From 98f3d08c6241756e7028c255c6968ad77c072fca Mon Sep 17 00:00:00 2001 From: annie-mac Date: Sat, 24 Feb 2024 13:52:17 -0800 Subject: [PATCH 01/44] some change --- eng/versioning/external_dependencies.txt | 3 + .../kafka/connect/CosmosDBSinkConnector.java | 60 ++++ .../connect/CosmosDBSourceConnector.java | 14 +- .../implementation/CosmosClientStore.java | 4 +- .../connect/implementation/CosmosConfig.java | 49 ++- .../CosmosExceptionsHelper.java | 24 ++ ...nstants.java => KafkaCosmosConstants.java} | 2 +- .../sink/CosmosDBWriteException.java | 14 + .../implementation/sink/CosmosSinkConfig.java | 336 ++++++++++++++++++ .../sink/CosmosSinkContainersConfig.java | 21 ++ .../implementation/sink/CosmosSinkTask.java | 147 ++++++++ .../sink/CosmosSinkTaskConfig.java | 13 + .../sink/CosmosSinkWriteConfig.java | 45 +++ .../connect/implementation/sink/IWriter.java | 10 + .../implementation/sink/IdStrategies.java | 31 ++ .../sink/ItemWriteStrategy.java | 32 ++ .../sink/KafkaCosmosBulkWriter.java | 228 ++++++++++++ .../sink/KafkaCosmosPointWriter.java | 167 +++++++++ .../sink/KafkaCosmosWriterBase.java | 68 ++++ .../sink/SinkOperationContext.java | 67 ++++ .../sink/SinkOperationFailedResponse.java | 22 ++ .../sink/SinkRecordTransformer.java | 74 ++++ .../sink/SinkWriteResponse.java | 28 ++ .../implementation/sink/StructToJsonMap.java | 114 ++++++ .../sink/ToleranceOnErrorLevel.java | 28 ++ .../source/CosmosChangeFeedModes.java | 2 +- .../CosmosChangeFeedStartFromModes.java | 2 +- .../source/CosmosSourceConfig.java | 184 ++++------ .../source/CosmosSourceContainersConfig.java | 11 +- .../source/CosmosSourceTask.java | 4 +- .../connect/CosmosDbSourceConnectorITest.java | 113 ++++++ .../KafkaCosmosTestConfigurations.java | 208 +++++++++++ .../source/MetadataMonitorThreadTest.java | 6 +- .../azure/cosmos/CosmosAsyncContainer.java | 5 + .../ImplementationBridgeHelpers.java | 2 + 35 files changed, 2000 insertions(+), 138 deletions(-) create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/{CosmosConstants.java => KafkaCosmosConstants.java} (94%) create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IdStrategies.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ItemWriteStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ToleranceOnErrorLevel.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 20f9b53a74589..3d1245d9f391a 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -393,6 +393,9 @@ cosmos_org.scalastyle:scalastyle-maven-plugin;1.0.0 cosmos_org.apache.kafka:connect-api;3.6.0 # Cosmos Kafka connector tests only cosmos_org.apache.kafka:connect-runtime;3.6.0 +cosmos_org.testcontainers:testcontainers;1.19.5 +cosmos_org.testcontainers:kafka;1.19.5 +cosmos_org.sourcelab:kafka-connect-client;4.0.4 # Maven Tools for Cosmos Kafka connector only cosmos_io.confluent:kafka-connect-maven-plugin;0.12.0 diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java new file mode 100644 index 0000000000000..72021103e24c8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java @@ -0,0 +1,60 @@ +package com.azure.cosmos.kafka.connect; + +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; +import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkConfig; +import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkTask; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.sink.SinkConnector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * A Sink connector that publishes topic messages to CosmosDB. + */ +public class CosmosDBSinkConnector extends SinkConnector { + private static final Logger LOGGER = LoggerFactory.getLogger(CosmosDBSinkConnector.class); + + private CosmosSinkConfig sinkConfig; + + @Override + public void start(Map props) { + LOGGER.info("Starting the kafka cosmos sink connector"); + this.sinkConfig = new CosmosSinkConfig(props); + } + + @Override + public Class taskClass() { + return CosmosSinkTask.class; + } + + @Override + public List> taskConfigs(int maxTasks) { + LOGGER.info("Setting task configurations with maxTasks {}", maxTasks); + List> configs = new ArrayList<>(); + for (int i = 0; i < maxTasks; i++) { + configs.add(this.sinkConfig.originalsStrings()); + } + + return configs; + } + + @Override + public void stop() { + LOGGER.debug("Kafka Cosmos sink connector {} is stopped."); + } + + @Override + public ConfigDef config() { + return CosmosSinkConfig.getConfigDef(); + } + + @Override + public String version() { + return KafkaCosmosConstants.CURRENT_VERSION; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java index 8545b65ed2029..65fdfa2ff94a6 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java @@ -14,7 +14,7 @@ import com.azure.cosmos.implementation.query.CompositeContinuationToken; import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; -import com.azure.cosmos.kafka.connect.implementation.CosmosConstants; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceOffsetStorageReader; @@ -102,7 +102,7 @@ public ConfigDef config() { @Override public String version() { - return CosmosConstants.CURRENT_VERSION; + return KafkaCosmosConstants.CURRENT_VERSION; } // TODO[public preview]: how this is being used private List> getTaskConfigs(int maxTasks) { @@ -324,15 +324,7 @@ private List> getFeedRanges(CosmosContainerProperties containerPro } private Map getContainersTopicMap(List allContainers) { - Map topicMapFromConfig = - this.config.getContainersConfig().getContainersTopicMap() - .stream() - .map(containerTopicMapString -> containerTopicMapString.split("#")) - .collect( - Collectors.toMap( - containerTopicMapArray -> containerTopicMapArray[1], - containerTopicMapArray -> containerTopicMapArray[0])); - + Map topicMapFromConfig = this.config.getContainersConfig().getContainersTopicMap(); Map effectiveContainersTopicMap = new HashMap<>(); allContainers.forEach(containerProperties -> { // by default, we are using container id as the topic name as well unless customer override through containers.topicMap diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java index 40812a54500ca..10589369d0e8d 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java @@ -36,9 +36,9 @@ public static CosmosAsyncClient getCosmosClient(CosmosAccountConfig accountConfi private static String getUserAgentSuffix(CosmosAccountConfig accountConfig) { if (StringUtils.isNotEmpty(accountConfig.getApplicationName())) { - return CosmosConstants.USER_AGENT_SUFFIX + "|" + accountConfig.getApplicationName(); + return KafkaCosmosConstants.USER_AGENT_SUFFIX + "|" + accountConfig.getApplicationName(); } - return CosmosConstants.USER_AGENT_SUFFIX; + return KafkaCosmosConstants.USER_AGENT_SUFFIX; } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java index 4aecdff89c87a..8b56ca1b017ea 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java @@ -5,6 +5,7 @@ import com.azure.cosmos.implementation.Strings; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceContainersConfig; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; @@ -151,6 +152,18 @@ public CosmosAccountConfig getAccountConfig() { return accountConfig; } + protected static List convertToList(String configValue) { + if (StringUtils.isNotEmpty(configValue)) { + if (configValue.startsWith("[") && configValue.endsWith("]")) { + configValue = configValue.substring(1, configValue.length() - 1); + } + + return Arrays.stream(configValue.split(",")).map(String::trim).collect(Collectors.toList()); + } + + return new ArrayList<>(); + } + public static class AccountEndpointValidator implements ConfigDef.Validator { @Override @SuppressWarnings("unchecked") @@ -173,15 +186,39 @@ public String toString() { } } - protected static List convertToList(String configValue) { - if (StringUtils.isNotEmpty(configValue)) { - if (configValue.startsWith("[") && configValue.endsWith("]")) { - configValue = configValue.substring(1, configValue.length() - 1); + public static class ContainersTopicMapValidator implements ConfigDef.Validator { + private static final String INVALID_TOPIC_MAP_FORMAT = + "Invalid entry for topic-container map. The topic-container map should be a comma-delimited " + + "list of Kafka topic to Cosmos containers. Each mapping should be a pair of Kafka " + + "topic and Cosmos container separated by '#'. For example: topic1#con1,topic2#con2."; + + @Override + @SuppressWarnings("unchecked") + public void ensureValid(String name, Object o) { + String configValue = (String) o; + if (StringUtils.isEmpty(configValue)) { + return; } - return Arrays.stream(configValue.split(",")).map(String::trim).collect(Collectors.toList()); + List containerTopicMapList = convertToList(configValue); + + // validate each item should be in topic#container format + boolean invalidFormatExists = + containerTopicMapList + .stream() + .anyMatch(containerTopicMap -> + containerTopicMap + .split(CosmosSourceContainersConfig.CONTAINER_TOPIC_MAP_SEPARATOR) + .length != 2); + + if (invalidFormatExists) { + throw new ConfigException(name, o, INVALID_TOPIC_MAP_FORMAT); + } } - return new ArrayList<>(); + @Override + public String toString() { + return "Containers topic map"; + } } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java index 7424128828408..9924864705b5e 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java @@ -49,4 +49,28 @@ public static ConnectException convertToConnectException(Throwable throwable, St return new ConnectException(message, throwable); } + + public static boolean isResourceExistsException(Throwable throwable) { + if (throwable instanceof CosmosException) { + return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.CONFLICT; + } + + return false; + } + + public static boolean isNotFoundException(Throwable throwable) { + if (throwable instanceof CosmosException) { + return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.NOTFOUND; + } + + return false; + } + + public static boolean isPreconditionFailedException(Throwable throwable) { + if (throwable instanceof CosmosException) { + return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.PRECONDITION_FAILED; + } + + return false; + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConstants.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java similarity index 94% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConstants.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java index afeac46866b95..01a89ef20f532 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConstants.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java @@ -5,7 +5,7 @@ import com.azure.core.util.CoreUtils; -public class CosmosConstants { +public class KafkaCosmosConstants { public static final String PROPERTIES_FILE_NAME = "azure-cosmos-kafka-connect.properties"; public static final String CURRENT_VERSION = CoreUtils.getProperties(PROPERTIES_FILE_NAME).get("version"); public static final String CURRENT_NAME = CoreUtils.getProperties(PROPERTIES_FILE_NAME).get("name"); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java new file mode 100644 index 0000000000000..6e30c4f0be08f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java @@ -0,0 +1,14 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import org.apache.kafka.connect.errors.ConnectException; + +public class CosmosDBWriteException extends ConnectException { + /** + * + */ + private static final long serialVersionUID = 1L; + + public CosmosDBWriteException(String message) { + super(message); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java new file mode 100644 index 0000000000000..808361b97a0e8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java @@ -0,0 +1,336 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.implementation.Strings; +import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.CosmosConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Common Configuration for Cosmos DB Kafka sink connector. + */ +public class CosmosSinkConfig extends CosmosConfig { + private static final String SINK_CONFIG_PREFIX = "kafka.connect.cosmos.sink."; + + // error tolerance + public static final String TOLERANCE_ON_ERROR_CONFIG = SINK_CONFIG_PREFIX + "errors.tolerance"; + public static final String TOLERANCE_ON_ERROR_DOC = + "Error tolerance level after exhausting all retries. 'None' for fail on error. 'All' for log and continue"; + public static final String TOLERANCE_ON_ERROR_DISPLAY = "Error tolerance level."; + public static final String DEFAULT_TOLERANCE_ON_ERROR = ToleranceOnErrorLevel.NONE.getName(); + + // sink bulk config + public static final String BULK_ENABLED_CONF = SINK_CONFIG_PREFIX + "bulk.enabled"; + private static final String BULK_ENABLED_DOC = + "Flag to indicate whether Cosmos DB bulk mode is enabled for Sink connector. By default it is true."; + private static final String BULK_ENABLED_DISPLAY = "enable bulk mode."; + private static final boolean DEFAULT_BULK_ENABLED = true; + + // TODO[Public Preview]: Add other write config, for example patch, bulkUpdate + public static final String BULK_MAX_CONCURRENT_PARTITIONS_CONF = SINK_CONFIG_PREFIX + "bulk.maxConcurrentCosmosPartitions"; + private static final String BULK_MAX_CONCURRENT_PARTITIONS_DOC = + "Cosmos DB Item Write Max Concurrent Cosmos Partitions." + + " If not specified it will be determined based on the number of the container's physical partitions -" + + " which would indicate every Spark partition is expected to have data from all Cosmos physical partitions." + + " If specified it indicates from at most how many Cosmos Physical Partitions each Spark partition contains" + + " data. So this config can be used to make bulk processing more efficient when input data in Spark has been" + + " repartitioned to balance to how many Cosmos partitions each Spark partition needs to write. This is mainly" + + " useful for very large containers (with hundreds of physical partitions)."; + private static final String BULK_MAX_CONCURRENT_PARTITIONS_DISPLAY = "Cosmos DB Item Write Max Concurrent Cosmos Partitions."; + private static final int DEFAULT_BULK_MAX_CONCURRENT_PARTITIONS = -1; + + public static final String BULK_INITIAL_BATCH_SIZE_CONF = SINK_CONFIG_PREFIX + "bulk.initialBatchSize"; + private static final String BULK_INITIAL_BATCH_SIZE_DOC = + "Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend " + + "when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch " + + "size is getting automatically tuned based on the throttling rate. By default the " + + "initial micro batch size is 100. Reduce this when you want to avoid that the first few requests consume " + + "too many RUs."; + private static final String BULK_INITIAL_BATCH_SIZE_DISPLAY = "Cosmos DB initial bulk micro batch size."; + private static final int DEFAULT_BULK_INITIAL_BATCH_SIZE = 1; // start with small value to avoid initial RU spike + + // write strategy + public static final String WRITE_STRATEGY_CONF = SINK_CONFIG_PREFIX + "write.strategy"; + private static final String WRITE_STRATEGY_DOC = "Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert)"; + private static final String WRITE_STRATEGY_DISPLAY = "Cosmos DB Item write Strategy."; + private static final String DEFAULT_WRITE_STRATEGY = ItemWriteStrategy.ITEM_OVERWRITE.getName(); + + // max retry + public static final String MAX_RETRY_COUNT_CONF = SINK_CONFIG_PREFIX + "maxRetryCount"; + private static final String MAX_RETRY_COUNT_DOC = + "Cosmos DB max retry attempts on write failures for Sink connector. By default, the connector will retry on transient write errors for up to 10 times."; + private static final String MAX_RETRY_COUNT_DISPLAY = "Cosmos DB max retry attempts on write failures for Sink connector."; + private static final int DEFAULT_MAX_RETRY_COUNT = 10; + + // database name + private static final String DATABASE_NAME_CONF = SINK_CONFIG_PREFIX + "database.name"; + private static final String DATABASE_NAME_CONF_DOC = "Cosmos DB database name."; + private static final String DATABASE_NAME_CONF_DISPLAY = "Cosmos DB database name."; + + // container topic map + public static final String CONTAINERS_TOPIC_MAP_CONF = SINK_CONFIG_PREFIX + "containers.topicMap"; + private static final String CONTAINERS_TOPIC_MAP_DOC = + "A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2."; + private static final String CONTAINERS_TOPIC_MAP_DISPLAY = "Topic-Container map"; + + // id.strategy + public static final String ID_STRATEGY_CONF = SINK_CONFIG_PREFIX + "id.strategy"; + public static final String ID_STRATEGY_DOC = + "A strategy used to populate the document with an ``id``. Valid strategies are: " + + "``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, " + + "``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. For each strategy, the full " + + "name of the strategy must be specified, e.g. " + + "com.azure.cosmos.kafka.connect.sink.id.strategy.TemplateStrategy. Configuration " + + "properties prefixed with``id.strategy`` are passed through to the strategy. For " + + "example, when using" + + "``id.strategy=com.azure.cosmos.kafka.connect.sink.id.strategy.TemplateStrategy`` , " + + "the property ``id.strategy.template`` is passed through to the template strategy " + + "and used to specify the template string to be used in constructing the ``id``."; + public static final String ID_STRATEGY_DISPLAY = "ID Strategy"; + public static final String DEFAULT_ID_STRATEGY = IdStrategies.PROVIDED_IN_VALUE_STRATEGY.getName(); + + // TODO[Public Preview] Verify whether compression need to happen in connector + + private final CosmosSinkWriteConfig writeConfig; + private final CosmosSinkContainersConfig containersConfig; + private final ToleranceOnErrorLevel toleranceOnErrorLevel; + private final IdStrategies idStrategy; + + public CosmosSinkConfig(Map parsedConfig) { + this(getConfigDef(), parsedConfig); + } + + public CosmosSinkConfig(ConfigDef config, Map parsedConfig) { + super(config, parsedConfig); + this.writeConfig = this.parseWriteConfig(); + this.containersConfig = this.parseContainersConfig(); + this.toleranceOnErrorLevel = this.parseToleranceOnErrorLevel(); + this.idStrategy = this.parseIdStrategy(); + } + + public static ConfigDef getConfigDef() { + ConfigDef configDef = CosmosConfig.getConfigDef(); + + defineWriteConfig(configDef); + defineContainersConfig(configDef); + defineToleranceOnErrorConfig(configDef); + defineIdStrategyConfig(configDef); + return configDef; + } + + private static void defineWriteConfig(ConfigDef configDef) { + final String writeConfigGroupName = "Write config"; + int writeConfigGroupOrder = 0; + configDef + .define( + BULK_ENABLED_CONF, + ConfigDef.Type.BOOLEAN, + DEFAULT_BULK_ENABLED, + ConfigDef.Importance.MEDIUM, + BULK_ENABLED_DOC, + writeConfigGroupName, + writeConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + BULK_ENABLED_DISPLAY + ) + .define( + BULK_MAX_CONCURRENT_PARTITIONS_CONF, + ConfigDef.Type.INT, + DEFAULT_BULK_MAX_CONCURRENT_PARTITIONS, + ConfigDef.Importance.LOW, + BULK_MAX_CONCURRENT_PARTITIONS_DOC, + writeConfigGroupName, + writeConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + BULK_MAX_CONCURRENT_PARTITIONS_DISPLAY + ) + .define( + BULK_INITIAL_BATCH_SIZE_CONF, + ConfigDef.Type.INT, + DEFAULT_BULK_INITIAL_BATCH_SIZE, + ConfigDef.Importance.MEDIUM, + BULK_INITIAL_BATCH_SIZE_DOC, + writeConfigGroupName, + writeConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + BULK_INITIAL_BATCH_SIZE_DISPLAY + ) + .define( + WRITE_STRATEGY_CONF, + ConfigDef.Type.STRING, + DEFAULT_WRITE_STRATEGY, + new ItemWriteStrategyValidator(), + ConfigDef.Importance.HIGH, + WRITE_STRATEGY_DOC, + writeConfigGroupName, + writeConfigGroupOrder++, + ConfigDef.Width.LONG, + WRITE_STRATEGY_DISPLAY + ) + .define( + MAX_RETRY_COUNT_CONF, + ConfigDef.Type.INT, + DEFAULT_MAX_RETRY_COUNT, + ConfigDef.Importance.MEDIUM, + MAX_RETRY_COUNT_DOC, + writeConfigGroupName, + writeConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + MAX_RETRY_COUNT_DISPLAY + ); + } + + private static void defineContainersConfig(ConfigDef configDef) { + final String containersGroupName = "Containers"; + int containersGroupOrder = 0; + + configDef + .define( + DATABASE_NAME_CONF, + ConfigDef.Type.STRING, + ConfigDef.NO_DEFAULT_VALUE, + NON_EMPTY_STRING, + ConfigDef.Importance.HIGH, + DATABASE_NAME_CONF_DOC, + containersGroupName, + containersGroupOrder++, + ConfigDef.Width.LONG, + DATABASE_NAME_CONF_DISPLAY + ) + .define( + CONTAINERS_TOPIC_MAP_CONF, + ConfigDef.Type.STRING, + Strings.Emtpy, + new ContainersTopicMapValidator(), + ConfigDef.Importance.MEDIUM, + CONTAINERS_TOPIC_MAP_DOC, + containersGroupName, + containersGroupOrder++, + ConfigDef.Width.LONG, + CONTAINERS_TOPIC_MAP_DISPLAY + ); + } + + private static void defineToleranceOnErrorConfig(ConfigDef configDef) { + final String toleranceOnErrorConfigGroupName = "Errors tolerance"; + int toleranceOnErrorConfigGroupOrder = 0; + configDef + .define( + TOLERANCE_ON_ERROR_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_TOLERANCE_ON_ERROR, + ConfigDef.Importance.HIGH, + TOLERANCE_ON_ERROR_DOC, + toleranceOnErrorConfigGroupName, + toleranceOnErrorConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + TOLERANCE_ON_ERROR_DISPLAY + ); + } + + private static void defineIdStrategyConfig(ConfigDef configDef) { + final String idStrategyConfigGroupName = "ID Strategy"; + int idStrategyConfigGroupOrder = 0; + configDef + .define( + ID_STRATEGY_CONF, + ConfigDef.Type.STRING, + DEFAULT_ID_STRATEGY, + ConfigDef.Importance.HIGH, + ID_STRATEGY_DOC, + idStrategyConfigGroupName, + idStrategyConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + ID_STRATEGY_DISPLAY + ); + } + + private CosmosSinkWriteConfig parseWriteConfig() { + boolean bulkEnabled = this.getBoolean(BULK_ENABLED_CONF); + int bulkMaxConcurrentCosmosPartitions = this.getInt(BULK_MAX_CONCURRENT_PARTITIONS_CONF); + int bulkInitialBatchSize = this.getInt(BULK_INITIAL_BATCH_SIZE_CONF); + ItemWriteStrategy writeStrategy = this.parseItemWriteStrategy(); + int maxRetryCount = this.getInt(MAX_RETRY_COUNT_CONF); + + return new CosmosSinkWriteConfig( + bulkEnabled, + bulkMaxConcurrentCosmosPartitions, + bulkInitialBatchSize, + writeStrategy, + maxRetryCount); + } + + private CosmosSinkContainersConfig parseContainersConfig() { + String databaseName = this.getString(DATABASE_NAME_CONF); + Map topicToContainerMap = this.getTopicToContainerMap(); + + return new CosmosSinkContainersConfig(databaseName, topicToContainerMap); + } + + private Map getTopicToContainerMap() { + List containersTopicMapList = convertToList(this.getString(CONTAINERS_TOPIC_MAP_CONF)); + return containersTopicMapList + .stream() + .map(containerTopicMapString -> containerTopicMapString.split("#")) + .collect( + Collectors.toMap( + containerTopicMapArray -> containerTopicMapArray[0], + containerTopicMapArray -> containerTopicMapArray[1])); + } + + private ItemWriteStrategy parseItemWriteStrategy() { + return ItemWriteStrategy.fromName(this.getString(WRITE_STRATEGY_CONF)); + } + + private ToleranceOnErrorLevel parseToleranceOnErrorLevel() { + return ToleranceOnErrorLevel.fromName(this.getString(TOLERANCE_ON_ERROR_CONFIG)); + } + + private IdStrategies parseIdStrategy() { + return IdStrategies.fromName(this.getString(ID_STRATEGY_CONF)); + } + + public CosmosSinkWriteConfig getWriteConfig() { + return writeConfig; + } + + public CosmosSinkContainersConfig getContainersConfig() { + return containersConfig; + } + + public ToleranceOnErrorLevel getToleranceOnErrorLevel() { + return toleranceOnErrorLevel; + } + + public IdStrategies getIdStrategy() { + return idStrategy; + } + + public static class ItemWriteStrategyValidator implements ConfigDef.Validator { + @Override + @SuppressWarnings("unchecked") + public void ensureValid(String name, Object o) { + String itemWriteStrategyString = (String) o; + if (StringUtils.isEmpty(itemWriteStrategyString)) { + throw new ConfigException(name, o, "WriteStrategy can not be empty or null"); + } + + ItemWriteStrategy itemWriteStrategy = ItemWriteStrategy.fromName(itemWriteStrategyString); + if (itemWriteStrategy == null) { + throw new ConfigException(name, o, "Invalid ItemWriteStrategy. Allowed values " + ItemWriteStrategy.values()); + } + } + + @Override + public String toString() { + return "ItemWriteStrategy. Only allow " + ItemWriteStrategy.values(); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java new file mode 100644 index 0000000000000..d14a90e8d9d9c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java @@ -0,0 +1,21 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import java.util.Map; + +public class CosmosSinkContainersConfig { + private final String databaseName; + private final Map topicToContainerMap; + + public CosmosSinkContainersConfig(String databaseName, Map topicToContainerMap) { + this.databaseName = databaseName; + this.topicToContainerMap = topicToContainerMap; + } + + public String getDatabaseName() { + return databaseName; + } + + public Map getTopicToContainerMap() { + return topicToContainerMap; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java new file mode 100644 index 0000000000000..b4d1b096414a9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java @@ -0,0 +1,147 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.CosmosAsyncClient; +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.MDC; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class CosmosSinkTask extends SinkTask { + private static final Logger LOGGER = LoggerFactory.getLogger(CosmosSinkTask.class); + private CosmosSinkTaskConfig sinkTaskConfig; + private CosmosAsyncClient cosmosClient; + private SinkRecordTransformer sinkRecordTransformer; + private IWriter cosmosWriter; + + @Override + public String version() { + return KafkaCosmosConstants.CURRENT_VERSION; + } + + @Override + public void start(Map props) { + LOGGER.info("Starting the kafka cosmos sink task..."); + this.sinkTaskConfig = new CosmosSinkTaskConfig(props); + this.cosmosClient = CosmosClientStore.getCosmosClient(this.sinkTaskConfig.getAccountConfig()); + this.sinkRecordTransformer = new SinkRecordTransformer(this.sinkTaskConfig.getIdStrategy()); + + if (this.sinkTaskConfig.getWriteConfig().isBulkEnabled()) { + this.cosmosWriter = new KafkaCosmosBulkWriter(this.sinkTaskConfig.getWriteConfig()); + } else { + this.cosmosWriter = new KafkaCosmosPointWriter( + this.sinkTaskConfig.getWriteConfig(), + this.sinkTaskConfig.getToleranceOnErrorLevel()); + } + + // TODO[this PR]: this is done in V1, verify what is the expected behavior + this.cosmosClient.createDatabaseIfNotExists(this.sinkTaskConfig.getContainersConfig().getDatabaseName()); + } + + @Override + public void put(Collection records) { + if (records == null || records.isEmpty()) { + LOGGER.debug("No records to be written"); + return; + } + + LOGGER.debug("Sending {} records to be written", records.size()); + + // group by container + Map> recordsByContainer = + records.stream().collect( + Collectors.groupingBy( + record -> this.sinkTaskConfig + .getContainersConfig() + .getTopicToContainerMap() + .getOrDefault(record.topic(), StringUtils.EMPTY))); + + if (recordsByContainer.containsKey(StringUtils.EMPTY)) { + throw new IllegalStateException("There is no container defined for topics " + recordsByContainer.get(StringUtils.EMPTY)); + } + + for (Map.Entry> entry : recordsByContainer.entrySet()) { + String containerName = entry.getKey(); + CosmosAsyncContainer container = + this.cosmosClient + .getDatabase(this.sinkTaskConfig.getContainersConfig().getDatabaseName()) + .getContainer(containerName); + + // transform sink records, for example populating id + List transformedRecords = sinkRecordTransformer.transform(containerName, entry.getValue()); + + try { + SinkWriteResponse response = this.cosmosWriter.write(container, transformedRecords); + if (response.getFailedRecordResponses().size() == 0) { + LOGGER.debug( + "Sink write completed, {} records succeeded.", + response.getSucceededRecords().size()); + } else { + LOGGER.info( + "Sink write completed, {} records succeeded, {} records failed.", + response.getSucceededRecords().size(), + response.getFailedRecordResponses().size()); + this.sendToDlqIfConfigured(response.getFailedRecordResponses()); + } + } finally { + MDC.clear(); + } + } + } + + /** + * Sends data to a dead letter queue + * + * @param failedResponses the kafka record that failed to write + */ + private void sendToDlqIfConfigured(List failedResponses) { + if (context != null && context.errantRecordReporter() != null) { + for (SinkOperationFailedResponse sinkRecordResponse : failedResponses) { + context.errantRecordReporter().report(sinkRecordResponse.getSinkRecord(), sinkRecordResponse.getException()); + } + } + + StringBuilder errorMessage = new StringBuilder(); + for (SinkOperationFailedResponse failedResponse : failedResponses) { + errorMessage + .append( + String.format( + "Unable to write record to CosmosDB: {%s}, value schema {%s}, exception {%s}", + failedResponse.getSinkRecord().key(), + failedResponse.getSinkRecord().valueSchema(), + failedResponse.getException().toString())) + .append("\n"); + } + + if (this.sinkTaskConfig.getToleranceOnErrorLevel() == ToleranceOnErrorLevel.ALL) { + LOGGER.error( + "Total {} records failed, tolerance on error level 'All'. Error message: {}" , + failedResponses.size(), + errorMessage); + } else { + throw new CosmosDBWriteException(errorMessage.toString()); + } + } + + @Override + public void stop() { + LOGGER.info("Stopping Kafka CosmosDB sink task..."); + if (this.cosmosClient != null) { + this.cosmosClient.close(); + } + + this.cosmosClient = null; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskConfig.java new file mode 100644 index 0000000000000..f438f3f620eb9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskConfig.java @@ -0,0 +1,13 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +import java.util.Map; + +// Currently the sink task config shares the same config as sink connector config +public class CosmosSinkTaskConfig extends CosmosSinkConfig { + public CosmosSinkTaskConfig(Map parsedConfig) { + super(parsedConfig); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java new file mode 100644 index 0000000000000..6137d4f957a19 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +public class CosmosSinkWriteConfig { + private final boolean bulkEnabled; + private final int bulkMaxConcurrentCosmosPartitions; + private final int bulkInitialBatchSize; + private final ItemWriteStrategy itemWriteStrategy; + private final int maxRetryCount; + + public CosmosSinkWriteConfig( + boolean bulkEnabled, + int bulkMaxConcurrentCosmosPartitions, + int bulkInitialBatchSize, + ItemWriteStrategy itemWriteStrategy, + int maxRetryCount) { + this.bulkEnabled = bulkEnabled; + this.bulkMaxConcurrentCosmosPartitions = bulkMaxConcurrentCosmosPartitions; + this.bulkInitialBatchSize = bulkInitialBatchSize; + this.itemWriteStrategy = itemWriteStrategy; + this.maxRetryCount = maxRetryCount; + } + + public boolean isBulkEnabled() { + return bulkEnabled; + } + + public int getBulkMaxConcurrentCosmosPartitions() { + return bulkMaxConcurrentCosmosPartitions; + } + + public int getBulkInitialBatchSize() { + return bulkInitialBatchSize; + } + + public ItemWriteStrategy getItemWriteStrategy() { + return itemWriteStrategy; + } + + public int getMaxRetryCount() { + return maxRetryCount; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java new file mode 100644 index 0000000000000..492380578bd9d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java @@ -0,0 +1,10 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.CosmosAsyncContainer; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.List; + +public interface IWriter { + SinkWriteResponse write(CosmosAsyncContainer container, List sinkRecords); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IdStrategies.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IdStrategies.java new file mode 100644 index 0000000000000..dcc3568dc2fb1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IdStrategies.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +public enum IdStrategies { + TEMPLATE_STRATEGY("TemplateStrategy"), + FULL_KEY_STRATEGY("FullKeyStrategy"), + KAFKA_METADATA_STRATEGY("KafkaMetadataStrategy"), + PROVIDED_IN_KEY_STRATEGY("ProvidedInKeyStrategy"), + PROVIDED_IN_VALUE_STRATEGY("ProvidedInValueStrategy"); + + private final String name; + + IdStrategies(String name) { + this.name = name; + } + + public String getName() { + return name; + } + + public static IdStrategies fromName(String name) { + for (IdStrategies mode : IdStrategies.values()) { + if (mode.getName().equalsIgnoreCase(name)) { + return mode; + } + } + return null; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ItemWriteStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ItemWriteStrategy.java new file mode 100644 index 0000000000000..988f577b5a267 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ItemWriteStrategy.java @@ -0,0 +1,32 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +public enum ItemWriteStrategy { + ITEM_OVERWRITE("ItemOverwrite"), + ITEM_APPEND("ItemAppend"), + ITEM_DELETE("ItemDelete"), + ITEM_DELETE_IF_NOT_MODIFIED("ItemDeleteIfNotModified"), + ITEM_OVERWRITE_IF_NOT_MODIFIED("ItemOverwriteIfNotModified"); + + // TODO[Public Preview] Add ItemPatch, ItemBulkUpdate + private final String name; + + ItemWriteStrategy(String name) { + this.name = name; + } + + public String getName() { + return name; + } + + public static ItemWriteStrategy fromName(String name) { + for (ItemWriteStrategy mode : ItemWriteStrategy.values()) { + if (mode.getName().equalsIgnoreCase(name)) { + return mode; + } + } + return null; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java new file mode 100644 index 0000000000000..46d659263f334 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java @@ -0,0 +1,228 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.BridgeInternal; +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.CosmosException; +import com.azure.cosmos.implementation.HttpConstants; +import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.models.CosmosBulkItemRequestOptions; +import com.azure.cosmos.models.CosmosBulkItemResponse; +import com.azure.cosmos.models.CosmosBulkOperations; +import com.azure.cosmos.models.CosmosItemOperation; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; + +public class KafkaCosmosBulkWriter extends KafkaCosmosWriterBase { + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosBulkWriter.class); + + private final CosmosSinkWriteConfig writeConfig; + + public KafkaCosmosBulkWriter(CosmosSinkWriteConfig writeConfig) { + checkNotNull(writeConfig, "Argument 'writeConfig' can not be null"); + + this.writeConfig = writeConfig; + } + + @Override + public SinkWriteResponse write(CosmosAsyncContainer container, List sinkRecords) { + LOGGER.debug("Write {} records to container {}", sinkRecords.size(), container.getId()); + + SinkWriteResponse sinkWriteResponse = new SinkWriteResponse(); + + if (sinkRecords == null || sinkRecords.isEmpty()) { + return sinkWriteResponse; + } + + List sinkOperationContexts = + sinkRecords + .stream() + .map(sinkRecord -> new SinkOperationContext(sinkRecord)) + .collect(Collectors.toList()); + + Mono + .defer(() -> Mono.just(getToBeProcessedSinkOperation(sinkOperationContexts))) + .flatMap(operationsToBeProcessed -> { + if (this.writeConfig.getItemWriteStrategy() == ItemWriteStrategy.ITEM_OVERWRITE) { + return this.getBulkOperations(container, operationsToBeProcessed) + .flatMap(itemOperations -> this.executeBulkOperations(container, itemOperations)); + } + + return Mono.empty(); + }) + .repeat(() -> getToBeProcessedSinkOperation(sinkOperationContexts).size() > 0)// only repeat when there are records still need to be processed + .blockLast(); + + return sinkWriteResponse; + } + + private List getToBeProcessedSinkOperation(List sinkOperationContexts) { + return sinkOperationContexts + .stream() + .filter( + sinkOperationContext -> { + return !sinkOperationContext.getIsSucceeded() + && shouldRetry(sinkOperationContext.getException(), sinkOperationContext.getRetryCount(), this.writeConfig.getMaxRetryCount()); + }) + .collect(Collectors.toList()); + } + + private Mono> getBulkOperations( + CosmosAsyncContainer container, + List sinkOperationContexts) { + + return this.getPartitionKeyDefinition(container) + .flatMap(partitionKeyDefinition -> { + List cosmosItemOperations = new ArrayList<>(); + switch (this.writeConfig.getItemWriteStrategy()) { + case ITEM_OVERWRITE: + sinkOperationContexts.forEach(sinkOperationContext -> + cosmosItemOperations.add( + CosmosBulkOperations.getUpsertItemOperation( + sinkOperationContext.getSinkRecord().value(), + this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), + sinkOperationContext + ) + )); + break; + case ITEM_OVERWRITE_IF_NOT_MODIFIED: + sinkOperationContexts.forEach(sinkOperationContext -> { + String etag = getEtag(sinkOperationContext.getSinkRecord().value()); + if (StringUtils.isEmpty(etag)) { + cosmosItemOperations.add( + CosmosBulkOperations.getCreateItemOperation( + sinkOperationContext.getSinkRecord().value(), + this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), + sinkOperationContext + ) + ); + } else { + cosmosItemOperations.add( + CosmosBulkOperations.getReplaceItemOperation( + getId(sinkOperationContext.getSinkRecord().value()), + sinkOperationContext.getSinkRecord(), + this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), + new CosmosBulkItemRequestOptions().setIfMatchETag(etag), + sinkOperationContext) + ); + } + }); + break; + case ITEM_APPEND: + sinkOperationContexts.forEach(sinkOperationContext -> { + cosmosItemOperations.add( + CosmosBulkOperations.getCreateItemOperation( + sinkOperationContext.getSinkRecord().value(), + this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), + sinkOperationContext + )); + + }); + break; + case ITEM_DELETE: + sinkOperationContexts.forEach(sinkOperationContext -> { + cosmosItemOperations.add( + CosmosBulkOperations.getDeleteItemOperation( + this.getId(sinkOperationContext.getSinkRecord().value()), + this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), + sinkOperationContext + )); + }); + break; + case ITEM_DELETE_IF_NOT_MODIFIED: + sinkOperationContexts.forEach(sinkOperationContext -> { + String etag = getEtag(sinkOperationContext.getSinkRecord().value()); + CosmosBulkItemRequestOptions itemRequestOptions = new CosmosBulkItemRequestOptions(); + if (StringUtils.isNotEmpty(etag)) { + itemRequestOptions.setIfMatchETag(etag); + } + + CosmosBulkOperations.getDeleteItemOperation( + getId(sinkOperationContext.getSinkRecord().value()), + this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), + itemRequestOptions, + sinkOperationContext); + }); + break; + default: + return Mono.error(new IllegalArgumentException(this.writeConfig.getItemWriteStrategy() + " is not supported")); + } + + return Mono.just(cosmosItemOperations); + }); + } + + private Mono executeBulkOperations(CosmosAsyncContainer container, List cosmosItemOperations) { + return container + .executeBulkOperations(Flux.fromIterable(cosmosItemOperations)) + .doOnNext(itemResponse -> { + SinkOperationContext context = itemResponse.getOperation().getContext(); + checkNotNull(context, "sinkOperationContext should not be null"); + + if (itemResponse.getException() != null + || itemResponse.getResponse() == null + || !itemResponse.getResponse().isSuccessStatusCode()) { + + BulkOperationFailedException exception = handleErrorStatusCode( + itemResponse.getResponse(), + itemResponse.getException(), + context); + + context.setException(exception); + } else { + context.setSucceeded(); + } + }) + .onErrorResume(throwable -> { + cosmosItemOperations.forEach(cosmosItemOperation -> { + ((SinkOperationContext)cosmosItemOperation.getContext()).setException(throwable); + }); + + return Mono.empty(); + }) + .then(); + } + + BulkOperationFailedException handleErrorStatusCode( + CosmosBulkItemResponse itemResponse, + Exception exception, + SinkOperationContext sinkOperationContext) { + + int effectiveStatusCode = + itemResponse != null + ? itemResponse.getStatusCode() + : (exception != null && exception instanceof CosmosException ? ((CosmosException)exception).getStatusCode() : HttpConstants.StatusCodes.REQUEST_TIMEOUT); + int effectiveSubStatusCode = + itemResponse != null + ? itemResponse.getSubStatusCode() + : (exception != null && exception instanceof CosmosException ? ((CosmosException)exception).getSubStatusCode() : 0); + + String errorMessage = + String.format( + "Request failed with effectiveStatusCode: {%s}, effectiveSubStatusCode: {%s}, kafkaOffset: {%s}, kafkaPartition: {%s}, topic: {%s}", + effectiveStatusCode, + effectiveSubStatusCode, + sinkOperationContext.getKafkaOffset(), + sinkOperationContext.getKafkaPartition(), + sinkOperationContext.getTopic()); + + + return new BulkOperationFailedException(effectiveStatusCode, effectiveSubStatusCode, errorMessage, exception); + } + + private static class BulkOperationFailedException extends CosmosException { + protected BulkOperationFailedException(int statusCode, int subStatusCode, String message, Throwable cause) { + super(statusCode, message, null, cause); + BridgeInternal.setSubStatusCode(this, subStatusCode); + } + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java new file mode 100644 index 0000000000000..660bac98aabd8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java @@ -0,0 +1,167 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.models.CosmosItemRequestOptions; +import org.apache.kafka.connect.sink.SinkRecord; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; + +import java.util.List; + +import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; + +public class KafkaCosmosPointWriter extends KafkaCosmosWriterBase { + private final CosmosSinkWriteConfig writeConfig; + private final ToleranceOnErrorLevel toleranceOnErrorLevel; + + public KafkaCosmosPointWriter(CosmosSinkWriteConfig writeConfig, ToleranceOnErrorLevel toleranceOnErrorLevel) { + checkNotNull(writeConfig, "Argument 'writeConfig' can not be null"); + this.writeConfig = writeConfig; + this.toleranceOnErrorLevel = toleranceOnErrorLevel; + } + + @Override + public SinkWriteResponse write(CosmosAsyncContainer container, List sinkRecords) { + checkNotNull(sinkRecords, "Argument 'sinkRecords' should not be null"); + SinkWriteResponse sinkWriteResponse = new SinkWriteResponse(); + + for (SinkRecord sinkRecord : sinkRecords) { + SinkOperationContext sinkOperationContext = new SinkOperationContext(sinkRecord); + switch (this.writeConfig.getItemWriteStrategy()) { + case ITEM_OVERWRITE: + this.upsertWithRetry(container, sinkOperationContext); + break; + case ITEM_OVERWRITE_IF_NOT_MODIFIED: + String etag = this.getEtag(sinkRecord.value()); + if (StringUtils.isNotEmpty(etag)) { + this.replaceIfNotModifiedWithRetry(container, sinkOperationContext, etag); + } else { + this.createWithRetry(container, sinkOperationContext); + } + break; + case ITEM_APPEND: + this.createWithRetry(container, sinkOperationContext); + break; + case ITEM_DELETE: + this.deleteWithRetry(container, sinkOperationContext, false); + break; + case ITEM_DELETE_IF_NOT_MODIFIED: + this.deleteWithRetry(container, sinkOperationContext, true); + break; + default: + throw new IllegalArgumentException(this.writeConfig.getItemWriteStrategy() + " is not supported"); + } + + if (sinkOperationContext.getIsSucceeded()) { + sinkWriteResponse.getSucceededRecords().add(sinkRecord); + } else { + sinkWriteResponse.getFailedRecordResponses().add(new SinkOperationFailedResponse(sinkRecord, sinkOperationContext.getException())); + if (this.toleranceOnErrorLevel == ToleranceOnErrorLevel.NONE) { + // if there is no tolerance, then fail fast on the first exception + return sinkWriteResponse; + } + } + } + + //We will only reach here is all the operations have succeeded or the ToleranceOnErrorLevel is ALL + return sinkWriteResponse; + } + + private void upsertWithRetry(CosmosAsyncContainer container, SinkOperationContext context) { + executeWithRetry( + (sinkOperationContext) -> container.upsertItem(context.getSinkRecord().value()).then(), + (throwable) -> false, // no exceptions should be ignored + context + ); + } + + private void createWithRetry(CosmosAsyncContainer container, SinkOperationContext context) { + executeWithRetry( + (sinkOperationContext) -> container.createItem(context.getSinkRecord().value()).then(), + (throwable) -> CosmosExceptionsHelper.isResourceExistsException(throwable), + context + ); + } + + private void replaceIfNotModifiedWithRetry(CosmosAsyncContainer container, SinkOperationContext context, String etag) { + executeWithRetry( + (sinkOperationContext) -> { + CosmosItemRequestOptions itemRequestOptions = new CosmosItemRequestOptions(); + itemRequestOptions.setIfMatchETag(etag); + + return this.getPartitionKeyDefinition(container) + .flatMap(partitionKeyDefinition -> { + return container.replaceItem( + context.getSinkRecord().value(), + getId(context.getSinkRecord().value()), + getPartitionKeyValue(context.getSinkRecord().value(), partitionKeyDefinition), + itemRequestOptions).then(); + }); + }, + (throwable) -> { + return CosmosExceptionsHelper.isResourceExistsException(throwable) || + CosmosExceptionsHelper.isNotFoundException(throwable) || + CosmosExceptionsHelper.isPreconditionFailedException(throwable); + }, + context + ); + } + + private void deleteWithRetry(CosmosAsyncContainer container, SinkOperationContext context, boolean onlyIfModified) { + executeWithRetry( + (sinkOperationContext) -> { + CosmosItemRequestOptions itemRequestOptions = new CosmosItemRequestOptions(); + if (onlyIfModified) { + String etag = this.getEtag(context.getSinkRecord().value()); + if (StringUtils.isNotEmpty(etag)) { + itemRequestOptions.setIfMatchETag(etag); + } + } + + return this.getPartitionKeyDefinition(container) + .flatMap(partitionKeyDefinition -> { + return container.deleteItem( + getId(context.getSinkRecord().value()), + getPartitionKeyValue(context.getSinkRecord().value(), partitionKeyDefinition), + itemRequestOptions + ); + }).then(); + }, + (throwable) -> CosmosExceptionsHelper.isNotFoundException(throwable), + context + ); + } + + private void executeWithRetry( + Function> execution, + Function shouldIgnoreFunc, + SinkOperationContext operationContext) { + + Mono.just(this) + .flatMap(data -> execution.apply(operationContext)) + .doOnNext(response -> operationContext.setSucceeded()) + .onErrorResume(throwable -> { + if (shouldIgnoreFunc.apply(throwable)) { + operationContext.setSucceeded(); + operationContext.complete(); + return Mono.empty(); + } + + operationContext.setException(throwable); + if (shouldRetry(throwable, operationContext.getRetryCount(), this.writeConfig.getMaxRetryCount())) { + return Mono.empty(); + } + + operationContext.complete(); + return Mono.empty(); + }) + .repeat(() -> !operationContext.isCompleted()) + .then() + .subscribeOn(Schedulers.boundedElastic()) + .block();// TODO: use customized schedulers + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java new file mode 100644 index 0000000000000..109c37276c516 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java @@ -0,0 +1,68 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.CosmosBridgeInternal; +import com.azure.cosmos.implementation.DocumentCollection; +import com.azure.cosmos.implementation.ImplementationBridgeHelpers; +import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.implementation.routing.PartitionKeyInternal; +import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.models.PartitionKey; +import com.azure.cosmos.models.PartitionKeyDefinition; +import reactor.core.publisher.Mono; + +import java.util.Collections; +import java.util.Map; + +import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; + +public abstract class KafkaCosmosWriterBase implements IWriter{ + protected String getId(Object recordValue) { + checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); + return ((Map) recordValue).get("id").toString(); + } + + protected String getEtag(Object recordValue) { + checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); + return ((Map) recordValue).get("_etag").toString(); + } + + protected PartitionKey getPartitionKeyValue(Object recordValue, PartitionKeyDefinition partitionKeyDefinition) { + checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); + + //TODO[Public Preview]: examine the code here for sub-partition + String partitionKeyPath = StringUtils.join(partitionKeyDefinition.getPaths(), ""); + Map recordMap = (Map) recordValue; + Object partitionKeyValue = recordMap.get(partitionKeyPath.substring(1)); + PartitionKeyInternal partitionKeyInternal = PartitionKeyInternal.fromObjectArray(Collections.singletonList(partitionKeyValue), false); + + return ImplementationBridgeHelpers + .PartitionKeyHelper + .getPartitionKeyAccessor() + .toPartitionKey(partitionKeyInternal); + } + + protected boolean shouldRetry(Throwable exception, int attemptedCount, int maxRetryCount) { + if (attemptedCount > maxRetryCount) { + return false; + } + + return CosmosExceptionsHelper.isTransientFailure(exception); + } + + protected Mono getPartitionKeyDefinition(CosmosAsyncContainer container) { + return Mono.just(CosmosBridgeInternal.getAsyncDocumentClient(container.getDatabase()).getCollectionCache()) + .flatMap(collectionCache -> { + return collectionCache + .resolveByNameAsync( + null, + ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .getLinkWithoutTrailingSlash(container), + null, + new DocumentCollection()) + .map(documentCollection -> documentCollection.getPartitionKey()); + }); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java new file mode 100644 index 0000000000000..d641eea8a2f29 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java @@ -0,0 +1,67 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class SinkOperationContext { + private final SinkRecord sinkRecord; + private final AtomicInteger retryCount; + private final AtomicBoolean isSucceeded; + private final AtomicReference exception; + private final AtomicBoolean completed; + + public SinkOperationContext(SinkRecord sinkRecord) { + this.sinkRecord = sinkRecord; + this.retryCount = new AtomicInteger(0); + this.isSucceeded = new AtomicBoolean(false); + this.exception = new AtomicReference<>(null); + this.completed = new AtomicBoolean(false); + } + + public SinkRecord getSinkRecord() { + return this.sinkRecord; + } + + public long getKafkaOffset() { + return this.sinkRecord.kafkaOffset(); + } + + public Integer getKafkaPartition() { + return this.sinkRecord.kafkaPartition(); + } + + public String getTopic() { + return this.sinkRecord.topic(); + } + + public boolean getIsSucceeded() { + return isSucceeded.get(); + } + + public int getRetryCount() { + return retryCount.get(); + } + + public Throwable getException() { + return this.exception.get(); + } + + public void setException(Throwable exception) { + this.exception.set(exception); + } + + public void setSucceeded() { + this.isSucceeded.set(true); + } + + public boolean isCompleted() { + return completed.get(); + } + + public void complete() { + completed.set(true); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java new file mode 100644 index 0000000000000..f4b0c181dea95 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java @@ -0,0 +1,22 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.CosmosException; +import org.apache.kafka.connect.sink.SinkRecord; + +public class SinkOperationFailedResponse { + private final SinkRecord sinkRecord; + private final Throwable exception; + + public SinkOperationFailedResponse(SinkRecord sinkRecord, Throwable cosmosException) { + this.sinkRecord = sinkRecord; + this.exception = cosmosException; + } + + public SinkRecord getSinkRecord() { + return this.sinkRecord; + } + + public Throwable getException() { + return this.exception; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java new file mode 100644 index 0000000000000..9feb527ff52e7 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java @@ -0,0 +1,74 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.slf4j.MDC; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class SinkRecordTransformer { + private static final Logger LOGGER = LoggerFactory.getLogger(SinkRecordTransformer.class); + + private final IdStrategies idStrategies; + + public SinkRecordTransformer(IdStrategies idStrategies) { + this.idStrategies = idStrategies; + } + + public List transform(String containerName, List sinkRecords) { + List toBeWrittenRecordList = new ArrayList<>(); + for (SinkRecord record : sinkRecords) { + if (record.key() != null) { + MDC.put(String.format("CosmosDbSink-%s", containerName), record.key().toString()); + } + + LOGGER.trace( + "Key Schema [{}], Key [{}], Value type [{}], Value schema [{}]", + record.keySchema(), + record.key(), + record.value() == null ? null : record.value().getClass().getName(), + record.value() == null ? null : record.valueSchema()); + + Object recordValue; + if (record.value() instanceof Struct) { + recordValue = StructToJsonMap.toJsonMap((Struct) record.value()); + // TODO: Do we need to update the value schema to map or keep it struct? + } else if (record.value() instanceof Map) { + recordValue = StructToJsonMap.handleMap((Map) record.value()); + } else { + recordValue = record.value(); + } + + maybeInsertId(recordValue, record); + + // Create an updated record with from the current record and the updated record value + final SinkRecord updatedRecord = new SinkRecord(record.topic(), + record.kafkaPartition(), + record.keySchema(), + record.key(), + record.valueSchema(), + recordValue, + record.kafkaOffset(), + record.timestamp(), + record.timestampType(), + record.headers()); + + toBeWrittenRecordList.add(updatedRecord); + } + + return toBeWrittenRecordList; + } + + private void maybeInsertId(Object recordValue, SinkRecord sinkRecord) { + if (!(recordValue instanceof Map)) { + return; + } + Map recordMap = (Map) recordValue; + IdStrategy idStrategy = config.idStrategy(); + recordMap.put(AbstractIdStrategyConfig.ID, idStrategy.generateId(sinkRecord)); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java new file mode 100644 index 0000000000000..eab452a2adae2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java @@ -0,0 +1,28 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.ArrayList; +import java.util.List; + +public class SinkWriteResponse { + private final List succeededRecords; + private List failedRecordResponses; + + public SinkWriteResponse() { + succeededRecords = new ArrayList<>(); + failedRecordResponses = new ArrayList<>(); + } + + public List getSucceededRecords() { + return succeededRecords; + } + + public List getFailedRecordResponses() { + return failedRecordResponses; + } + + public void setFailedRecordResponses(List failedRecordResponses) { + this.failedRecordResponses = failedRecordResponses; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java new file mode 100644 index 0000000000000..ef4e1af98a3ba --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java @@ -0,0 +1,114 @@ +package com.azure.cosmos.kafka.connect.implementation.sink; + +import org.apache.kafka.connect.data.Date; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +// TODO[Public Preview]: Double check logic here, copied over from V1 +public class StructToJsonMap { + + public static Map toJsonMap(Struct struct) { + if (struct == null) { + return null; + } + Map jsonMap = new HashMap(0); + List fields = struct.schema().fields(); + for (Field field : fields) { + String fieldName = field.name(); + Schema.Type fieldType = field.schema().type(); + String schemaName = field.schema().name(); + switch (fieldType) { + case STRING: + jsonMap.put(fieldName, struct.getString(fieldName)); + break; + case INT32: + if (Date.LOGICAL_NAME.equals(schemaName) || Time.LOGICAL_NAME.equals(schemaName)) { + jsonMap.put(fieldName, (java.util.Date) struct.get(fieldName)); + } else { + jsonMap.put(fieldName, struct.getInt32(fieldName)); + } + break; + case INT16: + jsonMap.put(fieldName, struct.getInt16(fieldName)); + break; + case INT64: + if (Timestamp.LOGICAL_NAME.equals(schemaName)) { + jsonMap.put(fieldName, (java.util.Date) struct.get(fieldName)); + } else { + jsonMap.put(fieldName, struct.getInt64(fieldName)); + } + break; + case FLOAT32: + jsonMap.put(fieldName, struct.getFloat32(fieldName)); + break; + case FLOAT64: + jsonMap.put(fieldName, struct.getFloat64(fieldName)); + break; + case BOOLEAN: + jsonMap.put(fieldName, struct.getBoolean(fieldName)); + break; + case ARRAY: + List fieldArray = struct.getArray(fieldName); + if (fieldArray != null && !fieldArray.isEmpty() && fieldArray.get(0) instanceof Struct) { + // If Array contains list of Structs + List jsonArray = new ArrayList<>(); + fieldArray.forEach(item -> { + jsonArray.add(toJsonMap((Struct) item)); + }); + jsonMap.put(fieldName, jsonArray); + } else { + jsonMap.put(fieldName, fieldArray); + } + break; + case STRUCT: + jsonMap.put(fieldName, toJsonMap(struct.getStruct(fieldName))); + break; + case MAP: + jsonMap.put(fieldName, handleMap(struct.getMap(fieldName))); + break; + default: + jsonMap.put(fieldName, struct.get(fieldName)); + break; + } + } + return jsonMap; + } + + public static Map handleMap(Map map) { + if (map == null) { + return null; + } + Map cacheMap = new HashMap<>(); + map.forEach((key, value) -> { + if (value instanceof Map) { + cacheMap.put(key, handleMap((Map) value)); + } else if (value instanceof Struct) { + cacheMap.put(key, toJsonMap((Struct) value)); + } else if (value instanceof List) { + List list = (List) value; + List jsonArray = new ArrayList<>(); + list.forEach(item -> { + if (item instanceof Struct) { + jsonArray.add(toJsonMap((Struct) item)); + } else if (item instanceof Map) { + jsonArray.add(handleMap((Map) item)); + } else { + jsonArray.add(item); + } + }); + cacheMap.put(key, jsonArray); + } else { + cacheMap.put(key, value); + } + }); + return cacheMap; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ToleranceOnErrorLevel.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ToleranceOnErrorLevel.java new file mode 100644 index 0000000000000..d169fd2484b0a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/ToleranceOnErrorLevel.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +public enum ToleranceOnErrorLevel { + NONE("None"), + ALL("All"); + + private final String name; + + ToleranceOnErrorLevel(String name) { + this.name = name; + } + + public String getName() { + return name; + } + + public static ToleranceOnErrorLevel fromName(String name) { + for (ToleranceOnErrorLevel mode : ToleranceOnErrorLevel.values()) { + if (mode.getName().equalsIgnoreCase(name)) { + return mode; + } + } + return null; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedModes.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedModes.java index 3753001f3e44e..9917a19145c8c 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedModes.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedModes.java @@ -18,7 +18,7 @@ public String getName() { public static CosmosChangeFeedModes fromName(String name) { for (CosmosChangeFeedModes mode : CosmosChangeFeedModes.values()) { - if (mode.getName().equals(name)) { + if (mode.getName().equalsIgnoreCase(name)) { return mode; } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedStartFromModes.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedStartFromModes.java index 439df9f826a5d..15be8610bceed 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedStartFromModes.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosChangeFeedStartFromModes.java @@ -19,7 +19,7 @@ public String getName() { public static CosmosChangeFeedStartFromModes fromName(String name) { for (CosmosChangeFeedStartFromModes startFromModes : CosmosChangeFeedStartFromModes.values()) { - if (startFromModes.getName().equals(name)) { + if (startFromModes.getName().equalsIgnoreCase(name)) { return startFromModes; } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java index 1ede731b5498c..e32d99ed1e8b3 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java @@ -14,6 +14,7 @@ import java.time.format.DateTimeParseException; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** * Common Configuration for Cosmos DB Kafka source connector. @@ -21,75 +22,75 @@ public class CosmosSourceConfig extends CosmosConfig { // configuration only targets to source connector - private static final String SOURCE_CONFIG_PREFIX = "kafka.connect.cosmos.source."; + private static final String SOURCE_CONF_PREFIX = "kafka.connect.cosmos.source."; // database name - private static final String DATABASE_NAME_CONF = SOURCE_CONFIG_PREFIX + "database.name"; + private static final String DATABASE_NAME_CONF = SOURCE_CONF_PREFIX + "database.name"; private static final String DATABASE_NAME_CONF_DOC = "Cosmos DB database name."; private static final String DATABASE_NAME_CONF_DISPLAY = "Cosmos DB database name."; // Source containers config - private static final String CONTAINERS_INCLUDE_ALL_CONFIG = SOURCE_CONFIG_PREFIX + "containers.includeAll"; - private static final String CONTAINERS_INCLUDE_ALL_CONFIG_DOC = "Flag to indicate whether reading from all containers."; - private static final String CONTAINERS_INCLUDE_ALL_CONFIG_DISPLAY = "Include all containers."; + private static final String CONTAINERS_INCLUDE_ALL_CONF = SOURCE_CONF_PREFIX + "containers.includeAll"; + private static final String CONTAINERS_INCLUDE_ALL_CONF_DOC = "Flag to indicate whether reading from all containers."; + private static final String CONTAINERS_INCLUDE_ALL_CONF_DISPLAY = "Include all containers."; private static final boolean DEFAULT_CONTAINERS_INCLUDE_ALL = false; - private static final String CONTAINERS_INCLUDED_LIST_CONFIG = SOURCE_CONFIG_PREFIX + "containers.includedList"; - private static final String CONTAINERS_INCLUDED_LIST_CONFIG_DOC = + private static final String CONTAINERS_INCLUDED_LIST_CONF = SOURCE_CONF_PREFIX + "containers.includedList"; + private static final String CONTAINERS_INCLUDED_LIST_CONF_DOC = "Containers included. This config will be ignored if kafka.connect.cosmos.source.includeAllContainers is true."; - private static final String CONTAINERS_INCLUDED_LIST_CONFIG_DISPLAY = "Containers included."; + private static final String CONTAINERS_INCLUDED_LIST_CONF_DISPLAY = "Containers included."; - private static final String CONTAINERS_TOPIC_MAP_CONFIG = SOURCE_CONFIG_PREFIX + "containers.topicMap"; - private static final String CONTAINERS_TOPIC_MAP_CONFIG_DOC = + private static final String CONTAINERS_TOPIC_MAP_CONF = SOURCE_CONF_PREFIX + "containers.topicMap"; + private static final String CONTAINERS_TOPIC_MAP_CONF_DOC = "A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2. " + "By default, container name is used as the name of the kafka topic to publish data to, " + "can use this property to override the default config "; - private static final String CONTAINERS_TOPIC_MAP_CONFIG_DISPLAY = "Cosmos container topic map."; + private static final String CONTAINERS_TOPIC_MAP_CONF_DISPLAY = "Cosmos container topic map."; // changeFeed config - private static final String CHANGE_FEED_START_FROM_CONFIG = SOURCE_CONFIG_PREFIX + "changeFeed.startFrom"; - private static final String CHANGE_FEED_START_FROM_CONFIG_DOC = "ChangeFeed Start from settings (Now, Beginning " + private static final String CHANGE_FEED_START_FROM_CONF = SOURCE_CONF_PREFIX + "changeFeed.startFrom"; + private static final String CHANGE_FEED_START_FROM_CONF_DOC = "ChangeFeed Start from settings (Now, Beginning " + "or a certain point in time (UTC) for example 2020-02-10T14:15:03) - the default value is 'Beginning'. "; - private static final String CHANGE_FEED_START_FROM_CONFIG_DISPLAY = "Change feed start from."; + private static final String CHANGE_FEED_START_FROM_CONF_DISPLAY = "Change feed start from."; private static final String DEFAULT_CHANGE_FEED_START_FROM = CosmosChangeFeedStartFromModes.BEGINNING.getName(); - private static final String CHANGE_FEED_MODE_CONFIG = SOURCE_CONFIG_PREFIX + "changeFeed.mode"; - private static final String CHANGE_FEED_MODE_CONFIG_DOC = "ChangeFeed mode (LatestVersion or AllVersionsAndDeletes)"; - private static final String CHANGE_FEED_MODE_CONFIG_DISPLAY = "ChangeFeed mode (LatestVersion or AllVersionsAndDeletes)"; + private static final String CHANGE_FEED_MODE_CONF = SOURCE_CONF_PREFIX + "changeFeed.mode"; + private static final String CHANGE_FEED_MODE_CONF_DOC = "ChangeFeed mode (LatestVersion or AllVersionsAndDeletes)"; + private static final String CHANGE_FEED_MODE_CONF_DISPLAY = "ChangeFeed mode (LatestVersion or AllVersionsAndDeletes)"; private static final String DEFAULT_CHANGE_FEED_MODE = CosmosChangeFeedModes.LATEST_VERSION.getName(); - private static final String CHANGE_FEED_MAX_ITEM_COUNT_CONFIG = SOURCE_CONFIG_PREFIX + "changeFeed.maxItemCountHint"; - private static final String CHANGE_FEED_MAX_ITEM_COUNT_CONFIG_DOC = + private static final String CHANGE_FEED_MAX_ITEM_COUNT_CONF = SOURCE_CONF_PREFIX + "changeFeed.maxItemCountHint"; + private static final String CHANGE_FEED_MAX_ITEM_COUNT_CONF_DOC = "The maximum number of documents returned in a single change feed request." + " But the number of items received might be higher than the specified value if multiple items are changed by the same transaction." + " The default is 1000."; - private static final String CHANGE_FEED_MAX_ITEM_COUNT_CONFIG_DISPLAY = "The maximum number hint of documents returned in a single request. "; + private static final String CHANGE_FEED_MAX_ITEM_COUNT_CONF_DISPLAY = "The maximum number hint of documents returned in a single request. "; private static final int DEFAULT_CHANGE_FEED_MAX_ITEM_COUNT = 1000; // Metadata config - private static final String METADATA_POLL_DELAY_MS_CONFIG = SOURCE_CONFIG_PREFIX + "metadata.poll.delay.ms"; - private static final String METADATA_POLL_DELAY_MS_CONFIG_DOC = + private static final String METADATA_POLL_DELAY_MS_CONF = SOURCE_CONF_PREFIX + "metadata.poll.delay.ms"; + private static final String METADATA_POLL_DELAY_MS_CONF_DOC = "Indicates how often to check the metadata changes (including container split/merge, adding/removing/recreated containers). " + "When changes are detected, it will reconfigure the tasks. Default is 5 minutes."; - private static final String METADATA_POLL_DELAY_MS_CONFIG_DISPLAY = "Metadata polling delay in ms."; + private static final String METADATA_POLL_DELAY_MS_CONF_DISPLAY = "Metadata polling delay in ms."; private static final int DEFAULT_METADATA_POLL_DELAY_MS = 5 * 60 * 1000; // default is every 5 minutes - private static final String METADATA_STORAGE_TOPIC_CONFIG = SOURCE_CONFIG_PREFIX + "metadata.storage.topic"; - private static final String METADATA_STORAGE_TOPIC_CONFIG_DOC = "The name of the topic where the metadata are stored. " + private static final String METADATA_STORAGE_TOPIC_CONF = SOURCE_CONF_PREFIX + "metadata.storage.topic"; + private static final String METADATA_STORAGE_TOPIC_CONF_DOC = "The name of the topic where the metadata are stored. " + "The metadata topic will be created if it does not already exist, else it will use the pre-created topic."; - private static final String METADATA_STORAGE_TOPIC_CONFIG_DISPLAY = "Metadata storage topic."; + private static final String METADATA_STORAGE_TOPIC_CONF_DISPLAY = "Metadata storage topic."; private static final String DEFAULT_METADATA_STORAGE_TOPIC = "_cosmos.metadata.topic"; // messageKey - private static final String MESSAGE_KEY_ENABLED_CONF = SOURCE_CONFIG_PREFIX + "messageKey.enabled"; + private static final String MESSAGE_KEY_ENABLED_CONF = SOURCE_CONF_PREFIX + "messageKey.enabled"; private static final String MESSAGE_KEY_ENABLED_CONF_DOC = "Whether to set the kafka record message key."; private static final String MESSAGE_KEY_ENABLED_CONF_DISPLAY = "Kafka record message key enabled."; private static final boolean DEFAULT_MESSAGE_KEY_ENABLED = true; - private static final String MESSAGE_KEY_FIELD_CONFIG = SOURCE_CONFIG_PREFIX + "messageKey.field"; - private static final String MESSAGE_KEY_FIELD_CONFIG_DOC = "The field to use as the message key."; - private static final String MESSAGE_KEY_FIELD_CONFIG_DISPLAY = "Kafka message key field."; - private static final String DEFAULT_MESSAGE_KEY_FIELD = "id"; // TODO: should we use pk instead? + private static final String MESSAGE_KEY_FIELD_CONF = SOURCE_CONF_PREFIX + "messageKey.field"; + private static final String MESSAGE_KEY_FIELD_CONF_DOC = "The field to use as the message key."; + private static final String MESSAGE_KEY_FIELD_CONF_DISPLAY = "Kafka message key field."; + private static final String DEFAULT_MESSAGE_KEY_FIELD = "id"; private final CosmosSourceContainersConfig containersConfig; private final CosmosMetadataConfig metadataConfig; @@ -137,38 +138,38 @@ private static void defineContainersConfig(ConfigDef result) { DATABASE_NAME_CONF_DISPLAY ) .define( - CONTAINERS_INCLUDE_ALL_CONFIG, + CONTAINERS_INCLUDE_ALL_CONF, ConfigDef.Type.BOOLEAN, DEFAULT_CONTAINERS_INCLUDE_ALL, ConfigDef.Importance.HIGH, - CONTAINERS_INCLUDE_ALL_CONFIG_DOC, + CONTAINERS_INCLUDE_ALL_CONF_DOC, containersGroupName, containersGroupOrder++, ConfigDef.Width.MEDIUM, - CONTAINERS_INCLUDE_ALL_CONFIG_DISPLAY + CONTAINERS_INCLUDE_ALL_CONF_DISPLAY ) .define( - CONTAINERS_INCLUDED_LIST_CONFIG, + CONTAINERS_INCLUDED_LIST_CONF, ConfigDef.Type.STRING, Strings.Emtpy, ConfigDef.Importance.MEDIUM, - CONTAINERS_INCLUDED_LIST_CONFIG_DOC, + CONTAINERS_INCLUDED_LIST_CONF_DOC, containersGroupName, containersGroupOrder++, ConfigDef.Width.LONG, - CONTAINERS_INCLUDED_LIST_CONFIG_DISPLAY + CONTAINERS_INCLUDED_LIST_CONF_DISPLAY ) .define( - CONTAINERS_TOPIC_MAP_CONFIG, + CONTAINERS_TOPIC_MAP_CONF, ConfigDef.Type.STRING, Strings.Emtpy, new ContainersTopicMapValidator(), ConfigDef.Importance.MEDIUM, - CONTAINERS_TOPIC_MAP_CONFIG_DOC, + CONTAINERS_TOPIC_MAP_CONF_DOC, containersGroupName, containersGroupOrder++, ConfigDef.Width.LONG, - CONTAINERS_TOPIC_MAP_CONFIG_DISPLAY + CONTAINERS_TOPIC_MAP_CONF_DISPLAY ); } @@ -178,28 +179,28 @@ private static void defineMetadataConfig(ConfigDef result) { result .define( - METADATA_POLL_DELAY_MS_CONFIG, + METADATA_POLL_DELAY_MS_CONF, ConfigDef.Type.INT, DEFAULT_METADATA_POLL_DELAY_MS, new PositiveValueValidator(), ConfigDef.Importance.MEDIUM, - METADATA_POLL_DELAY_MS_CONFIG_DOC, + METADATA_POLL_DELAY_MS_CONF_DOC, metadataGroupName, metadataGroupOrder++, ConfigDef.Width.MEDIUM, - METADATA_POLL_DELAY_MS_CONFIG_DISPLAY + METADATA_POLL_DELAY_MS_CONF_DISPLAY ) .define( - METADATA_STORAGE_TOPIC_CONFIG, + METADATA_STORAGE_TOPIC_CONF, ConfigDef.Type.STRING, DEFAULT_METADATA_STORAGE_TOPIC, NON_EMPTY_STRING, ConfigDef.Importance.HIGH, - METADATA_STORAGE_TOPIC_CONFIG_DOC, + METADATA_STORAGE_TOPIC_CONF_DOC, metadataGroupName, metadataGroupOrder++, ConfigDef.Width.LONG, - METADATA_STORAGE_TOPIC_CONFIG_DISPLAY + METADATA_STORAGE_TOPIC_CONF_DISPLAY ); } @@ -209,40 +210,40 @@ private static void defineChangeFeedConfig(ConfigDef result) { result .define( - CHANGE_FEED_MODE_CONFIG, + CHANGE_FEED_MODE_CONF, ConfigDef.Type.STRING, DEFAULT_CHANGE_FEED_MODE, new ChangeFeedModeValidator(), ConfigDef.Importance.HIGH, - CHANGE_FEED_MODE_CONFIG_DOC, + CHANGE_FEED_MODE_CONF_DOC, changeFeedGroupName, changeFeedGroupOrder++, ConfigDef.Width.MEDIUM, - CHANGE_FEED_MODE_CONFIG_DISPLAY + CHANGE_FEED_MODE_CONF_DISPLAY ) .define( - CHANGE_FEED_START_FROM_CONFIG, + CHANGE_FEED_START_FROM_CONF, ConfigDef.Type.STRING, DEFAULT_CHANGE_FEED_START_FROM, new ChangeFeedStartFromValidator(), ConfigDef.Importance.HIGH, - CHANGE_FEED_START_FROM_CONFIG_DOC, + CHANGE_FEED_START_FROM_CONF_DOC, changeFeedGroupName, changeFeedGroupOrder++, ConfigDef.Width.MEDIUM, - CHANGE_FEED_START_FROM_CONFIG_DISPLAY + CHANGE_FEED_START_FROM_CONF_DISPLAY ) .define( - CHANGE_FEED_MAX_ITEM_COUNT_CONFIG, + CHANGE_FEED_MAX_ITEM_COUNT_CONF, ConfigDef.Type.INT, DEFAULT_CHANGE_FEED_MAX_ITEM_COUNT, new PositiveValueValidator(), ConfigDef.Importance.MEDIUM, - CHANGE_FEED_MAX_ITEM_COUNT_CONFIG_DOC, + CHANGE_FEED_MAX_ITEM_COUNT_CONF_DOC, changeFeedGroupName, changeFeedGroupOrder++, ConfigDef.Width.MEDIUM, - CHANGE_FEED_MAX_ITEM_COUNT_CONFIG_DISPLAY + CHANGE_FEED_MAX_ITEM_COUNT_CONF_DISPLAY ); } @@ -263,23 +264,23 @@ private static void defineMessageKeyConfig(ConfigDef result) { MESSAGE_KEY_ENABLED_CONF_DISPLAY ) .define( - MESSAGE_KEY_FIELD_CONFIG, + MESSAGE_KEY_FIELD_CONF, ConfigDef.Type.STRING, DEFAULT_MESSAGE_KEY_FIELD, ConfigDef.Importance.HIGH, - MESSAGE_KEY_FIELD_CONFIG_DOC, + MESSAGE_KEY_FIELD_CONF_DOC, messageGroupName, messageGroupOrder++, ConfigDef.Width.MEDIUM, - MESSAGE_KEY_FIELD_CONFIG_DISPLAY + MESSAGE_KEY_FIELD_CONF_DISPLAY ); } private CosmosSourceContainersConfig parseContainersConfig() { String databaseName = this.getString(DATABASE_NAME_CONF); - boolean includeAllContainers = this.getBoolean(CONTAINERS_INCLUDE_ALL_CONFIG); + boolean includeAllContainers = this.getBoolean(CONTAINERS_INCLUDE_ALL_CONF); List containersIncludedList = this.getContainersIncludedList(); - List containersTopicMap = this.getContainersTopicMap(); + Map containersTopicMap = this.getContainerToTopicMap(); return new CosmosSourceContainersConfig( databaseName, @@ -290,16 +291,23 @@ private CosmosSourceContainersConfig parseContainersConfig() { } private List getContainersIncludedList() { - return convertToList(this.getString(CONTAINERS_INCLUDED_LIST_CONFIG)); + return convertToList(this.getString(CONTAINERS_INCLUDED_LIST_CONF)); } - private List getContainersTopicMap() { - return convertToList(this.getString(CONTAINERS_TOPIC_MAP_CONFIG)); + private Map getContainerToTopicMap() { + List containerTopicMapList = convertToList(this.getString(CONTAINERS_TOPIC_MAP_CONF)); + return containerTopicMapList + .stream() + .map(containerTopicMapString -> containerTopicMapString.split("#")) + .collect( + Collectors.toMap( + containerTopicMapArray -> containerTopicMapArray[1], + containerTopicMapArray -> containerTopicMapArray[0])); } private CosmosMetadataConfig parseMetadataConfig() { - int metadataPollDelayInMs = this.getInt(METADATA_POLL_DELAY_MS_CONFIG); - String metadataTopicName = this.getString(METADATA_STORAGE_TOPIC_CONFIG); + int metadataPollDelayInMs = this.getInt(METADATA_POLL_DELAY_MS_CONF); + String metadataTopicName = this.getString(METADATA_STORAGE_TOPIC_CONF); return new CosmosMetadataConfig(metadataPollDelayInMs, metadataTopicName); } @@ -308,7 +316,7 @@ private CosmosSourceChangeFeedConfig parseChangeFeedConfig() { CosmosChangeFeedModes changeFeedModes = this.parseChangeFeedMode(); CosmosChangeFeedStartFromModes changeFeedStartFromMode = this.parseChangeFeedStartFromMode(); Instant changeFeedStartFrom = this.parseChangeFeedStartFrom(changeFeedStartFromMode); - Integer changeFeedMaxItemCountHint = this.getInt(CHANGE_FEED_MAX_ITEM_COUNT_CONFIG); + Integer changeFeedMaxItemCountHint = this.getInt(CHANGE_FEED_MAX_ITEM_COUNT_CONF); return new CosmosSourceChangeFeedConfig( changeFeedModes, @@ -319,12 +327,12 @@ private CosmosSourceChangeFeedConfig parseChangeFeedConfig() { private CosmosSourceMessageKeyConfig parseMessageKeyConfig() { boolean messageKeyEnabled = this.getBoolean(MESSAGE_KEY_ENABLED_CONF); - String messageKeyField = this.getString(MESSAGE_KEY_FIELD_CONFIG); + String messageKeyField = this.getString(MESSAGE_KEY_FIELD_CONF); return new CosmosSourceMessageKeyConfig(messageKeyEnabled, messageKeyField); } private CosmosChangeFeedStartFromModes parseChangeFeedStartFromMode() { - String changeFeedStartFrom = this.getString(CHANGE_FEED_START_FROM_CONFIG); + String changeFeedStartFrom = this.getString(CHANGE_FEED_START_FROM_CONF); if (changeFeedStartFrom.equalsIgnoreCase(CosmosChangeFeedStartFromModes.BEGINNING.getName())) { return CosmosChangeFeedStartFromModes.BEGINNING; } @@ -338,7 +346,7 @@ private CosmosChangeFeedStartFromModes parseChangeFeedStartFromMode() { private Instant parseChangeFeedStartFrom(CosmosChangeFeedStartFromModes startFromMode) { if (startFromMode == CosmosChangeFeedStartFromModes.POINT_IN_TIME) { - String changeFeedStartFrom = this.getString(CHANGE_FEED_START_FROM_CONFIG); + String changeFeedStartFrom = this.getString(CHANGE_FEED_START_FROM_CONF); return Instant.from(DateTimeFormatter.ISO_INSTANT.parse(changeFeedStartFrom.trim())); } @@ -346,7 +354,7 @@ private Instant parseChangeFeedStartFrom(CosmosChangeFeedStartFromModes startFro } private CosmosChangeFeedModes parseChangeFeedMode() { - String changeFeedMode = this.getString(CHANGE_FEED_MODE_CONFIG); + String changeFeedMode = this.getString(CHANGE_FEED_MODE_CONF); return CosmosChangeFeedModes.fromName(changeFeedMode); } @@ -366,42 +374,6 @@ public CosmosSourceMessageKeyConfig getMessageKeyConfig() { return messageKeyConfig; } - public static class ContainersTopicMapValidator implements ConfigDef.Validator { - private static final String INVALID_TOPIC_MAP_FORMAT = - "Invalid entry for topic-container map. The topic-container map should be a comma-delimited " - + "list of Kafka topic to Cosmos containers. Each mapping should be a pair of Kafka " - + "topic and Cosmos container separated by '#'. For example: topic1#con1,topic2#con2."; - - @Override - @SuppressWarnings("unchecked") - public void ensureValid(String name, Object o) { - String configValue = (String) o; - if (StringUtils.isEmpty(configValue)) { - return; - } - - List containerTopicMapList = convertToList(configValue); - - // validate each item should be in topic#container format - boolean invalidFormatExists = - containerTopicMapList - .stream() - .anyMatch(containerTopicMap -> - containerTopicMap - .split(CosmosSourceContainersConfig.CONTAINER_TOPIC_MAP_SEPARATOR) - .length != 2); - - if (invalidFormatExists) { - throw new ConfigException(name, o, INVALID_TOPIC_MAP_FORMAT); - } - } - - @Override - public String toString() { - return "Containers topic map"; - } - } - public static class ChangeFeedModeValidator implements ConfigDef.Validator { @Override @SuppressWarnings("unchecked") diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java index 2d61b49fedb06..7f9a2d4284e85 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java @@ -6,6 +6,7 @@ import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import java.util.List; +import java.util.Map; import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; @@ -16,13 +17,13 @@ public class CosmosSourceContainersConfig { private final String databaseName; private final boolean includeAllContainers; private final List includedContainers; - private final List containersTopicMap; + private final Map containerToTopicMap; public CosmosSourceContainersConfig( String databaseName, boolean includeAllContainers, List includedContainers, - List containersTopicMap) { + Map containerToTopicMap) { checkArgument(StringUtils.isNotEmpty(databaseName), "Argument 'databaseName' can not be null"); checkNotNull(includedContainers, "Argument 'includedContainers' can not be null"); @@ -30,7 +31,7 @@ public CosmosSourceContainersConfig( this.databaseName = databaseName; this.includeAllContainers = includeAllContainers; this.includedContainers = includedContainers; - this.containersTopicMap = containersTopicMap; + this.containerToTopicMap = containerToTopicMap; } public String getDatabaseName() { @@ -45,7 +46,7 @@ public List getIncludedContainers() { return includedContainers; } - public List getContainersTopicMap() { - return containersTopicMap; + public Map getContainerToTopicMap() { + return containerToTopicMap; } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java index 61a6ceabd7163..0372af2bebbd4 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java @@ -14,7 +14,7 @@ import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.guava25.base.Stopwatch; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; -import com.azure.cosmos.kafka.connect.implementation.CosmosConstants; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; import com.azure.cosmos.models.CosmosChangeFeedRequestOptions; import com.azure.cosmos.models.FeedRange; @@ -46,7 +46,7 @@ public class CosmosSourceTask extends SourceTask { @Override public String version() { - return CosmosConstants.CURRENT_VERSION; + return KafkaCosmosConstants.CURRENT_VERSION; } @Override diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java new file mode 100644 index 0000000000000..cfc959db4ef54 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java @@ -0,0 +1,113 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect; + +import com.azure.cosmos.CosmosAsyncClient; +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; +import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.rnorth.ducttape.unreliables.Unreliables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +public class CosmosDbSourceConnectorITest extends KafkaCosmosIntegrationTestSuiteBase { + private static final Logger logger = LoggerFactory.getLogger(CosmosDbSourceConnectorITest.class); + + // TODO[public preview]: add more integration tests + @Test(groups = { "kafka-integration"}, timeOut = TIMEOUT) + public void readFromSingleContainer() { + Map sourceConnectorConfig = new HashMap<>(); + sourceConnectorConfig.put("connector.class", "com.azure.cosmos.kafka.connect.CosmosDBSourceConnector"); + sourceConnectorConfig.put("kafka.connect.cosmos.accountEndpoint", KafkaCosmosTestConfigurations.HOST); + sourceConnectorConfig.put("kafka.connect.cosmos.accountKey", KafkaCosmosTestConfigurations.MASTER_KEY); + sourceConnectorConfig.put("kafka.connect.cosmos.applicationName", "Test"); + sourceConnectorConfig.put("kafka.connect.cosmos.source.database.name", databaseName); + sourceConnectorConfig.put("kafka.connect.cosmos.source.containers.includeAll", "false"); + sourceConnectorConfig.put("kafka.connect.cosmos.source.containers.includedList", singlePartitionContainerName); + sourceConnectorConfig.put("kafka.connect.cosmos.source.containers.topicMap", "TestTopic#" + singlePartitionContainerName); + + + // TODO[this PR]: create the topic ahead of time + CosmosSourceConfig sourceConfig = new CosmosSourceConfig(sourceConnectorConfig); + CosmosAsyncClient client = CosmosClientStore.getCosmosClient(sourceConfig.getAccountConfig()); + CosmosAsyncContainer container = client.getDatabase(databaseName).getContainer(singlePartitionContainerName); + + String connectorName = "simpleTest-" + UUID.randomUUID(); + try { + // create few items in the container + logger.info("creating items in container {}", singlePartitionContainerName); + List createdItems = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + TestItem testItem = TestItem.createNewItem(); + container.createItem(testItem).block(); + createdItems.add(testItem.getId()); + } + + kafkaCosmosConnectContainer.registerConnector(connectorName, sourceConnectorConfig); + + logger.info("Getting consumer and subscribe to topic {}", singlePartitionContainerName); + KafkaConsumer kafkaConsumer = kafkaCosmosConnectContainer.getConsumer(); + kafkaConsumer.subscribe( + Arrays.asList( + "TestTopic", + sourceConfig.getMetadataConfig().getMetadataTopicName())); + + List> metadataRecords = new ArrayList<>(); + List> itemRecords = new ArrayList<>(); + + Unreliables.retryUntilTrue(30, TimeUnit.SECONDS, () -> {; + kafkaConsumer.poll(Duration.ofMillis(1000)) + .iterator() + .forEachRemaining(consumerRecord -> { + if (consumerRecord.topic().equals("TestTopic")) { + itemRecords.add(consumerRecord); + } else if (consumerRecord.topic().equals(sourceConfig.getMetadataConfig().getMetadataTopicName())) { + metadataRecords.add(consumerRecord); + } + }); + return metadataRecords.size() >= 2 && itemRecords.size() >= createdItems.size(); + }); + + //TODO[public preview]currently the metadata record value is null, populate it with metadata and validate the content here + assertThat(metadataRecords.size()).isEqualTo(2); + assertThat(itemRecords.size()).isEqualTo(createdItems.size()); + + List receivedItems = + itemRecords.stream().map(consumerRecord -> { + JsonNode jsonNode = consumerRecord.value(); + return jsonNode.get("payload").get("id").asText(); + }).collect(Collectors.toList()); + + assertThat(receivedItems.containsAll(createdItems)).isTrue(); + + } finally { + if (client != null) { + logger.info("cleaning container {}", singlePartitionContainerName); + cleanUpContainer(client, databaseName, singlePartitionContainerName); + client.close(); + } + + // IMPORTANT: remove the connector after use + if (kafkaCosmosConnectContainer != null) { + kafkaCosmosConnectContainer.deleteConnector(connectorName); + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java new file mode 100644 index 0000000000000..32555f32a7f9d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java @@ -0,0 +1,208 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect; + +import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.implementation.guava25.base.Strings; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.InputStream; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Properties; + +public class KafkaCosmosTestConfigurations { + private static final Logger logger = LoggerFactory.getLogger(KafkaCosmosTestConfigurations.class); + private static Properties properties = loadProperties(); + + private static final String COSMOS_EMULATOR_KEY = "C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=="; + private static final String COSMOS_EMULATOR_HOST = "https://localhost:8081/"; + public static final String DEFAULT_CONFLUENT_VERSION = "7.6.0"; //https://docs.confluent.io/platform/current/installation/versions-interoperability.html + public static final String DEFAULT_CONNECT_GROUP_ID = "1"; + public static final String DEFAULT_CONNECT_CONFIG_STORAGE_TOPIC = "docker-connect-configs"; + public static final String DEFAULT_CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR = "1"; + public static final String DEFAULT_CONNECT_OFFSET_STORAGE_TOPIC = "docker-connect-offsets"; + public static final String DEFAULT_CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR = "1"; + public static final String DEFAULT_CONNECT_STATUS_STORAGE_TOPIC = "docker-connect-status"; + public static final String DEFAULT_CONNECT_STATUS_STORAGE_REPLICATION_FACTOR = "1"; + public static final String DEFAULT_CONNECT_KEY_CONVERTER = "org.apache.kafka.connect.storage.StringConverter"; + public static final String DEFAULT_CONNECT_VALUE_CONVERTER = "org.apache.kafka.connect.json.JsonConverter"; + public static final String DEFAULT_CONNECT_PLUGIN_PATH = "/usr/share/java,/kafka/connect/cosmos-connector"; + public static final String DEFAULT_CONNECT_REST_ADVERTISED_HOST_NAME = "connect"; + + public final static String MASTER_KEY = + properties + .getProperty( + "ACCOUNT_KEY", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("ACCOUNT_KEY")), COSMOS_EMULATOR_KEY)); + + public final static String SECONDARY_MASTER_KEY = + properties + .getProperty( + "SECONDARY_ACCOUNT_KEY", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("SECONDARY_ACCOUNT_KEY")), COSMOS_EMULATOR_KEY)); + + public final static String HOST = + properties + .getProperty( + "ACCOUNT_HOST", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("ACCOUNT_HOST")), COSMOS_EMULATOR_HOST)); + + public final static String KAFKA_CLUSTER_KEY = + properties + .getProperty( + "KAFKA_CLUSTER_KEY", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("KAFKA_CLUSTER_KEY")), "")); + + public final static String KAFKA_CLUSTER_SECRET = + properties + .getProperty( + "KAFKA_CLUSTER_SECRET", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("KAFKA_CLUSTER_SECRET")), "")); + + public final static String SCHEMA_REGISTRY_KEY = + properties + .getProperty( + "SCHEMA_REGISTRY_KEY", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("SCHEMA_REGISTRY_KEY")), "")); + + public final static String SCHEMA_REGISTRY_SECRET = + properties + .getProperty( + "SCHEMA_REGISTRY_SECRET", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("SCHEMA_REGISTRY_SECRET")), "")); + + public final static String SCHEMA_REGISTRY_URL = + properties + .getProperty( + "SCHEMA_REGISTRY_URL", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("SCHEMA_REGISTRY_URL")), "")); + + public final static String BOOTSTRAP_SERVER = + properties + .getProperty( + "BOOTSTRAP_SERVER", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("BOOTSTRAP_SERVER")), "")); + + public final static String SASL_JAAS = + properties + .getProperty( + "SASL_JAAS", + StringUtils.defaultString( + Strings.emptyToNull(System.getenv().get("SASL_JAAS")), "")); + + public final static String CONFLUENT_VERSION = + properties + .getProperty( + "CONFLUENT_VERSION", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONFLUENT_VERSION")), DEFAULT_CONFLUENT_VERSION)); + + public final static String CONNECT_GROUP_ID = + properties + .getProperty( + "CONNECT_GROUP_ID", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_GROUP_ID")), DEFAULT_CONNECT_GROUP_ID)); + + public final static String CONNECT_CONFIG_STORAGE_TOPIC = + properties + .getProperty( + "CONNECT_CONFIG_STORAGE_TOPIC", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_CONFIG_STORAGE_TOPIC")), DEFAULT_CONNECT_CONFIG_STORAGE_TOPIC)); + + public final static String CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR = + properties + .getProperty( + "CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR")), DEFAULT_CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR)); + + public final static String CONNECT_OFFSET_STORAGE_TOPIC = + properties + .getProperty( + "CONNECT_OFFSET_STORAGE_TOPIC", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_OFFSET_STORAGE_TOPIC")), DEFAULT_CONNECT_OFFSET_STORAGE_TOPIC)); + + public final static String CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR = + properties + .getProperty( + "CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR")), DEFAULT_CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR)); + + public final static String CONNECT_STATUS_STORAGE_TOPIC = + properties + .getProperty( + "CONNECT_STATUS_STORAGE_TOPIC", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_STATUS_STORAGE_TOPIC")), DEFAULT_CONNECT_STATUS_STORAGE_TOPIC)); + + public final static String CONNECT_STATUS_STORAGE_REPLICATION_FACTOR = + properties + .getProperty( + "CONNECT_STATUS_STORAGE_REPLICATION_FACTOR", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_STATUS_STORAGE_REPLICATION_FACTOR")), DEFAULT_CONNECT_STATUS_STORAGE_REPLICATION_FACTOR)); + + public final static String CONNECT_KEY_CONVERTER = + properties + .getProperty( + "CONNECT_KEY_CONVERTER", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_KEY_CONVERTER")), DEFAULT_CONNECT_KEY_CONVERTER)); + + public final static String CONNECT_VALUE_CONVERTER = + properties + .getProperty( + "CONNECT_VALUE_CONVERTER", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_VALUE_CONVERTER")), DEFAULT_CONNECT_VALUE_CONVERTER)); + + public final static String CONNECT_PLUGIN_PATH = + properties + .getProperty( + "CONNECT_PLUGIN_PATH", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_PLUGIN_PATH")), DEFAULT_CONNECT_PLUGIN_PATH)); + + public final static String CONNECT_REST_ADVERTISED_HOST_NAME = + properties + .getProperty( + "CONNECT_REST_ADVERTISED_HOST_NAME", + StringUtils.defaultString(Strings.emptyToNull(System.getenv().get("CONNECT_REST_ADVERTISED_HOST_NAME")), DEFAULT_CONNECT_REST_ADVERTISED_HOST_NAME)); + + private static Properties loadProperties() { + Path root = FileSystems.getDefault().getPath("").toAbsolutePath(); + Path propertiesInProject = Paths.get(root.toString(),"../kafka-cosmos.properties"); + + Properties props = loadFromPathIfExists(propertiesInProject); + if (props != null) { + return props; + } + + Path propertiesInUserHome = Paths.get(System.getProperty("user.home"), "kafka-cosmos.properties"); + props = loadFromPathIfExists(propertiesInUserHome); + if (props != null) { + return props; + } + + return System.getProperties(); + } + + private static Properties loadFromPathIfExists(Path propertiesFilePath) { + if (Files.exists(propertiesFilePath)) { + try (InputStream in = Files.newInputStream(propertiesFilePath)) { + Properties props = new Properties(); + props.load(in); + logger.info("properties loaded from {}", propertiesFilePath); + return props; + } catch (Exception e) { + logger.error("Loading properties {} failed", propertiesFilePath, e); + } + } + return null; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java index e73dbdb628f69..76479de109810 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java @@ -56,7 +56,7 @@ public void requestTaskReconfigurationOnContainersChange() throws InterruptedExc databaseName, true, new ArrayList(), - new ArrayList()); + new HashMap()); CosmosMetadataConfig metadataConfig = new CosmosMetadataConfig(500, "_cosmos.metadata.topic"); SourceConnectorContext sourceConnectorContext = Mockito.mock(SourceConnectorContext.class); @@ -105,7 +105,7 @@ public void requestTaskReconfigurationOnSplit() throws InterruptedException { databaseName, false, Arrays.asList(multiPartitionContainerName), - new ArrayList()); + new HashMap()); CosmosMetadataConfig metadataConfig = new CosmosMetadataConfig(500, "_cosmos.metadata.topic"); SourceConnectorContext sourceConnectorContext = Mockito.mock(SourceConnectorContext.class); @@ -176,7 +176,7 @@ public void requestTaskReconfigurationOnMerge() throws InterruptedException { databaseName, false, Arrays.asList(singlePartitionContainerName), - new ArrayList()); + new HashMap()); CosmosMetadataConfig metadataConfig = new CosmosMetadataConfig(500, "_cosmos.metadata.topic"); SourceConnectorContext sourceConnectorContext = Mockito.mock(SourceConnectorContext.class); diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java index c6aa862610c94..bd7a295bc9aeb 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java @@ -2784,6 +2784,11 @@ public Mono> trySplitFeedRange( return cosmosAsyncContainer.trySplitFeedRange(feedRange, targetedCountAfterSplit); } + + @Override + public String getLinkWithoutTrailingSlash(CosmosAsyncContainer cosmosAsyncContainer) { + return cosmosAsyncContainer.getLinkWithoutTrailingSlash(); + } }); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java index bfa5c6541777d..5378feeadf5f3 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java @@ -965,6 +965,8 @@ Mono> trySplitFeedRange( CosmosAsyncContainer cosmosAsyncContainer, FeedRange feedRange, int targetedCountAfterSplit); + + String getLinkWithoutTrailingSlash(CosmosAsyncContainer cosmosAsyncContainer); } } From 8838eddaa8f39a95348ece8c1c948331896185b7 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 27 Feb 2024 12:37:31 -0800 Subject: [PATCH 02/44] add sink connector v2 implementation --- .../doc/configuration-reference.md | 13 + sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 17 + .../connect/CosmosDBSourceConnector.java | 6 +- ....java => KafkaCosmosExceptionsHelper.java} | 12 +- .../sink/CosmosDBWriteException.java | 6 + .../implementation/sink/CosmosSinkConfig.java | 57 +- .../sink/CosmosSinkContainersConfig.java | 3 + .../implementation/sink/CosmosSinkTask.java | 66 +- .../sink/CosmosSinkWriteConfig.java | 10 +- .../connect/implementation/sink/IWriter.java | 5 +- .../sink/KafkaCosmosBulkWriter.java | 350 +++++++---- .../sink/KafkaCosmosPointWriter.java | 146 +++-- .../sink/KafkaCosmosWriterBase.java | 64 +- ...erationContext.java => SinkOperation.java} | 25 +- .../sink/SinkOperationFailedResponse.java | 22 - .../sink/SinkRecordTransformer.java | 45 +- .../sink/SinkWriteResponse.java | 28 - .../implementation/sink/StructToJsonMap.java | 3 + .../sink/idStrategy/AbstractIdStrategy.java | 27 + .../idStrategy/AbstractIdStrategyConfig.java | 19 + .../sink/idStrategy/FullKeyStrategy.java | 16 + .../sink/idStrategy/IdStrategy.java | 11 + .../idStrategy/KafkaMetadataStrategy.java | 22 + .../KafkaMetadataStrategyConfig.java | 53 ++ .../sink/idStrategy/ProvidedInConfig.java | 53 ++ .../idStrategy/ProvidedInKeyStrategy.java | 10 + .../sink/idStrategy/ProvidedInStrategy.java | 46 ++ .../idStrategy/ProvidedInValueStrategy.java | 10 + .../sink/idStrategy/TemplateStrategy.java | 71 +++ .../idStrategy/TemplateStrategyConfig.java | 58 ++ .../source/CosmosSourceTask.java | 6 +- .../source/MetadataMonitorThread.java | 4 +- .../src/main/java/module-info.java | 1 + .../connect/CosmosDBSinkConnectorTest.java | 154 +++++ .../connect/CosmosDBSourceConnectorTest.java | 70 +-- .../connect/CosmosDbSourceConnectorITest.java | 113 ---- .../kafka/connect/KafkaCosmosConfigEntry.java | 29 + .../connect/KafkaCosmosReflectionUtils.java | 10 + .../sink/CosmosSinkTaskTest.java | 571 ++++++++++++++++++ .../idStrategy/ProvidedInStrategyTest.java | 223 +++++++ 40 files changed, 1923 insertions(+), 532 deletions(-) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/{CosmosExceptionsHelper.java => KafkaCosmosExceptionsHelper.java} (88%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{SinkOperationContext.java => SinkOperation.java} (74%) delete mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java delete mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java delete mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConfigEntry.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskTest.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md b/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md index 8512ba8d7af79..eb00fb6a3e593 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md +++ b/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md @@ -23,3 +23,16 @@ | `kafka.connect.cosmos.source.metadata.storage.topic` | `_cosmos.metadata.topic` | The name of the topic where the metadata are stored. The metadata topic will be created if it does not already exist, else it will use the pre-created topic. | | `kafka.connect.cosmos.source.messageKey.enabled` | `true` | Whether to set the kafka record message key. | | `kafka.connect.cosmos.source.messageKey.field` | `id` | The field to use as the message key. | + +## Sink Connector Configuration +| Config Property Name | Default | Description | +|:---------------------------------------------------------------|:----------------|:-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `kafka.connect.cosmos.sink.database.name` | None | Cosmos DB database name. | +| `kafka.connect.cosmos.sink.containers.topicMap` | `[]` | A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2. | +| `kafka.connect.cosmos.sink.errors.tolerance` | `None` | Error tolerance level after exhausting all retries. `None` for fail on error. `All` for log and continue | +| `kafka.connect.cosmos.sink.bulk.enabled` | `true` | Flag to indicate whether Cosmos DB bulk mode is enabled for Sink connector. By default it is true. | +| `kafka.connect.cosmos.sink.bulk.maxConcurrentCosmosPartitions` | `-1` | Cosmos DB Item Write Max Concurrent Cosmos Partitions. If not specified it will be determined based on the number of the container's physical partitions which would indicate every Spark partition is expected to have data from all Cosmos physical partitions. If specified it indicates from at most how many Cosmos Physical Partitions each Spark partition contains data. So this config can be used to make bulk processing more efficient when input data in Spark has been repartitioned to balance to how many Cosmos partitions each Spark partition needs to write. This is mainly useful for very large containers (with hundreds of physical partitions. | +| `kafka.connect.cosmos.sink.bulk.initialBatchSize` | `1` | Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch size is getting automatically tuned based on the throttling rate. By default the initial micro batch size is 1. Reduce this when you want to avoid that the first few requests consume too many RUs. | +| `kafka.connect.cosmos.sink.write.strategy` | `ItemOverwrite` | "Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert), `ItemAppend` (using create, ignore pre-existing items i.e., Conflicts), `ItemDelete` (deletes based on id/pk of data frame), `ItemDeleteIfNotModified` (deletes based on id/pk of data frame if etag hasn't changed since collecting id/pk), `ItemOverwriteIfNotModified` (using create if etag is empty, update/replace with etag pre-condition otherwise, if document was updated the pre-condition failure is ignored) | +| `kafka.connect.cosmos.sink.maxRetryCount` | `10` | Cosmos DB max retry attempts on write failures for Sink connector. By default, the connector will retry on transient write errors for up to 10 times. | +| `kafka.connect.cosmos.sink.id.strategy` | `ProvidedInValueStrategy` | A strategy used to populate the document with an ``id``. Valid strategies are: ``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, ``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. Configuration properties prefixed with``id.strategy`` are passed through to the strategy. For example, when using ``id.strategy=TemplateStrategy`` , the property ``id.strategy.template`` is passed through to the template strategy and used to specify the template string to be used in constructing the ``id``. | diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index d62b13b01e530..8a211b230c1f8 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -47,8 +47,12 @@ Licensed under the MIT License. --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect=ALL-UNNAMED --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect.implementation=ALL-UNNAMED --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect.implementation.source=com.fasterxml.jackson.databind,ALL-UNNAMED + --add-opens com.azure.cosmos/com.azure.cosmos.implementation=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.routing=ALL-UNNAMED + --add-opens com.azure.cosmos/com.azure.cosmos.implementation.caches=ALL-UNNAMED + --add-opens com.azure.cosmos/com.azure.cosmos.implementation.faultinjection=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.apachecommons.lang=ALL-UNNAMED + --add-opens com.azure.cosmos/com.azure.cosmos.implementation.guava25.base=ALL-UNNAMED @@ -80,6 +84,13 @@ Licensed under the MIT License. provided + + com.azure + azure-cosmos-test + 1.0.0-beta.7 + test + + org.apache.commons commons-collections4 @@ -93,6 +104,11 @@ Licensed under the MIT License. test 1.10.0 + + com.jayway.jsonpath + json-path + 2.9.0 + org.apache.kafka @@ -218,6 +234,7 @@ Licensed under the MIT License. com.azure:* org.apache.kafka:connect-api:[3.6.0] io.confluent:kafka-connect-maven-plugin:[0.12.0] + com.fasterxml.jackson.core:jackson-databind:[2.13.5] diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java index 65fdfa2ff94a6..dc1dd43f730df 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java @@ -15,7 +15,7 @@ import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; -import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceOffsetStorageReader; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceTask; @@ -314,7 +314,7 @@ private List> getFeedRanges(CosmosContainerProperties containerPro .getContainer(containerProperties.getId()) .getFeedRanges() .onErrorMap(throwable -> - CosmosExceptionsHelper.convertToConnectException( + KafkaCosmosExceptionsHelper.convertToConnectException( throwable, "GetFeedRanges failed for container " + containerProperties.getId())) .block() @@ -324,7 +324,7 @@ private List> getFeedRanges(CosmosContainerProperties containerPro } private Map getContainersTopicMap(List allContainers) { - Map topicMapFromConfig = this.config.getContainersConfig().getContainersTopicMap(); + Map topicMapFromConfig = this.config.getContainersConfig().getContainerToTopicMap(); Map effectiveContainersTopicMap = new HashMap<>(); allContainers.forEach(containerProperties -> { // by default, we are using container id as the topic name as well unless customer override through containers.topicMap diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java similarity index 88% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java index 9924864705b5e..0d5a8bb8759e3 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosExceptionsHelper.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java @@ -8,7 +8,7 @@ import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; -public class CosmosExceptionsHelper { +public class KafkaCosmosExceptionsHelper { public static boolean isTransientFailure(int statusCode, int substatusCode) { return statusCode == HttpConstants.StatusCodes.GONE || statusCode == HttpConstants.StatusCodes.SERVICE_UNAVAILABLE @@ -43,7 +43,7 @@ public static boolean isFeedRangeGoneException(int statusCode, int substatusCode } public static ConnectException convertToConnectException(Throwable throwable, String message) { - if (CosmosExceptionsHelper.isTransientFailure(throwable)) { + if (KafkaCosmosExceptionsHelper.isTransientFailure(throwable)) { return new RetriableException(message, throwable); } @@ -73,4 +73,12 @@ public static boolean isPreconditionFailedException(Throwable throwable) { return false; } + + public static boolean isTimeoutException(Throwable throwable) { + if (throwable instanceof CosmosException) { + return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.REQUEST_TIMEOUT; + } + + return false; + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java index 6e30c4f0be08f..69f99f35f8baf 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosDBWriteException.java @@ -1,7 +1,13 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import org.apache.kafka.connect.errors.ConnectException; +/** + * Generic CosmosDb sink write exceptions. + */ public class CosmosDBWriteException extends ConnectException { /** * diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java index 808361b97a0e8..9d87a937148c2 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java @@ -51,14 +51,18 @@ public class CosmosSinkConfig extends CosmosConfig { "Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend " + "when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch " + "size is getting automatically tuned based on the throttling rate. By default the " - + "initial micro batch size is 100. Reduce this when you want to avoid that the first few requests consume " + + "initial micro batch size is 1. Reduce this when you want to avoid that the first few requests consume " + "too many RUs."; private static final String BULK_INITIAL_BATCH_SIZE_DISPLAY = "Cosmos DB initial bulk micro batch size."; private static final int DEFAULT_BULK_INITIAL_BATCH_SIZE = 1; // start with small value to avoid initial RU spike // write strategy public static final String WRITE_STRATEGY_CONF = SINK_CONFIG_PREFIX + "write.strategy"; - private static final String WRITE_STRATEGY_DOC = "Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert)"; + private static final String WRITE_STRATEGY_DOC = "Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert), `ItemAppend` (using create, " + + "ignore pre-existing items i.e., Conflicts), `ItemDelete` (deletes based on id/pk of data frame), " + + "`ItemDeleteIfNotModified` (deletes based on id/pk of data frame if etag hasn't changed since collecting " + + "id/pk), `ItemOverwriteIfNotModified` (using create if etag is empty, update/replace with etag pre-condition " + + "otherwise, if document was updated the pre-condition failure is ignored)"; private static final String WRITE_STRATEGY_DISPLAY = "Cosmos DB Item write Strategy."; private static final String DEFAULT_WRITE_STRATEGY = ItemWriteStrategy.ITEM_OVERWRITE.getName(); @@ -80,17 +84,15 @@ public class CosmosSinkConfig extends CosmosConfig { "A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2."; private static final String CONTAINERS_TOPIC_MAP_DISPLAY = "Topic-Container map"; + // TODO[Public preview]: re-examine idStrategy implementation // id.strategy public static final String ID_STRATEGY_CONF = SINK_CONFIG_PREFIX + "id.strategy"; public static final String ID_STRATEGY_DOC = "A strategy used to populate the document with an ``id``. Valid strategies are: " + "``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, " - + "``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. For each strategy, the full " - + "name of the strategy must be specified, e.g. " - + "com.azure.cosmos.kafka.connect.sink.id.strategy.TemplateStrategy. Configuration " + + "``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. Configuration " + "properties prefixed with``id.strategy`` are passed through to the strategy. For " - + "example, when using" - + "``id.strategy=com.azure.cosmos.kafka.connect.sink.id.strategy.TemplateStrategy`` , " + + "example, when using ``id.strategy=TemplateStrategy`` , " + "the property ``id.strategy.template`` is passed through to the template strategy " + "and used to specify the template string to be used in constructing the ``id``."; public static final String ID_STRATEGY_DISPLAY = "ID Strategy"; @@ -100,7 +102,6 @@ public class CosmosSinkConfig extends CosmosConfig { private final CosmosSinkWriteConfig writeConfig; private final CosmosSinkContainersConfig containersConfig; - private final ToleranceOnErrorLevel toleranceOnErrorLevel; private final IdStrategies idStrategy; public CosmosSinkConfig(Map parsedConfig) { @@ -111,7 +112,6 @@ public CosmosSinkConfig(ConfigDef config, Map parsedConfig) { super(config, parsedConfig); this.writeConfig = this.parseWriteConfig(); this.containersConfig = this.parseContainersConfig(); - this.toleranceOnErrorLevel = this.parseToleranceOnErrorLevel(); this.idStrategy = this.parseIdStrategy(); } @@ -120,7 +120,6 @@ public static ConfigDef getConfigDef() { defineWriteConfig(configDef); defineContainersConfig(configDef); - defineToleranceOnErrorConfig(configDef); defineIdStrategyConfig(configDef); return configDef; } @@ -184,6 +183,17 @@ private static void defineWriteConfig(ConfigDef configDef) { writeConfigGroupOrder++, ConfigDef.Width.MEDIUM, MAX_RETRY_COUNT_DISPLAY + ) + .define( + TOLERANCE_ON_ERROR_CONFIG, + ConfigDef.Type.STRING, + DEFAULT_TOLERANCE_ON_ERROR, + ConfigDef.Importance.HIGH, + TOLERANCE_ON_ERROR_DOC, + writeConfigGroupName, + writeConfigGroupOrder++, + ConfigDef.Width.MEDIUM, + TOLERANCE_ON_ERROR_DISPLAY ); } @@ -207,7 +217,7 @@ private static void defineContainersConfig(ConfigDef configDef) { .define( CONTAINERS_TOPIC_MAP_CONF, ConfigDef.Type.STRING, - Strings.Emtpy, + ConfigDef.NO_DEFAULT_VALUE, new ContainersTopicMapValidator(), ConfigDef.Importance.MEDIUM, CONTAINERS_TOPIC_MAP_DOC, @@ -218,23 +228,6 @@ private static void defineContainersConfig(ConfigDef configDef) { ); } - private static void defineToleranceOnErrorConfig(ConfigDef configDef) { - final String toleranceOnErrorConfigGroupName = "Errors tolerance"; - int toleranceOnErrorConfigGroupOrder = 0; - configDef - .define( - TOLERANCE_ON_ERROR_CONFIG, - ConfigDef.Type.STRING, - DEFAULT_TOLERANCE_ON_ERROR, - ConfigDef.Importance.HIGH, - TOLERANCE_ON_ERROR_DOC, - toleranceOnErrorConfigGroupName, - toleranceOnErrorConfigGroupOrder++, - ConfigDef.Width.MEDIUM, - TOLERANCE_ON_ERROR_DISPLAY - ); - } - private static void defineIdStrategyConfig(ConfigDef configDef) { final String idStrategyConfigGroupName = "ID Strategy"; int idStrategyConfigGroupOrder = 0; @@ -258,13 +251,15 @@ private CosmosSinkWriteConfig parseWriteConfig() { int bulkInitialBatchSize = this.getInt(BULK_INITIAL_BATCH_SIZE_CONF); ItemWriteStrategy writeStrategy = this.parseItemWriteStrategy(); int maxRetryCount = this.getInt(MAX_RETRY_COUNT_CONF); + ToleranceOnErrorLevel toleranceOnErrorLevel = this.parseToleranceOnErrorLevel(); return new CosmosSinkWriteConfig( bulkEnabled, bulkMaxConcurrentCosmosPartitions, bulkInitialBatchSize, writeStrategy, - maxRetryCount); + maxRetryCount, + toleranceOnErrorLevel); } private CosmosSinkContainersConfig parseContainersConfig() { @@ -305,10 +300,6 @@ public CosmosSinkContainersConfig getContainersConfig() { return containersConfig; } - public ToleranceOnErrorLevel getToleranceOnErrorLevel() { - return toleranceOnErrorLevel; - } - public IdStrategies getIdStrategy() { return idStrategy; } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java index d14a90e8d9d9c..9cb3273f8ebda 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkContainersConfig.java @@ -1,3 +1,6 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import java.util.Map; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java index b4d1b096414a9..79d881169b3da 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java @@ -12,7 +12,6 @@ import org.apache.kafka.connect.sink.SinkTask; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.slf4j.MDC; import java.util.Collection; import java.util.List; @@ -36,18 +35,17 @@ public void start(Map props) { LOGGER.info("Starting the kafka cosmos sink task..."); this.sinkTaskConfig = new CosmosSinkTaskConfig(props); this.cosmosClient = CosmosClientStore.getCosmosClient(this.sinkTaskConfig.getAccountConfig()); - this.sinkRecordTransformer = new SinkRecordTransformer(this.sinkTaskConfig.getIdStrategy()); + this.sinkRecordTransformer = new SinkRecordTransformer(this.sinkTaskConfig); if (this.sinkTaskConfig.getWriteConfig().isBulkEnabled()) { - this.cosmosWriter = new KafkaCosmosBulkWriter(this.sinkTaskConfig.getWriteConfig()); + this.cosmosWriter = + new KafkaCosmosBulkWriter(this.sinkTaskConfig.getWriteConfig(), this.context.errantRecordReporter()); } else { - this.cosmosWriter = new KafkaCosmosPointWriter( - this.sinkTaskConfig.getWriteConfig(), - this.sinkTaskConfig.getToleranceOnErrorLevel()); + this.cosmosWriter = + new KafkaCosmosPointWriter(this.sinkTaskConfig.getWriteConfig(), context.errantRecordReporter()); } - // TODO[this PR]: this is done in V1, verify what is the expected behavior - this.cosmosClient.createDatabaseIfNotExists(this.sinkTaskConfig.getContainersConfig().getDatabaseName()); + // TODO[public preview]: in V1, it will create the database if does not exists, but why? } @Override @@ -81,57 +79,7 @@ record -> this.sinkTaskConfig // transform sink records, for example populating id List transformedRecords = sinkRecordTransformer.transform(containerName, entry.getValue()); - - try { - SinkWriteResponse response = this.cosmosWriter.write(container, transformedRecords); - if (response.getFailedRecordResponses().size() == 0) { - LOGGER.debug( - "Sink write completed, {} records succeeded.", - response.getSucceededRecords().size()); - } else { - LOGGER.info( - "Sink write completed, {} records succeeded, {} records failed.", - response.getSucceededRecords().size(), - response.getFailedRecordResponses().size()); - this.sendToDlqIfConfigured(response.getFailedRecordResponses()); - } - } finally { - MDC.clear(); - } - } - } - - /** - * Sends data to a dead letter queue - * - * @param failedResponses the kafka record that failed to write - */ - private void sendToDlqIfConfigured(List failedResponses) { - if (context != null && context.errantRecordReporter() != null) { - for (SinkOperationFailedResponse sinkRecordResponse : failedResponses) { - context.errantRecordReporter().report(sinkRecordResponse.getSinkRecord(), sinkRecordResponse.getException()); - } - } - - StringBuilder errorMessage = new StringBuilder(); - for (SinkOperationFailedResponse failedResponse : failedResponses) { - errorMessage - .append( - String.format( - "Unable to write record to CosmosDB: {%s}, value schema {%s}, exception {%s}", - failedResponse.getSinkRecord().key(), - failedResponse.getSinkRecord().valueSchema(), - failedResponse.getException().toString())) - .append("\n"); - } - - if (this.sinkTaskConfig.getToleranceOnErrorLevel() == ToleranceOnErrorLevel.ALL) { - LOGGER.error( - "Total {} records failed, tolerance on error level 'All'. Error message: {}" , - failedResponses.size(), - errorMessage); - } else { - throw new CosmosDBWriteException(errorMessage.toString()); + this.cosmosWriter.write(container, transformedRecords); } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java index 6137d4f957a19..08d8ee1cd0bd7 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java @@ -10,17 +10,21 @@ public class CosmosSinkWriteConfig { private final ItemWriteStrategy itemWriteStrategy; private final int maxRetryCount; + private final ToleranceOnErrorLevel toleranceOnErrorLevel; + public CosmosSinkWriteConfig( boolean bulkEnabled, int bulkMaxConcurrentCosmosPartitions, int bulkInitialBatchSize, ItemWriteStrategy itemWriteStrategy, - int maxRetryCount) { + int maxRetryCount, + ToleranceOnErrorLevel toleranceOnErrorLevel) { this.bulkEnabled = bulkEnabled; this.bulkMaxConcurrentCosmosPartitions = bulkMaxConcurrentCosmosPartitions; this.bulkInitialBatchSize = bulkInitialBatchSize; this.itemWriteStrategy = itemWriteStrategy; this.maxRetryCount = maxRetryCount; + this.toleranceOnErrorLevel = toleranceOnErrorLevel; } public boolean isBulkEnabled() { @@ -42,4 +46,8 @@ public ItemWriteStrategy getItemWriteStrategy() { public int getMaxRetryCount() { return maxRetryCount; } + + public ToleranceOnErrorLevel getToleranceOnErrorLevel() { + return toleranceOnErrorLevel; + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java index 492380578bd9d..18cee5577b7d6 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/IWriter.java @@ -1,3 +1,6 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import com.azure.cosmos.CosmosAsyncContainer; @@ -6,5 +9,5 @@ import java.util.List; public interface IWriter { - SinkWriteResponse write(CosmosAsyncContainer container, List sinkRecords); + void write(CosmosAsyncContainer container, List sinkRecords); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java index 46d659263f334..1723963cb95d8 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java @@ -1,200 +1,253 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import com.azure.cosmos.BridgeInternal; import com.azure.cosmos.CosmosAsyncContainer; import com.azure.cosmos.CosmosException; import com.azure.cosmos.implementation.HttpConstants; +import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; +import com.azure.cosmos.models.CosmosBulkExecutionOptions; import com.azure.cosmos.models.CosmosBulkItemRequestOptions; import com.azure.cosmos.models.CosmosBulkItemResponse; +import com.azure.cosmos.models.CosmosBulkOperationResponse; import com.azure.cosmos.models.CosmosBulkOperations; import com.azure.cosmos.models.CosmosItemOperation; -import org.apache.kafka.connect.sink.SinkRecord; +import com.azure.cosmos.models.PartitionKeyDefinition; +import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; +import reactor.core.publisher.SignalType; +import reactor.core.publisher.Sinks; -import java.util.ArrayList; +import java.time.Duration; import java.util.List; -import java.util.stream.Collectors; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; public class KafkaCosmosBulkWriter extends KafkaCosmosWriterBase { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosBulkWriter.class); + private static final int MAX_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS = 10000; + private static final int MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS = 1000; + private static final Random random = new Random(); private final CosmosSinkWriteConfig writeConfig; + private final Sinks.EmitFailureHandler emitFailureHandler; - public KafkaCosmosBulkWriter(CosmosSinkWriteConfig writeConfig) { + public KafkaCosmosBulkWriter( + CosmosSinkWriteConfig writeConfig, + ErrantRecordReporter errantRecordReporter) { + super(errantRecordReporter); checkNotNull(writeConfig, "Argument 'writeConfig' can not be null"); this.writeConfig = writeConfig; + this.emitFailureHandler = new KafkaCosmosEmitFailureHandler(); } @Override - public SinkWriteResponse write(CosmosAsyncContainer container, List sinkRecords) { - LOGGER.debug("Write {} records to container {}", sinkRecords.size(), container.getId()); - - SinkWriteResponse sinkWriteResponse = new SinkWriteResponse(); - - if (sinkRecords == null || sinkRecords.isEmpty()) { - return sinkWriteResponse; - } + public void writeCore(CosmosAsyncContainer container, List sinkOperations) { + Sinks.Many bulkRetryEmitter = Sinks.many().unicast().onBackpressureBuffer(); + CosmosBulkExecutionOptions bulkExecutionOptions = this.getBulkExecutionOperations(); + AtomicInteger totalPendingRecords = new AtomicInteger(sinkOperations.size()); + Runnable onTaskCompleteCheck = () -> { + if (totalPendingRecords.decrementAndGet() <= 0) { + bulkRetryEmitter.emitComplete(emitFailureHandler); + } + }; + + Flux.fromIterable(sinkOperations) + .flatMap(sinkOperation -> this.getBulkOperation(container, sinkOperation)) + .collectList() + .flatMapMany(itemOperations -> { + + Flux> cosmosBulkOperationResponseFlux = + container.executeBulkOperations( + Flux.fromIterable(itemOperations).mergeWith(bulkRetryEmitter.asFlux()), + bulkExecutionOptions); + return cosmosBulkOperationResponseFlux; + }) + .flatMap(itemResponse -> { + SinkOperation sinkOperation = itemResponse.getOperation().getContext(); + checkNotNull(sinkOperation, "sinkOperation should not be null"); - List sinkOperationContexts = - sinkRecords - .stream() - .map(sinkRecord -> new SinkOperationContext(sinkRecord)) - .collect(Collectors.toList()); - - Mono - .defer(() -> Mono.just(getToBeProcessedSinkOperation(sinkOperationContexts))) - .flatMap(operationsToBeProcessed -> { - if (this.writeConfig.getItemWriteStrategy() == ItemWriteStrategy.ITEM_OVERWRITE) { - return this.getBulkOperations(container, operationsToBeProcessed) - .flatMap(itemOperations -> this.executeBulkOperations(container, itemOperations)); + if (itemResponse.getResponse() != null && itemResponse.getResponse().isSuccessStatusCode()) { + // success + this.completeSinkOperation(sinkOperation, onTaskCompleteCheck); + } else { + BulkOperationFailedException exception = handleErrorStatusCode( + itemResponse.getResponse(), + itemResponse.getException(), + sinkOperation); + + if (shouldIgnore(exception)) { + this.completeSinkOperation(sinkOperation, onTaskCompleteCheck); + } else { + if (shouldRetry(exception, sinkOperation.getRetryCount(), this.writeConfig.getMaxRetryCount())) { + sinkOperation.setException(exception); + return this.scheduleRetry(container, itemResponse.getOperation().getContext(), bulkRetryEmitter, exception); + } else { + // operation failed after exhausting all retries + this.completeSinkOperationWithFailure(sinkOperation, exception, onTaskCompleteCheck); + if (this.writeConfig.getToleranceOnErrorLevel() == ToleranceOnErrorLevel.ALL) { + LOGGER.warn( + "Could not upload record {} to CosmosDB after exhausting all retries, " + + "but ToleranceOnErrorLevel is all, will only log the error message. ", + sinkOperation.getSinkRecord().key(), + sinkOperation.getException()); + return Mono.empty(); + } else { + return Mono.error(exception); + } + } + } } return Mono.empty(); }) - .repeat(() -> getToBeProcessedSinkOperation(sinkOperationContexts).size() > 0)// only repeat when there are records still need to be processed .blockLast(); - - return sinkWriteResponse; } - private List getToBeProcessedSinkOperation(List sinkOperationContexts) { - return sinkOperationContexts - .stream() - .filter( - sinkOperationContext -> { - return !sinkOperationContext.getIsSucceeded() - && shouldRetry(sinkOperationContext.getException(), sinkOperationContext.getRetryCount(), this.writeConfig.getMaxRetryCount()); - }) - .collect(Collectors.toList()); + private CosmosBulkExecutionOptions getBulkExecutionOperations() { + CosmosBulkExecutionOptions bulkExecutionOptions = new CosmosBulkExecutionOptions(); + bulkExecutionOptions.setInitialMicroBatchSize(this.writeConfig.getBulkInitialBatchSize()); + if (this.writeConfig.getBulkMaxConcurrentCosmosPartitions() > 0) { + ImplementationBridgeHelpers + .CosmosBulkExecutionOptionsHelper + .getCosmosBulkExecutionOptionsAccessor() + .setMaxConcurrentCosmosPartitions(bulkExecutionOptions, this.writeConfig.getBulkMaxConcurrentCosmosPartitions()); + } + + return bulkExecutionOptions; } - private Mono> getBulkOperations( + private Mono getBulkOperation( CosmosAsyncContainer container, - List sinkOperationContexts) { + SinkOperation sinkOperation) { return this.getPartitionKeyDefinition(container) .flatMap(partitionKeyDefinition -> { - List cosmosItemOperations = new ArrayList<>(); + CosmosItemOperation cosmosItemOperation; + switch (this.writeConfig.getItemWriteStrategy()) { case ITEM_OVERWRITE: - sinkOperationContexts.forEach(sinkOperationContext -> - cosmosItemOperations.add( - CosmosBulkOperations.getUpsertItemOperation( - sinkOperationContext.getSinkRecord().value(), - this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), - sinkOperationContext - ) - )); + cosmosItemOperation = this.getUpsertItemOperation(sinkOperation, partitionKeyDefinition); break; case ITEM_OVERWRITE_IF_NOT_MODIFIED: - sinkOperationContexts.forEach(sinkOperationContext -> { - String etag = getEtag(sinkOperationContext.getSinkRecord().value()); - if (StringUtils.isEmpty(etag)) { - cosmosItemOperations.add( - CosmosBulkOperations.getCreateItemOperation( - sinkOperationContext.getSinkRecord().value(), - this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), - sinkOperationContext - ) - ); - } else { - cosmosItemOperations.add( - CosmosBulkOperations.getReplaceItemOperation( - getId(sinkOperationContext.getSinkRecord().value()), - sinkOperationContext.getSinkRecord(), - this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), - new CosmosBulkItemRequestOptions().setIfMatchETag(etag), - sinkOperationContext) - ); - } - }); + String etag = getEtag(sinkOperation.getSinkRecord().value()); + if (StringUtils.isEmpty(etag)) { + cosmosItemOperation = this.getCreateItemOperation(sinkOperation, partitionKeyDefinition); + } else { + cosmosItemOperation = this.getReplaceItemOperation(sinkOperation, partitionKeyDefinition, etag); + } break; case ITEM_APPEND: - sinkOperationContexts.forEach(sinkOperationContext -> { - cosmosItemOperations.add( - CosmosBulkOperations.getCreateItemOperation( - sinkOperationContext.getSinkRecord().value(), - this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), - sinkOperationContext - )); - - }); + cosmosItemOperation = this.getCreateItemOperation(sinkOperation, partitionKeyDefinition); break; case ITEM_DELETE: - sinkOperationContexts.forEach(sinkOperationContext -> { - cosmosItemOperations.add( - CosmosBulkOperations.getDeleteItemOperation( - this.getId(sinkOperationContext.getSinkRecord().value()), - this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), - sinkOperationContext - )); - }); + cosmosItemOperation = this.getDeleteItemOperation(sinkOperation, partitionKeyDefinition, null); break; case ITEM_DELETE_IF_NOT_MODIFIED: - sinkOperationContexts.forEach(sinkOperationContext -> { - String etag = getEtag(sinkOperationContext.getSinkRecord().value()); - CosmosBulkItemRequestOptions itemRequestOptions = new CosmosBulkItemRequestOptions(); - if (StringUtils.isNotEmpty(etag)) { - itemRequestOptions.setIfMatchETag(etag); - } - - CosmosBulkOperations.getDeleteItemOperation( - getId(sinkOperationContext.getSinkRecord().value()), - this.getPartitionKeyValue(sinkOperationContext.getSinkRecord().value(), partitionKeyDefinition), - itemRequestOptions, - sinkOperationContext); - }); + String itemDeleteEtag = getEtag(sinkOperation.getSinkRecord().value()); + cosmosItemOperation = this.getDeleteItemOperation(sinkOperation, partitionKeyDefinition, itemDeleteEtag); break; default: return Mono.error(new IllegalArgumentException(this.writeConfig.getItemWriteStrategy() + " is not supported")); } - return Mono.just(cosmosItemOperations); + return Mono.just(cosmosItemOperation); }); } - private Mono executeBulkOperations(CosmosAsyncContainer container, List cosmosItemOperations) { - return container - .executeBulkOperations(Flux.fromIterable(cosmosItemOperations)) - .doOnNext(itemResponse -> { - SinkOperationContext context = itemResponse.getOperation().getContext(); - checkNotNull(context, "sinkOperationContext should not be null"); + private CosmosItemOperation getUpsertItemOperation( + SinkOperation sinkOperation, + PartitionKeyDefinition partitionKeyDefinition) { - if (itemResponse.getException() != null - || itemResponse.getResponse() == null - || !itemResponse.getResponse().isSuccessStatusCode()) { + return CosmosBulkOperations.getUpsertItemOperation( + sinkOperation.getSinkRecord().value(), + this.getPartitionKeyValue(sinkOperation.getSinkRecord().value(), partitionKeyDefinition), + sinkOperation); + } - BulkOperationFailedException exception = handleErrorStatusCode( - itemResponse.getResponse(), - itemResponse.getException(), - context); + private CosmosItemOperation getCreateItemOperation( + SinkOperation sinkOperation, + PartitionKeyDefinition partitionKeyDefinition) { + return CosmosBulkOperations.getCreateItemOperation( + sinkOperation.getSinkRecord().value(), + this.getPartitionKeyValue(sinkOperation.getSinkRecord().value(), partitionKeyDefinition), + sinkOperation); + } - context.setException(exception); - } else { - context.setSucceeded(); - } - }) - .onErrorResume(throwable -> { - cosmosItemOperations.forEach(cosmosItemOperation -> { - ((SinkOperationContext)cosmosItemOperation.getContext()).setException(throwable); + private CosmosItemOperation getReplaceItemOperation( + SinkOperation sinkOperation, + PartitionKeyDefinition partitionKeyDefinition, + String etag) { + + CosmosBulkItemRequestOptions itemRequestOptions = new CosmosBulkItemRequestOptions(); + if (StringUtils.isNotEmpty(etag)) { + itemRequestOptions.setIfMatchETag(etag); + } + + return CosmosBulkOperations.getReplaceItemOperation( + getId(sinkOperation.getSinkRecord().value()), + sinkOperation.getSinkRecord().value(), + this.getPartitionKeyValue(sinkOperation.getSinkRecord().value(), partitionKeyDefinition), + new CosmosBulkItemRequestOptions().setIfMatchETag(etag), + sinkOperation); + } + + private CosmosItemOperation getDeleteItemOperation( + SinkOperation sinkOperation, + PartitionKeyDefinition partitionKeyDefinition, + String etag) { + + CosmosBulkItemRequestOptions itemRequestOptions = new CosmosBulkItemRequestOptions(); + if (StringUtils.isNotEmpty(etag)) { + itemRequestOptions.setIfMatchETag(etag); + } + + return CosmosBulkOperations.getDeleteItemOperation( + this.getId(sinkOperation.getSinkRecord().value()), + this.getPartitionKeyValue(sinkOperation.getSinkRecord().value(), partitionKeyDefinition), + itemRequestOptions, + sinkOperation); + } + + private Mono scheduleRetry( + CosmosAsyncContainer container, + SinkOperation sinkOperation, + Sinks.Many bulkRetryEmitter, + BulkOperationFailedException exception) { + + sinkOperation.retry(); + Mono retryMono = + getBulkOperation(container, sinkOperation) + .flatMap(itemOperation -> { + bulkRetryEmitter.emitNext(itemOperation, emitFailureHandler); + return Mono.empty(); }); - return Mono.empty(); - }) - .then(); + if (KafkaCosmosExceptionsHelper.isTimeoutException(exception)) { + Duration delayDuration = Duration.ofMillis( + MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS + + random.nextInt(MAX_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS - MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS)); + + return retryMono.delaySubscription(delayDuration); + } + + return retryMono; } BulkOperationFailedException handleErrorStatusCode( CosmosBulkItemResponse itemResponse, Exception exception, - SinkOperationContext sinkOperationContext) { + SinkOperation sinkOperationContext) { int effectiveStatusCode = itemResponse != null @@ -218,11 +271,60 @@ BulkOperationFailedException handleErrorStatusCode( return new BulkOperationFailedException(effectiveStatusCode, effectiveSubStatusCode, errorMessage, exception); } + private boolean shouldIgnore(BulkOperationFailedException failedException) { + switch (this.writeConfig.getItemWriteStrategy()) { + case ITEM_APPEND: + return KafkaCosmosExceptionsHelper.isResourceExistsException(failedException); + case ITEM_DELETE: + return KafkaCosmosExceptionsHelper.isNotFoundException(failedException); + case ITEM_DELETE_IF_NOT_MODIFIED: + return KafkaCosmosExceptionsHelper.isNotFoundException(failedException) + || KafkaCosmosExceptionsHelper.isPreconditionFailedException(failedException); + case ITEM_OVERWRITE_IF_NOT_MODIFIED: + return KafkaCosmosExceptionsHelper.isResourceExistsException(failedException) + || KafkaCosmosExceptionsHelper.isNotFoundException(failedException) + || KafkaCosmosExceptionsHelper.isPreconditionFailedException(failedException); + default: + return false; + } + } + + private void completeSinkOperation(SinkOperation sinkOperationContext, Runnable onCompleteRunnable) { + sinkOperationContext.complete(); + onCompleteRunnable.run(); + } + + public void completeSinkOperationWithFailure( + SinkOperation sinkOperationContext, + Exception exception, + Runnable onCompleteRunnable) { + + sinkOperationContext.setException(exception); + sinkOperationContext.complete(); + onCompleteRunnable.run(); + + this.sendToDlqIfConfigured(sinkOperationContext); + } + private static class BulkOperationFailedException extends CosmosException { protected BulkOperationFailedException(int statusCode, int subStatusCode, String message, Throwable cause) { super(statusCode, message, null, cause); BridgeInternal.setSubStatusCode(this, subStatusCode); } } + + private static class KafkaCosmosEmitFailureHandler implements Sinks.EmitFailureHandler { + + @Override + public boolean onEmitFailure(SignalType signalType, Sinks.EmitResult emitResult) { + if (emitResult.equals(Sinks.EmitResult.FAIL_NON_SERIALIZED)) { + LOGGER.debug("emitFailureHandler - Signal: {}, Result: {}", signalType, emitResult.toString()); + return true; + } else { + LOGGER.error("emitFailureHandler - Signal: {}, Result: {}", signalType, emitResult.toString()); + return false; + } + } + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java index 660bac98aabd8..8459fcaa925be 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java @@ -1,11 +1,16 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import com.azure.cosmos.CosmosAsyncContainer; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.implementation.guava25.base.Function; -import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.models.CosmosItemRequestOptions; -import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.ErrantRecordReporter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import reactor.core.publisher.Mono; import reactor.core.scheduler.Schedulers; @@ -14,108 +19,93 @@ import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; public class KafkaCosmosPointWriter extends KafkaCosmosWriterBase { + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosPointWriter.class); + private final CosmosSinkWriteConfig writeConfig; - private final ToleranceOnErrorLevel toleranceOnErrorLevel; - public KafkaCosmosPointWriter(CosmosSinkWriteConfig writeConfig, ToleranceOnErrorLevel toleranceOnErrorLevel) { + public KafkaCosmosPointWriter( + CosmosSinkWriteConfig writeConfig, + ErrantRecordReporter errantRecordReporter) { + super(errantRecordReporter); checkNotNull(writeConfig, "Argument 'writeConfig' can not be null"); this.writeConfig = writeConfig; - this.toleranceOnErrorLevel = toleranceOnErrorLevel; } @Override - public SinkWriteResponse write(CosmosAsyncContainer container, List sinkRecords) { - checkNotNull(sinkRecords, "Argument 'sinkRecords' should not be null"); - SinkWriteResponse sinkWriteResponse = new SinkWriteResponse(); - - for (SinkRecord sinkRecord : sinkRecords) { - SinkOperationContext sinkOperationContext = new SinkOperationContext(sinkRecord); + public void writeCore(CosmosAsyncContainer container, List sinkOperations) { + for (SinkOperation sinkOperation : sinkOperations) { switch (this.writeConfig.getItemWriteStrategy()) { case ITEM_OVERWRITE: - this.upsertWithRetry(container, sinkOperationContext); + this.upsertWithRetry(container, sinkOperation); break; case ITEM_OVERWRITE_IF_NOT_MODIFIED: - String etag = this.getEtag(sinkRecord.value()); + String etag = this.getEtag(sinkOperation.getSinkRecord().value()); if (StringUtils.isNotEmpty(etag)) { - this.replaceIfNotModifiedWithRetry(container, sinkOperationContext, etag); + this.replaceIfNotModifiedWithRetry(container, sinkOperation, etag); } else { - this.createWithRetry(container, sinkOperationContext); + this.createWithRetry(container, sinkOperation); } break; case ITEM_APPEND: - this.createWithRetry(container, sinkOperationContext); + this.createWithRetry(container, sinkOperation); break; case ITEM_DELETE: - this.deleteWithRetry(container, sinkOperationContext, false); + this.deleteWithRetry(container, sinkOperation, false); break; case ITEM_DELETE_IF_NOT_MODIFIED: - this.deleteWithRetry(container, sinkOperationContext, true); + this.deleteWithRetry(container, sinkOperation, true); break; default: throw new IllegalArgumentException(this.writeConfig.getItemWriteStrategy() + " is not supported"); } - - if (sinkOperationContext.getIsSucceeded()) { - sinkWriteResponse.getSucceededRecords().add(sinkRecord); - } else { - sinkWriteResponse.getFailedRecordResponses().add(new SinkOperationFailedResponse(sinkRecord, sinkOperationContext.getException())); - if (this.toleranceOnErrorLevel == ToleranceOnErrorLevel.NONE) { - // if there is no tolerance, then fail fast on the first exception - return sinkWriteResponse; - } - } } - - //We will only reach here is all the operations have succeeded or the ToleranceOnErrorLevel is ALL - return sinkWriteResponse; } - private void upsertWithRetry(CosmosAsyncContainer container, SinkOperationContext context) { + private void upsertWithRetry(CosmosAsyncContainer container, SinkOperation sinkOperation) { executeWithRetry( - (sinkOperationContext) -> container.upsertItem(context.getSinkRecord().value()).then(), + (operation) -> container.upsertItem(operation.getSinkRecord().value()).then(), (throwable) -> false, // no exceptions should be ignored - context + sinkOperation ); } - private void createWithRetry(CosmosAsyncContainer container, SinkOperationContext context) { + private void createWithRetry(CosmosAsyncContainer container, SinkOperation sinkOperation) { executeWithRetry( - (sinkOperationContext) -> container.createItem(context.getSinkRecord().value()).then(), - (throwable) -> CosmosExceptionsHelper.isResourceExistsException(throwable), - context + (operation) -> container.createItem(operation.getSinkRecord().value()).then(), + (throwable) -> KafkaCosmosExceptionsHelper.isResourceExistsException(throwable), + sinkOperation ); } - private void replaceIfNotModifiedWithRetry(CosmosAsyncContainer container, SinkOperationContext context, String etag) { + private void replaceIfNotModifiedWithRetry(CosmosAsyncContainer container, SinkOperation sinkOperation, String etag) { executeWithRetry( - (sinkOperationContext) -> { + (operation) -> { CosmosItemRequestOptions itemRequestOptions = new CosmosItemRequestOptions(); itemRequestOptions.setIfMatchETag(etag); return this.getPartitionKeyDefinition(container) .flatMap(partitionKeyDefinition -> { return container.replaceItem( - context.getSinkRecord().value(), - getId(context.getSinkRecord().value()), - getPartitionKeyValue(context.getSinkRecord().value(), partitionKeyDefinition), - itemRequestOptions).then(); + operation.getSinkRecord().value(), + getId(operation.getSinkRecord().value()), + getPartitionKeyValue(operation.getSinkRecord().value(), partitionKeyDefinition), + itemRequestOptions).then(); }); }, (throwable) -> { - return CosmosExceptionsHelper.isResourceExistsException(throwable) || - CosmosExceptionsHelper.isNotFoundException(throwable) || - CosmosExceptionsHelper.isPreconditionFailedException(throwable); + return KafkaCosmosExceptionsHelper.isNotFoundException(throwable) || + KafkaCosmosExceptionsHelper.isPreconditionFailedException(throwable); }, - context + sinkOperation ); } - private void deleteWithRetry(CosmosAsyncContainer container, SinkOperationContext context, boolean onlyIfModified) { + private void deleteWithRetry(CosmosAsyncContainer container, SinkOperation sinkOperation, boolean onlyIfModified) { executeWithRetry( - (sinkOperationContext) -> { + (operation) -> { CosmosItemRequestOptions itemRequestOptions = new CosmosItemRequestOptions(); if (onlyIfModified) { - String etag = this.getEtag(context.getSinkRecord().value()); + String etag = this.getEtag(operation.getSinkRecord().value()); if (StringUtils.isNotEmpty(etag)) { itemRequestOptions.setIfMatchETag(etag); } @@ -124,41 +114,63 @@ private void deleteWithRetry(CosmosAsyncContainer container, SinkOperationContex return this.getPartitionKeyDefinition(container) .flatMap(partitionKeyDefinition -> { return container.deleteItem( - getId(context.getSinkRecord().value()), - getPartitionKeyValue(context.getSinkRecord().value(), partitionKeyDefinition), + getId(operation.getSinkRecord().value()), + getPartitionKeyValue(operation.getSinkRecord().value(), partitionKeyDefinition), itemRequestOptions ); }).then(); }, - (throwable) -> CosmosExceptionsHelper.isNotFoundException(throwable), - context + (throwable) -> { + return KafkaCosmosExceptionsHelper.isNotFoundException(throwable) + || KafkaCosmosExceptionsHelper.isPreconditionFailedException(throwable); + }, + sinkOperation ); } private void executeWithRetry( - Function> execution, + Function> execution, Function shouldIgnoreFunc, - SinkOperationContext operationContext) { + SinkOperation sinkOperation) { Mono.just(this) - .flatMap(data -> execution.apply(operationContext)) - .doOnNext(response -> operationContext.setSucceeded()) + .flatMap(data -> { + if (sinkOperation.getRetryCount() > 0) { + LOGGER.debug("Retry attempted for sinkRecord {}", sinkOperation.getSinkRecord().key()); + } + return execution.apply(sinkOperation); + }) + .doOnSuccess(response -> sinkOperation.complete()) .onErrorResume(throwable -> { if (shouldIgnoreFunc.apply(throwable)) { - operationContext.setSucceeded(); - operationContext.complete(); + sinkOperation.complete(); return Mono.empty(); } - operationContext.setException(throwable); - if (shouldRetry(throwable, operationContext.getRetryCount(), this.writeConfig.getMaxRetryCount())) { + if (shouldRetry(throwable, sinkOperation.getRetryCount(), this.writeConfig.getMaxRetryCount())) { + sinkOperation.setException(throwable); + sinkOperation.retry(); + return Mono.empty(); + } else { + // request failed after exhausted all retries + this.sendToDlqIfConfigured(sinkOperation); + + sinkOperation.setException(throwable); + sinkOperation.complete(); + + if (this.writeConfig.getToleranceOnErrorLevel() == ToleranceOnErrorLevel.ALL) { + LOGGER.warn( + "Could not upload record {} to CosmosDB after exhausting all retries, but ToleranceOnErrorLevel is all, will only log the error message. ", + sinkOperation.getSinkRecord().key(), + sinkOperation.getException()); + return Mono.empty(); + } else { + return Mono.error(sinkOperation.getException()); + } } - - operationContext.complete(); - return Mono.empty(); }) - .repeat(() -> !operationContext.isCompleted()) + .repeat(() -> !sinkOperation.isCompleted()) .then() .subscribeOn(Schedulers.boundedElastic()) .block();// TODO: use customized schedulers diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java index 109c37276c516..04eb22b8a8639 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java @@ -1,36 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import com.azure.cosmos.CosmosAsyncContainer; import com.azure.cosmos.CosmosBridgeInternal; import com.azure.cosmos.implementation.DocumentCollection; import com.azure.cosmos.implementation.ImplementationBridgeHelpers; +import com.azure.cosmos.implementation.Strings; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.implementation.routing.PartitionKeyInternal; -import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.models.PartitionKey; import com.azure.cosmos.models.PartitionKeyDefinition; +import org.apache.kafka.connect.sink.ErrantRecordReporter; +import org.apache.kafka.connect.sink.SinkRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import reactor.core.publisher.Mono; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -public abstract class KafkaCosmosWriterBase implements IWriter{ +public abstract class KafkaCosmosWriterBase implements IWriter { + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosWriterBase.class); + private static final String ID = "id"; + private static final String ETAG = "_etag"; + private final ErrantRecordReporter errantRecordReporter; + + public KafkaCosmosWriterBase(ErrantRecordReporter errantRecordReporter) { + this.errantRecordReporter = errantRecordReporter; + } + + abstract void writeCore(CosmosAsyncContainer container, List sinkOperations); + + @Override + public void write(CosmosAsyncContainer container, List sinkRecords) { + if (sinkRecords == null || sinkRecords.isEmpty()) { + LOGGER.debug("No records to be written to container {}", container.getId()); + return; + } + LOGGER.debug("Write {} records to container {}", sinkRecords.size(), container.getId()); + + // For each sinkRecord, it has a 1:1 mapping SinkOperation which contains sinkRecord and related context: retryCount, succeeded or failure. + List sinkOperations = + sinkRecords + .stream() + .map(sinkRecord -> new SinkOperation(sinkRecord)) + .collect(Collectors.toList()); + + try { + writeCore(container, sinkOperations); + } catch (Exception e) { + LOGGER.error("Write failed. ", e); + throw new CosmosDBWriteException(e.getMessage()); + } + } + protected String getId(Object recordValue) { checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); - return ((Map) recordValue).get("id").toString(); + return ((Map) recordValue).get(ID).toString(); } protected String getEtag(Object recordValue) { checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); - return ((Map) recordValue).get("_etag").toString(); + return ((Map) recordValue).getOrDefault(ETAG, Strings.Emtpy).toString(); } protected PartitionKey getPartitionKeyValue(Object recordValue, PartitionKeyDefinition partitionKeyDefinition) { checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); - //TODO[Public Preview]: examine the code here for sub-partition + //TODO[Public Preview]: add support for sub-partition String partitionKeyPath = StringUtils.join(partitionKeyDefinition.getPaths(), ""); Map recordMap = (Map) recordValue; Object partitionKeyValue = recordMap.get(partitionKeyPath.substring(1)); @@ -43,11 +87,11 @@ protected PartitionKey getPartitionKeyValue(Object recordValue, PartitionKeyDefi } protected boolean shouldRetry(Throwable exception, int attemptedCount, int maxRetryCount) { - if (attemptedCount > maxRetryCount) { + if (attemptedCount >= maxRetryCount) { return false; } - return CosmosExceptionsHelper.isTransientFailure(exception); + return KafkaCosmosExceptionsHelper.isTransientFailure(exception); } protected Mono getPartitionKeyDefinition(CosmosAsyncContainer container) { @@ -65,4 +109,10 @@ protected Mono getPartitionKeyDefinition(CosmosAsyncCont .map(documentCollection -> documentCollection.getPartitionKey()); }); } + + protected void sendToDlqIfConfigured(SinkOperation sinkOperationContext) { + if (this.errantRecordReporter != null) { + errantRecordReporter.report(sinkOperationContext.getSinkRecord(), sinkOperationContext.getException()); + } + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java similarity index 74% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java index d641eea8a2f29..0319ec2de0338 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationContext.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java @@ -1,3 +1,6 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import org.apache.kafka.connect.sink.SinkRecord; @@ -6,17 +9,15 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -public class SinkOperationContext { +public class SinkOperation { private final SinkRecord sinkRecord; private final AtomicInteger retryCount; - private final AtomicBoolean isSucceeded; private final AtomicReference exception; private final AtomicBoolean completed; - public SinkOperationContext(SinkRecord sinkRecord) { + public SinkOperation(SinkRecord sinkRecord) { this.sinkRecord = sinkRecord; this.retryCount = new AtomicInteger(0); - this.isSucceeded = new AtomicBoolean(false); this.exception = new AtomicReference<>(null); this.completed = new AtomicBoolean(false); } @@ -37,14 +38,12 @@ public String getTopic() { return this.sinkRecord.topic(); } - public boolean getIsSucceeded() { - return isSucceeded.get(); - } - public int getRetryCount() { - return retryCount.get(); + return this.retryCount.get(); } + public void retry() { this.retryCount.incrementAndGet(); } + public Throwable getException() { return this.exception.get(); } @@ -53,15 +52,11 @@ public void setException(Throwable exception) { this.exception.set(exception); } - public void setSucceeded() { - this.isSucceeded.set(true); - } - public boolean isCompleted() { - return completed.get(); + return this.completed.get(); } public void complete() { - completed.set(true); + this.completed.set(true); } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java deleted file mode 100644 index f4b0c181dea95..0000000000000 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperationFailedResponse.java +++ /dev/null @@ -1,22 +0,0 @@ -package com.azure.cosmos.kafka.connect.implementation.sink; - -import com.azure.cosmos.CosmosException; -import org.apache.kafka.connect.sink.SinkRecord; - -public class SinkOperationFailedResponse { - private final SinkRecord sinkRecord; - private final Throwable exception; - - public SinkOperationFailedResponse(SinkRecord sinkRecord, Throwable cosmosException) { - this.sinkRecord = sinkRecord; - this.exception = cosmosException; - } - - public SinkRecord getSinkRecord() { - return this.sinkRecord; - } - - public Throwable getException() { - return this.exception; - } -} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java index 9feb527ff52e7..ce824f12c5c41 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java @@ -1,5 +1,15 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.AbstractIdStrategyConfig; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.FullKeyStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.IdStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.KafkaMetadataStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.ProvidedInKeyStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.ProvidedInValueStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.TemplateStrategy; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; @@ -13,10 +23,10 @@ public class SinkRecordTransformer { private static final Logger LOGGER = LoggerFactory.getLogger(SinkRecordTransformer.class); - private final IdStrategies idStrategies; + private final IdStrategy idStrategy; - public SinkRecordTransformer(IdStrategies idStrategies) { - this.idStrategies = idStrategies; + public SinkRecordTransformer(CosmosSinkTaskConfig sinkTaskConfig) { + this.idStrategy = this.createIdStrategy(sinkTaskConfig); } public List transform(String containerName, List sinkRecords) { @@ -68,7 +78,32 @@ private void maybeInsertId(Object recordValue, SinkRecord sinkRecord) { return; } Map recordMap = (Map) recordValue; - IdStrategy idStrategy = config.idStrategy(); - recordMap.put(AbstractIdStrategyConfig.ID, idStrategy.generateId(sinkRecord)); + recordMap.put(AbstractIdStrategyConfig.ID, this.idStrategy.generateId(sinkRecord)); + } + + private IdStrategy createIdStrategy(CosmosSinkTaskConfig sinkTaskConfig) { + IdStrategy idStrategyClass; + switch (sinkTaskConfig.getIdStrategy()) { + case FULL_KEY_STRATEGY: + idStrategyClass = new FullKeyStrategy(); + break; + case TEMPLATE_STRATEGY: + idStrategyClass = new TemplateStrategy(); + break; + case KAFKA_METADATA_STRATEGY: + idStrategyClass = new KafkaMetadataStrategy(); + break; + case PROVIDED_IN_VALUE_STRATEGY: + idStrategyClass = new ProvidedInValueStrategy(); + break; + case PROVIDED_IN_KEY_STRATEGY: + idStrategyClass = new ProvidedInKeyStrategy(); + break; + default: + throw new IllegalArgumentException(sinkTaskConfig.getIdStrategy() + " is not supported"); + } + + idStrategyClass.configure(sinkTaskConfig.originalsWithPrefix(AbstractIdStrategyConfig.PREFIX)); + return idStrategyClass; } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java deleted file mode 100644 index eab452a2adae2..0000000000000 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkWriteResponse.java +++ /dev/null @@ -1,28 +0,0 @@ -package com.azure.cosmos.kafka.connect.implementation.sink; - -import org.apache.kafka.connect.sink.SinkRecord; - -import java.util.ArrayList; -import java.util.List; - -public class SinkWriteResponse { - private final List succeededRecords; - private List failedRecordResponses; - - public SinkWriteResponse() { - succeededRecords = new ArrayList<>(); - failedRecordResponses = new ArrayList<>(); - } - - public List getSucceededRecords() { - return succeededRecords; - } - - public List getFailedRecordResponses() { - return failedRecordResponses; - } - - public void setFailedRecordResponses(List failedRecordResponses) { - this.failedRecordResponses = failedRecordResponses; - } -} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java index ef4e1af98a3ba..c0ee2031e85d7 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java @@ -1,3 +1,6 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect.implementation.sink; import org.apache.kafka.connect.data.Date; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java new file mode 100644 index 0000000000000..c44c4866c8553 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import java.util.Map; +import java.util.regex.Pattern; + +public abstract class AbstractIdStrategy implements IdStrategy { + private static final String SANITIZED_CHAR = "_"; + private static final Pattern SANITIZE_ID_PATTERN = Pattern.compile("[/\\\\?#]"); + + protected Map configs; + + @Override + public void configure(Map configs) { + this.configs = configs; + } + + /** + * Replaces all characters that cannot be part of the ID with {@value SANITIZED_CHAR}. + *

The following characters are restricted and cannot be used in the Id property: '/', '\\', '?', '#' + */ + public static String sanitizeId(String unsanitized) { + return SANITIZE_ID_PATTERN.matcher(unsanitized).replaceAll(SANITIZED_CHAR); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java new file mode 100644 index 0000000000000..a90773ccad9f1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java @@ -0,0 +1,19 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; + +import java.util.Map; + +public class AbstractIdStrategyConfig extends AbstractConfig { + public static final String ID = "id"; + public static final String ID_STRATEGY = ID + ".strategy"; + public static final String PREFIX = ID_STRATEGY + "."; + + public AbstractIdStrategyConfig(ConfigDef definition, Map originals) { + super(definition, originals); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java new file mode 100644 index 0000000000000..9b2823e9b6dd4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java @@ -0,0 +1,16 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import java.util.HashMap; +import java.util.Map; + +public class FullKeyStrategy extends TemplateStrategy { + @Override + public void configure(Map configs) { + Map conf = new HashMap<>(configs); + conf.put(TemplateStrategyConfig.TEMPLATE_CONFIG, "${key}"); + super.configure(conf); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java new file mode 100644 index 0000000000000..e60f874afaa2f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java @@ -0,0 +1,11 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.common.Configurable; +import org.apache.kafka.connect.sink.SinkRecord; + +public interface IdStrategy extends Configurable { + String generateId(SinkRecord record); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java new file mode 100644 index 0000000000000..2f1f74c064017 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java @@ -0,0 +1,22 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import java.util.HashMap; +import java.util.Map; + +public class KafkaMetadataStrategy extends TemplateStrategy { + private KafkaMetadataStrategyConfig config; + + @Override + public void configure(Map configs) { + config = new KafkaMetadataStrategyConfig(configs); + Map conf = new HashMap<>(configs); + conf.put(TemplateStrategyConfig.TEMPLATE_CONFIG, + "${topic}" + config.delimiter() + + "${partition}" + config.delimiter() + "${offset}"); + + super.configure(conf); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java new file mode 100644 index 0000000000000..4d51afc62497b --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.common.config.ConfigDef; + +import java.util.Map; + +public class KafkaMetadataStrategyConfig extends AbstractIdStrategyConfig { + public static final String DELIMITER_CONFIG = "delimiter"; + public static final String DELIMITER_CONFIG_DEFAULT = "-"; + public static final String DELIMITER_CONFIG_DOC = "The delimiter between metadata components"; + public static final String DELIMITER_CONFIG_DISPLAY = "Kafka Metadata"; + + private String delimiter; + + public KafkaMetadataStrategyConfig(Map props) { + this(getConfig(), props); + } + + public KafkaMetadataStrategyConfig(ConfigDef definition, Map originals) { + super(definition, originals); + + this.delimiter = getString(DELIMITER_CONFIG); + } + + public static ConfigDef getConfig() { + ConfigDef result = new ConfigDef(); + + final String groupName = "Kafka Metadata Parameters"; + int groupOrder = 0; + + result.define( + DELIMITER_CONFIG, + ConfigDef.Type.STRING, + DELIMITER_CONFIG_DEFAULT, + ConfigDef.Importance.MEDIUM, + DELIMITER_CONFIG_DOC, + groupName, + groupOrder++, + ConfigDef.Width.MEDIUM, + DELIMITER_CONFIG_DISPLAY + ); + + return result; + } + + public String delimiter() { + return delimiter; + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java new file mode 100644 index 0000000000000..90033fbfaef43 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.common.config.ConfigDef; + +import java.util.Map; + +public class ProvidedInConfig extends AbstractIdStrategyConfig { + public static final String JSON_PATH_CONFIG = "jsonPath"; + public static final String JSON_PATH_CONFIG_DEFAULT = "$.id"; + public static final String JSON_PATH_CONFIG_DOC = "A JsonPath expression to select the desired component to use as ``id``"; + public static final String JSON_PATH_CONFIG_DISPLAY = "JSON Path"; + private final String jsonPath; + + public ProvidedInConfig(Map props) { + this(getConfig(), props); + } + + public ProvidedInConfig(ConfigDef definition, Map originals) { + super(definition, originals); + + this.jsonPath = getString(JSON_PATH_CONFIG); + } + + + public static ConfigDef getConfig() { + ConfigDef result = new ConfigDef(); + + final String groupName = "JsonPath Parameters"; + int groupOrder = 0; + + result.define( + JSON_PATH_CONFIG, + ConfigDef.Type.STRING, + JSON_PATH_CONFIG_DEFAULT, + ConfigDef.Importance.MEDIUM, + JSON_PATH_CONFIG_DOC, + groupName, + groupOrder++, + ConfigDef.Width.MEDIUM, + JSON_PATH_CONFIG_DISPLAY + ); + + return result; + } + + public String jsonPath() { + return jsonPath; + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java new file mode 100644 index 0000000000000..399267902b6f8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java @@ -0,0 +1,10 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +public class ProvidedInKeyStrategy extends ProvidedInStrategy { + public ProvidedInKeyStrategy() { + super(ProvidedIn.KEY); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java new file mode 100644 index 0000000000000..f081f9f3a1f19 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.connect.data.Values; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; +import com.jayway.jsonpath.JsonPath; + +import java.util.Map; + +class ProvidedInStrategy extends AbstractIdStrategy { + protected enum ProvidedIn { + KEY, + VALUE + } + + private final ProvidedIn where; + + private ProvidedInConfig config; + + ProvidedInStrategy(ProvidedIn where) { + this.where = where; + } + + @Override + public String generateId(SinkRecord record) { + String value = where == ProvidedIn.KEY + ? Values.convertToString(record.keySchema(), record.key()) + : Values.convertToString(record.valueSchema(), record.value()); + try { + Object object = JsonPath.parse(value).read(config.jsonPath()); + return sanitizeId(Values.convertToString(null, object)); + } catch (Exception e) { + throw new ConnectException("Could not evaluate JsonPath " + config.jsonPath(), e); + } + } + + @Override + public void configure(Map configs) { + config = new ProvidedInConfig(configs); + super.configure(configs); + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java new file mode 100644 index 0000000000000..ff35b4662a838 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java @@ -0,0 +1,10 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +public class ProvidedInValueStrategy extends ProvidedInStrategy { + public ProvidedInValueStrategy() { + super(ProvidedIn.VALUE); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java new file mode 100644 index 0000000000000..f81554e4eef10 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java @@ -0,0 +1,71 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import org.apache.kafka.connect.data.Values; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.Map; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +public class TemplateStrategy extends AbstractIdStrategy { + private static final String KEY = "key"; + private static final String TOPIC = "topic"; + private static final String PARTITION = "partition"; + private static final String OFFSET = "offset"; + + private static final String PATTERN_TEMPLATE = "\\$\\{(%s)\\}"; + private static final Pattern PATTERN; + + private TemplateStrategyConfig config; + + private static final Map> METHODS_BY_VARIABLE; + + static { + ImmutableMap.Builder> builder = ImmutableMap.builder(); + builder.put(KEY, (r) -> Values.convertToString(r.keySchema(), r.key())); + builder.put(TOPIC, SinkRecord::topic); + builder.put(PARTITION, (r) -> r.kafkaPartition().toString()); + builder.put(OFFSET, (r) -> Long.toString(r.kafkaOffset())); + METHODS_BY_VARIABLE = builder.build(); + + String pattern = String.format(PATTERN_TEMPLATE, + METHODS_BY_VARIABLE.keySet().stream().collect(Collectors.joining("|"))); + PATTERN = Pattern.compile(pattern); + } + + @Override + public String generateId(SinkRecord record) { + String template = config.template(); + return sanitizeId(resolveAll(template, record)); + } + + @Override + public void configure(Map configs) { + config = new TemplateStrategyConfig(configs); + + super.configure(configs); + } + + private String resolveAll(String template, SinkRecord record) { + int lastIndex = 0; + StringBuilder output = new StringBuilder(); + Matcher matcher = PATTERN.matcher(template); + while (matcher.find()) { + output.append(template, lastIndex, matcher.start()) + .append(METHODS_BY_VARIABLE.get(matcher.group(1)).apply(record)); + + lastIndex = matcher.end(); + } + if (lastIndex < template.length()) { + output.append(template, lastIndex, template.length()); + } + return output.toString(); + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java new file mode 100644 index 0000000000000..68dc2af127767 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.common.config.ConfigDef; + +import java.util.Map; + +public class TemplateStrategyConfig extends AbstractIdStrategyConfig { + public static final String TEMPLATE_CONFIG = "template"; + public static final String TEMPLATE_CONFIG_DEFAULT = ""; + public static final String TEMPLATE_CONFIG_DOC = + "The template string to use for determining the ``id``. The template can contain the " + + "following variables that are bound to their values on the Kafka record:" + + "${topic}, ${partition}, ${offset}, ${key}. For example, the template " + + "``${topic}-${key}`` would use the topic name and the entire key in the ``id``, " + + "separated by '-'"; + public static final String TEMPLATE_CONFIG_DISPLAY = "Template"; + private final String template; + + public TemplateStrategyConfig(Map props) { + this(getConfig(), props); + } + + public TemplateStrategyConfig(ConfigDef definition, Map originals) { + super(definition, originals); + + this.template = getString(TEMPLATE_CONFIG); + } + + + public static ConfigDef getConfig() { + ConfigDef result = new ConfigDef(); + + final String groupName = "Template Parameters"; + int groupOrder = 0; + + result.define( + TEMPLATE_CONFIG, + ConfigDef.Type.STRING, + TEMPLATE_CONFIG_DEFAULT, + ConfigDef.Importance.MEDIUM, + TEMPLATE_CONFIG_DOC, + groupName, + groupOrder++, + ConfigDef.Width.MEDIUM, + TEMPLATE_CONFIG_DISPLAY + ); + + return result; + } + + public String template() { + return template; + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java index 0372af2bebbd4..06bb02aa50f30 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java @@ -15,7 +15,7 @@ import com.azure.cosmos.implementation.guava25.base.Stopwatch; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; -import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.models.CosmosChangeFeedRequestOptions; import com.azure.cosmos.models.FeedRange; import com.azure.cosmos.models.FeedResponse; @@ -110,7 +110,7 @@ public List poll() { this.taskUnitsQueue.add(taskUnit); // TODO[Public Preview]: add checking for max retries checking - throw CosmosExceptionsHelper.convertToConnectException(e, "PollTask failed"); + throw KafkaCosmosExceptionsHelper.convertToConnectException(e, "PollTask failed"); } } @@ -172,7 +172,7 @@ private Pair, Boolean> executeFeedRangeTask(FeedRangeTaskUnit return Pair.of(records, false); }) .onErrorResume(throwable -> { - if (CosmosExceptionsHelper.isFeedRangeGoneException(throwable)) { + if (KafkaCosmosExceptionsHelper.isFeedRangeGoneException(throwable)) { return this.handleFeedRangeGone(feedRangeTaskUnit) .map(shouldRemoveOriginalTaskUnit -> Pair.of(new ArrayList<>(), shouldRemoveOriginalTaskUnit)); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java index b9dc02ae85a37..699b3ad9375d5 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java @@ -11,7 +11,7 @@ import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.feedranges.FeedRangeInternal; import com.azure.cosmos.implementation.routing.Range; -import com.azure.cosmos.kafka.connect.implementation.CosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.models.CosmosContainerProperties; import com.azure.cosmos.models.SqlParameter; import com.azure.cosmos.models.SqlQuerySpec; @@ -140,7 +140,7 @@ public Mono> getAllContainers() { .byPage() .flatMapIterable(response -> response.getResults()) .collectList() - .onErrorMap(throwable -> CosmosExceptionsHelper.convertToConnectException(throwable, "getAllContainers failed.")); + .onErrorMap(throwable -> KafkaCosmosExceptionsHelper.convertToConnectException(throwable, "getAllContainers failed.")); } public List getContainerRidsFromOffset() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java index 5d8cd88ccae6a..cfea2ab8a3213 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java @@ -7,6 +7,7 @@ requires transitive com.azure.cosmos; requires kafka.clients; requires connect.api; + requires json.path; // public API surface area diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java new file mode 100644 index 0000000000000..cc332096b5e16 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java @@ -0,0 +1,154 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect; + +import com.azure.cosmos.implementation.Strings; +import com.azure.cosmos.implementation.TestConfigurations; +import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkTask; +import com.azure.cosmos.kafka.connect.implementation.sink.IdStrategies; +import com.azure.cosmos.kafka.connect.implementation.sink.ItemWriteStrategy; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigValue; +import org.testng.annotations.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +import static com.azure.cosmos.kafka.connect.CosmosDBSinkConnectorTest.SinkConfigs.ALL_VALID_CONFIGS; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; +import static org.testng.Assert.assertEquals; + +public class CosmosDBSinkConnectorTest extends KafkaCosmosTestSuiteBase { + @Test(groups = "unit") + public void taskClass() { + CosmosDBSinkConnector sinkConnector = new CosmosDBSinkConnector(); + assertEquals(sinkConnector.taskClass(), CosmosSinkTask.class); + } + + @Test(groups = "unit") + public void config() { + CosmosDBSinkConnector sinkConnector = new CosmosDBSinkConnector(); + ConfigDef configDef = sinkConnector.config(); + Map configs = configDef.configKeys(); + List> allValidConfigs = ALL_VALID_CONFIGS; + + for (KafkaCosmosConfigEntry sinkConfigEntry : allValidConfigs) { + assertThat(configs.containsKey(sinkConfigEntry.getName())).isTrue(); + + configs.containsKey(sinkConfigEntry.getName()); + if (sinkConfigEntry.isOptional()) { + assertThat(configs.get(sinkConfigEntry.getName()).defaultValue).isEqualTo(sinkConfigEntry.getDefaultValue()); + } else { + assertThat(configs.get(sinkConfigEntry.getName()).defaultValue).isEqualTo(ConfigDef.NO_DEFAULT_VALUE); + } + } + } + + @Test(groups = "unit") + public void requiredConfig() { + Config config = new CosmosDBSinkConnector().validate(Collections.emptyMap()); + Map> errorMessages = config.configValues().stream() + .collect(Collectors.toMap(ConfigValue::name, ConfigValue::errorMessages)); + assertThat(errorMessages.get("kafka.connect.cosmos.accountEndpoint").size()).isGreaterThan(0); + assertThat(errorMessages.get("kafka.connect.cosmos.accountKey").size()).isGreaterThan(0); + assertThat(errorMessages.get("kafka.connect.cosmos.sink.database.name").size()).isGreaterThan(0); + assertThat(errorMessages.get("kafka.connect.cosmos.sink.containers.topicMap").size()).isGreaterThan(0); + } + + @Test(groups = "unit") + public void taskConfigs() { + CosmosDBSinkConnector sinkConnector = new CosmosDBSinkConnector(); + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", singlePartitionContainerName + "#" + singlePartitionContainerName); + sinkConnector.start(sinkConfigMap); + + int maxTask = 2; + List> taskConfigs = sinkConnector.taskConfigs(maxTask); + assertThat(taskConfigs.size()).isEqualTo(maxTask); + + for (Map taskConfig : taskConfigs) { + assertThat(taskConfig.get("kafka.connect.cosmos.accountEndpoint")).isEqualTo(TestConfigurations.HOST); + assertThat(taskConfig.get("kafka.connect.cosmos.accountKey")).isEqualTo(TestConfigurations.MASTER_KEY); + assertThat(taskConfig.get("kafka.connect.cosmos.sink.database.name")).isEqualTo(databaseName); + assertThat(taskConfig.get("kafka.connect.cosmos.sink.containers.topicMap")) + .isEqualTo(singlePartitionContainerName + "#" + singlePartitionContainerName); + } + } + + @Test(groups = "unit") + public void misFormattedConfig() { + CosmosDBSinkConnector sinkConnector = new CosmosDBSinkConnector(); + Map sinkConfigMap = this.getValidSinkConfig(); + + String topicMapConfigName = "kafka.connect.cosmos.sink.containers.topicMap"; + sinkConfigMap.put(topicMapConfigName, UUID.randomUUID().toString()); + + Config validatedConfig = sinkConnector.validate(sinkConfigMap); + ConfigValue configValue = + validatedConfig + .configValues() + .stream() + .filter(config -> config.name().equalsIgnoreCase(topicMapConfigName)) + .findFirst() + .get(); + + assertThat(configValue.errorMessages()).isNotNull(); + assertThat( + configValue + .errorMessages() + .get(0) + .contains( + "The topic-container map should be a comma-delimited list of Kafka topic to Cosmos containers." + + " Each mapping should be a pair of Kafka topic and Cosmos container separated by '#'." + + " For example: topic1#con1,topic2#con2.")) + .isTrue(); + + // TODO[Public Preview]: add other config validations + } + + private Map getValidSinkConfig() { + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", singlePartitionContainerName + "#" + singlePartitionContainerName); + + return sinkConfigMap; + } + + public static class SinkConfigs { + public static final List> ALL_VALID_CONFIGS = Arrays.asList( + new KafkaCosmosConfigEntry("kafka.connect.cosmos.accountEndpoint", null, false), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.accountKey", null, false), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.useGatewayMode", false, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.preferredRegionsList", Strings.Emtpy, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.applicationName", Strings.Emtpy, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.errors.tolerance", "None", true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.bulk.enabled", true, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.bulk.maxConcurrentCosmosPartitions", -1, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.bulk.initialBatchSize", 1, true), + new KafkaCosmosConfigEntry( + "kafka.connect.cosmos.sink.write.strategy", + ItemWriteStrategy.ITEM_OVERWRITE.getName(), + true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.maxRetryCount", 10, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.database.name", null, false), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.sink.containers.topicMap", null, false), + new KafkaCosmosConfigEntry( + "kafka.connect.cosmos.sink.id.strategy", + IdStrategies.PROVIDED_IN_VALUE_STRATEGY.getName(), + true) + ); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java index 9bddf4b229e19..a69da0cc6bf4b 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java @@ -66,9 +66,9 @@ public void config() { CosmosDBSourceConnector sourceConnector = new CosmosDBSourceConnector(); ConfigDef configDef = sourceConnector.config(); Map configs = configDef.configKeys(); - List> allValidConfigs = ALL_VALID_CONFIGS; + List> allValidConfigs = ALL_VALID_CONFIGS; - for (SourceConfigEntry sourceConfigEntry : allValidConfigs) { + for (KafkaCosmosConfigEntry sourceConfigEntry : allValidConfigs) { assertThat(configs.containsKey(sourceConfigEntry.getName())).isTrue(); configs.containsKey(sourceConfigEntry.getName()); @@ -374,15 +374,15 @@ public void getTaskConfigsAfterMerge() throws JsonProcessingException { @Test(groups = "unit") public void missingRequiredConfig() { - List> requiredConfigs = + List> requiredConfigs = ALL_VALID_CONFIGS .stream() - .filter(sourceConfigEntry -> !sourceConfigEntry.isOptional) + .filter(sourceConfigEntry -> !sourceConfigEntry.isOptional()) .collect(Collectors.toList()); assertThat(requiredConfigs.size()).isGreaterThan(1); CosmosDBSourceConnector sourceConnector = new CosmosDBSourceConnector(); - for (SourceConfigEntry configEntry : requiredConfigs) { + for (KafkaCosmosConfigEntry configEntry : requiredConfigs) { Map sourceConfigMap = this.getValidSourceConfig(); sourceConfigMap.remove(configEntry.getName()); @@ -391,7 +391,7 @@ public void missingRequiredConfig() { validatedConfig .configValues() .stream() - .filter(config -> config.name().equalsIgnoreCase(configEntry.name)) + .filter(config -> config.name().equalsIgnoreCase(configEntry.getName())) .findFirst() .get(); @@ -579,57 +579,33 @@ private void validateMetadataTask( assertThat(expectedMetadataTaskUnit).isEqualTo(metadataTaskUnitFromTaskConfig); } - public static class SourceConfigEntry { - private final String name; - private final T defaultValue; - private final boolean isOptional; - - public SourceConfigEntry(String name, T defaultValue, boolean isOptional) { - this.name = name; - this.defaultValue = defaultValue; - this.isOptional = isOptional; - } - - public String getName() { - return name; - } - - public T getDefaultValue() { - return defaultValue; - } - - public boolean isOptional() { - return isOptional; - } - } - public static class SourceConfigs { - public static final List> ALL_VALID_CONFIGS = Arrays.asList( - new SourceConfigEntry("kafka.connect.cosmos.accountEndpoint", null, false), - new SourceConfigEntry("kafka.connect.cosmos.accountKey", null, false), - new SourceConfigEntry("kafka.connect.cosmos.useGatewayMode", false, true), - new SourceConfigEntry("kafka.connect.cosmos.preferredRegionsList", Strings.Emtpy, true), - new SourceConfigEntry("kafka.connect.cosmos.applicationName", Strings.Emtpy, true), - new SourceConfigEntry("kafka.connect.cosmos.source.database.name", null, false), - new SourceConfigEntry("kafka.connect.cosmos.source.containers.includeAll", false, true), - new SourceConfigEntry("kafka.connect.cosmos.source.containers.includedList", Strings.Emtpy, true), - new SourceConfigEntry("kafka.connect.cosmos.source.containers.topicMap", Strings.Emtpy, true), - new SourceConfigEntry( + public static final List> ALL_VALID_CONFIGS = Arrays.asList( + new KafkaCosmosConfigEntry("kafka.connect.cosmos.accountEndpoint", null, false), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.accountKey", null, false), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.useGatewayMode", false, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.preferredRegionsList", Strings.Emtpy, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.applicationName", Strings.Emtpy, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.database.name", null, false), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.containers.includeAll", false, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.containers.includedList", Strings.Emtpy, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.containers.topicMap", Strings.Emtpy, true), + new KafkaCosmosConfigEntry( "kafka.connect.cosmos.source.changeFeed.startFrom", CosmosChangeFeedStartFromModes.BEGINNING.getName(), true), - new SourceConfigEntry( + new KafkaCosmosConfigEntry( "kafka.connect.cosmos.source.changeFeed.mode", CosmosChangeFeedModes.LATEST_VERSION.getName(), true), - new SourceConfigEntry("kafka.connect.cosmos.source.changeFeed.maxItemCountHint", 1000, true), - new SourceConfigEntry("kafka.connect.cosmos.source.metadata.poll.delay.ms", 5 * 60 * 1000, true), - new SourceConfigEntry( + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.changeFeed.maxItemCountHint", 1000, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.metadata.poll.delay.ms", 5 * 60 * 1000, true), + new KafkaCosmosConfigEntry( "kafka.connect.cosmos.source.metadata.storage.topic", "_cosmos.metadata.topic", true), - new SourceConfigEntry("kafka.connect.cosmos.source.messageKey.enabled", true, true), - new SourceConfigEntry("kafka.connect.cosmos.source.messageKey.field", "id", true) + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.messageKey.enabled", true, true), + new KafkaCosmosConfigEntry("kafka.connect.cosmos.source.messageKey.field", "id", true) ); } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java deleted file mode 100644 index cfc959db4ef54..0000000000000 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSourceConnectorITest.java +++ /dev/null @@ -1,113 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. - -package com.azure.cosmos.kafka.connect; - -import com.azure.cosmos.CosmosAsyncClient; -import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; -import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; -import com.fasterxml.jackson.databind.JsonNode; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.rnorth.ducttape.unreliables.Unreliables; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testng.annotations.Test; - -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; - -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; - -public class CosmosDbSourceConnectorITest extends KafkaCosmosIntegrationTestSuiteBase { - private static final Logger logger = LoggerFactory.getLogger(CosmosDbSourceConnectorITest.class); - - // TODO[public preview]: add more integration tests - @Test(groups = { "kafka-integration"}, timeOut = TIMEOUT) - public void readFromSingleContainer() { - Map sourceConnectorConfig = new HashMap<>(); - sourceConnectorConfig.put("connector.class", "com.azure.cosmos.kafka.connect.CosmosDBSourceConnector"); - sourceConnectorConfig.put("kafka.connect.cosmos.accountEndpoint", KafkaCosmosTestConfigurations.HOST); - sourceConnectorConfig.put("kafka.connect.cosmos.accountKey", KafkaCosmosTestConfigurations.MASTER_KEY); - sourceConnectorConfig.put("kafka.connect.cosmos.applicationName", "Test"); - sourceConnectorConfig.put("kafka.connect.cosmos.source.database.name", databaseName); - sourceConnectorConfig.put("kafka.connect.cosmos.source.containers.includeAll", "false"); - sourceConnectorConfig.put("kafka.connect.cosmos.source.containers.includedList", singlePartitionContainerName); - sourceConnectorConfig.put("kafka.connect.cosmos.source.containers.topicMap", "TestTopic#" + singlePartitionContainerName); - - - // TODO[this PR]: create the topic ahead of time - CosmosSourceConfig sourceConfig = new CosmosSourceConfig(sourceConnectorConfig); - CosmosAsyncClient client = CosmosClientStore.getCosmosClient(sourceConfig.getAccountConfig()); - CosmosAsyncContainer container = client.getDatabase(databaseName).getContainer(singlePartitionContainerName); - - String connectorName = "simpleTest-" + UUID.randomUUID(); - try { - // create few items in the container - logger.info("creating items in container {}", singlePartitionContainerName); - List createdItems = new ArrayList<>(); - for (int i = 0; i < 10; i++) { - TestItem testItem = TestItem.createNewItem(); - container.createItem(testItem).block(); - createdItems.add(testItem.getId()); - } - - kafkaCosmosConnectContainer.registerConnector(connectorName, sourceConnectorConfig); - - logger.info("Getting consumer and subscribe to topic {}", singlePartitionContainerName); - KafkaConsumer kafkaConsumer = kafkaCosmosConnectContainer.getConsumer(); - kafkaConsumer.subscribe( - Arrays.asList( - "TestTopic", - sourceConfig.getMetadataConfig().getMetadataTopicName())); - - List> metadataRecords = new ArrayList<>(); - List> itemRecords = new ArrayList<>(); - - Unreliables.retryUntilTrue(30, TimeUnit.SECONDS, () -> {; - kafkaConsumer.poll(Duration.ofMillis(1000)) - .iterator() - .forEachRemaining(consumerRecord -> { - if (consumerRecord.topic().equals("TestTopic")) { - itemRecords.add(consumerRecord); - } else if (consumerRecord.topic().equals(sourceConfig.getMetadataConfig().getMetadataTopicName())) { - metadataRecords.add(consumerRecord); - } - }); - return metadataRecords.size() >= 2 && itemRecords.size() >= createdItems.size(); - }); - - //TODO[public preview]currently the metadata record value is null, populate it with metadata and validate the content here - assertThat(metadataRecords.size()).isEqualTo(2); - assertThat(itemRecords.size()).isEqualTo(createdItems.size()); - - List receivedItems = - itemRecords.stream().map(consumerRecord -> { - JsonNode jsonNode = consumerRecord.value(); - return jsonNode.get("payload").get("id").asText(); - }).collect(Collectors.toList()); - - assertThat(receivedItems.containsAll(createdItems)).isTrue(); - - } finally { - if (client != null) { - logger.info("cleaning container {}", singlePartitionContainerName); - cleanUpContainer(client, databaseName, singlePartitionContainerName); - client.close(); - } - - // IMPORTANT: remove the connector after use - if (kafkaCosmosConnectContainer != null) { - kafkaCosmosConnectContainer.deleteConnector(connectorName); - } - } - } -} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConfigEntry.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConfigEntry.java new file mode 100644 index 0000000000000..ec0ee09b3e65f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConfigEntry.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect; + +public class KafkaCosmosConfigEntry { + private final String name; + private final T defaultValue; + private final boolean isOptional; + + public KafkaCosmosConfigEntry(String name, T defaultValue, boolean isOptional) { + this.name = name; + this.defaultValue = defaultValue; + this.isOptional = isOptional; + } + + public String getName() { + return name; + } + + public T getDefaultValue() { + return defaultValue; + } + + public boolean isOptional() { + return isOptional; + } + +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosReflectionUtils.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosReflectionUtils.java index e15f33693460c..cce96e741deb3 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosReflectionUtils.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosReflectionUtils.java @@ -4,10 +4,12 @@ package com.azure.cosmos.kafka.connect; import com.azure.cosmos.CosmosAsyncClient; +import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkTask; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceOffsetStorageReader; import com.azure.cosmos.kafka.connect.implementation.source.MetadataMonitorThread; import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.kafka.connect.sink.SinkTaskContext; import org.apache.kafka.connect.storage.OffsetStorageReader; public class KafkaCosmosReflectionUtils { @@ -59,4 +61,12 @@ public static CosmosSourceOffsetStorageReader getSourceOffsetStorageReader(Cosmo public static OffsetStorageReader getOffsetStorageReader(CosmosSourceOffsetStorageReader sourceOffsetStorageReader) { return get(sourceOffsetStorageReader,"offsetStorageReader"); } + + public static void setSinkTaskContext(CosmosSinkTask sinkTask, SinkTaskContext sinkTaskContext) { + set(sinkTask, sinkTaskContext, "context"); + } + + public static CosmosAsyncClient getSinkTaskCosmosClient(CosmosSinkTask sinkTask) { + return get(sinkTask,"cosmosClient"); + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskTest.java new file mode 100644 index 0000000000000..89cc8322b0506 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTaskTest.java @@ -0,0 +1,571 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink; + +import com.azure.cosmos.CosmosAsyncClient; +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.implementation.TestConfigurations; +import com.azure.cosmos.implementation.Utils; +import com.azure.cosmos.kafka.connect.KafkaCosmosReflectionUtils; +import com.azure.cosmos.kafka.connect.KafkaCosmosTestSuiteBase; +import com.azure.cosmos.kafka.connect.TestItem; +import com.azure.cosmos.kafka.connect.implementation.source.JsonToStruct; +import com.azure.cosmos.models.CosmosContainerProperties; +import com.azure.cosmos.models.PartitionKey; +import com.azure.cosmos.test.faultinjection.CosmosFaultInjectionHelper; +import com.azure.cosmos.test.faultinjection.FaultInjectionConditionBuilder; +import com.azure.cosmos.test.faultinjection.FaultInjectionResultBuilders; +import com.azure.cosmos.test.faultinjection.FaultInjectionRule; +import com.azure.cosmos.test.faultinjection.FaultInjectionRuleBuilder; +import com.azure.cosmos.test.faultinjection.FaultInjectionServerErrorType; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.mockito.Mockito; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; +import reactor.core.publisher.Mono; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + + +public class CosmosSinkTaskTest extends KafkaCosmosTestSuiteBase { + @DataProvider(name = "sinkTaskParameterProvider") + public Object[][] sinkTaskParameterProvider() { + return new Object[][]{ + // flag to indicate whether bulk enabled or not, sink record value schema + { true, Schema.Type.MAP }, + { false, Schema.Type.MAP }, + { true, Schema.Type.STRUCT }, + { false, Schema.Type.STRUCT } + }; + } + + @DataProvider(name = "bulkEnableParameterProvider") + public Object[][] bulkEnableParameterProvider() { + return new Object[][]{ + // flag to indicate whether bulk enabled or not + { true }, + { false } + }; + } + + @Test(groups = { "kafka" }, dataProvider = "sinkTaskParameterProvider", timeOut = TIMEOUT) + public void sinkWithValidRecords(boolean bulkEnabled, Schema.Type valueSchemaType) { + String topicName = singlePartitionContainerName; + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", topicName + "#" + singlePartitionContainerName); + sinkConfigMap.put("kafka.connect.cosmos.sink.bulk.enabled", String.valueOf(bulkEnabled)); + + CosmosSinkTask sinkTask = new CosmosSinkTask(); + SinkTaskContext sinkTaskContext = Mockito.mock(SinkTaskContext.class); + Mockito.when(sinkTaskContext.errantRecordReporter()).thenReturn(null); + KafkaCosmosReflectionUtils.setSinkTaskContext(sinkTask, sinkTaskContext); + sinkTask.start(sinkConfigMap); + + CosmosAsyncClient cosmosClient = KafkaCosmosReflectionUtils.getSinkTaskCosmosClient(sinkTask); + CosmosContainerProperties singlePartitionContainerProperties = getSinglePartitionContainer(cosmosClient); + CosmosAsyncContainer container = cosmosClient.getDatabase(databaseName).getContainer(singlePartitionContainerProperties.getId()); + + try { + List sinkRecordList = new ArrayList<>(); + List toBeCreateItems = new ArrayList<>(); + this.createSinkRecords( + 10, + topicName, + valueSchemaType, + toBeCreateItems, + sinkRecordList); + + sinkTask.put(sinkRecordList); + + // get all the items + List writtenItemIds = new ArrayList<>(); + String query = "select * from c"; + container.queryItems(query, TestItem.class) + .byPage() + .flatMap(response -> { + writtenItemIds.addAll( + response.getResults().stream().map(TestItem::getId).collect(Collectors.toList())); + return Mono.empty(); + }) + .blockLast(); + + assertThat(writtenItemIds.size()).isEqualTo(toBeCreateItems.size()); + List toBeCreateItemIds = toBeCreateItems.stream().map(TestItem::getId).collect(Collectors.toList()); + assertThat(writtenItemIds.containsAll(toBeCreateItemIds)).isTrue(); + + } finally { + if (cosmosClient != null) { + cleanUpContainer(cosmosClient, databaseName, singlePartitionContainerProperties.getId()); + sinkTask.stop(); + } + } + } + + @Test(groups = { "kafka" }, dataProvider = "bulkEnableParameterProvider", timeOut = 10 * TIMEOUT) + public void retryOnServiceUnavailable(boolean bulkEnabled) { + String topicName = singlePartitionContainerName; + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", topicName + "#" + singlePartitionContainerName); + sinkConfigMap.put("kafka.connect.cosmos.sink.bulk.enabled", String.valueOf(bulkEnabled)); + + CosmosSinkTask sinkTask = new CosmosSinkTask(); + SinkTaskContext sinkTaskContext = Mockito.mock(SinkTaskContext.class); + Mockito.when(sinkTaskContext.errantRecordReporter()).thenReturn(null); + KafkaCosmosReflectionUtils.setSinkTaskContext(sinkTask, sinkTaskContext); + sinkTask.start(sinkConfigMap); + + CosmosAsyncClient cosmosClient = KafkaCosmosReflectionUtils.getSinkTaskCosmosClient(sinkTask); + CosmosContainerProperties singlePartitionContainerProperties = getSinglePartitionContainer(cosmosClient); + CosmosAsyncContainer container = cosmosClient.getDatabase(databaseName).getContainer(singlePartitionContainerProperties.getId()); + + // configure fault injection rule + FaultInjectionRule goneExceptionRule = + new FaultInjectionRuleBuilder("goneExceptionRule-" + UUID.randomUUID()) + .condition(new FaultInjectionConditionBuilder().build()) + .result( + FaultInjectionResultBuilders + .getResultBuilder(FaultInjectionServerErrorType.GONE) + .build()) + // high enough so the batch requests will fail with 503 in the first time but low enough so the second retry from kafka connector can succeed + .hitLimit(10) + .build(); + + try { + List sinkRecordList = new ArrayList<>(); + List toBeCreateItems = new ArrayList<>(); + this.createSinkRecords( + 10, + topicName, + Schema.Type.STRUCT, + toBeCreateItems, + sinkRecordList); + + CosmosFaultInjectionHelper.configureFaultInjectionRules(container, Arrays.asList(goneExceptionRule)).block(); + sinkTask.put(sinkRecordList); + + // get all the items + List writtenItemIds = new ArrayList<>(); + String query = "select * from c"; + container.queryItems(query, TestItem.class) + .byPage() + .flatMap(response -> { + writtenItemIds.addAll( + response.getResults().stream().map(TestItem::getId).collect(Collectors.toList())); + return Mono.empty(); + }) + .blockLast(); + + assertThat(writtenItemIds.size()).isEqualTo(toBeCreateItems.size()); + List toBeCreateItemIds = toBeCreateItems.stream().map(TestItem::getId).collect(Collectors.toList()); + assertThat(toBeCreateItemIds.containsAll(writtenItemIds)).isTrue(); + + } finally { + goneExceptionRule.disable(); + if (cosmosClient != null) { + cleanUpContainer(cosmosClient, databaseName, singlePartitionContainerProperties.getId()); + cosmosClient.close(); + } + } + } + + @Test(groups = { "kafka" }, dataProvider = "bulkEnableParameterProvider", timeOut = TIMEOUT) + public void sinkWithItemAppend(boolean bulkEnabled) { + String topicName = singlePartitionContainerName; + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", topicName + "#" + singlePartitionContainerName); + sinkConfigMap.put("kafka.connect.cosmos.sink.bulk.enabled", String.valueOf(bulkEnabled)); + sinkConfigMap.put("kafka.connect.cosmos.sink.write.strategy", ItemWriteStrategy.ITEM_APPEND.getName()); + + CosmosSinkTask sinkTask = new CosmosSinkTask(); + SinkTaskContext sinkTaskContext = Mockito.mock(SinkTaskContext.class); + Mockito.when(sinkTaskContext.errantRecordReporter()).thenReturn(null); + KafkaCosmosReflectionUtils.setSinkTaskContext(sinkTask, sinkTaskContext); + sinkTask.start(sinkConfigMap); + + CosmosAsyncClient cosmosClient = KafkaCosmosReflectionUtils.getSinkTaskCosmosClient(sinkTask); + CosmosContainerProperties singlePartitionContainerProperties = getSinglePartitionContainer(cosmosClient); + CosmosAsyncContainer container = cosmosClient.getDatabase(databaseName).getContainer(singlePartitionContainerProperties.getId()); + + try { + List sinkRecordList = new ArrayList<>(); + List toBeCreateItems = new ArrayList<>(); + this.createSinkRecords( + 10, + topicName, + Schema.Type.MAP, + toBeCreateItems, + sinkRecordList); + + sinkTask.put(sinkRecordList); + + // get all the items + List writtenItemIds = this.getAllItemIds(container); + + assertThat(toBeCreateItems.size()).isEqualTo(writtenItemIds.size()); + List toBeCreateItemIds = toBeCreateItems.stream().map(TestItem::getId).collect(Collectors.toList()); + assertThat(toBeCreateItemIds.containsAll(writtenItemIds)).isTrue(); + + // add the same batch sink records, 409 should be ignored + sinkTask.put(sinkRecordList); + } finally { + if (cosmosClient != null) { + cleanUpContainer(cosmosClient, databaseName, singlePartitionContainerProperties.getId()); + sinkTask.stop(); + } + } + } + + @Test(groups = { "kafka" }, dataProvider = "bulkEnableParameterProvider", timeOut = 3 * TIMEOUT) + public void sinkWithItemOverwriteIfNotModified(boolean bulkEnabled) { + String topicName = singlePartitionContainerName; + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", topicName + "#" + singlePartitionContainerName); + sinkConfigMap.put("kafka.connect.cosmos.sink.bulk.enabled", String.valueOf(bulkEnabled)); + sinkConfigMap.put("kafka.connect.cosmos.sink.write.strategy", ItemWriteStrategy.ITEM_OVERWRITE_IF_NOT_MODIFIED.getName()); + + CosmosSinkTask sinkTask = new CosmosSinkTask(); + SinkTaskContext sinkTaskContext = Mockito.mock(SinkTaskContext.class); + Mockito.when(sinkTaskContext.errantRecordReporter()).thenReturn(null); + KafkaCosmosReflectionUtils.setSinkTaskContext(sinkTask, sinkTaskContext); + sinkTask.start(sinkConfigMap); + + CosmosAsyncClient cosmosClient = KafkaCosmosReflectionUtils.getSinkTaskCosmosClient(sinkTask); + CosmosContainerProperties singlePartitionContainerProperties = getSinglePartitionContainer(cosmosClient); + CosmosAsyncContainer container = cosmosClient.getDatabase(databaseName).getContainer(singlePartitionContainerProperties.getId()); + + try { + List sinkRecordList = new ArrayList<>(); + List toBeCreateItems = new ArrayList<>(); + this.createSinkRecords( + 10, + topicName, + Schema.Type.MAP, + toBeCreateItems, + sinkRecordList); + + sinkTask.put(sinkRecordList); + + // get all the items + List writtenItemIds = this.getAllItemIds(container); + + assertThat(toBeCreateItems.size()).isEqualTo(writtenItemIds.size()); + List toBeCreateItemIds = toBeCreateItems.stream().map(TestItem::getId).collect(Collectors.toList()); + assertThat(toBeCreateItemIds.containsAll(writtenItemIds)).isTrue(); + + ObjectNode existedItem = + container + .readItem(toBeCreateItems.get(0).getId(), new PartitionKey(toBeCreateItems.get(0).getMypk()), ObjectNode.class) + .block() + .getItem(); + + // test precondition-failed exception will be ignored + logger.info( + "Testing precondition-failed exception will be ignored for ItemWriteStrategy " + + ItemWriteStrategy.ITEM_OVERWRITE_IF_NOT_MODIFIED.getName()); + + ObjectNode itemWithWrongEtag = Utils.getSimpleObjectMapper().createObjectNode(); + itemWithWrongEtag.setAll(existedItem); + itemWithWrongEtag.put("_etag", UUID.randomUUID().toString()); + SinkRecord sinkRecordWithWrongEtag = + this.getSinkRecord( + topicName, + itemWithWrongEtag, + new ConnectSchema(Schema.Type.STRING), + itemWithWrongEtag.get("id").asText(), + Schema.Type.MAP); + + sinkTask.put(Arrays.asList(sinkRecordWithWrongEtag)); + + // test with correct etag, the item can be modified + logger.info( + "Testing item can be modified with correct etag for ItemWriteStrategy " + + ItemWriteStrategy.ITEM_OVERWRITE_IF_NOT_MODIFIED.getName()); + ObjectNode modifiedItem = Utils.getSimpleObjectMapper().createObjectNode(); + modifiedItem.setAll(existedItem); + modifiedItem.put("prop", UUID.randomUUID().toString()); + SinkRecord sinkRecordWithModifiedItem = + this.getSinkRecord( + topicName, + modifiedItem, + new ConnectSchema(Schema.Type.STRING), + modifiedItem.get("id").asText(), + Schema.Type.MAP); + sinkTask.put(Arrays.asList(sinkRecordWithModifiedItem)); + + existedItem = + container + .readItem(toBeCreateItems.get(0).getId(), new PartitionKey(toBeCreateItems.get(0).getMypk()), ObjectNode.class) + .block() + .getItem(); + assertThat(existedItem.get("prop").asText()).isEqualTo(modifiedItem.get("prop").asText()); + + } finally { + if (cosmosClient != null) { + cleanUpContainer(cosmosClient, databaseName, singlePartitionContainerProperties.getId()); + sinkTask.stop(); + } + } + } + + @Test(groups = { "kafka" }, dataProvider = "bulkEnableParameterProvider", timeOut = 3 * TIMEOUT) + public void sinkWithItemDelete(boolean bulkEnabled) { + String topicName = singlePartitionContainerName; + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", topicName + "#" + singlePartitionContainerName); + sinkConfigMap.put("kafka.connect.cosmos.sink.bulk.enabled", String.valueOf(bulkEnabled)); + sinkConfigMap.put("kafka.connect.cosmos.sink.write.strategy", ItemWriteStrategy.ITEM_DELETE.getName()); + + CosmosSinkTask sinkTask = new CosmosSinkTask(); + SinkTaskContext sinkTaskContext = Mockito.mock(SinkTaskContext.class); + Mockito.when(sinkTaskContext.errantRecordReporter()).thenReturn(null); + KafkaCosmosReflectionUtils.setSinkTaskContext(sinkTask, sinkTaskContext); + sinkTask.start(sinkConfigMap); + + CosmosAsyncClient cosmosClient = KafkaCosmosReflectionUtils.getSinkTaskCosmosClient(sinkTask); + CosmosContainerProperties singlePartitionContainerProperties = getSinglePartitionContainer(cosmosClient); + CosmosAsyncContainer container = cosmosClient.getDatabase(databaseName).getContainer(singlePartitionContainerProperties.getId()); + + try { + List sinkRecordList = new ArrayList<>(); + List toBeCreateItems = new ArrayList<>(); + this.createSinkRecords( + 10, + topicName, + Schema.Type.MAP, + toBeCreateItems, + sinkRecordList); + + // first time delete, ignore 404 exceptions + sinkTask.put(sinkRecordList); + + // creating the items in the container + for (TestItem testItem : toBeCreateItems) { + container.createItem(testItem).block(); + } + + // get all the items + List createdItemIds = this.getAllItemIds(container); + + assertThat(toBeCreateItems.size()).isEqualTo(createdItemIds.size()); + List toBeCreateItemIds = toBeCreateItems.stream().map(TestItem::getId).collect(Collectors.toList()); + assertThat(toBeCreateItemIds.containsAll(createdItemIds)).isTrue(); + + // now using the connector to delete the items + sinkTask.put(sinkRecordList); + + // verify all the items have deleted + List existingItemIds = this.getAllItemIds(container); + + assertThat(existingItemIds.isEmpty()).isTrue(); + + } finally { + if (cosmosClient != null) { + cleanUpContainer(cosmosClient, databaseName, singlePartitionContainerProperties.getId()); + sinkTask.stop(); + } + } + } + + @Test(groups = { "kafka" }, dataProvider = "bulkEnableParameterProvider", timeOut = 3 * TIMEOUT) + public void sinkWithItemDeleteIfNotModified(boolean bulkEnabled) throws InterruptedException { + String topicName = singlePartitionContainerName; + + Map sinkConfigMap = new HashMap<>(); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", topicName + "#" + singlePartitionContainerName); + sinkConfigMap.put("kafka.connect.cosmos.sink.bulk.enabled", String.valueOf(bulkEnabled)); + sinkConfigMap.put("kafka.connect.cosmos.sink.write.strategy", ItemWriteStrategy.ITEM_DELETE_IF_NOT_MODIFIED.getName()); + + CosmosSinkTask sinkTask = new CosmosSinkTask(); + SinkTaskContext sinkTaskContext = Mockito.mock(SinkTaskContext.class); + Mockito.when(sinkTaskContext.errantRecordReporter()).thenReturn(null); + KafkaCosmosReflectionUtils.setSinkTaskContext(sinkTask, sinkTaskContext); + sinkTask.start(sinkConfigMap); + + CosmosAsyncClient cosmosClient = KafkaCosmosReflectionUtils.getSinkTaskCosmosClient(sinkTask); + CosmosContainerProperties singlePartitionContainerProperties = getSinglePartitionContainer(cosmosClient); + CosmosAsyncContainer container = cosmosClient.getDatabase(databaseName).getContainer(singlePartitionContainerProperties.getId()); + + try { + List sinkRecordList = new ArrayList<>(); + List toBeCreateItems = new ArrayList<>(); + this.createSinkRecords( + 10, + topicName, + Schema.Type.MAP, + toBeCreateItems, + sinkRecordList); + + // first time delete, ignore 404 exceptions + sinkTask.put(sinkRecordList); + + // creating the items in the container + for (TestItem testItem : toBeCreateItems) { + container.createItem(testItem).block(); + } + + // get all the items + List createdItems = this.getAllItems(container); + List createdItemIds = + createdItems + .stream() + .map(objectNode -> objectNode.get("id").asText()) + .collect(Collectors.toList()); + List expectItemIds = toBeCreateItems.stream().map(TestItem::getId).collect(Collectors.toList()); + assertThat(toBeCreateItems.size()).isEqualTo(createdItemIds.size()); + assertThat(expectItemIds.containsAll(createdItemIds)).isTrue(); + + // using wrong etag to delete the items, verify no item will be deleted + List sinkRecordsWithWrongEtag = new ArrayList<>(); + for (ObjectNode createdItem : createdItems) { + ObjectNode testItemWithWrongEtag = Utils.getSimpleObjectMapper().createObjectNode(); + testItemWithWrongEtag.setAll(createdItem); + testItemWithWrongEtag.put("_etag", UUID.randomUUID().toString()); + sinkRecordsWithWrongEtag.add( + this.getSinkRecord( + topicName, + testItemWithWrongEtag, + new ConnectSchema(Schema.Type.STRING), + createdItem.get("id").asText(), + Schema.Type.STRUCT) + ); + } + sinkTask.put(sinkRecordsWithWrongEtag); + Thread.sleep(500); // delete happens in the background + List existingItemIds = this.getAllItemIds(container); + assertThat(existingItemIds.size()).isEqualTo(createdItemIds.size()); + assertThat(existingItemIds.containsAll(createdItemIds)).isTrue(); + + // verify all the items have deleted + List sinkRecordsWithCorrectEtag = new ArrayList<>(); + for (ObjectNode createdItem : createdItems) { + sinkRecordsWithCorrectEtag.add( + this.getSinkRecord( + topicName, + createdItem, + new ConnectSchema(Schema.Type.STRING), + createdItem.get("id").asText(), + Schema.Type.STRUCT) + ); + } + + sinkTask.put(sinkRecordsWithCorrectEtag); + Thread.sleep(500); // delete happens in the background + existingItemIds = this.getAllItemIds(container); + assertThat(existingItemIds.isEmpty()).isTrue(); + + } finally { + if (cosmosClient != null) { + cleanUpContainer(cosmosClient, databaseName, singlePartitionContainerProperties.getId()); + sinkTask.stop(); + } + } + } + + private SinkRecord getSinkRecord( + String topicName, + ObjectNode objectNode, + Schema keySchema, + String keyValue, + Schema.Type valueSchemaType) { + if (valueSchemaType == Schema.Type.STRUCT) { + SchemaAndValue schemaAndValue = + JsonToStruct.recordToSchemaAndValue(objectNode); + + return new SinkRecord( + topicName, + 1, + keySchema, + keyValue, + schemaAndValue.schema(), + schemaAndValue.value(), + 0L); + } else { + return new SinkRecord( + topicName, + 1, + keySchema, + keyValue, + new ConnectSchema(Schema.Type.MAP), + Utils.getSimpleObjectMapper().convertValue(objectNode, new TypeReference>() {}), + 0L); + } + } + + private void createSinkRecords( + int numberOfItems, + String topicName, + Schema.Type valueSchemaType, + List createdItems, + List sinkRecordList) { + + Schema keySchema = new ConnectSchema(Schema.Type.STRING); + + for (int i = 0; i < numberOfItems; i++) { + TestItem testItem = TestItem.createNewItem(); + createdItems.add(testItem); + + SinkRecord sinkRecord = + this.getSinkRecord( + topicName, + Utils.getSimpleObjectMapper().convertValue(testItem, ObjectNode.class), + keySchema, + testItem.getId(), + valueSchemaType); + sinkRecordList.add(sinkRecord); + } + } + + private List getAllItemIds(CosmosAsyncContainer container) { + return getAllItems(container) + .stream() + .map(objectNode -> objectNode.get("id").asText()) + .collect(Collectors.toList()); + } + + private List getAllItems(CosmosAsyncContainer container) { + String query = "select * from c"; + return container.queryItems(query, ObjectNode.class) + .byPage() + .flatMapIterable(response -> response.getResults()) + .collectList() + .block(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java new file mode 100644 index 0000000000000..82303b2067002 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java @@ -0,0 +1,223 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.mockito.Mockito; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; +import static org.mockito.Mockito.when; + +public class ProvidedInStrategyTest { + protected static final int TIMEOUT = 60000; + + @DataProvider(name = "idStrategyParameterProvider") + public static Object[][] idStrategyParameterProvider() { + return new Object[][]{ + { new ProvidedInValueStrategy() }, + { new ProvidedInKeyStrategy() }, + }; + } + + private void returnOnKeyOrValue( + Schema schema, + Object ret, + IdStrategy idStrategy, + SinkRecord sinkRecord) { + if (idStrategy.getClass() == ProvidedInKeyStrategy.class) { + when(sinkRecord.keySchema()).thenReturn(schema); + when(sinkRecord.key()).thenReturn(ret); + } else { + when(sinkRecord.valueSchema()).thenReturn(schema); + when(sinkRecord.value()).thenReturn(ret); + } + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) + public void valueNotStructOrMapShouldFail(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue(Schema.STRING_SCHEMA, "a string", idStrategy, sinkRecord); + idStrategy.generateId(sinkRecord); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) + public void noIdInValueShouldFail(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue(null, Map.of(), idStrategy, sinkRecord); + idStrategy.generateId(sinkRecord); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void stringIdOnMapShouldReturn(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", "1234567"), + idStrategy, + sinkRecord); + assertThat("1234567").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void nonStringIdOnMapShouldReturn(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", 1234567), + idStrategy, + sinkRecord); + assertThat("1234567").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void stringIdOnStructShouldReturn(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + Schema schema = SchemaBuilder.struct() + .field("id", Schema.STRING_SCHEMA) + .build(); + + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + Struct struct = new Struct(schema).put("id", "1234567"); + returnOnKeyOrValue(struct.schema(), struct, idStrategy, sinkRecord); + + assertThat("1234567").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void structIdOnStructShouldReturn(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + Schema idSchema = SchemaBuilder.struct() + .field("name", Schema.STRING_SCHEMA) + .build(); + Schema schema = SchemaBuilder.struct() + .field("id", idSchema) + .build(); + Struct struct = new Struct(schema) + .put("id", new Struct(idSchema).put("name", "cosmos kramer")); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue(struct.schema(), struct, idStrategy, sinkRecord); + + assertThat("{\"name\":\"cosmos kramer\"}").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void jsonPathOnStruct(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.name")); + + Schema idSchema = SchemaBuilder.struct() + .field("name", Schema.STRING_SCHEMA) + .build(); + Schema schema = SchemaBuilder.struct() + .field("id", idSchema) + .build(); + Struct struct = new Struct(schema) + .put("id", new Struct(idSchema).put("name", "franz kafka")); + + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue(struct.schema(), struct, idStrategy, sinkRecord); + assertThat("franz kafka").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void jsonPathOnMap(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.name")); + + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", Map.of("name", "franz kafka")), + idStrategy, + sinkRecord); + assertThat("franz kafka").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) + public void invalidJsonPathThrows(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "invalid.path")); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", Map.of("name", "franz kafka")), + idStrategy, + sinkRecord); + + idStrategy.generateId(sinkRecord); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) + public void jsonPathNotExistThrows(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.not.exist")); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", Map.of("name", "franz kafka")), + idStrategy, + sinkRecord); + + idStrategy.generateId(sinkRecord); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void complexJsonPath(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + Map map1 = new LinkedHashMap<>(); + map1.put("id", 0); + map1.put("name", "cosmos kramer"); + map1.put("occupation", "unknown"); + Map map2 = new LinkedHashMap<>(); + map2.put("id", 1); + map2.put("name", "franz kafka"); + map2.put("occupation", "writer"); + + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", List.of(map1, map2)), + idStrategy, + sinkRecord); + + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[0].name")); + assertThat("cosmos kramer").isEqualTo(idStrategy.generateId(sinkRecord)); + + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[1].name")); + assertThat("franz kafka").isEqualTo(idStrategy.generateId(sinkRecord)); + + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[*].id")); + assertThat("[0,1]").isEqualTo(idStrategy.generateId(sinkRecord)); + + idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id")); + assertThat("[{\"id\":0,\"name\":\"cosmos kramer\",\"occupation\":\"unknown\"},{\"id\":1,\"name\":\"franz kafka\",\"occupation\":\"writer\"}]").isEqualTo(idStrategy.generateId(sinkRecord)); + } + + @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) + public void generatedIdSanitized(IdStrategy idStrategy) { + idStrategy.configure(Map.of()); + SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); + returnOnKeyOrValue( + null, + Map.of("id", "#my/special\\id?"), + idStrategy, + sinkRecord); + + assertThat("_my_special_id_").isEqualTo(idStrategy.generateId(sinkRecord)); + } +} From 1c7d9b54ee5100e2d447b707842c4573f47a55a5 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 27 Feb 2024 12:52:28 -0800 Subject: [PATCH 03/44] changes --- eng/versioning/external_dependencies.txt | 4 +--- sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md | 1 + sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 3 +-- .../com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java | 3 +++ .../azure/cosmos/kafka/connect/CosmosDBSourceConnector.java | 2 +- .../connect/implementation/sink/CosmosSinkWriteConfig.java | 1 + .../connect/implementation/sink/KafkaCosmosBulkWriter.java | 2 +- .../connect/implementation/sink/KafkaCosmosPointWriter.java | 4 ++-- .../connect/implementation/sink/SinkRecordTransformer.java | 1 - 9 files changed, 11 insertions(+), 10 deletions(-) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 3d1245d9f391a..462caf0226412 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -391,11 +391,9 @@ cosmos_org.scalastyle:scalastyle-maven-plugin;1.0.0 ## Cosmos Kafka connector under sdk\cosmos\azure-cosmos-kafka-connect\pom.xml # Cosmos Kafka connector runtime dependencies cosmos_org.apache.kafka:connect-api;3.6.0 +cosmos_org.com.jayway.jsonpath:json-path;2.9.0 # Cosmos Kafka connector tests only cosmos_org.apache.kafka:connect-runtime;3.6.0 -cosmos_org.testcontainers:testcontainers;1.19.5 -cosmos_org.testcontainers:kafka;1.19.5 -cosmos_org.sourcelab:kafka-connect-client;4.0.4 # Maven Tools for Cosmos Kafka connector only cosmos_io.confluent:kafka-connect-maven-plugin;0.12.0 diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md b/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md index 4797133def4bc..840ffe29ca47f 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md @@ -4,6 +4,7 @@ #### Features Added * Added Source connector. See [PR 38748](https://github.com/Azure/azure-sdk-for-java/pull/38748) +* Added Sink connector. See [PR 38973](https://github.com/Azure/azure-sdk-for-java/pull/38973) #### Breaking Changes diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index 8a211b230c1f8..c7d01db047c33 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -106,7 +106,7 @@ Licensed under the MIT License. com.jayway.jsonpath - json-path + json-path 2.9.0 @@ -234,7 +234,6 @@ Licensed under the MIT License. com.azure:* org.apache.kafka:connect-api:[3.6.0] io.confluent:kafka-connect-maven-plugin:[0.12.0] - com.fasterxml.jackson.core:jackson-databind:[2.13.5] diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java index 72021103e24c8..91319959ba691 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnector.java @@ -1,3 +1,6 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + package com.azure.cosmos.kafka.connect; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java index dc1dd43f730df..4bbb794a5906a 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java @@ -103,7 +103,7 @@ public ConfigDef config() { @Override public String version() { return KafkaCosmosConstants.CURRENT_VERSION; - } // TODO[public preview]: how this is being used + } private List> getTaskConfigs(int maxTasks) { Pair> taskUnits = this.getAllTaskUnits(); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java index 08d8ee1cd0bd7..98d758deb6e3d 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkWriteConfig.java @@ -19,6 +19,7 @@ public CosmosSinkWriteConfig( ItemWriteStrategy itemWriteStrategy, int maxRetryCount, ToleranceOnErrorLevel toleranceOnErrorLevel) { + this.bulkEnabled = bulkEnabled; this.bulkMaxConcurrentCosmosPartitions = bulkMaxConcurrentCosmosPartitions; this.bulkInitialBatchSize = bulkInitialBatchSize; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java index 1723963cb95d8..6fe94c2d52f73 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java @@ -111,7 +111,7 @@ public void writeCore(CosmosAsyncContainer container, List sinkOp return Mono.empty(); }) - .blockLast(); + .blockLast(); // TODO[This PR]: using customized schedulers } private CosmosBulkExecutionOptions getBulkExecutionOperations() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java index 8459fcaa925be..342b44aa4d108 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java @@ -136,7 +136,7 @@ private void executeWithRetry( Mono.just(this) .flatMap(data -> { if (sinkOperation.getRetryCount() > 0) { - LOGGER.debug("Retry attempted for sinkRecord {}", sinkOperation.getSinkRecord().key()); + LOGGER.debug("Retry for sinkRecord {}", sinkOperation.getSinkRecord().key()); } return execution.apply(sinkOperation); }) @@ -173,7 +173,7 @@ private void executeWithRetry( .repeat(() -> !sinkOperation.isCompleted()) .then() .subscribeOn(Schedulers.boundedElastic()) - .block();// TODO: use customized schedulers + .block();// TODO[This PR]: use customized schedulers } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java index ce824f12c5c41..9cd29a2571049 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java @@ -46,7 +46,6 @@ public List transform(String containerName, List sinkRec Object recordValue; if (record.value() instanceof Struct) { recordValue = StructToJsonMap.toJsonMap((Struct) record.value()); - // TODO: Do we need to update the value schema to map or keep it struct? } else if (record.value() instanceof Map) { recordValue = StructToJsonMap.handleMap((Map) record.value()); } else { From ac1a96fabad1795ed3edb592ab7d8fadd8385488 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 27 Feb 2024 14:30:37 -0800 Subject: [PATCH 04/44] update pom --- eng/versioning/external_dependencies.txt | 2 +- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 3 +- .../sink/KafkaCosmosWriterBase.java | 3 ++ .../sink/SinkRecordTransformer.java | 2 + .../implementation/sink/StructToJsonMap.java | 1 + .../idStrategy/ProvidedInStrategyTest.java | 54 +++++++++---------- 6 files changed, 34 insertions(+), 31 deletions(-) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 462caf0226412..463adf813ba6b 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -391,7 +391,7 @@ cosmos_org.scalastyle:scalastyle-maven-plugin;1.0.0 ## Cosmos Kafka connector under sdk\cosmos\azure-cosmos-kafka-connect\pom.xml # Cosmos Kafka connector runtime dependencies cosmos_org.apache.kafka:connect-api;3.6.0 -cosmos_org.com.jayway.jsonpath:json-path;2.9.0 +cosmos_com.jayway.jsonpath:json-path;2.9.0 # Cosmos Kafka connector tests only cosmos_org.apache.kafka:connect-runtime;3.6.0 # Maven Tools for Cosmos Kafka connector only diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index c7d01db047c33..379282953e636 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -106,7 +106,7 @@ Licensed under the MIT License. com.jayway.jsonpath - json-path + json-path 2.9.0 @@ -234,6 +234,7 @@ Licensed under the MIT License. com.azure:* org.apache.kafka:connect-api:[3.6.0] io.confluent:kafka-connect-maven-plugin:[0.12.0] + com.jayway.jsonpath:json-path:[2.9.0] diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java index 04eb22b8a8639..108f3fefa7cac 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java @@ -61,16 +61,19 @@ public void write(CosmosAsyncContainer container, List sinkRecords) } } + @SuppressWarnings("unchecked") protected String getId(Object recordValue) { checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); return ((Map) recordValue).get(ID).toString(); } + @SuppressWarnings("unchecked") protected String getEtag(Object recordValue) { checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); return ((Map) recordValue).getOrDefault(ETAG, Strings.Emtpy).toString(); } + @SuppressWarnings("unchecked") protected PartitionKey getPartitionKeyValue(Object recordValue, PartitionKeyDefinition partitionKeyDefinition) { checkArgument(recordValue instanceof Map, "Argument 'recordValue' is not valid map format."); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java index 9cd29a2571049..e693a8ccde6da 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java @@ -29,6 +29,7 @@ public SinkRecordTransformer(CosmosSinkTaskConfig sinkTaskConfig) { this.idStrategy = this.createIdStrategy(sinkTaskConfig); } + @SuppressWarnings("unchecked") public List transform(String containerName, List sinkRecords) { List toBeWrittenRecordList = new ArrayList<>(); for (SinkRecord record : sinkRecords) { @@ -72,6 +73,7 @@ public List transform(String containerName, List sinkRec return toBeWrittenRecordList; } + @SuppressWarnings("unchecked") private void maybeInsertId(Object recordValue, SinkRecord sinkRecord) { if (!(recordValue instanceof Map)) { return; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java index c0ee2031e85d7..388baa3778f8d 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/StructToJsonMap.java @@ -85,6 +85,7 @@ public static Map toJsonMap(Struct struct) { return jsonMap; } + @SuppressWarnings("unchecked") public static Map handleMap(Map map) { if (map == null) { return null; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java index 82303b2067002..d382abedddfe4 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java @@ -12,8 +12,9 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +import java.util.Arrays; +import java.util.HashMap; import java.util.LinkedHashMap; -import java.util.List; import java.util.Map; import static org.assertj.core.api.AssertionsForClassTypes.assertThat; @@ -46,7 +47,7 @@ private void returnOnKeyOrValue( @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) public void valueNotStructOrMapShouldFail(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue(Schema.STRING_SCHEMA, "a string", idStrategy, sinkRecord); idStrategy.generateId(sinkRecord); @@ -54,19 +55,19 @@ public void valueNotStructOrMapShouldFail(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) public void noIdInValueShouldFail(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); - returnOnKeyOrValue(null, Map.of(), idStrategy, sinkRecord); + returnOnKeyOrValue(null, new HashMap<>(), idStrategy, sinkRecord); idStrategy.generateId(sinkRecord); } @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void stringIdOnMapShouldReturn(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", "1234567"), + new HashMap(){{ put("id", "1234567"); }}, idStrategy, sinkRecord); assertThat("1234567").isEqualTo(idStrategy.generateId(sinkRecord)); @@ -74,11 +75,11 @@ public void stringIdOnMapShouldReturn(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void nonStringIdOnMapShouldReturn(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", 1234567), + new HashMap(){{ put("id", 1234567); }}, idStrategy, sinkRecord); assertThat("1234567").isEqualTo(idStrategy.generateId(sinkRecord)); @@ -86,7 +87,7 @@ public void nonStringIdOnMapShouldReturn(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void stringIdOnStructShouldReturn(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); Schema schema = SchemaBuilder.struct() .field("id", Schema.STRING_SCHEMA) .build(); @@ -100,7 +101,7 @@ public void stringIdOnStructShouldReturn(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void structIdOnStructShouldReturn(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); Schema idSchema = SchemaBuilder.struct() .field("name", Schema.STRING_SCHEMA) .build(); @@ -117,8 +118,7 @@ public void structIdOnStructShouldReturn(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void jsonPathOnStruct(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.name")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.name"); }}); Schema idSchema = SchemaBuilder.struct() .field("name", Schema.STRING_SCHEMA) @@ -136,13 +136,12 @@ public void jsonPathOnStruct(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void jsonPathOnMap(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.name")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.name"); }}); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", Map.of("name", "franz kafka")), + new HashMap(){{ put("id", new HashMap(){{ put("name", "franz kafka"); }}); }}, idStrategy, sinkRecord); assertThat("franz kafka").isEqualTo(idStrategy.generateId(sinkRecord)); @@ -150,12 +149,11 @@ public void jsonPathOnMap(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) public void invalidJsonPathThrows(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "invalid.path")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "invalid.path"); }}); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", Map.of("name", "franz kafka")), + new HashMap(){{ put("id", new HashMap(){{ put("name", "franz kafka"); }}); }}, idStrategy, sinkRecord); @@ -164,12 +162,11 @@ public void invalidJsonPathThrows(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", expectedExceptions = ConnectException.class, timeOut = TIMEOUT) public void jsonPathNotExistThrows(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.not.exist")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id.not.exist"); }}); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", Map.of("name", "franz kafka")), + new HashMap(){{ put("id", new HashMap(){{ put("name", "franz kafka"); }}); }}, idStrategy, sinkRecord); @@ -178,7 +175,6 @@ public void jsonPathNotExistThrows(IdStrategy idStrategy) { @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void complexJsonPath(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); Map map1 = new LinkedHashMap<>(); map1.put("id", 0); map1.put("name", "cosmos kramer"); @@ -191,30 +187,30 @@ public void complexJsonPath(IdStrategy idStrategy) { SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", List.of(map1, map2)), + new HashMap() {{ put("id", Arrays.asList(map1, map2)); }}, idStrategy, sinkRecord); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[0].name")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[0].name"); }}); assertThat("cosmos kramer").isEqualTo(idStrategy.generateId(sinkRecord)); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[1].name")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[1].name"); }}); assertThat("franz kafka").isEqualTo(idStrategy.generateId(sinkRecord)); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[*].id")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id[*].id"); }}); assertThat("[0,1]").isEqualTo(idStrategy.generateId(sinkRecord)); - idStrategy.configure(Map.of(ProvidedInConfig.JSON_PATH_CONFIG, "$.id")); + idStrategy.configure(new HashMap(){{ put(ProvidedInConfig.JSON_PATH_CONFIG, "$.id"); }}); assertThat("[{\"id\":0,\"name\":\"cosmos kramer\",\"occupation\":\"unknown\"},{\"id\":1,\"name\":\"franz kafka\",\"occupation\":\"writer\"}]").isEqualTo(idStrategy.generateId(sinkRecord)); } @Test(groups = { "unit" }, dataProvider = "idStrategyParameterProvider", timeOut = TIMEOUT) public void generatedIdSanitized(IdStrategy idStrategy) { - idStrategy.configure(Map.of()); + idStrategy.configure(new HashMap<>()); SinkRecord sinkRecord = Mockito.mock(SinkRecord.class); returnOnKeyOrValue( null, - Map.of("id", "#my/special\\id?"), + new HashMap() {{put("id", "#my/special\\id?");}}, idStrategy, sinkRecord); From b1bc7eed828f69a11bd4cba3efe5673960c3a893 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 27 Feb 2024 15:22:50 -0800 Subject: [PATCH 05/44] change to use customized schedulers --- ...smosConfig.java => KafkaCosmosConfig.java} | 4 ++-- .../implementation/KafkaCosmosSchedulers.java | 19 +++++++++++++++++++ .../implementation/sink/CosmosSinkConfig.java | 7 +++---- .../sink/KafkaCosmosBulkWriter.java | 13 +++++++++---- .../sink/KafkaCosmosPointWriter.java | 6 +++--- .../source/CosmosSourceConfig.java | 6 +++--- 6 files changed, 39 insertions(+), 16 deletions(-) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/{CosmosConfig.java => KafkaCosmosConfig.java} (98%) create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java similarity index 98% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java index 8b56ca1b017ea..5c75056b6f7be 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java @@ -21,7 +21,7 @@ /** * Common Configuration for Cosmos DB Kafka source connector and sink connector. */ -public class CosmosConfig extends AbstractConfig { +public class KafkaCosmosConfig extends AbstractConfig { protected static final ConfigDef.Validator NON_EMPTY_STRING = new ConfigDef.NonEmptyString(); private static final String CONFIG_PREFIX = "kafka.connect.cosmos."; @@ -52,7 +52,7 @@ public class CosmosConfig extends AbstractConfig { private final CosmosAccountConfig accountConfig; - public CosmosConfig(ConfigDef config, Map parsedConfig) { + public KafkaCosmosConfig(ConfigDef config, Map parsedConfig) { super(config, parsedConfig); this.accountConfig = this.parseAccountConfig(); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java new file mode 100644 index 0000000000000..e5e44128a8af3 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java @@ -0,0 +1,19 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation; + +import reactor.core.scheduler.Scheduler; +import reactor.core.scheduler.Schedulers; + +public class KafkaCosmosSchedulers { + private final static String SINK_BOUNDED_ELASTIC_THREAD_NAME = "sink-bounded-elastic"; + private final static int TTL_FOR_SCHEDULER_WORKER_IN_SECONDS = 60; // same as BoundedElasticScheduler.DEFAULT_TTL_SECONDS + public final static Scheduler SINK_BOUNDED_ELASTIC = Schedulers.newBoundedElastic( + Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE, + Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, + SINK_BOUNDED_ELASTIC_THREAD_NAME, + TTL_FOR_SCHEDULER_WORKER_IN_SECONDS, + true + ); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java index 9d87a937148c2..a54831c867c45 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java @@ -3,9 +3,8 @@ package com.azure.cosmos.kafka.connect.implementation.sink; -import com.azure.cosmos.implementation.Strings; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.kafka.connect.implementation.CosmosConfig; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; @@ -16,7 +15,7 @@ /** * Common Configuration for Cosmos DB Kafka sink connector. */ -public class CosmosSinkConfig extends CosmosConfig { +public class CosmosSinkConfig extends KafkaCosmosConfig { private static final String SINK_CONFIG_PREFIX = "kafka.connect.cosmos.sink."; // error tolerance @@ -116,7 +115,7 @@ public CosmosSinkConfig(ConfigDef config, Map parsedConfig) { } public static ConfigDef getConfigDef() { - ConfigDef configDef = CosmosConfig.getConfigDef(); + ConfigDef configDef = KafkaCosmosConfig.getConfigDef(); defineWriteConfig(configDef); defineContainersConfig(configDef); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java index 6fe94c2d52f73..b1ac3d06ef039 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java @@ -10,6 +10,7 @@ import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosSchedulers; import com.azure.cosmos.models.CosmosBulkExecutionOptions; import com.azure.cosmos.models.CosmosBulkItemRequestOptions; import com.azure.cosmos.models.CosmosBulkItemResponse; @@ -68,9 +69,12 @@ public void writeCore(CosmosAsyncContainer container, List sinkOp .flatMapMany(itemOperations -> { Flux> cosmosBulkOperationResponseFlux = - container.executeBulkOperations( - Flux.fromIterable(itemOperations).mergeWith(bulkRetryEmitter.asFlux()), - bulkExecutionOptions); + container + .executeBulkOperations( + Flux.fromIterable(itemOperations) + .mergeWith(bulkRetryEmitter.asFlux()) + .publishOn(KafkaCosmosSchedulers.SINK_BOUNDED_ELASTIC), + bulkExecutionOptions); return cosmosBulkOperationResponseFlux; }) .flatMap(itemResponse -> { @@ -111,7 +115,8 @@ public void writeCore(CosmosAsyncContainer container, List sinkOp return Mono.empty(); }) - .blockLast(); // TODO[This PR]: using customized schedulers + .subscribeOn(KafkaCosmosSchedulers.SINK_BOUNDED_ELASTIC) + .blockLast(); } private CosmosBulkExecutionOptions getBulkExecutionOperations() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java index 342b44aa4d108..f55d45102b8e9 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java @@ -7,12 +7,12 @@ import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.implementation.guava25.base.Function; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosSchedulers; import com.azure.cosmos.models.CosmosItemRequestOptions; import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import reactor.core.publisher.Mono; -import reactor.core.scheduler.Schedulers; import java.util.List; @@ -172,8 +172,8 @@ private void executeWithRetry( }) .repeat(() -> !sinkOperation.isCompleted()) .then() - .subscribeOn(Schedulers.boundedElastic()) - .block();// TODO[This PR]: use customized schedulers + .subscribeOn(KafkaCosmosSchedulers.SINK_BOUNDED_ELASTIC) + .block(); } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java index e32d99ed1e8b3..1365d1e53575e 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java @@ -5,7 +5,7 @@ import com.azure.cosmos.implementation.Strings; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.kafka.connect.implementation.CosmosConfig; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; @@ -19,7 +19,7 @@ /** * Common Configuration for Cosmos DB Kafka source connector. */ -public class CosmosSourceConfig extends CosmosConfig { +public class CosmosSourceConfig extends KafkaCosmosConfig { // configuration only targets to source connector private static final String SOURCE_CONF_PREFIX = "kafka.connect.cosmos.source."; @@ -110,7 +110,7 @@ public CosmosSourceConfig(ConfigDef configDef, Map parsedConfigs) { } public static ConfigDef getConfigDef() { - ConfigDef configDef = CosmosConfig.getConfigDef(); + ConfigDef configDef = KafkaCosmosConfig.getConfigDef(); defineContainersConfig(configDef); defineMetadataConfig(configDef); From 223d6d720c6e1621150b740c9e350c252b850dc6 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Thu, 29 Feb 2024 17:43:29 -0800 Subject: [PATCH 06/44] pom file update --- .../doc/configuration-reference.md | 22 +++++++++---------- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 2 +- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md b/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md index eb00fb6a3e593..08e879937c06f 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md +++ b/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md @@ -25,14 +25,14 @@ | `kafka.connect.cosmos.source.messageKey.field` | `id` | The field to use as the message key. | ## Sink Connector Configuration -| Config Property Name | Default | Description | -|:---------------------------------------------------------------|:----------------|:-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `kafka.connect.cosmos.sink.database.name` | None | Cosmos DB database name. | -| `kafka.connect.cosmos.sink.containers.topicMap` | `[]` | A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2. | -| `kafka.connect.cosmos.sink.errors.tolerance` | `None` | Error tolerance level after exhausting all retries. `None` for fail on error. `All` for log and continue | -| `kafka.connect.cosmos.sink.bulk.enabled` | `true` | Flag to indicate whether Cosmos DB bulk mode is enabled for Sink connector. By default it is true. | -| `kafka.connect.cosmos.sink.bulk.maxConcurrentCosmosPartitions` | `-1` | Cosmos DB Item Write Max Concurrent Cosmos Partitions. If not specified it will be determined based on the number of the container's physical partitions which would indicate every Spark partition is expected to have data from all Cosmos physical partitions. If specified it indicates from at most how many Cosmos Physical Partitions each Spark partition contains data. So this config can be used to make bulk processing more efficient when input data in Spark has been repartitioned to balance to how many Cosmos partitions each Spark partition needs to write. This is mainly useful for very large containers (with hundreds of physical partitions. | -| `kafka.connect.cosmos.sink.bulk.initialBatchSize` | `1` | Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch size is getting automatically tuned based on the throttling rate. By default the initial micro batch size is 1. Reduce this when you want to avoid that the first few requests consume too many RUs. | -| `kafka.connect.cosmos.sink.write.strategy` | `ItemOverwrite` | "Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert), `ItemAppend` (using create, ignore pre-existing items i.e., Conflicts), `ItemDelete` (deletes based on id/pk of data frame), `ItemDeleteIfNotModified` (deletes based on id/pk of data frame if etag hasn't changed since collecting id/pk), `ItemOverwriteIfNotModified` (using create if etag is empty, update/replace with etag pre-condition otherwise, if document was updated the pre-condition failure is ignored) | -| `kafka.connect.cosmos.sink.maxRetryCount` | `10` | Cosmos DB max retry attempts on write failures for Sink connector. By default, the connector will retry on transient write errors for up to 10 times. | -| `kafka.connect.cosmos.sink.id.strategy` | `ProvidedInValueStrategy` | A strategy used to populate the document with an ``id``. Valid strategies are: ``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, ``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. Configuration properties prefixed with``id.strategy`` are passed through to the strategy. For example, when using ``id.strategy=TemplateStrategy`` , the property ``id.strategy.template`` is passed through to the template strategy and used to specify the template string to be used in constructing the ``id``. | +| Config Property Name | Default | Description | +|:---------------------------------------------------------------|:--------------------------|:------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `kafka.connect.cosmos.sink.database.name` | None | Cosmos DB database name. | +| `kafka.connect.cosmos.sink.containers.topicMap` | None | A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2. | +| `kafka.connect.cosmos.sink.errors.tolerance` | `None` | Error tolerance level after exhausting all retries. `None` for fail on error. `All` for log and continue | +| `kafka.connect.cosmos.sink.bulk.enabled` | `true` | Flag to indicate whether Cosmos DB bulk mode is enabled for Sink connector. By default it is true. | +| `kafka.connect.cosmos.sink.bulk.maxConcurrentCosmosPartitions` | `-1` | Cosmos DB Item Write Max Concurrent Cosmos Partitions. If not specified it will be determined based on the number of the container's physical partitions which would indicate every Spark partition is expected to have data from all Cosmos physical partitions. If specified it indicates from at most how many Cosmos Physical Partitions each Spark partition contains data. So this config can be used to make bulk processing more efficient when input data in Spark has been repartitioned to balance to how many Cosmos partitions each Spark partition needs to write. This is mainly useful for very large containers (with hundreds of physical partitions. | +| `kafka.connect.cosmos.sink.bulk.initialBatchSize` | `1` | Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch size is getting automatically tuned based on the throttling rate. By default the initial micro batch size is 1. Reduce this when you want to avoid that the first few requests consume too many RUs. | +| `kafka.connect.cosmos.sink.write.strategy` | `ItemOverwrite` | Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert), `ItemAppend` (using create, ignore pre-existing items i.e., Conflicts), `ItemDelete` (deletes based on id/pk of data frame), `ItemDeleteIfNotModified` (deletes based on id/pk of data frame if etag hasn't changed since collecting id/pk), `ItemOverwriteIfNotModified` (using create if etag is empty, update/replace with etag pre-condition otherwise, if document was updated the pre-condition failure is ignored) | +| `kafka.connect.cosmos.sink.maxRetryCount` | `10` | Cosmos DB max retry attempts on write failures for Sink connector. By default, the connector will retry on transient write errors for up to 10 times. | +| `kafka.connect.cosmos.sink.id.strategy` | `ProvidedInValueStrategy` | A strategy used to populate the document with an ``id``. Valid strategies are: ``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, ``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. Configuration properties prefixed with``id.strategy`` are passed through to the strategy. For example, when using ``id.strategy=TemplateStrategy`` , the property ``id.strategy.template`` is passed through to the template strategy and used to specify the template string to be used in constructing the ``id``. | diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index cc92b5d331d9f..f52cce022faf4 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -234,7 +234,7 @@ Licensed under the MIT License. com.azure:* org.apache.kafka:connect-api:[3.6.0] io.confluent:kafka-connect-maven-plugin:[0.12.0] - com.jayway.jsonpath:json-path:[2.9.0] + com.jayway.jsonpath:json-path:[2.9.0] From a5116b28a35cba744e23b4dceadf22c070882f01 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Thu, 29 Feb 2024 18:49:02 -0800 Subject: [PATCH 07/44] update pom file --- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index f52cce022faf4..fa4250e355894 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -106,8 +106,8 @@ Licensed under the MIT License. com.jayway.jsonpath - json-path - 2.9.0 + json-path + 2.9.0 From 490d550c75d50c9caccb12c578cf85bc261247ae Mon Sep 17 00:00:00 2001 From: annie-mac Date: Fri, 1 Mar 2024 08:32:45 -0800 Subject: [PATCH 08/44] fix --- .../checkstyle/checkstyle-suppressions.xml | 1 + .../implementation/KafkaCosmosSchedulers.java | 6 +++--- .../implementation/sink/CosmosSinkConfig.java | 3 +-- .../implementation/sink/KafkaCosmosBulkWriter.java | 8 ++++---- .../sink/KafkaCosmosPointWriter.java | 4 ++-- .../connect/implementation/sink/SinkOperation.java | 4 +++- .../implementation/sink/SinkRecordTransformer.java | 14 +++++++------- .../AbstractIdStrategy.java | 2 +- .../AbstractIdStrategyConfig.java | 2 +- .../FullKeyStrategy.java | 2 +- .../{idStrategy => idstrategy}/IdStrategy.java | 2 +- .../KafkaMetadataStrategy.java | 2 +- .../KafkaMetadataStrategyConfig.java | 2 +- .../ProvidedInConfig.java | 2 +- .../ProvidedInKeyStrategy.java | 2 +- .../ProvidedInStrategy.java | 2 +- .../ProvidedInValueStrategy.java | 2 +- .../TemplateStrategy.java | 2 +- .../TemplateStrategyConfig.java | 2 +- .../kafka/connect/KafkaCosmosTestSuiteBase.java | 9 +++++++++ .../sink/idStrategy/ProvidedInStrategyTest.java | 4 ++++ 21 files changed, 46 insertions(+), 31 deletions(-) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/AbstractIdStrategy.java (93%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/AbstractIdStrategyConfig.java (90%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/FullKeyStrategy.java (87%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/IdStrategy.java (82%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/KafkaMetadataStrategy.java (91%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/KafkaMetadataStrategyConfig.java (95%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/ProvidedInConfig.java (95%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/ProvidedInKeyStrategy.java (79%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/ProvidedInStrategy.java (95%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/ProvidedInValueStrategy.java (79%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/TemplateStrategy.java (97%) rename sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/{idStrategy => idstrategy}/TemplateStrategyConfig.java (96%) diff --git a/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle-suppressions.xml b/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle-suppressions.xml index db3789c74a934..b11579484fd0a 100755 --- a/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle-suppressions.xml +++ b/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle-suppressions.xml @@ -319,6 +319,7 @@ the main ServiceBusClientBuilder. --> + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java index e5e44128a8af3..081d0baa30f2a 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosSchedulers.java @@ -7,9 +7,9 @@ import reactor.core.scheduler.Schedulers; public class KafkaCosmosSchedulers { - private final static String SINK_BOUNDED_ELASTIC_THREAD_NAME = "sink-bounded-elastic"; - private final static int TTL_FOR_SCHEDULER_WORKER_IN_SECONDS = 60; // same as BoundedElasticScheduler.DEFAULT_TTL_SECONDS - public final static Scheduler SINK_BOUNDED_ELASTIC = Schedulers.newBoundedElastic( + private static final String SINK_BOUNDED_ELASTIC_THREAD_NAME = "sink-bounded-elastic"; + private static final int TTL_FOR_SCHEDULER_WORKER_IN_SECONDS = 60; // same as BoundedElasticScheduler.DEFAULT_TTL_SECONDS + public static final Scheduler SINK_BOUNDED_ELASTIC = Schedulers.newBoundedElastic( Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE, Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, SINK_BOUNDED_ELASTIC_THREAD_NAME, diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java index a54831c867c45..1b70e823573c5 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java @@ -240,8 +240,7 @@ private static void defineIdStrategyConfig(ConfigDef configDef) { idStrategyConfigGroupName, idStrategyConfigGroupOrder++, ConfigDef.Width.MEDIUM, - ID_STRATEGY_DISPLAY - ); + ID_STRATEGY_DISPLAY); } private CosmosSinkWriteConfig parseWriteConfig() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java index b1ac3d06ef039..97515fa530feb 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java @@ -37,7 +37,7 @@ public class KafkaCosmosBulkWriter extends KafkaCosmosWriterBase { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosBulkWriter.class); private static final int MAX_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS = 10000; private static final int MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS = 1000; - private static final Random random = new Random(); + private static final Random RANDOM = new Random(); private final CosmosSinkWriteConfig writeConfig; private final Sinks.EmitFailureHandler emitFailureHandler; @@ -241,7 +241,7 @@ private Mono scheduleRetry( if (KafkaCosmosExceptionsHelper.isTimeoutException(exception)) { Duration delayDuration = Duration.ofMillis( MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS - + random.nextInt(MAX_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS - MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS)); + + RANDOM.nextInt(MAX_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS - MIN_DELAY_ON_408_REQUEST_TIMEOUT_IN_MS)); return retryMono.delaySubscription(delayDuration); } @@ -257,11 +257,11 @@ BulkOperationFailedException handleErrorStatusCode( int effectiveStatusCode = itemResponse != null ? itemResponse.getStatusCode() - : (exception != null && exception instanceof CosmosException ? ((CosmosException)exception).getStatusCode() : HttpConstants.StatusCodes.REQUEST_TIMEOUT); + : (exception != null && exception instanceof CosmosException ? ((CosmosException) exception).getStatusCode() : HttpConstants.StatusCodes.REQUEST_TIMEOUT); int effectiveSubStatusCode = itemResponse != null ? itemResponse.getSubStatusCode() - : (exception != null && exception instanceof CosmosException ? ((CosmosException)exception).getSubStatusCode() : 0); + : (exception != null && exception instanceof CosmosException ? ((CosmosException) exception).getSubStatusCode() : 0); String errorMessage = String.format( diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java index f55d45102b8e9..a13c5d1a5266e 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java @@ -93,8 +93,8 @@ private void replaceIfNotModifiedWithRetry(CosmosAsyncContainer container, SinkO }); }, (throwable) -> { - return KafkaCosmosExceptionsHelper.isNotFoundException(throwable) || - KafkaCosmosExceptionsHelper.isPreconditionFailedException(throwable); + return KafkaCosmosExceptionsHelper.isNotFoundException(throwable) + || KafkaCosmosExceptionsHelper.isPreconditionFailedException(throwable); }, sinkOperation ); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java index 0319ec2de0338..599a71ec99848 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkOperation.java @@ -42,7 +42,9 @@ public int getRetryCount() { return this.retryCount.get(); } - public void retry() { this.retryCount.incrementAndGet(); } + public void retry() { + this.retryCount.incrementAndGet(); + } public Throwable getException() { return this.exception.get(); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java index e693a8ccde6da..007d09bb793d7 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/SinkRecordTransformer.java @@ -3,13 +3,13 @@ package com.azure.cosmos.kafka.connect.implementation.sink; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.AbstractIdStrategyConfig; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.FullKeyStrategy; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.IdStrategy; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.KafkaMetadataStrategy; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.ProvidedInKeyStrategy; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.ProvidedInValueStrategy; -import com.azure.cosmos.kafka.connect.implementation.sink.idStrategy.TemplateStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.AbstractIdStrategyConfig; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.FullKeyStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.IdStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.KafkaMetadataStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.ProvidedInKeyStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.ProvidedInValueStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.TemplateStrategy; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/AbstractIdStrategy.java similarity index 93% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/AbstractIdStrategy.java index c44c4866c8553..41bc401b81360 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/AbstractIdStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import java.util.Map; import java.util.regex.Pattern; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/AbstractIdStrategyConfig.java similarity index 90% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/AbstractIdStrategyConfig.java index a90773ccad9f1..1713ecc79636e 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/AbstractIdStrategyConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/AbstractIdStrategyConfig.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/FullKeyStrategy.java similarity index 87% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/FullKeyStrategy.java index 9b2823e9b6dd4..62fc6f72a3406 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/FullKeyStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/FullKeyStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import java.util.HashMap; import java.util.Map; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/IdStrategy.java similarity index 82% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/IdStrategy.java index e60f874afaa2f..b4ce03d4f73e3 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/IdStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/IdStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import org.apache.kafka.common.Configurable; import org.apache.kafka.connect.sink.SinkRecord; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/KafkaMetadataStrategy.java similarity index 91% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/KafkaMetadataStrategy.java index 2f1f74c064017..99d705f062f4c 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/KafkaMetadataStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import java.util.HashMap; import java.util.Map; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/KafkaMetadataStrategyConfig.java similarity index 95% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/KafkaMetadataStrategyConfig.java index 4d51afc62497b..b29d59e4409ca 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/KafkaMetadataStrategyConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/KafkaMetadataStrategyConfig.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import org.apache.kafka.common.config.ConfigDef; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInConfig.java similarity index 95% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInConfig.java index 90033fbfaef43..de3892baa1d2e 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInConfig.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import org.apache.kafka.common.config.ConfigDef; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInKeyStrategy.java similarity index 79% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInKeyStrategy.java index 399267902b6f8..930e2435351e8 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInKeyStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInKeyStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; public class ProvidedInKeyStrategy extends ProvidedInStrategy { public ProvidedInKeyStrategy() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInStrategy.java similarity index 95% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInStrategy.java index f081f9f3a1f19..79b4ed19f655d 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.ConnectException; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInValueStrategy.java similarity index 79% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInValueStrategy.java index ff35b4662a838..ca5b794fc8bd8 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInValueStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/ProvidedInValueStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; public class ProvidedInValueStrategy extends ProvidedInStrategy { public ProvidedInValueStrategy() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java similarity index 97% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java index f81554e4eef10..1f40827050dcc 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; import org.apache.kafka.connect.data.Values; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java similarity index 96% rename from sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java rename to sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java index 68dc2af127767..022b38ac83133 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/TemplateStrategyConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java @@ -1,7 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. -package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; import org.apache.kafka.common.config.ConfigDef; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java index 8b62d01fb0cb8..be655173cbc25 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java @@ -120,6 +120,15 @@ public static void beforeSuite() { } } + @BeforeSuite(groups = { "unit" }, timeOut = SUITE_SETUP_TIMEOUT) + public static void beforeSuiteUnit() { + logger.info("beforeSuite for unit tests started"); + + databaseName = "KafkaCosmosTest-" + UUID.randomUUID(); + multiPartitionContainerName = UUID.randomUUID().toString(); + singlePartitionContainerName = UUID.randomUUID().toString(); + } + @AfterSuite(groups = { "kafka" }, timeOut = SUITE_SHUTDOWN_TIMEOUT) public static void afterSuite() { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java index d382abedddfe4..19c8fb92dd374 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/sink/idStrategy/ProvidedInStrategyTest.java @@ -3,6 +3,10 @@ package com.azure.cosmos.kafka.connect.implementation.sink.idStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.IdStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.ProvidedInConfig; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.ProvidedInKeyStrategy; +import com.azure.cosmos.kafka.connect.implementation.sink.idstrategy.ProvidedInValueStrategy; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; From 1c5b5f0235707f8946276fd06c9bebfad82958d2 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 12 Mar 2024 07:57:53 -0700 Subject: [PATCH 09/44] resolve comments --- .../doc/configuration-reference.md | 22 +++++++++---------- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 4 ++++ .../implementation/sink/CosmosSinkConfig.java | 8 +++---- .../idstrategy/TemplateStrategyConfig.java | 1 - 4 files changed, 19 insertions(+), 16 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md b/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md index 08e879937c06f..064aaab81a76e 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md +++ b/sdk/cosmos/azure-cosmos-kafka-connect/doc/configuration-reference.md @@ -25,14 +25,14 @@ | `kafka.connect.cosmos.source.messageKey.field` | `id` | The field to use as the message key. | ## Sink Connector Configuration -| Config Property Name | Default | Description | -|:---------------------------------------------------------------|:--------------------------|:------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| `kafka.connect.cosmos.sink.database.name` | None | Cosmos DB database name. | -| `kafka.connect.cosmos.sink.containers.topicMap` | None | A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2. | -| `kafka.connect.cosmos.sink.errors.tolerance` | `None` | Error tolerance level after exhausting all retries. `None` for fail on error. `All` for log and continue | -| `kafka.connect.cosmos.sink.bulk.enabled` | `true` | Flag to indicate whether Cosmos DB bulk mode is enabled for Sink connector. By default it is true. | -| `kafka.connect.cosmos.sink.bulk.maxConcurrentCosmosPartitions` | `-1` | Cosmos DB Item Write Max Concurrent Cosmos Partitions. If not specified it will be determined based on the number of the container's physical partitions which would indicate every Spark partition is expected to have data from all Cosmos physical partitions. If specified it indicates from at most how many Cosmos Physical Partitions each Spark partition contains data. So this config can be used to make bulk processing more efficient when input data in Spark has been repartitioned to balance to how many Cosmos partitions each Spark partition needs to write. This is mainly useful for very large containers (with hundreds of physical partitions. | -| `kafka.connect.cosmos.sink.bulk.initialBatchSize` | `1` | Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch size is getting automatically tuned based on the throttling rate. By default the initial micro batch size is 1. Reduce this when you want to avoid that the first few requests consume too many RUs. | -| `kafka.connect.cosmos.sink.write.strategy` | `ItemOverwrite` | Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert), `ItemAppend` (using create, ignore pre-existing items i.e., Conflicts), `ItemDelete` (deletes based on id/pk of data frame), `ItemDeleteIfNotModified` (deletes based on id/pk of data frame if etag hasn't changed since collecting id/pk), `ItemOverwriteIfNotModified` (using create if etag is empty, update/replace with etag pre-condition otherwise, if document was updated the pre-condition failure is ignored) | -| `kafka.connect.cosmos.sink.maxRetryCount` | `10` | Cosmos DB max retry attempts on write failures for Sink connector. By default, the connector will retry on transient write errors for up to 10 times. | -| `kafka.connect.cosmos.sink.id.strategy` | `ProvidedInValueStrategy` | A strategy used to populate the document with an ``id``. Valid strategies are: ``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, ``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. Configuration properties prefixed with``id.strategy`` are passed through to the strategy. For example, when using ``id.strategy=TemplateStrategy`` , the property ``id.strategy.template`` is passed through to the template strategy and used to specify the template string to be used in constructing the ``id``. | +| Config Property Name | Default | Description | +|:---------------------------------------------------------------|:--------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `kafka.connect.cosmos.sink.database.name` | None | Cosmos DB database name. | +| `kafka.connect.cosmos.sink.containers.topicMap` | None | A comma delimited list of Kafka topics mapped to Cosmos containers. For example: topic1#con1,topic2#con2. | +| `kafka.connect.cosmos.sink.errors.tolerance` | `None` | Error tolerance level after exhausting all retries. `None` for fail on error. `All` for log and continue | +| `kafka.connect.cosmos.sink.bulk.enabled` | `true` | Flag to indicate whether Cosmos DB bulk mode is enabled for Sink connector. By default it is true. | +| `kafka.connect.cosmos.sink.bulk.maxConcurrentCosmosPartitions` | `-1` | Cosmos DB Item Write Max Concurrent Cosmos Partitions. If not specified it will be determined based on the number of the container's physical partitions which would indicate every batch is expected to have data from all Cosmos physical partitions. If specified it indicates from at most how many Cosmos Physical Partitions each batch contains data. So this config can be used to make bulk processing more efficient when input data in each batch has been repartitioned to balance to how many Cosmos partitions each batch needs to write. This is mainly useful for very large containers (with hundreds of physical partitions. | +| `kafka.connect.cosmos.sink.bulk.initialBatchSize` | `1` | Cosmos DB initial bulk micro batch size - a micro batch will be flushed to the backend when the number of documents enqueued exceeds this size - or the target payload size is met. The micro batch size is getting automatically tuned based on the throttling rate. By default the initial micro batch size is 1. Reduce this when you want to avoid that the first few requests consume too many RUs. | +| `kafka.connect.cosmos.sink.write.strategy` | `ItemOverwrite` | Cosmos DB Item write Strategy: `ItemOverwrite` (using upsert), `ItemAppend` (using create, ignore pre-existing items i.e., Conflicts), `ItemDelete` (deletes based on id/pk of data frame), `ItemDeleteIfNotModified` (deletes based on id/pk of data frame if etag hasn't changed since collecting id/pk), `ItemOverwriteIfNotModified` (using create if etag is empty, update/replace with etag pre-condition otherwise, if document was updated the pre-condition failure is ignored) | +| `kafka.connect.cosmos.sink.maxRetryCount` | `10` | Cosmos DB max retry attempts on write failures for Sink connector. By default, the connector will retry on transient write errors for up to 10 times. | +| `kafka.connect.cosmos.sink.id.strategy` | `ProvidedInValueStrategy` | A strategy used to populate the document with an ``id``. Valid strategies are: ``TemplateStrategy``, ``FullKeyStrategy``, ``KafkaMetadataStrategy``, ``ProvidedInKeyStrategy``, ``ProvidedInValueStrategy``. Configuration properties prefixed with``id.strategy`` are passed through to the strategy. For example, when using ``id.strategy=TemplateStrategy`` , the property ``id.strategy.template`` is passed through to the template strategy and used to specify the template string to be used in constructing the ``id``. | diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index cf165173e82a8..3f2cb549f60ef 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -336,6 +336,10 @@ Licensed under the MIT License. reactor ${shadingPrefix}.reactor + + com.jayway.jsonpath + ${shadingPrefix}.com.jayway.jsonpath + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java index 1b70e823573c5..d029105846a56 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java @@ -37,10 +37,10 @@ public class CosmosSinkConfig extends KafkaCosmosConfig { private static final String BULK_MAX_CONCURRENT_PARTITIONS_DOC = "Cosmos DB Item Write Max Concurrent Cosmos Partitions." + " If not specified it will be determined based on the number of the container's physical partitions -" - + " which would indicate every Spark partition is expected to have data from all Cosmos physical partitions." - + " If specified it indicates from at most how many Cosmos Physical Partitions each Spark partition contains" - + " data. So this config can be used to make bulk processing more efficient when input data in Spark has been" - + " repartitioned to balance to how many Cosmos partitions each Spark partition needs to write. This is mainly" + + " which would indicate every batch is expected to have data from all Cosmos physical partitions." + + " If specified it indicates from at most how many Cosmos Physical Partitions each batch contains data." + + " So this config can be used to make bulk processing more efficient when input data in each batch has been" + + " repartitioned to balance to how many Cosmos partitions each batch needs to write. This is mainly" + " useful for very large containers (with hundreds of physical partitions)."; private static final String BULK_MAX_CONCURRENT_PARTITIONS_DISPLAY = "Cosmos DB Item Write Max Concurrent Cosmos Partitions."; private static final int DEFAULT_BULK_MAX_CONCURRENT_PARTITIONS = -1; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java index 022b38ac83133..c03c605438d9b 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategyConfig.java @@ -29,7 +29,6 @@ public TemplateStrategyConfig(ConfigDef definition, Map originals) { this.template = getString(TEMPLATE_CONFIG); } - public static ConfigDef getConfig() { ConfigDef result = new ConfigDef(); From faf25ed8a7127d5691552a0b9853911888c7f09d Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 12 Mar 2024 14:16:53 -0700 Subject: [PATCH 10/44] add e2e test for sink connector --- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 23 ++++- .../connect/CosmosDbSinkConnectorITest.java | 99 +++++++++++++++++++ .../connect/KafkaCosmosConnectContainer.java | 39 +++++++- .../connect/KafkaCosmosTestSuiteBase.java | 13 ++- .../resources/kafka-integration-testng.xml | 35 +++++++ 5 files changed, 202 insertions(+), 7 deletions(-) create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index 3f2cb549f60ef..c19ed713b56eb 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -477,7 +477,7 @@ Licensed under the MIT License. - kafka-integration + kafka kafka @@ -496,5 +496,26 @@ Licensed under the MIT License. + + + kafka-integration + + kafka-integration + + + + + org.apache.maven.plugins + maven-failsafe-plugin + 3.2.5 + + + src/test/resources/kafka-integration-testng.xml + + + + + + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java new file mode 100644 index 0000000000000..9a0714e9f5d03 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java @@ -0,0 +1,99 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect; + +import com.azure.cosmos.CosmosAsyncClient; +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.implementation.Utils; +import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; +import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkConfig; +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.storage.StringConverter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +public class CosmosDbSinkConnectorITest extends KafkaCosmosIntegrationTestSuiteBase { + private static final Logger logger = LoggerFactory.getLogger(CosmosDbSinkConnectorITest.class); + + // TODO[public preview]: add more integration tests + @Test(groups = { "kafka-integration"}, timeOut = TIMEOUT) + public void sinkToSingleContainer() throws InterruptedException { + Map sinkConnectorConfig = new HashMap<>(); + + sinkConnectorConfig.put("topics", singlePartitionContainerName); + sinkConnectorConfig.put("value.converter", JsonConverter.class.getName()); + // TODO[Public Preview]: add tests for with schema + sinkConnectorConfig.put("value.converter.schemas.enable", "false"); + sinkConnectorConfig.put("key.converter", StringConverter.class.getName()); + sinkConnectorConfig.put("connector.class", "com.azure.cosmos.kafka.connect.CosmosDBSinkConnector"); + sinkConnectorConfig.put("kafka.connect.cosmos.accountEndpoint", KafkaCosmosTestConfigurations.HOST); + sinkConnectorConfig.put("kafka.connect.cosmos.accountKey", KafkaCosmosTestConfigurations.MASTER_KEY); + sinkConnectorConfig.put("kafka.connect.cosmos.applicationName", "Test"); + sinkConnectorConfig.put("kafka.connect.cosmos.sink.database.name", databaseName); + sinkConnectorConfig.put("kafka.connect.cosmos.sink.containers.topicMap", singlePartitionContainerName + "#" + singlePartitionContainerName); + + // Create topic ahead of time + kafkaCosmosConnectContainer.createTopic(singlePartitionContainerName, 1); + + CosmosSinkConfig sinkConfig = new CosmosSinkConfig(sinkConnectorConfig); + CosmosAsyncClient client = CosmosClientStore.getCosmosClient(sinkConfig.getAccountConfig()); + CosmosAsyncContainer container = client.getDatabase(databaseName).getContainer(singlePartitionContainerName); + + String connectorName = "simpleTest-" + UUID.randomUUID(); + try { + // register the sink connector + kafkaCosmosConnectContainer.registerConnector(connectorName, sinkConnectorConfig); + + KafkaProducer kafkaProducer = kafkaCosmosConnectContainer.getProducer(); + + // first create few records in the topic + logger.info("Creating sink records..."); + List recordValueIds = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + TestItem testItem = TestItem.createNewItem(); + ProducerRecord record = + new ProducerRecord<>(singlePartitionContainerName, testItem.getId(), Utils.getSimpleObjectMapper().valueToTree(testItem)); + kafkaProducer.send(record); + recordValueIds.add(testItem.getId()); + } + + // Wait for some time for the sink connector to process all records + Thread.sleep(5000); + // read from the container and verify all the items are created + String query = "select * from c"; + List createdItemIds = container.queryItems(query, TestItem.class) + .byPage() + .flatMapIterable(response -> response.getResults()) + .map(TestItem::getId) + .collectList() + .block(); + assertThat(createdItemIds.size()).isEqualTo(recordValueIds.size()); + assertThat(createdItemIds.containsAll(recordValueIds)).isTrue(); + + } finally { + if (client != null) { + logger.info("cleaning container {}", singlePartitionContainerName); + cleanUpContainer(client, databaseName, singlePartitionContainerName); + client.close(); + } + + // IMPORTANT: remove the connector after use + if (kafkaCosmosConnectContainer != null) { + kafkaCosmosConnectContainer.deleteConnector(connectorName); + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConnectContainer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConnectContainer.java index 6587222aaee5e..f0bef9d3d65cc 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConnectContainer.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosConnectContainer.java @@ -8,8 +8,12 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.connect.json.JsonDeserializer; +import org.apache.kafka.connect.json.JsonSerializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.sourcelab.kafka.connect.apiclient.Configuration; @@ -28,6 +32,7 @@ public class KafkaCosmosConnectContainer extends GenericContainer kafkaConsumer; + private KafkaProducer kafkaProducer; private AdminClient adminClient; private int replicationFactor = 1; @@ -56,7 +61,7 @@ private void defaultConfig() { private Properties defaultConsumerConfig() { Properties kafkaConsumerProperties = new Properties(); - kafkaConsumerProperties.put("group.id", "IntegrationTest"); + kafkaConsumerProperties.put("group.id", "IntegrationTest-Consumer"); kafkaConsumerProperties.put("value.deserializer", JsonDeserializer.class.getName()); kafkaConsumerProperties.put("key.deserializer", StringDeserializer.class.getName()); kafkaConsumerProperties.put("sasl.mechanism", "PLAIN"); @@ -65,6 +70,21 @@ private Properties defaultConsumerConfig() { return kafkaConsumerProperties; } + private Properties defaultProducerConfig() { + Properties kafkaProducerProperties = new Properties(); + + kafkaProducerProperties.put(ProducerConfig.CLIENT_ID_CONFIG, "IntegrationTest-producer"); + kafkaProducerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + kafkaProducerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonSerializer.class.getName()); + kafkaProducerProperties.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 2000L); + kafkaProducerProperties.put(ProducerConfig.ACKS_CONFIG, "all"); + kafkaProducerProperties.put("sasl.mechanism", "PLAIN"); + kafkaProducerProperties.put("client.dns.lookup", "use_all_dns_ips"); + kafkaProducerProperties.put("session.timeout.ms", "45000"); + + return kafkaProducerProperties; + } + public KafkaCosmosConnectContainer withLocalKafkaContainer(final KafkaContainer kafkaContainer) { withNetwork(kafkaContainer.getNetwork()); @@ -92,6 +112,11 @@ public KafkaCosmosConnectContainer withLocalBootstrapServer(String localBootstra Properties consumerProperties = defaultConsumerConfig(); consumerProperties.put("bootstrap.servers", localBootstrapServer); this.kafkaConsumer = new KafkaConsumer<>(consumerProperties); + + Properties producerProperties = defaultProducerConfig(); + producerProperties.put("bootstrap.servers", localBootstrapServer); + this.kafkaProducer = new KafkaProducer<>(producerProperties); + this.adminClient = this.getAdminClient(localBootstrapServer); return self(); } @@ -104,6 +129,14 @@ public KafkaCosmosConnectContainer withCloudBootstrapServer() { consumerProperties.put("sasl.mechanism", "PLAIN"); this.kafkaConsumer = new KafkaConsumer<>(consumerProperties); + + Properties producerProperties = defaultProducerConfig(); + producerProperties.put("bootstrap.servers", KafkaCosmosTestConfigurations.BOOTSTRAP_SERVER); + producerProperties.put("sasl.jaas.config", KafkaCosmosTestConfigurations.SASL_JAAS); + producerProperties.put("security.protocol", "SASL_SSL"); + producerProperties.put("sasl.mechanism", "PLAIN"); + this.kafkaProducer = new KafkaProducer<>(producerProperties); + this.adminClient = this.getAdminClient(KafkaCosmosTestConfigurations.BOOTSTRAP_SERVER); this.replicationFactor = 3; return self(); @@ -142,6 +175,10 @@ public KafkaConsumer getConsumer() { return this.kafkaConsumer; } + public KafkaProducer getProducer() { + return this.kafkaProducer; + } + public String getTarget() { return "http://" + getContainerIpAddress() + ":" + getMappedPort(KAFKA_CONNECT_PORT); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java index 67b643b4b63df..8e601948a6c31 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java @@ -88,7 +88,7 @@ protected static CosmosContainerProperties getSinglePartitionContainer(CosmosAsy credential = new AzureKeyCredential(KafkaCosmosTestConfigurations.MASTER_KEY); } - @BeforeSuite(groups = { "kafka" }, timeOut = SUITE_SETUP_TIMEOUT) + @BeforeSuite(groups = { "kafka", "kafka-integration" }, timeOut = SUITE_SETUP_TIMEOUT) public static void beforeSuite() { logger.info("beforeSuite Started"); @@ -124,12 +124,15 @@ public static void beforeSuite() { public static void beforeSuiteUnit() { logger.info("beforeSuite for unit tests started"); - databaseName = "KafkaCosmosTest-" + UUID.randomUUID(); - multiPartitionContainerName = UUID.randomUUID().toString(); - singlePartitionContainerName = UUID.randomUUID().toString(); + databaseName = + StringUtils.isEmpty(databaseName) ? "KafkaCosmosTest-" + UUID.randomUUID() : databaseName; + multiPartitionContainerName = + StringUtils.isEmpty(multiPartitionContainerName) ? UUID.randomUUID().toString() : multiPartitionContainerName; + singlePartitionContainerName = + StringUtils.isEmpty(singlePartitionContainerName) ? UUID.randomUUID().toString() : singlePartitionContainerName; } - @AfterSuite(groups = { "kafka" }, timeOut = SUITE_SHUTDOWN_TIMEOUT) + @AfterSuite(groups = { "kafka", "kafka-integration" }, timeOut = SUITE_SHUTDOWN_TIMEOUT) public static void afterSuite() { logger.info("afterSuite Started"); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml new file mode 100644 index 0000000000000..d30dd96be8e4a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml @@ -0,0 +1,35 @@ + + + + + + + + + + + + + + From adc0235587d76328956d89028fd917a4bbaa1654 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 12 Mar 2024 14:44:56 -0700 Subject: [PATCH 11/44] fix test --- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index c19ed713b56eb..725d223192098 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -47,6 +47,8 @@ Licensed under the MIT License. --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect=ALL-UNNAMED --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect.implementation=ALL-UNNAMED --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect.implementation.source=com.fasterxml.jackson.databind,ALL-UNNAMED + --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect.implementation.sink=ALL-UNNAMED + --add-opens com.azure.cosmos.kafka.connect/com.azure.cosmos.kafka.connect.implementation.sink.idStrategy=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.routing=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.caches=ALL-UNNAMED From 3489467f969df9f2726ed52516f231b93e778ec3 Mon Sep 17 00:00:00 2001 From: annie-mac Date: Tue, 12 Mar 2024 15:03:39 -0700 Subject: [PATCH 12/44] remove kafka-integration test suites --- sdk/cosmos/azure-cosmos-kafka-connect/pom.xml | 21 ----------- .../resources/kafka-integration-testng.xml | 35 ------------------- 2 files changed, 56 deletions(-) delete mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index 725d223192098..e89355097659b 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -498,26 +498,5 @@ Licensed under the MIT License. - - - kafka-integration - - kafka-integration - - - - - org.apache.maven.plugins - maven-failsafe-plugin - 3.2.5 - - - src/test/resources/kafka-integration-testng.xml - - - - - - diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml deleted file mode 100644 index d30dd96be8e4a..0000000000000 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/resources/kafka-integration-testng.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - - - - - - - - - - - From 28669160e6952e57f2549c07c275a4ff3cf670d1 Mon Sep 17 00:00:00 2001 From: Shawn Fang <45607042+mssfang@users.noreply.github.com> Date: Tue, 26 Mar 2024 13:48:49 -0700 Subject: [PATCH 13/44] upgrade bom version (#39412) --- sdk/boms/azure-sdk-bom/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/boms/azure-sdk-bom/pom.xml b/sdk/boms/azure-sdk-bom/pom.xml index 04dde96cf0467..4ccc16ebb201f 100644 --- a/sdk/boms/azure-sdk-bom/pom.xml +++ b/sdk/boms/azure-sdk-bom/pom.xml @@ -2,7 +2,7 @@ 4.0.0 com.azure azure-sdk-bom - 1.2.21 + 1.2.22 pom Azure Java SDK BOM (Bill of Materials) Azure Java SDK BOM (Bill of Materials) From f512c9c79f4af0d1f9af4fe066118dca44704bc3 Mon Sep 17 00:00:00 2001 From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com> Date: Tue, 26 Mar 2024 17:26:03 -0400 Subject: [PATCH 14/44] Increment versions for cosmos releases (#39411) * Increment package versions for cosmos releases * Updated azure-cosmos dependency to current --------- Co-authored-by: Kushagra Thapar --- eng/versioning/version_client.txt | 2 +- sdk/cosmos/azure-cosmos-benchmark/pom.xml | 2 +- sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md | 10 ++++++++++ sdk/cosmos/azure-cosmos-encryption/pom.xml | 4 ++-- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index 4b793ee964b19..08afb10916085 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -109,7 +109,7 @@ com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12;4.28.4;4.29.0-beta.1 com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12;4.28.4;4.29.0-beta.1 com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12;4.28.4;4.29.0-beta.1 com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12;4.28.4;4.29.0-beta.1 -com.azure:azure-cosmos-encryption;2.8.0;2.9.0 +com.azure:azure-cosmos-encryption;2.9.0;2.10.0-beta.1 com.azure:azure-cosmos-test;1.0.0-beta.6;1.0.0-beta.7 com.azure:azure-cosmos-tests;1.0.0-beta.1;1.0.0-beta.1 com.azure.cosmos.kafka:azure-cosmos-kafka-connect;1.0.0-beta.1;1.0.0-beta.1 diff --git a/sdk/cosmos/azure-cosmos-benchmark/pom.xml b/sdk/cosmos/azure-cosmos-benchmark/pom.xml index b4c9da6dc97cd..e0c704b43599a 100644 --- a/sdk/cosmos/azure-cosmos-benchmark/pom.xml +++ b/sdk/cosmos/azure-cosmos-benchmark/pom.xml @@ -57,7 +57,7 @@ Licensed under the MIT License. com.azure azure-cosmos-encryption - 2.9.0 + 2.10.0-beta.1 diff --git a/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md b/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md index cac0209b00fad..763f39923c3aa 100644 --- a/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md @@ -1,5 +1,15 @@ ## Release History +### 2.10.0-beta.1 (Unreleased) + +#### Features Added + +#### Breaking Changes + +#### Bugs Fixed + +#### Other Changes + ### 2.9.0 (2024-03-26) #### Other Changes * Updated `azure-cosmos` to version `4.57.0`. diff --git a/sdk/cosmos/azure-cosmos-encryption/pom.xml b/sdk/cosmos/azure-cosmos-encryption/pom.xml index ea1d311c501a5..0ceb980d9e13b 100644 --- a/sdk/cosmos/azure-cosmos-encryption/pom.xml +++ b/sdk/cosmos/azure-cosmos-encryption/pom.xml @@ -13,7 +13,7 @@ Licensed under the MIT License. com.azure azure-cosmos-encryption - 2.9.0 + 2.10.0-beta.1 Encryption Plugin for Azure Cosmos DB SDK This Package contains Encryption Plugin for Microsoft Azure Cosmos SDK jar @@ -57,7 +57,7 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0 + 4.58.0-beta.1 From 10d323b75f4138cbca8b4fe667e6af396233c29a Mon Sep 17 00:00:00 2001 From: Jair Myree Date: Tue, 26 Mar 2024 14:38:35 -0700 Subject: [PATCH 15/44] Autorest Regen to update scope to include non-public clouds (#39224) * Autorest Regen to update scope to include non-public cloud. * Restoring module-info.java * Updating the README.md * Adding `audience` to the Public API via an autorest code-customization * Adding javadoc comments to LogIngestionAudience * Changing method casing * Update sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogIngestionAudience.java Co-authored-by: Srikanta <51379715+srnagar@users.noreply.github.com> * Update sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogIngestionAudience.java Co-authored-by: Srikanta <51379715+srnagar@users.noreply.github.com> * Update sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogIngestionAudience.java Co-authored-by: Srikanta <51379715+srnagar@users.noreply.github.com> * Updates to PR based on comments * Updates to PR based on comments * Updates to PR based on comments * Updates to PR based on comments * Revert module-info.java * Javadoc update * Javadoc update * Javadoc update * Javadoc update * Updates to CHANGELOG.md --------- Co-authored-by: Srikanta <51379715+srnagar@users.noreply.github.com> --- .../azure-monitor-ingestion/CHANGELOG.md | 3 + .../ingestion/LogsIngestionClientBuilder.java | 14 ++ ...onUsingDataCollectionRulesAsyncClient.java | 28 +-- ...gestionUsingDataCollectionRulesClient.java | 34 ++- ...UsingDataCollectionRulesClientBuilder.java | 186 +++++++++------ ...ionUsingDataCollectionRulesClientImpl.java | 224 +++++++----------- ...singDataCollectionRulesServiceVersion.java | 14 +- .../implementation/package-info.java | 5 +- .../models/LogsIngestionAudience.java | 56 +++++ .../src/main/java/module-info.java | 2 +- .../azure-monitor-ingestion/swagger/README.md | 9 +- .../azure-monitor-ingestion/swagger/pom.xml | 30 +++ .../java/MonitorIngestionCustomizations.java | 150 ++++++++++++ 13 files changed, 508 insertions(+), 247 deletions(-) create mode 100644 sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogsIngestionAudience.java create mode 100644 sdk/monitor/azure-monitor-ingestion/swagger/pom.xml create mode 100644 sdk/monitor/azure-monitor-ingestion/swagger/src/main/java/MonitorIngestionCustomizations.java diff --git a/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md b/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md index 53d412567e5e5..30a74593c1f25 100644 --- a/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md +++ b/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md @@ -4,6 +4,9 @@ ### Features Added +- Introduced `LogsIngestionAudience` to allow specification of the audience of logs ingestion clients. +- Support for the scopes of non-public clouds. + ### Breaking Changes ### Bugs Fixed diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/LogsIngestionClientBuilder.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/LogsIngestionClientBuilder.java index 9d27d086e59c7..071e0eb274d3c 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/LogsIngestionClientBuilder.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/LogsIngestionClientBuilder.java @@ -20,6 +20,7 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.monitor.ingestion.implementation.IngestionUsingDataCollectionRulesClientBuilder; import com.azure.monitor.ingestion.implementation.IngestionUsingDataCollectionRulesServiceVersion; +import com.azure.monitor.ingestion.models.LogsIngestionAudience; import java.net.MalformedURLException; import java.net.URL; @@ -171,6 +172,19 @@ public LogsIngestionClientBuilder credential(TokenCredential tokenCredential) { return this; } + + /** + * Sets the audience for the authorization scope of log ingestion clients. If this value is not set, the default + * audience will be the azure public cloud. + * + * @param audience the audience value. + * @return the updated {@link LogsIngestionClientBuilder}. + */ + public LogsIngestionClientBuilder audience(LogsIngestionAudience audience) { + innerLogBuilder.audience(audience); + return this; + } + /** * {@inheritDoc} */ diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesAsyncClient.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesAsyncClient.java index 736c735ca2e4e..6410374635f80 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesAsyncClient.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesAsyncClient.java @@ -17,14 +17,17 @@ import com.azure.core.util.BinaryData; import reactor.core.publisher.Mono; -/** Initializes a new instance of the asynchronous IngestionUsingDataCollectionRulesClient type. */ +/** + * Initializes a new instance of the asynchronous IngestionUsingDataCollectionRulesClient type. + */ @ServiceClient(builder = IngestionUsingDataCollectionRulesClientBuilder.class, isAsync = true) public final class IngestionUsingDataCollectionRulesAsyncClient { - @Generated private final IngestionUsingDataCollectionRulesClientImpl serviceClient; + @Generated + private final IngestionUsingDataCollectionRulesClientImpl serviceClient; /** * Initializes an instance of IngestionUsingDataCollectionRulesAsyncClient class. - * + * * @param serviceClient the service client implementation. */ @Generated @@ -34,28 +37,23 @@ public final class IngestionUsingDataCollectionRulesAsyncClient { /** * Ingestion API used to directly ingest data using Data Collection Rules - * - *

See error response code and error response message for more detail. - * - *

Header Parameters - * + * + * See error response code and error response message for more detail. + *

Header Parameters

* * * * * *
Header Parameters
NameTypeRequiredDescription
Content-EncodingStringNogzip
x-ms-client-request-idStringNoClient request Id
- * * You can add these to a request with {@link RequestOptions#addHeader} - * - *

Request Body Schema - * + *

Request Body Schema

*
{@code
      * [
      *     Object (Required)
      * ]
      * }
- * + * * @param ruleId The immutable Id of the Data Collection Rule resource. * @param stream The streamDeclaration name as defined in the Data Collection Rule. * @param body An array of objects matching the schema defined by the provided stream. @@ -68,8 +66,8 @@ public final class IngestionUsingDataCollectionRulesAsyncClient { */ @Generated @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> uploadWithResponse( - String ruleId, String stream, BinaryData body, RequestOptions requestOptions) { + public Mono> uploadWithResponse(String ruleId, String stream, BinaryData body, + RequestOptions requestOptions) { return this.serviceClient.uploadWithResponseAsync(ruleId, stream, body, requestOptions); } } diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClient.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClient.java index 4fe8cb55eb44b..9af4b9ba8ed33 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClient.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClient.java @@ -16,45 +16,43 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.BinaryData; -/** Initializes a new instance of the synchronous IngestionUsingDataCollectionRulesClient type. */ +/** + * Initializes a new instance of the synchronous IngestionUsingDataCollectionRulesClient type. + */ @ServiceClient(builder = IngestionUsingDataCollectionRulesClientBuilder.class) public final class IngestionUsingDataCollectionRulesClient { - @Generated private final IngestionUsingDataCollectionRulesClientImpl serviceClient; + @Generated + private final IngestionUsingDataCollectionRulesClientImpl serviceClient; /** * Initializes an instance of IngestionUsingDataCollectionRulesClient class. - * - * @param client the async client. + * + * @param serviceClient the service client implementation. */ @Generated - IngestionUsingDataCollectionRulesClient(IngestionUsingDataCollectionRulesClientImpl client) { - this.serviceClient = client; + IngestionUsingDataCollectionRulesClient(IngestionUsingDataCollectionRulesClientImpl serviceClient) { + this.serviceClient = serviceClient; } /** * Ingestion API used to directly ingest data using Data Collection Rules - * - *

See error response code and error response message for more detail. - * - *

Header Parameters - * + * + * See error response code and error response message for more detail. + *

Header Parameters

* * * * * *
Header Parameters
NameTypeRequiredDescription
Content-EncodingStringNogzip
x-ms-client-request-idStringNoClient request Id
- * * You can add these to a request with {@link RequestOptions#addHeader} - * - *

Request Body Schema - * + *

Request Body Schema

*
{@code
      * [
      *     Object (Required)
      * ]
      * }
- * + * * @param ruleId The immutable Id of the Data Collection Rule resource. * @param stream The streamDeclaration name as defined in the Data Collection Rule. * @param body An array of objects matching the schema defined by the provided stream. @@ -67,8 +65,8 @@ public final class IngestionUsingDataCollectionRulesClient { */ @Generated @ServiceMethod(returns = ReturnType.SINGLE) - public Response uploadWithResponse( - String ruleId, String stream, BinaryData body, RequestOptions requestOptions) { + public Response uploadWithResponse(String ruleId, String stream, BinaryData body, + RequestOptions requestOptions) { return this.serviceClient.uploadWithResponse(ruleId, stream, body, requestOptions); } } diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientBuilder.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientBuilder.java index fba9adabadcf0..e42664a704975 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientBuilder.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientBuilder.java @@ -1,9 +1,9 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. - package com.azure.monitor.ingestion.implementation; +import com.azure.monitor.ingestion.models.LogsIngestionAudience; import com.azure.core.annotation.Generated; import com.azure.core.annotation.ServiceClientBuilder; import com.azure.core.client.traits.ConfigurationTrait; @@ -12,6 +12,7 @@ import com.azure.core.client.traits.TokenCredentialTrait; import com.azure.core.credential.TokenCredential; import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaderName; import com.azure.core.http.HttpHeaders; import com.azure.core.http.HttpPipeline; import com.azure.core.http.HttpPipelineBuilder; @@ -20,9 +21,8 @@ import com.azure.core.http.policy.AddHeadersFromContextPolicy; import com.azure.core.http.policy.AddHeadersPolicy; import com.azure.core.http.policy.BearerTokenAuthenticationPolicy; -import com.azure.core.http.policy.CookiePolicy; -import com.azure.core.http.policy.HttpLogOptions; import com.azure.core.http.policy.HttpLoggingPolicy; +import com.azure.core.http.policy.HttpLogOptions; import com.azure.core.http.policy.HttpPipelinePolicy; import com.azure.core.http.policy.HttpPolicyProviders; import com.azure.core.http.policy.RequestIdPolicy; @@ -33,36 +33,44 @@ import com.azure.core.util.Configuration; import com.azure.core.util.CoreUtils; import com.azure.core.util.builder.ClientBuilderUtil; +import com.azure.core.util.logging.ClientLogger; import com.azure.core.util.serializer.JacksonAdapter; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.stream.Collectors; -/** A builder for creating a new instance of the IngestionUsingDataCollectionRulesClient type. */ +/** + * A builder for creating a new instance of the IngestionUsingDataCollectionRulesClient type. + */ @ServiceClientBuilder( - serviceClients = { - IngestionUsingDataCollectionRulesClient.class, - IngestionUsingDataCollectionRulesAsyncClient.class - }) + serviceClients = { + IngestionUsingDataCollectionRulesClient.class, + IngestionUsingDataCollectionRulesAsyncClient.class }) public final class IngestionUsingDataCollectionRulesClientBuilder - implements HttpTrait, - ConfigurationTrait, - TokenCredentialTrait, - EndpointTrait { - @Generated private static final String SDK_NAME = "name"; + implements HttpTrait, + ConfigurationTrait, + TokenCredentialTrait, + EndpointTrait { - @Generated private static final String SDK_VERSION = "version"; + @Generated + private static final String SDK_NAME = "name"; - @Generated private static final String[] DEFAULT_SCOPES = new String[] {"https://monitor.azure.com//.default"}; + @Generated + private static final String SDK_VERSION = "version"; + + @Generated + private static final String[] DEFAULT_SCOPES = new String[] { "https://monitor.azure.com//.default" }; @Generated private static final Map PROPERTIES = CoreUtils.getProperties("azure-monitor-ingestion.properties"); - @Generated private final List pipelinePolicies; + @Generated + private final List pipelinePolicies; - /** Create an instance of the IngestionUsingDataCollectionRulesClientBuilder. */ + /** + * Create an instance of the IngestionUsingDataCollectionRulesClientBuilder. + */ @Generated public IngestionUsingDataCollectionRulesClientBuilder() { this.pipelinePolicies = new ArrayList<>(); @@ -71,12 +79,18 @@ public IngestionUsingDataCollectionRulesClientBuilder() { /* * The HTTP pipeline to send requests through. */ - @Generated private HttpPipeline pipeline; + @Generated + private HttpPipeline pipeline; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder pipeline(HttpPipeline pipeline) { + if (this.pipeline != null && pipeline == null) { + LOGGER.info("HttpPipeline is being set to 'null' when it was previously configured."); + } this.pipeline = pipeline; return this; } @@ -84,9 +98,12 @@ public IngestionUsingDataCollectionRulesClientBuilder pipeline(HttpPipeline pipe /* * The HTTP client used to send the request. */ - @Generated private HttpClient httpClient; + @Generated + private HttpClient httpClient; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder httpClient(HttpClient httpClient) { @@ -97,9 +114,12 @@ public IngestionUsingDataCollectionRulesClientBuilder httpClient(HttpClient http /* * The logging configuration for HTTP requests and responses. */ - @Generated private HttpLogOptions httpLogOptions; + @Generated + private HttpLogOptions httpLogOptions; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder httpLogOptions(HttpLogOptions httpLogOptions) { @@ -110,9 +130,12 @@ public IngestionUsingDataCollectionRulesClientBuilder httpLogOptions(HttpLogOpti /* * The client options such as application ID and custom headers to set on a request. */ - @Generated private ClientOptions clientOptions; + @Generated + private ClientOptions clientOptions; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder clientOptions(ClientOptions clientOptions) { @@ -123,9 +146,12 @@ public IngestionUsingDataCollectionRulesClientBuilder clientOptions(ClientOption /* * The retry options to configure retry policy for failed requests. */ - @Generated private RetryOptions retryOptions; + @Generated + private RetryOptions retryOptions; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder retryOptions(RetryOptions retryOptions) { @@ -133,7 +159,9 @@ public IngestionUsingDataCollectionRulesClientBuilder retryOptions(RetryOptions return this; } - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder addPolicy(HttpPipelinePolicy customPolicy) { @@ -145,9 +173,12 @@ public IngestionUsingDataCollectionRulesClientBuilder addPolicy(HttpPipelinePoli /* * The configuration store that is used during construction of the service client. */ - @Generated private Configuration configuration; + @Generated + private Configuration configuration; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder configuration(Configuration configuration) { @@ -158,9 +189,12 @@ public IngestionUsingDataCollectionRulesClientBuilder configuration(Configuratio /* * The TokenCredential used for authentication. */ - @Generated private TokenCredential tokenCredential; + @Generated + private TokenCredential tokenCredential; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder credential(TokenCredential tokenCredential) { @@ -171,9 +205,12 @@ public IngestionUsingDataCollectionRulesClientBuilder credential(TokenCredential /* * The service endpoint */ - @Generated private String endpoint; + @Generated + private String endpoint; - /** {@inheritDoc}. */ + /** + * {@inheritDoc}. + */ @Generated @Override public IngestionUsingDataCollectionRulesClientBuilder endpoint(String endpoint) { @@ -184,7 +221,8 @@ public IngestionUsingDataCollectionRulesClientBuilder endpoint(String endpoint) /* * Service version */ - @Generated private IngestionUsingDataCollectionRulesServiceVersion serviceVersion; + @Generated + private IngestionUsingDataCollectionRulesServiceVersion serviceVersion; /** * Sets Service version. @@ -193,8 +231,8 @@ public IngestionUsingDataCollectionRulesClientBuilder endpoint(String endpoint) * @return the IngestionUsingDataCollectionRulesClientBuilder. */ @Generated - public IngestionUsingDataCollectionRulesClientBuilder serviceVersion( - IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { + public IngestionUsingDataCollectionRulesClientBuilder + serviceVersion(IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { this.serviceVersion = serviceVersion; return this; } @@ -202,7 +240,8 @@ public IngestionUsingDataCollectionRulesClientBuilder serviceVersion( /* * The retry policy that will attempt to retry failed requests, if applicable. */ - @Generated private RetryPolicy retryPolicy; + @Generated + private RetryPolicy retryPolicy; /** * Sets The retry policy that will attempt to retry failed requests, if applicable. @@ -224,18 +263,17 @@ public IngestionUsingDataCollectionRulesClientBuilder retryPolicy(RetryPolicy re @Generated private IngestionUsingDataCollectionRulesClientImpl buildInnerClient() { HttpPipeline localPipeline = (pipeline != null) ? pipeline : createHttpPipeline(); - IngestionUsingDataCollectionRulesServiceVersion localServiceVersion = - (serviceVersion != null) ? serviceVersion : IngestionUsingDataCollectionRulesServiceVersion.getLatest(); - IngestionUsingDataCollectionRulesClientImpl client = - new IngestionUsingDataCollectionRulesClientImpl( - localPipeline, JacksonAdapter.createDefaultSerializerAdapter(), endpoint, localServiceVersion); + IngestionUsingDataCollectionRulesServiceVersion localServiceVersion + = (serviceVersion != null) ? serviceVersion : IngestionUsingDataCollectionRulesServiceVersion.getLatest(); + IngestionUsingDataCollectionRulesClientImpl client = new IngestionUsingDataCollectionRulesClientImpl( + localPipeline, JacksonAdapter.createDefaultSerializerAdapter(), this.endpoint, localServiceVersion); return client; } @Generated private HttpPipeline createHttpPipeline() { - Configuration buildConfiguration = - (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; + Configuration buildConfiguration + = (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; HttpLogOptions localHttpLogOptions = this.httpLogOptions == null ? new HttpLogOptions() : this.httpLogOptions; ClientOptions localClientOptions = this.clientOptions == null ? new ClientOptions() : this.clientOptions; List policies = new ArrayList<>(); @@ -246,33 +284,30 @@ private HttpPipeline createHttpPipeline() { policies.add(new RequestIdPolicy()); policies.add(new AddHeadersFromContextPolicy()); HttpHeaders headers = new HttpHeaders(); - localClientOptions.getHeaders().forEach(header -> headers.set(header.getName(), header.getValue())); + localClientOptions.getHeaders() + .forEach(header -> headers.set(HttpHeaderName.fromString(header.getName()), header.getValue())); if (headers.getSize() > 0) { policies.add(new AddHeadersPolicy(headers)); } - policies.addAll( - this.pipelinePolicies.stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + this.pipelinePolicies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .forEach(p -> policies.add(p)); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(ClientBuilderUtil.validateAndGetRetryPolicy(retryPolicy, retryOptions, new RetryPolicy())); policies.add(new AddDatePolicy()); - policies.add(new CookiePolicy()); if (tokenCredential != null) { - policies.add(new BearerTokenAuthenticationPolicy(tokenCredential, DEFAULT_SCOPES)); + policies.add(new BearerTokenAuthenticationPolicy(tokenCredential, + audience == null ? DEFAULT_SCOPES : new String[] { audience.toString() })); } - policies.addAll( - this.pipelinePolicies.stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + this.pipelinePolicies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) + .forEach(p -> policies.add(p)); HttpPolicyProviders.addAfterRetryPolicies(policies); - policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .httpClient(httpClient) - .clientOptions(localClientOptions) - .build(); + policies.add(new HttpLoggingPolicy(localHttpLogOptions)); + HttpPipeline httpPipeline = new HttpPipelineBuilder().policies(policies.toArray(new HttpPipelinePolicy[0])) + .httpClient(httpClient) + .clientOptions(localClientOptions) + .build(); return httpPipeline; } @@ -293,7 +328,26 @@ public IngestionUsingDataCollectionRulesAsyncClient buildAsyncClient() { */ @Generated public IngestionUsingDataCollectionRulesClient buildClient() { - return new IngestionUsingDataCollectionRulesClient( - buildInnerClient()); + return new IngestionUsingDataCollectionRulesClient(buildInnerClient()); + } + + private static final ClientLogger LOGGER = new ClientLogger(IngestionUsingDataCollectionRulesClientBuilder.class); + + /** + * The audience indicating the authorization scope of log ingestion clients. + */ + @Generated() + private LogsIngestionAudience audience; + + /** + * Sets The audience. + * + * @param audience the audience indicating the authorization scope of log ingestion clients. + * @return the IngestionUsingDataCollectionRulesClientBuilder. + */ + @Generated() + public IngestionUsingDataCollectionRulesClientBuilder audience(LogsIngestionAudience audience) { + this.audience = audience; + return this; } } diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientImpl.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientImpl.java index f563f013c9cd8..0ae66b24f53ae 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientImpl.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesClientImpl.java @@ -22,7 +22,6 @@ import com.azure.core.exception.ResourceNotFoundException; import com.azure.core.http.HttpPipeline; import com.azure.core.http.HttpPipelineBuilder; -import com.azure.core.http.policy.CookiePolicy; import com.azure.core.http.policy.RetryPolicy; import com.azure.core.http.policy.UserAgentPolicy; import com.azure.core.http.rest.RequestOptions; @@ -36,57 +35,65 @@ import com.azure.core.util.serializer.SerializerAdapter; import reactor.core.publisher.Mono; -/** Initializes a new instance of the IngestionUsingDataCollectionRulesClient type. */ +/** + * Initializes a new instance of the IngestionUsingDataCollectionRulesClient type. + */ public final class IngestionUsingDataCollectionRulesClientImpl { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final IngestionUsingDataCollectionRulesClientService service; /** - * The Data Collection Endpoint for the Data Collection Rule, for example - * https://dce-name.eastus-2.ingest.monitor.azure.com. + * The Data Collection Endpoint for the Data Collection Rule, for example https://dce-name.eastus-2.ingest.monitor.azure.com. */ private final String endpoint; /** - * Gets The Data Collection Endpoint for the Data Collection Rule, for example - * https://dce-name.eastus-2.ingest.monitor.azure.com. - * + * Gets The Data Collection Endpoint for the Data Collection Rule, for example https://dce-name.eastus-2.ingest.monitor.azure.com. + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Service version. */ + /** + * Service version. + */ private final IngestionUsingDataCollectionRulesServiceVersion serviceVersion; /** * Gets Service version. - * + * * @return the serviceVersion value. */ public IngestionUsingDataCollectionRulesServiceVersion getServiceVersion() { return this.serviceVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ public SerializerAdapter getSerializerAdapter() { @@ -95,138 +102,94 @@ public SerializerAdapter getSerializerAdapter() { /** * Initializes an instance of IngestionUsingDataCollectionRulesClient client. - * - * @param endpoint The Data Collection Endpoint for the Data Collection Rule, for example - * https://dce-name.eastus-2.ingest.monitor.azure.com. + * + * @param endpoint The Data Collection Endpoint for the Data Collection Rule, for example https://dce-name.eastus-2.ingest.monitor.azure.com. * @param serviceVersion Service version. */ - public IngestionUsingDataCollectionRulesClientImpl( - String endpoint, IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { - this( - new HttpPipelineBuilder() - .policies(new UserAgentPolicy(), new RetryPolicy(), new CookiePolicy()) - .build(), - JacksonAdapter.createDefaultSerializerAdapter(), - endpoint, - serviceVersion); + IngestionUsingDataCollectionRulesClientImpl(String endpoint, + IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { + this(new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(), + JacksonAdapter.createDefaultSerializerAdapter(), endpoint, serviceVersion); } /** * Initializes an instance of IngestionUsingDataCollectionRulesClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. - * @param endpoint The Data Collection Endpoint for the Data Collection Rule, for example - * https://dce-name.eastus-2.ingest.monitor.azure.com. + * @param endpoint The Data Collection Endpoint for the Data Collection Rule, for example https://dce-name.eastus-2.ingest.monitor.azure.com. * @param serviceVersion Service version. */ - public IngestionUsingDataCollectionRulesClientImpl( - HttpPipeline httpPipeline, - String endpoint, + IngestionUsingDataCollectionRulesClientImpl(HttpPipeline httpPipeline, String endpoint, IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { this(httpPipeline, JacksonAdapter.createDefaultSerializerAdapter(), endpoint, serviceVersion); } /** * Initializes an instance of IngestionUsingDataCollectionRulesClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. - * @param endpoint The Data Collection Endpoint for the Data Collection Rule, for example - * https://dce-name.eastus-2.ingest.monitor.azure.com. + * @param endpoint The Data Collection Endpoint for the Data Collection Rule, for example https://dce-name.eastus-2.ingest.monitor.azure.com. * @param serviceVersion Service version. */ - public IngestionUsingDataCollectionRulesClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - String endpoint, - IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { + IngestionUsingDataCollectionRulesClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + String endpoint, IngestionUsingDataCollectionRulesServiceVersion serviceVersion) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.endpoint = endpoint; this.serviceVersion = serviceVersion; - this.service = - RestProxy.create( - IngestionUsingDataCollectionRulesClientService.class, - this.httpPipeline, - this.getSerializerAdapter()); + this.service = RestProxy.create(IngestionUsingDataCollectionRulesClientService.class, this.httpPipeline, + this.getSerializerAdapter()); } /** - * The interface defining all the services for IngestionUsingDataCollectionRulesClient to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for IngestionUsingDataCollectionRulesClient to be used by the proxy service to perform REST calls. */ @Host("{endpoint}") @ServiceInterface(name = "IngestionUsingDataCo") public interface IngestionUsingDataCollectionRulesClientService { @Post("/dataCollectionRules/{ruleId}/streams/{stream}") - @ExpectedResponses({204}) - @UnexpectedResponseExceptionType( - value = ClientAuthenticationException.class, - code = {401}) - @UnexpectedResponseExceptionType( - value = ResourceNotFoundException.class, - code = {404}) - @UnexpectedResponseExceptionType( - value = ResourceModifiedException.class, - code = {409}) + @ExpectedResponses({ 204 }) + @UnexpectedResponseExceptionType(value = ClientAuthenticationException.class, code = { 401 }) + @UnexpectedResponseExceptionType(value = ResourceNotFoundException.class, code = { 404 }) + @UnexpectedResponseExceptionType(value = ResourceModifiedException.class, code = { 409 }) @UnexpectedResponseExceptionType(HttpResponseException.class) - Mono> upload( - @HostParam("endpoint") String endpoint, - @PathParam("ruleId") String ruleId, - @PathParam("stream") String stream, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") BinaryData body, - @HeaderParam("Accept") String accept, - RequestOptions requestOptions, - Context context); + Mono> upload(@HostParam("endpoint") String endpoint, @PathParam("ruleId") String ruleId, + @PathParam("stream") String stream, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") BinaryData body, @HeaderParam("Accept") String accept, + RequestOptions requestOptions, Context context); @Post("/dataCollectionRules/{ruleId}/streams/{stream}") - @ExpectedResponses({204}) - @UnexpectedResponseExceptionType( - value = ClientAuthenticationException.class, - code = {401}) - @UnexpectedResponseExceptionType( - value = ResourceNotFoundException.class, - code = {404}) - @UnexpectedResponseExceptionType( - value = ResourceModifiedException.class, - code = {409}) + @ExpectedResponses({ 204 }) + @UnexpectedResponseExceptionType(value = ClientAuthenticationException.class, code = { 401 }) + @UnexpectedResponseExceptionType(value = ResourceNotFoundException.class, code = { 404 }) + @UnexpectedResponseExceptionType(value = ResourceModifiedException.class, code = { 409 }) @UnexpectedResponseExceptionType(HttpResponseException.class) - Response uploadSync( - @HostParam("endpoint") String endpoint, - @PathParam("ruleId") String ruleId, - @PathParam("stream") String stream, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") BinaryData body, - @HeaderParam("Accept") String accept, - RequestOptions requestOptions, - Context context); + Response uploadSync(@HostParam("endpoint") String endpoint, @PathParam("ruleId") String ruleId, + @PathParam("stream") String stream, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") BinaryData body, @HeaderParam("Accept") String accept, + RequestOptions requestOptions, Context context); } /** * Ingestion API used to directly ingest data using Data Collection Rules - * - *

See error response code and error response message for more detail. - * - *

Header Parameters - * + * + * See error response code and error response message for more detail. + *

Header Parameters

* * * * * *
Header Parameters
NameTypeRequiredDescription
Content-EncodingStringNogzip
x-ms-client-request-idStringNoClient request Id
- * * You can add these to a request with {@link RequestOptions#addHeader} - * - *

Request Body Schema - * + *

Request Body Schema

*
{@code
      * [
      *     Object (Required)
      * ]
      * }
- * + * * @param ruleId The immutable Id of the Data Collection Rule resource. * @param stream The streamDeclaration name as defined in the Data Collection Rule. * @param body An array of objects matching the schema defined by the provided stream. @@ -238,58 +201,44 @@ Response uploadSync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> uploadWithResponseAsync( - String ruleId, String stream, BinaryData body, RequestOptions requestOptions) { + public Mono> uploadWithResponseAsync(String ruleId, String stream, BinaryData body, + RequestOptions requestOptions) { if (ruleId == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); } if (stream == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Parameter stream is required and cannot be null.")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Parameter stream is required and cannot be null.")); } if (body == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Parameter body is required and cannot be null.")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Parameter body is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil.withContext( - context -> - service.upload( - this.getEndpoint(), - ruleId, - stream, - this.getServiceVersion().getVersion(), - body, - accept, - requestOptions, - context)); + return FluxUtil.withContext(context -> service.upload(this.getEndpoint(), ruleId, stream, + this.getServiceVersion().getVersion(), body, accept, requestOptions, context)); } /** * Ingestion API used to directly ingest data using Data Collection Rules - * - *

See error response code and error response message for more detail. - * - *

Header Parameters - * + * + * See error response code and error response message for more detail. + *

Header Parameters

* * * * * *
Header Parameters
NameTypeRequiredDescription
Content-EncodingStringNogzip
x-ms-client-request-idStringNoClient request Id
- * * You can add these to a request with {@link RequestOptions#addHeader} - * - *

Request Body Schema - * + *

Request Body Schema

*
{@code
      * [
      *     Object (Required)
      * ]
      * }
- * + * * @param ruleId The immutable Id of the Data Collection Rule resource. * @param stream The streamDeclaration name as defined in the Data Collection Rule. * @param body An array of objects matching the schema defined by the provided stream. @@ -301,30 +250,23 @@ public Mono> uploadWithResponseAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response uploadWithResponse( - String ruleId, String stream, BinaryData body, RequestOptions requestOptions) { + public Response uploadWithResponse(String ruleId, String stream, BinaryData body, + RequestOptions requestOptions) { if (ruleId == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); } if (stream == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Parameter stream is required and cannot be null.")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Parameter stream is required and cannot be null.")); } if (body == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Parameter body is required and cannot be null.")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Parameter body is required and cannot be null.")); } final String accept = "application/json"; - return service.uploadSync( - this.getEndpoint(), - ruleId, - stream, - this.getServiceVersion().getVersion(), - body, - accept, - requestOptions, - requestOptions.getContext()); + return service.uploadSync(this.getEndpoint(), ruleId, stream, this.getServiceVersion().getVersion(), body, + accept, requestOptions, Context.NONE); } private static final ClientLogger LOGGER = new ClientLogger(IngestionUsingDataCollectionRulesClientImpl.class); diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesServiceVersion.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesServiceVersion.java index 706e1f3a03402..1a6e5ef84d752 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesServiceVersion.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/IngestionUsingDataCollectionRulesServiceVersion.java @@ -6,9 +6,13 @@ import com.azure.core.util.ServiceVersion; -/** Service version of IngestionUsingDataCollectionRulesClient. */ +/** + * Service version of IngestionUsingDataCollectionRulesClient. + */ public enum IngestionUsingDataCollectionRulesServiceVersion implements ServiceVersion { - /** Enum value 2023-01-01. */ + /** + * Enum value 2023-01-01. + */ V2023_01_01("2023-01-01"); private final String version; @@ -17,7 +21,9 @@ public enum IngestionUsingDataCollectionRulesServiceVersion implements ServiceVe this.version = version; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public String getVersion() { return this.version; @@ -25,7 +31,7 @@ public String getVersion() { /** * Gets the latest service version supported by this client library. - * + * * @return The latest {@link IngestionUsingDataCollectionRulesServiceVersion}. */ public static IngestionUsingDataCollectionRulesServiceVersion getLatest() { diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/package-info.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/package-info.java index f4ceffac4b6b5..e7584cdc91ee6 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/package-info.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for IngestionUsingDataCollectionRules. null. */ +/** + * Package containing the classes for IngestionUsingDataCollectionRules. + * null. + */ package com.azure.monitor.ingestion.implementation; diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogsIngestionAudience.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogsIngestionAudience.java new file mode 100644 index 0000000000000..4b3ed2c370b97 --- /dev/null +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/com/azure/monitor/ingestion/models/LogsIngestionAudience.java @@ -0,0 +1,56 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.monitor.ingestion.models; + +import com.azure.core.util.ExpandableStringEnum; + +import java.util.Collection; + +/** + * The audience indicating the authorization scope of log ingestion clients. + */ +public class LogsIngestionAudience extends ExpandableStringEnum { + + /** + * Static value for Azure Public Cloud. + */ + public static final LogsIngestionAudience AZURE_PUBLIC_CLOUD = fromString("https://monitor.azure.com//.default"); + + /** + * Static value for Azure US Government. + */ + private static final LogsIngestionAudience AZURE_GOVERNMENT = fromString("https://monitor.azure.us//.default"); + + /** + * Static value for Azure China. + */ + private static final LogsIngestionAudience AZURE_CHINA = fromString("https://monitor.azure.cn//.default"); + + /** + * @deprecated Creates an instance of LogsIngestionAudience. + */ + @Deprecated + LogsIngestionAudience() { + } + + /** + * Creates an instance of LogsIngestionAudience. + * + * @param name the string value. + * @return the LogsIngestionAudience. + */ + public static LogsIngestionAudience fromString(String name) { + return fromString(name, LogsIngestionAudience.class); + } + + /** + * Get the collection of LogsIngestionAudience values. + * + * @return the collection of LogsIngestionAudience values. + */ + public static Collection values() { + return values(LogsIngestionAudience.class); + } + +} diff --git a/sdk/monitor/azure-monitor-ingestion/src/main/java/module-info.java b/sdk/monitor/azure-monitor-ingestion/src/main/java/module-info.java index d392257ae5dd0..2b4903531d628 100644 --- a/sdk/monitor/azure-monitor-ingestion/src/main/java/module-info.java +++ b/sdk/monitor/azure-monitor-ingestion/src/main/java/module-info.java @@ -1,9 +1,9 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. module com.azure.monitor.ingestion { requires transitive com.azure.core; - exports com.azure.monitor.ingestion; exports com.azure.monitor.ingestion.models; diff --git a/sdk/monitor/azure-monitor-ingestion/swagger/README.md b/sdk/monitor/azure-monitor-ingestion/swagger/README.md index 4610d90a08391..b42ec32ab9824 100644 --- a/sdk/monitor/azure-monitor-ingestion/swagger/README.md +++ b/sdk/monitor/azure-monitor-ingestion/swagger/README.md @@ -2,13 +2,19 @@ ## Code generation settings +### Manual Modifications + +The following edits need to be made manually after code generation: +- Rollback the edits to `module-info` file + ```yaml java: true -use: '@autorest/java@4.1.9' +use: '@autorest/java@4.26.2' output-folder: ../ license-header: MICROSOFT_MIT_SMALL input-file: https://github.com/Azure/azure-rest-api-specs/blob/main/specification/monitor/data-plane/ingestion/stable/2023-01-01/DataCollectionRules.json namespace: com.azure.monitor.ingestion.implementation +implementation-subpackage: "" generate-client-interfaces: false sync-methods: all add-context-parameter: true @@ -21,4 +27,5 @@ client-side-validations: true artifact-id: azure-monitor-ingestion data-plane: true enable-sync-stack: true +customization-class: src/main/java/MonitorIngestionCustomizations.java ``` diff --git a/sdk/monitor/azure-monitor-ingestion/swagger/pom.xml b/sdk/monitor/azure-monitor-ingestion/swagger/pom.xml new file mode 100644 index 0000000000000..8876e3dcaf2d0 --- /dev/null +++ b/sdk/monitor/azure-monitor-ingestion/swagger/pom.xml @@ -0,0 +1,30 @@ + + + 4.0.0 + + Microsoft Azure Monitor Ingestion client for Java + This package contains client functionality for Microsoft Monitor Ingestion + + com.azure.tools + azure-monitor-ingestion-autorest-customization + 1.0.0-beta.1 + jar + + + com.azure + azure-code-customization-parent + 1.0.0-beta.1 + ../../../parents/azure-code-customization-parent + + + + + com.azure.tools + azure-autorest-customization + 1.0.0-beta.8 + + + + diff --git a/sdk/monitor/azure-monitor-ingestion/swagger/src/main/java/MonitorIngestionCustomizations.java b/sdk/monitor/azure-monitor-ingestion/swagger/src/main/java/MonitorIngestionCustomizations.java new file mode 100644 index 0000000000000..fec434fbc6470 --- /dev/null +++ b/sdk/monitor/azure-monitor-ingestion/swagger/src/main/java/MonitorIngestionCustomizations.java @@ -0,0 +1,150 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +import com.azure.autorest.customization.ClassCustomization; +import com.azure.autorest.customization.Customization; +import com.azure.autorest.customization.LibraryCustomization; +import com.azure.autorest.customization.PackageCustomization; +import com.github.javaparser.ast.body.ClassOrInterfaceDeclaration; +import com.github.javaparser.ast.body.TypeDeclaration; +import com.github.javaparser.ast.stmt.BlockStmt; +import org.slf4j.Logger; + +import java.util.function.Consumer; + +/** + * Customization class for Monitor. These customizations will be applied on top of the generated code. + */ +public class MonitorIngestionCustomizations extends Customization { + + /** + * Customizes the generated code. + * + *
+ * + * The following customizations are applied: + * + *
    + *
  1. The package customization for the package `com.azure.monitor.ingestion.implementation`.
  2. + *
+ * + * @param libraryCustomization The library customization. + * @param logger The logger. + */ + @Override + public void customize(LibraryCustomization libraryCustomization, Logger logger) { + monitorIngestionImplementation(libraryCustomization.getPackage("com.azure.monitor.ingestion.implementation"), logger); + } + + /** + * Customizes the generated code for the package com.azure.monitor.ingestion.implementation. + * + *
+ * + * The following classes are customized: + *
    + *
  1. IngestionUsingDataCollectionRulesClientBuilder
  2. + *
+ * + * @param packageCustomization The package customization. + * @param logger The logger. + */ + private void monitorIngestionImplementation(PackageCustomization packageCustomization, Logger logger) { + IngestionUsingDataCollectionRulesClientBuilderCustomization(packageCustomization.getClass("IngestionUsingDataCollectionRulesClientBuilder"), logger); + } + + /** + * Customizes the generated code for `IngestionUsingDataCollectionRulesClientBuilder`. + * + *
+ * + * The following customizations are applied: + * + *
    + *
  1. Adds an import statement for the class `LogsIngestionAudience`.
  2. + *
  3. Adds a field `audience` of type `LogsIngestionAudience` to the class.
  4. + *
  5. Adds a Javadoc for the field `audience`.
  6. + *
  7. Adds the generated annotation to the field `audience`.
  8. + *
  9. Adds a setter for the field `audience`.
  10. + *
  11. Adds a Javadoc for the setter.
  12. + *
  13. Adds the generated annotation to the setter.
  14. + *
  15. Replaces the body of the method `createHttpPipeline()` with a custom implementation that sets the + * audience in the `BearerTokenAuthenticationPolicy`.
  16. + *
+ * + * @param classCustomization The class customization. + * @param logger The logger. + */ + private void IngestionUsingDataCollectionRulesClientBuilderCustomization(ClassCustomization classCustomization, Logger logger) { + classCustomization.addImports("com.azure.monitor.ingestion.models.LogsIngestionAudience"); + + + + customizeAst(classCustomization, clazz -> { + clazz.addPrivateField("LogsIngestionAudience", "audience") + .addAnnotation("Generated") + .setJavadocComment("The audience indicating the authorization scope of log ingestion clients.") + .createSetter() + .setName("audience") + .setType("IngestionUsingDataCollectionRulesClientBuilder") + .setBody(new BlockStmt() + .addStatement("this.audience = audience;") + .addStatement("return this;")) + .addAnnotation("Generated") + .setJavadocComment("Sets The audience.\n" + + " *\n" + + " * @param audience the audience indicating the authorization scope of log ingestion clients.\n" + + " * @return the IngestionUsingDataCollectionRulesClientBuilder."); + }); + + + classCustomization.getMethod("createHttpPipeline").replaceBody("Configuration buildConfiguration\n" + + " = (configuration == null) ? Configuration.getGlobalConfiguration() : configuration;\n" + + " HttpLogOptions localHttpLogOptions = this.httpLogOptions == null ? new HttpLogOptions() : this.httpLogOptions;\n" + + " ClientOptions localClientOptions = this.clientOptions == null ? new ClientOptions() : this.clientOptions;\n" + + " List policies = new ArrayList<>();\n" + + " String clientName = PROPERTIES.getOrDefault(SDK_NAME, \"UnknownName\");\n" + + " String clientVersion = PROPERTIES.getOrDefault(SDK_VERSION, \"UnknownVersion\");\n" + + " String applicationId = CoreUtils.getApplicationId(localClientOptions, localHttpLogOptions);\n" + + " policies.add(new UserAgentPolicy(applicationId, clientName, clientVersion, buildConfiguration));\n" + + " policies.add(new RequestIdPolicy());\n" + + " policies.add(new AddHeadersFromContextPolicy());\n" + + " HttpHeaders headers = new HttpHeaders();\n" + + " localClientOptions.getHeaders()\n" + + " .forEach(header -> headers.set(HttpHeaderName.fromString(header.getName()), header.getValue()));\n" + + " if (headers.getSize() > 0) {\n" + + " policies.add(new AddHeadersPolicy(headers));\n" + + " }\n" + + " this.pipelinePolicies.stream()\n" + + " .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL)\n" + + " .forEach(p -> policies.add(p));\n" + + " HttpPolicyProviders.addBeforeRetryPolicies(policies);\n" + + " policies.add(ClientBuilderUtil.validateAndGetRetryPolicy(retryPolicy, retryOptions, new RetryPolicy()));\n" + + " policies.add(new AddDatePolicy());\n" + + " if (tokenCredential != null) {\n" + + " policies.add(new BearerTokenAuthenticationPolicy(tokenCredential, audience == null ? DEFAULT_SCOPES : new String[] { audience.toString() }));\n" + + " }\n" + + " this.pipelinePolicies.stream()\n" + + " .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY)\n" + + " .forEach(p -> policies.add(p));\n" + + " HttpPolicyProviders.addAfterRetryPolicies(policies);\n" + + " policies.add(new HttpLoggingPolicy(localHttpLogOptions));\n" + + " HttpPipeline httpPipeline = new HttpPipelineBuilder().policies(policies.toArray(new HttpPipelinePolicy[0]))\n" + + " .httpClient(httpClient)\n" + + " .clientOptions(localClientOptions)\n" + + " .build();\n" + + " return httpPipeline;"); + + } + + + /** + * Customizes the abstract syntax tree of a class. + * @param classCustomization The class customization. + * @param consumer The consumer. + */ + private static void customizeAst(ClassCustomization classCustomization, Consumer consumer) { + classCustomization.customizeAst(ast -> consumer.accept(ast.getClassByName(classCustomization.getClassName()) + .orElseThrow(() -> new RuntimeException("Class not found. " + classCustomization.getClassName())))); + } +} From 056e0d9e6cf6408117fc28dd3a3bef8ea0489b84 Mon Sep 17 00:00:00 2001 From: Srikanta <51379715+srnagar@users.noreply.github.com> Date: Tue, 26 Mar 2024 17:22:08 -0700 Subject: [PATCH 16/44] Azure Monitor Query: Prepare for GA release (#39413) --- eng/versioning/version_client.txt | 2 +- sdk/monitor/azure-monitor-query-perf/pom.xml | 2 +- sdk/monitor/azure-monitor-query/CHANGELOG.md | 8 ++------ sdk/monitor/azure-monitor-query/README.md | 2 +- sdk/monitor/azure-monitor-query/pom.xml | 2 +- 5 files changed, 6 insertions(+), 10 deletions(-) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index 08afb10916085..f86b6f8a3e9f6 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -165,7 +165,7 @@ com.azure:azure-mixedreality-remoterendering;1.1.27;1.2.0-beta.1 com.azure:azure-monitor-opentelemetry-exporter;1.0.0-beta.21;1.0.0-beta.22 com.azure:azure-monitor-ingestion;1.1.5;1.2.0-beta.1 com.azure:azure-monitor-ingestion-perf;1.0.0-beta.1;1.0.0-beta.1 -com.azure:azure-monitor-query;1.2.10;1.3.0-beta.3 +com.azure:azure-monitor-query;1.2.10;1.3.0 com.azure:azure-monitor-query-perf;1.0.0-beta.1;1.0.0-beta.1 com.azure:azure-perf-test-parent;1.0.0-beta.1;1.0.0-beta.1 com.azure:azure-quantum-jobs;1.0.0-beta.1;1.0.0-beta.2 diff --git a/sdk/monitor/azure-monitor-query-perf/pom.xml b/sdk/monitor/azure-monitor-query-perf/pom.xml index 796f03fd124c7..0120f9f2e3940 100644 --- a/sdk/monitor/azure-monitor-query-perf/pom.xml +++ b/sdk/monitor/azure-monitor-query-perf/pom.xml @@ -31,7 +31,7 @@ com.azure azure-monitor-query - 1.3.0-beta.3 + 1.3.0 com.azure diff --git a/sdk/monitor/azure-monitor-query/CHANGELOG.md b/sdk/monitor/azure-monitor-query/CHANGELOG.md index 4502c7202cbfe..c7946f7748e60 100644 --- a/sdk/monitor/azure-monitor-query/CHANGELOG.md +++ b/sdk/monitor/azure-monitor-query/CHANGELOG.md @@ -1,19 +1,16 @@ # Release History -## 1.3.0-beta.3 (Unreleased) +## 1.3.0 (2024-03-26) ### Features Added -### Breaking Changes +- Added `MetricsClient` and `MetricsAsyncClient` to support querying metrics for multiple resources in a single request. ### Bugs Fixed - Fixed the issue with `MetricsQueryClient` and `MetricsQueryAsyncClient` where the `listMetricDefinitions` method was returning `MetricsDefinition` objects with null values for `supportedAggregationTypes`.[(#36698)](https://github.com/Azure/azure-sdk-for-java/issues/36698) -### Other Changes - - ## 1.2.10 (2024-03-20) ### Other Changes @@ -23,7 +20,6 @@ - Upgraded `azure-core` from `1.46.0` to version `1.47.0`. - Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. - ## 1.2.9 (2024-02-20) ### Other Changes diff --git a/sdk/monitor/azure-monitor-query/README.md b/sdk/monitor/azure-monitor-query/README.md index e506e2f9c2cc0..c186fc3caeee2 100644 --- a/sdk/monitor/azure-monitor-query/README.md +++ b/sdk/monitor/azure-monitor-query/README.md @@ -66,7 +66,7 @@ add the direct dependency to your project as follows. com.azure azure-monitor-query - 1.3.0-beta.2 + 1.3.0 ``` diff --git a/sdk/monitor/azure-monitor-query/pom.xml b/sdk/monitor/azure-monitor-query/pom.xml index f053812ffff10..4875dc93a8c39 100644 --- a/sdk/monitor/azure-monitor-query/pom.xml +++ b/sdk/monitor/azure-monitor-query/pom.xml @@ -11,7 +11,7 @@ com.azure azure-monitor-query - 1.3.0-beta.3 + 1.3.0 Microsoft Azure SDK for Azure Monitor Logs and Metrics Query This package contains the Microsoft Azure SDK for querying Azure Monitor's Logs and Metrics data sources. From 3fae54e0e66bb3686bd5a19d48199d4e21738681 Mon Sep 17 00:00:00 2001 From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com> Date: Tue, 26 Mar 2024 22:31:53 -0400 Subject: [PATCH 17/44] [Automation] Generate Fluent Lite from sphere#package-2024-04-01 (#39401) * [Automation] External Change * [Automation] Generate Fluent Lite from sphere#package-2024-04-01 --- eng/versioning/version_client.txt | 2 +- .../azure-resourcemanager-sphere/CHANGELOG.md | 304 ++- .../azure-resourcemanager-sphere/README.md | 8 +- .../azure-resourcemanager-sphere/SAMPLE.md | 737 +++--- .../azure-resourcemanager-sphere/pom.xml | 6 +- .../sphere/AzureSphereManager.java | 107 +- ...Client.java => AzureSphereMgmtClient.java} | 32 +- .../sphere/fluent/CatalogsClient.java | 183 +- .../sphere/fluent/CertificatesClient.java | 52 +- .../sphere/fluent/DeploymentsClient.java | 147 +- .../sphere/fluent/DeviceGroupsClient.java | 183 +- .../sphere/fluent/DevicesClient.java | 204 +- .../sphere/fluent/ImagesClient.java | 78 +- .../sphere/fluent/OperationsClient.java | 16 +- .../sphere/fluent/ProductsClient.java | 96 +- .../sphere/fluent/models/CatalogInner.java | 58 +- .../models/CertificateChainResponseInner.java | 12 +- .../fluent/models/CertificateInner.java | 108 +- ...er.java => CountDevicesResponseInner.java} | 20 +- .../sphere/fluent/models/DeploymentInner.java | 95 +- .../fluent/models/DeviceGroupInner.java | 166 +- .../sphere/fluent/models/DeviceInner.java | 109 +- .../fluent/models/DeviceInsightInner.java | 80 +- .../sphere/fluent/models/ImageInner.java | 156 +- .../sphere/fluent/models/OperationInner.java | 22 +- .../sphere/fluent/models/ProductInner.java | 63 +- .../ProofOfPossessionNonceResponseInner.java | 11 +- .../SignedCapabilityImageResponseInner.java | 12 +- .../sphere/fluent/models/package-info.java | 5 +- .../sphere/fluent/package-info.java | 5 +- ...java => AzureSphereMgmtClientBuilder.java} | 78 +- ...pl.java => AzureSphereMgmtClientImpl.java} | 149 +- .../sphere/implementation/CatalogImpl.java | 122 +- .../implementation/CatalogsClientImpl.java | 2107 +++++++---------- .../sphere/implementation/CatalogsImpl.java | 217 +- .../CertificateChainResponseImpl.java | 4 +- .../implementation/CertificateImpl.java | 36 +- .../CertificatesClientImpl.java | 585 ++--- .../implementation/CertificatesImpl.java | 90 +- ...mpl.java => CountDevicesResponseImpl.java} | 14 +- .../sphere/implementation/DeploymentImpl.java | 130 +- .../implementation/DeploymentsClientImpl.java | 974 +++----- .../implementation/DeploymentsImpl.java | 261 +- .../implementation/DeviceGroupImpl.java | 171 +- .../DeviceGroupsClientImpl.java | 1332 ++++------- .../implementation/DeviceGroupsImpl.java | 237 +- .../sphere/implementation/DeviceImpl.java | 168 +- .../implementation/DeviceInsightImpl.java | 4 +- .../implementation/DevicesClientImpl.java | 1362 ++++------- .../sphere/implementation/DevicesImpl.java | 271 +-- .../sphere/implementation/ImageImpl.java | 105 +- .../implementation/ImagesClientImpl.java | 692 ++---- .../sphere/implementation/ImagesImpl.java | 131 +- .../implementation/OperationsClientImpl.java | 177 +- .../sphere/implementation/OperationsImpl.java | 8 +- .../sphere/implementation/ProductImpl.java | 89 +- .../implementation/ProductsClientImpl.java | 1082 +++------ .../sphere/implementation/ProductsImpl.java | 167 +- .../ProofOfPossessionNonceResponseImpl.java | 3 +- .../{Utils.java => ResourceManagerUtils.java} | 45 +- .../SignedCapabilityImageResponseImpl.java | 3 +- .../sphere/implementation/package-info.java | 5 +- .../sphere/models/ActionType.java | 14 +- .../models/AllowCrashDumpCollection.java | 18 +- .../sphere/models/CapabilityType.java | 18 +- .../sphere/models/Catalog.java | 183 +- .../sphere/models/CatalogListResult.java | 34 +- .../models/CatalogProperties.java | 30 +- .../sphere/models/CatalogUpdate.java | 14 +- .../sphere/models/Catalogs.java | 126 +- .../sphere/models/Certificate.java | 71 +- .../models/CertificateChainResponse.java | 8 +- .../sphere/models/CertificateListResult.java | 34 +- .../models/CertificateProperties.java | 28 +- .../sphere/models/CertificateStatus.java | 26 +- .../sphere/models/Certificates.java | 52 +- .../sphere/models/ClaimDevicesRequest.java | 20 +- .../sphere/models/CountDeviceResponse.java | 36 +- .../sphere/models/CountDevicesResponse.java | 26 + .../sphere/models/CountElementsResponse.java | 14 +- .../sphere/models/Deployment.java | 119 +- .../sphere/models/DeploymentListResult.java | 34 +- .../models/DeploymentProperties.java | 26 +- .../sphere/models/Deployments.java | 80 +- .../resourcemanager/sphere/models/Device.java | 155 +- .../sphere/models/DeviceGroup.java | 250 +- .../sphere/models/DeviceGroupListResult.java | 34 +- .../models/DeviceGroupProperties.java | 41 +- .../sphere/models/DeviceGroupUpdate.java | 139 +- .../models/DeviceGroupUpdateProperties.java | 40 +- .../sphere/models/DeviceGroups.java | 76 +- .../sphere/models/DeviceInsight.java | 22 +- .../sphere/models/DeviceListResult.java | 34 +- .../{fluent => }/models/DeviceProperties.java | 29 +- .../sphere/models/DeviceUpdate.java | 47 +- .../models/DeviceUpdateProperties.java | 16 +- .../sphere/models/Devices.java | 83 +- .../GenerateCapabilityImageRequest.java | 20 +- .../resourcemanager/sphere/models/Image.java | 164 +- .../sphere/models/ImageListResult.java | 34 +- .../{fluent => }/models/ImageProperties.java | 39 +- .../sphere/models/ImageType.java | 106 +- .../resourcemanager/sphere/models/Images.java | 44 +- .../models/ListDeviceGroupsRequest.java | 14 +- .../sphere/models/OSFeedType.java | 18 +- .../sphere/models/Operation.java | 16 +- .../sphere/models/OperationDisplay.java | 18 +- .../sphere/models/OperationListResult.java | 14 +- .../sphere/models/Operations.java | 16 +- .../resourcemanager/sphere/models/Origin.java | 18 +- .../sphere/models/PagedDeviceInsight.java | 34 +- .../sphere/models/Product.java | 115 +- .../sphere/models/ProductListResult.java | 34 +- .../models/ProductProperties.java | 29 +- .../sphere/models/ProductUpdate.java | 47 +- .../models/ProductUpdateProperties.java | 16 +- .../sphere/models/Products.java | 52 +- .../models/ProofOfPossessionNonceRequest.java | 20 +- .../ProofOfPossessionNonceResponse.java | 20 +- .../sphere/models/ProvisioningState.java | 38 +- .../sphere/models/RegionalDataBoundary.java | 18 +- .../models/SignedCapabilityImageResponse.java | 8 +- .../sphere/models/UpdatePolicy.java | 18 +- .../sphere/models/package-info.java | 5 +- .../resourcemanager/sphere/package-info.java | 5 +- .../src/main/java/module-info.java | 10 +- .../proxy-config.json | 1 + .../reflect-config.json | 266 +++ .../CatalogsCountDevicesSamples.java | 10 +- .../CatalogsCreateOrUpdateSamples.java | 15 +- .../generated/CatalogsDeleteSamples.java | 9 +- .../CatalogsGetByResourceGroupSamples.java | 14 +- .../CatalogsListByResourceGroupSamples.java | 9 +- .../CatalogsListDeploymentsSamples.java | 15 +- .../CatalogsListDeviceGroupsSamples.java | 23 +- .../CatalogsListDeviceInsightsSamples.java | 15 +- .../generated/CatalogsListDevicesSamples.java | 15 +- .../sphere/generated/CatalogsListSamples.java | 9 +- .../generated/CatalogsUpdateSamples.java | 17 +- .../generated/CatalogsUploadImageSamples.java | 29 + .../generated/CertificatesGetSamples.java | 14 +- .../CertificatesListByCatalogSamples.java | 14 +- .../CertificatesRetrieveCertChainSamples.java | 15 +- ...RetrieveProofOfPossessionNonceSamples.java | 24 +- .../DeploymentsCreateOrUpdateSamples.java | 16 +- .../generated/DeploymentsDeleteSamples.java | 20 +- .../generated/DeploymentsGetSamples.java | 20 +- .../DeploymentsListByDeviceGroupSamples.java | 23 +- .../DeviceGroupsClaimDevicesSamples.java | 26 +- .../DeviceGroupsCountDevicesSamples.java | 15 +- .../DeviceGroupsCreateOrUpdateSamples.java | 19 +- .../generated/DeviceGroupsDeleteSamples.java | 14 +- .../generated/DeviceGroupsGetSamples.java | 15 +- .../DeviceGroupsListByProductSamples.java | 22 +- .../generated/DeviceGroupsUpdateSamples.java | 17 +- .../DevicesCreateOrUpdateSamples.java | 18 +- .../generated/DevicesDeleteSamples.java | 21 +- ...DevicesGenerateCapabilityImageSamples.java | 25 +- .../sphere/generated/DevicesGetSamples.java | 21 +- .../DevicesListByDeviceGroupSamples.java | 15 +- .../generated/DevicesUpdateSamples.java | 24 +- .../ImagesCreateOrUpdateSamples.java | 18 +- .../sphere/generated/ImagesDeleteSamples.java | 12 +- .../sphere/generated/ImagesGetSamples.java | 14 +- .../generated/ImagesListByCatalogSamples.java | 14 +- .../generated/OperationsListSamples.java | 9 +- .../ProductsCountDevicesSamples.java | 15 +- .../ProductsCreateOrUpdateSamples.java | 9 +- .../generated/ProductsDeleteSamples.java | 9 +- ...ctsGenerateDefaultDeviceGroupsSamples.java | 19 +- .../sphere/generated/ProductsGetSamples.java | 14 +- .../ProductsListByCatalogSamples.java | 9 +- .../generated/ProductsUpdateSamples.java | 17 +- .../sphere/generated/CatalogInnerTests.java | 34 +- .../generated/CatalogListResultTests.java | 34 +- .../generated/CatalogPropertiesTests.java | 6 +- .../sphere/generated/CatalogUpdateTests.java | 19 +- ...logsCountDevicesWithResponseMockTests.java | 39 +- .../CatalogsCreateOrUpdateMockTests.java | 51 +- .../generated/CatalogsDeleteMockTests.java | 36 +- ...tByResourceGroupWithResponseMockTests.java | 41 +- .../CatalogsListByResourceGroupMockTests.java | 40 +- .../CatalogsListDeploymentsMockTests.java | 53 +- .../CatalogsListDeviceGroupsMockTests.java | 62 +- .../CatalogsListDeviceInsightsMockTests.java | 66 +- .../CatalogsListDevicesMockTests.java | 48 +- .../generated/CatalogsListMockTests.java | 37 +- .../CertificateChainResponseInnerTests.java | 6 +- .../generated/CertificateInnerTests.java | 11 +- .../generated/CertificateListResultTests.java | 18 +- .../generated/CertificatePropertiesTests.java | 10 +- .../CertificatesGetWithResponseMockTests.java | 41 +- .../CertificatesListByCatalogMockTests.java | 47 +- ...etrieveCertChainWithResponseMockTests.java | 39 +- ...fPossessionNonceWithResponseMockTests.java | 49 +- .../generated/ClaimDevicesRequestTests.java | 15 +- .../generated/CountDeviceResponseTests.java | 24 + ...va => CountDevicesResponseInnerTests.java} | 16 +- .../generated/CountElementsResponseTests.java | 10 +- .../generated/DeploymentInnerTests.java | 61 +- .../generated/DeploymentListResultTests.java | 34 +- .../generated/DeploymentPropertiesTests.java | 49 +- .../DeploymentsCreateOrUpdateMockTests.java | 71 +- .../generated/DeploymentsDeleteMockTests.java | 37 +- .../DeploymentsGetWithResponseMockTests.java | 56 +- ...DeploymentsListByDeviceGroupMockTests.java | 61 +- .../generated/DeviceGroupInnerTests.java | 40 +- .../generated/DeviceGroupListResultTests.java | 76 +- .../generated/DeviceGroupPropertiesTests.java | 29 +- .../DeviceGroupUpdatePropertiesTests.java | 29 +- .../generated/DeviceGroupUpdateTests.java | 41 +- ...oupsCountDevicesWithResponseMockTests.java | 42 +- .../DeviceGroupsCreateOrUpdateMockTests.java | 60 +- .../DeviceGroupsDeleteMockTests.java | 39 +- .../DeviceGroupsGetWithResponseMockTests.java | 50 +- .../DeviceGroupsListByProductMockTests.java | 61 +- .../sphere/generated/DeviceInnerTests.java | 15 +- .../generated/DeviceInsightInnerTests.java | 54 +- .../generated/DeviceListResultTests.java | 30 +- .../generated/DevicePropertiesTests.java | 16 +- .../DeviceUpdatePropertiesTests.java | 12 +- .../sphere/generated/DeviceUpdateTests.java | 12 +- .../DevicesCreateOrUpdateMockTests.java | 46 +- .../generated/DevicesDeleteMockTests.java | 39 +- ...vicesGenerateCapabilityImageMockTests.java | 54 +- .../DevicesGetWithResponseMockTests.java | 48 +- .../DevicesListByDeviceGroupMockTests.java | 42 +- .../GenerateCapabilityImageRequestTests.java | 14 +- .../sphere/generated/ImageInnerTests.java | 28 +- .../generated/ImageListResultTests.java | 46 +- .../generated/ImagePropertiesTests.java | 29 +- .../ImagesCreateOrUpdateMockTests.java | 59 +- .../generated/ImagesDeleteMockTests.java | 36 +- .../ImagesGetWithResponseMockTests.java | 46 +- .../ImagesListByCatalogMockTests.java | 59 +- .../ListDeviceGroupsRequestTests.java | 10 +- .../generated/OperationDisplayTests.java | 8 +- .../sphere/generated/OperationInnerTests.java | 8 +- .../generated/OperationListResultTests.java | 8 +- .../generated/OperationsListMockTests.java | 34 +- .../generated/PagedDeviceInsightTests.java | 79 +- .../sphere/generated/ProductInnerTests.java | 15 +- .../generated/ProductListResultTests.java | 21 +- .../generated/ProductPropertiesTests.java | 14 +- .../ProductUpdatePropertiesTests.java | 12 +- .../sphere/generated/ProductUpdateTests.java | 12 +- ...uctsCountDevicesWithResponseMockTests.java | 43 +- .../ProductsCreateOrUpdateMockTests.java | 46 +- .../generated/ProductsDeleteMockTests.java | 36 +- ...sGenerateDefaultDeviceGroupsMockTests.java | 51 +- .../ProductsGetWithResponseMockTests.java | 42 +- .../ProductsListByCatalogMockTests.java | 39 +- .../ProofOfPossessionNonceRequestTests.java | 10 +- ...ofOfPossessionNonceResponseInnerTests.java | 8 +- ...gnedCapabilityImageResponseInnerTests.java | 4 +- 255 files changed, 8363 insertions(+), 12377 deletions(-) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/{AzureSphereManagementClient.java => AzureSphereMgmtClient.java} (91%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/{CountDeviceResponseInner.java => CountDevicesResponseInner.java} (62%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/{AzureSphereManagementClientBuilder.java => AzureSphereMgmtClientBuilder.java} (54%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/{AzureSphereManagementClientImpl.java => AzureSphereMgmtClientImpl.java} (80%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/{CountDeviceResponseImpl.java => CountDevicesResponseImpl.java} (62%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/{Utils.java => ResourceManagerUtils.java} (79%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/CatalogProperties.java (62%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/CertificateProperties.java (90%) create mode 100644 sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDevicesResponse.java rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/DeploymentProperties.java (90%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/DeviceGroupProperties.java (90%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/DeviceGroupUpdateProperties.java (87%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/DeviceProperties.java (93%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/DeviceUpdateProperties.java (85%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/ImageProperties.java (91%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/ProductProperties.java (69%) rename sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/{fluent => }/models/ProductUpdateProperties.java (85%) create mode 100644 sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/proxy-config.json create mode 100644 sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/reflect-config.json create mode 100644 sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUploadImageSamples.java create mode 100644 sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseTests.java rename sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/{CountDeviceResponseInnerTests.java => CountDevicesResponseInnerTests.java} (56%) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index f86b6f8a3e9f6..6d83cc6fec48b 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -435,7 +435,7 @@ com.azure.resourcemanager:azure-resourcemanager-managementgroups;1.0.0-beta.1;1. com.azure.resourcemanager:azure-resourcemanager-managednetworkfabric;1.0.0;1.1.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-iotfirmwaredefense;1.0.0;1.1.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-quantum;1.0.0-beta.2;1.0.0-beta.3 -com.azure.resourcemanager:azure-resourcemanager-sphere;1.0.0-beta.1;1.0.0-beta.2 +com.azure.resourcemanager:azure-resourcemanager-sphere;1.0.0-beta.1;1.0.0 com.azure.resourcemanager:azure-resourcemanager-chaos;1.1.0;1.2.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-defendereasm;1.0.0-beta.1;1.0.0-beta.2 com.azure.resourcemanager:azure-resourcemanager-hdinsight-containers;1.0.0-beta.1;1.0.0-beta.2 diff --git a/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md b/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md index db7567eac8ac1..50b71037fac9e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md +++ b/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md @@ -1,14 +1,310 @@ # Release History -## 1.0.0-beta.2 (Unreleased) +## 1.0.0 (2024-03-26) -### Features Added +- Azure Resource Manager AzureSphere client library for Java. This package contains Microsoft Azure SDK for AzureSphere Management SDK. Azure Sphere resource management API. Package tag package-2024-04-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ### Breaking Changes -### Bugs Fixed +#### `models.CountDeviceResponse` was modified + +* `value()` was removed +* `innerModel()` was removed + +#### `models.Product$Definition` was modified + +* `withDescription(java.lang.String)` was removed + +#### `models.ImageListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.PagedDeviceInsight` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `AzureSphereManager` was modified + +* `fluent.AzureSphereManagementClient serviceClient()` -> `fluent.AzureSphereMgmtClient serviceClient()` + +#### `models.DeviceGroup$Update` was modified + +* `withAllowCrashDumpsCollection(models.AllowCrashDumpCollection)` was removed +* `withDescription(java.lang.String)` was removed +* `withUpdatePolicy(models.UpdatePolicy)` was removed +* `withOsFeedType(models.OSFeedType)` was removed +* `withRegionalDataBoundary(models.RegionalDataBoundary)` was removed + +#### `models.DeviceGroupListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.Catalogs` was modified + +* `models.CountDeviceResponse countDevices(java.lang.String,java.lang.String)` -> `models.CountDevicesResponse countDevices(java.lang.String,java.lang.String)` + +#### `models.DeploymentListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.Device$Definition` was modified + +* `withDeviceId(java.lang.String)` was removed + +#### `models.Catalog` was modified + +* `provisioningState()` was removed +* `models.CountDeviceResponse countDevices()` -> `models.CountDevicesResponse countDevices()` + +#### `models.Device` was modified + +* `chipSku()` was removed +* `provisioningState()` was removed +* `lastAvailableOsVersion()` was removed +* `lastOsUpdateUtc()` was removed +* `deviceId()` was removed +* `lastUpdateRequestUtc()` was removed +* `lastInstalledOsVersion()` was removed + +#### `models.Product$Update` was modified + +* `withDescription(java.lang.String)` was removed + +#### `models.ProductListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.DeviceGroup$Definition` was modified + +* `withAllowCrashDumpsCollection(models.AllowCrashDumpCollection)` was removed +* `withDescription(java.lang.String)` was removed +* `withUpdatePolicy(models.UpdatePolicy)` was removed +* `withRegionalDataBoundary(models.RegionalDataBoundary)` was removed +* `withOsFeedType(models.OSFeedType)` was removed + +#### `models.DeviceGroup` was modified + +* `osFeedType()` was removed +* `models.CountDeviceResponse countDevices()` -> `models.CountDevicesResponse countDevices()` +* `provisioningState()` was removed +* `hasDeployment()` was removed +* `allowCrashDumpsCollection()` was removed +* `regionalDataBoundary()` was removed +* `description()` was removed +* `updatePolicy()` was removed + +#### `models.DeviceUpdate` was modified + +* `withDeviceGroupId(java.lang.String)` was removed +* `deviceGroupId()` was removed + +#### `models.CatalogListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.DeviceListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.Product` was modified + +* `description()` was removed +* `models.CountDeviceResponse countDevices()` -> `models.CountDevicesResponse countDevices()` +* `provisioningState()` was removed + +#### `models.Device$Update` was modified + +* `withDeviceGroupId(java.lang.String)` was removed + +#### `models.Image$Definition` was modified + +* `withRegionalDataBoundary(models.RegionalDataBoundary)` was removed +* `withImage(java.lang.String)` was removed +* `withImageId(java.lang.String)` was removed + +#### `models.Deployment` was modified + +* `deploymentId()` was removed +* `deploymentDateUtc()` was removed +* `provisioningState()` was removed +* `deployedImages()` was removed + +#### `models.DeviceGroups` was modified + +* `models.CountDeviceResponse countDevices(java.lang.String,java.lang.String,java.lang.String,java.lang.String)` -> `models.CountDevicesResponse countDevices(java.lang.String,java.lang.String,java.lang.String,java.lang.String)` + +#### `models.Products` was modified + +* `models.CountDeviceResponse countDevices(java.lang.String,java.lang.String,java.lang.String)` -> `models.CountDevicesResponse countDevices(java.lang.String,java.lang.String,java.lang.String)` + +#### `models.ProductUpdate` was modified + +* `withDescription(java.lang.String)` was removed +* `description()` was removed + +#### `models.DeviceGroupUpdate` was modified + +* `regionalDataBoundary()` was removed +* `withRegionalDataBoundary(models.RegionalDataBoundary)` was removed +* `allowCrashDumpsCollection()` was removed +* `updatePolicy()` was removed +* `withAllowCrashDumpsCollection(models.AllowCrashDumpCollection)` was removed +* `withOsFeedType(models.OSFeedType)` was removed +* `withUpdatePolicy(models.UpdatePolicy)` was removed +* `withDescription(java.lang.String)` was removed +* `description()` was removed +* `osFeedType()` was removed + +#### `models.CertificateListResult` was modified + +* `withNextLink(java.lang.String)` was removed + +#### `models.Image` was modified + +* `regionalDataBoundary()` was removed +* `image()` was removed +* `provisioningState()` was removed +* `uri()` was removed +* `imageName()` was removed +* `imageId()` was removed +* `description()` was removed +* `componentId()` was removed +* `imageType()` was removed + +#### `models.Deployment$Definition` was modified + +* `withDeploymentId(java.lang.String)` was removed +* `withDeployedImages(java.util.List)` was removed + +#### `models.Certificate` was modified + +* `notBeforeUtc()` was removed +* `subject()` was removed +* `expiryUtc()` was removed +* `certificate()` was removed +* `thumbprint()` was removed +* `provisioningState()` was removed +* `status()` was removed + +### Features Added + +* `models.DeviceGroupProperties` was added + +* `models.DeviceProperties` was added + +* `models.CertificateProperties` was added + +* `models.CountDevicesResponse` was added + +* `models.DeploymentProperties` was added + +* `models.DeviceUpdateProperties` was added + +* `models.DeviceGroupUpdateProperties` was added + +* `models.CatalogProperties` was added + +* `models.ProductUpdateProperties` was added + +* `models.ImageProperties` was added + +* `models.ProductProperties` was added + +#### `models.CountDeviceResponse` was modified + +* `withValue(int)` was added +* `validate()` was added + +#### `models.Product$Definition` was modified + +* `withProperties(models.ProductProperties)` was added + +#### `models.DeviceGroup$Update` was modified + +* `withProperties(models.DeviceGroupUpdateProperties)` was added + +#### `models.Catalogs` was modified + +* `uploadImage(java.lang.String,java.lang.String,fluent.models.ImageInner,com.azure.core.util.Context)` was added +* `uploadImage(java.lang.String,java.lang.String,fluent.models.ImageInner)` was added + +#### `models.Device$Definition` was modified + +* `withProperties(models.DeviceProperties)` was added + +#### `models.Catalog` was modified + +* `uploadImage(fluent.models.ImageInner,com.azure.core.util.Context)` was added +* `properties()` was added +* `uploadImage(fluent.models.ImageInner)` was added + +#### `models.Device` was modified + +* `systemData()` was added +* `properties()` was added + +#### `models.Product$Update` was modified + +* `withProperties(models.ProductUpdateProperties)` was added + +#### `models.DeviceGroup$Definition` was modified + +* `withProperties(models.DeviceGroupProperties)` was added + +#### `models.DeviceGroup` was modified + +* `properties()` was added +* `systemData()` was added + +#### `models.DeviceUpdate` was modified + +* `properties()` was added +* `withProperties(models.DeviceUpdateProperties)` was added + +#### `models.Product` was modified + +* `properties()` was added +* `systemData()` was added + +#### `models.Device$Update` was modified + +* `withProperties(models.DeviceUpdateProperties)` was added + +#### `models.Image$Definition` was modified + +* `withProperties(models.ImageProperties)` was added + +#### `models.Deployment` was modified + +* `systemData()` was added +* `properties()` was added + +#### `models.ProductUpdate` was modified + +* `withProperties(models.ProductUpdateProperties)` was added +* `properties()` was added + +#### `models.DeviceGroupUpdate` was modified + +* `withProperties(models.DeviceGroupUpdateProperties)` was added +* `properties()` was added + +#### `models.Catalog$Definition` was modified + +* `withProperties(models.CatalogProperties)` was added + +#### `models.Image` was modified + +* `systemData()` was added +* `properties()` was added + +#### `models.Deployment$Definition` was modified + +* `withProperties(models.DeploymentProperties)` was added + +#### `models.Certificate` was modified -### Other Changes +* `properties()` was added ## 1.0.0-beta.1 (2023-07-21) diff --git a/sdk/sphere/azure-resourcemanager-sphere/README.md b/sdk/sphere/azure-resourcemanager-sphere/README.md index 2c270ea9f71a9..8918acb6fd371 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/README.md +++ b/sdk/sphere/azure-resourcemanager-sphere/README.md @@ -2,7 +2,7 @@ Azure Resource Manager AzureSphere client library for Java. -This package contains Microsoft Azure SDK for AzureSphere Management SDK. Azure Sphere resource management API. Package tag package-2022-09-01-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for AzureSphere Management SDK. Azure Sphere resource management API. Package tag package-2024-04-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-sphere - 1.0.0-beta.1 + 1.0.0 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -94,7 +94,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty diff --git a/sdk/sphere/azure-resourcemanager-sphere/SAMPLE.md b/sdk/sphere/azure-resourcemanager-sphere/SAMPLE.md index 5bf2b6bf9674a..0f9c9ae9fc3c8 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/SAMPLE.md +++ b/sdk/sphere/azure-resourcemanager-sphere/SAMPLE.md @@ -14,6 +14,7 @@ - [ListDeviceInsights](#catalogs_listdeviceinsights) - [ListDevices](#catalogs_listdevices) - [Update](#catalogs_update) +- [UploadImage](#catalogs_uploadimage) ## Certificates @@ -71,14 +72,18 @@ ### Catalogs_CountDevices ```java -/** Samples for Catalogs CountDevices. */ +/** + * Samples for Catalogs CountDevices. + */ public final class CatalogsCountDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostCountDevicesCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostCountDevicesCatalog. + * json */ /** * Sample code: Catalogs_CountDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsCountDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -90,22 +95,21 @@ public final class CatalogsCountDevicesSamples { ### Catalogs_CreateOrUpdate ```java -/** Samples for Catalogs CreateOrUpdate. */ +/** + * Samples for Catalogs CreateOrUpdate. + */ public final class CatalogsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutCatalog.json */ /** * Sample code: Catalogs_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .define("MyCatalog1") - .withRegion("global") - .withExistingResourceGroup("MyResourceGroup1") + manager.catalogs().define("MyCatalog1").withRegion("global").withExistingResourceGroup("MyResourceGroup1") .create(); } } @@ -114,14 +118,17 @@ public final class CatalogsCreateOrUpdateSamples { ### Catalogs_Delete ```java -/** Samples for Catalogs Delete. */ +/** + * Samples for Catalogs Delete. + */ public final class CatalogsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteCatalog.json */ /** * Sample code: Catalogs_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -133,20 +140,22 @@ public final class CatalogsDeleteSamples { ### Catalogs_GetByResourceGroup ```java -/** Samples for Catalogs GetByResourceGroup. */ +/** + * Samples for Catalogs GetByResourceGroup. + */ public final class CatalogsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCatalog.json */ /** * Sample code: Catalogs_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", com.azure.core.util.Context.NONE); + manager.catalogs().getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", + com.azure.core.util.Context.NONE); } } ``` @@ -154,14 +163,17 @@ public final class CatalogsGetByResourceGroupSamples { ### Catalogs_List ```java -/** Samples for Catalogs List. */ +/** + * Samples for Catalogs List. + */ public final class CatalogsListSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCatalogsSub.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCatalogsSub.json */ /** * Sample code: Catalogs_ListBySubscription. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListBySubscription(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -173,14 +185,17 @@ public final class CatalogsListSamples { ### Catalogs_ListByResourceGroup ```java -/** Samples for Catalogs ListByResourceGroup. */ +/** + * Samples for Catalogs ListByResourceGroup. + */ public final class CatalogsListByResourceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCatalogsRG.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCatalogsRG.json */ /** * Sample code: Catalogs_ListByResourceGroup. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListByResourceGroup(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -192,21 +207,22 @@ public final class CatalogsListByResourceGroupSamples { ### Catalogs_ListDeployments ```java -/** Samples for Catalogs ListDeployments. */ +/** + * Samples for Catalogs ListDeployments. + */ public final class CatalogsListDeploymentsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDeploymentsByCatalog.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostListDeploymentsByCatalog.json */ /** * Sample code: Catalogs_ListDeployments. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDeployments(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDeployments( - "MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.catalogs().listDeployments("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -216,28 +232,23 @@ public final class CatalogsListDeploymentsSamples { ```java import com.azure.resourcemanager.sphere.models.ListDeviceGroupsRequest; -/** Samples for Catalogs ListDeviceGroups. */ +/** + * Samples for Catalogs ListDeviceGroups. + */ public final class CatalogsListDeviceGroupsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDeviceGroupsCatalog.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostListDeviceGroupsCatalog.json */ /** * Sample code: Catalogs_ListDeviceGroups. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDeviceGroups(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDeviceGroups( - "MyResourceGroup1", - "MyCatalog1", - new ListDeviceGroupsRequest().withDeviceGroupName("MyDeviceGroup1"), - null, - null, - null, - null, - com.azure.core.util.Context.NONE); + manager.catalogs().listDeviceGroups("MyResourceGroup1", "MyCatalog1", + new ListDeviceGroupsRequest().withDeviceGroupName("MyDeviceGroup1"), null, null, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -245,21 +256,22 @@ public final class CatalogsListDeviceGroupsSamples { ### Catalogs_ListDeviceInsights ```java -/** Samples for Catalogs ListDeviceInsights. */ +/** + * Samples for Catalogs ListDeviceInsights. + */ public final class CatalogsListDeviceInsightsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDeviceInsightsCatalog.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostListDeviceInsightsCatalog.json */ /** * Sample code: Catalogs_ListDeviceInsights. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDeviceInsights(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDeviceInsights( - "MyResourceGroup1", "MyCatalog1", null, 10, null, null, com.azure.core.util.Context.NONE); + manager.catalogs().listDeviceInsights("MyResourceGroup1", "MyCatalog1", null, 10, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -267,20 +279,23 @@ public final class CatalogsListDeviceInsightsSamples { ### Catalogs_ListDevices ```java -/** Samples for Catalogs ListDevices. */ +/** + * Samples for Catalogs ListDevices. + */ public final class CatalogsListDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDevicesByCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostListDevicesByCatalog. + * json */ /** * Sample code: Catalogs_ListDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDevices("MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.catalogs().listDevices("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -290,44 +305,75 @@ public final class CatalogsListDevicesSamples { ```java import com.azure.resourcemanager.sphere.models.Catalog; -/** Samples for Catalogs Update. */ +/** + * Samples for Catalogs Update. + */ public final class CatalogsUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchCatalog.json */ /** * Sample code: Catalogs_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - Catalog resource = - manager - .catalogs() - .getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", com.azure.core.util.Context.NONE) - .getValue(); + Catalog resource = manager.catalogs() + .getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", com.azure.core.util.Context.NONE) + .getValue(); resource.update().apply(); } } ``` +### Catalogs_UploadImage + +```java +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; +import com.azure.resourcemanager.sphere.models.ImageProperties; + +/** + * Samples for Catalogs UploadImage. + */ +public final class CatalogsUploadImageSamples { + /* + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostUploadImageCatalog. + * json + */ + /** + * Sample code: Catalogs_UploadImage. + * + * @param manager Entry point to AzureSphereManager. + */ + public static void catalogsUploadImage(com.azure.resourcemanager.sphere.AzureSphereManager manager) { + manager.catalogs().uploadImage("MyResourceGroup1", "MyCatalog1", + new ImageInner().withProperties(new ImageProperties().withImage("bXliYXNlNjRzdHJpbmc=")), + com.azure.core.util.Context.NONE); + } +} +``` + ### Certificates_Get ```java -/** Samples for Certificates Get. */ +/** + * Samples for Certificates Get. + */ public final class CertificatesGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCertificate.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCertificate.json */ /** * Sample code: Certificates_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void certificatesGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "default", com.azure.core.util.Context.NONE); + manager.certificates().getWithResponse("MyResourceGroup1", "MyCatalog1", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -335,20 +381,22 @@ public final class CertificatesGetSamples { ### Certificates_ListByCatalog ```java -/** Samples for Certificates ListByCatalog. */ +/** + * Samples for Certificates ListByCatalog. + */ public final class CertificatesListByCatalogSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCertificates.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCertificates.json */ /** * Sample code: Certificates_ListByCatalog. - * + * * @param manager Entry point to AzureSphereManager. */ public static void certificatesListByCatalog(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.certificates().listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -356,21 +404,22 @@ public final class CertificatesListByCatalogSamples { ### Certificates_RetrieveCertChain ```java -/** Samples for Certificates RetrieveCertChain. */ +/** + * Samples for Certificates RetrieveCertChain. + */ public final class CertificatesRetrieveCertChainSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostRetrieveCatalogCertChain.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostRetrieveCatalogCertChain.json */ /** * Sample code: Certificates_RetrieveCertChain. - * + * * @param manager Entry point to AzureSphereManager. */ public static void certificatesRetrieveCertChain(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .retrieveCertChainWithResponse( - "MyResourceGroup1", "MyCatalog1", "active", com.azure.core.util.Context.NONE); + manager.certificates().retrieveCertChainWithResponse("MyResourceGroup1", "MyCatalog1", "active", + com.azure.core.util.Context.NONE); } } ``` @@ -380,26 +429,24 @@ public final class CertificatesRetrieveCertChainSamples { ```java import com.azure.resourcemanager.sphere.models.ProofOfPossessionNonceRequest; -/** Samples for Certificates RetrieveProofOfPossessionNonce. */ +/** + * Samples for Certificates RetrieveProofOfPossessionNonce. + */ public final class CertificatesRetrieveProofOfPossessionNonceSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostRetrieveProofOfPossessionNonce.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostRetrieveProofOfPossessionNonce.json */ /** * Sample code: Certificates_RetrieveProofOfPossessionNonce. - * + * * @param manager Entry point to AzureSphereManager. */ - public static void certificatesRetrieveProofOfPossessionNonce( - com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .retrieveProofOfPossessionNonceWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "active", - new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("proofOfPossessionNonce"), - com.azure.core.util.Context.NONE); + public static void + certificatesRetrieveProofOfPossessionNonce(com.azure.resourcemanager.sphere.AzureSphereManager manager) { + manager.certificates().retrieveProofOfPossessionNonceWithResponse("MyResourceGroup1", "MyCatalog1", "active", + new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("proofOfPossessionNonce"), + com.azure.core.util.Context.NONE); } } ``` @@ -407,22 +454,22 @@ public final class CertificatesRetrieveProofOfPossessionNonceSamples { ### Deployments_CreateOrUpdate ```java -/** Samples for Deployments CreateOrUpdate. */ +/** + * Samples for Deployments CreateOrUpdate. + */ public final class DeploymentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutDeployment.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutDeployment.json */ /** * Sample code: Deployments_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .define("MyDeployment1") - .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1") - .create(); + manager.deployments().define("MyDeployment1") + .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1").create(); } } ``` @@ -430,26 +477,22 @@ public final class DeploymentsCreateOrUpdateSamples { ### Deployments_Delete ```java -/** Samples for Deployments Delete. */ +/** + * Samples for Deployments Delete. + */ public final class DeploymentsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteDeployment.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteDeployment.json */ /** * Sample code: Deployments_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .delete( - "MyResourceGroup1", - "MyCatalog1", - "MyProductName1", - "DeviceGroupName1", - "MyDeploymentName1", - com.azure.core.util.Context.NONE); + manager.deployments().delete("MyResourceGroup1", "MyCatalog1", "MyProductName1", "DeviceGroupName1", + "MyDeploymentName1", com.azure.core.util.Context.NONE); } } ``` @@ -457,26 +500,22 @@ public final class DeploymentsDeleteSamples { ### Deployments_Get ```java -/** Samples for Deployments Get. */ +/** + * Samples for Deployments Get. + */ public final class DeploymentsGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeployment.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeployment.json */ /** * Sample code: Deployments_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .getWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - "MyDeployment1", - com.azure.core.util.Context.NONE); + manager.deployments().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + "MyDeployment1", com.azure.core.util.Context.NONE); } } ``` @@ -484,29 +523,22 @@ public final class DeploymentsGetSamples { ### Deployments_ListByDeviceGroup ```java -/** Samples for Deployments ListByDeviceGroup. */ +/** + * Samples for Deployments ListByDeviceGroup. + */ public final class DeploymentsListByDeviceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeployments.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeployments.json */ /** * Sample code: Deployments_ListByDeviceGroup. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsListByDeviceGroup(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .listByDeviceGroup( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - null, - null, - null, - null, - com.azure.core.util.Context.NONE); + manager.deployments().listByDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", null, + null, null, null, com.azure.core.util.Context.NONE); } } ``` @@ -517,30 +549,24 @@ public final class DeploymentsListByDeviceGroupSamples { import com.azure.resourcemanager.sphere.models.ClaimDevicesRequest; import java.util.Arrays; -/** Samples for DeviceGroups ClaimDevices. */ +/** + * Samples for DeviceGroups ClaimDevices. + */ public final class DeviceGroupsClaimDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostClaimDevices.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostClaimDevices.json */ /** * Sample code: DeviceGroups_ClaimDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsClaimDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .claimDevices( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "MyDeviceGroup1", - new ClaimDevicesRequest() - .withDeviceIdentifiers( - Arrays - .asList( - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000")), - com.azure.core.util.Context.NONE); + manager.deviceGroups().claimDevices("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", + new ClaimDevicesRequest().withDeviceIdentifiers(Arrays.asList( + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000")), + com.azure.core.util.Context.NONE); } } ``` @@ -548,21 +574,22 @@ public final class DeviceGroupsClaimDevicesSamples { ### DeviceGroups_CountDevices ```java -/** Samples for DeviceGroups CountDevices. */ +/** + * Samples for DeviceGroups CountDevices. + */ public final class DeviceGroupsCountDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostCountDevicesDeviceGroup.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostCountDevicesDeviceGroup.json */ /** * Sample code: DeviceGroups_CountDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsCountDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .countDevicesWithResponse( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE); + manager.deviceGroups().countDevicesWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + "MyDeviceGroup1", com.azure.core.util.Context.NONE); } } ``` @@ -570,27 +597,28 @@ public final class DeviceGroupsCountDevicesSamples { ### DeviceGroups_CreateOrUpdate ```java +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.UpdatePolicy; -/** Samples for DeviceGroups CreateOrUpdate. */ +/** + * Samples for DeviceGroups CreateOrUpdate. + */ public final class DeviceGroupsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutDeviceGroup.json */ /** * Sample code: DeviceGroups_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .define("MyDeviceGroup1") + manager.deviceGroups().define("MyDeviceGroup1") .withExistingProduct("MyResourceGroup1", "MyCatalog1", "MyProduct1") - .withDescription("Description for MyDeviceGroup1") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.UPDATE_ALL) + .withProperties(new DeviceGroupProperties().withDescription("Description for MyDeviceGroup1") + .withOsFeedType(OSFeedType.RETAIL).withUpdatePolicy(UpdatePolicy.UPDATE_ALL)) .create(); } } @@ -599,20 +627,22 @@ public final class DeviceGroupsCreateOrUpdateSamples { ### DeviceGroups_Delete ```java -/** Samples for DeviceGroups Delete. */ +/** + * Samples for DeviceGroups Delete. + */ public final class DeviceGroupsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteDeviceGroup.json */ /** * Sample code: DeviceGroups_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .delete("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE); + manager.deviceGroups().delete("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", + com.azure.core.util.Context.NONE); } } ``` @@ -620,21 +650,22 @@ public final class DeviceGroupsDeleteSamples { ### DeviceGroups_Get ```java -/** Samples for DeviceGroups Get. */ +/** + * Samples for DeviceGroups Get. + */ public final class DeviceGroupsGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeviceGroup.json */ /** * Sample code: DeviceGroups_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .getWithResponse( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE); + manager.deviceGroups().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", + com.azure.core.util.Context.NONE); } } ``` @@ -642,28 +673,22 @@ public final class DeviceGroupsGetSamples { ### DeviceGroups_ListByProduct ```java -/** Samples for DeviceGroups ListByProduct. */ +/** + * Samples for DeviceGroups ListByProduct. + */ public final class DeviceGroupsListByProductSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeviceGroups.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeviceGroups.json */ /** * Sample code: DeviceGroups_ListByProduct. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsListByProduct(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .listByProduct( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - null, - null, - null, - null, - com.azure.core.util.Context.NONE); + manager.deviceGroups().listByProduct("MyResourceGroup1", "MyCatalog1", "MyProduct1", null, null, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -673,23 +698,22 @@ public final class DeviceGroupsListByProductSamples { ```java import com.azure.resourcemanager.sphere.models.DeviceGroup; -/** Samples for DeviceGroups Update. */ +/** + * Samples for DeviceGroups Update. + */ public final class DeviceGroupsUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchDeviceGroup.json */ /** * Sample code: DeviceGroups_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - DeviceGroup resource = - manager - .deviceGroups() - .getWithResponse( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE) - .getValue(); + DeviceGroup resource = manager.deviceGroups().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + "MyDeviceGroup1", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } @@ -698,23 +722,23 @@ public final class DeviceGroupsUpdateSamples { ### Devices_CreateOrUpdate ```java -/** Samples for Devices CreateOrUpdate. */ +/** + * Samples for Devices CreateOrUpdate. + */ public final class DevicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutDevice.json */ /** * Sample code: Devices_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .define( - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000") - .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1") - .create(); + manager.devices().define( + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000") + .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1").create(); } } ``` @@ -722,26 +746,23 @@ public final class DevicesCreateOrUpdateSamples { ### Devices_Delete ```java -/** Samples for Devices Delete. */ +/** + * Samples for Devices Delete. + */ public final class DevicesDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteDevice.json */ /** * Sample code: Devices_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .delete( - "MyResourceGroup1", - "MyCatalog1", - "MyProductName1", - "DeviceGroupName1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - com.azure.core.util.Context.NONE); + manager.devices().delete("MyResourceGroup1", "MyCatalog1", "MyProductName1", "DeviceGroupName1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + com.azure.core.util.Context.NONE); } } ``` @@ -753,28 +774,25 @@ import com.azure.resourcemanager.sphere.models.CapabilityType; import com.azure.resourcemanager.sphere.models.GenerateCapabilityImageRequest; import java.util.Arrays; -/** Samples for Devices GenerateCapabilityImage. */ +/** + * Samples for Devices GenerateCapabilityImage. + */ public final class DevicesGenerateCapabilityImageSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostGenerateDeviceCapabilityImage.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostGenerateDeviceCapabilityImage.json */ /** * Sample code: Devices_GenerateCapabilityImage. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesGenerateCapabilityImage(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .generateCapabilityImage( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - new GenerateCapabilityImageRequest() - .withCapabilities(Arrays.asList(CapabilityType.APPLICATION_DEVELOPMENT)), - com.azure.core.util.Context.NONE); + manager.devices().generateCapabilityImage("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + new GenerateCapabilityImageRequest().withCapabilities( + Arrays.asList(CapabilityType.APPLICATION_DEVELOPMENT)), + com.azure.core.util.Context.NONE); } } ``` @@ -782,26 +800,23 @@ public final class DevicesGenerateCapabilityImageSamples { ### Devices_Get ```java -/** Samples for Devices Get. */ +/** + * Samples for Devices Get. + */ public final class DevicesGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDevice.json */ /** * Sample code: Devices_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .getWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - com.azure.core.util.Context.NONE); + manager.devices().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + com.azure.core.util.Context.NONE); } } ``` @@ -809,21 +824,22 @@ public final class DevicesGetSamples { ### Devices_ListByDeviceGroup ```java -/** Samples for Devices ListByDeviceGroup. */ +/** + * Samples for Devices ListByDeviceGroup. + */ public final class DevicesListByDeviceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDevices.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDevices.json */ /** * Sample code: Devices_ListByDeviceGroup. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesListByDeviceGroup(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .listByDeviceGroup( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", com.azure.core.util.Context.NONE); + manager.devices().listByDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + com.azure.core.util.Context.NONE); } } ``` @@ -833,28 +849,24 @@ public final class DevicesListByDeviceGroupSamples { ```java import com.azure.resourcemanager.sphere.models.Device; -/** Samples for Devices Update. */ +/** + * Samples for Devices Update. + */ public final class DevicesUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchDevice.json */ /** * Sample code: Devices_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - Device resource = - manager - .devices() - .getWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "MyDeviceGroup1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - com.azure.core.util.Context.NONE) - .getValue(); + Device resource = manager.devices().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + "MyDeviceGroup1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } @@ -863,23 +875,25 @@ public final class DevicesUpdateSamples { ### Images_CreateOrUpdate ```java -/** Samples for Images CreateOrUpdate. */ +import com.azure.resourcemanager.sphere.models.ImageProperties; + +/** + * Samples for Images CreateOrUpdate. + */ public final class ImagesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutImage.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutImage.json */ /** * Sample code: Image_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imageCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .images() - .define("default") + manager.images().define("00000000-0000-0000-0000-000000000000") .withExistingCatalog("MyResourceGroup1", "MyCatalog1") - .withImage("bXliYXNlNjRzdHJpbmc=") - .create(); + .withProperties(new ImageProperties().withImage("bXliYXNlNjRzdHJpbmc=")).create(); } } ``` @@ -887,18 +901,22 @@ public final class ImagesCreateOrUpdateSamples { ### Images_Delete ```java -/** Samples for Images Delete. */ +/** + * Samples for Images Delete. + */ public final class ImagesDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteImage.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteImage.json */ /** * Sample code: Images_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imagesDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager.images().delete("MyResourceGroup1", "MyCatalog1", "imageID", com.azure.core.util.Context.NONE); + manager.images().delete("MyResourceGroup1", "MyCatalog1", "00000000-0000-0000-0000-000000000000", + com.azure.core.util.Context.NONE); } } ``` @@ -906,20 +924,22 @@ public final class ImagesDeleteSamples { ### Images_Get ```java -/** Samples for Images Get. */ +/** + * Samples for Images Get. + */ public final class ImagesGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetImage.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetImage.json */ /** * Sample code: Images_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imagesGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .images() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "myImageId", com.azure.core.util.Context.NONE); + manager.images().getWithResponse("MyResourceGroup1", "MyCatalog1", "00000000-0000-0000-0000-000000000000", + com.azure.core.util.Context.NONE); } } ``` @@ -927,20 +947,22 @@ public final class ImagesGetSamples { ### Images_ListByCatalog ```java -/** Samples for Images ListByCatalog. */ +/** + * Samples for Images ListByCatalog. + */ public final class ImagesListByCatalogSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetImages.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetImages.json */ /** * Sample code: Images_ListByCatalog. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imagesListByCatalog(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .images() - .listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.images().listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -948,14 +970,17 @@ public final class ImagesListByCatalogSamples { ### Operations_List ```java -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetOperations.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetOperations.json */ /** * Sample code: Operations_List. - * + * * @param manager Entry point to AzureSphereManager. */ public static void operationsList(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -967,20 +992,23 @@ public final class OperationsListSamples { ### Products_CountDevices ```java -/** Samples for Products CountDevices. */ +/** + * Samples for Products CountDevices. + */ public final class ProductsCountDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostCountDevicesProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostCountDevicesProduct. + * json */ /** * Sample code: Products_CountDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsCountDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .products() - .countDevicesWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE); + manager.products().countDevicesWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + com.azure.core.util.Context.NONE); } } ``` @@ -988,14 +1016,17 @@ public final class ProductsCountDevicesSamples { ### Products_CreateOrUpdate ```java -/** Samples for Products CreateOrUpdate. */ +/** + * Samples for Products CreateOrUpdate. + */ public final class ProductsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutProduct.json */ /** * Sample code: Products_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -1007,14 +1038,17 @@ public final class ProductsCreateOrUpdateSamples { ### Products_Delete ```java -/** Samples for Products Delete. */ +/** + * Samples for Products Delete. + */ public final class ProductsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteProduct.json */ /** * Sample code: Products_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -1026,22 +1060,23 @@ public final class ProductsDeleteSamples { ### Products_GenerateDefaultDeviceGroups ```java -/** Samples for Products GenerateDefaultDeviceGroups. */ +/** + * Samples for Products GenerateDefaultDeviceGroups. + */ public final class ProductsGenerateDefaultDeviceGroupsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostGenerateDefaultDeviceGroups.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostGenerateDefaultDeviceGroups.json */ /** * Sample code: Products_GenerateDefaultDeviceGroups. - * + * * @param manager Entry point to AzureSphereManager. */ - public static void productsGenerateDefaultDeviceGroups( - com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .products() - .generateDefaultDeviceGroups( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE); + public static void + productsGenerateDefaultDeviceGroups(com.azure.resourcemanager.sphere.AzureSphereManager manager) { + manager.products().generateDefaultDeviceGroups("MyResourceGroup1", "MyCatalog1", "MyProduct1", + com.azure.core.util.Context.NONE); } } ``` @@ -1049,20 +1084,22 @@ public final class ProductsGenerateDefaultDeviceGroupsSamples { ### Products_Get ```java -/** Samples for Products Get. */ +/** + * Samples for Products Get. + */ public final class ProductsGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetProduct.json */ /** * Sample code: Products_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .products() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE); + manager.products().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + com.azure.core.util.Context.NONE); } } ``` @@ -1070,14 +1107,17 @@ public final class ProductsGetSamples { ### Products_ListByCatalog ```java -/** Samples for Products ListByCatalog. */ +/** + * Samples for Products ListByCatalog. + */ public final class ProductsListByCatalogSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetProducts.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetProducts.json */ /** * Sample code: Products_ListByCatalog. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsListByCatalog(com.azure.resourcemanager.sphere.AzureSphereManager manager) { @@ -1091,22 +1131,23 @@ public final class ProductsListByCatalogSamples { ```java import com.azure.resourcemanager.sphere.models.Product; -/** Samples for Products Update. */ +/** + * Samples for Products Update. + */ public final class ProductsUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchProduct.json */ /** * Sample code: Products_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - Product resource = - manager - .products() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE) - .getValue(); + Product resource = manager.products() + .getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE) + .getValue(); resource.update().apply(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/pom.xml b/sdk/sphere/azure-resourcemanager-sphere/pom.xml index 181c59053ef55..a63aa7bf32057 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/pom.xml +++ b/sdk/sphere/azure-resourcemanager-sphere/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-sphere - 1.0.0-beta.2 + 1.0.0 jar Microsoft Azure SDK for AzureSphere Management - This package contains Microsoft Azure SDK for AzureSphere Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Azure Sphere resource management API. Package tag package-2022-09-01-preview. + This package contains Microsoft Azure SDK for AzureSphere Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Azure Sphere resource management API. Package tag package-2024-04-01. https://github.com/Azure/azure-sdk-for-java @@ -87,8 +87,6 @@ 4.11.0 test - - net.bytebuddy byte-buddy diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/AzureSphereManager.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/AzureSphereManager.java index 6b49b7f12dc7d..e13e0d5d2607b 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/AzureSphereManager.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/AzureSphereManager.java @@ -23,8 +23,8 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.core.util.Configuration; import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.sphere.fluent.AzureSphereManagementClient; -import com.azure.resourcemanager.sphere.implementation.AzureSphereManagementClientBuilder; +import com.azure.resourcemanager.sphere.fluent.AzureSphereMgmtClient; +import com.azure.resourcemanager.sphere.implementation.AzureSphereMgmtClientBuilder; import com.azure.resourcemanager.sphere.implementation.CatalogsImpl; import com.azure.resourcemanager.sphere.implementation.CertificatesImpl; import com.azure.resourcemanager.sphere.implementation.DeploymentsImpl; @@ -48,7 +48,10 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to AzureSphereManager. Azure Sphere resource management API. */ +/** + * Entry point to AzureSphereManager. + * Azure Sphere resource management API. + */ public final class AzureSphereManager { private Operations operations; @@ -66,23 +69,19 @@ public final class AzureSphereManager { private Devices devices; - private final AzureSphereManagementClient clientObject; + private final AzureSphereMgmtClient clientObject; private AzureSphereManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new AzureSphereManagementClientBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new AzureSphereMgmtClientBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of AzureSphere service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the AzureSphere service API instance. @@ -95,7 +94,7 @@ public static AzureSphereManager authenticate(TokenCredential credential, AzureP /** * Creates an instance of AzureSphere service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the AzureSphere service API instance. @@ -108,14 +107,16 @@ public static AzureSphereManager authenticate(HttpPipeline httpPipeline, AzurePr /** * Gets a Configurable instance that can be used to create AzureSphereManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new AzureSphereManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -187,8 +188,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -205,8 +206,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -226,21 +227,12 @@ public AzureSphereManager authenticate(TokenCredential credential, AzureProfile Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.sphere") - .append("/") - .append("1.0.0-beta.1"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.sphere").append("/") + .append("1.0.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -259,38 +251,25 @@ public AzureSphereManager authenticate(TokenCredential credential, AzureProfile policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new AzureSphereManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -302,7 +281,7 @@ public Operations operations() { /** * Gets the resource collection API of Catalogs. It manages Catalog. - * + * * @return Resource collection API of Catalogs. */ public Catalogs catalogs() { @@ -314,7 +293,7 @@ public Catalogs catalogs() { /** * Gets the resource collection API of Certificates. - * + * * @return Resource collection API of Certificates. */ public Certificates certificates() { @@ -326,7 +305,7 @@ public Certificates certificates() { /** * Gets the resource collection API of Images. It manages Image. - * + * * @return Resource collection API of Images. */ public Images images() { @@ -338,7 +317,7 @@ public Images images() { /** * Gets the resource collection API of Products. It manages Product. - * + * * @return Resource collection API of Products. */ public Products products() { @@ -350,7 +329,7 @@ public Products products() { /** * Gets the resource collection API of DeviceGroups. It manages DeviceGroup. - * + * * @return Resource collection API of DeviceGroups. */ public DeviceGroups deviceGroups() { @@ -362,7 +341,7 @@ public DeviceGroups deviceGroups() { /** * Gets the resource collection API of Deployments. It manages Deployment. - * + * * @return Resource collection API of Deployments. */ public Deployments deployments() { @@ -374,7 +353,7 @@ public Deployments deployments() { /** * Gets the resource collection API of Devices. It manages Device. - * + * * @return Resource collection API of Devices. */ public Devices devices() { @@ -385,10 +364,12 @@ public Devices devices() { } /** - * @return Wrapped service client AzureSphereManagementClient providing direct access to the underlying - * auto-generated API implementation, based on Azure REST API. + * Gets wrapped service client AzureSphereMgmtClient providing direct access to the underlying auto-generated API + * implementation, based on Azure REST API. + * + * @return Wrapped service client AzureSphereMgmtClient. */ - public AzureSphereManagementClient serviceClient() { + public AzureSphereMgmtClient serviceClient() { return this.clientObject; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/AzureSphereManagementClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/AzureSphereMgmtClient.java similarity index 91% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/AzureSphereManagementClient.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/AzureSphereMgmtClient.java index 82f217261953a..5a4f1d01f66cf 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/AzureSphereManagementClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/AzureSphereMgmtClient.java @@ -7,95 +7,97 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for AzureSphereManagementClient class. */ -public interface AzureSphereManagementClient { +/** + * The interface for AzureSphereMgmtClient class. + */ +public interface AzureSphereMgmtClient { /** * Gets The ID of the target subscription. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the CatalogsClient object to access its operations. - * + * * @return the CatalogsClient object. */ CatalogsClient getCatalogs(); /** * Gets the CertificatesClient object to access its operations. - * + * * @return the CertificatesClient object. */ CertificatesClient getCertificates(); /** * Gets the ImagesClient object to access its operations. - * + * * @return the ImagesClient object. */ ImagesClient getImages(); /** * Gets the ProductsClient object to access its operations. - * + * * @return the ProductsClient object. */ ProductsClient getProducts(); /** * Gets the DeviceGroupsClient object to access its operations. - * + * * @return the DeviceGroupsClient object. */ DeviceGroupsClient getDeviceGroups(); /** * Gets the DeploymentsClient object to access its operations. - * + * * @return the DeploymentsClient object. */ DeploymentsClient getDeployments(); /** * Gets the DevicesClient object to access its operations. - * + * * @return the DevicesClient object. */ DevicesClient getDevices(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CatalogsClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CatalogsClient.java index b06050790d6c4..d66f6cffc0bce 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CatalogsClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CatalogsClient.java @@ -12,19 +12,22 @@ import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceInsightInner; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.CatalogUpdate; import com.azure.resourcemanager.sphere.models.ListDeviceGroupsRequest; -/** An instance of this class provides access to all the operations defined in CatalogsClient. */ +/** + * An instance of this class provides access to all the operations defined in CatalogsClient. + */ public interface CatalogsClient { /** * List Catalog resources by subscription ID. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation as paginated response with {@link PagedIterable}. @@ -34,7 +37,7 @@ public interface CatalogsClient { /** * List Catalog resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -46,7 +49,7 @@ public interface CatalogsClient { /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -58,7 +61,7 @@ public interface CatalogsClient { /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -71,7 +74,7 @@ public interface CatalogsClient { /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -81,12 +84,12 @@ public interface CatalogsClient { * @return a Catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String catalogName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String catalogName, + Context context); /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -99,7 +102,7 @@ Response getByResourceGroupWithResponse( /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -109,12 +112,12 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, CatalogInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, CatalogInner resource); + SyncPoller, CatalogInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + CatalogInner resource); /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -125,12 +128,12 @@ SyncPoller, CatalogInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, CatalogInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, CatalogInner resource, Context context); + SyncPoller, CatalogInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + CatalogInner resource, Context context); /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -144,7 +147,7 @@ SyncPoller, CatalogInner> beginCreateOrUpdate( /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -159,7 +162,7 @@ SyncPoller, CatalogInner> beginCreateOrUpdate( /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -170,12 +173,12 @@ SyncPoller, CatalogInner> beginCreateOrUpdate( * @return an Azure Sphere catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, String catalogName, CatalogUpdate properties, Context context); + Response updateWithResponse(String resourceGroupName, String catalogName, CatalogUpdate properties, + Context context); /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -189,7 +192,7 @@ Response updateWithResponse( /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -202,7 +205,7 @@ Response updateWithResponse( /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -216,7 +219,7 @@ Response updateWithResponse( /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -228,7 +231,7 @@ Response updateWithResponse( /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -241,7 +244,7 @@ Response updateWithResponse( /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -251,12 +254,12 @@ Response updateWithResponse( * @return response to the action call for count devices in a catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response countDevicesWithResponse( - String resourceGroupName, String catalogName, Context context); + Response countDevicesWithResponse(String resourceGroupName, String catalogName, + Context context); /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -265,11 +268,11 @@ Response countDevicesWithResponse( * @return response to the action call for count devices in a catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - CountDeviceResponseInner countDevices(String resourceGroupName, String catalogName); + CountDevicesResponseInner countDevices(String resourceGroupName, String catalogName); /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -282,7 +285,7 @@ Response countDevicesWithResponse( /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -296,18 +299,12 @@ Response countDevicesWithResponse( * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listDeployments( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDeployments(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context); /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -317,12 +314,12 @@ PagedIterable listDeployments( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listDeviceGroups( - String resourceGroupName, String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest); + PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest); /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -337,19 +334,13 @@ PagedIterable listDeviceGroups( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listDeviceGroups( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize, Context context); /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -362,7 +353,7 @@ PagedIterable listDeviceGroups( /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -376,18 +367,12 @@ PagedIterable listDeviceGroups( * @return paged collection of DeviceInsight items as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listDeviceInsights( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDeviceInsights(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context); /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -400,7 +385,7 @@ PagedIterable listDeviceInsights( /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -414,12 +399,64 @@ PagedIterable listDeviceInsights( * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listDevices( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDevices(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginUploadImage(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginUploadImage(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest, Context context); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest, Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CertificatesClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CertificatesClient.java index a25275cddf218..b76a551fcd5e8 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CertificatesClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/CertificatesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.sphere.fluent.models.ProofOfPossessionNonceResponseInner; import com.azure.resourcemanager.sphere.models.ProofOfPossessionNonceRequest; -/** An instance of this class provides access to all the operations defined in CertificatesClient. */ +/** + * An instance of this class provides access to all the operations defined in CertificatesClient. + */ public interface CertificatesClient { /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -31,7 +33,7 @@ public interface CertificatesClient { /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -45,18 +47,12 @@ public interface CertificatesClient { * @return the response of a Certificate list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context); /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -67,12 +63,12 @@ PagedIterable listByCatalog( * @return a Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String serialNumber, + Context context); /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -86,7 +82,7 @@ Response getWithResponse( /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -97,12 +93,12 @@ Response getWithResponse( * @return the certificate chain response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response retrieveCertChainWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context); + Response retrieveCertChainWithResponse(String resourceGroupName, String catalogName, + String serialNumber, Context context); /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -116,7 +112,7 @@ Response retrieveCertChainWithResponse( /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -128,16 +124,13 @@ Response retrieveCertChainWithResponse( * @return the proof of possession nonce along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response retrieveProofOfPossessionNonceWithResponse( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, + Response retrieveProofOfPossessionNonceWithResponse(String resourceGroupName, + String catalogName, String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, Context context); /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -148,9 +141,6 @@ Response retrieveProofOfPossessionNonceWith * @return the proof of possession nonce. */ @ServiceMethod(returns = ReturnType.SINGLE) - ProofOfPossessionNonceResponseInner retrieveProofOfPossessionNonce( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest); + ProofOfPossessionNonceResponseInner retrieveProofOfPossessionNonce(String resourceGroupName, String catalogName, + String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeploymentsClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeploymentsClient.java index 0d59f94edfa18..c02caa703a712 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeploymentsClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeploymentsClient.java @@ -13,12 +13,14 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; -/** An instance of this class provides access to all the operations defined in DeploymentsClient. */ +/** + * An instance of this class provides access to all the operations defined in DeploymentsClient. + */ public interface DeploymentsClient { /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -29,13 +31,13 @@ public interface DeploymentsClient { * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -51,27 +53,19 @@ PagedIterable listByDeviceGroup( * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByDeviceGroup( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context); /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -79,47 +73,38 @@ PagedIterable listByDeviceGroup( * @return a Deployment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, Context context); /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Deployment. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeploymentInner get( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + DeploymentInner get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName); /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -127,24 +112,20 @@ DeploymentInner get( * @return the {@link SyncPoller} for polling of an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeploymentInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, + SyncPoller, DeploymentInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource); /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -153,25 +134,20 @@ SyncPoller, DeploymentInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeploymentInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, + SyncPoller, DeploymentInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource, Context context); /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -179,24 +155,19 @@ SyncPoller, DeploymentInner> beginCreateOrUpdate( * @return an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeploymentInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource); + DeploymentInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, DeploymentInner resource); /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -205,48 +176,38 @@ DeploymentInner createOrUpdate( * @return an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeploymentInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, - Context context); + DeploymentInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, DeploymentInner resource, Context context); /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName); /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -254,57 +215,43 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, Context context); /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName); /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deploymentName, Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeviceGroupsClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeviceGroupsClient.java index 4e210133bb5c0..5de380b9aaf79 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeviceGroupsClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DeviceGroupsClient.java @@ -11,17 +11,19 @@ import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.models.ClaimDevicesRequest; import com.azure.resourcemanager.sphere.models.DeviceGroupUpdate; -/** An instance of this class provides access to all the operations defined in DeviceGroupsClient. */ +/** + * An instance of this class provides access to all the operations defined in DeviceGroupsClient. + */ public interface DeviceGroupsClient { /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -36,7 +38,7 @@ public interface DeviceGroupsClient { /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -51,20 +53,13 @@ public interface DeviceGroupsClient { * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByProduct( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByProduct(String resourceGroupName, String catalogName, String productName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context); /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -76,13 +71,13 @@ PagedIterable listByProduct( * @return a DeviceGroup along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context); /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -98,7 +93,7 @@ Response getWithResponse( /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -110,17 +105,13 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceGroupInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource); + SyncPoller, DeviceGroupInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource); /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -133,18 +124,13 @@ SyncPoller, DeviceGroupInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceGroupInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context); + SyncPoller, DeviceGroupInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource, Context context); /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -156,17 +142,13 @@ SyncPoller, DeviceGroupInner> beginCreateOrUpdate( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceGroupInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource); + DeviceGroupInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupInner resource); /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -179,18 +161,13 @@ DeviceGroupInner createOrUpdate( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceGroupInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context); + DeviceGroupInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupInner resource, Context context); /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -202,17 +179,13 @@ DeviceGroupInner createOrUpdate( * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceGroupInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties); + SyncPoller, DeviceGroupInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, DeviceGroupUpdate properties); /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -225,18 +198,13 @@ SyncPoller, DeviceGroupInner> beginUpdate( * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceGroupInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context); + SyncPoller, DeviceGroupInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, DeviceGroupUpdate properties, Context context); /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -248,17 +216,13 @@ SyncPoller, DeviceGroupInner> beginUpdate( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceGroupInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + DeviceGroupInner update(String resourceGroupName, String catalogName, String productName, String deviceGroupName, DeviceGroupUpdate properties); /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -271,18 +235,13 @@ DeviceGroupInner update( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceGroupInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context); + DeviceGroupInner update(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + DeviceGroupUpdate properties, Context context); /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -293,13 +252,13 @@ DeviceGroupInner update( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -311,13 +270,13 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context); /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -332,7 +291,7 @@ SyncPoller, Void> beginDelete( /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -343,13 +302,13 @@ SyncPoller, Void> beginDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + Context context); /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -361,17 +320,13 @@ void delete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginClaimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest); + SyncPoller, Void> beginClaimDevices(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest); /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -384,18 +339,13 @@ SyncPoller, Void> beginClaimDevices( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginClaimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context); + SyncPoller, Void> beginClaimDevices(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest, Context context); /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -406,17 +356,13 @@ SyncPoller, Void> beginClaimDevices( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest); /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -428,18 +374,13 @@ void claimDevices( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context); + void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + ClaimDevicesRequest claimDevicesRequest, Context context); /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -451,13 +392,13 @@ void claimDevices( * @return response to the action call for count devices in a catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context); /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -468,6 +409,6 @@ Response countDevicesWithResponse( * @return response to the action call for count devices in a catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - CountDeviceResponseInner countDevices( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + CountDevicesResponseInner countDevices(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DevicesClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DevicesClient.java index b053aa5cadace..9e6322247bf44 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DevicesClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/DevicesClient.java @@ -16,12 +16,14 @@ import com.azure.resourcemanager.sphere.models.DeviceUpdate; import com.azure.resourcemanager.sphere.models.GenerateCapabilityImageRequest; -/** An instance of this class provides access to all the operations defined in DevicesClient. */ +/** + * An instance of this class provides access to all the operations defined in DevicesClient. + */ public interface DevicesClient { /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -32,13 +34,13 @@ public interface DevicesClient { * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -50,13 +52,13 @@ PagedIterable listByDeviceGroup( * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context); /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -69,18 +71,13 @@ PagedIterable listByDeviceGroup( * @return a Device along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, Context context); /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -92,13 +89,13 @@ Response getWithResponse( * @return a Device. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceInner get( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName); + DeviceInner get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName); /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -111,18 +108,13 @@ DeviceInner get( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource); + SyncPoller, DeviceInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceInner resource); /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -136,19 +128,13 @@ SyncPoller, DeviceInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, - Context context); + SyncPoller, DeviceInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceInner resource, Context context); /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -161,18 +147,13 @@ SyncPoller, DeviceInner> beginCreateOrUpdate( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource); + DeviceInner createOrUpdate(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, DeviceInner resource); /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -186,19 +167,13 @@ DeviceInner createOrUpdate( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, - Context context); + DeviceInner createOrUpdate(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, DeviceInner resource, Context context); /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -211,18 +186,13 @@ DeviceInner createOrUpdate( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties); + SyncPoller, DeviceInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceUpdate properties); /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -236,19 +206,13 @@ SyncPoller, DeviceInner> beginUpdate( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DeviceInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, - Context context); + SyncPoller, DeviceInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceUpdate properties, Context context); /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -261,18 +225,13 @@ SyncPoller, DeviceInner> beginUpdate( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties); + DeviceInner update(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, DeviceUpdate properties); /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -286,18 +245,12 @@ DeviceInner update( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, - Context context); + DeviceInner update(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, DeviceUpdate properties, Context context); /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -309,12 +262,12 @@ DeviceInner update( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName); /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -327,17 +280,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, Context context); /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -348,12 +296,12 @@ SyncPoller, Void> beginDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName); /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -365,18 +313,13 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, Context context); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -390,18 +333,13 @@ void delete( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, SignedCapabilityImageResponseInner> - beginGenerateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest); + beginGenerateCapabilityImage(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -416,19 +354,14 @@ void delete( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, SignedCapabilityImageResponseInner> - beginGenerateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, + beginGenerateCapabilityImage(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -441,18 +374,14 @@ void delete( * @return signed device capability image response. */ @ServiceMethod(returns = ReturnType.SINGLE) - SignedCapabilityImageResponseInner generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, + SignedCapabilityImageResponseInner generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -466,12 +395,7 @@ SignedCapabilityImageResponseInner generateCapabilityImage( * @return signed device capability image response. */ @ServiceMethod(returns = ReturnType.SINGLE) - SignedCapabilityImageResponseInner generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - Context context); + SignedCapabilityImageResponseInner generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, + GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ImagesClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ImagesClient.java index 205ceca662eb9..de16245b13d1f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ImagesClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ImagesClient.java @@ -13,11 +13,13 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; -/** An instance of this class provides access to all the operations defined in ImagesClient. */ +/** + * An instance of this class provides access to all the operations defined in ImagesClient. + */ public interface ImagesClient { /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -30,7 +32,7 @@ public interface ImagesClient { /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -44,21 +46,15 @@ public interface ImagesClient { * @return the response of a Image list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context); /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -66,15 +62,15 @@ PagedIterable listByCatalog( * @return a Image along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String catalogName, String imageName, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String imageName, + Context context); /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -85,10 +81,10 @@ Response getWithResponse( /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -96,15 +92,15 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ImageInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource); + SyncPoller, ImageInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + String imageName, ImageInner resource); /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,15 +109,15 @@ SyncPoller, ImageInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ImageInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context); + SyncPoller, ImageInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + String imageName, ImageInner resource, Context context); /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -133,10 +129,10 @@ SyncPoller, ImageInner> beginCreateOrUpdate( /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -145,15 +141,15 @@ SyncPoller, ImageInner> beginCreateOrUpdate( * @return an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ImageInner createOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context); + ImageInner createOrUpdate(String resourceGroupName, String catalogName, String imageName, ImageInner resource, + Context context); /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -164,10 +160,10 @@ ImageInner createOrUpdate( /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -175,15 +171,15 @@ ImageInner createOrUpdate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String imageName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String imageName, + Context context); /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -193,10 +189,10 @@ SyncPoller, Void> beginDelete( /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/OperationsClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/OperationsClient.java index 628bd10d47fca..fa904d1972694 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/OperationsClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/OperationsClient.java @@ -10,28 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * List the operations for the provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ProductsClient.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ProductsClient.java index bcf5c9ad48259..684c9df9b38b4 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ProductsClient.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/ProductsClient.java @@ -11,16 +11,18 @@ import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; import com.azure.resourcemanager.sphere.models.ProductUpdate; -/** An instance of this class provides access to all the operations defined in ProductsClient. */ +/** + * An instance of this class provides access to all the operations defined in ProductsClient. + */ public interface ProductsClient { /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -33,7 +35,7 @@ public interface ProductsClient { /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -47,7 +49,7 @@ public interface ProductsClient { /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -58,12 +60,12 @@ public interface ProductsClient { * @return a Product along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String catalogName, String productName, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + Context context); /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -88,12 +90,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ProductInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource); + SyncPoller, ProductInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + String productName, ProductInner resource); /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -105,12 +107,12 @@ SyncPoller, ProductInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ProductInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context); + SyncPoller, ProductInner> beginCreateOrUpdate(String resourceGroupName, String catalogName, + String productName, ProductInner resource, Context context); /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -121,12 +123,12 @@ SyncPoller, ProductInner> beginCreateOrUpdate( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ProductInner createOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource); + ProductInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + ProductInner resource); /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -138,12 +140,12 @@ ProductInner createOrUpdate( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ProductInner createOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context); + ProductInner createOrUpdate(String resourceGroupName, String catalogName, String productName, ProductInner resource, + Context context); /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -154,12 +156,12 @@ ProductInner createOrUpdate( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ProductInner> beginUpdate( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties); + SyncPoller, ProductInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, ProductUpdate properties); /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -171,12 +173,12 @@ SyncPoller, ProductInner> beginUpdate( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ProductInner> beginUpdate( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context); + SyncPoller, ProductInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, ProductUpdate properties, Context context); /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -191,7 +193,7 @@ SyncPoller, ProductInner> beginUpdate( /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -203,12 +205,12 @@ SyncPoller, ProductInner> beginUpdate( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ProductInner update( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context); + ProductInner update(String resourceGroupName, String catalogName, String productName, ProductUpdate properties, + Context context); /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -222,7 +224,7 @@ ProductInner update( /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -233,12 +235,12 @@ ProductInner update( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, String productName, + Context context); /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -251,7 +253,7 @@ SyncPoller, Void> beginDelete( /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -266,7 +268,7 @@ SyncPoller, Void> beginDelete( /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -277,13 +279,13 @@ SyncPoller, Void> beginDelete( * @return response to the action call for count devices in a catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, Context context); + Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, Context context); /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -293,12 +295,12 @@ Response countDevicesWithResponse( * @return response to the action call for count devices in a catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - CountDeviceResponseInner countDevices(String resourceGroupName, String catalogName, String productName); + CountDevicesResponseInner countDevices(String resourceGroupName, String catalogName, String productName); /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -308,13 +310,13 @@ Response countDevicesWithResponse( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName); + PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName); /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -325,6 +327,6 @@ PagedIterable generateDefaultDeviceGroups( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName, Context context); + PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName, Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogInner.java index a639e4b67f15c..ccedd272070fc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogInner.java @@ -7,18 +7,20 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; import com.azure.core.management.SystemData; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.resourcemanager.sphere.models.CatalogProperties; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** An Azure Sphere catalog. */ +/** + * An Azure Sphere catalog. + */ @Fluent public final class CatalogInner extends Resource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private CatalogProperties innerProperties; + private CatalogProperties properties; /* * Azure Resource Manager metadata containing createdBy and modifiedBy information. @@ -26,59 +28,67 @@ public final class CatalogInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of CatalogInner class. */ + /** + * Creates an instance of CatalogInner class. + */ public CatalogInner() { } /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. + */ + public CatalogProperties properties() { + return this.properties; + } + + /** + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. + * @return the CatalogInner object itself. */ - private CatalogProperties innerProperties() { - return this.innerProperties; + public CatalogInner withProperties(CatalogProperties properties) { + this.properties = properties; + return this; } /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CatalogInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CatalogInner withTags(Map tags) { super.withTags(tags); return this; } - /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. - */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateChainResponseInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateChainResponseInner.java index e5e299011709c..645198a6ac0db 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateChainResponseInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateChainResponseInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The certificate chain response. */ +/** + * The certificate chain response. + */ @Immutable public final class CertificateChainResponseInner { /* @@ -16,13 +18,15 @@ public final class CertificateChainResponseInner { @JsonProperty(value = "certificateChain", access = JsonProperty.Access.WRITE_ONLY) private String certificateChain; - /** Creates an instance of CertificateChainResponseInner class. */ + /** + * Creates an instance of CertificateChainResponseInner class. + */ public CertificateChainResponseInner() { } /** * Get the certificateChain property: The certificate chain. - * + * * @return the certificateChain value. */ public String certificateChain() { @@ -31,7 +35,7 @@ public String certificateChain() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateInner.java index 9f50de6cf610b..8cb6c15c3e929 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateInner.java @@ -4,22 +4,22 @@ package com.azure.resourcemanager.sphere.fluent.models; -import com.azure.core.annotation.Immutable; +import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; import com.azure.core.management.SystemData; -import com.azure.resourcemanager.sphere.models.CertificateStatus; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.resourcemanager.sphere.models.CertificateProperties; import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; -/** An certificate resource belonging to a catalog resource. */ -@Immutable +/** + * An certificate resource belonging to a catalog resource. + */ +@Fluent public final class CertificateInner extends ProxyResource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private CertificateProperties innerProperties; + private CertificateProperties properties; /* * Azure Resource Manager metadata containing createdBy and modifiedBy information. @@ -27,99 +27,49 @@ public final class CertificateInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of CertificateInner class. */ - public CertificateInner() { - } - - /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. - */ - private CertificateProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - - /** - * Get the certificate property: The certificate as a UTF-8 encoded base 64 string. - * - * @return the certificate value. - */ - public String certificate() { - return this.innerProperties() == null ? null : this.innerProperties().certificate(); - } - - /** - * Get the status property: The certificate status. - * - * @return the status value. - */ - public CertificateStatus status() { - return this.innerProperties() == null ? null : this.innerProperties().status(); - } - - /** - * Get the subject property: The certificate subject. - * - * @return the subject value. - */ - public String subject() { - return this.innerProperties() == null ? null : this.innerProperties().subject(); - } - /** - * Get the thumbprint property: The certificate thumbprint. - * - * @return the thumbprint value. + * Creates an instance of CertificateInner class. */ - public String thumbprint() { - return this.innerProperties() == null ? null : this.innerProperties().thumbprint(); + public CertificateInner() { } /** - * Get the expiryUtc property: The certificate expiry date. - * - * @return the expiryUtc value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - public OffsetDateTime expiryUtc() { - return this.innerProperties() == null ? null : this.innerProperties().expiryUtc(); + public CertificateProperties properties() { + return this.properties; } /** - * Get the notBeforeUtc property: The certificate not before date. - * - * @return the notBeforeUtc value. + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. + * @return the CertificateInner object itself. */ - public OffsetDateTime notBeforeUtc() { - return this.innerProperties() == null ? null : this.innerProperties().notBeforeUtc(); + public CertificateInner withProperties(CertificateProperties properties) { + this.properties = properties; + return this; } /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CountDeviceResponseInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CountDevicesResponseInner.java similarity index 62% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CountDeviceResponseInner.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CountDevicesResponseInner.java index 80c30909e9da4..04b3280e7b55e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CountDeviceResponseInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CountDevicesResponseInner.java @@ -7,23 +7,29 @@ import com.azure.core.annotation.Fluent; import com.azure.resourcemanager.sphere.models.CountElementsResponse; -/** Response to the action call for count devices in a catalog. */ +/** + * Response to the action call for count devices in a catalog. + */ @Fluent -public final class CountDeviceResponseInner extends CountElementsResponse { - /** Creates an instance of CountDeviceResponseInner class. */ - public CountDeviceResponseInner() { +public final class CountDevicesResponseInner extends CountElementsResponse { + /** + * Creates an instance of CountDevicesResponseInner class. + */ + public CountDevicesResponseInner() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override - public CountDeviceResponseInner withValue(int value) { + public CountDevicesResponseInner withValue(int value) { super.withValue(value); return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentInner.java index ed70be5ad59c9..cc20d198b15b6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentInner.java @@ -6,105 +6,70 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.sphere.models.DeploymentProperties; import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; -import java.util.List; -/** An deployment resource belonging to a device group resource. */ +/** + * An deployment resource belonging to a device group resource. + */ @Fluent public final class DeploymentInner extends ProxyResource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private DeploymentProperties innerProperties; + private DeploymentProperties properties; - /** Creates an instance of DeploymentInner class. */ - public DeploymentInner() { - } - - /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. */ - private DeploymentProperties innerProperties() { - return this.innerProperties; - } + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; /** - * Get the deploymentId property: Deployment ID. - * - * @return the deploymentId value. + * Creates an instance of DeploymentInner class. */ - public String deploymentId() { - return this.innerProperties() == null ? null : this.innerProperties().deploymentId(); - } - - /** - * Set the deploymentId property: Deployment ID. - * - * @param deploymentId the deploymentId value to set. - * @return the DeploymentInner object itself. - */ - public DeploymentInner withDeploymentId(String deploymentId) { - if (this.innerProperties() == null) { - this.innerProperties = new DeploymentProperties(); - } - this.innerProperties().withDeploymentId(deploymentId); - return this; + public DeploymentInner() { } /** - * Get the deployedImages property: Images deployed. - * - * @return the deployedImages value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - public List deployedImages() { - return this.innerProperties() == null ? null : this.innerProperties().deployedImages(); + public DeploymentProperties properties() { + return this.properties; } /** - * Set the deployedImages property: Images deployed. - * - * @param deployedImages the deployedImages value to set. + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. * @return the DeploymentInner object itself. */ - public DeploymentInner withDeployedImages(List deployedImages) { - if (this.innerProperties() == null) { - this.innerProperties = new DeploymentProperties(); - } - this.innerProperties().withDeployedImages(deployedImages); + public DeploymentInner withProperties(DeploymentProperties properties) { + this.properties = properties; return this; } /** - * Get the deploymentDateUtc property: Deployment date UTC. - * - * @return the deploymentDateUtc value. - */ - public OffsetDateTime deploymentDateUtc() { - return this.innerProperties() == null ? null : this.innerProperties().deploymentDateUtc(); - } - - /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupInner.java index c09d49d6624ea..5bce6f135bb14 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupInner.java @@ -6,176 +6,70 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; -import com.azure.resourcemanager.sphere.models.OSFeedType; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; -import com.azure.resourcemanager.sphere.models.UpdatePolicy; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** An device group resource belonging to a product resource. */ +/** + * An device group resource belonging to a product resource. + */ @Fluent public final class DeviceGroupInner extends ProxyResource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private DeviceGroupProperties innerProperties; + private DeviceGroupProperties properties; - /** Creates an instance of DeviceGroupInner class. */ - public DeviceGroupInner() { - } - - /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. - */ - private DeviceGroupProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the description property: Description of the device group. - * - * @return the description value. - */ - public String description() { - return this.innerProperties() == null ? null : this.innerProperties().description(); - } - - /** - * Set the description property: Description of the device group. - * - * @param description the description value to set. - * @return the DeviceGroupInner object itself. - */ - public DeviceGroupInner withDescription(String description) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupProperties(); - } - this.innerProperties().withDescription(description); - return this; - } - - /** - * Get the osFeedType property: Operating system feed type of the device group. - * - * @return the osFeedType value. - */ - public OSFeedType osFeedType() { - return this.innerProperties() == null ? null : this.innerProperties().osFeedType(); - } - - /** - * Set the osFeedType property: Operating system feed type of the device group. - * - * @param osFeedType the osFeedType value to set. - * @return the DeviceGroupInner object itself. - */ - public DeviceGroupInner withOsFeedType(OSFeedType osFeedType) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupProperties(); - } - this.innerProperties().withOsFeedType(osFeedType); - return this; - } - - /** - * Get the updatePolicy property: Update policy of the device group. - * - * @return the updatePolicy value. - */ - public UpdatePolicy updatePolicy() { - return this.innerProperties() == null ? null : this.innerProperties().updatePolicy(); - } - - /** - * Set the updatePolicy property: Update policy of the device group. - * - * @param updatePolicy the updatePolicy value to set. - * @return the DeviceGroupInner object itself. - */ - public DeviceGroupInner withUpdatePolicy(UpdatePolicy updatePolicy) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupProperties(); - } - this.innerProperties().withUpdatePolicy(updatePolicy); - return this; - } - - /** - * Get the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * - * @return the allowCrashDumpsCollection value. + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. */ - public AllowCrashDumpCollection allowCrashDumpsCollection() { - return this.innerProperties() == null ? null : this.innerProperties().allowCrashDumpsCollection(); - } + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; /** - * Set the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * - * @param allowCrashDumpsCollection the allowCrashDumpsCollection value to set. - * @return the DeviceGroupInner object itself. + * Creates an instance of DeviceGroupInner class. */ - public DeviceGroupInner withAllowCrashDumpsCollection(AllowCrashDumpCollection allowCrashDumpsCollection) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupProperties(); - } - this.innerProperties().withAllowCrashDumpsCollection(allowCrashDumpsCollection); - return this; + public DeviceGroupInner() { } /** - * Get the regionalDataBoundary property: Regional data boundary for the device group. - * - * @return the regionalDataBoundary value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - public RegionalDataBoundary regionalDataBoundary() { - return this.innerProperties() == null ? null : this.innerProperties().regionalDataBoundary(); + public DeviceGroupProperties properties() { + return this.properties; } /** - * Set the regionalDataBoundary property: Regional data boundary for the device group. - * - * @param regionalDataBoundary the regionalDataBoundary value to set. + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. * @return the DeviceGroupInner object itself. */ - public DeviceGroupInner withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupProperties(); - } - this.innerProperties().withRegionalDataBoundary(regionalDataBoundary); + public DeviceGroupInner withProperties(DeviceGroupProperties properties) { + this.properties = properties; return this; } /** - * Get the hasDeployment property: Deployment status for the device group. - * - * @return the hasDeployment value. - */ - public Boolean hasDeployment() { - return this.innerProperties() == null ? null : this.innerProperties().hasDeployment(); - } - - /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInner.java index 4d6a8dfe174b8..5e37559821520 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInner.java @@ -6,117 +6,70 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.sphere.models.DeviceProperties; import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; -/** An device resource belonging to a device group resource. */ +/** + * An device resource belonging to a device group resource. + */ @Fluent public final class DeviceInner extends ProxyResource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private DeviceProperties innerProperties; + private DeviceProperties properties; - /** Creates an instance of DeviceInner class. */ - public DeviceInner() { - } + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. + * Creates an instance of DeviceInner class. */ - private DeviceProperties innerProperties() { - return this.innerProperties; + public DeviceInner() { } /** - * Get the deviceId property: Device ID. - * - * @return the deviceId value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - public String deviceId() { - return this.innerProperties() == null ? null : this.innerProperties().deviceId(); + public DeviceProperties properties() { + return this.properties; } /** - * Set the deviceId property: Device ID. - * - * @param deviceId the deviceId value to set. + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. * @return the DeviceInner object itself. */ - public DeviceInner withDeviceId(String deviceId) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceProperties(); - } - this.innerProperties().withDeviceId(deviceId); + public DeviceInner withProperties(DeviceProperties properties) { + this.properties = properties; return this; } /** - * Get the chipSku property: SKU of the chip. - * - * @return the chipSku value. - */ - public String chipSku() { - return this.innerProperties() == null ? null : this.innerProperties().chipSku(); - } - - /** - * Get the lastAvailableOsVersion property: OS version available for installation when update requested. - * - * @return the lastAvailableOsVersion value. - */ - public String lastAvailableOsVersion() { - return this.innerProperties() == null ? null : this.innerProperties().lastAvailableOsVersion(); - } - - /** - * Get the lastInstalledOsVersion property: OS version running on device when update requested. - * - * @return the lastInstalledOsVersion value. - */ - public String lastInstalledOsVersion() { - return this.innerProperties() == null ? null : this.innerProperties().lastInstalledOsVersion(); - } - - /** - * Get the lastOsUpdateUtc property: Time when update requested and new OS version available. - * - * @return the lastOsUpdateUtc value. - */ - public OffsetDateTime lastOsUpdateUtc() { - return this.innerProperties() == null ? null : this.innerProperties().lastOsUpdateUtc(); - } - - /** - * Get the lastUpdateRequestUtc property: Time when update was last requested. - * - * @return the lastUpdateRequestUtc value. - */ - public OffsetDateTime lastUpdateRequestUtc() { - return this.innerProperties() == null ? null : this.innerProperties().lastUpdateRequestUtc(); - } - - /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInsightInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInsightInner.java index 4290335b8721a..26e8172ee6491 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInsightInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceInsightInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Device insight report. */ +/** + * Device insight report. + */ @Fluent public final class DeviceInsightInner { /* @@ -60,13 +62,15 @@ public final class DeviceInsightInner { @JsonProperty(value = "eventCount", required = true) private int eventCount; - /** Creates an instance of DeviceInsightInner class. */ + /** + * Creates an instance of DeviceInsightInner class. + */ public DeviceInsightInner() { } /** * Get the deviceId property: Device ID. - * + * * @return the deviceId value. */ public String deviceId() { @@ -75,7 +79,7 @@ public String deviceId() { /** * Set the deviceId property: Device ID. - * + * * @param deviceId the deviceId value to set. * @return the DeviceInsightInner object itself. */ @@ -86,7 +90,7 @@ public DeviceInsightInner withDeviceId(String deviceId) { /** * Get the description property: Event description. - * + * * @return the description value. */ public String description() { @@ -95,7 +99,7 @@ public String description() { /** * Set the description property: Event description. - * + * * @param description the description value to set. * @return the DeviceInsightInner object itself. */ @@ -106,7 +110,7 @@ public DeviceInsightInner withDescription(String description) { /** * Get the startTimestampUtc property: Event start timestamp. - * + * * @return the startTimestampUtc value. */ public OffsetDateTime startTimestampUtc() { @@ -115,7 +119,7 @@ public OffsetDateTime startTimestampUtc() { /** * Set the startTimestampUtc property: Event start timestamp. - * + * * @param startTimestampUtc the startTimestampUtc value to set. * @return the DeviceInsightInner object itself. */ @@ -126,7 +130,7 @@ public DeviceInsightInner withStartTimestampUtc(OffsetDateTime startTimestampUtc /** * Get the endTimestampUtc property: Event end timestamp. - * + * * @return the endTimestampUtc value. */ public OffsetDateTime endTimestampUtc() { @@ -135,7 +139,7 @@ public OffsetDateTime endTimestampUtc() { /** * Set the endTimestampUtc property: Event end timestamp. - * + * * @param endTimestampUtc the endTimestampUtc value to set. * @return the DeviceInsightInner object itself. */ @@ -146,7 +150,7 @@ public DeviceInsightInner withEndTimestampUtc(OffsetDateTime endTimestampUtc) { /** * Get the eventCategory property: Event category. - * + * * @return the eventCategory value. */ public String eventCategory() { @@ -155,7 +159,7 @@ public String eventCategory() { /** * Set the eventCategory property: Event category. - * + * * @param eventCategory the eventCategory value to set. * @return the DeviceInsightInner object itself. */ @@ -166,7 +170,7 @@ public DeviceInsightInner withEventCategory(String eventCategory) { /** * Get the eventClass property: Event class. - * + * * @return the eventClass value. */ public String eventClass() { @@ -175,7 +179,7 @@ public String eventClass() { /** * Set the eventClass property: Event class. - * + * * @param eventClass the eventClass value to set. * @return the DeviceInsightInner object itself. */ @@ -186,7 +190,7 @@ public DeviceInsightInner withEventClass(String eventClass) { /** * Get the eventType property: Event type. - * + * * @return the eventType value. */ public String eventType() { @@ -195,7 +199,7 @@ public String eventType() { /** * Set the eventType property: Event type. - * + * * @param eventType the eventType value to set. * @return the DeviceInsightInner object itself. */ @@ -206,7 +210,7 @@ public DeviceInsightInner withEventType(String eventType) { /** * Get the eventCount property: Event count. - * + * * @return the eventCount value. */ public int eventCount() { @@ -215,7 +219,7 @@ public int eventCount() { /** * Set the eventCount property: Event count. - * + * * @param eventCount the eventCount value to set. * @return the DeviceInsightInner object itself. */ @@ -226,47 +230,37 @@ public DeviceInsightInner withEventCount(int eventCount) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (deviceId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property deviceId in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property deviceId in model DeviceInsightInner")); } if (description() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property description in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property description in model DeviceInsightInner")); } if (startTimestampUtc() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property startTimestampUtc in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property startTimestampUtc in model DeviceInsightInner")); } if (endTimestampUtc() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property endTimestampUtc in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property endTimestampUtc in model DeviceInsightInner")); } if (eventCategory() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property eventCategory in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property eventCategory in model DeviceInsightInner")); } if (eventClass() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property eventClass in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property eventClass in model DeviceInsightInner")); } if (eventType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property eventType in model DeviceInsightInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property eventType in model DeviceInsightInner")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageInner.java index 7f5a89784b1f0..d7bb5032b0f7a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageInner.java @@ -6,166 +6,70 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.resourcemanager.sphere.models.ImageType; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** An image resource belonging to a catalog resource. */ +/** + * An image resource belonging to a catalog resource. + */ @Fluent public final class ImageInner extends ProxyResource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private ImageProperties innerProperties; + private ImageProperties properties; - /** Creates an instance of ImageInner class. */ - public ImageInner() { - } - - /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. - */ - private ImageProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the image property: Image as a UTF-8 encoded base 64 string on image create. This field contains the image - * URI on image reads. - * - * @return the image value. - */ - public String image() { - return this.innerProperties() == null ? null : this.innerProperties().image(); - } - - /** - * Set the image property: Image as a UTF-8 encoded base 64 string on image create. This field contains the image - * URI on image reads. - * - * @param image the image value to set. - * @return the ImageInner object itself. - */ - public ImageInner withImage(String image) { - if (this.innerProperties() == null) { - this.innerProperties = new ImageProperties(); - } - this.innerProperties().withImage(image); - return this; - } - - /** - * Get the imageId property: Image ID. - * - * @return the imageId value. - */ - public String imageId() { - return this.innerProperties() == null ? null : this.innerProperties().imageId(); - } - - /** - * Set the imageId property: Image ID. - * - * @param imageId the imageId value to set. - * @return the ImageInner object itself. + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. */ - public ImageInner withImageId(String imageId) { - if (this.innerProperties() == null) { - this.innerProperties = new ImageProperties(); - } - this.innerProperties().withImageId(imageId); - return this; - } + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; /** - * Get the imageName property: Image name. - * - * @return the imageName value. + * Creates an instance of ImageInner class. */ - public String imageName() { - return this.innerProperties() == null ? null : this.innerProperties().imageName(); + public ImageInner() { } /** - * Get the regionalDataBoundary property: Regional data boundary for an image. - * - * @return the regionalDataBoundary value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - public RegionalDataBoundary regionalDataBoundary() { - return this.innerProperties() == null ? null : this.innerProperties().regionalDataBoundary(); + public ImageProperties properties() { + return this.properties; } /** - * Set the regionalDataBoundary property: Regional data boundary for an image. - * - * @param regionalDataBoundary the regionalDataBoundary value to set. + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. * @return the ImageInner object itself. */ - public ImageInner withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary) { - if (this.innerProperties() == null) { - this.innerProperties = new ImageProperties(); - } - this.innerProperties().withRegionalDataBoundary(regionalDataBoundary); + public ImageInner withProperties(ImageProperties properties) { + this.properties = properties; return this; } /** - * Get the uri property: Location the image. - * - * @return the uri value. - */ - public String uri() { - return this.innerProperties() == null ? null : this.innerProperties().uri(); - } - - /** - * Get the description property: The image description. - * - * @return the description value. - */ - public String description() { - return this.innerProperties() == null ? null : this.innerProperties().description(); - } - - /** - * Get the componentId property: The image component id. - * - * @return the componentId value. - */ - public String componentId() { - return this.innerProperties() == null ? null : this.innerProperties().componentId(); - } - - /** - * Get the imageType property: The image type. - * - * @return the imageType value. - */ - public ImageType imageType() { - return this.innerProperties() == null ? null : this.innerProperties().imageType(); - } - - /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/OperationInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/OperationInner.java index 239055d7f0702..dfc05b47ebf82 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/OperationInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/OperationInner.java @@ -12,8 +12,8 @@ /** * REST API Operation - * - *

Details of a REST API operation, returned from the Resource Provider Operations API. + * + * Details of a REST API operation, returned from the Resource Provider Operations API. */ @Fluent public final class OperationInner { @@ -50,14 +50,16 @@ public final class OperationInner { @JsonProperty(value = "actionType", access = JsonProperty.Access.WRITE_ONLY) private ActionType actionType; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ public String name() { @@ -67,7 +69,7 @@ public String name() { /** * Get the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -76,7 +78,7 @@ public Boolean isDataAction() { /** * Get the display property: Localized display information for this particular operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -85,7 +87,7 @@ public OperationDisplay display() { /** * Set the display property: Localized display information for this particular operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -97,7 +99,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Get the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ public Origin origin() { @@ -107,7 +109,7 @@ public Origin origin() { /** * Get the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ public ActionType actionType() { @@ -116,7 +118,7 @@ public ActionType actionType() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductInner.java index c5a3f9cc1d70a..a4bb450d3b9a4 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductInner.java @@ -6,71 +6,70 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.sphere.models.ProductProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** An product resource belonging to a catalog resource. */ +/** + * An product resource belonging to a catalog resource. + */ @Fluent public final class ProductInner extends ProxyResource { /* * The resource-specific properties for this resource. */ @JsonProperty(value = "properties") - private ProductProperties innerProperties; + private ProductProperties properties; - /** Creates an instance of ProductInner class. */ - public ProductInner() { - } + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; /** - * Get the innerProperties property: The resource-specific properties for this resource. - * - * @return the innerProperties value. + * Creates an instance of ProductInner class. */ - private ProductProperties innerProperties() { - return this.innerProperties; + public ProductInner() { } /** - * Get the description property: Description of the product. - * - * @return the description value. + * Get the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - public String description() { - return this.innerProperties() == null ? null : this.innerProperties().description(); + public ProductProperties properties() { + return this.properties; } /** - * Set the description property: Description of the product. - * - * @param description the description value to set. + * Set the properties property: The resource-specific properties for this resource. + * + * @param properties the properties value to set. * @return the ProductInner object itself. */ - public ProductInner withDescription(String description) { - if (this.innerProperties() == null) { - this.innerProperties = new ProductProperties(); - } - this.innerProperties().withDescription(description); + public ProductInner withProperties(ProductProperties properties) { + this.properties = properties; return this; } /** - * Get the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public ProvisioningState provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProofOfPossessionNonceResponseInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProofOfPossessionNonceResponseInner.java index ee5f94ce18c2f..a1582786030f5 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProofOfPossessionNonceResponseInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProofOfPossessionNonceResponseInner.java @@ -5,17 +5,22 @@ package com.azure.resourcemanager.sphere.fluent.models; import com.azure.core.annotation.Immutable; +import com.azure.resourcemanager.sphere.models.CertificateProperties; -/** Result of the action to generate a proof of possession nonce. */ +/** + * Result of the action to generate a proof of possession nonce. + */ @Immutable public final class ProofOfPossessionNonceResponseInner extends CertificateProperties { - /** Creates an instance of ProofOfPossessionNonceResponseInner class. */ + /** + * Creates an instance of ProofOfPossessionNonceResponseInner class. + */ public ProofOfPossessionNonceResponseInner() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/SignedCapabilityImageResponseInner.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/SignedCapabilityImageResponseInner.java index 1822b0e457228..e493a98d9a8e7 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/SignedCapabilityImageResponseInner.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/SignedCapabilityImageResponseInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Signed device capability image response. */ +/** + * Signed device capability image response. + */ @Immutable public final class SignedCapabilityImageResponseInner { /* @@ -16,13 +18,15 @@ public final class SignedCapabilityImageResponseInner { @JsonProperty(value = "image", access = JsonProperty.Access.WRITE_ONLY) private String image; - /** Creates an instance of SignedCapabilityImageResponseInner class. */ + /** + * Creates an instance of SignedCapabilityImageResponseInner class. + */ public SignedCapabilityImageResponseInner() { } /** * Get the image property: The signed device capability image as a UTF-8 encoded base 64 string. - * + * * @return the image value. */ public String image() { @@ -31,7 +35,7 @@ public String image() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/package-info.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/package-info.java index 6c295b60279c0..44b34f3f416cd 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/package-info.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the inner data models for AzureSphereManagementClient. Azure Sphere resource management API. */ +/** + * Package containing the inner data models for AzureSphereMgmtClient. + * Azure Sphere resource management API. + */ package com.azure.resourcemanager.sphere.fluent.models; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/package-info.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/package-info.java index 9853cc0ca23a1..725f0e0fae443 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/package-info.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the service clients for AzureSphereManagementClient. Azure Sphere resource management API. */ +/** + * Package containing the service clients for AzureSphereMgmtClient. + * Azure Sphere resource management API. + */ package com.azure.resourcemanager.sphere.fluent; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereManagementClientBuilder.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereMgmtClientBuilder.java similarity index 54% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereManagementClientBuilder.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereMgmtClientBuilder.java index 6f5c108a3bba2..b241c95719877 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereManagementClientBuilder.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereMgmtClientBuilder.java @@ -14,9 +14,11 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the AzureSphereManagementClientImpl type. */ -@ServiceClientBuilder(serviceClients = {AzureSphereManagementClientImpl.class}) -public final class AzureSphereManagementClientBuilder { +/** + * A builder for creating a new instance of the AzureSphereMgmtClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { AzureSphereMgmtClientImpl.class }) +public final class AzureSphereMgmtClientBuilder { /* * The ID of the target subscription. */ @@ -24,11 +26,11 @@ public final class AzureSphereManagementClientBuilder { /** * Sets The ID of the target subscription. - * + * * @param subscriptionId the subscriptionId value. - * @return the AzureSphereManagementClientBuilder. + * @return the AzureSphereMgmtClientBuilder. */ - public AzureSphereManagementClientBuilder subscriptionId(String subscriptionId) { + public AzureSphereMgmtClientBuilder subscriptionId(String subscriptionId) { this.subscriptionId = subscriptionId; return this; } @@ -40,11 +42,11 @@ public AzureSphereManagementClientBuilder subscriptionId(String subscriptionId) /** * Sets server parameter. - * + * * @param endpoint the endpoint value. - * @return the AzureSphereManagementClientBuilder. + * @return the AzureSphereMgmtClientBuilder. */ - public AzureSphereManagementClientBuilder endpoint(String endpoint) { + public AzureSphereMgmtClientBuilder endpoint(String endpoint) { this.endpoint = endpoint; return this; } @@ -56,11 +58,11 @@ public AzureSphereManagementClientBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. - * @return the AzureSphereManagementClientBuilder. + * @return the AzureSphereMgmtClientBuilder. */ - public AzureSphereManagementClientBuilder environment(AzureEnvironment environment) { + public AzureSphereMgmtClientBuilder environment(AzureEnvironment environment) { this.environment = environment; return this; } @@ -72,11 +74,11 @@ public AzureSphereManagementClientBuilder environment(AzureEnvironment environme /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. - * @return the AzureSphereManagementClientBuilder. + * @return the AzureSphereMgmtClientBuilder. */ - public AzureSphereManagementClientBuilder pipeline(HttpPipeline pipeline) { + public AzureSphereMgmtClientBuilder pipeline(HttpPipeline pipeline) { this.pipeline = pipeline; return this; } @@ -88,11 +90,11 @@ public AzureSphereManagementClientBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. - * @return the AzureSphereManagementClientBuilder. + * @return the AzureSphereMgmtClientBuilder. */ - public AzureSphereManagementClientBuilder defaultPollInterval(Duration defaultPollInterval) { + public AzureSphereMgmtClientBuilder defaultPollInterval(Duration defaultPollInterval) { this.defaultPollInterval = defaultPollInterval; return this; } @@ -104,41 +106,31 @@ public AzureSphereManagementClientBuilder defaultPollInterval(Duration defaultPo /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. - * @return the AzureSphereManagementClientBuilder. + * @return the AzureSphereMgmtClientBuilder. */ - public AzureSphereManagementClientBuilder serializerAdapter(SerializerAdapter serializerAdapter) { + public AzureSphereMgmtClientBuilder serializerAdapter(SerializerAdapter serializerAdapter) { this.serializerAdapter = serializerAdapter; return this; } /** - * Builds an instance of AzureSphereManagementClientImpl with the provided parameters. - * - * @return an instance of AzureSphereManagementClientImpl. + * Builds an instance of AzureSphereMgmtClientImpl with the provided parameters. + * + * @return an instance of AzureSphereMgmtClientImpl. */ - public AzureSphereManagementClientImpl buildClient() { + public AzureSphereMgmtClientImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - AzureSphereManagementClientImpl client = - new AzureSphereManagementClientImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + AzureSphereMgmtClientImpl client = new AzureSphereMgmtClientImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereManagementClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereMgmtClientImpl.java similarity index 80% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereManagementClientImpl.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereMgmtClientImpl.java index 251919142566f..17c8f69d4d40d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereManagementClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/AzureSphereMgmtClientImpl.java @@ -22,7 +22,7 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.serializer.SerializerAdapter; import com.azure.core.util.serializer.SerializerEncoding; -import com.azure.resourcemanager.sphere.fluent.AzureSphereManagementClient; +import com.azure.resourcemanager.sphere.fluent.AzureSphereMgmtClient; import com.azure.resourcemanager.sphere.fluent.CatalogsClient; import com.azure.resourcemanager.sphere.fluent.CertificatesClient; import com.azure.resourcemanager.sphere.fluent.DeploymentsClient; @@ -40,171 +40,201 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the AzureSphereManagementClientImpl type. */ -@ServiceClient(builder = AzureSphereManagementClientBuilder.class) -public final class AzureSphereManagementClientImpl implements AzureSphereManagementClient { - /** The ID of the target subscription. */ +/** + * Initializes a new instance of the AzureSphereMgmtClientImpl type. + */ +@ServiceClient(builder = AzureSphereMgmtClientBuilder.class) +public final class AzureSphereMgmtClientImpl implements AzureSphereMgmtClient { + /** + * The ID of the target subscription. + */ private final String subscriptionId; /** * Gets The ID of the target subscription. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The CatalogsClient object to access its operations. */ + /** + * The CatalogsClient object to access its operations. + */ private final CatalogsClient catalogs; /** * Gets the CatalogsClient object to access its operations. - * + * * @return the CatalogsClient object. */ public CatalogsClient getCatalogs() { return this.catalogs; } - /** The CertificatesClient object to access its operations. */ + /** + * The CertificatesClient object to access its operations. + */ private final CertificatesClient certificates; /** * Gets the CertificatesClient object to access its operations. - * + * * @return the CertificatesClient object. */ public CertificatesClient getCertificates() { return this.certificates; } - /** The ImagesClient object to access its operations. */ + /** + * The ImagesClient object to access its operations. + */ private final ImagesClient images; /** * Gets the ImagesClient object to access its operations. - * + * * @return the ImagesClient object. */ public ImagesClient getImages() { return this.images; } - /** The ProductsClient object to access its operations. */ + /** + * The ProductsClient object to access its operations. + */ private final ProductsClient products; /** * Gets the ProductsClient object to access its operations. - * + * * @return the ProductsClient object. */ public ProductsClient getProducts() { return this.products; } - /** The DeviceGroupsClient object to access its operations. */ + /** + * The DeviceGroupsClient object to access its operations. + */ private final DeviceGroupsClient deviceGroups; /** * Gets the DeviceGroupsClient object to access its operations. - * + * * @return the DeviceGroupsClient object. */ public DeviceGroupsClient getDeviceGroups() { return this.deviceGroups; } - /** The DeploymentsClient object to access its operations. */ + /** + * The DeploymentsClient object to access its operations. + */ private final DeploymentsClient deployments; /** * Gets the DeploymentsClient object to access its operations. - * + * * @return the DeploymentsClient object. */ public DeploymentsClient getDeployments() { return this.deployments; } - /** The DevicesClient object to access its operations. */ + /** + * The DevicesClient object to access its operations. + */ private final DevicesClient devices; /** * Gets the DevicesClient object to access its operations. - * + * * @return the DevicesClient object. */ public DevicesClient getDevices() { @@ -212,8 +242,8 @@ public DevicesClient getDevices() { } /** - * Initializes an instance of AzureSphereManagementClient client. - * + * Initializes an instance of AzureSphereMgmtClient client. + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -221,19 +251,14 @@ public DevicesClient getDevices() { * @param subscriptionId The ID of the target subscription. * @param endpoint server parameter. */ - AzureSphereManagementClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + AzureSphereMgmtClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2022-09-01-preview"; + this.apiVersion = "2024-04-01"; this.operations = new OperationsClientImpl(this); this.catalogs = new CatalogsClientImpl(this); this.certificates = new CertificatesClientImpl(this); @@ -246,7 +271,7 @@ public DevicesClient getDevices() { /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -255,7 +280,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -265,7 +290,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -275,26 +300,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -307,19 +321,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } @@ -384,5 +395,5 @@ public Mono getBodyAsString(Charset charset) { } } - private static final ClientLogger LOGGER = new ClientLogger(AzureSphereManagementClientImpl.class); + private static final ClientLogger LOGGER = new ClientLogger(AzureSphereMgmtClientImpl.class); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogImpl.java index e36df7131263a..9ffba680cbfe7 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogImpl.java @@ -10,15 +10,16 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.Catalog; +import com.azure.resourcemanager.sphere.models.CatalogProperties; import com.azure.resourcemanager.sphere.models.CatalogUpdate; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import com.azure.resourcemanager.sphere.models.Deployment; import com.azure.resourcemanager.sphere.models.Device; import com.azure.resourcemanager.sphere.models.DeviceGroup; import com.azure.resourcemanager.sphere.models.DeviceInsight; import com.azure.resourcemanager.sphere.models.ListDeviceGroupsRequest; -import com.azure.resourcemanager.sphere.models.ProvisioningState; import java.util.Collections; import java.util.Map; @@ -52,12 +53,12 @@ public Map tags() { } } - public SystemData systemData() { - return this.innerModel().systemData(); + public CatalogProperties properties() { + return this.innerModel().properties(); } - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public Region region() { @@ -92,20 +93,14 @@ public CatalogImpl withExistingResourceGroup(String resourceGroupName) { } public Catalog create() { - this.innerObject = - serviceManager - .serviceClient() - .getCatalogs() - .createOrUpdate(resourceGroupName, catalogName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getCatalogs().createOrUpdate(resourceGroupName, catalogName, + this.innerModel(), Context.NONE); return this; } public Catalog create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCatalogs() - .createOrUpdate(resourceGroupName, catalogName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getCatalogs().createOrUpdate(resourceGroupName, catalogName, + this.innerModel(), context); return this; } @@ -121,57 +116,41 @@ public CatalogImpl update() { } public Catalog apply() { - this.innerObject = - serviceManager - .serviceClient() - .getCatalogs() - .updateWithResponse(resourceGroupName, catalogName, updateProperties, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCatalogs() + .updateWithResponse(resourceGroupName, catalogName, updateProperties, Context.NONE).getValue(); return this; } public Catalog apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCatalogs() - .updateWithResponse(resourceGroupName, catalogName, updateProperties, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCatalogs() + .updateWithResponse(resourceGroupName, catalogName, updateProperties, context).getValue(); return this; } CatalogImpl(CatalogInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.catalogName = Utils.getValueFromIdByName(innerObject.id(), "catalogs"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.catalogName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "catalogs"); } public Catalog refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getCatalogs() - .getByResourceGroupWithResponse(resourceGroupName, catalogName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCatalogs() + .getByResourceGroupWithResponse(resourceGroupName, catalogName, Context.NONE).getValue(); return this; } public Catalog refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCatalogs() - .getByResourceGroupWithResponse(resourceGroupName, catalogName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCatalogs() + .getByResourceGroupWithResponse(resourceGroupName, catalogName, context).getValue(); return this; } - public Response countDevicesWithResponse(Context context) { + public Response countDevicesWithResponse(Context context) { return serviceManager.catalogs().countDevicesWithResponse(resourceGroupName, catalogName, context); } - public CountDeviceResponse countDevices() { + public CountDevicesResponse countDevices() { return serviceManager.catalogs().countDevices(resourceGroupName, catalogName); } @@ -179,50 +158,48 @@ public PagedIterable listDeployments() { return serviceManager.catalogs().listDeployments(resourceGroupName, catalogName); } - public PagedIterable listDeployments( - String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return serviceManager - .catalogs() - .listDeployments(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); + public PagedIterable listDeployments(String filter, Integer top, Integer skip, Integer maxpagesize, + Context context) { + return serviceManager.catalogs().listDeployments(resourceGroupName, catalogName, filter, top, skip, maxpagesize, + context); } public PagedIterable listDeviceGroups(ListDeviceGroupsRequest listDeviceGroupsRequest) { return serviceManager.catalogs().listDeviceGroups(resourceGroupName, catalogName, listDeviceGroupsRequest); } - public PagedIterable listDeviceGroups( - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - return serviceManager - .catalogs() - .listDeviceGroups( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize, context); + public PagedIterable listDeviceGroups(ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { + return serviceManager.catalogs().listDeviceGroups(resourceGroupName, catalogName, listDeviceGroupsRequest, + filter, top, skip, maxpagesize, context); } public PagedIterable listDeviceInsights() { return serviceManager.catalogs().listDeviceInsights(resourceGroupName, catalogName); } - public PagedIterable listDeviceInsights( - String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return serviceManager - .catalogs() - .listDeviceInsights(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); + public PagedIterable listDeviceInsights(String filter, Integer top, Integer skip, + Integer maxpagesize, Context context) { + return serviceManager.catalogs().listDeviceInsights(resourceGroupName, catalogName, filter, top, skip, + maxpagesize, context); } public PagedIterable listDevices() { return serviceManager.catalogs().listDevices(resourceGroupName, catalogName); } - public PagedIterable listDevices( - String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return serviceManager - .catalogs() - .listDevices(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); + public PagedIterable listDevices(String filter, Integer top, Integer skip, Integer maxpagesize, + Context context) { + return serviceManager.catalogs().listDevices(resourceGroupName, catalogName, filter, top, skip, maxpagesize, + context); + } + + public void uploadImage(ImageInner uploadImageRequest) { + serviceManager.catalogs().uploadImage(resourceGroupName, catalogName, uploadImageRequest); + } + + public void uploadImage(ImageInner uploadImageRequest, Context context) { + serviceManager.catalogs().uploadImage(resourceGroupName, catalogName, uploadImageRequest, context); } public CatalogImpl withRegion(Region location) { @@ -245,6 +222,11 @@ public CatalogImpl withTags(Map tags) { } } + public CatalogImpl withProperties(CatalogProperties properties) { + this.innerModel().withProperties(properties); + return this; + } + private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsClientImpl.java index aab63c3693db3..2c20fb1e1cb77 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsClientImpl.java @@ -35,11 +35,12 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.sphere.fluent.CatalogsClient; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceInsightInner; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.CatalogListResult; import com.azure.resourcemanager.sphere.models.CatalogUpdate; import com.azure.resourcemanager.sphere.models.DeploymentListResult; @@ -51,366 +52,276 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CatalogsClient. */ +/** + * An instance of this class provides access to all the operations defined in CatalogsClient. + */ public final class CatalogsClientImpl implements CatalogsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CatalogsService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of CatalogsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - CatalogsClientImpl(AzureSphereManagementClientImpl client) { + CatalogsClientImpl(AzureSphereMgmtClientImpl client) { this.service = RestProxy.create(CatalogsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientCatalogs to be used by the proxy service - * to perform REST calls. + * The interface defining all the services for AzureSphereMgmtClientCatalogs to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface CatalogsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.AzureSphere/catalogs") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") - @ExpectedResponses({200, 201}) + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @BodyParam("application/json") CatalogInner resource, - @HeaderParam("Accept") String accept, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @BodyParam("application/json") CatalogInner resource, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @BodyParam("application/json") CatalogUpdate properties, - @HeaderParam("Accept") String accept, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @BodyParam("application/json") CatalogUpdate properties, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/countDevices") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/countDevices") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> countDevices( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDeployments") - @ExpectedResponses({200}) + Mono> countDevices(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDeployments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listDeployments( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDeviceGroups") - @ExpectedResponses({200}) + Mono> listDeployments(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDeviceGroups") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listDeviceGroups( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, + Mono> listDeviceGroups(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, @BodyParam("application/json") ListDeviceGroupsRequest listDeviceGroupsRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDeviceInsights") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDeviceInsights") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listDeviceInsights( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDevices") - @ExpectedResponses({200}) + Mono> listDeviceInsights(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/listDevices") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listDevices(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/uploadImage") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listDevices( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @HeaderParam("Accept") String accept, + Mono>> uploadImage(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @BodyParam("application/json") ImageInner uploadImageRequest, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listDeploymentsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listDeviceGroupsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listDeviceInsightsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listDevicesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listDevicesNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List Catalog resources by subscription ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Catalog resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Catalog resources by subscription ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * List Catalog resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -419,13 +330,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * List Catalog resources by subscription ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation as paginated response with {@link PagedIterable}. @@ -437,7 +348,7 @@ public PagedIterable list() { /** * List Catalog resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -451,27 +362,23 @@ public PagedIterable list(Context context) { /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -479,53 +386,34 @@ private Mono> listByResourceGroupSinglePageAsync(Str } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -534,27 +422,15 @@ private Mono> listByResourceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -563,14 +439,13 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -580,14 +455,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupNam */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -601,7 +475,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName) /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -616,7 +490,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName, /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -625,19 +499,15 @@ public PagedIterable listByResourceGroup(String resourceGroupName, * @return a Catalog along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String catalogName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String catalogName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -648,23 +518,14 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -674,19 +535,15 @@ private Mono> getByResourceGroupWithResponseAsync( * @return a Catalog along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String catalogName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String catalogName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -697,20 +554,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, accept, context); } /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -726,7 +576,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, Str /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -736,14 +586,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName, Str * @return a Catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String catalogName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String catalogName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, catalogName, context).block(); } /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -758,7 +608,7 @@ public CatalogInner getByResourceGroup(String resourceGroupName, String catalogN /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -768,19 +618,15 @@ public CatalogInner getByResourceGroup(String resourceGroupName, String catalogN * @return an Azure Sphere catalog along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String catalogName, CatalogInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, CatalogInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -796,24 +642,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, resource, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -824,19 +660,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return an Azure Sphere catalog along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String catalogName, CatalogInner resource, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, CatalogInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -852,21 +684,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, resource, accept, context); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -876,19 +700,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, CatalogInner> beginCreateOrUpdateAsync( - String resourceGroupName, String catalogName, CatalogInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, resource); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), CatalogInner.class, CatalogInner.class, this.client.getContext()); + private PollerFlux, CatalogInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, CatalogInner resource) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + CatalogInner.class, CatalogInner.class, this.client.getContext()); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -899,20 +721,18 @@ private PollerFlux, CatalogInner> beginCreateOrUpdateAs * @return the {@link PollerFlux} for polling of an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, CatalogInner> beginCreateOrUpdateAsync( - String resourceGroupName, String catalogName, CatalogInner resource, Context context) { + private PollerFlux, CatalogInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, CatalogInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), CatalogInner.class, CatalogInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + CatalogInner.class, CatalogInner.class, context); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -922,14 +742,14 @@ private PollerFlux, CatalogInner> beginCreateOrUpdateAs * @return the {@link SyncPoller} for polling of an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, CatalogInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, CatalogInner resource) { + public SyncPoller, CatalogInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, CatalogInner resource) { return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, resource).getSyncPoller(); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -940,14 +760,14 @@ public SyncPoller, CatalogInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, CatalogInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, CatalogInner resource, Context context) { + public SyncPoller, CatalogInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, CatalogInner resource, Context context) { return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, resource, context).getSyncPoller(); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -957,16 +777,15 @@ public SyncPoller, CatalogInner> beginCreateOrUpdate( * @return an Azure Sphere catalog on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String catalogName, CatalogInner resource) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, resource) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, + CatalogInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, resource).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -977,16 +796,15 @@ private Mono createOrUpdateAsync( * @return an Azure Sphere catalog on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String catalogName, CatalogInner resource, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, resource, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, CatalogInner resource, + Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, resource, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -1002,7 +820,7 @@ public CatalogInner createOrUpdate(String resourceGroupName, String catalogName, /** * Create a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param resource Resource create parameters. @@ -1013,14 +831,14 @@ public CatalogInner createOrUpdate(String resourceGroupName, String catalogName, * @return an Azure Sphere catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CatalogInner createOrUpdate( - String resourceGroupName, String catalogName, CatalogInner resource, Context context) { + public CatalogInner createOrUpdate(String resourceGroupName, String catalogName, CatalogInner resource, + Context context) { return createOrUpdateAsync(resourceGroupName, catalogName, resource, context).block(); } /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -1030,19 +848,15 @@ public CatalogInner createOrUpdate( * @return an Azure Sphere catalog along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String catalogName, CatalogUpdate properties) { + private Mono> updateWithResponseAsync(String resourceGroupName, String catalogName, + CatalogUpdate properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1058,24 +872,14 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - properties, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, properties, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -1086,19 +890,15 @@ private Mono> updateWithResponseAsync( * @return an Azure Sphere catalog along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String catalogName, CatalogUpdate properties, Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, String catalogName, + CatalogUpdate properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1114,21 +914,13 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - properties, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, properties, accept, context); } /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -1145,7 +937,7 @@ private Mono updateAsync(String resourceGroupName, String catalogN /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -1156,14 +948,14 @@ private Mono updateAsync(String resourceGroupName, String catalogN * @return an Azure Sphere catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, String catalogName, CatalogUpdate properties, Context context) { + public Response updateWithResponse(String resourceGroupName, String catalogName, + CatalogUpdate properties, Context context) { return updateWithResponseAsync(resourceGroupName, catalogName, properties, context).block(); } /** * Update a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param properties The resource properties to be updated. @@ -1179,7 +971,7 @@ public CatalogInner update(String resourceGroupName, String catalogName, Catalog /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1190,16 +982,12 @@ public CatalogInner update(String resourceGroupName, String catalogName, Catalog @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1210,23 +998,14 @@ private Mono>> deleteWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1236,19 +1015,15 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1259,20 +1034,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, accept, context); } /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1283,15 +1051,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, catalogName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1301,18 +1067,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, catalogName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1327,7 +1092,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1337,14 +1102,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + Context context) { return this.beginDeleteAsync(resourceGroupName, catalogName, context).getSyncPoller(); } /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1359,7 +1124,7 @@ private Mono deleteAsync(String resourceGroupName, String catalogName) { /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1370,14 +1135,13 @@ private Mono deleteAsync(String resourceGroupName, String catalogName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String catalogName, Context context) { - return beginDeleteAsync(resourceGroupName, catalogName, context) - .last() + return beginDeleteAsync(resourceGroupName, catalogName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1391,7 +1155,7 @@ public void delete(String resourceGroupName, String catalogName) { /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1406,29 +1170,25 @@ public void delete(String resourceGroupName, String catalogName, Context context /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> countDevicesWithResponseAsync( - String resourceGroupName, String catalogName) { + private Mono> countDevicesWithResponseAsync(String resourceGroupName, + String catalogName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1439,23 +1199,14 @@ private Mono> countDevicesWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .countDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context)) + .withContext(context -> service.countDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1463,22 +1214,18 @@ private Mono> countDevicesWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> countDevicesWithResponseAsync( - String resourceGroupName, String catalogName, Context context) { + private Mono> countDevicesWithResponseAsync(String resourceGroupName, + String catalogName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1489,20 +1236,13 @@ private Mono> countDevicesWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .countDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context); + return service.countDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, accept, context); } /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1511,14 +1251,14 @@ private Mono> countDevicesWithResponseAsync( * @return response to the action call for count devices in a catalog on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono countDevicesAsync(String resourceGroupName, String catalogName) { + private Mono countDevicesAsync(String resourceGroupName, String catalogName) { return countDevicesWithResponseAsync(resourceGroupName, catalogName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -1528,14 +1268,14 @@ private Mono countDevicesAsync(String resourceGroupNam * @return response to the action call for count devices in a catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response countDevicesWithResponse( - String resourceGroupName, String catalogName, Context context) { + public Response countDevicesWithResponse(String resourceGroupName, String catalogName, + Context context) { return countDevicesWithResponseAsync(resourceGroupName, catalogName, context).block(); } /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1544,13 +1284,13 @@ public Response countDevicesWithResponse( * @return response to the action call for count devices in a catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CountDeviceResponseInner countDevices(String resourceGroupName, String catalogName) { + public CountDevicesResponseInner countDevices(String resourceGroupName, String catalogName) { return countDevicesWithResponse(resourceGroupName, catalogName, Context.NONE).getValue(); } /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -1561,22 +1301,18 @@ public CountDeviceResponseInner countDevices(String resourceGroupName, String ca * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeploymentsSinglePageAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private Mono> listDeploymentsSinglePageAsync(String resourceGroupName, + String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1587,36 +1323,17 @@ private Mono> listDeploymentsSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listDeployments( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listDeployments(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -1628,28 +1345,18 @@ private Mono> listDeploymentsSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeploymentsSinglePageAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listDeploymentsSinglePageAsync(String resourceGroupName, + String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1661,32 +1368,15 @@ private Mono> listDeploymentsSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listDeployments( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listDeployments(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -1699,8 +1389,8 @@ private Mono> listDeploymentsSinglePageAsync( * @return the response of a Deployment list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeploymentsAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private PagedFlux listDeploymentsAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize) { return new PagedFlux<>( () -> listDeploymentsSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize), nextLink -> listDeploymentsNextSinglePageAsync(nextLink)); @@ -1708,7 +1398,7 @@ private PagedFlux listDeploymentsAsync( /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1729,7 +1419,7 @@ private PagedFlux listDeploymentsAsync(String resourceGroupName /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -1743,23 +1433,15 @@ private PagedFlux listDeploymentsAsync(String resourceGroupName * @return the response of a Deployment list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeploymentsAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - return new PagedFlux<>( - () -> - listDeploymentsSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context), - nextLink -> listDeploymentsNextSinglePageAsync(nextLink, context)); + private PagedFlux listDeploymentsAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { + return new PagedFlux<>(() -> listDeploymentsSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, + maxpagesize, context), nextLink -> listDeploymentsNextSinglePageAsync(nextLink, context)); } /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1779,7 +1461,7 @@ public PagedIterable listDeployments(String resourceGroupName, /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -1793,21 +1475,15 @@ public PagedIterable listDeployments(String resourceGroupName, * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listDeployments( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + public PagedIterable listDeployments(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedIterable<>( listDeploymentsAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context)); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -1819,28 +1495,19 @@ public PagedIterable listDeployments( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeviceGroupsSinglePageAsync( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, + private Mono> listDeviceGroupsSinglePageAsync(String resourceGroupName, + String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1850,45 +1517,24 @@ private Mono> listDeviceGroupsSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter catalogName is required and cannot be null.")); } if (listDeviceGroupsRequest == null) { - return Mono - .error( - new IllegalArgumentException("Parameter listDeviceGroupsRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter listDeviceGroupsRequest is required and cannot be null.")); } else { listDeviceGroupsRequest.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listDeviceGroups( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - listDeviceGroupsRequest, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listDeviceGroups(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, + listDeviceGroupsRequest, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -1901,29 +1547,19 @@ private Mono> listDeviceGroupsSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeviceGroupsSinglePageAsync( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listDeviceGroupsSinglePageAsync(String resourceGroupName, + String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, + Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1933,42 +1569,24 @@ private Mono> listDeviceGroupsSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter catalogName is required and cannot be null.")); } if (listDeviceGroupsRequest == null) { - return Mono - .error( - new IllegalArgumentException("Parameter listDeviceGroupsRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter listDeviceGroupsRequest is required and cannot be null.")); } else { listDeviceGroupsRequest.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listDeviceGroups( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - listDeviceGroupsRequest, - accept, + .listDeviceGroups(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, listDeviceGroupsRequest, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -1982,24 +1600,17 @@ private Mono> listDeviceGroupsSinglePageAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeviceGroupsAsync( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, + private PagedFlux listDeviceGroupsAsync(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize) { - return new PagedFlux<>( - () -> - listDeviceGroupsSinglePageAsync( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize), + return new PagedFlux<>(() -> listDeviceGroupsSinglePageAsync(resourceGroupName, catalogName, + listDeviceGroupsRequest, filter, top, skip, maxpagesize), nextLink -> listDeviceGroupsNextSinglePageAsync(nextLink)); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -2009,22 +1620,20 @@ private PagedFlux listDeviceGroupsAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeviceGroupsAsync( - String resourceGroupName, String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest) { + private PagedFlux listDeviceGroupsAsync(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest) { final String filter = null; final Integer top = null; final Integer skip = null; final Integer maxpagesize = null; - return new PagedFlux<>( - () -> - listDeviceGroupsSinglePageAsync( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize), + return new PagedFlux<>(() -> listDeviceGroupsSinglePageAsync(resourceGroupName, catalogName, + listDeviceGroupsRequest, filter, top, skip, maxpagesize), nextLink -> listDeviceGroupsNextSinglePageAsync(nextLink)); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -2039,25 +1648,17 @@ private PagedFlux listDeviceGroupsAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeviceGroupsAsync( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + private PagedFlux listDeviceGroupsAsync(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return new PagedFlux<>( - () -> - listDeviceGroupsSinglePageAsync( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize, context), + return new PagedFlux<>(() -> listDeviceGroupsSinglePageAsync(resourceGroupName, catalogName, + listDeviceGroupsRequest, filter, top, skip, maxpagesize, context), nextLink -> listDeviceGroupsNextSinglePageAsync(nextLink, context)); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -2067,20 +1668,19 @@ private PagedFlux listDeviceGroupsAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listDeviceGroups( - String resourceGroupName, String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest) { + public PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest) { final String filter = null; final Integer top = null; final Integer skip = null; final Integer maxpagesize = null; - return new PagedIterable<>( - listDeviceGroupsAsync( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize)); + return new PagedIterable<>(listDeviceGroupsAsync(resourceGroupName, catalogName, listDeviceGroupsRequest, + filter, top, skip, maxpagesize)); } /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -2095,23 +1695,16 @@ public PagedIterable listDeviceGroups( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listDeviceGroups( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + public PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return new PagedIterable<>( - listDeviceGroupsAsync( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize, context)); + return new PagedIterable<>(listDeviceGroupsAsync(resourceGroupName, catalogName, listDeviceGroupsRequest, + filter, top, skip, maxpagesize, context)); } /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2122,22 +1715,18 @@ public PagedIterable listDeviceGroups( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeviceInsightsSinglePageAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private Mono> listDeviceInsightsSinglePageAsync(String resourceGroupName, + String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2148,36 +1737,17 @@ private Mono> listDeviceInsightsSinglePageAsyn } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listDeviceInsights( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listDeviceInsights(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2189,28 +1759,18 @@ private Mono> listDeviceInsightsSinglePageAsyn * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeviceInsightsSinglePageAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listDeviceInsightsSinglePageAsync(String resourceGroupName, + String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2222,32 +1782,15 @@ private Mono> listDeviceInsightsSinglePageAsyn final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listDeviceInsights( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listDeviceInsights(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2260,8 +1803,8 @@ private Mono> listDeviceInsightsSinglePageAsyn * @return paged collection of DeviceInsight items as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeviceInsightsAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private PagedFlux listDeviceInsightsAsync(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize) { return new PagedFlux<>( () -> listDeviceInsightsSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize), nextLink -> listDeviceInsightsNextSinglePageAsync(nextLink)); @@ -2269,7 +1812,7 @@ private PagedFlux listDeviceInsightsAsync( /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2290,7 +1833,7 @@ private PagedFlux listDeviceInsightsAsync(String resourceGro /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2304,24 +1847,15 @@ private PagedFlux listDeviceInsightsAsync(String resourceGro * @return paged collection of DeviceInsight items as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDeviceInsightsAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - return new PagedFlux<>( - () -> - listDeviceInsightsSinglePageAsync( - resourceGroupName, catalogName, filter, top, skip, maxpagesize, context), - nextLink -> listDeviceInsightsNextSinglePageAsync(nextLink, context)); + private PagedFlux listDeviceInsightsAsync(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { + return new PagedFlux<>(() -> listDeviceInsightsSinglePageAsync(resourceGroupName, catalogName, filter, top, + skip, maxpagesize, context), nextLink -> listDeviceInsightsNextSinglePageAsync(nextLink, context)); } /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2341,7 +1875,7 @@ public PagedIterable listDeviceInsights(String resourceGroup /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2355,21 +1889,15 @@ public PagedIterable listDeviceInsights(String resourceGroup * @return paged collection of DeviceInsight items as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listDeviceInsights( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + public PagedIterable listDeviceInsights(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedIterable<>( listDeviceInsightsAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context)); } /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2380,22 +1908,18 @@ public PagedIterable listDeviceInsights( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDevicesSinglePageAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private Mono> listDevicesSinglePageAsync(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2406,36 +1930,17 @@ private Mono> listDevicesSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2447,28 +1952,18 @@ private Mono> listDevicesSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDevicesSinglePageAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listDevicesSinglePageAsync(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2480,32 +1975,15 @@ private Mono> listDevicesSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - filter, - top, - skip, - maxpagesize, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listDevices(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2518,8 +1996,8 @@ private Mono> listDevicesSinglePageAsync( * @return the response of a Device list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDevicesAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private PagedFlux listDevicesAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize) { return new PagedFlux<>( () -> listDevicesSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize), nextLink -> listDevicesNextSinglePageAsync(nextLink)); @@ -2527,7 +2005,7 @@ private PagedFlux listDevicesAsync( /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2548,7 +2026,7 @@ private PagedFlux listDevicesAsync(String resourceGroupName, String /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2562,14 +2040,8 @@ private PagedFlux listDevicesAsync(String resourceGroupName, String * @return the response of a Device list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDevicesAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private PagedFlux listDevicesAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedFlux<>( () -> listDevicesSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context), nextLink -> listDevicesNextSinglePageAsync(nextLink, context)); @@ -2577,7 +2049,7 @@ private PagedFlux listDevicesAsync( /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2596,7 +2068,7 @@ public PagedIterable listDevices(String resourceGroupName, String c /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -2610,28 +2082,252 @@ public PagedIterable listDevices(String resourceGroupName, String c * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listDevices( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + public PagedIterable listDevices(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedIterable<>( listDevicesAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context)); } + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> uploadImageWithResponseAsync(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (catalogName == null) { + return Mono.error(new IllegalArgumentException("Parameter catalogName is required and cannot be null.")); + } + if (uploadImageRequest == null) { + return Mono + .error(new IllegalArgumentException("Parameter uploadImageRequest is required and cannot be null.")); + } else { + uploadImageRequest.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.uploadImage(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, uploadImageRequest, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> uploadImageWithResponseAsync(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (catalogName == null) { + return Mono.error(new IllegalArgumentException("Parameter catalogName is required and cannot be null.")); + } + if (uploadImageRequest == null) { + return Mono + .error(new IllegalArgumentException("Parameter uploadImageRequest is required and cannot be null.")); + } else { + uploadImageRequest.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.uploadImage(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, uploadImageRequest, accept, context); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginUploadImageAsync(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest) { + Mono>> mono + = uploadImageWithResponseAsync(resourceGroupName, catalogName, uploadImageRequest); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginUploadImageAsync(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = uploadImageWithResponseAsync(resourceGroupName, catalogName, uploadImageRequest, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginUploadImage(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest) { + return this.beginUploadImageAsync(resourceGroupName, catalogName, uploadImageRequest).getSyncPoller(); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginUploadImage(String resourceGroupName, String catalogName, + ImageInner uploadImageRequest, Context context) { + return this.beginUploadImageAsync(resourceGroupName, catalogName, uploadImageRequest, context).getSyncPoller(); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono uploadImageAsync(String resourceGroupName, String catalogName, ImageInner uploadImageRequest) { + return beginUploadImageAsync(resourceGroupName, catalogName, uploadImageRequest).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono uploadImageAsync(String resourceGroupName, String catalogName, ImageInner uploadImageRequest, + Context context) { + return beginUploadImageAsync(resourceGroupName, catalogName, uploadImageRequest, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest) { + uploadImageAsync(resourceGroupName, catalogName, uploadImageRequest).block(); + } + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest, + Context context) { + uploadImageAsync(resourceGroupName, catalogName, uploadImageRequest, context).block(); + } + /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -2639,38 +2335,30 @@ private Mono> listBySubscriptionNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { @@ -2678,36 +2366,27 @@ private Mono> listBySubscriptionNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -2715,38 +2394,30 @@ private Mono> listByResourceGroupNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, Context context) { @@ -2754,36 +2425,27 @@ private Mono> listByResourceGroupNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDeploymentsNextSinglePageAsync(String nextLink) { @@ -2791,37 +2453,29 @@ private Mono> listDeploymentsNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listDeploymentsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDeploymentsNextSinglePageAsync(String nextLink, Context context) { @@ -2829,36 +2483,27 @@ private Mono> listDeploymentsNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listDeploymentsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listDeploymentsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDeviceGroupsNextSinglePageAsync(String nextLink) { @@ -2866,75 +2511,58 @@ private Mono> listDeviceGroupsNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeviceGroupsNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listDeviceGroupsNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDeviceInsightsNextSinglePageAsync(String nextLink) { @@ -2942,76 +2570,59 @@ private Mono> listDeviceInsightsNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listDeviceInsightsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDeviceInsightsNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listDeviceInsightsNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listDeviceInsightsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listDeviceInsightsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDevicesNextSinglePageAsync(String nextLink) { @@ -3019,37 +2630,29 @@ private Mono> listDevicesNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listDevicesNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDevicesNextSinglePageAsync(String nextLink, Context context) { @@ -3057,23 +2660,13 @@ private Mono> listDevicesNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listDevicesNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listDevicesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsImpl.java index 7204f29b10d98..69ae4fb4568f3 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CatalogsImpl.java @@ -11,14 +11,15 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.sphere.fluent.CatalogsClient; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceInsightInner; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.Catalog; import com.azure.resourcemanager.sphere.models.Catalogs; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import com.azure.resourcemanager.sphere.models.Deployment; import com.azure.resourcemanager.sphere.models.Device; import com.azure.resourcemanager.sphere.models.DeviceGroup; @@ -32,41 +33,38 @@ public final class CatalogsImpl implements Catalogs { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - public CatalogsImpl( - CatalogsClient innerClient, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + public CatalogsImpl(CatalogsClient innerClient, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CatalogImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String catalogName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, catalogName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String catalogName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, catalogName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CatalogImpl(inner.getValue(), this.manager())); } else { return null; @@ -90,25 +88,22 @@ public void delete(String resourceGroupName, String catalogName, Context context this.serviceClient().delete(resourceGroupName, catalogName, context); } - public Response countDevicesWithResponse( - String resourceGroupName, String catalogName, Context context) { - Response inner = - this.serviceClient().countDevicesWithResponse(resourceGroupName, catalogName, context); + public Response countDevicesWithResponse(String resourceGroupName, String catalogName, + Context context) { + Response inner + = this.serviceClient().countDevicesWithResponse(resourceGroupName, catalogName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CountDeviceResponseImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new CountDevicesResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public CountDeviceResponse countDevices(String resourceGroupName, String catalogName) { - CountDeviceResponseInner inner = this.serviceClient().countDevices(resourceGroupName, catalogName); + public CountDevicesResponse countDevices(String resourceGroupName, String catalogName) { + CountDevicesResponseInner inner = this.serviceClient().countDevices(resourceGroupName, catalogName); if (inner != null) { - return new CountDeviceResponseImpl(inner, this.manager()); + return new CountDevicesResponseImpl(inner, this.manager()); } else { return null; } @@ -116,159 +111,117 @@ public CountDeviceResponse countDevices(String resourceGroupName, String catalog public PagedIterable listDeployments(String resourceGroupName, String catalogName) { PagedIterable inner = this.serviceClient().listDeployments(resourceGroupName, catalogName); - return Utils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); } - public PagedIterable listDeployments( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - PagedIterable inner = - this - .serviceClient() - .listDeployments(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); + public PagedIterable listDeployments(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner = this.serviceClient().listDeployments(resourceGroupName, catalogName, + filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); } - public PagedIterable listDeviceGroups( - String resourceGroupName, String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest) { - PagedIterable inner = - this.serviceClient().listDeviceGroups(resourceGroupName, catalogName, listDeviceGroupsRequest); - return Utils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); + public PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest) { + PagedIterable inner + = this.serviceClient().listDeviceGroups(resourceGroupName, catalogName, listDeviceGroupsRequest); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); } - public PagedIterable listDeviceGroups( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + public PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - PagedIterable inner = - this - .serviceClient() - .listDeviceGroups( - resourceGroupName, catalogName, listDeviceGroupsRequest, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); + PagedIterable inner = this.serviceClient().listDeviceGroups(resourceGroupName, catalogName, + listDeviceGroupsRequest, filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); } public PagedIterable listDeviceInsights(String resourceGroupName, String catalogName) { - PagedIterable inner = - this.serviceClient().listDeviceInsights(resourceGroupName, catalogName); - return Utils.mapPage(inner, inner1 -> new DeviceInsightImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listDeviceInsights(resourceGroupName, catalogName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceInsightImpl(inner1, this.manager())); } - public PagedIterable listDeviceInsights( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - PagedIterable inner = - this - .serviceClient() - .listDeviceInsights(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new DeviceInsightImpl(inner1, this.manager())); + public PagedIterable listDeviceInsights(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner = this.serviceClient().listDeviceInsights(resourceGroupName, + catalogName, filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceInsightImpl(inner1, this.manager())); } public PagedIterable listDevices(String resourceGroupName, String catalogName) { PagedIterable inner = this.serviceClient().listDevices(resourceGroupName, catalogName); - return Utils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); + } + + public PagedIterable listDevices(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner + = this.serviceClient().listDevices(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); + } + + public void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest) { + this.serviceClient().uploadImage(resourceGroupName, catalogName, uploadImageRequest); } - public PagedIterable listDevices( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + public void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest, Context context) { - PagedIterable inner = - this.serviceClient().listDevices(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); + this.serviceClient().uploadImage(resourceGroupName, catalogName, uploadImageRequest, context); } public Catalog getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, catalogName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, catalogName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } this.delete(resourceGroupName, catalogName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } this.delete(resourceGroupName, catalogName, context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateChainResponseImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateChainResponseImpl.java index c4f161c524887..42b6500cfaf00 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateChainResponseImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateChainResponseImpl.java @@ -12,8 +12,8 @@ public final class CertificateChainResponseImpl implements CertificateChainRespo private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - CertificateChainResponseImpl( - CertificateChainResponseInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + CertificateChainResponseImpl(CertificateChainResponseInner innerObject, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateImpl.java index 89bed5e2be67d..91f8136a0edd8 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificateImpl.java @@ -7,9 +7,7 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.sphere.fluent.models.CertificateInner; import com.azure.resourcemanager.sphere.models.Certificate; -import com.azure.resourcemanager.sphere.models.CertificateStatus; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import java.time.OffsetDateTime; +import com.azure.resourcemanager.sphere.models.CertificateProperties; public final class CertificateImpl implements Certificate { private CertificateInner innerObject; @@ -33,36 +31,12 @@ public String type() { return this.innerModel().type(); } - public SystemData systemData() { - return this.innerModel().systemData(); - } - - public String certificate() { - return this.innerModel().certificate(); - } - - public CertificateStatus status() { - return this.innerModel().status(); - } - - public String subject() { - return this.innerModel().subject(); + public CertificateProperties properties() { + return this.innerModel().properties(); } - public String thumbprint() { - return this.innerModel().thumbprint(); - } - - public OffsetDateTime expiryUtc() { - return this.innerModel().expiryUtc(); - } - - public OffsetDateTime notBeforeUtc() { - return this.innerModel().notBeforeUtc(); - } - - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public CertificateInner innerModel() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesClientImpl.java index 317dfa9c52f3e..0aef1cf556fa3 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesClientImpl.java @@ -35,110 +35,91 @@ import com.azure.resourcemanager.sphere.models.ProofOfPossessionNonceRequest; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CertificatesClient. */ +/** + * An instance of this class provides access to all the operations defined in CertificatesClient. + */ public final class CertificatesClientImpl implements CertificatesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CertificatesService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of CertificatesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - CertificatesClientImpl(AzureSphereManagementClientImpl client) { - this.service = - RestProxy.create(CertificatesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + CertificatesClientImpl(AzureSphereMgmtClientImpl client) { + this.service + = RestProxy.create(CertificatesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientCertificates to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for AzureSphereMgmtClientCertificates to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface CertificatesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCatalog( - @HostParam("$host") String endpoint, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates/{serialNumber}") - @ExpectedResponses({200}) + Mono> listByCatalog(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates/{serialNumber}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("serialNumber") String serialNumber, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates/{serialNumber}/retrieveCertChain") - @ExpectedResponses({200}) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("serialNumber") String serialNumber, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates/{serialNumber}/retrieveCertChain") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> retrieveCertChain( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("serialNumber") String serialNumber, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates/{serialNumber}/retrieveProofOfPossessionNonce") - @ExpectedResponses({200}) + Mono> retrieveCertChain(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("serialNumber") String serialNumber, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/certificates/{serialNumber}/retrieveProofOfPossessionNonce") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> retrieveProofOfPossessionNonce( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + @HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, @PathParam("serialNumber") String serialNumber, @BodyParam("application/json") ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByCatalogNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -149,22 +130,18 @@ Mono> listByCatalogNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Certificate list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByCatalogSinglePageAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private Mono> listByCatalogSinglePageAsync(String resourceGroupName, + String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -175,36 +152,17 @@ private Mono> listByCatalogSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCatalog( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByCatalog(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -216,28 +174,18 @@ private Mono> listByCatalogSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Certificate list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByCatalogSinglePageAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listByCatalogSinglePageAsync(String resourceGroupName, + String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -249,32 +197,15 @@ private Mono> listByCatalogSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCatalog( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByCatalog(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -287,8 +218,8 @@ private Mono> listByCatalogSinglePageAsync( * @return the response of a Certificate list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByCatalogAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private PagedFlux listByCatalogAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize) { return new PagedFlux<>( () -> listByCatalogSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize), nextLink -> listByCatalogNextSinglePageAsync(nextLink)); @@ -296,7 +227,7 @@ private PagedFlux listByCatalogAsync( /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -317,7 +248,7 @@ private PagedFlux listByCatalogAsync(String resourceGroupName, /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -331,14 +262,8 @@ private PagedFlux listByCatalogAsync(String resourceGroupName, * @return the response of a Certificate list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByCatalogAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private PagedFlux listByCatalogAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedFlux<>( () -> listByCatalogSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context), nextLink -> listByCatalogNextSinglePageAsync(nextLink, context)); @@ -346,7 +271,7 @@ private PagedFlux listByCatalogAsync( /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -365,7 +290,7 @@ public PagedIterable listByCatalog(String resourceGroupName, S /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -379,21 +304,15 @@ public PagedIterable listByCatalog(String resourceGroupName, S * @return the response of a Certificate list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + public PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedIterable<>( listByCatalogAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context)); } /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -403,19 +322,15 @@ public PagedIterable listByCatalog( * @return a Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String serialNumber) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String serialNumber) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -429,24 +344,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - serialNumber, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, serialNumber, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -457,19 +362,15 @@ private Mono> getWithResponseAsync( * @return a Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String serialNumber, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String serialNumber, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -483,21 +384,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - serialNumber, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, serialNumber, accept, context); } /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -514,7 +407,7 @@ private Mono getAsync(String resourceGroupName, String catalog /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -525,14 +418,14 @@ private Mono getAsync(String resourceGroupName, String catalog * @return a Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context) { + public Response getWithResponse(String resourceGroupName, String catalogName, String serialNumber, + Context context) { return getWithResponseAsync(resourceGroupName, catalogName, serialNumber, context).block(); } /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -548,7 +441,7 @@ public CertificateInner get(String resourceGroupName, String catalogName, String /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -558,19 +451,15 @@ public CertificateInner get(String resourceGroupName, String catalogName, String * @return the certificate chain response along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> retrieveCertChainWithResponseAsync( - String resourceGroupName, String catalogName, String serialNumber) { + private Mono> retrieveCertChainWithResponseAsync(String resourceGroupName, + String catalogName, String serialNumber) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -584,24 +473,14 @@ private Mono> retrieveCertChainWithRespo } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .retrieveCertChain( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - serialNumber, - accept, - context)) + .withContext(context -> service.retrieveCertChain(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, serialNumber, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -612,19 +491,15 @@ private Mono> retrieveCertChainWithRespo * @return the certificate chain response along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> retrieveCertChainWithResponseAsync( - String resourceGroupName, String catalogName, String serialNumber, Context context) { + private Mono> retrieveCertChainWithResponseAsync(String resourceGroupName, + String catalogName, String serialNumber, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -638,21 +513,13 @@ private Mono> retrieveCertChainWithRespo } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .retrieveCertChain( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - serialNumber, - accept, - context); + return service.retrieveCertChain(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, serialNumber, accept, context); } /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -662,15 +529,15 @@ private Mono> retrieveCertChainWithRespo * @return the certificate chain response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono retrieveCertChainAsync( - String resourceGroupName, String catalogName, String serialNumber) { + private Mono retrieveCertChainAsync(String resourceGroupName, String catalogName, + String serialNumber) { return retrieveCertChainWithResponseAsync(resourceGroupName, catalogName, serialNumber) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -681,14 +548,14 @@ private Mono retrieveCertChainAsync( * @return the certificate chain response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response retrieveCertChainWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context) { + public Response retrieveCertChainWithResponse(String resourceGroupName, + String catalogName, String serialNumber, Context context) { return retrieveCertChainWithResponseAsync(resourceGroupName, catalogName, serialNumber, context).block(); } /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -698,14 +565,14 @@ public Response retrieveCertChainWithResponse( * @return the certificate chain response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CertificateChainResponseInner retrieveCertChain( - String resourceGroupName, String catalogName, String serialNumber) { + public CertificateChainResponseInner retrieveCertChain(String resourceGroupName, String catalogName, + String serialNumber) { return retrieveCertChainWithResponse(resourceGroupName, catalogName, serialNumber, Context.NONE).getValue(); } /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -717,21 +584,15 @@ public CertificateChainResponseInner retrieveCertChain( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> retrieveProofOfPossessionNonceWithResponseAsync( - String resourceGroupName, - String catalogName, - String serialNumber, + String resourceGroupName, String catalogName, String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -744,34 +605,22 @@ private Mono> retrieveProofOfPosse return Mono.error(new IllegalArgumentException("Parameter serialNumber is required and cannot be null.")); } if (proofOfPossessionNonceRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter proofOfPossessionNonceRequest is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter proofOfPossessionNonceRequest is required and cannot be null.")); } else { proofOfPossessionNonceRequest.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .retrieveProofOfPossessionNonce( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - serialNumber, - proofOfPossessionNonceRequest, - accept, - context)) + .withContext(context -> service.retrieveProofOfPossessionNonce(this.client.getEndpoint(), + this.client.getApiVersion(), this.client.getSubscriptionId(), resourceGroupName, catalogName, + serialNumber, proofOfPossessionNonceRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -784,22 +633,15 @@ private Mono> retrieveProofOfPosse */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> retrieveProofOfPossessionNonceWithResponseAsync( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, - Context context) { + String resourceGroupName, String catalogName, String serialNumber, + ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -812,31 +654,21 @@ private Mono> retrieveProofOfPosse return Mono.error(new IllegalArgumentException("Parameter serialNumber is required and cannot be null.")); } if (proofOfPossessionNonceRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter proofOfPossessionNonceRequest is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter proofOfPossessionNonceRequest is required and cannot be null.")); } else { proofOfPossessionNonceRequest.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .retrieveProofOfPossessionNonce( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - serialNumber, - proofOfPossessionNonceRequest, - accept, - context); + return service.retrieveProofOfPossessionNonce(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, serialNumber, + proofOfPossessionNonceRequest, accept, context); } /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -847,19 +679,15 @@ private Mono> retrieveProofOfPosse * @return the proof of possession nonce on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono retrieveProofOfPossessionNonceAsync( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { - return retrieveProofOfPossessionNonceWithResponseAsync( - resourceGroupName, catalogName, serialNumber, proofOfPossessionNonceRequest) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono retrieveProofOfPossessionNonceAsync(String resourceGroupName, + String catalogName, String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { + return retrieveProofOfPossessionNonceWithResponseAsync(resourceGroupName, catalogName, serialNumber, + proofOfPossessionNonceRequest).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -872,19 +700,15 @@ private Mono retrieveProofOfPossessionNonce */ @ServiceMethod(returns = ReturnType.SINGLE) public Response retrieveProofOfPossessionNonceWithResponse( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, - Context context) { - return retrieveProofOfPossessionNonceWithResponseAsync( - resourceGroupName, catalogName, serialNumber, proofOfPossessionNonceRequest, context) - .block(); + String resourceGroupName, String catalogName, String serialNumber, + ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, Context context) { + return retrieveProofOfPossessionNonceWithResponseAsync(resourceGroupName, catalogName, serialNumber, + proofOfPossessionNonceRequest, context).block(); } /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -895,26 +719,23 @@ public Response retrieveProofOfPossessionNo * @return the proof of possession nonce. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ProofOfPossessionNonceResponseInner retrieveProofOfPossessionNonce( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { - return retrieveProofOfPossessionNonceWithResponse( - resourceGroupName, catalogName, serialNumber, proofOfPossessionNonceRequest, Context.NONE) - .getValue(); + public ProofOfPossessionNonceResponseInner retrieveProofOfPossessionNonce(String resourceGroupName, + String catalogName, String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { + return retrieveProofOfPossessionNonceWithResponse(resourceGroupName, catalogName, serialNumber, + proofOfPossessionNonceRequest, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Certificate list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByCatalogNextSinglePageAsync(String nextLink) { @@ -922,37 +743,29 @@ private Mono> listByCatalogNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Certificate list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByCatalogNextSinglePageAsync(String nextLink, Context context) { @@ -960,23 +773,13 @@ private Mono> listByCatalogNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesImpl.java index d8edb696338e5..d0bfd69c21d81 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CertificatesImpl.java @@ -26,39 +26,30 @@ public final class CertificatesImpl implements Certificates { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - public CertificatesImpl( - CertificatesClient innerClient, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + public CertificatesImpl(CertificatesClient innerClient, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByCatalog(String resourceGroupName, String catalogName) { PagedIterable inner = this.serviceClient().listByCatalog(resourceGroupName, catalogName); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } - public PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - PagedIterable inner = - this.serviceClient().listByCatalog(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + public PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner = this.serviceClient().listByCatalog(resourceGroupName, catalogName, + filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, catalogName, serialNumber, context); + public Response getWithResponse(String resourceGroupName, String catalogName, String serialNumber, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, catalogName, serialNumber, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CertificateImpl(inner.getValue(), this.manager())); } else { return null; @@ -74,25 +65,22 @@ public Certificate get(String resourceGroupName, String catalogName, String seri } } - public Response retrieveCertChainWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context) { - Response inner = - this.serviceClient().retrieveCertChainWithResponse(resourceGroupName, catalogName, serialNumber, context); + public Response retrieveCertChainWithResponse(String resourceGroupName, + String catalogName, String serialNumber, Context context) { + Response inner + = this.serviceClient().retrieveCertChainWithResponse(resourceGroupName, catalogName, serialNumber, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CertificateChainResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public CertificateChainResponse retrieveCertChain( - String resourceGroupName, String catalogName, String serialNumber) { - CertificateChainResponseInner inner = - this.serviceClient().retrieveCertChain(resourceGroupName, catalogName, serialNumber); + public CertificateChainResponse retrieveCertChain(String resourceGroupName, String catalogName, + String serialNumber) { + CertificateChainResponseInner inner + = this.serviceClient().retrieveCertChain(resourceGroupName, catalogName, serialNumber); if (inner != null) { return new CertificateChainResponseImpl(inner, this.manager()); } else { @@ -100,38 +88,24 @@ public CertificateChainResponse retrieveCertChain( } } - public Response retrieveProofOfPossessionNonceWithResponse( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, + public Response retrieveProofOfPossessionNonceWithResponse(String resourceGroupName, + String catalogName, String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, Context context) { - Response inner = - this - .serviceClient() - .retrieveProofOfPossessionNonceWithResponse( - resourceGroupName, catalogName, serialNumber, proofOfPossessionNonceRequest, context); + Response inner + = this.serviceClient().retrieveProofOfPossessionNonceWithResponse(resourceGroupName, catalogName, + serialNumber, proofOfPossessionNonceRequest, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ProofOfPossessionNonceResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public ProofOfPossessionNonceResponse retrieveProofOfPossessionNonce( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { - ProofOfPossessionNonceResponseInner inner = - this - .serviceClient() - .retrieveProofOfPossessionNonce( - resourceGroupName, catalogName, serialNumber, proofOfPossessionNonceRequest); + public ProofOfPossessionNonceResponse retrieveProofOfPossessionNonce(String resourceGroupName, String catalogName, + String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest) { + ProofOfPossessionNonceResponseInner inner = this.serviceClient().retrieveProofOfPossessionNonce( + resourceGroupName, catalogName, serialNumber, proofOfPossessionNonceRequest); if (inner != null) { return new ProofOfPossessionNonceResponseImpl(inner, this.manager()); } else { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CountDeviceResponseImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CountDevicesResponseImpl.java similarity index 62% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CountDeviceResponseImpl.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CountDevicesResponseImpl.java index fb2b9e03cdc00..b27e7d871641f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CountDeviceResponseImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/CountDevicesResponseImpl.java @@ -4,16 +4,16 @@ package com.azure.resourcemanager.sphere.implementation; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; -public final class CountDeviceResponseImpl implements CountDeviceResponse { - private CountDeviceResponseInner innerObject; +public final class CountDevicesResponseImpl implements CountDevicesResponse { + private CountDevicesResponseInner innerObject; private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - CountDeviceResponseImpl( - CountDeviceResponseInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + CountDevicesResponseImpl(CountDevicesResponseInner innerObject, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -22,7 +22,7 @@ public int value() { return this.innerModel().value(); } - public CountDeviceResponseInner innerModel() { + public CountDevicesResponseInner innerModel() { return this.innerObject; } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentImpl.java index 1f1923be9b2be..7a5c2b0f848eb 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentImpl.java @@ -4,16 +4,11 @@ package com.azure.resourcemanager.sphere.implementation; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; -import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.Deployment; -import com.azure.resourcemanager.sphere.models.Image; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import java.time.OffsetDateTime; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; +import com.azure.resourcemanager.sphere.models.DeploymentProperties; public final class DeploymentImpl implements Deployment, Deployment.Definition, Deployment.Update { private DeploymentInner innerObject; @@ -32,27 +27,12 @@ public String type() { return this.innerModel().type(); } - public String deploymentId() { - return this.innerModel().deploymentId(); + public DeploymentProperties properties() { + return this.innerModel().properties(); } - public List deployedImages() { - List inner = this.innerModel().deployedImages(); - if (inner != null) { - return Collections - .unmodifiableList( - inner.stream().map(inner1 -> new ImageImpl(inner1, this.manager())).collect(Collectors.toList())); - } else { - return Collections.emptyList(); - } - } - - public OffsetDateTime deploymentDateUtc() { - return this.innerModel().deploymentDateUtc(); - } - - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public String resourceGroupName() { @@ -77,8 +57,8 @@ private com.azure.resourcemanager.sphere.AzureSphereManager manager() { private String deploymentName; - public DeploymentImpl withExistingDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public DeploymentImpl withExistingDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { this.resourceGroupName = resourceGroupName; this.catalogName = catalogName; this.productName = productName; @@ -87,34 +67,14 @@ public DeploymentImpl withExistingDeviceGroup( } public Deployment create() { - this.innerObject = - serviceManager - .serviceClient() - .getDeployments() - .createOrUpdate( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - this.innerModel(), - Context.NONE); + this.innerObject = serviceManager.serviceClient().getDeployments().createOrUpdate(resourceGroupName, + catalogName, productName, deviceGroupName, deploymentName, this.innerModel(), Context.NONE); return this; } public Deployment create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeployments() - .createOrUpdate( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - this.innerModel(), - context); + this.innerObject = serviceManager.serviceClient().getDeployments().createOrUpdate(resourceGroupName, + catalogName, productName, deviceGroupName, deploymentName, this.innerModel(), context); return this; } @@ -129,75 +89,43 @@ public DeploymentImpl update() { } public Deployment apply() { - this.innerObject = - serviceManager - .serviceClient() - .getDeployments() - .createOrUpdate( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - this.innerModel(), - Context.NONE); + this.innerObject = serviceManager.serviceClient().getDeployments().createOrUpdate(resourceGroupName, + catalogName, productName, deviceGroupName, deploymentName, this.innerModel(), Context.NONE); return this; } public Deployment apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeployments() - .createOrUpdate( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - this.innerModel(), - context); + this.innerObject = serviceManager.serviceClient().getDeployments().createOrUpdate(resourceGroupName, + catalogName, productName, deviceGroupName, deploymentName, this.innerModel(), context); return this; } DeploymentImpl(DeploymentInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.catalogName = Utils.getValueFromIdByName(innerObject.id(), "catalogs"); - this.productName = Utils.getValueFromIdByName(innerObject.id(), "products"); - this.deviceGroupName = Utils.getValueFromIdByName(innerObject.id(), "deviceGroups"); - this.deploymentName = Utils.getValueFromIdByName(innerObject.id(), "deployments"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.catalogName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "catalogs"); + this.productName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "products"); + this.deviceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "deviceGroups"); + this.deploymentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "deployments"); } public Deployment refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getDeployments() - .getWithResponse( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeployments() + .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, Context.NONE) + .getValue(); return this; } public Deployment refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeployments() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context) - .getValue(); - return this; - } - - public DeploymentImpl withDeploymentId(String deploymentId) { - this.innerModel().withDeploymentId(deploymentId); + this.innerObject = serviceManager.serviceClient().getDeployments() + .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context) + .getValue(); return this; } - public DeploymentImpl withDeployedImages(List deployedImages) { - this.innerModel().withDeployedImages(deployedImages); + public DeploymentImpl withProperties(DeploymentProperties properties) { + this.innerModel().withProperties(properties); return this; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsClientImpl.java index 8e614c30d24c4..a1f267992d0f9 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsClientImpl.java @@ -38,119 +38,94 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DeploymentsClient. */ +/** + * An instance of this class provides access to all the operations defined in DeploymentsClient. + */ public final class DeploymentsClientImpl implements DeploymentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DeploymentsService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of DeploymentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - DeploymentsClientImpl(AzureSphereManagementClientImpl client) { - this.service = - RestProxy.create(DeploymentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + DeploymentsClientImpl(AzureSphereMgmtClientImpl client) { + this.service + = RestProxy.create(DeploymentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientDeployments to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for AzureSphereMgmtClientDeployments to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface DeploymentsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByDeviceGroup( - @HostParam("$host") String endpoint, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByDeviceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments/{deploymentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments/{deploymentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deploymentName") String deploymentName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deploymentName") String deploymentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments/{deploymentName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments/{deploymentName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deploymentName") String deploymentName, - @BodyParam("application/json") DeploymentInner resource, - @HeaderParam("Accept") String accept, - Context context); + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deploymentName") String deploymentName, @BodyParam("application/json") DeploymentInner resource, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments/{deploymentName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/deployments/{deploymentName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deploymentName") String deploymentName, - @HeaderParam("Accept") String accept, - Context context); + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deploymentName") String deploymentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByDeviceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -163,29 +138,19 @@ Mono> listByDeviceGroupNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDeviceGroupSinglePageAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, + private Mono> listByDeviceGroupSinglePageAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -203,39 +168,18 @@ private Mono> listByDeviceGroupSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByDeviceGroup( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByDeviceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, + productName, deviceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -249,30 +193,19 @@ private Mono> listByDeviceGroupSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDeviceGroupSinglePageAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listByDeviceGroupSinglePageAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String filter, Integer top, Integer skip, + Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -291,35 +224,17 @@ private Mono> listByDeviceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByDeviceGroup( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, + .listByDeviceGroup(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, productName, deviceGroupName, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -334,26 +249,17 @@ private Mono> listByDeviceGroupSinglePageAsync( * @return the response of a Deployment list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDeviceGroupAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize) { - return new PagedFlux<>( - () -> - listByDeviceGroupSinglePageAsync( - resourceGroupName, catalogName, productName, deviceGroupName, filter, top, skip, maxpagesize), + private PagedFlux listByDeviceGroupAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize) { + return new PagedFlux<>(() -> listByDeviceGroupSinglePageAsync(resourceGroupName, catalogName, productName, + deviceGroupName, filter, top, skip, maxpagesize), nextLink -> listByDeviceGroupNextSinglePageAsync(nextLink)); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -364,23 +270,21 @@ private PagedFlux listByDeviceGroupAsync( * @return the response of a Deployment list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDeviceGroupAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private PagedFlux listByDeviceGroupAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { final String filter = null; final Integer top = null; final Integer skip = null; final Integer maxpagesize = null; - return new PagedFlux<>( - () -> - listByDeviceGroupSinglePageAsync( - resourceGroupName, catalogName, productName, deviceGroupName, filter, top, skip, maxpagesize), + return new PagedFlux<>(() -> listByDeviceGroupSinglePageAsync(resourceGroupName, catalogName, productName, + deviceGroupName, filter, top, skip, maxpagesize), nextLink -> listByDeviceGroupNextSinglePageAsync(nextLink)); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -396,35 +300,18 @@ private PagedFlux listByDeviceGroupAsync( * @return the response of a Deployment list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDeviceGroupAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + private PagedFlux listByDeviceGroupAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return new PagedFlux<>( - () -> - listByDeviceGroupSinglePageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - filter, - top, - skip, - maxpagesize, - context), + return new PagedFlux<>(() -> listByDeviceGroupSinglePageAsync(resourceGroupName, catalogName, productName, + deviceGroupName, filter, top, skip, maxpagesize, context), nextLink -> listByDeviceGroupNextSinglePageAsync(nextLink, context)); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -435,21 +322,20 @@ private PagedFlux listByDeviceGroupAsync( * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { final String filter = null; final Integer top = null; final Integer skip = null; final Integer maxpagesize = null; - return new PagedIterable<>( - listByDeviceGroupAsync( - resourceGroupName, catalogName, productName, deviceGroupName, filter, top, skip, maxpagesize)); + return new PagedIterable<>(listByDeviceGroupAsync(resourceGroupName, catalogName, productName, deviceGroupName, + filter, top, skip, maxpagesize)); } /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -465,54 +351,38 @@ public PagedIterable listByDeviceGroup( * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByDeviceGroup( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { - return new PagedIterable<>( - listByDeviceGroupAsync( - resourceGroupName, catalogName, productName, deviceGroupName, filter, top, skip, maxpagesize, context)); + return new PagedIterable<>(listByDeviceGroupAsync(resourceGroupName, catalogName, productName, deviceGroupName, + filter, top, skip, maxpagesize, context)); } /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Deployment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -533,33 +403,22 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deploymentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -567,24 +426,15 @@ private Mono> getWithResponseAsync( * @return a Deployment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -605,42 +455,28 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, accept, context); } /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Deployment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName) { + private Mono getAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName) { return getWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -648,13 +484,13 @@ private Mono getAsync( /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -662,81 +498,62 @@ private Mono getAsync( * @return a Deployment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { - return getWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context) - .block(); + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, Context context) { + return getWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + context).block(); } /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Deployment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeploymentInner get( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + public DeploymentInner get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName) { - return getWithResponse( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, Context.NONE) - .getValue(); + return getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + Context.NONE).getValue(); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an deployment resource belonging to a device group resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -762,62 +579,41 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deploymentName, resource, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an deployment resource belonging to a device group resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -843,31 +639,21 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deploymentName, resource, accept, context); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -875,36 +661,25 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeploymentInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, + private PollerFlux, DeploymentInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DeploymentInner.class, - DeploymentInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, deploymentName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeploymentInner.class, DeploymentInner.class, this.client.getContext()); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -913,34 +688,26 @@ private PollerFlux, DeploymentInner> beginCreateOrUp * @return the {@link PollerFlux} for polling of an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeploymentInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, + private PollerFlux, DeploymentInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeploymentInner.class, DeploymentInner.class, context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, deploymentName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeploymentInner.class, DeploymentInner.class, context); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -948,29 +715,23 @@ private PollerFlux, DeploymentInner> beginCreateOrUp * @return the {@link SyncPoller} for polling of an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeploymentInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, + public SyncPoller, DeploymentInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource) - .getSyncPoller(); + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, + deploymentName, resource).getSyncPoller(); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -979,30 +740,23 @@ public SyncPoller, DeploymentInner> beginCreateOrUpd * @return the {@link SyncPoller} for polling of an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeploymentInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, + public SyncPoller, DeploymentInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deploymentName, DeploymentInner resource, Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource, context) - .getSyncPoller(); + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, + deploymentName, resource, context).getSyncPoller(); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1010,29 +764,22 @@ public SyncPoller, DeploymentInner> beginCreateOrUpd * @return an deployment resource belonging to a device group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource) { - return beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, DeploymentInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + resource).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1041,30 +788,22 @@ private Mono createOrUpdateAsync( * @return an deployment resource belonging to a device group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, DeploymentInner resource, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + resource, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1072,28 +811,22 @@ private Mono createOrUpdateAsync( * @return an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeploymentInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource) { - return createOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource) - .block(); + public DeploymentInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, DeploymentInner resource) { + return createOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + resource).block(); } /** * Create a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1102,52 +835,37 @@ public DeploymentInner createOrUpdate( * @return an deployment resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeploymentInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - DeploymentInner resource, - Context context) { - return createOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, resource, context) - .block(); + public DeploymentInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, DeploymentInner resource, Context context) { + return createOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + resource, context).block(); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1168,33 +886,22 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deploymentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1202,24 +909,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1240,60 +938,44 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deploymentName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, accept, context); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1301,59 +983,47 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono = deleteWithResponseAsync(resourceGroupName, catalogName, productName, + deviceGroupName, deploymentName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName) { - return this - .beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName) + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName) { + return this.beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName) .getSyncPoller(); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1361,13 +1031,8 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deploymentName, Context context) { return this .beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context) .getSyncPoller(); @@ -1376,40 +1041,35 @@ public SyncPoller, Void> beginDelete( /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName) { - return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName) - .last() + private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName) { + return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1417,38 +1077,28 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { + private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, Context context) { return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName) { deleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName).block(); } @@ -1456,39 +1106,35 @@ public void delete( /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deploymentName, Context context) { deleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByDeviceGroupNextSinglePageAsync(String nextLink) { @@ -1496,62 +1142,44 @@ private Mono> listByDeviceGroupNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByDeviceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDeviceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByDeviceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByDeviceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByDeviceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsImpl.java index eb743c5dd580b..8c7fadcc8dcf1 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeploymentsImpl.java @@ -21,75 +21,42 @@ public final class DeploymentsImpl implements Deployments { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - public DeploymentsImpl( - DeploymentsClient innerClient, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + public DeploymentsImpl(DeploymentsClient innerClient, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { - PagedIterable inner = - this.serviceClient().listByDeviceGroup(resourceGroupName, catalogName, productName, deviceGroupName); - return Utils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { + PagedIterable inner + = this.serviceClient().listByDeviceGroup(resourceGroupName, catalogName, productName, deviceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); } - public PagedIterable listByDeviceGroup( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - PagedIterable inner = - this - .serviceClient() - .listByDeviceGroup( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - filter, - top, - skip, - maxpagesize, - context); - return Utils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner = this.serviceClient().listByDeviceGroup(resourceGroupName, catalogName, + productName, deviceGroupName, filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeploymentImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context); + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, catalogName, + productName, deviceGroupName, deploymentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DeploymentImpl(inner.getValue(), this.manager())); } else { return null; } } - public Deployment get( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + public Deployment get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName) { - DeploymentInner inner = - this.serviceClient().get(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName); + DeploymentInner inner + = this.serviceClient().get(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName); if (inner != null) { return new DeploymentImpl(inner, this.manager()); } else { @@ -97,63 +64,42 @@ public Deployment get( } } - public void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName) { this.serviceClient().delete(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName); } - public void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context) { - this - .serviceClient() - .delete(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context); + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deploymentName, Context context) { + this.serviceClient().delete(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + context); } public Deployment getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deploymentName = Utils.getValueFromIdByName(id, "deployments"); + String deploymentName = ResourceManagerUtils.getValueFromIdByName(id, "deployments"); if (deploymentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); } return this .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, Context.NONE) @@ -161,122 +107,89 @@ public Deployment getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deploymentName = Utils.getValueFromIdByName(id, "deployments"); + String deploymentName = ResourceManagerUtils.getValueFromIdByName(id, "deployments"); if (deploymentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); } - return this - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context); + return this.getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, + context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deploymentName = Utils.getValueFromIdByName(id, "deployments"); + String deploymentName = ResourceManagerUtils.getValueFromIdByName(id, "deployments"); if (deploymentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); } this.delete(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deploymentName = Utils.getValueFromIdByName(id, "deployments"); + String deploymentName = ResourceManagerUtils.getValueFromIdByName(id, "deployments"); if (deploymentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deployments'.", id))); } this.delete(resourceGroupName, catalogName, productName, deviceGroupName, deploymentName, context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupImpl.java index b62722fd63eeb..a2563e1f77859 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupImpl.java @@ -5,17 +5,15 @@ package com.azure.resourcemanager.sphere.implementation; import com.azure.core.http.rest.Response; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; -import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; import com.azure.resourcemanager.sphere.models.ClaimDevicesRequest; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import com.azure.resourcemanager.sphere.models.DeviceGroup; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.DeviceGroupUpdate; -import com.azure.resourcemanager.sphere.models.OSFeedType; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; -import com.azure.resourcemanager.sphere.models.UpdatePolicy; +import com.azure.resourcemanager.sphere.models.DeviceGroupUpdateProperties; public final class DeviceGroupImpl implements DeviceGroup, DeviceGroup.Definition, DeviceGroup.Update { private DeviceGroupInner innerObject; @@ -34,32 +32,12 @@ public String type() { return this.innerModel().type(); } - public String description() { - return this.innerModel().description(); + public DeviceGroupProperties properties() { + return this.innerModel().properties(); } - public OSFeedType osFeedType() { - return this.innerModel().osFeedType(); - } - - public UpdatePolicy updatePolicy() { - return this.innerModel().updatePolicy(); - } - - public AllowCrashDumpCollection allowCrashDumpsCollection() { - return this.innerModel().allowCrashDumpsCollection(); - } - - public RegionalDataBoundary regionalDataBoundary() { - return this.innerModel().regionalDataBoundary(); - } - - public Boolean hasDeployment() { - return this.innerModel().hasDeployment(); - } - - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public String resourceGroupName() { @@ -92,22 +70,14 @@ public DeviceGroupImpl withExistingProduct(String resourceGroupName, String cata } public DeviceGroup create() { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceGroups() - .createOrUpdate( - resourceGroupName, catalogName, productName, deviceGroupName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getDeviceGroups().createOrUpdate(resourceGroupName, + catalogName, productName, deviceGroupName, this.innerModel(), Context.NONE); return this; } public DeviceGroup create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceGroups() - .createOrUpdate( - resourceGroupName, catalogName, productName, deviceGroupName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getDeviceGroups().createOrUpdate(resourceGroupName, + catalogName, productName, deviceGroupName, this.innerModel(), context); return this; } @@ -123,125 +93,64 @@ public DeviceGroupImpl update() { } public DeviceGroup apply() { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceGroups() - .update(resourceGroupName, catalogName, productName, deviceGroupName, updateProperties, Context.NONE); + this.innerObject = serviceManager.serviceClient().getDeviceGroups().update(resourceGroupName, catalogName, + productName, deviceGroupName, updateProperties, Context.NONE); return this; } public DeviceGroup apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceGroups() - .update(resourceGroupName, catalogName, productName, deviceGroupName, updateProperties, context); + this.innerObject = serviceManager.serviceClient().getDeviceGroups().update(resourceGroupName, catalogName, + productName, deviceGroupName, updateProperties, context); return this; } DeviceGroupImpl(DeviceGroupInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.catalogName = Utils.getValueFromIdByName(innerObject.id(), "catalogs"); - this.productName = Utils.getValueFromIdByName(innerObject.id(), "products"); - this.deviceGroupName = Utils.getValueFromIdByName(innerObject.id(), "deviceGroups"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.catalogName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "catalogs"); + this.productName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "products"); + this.deviceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "deviceGroups"); } public DeviceGroup refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceGroups() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceGroups() + .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE).getValue(); return this; } public DeviceGroup refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceGroups() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceGroups() + .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, context).getValue(); return this; } public void claimDevices(ClaimDevicesRequest claimDevicesRequest) { - serviceManager - .deviceGroups() - .claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest); + serviceManager.deviceGroups().claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest); } public void claimDevices(ClaimDevicesRequest claimDevicesRequest, Context context) { - serviceManager - .deviceGroups() - .claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, context); + serviceManager.deviceGroups().claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest, context); } - public Response countDevicesWithResponse(Context context) { - return serviceManager - .deviceGroups() - .countDevicesWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, context); + public Response countDevicesWithResponse(Context context) { + return serviceManager.deviceGroups().countDevicesWithResponse(resourceGroupName, catalogName, productName, + deviceGroupName, context); } - public CountDeviceResponse countDevices() { + public CountDevicesResponse countDevices() { return serviceManager.deviceGroups().countDevices(resourceGroupName, catalogName, productName, deviceGroupName); } - public DeviceGroupImpl withDescription(String description) { - if (isInCreateMode()) { - this.innerModel().withDescription(description); - return this; - } else { - this.updateProperties.withDescription(description); - return this; - } - } - - public DeviceGroupImpl withOsFeedType(OSFeedType osFeedType) { - if (isInCreateMode()) { - this.innerModel().withOsFeedType(osFeedType); - return this; - } else { - this.updateProperties.withOsFeedType(osFeedType); - return this; - } - } - - public DeviceGroupImpl withUpdatePolicy(UpdatePolicy updatePolicy) { - if (isInCreateMode()) { - this.innerModel().withUpdatePolicy(updatePolicy); - return this; - } else { - this.updateProperties.withUpdatePolicy(updatePolicy); - return this; - } - } - - public DeviceGroupImpl withAllowCrashDumpsCollection(AllowCrashDumpCollection allowCrashDumpsCollection) { - if (isInCreateMode()) { - this.innerModel().withAllowCrashDumpsCollection(allowCrashDumpsCollection); - return this; - } else { - this.updateProperties.withAllowCrashDumpsCollection(allowCrashDumpsCollection); - return this; - } - } - - public DeviceGroupImpl withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary) { - if (isInCreateMode()) { - this.innerModel().withRegionalDataBoundary(regionalDataBoundary); - return this; - } else { - this.updateProperties.withRegionalDataBoundary(regionalDataBoundary); - return this; - } - } - - private boolean isInCreateMode() { - return this.innerModel().id() == null; + public DeviceGroupImpl withProperties(DeviceGroupProperties properties) { + this.innerModel().withProperties(properties); + return this; + } + + public DeviceGroupImpl withProperties(DeviceGroupUpdateProperties properties) { + this.updateProperties.withProperties(properties); + return this; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsClientImpl.java index 2e01c5ca4e09a..19c91bdbaf5fe 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsClientImpl.java @@ -34,7 +34,7 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.sphere.fluent.DeviceGroupsClient; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.models.ClaimDevicesRequest; import com.azure.resourcemanager.sphere.models.DeviceGroupListResult; @@ -43,165 +43,125 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DeviceGroupsClient. */ +/** + * An instance of this class provides access to all the operations defined in DeviceGroupsClient. + */ public final class DeviceGroupsClientImpl implements DeviceGroupsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DeviceGroupsService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of DeviceGroupsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - DeviceGroupsClientImpl(AzureSphereManagementClientImpl client) { - this.service = - RestProxy.create(DeviceGroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + DeviceGroupsClientImpl(AzureSphereMgmtClientImpl client) { + this.service + = RestProxy.create(DeviceGroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientDeviceGroups to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for AzureSphereMgmtClientDeviceGroups to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface DeviceGroupsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByProduct( - @HostParam("$host") String endpoint, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") - @ExpectedResponses({200}) + Mono> listByProduct(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") - @ExpectedResponses({200, 201}) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @BodyParam("application/json") DeviceGroupInner resource, - @HeaderParam("Accept") String accept, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @BodyParam("application/json") DeviceGroupInner resource, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @BodyParam("application/json") DeviceGroupUpdate properties, - @HeaderParam("Accept") String accept, + Mono>> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @BodyParam("application/json") DeviceGroupUpdate properties, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/claimDevices") - @ExpectedResponses({202}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/claimDevices") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> claimDevices( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, + Mono>> claimDevices(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, @BodyParam("application/json") ClaimDevicesRequest claimDevicesRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/countDevices") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/countDevices") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> countDevices( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @HeaderParam("Accept") String accept, - Context context); + Mono> countDevices(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByProductNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -213,28 +173,18 @@ Mono> listByProductNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByProductSinglePageAsync( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize) { + private Mono> listByProductSinglePageAsync(String resourceGroupName, + String catalogName, String productName, String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -248,38 +198,18 @@ private Mono> listByProductSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByProduct( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByProduct(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, + productName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -292,29 +222,19 @@ private Mono> listByProductSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByProductSinglePageAsync( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + private Mono> listByProductSinglePageAsync(String resourceGroupName, + String catalogName, String productName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -329,34 +249,16 @@ private Mono> listByProductSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByProduct( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByProduct(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, productName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -370,25 +272,16 @@ private Mono> listByProductSinglePageAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByProductAsync( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize) { - return new PagedFlux<>( - () -> - listByProductSinglePageAsync( - resourceGroupName, catalogName, productName, filter, top, skip, maxpagesize), - nextLink -> listByProductNextSinglePageAsync(nextLink)); + private PagedFlux listByProductAsync(String resourceGroupName, String catalogName, + String productName, String filter, Integer top, Integer skip, Integer maxpagesize) { + return new PagedFlux<>(() -> listByProductSinglePageAsync(resourceGroupName, catalogName, productName, filter, + top, skip, maxpagesize), nextLink -> listByProductNextSinglePageAsync(nextLink)); } /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -398,23 +291,20 @@ private PagedFlux listByProductAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByProductAsync( - String resourceGroupName, String catalogName, String productName) { + private PagedFlux listByProductAsync(String resourceGroupName, String catalogName, + String productName) { final String filter = null; final Integer top = null; final Integer skip = null; final Integer maxpagesize = null; - return new PagedFlux<>( - () -> - listByProductSinglePageAsync( - resourceGroupName, catalogName, productName, filter, top, skip, maxpagesize), - nextLink -> listByProductNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listByProductSinglePageAsync(resourceGroupName, catalogName, productName, filter, + top, skip, maxpagesize), nextLink -> listByProductNextSinglePageAsync(nextLink)); } /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -429,26 +319,16 @@ private PagedFlux listByProductAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByProductAsync( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - return new PagedFlux<>( - () -> - listByProductSinglePageAsync( - resourceGroupName, catalogName, productName, filter, top, skip, maxpagesize, context), - nextLink -> listByProductNextSinglePageAsync(nextLink, context)); + private PagedFlux listByProductAsync(String resourceGroupName, String catalogName, + String productName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { + return new PagedFlux<>(() -> listByProductSinglePageAsync(resourceGroupName, catalogName, productName, filter, + top, skip, maxpagesize, context), nextLink -> listByProductNextSinglePageAsync(nextLink, context)); } /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -458,8 +338,8 @@ private PagedFlux listByProductAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByProduct( - String resourceGroupName, String catalogName, String productName) { + public PagedIterable listByProduct(String resourceGroupName, String catalogName, + String productName) { final String filter = null; final Integer top = null; final Integer skip = null; @@ -471,7 +351,7 @@ public PagedIterable listByProduct( /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -486,15 +366,8 @@ public PagedIterable listByProduct( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByProduct( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + public PagedIterable listByProduct(String resourceGroupName, String catalogName, + String productName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedIterable<>( listByProductAsync(resourceGroupName, catalogName, productName, filter, top, skip, maxpagesize, context)); } @@ -502,7 +375,7 @@ public PagedIterable listByProduct( /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -513,19 +386,15 @@ public PagedIterable listByProduct( * @return a DeviceGroup along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -543,26 +412,16 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -574,19 +433,15 @@ private Mono> getWithResponseAsync( * @return a DeviceGroup along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -604,23 +459,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, accept, context); } /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -631,8 +477,8 @@ private Mono> getWithResponseAsync( * @return a DeviceGroup on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private Mono getAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { return getWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -640,7 +486,7 @@ private Mono getAsync( /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -652,15 +498,15 @@ private Mono getAsync( * @return a DeviceGroup along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context) { return getWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, context).block(); } /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -671,15 +517,15 @@ public Response getWithResponse( * @return a DeviceGroup. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceGroupInner get( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public DeviceGroupInner get(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { return getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE).getValue(); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -689,26 +535,18 @@ public DeviceGroupInner get( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device group resource belonging to a product resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -731,27 +569,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, resource, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -762,27 +589,18 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device group resource belonging to a product resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -805,24 +623,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, resource, + accept, context); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -835,27 +644,18 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, DeviceGroupInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + String resourceGroupName, String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DeviceGroupInner.class, - DeviceGroupInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceGroupInner.class, DeviceGroupInner.class, this.client.getContext()); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -869,26 +669,19 @@ private PollerFlux, DeviceGroupInner> beginCreateOr */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, DeviceGroupInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context) { + String resourceGroupName, String catalogName, String productName, String deviceGroupName, + DeviceGroupInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeviceGroupInner.class, DeviceGroupInner.class, context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceGroupInner.class, DeviceGroupInner.class, context); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -900,21 +693,16 @@ private PollerFlux, DeviceGroupInner> beginCreateOr * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceGroupInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource) + public SyncPoller, DeviceGroupInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource) { + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource) .getSyncPoller(); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -927,13 +715,8 @@ public SyncPoller, DeviceGroupInner> beginCreateOrU * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceGroupInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context) { + public SyncPoller, DeviceGroupInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupInner resource, Context context) { return this .beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource, context) .getSyncPoller(); @@ -942,7 +725,7 @@ public SyncPoller, DeviceGroupInner> beginCreateOrU /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -954,21 +737,16 @@ public SyncPoller, DeviceGroupInner> beginCreateOrU * @return an device group resource belonging to a product resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -981,22 +759,16 @@ private Mono createOrUpdateAsync( * @return an device group resource belonging to a product resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context) { + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupInner resource, Context context) { return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1008,19 +780,15 @@ private Mono createOrUpdateAsync( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceGroupInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource) { + public DeviceGroupInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupInner resource) { return createOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource).block(); } /** * Create a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1033,13 +801,8 @@ public DeviceGroupInner createOrUpdate( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceGroupInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupInner resource, - Context context) { + public DeviceGroupInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupInner resource, Context context) { return createOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, resource, context) .block(); } @@ -1047,7 +810,7 @@ public DeviceGroupInner createOrUpdate( /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1057,26 +820,18 @@ public DeviceGroupInner createOrUpdate( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device group resource belonging to a product resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, DeviceGroupUpdate properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1099,27 +854,16 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - properties, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + properties, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1130,27 +874,18 @@ private Mono>> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device group resource belonging to a product resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, DeviceGroupUpdate properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1173,24 +908,14 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - properties, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, properties, accept, context); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1202,28 +927,18 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeviceGroupInner> beginUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties) { - Mono>> mono = - updateWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DeviceGroupInner.class, - DeviceGroupInner.class, - this.client.getContext()); + private PollerFlux, DeviceGroupInner> beginUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupUpdate properties) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceGroupInner.class, DeviceGroupInner.class, this.client.getContext()); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1236,26 +951,19 @@ private PollerFlux, DeviceGroupInner> beginUpdateAs * @return the {@link PollerFlux} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeviceGroupInner> beginUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context) { + private PollerFlux, DeviceGroupInner> beginUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupUpdate properties, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeviceGroupInner.class, DeviceGroupInner.class, context); + Mono>> mono = updateWithResponseAsync(resourceGroupName, catalogName, productName, + deviceGroupName, properties, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceGroupInner.class, DeviceGroupInner.class, context); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1267,21 +975,16 @@ private PollerFlux, DeviceGroupInner> beginUpdateAs * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceGroupInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties) { - return this - .beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties) + public SyncPoller, DeviceGroupInner> beginUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupUpdate properties) { + return this.beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties) .getSyncPoller(); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1294,22 +997,16 @@ public SyncPoller, DeviceGroupInner> beginUpdate( * @return the {@link SyncPoller} for polling of an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceGroupInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context) { - return this - .beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties, context) + public SyncPoller, DeviceGroupInner> beginUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, DeviceGroupUpdate properties, Context context) { + return this.beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties, context) .getSyncPoller(); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1321,21 +1018,16 @@ public SyncPoller, DeviceGroupInner> beginUpdate( * @return an device group resource belonging to a product resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties) { - return beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties) - .last() + private Mono updateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupUpdate properties) { + return beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1348,22 +1040,16 @@ private Mono updateAsync( * @return an device group resource belonging to a product resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context) { + private Mono updateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupUpdate properties, Context context) { return beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1375,19 +1061,15 @@ private Mono updateAsync( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceGroupInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties) { + public DeviceGroupInner update(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupUpdate properties) { return updateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties).block(); } /** * Update a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1400,20 +1082,15 @@ public DeviceGroupInner update( * @return an device group resource belonging to a product resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceGroupInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - DeviceGroupUpdate properties, - Context context) { + public DeviceGroupInner update(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, DeviceGroupUpdate properties, Context context) { return updateAsync(resourceGroupName, catalogName, productName, deviceGroupName, properties, context).block(); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1424,19 +1101,15 @@ public DeviceGroupInner update( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1454,26 +1127,16 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1485,19 +1148,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1515,23 +1174,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, accept, context); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1542,20 +1192,18 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1567,20 +1215,19 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1591,15 +1238,15 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { return this.beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName).getSyncPoller(); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1611,17 +1258,16 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { - return this - .beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { + return this.beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, context) .getSyncPoller(); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1632,17 +1278,16 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { - return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName) - .last() + private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { + return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1654,17 +1299,16 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { - return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context) { + return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1681,7 +1325,7 @@ public void delete(String resourceGroupName, String catalogName, String productN /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1692,15 +1336,15 @@ public void delete(String resourceGroupName, String catalogName, String productN * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + Context context) { deleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, context).block(); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1712,23 +1356,15 @@ public void delete( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> claimDevicesWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest) { + private Mono>> claimDevicesWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1752,27 +1388,16 @@ private Mono>> claimDevicesWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .claimDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - claimDevicesRequest, - accept, - context)) + .withContext(context -> service.claimDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1785,24 +1410,15 @@ private Mono>> claimDevicesWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> claimDevicesWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context) { + private Mono>> claimDevicesWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1826,24 +1442,15 @@ private Mono>> claimDevicesWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .claimDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - claimDevicesRequest, - accept, - context); + return service.claimDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest, accept, context); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1855,25 +1462,18 @@ private Mono>> claimDevicesWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginClaimDevicesAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest) { - Mono>> mono = - claimDevicesWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginClaimDevicesAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest) { + Mono>> mono = claimDevicesWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, claimDevicesRequest); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1886,26 +1486,19 @@ private PollerFlux, Void> beginClaimDevicesAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginClaimDevicesAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context) { + private PollerFlux, Void> beginClaimDevicesAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - claimDevicesWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono = claimDevicesWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, claimDevicesRequest, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1917,12 +1510,8 @@ private PollerFlux, Void> beginClaimDevicesAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginClaimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest) { + public SyncPoller, Void> beginClaimDevices(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest) { return this .beginClaimDevicesAsync(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest) .getSyncPoller(); @@ -1931,7 +1520,7 @@ public SyncPoller, Void> beginClaimDevices( /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1944,23 +1533,16 @@ public SyncPoller, Void> beginClaimDevices( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginClaimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context) { - return this - .beginClaimDevicesAsync( - resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, context) - .getSyncPoller(); + public SyncPoller, Void> beginClaimDevices(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest, Context context) { + return this.beginClaimDevicesAsync(resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest, context).getSyncPoller(); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1972,21 +1554,16 @@ public SyncPoller, Void> beginClaimDevices( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono claimDevicesAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest) { + private Mono claimDevicesAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, ClaimDevicesRequest claimDevicesRequest) { return beginClaimDevicesAsync(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1999,23 +1576,16 @@ private Mono claimDevicesAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono claimDevicesAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context) { - return beginClaimDevicesAsync( - resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono claimDevicesAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, ClaimDevicesRequest claimDevicesRequest, Context context) { + return beginClaimDevicesAsync(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, + context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2026,11 +1596,7 @@ private Mono claimDevicesAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + public void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest) { claimDevicesAsync(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest).block(); } @@ -2038,7 +1604,7 @@ public void claimDevices( /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2050,13 +1616,8 @@ public void claimDevices( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context) { + public void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + ClaimDevicesRequest claimDevicesRequest, Context context) { claimDevicesAsync(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, context) .block(); } @@ -2064,7 +1625,7 @@ public void claimDevices( /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2073,22 +1634,18 @@ public void claimDevices( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> countDevicesWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private Mono> countDevicesWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2106,26 +1663,16 @@ private Mono> countDevicesWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .countDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context)) + .withContext(context -> service.countDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2135,22 +1682,18 @@ private Mono> countDevicesWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> countDevicesWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + private Mono> countDevicesWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2168,23 +1711,15 @@ private Mono> countDevicesWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .countDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context); + return service.countDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, accept, + context); } /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2195,8 +1730,8 @@ private Mono> countDevicesWithResponseAsync( * @return response to the action call for count devices in a catalog on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono countDevicesAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private Mono countDevicesAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { return countDevicesWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -2204,7 +1739,7 @@ private Mono countDevicesAsync( /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2216,8 +1751,8 @@ private Mono countDevicesAsync( * @return response to the action call for count devices in a catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + public Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { return countDevicesWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, context) .block(); } @@ -2225,7 +1760,7 @@ public Response countDevicesWithResponse( /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2236,22 +1771,23 @@ public Response countDevicesWithResponse( * @return response to the action call for count devices in a catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CountDeviceResponseInner countDevices( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public CountDevicesResponseInner countDevices(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { return countDevicesWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE) .getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByProductNextSinglePageAsync(String nextLink) { @@ -2259,37 +1795,29 @@ private Mono> listByProductNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByProductNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByProductNextSinglePageAsync(String nextLink, Context context) { @@ -2297,23 +1825,13 @@ private Mono> listByProductNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByProductNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByProductNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsImpl.java index 3e768fc39155e..cc065b225694c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceGroupsImpl.java @@ -10,10 +10,10 @@ import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.sphere.fluent.DeviceGroupsClient; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.models.ClaimDevicesRequest; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import com.azure.resourcemanager.sphere.models.DeviceGroup; import com.azure.resourcemanager.sphere.models.DeviceGroups; @@ -24,43 +24,31 @@ public final class DeviceGroupsImpl implements DeviceGroups { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - public DeviceGroupsImpl( - DeviceGroupsClient innerClient, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + public DeviceGroupsImpl(DeviceGroupsClient innerClient, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByProduct(String resourceGroupName, String catalogName, String productName) { - PagedIterable inner = - this.serviceClient().listByProduct(resourceGroupName, catalogName, productName); - return Utils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByProduct(resourceGroupName, catalogName, productName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); } - public PagedIterable listByProduct( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - PagedIterable inner = - this - .serviceClient() - .listByProduct(resourceGroupName, catalogName, productName, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); + public PagedIterable listByProduct(String resourceGroupName, String catalogName, String productName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner = this.serviceClient().listByProduct(resourceGroupName, catalogName, + productName, filter, top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, context); + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, catalogName, + productName, deviceGroupName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DeviceGroupImpl(inner.getValue(), this.manager())); } else { return null; @@ -80,192 +68,139 @@ public void delete(String resourceGroupName, String catalogName, String productN this.serviceClient().delete(resourceGroupName, catalogName, productName, deviceGroupName); } - public void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + Context context) { this.serviceClient().delete(resourceGroupName, catalogName, productName, deviceGroupName, context); } - public void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + public void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest) { - this - .serviceClient() - .claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest); + this.serviceClient().claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest); } - public void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context) { - this - .serviceClient() - .claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, claimDevicesRequest, context); + public void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + ClaimDevicesRequest claimDevicesRequest, Context context) { + this.serviceClient().claimDevices(resourceGroupName, catalogName, productName, deviceGroupName, + claimDevicesRequest, context); } - public Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { - Response inner = - this - .serviceClient() - .countDevicesWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, context); + public Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { + Response inner = this.serviceClient().countDevicesWithResponse(resourceGroupName, + catalogName, productName, deviceGroupName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CountDeviceResponseImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new CountDevicesResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public CountDeviceResponse countDevices( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { - CountDeviceResponseInner inner = - this.serviceClient().countDevices(resourceGroupName, catalogName, productName, deviceGroupName); + public CountDevicesResponse countDevices(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { + CountDevicesResponseInner inner + = this.serviceClient().countDevices(resourceGroupName, catalogName, productName, deviceGroupName); if (inner != null) { - return new CountDeviceResponseImpl(inner, this.manager()); + return new CountDevicesResponseImpl(inner, this.manager()); } else { return null; } } public DeviceGroup getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - return this - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE) + return this.getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } return this.getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } this.delete(resourceGroupName, catalogName, productName, deviceGroupName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } this.delete(resourceGroupName, catalogName, productName, deviceGroupName, context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceImpl.java index a7b8dd47135f1..76751b21d613f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceImpl.java @@ -4,14 +4,15 @@ package com.azure.resourcemanager.sphere.implementation; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; import com.azure.resourcemanager.sphere.models.Device; +import com.azure.resourcemanager.sphere.models.DeviceProperties; import com.azure.resourcemanager.sphere.models.DeviceUpdate; +import com.azure.resourcemanager.sphere.models.DeviceUpdateProperties; import com.azure.resourcemanager.sphere.models.GenerateCapabilityImageRequest; -import com.azure.resourcemanager.sphere.models.ProvisioningState; import com.azure.resourcemanager.sphere.models.SignedCapabilityImageResponse; -import java.time.OffsetDateTime; public final class DeviceImpl implements Device, Device.Definition, Device.Update { private DeviceInner innerObject; @@ -30,32 +31,12 @@ public String type() { return this.innerModel().type(); } - public String deviceId() { - return this.innerModel().deviceId(); + public DeviceProperties properties() { + return this.innerModel().properties(); } - public String chipSku() { - return this.innerModel().chipSku(); - } - - public String lastAvailableOsVersion() { - return this.innerModel().lastAvailableOsVersion(); - } - - public String lastInstalledOsVersion() { - return this.innerModel().lastInstalledOsVersion(); - } - - public OffsetDateTime lastOsUpdateUtc() { - return this.innerModel().lastOsUpdateUtc(); - } - - public OffsetDateTime lastUpdateRequestUtc() { - return this.innerModel().lastUpdateRequestUtc(); - } - - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public String resourceGroupName() { @@ -82,8 +63,8 @@ private com.azure.resourcemanager.sphere.AzureSphereManager manager() { private DeviceUpdate updateProperties; - public DeviceImpl withExistingDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public DeviceImpl withExistingDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { this.resourceGroupName = resourceGroupName; this.catalogName = catalogName; this.productName = productName; @@ -92,34 +73,14 @@ public DeviceImpl withExistingDeviceGroup( } public Device create() { - this.innerObject = - serviceManager - .serviceClient() - .getDevices() - .createOrUpdate( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - this.innerModel(), - Context.NONE); + this.innerObject = serviceManager.serviceClient().getDevices().createOrUpdate(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, this.innerModel(), Context.NONE); return this; } public Device create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDevices() - .createOrUpdate( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - this.innerModel(), - context); + this.innerObject = serviceManager.serviceClient().getDevices().createOrUpdate(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, this.innerModel(), context); return this; } @@ -135,101 +96,60 @@ public DeviceImpl update() { } public Device apply() { - this.innerObject = - serviceManager - .serviceClient() - .getDevices() - .update( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - updateProperties, - Context.NONE); + this.innerObject = serviceManager.serviceClient().getDevices().update(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, updateProperties, Context.NONE); return this; } public Device apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDevices() - .update( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - updateProperties, - context); + this.innerObject = serviceManager.serviceClient().getDevices().update(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, updateProperties, context); return this; } DeviceImpl(DeviceInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.catalogName = Utils.getValueFromIdByName(innerObject.id(), "catalogs"); - this.productName = Utils.getValueFromIdByName(innerObject.id(), "products"); - this.deviceGroupName = Utils.getValueFromIdByName(innerObject.id(), "deviceGroups"); - this.deviceName = Utils.getValueFromIdByName(innerObject.id(), "devices"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.catalogName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "catalogs"); + this.productName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "products"); + this.deviceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "deviceGroups"); + this.deviceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "devices"); } public Device refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getDevices() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDevices() + .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, Context.NONE) + .getValue(); return this; } public Device refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDevices() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDevices() + .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context) + .getValue(); return this; } - public SignedCapabilityImageResponse generateCapabilityImage( - GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { - return serviceManager - .devices() - .generateCapabilityImage( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest); - } - - public SignedCapabilityImageResponse generateCapabilityImage( - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { - return serviceManager - .devices() - .generateCapabilityImage( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - context); - } - - public DeviceImpl withDeviceId(String deviceId) { - this.innerModel().withDeviceId(deviceId); + public SignedCapabilityImageResponse + generateCapabilityImage(GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { + return serviceManager.devices().generateCapabilityImage(resourceGroupName, catalogName, productName, + deviceGroupName, deviceName, generateDeviceCapabilityRequest); + } + + public SignedCapabilityImageResponse + generateCapabilityImage(GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { + return serviceManager.devices().generateCapabilityImage(resourceGroupName, catalogName, productName, + deviceGroupName, deviceName, generateDeviceCapabilityRequest, context); + } + + public DeviceImpl withProperties(DeviceProperties properties) { + this.innerModel().withProperties(properties); return this; } - public DeviceImpl withDeviceGroupId(String deviceGroupId) { - this.updateProperties.withDeviceGroupId(deviceGroupId); + public DeviceImpl withProperties(DeviceUpdateProperties properties) { + this.updateProperties.withProperties(properties); return this; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceInsightImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceInsightImpl.java index 8ad28a3361318..1671950314c7f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceInsightImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DeviceInsightImpl.java @@ -13,8 +13,8 @@ public final class DeviceInsightImpl implements DeviceInsight { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - DeviceInsightImpl( - DeviceInsightInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + DeviceInsightImpl(DeviceInsightInner innerObject, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesClientImpl.java index ab9b3fdad9735..96ee4df6e0624 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesClientImpl.java @@ -43,150 +43,114 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DevicesClient. */ +/** + * An instance of this class provides access to all the operations defined in DevicesClient. + */ public final class DevicesClientImpl implements DevicesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DevicesService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of DevicesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - DevicesClientImpl(AzureSphereManagementClientImpl client) { + DevicesClientImpl(AzureSphereMgmtClientImpl client) { this.service = RestProxy.create(DevicesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientDevices to be used by the proxy service to + * The interface defining all the services for AzureSphereMgmtClientDevices to be used by the proxy service to * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface DevicesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByDeviceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") - @ExpectedResponses({200}) + Mono> listByDeviceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deviceName") String deviceName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") - @ExpectedResponses({200, 201}) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deviceName") String deviceName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deviceName") String deviceName, - @BodyParam("application/json") DeviceInner resource, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") - @ExpectedResponses({200, 202}) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deviceName") String deviceName, @BodyParam("application/json") DeviceInner resource, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deviceName") String deviceName, - @BodyParam("application/json") DeviceUpdate properties, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") - @ExpectedResponses({200, 202, 204}) + Mono>> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deviceName") String deviceName, @BodyParam("application/json") DeviceUpdate properties, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, - @PathParam("deviceName") String deviceName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}/generateCapabilityImage") - @ExpectedResponses({200, 202}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, + @PathParam("deviceName") String deviceName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/deviceGroups/{deviceGroupName}/devices/{deviceName}/generateCapabilityImage") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> generateCapabilityImage( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @PathParam("deviceGroupName") String deviceGroupName, + Mono>> generateCapabilityImage(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @PathParam("deviceGroupName") String deviceGroupName, @PathParam("deviceName") String deviceName, @BodyParam("application/json") GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByDeviceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -195,22 +159,18 @@ Mono> listByDeviceGroupNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDeviceGroupSinglePageAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private Mono> listByDeviceGroupSinglePageAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -228,35 +188,18 @@ private Mono> listByDeviceGroupSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByDeviceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByDeviceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -266,22 +209,18 @@ private Mono> listByDeviceGroupSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDeviceGroupSinglePageAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + private Mono> listByDeviceGroupSinglePageAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -300,31 +239,16 @@ private Mono> listByDeviceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByDeviceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByDeviceGroup(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -335,8 +259,8 @@ private Mono> listByDeviceGroupSinglePageAsync( * @return the response of a Device list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDeviceGroupAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + private PagedFlux listByDeviceGroupAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { return new PagedFlux<>( () -> listByDeviceGroupSinglePageAsync(resourceGroupName, catalogName, productName, deviceGroupName), nextLink -> listByDeviceGroupNextSinglePageAsync(nextLink)); @@ -345,7 +269,7 @@ private PagedFlux listByDeviceGroupAsync( /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -357,18 +281,16 @@ private PagedFlux listByDeviceGroupAsync( * @return the response of a Device list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDeviceGroupAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { - return new PagedFlux<>( - () -> - listByDeviceGroupSinglePageAsync(resourceGroupName, catalogName, productName, deviceGroupName, context), - nextLink -> listByDeviceGroupNextSinglePageAsync(nextLink, context)); + private PagedFlux listByDeviceGroupAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { + return new PagedFlux<>(() -> listByDeviceGroupSinglePageAsync(resourceGroupName, catalogName, productName, + deviceGroupName, context), nextLink -> listByDeviceGroupNextSinglePageAsync(nextLink, context)); } /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -379,8 +301,8 @@ private PagedFlux listByDeviceGroupAsync( * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, + String productName, String deviceGroupName) { return new PagedIterable<>( listByDeviceGroupAsync(resourceGroupName, catalogName, productName, deviceGroupName)); } @@ -388,7 +310,7 @@ public PagedIterable listByDeviceGroup( /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -400,8 +322,8 @@ public PagedIterable listByDeviceGroup( * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context) { return new PagedIterable<>( listByDeviceGroupAsync(resourceGroupName, catalogName, productName, deviceGroupName, context)); } @@ -409,7 +331,7 @@ public PagedIterable listByDeviceGroup( /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -421,19 +343,15 @@ public PagedIterable listByDeviceGroup( * @return a Device along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -454,27 +372,16 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -487,24 +394,15 @@ private Mono> getWithResponseAsync( * @return a Device along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -525,24 +423,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, deviceName, accept, context); } /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -554,8 +442,8 @@ private Mono> getWithResponseAsync( * @return a Device on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { + private Mono getAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName) { return getWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -563,7 +451,7 @@ private Mono getAsync( /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -576,13 +464,8 @@ private Mono getAsync( * @return a Device along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, Context context) { return getWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context) .block(); } @@ -590,7 +473,7 @@ public Response getWithResponse( /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -602,8 +485,8 @@ public Response getWithResponse( * @return a Device. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceInner get( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { + public DeviceInner get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName) { return getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, Context.NONE) .getValue(); } @@ -611,7 +494,7 @@ public DeviceInner get( /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -622,27 +505,18 @@ public DeviceInner get( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device resource belonging to a device group resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -668,28 +542,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, resource, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -701,28 +563,19 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device resource belonging to a device group resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -748,25 +601,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + resource, accept, context); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -779,26 +622,18 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeviceInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeviceInner.class, DeviceInner.class, this.client.getContext()); + private PollerFlux, DeviceInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceInner resource) { + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceInner.class, DeviceInner.class, this.client.getContext()); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -812,28 +647,20 @@ private PollerFlux, DeviceInner> beginCreateOrUpdateAsyn * @return the {@link PollerFlux} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeviceInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, + private PollerFlux, DeviceInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeviceInner.class, DeviceInner.class, context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceInner.class, DeviceInner.class, context); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -846,23 +673,16 @@ private PollerFlux, DeviceInner> beginCreateOrUpdateAsyn * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource) - .getSyncPoller(); + public SyncPoller, DeviceInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceInner resource) { + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + resource).getSyncPoller(); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -876,24 +696,17 @@ public SyncPoller, DeviceInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceInner> beginCreateOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, + public SyncPoller, DeviceInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceInner resource, Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource, context) - .getSyncPoller(); + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + resource, context).getSyncPoller(); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -906,23 +719,16 @@ public SyncPoller, DeviceInner> beginCreateOrUpdate( * @return an device resource belonging to a device group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource) { - return beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, DeviceInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + resource).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -936,24 +742,16 @@ private Mono createOrUpdateAsync( * @return an device resource belonging to a device group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, DeviceInner resource, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + resource, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -966,13 +764,8 @@ private Mono createOrUpdateAsync( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource) { + public DeviceInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, DeviceInner resource) { return createOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource) .block(); } @@ -980,7 +773,7 @@ public DeviceInner createOrUpdate( /** * Create a Device. Use '.unassigned' or '.default' for the device group and product names to claim a device to the * catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -994,23 +787,16 @@ public DeviceInner createOrUpdate( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceInner createOrUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceInner resource, - Context context) { - return createOrUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource, context) - .block(); + public DeviceInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, DeviceInner resource, Context context) { + return createOrUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, resource, + context).block(); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1021,27 +807,18 @@ public DeviceInner createOrUpdate( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device resource belonging to a device group resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceUpdate properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1067,28 +844,16 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - properties, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, properties, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1100,28 +865,18 @@ private Mono>> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an device resource belonging to a device group resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, - Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceUpdate properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1147,25 +902,14 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - properties, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, accept, context); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1178,26 +922,18 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeviceInner> beginUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties) { - Mono>> mono = - updateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeviceInner.class, DeviceInner.class, this.client.getContext()); + private PollerFlux, DeviceInner> beginUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceUpdate properties) { + Mono>> mono = updateWithResponseAsync(resourceGroupName, catalogName, productName, + deviceGroupName, deviceName, properties); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceInner.class, DeviceInner.class, this.client.getContext()); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1211,28 +947,20 @@ private PollerFlux, DeviceInner> beginUpdateAsync( * @return the {@link PollerFlux} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DeviceInner> beginUpdateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, + private PollerFlux, DeviceInner> beginUpdateAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, DeviceUpdate properties, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DeviceInner.class, DeviceInner.class, context); + Mono>> mono = updateWithResponseAsync(resourceGroupName, catalogName, productName, + deviceGroupName, deviceName, properties, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DeviceInner.class, DeviceInner.class, context); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1245,13 +973,8 @@ private PollerFlux, DeviceInner> beginUpdateAsync( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties) { + public SyncPoller, DeviceInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceUpdate properties) { return this .beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties) .getSyncPoller(); @@ -1260,7 +983,7 @@ public SyncPoller, DeviceInner> beginUpdate( /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1274,24 +997,16 @@ public SyncPoller, DeviceInner> beginUpdate( * @return the {@link SyncPoller} for polling of an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DeviceInner> beginUpdate( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, - Context context) { - return this - .beginUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, context) - .getSyncPoller(); + public SyncPoller, DeviceInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, DeviceUpdate properties, Context context) { + return this.beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + properties, context).getSyncPoller(); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1304,22 +1019,16 @@ public SyncPoller, DeviceInner> beginUpdate( * @return an device resource belonging to a device group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties) { + private Mono updateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, DeviceUpdate properties) { return beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1333,24 +1042,16 @@ private Mono updateAsync( * @return an device resource belonging to a device group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, - Context context) { - return beginUpdateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono updateAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, DeviceUpdate properties, Context context) { + return beginUpdateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, + context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1363,13 +1064,8 @@ private Mono updateAsync( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties) { + public DeviceInner update(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, DeviceUpdate properties) { return updateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties) .block(); } @@ -1377,7 +1073,7 @@ public DeviceInner update( /** * Update a Device. Use '.unassigned' or '.default' for the device group and product names to move a device to the * catalog level. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1391,22 +1087,15 @@ public DeviceInner update( * @return an device resource belonging to a device group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceInner update( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - DeviceUpdate properties, - Context context) { - return updateAsync( - resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, context) - .block(); + public DeviceInner update(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, DeviceUpdate properties, Context context) { + return updateAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, properties, + context).block(); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1418,19 +1107,15 @@ public DeviceInner update( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1451,26 +1136,15 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1483,24 +1157,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1521,23 +1186,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, deviceGroupName, deviceName, accept, context); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1549,19 +1204,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1574,24 +1227,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono = deleteWithResponseAsync(resourceGroupName, catalogName, productName, + deviceGroupName, deviceName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1603,16 +1250,15 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { - return this - .beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName) + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName) { + return this.beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName) .getSyncPoller(); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1625,21 +1271,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { - return this - .beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, Context context) { + return this.beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context) .getSyncPoller(); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1651,16 +1291,15 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { - return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName) - .last() + private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName) { + return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1673,21 +1312,15 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, Context context) { return beginDeleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1698,14 +1331,14 @@ private Mono deleteAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName) { deleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName).block(); } /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1717,20 +1350,15 @@ public void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, Context context) { deleteAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context).block(); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1740,28 +1368,20 @@ public void delete( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return signed device capability image response along with {@link Response} on successful completion of {@link - * Mono}. + * @return signed device capability image response along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> generateCapabilityImageWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, + private Mono>> generateCapabilityImageWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1781,37 +1401,23 @@ private Mono>> generateCapabilityImageWithResponseAsyn return Mono.error(new IllegalArgumentException("Parameter deviceName is required and cannot be null.")); } if (generateDeviceCapabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter generateDeviceCapabilityRequest is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter generateDeviceCapabilityRequest is required and cannot be null.")); } else { generateDeviceCapabilityRequest.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .generateCapabilityImage( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - accept, - context)) + .withContext(context -> service.generateCapabilityImage(this.client.getEndpoint(), + this.client.getApiVersion(), this.client.getSubscriptionId(), resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, generateDeviceCapabilityRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1822,29 +1428,20 @@ private Mono>> generateCapabilityImageWithResponseAsyn * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return signed device capability image response along with {@link Response} on successful completion of {@link - * Mono}. + * @return signed device capability image response along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> generateCapabilityImageWithResponseAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - Context context) { + private Mono>> generateCapabilityImageWithResponseAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, + GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1864,34 +1461,22 @@ private Mono>> generateCapabilityImageWithResponseAsyn return Mono.error(new IllegalArgumentException("Parameter deviceName is required and cannot be null.")); } if (generateDeviceCapabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter generateDeviceCapabilityRequest is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter generateDeviceCapabilityRequest is required and cannot be null.")); } else { generateDeviceCapabilityRequest.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .generateCapabilityImage( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - accept, - context); + return service.generateCapabilityImage(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + generateDeviceCapabilityRequest, accept, context); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1905,35 +1490,19 @@ private Mono>> generateCapabilityImageWithResponseAsyn */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, SignedCapabilityImageResponseInner> - beginGenerateCapabilityImageAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { - Mono>> mono = - generateCapabilityImageWithResponseAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - SignedCapabilityImageResponseInner.class, - SignedCapabilityImageResponseInner.class, - this.client.getContext()); + beginGenerateCapabilityImageAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { + Mono>> mono = generateCapabilityImageWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, generateDeviceCapabilityRequest); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), SignedCapabilityImageResponseInner.class, + SignedCapabilityImageResponseInner.class, this.client.getContext()); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1948,38 +1517,21 @@ private Mono>> generateCapabilityImageWithResponseAsyn */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, SignedCapabilityImageResponseInner> - beginGenerateCapabilityImageAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, + beginGenerateCapabilityImageAsync(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - generateCapabilityImageWithResponseAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - SignedCapabilityImageResponseInner.class, - SignedCapabilityImageResponseInner.class, - context); + Mono>> mono = generateCapabilityImageWithResponseAsync(resourceGroupName, catalogName, + productName, deviceGroupName, deviceName, generateDeviceCapabilityRequest, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), SignedCapabilityImageResponseInner.class, + SignedCapabilityImageResponseInner.class, context); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1993,28 +1545,16 @@ private Mono>> generateCapabilityImageWithResponseAsyn */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, SignedCapabilityImageResponseInner> - beginGenerateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { - return this - .beginGenerateCapabilityImageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest) - .getSyncPoller(); + beginGenerateCapabilityImage(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { + return this.beginGenerateCapabilityImageAsync(resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, generateDeviceCapabilityRequest).getSyncPoller(); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2029,30 +1569,17 @@ private Mono>> generateCapabilityImageWithResponseAsyn */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, SignedCapabilityImageResponseInner> - beginGenerateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, + beginGenerateCapabilityImage(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { - return this - .beginGenerateCapabilityImageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - context) - .getSyncPoller(); + return this.beginGenerateCapabilityImageAsync(resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, generateDeviceCapabilityRequest, context).getSyncPoller(); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2065,28 +1592,17 @@ private Mono>> generateCapabilityImageWithResponseAsyn * @return signed device capability image response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateCapabilityImageAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, + private Mono generateCapabilityImageAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { - return beginGenerateCapabilityImageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest) - .last() - .flatMap(this.client::getLroFinalResultOrError); + return beginGenerateCapabilityImageAsync(resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, generateDeviceCapabilityRequest).last().flatMap(this.client::getLroFinalResultOrError); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2100,30 +1616,17 @@ private Mono generateCapabilityImageAsync( * @return signed device capability image response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateCapabilityImageAsync( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - Context context) { - return beginGenerateCapabilityImageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono generateCapabilityImageAsync(String resourceGroupName, + String catalogName, String productName, String deviceGroupName, String deviceName, + GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { + return beginGenerateCapabilityImageAsync(resourceGroupName, catalogName, productName, deviceGroupName, + deviceName, generateDeviceCapabilityRequest, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2136,27 +1639,17 @@ private Mono generateCapabilityImageAsync( * @return signed device capability image response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SignedCapabilityImageResponseInner generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, + public SignedCapabilityImageResponseInner generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { - return generateCapabilityImageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest) - .block(); + return generateCapabilityImageAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + generateDeviceCapabilityRequest).block(); } /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -2170,35 +1663,24 @@ public SignedCapabilityImageResponseInner generateCapabilityImage( * @return signed device capability image response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SignedCapabilityImageResponseInner generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - Context context) { - return generateCapabilityImageAsync( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - context) - .block(); + public SignedCapabilityImageResponseInner generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, + GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { + return generateCapabilityImageAsync(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, + generateDeviceCapabilityRequest, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByDeviceGroupNextSinglePageAsync(String nextLink) { @@ -2206,37 +1688,29 @@ private Mono> listByDeviceGroupNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByDeviceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByDeviceGroupNextSinglePageAsync(String nextLink, Context context) { @@ -2244,23 +1718,13 @@ private Mono> listByDeviceGroupNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByDeviceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByDeviceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesImpl.java index 2da011f018f7c..9c9fd2a75d7e2 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/DevicesImpl.java @@ -29,48 +29,36 @@ public DevicesImpl(DevicesClient innerClient, com.azure.resourcemanager.sphere.A this.serviceManager = serviceManager; } - public PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName) { - PagedIterable inner = - this.serviceClient().listByDeviceGroup(resourceGroupName, catalogName, productName, deviceGroupName); - return Utils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName) { + PagedIterable inner + = this.serviceClient().listByDeviceGroup(resourceGroupName, catalogName, productName, deviceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); } - public PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context) { - PagedIterable inner = - this - .serviceClient() - .listByDeviceGroup(resourceGroupName, catalogName, productName, deviceGroupName, context); - return Utils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); + public PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context) { + PagedIterable inner = this.serviceClient().listByDeviceGroup(resourceGroupName, catalogName, + productName, deviceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context); + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, catalogName, productName, + deviceGroupName, deviceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DeviceImpl(inner.getValue(), this.manager())); } else { return null; } } - public Device get( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { - DeviceInner inner = - this.serviceClient().get(resourceGroupName, catalogName, productName, deviceGroupName, deviceName); + public Device get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName) { + DeviceInner inner + = this.serviceClient().get(resourceGroupName, catalogName, productName, deviceGroupName, deviceName); if (inner != null) { return new DeviceImpl(inner, this.manager()); } else { @@ -78,38 +66,21 @@ public Device get( } } - public void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName) { this.serviceClient().delete(resourceGroupName, catalogName, productName, deviceGroupName, deviceName); } - public void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context) { + public void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, Context context) { this.serviceClient().delete(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context); } - public SignedCapabilityImageResponse generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, + public SignedCapabilityImageResponse generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest) { - SignedCapabilityImageResponseInner inner = - this - .serviceClient() - .generateCapabilityImage( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest); + SignedCapabilityImageResponseInner inner = this.serviceClient().generateCapabilityImage(resourceGroupName, + catalogName, productName, deviceGroupName, deviceName, generateDeviceCapabilityRequest); if (inner != null) { return new SignedCapabilityImageResponseImpl(inner, this.manager()); } else { @@ -117,25 +88,11 @@ public SignedCapabilityImageResponse generateCapabilityImage( } } - public SignedCapabilityImageResponse generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - Context context) { - SignedCapabilityImageResponseInner inner = - this - .serviceClient() - .generateCapabilityImage( - resourceGroupName, - catalogName, - productName, - deviceGroupName, - deviceName, - generateDeviceCapabilityRequest, - context); + public SignedCapabilityImageResponse generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, + GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context) { + SignedCapabilityImageResponseInner inner = this.serviceClient().generateCapabilityImage(resourceGroupName, + catalogName, productName, deviceGroupName, deviceName, generateDeviceCapabilityRequest, context); if (inner != null) { return new SignedCapabilityImageResponseImpl(inner, this.manager()); } else { @@ -144,41 +101,30 @@ public SignedCapabilityImageResponse generateCapabilityImage( } public Device getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deviceName = Utils.getValueFromIdByName(id, "devices"); + String deviceName = ResourceManagerUtils.getValueFromIdByName(id, "devices"); if (deviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); } return this .getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, Context.NONE) @@ -186,121 +132,88 @@ public Device getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deviceName = Utils.getValueFromIdByName(id, "devices"); + String deviceName = ResourceManagerUtils.getValueFromIdByName(id, "devices"); if (deviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); } return this.getWithResponse(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deviceName = Utils.getValueFromIdByName(id, "devices"); + String deviceName = ResourceManagerUtils.getValueFromIdByName(id, "devices"); if (deviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); } this.delete(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } - String deviceGroupName = Utils.getValueFromIdByName(id, "deviceGroups"); + String deviceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "deviceGroups"); if (deviceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceGroups'.", id))); } - String deviceName = Utils.getValueFromIdByName(id, "devices"); + String deviceName = ResourceManagerUtils.getValueFromIdByName(id, "devices"); if (deviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'devices'.", id))); } this.delete(resourceGroupName, catalogName, productName, deviceGroupName, deviceName, context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImageImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImageImpl.java index a1d69b14dd2ab..07aad183addee 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImageImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImageImpl.java @@ -4,12 +4,11 @@ package com.azure.resourcemanager.sphere.implementation; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.Image; -import com.azure.resourcemanager.sphere.models.ImageType; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; +import com.azure.resourcemanager.sphere.models.ImageProperties; public final class ImageImpl implements Image, Image.Definition, Image.Update { private ImageInner innerObject; @@ -28,40 +27,12 @@ public String type() { return this.innerModel().type(); } - public String image() { - return this.innerModel().image(); + public ImageProperties properties() { + return this.innerModel().properties(); } - public String imageId() { - return this.innerModel().imageId(); - } - - public String imageName() { - return this.innerModel().imageName(); - } - - public RegionalDataBoundary regionalDataBoundary() { - return this.innerModel().regionalDataBoundary(); - } - - public String uri() { - return this.innerModel().uri(); - } - - public String description() { - return this.innerModel().description(); - } - - public String componentId() { - return this.innerModel().componentId(); - } - - public ImageType imageType() { - return this.innerModel().imageType(); - } - - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public String resourceGroupName() { @@ -89,20 +60,14 @@ public ImageImpl withExistingCatalog(String resourceGroupName, String catalogNam } public Image create() { - this.innerObject = - serviceManager - .serviceClient() - .getImages() - .createOrUpdate(resourceGroupName, catalogName, imageName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getImages().createOrUpdate(resourceGroupName, catalogName, + imageName, this.innerModel(), Context.NONE); return this; } public Image create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getImages() - .createOrUpdate(resourceGroupName, catalogName, imageName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getImages().createOrUpdate(resourceGroupName, catalogName, + imageName, this.innerModel(), context); return this; } @@ -117,63 +82,39 @@ public ImageImpl update() { } public Image apply() { - this.innerObject = - serviceManager - .serviceClient() - .getImages() - .createOrUpdate(resourceGroupName, catalogName, imageName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getImages().createOrUpdate(resourceGroupName, catalogName, + imageName, this.innerModel(), Context.NONE); return this; } public Image apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getImages() - .createOrUpdate(resourceGroupName, catalogName, imageName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getImages().createOrUpdate(resourceGroupName, catalogName, + imageName, this.innerModel(), context); return this; } ImageImpl(ImageInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.catalogName = Utils.getValueFromIdByName(innerObject.id(), "catalogs"); - this.imageName = Utils.getValueFromIdByName(innerObject.id(), "images"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.catalogName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "catalogs"); + this.imageName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "images"); } public Image refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getImages() - .getWithResponse(resourceGroupName, catalogName, imageName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getImages() + .getWithResponse(resourceGroupName, catalogName, imageName, Context.NONE).getValue(); return this; } public Image refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getImages() - .getWithResponse(resourceGroupName, catalogName, imageName, context) - .getValue(); - return this; - } - - public ImageImpl withImage(String image) { - this.innerModel().withImage(image); - return this; - } - - public ImageImpl withImageId(String imageId) { - this.innerModel().withImageId(imageId); + this.innerObject = serviceManager.serviceClient().getImages() + .getWithResponse(resourceGroupName, catalogName, imageName, context).getValue(); return this; } - public ImageImpl withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary) { - this.innerModel().withRegionalDataBoundary(regionalDataBoundary); + public ImageImpl withProperties(ImageProperties properties) { + this.innerModel().withProperties(properties); return this; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesClientImpl.java index fa5d3497afd07..c6c1aa74e5aec 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesClientImpl.java @@ -38,109 +38,88 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ImagesClient. */ +/** + * An instance of this class provides access to all the operations defined in ImagesClient. + */ public final class ImagesClientImpl implements ImagesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ImagesService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of ImagesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - ImagesClientImpl(AzureSphereManagementClientImpl client) { + ImagesClientImpl(AzureSphereMgmtClientImpl client) { this.service = RestProxy.create(ImagesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientImages to be used by the proxy service to + * The interface defining all the services for AzureSphereMgmtClientImages to be used by the proxy service to * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface ImagesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCatalog( - @HostParam("$host") String endpoint, - @QueryParam("$filter") String filter, - @QueryParam("$top") Integer top, - @QueryParam("$skip") Integer skip, - @QueryParam("$maxpagesize") Integer maxpagesize, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images/{imageName}") - @ExpectedResponses({200}) + Mono> listByCatalog(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @QueryParam("$top") Integer top, @QueryParam("$skip") Integer skip, + @QueryParam("$maxpagesize") Integer maxpagesize, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images/{imageName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("imageName") String imageName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images/{imageName}") - @ExpectedResponses({200, 201}) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("imageName") String imageName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images/{imageName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("imageName") String imageName, - @BodyParam("application/json") ImageInner resource, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images/{imageName}") - @ExpectedResponses({200, 202, 204}) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("imageName") String imageName, @BodyParam("application/json") ImageInner resource, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/images/{imageName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("imageName") String imageName, - @HeaderParam("Accept") String accept, - Context context); + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("imageName") String imageName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByCatalogNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -151,22 +130,18 @@ Mono> listByCatalogNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Image list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByCatalogSinglePageAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private Mono> listByCatalogSinglePageAsync(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -177,36 +152,17 @@ private Mono> listByCatalogSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCatalog( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByCatalog(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -218,28 +174,18 @@ private Mono> listByCatalogSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Image list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByCatalogSinglePageAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private Mono> listByCatalogSinglePageAsync(String resourceGroupName, String catalogName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -251,32 +197,15 @@ private Mono> listByCatalogSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCatalog( - this.client.getEndpoint(), - filter, - top, - skip, - maxpagesize, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByCatalog(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, filter, top, skip, maxpagesize, catalogName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -289,8 +218,8 @@ private Mono> listByCatalogSinglePageAsync( * @return the response of a Image list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByCatalogAsync( - String resourceGroupName, String catalogName, String filter, Integer top, Integer skip, Integer maxpagesize) { + private PagedFlux listByCatalogAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize) { return new PagedFlux<>( () -> listByCatalogSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize), nextLink -> listByCatalogNextSinglePageAsync(nextLink)); @@ -298,7 +227,7 @@ private PagedFlux listByCatalogAsync( /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -319,7 +248,7 @@ private PagedFlux listByCatalogAsync(String resourceGroupName, Strin /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -333,14 +262,8 @@ private PagedFlux listByCatalogAsync(String resourceGroupName, Strin * @return the response of a Image list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByCatalogAsync( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + private PagedFlux listByCatalogAsync(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedFlux<>( () -> listByCatalogSinglePageAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context), nextLink -> listByCatalogNextSinglePageAsync(nextLink, context)); @@ -348,7 +271,7 @@ private PagedFlux listByCatalogAsync( /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -367,7 +290,7 @@ public PagedIterable listByCatalog(String resourceGroupName, String /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -381,43 +304,33 @@ public PagedIterable listByCatalog(String resourceGroupName, String * @return the response of a Image list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { + public PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context) { return new PagedIterable<>( listByCatalogAsync(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context)); } /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Image along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String imageName) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String imageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -431,27 +344,17 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - imageName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, imageName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -459,19 +362,15 @@ private Mono> getWithResponseAsync( * @return a Image along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String imageName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String imageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -485,24 +384,16 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - imageName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, imageName, accept, context); } /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -516,10 +407,10 @@ private Mono getAsync(String resourceGroupName, String catalogName, /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -527,17 +418,17 @@ private Mono getAsync(String resourceGroupName, String catalogName, * @return a Image along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String catalogName, String imageName, Context context) { + public Response getWithResponse(String resourceGroupName, String catalogName, String imageName, + Context context) { return getWithResponseAsync(resourceGroupName, catalogName, imageName, context).block(); } /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -550,31 +441,27 @@ public ImageInner get(String resourceGroupName, String catalogName, String image /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an image resource belonging to a catalog resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String catalogName, String imageName, ImageInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String imageName, ImageInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -593,50 +480,35 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - imageName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, imageName, resource, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an image resource belonging to a catalog resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String imageName, ImageInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -655,25 +527,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - imageName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, imageName, resource, accept, context); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -681,22 +544,20 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ImageInner> beginCreateOrUpdateAsync( - String resourceGroupName, String catalogName, String imageName, ImageInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, imageName, resource); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ImageInner.class, ImageInner.class, this.client.getContext()); + private PollerFlux, ImageInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String imageName, ImageInner resource) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, imageName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), ImageInner.class, + ImageInner.class, this.client.getContext()); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -705,23 +566,21 @@ private PollerFlux, ImageInner> beginCreateOrUpdateAsync( * @return the {@link PollerFlux} for polling of an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ImageInner> beginCreateOrUpdateAsync( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context) { + private PollerFlux, ImageInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String imageName, ImageInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, imageName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ImageInner.class, ImageInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, imageName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), ImageInner.class, + ImageInner.class, context); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -729,17 +588,17 @@ private PollerFlux, ImageInner> beginCreateOrUpdateAsync( * @return the {@link SyncPoller} for polling of an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ImageInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource) { + public SyncPoller, ImageInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String imageName, ImageInner resource) { return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource).getSyncPoller(); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -748,19 +607,18 @@ public SyncPoller, ImageInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ImageInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource, context) + public SyncPoller, ImageInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String imageName, ImageInner resource, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource, context) .getSyncPoller(); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -768,19 +626,18 @@ public SyncPoller, ImageInner> beginCreateOrUpdate( * @return an image resource belonging to a catalog resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String catalogName, String imageName, ImageInner resource) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String imageName, + ImageInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -789,19 +646,18 @@ private Mono createOrUpdateAsync( * @return an image resource belonging to a catalog resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String imageName, + ImageInner resource, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, imageName, resource, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -809,17 +665,17 @@ private Mono createOrUpdateAsync( * @return an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ImageInner createOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource) { + public ImageInner createOrUpdate(String resourceGroupName, String catalogName, String imageName, + ImageInner resource) { return createOrUpdateAsync(resourceGroupName, catalogName, imageName, resource).block(); } /** * Create a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param resource Resource create parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -828,36 +684,32 @@ public ImageInner createOrUpdate( * @return an image resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ImageInner createOrUpdate( - String resourceGroupName, String catalogName, String imageName, ImageInner resource, Context context) { + public ImageInner createOrUpdate(String resourceGroupName, String catalogName, String imageName, + ImageInner resource, Context context) { return createOrUpdateAsync(resourceGroupName, catalogName, imageName, resource, context).block(); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String imageName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String imageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -871,27 +723,17 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - imageName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, imageName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -899,19 +741,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String imageName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String imageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -925,45 +763,35 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - imageName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, imageName, accept, context); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String imageName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String imageName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, catalogName, imageName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -971,39 +799,38 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String imageName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String imageName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, imageName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, catalogName, imageName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String imageName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String imageName) { return this.beginDeleteAsync(resourceGroupName, catalogName, imageName).getSyncPoller(); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1011,17 +838,17 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String imageName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String imageName, Context context) { return this.beginDeleteAsync(resourceGroupName, catalogName, imageName, context).getSyncPoller(); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1029,17 +856,16 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String catalogName, String imageName) { - return beginDeleteAsync(resourceGroupName, catalogName, imageName) - .last() + return beginDeleteAsync(resourceGroupName, catalogName, imageName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1048,17 +874,16 @@ private Mono deleteAsync(String resourceGroupName, String catalogName, Str */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String catalogName, String imageName, Context context) { - return beginDeleteAsync(resourceGroupName, catalogName, imageName, context) - .last() + return beginDeleteAsync(resourceGroupName, catalogName, imageName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1070,10 +895,10 @@ public void delete(String resourceGroupName, String catalogName, String imageNam /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1086,14 +911,15 @@ public void delete(String resourceGroupName, String catalogName, String imageNam /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Image list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByCatalogNextSinglePageAsync(String nextLink) { @@ -1101,37 +927,29 @@ private Mono> listByCatalogNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Image list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByCatalogNextSinglePageAsync(String nextLink, Context context) { @@ -1139,23 +957,13 @@ private Mono> listByCatalogNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesImpl.java index 827d4f6b672f7..937e88be2d4be 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ImagesImpl.java @@ -28,31 +28,22 @@ public ImagesImpl(ImagesClient innerClient, com.azure.resourcemanager.sphere.Azu public PagedIterable listByCatalog(String resourceGroupName, String catalogName) { PagedIterable inner = this.serviceClient().listByCatalog(resourceGroupName, catalogName); - return Utils.mapPage(inner, inner1 -> new ImageImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ImageImpl(inner1, this.manager())); } - public PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context) { - PagedIterable inner = - this.serviceClient().listByCatalog(resourceGroupName, catalogName, filter, top, skip, maxpagesize, context); - return Utils.mapPage(inner, inner1 -> new ImageImpl(inner1, this.manager())); + public PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context) { + PagedIterable inner = this.serviceClient().listByCatalog(resourceGroupName, catalogName, filter, + top, skip, maxpagesize, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ImageImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String catalogName, String imageName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, catalogName, imageName, context); + public Response getWithResponse(String resourceGroupName, String catalogName, String imageName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, catalogName, imageName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ImageImpl(inner.getValue(), this.manager())); } else { return null; @@ -77,105 +68,77 @@ public void delete(String resourceGroupName, String catalogName, String imageNam } public Image getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String imageName = Utils.getValueFromIdByName(id, "images"); + String imageName = ResourceManagerUtils.getValueFromIdByName(id, "images"); if (imageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); } return this.getWithResponse(resourceGroupName, catalogName, imageName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String imageName = Utils.getValueFromIdByName(id, "images"); + String imageName = ResourceManagerUtils.getValueFromIdByName(id, "images"); if (imageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); } return this.getWithResponse(resourceGroupName, catalogName, imageName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String imageName = Utils.getValueFromIdByName(id, "images"); + String imageName = ResourceManagerUtils.getValueFromIdByName(id, "images"); if (imageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); } this.delete(resourceGroupName, catalogName, imageName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String imageName = Utils.getValueFromIdByName(id, "images"); + String imageName = ResourceManagerUtils.getValueFromIdByName(id, "images"); if (imageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'images'.", id))); } this.delete(resourceGroupName, catalogName, imageName, context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsClientImpl.java index ddee369b0405a..5ddb4fe50d7ca 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsClientImpl.java @@ -30,125 +30,106 @@ import com.azure.resourcemanager.sphere.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - OperationsClientImpl(AzureSphereManagementClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + OperationsClientImpl(AzureSphereMgmtClientImpl client) { + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientOperations to be used by the proxy service - * to perform REST calls. + * The interface defining all the services for AzureSphereMgmtClientOperations to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.AzureSphere/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List the operations for the provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the operations for the provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -157,27 +138,27 @@ private PagedFlux listAsync() { /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List the operations for the provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -186,13 +167,13 @@ public PagedIterable list() { /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -201,14 +182,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -216,37 +198,28 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -254,23 +227,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsImpl.java index 6ae70f0841532..c9356b30c78ae 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/OperationsImpl.java @@ -19,20 +19,20 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + public OperationsImpl(OperationsClient innerClient, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductImpl.java index 26314d6c9732e..ff4bcdf5720fe 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductImpl.java @@ -6,13 +6,15 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import com.azure.resourcemanager.sphere.models.DeviceGroup; import com.azure.resourcemanager.sphere.models.Product; +import com.azure.resourcemanager.sphere.models.ProductProperties; import com.azure.resourcemanager.sphere.models.ProductUpdate; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.resourcemanager.sphere.models.ProductUpdateProperties; public final class ProductImpl implements Product, Product.Definition, Product.Update { private ProductInner innerObject; @@ -31,12 +33,12 @@ public String type() { return this.innerModel().type(); } - public String description() { - return this.innerModel().description(); + public ProductProperties properties() { + return this.innerModel().properties(); } - public ProvisioningState provisioningState() { - return this.innerModel().provisioningState(); + public SystemData systemData() { + return this.innerModel().systemData(); } public String resourceGroupName() { @@ -66,20 +68,14 @@ public ProductImpl withExistingCatalog(String resourceGroupName, String catalogN } public Product create() { - this.innerObject = - serviceManager - .serviceClient() - .getProducts() - .createOrUpdate(resourceGroupName, catalogName, productName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getProducts().createOrUpdate(resourceGroupName, catalogName, + productName, this.innerModel(), Context.NONE); return this; } public Product create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getProducts() - .createOrUpdate(resourceGroupName, catalogName, productName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getProducts().createOrUpdate(resourceGroupName, catalogName, + productName, this.innerModel(), context); return this; } @@ -95,56 +91,42 @@ public ProductImpl update() { } public Product apply() { - this.innerObject = - serviceManager - .serviceClient() - .getProducts() - .update(resourceGroupName, catalogName, productName, updateProperties, Context.NONE); + this.innerObject = serviceManager.serviceClient().getProducts().update(resourceGroupName, catalogName, + productName, updateProperties, Context.NONE); return this; } public Product apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getProducts() - .update(resourceGroupName, catalogName, productName, updateProperties, context); + this.innerObject = serviceManager.serviceClient().getProducts().update(resourceGroupName, catalogName, + productName, updateProperties, context); return this; } ProductImpl(ProductInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.catalogName = Utils.getValueFromIdByName(innerObject.id(), "catalogs"); - this.productName = Utils.getValueFromIdByName(innerObject.id(), "products"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.catalogName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "catalogs"); + this.productName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "products"); } public Product refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getProducts() - .getWithResponse(resourceGroupName, catalogName, productName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getProducts() + .getWithResponse(resourceGroupName, catalogName, productName, Context.NONE).getValue(); return this; } public Product refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getProducts() - .getWithResponse(resourceGroupName, catalogName, productName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getProducts() + .getWithResponse(resourceGroupName, catalogName, productName, context).getValue(); return this; } - public Response countDevicesWithResponse(Context context) { + public Response countDevicesWithResponse(Context context) { return serviceManager.products().countDevicesWithResponse(resourceGroupName, catalogName, productName, context); } - public CountDeviceResponse countDevices() { + public CountDevicesResponse countDevices() { return serviceManager.products().countDevices(resourceGroupName, catalogName, productName); } @@ -153,22 +135,17 @@ public PagedIterable generateDefaultDeviceGroups() { } public PagedIterable generateDefaultDeviceGroups(Context context) { - return serviceManager - .products() - .generateDefaultDeviceGroups(resourceGroupName, catalogName, productName, context); + return serviceManager.products().generateDefaultDeviceGroups(resourceGroupName, catalogName, productName, + context); } - public ProductImpl withDescription(String description) { - if (isInCreateMode()) { - this.innerModel().withDescription(description); - return this; - } else { - this.updateProperties.withDescription(description); - return this; - } + public ProductImpl withProperties(ProductProperties properties) { + this.innerModel().withProperties(properties); + return this; } - private boolean isInCreateMode() { - return this.innerModel().id() == null; + public ProductImpl withProperties(ProductUpdateProperties properties) { + this.updateProperties.withProperties(properties); + return this; } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsClientImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsClientImpl.java index 1a9ea5f50753b..9034a2b7a033c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsClientImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsClientImpl.java @@ -34,7 +34,7 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.sphere.fluent.ProductsClient; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; import com.azure.resourcemanager.sphere.models.DeviceGroupListResult; @@ -44,183 +44,140 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ProductsClient. */ +/** + * An instance of this class provides access to all the operations defined in ProductsClient. + */ public final class ProductsClientImpl implements ProductsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ProductsService service; - /** The service client containing this operation class. */ - private final AzureSphereManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final AzureSphereMgmtClientImpl client; /** * Initializes an instance of ProductsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - ProductsClientImpl(AzureSphereManagementClientImpl client) { + ProductsClientImpl(AzureSphereMgmtClientImpl client) { this.service = RestProxy.create(ProductsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureSphereManagementClientProducts to be used by the proxy service - * to perform REST calls. + * The interface defining all the services for AzureSphereMgmtClientProducts to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "AzureSphereManagemen") + @ServiceInterface(name = "AzureSphereMgmtClien") public interface ProductsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCatalog( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") - @ExpectedResponses({200}) + Mono> listByCatalog(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") - @ExpectedResponses({200, 201}) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @BodyParam("application/json") ProductInner resource, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") - @ExpectedResponses({200, 202}) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @BodyParam("application/json") ProductInner resource, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @BodyParam("application/json") ProductUpdate properties, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") - @ExpectedResponses({200, 202, 204}) + Mono>> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @BodyParam("application/json") ProductUpdate properties, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/countDevices") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/countDevices") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> countDevices( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/generateDefaultDeviceGroups") - @ExpectedResponses({200}) + Mono> countDevices(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.AzureSphere/catalogs/{catalogName}/products/{productName}/generateDefaultDeviceGroups") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateDefaultDeviceGroups( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("catalogName") String catalogName, - @PathParam("productName") String productName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> generateDefaultDeviceGroups(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("catalogName") String catalogName, + @PathParam("productName") String productName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByCatalogNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> generateDefaultDeviceGroupsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Product list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByCatalogSinglePageAsync( - String resourceGroupName, String catalogName) { + private Mono> listByCatalogSinglePageAsync(String resourceGroupName, + String catalogName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -231,32 +188,16 @@ private Mono> listByCatalogSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCatalog( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByCatalog(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -264,22 +205,18 @@ private Mono> listByCatalogSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Product list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByCatalogSinglePageAsync( - String resourceGroupName, String catalogName, Context context) { + private Mono> listByCatalogSinglePageAsync(String resourceGroupName, String catalogName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -291,28 +228,15 @@ private Mono> listByCatalogSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCatalog( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByCatalog(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -322,14 +246,13 @@ private Mono> listByCatalogSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByCatalogAsync(String resourceGroupName, String catalogName) { - return new PagedFlux<>( - () -> listByCatalogSinglePageAsync(resourceGroupName, catalogName), + return new PagedFlux<>(() -> listByCatalogSinglePageAsync(resourceGroupName, catalogName), nextLink -> listByCatalogNextSinglePageAsync(nextLink)); } /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -340,14 +263,13 @@ private PagedFlux listByCatalogAsync(String resourceGroupName, Str */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByCatalogAsync(String resourceGroupName, String catalogName, Context context) { - return new PagedFlux<>( - () -> listByCatalogSinglePageAsync(resourceGroupName, catalogName, context), + return new PagedFlux<>(() -> listByCatalogSinglePageAsync(resourceGroupName, catalogName, context), nextLink -> listByCatalogNextSinglePageAsync(nextLink, context)); } /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -362,7 +284,7 @@ public PagedIterable listByCatalog(String resourceGroupName, Strin /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -378,7 +300,7 @@ public PagedIterable listByCatalog(String resourceGroupName, Strin /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -388,19 +310,15 @@ public PagedIterable listByCatalog(String resourceGroupName, Strin * @return a Product along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String productName) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -414,24 +332,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -442,19 +350,15 @@ private Mono> getWithResponseAsync( * @return a Product along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String catalogName, String productName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String catalogName, + String productName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -468,21 +372,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, accept, context); } /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -499,7 +395,7 @@ private Mono getAsync(String resourceGroupName, String catalogName /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -510,14 +406,14 @@ private Mono getAsync(String resourceGroupName, String catalogName * @return a Product along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String catalogName, String productName, Context context) { + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + Context context) { return getWithResponseAsync(resourceGroupName, catalogName, productName, context).block(); } /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -533,7 +429,7 @@ public ProductInner get(String resourceGroupName, String catalogName, String pro /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -542,22 +438,18 @@ public ProductInner get(String resourceGroupName, String catalogName, String pro * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an product resource belonging to a catalog resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String catalogName, String productName, ProductInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, ProductInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -576,25 +468,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, resource, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -604,22 +486,18 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an product resource belonging to a catalog resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String catalogName, String productName, ProductInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -638,22 +516,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, resource, accept, context); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -664,19 +533,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ProductInner> beginCreateOrUpdateAsync( - String resourceGroupName, String catalogName, String productName, ProductInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, productName, resource); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ProductInner.class, ProductInner.class, this.client.getContext()); + private PollerFlux, ProductInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String productName, ProductInner resource) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, productName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ProductInner.class, ProductInner.class, this.client.getContext()); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -688,20 +555,18 @@ private PollerFlux, ProductInner> beginCreateOrUpdateAs * @return the {@link PollerFlux} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ProductInner> beginCreateOrUpdateAsync( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context) { + private PollerFlux, ProductInner> beginCreateOrUpdateAsync(String resourceGroupName, + String catalogName, String productName, ProductInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, catalogName, productName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ProductInner.class, ProductInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, catalogName, productName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ProductInner.class, ProductInner.class, context); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -712,14 +577,14 @@ private PollerFlux, ProductInner> beginCreateOrUpdateAs * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ProductInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource) { + public SyncPoller, ProductInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, ProductInner resource) { return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource).getSyncPoller(); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -731,16 +596,15 @@ public SyncPoller, ProductInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ProductInner> beginCreateOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource, context) + public SyncPoller, ProductInner> beginCreateOrUpdate(String resourceGroupName, + String catalogName, String productName, ProductInner resource, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource, context) .getSyncPoller(); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -751,16 +615,15 @@ public SyncPoller, ProductInner> beginCreateOrUpdate( * @return an product resource belonging to a catalog resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String catalogName, String productName, ProductInner resource) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + ProductInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -772,16 +635,15 @@ private Mono createOrUpdateAsync( * @return an product resource belonging to a catalog resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String catalogName, String productName, + ProductInner resource, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, catalogName, productName, resource, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -792,14 +654,14 @@ private Mono createOrUpdateAsync( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ProductInner createOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource) { + public ProductInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + ProductInner resource) { return createOrUpdateAsync(resourceGroupName, catalogName, productName, resource).block(); } /** * Create a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -811,14 +673,14 @@ public ProductInner createOrUpdate( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ProductInner createOrUpdate( - String resourceGroupName, String catalogName, String productName, ProductInner resource, Context context) { + public ProductInner createOrUpdate(String resourceGroupName, String catalogName, String productName, + ProductInner resource, Context context) { return createOrUpdateAsync(resourceGroupName, catalogName, productName, resource, context).block(); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -827,22 +689,18 @@ public ProductInner createOrUpdate( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an product resource belonging to a catalog resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String catalogName, + String productName, ProductUpdate properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -860,26 +718,14 @@ private Mono>> updateWithResponseAsync( properties.validate(); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - properties, - accept, - context)) + return FluxUtil.withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, properties, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -889,22 +735,18 @@ private Mono>> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an product resource belonging to a catalog resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String catalogName, + String productName, ProductUpdate properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -923,22 +765,13 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - properties, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, properties, accept, context); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -949,19 +782,17 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ProductInner> beginUpdateAsync( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties) { - Mono>> mono = - updateWithResponseAsync(resourceGroupName, catalogName, productName, properties); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ProductInner.class, ProductInner.class, this.client.getContext()); + private PollerFlux, ProductInner> beginUpdateAsync(String resourceGroupName, + String catalogName, String productName, ProductUpdate properties) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, catalogName, productName, properties); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ProductInner.class, ProductInner.class, this.client.getContext()); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -973,20 +804,18 @@ private PollerFlux, ProductInner> beginUpdateAsync( * @return the {@link PollerFlux} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ProductInner> beginUpdateAsync( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context) { + private PollerFlux, ProductInner> beginUpdateAsync(String resourceGroupName, + String catalogName, String productName, ProductUpdate properties, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, catalogName, productName, properties, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ProductInner.class, ProductInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, catalogName, productName, properties, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ProductInner.class, ProductInner.class, context); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -997,14 +826,14 @@ private PollerFlux, ProductInner> beginUpdateAsync( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ProductInner> beginUpdate( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties) { + public SyncPoller, ProductInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, ProductUpdate properties) { return this.beginUpdateAsync(resourceGroupName, catalogName, productName, properties).getSyncPoller(); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1016,14 +845,14 @@ public SyncPoller, ProductInner> beginUpdate( * @return the {@link SyncPoller} for polling of an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ProductInner> beginUpdate( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context) { + public SyncPoller, ProductInner> beginUpdate(String resourceGroupName, String catalogName, + String productName, ProductUpdate properties, Context context) { return this.beginUpdateAsync(resourceGroupName, catalogName, productName, properties, context).getSyncPoller(); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1034,16 +863,15 @@ public SyncPoller, ProductInner> beginUpdate( * @return an product resource belonging to a catalog resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties) { - return beginUpdateAsync(resourceGroupName, catalogName, productName, properties) - .last() + private Mono updateAsync(String resourceGroupName, String catalogName, String productName, + ProductUpdate properties) { + return beginUpdateAsync(resourceGroupName, catalogName, productName, properties).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1055,16 +883,15 @@ private Mono updateAsync( * @return an product resource belonging to a catalog resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context) { - return beginUpdateAsync(resourceGroupName, catalogName, productName, properties, context) - .last() + private Mono updateAsync(String resourceGroupName, String catalogName, String productName, + ProductUpdate properties, Context context) { + return beginUpdateAsync(resourceGroupName, catalogName, productName, properties, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1075,14 +902,14 @@ private Mono updateAsync( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ProductInner update( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties) { + public ProductInner update(String resourceGroupName, String catalogName, String productName, + ProductUpdate properties) { return updateAsync(resourceGroupName, catalogName, productName, properties).block(); } /** * Update a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1094,14 +921,14 @@ public ProductInner update( * @return an product resource belonging to a catalog resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ProductInner update( - String resourceGroupName, String catalogName, String productName, ProductUpdate properties, Context context) { + public ProductInner update(String resourceGroupName, String catalogName, String productName, + ProductUpdate properties, Context context) { return updateAsync(resourceGroupName, catalogName, productName, properties, context).block(); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1111,19 +938,15 @@ public ProductInner update( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String productName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1137,24 +960,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1165,19 +978,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String catalogName, String productName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String catalogName, + String productName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1191,21 +1000,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, catalogName, productName, accept, context); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1215,18 +1016,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String productName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, catalogName, productName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1237,19 +1036,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String catalogName, String productName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String catalogName, + String productName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, catalogName, productName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, catalogName, productName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1259,14 +1057,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName) { return this.beginDeleteAsync(resourceGroupName, catalogName, productName).getSyncPoller(); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1277,14 +1075,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String catalogName, String productName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String catalogName, + String productName, Context context) { return this.beginDeleteAsync(resourceGroupName, catalogName, productName, context).getSyncPoller(); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1295,14 +1093,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String catalogName, String productName) { - return beginDeleteAsync(resourceGroupName, catalogName, productName) - .last() + return beginDeleteAsync(resourceGroupName, catalogName, productName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1314,14 +1111,13 @@ private Mono deleteAsync(String resourceGroupName, String catalogName, Str */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String catalogName, String productName, Context context) { - return beginDeleteAsync(resourceGroupName, catalogName, productName, context) - .last() + return beginDeleteAsync(resourceGroupName, catalogName, productName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1336,7 +1132,7 @@ public void delete(String resourceGroupName, String catalogName, String productN /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1353,7 +1149,7 @@ public void delete(String resourceGroupName, String catalogName, String productN /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1361,22 +1157,18 @@ public void delete(String resourceGroupName, String catalogName, String productN * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> countDevicesWithResponseAsync( - String resourceGroupName, String catalogName, String productName) { + private Mono> countDevicesWithResponseAsync(String resourceGroupName, + String catalogName, String productName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1390,25 +1182,15 @@ private Mono> countDevicesWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .countDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context)) + .withContext(context -> service.countDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1417,22 +1199,18 @@ private Mono> countDevicesWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> countDevicesWithResponseAsync( - String resourceGroupName, String catalogName, String productName, Context context) { + private Mono> countDevicesWithResponseAsync(String resourceGroupName, + String catalogName, String productName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1446,22 +1224,14 @@ private Mono> countDevicesWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .countDevices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context); + return service.countDevices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, accept, context); } /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1471,8 +1241,8 @@ private Mono> countDevicesWithResponseAsync( * @return response to the action call for count devices in a catalog on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono countDevicesAsync( - String resourceGroupName, String catalogName, String productName) { + private Mono countDevicesAsync(String resourceGroupName, String catalogName, + String productName) { return countDevicesWithResponseAsync(resourceGroupName, catalogName, productName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -1480,7 +1250,7 @@ private Mono countDevicesAsync( /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1491,15 +1261,15 @@ private Mono countDevicesAsync( * @return response to the action call for count devices in a catalog along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, Context context) { + public Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, Context context) { return countDevicesWithResponseAsync(resourceGroupName, catalogName, productName, context).block(); } /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1509,14 +1279,14 @@ public Response countDevicesWithResponse( * @return response to the action call for count devices in a catalog. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CountDeviceResponseInner countDevices(String resourceGroupName, String catalogName, String productName) { + public CountDevicesResponseInner countDevices(String resourceGroupName, String catalogName, String productName) { return countDevicesWithResponse(resourceGroupName, catalogName, productName, Context.NONE).getValue(); } /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1524,22 +1294,18 @@ public CountDeviceResponseInner countDevices(String resourceGroupName, String ca * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateDefaultDeviceGroupsSinglePageAsync( - String resourceGroupName, String catalogName, String productName) { + private Mono> generateDefaultDeviceGroupsSinglePageAsync(String resourceGroupName, + String catalogName, String productName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1554,33 +1320,17 @@ private Mono> generateDefaultDeviceGroupsSingleP final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .generateDefaultDeviceGroups( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.generateDefaultDeviceGroups(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1589,22 +1339,18 @@ private Mono> generateDefaultDeviceGroupsSingleP * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateDefaultDeviceGroupsSinglePageAsync( - String resourceGroupName, String catalogName, String productName, Context context) { + private Mono> generateDefaultDeviceGroupsSinglePageAsync(String resourceGroupName, + String catalogName, String productName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1619,30 +1365,16 @@ private Mono> generateDefaultDeviceGroupsSingleP final String accept = "application/json"; context = this.client.mergeContext(context); return service - .generateDefaultDeviceGroups( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - catalogName, - productName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .generateDefaultDeviceGroups(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, catalogName, productName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1652,8 +1384,8 @@ private Mono> generateDefaultDeviceGroupsSingleP * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux generateDefaultDeviceGroupsAsync( - String resourceGroupName, String catalogName, String productName) { + private PagedFlux generateDefaultDeviceGroupsAsync(String resourceGroupName, String catalogName, + String productName) { return new PagedFlux<>( () -> generateDefaultDeviceGroupsSinglePageAsync(resourceGroupName, catalogName, productName), nextLink -> generateDefaultDeviceGroupsNextSinglePageAsync(nextLink)); @@ -1662,7 +1394,7 @@ private PagedFlux generateDefaultDeviceGroupsAsync( /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1673,8 +1405,8 @@ private PagedFlux generateDefaultDeviceGroupsAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux generateDefaultDeviceGroupsAsync( - String resourceGroupName, String catalogName, String productName, Context context) { + private PagedFlux generateDefaultDeviceGroupsAsync(String resourceGroupName, String catalogName, + String productName, Context context) { return new PagedFlux<>( () -> generateDefaultDeviceGroupsSinglePageAsync(resourceGroupName, catalogName, productName, context), nextLink -> generateDefaultDeviceGroupsNextSinglePageAsync(nextLink, context)); @@ -1683,7 +1415,7 @@ private PagedFlux generateDefaultDeviceGroupsAsync( /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1693,15 +1425,15 @@ private PagedFlux generateDefaultDeviceGroupsAsync( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName) { + public PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName) { return new PagedIterable<>(generateDefaultDeviceGroupsAsync(resourceGroupName, catalogName, productName)); } /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -1712,22 +1444,23 @@ public PagedIterable generateDefaultDeviceGroups( * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName, Context context) { + public PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName, Context context) { return new PagedIterable<>( generateDefaultDeviceGroupsAsync(resourceGroupName, catalogName, productName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Product list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByCatalogNextSinglePageAsync(String nextLink) { @@ -1735,37 +1468,29 @@ private Mono> listByCatalogNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Product list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByCatalogNextSinglePageAsync(String nextLink, Context context) { @@ -1773,36 +1498,27 @@ private Mono> listByCatalogNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByCatalogNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> generateDefaultDeviceGroupsNextSinglePageAsync(String nextLink) { @@ -1810,64 +1526,44 @@ private Mono> generateDefaultDeviceGroupsNextSin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.generateDefaultDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.generateDefaultDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateDefaultDeviceGroupsNextSinglePageAsync( - String nextLink, Context context) { + private Mono> generateDefaultDeviceGroupsNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .generateDefaultDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.generateDefaultDeviceGroupsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsImpl.java index c58e2369b00ca..4e76a66cc0174 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProductsImpl.java @@ -10,10 +10,10 @@ import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.sphere.fluent.ProductsClient; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import com.azure.resourcemanager.sphere.models.DeviceGroup; import com.azure.resourcemanager.sphere.models.Product; import com.azure.resourcemanager.sphere.models.Products; @@ -25,31 +25,28 @@ public final class ProductsImpl implements Products { private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - public ProductsImpl( - ProductsClient innerClient, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { + public ProductsImpl(ProductsClient innerClient, + com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByCatalog(String resourceGroupName, String catalogName) { PagedIterable inner = this.serviceClient().listByCatalog(resourceGroupName, catalogName); - return Utils.mapPage(inner, inner1 -> new ProductImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ProductImpl(inner1, this.manager())); } public PagedIterable listByCatalog(String resourceGroupName, String catalogName, Context context) { PagedIterable inner = this.serviceClient().listByCatalog(resourceGroupName, catalogName, context); - return Utils.mapPage(inner, inner1 -> new ProductImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ProductImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String catalogName, String productName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, catalogName, productName, context); + public Response getWithResponse(String resourceGroupName, String catalogName, String productName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, catalogName, productName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ProductImpl(inner.getValue(), this.manager())); } else { return null; @@ -73,144 +70,114 @@ public void delete(String resourceGroupName, String catalogName, String productN this.serviceClient().delete(resourceGroupName, catalogName, productName, context); } - public Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, Context context) { - Response inner = - this.serviceClient().countDevicesWithResponse(resourceGroupName, catalogName, productName, context); + public Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, Context context) { + Response inner + = this.serviceClient().countDevicesWithResponse(resourceGroupName, catalogName, productName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CountDeviceResponseImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new CountDevicesResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public CountDeviceResponse countDevices(String resourceGroupName, String catalogName, String productName) { - CountDeviceResponseInner inner = this.serviceClient().countDevices(resourceGroupName, catalogName, productName); + public CountDevicesResponse countDevices(String resourceGroupName, String catalogName, String productName) { + CountDevicesResponseInner inner + = this.serviceClient().countDevices(resourceGroupName, catalogName, productName); if (inner != null) { - return new CountDeviceResponseImpl(inner, this.manager()); + return new CountDevicesResponseImpl(inner, this.manager()); } else { return null; } } - public PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName) { - PagedIterable inner = - this.serviceClient().generateDefaultDeviceGroups(resourceGroupName, catalogName, productName); - return Utils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); + public PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName) { + PagedIterable inner + = this.serviceClient().generateDefaultDeviceGroups(resourceGroupName, catalogName, productName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); } - public PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName, Context context) { - PagedIterable inner = - this.serviceClient().generateDefaultDeviceGroups(resourceGroupName, catalogName, productName, context); - return Utils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); + public PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName, Context context) { + PagedIterable inner + = this.serviceClient().generateDefaultDeviceGroups(resourceGroupName, catalogName, productName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceGroupImpl(inner1, this.manager())); } public Product getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } return this.getWithResponse(resourceGroupName, catalogName, productName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } return this.getWithResponse(resourceGroupName, catalogName, productName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } this.delete(resourceGroupName, catalogName, productName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String catalogName = Utils.getValueFromIdByName(id, "catalogs"); + String catalogName = ResourceManagerUtils.getValueFromIdByName(id, "catalogs"); if (catalogName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'catalogs'.", id))); } - String productName = Utils.getValueFromIdByName(id, "products"); + String productName = ResourceManagerUtils.getValueFromIdByName(id, "products"); if (productName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'products'.", id))); } this.delete(resourceGroupName, catalogName, productName, context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProofOfPossessionNonceResponseImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProofOfPossessionNonceResponseImpl.java index ac1feb2ee3e36..431290bc7a552 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProofOfPossessionNonceResponseImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ProofOfPossessionNonceResponseImpl.java @@ -15,8 +15,7 @@ public final class ProofOfPossessionNonceResponseImpl implements ProofOfPossessi private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - ProofOfPossessionNonceResponseImpl( - ProofOfPossessionNonceResponseInner innerObject, + ProofOfPossessionNonceResponseImpl(ProofOfPossessionNonceResponseInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/Utils.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ResourceManagerUtils.java similarity index 79% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/Utils.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ResourceManagerUtils.java index 505f4413ce8df..a0c9f67958a66 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/Utils.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/ResourceManagerUtils.java @@ -19,7 +19,10 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + static String getValueFromIdByName(String id, String name) { if (id == null) { return null; @@ -38,6 +41,7 @@ static String getValueFromIdByName(String id, String name) { } } return null; + } static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { @@ -60,7 +64,7 @@ static String getValueFromIdByParameterName(String id, String pathTemplate, Stri segments.add(idSegment); idItrReverted.forEachRemaining(segments::add); Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { + if (!segments.isEmpty() && segments.get(0).isEmpty()) { segments.remove(0); } return String.join("/", segments); @@ -71,10 +75,11 @@ static String getValueFromIdByParameterName(String id, String pathTemplate, Stri } } return null; + } static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); + return new PagedIterableImpl<>(pageIterable, mapper); } private static final class PagedIterableImpl extends PagedIterable { @@ -84,26 +89,17 @@ private static final class PagedIterableImpl extends PagedIterable { private final Function, PagedResponse> pageMapper; private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); this.pagedIterable = pagedIterable; this.mapper = mapper; this.pageMapper = getPageMapper(mapper); } private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); } @Override @@ -133,30 +129,27 @@ public Stream> streamByPage(String continuationToken, int prefe @Override public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); + return new IteratorImpl<>(pagedIterable.iterator(), mapper); } @Override public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); } @Override public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); } @Override public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); } @Override public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); } } @@ -198,7 +191,7 @@ private IterableImpl(Iterable iterable, Function mapper) { @Override public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); + return new IteratorImpl<>(iterable.iterator(), mapper); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/SignedCapabilityImageResponseImpl.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/SignedCapabilityImageResponseImpl.java index 95d73dc3ec3eb..d5767277af923 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/SignedCapabilityImageResponseImpl.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/SignedCapabilityImageResponseImpl.java @@ -12,8 +12,7 @@ public final class SignedCapabilityImageResponseImpl implements SignedCapability private final com.azure.resourcemanager.sphere.AzureSphereManager serviceManager; - SignedCapabilityImageResponseImpl( - SignedCapabilityImageResponseInner innerObject, + SignedCapabilityImageResponseImpl(SignedCapabilityImageResponseInner innerObject, com.azure.resourcemanager.sphere.AzureSphereManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/package-info.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/package-info.java index 0f24134af586d..878a1be7d183a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/package-info.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the implementations for AzureSphereManagementClient. Azure Sphere resource management API. */ +/** + * Package containing the implementations for AzureSphereMgmtClient. + * Azure Sphere resource management API. + */ package com.azure.resourcemanager.sphere.implementation; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ActionType.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ActionType.java index 16187bc8760ca..8a331ea51dcfc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ActionType.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ActionType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. */ +/** + * Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. + */ public final class ActionType extends ExpandableStringEnum { - /** Static value Internal for ActionType. */ + /** + * Static value Internal for ActionType. + */ public static final ActionType INTERNAL = fromString("Internal"); /** * Creates a new instance of ActionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ActionType() { /** * Creates or finds a ActionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActionType. */ @@ -35,7 +39,7 @@ public static ActionType fromString(String name) { /** * Gets known ActionType values. - * + * * @return known ActionType values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/AllowCrashDumpCollection.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/AllowCrashDumpCollection.java index 421f6c355637a..3fb7732e3328e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/AllowCrashDumpCollection.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/AllowCrashDumpCollection.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Allow crash dumps values. */ +/** + * Allow crash dumps values. + */ public final class AllowCrashDumpCollection extends ExpandableStringEnum { - /** Static value Enabled for AllowCrashDumpCollection. */ + /** + * Static value Enabled for AllowCrashDumpCollection. + */ public static final AllowCrashDumpCollection ENABLED = fromString("Enabled"); - /** Static value Disabled for AllowCrashDumpCollection. */ + /** + * Static value Disabled for AllowCrashDumpCollection. + */ public static final AllowCrashDumpCollection DISABLED = fromString("Disabled"); /** * Creates a new instance of AllowCrashDumpCollection value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AllowCrashDumpCollection() { /** * Creates or finds a AllowCrashDumpCollection from its string representation. - * + * * @param name a name to look for. * @return the corresponding AllowCrashDumpCollection. */ @@ -38,7 +44,7 @@ public static AllowCrashDumpCollection fromString(String name) { /** * Gets known AllowCrashDumpCollection values. - * + * * @return known AllowCrashDumpCollection values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CapabilityType.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CapabilityType.java index 20606302764a2..a3f2ec5709f2e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CapabilityType.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CapabilityType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Capability image type. */ +/** + * Capability image type. + */ public final class CapabilityType extends ExpandableStringEnum { - /** Static value ApplicationDevelopment for CapabilityType. */ + /** + * Static value ApplicationDevelopment for CapabilityType. + */ public static final CapabilityType APPLICATION_DEVELOPMENT = fromString("ApplicationDevelopment"); - /** Static value FieldServicing for CapabilityType. */ + /** + * Static value FieldServicing for CapabilityType. + */ public static final CapabilityType FIELD_SERVICING = fromString("FieldServicing"); /** * Creates a new instance of CapabilityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public CapabilityType() { /** * Creates or finds a CapabilityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding CapabilityType. */ @@ -38,7 +44,7 @@ public static CapabilityType fromString(String name) { /** * Gets known CapabilityType values. - * + * * @return known CapabilityType values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalog.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalog.java index f93fe02c2a042..4cfb60a7b47e8 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalog.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalog.java @@ -10,106 +10,114 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import java.util.Map; -/** An immutable client-side representation of Catalog. */ +/** + * An immutable client-side representation of Catalog. + */ public interface Catalog { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - SystemData systemData(); + CatalogProperties properties(); /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.CatalogInner object. - * + * * @return the inner object. */ CatalogInner innerModel(); - /** The entirety of the Catalog definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the Catalog definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The Catalog definition stages. */ + /** + * The Catalog definition stages. + */ interface DefinitionStages { - /** The first stage of the Catalog definition. */ + /** + * The first stage of the Catalog definition. + */ interface Blank extends WithLocation { } - /** The stage of the Catalog definition allowing to specify location. */ + /** + * The stage of the Catalog definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -117,18 +125,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Catalog definition allowing to specify parent resource. */ + /** + * The stage of the Catalog definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -139,67 +149,88 @@ interface WithResourceGroup { * The stage of the Catalog definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate extends DefinitionStages.WithTags { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ Catalog create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Catalog create(Context context); } - /** The stage of the Catalog definition allowing to specify tags. */ + /** + * The stage of the Catalog definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } + + /** + * The stage of the Catalog definition allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: The resource-specific properties for this resource.. + * + * @param properties The resource-specific properties for this resource. + * @return the next definition stage. + */ + WithCreate withProperties(CatalogProperties properties); + } } /** * Begins update for the Catalog resource. - * + * * @return the stage of resource update. */ Catalog.Update update(); - /** The template for Catalog update. */ + /** + * The template for Catalog update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ Catalog apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Catalog apply(Context context); } - /** The Catalog update stages. */ + /** + * The Catalog update stages. + */ interface UpdateStages { - /** The stage of the Catalog update allowing to specify tags. */ + /** + * The stage of the Catalog update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -209,14 +240,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Catalog refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -224,27 +255,27 @@ interface WithTags { /** * Counts devices in catalog. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response}. */ - Response countDevicesWithResponse(Context context); + Response countDevicesWithResponse(Context context); /** * Counts devices in catalog. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog. */ - CountDeviceResponse countDevices(); + CountDevicesResponse countDevices(); /** * Lists deployments for catalog. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. @@ -253,7 +284,7 @@ interface WithTags { /** * Lists deployments for catalog. - * + * * @param filter Filter the result list using the given expression. * @param top The number of result items to return. * @param skip The number of result items to skip. @@ -264,12 +295,12 @@ interface WithTags { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listDeployments( - String filter, Integer top, Integer skip, Integer maxpagesize, Context context); + PagedIterable listDeployments(String filter, Integer top, Integer skip, Integer maxpagesize, + Context context); /** * List the device groups for the catalog. - * + * * @param listDeviceGroupsRequest List device groups for catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -280,7 +311,7 @@ PagedIterable listDeployments( /** * List the device groups for the catalog. - * + * * @param listDeviceGroupsRequest List device groups for catalog. * @param filter Filter the result list using the given expression. * @param top The number of result items to return. @@ -292,17 +323,12 @@ PagedIterable listDeployments( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listDeviceGroups( - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDeviceGroups(ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context); /** * Lists device insights for catalog. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items as paginated response with {@link PagedIterable}. @@ -311,7 +337,7 @@ PagedIterable listDeviceGroups( /** * Lists device insights for catalog. - * + * * @param filter Filter the result list using the given expression. * @param top The number of result items to return. * @param skip The number of result items to skip. @@ -322,12 +348,12 @@ PagedIterable listDeviceGroups( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items as paginated response with {@link PagedIterable}. */ - PagedIterable listDeviceInsights( - String filter, Integer top, Integer skip, Integer maxpagesize, Context context); + PagedIterable listDeviceInsights(String filter, Integer top, Integer skip, Integer maxpagesize, + Context context); /** * Lists devices for catalog. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation as paginated response with {@link PagedIterable}. @@ -336,7 +362,7 @@ PagedIterable listDeviceInsights( /** * Lists devices for catalog. - * + * * @param filter Filter the result list using the given expression. * @param top The number of result items to return. * @param skip The number of result items to skip. @@ -348,4 +374,25 @@ PagedIterable listDeviceInsights( * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ PagedIterable listDevices(String filter, Integer top, Integer skip, Integer maxpagesize, Context context); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void uploadImage(ImageInner uploadImageRequest); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void uploadImage(ImageInner uploadImageRequest, Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogListResult.java index 9c82d8fefe24f..2af7d44df5d3b 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a Catalog list operation. */ +/** + * The response of a Catalog list operation. + */ @Fluent public final class CatalogListResult { /* @@ -22,16 +24,18 @@ public final class CatalogListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of CatalogListResult class. */ + /** + * Creates an instance of CatalogListResult class. + */ public CatalogListResult() { } /** * Get the value property: The Catalog items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The Catalog items on this page. - * + * * @param value the value value to set. * @return the CatalogListResult object itself. */ @@ -51,34 +55,22 @@ public CatalogListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the CatalogListResult object itself. - */ - public CatalogListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model CatalogListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model CatalogListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogProperties.java similarity index 62% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogProperties.java index e5e5c035f83a0..e176c812f6f3d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CatalogProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogProperties.java @@ -2,28 +2,46 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Immutable; -import com.azure.resourcemanager.sphere.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Catalog properties. */ +/** + * Catalog properties. + */ @Immutable public final class CatalogProperties { + /* + * The Azure Sphere tenant ID associated with the catalog. + */ + @JsonProperty(value = "tenantId", access = JsonProperty.Access.WRITE_ONLY) + private String tenantId; + /* * The status of the last operation. */ @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of CatalogProperties class. */ + /** + * Creates an instance of CatalogProperties class. + */ public CatalogProperties() { } + /** + * Get the tenantId property: The Azure Sphere tenant ID associated with the catalog. + * + * @return the tenantId value. + */ + public String tenantId() { + return this.tenantId; + } + /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -32,7 +50,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogUpdate.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogUpdate.java index a8798215ccb2c..63df52959467a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogUpdate.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CatalogUpdate.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** The type used for update operations of the Catalog. */ +/** + * The type used for update operations of the Catalog. + */ @Fluent public final class CatalogUpdate { /* @@ -19,13 +21,15 @@ public final class CatalogUpdate { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of CatalogUpdate class. */ + /** + * Creates an instance of CatalogUpdate class. + */ public CatalogUpdate() { } /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -34,7 +38,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the CatalogUpdate object itself. */ @@ -45,7 +49,7 @@ public CatalogUpdate withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalogs.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalogs.java index b7919b56ccf95..d399d00ab62d4 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalogs.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Catalogs.java @@ -7,12 +7,15 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; -/** Resource collection API of Catalogs. */ +/** + * Resource collection API of Catalogs. + */ public interface Catalogs { /** * List Catalog resources by subscription ID. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Catalog list operation as paginated response with {@link PagedIterable}. @@ -21,7 +24,7 @@ public interface Catalogs { /** * List Catalog resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,7 +35,7 @@ public interface Catalogs { /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -43,7 +46,7 @@ public interface Catalogs { /** * List Catalog resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -55,7 +58,7 @@ public interface Catalogs { /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -68,7 +71,7 @@ public interface Catalogs { /** * Get a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -80,7 +83,7 @@ public interface Catalogs { /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -91,7 +94,7 @@ public interface Catalogs { /** * Delete a Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -103,7 +106,7 @@ public interface Catalogs { /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -112,12 +115,12 @@ public interface Catalogs { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response}. */ - Response countDevicesWithResponse( - String resourceGroupName, String catalogName, Context context); + Response countDevicesWithResponse(String resourceGroupName, String catalogName, + Context context); /** * Counts devices in catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -125,11 +128,11 @@ Response countDevicesWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog. */ - CountDeviceResponse countDevices(String resourceGroupName, String catalogName); + CountDevicesResponse countDevices(String resourceGroupName, String catalogName); /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -141,7 +144,7 @@ Response countDevicesWithResponse( /** * Lists deployments for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -154,18 +157,12 @@ Response countDevicesWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listDeployments( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDeployments(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context); /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -174,12 +171,12 @@ PagedIterable listDeployments( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listDeviceGroups( - String resourceGroupName, String catalogName, ListDeviceGroupsRequest listDeviceGroupsRequest); + PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest); /** * List the device groups for the catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param listDeviceGroupsRequest List device groups for catalog. @@ -193,19 +190,13 @@ PagedIterable listDeviceGroups( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listDeviceGroups( - String resourceGroupName, - String catalogName, - ListDeviceGroupsRequest listDeviceGroupsRequest, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, + PagedIterable listDeviceGroups(String resourceGroupName, String catalogName, + ListDeviceGroupsRequest listDeviceGroupsRequest, String filter, Integer top, Integer skip, Integer maxpagesize, Context context); /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -217,7 +208,7 @@ PagedIterable listDeviceGroups( /** * Lists device insights for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -230,18 +221,12 @@ PagedIterable listDeviceGroups( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return paged collection of DeviceInsight items as paginated response with {@link PagedIterable}. */ - PagedIterable listDeviceInsights( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDeviceInsights(String resourceGroupName, String catalogName, String filter, + Integer top, Integer skip, Integer maxpagesize, Context context); /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -253,7 +238,7 @@ PagedIterable listDeviceInsights( /** * Lists devices for catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -266,18 +251,37 @@ PagedIterable listDeviceInsights( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listDevices( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listDevices(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest); + + /** + * Creates an image. Use this action when the image ID is unknown. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param catalogName Name of catalog. + * @param uploadImageRequest Image upload request body. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void uploadImage(String resourceGroupName, String catalogName, ImageInner uploadImageRequest, Context context); /** * Get a Catalog. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -288,7 +292,7 @@ PagedIterable listDevices( /** * Get a Catalog. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -300,7 +304,7 @@ PagedIterable listDevices( /** * Delete a Catalog. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -310,7 +314,7 @@ PagedIterable listDevices( /** * Delete a Catalog. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -321,7 +325,7 @@ PagedIterable listDevices( /** * Begins definition for a new Catalog resource. - * + * * @param name resource name. * @return the first stage of the new Catalog definition. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificate.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificate.java index 458d70f643c8b..1aee6fd4d576d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificate.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificate.java @@ -6,90 +6,49 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.sphere.fluent.models.CertificateInner; -import java.time.OffsetDateTime; -/** An immutable client-side representation of Certificate. */ +/** + * An immutable client-side representation of Certificate. + */ public interface Certificate { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - - /** - * Gets the certificate property: The certificate as a UTF-8 encoded base 64 string. - * - * @return the certificate value. - */ - String certificate(); - - /** - * Gets the status property: The certificate status. - * - * @return the status value. - */ - CertificateStatus status(); - - /** - * Gets the subject property: The certificate subject. - * - * @return the subject value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - String subject(); + CertificateProperties properties(); /** - * Gets the thumbprint property: The certificate thumbprint. - * - * @return the thumbprint value. - */ - String thumbprint(); - - /** - * Gets the expiryUtc property: The certificate expiry date. - * - * @return the expiryUtc value. - */ - OffsetDateTime expiryUtc(); - - /** - * Gets the notBeforeUtc property: The certificate not before date. - * - * @return the notBeforeUtc value. - */ - OffsetDateTime notBeforeUtc(); - - /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.CertificateInner object. - * + * * @return the inner object. */ CertificateInner innerModel(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateChainResponse.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateChainResponse.java index 9ecd4c11ccc0f..940f393fa3a8d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateChainResponse.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateChainResponse.java @@ -6,18 +6,20 @@ import com.azure.resourcemanager.sphere.fluent.models.CertificateChainResponseInner; -/** An immutable client-side representation of CertificateChainResponse. */ +/** + * An immutable client-side representation of CertificateChainResponse. + */ public interface CertificateChainResponse { /** * Gets the certificateChain property: The certificate chain. - * + * * @return the certificateChain value. */ String certificateChain(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.CertificateChainResponseInner object. - * + * * @return the inner object. */ CertificateChainResponseInner innerModel(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateListResult.java index 6325d017199a2..b303342e195e9 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a Certificate list operation. */ +/** + * The response of a Certificate list operation. + */ @Fluent public final class CertificateListResult { /* @@ -22,16 +24,18 @@ public final class CertificateListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of CertificateListResult class. */ + /** + * Creates an instance of CertificateListResult class. + */ public CertificateListResult() { } /** * Get the value property: The Certificate items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The Certificate items on this page. - * + * * @param value the value value to set. * @return the CertificateListResult object itself. */ @@ -51,34 +55,22 @@ public CertificateListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the CertificateListResult object itself. - */ - public CertificateListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model CertificateListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model CertificateListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateProperties.java similarity index 90% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateProperties.java index 94db576f28bde..ff0a7b9f59c4e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/CertificateProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateProperties.java @@ -2,15 +2,15 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Immutable; -import com.azure.resourcemanager.sphere.models.CertificateStatus; -import com.azure.resourcemanager.sphere.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** The properties of certificate. */ +/** + * The properties of certificate. + */ @Immutable public class CertificateProperties { /* @@ -55,13 +55,15 @@ public class CertificateProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of CertificateProperties class. */ + /** + * Creates an instance of CertificateProperties class. + */ public CertificateProperties() { } /** * Get the certificate property: The certificate as a UTF-8 encoded base 64 string. - * + * * @return the certificate value. */ public String certificate() { @@ -70,7 +72,7 @@ public String certificate() { /** * Get the status property: The certificate status. - * + * * @return the status value. */ public CertificateStatus status() { @@ -79,7 +81,7 @@ public CertificateStatus status() { /** * Get the subject property: The certificate subject. - * + * * @return the subject value. */ public String subject() { @@ -88,7 +90,7 @@ public String subject() { /** * Get the thumbprint property: The certificate thumbprint. - * + * * @return the thumbprint value. */ public String thumbprint() { @@ -97,7 +99,7 @@ public String thumbprint() { /** * Get the expiryUtc property: The certificate expiry date. - * + * * @return the expiryUtc value. */ public OffsetDateTime expiryUtc() { @@ -106,7 +108,7 @@ public OffsetDateTime expiryUtc() { /** * Get the notBeforeUtc property: The certificate not before date. - * + * * @return the notBeforeUtc value. */ public OffsetDateTime notBeforeUtc() { @@ -115,7 +117,7 @@ public OffsetDateTime notBeforeUtc() { /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -124,7 +126,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateStatus.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateStatus.java index e38e138072baf..0ef99819e1d63 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateStatus.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CertificateStatus.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Certificate status values. */ +/** + * Certificate status values. + */ public final class CertificateStatus extends ExpandableStringEnum { - /** Static value Active for CertificateStatus. */ + /** + * Static value Active for CertificateStatus. + */ public static final CertificateStatus ACTIVE = fromString("Active"); - /** Static value Inactive for CertificateStatus. */ + /** + * Static value Inactive for CertificateStatus. + */ public static final CertificateStatus INACTIVE = fromString("Inactive"); - /** Static value Expired for CertificateStatus. */ + /** + * Static value Expired for CertificateStatus. + */ public static final CertificateStatus EXPIRED = fromString("Expired"); - /** Static value Revoked for CertificateStatus. */ + /** + * Static value Revoked for CertificateStatus. + */ public static final CertificateStatus REVOKED = fromString("Revoked"); /** * Creates a new instance of CertificateStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public CertificateStatus() { /** * Creates or finds a CertificateStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding CertificateStatus. */ @@ -44,7 +54,7 @@ public static CertificateStatus fromString(String name) { /** * Gets known CertificateStatus values. - * + * * @return known CertificateStatus values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificates.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificates.java index 0ed52f203f244..bf130ef1ce0fe 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificates.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Certificates.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Certificates. */ +/** + * Resource collection API of Certificates. + */ public interface Certificates { /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface Certificates { /** * List Certificate resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -37,18 +39,12 @@ public interface Certificates { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Certificate list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context); /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -58,12 +54,12 @@ PagedIterable listByCatalog( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Certificate along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String serialNumber, + Context context); /** * Get a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -76,7 +72,7 @@ Response getWithResponse( /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -86,12 +82,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the certificate chain response along with {@link Response}. */ - Response retrieveCertChainWithResponse( - String resourceGroupName, String catalogName, String serialNumber, Context context); + Response retrieveCertChainWithResponse(String resourceGroupName, String catalogName, + String serialNumber, Context context); /** * Retrieves cert chain. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -104,7 +100,7 @@ Response retrieveCertChainWithResponse( /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -115,16 +111,13 @@ Response retrieveCertChainWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the proof of possession nonce along with {@link Response}. */ - Response retrieveProofOfPossessionNonceWithResponse( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, + Response retrieveProofOfPossessionNonceWithResponse(String resourceGroupName, + String catalogName, String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest, Context context); /** * Gets the proof of possession nonce. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param serialNumber Serial number of the certificate. Use '.default' to get current active certificate. @@ -134,9 +127,6 @@ Response retrieveProofOfPossessionNonceWithRespo * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the proof of possession nonce. */ - ProofOfPossessionNonceResponse retrieveProofOfPossessionNonce( - String resourceGroupName, - String catalogName, - String serialNumber, - ProofOfPossessionNonceRequest proofOfPossessionNonceRequest); + ProofOfPossessionNonceResponse retrieveProofOfPossessionNonce(String resourceGroupName, String catalogName, + String serialNumber, ProofOfPossessionNonceRequest proofOfPossessionNonceRequest); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ClaimDevicesRequest.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ClaimDevicesRequest.java index ae3e8c806059d..12cd6f0eb35df 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ClaimDevicesRequest.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ClaimDevicesRequest.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Request to the action call to bulk claim devices. */ +/** + * Request to the action call to bulk claim devices. + */ @Fluent public final class ClaimDevicesRequest { /* @@ -18,13 +20,15 @@ public final class ClaimDevicesRequest { @JsonProperty(value = "deviceIdentifiers", required = true) private List deviceIdentifiers; - /** Creates an instance of ClaimDevicesRequest class. */ + /** + * Creates an instance of ClaimDevicesRequest class. + */ public ClaimDevicesRequest() { } /** * Get the deviceIdentifiers property: Device identifiers of the devices to be claimed. - * + * * @return the deviceIdentifiers value. */ public List deviceIdentifiers() { @@ -33,7 +37,7 @@ public List deviceIdentifiers() { /** * Set the deviceIdentifiers property: Device identifiers of the devices to be claimed. - * + * * @param deviceIdentifiers the deviceIdentifiers value to set. * @return the ClaimDevicesRequest object itself. */ @@ -44,15 +48,13 @@ public ClaimDevicesRequest withDeviceIdentifiers(List deviceIdentifiers) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (deviceIdentifiers() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property deviceIdentifiers in model ClaimDevicesRequest")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property deviceIdentifiers in model ClaimDevicesRequest")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDeviceResponse.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDeviceResponse.java index 7cadda61b459f..ab6470e8b571b 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDeviceResponse.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDeviceResponse.java @@ -4,21 +4,35 @@ package com.azure.resourcemanager.sphere.models; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.core.annotation.Fluent; -/** An immutable client-side representation of CountDeviceResponse. */ -public interface CountDeviceResponse { +/** + * Response to the action call for count devices in a catalog (preview API). + */ +@Fluent +public final class CountDeviceResponse extends CountElementsResponse { /** - * Gets the value property: Number of children resources in parent resource. - * - * @return the value value. + * Creates an instance of CountDeviceResponse class. */ - int value(); + public CountDeviceResponse() { + } /** - * Gets the inner com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner object. - * - * @return the inner object. + * {@inheritDoc} */ - CountDeviceResponseInner innerModel(); + @Override + public CountDeviceResponse withValue(int value) { + super.withValue(value); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDevicesResponse.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDevicesResponse.java new file mode 100644 index 0000000000000..61ba11c3736ba --- /dev/null +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountDevicesResponse.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.sphere.models; + +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; + +/** + * An immutable client-side representation of CountDevicesResponse. + */ +public interface CountDevicesResponse { + /** + * Gets the value property: Number of children resources in parent resource. + * + * @return the value value. + */ + int value(); + + /** + * Gets the inner com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner object. + * + * @return the inner object. + */ + CountDevicesResponseInner innerModel(); +} diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountElementsResponse.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountElementsResponse.java index 8f716d7182f12..1c4ca678f5fe2 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountElementsResponse.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/CountElementsResponse.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Response of the count for elements. */ +/** + * Response of the count for elements. + */ @Fluent public class CountElementsResponse { /* @@ -16,13 +18,15 @@ public class CountElementsResponse { @JsonProperty(value = "value", required = true) private int value; - /** Creates an instance of CountElementsResponse class. */ + /** + * Creates an instance of CountElementsResponse class. + */ public CountElementsResponse() { } /** * Get the value property: Number of children resources in parent resource. - * + * * @return the value value. */ public int value() { @@ -31,7 +35,7 @@ public int value() { /** * Set the value property: Number of children resources in parent resource. - * + * * @param value the value value to set. * @return the CountElementsResponse object itself. */ @@ -42,7 +46,7 @@ public CountElementsResponse withValue(int value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployment.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployment.java index 52b52dcfd10bc..50a4bb9c4ac16 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployment.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployment.java @@ -4,186 +4,175 @@ package com.azure.resourcemanager.sphere.models; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; -import com.azure.resourcemanager.sphere.fluent.models.ImageInner; -import java.time.OffsetDateTime; -import java.util.List; -/** An immutable client-side representation of Deployment. */ +/** + * An immutable client-side representation of Deployment. + */ public interface Deployment { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the deploymentId property: Deployment ID. - * - * @return the deploymentId value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - String deploymentId(); + DeploymentProperties properties(); /** - * Gets the deployedImages property: Images deployed. - * - * @return the deployedImages value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - List deployedImages(); - - /** - * Gets the deploymentDateUtc property: Deployment date UTC. - * - * @return the deploymentDateUtc value. - */ - OffsetDateTime deploymentDateUtc(); - - /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. - */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.DeploymentInner object. - * + * * @return the inner object. */ DeploymentInner innerModel(); - /** The entirety of the Deployment definition. */ + /** + * The entirety of the Deployment definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Deployment definition stages. */ + /** + * The Deployment definition stages. + */ interface DefinitionStages { - /** The first stage of the Deployment definition. */ + /** + * The first stage of the Deployment definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Deployment definition allowing to specify parent resource. */ + /** + * The stage of the Deployment definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, catalogName, productName, deviceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @return the next definition stage. */ - WithCreate withExistingDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + WithCreate withExistingDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); } /** * The stage of the Deployment definition which contains all the minimum required properties for the resource to * be created, but also allows for any other optional properties to be specified. */ - interface WithCreate extends DefinitionStages.WithDeploymentId, DefinitionStages.WithDeployedImages { + interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ Deployment create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Deployment create(Context context); } - /** The stage of the Deployment definition allowing to specify deploymentId. */ - interface WithDeploymentId { - /** - * Specifies the deploymentId property: Deployment ID. - * - * @param deploymentId Deployment ID. - * @return the next definition stage. - */ - WithCreate withDeploymentId(String deploymentId); - } - - /** The stage of the Deployment definition allowing to specify deployedImages. */ - interface WithDeployedImages { + /** + * The stage of the Deployment definition allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the deployedImages property: Images deployed. - * - * @param deployedImages Images deployed. + * Specifies the properties property: The resource-specific properties for this resource.. + * + * @param properties The resource-specific properties for this resource. * @return the next definition stage. */ - WithCreate withDeployedImages(List deployedImages); + WithCreate withProperties(DeploymentProperties properties); } } /** * Begins update for the Deployment resource. - * + * * @return the stage of resource update. */ Deployment.Update update(); - /** The template for Deployment update. */ + /** + * The template for Deployment update. + */ interface Update { /** * Executes the update request. - * + * * @return the updated resource. */ Deployment apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Deployment apply(Context context); } - /** The Deployment update stages. */ + /** + * The Deployment update stages. + */ interface UpdateStages { } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Deployment refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentListResult.java index 6c7498792bfa5..d35e323cdcf60 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a Deployment list operation. */ +/** + * The response of a Deployment list operation. + */ @Fluent public final class DeploymentListResult { /* @@ -22,16 +24,18 @@ public final class DeploymentListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DeploymentListResult class. */ + /** + * Creates an instance of DeploymentListResult class. + */ public DeploymentListResult() { } /** * Get the value property: The Deployment items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The Deployment items on this page. - * + * * @param value the value value to set. * @return the DeploymentListResult object itself. */ @@ -51,34 +55,22 @@ public DeploymentListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the DeploymentListResult object itself. - */ - public DeploymentListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model DeploymentListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DeploymentListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentProperties.java similarity index 90% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentProperties.java index 426c75a317fd1..301897ce51c8e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeploymentProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeploymentProperties.java @@ -2,15 +2,17 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.models.ProvisioningState; +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; import java.util.List; -/** The properties of deployment. */ +/** + * The properties of deployment. + */ @Fluent public final class DeploymentProperties { /* @@ -37,13 +39,15 @@ public final class DeploymentProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of DeploymentProperties class. */ + /** + * Creates an instance of DeploymentProperties class. + */ public DeploymentProperties() { } /** * Get the deploymentId property: Deployment ID. - * + * * @return the deploymentId value. */ public String deploymentId() { @@ -52,7 +56,7 @@ public String deploymentId() { /** * Set the deploymentId property: Deployment ID. - * + * * @param deploymentId the deploymentId value to set. * @return the DeploymentProperties object itself. */ @@ -63,7 +67,7 @@ public DeploymentProperties withDeploymentId(String deploymentId) { /** * Get the deployedImages property: Images deployed. - * + * * @return the deployedImages value. */ public List deployedImages() { @@ -72,7 +76,7 @@ public List deployedImages() { /** * Set the deployedImages property: Images deployed. - * + * * @param deployedImages the deployedImages value to set. * @return the DeploymentProperties object itself. */ @@ -83,7 +87,7 @@ public DeploymentProperties withDeployedImages(List deployedImages) /** * Get the deploymentDateUtc property: Deployment date UTC. - * + * * @return the deploymentDateUtc value. */ public OffsetDateTime deploymentDateUtc() { @@ -92,7 +96,7 @@ public OffsetDateTime deploymentDateUtc() { /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -101,7 +105,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployments.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployments.java index 082f28f191fe7..71d8fff814d50 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployments.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Deployments.java @@ -8,12 +8,14 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Deployments. */ +/** + * Resource collection API of Deployments. + */ public interface Deployments { /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -23,13 +25,13 @@ public interface Deployments { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); /** * List Deployment resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be * used for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -44,111 +46,85 @@ PagedIterable listByDeviceGroup( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Deployment list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByDeviceGroup( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String filter, Integer top, Integer skip, Integer maxpagesize, Context context); /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Deployment along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deploymentName, Context context); /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Deployment. */ - Deployment get( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + Deployment get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName); /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deploymentName); /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @param deploymentName Deployment name. Use .default for deployment creation and to get the current deployment for - * the associated device group. + * the associated device group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deploymentName, - Context context); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deploymentName, Context context); /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -160,7 +136,7 @@ void delete( /** * Get a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or device * group name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -173,7 +149,7 @@ void delete( /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -184,7 +160,7 @@ void delete( /** * Delete a Deployment. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -195,7 +171,7 @@ void delete( /** * Begins definition for a new Deployment resource. - * + * * @param name resource name. * @return the first stage of the new Deployment definition. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Device.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Device.java index 073290d5c9968..0228d79e48579 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Device.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Device.java @@ -4,204 +4,187 @@ package com.azure.resourcemanager.sphere.models; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; -import java.time.OffsetDateTime; -/** An immutable client-side representation of Device. */ +/** + * An immutable client-side representation of Device. + */ public interface Device { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the deviceId property: Device ID. - * - * @return the deviceId value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - String deviceId(); + DeviceProperties properties(); /** - * Gets the chipSku property: SKU of the chip. - * - * @return the chipSku value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - String chipSku(); - - /** - * Gets the lastAvailableOsVersion property: OS version available for installation when update requested. - * - * @return the lastAvailableOsVersion value. - */ - String lastAvailableOsVersion(); - - /** - * Gets the lastInstalledOsVersion property: OS version running on device when update requested. - * - * @return the lastInstalledOsVersion value. - */ - String lastInstalledOsVersion(); - - /** - * Gets the lastOsUpdateUtc property: Time when update requested and new OS version available. - * - * @return the lastOsUpdateUtc value. - */ - OffsetDateTime lastOsUpdateUtc(); - - /** - * Gets the lastUpdateRequestUtc property: Time when update was last requested. - * - * @return the lastUpdateRequestUtc value. - */ - OffsetDateTime lastUpdateRequestUtc(); - - /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. - */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.DeviceInner object. - * + * * @return the inner object. */ DeviceInner innerModel(); - /** The entirety of the Device definition. */ + /** + * The entirety of the Device definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Device definition stages. */ + /** + * The Device definition stages. + */ interface DefinitionStages { - /** The first stage of the Device definition. */ + /** + * The first stage of the Device definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Device definition allowing to specify parent resource. */ + /** + * The stage of the Device definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, catalogName, productName, deviceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. * @param deviceGroupName Name of device group. * @return the next definition stage. */ - WithCreate withExistingDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + WithCreate withExistingDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); } /** * The stage of the Device definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate extends DefinitionStages.WithDeviceId { + interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ Device create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Device create(Context context); } - /** The stage of the Device definition allowing to specify deviceId. */ - interface WithDeviceId { + /** + * The stage of the Device definition allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the deviceId property: Device ID. - * - * @param deviceId Device ID. + * Specifies the properties property: The resource-specific properties for this resource.. + * + * @param properties The resource-specific properties for this resource. * @return the next definition stage. */ - WithCreate withDeviceId(String deviceId); + WithCreate withProperties(DeviceProperties properties); } } /** * Begins update for the Device resource. - * + * * @return the stage of resource update. */ Device.Update update(); - /** The template for Device update. */ - interface Update extends UpdateStages.WithDeviceGroupId { + /** + * The template for Device update. + */ + interface Update extends UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ Device apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Device apply(Context context); } - /** The Device update stages. */ + /** + * The Device update stages. + */ interface UpdateStages { - /** The stage of the Device update allowing to specify deviceGroupId. */ - interface WithDeviceGroupId { + /** + * The stage of the Device update allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the deviceGroupId property: Device group id. - * - * @param deviceGroupId Device group id. + * Specifies the properties property: The updatable properties of the Device.. + * + * @param properties The updatable properties of the Device. * @return the next definition stage. */ - Update withDeviceGroupId(String deviceGroupId); + Update withProperties(DeviceUpdateProperties properties); } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Device refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -210,20 +193,20 @@ interface WithDeviceGroupId { /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param generateDeviceCapabilityRequest Generate capability image request body. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return signed device capability image response. */ - SignedCapabilityImageResponse generateCapabilityImage( - GenerateCapabilityImageRequest generateDeviceCapabilityRequest); + SignedCapabilityImageResponse + generateCapabilityImage(GenerateCapabilityImageRequest generateDeviceCapabilityRequest); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param generateDeviceCapabilityRequest Generate capability image request body. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -231,6 +214,6 @@ SignedCapabilityImageResponse generateCapabilityImage( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return signed device capability image response. */ - SignedCapabilityImageResponse generateCapabilityImage( - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context); + SignedCapabilityImageResponse + generateCapabilityImage(GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroup.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroup.java index 19be2d25dcec9..b94eba078941a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroup.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroup.java @@ -5,111 +5,87 @@ package com.azure.resourcemanager.sphere.models; import com.azure.core.http.rest.Response; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; -/** An immutable client-side representation of DeviceGroup. */ +/** + * An immutable client-side representation of DeviceGroup. + */ public interface DeviceGroup { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the description property: Description of the device group. - * - * @return the description value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - String description(); + DeviceGroupProperties properties(); /** - * Gets the osFeedType property: Operating system feed type of the device group. - * - * @return the osFeedType value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - OSFeedType osFeedType(); - - /** - * Gets the updatePolicy property: Update policy of the device group. - * - * @return the updatePolicy value. - */ - UpdatePolicy updatePolicy(); - - /** - * Gets the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * - * @return the allowCrashDumpsCollection value. - */ - AllowCrashDumpCollection allowCrashDumpsCollection(); - - /** - * Gets the regionalDataBoundary property: Regional data boundary for the device group. - * - * @return the regionalDataBoundary value. - */ - RegionalDataBoundary regionalDataBoundary(); - - /** - * Gets the hasDeployment property: Deployment status for the device group. - * - * @return the hasDeployment value. - */ - Boolean hasDeployment(); - - /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. - */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner object. - * + * * @return the inner object. */ DeviceGroupInner innerModel(); - /** The entirety of the DeviceGroup definition. */ + /** + * The entirety of the DeviceGroup definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The DeviceGroup definition stages. */ + /** + * The DeviceGroup definition stages. + */ interface DefinitionStages { - /** The first stage of the DeviceGroup definition. */ + /** + * The first stage of the DeviceGroup definition. + */ interface Blank extends WithParentResource { } - /** The stage of the DeviceGroup definition allowing to specify parent resource. */ + /** + * The stage of the DeviceGroup definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, catalogName, productName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -122,184 +98,92 @@ interface WithParentResource { * The stage of the DeviceGroup definition which contains all the minimum required properties for the resource * to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithDescription, - DefinitionStages.WithOsFeedType, - DefinitionStages.WithUpdatePolicy, - DefinitionStages.WithAllowCrashDumpsCollection, - DefinitionStages.WithRegionalDataBoundary { + interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ DeviceGroup create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ DeviceGroup create(Context context); } - /** The stage of the DeviceGroup definition allowing to specify description. */ - interface WithDescription { - /** - * Specifies the description property: Description of the device group.. - * - * @param description Description of the device group. - * @return the next definition stage. - */ - WithCreate withDescription(String description); - } - - /** The stage of the DeviceGroup definition allowing to specify osFeedType. */ - interface WithOsFeedType { - /** - * Specifies the osFeedType property: Operating system feed type of the device group.. - * - * @param osFeedType Operating system feed type of the device group. - * @return the next definition stage. - */ - WithCreate withOsFeedType(OSFeedType osFeedType); - } - - /** The stage of the DeviceGroup definition allowing to specify updatePolicy. */ - interface WithUpdatePolicy { - /** - * Specifies the updatePolicy property: Update policy of the device group.. - * - * @param updatePolicy Update policy of the device group. - * @return the next definition stage. - */ - WithCreate withUpdatePolicy(UpdatePolicy updatePolicy); - } - - /** The stage of the DeviceGroup definition allowing to specify allowCrashDumpsCollection. */ - interface WithAllowCrashDumpsCollection { - /** - * Specifies the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump - * collection.. - * - * @param allowCrashDumpsCollection Flag to define if the user allows for crash dump collection. - * @return the next definition stage. - */ - WithCreate withAllowCrashDumpsCollection(AllowCrashDumpCollection allowCrashDumpsCollection); - } - - /** The stage of the DeviceGroup definition allowing to specify regionalDataBoundary. */ - interface WithRegionalDataBoundary { + /** + * The stage of the DeviceGroup definition allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the regionalDataBoundary property: Regional data boundary for the device group.. - * - * @param regionalDataBoundary Regional data boundary for the device group. + * Specifies the properties property: The resource-specific properties for this resource.. + * + * @param properties The resource-specific properties for this resource. * @return the next definition stage. */ - WithCreate withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary); + WithCreate withProperties(DeviceGroupProperties properties); } } /** * Begins update for the DeviceGroup resource. - * + * * @return the stage of resource update. */ DeviceGroup.Update update(); - /** The template for DeviceGroup update. */ - interface Update - extends UpdateStages.WithDescription, - UpdateStages.WithOsFeedType, - UpdateStages.WithUpdatePolicy, - UpdateStages.WithAllowCrashDumpsCollection, - UpdateStages.WithRegionalDataBoundary { + /** + * The template for DeviceGroup update. + */ + interface Update extends UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ DeviceGroup apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ DeviceGroup apply(Context context); } - /** The DeviceGroup update stages. */ + /** + * The DeviceGroup update stages. + */ interface UpdateStages { - /** The stage of the DeviceGroup update allowing to specify description. */ - interface WithDescription { - /** - * Specifies the description property: Description of the device group.. - * - * @param description Description of the device group. - * @return the next definition stage. - */ - Update withDescription(String description); - } - - /** The stage of the DeviceGroup update allowing to specify osFeedType. */ - interface WithOsFeedType { - /** - * Specifies the osFeedType property: Operating system feed type of the device group.. - * - * @param osFeedType Operating system feed type of the device group. - * @return the next definition stage. - */ - Update withOsFeedType(OSFeedType osFeedType); - } - - /** The stage of the DeviceGroup update allowing to specify updatePolicy. */ - interface WithUpdatePolicy { - /** - * Specifies the updatePolicy property: Update policy of the device group.. - * - * @param updatePolicy Update policy of the device group. - * @return the next definition stage. - */ - Update withUpdatePolicy(UpdatePolicy updatePolicy); - } - - /** The stage of the DeviceGroup update allowing to specify allowCrashDumpsCollection. */ - interface WithAllowCrashDumpsCollection { - /** - * Specifies the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump - * collection.. - * - * @param allowCrashDumpsCollection Flag to define if the user allows for crash dump collection. - * @return the next definition stage. - */ - Update withAllowCrashDumpsCollection(AllowCrashDumpCollection allowCrashDumpsCollection); - } - - /** The stage of the DeviceGroup update allowing to specify regionalDataBoundary. */ - interface WithRegionalDataBoundary { + /** + * The stage of the DeviceGroup update allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the regionalDataBoundary property: Regional data boundary for the device group.. - * - * @param regionalDataBoundary Regional data boundary for the device group. + * Specifies the properties property: The updatable properties of the DeviceGroup.. + * + * @param properties The updatable properties of the DeviceGroup. * @return the next definition stage. */ - Update withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary); + Update withProperties(DeviceGroupUpdateProperties properties); } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ DeviceGroup refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -308,7 +192,7 @@ interface WithRegionalDataBoundary { /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param claimDevicesRequest Bulk claim devices request body. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -319,7 +203,7 @@ interface WithRegionalDataBoundary { /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param claimDevicesRequest Bulk claim devices request body. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -331,22 +215,22 @@ interface WithRegionalDataBoundary { /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response}. */ - Response countDevicesWithResponse(Context context); + Response countDevicesWithResponse(Context context); /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog. */ - CountDeviceResponse countDevices(); + CountDevicesResponse countDevices(); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupListResult.java index 5010b57a9b072..c84c197fd3755 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a DeviceGroup list operation. */ +/** + * The response of a DeviceGroup list operation. + */ @Fluent public final class DeviceGroupListResult { /* @@ -22,16 +24,18 @@ public final class DeviceGroupListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DeviceGroupListResult class. */ + /** + * Creates an instance of DeviceGroupListResult class. + */ public DeviceGroupListResult() { } /** * Get the value property: The DeviceGroup items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The DeviceGroup items on this page. - * + * * @param value the value value to set. * @return the DeviceGroupListResult object itself. */ @@ -51,34 +55,22 @@ public DeviceGroupListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the DeviceGroupListResult object itself. - */ - public DeviceGroupListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model DeviceGroupListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DeviceGroupListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupProperties.java similarity index 90% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupProperties.java index dfab130e9fa89..007e5cbf4de8e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupProperties.java @@ -2,17 +2,14 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; -import com.azure.resourcemanager.sphere.models.OSFeedType; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; -import com.azure.resourcemanager.sphere.models.UpdatePolicy; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of deviceGroup. */ +/** + * The properties of deviceGroup. + */ @Fluent public final class DeviceGroupProperties { /* @@ -57,13 +54,15 @@ public final class DeviceGroupProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of DeviceGroupProperties class. */ + /** + * Creates an instance of DeviceGroupProperties class. + */ public DeviceGroupProperties() { } /** * Get the description property: Description of the device group. - * + * * @return the description value. */ public String description() { @@ -72,7 +71,7 @@ public String description() { /** * Set the description property: Description of the device group. - * + * * @param description the description value to set. * @return the DeviceGroupProperties object itself. */ @@ -83,7 +82,7 @@ public DeviceGroupProperties withDescription(String description) { /** * Get the osFeedType property: Operating system feed type of the device group. - * + * * @return the osFeedType value. */ public OSFeedType osFeedType() { @@ -92,7 +91,7 @@ public OSFeedType osFeedType() { /** * Set the osFeedType property: Operating system feed type of the device group. - * + * * @param osFeedType the osFeedType value to set. * @return the DeviceGroupProperties object itself. */ @@ -103,7 +102,7 @@ public DeviceGroupProperties withOsFeedType(OSFeedType osFeedType) { /** * Get the updatePolicy property: Update policy of the device group. - * + * * @return the updatePolicy value. */ public UpdatePolicy updatePolicy() { @@ -112,7 +111,7 @@ public UpdatePolicy updatePolicy() { /** * Set the updatePolicy property: Update policy of the device group. - * + * * @param updatePolicy the updatePolicy value to set. * @return the DeviceGroupProperties object itself. */ @@ -123,7 +122,7 @@ public DeviceGroupProperties withUpdatePolicy(UpdatePolicy updatePolicy) { /** * Get the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * + * * @return the allowCrashDumpsCollection value. */ public AllowCrashDumpCollection allowCrashDumpsCollection() { @@ -132,7 +131,7 @@ public AllowCrashDumpCollection allowCrashDumpsCollection() { /** * Set the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * + * * @param allowCrashDumpsCollection the allowCrashDumpsCollection value to set. * @return the DeviceGroupProperties object itself. */ @@ -143,7 +142,7 @@ public DeviceGroupProperties withAllowCrashDumpsCollection(AllowCrashDumpCollect /** * Get the regionalDataBoundary property: Regional data boundary for the device group. - * + * * @return the regionalDataBoundary value. */ public RegionalDataBoundary regionalDataBoundary() { @@ -152,7 +151,7 @@ public RegionalDataBoundary regionalDataBoundary() { /** * Set the regionalDataBoundary property: Regional data boundary for the device group. - * + * * @param regionalDataBoundary the regionalDataBoundary value to set. * @return the DeviceGroupProperties object itself. */ @@ -163,7 +162,7 @@ public DeviceGroupProperties withRegionalDataBoundary(RegionalDataBoundary regio /** * Get the hasDeployment property: Deployment status for the device group. - * + * * @return the hasDeployment value. */ public Boolean hasDeployment() { @@ -172,7 +171,7 @@ public Boolean hasDeployment() { /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -181,7 +180,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdate.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdate.java index b29005860436c..6b8a3dedd1714 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdate.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdate.java @@ -5,154 +5,53 @@ package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupUpdateProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** The type used for update operations of the DeviceGroup. */ +/** + * The type used for update operations of the DeviceGroup. + */ @Fluent public final class DeviceGroupUpdate { /* * The updatable properties of the DeviceGroup. */ @JsonProperty(value = "properties") - private DeviceGroupUpdateProperties innerProperties; - - /** Creates an instance of DeviceGroupUpdate class. */ - public DeviceGroupUpdate() { - } - - /** - * Get the innerProperties property: The updatable properties of the DeviceGroup. - * - * @return the innerProperties value. - */ - private DeviceGroupUpdateProperties innerProperties() { - return this.innerProperties; - } + private DeviceGroupUpdateProperties properties; /** - * Get the description property: Description of the device group. - * - * @return the description value. + * Creates an instance of DeviceGroupUpdate class. */ - public String description() { - return this.innerProperties() == null ? null : this.innerProperties().description(); - } - - /** - * Set the description property: Description of the device group. - * - * @param description the description value to set. - * @return the DeviceGroupUpdate object itself. - */ - public DeviceGroupUpdate withDescription(String description) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupUpdateProperties(); - } - this.innerProperties().withDescription(description); - return this; - } - - /** - * Get the osFeedType property: Operating system feed type of the device group. - * - * @return the osFeedType value. - */ - public OSFeedType osFeedType() { - return this.innerProperties() == null ? null : this.innerProperties().osFeedType(); - } - - /** - * Set the osFeedType property: Operating system feed type of the device group. - * - * @param osFeedType the osFeedType value to set. - * @return the DeviceGroupUpdate object itself. - */ - public DeviceGroupUpdate withOsFeedType(OSFeedType osFeedType) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupUpdateProperties(); - } - this.innerProperties().withOsFeedType(osFeedType); - return this; - } - - /** - * Get the updatePolicy property: Update policy of the device group. - * - * @return the updatePolicy value. - */ - public UpdatePolicy updatePolicy() { - return this.innerProperties() == null ? null : this.innerProperties().updatePolicy(); - } - - /** - * Set the updatePolicy property: Update policy of the device group. - * - * @param updatePolicy the updatePolicy value to set. - * @return the DeviceGroupUpdate object itself. - */ - public DeviceGroupUpdate withUpdatePolicy(UpdatePolicy updatePolicy) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupUpdateProperties(); - } - this.innerProperties().withUpdatePolicy(updatePolicy); - return this; - } - - /** - * Get the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * - * @return the allowCrashDumpsCollection value. - */ - public AllowCrashDumpCollection allowCrashDumpsCollection() { - return this.innerProperties() == null ? null : this.innerProperties().allowCrashDumpsCollection(); - } - - /** - * Set the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * - * @param allowCrashDumpsCollection the allowCrashDumpsCollection value to set. - * @return the DeviceGroupUpdate object itself. - */ - public DeviceGroupUpdate withAllowCrashDumpsCollection(AllowCrashDumpCollection allowCrashDumpsCollection) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupUpdateProperties(); - } - this.innerProperties().withAllowCrashDumpsCollection(allowCrashDumpsCollection); - return this; + public DeviceGroupUpdate() { } /** - * Get the regionalDataBoundary property: Regional data boundary for the device group. - * - * @return the regionalDataBoundary value. + * Get the properties property: The updatable properties of the DeviceGroup. + * + * @return the properties value. */ - public RegionalDataBoundary regionalDataBoundary() { - return this.innerProperties() == null ? null : this.innerProperties().regionalDataBoundary(); + public DeviceGroupUpdateProperties properties() { + return this.properties; } /** - * Set the regionalDataBoundary property: Regional data boundary for the device group. - * - * @param regionalDataBoundary the regionalDataBoundary value to set. + * Set the properties property: The updatable properties of the DeviceGroup. + * + * @param properties the properties value to set. * @return the DeviceGroupUpdate object itself. */ - public DeviceGroupUpdate withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceGroupUpdateProperties(); - } - this.innerProperties().withRegionalDataBoundary(regionalDataBoundary); + public DeviceGroupUpdate withProperties(DeviceGroupUpdateProperties properties) { + this.properties = properties; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupUpdateProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdateProperties.java similarity index 87% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupUpdateProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdateProperties.java index 055f245fd2369..0761b1534557d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceGroupUpdateProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroupUpdateProperties.java @@ -2,16 +2,14 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; -import com.azure.resourcemanager.sphere.models.OSFeedType; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; -import com.azure.resourcemanager.sphere.models.UpdatePolicy; import com.fasterxml.jackson.annotation.JsonProperty; -/** The updatable properties of the DeviceGroup. */ +/** + * The updatable properties of the DeviceGroup. + */ @Fluent public final class DeviceGroupUpdateProperties { /* @@ -44,13 +42,15 @@ public final class DeviceGroupUpdateProperties { @JsonProperty(value = "regionalDataBoundary") private RegionalDataBoundary regionalDataBoundary; - /** Creates an instance of DeviceGroupUpdateProperties class. */ + /** + * Creates an instance of DeviceGroupUpdateProperties class. + */ public DeviceGroupUpdateProperties() { } /** * Get the description property: Description of the device group. - * + * * @return the description value. */ public String description() { @@ -59,7 +59,7 @@ public String description() { /** * Set the description property: Description of the device group. - * + * * @param description the description value to set. * @return the DeviceGroupUpdateProperties object itself. */ @@ -70,7 +70,7 @@ public DeviceGroupUpdateProperties withDescription(String description) { /** * Get the osFeedType property: Operating system feed type of the device group. - * + * * @return the osFeedType value. */ public OSFeedType osFeedType() { @@ -79,7 +79,7 @@ public OSFeedType osFeedType() { /** * Set the osFeedType property: Operating system feed type of the device group. - * + * * @param osFeedType the osFeedType value to set. * @return the DeviceGroupUpdateProperties object itself. */ @@ -90,7 +90,7 @@ public DeviceGroupUpdateProperties withOsFeedType(OSFeedType osFeedType) { /** * Get the updatePolicy property: Update policy of the device group. - * + * * @return the updatePolicy value. */ public UpdatePolicy updatePolicy() { @@ -99,7 +99,7 @@ public UpdatePolicy updatePolicy() { /** * Set the updatePolicy property: Update policy of the device group. - * + * * @param updatePolicy the updatePolicy value to set. * @return the DeviceGroupUpdateProperties object itself. */ @@ -110,7 +110,7 @@ public DeviceGroupUpdateProperties withUpdatePolicy(UpdatePolicy updatePolicy) { /** * Get the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * + * * @return the allowCrashDumpsCollection value. */ public AllowCrashDumpCollection allowCrashDumpsCollection() { @@ -119,19 +119,19 @@ public AllowCrashDumpCollection allowCrashDumpsCollection() { /** * Set the allowCrashDumpsCollection property: Flag to define if the user allows for crash dump collection. - * + * * @param allowCrashDumpsCollection the allowCrashDumpsCollection value to set. * @return the DeviceGroupUpdateProperties object itself. */ - public DeviceGroupUpdateProperties withAllowCrashDumpsCollection( - AllowCrashDumpCollection allowCrashDumpsCollection) { + public DeviceGroupUpdateProperties + withAllowCrashDumpsCollection(AllowCrashDumpCollection allowCrashDumpsCollection) { this.allowCrashDumpsCollection = allowCrashDumpsCollection; return this; } /** * Get the regionalDataBoundary property: Regional data boundary for the device group. - * + * * @return the regionalDataBoundary value. */ public RegionalDataBoundary regionalDataBoundary() { @@ -140,7 +140,7 @@ public RegionalDataBoundary regionalDataBoundary() { /** * Set the regionalDataBoundary property: Regional data boundary for the device group. - * + * * @param regionalDataBoundary the regionalDataBoundary value to set. * @return the DeviceGroupUpdateProperties object itself. */ @@ -151,7 +151,7 @@ public DeviceGroupUpdateProperties withRegionalDataBoundary(RegionalDataBoundary /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroups.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroups.java index ece2c86ce5a7b..afed0f5f9dd90 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroups.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceGroups.java @@ -8,12 +8,14 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of DeviceGroups. */ +/** + * Resource collection API of DeviceGroups. + */ public interface DeviceGroups { /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -27,7 +29,7 @@ public interface DeviceGroups { /** * List DeviceGroup resources by Product. '.default' and '.unassigned' are system defined values and cannot be used * for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -41,20 +43,13 @@ public interface DeviceGroups { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByProduct( - String resourceGroupName, - String catalogName, - String productName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByProduct(String resourceGroupName, String catalogName, String productName, + String filter, Integer top, Integer skip, Integer maxpagesize, Context context); /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -65,13 +60,13 @@ PagedIterable listByProduct( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a DeviceGroup along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context); /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -86,7 +81,7 @@ Response getWithResponse( /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -100,7 +95,7 @@ Response getWithResponse( /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -110,13 +105,13 @@ Response getWithResponse( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + Context context); /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -126,17 +121,13 @@ void delete( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, + void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, ClaimDevicesRequest claimDevicesRequest); /** * Bulk claims the devices. Use '.unassigned' or '.default' for the device group and product names when bulk * claiming devices to a catalog only. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -147,18 +138,13 @@ void claimDevices( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void claimDevices( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - ClaimDevicesRequest claimDevicesRequest, - Context context); + void claimDevices(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + ClaimDevicesRequest claimDevicesRequest, Context context); /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -169,13 +155,13 @@ void claimDevices( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response}. */ - Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, Context context); /** * Counts devices in device group. '.default' and '.unassigned' are system defined values and cannot be used for * product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -185,13 +171,13 @@ Response countDevicesWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog. */ - CountDeviceResponse countDevices( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + CountDevicesResponse countDevices(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -203,7 +189,7 @@ CountDeviceResponse countDevices( /** * Get a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -216,7 +202,7 @@ CountDeviceResponse countDevices( /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -227,7 +213,7 @@ CountDeviceResponse countDevices( /** * Delete a DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used for product or * device group name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -238,7 +224,7 @@ CountDeviceResponse countDevices( /** * Begins definition for a new DeviceGroup resource. - * + * * @param name resource name. * @return the first stage of the new DeviceGroup definition. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceInsight.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceInsight.java index adf199386321d..1ddc5e3df51f6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceInsight.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceInsight.java @@ -7,67 +7,69 @@ import com.azure.resourcemanager.sphere.fluent.models.DeviceInsightInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of DeviceInsight. */ +/** + * An immutable client-side representation of DeviceInsight. + */ public interface DeviceInsight { /** * Gets the deviceId property: Device ID. - * + * * @return the deviceId value. */ String deviceId(); /** * Gets the description property: Event description. - * + * * @return the description value. */ String description(); /** * Gets the startTimestampUtc property: Event start timestamp. - * + * * @return the startTimestampUtc value. */ OffsetDateTime startTimestampUtc(); /** * Gets the endTimestampUtc property: Event end timestamp. - * + * * @return the endTimestampUtc value. */ OffsetDateTime endTimestampUtc(); /** * Gets the eventCategory property: Event category. - * + * * @return the eventCategory value. */ String eventCategory(); /** * Gets the eventClass property: Event class. - * + * * @return the eventClass value. */ String eventClass(); /** * Gets the eventType property: Event type. - * + * * @return the eventType value. */ String eventType(); /** * Gets the eventCount property: Event count. - * + * * @return the eventCount value. */ int eventCount(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.DeviceInsightInner object. - * + * * @return the inner object. */ DeviceInsightInner innerModel(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceListResult.java index 5f6ddd913eec1..731fcd4e2491c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a Device list operation. */ +/** + * The response of a Device list operation. + */ @Fluent public final class DeviceListResult { /* @@ -22,16 +24,18 @@ public final class DeviceListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DeviceListResult class. */ + /** + * Creates an instance of DeviceListResult class. + */ public DeviceListResult() { } /** * Get the value property: The Device items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The Device items on this page. - * + * * @param value the value value to set. * @return the DeviceListResult object itself. */ @@ -51,34 +55,22 @@ public DeviceListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the DeviceListResult object itself. - */ - public DeviceListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model DeviceListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DeviceListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceProperties.java similarity index 93% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceProperties.java index 007b9a67c80d8..0367cccef4a0a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceProperties.java @@ -2,14 +2,15 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** The properties of device. */ +/** + * The properties of device. + */ @Fluent public final class DeviceProperties { /* @@ -54,13 +55,15 @@ public final class DeviceProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of DeviceProperties class. */ + /** + * Creates an instance of DeviceProperties class. + */ public DeviceProperties() { } /** * Get the deviceId property: Device ID. - * + * * @return the deviceId value. */ public String deviceId() { @@ -69,7 +72,7 @@ public String deviceId() { /** * Set the deviceId property: Device ID. - * + * * @param deviceId the deviceId value to set. * @return the DeviceProperties object itself. */ @@ -80,7 +83,7 @@ public DeviceProperties withDeviceId(String deviceId) { /** * Get the chipSku property: SKU of the chip. - * + * * @return the chipSku value. */ public String chipSku() { @@ -89,7 +92,7 @@ public String chipSku() { /** * Get the lastAvailableOsVersion property: OS version available for installation when update requested. - * + * * @return the lastAvailableOsVersion value. */ public String lastAvailableOsVersion() { @@ -98,7 +101,7 @@ public String lastAvailableOsVersion() { /** * Get the lastInstalledOsVersion property: OS version running on device when update requested. - * + * * @return the lastInstalledOsVersion value. */ public String lastInstalledOsVersion() { @@ -107,7 +110,7 @@ public String lastInstalledOsVersion() { /** * Get the lastOsUpdateUtc property: Time when update requested and new OS version available. - * + * * @return the lastOsUpdateUtc value. */ public OffsetDateTime lastOsUpdateUtc() { @@ -116,7 +119,7 @@ public OffsetDateTime lastOsUpdateUtc() { /** * Get the lastUpdateRequestUtc property: Time when update was last requested. - * + * * @return the lastUpdateRequestUtc value. */ public OffsetDateTime lastUpdateRequestUtc() { @@ -125,7 +128,7 @@ public OffsetDateTime lastUpdateRequestUtc() { /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -134,7 +137,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdate.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdate.java index 9217bec762a3a..513a2533d53e6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdate.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdate.java @@ -5,62 +5,53 @@ package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.fluent.models.DeviceUpdateProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** The type used for update operations of the Device. */ +/** + * The type used for update operations of the Device. + */ @Fluent public final class DeviceUpdate { /* * The updatable properties of the Device. */ @JsonProperty(value = "properties") - private DeviceUpdateProperties innerProperties; - - /** Creates an instance of DeviceUpdate class. */ - public DeviceUpdate() { - } + private DeviceUpdateProperties properties; /** - * Get the innerProperties property: The updatable properties of the Device. - * - * @return the innerProperties value. + * Creates an instance of DeviceUpdate class. */ - private DeviceUpdateProperties innerProperties() { - return this.innerProperties; + public DeviceUpdate() { } /** - * Get the deviceGroupId property: Device group id. - * - * @return the deviceGroupId value. + * Get the properties property: The updatable properties of the Device. + * + * @return the properties value. */ - public String deviceGroupId() { - return this.innerProperties() == null ? null : this.innerProperties().deviceGroupId(); + public DeviceUpdateProperties properties() { + return this.properties; } /** - * Set the deviceGroupId property: Device group id. - * - * @param deviceGroupId the deviceGroupId value to set. + * Set the properties property: The updatable properties of the Device. + * + * @param properties the properties value to set. * @return the DeviceUpdate object itself. */ - public DeviceUpdate withDeviceGroupId(String deviceGroupId) { - if (this.innerProperties() == null) { - this.innerProperties = new DeviceUpdateProperties(); - } - this.innerProperties().withDeviceGroupId(deviceGroupId); + public DeviceUpdate withProperties(DeviceUpdateProperties properties) { + this.properties = properties; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceUpdateProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdateProperties.java similarity index 85% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceUpdateProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdateProperties.java index d89b5654acf6a..9b6696d7d517d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/DeviceUpdateProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/DeviceUpdateProperties.java @@ -2,12 +2,14 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The updatable properties of the Device. */ +/** + * The updatable properties of the Device. + */ @Fluent public final class DeviceUpdateProperties { /* @@ -16,13 +18,15 @@ public final class DeviceUpdateProperties { @JsonProperty(value = "deviceGroupId") private String deviceGroupId; - /** Creates an instance of DeviceUpdateProperties class. */ + /** + * Creates an instance of DeviceUpdateProperties class. + */ public DeviceUpdateProperties() { } /** * Get the deviceGroupId property: Device group id. - * + * * @return the deviceGroupId value. */ public String deviceGroupId() { @@ -31,7 +35,7 @@ public String deviceGroupId() { /** * Set the deviceGroupId property: Device group id. - * + * * @param deviceGroupId the deviceGroupId value to set. * @return the DeviceUpdateProperties object itself. */ @@ -42,7 +46,7 @@ public DeviceUpdateProperties withDeviceGroupId(String deviceGroupId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Devices.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Devices.java index 120b73eaa9d46..ef2bd6c513eeb 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Devices.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Devices.java @@ -8,12 +8,14 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Devices. */ +/** + * Resource collection API of Devices. + */ public interface Devices { /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -23,13 +25,13 @@ public interface Devices { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName); /** * List Device resources by DeviceGroup. '.default' and '.unassigned' are system defined values and cannot be used * for product or device group name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -40,13 +42,13 @@ PagedIterable listByDeviceGroup( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Device list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByDeviceGroup( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, Context context); + PagedIterable listByDeviceGroup(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, Context context); /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -58,18 +60,13 @@ PagedIterable listByDeviceGroup( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Device along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + String deviceGroupName, String deviceName, Context context); /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -80,12 +77,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Device. */ - Device get( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName); + Device get(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName); /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -95,12 +92,12 @@ Device get( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, String catalogName, String productName, String deviceGroupName, String deviceName); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName); /** * Delete a Device. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -111,18 +108,13 @@ void delete( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - Context context); + void delete(String resourceGroupName, String catalogName, String productName, String deviceGroupName, + String deviceName, Context context); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -134,18 +126,14 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return signed device capability image response. */ - SignedCapabilityImageResponse generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, + SignedCapabilityImageResponse generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, GenerateCapabilityImageRequest generateDeviceCapabilityRequest); /** * Generates the capability image for the device. Use '.unassigned' or '.default' for the device group and product * names to generate the image for a device that does not belong to a specific device group and product. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -158,19 +146,14 @@ SignedCapabilityImageResponse generateCapabilityImage( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return signed device capability image response. */ - SignedCapabilityImageResponse generateCapabilityImage( - String resourceGroupName, - String catalogName, - String productName, - String deviceGroupName, - String deviceName, - GenerateCapabilityImageRequest generateDeviceCapabilityRequest, - Context context); + SignedCapabilityImageResponse generateCapabilityImage(String resourceGroupName, String catalogName, + String productName, String deviceGroupName, String deviceName, + GenerateCapabilityImageRequest generateDeviceCapabilityRequest, Context context); /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -182,7 +165,7 @@ SignedCapabilityImageResponse generateCapabilityImage( /** * Get a Device. Use '.unassigned' or '.default' for the device group and product names when a device does not * belong to a device group and product. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -194,7 +177,7 @@ SignedCapabilityImageResponse generateCapabilityImage( /** * Delete a Device. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -204,7 +187,7 @@ SignedCapabilityImageResponse generateCapabilityImage( /** * Delete a Device. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -215,7 +198,7 @@ SignedCapabilityImageResponse generateCapabilityImage( /** * Begins definition for a new Device resource. - * + * * @param name resource name. * @return the first stage of the new Device definition. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/GenerateCapabilityImageRequest.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/GenerateCapabilityImageRequest.java index e795eb1a071e1..149670e052b4d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/GenerateCapabilityImageRequest.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/GenerateCapabilityImageRequest.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Request of the action to create a signed device capability image. */ +/** + * Request of the action to create a signed device capability image. + */ @Fluent public final class GenerateCapabilityImageRequest { /* @@ -18,13 +20,15 @@ public final class GenerateCapabilityImageRequest { @JsonProperty(value = "capabilities", required = true) private List capabilities; - /** Creates an instance of GenerateCapabilityImageRequest class. */ + /** + * Creates an instance of GenerateCapabilityImageRequest class. + */ public GenerateCapabilityImageRequest() { } /** * Get the capabilities property: List of capabilities to create. - * + * * @return the capabilities value. */ public List capabilities() { @@ -33,7 +37,7 @@ public List capabilities() { /** * Set the capabilities property: List of capabilities to create. - * + * * @param capabilities the capabilities value to set. * @return the GenerateCapabilityImageRequest object itself. */ @@ -44,15 +48,13 @@ public GenerateCapabilityImageRequest withCapabilities(List capa /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (capabilities() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property capabilities in model GenerateCapabilityImageRequest")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property capabilities in model GenerateCapabilityImageRequest")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Image.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Image.java index 03c294f282cfb..c9834ae799a37 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Image.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Image.java @@ -4,126 +4,87 @@ package com.azure.resourcemanager.sphere.models; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; -/** An immutable client-side representation of Image. */ +/** + * An immutable client-side representation of Image. + */ public interface Image { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the image property: Image as a UTF-8 encoded base 64 string on image create. This field contains the image - * URI on image reads. - * - * @return the image value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - String image(); + ImageProperties properties(); /** - * Gets the imageId property: Image ID. - * - * @return the imageId value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - String imageId(); - - /** - * Gets the imageName property: Image name. - * - * @return the imageName value. - */ - String imageName(); - - /** - * Gets the regionalDataBoundary property: Regional data boundary for an image. - * - * @return the regionalDataBoundary value. - */ - RegionalDataBoundary regionalDataBoundary(); - - /** - * Gets the uri property: Location the image. - * - * @return the uri value. - */ - String uri(); - - /** - * Gets the description property: The image description. - * - * @return the description value. - */ - String description(); - - /** - * Gets the componentId property: The image component id. - * - * @return the componentId value. - */ - String componentId(); - - /** - * Gets the imageType property: The image type. - * - * @return the imageType value. - */ - ImageType imageType(); - - /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. - */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.ImageInner object. - * + * * @return the inner object. */ ImageInner innerModel(); - /** The entirety of the Image definition. */ + /** + * The entirety of the Image definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Image definition stages. */ + /** + * The Image definition stages. + */ interface DefinitionStages { - /** The first stage of the Image definition. */ + /** + * The first stage of the Image definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Image definition allowing to specify parent resource. */ + /** + * The stage of the Image definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, catalogName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @return the next definition stage. @@ -135,101 +96,80 @@ interface WithParentResource { * The stage of the Image definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithImage, - DefinitionStages.WithImageId, - DefinitionStages.WithRegionalDataBoundary { + interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ Image create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Image create(Context context); } - /** The stage of the Image definition allowing to specify image. */ - interface WithImage { - /** - * Specifies the image property: Image as a UTF-8 encoded base 64 string on image create. This field - * contains the image URI on image reads.. - * - * @param image Image as a UTF-8 encoded base 64 string on image create. This field contains the image URI - * on image reads. - * @return the next definition stage. - */ - WithCreate withImage(String image); - } - - /** The stage of the Image definition allowing to specify imageId. */ - interface WithImageId { - /** - * Specifies the imageId property: Image ID. - * - * @param imageId Image ID. - * @return the next definition stage. - */ - WithCreate withImageId(String imageId); - } - - /** The stage of the Image definition allowing to specify regionalDataBoundary. */ - interface WithRegionalDataBoundary { + /** + * The stage of the Image definition allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the regionalDataBoundary property: Regional data boundary for an image. - * - * @param regionalDataBoundary Regional data boundary for an image. + * Specifies the properties property: The resource-specific properties for this resource.. + * + * @param properties The resource-specific properties for this resource. * @return the next definition stage. */ - WithCreate withRegionalDataBoundary(RegionalDataBoundary regionalDataBoundary); + WithCreate withProperties(ImageProperties properties); } } /** * Begins update for the Image resource. - * + * * @return the stage of resource update. */ Image.Update update(); - /** The template for Image update. */ + /** + * The template for Image update. + */ interface Update { /** * Executes the update request. - * + * * @return the updated resource. */ Image apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Image apply(Context context); } - /** The Image update stages. */ + /** + * The Image update stages. + */ interface UpdateStages { } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Image refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageListResult.java index a6e018f8b2eec..a88e846764dcd 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a Image list operation. */ +/** + * The response of a Image list operation. + */ @Fluent public final class ImageListResult { /* @@ -22,16 +24,18 @@ public final class ImageListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ImageListResult class. */ + /** + * Creates an instance of ImageListResult class. + */ public ImageListResult() { } /** * Get the value property: The Image items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The Image items on this page. - * + * * @param value the value value to set. * @return the ImageListResult object itself. */ @@ -51,34 +55,22 @@ public ImageListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the ImageListResult object itself. - */ - public ImageListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model ImageListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ImageListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageProperties.java similarity index 91% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageProperties.java index 26464c8d0a60a..a2c63b632aa81 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ImageProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageProperties.java @@ -2,15 +2,14 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.models.ImageType; -import com.azure.resourcemanager.sphere.models.ProvisioningState; -import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of image. */ +/** + * The properties of image. + */ @Fluent public final class ImageProperties { /* @@ -67,14 +66,16 @@ public final class ImageProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of ImageProperties class. */ + /** + * Creates an instance of ImageProperties class. + */ public ImageProperties() { } /** * Get the image property: Image as a UTF-8 encoded base 64 string on image create. This field contains the image * URI on image reads. - * + * * @return the image value. */ public String image() { @@ -84,7 +85,7 @@ public String image() { /** * Set the image property: Image as a UTF-8 encoded base 64 string on image create. This field contains the image * URI on image reads. - * + * * @param image the image value to set. * @return the ImageProperties object itself. */ @@ -95,7 +96,7 @@ public ImageProperties withImage(String image) { /** * Get the imageId property: Image ID. - * + * * @return the imageId value. */ public String imageId() { @@ -104,7 +105,7 @@ public String imageId() { /** * Set the imageId property: Image ID. - * + * * @param imageId the imageId value to set. * @return the ImageProperties object itself. */ @@ -115,7 +116,7 @@ public ImageProperties withImageId(String imageId) { /** * Get the imageName property: Image name. - * + * * @return the imageName value. */ public String imageName() { @@ -124,7 +125,7 @@ public String imageName() { /** * Get the regionalDataBoundary property: Regional data boundary for an image. - * + * * @return the regionalDataBoundary value. */ public RegionalDataBoundary regionalDataBoundary() { @@ -133,7 +134,7 @@ public RegionalDataBoundary regionalDataBoundary() { /** * Set the regionalDataBoundary property: Regional data boundary for an image. - * + * * @param regionalDataBoundary the regionalDataBoundary value to set. * @return the ImageProperties object itself. */ @@ -144,7 +145,7 @@ public ImageProperties withRegionalDataBoundary(RegionalDataBoundary regionalDat /** * Get the uri property: Location the image. - * + * * @return the uri value. */ public String uri() { @@ -153,7 +154,7 @@ public String uri() { /** * Get the description property: The image description. - * + * * @return the description value. */ public String description() { @@ -162,7 +163,7 @@ public String description() { /** * Get the componentId property: The image component id. - * + * * @return the componentId value. */ public String componentId() { @@ -171,7 +172,7 @@ public String componentId() { /** * Get the imageType property: The image type. - * + * * @return the imageType value. */ public ImageType imageType() { @@ -180,7 +181,7 @@ public ImageType imageType() { /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -189,7 +190,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageType.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageType.java index b12f2162886a8..6d4989951a21e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageType.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ImageType.java @@ -8,83 +8,133 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Image type values. */ +/** + * Image type values. + */ public final class ImageType extends ExpandableStringEnum { - /** Static value InvalidImageType for ImageType. */ + /** + * Static value InvalidImageType for ImageType. + */ public static final ImageType INVALID_IMAGE_TYPE = fromString("InvalidImageType"); - /** Static value OneBl for ImageType. */ + /** + * Static value OneBl for ImageType. + */ public static final ImageType ONE_BL = fromString("OneBl"); - /** Static value PlutonRuntime for ImageType. */ + /** + * Static value PlutonRuntime for ImageType. + */ public static final ImageType PLUTON_RUNTIME = fromString("PlutonRuntime"); - /** Static value WifiFirmware for ImageType. */ + /** + * Static value WifiFirmware for ImageType. + */ public static final ImageType WIFI_FIRMWARE = fromString("WifiFirmware"); - /** Static value SecurityMonitor for ImageType. */ + /** + * Static value SecurityMonitor for ImageType. + */ public static final ImageType SECURITY_MONITOR = fromString("SecurityMonitor"); - /** Static value NormalWorldLoader for ImageType. */ + /** + * Static value NormalWorldLoader for ImageType. + */ public static final ImageType NORMAL_WORLD_LOADER = fromString("NormalWorldLoader"); - /** Static value NormalWorldDtb for ImageType. */ + /** + * Static value NormalWorldDtb for ImageType. + */ public static final ImageType NORMAL_WORLD_DTB = fromString("NormalWorldDtb"); - /** Static value NormalWorldKernel for ImageType. */ + /** + * Static value NormalWorldKernel for ImageType. + */ public static final ImageType NORMAL_WORLD_KERNEL = fromString("NormalWorldKernel"); - /** Static value RootFs for ImageType. */ + /** + * Static value RootFs for ImageType. + */ public static final ImageType ROOT_FS = fromString("RootFs"); - /** Static value Services for ImageType. */ + /** + * Static value Services for ImageType. + */ public static final ImageType SERVICES = fromString("Services"); - /** Static value Applications for ImageType. */ + /** + * Static value Applications for ImageType. + */ public static final ImageType APPLICATIONS = fromString("Applications"); - /** Static value FwConfig for ImageType. */ + /** + * Static value FwConfig for ImageType. + */ public static final ImageType FW_CONFIG = fromString("FwConfig"); - /** Static value BootManifest for ImageType. */ + /** + * Static value BootManifest for ImageType. + */ public static final ImageType BOOT_MANIFEST = fromString("BootManifest"); - /** Static value Nwfs for ImageType. */ + /** + * Static value Nwfs for ImageType. + */ public static final ImageType NWFS = fromString("Nwfs"); - /** Static value TrustedKeystore for ImageType. */ + /** + * Static value TrustedKeystore for ImageType. + */ public static final ImageType TRUSTED_KEYSTORE = fromString("TrustedKeystore"); - /** Static value Policy for ImageType. */ + /** + * Static value Policy for ImageType. + */ public static final ImageType POLICY = fromString("Policy"); - /** Static value CustomerBoardConfig for ImageType. */ + /** + * Static value CustomerBoardConfig for ImageType. + */ public static final ImageType CUSTOMER_BOARD_CONFIG = fromString("CustomerBoardConfig"); - /** Static value UpdateCertStore for ImageType. */ + /** + * Static value UpdateCertStore for ImageType. + */ public static final ImageType UPDATE_CERT_STORE = fromString("UpdateCertStore"); - /** Static value BaseSystemUpdateManifest for ImageType. */ + /** + * Static value BaseSystemUpdateManifest for ImageType. + */ public static final ImageType BASE_SYSTEM_UPDATE_MANIFEST = fromString("BaseSystemUpdateManifest"); - /** Static value FirmwareUpdateManifest for ImageType. */ + /** + * Static value FirmwareUpdateManifest for ImageType. + */ public static final ImageType FIRMWARE_UPDATE_MANIFEST = fromString("FirmwareUpdateManifest"); - /** Static value CustomerUpdateManifest for ImageType. */ + /** + * Static value CustomerUpdateManifest for ImageType. + */ public static final ImageType CUSTOMER_UPDATE_MANIFEST = fromString("CustomerUpdateManifest"); - /** Static value RecoveryManifest for ImageType. */ + /** + * Static value RecoveryManifest for ImageType. + */ public static final ImageType RECOVERY_MANIFEST = fromString("RecoveryManifest"); - /** Static value ManifestSet for ImageType. */ + /** + * Static value ManifestSet for ImageType. + */ public static final ImageType MANIFEST_SET = fromString("ManifestSet"); - /** Static value Other for ImageType. */ + /** + * Static value Other for ImageType. + */ public static final ImageType OTHER = fromString("Other"); /** * Creates a new instance of ImageType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -93,7 +143,7 @@ public ImageType() { /** * Creates or finds a ImageType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ImageType. */ @@ -104,7 +154,7 @@ public static ImageType fromString(String name) { /** * Gets known ImageType values. - * + * * @return known ImageType values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Images.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Images.java index 482e9daa70998..da17aaf6899ac 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Images.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Images.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Images. */ +/** + * Resource collection API of Images. + */ public interface Images { /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface Images { /** * List Image resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param filter Filter the result list using the given expression. @@ -37,21 +39,15 @@ public interface Images { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a Image list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByCatalog( - String resourceGroupName, - String catalogName, - String filter, - Integer top, - Integer skip, - Integer maxpagesize, - Context context); + PagedIterable listByCatalog(String resourceGroupName, String catalogName, String filter, Integer top, + Integer skip, Integer maxpagesize, Context context); /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -62,10 +58,10 @@ PagedIterable listByCatalog( /** * Get a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -75,10 +71,10 @@ PagedIterable listByCatalog( /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -87,10 +83,10 @@ PagedIterable listByCatalog( /** * Delete a Image. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. - * @param imageName Image name. Use .default for image creation. + * @param imageName Image name. Use an image GUID for GA versions of the API. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -100,7 +96,7 @@ PagedIterable listByCatalog( /** * Get a Image. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -111,7 +107,7 @@ PagedIterable listByCatalog( /** * Get a Image. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -123,7 +119,7 @@ PagedIterable listByCatalog( /** * Delete a Image. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -133,7 +129,7 @@ PagedIterable listByCatalog( /** * Delete a Image. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -144,7 +140,7 @@ PagedIterable listByCatalog( /** * Begins definition for a new Image resource. - * + * * @param name resource name. * @return the first stage of the new Image definition. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ListDeviceGroupsRequest.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ListDeviceGroupsRequest.java index aeac84fe7a660..577a0d17c6feb 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ListDeviceGroupsRequest.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ListDeviceGroupsRequest.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Request of the action to list device groups for a catalog. */ +/** + * Request of the action to list device groups for a catalog. + */ @Fluent public final class ListDeviceGroupsRequest { /* @@ -16,13 +18,15 @@ public final class ListDeviceGroupsRequest { @JsonProperty(value = "deviceGroupName") private String deviceGroupName; - /** Creates an instance of ListDeviceGroupsRequest class. */ + /** + * Creates an instance of ListDeviceGroupsRequest class. + */ public ListDeviceGroupsRequest() { } /** * Get the deviceGroupName property: Device Group name. - * + * * @return the deviceGroupName value. */ public String deviceGroupName() { @@ -31,7 +35,7 @@ public String deviceGroupName() { /** * Set the deviceGroupName property: Device Group name. - * + * * @param deviceGroupName the deviceGroupName value to set. * @return the ListDeviceGroupsRequest object itself. */ @@ -42,7 +46,7 @@ public ListDeviceGroupsRequest withDeviceGroupName(String deviceGroupName) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OSFeedType.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OSFeedType.java index b22ba3545e505..df86906705bf7 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OSFeedType.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OSFeedType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** OS feed type values. */ +/** + * OS feed type values. + */ public final class OSFeedType extends ExpandableStringEnum { - /** Static value Retail for OSFeedType. */ + /** + * Static value Retail for OSFeedType. + */ public static final OSFeedType RETAIL = fromString("Retail"); - /** Static value RetailEval for OSFeedType. */ + /** + * Static value RetailEval for OSFeedType. + */ public static final OSFeedType RETAIL_EVAL = fromString("RetailEval"); /** * Creates a new instance of OSFeedType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public OSFeedType() { /** * Creates or finds a OSFeedType from its string representation. - * + * * @param name a name to look for. * @return the corresponding OSFeedType. */ @@ -38,7 +44,7 @@ public static OSFeedType fromString(String name) { /** * Gets known OSFeedType values. - * + * * @return known OSFeedType values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operation.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operation.java index 7fd48e4bde394..f39a2845910dc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operation.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operation.java @@ -6,12 +6,14 @@ import com.azure.resourcemanager.sphere.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ String name(); @@ -19,14 +21,14 @@ public interface Operation { /** * Gets the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Localized display information for this particular operation. - * + * * @return the display value. */ OperationDisplay display(); @@ -34,7 +36,7 @@ public interface Operation { /** * Gets the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ Origin origin(); @@ -42,14 +44,14 @@ public interface Operation { /** * Gets the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ ActionType actionType(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationDisplay.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationDisplay.java index 562d7f5f396c8..176049cf9e0d7 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationDisplay.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Localized display information for this particular operation. */ +/** + * Localized display information for this particular operation. + */ @Immutable public final class OperationDisplay { /* @@ -37,14 +39,16 @@ public final class OperationDisplay { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: The localized friendly form of the resource provider name, e.g. "Microsoft Monitoring * Insights" or "Microsoft Compute". - * + * * @return the provider value. */ public String provider() { @@ -54,7 +58,7 @@ public String provider() { /** * Get the resource property: The localized friendly name of the resource type related to this operation. E.g. * "Virtual Machines" or "Job Schedule Collections". - * + * * @return the resource value. */ public String resource() { @@ -64,7 +68,7 @@ public String resource() { /** * Get the operation property: The concise, localized friendly name for the operation; suitable for dropdowns. E.g. * "Create or Update Virtual Machine", "Restart Virtual Machine". - * + * * @return the operation value. */ public String operation() { @@ -74,7 +78,7 @@ public String operation() { /** * Get the description property: The short, localized friendly description of the operation; suitable for tool tips * and detailed views. - * + * * @return the description value. */ public String description() { @@ -83,7 +87,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationListResult.java index e9291b9358f4f..01821c5564be6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/OperationListResult.java @@ -10,8 +10,8 @@ import java.util.List; /** - * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set of - * results. + * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set + * of results. */ @Immutable public final class OperationListResult { @@ -27,13 +27,15 @@ public final class OperationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationListResult class. */ + /** + * Creates an instance of OperationListResult class. + */ public OperationListResult() { } /** * Get the value property: List of operations supported by the resource provider. - * + * * @return the value value. */ public List value() { @@ -42,7 +44,7 @@ public List value() { /** * Get the nextLink property: URL to get the next set of operation list results (if there are any). - * + * * @return the nextLink value. */ public String nextLink() { @@ -51,7 +53,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operations.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operations.java index 53ce30f3ca799..fad6812e8ca83 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operations.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Operations.java @@ -7,27 +7,29 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * List the operations for the provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Origin.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Origin.java index a5871fb788c75..7b023584191ff 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Origin.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Origin.java @@ -13,18 +13,24 @@ * is "user,system". */ public final class Origin extends ExpandableStringEnum { - /** Static value user for Origin. */ + /** + * Static value user for Origin. + */ public static final Origin USER = fromString("user"); - /** Static value system for Origin. */ + /** + * Static value system for Origin. + */ public static final Origin SYSTEM = fromString("system"); - /** Static value user,system for Origin. */ + /** + * Static value user,system for Origin. + */ public static final Origin USER_SYSTEM = fromString("user,system"); /** * Creates a new instance of Origin value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +39,7 @@ public Origin() { /** * Creates or finds a Origin from its string representation. - * + * * @param name a name to look for. * @return the corresponding Origin. */ @@ -44,7 +50,7 @@ public static Origin fromString(String name) { /** * Gets known Origin values. - * + * * @return known Origin values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/PagedDeviceInsight.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/PagedDeviceInsight.java index 36810c2a7ab54..518bec40d5447 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/PagedDeviceInsight.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/PagedDeviceInsight.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Paged collection of DeviceInsight items. */ +/** + * Paged collection of DeviceInsight items. + */ @Fluent public final class PagedDeviceInsight { /* @@ -22,16 +24,18 @@ public final class PagedDeviceInsight { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of PagedDeviceInsight class. */ + /** + * Creates an instance of PagedDeviceInsight class. + */ public PagedDeviceInsight() { } /** * Get the value property: The DeviceInsight items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The DeviceInsight items on this page. - * + * * @param value the value value to set. * @return the PagedDeviceInsight object itself. */ @@ -51,34 +55,22 @@ public PagedDeviceInsight withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the PagedDeviceInsight object itself. - */ - public PagedDeviceInsight withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model PagedDeviceInsight")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model PagedDeviceInsight")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Product.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Product.java index d88525e092fc0..e57616eacea67 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Product.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Product.java @@ -6,76 +6,87 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; -/** An immutable client-side representation of Product. */ +/** + * An immutable client-side representation of Product. + */ public interface Product { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the description property: Description of the product. - * - * @return the description value. + * Gets the properties property: The resource-specific properties for this resource. + * + * @return the properties value. */ - String description(); + ProductProperties properties(); /** - * Gets the provisioningState property: The status of the last operation. - * - * @return the provisioningState value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - ProvisioningState provisioningState(); + SystemData systemData(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.ProductInner object. - * + * * @return the inner object. */ ProductInner innerModel(); - /** The entirety of the Product definition. */ + /** + * The entirety of the Product definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Product definition stages. */ + /** + * The Product definition stages. + */ interface DefinitionStages { - /** The first stage of the Product definition. */ + /** + * The first stage of the Product definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Product definition allowing to specify parent resource. */ + /** + * The stage of the Product definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, catalogName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @return the next definition stage. @@ -87,84 +98,92 @@ interface WithParentResource { * The stage of the Product definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate extends DefinitionStages.WithDescription { + interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ Product create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Product create(Context context); } - /** The stage of the Product definition allowing to specify description. */ - interface WithDescription { + /** + * The stage of the Product definition allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the description property: Description of the product. - * - * @param description Description of the product. + * Specifies the properties property: The resource-specific properties for this resource.. + * + * @param properties The resource-specific properties for this resource. * @return the next definition stage. */ - WithCreate withDescription(String description); + WithCreate withProperties(ProductProperties properties); } } /** * Begins update for the Product resource. - * + * * @return the stage of resource update. */ Product.Update update(); - /** The template for Product update. */ - interface Update extends UpdateStages.WithDescription { + /** + * The template for Product update. + */ + interface Update extends UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ Product apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Product apply(Context context); } - /** The Product update stages. */ + /** + * The Product update stages. + */ interface UpdateStages { - /** The stage of the Product update allowing to specify description. */ - interface WithDescription { + /** + * The stage of the Product update allowing to specify properties. + */ + interface WithProperties { /** - * Specifies the description property: Description of the product. - * - * @param description Description of the product. + * Specifies the properties property: The updatable properties of the Product.. + * + * @param properties The updatable properties of the Product. * @return the next definition stage. */ - Update withDescription(String description); + Update withProperties(ProductUpdateProperties properties); } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Product refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -173,29 +192,29 @@ interface WithDescription { /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response}. */ - Response countDevicesWithResponse(Context context); + Response countDevicesWithResponse(Context context); /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog. */ - CountDeviceResponse countDevices(); + CountDevicesResponse countDevices(); /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. @@ -205,7 +224,7 @@ interface WithDescription { /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductListResult.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductListResult.java index 37e441bd57a68..1008e53514ebe 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductListResult.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a Product list operation. */ +/** + * The response of a Product list operation. + */ @Fluent public final class ProductListResult { /* @@ -22,16 +24,18 @@ public final class ProductListResult { /* * The link to the next page of items */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ProductListResult class. */ + /** + * Creates an instance of ProductListResult class. + */ public ProductListResult() { } /** * Get the value property: The Product items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The Product items on this page. - * + * * @param value the value value to set. * @return the ProductListResult object itself. */ @@ -51,34 +55,22 @@ public ProductListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: The link to the next page of items. - * - * @param nextLink the nextLink value to set. - * @return the ProductListResult object itself. - */ - public ProductListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model ProductListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ProductListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductProperties.java similarity index 69% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductProperties.java index c9e8cd2176f70..477c971fdda9e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductProperties.java @@ -2,20 +2,20 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.sphere.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of product. */ +/** + * The properties of product. + */ @Fluent public final class ProductProperties { /* * Description of the product */ - @JsonProperty(value = "description", required = true) + @JsonProperty(value = "description") private String description; /* @@ -24,13 +24,15 @@ public final class ProductProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of ProductProperties class. */ + /** + * Creates an instance of ProductProperties class. + */ public ProductProperties() { } /** * Get the description property: Description of the product. - * + * * @return the description value. */ public String description() { @@ -39,7 +41,7 @@ public String description() { /** * Set the description property: Description of the product. - * + * * @param description the description value to set. * @return the ProductProperties object itself. */ @@ -50,7 +52,7 @@ public ProductProperties withDescription(String description) { /** * Get the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -59,16 +61,9 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (description() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property description in model ProductProperties")); - } } - - private static final ClientLogger LOGGER = new ClientLogger(ProductProperties.class); } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdate.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdate.java index 728b43305eb4a..344d810850829 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdate.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdate.java @@ -5,62 +5,53 @@ package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.sphere.fluent.models.ProductUpdateProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** The type used for update operations of the Product. */ +/** + * The type used for update operations of the Product. + */ @Fluent public final class ProductUpdate { /* * The updatable properties of the Product. */ @JsonProperty(value = "properties") - private ProductUpdateProperties innerProperties; - - /** Creates an instance of ProductUpdate class. */ - public ProductUpdate() { - } + private ProductUpdateProperties properties; /** - * Get the innerProperties property: The updatable properties of the Product. - * - * @return the innerProperties value. + * Creates an instance of ProductUpdate class. */ - private ProductUpdateProperties innerProperties() { - return this.innerProperties; + public ProductUpdate() { } /** - * Get the description property: Description of the product. - * - * @return the description value. + * Get the properties property: The updatable properties of the Product. + * + * @return the properties value. */ - public String description() { - return this.innerProperties() == null ? null : this.innerProperties().description(); + public ProductUpdateProperties properties() { + return this.properties; } /** - * Set the description property: Description of the product. - * - * @param description the description value to set. + * Set the properties property: The updatable properties of the Product. + * + * @param properties the properties value to set. * @return the ProductUpdate object itself. */ - public ProductUpdate withDescription(String description) { - if (this.innerProperties() == null) { - this.innerProperties = new ProductUpdateProperties(); - } - this.innerProperties().withDescription(description); + public ProductUpdate withProperties(ProductUpdateProperties properties) { + this.properties = properties; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); + if (properties() != null) { + properties().validate(); } } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductUpdateProperties.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdateProperties.java similarity index 85% rename from sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductUpdateProperties.java rename to sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdateProperties.java index 77e75f019861f..fd4cc19da2ffe 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/fluent/models/ProductUpdateProperties.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProductUpdateProperties.java @@ -2,12 +2,14 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.sphere.fluent.models; +package com.azure.resourcemanager.sphere.models; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The updatable properties of the Product. */ +/** + * The updatable properties of the Product. + */ @Fluent public final class ProductUpdateProperties { /* @@ -16,13 +18,15 @@ public final class ProductUpdateProperties { @JsonProperty(value = "description") private String description; - /** Creates an instance of ProductUpdateProperties class. */ + /** + * Creates an instance of ProductUpdateProperties class. + */ public ProductUpdateProperties() { } /** * Get the description property: Description of the product. - * + * * @return the description value. */ public String description() { @@ -31,7 +35,7 @@ public String description() { /** * Set the description property: Description of the product. - * + * * @param description the description value to set. * @return the ProductUpdateProperties object itself. */ @@ -42,7 +46,7 @@ public ProductUpdateProperties withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Products.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Products.java index 2d19d86941ed4..c6ebe0df8eafa 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Products.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/Products.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Products. */ +/** + * Resource collection API of Products. + */ public interface Products { /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface Products { /** * List Product resources by Catalog. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param context The context to associate with this operation. @@ -37,7 +39,7 @@ public interface Products { /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -47,12 +49,12 @@ public interface Products { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Product along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String catalogName, String productName, Context context); + Response getWithResponse(String resourceGroupName, String catalogName, String productName, + Context context); /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -91,7 +93,7 @@ Response getWithResponse( /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -101,13 +103,13 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog along with {@link Response}. */ - Response countDevicesWithResponse( - String resourceGroupName, String catalogName, String productName, Context context); + Response countDevicesWithResponse(String resourceGroupName, String catalogName, + String productName, Context context); /** * Counts devices in product. '.default' and '.unassigned' are system defined values and cannot be used for product * name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -116,12 +118,12 @@ Response countDevicesWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response to the action call for count devices in a catalog. */ - CountDeviceResponse countDevices(String resourceGroupName, String catalogName, String productName); + CountDevicesResponse countDevices(String resourceGroupName, String catalogName, String productName); /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -130,13 +132,13 @@ Response countDevicesWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ - PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName); + PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName); /** * Generates default device groups for the product. '.default' and '.unassigned' are system defined values and * cannot be used for product name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param catalogName Name of catalog. * @param productName Name of product. @@ -146,12 +148,12 @@ PagedIterable generateDefaultDeviceGroups( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a DeviceGroup list operation as paginated response with {@link PagedIterable}. */ - PagedIterable generateDefaultDeviceGroups( - String resourceGroupName, String catalogName, String productName, Context context); + PagedIterable generateDefaultDeviceGroups(String resourceGroupName, String catalogName, + String productName, Context context); /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -162,7 +164,7 @@ PagedIterable generateDefaultDeviceGroups( /** * Get a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -174,7 +176,7 @@ PagedIterable generateDefaultDeviceGroups( /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -184,7 +186,7 @@ PagedIterable generateDefaultDeviceGroups( /** * Delete a Product. '.default' and '.unassigned' are system defined values and cannot be used for product name'. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -195,7 +197,7 @@ PagedIterable generateDefaultDeviceGroups( /** * Begins definition for a new Product resource. - * + * * @param name resource name. * @return the first stage of the new Product definition. */ diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceRequest.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceRequest.java index 259f942574dae..04c79b6216470 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceRequest.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceRequest.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Request for the proof of possession nonce. */ +/** + * Request for the proof of possession nonce. + */ @Fluent public final class ProofOfPossessionNonceRequest { /* @@ -17,13 +19,15 @@ public final class ProofOfPossessionNonceRequest { @JsonProperty(value = "proofOfPossessionNonce", required = true) private String proofOfPossessionNonce; - /** Creates an instance of ProofOfPossessionNonceRequest class. */ + /** + * Creates an instance of ProofOfPossessionNonceRequest class. + */ public ProofOfPossessionNonceRequest() { } /** * Get the proofOfPossessionNonce property: The proof of possession nonce. - * + * * @return the proofOfPossessionNonce value. */ public String proofOfPossessionNonce() { @@ -32,7 +36,7 @@ public String proofOfPossessionNonce() { /** * Set the proofOfPossessionNonce property: The proof of possession nonce. - * + * * @param proofOfPossessionNonce the proofOfPossessionNonce value to set. * @return the ProofOfPossessionNonceRequest object itself. */ @@ -43,15 +47,13 @@ public ProofOfPossessionNonceRequest withProofOfPossessionNonce(String proofOfPo /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (proofOfPossessionNonce() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property proofOfPossessionNonce in model ProofOfPossessionNonceRequest")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property proofOfPossessionNonce in model ProofOfPossessionNonceRequest")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceResponse.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceResponse.java index 74245243ac808..3bcdf8d2100d1 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceResponse.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProofOfPossessionNonceResponse.java @@ -7,60 +7,62 @@ import com.azure.resourcemanager.sphere.fluent.models.ProofOfPossessionNonceResponseInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of ProofOfPossessionNonceResponse. */ +/** + * An immutable client-side representation of ProofOfPossessionNonceResponse. + */ public interface ProofOfPossessionNonceResponse { /** * Gets the certificate property: The certificate as a UTF-8 encoded base 64 string. - * + * * @return the certificate value. */ String certificate(); /** * Gets the status property: The certificate status. - * + * * @return the status value. */ CertificateStatus status(); /** * Gets the subject property: The certificate subject. - * + * * @return the subject value. */ String subject(); /** * Gets the thumbprint property: The certificate thumbprint. - * + * * @return the thumbprint value. */ String thumbprint(); /** * Gets the expiryUtc property: The certificate expiry date. - * + * * @return the expiryUtc value. */ OffsetDateTime expiryUtc(); /** * Gets the notBeforeUtc property: The certificate not before date. - * + * * @return the notBeforeUtc value. */ OffsetDateTime notBeforeUtc(); /** * Gets the provisioningState property: The status of the last operation. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.ProofOfPossessionNonceResponseInner object. - * + * * @return the inner object. */ ProofOfPossessionNonceResponseInner innerModel(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProvisioningState.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProvisioningState.java index 2b6e552a5b3e4..83fe4afb8936e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProvisioningState.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/ProvisioningState.java @@ -8,32 +8,48 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning state of the resource. */ +/** + * Provisioning state of resource. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value Provisioning for ProvisioningState. */ + /** + * Static value Provisioning for ProvisioningState. + */ public static final ProvisioningState PROVISIONING = fromString("Provisioning"); - /** Static value Updating for ProvisioningState. */ + /** + * Static value Updating for ProvisioningState. + */ public static final ProvisioningState UPDATING = fromString("Updating"); - /** Static value Deleting for ProvisioningState. */ + /** + * Static value Deleting for ProvisioningState. + */ public static final ProvisioningState DELETING = fromString("Deleting"); - /** Static value Accepted for ProvisioningState. */ + /** + * Static value Accepted for ProvisioningState. + */ public static final ProvisioningState ACCEPTED = fromString("Accepted"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -42,7 +58,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -53,7 +69,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/RegionalDataBoundary.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/RegionalDataBoundary.java index 37ff9a000820c..d91169299d5fc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/RegionalDataBoundary.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/RegionalDataBoundary.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Regional data boundary values. */ +/** + * Regional data boundary values. + */ public final class RegionalDataBoundary extends ExpandableStringEnum { - /** Static value None for RegionalDataBoundary. */ + /** + * Static value None for RegionalDataBoundary. + */ public static final RegionalDataBoundary NONE = fromString("None"); - /** Static value EU for RegionalDataBoundary. */ + /** + * Static value EU for RegionalDataBoundary. + */ public static final RegionalDataBoundary EU = fromString("EU"); /** * Creates a new instance of RegionalDataBoundary value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public RegionalDataBoundary() { /** * Creates or finds a RegionalDataBoundary from its string representation. - * + * * @param name a name to look for. * @return the corresponding RegionalDataBoundary. */ @@ -38,7 +44,7 @@ public static RegionalDataBoundary fromString(String name) { /** * Gets known RegionalDataBoundary values. - * + * * @return known RegionalDataBoundary values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/SignedCapabilityImageResponse.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/SignedCapabilityImageResponse.java index 3e93a7ae3156b..d2477da612aeb 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/SignedCapabilityImageResponse.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/SignedCapabilityImageResponse.java @@ -6,18 +6,20 @@ import com.azure.resourcemanager.sphere.fluent.models.SignedCapabilityImageResponseInner; -/** An immutable client-side representation of SignedCapabilityImageResponse. */ +/** + * An immutable client-side representation of SignedCapabilityImageResponse. + */ public interface SignedCapabilityImageResponse { /** * Gets the image property: The signed device capability image as a UTF-8 encoded base 64 string. - * + * * @return the image value. */ String image(); /** * Gets the inner com.azure.resourcemanager.sphere.fluent.models.SignedCapabilityImageResponseInner object. - * + * * @return the inner object. */ SignedCapabilityImageResponseInner innerModel(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/UpdatePolicy.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/UpdatePolicy.java index 3153c79aed9a0..dfcee7e165628 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/UpdatePolicy.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/UpdatePolicy.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Update policy values. */ +/** + * Update policy values. + */ public final class UpdatePolicy extends ExpandableStringEnum { - /** Static value UpdateAll for UpdatePolicy. */ + /** + * Static value UpdateAll for UpdatePolicy. + */ public static final UpdatePolicy UPDATE_ALL = fromString("UpdateAll"); - /** Static value No3rdPartyAppUpdates for UpdatePolicy. */ + /** + * Static value No3rdPartyAppUpdates for UpdatePolicy. + */ public static final UpdatePolicy NO3RD_PARTY_APP_UPDATES = fromString("No3rdPartyAppUpdates"); /** * Creates a new instance of UpdatePolicy value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public UpdatePolicy() { /** * Creates or finds a UpdatePolicy from its string representation. - * + * * @param name a name to look for. * @return the corresponding UpdatePolicy. */ @@ -38,7 +44,7 @@ public static UpdatePolicy fromString(String name) { /** * Gets known UpdatePolicy values. - * + * * @return known UpdatePolicy values. */ public static Collection values() { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/package-info.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/package-info.java index 1434cd8dbacf7..5ee8c5416a246 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/package-info.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the data models for AzureSphereManagementClient. Azure Sphere resource management API. */ +/** + * Package containing the data models for AzureSphereMgmtClient. + * Azure Sphere resource management API. + */ package com.azure.resourcemanager.sphere.models; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/package-info.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/package-info.java index 0b51b3be880aa..229d09be942a6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/package-info.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/com/azure/resourcemanager/sphere/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for AzureSphereManagementClient. Azure Sphere resource management API. */ +/** + * Package containing the classes for AzureSphereMgmtClient. + * Azure Sphere resource management API. + */ package com.azure.resourcemanager.sphere; diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/module-info.java b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/module-info.java index bd2ed817d1ea2..e7b19257c6ccd 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/main/java/module-info.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.sphere { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.sphere; exports com.azure.resourcemanager.sphere.fluent; exports com.azure.resourcemanager.sphere.fluent.models; exports com.azure.resourcemanager.sphere.models; - - opens com.azure.resourcemanager.sphere.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.sphere.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.sphere.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.sphere.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/proxy-config.json b/sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/proxy-config.json new file mode 100644 index 0000000000000..059551aba04b7 --- /dev/null +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.sphere.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.sphere.implementation.CatalogsClientImpl$CatalogsService" ], [ "com.azure.resourcemanager.sphere.implementation.CertificatesClientImpl$CertificatesService" ], [ "com.azure.resourcemanager.sphere.implementation.ImagesClientImpl$ImagesService" ], [ "com.azure.resourcemanager.sphere.implementation.ProductsClientImpl$ProductsService" ], [ "com.azure.resourcemanager.sphere.implementation.DeviceGroupsClientImpl$DeviceGroupsService" ], [ "com.azure.resourcemanager.sphere.implementation.DeploymentsClientImpl$DeploymentsService" ], [ "com.azure.resourcemanager.sphere.implementation.DevicesClientImpl$DevicesService" ] ] \ No newline at end of file diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/reflect-config.json b/sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/reflect-config.json new file mode 100644 index 0000000000000..9bba8e24f5a1f --- /dev/null +++ b/sdk/sphere/azure-resourcemanager-sphere/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-sphere/reflect-config.json @@ -0,0 +1,266 @@ +[ { + "name" : "com.azure.resourcemanager.sphere.models.OperationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CatalogListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.CatalogInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CatalogProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CatalogUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CertificateListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.CertificateInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CertificateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.CertificateChainResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ProofOfPossessionNonceRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.ProofOfPossessionNonceResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CountElementsResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ImageListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.ImageInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ImageProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeploymentListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.DeploymentInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeploymentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ListDeviceGroupsRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceGroupListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceGroupProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.PagedDeviceInsight", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.DeviceInsightInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.DeviceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ProductListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.ProductInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ProductProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ProductUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ProductUpdateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceGroupUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceGroupUpdateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ClaimDevicesRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.DeviceUpdateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.GenerateCapabilityImageRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.fluent.models.SignedCapabilityImageResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CountDeviceResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.Origin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ActionType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CertificateStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.RegionalDataBoundary", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.ImageType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.OSFeedType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.UpdatePolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.sphere.models.CapabilityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesSamples.java index a953f4a717844..564480312993f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs CountDevices. */ +/** + * Samples for Catalogs CountDevices. + */ public final class CatalogsCountDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostCountDevicesCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostCountDevicesCatalog. + * json */ /** * Sample code: Catalogs_CountDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsCountDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateSamples.java index 66a5ddf183d95..5a3cc146ac6e0 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateSamples.java @@ -4,22 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs CreateOrUpdate. */ +/** + * Samples for Catalogs CreateOrUpdate. + */ public final class CatalogsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutCatalog.json */ /** * Sample code: Catalogs_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .define("MyCatalog1") - .withRegion("global") - .withExistingResourceGroup("MyResourceGroup1") + manager.catalogs().define("MyCatalog1").withRegion("global").withExistingResourceGroup("MyResourceGroup1") .create(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteSamples.java index 4fbc13d2ade57..c5433706b759e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs Delete. */ +/** + * Samples for Catalogs Delete. + */ public final class CatalogsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteCatalog.json */ /** * Sample code: Catalogs_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupSamples.java index 009565ecd4de5..0420cbde35826 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs GetByResourceGroup. */ +/** + * Samples for Catalogs GetByResourceGroup. + */ public final class CatalogsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCatalog.json */ /** * Sample code: Catalogs_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", com.azure.core.util.Context.NONE); + manager.catalogs().getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupSamples.java index 9a546c715b609..da2880834e187 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs ListByResourceGroup. */ +/** + * Samples for Catalogs ListByResourceGroup. + */ public final class CatalogsListByResourceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCatalogsRG.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCatalogsRG.json */ /** * Sample code: Catalogs_ListByResourceGroup. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListByResourceGroup(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsSamples.java index bffa0c8b515d4..7acbb393e10d4 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs ListDeployments. */ +/** + * Samples for Catalogs ListDeployments. + */ public final class CatalogsListDeploymentsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDeploymentsByCatalog.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostListDeploymentsByCatalog.json */ /** * Sample code: Catalogs_ListDeployments. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDeployments(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDeployments( - "MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.catalogs().listDeployments("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsSamples.java index 3876b11b2765c..f15657f10a594 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsSamples.java @@ -6,27 +6,22 @@ import com.azure.resourcemanager.sphere.models.ListDeviceGroupsRequest; -/** Samples for Catalogs ListDeviceGroups. */ +/** + * Samples for Catalogs ListDeviceGroups. + */ public final class CatalogsListDeviceGroupsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDeviceGroupsCatalog.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostListDeviceGroupsCatalog.json */ /** * Sample code: Catalogs_ListDeviceGroups. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDeviceGroups(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDeviceGroups( - "MyResourceGroup1", - "MyCatalog1", - new ListDeviceGroupsRequest().withDeviceGroupName("MyDeviceGroup1"), - null, - null, - null, - null, - com.azure.core.util.Context.NONE); + manager.catalogs().listDeviceGroups("MyResourceGroup1", "MyCatalog1", + new ListDeviceGroupsRequest().withDeviceGroupName("MyDeviceGroup1"), null, null, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsSamples.java index 79f061be11916..ef493a48a9053 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs ListDeviceInsights. */ +/** + * Samples for Catalogs ListDeviceInsights. + */ public final class CatalogsListDeviceInsightsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDeviceInsightsCatalog.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostListDeviceInsightsCatalog.json */ /** * Sample code: Catalogs_ListDeviceInsights. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDeviceInsights(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDeviceInsights( - "MyResourceGroup1", "MyCatalog1", null, 10, null, null, com.azure.core.util.Context.NONE); + manager.catalogs().listDeviceInsights("MyResourceGroup1", "MyCatalog1", null, 10, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesSamples.java index efacfe7b38595..ba19e546a2b54 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs ListDevices. */ +/** + * Samples for Catalogs ListDevices. + */ public final class CatalogsListDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostListDevicesByCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostListDevicesByCatalog. + * json */ /** * Sample code: Catalogs_ListDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .catalogs() - .listDevices("MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.catalogs().listDevices("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListSamples.java index cbca8f065c0ea..5171e9f9ee640 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Catalogs List. */ +/** + * Samples for Catalogs List. + */ public final class CatalogsListSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCatalogsSub.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCatalogsSub.json */ /** * Sample code: Catalogs_ListBySubscription. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsListBySubscription(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUpdateSamples.java index b403f511181dc..e22025f5572cc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUpdateSamples.java @@ -6,22 +6,23 @@ import com.azure.resourcemanager.sphere.models.Catalog; -/** Samples for Catalogs Update. */ +/** + * Samples for Catalogs Update. + */ public final class CatalogsUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchCatalog.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchCatalog.json */ /** * Sample code: Catalogs_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void catalogsUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - Catalog resource = - manager - .catalogs() - .getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", com.azure.core.util.Context.NONE) - .getValue(); + Catalog resource = manager.catalogs() + .getByResourceGroupWithResponse("MyResourceGroup1", "MyCatalog1", com.azure.core.util.Context.NONE) + .getValue(); resource.update().apply(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUploadImageSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUploadImageSamples.java new file mode 100644 index 0000000000000..dcc15f12a1375 --- /dev/null +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CatalogsUploadImageSamples.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.sphere.generated; + +import com.azure.resourcemanager.sphere.fluent.models.ImageInner; +import com.azure.resourcemanager.sphere.models.ImageProperties; + +/** + * Samples for Catalogs UploadImage. + */ +public final class CatalogsUploadImageSamples { + /* + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostUploadImageCatalog. + * json + */ + /** + * Sample code: Catalogs_UploadImage. + * + * @param manager Entry point to AzureSphereManager. + */ + public static void catalogsUploadImage(com.azure.resourcemanager.sphere.AzureSphereManager manager) { + manager.catalogs().uploadImage("MyResourceGroup1", "MyCatalog1", + new ImageInner().withProperties(new ImageProperties().withImage("bXliYXNlNjRzdHJpbmc=")), + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesGetSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesGetSamples.java index dd297530c5ce1..8a7d5fc255e37 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesGetSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Certificates Get. */ +/** + * Samples for Certificates Get. + */ public final class CertificatesGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCertificate.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCertificate.json */ /** * Sample code: Certificates_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void certificatesGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "default", com.azure.core.util.Context.NONE); + manager.certificates().getWithResponse("MyResourceGroup1", "MyCatalog1", "default", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogSamples.java index 5e3fb21f0a195..3d33950347e85 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Certificates ListByCatalog. */ +/** + * Samples for Certificates ListByCatalog. + */ public final class CertificatesListByCatalogSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetCertificates.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetCertificates.json */ /** * Sample code: Certificates_ListByCatalog. - * + * * @param manager Entry point to AzureSphereManager. */ public static void certificatesListByCatalog(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.certificates().listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainSamples.java index a93dae6623043..7fdfb2a053605 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Certificates RetrieveCertChain. */ +/** + * Samples for Certificates RetrieveCertChain. + */ public final class CertificatesRetrieveCertChainSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostRetrieveCatalogCertChain.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostRetrieveCatalogCertChain.json */ /** * Sample code: Certificates_RetrieveCertChain. - * + * * @param manager Entry point to AzureSphereManager. */ public static void certificatesRetrieveCertChain(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .retrieveCertChainWithResponse( - "MyResourceGroup1", "MyCatalog1", "active", com.azure.core.util.Context.NONE); + manager.certificates().retrieveCertChainWithResponse("MyResourceGroup1", "MyCatalog1", "active", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceSamples.java index e0d1984755302..f3893547209f9 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceSamples.java @@ -6,25 +6,23 @@ import com.azure.resourcemanager.sphere.models.ProofOfPossessionNonceRequest; -/** Samples for Certificates RetrieveProofOfPossessionNonce. */ +/** + * Samples for Certificates RetrieveProofOfPossessionNonce. + */ public final class CertificatesRetrieveProofOfPossessionNonceSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostRetrieveProofOfPossessionNonce.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostRetrieveProofOfPossessionNonce.json */ /** * Sample code: Certificates_RetrieveProofOfPossessionNonce. - * + * * @param manager Entry point to AzureSphereManager. */ - public static void certificatesRetrieveProofOfPossessionNonce( - com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .certificates() - .retrieveProofOfPossessionNonceWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "active", - new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("proofOfPossessionNonce"), - com.azure.core.util.Context.NONE); + public static void + certificatesRetrieveProofOfPossessionNonce(com.azure.resourcemanager.sphere.AzureSphereManager manager) { + manager.certificates().retrieveProofOfPossessionNonceWithResponse("MyResourceGroup1", "MyCatalog1", "active", + new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("proofOfPossessionNonce"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateSamples.java index f7c1b3f6f9ba2..f88e47e470a64 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateSamples.java @@ -4,21 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Deployments CreateOrUpdate. */ +/** + * Samples for Deployments CreateOrUpdate. + */ public final class DeploymentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutDeployment.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutDeployment.json */ /** * Sample code: Deployments_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .define("MyDeployment1") - .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1") - .create(); + manager.deployments().define("MyDeployment1") + .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1").create(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteSamples.java index 76bccb55eba05..72ae93fe3f1df 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteSamples.java @@ -4,25 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Deployments Delete. */ +/** + * Samples for Deployments Delete. + */ public final class DeploymentsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteDeployment.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteDeployment.json */ /** * Sample code: Deployments_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .delete( - "MyResourceGroup1", - "MyCatalog1", - "MyProductName1", - "DeviceGroupName1", - "MyDeploymentName1", - com.azure.core.util.Context.NONE); + manager.deployments().delete("MyResourceGroup1", "MyCatalog1", "MyProductName1", "DeviceGroupName1", + "MyDeploymentName1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetSamples.java index 275984e3dc927..9179cd7968628 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetSamples.java @@ -4,25 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Deployments Get. */ +/** + * Samples for Deployments Get. + */ public final class DeploymentsGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeployment.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeployment.json */ /** * Sample code: Deployments_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .getWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - "MyDeployment1", - com.azure.core.util.Context.NONE); + manager.deployments().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + "MyDeployment1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupSamples.java index 609509e0cbb9b..35084689ffd01 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupSamples.java @@ -4,28 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Deployments ListByDeviceGroup. */ +/** + * Samples for Deployments ListByDeviceGroup. + */ public final class DeploymentsListByDeviceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeployments.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeployments.json */ /** * Sample code: Deployments_ListByDeviceGroup. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deploymentsListByDeviceGroup(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deployments() - .listByDeviceGroup( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - null, - null, - null, - null, - com.azure.core.util.Context.NONE); + manager.deployments().listByDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", null, + null, null, null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsClaimDevicesSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsClaimDevicesSamples.java index 75face00b784e..e41bfdfa6b1e2 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsClaimDevicesSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsClaimDevicesSamples.java @@ -7,29 +7,23 @@ import com.azure.resourcemanager.sphere.models.ClaimDevicesRequest; import java.util.Arrays; -/** Samples for DeviceGroups ClaimDevices. */ +/** + * Samples for DeviceGroups ClaimDevices. + */ public final class DeviceGroupsClaimDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostClaimDevices.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostClaimDevices.json */ /** * Sample code: DeviceGroups_ClaimDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsClaimDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .claimDevices( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "MyDeviceGroup1", - new ClaimDevicesRequest() - .withDeviceIdentifiers( - Arrays - .asList( - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000")), - com.azure.core.util.Context.NONE); + manager.deviceGroups().claimDevices("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", + new ClaimDevicesRequest().withDeviceIdentifiers(Arrays.asList( + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000")), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesSamples.java index b8f9b52c5f86a..417723a372ecc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for DeviceGroups CountDevices. */ +/** + * Samples for DeviceGroups CountDevices. + */ public final class DeviceGroupsCountDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostCountDevicesDeviceGroup.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostCountDevicesDeviceGroup.json */ /** * Sample code: DeviceGroups_CountDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsCountDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .countDevicesWithResponse( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE); + manager.deviceGroups().countDevicesWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + "MyDeviceGroup1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateSamples.java index 7d080fbac1ffe..757c2d3de33a6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateSamples.java @@ -4,27 +4,28 @@ package com.azure.resourcemanager.sphere.generated; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.UpdatePolicy; -/** Samples for DeviceGroups CreateOrUpdate. */ +/** + * Samples for DeviceGroups CreateOrUpdate. + */ public final class DeviceGroupsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutDeviceGroup.json */ /** * Sample code: DeviceGroups_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .define("MyDeviceGroup1") + manager.deviceGroups().define("MyDeviceGroup1") .withExistingProduct("MyResourceGroup1", "MyCatalog1", "MyProduct1") - .withDescription("Description for MyDeviceGroup1") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.UPDATE_ALL) + .withProperties(new DeviceGroupProperties().withDescription("Description for MyDeviceGroup1") + .withOsFeedType(OSFeedType.RETAIL).withUpdatePolicy(UpdatePolicy.UPDATE_ALL)) .create(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteSamples.java index 8bca32f3dd902..63394eb71fc62 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for DeviceGroups Delete. */ +/** + * Samples for DeviceGroups Delete. + */ public final class DeviceGroupsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteDeviceGroup.json */ /** * Sample code: DeviceGroups_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .delete("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE); + manager.deviceGroups().delete("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetSamples.java index f670f901cc1a4..2f49c0aeb21dd 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for DeviceGroups Get. */ +/** + * Samples for DeviceGroups Get. + */ public final class DeviceGroupsGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeviceGroup.json */ /** * Sample code: DeviceGroups_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .getWithResponse( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE); + manager.deviceGroups().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductSamples.java index 58f6957cba471..830df666d347b 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductSamples.java @@ -4,27 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for DeviceGroups ListByProduct. */ +/** + * Samples for DeviceGroups ListByProduct. + */ public final class DeviceGroupsListByProductSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDeviceGroups.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDeviceGroups.json */ /** * Sample code: DeviceGroups_ListByProduct. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsListByProduct(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .deviceGroups() - .listByProduct( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - null, - null, - null, - null, - com.azure.core.util.Context.NONE); + manager.deviceGroups().listByProduct("MyResourceGroup1", "MyCatalog1", "MyProduct1", null, null, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsUpdateSamples.java index de3020b0344c9..38228e0b95184 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsUpdateSamples.java @@ -6,23 +6,22 @@ import com.azure.resourcemanager.sphere.models.DeviceGroup; -/** Samples for DeviceGroups Update. */ +/** + * Samples for DeviceGroups Update. + */ public final class DeviceGroupsUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchDeviceGroup.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchDeviceGroup.json */ /** * Sample code: DeviceGroups_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void deviceGroupsUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - DeviceGroup resource = - manager - .deviceGroups() - .getWithResponse( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "MyDeviceGroup1", com.azure.core.util.Context.NONE) - .getValue(); + DeviceGroup resource = manager.deviceGroups().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + "MyDeviceGroup1", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateSamples.java index fa3f3e079bf00..b549d4b03c751 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateSamples.java @@ -4,22 +4,22 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Devices CreateOrUpdate. */ +/** + * Samples for Devices CreateOrUpdate. + */ public final class DevicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutDevice.json */ /** * Sample code: Devices_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .define( - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000") - .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1") - .create(); + manager.devices().define( + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000") + .withExistingDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1").create(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteSamples.java index 07cb659d05313..0f05b19ae8e9d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteSamples.java @@ -4,25 +4,22 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Devices Delete. */ +/** + * Samples for Devices Delete. + */ public final class DevicesDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteDevice.json */ /** * Sample code: Devices_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .delete( - "MyResourceGroup1", - "MyCatalog1", - "MyProductName1", - "DeviceGroupName1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - com.azure.core.util.Context.NONE); + manager.devices().delete("MyResourceGroup1", "MyCatalog1", "MyProductName1", "DeviceGroupName1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageSamples.java index f1cd433d951b2..c50b6947e56e7 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageSamples.java @@ -8,27 +8,24 @@ import com.azure.resourcemanager.sphere.models.GenerateCapabilityImageRequest; import java.util.Arrays; -/** Samples for Devices GenerateCapabilityImage. */ +/** + * Samples for Devices GenerateCapabilityImage. + */ public final class DevicesGenerateCapabilityImageSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostGenerateDeviceCapabilityImage.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostGenerateDeviceCapabilityImage.json */ /** * Sample code: Devices_GenerateCapabilityImage. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesGenerateCapabilityImage(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .generateCapabilityImage( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - new GenerateCapabilityImageRequest() - .withCapabilities(Arrays.asList(CapabilityType.APPLICATION_DEVELOPMENT)), - com.azure.core.util.Context.NONE); + manager.devices().generateCapabilityImage("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + new GenerateCapabilityImageRequest().withCapabilities( + Arrays.asList(CapabilityType.APPLICATION_DEVELOPMENT)), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGetSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGetSamples.java index 6e13f61296581..d1d4264593aa4 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGetSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesGetSamples.java @@ -4,25 +4,22 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Devices Get. */ +/** + * Samples for Devices Get. + */ public final class DevicesGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDevice.json */ /** * Sample code: Devices_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .getWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "myDeviceGroup1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - com.azure.core.util.Context.NONE); + manager.devices().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupSamples.java index 0ebc32822d157..4efea809c5825 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Devices ListByDeviceGroup. */ +/** + * Samples for Devices ListByDeviceGroup. + */ public final class DevicesListByDeviceGroupSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetDevices.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetDevices.json */ /** * Sample code: Devices_ListByDeviceGroup. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesListByDeviceGroup(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .devices() - .listByDeviceGroup( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", com.azure.core.util.Context.NONE); + manager.devices().listByDeviceGroup("MyResourceGroup1", "MyCatalog1", "MyProduct1", "myDeviceGroup1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesUpdateSamples.java index fac4d4f386721..a7d30f3514f72 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/DevicesUpdateSamples.java @@ -6,28 +6,24 @@ import com.azure.resourcemanager.sphere.models.Device; -/** Samples for Devices Update. */ +/** + * Samples for Devices Update. + */ public final class DevicesUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchDevice.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchDevice.json */ /** * Sample code: Devices_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void devicesUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - Device resource = - manager - .devices() - .getWithResponse( - "MyResourceGroup1", - "MyCatalog1", - "MyProduct1", - "MyDeviceGroup1", - "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", - com.azure.core.util.Context.NONE) - .getValue(); + Device resource = manager.devices().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + "MyDeviceGroup1", + "00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000", + com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateSamples.java index 439986767c4d7..1637da43bbb6e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateSamples.java @@ -4,22 +4,24 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Images CreateOrUpdate. */ +import com.azure.resourcemanager.sphere.models.ImageProperties; + +/** + * Samples for Images CreateOrUpdate. + */ public final class ImagesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutImage.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutImage.json */ /** * Sample code: Image_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imageCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .images() - .define("default") + manager.images().define("00000000-0000-0000-0000-000000000000") .withExistingCatalog("MyResourceGroup1", "MyCatalog1") - .withImage("bXliYXNlNjRzdHJpbmc=") - .create(); + .withProperties(new ImageProperties().withImage("bXliYXNlNjRzdHJpbmc=")).create(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteSamples.java index 2b145b19ae591..ca298c7ffa88c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteSamples.java @@ -4,17 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Images Delete. */ +/** + * Samples for Images Delete. + */ public final class ImagesDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteImage.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteImage.json */ /** * Sample code: Images_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imagesDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager.images().delete("MyResourceGroup1", "MyCatalog1", "imageID", com.azure.core.util.Context.NONE); + manager.images().delete("MyResourceGroup1", "MyCatalog1", "00000000-0000-0000-0000-000000000000", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesGetSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesGetSamples.java index 6834a1b62380c..5941a50677feb 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesGetSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Images Get. */ +/** + * Samples for Images Get. + */ public final class ImagesGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetImage.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetImage.json */ /** * Sample code: Images_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imagesGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .images() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "myImageId", com.azure.core.util.Context.NONE); + manager.images().getWithResponse("MyResourceGroup1", "MyCatalog1", "00000000-0000-0000-0000-000000000000", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogSamples.java index 3df520e62c0ae..d3afc98bbd240 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Images ListByCatalog. */ +/** + * Samples for Images ListByCatalog. + */ public final class ImagesListByCatalogSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetImages.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetImages.json */ /** * Sample code: Images_ListByCatalog. - * + * * @param manager Entry point to AzureSphereManager. */ public static void imagesListByCatalog(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .images() - .listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, com.azure.core.util.Context.NONE); + manager.images().listByCatalog("MyResourceGroup1", "MyCatalog1", null, null, null, null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/OperationsListSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/OperationsListSamples.java index 1644d16ecb87b..a11da63ebadb2 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/OperationsListSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/OperationsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetOperations.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetOperations.json */ /** * Sample code: Operations_List. - * + * * @param manager Entry point to AzureSphereManager. */ public static void operationsList(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesSamples.java index c2fbb80de7d03..0929cda53110b 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Products CountDevices. */ +/** + * Samples for Products CountDevices. + */ public final class ProductsCountDevicesSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostCountDevicesProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PostCountDevicesProduct. + * json */ /** * Sample code: Products_CountDevices. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsCountDevices(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .products() - .countDevicesWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE); + manager.products().countDevicesWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateSamples.java index f4b464e1d8b56..c487141ce71cc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Products CreateOrUpdate. */ +/** + * Samples for Products CreateOrUpdate. + */ public final class ProductsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PutProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PutProduct.json */ /** * Sample code: Products_CreateOrUpdate. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsCreateOrUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteSamples.java index bf1b143a5ec91..b8664ba9eaf4c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Products Delete. */ +/** + * Samples for Products Delete. + */ public final class ProductsDeleteSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/DeleteProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/DeleteProduct.json */ /** * Sample code: Products_Delete. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsDelete(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsSamples.java index dea33d91b4257..ed749735aac97 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsSamples.java @@ -4,21 +4,22 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Products GenerateDefaultDeviceGroups. */ +/** + * Samples for Products GenerateDefaultDeviceGroups. + */ public final class ProductsGenerateDefaultDeviceGroupsSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PostGenerateDefaultDeviceGroups.json + * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/ + * PostGenerateDefaultDeviceGroups.json */ /** * Sample code: Products_GenerateDefaultDeviceGroups. - * + * * @param manager Entry point to AzureSphereManager. */ - public static void productsGenerateDefaultDeviceGroups( - com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .products() - .generateDefaultDeviceGroups( - "MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE); + public static void + productsGenerateDefaultDeviceGroups(com.azure.resourcemanager.sphere.AzureSphereManager manager) { + manager.products().generateDefaultDeviceGroups("MyResourceGroup1", "MyCatalog1", "MyProduct1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGetSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGetSamples.java index 0adf122c027f1..02d84879b5931 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGetSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Products Get. */ +/** + * Samples for Products Get. + */ public final class ProductsGetSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetProduct.json */ /** * Sample code: Products_Get. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsGet(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - manager - .products() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE); + manager.products().getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogSamples.java index 77d7f7e2b1062..0518ce64f8725 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.sphere.generated; -/** Samples for Products ListByCatalog. */ +/** + * Samples for Products ListByCatalog. + */ public final class ProductsListByCatalogSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/GetProducts.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/GetProducts.json */ /** * Sample code: Products_ListByCatalog. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsListByCatalog(com.azure.resourcemanager.sphere.AzureSphereManager manager) { diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsUpdateSamples.java b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsUpdateSamples.java index 96e2a0ca437ae..58b204a769434 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsUpdateSamples.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/samples/java/com/azure/resourcemanager/sphere/generated/ProductsUpdateSamples.java @@ -6,22 +6,23 @@ import com.azure.resourcemanager.sphere.models.Product; -/** Samples for Products Update. */ +/** + * Samples for Products Update. + */ public final class ProductsUpdateSamples { /* - * x-ms-original-file: specification/sphere/resource-manager/Microsoft.AzureSphere/preview/2022-09-01-preview/examples/PatchProduct.json + * x-ms-original-file: + * specification/sphere/resource-manager/Microsoft.AzureSphere/stable/2024-04-01/examples/PatchProduct.json */ /** * Sample code: Products_Update. - * + * * @param manager Entry point to AzureSphereManager. */ public static void productsUpdate(com.azure.resourcemanager.sphere.AzureSphereManager manager) { - Product resource = - manager - .products() - .getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE) - .getValue(); + Product resource = manager.products() + .getWithResponse("MyResourceGroup1", "MyCatalog1", "MyProduct1", com.azure.core.util.Context.NONE) + .getValue(); resource.update().apply(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogInnerTests.java index 6962083bbbef2..f5860910910d6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogInnerTests.java @@ -6,6 +6,7 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; +import com.azure.resourcemanager.sphere.models.CatalogProperties; import java.util.HashMap; import java.util.Map; import org.junit.jupiter.api.Assertions; @@ -13,35 +14,24 @@ public final class CatalogInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CatalogInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\"},\"location\":\"lokjyemkk\",\"tags\":{\"ejspodmail\":\"pjoxzjnch\",\"yahux\":\"ydehoj\",\"vcputegj\":\"npmqnjaqwixjspro\",\"uuvmkjozkrwfnd\":\"wmfdatscmdvpjhul\"},\"id\":\"odjpslwejd\",\"name\":\"vwryoqpso\",\"type\":\"cctazakljlahbc\"}") - .toObject(CatalogInner.class); - Assertions.assertEquals("lokjyemkk", model.location()); - Assertions.assertEquals("pjoxzjnch", model.tags().get("ejspodmail")); + CatalogInner model = BinaryData.fromString( + "{\"properties\":{\"tenantId\":\"vpjhulsuuv\",\"provisioningState\":\"Failed\"},\"location\":\"zkrwfn\",\"tags\":{\"jdpvwryo\":\"djpslw\",\"hbcryffdfdosyge\":\"psoacctazakljl\",\"rzevdphlxaol\":\"paojakhmsbzjh\"},\"id\":\"hqtrgqjbpf\",\"name\":\"fsinzgvfcjrwzoxx\",\"type\":\"tfell\"}") + .toObject(CatalogInner.class); + Assertions.assertEquals("zkrwfn", model.location()); + Assertions.assertEquals("djpslw", model.tags().get("jdpvwryo")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CatalogInner model = - new CatalogInner() - .withLocation("lokjyemkk") - .withTags( - mapOf( - "ejspodmail", - "pjoxzjnch", - "yahux", - "ydehoj", - "vcputegj", - "npmqnjaqwixjspro", - "uuvmkjozkrwfnd", - "wmfdatscmdvpjhul")); + CatalogInner model = new CatalogInner().withLocation("zkrwfn") + .withTags(mapOf("jdpvwryo", "djpslw", "hbcryffdfdosyge", "psoacctazakljl", "rzevdphlxaol", "paojakhmsbzjh")) + .withProperties(new CatalogProperties()); model = BinaryData.fromObject(model).toObject(CatalogInner.class); - Assertions.assertEquals("lokjyemkk", model.location()); - Assertions.assertEquals("pjoxzjnch", model.tags().get("ejspodmail")); + Assertions.assertEquals("zkrwfn", model.location()); + Assertions.assertEquals("djpslw", model.tags().get("jdpvwryo")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogListResultTests.java index a8865709acb79..56fd7b612858e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogListResultTests.java @@ -7,6 +7,7 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.CatalogInner; import com.azure.resourcemanager.sphere.models.CatalogListResult; +import com.azure.resourcemanager.sphere.models.CatalogProperties; import java.util.Arrays; import java.util.HashMap; import java.util.Map; @@ -15,33 +16,28 @@ public final class CatalogListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CatalogListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Updating\"},\"location\":\"hxqh\",\"tags\":{\"scnpqxuhivy\":\"fpikxwczb\",\"wby\":\"n\",\"grtfwvu\":\"rkxvdum\"},\"id\":\"xgaudccs\",\"name\":\"h\",\"type\":\"jcny\"}],\"nextLink\":\"hkryhtn\"}") - .toObject(CatalogListResult.class); - Assertions.assertEquals("hxqh", model.value().get(0).location()); - Assertions.assertEquals("fpikxwczb", model.value().get(0).tags().get("scnpqxuhivy")); - Assertions.assertEquals("hkryhtn", model.nextLink()); + CatalogListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"tenantId\":\"ithxqhabifpi\",\"provisioningState\":\"Accepted\"},\"location\":\"zb\",\"tags\":{\"vyq\":\"npqxuh\",\"tfwvukxgaudc\":\"iwbybrkxvdumjg\",\"napczwlokjy\":\"snhsjcnyejhkryh\",\"oxzjnchgejspod\":\"mkkvnip\"},\"id\":\"ailzydehojwyahu\",\"name\":\"inpm\",\"type\":\"njaqwixjspro\"}],\"nextLink\":\"cputegjvwmfdats\"}") + .toObject(CatalogListResult.class); + Assertions.assertEquals("zb", model.value().get(0).location()); + Assertions.assertEquals("npqxuh", model.value().get(0).tags().get("vyq")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CatalogListResult model = - new CatalogListResult() + CatalogListResult model + = new CatalogListResult() .withValue( - Arrays - .asList( - new CatalogInner() - .withLocation("hxqh") - .withTags(mapOf("scnpqxuhivy", "fpikxwczb", "wby", "n", "grtfwvu", "rkxvdum")))) - .withNextLink("hkryhtn"); + Arrays.asList(new CatalogInner() + .withLocation("zb").withTags(mapOf("vyq", "npqxuh", "tfwvukxgaudc", "iwbybrkxvdumjg", + "napczwlokjy", "snhsjcnyejhkryh", "oxzjnchgejspod", "mkkvnip")) + .withProperties(new CatalogProperties()))); model = BinaryData.fromObject(model).toObject(CatalogListResult.class); - Assertions.assertEquals("hxqh", model.value().get(0).location()); - Assertions.assertEquals("fpikxwczb", model.value().get(0).tags().get("scnpqxuhivy")); - Assertions.assertEquals("hkryhtn", model.nextLink()); + Assertions.assertEquals("zb", model.value().get(0).location()); + Assertions.assertEquals("npqxuh", model.value().get(0).tags().get("vyq")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogPropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogPropertiesTests.java index 92506c4b6bb33..6b9c7983e55ae 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogPropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogPropertiesTests.java @@ -5,13 +5,13 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.CatalogProperties; +import com.azure.resourcemanager.sphere.models.CatalogProperties; public final class CatalogPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CatalogProperties model = - BinaryData.fromString("{\"provisioningState\":\"Provisioning\"}").toObject(CatalogProperties.class); + CatalogProperties model = BinaryData.fromString("{\"tenantId\":\"fziton\",\"provisioningState\":\"Failed\"}") + .toObject(CatalogProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogUpdateTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogUpdateTests.java index a1076b1954073..94d41f62fe14a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogUpdateTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogUpdateTests.java @@ -13,23 +13,22 @@ public final class CatalogUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CatalogUpdate model = - BinaryData - .fromString( - "{\"tags\":{\"xpaojakhmsbz\":\"fdosyg\",\"hlxaolthqtr\":\"hcrzevd\",\"gvfcj\":\"qjbpfzfsin\"}}") - .toObject(CatalogUpdate.class); - Assertions.assertEquals("fdosyg", model.tags().get("xpaojakhmsbz")); + CatalogUpdate model = BinaryData + .fromString( + "{\"tags\":{\"ypininm\":\"jkjlxofpdvhpfx\",\"po\":\"yhuybbkpod\",\"ognarxzxtheotus\":\"ginuvamih\"}}") + .toObject(CatalogUpdate.class); + Assertions.assertEquals("jkjlxofpdvhpfx", model.tags().get("ypininm")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CatalogUpdate model = - new CatalogUpdate() - .withTags(mapOf("xpaojakhmsbz", "fdosyg", "hlxaolthqtr", "hcrzevd", "gvfcj", "qjbpfzfsin")); + CatalogUpdate model = new CatalogUpdate() + .withTags(mapOf("ypininm", "jkjlxofpdvhpfx", "po", "yhuybbkpod", "ognarxzxtheotus", "ginuvamih")); model = BinaryData.fromObject(model).toObject(CatalogUpdate.class); - Assertions.assertEquals("fdosyg", model.tags().get("xpaojakhmsbz")); + Assertions.assertEquals("jkjlxofpdvhpfx", model.tags().get("ypininm")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesWithResponseMockTests.java index b65131b0794fd..819919f6531dc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCountDevicesWithResponseMockTests.java @@ -12,7 +12,7 @@ import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -30,37 +30,26 @@ public void testCountDevicesWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"value\":427278595}"; + String responseStr = "{\"value\":1560245881}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - CountDeviceResponse response = - manager.catalogs().countDevicesWithResponse("bkyvp", "ca", com.azure.core.util.Context.NONE).getValue(); + CountDevicesResponse response = manager.catalogs() + .countDevicesWithResponse("cjooxdjebwpucwwf", "ovbvmeueciv", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(427278595, response.value()); + Assertions.assertEquals(1560245881, response.value()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateMockTests.java index 8b2d78d783367..d685320695650 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsCreateOrUpdateMockTests.java @@ -13,6 +13,7 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; import com.azure.resourcemanager.sphere.models.Catalog; +import com.azure.resourcemanager.sphere.models.CatalogProperties; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -32,48 +33,34 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\"},\"location\":\"lmpewwwfbkr\",\"tags\":{\"rsbfovasrruvw\":\"svshqjohx\"},\"id\":\"hsqfsubcgjbirxbp\",\"name\":\"bsrfbj\",\"type\":\"dtws\"}"; + String responseStr + = "{\"properties\":{\"tenantId\":\"yngudivk\",\"provisioningState\":\"Succeeded\"},\"location\":\"bxqz\",\"tags\":{\"e\":\"jfauvjfdxxi\"},\"id\":\"vtcqaqtdo\",\"name\":\"mcbxvwvxysl\",\"type\":\"bhsfxob\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Catalog response = - manager - .catalogs() - .define("mjh") - .withRegion("gudivkrtswbxqz") - .withExistingResourceGroup("rmfqjhhkxbpvj") - .withTags(mapOf("e", "jfauvjfdxxi")) - .create(); + Catalog response = manager.catalogs().define("zhxgktrmgucn").withRegion("lwptfdy") + .withExistingResourceGroup("medjvcslynqwwncw") + .withTags(mapOf("huaoppp", "qbuaceopzfqr", "z", "qeqxo", "moizpos", "ahzxctobgbk")) + .withProperties(new CatalogProperties()).create(); - Assertions.assertEquals("lmpewwwfbkr", response.location()); - Assertions.assertEquals("svshqjohx", response.tags().get("rsbfovasrruvw")); + Assertions.assertEquals("bxqz", response.location()); + Assertions.assertEquals("jfauvjfdxxi", response.tags().get("e")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteMockTests.java index a43539e369588..e9bba9bed6403 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.catalogs().delete("hdzhlrqj", "hckfrlhrx", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.catalogs().delete("hashsfwxosow", "xcug", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupWithResponseMockTests.java index be9d6977414b4..a5ec592630687 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsGetByResourceGroupWithResponseMockTests.java @@ -30,39 +30,28 @@ public void testGetByResourceGroupWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Accepted\"},\"location\":\"gxywpmue\",\"tags\":{\"bglaocqxtccm\":\"zwfqkqujidsuyon\"},\"id\":\"yudxytlmoy\",\"name\":\"xv\",\"type\":\"fudwpznt\"}"; + String responseStr + = "{\"properties\":{\"tenantId\":\"ag\",\"provisioningState\":\"Accepted\"},\"location\":\"elmqk\",\"tags\":{\"dhmdua\":\"hvljuahaquh\",\"pvfadmwsrcr\":\"aex\",\"fmisg\":\"vxpvgomz\"},\"id\":\"bnbbeldawkz\",\"name\":\"ali\",\"type\":\"urqhaka\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Catalog response = - manager.catalogs().getByResourceGroupWithResponse("ye", "sbj", com.azure.core.util.Context.NONE).getValue(); + Catalog response = manager.catalogs() + .getByResourceGroupWithResponse("ehhseyvjusrts", "hspkdeemao", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("gxywpmue", response.location()); - Assertions.assertEquals("zwfqkqujidsuyon", response.tags().get("bglaocqxtccm")); + Assertions.assertEquals("elmqk", response.location()); + Assertions.assertEquals("hvljuahaquh", response.tags().get("dhmdua")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupMockTests.java index 4c933ee635d93..0a3320a64b285 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListByResourceGroupMockTests.java @@ -31,39 +31,27 @@ public void testListByResourceGroup() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\"},\"location\":\"bkc\",\"tags\":{\"nv\":\"hbttkphyw\"},\"id\":\"t\",\"name\":\"qnermclfplphoxu\",\"type\":\"crpab\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"tenantId\":\"xy\",\"provisioningState\":\"Deleting\"},\"location\":\"yrxvwfudwpznt\",\"tags\":{\"ck\":\"zhlrqjb\",\"kyv\":\"rlhrxs\"},\"id\":\"ycanuzbpzkafku\",\"name\":\"b\",\"type\":\"rnwb\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.catalogs().listByResourceGroup("kvceoveilovnotyf", com.azure.core.util.Context.NONE); + PagedIterable response = manager.catalogs().listByResourceGroup("g", com.azure.core.util.Context.NONE); - Assertions.assertEquals("bkc", response.iterator().next().location()); - Assertions.assertEquals("hbttkphyw", response.iterator().next().tags().get("nv")); + Assertions.assertEquals("yrxvwfudwpznt", response.iterator().next().location()); + Assertions.assertEquals("zhlrqjb", response.iterator().next().tags().get("ck")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsMockTests.java index cb6c4fbc61b39..7f22830d823b9 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeploymentsMockTests.java @@ -32,46 +32,33 @@ public void testListDeployments() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"deploymentId\":\"v\",\"deployedImages\":[{\"properties\":{\"image\":\"slhs\",\"imageId\":\"deemao\",\"imageName\":\"xagkvtmelmqkrh\",\"regionalDataBoundary\":\"None\",\"uri\":\"juahaquhcdhmdual\",\"description\":\"xqpvfadmw\",\"componentId\":\"crgvxpvgom\",\"imageType\":\"WifiFirmware\",\"provisioningState\":\"Canceled\"},\"id\":\"sgwbnbbeld\",\"name\":\"wkz\",\"type\":\"ali\"},{\"properties\":{\"image\":\"qhakauhashsf\",\"imageId\":\"osow\",\"imageName\":\"cugicjoox\",\"regionalDataBoundary\":\"EU\",\"uri\":\"wpucwwfvovbv\",\"description\":\"uecivyhz\",\"componentId\":\"uojgj\",\"imageType\":\"SecurityMonitor\",\"provisioningState\":\"Succeeded\"},\"id\":\"iotwmcdytdxwit\",\"name\":\"nrjawgqwg\",\"type\":\"hniskxfbkpyc\"},{\"properties\":{\"image\":\"wndnhj\",\"imageId\":\"uwhvylwzbtdhxujz\",\"imageName\":\"mpowuwpr\",\"regionalDataBoundary\":\"None\",\"uri\":\"eualupjmkhf\",\"description\":\"bbcswsrtjri\",\"componentId\":\"rbpbewtghfgblcg\",\"imageType\":\"OneBl\",\"provisioningState\":\"Canceled\"},\"id\":\"v\",\"name\":\"hjkbegibtnmxieb\",\"type\":\"waloayqcgwr\"},{\"properties\":{\"image\":\"uzgwyzmhtx\",\"imageId\":\"gmtsavjcbpwxqpsr\",\"imageName\":\"ftguv\",\"regionalDataBoundary\":\"EU\",\"uri\":\"prwmdyvxqt\",\"description\":\"riwwroy\",\"componentId\":\"exrmcqibycnojvk\",\"imageType\":\"PlutonRuntime\",\"provisioningState\":\"Updating\"},\"id\":\"sgzvahapjyzhpv\",\"name\":\"qzcjrvxdj\",\"type\":\"lmwlxkvugfhzo\"}],\"deploymentDateUtc\":\"2021-09-30T03:16:16Z\",\"provisioningState\":\"Canceled\"},\"id\":\"zunlu\",\"name\":\"hnnpr\",\"type\":\"xipeilpjzuaejx\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"deploymentId\":\"jawgqwg\",\"deployedImages\":[{\"properties\":{\"image\":\"kxfbkpycgklwndn\",\"imageId\":\"dauwhvylwzbtd\",\"imageName\":\"ujznb\",\"regionalDataBoundary\":\"None\",\"uri\":\"uwprzql\",\"description\":\"ualupjmkh\",\"componentId\":\"obbc\",\"imageType\":\"CustomerBoardConfig\",\"provisioningState\":\"Updating\"},\"id\":\"riplrbpbewtg\",\"name\":\"fgb\",\"type\":\"c\"}],\"deploymentDateUtc\":\"2021-02-14T22:48:14Z\",\"provisioningState\":\"Accepted\"},\"id\":\"v\",\"name\":\"hjkbegibtnmxieb\",\"type\":\"waloayqcgwr\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .catalogs() - .listDeployments( - "z", "p", "kafkuwbcrnwbm", 1542253292, 1688496747, 1198251199, com.azure.core.util.Context.NONE); + PagedIterable response = manager.catalogs().listDeployments("zceuojgjrw", "ueiotwmcdyt", "x", + 384724385, 1665373545, 838785311, com.azure.core.util.Context.NONE); - Assertions.assertEquals("v", response.iterator().next().deploymentId()); - Assertions.assertEquals("slhs", response.iterator().next().deployedImages().get(0).image()); - Assertions.assertEquals("deemao", response.iterator().next().deployedImages().get(0).imageId()); - Assertions - .assertEquals( - RegionalDataBoundary.NONE, response.iterator().next().deployedImages().get(0).regionalDataBoundary()); + Assertions.assertEquals("jawgqwg", response.iterator().next().properties().deploymentId()); + Assertions.assertEquals("kxfbkpycgklwndn", + response.iterator().next().properties().deployedImages().get(0).properties().image()); + Assertions.assertEquals("dauwhvylwzbtd", + response.iterator().next().properties().deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + response.iterator().next().properties().deployedImages().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsMockTests.java index 5d57cacf9ab98..cba15c9b49563 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceGroupsMockTests.java @@ -36,53 +36,35 @@ public void testListDeviceGroups() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"description\":\"luu\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":false,\"provisioningState\":\"Updating\"},\"id\":\"v\",\"name\":\"elnsmvbxw\",\"type\":\"jsflhhcaalnjix\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"description\":\"iwwroyqbexrmc\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Accepted\"},\"id\":\"qsgzvahapj\",\"name\":\"zhpvgqzcjrvxd\",\"type\":\"zlmwlxkvugfhz\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .catalogs() - .listDeviceGroups( - "ultskzbbtdz", - "mv", - new ListDeviceGroupsRequest().withDeviceGroupName("kgpwoz"), - "hkfpbs", - 820009390, - 657936288, - 205303620, - com.azure.core.util.Context.NONE); + PagedIterable response = manager.catalogs().listDeviceGroups("zjuzgwyz", "htxongmtsavjc", + new ListDeviceGroupsRequest().withDeviceGroupName("wxqpsrknftguvri"), "hprwmdyv", 1468047146, 1392355180, + 1817218794, com.azure.core.util.Context.NONE); - Assertions.assertEquals("luu", response.iterator().next().description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.iterator().next().osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, response.iterator().next().updatePolicy()); - Assertions - .assertEquals(AllowCrashDumpCollection.ENABLED, response.iterator().next().allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, response.iterator().next().regionalDataBoundary()); + Assertions.assertEquals("iwwroyqbexrmc", response.iterator().next().properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.iterator().next().properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, + response.iterator().next().properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, + response.iterator().next().properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + response.iterator().next().properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsMockTests.java index 924deb5c4dad5..210566f5ad09c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDeviceInsightsMockTests.java @@ -31,56 +31,36 @@ public void testListDeviceInsights() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"deviceId\":\"ztfolhbnxk\",\"description\":\"alaulppggdtpnapn\",\"startTimestampUtc\":\"2021-01-27T00:10:34Z\",\"endTimestampUtc\":\"2021-04-06T21:45:28Z\",\"eventCategory\":\"opuhpig\",\"eventClass\":\"pgylg\",\"eventType\":\"git\",\"eventCount\":1973221194}]}"; + String responseStr + = "{\"value\":[{\"deviceId\":\"skzbb\",\"description\":\"dzumveekg\",\"startTimestampUtc\":\"2021-02-08T12:29:03Z\",\"endTimestampUtc\":\"2021-07-01T08:04:50Z\",\"eventCategory\":\"zuhkfpbsjyof\",\"eventClass\":\"xl\",\"eventType\":\"us\",\"eventCount\":1856979158}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .catalogs() - .listDeviceInsights( - "sxyawjoyaqcs", - "yjpkiidzyexz", - "eli", - 1911330715, - 462425094, - 705550222, - com.azure.core.util.Context.NONE); + PagedIterable response = manager.catalogs().listDeviceInsights("vawjvzunlu", "hnnpr", + "xipeilpjzuaejx", 677451354, 122222845, 1732418843, com.azure.core.util.Context.NONE); - Assertions.assertEquals("ztfolhbnxk", response.iterator().next().deviceId()); - Assertions.assertEquals("alaulppggdtpnapn", response.iterator().next().description()); - Assertions - .assertEquals(OffsetDateTime.parse("2021-01-27T00:10:34Z"), response.iterator().next().startTimestampUtc()); - Assertions - .assertEquals(OffsetDateTime.parse("2021-04-06T21:45:28Z"), response.iterator().next().endTimestampUtc()); - Assertions.assertEquals("opuhpig", response.iterator().next().eventCategory()); - Assertions.assertEquals("pgylg", response.iterator().next().eventClass()); - Assertions.assertEquals("git", response.iterator().next().eventType()); - Assertions.assertEquals(1973221194, response.iterator().next().eventCount()); + Assertions.assertEquals("skzbb", response.iterator().next().deviceId()); + Assertions.assertEquals("dzumveekg", response.iterator().next().description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-08T12:29:03Z"), + response.iterator().next().startTimestampUtc()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-01T08:04:50Z"), + response.iterator().next().endTimestampUtc()); + Assertions.assertEquals("zuhkfpbsjyof", response.iterator().next().eventCategory()); + Assertions.assertEquals("xl", response.iterator().next().eventClass()); + Assertions.assertEquals("us", response.iterator().next().eventType()); + Assertions.assertEquals(1856979158, response.iterator().next().eventCount()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesMockTests.java index cbd33be295984..77c223d9ca877 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListDevicesMockTests.java @@ -31,47 +31,27 @@ public void testListDevices() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"deviceId\":\"ellwptfdy\",\"chipSku\":\"fqbuaceopzf\",\"lastAvailableOsVersion\":\"hhuao\",\"lastInstalledOsVersion\":\"pcqeqx\",\"lastOsUpdateUtc\":\"2021-02-19T08:33:13Z\",\"lastUpdateRequestUtc\":\"2021-12-05T23:59:55Z\",\"provisioningState\":\"Failed\"},\"id\":\"xcto\",\"name\":\"gbkdmoizpos\",\"type\":\"mgrcfbu\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"deviceId\":\"joya\",\"chipSku\":\"slyjpkiid\",\"lastAvailableOsVersion\":\"exznelixhnr\",\"lastInstalledOsVersion\":\"folhbnxknal\",\"lastOsUpdateUtc\":\"2021-06-16T03:28:56Z\",\"lastUpdateRequestUtc\":\"2021-05-05T13:18:41Z\",\"provisioningState\":\"Deleting\"},\"id\":\"tpnapnyiropuhpig\",\"name\":\"pgylg\",\"type\":\"git\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .catalogs() - .listDevices( - "edjvcslynqw", - "ncw", - "zhxgktrmgucn", - 1295716293, - 994320059, - 47772472, - com.azure.core.util.Context.NONE); + PagedIterable response = manager.catalogs().listDevices("touwaboekqv", "elnsmvbxw", "jsflhhcaalnjix", + 923639125, 1143509001, 589324262, com.azure.core.util.Context.NONE); - Assertions.assertEquals("ellwptfdy", response.iterator().next().deviceId()); + Assertions.assertEquals("joya", response.iterator().next().properties().deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListMockTests.java index ed17d3f62a22c..8784e4046ca01 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CatalogsListMockTests.java @@ -31,38 +31,27 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\"},\"location\":\"kde\",\"tags\":{\"kdwzbaiuebbaumny\":\"vlopwiyighx\",\"txp\":\"upedeojnabckhs\",\"tfhvpesapskrdqmh\":\"ie\",\"tkncwsc\":\"jdhtldwkyzxu\"},\"id\":\"svlxotogtwrup\",\"name\":\"sx\",\"type\":\"nmic\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"tenantId\":\"nxdhbt\",\"provisioningState\":\"Provisioning\"},\"location\":\"ywpnvjt\",\"tags\":{\"abgy\":\"ermclfplphoxuscr\",\"qugxywpmueefjzwf\":\"psbjta\"},\"id\":\"kqujidsuyono\",\"name\":\"glaocq\",\"type\":\"tcc\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.catalogs().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("kde", response.iterator().next().location()); - Assertions.assertEquals("vlopwiyighx", response.iterator().next().tags().get("kdwzbaiuebbaumny")); + Assertions.assertEquals("ywpnvjt", response.iterator().next().location()); + Assertions.assertEquals("ermclfplphoxuscr", response.iterator().next().tags().get("abgy")); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateChainResponseInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateChainResponseInnerTests.java index 20eb83b8c343b..8f4a231aa5546 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateChainResponseInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateChainResponseInnerTests.java @@ -10,10 +10,8 @@ public final class CertificateChainResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CertificateChainResponseInner model = - BinaryData - .fromString("{\"certificateChain\":\"uskcqvkocrcj\"}") - .toObject(CertificateChainResponseInner.class); + CertificateChainResponseInner model + = BinaryData.fromString("{\"certificateChain\":\"cph\"}").toObject(CertificateChainResponseInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateInnerTests.java index 6754027061f50..9b8b53b83170c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateInnerTests.java @@ -6,20 +6,19 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.CertificateInner; +import com.azure.resourcemanager.sphere.models.CertificateProperties; public final class CertificateInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CertificateInner model = - BinaryData - .fromString( - "{\"properties\":{\"certificate\":\"eotusivyevc\",\"status\":\"Active\",\"subject\":\"hn\",\"thumbprint\":\"ngbwjz\",\"expiryUtc\":\"2020-12-28T13:19:57Z\",\"notBeforeUtc\":\"2021-11-04T09:44:48Z\",\"provisioningState\":\"Failed\"},\"id\":\"spemvtzfk\",\"name\":\"fublj\",\"type\":\"fxqeof\"}") - .toObject(CertificateInner.class); + CertificateInner model = BinaryData.fromString( + "{\"properties\":{\"certificate\":\"oc\",\"status\":\"Inactive\",\"subject\":\"blgphuticn\",\"thumbprint\":\"kao\",\"expiryUtc\":\"2021-10-08T07:48:08Z\",\"notBeforeUtc\":\"2021-02-22T11:39:16Z\",\"provisioningState\":\"Updating\"},\"id\":\"xhurok\",\"name\":\"tyxolniwpwc\",\"type\":\"kjfkg\"}") + .toObject(CertificateInner.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CertificateInner model = new CertificateInner(); + CertificateInner model = new CertificateInner().withProperties(new CertificateProperties()); model = BinaryData.fromObject(model).toObject(CertificateInner.class); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateListResultTests.java index 7446a0f2cf621..a0c05cbbd6f7a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificateListResultTests.java @@ -7,25 +7,23 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.CertificateInner; import com.azure.resourcemanager.sphere.models.CertificateListResult; +import com.azure.resourcemanager.sphere.models.CertificateProperties; import java.util.Arrays; -import org.junit.jupiter.api.Assertions; public final class CertificateListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CertificateListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"certificate\":\"xxjtfe\",\"status\":\"Expired\",\"subject\":\"fziton\",\"thumbprint\":\"qfpjk\",\"expiryUtc\":\"2021-05-09T13:58:23Z\",\"notBeforeUtc\":\"2021-11-24T18:52:18Z\",\"provisioningState\":\"Failed\"},\"id\":\"hpf\",\"name\":\"xypininmayhuybbk\",\"type\":\"odepoogin\"}],\"nextLink\":\"amiheognarxz\"}") - .toObject(CertificateListResult.class); - Assertions.assertEquals("amiheognarxz", model.nextLink()); + CertificateListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"certificate\":\"v\",\"status\":\"Inactive\",\"subject\":\"ihnhun\",\"thumbprint\":\"wjzrnfygxgisp\",\"expiryUtc\":\"2021-05-03T11:09:18Z\",\"notBeforeUtc\":\"2021-01-12T09:11:57Z\",\"provisioningState\":\"Accepted\"},\"id\":\"fublj\",\"name\":\"fxqeof\",\"type\":\"aeqjhqjbasvms\"},{\"properties\":{\"certificate\":\"ulngsntn\",\"status\":\"Expired\",\"subject\":\"zgcwrw\",\"thumbprint\":\"xxwr\",\"expiryUtc\":\"2021-07-19T21:40:35Z\",\"notBeforeUtc\":\"2021-04-09T18:31:52Z\",\"provisioningState\":\"Canceled\"},\"id\":\"qvkoc\",\"name\":\"cjdkwtnhxbnjbi\",\"type\":\"sqrglssainq\"},{\"properties\":{\"certificate\":\"nzl\",\"status\":\"Inactive\",\"subject\":\"ppeebvmgxsab\",\"thumbprint\":\"qduujitcjczdz\",\"expiryUtc\":\"2021-11-12T14:41:45Z\",\"notBeforeUtc\":\"2021-05-08T08:52:13Z\",\"provisioningState\":\"Deleting\"},\"id\":\"pdappds\",\"name\":\"dkvwrwjfe\",\"type\":\"snhu\"}],\"nextLink\":\"eltmrldhugjzzdat\"}") + .toObject(CertificateListResult.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CertificateListResult model = - new CertificateListResult().withValue(Arrays.asList(new CertificateInner())).withNextLink("amiheognarxz"); + CertificateListResult model = new CertificateListResult() + .withValue(Arrays.asList(new CertificateInner().withProperties(new CertificateProperties()), + new CertificateInner().withProperties(new CertificateProperties()), + new CertificateInner().withProperties(new CertificateProperties()))); model = BinaryData.fromObject(model).toObject(CertificateListResult.class); - Assertions.assertEquals("amiheognarxz", model.nextLink()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatePropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatePropertiesTests.java index 466e9b7ffa5b1..c49965d7ba157 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatePropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatePropertiesTests.java @@ -5,16 +5,14 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.CertificateProperties; +import com.azure.resourcemanager.sphere.models.CertificateProperties; public final class CertificatePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CertificateProperties model = - BinaryData - .fromString( - "{\"certificate\":\"e\",\"status\":\"Expired\",\"subject\":\"jbasvmsmjqulngs\",\"thumbprint\":\"nbybkzgcwrwcl\",\"expiryUtc\":\"2021-06-21T13:48:53Z\",\"notBeforeUtc\":\"2021-01-30T10:40:21Z\",\"provisioningState\":\"Canceled\"}") - .toObject(CertificateProperties.class); + CertificateProperties model = BinaryData.fromString( + "{\"certificate\":\"w\",\"status\":\"Revoked\",\"subject\":\"ypl\",\"thumbprint\":\"kbasyypn\",\"expiryUtc\":\"2021-11-02T07:55:59Z\",\"notBeforeUtc\":\"2021-11-25T20:26:44Z\",\"provisioningState\":\"Accepted\"}") + .toObject(CertificateProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesGetWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesGetWithResponseMockTests.java index c4b74f5c71434..5fb49d726ef15 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesGetWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesGetWithResponseMockTests.java @@ -29,39 +29,26 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"certificate\":\"d\",\"status\":\"Inactive\",\"subject\":\"hzdxssadbzm\",\"thumbprint\":\"dfznudaodv\",\"expiryUtc\":\"2021-05-29T13:49:41Z\",\"notBeforeUtc\":\"2021-08-27T15:54:03Z\",\"provisioningState\":\"Provisioning\"},\"id\":\"lpstdbhhxsrzdz\",\"name\":\"cers\",\"type\":\"dntnevf\"}"; + String responseStr + = "{\"properties\":{\"certificate\":\"fthnzdn\",\"status\":\"Active\",\"subject\":\"nayqi\",\"thumbprint\":\"nduhavhqlkthum\",\"expiryUtc\":\"2021-08-13T06:24:50Z\",\"notBeforeUtc\":\"2021-04-19T22:18:17Z\",\"provisioningState\":\"Accepted\"},\"id\":\"duiertgcc\",\"name\":\"mvaolps\",\"type\":\"lqlfm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Certificate response = manager.certificates() + .getWithResponse("wqytjrybnwjewgdr", "ervnaenqpehi", "doy", com.azure.core.util.Context.NONE).getValue(); - Certificate response = - manager - .certificates() - .getWithResponse("aolps", "lqlfm", "dnbbglzps", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogMockTests.java index 046b9f5afd56d..6364826e8ec04 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesListByCatalogMockTests.java @@ -30,45 +30,26 @@ public void testListByCatalog() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"certificate\":\"n\",\"status\":\"Expired\",\"subject\":\"gdrjervnaenqpe\",\"thumbprint\":\"ndoygmifthnzdnd\",\"expiryUtc\":\"2021-07-04T07:51:31Z\",\"notBeforeUtc\":\"2021-07-27T07:40:17Z\",\"provisioningState\":\"Canceled\"},\"id\":\"gynduha\",\"name\":\"hqlkthumaqo\",\"type\":\"bgycduiertgccym\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"certificate\":\"uvwbhsqfs\",\"status\":\"Expired\",\"subject\":\"jbi\",\"thumbprint\":\"bpybsrfbjf\",\"expiryUtc\":\"2021-07-05T13:20:17Z\",\"notBeforeUtc\":\"2021-10-31T01:03:52Z\",\"provisioningState\":\"Updating\"},\"id\":\"tpvjzbexilzznfqq\",\"name\":\"vwpm\",\"type\":\"taruoujmkcj\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.certificates().listByCatalog("ytkblmpew", "wfbkrvrns", + "shqjohxcrsbf", 249445305, 1210534043, 1966093769, com.azure.core.util.Context.NONE); - PagedIterable response = - manager - .certificates() - .listByCatalog( - "otftpvjzbexilz", - "nfqqnvwp", - "qtaruoujmkcjhwq", - 1209878928, - 1656357002, - 75465955, - com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainWithResponseMockTests.java index 29c0fee1fc770..5b3b5a8e49d28 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveCertChainWithResponseMockTests.java @@ -29,38 +29,25 @@ public void testRetrieveCertChainWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"certificateChain\":\"mweriofzpy\"}"; + String responseStr = "{\"certificateChain\":\"dvxzbncblylpst\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + CertificateChainResponse response = manager.certificates().retrieveCertChainWithResponse("dnbbglzps", + "iydmcwyhzdxs", "adbzmnvdfznud", com.azure.core.util.Context.NONE).getValue(); - CertificateChainResponse response = - manager - .certificates() - .retrieveCertChainWithResponse("wjmy", "tdss", "s", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceWithResponseMockTests.java index 012c89bb50ee5..7e21c84c01e39 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CertificatesRetrieveProofOfPossessionNonceWithResponseMockTests.java @@ -30,44 +30,29 @@ public void testRetrieveProofOfPossessionNonceWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"certificate\":\"k\",\"status\":\"Inactive\",\"subject\":\"pjflcxogao\",\"thumbprint\":\"nzmnsikvm\",\"expiryUtc\":\"2021-05-13T18:51:38Z\",\"notBeforeUtc\":\"2021-02-18T23:34:18Z\",\"provisioningState\":\"Failed\"}"; + String responseStr + = "{\"certificate\":\"mweriofzpy\",\"status\":\"Inactive\",\"subject\":\"wab\",\"thumbprint\":\"tshhszhedp\",\"expiryUtc\":\"2021-10-13T23:51:57Z\",\"notBeforeUtc\":\"2021-09-12T04:45:44Z\",\"provisioningState\":\"Succeeded\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + ProofOfPossessionNonceResponse response = manager.certificates() + .retrieveProofOfPossessionNonceWithResponse("bhhxsrzdzuc", "rsc", "ntnev", + new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("iwjmygtdssls"), + com.azure.core.util.Context.NONE) + .getValue(); - ProofOfPossessionNonceResponse response = - manager - .certificates() - .retrieveProofOfPossessionNonceWithResponse( - "semwabnet", - "hhszh", - "d", - new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("lvwiwubmwmbesl"), - com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ClaimDevicesRequestTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ClaimDevicesRequestTests.java index 987e77431a939..157bd75444d16 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ClaimDevicesRequestTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ClaimDevicesRequestTests.java @@ -12,19 +12,16 @@ public final class ClaimDevicesRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ClaimDevicesRequest model = - BinaryData - .fromString("{\"deviceIdentifiers\":[\"ckzywbiexzfeyue\",\"xibxujwbhqwalm\",\"zyoxaepdkzjan\"]}") - .toObject(ClaimDevicesRequest.class); - Assertions.assertEquals("ckzywbiexzfeyue", model.deviceIdentifiers().get(0)); + ClaimDevicesRequest model = BinaryData.fromString("{\"deviceIdentifiers\":[\"wiyighxpkdw\",\"baiuebbaumny\"]}") + .toObject(ClaimDevicesRequest.class); + Assertions.assertEquals("wiyighxpkdw", model.deviceIdentifiers().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ClaimDevicesRequest model = - new ClaimDevicesRequest() - .withDeviceIdentifiers(Arrays.asList("ckzywbiexzfeyue", "xibxujwbhqwalm", "zyoxaepdkzjan")); + ClaimDevicesRequest model + = new ClaimDevicesRequest().withDeviceIdentifiers(Arrays.asList("wiyighxpkdw", "baiuebbaumny")); model = BinaryData.fromObject(model).toObject(ClaimDevicesRequest.class); - Assertions.assertEquals("ckzywbiexzfeyue", model.deviceIdentifiers().get(0)); + Assertions.assertEquals("wiyighxpkdw", model.deviceIdentifiers().get(0)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseTests.java new file mode 100644 index 0000000000000..5b91fb03e7476 --- /dev/null +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.sphere.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import org.junit.jupiter.api.Assertions; + +public final class CountDeviceResponseTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CountDeviceResponse model = BinaryData.fromString("{\"value\":329870770}").toObject(CountDeviceResponse.class); + Assertions.assertEquals(329870770, model.value()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CountDeviceResponse model = new CountDeviceResponse().withValue(329870770); + model = BinaryData.fromObject(model).toObject(CountDeviceResponse.class); + Assertions.assertEquals(329870770, model.value()); + } +} diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDevicesResponseInnerTests.java similarity index 56% rename from sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseInnerTests.java rename to sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDevicesResponseInnerTests.java index 2ffe088cd6d62..234ac109554c5 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDeviceResponseInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountDevicesResponseInnerTests.java @@ -5,21 +5,21 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.CountDeviceResponseInner; +import com.azure.resourcemanager.sphere.fluent.models.CountDevicesResponseInner; import org.junit.jupiter.api.Assertions; -public final class CountDeviceResponseInnerTests { +public final class CountDevicesResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CountDeviceResponseInner model = - BinaryData.fromString("{\"value\":1762114496}").toObject(CountDeviceResponseInner.class); - Assertions.assertEquals(1762114496, model.value()); + CountDevicesResponseInner model + = BinaryData.fromString("{\"value\":1041920176}").toObject(CountDevicesResponseInner.class); + Assertions.assertEquals(1041920176, model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CountDeviceResponseInner model = new CountDeviceResponseInner().withValue(1762114496); - model = BinaryData.fromObject(model).toObject(CountDeviceResponseInner.class); - Assertions.assertEquals(1762114496, model.value()); + CountDevicesResponseInner model = new CountDevicesResponseInner().withValue(1041920176); + model = BinaryData.fromObject(model).toObject(CountDevicesResponseInner.class); + Assertions.assertEquals(1041920176, model.value()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountElementsResponseTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountElementsResponseTests.java index 41e3672f50df1..126ce4506c957 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountElementsResponseTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/CountElementsResponseTests.java @@ -11,15 +11,15 @@ public final class CountElementsResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CountElementsResponse model = - BinaryData.fromString("{\"value\":1742159588}").toObject(CountElementsResponse.class); - Assertions.assertEquals(1742159588, model.value()); + CountElementsResponse model + = BinaryData.fromString("{\"value\":494669618}").toObject(CountElementsResponse.class); + Assertions.assertEquals(494669618, model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CountElementsResponse model = new CountElementsResponse().withValue(1742159588); + CountElementsResponse model = new CountElementsResponse().withValue(494669618); model = BinaryData.fromObject(model).toObject(CountElementsResponse.class); - Assertions.assertEquals(1742159588, model.value()); + Assertions.assertEquals(494669618, model.value()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentInnerTests.java index 08bb9bfd280b6..b234299ca4c30 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentInnerTests.java @@ -7,6 +7,8 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; +import com.azure.resourcemanager.sphere.models.DeploymentProperties; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import java.util.Arrays; import org.junit.jupiter.api.Assertions; @@ -14,45 +16,34 @@ public final class DeploymentInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeploymentInner model = - BinaryData - .fromString( - "{\"properties\":{\"deploymentId\":\"jrmvdjwzrlo\",\"deployedImages\":[{\"properties\":{\"image\":\"hijco\",\"imageId\":\"ctbzaq\",\"imageName\":\"sycbkbfk\",\"regionalDataBoundary\":\"None\",\"uri\":\"kexxppof\",\"description\":\"axcfjpgddtocjjx\",\"componentId\":\"pmouexhdz\",\"imageType\":\"InvalidImageType\",\"provisioningState\":\"Accepted\"},\"id\":\"ojnxqbzvdd\",\"name\":\"t\",\"type\":\"ndei\"},{\"properties\":{\"image\":\"w\",\"imageId\":\"zao\",\"imageName\":\"uhrhcffcyddgl\",\"regionalDataBoundary\":\"None\",\"uri\":\"jqkwpyeicx\",\"description\":\"ciwqvhk\",\"componentId\":\"xuigdtopbobj\",\"imageType\":\"ManifestSet\",\"provisioningState\":\"Deleting\"},\"id\":\"w\",\"name\":\"a\",\"type\":\"a\"},{\"properties\":{\"image\":\"z\",\"imageId\":\"vvtpgvdfgio\",\"imageName\":\"ftutqxlngxlefgu\",\"regionalDataBoundary\":\"None\",\"uri\":\"rxdq\",\"description\":\"dt\",\"componentId\":\"zrvqdr\",\"imageType\":\"NormalWorldKernel\",\"provisioningState\":\"Provisioning\"},\"id\":\"big\",\"name\":\"h\",\"type\":\"qfbow\"},{\"properties\":{\"image\":\"nyktzlcuiy\",\"imageId\":\"qyw\",\"imageName\":\"drvyn\",\"regionalDataBoundary\":\"EU\",\"uri\":\"phrcgyncoc\",\"description\":\"cfvmmco\",\"componentId\":\"sxlzevgbmqj\",\"imageType\":\"PlutonRuntime\",\"provisioningState\":\"Deleting\"},\"id\":\"pmivkwlzu\",\"name\":\"ccfwnfnbacfion\",\"type\":\"ebxetqgtzxdp\"}],\"deploymentDateUtc\":\"2021-04-08T18:16:48Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"wxrjfeallnwsub\",\"name\":\"snjampmng\",\"type\":\"zscxaqwo\"}") - .toObject(DeploymentInner.class); - Assertions.assertEquals("jrmvdjwzrlo", model.deploymentId()); - Assertions.assertEquals("hijco", model.deployedImages().get(0).image()); - Assertions.assertEquals("ctbzaq", model.deployedImages().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.deployedImages().get(0).regionalDataBoundary()); + DeploymentInner model = BinaryData.fromString( + "{\"properties\":{\"deploymentId\":\"jrmvdjwzrlo\",\"deployedImages\":[{\"properties\":{\"image\":\"hijco\",\"imageId\":\"ctbzaq\",\"imageName\":\"sycbkbfk\",\"regionalDataBoundary\":\"None\",\"uri\":\"kexxppof\",\"description\":\"axcfjpgddtocjjx\",\"componentId\":\"pmouexhdz\",\"imageType\":\"InvalidImageType\",\"provisioningState\":\"Accepted\"},\"id\":\"jnxqbzvddntwn\",\"name\":\"eic\",\"type\":\"twnpzaoqvuhrhcf\"},{\"properties\":{\"image\":\"ddglm\",\"imageId\":\"hjq\",\"imageName\":\"pyeicxm\",\"regionalDataBoundary\":\"None\",\"uri\":\"q\",\"description\":\"khixuigdtopbo\",\"componentId\":\"og\",\"imageType\":\"InvalidImageType\",\"provisioningState\":\"Deleting\"},\"id\":\"m\",\"name\":\"uhrzayvvt\",\"type\":\"gvdfgiotkftutq\"},{\"properties\":{\"image\":\"gxlefgugnxkrxd\",\"imageId\":\"i\",\"imageName\":\"thz\",\"regionalDataBoundary\":\"EU\",\"uri\":\"rabhjybigeho\",\"description\":\"bowsk\",\"componentId\":\"yktz\",\"imageType\":\"SecurityMonitor\",\"provisioningState\":\"Provisioning\"},\"id\":\"gqywgndrv\",\"name\":\"nhzgpphrcgyn\",\"type\":\"ocpecfvmmco\"},{\"properties\":{\"image\":\"xlzevgbmqjqabcy\",\"imageId\":\"ivkwlzuvccfwnfnb\",\"imageName\":\"fionl\",\"regionalDataBoundary\":\"None\",\"uri\":\"tqgtzxdpnqbqq\",\"description\":\"rjfeallnwsubisnj\",\"componentId\":\"pmng\",\"imageType\":\"FwConfig\",\"provisioningState\":\"Provisioning\"},\"id\":\"qwoochcbon\",\"name\":\"vpk\",\"type\":\"lrxnjeaseiphe\"}],\"deploymentDateUtc\":\"2021-04-05T14:36:27Z\",\"provisioningState\":\"Failed\"},\"id\":\"yyien\",\"name\":\"bdlwtgrhpdjpj\",\"type\":\"masxazjpqyegu\"}") + .toObject(DeploymentInner.class); + Assertions.assertEquals("jrmvdjwzrlo", model.properties().deploymentId()); + Assertions.assertEquals("hijco", model.properties().deployedImages().get(0).properties().image()); + Assertions.assertEquals("ctbzaq", model.properties().deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + model.properties().deployedImages().get(0).properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeploymentInner model = - new DeploymentInner() - .withDeploymentId("jrmvdjwzrlo") - .withDeployedImages( - Arrays - .asList( - new ImageInner() - .withImage("hijco") - .withImageId("ctbzaq") - .withRegionalDataBoundary(RegionalDataBoundary.NONE), - new ImageInner() - .withImage("w") - .withImageId("zao") - .withRegionalDataBoundary(RegionalDataBoundary.NONE), - new ImageInner() - .withImage("z") - .withImageId("vvtpgvdfgio") - .withRegionalDataBoundary(RegionalDataBoundary.NONE), - new ImageInner() - .withImage("nyktzlcuiy") - .withImageId("qyw") - .withRegionalDataBoundary(RegionalDataBoundary.EU))); + DeploymentInner model + = new DeploymentInner().withProperties(new DeploymentProperties().withDeploymentId("jrmvdjwzrlo") + .withDeployedImages(Arrays.asList( + new ImageInner().withProperties(new ImageProperties().withImage("hijco").withImageId("ctbzaq") + .withRegionalDataBoundary(RegionalDataBoundary.NONE)), + new ImageInner().withProperties(new ImageProperties().withImage("ddglm").withImageId("hjq") + .withRegionalDataBoundary(RegionalDataBoundary.NONE)), + new ImageInner().withProperties(new ImageProperties().withImage("gxlefgugnxkrxd").withImageId("i") + .withRegionalDataBoundary(RegionalDataBoundary.EU)), + new ImageInner().withProperties(new ImageProperties().withImage("xlzevgbmqjqabcy") + .withImageId("ivkwlzuvccfwnfnb").withRegionalDataBoundary(RegionalDataBoundary.NONE))))); model = BinaryData.fromObject(model).toObject(DeploymentInner.class); - Assertions.assertEquals("jrmvdjwzrlo", model.deploymentId()); - Assertions.assertEquals("hijco", model.deployedImages().get(0).image()); - Assertions.assertEquals("ctbzaq", model.deployedImages().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.deployedImages().get(0).regionalDataBoundary()); + Assertions.assertEquals("jrmvdjwzrlo", model.properties().deploymentId()); + Assertions.assertEquals("hijco", model.properties().deployedImages().get(0).properties().image()); + Assertions.assertEquals("ctbzaq", model.properties().deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + model.properties().deployedImages().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentListResultTests.java index 6fee8c609012e..eb6600f03f0d0 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentListResultTests.java @@ -8,40 +8,26 @@ import com.azure.resourcemanager.sphere.fluent.models.DeploymentInner; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.DeploymentListResult; +import com.azure.resourcemanager.sphere.models.DeploymentProperties; +import com.azure.resourcemanager.sphere.models.ImageProperties; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class DeploymentListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeploymentListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"deploymentId\":\"idnsezcxtb\",\"deployedImages\":[{\"properties\":{},\"id\":\"yc\",\"name\":\"sne\",\"type\":\"mdwzjeiachboo\"},{\"properties\":{},\"id\":\"lnrosfqp\",\"name\":\"eeh\",\"type\":\"zvypyqrimzinp\"}],\"deploymentDateUtc\":\"2021-01-25T11:13:44Z\",\"provisioningState\":\"Deleting\"},\"id\":\"kirsoodqxhc\",\"name\":\"mnoh\",\"type\":\"t\"},{\"properties\":{\"deploymentId\":\"h\",\"deployedImages\":[{\"properties\":{},\"id\":\"fiyipjxsqwpgrj\",\"name\":\"znorcj\",\"type\":\"vsnb\"},{\"properties\":{},\"id\":\"qabnmoc\",\"name\":\"cyshurzafbljjgp\",\"type\":\"toqcjmklja\"},{\"properties\":{},\"id\":\"qidtqajzyu\",\"name\":\"pku\",\"type\":\"jkrlkhbzhfepg\"},{\"properties\":{},\"id\":\"qex\",\"name\":\"locx\",\"type\":\"c\"}],\"deploymentDateUtc\":\"2021-01-04T21:09:36Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"hhbcsglummajtjao\",\"name\":\"xobnbdxkqpxok\",\"type\":\"jionpimexgstxgc\"}],\"nextLink\":\"dg\"}") - .toObject(DeploymentListResult.class); - Assertions.assertEquals("idnsezcxtb", model.value().get(0).deploymentId()); - Assertions.assertEquals("dg", model.nextLink()); + DeploymentListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"deploymentId\":\"jmkljavbqidtqajz\",\"deployedImages\":[{\"properties\":{},\"id\":\"u\",\"name\":\"jkrlkhbzhfepg\",\"type\":\"gqexzlocxs\"}],\"deploymentDateUtc\":\"2021-11-18T07:05:01Z\",\"provisioningState\":\"Canceled\"},\"id\":\"hhbcsglummajtjao\",\"name\":\"xobnbdxkqpxok\",\"type\":\"jionpimexgstxgc\"}],\"nextLink\":\"dg\"}") + .toObject(DeploymentListResult.class); + Assertions.assertEquals("jmkljavbqidtqajz", model.value().get(0).properties().deploymentId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeploymentListResult model = - new DeploymentListResult() - .withValue( - Arrays - .asList( - new DeploymentInner() - .withDeploymentId("idnsezcxtb") - .withDeployedImages(Arrays.asList(new ImageInner(), new ImageInner())), - new DeploymentInner() - .withDeploymentId("h") - .withDeployedImages( - Arrays - .asList( - new ImageInner(), new ImageInner(), new ImageInner(), new ImageInner())))) - .withNextLink("dg"); + DeploymentListResult model = new DeploymentListResult().withValue(Arrays + .asList(new DeploymentInner().withProperties(new DeploymentProperties().withDeploymentId("jmkljavbqidtqajz") + .withDeployedImages(Arrays.asList(new ImageInner().withProperties(new ImageProperties())))))); model = BinaryData.fromObject(model).toObject(DeploymentListResult.class); - Assertions.assertEquals("idnsezcxtb", model.value().get(0).deploymentId()); - Assertions.assertEquals("dg", model.nextLink()); + Assertions.assertEquals("jmkljavbqidtqajz", model.value().get(0).properties().deploymentId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentPropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentPropertiesTests.java index 7073c470e6569..4a33e1481043c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentPropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentPropertiesTests.java @@ -5,8 +5,9 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.DeploymentProperties; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; +import com.azure.resourcemanager.sphere.models.DeploymentProperties; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import java.util.Arrays; import org.junit.jupiter.api.Assertions; @@ -14,37 +15,29 @@ public final class DeploymentPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeploymentProperties model = - BinaryData - .fromString( - "{\"deploymentId\":\"hcbonqvpkvlr\",\"deployedImages\":[{\"properties\":{\"image\":\"seiphe\",\"imageId\":\"lokeyy\",\"imageName\":\"nj\",\"regionalDataBoundary\":\"EU\",\"uri\":\"tgrhpdjpjumas\",\"description\":\"zj\",\"componentId\":\"yegu\",\"imageType\":\"FirmwareUpdateManifest\",\"provisioningState\":\"Provisioning\"},\"id\":\"xhejjzzvdud\",\"name\":\"wdslfhotwmcy\",\"type\":\"pwlbjnpg\"},{\"properties\":{\"image\":\"tadehxnltyfsopp\",\"imageId\":\"uesnzwdejbavo\",\"imageName\":\"zdmohctbqvu\",\"regionalDataBoundary\":\"None\",\"uri\":\"ndnvo\",\"description\":\"ujjugwdkcglh\",\"componentId\":\"azjdyggd\",\"imageType\":\"WifiFirmware\",\"provisioningState\":\"Provisioning\"},\"id\":\"b\",\"name\":\"uofqwe\",\"type\":\"kh\"}],\"deploymentDateUtc\":\"2021-06-19T17:30:56Z\",\"provisioningState\":\"Succeeded\"}") - .toObject(DeploymentProperties.class); - Assertions.assertEquals("hcbonqvpkvlr", model.deploymentId()); - Assertions.assertEquals("seiphe", model.deployedImages().get(0).image()); - Assertions.assertEquals("lokeyy", model.deployedImages().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.deployedImages().get(0).regionalDataBoundary()); + DeploymentProperties model = BinaryData.fromString( + "{\"deploymentId\":\"hb\",\"deployedImages\":[{\"properties\":{\"image\":\"jzzvdud\",\"imageId\":\"dslfhotwmcy\",\"imageName\":\"wlbjnpgacftade\",\"regionalDataBoundary\":\"EU\",\"uri\":\"tyfsoppusuesn\",\"description\":\"dejbavo\",\"componentId\":\"zdmohctbqvu\",\"imageType\":\"BaseSystemUpdateManifest\",\"provisioningState\":\"Canceled\"},\"id\":\"nvowgujju\",\"name\":\"wdkcglhsl\",\"type\":\"zj\"},{\"properties\":{\"image\":\"gdtjixhbkuofqwey\",\"imageId\":\"menevfyexfwh\",\"imageName\":\"cibvyvdcsitynn\",\"regionalDataBoundary\":\"EU\",\"uri\":\"ectehf\",\"description\":\"scjeypv\",\"componentId\":\"zrkgqhcjrefovg\",\"imageType\":\"CustomerBoardConfig\",\"provisioningState\":\"Succeeded\"},\"id\":\"yyvxyqjpkcattpn\",\"name\":\"jcrcczsqpjhvmda\",\"type\":\"v\"}],\"deploymentDateUtc\":\"2021-09-30T19:57Z\",\"provisioningState\":\"Failed\"}") + .toObject(DeploymentProperties.class); + Assertions.assertEquals("hb", model.deploymentId()); + Assertions.assertEquals("jzzvdud", model.deployedImages().get(0).properties().image()); + Assertions.assertEquals("dslfhotwmcy", model.deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, + model.deployedImages().get(0).properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeploymentProperties model = - new DeploymentProperties() - .withDeploymentId("hcbonqvpkvlr") - .withDeployedImages( - Arrays - .asList( - new ImageInner() - .withImage("seiphe") - .withImageId("lokeyy") - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new ImageInner() - .withImage("tadehxnltyfsopp") - .withImageId("uesnzwdejbavo") - .withRegionalDataBoundary(RegionalDataBoundary.NONE))); + DeploymentProperties model = new DeploymentProperties().withDeploymentId("hb") + .withDeployedImages(Arrays.asList( + new ImageInner().withProperties(new ImageProperties().withImage("jzzvdud").withImageId("dslfhotwmcy") + .withRegionalDataBoundary(RegionalDataBoundary.EU)), + new ImageInner().withProperties(new ImageProperties().withImage("gdtjixhbkuofqwey") + .withImageId("menevfyexfwh").withRegionalDataBoundary(RegionalDataBoundary.EU)))); model = BinaryData.fromObject(model).toObject(DeploymentProperties.class); - Assertions.assertEquals("hcbonqvpkvlr", model.deploymentId()); - Assertions.assertEquals("seiphe", model.deployedImages().get(0).image()); - Assertions.assertEquals("lokeyy", model.deployedImages().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.deployedImages().get(0).regionalDataBoundary()); + Assertions.assertEquals("hb", model.deploymentId()); + Assertions.assertEquals("jzzvdud", model.deployedImages().get(0).properties().image()); + Assertions.assertEquals("dslfhotwmcy", model.deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, + model.deployedImages().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateMockTests.java index 3341621f04508..782435cedb3dd 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsCreateOrUpdateMockTests.java @@ -14,6 +14,8 @@ import com.azure.resourcemanager.sphere.AzureSphereManager; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.Deployment; +import com.azure.resourcemanager.sphere.models.DeploymentProperties; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -33,57 +35,38 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"deploymentId\":\"dsrezpdrhneuyow\",\"deployedImages\":[{\"properties\":{\"image\":\"t\",\"imageId\":\"ib\",\"imageName\":\"cgpik\",\"regionalDataBoundary\":\"None\",\"uri\":\"ejzanlfz\",\"description\":\"av\",\"componentId\":\"bzonok\",\"imageType\":\"FirmwareUpdateManifest\",\"provisioningState\":\"Deleting\"},\"id\":\"cirgzp\",\"name\":\"rlazszrnw\",\"type\":\"iin\"}],\"deploymentDateUtc\":\"2021-01-06T22:30:17Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"jylwbtlhflsj\",\"name\":\"dhszfjv\",\"type\":\"bgofeljag\"}"; + String responseStr + = "{\"properties\":{\"deploymentId\":\"wib\",\"deployedImages\":[{\"properties\":{\"image\":\"bhshfwpracstwity\",\"imageId\":\"evxccedcp\",\"imageName\":\"dyodnwzxltj\",\"regionalDataBoundary\":\"EU\",\"uri\":\"ltiugcxnavv\",\"description\":\"qiby\",\"componentId\":\"nyowxwlmdjrkvfg\",\"imageType\":\"FwConfig\",\"provisioningState\":\"Updating\"},\"id\":\"bodacizsjq\",\"name\":\"hkr\",\"type\":\"ibdeibq\"}],\"deploymentDateUtc\":\"2021-10-05T18:29:09Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"vxndz\",\"name\":\"mkrefajpjorwkq\",\"type\":\"yhgbijtjivfx\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Deployment response = - manager - .deployments() - .define("nmdyodnwzxl") - .withExistingDeviceGroup("dl", "h", "hfwpracstwit", "khevxccedc") - .withDeploymentId("vnhltiugcx") - .withDeployedImages( - Arrays - .asList( - new ImageInner() - .withImage("xqi") - .withImageId("qunyowxwlmdjr") - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new ImageInner() - .withImage("jivfxzsjabib") - .withImageId("stawfsdjpvkv") - .withRegionalDataBoundary(RegionalDataBoundary.NONE))) - .create(); + Deployment response = manager.deployments().define("wpusdsttwvogv") + .withExistingDeviceGroup("bykutw", "fhpagmhrskdsnf", "sd", "akgtdlmkkzevdlh") + .withProperties(new DeploymentProperties().withDeploymentId("jdcngqqm") + .withDeployedImages(Arrays.asList( + new ImageInner().withProperties(new ImageProperties().withImage("gm").withImageId("rwr") + .withRegionalDataBoundary(RegionalDataBoundary.NONE)), + new ImageInner().withProperties(new ImageProperties().withImage("kaivwit").withImageId("cywuggwol") + .withRegionalDataBoundary(RegionalDataBoundary.EU))))) + .create(); - Assertions.assertEquals("dsrezpdrhneuyow", response.deploymentId()); - Assertions.assertEquals("t", response.deployedImages().get(0).image()); - Assertions.assertEquals("ib", response.deployedImages().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.deployedImages().get(0).regionalDataBoundary()); + Assertions.assertEquals("wib", response.properties().deploymentId()); + Assertions.assertEquals("bhshfwpracstwity", response.properties().deployedImages().get(0).properties().image()); + Assertions.assertEquals("evxccedcp", response.properties().deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, + response.properties().deployedImages().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteMockTests.java index 4ce227f68739a..6513e5ba1bb55 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsDeleteMockTests.java @@ -32,30 +32,21 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.deployments().delete("lolp", "vk", "r", "qvujzraehtwdwrf", "swibyr", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.deployments().delete("toego", "dwbwhkszzcmrvexz", "vbtqgsfraoyzk", "owtlmnguxawqald", + "yuuximerqfobwyzn", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetWithResponseMockTests.java index 068679d45e304..27a15bb56030e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsGetWithResponseMockTests.java @@ -31,45 +31,31 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"deploymentId\":\"doamciodhkha\",\"deployedImages\":[{\"properties\":{\"image\":\"zbonlwnt\",\"imageId\":\"gokdwbwhks\",\"imageName\":\"cmrvexzt\",\"regionalDataBoundary\":\"None\",\"uri\":\"gsfraoyzkoow\",\"description\":\"mnguxawqaldsyu\",\"componentId\":\"imerqfobwyznk\",\"imageType\":\"InvalidImageType\",\"provisioningState\":\"Updating\"},\"id\":\"wpfhpagmhrskd\",\"name\":\"nfd\",\"type\":\"doakgtdlmkkzevdl\"},{\"properties\":{\"image\":\"pusdstt\",\"imageId\":\"ogvbbejdcngq\",\"imageName\":\"oakufgm\",\"regionalDataBoundary\":\"None\",\"uri\":\"rdgrtw\",\"description\":\"nuuzkopbm\",\"componentId\":\"rfdwoyu\",\"imageType\":\"Nwfs\",\"provisioningState\":\"Failed\"},\"id\":\"iefozbhdmsml\",\"name\":\"zqhof\",\"type\":\"rmaequ\"},{\"properties\":{\"image\":\"xicslfao\",\"imageId\":\"piyylhalnswhccsp\",\"imageName\":\"aivwitqscywu\",\"regionalDataBoundary\":\"EU\",\"uri\":\"luhczbw\",\"description\":\"hairsbrgzdwms\",\"componentId\":\"ypqwdxggiccc\",\"imageType\":\"RecoveryManifest\",\"provisioningState\":\"Succeeded\"},\"id\":\"exmk\",\"name\":\"tlstvlzywem\",\"type\":\"zrncsdt\"}],\"deploymentDateUtc\":\"2021-07-01T02:09:13Z\",\"provisioningState\":\"Deleting\"},\"id\":\"ypbsfgytguslfead\",\"name\":\"ygqukyhejh\",\"type\":\"isxgfp\"}"; + String responseStr + = "{\"properties\":{\"deploymentId\":\"qdpfuvglsbjjca\",\"deployedImages\":[{\"properties\":{\"image\":\"t\",\"imageId\":\"dut\",\"imageName\":\"ormrlxqtvcofudfl\",\"regionalDataBoundary\":\"None\",\"uri\":\"u\",\"description\":\"dknnqvsazn\",\"componentId\":\"tor\",\"imageType\":\"Policy\",\"provisioningState\":\"Succeeded\"},\"id\":\"hmk\",\"name\":\"c\",\"type\":\"rauwjuetaebu\"},{\"properties\":{\"image\":\"dmovsm\",\"imageId\":\"xwabmqoe\",\"imageName\":\"ifrvtpu\",\"regionalDataBoundary\":\"None\",\"uri\":\"qlgkfbtn\",\"description\":\"aongbj\",\"componentId\":\"tujitcjedft\",\"imageType\":\"BaseSystemUpdateManifest\",\"provisioningState\":\"Updating\"},\"id\":\"ojvdcpzfoqo\",\"name\":\"i\",\"type\":\"ybxarzgszu\"}],\"deploymentDateUtc\":\"2021-07-26T12:22:12Z\",\"provisioningState\":\"Canceled\"},\"id\":\"opidoamciodh\",\"name\":\"haz\",\"type\":\"khnzbonlw\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Deployment response = - manager - .deployments() - .getWithResponse( - "bjcntujitc", "ed", "twwaezkojvdcpzf", "qouicybxarzgsz", "foxciq", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("doamciodhkha", response.deploymentId()); - Assertions.assertEquals("zbonlwnt", response.deployedImages().get(0).image()); - Assertions.assertEquals("gokdwbwhks", response.deployedImages().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.deployedImages().get(0).regionalDataBoundary()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Deployment response = manager.deployments().getWithResponse("cckwyfzqwhxxbu", "qa", "zfeqztppri", + "lxorjaltolmncws", "bqwcsdbnwdcf", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("qdpfuvglsbjjca", response.properties().deploymentId()); + Assertions.assertEquals("t", response.properties().deployedImages().get(0).properties().image()); + Assertions.assertEquals("dut", response.properties().deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + response.properties().deployedImages().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupMockTests.java index 861f00dbdfdfa..f0e6d9472c68a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeploymentsListByDeviceGroupMockTests.java @@ -32,54 +32,33 @@ public void testListByDeviceGroup() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"deploymentId\":\"rx\",\"deployedImages\":[{\"properties\":{\"image\":\"vaytdwkqbrq\",\"imageId\":\"paxh\",\"imageName\":\"iilivpdtiirqtd\",\"regionalDataBoundary\":\"EU\",\"uri\":\"oruzfgsquyfxrxx\",\"description\":\"ptramxj\",\"componentId\":\"wlwnwxuqlcv\",\"imageType\":\"UpdateCertStore\",\"provisioningState\":\"Failed\"},\"id\":\"tdooaoj\",\"name\":\"niodkooeb\",\"type\":\"nuj\"},{\"properties\":{\"image\":\"msbvdkcrodtjinf\",\"imageId\":\"lfltka\",\"imageName\":\"vefkdlfoakggk\",\"regionalDataBoundary\":\"EU\",\"uri\":\"ao\",\"description\":\"ulpqblylsyxkqjn\",\"componentId\":\"ervtiagxs\",\"imageType\":\"SecurityMonitor\",\"provisioningState\":\"Accepted\"},\"id\":\"mpsbzkfzbeyv\",\"name\":\"nqicvinvkjjxdxrb\",\"type\":\"ukzclewyhmlwpaz\"},{\"properties\":{\"image\":\"ofncckwyfzqwhxxb\",\"imageId\":\"qa\",\"imageName\":\"feqztppriol\",\"regionalDataBoundary\":\"EU\",\"uri\":\"altol\",\"description\":\"cwsobqwcs\",\"componentId\":\"nwdcfhu\",\"imageType\":\"PlutonRuntime\",\"provisioningState\":\"Updating\"},\"id\":\"uvglsbjjcanvx\",\"name\":\"vtvudutncormr\",\"type\":\"xqtvcofu\"},{\"properties\":{\"image\":\"vkg\",\"imageId\":\"bgdknnqv\",\"imageName\":\"znqntoru\",\"regionalDataBoundary\":\"EU\",\"uri\":\"a\",\"description\":\"kycgrauwj\",\"componentId\":\"taeburuvdm\",\"imageType\":\"NormalWorldKernel\",\"provisioningState\":\"Provisioning\"},\"id\":\"l\",\"name\":\"wabm\",\"type\":\"oefki\"}],\"deploymentDateUtc\":\"2021-07-14T12:33:43Z\",\"provisioningState\":\"Canceled\"},\"id\":\"u\",\"name\":\"ujmqlgkfbtndoa\",\"type\":\"n\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"deploymentId\":\"wmewzsyy\",\"deployedImages\":[{\"properties\":{\"image\":\"oibjudpfrxtrthz\",\"imageId\":\"ytdw\",\"imageName\":\"brqubp\",\"regionalDataBoundary\":\"None\",\"uri\":\"xiilivpdtiirqt\",\"description\":\"oaxoruzfgsqu\",\"componentId\":\"xrxxlep\",\"imageType\":\"OneBl\",\"provisioningState\":\"Deleting\"},\"id\":\"ezw\",\"name\":\"wnwxuqlcvyd\",\"type\":\"patdooaojkniodko\"},{\"properties\":{\"image\":\"wnujhemmsbvdk\",\"imageId\":\"odtji\",\"imageName\":\"wj\",\"regionalDataBoundary\":\"None\",\"uri\":\"kacjvefkdlfo\",\"description\":\"ggkfpagaowpul\",\"componentId\":\"blylsyxkqjnsj\",\"imageType\":\"UpdateCertStore\",\"provisioningState\":\"Succeeded\"},\"id\":\"gxsds\",\"name\":\"uem\",\"type\":\"sbzkf\"}],\"deploymentDateUtc\":\"2021-06-16T01:14:17Z\",\"provisioningState\":\"Failed\"},\"id\":\"nqicvinvkjjxdxrb\",\"name\":\"ukzclewyhmlwpaz\",\"type\":\"zpof\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .deployments() - .listByDeviceGroup( - "nbyxbaaabjyv", - "yffimrzrtuzqogs", - "xnevfdnwn", - "mewzsyyc", - "uzsoi", - 2025880638, - 414197490, - 2135802257, - com.azure.core.util.Context.NONE); + PagedIterable response = manager.deployments().listByDeviceGroup("cqqudf", "byxbaaabjy", "ayffim", + "zrtuzq", "gsexne", 850324011, 779817958, 1536688095, com.azure.core.util.Context.NONE); - Assertions.assertEquals("rx", response.iterator().next().deploymentId()); - Assertions.assertEquals("vaytdwkqbrq", response.iterator().next().deployedImages().get(0).image()); - Assertions.assertEquals("paxh", response.iterator().next().deployedImages().get(0).imageId()); - Assertions - .assertEquals( - RegionalDataBoundary.EU, response.iterator().next().deployedImages().get(0).regionalDataBoundary()); + Assertions.assertEquals("wmewzsyy", response.iterator().next().properties().deploymentId()); + Assertions.assertEquals("oibjudpfrxtrthz", + response.iterator().next().properties().deployedImages().get(0).properties().image()); + Assertions.assertEquals("ytdw", + response.iterator().next().properties().deployedImages().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + response.iterator().next().properties().deployedImages().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupInnerTests.java index 3c7b0effbe192..514635a5b1798 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupInnerTests.java @@ -7,6 +7,7 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.azure.resourcemanager.sphere.models.UpdatePolicy; @@ -15,32 +16,27 @@ public final class DeviceGroupInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceGroupInner model = - BinaryData - .fromString( - "{\"properties\":{\"description\":\"hqyudxorrqnbpoc\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Succeeded\"},\"id\":\"llr\",\"name\":\"vvdfwatkpnpul\",\"type\":\"xxbczwtr\"}") - .toObject(DeviceGroupInner.class); - Assertions.assertEquals("hqyudxorrqnbpoc", model.description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); + DeviceGroupInner model = BinaryData.fromString( + "{\"properties\":{\"description\":\"d\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Canceled\"},\"id\":\"czwtruwiqzbqjv\",\"name\":\"ovm\",\"type\":\"okacspk\"}") + .toObject(DeviceGroupInner.class); + Assertions.assertEquals("d", model.properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL, model.properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, model.properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceGroupInner model = - new DeviceGroupInner() - .withDescription("hqyudxorrqnbpoc") - .withOsFeedType(OSFeedType.RETAIL_EVAL) - .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) - .withRegionalDataBoundary(RegionalDataBoundary.NONE); + DeviceGroupInner model = new DeviceGroupInner().withProperties(new DeviceGroupProperties().withDescription("d") + .withOsFeedType(OSFeedType.RETAIL).withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.ENABLED) + .withRegionalDataBoundary(RegionalDataBoundary.EU)); model = BinaryData.fromObject(model).toObject(DeviceGroupInner.class); - Assertions.assertEquals("hqyudxorrqnbpoc", model.description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); + Assertions.assertEquals("d", model.properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL, model.properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, model.properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupListResultTests.java index 3f11315981974..86e73b1470602 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupListResultTests.java @@ -8,6 +8,7 @@ import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupInner; import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; import com.azure.resourcemanager.sphere.models.DeviceGroupListResult; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.azure.resourcemanager.sphere.models.UpdatePolicy; @@ -17,57 +18,38 @@ public final class DeviceGroupListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceGroupListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"description\":\"bvyvdcsity\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Provisioning\"},\"id\":\"qsc\",\"name\":\"eypvhezrkg\",\"type\":\"hcjrefovgmk\"},{\"properties\":{\"description\":\"eyyvxyqjpkcat\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Succeeded\"},\"id\":\"jh\",\"name\":\"mdajv\",\"type\":\"ysou\"},{\"properties\":{\"description\":\"canoaeupf\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"UpdateAll\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Updating\"},\"id\":\"matuok\",\"name\":\"hfuiuaodsfc\",\"type\":\"kvxod\"},{\"properties\":{\"description\":\"zmyzydagf\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Deleting\"},\"id\":\"whrdxwzywqsmbsu\",\"name\":\"exim\",\"type\":\"ryocfsfksymdd\"}],\"nextLink\":\"tki\"}") - .toObject(DeviceGroupListResult.class); - Assertions.assertEquals("bvyvdcsity", model.value().get(0).description()); - Assertions.assertEquals(OSFeedType.RETAIL, model.value().get(0).osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.value().get(0).updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, model.value().get(0).allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.value().get(0).regionalDataBoundary()); - Assertions.assertEquals("tki", model.nextLink()); + DeviceGroupListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"description\":\"oaeupfhyhltrpmo\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"UpdateAll\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Failed\"},\"id\":\"iuaod\",\"name\":\"fcp\",\"type\":\"vxodpu\"},{\"properties\":{\"description\":\"yzydagfuaxbezyi\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":false,\"provisioningState\":\"Failed\"},\"id\":\"q\",\"name\":\"mbsureximo\",\"type\":\"yocf\"},{\"properties\":{\"description\":\"s\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"UpdateAll\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":false,\"provisioningState\":\"Failed\"},\"id\":\"yudxorrqnbp\",\"name\":\"czvyifq\",\"type\":\"vkd\"}],\"nextLink\":\"sllr\"}") + .toObject(DeviceGroupListResult.class); + Assertions.assertEquals("oaeupfhyhltrpmo", model.value().get(0).properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.value().get(0).properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.UPDATE_ALL, model.value().get(0).properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, + model.value().get(0).properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.value().get(0).properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceGroupListResult model = - new DeviceGroupListResult() - .withValue( - Arrays - .asList( - new DeviceGroupInner() - .withDescription("bvyvdcsity") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.ENABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new DeviceGroupInner() - .withDescription("eyyvxyqjpkcat") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new DeviceGroupInner() - .withDescription("canoaeupf") - .withOsFeedType(OSFeedType.RETAIL_EVAL) - .withUpdatePolicy(UpdatePolicy.UPDATE_ALL) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.ENABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new DeviceGroupInner() - .withDescription("zmyzydagf") - .withOsFeedType(OSFeedType.RETAIL_EVAL) - .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU))) - .withNextLink("tki"); + DeviceGroupListResult model = new DeviceGroupListResult().withValue(Arrays.asList( + new DeviceGroupInner().withProperties(new DeviceGroupProperties().withDescription("oaeupfhyhltrpmo") + .withOsFeedType(OSFeedType.RETAIL_EVAL).withUpdatePolicy(UpdatePolicy.UPDATE_ALL) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.ENABLED) + .withRegionalDataBoundary(RegionalDataBoundary.NONE)), + new DeviceGroupInner().withProperties(new DeviceGroupProperties().withDescription("yzydagfuaxbezyi") + .withOsFeedType(OSFeedType.RETAIL_EVAL).withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) + .withRegionalDataBoundary(RegionalDataBoundary.EU)), + new DeviceGroupInner().withProperties(new DeviceGroupProperties().withDescription("s") + .withOsFeedType(OSFeedType.RETAIL_EVAL).withUpdatePolicy(UpdatePolicy.UPDATE_ALL) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.ENABLED) + .withRegionalDataBoundary(RegionalDataBoundary.NONE)))); model = BinaryData.fromObject(model).toObject(DeviceGroupListResult.class); - Assertions.assertEquals("bvyvdcsity", model.value().get(0).description()); - Assertions.assertEquals(OSFeedType.RETAIL, model.value().get(0).osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.value().get(0).updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, model.value().get(0).allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.value().get(0).regionalDataBoundary()); - Assertions.assertEquals("tki", model.nextLink()); + Assertions.assertEquals("oaeupfhyhltrpmo", model.value().get(0).properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.value().get(0).properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.UPDATE_ALL, model.value().get(0).properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, + model.value().get(0).properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.value().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupPropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupPropertiesTests.java index 445027d88fdb7..68cf41bc666ae 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupPropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupPropertiesTests.java @@ -5,8 +5,8 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.azure.resourcemanager.sphere.models.UpdatePolicy; @@ -15,32 +15,27 @@ public final class DeviceGroupPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceGroupProperties model = - BinaryData - .fromString( - "{\"description\":\"iqzbq\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Deleting\"}") - .toObject(DeviceGroupProperties.class); - Assertions.assertEquals("iqzbq", model.description()); + DeviceGroupProperties model = BinaryData.fromString( + "{\"description\":\"hzdobpxjmflbvvnc\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Failed\"}") + .toObject(DeviceGroupProperties.class); + Assertions.assertEquals("hzdobpxjmflbvvnc", model.description()); Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.osFeedType()); Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.updatePolicy()); Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceGroupProperties model = - new DeviceGroupProperties() - .withDescription("iqzbq") - .withOsFeedType(OSFeedType.RETAIL_EVAL) - .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) - .withRegionalDataBoundary(RegionalDataBoundary.NONE); + DeviceGroupProperties model = new DeviceGroupProperties().withDescription("hzdobpxjmflbvvnc") + .withOsFeedType(OSFeedType.RETAIL_EVAL).withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) + .withRegionalDataBoundary(RegionalDataBoundary.EU); model = BinaryData.fromObject(model).toObject(DeviceGroupProperties.class); - Assertions.assertEquals("iqzbq", model.description()); + Assertions.assertEquals("hzdobpxjmflbvvnc", model.description()); Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.osFeedType()); Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.updatePolicy()); Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdatePropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdatePropertiesTests.java index b0e15ed3d3827..d5aaea99a647d 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdatePropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdatePropertiesTests.java @@ -5,8 +5,8 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.DeviceGroupUpdateProperties; import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; +import com.azure.resourcemanager.sphere.models.DeviceGroupUpdateProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.azure.resourcemanager.sphere.models.UpdatePolicy; @@ -15,32 +15,27 @@ public final class DeviceGroupUpdatePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceGroupUpdateProperties model = - BinaryData - .fromString( - "{\"description\":\"cocmnyyaztt\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"UpdateAll\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\"}") - .toObject(DeviceGroupUpdateProperties.class); - Assertions.assertEquals("cocmnyyaztt", model.description()); + DeviceGroupUpdateProperties model = BinaryData.fromString( + "{\"description\":\"kjz\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"UpdateAll\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"None\"}") + .toObject(DeviceGroupUpdateProperties.class); + Assertions.assertEquals("kjz", model.description()); Assertions.assertEquals(OSFeedType.RETAIL, model.osFeedType()); Assertions.assertEquals(UpdatePolicy.UPDATE_ALL, model.updatePolicy()); Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceGroupUpdateProperties model = - new DeviceGroupUpdateProperties() - .withDescription("cocmnyyaztt") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.UPDATE_ALL) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU); + DeviceGroupUpdateProperties model = new DeviceGroupUpdateProperties().withDescription("kjz") + .withOsFeedType(OSFeedType.RETAIL).withUpdatePolicy(UpdatePolicy.UPDATE_ALL) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) + .withRegionalDataBoundary(RegionalDataBoundary.NONE); model = BinaryData.fromObject(model).toObject(DeviceGroupUpdateProperties.class); - Assertions.assertEquals("cocmnyyaztt", model.description()); + Assertions.assertEquals("kjz", model.description()); Assertions.assertEquals(OSFeedType.RETAIL, model.osFeedType()); Assertions.assertEquals(UpdatePolicy.UPDATE_ALL, model.updatePolicy()); Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdateTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdateTests.java index 2fef5ffd910aa..1f09bd0a1f540 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdateTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupUpdateTests.java @@ -7,6 +7,7 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; import com.azure.resourcemanager.sphere.models.DeviceGroupUpdate; +import com.azure.resourcemanager.sphere.models.DeviceGroupUpdateProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.azure.resourcemanager.sphere.models.UpdatePolicy; @@ -15,32 +16,28 @@ public final class DeviceGroupUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceGroupUpdate model = - BinaryData - .fromString( - "{\"properties\":{\"description\":\"xdbabphlwr\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\"}}") - .toObject(DeviceGroupUpdate.class); - Assertions.assertEquals("xdbabphlwr", model.description()); - Assertions.assertEquals(OSFeedType.RETAIL, model.osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); + DeviceGroupUpdate model = BinaryData.fromString( + "{\"properties\":{\"description\":\"xquk\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\"}}") + .toObject(DeviceGroupUpdate.class); + Assertions.assertEquals("xquk", model.properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceGroupUpdate model = - new DeviceGroupUpdate() - .withDescription("xdbabphlwr") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) - .withAllowCrashDumpsCollection(AllowCrashDumpCollection.ENABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU); + DeviceGroupUpdate model + = new DeviceGroupUpdate().withProperties(new DeviceGroupUpdateProperties().withDescription("xquk") + .withOsFeedType(OSFeedType.RETAIL_EVAL).withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) + .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) + .withRegionalDataBoundary(RegionalDataBoundary.EU)); model = BinaryData.fromObject(model).toObject(DeviceGroupUpdate.class); - Assertions.assertEquals("xdbabphlwr", model.description()); - Assertions.assertEquals(OSFeedType.RETAIL, model.osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, model.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); + Assertions.assertEquals("xquk", model.properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, model.properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, model.properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, model.properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesWithResponseMockTests.java index cf62cf1fa5eb8..829d574b47b02 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCountDevicesWithResponseMockTests.java @@ -12,7 +12,7 @@ import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -30,40 +30,26 @@ public void testCountDevicesWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"value\":802895356}"; + String responseStr = "{\"value\":326673667}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - CountDeviceResponse response = - manager - .deviceGroups() - .countDevicesWithResponse("sphyoulpjrvxa", "l", "vimjwos", "tx", com.azure.core.util.Context.NONE) - .getValue(); + CountDevicesResponse response = manager.deviceGroups().countDevicesWithResponse("elpcirelsfeaenwa", "fatkld", + "xbjhwuaanozjosph", "oulpjrv", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(802895356, response.value()); + Assertions.assertEquals(326673667, response.value()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateMockTests.java index cb460507b2e43..9f6376ba8a795 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsCreateOrUpdateMockTests.java @@ -14,6 +14,7 @@ import com.azure.resourcemanager.sphere.AzureSphereManager; import com.azure.resourcemanager.sphere.models.AllowCrashDumpCollection; import com.azure.resourcemanager.sphere.models.DeviceGroup; +import com.azure.resourcemanager.sphere.models.DeviceGroupProperties; import com.azure.resourcemanager.sphere.models.OSFeedType; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import com.azure.resourcemanager.sphere.models.UpdatePolicy; @@ -34,51 +35,36 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"description\":\"oduhp\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":false,\"provisioningState\":\"Succeeded\"},\"id\":\"xqugjhkycubedd\",\"name\":\"ssofwqmzqa\",\"type\":\"krmnjijpxacqqud\"}"; + String responseStr + = "{\"properties\":{\"description\":\"oduhp\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":false,\"provisioningState\":\"Succeeded\"},\"id\":\"qugjhkycube\",\"name\":\"dgssofwqmzqal\",\"type\":\"rmnjijpx\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - DeviceGroup response = - manager - .deviceGroups() - .define("dunyg") - .withExistingProduct("cskfcktqumiekk", "zzikhlyfjhdg", "gge") - .withDescription("idb") - .withOsFeedType(OSFeedType.RETAIL) - .withUpdatePolicy(UpdatePolicy.UPDATE_ALL) + DeviceGroup response = manager.deviceGroups().define("gge") + .withExistingProduct("glrvimjwosytxi", "cskfcktqumiekk", "zzikhlyfjhdg") + .withProperties(new DeviceGroupProperties().withDescription("nyga").withOsFeedType(OSFeedType.RETAIL) + .withUpdatePolicy(UpdatePolicy.NO3RD_PARTY_APP_UPDATES) .withAllowCrashDumpsCollection(AllowCrashDumpCollection.DISABLED) - .withRegionalDataBoundary(RegionalDataBoundary.EU) - .create(); + .withRegionalDataBoundary(RegionalDataBoundary.NONE)) + .create(); - Assertions.assertEquals("oduhp", response.description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, response.updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, response.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.EU, response.regionalDataBoundary()); + Assertions.assertEquals("oduhp", response.properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, response.properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, response.properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.EU, response.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteMockTests.java index ad74f2a3ce680..356103c2be9e5 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsDeleteMockTests.java @@ -32,32 +32,21 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .deviceGroups() - .delete("wczelpci", "elsfeaen", "abfatkl", "dxbjhwuaanozj", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.deviceGroups().delete("imfnjhfjx", "mszkkfo", "rey", "kzikfjawneaivxwc", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetWithResponseMockTests.java index c15479bbbd942..414e69870be1c 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsGetWithResponseMockTests.java @@ -34,45 +34,31 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"description\":\"npkukghimdblx\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":false,\"provisioningState\":\"Accepted\"},\"id\":\"szkkfoqre\",\"name\":\"fkzikfj\",\"type\":\"wneaiv\"}"; + String responseStr + = "{\"properties\":{\"description\":\"v\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Updating\"},\"id\":\"dxepxgyq\",\"name\":\"gvr\",\"type\":\"mnpkukghimdblxg\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - DeviceGroup response = - manager - .deviceGroups() - .getWithResponse("mtdaa", "gdv", "vgpiohgwxrt", "udxepxgyqagv", com.azure.core.util.Context.NONE) - .getValue(); + DeviceGroup response = manager.deviceGroups() + .getWithResponse("xxmueedn", "rdvstkwqqtch", "alm", "mtdaa", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("npkukghimdblx", response.description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, response.updatePolicy()); - Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, response.allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.regionalDataBoundary()); + Assertions.assertEquals("v", response.properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, response.properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, response.properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.NONE, response.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductMockTests.java index a5388dfb9b504..078108c614720 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceGroupsListByProductMockTests.java @@ -35,53 +35,34 @@ public void testListByProduct() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"description\":\"ggxkallatmelwuip\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":false,\"provisioningState\":\"Accepted\"},\"id\":\"nayrhyrnxxmueedn\",\"name\":\"rdvstkwqqtch\",\"type\":\"alm\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"description\":\"mqg\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Enabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Provisioning\"},\"id\":\"allatmelwuipic\",\"name\":\"jzkzi\",\"type\":\"gvvcnayrhyr\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .deviceGroups() - .listByProduct( - "tdtbnnhadooc", - "kvci", - "hnvpamqgxq", - "u", - 1862318440, - 443622503, - 1817330156, - com.azure.core.util.Context.NONE); + PagedIterable response = manager.deviceGroups().listByProduct("lxdy", "gsyocogj", "tdtbnnhadooc", + "kvci", 464313768, 1515797519, 600474862, com.azure.core.util.Context.NONE); - Assertions.assertEquals("ggxkallatmelwuip", response.iterator().next().description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.iterator().next().osFeedType()); - Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, response.iterator().next().updatePolicy()); - Assertions - .assertEquals(AllowCrashDumpCollection.ENABLED, response.iterator().next().allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.iterator().next().regionalDataBoundary()); + Assertions.assertEquals("mqg", response.iterator().next().properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL, response.iterator().next().properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, + response.iterator().next().properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.ENABLED, + response.iterator().next().properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.NONE, + response.iterator().next().properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInnerTests.java index e7cd923a440ba..57dbd6f596d0f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInnerTests.java @@ -6,23 +6,22 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; +import com.azure.resourcemanager.sphere.models.DeviceProperties; import org.junit.jupiter.api.Assertions; public final class DeviceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceInner model = - BinaryData - .fromString( - "{\"properties\":{\"deviceId\":\"wutttxfvjrbi\",\"chipSku\":\"hxepcyvahfnlj\",\"lastAvailableOsVersion\":\"qxj\",\"lastInstalledOsVersion\":\"ujqgidok\",\"lastOsUpdateUtc\":\"2021-09-28T11:40:31Z\",\"lastUpdateRequestUtc\":\"2021-11-13T00:56:49Z\",\"provisioningState\":\"Accepted\"},\"id\":\"gvcl\",\"name\":\"bgsncghkjeszzhb\",\"type\":\"jhtxfvgxbfsmxne\"}") - .toObject(DeviceInner.class); - Assertions.assertEquals("wutttxfvjrbi", model.deviceId()); + DeviceInner model = BinaryData.fromString( + "{\"properties\":{\"deviceId\":\"kpikadrgvt\",\"chipSku\":\"gnbuy\",\"lastAvailableOsVersion\":\"ijggmebfsiar\",\"lastInstalledOsVersion\":\"trcvpnazzmh\",\"lastOsUpdateUtc\":\"2021-11-09T20:12:13Z\",\"lastUpdateRequestUtc\":\"2021-02-24T15:52:09Z\",\"provisioningState\":\"Provisioning\"},\"id\":\"tdbhrbnla\",\"name\":\"kx\",\"type\":\"yskpbhen\"}") + .toObject(DeviceInner.class); + Assertions.assertEquals("kpikadrgvt", model.properties().deviceId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceInner model = new DeviceInner().withDeviceId("wutttxfvjrbi"); + DeviceInner model = new DeviceInner().withProperties(new DeviceProperties().withDeviceId("kpikadrgvt")); model = BinaryData.fromObject(model).toObject(DeviceInner.class); - Assertions.assertEquals("wutttxfvjrbi", model.deviceId()); + Assertions.assertEquals("kpikadrgvt", model.properties().deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInsightInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInsightInnerTests.java index 3241802fb0647..d4df5de833d42 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInsightInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceInsightInnerTests.java @@ -12,41 +12,33 @@ public final class DeviceInsightInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceInsightInner model = - BinaryData - .fromString( - "{\"deviceId\":\"eggzfb\",\"description\":\"hfmvfaxkffe\",\"startTimestampUtc\":\"2021-04-21T05:15:54Z\",\"endTimestampUtc\":\"2021-05-15T04:16:32Z\",\"eventCategory\":\"hl\",\"eventClass\":\"m\",\"eventType\":\"zy\",\"eventCount\":2122365511}") - .toObject(DeviceInsightInner.class); - Assertions.assertEquals("eggzfb", model.deviceId()); - Assertions.assertEquals("hfmvfaxkffe", model.description()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-21T05:15:54Z"), model.startTimestampUtc()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-15T04:16:32Z"), model.endTimestampUtc()); - Assertions.assertEquals("hl", model.eventCategory()); - Assertions.assertEquals("m", model.eventClass()); - Assertions.assertEquals("zy", model.eventType()); - Assertions.assertEquals(2122365511, model.eventCount()); + DeviceInsightInner model = BinaryData.fromString( + "{\"deviceId\":\"ntypmrbpizcdrqj\",\"description\":\"dpydn\",\"startTimestampUtc\":\"2021-02-06T09:08:09Z\",\"endTimestampUtc\":\"2021-07-10T19:26:28Z\",\"eventCategory\":\"xdeoejzic\",\"eventClass\":\"ifsjttgzfbishcb\",\"eventType\":\"hajdeyeamdpha\",\"eventCount\":1536073800}") + .toObject(DeviceInsightInner.class); + Assertions.assertEquals("ntypmrbpizcdrqj", model.deviceId()); + Assertions.assertEquals("dpydn", model.description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-06T09:08:09Z"), model.startTimestampUtc()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-10T19:26:28Z"), model.endTimestampUtc()); + Assertions.assertEquals("xdeoejzic", model.eventCategory()); + Assertions.assertEquals("ifsjttgzfbishcb", model.eventClass()); + Assertions.assertEquals("hajdeyeamdpha", model.eventType()); + Assertions.assertEquals(1536073800, model.eventCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceInsightInner model = - new DeviceInsightInner() - .withDeviceId("eggzfb") - .withDescription("hfmvfaxkffe") - .withStartTimestampUtc(OffsetDateTime.parse("2021-04-21T05:15:54Z")) - .withEndTimestampUtc(OffsetDateTime.parse("2021-05-15T04:16:32Z")) - .withEventCategory("hl") - .withEventClass("m") - .withEventType("zy") - .withEventCount(2122365511); + DeviceInsightInner model = new DeviceInsightInner().withDeviceId("ntypmrbpizcdrqj").withDescription("dpydn") + .withStartTimestampUtc(OffsetDateTime.parse("2021-02-06T09:08:09Z")) + .withEndTimestampUtc(OffsetDateTime.parse("2021-07-10T19:26:28Z")).withEventCategory("xdeoejzic") + .withEventClass("ifsjttgzfbishcb").withEventType("hajdeyeamdpha").withEventCount(1536073800); model = BinaryData.fromObject(model).toObject(DeviceInsightInner.class); - Assertions.assertEquals("eggzfb", model.deviceId()); - Assertions.assertEquals("hfmvfaxkffe", model.description()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-21T05:15:54Z"), model.startTimestampUtc()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-15T04:16:32Z"), model.endTimestampUtc()); - Assertions.assertEquals("hl", model.eventCategory()); - Assertions.assertEquals("m", model.eventClass()); - Assertions.assertEquals("zy", model.eventType()); - Assertions.assertEquals(2122365511, model.eventCount()); + Assertions.assertEquals("ntypmrbpizcdrqj", model.deviceId()); + Assertions.assertEquals("dpydn", model.description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-06T09:08:09Z"), model.startTimestampUtc()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-10T19:26:28Z"), model.endTimestampUtc()); + Assertions.assertEquals("xdeoejzic", model.eventCategory()); + Assertions.assertEquals("ifsjttgzfbishcb", model.eventClass()); + Assertions.assertEquals("hajdeyeamdpha", model.eventType()); + Assertions.assertEquals(1536073800, model.eventCount()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceListResultTests.java index 0fb84591ce8b2..f4eada2463b19 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceListResultTests.java @@ -7,35 +7,27 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.DeviceInner; import com.azure.resourcemanager.sphere.models.DeviceListResult; +import com.azure.resourcemanager.sphere.models.DeviceProperties; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class DeviceListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"deviceId\":\"zsbbzoggigrxwb\",\"chipSku\":\"vjxxjnsp\",\"lastAvailableOsVersion\":\"ptkoenkoukn\",\"lastInstalledOsVersion\":\"dwtiukbldngkp\",\"lastOsUpdateUtc\":\"2021-05-04T07:17:38Z\",\"lastUpdateRequestUtc\":\"2021-05-19T00:57:25Z\",\"provisioningState\":\"Canceled\"},\"id\":\"xoegukgjnpiucgy\",\"name\":\"evqzntypmrbp\",\"type\":\"zcdrqjsdpydnfyhx\"},{\"properties\":{\"deviceId\":\"ejzicwifsjtt\",\"chipSku\":\"fbishcbkha\",\"lastAvailableOsVersion\":\"eyeam\",\"lastInstalledOsVersion\":\"hagalpbuxwgipwh\",\"lastOsUpdateUtc\":\"2021-04-24T13:32:45Z\",\"lastUpdateRequestUtc\":\"2021-08-18T16:08:42Z\",\"provisioningState\":\"Deleting\"},\"id\":\"hwankixzbinjepu\",\"name\":\"tmryw\",\"type\":\"uzoqft\"},{\"properties\":{\"deviceId\":\"zrnkcqvyxlwh\",\"chipSku\":\"sicohoqqnwvlry\",\"lastAvailableOsVersion\":\"w\",\"lastInstalledOsVersion\":\"eun\",\"lastOsUpdateUtc\":\"2021-02-20T01:45:10Z\",\"lastUpdateRequestUtc\":\"2021-10-16T07:10:15Z\",\"provisioningState\":\"Canceled\"},\"id\":\"zko\",\"name\":\"ocukoklyax\",\"type\":\"conuqszfkbeype\"},{\"properties\":{\"deviceId\":\"jmwvvj\",\"chipSku\":\"tcxsenhwlrs\",\"lastAvailableOsVersion\":\"rzpwvlqdqgbiq\",\"lastInstalledOsVersion\":\"ihkaetcktvfc\",\"lastOsUpdateUtc\":\"2021-07-13T01:18:13Z\",\"lastUpdateRequestUtc\":\"2021-09-20T06:05:50Z\",\"provisioningState\":\"Provisioning\"},\"id\":\"m\",\"name\":\"ctq\",\"type\":\"jf\"}],\"nextLink\":\"brjcxe\"}") - .toObject(DeviceListResult.class); - Assertions.assertEquals("zsbbzoggigrxwb", model.value().get(0).deviceId()); - Assertions.assertEquals("brjcxe", model.nextLink()); + DeviceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"deviceId\":\"uxwgipwho\",\"chipSku\":\"wkgshwa\",\"lastAvailableOsVersion\":\"ixzbinjeputtmryw\",\"lastInstalledOsVersion\":\"zoqftiyqzrnkcqvy\",\"lastOsUpdateUtc\":\"2021-11-01T04:15:41Z\",\"lastUpdateRequestUtc\":\"2021-04-08T10:43:47Z\",\"provisioningState\":\"Provisioning\"},\"id\":\"cohoq\",\"name\":\"nwvlryavwhheunmm\",\"type\":\"hgyxzkonoc\"},{\"properties\":{\"deviceId\":\"klyaxuconu\",\"chipSku\":\"zf\",\"lastAvailableOsVersion\":\"eyp\",\"lastInstalledOsVersion\":\"rmjmwvvjektc\",\"lastOsUpdateUtc\":\"2021-04-14T12:00:30Z\",\"lastUpdateRequestUtc\":\"2021-03-21T19:21:01Z\",\"provisioningState\":\"Provisioning\"},\"id\":\"s\",\"name\":\"frzpwvlqdqgb\",\"type\":\"qylihkaetckt\"},{\"properties\":{\"deviceId\":\"ivfsnk\",\"chipSku\":\"uctqhjfbe\",\"lastAvailableOsVersion\":\"jcxerfuwu\",\"lastInstalledOsVersion\":\"txfvjrbirph\",\"lastOsUpdateUtc\":\"2020-12-25T01:34:04Z\",\"lastUpdateRequestUtc\":\"2021-09-05T13:55:56Z\",\"provisioningState\":\"Provisioning\"},\"id\":\"fnljky\",\"name\":\"xjvuujqgidokg\",\"type\":\"ljyoxgvcltb\"},{\"properties\":{\"deviceId\":\"c\",\"chipSku\":\"kjeszz\",\"lastAvailableOsVersion\":\"ijhtxf\",\"lastInstalledOsVersion\":\"xbf\",\"lastOsUpdateUtc\":\"2021-01-19T10:57:10Z\",\"lastUpdateRequestUtc\":\"2021-05-10T05:23:15Z\",\"provisioningState\":\"Failed\"},\"id\":\"vecxgodebfqkk\",\"name\":\"bmpukgriwflz\",\"type\":\"fbxzpuzycisp\"}],\"nextLink\":\"zahmgkbrpyydhibn\"}") + .toObject(DeviceListResult.class); + Assertions.assertEquals("uxwgipwho", model.value().get(0).properties().deviceId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceListResult model = - new DeviceListResult() - .withValue( - Arrays - .asList( - new DeviceInner().withDeviceId("zsbbzoggigrxwb"), - new DeviceInner().withDeviceId("ejzicwifsjtt"), - new DeviceInner().withDeviceId("zrnkcqvyxlwh"), - new DeviceInner().withDeviceId("jmwvvj"))) - .withNextLink("brjcxe"); + DeviceListResult model = new DeviceListResult() + .withValue(Arrays.asList(new DeviceInner().withProperties(new DeviceProperties().withDeviceId("uxwgipwho")), + new DeviceInner().withProperties(new DeviceProperties().withDeviceId("klyaxuconu")), + new DeviceInner().withProperties(new DeviceProperties().withDeviceId("ivfsnk")), + new DeviceInner().withProperties(new DeviceProperties().withDeviceId("c")))); model = BinaryData.fromObject(model).toObject(DeviceListResult.class); - Assertions.assertEquals("zsbbzoggigrxwb", model.value().get(0).deviceId()); - Assertions.assertEquals("brjcxe", model.nextLink()); + Assertions.assertEquals("uxwgipwho", model.value().get(0).properties().deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicePropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicePropertiesTests.java index 20be7d4c930e1..76907e2d7e3cf 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicePropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicePropertiesTests.java @@ -5,24 +5,22 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.DeviceProperties; +import com.azure.resourcemanager.sphere.models.DeviceProperties; import org.junit.jupiter.api.Assertions; public final class DevicePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceProperties model = - BinaryData - .fromString( - "{\"deviceId\":\"pvecxgodeb\",\"chipSku\":\"kk\",\"lastAvailableOsVersion\":\"mpukgriw\",\"lastInstalledOsVersion\":\"zlfbxzpuzycispnq\",\"lastOsUpdateUtc\":\"2021-11-25T03:44:38Z\",\"lastUpdateRequestUtc\":\"2021-01-22T12:22:34Z\",\"provisioningState\":\"Succeeded\"}") - .toObject(DeviceProperties.class); - Assertions.assertEquals("pvecxgodeb", model.deviceId()); + DeviceProperties model = BinaryData.fromString( + "{\"deviceId\":\"kcxywnyt\",\"chipSku\":\"synlqidybyxczfc\",\"lastAvailableOsVersion\":\"aaxdbabphlwrq\",\"lastInstalledOsVersion\":\"ktsthsucocmny\",\"lastOsUpdateUtc\":\"2021-08-16T03:52:30Z\",\"lastUpdateRequestUtc\":\"2021-08-27T13:10:53Z\",\"provisioningState\":\"Accepted\"}") + .toObject(DeviceProperties.class); + Assertions.assertEquals("kcxywnyt", model.deviceId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceProperties model = new DeviceProperties().withDeviceId("pvecxgodeb"); + DeviceProperties model = new DeviceProperties().withDeviceId("kcxywnyt"); model = BinaryData.fromObject(model).toObject(DeviceProperties.class); - Assertions.assertEquals("pvecxgodeb", model.deviceId()); + Assertions.assertEquals("kcxywnyt", model.deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdatePropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdatePropertiesTests.java index 3422ec72cc78d..9391bfb2e45d1 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdatePropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdatePropertiesTests.java @@ -5,21 +5,21 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.DeviceUpdateProperties; +import com.azure.resourcemanager.sphere.models.DeviceUpdateProperties; import org.junit.jupiter.api.Assertions; public final class DeviceUpdatePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceUpdateProperties model = - BinaryData.fromString("{\"deviceGroupId\":\"swzts\"}").toObject(DeviceUpdateProperties.class); - Assertions.assertEquals("swzts", model.deviceGroupId()); + DeviceUpdateProperties model + = BinaryData.fromString("{\"deviceGroupId\":\"smtxpsieb\"}").toObject(DeviceUpdateProperties.class); + Assertions.assertEquals("smtxpsieb", model.deviceGroupId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceUpdateProperties model = new DeviceUpdateProperties().withDeviceGroupId("swzts"); + DeviceUpdateProperties model = new DeviceUpdateProperties().withDeviceGroupId("smtxpsieb"); model = BinaryData.fromObject(model).toObject(DeviceUpdateProperties.class); - Assertions.assertEquals("swzts", model.deviceGroupId()); + Assertions.assertEquals("smtxpsieb", model.deviceGroupId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdateTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdateTests.java index c79fa66c09ae7..6847d428acc5a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdateTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DeviceUpdateTests.java @@ -6,20 +6,22 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.models.DeviceUpdate; +import com.azure.resourcemanager.sphere.models.DeviceUpdateProperties; import org.junit.jupiter.api.Assertions; public final class DeviceUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DeviceUpdate model = - BinaryData.fromString("{\"properties\":{\"deviceGroupId\":\"rhdwbavxbniw\"}}").toObject(DeviceUpdate.class); - Assertions.assertEquals("rhdwbavxbniw", model.deviceGroupId()); + DeviceUpdate model + = BinaryData.fromString("{\"properties\":{\"deviceGroupId\":\"edeojnabc\"}}").toObject(DeviceUpdate.class); + Assertions.assertEquals("edeojnabc", model.properties().deviceGroupId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeviceUpdate model = new DeviceUpdate().withDeviceGroupId("rhdwbavxbniw"); + DeviceUpdate model + = new DeviceUpdate().withProperties(new DeviceUpdateProperties().withDeviceGroupId("edeojnabc")); model = BinaryData.fromObject(model).toObject(DeviceUpdate.class); - Assertions.assertEquals("rhdwbavxbniw", model.deviceGroupId()); + Assertions.assertEquals("edeojnabc", model.properties().deviceGroupId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateMockTests.java index 520346241d37c..510dca4e24b13 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesCreateOrUpdateMockTests.java @@ -13,6 +13,7 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; import com.azure.resourcemanager.sphere.models.Device; +import com.azure.resourcemanager.sphere.models.DeviceProperties; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -30,43 +31,28 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"deviceId\":\"wijnh\",\"chipSku\":\"svfycxzbfv\",\"lastAvailableOsVersion\":\"wvrvmtg\",\"lastInstalledOsVersion\":\"ppyostronzmyhgf\",\"lastOsUpdateUtc\":\"2021-12-09T16:00:11Z\",\"lastUpdateRequestUtc\":\"2021-05-13T00:47:52Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"cwaekrrjre\",\"name\":\"fxtsgum\",\"type\":\"jglikkxwslolb\"}"; + String responseStr + = "{\"properties\":{\"deviceId\":\"lvez\",\"chipSku\":\"pqlmfe\",\"lastAvailableOsVersion\":\"erqwkyhkobopg\",\"lastInstalledOsVersion\":\"dkow\",\"lastOsUpdateUtc\":\"2021-04-05T06:25:25Z\",\"lastUpdateRequestUtc\":\"2021-01-20T13:15:44Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"kbwcc\",\"name\":\"njv\",\"type\":\"dw\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Device response = - manager - .devices() - .define("vblm") - .withExistingDeviceGroup("mfiibfggj", "ool", "rwxkvtkkgl", "qwjygvja") - .withDeviceId("zuhbxvvyhgsopb") - .create(); + Device response = manager.devices().define("vtbvkayh") + .withExistingDeviceGroup("whbotzingamv", "phoszqz", "dphqamv", "kfwynw") + .withProperties(new DeviceProperties().withDeviceId("vyqia")).create(); - Assertions.assertEquals("wijnh", response.deviceId()); + Assertions.assertEquals("lvez", response.properties().deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteMockTests.java index b8d82fa33e5b1..956e9375ddc33 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesDeleteMockTests.java @@ -32,32 +32,21 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .devices() - .delete("ob", "pg", "edkowepbqpcrfk", "wccsnjvcdwxlpqek", "tn", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.devices().delete("dhszfjv", "bgofeljag", "qmqhldvriii", "jnalghf", "vtvsexsowueluq", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageMockTests.java index a860a2190fcb6..6266db384a8c7 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGenerateCapabilityImageMockTests.java @@ -32,50 +32,28 @@ public void testGenerateCapabilityImage() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"image\":\"qqaatjinrvgou\"}"; + String responseStr = "{\"image\":\"fi\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + SignedCapabilityImageResponse response + = manager.devices().generateCapabilityImage("hahhxvrhmzkwpj", "wws", "ughftqsx", "qxujxukndxd", "grjguufzd", + new GenerateCapabilityImageRequest().withCapabilities( + Arrays.asList(CapabilityType.APPLICATION_DEVELOPMENT, CapabilityType.FIELD_SERVICING)), + com.azure.core.util.Context.NONE); - SignedCapabilityImageResponse response = - manager - .devices() - .generateCapabilityImage( - "htjsying", - "fq", - "tmtdhtmdvypgik", - "gszywk", - "irryuzhlh", - new GenerateCapabilityImageRequest() - .withCapabilities( - Arrays - .asList( - CapabilityType.FIELD_SERVICING, - CapabilityType.APPLICATION_DEVELOPMENT, - CapabilityType.APPLICATION_DEVELOPMENT)), - com.azure.core.util.Context.NONE); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGetWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGetWithResponseMockTests.java index bbe62979f7baf..f8ec6643bb4ad 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGetWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesGetWithResponseMockTests.java @@ -30,47 +30,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"deviceId\":\"ygz\",\"chipSku\":\"dnkfx\",\"lastAvailableOsVersion\":\"emdwzrmuhapfc\",\"lastInstalledOsVersion\":\"psqxq\",\"lastOsUpdateUtc\":\"2021-08-14T15:35:59Z\",\"lastUpdateRequestUtc\":\"2021-05-20T12:44:27Z\",\"provisioningState\":\"Canceled\"},\"id\":\"mgccelvezrypq\",\"name\":\"mfe\",\"type\":\"kerqwkyh\"}"; + String responseStr + = "{\"properties\":{\"deviceId\":\"ikpzimejza\",\"chipSku\":\"fzxiavrmb\",\"lastAvailableOsVersion\":\"nokixrjqcirgz\",\"lastInstalledOsVersion\":\"rlazszrnw\",\"lastOsUpdateUtc\":\"2021-07-10T05:57:56Z\",\"lastUpdateRequestUtc\":\"2021-03-11T03:05:15Z\",\"provisioningState\":\"Accepted\"},\"id\":\"pj\",\"name\":\"lwbtlhf\",\"type\":\"sj\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Device response = - manager - .devices() - .getWithResponse( - "amvdkfwynwcvtbv", - "ayhmtnvyqiatkz", - "pcnp", - "zcjaesgvvsccy", - "jguq", - com.azure.core.util.Context.NONE) - .getValue(); + Device response = manager.devices().getWithResponse("slthaq", "x", "smwutwbdsrezpd", "hneuyowqkd", "ytisibir", + com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("ygz", response.deviceId()); + Assertions.assertEquals("ikpzimejza", response.properties().deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupMockTests.java index 4d4bb7c5ec89d..be59d4cf54a59 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/DevicesListByDeviceGroupMockTests.java @@ -31,41 +31,27 @@ public void testListByDeviceGroup() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"deviceId\":\"spughftqsxhq\",\"chipSku\":\"j\",\"lastAvailableOsVersion\":\"kndxdigrjgu\",\"lastInstalledOsVersion\":\"zdmsyqtfi\",\"lastOsUpdateUtc\":\"2020-12-25T23:20:44Z\",\"lastUpdateRequestUtc\":\"2021-09-23T10:33:57Z\",\"provisioningState\":\"Updating\"},\"id\":\"ingamvp\",\"name\":\"ho\",\"type\":\"zqzudph\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"deviceId\":\"o\",\"chipSku\":\"hjjklff\",\"lastAvailableOsVersion\":\"ouw\",\"lastInstalledOsVersion\":\"gzrf\",\"lastOsUpdateUtc\":\"2021-01-08T15:42:56Z\",\"lastUpdateRequestUtc\":\"2021-04-04T13:09:14Z\",\"provisioningState\":\"Canceled\"},\"id\":\"ikayuhqlbjbsybb\",\"name\":\"wrv\",\"type\":\"ldgmfpgvmpip\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .devices() - .listByDeviceGroup( - "qmqhldvriii", "jnalghf", "vtvsexsowueluq", "hahhxvrhmzkwpj", com.azure.core.util.Context.NONE); + PagedIterable response = manager.devices().listByDeviceGroup("sjabibs", "stawfsdjpvkv", "bjxbkzbzk", + "vncjabudurgk", com.azure.core.util.Context.NONE); - Assertions.assertEquals("spughftqsxhq", response.iterator().next().deviceId()); + Assertions.assertEquals("o", response.iterator().next().properties().deviceId()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/GenerateCapabilityImageRequestTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/GenerateCapabilityImageRequestTests.java index 94189e9c6adc7..53e36e9b5bea8 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/GenerateCapabilityImageRequestTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/GenerateCapabilityImageRequestTests.java @@ -13,19 +13,17 @@ public final class GenerateCapabilityImageRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GenerateCapabilityImageRequest model = - BinaryData - .fromString("{\"capabilities\":[\"ApplicationDevelopment\"]}") + GenerateCapabilityImageRequest model + = BinaryData.fromString("{\"capabilities\":[\"FieldServicing\",\"FieldServicing\",\"FieldServicing\"]}") .toObject(GenerateCapabilityImageRequest.class); - Assertions.assertEquals(CapabilityType.APPLICATION_DEVELOPMENT, model.capabilities().get(0)); + Assertions.assertEquals(CapabilityType.FIELD_SERVICING, model.capabilities().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GenerateCapabilityImageRequest model = - new GenerateCapabilityImageRequest() - .withCapabilities(Arrays.asList(CapabilityType.APPLICATION_DEVELOPMENT)); + GenerateCapabilityImageRequest model = new GenerateCapabilityImageRequest().withCapabilities(Arrays + .asList(CapabilityType.FIELD_SERVICING, CapabilityType.FIELD_SERVICING, CapabilityType.FIELD_SERVICING)); model = BinaryData.fromObject(model).toObject(GenerateCapabilityImageRequest.class); - Assertions.assertEquals(CapabilityType.APPLICATION_DEVELOPMENT, model.capabilities().get(0)); + Assertions.assertEquals(CapabilityType.FIELD_SERVICING, model.capabilities().get(0)); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageInnerTests.java index f234938a021bc..e2d472cd92f19 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageInnerTests.java @@ -6,32 +6,28 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import org.junit.jupiter.api.Assertions; public final class ImageInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ImageInner model = - BinaryData - .fromString( - "{\"properties\":{\"image\":\"mbe\",\"imageId\":\"pbhtqqrolfpfpsa\",\"imageName\":\"bquxigjy\",\"regionalDataBoundary\":\"EU\",\"uri\":\"aoyfhrtxilnerkuj\",\"description\":\"vlejuvfqa\",\"componentId\":\"lyxwjkcprbnwbx\",\"imageType\":\"OneBl\",\"provisioningState\":\"Provisioning\"},\"id\":\"vpys\",\"name\":\"zdn\",\"type\":\"uj\"}") - .toObject(ImageInner.class); - Assertions.assertEquals("mbe", model.image()); - Assertions.assertEquals("pbhtqqrolfpfpsa", model.imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); + ImageInner model = BinaryData.fromString( + "{\"properties\":{\"image\":\"ymwisdkft\",\"imageId\":\"xmnteiwaop\",\"imageName\":\"mijcmmxdcufufs\",\"regionalDataBoundary\":\"None\",\"uri\":\"zidnsezcxtbzsgfy\",\"description\":\"sne\",\"componentId\":\"dwzjeiach\",\"imageType\":\"CustomerBoardConfig\",\"provisioningState\":\"Failed\"},\"id\":\"nrosfqpte\",\"name\":\"hzzvypyq\",\"type\":\"i\"}") + .toObject(ImageInner.class); + Assertions.assertEquals("ymwisdkft", model.properties().image()); + Assertions.assertEquals("xmnteiwaop", model.properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImageInner model = - new ImageInner() - .withImage("mbe") - .withImageId("pbhtqqrolfpfpsa") - .withRegionalDataBoundary(RegionalDataBoundary.EU); + ImageInner model = new ImageInner().withProperties(new ImageProperties().withImage("ymwisdkft") + .withImageId("xmnteiwaop").withRegionalDataBoundary(RegionalDataBoundary.NONE)); model = BinaryData.fromObject(model).toObject(ImageInner.class); - Assertions.assertEquals("mbe", model.image()); - Assertions.assertEquals("pbhtqqrolfpfpsa", model.imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); + Assertions.assertEquals("ymwisdkft", model.properties().image()); + Assertions.assertEquals("xmnteiwaop", model.properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageListResultTests.java index 1178f1b98e760..e94c1ff4c9ca6 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImageListResultTests.java @@ -7,6 +7,7 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.ImageInner; import com.azure.resourcemanager.sphere.models.ImageListResult; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import java.util.Arrays; import org.junit.jupiter.api.Assertions; @@ -14,41 +15,24 @@ public final class ImageListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ImageListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"image\":\"duuji\",\"imageId\":\"jczdzevndh\",\"imageName\":\"wpdappdsbdkv\",\"regionalDataBoundary\":\"EU\",\"uri\":\"feusnhut\",\"description\":\"ltmrldh\",\"componentId\":\"jzzd\",\"imageType\":\"Services\",\"provisioningState\":\"Provisioning\"},\"id\":\"oc\",\"name\":\"geablgphuticndvk\",\"type\":\"ozwyiftyhxhuro\"},{\"properties\":{\"image\":\"yxolniwp\",\"imageId\":\"ukjfkgiawxklr\",\"imageName\":\"lwckbasyypnddhs\",\"regionalDataBoundary\":\"EU\",\"uri\":\"cph\",\"description\":\"koty\",\"componentId\":\"gou\",\"imageType\":\"UpdateCertStore\",\"provisioningState\":\"Accepted\"},\"id\":\"i\",\"name\":\"wyqkgfgibm\",\"type\":\"dgak\"},{\"properties\":{\"image\":\"rxybz\",\"imageId\":\"e\",\"imageName\":\"ytb\",\"regionalDataBoundary\":\"None\",\"uri\":\"ouf\",\"description\":\"mnkzsmod\",\"componentId\":\"lougpbkw\",\"imageType\":\"ManifestSet\",\"provisioningState\":\"Failed\"},\"id\":\"uqktap\",\"name\":\"pwgcuertu\",\"type\":\"kdosvqw\"}],\"nextLink\":\"mdgbbjfdd\"}") - .toObject(ImageListResult.class); - Assertions.assertEquals("duuji", model.value().get(0).image()); - Assertions.assertEquals("jczdzevndh", model.value().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.value().get(0).regionalDataBoundary()); - Assertions.assertEquals("mdgbbjfdd", model.nextLink()); + ImageListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"image\":\"fouflmmnkzsmo\",\"imageId\":\"glougpbk\",\"imageName\":\"mutduqktaps\",\"regionalDataBoundary\":\"None\",\"uri\":\"uertumk\",\"description\":\"svqwhbmdgbbjfd\",\"componentId\":\"mbmbexppbh\",\"imageType\":\"SecurityMonitor\",\"provisioningState\":\"Accepted\"},\"id\":\"fpfpsalgbquxigj\",\"name\":\"jgzjaoyfhrtx\",\"type\":\"lnerkujysvleju\"},{\"properties\":{\"image\":\"awrlyx\",\"imageId\":\"kcprbnw\",\"imageName\":\"gjvtbv\",\"regionalDataBoundary\":\"None\",\"uri\":\"zdn\",\"description\":\"jq\",\"componentId\":\"hmuouqfprwzwbn\",\"imageType\":\"Applications\",\"provisioningState\":\"Provisioning\"},\"id\":\"uizga\",\"name\":\"x\",\"type\":\"fizuckyf\"}],\"nextLink\":\"rfidfvzwdz\"}") + .toObject(ImageListResult.class); + Assertions.assertEquals("fouflmmnkzsmo", model.value().get(0).properties().image()); + Assertions.assertEquals("glougpbk", model.value().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.value().get(0).properties().regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImageListResult model = - new ImageListResult() - .withValue( - Arrays - .asList( - new ImageInner() - .withImage("duuji") - .withImageId("jczdzevndh") - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new ImageInner() - .withImage("yxolniwp") - .withImageId("ukjfkgiawxklr") - .withRegionalDataBoundary(RegionalDataBoundary.EU), - new ImageInner() - .withImage("rxybz") - .withImageId("e") - .withRegionalDataBoundary(RegionalDataBoundary.NONE))) - .withNextLink("mdgbbjfdd"); + ImageListResult model = new ImageListResult().withValue(Arrays.asList( + new ImageInner().withProperties(new ImageProperties().withImage("fouflmmnkzsmo").withImageId("glougpbk") + .withRegionalDataBoundary(RegionalDataBoundary.NONE)), + new ImageInner().withProperties(new ImageProperties().withImage("awrlyx").withImageId("kcprbnw") + .withRegionalDataBoundary(RegionalDataBoundary.NONE)))); model = BinaryData.fromObject(model).toObject(ImageListResult.class); - Assertions.assertEquals("duuji", model.value().get(0).image()); - Assertions.assertEquals("jczdzevndh", model.value().get(0).imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, model.value().get(0).regionalDataBoundary()); - Assertions.assertEquals("mdgbbjfdd", model.nextLink()); + Assertions.assertEquals("fouflmmnkzsmo", model.value().get(0).properties().image()); + Assertions.assertEquals("glougpbk", model.value().get(0).properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, model.value().get(0).properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagePropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagePropertiesTests.java index 2ccbbac3225ad..e62619e021c31 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagePropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagePropertiesTests.java @@ -5,33 +5,28 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.ImageProperties; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import org.junit.jupiter.api.Assertions; public final class ImagePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ImageProperties model = - BinaryData - .fromString( - "{\"image\":\"uhmuouqfprwzwbn\",\"imageId\":\"itnwuizgazxufi\",\"imageName\":\"ckyfih\",\"regionalDataBoundary\":\"None\",\"uri\":\"fvzwdzuhty\",\"description\":\"isdkfthwxmnteiw\",\"componentId\":\"pvkmijcmmxdcuf\",\"imageType\":\"FwConfig\",\"provisioningState\":\"Accepted\"}") - .toObject(ImageProperties.class); - Assertions.assertEquals("uhmuouqfprwzwbn", model.image()); - Assertions.assertEquals("itnwuizgazxufi", model.imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); + ImageProperties model = BinaryData.fromString( + "{\"image\":\"inpvswjdkirsoodq\",\"imageId\":\"crmnohjtckwhds\",\"imageName\":\"fiyipjxsqwpgrj\",\"regionalDataBoundary\":\"EU\",\"uri\":\"rcjxvsnbyxqabn\",\"description\":\"cpc\",\"componentId\":\"hurzafblj\",\"imageType\":\"InvalidImageType\",\"provisioningState\":\"Provisioning\"}") + .toObject(ImageProperties.class); + Assertions.assertEquals("inpvswjdkirsoodq", model.image()); + Assertions.assertEquals("crmnohjtckwhds", model.imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImageProperties model = - new ImageProperties() - .withImage("uhmuouqfprwzwbn") - .withImageId("itnwuizgazxufi") - .withRegionalDataBoundary(RegionalDataBoundary.NONE); + ImageProperties model = new ImageProperties().withImage("inpvswjdkirsoodq").withImageId("crmnohjtckwhds") + .withRegionalDataBoundary(RegionalDataBoundary.EU); model = BinaryData.fromObject(model).toObject(ImageProperties.class); - Assertions.assertEquals("uhmuouqfprwzwbn", model.image()); - Assertions.assertEquals("itnwuizgazxufi", model.imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, model.regionalDataBoundary()); + Assertions.assertEquals("inpvswjdkirsoodq", model.image()); + Assertions.assertEquals("crmnohjtckwhds", model.imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, model.regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateMockTests.java index f4ae052642594..d7da725227b10 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesCreateOrUpdateMockTests.java @@ -13,6 +13,7 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; import com.azure.resourcemanager.sphere.models.Image; +import com.azure.resourcemanager.sphere.models.ImageProperties; import com.azure.resourcemanager.sphere.models.RegionalDataBoundary; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -31,47 +32,31 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"image\":\"smond\",\"imageId\":\"quxvypomgkop\",\"imageName\":\"hojvpajqgxysmocm\",\"regionalDataBoundary\":\"None\",\"uri\":\"vmkcx\",\"description\":\"apvhelxprgly\",\"componentId\":\"dd\",\"imageType\":\"InvalidImageType\",\"provisioningState\":\"Succeeded\"},\"id\":\"uejrjxgc\",\"name\":\"qibrhosxsdqrhzoy\",\"type\":\"i\"}"; + String responseStr + = "{\"properties\":{\"image\":\"vqtmnub\",\"imageId\":\"kpzksmondjmq\",\"imageName\":\"vypomgkopkwho\",\"regionalDataBoundary\":\"EU\",\"uri\":\"jqg\",\"description\":\"smocmbq\",\"componentId\":\"vmkcx\",\"imageType\":\"Nwfs\",\"provisioningState\":\"Succeeded\"},\"id\":\"elxprglyatddck\",\"name\":\"bcuejrjxgci\",\"type\":\"ibrhosxsdqr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Image response = - manager - .images() - .define("ld") - .withExistingCatalog("sotbob", "dopcjwvnh") - .withImage("xcxrsl") - .withImageId("utwu") - .withRegionalDataBoundary(RegionalDataBoundary.NONE) - .create(); - - Assertions.assertEquals("smond", response.image()); - Assertions.assertEquals("quxvypomgkop", response.imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.regionalDataBoundary()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Image response = manager.images().define("apnedgfbcvkc").withExistingCatalog("hdlxyjrxsagafcn", "hgw") + .withProperties(new ImageProperties().withImage("pkeqdcvdrhvoo").withImageId("otbobzdopcj") + .withRegionalDataBoundary(RegionalDataBoundary.EU)) + .create(); + + Assertions.assertEquals("vqtmnub", response.properties().image()); + Assertions.assertEquals("kpzksmondjmq", response.properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, response.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteMockTests.java index f240ed4fe1e8c..f171bb9b80888 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.images().delete("apnedgfbcvkc", "q", "pkeqdcvdrhvoo", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.images().delete("a", "lvpnpp", "uflrwd", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesGetWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesGetWithResponseMockTests.java index 42ed4f7bc3cee..c1a692c3bbf74 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesGetWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesGetWithResponseMockTests.java @@ -31,43 +31,29 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"image\":\"clvit\",\"imageId\":\"qzonosggbhcohf\",\"imageName\":\"sjnkal\",\"regionalDataBoundary\":\"EU\",\"uri\":\"iswac\",\"description\":\"gdkz\",\"componentId\":\"wkfvhqcrailvp\",\"imageType\":\"UpdateCertStore\",\"provisioningState\":\"Succeeded\"},\"id\":\"flrwd\",\"name\":\"hdlxyjrxsagafcn\",\"type\":\"hgw\"}"; + String responseStr + = "{\"properties\":{\"image\":\"lkfg\",\"imageId\":\"dneu\",\"imageName\":\"fphsdyhtozfikdow\",\"regionalDataBoundary\":\"None\",\"uri\":\"v\",\"description\":\"xclvit\",\"componentId\":\"qzonosggbhcohf\",\"imageType\":\"OneBl\",\"provisioningState\":\"Accepted\"},\"id\":\"aljutiiswac\",\"name\":\"fgdkzzew\",\"type\":\"fvhqc\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Image response = - manager - .images() - .getWithResponse("fgohdneuelfphs", "yhtozfikdowwqu", "v", com.azure.core.util.Context.NONE) - .getValue(); + Image response = manager.images() + .getWithResponse("mpgcjefuzmuvpbt", "d", "morppxebmnzbtbh", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("clvit", response.image()); - Assertions.assertEquals("qzonosggbhcohf", response.imageId()); - Assertions.assertEquals(RegionalDataBoundary.EU, response.regionalDataBoundary()); + Assertions.assertEquals("lkfg", response.properties().image()); + Assertions.assertEquals("dneu", response.properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.NONE, response.properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogMockTests.java index 7ca2213eed516..84e496ebe6112 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ImagesListByCatalogMockTests.java @@ -32,49 +32,30 @@ public void testListByCatalog() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"image\":\"xbxwa\",\"imageId\":\"ogqxndlkzgxhuri\",\"imageName\":\"bpodxunkbebxm\",\"regionalDataBoundary\":\"None\",\"uri\":\"ntwlrbqtkoie\",\"description\":\"eotg\",\"componentId\":\"l\",\"imageType\":\"RecoveryManifest\",\"provisioningState\":\"Canceled\"},\"id\":\"lauwzizxbmpgcjef\",\"name\":\"zmuvpbttdumorppx\",\"type\":\"bmnzbtbhjpgl\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"image\":\"tfz\",\"imageId\":\"hhvh\",\"imageName\":\"r\",\"regionalDataBoundary\":\"EU\",\"uri\":\"wobdagxtibqdx\",\"description\":\"wakbogqxndl\",\"componentId\":\"gxhuriplbp\",\"imageType\":\"Services\",\"provisioningState\":\"Deleting\"},\"id\":\"bebxmubyyntwl\",\"name\":\"bqtkoievseotgqr\",\"type\":\"ltmuwlauwzizx\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .images() - .listByCatalog( - "dltfz", - "mhhv", - "gureodkwobdag", - 933543231, - 1861593827, - 1583051460, - com.azure.core.util.Context.NONE); - - Assertions.assertEquals("xbxwa", response.iterator().next().image()); - Assertions.assertEquals("ogqxndlkzgxhuri", response.iterator().next().imageId()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.iterator().next().regionalDataBoundary()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.images().listByCatalog("mwmbes", "dnkwwtppjflcxog", "okonzmnsikvmkqz", + 1984679669, 1625673068, 1138945921, com.azure.core.util.Context.NONE); + + Assertions.assertEquals("tfz", response.iterator().next().properties().image()); + Assertions.assertEquals("hhvh", response.iterator().next().properties().imageId()); + Assertions.assertEquals(RegionalDataBoundary.EU, + response.iterator().next().properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ListDeviceGroupsRequestTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ListDeviceGroupsRequestTests.java index 388386523a1c7..70968a65ae0eb 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ListDeviceGroupsRequestTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ListDeviceGroupsRequestTests.java @@ -11,15 +11,15 @@ public final class ListDeviceGroupsRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ListDeviceGroupsRequest model = - BinaryData.fromString("{\"deviceGroupName\":\"yexfwh\"}").toObject(ListDeviceGroupsRequest.class); - Assertions.assertEquals("yexfwh", model.deviceGroupName()); + ListDeviceGroupsRequest model + = BinaryData.fromString("{\"deviceGroupName\":\"q\"}").toObject(ListDeviceGroupsRequest.class); + Assertions.assertEquals("q", model.deviceGroupName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ListDeviceGroupsRequest model = new ListDeviceGroupsRequest().withDeviceGroupName("yexfwh"); + ListDeviceGroupsRequest model = new ListDeviceGroupsRequest().withDeviceGroupName("q"); model = BinaryData.fromObject(model).toObject(ListDeviceGroupsRequest.class); - Assertions.assertEquals("yexfwh", model.deviceGroupName()); + Assertions.assertEquals("q", model.deviceGroupName()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationDisplayTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationDisplayTests.java index 7b49a47063ce6..919921646ba46 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationDisplayTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationDisplayTests.java @@ -10,11 +10,9 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"yrtih\",\"resource\":\"tijbpzvgnwzsymgl\",\"operation\":\"fcyzkohdbihanufh\",\"description\":\"bj\"}") - .toObject(OperationDisplay.class); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"yrtih\",\"resource\":\"tijbpzvgnwzsymgl\",\"operation\":\"fcyzkohdbihanufh\",\"description\":\"bj\"}") + .toObject(OperationDisplay.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationInnerTests.java index 2440972f8cc6f..f30b7af5fbae9 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationInnerTests.java @@ -11,11 +11,9 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"usarhmofc\",\"isDataAction\":false,\"display\":{\"provider\":\"urkdtmlx\",\"resource\":\"kuksjtxukcdm\",\"operation\":\"rcryuanzwuxzdxta\",\"description\":\"lhmwhfpmrqobm\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") - .toObject(OperationInner.class); + OperationInner model = BinaryData.fromString( + "{\"name\":\"usarhmofc\",\"isDataAction\":false,\"display\":{\"provider\":\"urkdtmlx\",\"resource\":\"kuksjtxukcdm\",\"operation\":\"rcryuanzwuxzdxta\",\"description\":\"lhmwhfpmrqobm\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") + .toObject(OperationInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationListResultTests.java index 1e4a38ab5f2e8..da51bda792f98 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationListResultTests.java @@ -10,11 +10,9 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"quvgjxpybczme\",\"isDataAction\":true,\"display\":{\"provider\":\"pbsphrupidgs\",\"resource\":\"bejhphoycmsxa\",\"operation\":\"hdxbmtqio\",\"description\":\"zehtbmu\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"izhwlrxy\",\"isDataAction\":false,\"display\":{\"provider\":\"ijgkdm\",\"resource\":\"azlobcufpdznrbt\",\"operation\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"rifkwm\",\"isDataAction\":true,\"display\":{\"provider\":\"izntocipao\",\"resource\":\"jpsq\",\"operation\":\"mpoyfd\",\"description\":\"ogknygjofjdd\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"upewnwreitjzy\"}") - .toObject(OperationListResult.class); + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"quvgjxpybczme\",\"isDataAction\":true,\"display\":{\"provider\":\"pbsphrupidgs\",\"resource\":\"bejhphoycmsxa\",\"operation\":\"hdxbmtqio\",\"description\":\"zehtbmu\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"izhwlrxy\",\"isDataAction\":false,\"display\":{\"provider\":\"ijgkdm\",\"resource\":\"azlobcufpdznrbt\",\"operation\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"rifkwm\",\"isDataAction\":true,\"display\":{\"provider\":\"izntocipao\",\"resource\":\"jpsq\",\"operation\":\"mpoyfd\",\"description\":\"ogknygjofjdd\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"upewnwreitjzy\"}") + .toObject(OperationListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationsListMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationsListMockTests.java index b2433e37aaf2f..07a588a03fa11 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationsListMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/OperationsListMockTests.java @@ -30,35 +30,25 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"xbzpfzab\",\"isDataAction\":false,\"display\":{\"provider\":\"xwtctyqiklbbovpl\",\"resource\":\"bhvgy\",\"operation\":\"uosvmkfssxqukk\",\"description\":\"l\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}]}"; + String responseStr + = "{\"value\":[{\"name\":\"hjjdhtldwkyzxuut\",\"isDataAction\":true,\"display\":{\"provider\":\"cwsvlxotog\",\"resource\":\"rupqsxvnmicy\",\"operation\":\"ceoveilovno\",\"description\":\"fj\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/PagedDeviceInsightTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/PagedDeviceInsightTests.java index dbef79838d8f0..60d98a88b949f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/PagedDeviceInsightTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/PagedDeviceInsightTests.java @@ -14,57 +14,42 @@ public final class PagedDeviceInsightTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PagedDeviceInsight model = - BinaryData - .fromString( - "{\"value\":[{\"deviceId\":\"w\",\"description\":\"hzdobpxjmflbvvnc\",\"startTimestampUtc\":\"2021-11-01T14:14:41Z\",\"endTimestampUtc\":\"2021-05-23T11:48:05Z\",\"eventCategory\":\"cciw\",\"eventClass\":\"zjuqkhrsaj\",\"eventType\":\"wkuofoskghsauu\",\"eventCount\":2118838793},{\"deviceId\":\"jmvxie\",\"description\":\"uugidyjrrfby\",\"startTimestampUtc\":\"2021-09-15T05:38:02Z\",\"endTimestampUtc\":\"2021-05-02T14:53:40Z\",\"eventCategory\":\"v\",\"eventClass\":\"xc\",\"eventType\":\"onpc\",\"eventCount\":316608134}],\"nextLink\":\"cohslkev\"}") - .toObject(PagedDeviceInsight.class); - Assertions.assertEquals("w", model.value().get(0).deviceId()); - Assertions.assertEquals("hzdobpxjmflbvvnc", model.value().get(0).description()); - Assertions.assertEquals(OffsetDateTime.parse("2021-11-01T14:14:41Z"), model.value().get(0).startTimestampUtc()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-23T11:48:05Z"), model.value().get(0).endTimestampUtc()); - Assertions.assertEquals("cciw", model.value().get(0).eventCategory()); - Assertions.assertEquals("zjuqkhrsaj", model.value().get(0).eventClass()); - Assertions.assertEquals("wkuofoskghsauu", model.value().get(0).eventType()); - Assertions.assertEquals(2118838793, model.value().get(0).eventCount()); - Assertions.assertEquals("cohslkev", model.nextLink()); + PagedDeviceInsight model = BinaryData.fromString( + "{\"value\":[{\"deviceId\":\"rsa\",\"description\":\"iwkuofos\",\"startTimestampUtc\":\"2021-06-28T14:02:06Z\",\"endTimestampUtc\":\"2021-03-08T03:12:48Z\",\"eventCategory\":\"sauuimj\",\"eventClass\":\"vxieduugidyj\",\"eventType\":\"rfbyaosvexcso\",\"eventCount\":771742090},{\"deviceId\":\"clhocohsl\",\"description\":\"ev\",\"startTimestampUtc\":\"2021-02-15T02:43:43Z\",\"endTimestampUtc\":\"2021-07-27T15:45:18Z\",\"eventCategory\":\"gz\",\"eventClass\":\"buhfmvfaxkffeiit\",\"eventType\":\"lvmezyvshxmzsbbz\",\"eventCount\":681165656},{\"deviceId\":\"gigr\",\"description\":\"wburvjxxjnspydpt\",\"startTimestampUtc\":\"2021-07-25T09:18:50Z\",\"endTimestampUtc\":\"2021-09-24T03:08Z\",\"eventCategory\":\"nkoukn\",\"eventClass\":\"udwtiukbl\",\"eventType\":\"ngkpocipazy\",\"eventCount\":212177726}],\"nextLink\":\"gukgjnpiucgygevq\"}") + .toObject(PagedDeviceInsight.class); + Assertions.assertEquals("rsa", model.value().get(0).deviceId()); + Assertions.assertEquals("iwkuofos", model.value().get(0).description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-28T14:02:06Z"), model.value().get(0).startTimestampUtc()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-08T03:12:48Z"), model.value().get(0).endTimestampUtc()); + Assertions.assertEquals("sauuimj", model.value().get(0).eventCategory()); + Assertions.assertEquals("vxieduugidyj", model.value().get(0).eventClass()); + Assertions.assertEquals("rfbyaosvexcso", model.value().get(0).eventType()); + Assertions.assertEquals(771742090, model.value().get(0).eventCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PagedDeviceInsight model = - new PagedDeviceInsight() - .withValue( - Arrays - .asList( - new DeviceInsightInner() - .withDeviceId("w") - .withDescription("hzdobpxjmflbvvnc") - .withStartTimestampUtc(OffsetDateTime.parse("2021-11-01T14:14:41Z")) - .withEndTimestampUtc(OffsetDateTime.parse("2021-05-23T11:48:05Z")) - .withEventCategory("cciw") - .withEventClass("zjuqkhrsaj") - .withEventType("wkuofoskghsauu") - .withEventCount(2118838793), - new DeviceInsightInner() - .withDeviceId("jmvxie") - .withDescription("uugidyjrrfby") - .withStartTimestampUtc(OffsetDateTime.parse("2021-09-15T05:38:02Z")) - .withEndTimestampUtc(OffsetDateTime.parse("2021-05-02T14:53:40Z")) - .withEventCategory("v") - .withEventClass("xc") - .withEventType("onpc") - .withEventCount(316608134))) - .withNextLink("cohslkev"); + PagedDeviceInsight model = new PagedDeviceInsight().withValue(Arrays.asList( + new DeviceInsightInner().withDeviceId("rsa").withDescription("iwkuofos") + .withStartTimestampUtc(OffsetDateTime.parse("2021-06-28T14:02:06Z")) + .withEndTimestampUtc(OffsetDateTime.parse("2021-03-08T03:12:48Z")).withEventCategory("sauuimj") + .withEventClass("vxieduugidyj").withEventType("rfbyaosvexcso").withEventCount(771742090), + new DeviceInsightInner().withDeviceId("clhocohsl").withDescription("ev") + .withStartTimestampUtc(OffsetDateTime.parse("2021-02-15T02:43:43Z")) + .withEndTimestampUtc(OffsetDateTime.parse("2021-07-27T15:45:18Z")).withEventCategory("gz") + .withEventClass("buhfmvfaxkffeiit").withEventType("lvmezyvshxmzsbbz").withEventCount(681165656), + new DeviceInsightInner().withDeviceId("gigr").withDescription("wburvjxxjnspydpt") + .withStartTimestampUtc(OffsetDateTime.parse("2021-07-25T09:18:50Z")) + .withEndTimestampUtc(OffsetDateTime.parse("2021-09-24T03:08Z")).withEventCategory("nkoukn") + .withEventClass("udwtiukbl").withEventType("ngkpocipazy").withEventCount(212177726))); model = BinaryData.fromObject(model).toObject(PagedDeviceInsight.class); - Assertions.assertEquals("w", model.value().get(0).deviceId()); - Assertions.assertEquals("hzdobpxjmflbvvnc", model.value().get(0).description()); - Assertions.assertEquals(OffsetDateTime.parse("2021-11-01T14:14:41Z"), model.value().get(0).startTimestampUtc()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-23T11:48:05Z"), model.value().get(0).endTimestampUtc()); - Assertions.assertEquals("cciw", model.value().get(0).eventCategory()); - Assertions.assertEquals("zjuqkhrsaj", model.value().get(0).eventClass()); - Assertions.assertEquals("wkuofoskghsauu", model.value().get(0).eventType()); - Assertions.assertEquals(2118838793, model.value().get(0).eventCount()); - Assertions.assertEquals("cohslkev", model.nextLink()); + Assertions.assertEquals("rsa", model.value().get(0).deviceId()); + Assertions.assertEquals("iwkuofos", model.value().get(0).description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-28T14:02:06Z"), model.value().get(0).startTimestampUtc()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-08T03:12:48Z"), model.value().get(0).endTimestampUtc()); + Assertions.assertEquals("sauuimj", model.value().get(0).eventCategory()); + Assertions.assertEquals("vxieduugidyj", model.value().get(0).eventClass()); + Assertions.assertEquals("rfbyaosvexcso", model.value().get(0).eventType()); + Assertions.assertEquals(771742090, model.value().get(0).eventCount()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductInnerTests.java index 7080cd4500841..aff0397719aaf 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductInnerTests.java @@ -6,23 +6,22 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; +import com.azure.resourcemanager.sphere.models.ProductProperties; import org.junit.jupiter.api.Assertions; public final class ProductInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProductInner model = - BinaryData - .fromString( - "{\"properties\":{\"description\":\"pnazzm\",\"provisioningState\":\"Updating\"},\"id\":\"unmpxttd\",\"name\":\"hrbnlankxmyskpbh\",\"type\":\"nbtkcxywnytnr\"}") - .toObject(ProductInner.class); - Assertions.assertEquals("pnazzm", model.description()); + ProductInner model = BinaryData.fromString( + "{\"properties\":{\"description\":\"s\",\"provisioningState\":\"Provisioning\"},\"id\":\"nxytxh\",\"name\":\"zxbzpfzabglc\",\"type\":\"hxw\"}") + .toObject(ProductInner.class); + Assertions.assertEquals("s", model.properties().description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProductInner model = new ProductInner().withDescription("pnazzm"); + ProductInner model = new ProductInner().withProperties(new ProductProperties().withDescription("s")); model = BinaryData.fromObject(model).toObject(ProductInner.class); - Assertions.assertEquals("pnazzm", model.description()); + Assertions.assertEquals("s", model.properties().description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductListResultTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductListResultTests.java index d3e581055b777..f720a65ce7e6f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductListResultTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductListResultTests.java @@ -7,29 +7,24 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.fluent.models.ProductInner; import com.azure.resourcemanager.sphere.models.ProductListResult; +import com.azure.resourcemanager.sphere.models.ProductProperties; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class ProductListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProductListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"description\":\"yydhibnuqqk\",\"provisioningState\":\"Accepted\"},\"id\":\"a\",\"name\":\"rgvtqag\",\"type\":\"buynhijggm\"}],\"nextLink\":\"fsiarbutr\"}") - .toObject(ProductListResult.class); - Assertions.assertEquals("yydhibnuqqk", model.value().get(0).description()); - Assertions.assertEquals("fsiarbutr", model.nextLink()); + ProductListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"description\":\"qpuedckzywbiex\",\"provisioningState\":\"Provisioning\"},\"id\":\"ue\",\"name\":\"xibxujwbhqwalm\",\"type\":\"zyoxaepdkzjan\"}],\"nextLink\":\"xrhdwbavxbniwdjs\"}") + .toObject(ProductListResult.class); + Assertions.assertEquals("qpuedckzywbiex", model.value().get(0).properties().description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProductListResult model = - new ProductListResult() - .withValue(Arrays.asList(new ProductInner().withDescription("yydhibnuqqk"))) - .withNextLink("fsiarbutr"); + ProductListResult model = new ProductListResult().withValue(Arrays + .asList(new ProductInner().withProperties(new ProductProperties().withDescription("qpuedckzywbiex")))); model = BinaryData.fromObject(model).toObject(ProductListResult.class); - Assertions.assertEquals("yydhibnuqqk", model.value().get(0).description()); - Assertions.assertEquals("fsiarbutr", model.nextLink()); + Assertions.assertEquals("qpuedckzywbiex", model.value().get(0).properties().description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductPropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductPropertiesTests.java index f87b202e7bc39..29d43cc532ecc 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductPropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductPropertiesTests.java @@ -5,23 +5,21 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.ProductProperties; +import com.azure.resourcemanager.sphere.models.ProductProperties; import org.junit.jupiter.api.Assertions; public final class ProductPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProductProperties model = - BinaryData - .fromString("{\"description\":\"yn\",\"provisioningState\":\"Provisioning\"}") - .toObject(ProductProperties.class); - Assertions.assertEquals("yn", model.description()); + ProductProperties model = BinaryData.fromString("{\"description\":\"tyq\",\"provisioningState\":\"Failed\"}") + .toObject(ProductProperties.class); + Assertions.assertEquals("tyq", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProductProperties model = new ProductProperties().withDescription("yn"); + ProductProperties model = new ProductProperties().withDescription("tyq"); model = BinaryData.fromObject(model).toObject(ProductProperties.class); - Assertions.assertEquals("yn", model.description()); + Assertions.assertEquals("tyq", model.description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdatePropertiesTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdatePropertiesTests.java index 5343929ecb72f..6078ea595687b 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdatePropertiesTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdatePropertiesTests.java @@ -5,21 +5,21 @@ package com.azure.resourcemanager.sphere.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.sphere.fluent.models.ProductUpdateProperties; +import com.azure.resourcemanager.sphere.models.ProductUpdateProperties; import org.junit.jupiter.api.Assertions; public final class ProductUpdatePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProductUpdateProperties model = - BinaryData.fromString("{\"description\":\"zfcl\"}").toObject(ProductUpdateProperties.class); - Assertions.assertEquals("zfcl", model.description()); + ProductUpdateProperties model + = BinaryData.fromString("{\"description\":\"hvgyuguosvmk\"}").toObject(ProductUpdateProperties.class); + Assertions.assertEquals("hvgyuguosvmk", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProductUpdateProperties model = new ProductUpdateProperties().withDescription("zfcl"); + ProductUpdateProperties model = new ProductUpdateProperties().withDescription("hvgyuguosvmk"); model = BinaryData.fromObject(model).toObject(ProductUpdateProperties.class); - Assertions.assertEquals("zfcl", model.description()); + Assertions.assertEquals("hvgyuguosvmk", model.description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdateTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdateTests.java index 0334345eb6613..8ca49441ba247 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdateTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductUpdateTests.java @@ -6,20 +6,22 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.sphere.models.ProductUpdate; +import com.azure.resourcemanager.sphere.models.ProductUpdateProperties; import org.junit.jupiter.api.Assertions; public final class ProductUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProductUpdate model = - BinaryData.fromString("{\"properties\":{\"description\":\"by\"}}").toObject(ProductUpdate.class); - Assertions.assertEquals("by", model.description()); + ProductUpdate model + = BinaryData.fromString("{\"properties\":{\"description\":\"ovplw\"}}").toObject(ProductUpdate.class); + Assertions.assertEquals("ovplw", model.properties().description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProductUpdate model = new ProductUpdate().withDescription("by"); + ProductUpdate model + = new ProductUpdate().withProperties(new ProductUpdateProperties().withDescription("ovplw")); model = BinaryData.fromObject(model).toObject(ProductUpdate.class); - Assertions.assertEquals("by", model.description()); + Assertions.assertEquals("ovplw", model.properties().description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesWithResponseMockTests.java index cd4aee8357873..d43fa39195c17 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCountDevicesWithResponseMockTests.java @@ -12,7 +12,7 @@ import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; -import com.azure.resourcemanager.sphere.models.CountDeviceResponse; +import com.azure.resourcemanager.sphere.models.CountDevicesResponse; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -30,40 +30,27 @@ public void testCountDevicesWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"value\":1309534736}"; + String responseStr = "{\"value\":890415173}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - CountDeviceResponse response = - manager - .products() - .countDevicesWithResponse("syrsndsytgadgvra", "aeneqnzarrwl", "uu", com.azure.core.util.Context.NONE) - .getValue(); + CountDevicesResponse response = manager.products() + .countDevicesWithResponse("reqnovvqfov", "jxywsuws", "rsndsytgadgvra", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals(1309534736, response.value()); + Assertions.assertEquals(890415173, response.value()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateMockTests.java index a8a1ec3e7089c..269b2c67e8009 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsCreateOrUpdateMockTests.java @@ -13,6 +13,7 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.sphere.AzureSphereManager; import com.azure.resourcemanager.sphere.models.Product; +import com.azure.resourcemanager.sphere.models.ProductProperties; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -30,43 +31,28 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"description\":\"orgjhxbldt\",\"provisioningState\":\"Succeeded\"},\"id\":\"rlkdmtncvokotl\",\"name\":\"xdy\",\"type\":\"gsyocogj\"}"; + String responseStr + = "{\"properties\":{\"description\":\"g\",\"provisioningState\":\"Succeeded\"},\"id\":\"mabiknsorgjhxb\",\"name\":\"dtlwwrlkd\",\"type\":\"tncvokot\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Product response = - manager - .products() - .define("ox") - .withExistingCatalog("swsrms", "yzrpzbchckqqzq") - .withDescription("suiizynkedyat") - .create(); + Product response + = manager.products().define("slyzrpzbchckqq").withExistingCatalog("jphuopxodlqi", "ntorzihleosjswsr") + .withProperties(new ProductProperties().withDescription("ox")).create(); - Assertions.assertEquals("orgjhxbldt", response.description()); + Assertions.assertEquals("g", response.properties().description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteMockTests.java index 4348987e39942..af8f664743c81 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.products().delete("dlpichkoymkcdyhb", "kkpwdreqnovvq", "ovljxywsu", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.products().delete("joxafnndlpi", "hkoymkcdyhbp", "kpw", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsMockTests.java index 837c76e3e7b04..302caaffcfb2a 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGenerateDefaultDeviceGroupsMockTests.java @@ -35,43 +35,34 @@ public void testGenerateDefaultDeviceGroups() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"description\":\"wifto\",\"osFeedType\":\"RetailEval\",\"updatePolicy\":\"UpdateAll\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"None\",\"hasDeployment\":true,\"provisioningState\":\"Failed\"},\"id\":\"aknynfsynljphuop\",\"name\":\"odlqiyntor\",\"type\":\"ihleos\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"description\":\"qkacewii\",\"osFeedType\":\"Retail\",\"updatePolicy\":\"No3rdPartyAppUpdates\",\"allowCrashDumpsCollection\":\"Disabled\",\"regionalDataBoundary\":\"EU\",\"hasDeployment\":true,\"provisioningState\":\"Updating\"},\"id\":\"hqkvpuvksgplsak\",\"name\":\"ynfs\",\"type\":\"n\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.products().generateDefaultDeviceGroups("fqka", "e", "iipfpubj", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.products().generateDefaultDeviceGroups("en", "qnzarrwl", "uu", com.azure.core.util.Context.NONE); - Assertions.assertEquals("wifto", response.iterator().next().description()); - Assertions.assertEquals(OSFeedType.RETAIL_EVAL, response.iterator().next().osFeedType()); - Assertions.assertEquals(UpdatePolicy.UPDATE_ALL, response.iterator().next().updatePolicy()); - Assertions - .assertEquals(AllowCrashDumpCollection.DISABLED, response.iterator().next().allowCrashDumpsCollection()); - Assertions.assertEquals(RegionalDataBoundary.NONE, response.iterator().next().regionalDataBoundary()); + Assertions.assertEquals("qkacewii", response.iterator().next().properties().description()); + Assertions.assertEquals(OSFeedType.RETAIL, response.iterator().next().properties().osFeedType()); + Assertions.assertEquals(UpdatePolicy.NO3RD_PARTY_APP_UPDATES, + response.iterator().next().properties().updatePolicy()); + Assertions.assertEquals(AllowCrashDumpCollection.DISABLED, + response.iterator().next().properties().allowCrashDumpsCollection()); + Assertions.assertEquals(RegionalDataBoundary.EU, + response.iterator().next().properties().regionalDataBoundary()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGetWithResponseMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGetWithResponseMockTests.java index f8137c9e85dca..ff87bdfb0c29e 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGetWithResponseMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsGetWithResponseMockTests.java @@ -30,41 +30,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"description\":\"asipqiio\",\"provisioningState\":\"Accepted\"},\"id\":\"qerpqlpqwcc\",\"name\":\"uqgbdbutauvfbt\",\"type\":\"uwhhmhykojoxafn\"}"; + String responseStr + = "{\"properties\":{\"description\":\"yuq\",\"provisioningState\":\"Provisioning\"},\"id\":\"lp\",\"name\":\"wcciuqgbdbu\",\"type\":\"auvfbtkuwhhmhyk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Product response = - manager - .products() - .getWithResponse("nwashrtd", "kcnqxwbpo", "ulpiuj", com.azure.core.util.Context.NONE) - .getValue(); + Product response = manager.products() + .getWithResponse("kcnqxwbpo", "ulpiuj", "aasipqi", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("asipqiio", response.description()); + Assertions.assertEquals("yuq", response.properties().description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogMockTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogMockTests.java index 7484fb588ac43..2e3f1473183af 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogMockTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProductsListByCatalogMockTests.java @@ -31,38 +31,27 @@ public void testListByCatalog() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"description\":\"wfluszdt\",\"provisioningState\":\"Succeeded\"},\"id\":\"kwofyyvoq\",\"name\":\"cpi\",\"type\":\"xpbtgiwbwo\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"description\":\"oqac\",\"provisioningState\":\"Accepted\"},\"id\":\"pbtg\",\"name\":\"wbwo\",\"type\":\"nwashrtd\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureSphereManager manager = - AzureSphereManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureSphereManager manager = AzureSphereManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.products().listByCatalog("m", "qyib", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.products().listByCatalog("zoymibmrqyibahw", "luszdtmhrkwof", com.azure.core.util.Context.NONE); - Assertions.assertEquals("wfluszdt", response.iterator().next().description()); + Assertions.assertEquals("oqac", response.iterator().next().properties().description()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceRequestTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceRequestTests.java index 6858ae6e401c7..34e5a6fe072f9 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceRequestTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceRequestTests.java @@ -11,15 +11,15 @@ public final class ProofOfPossessionNonceRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProofOfPossessionNonceRequest model = - BinaryData.fromString("{\"proofOfPossessionNonce\":\"kwt\"}").toObject(ProofOfPossessionNonceRequest.class); - Assertions.assertEquals("kwt", model.proofOfPossessionNonce()); + ProofOfPossessionNonceRequest model = BinaryData.fromString("{\"proofOfPossessionNonce\":\"jkot\"}") + .toObject(ProofOfPossessionNonceRequest.class); + Assertions.assertEquals("jkot", model.proofOfPossessionNonce()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProofOfPossessionNonceRequest model = new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("kwt"); + ProofOfPossessionNonceRequest model = new ProofOfPossessionNonceRequest().withProofOfPossessionNonce("jkot"); model = BinaryData.fromObject(model).toObject(ProofOfPossessionNonceRequest.class); - Assertions.assertEquals("kwt", model.proofOfPossessionNonce()); + Assertions.assertEquals("jkot", model.proofOfPossessionNonce()); } } diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceResponseInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceResponseInnerTests.java index 4295b284bcd1d..a27a3f0b1011f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceResponseInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/ProofOfPossessionNonceResponseInnerTests.java @@ -10,11 +10,9 @@ public final class ProofOfPossessionNonceResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProofOfPossessionNonceResponseInner model = - BinaryData - .fromString( - "{\"certificate\":\"xbnjbiksq\",\"status\":\"Inactive\",\"subject\":\"sainqpjwnzl\",\"thumbprint\":\"fmppe\",\"expiryUtc\":\"2021-03-28T22:28:31Z\",\"notBeforeUtc\":\"2021-11-23T16:03:48Z\",\"provisioningState\":\"Provisioning\"}") - .toObject(ProofOfPossessionNonceResponseInner.class); + ProofOfPossessionNonceResponseInner model = BinaryData.fromString( + "{\"certificate\":\"qgoulznd\",\"status\":\"Active\",\"subject\":\"yqkgfg\",\"thumbprint\":\"madgakeqsrxyb\",\"expiryUtc\":\"2021-03-26T22:16:08Z\",\"notBeforeUtc\":\"2021-02-21T09:55:36Z\",\"provisioningState\":\"Deleting\"}") + .toObject(ProofOfPossessionNonceResponseInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/SignedCapabilityImageResponseInnerTests.java b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/SignedCapabilityImageResponseInnerTests.java index 54c09a9375131..eadaa20bd300f 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/SignedCapabilityImageResponseInnerTests.java +++ b/sdk/sphere/azure-resourcemanager-sphere/src/test/java/com/azure/resourcemanager/sphere/generated/SignedCapabilityImageResponseInnerTests.java @@ -10,8 +10,8 @@ public final class SignedCapabilityImageResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SignedCapabilityImageResponseInner model = - BinaryData.fromString("{\"image\":\"nxytxh\"}").toObject(SignedCapabilityImageResponseInner.class); + SignedCapabilityImageResponseInner model + = BinaryData.fromString("{\"image\":\"sapskr\"}").toObject(SignedCapabilityImageResponseInner.class); } @org.junit.jupiter.api.Test From 15f7c5894e51af5f207f11fcb365aa5a19006261 Mon Sep 17 00:00:00 2001 From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com> Date: Tue, 26 Mar 2024 22:51:30 -0400 Subject: [PATCH 18/44] Increment package versions for monitor releases (#39419) --- eng/versioning/version_client.txt | 2 +- sdk/monitor/azure-monitor-query-perf/pom.xml | 2 +- sdk/monitor/azure-monitor-query/CHANGELOG.md | 10 ++++++++++ sdk/monitor/azure-monitor-query/pom.xml | 2 +- 4 files changed, 13 insertions(+), 3 deletions(-) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index 6d83cc6fec48b..61da732bf09d3 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -165,7 +165,7 @@ com.azure:azure-mixedreality-remoterendering;1.1.27;1.2.0-beta.1 com.azure:azure-monitor-opentelemetry-exporter;1.0.0-beta.21;1.0.0-beta.22 com.azure:azure-monitor-ingestion;1.1.5;1.2.0-beta.1 com.azure:azure-monitor-ingestion-perf;1.0.0-beta.1;1.0.0-beta.1 -com.azure:azure-monitor-query;1.2.10;1.3.0 +com.azure:azure-monitor-query;1.3.0;1.4.0-beta.1 com.azure:azure-monitor-query-perf;1.0.0-beta.1;1.0.0-beta.1 com.azure:azure-perf-test-parent;1.0.0-beta.1;1.0.0-beta.1 com.azure:azure-quantum-jobs;1.0.0-beta.1;1.0.0-beta.2 diff --git a/sdk/monitor/azure-monitor-query-perf/pom.xml b/sdk/monitor/azure-monitor-query-perf/pom.xml index 0120f9f2e3940..e40ac8ae211c8 100644 --- a/sdk/monitor/azure-monitor-query-perf/pom.xml +++ b/sdk/monitor/azure-monitor-query-perf/pom.xml @@ -31,7 +31,7 @@ com.azure azure-monitor-query - 1.3.0 + 1.4.0-beta.1 com.azure diff --git a/sdk/monitor/azure-monitor-query/CHANGELOG.md b/sdk/monitor/azure-monitor-query/CHANGELOG.md index c7946f7748e60..8232947998b2f 100644 --- a/sdk/monitor/azure-monitor-query/CHANGELOG.md +++ b/sdk/monitor/azure-monitor-query/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 1.4.0-beta.1 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 1.3.0 (2024-03-26) ### Features Added diff --git a/sdk/monitor/azure-monitor-query/pom.xml b/sdk/monitor/azure-monitor-query/pom.xml index 4875dc93a8c39..3a1ff8146fc49 100644 --- a/sdk/monitor/azure-monitor-query/pom.xml +++ b/sdk/monitor/azure-monitor-query/pom.xml @@ -11,7 +11,7 @@ com.azure azure-monitor-query - 1.3.0 + 1.4.0-beta.1 Microsoft Azure SDK for Azure Monitor Logs and Metrics Query This package contains the Microsoft Azure SDK for querying Azure Monitor's Logs and Metrics data sources. From b8a9d8c1da8114ed9ed1c451a0549f56d4d9b510 Mon Sep 17 00:00:00 2001 From: Liudmila Molkova Date: Tue, 26 Mar 2024 19:58:23 -0700 Subject: [PATCH 19/44] Tracing: end HTTP spans when body is fully consumed, add missing methods for OpenAI instrumentation (#39381) * Tracing fixes, add isRecording, add setAttribute(String, Object, stream reading fix --- .../CHANGELOG.md | 2 + .../opentelemetry/OpenTelemetryTracer.java | 52 +++++++++++- .../OpenTelemetryHttpPolicyTests.java | 24 +++--- .../OpenTelemetryTracerTest.java | 72 +++++++++++++--- sdk/core/azure-core/CHANGELOG.md | 2 + .../http/policy/InstrumentationPolicy.java | 84 +++++++++++++------ .../http/rest/AsyncRestProxy.java | 3 +- .../http/rest/RestProxyBase.java | 2 +- .../com/azure/core/util/tracing/Tracer.java | 33 +++++++- .../util/tracing/RestProxyTracingTests.java | 29 +++++-- 10 files changed, 243 insertions(+), 60 deletions(-) diff --git a/sdk/core/azure-core-tracing-opentelemetry/CHANGELOG.md b/sdk/core/azure-core-tracing-opentelemetry/CHANGELOG.md index 1f029f24c773f..da3ba92a0dddc 100644 --- a/sdk/core/azure-core-tracing-opentelemetry/CHANGELOG.md +++ b/sdk/core/azure-core-tracing-opentelemetry/CHANGELOG.md @@ -8,6 +8,8 @@ ### Bugs Fixed +- Fixed unreliable HTTP span reporting when response is not closed. + ### Other Changes ## 1.0.0-beta.44 (2024-03-01) diff --git a/sdk/core/azure-core-tracing-opentelemetry/src/main/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracer.java b/sdk/core/azure-core-tracing-opentelemetry/src/main/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracer.java index 6b29677b51686..1a0389024a3cf 100644 --- a/sdk/core/azure-core-tracing-opentelemetry/src/main/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracer.java +++ b/sdk/core/azure-core-tracing-opentelemetry/src/main/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracer.java @@ -102,7 +102,7 @@ public Context start(String spanName, StartSpanOptions options, Context context) return startSuppressedSpan(context); } context = unsuppress(context); - if (spanKind == SpanKind.INTERNAL && !context.getData(CLIENT_METHOD_CALL_FLAG).isPresent()) { + if (isInternalOrClientSpan(spanKind) && !context.getData(CLIENT_METHOD_CALL_FLAG).isPresent()) { context = context.addData(CLIENT_METHOD_CALL_FLAG, true); } @@ -184,6 +184,9 @@ private SpanBuilder createSpanBuilder(String spanName, StartSpanOptions options, return spanBuilder; } + /** + * {@inheritDoc} + */ @Override public void injectContext(BiConsumer headerSetter, Context context) { io.opentelemetry.context.Context otelContext = getTraceContextOrDefault(context, null); @@ -192,6 +195,9 @@ public void injectContext(BiConsumer headerSetter, Context conte } } + /** + * {@inheritDoc} + */ @Override public void setAttribute(String key, long value, Context context) { Objects.requireNonNull(context, "'context' cannot be null"); @@ -235,6 +241,28 @@ public void setAttribute(String key, String value, Context context) { } } + /** + * {@inheritDoc} + */ + @Override + public void setAttribute(String key, Object value, Context context) { + Objects.requireNonNull(value, "'value' cannot be null"); + Objects.requireNonNull(context, "'context' cannot be null"); + + if (!isEnabled) { + return; + } + + final Span span = getSpanOrNull(context); + if (span == null) { + return; + } + + if (span.isRecording()) { + OpenTelemetryUtils.addAttribute(span, key, value); + } + } + /** * {@inheritDoc} */ @@ -267,6 +295,24 @@ public Context extractContext(Function headerGetter) { return new Context(SPAN_CONTEXT_KEY, Span.fromContext(traceContext).getSpanContext()); } + /** + * {@inheritDoc} + */ + @Override + public boolean isRecording(Context context) { + Objects.requireNonNull(context, "'context' cannot be null"); + if (!isEnabled) { + return false; + } + + Span span = getSpanOrNull(context); + if (span != null) { + return span.isRecording(); + } + + return false; + } + private static class Getter implements TextMapGetter> { public static final TextMapGetter> INSTANCE = new Getter(); @@ -420,4 +466,8 @@ private static TracerProvider getTracerProvider(TracingOptions options) { return GlobalOpenTelemetry.getTracerProvider(); } + + private static boolean isInternalOrClientSpan(SpanKind kind) { + return kind == SpanKind.INTERNAL || kind == SpanKind.CLIENT; + } } diff --git a/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryHttpPolicyTests.java b/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryHttpPolicyTests.java index c452de4d6387c..9b5fc6309144a 100644 --- a/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryHttpPolicyTests.java +++ b/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryHttpPolicyTests.java @@ -112,7 +112,7 @@ public void openTelemetryHttpPolicyTest() { request.setHeader(HttpHeaderName.USER_AGENT, "user-agent"); try (Scope scope = parentSpan.makeCurrent()) { - createHttpPipeline(azTracer).send(request, tracingContext).block().close(); + createHttpPipeline(azTracer).send(request, tracingContext).block().getBodyAsBinaryData(); } // Assert List exportedSpans = exporter.getFinishedSpanItems(); @@ -181,8 +181,9 @@ public String getDescription() { public void clientRequestIdIsStamped() { try (Scope scope = tracer.spanBuilder("test").startSpan().makeCurrent()) { HttpRequest request = new HttpRequest(HttpMethod.PUT, "https://httpbin.org/hello?there#otel"); - HttpResponse response = createHttpPipeline(azTracer, new RequestIdPolicy()).send(request).block(); - response.close(); + HttpResponse response = createHttpPipeline(azTracer, new RequestIdPolicy()).send(request) + .flatMap(r -> r.getBodyAsByteArray().thenReturn(r)) + .block(); // Assert List exportedSpans = exporter.getFinishedSpanItems(); @@ -293,8 +294,8 @@ public void endStatusDependingOnStatusCode(int statusCode, StatusCode status) { StepVerifier.create(pipeline.send(new HttpRequest(HttpMethod.GET, "http://localhost/hello"))) .assertNext(response -> { + response.getBodyAsByteArray().block(); assertEquals(statusCode, response.getStatusCode()); - response.close(); }) .verifyComplete(); @@ -366,12 +367,10 @@ public void timeoutIsTraced() { = new Context(PARENT_TRACE_CONTEXT_KEY, io.opentelemetry.context.Context.root().with(parentSpan)) .addData("az.namespace", "foo"); - StepVerifier.create(pipeline.send(new HttpRequest(HttpMethod.GET, "http://localhost/hello"), tracingContext)) - .assertNext(response -> { + StepVerifier.create(pipeline.send(new HttpRequest(HttpMethod.GET, "http://localhost/hello"), tracingContext) + .flatMap(r -> r.getBody().collectList().thenReturn(r))).assertNext(response -> { assertEquals(200, response.getStatusCode()); - response.close(); - }) - .verifyComplete(); + }).verifyComplete(); List exportedSpans = exporter.getFinishedSpanItems(); assertEquals(2, exportedSpans.size()); @@ -398,11 +397,8 @@ public void connectionErrorAfterResponseCodeIsTraced() { .tracer(azTracer) .build(); - StepVerifier - .create(pipeline.send(new HttpRequest(HttpMethod.GET, "http://localhost/hello"), Context.NONE) - .flatMap(response -> response.getBodyAsInputStream().doFinally(i -> response.close()))) - .expectError(IOException.class) - .verify(); + StepVerifier.create(pipeline.send(new HttpRequest(HttpMethod.GET, "http://localhost/hello"), Context.NONE) + .flatMap(response -> response.getBodyAsInputStream())).expectError(IOException.class).verify(); List exportedSpans = exporter.getFinishedSpanItems(); assertEquals(1, exportedSpans.size()); diff --git a/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracerTest.java b/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracerTest.java index c2f5c76ae12c8..63663e2f04368 100644 --- a/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracerTest.java +++ b/sdk/core/azure-core-tracing-opentelemetry/src/test/java/com/azure/core/tracing/opentelemetry/OpenTelemetryTracerTest.java @@ -36,6 +36,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import reactor.core.Exceptions; import java.io.IOException; @@ -285,6 +286,28 @@ public void startWithRemoteParent() { assertEquals(SpanKind.CONSUMER, spanData.getKind()); } + @ParameterizedTest + @ValueSource(booleans = { true, false }) + public void testIsRecording(boolean isRecording) { + // Arrange + SpanContext remoteParent + = SpanContext.create(IdGenerator.random().generateTraceId(), IdGenerator.random().generateSpanId(), + isRecording ? TraceFlags.getSampled() : TraceFlags.getDefault(), TraceState.getDefault()); + StartSpanOptions options = new StartSpanOptions(com.azure.core.util.tracing.SpanKind.CLIENT) + .setRemoteParent(new Context(SPAN_CONTEXT_KEY, remoteParent)); + + // Act + final Context span = openTelemetryTracer.start(METHOD_NAME, options, Context.NONE); + + // Assert + assertEquals(isRecording, openTelemetryTracer.isRecording(span)); + } + + @Test + public void testIsRecordingNoSpan() { + assertFalse(openTelemetryTracer.isRecording(Context.NONE)); + } + @Test @SuppressWarnings("deprecation") public void startSpanProcessKindSend() { @@ -462,7 +485,6 @@ public void startConsumeSpanWitStartTimeInContext() { @Test @SuppressWarnings("deprecation") public void startSpanOverloadNullPointerException() { - // Assert assertThrows(NullPointerException.class, () -> openTelemetryTracer.start("", Context.NONE, null)); } @@ -476,23 +498,19 @@ public void startSpanInvalid() { new StartSpanOptions(com.azure.core.util.tracing.SpanKind.CONSUMER), Context.NONE)); assertThrows(NullPointerException.class, () -> openTelemetryTracer.start("span", new StartSpanOptions(com.azure.core.util.tracing.SpanKind.CONSUMER), null)); - } @Test - @SuppressWarnings("deprecation") public void addLinkTest() { // Arrange - StartSpanOptions spanBuilder = new StartSpanOptions(com.azure.core.util.tracing.SpanKind.INTERNAL); Span toLinkSpan = tracer.spanBuilder("new test span").startSpan(); - Context linkContext = new Context(SPAN_CONTEXT_KEY, toLinkSpan.getSpanContext()); LinkData expectedLink = LinkData.create(toLinkSpan.getSpanContext()); + StartSpanOptions startOptions + = new StartSpanOptions(com.azure.core.util.tracing.SpanKind.INTERNAL).addLink(new TracingLink(linkContext)); // Act - openTelemetryTracer.addLink(linkContext.addData(SPAN_BUILDER_KEY, spanBuilder)); - - Context span = openTelemetryTracer.start(METHOD_NAME, spanBuilder, Context.NONE); + Context span = openTelemetryTracer.start(METHOD_NAME, startOptions, Context.NONE); ReadableSpan span1 = getSpan(span); // Assert @@ -886,6 +904,36 @@ public void startSpanWithAttributes() { verifySpanAttributes(expectedAttributes, span.toSpanData().getAttributes()); } + @Test + public void spanAttributes() { + Map attributes = new HashMap<>(); + attributes.put("S", "foo"); + attributes.put("I", 1); + attributes.put("L", 10L); + attributes.put("D", 0.1d); + attributes.put("B", true); + attributes.put("S[]", new String[] { "foo" }); + attributes.put("L[]", new long[] { 10L }); + attributes.put("D[]", new double[] { 0.1d }); + attributes.put("B[]", new boolean[] { true }); + attributes.put("I[]", new int[] { 1 }); + attributes.put("Complex", Collections.singletonMap("key", "value")); + + Attributes expectedAttributes = Attributes.builder() + .put("S", "foo") + .put("L", 10L) + .put("I", 1) + .put("D", 0.1d) + .put("B", true) + .put("az.namespace", AZ_NAMESPACE_VALUE) + .build(); + + Context spanCtx = openTelemetryTracer.start(METHOD_NAME, tracingContext); + attributes.forEach((key, value) -> openTelemetryTracer.setAttribute(key, value, spanCtx)); + final ReadableSpan span = getSpan(spanCtx); + verifySpanAttributes(expectedAttributes, span.toSpanData().getAttributes()); + } + @Test public void suppressNestedClientSpan() { Context outer = openTelemetryTracer.start("outer", Context.NONE); @@ -909,9 +957,9 @@ public void suppressNestedClientSpan() { } @Test - @SuppressWarnings("deprecation") public void suppressNestedInterleavedClientSpan() { - Context outer = openTelemetryTracer.start("outer", Context.NONE, com.azure.core.util.tracing.ProcessKind.SEND); + Context outer = openTelemetryTracer.start("outer", + new StartSpanOptions(com.azure.core.util.tracing.SpanKind.CONSUMER), Context.NONE); Context inner1NotSuppressed = openTelemetryTracer.start("innerSuppressed", outer); Context inner2Suppressed = openTelemetryTracer.start("innerSuppressed", inner1NotSuppressed); @@ -940,7 +988,7 @@ public void suppressNestedInterleavedClientSpanWithOptions() { Context inner1Suppressed = openTelemetryTracer.start("innerSuppressed", outer); Context inner1NotSuppressed = openTelemetryTracer.start("innerNotSuppressed", - new StartSpanOptions(com.azure.core.util.tracing.SpanKind.CLIENT), inner1Suppressed); + new StartSpanOptions(com.azure.core.util.tracing.SpanKind.PRODUCER), inner1Suppressed); Context inner2Suppressed = openTelemetryTracer.start("innerSuppressed", inner1NotSuppressed); openTelemetryTracer.end("ok", null, inner2Suppressed); @@ -1009,7 +1057,7 @@ public static Stream spanKinds() { Arguments.of(com.azure.core.util.tracing.SpanKind.CLIENT, com.azure.core.util.tracing.SpanKind.CLIENT, false), Arguments.of(com.azure.core.util.tracing.SpanKind.CLIENT, com.azure.core.util.tracing.SpanKind.INTERNAL, - false), + true), Arguments.of(com.azure.core.util.tracing.SpanKind.CLIENT, com.azure.core.util.tracing.SpanKind.PRODUCER, false), Arguments.of(com.azure.core.util.tracing.SpanKind.CLIENT, com.azure.core.util.tracing.SpanKind.CONSUMER, diff --git a/sdk/core/azure-core/CHANGELOG.md b/sdk/core/azure-core/CHANGELOG.md index 7b8546e9b8cae..246794c098fae 100644 --- a/sdk/core/azure-core/CHANGELOG.md +++ b/sdk/core/azure-core/CHANGELOG.md @@ -4,6 +4,8 @@ ### Features Added +- Added new methods on `com.azure.core.util.tracing.Tracer` - `isRecording` and `addAttribute(String, Object, Context)`. + ### Breaking Changes ### Bugs Fixed diff --git a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/policy/InstrumentationPolicy.java b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/policy/InstrumentationPolicy.java index 9a3b22760d2e7..3dc2f4e665b87 100644 --- a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/policy/InstrumentationPolicy.java +++ b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/policy/InstrumentationPolicy.java @@ -25,6 +25,7 @@ import java.net.URL; import java.nio.ByteBuffer; import java.nio.charset.Charset; +import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import static com.azure.core.http.HttpHeaderName.X_MS_CLIENT_REQUEST_ID; import static com.azure.core.http.HttpHeaderName.X_MS_REQUEST_ID; @@ -54,6 +55,8 @@ public class InstrumentationPolicy implements HttpPipelinePolicy { private static final String SERVER_PORT = "server.port"; private static final ClientLogger LOGGER = new ClientLogger(InstrumentationPolicy.class); + // magic OpenTelemetry string that represents unknown error. + private static final String OTHER_ERROR_TYPE = "_OTHER"; private Tracer tracer; /** @@ -82,7 +85,7 @@ public Mono process(HttpPipelineCallContext context, HttpPipelineN return next.process() .doOnSuccess(response -> onResponseCode(response, span)) // TODO: maybe we can optimize it? https://github.com/Azure/azure-sdk-for-java/issues/38228 - .map(response -> new TraceableResponse(response, span)) + .map(response -> TraceableResponse.create(response, tracer, span)) .doOnCancel(() -> tracer.end(CANCELLED_ERROR_TYPE, null, span)) .doOnError(exception -> tracer.end(null, exception, span)); }); @@ -100,7 +103,7 @@ public HttpResponse processSync(HttpPipelineCallContext context, HttpPipelineNex HttpResponse response = next.processSync(); onResponseCode(response, span); // TODO: maybe we can optimize it? https://github.com/Azure/azure-sdk-for-java/issues/38228 - return new TraceableResponse(response, span); + return TraceableResponse.create(response, tracer, span); } catch (RuntimeException ex) { tracer.end(null, ex, span); throw ex; @@ -150,7 +153,7 @@ private void addPostSamplingAttributes(Context span, HttpRequest request) { } private void onResponseCode(HttpResponse response, Context span) { - if (response != null) { + if (response != null && tracer.isRecording(span)) { int statusCode = response.getStatusCode(); tracer.setAttribute(HTTP_STATUS_CODE, statusCode, span); String requestId = response.getHeaderValue(X_MS_REQUEST_ID); @@ -164,16 +167,29 @@ private boolean isTracingEnabled(HttpPipelineCallContext context) { return tracer != null && tracer.isEnabled() && !((boolean) context.getData(DISABLE_TRACING_KEY).orElse(false)); } - private final class TraceableResponse extends HttpResponse { + private static final class TraceableResponse extends HttpResponse { private final HttpResponse response; private final Context span; - private Throwable exception; - private String errorType; + private final Tracer tracer; + private volatile int ended = 0; + private static final AtomicIntegerFieldUpdater ENDED_UPDATER + = AtomicIntegerFieldUpdater.newUpdater(TraceableResponse.class, "ended"); - TraceableResponse(HttpResponse response, Context span) { + private TraceableResponse(HttpResponse response, Tracer tracer, Context span) { super(response.getRequest()); this.response = response; this.span = span; + this.tracer = tracer; + } + + public static HttpResponse create(HttpResponse response, Tracer tracer, Context span) { + if (tracer.isRecording(span)) { + return new TraceableResponse(response, tracer, span); + } + + // OTel does not need to end sampled-out spans, but let's do it just in case + tracer.end(null, null, span); + return response; } @Override @@ -199,21 +215,21 @@ public HttpHeaders getHeaders() { @Override public Flux getBody() { - return response.getBody().doOnError(e -> exception = e).doOnCancel(() -> errorType = CANCELLED_ERROR_TYPE); + return Flux.using(() -> span, + s -> response.getBody() + .doOnError(e -> onError(null, e)) + .doOnCancel(() -> onError(CANCELLED_ERROR_TYPE, null)), + s -> endNoError()); } @Override public Mono getBodyAsByteArray() { - return response.getBodyAsByteArray() - .doOnError(e -> exception = e) - .doOnCancel(() -> errorType = CANCELLED_ERROR_TYPE); + return endSpanWhen(response.getBodyAsByteArray()); } @Override public Mono getBodyAsString() { - return response.getBodyAsString() - .doOnError(e -> exception = e) - .doOnCancel(() -> errorType = CANCELLED_ERROR_TYPE); + return endSpanWhen(response.getBodyAsString()); } @Override @@ -221,34 +237,52 @@ public BinaryData getBodyAsBinaryData() { try { return response.getBodyAsBinaryData(); } catch (Exception e) { - exception = e; + onError(null, e); throw e; + } finally { + endNoError(); } } @Override public Mono getBodyAsString(Charset charset) { - return response.getBodyAsString(charset) - .doOnError(e -> exception = e) - .doOnCancel(() -> errorType = CANCELLED_ERROR_TYPE); + return endSpanWhen(response.getBodyAsString(charset)); } @Override public Mono getBodyAsInputStream() { - return response.getBodyAsInputStream() - .doOnError(e -> exception = e) - .doOnCancel(() -> errorType = CANCELLED_ERROR_TYPE); + return endSpanWhen(response.getBodyAsInputStream()); } @Override public void close() { response.close(); - int statusCode = response.getStatusCode(); + endNoError(); + } + + private Mono endSpanWhen(Mono publisher) { + return Mono.using(() -> span, + s -> publisher.doOnError(e -> onError(null, e)).doOnCancel(() -> onError(CANCELLED_ERROR_TYPE, null)), + s -> endNoError()); + } + + private void onError(String errorType, Throwable error) { + if (ENDED_UPDATER.compareAndSet(this, 0, 1)) { + tracer.end(errorType, error, span); + } + } + + private void endNoError() { + if (ENDED_UPDATER.compareAndSet(this, 0, 1)) { + String errorType = null; + if (response == null) { + errorType = OTHER_ERROR_TYPE; + } else if (response.getStatusCode() >= 400) { + errorType = String.valueOf(response.getStatusCode()); + } - if (errorType == null && statusCode >= 400) { - errorType = String.valueOf(statusCode); + tracer.end(errorType, null, span); } - tracer.end(errorType, exception, span); } } } diff --git a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java index 37e7693d40b87..e45a5a7dec7be 100644 --- a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java +++ b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java @@ -19,6 +19,7 @@ import com.azure.core.util.FluxUtil; import com.azure.core.util.serializer.SerializerAdapter; import com.azure.core.util.serializer.SerializerEncoding; +import com.azure.core.util.tracing.Tracer; import com.azure.json.JsonSerializable; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -277,7 +278,7 @@ private Object handleRestReturnType(Mono tracer.end(CANCELLED_ERROR_TYPE, null, span)) - .contextWrite(reactor.util.context.Context.of("TRACING_CONTEXT", span)); + .contextWrite(reactor.util.context.Context.of(Tracer.PARENT_TRACE_CONTEXT_KEY, span)); } return getResponse; diff --git a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/RestProxyBase.java b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/RestProxyBase.java index 2314b0f46d269..1aa122f8b38b5 100644 --- a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/RestProxyBase.java +++ b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/RestProxyBase.java @@ -227,7 +227,7 @@ public Response createResponse(HttpResponseDecoder.HttpDecodedResponse response, */ Context startTracingSpan(SwaggerMethodParser method, Context context) { if (isTracingEnabled(context)) { - Object tracingContextObj = context.getData("TRACING_CONTEXT").orElse(null); + Object tracingContextObj = context.getData(Tracer.PARENT_TRACE_CONTEXT_KEY).orElse(null); Context tracingContext = tracingContextObj instanceof Context ? (Context) tracingContextObj : context; return tracer.start(method.getSpanName(), tracingContext); } diff --git a/sdk/core/azure-core/src/main/java/com/azure/core/util/tracing/Tracer.java b/sdk/core/azure-core/src/main/java/com/azure/core/util/tracing/Tracer.java index 4409333255242..ea343c3a75278 100644 --- a/sdk/core/azure-core/src/main/java/com/azure/core/util/tracing/Tracer.java +++ b/sdk/core/azure-core/src/main/java/com/azure/core/util/tracing/Tracer.java @@ -371,7 +371,6 @@ default void end(int responseCode, Throwable error, Context context) { * tracer.setAttribute("foo", 42, span); * * - * @param key attribute name * @param value atteribute value * @param context tracing context @@ -380,6 +379,28 @@ default void setAttribute(String key, long value, Context context) { setAttribute(key, Long.toString(value), context); } + /** + * Sets an attribute on span. + * Adding duplicate attributes, update, or removal is discouraged, since underlying implementations + * behavior can vary. + * + * @param key attribute key. + * @param value attribute value. Note that underlying tracer implementations limit supported value types. + * OpenTelemetry implementation supports following types: + *

    + *
  • {@link String}
  • + *
  • {@code int}
  • + *
  • {@code double}
  • + *
  • {@code boolean}
  • + *
  • {@code long}
  • + *
+ * @param context context containing span to which attribute is added. + */ + default void setAttribute(String key, Object value, Context context) { + Objects.requireNonNull(value, "'value' cannot be null."); + setAttribute(key, value.toString(), context); + } + /** * Sets the name for spans that are created. * @@ -620,6 +641,16 @@ default AutoCloseable makeSpanCurrent(Context context) { return NoopTracer.INSTANCE.makeSpanCurrent(context); } + /** + * Checks if span is sampled in. + * + * @param span Span to check. + * @return true if span is recording, false otherwise. + */ + default boolean isRecording(Context span) { + return true; + } + /** * Checks if tracer is enabled. * diff --git a/sdk/core/azure-core/src/test/java/com/azure/core/util/tracing/RestProxyTracingTests.java b/sdk/core/azure-core/src/test/java/com/azure/core/util/tracing/RestProxyTracingTests.java index 933deddbdc979..98f2faeb48f37 100644 --- a/sdk/core/azure-core/src/test/java/com/azure/core/util/tracing/RestProxyTracingTests.java +++ b/sdk/core/azure-core/src/test/java/com/azure/core/util/tracing/RestProxyTracingTests.java @@ -56,8 +56,8 @@ void beforeEach() { @SyncAsyncTest public void restProxySuccess() throws Exception { - SyncAsyncExtension.execute(() -> testInterface.testMethodReturnsMonoVoidSync(), - () -> testInterface.testMethodReturnsMonoVoid().block()); + SyncAsyncExtension.execute(() -> testInterface.testMethodReturnsMonoVoidSync(Context.NONE), + () -> testInterface.testMethodReturnsMonoVoid(Context.NONE).block()); assertEquals(2, tracer.getSpans().size()); Span restProxy = tracer.getSpans().get(0); @@ -69,9 +69,28 @@ public void restProxySuccess() throws Exception { assertNull(restProxy.getErrorMessage()); } + @SyncAsyncTest + public void restProxyNested() throws Exception { + Context outerSpan = tracer.start("outer", Context.NONE); + SyncAsyncExtension.execute(() -> testInterface.testMethodReturnsMonoVoidSync(outerSpan), + () -> testInterface.testMethodReturnsMonoVoid(outerSpan).block()); + tracer.end(null, null, outerSpan); + + assertEquals(3, tracer.getSpans().size()); + Span outer = tracer.getSpans().get(0); + Span restProxy = tracer.getSpans().get(1); + Span http = tracer.getSpans().get(2); + + assertEquals(getSpan(restProxy.getStartContext()), outer); + assertEquals(getSpan(http.getStartContext()), restProxy); + assertTrue(restProxy.getName().startsWith("myService.testMethodReturnsMonoVoid")); + assertNull(restProxy.getThrowable()); + assertNull(restProxy.getErrorMessage()); + } + @Test public void restProxyCancelAsync() { - testInterface.testMethodDelays().timeout(Duration.ofMillis(10)).toFuture().cancel(true); + StepVerifier.create(testInterface.testMethodDelays()).expectSubscription().thenCancel().verify(); assertEquals(2, tracer.getSpans().size()); Span restProxy = tracer.getSpans().get(0); @@ -235,11 +254,11 @@ public HttpResponse sendSync(HttpRequest request, Context context) { interface TestInterface { @Get("my/url/path") @ExpectedResponses({ 200 }) - Mono testMethodReturnsMonoVoid(); + Mono testMethodReturnsMonoVoid(Context context); @Get("my/url/path") @ExpectedResponses({ 200 }) - Response testMethodReturnsMonoVoidSync(); + Response testMethodReturnsMonoVoidSync(Context context); @Post("my/url/path") @ExpectedResponses({ 500 }) From e6e6f21c87820fa8d0be5e5afa916502b65bb8fc Mon Sep 17 00:00:00 2001 From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com> Date: Tue, 26 Mar 2024 23:45:23 -0400 Subject: [PATCH 20/44] Increment package versions for sphere releases (#39421) --- eng/versioning/version_client.txt | 2 +- sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md | 10 ++++++++++ sdk/sphere/azure-resourcemanager-sphere/pom.xml | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index 61da732bf09d3..e9583504ddd08 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -435,7 +435,7 @@ com.azure.resourcemanager:azure-resourcemanager-managementgroups;1.0.0-beta.1;1. com.azure.resourcemanager:azure-resourcemanager-managednetworkfabric;1.0.0;1.1.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-iotfirmwaredefense;1.0.0;1.1.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-quantum;1.0.0-beta.2;1.0.0-beta.3 -com.azure.resourcemanager:azure-resourcemanager-sphere;1.0.0-beta.1;1.0.0 +com.azure.resourcemanager:azure-resourcemanager-sphere;1.0.0;1.1.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-chaos;1.1.0;1.2.0-beta.1 com.azure.resourcemanager:azure-resourcemanager-defendereasm;1.0.0-beta.1;1.0.0-beta.2 com.azure.resourcemanager:azure-resourcemanager-hdinsight-containers;1.0.0-beta.1;1.0.0-beta.2 diff --git a/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md b/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md index 50b71037fac9e..5a13d46e2c228 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md +++ b/sdk/sphere/azure-resourcemanager-sphere/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 1.1.0-beta.1 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 1.0.0 (2024-03-26) - Azure Resource Manager AzureSphere client library for Java. This package contains Microsoft Azure SDK for AzureSphere Management SDK. Azure Sphere resource management API. Package tag package-2024-04-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/sphere/azure-resourcemanager-sphere/pom.xml b/sdk/sphere/azure-resourcemanager-sphere/pom.xml index a63aa7bf32057..25fac3c84f3db 100644 --- a/sdk/sphere/azure-resourcemanager-sphere/pom.xml +++ b/sdk/sphere/azure-resourcemanager-sphere/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-sphere - 1.0.0 + 1.1.0-beta.1 jar Microsoft Azure SDK for AzureSphere Management From 00db29e1e0d082e62e15cd9936bf730aa7418ac6 Mon Sep 17 00:00:00 2001 From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com> Date: Wed, 27 Mar 2024 04:55:23 -0400 Subject: [PATCH 21/44] [Automation] Generate Fluent Lite from batch#package-2024-02 (#39424) * [Automation] Generate Fluent Lite from batch#package-2024-02 * fix mgmt live test * imports --------- Co-authored-by: XiaofeiCao --- .../azure-resourcemanager-batch/CHANGELOG.md | 28 +- .../azure-resourcemanager-batch/README.md | 4 +- .../azure-resourcemanager-batch/SAMPLE.md | 829 ++++++++++++++---- sdk/batch/azure-resourcemanager-batch/pom.xml | 4 +- .../resourcemanager/batch/BatchManager.java | 2 +- .../batch/fluent/models/PoolInner.java | 24 + .../batch/fluent/models/PoolProperties.java | 30 + .../fluent/models/SupportedSkuInner.java | 16 + .../batch/implementation/ApplicationImpl.java | 6 +- .../ApplicationPackagesImpl.java | 36 +- .../implementation/ApplicationsImpl.java | 28 +- .../implementation/BatchAccountImpl.java | 4 +- .../implementation/BatchAccountsImpl.java | 34 +- .../BatchManagementClientImpl.java | 2 +- .../batch/implementation/CertificateImpl.java | 6 +- .../implementation/CertificatesImpl.java | 28 +- .../batch/implementation/LocationsImpl.java | 8 +- .../batch/implementation/OperationsImpl.java | 4 +- .../batch/implementation/PoolImpl.java | 16 +- .../batch/implementation/PoolsImpl.java | 28 +- .../PrivateEndpointConnectionsImpl.java | 4 +- .../PrivateLinkResourcesImpl.java | 4 +- .../{Utils.java => ResourceManagerUtils.java} | 4 +- .../implementation/SupportedSkuImpl.java | 5 + .../models/AutomaticOSUpgradePolicy.java | 147 ++++ .../resourcemanager/batch/models/Pool.java | 40 +- .../batch/models/RollingUpgradePolicy.java | 267 ++++++ .../batch/models/SupportedSku.java | 8 + .../batch/models/UpgradeMode.java | 70 ++ .../batch/models/UpgradePolicy.java | 135 +++ .../src/main/java/module-info.java | 2 - .../reflect-config.json | 20 + .../generated/ApplicationCreateSamples.java | 2 +- .../generated/ApplicationDeleteSamples.java | 2 +- .../generated/ApplicationGetSamples.java | 2 +- .../generated/ApplicationListSamples.java | 2 +- .../ApplicationPackageActivateSamples.java | 2 +- .../ApplicationPackageCreateSamples.java | 2 +- .../ApplicationPackageDeleteSamples.java | 2 +- .../ApplicationPackageGetSamples.java | 2 +- .../ApplicationPackageListSamples.java | 2 +- .../generated/ApplicationUpdateSamples.java | 2 +- .../generated/BatchAccountCreateSamples.java | 10 +- .../generated/BatchAccountDeleteSamples.java | 2 +- ...BatchAccountGetByResourceGroupSamples.java | 4 +- .../BatchAccountGetDetectorSamples.java | 2 +- .../generated/BatchAccountGetKeysSamples.java | 2 +- ...atchAccountListByResourceGroupSamples.java | 2 +- .../BatchAccountListDetectorsSamples.java | 2 +- ...ndNetworkDependenciesEndpointsSamples.java | 2 +- .../generated/BatchAccountListSamples.java | 2 +- .../BatchAccountRegenerateKeySamples.java | 2 +- ...ountSynchronizeAutoStorageKeysSamples.java | 2 +- .../generated/BatchAccountUpdateSamples.java | 2 +- .../CertificateCancelDeletionSamples.java | 2 +- .../generated/CertificateCreateSamples.java | 6 +- .../generated/CertificateDeleteSamples.java | 2 +- .../generated/CertificateGetSamples.java | 4 +- .../CertificateListByBatchAccountSamples.java | 4 +- .../generated/CertificateUpdateSamples.java | 2 +- .../LocationCheckNameAvailabilitySamples.java | 4 +- .../generated/LocationGetQuotasSamples.java | 2 +- ...nListSupportedCloudServiceSkusSamples.java | 2 +- ...istSupportedVirtualMachineSkusSamples.java | 2 +- .../generated/OperationsListSamples.java | 2 +- .../batch/generated/PoolCreateSamples.java | 68 +- .../batch/generated/PoolDeleteSamples.java | 2 +- .../PoolDisableAutoScaleSamples.java | 2 +- .../batch/generated/PoolGetSamples.java | 26 +- .../PoolListByBatchAccountSamples.java | 4 +- .../generated/PoolStopResizeSamples.java | 2 +- .../batch/generated/PoolUpdateSamples.java | 8 +- ...rivateEndpointConnectionDeleteSamples.java | 2 +- .../PrivateEndpointConnectionGetSamples.java | 2 +- ...ntConnectionListByBatchAccountSamples.java | 2 +- ...rivateEndpointConnectionUpdateSamples.java | 2 +- .../PrivateLinkResourceGetSamples.java | 2 +- ...LinkResourceListByBatchAccountSamples.java | 2 +- .../resourcemanager/batch/BatchTests.java | 10 +- ...PackagesActivateWithResponseMockTests.java | 6 +- ...onPackagesCreateWithResponseMockTests.java | 6 +- ...onPackagesDeleteWithResponseMockTests.java | 2 +- ...ationPackagesGetWithResponseMockTests.java | 4 +- .../ApplicationPackagesListMockTests.java | 6 +- ...plicationsCreateWithResponseMockTests.java | 12 +- ...plicationsDeleteWithResponseMockTests.java | 3 +- .../ApplicationsGetWithResponseMockTests.java | 8 +- .../generated/ApplicationsListMockTests.java | 10 +- .../AutomaticOSUpgradePolicyTests.java | 33 + .../BatchAccountsDeleteMockTests.java | 2 +- ...ountsGetDetectorWithResponseMockTests.java | 7 +- ...hAccountsGetKeysWithResponseMockTests.java | 5 +- .../BatchAccountsListDetectorsMockTests.java | 6 +- ...NetworkDependenciesEndpointsMockTests.java | 7 +- ...eAutoStorageKeysWithResponseMockTests.java | 2 +- .../generated/BatchPoolIdentityTests.java | 8 +- .../CertificateBasePropertiesTests.java | 18 +- .../CertificatesDeleteMockTests.java | 2 +- .../CheckNameAvailabilityParametersTests.java | 8 +- ...CheckNameAvailabilityResultInnerTests.java | 2 +- .../generated/DetectorListResultTests.java | 17 +- .../generated/DetectorResponseInnerTests.java | 8 +- .../DetectorResponsePropertiesTests.java | 8 +- .../generated/EndpointDependencyTests.java | 2 +- .../batch/generated/EndpointDetailTests.java | 2 +- .../ListPrivateLinkResourcesResultTests.java | 5 +- ...NameAvailabilityWithResponseMockTests.java | 9 +- ...cationsGetQuotasWithResponseMockTests.java | 4 +- ...istSupportedCloudServiceSkusMockTests.java | 6 +- ...tSupportedVirtualMachineSkusMockTests.java | 6 +- .../generated/OperationDisplayTests.java | 22 +- .../batch/generated/OperationInnerTests.java | 34 +- .../generated/OperationListResultTests.java | 58 +- .../generated/OperationsListMockTests.java | 16 +- ...undEnvironmentEndpointCollectionTests.java | 8 +- ...OutboundEnvironmentEndpointInnerTests.java | 2 +- .../batch/generated/PoolsDeleteMockTests.java | 2 +- ...ntConnectionsGetWithResponseMockTests.java | 8 +- ...onnectionsListByBatchAccountMockTests.java | 9 +- ...ateEndpointConnectionsUpdateMockTests.java | 14 +- ...LinkResourcesGetWithResponseMockTests.java | 4 +- ...kResourcesListByBatchAccountMockTests.java | 6 +- .../generated/RollingUpgradePolicyTests.java | 41 + .../batch/generated/SkuCapabilityTests.java | 2 +- .../generated/SupportedSkuInnerTests.java | 2 +- .../generated/SupportedSkusResultTests.java | 2 +- .../batch/generated/UpgradePolicyTests.java | 59 ++ 127 files changed, 2009 insertions(+), 567 deletions(-) rename sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/{Utils.java => ResourceManagerUtils.java} (99%) create mode 100644 sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/AutomaticOSUpgradePolicy.java create mode 100644 sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/RollingUpgradePolicy.java create mode 100644 sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradeMode.java create mode 100644 sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradePolicy.java create mode 100644 sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/AutomaticOSUpgradePolicyTests.java create mode 100644 sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/RollingUpgradePolicyTests.java create mode 100644 sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/UpgradePolicyTests.java diff --git a/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md b/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md index 8ce9a2ef2121a..f63e592fc833d 100644 --- a/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md +++ b/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md @@ -1,14 +1,34 @@ # Release History -## 1.1.0-beta.4 (Unreleased) +## 1.1.0-beta.4 (2024-03-27) + +- Azure Resource Manager Batch client library for Java. This package contains Microsoft Azure SDK for Batch Management SDK. Batch Client. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ### Features Added -### Breaking Changes +* `models.UpgradePolicy` was added + +* `models.RollingUpgradePolicy` was added + +* `models.UpgradeMode` was added + +* `models.AutomaticOSUpgradePolicy` was added + +#### `models.SupportedSku` was modified + +* `batchSupportEndOfLife()` was added -### Bugs Fixed +#### `models.Pool$Definition` was modified + +* `withUpgradePolicy(models.UpgradePolicy)` was added + +#### `models.Pool` was modified + +* `upgradePolicy()` was added + +#### `models.Pool$Update` was modified -### Other Changes +* `withUpgradePolicy(models.UpgradePolicy)` was added ## 1.1.0-beta.3 (2023-12-22) diff --git a/sdk/batch/azure-resourcemanager-batch/README.md b/sdk/batch/azure-resourcemanager-batch/README.md index f908ce11a8f11..7144bd45b7cb8 100644 --- a/sdk/batch/azure-resourcemanager-batch/README.md +++ b/sdk/batch/azure-resourcemanager-batch/README.md @@ -2,7 +2,7 @@ Azure Resource Manager Batch client library for Java. -This package contains Microsoft Azure SDK for Batch Management SDK. Batch Client. Package tag package-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for Batch Management SDK. Batch Client. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-batch - 1.1.0-beta.3 + 1.1.0-beta.4 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/batch/azure-resourcemanager-batch/SAMPLE.md b/sdk/batch/azure-resourcemanager-batch/SAMPLE.md index 547c72d529c7b..98a2b88c9c403 100644 --- a/sdk/batch/azure-resourcemanager-batch/SAMPLE.md +++ b/sdk/batch/azure-resourcemanager-batch/SAMPLE.md @@ -81,7 +81,8 @@ */ public final class ApplicationCreateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationCreate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationCreate.json */ /** * Sample code: ApplicationCreate. @@ -89,7 +90,8 @@ public final class ApplicationCreateSamples { * @param manager Entry point to BatchManager. */ public static void applicationCreate(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applications().define("app1").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withDisplayName("myAppName").withAllowUpdates(false).create(); + manager.applications().define("app1").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withDisplayName("myAppName").withAllowUpdates(false).create(); } } ``` @@ -102,7 +104,8 @@ public final class ApplicationCreateSamples { */ public final class ApplicationDeleteSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationDelete.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationDelete.json */ /** * Sample code: ApplicationDelete. @@ -110,7 +113,8 @@ public final class ApplicationDeleteSamples { * @param manager Entry point to BatchManager. */ public static void applicationDelete(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applications().deleteWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", com.azure.core.util.Context.NONE); + manager.applications().deleteWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", + com.azure.core.util.Context.NONE); } } ``` @@ -123,7 +127,8 @@ public final class ApplicationDeleteSamples { */ public final class ApplicationGetSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationGet.json */ /** * Sample code: ApplicationGet. @@ -131,7 +136,8 @@ public final class ApplicationGetSamples { * @param manager Entry point to BatchManager. */ public static void applicationGet(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applications().getWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", com.azure.core.util.Context.NONE); + manager.applications().getWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", + com.azure.core.util.Context.NONE); } } ``` @@ -144,7 +150,8 @@ public final class ApplicationGetSamples { */ public final class ApplicationListSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationList.json */ /** * Sample code: ApplicationList. @@ -152,7 +159,8 @@ public final class ApplicationListSamples { * @param manager Entry point to BatchManager. */ public static void applicationList(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applications().list("default-azurebatch-japaneast", "sampleacct", null, com.azure.core.util.Context.NONE); + manager.applications().list("default-azurebatch-japaneast", "sampleacct", null, + com.azure.core.util.Context.NONE); } } ``` @@ -167,7 +175,8 @@ import com.azure.resourcemanager.batch.models.Application; */ public final class ApplicationUpdateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationUpdate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationUpdate.json */ /** * Sample code: ApplicationUpdate. @@ -175,7 +184,9 @@ public final class ApplicationUpdateSamples { * @param manager Entry point to BatchManager. */ public static void applicationUpdate(com.azure.resourcemanager.batch.BatchManager manager) { - Application resource = manager.applications().getWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", com.azure.core.util.Context.NONE).getValue(); + Application resource = manager.applications() + .getWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withDisplayName("myAppName").withAllowUpdates(true).withDefaultVersion("2").apply(); } } @@ -191,7 +202,8 @@ import com.azure.resourcemanager.batch.models.ActivateApplicationPackageParamete */ public final class ApplicationPackageActivateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageActivate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageActivate.json */ /** * Sample code: ApplicationPackageActivate. @@ -199,7 +211,8 @@ public final class ApplicationPackageActivateSamples { * @param manager Entry point to BatchManager. */ public static void applicationPackageActivate(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applicationPackages().activateWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", "1", new ActivateApplicationPackageParameters().withFormat("zip"), com.azure.core.util.Context.NONE); + manager.applicationPackages().activateWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", "1", + new ActivateApplicationPackageParameters().withFormat("zip"), com.azure.core.util.Context.NONE); } } ``` @@ -212,7 +225,8 @@ public final class ApplicationPackageActivateSamples { */ public final class ApplicationPackageCreateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageCreate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageCreate.json */ /** * Sample code: ApplicationPackageCreate. @@ -220,7 +234,8 @@ public final class ApplicationPackageCreateSamples { * @param manager Entry point to BatchManager. */ public static void applicationPackageCreate(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applicationPackages().define("1").withExistingApplication("default-azurebatch-japaneast", "sampleacct", "app1").create(); + manager.applicationPackages().define("1") + .withExistingApplication("default-azurebatch-japaneast", "sampleacct", "app1").create(); } } ``` @@ -233,7 +248,8 @@ public final class ApplicationPackageCreateSamples { */ public final class ApplicationPackageDeleteSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageDelete.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageDelete.json */ /** * Sample code: ApplicationPackageDelete. @@ -241,7 +257,8 @@ public final class ApplicationPackageDeleteSamples { * @param manager Entry point to BatchManager. */ public static void applicationPackageDelete(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applicationPackages().deleteWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", "1", com.azure.core.util.Context.NONE); + manager.applicationPackages().deleteWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", "1", + com.azure.core.util.Context.NONE); } } ``` @@ -254,7 +271,8 @@ public final class ApplicationPackageDeleteSamples { */ public final class ApplicationPackageGetSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageGet.json */ /** * Sample code: ApplicationPackageGet. @@ -262,7 +280,8 @@ public final class ApplicationPackageGetSamples { * @param manager Entry point to BatchManager. */ public static void applicationPackageGet(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applicationPackages().getWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", "1", com.azure.core.util.Context.NONE); + manager.applicationPackages().getWithResponse("default-azurebatch-japaneast", "sampleacct", "app1", "1", + com.azure.core.util.Context.NONE); } } ``` @@ -275,7 +294,8 @@ public final class ApplicationPackageGetSamples { */ public final class ApplicationPackageListSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageList.json */ /** * Sample code: ApplicationPackageList. @@ -283,7 +303,8 @@ public final class ApplicationPackageListSamples { * @param manager Entry point to BatchManager. */ public static void applicationPackageList(com.azure.resourcemanager.batch.BatchManager manager) { - manager.applicationPackages().list("default-azurebatch-japaneast", "sampleacct", "app1", null, com.azure.core.util.Context.NONE); + manager.applicationPackages().list("default-azurebatch-japaneast", "sampleacct", "app1", null, + com.azure.core.util.Context.NONE); } } ``` @@ -300,14 +321,14 @@ import com.azure.resourcemanager.batch.models.ResourceIdentityType; import com.azure.resourcemanager.batch.models.UserAssignedIdentities; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for BatchAccount Create. */ public final class BatchAccountCreateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountCreate_BYOS.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountCreate_BYOS.json */ /** * Sample code: BatchAccountCreate_BYOS. @@ -315,11 +336,20 @@ public final class BatchAccountCreateSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountCreateBYOS(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().define("sampleacct").withRegion("japaneast").withExistingResourceGroup("default-azurebatch-japaneast").withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")).withPoolAllocationMode(PoolAllocationMode.USER_SUBSCRIPTION).withKeyVaultReference(new KeyVaultReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.KeyVault/vaults/sample").withUrl("http://sample.vault.azure.net/")).create(); + manager.batchAccounts().define("sampleacct").withRegion("japaneast") + .withExistingResourceGroup("default-azurebatch-japaneast") + .withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")) + .withPoolAllocationMode(PoolAllocationMode.USER_SUBSCRIPTION) + .withKeyVaultReference(new KeyVaultReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.KeyVault/vaults/sample") + .withUrl("http://sample.vault.azure.net/")) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountCreate_UserAssignedIdentity.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * BatchAccountCreate_UserAssignedIdentity.json */ /** * Sample code: BatchAccountCreate_UserAssignedIdentity. @@ -327,11 +357,20 @@ public final class BatchAccountCreateSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountCreateUserAssignedIdentity(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().define("sampleacct").withRegion("japaneast").withExistingResourceGroup("default-azurebatch-japaneast").withIdentity(new BatchAccountIdentity().withType(ResourceIdentityType.USER_ASSIGNED).withUserAssignedIdentities(mapOf("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.ManagedIdentity/userAssignedIdentities/id1", new UserAssignedIdentities()))).withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")).create(); + manager.batchAccounts().define("sampleacct").withRegion("japaneast") + .withExistingResourceGroup("default-azurebatch-japaneast") + .withIdentity(new BatchAccountIdentity().withType(ResourceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.ManagedIdentity/userAssignedIdentities/id1", + new UserAssignedIdentities()))) + .withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateBatchAccountCreate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateBatchAccountCreate.json */ /** * Sample code: PrivateBatchAccountCreate. @@ -339,11 +378,19 @@ public final class BatchAccountCreateSamples { * @param manager Entry point to BatchManager. */ public static void privateBatchAccountCreate(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().define("sampleacct").withRegion("japaneast").withExistingResourceGroup("default-azurebatch-japaneast").withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")).withKeyVaultReference(new KeyVaultReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.KeyVault/vaults/sample").withUrl("http://sample.vault.azure.net/")).withPublicNetworkAccess(PublicNetworkAccessType.DISABLED).create(); + manager.batchAccounts().define("sampleacct").withRegion("japaneast") + .withExistingResourceGroup("default-azurebatch-japaneast") + .withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")) + .withKeyVaultReference(new KeyVaultReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.KeyVault/vaults/sample") + .withUrl("http://sample.vault.azure.net/")) + .withPublicNetworkAccess(PublicNetworkAccessType.DISABLED).create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountCreate_SystemAssignedIdentity.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * BatchAccountCreate_SystemAssignedIdentity.json */ /** * Sample code: BatchAccountCreate_SystemAssignedIdentity. @@ -351,11 +398,17 @@ public final class BatchAccountCreateSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountCreateSystemAssignedIdentity(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().define("sampleacct").withRegion("japaneast").withExistingResourceGroup("default-azurebatch-japaneast").withIdentity(new BatchAccountIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)).withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")).create(); + manager.batchAccounts().define("sampleacct").withRegion("japaneast") + .withExistingResourceGroup("default-azurebatch-japaneast") + .withIdentity(new BatchAccountIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)) + .withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountCreate_Default.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountCreate_Default.json */ /** * Sample code: BatchAccountCreate_Default. @@ -363,7 +416,11 @@ public final class BatchAccountCreateSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountCreateDefault(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().define("sampleacct").withRegion("japaneast").withExistingResourceGroup("default-azurebatch-japaneast").withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")).create(); + manager.batchAccounts().define("sampleacct").withRegion("japaneast") + .withExistingResourceGroup("default-azurebatch-japaneast") + .withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")) + .create(); } // Use "Map.of" if available @@ -388,7 +445,8 @@ public final class BatchAccountCreateSamples { */ public final class BatchAccountDeleteSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountDelete.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountDelete.json */ /** * Sample code: BatchAccountDelete. @@ -409,7 +467,8 @@ public final class BatchAccountDeleteSamples { */ public final class BatchAccountGetByResourceGroupSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateBatchAccountGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateBatchAccountGet.json */ /** * Sample code: PrivateBatchAccountGet. @@ -417,11 +476,13 @@ public final class BatchAccountGetByResourceGroupSamples { * @param manager Entry point to BatchManager. */ public static void privateBatchAccountGet(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().getByResourceGroupWithResponse("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE); + manager.batchAccounts().getByResourceGroupWithResponse("default-azurebatch-japaneast", "sampleacct", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountGet.json */ /** * Sample code: BatchAccountGet. @@ -429,7 +490,8 @@ public final class BatchAccountGetByResourceGroupSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountGet(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().getByResourceGroupWithResponse("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE); + manager.batchAccounts().getByResourceGroupWithResponse("default-azurebatch-japaneast", "sampleacct", + com.azure.core.util.Context.NONE); } } ``` @@ -442,7 +504,8 @@ public final class BatchAccountGetByResourceGroupSamples { */ public final class BatchAccountGetDetectorSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/DetectorGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/DetectorGet.json */ /** * Sample code: GetDetector. @@ -450,7 +513,8 @@ public final class BatchAccountGetDetectorSamples { * @param manager Entry point to BatchManager. */ public static void getDetector(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().getDetectorWithResponse("default-azurebatch-japaneast", "sampleacct", "poolsAndNodes", com.azure.core.util.Context.NONE); + manager.batchAccounts().getDetectorWithResponse("default-azurebatch-japaneast", "sampleacct", "poolsAndNodes", + com.azure.core.util.Context.NONE); } } ``` @@ -463,7 +527,8 @@ public final class BatchAccountGetDetectorSamples { */ public final class BatchAccountGetKeysSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountGetKeys.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountGetKeys.json */ /** * Sample code: BatchAccountGetKeys. @@ -471,7 +536,8 @@ public final class BatchAccountGetKeysSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountGetKeys(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().getKeysWithResponse("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE); + manager.batchAccounts().getKeysWithResponse("default-azurebatch-japaneast", "sampleacct", + com.azure.core.util.Context.NONE); } } ``` @@ -484,7 +550,8 @@ public final class BatchAccountGetKeysSamples { */ public final class BatchAccountListSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountList.json */ /** * Sample code: BatchAccountList. @@ -505,7 +572,9 @@ public final class BatchAccountListSamples { */ public final class BatchAccountListByResourceGroupSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountListByResourceGroup.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountListByResourceGroup. + * json */ /** * Sample code: BatchAccountListByResourceGroup. @@ -526,7 +595,8 @@ public final class BatchAccountListByResourceGroupSamples { */ public final class BatchAccountListDetectorsSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/DetectorList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/DetectorList.json */ /** * Sample code: ListDetectors. @@ -534,7 +604,8 @@ public final class BatchAccountListDetectorsSamples { * @param manager Entry point to BatchManager. */ public static void listDetectors(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().listDetectors("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE); + manager.batchAccounts().listDetectors("default-azurebatch-japaneast", "sampleacct", + com.azure.core.util.Context.NONE); } } ``` @@ -547,7 +618,8 @@ public final class BatchAccountListDetectorsSamples { */ public final class BatchAccountListOutboundNetworkDependenciesEndpointsSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountListOutboundNetworkDependenciesEndpoints.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * BatchAccountListOutboundNetworkDependenciesEndpoints.json */ /** * Sample code: ListOutboundNetworkDependencies. @@ -555,7 +627,8 @@ public final class BatchAccountListOutboundNetworkDependenciesEndpointsSamples { * @param manager Entry point to BatchManager. */ public static void listOutboundNetworkDependencies(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().listOutboundNetworkDependenciesEndpoints("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE); + manager.batchAccounts().listOutboundNetworkDependenciesEndpoints("default-azurebatch-japaneast", "sampleacct", + com.azure.core.util.Context.NONE); } } ``` @@ -565,14 +638,14 @@ public final class BatchAccountListOutboundNetworkDependenciesEndpointsSamples { ```java import com.azure.resourcemanager.batch.models.AccountKeyType; import com.azure.resourcemanager.batch.models.BatchAccountRegenerateKeyParameters; -import java.util.stream.Collectors; /** * Samples for BatchAccount RegenerateKey. */ public final class BatchAccountRegenerateKeySamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountRegenerateKey.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountRegenerateKey.json */ /** * Sample code: BatchAccountRegenerateKey. @@ -580,7 +653,9 @@ public final class BatchAccountRegenerateKeySamples { * @param manager Entry point to BatchManager. */ public static void batchAccountRegenerateKey(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().regenerateKeyWithResponse("default-azurebatch-japaneast", "sampleacct", new BatchAccountRegenerateKeyParameters().withKeyName(AccountKeyType.PRIMARY), com.azure.core.util.Context.NONE); + manager.batchAccounts().regenerateKeyWithResponse("default-azurebatch-japaneast", "sampleacct", + new BatchAccountRegenerateKeyParameters().withKeyName(AccountKeyType.PRIMARY), + com.azure.core.util.Context.NONE); } } ``` @@ -593,7 +668,8 @@ public final class BatchAccountRegenerateKeySamples { */ public final class BatchAccountSynchronizeAutoStorageKeysSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountSynchronizeAutoStorageKeys.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * BatchAccountSynchronizeAutoStorageKeys.json */ /** * Sample code: BatchAccountSynchronizeAutoStorageKeys. @@ -601,7 +677,8 @@ public final class BatchAccountSynchronizeAutoStorageKeysSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountSynchronizeAutoStorageKeys(com.azure.resourcemanager.batch.BatchManager manager) { - manager.batchAccounts().synchronizeAutoStorageKeysWithResponse("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE); + manager.batchAccounts().synchronizeAutoStorageKeysWithResponse("default-azurebatch-japaneast", "sampleacct", + com.azure.core.util.Context.NONE); } } ``` @@ -617,7 +694,8 @@ import com.azure.resourcemanager.batch.models.BatchAccount; */ public final class BatchAccountUpdateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountUpdate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountUpdate.json */ /** * Sample code: BatchAccountUpdate. @@ -625,8 +703,11 @@ public final class BatchAccountUpdateSamples { * @param manager Entry point to BatchManager. */ public static void batchAccountUpdate(com.azure.resourcemanager.batch.BatchManager manager) { - BatchAccount resource = manager.batchAccounts().getByResourceGroupWithResponse("default-azurebatch-japaneast", "sampleacct", com.azure.core.util.Context.NONE).getValue(); - resource.update().withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")).apply(); + BatchAccount resource = manager.batchAccounts().getByResourceGroupWithResponse("default-azurebatch-japaneast", + "sampleacct", com.azure.core.util.Context.NONE).getValue(); + resource.update().withAutoStorage(new AutoStorageBaseProperties().withStorageAccountId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Storage/storageAccounts/samplestorage")) + .apply(); } } ``` @@ -639,7 +720,8 @@ public final class BatchAccountUpdateSamples { */ public final class CertificateCancelDeletionSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCancelDeletion.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCancelDeletion.json */ /** * Sample code: CertificateCancelDeletion. @@ -647,7 +729,8 @@ public final class CertificateCancelDeletionSamples { * @param manager Entry point to BatchManager. */ public static void certificateCancelDeletion(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().cancelDeletionWithResponse("default-azurebatch-japaneast", "sampleacct", "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); + manager.certificates().cancelDeletionWithResponse("default-azurebatch-japaneast", "sampleacct", + "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); } } ``` @@ -656,14 +739,14 @@ public final class CertificateCancelDeletionSamples { ```java import com.azure.resourcemanager.batch.models.CertificateFormat; -import java.util.stream.Collectors; /** * Samples for Certificate Create. */ public final class CertificateCreateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCreate_Full.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCreate_Full.json */ /** * Sample code: CreateCertificate - Full. @@ -671,11 +754,15 @@ public final class CertificateCreateSamples { * @param manager Entry point to BatchManager. */ public static void createCertificateFull(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().define("sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withPassword("").withThumbprintAlgorithm("sha1").withThumbprint("0a0e4f50d51beadeac1d35afc5116098e7902e6e").withFormat(CertificateFormat.PFX).create(); + manager.certificates().define("sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e") + .withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withPassword("") + .withThumbprintAlgorithm("sha1").withThumbprint("0a0e4f50d51beadeac1d35afc5116098e7902e6e") + .withFormat(CertificateFormat.PFX).create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCreate_Minimal.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCreate_Minimal.json */ /** * Sample code: CreateCertificate - Minimal Pfx. @@ -683,11 +770,14 @@ public final class CertificateCreateSamples { * @param manager Entry point to BatchManager. */ public static void createCertificateMinimalPfx(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().define("sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withPassword("").create(); + manager.certificates().define("sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e") + .withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withPassword("") + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCreate_MinimalCer.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCreate_MinimalCer.json */ /** * Sample code: CreateCertificate - Minimal Cer. @@ -695,7 +785,9 @@ public final class CertificateCreateSamples { * @param manager Entry point to BatchManager. */ public static void createCertificateMinimalCer(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().define("sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withFormat(CertificateFormat.CER).create(); + manager.certificates().define("sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e") + .withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withFormat(CertificateFormat.CER) + .create(); } } ``` @@ -708,7 +800,8 @@ public final class CertificateCreateSamples { */ public final class CertificateDeleteSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateDelete.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateDelete.json */ /** * Sample code: CertificateDelete. @@ -716,7 +809,8 @@ public final class CertificateDeleteSamples { * @param manager Entry point to BatchManager. */ public static void certificateDelete(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().delete("default-azurebatch-japaneast", "sampleacct", "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); + manager.certificates().delete("default-azurebatch-japaneast", "sampleacct", + "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); } } ``` @@ -729,7 +823,9 @@ public final class CertificateDeleteSamples { */ public final class CertificateGetSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateGetWithDeletionError.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateGetWithDeletionError. + * json */ /** * Sample code: Get Certificate with Deletion Error. @@ -737,11 +833,13 @@ public final class CertificateGetSamples { * @param manager Entry point to BatchManager. */ public static void getCertificateWithDeletionError(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().getWithResponse("default-azurebatch-japaneast", "sampleacct", "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); + manager.certificates().getWithResponse("default-azurebatch-japaneast", "sampleacct", + "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateGet.json */ /** * Sample code: Get Certificate. @@ -749,7 +847,8 @@ public final class CertificateGetSamples { * @param manager Entry point to BatchManager. */ public static void getCertificate(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().getWithResponse("default-azurebatch-japaneast", "sampleacct", "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); + manager.certificates().getWithResponse("default-azurebatch-japaneast", "sampleacct", + "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE); } } ``` @@ -762,7 +861,8 @@ public final class CertificateGetSamples { */ public final class CertificateListByBatchAccountSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateListWithFilter.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateListWithFilter.json */ /** * Sample code: ListCertificates - Filter and Select. @@ -770,11 +870,15 @@ public final class CertificateListByBatchAccountSamples { * @param manager Entry point to BatchManager. */ public static void listCertificatesFilterAndSelect(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, "properties/format,properties/provisioningState", "properties/provisioningStateTransitionTime gt '2017-05-01' or properties/provisioningState eq 'Failed'", com.azure.core.util.Context.NONE); + manager.certificates().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, + "properties/format,properties/provisioningState", + "properties/provisioningStateTransitionTime gt '2017-05-01' or properties/provisioningState eq 'Failed'", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateList.json */ /** * Sample code: ListCertificates. @@ -782,7 +886,8 @@ public final class CertificateListByBatchAccountSamples { * @param manager Entry point to BatchManager. */ public static void listCertificates(com.azure.resourcemanager.batch.BatchManager manager) { - manager.certificates().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", 1, null, null, com.azure.core.util.Context.NONE); + manager.certificates().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", 1, null, null, + com.azure.core.util.Context.NONE); } } ``` @@ -797,7 +902,8 @@ import com.azure.resourcemanager.batch.models.Certificate; */ public final class CertificateUpdateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateUpdate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateUpdate.json */ /** * Sample code: UpdateCertificate. @@ -805,7 +911,8 @@ public final class CertificateUpdateSamples { * @param manager Entry point to BatchManager. */ public static void updateCertificate(com.azure.resourcemanager.batch.BatchManager manager) { - Certificate resource = manager.certificates().getWithResponse("default-azurebatch-japaneast", "sampleacct", "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE).getValue(); + Certificate resource = manager.certificates().getWithResponse("default-azurebatch-japaneast", "sampleacct", + "sha1-0a0e4f50d51beadeac1d35afc5116098e7902e6e", com.azure.core.util.Context.NONE).getValue(); resource.update().withData("MIIJsgIBAzCCCW4GCSqGSIb3DQE...").withPassword("").apply(); } } @@ -821,19 +928,23 @@ import com.azure.resourcemanager.batch.models.CheckNameAvailabilityParameters; */ public final class LocationCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationCheckNameAvailability_AlreadyExists.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * LocationCheckNameAvailability_AlreadyExists.json */ /** * Sample code: LocationCheckNameAvailability_AlreadyExists. * * @param manager Entry point to BatchManager. */ - public static void locationCheckNameAvailabilityAlreadyExists(com.azure.resourcemanager.batch.BatchManager manager) { - manager.locations().checkNameAvailabilityWithResponse("japaneast", new CheckNameAvailabilityParameters().withName("existingaccountname"), com.azure.core.util.Context.NONE); + public static void + locationCheckNameAvailabilityAlreadyExists(com.azure.resourcemanager.batch.BatchManager manager) { + manager.locations().checkNameAvailabilityWithResponse("japaneast", + new CheckNameAvailabilityParameters().withName("existingaccountname"), com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationCheckNameAvailability_Available.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * LocationCheckNameAvailability_Available.json */ /** * Sample code: LocationCheckNameAvailability_Available. @@ -841,7 +952,8 @@ public final class LocationCheckNameAvailabilitySamples { * @param manager Entry point to BatchManager. */ public static void locationCheckNameAvailabilityAvailable(com.azure.resourcemanager.batch.BatchManager manager) { - manager.locations().checkNameAvailabilityWithResponse("japaneast", new CheckNameAvailabilityParameters().withName("newaccountname"), com.azure.core.util.Context.NONE); + manager.locations().checkNameAvailabilityWithResponse("japaneast", + new CheckNameAvailabilityParameters().withName("newaccountname"), com.azure.core.util.Context.NONE); } } ``` @@ -854,7 +966,8 @@ public final class LocationCheckNameAvailabilitySamples { */ public final class LocationGetQuotasSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationGetQuotas.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/LocationGetQuotas.json */ /** * Sample code: LocationGetQuotas. @@ -875,7 +988,8 @@ public final class LocationGetQuotasSamples { */ public final class LocationListSupportedCloudServiceSkusSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationListCloudServiceSkus.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/LocationListCloudServiceSkus.json */ /** * Sample code: LocationListCloudServiceSkus. @@ -896,7 +1010,9 @@ public final class LocationListSupportedCloudServiceSkusSamples { */ public final class LocationListSupportedVirtualMachineSkusSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationListVirtualMachineSkus.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/LocationListVirtualMachineSkus. + * json */ /** * Sample code: LocationListVirtualMachineSkus. @@ -917,7 +1033,8 @@ public final class LocationListSupportedVirtualMachineSkusSamples { */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/OperationsList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/OperationsList.json */ /** * Sample code: OperationsList. @@ -939,6 +1056,7 @@ import com.azure.resourcemanager.batch.models.ApplicationPackageReference; import com.azure.resourcemanager.batch.models.AutoScaleSettings; import com.azure.resourcemanager.batch.models.AutoUserScope; import com.azure.resourcemanager.batch.models.AutoUserSpecification; +import com.azure.resourcemanager.batch.models.AutomaticOSUpgradePolicy; import com.azure.resourcemanager.batch.models.BatchPoolIdentity; import com.azure.resourcemanager.batch.models.CachingType; import com.azure.resourcemanager.batch.models.CertificateReference; @@ -975,6 +1093,7 @@ import com.azure.resourcemanager.batch.models.PoolEndpointConfiguration; import com.azure.resourcemanager.batch.models.PoolIdentityType; import com.azure.resourcemanager.batch.models.PublicIpAddressConfiguration; import com.azure.resourcemanager.batch.models.ResourceFile; +import com.azure.resourcemanager.batch.models.RollingUpgradePolicy; import com.azure.resourcemanager.batch.models.ScaleSettings; import com.azure.resourcemanager.batch.models.SecurityProfile; import com.azure.resourcemanager.batch.models.SecurityTypes; @@ -983,37 +1102,55 @@ import com.azure.resourcemanager.batch.models.StartTask; import com.azure.resourcemanager.batch.models.StorageAccountType; import com.azure.resourcemanager.batch.models.TaskSchedulingPolicy; import com.azure.resourcemanager.batch.models.UefiSettings; +import com.azure.resourcemanager.batch.models.UpgradeMode; +import com.azure.resourcemanager.batch.models.UpgradePolicy; import com.azure.resourcemanager.batch.models.UserAccount; import com.azure.resourcemanager.batch.models.UserAssignedIdentities; import com.azure.resourcemanager.batch.models.UserIdentity; -import com.azure.resourcemanager.batch.models.VirtualMachineConfiguration; import com.azure.resourcemanager.batch.models.VMExtension; +import com.azure.resourcemanager.batch.models.VirtualMachineConfiguration; import com.azure.resourcemanager.batch.models.WindowsConfiguration; import java.io.IOException; import java.time.Duration; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for Pool Create. */ public final class PoolCreateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_VirtualMachineConfiguration_ServiceArtifactReference.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_VirtualMachineConfiguration_ServiceArtifactReference.json */ /** * Sample code: CreatePool - VirtualMachineConfiguration ServiceArtifactReference. * * @param manager Entry point to BatchManager. */ - public static void createPoolVirtualMachineConfigurationServiceArtifactReference(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("Standard_d4s_v3").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer").withOffer("WindowsServer").withSku("2019-datacenter-smalldisk").withVersion("latest")).withNodeAgentSkuId("batch.node.windows amd64").withWindowsConfiguration(new WindowsConfiguration().withEnableAutomaticUpdates(false)).withServiceArtifactReference(new ServiceArtifactReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Compute/galleries/myGallery/serviceArtifacts/myServiceArtifact/vmArtifactsProfiles/vmArtifactsProfile")))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(2).withTargetLowPriorityNodes(0))).create(); + public static void createPoolVirtualMachineConfigurationServiceArtifactReference( + com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("Standard_d4s_v3") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer") + .withOffer("WindowsServer").withSku("2019-datacenter-smalldisk").withVersion("latest")) + .withNodeAgentSkuId("batch.node.windows amd64") + .withWindowsConfiguration(new WindowsConfiguration().withEnableAutomaticUpdates(false)) + .withServiceArtifactReference(new ServiceArtifactReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Compute/galleries/myGallery/serviceArtifacts/myServiceArtifact/vmArtifactsProfiles/vmArtifactsProfile")))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(2).withTargetLowPriorityNodes(0))) + .withUpgradePolicy(new UpgradePolicy().withMode(UpgradeMode.AUTOMATIC) + .withAutomaticOSUpgradePolicy(new AutomaticOSUpgradePolicy().withEnableAutomaticOSUpgrade(true))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_SecurityProfile.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_SecurityProfile.json */ /** * Sample code: CreatePool - SecurityProfile. @@ -1021,59 +1158,125 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolSecurityProfile(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("Standard_d4s_v3").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("Canonical").withOffer("UbuntuServer").withSku("18_04-lts-gen2").withVersion("latest")).withNodeAgentSkuId("batch.node.ubuntu 18.04").withSecurityProfile(new SecurityProfile().withSecurityType(SecurityTypes.TRUSTED_LAUNCH).withEncryptionAtHost(true).withUefiSettings(new UefiSettings().withVTpmEnabled(false))))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))).create(); + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("Standard_d4s_v3") + .withDeploymentConfiguration( + new DeploymentConfiguration() + .withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("Canonical").withOffer("UbuntuServer") + .withSku("18_04-lts-gen2").withVersion("latest")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04") + .withSecurityProfile(new SecurityProfile().withSecurityType(SecurityTypes.TRUSTED_LAUNCH) + .withEncryptionAtHost(true).withUefiSettings(new UefiSettings().withVTpmEnabled(false))))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_VirtualMachineConfiguration_ManagedOSDisk.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_VirtualMachineConfiguration_ManagedOSDisk.json */ /** * Sample code: CreatePool - VirtualMachineConfiguration OSDisk. * * @param manager Entry point to BatchManager. */ - public static void createPoolVirtualMachineConfigurationOSDisk(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("Standard_d2s_v3").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("microsoftwindowsserver").withOffer("windowsserver").withSku("2022-datacenter-smalldisk")).withNodeAgentSkuId("batch.node.windows amd64").withOsDisk(new OSDisk().withCaching(CachingType.READ_WRITE).withManagedDisk(new ManagedDisk().withStorageAccountType(StorageAccountType.STANDARD_SSD_LRS)).withDiskSizeGB(100).withWriteAcceleratorEnabled(false)))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))).create(); + public static void + createPoolVirtualMachineConfigurationOSDisk(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("Standard_d2s_v3") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("microsoftwindowsserver") + .withOffer("windowsserver").withSku("2022-datacenter-smalldisk")) + .withNodeAgentSkuId("batch.node.windows amd64") + .withOsDisk(new OSDisk().withCaching(CachingType.READ_WRITE) + .withManagedDisk(new ManagedDisk().withStorageAccountType(StorageAccountType.STANDARD_SSD_LRS)) + .withDiskSizeGB(100).withWriteAcceleratorEnabled(false)))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_MinimalCloudServiceConfiguration.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_MinimalCloudServiceConfiguration.json */ /** * Sample code: CreatePool - Minimal CloudServiceConfiguration. * * @param manager Entry point to BatchManager. */ - public static void createPoolMinimalCloudServiceConfiguration(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withCloudServiceConfiguration(new CloudServiceConfiguration().withOsFamily("5"))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(3))).create(); + public static void + createPoolMinimalCloudServiceConfiguration(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration(new DeploymentConfiguration() + .withCloudServiceConfiguration(new CloudServiceConfiguration().withOsFamily("5"))) + .withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(3))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_MinimalVirtualMachineConfiguration.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_MinimalVirtualMachineConfiguration.json */ /** * Sample code: CreatePool - Minimal VirtualMachineConfiguration. * * @param manager Entry point to BatchManager. */ - public static void createPoolMinimalVirtualMachineConfiguration(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("Canonical").withOffer("UbuntuServer").withSku("18.04-LTS").withVersion("latest")).withNodeAgentSkuId("batch.node.ubuntu 18.04"))).withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings().withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))).create(); + public static void + createPoolMinimalVirtualMachineConfiguration(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration( + new DeploymentConfiguration() + .withVirtualMachineConfiguration( + new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("Canonical") + .withOffer("UbuntuServer").withSku("18.04-LTS").withVersion("latest")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) + .withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings() + .withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_VirtualMachineConfiguration_Extensions.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_VirtualMachineConfiguration_Extensions.json */ /** * Sample code: CreatePool - VirtualMachineConfiguration Extensions. * * @param manager Entry point to BatchManager. */ - public static void createPoolVirtualMachineConfigurationExtensions(com.azure.resourcemanager.batch.BatchManager manager) throws IOException { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("Canonical").withOffer("0001-com-ubuntu-server-focal").withSku("20_04-lts")).withNodeAgentSkuId("batch.node.ubuntu 20.04").withExtensions(Arrays.asList(new VMExtension().withName("batchextension1").withPublisher("Microsoft.Azure.KeyVault").withType("KeyVaultForLinux").withTypeHandlerVersion("2.0").withAutoUpgradeMinorVersion(true).withEnableAutomaticUpgrade(true).withSettings(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"authenticationSettingsKey\":\"authenticationSettingsValue\",\"secretsManagementSettingsKey\":\"secretsManagementSettingsValue\"}", Object.class, SerializerEncoding.JSON)))))).withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings().withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))).withTargetNodeCommunicationMode(NodeCommunicationMode.DEFAULT).create(); + public static void createPoolVirtualMachineConfigurationExtensions( + com.azure.resourcemanager.batch.BatchManager manager) throws IOException { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("Canonical") + .withOffer("0001-com-ubuntu-server-focal").withSku("20_04-lts")) + .withNodeAgentSkuId("batch.node.ubuntu 20.04") + .withExtensions(Arrays.asList(new VMExtension().withName("batchextension1") + .withPublisher("Microsoft.Azure.KeyVault").withType("KeyVaultForLinux") + .withTypeHandlerVersion("2.0").withAutoUpgradeMinorVersion(true) + .withEnableAutomaticUpgrade(true) + .withSettings(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"authenticationSettingsKey\":\"authenticationSettingsValue\",\"secretsManagementSettingsKey\":\"secretsManagementSettingsValue\"}", + Object.class, SerializerEncoding.JSON)))))) + .withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings() + .withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))) + .withTargetNodeCommunicationMode(NodeCommunicationMode.DEFAULT).create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_UserAssignedIdentities.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_UserAssignedIdentities + * .json */ /** * Sample code: CreatePool - UserAssignedIdentities. @@ -1081,11 +1284,64 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolUserAssignedIdentities(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withIdentity(new BatchPoolIdentity().withType(PoolIdentityType.USER_ASSIGNED).withUserAssignedIdentities(mapOf("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.ManagedIdentity/userAssignedIdentities/id1", new UserAssignedIdentities(), "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.ManagedIdentity/userAssignedIdentities/id2", new UserAssignedIdentities()))).withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("Canonical").withOffer("UbuntuServer").withSku("18.04-LTS").withVersion("latest")).withNodeAgentSkuId("batch.node.ubuntu 18.04"))).withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings().withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))).create(); - } - - /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_AcceleratedNetworking.json + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withIdentity(new BatchPoolIdentity().withType(PoolIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.ManagedIdentity/userAssignedIdentities/id1", + new UserAssignedIdentities(), + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.ManagedIdentity/userAssignedIdentities/id2", + new UserAssignedIdentities()))) + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration( + new DeploymentConfiguration() + .withVirtualMachineConfiguration( + new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("Canonical") + .withOffer("UbuntuServer").withSku("18.04-LTS").withVersion("latest")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) + .withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings() + .withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))) + .create(); + } + + /* + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_UpgradePolicy.json + */ + /** + * Sample code: CreatePool - UpgradePolicy. + * + * @param manager Entry point to BatchManager. + */ + public static void createPoolUpgradePolicy(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("Standard_d4s_v3") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer") + .withOffer("WindowsServer").withSku("2019-datacenter-smalldisk").withVersion("latest")) + .withNodeAgentSkuId("batch.node.windows amd64") + .withWindowsConfiguration(new WindowsConfiguration().withEnableAutomaticUpdates(false)) + .withNodePlacementConfiguration( + new NodePlacementConfiguration().withPolicy(NodePlacementPolicyType.ZONAL)))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(2).withTargetLowPriorityNodes(0))) + .withUpgradePolicy( + new UpgradePolicy().withMode(UpgradeMode.AUTOMATIC) + .withAutomaticOSUpgradePolicy(new AutomaticOSUpgradePolicy().withDisableAutomaticRollback(true) + .withEnableAutomaticOSUpgrade(true).withUseRollingUpgradePolicy(true) + .withOsRollingUpgradeDeferral(true)) + .withRollingUpgradePolicy(new RollingUpgradePolicy().withEnableCrossZoneUpgrade(true) + .withMaxBatchInstancePercent(20).withMaxUnhealthyInstancePercent(20) + .withMaxUnhealthyUpgradedInstancePercent(20).withPauseTimeBetweenBatches("PT0S") + .withPrioritizeUnhealthyInstances(false).withRollbackFailedInstancesOnPolicyBreach(false))) + .create(); + } + + /* + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_AcceleratedNetworking. + * json */ /** * Sample code: CreatePool - accelerated networking. @@ -1093,11 +1349,24 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolAcceleratedNetworking(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D1_V2").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer").withOffer("WindowsServer").withSku("2016-datacenter-smalldisk").withVersion("latest")).withNodeAgentSkuId("batch.node.windows amd64"))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))).withNetworkConfiguration(new NetworkConfiguration().withSubnetId("/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123").withEnableAcceleratedNetworking(true)).create(); + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D1_V2") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer") + .withOffer("WindowsServer").withSku("2016-datacenter-smalldisk").withVersion("latest")) + .withNodeAgentSkuId("batch.node.windows amd64"))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))) + .withNetworkConfiguration(new NetworkConfiguration().withSubnetId( + "/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123") + .withEnableAcceleratedNetworking(true)) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_VirtualMachineConfiguration.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_VirtualMachineConfiguration.json */ /** * Sample code: CreatePool - Full VirtualMachineConfiguration. @@ -1105,11 +1374,51 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolFullVirtualMachineConfiguration(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer").withOffer("WindowsServer").withSku("2016-Datacenter-SmallDisk").withVersion("latest")).withNodeAgentSkuId("batch.node.windows amd64").withWindowsConfiguration(new WindowsConfiguration().withEnableAutomaticUpdates(false)).withDataDisks(Arrays.asList(new DataDisk().withLun(0).withCaching(CachingType.READ_WRITE).withDiskSizeGB(30).withStorageAccountType(StorageAccountType.PREMIUM_LRS), new DataDisk().withLun(1).withCaching(CachingType.NONE).withDiskSizeGB(200).withStorageAccountType(StorageAccountType.STANDARD_LRS))).withLicenseType("Windows_Server").withDiskEncryptionConfiguration(new DiskEncryptionConfiguration().withTargets(Arrays.asList(DiskEncryptionTarget.OS_DISK, DiskEncryptionTarget.TEMPORARY_DISK))).withNodePlacementConfiguration(new NodePlacementConfiguration().withPolicy(NodePlacementPolicyType.ZONAL)).withOsDisk(new OSDisk().withEphemeralOSDiskSettings(new DiffDiskSettings().withPlacement(DiffDiskPlacement.CACHE_DISK))))).withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings().withFormula("$TargetDedicatedNodes=1").withEvaluationInterval(Duration.parse("PT5M")))).withNetworkConfiguration(new NetworkConfiguration().withEndpointConfiguration(new PoolEndpointConfiguration().withInboundNatPools(Arrays.asList(new InboundNatPool().withName("testnat").withProtocol(InboundEndpointProtocol.TCP).withBackendPort(12001).withFrontendPortRangeStart(15000).withFrontendPortRangeEnd(15100).withNetworkSecurityGroupRules(Arrays.asList(new NetworkSecurityGroupRule().withPriority(150).withAccess(NetworkSecurityGroupRuleAccess.ALLOW).withSourceAddressPrefix("192.100.12.45").withSourcePortRanges(Arrays.asList("1", "2")), new NetworkSecurityGroupRule().withPriority(3500).withAccess(NetworkSecurityGroupRuleAccess.DENY).withSourceAddressPrefix("*").withSourcePortRanges(Arrays.asList("*")))))))).create(); - } - - /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_SharedImageGallery.json + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration( + new DeploymentConfiguration() + .withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer") + .withOffer("WindowsServer").withSku("2016-Datacenter-SmallDisk").withVersion("latest")) + .withNodeAgentSkuId("batch.node.windows amd64") + .withWindowsConfiguration(new WindowsConfiguration().withEnableAutomaticUpdates(false)) + .withDataDisks(Arrays.asList( + new DataDisk().withLun(0).withCaching(CachingType.READ_WRITE).withDiskSizeGB(30) + .withStorageAccountType(StorageAccountType.PREMIUM_LRS), + new DataDisk().withLun(1).withCaching(CachingType.NONE).withDiskSizeGB(200) + .withStorageAccountType(StorageAccountType.STANDARD_LRS))) + .withLicenseType("Windows_Server") + .withDiskEncryptionConfiguration(new DiskEncryptionConfiguration().withTargets( + Arrays.asList(DiskEncryptionTarget.OS_DISK, DiskEncryptionTarget.TEMPORARY_DISK))) + .withNodePlacementConfiguration( + new NodePlacementConfiguration().withPolicy(NodePlacementPolicyType.ZONAL)) + .withOsDisk(new OSDisk().withEphemeralOSDiskSettings( + new DiffDiskSettings().withPlacement(DiffDiskPlacement.CACHE_DISK))))) + .withScaleSettings( + new ScaleSettings() + .withAutoScale( + new AutoScaleSettings().withFormula( + "$TargetDedicatedNodes=1").withEvaluationInterval( + Duration.parse("PT5M")))) + .withNetworkConfiguration( + new NetworkConfiguration() + .withEndpointConfiguration(new PoolEndpointConfiguration().withInboundNatPools( + Arrays.asList(new InboundNatPool().withName("testnat").withProtocol(InboundEndpointProtocol.TCP) + .withBackendPort(12001).withFrontendPortRangeStart(15000).withFrontendPortRangeEnd(15100) + .withNetworkSecurityGroupRules(Arrays.asList(new NetworkSecurityGroupRule() + .withPriority(150).withAccess(NetworkSecurityGroupRuleAccess.ALLOW) + .withSourceAddressPrefix("192.100.12.45").withSourcePortRanges(Arrays.asList("1", "2")), + new NetworkSecurityGroupRule().withPriority(3500) + .withAccess(NetworkSecurityGroupRuleAccess.DENY).withSourceAddressPrefix("*") + .withSourcePortRanges(Arrays.asList("*")))))))) + .create(); + } + + /* + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_SharedImageGallery. + * json */ /** * Sample code: CreatePool - Custom Image. @@ -1117,11 +1426,18 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolCustomImage(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withId("/subscriptions/subid/resourceGroups/networking-group/providers/Microsoft.Compute/galleries/testgallery/images/testimagedef/versions/0.0.1")).withNodeAgentSkuId("batch.node.ubuntu 18.04"))).create(); + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration( + new VirtualMachineConfiguration().withImageReference(new ImageReference().withId( + "/subscriptions/subid/resourceGroups/networking-group/providers/Microsoft.Compute/galleries/testgallery/images/testimagedef/versions/0.0.1")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_CloudServiceConfiguration.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolCreate_CloudServiceConfiguration.json */ /** * Sample code: CreatePool - Full CloudServiceConfiguration. @@ -1129,11 +1445,51 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolFullCloudServiceConfiguration(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withDisplayName("my-pool-name").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withCloudServiceConfiguration(new CloudServiceConfiguration().withOsFamily("4").withOsVersion("WA-GUEST-OS-4.45_201708-01"))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withResizeTimeout(Duration.parse("PT8M")).withTargetDedicatedNodes(6).withTargetLowPriorityNodes(28).withNodeDeallocationOption(ComputeNodeDeallocationOption.TASK_COMPLETION))).withInterNodeCommunication(InterNodeCommunicationState.ENABLED).withNetworkConfiguration(new NetworkConfiguration().withSubnetId("/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123").withPublicIpAddressConfiguration(new PublicIpAddressConfiguration().withProvision(IpAddressProvisioningType.USER_MANAGED).withIpAddressIds(Arrays.asList("/subscriptions/subid1/resourceGroups/rg13/providers/Microsoft.Network/publicIPAddresses/ip135", "/subscriptions/subid2/resourceGroups/rg24/providers/Microsoft.Network/publicIPAddresses/ip268")))).withTaskSlotsPerNode(13).withTaskSchedulingPolicy(new TaskSchedulingPolicy().withNodeFillType(ComputeNodeFillType.PACK)).withUserAccounts(Arrays.asList(new UserAccount().withName("username1").withPassword("fakeTokenPlaceholder").withElevationLevel(ElevationLevel.ADMIN).withLinuxUserConfiguration(new LinuxUserConfiguration().withUid(1234).withGid(4567).withSshPrivateKey("fakeTokenPlaceholder")))).withMetadata(Arrays.asList(new MetadataItem().withName("metadata-1").withValue("value-1"), new MetadataItem().withName("metadata-2").withValue("value-2"))).withStartTask(new StartTask().withCommandLine("cmd /c SET").withResourceFiles(Arrays.asList(new ResourceFile().withHttpUrl("https://testaccount.blob.core.windows.net/example-blob-file").withFilePath("c:\\temp\\gohere").withFileMode("777"))).withEnvironmentSettings(Arrays.asList(new EnvironmentSetting().withName("MYSET").withValue("1234"))).withUserIdentity(new UserIdentity().withAutoUser(new AutoUserSpecification().withScope(AutoUserScope.POOL).withElevationLevel(ElevationLevel.ADMIN))).withMaxTaskRetryCount(6).withWaitForSuccess(true)).withCertificates(Arrays.asList(new CertificateReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/certificates/sha1-1234567").withStoreLocation(CertificateStoreLocation.LOCAL_MACHINE).withStoreName("MY").withVisibility(Arrays.asList(CertificateVisibility.REMOTE_USER)))).withApplicationPackages(Arrays.asList(new ApplicationPackageReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/applications/app_1234").withVersion("asdf"))).withApplicationLicenses(Arrays.asList("app-license0", "app-license1")).create(); - } - - /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_NoPublicIPAddresses.json + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withDisplayName("my-pool-name").withVmSize("STANDARD_D4") + .withDeploymentConfiguration(new DeploymentConfiguration().withCloudServiceConfiguration( + new CloudServiceConfiguration().withOsFamily("4").withOsVersion("WA-GUEST-OS-4.45_201708-01"))) + .withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings() + .withResizeTimeout(Duration.parse("PT8M")).withTargetDedicatedNodes(6).withTargetLowPriorityNodes(28) + .withNodeDeallocationOption(ComputeNodeDeallocationOption.TASK_COMPLETION))) + .withInterNodeCommunication(InterNodeCommunicationState.ENABLED) + .withNetworkConfiguration(new NetworkConfiguration().withSubnetId( + "/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123") + .withPublicIpAddressConfiguration(new PublicIpAddressConfiguration() + .withProvision(IpAddressProvisioningType.USER_MANAGED) + .withIpAddressIds(Arrays.asList( + "/subscriptions/subid1/resourceGroups/rg13/providers/Microsoft.Network/publicIPAddresses/ip135", + "/subscriptions/subid2/resourceGroups/rg24/providers/Microsoft.Network/publicIPAddresses/ip268")))) + .withTaskSlotsPerNode(13) + .withTaskSchedulingPolicy(new TaskSchedulingPolicy().withNodeFillType(ComputeNodeFillType.PACK)) + .withUserAccounts(Arrays.asList(new UserAccount().withName("username1").withPassword("fakeTokenPlaceholder") + .withElevationLevel(ElevationLevel.ADMIN) + .withLinuxUserConfiguration(new LinuxUserConfiguration().withUid(1234).withGid(4567) + .withSshPrivateKey("fakeTokenPlaceholder")))) + .withMetadata(Arrays.asList(new MetadataItem().withName("metadata-1").withValue("value-1"), + new MetadataItem().withName("metadata-2").withValue("value-2"))) + .withStartTask(new StartTask().withCommandLine("cmd /c SET") + .withResourceFiles(Arrays.asList( + new ResourceFile().withHttpUrl("https://testaccount.blob.core.windows.net/example-blob-file") + .withFilePath("c:\\temp\\gohere").withFileMode("777"))) + .withEnvironmentSettings(Arrays.asList(new EnvironmentSetting().withName("MYSET").withValue("1234"))) + .withUserIdentity(new UserIdentity().withAutoUser( + new AutoUserSpecification().withScope(AutoUserScope.POOL).withElevationLevel(ElevationLevel.ADMIN))) + .withMaxTaskRetryCount(6).withWaitForSuccess(true)) + .withCertificates(Arrays.asList(new CertificateReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/certificates/sha1-1234567") + .withStoreLocation(CertificateStoreLocation.LOCAL_MACHINE).withStoreName("MY") + .withVisibility(Arrays.asList(CertificateVisibility.REMOTE_USER)))) + .withApplicationPackages(Arrays.asList(new ApplicationPackageReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/applications/app_1234") + .withVersion("asdf"))) + .withApplicationLicenses(Arrays.asList("app-license0", "app-license1")).create(); + } + + /* + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_NoPublicIPAddresses. + * json */ /** * Sample code: CreatePool - No public IP. @@ -1141,11 +1497,22 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolNoPublicIP(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withId("/subscriptions/subid/resourceGroups/networking-group/providers/Microsoft.Compute/galleries/testgallery/images/testimagedef/versions/0.0.1")).withNodeAgentSkuId("batch.node.ubuntu 18.04"))).withNetworkConfiguration(new NetworkConfiguration().withSubnetId("/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123").withPublicIpAddressConfiguration(new PublicIpAddressConfiguration().withProvision(IpAddressProvisioningType.NO_PUBLIC_IPADDRESSES))).create(); + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration( + new VirtualMachineConfiguration().withImageReference(new ImageReference().withId( + "/subscriptions/subid/resourceGroups/networking-group/providers/Microsoft.Compute/galleries/testgallery/images/testimagedef/versions/0.0.1")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) + .withNetworkConfiguration(new NetworkConfiguration().withSubnetId( + "/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123") + .withPublicIpAddressConfiguration( + new PublicIpAddressConfiguration().withProvision(IpAddressProvisioningType.NO_PUBLIC_IPADDRESSES))) + .create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_ResourceTags.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_ResourceTags.json */ /** * Sample code: CreatePool - ResourceTags. @@ -1153,11 +1520,21 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolResourceTags(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("Standard_d4s_v3").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withPublisher("Canonical").withOffer("UbuntuServer").withSku("18_04-lts-gen2").withVersion("latest")).withNodeAgentSkuId("batch.node.ubuntu 18.04"))).withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))).withResourceTags(mapOf("TagName1", "TagValue1", "TagName2", "TagValue2")).create(); + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("Standard_d4s_v3") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("Canonical").withOffer("UbuntuServer") + .withSku("18_04-lts-gen2").withVersion("latest")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(1).withTargetLowPriorityNodes(0))) + .withResourceTags(mapOf("TagName1", "TagValue1", "TagName2", "TagValue2")).create(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_PublicIPs.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_PublicIPs.json */ /** * Sample code: CreatePool - Public IPs. @@ -1165,7 +1542,19 @@ public final class PoolCreateSamples { * @param manager Entry point to BatchManager. */ public static void createPoolPublicIPs(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct").withVmSize("STANDARD_D4").withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration().withImageReference(new ImageReference().withId("/subscriptions/subid/resourceGroups/networking-group/providers/Microsoft.Compute/galleries/testgallery/images/testimagedef/versions/0.0.1")).withNodeAgentSkuId("batch.node.ubuntu 18.04"))).withNetworkConfiguration(new NetworkConfiguration().withSubnetId("/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123").withPublicIpAddressConfiguration(new PublicIpAddressConfiguration().withProvision(IpAddressProvisioningType.USER_MANAGED).withIpAddressIds(Arrays.asList("/subscriptions/subid1/resourceGroups/rg13/providers/Microsoft.Network/publicIPAddresses/ip135")))).create(); + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("STANDARD_D4") + .withDeploymentConfiguration(new DeploymentConfiguration().withVirtualMachineConfiguration( + new VirtualMachineConfiguration().withImageReference(new ImageReference().withId( + "/subscriptions/subid/resourceGroups/networking-group/providers/Microsoft.Compute/galleries/testgallery/images/testimagedef/versions/0.0.1")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) + .withNetworkConfiguration(new NetworkConfiguration().withSubnetId( + "/subscriptions/subid/resourceGroups/rg1234/providers/Microsoft.Network/virtualNetworks/network1234/subnets/subnet123") + .withPublicIpAddressConfiguration(new PublicIpAddressConfiguration() + .withProvision(IpAddressProvisioningType.USER_MANAGED) + .withIpAddressIds(Arrays.asList( + "/subscriptions/subid1/resourceGroups/rg13/providers/Microsoft.Network/publicIPAddresses/ip135")))) + .create(); } // Use "Map.of" if available @@ -1190,7 +1579,8 @@ public final class PoolCreateSamples { */ public final class PoolDeleteSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolDelete.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolDelete.json */ /** * Sample code: DeletePool. @@ -1198,7 +1588,8 @@ public final class PoolDeleteSamples { * @param manager Entry point to BatchManager. */ public static void deletePool(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().delete("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().delete("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } } ``` @@ -1211,7 +1602,8 @@ public final class PoolDeleteSamples { */ public final class PoolDisableAutoScaleSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolDisableAutoScale.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolDisableAutoScale.json */ /** * Sample code: Disable AutoScale. @@ -1219,7 +1611,8 @@ public final class PoolDisableAutoScaleSamples { * @param manager Entry point to BatchManager. */ public static void disableAutoScale(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().disableAutoScaleWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().disableAutoScaleWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } } ``` @@ -1232,7 +1625,8 @@ public final class PoolDisableAutoScaleSamples { */ public final class PoolGetSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_SecurityProfile.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet_SecurityProfile.json */ /** * Sample code: GetPool - SecurityProfile. @@ -1240,23 +1634,28 @@ public final class PoolGetSamples { * @param manager Entry point to BatchManager. */ public static void getPoolSecurityProfile(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_VirtualMachineConfiguration_Extensions.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolGet_VirtualMachineConfiguration_Extensions.json */ /** * Sample code: GetPool - VirtualMachineConfiguration Extensions. * * @param manager Entry point to BatchManager. */ - public static void getPoolVirtualMachineConfigurationExtensions(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + public static void + getPoolVirtualMachineConfigurationExtensions(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_VirtualMachineConfiguration_MangedOSDisk.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolGet_VirtualMachineConfiguration_MangedOSDisk.json */ /** * Sample code: GetPool - VirtualMachineConfiguration OSDisk. @@ -1264,23 +1663,43 @@ public final class PoolGetSamples { * @param manager Entry point to BatchManager. */ public static void getPoolVirtualMachineConfigurationOSDisk(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet_UpgradePolicy.json + */ + /** + * Sample code: GetPool - UpgradePolicy. + * + * @param manager Entry point to BatchManager. + */ + public static void getPoolUpgradePolicy(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_VirtualMachineConfiguration_ServiceArtifactReference.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ + * PoolGet_VirtualMachineConfiguration_ServiceArtifactReference.json */ /** * Sample code: GetPool - VirtualMachineConfiguration ServiceArtifactReference. * * @param manager Entry point to BatchManager. */ - public static void getPoolVirtualMachineConfigurationServiceArtifactReference(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + public static void getPoolVirtualMachineConfigurationServiceArtifactReference( + com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_AcceleratedNetworking.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet_AcceleratedNetworking. + * json */ /** * Sample code: GetPool - AcceleratedNetworking. @@ -1288,11 +1707,12 @@ public final class PoolGetSamples { * @param manager Entry point to BatchManager. */ public static void getPoolAcceleratedNetworking(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet.json */ /** * Sample code: GetPool. @@ -1300,7 +1720,8 @@ public final class PoolGetSamples { * @param manager Entry point to BatchManager. */ public static void getPool(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } } ``` @@ -1313,7 +1734,7 @@ public final class PoolGetSamples { */ public final class PoolListByBatchAccountSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolList.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolList.json */ /** * Sample code: ListPool. @@ -1321,11 +1742,13 @@ public final class PoolListByBatchAccountSamples { * @param manager Entry point to BatchManager. */ public static void listPool(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, null, null, com.azure.core.util.Context.NONE); + manager.pools().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, null, null, + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolListWithFilter.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolListWithFilter.json */ /** * Sample code: ListPoolWithFilter. @@ -1333,7 +1756,10 @@ public final class PoolListByBatchAccountSamples { * @param manager Entry point to BatchManager. */ public static void listPoolWithFilter(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", 50, "properties/allocationState,properties/provisioningStateTransitionTime,properties/currentDedicatedNodes,properties/currentLowPriorityNodes", "startswith(name, 'po') or (properties/allocationState eq 'Steady' and properties/provisioningStateTransitionTime lt datetime'2017-02-02')", com.azure.core.util.Context.NONE); + manager.pools().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", 50, + "properties/allocationState,properties/provisioningStateTransitionTime,properties/currentDedicatedNodes,properties/currentLowPriorityNodes", + "startswith(name, 'po') or (properties/allocationState eq 'Steady' and properties/provisioningStateTransitionTime lt datetime'2017-02-02')", + com.azure.core.util.Context.NONE); } } ``` @@ -1346,7 +1772,8 @@ public final class PoolListByBatchAccountSamples { */ public final class PoolStopResizeSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolStopResize.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolStopResize.json */ /** * Sample code: StopPoolResize. @@ -1354,7 +1781,8 @@ public final class PoolStopResizeSamples { * @param manager Entry point to BatchManager. */ public static void stopPoolResize(com.azure.resourcemanager.batch.BatchManager manager) { - manager.pools().stopResizeWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE); + manager.pools().stopResizeWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); } } ``` @@ -1375,14 +1803,14 @@ import com.azure.resourcemanager.batch.models.ScaleSettings; import com.azure.resourcemanager.batch.models.StartTask; import java.time.Duration; import java.util.Arrays; -import java.util.stream.Collectors; /** * Samples for Pool Update. */ public final class PoolUpdateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_EnableAutoScale.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_EnableAutoScale.json */ /** * Sample code: UpdatePool - Enable Autoscale. @@ -1390,12 +1818,18 @@ public final class PoolUpdateSamples { * @param manager Entry point to BatchManager. */ public static void updatePoolEnableAutoscale(com.azure.resourcemanager.batch.BatchManager manager) { - Pool resource = manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE).getValue(); - resource.update().withScaleSettings(new ScaleSettings().withAutoScale(new AutoScaleSettings().withFormula("$TargetDedicatedNodes=34"))).apply(); + Pool resource = manager.pools() + .getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE) + .getValue(); + resource.update() + .withScaleSettings( + new ScaleSettings().withAutoScale(new AutoScaleSettings().withFormula("$TargetDedicatedNodes=34"))) + .apply(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_RemoveStartTask.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_RemoveStartTask.json */ /** * Sample code: UpdatePool - Remove Start Task. @@ -1403,12 +1837,15 @@ public final class PoolUpdateSamples { * @param manager Entry point to BatchManager. */ public static void updatePoolRemoveStartTask(com.azure.resourcemanager.batch.BatchManager manager) { - Pool resource = manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE).getValue(); + Pool resource = manager.pools() + .getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withStartTask(new StartTask()).apply(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_ResizePool.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_ResizePool.json */ /** * Sample code: UpdatePool - Resize Pool. @@ -1416,12 +1853,19 @@ public final class PoolUpdateSamples { * @param manager Entry point to BatchManager. */ public static void updatePoolResizePool(com.azure.resourcemanager.batch.BatchManager manager) { - Pool resource = manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE).getValue(); - resource.update().withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings().withResizeTimeout(Duration.parse("PT8M")).withTargetDedicatedNodes(5).withTargetLowPriorityNodes(0).withNodeDeallocationOption(ComputeNodeDeallocationOption.TASK_COMPLETION))).apply(); + Pool resource = manager.pools() + .getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE) + .getValue(); + resource.update() + .withScaleSettings(new ScaleSettings().withFixedScale(new FixedScaleSettings() + .withResizeTimeout(Duration.parse("PT8M")).withTargetDedicatedNodes(5).withTargetLowPriorityNodes(0) + .withNodeDeallocationOption(ComputeNodeDeallocationOption.TASK_COMPLETION))) + .apply(); } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_OtherProperties.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_OtherProperties.json */ /** * Sample code: UpdatePool - Other Properties. @@ -1429,8 +1873,19 @@ public final class PoolUpdateSamples { * @param manager Entry point to BatchManager. */ public static void updatePoolOtherProperties(com.azure.resourcemanager.batch.BatchManager manager) { - Pool resource = manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE).getValue(); - resource.update().withMetadata(Arrays.asList(new MetadataItem().withName("key1").withValue("value1"))).withCertificates(Arrays.asList(new CertificateReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/certificates/sha1-1234567").withStoreLocation(CertificateStoreLocation.LOCAL_MACHINE).withStoreName("MY"))).withApplicationPackages(Arrays.asList(new ApplicationPackageReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/applications/app_1234"), new ApplicationPackageReference().withId("/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/applications/app_5678").withVersion("1.0"))).withTargetNodeCommunicationMode(NodeCommunicationMode.SIMPLIFIED).apply(); + Pool resource = manager.pools() + .getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withMetadata(Arrays.asList(new MetadataItem().withName("key1").withValue("value1"))) + .withCertificates(Arrays.asList(new CertificateReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/certificates/sha1-1234567") + .withStoreLocation(CertificateStoreLocation.LOCAL_MACHINE).withStoreName("MY"))) + .withApplicationPackages(Arrays.asList(new ApplicationPackageReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/applications/app_1234"), + new ApplicationPackageReference().withId( + "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Batch/batchAccounts/sampleacct/pools/testpool/applications/app_5678") + .withVersion("1.0"))) + .withTargetNodeCommunicationMode(NodeCommunicationMode.SIMPLIFIED).apply(); } } ``` @@ -1443,7 +1898,9 @@ public final class PoolUpdateSamples { */ public final class PrivateEndpointConnectionDeleteSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionDelete.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionDelete. + * json */ /** * Sample code: PrivateEndpointConnectionDelete. @@ -1451,7 +1908,9 @@ public final class PrivateEndpointConnectionDeleteSamples { * @param manager Entry point to BatchManager. */ public static void privateEndpointConnectionDelete(com.azure.resourcemanager.batch.BatchManager manager) { - manager.privateEndpointConnections().delete("default-azurebatch-japaneast", "sampleacct", "testprivateEndpointConnection5testprivateEndpointConnection5.24d6b4b5-e65c-4330-bbe9-3a290d62f8e0", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().delete("default-azurebatch-japaneast", "sampleacct", + "testprivateEndpointConnection5testprivateEndpointConnection5.24d6b4b5-e65c-4330-bbe9-3a290d62f8e0", + com.azure.core.util.Context.NONE); } } ``` @@ -1464,7 +1923,8 @@ public final class PrivateEndpointConnectionDeleteSamples { */ public final class PrivateEndpointConnectionGetSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionGet.json */ /** * Sample code: GetPrivateEndpointConnection. @@ -1472,7 +1932,9 @@ public final class PrivateEndpointConnectionGetSamples { * @param manager Entry point to BatchManager. */ public static void getPrivateEndpointConnection(com.azure.resourcemanager.batch.BatchManager manager) { - manager.privateEndpointConnections().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testprivateEndpointConnection5testprivateEndpointConnection5.24d6b4b5-e65c-4330-bbe9-3a290d62f8e0", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().getWithResponse("default-azurebatch-japaneast", "sampleacct", + "testprivateEndpointConnection5testprivateEndpointConnection5.24d6b4b5-e65c-4330-bbe9-3a290d62f8e0", + com.azure.core.util.Context.NONE); } } ``` @@ -1485,7 +1947,9 @@ public final class PrivateEndpointConnectionGetSamples { */ public final class PrivateEndpointConnectionListByBatchAccountSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionsList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionsList. + * json */ /** * Sample code: ListPrivateEndpointConnections. @@ -1493,7 +1957,8 @@ public final class PrivateEndpointConnectionListByBatchAccountSamples { * @param manager Entry point to BatchManager. */ public static void listPrivateEndpointConnections(com.azure.resourcemanager.batch.BatchManager manager) { - manager.privateEndpointConnections().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, + com.azure.core.util.Context.NONE); } } ``` @@ -1504,14 +1969,15 @@ public final class PrivateEndpointConnectionListByBatchAccountSamples { import com.azure.resourcemanager.batch.fluent.models.PrivateEndpointConnectionInner; import com.azure.resourcemanager.batch.models.PrivateLinkServiceConnectionState; import com.azure.resourcemanager.batch.models.PrivateLinkServiceConnectionStatus; -import java.util.stream.Collectors; /** * Samples for PrivateEndpointConnection Update. */ public final class PrivateEndpointConnectionUpdateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionUpdate.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionUpdate. + * json */ /** * Sample code: UpdatePrivateEndpointConnection. @@ -1519,7 +1985,12 @@ public final class PrivateEndpointConnectionUpdateSamples { * @param manager Entry point to BatchManager. */ public static void updatePrivateEndpointConnection(com.azure.resourcemanager.batch.BatchManager manager) { - manager.privateEndpointConnections().update("default-azurebatch-japaneast", "sampleacct", "testprivateEndpointConnection5.24d6b4b5-e65c-4330-bbe9-3a290d62f8e0", new PrivateEndpointConnectionInner().withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState().withStatus(PrivateLinkServiceConnectionStatus.APPROVED).withDescription("Approved by xyz.abc@company.com")), null, com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().update("default-azurebatch-japaneast", "sampleacct", + "testprivateEndpointConnection5.24d6b4b5-e65c-4330-bbe9-3a290d62f8e0", + new PrivateEndpointConnectionInner().withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateLinkServiceConnectionStatus.APPROVED) + .withDescription("Approved by xyz.abc@company.com")), + null, com.azure.core.util.Context.NONE); } } ``` @@ -1532,7 +2003,8 @@ public final class PrivateEndpointConnectionUpdateSamples { */ public final class PrivateLinkResourceGetSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateLinkResourceGet.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateLinkResourceGet.json */ /** * Sample code: GetPrivateLinkResource. @@ -1540,7 +2012,8 @@ public final class PrivateLinkResourceGetSamples { * @param manager Entry point to BatchManager. */ public static void getPrivateLinkResource(com.azure.resourcemanager.batch.BatchManager manager) { - manager.privateLinkResources().getWithResponse("default-azurebatch-japaneast", "sampleacct", "batchAccount", com.azure.core.util.Context.NONE); + manager.privateLinkResources().getWithResponse("default-azurebatch-japaneast", "sampleacct", "batchAccount", + com.azure.core.util.Context.NONE); } } ``` @@ -1553,7 +2026,8 @@ public final class PrivateLinkResourceGetSamples { */ public final class PrivateLinkResourceListByBatchAccountSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateLinkResourcesList.json + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateLinkResourcesList.json */ /** * Sample code: ListPrivateLinkResource. @@ -1561,7 +2035,8 @@ public final class PrivateLinkResourceListByBatchAccountSamples { * @param manager Entry point to BatchManager. */ public static void listPrivateLinkResource(com.azure.resourcemanager.batch.BatchManager manager) { - manager.privateLinkResources().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, com.azure.core.util.Context.NONE); + manager.privateLinkResources().listByBatchAccount("default-azurebatch-japaneast", "sampleacct", null, + com.azure.core.util.Context.NONE); } } ``` diff --git a/sdk/batch/azure-resourcemanager-batch/pom.xml b/sdk/batch/azure-resourcemanager-batch/pom.xml index 1dd3584e5f069..2eda87cd4a30f 100644 --- a/sdk/batch/azure-resourcemanager-batch/pom.xml +++ b/sdk/batch/azure-resourcemanager-batch/pom.xml @@ -18,7 +18,7 @@ jar Microsoft Azure SDK for Batch Management - This package contains Microsoft Azure SDK for Batch Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Batch Client. Package tag package-2023-11. + This package contains Microsoft Azure SDK for Batch Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Batch Client. Package tag package-2024-02. https://github.com/Azure/azure-sdk-for-java @@ -88,8 +88,6 @@ 4.11.0 test
- - net.bytebuddy byte-buddy diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/BatchManager.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/BatchManager.java index 967c18cbf3cff..44e9afc66c17a 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/BatchManager.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/BatchManager.java @@ -232,7 +232,7 @@ public BatchManager authenticate(TokenCredential credential, AzureProfile profil StringBuilder userAgentBuilder = new StringBuilder(); userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.batch").append("/") - .append("1.1.0-beta.3"); + .append("1.1.0-beta.4"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolInner.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolInner.java index 859c9eb66fefe..e14b5c0d97a2d 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolInner.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolInner.java @@ -22,6 +22,7 @@ import com.azure.resourcemanager.batch.models.ScaleSettings; import com.azure.resourcemanager.batch.models.StartTask; import com.azure.resourcemanager.batch.models.TaskSchedulingPolicy; +import com.azure.resourcemanager.batch.models.UpgradePolicy; import com.azure.resourcemanager.batch.models.UserAccount; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; @@ -684,6 +685,29 @@ public NodeCommunicationMode currentNodeCommunicationMode() { return this.innerProperties() == null ? null : this.innerProperties().currentNodeCommunicationMode(); } + /** + * Get the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling. + * + * @return the upgradePolicy value. + */ + public UpgradePolicy upgradePolicy() { + return this.innerProperties() == null ? null : this.innerProperties().upgradePolicy(); + } + + /** + * Set the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling. + * + * @param upgradePolicy the upgradePolicy value to set. + * @return the PoolInner object itself. + */ + public PoolInner withUpgradePolicy(UpgradePolicy upgradePolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new PoolProperties(); + } + this.innerProperties().withUpgradePolicy(upgradePolicy); + return this; + } + /** * Get the resourceTags property: The user-defined tags to be associated with the Azure Batch Pool. When specified, * these tags are propagated to the backing Azure resources associated with the pool. This property can only be diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolProperties.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolProperties.java index 30e881d4035b8..3cf497e8b7ed4 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolProperties.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/PoolProperties.java @@ -20,6 +20,7 @@ import com.azure.resourcemanager.batch.models.ScaleSettings; import com.azure.resourcemanager.batch.models.StartTask; import com.azure.resourcemanager.batch.models.TaskSchedulingPolicy; +import com.azure.resourcemanager.batch.models.UpgradePolicy; import com.azure.resourcemanager.batch.models.UserAccount; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; @@ -256,6 +257,12 @@ public final class PoolProperties { @JsonProperty(value = "currentNodeCommunicationMode", access = JsonProperty.Access.WRITE_ONLY) private NodeCommunicationMode currentNodeCommunicationMode; + /* + * Describes an upgrade policy - automatic, manual, or rolling. + */ + @JsonProperty(value = "upgradePolicy") + private UpgradePolicy upgradePolicy; + /* * The user-defined tags to be associated with the Azure Batch Pool. When specified, these tags are propagated to * the backing Azure resources associated with the pool. This property can only be specified when the Batch account @@ -812,6 +819,26 @@ public NodeCommunicationMode currentNodeCommunicationMode() { return this.currentNodeCommunicationMode; } + /** + * Get the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling. + * + * @return the upgradePolicy value. + */ + public UpgradePolicy upgradePolicy() { + return this.upgradePolicy; + } + + /** + * Set the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling. + * + * @param upgradePolicy the upgradePolicy value to set. + * @return the PoolProperties object itself. + */ + public PoolProperties withUpgradePolicy(UpgradePolicy upgradePolicy) { + this.upgradePolicy = upgradePolicy; + return this; + } + /** * Get the resourceTags property: The user-defined tags to be associated with the Azure Batch Pool. When specified, * these tags are propagated to the backing Azure resources associated with the pool. This property can only be @@ -878,5 +905,8 @@ public void validate() { if (mountConfiguration() != null) { mountConfiguration().forEach(e -> e.validate()); } + if (upgradePolicy() != null) { + upgradePolicy().validate(); + } } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/SupportedSkuInner.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/SupportedSkuInner.java index 9f6c68567eda2..7653f77443b16 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/SupportedSkuInner.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/fluent/models/SupportedSkuInner.java @@ -7,6 +7,7 @@ import com.azure.core.annotation.Immutable; import com.azure.resourcemanager.batch.models.SkuCapability; import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; import java.util.List; /** @@ -32,6 +33,12 @@ public final class SupportedSkuInner { @JsonProperty(value = "capabilities", access = JsonProperty.Access.WRITE_ONLY) private List capabilities; + /* + * The time when Azure Batch service will retire this SKU. + */ + @JsonProperty(value = "batchSupportEndOfLife", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime batchSupportEndOfLife; + /** * Creates an instance of SupportedSkuInner class. */ @@ -65,6 +72,15 @@ public List capabilities() { return this.capabilities; } + /** + * Get the batchSupportEndOfLife property: The time when Azure Batch service will retire this SKU. + * + * @return the batchSupportEndOfLife value. + */ + public OffsetDateTime batchSupportEndOfLife() { + return this.batchSupportEndOfLife; + } + /** * Validates the instance. * diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationImpl.java index 951a5c31d7791..e12cda1fd8e16 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationImpl.java @@ -104,9 +104,9 @@ public Application apply(Context context) { ApplicationImpl(ApplicationInner innerObject, com.azure.resourcemanager.batch.BatchManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "batchAccounts"); - this.applicationName = Utils.getValueFromIdByName(innerObject.id(), "applications"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "batchAccounts"); + this.applicationName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "applications"); } public Application refresh() { diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationPackagesImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationPackagesImpl.java index 6e4f82e27ebe0..97b9f417b1122 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationPackagesImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationPackagesImpl.java @@ -88,33 +88,33 @@ public PagedIterable list(String resourceGroupName, String a String applicationName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, applicationName); - return Utils.mapPage(inner, inner1 -> new ApplicationPackageImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationPackageImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, String applicationName, Integer maxresults, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, applicationName, maxresults, context); - return Utils.mapPage(inner, inner1 -> new ApplicationPackageImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationPackageImpl(inner1, this.manager())); } public ApplicationPackage getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } - String versionName = Utils.getValueFromIdByName(id, "versions"); + String versionName = ResourceManagerUtils.getValueFromIdByName(id, "versions"); if (versionName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'versions'.", id))); @@ -124,22 +124,22 @@ public ApplicationPackage getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } - String versionName = Utils.getValueFromIdByName(id, "versions"); + String versionName = ResourceManagerUtils.getValueFromIdByName(id, "versions"); if (versionName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'versions'.", id))); @@ -148,22 +148,22 @@ public Response getByIdWithResponse(String id, Context conte } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } - String versionName = Utils.getValueFromIdByName(id, "versions"); + String versionName = ResourceManagerUtils.getValueFromIdByName(id, "versions"); if (versionName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'versions'.", id))); @@ -172,22 +172,22 @@ public void deleteById(String id) { } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } - String versionName = Utils.getValueFromIdByName(id, "versions"); + String versionName = ResourceManagerUtils.getValueFromIdByName(id, "versions"); if (versionName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'versions'.", id))); diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationsImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationsImpl.java index cdb4c6cbe2e2c..221e7244875f9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationsImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ApplicationsImpl.java @@ -59,28 +59,28 @@ public Application get(String resourceGroupName, String accountName, String appl public PagedIterable list(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, Integer maxresults, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, maxresults, context); - return Utils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); } public Application getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); @@ -89,17 +89,17 @@ public Application getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); @@ -108,17 +108,17 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); @@ -127,17 +127,17 @@ public void deleteById(String id) { } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String applicationName = Utils.getValueFromIdByName(id, "applications"); + String applicationName = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountImpl.java index 062eb68393016..2e156870c1025 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountImpl.java @@ -221,8 +221,8 @@ public BatchAccount apply(Context context) { BatchAccountImpl(BatchAccountInner innerObject, com.azure.resourcemanager.batch.BatchManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "batchAccounts"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "batchAccounts"); } public BatchAccount refresh() { diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountsImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountsImpl.java index d3a046a9e85a5..d232d54743697 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountsImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchAccountsImpl.java @@ -65,22 +65,22 @@ public BatchAccount getByResourceGroup(String resourceGroupName, String accountN public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BatchAccountImpl(inner1, this.manager())); } public Response synchronizeAutoStorageKeysWithResponse(String resourceGroupName, String accountName, @@ -137,14 +137,14 @@ public BatchAccountKeys getKeys(String resourceGroupName, String accountName) { public PagedIterable listDetectors(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().listDetectors(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new DetectorResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DetectorResponseImpl(inner1, this.manager())); } public PagedIterable listDetectors(String resourceGroupName, String accountName, Context context) { PagedIterable inner = this.serviceClient().listDetectors(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new DetectorResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DetectorResponseImpl(inner1, this.manager())); } public Response getDetectorWithResponse(String resourceGroupName, String accountName, @@ -172,23 +172,25 @@ public PagedIterable listOutboundNetworkDependencie String accountName) { PagedIterable inner = this.serviceClient().listOutboundNetworkDependenciesEndpoints(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new OutboundEnvironmentEndpointImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new OutboundEnvironmentEndpointImpl(inner1, this.manager())); } public PagedIterable listOutboundNetworkDependenciesEndpoints(String resourceGroupName, String accountName, Context context) { PagedIterable inner = this.serviceClient().listOutboundNetworkDependenciesEndpoints(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new OutboundEnvironmentEndpointImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new OutboundEnvironmentEndpointImpl(inner1, this.manager())); } public BatchAccount getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); @@ -197,12 +199,12 @@ public BatchAccount getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); @@ -211,12 +213,12 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); @@ -225,12 +227,12 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchManagementClientImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchManagementClientImpl.java index 948a6a1284da3..dc425322e44e6 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchManagementClientImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/BatchManagementClientImpl.java @@ -274,7 +274,7 @@ public PoolsClient getPools() { this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-11-01"; + this.apiVersion = "2024-02-01"; this.batchAccounts = new BatchAccountsClientImpl(this); this.applicationPackages = new ApplicationPackagesClientImpl(this); this.applications = new ApplicationsClientImpl(this); diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificateImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificateImpl.java index ee3685216e0f6..f8b5492f47af1 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificateImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificateImpl.java @@ -147,9 +147,9 @@ public Certificate apply(Context context) { CertificateImpl(CertificateInner innerObject, com.azure.resourcemanager.batch.BatchManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "batchAccounts"); - this.certificateName = Utils.getValueFromIdByName(innerObject.id(), "certificates"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "batchAccounts"); + this.certificateName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "certificates"); } public Certificate refresh() { diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificatesImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificatesImpl.java index ec7723eb233a8..f8a926a5636b7 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificatesImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/CertificatesImpl.java @@ -31,14 +31,14 @@ public CertificatesImpl(CertificatesClient innerClient, public PagedIterable listByBatchAccount(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } public PagedIterable listByBatchAccount(String resourceGroupName, String accountName, Integer maxresults, String select, String filter, Context context) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName, maxresults, select, filter, context); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } public void delete(String resourceGroupName, String accountName, String certificateName) { @@ -92,17 +92,17 @@ public Certificate cancelDeletion(String resourceGroupName, String accountName, } public Certificate getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); @@ -111,17 +111,17 @@ public Certificate getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); @@ -130,17 +130,17 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); @@ -149,17 +149,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/LocationsImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/LocationsImpl.java index 37e7331e32a2b..46d0f835da714 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/LocationsImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/LocationsImpl.java @@ -52,26 +52,26 @@ public BatchLocationQuota getQuotas(String locationName) { public PagedIterable listSupportedVirtualMachineSkus(String locationName) { PagedIterable inner = this.serviceClient().listSupportedVirtualMachineSkus(locationName); - return Utils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); } public PagedIterable listSupportedVirtualMachineSkus(String locationName, Integer maxresults, String filter, Context context) { PagedIterable inner = this.serviceClient().listSupportedVirtualMachineSkus(locationName, maxresults, filter, context); - return Utils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); } public PagedIterable listSupportedCloudServiceSkus(String locationName) { PagedIterable inner = this.serviceClient().listSupportedCloudServiceSkus(locationName); - return Utils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); } public PagedIterable listSupportedCloudServiceSkus(String locationName, Integer maxresults, String filter, Context context) { PagedIterable inner = this.serviceClient().listSupportedCloudServiceSkus(locationName, maxresults, filter, context); - return Utils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SupportedSkuImpl(inner1, this.manager())); } public Response checkNameAvailabilityWithResponse(String locationName, diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/OperationsImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/OperationsImpl.java index 8a2c1788468ce..09bc2618980af 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/OperationsImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/OperationsImpl.java @@ -26,12 +26,12 @@ public OperationsImpl(OperationsClient innerClient, com.azure.resourcemanager.ba public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolImpl.java index 2d2ac3b6cfc2c..c0e593b45e43c 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolImpl.java @@ -24,6 +24,7 @@ import com.azure.resourcemanager.batch.models.ScaleSettings; import com.azure.resourcemanager.batch.models.StartTask; import com.azure.resourcemanager.batch.models.TaskSchedulingPolicy; +import com.azure.resourcemanager.batch.models.UpgradePolicy; import com.azure.resourcemanager.batch.models.UserAccount; import java.time.OffsetDateTime; import java.util.Collections; @@ -193,6 +194,10 @@ public NodeCommunicationMode currentNodeCommunicationMode() { return this.innerModel().currentNodeCommunicationMode(); } + public UpgradePolicy upgradePolicy() { + return this.innerModel().upgradePolicy(); + } + public Map resourceTags() { Map inner = this.innerModel().resourceTags(); if (inner != null) { @@ -273,9 +278,9 @@ public Pool apply(Context context) { PoolImpl(PoolInner innerObject, com.azure.resourcemanager.batch.BatchManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "batchAccounts"); - this.poolName = Utils.getValueFromIdByName(innerObject.id(), "pools"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "batchAccounts"); + this.poolName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "pools"); } public Pool refresh() { @@ -391,6 +396,11 @@ public PoolImpl withTargetNodeCommunicationMode(NodeCommunicationMode targetNode return this; } + public PoolImpl withUpgradePolicy(UpgradePolicy upgradePolicy) { + this.innerModel().withUpgradePolicy(upgradePolicy); + return this; + } + public PoolImpl withResourceTags(Map resourceTags) { this.innerModel().withResourceTags(resourceTags); return this; diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolsImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolsImpl.java index 3a9ebbcebcca9..05352143dee82 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolsImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PoolsImpl.java @@ -31,14 +31,14 @@ public PoolsImpl(PoolsClient innerClient, com.azure.resourcemanager.batch.BatchM public PagedIterable listByBatchAccount(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new PoolImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PoolImpl(inner1, this.manager())); } public PagedIterable listByBatchAccount(String resourceGroupName, String accountName, Integer maxresults, String select, String filter, Context context) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName, maxresults, select, filter, context); - return Utils.mapPage(inner, inner1 -> new PoolImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PoolImpl(inner1, this.manager())); } public void delete(String resourceGroupName, String accountName, String poolName) { @@ -113,17 +113,17 @@ public Pool stopResize(String resourceGroupName, String accountName, String pool } public Pool getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "pools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "pools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'pools'.", id))); @@ -132,17 +132,17 @@ public Pool getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "pools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "pools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'pools'.", id))); @@ -151,17 +151,17 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "pools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "pools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'pools'.", id))); @@ -170,17 +170,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "batchAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "batchAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'batchAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "pools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "pools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'pools'.", id))); diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateEndpointConnectionsImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateEndpointConnectionsImpl.java index efc34c57e5284..d406f5a15d860 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateEndpointConnectionsImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateEndpointConnectionsImpl.java @@ -30,14 +30,14 @@ public PrivateEndpointConnectionsImpl(PrivateEndpointConnectionsClient innerClie public PagedIterable listByBatchAccount(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); } public PagedIterable listByBatchAccount(String resourceGroupName, String accountName, Integer maxresults, Context context) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName, maxresults, context); - return Utils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateLinkResourcesImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateLinkResourcesImpl.java index b310d6a4a11d4..7e2a729b812e3 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateLinkResourcesImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/PrivateLinkResourcesImpl.java @@ -30,14 +30,14 @@ public PrivateLinkResourcesImpl(PrivateLinkResourcesClient innerClient, public PagedIterable listByBatchAccount(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); } public PagedIterable listByBatchAccount(String resourceGroupName, String accountName, Integer maxresults, Context context) { PagedIterable inner = this.serviceClient().listByBatchAccount(resourceGroupName, accountName, maxresults, context); - return Utils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/Utils.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ResourceManagerUtils.java similarity index 99% rename from sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/Utils.java rename to sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ResourceManagerUtils.java index 98c1d460a912c..b12b25795b245 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/Utils.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/ResourceManagerUtils.java @@ -19,8 +19,8 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { - private Utils() { +final class ResourceManagerUtils { + private ResourceManagerUtils() { } static String getValueFromIdByName(String id, String name) { diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/SupportedSkuImpl.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/SupportedSkuImpl.java index adc5878d40ec5..bf6aa5f40b17b 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/SupportedSkuImpl.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/implementation/SupportedSkuImpl.java @@ -7,6 +7,7 @@ import com.azure.resourcemanager.batch.fluent.models.SupportedSkuInner; import com.azure.resourcemanager.batch.models.SkuCapability; import com.azure.resourcemanager.batch.models.SupportedSku; +import java.time.OffsetDateTime; import java.util.Collections; import java.util.List; @@ -37,6 +38,10 @@ public List capabilities() { } } + public OffsetDateTime batchSupportEndOfLife() { + return this.innerModel().batchSupportEndOfLife(); + } + public SupportedSkuInner innerModel() { return this.innerObject; } diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/AutomaticOSUpgradePolicy.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/AutomaticOSUpgradePolicy.java new file mode 100644 index 0000000000000..3507367bbce78 --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/AutomaticOSUpgradePolicy.java @@ -0,0 +1,147 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The configuration parameters used for performing automatic OS upgrade. + */ +@Fluent +public final class AutomaticOSUpgradePolicy { + /* + * Whether OS image rollback feature should be disabled. + */ + @JsonProperty(value = "disableAutomaticRollback") + private Boolean disableAutomaticRollback; + + /* + * Indicates whether OS upgrades should automatically be applied to scale set instances in a rolling fashion when a + * newer version of the OS image becomes available.

If this is set to true for Windows based pools, + * [WindowsConfiguration.enableAutomaticUpdates](https://learn.microsoft.com/en-us/rest/api/batchmanagement/pool/ + * create?tabs=HTTP#windowsconfiguration) + * cannot be set to true. + */ + @JsonProperty(value = "enableAutomaticOSUpgrade") + private Boolean enableAutomaticOSUpgrade; + + /* + * Indicates whether rolling upgrade policy should be used during Auto OS Upgrade. Auto OS Upgrade will fallback to + * the default policy if no policy is defined on the VMSS. + */ + @JsonProperty(value = "useRollingUpgradePolicy") + private Boolean useRollingUpgradePolicy; + + /* + * Defer OS upgrades on the TVMs if they are running tasks. + */ + @JsonProperty(value = "osRollingUpgradeDeferral") + private Boolean osRollingUpgradeDeferral; + + /** + * Creates an instance of AutomaticOSUpgradePolicy class. + */ + public AutomaticOSUpgradePolicy() { + } + + /** + * Get the disableAutomaticRollback property: Whether OS image rollback feature should be disabled. + * + * @return the disableAutomaticRollback value. + */ + public Boolean disableAutomaticRollback() { + return this.disableAutomaticRollback; + } + + /** + * Set the disableAutomaticRollback property: Whether OS image rollback feature should be disabled. + * + * @param disableAutomaticRollback the disableAutomaticRollback value to set. + * @return the AutomaticOSUpgradePolicy object itself. + */ + public AutomaticOSUpgradePolicy withDisableAutomaticRollback(Boolean disableAutomaticRollback) { + this.disableAutomaticRollback = disableAutomaticRollback; + return this; + } + + /** + * Get the enableAutomaticOSUpgrade property: Indicates whether OS upgrades should automatically be applied to + * scale set instances in a rolling fashion when a newer version of the OS image becomes available. <br + * /><br /> If this is set to true for Windows based pools, + * [WindowsConfiguration.enableAutomaticUpdates](https://learn.microsoft.com/en-us/rest/api/batchmanagement/pool/create?tabs=HTTP#windowsconfiguration) + * cannot be set to true. + * + * @return the enableAutomaticOSUpgrade value. + */ + public Boolean enableAutomaticOSUpgrade() { + return this.enableAutomaticOSUpgrade; + } + + /** + * Set the enableAutomaticOSUpgrade property: Indicates whether OS upgrades should automatically be applied to + * scale set instances in a rolling fashion when a newer version of the OS image becomes available. <br + * /><br /> If this is set to true for Windows based pools, + * [WindowsConfiguration.enableAutomaticUpdates](https://learn.microsoft.com/en-us/rest/api/batchmanagement/pool/create?tabs=HTTP#windowsconfiguration) + * cannot be set to true. + * + * @param enableAutomaticOSUpgrade the enableAutomaticOSUpgrade value to set. + * @return the AutomaticOSUpgradePolicy object itself. + */ + public AutomaticOSUpgradePolicy withEnableAutomaticOSUpgrade(Boolean enableAutomaticOSUpgrade) { + this.enableAutomaticOSUpgrade = enableAutomaticOSUpgrade; + return this; + } + + /** + * Get the useRollingUpgradePolicy property: Indicates whether rolling upgrade policy should be used during Auto OS + * Upgrade. Auto OS Upgrade will fallback to the default policy if no policy is defined on the VMSS. + * + * @return the useRollingUpgradePolicy value. + */ + public Boolean useRollingUpgradePolicy() { + return this.useRollingUpgradePolicy; + } + + /** + * Set the useRollingUpgradePolicy property: Indicates whether rolling upgrade policy should be used during Auto OS + * Upgrade. Auto OS Upgrade will fallback to the default policy if no policy is defined on the VMSS. + * + * @param useRollingUpgradePolicy the useRollingUpgradePolicy value to set. + * @return the AutomaticOSUpgradePolicy object itself. + */ + public AutomaticOSUpgradePolicy withUseRollingUpgradePolicy(Boolean useRollingUpgradePolicy) { + this.useRollingUpgradePolicy = useRollingUpgradePolicy; + return this; + } + + /** + * Get the osRollingUpgradeDeferral property: Defer OS upgrades on the TVMs if they are running tasks. + * + * @return the osRollingUpgradeDeferral value. + */ + public Boolean osRollingUpgradeDeferral() { + return this.osRollingUpgradeDeferral; + } + + /** + * Set the osRollingUpgradeDeferral property: Defer OS upgrades on the TVMs if they are running tasks. + * + * @param osRollingUpgradeDeferral the osRollingUpgradeDeferral value to set. + * @return the AutomaticOSUpgradePolicy object itself. + */ + public AutomaticOSUpgradePolicy withOsRollingUpgradeDeferral(Boolean osRollingUpgradeDeferral) { + this.osRollingUpgradeDeferral = osRollingUpgradeDeferral; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/Pool.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/Pool.java index a8b5f852c8f69..8493f4c442bb0 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/Pool.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/Pool.java @@ -304,6 +304,13 @@ public interface Pool { */ NodeCommunicationMode currentNodeCommunicationMode(); + /** + * Gets the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling. + * + * @return the upgradePolicy value. + */ + UpgradePolicy upgradePolicy(); + /** * Gets the resourceTags property: The user-defined tags to be associated with the Azure Batch Pool. When specified, * these tags are propagated to the backing Azure resources associated with the pool. This property can only be @@ -369,8 +376,8 @@ interface WithCreate extends DefinitionStages.WithIdentity, DefinitionStages.Wit DefinitionStages.WithTaskSchedulingPolicy, DefinitionStages.WithUserAccounts, DefinitionStages.WithMetadata, DefinitionStages.WithStartTask, DefinitionStages.WithCertificates, DefinitionStages.WithApplicationPackages, DefinitionStages.WithApplicationLicenses, DefinitionStages.WithMountConfiguration, - DefinitionStages.WithTargetNodeCommunicationMode, DefinitionStages.WithResourceTags, - DefinitionStages.WithIfMatch, DefinitionStages.WithIfNoneMatch { + DefinitionStages.WithTargetNodeCommunicationMode, DefinitionStages.WithUpgradePolicy, + DefinitionStages.WithResourceTags, DefinitionStages.WithIfMatch, DefinitionStages.WithIfNoneMatch { /** * Executes the create request. * @@ -733,6 +740,19 @@ interface WithTargetNodeCommunicationMode { WithCreate withTargetNodeCommunicationMode(NodeCommunicationMode targetNodeCommunicationMode); } + /** + * The stage of the Pool definition allowing to specify upgradePolicy. + */ + interface WithUpgradePolicy { + /** + * Specifies the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling.. + * + * @param upgradePolicy Describes an upgrade policy - automatic, manual, or rolling. + * @return the next definition stage. + */ + WithCreate withUpgradePolicy(UpgradePolicy upgradePolicy); + } + /** * The stage of the Pool definition allowing to specify resourceTags. */ @@ -800,7 +820,8 @@ interface Update extends UpdateStages.WithIdentity, UpdateStages.WithDisplayName UpdateStages.WithTaskSlotsPerNode, UpdateStages.WithTaskSchedulingPolicy, UpdateStages.WithUserAccounts, UpdateStages.WithMetadata, UpdateStages.WithStartTask, UpdateStages.WithCertificates, UpdateStages.WithApplicationPackages, UpdateStages.WithApplicationLicenses, UpdateStages.WithMountConfiguration, - UpdateStages.WithTargetNodeCommunicationMode, UpdateStages.WithResourceTags, UpdateStages.WithIfMatch { + UpdateStages.WithTargetNodeCommunicationMode, UpdateStages.WithUpgradePolicy, UpdateStages.WithResourceTags, + UpdateStages.WithIfMatch { /** * Executes the update request. * @@ -1167,6 +1188,19 @@ interface WithTargetNodeCommunicationMode { Update withTargetNodeCommunicationMode(NodeCommunicationMode targetNodeCommunicationMode); } + /** + * The stage of the Pool update allowing to specify upgradePolicy. + */ + interface WithUpgradePolicy { + /** + * Specifies the upgradePolicy property: Describes an upgrade policy - automatic, manual, or rolling.. + * + * @param upgradePolicy Describes an upgrade policy - automatic, manual, or rolling. + * @return the next definition stage. + */ + Update withUpgradePolicy(UpgradePolicy upgradePolicy); + } + /** * The stage of the Pool update allowing to specify resourceTags. */ diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/RollingUpgradePolicy.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/RollingUpgradePolicy.java new file mode 100644 index 0000000000000..67fb8e8650b3c --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/RollingUpgradePolicy.java @@ -0,0 +1,267 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The configuration parameters used while performing a rolling upgrade. + */ +@Fluent +public final class RollingUpgradePolicy { + /* + * Allow VMSS to ignore AZ boundaries when constructing upgrade batches. Take into consideration the Update Domain + * and maxBatchInstancePercent to determine the batch size. If this field is not set, Azure Azure Batch will not + * set its default value. The value of enableCrossZoneUpgrade on the created VirtualMachineScaleSet will be decided + * by the default configurations on VirtualMachineScaleSet. This field is able to be set to true or false only when + * using NodePlacementConfiguration as Zonal. + */ + @JsonProperty(value = "enableCrossZoneUpgrade") + private Boolean enableCrossZoneUpgrade; + + /* + * The maximum percent of total virtual machine instances that will be upgraded simultaneously by the rolling + * upgrade in one batch. As this is a maximum, unhealthy instances in previous or future batches can cause the + * percentage of instances in a batch to decrease to ensure higher reliability. The value of this field should be + * between 5 and 100, inclusive. If both maxBatchInstancePercent and maxUnhealthyInstancePercent are assigned with + * value, the value of maxBatchInstancePercent should not be more than maxUnhealthyInstancePercent. + */ + @JsonProperty(value = "maxBatchInstancePercent") + private Integer maxBatchInstancePercent; + + /* + * The maximum percentage of the total virtual machine instances in the scale set that can be simultaneously + * unhealthy, either as a result of being upgraded, or by being found in an unhealthy state by the virtual machine + * health checks before the rolling upgrade aborts. This constraint will be checked prior to starting any batch. + * The value of this field should be between 5 and 100, inclusive. If both maxBatchInstancePercent and + * maxUnhealthyInstancePercent are assigned with value, the value of maxBatchInstancePercent should not be more + * than maxUnhealthyInstancePercent. + */ + @JsonProperty(value = "maxUnhealthyInstancePercent") + private Integer maxUnhealthyInstancePercent; + + /* + * The maximum percentage of upgraded virtual machine instances that can be found to be in an unhealthy state. This + * check will happen after each batch is upgraded. If this percentage is ever exceeded, the rolling update aborts. + * The value of this field should be between 0 and 100, inclusive. + */ + @JsonProperty(value = "maxUnhealthyUpgradedInstancePercent") + private Integer maxUnhealthyUpgradedInstancePercent; + + /* + * The wait time between completing the update for all virtual machines in one batch and starting the next batch. + * The time duration should be specified in ISO 8601 format. + */ + @JsonProperty(value = "pauseTimeBetweenBatches") + private String pauseTimeBetweenBatches; + + /* + * Upgrade all unhealthy instances in a scale set before any healthy instances. + */ + @JsonProperty(value = "prioritizeUnhealthyInstances") + private Boolean prioritizeUnhealthyInstances; + + /* + * Rollback failed instances to previous model if the Rolling Upgrade policy is violated. + */ + @JsonProperty(value = "rollbackFailedInstancesOnPolicyBreach") + private Boolean rollbackFailedInstancesOnPolicyBreach; + + /** + * Creates an instance of RollingUpgradePolicy class. + */ + public RollingUpgradePolicy() { + } + + /** + * Get the enableCrossZoneUpgrade property: Allow VMSS to ignore AZ boundaries when constructing upgrade batches. + * Take into consideration the Update Domain and maxBatchInstancePercent to determine the batch size. If this field + * is not set, Azure Azure Batch will not set its default value. The value of enableCrossZoneUpgrade on the created + * VirtualMachineScaleSet will be decided by the default configurations on VirtualMachineScaleSet. This field is + * able to be set to true or false only when using NodePlacementConfiguration as Zonal. + * + * @return the enableCrossZoneUpgrade value. + */ + public Boolean enableCrossZoneUpgrade() { + return this.enableCrossZoneUpgrade; + } + + /** + * Set the enableCrossZoneUpgrade property: Allow VMSS to ignore AZ boundaries when constructing upgrade batches. + * Take into consideration the Update Domain and maxBatchInstancePercent to determine the batch size. If this field + * is not set, Azure Azure Batch will not set its default value. The value of enableCrossZoneUpgrade on the created + * VirtualMachineScaleSet will be decided by the default configurations on VirtualMachineScaleSet. This field is + * able to be set to true or false only when using NodePlacementConfiguration as Zonal. + * + * @param enableCrossZoneUpgrade the enableCrossZoneUpgrade value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy withEnableCrossZoneUpgrade(Boolean enableCrossZoneUpgrade) { + this.enableCrossZoneUpgrade = enableCrossZoneUpgrade; + return this; + } + + /** + * Get the maxBatchInstancePercent property: The maximum percent of total virtual machine instances that will be + * upgraded simultaneously by the rolling upgrade in one batch. As this is a maximum, unhealthy instances in + * previous or future batches can cause the percentage of instances in a batch to decrease to ensure higher + * reliability. The value of this field should be between 5 and 100, inclusive. If both maxBatchInstancePercent and + * maxUnhealthyInstancePercent are assigned with value, the value of maxBatchInstancePercent should not be more + * than maxUnhealthyInstancePercent. + * + * @return the maxBatchInstancePercent value. + */ + public Integer maxBatchInstancePercent() { + return this.maxBatchInstancePercent; + } + + /** + * Set the maxBatchInstancePercent property: The maximum percent of total virtual machine instances that will be + * upgraded simultaneously by the rolling upgrade in one batch. As this is a maximum, unhealthy instances in + * previous or future batches can cause the percentage of instances in a batch to decrease to ensure higher + * reliability. The value of this field should be between 5 and 100, inclusive. If both maxBatchInstancePercent and + * maxUnhealthyInstancePercent are assigned with value, the value of maxBatchInstancePercent should not be more + * than maxUnhealthyInstancePercent. + * + * @param maxBatchInstancePercent the maxBatchInstancePercent value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy withMaxBatchInstancePercent(Integer maxBatchInstancePercent) { + this.maxBatchInstancePercent = maxBatchInstancePercent; + return this; + } + + /** + * Get the maxUnhealthyInstancePercent property: The maximum percentage of the total virtual machine instances in + * the scale set that can be simultaneously unhealthy, either as a result of being upgraded, or by being found in + * an unhealthy state by the virtual machine health checks before the rolling upgrade aborts. This constraint will + * be checked prior to starting any batch. The value of this field should be between 5 and 100, inclusive. If both + * maxBatchInstancePercent and maxUnhealthyInstancePercent are assigned with value, the value of + * maxBatchInstancePercent should not be more than maxUnhealthyInstancePercent. + * + * @return the maxUnhealthyInstancePercent value. + */ + public Integer maxUnhealthyInstancePercent() { + return this.maxUnhealthyInstancePercent; + } + + /** + * Set the maxUnhealthyInstancePercent property: The maximum percentage of the total virtual machine instances in + * the scale set that can be simultaneously unhealthy, either as a result of being upgraded, or by being found in + * an unhealthy state by the virtual machine health checks before the rolling upgrade aborts. This constraint will + * be checked prior to starting any batch. The value of this field should be between 5 and 100, inclusive. If both + * maxBatchInstancePercent and maxUnhealthyInstancePercent are assigned with value, the value of + * maxBatchInstancePercent should not be more than maxUnhealthyInstancePercent. + * + * @param maxUnhealthyInstancePercent the maxUnhealthyInstancePercent value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy withMaxUnhealthyInstancePercent(Integer maxUnhealthyInstancePercent) { + this.maxUnhealthyInstancePercent = maxUnhealthyInstancePercent; + return this; + } + + /** + * Get the maxUnhealthyUpgradedInstancePercent property: The maximum percentage of upgraded virtual machine + * instances that can be found to be in an unhealthy state. This check will happen after each batch is upgraded. If + * this percentage is ever exceeded, the rolling update aborts. The value of this field should be between 0 and + * 100, inclusive. + * + * @return the maxUnhealthyUpgradedInstancePercent value. + */ + public Integer maxUnhealthyUpgradedInstancePercent() { + return this.maxUnhealthyUpgradedInstancePercent; + } + + /** + * Set the maxUnhealthyUpgradedInstancePercent property: The maximum percentage of upgraded virtual machine + * instances that can be found to be in an unhealthy state. This check will happen after each batch is upgraded. If + * this percentage is ever exceeded, the rolling update aborts. The value of this field should be between 0 and + * 100, inclusive. + * + * @param maxUnhealthyUpgradedInstancePercent the maxUnhealthyUpgradedInstancePercent value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy withMaxUnhealthyUpgradedInstancePercent(Integer maxUnhealthyUpgradedInstancePercent) { + this.maxUnhealthyUpgradedInstancePercent = maxUnhealthyUpgradedInstancePercent; + return this; + } + + /** + * Get the pauseTimeBetweenBatches property: The wait time between completing the update for all virtual machines + * in one batch and starting the next batch. The time duration should be specified in ISO 8601 format. + * + * @return the pauseTimeBetweenBatches value. + */ + public String pauseTimeBetweenBatches() { + return this.pauseTimeBetweenBatches; + } + + /** + * Set the pauseTimeBetweenBatches property: The wait time between completing the update for all virtual machines + * in one batch and starting the next batch. The time duration should be specified in ISO 8601 format. + * + * @param pauseTimeBetweenBatches the pauseTimeBetweenBatches value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy withPauseTimeBetweenBatches(String pauseTimeBetweenBatches) { + this.pauseTimeBetweenBatches = pauseTimeBetweenBatches; + return this; + } + + /** + * Get the prioritizeUnhealthyInstances property: Upgrade all unhealthy instances in a scale set before any healthy + * instances. + * + * @return the prioritizeUnhealthyInstances value. + */ + public Boolean prioritizeUnhealthyInstances() { + return this.prioritizeUnhealthyInstances; + } + + /** + * Set the prioritizeUnhealthyInstances property: Upgrade all unhealthy instances in a scale set before any healthy + * instances. + * + * @param prioritizeUnhealthyInstances the prioritizeUnhealthyInstances value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy withPrioritizeUnhealthyInstances(Boolean prioritizeUnhealthyInstances) { + this.prioritizeUnhealthyInstances = prioritizeUnhealthyInstances; + return this; + } + + /** + * Get the rollbackFailedInstancesOnPolicyBreach property: Rollback failed instances to previous model if the + * Rolling Upgrade policy is violated. + * + * @return the rollbackFailedInstancesOnPolicyBreach value. + */ + public Boolean rollbackFailedInstancesOnPolicyBreach() { + return this.rollbackFailedInstancesOnPolicyBreach; + } + + /** + * Set the rollbackFailedInstancesOnPolicyBreach property: Rollback failed instances to previous model if the + * Rolling Upgrade policy is violated. + * + * @param rollbackFailedInstancesOnPolicyBreach the rollbackFailedInstancesOnPolicyBreach value to set. + * @return the RollingUpgradePolicy object itself. + */ + public RollingUpgradePolicy + withRollbackFailedInstancesOnPolicyBreach(Boolean rollbackFailedInstancesOnPolicyBreach) { + this.rollbackFailedInstancesOnPolicyBreach = rollbackFailedInstancesOnPolicyBreach; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/SupportedSku.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/SupportedSku.java index d314a153bce16..72afbfb41fde3 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/SupportedSku.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/SupportedSku.java @@ -5,6 +5,7 @@ package com.azure.resourcemanager.batch.models; import com.azure.resourcemanager.batch.fluent.models.SupportedSkuInner; +import java.time.OffsetDateTime; import java.util.List; /** @@ -32,6 +33,13 @@ public interface SupportedSku { */ List capabilities(); + /** + * Gets the batchSupportEndOfLife property: The time when Azure Batch service will retire this SKU. + * + * @return the batchSupportEndOfLife value. + */ + OffsetDateTime batchSupportEndOfLife(); + /** * Gets the inner com.azure.resourcemanager.batch.fluent.models.SupportedSkuInner object. * diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradeMode.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradeMode.java new file mode 100644 index 0000000000000..532e215d80418 --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradeMode.java @@ -0,0 +1,70 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.models; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; + +/** + * Specifies the mode of an upgrade to virtual machines in the scale set.<br /><br /> Possible values + * are:<br /><br /> **Manual** - You control the application of updates to virtual machines in the scale + * set. You do this by using the manualUpgrade action.<br /><br /> **Automatic** - All virtual machines in + * the scale set are automatically updated at the same time.<br /><br /> **Rolling** - Scale set performs + * updates in batches with an optional pause time in between. + */ +public enum UpgradeMode { + /** + * Enum value automatic. + */ + AUTOMATIC("automatic"), + + /** + * Enum value manual. + */ + MANUAL("manual"), + + /** + * Enum value rolling. + */ + ROLLING("rolling"); + + /** + * The actual serialized value for a UpgradeMode instance. + */ + private final String value; + + UpgradeMode(String value) { + this.value = value; + } + + /** + * Parses a serialized value to a UpgradeMode instance. + * + * @param value the serialized value to parse. + * @return the parsed UpgradeMode object, or null if unable to parse. + */ + @JsonCreator + public static UpgradeMode fromString(String value) { + if (value == null) { + return null; + } + UpgradeMode[] items = UpgradeMode.values(); + for (UpgradeMode item : items) { + if (item.toString().equalsIgnoreCase(value)) { + return item; + } + } + return null; + } + + /** + * {@inheritDoc} + */ + @JsonValue + @Override + public String toString() { + return this.value; + } +} diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradePolicy.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradePolicy.java new file mode 100644 index 0000000000000..bd5bbf2ba9a0d --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/com/azure/resourcemanager/batch/models/UpgradePolicy.java @@ -0,0 +1,135 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Describes an upgrade policy - automatic, manual, or rolling. + */ +@Fluent +public final class UpgradePolicy { + /* + * Specifies the mode of an upgrade to virtual machines in the scale set.

Possible values are:

**Manual** - You control the application of updates to virtual machines in the scale set. You do this by + * using the manualUpgrade action.

**Automatic** - All virtual machines in the scale set are + * automatically updated at the same time.

**Rolling** - Scale set performs updates in batches with an + * optional pause time in between. + */ + @JsonProperty(value = "mode", required = true) + private UpgradeMode mode; + + /* + * The configuration parameters used for performing automatic OS upgrade. + */ + @JsonProperty(value = "automaticOSUpgradePolicy") + private AutomaticOSUpgradePolicy automaticOSUpgradePolicy; + + /* + * This property is only supported on Pools with the virtualMachineConfiguration property. + */ + @JsonProperty(value = "rollingUpgradePolicy") + private RollingUpgradePolicy rollingUpgradePolicy; + + /** + * Creates an instance of UpgradePolicy class. + */ + public UpgradePolicy() { + } + + /** + * Get the mode property: Specifies the mode of an upgrade to virtual machines in the scale set.<br /><br + * /> Possible values are:<br /><br /> **Manual** - You control the application of updates to + * virtual machines in the scale set. You do this by using the manualUpgrade action.<br /><br /> + * **Automatic** - All virtual machines in the scale set are automatically updated at the same time.<br + * /><br /> **Rolling** - Scale set performs updates in batches with an optional pause time in between. + * + * @return the mode value. + */ + public UpgradeMode mode() { + return this.mode; + } + + /** + * Set the mode property: Specifies the mode of an upgrade to virtual machines in the scale set.<br /><br + * /> Possible values are:<br /><br /> **Manual** - You control the application of updates to + * virtual machines in the scale set. You do this by using the manualUpgrade action.<br /><br /> + * **Automatic** - All virtual machines in the scale set are automatically updated at the same time.<br + * /><br /> **Rolling** - Scale set performs updates in batches with an optional pause time in between. + * + * @param mode the mode value to set. + * @return the UpgradePolicy object itself. + */ + public UpgradePolicy withMode(UpgradeMode mode) { + this.mode = mode; + return this; + } + + /** + * Get the automaticOSUpgradePolicy property: The configuration parameters used for performing automatic OS + * upgrade. + * + * @return the automaticOSUpgradePolicy value. + */ + public AutomaticOSUpgradePolicy automaticOSUpgradePolicy() { + return this.automaticOSUpgradePolicy; + } + + /** + * Set the automaticOSUpgradePolicy property: The configuration parameters used for performing automatic OS + * upgrade. + * + * @param automaticOSUpgradePolicy the automaticOSUpgradePolicy value to set. + * @return the UpgradePolicy object itself. + */ + public UpgradePolicy withAutomaticOSUpgradePolicy(AutomaticOSUpgradePolicy automaticOSUpgradePolicy) { + this.automaticOSUpgradePolicy = automaticOSUpgradePolicy; + return this; + } + + /** + * Get the rollingUpgradePolicy property: This property is only supported on Pools with the + * virtualMachineConfiguration property. + * + * @return the rollingUpgradePolicy value. + */ + public RollingUpgradePolicy rollingUpgradePolicy() { + return this.rollingUpgradePolicy; + } + + /** + * Set the rollingUpgradePolicy property: This property is only supported on Pools with the + * virtualMachineConfiguration property. + * + * @param rollingUpgradePolicy the rollingUpgradePolicy value to set. + * @return the UpgradePolicy object itself. + */ + public UpgradePolicy withRollingUpgradePolicy(RollingUpgradePolicy rollingUpgradePolicy) { + this.rollingUpgradePolicy = rollingUpgradePolicy; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (mode() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property mode in model UpgradePolicy")); + } + if (automaticOSUpgradePolicy() != null) { + automaticOSUpgradePolicy().validate(); + } + if (rollingUpgradePolicy() != null) { + rollingUpgradePolicy().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UpgradePolicy.class); +} diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/java/module-info.java b/sdk/batch/azure-resourcemanager-batch/src/main/java/module-info.java index 57ef56c592101..a9a05594156ef 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/java/module-info.java +++ b/sdk/batch/azure-resourcemanager-batch/src/main/java/module-info.java @@ -4,12 +4,10 @@ module com.azure.resourcemanager.batch { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.batch; exports com.azure.resourcemanager.batch.fluent; exports com.azure.resourcemanager.batch.fluent.models; exports com.azure.resourcemanager.batch.models; - opens com.azure.resourcemanager.batch.fluent.models to com.azure.core, com.fasterxml.jackson.databind; opens com.azure.resourcemanager.batch.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/batch/azure-resourcemanager-batch/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-batch/reflect-config.json b/sdk/batch/azure-resourcemanager-batch/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-batch/reflect-config.json index 89db28166538e..04f6c9d0b5cb2 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-batch/reflect-config.json +++ b/sdk/batch/azure-resourcemanager-batch/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-batch/reflect-config.json @@ -523,6 +523,21 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.batch.models.UpgradePolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.batch.models.AutomaticOSUpgradePolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.batch.models.RollingUpgradePolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.batch.models.BatchPoolIdentity", "allDeclaredConstructors" : true, @@ -773,6 +788,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.batch.models.UpgradeMode", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.batch.models.PoolIdentityType", "allDeclaredConstructors" : true, diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationCreateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationCreateSamples.java index 32d4cb553d544..664c1eab17a2e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationCreateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationCreateSamples.java @@ -10,7 +10,7 @@ public final class ApplicationCreateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationCreate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationCreate.json */ /** * Sample code: ApplicationCreate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationDeleteSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationDeleteSamples.java index 4e40b0984e073..29ccc1bfadc93 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationDeleteSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationDeleteSamples.java @@ -10,7 +10,7 @@ public final class ApplicationDeleteSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationDelete.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationDelete.json */ /** * Sample code: ApplicationDelete. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationGetSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationGetSamples.java index 037cd91b495dc..6374eea493e08 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationGetSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationGetSamples.java @@ -10,7 +10,7 @@ public final class ApplicationGetSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationGet.json */ /** * Sample code: ApplicationGet. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationListSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationListSamples.java index 02826c696b5b1..2d2bbecb46d08 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationListSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationListSamples.java @@ -10,7 +10,7 @@ public final class ApplicationListSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationList.json */ /** * Sample code: ApplicationList. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageActivateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageActivateSamples.java index db320c277477b..66b0b6809384d 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageActivateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageActivateSamples.java @@ -12,7 +12,7 @@ public final class ApplicationPackageActivateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageActivate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageActivate.json */ /** * Sample code: ApplicationPackageActivate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageCreateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageCreateSamples.java index 65b51fb9d8823..70f76a679edfb 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageCreateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageCreateSamples.java @@ -10,7 +10,7 @@ public final class ApplicationPackageCreateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageCreate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageCreate.json */ /** * Sample code: ApplicationPackageCreate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageDeleteSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageDeleteSamples.java index a0cf15f093482..c3e355906d9a3 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageDeleteSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageDeleteSamples.java @@ -10,7 +10,7 @@ public final class ApplicationPackageDeleteSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageDelete.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageDelete.json */ /** * Sample code: ApplicationPackageDelete. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageGetSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageGetSamples.java index 759f90fe9950c..95f765f3860f2 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageGetSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageGetSamples.java @@ -10,7 +10,7 @@ public final class ApplicationPackageGetSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageGet.json */ /** * Sample code: ApplicationPackageGet. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageListSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageListSamples.java index 4d308b4a4618c..65a01341b69b0 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageListSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationPackageListSamples.java @@ -10,7 +10,7 @@ public final class ApplicationPackageListSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationPackageList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationPackageList.json */ /** * Sample code: ApplicationPackageList. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationUpdateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationUpdateSamples.java index 7154dca608cbb..8d6385166d4c2 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationUpdateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/ApplicationUpdateSamples.java @@ -12,7 +12,7 @@ public final class ApplicationUpdateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ApplicationUpdate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ApplicationUpdate.json */ /** * Sample code: ApplicationUpdate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountCreateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountCreateSamples.java index 4edc77637ae83..d11dcfd21f8d9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountCreateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountCreateSamples.java @@ -20,7 +20,7 @@ public final class BatchAccountCreateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountCreate_BYOS.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountCreate_BYOS.json */ /** * Sample code: BatchAccountCreate_BYOS. @@ -40,7 +40,7 @@ public static void batchAccountCreateBYOS(com.azure.resourcemanager.batch.BatchM } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * BatchAccountCreate_UserAssignedIdentity.json */ /** @@ -62,7 +62,7 @@ public static void batchAccountCreateUserAssignedIdentity(com.azure.resourcemana /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateBatchAccountCreate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateBatchAccountCreate.json */ /** * Sample code: PrivateBatchAccountCreate. @@ -81,7 +81,7 @@ public static void privateBatchAccountCreate(com.azure.resourcemanager.batch.Bat } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * BatchAccountCreate_SystemAssignedIdentity.json */ /** @@ -100,7 +100,7 @@ public static void batchAccountCreateSystemAssignedIdentity(com.azure.resourcema /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountCreate_Default.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountCreate_Default.json */ /** * Sample code: BatchAccountCreate_Default. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountDeleteSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountDeleteSamples.java index c03337039a0b1..efe94f257451c 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountDeleteSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountDeleteSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountDeleteSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountDelete.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountDelete.json */ /** * Sample code: BatchAccountDelete. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetByResourceGroupSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetByResourceGroupSamples.java index c4e4d49380a1e..92b1d3620a4b9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetByResourceGroupSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountGetByResourceGroupSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateBatchAccountGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateBatchAccountGet.json */ /** * Sample code: PrivateBatchAccountGet. @@ -24,7 +24,7 @@ public static void privateBatchAccountGet(com.azure.resourcemanager.batch.BatchM /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountGet.json */ /** * Sample code: BatchAccountGet. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetDetectorSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetDetectorSamples.java index 4bd561ccac764..c8b6f13e7001e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetDetectorSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetDetectorSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountGetDetectorSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/DetectorGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/DetectorGet.json */ /** * Sample code: GetDetector. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetKeysSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetKeysSamples.java index 58e949c158395..6d68196dfec71 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetKeysSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountGetKeysSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountGetKeysSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountGetKeys.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountGetKeys.json */ /** * Sample code: BatchAccountGetKeys. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListByResourceGroupSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListByResourceGroupSamples.java index 2347a71c21010..1a537cdecb7dc 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListByResourceGroupSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountListByResourceGroupSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountListByResourceGroup. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountListByResourceGroup. * json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListDetectorsSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListDetectorsSamples.java index 157dda2a3834f..1489167ee81ff 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListDetectorsSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListDetectorsSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountListDetectorsSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/DetectorList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/DetectorList.json */ /** * Sample code: ListDetectors. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListOutboundNetworkDependenciesEndpointsSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListOutboundNetworkDependenciesEndpointsSamples.java index 184d6d76ef5b6..2402bed749ef9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListOutboundNetworkDependenciesEndpointsSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListOutboundNetworkDependenciesEndpointsSamples.java @@ -9,7 +9,7 @@ */ public final class BatchAccountListOutboundNetworkDependenciesEndpointsSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * BatchAccountListOutboundNetworkDependenciesEndpoints.json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListSamples.java index 7ac48545b3bc7..ff1e81fdb79e6 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountListSamples.java @@ -10,7 +10,7 @@ public final class BatchAccountListSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountList.json */ /** * Sample code: BatchAccountList. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountRegenerateKeySamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountRegenerateKeySamples.java index 38c7472ce3e0d..f80e9ba98f752 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountRegenerateKeySamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountRegenerateKeySamples.java @@ -13,7 +13,7 @@ public final class BatchAccountRegenerateKeySamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountRegenerateKey.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountRegenerateKey.json */ /** * Sample code: BatchAccountRegenerateKey. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountSynchronizeAutoStorageKeysSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountSynchronizeAutoStorageKeysSamples.java index a6b07fb5e752b..94ab3d4872047 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountSynchronizeAutoStorageKeysSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountSynchronizeAutoStorageKeysSamples.java @@ -9,7 +9,7 @@ */ public final class BatchAccountSynchronizeAutoStorageKeysSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * BatchAccountSynchronizeAutoStorageKeys.json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountUpdateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountUpdateSamples.java index ee82432785c2a..05091a9dd807e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountUpdateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/BatchAccountUpdateSamples.java @@ -13,7 +13,7 @@ public final class BatchAccountUpdateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/BatchAccountUpdate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/BatchAccountUpdate.json */ /** * Sample code: BatchAccountUpdate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCancelDeletionSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCancelDeletionSamples.java index bff20c21ce095..c39302a13b174 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCancelDeletionSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCancelDeletionSamples.java @@ -10,7 +10,7 @@ public final class CertificateCancelDeletionSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCancelDeletion.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCancelDeletion.json */ /** * Sample code: CertificateCancelDeletion. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCreateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCreateSamples.java index edcddf4cde1eb..04096518233d4 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCreateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateCreateSamples.java @@ -12,7 +12,7 @@ public final class CertificateCreateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCreate_Full.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCreate_Full.json */ /** * Sample code: CreateCertificate - Full. @@ -28,7 +28,7 @@ public static void createCertificateFull(com.azure.resourcemanager.batch.BatchMa /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCreate_Minimal.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCreate_Minimal.json */ /** * Sample code: CreateCertificate - Minimal Pfx. @@ -43,7 +43,7 @@ public static void createCertificateMinimalPfx(com.azure.resourcemanager.batch.B /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateCreate_MinimalCer.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateCreate_MinimalCer.json */ /** * Sample code: CreateCertificate - Minimal Cer. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateDeleteSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateDeleteSamples.java index d95f3e81dc7c3..3006d3455f205 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateDeleteSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateDeleteSamples.java @@ -10,7 +10,7 @@ public final class CertificateDeleteSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateDelete.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateDelete.json */ /** * Sample code: CertificateDelete. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateGetSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateGetSamples.java index c24c4986028dd..12348be264276 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateGetSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateGetSamples.java @@ -10,7 +10,7 @@ public final class CertificateGetSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateGetWithDeletionError. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateGetWithDeletionError. * json */ /** @@ -25,7 +25,7 @@ public static void getCertificateWithDeletionError(com.azure.resourcemanager.bat /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateGet.json */ /** * Sample code: Get Certificate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateListByBatchAccountSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateListByBatchAccountSamples.java index 2a52015070fbc..21d607d80986b 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateListByBatchAccountSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateListByBatchAccountSamples.java @@ -10,7 +10,7 @@ public final class CertificateListByBatchAccountSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateListWithFilter.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateListWithFilter.json */ /** * Sample code: ListCertificates - Filter and Select. @@ -26,7 +26,7 @@ public static void listCertificatesFilterAndSelect(com.azure.resourcemanager.bat /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateList.json */ /** * Sample code: ListCertificates. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateUpdateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateUpdateSamples.java index ca46276aebcb7..0420c88a51094 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateUpdateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/CertificateUpdateSamples.java @@ -12,7 +12,7 @@ public final class CertificateUpdateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/CertificateUpdate.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/CertificateUpdate.json */ /** * Sample code: UpdateCertificate. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationCheckNameAvailabilitySamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationCheckNameAvailabilitySamples.java index f3d6c868b58d1..2bac07af7685e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationCheckNameAvailabilitySamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationCheckNameAvailabilitySamples.java @@ -11,7 +11,7 @@ */ public final class LocationCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * LocationCheckNameAvailability_AlreadyExists.json */ /** @@ -26,7 +26,7 @@ public final class LocationCheckNameAvailabilitySamples { } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * LocationCheckNameAvailability_Available.json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationGetQuotasSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationGetQuotasSamples.java index 56d6747383a4a..36b3063c9a6b9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationGetQuotasSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationGetQuotasSamples.java @@ -10,7 +10,7 @@ public final class LocationGetQuotasSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationGetQuotas.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/LocationGetQuotas.json */ /** * Sample code: LocationGetQuotas. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedCloudServiceSkusSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedCloudServiceSkusSamples.java index b75573c0f922a..751196c6a4245 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedCloudServiceSkusSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedCloudServiceSkusSamples.java @@ -10,7 +10,7 @@ public final class LocationListSupportedCloudServiceSkusSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationListCloudServiceSkus.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/LocationListCloudServiceSkus.json */ /** * Sample code: LocationListCloudServiceSkus. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedVirtualMachineSkusSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedVirtualMachineSkusSamples.java index f3d0ad557beb6..a55532fabc945 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedVirtualMachineSkusSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/LocationListSupportedVirtualMachineSkusSamples.java @@ -10,7 +10,7 @@ public final class LocationListSupportedVirtualMachineSkusSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/LocationListVirtualMachineSkus. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/LocationListVirtualMachineSkus. * json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/OperationsListSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/OperationsListSamples.java index 55a9e3a500207..586078bf2c6e9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/OperationsListSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/OperationsListSamples.java @@ -10,7 +10,7 @@ public final class OperationsListSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/OperationsList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/OperationsList.json */ /** * Sample code: OperationsList. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolCreateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolCreateSamples.java index da759c8f33767..678978951dd1a 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolCreateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolCreateSamples.java @@ -10,6 +10,7 @@ import com.azure.resourcemanager.batch.models.AutoScaleSettings; import com.azure.resourcemanager.batch.models.AutoUserScope; import com.azure.resourcemanager.batch.models.AutoUserSpecification; +import com.azure.resourcemanager.batch.models.AutomaticOSUpgradePolicy; import com.azure.resourcemanager.batch.models.BatchPoolIdentity; import com.azure.resourcemanager.batch.models.CachingType; import com.azure.resourcemanager.batch.models.CertificateReference; @@ -46,6 +47,7 @@ import com.azure.resourcemanager.batch.models.PoolIdentityType; import com.azure.resourcemanager.batch.models.PublicIpAddressConfiguration; import com.azure.resourcemanager.batch.models.ResourceFile; +import com.azure.resourcemanager.batch.models.RollingUpgradePolicy; import com.azure.resourcemanager.batch.models.ScaleSettings; import com.azure.resourcemanager.batch.models.SecurityProfile; import com.azure.resourcemanager.batch.models.SecurityTypes; @@ -54,6 +56,8 @@ import com.azure.resourcemanager.batch.models.StorageAccountType; import com.azure.resourcemanager.batch.models.TaskSchedulingPolicy; import com.azure.resourcemanager.batch.models.UefiSettings; +import com.azure.resourcemanager.batch.models.UpgradeMode; +import com.azure.resourcemanager.batch.models.UpgradePolicy; import com.azure.resourcemanager.batch.models.UserAccount; import com.azure.resourcemanager.batch.models.UserAssignedIdentities; import com.azure.resourcemanager.batch.models.UserIdentity; @@ -71,7 +75,7 @@ */ public final class PoolCreateSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_VirtualMachineConfiguration_ServiceArtifactReference.json */ /** @@ -93,12 +97,14 @@ public static void createPoolVirtualMachineConfigurationServiceArtifactReference "/subscriptions/subid/resourceGroups/default-azurebatch-japaneast/providers/Microsoft.Compute/galleries/myGallery/serviceArtifacts/myServiceArtifact/vmArtifactsProfiles/vmArtifactsProfile")))) .withScaleSettings(new ScaleSettings() .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(2).withTargetLowPriorityNodes(0))) + .withUpgradePolicy(new UpgradePolicy().withMode(UpgradeMode.AUTOMATIC) + .withAutomaticOSUpgradePolicy(new AutomaticOSUpgradePolicy().withEnableAutomaticOSUpgrade(true))) .create(); } /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_SecurityProfile.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_SecurityProfile.json */ /** * Sample code: CreatePool - SecurityProfile. @@ -122,7 +128,7 @@ public static void createPoolSecurityProfile(com.azure.resourcemanager.batch.Bat } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_VirtualMachineConfiguration_ManagedOSDisk.json */ /** @@ -148,7 +154,7 @@ public static void createPoolSecurityProfile(com.azure.resourcemanager.batch.Bat } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_MinimalCloudServiceConfiguration.json */ /** @@ -167,7 +173,7 @@ public static void createPoolSecurityProfile(com.azure.resourcemanager.batch.Bat } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_MinimalVirtualMachineConfiguration.json */ /** @@ -192,7 +198,7 @@ public static void createPoolSecurityProfile(com.azure.resourcemanager.batch.Bat } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_VirtualMachineConfiguration_Extensions.json */ /** @@ -223,7 +229,7 @@ public static void createPoolVirtualMachineConfigurationExtensions( /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_UserAssignedIdentities + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_UserAssignedIdentities * .json */ /** @@ -254,7 +260,41 @@ public static void createPoolUserAssignedIdentities(com.azure.resourcemanager.ba /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_AcceleratedNetworking. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_UpgradePolicy.json + */ + /** + * Sample code: CreatePool - UpgradePolicy. + * + * @param manager Entry point to BatchManager. + */ + public static void createPoolUpgradePolicy(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().define("testpool").withExistingBatchAccount("default-azurebatch-japaneast", "sampleacct") + .withVmSize("Standard_d4s_v3") + .withDeploymentConfiguration( + new DeploymentConfiguration().withVirtualMachineConfiguration(new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("MicrosoftWindowsServer") + .withOffer("WindowsServer").withSku("2019-datacenter-smalldisk").withVersion("latest")) + .withNodeAgentSkuId("batch.node.windows amd64") + .withWindowsConfiguration(new WindowsConfiguration().withEnableAutomaticUpdates(false)) + .withNodePlacementConfiguration( + new NodePlacementConfiguration().withPolicy(NodePlacementPolicyType.ZONAL)))) + .withScaleSettings(new ScaleSettings() + .withFixedScale(new FixedScaleSettings().withTargetDedicatedNodes(2).withTargetLowPriorityNodes(0))) + .withUpgradePolicy( + new UpgradePolicy().withMode(UpgradeMode.AUTOMATIC) + .withAutomaticOSUpgradePolicy(new AutomaticOSUpgradePolicy().withDisableAutomaticRollback(true) + .withEnableAutomaticOSUpgrade(true).withUseRollingUpgradePolicy(true) + .withOsRollingUpgradeDeferral(true)) + .withRollingUpgradePolicy(new RollingUpgradePolicy().withEnableCrossZoneUpgrade(true) + .withMaxBatchInstancePercent(20).withMaxUnhealthyInstancePercent(20) + .withMaxUnhealthyUpgradedInstancePercent(20).withPauseTimeBetweenBatches("PT0S") + .withPrioritizeUnhealthyInstances(false).withRollbackFailedInstancesOnPolicyBreach(false))) + .create(); + } + + /* + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_AcceleratedNetworking. * json */ /** @@ -279,7 +319,7 @@ public static void createPoolAcceleratedNetworking(com.azure.resourcemanager.bat } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_VirtualMachineConfiguration.json */ /** @@ -331,7 +371,7 @@ public static void createPoolFullVirtualMachineConfiguration(com.azure.resourcem /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_SharedImageGallery. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_SharedImageGallery. * json */ /** @@ -350,7 +390,7 @@ public static void createPoolCustomImage(com.azure.resourcemanager.batch.BatchMa } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolCreate_CloudServiceConfiguration.json */ /** @@ -402,7 +442,7 @@ public static void createPoolFullCloudServiceConfiguration(com.azure.resourceman /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_NoPublicIPAddresses. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_NoPublicIPAddresses. * json */ /** @@ -426,7 +466,7 @@ public static void createPoolNoPublicIP(com.azure.resourcemanager.batch.BatchMan /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_ResourceTags.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_ResourceTags.json */ /** * Sample code: CreatePool - ResourceTags. @@ -448,7 +488,7 @@ public static void createPoolResourceTags(com.azure.resourcemanager.batch.BatchM /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolCreate_PublicIPs.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolCreate_PublicIPs.json */ /** * Sample code: CreatePool - Public IPs. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDeleteSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDeleteSamples.java index 5fbcf895a5074..bf5021ddfced0 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDeleteSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDeleteSamples.java @@ -10,7 +10,7 @@ public final class PoolDeleteSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolDelete.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolDelete.json */ /** * Sample code: DeletePool. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDisableAutoScaleSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDisableAutoScaleSamples.java index 3b70b3c5ae625..943106d6b503a 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDisableAutoScaleSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolDisableAutoScaleSamples.java @@ -10,7 +10,7 @@ public final class PoolDisableAutoScaleSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolDisableAutoScale.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolDisableAutoScale.json */ /** * Sample code: Disable AutoScale. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolGetSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolGetSamples.java index 0b72d8b4a63b5..fdc301d2c3ac2 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolGetSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolGetSamples.java @@ -10,7 +10,7 @@ public final class PoolGetSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_SecurityProfile.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet_SecurityProfile.json */ /** * Sample code: GetPool - SecurityProfile. @@ -23,7 +23,7 @@ public static void getPoolSecurityProfile(com.azure.resourcemanager.batch.BatchM } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolGet_VirtualMachineConfiguration_Extensions.json */ /** @@ -38,7 +38,7 @@ public static void getPoolSecurityProfile(com.azure.resourcemanager.batch.BatchM } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolGet_VirtualMachineConfiguration_MangedOSDisk.json */ /** @@ -52,7 +52,21 @@ public static void getPoolVirtualMachineConfigurationOSDisk(com.azure.resourcema } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/ + * x-ms-original-file: + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet_UpgradePolicy.json + */ + /** + * Sample code: GetPool - UpgradePolicy. + * + * @param manager Entry point to BatchManager. + */ + public static void getPoolUpgradePolicy(com.azure.resourcemanager.batch.BatchManager manager) { + manager.pools().getWithResponse("default-azurebatch-japaneast", "sampleacct", "testpool", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/ * PoolGet_VirtualMachineConfiguration_ServiceArtifactReference.json */ /** @@ -68,7 +82,7 @@ public static void getPoolVirtualMachineConfigurationServiceArtifactReference( /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet_AcceleratedNetworking. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet_AcceleratedNetworking. * json */ /** @@ -82,7 +96,7 @@ public static void getPoolAcceleratedNetworking(com.azure.resourcemanager.batch. } /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolGet.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolGet.json */ /** * Sample code: GetPool. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolListByBatchAccountSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolListByBatchAccountSamples.java index d71a1c506acce..32ca7c7372c9e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolListByBatchAccountSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolListByBatchAccountSamples.java @@ -9,7 +9,7 @@ */ public final class PoolListByBatchAccountSamples { /* - * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolList.json + * x-ms-original-file: specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolList.json */ /** * Sample code: ListPool. @@ -23,7 +23,7 @@ public static void listPool(com.azure.resourcemanager.batch.BatchManager manager /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolListWithFilter.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolListWithFilter.json */ /** * Sample code: ListPoolWithFilter. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolStopResizeSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolStopResizeSamples.java index 02f0e01ac2a7f..85d6ada39ed08 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolStopResizeSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolStopResizeSamples.java @@ -10,7 +10,7 @@ public final class PoolStopResizeSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolStopResize.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolStopResize.json */ /** * Sample code: StopPoolResize. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolUpdateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolUpdateSamples.java index 81ae0776c3e61..781011f9a6a5d 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolUpdateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PoolUpdateSamples.java @@ -24,7 +24,7 @@ public final class PoolUpdateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_EnableAutoScale.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_EnableAutoScale.json */ /** * Sample code: UpdatePool - Enable Autoscale. @@ -43,7 +43,7 @@ public static void updatePoolEnableAutoscale(com.azure.resourcemanager.batch.Bat /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_RemoveStartTask.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_RemoveStartTask.json */ /** * Sample code: UpdatePool - Remove Start Task. @@ -59,7 +59,7 @@ public static void updatePoolRemoveStartTask(com.azure.resourcemanager.batch.Bat /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_ResizePool.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_ResizePool.json */ /** * Sample code: UpdatePool - Resize Pool. @@ -79,7 +79,7 @@ public static void updatePoolResizePool(com.azure.resourcemanager.batch.BatchMan /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PoolUpdate_OtherProperties.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PoolUpdate_OtherProperties.json */ /** * Sample code: UpdatePool - Other Properties. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionDeleteSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionDeleteSamples.java index cf8ce2f125075..4d3464b08e828 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionDeleteSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionDeleteSamples.java @@ -10,7 +10,7 @@ public final class PrivateEndpointConnectionDeleteSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionDelete. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionDelete. * json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionGetSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionGetSamples.java index c1b9c2b8bae08..98c7ebf4bf58a 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionGetSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionGetSamples.java @@ -10,7 +10,7 @@ public final class PrivateEndpointConnectionGetSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionGet.json */ /** * Sample code: GetPrivateEndpointConnection. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionListByBatchAccountSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionListByBatchAccountSamples.java index 596bca57c53ce..f9130e75850db 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionListByBatchAccountSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionListByBatchAccountSamples.java @@ -10,7 +10,7 @@ public final class PrivateEndpointConnectionListByBatchAccountSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionsList. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionsList. * json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionUpdateSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionUpdateSamples.java index 7f16886536b41..b66100f9a731f 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionUpdateSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionUpdateSamples.java @@ -14,7 +14,7 @@ public final class PrivateEndpointConnectionUpdateSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateEndpointConnectionUpdate. + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateEndpointConnectionUpdate. * json */ /** diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceGetSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceGetSamples.java index 2bf527a562b02..e6085cc9840e5 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceGetSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceGetSamples.java @@ -10,7 +10,7 @@ public final class PrivateLinkResourceGetSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateLinkResourceGet.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateLinkResourceGet.json */ /** * Sample code: GetPrivateLinkResource. diff --git a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceListByBatchAccountSamples.java b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceListByBatchAccountSamples.java index 0c56af29b60ba..882ef621d597f 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceListByBatchAccountSamples.java +++ b/sdk/batch/azure-resourcemanager-batch/src/samples/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourceListByBatchAccountSamples.java @@ -10,7 +10,7 @@ public final class PrivateLinkResourceListByBatchAccountSamples { /* * x-ms-original-file: - * specification/batch/resource-manager/Microsoft.Batch/stable/2023-11-01/examples/PrivateLinkResourcesList.json + * specification/batch/resource-manager/Microsoft.Batch/stable/2024-02-01/examples/PrivateLinkResourcesList.json */ /** * Sample code: ListPrivateLinkResource. diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/BatchTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/BatchTests.java index f477ccdb10868..4ca96c955d849 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/BatchTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/BatchTests.java @@ -20,12 +20,13 @@ import com.azure.resourcemanager.batch.models.BatchAccount; import com.azure.resourcemanager.batch.models.BatchAccountKeys; import com.azure.resourcemanager.batch.models.BatchAccountRegenerateKeyParameters; -import com.azure.resourcemanager.batch.models.CloudServiceConfiguration; import com.azure.resourcemanager.batch.models.ComputeNodeDeallocationOption; import com.azure.resourcemanager.batch.models.DeploymentConfiguration; import com.azure.resourcemanager.batch.models.FixedScaleSettings; +import com.azure.resourcemanager.batch.models.ImageReference; import com.azure.resourcemanager.batch.models.Pool; import com.azure.resourcemanager.batch.models.ScaleSettings; +import com.azure.resourcemanager.batch.models.VirtualMachineConfiguration; import com.azure.resourcemanager.storage.StorageManager; import com.azure.resourcemanager.storage.models.StorageAccount; import org.junit.jupiter.api.Assertions; @@ -276,8 +277,11 @@ public void testCRUDBatchPool() { .withDisplayName(poolDisplayName) .withDeploymentConfiguration( new DeploymentConfiguration() - .withCloudServiceConfiguration( - new CloudServiceConfiguration().withOsFamily("4"))) + .withVirtualMachineConfiguration( + new VirtualMachineConfiguration() + .withImageReference(new ImageReference().withPublisher("Canonical") + .withOffer("UbuntuServer").withSku("18.04-LTS").withVersion("latest")) + .withNodeAgentSkuId("batch.node.ubuntu 18.04"))) .withScaleSettings( new ScaleSettings() .withFixedScale( diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesActivateWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesActivateWithResponseMockTests.java index f063dc9288122..da571bc8ad172 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesActivateWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesActivateWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testActivateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"state\":\"Active\",\"format\":\"mtdh\",\"storageUrl\":\"dvypgikdgsz\",\"storageUrlExpiry\":\"2021-09-09T14:02:20Z\",\"lastActivationTime\":\"2021-01-23T14:08:39Z\"},\"etag\":\"ryuzh\",\"id\":\"hkjoqr\",\"name\":\"qqaatjinrvgou\",\"type\":\"mfiibfggj\"}"; + = "{\"properties\":{\"state\":\"Pending\",\"format\":\"lazszrn\",\"storageUrl\":\"iin\",\"storageUrlExpiry\":\"2021-01-06T22:30:17Z\",\"lastActivationTime\":\"2021-03-15T09:11:44Z\"},\"etag\":\"ylwbtlhflsjcdhsz\",\"id\":\"jvfbgofelja\",\"name\":\"rqmq\",\"type\":\"ldvriiiojnalghfk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,8 +49,8 @@ public void testActivateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ApplicationPackage response = manager.applicationPackages() - .activateWithResponse("erqwkyhkobopg", "edkowepbqpcrfk", "wccsnjvcdwxlpqek", "tn", - new ActivateApplicationPackageParameters().withFormat("htjsying"), com.azure.core.util.Context.NONE) + .activateWithResponse("owqkdwytisi", "ircgpikpz", "mejzanlfzxia", "rmbzo", + new ActivateApplicationPackageParameters().withFormat("okixrjqcir"), com.azure.core.util.Context.NONE) .getValue(); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesCreateWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesCreateWithResponseMockTests.java index 8d11d244fd3e9..23f3e3df80f51 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesCreateWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesCreateWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testCreateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"state\":\"Pending\",\"format\":\"kpj\",\"storageUrl\":\"qmt\",\"storageUrlExpiry\":\"2021-03-16T18:33:01Z\",\"lastActivationTime\":\"2021-09-01T21:15:43Z\"},\"etag\":\"jihy\",\"id\":\"ozphvwauyqncygu\",\"name\":\"kvi\",\"type\":\"mdscwxqupev\"}"; + = "{\"properties\":{\"state\":\"Active\",\"format\":\"gvjayvblmh\",\"storageUrl\":\"zuhbxvvyhgsopb\",\"storageUrlExpiry\":\"2021-10-16T16:37:11Z\",\"lastActivationTime\":\"2021-05-14T00:06:10Z\"},\"etag\":\"g\",\"id\":\"uvwzfbnh\",\"name\":\"mctlpdngitv\",\"type\":\"bmhrixkwmyijejv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,8 +47,8 @@ public void testCreateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - ApplicationPackage response - = manager.applicationPackages().define("jjoqkagf").withExistingApplication("brnjwmw", "pn", "saz").create(); + ApplicationPackage response = manager.applicationPackages().define("tmtdhtmdvypgik") + .withExistingApplication("tn", "htjsying", "fq").create(); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesDeleteWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesDeleteWithResponseMockTests.java index 647c7cc7b3467..162f62cfb3b69 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesDeleteWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.applicationPackages().deleteWithResponse("ool", "rwxkvtkkgl", "qwjygvja", "vblm", + manager.applicationPackages().deleteWithResponse("tvsexsowuel", "qhhahhxvrhmzkwpj", "wws", "ughftqsx", com.azure.core.util.Context.NONE); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesGetWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesGetWithResponseMockTests.java index c50fd477fb99e..3543fa5e16903 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesGetWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesGetWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"state\":\"Pending\",\"format\":\"ixkwmyijejveg\",\"storageUrl\":\"bpnaixexccbdre\",\"storageUrlExpiry\":\"2021-04-07T14:01:01Z\",\"lastActivationTime\":\"2021-09-06T10:46:02Z\"},\"etag\":\"drrvqahqkght\",\"id\":\"wijnh\",\"name\":\"jsvfycxzbfvoowv\",\"type\":\"vmtgjqppy\"}"; + = "{\"properties\":{\"state\":\"Pending\",\"format\":\"zqzudph\",\"storageUrl\":\"mvdk\",\"storageUrlExpiry\":\"2021-09-19T23:00:24Z\",\"lastActivationTime\":\"2021-08-12T04:12:29Z\"},\"etag\":\"vtbvkayh\",\"id\":\"tnvyqiatkzwp\",\"name\":\"npwzcjaes\",\"type\":\"vvsccyajguq\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,7 +48,7 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ApplicationPackage response = manager.applicationPackages() - .getWithResponse("vkzuhbxvvyhgso", "byrqufeg", "uvwzfbnh", "mctlpdngitv", com.azure.core.util.Context.NONE) + .getWithResponse("qxujxukndxd", "grjguufzd", "syqtfi", "whbotzingamv", com.azure.core.util.Context.NONE) .getValue(); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesListMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesListMockTests.java index 2f2a271484561..42e0956db9d08 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesListMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationPackagesListMockTests.java @@ -31,7 +31,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"state\":\"Pending\",\"format\":\"rjreafxts\",\"storageUrl\":\"mhjglikkxwslolb\",\"storageUrlExpiry\":\"2021-11-05T04:06:51Z\",\"lastActivationTime\":\"2021-08-07T04:15:04Z\"},\"etag\":\"m\",\"id\":\"felfktg\",\"name\":\"lcrpw\",\"type\":\"xeznoi\"}]}"; + = "{\"value\":[{\"properties\":{\"state\":\"Active\",\"format\":\"dpsqx\",\"storageUrl\":\"psvuoymgc\",\"storageUrlExpiry\":\"2021-08-18T08:31:36Z\",\"lastActivationTime\":\"2021-01-01T10:52:13Z\"},\"etag\":\"rypqlmfeo\",\"id\":\"erqwkyhkobopg\",\"name\":\"edkowepbqpcrfk\",\"type\":\"wccsnjvcdwxlpqek\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,8 @@ public void testList() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.applicationPackages().list("s", "ronzmyhgfip", "sxkm", - 2130942717, com.azure.core.util.Context.NONE); + PagedIterable response = manager.applicationPackages().list("hwyg", "lvdnkfx", "semdwzrmu", + 114396091, com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsCreateWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsCreateWithResponseMockTests.java index 2a9bb29d84c6b..aae7b66c37efb 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsCreateWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsCreateWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testCreateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"displayName\":\"zejntps\",\"allowUpdates\":false,\"defaultVersion\":\"oi\"},\"etag\":\"ukry\",\"id\":\"xtqmieoxor\",\"name\":\"gufhyaomtbg\",\"type\":\"havgrvk\"}"; + = "{\"properties\":{\"displayName\":\"txhojujb\",\"allowUpdates\":true,\"defaultVersion\":\"mc\"},\"etag\":\"hixbjxyfwnyl\",\"id\":\"coolsttpkiwkkb\",\"name\":\"ujrywvtyl\",\"type\":\"fpncurdo\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,11 +48,11 @@ public void testCreateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - Application response = manager.applications().define("lgnyhmo").withExistingBatchAccount("tw", "sgogczhonnxk") - .withDisplayName("kkgthr").withAllowUpdates(true).withDefaultVersion("jbdhqxvc").create(); + Application response = manager.applications().define("aays").withExistingBatchAccount("ou", "ibreb") + .withDisplayName("xqtnq").withAllowUpdates(false).withDefaultVersion("lwfffi").create(); - Assertions.assertEquals("zejntps", response.displayName()); - Assertions.assertEquals(false, response.allowUpdates()); - Assertions.assertEquals("oi", response.defaultVersion()); + Assertions.assertEquals("txhojujb", response.displayName()); + Assertions.assertEquals(true, response.allowUpdates()); + Assertions.assertEquals("mc", response.defaultVersion()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsDeleteWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsDeleteWithResponseMockTests.java index ccbf6bb90f703..6da9dea6a26a5 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsDeleteWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsDeleteWithResponseMockTests.java @@ -45,8 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.applications().deleteWithResponse("hfstotxhojujbyp", "lmcuvhixb", "xyfwnylrcool", - com.azure.core.util.Context.NONE); + manager.applications().deleteWithResponse("grhbpn", "ixexcc", "dreaxh", com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsGetWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsGetWithResponseMockTests.java index bc192778bb201..2cf6598896a8e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsGetWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"displayName\":\"ithtywu\",\"allowUpdates\":false,\"defaultVersion\":\"ihwqknfdntwjchr\"},\"etag\":\"oihxumwctondzjlu\",\"id\":\"dfdlwggyts\",\"name\":\"wtovvtgsein\",\"type\":\"fiufx\"}"; + = "{\"properties\":{\"displayName\":\"t\",\"allowUpdates\":false,\"defaultVersion\":\"pyostronzmyhgfi\"},\"etag\":\"sxkm\",\"id\":\"waekrrjreafxtsgu\",\"name\":\"hjglikk\",\"type\":\"wslolbqp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,10 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); Application response = manager.applications() - .getWithResponse("ttpkiwkkbnujrywv", "y", "bfpncurdo", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("exdrrvqahqkg", "tpwijnh", "jsvfycxzbfvoowv", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("ithtywu", response.displayName()); + Assertions.assertEquals("t", response.displayName()); Assertions.assertEquals(false, response.allowUpdates()); - Assertions.assertEquals("ihwqknfdntwjchr", response.defaultVersion()); + Assertions.assertEquals("pyostronzmyhgfi", response.defaultVersion()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsListMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsListMockTests.java index a15d305814d72..6b36791995a8e 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsListMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ApplicationsListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"displayName\":\"r\",\"allowUpdates\":true,\"defaultVersion\":\"ijnkrxfrdd\"},\"etag\":\"ratiz\",\"id\":\"ronasxift\",\"name\":\"zq\",\"type\":\"zh\"}]}"; + = "{\"value\":[{\"properties\":{\"displayName\":\"pwjxezn\",\"allowUpdates\":false,\"defaultVersion\":\"rnjwmw\"},\"etag\":\"nbsazejjoqkag\",\"id\":\"hsxttaugzxnf\",\"name\":\"azpxdtnkdmkqjjl\",\"type\":\"uenvrkp\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,10 +50,10 @@ public void testList() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.applications().list("knpirgnepttwq", "sniffc", 1105957042, com.azure.core.util.Context.NONE); + = manager.applications().list("uzlm", "felfktg", 1170728394, com.azure.core.util.Context.NONE); - Assertions.assertEquals("r", response.iterator().next().displayName()); - Assertions.assertEquals(true, response.iterator().next().allowUpdates()); - Assertions.assertEquals("ijnkrxfrdd", response.iterator().next().defaultVersion()); + Assertions.assertEquals("pwjxezn", response.iterator().next().displayName()); + Assertions.assertEquals(false, response.iterator().next().allowUpdates()); + Assertions.assertEquals("rnjwmw", response.iterator().next().defaultVersion()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/AutomaticOSUpgradePolicyTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/AutomaticOSUpgradePolicyTests.java new file mode 100644 index 0000000000000..562ab12ce8f9b --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/AutomaticOSUpgradePolicyTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.batch.models.AutomaticOSUpgradePolicy; +import org.junit.jupiter.api.Assertions; + +public final class AutomaticOSUpgradePolicyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + AutomaticOSUpgradePolicy model = BinaryData.fromString( + "{\"disableAutomaticRollback\":true,\"enableAutomaticOSUpgrade\":true,\"useRollingUpgradePolicy\":true,\"osRollingUpgradeDeferral\":false}") + .toObject(AutomaticOSUpgradePolicy.class); + Assertions.assertEquals(true, model.disableAutomaticRollback()); + Assertions.assertEquals(true, model.enableAutomaticOSUpgrade()); + Assertions.assertEquals(true, model.useRollingUpgradePolicy()); + Assertions.assertEquals(false, model.osRollingUpgradeDeferral()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + AutomaticOSUpgradePolicy model = new AutomaticOSUpgradePolicy().withDisableAutomaticRollback(true) + .withEnableAutomaticOSUpgrade(true).withUseRollingUpgradePolicy(true).withOsRollingUpgradeDeferral(false); + model = BinaryData.fromObject(model).toObject(AutomaticOSUpgradePolicy.class); + Assertions.assertEquals(true, model.disableAutomaticRollback()); + Assertions.assertEquals(true, model.enableAutomaticOSUpgrade()); + Assertions.assertEquals(true, model.useRollingUpgradePolicy()); + Assertions.assertEquals(false, model.osRollingUpgradeDeferral()); + } +} diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsDeleteMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsDeleteMockTests.java index c7750a469eb1e..d5255c48aba32 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsDeleteMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.batchAccounts().delete("cccnxqhuexmktt", "stvlzywemhzrnc", com.azure.core.util.Context.NONE); + manager.batchAccounts().delete("ngqqmoakuf", "m", com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetDetectorWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetDetectorWithResponseMockTests.java index b249813a232ae..c64613103d0a5 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetDetectorWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetDetectorWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetDetectorWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"value\":\"j\"},\"etag\":\"nlfzxiavrmbz\",\"id\":\"nokixrjqcirgz\",\"name\":\"frl\",\"type\":\"zszrnwoiindfpw\"}"; + = "{\"properties\":{\"value\":\"wxqibyq\"},\"etag\":\"y\",\"id\":\"wxwlmdjrkvfgb\",\"name\":\"fvpdbo\",\"type\":\"acizsjqlhkrr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,9 +49,8 @@ public void testGetDetectorWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); DetectorResponse response = manager.batchAccounts() - .getDetectorWithResponse("zpdrhneu", "owqkdwytisi", "ircgpikpz", com.azure.core.util.Context.NONE) - .getValue(); + .getDetectorWithResponse("nmdyodnwzxl", "jc", "nhltiugcxn", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("j", response.value()); + Assertions.assertEquals("wxqibyq", response.value()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetKeysWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetKeysWithResponseMockTests.java index 81b311c479944..09630782559ba 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetKeysWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsGetKeysWithResponseMockTests.java @@ -29,8 +29,7 @@ public void testGetKeysWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr - = "{\"accountName\":\"abudurgk\",\"primary\":\"mokzhjjklf\",\"secondary\":\"mouwqlgzrfzeey\"}"; + String responseStr = "{\"accountName\":\"sx\",\"primary\":\"pelol\",\"secondary\":\"vk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,7 +47,7 @@ public void testGetKeysWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); BatchAccountKeys response = manager.batchAccounts() - .getKeysWithResponse("tawfsdjpvkvp", "jxbkzbzkdvn", com.azure.core.util.Context.NONE).getValue(); + .getKeysWithResponse("sfgytguslfead", "ygqukyhejh", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListDetectorsMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListDetectorsMockTests.java index c729a65ad502e..93b11fd78e601 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListDetectorsMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListDetectorsMockTests.java @@ -32,7 +32,7 @@ public void testListDetectors() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"value\":\"vtldgmfpgvmpip\"},\"etag\":\"ltha\",\"id\":\"fxssm\",\"name\":\"u\",\"type\":\"wbdsr\"}]}"; + = "{\"value\":[{\"properties\":{\"value\":\"iby\"},\"etag\":\"dl\",\"id\":\"h\",\"name\":\"hfwpracstwit\",\"type\":\"khevxccedc\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,8 +50,8 @@ public void testListDetectors() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.batchAccounts().listDetectors("bizikayuhq", "bjbsybb", com.azure.core.util.Context.NONE); + = manager.batchAccounts().listDetectors("r", "qvujzraehtwdwrf", com.azure.core.util.Context.NONE); - Assertions.assertEquals("vtldgmfpgvmpip", response.iterator().next().value()); + Assertions.assertEquals("iby", response.iterator().next().value()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListOutboundNetworkDependenciesEndpointsMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListOutboundNetworkDependenciesEndpointsMockTests.java index ae30548c6f1c5..b44ad7e627313 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListOutboundNetworkDependenciesEndpointsMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsListOutboundNetworkDependenciesEndpointsMockTests.java @@ -31,7 +31,7 @@ public void testListOutboundNetworkDependenciesEndpoints() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"category\":\"gofel\",\"endpoints\":[{\"domainName\":\"qmqhldvriii\",\"description\":\"nalghfkvtvsexso\",\"endpointDetails\":[{\"port\":750613603}]},{\"domainName\":\"hhahhxvrhmzkwpjg\",\"description\":\"spughftqsxhq\",\"endpointDetails\":[{\"port\":1043594418}]},{\"domainName\":\"ndxdigrjguufzdm\",\"description\":\"qtfihwhbotzinga\",\"endpointDetails\":[{\"port\":313573135}]}]}]}"; + = "{\"value\":[{\"category\":\"wkqnyhg\",\"endpoints\":[{\"domainName\":\"jivfxzsjabib\",\"description\":\"stawfsdjpvkv\",\"endpointDetails\":[{\"port\":1731157683},{\"port\":2005315516},{\"port\":2094987442}]}]}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,9 @@ public void testListOutboundNetworkDependenciesEndpoints() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.batchAccounts() - .listOutboundNetworkDependenciesEndpoints("jylwbtlhflsj", "dhszfjv", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.batchAccounts().listOutboundNetworkDependenciesEndpoints("bdeibqipqk", "hvxndzwmkrefajpj", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsSynchronizeAutoStorageKeysWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsSynchronizeAutoStorageKeysWithResponseMockTests.java index 00059fc180263..6536da3ee7cd7 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsSynchronizeAutoStorageKeysWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchAccountsSynchronizeAutoStorageKeysWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testSynchronizeAutoStorageKeysWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.batchAccounts().synchronizeAutoStorageKeysWithResponse("hvxndzwmkrefajpj", "rwkq", + manager.batchAccounts().synchronizeAutoStorageKeysWithResponse("xggicccnxqhuexmk", "tlstvlzywem", com.azure.core.util.Context.NONE); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchPoolIdentityTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchPoolIdentityTests.java index e0c59247d599a..8f8da926fc6bf 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchPoolIdentityTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/BatchPoolIdentityTests.java @@ -16,16 +16,16 @@ public final class BatchPoolIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BatchPoolIdentity model = BinaryData.fromString( - "{\"type\":\"None\",\"userAssignedIdentities\":{\"dxrbuukzcle\":{\"principalId\":\"kfzbeyvpnqicvi\",\"clientId\":\"kjj\"},\"qa\":{\"principalId\":\"hmlwpaztzpo\",\"clientId\":\"cckwyfzqwhxxbu\"},\"obqwcsdbnwdcfh\":{\"principalId\":\"feqztppriol\",\"clientId\":\"rjaltolmncw\"}}}") + "{\"type\":\"None\",\"userAssignedIdentities\":{\"ppriol\":{\"principalId\":\"whxxbuyqax\",\"clientId\":\"eqz\"},\"ucqdpfuvglsb\":{\"principalId\":\"rjaltolmncw\",\"clientId\":\"bqwcsdbnwdcf\"},\"ncormrlxqtvcof\":{\"principalId\":\"ca\",\"clientId\":\"xbvtvudu\"},\"n\":{\"principalId\":\"f\",\"clientId\":\"kgjubgdknnqvsazn\"}}}") .toObject(BatchPoolIdentity.class); Assertions.assertEquals(PoolIdentityType.NONE, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BatchPoolIdentity model = new BatchPoolIdentity().withType(PoolIdentityType.NONE) - .withUserAssignedIdentities(mapOf("dxrbuukzcle", new UserAssignedIdentities(), "qa", - new UserAssignedIdentities(), "obqwcsdbnwdcfh", new UserAssignedIdentities())); + BatchPoolIdentity model = new BatchPoolIdentity().withType(PoolIdentityType.NONE).withUserAssignedIdentities( + mapOf("ppriol", new UserAssignedIdentities(), "ucqdpfuvglsb", new UserAssignedIdentities(), + "ncormrlxqtvcof", new UserAssignedIdentities(), "n", new UserAssignedIdentities())); model = BinaryData.fromObject(model).toObject(BatchPoolIdentity.class); Assertions.assertEquals(PoolIdentityType.NONE, model.type()); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificateBasePropertiesTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificateBasePropertiesTests.java index a173f0d6f7973..7e01a08829471 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificateBasePropertiesTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificateBasePropertiesTests.java @@ -13,20 +13,20 @@ public final class CertificateBasePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CertificateBaseProperties model = BinaryData - .fromString("{\"thumbprintAlgorithm\":\"tki\",\"thumbprint\":\"xhqyudxorrqnb\",\"format\":\"Pfx\"}") + .fromString("{\"thumbprintAlgorithm\":\"mdectehfiqscjey\",\"thumbprint\":\"hezrkgq\",\"format\":\"Cer\"}") .toObject(CertificateBaseProperties.class); - Assertions.assertEquals("tki", model.thumbprintAlgorithm()); - Assertions.assertEquals("xhqyudxorrqnb", model.thumbprint()); - Assertions.assertEquals(CertificateFormat.PFX, model.format()); + Assertions.assertEquals("mdectehfiqscjey", model.thumbprintAlgorithm()); + Assertions.assertEquals("hezrkgq", model.thumbprint()); + Assertions.assertEquals(CertificateFormat.CER, model.format()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CertificateBaseProperties model = new CertificateBaseProperties().withThumbprintAlgorithm("tki") - .withThumbprint("xhqyudxorrqnb").withFormat(CertificateFormat.PFX); + CertificateBaseProperties model = new CertificateBaseProperties().withThumbprintAlgorithm("mdectehfiqscjey") + .withThumbprint("hezrkgq").withFormat(CertificateFormat.CER); model = BinaryData.fromObject(model).toObject(CertificateBaseProperties.class); - Assertions.assertEquals("tki", model.thumbprintAlgorithm()); - Assertions.assertEquals("xhqyudxorrqnb", model.thumbprint()); - Assertions.assertEquals(CertificateFormat.PFX, model.format()); + Assertions.assertEquals("mdectehfiqscjey", model.thumbprintAlgorithm()); + Assertions.assertEquals("hezrkgq", model.thumbprint()); + Assertions.assertEquals(CertificateFormat.CER, model.format()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificatesDeleteMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificatesDeleteMockTests.java index 2d674b1019a46..8ad808c57ee54 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificatesDeleteMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CertificatesDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.certificates().delete("novqfzge", "jdftuljltd", "ceamtm", com.azure.core.util.Context.NONE); + manager.certificates().delete("bgye", "rymsgaojfmw", "cotmr", com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityParametersTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityParametersTests.java index 417d030d8deac..4e498e8257fd8 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityParametersTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityParametersTests.java @@ -12,14 +12,14 @@ public final class CheckNameAvailabilityParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CheckNameAvailabilityParameters model - = BinaryData.fromString("{\"name\":\"cczsq\"}").toObject(CheckNameAvailabilityParameters.class); - Assertions.assertEquals("cczsq", model.name()); + = BinaryData.fromString("{\"name\":\"cftadeh\"}").toObject(CheckNameAvailabilityParameters.class); + Assertions.assertEquals("cftadeh", model.name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CheckNameAvailabilityParameters model = new CheckNameAvailabilityParameters().withName("cczsq"); + CheckNameAvailabilityParameters model = new CheckNameAvailabilityParameters().withName("cftadeh"); model = BinaryData.fromObject(model).toObject(CheckNameAvailabilityParameters.class); - Assertions.assertEquals("cczsq", model.name()); + Assertions.assertEquals("cftadeh", model.name()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityResultInnerTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityResultInnerTests.java index 5c934faa58361..917e73d354018 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityResultInnerTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/CheckNameAvailabilityResultInnerTests.java @@ -11,7 +11,7 @@ public final class CheckNameAvailabilityResultInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CheckNameAvailabilityResultInner model - = BinaryData.fromString("{\"nameAvailable\":true,\"reason\":\"Invalid\",\"message\":\"ajvnysounqe\"}") + = BinaryData.fromString("{\"nameAvailable\":false,\"reason\":\"Invalid\",\"message\":\"sop\"}") .toObject(CheckNameAvailabilityResultInner.class); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorListResultTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorListResultTests.java index 213227942e5a6..495633c536052 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorListResultTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorListResultTests.java @@ -14,19 +14,20 @@ public final class DetectorListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DetectorListResult model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"value\":\"spkwlhzdobpxjm\"},\"etag\":\"bvvnchrkcciw\",\"id\":\"zjuqkhrsaj\",\"name\":\"wkuofoskghsauu\",\"type\":\"mjmvxieduugidyjr\"},{\"properties\":{\"value\":\"y\"},\"etag\":\"svexcsonpclhoco\",\"id\":\"slkevle\",\"name\":\"gz\",\"type\":\"buhfmvfaxkffeiit\"}],\"nextLink\":\"vmezy\"}") + "{\"value\":[{\"properties\":{\"value\":\"q\"},\"etag\":\"a\",\"id\":\"oaeupfhyhltrpmo\",\"name\":\"jmcmatuokthfu\",\"type\":\"uaodsfcpk\"},{\"properties\":{\"value\":\"dpuozmyz\"},\"etag\":\"agfuaxbezyiu\",\"id\":\"kktwhrdxw\",\"name\":\"ywqsmbsurexim\",\"type\":\"ryocfsfksymdd\"},{\"properties\":{\"value\":\"kiiuxhqyudxor\"},\"etag\":\"nbpoczvyifqrvkdv\",\"id\":\"sllr\",\"name\":\"vvdfwatkpnpul\",\"type\":\"xxbczwtr\"}],\"nextLink\":\"iqzbq\"}") .toObject(DetectorListResult.class); - Assertions.assertEquals("spkwlhzdobpxjm", model.value().get(0).value()); - Assertions.assertEquals("vmezy", model.nextLink()); + Assertions.assertEquals("q", model.value().get(0).value()); + Assertions.assertEquals("iqzbq", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DetectorListResult model - = new DetectorListResult().withValue(Arrays.asList(new DetectorResponseInner().withValue("spkwlhzdobpxjm"), - new DetectorResponseInner().withValue("y"))).withNextLink("vmezy"); + DetectorListResult model = new DetectorListResult().withValue( + Arrays.asList(new DetectorResponseInner().withValue("q"), new DetectorResponseInner().withValue("dpuozmyz"), + new DetectorResponseInner().withValue("kiiuxhqyudxor"))) + .withNextLink("iqzbq"); model = BinaryData.fromObject(model).toObject(DetectorListResult.class); - Assertions.assertEquals("spkwlhzdobpxjm", model.value().get(0).value()); - Assertions.assertEquals("vmezy", model.nextLink()); + Assertions.assertEquals("q", model.value().get(0).value()); + Assertions.assertEquals("iqzbq", model.nextLink()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponseInnerTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponseInnerTests.java index fce80d78707f4..935e4a87e7a55 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponseInnerTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponseInnerTests.java @@ -12,15 +12,15 @@ public final class DetectorResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DetectorResponseInner model = BinaryData.fromString( - "{\"properties\":{\"value\":\"xmzsbbzogg\"},\"etag\":\"rxwburv\",\"id\":\"xxjnspydptk\",\"name\":\"enkouknvudw\",\"type\":\"iukbldngkpoci\"}") + "{\"properties\":{\"value\":\"ovm\"},\"etag\":\"kacspkw\",\"id\":\"hzdobpxjmflbvvnc\",\"name\":\"rkcciwwzjuqk\",\"type\":\"rsa\"}") .toObject(DetectorResponseInner.class); - Assertions.assertEquals("xmzsbbzogg", model.value()); + Assertions.assertEquals("ovm", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DetectorResponseInner model = new DetectorResponseInner().withValue("xmzsbbzogg"); + DetectorResponseInner model = new DetectorResponseInner().withValue("ovm"); model = BinaryData.fromObject(model).toObject(DetectorResponseInner.class); - Assertions.assertEquals("xmzsbbzogg", model.value()); + Assertions.assertEquals("ovm", model.value()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponsePropertiesTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponsePropertiesTests.java index 061c29da869d5..329ecb2e5f644 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponsePropertiesTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/DetectorResponsePropertiesTests.java @@ -12,14 +12,14 @@ public final class DetectorResponsePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DetectorResponseProperties model - = BinaryData.fromString("{\"value\":\"z\"}").toObject(DetectorResponseProperties.class); - Assertions.assertEquals("z", model.value()); + = BinaryData.fromString("{\"value\":\"wkuofoskghsauu\"}").toObject(DetectorResponseProperties.class); + Assertions.assertEquals("wkuofoskghsauu", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DetectorResponseProperties model = new DetectorResponseProperties().withValue("z"); + DetectorResponseProperties model = new DetectorResponseProperties().withValue("wkuofoskghsauu"); model = BinaryData.fromObject(model).toObject(DetectorResponseProperties.class); - Assertions.assertEquals("z", model.value()); + Assertions.assertEquals("wkuofoskghsauu", model.value()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDependencyTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDependencyTests.java index 7c02e4a58f271..9533f87d55e9c 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDependencyTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDependencyTests.java @@ -11,7 +11,7 @@ public final class EndpointDependencyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { EndpointDependency model = BinaryData.fromString( - "{\"domainName\":\"luhczbw\",\"description\":\"hairsbrgzdwms\",\"endpointDetails\":[{\"port\":151812446},{\"port\":2076786264},{\"port\":840450646}]}") + "{\"domainName\":\"akgtdlmkkzevdlh\",\"description\":\"pusdstt\",\"endpointDetails\":[{\"port\":221713689},{\"port\":1374887959},{\"port\":2086607804}]}") .toObject(EndpointDependency.class); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDetailTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDetailTests.java index b0f32953de99e..4a700d2eaeb98 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDetailTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/EndpointDetailTests.java @@ -10,7 +10,7 @@ public final class EndpointDetailTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - EndpointDetail model = BinaryData.fromString("{\"port\":1308163389}").toObject(EndpointDetail.class); + EndpointDetail model = BinaryData.fromString("{\"port\":1690092543}").toObject(EndpointDetail.class); } @org.junit.jupiter.api.Test diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ListPrivateLinkResourcesResultTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ListPrivateLinkResourcesResultTests.java index 41a4915fa9a5b..be5fb909f8e2c 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ListPrivateLinkResourcesResultTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/ListPrivateLinkResourcesResultTests.java @@ -14,7 +14,7 @@ public final class ListPrivateLinkResourcesResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ListPrivateLinkResourcesResult model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"groupId\":\"ukgjnpiucgygevq\",\"requiredMembers\":[\"yp\",\"rbpizc\",\"r\",\"j\"],\"requiredZoneNames\":[\"ydnfyhxdeoejz\"]},\"etag\":\"w\",\"id\":\"fsj\",\"name\":\"tgzfbishcbkh\",\"type\":\"jdeyeamdpha\"}],\"nextLink\":\"lpbuxwgipwhonowk\"}") + "{\"value\":[{\"properties\":{\"groupId\":\"xieduugidyjrr\",\"requiredMembers\":[\"aos\"],\"requiredZoneNames\":[\"csonpclhoco\"]},\"etag\":\"lkevle\",\"id\":\"gz\",\"name\":\"buhfmvfaxkffeiit\",\"type\":\"lvmezyvshxmzsbbz\"},{\"properties\":{\"groupId\":\"igrxwburvjxxjn\",\"requiredMembers\":[\"dptkoenkouk\",\"vudwtiukbldng\"],\"requiredZoneNames\":[\"cipazyxoegukgjnp\",\"ucgygevqz\",\"typmrbpizcdrqjsd\"]},\"etag\":\"dnfyhxdeoejzicwi\",\"id\":\"sjttgzfbish\",\"name\":\"bkh\",\"type\":\"jdeyeamdpha\"}],\"nextLink\":\"lpbuxwgipwhonowk\"}") .toObject(ListPrivateLinkResourcesResult.class); Assertions.assertEquals("lpbuxwgipwhonowk", model.nextLink()); } @@ -22,7 +22,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ListPrivateLinkResourcesResult model = new ListPrivateLinkResourcesResult() - .withValue(Arrays.asList(new PrivateLinkResourceInner())).withNextLink("lpbuxwgipwhonowk"); + .withValue(Arrays.asList(new PrivateLinkResourceInner(), new PrivateLinkResourceInner())) + .withNextLink("lpbuxwgipwhonowk"); model = BinaryData.fromObject(model).toObject(ListPrivateLinkResourcesResult.class); Assertions.assertEquals("lpbuxwgipwhonowk", model.nextLink()); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsCheckNameAvailabilityWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsCheckNameAvailabilityWithResponseMockTests.java index 0f96c65c9aeb1..51e1a9f2c7fb4 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsCheckNameAvailabilityWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"nameAvailable\":false,\"reason\":\"AlreadyExists\",\"message\":\"sutrgjup\"}"; + String responseStr = "{\"nameAvailable\":true,\"reason\":\"AlreadyExists\",\"message\":\"qg\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,8 +47,11 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - CheckNameAvailabilityResult response = manager.locations().checkNameAvailabilityWithResponse("xifqjzgxm", - new CheckNameAvailabilityParameters().withName("hu"), com.azure.core.util.Context.NONE).getValue(); + CheckNameAvailabilityResult response + = manager.locations() + .checkNameAvailabilityWithResponse("swdvzyybycnun", + new CheckNameAvailabilityParameters().withName("jsrtk"), com.azure.core.util.Context.NONE) + .getValue(); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsGetQuotasWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsGetQuotasWithResponseMockTests.java index e66cd25220b8f..da9bf35900117 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsGetQuotasWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsGetQuotasWithResponseMockTests.java @@ -29,7 +29,7 @@ public void testGetQuotasWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"accountQuota\":284143566}"; + String responseStr = "{\"accountQuota\":709721668}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,7 +47,7 @@ public void testGetQuotasWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); BatchLocationQuota response - = manager.locations().getQuotasWithResponse("fo", com.azure.core.util.Context.NONE).getValue(); + = manager.locations().getQuotasWithResponse("wiithtywub", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedCloudServiceSkusMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedCloudServiceSkusMockTests.java index c9d4346e4e8c6..efc6328c168f6 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedCloudServiceSkusMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedCloudServiceSkusMockTests.java @@ -31,7 +31,7 @@ public void testListSupportedCloudServiceSkus() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"cpqjlihhyu\",\"familyName\":\"skasdvlmfwdgzxu\",\"capabilities\":[{\"name\":\"pamrsr\",\"value\":\"zvxurisjnhny\"}]}]}"; + = "{\"value\":[{\"name\":\"tizzronasxif\",\"familyName\":\"zq\",\"capabilities\":[{\"name\":\"tw\",\"value\":\"gogczhonnxkrlgny\"},{\"name\":\"ossxk\",\"value\":\"thrrgh\"},{\"name\":\"bdhqxvcxgf\",\"value\":\"dsofbshrns\"}],\"batchSupportEndOfLife\":\"2021-07-30T15:10:49Z\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,8 @@ public void testListSupportedCloudServiceSkus() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.locations().listSupportedCloudServiceSkus("tymoxoftp", - 1701654181, "iwyczuh", com.azure.core.util.Context.NONE); + PagedIterable response = manager.locations().listSupportedCloudServiceSkus("qnrojlpijnkrxfrd", + 1333681706, "c", com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedVirtualMachineSkusMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedVirtualMachineSkusMockTests.java index 1b82cb8c0e41e..03cac32e4c753 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedVirtualMachineSkusMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/LocationsListSupportedVirtualMachineSkusMockTests.java @@ -31,7 +31,7 @@ public void testListSupportedVirtualMachineSkus() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"bkfezzxscyhwzdgi\",\"familyName\":\"jbzbomvzzbtdcq\",\"capabilities\":[{\"name\":\"yujviylwdshfssn\",\"value\":\"gy\"},{\"name\":\"rymsgaojfmw\",\"value\":\"otmrfhir\"}]}]}"; + = "{\"value\":[{\"name\":\"fdlwg\",\"familyName\":\"tsbwtovvtgse\",\"capabilities\":[{\"name\":\"iufxqknpir\",\"value\":\"epttwqmsniff\"}],\"batchSupportEndOfLife\":\"2021-10-31T18:45:45Z\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,8 @@ public void testListSupportedVirtualMachineSkus() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.locations().listSupportedVirtualMachineSkus("hpjbib", 1956695064, - "mfxumvfcluyovw", com.azure.core.util.Context.NONE); + PagedIterable response = manager.locations().listSupportedVirtualMachineSkus("ihwqknfdntwjchr", + 1139414595, "oihxumwctondzjlu", com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationDisplayTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationDisplayTests.java index 70758bcd06b95..13fb49cecd22f 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationDisplayTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationDisplayTests.java @@ -12,22 +12,22 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OperationDisplay model = BinaryData.fromString( - "{\"provider\":\"mdectehfiqscjey\",\"operation\":\"hezrkgq\",\"resource\":\"jrefovgmkqsle\",\"description\":\"vxyqjpkcattpngjc\"}") + "{\"provider\":\"gual\",\"operation\":\"xxhejjzzvd\",\"resource\":\"gwdslfhotwm\",\"description\":\"npwlbjnpg\"}") .toObject(OperationDisplay.class); - Assertions.assertEquals("mdectehfiqscjey", model.provider()); - Assertions.assertEquals("hezrkgq", model.operation()); - Assertions.assertEquals("jrefovgmkqsle", model.resource()); - Assertions.assertEquals("vxyqjpkcattpngjc", model.description()); + Assertions.assertEquals("gual", model.provider()); + Assertions.assertEquals("xxhejjzzvd", model.operation()); + Assertions.assertEquals("gwdslfhotwm", model.resource()); + Assertions.assertEquals("npwlbjnpg", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationDisplay model = new OperationDisplay().withProvider("mdectehfiqscjey").withOperation("hezrkgq") - .withResource("jrefovgmkqsle").withDescription("vxyqjpkcattpngjc"); + OperationDisplay model = new OperationDisplay().withProvider("gual").withOperation("xxhejjzzvd") + .withResource("gwdslfhotwm").withDescription("npwlbjnpg"); model = BinaryData.fromObject(model).toObject(OperationDisplay.class); - Assertions.assertEquals("mdectehfiqscjey", model.provider()); - Assertions.assertEquals("hezrkgq", model.operation()); - Assertions.assertEquals("jrefovgmkqsle", model.resource()); - Assertions.assertEquals("vxyqjpkcattpngjc", model.description()); + Assertions.assertEquals("gual", model.provider()); + Assertions.assertEquals("xxhejjzzvd", model.operation()); + Assertions.assertEquals("gwdslfhotwm", model.resource()); + Assertions.assertEquals("npwlbjnpg", model.description()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationInnerTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationInnerTests.java index 5f6d13fed664c..ba154f1214331 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationInnerTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationInnerTests.java @@ -13,30 +13,30 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OperationInner model = BinaryData.fromString( - "{\"name\":\"dnvowg\",\"isDataAction\":true,\"display\":{\"provider\":\"wdkcglhsl\",\"operation\":\"jdyggdtji\",\"resource\":\"b\",\"description\":\"ofqweykhmenevfye\"},\"origin\":\"whybcib\",\"properties\":\"datavdcsitynn\"}") + "{\"name\":\"lnwsubisn\",\"isDataAction\":true,\"display\":{\"provider\":\"ngnzscxaqwoochc\",\"operation\":\"nqvpkvlrxnje\",\"resource\":\"eipheoflokeyy\",\"description\":\"nj\"},\"origin\":\"lwtgrhpdj\",\"properties\":\"dataumasxazjpq\"}") .toObject(OperationInner.class); - Assertions.assertEquals("dnvowg", model.name()); + Assertions.assertEquals("lnwsubisn", model.name()); Assertions.assertEquals(true, model.isDataAction()); - Assertions.assertEquals("wdkcglhsl", model.display().provider()); - Assertions.assertEquals("jdyggdtji", model.display().operation()); - Assertions.assertEquals("b", model.display().resource()); - Assertions.assertEquals("ofqweykhmenevfye", model.display().description()); - Assertions.assertEquals("whybcib", model.origin()); + Assertions.assertEquals("ngnzscxaqwoochc", model.display().provider()); + Assertions.assertEquals("nqvpkvlrxnje", model.display().operation()); + Assertions.assertEquals("eipheoflokeyy", model.display().resource()); + Assertions.assertEquals("nj", model.display().description()); + Assertions.assertEquals("lwtgrhpdj", model.origin()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationInner model = new OperationInner() - .withName("dnvowg").withIsDataAction(true).withDisplay(new OperationDisplay().withProvider("wdkcglhsl") - .withOperation("jdyggdtji").withResource("b").withDescription("ofqweykhmenevfye")) - .withOrigin("whybcib").withProperties("datavdcsitynn"); + OperationInner model = new OperationInner().withName("lnwsubisn").withIsDataAction(true) + .withDisplay(new OperationDisplay().withProvider("ngnzscxaqwoochc").withOperation("nqvpkvlrxnje") + .withResource("eipheoflokeyy").withDescription("nj")) + .withOrigin("lwtgrhpdj").withProperties("dataumasxazjpq"); model = BinaryData.fromObject(model).toObject(OperationInner.class); - Assertions.assertEquals("dnvowg", model.name()); + Assertions.assertEquals("lnwsubisn", model.name()); Assertions.assertEquals(true, model.isDataAction()); - Assertions.assertEquals("wdkcglhsl", model.display().provider()); - Assertions.assertEquals("jdyggdtji", model.display().operation()); - Assertions.assertEquals("b", model.display().resource()); - Assertions.assertEquals("ofqweykhmenevfye", model.display().description()); - Assertions.assertEquals("whybcib", model.origin()); + Assertions.assertEquals("ngnzscxaqwoochc", model.display().provider()); + Assertions.assertEquals("nqvpkvlrxnje", model.display().operation()); + Assertions.assertEquals("eipheoflokeyy", model.display().resource()); + Assertions.assertEquals("nj", model.display().description()); + Assertions.assertEquals("lwtgrhpdj", model.origin()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationListResultTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationListResultTests.java index 62316fe5b9800..108565ab2a0dd 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationListResultTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationListResultTests.java @@ -15,46 +15,34 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OperationListResult model = BinaryData.fromString( - "{\"value\":[{\"name\":\"ugnxkrxdqmi\",\"isDataAction\":false,\"display\":{\"provider\":\"rvqdra\",\"operation\":\"jybige\",\"resource\":\"qfbow\",\"description\":\"anyktzlcuiywg\"},\"origin\":\"wgndrvynhzgpp\",\"properties\":\"datacgyncocpecf\"},{\"name\":\"mcoo\",\"isDataAction\":true,\"display\":{\"provider\":\"evgbmqjq\",\"operation\":\"c\",\"resource\":\"mivkwlzuvcc\",\"description\":\"nfnbacfionlebxe\"},\"origin\":\"gtzxdpn\",\"properties\":\"dataqqwx\"},{\"name\":\"feallnwsu\",\"isDataAction\":true,\"display\":{\"provider\":\"ampmngnz\",\"operation\":\"xaqwoochcbonqv\",\"resource\":\"vlrxnjeaseiph\",\"description\":\"f\"},\"origin\":\"keyyi\",\"properties\":\"datajbdlwtgrhpdjpju\"},{\"name\":\"sxazjpq\",\"isDataAction\":false,\"display\":{\"provider\":\"lhbxxhejjzzvdud\",\"operation\":\"dslfhotwmcy\",\"resource\":\"wlbjnpgacftade\",\"description\":\"nltyfsoppusuesnz\"},\"origin\":\"ej\",\"properties\":\"datavorxzdmohct\"}],\"nextLink\":\"vudwx\"}") + "{\"value\":[{\"name\":\"ocpecfvmmco\",\"isDataAction\":true,\"display\":{\"provider\":\"zevgb\",\"operation\":\"jqabcypmivkwlzuv\",\"resource\":\"fwnfnb\",\"description\":\"fionl\"},\"origin\":\"x\",\"properties\":\"dataqgtz\"}],\"nextLink\":\"pnqbqqwxrjfe\"}") .toObject(OperationListResult.class); - Assertions.assertEquals("ugnxkrxdqmi", model.value().get(0).name()); - Assertions.assertEquals(false, model.value().get(0).isDataAction()); - Assertions.assertEquals("rvqdra", model.value().get(0).display().provider()); - Assertions.assertEquals("jybige", model.value().get(0).display().operation()); - Assertions.assertEquals("qfbow", model.value().get(0).display().resource()); - Assertions.assertEquals("anyktzlcuiywg", model.value().get(0).display().description()); - Assertions.assertEquals("wgndrvynhzgpp", model.value().get(0).origin()); - Assertions.assertEquals("vudwx", model.nextLink()); + Assertions.assertEquals("ocpecfvmmco", model.value().get(0).name()); + Assertions.assertEquals(true, model.value().get(0).isDataAction()); + Assertions.assertEquals("zevgb", model.value().get(0).display().provider()); + Assertions.assertEquals("jqabcypmivkwlzuv", model.value().get(0).display().operation()); + Assertions.assertEquals("fwnfnb", model.value().get(0).display().resource()); + Assertions.assertEquals("fionl", model.value().get(0).display().description()); + Assertions.assertEquals("x", model.value().get(0).origin()); + Assertions.assertEquals("pnqbqqwxrjfe", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationListResult model = new OperationListResult().withValue(Arrays.asList( - new OperationInner().withName("ugnxkrxdqmi").withIsDataAction(false) - .withDisplay(new OperationDisplay().withProvider("rvqdra").withOperation("jybige").withResource("qfbow") - .withDescription("anyktzlcuiywg")) - .withOrigin("wgndrvynhzgpp").withProperties("datacgyncocpecf"), - new OperationInner().withName("mcoo").withIsDataAction(true) - .withDisplay(new OperationDisplay().withProvider("evgbmqjq").withOperation("c") - .withResource("mivkwlzuvcc").withDescription("nfnbacfionlebxe")) - .withOrigin("gtzxdpn").withProperties("dataqqwx"), - new OperationInner().withName("feallnwsu").withIsDataAction(true) - .withDisplay(new OperationDisplay().withProvider("ampmngnz").withOperation("xaqwoochcbonqv") - .withResource("vlrxnjeaseiph").withDescription("f")) - .withOrigin("keyyi").withProperties("datajbdlwtgrhpdjpju"), - new OperationInner().withName("sxazjpq").withIsDataAction(false) - .withDisplay(new OperationDisplay().withProvider("lhbxxhejjzzvdud").withOperation("dslfhotwmcy") - .withResource("wlbjnpgacftade").withDescription("nltyfsoppusuesnz")) - .withOrigin("ej").withProperties("datavorxzdmohct"))) - .withNextLink("vudwx"); + OperationListResult model = new OperationListResult() + .withValue(Arrays.asList(new OperationInner().withName("ocpecfvmmco").withIsDataAction(true) + .withDisplay(new OperationDisplay().withProvider("zevgb").withOperation("jqabcypmivkwlzuv") + .withResource("fwnfnb").withDescription("fionl")) + .withOrigin("x").withProperties("dataqgtz"))) + .withNextLink("pnqbqqwxrjfe"); model = BinaryData.fromObject(model).toObject(OperationListResult.class); - Assertions.assertEquals("ugnxkrxdqmi", model.value().get(0).name()); - Assertions.assertEquals(false, model.value().get(0).isDataAction()); - Assertions.assertEquals("rvqdra", model.value().get(0).display().provider()); - Assertions.assertEquals("jybige", model.value().get(0).display().operation()); - Assertions.assertEquals("qfbow", model.value().get(0).display().resource()); - Assertions.assertEquals("anyktzlcuiywg", model.value().get(0).display().description()); - Assertions.assertEquals("wgndrvynhzgpp", model.value().get(0).origin()); - Assertions.assertEquals("vudwx", model.nextLink()); + Assertions.assertEquals("ocpecfvmmco", model.value().get(0).name()); + Assertions.assertEquals(true, model.value().get(0).isDataAction()); + Assertions.assertEquals("zevgb", model.value().get(0).display().provider()); + Assertions.assertEquals("jqabcypmivkwlzuv", model.value().get(0).display().operation()); + Assertions.assertEquals("fwnfnb", model.value().get(0).display().resource()); + Assertions.assertEquals("fionl", model.value().get(0).display().description()); + Assertions.assertEquals("x", model.value().get(0).origin()); + Assertions.assertEquals("pnqbqqwxrjfe", model.nextLink()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationsListMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationsListMockTests.java index 59b231c9b1618..6264844b95aa4 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationsListMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OperationsListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"utpwoqhihejqgw\",\"isDataAction\":false,\"display\":{\"provider\":\"n\",\"operation\":\"ypsxjvfoim\",\"resource\":\"slirciz\",\"description\":\"vydfceacvlhvygdy\"},\"origin\":\"umrtwnawjsl\",\"properties\":\"datawkojgcyztsfmzn\"}]}"; + = "{\"value\":[{\"name\":\"yzirtxdyuxzejn\",\"isDataAction\":true,\"display\":{\"provider\":\"gioilqu\",\"operation\":\"ydxtqm\",\"resource\":\"ox\",\"description\":\"ggufhyaomtb\"},\"origin\":\"havgrvk\",\"properties\":\"dataovjzhpjbibgjmfx\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,12 +51,12 @@ public void testList() throws Exception { PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("utpwoqhihejqgw", response.iterator().next().name()); - Assertions.assertEquals(false, response.iterator().next().isDataAction()); - Assertions.assertEquals("n", response.iterator().next().display().provider()); - Assertions.assertEquals("ypsxjvfoim", response.iterator().next().display().operation()); - Assertions.assertEquals("slirciz", response.iterator().next().display().resource()); - Assertions.assertEquals("vydfceacvlhvygdy", response.iterator().next().display().description()); - Assertions.assertEquals("umrtwnawjsl", response.iterator().next().origin()); + Assertions.assertEquals("yzirtxdyuxzejn", response.iterator().next().name()); + Assertions.assertEquals(true, response.iterator().next().isDataAction()); + Assertions.assertEquals("gioilqu", response.iterator().next().display().provider()); + Assertions.assertEquals("ydxtqm", response.iterator().next().display().operation()); + Assertions.assertEquals("ox", response.iterator().next().display().resource()); + Assertions.assertEquals("ggufhyaomtb", response.iterator().next().display().description()); + Assertions.assertEquals("havgrvk", response.iterator().next().origin()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointCollectionTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointCollectionTests.java index 92fc51b6847ba..f5969aea9151a 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointCollectionTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointCollectionTests.java @@ -12,16 +12,16 @@ public final class OutboundEnvironmentEndpointCollectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OutboundEnvironmentEndpointCollection model = BinaryData.fromString( - "{\"value\":[{\"category\":\"pfuvglsbjjca\",\"endpoints\":[{\"domainName\":\"vtvudutncormr\",\"description\":\"qtvcofudflvkgj\",\"endpointDetails\":[{},{}]},{\"domainName\":\"knnqvsaznq\",\"description\":\"orudsgsa\",\"endpointDetails\":[{},{},{}]}]},{\"category\":\"c\",\"endpoints\":[{\"domainName\":\"wjue\",\"description\":\"eburu\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"vsmzlxwab\",\"description\":\"oefki\",\"endpointDetails\":[{},{},{}]},{\"domainName\":\"puqujmqlgkfbtn\",\"description\":\"aongbj\",\"endpointDetails\":[{},{},{}]}]},{\"category\":\"jitcjedftwwaez\",\"endpoints\":[{\"domainName\":\"dcpzfoqo\",\"description\":\"cybxa\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"zuf\",\"description\":\"ciqopidoa\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"dhkha\",\"description\":\"khnzbonlw\",\"endpointDetails\":[{},{},{}]},{\"domainName\":\"gokdwbwhks\",\"description\":\"cmrvexzt\",\"endpointDetails\":[{}]}]},{\"category\":\"gsfraoyzkoow\",\"endpoints\":[{\"domainName\":\"guxawqaldsyuuxi\",\"description\":\"rqf\",\"endpointDetails\":[{},{},{}]},{\"domainName\":\"znkbykutwpfhpagm\",\"description\":\"skdsnfdsdoakg\",\"endpointDetails\":[{}]},{\"domainName\":\"kkze\",\"description\":\"l\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"usdsttwv\",\"description\":\"vbbejdcng\",\"endpointDetails\":[{}]}]}],\"nextLink\":\"akufgmjz\"}") + "{\"value\":[{\"category\":\"dsg\",\"endpoints\":[{\"domainName\":\"kycgrauwj\",\"description\":\"taeburuvdm\",\"endpointDetails\":[{}]}]},{\"category\":\"zlxwabmqoefkifr\",\"endpoints\":[{\"domainName\":\"qujmqlgkf\",\"description\":\"ndo\",\"endpointDetails\":[{}]},{\"domainName\":\"bjcntujitc\",\"description\":\"df\",\"endpointDetails\":[{},{},{}]},{\"domainName\":\"ezkojvdcp\",\"description\":\"oqouicybxarzgszu\",\"endpointDetails\":[{}]},{\"domainName\":\"iqopidoamciod\",\"description\":\"haz\",\"endpointDetails\":[{},{}]}]}],\"nextLink\":\"zbonlwnt\"}") .toObject(OutboundEnvironmentEndpointCollection.class); - Assertions.assertEquals("akufgmjz", model.nextLink()); + Assertions.assertEquals("zbonlwnt", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { OutboundEnvironmentEndpointCollection model - = new OutboundEnvironmentEndpointCollection().withNextLink("akufgmjz"); + = new OutboundEnvironmentEndpointCollection().withNextLink("zbonlwnt"); model = BinaryData.fromObject(model).toObject(OutboundEnvironmentEndpointCollection.class); - Assertions.assertEquals("akufgmjz", model.nextLink()); + Assertions.assertEquals("zbonlwnt", model.nextLink()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointInnerTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointInnerTests.java index e63bbcab93a69..2b4e8745ac7d7 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointInnerTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/OutboundEnvironmentEndpointInnerTests.java @@ -11,7 +11,7 @@ public final class OutboundEnvironmentEndpointInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OutboundEnvironmentEndpointInner model = BinaryData.fromString( - "{\"category\":\"rdgrtw\",\"endpoints\":[{\"domainName\":\"uzkopbminrfd\",\"description\":\"yuhhziu\",\"endpointDetails\":[{\"port\":900012181},{\"port\":1689863270},{\"port\":1792601934},{\"port\":962807389}]},{\"domainName\":\"mzqhoftrmaequi\",\"description\":\"xicslfao\",\"endpointDetails\":[{\"port\":1413149705},{\"port\":2064945633},{\"port\":980018563},{\"port\":1404219514}]},{\"domainName\":\"whccs\",\"description\":\"kaivwit\",\"endpointDetails\":[{\"port\":2016722653},{\"port\":494883497},{\"port\":1839231877}]}]}") + "{\"category\":\"gokdwbwhks\",\"endpoints\":[{\"domainName\":\"rvexztvb\",\"description\":\"gsfraoyzkoow\",\"endpointDetails\":[{\"port\":1663606623},{\"port\":1523878268},{\"port\":1953532777},{\"port\":1275486297}]},{\"domainName\":\"dsyuuximerqfob\",\"description\":\"znkbykutwpfhpagm\",\"endpointDetails\":[{\"port\":1595958724},{\"port\":803916344},{\"port\":1293750730},{\"port\":2086518628}]}]}") .toObject(OutboundEnvironmentEndpointInner.class); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PoolsDeleteMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PoolsDeleteMockTests.java index 8e5be5b0d7822..567d056e865f9 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PoolsDeleteMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PoolsDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.pools().delete("rriloz", "peewchpxlkt", "kuziycsle", com.azure.core.util.Context.NONE); + manager.pools().delete("yhddvia", "egfnmntfpmvmemfn", "zdwvvbalxl", com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java index e98e40b610aff..a343bfe80d35a 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Updating\",\"privateEndpoint\":{\"id\":\"vcimpev\"},\"groupIds\":[\"b\",\"rrilbywdxsmic\",\"wrwfscjfnyns\"],\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"i\",\"actionsRequired\":\"voqyt\"}},\"etag\":\"yo\",\"id\":\"bblgyavut\",\"name\":\"thjoxoism\",\"type\":\"ksbpimlqoljx\"}"; + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"privateEndpoint\":{\"id\":\"n\"},\"groupIds\":[\"ewdjcvbquwrb\"],\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"gohbuffkmrq\",\"actionsRequired\":\"vvhmxtdrj\"}},\"etag\":\"tac\",\"id\":\"ebjvewzcjzn\",\"name\":\"wcpmguaadraufac\",\"type\":\"kahzo\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,10 +50,10 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PrivateEndpointConnection response = manager.privateEndpointConnections() - .getWithResponse("cubiipuipw", "qonmacj", "k", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("qybaryeua", "jkqa", "qgzsles", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(PrivateLinkServiceConnectionStatus.APPROVED, + Assertions.assertEquals(PrivateLinkServiceConnectionStatus.REJECTED, response.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("i", response.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("gohbuffkmrq", response.privateLinkServiceConnectionState().description()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsListByBatchAccountMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsListByBatchAccountMockTests.java index a6287a54012aa..43782611eb8f1 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsListByBatchAccountMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsListByBatchAccountMockTests.java @@ -33,7 +33,7 @@ public void testListByBatchAccount() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"privateEndpoint\":{\"id\":\"rsukokwbqplh\"},\"groupIds\":[\"uuepzlrphwzsoldw\",\"yuqdu\",\"vmnnrw\"],\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"rk\",\"actionsRequired\":\"lywjhh\"}},\"etag\":\"nhxmsi\",\"id\":\"fomiloxgg\",\"name\":\"ufiqndieuzaof\",\"type\":\"chvcyyysfgdo\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Updating\",\"privateEndpoint\":{\"id\":\"stvdxeclz\"},\"groupIds\":[\"bcvhzlhpl\",\"dqkdlwwqfbu\",\"lkxt\"],\"privateLinkServiceConnectionState\":{\"status\":\"Disconnected\",\"description\":\"smlmbtxhwgfwsrta\",\"actionsRequired\":\"oezbrhubsk\"}},\"etag\":\"dyg\",\"id\":\"ookk\",\"name\":\"fqjbvleo\",\"type\":\"fmluiqtqzfavyvn\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,10 +51,11 @@ public void testListByBatchAccount() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.privateEndpointConnections() - .listByBatchAccount("epn", "bjcrxgibbdaxco", 456255387, com.azure.core.util.Context.NONE); + .listByBatchAccount("uudl", "zibt", 1621588970, com.azure.core.util.Context.NONE); - Assertions.assertEquals(PrivateLinkServiceConnectionStatus.APPROVED, + Assertions.assertEquals(PrivateLinkServiceConnectionStatus.DISCONNECTED, response.iterator().next().privateLinkServiceConnectionState().status()); - Assertions.assertEquals("rk", response.iterator().next().privateLinkServiceConnectionState().description()); + Assertions.assertEquals("smlmbtxhwgfwsrta", + response.iterator().next().privateLinkServiceConnectionState().description()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsUpdateMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsUpdateMockTests.java index 42978594b213b..ce1e7c7853e62 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsUpdateMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateEndpointConnectionsUpdateMockTests.java @@ -34,7 +34,7 @@ public void testUpdate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"privateEndpoint\":{\"id\":\"uvscxkdmligov\"},\"groupIds\":[\"xk\",\"mloazuru\",\"cbgoor\",\"te\"],\"privateLinkServiceConnectionState\":{\"status\":\"Disconnected\",\"description\":\"hjxa\",\"actionsRequired\":\"vjgsl\"}},\"etag\":\"dilmyww\",\"id\":\"kgkxn\",\"name\":\"edabgyvudtjue\",\"type\":\"bcihxuuwhc\"}"; + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"privateEndpoint\":{\"id\":\"dnhxmsi\"},\"groupIds\":[\"miloxggdufiqndie\"],\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"fjchvc\",\"actionsRequired\":\"ys\"}},\"etag\":\"dotcubiipuip\",\"id\":\"oqonma\",\"name\":\"jeknizshq\",\"type\":\"cimpevfg\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,14 +52,14 @@ public void testUpdate() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PrivateEndpointConnection response - = manager.privateEndpointConnections().update("cgxxlxs", "fgcviz", "zdwlvwlyoupfgfb", + = manager.privateEndpointConnections().update("ajjziuxxpshne", "kulfg", "lqubkwdlen", new PrivateEndpointConnectionInner().withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState().withStatus(PrivateLinkServiceConnectionStatus.APPROVED) - .withDescription("tg")), - "atnwxyiopi", com.azure.core.util.Context.NONE); + new PrivateLinkServiceConnectionState().withStatus(PrivateLinkServiceConnectionStatus.DISCONNECTED) + .withDescription("rxgibbd")), + "biorktal", com.azure.core.util.Context.NONE); - Assertions.assertEquals(PrivateLinkServiceConnectionStatus.DISCONNECTED, + Assertions.assertEquals(PrivateLinkServiceConnectionStatus.APPROVED, response.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("hjxa", response.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("fjchvc", response.privateLinkServiceConnectionState().description()); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesGetWithResponseMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesGetWithResponseMockTests.java index 9c5c7cf63c121..dbb842d5d4055 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesGetWithResponseMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesGetWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"groupId\":\"m\",\"requiredMembers\":[\"mguaadraufa\",\"tkahzo\",\"ajjziuxxpshne\",\"kulfg\"],\"requiredZoneNames\":[\"ubkwdle\",\"rds\"]},\"etag\":\"ujbazpjuohminyfl\",\"id\":\"orwmduvwpklv\",\"name\":\"w\",\"type\":\"ygdxpgpqchis\"}"; + = "{\"properties\":{\"groupId\":\"rvkwc\",\"requiredMembers\":[\"ljyxgtczhe\"],\"requiredZoneNames\":[\"sdshmkxmaehvb\",\"xu\",\"iplt\",\"n\"]},\"etag\":\"baxk\",\"id\":\"xywr\",\"name\":\"kpyklyhp\",\"type\":\"uodpv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,7 +48,7 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PrivateLinkResource response = manager.privateLinkResources() - .getWithResponse("uffkmrqemvvh", "xtdr", "futacoebjvewzc", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("ejwcwwqiok", "ssxmojms", "p", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesListByBatchAccountMockTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesListByBatchAccountMockTests.java index 4d68a1cabacf3..360a3f7a0a86d 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesListByBatchAccountMockTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/PrivateLinkResourcesListByBatchAccountMockTests.java @@ -31,7 +31,7 @@ public void testListByBatchAccount() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"groupId\":\"oookkqfq\",\"requiredMembers\":[\"leorfmluiqtqz\"],\"requiredZoneNames\":[\"yvnqqybaryeuay\",\"kq\"]},\"etag\":\"qgzsles\",\"id\":\"cbhernntiewdj\",\"name\":\"vbquwr\",\"type\":\"ehwagoh\"}]}"; + = "{\"value\":[{\"properties\":{\"groupId\":\"ifmviklbydvk\",\"requiredMembers\":[\"jdz\"],\"requiredZoneNames\":[\"vdsrhnjiv\",\"lvtno\"]},\"etag\":\"fzg\",\"id\":\"mjdftu\",\"name\":\"jltduceam\",\"type\":\"mczuo\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,8 @@ public void testListByBatchAccount() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.privateLinkResources().listByBatchAccount("a", - "coezbrhubskh", 547871572, com.azure.core.util.Context.NONE); + PagedIterable response = manager.privateLinkResources().listByBatchAccount("ztsfmznbaeqp", + "chqnrnrpxehuwry", 1484203808, com.azure.core.util.Context.NONE); } } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/RollingUpgradePolicyTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/RollingUpgradePolicyTests.java new file mode 100644 index 0000000000000..ac2503318ce7a --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/RollingUpgradePolicyTests.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.batch.models.RollingUpgradePolicy; +import org.junit.jupiter.api.Assertions; + +public final class RollingUpgradePolicyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RollingUpgradePolicy model = BinaryData.fromString( + "{\"enableCrossZoneUpgrade\":true,\"maxBatchInstancePercent\":332964812,\"maxUnhealthyInstancePercent\":969914681,\"maxUnhealthyUpgradedInstancePercent\":1059529277,\"pauseTimeBetweenBatches\":\"zpof\",\"prioritizeUnhealthyInstances\":false,\"rollbackFailedInstancesOnPolicyBreach\":true}") + .toObject(RollingUpgradePolicy.class); + Assertions.assertEquals(true, model.enableCrossZoneUpgrade()); + Assertions.assertEquals(332964812, model.maxBatchInstancePercent()); + Assertions.assertEquals(969914681, model.maxUnhealthyInstancePercent()); + Assertions.assertEquals(1059529277, model.maxUnhealthyUpgradedInstancePercent()); + Assertions.assertEquals("zpof", model.pauseTimeBetweenBatches()); + Assertions.assertEquals(false, model.prioritizeUnhealthyInstances()); + Assertions.assertEquals(true, model.rollbackFailedInstancesOnPolicyBreach()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RollingUpgradePolicy model = new RollingUpgradePolicy().withEnableCrossZoneUpgrade(true) + .withMaxBatchInstancePercent(332964812).withMaxUnhealthyInstancePercent(969914681) + .withMaxUnhealthyUpgradedInstancePercent(1059529277).withPauseTimeBetweenBatches("zpof") + .withPrioritizeUnhealthyInstances(false).withRollbackFailedInstancesOnPolicyBreach(true); + model = BinaryData.fromObject(model).toObject(RollingUpgradePolicy.class); + Assertions.assertEquals(true, model.enableCrossZoneUpgrade()); + Assertions.assertEquals(332964812, model.maxBatchInstancePercent()); + Assertions.assertEquals(969914681, model.maxUnhealthyInstancePercent()); + Assertions.assertEquals(1059529277, model.maxUnhealthyUpgradedInstancePercent()); + Assertions.assertEquals("zpof", model.pauseTimeBetweenBatches()); + Assertions.assertEquals(false, model.prioritizeUnhealthyInstances()); + Assertions.assertEquals(true, model.rollbackFailedInstancesOnPolicyBreach()); + } +} diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SkuCapabilityTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SkuCapabilityTests.java index 224265044527e..dba08f6cb0379 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SkuCapabilityTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SkuCapabilityTests.java @@ -11,7 +11,7 @@ public final class SkuCapabilityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SkuCapability model - = BinaryData.fromString("{\"name\":\"pgvdf\",\"value\":\"otkftutqxlngx\"}").toObject(SkuCapability.class); + = BinaryData.fromString("{\"name\":\"ndrvynhzg\",\"value\":\"hrc\"}").toObject(SkuCapability.class); } @org.junit.jupiter.api.Test diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkuInnerTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkuInnerTests.java index 71036049aab27..a0258e638e234 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkuInnerTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkuInnerTests.java @@ -11,7 +11,7 @@ public final class SupportedSkuInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SupportedSkuInner model = BinaryData.fromString( - "{\"name\":\"uhrhcffcyddgl\",\"familyName\":\"t\",\"capabilities\":[{\"name\":\"wpyeicxmqciwqvh\",\"value\":\"ixuigdtopbobj\"},{\"name\":\"hm\",\"value\":\"u\"},{\"name\":\"a\",\"value\":\"rzayv\"}]}") + "{\"name\":\"w\",\"familyName\":\"m\",\"capabilities\":[{\"name\":\"z\",\"value\":\"vvtpgvdfgio\"},{\"name\":\"ftutqxlngxlefgu\",\"value\":\"xkrxdqmi\"},{\"name\":\"thz\",\"value\":\"qdrabhjybigehoqf\"},{\"name\":\"wska\",\"value\":\"ktzlcuiywg\"}],\"batchSupportEndOfLife\":\"2021-06-17T20:14:54Z\"}") .toObject(SupportedSkuInner.class); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkusResultTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkusResultTests.java index 6aad0713b4623..89488781a8b51 100644 --- a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkusResultTests.java +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/SupportedSkusResultTests.java @@ -13,7 +13,7 @@ public final class SupportedSkusResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SupportedSkusResult model = BinaryData.fromString( - "{\"value\":[{\"name\":\"urzafb\",\"familyName\":\"j\",\"capabilities\":[{\"name\":\"oq\",\"value\":\"mkljavb\"},{\"name\":\"dtqajzyulpkudj\",\"value\":\"lkhbz\"},{\"name\":\"epgzgqexz\",\"value\":\"c\"},{\"name\":\"c\",\"value\":\"ierhhbcsglummaj\"}]},{\"name\":\"aodxo\",\"familyName\":\"bdxkqpxokaj\",\"capabilities\":[{\"name\":\"imexgstxgcpodgma\",\"value\":\"r\"},{\"name\":\"djwzrlov\",\"value\":\"lwhijcoejctbzaq\"}]},{\"name\":\"sycbkbfk\",\"familyName\":\"kdkexxp\",\"capabilities\":[{\"name\":\"xaxcfjpgddtocjjx\",\"value\":\"pmouexhdz\"}]},{\"name\":\"bqe\",\"familyName\":\"nxqbzvddn\",\"capabilities\":[{\"name\":\"eic\",\"value\":\"w\"}]}],\"nextLink\":\"zao\"}") + "{\"value\":[{\"name\":\"urzafb\",\"familyName\":\"j\",\"capabilities\":[{\"name\":\"oq\",\"value\":\"mkljavb\"},{\"name\":\"dtqajzyulpkudj\",\"value\":\"lkhbz\"},{\"name\":\"epgzgqexz\",\"value\":\"c\"},{\"name\":\"c\",\"value\":\"ierhhbcsglummaj\"}],\"batchSupportEndOfLife\":\"2021-08-25T02:16:19Z\"},{\"name\":\"dxob\",\"familyName\":\"dxkqpx\",\"capabilities\":[{\"name\":\"ionpimexg\",\"value\":\"xgcp\"},{\"name\":\"gmaajrm\",\"value\":\"jwzrl\"},{\"name\":\"mcl\",\"value\":\"ijcoejctb\"},{\"name\":\"qsqsy\",\"value\":\"kbfkg\"}],\"batchSupportEndOfLife\":\"2021-04-20T10:26:18Z\"},{\"name\":\"exxppofmxaxcfjp\",\"familyName\":\"dtocj\",\"capabilities\":[{\"name\":\"pmouexhdz\",\"value\":\"bqe\"},{\"name\":\"nxqbzvddn\",\"value\":\"ndei\"}],\"batchSupportEndOfLife\":\"2021-07-03T11:10:37Z\"},{\"name\":\"npzaoq\",\"familyName\":\"hrhcffcyddglmjth\",\"capabilities\":[{\"name\":\"pyeicxm\",\"value\":\"iwqvhkh\"}],\"batchSupportEndOfLife\":\"2021-04-08T23:48:15Z\"}],\"nextLink\":\"gdtopbobjogh\"}") .toObject(SupportedSkusResult.class); } diff --git a/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/UpgradePolicyTests.java b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/UpgradePolicyTests.java new file mode 100644 index 0000000000000..c80836a4bfec3 --- /dev/null +++ b/sdk/batch/azure-resourcemanager-batch/src/test/java/com/azure/resourcemanager/batch/generated/UpgradePolicyTests.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.batch.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.batch.models.AutomaticOSUpgradePolicy; +import com.azure.resourcemanager.batch.models.RollingUpgradePolicy; +import com.azure.resourcemanager.batch.models.UpgradeMode; +import com.azure.resourcemanager.batch.models.UpgradePolicy; +import org.junit.jupiter.api.Assertions; + +public final class UpgradePolicyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UpgradePolicy model = BinaryData.fromString( + "{\"mode\":\"manual\",\"automaticOSUpgradePolicy\":{\"disableAutomaticRollback\":false,\"enableAutomaticOSUpgrade\":false,\"useRollingUpgradePolicy\":true,\"osRollingUpgradeDeferral\":true},\"rollingUpgradePolicy\":{\"enableCrossZoneUpgrade\":true,\"maxBatchInstancePercent\":881779477,\"maxUnhealthyInstancePercent\":1460158080,\"maxUnhealthyUpgradedInstancePercent\":1973822753,\"pauseTimeBetweenBatches\":\"kjj\",\"prioritizeUnhealthyInstances\":true,\"rollbackFailedInstancesOnPolicyBreach\":true}}") + .toObject(UpgradePolicy.class); + Assertions.assertEquals(UpgradeMode.MANUAL, model.mode()); + Assertions.assertEquals(false, model.automaticOSUpgradePolicy().disableAutomaticRollback()); + Assertions.assertEquals(false, model.automaticOSUpgradePolicy().enableAutomaticOSUpgrade()); + Assertions.assertEquals(true, model.automaticOSUpgradePolicy().useRollingUpgradePolicy()); + Assertions.assertEquals(true, model.automaticOSUpgradePolicy().osRollingUpgradeDeferral()); + Assertions.assertEquals(true, model.rollingUpgradePolicy().enableCrossZoneUpgrade()); + Assertions.assertEquals(881779477, model.rollingUpgradePolicy().maxBatchInstancePercent()); + Assertions.assertEquals(1460158080, model.rollingUpgradePolicy().maxUnhealthyInstancePercent()); + Assertions.assertEquals(1973822753, model.rollingUpgradePolicy().maxUnhealthyUpgradedInstancePercent()); + Assertions.assertEquals("kjj", model.rollingUpgradePolicy().pauseTimeBetweenBatches()); + Assertions.assertEquals(true, model.rollingUpgradePolicy().prioritizeUnhealthyInstances()); + Assertions.assertEquals(true, model.rollingUpgradePolicy().rollbackFailedInstancesOnPolicyBreach()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UpgradePolicy model + = new UpgradePolicy().withMode(UpgradeMode.MANUAL) + .withAutomaticOSUpgradePolicy(new AutomaticOSUpgradePolicy().withDisableAutomaticRollback(false) + .withEnableAutomaticOSUpgrade(false).withUseRollingUpgradePolicy(true) + .withOsRollingUpgradeDeferral(true)) + .withRollingUpgradePolicy(new RollingUpgradePolicy().withEnableCrossZoneUpgrade(true) + .withMaxBatchInstancePercent(881779477).withMaxUnhealthyInstancePercent(1460158080) + .withMaxUnhealthyUpgradedInstancePercent(1973822753).withPauseTimeBetweenBatches("kjj") + .withPrioritizeUnhealthyInstances(true).withRollbackFailedInstancesOnPolicyBreach(true)); + model = BinaryData.fromObject(model).toObject(UpgradePolicy.class); + Assertions.assertEquals(UpgradeMode.MANUAL, model.mode()); + Assertions.assertEquals(false, model.automaticOSUpgradePolicy().disableAutomaticRollback()); + Assertions.assertEquals(false, model.automaticOSUpgradePolicy().enableAutomaticOSUpgrade()); + Assertions.assertEquals(true, model.automaticOSUpgradePolicy().useRollingUpgradePolicy()); + Assertions.assertEquals(true, model.automaticOSUpgradePolicy().osRollingUpgradeDeferral()); + Assertions.assertEquals(true, model.rollingUpgradePolicy().enableCrossZoneUpgrade()); + Assertions.assertEquals(881779477, model.rollingUpgradePolicy().maxBatchInstancePercent()); + Assertions.assertEquals(1460158080, model.rollingUpgradePolicy().maxUnhealthyInstancePercent()); + Assertions.assertEquals(1973822753, model.rollingUpgradePolicy().maxUnhealthyUpgradedInstancePercent()); + Assertions.assertEquals("kjj", model.rollingUpgradePolicy().pauseTimeBetweenBatches()); + Assertions.assertEquals(true, model.rollingUpgradePolicy().prioritizeUnhealthyInstances()); + Assertions.assertEquals(true, model.rollingUpgradePolicy().rollbackFailedInstancesOnPolicyBreach()); + } +} From 38cf774a7accb8115c91ad2ce314db452f42daa2 Mon Sep 17 00:00:00 2001 From: "Hong Li(MSFT)" <74638143+v-hongli1@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:16:01 +0800 Subject: [PATCH 22/44] Fix DeleteOptions on Public Ip Address (issue#38806) (#39096) Fix DeleteOptions on Public Ip Address (issue#38806) --- .../CHANGELOG.md | 2 + .../azure-resourcemanager-network/assets.json | 2 +- .../implementation/NetworkInterfaceImpl.java | 23 +++++- .../NicIpConfigurationImpl.java | 21 +++++- .../network/models/NetworkInterface.java | 29 ++++++- .../network/models/NicIpConfiguration.java | 52 ++++++++++++- .../NetworkInterfaceOperationsTests.java | 75 +++++++++++++++++++ 7 files changed, 194 insertions(+), 10 deletions(-) diff --git a/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md index 350f7c5721d1e..519b878fdc796 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md @@ -4,6 +4,8 @@ ### Features Added +- Supported setting `DeleteOptions` for public IP addresses associated with `NetworkInterface`. + ### Breaking Changes ### Bugs Fixed diff --git a/sdk/resourcemanager/azure-resourcemanager-network/assets.json b/sdk/resourcemanager/azure-resourcemanager-network/assets.json index ccfb7e0098d77..daf9a4133b214 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/assets.json +++ b/sdk/resourcemanager/azure-resourcemanager-network/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/resourcemanager/azure-resourcemanager-network", - "Tag": "java/resourcemanager/azure-resourcemanager-network_271269e41a" + "Tag": "java/resourcemanager/azure-resourcemanager-network_03b4909e5e" } diff --git a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NetworkInterfaceImpl.java b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NetworkInterfaceImpl.java index 3b2d2c6e2f5ea..1a8793482863c 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NetworkInterfaceImpl.java +++ b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NetworkInterfaceImpl.java @@ -9,6 +9,7 @@ import com.azure.resourcemanager.network.NetworkManager; import com.azure.resourcemanager.network.fluent.models.ApplicationSecurityGroupInner; import com.azure.resourcemanager.network.models.ApplicationSecurityGroup; +import com.azure.resourcemanager.network.models.DeleteOptions; import com.azure.resourcemanager.network.models.IpAllocationMethod; import com.azure.resourcemanager.network.models.LoadBalancer; import com.azure.resourcemanager.network.models.Network; @@ -32,6 +33,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -60,11 +62,14 @@ class NetworkInterfaceImpl private NetworkSecurityGroup existingNetworkSecurityGroupToAssociate; /** cached related resources. */ private NetworkSecurityGroup networkSecurityGroup; + /** the name of specified ip config name */ + private Map specifiedIpConfigNames; NetworkInterfaceImpl(String name, NetworkInterfaceInner innerModel, final NetworkManager networkManager) { super(name, innerModel, networkManager); this.nicName = name; this.namer = this.manager().resourceManager().internalContext().createIdentifierProvider(this.nicName); + this.specifiedIpConfigNames = new HashMap(); initializeChildrenFromInner(); } @@ -563,9 +568,25 @@ protected void beforeCreating() { .withNetworkSecurityGroup(new NetworkSecurityGroupInner().withId(networkSecurityGroup.id())); } - NicIpConfigurationImpl.ensureConfigurations(this.nicIPConfigurations.values()); + NicIpConfigurationImpl.ensureConfigurations(this.nicIPConfigurations.values(), this.specifiedIpConfigNames); // Reset and update IP configs this.innerModel().withIpConfigurations(innersFromWrappers(this.nicIPConfigurations.values())); } + + @Override + public NetworkInterfaceImpl withPrimaryPublicIPAddressDeleteOptions(DeleteOptions deleteOptions) { + this.ensureDeleteOptions(deleteOptions, "primary"); + return this; + } + + @Override + public NetworkInterfaceImpl update() { + this.specifiedIpConfigNames = new HashMap(); + return super.update(); + } + + public void ensureDeleteOptions(DeleteOptions deleteOptions, String ipConfigName) { + this.specifiedIpConfigNames.put(ipConfigName, deleteOptions); + } } diff --git a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NicIpConfigurationImpl.java b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NicIpConfigurationImpl.java index aa05e3df0d123..ecca0e730cf00 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NicIpConfigurationImpl.java +++ b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/implementation/NicIpConfigurationImpl.java @@ -9,6 +9,7 @@ import com.azure.resourcemanager.network.models.ApplicationGateway; import com.azure.resourcemanager.network.models.ApplicationGatewayBackendAddressPool; import com.azure.resourcemanager.network.models.ApplicationSecurityGroup; +import com.azure.resourcemanager.network.models.DeleteOptions; import com.azure.resourcemanager.network.models.IpAllocationMethod; import com.azure.resourcemanager.network.models.IpVersion; import com.azure.resourcemanager.network.models.LoadBalancer; @@ -28,6 +29,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Objects; /** Implementation for NicIPConfiguration and its create and update interfaces. */ @@ -257,11 +259,11 @@ private List ensureInboundNatRules() { return natRefs; } - protected static void ensureConfigurations(Collection nicIPConfigurations) { + protected static void ensureConfigurations(Collection nicIPConfigurations, Map specifiedIpConfigNames) { for (NicIpConfiguration nicIPConfiguration : nicIPConfigurations) { NicIpConfigurationImpl config = (NicIpConfigurationImpl) nicIPConfiguration; config.innerModel().withSubnet(config.subnetToAssociate()); - config.innerModel().withPublicIpAddress(config.publicIPToAssociate()); + config.innerModel().withPublicIpAddress(config.publicIPToAssociate(specifiedIpConfigNames.getOrDefault(config.name(), null))); } } @@ -331,9 +333,10 @@ private SubnetInner subnetToAssociate() { * public IP in create fluent chain. In case of update chain, if withoutPublicIP(..) is not specified then existing * associated (if any) public IP will be returned. * + * @param deleteOptions what happens to the public IP address when the VM using it is deleted * @return public IP SubResource */ - private PublicIpAddressInner publicIPToAssociate() { + private PublicIpAddressInner publicIPToAssociate(DeleteOptions deleteOptions) { String pipId = null; if (this.removePrimaryPublicIPAssociation) { return null; @@ -344,8 +347,14 @@ private PublicIpAddressInner publicIPToAssociate() { } if (pipId != null) { + if (Objects.nonNull(deleteOptions)) { + return new PublicIpAddressInner().withId(pipId).withDeleteOption(deleteOptions); + } return new PublicIpAddressInner().withId(pipId); } else if (!this.isInCreateMode) { + if (Objects.nonNull(this.innerModel().publicIpAddress()) && Objects.nonNull(deleteOptions)) { + return this.innerModel().publicIpAddress().withDeleteOption(deleteOptions); + } return this.innerModel().publicIpAddress(); } else { return null; @@ -400,4 +409,10 @@ NicIpConfigurationImpl withoutApplicationSecurityGroup(String name) { } return this; } + + @Override + public NicIpConfigurationImpl withPublicIPAddressDeleteOptions(DeleteOptions deleteOptions) { + this.parent().ensureDeleteOptions(deleteOptions, this.innerModel().name()); + return this; + } } diff --git a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NetworkInterface.java b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NetworkInterface.java index 84f1a5cb888d8..e080c4b56b1b7 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NetworkInterface.java +++ b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NetworkInterface.java @@ -256,6 +256,17 @@ interface WithAcceleratedNetworking { WithCreate withAcceleratedNetworking(); } + /** The stage of the definition allowing to specify delete options for the public ip address. */ + interface WithPublicIPAddressDeleteOptions { + /** + * Sets delete options for public ip address. + * + * @param deleteOptions the delete options for primary network interfaces + * @return the next stage of the update + */ + WithCreate withPrimaryPublicIPAddressDeleteOptions(DeleteOptions deleteOptions); + } + /** * The stage of the network interface definition which contains all the minimum required inputs for the resource * to be created, but also allows for any other optional settings to be specified. @@ -268,7 +279,8 @@ interface WithCreate WithSecondaryIPConfiguration, WithAcceleratedNetworking, WithLoadBalancer, - WithApplicationSecurityGroup { + WithApplicationSecurityGroup, + WithPublicIPAddressDeleteOptions { /** * Enables IP forwarding in the network interface. * @@ -576,6 +588,18 @@ interface WithLoadBalancer { */ Update withoutLoadBalancerInboundNatRules(); } + + /** The stage of the network interface update allowing to specify delete options for the public ip address. */ + interface WithPublicIPAddressDeleteOptions { + + /** + * Sets delete options for public ip address. + * + * @param deleteOptions the delete options for primary network interfaces + * @return the next stage of the update + */ + Update withPrimaryPublicIPAddressDeleteOptions(DeleteOptions deleteOptions); + } } /** The template for an update operation, containing all the settings that can be modified. */ @@ -591,6 +615,7 @@ interface Update UpdateStages.WithIPConfiguration, UpdateStages.WithLoadBalancer, UpdateStages.WithAcceleratedNetworking, - UpdateStages.WithApplicationSecurityGroup { + UpdateStages.WithApplicationSecurityGroup, + UpdateStages.WithPublicIPAddressDeleteOptions { } } diff --git a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NicIpConfiguration.java b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NicIpConfiguration.java index 2441c4d31d57d..fb6dcb288b463 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NicIpConfiguration.java +++ b/sdk/resourcemanager/azure-resourcemanager-network/src/main/java/com/azure/resourcemanager/network/models/NicIpConfiguration.java @@ -188,6 +188,22 @@ WithAttach withExistingApplicationGatewayBackend( ApplicationGateway appGateway, String backendName); } + + /** The stage of the definition allowing to specify delete options for the public ip address. + * + * @param the stage of the parent network interface definition to return to after attaching this + * definition + * */ + interface WithPublicIPAddressDeleteOptions { + /** + * Sets delete options for public ip address. + * + * @param deleteOptions the delete options for primary network interfaces + * @return the next stage of the definition + */ + WithAttach withPublicIPAddressDeleteOptions(DeleteOptions deleteOptions); + } + /** * The final stage of network interface IP configuration. * @@ -201,7 +217,8 @@ interface WithAttach extends Attachable.InDefinition, WithPublicIPAddress, WithLoadBalancer, - WithApplicationGateway { + WithApplicationGateway, + WithPublicIPAddressDeleteOptions { } } @@ -372,6 +389,22 @@ WithAttach withExistingApplicationGatewayBackend( ApplicationGateway appGateway, String backendName); } + /** + * The stage of the definition allowing to specify delete options for the public ip address. + * + * @param the stage of the parent network interface update to return to after attaching this + * definition + * */ + interface WithPublicIPAddressDeleteOptions { + /** + * Sets delete options for public ip address. + * + * @param deleteOptions the delete options for primary network interfaces + * @return the next stage of the update + */ + WithAttach withPublicIPAddressDeleteOptions(DeleteOptions deleteOptions); + } + /** * The final stage of network interface IP configuration. * @@ -385,7 +418,8 @@ interface WithAttach extends Attachable.InUpdate, WithPublicIPAddress, WithLoadBalancer, - WithApplicationGateway { + WithApplicationGateway, + WithPublicIPAddressDeleteOptions { } } @@ -396,7 +430,8 @@ interface Update UpdateStages.WithPrivateIP, UpdateStages.WithPublicIPAddress, UpdateStages.WithLoadBalancer, - UpdateStages.WithApplicationGateway { + UpdateStages.WithApplicationGateway, + UpdateStages.WithPublicIPAddressDeleteOptions { } /** Grouping of network interface IP configuration update stages. */ @@ -486,5 +521,16 @@ interface WithApplicationGateway { */ Update withoutApplicationGatewayBackends(); } + + /** The stage of the network interface update allowing to specify delete options for the public ip address. */ + interface WithPublicIPAddressDeleteOptions { + /** + * Sets delete options for public ip address. + * + * @param deleteOptions the delete options for primary network interfaces + * @return the next stage of the update + */ + Update withPublicIPAddressDeleteOptions(DeleteOptions deleteOptions); + } } } diff --git a/sdk/resourcemanager/azure-resourcemanager-network/src/test/java/com/azure/resourcemanager/network/NetworkInterfaceOperationsTests.java b/sdk/resourcemanager/azure-resourcemanager-network/src/test/java/com/azure/resourcemanager/network/NetworkInterfaceOperationsTests.java index 4f74986fe237c..41523fe1a7d4d 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/src/test/java/com/azure/resourcemanager/network/NetworkInterfaceOperationsTests.java +++ b/sdk/resourcemanager/azure-resourcemanager-network/src/test/java/com/azure/resourcemanager/network/NetworkInterfaceOperationsTests.java @@ -6,6 +6,7 @@ import com.azure.core.management.Region; import com.azure.resourcemanager.network.fluent.models.NatGatewayInner; import com.azure.resourcemanager.network.models.ApplicationSecurityGroup; +import com.azure.resourcemanager.network.models.DeleteOptions; import com.azure.resourcemanager.network.models.NatGatewaySku; import com.azure.resourcemanager.network.models.NatGatewaySkuName; import com.azure.resourcemanager.network.models.Network; @@ -509,6 +510,80 @@ public void canAssociateNatGateway() { Assertions.assertEquals(gateway2.id(), subnet2.natGatewayId()); } + @Test + public void canCreateAndUpdateNicWithMultipleDeleteOptions() { + String subnetName = generateRandomResourceName("subnet-", 15); + resourceManager.resourceGroups().define(rgName).withRegion(Region.US_EAST).create(); + Network vnet = networkManager.networks() + .define(generateRandomResourceName("vnet-", 15)) + .withRegion(Region.US_EAST) + .withExistingResourceGroup(rgName) + .withAddressSpace("10.0.0.0/28") + .withSubnet(subnetName, "10.0.0.0/28") + .create(); + + NetworkInterface nic = networkManager.networkInterfaces() + .define(generateRandomResourceName("nic-", 15)) + .withRegion(Region.US_EAST) + .withExistingResourceGroup(rgName) + .withExistingPrimaryNetwork(vnet) + .withSubnet(subnetName) + .withPrimaryPrivateIPAddressDynamic() + .withNewPrimaryPublicIPAddress() + .withPrimaryPublicIPAddressDeleteOptions(DeleteOptions.DELETE) + .defineSecondaryIPConfiguration("secondary1") + .withExistingNetwork(vnet) + .withSubnet(subnetName) + .withPrivateIpAddressDynamic() + .withNewPublicIpAddress() + .withPublicIPAddressDeleteOptions(DeleteOptions.DETACH) + .attach() + .defineSecondaryIPConfiguration("secondary2") + .withExistingNetwork(vnet) + .withSubnet(subnetName) + .withPrivateIpAddressDynamic() + .withNewPublicIpAddress() + .withPublicIPAddressDeleteOptions(DeleteOptions.DETACH) + .attach() + .create(); + + nic.refresh(); + Assertions.assertEquals(DeleteOptions.DELETE, nic.primaryIPConfiguration().innerModel().publicIpAddress().deleteOption()); + Assertions.assertEquals(DeleteOptions.DETACH, nic.ipConfigurations().get("secondary1").innerModel().publicIpAddress().deleteOption()); + Assertions.assertEquals(DeleteOptions.DETACH, nic.ipConfigurations().get("secondary2").innerModel().publicIpAddress().deleteOption()); + + String existingPrimaryIpAddressId = nic.primaryIPConfiguration().publicIpAddressId(); + nic.update().withNewPrimaryPublicIPAddress().withPrimaryPublicIPAddressDeleteOptions(DeleteOptions.DETACH).apply(); + nic.refresh(); + Assertions.assertFalse(existingPrimaryIpAddressId.equalsIgnoreCase(nic.primaryIPConfiguration().publicIpAddressId())); + Assertions.assertEquals(DeleteOptions.DETACH, nic.primaryIPConfiguration().innerModel().publicIpAddress().deleteOption()); + + String existingSecondary1IpAddressId = nic.ipConfigurations().get("secondary1").publicIpAddressId(); + nic.update() + .withPrimaryPublicIPAddressDeleteOptions(DeleteOptions.DELETE) + .updateIPConfiguration("secondary1") + .withNewPublicIpAddress() + .withPublicIPAddressDeleteOptions(DeleteOptions.DELETE) + .parent() + .updateIPConfiguration("secondary2") + .withPublicIPAddressDeleteOptions(DeleteOptions.DELETE) + .parent() + .defineSecondaryIPConfiguration("secondary3") + .withExistingNetwork(vnet) + .withSubnet(subnetName) + .withPrivateIpAddressDynamic() + .withNewPublicIpAddress() + .withPublicIPAddressDeleteOptions(DeleteOptions.DELETE) + .attach() + .apply(); + nic.refresh(); + Assertions.assertFalse(existingSecondary1IpAddressId.equalsIgnoreCase(nic.ipConfigurations().get("secondary1").publicIpAddressId())); + Assertions.assertEquals(DeleteOptions.DELETE, nic.primaryIPConfiguration().innerModel().publicIpAddress().deleteOption()); + Assertions.assertEquals(DeleteOptions.DELETE, nic.ipConfigurations().get("secondary1").innerModel().publicIpAddress().deleteOption()); + Assertions.assertEquals(DeleteOptions.DELETE, nic.ipConfigurations().get("secondary2").innerModel().publicIpAddress().deleteOption()); + Assertions.assertEquals(DeleteOptions.DELETE, nic.ipConfigurations().get("secondary3").innerModel().publicIpAddress().deleteOption()); + } + private NatGatewayInner createNatGateway() { String natGatewayName = generateRandomResourceName("natgw", 10); return networkManager.serviceClient() From 7176df06036d2095521cb166b55884303d10fb0b Mon Sep 17 00:00:00 2001 From: Michael Zappe <84374786+MichaelZp0@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:00:01 +0100 Subject: [PATCH 23/44] Fix Azure Remote Rendering tests by adding now missing dependencies and changed conversion error messages (#39403) Fix Azure Remote Rendering tests by adding now missing dependencies and changed conversion error messages --- .../assets.json | 2 +- .../pom.xml | 22 +++++++++++++++++++ .../RemoteRenderingAsyncClientTest.java | 7 +++--- .../RemoteRenderingClientTest.java | 7 +++--- .../RemoteRenderingTestBase.java | 2 +- 5 files changed, 32 insertions(+), 8 deletions(-) diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/assets.json b/sdk/remoterendering/azure-mixedreality-remoterendering/assets.json index 248f3dbe9f3e8..97bec81035630 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/assets.json +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/remoterendering/azure-mixedreality-remoterendering", - "Tag": "java/remoterendering/azure-mixedreality-remoterendering_24a8dbb81a" + "Tag": "java/remoterendering/azure-mixedreality-remoterendering_2fbd5a57df" } diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml b/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml index 730dcb99022fd..bac6e2e6bbf08 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml @@ -52,6 +52,12 @@ 1.11.19 test
+ + com.azure + azure-core-http-vertx + 1.0.0-beta.16 + test + org.junit.jupiter junit-jupiter-api @@ -89,4 +95,20 @@ test + + + java12plus + + [12,) + + + + com.azure + azure-core-http-jdk-httpclient + 1.0.0-beta.11 + test + + + + diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingAsyncClientTest.java b/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingAsyncClientTest.java index 65a6fc7ac6d18..dec2e83187211 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingAsyncClientTest.java +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingAsyncClientTest.java @@ -146,9 +146,10 @@ public void failedConversionMissingAssetTest(HttpClient httpClient) { assertEquals(AssetConversionStatus.FAILED, conversion.getStatus()); assertNotNull(conversion.getError()); - // Invalid input provided. Check logs in output container for details. - assertTrue(conversion.getError().getMessage().toLowerCase(Locale.ROOT).contains("invalid input")); - assertTrue(conversion.getError().getMessage().toLowerCase(Locale.ROOT).contains("logs")); + assertEquals(conversion.getError().getCode(), "InputContainerError"); + // Message: "Could not find the asset file in the storage account. Please make sure all paths and names are correct and the file is uploaded to storage." + assertNotNull(conversion.getError().getMessage()); + assertTrue(conversion.getError().getMessage().toLowerCase(Locale.ROOT).contains("could not find the asset file in the storage account")); }) .verifyComplete(); } diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingClientTest.java b/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingClientTest.java index 3676fe54f862a..b37bab4df0118 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingClientTest.java +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingClientTest.java @@ -126,9 +126,10 @@ public void failedConversionMissingAssetTest(HttpClient httpClient) { assertEquals(AssetConversionStatus.FAILED, conversion.getStatus()); assertNotNull(conversion.getError()); - // Invalid input provided. Check logs in output container for details. - assertTrue(conversion.getError().getMessage().toLowerCase(Locale.ROOT).contains("invalid input")); - assertTrue(conversion.getError().getMessage().toLowerCase(Locale.ROOT).contains("logs")); + assertEquals(conversion.getError().getCode(), "InputContainerError"); + // Message: "Could not find the asset file in the storage account. Please make sure all paths and names are correct and the file is uploaded to storage." + assertNotNull(conversion.getError().getMessage()); + assertTrue(conversion.getError().getMessage().toLowerCase(Locale.ROOT).contains("could not find the asset file in the storage account")); } @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingTestBase.java b/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingTestBase.java index 2765aaaa29546..208a9329d6407 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingTestBase.java +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/src/test/java/com/azure/mixedreality/remoterendering/RemoteRenderingTestBase.java @@ -41,7 +41,7 @@ public class RemoteRenderingTestBase extends TestProxyTestBase { private final String serviceEndpoint = Configuration.getGlobalConfiguration().get("REMOTERENDERING_ARR_SERVICE_ENDPOINT"); // NOT REAL ACCOUNT DETAILS - private final String playbackAccountId = "f5b3e69f-1e1b-46a5-a718-aea58a7a0f8e"; + private final String playbackAccountId = "40831821-9a8b-4f81-b85f-018809a1f727"; private final String playbackAccountDomain = "mixedreality.azure.com"; private final String playbackAccountKey = "Sanitized"; private final String playbackStorageAccountName = "sdkTest"; From 3c31ba38fe98fa681bf4da0e265b0b87fd8f2195 Mon Sep 17 00:00:00 2001 From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com> Date: Wed, 27 Mar 2024 12:00:34 -0400 Subject: [PATCH 24/44] Increment package versions for batch releases (#39426) --- eng/versioning/version_client.txt | 2 +- sdk/batch/azure-resourcemanager-batch/CHANGELOG.md | 10 ++++++++++ sdk/batch/azure-resourcemanager-batch/pom.xml | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index e9583504ddd08..c08920def8487 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -355,7 +355,7 @@ com.azure.resourcemanager:azure-resourcemanager-marketplaceordering;1.0.0-beta.2 com.azure.resourcemanager:azure-resourcemanager-timeseriesinsights;1.0.0-beta.2;1.0.0-beta.3 com.azure.resourcemanager:azure-resourcemanager-streamanalytics;1.0.0-beta.3;1.0.0-beta.4 com.azure.resourcemanager:azure-resourcemanager-operationsmanagement;1.0.0-beta.2;1.0.0-beta.3 -com.azure.resourcemanager:azure-resourcemanager-batch;1.0.0;1.1.0-beta.4 +com.azure.resourcemanager:azure-resourcemanager-batch;1.0.0;1.1.0-beta.5 com.azure.resourcemanager:azure-resourcemanager-datalakeanalytics;1.0.0-beta.2;1.0.0-beta.3 com.azure.resourcemanager:azure-resourcemanager-datalakestore;1.0.0-beta.2;1.0.0-beta.3 com.azure.resourcemanager:azure-resourcemanager-iotcentral;1.0.0;1.1.0-beta.2 diff --git a/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md b/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md index f63e592fc833d..a27735d6a0d19 100644 --- a/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md +++ b/sdk/batch/azure-resourcemanager-batch/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 1.1.0-beta.5 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 1.1.0-beta.4 (2024-03-27) - Azure Resource Manager Batch client library for Java. This package contains Microsoft Azure SDK for Batch Management SDK. Batch Client. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/batch/azure-resourcemanager-batch/pom.xml b/sdk/batch/azure-resourcemanager-batch/pom.xml index 2eda87cd4a30f..483f97dd1cbc5 100644 --- a/sdk/batch/azure-resourcemanager-batch/pom.xml +++ b/sdk/batch/azure-resourcemanager-batch/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-batch - 1.1.0-beta.4 + 1.1.0-beta.5 jar Microsoft Azure SDK for Batch Management From c8b985377cdca6c0998bd7b0a0f715f37c036ce8 Mon Sep 17 00:00:00 2001 From: Scott Addie <10702007+scottaddie@users.noreply.github.com> Date: Wed, 27 Mar 2024 12:38:13 -0500 Subject: [PATCH 25/44] Update Azure AD Learn links in Identity (#39405) --- sdk/identity/azure-identity/README.md | 24 ++++++++--------- .../azure-identity/TROUBLESHOOTING.md | 26 +++++++++---------- .../identity/AadCredentialBuilderBase.java | 2 +- .../identity/AuthorizationCodeCredential.java | 2 +- .../AuthorizationCodeCredentialBuilder.java | 2 +- .../azure/identity/AzureCliCredential.java | 2 +- .../identity/AzureCliCredentialBuilder.java | 2 +- .../identity/AzureDeveloperCliCredential.java | 2 +- .../AzureDeveloperCliCredentialBuilder.java | 2 +- .../identity/AzurePowerShellCredential.java | 2 +- .../AzurePowerShellCredentialBuilder.java | 2 +- .../identity/ClientAssertionCredential.java | 2 +- .../ClientAssertionCredentialBuilder.java | 2 +- .../identity/ClientCertificateCredential.java | 4 +-- .../ClientCertificateCredentialBuilder.java | 4 +-- .../identity/ClientSecretCredential.java | 4 +-- .../ClientSecretCredentialBuilder.java | 4 +-- .../identity/DefaultAzureCredential.java | 2 +- .../DefaultAzureCredentialBuilder.java | 2 +- .../azure/identity/DeviceCodeCredential.java | 2 +- .../identity/DeviceCodeCredentialBuilder.java | 2 +- .../azure/identity/IntelliJCredential.java | 2 +- .../identity/IntelliJCredentialBuilder.java | 2 +- .../InteractiveBrowserCredential.java | 2 +- .../InteractiveBrowserCredentialBuilder.java | 2 +- .../identity/ManagedIdentityCredential.java | 6 ++--- .../ManagedIdentityCredentialBuilder.java | 6 ++--- .../identity/UsernamePasswordCredential.java | 2 +- .../UsernamePasswordCredentialBuilder.java | 2 +- .../java/com/azure/identity/package-info.java | 12 ++++----- 30 files changed, 66 insertions(+), 66 deletions(-) diff --git a/sdk/identity/azure-identity/README.md b/sdk/identity/azure-identity/README.md index 660aaa8ef471e..3a98cbe29b878 100644 --- a/sdk/identity/azure-identity/README.md +++ b/sdk/identity/azure-identity/README.md @@ -1,6 +1,6 @@ # Azure Identity client library for Java -The Azure Identity library provides [Microsoft Entra ID](https://learn.microsoft.com/azure/active-directory/fundamentals/active-directory-whatis) ([formerly Azure Active Directory](https://learn.microsoft.com/azure/active-directory/fundamentals/new-name)) token authentication support across the Azure SDK. It provides a set of [TokenCredential](https://learn.microsoft.com/java/api/com.azure.core.credential.tokencredential?view=azure-java-stable) implementations that can be used to construct Azure SDK clients that support Microsoft Entra token authentication. +The Azure Identity library provides [Microsoft Entra ID](https://learn.microsoft.com/entra/fundamentals/whatis) ([formerly Azure Active Directory](https://learn.microsoft.com/entra/fundamentals/new-name)) token authentication support across the Azure SDK. It provides a set of [TokenCredential](https://learn.microsoft.com/java/api/com.azure.core.credential.tokencredential?view=azure-java-stable) implementations that can be used to construct Azure SDK clients that support Microsoft Entra token authentication. [Source code][source] | [API reference documentation][javadoc] | [Microsoft Entra ID documentation][entraid_doc] @@ -187,15 +187,15 @@ public void createDefaultAzureCredentialForIntelliJ() { ## Managed Identity support -The [Managed identity authentication](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/overview) is supported via either the `DefaultAzureCredential` or the `ManagedIdentityCredential` directly for the following Azure Services: +The [Managed identity authentication](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/overview) is supported via either the `DefaultAzureCredential` or the `ManagedIdentityCredential` directly for the following Azure Services: - [Azure App Service and Azure Functions](https://learn.microsoft.com/azure/app-service/overview-managed-identity?tabs=dotnet) - [Azure Arc](https://learn.microsoft.com/azure/azure-arc/servers/managed-identity-authentication) - [Azure Cloud Shell](https://learn.microsoft.com/azure/cloud-shell/msi-authorization) - [Azure Kubernetes Service](https://learn.microsoft.com/azure/aks/use-managed-identity) - [Azure Service Fabric](https://learn.microsoft.com/azure/service-fabric/concepts-managed-identity) -- [Azure Virtual Machines](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token) -- [Azure Virtual Machines Scale Sets](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/qs-configure-powershell-windows-vmss) +- [Azure Virtual Machines](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/how-to-use-vm-token) +- [Azure Virtual Machines Scale Sets](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-powershell-windows-vmss) **Note:** Use `azure-identity` version `1.7.0` or later to utilize [token caching](https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity/TOKEN_CACHING.md) support for managed identity authentication. @@ -339,13 +339,13 @@ Not all credentials require this configuration. Credentials that authenticate th ClientCertificateCredential authenticates a service principal using a certificate example - Service principal authentication + Service principal authentication ClientSecretCredential authenticates a service principal using a secret example - Service principal authentication + Service principal authentication @@ -367,31 +367,31 @@ Not all credentials require this configuration. Credentials that authenticate th AuthorizationCodeCredential authenticate a user with a previously obtained authorization code as part of an Oauth 2 flow - OAuth2 authentication code + OAuth2 authentication code DeviceCodeCredential interactively authenticates a user on devices with limited UI example - Device code authentication + Device code authentication InteractiveBrowserCredential interactively authenticates a user with the default system browser example - OAuth2 authentication code + OAuth2 authentication code OnBehalfOfCredential propagates the delegated user identity and permissions through the request chain - On-behalf-of authentication + On-behalf-of authentication UsernamePasswordCredential authenticates a user with a username and password without multi-factored auth example - Username + password authentication + Username + password authentication @@ -540,7 +540,7 @@ Configuration is attempted in the above order. For example, if values for a clie ## Continuous Access Evaluation -As of v1.10.0, accessing resources protected by [Continuous Access Evaluation](https://learn.microsoft.com/azure/active-directory/conditional-access/concept-continuous-access-evaluation) (CAE) is possible on a per-request basis. This can be enabled using the [`TokenRequestContext.setCaeEnabled(boolean)` API](https://learn.microsoft.com/java/api/com.azure.core.credential.tokenrequestcontext?view=azure-java-stable#com-azure-core-credential-tokenrequestcontext-setcaeenabled(boolean)). CAE isn't supported for developer credentials. +As of v1.10.0, accessing resources protected by [Continuous Access Evaluation](https://learn.microsoft.com/entra/identity/conditional-access/concept-continuous-access-evaluation) (CAE) is possible on a per-request basis. This can be enabled using the [`TokenRequestContext.setCaeEnabled(boolean)` API](https://learn.microsoft.com/java/api/com.azure.core.credential.tokenrequestcontext?view=azure-java-stable#com-azure-core-credential-tokenrequestcontext-setcaeenabled(boolean)). CAE isn't supported for developer credentials. ## Token caching Token caching is a feature provided by the Azure Identity library that allows apps to: diff --git a/sdk/identity/azure-identity/TROUBLESHOOTING.md b/sdk/identity/azure-identity/TROUBLESHOOTING.md index 51b9636d9c6d8..163727c51ff3d 100644 --- a/sdk/identity/azure-identity/TROUBLESHOOTING.md +++ b/sdk/identity/azure-identity/TROUBLESHOOTING.md @@ -66,7 +66,7 @@ This error contains several pieces of information: - __Failing Credential Type__: The type of credential that failed to authenticate. This can be helpful when diagnosing issues with chained credential types such as `DefaultAzureCredential` or `ChainedTokenCredential`. -- __STS Error Code and Message__: The error code and message returned from the Microsoft Entra STS. This can give insight into the specific reason the request failed. For instance, in this specific case because the provided client secret is incorrect. More information on STS error codes can be found [here](https://learn.microsoft.com/azure/active-directory/develop/reference-aadsts-error-codes#aadsts-error-codes). +- __STS Error Code and Message__: The error code and message returned from the Microsoft Entra STS. This can give insight into the specific reason the request failed. For instance, in this specific case because the provided client secret is incorrect. More information on STS error codes can be found [here](https://learn.microsoft.com/entra/identity-platform/reference-error-codes#aadsts-error-codes). - __Correlation ID and Timestamp__: The correlation ID and call Timestamp used to identify the request in server-side logs. This information can be useful to support engineers when diagnosing unexpected STS failures. @@ -97,17 +97,17 @@ The underlying MSAL library, MSAL4J, also has detailed logging. It is highly ver | Error Code | Issue | Mitigation | |---|---|---| -|AADSTS7000215|An invalid client secret was provided.|Ensure the `clientSecret` provided when constructing the credential is valid. If unsure, create a new client secret using the Azure portal. Details on creating a new client secret can be found [here](https://learn.microsoft.com/azure/active-directory/develop/howto-create-service-principal-portal#option-2-create-a-new-application-secret).| -|AADSTS7000222|An expired client secret was provided.|Create a new client secret using the Azure portal. Details on creating a new client secret can be found [here](https://learn.microsoft.com/azure/active-directory/develop/howto-create-service-principal-portal#option-2-create-a-new-application-secret).| -|AADSTS700016|The specified application wasn't found in the specified tenant.|Ensure the specified `clientId` and `tenantId` are correct for your application registration. For multi-tenant apps, ensure the application has been added to the desired tenant by a tenant admin. To add a new application in the desired tenant, follow the instructions [here](https://learn.microsoft.com/azure/active-directory/develop/howto-create-service-principal-portal).| +|AADSTS7000215|An invalid client secret was provided.|Ensure the `clientSecret` provided when constructing the credential is valid. If unsure, create a new client secret using the Azure portal. Details on creating a new client secret can be found [here](https://learn.microsoft.com/entra/identity-platform/howto-create-service-principal-portal#option-2-create-a-new-application-secret).| +|AADSTS7000222|An expired client secret was provided.|Create a new client secret using the Azure portal. Details on creating a new client secret can be found [here](https://learn.microsoft.com/entra/identity-platform/howto-create-service-principal-portal#option-2-create-a-new-application-secret).| +|AADSTS700016|The specified application wasn't found in the specified tenant.|Ensure the specified `clientId` and `tenantId` are correct for your application registration. For multi-tenant apps, ensure the application has been added to the desired tenant by a tenant admin. To add a new application in the desired tenant, follow the instructions [here](https://learn.microsoft.com/entra/identity-platform/howto-create-service-principal-portal).| ## Troubleshoot `ClientCertificateCredential` authentication issues `ClientAuthenticationException` | Error Code | Description | Mitigation | |---|---|---| -|AADSTS700027|Client assertion contains an invalid signature.|Ensure the specified certificate has been uploaded to the Microsoft Entra application registration. Instructions for uploading certificates to the application registration can be found [here](https://learn.microsoft.com/azure/active-directory/develop/howto-create-service-principal-portal#option-1-upload-a-certificate).| -|AADSTS700016|The specified application wasn't found in the specified tenant.| Ensure the specified `clientId` and `tenantId` are correct for your application registration. For multi-tenant apps, ensure the application has been added to the desired tenant by a tenant admin. To add a new application in the desired tenant, follow the instructions [here](https://learn.microsoft.com/azure/active-directory/develop/howto-create-service-principal-portal).| +|AADSTS700027|Client assertion contains an invalid signature.|Ensure the specified certificate has been uploaded to the Microsoft Entra application registration. Instructions for uploading certificates to the application registration can be found [here](https://learn.microsoft.com/entra/identity-platform/howto-create-service-principal-portal#option-1-upload-a-certificate).| +|AADSTS700016|The specified application wasn't found in the specified tenant.| Ensure the specified `clientId` and `tenantId` are correct for your application registration. For multi-tenant apps, ensure the application has been added to the desired tenant by a tenant admin. To add a new application in the desired tenant, follow the instructions [here](https://learn.microsoft.com/entra/identity-platform/howto-create-service-principal-portal).| ## Troubleshoot `ClientAssertionCredential` authentication issues @@ -115,9 +115,9 @@ The underlying MSAL library, MSAL4J, also has detailed logging. It is highly ver | Error Code | Description | Mitigation | |---|---|---| -|AADSTS700021| Client assertion application identifier doesn't match 'client_id' parameter. Review the documentation at https://learn.microsoft.com/azure/active-directory/develop/active-directory-certificate-credentials | Ensure the JWT assertion created has the correct values specified for the `sub` and `issuer` value of the payload, both of these should have the value be equal to `clientId`. Refer documentation for [client assertion format](https://learn.microsoft.com/azure/active-directory/develop/active-directory-certificate-credentials)| -|AADSTS700023| Client assertion audience claim does not match Realm issuer. Review the documentation at https://learn.microsoft.com/azure/active-directory/develop/active-directory-certificate-credentials. | Ensure the audience `aud` field in the JWT assertion created has the correct value for the audience specified in the payload. This should be set to `https://login.microsoftonline.com/{tenantId}/v2`.| -|AADSTS50027| JWT token is invalid or malformed. | Ensure the JWT assertion token is in the valid format. Refer to the documentation for [client assertion format](https://learn.microsoft.com/azure/active-directory/develop/active-directory-certificate-credentials).| +|AADSTS700021| Client assertion application identifier doesn't match 'client_id' parameter. Review the documentation at https://learn.microsoft.com/entra/identity-platform/certificate-credentials | Ensure the JWT assertion created has the correct values specified for the `sub` and `issuer` value of the payload, both of these should have the value be equal to `clientId`. Refer documentation for [client assertion format](https://learn.microsoft.com/entra/identity-platform/certificate-credentials)| +|AADSTS700023| Client assertion audience claim does not match Realm issuer. Review the documentation at https://learn.microsoft.com/entra/identity-platform/certificate-credentials. | Ensure the audience `aud` field in the JWT assertion created has the correct value for the audience specified in the payload. This should be set to `https://login.microsoftonline.com/{tenantId}/v2`.| +|AADSTS50027| JWT token is invalid or malformed. | Ensure the JWT assertion token is in the valid format. Refer to the documentation for [client assertion format](https://learn.microsoft.com/entra/identity-platform/certificate-credentials).| ## Troubleshoot `UsernamePasswordCredential` authentication issues `ClientAuthenticationException` @@ -136,7 +136,7 @@ The `ManagedIdentityCredential` is designed to work on a variety of Azure hosts |Azure Arc|[Configuration](https://learn.microsoft.com/azure/azure-arc/servers/managed-identity-authentication)|| |Azure Kubernetes Service|[Configuration](https://azure.github.io/aad-pod-identity/docs/)|[Troubleshooting](#azure-kubernetes-service-managed-identity)| |Azure Service Fabric|[Configuration](https://learn.microsoft.com/azure/service-fabric/concepts-managed-identity)|| -|Azure Virtual Machines and Scale Sets|[Configuration](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/qs-configure-portal-windows-vm)|[Troubleshooting](#azure-virtual-machine-managed-identity)| +|Azure Virtual Machines and Scale Sets|[Configuration](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm)|[Troubleshooting](#azure-virtual-machine-managed-identity)| ### Azure Virtual Machine Managed Identity @@ -144,10 +144,10 @@ The `ManagedIdentityCredential` is designed to work on a variety of Azure hosts | Error Message |Description| Mitigation | |---|---|---| -|The requested identity hasn't been assigned to this resource.|The IMDS endpoint responded with a status code of 400, indicating the requested identity isn't assigned to the VM.|If using a user assigned identity, ensure the specified `clientId` is correct.

If using a system assigned identity, make sure it has been enabled properly. Instructions to enable the system assigned identity on an Azure VM can be found [here](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/qs-configure-portal-windows-vm#enable-system-assigned-managed-identity-on-an-existing-vm).| +|The requested identity hasn't been assigned to this resource.|The IMDS endpoint responded with a status code of 400, indicating the requested identity isn't assigned to the VM.|If using a user assigned identity, ensure the specified `clientId` is correct.

If using a system assigned identity, make sure it has been enabled properly. Instructions to enable the system assigned identity on an Azure VM can be found [here](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm#enable-system-assigned-managed-identity-on-an-existing-vm).| |The request failed due to a gateway error.|The request to the IMDS endpoint failed due to a gateway error, 502 or 504 status code.|Calls via proxy or gateway aren't supported by IMDS. Disable proxies or gateways running on the VM for calls to the IMDS endpoint `http://169.254.169.254/`| -|No response received from the managed identity endpoint.|No response was received for the request to IMDS or the request timed out.|

  • Ensure managed identity has been properly configured on the VM. Instructions for configuring the manged identity can be found [here](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/qs-configure-portal-windows-vm).
  • Verify the IMDS endpoint is reachable on the VM, see [below](#verify-imds-is-available-on-the-vm) for instructions.
| -|Multiple attempts failed to obtain a token from the managed identity endpoint.|Retries to retrieve a token from the IMDS endpoint have been exhausted.|
  • Refer to inner exception messages for more details on specific failures. If the data has been truncated, more detail can be obtained by [collecting logs](https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity/README.md#enable-client-logging).
  • Ensure managed identity has been properly configured on the VM. Instructions for configuring the manged identity can be found [here](https://learn.microsoft.com/azure/active-directory/managed-identities-azure-resources/qs-configure-portal-windows-vm).
  • Verify the IMDS endpoint is reachable on the VM, see [below](#verify-imds-is-available-on-the-vm) for instructions.
| +|No response received from the managed identity endpoint.|No response was received for the request to IMDS or the request timed out.|
  • Ensure managed identity has been properly configured on the VM. Instructions for configuring the manged identity can be found [here](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm).
  • Verify the IMDS endpoint is reachable on the VM, see [below](#verify-imds-is-available-on-the-vm) for instructions.
| +|Multiple attempts failed to obtain a token from the managed identity endpoint.|Retries to retrieve a token from the IMDS endpoint have been exhausted.|
  • Refer to inner exception messages for more details on specific failures. If the data has been truncated, more detail can be obtained by [collecting logs](https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity/README.md#enable-client-logging).
  • Ensure managed identity has been properly configured on the VM. Instructions for configuring the manged identity can be found [here](https://learn.microsoft.com/entra/identity/managed-identities-azure-resources/qs-configure-portal-windows-vm).
  • Verify the IMDS endpoint is reachable on the VM, see [below](#verify-imds-is-available-on-the-vm) for instructions.
| #### Verify IMDS is available on the VM diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AadCredentialBuilderBase.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AadCredentialBuilderBase.java index 19f5f26bb1713..361f7a615a9a8 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AadCredentialBuilderBase.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AadCredentialBuilderBase.java @@ -136,7 +136,7 @@ public T disableInstanceDiscovery() { /** * Enables additional support logging for public and confidential client applications. This enables - * PII logging in MSAL4J as described here. + * PII logging in MSAL4J as described here. * *

This operation will log PII including tokens. It should only be used when directed by support. * diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredential.java index 87e7ea51bd142..00c23d95f5c92 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredential.java @@ -22,7 +22,7 @@ /** *

Authorization Code authentication in Azure is a type of authentication mechanism that allows users to - * authenticate with Microsoft Entra ID + * authenticate with Microsoft Entra ID * and obtain an authorization code that can be used to request an access token to access * Azure resources. It is a widely used authentication mechanism and is supported by a wide range of Azure services * and applications. It provides a secure and scalable way to authenticate users and grant them access to Azure diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredentialBuilder.java index e0b727cadb50e..330d04e31c64d 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AuthorizationCodeCredentialBuilder.java @@ -13,7 +13,7 @@ *

Fluent credential builder for instantiating a {@link AuthorizationCodeCredential}.

* *

Authorization Code authentication in Azure is a type of authentication mechanism that allows users to - * authenticate with Microsoft Entra ID + * authenticate with Microsoft Entra ID * and obtain an authorization code that can be used to request an access token to access * Azure resources. It is a widely used authentication mechanism and is supported by a wide range of Azure services * and applications. It provides a secure and scalable way to authenticate users and grant them access to Azure diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredential.java index 56d9f0b8de541..0818a9d7b22cd 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredential.java @@ -20,7 +20,7 @@ * terminal. It allows users to * authenticate interactively as a * user and/or a service principal against - * Microsoft Entra ID. + * Microsoft Entra ID. * The AzureCliCredential authenticates in a development environment and acquires a token on behalf of the * logged-in user or service principal in Azure CLI. It acts as the Azure CLI logged in user or service principal * and executes an Azure CLI command underneath to authenticate the application against Microsoft Entra ID.

diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredentialBuilder.java index da07d5cad70cf..d41caf547f26b 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureCliCredentialBuilder.java @@ -19,7 +19,7 @@ * terminal. It allows users to * authenticate interactively as a * user and/or a service principal against - * Microsoft Entra ID. + * Microsoft Entra ID. * The AzureCliCredential authenticates in a development environment and acquires a token on behalf of the * logged-in user or service principal in Azure CLI. It acts as the Azure CLI logged in user or service principal * and executes an Azure CLI command underneath to authenticate the application against Microsoft Entra ID.

diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredential.java index f8421266a01c4..df0628d86996f 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredential.java @@ -19,7 +19,7 @@ *

Azure Developer CLI is a command-line interface tool that allows developers to create, manage, and deploy * resources in Azure. It's built on top of the Azure CLI and provides additional functionality specific * to Azure developers. It allows users to authenticate as a user and/or a service principal against - * Microsoft Entra ID. + * Microsoft Entra ID. * The AzureDeveloperCliCredential authenticates in a development environment and acquires a token on behalf of * the logged-in user or service principal in Azure Developer CLI. It acts as the Azure Developer CLI logged in user or * service principal and executes an Azure CLI command underneath to authenticate the application against diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredentialBuilder.java index d1f3f51cf30d4..4fc66874eadc1 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzureDeveloperCliCredentialBuilder.java @@ -18,7 +18,7 @@ *

Azure Developer CLI is a command-line interface tool that allows developers to create, manage, and deploy * resources in Azure. It's built on top of the Azure CLI and provides additional functionality specific * to Azure developers. It allows users to authenticate as a user and/or a service principal against - * Microsoft Entra ID. + * Microsoft Entra ID. * The AzureDeveloperCliCredential authenticates in a development environment and acquires a token on behalf of * the logged-in user or service principal in Azure Developer CLI. It acts as the Azure Developer CLI logged in user or * service principal and executes an Azure CLI command underneath to authenticate the application against diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredential.java index 16fd494da1fda..365e18fd5c0cf 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredential.java @@ -19,7 +19,7 @@ * or terminal. It allows users to * authenticate interactively * as a user and/or a service principal against - * Microsoft Entra ID. + * Microsoft Entra ID. * The AzurePowershellCredential authenticates in a development environment and acquires a token on behalf of the * logged-in user or service principal in Azure Powershell. It acts as the Azure Powershell logged in user or * service principal and executes an Azure Powershell command underneath to authenticate the application against diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredentialBuilder.java index 5bc634f6ec944..77f431ddb19ea 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/AzurePowerShellCredentialBuilder.java @@ -17,7 +17,7 @@ * or terminal. It allows users to * authenticate interactively * as a user and/or a service principal against - * Microsoft Entra ID. + * Microsoft Entra ID. * The {@link AzurePowerShellCredential} authenticates in a development environment and acquires a token on * behalf of the logged-in user or service principal in Azure Powershell. It acts as the Azure Powershell logged in * user or service principal and executes an Azure Powershell command underneath to authenticate the application diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredential.java index bf788fcad345a..5dd062b019c6a 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredential.java @@ -25,7 +25,7 @@ * In this authentication method, the client application creates a JSON Web Token (JWT) that includes information about * the service principal (such as its client ID and tenant ID) and signs it using a client secret. The client then * sends this token to - * Microsoft Entra ID as proof of its + * Microsoft Entra ID as proof of its * identity. Microsoft Entra ID verifies the token signature and checks that the service principal has * the necessary permissions to access the requested Azure resource. If the token is valid and the service principal is * authorized, Microsoft Entra ID issues an access token that the client application can use to access the requested resource. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredentialBuilder.java index 57c9744d2c6cd..2bf56a64ddc3f 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientAssertionCredentialBuilder.java @@ -18,7 +18,7 @@ * In this authentication method, the client application creates a JSON Web Token (JWT) that includes information about * the service principal (such as its client ID and tenant ID) and signs it using a client secret. The client then * sends this token to - * Microsoft Entra ID as proof of its + * Microsoft Entra ID as proof of its * identity. Microsoft Entra ID verifies the token signature and checks that the service principal has * the necessary permissions to access the requested Azure resource. If the token is valid and the service principal is * authorized, Microsoft Entra ID issues an access token that the client application can use to access the requested resource. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredential.java index 471725f45fd89..0ce162f1fa594 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredential.java @@ -21,12 +21,12 @@ /** *

The ClientCertificateCredential acquires a token via service principal authentication. It is a type of * authentication in Azure that enables a non-interactive login to - * Microsoft Entra ID, allowing + * Microsoft Entra ID, allowing * an application or service to authenticate itself with Azure resources. * A Service Principal is essentially an identity created for an application in Microsoft Entra ID that can be used to * authenticate with Azure resources. It's like a "user identity" for the application or service, and it provides * a way for the application to authenticate itself with Azure resources without needing to use a user's credentials. - * Microsoft Entra ID allows users + * Microsoft Entra ID allows users * to register service principals which can be used as an identity for authentication. * A client certificate associated with the registered service principal is used as the password when authenticating * the service principal. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredentialBuilder.java index 6c3337a21aa1c..523f14ca1c236 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientCertificateCredentialBuilder.java @@ -15,12 +15,12 @@ * *

The ClientCertificateCredential acquires a token via service principal authentication. It is a type of * authentication in Azure that enables a non-interactive login to - * Microsoft Entra ID, allowing an + * Microsoft Entra ID, allowing an * application or service to authenticate itself with Azure resources. * A Service Principal is essentially an identity created for an application in Microsoft Entra ID that can be used to * authenticate with Azure resources. It's like a "user identity" for the application or service, and it provides * a way for the application to authenticate itself with Azure resources without needing to use a user's credentials. - * Microsoft Entra ID allows users to + * Microsoft Entra ID allows users to * register service principals which can be used as an identity for authentication. * A client certificate associated with the registered service principal is used as the password when authenticating * the service principal. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredential.java index 3450e10676669..37e43a328ff17 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredential.java @@ -21,12 +21,12 @@ /** *

The ClientSecretCredential acquires a token via service principal authentication. It is a type of authentication * in Azure that enables a non-interactive login to - * Microsoft Entra ID, allowing an + * Microsoft Entra ID, allowing an * application or service to authenticate itself with Azure resources. * A Service Principal is essentially an identity created for an application in Microsoft Entra ID that can be used to * authenticate with Azure resources. It's like a "user identity" for the application or service, and it provides * a way for the application to authenticate itself with Azure resources without needing to use a user's credentials. - * Microsoft Entra ID allows users to + * Microsoft Entra ID allows users to * register service principals which can be used as an identity for authentication. * A client secret associated with the registered service principal is used as the password when authenticating the * service principal. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredentialBuilder.java index b77e79c032a46..b5f099d20d751 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ClientSecretCredentialBuilder.java @@ -11,12 +11,12 @@ * *

The {@link ClientSecretCredential} acquires a token via service principal authentication. It is a type of * authentication in Azure that enables a non-interactive login to - * Microsoft Entra ID, allowing an + * Microsoft Entra ID, allowing an * application or service to authenticate itself with Azure resources. * A Service Principal is essentially an identity created for an application in Microsoft Entra ID that can be used to * authenticate with Azure resources. It's like a "user identity" for the application or service, and it provides * a way for the application to authenticate itself with Azure resources without needing to use a user's credentials. - * Microsoft Entra ID allows users to + * Microsoft Entra ID allows users to * register service principals which can be used as an identity for authentication. * A client secret associated with the registered service principal is used as the password when authenticating the * service principal. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredential.java index 938ecd7175b2a..04dd2052e5720 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredential.java @@ -72,7 +72,7 @@ *

Sample: Construct DefaultAzureCredential with User Assigned Managed Identity

* *

User-Assigned Managed Identity (UAMI) in Azure is a feature that allows you to create an identity in - * Microsoft Entra ID that is + * Microsoft Entra ID that is * associated with one or more Azure resources. This identity can then be used to authenticate and * authorize access to various Azure services and resources. The following code sample demonstrates the creation of * a DefaultAzureCredential to target a user assigned managed identity, using the diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredentialBuilder.java index 56c15539aa095..40c2193924f3e 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DefaultAzureCredentialBuilder.java @@ -43,7 +43,7 @@ *

Sample: Construct DefaultAzureCredential with User Assigned Managed Identity

* *

User-Assigned Managed Identity (UAMI) in Azure is a feature that allows you to create an identity in - * Microsoft Entra ID that is + * Microsoft Entra ID that is * associated with one or more Azure resources. This identity can then be used to authenticate and * authorize access to various Azure services and resources. The following code sample demonstrates the creation of * a {@link DefaultAzureCredential} to target a user assigned managed identity, using the DefaultAzureCredentialBuilder diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredential.java index 60fb3819f6917..281e96eab17f0 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredential.java @@ -22,7 +22,7 @@ /** *

Device code authentication is a type of authentication flow offered by - * Microsoft Entra ID that + * Microsoft Entra ID that * allows users to sign in to applications on devices that don't have a web browser or a keyboard. * This authentication method is particularly useful for devices such as smart TVs, gaming consoles, and * Internet of Things (IoT) devices that may not have the capability to enter a username and password. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredentialBuilder.java index 47b61f240490f..051a8fd03af4b 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/DeviceCodeCredentialBuilder.java @@ -15,7 +15,7 @@ * Fluent credential builder for instantiating a {@link DeviceCodeCredential}. * *

Device code authentication is a type of authentication flow offered by - * Microsoft Entra ID that + * Microsoft Entra ID that * allows users to sign in to applications on devices that don't have a web browser or a keyboard. * This authentication method is particularly useful for devices such as smart TVs, gaming consoles, and * Internet of Things (IoT) devices that may not have the capability to enter a username and password. diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredential.java index da6fa2e039679..73785dbf6c26e 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredential.java @@ -28,7 +28,7 @@ * for IntelliJ plugin for the IntelliJ IDEA development environment. It * enables developers to create, test, and deploy Java applications to the Azure cloud platform. In order to * use the plugin authentication as a user or service principal against - * Microsoft Entra ID is required. + * Microsoft Entra ID is required. * The IntelliJCredential authenticates in a development environment and acquires a token on behalf of the * logged-in account in Azure Toolkit for IntelliJ. It uses the logged in user information on the IntelliJ IDE and uses * it to authenticate the application against Microsoft Entra ID.

diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredentialBuilder.java index 1bb985361ddfc..f19677c4432e2 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/IntelliJCredentialBuilder.java @@ -20,7 +20,7 @@ * for IntelliJ plugin for the IntelliJ IDEA development environment. It enables developers to create, test, and * deploy Java applications to the Azure cloud platform. In order to use the plugin authentication as a user or * service principal against - * Microsoft Entra ID is required. + * Microsoft Entra ID is required. * The {@link IntelliJCredential} authenticates in a development environment and acquires a token on behalf of the * logged-in account in Azure Toolkit for IntelliJ. It uses the logged in user information on the IntelliJ IDE and uses * it to authenticate the application against Microsoft Entra ID.

diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredential.java index 7d759b93e0e21..ed1212233330a 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredential.java @@ -21,7 +21,7 @@ /** *

Interactive browser authentication is a type of authentication flow offered by - * Microsoft Entra ID + * Microsoft Entra ID * that enables users to sign in to applications and services using a web browser. This authentication method is * commonly used for web applications, where users enter their credentials directly into a web page. * With interactive browser authentication, the user navigates to a web application and is prompted to enter their diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredentialBuilder.java index 33cd7966217bf..1b13ff9c7cdd6 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/InteractiveBrowserCredentialBuilder.java @@ -16,7 +16,7 @@ * Fluent credential builder for instantiating a {@link InteractiveBrowserCredential}. * *

Interactive browser authentication is a type of authentication flow offered by - * Microsoft Entra ID + * Microsoft Entra ID * that enables users to sign in to applications and services using a web browser. This authentication method is * commonly used for web applications, where users enter their credentials directly into a web page. * With interactive browser authentication, the user navigates to a web application and is prompted to enter their diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredential.java index 3b9b27d1b9f72..22201b8c8778a 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredential.java @@ -19,9 +19,9 @@ import java.time.Duration; /** - *

Azure + *

Azure * Managed Identity is a feature in - * Microsoft Entra ID + * Microsoft Entra ID * that provides a way for applications running on Azure to authenticate themselves with Azure resources without * needing to manage or store any secrets like passwords or keys. * The ManagedIdentityCredential authenticates the configured managed identity (system or user assigned) of an @@ -62,7 +62,7 @@ *

Sample: Construct a User Assigned ManagedIdentityCredential

* *

User-Assigned Managed Identity (UAMI) in Azure is a feature that allows you to create an identity in - * Microsoft Entra ID + * Microsoft Entra ID * that is associated with one or more Azure resources. This identity can then be * used to authenticate and authorize access to various Azure services and resources. The following code sample * demonstrates the creation of a ManagedIdentityCredential to target a user assigned managed identity, using the diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredentialBuilder.java index 6d6b57ed66fea..1fa8f853c6215 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/ManagedIdentityCredentialBuilder.java @@ -8,9 +8,9 @@ /** *

Fluent credential builder for instantiating a {@link ManagedIdentityCredential}.

* - *

Azure + *

Azure * Managed Identity is a feature in - * Microsoft Entra ID + * Microsoft Entra ID * that provides a way for applications running on Azure to authenticate themselves with Azure resources without * needing to manage or store any secrets like passwords or keys. * The {@link ManagedIdentityCredential} authenticates the configured managed identity (system or user assigned) of an @@ -36,7 +36,7 @@ *

Sample: Construct a User Assigned ManagedIdentityCredential

* *

User-Assigned Managed Identity (UAMI) in Azure is a feature that allows you to create an identity in - * Microsoft Entra ID + * Microsoft Entra ID * that is associated with one or more Azure resources. This identity can then be used to authenticate and * authorize access to various Azure services and resources. The following code sample demonstrates the creation of a * {@link ManagedIdentityCredential} to target a user assigned managed identity, using the diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredential.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredential.java index cfad0d11b1194..24d8645c6e4e0 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredential.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredential.java @@ -22,7 +22,7 @@ /** *

Username password authentication is a common type of authentication flow used by many applications and services, - * including Microsoft Entra ID. + * including Microsoft Entra ID. * With username password authentication, users enter their username and password credentials to sign * in to an application or service. * The UsernamePasswordCredential authenticates a public client application and acquires a token using the diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredentialBuilder.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredentialBuilder.java index efddd8327f65d..a273855fd8005 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredentialBuilder.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/UsernamePasswordCredentialBuilder.java @@ -14,7 +14,7 @@ * Fluent credential builder for instantiating a {@link UsernamePasswordCredential}. * *

Username password authentication is a common type of authentication flow used by many applications and services, - * including Microsoft Entra ID. + * including Microsoft Entra ID. * With username password authentication, users enter their username and password credentials to sign * in to an application or service. * The {@link UsernamePasswordCredential} authenticates a public client application and acquires a token using the diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/package-info.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/package-info.java index 985ea4dd706cf..f3ab5e6949b3a 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/package-info.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/package-info.java @@ -3,7 +3,7 @@ /** *

The Azure Identity library provides - * Microsoft Entra ID token + * Microsoft Entra ID token * authentication support across the * Azure SDK. The library focuses on * OAuth authentication with Microsoft Entra ID, and it offers various credential classes capable of acquiring a Microsoft Entra token @@ -120,9 +120,9 @@ * *

Authenticating on Azure Hosted Platforms via Managed Identity

* - *

Azure + *

Azure * Managed Identity is a feature in - * Microsoft Entra ID + * Microsoft Entra ID * that provides a way for applications running on Azure to authenticate themselves with Azure resources without * needing to manage or store any secrets like passwords or keys.

* @@ -192,12 +192,12 @@ *

Authenticate with Service Principals

* *

Service Principal authentication is a type of authentication in Azure that enables a non-interactive login to - * Microsoft Entra ID, allowing an + * Microsoft Entra ID, allowing an * application or service to authenticate itself with Azure resources. * A Service Principal is essentially an identity created for an application in Microsoft Entra ID that can be used to * authenticate with Azure resources. It's like a "user identity" for the application or service, and it provides * a way for the application to authenticate itself with Azure resources without needing to use a user's credentials. - * Microsoft Entra ID allows users to + * Microsoft Entra ID allows users to * register service principals which can be used as an identity for authentication. * A client secret and/or a client certificate associated with the registered service principal is used as the password * when authenticating the service principal.

@@ -269,7 +269,7 @@ * *

User credential authentication is a type of authentication in Azure that involves a user providing their * username and password to authenticate with Azure resources. In Azure, user credential authentication can be used to - * authenticate with Microsoft Entra ID.

+ * authenticate with Microsoft Entra ID.

* *

The Azure Identity library supports user credentials based authentication via * {@link com.azure.identity.InteractiveBrowserCredential}, {@link com.azure.identity.DeviceCodeCredential} and From 483e43ce7f6ad5ee4ad7b3089cc9723eb39e0b4e Mon Sep 17 00:00:00 2001 From: "Xilong Feng(MSFT)" <138087452+faynef@users.noreply.github.com> Date: Thu, 28 Mar 2024 02:01:19 +0800 Subject: [PATCH 26/44] Fix Tables Samples Issue (#38952) * Fix Tables Samples Issue #38179 --- .../src/main/java/com/azure/data/tables/TableClient.java | 4 ++-- .../tables/codesnippets/TableClientJavaDocCodeSnippets.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdk/tables/azure-data-tables/src/main/java/com/azure/data/tables/TableClient.java b/sdk/tables/azure-data-tables/src/main/java/com/azure/data/tables/TableClient.java index e1656181ab773..c1ba28dbe0c8d 100644 --- a/sdk/tables/azure-data-tables/src/main/java/com/azure/data/tables/TableClient.java +++ b/sdk/tables/azure-data-tables/src/main/java/com/azure/data/tables/TableClient.java @@ -159,7 +159,7 @@ * *

  *
- * TableEntity myTableEntity = new TableEntity("paritionKey", "rowKey")
+ * TableEntity myTableEntity = new TableEntity("partitionKey", "rowKey")
  *     .addProperty("Property", "Value");
  *
  * tableClient.updateEntity(myTableEntity, TableEntityUpdateMode.REPLACE);
@@ -772,7 +772,7 @@ public void updateEntity(TableEntity entity) {
      * 
      * 
      *
-     * TableEntity myTableEntity = new TableEntity("paritionKey", "rowKey")
+     * TableEntity myTableEntity = new TableEntity("partitionKey", "rowKey")
      *     .addProperty("Property", "Value");
      *
      * tableClient.updateEntity(myTableEntity, TableEntityUpdateMode.REPLACE);
diff --git a/sdk/tables/azure-data-tables/src/samples/java/com/azure/data/tables/codesnippets/TableClientJavaDocCodeSnippets.java b/sdk/tables/azure-data-tables/src/samples/java/com/azure/data/tables/codesnippets/TableClientJavaDocCodeSnippets.java
index 9cd3361d99bbe..82c26689281a6 100644
--- a/sdk/tables/azure-data-tables/src/samples/java/com/azure/data/tables/codesnippets/TableClientJavaDocCodeSnippets.java
+++ b/sdk/tables/azure-data-tables/src/samples/java/com/azure/data/tables/codesnippets/TableClientJavaDocCodeSnippets.java
@@ -185,7 +185,7 @@ public void updateEntity() {
 
         // BEGIN: com.azure.data.tables.tableClient.updateEntity#TableEntity-TableEntityUpdateMode
 
-        TableEntity myTableEntity = new TableEntity("paritionKey", "rowKey")
+        TableEntity myTableEntity = new TableEntity("partitionKey", "rowKey")
             .addProperty("Property", "Value");
 
         tableClient.updateEntity(myTableEntity, TableEntityUpdateMode.REPLACE);

From a6d772352e159d2dd469a05c67ab42bfcac42b0e Mon Sep 17 00:00:00 2001
From: Azure SDK Bot <53356347+azure-sdk@users.noreply.github.com>
Date: Wed, 27 Mar 2024 15:01:17 -0400
Subject: [PATCH 27/44] Sync eng/common directory with azure-sdk-tools for PR
 7855 (#39324)

* Pipeline template to validate package and update package work item

* Changes to restructure validations

* Additional fixes as per comments

* Remove explicit exit code

* Set erroractionpreference for change log check

---------

Co-authored-by: Praveen Kuttappan 
---
 .../templates/steps/validate-all-packages.yml |  34 +++
 eng/common/scripts/ChangeLog-Operations.ps1   |  73 +++--
 .../scripts/Helpers/ApiView-Helpers.ps1       |  18 +-
 .../Helpers/DevOps-WorkItem-Helpers.ps1       |  42 +++
 .../Update-DevOps-Release-WorkItem.ps1        |  10 +-
 eng/common/scripts/Validate-All-Packages.ps1  |  52 ++++
 eng/common/scripts/Validate-Package.ps1       | 252 ++++++++++++++++++
 7 files changed, 447 insertions(+), 34 deletions(-)
 create mode 100644 eng/common/pipelines/templates/steps/validate-all-packages.yml
 create mode 100644 eng/common/scripts/Validate-All-Packages.ps1
 create mode 100644 eng/common/scripts/Validate-Package.ps1

diff --git a/eng/common/pipelines/templates/steps/validate-all-packages.yml b/eng/common/pipelines/templates/steps/validate-all-packages.yml
new file mode 100644
index 0000000000000..db374478a06a1
--- /dev/null
+++ b/eng/common/pipelines/templates/steps/validate-all-packages.yml
@@ -0,0 +1,34 @@
+parameters:
+  ArtifactPath: $(Build.ArtifactStagingDirectory)
+  Artifacts: []
+  ConfigFileDir: $(Build.ArtifactStagingDirectory)/PackageInfo
+
+steps:
+ - ${{ if and(ne(variables['Skip.PackageValidation'], 'true'), eq(variables['System.TeamProject'], 'internal')) }}:
+    - pwsh: |
+        echo "##vso[task.setvariable variable=SetAsReleaseBuild]false"
+      displayName: "Set as release build"
+      condition: and(succeeded(), eq(variables['SetAsReleaseBuild'], ''))
+
+    - task: Powershell@2
+      inputs:
+        filePath: $(Build.SourcesDirectory)/eng/common/scripts/Validate-All-Packages.ps1
+        arguments: >
+          -ArtifactList ('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object Name)
+          -ArtifactPath ${{ parameters.ArtifactPath }}
+          -RepoRoot $(Build.SourcesDirectory)
+          -APIKey $(azuresdk-apiview-apikey)
+          -ConfigFileDir '${{ parameters.ConfigFileDir }}'
+          -BuildDefinition $(System.CollectionUri)$(System.TeamProject)/_build?definitionId=$(System.DefinitionId)
+          -PipelineUrl $(System.CollectionUri)$(System.TeamProject)/_build/results?buildId=$(Build.BuildId)
+          -Devops_pat '$(azuresdk-azure-sdk-devops-release-work-item-pat)'
+          -IsReleaseBuild $$(SetAsReleaseBuild)
+        pwsh: true
+        workingDirectory: $(Pipeline.Workspace)
+      displayName: Validate packages and update work items
+      continueOnError: true
+      condition: >-
+        and(
+          succeededOrFailed(),
+          not(endsWith(variables['Build.Repository.Name'], '-pr'))
+        )
diff --git a/eng/common/scripts/ChangeLog-Operations.ps1 b/eng/common/scripts/ChangeLog-Operations.ps1
index e31d10dc87184..fb5f85c9a49fc 100644
--- a/eng/common/scripts/ChangeLog-Operations.ps1
+++ b/eng/common/scripts/ChangeLog-Operations.ps1
@@ -138,14 +138,27 @@ function Confirm-ChangeLogEntry {
     [Parameter(Mandatory = $true)]
     [String]$VersionString,
     [boolean]$ForRelease = $false,
-    [Switch]$SantizeEntry
+    [Switch]$SantizeEntry,
+    [PSCustomObject]$ChangeLogStatus = $null
   )
 
+  if (!$ChangeLogStatus) {
+    $ChangeLogStatus = [PSCustomObject]@{
+      IsValid = $false
+      Message = ""
+    }
+  }
+  else {
+    # Do not stop the script on error when status object is passed as param
+    $ErrorActionPreference = 'Continue'
+  }
   $changeLogEntries = Get-ChangeLogEntries -ChangeLogLocation $ChangeLogLocation
   $changeLogEntry = $changeLogEntries[$VersionString]
 
   if (!$changeLogEntry) {
-    LogError "ChangeLog[${ChangeLogLocation}] does not have an entry for version ${VersionString}."
+    $ChangeLogStatus.Message = "ChangeLog[${ChangeLogLocation}] does not have an entry for version ${VersionString}."
+    $ChangeLogStatus.IsValid = $false
+    LogError "$($ChangeLogStatus.Message)"
     return $false
   }
 
@@ -161,14 +174,16 @@ function Confirm-ChangeLogEntry {
   Write-Host "-----"
 
   if ([System.String]::IsNullOrEmpty($changeLogEntry.ReleaseStatus)) {
-    LogError "Entry does not have a correct release status. Please ensure the status is set to a date '($CHANGELOG_DATE_FORMAT)' or '$CHANGELOG_UNRELEASED_STATUS' if not yet released. See https://aka.ms/azsdk/guideline/changelogs for more info."
+    $ChangeLogStatus.Message = "Entry does not have a release status. Please ensure the status is set to a date '($CHANGELOG_DATE_FORMAT)' or '$CHANGELOG_UNRELEASED_STATUS' if not yet released. See https://aka.ms/azsdk/guideline/changelogs for more info."
+    $ChangeLogStatus.IsValid = $false
+    LogError "$($ChangeLogStatus.Message)"
     return $false
   }
 
   if ($ForRelease -eq $True)
   {
     LogDebug "Verifying as a release build because ForRelease parameter is set to true"
-    return Confirm-ChangeLogForRelease -changeLogEntry $changeLogEntry -changeLogEntries $changeLogEntries
+    return Confirm-ChangeLogForRelease -changeLogEntry $changeLogEntry -changeLogEntries $changeLogEntries -ChangeLogStatus $ChangeLogStatus
   }
 
   # If the release status is a valid date then verify like its about to be released
@@ -176,9 +191,11 @@ function Confirm-ChangeLogEntry {
   if ($status -as [DateTime])
   {
     LogDebug "Verifying as a release build because the changelog entry has a valid date."
-    return Confirm-ChangeLogForRelease -changeLogEntry $changeLogEntry -changeLogEntries $changeLogEntries
+    return Confirm-ChangeLogForRelease -changeLogEntry $changeLogEntry -changeLogEntries $changeLogEntries -ChangeLogStatus $ChangeLogStatus
   }
 
+  $ChangeLogStatus.Message = "ChangeLog[${ChangeLogLocation}] has an entry for version ${VersionString}."
+  $ChangeLogStatus.IsValid = $true
   return $true
 }
 
@@ -338,15 +355,23 @@ function Confirm-ChangeLogForRelease {
     [Parameter(Mandatory = $true)]
     $changeLogEntry,
     [Parameter(Mandatory = $true)]
-    $changeLogEntries
+    $changeLogEntries,
+    $ChangeLogStatus = $null
   )
 
+  if (!$ChangeLogStatus) {
+    $ChangeLogStatus = [PSCustomObject]@{
+      IsValid = $false
+      Message = ""
+    }
+  }
   $entries = Sort-ChangeLogEntries -changeLogEntries $changeLogEntries
 
-  $isValid = $true
+  $ChangeLogStatus.IsValid = $true
   if ($changeLogEntry.ReleaseStatus -eq $CHANGELOG_UNRELEASED_STATUS) {
-    LogError "Entry has no release date set. Please ensure to set a release date with format '$CHANGELOG_DATE_FORMAT'. See https://aka.ms/azsdk/guideline/changelogs for more info."
-    $isValid = $false
+    $ChangeLogStatus.Message = "Entry has no release date set. Please ensure to set a release date with format '$CHANGELOG_DATE_FORMAT'. See https://aka.ms/azsdk/guideline/changelogs for more info."
+    $ChangeLogStatus.IsValid = $false
+    LogError "$($ChangeLogStatus.Message)"
   }
   else {
     $status = $changeLogEntry.ReleaseStatus.Trim().Trim("()")
@@ -354,25 +379,29 @@ function Confirm-ChangeLogForRelease {
       $releaseDate = [DateTime]$status
       if ($status -ne ($releaseDate.ToString($CHANGELOG_DATE_FORMAT)))
       {
-        LogError "Date must be in the format $($CHANGELOG_DATE_FORMAT). See https://aka.ms/azsdk/guideline/changelogs for more info."
-        $isValid = $false
+        $ChangeLogStatus.Message = "Date must be in the format $($CHANGELOG_DATE_FORMAT). See https://aka.ms/azsdk/guideline/changelogs for more info."
+        $ChangeLogStatus.IsValid = $false
+        LogError "$($ChangeLogStatus.Message)"
       }
 
       if (@($entries.ReleaseStatus)[0] -ne $changeLogEntry.ReleaseStatus)
       {
-        LogError "Invalid date [ $status ]. The date for the changelog being released must be the latest in the file."
-        $isValid = $false
+        $ChangeLogStatus.Message = "Invalid date [ $status ]. The date for the changelog being released must be the latest in the file."
+        $ChangeLogStatus.IsValid = $false
+        LogError "$($ChangeLogStatus.Message)"
       }
     }
     catch {
-        LogError "Invalid date [ $status ] passed as status for Version [$($changeLogEntry.ReleaseVersion)]. See https://aka.ms/azsdk/guideline/changelogs for more info."
-        $isValid = $false
+        $ChangeLogStatus.Message = "Invalid date [ $status ] passed as status for Version [$($changeLogEntry.ReleaseVersion)]. See https://aka.ms/azsdk/guideline/changelogs for more info."
+        $ChangeLogStatus.IsValid = $false
+        LogError "$($ChangeLogStatus.Message)"
     }
   }
 
   if ([System.String]::IsNullOrWhiteSpace($changeLogEntry.ReleaseContent)) {
-    LogError "Entry has no content. Please ensure to provide some content of what changed in this version. See https://aka.ms/azsdk/guideline/changelogs for more info."
-    $isValid = $false
+    $ChangeLogStatus.Message = "Entry has no content. Please ensure to provide some content of what changed in this version. See https://aka.ms/azsdk/guideline/changelogs for more info."
+    $ChangeLogStatus.IsValid = $false
+    LogError "$($ChangeLogStatus.Message)"
   }
 
   $foundRecommendedSection = $false
@@ -391,12 +420,14 @@ function Confirm-ChangeLogForRelease {
   }
   if ($emptySections.Count -gt 0)
   {
-    LogError "The changelog entry has the following sections with no content ($($emptySections -join ', ')). Please ensure to either remove the empty sections or add content to the section."
-    $isValid = $false
+    $ChangeLogStatus.Message = "The changelog entry has the following sections with no content ($($emptySections -join ', ')). Please ensure to either remove the empty sections or add content to the section."
+    $ChangeLogStatus.IsValid = $false
+    LogError "$($ChangeLogStatus.Message)"
   }
   if (!$foundRecommendedSection)
   {
-    LogWarning "The changelog entry did not contain any of the recommended sections ($($RecommendedSectionHeaders -join ', ')), please add at least one. See https://aka.ms/azsdk/guideline/changelogs for more info."
+    $ChangeLogStatus.Message = "The changelog entry did not contain any of the recommended sections ($($RecommendedSectionHeaders -join ', ')), please add at least one. See https://aka.ms/azsdk/guideline/changelogs for more info."
+    LogWarning "$($ChangeLogStatus.Message)"
   }
-  return $isValid
+  return $ChangeLogStatus.IsValid
 }
\ No newline at end of file
diff --git a/eng/common/scripts/Helpers/ApiView-Helpers.ps1 b/eng/common/scripts/Helpers/ApiView-Helpers.ps1
index bf8b16a99e021..58ee8ee19ca1c 100644
--- a/eng/common/scripts/Helpers/ApiView-Helpers.ps1
+++ b/eng/common/scripts/Helpers/ApiView-Helpers.ps1
@@ -1,4 +1,4 @@
-function MapLanguageName($language)
+function MapLanguageToRequestParam($language)
 {
     $lang = $language
     # Update language name to match those in API cosmos DB. Cosmos SQL is case sensitive and handling this within the query makes it slow.
@@ -6,7 +6,7 @@ function MapLanguageName($language)
         $lang = "JavaScript"
     }
     elseif ($lang -eq "dotnet"){
-        $lang = "C#"
+        $lang = "C%23"
     }
     elseif ($lang -eq "java"){
         $lang = "Java"
@@ -23,17 +23,12 @@ function MapLanguageName($language)
 function Check-ApiReviewStatus($packageName, $packageVersion, $language, $url, $apiKey, $apiApprovalStatus = $null, $packageNameStatus = $null)
 {
   # Get API view URL and API Key to check status
-  Write-Host "Checking API review status"
-  $lang = MapLanguageName -language $language
+  Write-Host "Checking API review status for package: ${packageName}"
+  $lang = MapLanguageToRequestParam -language $language
   if ($lang -eq $null) {
     return
   }
   $headers = @{ "ApiKey" = $apiKey }
-  $body = @{
-    language = $lang
-    packageName = $packageName
-    packageVersion = $packageVersion
-  }
 
   if (!$apiApprovalStatus) {
     $apiApprovalStatus = [PSCustomObject]@{
@@ -51,7 +46,10 @@ function Check-ApiReviewStatus($packageName, $packageVersion, $language, $url, $
 
   try
   {
-    $response = Invoke-WebRequest $url -Method 'GET' -Headers $headers -Body $body
+    $requestUrl = "${url}?language=${lang}&packageName=${packageName}&packageVersion=${packageVersion}"
+    Write-Host "Request to APIView: [${requestUrl}]"
+    $response = Invoke-WebRequest $requestUrl -Method 'GET' -Headers $headers
+    Write-Host "Response: $($response.StatusCode)"
     Process-ReviewStatusCode -statusCode $response.StatusCode -packageName $packageName -apiApprovalStatus $apiApprovalStatus -packageNameStatus $packageNameStatus
     if ($apiApprovalStatus.IsApproved) {
       Write-Host $($apiApprovalStatus.Details)
diff --git a/eng/common/scripts/Helpers/DevOps-WorkItem-Helpers.ps1 b/eng/common/scripts/Helpers/DevOps-WorkItem-Helpers.ps1
index c03b6693edd41..805486245c5cc 100644
--- a/eng/common/scripts/Helpers/DevOps-WorkItem-Helpers.ps1
+++ b/eng/common/scripts/Helpers/DevOps-WorkItem-Helpers.ps1
@@ -985,4 +985,46 @@ function UpdatePackageVersions($pkgWorkItem, $plannedVersions, $shippedVersions)
     -Uri "https://dev.azure.com/azure-sdk/_apis/wit/workitems/${id}?api-version=6.0" `
     -Headers (Get-DevOpsRestHeaders) -Body $body -ContentType "application/json-patch+json" | ConvertTo-Json -Depth 10 | ConvertFrom-Json -AsHashTable
   return $response
+}
+
+function UpdateValidationStatus($pkgvalidationDetails, $BuildDefinition, $PipelineUrl)
+{
+    $pkgName = $pkgValidationDetails.Name
+    $versionString = $pkgValidationDetails.Version
+
+    $parsedNewVersion = [AzureEngSemanticVersion]::new($versionString)
+    $versionMajorMinor = "" + $parsedNewVersion.Major + "." + $parsedNewVersion.Minor
+    $workItem = FindPackageWorkItem -lang $LanguageDisplayName -packageName $pkgName -version $versionMajorMinor -includeClosed $true -outputCommand $false
+
+    if (!$workItem)
+    {
+        Write-Host"No work item found for package [$pkgName]."
+        return $false
+    }
+
+    $changeLogStatus = $pkgValidationDetails.ChangeLogValidation.Status
+    $changeLogDetails  = $pkgValidationDetails.ChangeLogValidation.Message
+    $apiReviewStatus = $pkgValidationDetails.APIReviewValidation.Status
+    $apiReviewDetails = $pkgValidationDetails.APIReviewValidation.Message
+    $packageNameStatus = $pkgValidationDetails.PackageNameValidation.Status
+    $packageNameDetails = $pkgValidationDetails.PackageNameValidation.Message
+
+    $fields = @()
+    $fields += "`"PackageVersion=${versionString}`""
+    $fields += "`"ChangeLogStatus=${changeLogStatus}`""
+    $fields += "`"ChangeLogValidationDetails=${changeLogDetails}`""
+    $fields += "`"APIReviewStatus=${apiReviewStatus}`""
+    $fields += "`"APIReviewStatusDetails=${apiReviewDetails}`""
+    $fields += "`"PackageNameApprovalStatus=${packageNameStatus}`""
+    $fields += "`"PackageNameApprovalDetails=${packageNameDetails}`""
+    if ($BuildDefinition) {
+        $fields += "`"PipelineDefinition=$BuildDefinition`""
+    }
+    if ($PipelineUrl) {
+        $fields += "`"LatestPipelineRun=$PipelineUrl`""
+    }
+
+    $workItem = UpdateWorkItem -id $workItem.id -fields $fields
+    Write-Host "[$($workItem.id)]$LanguageDisplayName - $pkgName($versionMajorMinor) - Updated"
+    return $true
 }
\ No newline at end of file
diff --git a/eng/common/scripts/Update-DevOps-Release-WorkItem.ps1 b/eng/common/scripts/Update-DevOps-Release-WorkItem.ps1
index c8ac48e4fa868..b3a0da8036bf0 100644
--- a/eng/common/scripts/Update-DevOps-Release-WorkItem.ps1
+++ b/eng/common/scripts/Update-DevOps-Release-WorkItem.ps1
@@ -15,7 +15,8 @@ param(
   [string]$packageNewLibrary = "true",
   [string]$relatedWorkItemId = $null,
   [string]$tag = $null,
-  [string]$devops_pat = $env:DEVOPS_PAT
+  [string]$devops_pat = $env:DEVOPS_PAT,
+  [bool]$inRelease = $true
 )
 #Requires -Version 6.0
 Set-StrictMode -Version 3
@@ -97,8 +98,11 @@ Write-Host "  PackageDisplayName: $($workItem.fields['Custom.PackageDisplayName'
 Write-Host "  ServiceName: $($workItem.fields['Custom.ServiceName'])"
 Write-Host "  PackageType: $($workItem.fields['Custom.PackageType'])"
 Write-Host ""
-Write-Host "Marking item [$($workItem.id)]$($workItem.fields['System.Title']) as '$state' for '$releaseType'"
-$updatedWI = UpdatePackageWorkItemReleaseState -id $workItem.id -state "In Release" -releaseType $releaseType -outputCommand $false
+if ($inRelease)
+{
+  Write-Host "Marking item [$($workItem.id)]$($workItem.fields['System.Title']) as '$state' for '$releaseType'"
+  $updatedWI = UpdatePackageWorkItemReleaseState -id $workItem.id -state "In Release" -releaseType $releaseType -outputCommand $false
+}
 $updatedWI = UpdatePackageVersions $workItem -plannedVersions $plannedVersions
 
 Write-Host "Release tracking item is at https://dev.azure.com/azure-sdk/Release/_workitems/edit/$($updatedWI.id)/"
diff --git a/eng/common/scripts/Validate-All-Packages.ps1 b/eng/common/scripts/Validate-All-Packages.ps1
new file mode 100644
index 0000000000000..46d76195ba143
--- /dev/null
+++ b/eng/common/scripts/Validate-All-Packages.ps1
@@ -0,0 +1,52 @@
+[CmdletBinding()]
+Param (
+  [Parameter(Mandatory=$True)]
+  [array]$ArtifactList,
+  [Parameter(Mandatory=$True)]
+  [string]$ArtifactPath,
+  [Parameter(Mandatory=$True)]
+  [string]$RepoRoot,
+  [Parameter(Mandatory=$True)]
+  [string]$APIKey,
+  [string]$ConfigFileDir,
+  [string]$BuildDefinition,
+  [string]$PipelineUrl,
+  [string]$APIViewUri  = "https://apiview.dev/AutoReview/GetReviewStatus",
+  [string]$Devops_pat = $env:DEVOPS_PAT,
+  [bool] $IsReleaseBuild = $false
+)
+
+Set-StrictMode -Version 3
+. (Join-Path $PSScriptRoot common.ps1)
+
+function ProcessPackage($PackageName, $ConfigFileDir)
+{
+    Write-Host "Artifact path: $($ArtifactPath)"
+    Write-Host "Package Name: $($PackageName)"
+    Write-Host "Config File directory: $($ConfigFileDir)"
+
+    &$EngCommonScriptsDir/Validate-Package.ps1 `
+        -PackageName $PackageName `
+        -ArtifactPath $ArtifactPath `
+        -RepoRoot $RepoRoot `
+        -APIViewUri $APIViewUri `
+        -APIKey $APIKey `
+        -BuildDefinition $BuildDefinition `
+        -PipelineUrl $PipelineUrl `
+        -ConfigFileDir $ConfigFileDir `
+        -Devops_pat $Devops_pat
+    if ($LASTEXITCODE -ne 0) {
+        Write-Error "Failed to validate package $PackageName"
+        exit 1
+    }
+}
+
+# Check if package config file is present. This file has package version, SDK type etc info.
+if (-not $ConfigFileDir) {
+    $ConfigFileDir = Join-Path -Path $ArtifactPath "PackageInfo"
+}
+foreach ($artifact in $ArtifactList)
+{
+    Write-Host "Processing $($artifact.name)"
+    ProcessPackage -PackageName $artifact.name -ConfigFileDir $ConfigFileDir
+}
\ No newline at end of file
diff --git a/eng/common/scripts/Validate-Package.ps1 b/eng/common/scripts/Validate-Package.ps1
new file mode 100644
index 0000000000000..57f093d76e695
--- /dev/null
+++ b/eng/common/scripts/Validate-Package.ps1
@@ -0,0 +1,252 @@
+#This script is responsible for release preparedness check that's run as part of build pipeline.
+
+[CmdletBinding()]
+param (
+  [Parameter(Mandatory = $true)]  
+  [string] $PackageName,
+  [Parameter(Mandatory = $true)] 
+  [string] $ArtifactPath,
+  [Parameter(Mandatory=$True)]
+  [string] $RepoRoot,
+  [Parameter(Mandatory=$True)]
+  [string] $APIKey,  
+  [Parameter(Mandatory=$True)]
+  [string] $ConfigFileDir,
+  [string] $BuildDefinition,
+  [string] $PipelineUrl,
+  [string] $APIViewUri,
+  [string] $Devops_pat = $env:DEVOPS_PAT,
+  [bool] $IsReleaseBuild = $false
+)
+Set-StrictMode -Version 3
+
+. (Join-Path $PSScriptRoot common.ps1)
+. ${PSScriptRoot}\Helpers\ApiView-Helpers.ps1
+. ${PSScriptRoot}\Helpers\DevOps-WorkItem-Helpers.ps1
+
+if (!$Devops_pat) {
+  az account show *> $null
+  if (!$?) {
+    Write-Host 'Running az login...'
+    az login *> $null
+  }
+}
+else {
+  # Login using PAT
+  LoginToAzureDevops $Devops_pat
+}
+
+az extension show -n azure-devops *> $null
+if (!$?){
+  az extension add --name azure-devops
+} else {
+  # Force update the extension to the latest version if it was already installed
+  # this is needed to ensure we have the authentication issue fixed from earlier versions
+  az extension update -n azure-devops *> $null
+}
+
+CheckDevOpsAccess
+
+# Function to validate change log
+function ValidateChangeLog($changeLogPath, $versionString, $validationStatus)
+{
+    try
+    {
+        $ChangeLogStatus = [PSCustomObject]@{
+            IsValid = $false
+            Message = ""
+        }
+        $changeLogFullPath = Join-Path $RepoRoot $changeLogPath
+        Write-Host "Path to change log: [$changeLogFullPath]"        
+        if (Test-Path $changeLogFullPath)
+        {
+            Confirm-ChangeLogEntry -ChangeLogLocation $changeLogFullPath -VersionString $versionString -ForRelease $true -ChangeLogStatus $ChangeLogStatus
+            $validationStatus.Status = if ($ChangeLogStatus.IsValid) { "Success" } else { "Failed" }
+            $validationStatus.Message = $ChangeLogStatus.Message 
+        }
+        else {
+            $validationStatus.Status = "Failed"
+            $validationStatus.Message = "Change log is not found in [$changeLogPath]. Change log file must be present in package root directory."
+        }
+    }
+    catch
+    {
+        Write-Host "Current directory: $(Get-Location)"
+        $validationStatus.Status = "Failed"
+        $validationStatus.Message = $_.Exception.Message
+    }
+}
+
+# Function to verify API review status
+function VerifyAPIReview($packageName, $packageVersion, $language)
+{
+    $APIReviewValidation = [PSCustomObject]@{
+        Name = "API Review Approval"
+        Status = "Pending"
+        Message = ""    
+    }
+    $PackageNameValidation = [PSCustomObject]@{
+        Name = "Package Name Approval"
+        Status = "Pending"
+        Message = ""
+    }
+
+    try
+    {
+        $apiStatus = [PSCustomObject]@{
+            IsApproved = $false
+            Details = ""
+        }
+        $packageNameStatus = [PSCustomObject]@{
+            IsApproved = $false
+            Details = ""
+        }
+        Write-Host "Checking API review status for package $packageName with version $packageVersion. language [$language]." 
+        Check-ApiReviewStatus $packageName $packageVersion $language $APIViewUri $APIKey $apiStatus $packageNameStatus
+
+        Write-Host "API review approval details: $($apiStatus.Details)"
+        Write-Host "Package name approval details: $($packageNameStatus.Details)"
+        #API review approval status
+        $APIReviewValidation.Message = $apiStatus.Details
+        $APIReviewValidation.Status = if ($apiStatus.IsApproved) { "Approved" } else { "Pending" }
+
+        # Package name approval status
+        $PackageNameValidation.Status = if ($packageNameStatus.IsApproved) { "Approved" } else { "Pending" }
+        $PackageNameValidation.Message = $packageNameStatus.Details
+    }
+    catch
+    {
+        Write-Warning "Failed to get API review status. Error: $_"
+        $PackageNameValidation.Status = "Failed"
+        $PackageNameValidation.Message = $_.Exception.Message
+        $APIReviewValidation.Status = "Failed"
+        $APIReviewValidation.Message = $_.Exception.Message
+    }
+
+    return [PSCustomObject]@{
+        ApiviewApproval = $APIReviewValidation
+        PackageNameApproval = $PackageNameValidation
+    }
+}
+
+
+function IsVersionShipped($packageName, $packageVersion)
+{
+    # This function will decide if a package version is already shipped or not  
+    Write-Host "Checking if a version is already shipped for package $packageName with version $packageVersion."
+    $parsedNewVersion = [AzureEngSemanticVersion]::new($packageVersion)
+    $versionMajorMinor = "" + $parsedNewVersion.Major + "." + $parsedNewVersion.Minor
+    $workItem = FindPackageWorkItem -lang $LanguageDisplayName -packageName $packageName -version $versionMajorMinor -includeClosed $true -outputCommand $false
+    if ($workItem)
+    {
+        # Check if the package version is already shipped    
+        $shippedVersionSet = ParseVersionSetFromMDField $workItem.fields["Custom.ShippedPackages"]
+        if ($shippedVersionSet.ContainsKey($packageVersion)) {
+            return $true
+        }
+    }
+    else {
+        Write-Host "No work item found for package [$packageName]. Creating new work item for package."
+    }
+    return $false
+}
+
+function CreateUpdatePackageWorkItem($pkgInfo)
+{
+    # This function will create or update package work item in Azure DevOps
+    $versionString = $pkgInfo.Version
+    $packageName = $pkgInfo.Name
+    $plannedDate = $pkgInfo.ReleaseStatus
+    $setReleaseState = $true
+    if (!$plannedDate -or $plannedDate -eq "Unreleased")
+    {
+        $setReleaseState = $false
+        $plannedDate = "unknown"
+    }
+        
+    # Create or update package work item  
+    &$EngCommonScriptsDir/Update-DevOps-Release-WorkItem.ps1 `
+        -language $LanguageDisplayName `
+        -packageName $packageName `
+        -version $versionString `
+        -plannedDate $plannedDate `
+        -packageRepoPath $pkgInfo.serviceDirectory `
+        -packageType $pkgInfo.SDKType `
+        -packageNewLibrary $pkgInfo.IsNewSDK `
+        -serviceName "unknown" `
+        -packageDisplayName "unknown" `
+        -inRelease $IsReleaseBuild `
+        -devops_pat $Devops_pat
+    
+    if ($LASTEXITCODE -ne 0)
+    {
+        Write-Host "Update of the Devops Release WorkItem failed."
+        return $false
+    }
+    return $true
+}
+
+# Read package property file and identify all packages to process
+Write-Host "Processing package: $PackageName"
+Write-Host "Is Release Build: $IsReleaseBuild"
+$packagePropertyFile = Join-Path $ConfigFileDir "$PackageName.json"
+$pkgInfo = Get-Content $packagePropertyFile | ConvertFrom-Json
+
+$changeLogPath = $pkgInfo.ChangeLogPath
+$versionString = $pkgInfo.Version
+Write-Host "Checking if we need to create or update work item for package $packageName with version $versionString."
+$isShipped = IsVersionShipped $packageName $versionString
+if ($isShipped) {
+    Write-Host "Package work item already exists for version [$versionString] that is marked as shipped. Skipping the update of package work item."
+    exit 0
+}
+
+Write-Host "Validating package $packageName with version $versionString."
+
+# Change log validation
+$changeLogStatus = [PSCustomObject]@{
+    Name = "Change Log Validation"
+    Status = "Success"
+    Message = ""
+}
+ValidateChangeLog $changeLogPath $versionString $changeLogStatus
+
+# API review and package name validation
+$apireviewDetails = VerifyAPIReview $PackageName $pkgInfo.Version $Language
+
+$pkgValidationDetails= [PSCustomObject]@{
+    Name = $PackageName
+    Version = $pkgInfo.Version
+    ChangeLogValidation = $changeLogStatus
+    APIReviewValidation = $apireviewDetails.ApiviewApproval
+    PackageNameValidation = $apireviewDetails.PackageNameApproval
+}
+
+$output = ConvertTo-Json $pkgValidationDetails
+Write-Host "Output: $($output)"
+
+# Create json token file in artifact path
+$tokenFile = Join-Path $ArtifactPath "$PackageName-Validation.json"
+$output | Out-File -FilePath $tokenFile -Encoding utf8
+
+# Create DevOps work item
+$updatedWi = CreateUpdatePackageWorkItem $pkgInfo
+
+# Update validation status in package work item
+if ($updatedWi) {
+    Write-Host "Updating validation status in package work item."
+    $updatedWi = UpdateValidationStatus $pkgValidationDetails $BuildDefinition $PipelineUrl    
+}
+
+# Fail the build if any validation is not successful for a release build
+Write-Host "Change log status:" $changelogStatus.Status
+Write-Host "API Review status:" $apireviewDetails.ApiviewApproval.Status
+Write-Host "Package Name status:" $apireviewDetails.PackageNameApproval.Status
+
+if ($IsReleaseBuild)
+{
+    if (!$updatedWi -or $changelogStatus.Status -ne "Success" -or $apireviewDetails.ApiviewApproval.Status -ne "Approved" -or $apireviewDetails.PackageNameApproval.Status -ne "Approved") {        
+        Write-Error "At least one of the Validations above failed for package $PackageName with version $versionString."
+        exit 1
+    }
+}
\ No newline at end of file

From 7eb17b0212c166a6e523bc35abb67a19f1aaf96c Mon Sep 17 00:00:00 2001
From: Kushagra Thapar 
Date: Wed, 27 Mar 2024 13:05:47 -0700
Subject: [PATCH 28/44] Updated Network failure logging to warning (#39429)

---
 .../com/azure/cosmos/implementation/RxGatewayStoreModel.java  | 2 +-
 .../directconnectivity/GatewayAddressCache.java               | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RxGatewayStoreModel.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RxGatewayStoreModel.java
index e17726deaf326..798e96353477c 100644
--- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RxGatewayStoreModel.java
+++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RxGatewayStoreModel.java
@@ -428,7 +428,7 @@ private Mono toDocumentServiceResponse(Mono> getServerAddressesViaGatewayInternalAsync(RxDocument
             CosmosException dce;
             if (!(exception instanceof CosmosException)) {
                 // wrap in CosmosException
-                logger.error("Network failure", exception);
+                logger.warn("Network failure", exception);
                 int statusCode = 0;
                 if (WebExceptionUtility.isNetworkFailure(exception)) {
                     if (WebExceptionUtility.isReadTimeoutException(exception)) {
@@ -864,7 +864,7 @@ private Mono> getMasterAddressesViaGatewayAsyncInternal(
             CosmosException dce;
             if (!(exception instanceof CosmosException)) {
                 // wrap in CosmosException
-                logger.error("Network failure", exception);
+                logger.warn("Network failure", exception);
                 int statusCode = 0;
                 if (WebExceptionUtility.isNetworkFailure(exception)) {
                     if (WebExceptionUtility.isReadTimeoutException(exception)) {

From c5786ecf1c382bb851bd029c4e66613670f4efb6 Mon Sep 17 00:00:00 2001
From: annie-mac 
Date: Wed, 27 Mar 2024 13:29:41 -0700
Subject: [PATCH 29/44] merge from original PR

---
 .github/event-processor.config                |    2 +-
 .gitignore                                    |    3 +-
 .vscode/cspell.json                           |    1 +
 SUPPORT.md                                    |    2 +
 common/perf-test-core/pom.xml                 |    6 +-
 common/smoke-tests/pom.xml                    |   12 +-
 eng/CredScanSuppression.json                  |   10 +
 eng/code-quality-reports/pom.xml              |    2 +-
 .../main/resources/checkstyle/checkstyle.xml  |    2 +
 .../templates/steps/bypass-local-dns.yml      |    2 +-
 .../templates/steps/create-apireview.yml      |   19 +-
 .../templates/steps/git-push-changes.yml      |   23 +-
 eng/common/scripts/Create-APIReview.ps1       |  313 +-
 .../scripts/Helpers/ApiView-Helpers.ps1       |   87 +-
 eng/common/scripts/Update-DocsMsMetadata.ps1  |   23 +-
 eng/common/scripts/check-for-git-changes.ps1  |   14 +
 .../onboarding/common-asset-functions.ps1     |   35 -
 .../onboarding/generate-assets-json.ps1       |    4 +-
 eng/common/testproxy/publish-proxy-logs.yml   |    7 +-
 eng/common/testproxy/target_version.txt       |    2 +-
 .../testproxy/test-proxy-tool-shutdown.yml    |   10 +
 eng/common/testproxy/test-proxy-tool.yml      |    8 +-
 eng/pipelines/aggregate-reports.yml           |   14 +-
 eng/pipelines/bompreparation.yml              |    8 +-
 eng/pipelines/code-quality-reports.yml        |   27 +-
 eng/pipelines/docindex.yml                    |   13 +-
 eng/pipelines/latest-jdk.yml                  |   90 -
 eng/pipelines/partner-release.yml             |  125 +-
 eng/pipelines/patch_release_client.txt        |    3 +-
 .../scripts/Get-Heap-Dump-Hprofs.ps1          |   32 +
 eng/pipelines/scripts/Get-Linting-Reports.ps1 |   15 +-
 eng/pipelines/scripts/Get-Test-Logs.ps1       |   32 +
 .../scripts/Invoke-Sparse-Checkout.ps1        |  104 +
 eng/pipelines/scripts/Set-Dev-Versioning.ps1  |   37 +
 .../scripts/generate-project-list.ps1         |   42 +
 .../templates/jobs/build-validate-pom.yml     |   27 +-
 eng/pipelines/templates/jobs/ci.tests.yml     |   67 +-
 .../templates/jobs/ci.versions.tests.yml      |   60 +-
 eng/pipelines/templates/jobs/ci.yml           |  177 +-
 eng/pipelines/templates/jobs/live.tests.yml   |   41 +-
 .../templates/jobs/native.live.tests.yml      |  117 +-
 .../templates/stages/1es-redirect.yml         |   60 +
 .../stages/archetype-java-release-batch.yml   |   90 +-
 .../stages/archetype-java-release-patch.yml   |   37 +-
 .../archetype-java-release-pom-only.yml       |   21 +-
 .../stages/archetype-java-release.yml         |   45 +-
 .../stages/archetype-sdk-client-patch.yml     |  583 +-
 .../templates/stages/archetype-sdk-client.yml |  335 +-
 .../stages/archetype-sdk-native-tests.yml     |   30 +-
 .../stages/archetype-sdk-pom-only.yml         |   81 +-
 .../stages/archetype-sdk-tests-isolated.yml   |  175 +
 .../templates/stages/archetype-sdk-tests.yml  |  280 +-
 .../stages/cosmos-emulator-matrix.json        |    2 +-
 .../templates/stages/cosmos-sdk-client.yml    |  263 +-
 .../stages/native-platform-matrix.json        |    4 +-
 .../templates/stages/platform-matrix.json     |   16 +-
 .../templates/steps/build-and-test-native.yml |    8 -
 .../templates/steps/build-and-test.yml        |    6 -
 ...ailure.yml => clean-maven-local-cache.yml} |    5 -
 .../steps/download-credscan-suppressions.yml  |   14 +
 ...roject-list-and-cache-maven-repository.yml |   34 +-
 .../templates/steps/http-fault-injector.yml   |   47 +-
 .../steps/initialize-test-environment.yml     |   21 +-
 .../publish-http-fault-injector-logs.yml      |   25 +-
 .../steps/retain-heap-dump-hprofs.yml         |   31 +-
 .../templates/steps/retain-test-logs.yml      |   26 +-
 .../steps/run-and-validate-linting.yml        |   12 +-
 .../sparse-checkout-repo-initialized.yml      |   30 +
 eng/pipelines/templates/variables/globals.yml |    2 +
 eng/pipelines/templates/variables/image.yml   |   26 +
 ...e-ServiceDirectories-From-Project-List.ps1 |    9 +-
 eng/scripts/Language-Settings.ps1             |    2 +-
 eng/scripts/Publish-MavenPackages.ps1         |   21 +-
 eng/scripts/generate_from_source_pom.py       |   36 +-
 eng/scripts/setup-http-fault-injector.ps1     |   30 +
 eng/versioning/external_dependencies.txt      |  136 +-
 eng/versioning/pom_file_version_scanner.ps1   |  119 +-
 eng/versioning/update_versions.py             |   10 +-
 eng/versioning/version_client.txt             |  200 +-
 eng/versioning/version_data.txt               |    1 -
 .../azure-resourcemanager-advisor/pom.xml     |    6 +-
 .../azure-resourcemanager-agrifood/pom.xml    |    6 +-
 .../azure-verticals-agrifood-farming/pom.xml  |    2 +-
 .../azure-ai-anomalydetector/pom.xml          |    2 +-
 .../azure-ai-anomalydetector/tests.yml        |   16 +-
 sdk/aot/azure-aot-graalvm-samples/pom.xml     |   20 +-
 .../azure-aot-graalvm-support-netty/pom.xml   |   32 +-
 sdk/aot/ci.yml                                |    2 -
 .../azure-resourcemanager-apicenter/pom.xml   |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/apimanagement/tests.mgmt.yml              |   24 +-
 .../pom.xml                                   |    6 +-
 .../azure-data-appconfiguration-perf/pom.xml  |    2 +-
 .../azure-data-appconfiguration/CHANGELOG.md  |   11 +
 .../azure-data-appconfiguration/pom.xml       |    4 +-
 .../pom.xml                                   |    6 +-
 sdk/appconfiguration/ci.yml                   |    5 +
 sdk/appconfiguration/tests.native.yml         |   35 +-
 sdk/appconfiguration/tests.yml                |   28 +-
 .../CHANGELOG.md                              |  401 +-
 .../README.md                                 |    8 +-
 .../SAMPLE.md                                 | 4520 ++++++++----
 .../pom.xml                                   |   12 +-
 .../ContainerAppsApiManager.java              |  380 +-
 .../fluent/AppResilienciesClient.java         |  198 +
 .../AvailableWorkloadProfilesClient.java      |   16 +-
 .../fluent/BillingMetersClient.java           |   20 +-
 .../fluent/BuildAuthTokensClient.java         |   47 +
 .../appcontainers/fluent/BuildersClient.java  |  270 +
 .../BuildsByBuilderResourcesClient.java}      |   31 +-
 .../appcontainers/fluent/BuildsClient.java}   |  139 +-
 .../fluent/CertificatesClient.java            |   88 +-
 ...nnectedEnvironmentsCertificatesClient.java |   85 +-
 .../fluent/ConnectedEnvironmentsClient.java   |  136 +-
 ...ectedEnvironmentsDaprComponentsClient.java |   88 +-
 .../ConnectedEnvironmentsStoragesClient.java  |   62 +-
 .../fluent/ContainerAppsApiClient.java        |  153 +-
 .../ContainerAppsAuthConfigsClient.java       |   62 +-
 .../fluent/ContainerAppsClient.java           |  216 +-
 .../ContainerAppsDiagnosticsClient.java       |   60 +-
 .../ContainerAppsRevisionReplicasClient.java  |   28 +-
 .../fluent/ContainerAppsRevisionsClient.java  |   64 +-
 .../ContainerAppsSourceControlsClient.java    |  104 +-
 ...DaprComponentResiliencyPoliciesClient.java |  164 +
 .../fluent/DaprComponentsClient.java          |   85 +-
 .../fluent/DaprSubscriptionsClient.java       |  151 +
 .../fluent/DotNetComponentsClient.java}       |  222 +-
 .../fluent/JavaComponentsClient.java          |  281 +
 .../appcontainers/fluent/JobsClient.java      |  301 +-
 .../fluent/JobsExecutionsClient.java          |   12 +-
 .../fluent/ManagedCertificatesClient.java     |  118 +-
 .../ManagedEnvironmentDiagnosticsClient.java  |   48 +-
 .../ManagedEnvironmentUsagesClient.java       |   47 +
 .../fluent/ManagedEnvironmentsClient.java     |  220 +-
 .../ManagedEnvironmentsDiagnosticsClient.java |   24 +-
 .../ManagedEnvironmentsStoragesClient.java    |   61 +-
 .../fluent/NamespacesClient.java              |   31 +-
 .../fluent/OperationsClient.java              |   12 +-
 .../fluent/ResourceProvidersClient.java       |   40 +-
 .../appcontainers/fluent/UsagesClient.java    |   45 +
 .../fluent/models/AppResiliencyInner.java     |  207 +
 .../models/AppResiliencyProperties.java       |  208 +
 .../fluent/models/AuthConfigInner.java        |   60 +-
 .../fluent/models/AuthConfigProperties.java   |   65 +-
 .../models/AvailableWorkloadProfileInner.java |   34 +-
 .../models/BillingMeterCollectionInner.java   |   20 +-
 .../fluent/models/BuildProperties.java        |  167 +
 .../fluent/models/BuildResourceInner.java     |  160 +
 .../fluent/models/BuildTokenInner.java        |   59 +
 .../fluent/models/BuilderProperties.java      |  112 +
 .../fluent/models/BuilderResourceInner.java   |  174 +
 .../BuilderResourceUpdateProperties.java      |   56 +
 .../fluent/models/CertificateInner.java       |   24 +-
 .../CheckNameAvailabilityResponseInner.java   |   22 +-
 .../models/ConnectedEnvironmentInner.java     |   56 +-
 .../ConnectedEnvironmentProperties.java       |   40 +-
 .../ConnectedEnvironmentStorageInner.java     |   30 +-
 ...tedEnvironmentStoragesCollectionInner.java |   20 +-
 .../models/ContainerAppAuthTokenInner.java    |   26 +-
 .../ContainerAppAuthTokenProperties.java      |   14 +-
 .../fluent/models/ContainerAppInner.java      |   88 +-
 .../ContainerAppJobExecutionsInner.java       |   22 +-
 .../fluent/models/ContainerAppProperties.java |   44 +-
 .../CustomHostnameAnalysisResultInner.java    |   48 +-
 .../fluent/models/DaprComponentInner.java     |   84 +-
 .../models/DaprComponentProperties.java       |   72 +-
 .../DaprComponentResiliencyPolicyInner.java   |  112 +
 ...prComponentResiliencyPolicyProperties.java |   89 +
 .../models/DaprSecretsCollectionInner.java    |   20 +-
 .../fluent/models/DaprSubscriptionInner.java  |  228 +
 .../models/DaprSubscriptionProperties.java    |  222 +
 .../models/DiagnosticsCollectionInner.java    |   22 +-
 .../fluent/models/DiagnosticsInner.java       |   30 +-
 .../fluent/models/DotNetComponentInner.java   |  146 +
 .../models/DotNetComponentProperties.java     |  132 +
 .../models/EnvironmentAuthTokenInner.java     |   26 +-
 .../EnvironmentAuthTokenProperties.java       |   14 +-
 .../fluent/models/HeaderMatchMatch.java       |  132 +
 .../fluent/models/HttpRetryPolicyMatches.java |  111 +
 .../models/HttpRetryPolicyRetryBackOff.java   |   80 +
 .../fluent/models/JavaComponentInner.java     |  146 +
 .../models/JavaComponentProperties.java       |  132 +
 .../fluent/models/JobExecutionBaseInner.java  |   18 +-
 .../fluent/models/JobExecutionInner.java      |  100 +-
 .../fluent/models/JobExecutionProperties.java |  127 +
 .../appcontainers/fluent/models/JobInner.java |   86 +-
 .../fluent/models/JobProperties.java          |   32 +-
 .../models/JobSecretsCollectionInner.java     |   19 +-
 .../models/ManagedCertificateInner.java       |   24 +-
 .../models/ManagedEnvironmentInner.java       |  196 +-
 .../models/ManagedEnvironmentProperties.java  |  164 +-
 .../ManagedEnvironmentStorageInner.java       |   30 +-
 ...gedEnvironmentStoragesCollectionInner.java |   20 +-
 .../fluent/models/OperationDetailInner.java   |   26 +-
 .../fluent/models/ReplicaCollectionInner.java |   19 +-
 .../fluent/models/ReplicaInner.java           |   42 +-
 .../fluent/models/ReplicaProperties.java      |   24 +-
 .../fluent/models/RevisionInner.java          |   58 +-
 .../fluent/models/RevisionProperties.java     |   40 +-
 .../fluent/models/SecretsCollectionInner.java |   19 +-
 .../fluent/models/SourceControlInner.java     |   48 +-
 .../models/SourceControlProperties.java       |   30 +-
 .../fluent/models/UsageInner.java             |  142 +
 .../models/WorkloadProfileStatesInner.java    |   30 +-
 .../fluent/models/package-info.java           |    5 +-
 .../appcontainers/fluent/package-info.java    |    5 +-
 .../AppResilienciesClientImpl.java            |  930 +++
 .../implementation/AppResilienciesImpl.java   |  156 +
 .../implementation/AppResiliencyImpl.java     |  171 +
 .../implementation/AuthConfigImpl.java        |   76 +-
 .../AvailableWorkloadProfileImpl.java         |    8 +-
 .../AvailableWorkloadProfilesClientImpl.java  |  216 +-
 .../AvailableWorkloadProfilesImpl.java        |    7 +-
 .../BillingMeterCollectionImpl.java           |    3 +-
 .../BillingMetersClientImpl.java              |  108 +-
 .../implementation/BillingMetersImpl.java     |    8 +-
 .../BuildAuthTokensClientImpl.java            |  202 +
 .../implementation/BuildAuthTokensImpl.java   |   57 +
 .../implementation/BuildResourceImpl.java     |  154 +
 .../implementation/BuildTokenImpl.java        |   37 +
 .../implementation/BuilderResourceImpl.java   |  209 +
 .../implementation/BuildersClientImpl.java    | 1290 ++++
 .../implementation/BuildersImpl.java          |  147 +
 .../BuildsByBuilderResourcesClientImpl.java   |  278 +
 .../BuildsByBuilderResourcesImpl.java         |   45 +
 .../implementation/BuildsClientImpl.java}     |  590 +-
 .../implementation/BuildsImpl.java            |  145 +
 .../implementation/CertificateImpl.java       |   74 +-
 .../CertificatesClientImpl.java               |  662 +-
 .../implementation/CertificatesImpl.java      |   76 +-
 .../CheckNameAvailabilityResponseImpl.java    |    3 +-
 .../ConnectedEnvironmentImpl.java             |   75 +-
 .../ConnectedEnvironmentStorageImpl.java      |   75 +-
 ...ctedEnvironmentStoragesCollectionImpl.java |   12 +-
 ...tedEnvironmentsCertificatesClientImpl.java |  733 +-
 ...ConnectedEnvironmentsCertificatesImpl.java |  148 +-
 .../ConnectedEnvironmentsClientImpl.java      | 1082 +--
 ...dEnvironmentsDaprComponentsClientImpl.java |  701 +-
 ...nnectedEnvironmentsDaprComponentsImpl.java |  193 +-
 .../ConnectedEnvironmentsImpl.java            |  159 +-
 ...nnectedEnvironmentsStoragesClientImpl.java |  483 +-
 .../ConnectedEnvironmentsStoragesImpl.java    |  170 +-
 .../ContainerAppAuthTokenImpl.java            |    3 +-
 .../implementation/ContainerAppImpl.java      |   61 +-
 .../ContainerAppJobExecutionsImpl.java        |   11 +-
 .../ContainerAppsApiClientBuilder.java        |   50 +-
 .../ContainerAppsApiClientImpl.java           |  433 +-
 .../ContainerAppsAuthConfigsClientImpl.java   |  513 +-
 .../ContainerAppsAuthConfigsImpl.java         |  136 +-
 .../ContainerAppsClientImpl.java              | 1440 ++--
 .../ContainerAppsDiagnosticsClientImpl.java   |  683 +-
 .../ContainerAppsDiagnosticsImpl.java         |   66 +-
 .../implementation/ContainerAppsImpl.java     |  137 +-
 ...ntainerAppsRevisionReplicasClientImpl.java |  233 +-
 .../ContainerAppsRevisionReplicasImpl.java    |   43 +-
 .../ContainerAppsRevisionsClientImpl.java     |  574 +-
 .../ContainerAppsRevisionsImpl.java           |   55 +-
 ...ContainerAppsSourceControlsClientImpl.java |  656 +-
 .../ContainerAppsSourceControlsImpl.java      |  136 +-
 .../CustomHostnameAnalysisResultImpl.java     |    3 +-
 .../implementation/DaprComponentImpl.java     |  100 +-
 ...ComponentResiliencyPoliciesClientImpl.java |  812 +++
 .../DaprComponentResiliencyPoliciesImpl.java  |  186 +
 .../DaprComponentResiliencyPolicyImpl.java    |  143 +
 .../DaprComponentsClientImpl.java             |  636 +-
 .../implementation/DaprComponentsImpl.java    |   76 +-
 .../DaprSecretsCollectionImpl.java            |    3 +-
 .../implementation/DaprSubscriptionImpl.java  |  194 +
 .../DaprSubscriptionsClientImpl.java          |  753 ++
 .../implementation/DaprSubscriptionsImpl.java |  158 +
 .../DiagnosticsCollectionImpl.java            |   11 +-
 .../implementation/DiagnosticsImpl.java       |    9 +-
 .../implementation/DotNetComponentImpl.java   |  158 +
 .../DotNetComponentsClientImpl.java           | 1238 ++++
 .../implementation/DotNetComponentsImpl.java  |  157 +
 .../EnvironmentAuthTokenImpl.java             |    3 +-
 .../implementation/JavaComponentImpl.java     |  158 +
 .../JavaComponentsClientImpl.java}            |  901 ++-
 .../implementation/JavaComponentsImpl.java    |  157 +
 .../implementation/JobExecutionBaseImpl.java  |    3 +-
 .../implementation/JobExecutionImpl.java      |    4 +-
 .../appcontainers/implementation/JobImpl.java |   60 +-
 .../JobSecretsCollectionImpl.java             |    3 +-
 .../implementation/JobsClientImpl.java        | 1851 ++---
 .../JobsExecutionsClientImpl.java             |  219 +-
 .../implementation/JobsExecutionsImpl.java    |    7 +-
 .../implementation/JobsImpl.java              |  175 +-
 .../ManagedCertificateImpl.java               |   71 +-
 .../ManagedCertificatesClientImpl.java        |  868 +--
 .../ManagedCertificatesImpl.java              |  156 +-
 ...nagedEnvironmentDiagnosticsClientImpl.java |  261 +-
 .../ManagedEnvironmentDiagnosticsImpl.java    |   29 +-
 .../ManagedEnvironmentImpl.java               |   95 +-
 .../ManagedEnvironmentStorageImpl.java        |   74 +-
 ...agedEnvironmentStoragesCollectionImpl.java |   11 +-
 .../ManagedEnvironmentUsagesClientImpl.java   |  283 +
 .../ManagedEnvironmentUsagesImpl.java         |   45 +
 .../ManagedEnvironmentsClientImpl.java        | 1420 ++--
 ...agedEnvironmentsDiagnosticsClientImpl.java |  136 +-
 .../ManagedEnvironmentsDiagnosticsImpl.java   |   16 +-
 .../ManagedEnvironmentsImpl.java              |  137 +-
 ...ManagedEnvironmentsStoragesClientImpl.java |  434 +-
 .../ManagedEnvironmentsStoragesImpl.java      |  153 +-
 .../implementation/NamespacesClientImpl.java  |  157 +-
 .../implementation/NamespacesImpl.java        |   33 +-
 .../implementation/OperationDetailImpl.java   |    3 +-
 .../implementation/OperationsClientImpl.java  |  159 +-
 .../implementation/OperationsImpl.java        |    8 +-
 .../implementation/ReplicaCollectionImpl.java |    8 +-
 .../implementation/ReplicaImpl.java           |    9 +-
 .../implementation/ResourceManagerUtils.java  |  197 +
 .../ResourceProvidersClientImpl.java          |  212 +-
 .../implementation/ResourceProvidersImpl.java |   24 +-
 .../implementation/RevisionImpl.java          |    9 +-
 .../implementation/SecretsCollectionImpl.java |    3 +-
 .../implementation/SourceControlImpl.java     |   60 +-
 .../implementation/UsageImpl.java             |   44 +
 .../implementation/UsagesClientImpl.java      |  262 +
 .../implementation/UsagesImpl.java            |   45 +
 .../appcontainers/implementation/Utils.java   |  204 -
 .../WorkloadProfileStatesImpl.java            |    8 +-
 .../implementation/package-info.java          |    5 +-
 .../appcontainers/models/AccessMode.java      |   18 +-
 .../appcontainers/models/Action.java          |   18 +-
 .../models/ActiveRevisionsMode.java           |   14 +-
 .../appcontainers/models/Affinity.java        |   18 +-
 .../models/AllowedAudiencesValidation.java    |   14 +-
 .../models/AllowedPrincipals.java             |   18 +-
 .../models/AppInsightsConfiguration.java      |   54 +
 .../models/AppLogsConfiguration.java          |   18 +-
 .../appcontainers/models/AppProtocol.java     |   18 +-
 .../appcontainers/models/AppRegistration.java |   18 +-
 .../appcontainers/models/AppResiliencies.java |  175 +
 .../appcontainers/models/AppResiliency.java   |  358 +
 .../models/AppResiliencyCollection.java       |   80 +
 .../appcontainers/models/Apple.java           |   22 +-
 .../models/AppleRegistration.java             |   18 +-
 .../appcontainers/models/Applicability.java   |   18 +-
 .../appcontainers/models/AuthConfig.java      |  200 +-
 .../models/AuthConfigCollection.java          |   21 +-
 .../appcontainers/models/AuthPlatform.java    |   28 +-
 .../models/AvailableOperations.java           |   26 +-
 .../models/AvailableWorkloadProfile.java      |   24 +-
 .../AvailableWorkloadProfileProperties.java   |   56 +-
 .../models/AvailableWorkloadProfiles.java     |   16 +-
 .../AvailableWorkloadProfilesCollection.java  |   22 +-
 .../models/AzureActiveDirectory.java          |   40 +-
 .../models/AzureActiveDirectoryLogin.java     |   34 +-
 .../AzureActiveDirectoryRegistration.java     |   98 +-
 .../AzureActiveDirectoryValidation.java       |   26 +-
 .../models/AzureCredentials.java              |   30 +-
 .../models/AzureFileProperties.java           |   26 +-
 .../models/AzureStaticWebApps.java            |   18 +-
 .../AzureStaticWebAppsRegistration.java       |   14 +-
 .../appcontainers/models/BaseContainer.java   |   38 +-
 .../appcontainers/models/BillingMeter.java    |   34 +-
 .../models/BillingMeterCollection.java        |    8 +-
 .../models/BillingMeterProperties.java        |   22 +-
 .../appcontainers/models/BillingMeters.java   |   20 +-
 .../appcontainers/models/BindingType.java     |   18 +-
 .../models/BlobStorageTokenStore.java         |   63 +
 .../appcontainers/models/BuildAuthTokens.java |   42 +
 .../appcontainers/models/BuildCollection.java |   91 +
 .../models/BuildConfiguration.java            |  167 +
 .../models/BuildProvisioningState.java        |   73 +
 .../appcontainers/models/BuildResource.java   |  261 +
 .../appcontainers/models/BuildStatus.java     |   68 +
 .../appcontainers/models/BuildToken.java      |   34 +
 .../models/BuilderCollection.java             |   91 +
 .../models/BuilderProvisioningState.java      |   73 +
 .../appcontainers/models/BuilderResource.java |  333 +
 .../models/BuilderResourceUpdate.java         |  130 +
 .../appcontainers/models/Builders.java}       |  132 +-
 .../appcontainers/models/Builds.java          |  118 +
 .../models/BuildsByBuilderResources.java}     |   27 +-
 .../appcontainers/models/Certificate.java     |   99 +-
 .../models/CertificateCollection.java         |   21 +-
 .../models/CertificateKeyVaultProperties.java |   83 +
 .../models/CertificatePatch.java              |   14 +-
 .../models/CertificateProperties.java         |   94 +-
 .../models/CertificateProvisioningState.java  |   30 +-
 .../appcontainers/models/CertificateType.java |   53 +
 .../appcontainers/models/Certificates.java    |   88 +-
 .../models/CheckNameAvailabilityReason.java   |   18 +-
 .../models/CheckNameAvailabilityRequest.java  |   18 +-
 .../models/CheckNameAvailabilityResponse.java |   12 +-
 .../models/CircuitBreakerPolicy.java          |  113 +
 .../models/ClientCredentialMethod.java        |   18 +-
 .../models/ClientRegistration.java            |   18 +-
 .../appcontainers/models/Configuration.java   |   52 +-
 .../models/ConnectedEnvironment.java          |  192 +-
 .../ConnectedEnvironmentCollection.java       |   16 +-
 ...ConnectedEnvironmentProvisioningState.java |   54 +-
 .../models/ConnectedEnvironmentStorage.java   |   76 +-
 ...ConnectedEnvironmentStorageProperties.java |   14 +-
 ...onnectedEnvironmentStoragesCollection.java |    8 +-
 .../models/ConnectedEnvironments.java         |  107 +-
 .../ConnectedEnvironmentsCertificates.java    |   54 +-
 .../ConnectedEnvironmentsDaprComponents.java  |   82 +-
 .../models/ConnectedEnvironmentsStorages.java |   58 +-
 .../appcontainers/models/Container.java       |   42 +-
 .../appcontainers/models/ContainerApp.java    |  290 +-
 .../models/ContainerAppAuthToken.java         |   22 +-
 .../models/ContainerAppCollection.java        |   21 +-
 .../ContainerAppContainerRunningState.java    |   22 +-
 .../models/ContainerAppJobExecutions.java     |   10 +-
 .../models/ContainerAppProbe.java             |   82 +-
 .../models/ContainerAppProbeHttpGet.java      |   30 +-
 ...ntainerAppProbeHttpGetHttpHeadersItem.java |   30 +-
 .../models/ContainerAppProbeTcpSocket.java    |   18 +-
 .../models/ContainerAppProvisioningState.java |   30 +-
 .../ContainerAppReplicaRunningState.java      |   22 +-
 .../models/ContainerAppSecret.java            |   18 +-
 .../appcontainers/models/ContainerApps.java   |  122 +-
 .../models/ContainerAppsAuthConfigs.java      |   54 +-
 .../models/ContainerAppsDiagnostics.java      |   60 +-
 .../models/ContainerAppsRevisionReplicas.java |   28 +-
 .../models/ContainerAppsRevisions.java        |   64 +-
 .../models/ContainerAppsSourceControls.java   |   58 +-
 .../models/ContainerRegistry.java             |   91 +
 .../ContainerRegistryWithCustomImage.java     |   90 +
 .../models/ContainerResources.java            |   20 +-
 .../models/CookieExpiration.java              |   18 +-
 .../models/CookieExpirationConvention.java    |   22 +-
 .../appcontainers/models/CorsPolicy.java      |   39 +-
 .../appcontainers/models/CustomDomain.java    |   27 +-
 .../models/CustomDomainConfiguration.java     |   60 +-
 .../models/CustomHostnameAnalysisResult.java  |   30 +-
 ...ltCustomDomainVerificationFailureInfo.java |   24 +-
 ...ainVerificationFailureInfoDetailsItem.java |   16 +-
 .../models/CustomOpenIdConnectProvider.java   |   22 +-
 .../appcontainers/models/CustomScaleRule.java |   28 +-
 .../appcontainers/models/Dapr.java            |   50 +-
 .../appcontainers/models/DaprComponent.java   |  240 +-
 .../DaprComponentResiliencyPolicies.java      |  174 +
 ...ComponentResiliencyPoliciesCollection.java |   80 +
 .../models/DaprComponentResiliencyPolicy.java |  224 +
 ...licyCircuitBreakerPolicyConfiguration.java |  114 +
 ...omponentResiliencyPolicyConfiguration.java |  118 +
 ...cyPolicyHttpRetryBackOffConfiguration.java |   86 +
 ...ncyPolicyHttpRetryPolicyConfiguration.java |   84 +
 ...iencyPolicyTimeoutPolicyConfiguration.java |   55 +
 .../models/DaprComponentServiceBinding.java   |  109 +
 .../appcontainers/models/DaprComponents.java  |   85 +-
 .../models/DaprComponentsCollection.java      |   21 +-
 .../models/DaprConfiguration.java             |   12 +-
 .../appcontainers/models/DaprMetadata.java    |   22 +-
 .../appcontainers/models/DaprSecret.java      |   14 +-
 .../models/DaprSecretsCollection.java         |    8 +-
 .../models/DaprServiceBindMetadata.java}      |   48 +-
 .../models/DaprSubscription.java              |  392 +
 .../DaprSubscriptionBulkSubscribeOptions.java |  108 +
 .../models/DaprSubscriptionRouteRule.java     |   86 +
 .../models/DaprSubscriptionRoutes.java        |   84 +
 .../models/DaprSubscriptions.java             |  164 +
 .../models/DaprSubscriptionsCollection.java   |   80 +
 .../models/DataDogConfiguration.java          |   80 +
 .../models/DefaultAuthorizationPolicy.java    |   18 +-
 .../models/DefaultErrorResponseError.java     |   12 +-
 .../DefaultErrorResponseErrorException.java   |   12 +-
 .../models/DestinationsConfiguration.java     |   87 +
 .../DiagnosticDataProviderMetadata.java       |   22 +-
 ...icDataProviderMetadataPropertyBagItem.java |   18 +-
 .../DiagnosticDataTableResponseColumn.java    |   22 +-
 .../DiagnosticDataTableResponseObject.java    |   22 +-
 .../models/DiagnosticRendering.java           |   26 +-
 .../models/DiagnosticSupportTopic.java        |   14 +-
 .../appcontainers/models/Diagnostics.java     |   22 +-
 .../models/DiagnosticsCollection.java         |   10 +-
 .../models/DiagnosticsDataApiResponse.java    |   18 +-
 .../models/DiagnosticsDefinition.java         |   32 +-
 .../models/DiagnosticsProperties.java         |   26 +-
 .../models/DiagnosticsStatus.java             |   18 +-
 .../models/DnsVerificationTestResult.java     |   26 +-
 .../appcontainers/models/DotNetComponent.java |  265 +
 .../DotNetComponentConfigurationProperty.java |   80 +
 .../DotNetComponentProvisioningState.java     |   68 +
 .../models/DotNetComponentServiceBind.java    |   80 +
 .../models/DotNetComponentType.java           |   53 +
 .../models/DotNetComponents.java}             |  100 +-
 .../models/DotNetComponentsCollection.java    |   80 +
 .../models/EncryptionSettings.java            |   81 +
 .../models/EnvironmentAuthToken.java          |   22 +-
 .../models/EnvironmentProvisioningState.java  |   62 +-
 .../appcontainers/models/EnvironmentVar.java  |   22 +-
 .../models/EnvironmentVariable.java           |   91 +
 .../models/ExtendedLocation.java              |   18 +-
 .../models/ExtendedLocationTypes.java         |   14 +-
 .../appcontainers/models/Facebook.java        |   26 +-
 .../appcontainers/models/ForwardProxy.java    |   22 +-
 .../models/ForwardProxyConvention.java        |   26 +-
 .../appcontainers/models/GitHub.java          |   30 +-
 .../models/GithubActionConfiguration.java     |   77 +-
 .../models/GlobalValidation.java              |   24 +-
 .../appcontainers/models/Google.java          |   34 +-
 .../appcontainers/models/Header.java          |   80 +
 .../appcontainers/models/HeaderMatch.java     |  165 +
 .../models/HttpConnectionPool.java            |   80 +
 .../appcontainers/models/HttpGet.java         |  114 +
 .../appcontainers/models/HttpRetryPolicy.java |  208 +
 .../appcontainers/models/HttpScaleRule.java   |   18 +-
 .../appcontainers/models/HttpSettings.java    |   18 +-
 .../models/HttpSettingsRoutes.java            |   14 +-
 .../models/IdentityProviders.java             |   61 +-
 .../appcontainers/models/Ingress.java         |  111 +-
 .../models/IngressClientCertificateMode.java  |   18 +-
 .../models/IngressPortMapping.java            |  108 +
 .../models/IngressStickySessions.java         |   14 +-
 .../models/IngressTargetPortHttpScheme.java   |   53 +
 .../models/IngressTransportMethod.java        |   26 +-
 .../appcontainers/models/InitContainer.java   |   38 +-
 .../models/IpSecurityRestrictionRule.java     |   43 +-
 .../appcontainers/models/JavaComponent.java   |  265 +
 .../JavaComponentConfigurationProperty.java   |   80 +
 .../JavaComponentProvisioningState.java       |   68 +
 .../models/JavaComponentServiceBind.java      |   80 +
 .../models/JavaComponentType.java             |   58 +
 .../appcontainers/models/JavaComponents.java} |  125 +-
 .../models/JavaComponentsCollection.java      |   80 +
 .../appcontainers/models/Job.java             |  223 +-
 .../models/JobConfiguration.java              |   47 +-
 .../JobConfigurationEventTriggerConfig.java   |   22 +-
 .../JobConfigurationManualTriggerConfig.java  |   14 +-
 ...JobConfigurationScheduleTriggerConfig.java |   28 +-
 .../appcontainers/models/JobExecution.java    |   22 +-
 .../models/JobExecutionBase.java              |   10 +-
 .../models/JobExecutionContainer.java         |   34 +-
 .../models/JobExecutionRunningState.java      |   38 +-
 .../models/JobExecutionTemplate.java          |   18 +-
 .../models/JobPatchProperties.java            |   59 +-
 .../models/JobPatchPropertiesProperties.java  |   30 +-
 .../models/JobProvisioningState.java          |   30 +-
 .../appcontainers/models/JobScale.java        |   26 +-
 .../appcontainers/models/JobScaleRule.java    |   32 +-
 .../models/JobSecretsCollection.java          |    8 +-
 .../appcontainers/models/JobTemplate.java     |   18 +-
 .../appcontainers/models/Jobs.java            |  183 +-
 .../appcontainers/models/JobsCollection.java  |   21 +-
 .../appcontainers/models/JobsExecutions.java  |   12 +-
 .../appcontainers/models/JwtClaimChecks.java  |   18 +-
 .../models/KedaConfiguration.java             |   12 +-
 .../models/ListUsagesResult.java              |   88 +
 .../models/LogAnalyticsConfiguration.java     |   46 +-
 .../appcontainers/models/LogLevel.java        |   26 +-
 .../appcontainers/models/Login.java           |   71 +-
 .../appcontainers/models/LoginRoutes.java     |   14 +-
 .../appcontainers/models/LoginScopes.java     |   14 +-
 .../models/LogsConfiguration.java             |   55 +
 .../models/ManagedCertificate.java            |   99 +-
 .../models/ManagedCertificateCollection.java  |   22 +-
 ...gedCertificateDomainControlValidation.java |   22 +-
 .../models/ManagedCertificatePatch.java       |   14 +-
 .../models/ManagedCertificateProperties.java  |   28 +-
 .../models/ManagedCertificates.java           |   54 +-
 .../models/ManagedEnvironment.java            |  450 +-
 .../models/ManagedEnvironmentDiagnostics.java |   48 +-
 ...vironmentPropertiesPeerAuthentication.java |   14 +-
 .../models/ManagedEnvironmentStorage.java     |   76 +-
 .../ManagedEnvironmentStorageProperties.java  |   43 +-
 .../ManagedEnvironmentStoragesCollection.java |    8 +-
 .../models/ManagedEnvironmentUsages.java      |   42 +
 .../models/ManagedEnvironments.java           |  146 +-
 .../models/ManagedEnvironmentsCollection.java |   22 +-
 .../ManagedEnvironmentsDiagnostics.java       |   20 +-
 .../models/ManagedEnvironmentsStorages.java   |   58 +-
 .../models/ManagedServiceIdentity.java        |   47 +-
 .../models/ManagedServiceIdentityType.java    |   30 +-
 .../models/MetricsConfiguration.java          |   55 +
 .../appcontainers/models/Mtls.java            |   14 +-
 .../appcontainers/models/Namespaces.java      |   31 +-
 .../models/NfsAzureFileProperties.java        |  106 +
 .../appcontainers/models/Nonce.java           |   18 +-
 .../models/OpenIdConnectClientCredential.java |   18 +-
 .../models/OpenIdConnectConfig.java           |   38 +-
 .../models/OpenIdConnectLogin.java            |   18 +-
 .../models/OpenIdConnectRegistration.java     |   30 +-
 .../models/OpenTelemetryConfiguration.java    |  145 +
 .../appcontainers/models/OperationDetail.java |   14 +-
 .../models/OperationDisplay.java              |   26 +-
 .../appcontainers/models/Operations.java      |   12 +-
 .../models/OtlpConfiguration.java             |  136 +
 .../appcontainers/models/PreBuildStep.java    |  110 +
 .../appcontainers/models/QueueScaleRule.java  |   22 +-
 .../models/RegistryCredentials.java           |   26 +-
 .../appcontainers/models/RegistryInfo.java    |   22 +-
 .../appcontainers/models/Replica.java         |   30 +-
 .../models/ReplicaCollection.java             |    8 +-
 .../models/ReplicaContainer.java              |   38 +-
 .../models/ResourceProviders.java             |   38 +-
 .../appcontainers/models/Revision.java        |   50 +-
 .../models/RevisionCollection.java            |   21 +-
 .../models/RevisionHealthState.java           |   22 +-
 .../models/RevisionProvisioningState.java     |   30 +-
 .../models/RevisionRunningState.java          |   34 +-
 .../appcontainers/models/Scale.java           |   22 +-
 .../appcontainers/models/ScaleRule.java       |   30 +-
 .../appcontainers/models/ScaleRuleAuth.java   |   18 +-
 .../appcontainers/models/Scheme.java          |   18 +-
 .../appcontainers/models/Secret.java          |   26 +-
 .../models/SecretVolumeItem.java              |   18 +-
 .../models/SecretsCollection.java             |    8 +-
 .../appcontainers/models/Service.java         |   14 +-
 .../appcontainers/models/ServiceBind.java     |   73 +-
 .../appcontainers/models/SourceControl.java   |  131 +-
 .../models/SourceControlCollection.java       |   21 +-
 .../models/SourceControlOperationState.java   |   26 +-
 .../appcontainers/models/StorageType.java     |   27 +-
 .../models/TcpConnectionPool.java             |   54 +
 .../appcontainers/models/TcpRetryPolicy.java  |   54 +
 .../appcontainers/models/TcpScaleRule.java    |   18 +-
 .../appcontainers/models/Template.java        |   39 +-
 .../appcontainers/models/TimeoutPolicy.java   |   80 +
 .../appcontainers/models/TokenStore.java      |  122 +
 .../models/TracesConfiguration.java           |   55 +
 .../appcontainers/models/TrafficWeight.java   |   26 +-
 .../appcontainers/models/TriggerType.java     |   22 +-
 .../appcontainers/models/Twitter.java         |   18 +-
 .../models/TwitterRegistration.java           |   36 +-
 .../appcontainers/models/Type.java            |   22 +-
 .../models/UnauthenticatedClientActionV2.java |   30 +-
 .../appcontainers/models/Usage.java           |   47 +
 .../appcontainers/models/UsageName.java       |   80 +
 .../appcontainers/models/Usages.java          |   40 +
 .../models/UserAssignedIdentity.java          |   14 +-
 .../models/VnetConfiguration.java             |   38 +-
 .../appcontainers/models/Volume.java          |   39 +-
 .../appcontainers/models/VolumeMount.java     |   36 +-
 .../appcontainers/models/WorkloadProfile.java |   37 +-
 .../models/WorkloadProfileStates.java         |   22 +-
 .../WorkloadProfileStatesCollection.java      |   22 +-
 .../WorkloadProfileStatesProperties.java      |   22 +-
 .../appcontainers/models/package-info.java    |    5 +-
 .../appcontainers/package-info.java           |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       | 1486 ++++
 .../AppResiliencyCreateOrUpdateSamples.java   |   47 +
 .../generated/AppResiliencyDeleteSamples.java |   24 +
 .../generated/AppResiliencyGetSamples.java    |   24 +
 .../generated/AppResiliencyListSamples.java   |   23 +
 .../generated/AppResiliencyUpdateSamples.java |   31 +
 .../AvailableWorkloadProfilesGetSamples.java  |    9 +-
 .../generated/BillingMetersGetSamples.java    |    9 +-
 .../generated/BuildAuthTokenListSamples.java  |   23 +
 .../BuildersCreateOrUpdateSamples.java        |   58 +
 .../generated/BuildersDeleteSamples.java      |   23 +
 .../BuildersGetByResourceGroupSamples.java    |   23 +
 .../BuildersListByResourceGroupSamples.java   |   25 +
 .../generated/BuildersListSamples.java        |   25 +
 .../generated/BuildersUpdateSamples.java      |   41 +
 .../BuildsByBuilderResourceListSamples.java   |   25 +
 .../BuildsCreateOrUpdateSamples.java          |   63 +
 .../generated/BuildsDeleteSamples.java        |   23 +
 .../generated/BuildsGetSamples.java           |   23 +
 .../CertificatesCreateOrUpdateSamples.java    |   54 +-
 .../generated/CertificatesDeleteSamples.java  |   15 +-
 .../generated/CertificatesGetSamples.java     |   15 +-
 .../generated/CertificatesListSamples.java    |   13 +-
 .../generated/CertificatesUpdateSamples.java  |   20 +-
 ...entsCertificatesCreateOrUpdateSamples.java |   18 +-
 ...EnvironmentsCertificatesDeleteSamples.java |   15 +-
 ...tedEnvironmentsCertificatesGetSamples.java |   15 +-
 ...edEnvironmentsCertificatesListSamples.java |   14 +-
 ...EnvironmentsCertificatesUpdateSamples.java |   17 +-
 ...ironmentsCheckNameAvailabilitySamples.java |   24 +-
 ...ctedEnvironmentsCreateOrUpdateSamples.java |   22 +-
 ...tsDaprComponentsCreateOrUpdateSamples.java |   41 +-
 ...vironmentsDaprComponentsDeleteSamples.java |   14 +-
 ...dEnvironmentsDaprComponentsGetSamples.java |   14 +-
 ...EnvironmentsDaprComponentsListSamples.java |   14 +-
 ...mentsDaprComponentsListSecretsSamples.java |   18 +-
 .../ConnectedEnvironmentsDeleteSamples.java   |   10 +-
 ...EnvironmentsGetByResourceGroupSamples.java |   15 +-
 ...nvironmentsListByResourceGroupSamples.java |   13 +-
 .../ConnectedEnvironmentsListSamples.java     |    9 +-
 ...ronmentsStoragesCreateOrUpdateSamples.java |   28 +-
 ...ctedEnvironmentsStoragesDeleteSamples.java |   14 +-
 ...nnectedEnvironmentsStoragesGetSamples.java |   14 +-
 ...nectedEnvironmentsStoragesListSamples.java |   14 +-
 .../ConnectedEnvironmentsUpdateSamples.java   |   14 +-
 ...rAppsAuthConfigsCreateOrUpdateSamples.java |   32 +-
 ...ContainerAppsAuthConfigsDeleteSamples.java |   18 +-
 .../ContainerAppsAuthConfigsGetSamples.java   |   18 +-
 ...sAuthConfigsListByContainerAppSamples.java |   19 +-
 .../ContainerAppsCreateOrUpdateSamples.java   |  398 +-
 .../generated/ContainerAppsDeleteSamples.java |    9 +-
 ...inerAppsDiagnosticsGetDetectorSamples.java |   23 +-
 ...inerAppsDiagnosticsGetRevisionSamples.java |   19 +-
 ...ontainerAppsDiagnosticsGetRootSamples.java |   14 +-
 ...erAppsDiagnosticsListDetectorsSamples.java |   14 +-
 ...erAppsDiagnosticsListRevisionsSamples.java |   18 +-
 .../ContainerAppsGetAuthTokenSamples.java     |   14 +-
 ...ontainerAppsGetByResourceGroupSamples.java |   14 +-
 ...ntainerAppsListByResourceGroupSamples.java |   13 +-
 ...AppsListCustomHostnameAnalysisSamples.java |   15 +-
 .../generated/ContainerAppsListSamples.java   |   13 +-
 .../ContainerAppsListSecretsSamples.java      |   14 +-
 ...AppsRevisionReplicasGetReplicaSamples.java |   23 +-
 ...psRevisionReplicasListReplicasSamples.java |   18 +-
 ...rAppsRevisionsActivateRevisionSamples.java |   19 +-
 ...ppsRevisionsDeactivateRevisionSamples.java |   19 +-
 ...tainerAppsRevisionsGetRevisionSamples.java |   19 +-
 ...inerAppsRevisionsListRevisionsSamples.java |   18 +-
 ...erAppsRevisionsRestartRevisionSamples.java |   19 +-
 ...psSourceControlsCreateOrUpdateSamples.java |   43 +-
 ...tainerAppsSourceControlsDeleteSamples.java |   18 +-
 ...ContainerAppsSourceControlsGetSamples.java |   18 +-
 ...urceControlsListByContainerAppSamples.java |   14 +-
 .../generated/ContainerAppsStartSamples.java  |    9 +-
 .../generated/ContainerAppsStopSamples.java   |    9 +-
 .../generated/ContainerAppsUpdateSamples.java |  161 +-
 ...siliencyPoliciesCreateOrUpdateSamples.java |   96 +
 ...ponentResiliencyPoliciesDeleteSamples.java |   25 +
 ...ComponentResiliencyPoliciesGetSamples.java |   25 +
 ...omponentResiliencyPoliciesListSamples.java |   25 +
 .../DaprComponentsCreateOrUpdateSamples.java  |   83 +-
 .../DaprComponentsDeleteSamples.java          |   14 +-
 .../generated/DaprComponentsGetSamples.java   |   29 +-
 .../generated/DaprComponentsListSamples.java  |    9 +-
 .../DaprComponentsListSecretsSamples.java     |   19 +-
 ...aprSubscriptionsCreateOrUpdateSamples.java |   87 +
 .../DaprSubscriptionsDeleteSamples.java       |   25 +
 .../DaprSubscriptionsGetSamples.java          |   55 +
 .../DaprSubscriptionsListSamples.java         |   23 +
 ...DotNetComponentsCreateOrUpdateSamples.java |   55 +
 .../DotNetComponentsDeleteSamples.java        |   24 +
 .../generated/DotNetComponentsGetSamples.java |   39 +
 .../DotNetComponentsListSamples.java          |   37 +
 .../DotNetComponentsUpdateSamples.java        |   57 +
 .../JavaComponentsCreateOrUpdateSamples.java  |   59 +
 .../JavaComponentsDeleteSamples.java          |   24 +
 .../generated/JavaComponentsGetSamples.java   |   39 +
 .../generated/JavaComponentsListSamples.java  |   37 +
 .../JavaComponentsUpdateSamples.java          |   63 +
 .../generated/JobsCreateOrUpdateSamples.java  |  187 +-
 .../generated/JobsDeleteSamples.java          |    9 +-
 .../generated/JobsExecutionsListSamples.java  |   13 +-
 .../JobsGetByResourceGroupSamples.java        |    9 +-
 .../generated/JobsGetDetectorSamples.java     |   25 +
 .../JobsListByResourceGroupSamples.java       |   14 +-
 .../generated/JobsListDetectorsSamples.java   |   25 +
 .../generated/JobsListSamples.java            |   13 +-
 .../generated/JobsListSecretsSamples.java     |   13 +-
 .../generated/JobsProxyGetSamples.java        |   24 +
 .../generated/JobsStartSamples.java           |   42 +-
 .../generated/JobsStopExecutionSamples.java   |   13 +-
 .../JobsStopMultipleExecutionsSamples.java    |   13 +-
 .../generated/JobsUpdateSamples.java          |   76 +-
 ...agedCertificatesCreateOrUpdateSamples.java |   24 +-
 .../ManagedCertificatesDeleteSamples.java     |   16 +-
 .../ManagedCertificatesGetSamples.java        |   15 +-
 .../ManagedCertificatesListSamples.java       |    9 +-
 .../ManagedCertificatesUpdateSamples.java     |   22 +-
 ...ironmentDiagnosticsGetDetectorSamples.java |   18 +-
 ...onmentDiagnosticsListDetectorsSamples.java |   14 +-
 .../ManagedEnvironmentUsagesListSamples.java  |   24 +
 ...agedEnvironmentsCreateOrUpdateSamples.java |  164 +-
 .../ManagedEnvironmentsDeleteSamples.java     |   14 +-
 ...EnvironmentsDiagnosticsGetRootSamples.java |   14 +-
 ...anagedEnvironmentsGetAuthTokenSamples.java |   13 +-
 ...EnvironmentsGetByResourceGroupSamples.java |   14 +-
 ...nvironmentsListByResourceGroupSamples.java |   13 +-
 .../ManagedEnvironmentsListSamples.java       |   13 +-
 ...mentsListWorkloadProfileStatesSamples.java |   18 +-
 ...ronmentsStoragesCreateOrUpdateSamples.java |   47 +-
 ...agedEnvironmentsStoragesDeleteSamples.java |   14 +-
 ...ManagedEnvironmentsStoragesGetSamples.java |   33 +-
 ...anagedEnvironmentsStoragesListSamples.java |   14 +-
 .../ManagedEnvironmentsUpdateSamples.java     |   22 +-
 ...amespacesCheckNameAvailabilitySamples.java |   43 +-
 .../generated/OperationsListSamples.java      |    9 +-
 ...rGetCustomDomainVerificationIdSamples.java |   23 +
 .../ResourceProviderJobExecutionSamples.java  |   14 +-
 .../generated/UsagesListSamples.java          |   23 +
 .../AllowedAudiencesValidationTests.java      |   13 +-
 .../generated/AllowedPrincipalsTests.java     |   21 +-
 .../AppInsightsConfigurationTests.java        |   25 +
 ...ilienciesDeleteWithResponseMockTests.java} |   10 +-
 .../generated/AuthConfigCollectionTests.java  |  364 -
 .../generated/AuthPlatformTests.java          |   10 +-
 .../generated/AvailableOperationsTests.java   |   67 +-
 .../AvailableWorkloadProfileInnerTests.java   |   47 +-
 ...ailableWorkloadProfilePropertiesTests.java |   40 +-
 ...ilableWorkloadProfilesCollectionTests.java |   79 +-
 ...AvailableWorkloadProfilesGetMockTests.java |   51 +-
 .../AzureActiveDirectoryLoginTests.java       |   18 +-
 .../AzureStaticWebAppsRegistrationTests.java  |   10 +-
 .../generated/AzureStaticWebAppsTests.java    |   19 +-
 .../BillingMeterCollectionInnerTests.java     |   62 +-
 .../BillingMeterPropertiesTests.java          |   23 +-
 .../generated/BillingMeterTests.java          |   35 +-
 ...BillingMetersGetWithResponseMockTests.java |   51 +-
 .../generated/BlobStorageTokenStoreTests.java |   25 +
 .../generated/BuildConfigurationTests.java    |   67 +
 .../generated/BuilderCollectionTests.java     |  108 +
 .../generated/BuilderPropertiesTests.java     |   37 +
 .../generated/BuilderResourceInnerTests.java  |   67 +
 .../BuilderResourceUpdatePropertiesTests.java |   25 +
 .../generated/BuilderResourceUpdateTests.java |   52 +
 .../BuildersCreateOrUpdateMockTests.java      |   88 +
 ...tByResourceGroupWithResponseMockTests.java |   62 +
 .../BuildersListByResourceGroupMockTests.java |   65 +
 .../generated/BuildersListMockTests.java      |   64 +
 .../generated/CertificatePatchTests.java      |   16 +-
 ...rtificatesDeleteWithResponseMockTests.java |   39 +-
 .../CheckNameAvailabilityRequestTests.java    |   16 +-
 ...eckNameAvailabilityResponseInnerTests.java |   20 +-
 .../generated/CircuitBreakerPolicyTests.java  |   32 +
 ...rtificatesDeleteWithResponseMockTests.java |   39 +-
 ...NameAvailabilityWithResponseMockTests.java |   56 +-
 ...ComponentsDeleteWithResponseMockTests.java |   39 +-
 ...nentsListSecretsWithResponseMockTests.java |   41 +-
 .../ConnectedEnvironmentsDeleteMockTests.java |   36 +-
 ...tsStoragesDeleteWithResponseMockTests.java |   39 +-
 .../ContainerAppJobExecutionsInnerTests.java  |  200 +-
 ...erAppProbeHttpGetHttpHeadersItemTests.java |   17 +-
 .../ContainerAppProbeHttpGetTests.java        |   47 +-
 .../ContainerAppProbeTcpSocketTests.java      |   18 +-
 .../generated/ContainerAppProbeTests.java     |   94 +-
 ...uthConfigsDeleteWithResponseMockTests.java |   38 +-
 .../ContainerAppsDeleteMockTests.java         |   36 +-
 ...sticsGetDetectorWithResponseMockTests.java |   65 +-
 ...AppsDiagnosticsListDetectorsMockTests.java |   81 +-
 ...plicasGetReplicaWithResponseMockTests.java |   54 +-
 ...icasListReplicasWithResponseMockTests.java |   43 +-
 ...ActivateRevisionWithResponseMockTests.java |   39 +-
 ...activateRevisionWithResponseMockTests.java |   39 +-
 ...sRestartRevisionWithResponseMockTests.java |   39 +-
 ...inerAppsSourceControlsDeleteMockTests.java |   39 +-
 .../generated/ContainerRegistryTests.java     |   28 +
 ...ContainerRegistryWithCustomImageTests.java |   29 +
 .../generated/ContainerResourcesTests.java    |   18 +-
 .../generated/CookieExpirationTests.java      |   17 +-
 .../generated/CustomDomainTests.java          |   26 +-
 ...nentResiliencyPoliciesCollectionTests.java |  144 +
 ...esCreateOrUpdateWithResponseMockTests.java |   97 +
 ...cyPoliciesDeleteWithResponseMockTests.java |   52 +
 ...iencyPoliciesGetWithResponseMockTests.java |   74 +
 ...ponentResiliencyPoliciesListMockTests.java |   82 +
 ...ircuitBreakerPolicyConfigurationTests.java |   34 +
 ...entResiliencyPolicyConfigurationTests.java |   50 +
 ...icyHttpRetryBackOffConfigurationTests.java |   30 +
 ...licyHttpRetryPolicyConfigurationTests.java |   34 +
 ...prComponentResiliencyPolicyInnerTests.java |   85 +
 ...ponentResiliencyPolicyPropertiesTests.java |   85 +
 ...PolicyTimeoutPolicyConfigurationTests.java |   27 +
 .../DaprComponentServiceBindingTests.java     |   35 +
 ...ComponentsDeleteWithResponseMockTests.java |   39 +-
 ...nentsListSecretsWithResponseMockTests.java |   40 +-
 .../generated/DaprConfigurationTests.java     |    3 +-
 .../generated/DaprSecretTests.java            |    4 +-
 .../DaprSecretsCollectionInnerTests.java      |   13 +-
 .../DaprServiceBindMetadataTests.java         |   28 +
 ...SubscriptionBulkSubscribeOptionsTests.java |   31 +
 .../generated/DaprSubscriptionInnerTests.java |   72 +
 .../DaprSubscriptionPropertiesTests.java      |   75 +
 .../DaprSubscriptionRouteRuleTests.java       |   27 +
 .../DaprSubscriptionRoutesTests.java          |   35 +
 .../DaprSubscriptionsCollectionTests.java     |  107 +
 ...nsCreateOrUpdateWithResponseMockTests.java |   94 +
 ...scriptionsDeleteWithResponseMockTests.java |   52 +
 ...SubscriptionsGetWithResponseMockTests.java |   66 +
 .../DaprSubscriptionsListMockTests.java       |   67 +
 .../appcontainers/generated/DaprTests.java    |   53 +-
 .../DefaultAuthorizationPolicyTests.java      |   31 +-
 .../DefaultErrorResponseErrorTests.java       |    4 +-
 ...aProviderMetadataPropertyBagItemTests.java |   17 +-
 .../DiagnosticDataProviderMetadataTests.java  |   37 +-
 ...iagnosticDataTableResponseColumnTests.java |   24 +-
 ...iagnosticDataTableResponseObjectTests.java |   58 +-
 .../generated/DiagnosticRenderingTests.java   |   28 +-
 .../DiagnosticSupportTopicTests.java          |    6 +-
 .../DiagnosticsCollectionInnerTests.java      |  165 +-
 .../DiagnosticsDataApiResponseTests.java      |   73 +-
 .../generated/DiagnosticsDefinitionTests.java |   19 +-
 .../generated/DiagnosticsInnerTests.java      |  186 +-
 .../generated/DiagnosticsPropertiesTests.java |  146 +-
 .../generated/DiagnosticsStatusTests.java     |   14 +-
 ...etComponentConfigurationPropertyTests.java |   29 +
 .../generated/DotNetComponentInnerTests.java  |   45 +
 .../DotNetComponentPropertiesTests.java       |   46 +
 .../DotNetComponentServiceBindTests.java      |   27 +
 .../DotNetComponentsCollectionTests.java      |   45 +
 ...tNetComponentsCreateOrUpdateMockTests.java |   75 +
 ...NetComponentsGetWithResponseMockTests.java |   61 +
 .../DotNetComponentsListMockTests.java        |   62 +
 .../generated/EnvironmentVariableTests.java   |   27 +
 .../generated/ExtendedLocationTests.java      |   11 +-
 .../generated/ForwardProxyTests.java          |   27 +-
 .../generated/GlobalValidationTests.java      |   30 +-
 .../generated/HeaderMatchMatchTests.java      |   33 +
 .../generated/HeaderMatchTests.java           |   35 +
 .../generated/HttpConnectionPoolTests.java    |   29 +
 .../appcontainers/generated/HttpGetTests.java |   33 +
 .../HttpRetryPolicyRetryBackOffTests.java     |   29 +
 .../generated/HttpSettingsRoutesTests.java    |   10 +-
 .../generated/HttpSettingsTests.java          |   33 +-
 .../generated/IngressPortMappingTests.java    |   31 +
 .../generated/IngressStickySessionsTests.java |   10 +-
 .../IpSecurityRestrictionRuleTests.java       |   32 +-
 ...vaComponentConfigurationPropertyTests.java |   29 +
 .../generated/JavaComponentInnerTests.java    |   44 +
 .../JavaComponentPropertiesTests.java         |   42 +
 .../JavaComponentServiceBindTests.java        |   28 +
 .../JavaComponentsCollectionTests.java        |   53 +
 ...JavaComponentsCreateOrUpdateMockTests.java |   74 +
 ...avaComponentsGetWithResponseMockTests.java |   62 +
 .../JavaComponentsListMockTests.java          |   62 +
 ...ConfigurationManualTriggerConfigTests.java |   17 +-
 ...nfigurationScheduleTriggerConfigTests.java |   27 +-
 .../generated/JobExecutionBaseInnerTests.java |   14 +-
 .../generated/JobExecutionInnerTests.java     |   92 +
 .../generated/JobsDeleteMockTests.java        |   36 +-
 .../JobsExecutionsListMockTests.java          |   78 +-
 .../JobsGetDetectorWithResponseMockTests.java |   66 +
 ...obsListDetectorsWithResponseMockTests.java |   59 +
 .../generated/JobsStopExecutionMockTests.java |   36 +-
 .../JobsStopMultipleExecutionsMockTests.java  |   53 +-
 .../generated/JwtClaimChecksTests.java        |   22 +-
 .../generated/KedaConfigurationTests.java     |    4 +-
 .../generated/ListUsagesResultTests.java      |   40 +
 .../generated/LoginRoutesTests.java           |    9 +-
 .../generated/LoginScopesTests.java           |    9 +-
 .../appcontainers/generated/LoginTests.java   |   56 -
 .../generated/LogsConfigurationTests.java     |   26 +
 .../ManagedCertificatePatchTests.java         |   10 +-
 ...rtificatesDeleteWithResponseMockTests.java |   39 +-
 ...sticsGetDetectorWithResponseMockTests.java |   65 +-
 ...icsListDetectorsWithResponseMockTests.java |   55 +-
 ...mentPropertiesPeerAuthenticationTests.java |   10 +-
 ...ManagedEnvironmentUsagesListMockTests.java |   60 +
 .../ManagedEnvironmentsDeleteMockTests.java   |   36 +-
 ...ntsListWorkloadProfileStatesMockTests.java |   45 +-
 ...tsStoragesDeleteWithResponseMockTests.java |   39 +-
 .../ManagedServiceIdentityTests.java          |   19 +-
 .../generated/MetricsConfigurationTests.java  |   28 +
 ...NameAvailabilityWithResponseMockTests.java |   49 +-
 .../NfsAzureFilePropertiesTests.java          |   32 +
 .../appcontainers/generated/NonceTests.java   |   12 +-
 .../generated/OpenIdConnectLoginTests.java    |   22 +-
 .../generated/OperationDetailInnerTests.java  |   51 +-
 .../generated/OperationDisplayTests.java      |   32 +-
 .../generated/OperationsListMockTests.java    |   47 +-
 .../generated/PreBuildStepTests.java          |   38 +
 .../ReplicaCollectionInnerTests.java          |  130 +-
 .../generated/ReplicaContainerTests.java      |   33 +-
 .../generated/ReplicaInnerTests.java          |   81 +-
 .../generated/ReplicaPropertiesTests.java     |   93 +-
 ...inVerificationIdWithResponseMockTests.java |   54 +
 ...dersJobExecutionWithResponseMockTests.java |   71 +-
 .../generated/RevisionCollectionTests.java    |   12 +-
 .../generated/ServiceBindTests.java           |   27 -
 .../appcontainers/generated/ServiceTests.java |    8 +-
 .../generated/TcpConnectionPoolTests.java     |   25 +
 .../generated/TcpRetryPolicyTests.java        |   25 +
 .../generated/TimeoutPolicyTests.java         |   29 +
 .../generated/TracesConfigurationTests.java   |   26 +
 .../generated/TrafficWeightTests.java         |   28 +-
 .../generated/UsageInnerTests.java            |   34 +
 .../generated/UsageNameTests.java             |   27 +
 .../generated/UsagesListMockTests.java        |   59 +
 .../generated/UserAssignedIdentityTests.java  |    8 +-
 .../generated/VnetConfigurationTests.java     |   33 +-
 .../generated/VolumeMountTests.java           |   23 +-
 .../WorkloadProfileStatesCollectionTests.java |   40 +-
 .../WorkloadProfileStatesInnerTests.java      |   30 +-
 .../WorkloadProfileStatesPropertiesTests.java |   26 +-
 .../generated/WorkloadProfileTests.java       |   32 +-
 .../pom.xml                                   |    8 +-
 sdk/applicationinsights/tests.mgmt.yml        |   24 +-
 sdk/astro/azure-resourcemanager-astro/pom.xml |    6 +-
 .../azure-resourcemanager-attestation/pom.xml |    6 +-
 .../azure-security-attestation/CHANGELOG.md   |   11 +
 .../azure-security-attestation/pom.xml        |   14 +-
 sdk/attestation/tests.yml                     |   26 +-
 sdk/avs/azure-resourcemanager-avs/pom.xml     |    8 +-
 sdk/avs/tests.mgmt.yml                        |   24 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-azurestack/pom.xml  |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 sdk/batch/azure-resourcemanager-batch/pom.xml |    8 +-
 sdk/batch/tests.mgmt.yml                      |   24 +-
 sdk/batch/tests.yml                           |   28 +-
 .../azure-resourcemanager-billing/pom.xml     |    6 +-
 .../pom.xml                                   |    6 +-
 sdk/boms/azure-sdk-bom/CHANGELOG.md           |    6 +
 sdk/boms/azure-sdk-bom/README.md              |    2 +-
 sdk/boms/azure-sdk-bom/pom.xml                |  123 +-
 .../spring-cloud-azure-dependencies/pom.xml   |    2 +-
 .../pom.xml                                   |    8 +-
 sdk/changeanalysis/tests.mgmt.yml             |   24 +-
 .../azure-resourcemanager-chaos/CHANGELOG.md  |   17 +-
 .../azure-resourcemanager-chaos/README.md     |    4 +-
 .../azure-resourcemanager-chaos/SAMPLE.md     |  450 +-
 sdk/chaos/azure-resourcemanager-chaos/pom.xml |   36 +-
 .../resourcemanager/chaos/ChaosManager.java   |   93 +-
 .../chaos/fluent/CapabilitiesClient.java      |  103 +-
 .../chaos/fluent/CapabilityTypesClient.java   |   28 +-
 .../chaos/fluent/ChaosManagementClient.java   |   28 +-
 .../chaos/fluent/ExperimentsClient.java       |  120 +-
 .../chaos/fluent/OperationStatusesClient.java |   12 +-
 .../chaos/fluent/OperationsClient.java        |    8 +-
 .../chaos/fluent/TargetTypesClient.java       |   12 +-
 .../chaos/fluent/TargetsClient.java           |   94 +-
 .../chaos/fluent/models/CapabilityInner.java  |   24 +-
 .../fluent/models/CapabilityProperties.java   |   20 +-
 .../fluent/models/CapabilityTypeInner.java    |   44 +-
 .../models/CapabilityTypeProperties.java      |   36 +-
 .../ExperimentExecutionDetailsInner.java      |   30 +-
 .../ExperimentExecutionDetailsProperties.java |   16 +-
 .../models/ExperimentExecutionInner.java      |   24 +-
 .../models/ExperimentExecutionProperties.java |   16 +-
 .../chaos/fluent/models/ExperimentInner.java  |   41 +-
 .../fluent/models/ExperimentProperties.java   |   30 +-
 .../chaos/fluent/models/OperationInner.java   |   22 +-
 .../fluent/models/OperationStatusInner.java   |   34 +-
 .../chaos/fluent/models/TargetInner.java      |   25 +-
 .../chaos/fluent/models/TargetTypeInner.java  |   31 +-
 .../fluent/models/TargetTypeProperties.java   |   18 +-
 .../chaos/fluent/models/package-info.java     |    5 +-
 .../chaos/fluent/package-info.java            |    5 +-
 .../CapabilitiesClientImpl.java               |  893 +--
 .../implementation/CapabilitiesImpl.java      |  187 +-
 .../CapabilityTypesClientImpl.java            |  338 +-
 .../implementation/CapabilityTypesImpl.java   |   29 +-
 .../ChaosManagementClientBuilder.java         |   46 +-
 .../ChaosManagementClientImpl.java            |  133 +-
 .../ExperimentExecutionDetailsImpl.java       |    4 +-
 .../ExperimentExecutionImpl.java              |    4 +-
 .../chaos/implementation/ExperimentImpl.java  |   57 +-
 .../implementation/ExperimentsClientImpl.java | 1559 ++--
 .../chaos/implementation/ExperimentsImpl.java |  151 +-
 .../OperationStatusesClientImpl.java          |  113 +-
 .../implementation/OperationStatusesImpl.java |   13 +-
 .../implementation/OperationsClientImpl.java  |  152 +-
 .../chaos/implementation/OperationsImpl.java  |    4 +-
 .../{Utils.java => ResourceManagerUtils.java} |   25 +-
 .../implementation/TargetTypesClientImpl.java |  294 +-
 .../chaos/implementation/TargetTypesImpl.java |    9 +-
 .../implementation/TargetsClientImpl.java     |  821 +--
 .../chaos/implementation/TargetsImpl.java     |  160 +-
 .../chaos/implementation/package-info.java    |    5 +-
 .../chaos/models/ActionStatus.java            |   22 +-
 .../chaos/models/ActionType.java              |   14 +-
 .../chaos/models/BranchStatus.java            |   18 +-
 .../chaos/models/Capabilities.java            |  104 +-
 .../chaos/models/Capability.java              |   24 +-
 .../chaos/models/CapabilityListResult.java    |   14 +-
 .../chaos/models/CapabilityType.java          |   36 +-
 .../models/CapabilityTypeListResult.java      |   14 +-
 ...bilityTypePropertiesRuntimeProperties.java |   12 +-
 .../chaos/models/CapabilityTypes.java         |   28 +-
 .../chaos/models/ChaosExperimentAction.java   |   22 +-
 .../chaos/models/ChaosExperimentBranch.java   |   28 +-
 .../chaos/models/ChaosExperimentStep.java     |   28 +-
 .../chaos/models/ChaosTargetFilter.java       |   12 +-
 .../chaos/models/ChaosTargetListSelector.java |   27 +-
 .../models/ChaosTargetQuerySelector.java      |   38 +-
 .../chaos/models/ChaosTargetSelector.java     |   33 +-
 .../chaos/models/ChaosTargetSimpleFilter.java |   14 +-
 .../ChaosTargetSimpleFilterParameters.java    |   14 +-
 .../chaos/models/ContinuousAction.java        |   41 +-
 .../chaos/models/DelayAction.java             |   23 +-
 .../chaos/models/DiscreteAction.java          |   32 +-
 .../chaos/models/Experiment.java              |  141 +-
 .../chaos/models/ExperimentExecution.java     |   18 +-
 ...mentExecutionActionTargetDetailsError.java |   14 +-
 ...xecutionActionTargetDetailsProperties.java |   20 +-
 .../models/ExperimentExecutionDetails.java    |   24 +-
 ...cutionDetailsPropertiesRunInformation.java |   12 +-
 .../models/ExperimentExecutionListResult.java |   14 +-
 .../chaos/models/ExperimentListResult.java    |   14 +-
 .../chaos/models/ExperimentUpdate.java        |   43 +-
 .../chaos/models/Experiments.java             |   74 +-
 .../chaos/models/FilterType.java              |   14 +-
 .../chaos/models/KeyValuePair.java            |   28 +-
 .../chaos/models/Operation.java               |   16 +-
 .../chaos/models/OperationDisplay.java        |   18 +-
 .../chaos/models/OperationListResult.java     |   14 +-
 .../chaos/models/OperationStatus.java         |   18 +-
 .../chaos/models/OperationStatuses.java       |   12 +-
 .../chaos/models/Operations.java              |    8 +-
 .../resourcemanager/chaos/models/Origin.java  |   18 +-
 .../chaos/models/ProvisioningState.java       |   34 +-
 .../chaos/models/ResourceIdentity.java        |   43 +-
 .../chaos/models/ResourceIdentityType.java    |   26 +-
 .../chaos/models/SelectorType.java            |   18 +-
 .../chaos/models/StepStatus.java              |   18 +-
 .../resourcemanager/chaos/models/Target.java  |   18 +-
 .../chaos/models/TargetListResult.java        |   14 +-
 .../chaos/models/TargetReference.java         |   28 +-
 .../chaos/models/TargetReferenceType.java     |   14 +-
 .../chaos/models/TargetType.java              |   24 +-
 .../chaos/models/TargetTypeListResult.java    |   14 +-
 .../chaos/models/TargetTypes.java             |   12 +-
 .../resourcemanager/chaos/models/Targets.java |   94 +-
 .../chaos/models/UserAssignedIdentity.java    |   14 +-
 .../chaos/models/package-info.java            |    5 +-
 .../resourcemanager/chaos/package-info.java   |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../CapabilitiesCreateOrUpdateSamples.java    |   23 +-
 .../generated/CapabilitiesDeleteSamples.java  |   21 +-
 .../generated/CapabilitiesGetSamples.java     |   25 +-
 .../generated/CapabilitiesListSamples.java    |   21 +-
 .../generated/CapabilityTypesGetSamples.java  |   14 +-
 .../generated/CapabilityTypesListSamples.java |    9 +-
 .../generated/ExperimentsCancelSamples.java   |    9 +-
 .../ExperimentsCreateOrUpdateSamples.java     |   62 +-
 .../generated/ExperimentsDeleteSamples.java   |    9 +-
 .../ExperimentsExecutionDetailsSamples.java   |   18 +-
 .../ExperimentsGetByResourceGroupSamples.java |   14 +-
 .../ExperimentsGetExecutionSamples.java       |   18 +-
 .../ExperimentsListAllExecutionsSamples.java  |    9 +-
 ...ExperimentsListByResourceGroupSamples.java |   10 +-
 .../generated/ExperimentsListSamples.java     |   10 +-
 .../generated/ExperimentsStartSamples.java    |    9 +-
 .../generated/ExperimentsUpdateSamples.java   |   30 +-
 .../OperationStatusesGetSamples.java          |   14 +-
 .../generated/TargetTypesGetSamples.java      |    9 +-
 .../generated/TargetTypesListSamples.java     |    9 +-
 .../TargetsCreateOrUpdateSamples.java         |   35 +-
 .../chaos/generated/TargetsDeleteSamples.java |   24 +-
 .../chaos/generated/TargetsGetSamples.java    |   24 +-
 .../chaos/generated/TargetsListSamples.java   |   24 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   49 +-
 ...pabilitiesDeleteWithResponseMockTests.java |   39 +-
 .../CapabilitiesGetWithResponseMockTests.java |   49 +-
 .../generated/CapabilitiesListMockTests.java  |   47 +-
 .../chaos/generated/CapabilityInnerTests.java |    8 +-
 .../generated/CapabilityListResultTests.java  |    8 +-
 .../generated/CapabilityPropertiesTests.java  |    8 +-
 .../generated/CapabilityTypeInnerTests.java   |    8 +-
 .../CapabilityTypeListResultTests.java        |    8 +-
 ...yTypePropertiesRuntimePropertiesTests.java |    6 +-
 .../CapabilityTypePropertiesTests.java        |   13 +-
 ...pabilityTypesGetWithResponseMockTests.java |   43 +-
 .../CapabilityTypesListMockTests.java         |   39 +-
 .../generated/ChaosExperimentActionTests.java |    5 +-
 .../generated/ChaosExperimentBranchTests.java |   22 +-
 .../generated/ChaosExperimentStepTests.java   |   27 +-
 .../generated/ChaosTargetFilterTests.java     |    4 +-
 .../ChaosTargetListSelectorTests.java         |   30 +-
 .../ChaosTargetQuerySelectorTests.java        |   29 +-
 .../generated/ChaosTargetSelectorTests.java   |   15 +-
 ...haosTargetSimpleFilterParametersTests.java |   13 +-
 .../ChaosTargetSimpleFilterTests.java         |   16 +-
 .../chaos/generated/DelayActionTests.java     |   16 +-
 .../ExperimentExecutionDetailsInnerTests.java |    8 +-
 ...nDetailsPropertiesRunInformationTests.java |   12 +-
 ...rimentExecutionDetailsPropertiesTests.java |    8 +-
 .../ExperimentExecutionInnerTests.java        |    8 +-
 .../ExperimentExecutionListResultTests.java   |    8 +-
 .../ExperimentExecutionPropertiesTests.java   |    8 +-
 .../chaos/generated/ExperimentInnerTests.java |  124 +-
 .../generated/ExperimentListResultTests.java  |    8 +-
 .../generated/ExperimentPropertiesTests.java  |  128 +-
 .../generated/ExperimentUpdateTests.java      |   31 +-
 .../ExperimentsCreateOrUpdateMockTests.java   |  123 +-
 ...ExecutionDetailsWithResponseMockTests.java |   43 +-
 ...tByResourceGroupWithResponseMockTests.java |   58 +-
 ...entsGetExecutionWithResponseMockTests.java |   42 +-
 ...ExperimentsListAllExecutionsMockTests.java |   38 +-
 ...perimentsListByResourceGroupMockTests.java |   57 +-
 .../generated/ExperimentsListMockTests.java   |   47 +-
 .../generated/OperationDisplayTests.java      |    9 +-
 .../chaos/generated/OperationInnerTests.java  |    8 +-
 .../generated/OperationListResultTests.java   |    8 +-
 .../generated/OperationStatusInnerTests.java  |   37 +-
 ...ationStatusesGetWithResponseMockTests.java |   56 +-
 .../generated/ResourceIdentityTests.java      |   22 +-
 .../chaos/generated/StepStatusTests.java      |    8 +-
 .../chaos/generated/TargetInnerTests.java     |   19 +-
 .../generated/TargetListResultTests.java      |    8 +-
 .../chaos/generated/TargetReferenceTests.java |   10 +-
 .../chaos/generated/TargetTypeInnerTests.java |   14 +-
 .../generated/TargetTypeListResultTests.java  |    8 +-
 .../generated/TargetTypePropertiesTests.java  |    8 +-
 .../TargetTypesGetWithResponseMockTests.java  |   42 +-
 .../generated/TargetTypesListMockTests.java   |   39 +-
 ...tsCreateOrUpdateWithResponseMockTests.java |   64 +-
 .../TargetsDeleteWithResponseMockTests.java   |   36 +-
 .../TargetsGetWithResponseMockTests.java      |   42 +-
 .../chaos/generated/TargetsListMockTests.java |   41 +-
 .../generated/UserAssignedIdentityTests.java  |    8 +-
 sdk/chaos/tests.mgmt.yml                      |   24 +-
 .../pom.xml                                   |    8 +-
 sdk/cognitiveservices/tests.mgmt.yml          |   24 +-
 .../azure-resourcemanager-commerce/pom.xml    |    6 +-
 .../CHANGELOG.md                              |   12 +
 .../pom.xml                                   |   12 +-
 .../tests.yml                                 |    4 +-
 .../azure-communication-callingserver/pom.xml |   12 +-
 .../tests.yml                                 |    6 +-
 .../azure-communication-chat/CHANGELOG.md     |   12 +
 .../azure-communication-chat/pom.xml          |    4 +-
 .../azure-communication-chat/tests.yml        |   10 +-
 .../azure-communication-common/CHANGELOG.md   |   11 +
 .../azure-communication-common/pom.xml        |    2 +-
 .../azure-communication-email/CHANGELOG.md    |   12 +
 .../azure-communication-email/pom.xml         |    4 +-
 .../azure-communication-email/tests.yml       |   50 +-
 .../azure-communication-identity/CHANGELOG.md |   12 +
 .../azure-communication-identity/pom.xml      |    6 +-
 .../azure-communication-identity/tests.yml    |   40 +-
 .../CHANGELOG.md                              |   12 +
 .../azure-communication-jobrouter/pom.xml     |    4 +-
 .../azure-communication-jobrouter/tests.yml   |   10 +-
 .../azure-communication-messages/CHANGELOG.md |   12 +
 .../azure-communication-messages/pom.xml      |    6 +-
 .../azure-communication-messages/tests.yml    |   42 +-
 .../pom.xml                                   |    8 +-
 .../tests.yml                                 |   12 +-
 .../CHANGELOG.md                              |   48 +-
 .../assets.json                               |    2 +-
 .../phone-numbers-livetest-matrix.json        |   16 +-
 .../azure-communication-phonenumbers/pom.xml  |   12 +-
 .../phonenumbers/PhoneNumbersAsyncClient.java |   74 +-
 .../phonenumbers/PhoneNumbersClient.java      |   72 +-
 .../PhoneNumbersServiceVersion.java           |    9 +-
 .../PhoneNumberAdminClientImpl.java           |   25 +-
 .../PhoneNumberAdminClientImplBuilder.java    |  235 +-
 .../implementation/PhoneNumbersImpl.java      |  752 +-
 .../models/CommunicationError.java            |    4 +-
 .../models/CommunicationErrorResponse.java    |    4 +-
 .../CommunicationErrorResponseException.java  |    2 -
 .../implementation/models/Error.java          |   83 +
 .../models/OfferingsResponse.java             |    4 +-
 .../models/OperatorInformationOptions.java    |   42 +
 .../models/OperatorInformationRequest.java    |   66 +
 .../OperatorInformationRequestOptions.java    |   43 +
 .../models/OperatorInformationResult.java     |   34 +
 .../models/PhoneNumberAreaCodes.java          |    4 +-
 .../PhoneNumberCapabilitiesRequest.java       |    4 +-
 .../models/PhoneNumberCountries.java          |    4 +-
 .../models/PhoneNumberLocalities.java         |    4 +-
 .../models/PhoneNumberPurchaseRequest.java    |    4 +-
 .../models/PhoneNumberRawOperation.java       |    2 -
 .../models/PhoneNumberSearchRequest.java      |    7 +-
 .../models/PhoneNumberSearchResultError.java  |   84 +
 .../PhoneNumbersGetOperationHeaders.java      |    2 -
 .../PhoneNumbersGetOperationResponse.java     |    6 +-
 ...oneNumbersPurchasePhoneNumbersHeaders.java |    2 -
 ...PhoneNumbersReleasePhoneNumberHeaders.java |    2 -
 ...ersSearchAvailablePhoneNumbersHeaders.java |    2 -
 ...rsSearchAvailablePhoneNumbersResponse.java |    6 +-
 ...PhoneNumbersUpdateCapabilitiesHeaders.java |    2 -
 ...honeNumbersUpdateCapabilitiesResponse.java |    6 +-
 .../models/PurchasedPhoneNumbers.java         |    4 +-
 .../implementation/package-info.java          |    4 +-
 .../phonenumbers/models/BillingFrequency.java |    9 +-
 .../phonenumbers/models/OperatorDetails.java  |   57 +
 .../models/OperatorInformation.java           |  103 +
 .../models/OperatorInformationOptions.java    |   42 +
 .../models/OperatorInformationResult.java     |   33 +
 .../models/OperatorNumberType.java            |   44 +
 .../PhoneNumberAdministrativeDivision.java    |    2 +-
 .../models/PhoneNumberAreaCode.java           |    4 +-
 .../models/PhoneNumberAssignmentType.java     |    6 +-
 .../models/PhoneNumberCapabilities.java       |    2 +-
 .../models/PhoneNumberCapabilityType.java     |    6 +-
 .../phonenumbers/models/PhoneNumberCost.java  |    2 +-
 .../models/PhoneNumberCountry.java            |    2 +-
 .../models/PhoneNumberLocality.java           |    2 +-
 .../models/PhoneNumberOffering.java           |    2 +-
 .../models/PhoneNumberOperationStatus.java    |    6 +-
 .../models/PhoneNumberOperationType.java      |    8 +-
 .../models/PhoneNumberSearchResult.java       |   32 +-
 .../models/PhoneNumberSearchResultError.java  |   84 +
 .../phonenumbers/models/PhoneNumberType.java  |    6 +-
 .../models/PurchasedPhoneNumber.java          |    2 +-
 ...honeNumbersAsyncClientIntegrationTest.java |   63 +
 .../PhoneNumbersClientBuilderTest.java        |    4 +-
 .../PhoneNumbersClientIntegrationTest.java    |   11 +
 .../PhoneNumbersIntegrationTestBase.java      |   10 +-
 .../swagger/README.md                         |   61 +-
 .../tests.yml                                 |   64 +-
 .../azure-communication-rooms/CHANGELOG.md    |   12 +
 .../azure-communication-rooms/pom.xml         |    8 +-
 .../azure-communication-rooms/tests.yml       |   36 +-
 .../azure-communication-sms/CHANGELOG.md      |   12 +
 .../azure-communication-sms/pom.xml           |    6 +-
 .../azure-communication-sms/tests.yml         |   36 +-
 .../CHANGELOG.md                              |   31 +-
 .../README.md                                 |    4 +-
 .../SAMPLE.md                                 |  504 +-
 .../pom.xml                                   |   13 +-
 .../communication/CommunicationManager.java   |   35 +-
 .../CommunicationServiceManagementClient.java |   14 -
 .../SuppressionListAddressesClient.java       |  174 -
 .../fluent/SuppressionListsClient.java        |  167 -
 .../SuppressionListAddressProperties.java     |  172 -
 .../SuppressionListAddressResourceInner.java  |  175 -
 .../models/SuppressionListProperties.java     |  104 -
 .../models/SuppressionListResourceInner.java  |  116 -
 ...municationServiceManagementClientImpl.java |   34 +-
 .../CommunicationServiceResourceImpl.java     |    5 +-
 .../CommunicationServicesImpl.java            |   28 +-
 .../implementation/DomainResourceImpl.java    |    6 +-
 .../implementation/DomainsImpl.java           |   28 +-
 .../EmailServiceResourceImpl.java             |    4 +-
 .../implementation/EmailServicesImpl.java     |   24 +-
 .../implementation/OperationsImpl.java        |    4 +-
 .../{Utils.java => ResourceManagerUtils.java} |    4 +-
 .../SenderUsernameResourceImpl.java           |    8 +-
 .../implementation/SenderUsernamesImpl.java   |   36 +-
 .../SuppressionListAddressResourceImpl.java   |  179 -
 .../SuppressionListAddressesClientImpl.java   |  900 ---
 .../SuppressionListAddressesImpl.java         |  209 -
 .../SuppressionListResourceImpl.java          |  144 -
 .../SuppressionListsClientImpl.java           |  856 ---
 .../implementation/SuppressionListsImpl.java  |  186 -
 .../SuppressionListAddressResource.java       |  311 -
 ...pressionListAddressResourceCollection.java |   85 -
 .../models/SuppressionListResource.java       |  217 -
 .../SuppressionListResourceCollection.java    |   85 -
 .../src/main/java/module-info.java            |    2 -
 .../proxy-config.json                         |    2 +-
 .../reflect-config.json                       |   30 -
 ...nServicesCheckNameAvailabilitySamples.java |    4 +-
 ...nicationServicesCreateOrUpdateSamples.java |    4 +-
 .../CommunicationServicesDeleteSamples.java   |    2 +-
 ...tionServicesGetByResourceGroupSamples.java |    2 +-
 ...ionServicesLinkNotificationHubSamples.java |    2 +-
 ...ionServicesListByResourceGroupSamples.java |    2 +-
 .../CommunicationServicesListKeysSamples.java |    2 +-
 .../CommunicationServicesListSamples.java     |    2 +-
 ...unicationServicesRegenerateKeySamples.java |    2 +-
 .../CommunicationServicesUpdateSamples.java   |   16 +-
 .../DomainsCancelVerificationSamples.java     |    2 +-
 .../DomainsCreateOrUpdateSamples.java         |    2 +-
 .../generated/DomainsDeleteSamples.java       |    4 +-
 .../generated/DomainsGetSamples.java          |    3 +-
 .../DomainsInitiateVerificationSamples.java   |    2 +-
 ...ainsListByEmailServiceResourceSamples.java |    2 +-
 .../generated/DomainsUpdateSamples.java       |    4 +-
 .../EmailServicesCreateOrUpdateSamples.java   |    4 +-
 .../generated/EmailServicesDeleteSamples.java |    4 +-
 ...mailServicesGetByResourceGroupSamples.java |    4 +-
 ...ailServicesListByResourceGroupSamples.java |    4 +-
 .../generated/EmailServicesListSamples.java   |    4 +-
 ...tVerifiedExchangeOnlineDomainsSamples.java |    4 +-
 .../generated/EmailServicesUpdateSamples.java |    4 +-
 .../generated/OperationsListSamples.java      |    2 +-
 .../SenderUsernamesCreateOrUpdateSamples.java |    4 +-
 .../SenderUsernamesDeleteSamples.java         |    4 +-
 .../generated/SenderUsernamesGetSamples.java  |    4 +-
 .../SenderUsernamesListByDomainsSamples.java  |    8 +-
 ...ionListAddressesCreateOrUpdateSamples.java |   28 -
 ...SuppressionListAddressesDeleteSamples.java |   27 -
 .../SuppressionListAddressesGetSamples.java   |   27 -
 .../SuppressionListAddressesListSamples.java  |   26 -
 ...SuppressionListsCreateOrUpdateSamples.java |   27 -
 .../SuppressionListsDeleteSamples.java        |   26 -
 .../generated/SuppressionListsGetSamples.java |   26 -
 .../SuppressionListsListByDomainSamples.java  |   26 -
 ...ommunicationServiceResourceInnerTests.java |    2 +-
 ...CommunicationServiceResourceListTests.java |    2 +-
 ...mmunicationServiceResourceUpdateTests.java |    2 +-
 ...NameAvailabilityWithResponseMockTests.java |   10 +-
 ...cationServicesCreateOrUpdateMockTests.java |   27 +-
 .../CommunicationServicesDeleteMockTests.java |    2 +-
 ...tByResourceGroupWithResponseMockTests.java |   14 +-
 ...kNotificationHubWithResponseMockTests.java |    8 +-
 ...nServicesListByResourceGroupMockTests.java |   17 +-
 .../CommunicationServicesListMockTests.java   |   13 +-
 .../generated/DomainsDeleteMockTests.java     |    2 +-
 .../EmailServicesCreateOrUpdateMockTests.java |   16 +-
 .../EmailServicesDeleteMockTests.java         |    2 +-
 ...tByResourceGroupWithResponseMockTests.java |   10 +-
 ...lServicesListByResourceGroupMockTests.java |   10 +-
 .../generated/EmailServicesListMockTests.java |    8 +-
 ...ngeOnlineDomainsWithResponseMockTests.java |    4 +-
 .../ManagedServiceIdentityTests.java          |    2 +-
 .../generated/OperationsListMockTests.java    |    2 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   10 +-
 ...rUsernamesDeleteWithResponseMockTests.java |    2 +-
 ...nderUsernamesGetWithResponseMockTests.java |    9 +-
 ...SenderUsernamesListByDomainsMockTests.java |   10 +-
 ...SuppressionListAddressPropertiesTests.java |   33 -
 ...ionListAddressResourceCollectionTests.java |   42 -
 ...pressionListAddressResourceInnerTests.java |   33 -
 .../SuppressionListPropertiesTests.java       |   26 -
 ...uppressionListResourceCollectionTests.java |   33 -
 .../SuppressionListResourceInnerTests.java    |   26 -
 .../generated/UserAssignedIdentityTests.java  |    2 +-
 .../communication-tests-template.yml          |   41 +-
 .../pom.xml                                   |    8 +-
 .../CHANGELOG.md                              |   11 +
 .../azure-security-confidentialledger/pom.xml |    2 +-
 .../tests.yml                                 |   22 +-
 .../CHANGELOG.md                              |   91 +-
 .../azure-resourcemanager-confluent/README.md |    4 +-
 .../azure-resourcemanager-confluent/SAMPLE.md |  496 +-
 .../azure-resourcemanager-confluent/pom.xml   |   14 +-
 .../confluent/ConfluentManager.java           |    2 +-
 .../confluent/fluent/AccessClient.java        |  122 +-
 .../fluent/ConfluentManagementClient.java     |    2 +-
 .../confluent/fluent/OrganizationsClient.java |  357 +-
 .../confluent/fluent/ValidationsClient.java   |    8 +-
 ...AccessListClusterSuccessResponseInner.java |    8 +-
 ...sListEnvironmentsSuccessResponseInner.java |   14 +-
 ...sListRoleBindingsSuccessResponseInner.java |    8 +-
 ...leBindingNameListSuccessResponseInner.java |  111 +
 .../fluent/models/ApiKeyProperties.java       |   88 +
 .../fluent/models/ApiKeyRecordInner.java      |  146 +
 .../fluent/models/ClusterProperties.java      |  118 +
 .../fluent/models/EnvironmentProperties.java  |   58 +
 .../ListRegionsSuccessResponseInner.java      |   59 +
 .../fluent/models/RegionProperties.java       |   88 +
 .../fluent/models/SCClusterRecordInner.java   |  196 +
 .../models/SCEnvironmentRecordInner.java      |  148 +
 .../SchemaRegistryClusterProperties.java      |  118 +
 .../SchemaRegistryClusterRecordInner.java     |  170 +
 .../implementation/AccessClientImpl.java      |  524 +-
 .../confluent/implementation/AccessImpl.java  |   47 +
 ...oleBindingNameListSuccessResponseImpl.java |   48 +
 .../implementation/ApiKeyRecordImpl.java      |   46 +
 .../ConfluentManagementClientBuilder.java     |    4 +-
 .../ConfluentManagementClientImpl.java        |    8 +-
 .../ListRegionsSuccessResponseImpl.java       |   40 +
 .../MarketplaceAgreementsImpl.java            |    6 +-
 .../OrganizationOperationsImpl.java           |    4 +-
 .../OrganizationResourceImpl.java             |   16 +-
 .../OrganizationsClientImpl.java              | 2086 +++++-
 .../implementation/OrganizationsImpl.java     |  226 +-
 .../{Utils.java => ResourceManagerUtils.java} |    4 +-
 .../implementation/SCClusterRecordImpl.java   |   55 +
 .../SCEnvironmentRecordImpl.java              |   45 +
 .../SchemaRegistryClusterRecordImpl.java      |   51 +
 .../implementation/ValidationsClientImpl.java |   20 +-
 .../confluent/models/Access.java              |  113 +-
 .../AccessCreateRoleBindingRequestModel.java  |  108 +
 .../AccessListClusterSuccessResponse.java     |    2 +-
 ...AccessListEnvironmentsSuccessResponse.java |    4 +-
 ...AccessListRoleBindingsSuccessResponse.java |    2 +-
 ...essRoleBindingNameListSuccessResponse.java |   42 +
 .../confluent/models/ApiKeyOwnerEntity.java   |  132 +
 .../confluent/models/ApiKeyRecord.java        |   47 +
 .../models/ApiKeyResourceEntity.java          |  158 +
 .../confluent/models/ApiKeySpecEntity.java    |  164 +
 .../confluent/models/ClusterRecord.java       |   20 +-
 .../confluent/models/CreateApiKeyModel.java   |   80 +
 .../confluent/models/EnvironmentRecord.java   |    2 +-
 .../models/GetEnvironmentsResponse.java       |   85 +
 .../models/ListClustersSuccessResponse.java   |   85 +
 .../models/ListRegionsSuccessResponse.java    |   27 +
 .../ListSchemaRegistryClustersResponse.java   |   85 +
 .../models/OrganizationResource.java          |   26 +-
 .../confluent/models/Organizations.java       |  313 +-
 .../confluent/models/RegionRecord.java        |  145 +
 .../confluent/models/RegionSpecEntity.java    |  133 +
 .../confluent/models/RoleBindingRecord.java   |    8 +-
 .../confluent/models/SCClusterByokEntity.java |  106 +
 .../SCClusterNetworkEnvironmentEntity.java    |  132 +
 .../confluent/models/SCClusterRecord.java     |   61 +
 .../confluent/models/SCClusterSpecEntity.java |  352 +
 .../confluent/models/SCEnvironmentRecord.java |   47 +
 .../confluent/models/SCMetadataEntity.java    |  158 +
 ...egistryClusterEnvironmentRegionEntity.java |  106 +
 .../models/SchemaRegistryClusterRecord.java   |   54 +
 .../SchemaRegistryClusterSpecEntity.java      |  190 +
 .../SchemaRegistryClusterStatusEntity.java    |   54 +
 .../confluent/models/Validations.java         |    8 +-
 .../src/main/java/module-info.java            |    2 -
 .../reflect-config.json                       |  140 +
 .../AccessCreateRoleBindingSamples.java       |   30 +
 .../AccessDeleteRoleBindingSamples.java       |   25 +
 .../generated/AccessInviteUserSamples.java    |    2 +-
 .../generated/AccessListClustersSamples.java  |    2 +-
 .../AccessListEnvironmentsSamples.java        |    2 +-
 .../AccessListInvitationsSamples.java         |    2 +-
 .../AccessListRoleBindingNameListSamples.java |   43 +
 .../AccessListRoleBindingsSamples.java        |    2 +-
 .../AccessListServiceAccountsSamples.java     |    2 +-
 .../generated/AccessListUsersSamples.java     |    2 +-
 .../MarketplaceAgreementsCreateSamples.java   |    2 +-
 .../MarketplaceAgreementsListSamples.java     |    2 +-
 .../OrganizationCreateApiKeySamples.java      |   28 +
 .../generated/OrganizationCreateSamples.java  |    2 +-
 ...rganizationDeleteClusterApiKeySamples.java |   24 +
 .../generated/OrganizationDeleteSamples.java  |    2 +-
 ...OrganizationGetByResourceGroupSamples.java |    2 +-
 .../OrganizationGetClusterApiKeySamples.java  |   24 +
 .../OrganizationGetClusterByIdSamples.java    |   24 +
 ...OrganizationGetEnvironmentByIdSamples.java |   24 +
 ...onGetSchemaRegistryClusterByIdSamples.java |   25 +
 ...rganizationListByResourceGroupSamples.java |    2 +-
 .../OrganizationListClustersSamples.java      |   25 +
 .../OrganizationListEnvironmentsSamples.java  |   24 +
 .../OrganizationListRegionsSamples.java       |   43 +
 .../generated/OrganizationListSamples.java    |    2 +-
 ...tionListSchemaRegistryClustersSamples.java |   25 +
 .../OrganizationOperationsListSamples.java    |    2 +-
 .../generated/OrganizationUpdateSamples.java  |    2 +-
 ...alidationsValidateOrganizationSamples.java |    2 +-
 ...idationsValidateOrganizationV2Samples.java |    2 +-
 ...essCreateRoleBindingRequestModelTests.java |   31 +
 ...reateRoleBindingWithResponseMockTests.java |   69 +
 ...eleteRoleBindingWithResponseMockTests.java |   52 +
 ...AccessInviteUserWithResponseMockTests.java |   41 +-
 ...cessListClustersWithResponseMockTests.java |   80 +-
 ...ListEnvironmentsWithResponseMockTests.java |   35 +-
 ...sListInvitationsWithResponseMockTests.java |   44 +-
 ...BindingNameListWithResponseMockTests.java} |   34 +-
 ...ListRoleBindingsWithResponseMockTests.java |   47 +-
 ...tServiceAccountsWithResponseMockTests.java |   37 +-
 .../AccessListUsersWithResponseMockTests.java |   41 +-
 ...dingNameListSuccessResponseInnerTests.java |   43 +
 .../generated/ApiKeyOwnerEntityTests.java     |   33 +
 .../generated/ApiKeyResourceEntityTests.java  |   35 +
 .../generated/ClusterPropertiesTests.java     |   97 +
 .../generated/CreateApiKeyModelTests.java     |   27 +
 .../generated/EnvironmentPropertiesTests.java |   37 +
 .../GetEnvironmentsResponseTests.java         |   51 +
 .../ListClustersSuccessResponseTests.java     |  140 +
 .../ListRegionsSuccessResponseInnerTests.java |   56 +
 ...stSchemaRegistryClustersResponseTests.java |   80 +
 ...AgreementsCreateWithResponseMockTests.java |   22 +-
 .../MarketplaceAgreementsListMockTests.java   |   16 +-
 .../OrganizationOperationsListMockTests.java  |   12 +-
 ...eteClusterApiKeyWithResponseMockTests.java |   52 +
 .../OrganizationsDeleteMockTests.java         |    2 +-
 ...nsGetClusterByIdWithResponseMockTests.java |   85 +
 ...EnvironmentByIdWithResponseMockTests.java} |   32 +-
 ...istryClusterByIdWithResponseMockTests.java |   74 +
 .../OrganizationsListClustersMockTests.java   |   86 +
 ...rganizationsListEnvironmentsMockTests.java |   64 +
 ...tionsListRegionsWithResponseMockTests.java |   84 +
 ...nsListSchemaRegistryClustersMockTests.java |   74 +
 .../generated/RegionPropertiesTests.java      |   49 +
 .../generated/RegionRecordTests.java          |   53 +
 .../generated/RegionSpecEntityTests.java      |   34 +
 .../generated/SCClusterByokEntityTests.java   |   31 +
 ...CClusterNetworkEnvironmentEntityTests.java |   33 +
 .../generated/SCClusterRecordInnerTests.java  |  105 +
 .../generated/SCClusterSpecEntityTests.java   |   76 +
 .../SCEnvironmentRecordInnerTests.java        |   43 +
 .../generated/SCMetadataEntityTests.java      |   36 +
 ...ryClusterEnvironmentRegionEntityTests.java |   31 +
 .../SchemaRegistryClusterPropertiesTests.java |   70 +
 ...SchemaRegistryClusterRecordInnerTests.java |   76 +
 .../SchemaRegistryClusterSpecEntityTests.java |   51 +
 ...chemaRegistryClusterStatusEntityTests.java |   25 +
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-consumption/pom.xml |    2 +-
 .../pom.xml                                   |    4 +-
 .../CHANGELOG.md                              |   21 +
 .../pom.xml                                   |   10 +-
 sdk/containerregistry/tests.yml               |   32 +-
 .../pom.xml                                   |    8 +-
 sdk/containerservicefleet/tests.mgmt.yml      |   24 +-
 .../azure-ai-contentsafety/CHANGELOG.md       |   11 +
 .../azure-ai-contentsafety/pom.xml            |    2 +-
 sdk/core/azure-core-amqp-experimental/pom.xml |    2 +-
 sdk/core/azure-core-amqp/pom.xml              |    6 +-
 sdk/core/azure-core-experimental/pom.xml      |    4 +-
 .../azure-core-http-jdk-httpclient/pom.xml    |    2 +-
 sdk/core/azure-core-http-netty/pom.xml        |   50 +-
 .../implementation/HttpProxyHandler.java      |    1 +
 sdk/core/azure-core-http-okhttp/pom.xml       |    6 +-
 sdk/core/azure-core-http-vertx/pom.xml        |   18 +-
 sdk/core/azure-core-management/pom.xml        |    2 +-
 .../azure-core-metrics-opentelemetry/pom.xml  |   18 +-
 sdk/core/azure-core-perf/pom.xml              |    2 +-
 .../azure-core-serializer-avro-apache/pom.xml |    2 +-
 .../pom.xml                                   |    2 +-
 .../azure-core-serializer-json-gson/pom.xml   |    6 +-
 .../pom.xml                                   |    2 +-
 sdk/core/azure-core-test/assets.json          |    2 +-
 sdk/core/azure-core-test/pom.xml              |    8 +-
 .../core/test/http/TestProxyTestServer.java   |    2 +-
 .../azure/core/test/utils/TestProxyUtils.java |    2 +-
 .../com/azure/core/test/TestProxyTests.java   |    1 +
 .../pom.xml                                   |   10 +-
 .../azure-core-tracing-opentelemetry/pom.xml  |   12 +-
 sdk/core/azure-core-version-tests/pom.xml     |    6 +-
 sdk/core/azure-core/CHANGELOG.md              |    3 +
 sdk/core/azure-core/pom.xml                   |    6 +-
 .../ReflectionSerializable.java               |   24 +-
 .../http/rest/AsyncRestProxy.java             |    2 +-
 .../http/rest/AsyncRestProxyTests.java        |    1 +
 sdk/core/ci.yml                               |   63 +-
 .../ctl-image-build.yml                       |  152 +-
 sdk/cosmos/azure-cosmos-benchmark/pom.xml     |    8 +-
 .../data/InvitationsEntityConfiguration.java  |    2 +-
 .../data/entity/InvitationDataGenerator.java  |    2 +-
 .../azure-cosmos-dotnet-benchmark/pom.xml     |    2 +-
 .../dotnet/benchmark/OperationResult.java     |    2 +-
 .../azure-cosmos-encryption/CHANGELOG.md      |   10 +-
 sdk/cosmos/azure-cosmos-encryption/README.md  |    2 +-
 sdk/cosmos/azure-cosmos-encryption/pom.xml    |   14 +-
 .../CosmosEncryptionAsyncContainer.java       |    4 +-
 .../encryption/CosmosEncryptionContainer.java |    2 +-
 .../com/azure/cosmos/encryption/Program.java  |    2 +-
 .../encryption/EncryptionSyncApiCrudTest.java |    2 +-
 .../cosmos/encryption/TestSuiteBase.java      |    4 +-
 .../azure-cosmos-kafka-connect/CHANGELOG.md   |    3 +-
 sdk/cosmos/azure-cosmos-kafka-connect/pom.xml |   12 +-
 .../connect/CosmosDBSourceConnector.java      |  193 +-
 .../implementation/CosmosAccountConfig.java   |    4 +-
 .../implementation/CosmosClientStore.java     |    2 +-
 .../implementation/KafkaCosmosConfig.java     |    3 +-
 .../implementation/KafkaCosmosConstants.java  |   16 +
 .../KafkaCosmosExceptionsHelper.java          |   25 +-
 .../implementation/KafkaCosmosUtils.java      |   66 +
 .../kafka/connect/implementation/Strings.java |  133 +
 .../apachecommons/lang/ArrayUtils.java        |  185 +
 .../apachecommons/lang/CharSequenceUtils.java |   82 +
 .../apachecommons/lang/StringUtils.java       | 1428 ++++
 .../apachecommons/lang/Validate.java          |   60 +
 .../apachecommons/lang/builder/Builder.java   |   29 +
 .../lang/builder/CompareToBuilder.java        |  715 ++
 .../lang/tuple/ImmutablePair.java             |  108 +
 .../apachecommons/lang/tuple/Pair.java        |  163 +
 .../implementation/guava25/base/Absent.java   |  102 +
 .../guava25/base/AbstractIterator.java        |   91 +
 .../implementation/guava25/base/Ascii.java    |  636 ++
 .../guava25/base/CaseFormat.java              |  211 +
 .../guava25/base/CharMatcher.java             | 1981 +++++
 .../implementation/guava25/base/Charsets.java |   99 +
 .../guava25/base/CommonMatcher.java           |   38 +
 .../guava25/base/CommonPattern.java           |   42 +
 .../guava25/base/Converter.java               |  518 ++
 .../implementation/guava25/base/Defaults.java |   65 +
 .../implementation/guava25/base/Enums.java    |  148 +
 .../guava25/base/Equivalence.java             |  377 +
 .../base/ExtraObjectsMethodsForWeb.java       |   26 +
 .../base/FinalizablePhantomReference.java     |   46 +
 .../guava25/base/FinalizableReference.java    |   35 +
 .../base/FinalizableReferenceQueue.java       |  356 +
 .../base/FinalizableSoftReference.java        |   44 +
 .../base/FinalizableWeakReference.java        |   44 +
 .../implementation/guava25/base/Function.java |   61 +
 .../guava25/base/FunctionalEquivalence.java   |   77 +
 .../guava25/base/Functions.java               |  405 ++
 .../guava25/base/JdkPattern.java              |  105 +
 .../implementation/guava25/base/Joiner.java   |  468 ++
 .../guava25/base/MoreObjects.java             |  373 +
 .../implementation/guava25/base/Objects.java  |   81 +
 .../implementation/guava25/base/Optional.java |  355 +
 .../guava25/base/PairwiseEquivalence.java     |   77 +
 .../guava25/base/PatternCompiler.java         |   33 +
 .../implementation/guava25/base/Platform.java |  110 +
 .../guava25/base/Preconditions.java           | 1454 ++++
 .../guava25/base/Predicate.java               |   80 +
 .../guava25/base/Predicates.java              |  695 ++
 .../implementation/guava25/base/Present.java  |  101 +
 .../guava25/base/SmallCharMatcher.java        |  146 +
 .../implementation/guava25/base/Splitter.java |  598 ++
 .../guava25/base/StandardSystemProperty.java  |  138 +
 .../guava25/base/Stopwatch.java               |  264 +
 .../implementation/guava25/base/Strings.java  |  223 +
 .../implementation/guava25/base/Supplier.java |   50 +
 .../guava25/base/Suppliers.java               |  364 +
 .../guava25/base/Throwables.java              |  523 ++
 .../implementation/guava25/base/Ticker.java   |   57 +
 .../implementation/guava25/base/Utf8.java     |  201 +
 .../implementation/guava25/base/Verify.java   |  524 ++
 .../guava25/base/VerifyException.java         |   56 +
 .../guava25/base/internal/Finalizer.java      |  239 +
 .../guava25/base/package-info.java            |   22 +
 .../guava25/collect/AbstractBiMap.java        |  470 ++
 .../collect/AbstractIndexedListIterator.java  |  104 +
 .../guava25/collect/AbstractIterator.java     |  172 +
 .../guava25/collect/AbstractListMultimap.java |  139 +
 .../collect/AbstractMapBasedMultimap.java     | 1630 +++++
 .../collect/AbstractMapBasedMultiset.java     |  333 +
 .../guava25/collect/AbstractMapEntry.java     |   68 +
 .../guava25/collect/AbstractMultimap.java     |  258 +
 .../guava25/collect/AbstractMultiset.java     |  220 +
 .../guava25/collect/AbstractNavigableMap.java |  180 +
 .../guava25/collect/AbstractRangeSet.java     |  102 +
 .../collect/AbstractSequentialIterator.java   |   79 +
 .../guava25/collect/AbstractSetMultimap.java  |  150 +
 .../AbstractSortedKeySortedSetMultimap.java   |   61 +
 .../collect/AbstractSortedMultiset.java       |  148 +
 .../collect/AbstractSortedSetMultimap.java    |  147 +
 .../guava25/collect/AllEqualOrdering.java     |   66 +
 .../guava25/collect/ArrayListMultimap.java    |  169 +
 ...tMultimapGwtSerializationDependencies.java |   41 +
 .../implementation/guava25/collect/BiMap.java |  107 +
 .../guava25/collect/BoundType.java            |   47 +
 .../guava25/collect/ByFunctionOrdering.java   |   72 +
 .../guava25/collect/CartesianList.java        |  118 +
 .../guava25/collect/ClassToInstanceMap.java   |   61 +
 .../guava25/collect/CollectCollectors.java    |  136 +
 .../guava25/collect/CollectPreconditions.java |   64 +
 .../guava25/collect/CollectSpliterators.java  |  293 +
 .../guava25/collect/Collections2.java         |  697 ++
 .../guava25/collect/CompactHashMap.java       |  849 +++
 .../guava25/collect/CompactHashSet.java       |  567 ++
 .../guava25/collect/CompactLinkedHashMap.java |  288 +
 .../guava25/collect/CompactLinkedHashSet.java |  239 +
 .../guava25/collect/ComparatorOrdering.java   |   65 +
 .../guava25/collect/Comparators.java          |  193 +
 .../guava25/collect/ComparisonChain.java      |  253 +
 .../guava25/collect/CompoundOrdering.java     |   75 +
 .../guava25/collect/ComputationException.java |   38 +
 .../collect/ConcurrentHashMultiset.java       |  596 ++
 .../collect/ConsumingQueueIterator.java       |   49 +
 .../guava25/collect/ContiguousSet.java        |  255 +
 .../implementation/guava25/collect/Count.java |   72 +
 .../implementation/guava25/collect/Cut.java   |  477 ++
 .../DescendingImmutableSortedMultiset.java    |   84 +
 .../collect/DescendingImmutableSortedSet.java |  117 +
 .../guava25/collect/DescendingMultiset.java   |  157 +
 .../guava25/collect/DiscreteDomain.java       |  322 +
 .../guava25/collect/EmptyContiguousSet.java   |  163 +
 .../collect/EmptyImmutableListMultimap.java   |   41 +
 .../collect/EmptyImmutableSetMultimap.java    |   41 +
 .../guava25/collect/EnumBiMap.java            |  144 +
 .../guava25/collect/EnumHashBiMap.java        |  122 +
 .../guava25/collect/EnumMultiset.java         |  312 +
 .../guava25/collect/EvictingQueue.java        |  139 +
 .../guava25/collect/ExplicitOrdering.java     |   72 +
 .../collect/FilteredEntryMultimap.java        |  413 ++
 .../collect/FilteredEntrySetMultimap.java     |   69 +
 .../collect/FilteredKeyListMultimap.java      |   58 +
 .../guava25/collect/FilteredKeyMultimap.java  |  219 +
 .../collect/FilteredKeySetMultimap.java       |   82 +
 .../guava25/collect/FilteredMultimap.java     |   36 +
 .../collect/FilteredMultimapValues.java       |   97 +
 .../guava25/collect/FilteredSetMultimap.java  |   32 +
 .../guava25/collect/FluentIterable.java       |  860 +++
 .../guava25/collect/ForwardingCollection.java |  252 +
 .../collect/ForwardingConcurrentMap.java      |   68 +
 .../ForwardingImmutableCollection.java        |   31 +
 .../collect/ForwardingImmutableList.java      |   31 +
 .../collect/ForwardingImmutableMap.java       |   31 +
 .../collect/ForwardingImmutableSet.java       |   30 +
 .../guava25/collect/ForwardingIterator.java   |   62 +
 .../guava25/collect/ForwardingList.java       |  238 +
 .../collect/ForwardingListIterator.java       |   78 +
 .../collect/ForwardingListMultimap.java       |   61 +
 .../guava25/collect/ForwardingMap.java        |  305 +
 .../guava25/collect/ForwardingMapEntry.java   |  126 +
 .../guava25/collect/ForwardingMultimap.java   |  148 +
 .../guava25/collect/ForwardingMultiset.java   |  312 +
 .../collect/ForwardingNavigableMap.java       |  409 ++
 .../collect/ForwardingNavigableSet.java       |  237 +
 .../guava25/collect/ForwardingObject.java     |   69 +
 .../guava25/collect/ForwardingQueue.java      |  122 +
 .../guava25/collect/ForwardingSet.java        |  102 +
 .../collect/ForwardingSetMultimap.java        |   64 +
 .../guava25/collect/ForwardingSortedMap.java  |  155 +
 .../collect/ForwardingSortedMultiset.java     |  228 +
 .../guava25/collect/ForwardingSortedSet.java  |  164 +
 .../collect/ForwardingSortedSetMultimap.java  |   67 +
 .../guava25/collect/GeneralRange.java         |  299 +
 .../guava25/collect/GwtTransient.java         |   38 +
 .../guava25/collect/HashBiMap.java            |  740 ++
 .../guava25/collect/HashMultimap.java         |  140 +
 ...hMultimapGwtSerializationDependencies.java |   39 +
 .../guava25/collect/HashMultiset.java         |   91 +
 .../guava25/collect/Hashing.java              |   77 +
 .../guava25/collect/ImmutableAsList.java      |   83 +
 .../guava25/collect/ImmutableBiMap.java       |  433 ++
 .../collect/ImmutableBiMapFauxverideShim.java |   65 +
 .../collect/ImmutableClassToInstanceMap.java  |  191 +
 .../guava25/collect/ImmutableCollection.java  |  441 ++
 .../guava25/collect/ImmutableEntry.java       |   54 +
 .../guava25/collect/ImmutableEnumMap.java     |  137 +
 .../guava25/collect/ImmutableEnumSet.java     |  157 +
 .../guava25/collect/ImmutableList.java        |  866 +++
 .../collect/ImmutableListMultimap.java        |  497 ++
 .../guava25/collect/ImmutableMap.java         |  920 +++
 .../guava25/collect/ImmutableMapEntry.java    |  115 +
 .../guava25/collect/ImmutableMapEntrySet.java |  132 +
 .../guava25/collect/ImmutableMapKeySet.java   |   97 +
 .../guava25/collect/ImmutableMapValues.java   |  121 +
 .../guava25/collect/ImmutableMultimap.java    |  733 ++
 .../guava25/collect/ImmutableMultiset.java    |  630 ++
 ...eMultisetGwtSerializationDependencies.java |   42 +
 .../guava25/collect/ImmutableRangeMap.java    |  411 ++
 .../guava25/collect/ImmutableRangeSet.java    |  831 +++
 .../guava25/collect/ImmutableSet.java         |  833 +++
 .../guava25/collect/ImmutableSetMultimap.java |  637 ++
 .../collect/ImmutableSortedAsList.java        |   93 +
 .../guava25/collect/ImmutableSortedMap.java   |  944 +++
 .../ImmutableSortedMapFauxverideShim.java     |  164 +
 .../collect/ImmutableSortedMultiset.java      |  586 ++
 ...ImmutableSortedMultisetFauxverideShim.java |  202 +
 .../guava25/collect/ImmutableSortedSet.java   |  826 +++
 .../ImmutableSortedSetFauxverideShim.java     |  199 +
 .../guava25/collect/IndexedImmutableSet.java  |   79 +
 .../guava25/collect/Interner.java             |   46 +
 .../guava25/collect/Interners.java            |  188 +
 .../guava25/collect/Iterables.java            | 1033 +++
 .../guava25/collect/Iterators.java            | 1388 ++++
 .../collect/JdkBackedImmutableBiMap.java      |  119 +
 .../collect/JdkBackedImmutableMap.java        |   95 +
 .../collect/JdkBackedImmutableMultiset.java   |   92 +
 .../collect/JdkBackedImmutableSet.java        |   59 +
 .../collect/LexicographicalOrdering.java      |   78 +
 .../guava25/collect/LinkedHashMultimap.java   |  608 ++
 ...hMultimapGwtSerializationDependencies.java |   40 +
 .../guava25/collect/LinkedHashMultiset.java   |  100 +
 .../guava25/collect/LinkedListMultimap.java   |  840 +++
 .../guava25/collect/ListMultimap.java         |   96 +
 .../implementation/guava25/collect/Lists.java | 1143 +++
 .../guava25/collect/MapDifference.java        |  109 +
 .../guava25/collect/MapMaker.java             |  302 +
 .../guava25/collect/MapMakerInternalMap.java  | 2939 ++++++++
 .../implementation/guava25/collect/Maps.java  | 4250 +++++++++++
 .../guava25/collect/MinMaxPriorityQueue.java  |  944 +++
 .../guava25/collect/MoreCollectors.java       |  172 +
 .../guava25/collect/Multimap.java             |  382 +
 .../guava25/collect/MultimapBuilder.java      |  475 ++
 .../guava25/collect/Multimaps.java            | 2195 ++++++
 .../guava25/collect/Multiset.java             |  467 ++
 .../guava25/collect/Multisets.java            | 1177 +++
 .../collect/MutableClassToInstanceMap.java    |  180 +
 .../guava25/collect/NaturalOrdering.java      |   79 +
 .../guava25/collect/NullsFirstOrdering.java   |   88 +
 .../guava25/collect/NullsLastOrdering.java    |   88 +
 .../guava25/collect/ObjectArrays.java         |  223 +
 .../guava25/collect/Ordering.java             |  938 +++
 .../guava25/collect/PeekingIterator.java      |   69 +
 .../guava25/collect/Platform.java             |  109 +
 .../implementation/guava25/collect/Range.java |  689 ++
 .../RangeGwtSerializationDependencies.java    |   35 +
 .../guava25/collect/RangeMap.java             |  166 +
 .../guava25/collect/RangeSet.java             |  279 +
 .../guava25/collect/RegularContiguousSet.java |  235 +
 .../collect/RegularImmutableAsList.java       |   74 +
 .../collect/RegularImmutableBiMap.java        |  305 +
 .../guava25/collect/RegularImmutableList.java |   79 +
 .../guava25/collect/RegularImmutableMap.java  |  309 +
 .../collect/RegularImmutableMultiset.java     |  196 +
 .../guava25/collect/RegularImmutableSet.java  |  110 +
 .../RegularImmutableSortedMultiset.java       |  136 +
 .../collect/RegularImmutableSortedSet.java    |  311 +
 .../collect/ReverseNaturalOrdering.java       |  103 +
 .../guava25/collect/ReverseOrdering.java      |  115 +
 .../guava25/collect/Serialization.java        |  221 +
 .../guava25/collect/SetMultimap.java          |  117 +
 .../implementation/guava25/collect/Sets.java  | 2121 ++++++
 .../collect/SingletonImmutableBiMap.java      |  105 +
 .../collect/SingletonImmutableList.java       |   80 +
 .../collect/SingletonImmutableSet.java        |  104 +
 .../guava25/collect/SortedIterable.java       |   43 +
 .../guava25/collect/SortedIterables.java      |   61 +
 .../guava25/collect/SortedLists.java          |  285 +
 .../guava25/collect/SortedMapDifference.java  |   44 +
 .../guava25/collect/SortedMultiset.java       |  146 +
 .../guava25/collect/SortedMultisetBridge.java |   35 +
 .../guava25/collect/SortedMultisets.java      |  171 +
 .../guava25/collect/SortedSetMultimap.java    |  107 +
 .../guava25/collect/Streams.java              |  871 +++
 .../guava25/collect/Synchronized.java         | 2153 ++++++
 .../implementation/guava25/collect/Table.java |  280 +
 .../guava25/collect/TopKSelector.java         |  275 +
 .../guava25/collect/TransformedIterator.java  |   56 +
 .../collect/TransformedListIterator.java      |   72 +
 .../guava25/collect/TreeMultimap.java         |  221 +
 .../guava25/collect/TreeMultiset.java         | 1030 +++
 .../guava25/collect/TreeRangeMap.java         |  718 ++
 .../guava25/collect/TreeRangeSet.java         |  934 +++
 .../guava25/collect/UnmodifiableIterator.java |   50 +
 .../collect/UnmodifiableListIterator.java     |   59 +
 .../collect/UnmodifiableSortedMultiset.java   |  113 +
 .../collect/UsingToStringOrdering.java        |   47 +
 .../guava25/collect/WellBehavedMap.java       |   99 +
 .../guava25/collect/package-info.java         |  218 +
 .../implementation/guava25/math/IntMath.java  |  724 ++
 .../implementation/guava25/math/LongMath.java | 1193 +++
 .../guava25/math/MathPreconditions.java       |  104 +
 .../guava25/math/package-info.java            |   28 +
 .../guava25/primitives/Booleans.java          |  552 ++
 .../guava25/primitives/Doubles.java           |  697 ++
 .../guava25/primitives/Ints.java              |  743 ++
 .../guava25/primitives/Longs.java             |  791 ++
 .../guava25/primitives/ParseRequest.java      |   57 +
 .../guava25/primitives/Primitives.java        |  140 +
 .../guava25/primitives/UnsignedLongs.java     |  497 ++
 .../guava25/primitives/package-info.java      |   24 +
 .../implementation/sink/CosmosSinkConfig.java |    2 +-
 .../implementation/sink/CosmosSinkTask.java   |    2 +-
 .../sink/KafkaCosmosBulkWriter.java           |    9 +-
 .../sink/KafkaCosmosPointWriter.java          |   31 +-
 .../sink/KafkaCosmosWriterBase.java           |   27 +-
 .../sink/idstrategy/TemplateStrategy.java     |    2 +-
 .../source/ContainersMetadataTopicOffset.java |    6 +-
 .../ContainersMetadataTopicPartition.java     |    4 +-
 .../source/CosmosMetadataConfig.java          |    4 +-
 .../source/CosmosSourceConfig.java            |    4 +-
 .../source/CosmosSourceContainersConfig.java  |    6 +-
 .../CosmosSourceOffsetStorageReader.java      |    4 +-
 .../source/CosmosSourceTask.java              |   69 +-
 .../source/CosmosSourceTaskConfig.java        |    6 +-
 .../FeedRangeContinuationTopicOffset.java     |   20 +-
 .../FeedRangeContinuationTopicPartition.java  |   16 +-
 .../source/FeedRangeTaskUnit.java             |   16 +-
 .../source/FeedRangesMetadataTopicOffset.java |   20 +-
 .../FeedRangesMetadataTopicPartition.java     |    4 +-
 .../source/KafkaCosmosChangeFeedState.java    |   73 +
 .../source/MetadataMonitorThread.java         |   60 +-
 .../source/MetadataTaskUnit.java              |   22 +-
 .../src/main/java/module-info.java            |    3 +-
 .../connect/CosmosDBSinkConnectorTest.java    |   15 +-
 .../connect/CosmosDBSourceConnectorTest.java  |  117 +-
 .../connect/CosmosDbSinkConnectorITest.java   |    4 +-
 .../KafkaCosmosIntegrationTestSuiteBase.java  |    2 +-
 .../KafkaCosmosTestConfigurations.java        |    4 +-
 .../connect/KafkaCosmosTestSuiteBase.java     |    2 +-
 .../source/CosmosSourceTaskTest.java          |   12 +-
 .../source/MetadataMonitorThreadTest.java     |    6 +-
 .../azure-cosmos-spark_3-1_2-12/CHANGELOG.md  |   15 +
 .../azure-cosmos-spark_3-1_2-12/README.md     |    8 +-
 .../azure-cosmos-spark_3-1_2-12/pom.xml       |    2 +-
 .../cosmos/spark/SparkInternalsBridge.scala   |    4 +-
 .../azure-cosmos-spark_3-2_2-12/CHANGELOG.md  |   15 +
 .../azure-cosmos-spark_3-2_2-12/README.md     |    8 +-
 .../azure-cosmos-spark_3-2_2-12/pom.xml       |    2 +-
 .../cosmos/spark/SparkInternalsBridge.scala   |    4 +-
 .../azure-cosmos-spark_3-3_2-12/CHANGELOG.md  |   15 +
 .../azure-cosmos-spark_3-3_2-12/README.md     |    8 +-
 .../azure-cosmos-spark_3-3_2-12/pom.xml       |    2 +-
 .../cosmos/spark/SparkInternalsBridge.scala   |    4 +-
 .../azure-cosmos-spark_3-4_2-12/CHANGELOG.md  |   15 +
 .../azure-cosmos-spark_3-4_2-12/README.md     |    8 +-
 .../azure-cosmos-spark_3-4_2-12/pom.xml       |    2 +-
 .../cosmos/spark/SparkInternalsBridge.scala   |    4 +-
 .../docs/quick-start.md                       |    8 +-
 sdk/cosmos/azure-cosmos-spark_3_2-12/pom.xml  |    8 +-
 .../SparkBridgeImplementationInternal.scala   |    4 +-
 .../com/azure/cosmos/spark/BulkWriter.scala   |   48 +-
 .../com/azure/cosmos/spark/PointWriter.scala  |    2 +-
 sdk/cosmos/azure-cosmos-test/pom.xml          |    6 +-
 .../FaultInjectionConditionBuilder.java       |    4 +-
 ...InjectionConnectionErrorResultBuilder.java |    4 +-
 .../FaultInjectionEndpointBuilder.java        |    4 +-
 .../faultinjection/FaultInjectionRule.java    |    2 +-
 .../FaultInjectionRuleBuilder.java            |    4 +-
 ...aultInjectionServerErrorResultBuilder.java |    2 +-
 .../FaultInjectionConnectionErrorRule.java    |    4 +-
 .../FaultInjectionRuleProcessor.java          |    4 +-
 .../FaultInjectionRuleStore.java              |    4 +-
 ...ultInjectionServerErrorResultInternal.java |    2 +-
 .../FaultInjectionServerErrorRule.java        |    4 +-
 .../faultinjection/FaultInjectorProvider.java |    2 +-
 .../RntbdConnectionErrorInjector.java         |    2 +-
 .../faultinjection/ServerErrorInjector.java   |    2 +-
 sdk/cosmos/azure-cosmos-tests/pom.xml         |   12 +-
 .../com/azure/cosmos/ClientMetricsTest.java   |    2 +-
 .../java/com/azure/cosmos/CosmosBulkTest.java |    4 +-
 .../cosmos/CosmosContainerChangeFeedTest.java |    4 +-
 .../cosmos/CosmosDiagnosticsE2ETest.java      |  102 +-
 .../com/azure/cosmos/CosmosExceptionTest.java |    2 +-
 .../com/azure/cosmos/CosmosTracerTest.java    |    2 +-
 .../azure/cosmos/DocumentCollectionTests.java |    2 +-
 .../SerializableDocumentCollectionTests.java  |    2 +-
 .../java/com/azure/cosmos/TestObject.java     |    2 +-
 .../azure/cosmos/TransactionalBatchTest.java  |    2 +-
 .../ClientConfigDiagnosticsTest.java          |    2 +-
 .../implementation/ClientRetryPolicyTest.java |   99 +-
 .../ImplementationBridgeHelpersTest.java      |  104 +
 .../RetryCreateDocumentTest.java              |    2 +-
 .../implementation/SessionContainerTest.java  |    4 +-
 .../implementation/SqlParameterTest.java      |    2 +-
 .../cosmos/implementation/TestSuiteBase.java  |    4 +-
 .../caches/AsyncCacheNonBlockingTest.java     |    2 +-
 .../caches/SerializableAsyncCacheTest.java    |    2 +-
 .../cpu/CpuLoadHistoryTest.java               |    2 +-
 .../AddressResolverTest.java                  |    4 +-
 .../AddressSelectorTest.java                  |    2 +-
 .../AddressSelectorWrapper.java               |    4 +-
 .../ConsistencyReaderTest.java                |    2 +-
 .../ConsistencyWriterTest.java                |    2 +-
 .../directconnectivity/EndpointMock.java      |    2 +-
 .../GatewayAddressCacheTest.java              |    4 +-
 .../GoneAndRetryWithRetryPolicyTest.java      |    2 +-
 .../MultiStoreResultValidator.java            |    2 +-
 .../PartitionKeyInternalTest.java             |    4 +-
 .../directconnectivity/PartitionKeyTest.java  |    2 +-
 .../directconnectivity/QuorumReaderTest.java  |    4 +-
 .../RntbdTransportClientTest.java             |    4 +-
 .../directconnectivity/StoreReaderTest.java   |    2 +-
 .../StoreReaderUnderTest.java                 |    2 +-
 .../rntbd/ServerRntbdConstants.java           |    6 +-
 .../rntbd/ServerRntbdContext.java             |    8 +-
 .../rntbd/ServerRntbdObjectMapper.java        |    2 +-
 .../rntbd/ServerRntbdRequest.java             |    2 +-
 .../TcpServerMock/rntbd/ServerRntbdToken.java |    6 +-
 .../rntbd/ServerRntbdTokenStream.java         |    8 +-
 .../rntbd/ServerRntbdTokenType.java           |    4 +-
 .../WebExceptionUtilityTest.java              |   17 +-
 .../query/DocumentProducerTest.java           |    8 +-
 .../InMemoryCollectionRoutingMapTest.java     |    2 +-
 .../routing/LocationCacheTest.java            |    4 +-
 .../routing/PartitionKeyInternalUtils.java    |    2 +-
 .../com/azure/cosmos/rx/ChangeFeedTest.java   |    6 +-
 .../cosmos/rx/ClientRetryPolicyE2ETests.java  |  141 +-
 .../azure/cosmos/rx/DistinctQueryTests.java   |    2 +-
 .../cosmos/rx/ParallelDocumentQueryTest.java  |    2 +-
 .../rx/SinglePartitionDocumentQueryTest.java  |    2 +-
 .../com/azure/cosmos/rx/TestSuiteBase.java    |   42 +-
 .../com/azure/cosmos/rx/UniqueIndexTest.java  |    4 +-
 sdk/cosmos/azure-cosmos/CHANGELOG.md          |   47 +-
 sdk/cosmos/azure-cosmos/README.md             |    4 +-
 sdk/cosmos/azure-cosmos/pom.xml               |    2 +-
 .../azure/cosmos/CosmosAsyncContainer.java    |   81 +-
 .../cosmos/CosmosDiagnosticsContext.java      |   14 +-
 ...ringSessionContainerClientRetryPolicy.java |    2 +-
 .../implementation/ClientRetryPolicy.java     |   10 +-
 .../CosmosPagedFluxOptions.java               |    4 +-
 .../implementation/DiagnosticsProvider.java   |   76 +-
 .../implementation/FeedOperationState.java    |   10 +-
 .../implementation/GlobalEndpointManager.java |    2 +-
 .../ImplementationBridgeHelpers.java          |   53 +-
 ...enameCollectionAwareClientRetryPolicy.java |    2 +-
 .../implementation/VectorSessionToken.java    |    2 +-
 .../ClientTelemetryMetrics.java               |    4 +-
 .../ChannelAcquisitionException.java          |   17 +
 .../directconnectivity/ConsistencyWriter.java |   11 +-
 .../directconnectivity/QuorumReader.java      |   20 +-
 .../rntbd/RntbdClientChannelPool.java         |   32 +-
 .../rntbd/RntbdServiceEndpoint.java           |    4 +-
 .../models/CosmosBulkExecutionOptions.java    |   22 +-
 .../CosmosChangeFeedRequestOptions.java       |   51 +
 .../com/azure/cosmos/models/PartitionKey.java |    8 +
 .../azure/cosmos/util/CosmosPagedFlux.java    |   46 +-
 .../src/main/java/module-info.java            |   19 +-
 sdk/cosmos/ci.yml                             |    2 +-
 sdk/cosmos/live-platform-matrix.json          |   16 +-
 sdk/cosmos/tests.yml                          |  179 +-
 .../CHANGELOG.md                              |   61 +-
 .../README.md                                 |   35 +-
 .../SAMPLE.md                                 |  731 +-
 .../pom.xml                                   |   25 +-
 .../CosmosDBForPostgreSqlManager.java         |  102 +-
 .../fluent/ClustersClient.java                |  114 +-
 .../fluent/ConfigurationsClient.java          |   94 +-
 .../fluent/CosmosDBForPostgreSql.java         |   30 +-
 .../fluent/FirewallRulesClient.java           |   64 +-
 .../fluent/OperationsClient.java              |    8 +-
 .../PrivateEndpointConnectionsClient.java     |   74 +-
 .../fluent/PrivateLinkResourcesClient.java    |   20 +-
 .../fluent/RolesClient.java                   |   44 +-
 .../fluent/ServersClient.java                 |   16 +-
 .../fluent/models/ClusterInner.java           |  192 +-
 .../fluent/models/ClusterProperties.java      |  193 +-
 .../models/ClusterPropertiesForUpdate.java    |   82 +-
 .../fluent/models/ClusterServerInner.java     |   58 +-
 .../models/ClusterServerProperties.java       |   48 +-
 .../fluent/models/ConfigurationInner.java     |   34 +-
 .../models/ConfigurationProperties.java       |   36 +-
 .../fluent/models/FirewallRuleInner.java      |   30 +-
 .../fluent/models/FirewallRuleProperties.java |   32 +-
 .../fluent/models/NameAvailabilityInner.java  |   26 +-
 .../fluent/models/OperationInner.java         |   22 +-
 .../PrivateEndpointConnectionInner.java       |   38 +-
 .../PrivateEndpointConnectionProperties.java  |   41 +-
 ...ateEndpointConnectionSimpleProperties.java |   34 +-
 .../models/PrivateLinkResourceInner.java      |   22 +-
 .../models/PrivateLinkResourceProperties.java |   18 +-
 .../fluent/models/RoleInner.java              |  125 +-
 .../fluent/models/RoleProperties.java         |  145 +-
 .../RolePropertiesExternalIdentity.java       |  118 +
 .../models/ServerConfigurationInner.java      |   32 +-
 .../models/ServerConfigurationProperties.java |   34 +-
 .../fluent/models/package-info.java           |    4 +-
 .../fluent/package-info.java                  |    4 +-
 .../implementation/ClusterImpl.java           |   88 +-
 .../implementation/ClusterServerImpl.java     |    3 +-
 .../implementation/ClustersClientImpl.java    | 1467 ++--
 .../implementation/ClustersImpl.java          |  115 +-
 .../implementation/ConfigurationImpl.java     |    3 +-
 .../ConfigurationsClientImpl.java             | 1082 +--
 .../implementation/ConfigurationsImpl.java    |  120 +-
 .../CosmosDBForPostgreSqlBuilder.java         |   46 +-
 .../CosmosDBForPostgreSqlImpl.java            |  139 +-
 .../implementation/FirewallRuleImpl.java      |   57 +-
 .../FirewallRulesClientImpl.java              |  551 +-
 .../implementation/FirewallRulesImpl.java     |  128 +-
 .../implementation/NameAvailabilityImpl.java  |    3 +-
 .../implementation/OperationImpl.java         |    3 +-
 .../implementation/OperationsClientImpl.java  |   72 +-
 .../implementation/OperationsImpl.java        |    7 +-
 .../PrivateEndpointConnectionImpl.java        |   66 +-
 .../PrivateEndpointConnectionsClientImpl.java |  636 +-
 .../PrivateEndpointConnectionsImpl.java       |  169 +-
 .../PrivateLinkResourceImpl.java              |    3 +-
 .../PrivateLinkResourcesClientImpl.java       |  235 +-
 .../PrivateLinkResourcesImpl.java             |   36 +-
 .../implementation/ResourceManagerUtils.java  |  197 +
 .../implementation/RoleImpl.java              |   71 +-
 .../implementation/RolesClientImpl.java       |  499 +-
 .../implementation/RolesImpl.java             |  124 +-
 .../ServerConfigurationImpl.java              |    3 +-
 .../implementation/ServersClientImpl.java     |  220 +-
 .../implementation/ServersImpl.java           |   24 +-
 .../implementation/Utils.java                 |  204 -
 .../implementation/package-info.java          |    4 +-
 .../models/ActiveDirectoryAuth.java           |   53 +
 .../models/AuthConfig.java                    |   80 +
 .../cosmosdbforpostgresql/models/Cluster.java |  510 +-
 .../ClusterConfigurationListResult.java       |   16 +-
 .../models/ClusterForUpdate.java              |   86 +-
 .../models/ClusterListResult.java             |   16 +-
 .../models/ClusterServer.java                 |   42 +-
 .../models/ClusterServerListResult.java       |   14 +-
 .../models/Clusters.java                      |   66 +-
 .../models/Configuration.java                 |   26 +-
 .../models/ConfigurationDataType.java         |   26 +-
 .../models/Configurations.java                |   72 +-
 .../models/FirewallRule.java                  |   94 +-
 .../models/FirewallRuleListResult.java        |   14 +-
 .../models/FirewallRules.java                 |   30 +-
 .../models/MaintenanceWindow.java             |   26 +-
 .../models/NameAvailability.java              |   14 +-
 .../models/NameAvailabilityRequest.java       |   24 +-
 .../models/Operation.java                     |   16 +-
 .../models/OperationDisplay.java              |   18 +-
 .../models/OperationListResult.java           |   20 +-
 .../models/OperationOrigin.java               |   22 +-
 .../models/Operations.java                    |    8 +-
 .../models/PasswordAuth.java                  |   53 +
 .../models/PrincipalType.java                 |   58 +
 .../models/PrivateEndpoint.java               |   12 +-
 .../models/PrivateEndpointConnection.java     |   88 +-
 .../PrivateEndpointConnectionListResult.java  |   14 +-
 ...teEndpointConnectionProvisioningState.java |   26 +-
 .../models/PrivateEndpointConnections.java    |   34 +-
 .../models/PrivateEndpointProperty.java       |   14 +-
 ...rivateEndpointServiceConnectionStatus.java |   22 +-
 .../models/PrivateLinkResource.java           |   20 +-
 .../models/PrivateLinkResourceListResult.java |   14 +-
 .../models/PrivateLinkResources.java          |   16 +-
 .../PrivateLinkServiceConnectionState.java    |   22 +-
 .../models/PromoteRequest.java                |   54 +
 .../models/ProvisioningState.java             |   26 +-
 .../cosmosdbforpostgresql/models/Role.java    |  162 +-
 .../models/RoleListResult.java                |   14 +-
 .../models/RoleType.java                      |   53 +
 .../cosmosdbforpostgresql/models/Roles.java   |   26 +-
 .../models/ServerConfiguration.java           |   30 +-
 .../models/ServerConfigurationListResult.java |   16 +-
 .../models/ServerNameItem.java                |   16 +-
 .../models/ServerProperties.java              |   32 +-
 .../models/ServerRole.java                    |   18 +-
 .../models/ServerRoleGroupConfiguration.java  |   34 +-
 .../cosmosdbforpostgresql/models/Servers.java |   16 +-
 .../SimplePrivateEndpointConnection.java      |   38 +-
 .../models/package-info.java                  |    4 +-
 .../cosmosdbforpostgresql/package-info.java   |    4 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       |  266 +
 .../ClustersCheckNameAvailabilitySamples.java |   20 +-
 .../generated/ClustersCreateSamples.java      |  157 +-
 .../generated/ClustersDeleteSamples.java      |   14 +-
 .../ClustersGetByResourceGroupSamples.java    |   19 +-
 .../ClustersListByResourceGroupSamples.java   |   10 +-
 .../generated/ClustersListSamples.java        |   14 +-
 .../ClustersPromoteReadReplicaSamples.java    |   12 +-
 .../generated/ClustersRestartSamples.java     |   10 +-
 .../generated/ClustersStartSamples.java       |   10 +-
 .../generated/ClustersStopSamples.java        |   10 +-
 .../generated/ClustersUpdateSamples.java      |   96 +-
 .../ConfigurationsGetCoordinatorSamples.java  |   16 +-
 .../ConfigurationsGetNodeSamples.java         |   15 +-
 .../generated/ConfigurationsGetSamples.java   |   19 +-
 .../ConfigurationsListByClusterSamples.java   |   10 +-
 .../ConfigurationsListByServerSamples.java    |   15 +-
 ...figurationsUpdateOnCoordinatorSamples.java |   20 +-
 .../ConfigurationsUpdateOnNodeSamples.java    |   20 +-
 .../FirewallRulesCreateOrUpdateSamples.java   |   19 +-
 .../generated/FirewallRulesDeleteSamples.java |   10 +-
 .../generated/FirewallRulesGetSamples.java    |   10 +-
 .../FirewallRulesListByClusterSamples.java    |   10 +-
 .../generated/OperationsListSamples.java      |   10 +-
 ...pointConnectionsCreateOrUpdateSamples.java |   17 +-
 ...ivateEndpointConnectionsDeleteSamples.java |   15 +-
 .../PrivateEndpointConnectionsGetSamples.java |   16 +-
 ...dpointConnectionsListByClusterSamples.java |   15 +-
 .../PrivateLinkResourcesGetSamples.java       |   15 +-
 ...vateLinkResourcesListByClusterSamples.java |   15 +-
 .../generated/RolesCreateSamples.java         |   20 +-
 .../generated/RolesDeleteSamples.java         |   14 +-
 .../generated/RolesGetSamples.java            |   14 +-
 .../generated/RolesListByClusterSamples.java  |   10 +-
 .../generated/ServersGetSamples.java          |   19 +-
 .../ServersListByClusterSamples.java          |   14 +-
 .../ClusterConfigurationListResultTests.java  |   58 +-
 .../generated/ClusterServerInnerTests.java    |   54 +-
 .../ClusterServerListResultTests.java         |   76 +-
 .../ClusterServerPropertiesTests.java         |   46 +-
 ...NameAvailabilityWithResponseMockTests.java |   47 +-
 .../generated/ClustersDeleteMockTests.java    |   36 +-
 .../generated/ClustersStartMockTests.java     |   36 +-
 .../generated/ClustersStopMockTests.java      |   36 +-
 .../generated/ConfigurationInnerTests.java    |   23 +-
 .../ConfigurationPropertiesTests.java         |   32 +-
 ...nsGetCoordinatorWithResponseMockTests.java |   44 +-
 ...gurationsGetNodeWithResponseMockTests.java |   43 +-
 ...onfigurationsGetWithResponseMockTests.java |   44 +-
 .../ConfigurationsListByClusterMockTests.java |   45 +-
 .../ConfigurationsListByServerMockTests.java  |   39 +-
 ...gurationsUpdateOnCoordinatorMockTests.java |   46 +-
 .../ConfigurationsUpdateOnNodeMockTests.java  |   46 +-
 .../generated/FirewallRuleInnerTests.java     |   20 +-
 .../FirewallRuleListResultTests.java          |   27 +-
 .../FirewallRulePropertiesTests.java          |   21 +-
 .../FirewallRulesCreateOrUpdateMockTests.java |   48 +-
 ...FirewallRulesGetWithResponseMockTests.java |   44 +-
 .../FirewallRulesListByClusterMockTests.java  |   41 +-
 .../generated/MaintenanceWindowTests.java     |   16 +-
 .../generated/NameAvailabilityInnerTests.java |   26 +-
 .../NameAvailabilityRequestTests.java         |   10 +-
 .../generated/OperationDisplayTests.java      |    8 +-
 .../generated/OperationInnerTests.java        |    8 +-
 .../generated/OperationListResultTests.java   |   19 +-
 .../generated/OperationsListMockTests.java    |   35 +-
 .../PrivateEndpointConnectionInnerTests.java  |   38 +-
 ...vateEndpointConnectionListResultTests.java |   80 +-
 ...vateEndpointConnectionPropertiesTests.java |   38 +-
 ...dpointConnectionSimplePropertiesTests.java |   32 +-
 ...intConnectionsCreateOrUpdateMockTests.java |   60 +-
 ...ntConnectionsGetWithResponseMockTests.java |   55 +-
 ...ointConnectionsListByClusterMockTests.java |   58 +-
 .../PrivateEndpointPropertyTests.java         |    4 +-
 .../generated/PrivateEndpointTests.java       |    2 +-
 .../PrivateLinkResourceInnerTests.java        |   15 +-
 .../PrivateLinkResourceListResultTests.java   |   23 +-
 .../PrivateLinkResourcePropertiesTests.java   |   16 +-
 ...LinkResourcesGetWithResponseMockTests.java |   42 +-
 ...teLinkResourcesListByClusterMockTests.java |   39 +-
 ...rivateLinkServiceConnectionStateTests.java |   16 +-
 .../generated/PromoteRequestTests.java        |   24 +
 .../RolePropertiesExternalIdentityTests.java  |   32 +
 .../ServerConfigurationInnerTests.java        |   14 +-
 .../ServerConfigurationListResultTests.java   |   25 +-
 .../ServerConfigurationPropertiesTests.java   |   14 +-
 .../generated/ServerNameItemTests.java        |    5 +-
 .../generated/ServerPropertiesTests.java      |   32 +-
 .../ServerRoleGroupConfigurationTests.java    |   20 +-
 .../ServersGetWithResponseMockTests.java      |   53 +-
 .../ServersListByClusterMockTests.java        |   51 +-
 .../SimplePrivateEndpointConnectionTests.java |   32 +-
 sdk/cosmosdbforpostgresql/tests.mgmt.yml      |   24 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-dashboard/pom.xml   |    8 +-
 sdk/dashboard/tests.mgmt.yml                  |   24 +-
 .../azure-resourcemanager-databox/pom.xml     |    6 +-
 .../azure-resourcemanager-databoxedge/pom.xml |    6 +-
 .../azure-resourcemanager-databricks/pom.xml  |    8 +-
 sdk/databricks/tests.mgmt.yml                 |   24 +-
 .../azure-resourcemanager-datadog/pom.xml     |    6 +-
 .../CHANGELOG.md                              |   34 +-
 .../README.md                                 |    2 +-
 .../SAMPLE.md                                 | 1050 ++-
 .../azure-resourcemanager-datafactory/pom.xml |   12 +-
 .../datafactory/DataFactoryManager.java       |    2 +-
 ...GoogleBigQueryV2DatasetTypeProperties.java |   84 +
 ...BigQueryV2LinkedServiceTypeProperties.java |  246 +
 ...stgreSqlV2LinkedServiceTypeProperties.java |  568 ++
 ...ostgreSqlV2TableDatasetTypeProperties.java |   80 +
 ...rviceNowV2LinkedServiceTypeProperties.java |  261 +
 .../ManagedPrivateEndpointsClientImpl.java    |    2 +-
 .../implementation/PipelinesClientImpl.java   |    2 +-
 .../datafactory/models/Dataset.java           |    5 +-
 .../datafactory/models/ExpressionV2.java      |  136 +
 .../datafactory/models/ExpressionV2Type.java  |   63 +
 .../GoogleBigQueryV2AuthenticationType.java   |   53 +
 .../models/GoogleBigQueryV2LinkedService.java |  269 +
 .../models/GoogleBigQueryV2ObjectDataset.java |  168 +
 .../models/GoogleBigQueryV2Source.java        |  116 +
 .../datafactory/models/LinkedService.java     |    5 +-
 .../models/PostgreSqlV2LinkedService.java     |  552 ++
 .../models/PostgreSqlV2Source.java            |  114 +
 .../models/PostgreSqlV2TableDataset.java      |  164 +
 .../ServiceNowV2AuthenticationType.java       |   53 +
 .../models/ServiceNowV2LinkedService.java     |  285 +
 .../models/ServiceNowV2ObjectDataset.java     |  141 +
 .../models/ServiceNowV2Source.java            |  117 +
 .../datafactory/models/TabularSource.java     |    5 +-
 .../src/main/java/module-info.java            |    2 -
 .../reflect-config.json                       |   90 +
 .../generated/ActivityPolicyTests.java        |   14 +-
 .../AmazonMwsObjectDatasetTests.java          |   43 +-
 .../generated/AmazonMwsSourceTests.java       |   10 +-
 ...zonRdsForOraclePartitionSettingsTests.java |    6 +-
 .../AmazonRdsForOracleSourceTests.java        |   18 +-
 .../AmazonRdsForSqlServerSourceTests.java     |   19 +-
 ...mazonRdsForSqlServerTableDatasetTests.java |   42 +-
 ...ServerTableDatasetTypePropertiesTests.java |    6 +-
 .../generated/AmazonRedshiftSourceTests.java  |   22 +-
 .../AmazonS3CompatibleReadSettingsTests.java  |   13 +-
 .../generated/AmazonS3ReadSettingsTests.java  |   14 +-
 .../AppendVariableActivityTests.java          |   49 +-
 ...ndVariableActivityTypePropertiesTests.java |   10 +-
 .../datafactory/generated/AvroSinkTests.java  |   31 +-
 .../generated/AvroSourceTests.java            |   13 +-
 .../generated/AvroWriteSettingsTests.java     |   14 +-
 .../generated/AzPowerShellSetupTests.java     |   12 +-
 .../AzPowerShellSetupTypePropertiesTests.java |    8 +-
 .../AzureBlobFSReadSettingsTests.java         |   14 +-
 .../generated/AzureBlobFSSinkTests.java       |   14 +-
 .../generated/AzureBlobFSSourceTests.java     |   10 +-
 .../AzureBlobFSWriteSettingsTests.java        |   13 +-
 .../AzureBlobStorageReadSettingsTests.java    |   12 +-
 .../AzureBlobStorageWriteSettingsTests.java   |   12 +-
 ...AzureDataExplorerCommandActivityTests.java |   83 +-
 ...rerCommandActivityTypePropertiesTests.java |    4 +-
 .../generated/AzureDataExplorerSinkTests.java |   12 +-
 .../AzureDataExplorerSourceTests.java         |   11 +-
 .../AzureDataLakeStoreReadSettingsTests.java  |   14 +-
 .../AzureDataLakeStoreSinkTests.java          |   11 +-
 .../AzureDataLakeStoreSourceTests.java        |    8 +-
 .../AzureDataLakeStoreWriteSettingsTests.java |   14 +-
 ...DatabricksDeltaLakeExportCommandTests.java |    4 +-
 ...DatabricksDeltaLakeImportCommandTests.java |    6 +-
 .../AzureDatabricksDeltaLakeSinkTests.java    |   14 +-
 .../AzureDatabricksDeltaLakeSourceTests.java  |   12 +-
 .../AzureFileStorageReadSettingsTests.java    |   13 +-
 .../AzureFileStorageWriteSettingsTests.java   |   13 +-
 .../generated/AzureFunctionActivityTests.java |   94 +-
 ...reFunctionActivityTypePropertiesTests.java |   14 +-
 .../AzureMLBatchExecutionActivityTests.java   |  118 +-
 ...hExecutionActivityTypePropertiesTests.java |   48 +-
 .../AzureMLExecutePipelineActivityTests.java  |   97 +-
 ...tePipelineActivityTypePropertiesTests.java |   12 +-
 .../AzureMLUpdateResourceActivityTests.java   |  105 +-
 ...teResourceActivityTypePropertiesTests.java |   15 +-
 .../generated/AzureMLWebServiceFileTests.java |   12 +-
 .../generated/AzureMariaDBSourceTests.java    |    8 +-
 .../AzureMariaDBTableDatasetTests.java        |   40 +-
 .../generated/AzureMySqlSinkTests.java        |   10 +-
 .../generated/AzureMySqlSourceTests.java      |    8 +-
 .../generated/AzurePostgreSqlSinkTests.java   |   10 +-
 .../generated/AzurePostgreSqlSourceTests.java |   10 +-
 .../AzurePostgreSqlTableDatasetTests.java     |   44 +-
 ...greSqlTableDatasetTypePropertiesTests.java |    4 +-
 .../generated/AzureQueueSinkTests.java        |    9 +-
 .../AzureSearchIndexDatasetTests.java         |   40 +-
 ...SearchIndexDatasetTypePropertiesTests.java |    4 +-
 .../generated/AzureSearchIndexSinkTests.java  |   10 +-
 .../generated/AzureSqlSourceTests.java        |   19 +-
 ...ureSynapseArtifactsLinkedServiceTests.java |   34 +-
 ...factsLinkedServiceTypePropertiesTests.java |    4 +-
 .../generated/AzureTableSourceTests.java      |   12 +-
 ...DataPoolParametrizationReferenceTests.java |    4 +-
 .../generated/BinaryReadSettingsTests.java    |    2 +-
 .../generated/BinarySinkTests.java            |   20 +-
 .../generated/BinarySourceTests.java          |   10 +-
 .../generated/BlobEventsTriggerTests.java     |   52 +-
 .../BlobEventsTriggerTypePropertiesTests.java |   29 +-
 .../datafactory/generated/BlobSinkTests.java  |   19 +-
 .../generated/BlobSourceTests.java            |    9 +-
 .../generated/BlobTriggerTests.java           |   51 +-
 .../BlobTriggerTypePropertiesTests.java       |   20 +-
 .../generated/CassandraSourceTests.java       |   14 +-
 .../generated/ChainingTriggerTests.java       |   40 +-
 .../ChainingTriggerTypePropertiesTests.java   |   23 +-
 ...esCreateOrUpdateWithResponseMockTests.java |  110 +-
 ...taCapturesDeleteWithResponseMockTests.java |    2 +-
 ...eDataCapturesGetWithResponseMockTests.java |   34 +-
 ...ngeDataCapturesListByFactoryMockTests.java |   30 +-
 ...ataCapturesStartWithResponseMockTests.java |    2 +-
 ...taCapturesStatusWithResponseMockTests.java |    6 +-
 ...DataCapturesStopWithResponseMockTests.java |    2 +-
 .../CommonDataServiceForAppsSourceTests.java  |    9 +-
 .../CompressionReadSettingsTests.java         |    4 +-
 .../generated/ConcurObjectDatasetTests.java   |   45 +-
 .../generated/ConcurSourceTests.java          |    9 +-
 .../generated/ControlActivityTests.java       |   55 +-
 .../CopyActivityLogSettingsTests.java         |    6 +-
 .../generated/CopyActivityTests.java          |  171 +-
 .../CopyActivityTypePropertiesTests.java      |   82 +-
 .../CopyComputeScalePropertiesTests.java      |   14 +-
 .../datafactory/generated/CopySinkTests.java  |    9 +-
 .../generated/CopySourceTests.java            |    6 +-
 .../generated/CopyTranslatorTests.java        |    3 +-
 .../CosmosDbMongoDbApiLinkedServiceTests.java |   34 +-
 ...DbApiLinkedServiceTypePropertiesTests.java |    7 +-
 .../CosmosDbMongoDbApiSinkTests.java          |   10 +-
 .../CosmosDbMongoDbApiSourceTests.java        |   17 +-
 .../generated/CosmosDbSqlApiSinkTests.java    |   10 +-
 .../generated/CosmosDbSqlApiSourceTests.java  |   12 +-
 .../generated/CouchbaseSourceTests.java       |   10 +-
 .../generated/CouchbaseTableDatasetTests.java |   37 +-
 ...nsCreateOrUpdateWithResponseMockTests.java |   20 +-
 ...OperationsDeleteWithResponseMockTests.java |    2 +-
 ...ialOperationsGetWithResponseMockTests.java |   10 +-
 ...ntialOperationsListByFactoryMockTests.java |   10 +-
 .../CustomActivityReferenceObjectTests.java   |   41 +-
 .../generated/CustomActivityTests.java        |  148 +-
 .../CustomActivityTypePropertiesTests.java    |   59 +-
 .../CustomDataSourceLinkedServiceTests.java   |   29 +-
 .../generated/CustomEventsTriggerTests.java   |   51 +-
 ...ustomEventsTriggerTypePropertiesTests.java |   20 +-
 .../DWCopyCommandDefaultValueTests.java       |    6 +-
 .../generated/DWCopyCommandSettingsTests.java |   15 +-
 ...sionsAddDataFlowWithResponseMockTests.java |  178 +-
 .../DataFlowDebugSessionsCreateMockTests.java |   21 +-
 ...ugSessionsDeleteWithResponseMockTests.java |    4 +-
 ...wDebugSessionsExecuteCommandMockTests.java |   17 +-
 ...wDebugSessionsQueryByFactoryMockTests.java |   24 +-
 ...wsCreateOrUpdateWithResponseMockTests.java |   19 +-
 .../DataFlowsDeleteWithResponseMockTests.java |    3 +-
 .../DataFlowsGetWithResponseMockTests.java    |   10 +-
 .../DataFlowsListByFactoryMockTests.java      |   10 +-
 .../DataLakeAnalyticsUsqlActivityTests.java   |   84 +-
 ...lyticsUsqlActivityTypePropertiesTests.java |   21 +-
 .../DatabricksNotebookActivityTests.java      |   91 +-
 ...ksNotebookActivityTypePropertiesTests.java |   12 +-
 .../DatabricksSparkJarActivityTests.java      |   79 +-
 ...ksSparkJarActivityTypePropertiesTests.java |   15 +-
 .../DatabricksSparkPythonActivityTests.java   |   97 +-
 ...parkPythonActivityTypePropertiesTests.java |   18 +-
 ...tsCreateOrUpdateWithResponseMockTests.java |   43 +-
 .../DatasetsDeleteWithResponseMockTests.java  |    3 +-
 .../DatasetsGetWithResponseMockTests.java     |   16 +-
 .../DatasetsListByFactoryMockTests.java       |   19 +-
 .../datafactory/generated/Db2SourceTests.java |    8 +-
 .../generated/DeleteActivityTests.java        |  109 +-
 .../DeleteActivityTypePropertiesTests.java    |   35 +-
 .../DelimitedTextReadSettingsTests.java       |    4 +-
 .../generated/DelimitedTextSinkTests.java     |   24 +-
 .../generated/DelimitedTextSourceTests.java   |   21 +-
 .../DelimitedTextWriteSettingsTests.java      |    6 +-
 .../generated/DistcpSettingsTests.java        |    6 +-
 .../DocumentDbCollectionSinkTests.java        |   12 +-
 .../DocumentDbCollectionSourceTests.java      |   11 +-
 .../DrillDatasetTypePropertiesTests.java      |    6 +-
 .../generated/DrillSourceTests.java           |    9 +-
 .../generated/DrillTableDatasetTests.java     |   44 +-
 .../generated/DynamicsAXSourceTests.java      |   10 +-
 .../generated/DynamicsCrmSourceTests.java     |    9 +-
 .../generated/DynamicsSourceTests.java        |    8 +-
 .../generated/EloquaObjectDatasetTests.java   |   43 +-
 .../generated/EloquaSourceTests.java          |    9 +-
 .../generated/EntityReferenceTests.java       |    8 +-
 .../EnvironmentVariableSetupTests.java        |   12 +-
 ...nmentVariableSetupTypePropertiesTests.java |   14 +-
 .../generated/ExcelSourceTests.java           |   12 +-
 .../ExecuteDataFlowActivityTests.java         |  123 +-
 ...lowActivityTypePropertiesComputeTests.java |    4 +-
 ...teDataFlowActivityTypePropertiesTests.java |   43 +-
 .../ExecutePipelineActivityPolicyTests.java   |    6 +-
 .../ExecutePipelineActivityTests.java         |   74 +-
 ...tePipelineActivityTypePropertiesTests.java |   16 +-
 ...PowerQueryActivityTypePropertiesTests.java |  473 +-
 ...ExecuteWranglingDataflowActivityTests.java |  471 +-
 .../generated/ExecutionActivityTests.java     |   77 +-
 .../generated/ExportSettingsTests.java        |    6 +-
 ...reValueByFactoryWithResponseMockTests.java |    6 +-
 ...sGetFeatureValueWithResponseMockTests.java |    6 +-
 ...eValuesByFactoryWithResponseMockTests.java |   10 +-
 .../generated/ExpressionV2Tests.java          |   86 +
 ...eByResourceGroupWithResponseMockTests.java |    2 +-
 .../generated/FactoryIdentityTests.java       |    2 +-
 .../FactoryUpdateParametersTests.java         |    2 +-
 .../FileServerReadSettingsTests.java          |   15 +-
 .../FileServerWriteSettingsTests.java         |   13 +-
 .../generated/FileSystemSinkTests.java        |   10 +-
 .../generated/FileSystemSourceTests.java      |    9 +-
 .../generated/FilterActivityTests.java        |   60 +-
 .../FilterActivityTypePropertiesTests.java    |   12 +-
 .../generated/ForEachActivityTests.java       |  173 +-
 .../ForEachActivityTypePropertiesTests.java   |  197 +-
 .../generated/FormatReadSettingsTests.java    |    4 +-
 .../generated/FormatWriteSettingsTests.java   |    2 +-
 .../generated/FtpReadSettingsTests.java       |   14 +-
 .../GenericDatasetTypePropertiesTests.java    |    6 +-
 .../generated/GetMetadataActivityTests.java   |   76 +-
 ...etMetadataActivityTypePropertiesTests.java |   17 +-
 ...rsCreateOrUpdateWithResponseMockTests.java |   20 +-
 ...ParametersDeleteWithResponseMockTests.java |    2 +-
 ...balParametersGetWithResponseMockTests.java |    8 +-
 ...lobalParametersListByFactoryMockTests.java |   12 +-
 .../generated/GoogleAdWordsSourceTests.java   |   10 +-
 ...gleBigQueryDatasetTypePropertiesTests.java |   11 +-
 .../GoogleBigQueryObjectDatasetTests.java     |   40 +-
 .../generated/GoogleBigQuerySourceTests.java  |   10 +-
 ...eBigQueryV2DatasetTypePropertiesTests.java |   24 +
 .../GoogleBigQueryV2ObjectDatasetTests.java   |   62 +
 .../GoogleBigQueryV2SourceTests.java          |   26 +
 .../GoogleCloudStorageReadSettingsTests.java  |   14 +-
 .../GreenplumDatasetTypePropertiesTests.java  |   10 +-
 .../generated/GreenplumSourceTests.java       |   10 +-
 .../generated/GreenplumTableDatasetTests.java |   41 +-
 .../generated/HBaseObjectDatasetTests.java    |   45 +-
 .../generated/HBaseSourceTests.java           |    9 +-
 .../generated/HDInsightHiveActivityTests.java |  125 +-
 ...nsightHiveActivityTypePropertiesTests.java |   49 +-
 .../HDInsightMapReduceActivityTests.java      |   97 +-
 ...tMapReduceActivityTypePropertiesTests.java |   34 +-
 .../generated/HDInsightPigActivityTests.java  |   97 +-
 ...InsightPigActivityTypePropertiesTests.java |   38 +-
 .../HDInsightSparkActivityTests.java          |   91 +-
 ...sightSparkActivityTypePropertiesTests.java |   26 +-
 .../HDInsightStreamingActivityTests.java      |  126 +-
 ...tStreamingActivityTypePropertiesTests.java |   45 +-
 .../generated/HdfsReadSettingsTests.java      |   19 +-
 .../generated/HdfsSourceTests.java            |   10 +-
 .../HiveDatasetTypePropertiesTests.java       |    8 +-
 .../generated/HiveObjectDatasetTests.java     |   38 +-
 .../generated/HiveSourceTests.java            |    9 +-
 .../generated/HttpDatasetTests.java           |   52 +-
 .../HttpDatasetTypePropertiesTests.java       |   10 +-
 .../generated/HttpReadSettingsTests.java      |   10 +-
 .../generated/HttpSourceTests.java            |    8 +-
 .../generated/HubspotObjectDatasetTests.java  |   38 +-
 .../generated/HubspotSourceTests.java         |   10 +-
 .../generated/IfConditionActivityTests.java   |  302 +-
 ...fConditionActivityTypePropertiesTests.java |  193 +-
 .../ImpalaDatasetTypePropertiesTests.java     |   10 +-
 .../generated/ImpalaObjectDatasetTests.java   |   40 +-
 .../generated/ImpalaSourceTests.java          |    9 +-
 .../generated/ImportSettingsTests.java        |    6 +-
 .../generated/InformixSinkTests.java          |   10 +-
 .../generated/InformixSourceTests.java        |    9 +-
 ...egrationRuntimeComputePropertiesTests.java |  110 +-
 ...ionRuntimeCustomerVirtualNetworkTests.java |   10 +-
 ...owPropertiesCustomPropertiesItemTests.java |   12 +-
 ...grationRuntimeDataFlowPropertiesTests.java |   44 +-
 ...rationRuntimeDataProxyPropertiesTests.java |   23 +-
 ...ntimeNodesDeleteWithResponseMockTests.java |    2 +-
 ...odesGetIpAddressWithResponseMockTests.java |    8 +-
 ...nRuntimeNodesGetWithResponseMockTests.java |    5 +-
 ...ntimeNodesUpdateWithResponseMockTests.java |    6 +-
 ...jectMetadatasGetWithResponseMockTests.java |   15 +-
 ...untimeObjectMetadatasRefreshMockTests.java |   14 +-
 ...IntegrationRuntimeVNetPropertiesTests.java |   24 +-
 ...tegrationRuntimeWithResponseMockTests.java |    8 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   15 +-
 ...onRuntimesDeleteWithResponseMockTests.java |    2 +-
 ...etMonitoringDataWithResponseMockTests.java |    6 +-
 ...untimesGetStatusWithResponseMockTests.java |    5 +-
 ...ationRuntimesGetWithResponseMockTests.java |    8 +-
 ...grationRuntimesListByFactoryMockTests.java |   10 +-
 ...denciesEndpointsWithResponseMockTests.java |   10 +-
 ...timesRemoveLinksWithResponseMockTests.java |    4 +-
 .../IntegrationRuntimesStartMockTests.java    |    6 +-
 .../IntegrationRuntimesStopMockTests.java     |    2 +-
 ...sSyncCredentialsWithResponseMockTests.java |    2 +-
 ...nRuntimesUpgradeWithResponseMockTests.java |    2 +-
 .../generated/JiraObjectDatasetTests.java     |   35 +-
 .../generated/JiraSourceTests.java            |    9 +-
 .../generated/JsonReadSettingsTests.java      |    2 +-
 .../datafactory/generated/JsonSinkTests.java  |   19 +-
 .../generated/JsonSourceTests.java            |   13 +-
 .../generated/JsonWriteSettingsTests.java     |    6 +-
 .../generated/LakeHouseReadSettingsTests.java |   14 +-
 .../generated/LakeHouseTableSinkTests.java    |   12 +-
 .../generated/LakeHouseTableSourceTests.java  |    8 +-
 .../LakeHouseWriteSettingsTests.java          |   12 +-
 .../LinkedIntegrationRuntimeTests.java        |    2 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   33 +-
 ...edServicesDeleteWithResponseMockTests.java |    2 +-
 ...inkedServicesGetWithResponseMockTests.java |   13 +-
 .../LinkedServicesListByFactoryMockTests.java |   14 +-
 .../generated/LogLocationSettingsTests.java   |   12 +-
 .../generated/LogSettingsTests.java           |   16 +-
 .../generated/LogStorageSettingsTests.java    |   13 +-
 .../generated/LookupActivityTests.java        |   82 +-
 .../LookupActivityTypePropertiesTests.java    |   16 +-
 .../generated/MagentoObjectDatasetTests.java  |   39 +-
 .../generated/MagentoSourceTests.java         |    9 +-
 ...tsCreateOrUpdateWithResponseMockTests.java |   22 +-
 ...eEndpointsDeleteWithResponseMockTests.java |    2 +-
 ...vateEndpointsGetWithResponseMockTests.java |   15 +-
 ...rivateEndpointsListByFactoryMockTests.java |   12 +-
 ...ksCreateOrUpdateWithResponseMockTests.java |   10 +-
 ...rtualNetworksGetWithResponseMockTests.java |    6 +-
 ...VirtualNetworksListByFactoryMockTests.java |    8 +-
 .../generated/MariaDBSourceTests.java         |    9 +-
 .../generated/MariaDBTableDatasetTests.java   |   41 +-
 .../generated/MarketoObjectDatasetTests.java  |   37 +-
 .../generated/MarketoSourceTests.java         |   10 +-
 .../generated/MetadataItemTests.java          |    4 +-
 .../generated/MicrosoftAccessSinkTests.java   |   10 +-
 .../generated/MicrosoftAccessSourceTests.java |    8 +-
 .../MicrosoftAccessTableDatasetTests.java     |   43 +-
 ...AccessTableDatasetTypePropertiesTests.java |    4 +-
 .../MongoDbAtlasLinkedServiceTests.java       |   42 +-
 ...AtlasLinkedServiceTypePropertiesTests.java |    4 +-
 .../generated/MongoDbAtlasSinkTests.java      |   10 +-
 .../generated/MongoDbAtlasSourceTests.java    |   15 +-
 .../MongoDbCursorMethodsPropertiesTests.java  |    6 +-
 .../generated/MongoDbSourceTests.java         |    8 +-
 .../MongoDbV2LinkedServiceTests.java          |   29 +-
 ...oDbV2LinkedServiceTypePropertiesTests.java |    6 +-
 .../generated/MongoDbV2SinkTests.java         |   10 +-
 .../generated/MongoDbV2SourceTests.java       |   15 +-
 .../MultiplePipelineTriggerTests.java         |   38 +-
 .../generated/MySqlSourceTests.java           |   10 +-
 .../NetezzaPartitionSettingsTests.java        |    6 +-
 .../generated/NetezzaSourceTests.java         |   14 +-
 .../generated/NotebookParameterTests.java     |    8 +-
 .../generated/ODataSourceTests.java           |   10 +-
 .../datafactory/generated/OdbcSinkTests.java  |   10 +-
 .../generated/OdbcSourceTests.java            |    9 +-
 .../generated/Office365SourceTests.java       |   12 +-
 .../generated/OperationsListMockTests.java    |   36 +-
 .../OracleCloudStorageReadSettingsTests.java  |   14 +-
 .../OraclePartitionSettingsTests.java         |    6 +-
 .../OracleServiceCloudSourceTests.java        |   10 +-
 .../generated/OracleSinkTests.java            |   10 +-
 .../generated/OracleSourceTests.java          |   17 +-
 .../datafactory/generated/OrcSinkTests.java   |   19 +-
 .../datafactory/generated/OrcSourceTests.java |   14 +-
 .../generated/OrcWriteSettingsTests.java      |    4 +-
 .../generated/PackageStoreTests.java          |   15 +-
 .../generated/ParquetReadSettingsTests.java   |    2 +-
 .../generated/ParquetSinkTests.java           |   20 +-
 .../generated/ParquetSourceTests.java         |   19 +-
 .../generated/ParquetWriteSettingsTests.java  |    4 +-
 .../generated/PaypalObjectDatasetTests.java   |   45 +-
 .../generated/PaypalSourceTests.java          |   10 +-
 .../PhoenixDatasetTypePropertiesTests.java    |   10 +-
 .../generated/PhoenixObjectDatasetTests.java  |   45 +-
 .../generated/PhoenixSourceTests.java         |    8 +-
 ...neExternalComputeScalePropertiesTests.java |   16 +-
 ...pelineRunsCancelWithResponseMockTests.java |    3 +-
 .../PipelineRunsGetWithResponseMockTests.java |    4 +-
 ...esCreateOrUpdateWithResponseMockTests.java |  115 +-
 ...pelinesCreateRunWithResponseMockTests.java |   10 +-
 .../PipelinesDeleteWithResponseMockTests.java |    2 +-
 .../PipelinesGetWithResponseMockTests.java    |   26 +-
 .../PipelinesListByFactoryMockTests.java      |   31 +-
 .../generated/PolybaseSettingsTests.java      |   12 +-
 .../generated/PostgreSqlSourceTests.java      |    8 +-
 .../generated/PostgreSqlV2SourceTests.java    |   26 +
 .../PostgreSqlV2TableDatasetTests.java        |   61 +
 ...eSqlV2TableDatasetTypePropertiesTests.java |   24 +
 .../generated/PowerQuerySinkMappingTests.java |  132 +-
 .../generated/PrestoObjectDatasetTests.java   |   32 +-
 .../generated/PrestoSourceTests.java          |    9 +-
 ...ointConnectionsListByFactoryMockTests.java |   14 +-
 ...nsCreateOrUpdateWithResponseMockTests.java |   26 +-
 ...OperationsDeleteWithResponseMockTests.java |    4 +-
 ...ionOperationsGetWithResponseMockTests.java |   13 +-
 ...LinkResourcesGetWithResponseMockTests.java |    6 +-
 .../generated/QuickBooksSourceTests.java      |    8 +-
 .../RecurrenceScheduleOccurrenceTests.java    |   14 +-
 .../generated/RecurrenceScheduleTests.java    |   45 +-
 .../RedirectIncompatibleRowSettingsTests.java |    6 +-
 .../RedshiftUnloadSettingsTests.java          |   13 +-
 .../generated/RelationalSourceTests.java      |    9 +-
 .../RerunTumblingWindowTriggerTests.java      |   27 +-
 ...blingWindowTriggerTypePropertiesTests.java |   18 +-
 .../generated/ResponsysSourceTests.java       |    8 +-
 .../generated/RestResourceDatasetTests.java   |   44 +-
 ...estResourceDatasetTypePropertiesTests.java |   11 +-
 .../datafactory/generated/RestSinkTests.java  |   13 +-
 .../generated/RestSourceTests.java            |   14 +-
 .../generated/RetryPolicyTests.java           |    8 +-
 .../SalesforceMarketingCloudSourceTests.java  |    8 +-
 .../SalesforceObjectDatasetTests.java         |   39 +-
 ...forceObjectDatasetTypePropertiesTests.java |    7 +-
 ...esforceServiceCloudObjectDatasetTests.java |   37 +-
 ...CloudObjectDatasetTypePropertiesTests.java |    4 +-
 .../SalesforceServiceCloudSinkTests.java      |   12 +-
 .../SalesforceServiceCloudSourceTests.java    |   10 +-
 .../SalesforceServiceCloudV2SinkTests.java    |   13 +-
 .../SalesforceServiceCloudV2SourceTests.java  |   10 +-
 .../generated/SalesforceSinkTests.java        |   12 +-
 .../generated/SalesforceSourceTests.java      |   10 +-
 .../generated/SalesforceV2SinkTests.java      |   16 +-
 .../generated/SalesforceV2SourceTests.java    |   11 +-
 .../generated/SapBwCubeDatasetTests.java      |   43 +-
 .../generated/SapBwSourceTests.java           |    8 +-
 ...pCloudForCustomerResourceDatasetTests.java |   36 +-
 ...merResourceDatasetTypePropertiesTests.java |    4 +-
 .../SapCloudForCustomerSinkTests.java         |   12 +-
 .../SapCloudForCustomerSourceTests.java       |   10 +-
 .../generated/SapEccResourceDatasetTests.java |   37 +-
 ...EccResourceDatasetTypePropertiesTests.java |    7 +-
 .../generated/SapEccSourceTests.java          |   10 +-
 .../SapHanaPartitionSettingsTests.java        |    4 +-
 .../generated/SapHanaSourceTests.java         |   13 +-
 .../generated/SapHanaTableDatasetTests.java   |   41 +-
 ...apHanaTableDatasetTypePropertiesTests.java |    4 +-
 .../generated/SapOdpResourceDatasetTests.java |   40 +-
 ...OdpResourceDatasetTypePropertiesTests.java |    4 +-
 .../generated/SapOdpSourceTests.java          |   11 +-
 .../generated/SapOpenHubSourceTests.java      |   12 +-
 .../SapOpenHubTableDatasetTests.java          |   41 +-
 ...penHubTableDatasetTypePropertiesTests.java |    6 +-
 .../SapTablePartitionSettingsTests.java       |    8 +-
 .../SapTableResourceDatasetTests.java         |   37 +-
 ...bleResourceDatasetTypePropertiesTests.java |    4 +-
 .../generated/SapTableSourceTests.java        |   21 +-
 .../ScheduleTriggerRecurrenceTests.java       |   77 +-
 .../generated/ScheduleTriggerTests.java       |   94 +-
 .../ScheduleTriggerTypePropertiesTests.java   |   77 +-
 .../generated/ScriptActionTests.java          |   18 +-
 .../ScriptActivityParameterTests.java         |   16 +-
 .../ScriptActivityScriptBlockTests.java       |   28 +-
 .../generated/ScriptActivityTests.java        |  157 +-
 ...ctivityTypePropertiesLogSettingsTests.java |   12 +-
 .../ScriptActivityTypePropertiesTests.java    |   75 +-
 .../SecureInputOutputPolicyTests.java         |    8 +-
 ...cyTumblingWindowTriggerReferenceTests.java |   15 +-
 ...lfHostedIntegrationRuntimeStatusTests.java |   61 +-
 ...ationRuntimeStatusTypePropertiesTests.java |   57 +-
 .../generated/ServiceNowSourceTests.java      |   10 +-
 .../ServiceNowV2ObjectDatasetTests.java       |   62 +
 .../generated/ServiceNowV2SourceTests.java    |   69 +
 .../generated/SetVariableActivityTests.java   |   71 +-
 ...etVariableActivityTypePropertiesTests.java |   12 +-
 .../generated/SftpReadSettingsTests.java      |   15 +-
 .../generated/SftpWriteSettingsTests.java     |   13 +-
 .../SharePointOnlineListSourceTests.java      |   10 +-
 .../generated/ShopifySourceTests.java         |    9 +-
 .../generated/SkipErrorFileTests.java         |    5 +-
 .../SnowflakeExportCopyCommandTests.java      |    9 +-
 .../SnowflakeImportCopyCommandTests.java      |    9 +-
 .../generated/SnowflakeSinkTests.java         |   22 +-
 .../generated/SnowflakeSourceTests.java       |   17 +-
 .../generated/SnowflakeV2SinkTests.java       |   15 +-
 .../generated/SnowflakeV2SourceTests.java     |   13 +-
 ...gurationParametrizationReferenceTests.java |    4 +-
 .../generated/SparkSourceTests.java           |    8 +-
 .../generated/SqlDWSourceTests.java           |   16 +-
 .../generated/SqlMISourceTests.java           |   18 +-
 .../generated/SqlPartitionSettingsTests.java  |    6 +-
 .../generated/SqlServerSourceTests.java       |   18 +-
 ...SqlServerStoredProcedureActivityTests.java |   75 +-
 ...dProcedureActivityTypePropertiesTests.java |    4 +-
 .../generated/SqlServerTableDatasetTests.java |   36 +-
 ...ServerTableDatasetTypePropertiesTests.java |   10 +-
 .../datafactory/generated/SqlSourceTests.java |   16 +-
 .../generated/SquareSourceTests.java          |    8 +-
 .../generated/SsisChildPackageTests.java      |   14 +-
 .../SsisEnvironmentReferenceTests.java        |   23 +-
 .../generated/SsisEnvironmentTests.java       |   69 +-
 .../SsisExecutionParameterTests.java          |    4 +-
 .../generated/SsisFolderTests.java            |   16 +-
 .../generated/SsisPackageTests.java           |  102 +-
 .../generated/SsisParameterTests.java         |   59 +-
 .../generated/SsisProjectTests.java           |  128 +-
 .../generated/SsisPropertyOverrideTests.java  |    8 +-
 .../generated/SsisVariableTests.java          |   32 +-
 .../generated/StagingSettingsTests.java       |   13 +-
 .../generated/StoreReadSettingsTests.java     |    7 +-
 .../generated/StoreWriteSettingsTests.java    |   12 +-
 .../generated/SwitchActivityTests.java        |  332 +-
 .../SwitchActivityTypePropertiesTests.java    |  408 +-
 .../generated/SwitchCaseTests.java            |  128 +-
 .../generated/SybaseSourceTests.java          |    9 +-
 .../generated/SybaseTableDatasetTests.java    |   36 +-
 ...SybaseTableDatasetTypePropertiesTests.java |    6 +-
 .../SynapseNotebookActivityTests.java         |  119 +-
 ...seNotebookActivityTypePropertiesTests.java |   31 +-
 .../SynapseNotebookReferenceTests.java        |    4 +-
 .../SynapseSparkJobReferenceTests.java        |    4 +-
 .../generated/TabularSourceTests.java         |    8 +-
 .../generated/TabularTranslatorTests.java     |   14 +-
 .../generated/TarGZipReadSettingsTests.java   |    5 +-
 .../generated/TarReadSettingsTests.java       |    4 +-
 .../TeradataPartitionSettingsTests.java       |    6 +-
 .../generated/TeradataSourceTests.java        |   14 +-
 .../TriggerDependencyReferenceTests.java      |    8 +-
 .../generated/TriggerReferenceTests.java      |   10 +-
 ...riggerRunsCancelWithResponseMockTests.java |    2 +-
 ...TriggerRunsRerunWithResponseMockTests.java |    2 +-
 ...rsCreateOrUpdateWithResponseMockTests.java |   13 +-
 .../TriggersDeleteWithResponseMockTests.java  |    2 +-
 ...bscriptionStatusWithResponseMockTests.java |    4 +-
 .../TriggersGetWithResponseMockTests.java     |    9 +-
 .../TriggersListByFactoryMockTests.java       |    8 +-
 .../generated/TriggersStartMockTests.java     |    2 +-
 .../generated/TriggersStopMockTests.java      |    2 +-
 .../TriggersSubscribeToEventsMockTests.java   |    6 +-
 ...riggersUnsubscribeFromEventsMockTests.java |    6 +-
 ...WindowTriggerDependencyReferenceTests.java |   20 +-
 .../generated/TumblingWindowTriggerTests.java |   56 +-
 ...blingWindowTriggerTypePropertiesTests.java |   37 +-
 .../TypeConversionSettingsTests.java          |    9 +-
 .../generated/UntilActivityTests.java         |  170 +-
 .../UntilActivityTypePropertiesTests.java     |  117 +-
 .../generated/ValidationActivityTests.java    |   65 +-
 ...ValidationActivityTypePropertiesTests.java |   16 +-
 .../generated/VerticaSourceTests.java         |   10 +-
 .../generated/WaitActivityTests.java          |   44 +-
 .../WaitActivityTypePropertiesTests.java      |    4 +-
 .../generated/WarehouseSinkTests.java         |   32 +-
 .../generated/WarehouseSourceTests.java       |   18 +-
 .../WebAnonymousAuthenticationTests.java      |    4 +-
 .../generated/WebLinkedServiceTests.java      |   37 +-
 .../WebLinkedServiceTypePropertiesTests.java  |    8 +-
 .../datafactory/generated/WebSourceTests.java |    8 +-
 .../generated/WebTableDatasetTests.java       |   39 +-
 .../WebTableDatasetTypePropertiesTests.java   |    4 +-
 .../generated/XeroSourceTests.java            |    8 +-
 .../generated/XmlReadSettingsTests.java       |    6 +-
 .../datafactory/generated/XmlSourceTests.java |   17 +-
 .../ZipDeflateReadSettingsTests.java          |    5 +-
 .../generated/ZohoSourceTests.java            |    9 +-
 sdk/datafactory/tests.mgmt.yml                |   18 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/dataprotection/tests.mgmt.yml             |   24 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../azure-developer-devcenter/pom.xml         |    2 +-
 .../azure-resourcemanager-devcenter/pom.xml   |    6 +-
 .../azure-resourcemanager-devhub/pom.xml      |    6 +-
 .../pom.xml                                   |    4 +-
 sdk/deviceprovisioningservices/tests.mgmt.yml |   18 +-
 .../azure-iot-deviceupdate/CHANGELOG.md       |   11 +
 .../azure-iot-deviceupdate/pom.xml            |    2 +-
 .../pom.xml                                   |    8 +-
 sdk/deviceupdate/tests.mgmt.yml               |   24 +-
 .../azure-resourcemanager-devspaces/pom.xml   |    6 +-
 .../azure-resourcemanager-devtestlabs/pom.xml |    8 +-
 sdk/devtestlabs/tests.mgmt.yml                |   24 +-
 .../azure-digitaltwins-core/CHANGELOG.md      |   12 +
 .../azure-digitaltwins-core/pom.xml           |    2 +-
 .../pom.xml                                   |    8 +-
 sdk/digitaltwins/tests.yml                    |   20 +-
 .../azure-resourcemanager-dnsresolver/pom.xml |    6 +-
 .../CHANGELOG.md                              |   10 +
 .../azure-ai-documentintelligence/pom.xml     |    4 +-
 sdk/documentintelligence/tests.native.yml     |   28 +-
 sdk/documentintelligence/tests.yml            |   28 +-
 .../azure-resourcemanager-dynatrace/pom.xml   |    6 +-
 sdk/e2e/pom.xml                               |    2 +-
 .../azure-analytics-defender-easm/pom.xml     |    6 +-
 .../azure-resourcemanager-education/pom.xml   |    6 +-
 .../azure-resourcemanager-elastic/pom.xml     |    6 +-
 .../azure-resourcemanager-elasticsan/pom.xml  |    8 +-
 sdk/elasticsan/tests.mgmt.yml                 |   24 +-
 .../pom.xml                                   |    2 +-
 .../azure-messaging-eventgrid/CHANGELOG.md    |   10 +
 .../azure-messaging-eventgrid/pom.xml         |    8 +-
 .../azure-resourcemanager-eventgrid/ci.yml    |   44 +
 .../azure-resourcemanager-eventgrid/pom.xml   |   12 +-
 sdk/eventgrid/ci.yml                          |   15 +-
 sdk/eventgrid/tests.yml                       |   26 +-
 .../CHANGELOG.md                              |   11 +
 .../pom.xml                                   |   10 +-
 .../pom.xml                                   |    8 +-
 .../azure-messaging-eventhubs-stress/pom.xml  |   10 +-
 .../pom.xml                                   |    2 +-
 .../azure-messaging-eventhubs/CHANGELOG.md    |   11 +
 .../azure-messaging-eventhubs/docs/pom.xml    |    4 +-
 .../azure-messaging-eventhubs/pom.xml         |   14 +-
 sdk/eventhubs/ci.yml                          |    2 -
 .../microsoft-azure-eventhubs-eph/pom.xml     |    4 +-
 .../pom.xml                                   |    4 +-
 .../microsoft-azure-eventhubs/pom.xml         |    6 +-
 sdk/eventhubs/tests.yml                       |   46 +-
 sdk/eventhubs/version-overrides-matrix.json   |    2 +-
 .../azure-resourcemanager-fluidrelay/pom.xml  |    4 +-
 sdk/fluidrelay/tests.mgmt.yml                 |   24 +-
 .../azure-ai-formrecognizer-perf/pom.xml      |    2 +-
 .../azure-ai-formrecognizer/CHANGELOG.md      |   11 +
 .../azure-ai-formrecognizer/pom.xml           |    2 +-
 sdk/formrecognizer/ci.yml                     |    4 +-
 sdk/formrecognizer/tests.native.yml           |   28 +-
 sdk/formrecognizer/tests.yml                  |   46 +-
 .../azure-resourcemanager-frontdoor/pom.xml   |    8 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-hanaonazure/pom.xml |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-hdinsight/pom.xml   |   12 +-
 sdk/hdinsight/tests.mgmt.yml                  |   24 +-
 .../pom.xml                                   |    8 +-
 sdk/healthcareapis/tests.mgmt.yml             |   24 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 sdk/healthinsights/tests.yml                  |   30 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../identity/broker/JavaDocCodeSnippets.java  |    8 +
 .../azure-identity-broker/CHANGELOG.md        |   11 +
 sdk/identity/azure-identity-broker/README.md  |   10 +
 sdk/identity/azure-identity-broker/pom.xml    |    4 +-
 ...ractiveBrowserBrokerCredentialBuilder.java |   12 +-
 ...iveBrowserBrokerCredentialBuilderTest.java |    4 +-
 .../azure-identity-extensions/CHANGELOG.md    |   19 +
 sdk/identity/azure-identity-extensions/ci.yml |   43 +
 .../azure-identity-extensions/pom.xml         |    6 +-
 sdk/identity/azure-identity/CHANGELOG.md      |    7 +
 sdk/identity/azure-identity/pom.xml           |   16 +-
 .../implementation/IdentityClient.java        |   34 +-
 .../implementation/IdentityClientOptions.java |   12 +-
 .../implementation/IdentitySyncClient.java    |   22 +-
 sdk/identity/ci.yml                           |   16 +-
 .../pom.xml                                   |   10 +-
 sdk/imagebuilder/tests.mgmt.yml               |   24 +-
 .../azure-resourcemanager-iotcentral/pom.xml  |    4 +-
 sdk/iotcentral/tests.mgmt.yml                 |   24 +-
 .../CHANGELOG.md                              |  240 +-
 .../README.md                                 |    8 +-
 .../SAMPLE.md                                 | 1107 ++-
 .../pom.xml                                   |   12 +-
 .../IoTFirmwareDefenseManager.java            |  234 +-
 .../fluent/BinaryHardeningsClient.java        |   47 +
 .../fluent/CryptoCertificatesClient.java      |   47 +
 .../fluent/CryptoKeysClient.java              |   47 +
 .../iotfirmwaredefense/fluent/CvesClient.java |   46 +
 .../fluent/FirmwaresClient.java               |  477 +-
 .../iotfirmwaredefense/fluent/Fist.java       |   67 -
 .../fluent/IoTFirmwareDefense.java            |  118 +
 .../fluent/OperationsClient.java              |   16 +-
 .../fluent/PasswordHashesClient.java          |   47 +
 .../fluent/SbomComponentsClient.java          |   47 +
 .../fluent/SummariesClient.java               |   82 +
 .../fluent/WorkspacesClient.java              |   52 +-
 .../models/BinaryHardeningFeatures.java       |   89 +-
 .../models/BinaryHardeningResourceInner.java  |  316 +
 ...gInner.java => BinaryHardeningResult.java} |  161 +-
 .../fluent/models/ComponentInner.java         |  209 -
 ...icateInner.java => CryptoCertificate.java} |  204 +-
 .../CryptoCertificateResourceInner.java       |  515 ++
 .../{CryptoKeyInner.java => CryptoKey.java}   |   87 +-
 .../fluent/models/CryptoKeyResourceInner.java |  235 +
 .../fluent/models/CveResourceInner.java       |  284 +
 .../models/{CveInner.java => CveResult.java}  |  203 +-
 .../fluent/models/CveSummaryInner.java        |  180 -
 .../fluent/models/FirmwareInner.java          |   53 +-
 .../fluent/models/FirmwareProperties.java     |   54 +-
 .../fluent/models/FirmwareSummaryInner.java   |  206 -
 .../fluent/models/OperationInner.java         |   22 +-
 ...sswordHashInner.java => PasswordHash.java} |   70 +-
 .../models/PasswordHashResourceInner.java     |  224 +
 .../fluent/models/SbomComponent.java          |  159 +
 .../models/SbomComponentResourceInner.java    |  179 +
 .../fluent/models/SummaryResourceInner.java   |   64 +
 .../fluent/models/UrlTokenInner.java          |   25 +-
 .../fluent/models/WorkspaceInner.java         |   24 +-
 .../fluent/models/WorkspaceProperties.java    |   12 +-
 .../fluent/models/package-info.java           |    4 +-
 .../fluent/package-info.java                  |    4 +-
 ....java => BinaryHardeningResourceImpl.java} |   49 +-
 .../BinaryHardeningSummaryImpl.java           |   53 -
 .../BinaryHardeningsClientImpl.java           |  300 +
 .../implementation/BinaryHardeningsImpl.java  |   49 +
 ...ava => CryptoCertificateResourceImpl.java} |   46 +-
 .../CryptoCertificateSummaryImpl.java         |   57 -
 .../CryptoCertificatesClientImpl.java         |  296 +
 .../CryptoCertificatesImpl.java               |   49 +
 ...eyImpl.java => CryptoKeyResourceImpl.java} |   33 +-
 .../implementation/CryptoKeySummaryImpl.java  |   49 -
 .../implementation/CryptoKeysClientImpl.java  |  296 +
 .../implementation/CryptoKeysImpl.java        |   49 +
 .../{CveImpl.java => CveResourceImpl.java}    |   47 +-
 .../implementation/CveSummaryImpl.java        |   53 -
 .../implementation/CvesClientImpl.java        |  293 +
 .../implementation/CvesImpl.java              |   49 +
 .../implementation/FirmwareImpl.java          |  219 +-
 .../implementation/FirmwareSummaryImpl.java   |   57 -
 .../implementation/FirmwaresClientImpl.java   | 3860 +---------
 .../implementation/FirmwaresImpl.java         |  454 +-
 ...er.java => IoTFirmwareDefenseBuilder.java} |   82 +-
 ...tImpl.java => IoTFirmwareDefenseImpl.java} |  253 +-
 .../implementation/OperationImpl.java         |    3 +-
 .../implementation/OperationsClientImpl.java  |  176 +-
 .../implementation/OperationsImpl.java        |    7 +-
 ...mpl.java => PasswordHashResourceImpl.java} |   30 +-
 .../PasswordHashesClientImpl.java             |  296 +
 .../implementation/PasswordHashesImpl.java    |   49 +
 .../implementation/ResourceManagerUtils.java  |  197 +
 ...pl.java => SbomComponentResourceImpl.java} |   44 +-
 .../SbomComponentsClientImpl.java             |  296 +
 .../implementation/SbomComponentsImpl.java    |   49 +
 .../implementation/SummariesClientImpl.java   |  452 ++
 .../implementation/SummariesImpl.java         |   75 +
 .../implementation/SummaryResourceImpl.java   |   50 +
 .../implementation/UrlTokenImpl.java          |    7 +-
 .../implementation/Utils.java                 |  204 -
 .../implementation/WorkspaceImpl.java         |   64 +-
 .../implementation/WorkspacesClientImpl.java  |  813 +--
 .../implementation/WorkspacesImpl.java        |  119 +-
 .../implementation/package-info.java          |    4 +-
 .../iotfirmwaredefense/models/ActionType.java |   14 +-
 .../models/BinaryHardening.java               |   94 -
 ...st.java => BinaryHardeningListResult.java} |   36 +-
 .../models/BinaryHardeningResource.java       |  125 +
 .../models/BinaryHardeningSummary.java        |   59 -
 .../BinaryHardeningSummaryResource.java}      |   72 +-
 .../models/BinaryHardenings.java              |   42 +
 .../iotfirmwaredefense/models/CanaryFlag.java |   47 -
 .../iotfirmwaredefense/models/Component.java  |   68 -
 .../models/CryptoCertificateEntity.java       |   30 +-
 .../models/CryptoCertificateListResult.java   |   74 +
 ...te.java => CryptoCertificateResource.java} |  103 +-
 .../models/CryptoCertificateSummary.java      |   67 -
 .../CryptoCertificateSummaryResource.java}    |   80 +-
 .../models/CryptoCertificates.java            |   42 +
 ...oKeyList.java => CryptoKeyListResult.java} |   40 +-
 ...{CryptoKey.java => CryptoKeyResource.java} |   65 +-
 .../models/CryptoKeySummary.java              |   52 -
 .../CryptoKeySummaryResource.java}            |   64 +-
 .../iotfirmwaredefense/models/CryptoKeys.java |   41 +
 .../iotfirmwaredefense/models/Cve.java        |  104 -
 .../models/CveComponent.java                  |  106 +
 .../iotfirmwaredefense/models/CveLink.java    |   30 +-
 .../{CveList.java => CveListResult.java}      |   36 +-
 .../models/CveResource.java                   |  120 +
 .../iotfirmwaredefense/models/CveSummary.java |  157 +-
 .../iotfirmwaredefense/models/Cves.java       |   41 +
 .../iotfirmwaredefense/models/Firmware.java   |  473 +-
 .../models/FirmwareList.java                  |   16 +-
 .../models/FirmwareSummary.java               |  206 +-
 .../models/FirmwareUpdateDefinition.java      |   50 +-
 .../iotfirmwaredefense/models/Firmwares.java  |  425 +-
 .../models/GenerateUploadUrlRequest.java      |   14 +-
 .../iotfirmwaredefense/models/IsExpired.java  |   47 -
 .../models/IsSelfSigned.java                  |   47 -
 .../models/IsShortKeySize.java                |   47 -
 .../models/IsUpdateAvailable.java             |   47 -
 .../models/IsWeakSignature.java               |   47 -
 .../iotfirmwaredefense/models/NxFlag.java     |   47 -
 .../iotfirmwaredefense/models/Operation.java  |   16 +-
 .../models/OperationDisplay.java              |   18 +-
 .../models/OperationListResult.java           |   14 +-
 .../iotfirmwaredefense/models/Operations.java |   16 +-
 .../iotfirmwaredefense/models/Origin.java     |   18 +-
 .../iotfirmwaredefense/models/PairedKey.java  |   42 +-
 .../models/PasswordHashList.java              |   70 -
 .../models/PasswordHashListResult.java        |   74 +
 ...ordHash.java => PasswordHashResource.java} |   57 +-
 .../models/PasswordHashes.java                |   42 +
 .../iotfirmwaredefense/models/PieFlag.java    |   47 -
 .../models/ProvisioningState.java             |   26 +-
 .../iotfirmwaredefense/models/RelroFlag.java  |   47 -
 ...List.java => SbomComponentListResult.java} |   40 +-
 .../models/SbomComponentResource.java         |   84 +
 .../models/SbomComponents.java                |   42 +
 .../iotfirmwaredefense/models/Status.java     |   30 +-
 .../models/StatusMessage.java                 |   80 +
 .../models/StrippedFlag.java                  |   47 -
 .../iotfirmwaredefense/models/Summaries.java  |   72 +
 ...ficateList.java => SummaryListResult.java} |   40 +-
 .../models/SummaryName.java                   |   68 +
 .../models/SummaryResource.java               |   55 +
 .../models/SummaryResourceProperties.java     |   42 +
 .../models/SummaryType.java                   |   68 +
 .../iotfirmwaredefense/models/UrlToken.java   |   15 +-
 .../iotfirmwaredefense/models/Workspace.java  |   91 +-
 .../models/WorkspaceList.java                 |   16 +-
 .../models/WorkspaceUpdateDefinition.java     |   14 +-
 .../iotfirmwaredefense/models/Workspaces.java |   42 +-
 .../models/package-info.java                  |    4 +-
 .../iotfirmwaredefense/package-info.java      |    4 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       |  256 +
 .../BinaryHardeningListByFirmwareSamples.java |   40 +
 ...yptoCertificatesListByFirmwareSamples.java |   40 +
 .../CryptoKeysListByFirmwareSamples.java      |   40 +
 .../generated/CvesListByFirmwareSamples.java  |   40 +
 .../generated/FirmwareCreateSamples.java      |   58 -
 .../generated/FirmwareDeleteSamples.java      |   38 -
 ...GenerateBinaryHardeningDetailsSamples.java |   40 -
 ...GenerateBinaryHardeningSummarySamples.java |   40 -
 ...rmwareGenerateComponentDetailsSamples.java |   40 -
 ...nerateCryptoCertificateSummarySamples.java |   43 -
 ...rmwareGenerateCryptoKeySummarySamples.java |   43 -
 .../FirmwareGenerateCveSummarySamples.java    |   40 -
 .../FirmwareGenerateDownloadUrlSamples.java   |   40 -
 ...eGenerateFilesystemDownloadUrlSamples.java |   40 -
 .../FirmwareGenerateSummarySamples.java       |   38 -
 .../generated/FirmwareGetSamples.java         |   38 -
 ...istGenerateBinaryHardeningListSamples.java |   40 -
 ...mwareListGenerateComponentListSamples.java |   38 -
 ...tGenerateCryptoCertificateListSamples.java |   43 -
 ...mwareListGenerateCryptoKeyListSamples.java |   43 -
 .../FirmwareListGenerateCveListSamples.java   |   38 -
 ...reListGeneratePasswordHashListSamples.java |   38 -
 .../generated/FirmwareUpdateSamples.java      |   67 -
 .../generated/FirmwaresCreateSamples.java     |   45 +
 .../generated/FirmwaresDeleteSamples.java     |   40 +
 .../FirmwaresGenerateDownloadUrlSamples.java  |   40 +
 ...sGenerateFilesystemDownloadUrlSamples.java |   40 +
 .../generated/FirmwaresGetSamples.java        |   40 +
 ...a => FirmwaresListByWorkspaceSamples.java} |   24 +-
 .../generated/FirmwaresUpdateSamples.java     |   49 +
 .../generated/OperationsListSamples.java      |   22 +-
 .../PasswordHashesListByFirmwareSamples.java  |   40 +
 .../SbomComponentsListByFirmwareSamples.java  |   40 +
 .../generated/SummariesGetSamples.java        |   42 +
 .../SummariesListByFirmwareSamples.java       |   40 +
 .../generated/WorkspacesCreateSamples.java    |   40 +-
 .../generated/WorkspacesDeleteSamples.java    |   32 +-
 .../WorkspacesGenerateUploadUrlSamples.java   |   29 +-
 .../WorkspacesGetByResourceGroupSamples.java  |   22 +-
 .../WorkspacesListByResourceGroupSamples.java |   14 +-
 .../generated/WorkspacesListSamples.java      |   14 +-
 .../generated/WorkspacesUpdateSamples.java    |   36 +-
 .../BinaryHardeningFeaturesTests.java         |   40 +-
 .../generated/BinaryHardeningInnerTests.java  |   65 -
 .../BinaryHardeningListResultTests.java       |   26 +
 .../generated/BinaryHardeningListTests.java   |   28 -
 .../BinaryHardeningResourceInnerTests.java    |   48 +
 .../generated/BinaryHardeningResultTests.java |   49 +
 .../BinaryHardeningSummaryInnerTests.java     |   45 -
 .../BinaryHardeningSummaryResourceTests.java  |   37 +
 ...naryHardeningsListByFirmwareMockTests.java |   67 +
 .../generated/ComponentInnerTests.java        |   51 -
 .../generated/ComponentListTests.java         |   28 -
 .../CryptoCertificateEntityTests.java         |   38 +-
 .../generated/CveComponentTests.java          |   31 +
 .../generated/CveInnerTests.java              |   59 -
 .../generated/CveLinkTests.java               |   13 +-
 .../generated/CveListResultTests.java         |   26 +
 .../generated/CveListTests.java               |   28 -
 .../generated/CveResourceInnerTests.java      |   51 +
 .../generated/CveResultTests.java             |   50 +
 .../generated/CveSummaryInnerTests.java       |   45 -
 .../generated/CveSummaryTests.java            |   35 +
 .../CvesListByFirmwareMockTests.java          |   67 +
 .../generated/FirmwareInnerTests.java         |   51 -
 .../generated/FirmwareListTests.java          |   28 -
 .../generated/FirmwarePropertiesTests.java    |   51 -
 .../generated/FirmwareSummaryInnerTests.java  |   48 -
 .../generated/FirmwareSummaryTests.java       |   41 +
 .../FirmwareUpdateDefinitionTests.java        |   51 -
 .../FirmwaresCreateWithResponseMockTests.java |   87 -
 .../FirmwaresDeleteWithResponseMockTests.java |   38 +-
 ...HardeningDetailsWithResponseMockTests.java |   85 -
 ...HardeningSummaryWithResponseMockTests.java |   76 -
 ...ComponentDetailsWithResponseMockTests.java |   77 -
 ...nerateCveSummaryWithResponseMockTests.java |   76 -
 ...erateDownloadUrlWithResponseMockTests.java |   39 +-
 ...ystemDownloadUrlWithResponseMockTests.java |   39 +-
 ...sGenerateSummaryWithResponseMockTests.java |   76 -
 .../FirmwaresGetWithResponseMockTests.java    |   77 -
 .../FirmwaresListByWorkspaceMockTests.java    |   75 -
 ...tGenerateBinaryHardeningListMockTests.java |   85 -
 ...resListGenerateComponentListMockTests.java |   78 -
 ...FirmwaresListGenerateCveListMockTests.java |   79 -
 .../GenerateUploadUrlRequestTests.java        |   10 +-
 .../generated/OperationDisplayTests.java      |    8 +-
 .../generated/OperationInnerTests.java        |    8 +-
 .../generated/OperationListResultTests.java   |    8 +-
 .../generated/OperationsListMockTests.java    |   34 +-
 .../generated/PairedKeyTests.java             |   15 +-
 .../SbomComponentListResultTests.java         |   26 +
 .../SbomComponentResourceInnerTests.java      |   37 +
 .../generated/SbomComponentTests.java         |   36 +
 ...SbomComponentsListByFirmwareMockTests.java |   61 +
 .../SummariesGetWithResponseMockTests.java    |   55 +
 .../SummariesListByFirmwareMockTests.java     |   55 +
 .../generated/SummaryListResultTests.java     |   26 +
 .../generated/SummaryResourceInnerTests.java  |   23 +
 .../SummaryResourcePropertiesTests.java       |   22 +
 .../generated/UrlTokenInnerTests.java         |    3 +-
 .../generated/WorkspaceInnerTests.java        |   21 +-
 .../generated/WorkspaceListTests.java         |   14 +-
 .../generated/WorkspacePropertiesTests.java   |    4 +-
 .../WorkspaceUpdateDefinitionTests.java       |    6 +-
 ...WorkspacesCreateWithResponseMockTests.java |   49 +-
 ...eByResourceGroupWithResponseMockTests.java |   37 +-
 ...enerateUploadUrlWithResponseMockTests.java |   45 +-
 ...tByResourceGroupWithResponseMockTests.java |   44 +-
 ...orkspacesListByResourceGroupMockTests.java |   41 +-
 .../generated/WorkspacesListMockTests.java    |   37 +-
 .../azure-resourcemanager-iothub/pom.xml      |    8 +-
 sdk/iothub/tests.mgmt.yml                     |   24 +-
 .../CHANGELOG.md                              |   11 +
 .../pom.xml                                   |    8 +-
 .../CHANGELOG.md                              |   11 +
 .../pom.xml                                   |    4 +-
 .../azure-security-keyvault-jca/pom.xml       |    4 +-
 .../azure-security-keyvault-keys/CHANGELOG.md |   11 +
 .../azure-security-keyvault-keys/pom.xml      |    8 +-
 .../azure-security-keyvault-perf/pom.xml      |    4 +-
 .../CHANGELOG.md                              |   11 +
 .../azure-security-keyvault-secrets/pom.xml   |    4 +-
 .../azure-security-test-keyvault-jca/pom.xml  |    2 +-
 sdk/keyvault/ci.yml                           |    2 +
 .../microsoft-azure-keyvault-core/pom.xml     |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../microsoft-azure-keyvault-webkey/pom.xml   |    2 +-
 sdk/keyvault/platform-matrix.json             |    8 +-
 sdk/keyvault/tests.yml                        |  108 +-
 .../pom.xml                                   |    6 +-
 sdk/kusto/azure-resourcemanager-kusto/pom.xml |    8 +-
 sdk/kusto/tests.mgmt.yml                      |   24 +-
 .../pom.xml                                   |    6 +-
 .../azure-developer-loadtesting/CHANGELOG.md  |   11 +
 .../azure-developer-loadtesting/pom.xml       |    2 +-
 .../azure-resourcemanager-loadtesting/pom.xml |    8 +-
 sdk/loadtesting/tests.mgmt.yml                |   18 +-
 .../pom.xml                                   |    8 +-
 sdk/loganalytics/tests.mgmt.yml               |   24 +-
 sdk/logic/azure-resourcemanager-logic/pom.xml |    6 +-
 sdk/logz/azure-resourcemanager-logz/pom.xml   |    6 +-
 .../pom.xml                                   |   12 +-
 sdk/machinelearning/tests.mgmt.yml            |   24 +-
 .../azure-resourcemanager-maintenance/pom.xml |    8 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/managednetworkfabric/tests.mgmt.yml       |   24 +-
 .../pom.xml                                   |    6 +-
 sdk/maps/azure-maps-elevation/pom.xml         |    4 +-
 sdk/maps/azure-maps-elevation/tests.yml       |   10 +-
 sdk/maps/azure-maps-geolocation/pom.xml       |    4 +-
 sdk/maps/azure-maps-geolocation/tests.yml     |   10 +-
 sdk/maps/azure-maps-render/pom.xml            |    4 +-
 sdk/maps/azure-maps-render/tests.yml          |   10 +-
 sdk/maps/azure-maps-route/pom.xml             |    4 +-
 sdk/maps/azure-maps-route/tests.yml           |   10 +-
 sdk/maps/azure-maps-search/pom.xml            |    4 +-
 sdk/maps/azure-maps-search/tests.yml          |   10 +-
 sdk/maps/azure-maps-timezone/pom.xml          |    4 +-
 sdk/maps/azure-maps-timezone/tests.yml        |   10 +-
 sdk/maps/azure-maps-traffic/pom.xml           |    4 +-
 sdk/maps/azure-maps-traffic/tests.yml         |   10 +-
 sdk/maps/azure-maps-weather/pom.xml           |    4 +-
 sdk/maps/azure-maps-weather/tests.yml         |   10 +-
 sdk/maps/azure-resourcemanager-maps/pom.xml   |    8 +-
 sdk/maps/maps-tests-template.yml              |   42 +-
 .../azure-resourcemanager-mariadb/pom.xml     |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/mediaservices/tests.mgmt.yml              |   18 +-
 .../azure-ai-metricsadvisor/CHANGELOG.md      |   11 +
 .../azure-ai-metricsadvisor/pom.xml           |    2 +-
 sdk/metricsadvisor/tests.yml                  |   66 +-
 .../CHANGELOG.md                              |   11 +
 .../azure-mixedreality-authentication/pom.xml |    4 +-
 .../pom.xml                                   |    6 +-
 sdk/mixedreality/tests.yml                    |   18 +-
 .../CHANGELOG.md                              |  101 +-
 .../README.md                                 |    4 +-
 .../SAMPLE.md                                 | 1684 ++---
 .../pom.xml                                   |   12 +-
 .../mobilenetwork/MobileNetworkManager.java   |  150 +-
 .../fluent/AttachedDataNetworksClient.java    |  140 +-
 .../fluent/DataNetworksClient.java            |   84 +-
 .../fluent/DiagnosticsPackagesClient.java     |   68 +-
 .../fluent/ExtendedUeInformationsClient.java  |   46 +
 .../fluent/MobileNetworkManagementClient.java |   58 +-
 .../fluent/MobileNetworksClient.java          |   72 +-
 .../fluent/OperationsClient.java              |    8 +-
 .../fluent/PacketCapturesClient.java          |  102 +-
 .../PacketCoreControlPlaneVersionsClient.java |   36 +-
 .../fluent/PacketCoreControlPlanesClient.java |  159 +-
 .../fluent/PacketCoreDataPlanesClient.java    |  101 +-
 .../mobilenetwork/fluent/ServicesClient.java  |  104 +-
 .../mobilenetwork/fluent/SimGroupsClient.java |   56 +-
 .../fluent/SimPoliciesClient.java             |   84 +-
 .../mobilenetwork/fluent/SimsClient.java      |  104 +-
 .../mobilenetwork/fluent/SitesClient.java     |   88 +-
 .../mobilenetwork/fluent/SlicesClient.java    |   64 +-
 .../fluent/UeInformationsClient.java          |   43 +
 .../models/AsyncOperationStatusInner.java     |   52 +-
 .../models/AttachedDataNetworkInner.java      |   96 +-
 .../AttachedDataNetworkPropertiesFormat.java  |   95 +-
 .../fluent/models/DataNetworkInner.java       |   28 +-
 .../models/DataNetworkPropertiesFormat.java   |   16 +-
 .../models/DiagnosticsPackageInner.java       |   26 +-
 .../DiagnosticsPackagePropertiesFormat.java   |   16 +-
 .../models/EncryptedSimPropertiesFormat.java  |   38 +-
 .../fluent/models/ExtendedUeInfoInner.java    |   81 +
 .../fluent/models/MobileNetworkInner.java     |  107 +-
 .../models/MobileNetworkPropertiesFormat.java |   81 +-
 .../fluent/models/OperationInner.java         |   18 +-
 .../fluent/models/PacketCaptureInner.java     |   58 +-
 .../models/PacketCapturePropertiesFormat.java |   44 +-
 .../models/PacketCoreControlPlaneInner.java   |  125 +-
 ...acketCoreControlPlanePropertiesFormat.java |  148 +-
 .../PacketCoreControlPlaneVersionInner.java   |   30 +-
 ...reControlPlaneVersionPropertiesFormat.java |   16 +-
 .../models/PacketCoreDataPlaneInner.java      |   58 +-
 .../PacketCoreDataPlanePropertiesFormat.java  |   51 +-
 .../fluent/models/ServiceInner.java           |   41 +-
 .../models/ServicePropertiesFormat.java       |   30 +-
 .../fluent/models/SimGroupInner.java          |   41 +-
 .../models/SimGroupPropertiesFormat.java      |   20 +-
 .../mobilenetwork/fluent/models/SimInner.java |   72 +-
 .../fluent/models/SimPolicyInner.java         |   59 +-
 .../models/SimPolicyPropertiesFormat.java     |   60 +-
 .../fluent/models/SimPropertiesFormat.java    |   38 +-
 .../fluent/models/SiteInner.java              |   26 +-
 .../fluent/models/SitePropertiesFormat.java   |   14 +-
 .../fluent/models/SliceInner.java             |   37 +-
 .../fluent/models/SlicePropertiesFormat.java  |   25 +-
 .../fluent/models/UeInfoInner.java            |   81 +
 .../fluent/models/package-info.java           |    6 +-
 .../mobilenetwork/fluent/package-info.java    |    6 +-
 .../AsyncOperationStatusImpl.java             |    3 +-
 .../AttachedDataNetworkImpl.java              |  108 +-
 .../AttachedDataNetworksClientImpl.java       | 1177 +--
 .../AttachedDataNetworksImpl.java             |  298 +-
 .../implementation/DataNetworkImpl.java       |   59 +-
 .../DataNetworksClientImpl.java               |  729 +-
 .../implementation/DataNetworksImpl.java      |  137 +-
 .../DiagnosticsPackageImpl.java               |    3 +-
 .../DiagnosticsPackagesClientImpl.java        |  686 +-
 .../DiagnosticsPackagesImpl.java              |   68 +-
 .../implementation/ExtendedUeInfoImpl.java    |   50 +
 .../ExtendedUeInformationsClientImpl.java     |  205 +
 .../ExtendedUeInformationsImpl.java           |   57 +
 .../implementation/MobileNetworkImpl.java     |   91 +-
 .../MobileNetworkManagementClientBuilder.java |   46 +-
 .../MobileNetworkManagementClientImpl.java    |  213 +-
 .../MobileNetworksClientImpl.java             |  832 +--
 .../implementation/MobileNetworksImpl.java    |   97 +-
 .../implementation/OperationImpl.java         |    4 +-
 .../implementation/OperationsClientImpl.java  |  143 +-
 .../implementation/OperationsImpl.java        |    8 +-
 .../implementation/PacketCaptureImpl.java     |   68 +-
 .../PacketCapturesClientImpl.java             |  886 +--
 .../implementation/PacketCapturesImpl.java    |  192 +-
 .../PacketCoreControlPlaneImpl.java           |   97 +-
 .../PacketCoreControlPlaneVersionImpl.java    |    3 +-
 ...ketCoreControlPlaneVersionsClientImpl.java |  509 +-
 .../PacketCoreControlPlaneVersionsImpl.java   |   29 +-
 .../PacketCoreControlPlanesClientImpl.java    | 1501 ++--
 .../PacketCoreControlPlanesImpl.java          |  148 +-
 .../PacketCoreDataPlaneImpl.java              |   81 +-
 .../PacketCoreDataPlanesClientImpl.java       |  918 +--
 .../PacketCoreDataPlanesImpl.java             |  174 +-
 .../{Utils.java => ResourceManagerUtils.java} |   25 +-
 .../implementation/ServiceImpl.java           |   55 +-
 .../implementation/ServicesClientImpl.java    |  802 +--
 .../implementation/ServicesImpl.java          |  137 +-
 .../implementation/SimGroupImpl.java          |   54 +-
 .../implementation/SimGroupsClientImpl.java   |  818 +--
 .../implementation/SimGroupsImpl.java         |   93 +-
 .../mobilenetwork/implementation/SimImpl.java |   50 +-
 .../implementation/SimPoliciesClientImpl.java |  746 +-
 .../implementation/SimPoliciesImpl.java       |  137 +-
 .../implementation/SimPolicyImpl.java         |   59 +-
 .../implementation/SimsClientImpl.java        | 1079 +--
 .../implementation/SimsImpl.java              |  153 +-
 .../implementation/SiteImpl.java              |   54 +-
 .../implementation/SitesClientImpl.java       |  893 +--
 .../implementation/SitesImpl.java             |  145 +-
 .../implementation/SliceImpl.java             |   54 +-
 .../implementation/SlicesClientImpl.java      |  701 +-
 .../implementation/SlicesImpl.java            |  137 +-
 .../implementation/UeInfoImpl.java            |   49 +
 .../UeInformationsClientImpl.java             |  284 +
 .../implementation/UeInformationsImpl.java    |   46 +
 .../implementation/package-info.java          |    6 +-
 .../mobilenetwork/models/Ambr.java            |   18 +-
 .../mobilenetwork/models/AmfId.java           |  106 +
 .../models/AsyncOperationId.java              |   19 +-
 .../models/AsyncOperationStatus.java          |   24 +-
 .../models/AttachedDataNetwork.java           |  210 +-
 .../models/AttachedDataNetworkListResult.java |   16 +-
 .../models/AttachedDataNetworkResourceId.java |   20 +-
 .../models/AttachedDataNetworks.java          |   68 +-
 .../models/AuthenticationType.java            |   18 +-
 .../AzureStackEdgeDeviceResourceId.java       |   20 +-
 .../AzureStackHciClusterResourceId.java       |   20 +-
 .../mobilenetwork/models/BillingSku.java      |   26 +-
 .../models/CertificateProvisioning.java       |   14 +-
 .../models/CertificateProvisioningState.java  |   22 +-
 .../models/CommonSimPropertiesFormat.java     |   66 +-
 .../models/ConnectedClusterResourceId.java    |   19 +-
 .../mobilenetwork/models/CoreNetworkType.java |   26 +-
 .../models/CustomLocationResourceId.java      |   19 +-
 .../mobilenetwork/models/DataNetwork.java     |  101 +-
 .../models/DataNetworkConfiguration.java      |  124 +-
 .../models/DataNetworkListResult.java         |   16 +-
 .../models/DataNetworkResourceId.java         |   19 +-
 .../mobilenetwork/models/DataNetworks.java    |   30 +-
 .../models/DesiredInstallationState.java      |   18 +-
 .../models/DiagnosticsPackage.java            |   20 +-
 .../models/DiagnosticsPackageListResult.java  |   16 +-
 .../models/DiagnosticsPackageStatus.java      |   26 +-
 .../models/DiagnosticsPackages.java           |   44 +-
 .../DiagnosticsUploadConfiguration.java       |   21 +-
 .../mobilenetwork/models/DnnIpPair.java       |   83 +
 .../models/EncryptedSimUploadList.java        |   57 +-
 .../models/EventHubConfiguration.java         |   25 +-
 .../mobilenetwork/models/ExtendedUeInfo.java  |   55 +
 .../models/ExtendedUeInfoProperties.java      |   67 +
 .../models/ExtendedUeInformations.java        |   41 +
 .../mobilenetwork/models/GNbId.java           |   80 +
 .../mobilenetwork/models/GlobalRanNodeId.java |  248 +
 .../mobilenetwork/models/Guti4G.java          |  127 +
 .../mobilenetwork/models/Guti5G.java          |  127 +
 .../HomeNetworkPrivateKeysProvisioning.java   |   43 +
 ...meNetworkPrivateKeysProvisioningState.java |   59 +
 .../models/HomeNetworkPublicKey.java          |   86 +
 .../models/HttpsServerCertificate.java        |   22 +-
 .../models/IdentityAndTagsObject.java         |   18 +-
 .../mobilenetwork/models/Installation.java    |   22 +-
 .../models/InstallationReason.java            |   70 +-
 .../models/InstallationState.java             |   46 +-
 .../models/InterfaceProperties.java           |   34 +-
 .../mobilenetwork/models/KeyVaultKey.java     |   14 +-
 .../LocalDiagnosticsAccessConfiguration.java  |   28 +-
 .../models/ManagedServiceIdentity.java        |   39 +-
 .../models/ManagedServiceIdentityType.java    |   18 +-
 .../mobilenetwork/models/MmeId.java           |   80 +
 .../mobilenetwork/models/MobileNetwork.java   |  186 +-
 .../models/MobileNetworkListResult.java       |   16 +-
 .../models/MobileNetworkResourceId.java       |   19 +-
 .../mobilenetwork/models/MobileNetworks.java  |   34 +-
 .../models/NaptConfiguration.java             |   60 +-
 .../mobilenetwork/models/NaptEnabled.java     |   18 +-
 .../models/NasRerouteConfiguration.java       |   22 +-
 .../mobilenetwork/models/ObsoleteVersion.java |   18 +-
 .../mobilenetwork/models/Operation.java       |   12 +-
 .../models/OperationDisplay.java              |   26 +-
 .../mobilenetwork/models/OperationList.java   |   14 +-
 .../mobilenetwork/models/Operations.java      |    8 +-
 .../mobilenetwork/models/PacketCapture.java   |  143 +-
 .../models/PacketCaptureListResult.java       |   16 +-
 .../models/PacketCaptureStatus.java           |   26 +-
 .../mobilenetwork/models/PacketCaptures.java  |   46 +-
 .../models/PacketCoreControlPlane.java        |  291 +-
 ...ControlPlaneCollectDiagnosticsPackage.java |   21 +-
 .../PacketCoreControlPlaneListResult.java     |   16 +-
 .../PacketCoreControlPlaneResourceId.java     |   20 +-
 .../models/PacketCoreControlPlaneVersion.java |   18 +-
 ...cketCoreControlPlaneVersionListResult.java |   16 +-
 .../PacketCoreControlPlaneVersions.java       |   36 +-
 .../models/PacketCoreControlPlanes.java       |   81 +-
 .../models/PacketCoreDataPlane.java           |  121 +-
 .../models/PacketCoreDataPlaneListResult.java |   16 +-
 .../models/PacketCoreDataPlanes.java          |   46 +-
 .../models/PccRuleConfiguration.java          |   41 +-
 .../models/PccRuleQosPolicy.java              |   34 +-
 .../mobilenetwork/models/PdnType.java         |   48 +
 .../mobilenetwork/models/PduSessionType.java  |   18 +-
 .../mobilenetwork/models/PinholeTimeouts.java |   22 +-
 .../mobilenetwork/models/Platform.java        |   34 +-
 .../models/PlatformConfiguration.java         |   37 +-
 .../mobilenetwork/models/PlatformType.java    |   18 +-
 .../mobilenetwork/models/PlmnId.java          |   22 +-
 .../mobilenetwork/models/PortRange.java       |   21 +-
 .../models/PortReuseHoldTimes.java            |   14 +-
 .../models/PreemptionCapability.java          |   18 +-
 .../models/PreemptionVulnerability.java       |   18 +-
 .../models/ProvisioningState.java             |   38 +-
 .../models/PublicLandMobileNetwork.java       |   78 +
 ...andMobileNetworkHomeNetworkPublicKeys.java |   93 +
 .../mobilenetwork/models/QosPolicy.java       |   45 +-
 .../mobilenetwork/models/RatType.java         |   53 +
 .../models/RecommendedVersion.java            |   18 +-
 .../models/ReinstallRequired.java             |   18 +-
 .../models/RrcEstablishmentCause.java         |   73 +
 .../mobilenetwork/models/SdfDirection.java    |   22 +-
 .../mobilenetwork/models/Service.java         |  126 +-
 .../models/ServiceDataFlowTemplate.java       |   70 +-
 .../models/ServiceListResult.java             |   16 +-
 .../models/ServiceResourceId.java             |   19 +-
 .../mobilenetwork/models/Services.java        |   38 +-
 .../models/SignalingConfiguration.java        |   14 +-
 .../mobilenetwork/models/Sim.java             |  203 +-
 .../mobilenetwork/models/SimDeleteList.java   |   19 +-
 .../mobilenetwork/models/SimGroup.java        |  132 +-
 .../models/SimGroupListResult.java            |   16 +-
 .../mobilenetwork/models/SimGroups.java       |   30 +-
 .../mobilenetwork/models/SimListResult.java   |   16 +-
 .../models/SimNameAndEncryptedProperties.java |   81 +-
 .../models/SimNameAndProperties.java          |   84 +-
 .../mobilenetwork/models/SimPolicies.java     |   30 +-
 .../mobilenetwork/models/SimPolicy.java       |  149 +-
 .../models/SimPolicyListResult.java           |   16 +-
 .../models/SimPolicyResourceId.java           |   19 +-
 .../mobilenetwork/models/SimState.java        |   22 +-
 .../models/SimStaticIpProperties.java         |   22 +-
 .../models/SimStaticIpPropertiesStaticIp.java |   22 +-
 .../mobilenetwork/models/SimUploadList.java   |   19 +-
 .../mobilenetwork/models/Sims.java            |   54 +-
 .../mobilenetwork/models/Site.java            |   99 +-
 .../models/SiteDeletePacketCore.java          |   14 +-
 .../mobilenetwork/models/SiteListResult.java  |   16 +-
 .../models/SiteProvisioningState.java         |   34 +-
 .../mobilenetwork/models/SiteResourceId.java  |   19 +-
 .../mobilenetwork/models/Sites.java           |   46 +-
 .../mobilenetwork/models/Slice.java           |  112 +-
 .../models/SliceConfiguration.java            |   39 +-
 .../mobilenetwork/models/SliceListResult.java |   16 +-
 .../mobilenetwork/models/SliceResourceId.java |   19 +-
 .../mobilenetwork/models/Slices.java          |   30 +-
 .../mobilenetwork/models/Snssai.java          |   18 +-
 .../mobilenetwork/models/TagsObject.java      |   14 +-
 .../models/TrafficControlPermission.java      |   18 +-
 .../models/UeConnectionInfo4G.java            |  309 +
 .../models/UeConnectionInfo5G.java            |  339 +
 .../mobilenetwork/models/UeInfo.java          |   55 +
 .../mobilenetwork/models/UeInfo4G.java        |   79 +
 .../models/UeInfo4GProperties.java            |  204 +
 .../mobilenetwork/models/UeInfo5G.java        |   79 +
 .../models/UeInfo5GProperties.java            |  178 +
 .../mobilenetwork/models/UeInfoList.java      |   74 +
 .../models/UeInfoPropertiesFormat.java        |  148 +
 .../mobilenetwork/models/UeInformations.java  |   38 +
 .../mobilenetwork/models/UeIpAddress.java     |   54 +
 .../mobilenetwork/models/UeLocationInfo.java  |  123 +
 .../mobilenetwork/models/UeQosFlow.java       |  138 +
 .../mobilenetwork/models/UeSessionInfo4G.java |  149 +
 .../mobilenetwork/models/UeSessionInfo5G.java |  246 +
 .../mobilenetwork/models/UeState.java         |   68 +
 .../mobilenetwork/models/UeUsageSetting.java  |   53 +
 .../models/UserAssignedIdentity.java          |   14 +-
 .../mobilenetwork/models/VersionState.java    |   34 +-
 .../mobilenetwork/models/package-info.java    |    5 +-
 .../mobilenetwork/package-info.java           |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    2 +-
 .../reflect-config.json                       |  189 +-
 ...chedDataNetworksCreateOrUpdateSamples.java |   29 +-
 .../AttachedDataNetworksDeleteSamples.java    |   24 +-
 .../AttachedDataNetworksGetSamples.java       |   20 +-
 ...worksListByPacketCoreDataPlaneSamples.java |   19 +-
 ...AttachedDataNetworksUpdateTagsSamples.java |   26 +-
 .../DataNetworksCreateOrUpdateSamples.java    |   19 +-
 .../generated/DataNetworksDeleteSamples.java  |   10 +-
 .../generated/DataNetworksGetSamples.java     |   15 +-
 ...ataNetworksListByMobileNetworkSamples.java |   14 +-
 .../DataNetworksUpdateTagsSamples.java        |   18 +-
 ...gnosticsPackagesCreateOrUpdateSamples.java |   15 +-
 .../DiagnosticsPackagesDeleteSamples.java     |   10 +-
 .../DiagnosticsPackagesGetSamples.java        |   15 +-
 ...esListByPacketCoreControlPlaneSamples.java |   15 +-
 .../ExtendedUeInformationGetSamples.java      |   40 +
 .../MobileNetworksCreateOrUpdateSamples.java  |   29 +-
 .../MobileNetworksDeleteSamples.java          |   10 +-
 ...bileNetworksGetByResourceGroupSamples.java |   15 +-
 ...ileNetworksListByResourceGroupSamples.java |   14 +-
 .../generated/MobileNetworksListSamples.java  |   14 +-
 .../MobileNetworksUpdateTagsSamples.java      |   17 +-
 .../generated/OperationsListSamples.java      |   10 +-
 .../PacketCapturesCreateOrUpdateSamples.java  |   30 +-
 .../PacketCapturesDeleteSamples.java          |   10 +-
 .../generated/PacketCapturesGetSamples.java   |   10 +-
 ...esListByPacketCoreControlPlaneSamples.java |   15 +-
 .../generated/PacketCapturesStopSamples.java  |   10 +-
 ...PlaneVersionsGetBySubscriptionSamples.java |   15 +-
 ...ketCoreControlPlaneVersionsGetSamples.java |   14 +-
 ...laneVersionsListBySubscriptionSamples.java |   10 +-
 ...etCoreControlPlaneVersionsListSamples.java |   10 +-
 ...lanesCollectDiagnosticsPackageSamples.java |   23 +-
 ...oreControlPlanesCreateOrUpdateSamples.java |   72 +-
 .../PacketCoreControlPlanesDeleteSamples.java |   14 +-
 ...ontrolPlanesGetByResourceGroupSamples.java |   15 +-
 ...ntrolPlanesListByResourceGroupSamples.java |   10 +-
 .../PacketCoreControlPlanesListSamples.java   |   10 +-
 ...cketCoreControlPlanesReinstallSamples.java |   14 +-
 ...acketCoreControlPlanesRollbackSamples.java |   14 +-
 ...ketCoreControlPlanesUpdateTagsSamples.java |   36 +-
 ...etCoreDataPlanesCreateOrUpdateSamples.java |   18 +-
 .../PacketCoreDataPlanesDeleteSamples.java    |   15 +-
 .../PacketCoreDataPlanesGetSamples.java       |   15 +-
 ...esListByPacketCoreControlPlaneSamples.java |   19 +-
 ...PacketCoreDataPlanesUpdateTagsSamples.java |   22 +-
 .../ServicesCreateOrUpdateSamples.java        |   55 +-
 .../generated/ServicesDeleteSamples.java      |   10 +-
 .../generated/ServicesGetSamples.java         |    9 +-
 .../ServicesListByMobileNetworkSamples.java   |   19 +-
 .../generated/ServicesUpdateTagsSamples.java  |   17 +-
 .../SimGroupsCreateOrUpdateSamples.java       |   33 +-
 .../generated/SimGroupsDeleteSamples.java     |   10 +-
 .../SimGroupsGetByResourceGroupSamples.java   |   15 +-
 .../SimGroupsListByResourceGroupSamples.java  |   14 +-
 .../generated/SimGroupsListSamples.java       |   14 +-
 .../generated/SimGroupsUpdateTagsSamples.java |   32 +-
 .../SimPoliciesCreateOrUpdateSamples.java     |   73 +-
 .../generated/SimPoliciesDeleteSamples.java   |   10 +-
 .../generated/SimPoliciesGetSamples.java      |   14 +-
 ...SimPoliciesListByMobileNetworkSamples.java |   19 +-
 .../SimPoliciesUpdateTagsSamples.java         |   17 +-
 .../generated/SimsBulkDeleteSamples.java      |   23 +-
 .../SimsBulkUploadEncryptedSamples.java       |  105 +-
 .../generated/SimsBulkUploadSamples.java      |   96 +-
 .../generated/SimsCreateOrUpdateSamples.java  |   42 +-
 .../generated/SimsDeleteSamples.java          |    9 +-
 .../generated/SimsGetSamples.java             |   14 +-
 .../generated/SimsListByGroupSamples.java     |   10 +-
 .../generated/SitesCreateOrUpdateSamples.java |   17 +-
 .../SitesDeletePacketCoreSamples.java         |   30 +-
 .../generated/SitesDeleteSamples.java         |    9 +-
 .../generated/SitesGetSamples.java            |    9 +-
 .../SitesListByMobileNetworkSamples.java      |   14 +-
 .../generated/SitesUpdateTagsSamples.java     |   21 +-
 .../SlicesCreateOrUpdateSamples.java          |   19 +-
 .../generated/SlicesDeleteSamples.java        |    9 +-
 .../generated/SlicesGetSamples.java           |    9 +-
 .../SlicesListByMobileNetworkSamples.java     |   14 +-
 .../generated/SlicesUpdateTagsSamples.java    |   17 +-
 .../generated/UeInformationListSamples.java   |   23 +
 .../mobilenetwork/generated/AmbrTests.java    |   13 +-
 .../mobilenetwork/generated/AmfIdTests.java   |   29 +
 .../generated/AsyncOperationIdTests.java      |    8 +-
 .../AsyncOperationStatusInnerTests.java       |   51 +-
 .../AttachedDataNetworkInnerTests.java        |   38 +-
 .../AttachedDataNetworkListResultTests.java   |  124 +-
 ...achedDataNetworkPropertiesFormatTests.java |   38 +-
 .../AttachedDataNetworkResourceIdTests.java   |   10 +-
 ...edDataNetworksCreateOrUpdateMockTests.java |  107 +-
 .../AttachedDataNetworksDeleteMockTests.java  |   39 +-
 ...dDataNetworksGetWithResponseMockTests.java |   76 +-
 ...rksListByPacketCoreDataPlaneMockTests.java |   76 +-
 .../AzureStackEdgeDeviceResourceIdTests.java  |   10 +-
 .../AzureStackHciClusterResourceIdTests.java  |   10 +-
 .../CertificateProvisioningTests.java         |    6 +-
 .../ConnectedClusterResourceIdTests.java      |   10 +-
 .../CustomLocationResourceIdTests.java        |   10 +-
 .../DataNetworkConfigurationTests.java        |   78 +-
 .../generated/DataNetworkInnerTests.java      |   15 +-
 .../generated/DataNetworkListResultTests.java |   26 +-
 .../DataNetworkPropertiesFormatTests.java     |    5 +-
 .../generated/DataNetworkResourceIdTests.java |   10 +-
 .../DataNetworksCreateOrUpdateMockTests.java  |   55 +-
 .../DataNetworksDeleteMockTests.java          |   38 +-
 .../DataNetworksGetWithResponseMockTests.java |   46 +-
 ...aNetworksListByMobileNetworkMockTests.java |   45 +-
 .../DiagnosticsPackageInnerTests.java         |    8 +-
 .../DiagnosticsPackageListResultTests.java    |   13 +-
 ...agnosticsPackagePropertiesFormatTests.java |    7 +-
 ...osticsPackagesCreateOrUpdateMockTests.java |   40 +-
 .../DiagnosticsPackagesDeleteMockTests.java   |   38 +-
 ...sticsPackagesGetWithResponseMockTests.java |   42 +-
 ...ListByPacketCoreControlPlaneMockTests.java |   40 +-
 .../DiagnosticsUploadConfigurationTests.java  |   13 +-
 .../generated/DnnIpPairTests.java             |   28 +
 .../generated/EventHubConfigurationTests.java |   17 +-
 .../generated/ExtendedUeInfoInnerTests.java   |   29 +
 .../ExtendedUeInfoPropertiesTests.java        |   28 +
 ...InformationsGetWithResponseMockTests.java} |   18 +-
 .../mobilenetwork/generated/GNbIdTests.java   |   27 +
 .../generated/GlobalRanNodeIdTests.java       |   48 +
 .../mobilenetwork/generated/Guti5GTests.java  |   40 +
 ...meNetworkPrivateKeysProvisioningTests.java |   22 +
 .../generated/HomeNetworkPublicKeyTests.java  |   27 +
 .../HttpsServerCertificateTests.java          |   15 +-
 .../generated/IdentityAndTagsObjectTests.java |   34 +-
 .../generated/InstallationTests.java          |   14 +-
 .../generated/InterfacePropertiesTests.java   |   17 +-
 ...alDiagnosticsAccessConfigurationTests.java |   23 +-
 .../ManagedServiceIdentityTests.java          |   20 +-
 .../MobileNetworkResourceIdTests.java         |   10 +-
 .../MobileNetworksDeleteMockTests.java        |   36 +-
 .../generated/NaptConfigurationTests.java     |   21 +-
 .../NasRerouteConfigurationTests.java         |   10 +-
 .../generated/OperationDisplayTests.java      |   32 +-
 .../generated/OperationInnerTests.java        |   14 +-
 .../generated/OperationListTests.java         |    8 +-
 .../generated/OperationsListMockTests.java    |   35 +-
 .../generated/PacketCaptureInnerTests.java    |   34 +-
 .../PacketCaptureListResultTests.java         |   39 +-
 .../PacketCapturePropertiesFormatTests.java   |   34 +-
 ...PacketCapturesCreateOrUpdateMockTests.java |   62 +-
 .../PacketCapturesDeleteMockTests.java        |   36 +-
 ...acketCapturesGetWithResponseMockTests.java |   54 +-
 ...ListByPacketCoreControlPlaneMockTests.java |   53 +-
 .../PacketCapturesStopMockTests.java          |   59 +-
 ...olPlaneCollectDiagnosticsPackageTests.java |   13 +-
 .../PacketCoreControlPlaneInnerTests.java     |  162 -
 ...PacketCoreControlPlaneListResultTests.java |  181 -
 ...CoreControlPlanePropertiesFormatTests.java |  121 -
 ...PacketCoreControlPlaneResourceIdTests.java |   10 +-
 ...cketCoreControlPlaneVersionInnerTests.java |   63 +-
 ...oreControlPlaneVersionListResultTests.java |  157 +-
 ...trolPlaneVersionPropertiesFormatTests.java |   56 +-
 ...etBySubscriptionWithResponseMockTests.java |   52 +-
 ...PlaneVersionsGetWithResponseMockTests.java |   48 +-
 ...neVersionsListBySubscriptionMockTests.java |   60 +-
 ...CoreControlPlaneVersionsListMockTests.java |   63 +-
 ...nesCollectDiagnosticsPackageMockTests.java |   58 +-
 ...eControlPlanesCreateOrUpdateMockTests.java |  173 -
 ...acketCoreControlPlanesDeleteMockTests.java |   36 +-
 ...tByResourceGroupWithResponseMockTests.java |  101 -
 ...rolPlanesListByResourceGroupMockTests.java |  106 -
 .../PacketCoreControlPlanesListMockTests.java |  105 -
 ...etCoreControlPlanesReinstallMockTests.java |   57 +-
 ...ketCoreControlPlanesRollbackMockTests.java |   57 +-
 .../PacketCoreDataPlaneInnerTests.java        |   54 +-
 .../PacketCoreDataPlaneListResultTests.java   |   78 +-
 ...ketCoreDataPlanePropertiesFormatTests.java |   41 +-
 ...CoreDataPlanesCreateOrUpdateMockTests.java |   69 +-
 .../PacketCoreDataPlanesDeleteMockTests.java  |   36 +-
 ...oreDataPlanesGetWithResponseMockTests.java |   60 +-
 ...ListByPacketCoreControlPlaneMockTests.java |   59 +-
 .../generated/PccRuleConfigurationTests.java  |  105 +-
 .../generated/PccRuleQosPolicyTests.java      |   54 +-
 .../generated/PinholeTimeoutsTests.java       |    6 +-
 .../generated/PlatformConfigurationTests.java |   36 +-
 .../generated/PlatformTests.java              |   43 +-
 .../mobilenetwork/generated/PlmnIdTests.java  |   13 +-
 .../generated/PortRangeTests.java             |    4 +-
 .../generated/PortReuseHoldTimesTests.java    |    4 +-
 ...bileNetworkHomeNetworkPublicKeysTests.java |   42 +
 .../generated/QosPolicyTests.java             |   35 +-
 .../ServiceDataFlowTemplateTests.java         |   39 +-
 .../generated/ServiceInnerTests.java          |  253 +-
 .../generated/ServiceListResultTests.java     |  539 +-
 .../ServicePropertiesFormatTests.java         |  343 +-
 .../generated/ServiceResourceIdTests.java     |   10 +-
 .../ServicesCreateOrUpdateMockTests.java      |  247 +-
 .../generated/ServicesDeleteMockTests.java    |   36 +-
 .../ServicesGetWithResponseMockTests.java     |  104 +-
 .../ServicesListByMobileNetworkMockTests.java |  129 +-
 .../SignalingConfigurationTests.java          |   14 +-
 .../generated/SimDeleteListTests.java         |    9 +-
 .../generated/SimGroupsDeleteMockTests.java   |   36 +-
 .../SimPoliciesCreateOrUpdateMockTests.java   |  163 +-
 .../generated/SimPoliciesDeleteMockTests.java |   36 +-
 .../SimPoliciesGetWithResponseMockTests.java  |   77 +-
 ...mPoliciesListByMobileNetworkMockTests.java |   99 +-
 .../generated/SimPolicyInnerTests.java        |  419 +-
 .../generated/SimPolicyListResultTests.java   |  444 +-
 .../SimPolicyPropertiesFormatTests.java       |  460 +-
 .../generated/SimPolicyResourceIdTests.java   |    8 +-
 .../SimStaticIpPropertiesStaticIpTests.java   |   10 +-
 .../generated/SimStaticIpPropertiesTests.java |   29 +-
 .../generated/SimsBulkDeleteMockTests.java    |   58 +-
 .../generated/SimsDeleteMockTests.java        |   36 +-
 .../generated/SiteDeletePacketCoreTests.java  |   12 +-
 .../generated/SiteInnerTests.java             |   23 +-
 .../generated/SiteListResultTests.java        |   53 +-
 .../generated/SitePropertiesFormatTests.java  |    8 +-
 .../generated/SiteResourceIdTests.java        |    8 +-
 .../SitesCreateOrUpdateMockTests.java         |   47 +-
 .../generated/SitesDeleteMockTests.java       |   36 +-
 .../SitesDeletePacketCoreMockTests.java       |   41 +-
 .../SitesGetWithResponseMockTests.java        |   42 +-
 .../SitesListByMobileNetworkMockTests.java    |   41 +-
 .../generated/SliceConfigurationTests.java    |  157 +-
 .../generated/SliceInnerTests.java            |   37 +-
 .../generated/SliceListResultTests.java       |   86 +-
 .../generated/SlicePropertiesFormatTests.java |   26 +-
 .../generated/SliceResourceIdTests.java       |    8 +-
 .../SlicesCreateOrUpdateMockTests.java        |   58 +-
 .../generated/SlicesDeleteMockTests.java      |   36 +-
 .../SlicesGetWithResponseMockTests.java       |   56 +-
 .../SlicesListByMobileNetworkMockTests.java   |   53 +-
 .../mobilenetwork/generated/SnssaiTests.java  |   12 +-
 .../generated/TagsObjectTests.java            |   14 +-
 .../generated/UeConnectionInfo4GTests.java    |   86 +
 .../generated/UeConnectionInfo5GTests.java    |   94 +
 .../generated/UeInfo5GPropertiesTests.java    |  169 +
 .../generated/UeInfo5GTests.java              |  166 +
 .../generated/UeInfoInnerTests.java           |   45 +
 .../generated/UeInfoListTests.java            |   46 +
 .../UeInfoPropertiesFormatTests.java          |   44 +
 .../UeInformationsListMockTests.java          |   66 +
 .../generated/UeIpAddressTests.java           |   24 +
 .../generated/UeLocationInfoTests.java        |   34 +
 .../generated/UeQosFlowTests.java             |   39 +
 .../generated/UeSessionInfo4GTests.java       |   35 +
 .../generated/UeSessionInfo5GTests.java       |   71 +
 .../generated/UserAssignedIdentityTests.java  |    8 +-
 .../azure-iot-modelsrepository/pom.xml        |    2 +-
 sdk/modelsrepository/tests.yml                |   18 +-
 .../azure-monitor-ingestion-perf/pom.xml      |    4 +-
 .../azure-monitor-ingestion/CHANGELOG.md      |   10 +
 sdk/monitor/azure-monitor-ingestion/pom.xml   |    6 +-
 .../CHANGELOG.md                              |   10 +
 .../pom.xml                                   |   44 +-
 .../tests.yml                                 |   24 +-
 sdk/monitor/azure-monitor-query-perf/pom.xml  |    2 +-
 sdk/monitor/azure-monitor-query/CHANGELOG.md  |   11 +
 sdk/monitor/azure-monitor-query/README.md     |   22 +-
 sdk/monitor/azure-monitor-query/pom.xml       |    4 +-
 .../monitor/query/LogsQueryAsyncClient.java   |   44 +-
 .../azure/monitor/query/LogsQueryClient.java  |   43 +-
 .../monitor/query/LogsQueryClientBuilder.java |   34 +-
 .../monitor/query/MetricsAsyncClient.java     |   34 +-
 .../azure/monitor/query/MetricsClient.java    |   34 +-
 .../monitor/query/MetricsClientBuilder.java   |    7 +-
 .../query/MetricsQueryAsyncClient.java        |   36 +-
 .../monitor/query/MetricsQueryClient.java     |   36 +-
 .../query/MetricsQueryClientBuilder.java      |   40 +-
 .../query/MetricsQueryServiceVersion.java     |    8 +-
 .../monitor/query/MetricsServiceVersion.java  |    4 +-
 .../metrics/models/MetricsHelper.java         |    2 +-
 .../monitor/query/models/LogsBatchQuery.java  |    5 +
 .../monitor/query/models/LogsColumnType.java  |   11 +-
 .../query/models/LogsQueryOptions.java        |    5 +
 .../query/models/MetricAvailability.java      |    5 +
 .../monitor/query/models/MetricClass.java     |   11 +-
 .../query/models/MetricDefinition.java        |    5 +
 .../monitor/query/models/MetricNamespace.java |    5 +
 .../monitor/query/models/MetricUnit.java      |   11 +-
 .../query/models/MetricsQueryOptions.java     |    5 +
 ...java => MetricsQueryResourcesOptions.java} |   22 +-
 ....java => MetricsQueryResourcesResult.java} |    4 +-
 .../query/models/NamespaceClassification.java |   11 +-
 .../com/azure/monitor/query/package-info.java |  117 +-
 ...tchQuerySample.java => MetricsSample.java} |   10 +-
 .../azure/monitor/query/ReadmeSamples.java    |   28 +-
 ...csBatchQueryClientJavaDocCodeSnippets.java |   10 +-
 .../monitor/query/MetricsClientTest.java      |   10 +-
 sdk/monitor/azure-monitor-query/tests.yml     |   24 +-
 sdk/monitor/tests.yml                         |   34 +-
 sdk/mysql/azure-resourcemanager-mysql/pom.xml |    4 +-
 sdk/mysql/tests.mgmt.yml                      |   24 +-
 .../pom.xml                                   |    8 +-
 sdk/mysqlflexibleserver/tests.mgmt.yml        |   24 +-
 .../azure-resourcemanager-netapp/CHANGELOG.md |   12 +-
 .../azure-resourcemanager-netapp/README.md    |    4 +-
 .../azure-resourcemanager-netapp/SAMPLE.md    | 1253 ++--
 .../azure-resourcemanager-netapp/pom.xml      |   14 +-
 .../netapp/NetAppFilesManager.java            |  102 +-
 .../netapp/fluent/AccountsClient.java         |   69 -
 .../netapp/fluent/BackupsClient.java          |  338 -
 .../fluent/BackupsUnderAccountsClient.java    |   83 -
 .../BackupsUnderBackupVaultsClient.java       |   93 -
 .../fluent/BackupsUnderVolumesClient.java     |   93 -
 .../netapp/fluent/NetAppManagementClient.java |   42 -
 .../netapp/fluent/VolumesClient.java          |   71 -
 .../fluent/models/AccountProperties.java      |   44 -
 .../netapp/fluent/models/BackupInner.java     |  234 -
 .../fluent/models/BackupPatchProperties.java  |   54 -
 .../fluent/models/BackupProperties.java       |  261 -
 .../fluent/models/BackupStatusInner.java      |  166 -
 .../fluent/models/BackupVaultInner.java       |   91 -
 .../fluent/models/BackupVaultProperties.java  |   43 -
 .../fluent/models/NetAppAccountInner.java     |   34 -
 .../models/RegionInfoResourceInner.java       |  122 -
 .../netapp/fluent/models/VolumeInner.java     |   15 +-
 .../fluent/models/VolumeProperties.java       |   26 +-
 .../implementation/AccountBackupsImpl.java    |   78 -
 .../implementation/AccountsClientImpl.java    |  329 +-
 .../netapp/implementation/AccountsImpl.java   |   34 +-
 .../netapp/implementation/BackupImpl.java     |  193 -
 .../BackupPoliciesClientImpl.java             |   10 +-
 .../implementation/BackupPoliciesImpl.java    |   28 +-
 .../implementation/BackupPolicyImpl.java      |    6 +-
 .../implementation/BackupStatusImpl.java      |   66 -
 .../implementation/BackupVaultImpl.java       |  166 -
 .../implementation/BackupVaultsImpl.java      |  159 -
 .../implementation/BackupsClientImpl.java     | 1536 ----
 .../netapp/implementation/BackupsImpl.java    |  172 -
 .../BackupsUnderAccountsClientImpl.java       |  322 -
 .../BackupsUnderAccountsImpl.java             |   42 -
 .../BackupsUnderBackupVaultsClientImpl.java   |  363 -
 .../BackupsUnderBackupVaultsImpl.java         |   43 -
 .../BackupsUnderVolumesClientImpl.java        |  361 -
 .../BackupsUnderVolumesImpl.java              |   43 -
 .../implementation/CapacityPoolImpl.java      |    6 +-
 .../implementation/NetAppAccountImpl.java     |   31 +-
 .../NetAppManagementClientImpl.java           |   98 +-
 .../NetAppResourceQuotaLimitsImpl.java        |    4 +-
 .../NetAppResourceRegionInfosClientImpl.java  |  390 -
 .../NetAppResourceRegionInfosImpl.java        |   66 -
 .../netapp/implementation/OperationsImpl.java |    4 +-
 .../implementation/PoolsClientImpl.java       |   11 +-
 .../netapp/implementation/PoolsImpl.java      |   28 +-
 .../RegionInfoResourceImpl.java               |   62 -
 .../{Utils.java => ResourceManagerUtils.java} |    4 +-
 .../SnapshotPoliciesClientImpl.java           |   10 +-
 .../implementation/SnapshotPoliciesImpl.java  |   28 +-
 .../implementation/SnapshotPolicyImpl.java    |    6 +-
 .../implementation/SnapshotsClientImpl.java   |   21 +-
 .../netapp/implementation/SnapshotsImpl.java  |   44 +-
 .../implementation/SubvolumeInfoImpl.java     |   10 +-
 .../implementation/SubvolumesClientImpl.java  |   10 +-
 .../netapp/implementation/SubvolumesImpl.java |   44 +-
 .../VolumeGroupsClientImpl.java               |   10 +-
 .../implementation/VolumeGroupsImpl.java      |   28 +-
 .../netapp/implementation/VolumeImpl.java     |   20 +-
 .../implementation/VolumeQuotaRuleImpl.java   |   10 +-
 .../VolumeQuotaRulesClientImpl.java           |   14 +-
 .../implementation/VolumeQuotaRulesImpl.java  |   44 +-
 .../implementation/VolumesClientImpl.java     |  427 +-
 .../netapp/implementation/VolumesImpl.java    |   49 +-
 .../netapp/models/Accounts.java               |   31 -
 .../resourcemanager/netapp/models/Backup.java |  311 -
 .../netapp/models/BackupPatch.java            |   70 -
 .../netapp/models/BackupRestoreFiles.java     |  125 -
 .../netapp/models/BackupStatus.java           |   83 -
 .../netapp/models/BackupType.java             |   55 -
 .../netapp/models/BackupVault.java            |  240 -
 .../netapp/models/BackupVaultPatch.java       |   57 -
 .../netapp/models/Backups.java                |  195 -
 .../models/BackupsMigrationRequest.java       |   61 -
 .../models/BackupsUnderBackupVaults.java      |   47 -
 .../netapp/models/BackupsUnderVolumes.java    |   47 -
 .../models/EncryptionMigrationRequest.java    |  103 -
 .../netapp/models/NetAppAccount.java          |   77 +-
 .../netapp/models/NetAppAccountPatch.java     |   34 -
 .../netapp/models/RegionInfoResource.java     |   67 -
 .../netapp/models/RelationshipStatus.java     |   10 +
 .../netapp/models/RemotePath.java             |  121 -
 .../netapp/models/ReplicationObject.java      |   39 +-
 .../UpdateNetworkSiblingSetRequest.java       |    6 +-
 .../resourcemanager/netapp/models/Volume.java |   37 +-
 .../netapp/models/VolumeBackupProperties.java |  132 -
 .../models/VolumeGroupVolumeProperties.java   |   15 +-
 .../VolumePatchPropertiesDataProtection.java  |   29 -
 .../VolumePropertiesDataProtection.java       |   29 -
 .../netapp/models/Volumes.java                |   32 -
 .../src/main/java/module-info.java            |    2 -
 .../proxy-config.json                         |    2 +-
 .../reflect-config.json                       |   90 -
 .../AccountBackupsDeleteSamples.java          |   24 -
 .../generated/AccountBackupsGetSamples.java   |   24 -
 ...ountBackupsListByNetAppAccountSamples.java |   24 -
 .../AccountsCreateOrUpdateSamples.java        |    8 +-
 .../generated/AccountsDeleteSamples.java      |    2 +-
 .../AccountsGetByResourceGroupSamples.java    |    2 +-
 .../AccountsListByResourceGroupSamples.java   |    2 +-
 .../netapp/generated/AccountsListSamples.java |    2 +-
 .../AccountsMigrateEncryptionKeySamples.java  |   30 -
 .../AccountsRenewCredentialsSamples.java      |    4 +-
 .../generated/AccountsUpdateSamples.java      |    2 +-
 .../BackupPoliciesCreateSamples.java          |    3 +-
 .../BackupPoliciesDeleteSamples.java          |    3 +-
 .../generated/BackupPoliciesGetSamples.java   |    3 +-
 .../generated/BackupPoliciesListSamples.java  |    3 +-
 .../BackupPoliciesUpdateSamples.java          |    3 +-
 .../BackupVaultsCreateOrUpdateSamples.java    |   25 -
 .../generated/BackupVaultsDeleteSamples.java  |   24 -
 .../generated/BackupVaultsGetSamples.java     |   23 -
 ...ackupVaultsListByNetAppAccountSamples.java |   23 -
 .../generated/BackupVaultsUpdateSamples.java  |   42 -
 .../generated/BackupsCreateSamples.java       |   26 -
 .../generated/BackupsDeleteSamples.java       |   24 -
 .../BackupsGetLatestStatusSamples.java        |   24 -
 .../netapp/generated/BackupsGetSamples.java   |   24 -
 .../BackupsGetVolumeRestoreStatusSamples.java |    3 +-
 .../generated/BackupsListByVaultSamples.java  |   23 -
 ...kupsUnderAccountMigrateBackupsSamples.java |   28 -
 ...psUnderBackupVaultRestoreFilesSamples.java |   30 -
 ...ckupsUnderVolumeMigrateBackupsSamples.java |   28 -
 .../generated/BackupsUpdateSamples.java       |   28 -
 ...ourceCheckFilePathAvailabilitySamples.java |    4 +-
 ...pResourceCheckNameAvailabilitySamples.java |    3 +-
 ...ResourceCheckQuotaAvailabilitySamples.java |    3 +-
 ...ResourceQueryNetworkSiblingSetSamples.java |    4 +-
 .../NetAppResourceQueryRegionInfoSamples.java |    2 +-
 .../NetAppResourceQuotaLimitsGetSamples.java  |    2 +-
 .../NetAppResourceQuotaLimitsListSamples.java |    2 +-
 .../NetAppResourceRegionInfosGetSamples.java  |   23 -
 .../NetAppResourceRegionInfosListSamples.java |   23 -
 ...esourceUpdateNetworkSiblingSetSamples.java |    4 +-
 .../generated/OperationsListSamples.java      |    2 +-
 .../generated/PoolsCreateOrUpdateSamples.java |    3 +-
 .../netapp/generated/PoolsDeleteSamples.java  |    2 +-
 .../netapp/generated/PoolsGetSamples.java     |    2 +-
 .../netapp/generated/PoolsListSamples.java    |    2 +-
 .../netapp/generated/PoolsUpdateSamples.java  |    2 +-
 .../SnapshotPoliciesCreateSamples.java        |    4 +-
 .../SnapshotPoliciesDeleteSamples.java        |    4 +-
 .../generated/SnapshotPoliciesGetSamples.java |    3 +-
 .../SnapshotPoliciesListSamples.java          |    3 +-
 .../SnapshotPoliciesListVolumesSamples.java   |    5 +-
 .../SnapshotPoliciesUpdateSamples.java        |    4 +-
 .../generated/SnapshotsCreateSamples.java     |    2 +-
 .../generated/SnapshotsDeleteSamples.java     |    2 +-
 .../netapp/generated/SnapshotsGetSamples.java |    2 +-
 .../generated/SnapshotsListSamples.java       |    2 +-
 .../SnapshotsRestoreFilesSamples.java         |    5 +-
 .../generated/SnapshotsUpdateSamples.java     |    2 +-
 .../generated/SubvolumesCreateSamples.java    |    2 +-
 .../generated/SubvolumesDeleteSamples.java    |    2 +-
 .../SubvolumesGetMetadataSamples.java         |    3 +-
 .../generated/SubvolumesGetSamples.java       |    2 +-
 .../SubvolumesListByVolumeSamples.java        |    2 +-
 .../generated/SubvolumesUpdateSamples.java    |    2 +-
 .../generated/VolumeGroupsCreateSamples.java  |    9 +-
 .../generated/VolumeGroupsDeleteSamples.java  |    3 +-
 .../generated/VolumeGroupsGetSamples.java     |    8 +-
 ...olumeGroupsListByNetAppAccountSamples.java |    8 +-
 .../VolumeQuotaRulesCreateSamples.java        |    4 +-
 .../VolumeQuotaRulesDeleteSamples.java        |    4 +-
 .../generated/VolumeQuotaRulesGetSamples.java |    3 +-
 .../VolumeQuotaRulesListByVolumeSamples.java  |    3 +-
 .../VolumeQuotaRulesUpdateSamples.java        |    4 +-
 .../VolumesAuthorizeReplicationSamples.java   |    5 +-
 .../VolumesBreakFileLocksSamples.java         |    3 +-
 .../VolumesBreakReplicationSamples.java       |    4 +-
 .../VolumesCreateOrUpdateSamples.java         |    7 +-
 .../VolumesDeleteReplicationSamples.java      |    4 +-
 .../generated/VolumesDeleteSamples.java       |    2 +-
 .../VolumesFinalizeRelocationSamples.java     |    4 +-
 .../netapp/generated/VolumesGetSamples.java   |    2 +-
 ...sListGetGroupIdListForLdapUserSamples.java |    3 +-
 .../VolumesListReplicationsSamples.java       |    4 +-
 .../netapp/generated/VolumesListSamples.java  |    2 +-
 .../generated/VolumesPoolChangeSamples.java   |    3 +-
 ...olumesPopulateAvailabilityZoneSamples.java |    2 +-
 ...VolumesReInitializeReplicationSamples.java |    5 +-
 .../VolumesReestablishReplicationSamples.java |    5 +-
 .../generated/VolumesRelocateSamples.java     |    2 +-
 .../VolumesReplicationStatusSamples.java      |    4 +-
 .../VolumesResetCifsPasswordSamples.java      |    4 +-
 .../VolumesResyncReplicationSamples.java      |    4 +-
 .../VolumesRevertRelocationSamples.java       |    4 +-
 .../generated/VolumesRevertSamples.java       |    2 +-
 .../VolumesSplitCloneFromParentSamples.java   |   25 -
 .../generated/VolumesUpdateSamples.java       |    9 +-
 .../AccountsRenewCredentialsMockTests.java    |    2 +-
 .../generated/AuthorizeRequestTests.java      |    8 +-
 .../netapp/generated/BackupInnerTests.java    |   33 -
 .../generated/BackupPatchPropertiesTests.java |   25 -
 .../netapp/generated/BackupPatchTests.java    |   25 -
 .../BackupPoliciesCreateMockTests.java        |   21 +-
 .../BackupPoliciesDeleteMockTests.java        |    2 +-
 ...ackupPoliciesGetWithResponseMockTests.java |   14 +-
 .../BackupPoliciesListMockTests.java          |   18 +-
 .../generated/BackupPoliciesListTests.java    |   45 +-
 .../generated/BackupPolicyInnerTests.java     |   29 +-
 .../generated/BackupPolicyPatchTests.java     |   30 +-
 .../BackupPolicyPropertiesTests.java          |   18 +-
 .../generated/BackupPropertiesTests.java      |   33 -
 .../generated/BackupRestoreFilesTests.java    |   33 -
 .../generated/BackupStatusInnerTests.java     |   23 -
 .../generated/BackupVaultInnerTests.java      |   43 -
 .../generated/BackupVaultPatchTests.java      |   41 -
 .../generated/BackupVaultPropertiesTests.java |   22 -
 .../generated/BackupVaultsListTests.java      |   51 -
 ...umeRestoreStatusWithResponseMockTests.java |    6 +-
 .../netapp/generated/BackupsListTests.java    |   45 -
 .../BackupsMigrationRequestTests.java         |   25 -
 .../generated/BreakFileLocksRequestTests.java |    8 +-
 .../generated/CapacityPoolInnerTests.java     |   34 +-
 .../generated/CapacityPoolListTests.java      |   54 +-
 .../generated/CapacityPoolPatchTests.java     |   24 +-
 .../netapp/generated/DailyScheduleTests.java  |   22 +-
 .../generated/EncryptionIdentityTests.java    |   11 +-
 .../EncryptionMigrationRequestTests.java      |   29 -
 .../generated/ExportPolicyRuleTests.java      |   52 +-
 ...GetGroupIdListForLdapUserRequestTests.java |    8 +-
 ...upIdListForLdapUserResponseInnerTests.java |   12 +-
 .../netapp/generated/HourlyScheduleTests.java |   18 +-
 .../generated/LdapSearchScopeOptTests.java    |   18 +-
 .../generated/ListReplicationsTests.java      |   26 +-
 .../ManagedServiceIdentityTests.java          |   14 +-
 .../generated/MonthlyScheduleTests.java       |   26 +-
 .../generated/MountTargetPropertiesTests.java |   12 +-
 ...ceQuotaLimitsGetWithResponseMockTests.java |    4 +-
 ...etAppResourceQuotaLimitsListMockTests.java |    4 +-
 ...ceRegionInfosGetWithResponseMockTests.java |   59 -
 ...PathAvailabilityWithResponseMockTests.java |    8 +-
 ...NameAvailabilityWithResponseMockTests.java |   12 +-
 ...uotaAvailabilityWithResponseMockTests.java |   14 +-
 ...etworkSiblingSetWithResponseMockTests.java |   16 +-
 ...sQueryRegionInfoWithResponseMockTests.java |   12 +-
 ...urcesUpdateNetworkSiblingSetMockTests.java |   18 +-
 .../NetworkSiblingSetInnerTests.java          |   34 +-
 .../netapp/generated/NicInfoTests.java        |   11 +-
 .../generated/OperationsListMockTests.java    |   46 +-
 .../generated/PoolChangeRequestTests.java     |    8 +-
 .../generated/PoolPatchPropertiesTests.java   |   18 +-
 .../netapp/generated/PoolPropertiesTests.java |   19 +-
 .../PoolsCreateOrUpdateMockTests.java         |   21 +-
 .../PoolsGetWithResponseMockTests.java        |   14 +-
 .../netapp/generated/PoolsListMockTests.java  |   18 +-
 .../QueryNetworkSiblingSetRequestTests.java   |   12 +-
 .../ReestablishReplicationRequestTests.java   |    8 +-
 ...InfoAvailabilityZoneMappingsItemTests.java |    8 +-
 .../generated/RegionInfoInnerTests.java       |   26 +-
 .../RegionInfoResourceInnerTests.java         |   37 -
 .../generated/RegionInfosListTests.java       |   69 -
 .../netapp/generated/RemotePathTests.java     |   31 -
 .../generated/ReplicationInnerTests.java      |   20 +-
 .../generated/ReplicationObjectTests.java     |   33 +-
 .../ReplicationStatusInnerTests.java          |   18 +-
 .../generated/RestoreStatusInnerTests.java    |    2 +-
 .../netapp/generated/SnapshotInnerTests.java  |    8 +-
 ...otPoliciesCreateWithResponseMockTests.java |   63 +-
 .../SnapshotPoliciesDeleteMockTests.java      |    2 +-
 ...pshotPoliciesGetWithResponseMockTests.java |   42 +-
 .../SnapshotPoliciesListMockTests.java        |   42 +-
 .../generated/SnapshotPoliciesListTests.java  |  137 +-
 .../generated/SnapshotPolicyInnerTests.java   |   98 +-
 .../generated/SnapshotPolicyPatchTests.java   |  100 +-
 .../SnapshotPolicyPropertiesTests.java        |   87 +-
 .../generated/SnapshotPropertiesTests.java    |    5 +-
 .../generated/SnapshotRestoreFilesTests.java  |   14 +-
 .../generated/SnapshotsDeleteMockTests.java   |    2 +-
 .../SnapshotsGetWithResponseMockTests.java    |    8 +-
 .../generated/SnapshotsListMockTests.java     |    8 +-
 .../netapp/generated/SnapshotsListTests.java  |   10 +-
 .../SnapshotsRestoreFilesMockTests.java       |    5 +-
 .../generated/SnapshotsUpdateMockTests.java   |    8 +-
 .../generated/SubvolumeInfoInnerTests.java    |   18 +-
 .../generated/SubvolumeModelInnerTests.java   |   54 +-
 .../SubvolumeModelPropertiesTests.java        |   54 +-
 .../generated/SubvolumePatchParamsTests.java  |   16 +-
 .../generated/SubvolumePatchRequestTests.java |   13 +-
 .../generated/SubvolumePropertiesTests.java   |   18 +-
 .../generated/SubvolumesCreateMockTests.java  |   14 +-
 .../generated/SubvolumesDeleteMockTests.java  |    3 +-
 .../SubvolumesGetMetadataMockTests.java       |   24 +-
 .../SubvolumesGetWithResponseMockTests.java   |   10 +-
 .../SubvolumesListByVolumeMockTests.java      |   12 +-
 .../netapp/generated/SubvolumesListTests.java |   27 +-
 .../UpdateNetworkSiblingSetRequestTests.java  |   24 +-
 .../generated/UserAssignedIdentityTests.java  |    2 +-
 .../VolumeBackupPropertiesTests.java          |   33 -
 .../netapp/generated/VolumeBackupsTests.java  |   16 +-
 .../VolumeGroupsDeleteMockTests.java          |    3 +-
 ...umePatchPropertiesDataProtectionTests.java |   20 +-
 ...olumePatchPropertiesExportPolicyTests.java |   72 +-
 .../generated/VolumePatchPropertiesTests.java |  119 +-
 .../netapp/generated/VolumePatchTests.java    |  151 +-
 .../VolumePropertiesDataProtectionTests.java  |   51 +-
 .../VolumePropertiesExportPolicyTests.java    |   47 +-
 .../generated/VolumeQuotaRuleInnerTests.java  |   28 +-
 .../generated/VolumeQuotaRulePatchTests.java  |   24 +-
 .../VolumeQuotaRulesCreateMockTests.java      |   22 +-
 .../VolumeQuotaRulesDeleteMockTests.java      |    2 +-
 ...umeQuotaRulesGetWithResponseMockTests.java |   16 +-
 ...VolumeQuotaRulesListByVolumeMockTests.java |   16 +-
 .../generated/VolumeQuotaRulesListTests.java  |   36 +-
 .../VolumeQuotaRulesPropertiesTests.java      |   18 +-
 .../VolumeRelocationPropertiesTests.java      |    8 +-
 .../netapp/generated/VolumeRevertTests.java   |    8 +-
 .../VolumeSnapshotPropertiesTests.java        |    8 +-
 .../VolumesAuthorizeReplicationMockTests.java |    4 +-
 .../VolumesBreakFileLocksMockTests.java       |    4 +-
 .../VolumesBreakReplicationMockTests.java     |    4 +-
 .../VolumesDeleteReplicationMockTests.java    |    2 +-
 .../VolumesFinalizeRelocationMockTests.java   |    3 +-
 ...istGetGroupIdListForLdapUserMockTests.java |    9 +-
 .../VolumesListReplicationsMockTests.java     |   12 +-
 .../generated/VolumesPoolChangeMockTests.java |    4 +-
 ...lumesReInitializeReplicationMockTests.java |    2 +-
 ...eplicationStatusWithResponseMockTests.java |   12 +-
 .../VolumesResyncReplicationMockTests.java    |    2 +-
 .../generated/VolumesRevertMockTests.java     |    4 +-
 .../VolumesRevertRelocationMockTests.java     |    2 +-
 .../netapp/generated/WeeklyScheduleTests.java |   26 +-
 sdk/netapp/tests.mgmt.yml                     |   24 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../CHANGELOG.md                              |   75 +-
 .../README.md                                 |   10 +-
 .../SAMPLE.md                                 |  955 +--
 .../pom.xml                                   |   12 +-
 .../NewRelicObservabilityManager.java         |  148 +-
 .../fluent/AccountsClient.java                |    8 +-
 .../fluent/BillingInfoesClient.java           |   43 +
 .../ConnectedPartnerResourcesClient.java      |   45 +
 .../fluent/MonitoredSubscriptionsClient.java  |  272 +
 .../fluent/MonitorsClient.java                |  185 +-
 .../fluent/NewRelicObservability.java         |   47 +-
 .../fluent/OperationsClient.java              |   16 +-
 .../fluent/OrganizationsClient.java           |    8 +-
 .../fluent/PlansClient.java                   |    8 +-
 .../fluent/TagRulesClient.java                |   72 +-
 .../fluent/models/AccountProperties.java      |   32 +-
 .../fluent/models/AccountResourceInner.java   |   34 +-
 .../fluent/models/AppServiceInfoInner.java    |   22 +-
 .../models/BillingInfoResponseInner.java      |   88 +
 ...nectedPartnerResourcesListFormatInner.java |   58 +
 .../fluent/models/LinkedResourceInner.java    |   54 +
 .../fluent/models/MetricRulesInner.java       |   22 +-
 .../models/MetricsStatusResponseInner.java    |   14 +-
 .../fluent/models/MonitorProperties.java      |   94 +-
 .../fluent/models/MonitoredResourceInner.java |   30 +-
 .../models/MonitoredSubscriptionInner.java    |  136 +
 .../MonitoredSubscriptionPropertiesInner.java |   61 +
 .../MonitoringTagRulesPropertiesInner.java    |   20 +-
 .../models/NewRelicMonitorResourceInner.java  |  114 +-
 ...wRelicMonitorResourceUpdateProperties.java |   38 +-
 .../fluent/models/OperationInner.java         |   22 +-
 .../fluent/models/OrganizationProperties.java |   22 +-
 .../models/OrganizationResourceInner.java     |   26 +-
 .../fluent/models/PlanDataProperties.java     |   22 +-
 .../fluent/models/PlanDataResourceInner.java  |   26 +-
 .../fluent/models/TagRuleInner.java           |   29 +-
 .../fluent/models/TagRuleUpdateInner.java     |   20 +-
 .../models/TagRuleUpdatePropertiesInner.java  |   18 +-
 .../models/VMExtensionPayloadInner.java       |   14 +-
 .../fluent/models/VMInfoInner.java            |   22 +-
 .../fluent/models/package-info.java           |    5 +-
 .../fluent/package-info.java                  |    5 +-
 .../implementation/AccountResourceImpl.java   |    3 +-
 .../implementation/AccountsClientImpl.java    |  191 +-
 .../implementation/AccountsImpl.java          |    7 +-
 .../implementation/AppServiceInfoImpl.java    |    3 +-
 .../BillingInfoResponseImpl.java              |   38 +
 .../BillingInfoesClientImpl.java              |  188 +
 .../implementation/BillingInfoesImpl.java     |   57 +
 .../ConnectedPartnerResourcesClientImpl.java  |  312 +
 .../ConnectedPartnerResourcesImpl.java        |   50 +
 ...nnectedPartnerResourcesListFormatImpl.java |   33 +
 .../implementation/LinkedResourceImpl.java    |   32 +
 .../implementation/MetricRulesImpl.java       |    3 +-
 .../MetricsStatusResponseImpl.java            |    3 +-
 .../implementation/MonitoredResourceImpl.java |    3 +-
 .../MonitoredSubscriptionImpl.java            |   52 +
 .../MonitoredSubscriptionPropertiesImpl.java  |  120 +
 .../MonitoredSubscriptionsClientImpl.java     | 1299 ++++
 .../MonitoredSubscriptionsImpl.java           |  169 +
 .../MonitoringTagRulesPropertiesImpl.java     |    3 +-
 .../implementation/MonitorsClientImpl.java    | 2076 +++---
 .../implementation/MonitorsImpl.java          |  193 +-
 .../NewRelicMonitorResourceImpl.java          |   88 +-
 .../NewRelicObservabilityBuilder.java         |   46 +-
 .../NewRelicObservabilityImpl.java            |  175 +-
 .../implementation/OperationImpl.java         |    3 +-
 .../implementation/OperationsClientImpl.java  |  167 +-
 .../implementation/OperationsImpl.java        |    7 +-
 .../OrganizationResourceImpl.java             |    3 +-
 .../OrganizationsClientImpl.java              |  198 +-
 .../implementation/OrganizationsImpl.java     |    7 +-
 .../implementation/PlanDataResourceImpl.java  |    3 +-
 .../implementation/PlansClientImpl.java       |  201 +-
 .../implementation/PlansImpl.java             |    7 +-
 .../implementation/ResourceManagerUtils.java  |  197 +
 .../implementation/TagRuleImpl.java           |   59 +-
 .../implementation/TagRuleUpdateImpl.java     |    3 +-
 .../TagRuleUpdatePropertiesImpl.java          |    3 +-
 .../implementation/TagRulesClientImpl.java    |  727 +-
 .../implementation/TagRulesImpl.java          |  132 +-
 .../implementation/Utils.java                 |  204 -
 .../VMExtensionPayloadImpl.java               |    3 +-
 .../implementation/VMInfoImpl.java            |    3 +-
 .../implementation/package-info.java          |    5 +-
 .../models/AccountCreationSource.java         |   18 +-
 .../models/AccountInfo.java                   |   28 +-
 .../models/AccountResource.java               |   24 +-
 .../models/Accounts.java                      |    8 +-
 .../models/AccountsListResponse.java          |   23 +-
 .../models/ActionType.java                    |   14 +-
 .../models/AppServiceInfo.java                |   12 +-
 .../models/AppServicesGetRequest.java         |   23 +-
 .../models/AppServicesListResponse.java       |   23 +-
 .../models/BillingCycle.java                  |   22 +-
 .../models/BillingInfoResponse.java           |   33 +
 .../models/BillingInfoes.java                 |   38 +
 .../models/BillingSource.java                 |   18 +-
 .../models/ConfigurationName.java             |   48 +
 .../ConnectedPartnerResourceProperties.java   |  132 +
 .../models/ConnectedPartnerResources.java     |   40 +
 .../ConnectedPartnerResourcesListFormat.java  |   27 +
 ...ConnectedPartnerResourcesListResponse.java |   85 +
 .../models/FilteringTag.java                  |   18 +-
 .../models/HostsGetRequest.java               |   23 +-
 .../models/LiftrResourceCategories.java       |   18 +-
 .../models/LinkedResource.java                |   26 +
 .../models/LinkedResourceListResponse.java    |   85 +
 .../models/LogRules.java                      |   38 +-
 .../models/ManagedServiceIdentity.java        |   47 +-
 .../models/ManagedServiceIdentityType.java    |   30 +-
 .../models/MarketplaceSaaSInfo.java           |  160 +
 .../models/MarketplaceSubscriptionStatus.java |   14 +-
 .../models/MetricRules.java                   |   12 +-
 .../models/MetricsRequest.java                |   19 +-
 .../models/MetricsStatusRequest.java          |   23 +-
 .../models/MetricsStatusResponse.java         |    8 +-
 .../models/MonitoredResource.java             |   16 +-
 .../models/MonitoredResourceListResponse.java |   24 +-
 .../models/MonitoredSubscription.java         |   47 +
 .../MonitoredSubscriptionProperties.java      |  187 +
 .../MonitoredSubscriptionPropertiesList.java  |   85 +
 .../models/MonitoredSubscriptions.java        |  148 +
 .../models/MonitoringStatus.java              |   18 +-
 .../models/MonitoringTagRulesProperties.java  |   12 +-
 .../models/Monitors.java                      |  139 +-
 .../models/MonitorsSwitchBillingHeaders.java  |   12 +-
 .../models/MonitorsSwitchBillingResponse.java |   16 +-
 .../models/NewRelicAccountProperties.java     |   26 +-
 .../models/NewRelicMonitorResource.java       |  329 +-
 .../NewRelicMonitorResourceListResult.java    |   24 +-
 .../models/NewRelicMonitorResourceUpdate.java |   44 +-
 .../NewRelicSingleSignOnProperties.java       |   26 +-
 .../models/Operation.java                     |   16 +-
 .../models/OperationDisplay.java              |   18 +-
 .../models/OperationListResult.java           |   14 +-
 .../models/Operations.java                    |   16 +-
 .../models/OrgCreationSource.java             |   18 +-
 .../models/OrganizationInfo.java              |   14 +-
 .../models/OrganizationResource.java          |   20 +-
 .../models/Organizations.java                 |    8 +-
 .../models/OrganizationsListResponse.java     |   23 +-
 .../newrelicobservability/models/Origin.java  |   18 +-
 .../models/PartnerBillingEntity.java          |   80 +
 .../models/PatchOperation.java                |   68 +
 .../models/PlanData.java                      |   26 +-
 .../models/PlanDataListResponse.java          |   23 +-
 .../models/PlanDataResource.java              |   20 +-
 .../newrelicobservability/models/Plans.java   |    8 +-
 .../models/ProvisioningState.java             |   46 +-
 .../models/SendAadLogsStatus.java             |   18 +-
 .../models/SendActivityLogsStatus.java        |   18 +-
 .../models/SendMetricsStatus.java             |   18 +-
 .../models/SendSubscriptionLogsStatus.java    |   18 +-
 .../models/SendingLogsStatus.java             |   18 +-
 .../models/SendingMetricsStatus.java          |   18 +-
 .../models/SingleSignOnStates.java            |   26 +-
 .../newrelicobservability/models/Status.java  |   63 +
 .../models/SubscriptionList.java              |  100 +
 .../models/SwitchBillingRequest.java          |   31 +-
 .../models/TagAction.java                     |   18 +-
 .../newrelicobservability/models/TagRule.java |   95 +-
 .../models/TagRuleListResult.java             |   23 +-
 .../models/TagRuleUpdate.java                 |   10 +-
 .../models/TagRuleUpdateProperties.java       |   10 +-
 .../models/TagRules.java                      |   30 +-
 .../models/UsageType.java                     |   18 +-
 .../models/UserAssignedIdentity.java          |   14 +-
 .../models/UserInfo.java                      |   30 +-
 .../models/VMExtensionPayload.java            |    8 +-
 .../models/VMHostsListResponse.java           |   23 +-
 .../newrelicobservability/models/VMInfo.java  |   12 +-
 .../models/package-info.java                  |    5 +-
 .../newrelicobservability/package-info.java   |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       |  411 ++
 .../generated/AccountsListSamples.java        |   14 +-
 .../generated/BillingInfoGetSamples.java      |   24 +
 .../ConnectedPartnerResourcesListSamples.java |   25 +
 ...redSubscriptionsCreateorUpdateSamples.java |   27 +
 .../MonitoredSubscriptionsDeleteSamples.java  |   27 +
 .../MonitoredSubscriptionsGetSamples.java     |   27 +
 .../MonitoredSubscriptionsListSamples.java    |   24 +
 .../MonitoredSubscriptionsUpdateSamples.java  |   29 +
 .../MonitorsCreateOrUpdateSamples.java        |   68 +-
 .../generated/MonitorsDeleteSamples.java      |   32 +-
 .../MonitorsGetByResourceGroupSamples.java    |   13 +-
 .../MonitorsGetMetricRulesSamples.java        |   32 +-
 .../MonitorsGetMetricStatusSamples.java       |   40 +-
 .../MonitorsListAppServicesSamples.java       |   40 +-
 .../MonitorsListByResourceGroupSamples.java   |    9 +-
 .../generated/MonitorsListHostsSamples.java   |   40 +-
 .../MonitorsListLinkedResourcesSamples.java   |   25 +
 ...MonitorsListMonitoredResourcesSamples.java |   24 +-
 .../generated/MonitorsListSamples.java        |    9 +-
 .../MonitorsSwitchBillingSamples.java         |   47 +-
 .../generated/MonitorsUpdateSamples.java      |   70 +-
 .../MonitorsVmHostPayloadSamples.java         |   24 +-
 .../generated/OperationsListSamples.java      |   14 +-
 .../generated/OrganizationsListSamples.java   |   14 +-
 .../generated/PlansListSamples.java           |   24 +-
 .../TagRulesCreateOrUpdateSamples.java        |   56 +-
 .../generated/TagRulesDeleteSamples.java      |   32 +-
 .../generated/TagRulesGetSamples.java         |   40 +-
 ...sListByNewRelicMonitorResourceSamples.java |   26 +-
 .../generated/TagRulesUpdateSamples.java      |   70 +-
 .../generated/AccountPropertiesTests.java     |   16 +-
 .../generated/AccountResourceInnerTests.java  |   16 +-
 .../generated/AccountsListMockTests.java      |   51 +-
 .../generated/AccountsListResponseTests.java  |   22 +-
 .../generated/AppServiceInfoInnerTests.java   |   27 +-
 .../generated/AppServicesGetRequestTests.java |   17 +-
 .../AppServicesListResponseTests.java         |   47 +-
 .../BillingInfoResponseInnerTests.java        |   46 +
 ...BillingInfoesGetWithResponseMockTests.java |   62 +
 ...nnectedPartnerResourcePropertiesTests.java |   33 +
 ...dPartnerResourcesListFormatInnerTests.java |   35 +
 ...onnectedPartnerResourcesListMockTests.java |   60 +
 ...ctedPartnerResourcesListResponseTests.java |   48 +
 .../generated/FilteringTagTests.java          |   21 +-
 .../generated/HostsGetRequestTests.java       |   18 +-
 .../generated/LinkedResourceInnerTests.java   |   24 +
 .../LinkedResourceListResponseTests.java      |   33 +
 .../generated/LogRulesTests.java              |   57 +-
 .../ManagedServiceIdentityTests.java          |   17 +-
 .../generated/MarketplaceSaaSInfoTests.java   |   36 +
 .../generated/MetricRulesInnerTests.java      |   40 +-
 .../generated/MetricsRequestTests.java        |    9 +-
 .../generated/MetricsStatusRequestTests.java  |   20 +-
 .../MetricsStatusResponseInnerTests.java      |   11 +-
 .../MonitoredResourceInnerTests.java          |   34 +-
 .../MonitoredResourceListResponseTests.java   |   66 +-
 .../MonitoredSubscriptionInnerTests.java      |   85 +
 ...toredSubscriptionPropertiesInnerTests.java |  101 +
 ...itoredSubscriptionPropertiesListTests.java |   85 +
 ...dSubscriptionsCreateorUpdateMockTests.java |   90 +
 ...SubscriptionsGetWithResponseMockTests.java |   62 +
 .../MonitoredSubscriptionsListMockTests.java  |   65 +
 ...onitoringTagRulesPropertiesInnerTests.java |   92 +-
 .../MonitorsCreateOrUpdateMockTests.java      |  136 -
 .../generated/MonitorsDeleteMockTests.java    |   36 +-
 ...tByResourceGroupWithResponseMockTests.java |   89 -
 ...rsGetMetricRulesWithResponseMockTests.java |   52 +-
 ...sGetMetricStatusWithResponseMockTests.java |   55 +-
 .../MonitorsListAppServicesMockTests.java     |   59 +-
 .../MonitorsListByResourceGroupMockTests.java |   89 -
 .../generated/MonitorsListHostsMockTests.java |   56 +-
 .../MonitorsListLinkedResourcesMockTests.java |   56 +
 .../generated/MonitorsListMockTests.java      |   88 -
 ...nitorsListMonitoredResourcesMockTests.java |   45 +-
 ...orsSwitchBillingWithResponseMockTests.java |  104 -
 .../NewRelicMonitorResourceInnerTests.java    |  112 -
 ...ewRelicMonitorResourceListResultTests.java |  105 -
 .../NewRelicMonitorResourceUpdateTests.java   |  101 -
 .../NewRelicSingleSignOnPropertiesTests.java  |   33 +-
 .../generated/OperationDisplayTests.java      |    8 +-
 .../generated/OperationInnerTests.java        |    8 +-
 .../generated/OperationListResultTests.java   |    8 +-
 .../generated/OperationsListMockTests.java    |   34 +-
 .../generated/OrganizationInfoTests.java      |    9 +-
 .../OrganizationPropertiesTests.java          |   23 +-
 .../OrganizationResourceInnerTests.java       |   27 +-
 .../generated/OrganizationsListMockTests.java |   43 +-
 .../OrganizationsListResponseTests.java       |   46 +-
 .../generated/PartnerBillingEntityTests.java  |   29 +
 .../generated/PlanDataListResponseTests.java  |   48 +-
 .../generated/PlanDataPropertiesTests.java    |   42 +-
 .../generated/PlanDataResourceInnerTests.java |   34 +-
 .../generated/PlanDataTests.java              |   32 +-
 .../generated/PlansListMockTests.java         |   48 +-
 .../generated/SubscriptionListTests.java      |   99 +
 .../generated/SwitchBillingRequestTests.java  |   48 +-
 .../generated/TagRuleInnerTests.java          |   62 +-
 .../generated/TagRuleListResultTests.java     |   50 +-
 .../generated/TagRuleUpdateInnerTests.java    |   57 +-
 .../TagRuleUpdatePropertiesInnerTests.java    |   87 +-
 .../TagRulesCreateOrUpdateMockTests.java      |   80 +-
 .../generated/TagRulesDeleteMockTests.java    |   36 +-
 .../TagRulesGetWithResponseMockTests.java     |   53 +-
 ...istByNewRelicMonitorResourceMockTests.java |   62 +-
 .../generated/UserAssignedIdentityTests.java  |    8 +-
 .../generated/UserInfoTests.java              |   37 +-
 .../generated/VMHostsListResponseTests.java   |   38 +-
 .../generated/VMInfoInnerTests.java           |   22 +-
 sdk/nginx/azure-resourcemanager-nginx/pom.xml |    6 +-
 .../CHANGELOG.md                              |  368 +-
 .../README.md                                 |   31 +-
 .../SAMPLE.md                                 |  905 ++-
 .../pom.xml                                   |   61 +-
 .../NotificationHubsManager.java              |  181 +-
 .../fluent/NamespacesClient.java              |  456 +-
 .../fluent/NotificationHubsClient.java        |  508 +-
 ...ent.java => NotificationHubsRPClient.java} |   44 +-
 .../fluent/OperationsClient.java              |   18 +-
 .../PrivateEndpointConnectionsClient.java     |  306 +
 .../models/AdmCredentialProperties.java       |   59 +-
 .../models/ApnsCredentialProperties.java      |  136 +-
 .../models/BaiduCredentialProperties.java     |   59 +-
 .../models/BrowserCredentialProperties.java   |  121 +
 .../models/CheckAvailabilityResultInner.java  |  126 +-
 .../fluent/models/DebugSendResponseInner.java |  158 +-
 .../fluent/models/DebugSendResult.java        |   90 +-
 .../models/FcmV1CredentialProperties.java     |  121 +
 .../models/GcmCredentialProperties.java       |   42 +-
 .../models/MpnsCredentialProperties.java      |   59 +-
 .../fluent/models/NamespaceProperties.java    |  433 +-
 .../fluent/models/NamespaceResourceInner.java |  379 +-
 .../models/NotificationHubProperties.java     |  221 +-
 .../models/NotificationHubResourceInner.java  |  220 +-
 .../fluent/models/OperationInner.java         |   72 +-
 .../fluent/models/PnsCredentials.java         |  298 +
 .../models/PnsCredentialsProperties.java      |  204 -
 .../models/PnsCredentialsResourceInner.java   |  276 +-
 ...rivateEndpointConnectionResourceInner.java |   75 +
 .../models/PrivateLinkResourceInner.java      |   75 +
 .../fluent/models/ResourceListKeysInner.java  |  115 +-
 ...aredAccessAuthorizationRuleProperties.java |  156 +-
 ...dAccessAuthorizationRuleResourceInner.java |  199 +-
 .../models/WnsCredentialProperties.java       |   98 +-
 .../models/XiaomiCredentialProperties.java    |   80 +
 .../fluent/models/package-info.java           |    5 +-
 .../notificationhubs/fluent/package-info.java |    5 +-
 .../CheckAvailabilityResultImpl.java          |   16 +-
 .../implementation/DebugSendResponseImpl.java |   24 +-
 .../implementation/NamespaceResourceImpl.java |  231 +-
 .../implementation/NamespacesClientImpl.java  | 2940 ++++----
 .../implementation/NamespacesImpl.java        |  439 +-
 .../NotificationHubResourceImpl.java          |  195 +-
 .../NotificationHubsClientImpl.java           | 3037 +++-----
 .../implementation/NotificationHubsImpl.java  |  555 +-
 ...tificationHubsManagementClientBuilder.java |  149 -
 .../NotificationHubsRPClientBuilder.java      |  136 +
 ...java => NotificationHubsRPClientImpl.java} |  175 +-
 .../implementation/OperationImpl.java         |   13 +-
 .../implementation/OperationsClientImpl.java  |  194 +-
 .../implementation/OperationsImpl.java        |   10 +-
 .../PnsCredentialsResourceImpl.java           |   36 +-
 ...PrivateEndpointConnectionResourceImpl.java |   50 +
 .../PrivateEndpointConnectionsClientImpl.java | 1288 ++++
 .../PrivateEndpointConnectionsImpl.java       |  143 +
 .../PrivateLinkResourceImpl.java              |   50 +
 .../implementation/ResourceListKeysImpl.java  |    3 +-
 .../implementation/ResourceManagerUtils.java  |  197 +
 ...edAccessAuthorizationRuleResourceImpl.java |  185 +-
 .../implementation/Utils.java                 |  204 -
 .../implementation/package-info.java          |    5 +-
 .../notificationhubs/models/AccessRights.java |   68 +-
 .../models/AdmCredential.java                 |   56 +-
 .../models/ApnsCredential.java                |  116 +-
 .../notificationhubs/models/Availability.java |   58 +
 .../models/BaiduCredential.java               |   56 +-
 .../models/BrowserCredential.java             |  122 +
 .../models/CheckAvailabilityParameters.java   |  119 +-
 .../models/CheckAvailabilityResult.java       |   46 +-
 .../models/DebugSendResponse.java             |   48 +-
 .../models/FcmV1Credential.java               |  122 +
 .../models/GcmCredential.java                 |   48 +-
 .../notificationhubs/models/IpRule.java       |   92 +
 .../models/LogSpecification.java              |  100 +
 .../models/MetricSpecification.java           |  171 +
 .../models/MpnsCredential.java                |   56 +-
 .../NamespaceCreateOrUpdateParameters.java    |  403 --
 .../models/NamespaceListResult.java           |   59 +-
 .../models/NamespacePatchParameters.java      |   94 +-
 .../models/NamespaceResource.java             |  537 +-
 .../models/NamespaceStatus.java               |   63 +
 .../models/NamespaceType.java                 |   62 +-
 .../notificationhubs/models/Namespaces.java   |  376 +-
 .../notificationhubs/models/NetworkAcls.java  |   89 +
 ...tificationHubCreateOrUpdateParameters.java |  303 -
 .../models/NotificationHubListResult.java     |   59 +-
 .../NotificationHubPatchParameters.java       |  230 +-
 .../models/NotificationHubResource.java       |  552 +-
 .../models/NotificationHubs.java              |  473 +-
 .../notificationhubs/models/Operation.java    |   27 +-
 .../models/OperationDisplay.java              |   49 +-
 .../models/OperationListResult.java           |   32 +-
 .../models/OperationProperties.java           |   57 +
 .../models/OperationProvisioningState.java    |   78 +
 .../notificationhubs/models/Operations.java   |   18 +-
 .../models/PnsCredentialsResource.java        |   86 +-
 .../models/PolicyKeyResource.java             |   61 +
 .../models/PolicyKeyType.java                 |   53 +
 .../models/PolicykeyResource.java             |   54 -
 .../PrivateEndpointConnectionProperties.java  |  134 +
 ...teEndpointConnectionProvisioningState.java |   84 +
 .../PrivateEndpointConnectionResource.java    |   56 +
 ...eEndpointConnectionResourceListResult.java |   63 +
 .../models/PrivateEndpointConnections.java    |  216 +
 .../models/PrivateLinkConnectionStatus.java   |   63 +
 .../models/PrivateLinkResource.java           |   55 +
 .../models/PrivateLinkResourceListResult.java |   63 +
 .../models/PrivateLinkResourceProperties.java |   77 +
 .../PublicInternetAuthorizationRule.java      |   62 +
 .../models/PublicNetworkAccess.java           |   53 +
 .../models/RegistrationResult.java            |   88 +
 .../RemotePrivateEndpointConnection.java      |   45 +
 ...motePrivateLinkServiceConnectionState.java |   84 +
 .../models/ReplicationRegion.java             |   83 +
 .../models/ResourceListKeys.java              |   27 +-
 .../models/ServiceSpecification.java          |   65 +
 ...orizationRuleCreateOrUpdateParameters.java |   62 -
 ...aredAccessAuthorizationRuleListResult.java |   59 +-
 ...SharedAccessAuthorizationRuleResource.java |  350 +-
 .../notificationhubs/models/Sku.java          |   69 +-
 .../notificationhubs/models/SkuName.java      |   33 +-
 .../models/WnsCredential.java                 |  102 +-
 .../models/XiaomiCredential.java              |   99 +
 .../models/ZoneRedundancyPreference.java      |   53 +
 .../notificationhubs/models/package-info.java |    5 +-
 .../notificationhubs/package-info.java        |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       |  346 +
 .../NamespacesCheckAvailabilitySamples.java   |   23 +-
 ...reateOrUpdateAuthorizationRuleSamples.java |   29 +-
 .../NamespacesCreateOrUpdateSamples.java      |   35 +-
 ...espacesDeleteAuthorizationRuleSamples.java |   23 +-
 .../generated/NamespacesDeleteSamples.java    |   19 +-
 ...NamespacesGetAuthorizationRuleSamples.java |   23 +-
 .../NamespacesGetByResourceGroupSamples.java  |   18 +-
 .../NamespacesGetPnsCredentialsSamples.java   |   25 +
 ...mespacesListAuthorizationRulesSamples.java |   20 +-
 .../NamespacesListByResourceGroupSamples.java |   19 +-
 .../generated/NamespacesListKeysSamples.java  |   20 +-
 .../generated/NamespacesListSamples.java      |   18 +-
 .../generated/NamespacesPatchSamples.java     |   44 -
 .../NamespacesRegenerateKeysSamples.java      |   30 +-
 .../generated/NamespacesUpdateSamples.java    |   51 +
 ...eckNotificationHubAvailabilitySamples.java |   25 +-
 ...reateOrUpdateAuthorizationRuleSamples.java |   32 +-
 ...NotificationHubsCreateOrUpdateSamples.java |   24 +-
 .../NotificationHubsDebugSendSamples.java     |   33 +-
 ...ionHubsDeleteAuthorizationRuleSamples.java |   22 +-
 .../NotificationHubsDeleteSamples.java        |   21 +-
 ...cationHubsGetAuthorizationRuleSamples.java |   22 +-
 ...ificationHubsGetPnsCredentialsSamples.java |   21 +-
 .../generated/NotificationHubsGetSamples.java |   19 +-
 ...tionHubsListAuthorizationRulesSamples.java |   19 +-
 .../NotificationHubsListKeysSamples.java      |   23 +-
 .../NotificationHubsListSamples.java          |   20 +-
 .../NotificationHubsPatchSamples.java         |   29 -
 ...NotificationHubsRegenerateKeysSamples.java |   32 +-
 .../NotificationHubsUpdateSamples.java        |   33 +
 .../generated/OperationsListSamples.java      |   16 +-
 ...ivateEndpointConnectionsDeleteSamples.java |   26 +
 ...eEndpointConnectionsGetGroupIdSamples.java |   26 +
 .../PrivateEndpointConnectionsGetSamples.java |   26 +
 ...ndpointConnectionsListGroupIdsSamples.java |   25 +
 ...PrivateEndpointConnectionsListSamples.java |   25 +
 ...ivateEndpointConnectionsUpdateSamples.java |   36 +
 .../generated/AvailabilityTests.java          |   22 +
 .../CheckAvailabilityParametersTests.java     |   61 +
 .../CheckAvailabilityResultInnerTests.java    |   61 +
 .../DebugSendResponseInnerTests.java          |   43 +
 .../generated/DebugSendResultTests.java       |   23 +
 .../generated/IpRuleTests.java                |   30 +
 .../generated/LogSpecificationTests.java      |   27 +
 .../generated/MetricSpecificationTests.java   |   23 +
 ...heckAvailabilityWithResponseMockTests.java |   86 +
 ...uthorizationRuleWithResponseMockTests.java |   52 +
 ...eByResourceGroupWithResponseMockTests.java |   52 +
 .../generated/NetworkAclsTests.java           |   40 +
 ...nHubAvailabilityWithResponseMockTests.java |   86 +
 ...ionHubsDebugSendWithResponseMockTests.java |   57 +
 ...uthorizationRuleWithResponseMockTests.java |   52 +
 ...ationHubsDeleteWithResponseMockTests.java} |   11 +-
 .../generated/OperationDisplayTests.java      |   23 +
 .../generated/OperationInnerTests.java        |   27 +
 .../generated/OperationListResultTests.java   |   23 +
 .../generated/OperationPropertiesTests.java   |   24 +
 .../generated/OperationsListMockTests.java    |   54 +
 ...vateEndpointConnectionPropertiesTests.java |   37 +
 ...eEndpointConnectionResourceInnerTests.java |   41 +
 ...ointConnectionResourceListResultTests.java |   23 +
 ...tionsGetGroupIdWithResponseMockTests.java} |   19 +-
 ...ntConnectionsGetWithResponseMockTests.java |   61 +
 ...ointConnectionsListGroupIdsMockTests.java} |   20 +-
 ...ivateEndpointConnectionsListMockTests.java |   62 +
 ...ateEndpointConnectionsUpdateMockTests.java |   70 +
 .../PrivateLinkResourceInnerTests.java        |   25 +
 .../PrivateLinkResourceListResultTests.java   |   23 +
 .../PrivateLinkResourcePropertiesTests.java   |   23 +
 .../PublicInternetAuthorizationRuleTests.java |   28 +
 .../generated/RegistrationResultTests.java    |   23 +
 .../RemotePrivateEndpointConnectionTests.java |   22 +
 ...rivateLinkServiceConnectionStateTests.java |   28 +
 .../generated/ServiceSpecificationTests.java  |   23 +
 .../notificationhubs/generated/SkuTests.java  |   37 +
 .../org.mockito.plugins.MockMaker             |    1 +
 sdk/openai/azure-ai-openai-assistants/pom.xml |    2 +-
 sdk/openai/azure-ai-openai/CHANGELOG.md       |    3 +
 sdk/openai/azure-ai-openai/README.md          |   32 +-
 sdk/openai/azure-ai-openai/assets.json        |    2 +-
 sdk/openai/azure-ai-openai/pom.xml            |    2 +-
 .../azure/ai/openai/OpenAIAsyncClient.java    |   71 +-
 .../com/azure/ai/openai/OpenAIClient.java     |   62 +-
 .../azure/ai/openai/StreamingChatSample.java  |    9 +-
 .../OpenAIAsyncClientJavaDocCodeSnippets.java |   40 +-
 .../impl/OpenAIClientJavaDocCodeSnippets.java |   33 +-
 .../azure/ai/openai/impl/ReadmeSamples.java   |   33 +-
 .../GetChatCompletionsStreamAsyncSample.java  |   19 +-
 .../usage/GetChatCompletionsStreamSample.java |   15 +-
 .../openai/NonAzureOpenAIAsyncClientTest.java |   18 +
 .../openai/NonAzureOpenAISyncClientTest.java  |   14 +
 .../ai/openai/OpenAIAsyncClientTest.java      |   19 +-
 .../azure/ai/openai/OpenAIClientTestBase.java |   42 +
 .../azure/ai/openai/OpenAISyncClientTest.java |   15 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-orbital/pom.xml     |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/paloaltonetworks/tests.mgmt.yml           |   24 +-
 .../azure-resourcemanager-peering/pom.xml     |    6 +-
 .../azure-ai-personalizer/pom.xml             |    2 +-
 sdk/personalizer/tests.yml                    |   30 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-postgresql/pom.xml  |    4 +-
 sdk/postgresql/tests.mgmt.yml                 |   24 +-
 .../pom.xml                                   |    8 +-
 sdk/postgresqlflexibleserver/tests.mgmt.yml   |   24 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-providerhub/pom.xml |    4 +-
 sdk/providerhub/tests.mgmt.yml                |   24 +-
 .../pom.xml                                   |    2 +-
 .../azure-analytics-purview-catalog/pom.xml   |    2 +-
 .../azure-analytics-purview-datamap/pom.xml   |    2 +-
 .../azure-analytics-purview-scanning/pom.xml  |    2 +-
 .../azure-analytics-purview-sharing/pom.xml   |    2 +-
 .../azure-analytics-purview-workflow/pom.xml  |    2 +-
 .../azure-resourcemanager-purview/pom.xml     |    6 +-
 sdk/quantum/azure-quantum-jobs/pom.xml        |    4 +-
 .../CHANGELOG.md                              |   31 +-
 .../azure-resourcemanager-quantum/README.md   |    8 +-
 .../azure-resourcemanager-quantum/SAMPLE.md   |  187 +-
 .../azure-resourcemanager-quantum/pom.xml     |   13 +-
 .../quantum/AzureQuantumManager.java          |   90 +-
 .../fluent/AzureQuantumManagementClient.java  |   24 +-
 .../quantum/fluent/OfferingsClient.java       |    8 +-
 .../quantum/fluent/OperationsClient.java      |    8 +-
 .../fluent/WorkspaceOperationsClient.java     |   78 +-
 .../quantum/fluent/WorkspacesClient.java      |   88 +-
 .../CheckNameAvailabilityResultInner.java     |   20 +-
 .../fluent/models/ListKeysResultInner.java    |  143 +
 .../quantum/fluent/models/OperationInner.java |   22 +-
 .../models/ProviderDescriptionInner.java      |   26 +-
 .../fluent/models/QuantumWorkspaceInner.java  |   67 +-
 .../models/WorkspaceResourceProperties.java   |   50 +-
 .../quantum/fluent/models/package-info.java   |    5 +-
 .../quantum/fluent/package-info.java          |    5 +-
 .../AzureQuantumManagementClientBuilder.java  |   50 +-
 .../AzureQuantumManagementClientImpl.java     |  119 +-
 .../CheckNameAvailabilityResultImpl.java      |    3 +-
 .../implementation/ListKeysResultImpl.java    |   49 +
 .../implementation/OfferingsClientImpl.java   |  188 +-
 .../quantum/implementation/OfferingsImpl.java |    8 +-
 .../implementation/OperationsClientImpl.java  |  143 +-
 .../implementation/OperationsImpl.java        |    8 +-
 .../ProviderDescriptionImpl.java              |    4 +-
 .../implementation/QuantumWorkspaceImpl.java  |   63 +-
 .../implementation/ResourceManagerUtils.java  |  197 +
 .../quantum/implementation/Utils.java         |  204 -
 .../WorkspaceOperationsClientImpl.java        |  406 +-
 .../WorkspaceOperationsImpl.java              |   61 +-
 .../implementation/WorkspacesClientImpl.java  |  890 +--
 .../implementation/WorkspacesImpl.java        |   97 +-
 .../quantum/implementation/package-info.java  |    5 +-
 .../quantum/models/ApiKey.java                |   70 +
 .../quantum/models/ApiKeys.java               |   55 +
 .../CheckNameAvailabilityParameters.java      |   18 +-
 .../models/CheckNameAvailabilityResult.java   |   12 +-
 .../quantum/models/KeyType.java               |   53 +
 .../quantum/models/ListKeysResult.java        |   54 +
 .../quantum/models/Offerings.java             |    8 +-
 .../quantum/models/OfferingsListResult.java   |   18 +-
 .../quantum/models/Operation.java             |   12 +-
 .../quantum/models/OperationDisplay.java      |   26 +-
 .../quantum/models/Operations.java            |    8 +-
 .../quantum/models/OperationsList.java        |   23 +-
 .../quantum/models/PricingDetail.java         |   18 +-
 .../quantum/models/PricingDimension.java      |   18 +-
 .../quantum/models/Provider.java              |   34 +-
 .../quantum/models/ProviderDescription.java   |   14 +-
 .../quantum/models/ProviderProperties.java    |   42 +-
 .../quantum/models/ProviderPropertiesAad.java |   14 +-
 .../ProviderPropertiesManagedApplication.java |   14 +-
 .../quantum/models/ProvisioningStatus.java    |   34 +-
 .../quantum/models/QuantumWorkspace.java      |  152 +-
 .../models/QuantumWorkspaceIdentity.java      |   18 +-
 .../quantum/models/QuotaDimension.java        |   42 +-
 .../quantum/models/ResourceIdentityType.java  |   18 +-
 .../quantum/models/SkuDescription.java        |   46 +-
 .../quantum/models/Status.java                |   34 +-
 .../quantum/models/TagsObject.java            |   14 +-
 .../quantum/models/TargetDescription.java     |   30 +-
 .../quantum/models/UsableStatus.java          |   22 +-
 .../quantum/models/WorkspaceListResult.java   |   18 +-
 .../quantum/models/WorkspaceOperations.java   |   72 +-
 .../quantum/models/Workspaces.java            |   46 +-
 .../quantum/models/package-info.java          |    5 +-
 .../resourcemanager/quantum/package-info.java |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       |  146 +
 .../generated/OfferingsListSamples.java       |    9 +-
 .../generated/OperationsListSamples.java      |    9 +-
 ...OperationCheckNameAvailabilitySamples.java |   23 +-
 .../WorkspaceOperationListKeysSamples.java    |   24 +
 ...rkspaceOperationRegenerateKeysSamples.java |   29 +
 .../WorkspacesCreateOrUpdateSamples.java      |   24 +-
 .../generated/WorkspacesDeleteSamples.java    |    9 +-
 .../WorkspacesGetByResourceGroupSamples.java  |   16 +-
 .../WorkspacesListByResourceGroupSamples.java |   13 +-
 .../generated/WorkspacesListSamples.java      |   13 +-
 .../WorkspacesUpdateTagsSamples.java          |   18 +-
 .../CheckNameAvailabilityParametersTests.java |   17 +-
 ...CheckNameAvailabilityResultInnerTests.java |   15 +-
 .../generated/OfferingsListMockTests.java     |   94 +-
 .../generated/OfferingsListResultTests.java   |  390 +-
 .../generated/OperationDisplayTests.java      |   32 +-
 .../generated/OperationInnerTests.java        |   45 +-
 .../generated/OperationsListMockTests.java    |   43 +-
 .../generated/OperationsListTests.java        |   59 +-
 .../quantum/generated/PricingDetailTests.java |   13 +-
 .../generated/PricingDimensionTests.java      |   14 +-
 .../ProviderDescriptionInnerTests.java        |  332 +-
 .../generated/ProviderPropertiesAadTests.java |    6 +-
 ...iderPropertiesManagedApplicationTests.java |    5 +-
 .../generated/ProviderPropertiesTests.java    |  300 +-
 .../quantum/generated/ProviderTests.java      |   38 +-
 .../QuantumWorkspaceIdentityTests.java        |    5 +-
 .../generated/QuantumWorkspaceInnerTests.java |   79 -
 .../generated/QuotaDimensionTests.java        |   53 +-
 .../generated/SkuDescriptionTests.java        |  108 +-
 .../quantum/generated/TagsObjectTests.java    |   13 +-
 .../generated/TargetDescriptionTests.java     |   38 +-
 .../generated/WorkspaceListResultTests.java   |  154 -
 ...NameAvailabilityWithResponseMockTests.java |   46 +-
 .../WorkspaceResourcePropertiesTests.java     |   62 -
 .../WorkspacesCreateOrUpdateMockTests.java    |  133 -
 .../generated/WorkspacesDeleteMockTests.java  |   36 +-
 ...tByResourceGroupWithResponseMockTests.java |   81 -
 ...orkspacesListByResourceGroupMockTests.java |   79 -
 .../generated/WorkspacesListMockTests.java    |   78 -
 .../azure-resourcemanager-qumulo/pom.xml      |    6 +-
 sdk/quota/azure-resourcemanager-quota/pom.xml |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/recoveryservices/tests.mgmt.yml           |   24 +-
 .../pom.xml                                   |    8 +-
 sdk/recoveryservicesbackup/tests.mgmt.yml     |   24 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../CHANGELOG.md                              |  119 +-
 .../README.md                                 |   20 +-
 .../SAMPLE.md                                 |  492 +-
 .../pom.xml                                   |   25 +-
 .../RedisEnterpriseManager.java               |  115 +-
 .../fluent/DatabasesClient.java               |  218 +-
 .../fluent/OperationsClient.java              |   16 +-
 .../fluent/OperationsStatusClient.java        |    8 +-
 .../PrivateEndpointConnectionsClient.java     |  121 +-
 .../fluent/PrivateLinkResourcesClient.java    |   16 +-
 .../RedisEnterpriseManagementClient.java      |   33 +-
 .../fluent/RedisEnterprisesClient.java        |   68 +-
 .../redisenterprise/fluent/SkusClient.java    |   41 -
 .../fluent/models/AccessKeysInner.java        |   14 +-
 .../fluent/models/ClusterInner.java           |   71 +-
 .../fluent/models/ClusterProperties.java      |   28 +-
 .../fluent/models/DatabaseInner.java          |   72 +-
 .../fluent/models/DatabaseProperties.java     |   50 +-
 .../fluent/models/OperationInner.java         |   22 +-
 .../fluent/models/OperationStatusInner.java   |   44 +-
 .../PrivateEndpointConnectionInner.java       |   48 +-
 .../PrivateEndpointConnectionProperties.java  |   39 +-
 .../models/PrivateLinkResourceInner.java      |   34 +-
 .../models/PrivateLinkResourceProperties.java |   18 +-
 .../fluent/models/RegionSkuDetailInner.java   |  120 -
 .../fluent/models/package-info.java           |    4 +-
 .../redisenterprise/fluent/package-info.java  |    4 +-
 .../implementation/AccessKeysImpl.java        |    4 +-
 .../implementation/ClusterImpl.java           |   65 +-
 .../implementation/DatabaseImpl.java          |   64 +-
 .../implementation/DatabasesClientImpl.java   | 1751 ++---
 .../implementation/DatabasesImpl.java         |  204 +-
 .../implementation/OperationImpl.java         |    4 +-
 .../implementation/OperationStatusImpl.java   |    3 +-
 .../implementation/OperationsClientImpl.java  |  167 +-
 .../implementation/OperationsImpl.java        |    8 +-
 .../OperationsStatusClientImpl.java           |   99 +-
 .../implementation/OperationsStatusImpl.java  |    8 +-
 .../PrivateEndpointConnectionImpl.java        |   67 +-
 .../PrivateEndpointConnectionsClientImpl.java |  748 +-
 .../PrivateEndpointConnectionsImpl.java       |  179 +-
 .../PrivateLinkResourceImpl.java              |    8 +-
 .../PrivateLinkResourcesClientImpl.java       |  145 +-
 .../PrivateLinkResourcesImpl.java             |   19 +-
 ...edisEnterpriseManagementClientBuilder.java |   46 +-
 .../RedisEnterpriseManagementClientImpl.java  |  141 +-
 .../RedisEnterprisesClientImpl.java           |  878 +--
 .../implementation/RedisEnterprisesImpl.java  |   96 +-
 .../implementation/RegionSkuDetailImpl.java   |   43 -
 .../implementation/ResourceManagerUtils.java  |  197 +
 .../implementation/SkusClientImpl.java        |  222 -
 .../implementation/SkusImpl.java              |   45 -
 .../redisenterprise/implementation/Utils.java |  204 -
 .../implementation/package-info.java          |    4 +-
 .../redisenterprise/models/AccessKeyType.java |   22 +-
 .../redisenterprise/models/AccessKeys.java    |   10 +-
 .../redisenterprise/models/ActionType.java    |   14 +-
 .../redisenterprise/models/AofFrequency.java  |   18 +-
 .../redisenterprise/models/Cluster.java       |  205 +-
 .../redisenterprise/models/ClusterList.java   |   16 +-
 .../models/ClusterPropertiesEncryption.java   |   14 +-
 ...ncryptionCustomerManagedKeyEncryption.java |   18 +-
 ...stomerManagedKeyEncryptionKeyIdentity.java |   22 +-
 .../redisenterprise/models/ClusterUpdate.java |   48 +-
 .../models/ClusteringPolicy.java              |   18 +-
 .../models/CmkIdentityType.java               |   18 +-
 .../redisenterprise/models/Database.java      |  196 +-
 .../redisenterprise/models/DatabaseList.java  |   16 +-
 .../DatabasePropertiesGeoReplication.java     |   18 +-
 .../models/DatabaseUpdate.java                |   58 +-
 .../redisenterprise/models/Databases.java     |  104 +-
 .../models/EvictionPolicy.java                |   42 +-
 .../models/ExportClusterParameters.java       |   21 +-
 .../models/FlushParameters.java               |   26 +-
 .../models/ForceUnlinkParameters.java         |   19 +-
 .../models/ImportClusterParameters.java       |   21 +-
 .../redisenterprise/models/LinkState.java     |   30 +-
 .../models/LinkedDatabase.java                |   16 +-
 .../redisenterprise/models/LocationInfo.java  |   80 -
 .../models/ManagedServiceIdentity.java        |   47 +-
 .../models/ManagedServiceIdentityType.java    |   30 +-
 .../redisenterprise/models/Module.java        |   20 +-
 .../redisenterprise/models/Operation.java     |   16 +-
 .../models/OperationDisplay.java              |   18 +-
 .../models/OperationListResult.java           |   14 +-
 .../models/OperationStatus.java               |   22 +-
 .../redisenterprise/models/Operations.java    |   16 +-
 .../models/OperationsStatus.java              |    8 +-
 .../redisenterprise/models/Origin.java        |   18 +-
 .../redisenterprise/models/Persistence.java   |   26 +-
 .../models/PrivateEndpoint.java               |   12 +-
 .../models/PrivateEndpointConnection.java     |  101 +-
 .../PrivateEndpointConnectionListResult.java  |   14 +-
 ...teEndpointConnectionProvisioningState.java |   26 +-
 .../models/PrivateEndpointConnections.java    |   65 +-
 ...rivateEndpointServiceConnectionStatus.java |   22 +-
 .../models/PrivateLinkResource.java           |   26 +-
 .../models/PrivateLinkResourceListResult.java |   14 +-
 .../models/PrivateLinkResources.java          |   12 +-
 .../PrivateLinkServiceConnectionState.java    |   22 +-
 .../redisenterprise/models/Protocol.java      |   14 +-
 .../models/ProvisioningState.java             |   34 +-
 .../redisenterprise/models/RdbFrequency.java  |   22 +-
 .../models/RedisEnterprises.java              |   38 +-
 .../models/RegenerateKeyParameters.java       |   19 +-
 .../models/RegionSkuDetail.java               |   40 -
 .../redisenterprise/models/ResourceState.java |   68 +-
 .../redisenterprise/models/Sku.java           |   18 +-
 .../redisenterprise/models/SkuDetail.java     |   56 -
 .../redisenterprise/models/SkuName.java       |   38 +-
 .../redisenterprise/models/TlsVersion.java    |   22 +-
 .../models/UserAssignedIdentity.java          |   14 +-
 .../redisenterprise/models/package-info.java  |    4 +-
 .../redisenterprise/package-info.java         |    4 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       |  266 +
 .../generated/DatabasesCreateSamples.java     |   66 +-
 .../generated/DatabasesDeleteSamples.java     |   13 +-
 .../generated/DatabasesExportSamples.java     |   26 +-
 .../generated/DatabasesFlushSamples.java      |   28 +-
 .../DatabasesForceUnlinkSamples.java          |   24 +-
 .../generated/DatabasesGetSamples.java        |   13 +-
 .../DatabasesImportMethodSamples.java         |   31 +-
 .../DatabasesListByClusterSamples.java        |    9 +-
 .../generated/DatabasesListKeysSamples.java   |   13 +-
 .../DatabasesRegenerateKeySamples.java        |   19 +-
 .../generated/DatabasesUpdateSamples.java     |   28 +-
 .../generated/OperationsListSamples.java      |    9 +-
 .../generated/OperationsStatusGetSamples.java |   10 +-
 ...ivateEndpointConnectionsDeleteSamples.java |   13 +-
 .../PrivateEndpointConnectionsGetSamples.java |   14 +-
 ...PrivateEndpointConnectionsListSamples.java |    9 +-
 .../PrivateEndpointConnectionsPutSamples.java |   20 +-
 ...vateLinkResourcesListByClusterSamples.java |    9 +-
 .../RedisEnterpriseCreateSamples.java         |   48 +-
 .../RedisEnterpriseDeleteSamples.java         |   10 +-
 ...isEnterpriseGetByResourceGroupSamples.java |    9 +-
 ...sEnterpriseListByResourceGroupSamples.java |   13 +-
 .../generated/RedisEnterpriseListSamples.java |   10 +-
 .../RedisEnterpriseUpdateSamples.java         |   25 +-
 .../generated/SkusListSamples.java            |   20 -
 .../generated/CapabilityTests.java            |   27 -
 .../generated/ClusterInnerTests.java          |   73 -
 .../generated/ClusterListTests.java           |  114 -
 ...rManagedKeyEncryptionKeyIdentityTests.java |   25 +-
 .../generated/ClusterUpdateTests.java         |   72 -
 .../generated/DatabaseInnerTests.java         |   72 +-
 .../generated/DatabaseListTests.java          |   99 +-
 ...DatabasePropertiesGeoReplicationTests.java |   30 +-
 .../generated/DatabasePropertiesTests.java    |   69 +-
 .../generated/DatabaseUpdateTests.java        |   77 +-
 .../generated/DatabasesCreateMockTests.java   |   84 +-
 .../generated/DatabasesDeleteMockTests.java   |   36 +-
 .../generated/DatabasesExportMockTests.java   |   40 +-
 .../generated/DatabasesFlushMockTests.java    |   41 +-
 .../DatabasesForceUnlinkMockTests.java        |   40 +-
 .../DatabasesGetWithResponseMockTests.java    |   57 +-
 .../DatabasesImportMethodMockTests.java       |   41 +-
 .../DatabasesListByClusterMockTests.java      |   52 +-
 .../ExportClusterParametersTests.java         |   10 +-
 .../generated/FlushParametersTests.java       |   12 +-
 .../generated/ForceUnlinkParametersTests.java |   13 +-
 .../ImportClusterParametersTests.java         |   10 +-
 .../generated/LinkedDatabaseTests.java        |   10 +-
 .../generated/LocationInfoTests.java          |   43 -
 .../ManagedServiceIdentityTests.java          |   22 +-
 .../generated/ModuleTests.java                |   14 +-
 .../generated/OperationDisplayTests.java      |    8 +-
 .../generated/OperationInnerTests.java        |    8 +-
 .../generated/OperationListResultTests.java   |    8 +-
 .../generated/OperationStatusInnerTests.java  |   17 +-
 .../generated/OperationsListMockTests.java    |   34 +-
 ...rationsStatusGetWithResponseMockTests.java |   56 +-
 .../generated/PersistenceTests.java           |   24 +-
 .../PrivateEndpointConnectionInnerTests.java  |   38 +-
 ...vateEndpointConnectionListResultTests.java |   44 +-
 ...vateEndpointConnectionPropertiesTests.java |   38 +-
 ...ateEndpointConnectionsDeleteMockTests.java |   51 +
 ...onnectionsDeleteWithResponseMockTests.java |   63 -
 ...ntConnectionsGetWithResponseMockTests.java |   55 +-
 ...ivateEndpointConnectionsListMockTests.java |   56 +-
 .../generated/PrivateEndpointTests.java       |    2 +-
 .../PrivateLinkResourceInnerTests.java        |   15 +-
 .../PrivateLinkResourceListResultTests.java   |   23 +-
 .../PrivateLinkResourcePropertiesTests.java   |   17 +-
 ...teLinkResourcesListByClusterMockTests.java |   39 +-
 ...rivateLinkServiceConnectionStateTests.java |   22 +-
 .../RedisEnterprisesCreateMockTests.java      |  107 -
 .../RedisEnterprisesDeleteMockTests.java      |   36 +-
 ...tByResourceGroupWithResponseMockTests.java |   79 -
 ...terprisesListByResourceGroupMockTests.java |   78 -
 .../RedisEnterprisesListMockTests.java        |   76 -
 .../generated/RegionSkuDetailInnerTests.java  |   52 -
 .../generated/RegionSkuDetailsTests.java      |   46 -
 .../generated/SkuDetailTests.java             |   25 -
 .../redisenterprise/generated/SkuTests.java   |   12 +-
 .../generated/SkusListMockTests.java          |   70 -
 .../generated/UserAssignedIdentityTests.java  |    8 +-
 sdk/redisenterprise/tests.mgmt.yml            |   24 +-
 .../CHANGELOG.md                              |   11 +
 .../pom.xml                                   |    6 +-
 sdk/remoterendering/tests.yml                 |   18 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/resourcehealth/tests.mgmt.yml             |   18 +-
 sdk/resourcemanager/README.md                 |    8 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../azure-resourcemanager-appplatform/pom.xml |   14 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../azure-resourcemanager-appservice/pom.xml  |   14 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../pom.xml                                   |    8 +-
 .../azure-resourcemanager-cdn/CHANGELOG.md    |   10 +-
 .../azure-resourcemanager-cdn/README.md       |    2 +-
 .../azure-resourcemanager-cdn/pom.xml         |    4 +-
 .../CHANGELOG.md                              |   10 +-
 .../azure-resourcemanager-compute/README.md   |    2 +-
 .../azure-resourcemanager-compute/pom.xml     |   16 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../pom.xml                                   |   14 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../pom.xml                                   |    4 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../pom.xml                                   |    4 +-
 .../azure-resourcemanager-cosmos/CHANGELOG.md |   18 +-
 .../azure-resourcemanager-cosmos/README.md    |    2 +-
 .../azure-resourcemanager-cosmos/pom.xml      |    8 +-
 .../azure-resourcemanager-dns/CHANGELOG.md    |   10 +-
 .../azure-resourcemanager-dns/README.md       |    2 +-
 .../azure-resourcemanager-dns/pom.xml         |    4 +-
 .../CHANGELOG.md                              |   10 +-
 .../azure-resourcemanager-eventhubs/README.md |    2 +-
 .../azure-resourcemanager-eventhubs/pom.xml   |    6 +-
 .../CHANGELOG.md                              |    8 +-
 .../azure-resourcemanager-keyvault/README.md  |    2 +-
 .../assets.json                               |    2 +-
 .../azure-resourcemanager-keyvault/pom.xml    |   10 +-
 .../keyvault/implementation/VaultImpl.java    |   26 +
 .../keyvault/models/Vault.java                |   26 +
 .../resourcemanager/keyvault/VaultTests.java  |   21 +
 .../CHANGELOG.md                              |   10 +-
 .../azure-resourcemanager-monitor/README.md   |    2 +-
 .../azure-resourcemanager-monitor/pom.xml     |   14 +-
 .../azure-resourcemanager-msi/CHANGELOG.md    |   10 +-
 .../azure-resourcemanager-msi/README.md       |    2 +-
 .../azure-resourcemanager-msi/pom.xml         |    6 +-
 .../CHANGELOG.md                              |   10 +-
 .../azure-resourcemanager-network/README.md   |    2 +-
 .../azure-resourcemanager-network/pom.xml     |   10 +-
 .../azure-resourcemanager-perf/pom.xml        |    2 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../azure-resourcemanager-privatedns/pom.xml  |    6 +-
 .../azure-resourcemanager-redis/CHANGELOG.md  |   10 +-
 .../azure-resourcemanager-redis/README.md     |    2 +-
 .../azure-resourcemanager-redis/pom.xml       |    6 +-
 .../CHANGELOG.md                              |   10 +-
 .../azure-resourcemanager-resources/README.md |    2 +-
 .../azure-resourcemanager-resources/pom.xml   |    8 +-
 .../azure-resourcemanager-samples/README.md   |    2 +-
 .../azure-resourcemanager-samples/pom.xml     |   18 +-
 .../azure-resourcemanager-search/CHANGELOG.md |   10 +-
 .../azure-resourcemanager-search/README.md    |    2 +-
 .../azure-resourcemanager-search/pom.xml      |    4 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../azure-resourcemanager-servicebus/pom.xml  |    4 +-
 .../azure-resourcemanager-sql/CHANGELOG.md    |   10 +-
 .../azure-resourcemanager-sql/README.md       |    2 +-
 .../azure-resourcemanager-sql/pom.xml         |    6 +-
 .../CHANGELOG.md                              |   10 +-
 .../azure-resourcemanager-storage/README.md   |    2 +-
 .../azure-resourcemanager-storage/pom.xml     |    8 +-
 .../azure-resourcemanager-test/pom.xml        |    2 +-
 .../CHANGELOG.md                              |   10 +-
 .../README.md                                 |    2 +-
 .../pom.xml                                   |    4 +-
 .../azure-resourcemanager/CHANGELOG.md        |   10 +-
 .../azure-resourcemanager/README.md           |    2 +-
 .../azure-resourcemanager/assets.json         |    2 +-
 .../azure-resourcemanager/pom.xml             |   52 +-
 .../resourcemanager/PrivateLinkTests.java     |    1 +
 sdk/resourcemanager/ci.yml                    |    2 +
 .../docs/SINGLE_SERVICE_PACKAGES.md           |   36 +
 .../azure-resourcemanager-compute/pom.xml     |    4 +-
 .../azure-resourcemanager-keyvault/pom.xml    |    4 +-
 .../azure-resourcemanager-network/pom.xml     |    4 +-
 .../azure-resourcemanager-resources/pom.xml   |    2 +-
 .../azure-resourcemanager-test/pom.xml        |    2 +-
 .../pom.xml                                   |    8 +-
 sdk/resourcemover/tests.mgmt.yml              |   24 +-
 .../CHANGELOG.md                              |   31 +
 .../pom.xml                                   |   10 +-
 .../pom.xml                                   |   10 +-
 .../azure-data-schemaregistry/CHANGELOG.md    |   21 +
 .../azure-data-schemaregistry/pom.xml         |    4 +-
 sdk/schemaregistry/tests.yml                  |   36 +-
 .../azure-search-documents/CHANGELOG.md       |   33 +
 sdk/search/azure-search-documents/pom.xml     |    6 +-
 sdk/search/azure-search-perf/pom.xml          |    2 +-
 sdk/search/tests.yml                          |   44 +-
 .../CHANGELOG.md                              |  475 +-
 .../azure-resourcemanager-security/README.md  |    8 +-
 .../azure-resourcemanager-security/SAMPLE.md  | 6365 ++++++++++-------
 .../azure-resourcemanager-security/pom.xml    |   11 +-
 .../security/SecurityManager.java             | 1299 ++--
 .../AdaptiveApplicationControlsClient.java    |   48 +-
 .../AdaptiveNetworkHardeningsClient.java      |  103 +-
 .../AdvancedThreatProtectionsClient.java      |   20 +-
 .../security/fluent/AlertsClient.java         |  204 +-
 .../fluent/AlertsSuppressionRulesClient.java  |   32 +-
 .../fluent/AllowedConnectionsClient.java      |   48 +-
 .../ApiCollectionOffboardingsClient.java      |   50 -
 .../ApiCollectionOnboardingsClient.java       |   52 -
 .../security/fluent/ApiCollectionsClient.java |  265 +-
 .../fluent/ApplicationOperationsClient.java   |   20 +-
 .../security/fluent/ApplicationsClient.java   |   16 +-
 .../security/fluent/AssessmentsClient.java    |   44 +-
 .../fluent/AssessmentsMetadatasClient.java    |   48 +-
 .../AutoProvisioningSettingsClient.java       |   20 +-
 .../security/fluent/AutomationsClient.java    |   97 +-
 .../fluent/AzureDevOpsOrgsClient.java         |  237 +
 .../fluent/AzureDevOpsProjectsClient.java     |  225 +
 .../fluent/AzureDevOpsReposClient.java        |  239 +
 .../fluent/ComplianceResultsClient.java       |   16 +-
 .../security/fluent/CompliancesClient.java    |   20 +-
 .../security/fluent/ConnectorsClient.java     |   28 +-
 .../CustomAssessmentAutomationsClient.java    |   96 +-
 .../CustomEntityStoreAssignmentsClient.java   |  104 +-
 .../fluent/DefenderForStoragesClient.java     |   76 +
 .../fluent/DevOpsConfigurationsClient.java    |  253 +
 .../fluent/DevOpsOperationResultsClient.java  |   46 +
 .../fluent/DeviceSecurityGroupsClient.java    |   47 +-
 .../DiscoveredSecuritySolutionsClient.java    |   48 +-
 .../ExternalSecuritySolutionsClient.java      |   52 +-
 .../security/fluent/GitHubOwnersClient.java   |  103 +
 .../security/fluent/GitHubReposClient.java    |   80 +
 .../security/fluent/GitLabGroupsClient.java   |  103 +
 .../security/fluent/GitLabProjectsClient.java |   82 +
 .../fluent/GitLabSubgroupsClient.java         |   46 +
 .../fluent/GovernanceAssignmentsClient.java   |   76 +-
 .../fluent/GovernanceRulesClient.java         |  144 +-
 .../security/fluent/HealthReportsClient.java  |   36 +-
 .../InformationProtectionPoliciesClient.java  |   41 +-
 .../fluent/IngestionSettingsClient.java       |  169 -
 .../IotSecuritySolutionAnalyticsClient.java   |   28 +-
 ...utionsAnalyticsAggregatedAlertsClient.java |   36 +-
 ...lutionsAnalyticsRecommendationsClient.java |   36 +-
 .../fluent/IotSecuritySolutionsClient.java    |   78 +-
 .../JitNetworkAccessPoliciesClient.java       |  130 +-
 .../security/fluent/LocationsClient.java      |   16 +-
 .../security/fluent/MdeOnboardingsClient.java |   16 +-
 .../security/fluent/OperationsClient.java     |   16 +-
 .../security/fluent/PricingsClient.java       |  132 +-
 ...RegulatoryComplianceAssessmentsClient.java |   36 +-
 .../RegulatoryComplianceControlsClient.java   |   24 +-
 .../RegulatoryComplianceStandardsClient.java  |   16 +-
 .../fluent/ResourceProvidersClient.java}      |   56 +-
 .../SecureScoreControlDefinitionsClient.java  |   12 +-
 .../fluent/SecureScoreControlsClient.java     |   28 +-
 .../security/fluent/SecureScoresClient.java   |   18 +-
 .../security/fluent/SecurityCenter.java       |  570 +-
 ...yConnectorApplicationOperationsClient.java |   44 +-
 .../SecurityConnectorApplicationsClient.java  |   16 +-
 .../fluent/SecurityConnectorsClient.java      |   74 +-
 .../fluent/SecurityContactsClient.java        |   37 +-
 .../fluent/SecurityOperatorsClient.java       |   24 +-
 .../fluent/SecuritySolutionsClient.java       |   24 +-
 ...SecuritySolutionsReferenceDatasClient.java |   16 +-
 .../fluent/SensitivitySettingsClient.java     |   38 +
 .../ServerVulnerabilityAssessmentsClient.java |   84 +-
 ...ulnerabilityAssessmentsSettingsClient.java |  124 +
 .../security/fluent/SettingsClient.java       |   27 +-
 .../fluent/SoftwareInventoriesClient.java     |   55 +-
 ...rabilityAssessmentBaselineRulesClient.java |   28 +-
 ...nerabilityAssessmentScanResultsClient.java |   12 +-
 ...SqlVulnerabilityAssessmentScansClient.java |   12 +-
 .../security/fluent/SubAssessmentsClient.java |   44 +-
 .../security/fluent/TasksClient.java          |  100 +-
 .../security/fluent/TopologiesClient.java     |   44 +-
 .../fluent/WorkspaceSettingsClient.java       |   32 +-
 .../AdaptiveApplicationControlGroupData.java  |   34 +-
 .../AdaptiveApplicationControlGroupInner.java |   44 +-
 ...AdaptiveApplicationControlGroupsInner.java |   10 +-
 .../models/AdaptiveNetworkHardeningInner.java |   28 +-
 .../AdaptiveNetworkHardeningProperties.java   |   26 +-
 .../AdvancedThreatProtectionProperties.java   |   14 +-
 .../AdvancedThreatProtectionSettingInner.java |   16 +-
 .../security/fluent/models/AlertInner.java    |   70 +-
 .../fluent/models/AlertProperties.java        |   68 +-
 .../models/AlertSyncSettingProperties.java    |   14 +-
 .../models/AlertsSuppressionRuleInner.java    |   42 +-
 .../AlertsSuppressionRuleProperties.java      |   61 +-
 .../AllowedConnectionsResourceInner.java      |   18 +-
 .../AllowedConnectionsResourceProperties.java |   14 +-
 .../fluent/models/ApiCollectionInner.java     |  145 +
 .../models/ApiCollectionProperties.java       |  179 +-
 .../models/ApiCollectionResponseInner.java    |   90 -
 .../fluent/models/ApplicationInner.java       |   28 +-
 .../fluent/models/ApplicationProperties.java  |   38 +-
 .../fluent/models/AscLocationInner.java       |   14 +-
 .../models/AutoProvisioningSettingInner.java  |   16 +-
 .../AutoProvisioningSettingProperties.java    |   20 +-
 .../fluent/models/AutomationInner.java        |   48 +-
 .../fluent/models/AutomationProperties.java   |   34 +-
 .../AutomationValidationStatusInner.java      |   18 +-
 .../fluent/models/AzureDevOpsOrgInner.java    |   75 +
 .../AzureDevOpsOrgListResponseInner.java}     |   35 +-
 .../models/AzureDevOpsProjectInner.java       |   75 +
 .../models/AzureDevOpsRepositoryInner.java    |   75 +
 .../fluent/models/ComplianceInner.java        |   22 +-
 .../fluent/models/ComplianceProperties.java   |   16 +-
 .../fluent/models/ComplianceResultInner.java  |   14 +-
 .../models/ComplianceResultProperties.java    |   12 +-
 .../fluent/models/ConnectionStringsInner.java |   62 -
 .../fluent/models/ConnectorSettingInner.java  |   20 +-
 .../models/ConnectorSettingProperties.java    |   18 +-
 .../CustomAssessmentAutomationInner.java      |   48 +-
 .../CustomAssessmentAutomationProperties.java |   44 +-
 ...AssessmentAutomationRequestProperties.java |   34 +-
 .../CustomEntityStoreAssignmentInner.java     |   22 +-
 ...CustomEntityStoreAssignmentProperties.java |   18 +-
 ...ntityStoreAssignmentRequestProperties.java |   22 +-
 .../models/DataExportSettingProperties.java   |   14 +-
 .../DefenderForStorageSettingInner.java       |  220 +
 .../DefenderForStorageSettingProperties.java  |  237 +
 .../models/DevOpsConfigurationInner.java      |   75 +
 .../models/DeviceSecurityGroupInner.java      |   28 +-
 .../models/DeviceSecurityGroupProperties.java |   26 +-
 .../DiscoveredSecuritySolutionInner.java      |   36 +-
 .../DiscoveredSecuritySolutionProperties.java |   50 +-
 .../models/ExternalSecuritySolutionInner.java |   11 +-
 ...SensitivitySettingsListResponseInner.java} |   35 +-
 .../GetSensitivitySettingsResponseInner.java  |   59 +
 .../fluent/models/GitHubOwnerInner.java       |   75 +
 .../models/GitHubOwnerListResponseInner.java} |   35 +-
 .../fluent/models/GitHubRepositoryInner.java  |   75 +
 .../fluent/models/GitLabGroupInner.java       |   75 +
 .../models/GitLabGroupListResponseInner.java} |   35 +-
 .../fluent/models/GitLabProjectInner.java     |   75 +
 .../models/GovernanceAssignmentInner.java     |   48 +-
 .../GovernanceAssignmentProperties.java       |   54 +-
 .../fluent/models/GovernanceRuleInner.java    |   80 +-
 .../models/GovernanceRuleProperties.java      |  108 +-
 .../fluent/models/HealthReportInner.java      |   70 +-
 .../fluent/models/HealthReportProperties.java |   80 +-
 .../InformationProtectionPolicyInner.java     |   24 +-
 ...InformationProtectionPolicyProperties.java |   48 +-
 .../fluent/models/IngestionSettingInner.java  |   51 -
 .../models/IngestionSettingTokenInner.java    |   40 -
 .../IoTSecurityAggregatedAlertInner.java      |   42 +-
 .../IoTSecurityAggregatedAlertProperties.java |   36 +-
 ...SecurityAggregatedRecommendationInner.java |   38 +-
 ...ityAggregatedRecommendationProperties.java |   32 +-
 ...oTSecuritySolutionAnalyticsModelInner.java |   40 +-
 ...curitySolutionAnalyticsModelListInner.java |   22 +-
 ...uritySolutionAnalyticsModelProperties.java |   36 +-
 .../models/IoTSecuritySolutionModelInner.java |   82 +-
 .../models/IoTSecuritySolutionProperties.java |   82 +-
 .../models/JitNetworkAccessPolicyInner.java   |   34 +-
 .../JitNetworkAccessPolicyProperties.java     |   30 +-
 .../models/JitNetworkAccessRequestInner.java  |   48 +-
 .../models/MalwareScanningProperties.java     |  141 +
 .../fluent/models/MdeOnboardingDataInner.java |   26 +-
 .../models/MdeOnboardingDataListInner.java    |   14 +-
 .../models/MdeOnboardingDataProperties.java   |   18 +-
 .../fluent/models/OnUploadProperties.java     |   80 +
 .../fluent/models/OperationInner.java         |   18 +-
 .../OperationResultAutoGeneratedInner.java    |   12 +-
 .../models/OperationStatusResultInner.java    |  249 +
 .../security/fluent/models/PricingInner.java  |  140 +-
 .../fluent/models/PricingListInner.java       |   19 +-
 .../fluent/models/PricingProperties.java      |  197 +-
 .../RegulatoryComplianceAssessmentInner.java  |   30 +-
 ...ulatoryComplianceAssessmentProperties.java |   28 +-
 .../RegulatoryComplianceControlInner.java     |   24 +-
 ...RegulatoryComplianceControlProperties.java |   22 +-
 .../RegulatoryComplianceStandardInner.java    |   24 +-
 ...egulatoryComplianceStandardProperties.java |   22 +-
 .../fluent/models/RuleResultsInner.java       |   14 +-
 .../fluent/models/RulesResultsInner.java      |   14 +-
 .../security/fluent/models/ScanInner.java     |   14 +-
 .../fluent/models/ScanResultInner.java        |   14 +-
 .../fluent/models/ScanResultsInner.java       |   14 +-
 .../security/fluent/models/ScansInner.java    |   14 +-
 .../security/fluent/models/ScoreDetails.java  |   16 +-
 ...SecureScoreControlDefinitionItemInner.java |   22 +-
 ...eScoreControlDefinitionItemProperties.java |   20 +-
 .../SecureScoreControlDetailsInner.java       |   32 +-
 .../SecureScoreControlScoreDetailsInner.java  |   32 +-
 .../fluent/models/SecureScoreItemInner.java   |   22 +-
 .../models/SecureScoreItemProperties.java     |   22 +-
 .../SecurityAssessmentMetadataProperties.java |   74 +-
 ...yAssessmentMetadataPropertiesResponse.java |   82 +-
 ...curityAssessmentMetadataResponseInner.java |   78 +-
 .../models/SecurityAssessmentProperties.java  |   36 +-
 .../SecurityAssessmentPropertiesResponse.java |   36 +-
 .../SecurityAssessmentResponseInner.java      |   36 +-
 .../fluent/models/SecurityConnectorInner.java |   40 +-
 .../models/SecurityConnectorProperties.java   |   28 +-
 .../fluent/models/SecurityContactInner.java   |   74 +-
 .../models/SecurityContactProperties.java     |   86 +-
 .../fluent/models/SecurityOperatorInner.java  |   14 +-
 .../models/SecurityOperatorListInner.java     |   19 +-
 .../fluent/models/SecuritySolutionInner.java  |   30 +-
 .../models/SecuritySolutionProperties.java    |   50 +-
 ...curitySolutionsReferenceDataListInner.java |   14 +-
 ...uritySolutionsReferenceDataProperties.java |   81 +-
 .../models/SecuritySubAssessmentInner.java    |   38 +-
 .../SecuritySubAssessmentProperties.java      |   36 +-
 .../fluent/models/SecurityTaskInner.java      |   24 +-
 .../fluent/models/SecurityTaskProperties.java |   22 +-
 .../SensitiveDataDiscoveryProperties.java     |   74 +
 .../ServerVulnerabilityAssessmentInner.java   |   14 +-
 ...rverVulnerabilityAssessmentProperties.java |   12 +-
 ...lityAssessmentsAzureSettingProperties.java |   66 +
 ...rverVulnerabilityAssessmentsListInner.java |   14 +-
 ...rVulnerabilityAssessmentsSettingInner.java |   56 +
 .../security/fluent/models/SettingInner.java  |   13 +-
 .../security/fluent/models/SoftwareInner.java |   53 +-
 .../fluent/models/SoftwareProperties.java     |   46 +-
 .../fluent/models/TopologyResourceInner.java  |   18 +-
 .../models/TopologyResourceProperties.java    |   14 +-
 .../UpdateIoTSecuritySolutionProperties.java  |   26 +-
 .../fluent/models/WorkspaceSettingInner.java  |   28 +-
 .../models/WorkspaceSettingProperties.java    |   38 +-
 .../security/fluent/models/package-info.java  |    4 +-
 .../security/fluent/package-info.java         |    4 +-
 .../AdaptiveApplicationControlGroupImpl.java  |   65 +-
 .../AdaptiveApplicationControlGroupsImpl.java |   12 +-
 ...AdaptiveApplicationControlsClientImpl.java |  415 +-
 .../AdaptiveApplicationControlsImpl.java      |  105 +-
 .../AdaptiveNetworkHardeningImpl.java         |    4 +-
 .../AdaptiveNetworkHardeningsClientImpl.java  |  743 +-
 .../AdaptiveNetworkHardeningsImpl.java        |  114 +-
 .../AdvancedThreatProtectionSettingImpl.java  |   35 +-
 .../AdvancedThreatProtectionsClientImpl.java  |  161 +-
 .../AdvancedThreatProtectionsImpl.java        |   40 +-
 .../implementation/AlertsClientImpl.java      | 2012 ++----
 .../security/implementation/AlertsImpl.java   |  115 +-
 .../AlertsSuppressionRuleImpl.java            |    4 +-
 .../AlertsSuppressionRulesClientImpl.java     |  469 +-
 .../AlertsSuppressionRulesImpl.java           |   38 +-
 .../AllowedConnectionsClientImpl.java         |  511 +-
 .../AllowedConnectionsImpl.java               |   37 +-
 .../AllowedConnectionsResourceImpl.java       |    3 +-
 .../implementation/ApiCollectionImpl.java     |   81 +
 .../ApiCollectionOffboardingsClientImpl.java  |  248 -
 .../ApiCollectionOffboardingsImpl.java        |   43 -
 .../ApiCollectionOnboardingsClientImpl.java   |  252 -
 .../ApiCollectionOnboardingsImpl.java         |   60 -
 .../ApiCollectionResponseImpl.java            |   55 -
 .../ApiCollectionsClientImpl.java             | 1409 +++-
 .../implementation/ApiCollectionsImpl.java    |   97 +-
 .../implementation/ApplicationImpl.java       |   50 +-
 .../ApplicationOperationsClientImpl.java      |  239 +-
 .../ApplicationOperationsImpl.java            |   41 +-
 .../ApplicationsClientImpl.java               |  195 +-
 .../implementation/ApplicationsImpl.java      |    8 +-
 .../implementation/AssessmentsClientImpl.java |  360 +-
 .../implementation/AssessmentsImpl.java       |  119 +-
 .../AssessmentsMetadatasClientImpl.java       |  677 +-
 .../AssessmentsMetadatasImpl.java             |   82 +-
 .../AutoProvisioningSettingImpl.java          |   36 +-
 .../AutoProvisioningSettingsClientImpl.java   |  322 +-
 .../AutoProvisioningSettingsImpl.java         |   35 +-
 .../implementation/AutomationImpl.java        |  119 +-
 .../AutomationValidationStatusImpl.java       |    3 +-
 .../implementation/AutomationsClientImpl.java |  941 ++-
 .../implementation/AutomationsImpl.java       |  118 +-
 .../implementation/AzureDevOpsOrgImpl.java    |  121 +
 .../AzureDevOpsOrgListResponseImpl.java       |   47 +
 .../AzureDevOpsOrgsClientImpl.java            | 1105 +++
 .../implementation/AzureDevOpsOrgsImpl.java   |  135 +
 .../AzureDevOpsProjectImpl.java               |  127 +
 .../AzureDevOpsProjectsClientImpl.java        | 1049 +++
 .../AzureDevOpsProjectsImpl.java              |  127 +
 .../AzureDevOpsReposClientImpl.java           | 1117 +++
 .../implementation/AzureDevOpsReposImpl.java  |  138 +
 .../AzureDevOpsRepositoryImpl.java            |  133 +
 .../implementation/ComplianceResultImpl.java  |    4 +-
 .../ComplianceResultsClientImpl.java          |  219 +-
 .../implementation/ComplianceResultsImpl.java |   17 +-
 .../implementation/CompliancesClientImpl.java |  219 +-
 .../implementation/CompliancesImpl.java       |   13 +-
 .../implementation/ConnectionStringsImpl.java |   40 -
 .../implementation/ConnectorSettingImpl.java  |   46 +-
 .../implementation/ConnectorsClientImpl.java  |  404 +-
 .../implementation/ConnectorsImpl.java        |   45 +-
 .../CustomAssessmentAutomationImpl.java       |   42 +-
 ...CustomAssessmentAutomationsClientImpl.java |  869 +--
 .../CustomAssessmentAutomationsImpl.java      |  141 +-
 .../CustomEntityStoreAssignmentImpl.java      |   46 +-
 ...ustomEntityStoreAssignmentsClientImpl.java |  868 +--
 .../CustomEntityStoreAssignmentsImpl.java     |  141 +-
 .../DefenderForStorageSettingImpl.java        |  149 +
 .../DefenderForStoragesClientImpl.java        |  323 +
 .../DefenderForStoragesImpl.java              |   96 +
 .../DevOpsConfigurationImpl.java              |   50 +
 .../DevOpsConfigurationsClientImpl.java       | 1152 +++
 .../DevOpsConfigurationsImpl.java             |  123 +
 .../DevOpsOperationResultsClientImpl.java     |  210 +
 .../DevOpsOperationResultsImpl.java           |   58 +
 .../DeviceSecurityGroupImpl.java              |   71 +-
 .../DeviceSecurityGroupsClientImpl.java       |  398 +-
 .../DeviceSecurityGroupsImpl.java             |  153 +-
 .../DiscoveredSecuritySolutionImpl.java       |    3 +-
 ...DiscoveredSecuritySolutionsClientImpl.java |  515 +-
 .../DiscoveredSecuritySolutionsImpl.java      |   42 +-
 .../ExternalSecuritySolutionImpl.java         |    4 +-
 .../ExternalSecuritySolutionsClientImpl.java  |  511 +-
 .../ExternalSecuritySolutionsImpl.java        |   38 +-
 ...etSensitivitySettingsListResponseImpl.java |   44 +
 .../GetSensitivitySettingsResponseImpl.java   |   45 +
 .../implementation/GitHubOwnerImpl.java       |   49 +
 .../GitHubOwnerListResponseImpl.java          |   47 +
 .../GitHubOwnersClientImpl.java               |  566 ++
 .../implementation/GitHubOwnersImpl.java      |   93 +
 .../implementation/GitHubReposClientImpl.java |  458 ++
 .../implementation/GitHubReposImpl.java       |   74 +
 .../implementation/GitHubRepositoryImpl.java  |   50 +
 .../implementation/GitLabGroupImpl.java       |   49 +
 .../GitLabGroupListResponseImpl.java          |   47 +
 .../GitLabGroupsClientImpl.java               |  568 ++
 .../implementation/GitLabGroupsImpl.java      |   93 +
 .../implementation/GitLabProjectImpl.java     |   50 +
 .../GitLabProjectsClientImpl.java             |  466 ++
 .../implementation/GitLabProjectsImpl.java    |   74 +
 .../GitLabSubgroupsClientImpl.java            |  209 +
 .../implementation/GitLabSubgroupsImpl.java   |   58 +
 .../GovernanceAssignmentImpl.java             |   85 +-
 .../GovernanceAssignmentsClientImpl.java      |  497 +-
 .../GovernanceAssignmentsImpl.java            |  217 +-
 .../implementation/GovernanceRuleImpl.java    |   60 +-
 .../GovernanceRulesClientImpl.java            |  776 +-
 .../implementation/GovernanceRulesImpl.java   |  130 +-
 .../implementation/HealthReportImpl.java      |   19 +
 .../HealthReportOperationsClientImpl.java     |  180 -
 .../HealthReportOperationsImpl.java           |   58 -
 .../HealthReportsClientImpl.java              |  267 +-
 .../implementation/HealthReportsImpl.java     |   29 +-
 ...formationProtectionPoliciesClientImpl.java |  395 +-
 .../InformationProtectionPoliciesImpl.java    |  102 +-
 .../InformationProtectionPolicyImpl.java      |   78 +-
 .../implementation/IngestionSettingImpl.java  |  117 -
 .../IngestionSettingTokenImpl.java            |   32 -
 .../IngestionSettingsClientImpl.java          | 1030 ---
 .../implementation/IngestionSettingsImpl.java |  183 -
 .../IoTSecurityAggregatedAlertImpl.java       |    3 +-
 ...TSecurityAggregatedRecommendationImpl.java |    3 +-
 ...IoTSecuritySolutionAnalyticsModelImpl.java |    3 +-
 ...ecuritySolutionAnalyticsModelListImpl.java |   12 +-
 .../IoTSecuritySolutionModelImpl.java         |   66 +-
 ...otSecuritySolutionAnalyticsClientImpl.java |  226 +-
 .../IotSecuritySolutionAnalyticsImpl.java     |   29 +-
 ...nsAnalyticsAggregatedAlertsClientImpl.java |  450 +-
 ...olutionsAnalyticsAggregatedAlertsImpl.java |   43 +-
 ...onsAnalyticsRecommendationsClientImpl.java |  377 +-
 ...SolutionsAnalyticsRecommendationsImpl.java |   41 +-
 .../IotSecuritySolutionsClientImpl.java       |  835 +--
 .../IotSecuritySolutionsImpl.java             |  121 +-
 .../JitNetworkAccessPoliciesClientImpl.java   | 1297 ++--
 .../JitNetworkAccessPoliciesImpl.java         |  211 +-
 .../JitNetworkAccessPolicyImpl.java           |   75 +-
 .../JitNetworkAccessRequestImpl.java          |    4 +-
 .../implementation/LocationsClientImpl.java   |  249 +-
 .../implementation/LocationsImpl.java         |   13 +-
 .../implementation/MdeOnboardingDataImpl.java |    4 +-
 .../MdeOnboardingDataListImpl.java            |   12 +-
 .../MdeOnboardingsClientImpl.java             |  146 +-
 .../implementation/MdeOnboardingsImpl.java    |   14 +-
 .../OperationResultAutoGeneratedImpl.java     |    3 +-
 .../OperationStatusResultImpl.java            |   71 +
 .../implementation/OperationsClientImpl.java  |  170 +-
 .../implementation/OperationsImpl.java        |    8 +-
 .../security/implementation/PricingImpl.java  |   31 +-
 .../implementation/PricingListImpl.java       |    5 +-
 .../implementation/PricingsClientImpl.java    |  614 +-
 .../security/implementation/PricingsImpl.java |   56 +-
 .../RegulatoryComplianceAssessmentImpl.java   |    3 +-
 ...latoryComplianceAssessmentsClientImpl.java |  427 +-
 .../RegulatoryComplianceAssessmentsImpl.java  |   68 +-
 .../RegulatoryComplianceControlImpl.java      |    3 +-
 ...egulatoryComplianceControlsClientImpl.java |  351 +-
 .../RegulatoryComplianceControlsImpl.java     |   44 +-
 .../RegulatoryComplianceStandardImpl.java     |    3 +-
 ...gulatoryComplianceStandardsClientImpl.java |  284 +-
 .../RegulatoryComplianceStandardsImpl.java    |   22 +-
 .../implementation/ResourceManagerUtils.java  |  197 +
 .../ResourceProvidersClientImpl.java          |  270 +
 .../implementation/ResourceProvidersImpl.java |   79 +
 .../implementation/RuleResultsImpl.java       |   50 +-
 .../implementation/RulesResultsImpl.java      |    8 +-
 .../implementation/ScanResultsImpl.java       |    8 +-
 .../security/implementation/ScansImpl.java    |    5 +-
 .../SecureScoreControlDefinitionItemImpl.java |    3 +-
 ...cureScoreControlDefinitionsClientImpl.java |  339 +-
 .../SecureScoreControlDefinitionsImpl.java    |   15 +-
 .../SecureScoreControlDetailsImpl.java        |    4 +-
 .../SecureScoreControlScoreDetailsImpl.java   |    3 +-
 .../SecureScoreControlsClientImpl.java        |  415 +-
 .../SecureScoreControlsImpl.java              |   20 +-
 .../implementation/SecureScoreItemImpl.java   |    4 +-
 .../SecureScoresClientImpl.java               |  254 +-
 .../implementation/SecureScoresImpl.java      |   13 +-
 ...ecurityAssessmentMetadataResponseImpl.java |   43 +-
 .../SecurityAssessmentResponseImpl.java       |   67 +-
 .../implementation/SecurityCenterBuilder.java |   46 +-
 .../implementation/SecurityCenterImpl.java    | 1333 ++--
 ...nectorApplicationOperationsClientImpl.java |  354 +-
 ...ityConnectorApplicationOperationsImpl.java |   53 +-
 ...curityConnectorApplicationsClientImpl.java |  238 +-
 .../SecurityConnectorApplicationsImpl.java    |   11 +-
 .../implementation/SecurityConnectorImpl.java |   58 +-
 .../SecurityConnectorsClientImpl.java         |  807 +--
 .../SecurityConnectorsImpl.java               |  113 +-
 .../implementation/SecurityContactImpl.java   |   71 +-
 .../SecurityContactsClientImpl.java           |  459 +-
 .../implementation/SecurityContactsImpl.java  |   76 +-
 .../implementation/SecurityOperatorImpl.java  |    4 +-
 .../SecurityOperatorListImpl.java             |   12 +-
 .../SecurityOperatorsClientImpl.java          |  341 +-
 .../implementation/SecurityOperatorsImpl.java |   39 +-
 .../implementation/SecuritySolutionImpl.java  |    4 +-
 .../SecuritySolutionsClientImpl.java          |  280 +-
 .../implementation/SecuritySolutionsImpl.java |   21 +-
 ...ecuritySolutionsReferenceDataListImpl.java |    3 +-
 ...ritySolutionsReferenceDatasClientImpl.java |  181 +-
 .../SecuritySolutionsReferenceDatasImpl.java  |   21 +-
 .../SecuritySubAssessmentImpl.java            |    4 +-
 .../SensitivitySettingsClientImpl.java        |  146 +
 .../SensitivitySettingsImpl.java              |   55 +
 .../ServerVulnerabilityAssessmentImpl.java    |    3 +-
 ...verVulnerabilityAssessmentsClientImpl.java |  574 +-
 .../ServerVulnerabilityAssessmentsImpl.java   |   80 +-
 ...erverVulnerabilityAssessmentsListImpl.java |   12 +-
 ...erVulnerabilityAssessmentsSettingImpl.java |   45 +
 ...rabilityAssessmentsSettingsClientImpl.java |  641 ++
 ...rVulnerabilityAssessmentsSettingsImpl.java |  105 +
 .../implementation/SettingsClientImpl.java    |  322 +-
 .../security/implementation/SettingsImpl.java |   25 +-
 .../SoftwareInventoriesClientImpl.java        |  556 +-
 .../SoftwareInventoriesImpl.java              |   63 +-
 ...lityAssessmentBaselineRulesClientImpl.java |  294 +-
 ...nerabilityAssessmentBaselineRulesImpl.java |   34 +-
 ...bilityAssessmentScanResultsClientImpl.java |  156 +-
 ...ulnerabilityAssessmentScanResultsImpl.java |   29 +-
 ...ulnerabilityAssessmentScansClientImpl.java |  121 +-
 .../SqlVulnerabilityAssessmentScansImpl.java  |   13 +-
 .../SubAssessmentsClientImpl.java             |  433 +-
 .../implementation/SubAssessmentsImpl.java    |   25 +-
 .../implementation/TasksClientImpl.java       | 1102 +--
 .../security/implementation/TasksImpl.java    |   92 +-
 .../implementation/TopologiesClientImpl.java  |  483 +-
 .../implementation/TopologiesImpl.java        |   25 +-
 .../implementation/TopologyResourceImpl.java  |    4 +-
 .../security/implementation/Utils.java        |  204 -
 .../implementation/WorkspaceSettingImpl.java  |   54 +-
 .../WorkspaceSettingsClientImpl.java          |  507 +-
 .../implementation/WorkspaceSettingsImpl.java |   53 +-
 .../security/implementation/package-info.java |    4 +-
 .../security/models/AadConnectivityState.java |   22 +-
 .../AadConnectivityStateAutoGenerated.java    |   14 +-
 .../models/AadExternalSecuritySolution.java   |   14 +-
 .../models/AadSolutionProperties.java         |   26 +-
 .../security/models/ActionType.java           |   22 +-
 .../models/ActionableRemediation.java         |  157 +
 .../models/ActionableRemediationState.java    |   61 +
 .../ActiveConnectionsNotInAllowedRange.java   |   26 +-
 .../AdaptiveApplicationControlGroup.java      |  155 +-
 .../AdaptiveApplicationControlGroups.java     |    8 +-
 .../AdaptiveApplicationControlIssue.java      |   46 +-
 ...daptiveApplicationControlIssueSummary.java |   18 +-
 .../models/AdaptiveApplicationControls.java   |   38 +-
 .../models/AdaptiveNetworkHardening.java      |   18 +-
 ...daptiveNetworkHardeningEnforceRequest.java |   39 +-
 .../models/AdaptiveNetworkHardenings.java     |   76 +-
 .../models/AdaptiveNetworkHardeningsList.java |   18 +-
 .../security/models/AdditionalData.java       |   13 +-
 .../models/AdditionalWorkspaceDataType.java   |   18 +-
 .../models/AdditionalWorkspaceType.java       |   14 +-
 .../AdditionalWorkspacesProperties.java       |   22 +-
 .../AdvancedThreatProtectionSetting.java      |   51 +-
 .../models/AdvancedThreatProtections.java     |   14 +-
 .../security/models/Alert.java                |   66 +-
 .../security/models/AlertEntity.java          |   19 +-
 .../security/models/AlertList.java            |   16 +-
 .../AlertPropertiesSupportingEvidence.java    |   19 +-
 .../security/models/AlertSeverity.java        |   22 +-
 ...lertSimulatorBundlesRequestProperties.java |   14 +-
 .../models/AlertSimulatorRequestBody.java     |   14 +-
 .../AlertSimulatorRequestProperties.java      |   19 +-
 .../security/models/AlertStatus.java          |   26 +-
 .../security/models/AlertSyncSettings.java    |   16 +-
 .../security/models/Alerts.java               |  184 +-
 .../models/AlertsSuppressionRule.java         |   28 +-
 .../models/AlertsSuppressionRules.java        |   28 +-
 .../models/AlertsSuppressionRulesList.java    |   22 +-
 .../security/models/AllowedConnections.java   |   48 +-
 .../models/AllowedConnectionsList.java        |   14 +-
 .../models/AllowedConnectionsResource.java    |   18 +-
 .../models/AllowlistCustomAlertRule.java      |   27 +-
 .../AmqpC2DMessagesNotInAllowedRange.java     |   26 +-
 ...pC2DRejectedMessagesNotInAllowedRange.java |   26 +-
 .../AmqpD2CMessagesNotInAllowedRange.java     |   26 +-
 .../models/AnnotateDefaultBranchState.java    |   56 +
 .../security/models/ApiCollection.java        |  116 +
 ...sponseList.java => ApiCollectionList.java} |   24 +-
 .../models/ApiCollectionOffboardings.java     |   46 -
 .../models/ApiCollectionOnboardings.java      |   47 -
 .../security/models/ApiCollections.java       |  212 +-
 .../security/models/Application.java          |  128 +-
 .../models/ApplicationOperations.java         |   22 +-
 .../models/ApplicationSourceResourceType.java |   14 +-
 .../security/models/Applications.java         |   16 +-
 .../security/models/ApplicationsList.java     |   14 +-
 .../security/models/AscLocation.java          |   14 +-
 .../security/models/AscLocationList.java      |   14 +-
 .../security/models/AssessedResourceType.java |   26 +-
 .../security/models/AssessmentLinks.java      |   12 +-
 .../security/models/AssessmentStatus.java     |   27 +-
 .../security/models/AssessmentStatusCode.java |   22 +-
 .../models/AssessmentStatusResponse.java      |   26 +-
 .../security/models/AssessmentType.java       |   22 +-
 .../security/models/Assessments.java          |   42 +-
 .../security/models/AssessmentsMetadatas.java |   46 +-
 .../models/AtaExternalSecuritySolution.java   |   14 +-
 .../models/AtaSolutionProperties.java         |   26 +-
 .../AuthenticationDetailsProperties.java      |   17 +-
 .../AuthenticationProvisioningState.java      |   26 +-
 .../security/models/AuthenticationType.java   |   18 +-
 .../security/models/Authorization.java        |   60 +
 .../security/models/AutoDiscovery.java        |   58 +
 .../security/models/AutoProvision.java        |   18 +-
 .../models/AutoProvisioningSetting.java       |   44 +-
 .../models/AutoProvisioningSettingList.java   |   16 +-
 .../models/AutoProvisioningSettings.java      |   18 +-
 .../security/models/Automation.java           |  220 +-
 .../security/models/AutomationAction.java     |   13 +-
 .../models/AutomationActionEventHub.java      |   42 +-
 .../models/AutomationActionLogicApp.java      |   18 +-
 .../models/AutomationActionWorkspace.java     |   18 +-
 .../security/models/AutomationList.java       |   21 +-
 .../security/models/AutomationRuleSet.java    |   14 +-
 .../security/models/AutomationScope.java      |   18 +-
 .../security/models/AutomationSource.java     |   14 +-
 .../models/AutomationTriggeringRule.java      |   22 +-
 .../models/AutomationUpdateModel.java         |  183 +
 .../models/AutomationValidationStatus.java    |   10 +-
 .../security/models/Automations.java          |   58 +-
 ...meRoleAuthenticationDetailsProperties.java |   38 +-
 ...sCredsAuthenticationDetailsProperties.java |   29 +-
 .../security/models/AwsEnvironmentData.java   |   46 +-
 .../models/AwsOrganizationalData.java         |   13 +-
 .../models/AwsOrganizationalDataMaster.java   |   18 +-
 .../models/AwsOrganizationalDataMember.java   |   14 +-
 .../security/models/AzureDevOpsOrg.java       |  189 +
 .../models/AzureDevOpsOrgListResponse.java    |   34 +
 .../models/AzureDevOpsOrgProperties.java      |  182 +
 .../security/models/AzureDevOpsOrgs.java      |  124 +
 .../security/models/AzureDevOpsProject.java   |  191 +
 .../AzureDevOpsProjectListResponse.java       |   85 +
 .../models/AzureDevOpsProjectProperties.java  |  223 +
 .../security/models/AzureDevOpsProjects.java  |  103 +
 .../security/models/AzureDevOpsRepos.java     |  109 +
 .../models/AzureDevOpsRepository.java         |  192 +
 .../AzureDevOpsRepositoryListResponse.java    |   85 +
 .../AzureDevOpsRepositoryProperties.java      |  280 +
 .../AzureDevOpsScopeEnvironmentData.java      |   10 +-
 .../security/models/AzureResourceDetails.java |   12 +-
 .../models/AzureResourceIdentifier.java       |   12 +-
 .../security/models/AzureResourceLink.java    |   12 +-
 .../security/models/AzureServersSetting.java  |   81 +
 .../models/AzureTrackedResourceLocation.java  |   14 +-
 .../security/models/Baseline.java             |   18 +-
 .../models/BaselineAdjustedResult.java        |   26 +-
 .../security/models/BenchmarkReference.java   |   18 +-
 .../security/models/BuiltInInfoType.java      |  107 +
 .../security/models/BundleType.java           |   46 +-
 .../security/models/Categories.java           |   30 +-
 .../models/CategoryConfiguration.java         |   98 +
 .../models/CefExternalSecuritySolution.java   |   14 +-
 .../models/CefSolutionProperties.java         |   34 +-
 .../security/models/CloudName.java            |   34 +-
 .../security/models/CloudOffering.java        |   15 +-
 .../resourcemanager/security/models/Code.java |   18 +-
 .../security/models/Compliance.java           |   18 +-
 .../security/models/ComplianceList.java       |   16 +-
 .../security/models/ComplianceResult.java     |   14 +-
 .../security/models/ComplianceResultList.java |   21 +-
 .../security/models/ComplianceResults.java    |   16 +-
 .../security/models/ComplianceSegment.java    |   14 +-
 .../security/models/Compliances.java          |   20 +-
 .../security/models/ConfigurationStatus.java  |   30 +-
 .../security/models/ConnectableResource.java  |   16 +-
 .../security/models/ConnectedResource.java    |   16 +-
 .../security/models/ConnectedWorkspace.java   |   14 +-
 .../models/ConnectionFromIpNotAllowed.java    |   18 +-
 .../security/models/ConnectionStrings.java    |   25 -
 .../models/ConnectionToIpNotAllowed.java      |   18 +-
 .../security/models/ConnectionType.java       |   18 +-
 .../security/models/ConnectorSetting.java     |   90 +-
 .../security/models/ConnectorSettingList.java |   16 +-
 .../security/models/Connectors.java           |   30 +-
 ...tainerRegistryVulnerabilityProperties.java |   39 +-
 .../security/models/ControlType.java          |   18 +-
 .../models/CspmMonitorAwsOffering.java        |   18 +-
 ...nitorAwsOfferingNativeCloudConnection.java |   14 +-
 .../CspmMonitorAzureDevOpsOffering.java       |   10 +-
 .../models/CspmMonitorGcpOffering.java        |   18 +-
 ...nitorGcpOfferingNativeCloudConnection.java |   26 +-
 .../models/CspmMonitorGitLabOffering.java     |   10 +-
 .../models/CspmMonitorGithubOffering.java     |   10 +-
 .../security/models/CustomAlertRule.java      |   21 +-
 .../models/CustomAssessmentAutomation.java    |  114 +-
 .../CustomAssessmentAutomationRequest.java    |   36 +-
 .../models/CustomAssessmentAutomations.java   |  104 +-
 ...CustomAssessmentAutomationsListResult.java |   16 +-
 .../models/CustomEntityStoreAssignment.java   |   59 +-
 .../CustomEntityStoreAssignmentRequest.java   |   24 +-
 .../models/CustomEntityStoreAssignments.java  |  104 +-
 ...ustomEntityStoreAssignmentsListResult.java |   16 +-
 .../resourcemanager/security/models/Cve.java  |   14 +-
 .../resourcemanager/security/models/Cvss.java |   12 +-
 .../security/models/DataExportSettings.java   |   16 +-
 .../security/models/DataSource.java           |   14 +-
 .../models/DefenderCspmAwsOffering.java       |  119 +-
 .../models/DefenderCspmAwsOfferingCiem.java   |   86 +
 .../DefenderCspmAwsOfferingCiemDiscovery.java |   54 +
 .../DefenderCspmAwsOfferingCiemOidc.java      |   82 +
 ...pmAwsOfferingDataSensitivityDiscovery.java |   18 +-
 .../DefenderCspmAwsOfferingDatabasesDspm.java |   18 +-
 ...ingMdcContainersAgentlessDiscoveryK8S.java |   80 +
 ...sOfferingMdcContainersImageAssessment.java |   80 +
 .../DefenderCspmAwsOfferingVmScanners.java    |   22 +-
 ...spmAwsOfferingVmScannersConfiguration.java |   22 +-
 .../models/DefenderCspmGcpOffering.java       |  167 +-
 .../DefenderCspmGcpOfferingCiemDiscovery.java |  110 +
 ...pmGcpOfferingDataSensitivityDiscovery.java |  108 +
 ...ingMdcContainersAgentlessDiscoveryK8S.java |  108 +
 ...pOfferingMdcContainersImageAssessment.java |  108 +
 .../DefenderCspmGcpOfferingVmScanners.java    |   84 +
 ...spmGcpOfferingVmScannersConfiguration.java |   83 +
 .../DefenderFoDatabasesAwsOffering.java       |   26 +-
 ...tabasesAwsOfferingArcAutoProvisioning.java |   26 +-
 ...eringArcAutoProvisioningConfiguration.java |   22 +-
 ...erFoDatabasesAwsOfferingDatabasesDspm.java |   18 +-
 .../DefenderFoDatabasesAwsOfferingRds.java    |   18 +-
 .../DefenderForContainersAwsOffering.java     |  134 +-
 ...tainersAwsOfferingCloudWatchToKinesis.java |   14 +-
 ...eringContainerVulnerabilityAssessment.java |   14 +-
 ...gContainerVulnerabilityAssessmentTask.java |   14 +-
 ...erForContainersAwsOfferingKinesisToS3.java |   14 +-
 ...inersAwsOfferingKubernetesScubaReader.java |   14 +-
 ...ontainersAwsOfferingKubernetesService.java |   14 +-
 ...ingMdcContainersAgentlessDiscoveryK8S.java |   80 +
 ...sOfferingMdcContainersImageAssessment.java |   80 +
 .../DefenderForContainersGcpOffering.java     |  104 +-
 ...ringDataPipelineNativeCloudConnection.java |   26 +-
 ...ingMdcContainersAgentlessDiscoveryK8S.java |  108 +
 ...pOfferingMdcContainersImageAssessment.java |  108 +
 ...inersGcpOfferingNativeCloudConnection.java |   26 +-
 .../DefenderForDatabasesGcpOffering.java      |   28 +-
 ...tabasesGcpOfferingArcAutoProvisioning.java |   22 +-
 ...eringArcAutoProvisioningConfiguration.java |   22 +-
 ...fenderForDatabasesArcAutoProvisioning.java |   26 +-
 .../DefenderForDevOpsAzureDevOpsOffering.java |   10 +-
 .../DefenderForDevOpsGitLabOffering.java      |   10 +-
 .../DefenderForDevOpsGithubOffering.java      |   10 +-
 .../models/DefenderForServersAwsOffering.java |   50 +-
 ...ServersAwsOfferingArcAutoProvisioning.java |   26 +-
 ...eringArcAutoProvisioningConfiguration.java |   18 +-
 ...rServersAwsOfferingDefenderForServers.java |   14 +-
 ...ServersAwsOfferingMdeAutoProvisioning.java |   18 +-
 .../DefenderForServersAwsOfferingSubPlan.java |   14 +-
 ...rServersAwsOfferingVaAutoProvisioning.java |   22 +-
 ...feringVaAutoProvisioningConfiguration.java |   14 +-
 ...fenderForServersAwsOfferingVmScanners.java |   22 +-
 ...ersAwsOfferingVmScannersConfiguration.java |   22 +-
 .../models/DefenderForServersGcpOffering.java |   50 +-
 ...ServersGcpOfferingArcAutoProvisioning.java |   22 +-
 ...eringArcAutoProvisioningConfiguration.java |   18 +-
 ...rServersGcpOfferingDefenderForServers.java |   26 +-
 ...ServersGcpOfferingMdeAutoProvisioning.java |   18 +-
 .../DefenderForServersGcpOfferingSubPlan.java |   14 +-
 ...rServersGcpOfferingVaAutoProvisioning.java |   22 +-
 ...feringVaAutoProvisioningConfiguration.java |   14 +-
 ...fenderForServersGcpOfferingVmScanners.java |   22 +-
 ...ersGcpOfferingVmScannersConfiguration.java |   18 +-
 .../models/DefenderForStorageSetting.java     |  262 +
 .../security/models/DefenderForStorages.java  |   69 +
 .../models/DenylistCustomAlertRule.java       |   24 +-
 .../security/models/DevOpsConfiguration.java  |   55 +
 .../DevOpsConfigurationListResponse.java      |   85 +
 .../models/DevOpsConfigurationProperties.java |  194 +
 .../security/models/DevOpsConfigurations.java |  147 +
 .../models/DevOpsOperationResults.java        |   41 +
 .../models/DevOpsProvisioningState.java       |   86 +
 .../security/models/DeviceSecurityGroup.java  |  135 +-
 .../models/DeviceSecurityGroupList.java       |   16 +-
 .../security/models/DeviceSecurityGroups.java |   38 +-
 .../DirectMethodInvokesNotInAllowedRange.java |   26 +-
 .../security/models/Direction.java            |   18 +-
 .../models/DiscoveredSecuritySolution.java    |   22 +-
 .../DiscoveredSecuritySolutionList.java       |   16 +-
 .../models/DiscoveredSecuritySolutions.java   |   44 +-
 .../EffectiveNetworkSecurityGroups.java       |   18 +-
 .../security/models/EndOfSupportStatus.java   |   34 +-
 .../security/models/Enforce.java              |   55 +
 .../security/models/EnforcementMode.java      |   22 +-
 .../security/models/EnforcementSupport.java   |   22 +-
 .../security/models/EnvironmentData.java      |   13 +-
 .../security/models/EnvironmentDetails.java   |   38 +-
 .../security/models/EnvironmentType.java      |   30 +-
 .../resourcemanager/security/models/Etag.java |   14 +-
 .../security/models/EventSource.java          |   68 +-
 .../models/ExecuteGovernanceRuleParams.java   |   14 +-
 .../security/models/ExpandControlsEnum.java   |   14 +-
 .../security/models/ExpandEnum.java           |   18 +-
 .../security/models/ExportData.java           |   14 +-
 .../security/models/Extension.java            |   73 +-
 .../models/ExternalSecuritySolution.java      |   14 +-
 .../models/ExternalSecuritySolutionKind.java  |   22 +-
 .../models/ExternalSecuritySolutionList.java  |   16 +-
 .../ExternalSecuritySolutionProperties.java   |   29 +-
 .../models/ExternalSecuritySolutions.java     |   48 +-
 .../FailedLocalLoginsNotInAllowedRange.java   |   26 +-
 .../security/models/FileType.java             |   34 +-
 .../models/FileUploadsNotInAllowedRange.java  |   26 +-
 .../GcpCredentialsDetailsProperties.java      |  120 +-
 .../models/GcpOrganizationalData.java         |   13 +-
 .../models/GcpOrganizationalDataMember.java   |   18 +-
 .../GcpOrganizationalDataOrganization.java    |   24 +-
 .../security/models/GcpProjectDetails.java    |   22 +-
 .../models/GcpProjectEnvironmentData.java     |   44 +-
 .../GetSensitivitySettingsListResponse.java   |   27 +
 .../GetSensitivitySettingsResponse.java       |   47 +
 ...SensitivitySettingsResponseProperties.java |  146 +
 ...tingsResponsePropertiesMipInformation.java |  144 +
 ...llectionResponse.java => GitHubOwner.java} |   38 +-
 .../models/GitHubOwnerListResponse.java       |   34 +
 .../models/GitHubOwnerProperties.java         |  183 +
 .../security/models/GitHubOwners.java}        |   81 +-
 .../security/models/GitHubRepos.java          |   72 +
 .../security/models/GitHubRepository.java     |   55 +
 .../models/GitHubRepositoryListResponse.java  |   85 +
 .../models/GitHubRepositoryProperties.java    |  251 +
 .../security/models/GitLabGroup.java          |   55 +
 .../models/GitLabGroupListResponse.java       |   34 +
 .../models/GitLabGroupProperties.java         |  209 +
 .../security/models/GitLabGroups.java         |   93 +
 .../security/models/GitLabProject.java        |   55 +
 .../models/GitLabProjectListResponse.java     |   85 +
 .../models/GitLabProjectProperties.java       |  229 +
 .../security/models/GitLabProjects.java       |   74 +
 .../security/models/GitLabSubgroups.java      |   41 +
 .../models/GithubScopeEnvironmentData.java    |   10 +-
 .../models/GitlabScopeEnvironmentData.java    |   10 +-
 .../security/models/GovernanceAssignment.java |  187 +-
 .../GovernanceAssignmentAdditionalData.java   |   22 +-
 .../models/GovernanceAssignments.java         |   62 +-
 .../models/GovernanceAssignmentsList.java     |   14 +-
 .../models/GovernanceEmailNotification.java   |   20 +-
 .../security/models/GovernanceRule.java       |  355 +-
 .../GovernanceRuleEmailNotification.java      |   22 +-
 .../security/models/GovernanceRuleList.java   |   14 +-
 .../models/GovernanceRuleMetadata.java        |   18 +-
 .../models/GovernanceRuleOwnerSource.java     |   18 +-
 .../models/GovernanceRuleOwnerSourceType.java |   18 +-
 .../GovernanceRuleSourceResourceType.java     |   14 +-
 .../security/models/GovernanceRuleType.java   |   18 +-
 .../security/models/GovernanceRules.java      |   98 +-
 ...overnanceRulesOperationResultsHeaders.java |   12 +-
 ...vernanceRulesOperationResultsResponse.java |   16 +-
 .../models/HealthDataClassification.java      |   28 +-
 .../security/models/HealthReport.java         |   40 +-
 .../security/models/HealthReports.java        |   34 +-
 .../security/models/HealthReportsList.java    |   14 +-
 .../HttpC2DMessagesNotInAllowedRange.java     |   26 +-
 ...pC2DRejectedMessagesNotInAllowedRange.java |   26 +-
 .../HttpD2CMessagesNotInAllowedRange.java     |   26 +-
 .../HybridComputeProvisioningState.java       |   22 +-
 .../HybridComputeSettingsProperties.java      |   38 +-
 .../security/models/Identity.java             |   18 +-
 .../security/models/ImplementationEffort.java |   22 +-
 .../security/models/InfoType.java             |  107 +
 .../InformationProtectionAwsOffering.java     |   18 +-
 ...ctionAwsOfferingInformationProtection.java |   14 +-
 .../models/InformationProtectionKeyword.java  |   26 +-
 .../models/InformationProtectionPolicies.java |   30 +-
 .../models/InformationProtectionPolicy.java   |   97 +-
 .../InformationProtectionPolicyList.java      |   16 +-
 .../InformationProtectionPolicyName.java      |   18 +-
 .../security/models/InformationType.java      |   38 +-
 .../models/IngestionConnectionString.java     |   54 -
 .../security/models/IngestionSetting.java     |  142 -
 .../models/IngestionSettingToken.java         |   25 -
 .../security/models/IngestionSettings.java    |  180 -
 .../models/InheritFromParentState.java        |   56 +
 .../security/models/Inherited.java            |   56 +
 .../security/models/Intent.java               |   70 +-
 .../models/IoTSecurityAggregatedAlert.java    |   40 +-
 .../IoTSecurityAggregatedAlertList.java       |   22 +-
 ...atedAlertPropertiesTopDevicesListItem.java |   16 +-
 .../IoTSecurityAggregatedRecommendation.java  |   34 +-
 ...TSecurityAggregatedRecommendationList.java |   22 +-
 .../models/IoTSecurityAlertedDevice.java      |   14 +-
 .../models/IoTSecurityDeviceAlert.java        |   16 +-
 .../IoTSecurityDeviceRecommendation.java      |   16 +-
 .../IoTSecuritySolutionAnalyticsModel.java    |   24 +-
 ...IoTSecuritySolutionAnalyticsModelList.java |   10 +-
 ...ticsModelPropertiesDevicesMetricsItem.java |   22 +-
 .../models/IoTSecuritySolutionModel.java      |  224 +-
 .../models/IoTSecuritySolutionsList.java      |   21 +-
 .../security/models/IoTSeverityMetrics.java   |   22 +-
 .../models/IotSecuritySolutionAnalytics.java  |   28 +-
 .../security/models/IotSecuritySolutions.java |   50 +-
 ...itySolutionsAnalyticsAggregatedAlerts.java |   40 +-
 ...ritySolutionsAnalyticsRecommendations.java |   40 +-
 .../security/models/IsEnabled.java            |   18 +-
 .../security/models/Issue.java                |   46 +-
 .../models/JitNetworkAccessPolicies.java      |  107 +-
 .../models/JitNetworkAccessPoliciesList.java  |   16 +-
 .../models/JitNetworkAccessPolicy.java        |  126 +-
 .../JitNetworkAccessPolicyInitiatePort.java   |   28 +-
 ...JitNetworkAccessPolicyInitiateRequest.java |   28 +-
 ...orkAccessPolicyInitiateVirtualMachine.java |   30 +-
 .../JitNetworkAccessPolicyVirtualMachine.java |   34 +-
 .../models/JitNetworkAccessPortRule.java      |   42 +-
 .../models/JitNetworkAccessRequest.java       |   14 +-
 .../models/JitNetworkAccessRequestPort.java   |   56 +-
 ...JitNetworkAccessRequestVirtualMachine.java |   30 +-
 .../resourcemanager/security/models/Kind.java |   14 +-
 .../security/models/KindAutoGenerated.java    |   14 +-
 .../security/models/Label.java                |  109 +
 .../security/models/ListCustomAlertRule.java  |   19 +-
 .../security/models/LocalUserNotAllowed.java  |   18 +-
 .../security/models/Location.java             |   12 +-
 .../security/models/Locations.java            |   16 +-
 .../models/LogAnalyticsIdentifier.java        |   18 +-
 .../security/models/MdeOnboardingData.java    |   16 +-
 .../models/MdeOnboardingDataList.java         |    8 +-
 .../security/models/MdeOnboardings.java       |   16 +-
 .../security/models/MinimalRiskLevel.java     |   63 +
 .../security/models/MinimalSeverity.java      |   22 +-
 .../security/models/MipIntegrationStatus.java |   63 +
 .../MqttC2DMessagesNotInAllowedRange.java     |   26 +-
 ...tC2DRejectedMessagesNotInAllowedRange.java |   26 +-
 .../MqttD2CMessagesNotInAllowedRange.java     |   26 +-
 .../security/models/NotificationsSource.java  |   39 +
 .../models/NotificationsSourceAlert.java      |   60 +
 .../models/NotificationsSourceAttackPath.java |   62 +
 .../security/models/OfferingType.java         |   78 +-
 .../models/OnPremiseResourceDetails.java      |   51 +-
 .../models/OnPremiseSqlResourceDetails.java   |   46 +-
 .../security/models/OnboardingState.java      |   69 +
 .../security/models/Operation.java            |   12 +-
 .../security/models/OperationDisplay.java     |   18 +-
 .../security/models/OperationList.java        |   16 +-
 .../security/models/OperationResult.java      |   22 +-
 .../models/OperationResultAutoGenerated.java  |    8 +-
 .../security/models/OperationStatus.java      |   24 +-
 .../models/OperationStatusAutoGenerated.java  |   80 +
 .../models/OperationStatusResult.java         |   78 +
 .../security/models/Operations.java           |   16 +-
 .../security/models/Operator.java             |   46 +-
 .../models/OrganizationMembershipType.java    |   18 +-
 .../security/models/PathRecommendation.java   |   46 +-
 .../security/models/PermissionProperty.java   |   34 +-
 .../security/models/Pricing.java              |   87 +-
 .../security/models/PricingList.java          |    8 +-
 .../security/models/PricingTier.java          |   19 +-
 .../security/models/Pricings.java             |  129 +-
 .../security/models/ProcessNotAllowed.java    |   18 +-
 .../security/models/PropertyType.java         |   26 +-
 .../security/models/ProtectionMode.java       |   22 +-
 .../security/models/Protocol.java             |   22 +-
 .../security/models/ProvisioningState.java    |   22 +-
 .../models/ProxyServerProperties.java         |   14 +-
 .../security/models/PublisherInfo.java        |   28 +-
 .../security/models/QueryCheck.java           |   22 +-
 .../models/QueuePurgesNotInAllowedRange.java  |   26 +-
 .../resourcemanager/security/models/Rank.java |   34 +-
 .../security/models/RecommendationAction.java |   22 +-
 .../models/RecommendationConfigStatus.java    |   18 +-
 ...RecommendationConfigurationProperties.java |   32 +-
 .../security/models/RecommendationStatus.java |   26 +-
 .../security/models/RecommendationType.java   |   90 +-
 .../RegulatoryComplianceAssessment.java       |   28 +-
 .../RegulatoryComplianceAssessmentList.java   |   22 +-
 .../RegulatoryComplianceAssessments.java      |   34 +-
 .../models/RegulatoryComplianceControl.java   |   22 +-
 .../RegulatoryComplianceControlList.java      |   22 +-
 .../models/RegulatoryComplianceControls.java  |   20 +-
 .../models/RegulatoryComplianceStandard.java  |   22 +-
 .../RegulatoryComplianceStandardList.java     |   22 +-
 .../models/RegulatoryComplianceStandards.java |   12 +-
 .../security/models/Remediation.java          |   26 +-
 .../security/models/RemediationEta.java       |   28 +-
 .../security/models/ReportedSeverity.java     |   26 +-
 .../security/models/ResourceDetails.java      |   13 +-
 .../models/ResourceDetailsAutoGenerated.java  |   18 +-
 .../security/models/ResourceIdentifier.java   |   13 +-
 .../models/ResourceIdentifierType.java        |   18 +-
 .../security/models/ResourceIdentityType.java |   18 +-
 .../security/models/ResourceProviders.java}   |   49 +-
 .../security/models/ResourceStatus.java       |   26 +-
 .../models/ResourcesCoverageStatus.java       |   61 +
 .../security/models/Roles.java                |   53 -
 .../resourcemanager/security/models/Rule.java |   26 +-
 .../security/models/RuleCategory.java         |   79 +
 .../security/models/RuleResults.java          |  112 +-
 .../security/models/RuleResultsInput.java     |   26 +-
 .../models/RuleResultsProperties.java         |   14 +-
 .../security/models/RuleSeverity.java         |   30 +-
 .../security/models/RuleState.java            |   26 +-
 .../security/models/RuleStatus.java           |   22 +-
 .../security/models/RuleType.java             |   26 +-
 .../security/models/RulesResults.java         |    8 +-
 .../security/models/RulesResultsInput.java    |   26 +-
 .../resourcemanager/security/models/Scan.java |   14 +-
 .../security/models/ScanProperties.java       |   70 +-
 .../security/models/ScanResult.java           |   14 +-
 .../security/models/ScanResultProperties.java |   38 +-
 .../security/models/ScanResults.java          |    8 +-
 .../security/models/ScanState.java            |   26 +-
 .../security/models/ScanTriggerType.java      |   18 +-
 .../security/models/ScanningMode.java         |   14 +-
 .../security/models/Scans.java                |    8 +-
 .../security/models/ScopeElement.java         |   21 +-
 .../security/models/ScopeName.java            |   53 -
 .../SecureScoreControlDefinitionItem.java     |   22 +-
 .../SecureScoreControlDefinitionList.java     |   14 +-
 .../SecureScoreControlDefinitionSource.java   |   14 +-
 .../models/SecureScoreControlDefinitions.java |   12 +-
 .../models/SecureScoreControlDetails.java     |   30 +-
 .../models/SecureScoreControlList.java        |   14 +-
 .../SecureScoreControlScoreDetails.java       |   24 +-
 .../security/models/SecureScoreControls.java  |   28 +-
 .../security/models/SecureScoreItem.java      |   22 +-
 .../security/models/SecureScores.java         |   18 +-
 .../security/models/SecureScoresList.java     |   14 +-
 .../security/models/SecurityAssessment.java   |   36 +-
 .../models/SecurityAssessmentList.java        |   14 +-
 ...SecurityAssessmentMetadataPartnerData.java |   36 +-
 ...etadataPropertiesResponsePublishDates.java |   29 +-
 .../SecurityAssessmentMetadataResponse.java   |  190 +-
 ...ecurityAssessmentMetadataResponseList.java |   14 +-
 .../models/SecurityAssessmentPartnerData.java |   32 +-
 .../SecurityAssessmentPropertiesBase.java     |   36 +-
 .../models/SecurityAssessmentResponse.java    |  157 +-
 .../security/models/SecurityConnector.java    |  187 +-
 ...ecurityConnectorApplicationOperations.java |   48 +-
 .../models/SecurityConnectorApplications.java |   16 +-
 .../security/models/SecurityConnectors.java   |   50 +-
 .../models/SecurityConnectorsList.java        |   21 +-
 .../security/models/SecurityContact.java      |  118 +-
 .../security/models/SecurityContactList.java  |   39 +-
 .../security/models/SecurityContactName.java  |   48 +
 ...tyContactPropertiesAlertNotifications.java |   76 -
 ...yContactPropertiesNotificationsByRole.java |   28 +-
 .../security/models/SecurityContactRole.java  |   63 +
 .../security/models/SecurityContacts.java     |   36 +-
 .../security/models/SecurityFamily.java       |   26 +-
 .../security/models/SecurityOperator.java     |   14 +-
 .../security/models/SecurityOperatorList.java |    8 +-
 .../security/models/SecurityOperators.java    |   24 +-
 .../security/models/SecuritySolution.java     |   22 +-
 .../security/models/SecuritySolutionList.java |   16 +-
 .../models/SecuritySolutionStatus.java        |   18 +-
 .../security/models/SecuritySolutions.java    |   24 +-
 .../SecuritySolutionsReferenceData.java       |   48 +-
 .../SecuritySolutionsReferenceDataList.java   |    8 +-
 .../SecuritySolutionsReferenceDatas.java      |   16 +-
 .../models/SecuritySubAssessment.java         |   32 +-
 .../models/SecuritySubAssessmentList.java     |   14 +-
 .../security/models/SecurityTask.java         |   22 +-
 .../security/models/SecurityTaskList.java     |   14 +-
 .../models/SecurityTaskParameters.java        |   19 +-
 .../security/models/SensitivityLabel.java     |   30 +-
 .../security/models/SensitivitySettings.java  |   33 +
 .../models/ServerVulnerabilityAssessment.java |   14 +-
 ...AssessmentPropertiesProvisioningState.java |   38 +-
 .../ServerVulnerabilityAssessments.java       |   68 +-
 ...sessmentsAzureSettingSelectedProvider.java |   49 +
 .../ServerVulnerabilityAssessmentsList.java   |    8 +-
 ...ServerVulnerabilityAssessmentsSetting.java |   49 +
 ...erVulnerabilityAssessmentsSettingKind.java |   50 +
 ...lnerabilityAssessmentsSettingKindName.java |   50 +
 ...erverVulnerabilityAssessmentsSettings.java |  112 +
 ...VulnerabilityAssessmentsSettingsList.java} |   30 +-
 .../models/ServerVulnerabilityProperties.java |   37 +-
 .../models/ServicePrincipalProperties.java    |   26 +-
 .../security/models/Setting.java              |   12 +-
 .../security/models/SettingKind.java          |   22 +-
 .../security/models/SettingName.java          |   29 +-
 .../models/SettingNameAutoGenerated.java      |   69 +
 .../security/models/Settings.java             |   24 +-
 .../security/models/SettingsList.java         |   16 +-
 .../security/models/Severity.java             |   22 +-
 .../security/models/SeverityEnum.java         |   22 +-
 .../security/models/Software.java             |   30 +-
 .../security/models/SoftwareInventories.java  |   55 +-
 .../security/models/SoftwaresList.java        |   16 +-
 .../security/models/Source.java               |   22 +-
 .../security/models/SourceSystem.java         |   30 +-
 .../security/models/SourceType.java           |   53 +
 .../SqlServerVulnerabilityProperties.java     |   14 +-
 ...lVulnerabilityAssessmentBaselineRules.java |   26 +-
 ...SqlVulnerabilityAssessmentScanResults.java |   16 +-
 .../SqlVulnerabilityAssessmentScans.java      |   12 +-
 .../security/models/State.java                |   26 +-
 .../security/models/Status.java               |   18 +-
 .../security/models/StatusAutoGenerated.java  |   55 +-
 .../security/models/StatusName.java           |   22 +-
 .../security/models/StatusReason.java         |   22 +-
 .../security/models/SubAssessmentStatus.java  |   18 +-
 .../models/SubAssessmentStatusCode.java       |   22 +-
 .../security/models/SubAssessments.java       |   44 +-
 .../security/models/SubPlan.java              |   18 +-
 .../security/models/SupportedCloudEnum.java   |   18 +-
 .../models/SuppressionAlertsScope.java        |   19 +-
 .../security/models/Tactics.java              |   66 +-
 .../resourcemanager/security/models/Tags.java |   14 +-
 .../security/models/TagsResource.java         |   14 +-
 .../models/TargetBranchConfiguration.java     |   90 +
 .../security/models/TaskUpdateActionType.java |   30 +-
 .../security/models/Tasks.java                |   96 +-
 .../security/models/Techniques.java           |  462 +-
 .../security/models/Threats.java              |   42 +-
 .../models/ThresholdCustomAlertRule.java      |   24 +-
 .../models/TimeWindowCustomAlertRule.java     |   31 +-
 .../security/models/Topologies.java           |   44 +-
 .../security/models/TopologyList.java         |   14 +-
 .../security/models/TopologyResource.java     |   18 +-
 .../models/TopologySingleResource.java        |   26 +-
 .../models/TopologySingleResourceChild.java   |   12 +-
 .../models/TopologySingleResourceParent.java  |   12 +-
 .../security/models/TrackedResource.java      |   26 +-
 .../security/models/TransportProtocol.java    |   18 +-
 .../models/TwinUpdatesNotInAllowedRange.java  |   26 +-
 .../resourcemanager/security/models/Type.java |   18 +-
 ...authorizedOperationsNotInAllowedRange.java |   26 +-
 .../models/UnmaskedIpLoggingStatus.java       |   18 +-
 .../models/UpdateIotSecuritySolutionData.java |   28 +-
 .../UpdateSensitivitySettingsRequest.java     |  120 +
 .../UserDefinedResourcesProperties.java       |   30 +-
 .../security/models/UserImpact.java           |   22 +-
 .../security/models/UserRecommendation.java   |   18 +-
 .../security/models/VaRule.java               |   46 +-
 .../security/models/ValueType.java            |   18 +-
 .../security/models/VendorReference.java      |   14 +-
 .../security/models/VmRecommendation.java     |   26 +-
 .../security/models/WorkspaceSetting.java     |   86 +-
 .../security/models/WorkspaceSettingList.java |   21 +-
 .../security/models/WorkspaceSettings.java    |   26 +-
 .../security/models/package-info.java         |    4 +-
 .../security/package-info.java                |    4 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    1 +
 .../reflect-config.json                       | 2856 ++++++++
 ...ptiveApplicationControlsDeleteSamples.java |   19 +-
 ...AdaptiveApplicationControlsGetSamples.java |   19 +-
 ...daptiveApplicationControlsListSamples.java |   10 +-
 ...AdaptiveApplicationControlsPutSamples.java |  154 +-
 ...aptiveNetworkHardeningsEnforceSamples.java |   56 +-
 .../AdaptiveNetworkHardeningsGetSamples.java  |   20 +-
 ...rdeningsListByExtendedResourceSamples.java |   16 +-
 ...AdvancedThreatProtectionCreateSamples.java |   20 +-
 .../AdvancedThreatProtectionGetSamples.java   |   18 +-
 .../AlertsGetResourceGroupLevelSamples.java   |   18 +-
 .../AlertsGetSubscriptionLevelSamples.java    |   17 +-
 .../AlertsListByResourceGroupSamples.java     |    9 +-
 ...ListResourceGroupLevelByRegionSamples.java |    9 +-
 .../security/generated/AlertsListSamples.java |    9 +-
 ...sListSubscriptionLevelByRegionSamples.java |    9 +-
 .../generated/AlertsSimulateSamples.java      |   39 +-
 .../AlertsSuppressionRulesDeleteSamples.java  |   14 +-
 .../AlertsSuppressionRulesGetSamples.java     |   14 +-
 .../AlertsSuppressionRulesListSamples.java    |   16 +-
 .../AlertsSuppressionRulesUpdateSamples.java  |   53 +-
 ...ourceGroupLevelStateToActivateSamples.java |   18 +-
 ...sourceGroupLevelStateToDismissSamples.java |   18 +-
 ...rceGroupLevelStateToInProgressSamples.java |   18 +-
 ...sourceGroupLevelStateToResolveSamples.java |   18 +-
 ...bscriptionLevelStateToActivateSamples.java |   17 +-
 ...ubscriptionLevelStateToDismissSamples.java |   17 +-
 ...criptionLevelStateToInProgressSamples.java |   17 +-
 ...ubscriptionLevelStateToResolveSamples.java |   17 +-
 .../AllowedConnectionsGetSamples.java         |   15 +-
 ...wedConnectionsListByHomeRegionSamples.java |   10 +-
 .../AllowedConnectionsListSamples.java        |   14 +-
 .../generated/ApiCollectionGetSamples.java    |   21 -
 .../generated/ApiCollectionListSamples.java   |   21 -
 ...ApiCollectionOffboardingDeleteSamples.java |   23 -
 .../ApiCollectionOnboardingCreateSamples.java |   23 -
 ...GetByAzureApiManagementServiceSamples.java |   26 +
 ...istByAzureApiManagementServiceSamples.java |   26 +
 ...CollectionsListByResourceGroupSamples.java |   26 +
 .../generated/ApiCollectionsListSamples.java  |   26 +
 ...sOffboardAzureApiManagementApiSamples.java |   26 +
 ...nsOnboardAzureApiManagementApiSamples.java |   26 +
 ...icationOperationCreateOrUpdateSamples.java |   29 +-
 .../ApplicationOperationDeleteSamples.java    |   15 +-
 .../ApplicationOperationGetSamples.java       |   19 +-
 .../generated/ApplicationsListSamples.java    |   14 +-
 .../AssessmentsCreateOrUpdateSamples.java     |   24 +-
 .../generated/AssessmentsDeleteSamples.java   |   23 +-
 .../generated/AssessmentsGetSamples.java      |   36 +-
 .../generated/AssessmentsListSamples.java     |   15 +-
 ...tsMetadataCreateInSubscriptionSamples.java |   34 +-
 ...tsMetadataDeleteInSubscriptionSamples.java |   19 +-
 ...mentsMetadataGetInSubscriptionSamples.java |   19 +-
 .../AssessmentsMetadataGetSamples.java        |   15 +-
 ...entsMetadataListBySubscriptionSamples.java |   14 +-
 .../AssessmentsMetadataListSamples.java       |   10 +-
 ...AutoProvisioningSettingsCreateSamples.java |   14 +-
 .../AutoProvisioningSettingsGetSamples.java   |   14 +-
 .../AutoProvisioningSettingsListSamples.java  |   14 +-
 .../AutomationsCreateOrUpdateSamples.java     |  167 +-
 .../generated/AutomationsDeleteSamples.java   |   15 +-
 .../AutomationsGetByResourceGroupSamples.java |   16 +-
 ...AutomationsListByResourceGroupSamples.java |   10 +-
 .../generated/AutomationsListSamples.java     |   14 +-
 .../generated/AutomationsUpdateSamples.java   |   56 +
 .../generated/AutomationsValidateSamples.java |   74 +-
 .../AzureDevOpsOrgsCreateOrUpdateSamples.java |   33 +
 .../generated/AzureDevOpsOrgsGetSamples.java  |   25 +
 .../AzureDevOpsOrgsListAvailableSamples.java  |   25 +
 .../generated/AzureDevOpsOrgsListSamples.java |   24 +
 .../AzureDevOpsOrgsUpdateSamples.java         |   36 +
 ...reDevOpsProjectsCreateOrUpdateSamples.java |   33 +
 .../AzureDevOpsProjectsGetSamples.java        |   25 +
 .../AzureDevOpsProjectsListSamples.java       |   25 +
 .../AzureDevOpsProjectsUpdateSamples.java     |   35 +
 ...AzureDevOpsReposCreateOrUpdateSamples.java |   33 +
 .../generated/AzureDevOpsReposGetSamples.java |   25 +
 .../AzureDevOpsReposListSamples.java          |   25 +
 .../AzureDevOpsReposUpdateSamples.java        |   35 +
 .../ComplianceResultsGetSamples.java          |   18 +-
 .../ComplianceResultsListSamples.java         |   15 +-
 .../generated/CompliancesGetSamples.java      |   16 +-
 .../generated/CompliancesListSamples.java     |   15 +-
 .../ConnectorsCreateOrUpdateSamples.java      |   99 +-
 .../generated/ConnectorsDeleteSamples.java    |   14 +-
 .../generated/ConnectorsGetSamples.java       |   14 +-
 .../generated/ConnectorsListSamples.java      |   14 +-
 ...tomAssessmentAutomationsCreateSamples.java |   23 +-
 ...tomAssessmentAutomationsDeleteSamples.java |   16 +-
 ...tAutomationsGetByResourceGroupSamples.java |   16 +-
 ...AutomationsListByResourceGroupSamples.java |   15 +-
 ...ustomAssessmentAutomationsListSamples.java |   14 +-
 ...omEntityStoreAssignmentsCreateSamples.java |   14 +-
 ...omEntityStoreAssignmentsDeleteSamples.java |   16 +-
 ...eAssignmentsGetByResourceGroupSamples.java |   16 +-
 ...AssignmentsListByResourceGroupSamples.java |   15 +-
 ...stomEntityStoreAssignmentsListSamples.java |   14 +-
 .../DefenderForStorageCreateSamples.java      |   33 +
 .../DefenderForStorageGetSamples.java         |   29 +
 ...psConfigurationsCreateOrUpdateSamples.java |   73 +
 .../DevOpsConfigurationsDeleteSamples.java    |   24 +
 .../DevOpsConfigurationsGetSamples.java       |   25 +
 .../DevOpsConfigurationsListSamples.java      |   24 +
 .../DevOpsConfigurationsUpdateSamples.java    |   31 +
 .../DevOpsOperationResultsGetSamples.java     |   40 +
 ...ceSecurityGroupsCreateOrUpdateSamples.java |   27 +-
 .../DeviceSecurityGroupsDeleteSamples.java    |   19 +-
 .../DeviceSecurityGroupsGetSamples.java       |   19 +-
 .../DeviceSecurityGroupsListSamples.java      |   18 +-
 ...DiscoveredSecuritySolutionsGetSamples.java |   15 +-
 ...uritySolutionsListByHomeRegionSamples.java |   10 +-
 ...iscoveredSecuritySolutionsListSamples.java |   10 +-
 .../ExternalSecuritySolutionsGetSamples.java  |   19 +-
 ...uritySolutionsListByHomeRegionSamples.java |   10 +-
 .../ExternalSecuritySolutionsListSamples.java |   14 +-
 .../generated/GitHubOwnersGetSamples.java     |   25 +
 .../GitHubOwnersListAvailableSamples.java     |   25 +
 .../generated/GitHubOwnersListSamples.java    |   24 +
 .../generated/GitHubReposGetSamples.java      |   25 +
 .../generated/GitHubReposListSamples.java     |   25 +
 .../generated/GitLabGroupsGetSamples.java     |   25 +
 .../GitLabGroupsListAvailableSamples.java     |   25 +
 .../generated/GitLabGroupsListSamples.java    |   24 +
 .../generated/GitLabProjectsGetSamples.java   |   25 +
 .../generated/GitLabProjectsListSamples.java  |   25 +
 .../generated/GitLabSubgroupsListSamples.java |   25 +
 ...nanceAssignmentsCreateOrUpdateSamples.java |   39 +-
 .../GovernanceAssignmentsDeleteSamples.java   |   21 +-
 .../GovernanceAssignmentsGetSamples.java      |   21 +-
 .../GovernanceAssignmentsListSamples.java     |   18 +-
 .../GovernanceRulesCreateOrUpdateSamples.java |  138 +-
 .../GovernanceRulesDeleteSamples.java         |   59 +-
 .../GovernanceRulesExecuteSamples.java        |   62 +-
 .../generated/GovernanceRulesGetSamples.java  |   59 +-
 .../generated/GovernanceRulesListSamples.java |   52 +-
 ...overnanceRulesOperationResultsSamples.java |   53 +-
 .../HealthReportOperationGetSamples.java      |   25 -
 .../generated/HealthReportsGetSamples.java    |   26 +
 .../generated/HealthReportsListSamples.java   |   15 +-
 ...otectionPoliciesCreateOrUpdateSamples.java |   63 +-
 ...formationProtectionPoliciesGetSamples.java |   34 +-
 ...ormationProtectionPoliciesListSamples.java |   18 +-
 .../IngestionSettingsCreateSamples.java       |   21 -
 .../IngestionSettingsDeleteSamples.java       |   21 -
 .../IngestionSettingsGetSamples.java          |   20 -
 ...nSettingsListConnectionStringsSamples.java |   21 -
 .../IngestionSettingsListSamples.java         |   20 -
 .../IngestionSettingsListTokensSamples.java   |   20 -
 ...otSecuritySolutionAnalyticsGetSamples.java |    9 +-
 ...tSecuritySolutionAnalyticsListSamples.java |    9 +-
 ...SecuritySolutionCreateOrUpdateSamples.java |   52 +-
 .../IotSecuritySolutionDeleteSamples.java     |   15 +-
 ...ritySolutionGetByResourceGroupSamples.java |   15 +-
 ...itySolutionListByResourceGroupSamples.java |   34 +-
 .../IotSecuritySolutionListSamples.java       |   29 +-
 .../IotSecuritySolutionUpdateSamples.java     |   48 +-
 ...nalyticsAggregatedAlertDismissSamples.java |   22 +-
 ...onsAnalyticsAggregatedAlertGetSamples.java |   14 +-
 ...nsAnalyticsAggregatedAlertListSamples.java |   14 +-
 ...ionsAnalyticsRecommendationGetSamples.java |   14 +-
 ...onsAnalyticsRecommendationListSamples.java |   14 +-
 ...rkAccessPoliciesCreateOrUpdateSamples.java |   72 +-
 ...JitNetworkAccessPoliciesDeleteSamples.java |   15 +-
 .../JitNetworkAccessPoliciesGetSamples.java   |   15 +-
 ...tNetworkAccessPoliciesInitiateSamples.java |   43 +-
 ...workAccessPoliciesListByRegionSamples.java |   10 +-
 ...esListByResourceGroupAndRegionSamples.java |   15 +-
 ...essPoliciesListByResourceGroupSamples.java |   14 +-
 .../JitNetworkAccessPoliciesListSamples.java  |   14 +-
 .../generated/LocationsGetSamples.java        |   10 +-
 .../generated/LocationsListSamples.java       |   10 +-
 .../generated/MdeOnboardingsGetSamples.java   |   10 +-
 .../generated/MdeOnboardingsListSamples.java  |   10 +-
 .../generated/PricingsDeleteSamples.java      |   26 +
 .../generated/PricingsGetSamples.java         |   82 +-
 .../generated/PricingsListSamples.java        |   45 +-
 .../generated/PricingsUpdateSamples.java      |   77 +-
 ...latoryComplianceAssessmentsGetSamples.java |   16 +-
 ...atoryComplianceAssessmentsListSamples.java |   10 +-
 ...egulatoryComplianceControlsGetSamples.java |   10 +-
 ...gulatoryComplianceControlsListSamples.java |   10 +-
 ...gulatoryComplianceStandardsGetSamples.java |   10 +-
 ...ulatoryComplianceStandardsListSamples.java |   10 +-
 ...ProviderGetSensitivitySettingsSamples.java |   24 +
 ...viderUpdateSensitivitySettingsSamples.java |   35 +
 ...lDefinitionsListBySubscriptionSamples.java |   13 +-
 ...ureScoreControlDefinitionsListSamples.java |    9 +-
 ...ScoreControlsListBySecureScoreSamples.java |   21 +-
 .../SecureScoreControlsListSamples.java       |   10 +-
 .../generated/SecureScoresGetSamples.java     |   10 +-
 .../generated/SecureScoresListSamples.java    |   10 +-
 ...icationOperationCreateOrUpdateSamples.java |   39 +-
 ...ctorApplicationOperationDeleteSamples.java |   19 +-
 ...nnectorApplicationOperationGetSamples.java |   23 +-
 ...urityConnectorApplicationsListSamples.java |   15 +-
 ...curityConnectorsCreateOrUpdateSamples.java |   35 +-
 .../SecurityConnectorsDeleteSamples.java      |   15 +-
 ...tyConnectorsGetByResourceGroupSamples.java |   16 +-
 ...yConnectorsListByResourceGroupSamples.java |   14 +-
 .../SecurityConnectorsListSamples.java        |   14 +-
 .../SecurityConnectorsUpdateSamples.java      |   37 +-
 .../SecurityContactsCreateSamples.java        |   37 +-
 .../SecurityContactsDeleteSamples.java        |   14 +-
 .../generated/SecurityContactsGetSamples.java |   14 +-
 .../SecurityContactsListSamples.java          |   10 +-
 ...ecurityOperatorsCreateOrUpdateSamples.java |   20 +-
 .../SecurityOperatorsDeleteSamples.java       |   20 +-
 .../SecurityOperatorsGetSamples.java          |   15 +-
 .../SecurityOperatorsListSamples.java         |   10 +-
 .../SecuritySolutionsGetSamples.java          |   19 +-
 .../SecuritySolutionsListSamples.java         |   10 +-
 ...sReferenceDataListByHomeRegionSamples.java |   18 +-
 ...ritySolutionsReferenceDataListSamples.java |    9 +-
 .../SensitivitySettingsListSamples.java       |   24 +
 ...bilityAssessmentCreateOrUpdateSamples.java |   15 +-
 ...rVulnerabilityAssessmentDeleteSamples.java |   14 +-
 ...rverVulnerabilityAssessmentGetSamples.java |   14 +-
 ...sessmentListByExtendedResourceSamples.java |   15 +-
 ...essmentsSettingsCreateOrUpdateSamples.java |   31 +
 ...ilityAssessmentsSettingsDeleteSamples.java |   27 +
 ...rabilityAssessmentsSettingsGetSamples.java |   28 +
 ...abilityAssessmentsSettingsListSamples.java |   24 +
 .../generated/SettingsGetSamples.java         |   14 +-
 .../generated/SettingsListSamples.java        |   10 +-
 .../generated/SettingsUpdateSamples.java      |   18 +-
 .../SoftwareInventoriesGetSamples.java        |   25 +-
 ...entoriesListByExtendedResourceSamples.java |   20 +-
 .../SoftwareInventoriesListSamples.java       |   10 +-
 ...lityAssessmentBaselineRulesAddSamples.java |   47 +-
 ...entBaselineRulesCreateOrUpdateSamples.java |   41 +-
 ...yAssessmentBaselineRulesDeleteSamples.java |   21 +-
 ...lityAssessmentBaselineRulesGetSamples.java |   21 +-
 ...ityAssessmentBaselineRulesListSamples.java |   19 +-
 ...bilityAssessmentScanResultsGetSamples.java |   40 +-
 ...ilityAssessmentScanResultsListSamples.java |   38 +-
 ...ulnerabilityAssessmentScansGetSamples.java |   37 +-
 ...lnerabilityAssessmentScansListSamples.java |   19 +-
 .../generated/SubAssessmentsGetSamples.java   |   21 +-
 .../SubAssessmentsListAllSamples.java         |   15 +-
 .../generated/SubAssessmentsListSamples.java  |   18 +-
 ...TasksGetResourceGroupLevelTaskSamples.java |   20 +-
 .../TasksGetSubscriptionLevelTaskSamples.java |   16 +-
 .../TasksListByHomeRegionSamples.java         |   10 +-
 .../TasksListByResourceGroupSamples.java      |   14 +-
 .../security/generated/TasksListSamples.java  |   10 +-
 ...ateResourceGroupLevelTaskStateSamples.java |   24 +-
 ...dateSubscriptionLevelTaskStateSamples.java |   23 +-
 .../generated/TopologyGetSamples.java         |   10 +-
 .../TopologyListByHomeRegionSamples.java      |   14 +-
 .../generated/TopologyListSamples.java        |   10 +-
 .../WorkspaceSettingsCreateSamples.java       |   24 +-
 .../WorkspaceSettingsDeleteSamples.java       |   14 +-
 .../WorkspaceSettingsGetSamples.java          |   10 +-
 .../WorkspaceSettingsListSamples.java         |   10 +-
 .../WorkspaceSettingsUpdateSamples.java       |   24 +-
 ...adConnectivityStateAutoGeneratedTests.java |   14 +-
 .../AadExternalSecuritySolutionTests.java     |   32 +-
 .../generated/AadSolutionPropertiesTests.java |   33 +-
 .../generated/ActionableRemediationTests.java |   55 +
 ...tiveConnectionsNotInAllowedRangeTests.java |   33 +-
 ...ptiveApplicationControlGroupDataTests.java |  179 +-
 ...tiveApplicationControlGroupInnerTests.java |  185 +-
 ...iveApplicationControlGroupsInnerTests.java |  155 +-
 ...veApplicationControlIssueSummaryTests.java |   19 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...ationControlsGetWithResponseMockTests.java |   71 +-
 ...tionControlsListWithResponseMockTests.java |   48 +-
 ...ationControlsPutWithResponseMockTests.java |  112 +-
 ...veNetworkHardeningEnforceRequestTests.java |   49 +-
 .../AdaptiveNetworkHardeningInnerTests.java   |   89 +-
 ...aptiveNetworkHardeningPropertiesTests.java |   92 +-
 ...tiveNetworkHardeningsEnforceMockTests.java |   60 +-
 ...orkHardeningsGetWithResponseMockTests.java |   52 +-
 ...eningsListByExtendedResourceMockTests.java |   56 +-
 .../AdaptiveNetworkHardeningsListTests.java   |  139 +-
 .../generated/AdditionalDataTests.java        |    4 +-
 .../AdditionalWorkspacesPropertiesTests.java  |   21 +-
 ...vancedThreatProtectionPropertiesTests.java |   10 +-
 ...ncedThreatProtectionSettingInnerTests.java |    9 +-
 ...rotectionsCreateWithResponseMockTests.java |   37 +-
 ...atProtectionsGetWithResponseMockTests.java |   37 +-
 .../security/generated/AlertEntityTests.java  |   12 +-
 ...lertPropertiesSupportingEvidenceTests.java |   11 +-
 ...imulatorBundlesRequestPropertiesTests.java |   16 +-
 .../AlertSimulatorRequestBodyTests.java       |   17 +-
 .../AlertSimulatorRequestPropertiesTests.java |   13 +-
 .../AlertSyncSettingPropertiesTests.java      |    4 +-
 .../generated/AlertSyncSettingsTests.java     |    8 +-
 .../AlertsSuppressionRuleInnerTests.java      |   61 +-
 .../AlertsSuppressionRulePropertiesTests.java |   52 +-
 ...ssionRulesDeleteWithResponseMockTests.java |   36 +-
 ...pressionRulesGetWithResponseMockTests.java |   46 +-
 .../AlertsSuppressionRulesListMockTests.java  |   50 +-
 .../AlertsSuppressionRulesListTests.java      |   96 +-
 ...ssionRulesUpdateWithResponseMockTests.java |   80 +-
 ...lStateToActivateWithResponseMockTests.java |   36 +-
 ...elStateToDismissWithResponseMockTests.java |   36 +-
 ...tateToInProgressWithResponseMockTests.java |   36 +-
 ...elStateToResolveWithResponseMockTests.java |   36 +-
 ...lStateToActivateWithResponseMockTests.java |   39 +-
 ...elStateToDismissWithResponseMockTests.java |   36 +-
 ...tateToInProgressWithResponseMockTests.java |   39 +-
 ...elStateToResolveWithResponseMockTests.java |   36 +-
 ...edConnectionsGetWithResponseMockTests.java |   42 +-
 ...dConnectionsListByHomeRegionMockTests.java |   38 +-
 .../AllowedConnectionsListMockTests.java      |   38 +-
 .../AllowedConnectionsListTests.java          |    8 +-
 .../AllowedConnectionsResourceInnerTests.java |    8 +-
 ...wedConnectionsResourcePropertiesTests.java |    8 +-
 .../AllowlistCustomAlertRuleTests.java        |   18 +-
 ...AmqpC2DMessagesNotInAllowedRangeTests.java |   28 +-
 ...ejectedMessagesNotInAllowedRangeTests.java |   33 +-
 ...AmqpD2CMessagesNotInAllowedRangeTests.java |   32 +-
 .../generated/ApiCollectionInnerTests.java    |   23 +
 .../generated/ApiCollectionListTests.java     |   23 +
 ...fboardingsDeleteWithResponseMockTests.java |   63 -
 ...nboardingsCreateWithResponseMockTests.java |   71 -
 .../ApiCollectionPropertiesTests.java         |   33 +-
 .../ApiCollectionResponseInnerTests.java      |   46 -
 .../ApiCollectionResponseListTests.java       |   25 -
 ...anagementServiceWithResponseMockTests.java |   55 +
 ...piCollectionsGetWithResponseMockTests.java |   71 -
 ...tByAzureApiManagementServiceMockTests.java |   55 +
 ...lectionsListByResourceGroupMockTests.java} |   21 +-
 .../ApiCollectionsListMockTests.java          |   41 +-
 ...ApiManagementApiWithResponseMockTests.java |   52 +
 ...OnboardAzureApiManagementApiMockTests.java |   54 +
 .../generated/ApplicationInnerTests.java      |   25 +-
 ...nsCreateOrUpdateWithResponseMockTests.java |   56 +-
 ...OperationsDeleteWithResponseMockTests.java |   36 +-
 ...ionOperationsGetWithResponseMockTests.java |   41 +-
 .../generated/ApplicationPropertiesTests.java |   25 +-
 .../generated/ApplicationsListMockTests.java  |   41 +-
 .../generated/ApplicationsListTests.java      |    8 +-
 .../generated/AscLocationInnerTests.java      |   11 +-
 .../generated/AscLocationListTests.java       |    8 +-
 .../generated/AssessmentLinksTests.java       |    4 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...teInSubscriptionWithResponseMockTests.java |   37 +-
 .../AtaExternalSecuritySolutionTests.java     |   35 +-
 .../generated/AtaSolutionPropertiesTests.java |   33 +-
 .../AuthenticationDetailsPropertiesTests.java |    8 +-
 .../AutoProvisioningSettingInnerTests.java    |   14 +-
 .../AutoProvisioningSettingListTests.java     |   19 +-
 ...utoProvisioningSettingPropertiesTests.java |   12 +-
 ...ngSettingsCreateWithResponseMockTests.java |   37 +-
 ...oningSettingsGetWithResponseMockTests.java |   40 +-
 ...AutoProvisioningSettingsListMockTests.java |   39 +-
 .../AutomationActionEventHubTests.java        |   24 +-
 .../AutomationActionLogicAppTests.java        |   19 +-
 .../generated/AutomationActionTests.java      |    4 +-
 .../AutomationActionWorkspaceTests.java       |   11 +-
 .../generated/AutomationInnerTests.java       |   77 +-
 .../generated/AutomationListTests.java        |  136 +-
 .../generated/AutomationPropertiesTests.java  |   98 +-
 .../generated/AutomationRuleSetTests.java     |   37 +-
 .../generated/AutomationScopeTests.java       |   16 +-
 .../generated/AutomationSourceTests.java      |   52 +-
 .../AutomationTriggeringRuleTests.java        |   32 +-
 .../generated/AutomationUpdateModelTests.java |   71 +
 .../AutomationValidationStatusInnerTests.java |   17 +-
 ...nsCreateOrUpdateWithResponseMockTests.java |   81 +-
 ...eByResourceGroupWithResponseMockTests.java |   41 +-
 ...tByResourceGroupWithResponseMockTests.java |   53 +-
 ...tomationsListByResourceGroupMockTests.java |   47 +-
 .../generated/AutomationsListMockTests.java   |   45 +-
 ...omationsValidateWithResponseMockTests.java |   67 +-
 ...eAuthenticationDetailsPropertiesTests.java |   21 +-
 .../generated/AwsEnvironmentDataTests.java    |   20 +-
 .../AwsOrganizationalDataMasterTests.java     |   22 +-
 .../AwsOrganizationalDataMemberTests.java     |   13 +-
 .../generated/AwsOrganizationalDataTests.java |    5 +-
 .../generated/AzureDevOpsOrgInnerTests.java   |   74 +
 .../AzureDevOpsOrgListResponseInnerTests.java |   96 +
 .../AzureDevOpsOrgPropertiesTests.java        |   67 +
 ...zureDevOpsOrgsCreateOrUpdateMockTests.java |   94 +
 ...ureDevOpsOrgsGetWithResponseMockTests.java |   76 +
 ...rgsListAvailableWithResponseMockTests.java |   68 +
 .../AzureDevOpsOrgsListMockTests.java         |   78 +
 .../AzureDevOpsProjectInnerTests.java         |   79 +
 .../AzureDevOpsProjectListResponseTests.java  |   99 +
 .../AzureDevOpsProjectPropertiesTests.java    |   73 +
 ...DevOpsProjectsCreateOrUpdateMockTests.java |   93 +
 ...evOpsProjectsGetWithResponseMockTests.java |   77 +
 .../AzureDevOpsProjectsListMockTests.java     |   79 +
 ...ureDevOpsReposCreateOrUpdateMockTests.java |  100 +
 ...reDevOpsReposGetWithResponseMockTests.java |   78 +
 .../AzureDevOpsReposListMockTests.java        |   80 +
 .../AzureDevOpsRepositoryInnerTests.java      |   76 +
 ...zureDevOpsRepositoryListResponseTests.java |  108 +
 .../AzureDevOpsRepositoryPropertiesTests.java |   74 +
 .../AzureDevOpsScopeEnvironmentDataTests.java |    6 +-
 .../generated/AzureResourceDetailsTests.java  |    4 +-
 .../AzureResourceIdentifierTests.java         |    5 +-
 .../generated/AzureResourceLinkTests.java     |    4 +-
 .../generated/AzureServersSettingTests.java   |   30 +
 .../AzureTrackedResourceLocationTests.java    |   10 +-
 .../BaselineAdjustedResultTests.java          |   61 +-
 .../security/generated/BaselineTests.java     |   28 +-
 .../generated/BenchmarkReferenceTests.java    |   15 +-
 .../generated/BuiltInInfoTypeTests.java       |   32 +
 .../generated/CategoryConfigurationTests.java |   30 +
 .../CefExternalSecuritySolutionTests.java     |   46 +-
 .../generated/CefSolutionPropertiesTests.java |   43 +-
 .../generated/CloudOfferingTests.java         |    6 +-
 .../generated/ComplianceInnerTests.java       |    8 +-
 .../generated/ComplianceListTests.java        |   19 +-
 .../generated/CompliancePropertiesTests.java  |    8 +-
 .../generated/ComplianceResultInnerTests.java |    8 +-
 .../generated/ComplianceResultListTests.java  |   13 +-
 .../ComplianceResultPropertiesTests.java      |    4 +-
 ...lianceResultsGetWithResponseMockTests.java |   38 +-
 .../ComplianceResultsListMockTests.java       |   38 +-
 .../generated/ComplianceSegmentTests.java     |    5 +-
 .../CompliancesGetWithResponseMockTests.java  |   38 +-
 .../generated/CompliancesListMockTests.java   |   37 +-
 .../generated/ConnectableResourceTests.java   |    8 +-
 .../generated/ConnectedResourceTests.java     |    7 +-
 .../generated/ConnectedWorkspaceTests.java    |    9 +-
 .../ConnectionFromIpNotAllowedTests.java      |   22 +-
 .../ConnectionStringsInnerTests.java          |   29 -
 .../ConnectionToIpNotAllowedTests.java        |   18 +-
 .../generated/ConnectorSettingInnerTests.java |   42 -
 .../generated/ConnectorSettingListTests.java  |   35 -
 ...rsCreateOrUpdateWithResponseMockTests.java |   81 -
 ...ConnectorsDeleteWithResponseMockTests.java |   36 +-
 .../ConnectorsGetWithResponseMockTests.java   |   70 -
 .../generated/ConnectorsListMockTests.java    |   70 -
 ...rRegistryVulnerabilityPropertiesTests.java |    8 +-
 ...AwsOfferingNativeCloudConnectionTests.java |   13 +-
 .../CspmMonitorAwsOfferingTests.java          |   18 +-
 .../CspmMonitorAzureDevOpsOfferingTests.java  |    5 +-
 ...GcpOfferingNativeCloudConnectionTests.java |   21 +-
 .../CspmMonitorGcpOfferingTests.java          |   25 +-
 .../CspmMonitorGitLabOfferingTests.java       |    5 +-
 .../CspmMonitorGithubOfferingTests.java       |    5 +-
 .../generated/CustomAlertRuleTests.java       |   14 +-
 ...smentAutomationRequestPropertiesTests.java |   44 +-
 ...ustomAssessmentAutomationRequestTests.java |   40 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...CustomEntityStoreAssignmentInnerTests.java |   22 +-
 ...mEntityStoreAssignmentPropertiesTests.java |   19 +-
 ...StoreAssignmentRequestPropertiesTests.java |   14 +-
 ...stomEntityStoreAssignmentRequestTests.java |   12 +-
 ...ssignmentsCreateWithResponseMockTests.java |   46 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...tByResourceGroupWithResponseMockTests.java |   44 +-
 ...signmentsListByResourceGroupMockTests.java |   41 +-
 ...omEntityStoreAssignmentsListMockTests.java |   41 +-
 ...EntityStoreAssignmentsListResultTests.java |   16 +-
 .../security/generated/CveTests.java          |    2 +-
 .../security/generated/CvssTests.java         |    2 +-
 .../DataExportSettingPropertiesTests.java     |    4 +-
 .../generated/DataExportSettingsTests.java    |    8 +-
 ...nderCspmAwsOfferingCiemDiscoveryTests.java |   25 +
 .../DefenderCspmAwsOfferingCiemOidcTests.java |   29 +
 .../DefenderCspmAwsOfferingCiemTests.java     |   35 +
 ...OfferingDataSensitivityDiscoveryTests.java |   17 +-
 ...nderCspmAwsOfferingDatabasesDspmTests.java |   17 +-
 ...cContainersAgentlessDiscoveryK8STests.java |   30 +
 ...ringMdcContainersImageAssessmentTests.java |   29 +
 .../DefenderCspmAwsOfferingTests.java         |   83 +-
 ...sOfferingVmScannersConfigurationTests.java |   26 +-
 ...efenderCspmAwsOfferingVmScannersTests.java |   38 +-
 ...nderCspmGcpOfferingCiemDiscoveryTests.java |   32 +
 ...OfferingDataSensitivityDiscoveryTests.java |   32 +
 ...cContainersAgentlessDiscoveryK8STests.java |   33 +
 ...ringMdcContainersImageAssessmentTests.java |   32 +
 .../DefenderCspmGcpOfferingTests.java         |   78 +-
 ...pOfferingVmScannersConfigurationTests.java |   46 +
 ...efenderCspmGcpOfferingVmScannersTests.java |   49 +
 ...ArcAutoProvisioningConfigurationTests.java |   24 +-
 ...esAwsOfferingArcAutoProvisioningTests.java |   32 +-
 ...atabasesAwsOfferingDatabasesDspmTests.java |   13 +-
 ...efenderFoDatabasesAwsOfferingRdsTests.java |   17 +-
 .../DefenderFoDatabasesAwsOfferingTests.java  |   55 +-
 ...rsAwsOfferingCloudWatchToKinesisTests.java |   13 +-
 ...ainerVulnerabilityAssessmentTaskTests.java |   21 +-
 ...ContainerVulnerabilityAssessmentTests.java |   19 +-
 ...ContainersAwsOfferingKinesisToS3Tests.java |   13 +-
 ...AwsOfferingKubernetesScubaReaderTests.java |   13 +-
 ...nersAwsOfferingKubernetesServiceTests.java |   13 +-
 ...cContainersAgentlessDiscoveryK8STests.java |   31 +
 ...ringMdcContainersImageAssessmentTests.java |   31 +
 ...DefenderForContainersAwsOfferingTests.java |   95 +-
 ...ataPipelineNativeCloudConnectionTests.java |   27 +-
 ...cContainersAgentlessDiscoveryK8STests.java |   34 +
 ...ringMdcContainersImageAssessmentTests.java |   34 +
 ...GcpOfferingNativeCloudConnectionTests.java |   23 +-
 ...DefenderForContainersGcpOfferingTests.java |   78 +-
 ...ArcAutoProvisioningConfigurationTests.java |   26 +-
 ...esGcpOfferingArcAutoProvisioningTests.java |   31 +-
 ...rForDatabasesArcAutoProvisioningTests.java |   29 +-
 .../DefenderForDatabasesGcpOfferingTests.java |   50 +-
 ...nderForDevOpsAzureDevOpsOfferingTests.java |    7 +-
 .../DefenderForDevOpsGitLabOfferingTests.java |    5 +-
 .../DefenderForDevOpsGithubOfferingTests.java |    5 +-
 ...ArcAutoProvisioningConfigurationTests.java |   24 +-
 ...rsAwsOfferingArcAutoProvisioningTests.java |   32 +-
 ...ersAwsOfferingDefenderForServersTests.java |   14 +-
 ...rsAwsOfferingMdeAutoProvisioningTests.java |    9 +-
 ...nderForServersAwsOfferingSubPlanTests.java |    4 +-
 .../DefenderForServersAwsOfferingTests.java   |   99 +-
 ...gVaAutoProvisioningConfigurationTests.java |   17 +-
 ...ersAwsOfferingVaAutoProvisioningTests.java |   17 +-
 ...sOfferingVmScannersConfigurationTests.java |   34 +-
 ...rForServersAwsOfferingVmScannersTests.java |   30 +-
 ...ArcAutoProvisioningConfigurationTests.java |   24 +-
 ...rsGcpOfferingArcAutoProvisioningTests.java |   27 +-
 ...ersGcpOfferingDefenderForServersTests.java |   21 +-
 ...rsGcpOfferingMdeAutoProvisioningTests.java |   13 +-
 ...nderForServersGcpOfferingSubPlanTests.java |   10 +-
 .../DefenderForServersGcpOfferingTests.java   |   99 +-
 ...gVaAutoProvisioningConfigurationTests.java |   18 +-
 ...ersGcpOfferingVaAutoProvisioningTests.java |   13 +-
 ...pOfferingVmScannersConfigurationTests.java |   22 +-
 ...rForServersGcpOfferingVmScannersTests.java |   28 +-
 .../DenylistCustomAlertRuleTests.java         |   22 +-
 .../DevOpsConfigurationsDeleteMockTests.java  |   51 +
 ...rationResultsGetWithResponseMockTests.java |   76 +
 .../DeviceSecurityGroupInnerTests.java        |   96 +-
 .../DeviceSecurityGroupListTests.java         |  121 +-
 .../DeviceSecurityGroupPropertiesTests.java   |   94 +-
 ...psCreateOrUpdateWithResponseMockTests.java |   85 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...ecurityGroupsGetWithResponseMockTests.java |   54 +-
 .../DeviceSecurityGroupsListMockTests.java    |   54 +-
 ...ctMethodInvokesNotInAllowedRangeTests.java |   32 +-
 .../DiscoveredSecuritySolutionInnerTests.java |   32 +-
 .../DiscoveredSecuritySolutionListTests.java  |   52 +-
 ...overedSecuritySolutionPropertiesTests.java |   30 +-
 ...ritySolutionsGetWithResponseMockTests.java |   54 +-
 ...itySolutionsListByHomeRegionMockTests.java |   51 +-
 ...coveredSecuritySolutionsListMockTests.java |   43 +-
 .../EffectiveNetworkSecurityGroupsTests.java  |   23 +-
 .../generated/EnvironmentDataTests.java       |    4 +-
 .../generated/EnvironmentDetailsTests.java    |   38 +-
 .../security/generated/EtagTests.java         |    8 +-
 .../ExecuteGovernanceRuleParamsTests.java     |    4 +-
 .../ExternalSecuritySolutionInnerTests.java   |    8 +-
 .../ExternalSecuritySolutionListTests.java    |   13 +-
 ...ternalSecuritySolutionPropertiesTests.java |   30 +-
 ...ritySolutionsGetWithResponseMockTests.java |   41 +-
 ...itySolutionsListByHomeRegionMockTests.java |   38 +-
 ...xternalSecuritySolutionsListMockTests.java |   38 +-
 ...iledLocalLoginsNotInAllowedRangeTests.java |   28 +-
 .../FileUploadsNotInAllowedRangeTests.java    |   32 +-
 .../GcpOrganizationalDataMemberTests.java     |   22 +-
 ...cpOrganizationalDataOrganizationTests.java |   28 +-
 .../generated/GcpOrganizationalDataTests.java |    5 +-
 .../generated/GcpProjectDetailsTests.java     |   19 +-
 .../GcpProjectEnvironmentDataTests.java       |   26 +-
 ...itivitySettingsListResponseInnerTests.java |   72 +
 ...SensitivitySettingsResponseInnerTests.java |   86 +
 ...ResponsePropertiesMipInformationTests.java |   74 +
 ...tivitySettingsResponsePropertiesTests.java |   92 +
 .../generated/GitHubOwnerInnerTests.java      |   33 +
 .../GitHubOwnerListResponseInnerTests.java    |   49 +
 .../generated/GitHubOwnerPropertiesTests.java |   32 +
 ...GitHubOwnersGetWithResponseMockTests.java} |   24 +-
 ...rsListAvailableWithResponseMockTests.java} |   29 +-
 .../generated/GitHubOwnersListMockTests.java} |   28 +-
 .../GitHubReposGetWithResponseMockTests.java} |   26 +-
 .../generated/GitHubReposListMockTests.java   |   62 +
 .../generated/GitHubRepositoryInnerTests.java |   35 +
 .../GitHubRepositoryListResponseTests.java    |   56 +
 .../GitHubRepositoryPropertiesTests.java      |   34 +
 .../generated/GitLabGroupInnerTests.java      |   33 +
 .../GitLabGroupListResponseInnerTests.java    |   41 +
 .../generated/GitLabGroupPropertiesTests.java |   32 +
 ...GitLabGroupsGetWithResponseMockTests.java} |   24 +-
 ...upsListAvailableWithResponseMockTests.java |   61 +
 .../generated/GitLabGroupsListMockTests.java  |   62 +
 .../generated/GitLabProjectInnerTests.java    |   33 +
 .../GitLabProjectListResponseTests.java       |   50 +
 .../GitLabProjectPropertiesTests.java         |   32 +
 ...itLabProjectsGetWithResponseMockTests.java |   60 +
 .../GitLabProjectsListMockTests.java}         |   28 +-
 ...LabSubgroupsListWithResponseMockTests.java |   63 +
 .../GithubScopeEnvironmentDataTests.java      |    4 +-
 .../GitlabScopeEnvironmentDataTests.java      |    4 +-
 ...vernanceAssignmentAdditionalDataTests.java |   24 +-
 .../GovernanceAssignmentInnerTests.java       |   67 +-
 .../GovernanceAssignmentPropertiesTests.java  |   75 +-
 ...tsCreateOrUpdateWithResponseMockTests.java |   84 +-
 ...ssignmentsDeleteWithResponseMockTests.java |   36 +-
 ...ceAssignmentsGetWithResponseMockTests.java |   56 +-
 .../GovernanceAssignmentsListMockTests.java   |   69 +-
 .../GovernanceAssignmentsListTests.java       |    8 +-
 .../GovernanceEmailNotificationTests.java     |   13 +-
 .../GovernanceRuleEmailNotificationTests.java |   17 +-
 .../generated/GovernanceRuleInnerTests.java   |   74 +-
 .../generated/GovernanceRuleListTests.java    |    8 +-
 .../GovernanceRuleMetadataTests.java          |    8 +-
 .../GovernanceRuleOwnerSourceTests.java       |   18 +-
 .../GovernanceRulePropertiesTests.java        |   77 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   91 +-
 .../GovernanceRulesDeleteMockTests.java       |   36 +-
 ...vernanceRulesGetWithResponseMockTests.java |   56 +-
 .../GovernanceRulesListMockTests.java         |   69 +-
 ...OperationResultsWithResponseMockTests.java |   39 +-
 .../HealthDataClassificationTests.java        |   25 +-
 .../generated/HealthReportsListTests.java     |   25 -
 ...HttpC2DMessagesNotInAllowedRangeTests.java |   33 +-
 ...ejectedMessagesNotInAllowedRangeTests.java |   29 +-
 ...HttpD2CMessagesNotInAllowedRangeTests.java |   28 +-
 .../security/generated/IdentityTests.java     |    7 +-
 .../security/generated/InfoTypeTests.java     |   33 +
 ...AwsOfferingInformationProtectionTests.java |   13 +-
 ...InformationProtectionAwsOfferingTests.java |   18 +-
 .../InformationProtectionKeywordTests.java    |   24 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   84 -
 ...ctionPoliciesGetWithResponseMockTests.java |   68 -
 ...mationProtectionPoliciesListMockTests.java |   65 -
 .../InformationProtectionPolicyListTests.java |   46 -
 .../IngestionConnectionStringTests.java       |   24 -
 .../generated/IngestionSettingInnerTests.java |   24 -
 .../generated/IngestionSettingListTests.java  |   25 -
 ...onSettingsCreateWithResponseMockTests.java |   63 -
 ...onSettingsDeleteWithResponseMockTests.java |   61 -
 ...stionSettingsGetWithResponseMockTests.java |   64 -
 ...onnectionStringsWithResponseMockTests.java |   67 -
 .../IngestionSettingsListMockTests.java       |   64 -
 .../IoTSecurityAggregatedAlertInnerTests.java |   17 +-
 .../IoTSecurityAggregatedAlertListTests.java  |   24 +-
 ...ecurityAggregatedAlertPropertiesTests.java |    8 +-
 ...lertPropertiesTopDevicesListItemTests.java |   12 +-
 ...ityAggregatedRecommendationInnerTests.java |   32 +-
 ...rityAggregatedRecommendationListTests.java |   46 +-
 ...gregatedRecommendationPropertiesTests.java |   16 +-
 .../IoTSecurityAlertedDeviceTests.java        |    5 +-
 .../IoTSecurityDeviceAlertTests.java          |    8 +-
 .../IoTSecurityDeviceRecommendationTests.java |    8 +-
 ...uritySolutionAnalyticsModelInnerTests.java |   31 +-
 ...ySolutionAnalyticsModelListInnerTests.java |   31 +-
 ...odelPropertiesDevicesMetricsItemTests.java |   40 +-
 ...SolutionAnalyticsModelPropertiesTests.java |   34 +-
 .../IoTSecuritySolutionModelInnerTests.java   |  138 +-
 .../IoTSecuritySolutionPropertiesTests.java   |  165 +-
 .../IoTSecuritySolutionsListTests.java        |  184 +-
 .../generated/IoTSeverityMetricsTests.java    |   26 +-
 ...tionAnalyticsGetWithResponseMockTests.java |   41 +-
 ...ionAnalyticsListWithResponseMockTests.java |   41 +-
 ...tedAlertsDismissWithResponseMockTests.java |   39 +-
 ...regatedAlertsGetWithResponseMockTests.java |   43 +-
 ...nalyticsAggregatedAlertsListMockTests.java |   41 +-
 ...commendationsGetWithResponseMockTests.java |   44 +-
 ...AnalyticsRecommendationsListMockTests.java |   44 +-
 ...nsCreateOrUpdateWithResponseMockTests.java |  126 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...tByResourceGroupWithResponseMockTests.java |   69 +-
 ...SolutionsListByResourceGroupMockTests.java |   64 +-
 .../IotSecuritySolutionsListMockTests.java    |   64 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   80 +-
 ...ssPoliciesDeleteWithResponseMockTests.java |   39 +-
 ...ccessPoliciesGetWithResponseMockTests.java |   54 +-
 ...PoliciesInitiateWithResponseMockTests.java |  108 +-
 ...rkAccessPoliciesListByRegionMockTests.java |   53 +-
 ...ListByResourceGroupAndRegionMockTests.java |   55 +-
 ...sPoliciesListByResourceGroupMockTests.java |   56 +-
 ...JitNetworkAccessPoliciesListMockTests.java |   54 +-
 .../JitNetworkAccessPoliciesListTests.java    |  166 +-
 ...tNetworkAccessPolicyInitiatePortTests.java |   27 +-
 ...tworkAccessPolicyInitiateRequestTests.java |   73 +-
 ...cessPolicyInitiateVirtualMachineTests.java |   43 +-
 .../JitNetworkAccessPolicyInnerTests.java     |  230 +-
 ...JitNetworkAccessPolicyPropertiesTests.java |  209 +-
 ...etworkAccessPolicyVirtualMachineTests.java |   79 +-
 .../JitNetworkAccessPortRuleTests.java        |   39 +-
 .../JitNetworkAccessRequestInnerTests.java    |   70 +-
 .../JitNetworkAccessRequestPortTests.java     |   50 +-
 ...tworkAccessRequestVirtualMachineTests.java |   90 +-
 .../generated/KindAutoGeneratedTests.java     |    9 +-
 .../security/generated/LabelTests.java        |   32 +
 .../generated/ListCustomAlertRuleTests.java   |   14 +-
 .../generated/LocalUserNotAllowedTests.java   |   22 +-
 .../security/generated/LocationTests.java     |    2 +-
 .../LocationsGetWithResponseMockTests.java    |   37 +-
 .../generated/LocationsListMockTests.java     |   34 +-
 .../LogAnalyticsIdentifierTests.java          |    8 +-
 .../MdeOnboardingDataInnerTests.java          |    6 +-
 .../MdeOnboardingDataListInnerTests.java      |   16 +-
 ...deOnboardingsGetWithResponseMockTests.java |   40 +-
 ...eOnboardingsListWithResponseMockTests.java |   41 +-
 ...MqttC2DMessagesNotInAllowedRangeTests.java |   32 +-
 ...ejectedMessagesNotInAllowedRangeTests.java |   29 +-
 ...MqttD2CMessagesNotInAllowedRangeTests.java |   33 +-
 .../NotificationsSourceAlertTests.java        |   26 +
 .../NotificationsSourceAttackPathTests.java   |   28 +
 .../generated/NotificationsSourceTests.java   |   22 +
 .../OnPremiseResourceDetailsTests.java        |   32 +-
 .../OnPremiseSqlResourceDetailsTests.java     |   43 +-
 .../generated/OnUploadPropertiesTests.java    |   27 +
 .../generated/OperationDisplayTests.java      |    8 +-
 .../generated/OperationInnerTests.java        |    8 +-
 .../generated/OperationListTests.java         |   18 +-
 ...perationResultAutoGeneratedInnerTests.java |    4 +-
 .../OperationStatusResultInnerTests.java      |  109 +
 .../generated/PathRecommendationTests.java    |   91 +-
 .../security/generated/PricingInnerTests.java |   61 -
 .../generated/PricingListInnerTests.java      |   61 +-
 ...eByResourceGroupWithResponseMockTests.java |   51 +
 .../PricingsGetWithResponseMockTests.java     |   68 -
 .../PricingsListWithResponseMockTests.java    |   50 +-
 .../PricingsUpdateWithResponseMockTests.java  |   80 -
 .../generated/ProcessNotAllowedTests.java     |   16 +-
 .../generated/ProtectionModeTests.java        |   19 +-
 .../generated/ProxyServerPropertiesTests.java |   14 +-
 .../generated/PublisherInfoTests.java         |   32 +-
 .../security/generated/QueryCheckTests.java   |   34 +-
 .../QueuePurgesNotInAllowedRangeTests.java    |   28 +-
 ...mendationConfigurationPropertiesTests.java |   23 +-
 ...ulatoryComplianceAssessmentInnerTests.java |   14 +-
 ...gulatoryComplianceAssessmentListTests.java |   24 +-
 ...ryComplianceAssessmentPropertiesTests.java |   16 +-
 ...ceAssessmentsGetWithResponseMockTests.java |   40 +-
 ...oryComplianceAssessmentsListMockTests.java |   39 +-
 ...RegulatoryComplianceControlInnerTests.java |   14 +-
 .../RegulatoryComplianceControlListTests.java |   20 +-
 ...atoryComplianceControlPropertiesTests.java |   12 +-
 ...ianceControlsGetWithResponseMockTests.java |   42 +-
 ...latoryComplianceControlsListMockTests.java |   41 +-
 ...egulatoryComplianceStandardInnerTests.java |   14 +-
 ...RegulatoryComplianceStandardListTests.java |   22 +-
 ...toryComplianceStandardPropertiesTests.java |   16 +-
 ...anceStandardsGetWithResponseMockTests.java |   42 +-
 ...atoryComplianceStandardsListMockTests.java |   39 +-
 .../generated/RemediationEtaTests.java        |   20 +-
 .../security/generated/RemediationTests.java  |   33 +-
 .../ResourceDetailsAutoGeneratedTests.java    |   13 +-
 .../generated/ResourceDetailsTests.java       |    4 +-
 .../generated/ResourceIdentifierTests.java    |    4 +-
 ...sitivitySettingsWithResponseMockTests.java |   77 +
 ...sitivitySettingsWithResponseMockTests.java |   85 +
 .../generated/RuleResultsInnerTests.java      |   19 +-
 .../generated/RuleResultsInputTests.java      |   22 +-
 .../generated/RuleResultsPropertiesTests.java |   17 +-
 .../security/generated/RuleTests.java         |   30 +-
 .../generated/RulesResultsInnerTests.java     |   29 +-
 .../generated/RulesResultsInputTests.java     |   39 +-
 .../security/generated/ScanInnerTests.java    |   94 +-
 .../generated/ScanPropertiesTests.java        |   88 +-
 .../generated/ScanResultInnerTests.java       |  153 +-
 .../generated/ScanResultPropertiesTests.java  |  182 +-
 .../generated/ScanResultsInnerTests.java      |  194 +-
 .../security/generated/ScansInnerTests.java   |  121 +-
 .../security/generated/ScopeElementTests.java |   15 +-
 .../security/generated/ScoreDetailsTests.java |    7 +-
 ...eScoreControlDefinitionItemInnerTests.java |    8 +-
 ...eControlDefinitionItemPropertiesTests.java |    8 +-
 ...SecureScoreControlDefinitionListTests.java |    8 +-
 ...cureScoreControlDefinitionSourceTests.java |   12 +-
 ...efinitionsListBySubscriptionMockTests.java |   38 +-
 ...eScoreControlDefinitionsListMockTests.java |   38 +-
 .../SecureScoreControlDetailsInnerTests.java  |   12 +-
 .../SecureScoreControlListTests.java          |    8 +-
 ...ureScoreControlScoreDetailsInnerTests.java |   12 +-
 ...oreControlsListBySecureScoreMockTests.java |   40 +-
 .../SecureScoreControlsListMockTests.java     |   38 +-
 .../generated/SecureScoreItemInnerTests.java  |    8 +-
 .../SecureScoreItemPropertiesTests.java       |    8 +-
 .../SecureScoresGetWithResponseMockTests.java |   38 +-
 .../generated/SecureScoresListMockTests.java  |   34 +-
 .../generated/SecureScoresListTests.java      |    8 +-
 .../SecurityAssessmentListTests.java          |   25 -
 ...taPropertiesResponsePublishDatesTests.java |   18 +-
 ...tyAssessmentMetadataResponseListTests.java |   25 -
 ...nsCreateOrUpdateWithResponseMockTests.java |   58 +-
 ...OperationsDeleteWithResponseMockTests.java |   39 +-
 ...ionOperationsGetWithResponseMockTests.java |   44 +-
 ...ityConnectorApplicationsListMockTests.java |   51 +-
 .../SecurityConnectorInnerTests.java          |   35 +-
 .../SecurityConnectorPropertiesTests.java     |   25 +-
 ...rsCreateOrUpdateWithResponseMockTests.java |   60 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...tByResourceGroupWithResponseMockTests.java |   54 +-
 ...onnectorsListByResourceGroupMockTests.java |   51 +-
 .../SecurityConnectorsListMockTests.java      |   41 +-
 .../SecurityConnectorsListTests.java          |   69 +-
 .../generated/SecurityContactInnerTests.java  |   47 +-
 .../generated/SecurityContactListTests.java   |   43 +-
 ...tactPropertiesAlertNotificationsTests.java |   34 -
 ...actPropertiesNotificationsByRoleTests.java |   21 +-
 .../SecurityContactPropertiesTests.java       |   53 +-
 ...tyContactsCreateWithResponseMockTests.java |   69 +-
 ...tyContactsDeleteWithResponseMockTests.java |   37 +-
 ...urityContactsGetWithResponseMockTests.java |   56 +-
 .../SecurityContactsListMockTests.java        |   47 +-
 .../generated/SecurityOperatorInnerTests.java |   12 +-
 .../SecurityOperatorListInnerTests.java       |   19 +-
 ...rsCreateOrUpdateWithResponseMockTests.java |   40 +-
 ...eByResourceGroupWithResponseMockTests.java |   39 +-
 ...rityOperatorsGetWithResponseMockTests.java |   37 +-
 ...ityOperatorsListWithResponseMockTests.java |   44 +-
 .../generated/SecuritySolutionInnerTests.java |   25 +-
 .../generated/SecuritySolutionListTests.java  |   54 +-
 .../SecuritySolutionPropertiesTests.java      |   32 +-
 ...ritySolutionsGetWithResponseMockTests.java |   46 +-
 .../SecuritySolutionsListMockTests.java       |   37 +-
 ...ySolutionsReferenceDataListInnerTests.java |   53 +-
 ...SolutionsReferenceDataPropertiesTests.java |   49 +-
 .../SecuritySolutionsReferenceDataTests.java  |   49 +-
 ...ListByHomeRegionWithResponseMockTests.java |   50 +-
 ...ferenceDatasListWithResponseMockTests.java |   47 +-
 .../SecuritySubAssessmentListTests.java       |   25 -
 .../generated/SecurityTaskInnerTests.java     |   15 +-
 .../generated/SecurityTaskListTests.java      |    8 +-
 .../SecurityTaskParametersTests.java          |   12 +-
 .../SecurityTaskPropertiesTests.java          |   15 +-
 .../generated/SensitivityLabelTests.java      |   33 +-
 ...vitySettingsListWithResponseMockTests.java |   64 +
 ...rverVulnerabilityAssessmentInnerTests.java |    8 +-
 ...ulnerabilityAssessmentPropertiesTests.java |    5 +-
 ...ssessmentsAzureSettingPropertiesTests.java |   31 +
 ...tsCreateOrUpdateWithResponseMockTests.java |   43 +-
 ...lnerabilityAssessmentsDeleteMockTests.java |   39 +-
 ...tyAssessmentsGetWithResponseMockTests.java |   42 +-
 ...ExtendedResourceWithResponseMockTests.java |   43 +-
 ...ulnerabilityAssessmentsListInnerTests.java |   14 +-
 ...erabilityAssessmentsSettingInnerTests.java |   23 +
 ...gsCreateOrUpdateWithResponseMockTests.java |   58 +
 ...tsSettingsDeleteWithResponseMockTests.java |   53 +
 ...entsSettingsGetWithResponseMockTests.java} |   22 +-
 ...ilityAssessmentsSettingsListMockTests.java |   55 +
 ...erabilityAssessmentsSettingsListTests.java |   23 +
 .../ServerVulnerabilityPropertiesTests.java   |    8 +-
 .../security/generated/SettingInnerTests.java |    8 +-
 .../SettingsGetWithResponseMockTests.java     |   42 +-
 .../generated/SettingsListMockTests.java      |   34 +-
 .../security/generated/SettingsListTests.java |   11 +-
 .../SettingsUpdateWithResponseMockTests.java  |   44 +-
 .../generated/SoftwareInnerTests.java         |   59 +-
 ...reInventoriesGetWithResponseMockTests.java |   59 +-
 ...toriesListByExtendedResourceMockTests.java |   60 +-
 .../SoftwareInventoriesListMockTests.java     |   53 +-
 .../generated/SoftwarePropertiesTests.java    |   55 +-
 .../generated/SoftwaresListTests.java         |   72 +-
 ...SqlServerVulnerabilityPropertiesTests.java |    9 +-
 ...BaselineRulesAddWithResponseMockTests.java |   63 +-
 ...esCreateOrUpdateWithResponseMockTests.java |   53 +-
 ...elineRulesDeleteWithResponseMockTests.java |   39 +-
 ...BaselineRulesGetWithResponseMockTests.java |   42 +-
 ...aselineRulesListWithResponseMockTests.java |   43 +-
 ...ntScanResultsGetWithResponseMockTests.java |   86 +-
 ...tScanResultsListWithResponseMockTests.java |   61 +-
 ...sessmentScansGetWithResponseMockTests.java |   70 +-
 ...essmentScansListWithResponseMockTests.java |   62 +-
 .../SuppressionAlertsScopeTests.java          |   20 +-
 .../security/generated/TagsResourceTests.java |   14 +-
 .../security/generated/TagsTests.java         |   13 +-
 .../TargetBranchConfigurationTests.java       |   32 +
 ...ceGroupLevelTaskWithResponseMockTests.java |   43 +-
 ...riptionLevelTaskWithResponseMockTests.java |   42 +-
 .../TasksListByHomeRegionMockTests.java       |   38 +-
 .../TasksListByResourceGroupMockTests.java    |   38 +-
 .../generated/TasksListMockTests.java         |   36 +-
 ...upLevelTaskStateWithResponseMockTests.java |   36 +-
 ...onLevelTaskStateWithResponseMockTests.java |   36 +-
 .../ThresholdCustomAlertRuleTests.java        |   23 +-
 .../TimeWindowCustomAlertRuleTests.java       |   28 +-
 .../TopologiesGetWithResponseMockTests.java   |   41 +-
 .../TopologiesListByHomeRegionMockTests.java  |   38 +-
 .../generated/TopologiesListMockTests.java    |   34 +-
 .../security/generated/TopologyListTests.java |    8 +-
 .../generated/TopologyResourceInnerTests.java |    8 +-
 .../TopologyResourcePropertiesTests.java      |    8 +-
 .../TopologySingleResourceChildTests.java     |    4 +-
 .../TopologySingleResourceParentTests.java    |    4 +-
 .../TopologySingleResourceTests.java          |    8 +-
 .../generated/TrackedResourceTests.java       |   33 +-
 .../TwinUpdatesNotInAllowedRangeTests.java    |   29 +-
 ...rizedOperationsNotInAllowedRangeTests.java |   29 +-
 ...ateIoTSecuritySolutionPropertiesTests.java |   60 +-
 .../UpdateIotSecuritySolutionDataTests.java   |   72 +-
 ...UpdateSensitivitySettingsRequestTests.java |   39 +
 .../UserDefinedResourcesPropertiesTests.java  |   20 +-
 .../generated/UserRecommendationTests.java    |   13 +-
 .../security/generated/VaRuleTests.java       |   90 +-
 .../generated/VendorReferenceTests.java       |    4 +-
 .../generated/VmRecommendationTests.java      |   29 +-
 .../generated/WorkspaceSettingInnerTests.java |   20 +-
 .../generated/WorkspaceSettingListTests.java  |   26 +-
 .../WorkspaceSettingPropertiesTests.java      |   17 +-
 ...ceSettingsCreateWithResponseMockTests.java |   46 +-
 ...ceSettingsDeleteWithResponseMockTests.java |   36 +-
 ...spaceSettingsGetWithResponseMockTests.java |   41 +-
 .../WorkspaceSettingsListMockTests.java       |   37 +-
 .../pom.xml                                   |    6 +-
 .../azure-resourcemanager-selfhelp/pom.xml    |    6 +-
 sdk/serialization/azure-json-gson/pom.xml     |    4 +-
 sdk/serialization/azure-json-reflect/pom.xml  |    2 +-
 .../implementation/DefaultJsonProvider.java   |    4 +-
 .../azure-xml/spotbugs-exclude.xml            |    2 +-
 .../main/java/com/azure/xml/XmlProvider.java  |   76 -
 .../main/java/com/azure/xml/XmlProviders.java |  273 -
 .../main/java/com/azure/xml/XmlReader.java    |  260 +-
 .../java/com/azure/xml/XmlSerializable.java   |   17 +-
 .../main/java/com/azure/xml/XmlWriter.java    |  266 +-
 .../xml/implementation/DefaultXmlReader.java  |  216 -
 .../xml/implementation/DefaultXmlWriter.java  |  161 -
 .../xml/implementation/package-info.java      |    7 -
 .../azure-xml/src/main/java/module-info.java  |    3 -
 .../xml/DefaultXmlReaderContractTests.java    |    5 +-
 .../xml/DefaultXmlWriterContractTests.java    |    5 +-
 .../java/com/azure/xml/PlaygroundTests.java   |   10 +-
 .../xml/contract/XmlReaderContractTests.java  |    4 +-
 .../azure/xml/storage/DateTimeRfc1123.java    |    2 +-
 .../storage/DeserializeListBlobsTests.java    |   12 +-
 sdk/serialization/ci.yml                      |    6 +-
 .../azure-messaging-servicebus-stress/pom.xml |   12 +-
 .../pom.xml                                   |    2 +-
 .../azure-messaging-servicebus/CHANGELOG.md   |   21 +-
 .../azure-messaging-servicebus/README.md      |    2 +-
 .../azure-messaging-servicebus/docs/pom.xml   |    4 +-
 .../azure-messaging-servicebus/pom.xml        |   18 +-
 .../implementation/EntityHelper.java          |   17 +-
 ...rviceBusAdministrationAsyncClientTest.java |    3 +-
 sdk/servicebus/tests.yml                      |   40 +-
 sdk/servicebus/version-overrides-matrix.json  |    2 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    4 +-
 .../pom.xml                                   |    8 +-
 sdk/servicenetworking/tests.mgmt.yml          |   24 +-
 .../azure-resourcemanager-signalr/pom.xml     |    6 +-
 .../azure-resourcemanager-sphere/pom.xml      |    6 +-
 sdk/spring/CHANGELOG.md                       |   14 +
 sdk/spring/azure-spring-data-cosmos/pom.xml   |   32 +-
 sdk/spring/compatibility-tests.yml            |    2 +-
 sdk/spring/monitor-tests.yml                  |    2 +-
 .../pipeline/cosmos-integration-matrix.json   |    4 +-
 .../monitor-supported-version-matrix.json     |    2 +-
 .../spring-cloud-azure-supported-spring.json  |   16 +
 .../pipeline/supported-version-matrix.json    |    2 +-
 ...t_2.7.18_managed_external_dependencies.txt |  424 +-
 .../pom.xml                                   |   20 +-
 .../spring-cloud-azure-actuator/pom.xml       |   20 +-
 .../pom.xml                                   |    8 +-
 .../pom.xml                                   |   22 +-
 .../spring-cloud-azure-autoconfigure/pom.xml  |   48 +-
 sdk/spring/spring-cloud-azure-core/pom.xml    |   18 +-
 .../pom.xml                                   |    8 +-
 .../pom.xml                                   |    8 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    8 +-
 sdk/spring/spring-cloud-azure-service/pom.xml |   28 +-
 .../pom.xml                                   |    2 +-
 .../spring-cloud-azure-starter-cosmos/pom.xml |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    4 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../java/com/azure/SpringMonitorTest.java     |    1 +
 .../pom.xml                                   |   38 +-
 .../spring-cloud-azure-starter-redis/pom.xml  |    2 +-
 .../pom.xml                                   |   20 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |   10 +-
 .../pom.xml                                   |    8 +-
 ...gHandlerMappingsProviderConfiguration.java |   35 +
 .../main/resources/META-INF/spring.factories  |    2 +
 ...appingsProviderAutoConfigurationTests.java |   93 +
 .../pom.xml                                   |   12 +-
 .../pom.xml                                   |    6 +-
 ...gHandlerMappingsProviderConfiguration.java |   35 +
 .../main/resources/META-INF/spring.factories  |    2 +
 ...appingsProviderAutoConfigurationTests.java |   86 +
 .../spring-cloud-azure-trace-sleuth/pom.xml   |   10 +-
 .../spring-integration-azure-core/pom.xml     |    6 +-
 .../pom.xml                                   |    8 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    4 +-
 .../spring-messaging-azure-eventhubs/pom.xml  |   18 +-
 .../spring-messaging-azure-servicebus/pom.xml |   17 +-
 .../pom.xml                                   |   11 +-
 sdk/spring/spring-messaging-azure/pom.xml     |    8 +-
 sdk/spring/tests.yml                          |  163 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../azure-storage-blob-batch/CHANGELOG.md     |   23 +
 .../checkstyle-suppressions.xml               |    7 +
 sdk/storage/azure-storage-blob-batch/pom.xml  |    9 +-
 .../spotbugs-exclude.xml                      |    9 +
 .../checkstyle-suppressions.xml               |   15 +
 .../azure-storage-blob-changefeed/pom.xml     |   13 +-
 .../spotbugs-exclude.xml                      |   20 +
 .../implementation/models/package-info.java   |    7 +
 .../implementation/util/package-info.java     |    7 +
 .../CHANGELOG.md                              |   24 +
 .../checkstyle-suppressions.xml               |    8 +
 .../azure-storage-blob-cryptography/pom.xml   |   11 +-
 .../spotbugs-exclude.xml                      |   13 +
 .../checkstyle-suppressions.xml               |   19 +
 sdk/storage/azure-storage-blob-nio/pom.xml    |   11 +-
 .../spotbugs-exclude.xml                      |   20 +
 sdk/storage/azure-storage-blob/CHANGELOG.md   |   24 +
 .../checkstyle-suppressions.xml               |   42 +
 sdk/storage/azure-storage-blob/pom.xml        |   13 +-
 .../azure-storage-blob/spotbugs-exclude.xml   |  175 +
 .../accesshelpers/package-info.java           |    7 +
 sdk/storage/azure-storage-common/CHANGELOG.md |   21 +
 .../checkstyle-suppressions.xml               |   25 +
 sdk/storage/azure-storage-common/pom.xml      |    9 +-
 .../azure-storage-common/spotbugs-exclude.xml |   25 +
 .../connectionstring/package-info.java        |    7 +
 .../azure-storage-file-datalake/CHANGELOG.md  |   24 +
 .../azure-storage-file-datalake/assets.json   |    2 +-
 .../checkstyle-suppressions.xml               |   27 +
 .../azure-storage-file-datalake/pom.xml       |    9 +-
 .../spotbugs-exclude.xml                      |   91 +
 .../storage/file/datalake/CpkAsyncTests.java  |   49 +-
 .../file/datalake/DataLakeTestBase.java       |   23 +-
 .../file/datalake/DirectoryAsyncApiTests.java | 1978 +++--
 .../file/datalake/FileAsyncApiTests.java      | 2081 +++---
 .../datalake/FileSystemAsyncApiTests.java     |  841 ++-
 .../storage/file/datalake/SasAsyncTests.java  |  236 +-
 .../file/datalake/ServiceAsyncApiTests.java   |  182 +-
 .../file/datalake/SoftDeleteAsyncTests.java   |   89 +-
 .../specialized/LeaseAsyncApiTests.java       |  493 +-
 .../azure-storage-file-share/CHANGELOG.md     |   23 +
 .../checkstyle-suppressions.xml               |   20 +
 sdk/storage/azure-storage-file-share/pom.xml  |    9 +-
 .../spotbugs-exclude.xml                      |   90 +
 .../storage/file/share/ShareFileClient.java   |    2 +-
 .../accesshelpers/package-info.java           |    7 +
 .../azure-storage-internal-avro/CHANGELOG.md  |   21 +
 .../checkstyle-suppressions.xml               |   21 +
 .../azure-storage-internal-avro/pom.xml       |    7 +-
 .../spotbugs-exclude.xml                      |   18 +
 .../schema/complex/package-info.java          |    7 +
 .../schema/file/package-info.java             |    7 +
 .../implementation/schema/package-info.java   |    7 +
 .../schema/primitive/package-info.java        |    7 +
 .../memory-stress-scenarios.ps1               |    5 +
 sdk/storage/azure-storage-perf/pom.xml        |    4 +-
 sdk/storage/azure-storage-queue/CHANGELOG.md  |   23 +
 .../checkstyle-suppressions.xml               |   17 +
 sdk/storage/azure-storage-queue/pom.xml       |   11 +-
 .../azure-storage-queue/spotbugs-exclude.xml  |   37 +
 sdk/storage/ci.yml                            |    2 +
 .../microsoft-azure-storage-perf/pom.xml      |    4 +-
 sdk/storage/platform-matrix-all-versions.json |    4 +-
 sdk/storage/platform-matrix-ci.json           |    2 +-
 sdk/storage/platform-matrix.json              |   10 +-
 sdk/storage/tests-template.yml                |    5 +-
 .../CHANGELOG.md                              |   13 +-
 .../pom.xml                                   |    8 +-
 .../reflect-config.json                       |    4 +-
 .../ManagedServiceIdentityTests.java          |    2 +-
 .../generated/StorageTaskInnerTests.java      |    2 +-
 .../StorageTaskUpdateParametersTests.java     |    2 +-
 .../StorageTasksCreateMockTests.java          |    2 +-
 ...tByResourceGroupWithResponseMockTests.java |    2 +-
 ...rageTasksListByResourceGroupMockTests.java |    2 +-
 .../generated/StorageTasksListMockTests.java  |    2 +-
 .../StorageTasksListResultTests.java          |    2 +-
 .../generated/UserAssignedIdentityTests.java  |    2 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    8 +-
 sdk/storagemover/tests.mgmt.yml               |   24 +-
 .../azure-resourcemanager-storagepool/pom.xml |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../CHANGELOG.md                              |  115 +-
 .../azure-resourcemanager-support/README.md   |    4 +-
 .../azure-resourcemanager-support/SAMPLE.md   | 2056 +++---
 .../azure-resourcemanager-support/pom.xml     |   12 +-
 .../support/SupportManager.java               |  228 +-
 .../support/fluent/ChatTranscriptsClient.java |   16 +-
 .../ChatTranscriptsNoSubscriptionsClient.java |   38 +-
 .../support/fluent/CommunicationsClient.java  |   68 +-
 .../CommunicationsNoSubscriptionsClient.java  |   96 +-
 .../support/fluent/FileWorkspacesClient.java  |   12 +-
 .../FileWorkspacesNoSubscriptionsClient.java  |   12 +-
 .../support/fluent/FilesClient.java           |   28 +-
 .../fluent/FilesNoSubscriptionsClient.java    |   28 +-
 .../fluent/LookUpResourceIdsClient.java}      |   34 +-
 .../support/fluent/MicrosoftSupport.java      |   84 +-
 .../support/fluent/OperationsClient.java      |   16 +-
 .../fluent/ProblemClassificationsClient.java  |   51 +-
 ...mClassificationsNoSubscriptionsClient.java |   47 +
 .../fluent/ServiceClassificationsClient.java  |   43 +
 ...eClassificationsNoSubscriptionsClient.java |   44 +
 .../support/fluent/ServicesClient.java        |   16 +-
 ...etCommunicationsNoSubscriptionsClient.java |   58 -
 .../support/fluent/SupportTicketsClient.java  |   64 +-
 .../SupportTicketsNoSubscriptionsClient.java  |   66 +-
 .../models/ChatTranscriptDetailsInner.java    |   20 +-
 .../ChatTranscriptDetailsProperties.java      |   16 +-
 .../CheckNameAvailabilityOutputInner.java     |   16 +-
 .../fluent/models/ClassificationService.java  |   85 +
 .../models/CommunicationDetailsInner.java     |   36 +-
 .../CommunicationDetailsProperties.java       |   46 +-
 .../fluent/models/FileDetailsInner.java       |   40 +-
 .../fluent/models/FileDetailsProperties.java  |   42 +-
 .../models/FileWorkspaceDetailsInner.java     |   18 +-
 .../FileWorkspaceDetailsProperties.java       |   14 +-
 .../models/LookUpResourceIdResponseInner.java |   54 +
 .../support/fluent/models/OperationInner.java |   16 +-
 .../models/ProblemClassificationInner.java    |   74 +-
 .../ProblemClassificationProperties.java      |   84 -
 .../ProblemClassificationPropertiesInner.java |  138 +
 ...ssificationsClassificationOutputInner.java |   60 +
 .../ServiceClassificationOutputInner.java     |   60 +
 .../support/fluent/models/ServiceInner.java   |   36 +-
 .../fluent/models/ServiceProperties.java      |   36 +-
 .../models/SupportTicketDetailsInner.java     |  124 +-
 .../SupportTicketDetailsProperties.java       |  163 +-
 .../support/fluent/models/package-info.java   |    5 +-
 .../support/fluent/package-info.java          |    5 +-
 .../ChatTranscriptDetailsImpl.java            |    4 +-
 .../ChatTranscriptsClientImpl.java            |  282 +-
 .../implementation/ChatTranscriptsImpl.java   |   21 +-
 ...tTranscriptsNoSubscriptionsClientImpl.java |  291 +-
 .../ChatTranscriptsNoSubscriptionsImpl.java   |   27 +-
 .../CheckNameAvailabilityOutputImpl.java      |    4 +-
 .../CommunicationDetailsImpl.java             |   34 +-
 .../CommunicationsClientImpl.java             |  641 +-
 .../implementation/CommunicationsImpl.java    |   88 +-
 ...mmunicationsNoSubscriptionsClientImpl.java |  636 +-
 .../CommunicationsNoSubscriptionsImpl.java    |   71 +-
 .../implementation/FileDetailsImpl.java       |   40 +-
 .../FileWorkspaceDetailsImpl.java             |    4 +-
 .../FileWorkspacesClientImpl.java             |  168 +-
 .../implementation/FileWorkspacesImpl.java    |   14 +-
 ...leWorkspacesNoSubscriptionsClientImpl.java |  130 +-
 .../FileWorkspacesNoSubscriptionsImpl.java    |   13 +-
 .../implementation/FilesClientImpl.java       |  463 +-
 .../support/implementation/FilesImpl.java     |   47 +-
 .../FilesNoSubscriptionsClientImpl.java       |  377 +-
 .../FilesNoSubscriptionsImpl.java             |   30 +-
 .../LookUpResourceIdResponseImpl.java         |   32 +
 .../LookUpResourceIdsClientImpl.java          |  172 +
 .../implementation/LookUpResourceIdsImpl.java |   58 +
 .../MicrosoftSupportBuilder.java              |   50 +-
 .../implementation/MicrosoftSupportImpl.java  |  259 +-
 .../implementation/OperationsClientImpl.java  |   92 +-
 .../implementation/OperationsImpl.java        |    8 +-
 .../ProblemClassificationImpl.java            |   23 +-
 .../ProblemClassificationPropertiesImpl.java  |   65 +
 ...assificationsClassificationOutputImpl.java |   41 +
 .../ProblemClassificationsClientImpl.java     |  321 +-
 .../ProblemClassificationsImpl.java           |   47 +-
 ...ssificationsNoSubscriptionsClientImpl.java |  199 +
 ...lemClassificationsNoSubscriptionsImpl.java |   60 +
 .../{Utils.java => ResourceManagerUtils.java} |   25 +-
 .../ServiceClassificationOutputImpl.java      |   40 +
 .../ServiceClassificationsClientImpl.java     |  186 +
 .../ServiceClassificationsImpl.java           |   58 +
 ...ssificationsNoSubscriptionsClientImpl.java |  179 +
 ...iceClassificationsNoSubscriptionsImpl.java |   58 +
 .../support/implementation/ServiceImpl.java   |   10 +
 .../implementation/ServicesClientImpl.java    |  126 +-
 .../support/implementation/ServicesImpl.java  |    9 +-
 ...tTranscriptsNoSubscriptionsClientImpl.java |  304 -
 ...ketChatTranscriptsNoSubscriptionsImpl.java |   47 -
 ...mmunicationsNoSubscriptionsClientImpl.java |  395 -
 ...cketCommunicationsNoSubscriptionsImpl.java |   49 -
 .../SupportTicketDetailsImpl.java             |   55 +-
 .../SupportTicketsClientImpl.java             |  692 +-
 .../implementation/SupportTicketsImpl.java    |   44 +-
 ...pportTicketsNoSubscriptionsClientImpl.java |  607 +-
 .../SupportTicketsNoSubscriptionsImpl.java    |   50 +-
 .../support/implementation/package-info.java  |    5 +-
 .../support/models/ChatTranscriptDetails.java |   18 +-
 .../support/models/ChatTranscripts.java       |   16 +-
 .../models/ChatTranscriptsListResult.java     |   18 +-
 .../ChatTranscriptsNoSubscriptions.java       |   36 +-
 .../models/CheckNameAvailabilityInput.java    |   28 +-
 .../models/CheckNameAvailabilityOutput.java   |   12 +-
 .../support/models/CommunicationDetails.java  |   68 +-
 .../models/CommunicationDirection.java        |   18 +-
 .../support/models/CommunicationType.java     |   18 +-
 .../support/models/Communications.java        |   36 +-
 .../models/CommunicationsListResult.java      |   18 +-
 .../models/CommunicationsNoSubscriptions.java |   74 +-
 .../support/models/Consent.java               |   18 +-
 .../support/models/ContactProfile.java        |  117 +-
 .../support/models/FileDetails.java           |   82 +-
 .../support/models/FileWorkspaceDetails.java  |   18 +-
 .../support/models/FileWorkspaces.java        |   12 +-
 .../models/FileWorkspacesNoSubscriptions.java |   12 +-
 .../resourcemanager/support/models/Files.java |   26 +-
 .../support/models/FilesListResult.java       |   18 +-
 .../support/models/FilesNoSubscriptions.java  |   28 +-
 .../support/models/IsTemporaryTicket.java     |   53 +
 .../models/LookUpResourceIdRequest.java       |   83 +
 .../models/LookUpResourceIdResponse.java      |   26 +
 ...scriptions.java => LookUpResourceIds.java} |   31 +-
 .../support/models/MessageProperties.java     |   29 +-
 .../support/models/Operation.java             |   10 +-
 .../support/models/OperationDisplay.java      |   18 +-
 .../support/models/Operations.java            |   16 +-
 .../support/models/OperationsListResult.java  |   14 +-
 .../models/PreferredContactMethod.java        |   18 +-
 .../support/models/ProblemClassification.java |   32 +-
 .../ProblemClassificationProperties.java      |   51 +
 .../models/ProblemClassifications.java        |   47 +-
 ...lemClassificationsClassificationInput.java |   87 +
 ...emClassificationsClassificationOutput.java |   28 +
 ...emClassificationsClassificationResult.java |  164 +
 .../ProblemClassificationsListResult.java     |   14 +-
 ...ProblemClassificationsNoSubscriptions.java |   40 +
 .../support/models/QuotaChangeRequest.java    |   18 +-
 .../support/models/QuotaTicketDetails.java    |   18 +-
 .../support/models/ResourceType.java          |   56 +
 .../support/models/SecondaryConsent.java      |   18 +-
 .../models/SecondaryConsentEnabled.java       |   18 +-
 .../support/models/Service.java               |   24 +-
 .../models/ServiceClassificationAnswer.java   |   70 +
 .../models/ServiceClassificationOutput.java   |   27 +
 .../models/ServiceClassificationRequest.java  |  106 +
 .../models/ServiceClassifications.java}       |   31 +-
 ...erviceClassificationsNoSubscriptions.java} |   35 +-
 .../support/models/ServiceLevelAgreement.java |   16 +-
 .../support/models/Services.java              |   16 +-
 .../support/models/ServicesListResult.java    |   14 +-
 .../support/models/SeverityLevel.java         |   22 +-
 .../support/models/Status.java                |   18 +-
 .../support/models/SupportEngineer.java       |   12 +-
 ...rtTicketCommunicationsNoSubscriptions.java |   49 -
 .../support/models/SupportTicketDetails.java  |  304 +-
 .../support/models/SupportTickets.java        |   42 +-
 .../models/SupportTicketsListResult.java      |   18 +-
 .../models/SupportTicketsNoSubscriptions.java |   54 +-
 .../models/TechnicalTicketDetails.java        |   14 +-
 .../support/models/TranscriptContentType.java |   10 +-
 .../resourcemanager/support/models/Type.java  |   22 +-
 .../support/models/UpdateContactProfile.java  |   70 +-
 .../support/models/UpdateSupportTicket.java   |   30 +-
 .../support/models/UploadFile.java            |   24 +-
 .../support/models/UserConsent.java           |   18 +-
 .../support/models/package-info.java          |    5 +-
 .../resourcemanager/support/package-info.java |    5 +-
 .../src/main/java/module-info.java            |   10 +-
 .../proxy-config.json                         |    2 +-
 .../reflect-config.json                       |   57 +-
 .../generated/ChatTranscriptsGetSamples.java  |   14 +-
 .../generated/ChatTranscriptsListSamples.java |   15 +-
 ...atTranscriptsNoSubscriptionGetSamples.java |   14 +-
 ...tTranscriptsNoSubscriptionListSamples.java |   24 +
 ...nicationsCheckNameAvailabilitySamples.java |   22 +-
 .../CommunicationsCreateSamples.java          |   20 +-
 .../generated/CommunicationsGetSamples.java   |   13 +-
 .../generated/CommunicationsListSamples.java  |   41 +-
 ...scriptionCheckNameAvailabilitySamples.java |   22 +-
 ...unicationsNoSubscriptionCreateSamples.java |   28 +-
 ...ommunicationsNoSubscriptionGetSamples.java |   14 +-
 ...mmunicationsNoSubscriptionListSamples.java |   54 +
 .../FileWorkspacesCreateSamples.java          |   14 +-
 .../generated/FileWorkspacesGetSamples.java   |   13 +-
 ...WorkspacesNoSubscriptionCreateSamples.java |   10 +-
 ...ileWorkspacesNoSubscriptionGetSamples.java |    9 +-
 .../support/generated/FilesCreateSamples.java |   23 +-
 .../support/generated/FilesGetSamples.java    |    9 +-
 .../support/generated/FilesListSamples.java   |   13 +-
 .../FilesNoSubscriptionCreateSamples.java     |   19 +-
 .../FilesNoSubscriptionGetSamples.java        |    9 +-
 .../FilesNoSubscriptionListSamples.java       |    9 +-
 .../FilesNoSubscriptionUploadSamples.java     |   22 +-
 .../support/generated/FilesUploadSamples.java |   22 +-
 .../LookUpResourceIdPostSamples.java          |   28 +
 .../generated/OperationsListSamples.java      |    9 +-
 ...lassificationsClassifyProblemsSamples.java |   30 +
 .../ProblemClassificationsGetSamples.java     |   18 +-
 .../ProblemClassificationsListSamples.java    |    9 +-
 ...NoSubscriptionClassifyProblemsSamples.java |   28 +
 ...lassificationsClassifyServicesSamples.java |   29 +
 ...NoSubscriptionClassifyServicesSamples.java |   29 +
 .../support/generated/ServicesGetSamples.java |    9 +-
 .../generated/ServicesListSamples.java        |   13 +-
 ...tTranscriptsNoSubscriptionListSamples.java |   21 -
 ...mmunicationsNoSubscriptionListSamples.java |   57 -
 ...rtTicketsCheckNameAvailabilitySamples.java |   23 +-
 .../SupportTicketsCreateSamples.java          |  784 +-
 .../generated/SupportTicketsGetSamples.java   |    9 +-
 .../generated/SupportTicketsListSamples.java  |   66 +-
 ...scriptionCheckNameAvailabilitySamples.java |   23 +-
 ...ortTicketsNoSubscriptionCreateSamples.java |  163 +-
 ...upportTicketsNoSubscriptionGetSamples.java |    9 +-
 ...pportTicketsNoSubscriptionListSamples.java |  102 +-
 ...ortTicketsNoSubscriptionUpdateSamples.java |   75 +-
 .../SupportTicketsUpdateSamples.java          |   83 +-
 .../ChatTranscriptDetailsInnerTests.java      |   27 +-
 .../ChatTranscriptDetailsPropertiesTests.java |   28 +-
 ...atTranscriptsGetWithResponseMockTests.java |   44 +-
 .../ChatTranscriptsListMockTests.java         |   41 +-
 .../ChatTranscriptsListResultTests.java       |   64 +-
 ...SubscriptionsGetWithResponseMockTests.java |   44 +-
 ...nscriptsNoSubscriptionsListMockTests.java} |   24 +-
 .../CheckNameAvailabilityInputTests.java      |   13 +-
 ...CheckNameAvailabilityOutputInnerTests.java |    5 +-
 .../generated/ClassificationServiceTests.java |   28 +
 .../CommunicationDetailsInnerTests.java       |   24 +-
 .../CommunicationDetailsPropertiesTests.java  |   27 +-
 ...NameAvailabilityWithResponseMockTests.java |   44 +-
 .../CommunicationsCreateMockTests.java        |   57 +-
 ...ommunicationsGetWithResponseMockTests.java |   43 +-
 .../CommunicationsListMockTests.java          |   43 +-
 .../CommunicationsListResultTests.java        |   41 +-
 ...NameAvailabilityWithResponseMockTests.java |   47 +-
 ...cationsNoSubscriptionsCreateMockTests.java |   59 +-
 ...SubscriptionsGetWithResponseMockTests.java |   46 +-
 ...icationsNoSubscriptionsListMockTests.java} |   23 +-
 .../generated/ContactProfileTests.java        |   55 +-
 .../generated/FileDetailsInnerTests.java      |   24 +-
 .../generated/FileDetailsPropertiesTests.java |   27 +-
 .../FileWorkspaceDetailsInnerTests.java       |    8 +-
 .../FileWorkspaceDetailsPropertiesTests.java  |    7 +-
 ...WorkspacesCreateWithResponseMockTests.java |   38 +-
 ...ileWorkspacesGetWithResponseMockTests.java |   38 +-
 ...scriptionsCreateWithResponseMockTests.java |   41 +-
 ...SubscriptionsGetWithResponseMockTests.java |   38 +-
 .../FilesCreateWithResponseMockTests.java     |   50 +-
 .../FilesGetWithResponseMockTests.java        |   43 +-
 .../support/generated/FilesListMockTests.java |   41 +-
 .../generated/FilesListResultTests.java       |   38 +-
 ...scriptionsCreateWithResponseMockTests.java |   58 +-
 ...SubscriptionsGetWithResponseMockTests.java |   46 +-
 .../FilesNoSubscriptionsListMockTests.java    |   43 +-
 ...scriptionsUploadWithResponseMockTests.java |   39 +-
 .../FilesUploadWithResponseMockTests.java     |   39 +-
 .../LookUpResourceIdRequestTests.java         |   30 +
 .../LookUpResourceIdResponseInnerTests.java   |   25 +
 ...ResourceIdsPostWithResponseMockTests.java} |   26 +-
 .../generated/MessagePropertiesTests.java     |   18 +-
 .../generated/OperationDisplayTests.java      |    8 +-
 .../generated/OperationInnerTests.java        |    8 +-
 .../generated/OperationsListMockTests.java    |   34 +-
 .../generated/OperationsListResultTests.java  |   20 +-
 .../ProblemClassificationInnerTests.java      |   56 +-
 ...lemClassificationPropertiesInnerTests.java |   61 +
 .../ProblemClassificationPropertiesTests.java |   42 -
 ...assificationsClassificationInputTests.java |   29 +
 ...cationsClassificationOutputInnerTests.java |   36 +
 ...ssificationsClassificationResultTests.java |   28 +
 ...ClassifyProblemsWithResponseMockTests.java |   59 +
 ...assificationsGetWithResponseMockTests.java |   53 +-
 .../ProblemClassificationsListMockTests.java  |   52 +-
 ...ProblemClassificationsListResultTests.java |   72 +-
 ...ClassifyProblemsWithResponseMockTests.java |   59 +
 .../generated/QuotaChangeRequestTests.java    |   16 +-
 .../generated/QuotaTicketDetailsTests.java    |   39 +-
 .../SecondaryConsentEnabledTests.java         |   18 +-
 .../generated/SecondaryConsentTests.java      |   10 +-
 .../ServiceClassificationAnswerTests.java     |   32 +
 ...ServiceClassificationOutputInnerTests.java |   38 +
 .../ServiceClassificationRequestTests.java    |   31 +
 ...ClassifyServicesWithResponseMockTests.java |   61 +
 ...ClassifyServicesWithResponseMockTests.java |   61 +
 .../support/generated/ServiceInnerTests.java  |   22 +-
 .../generated/ServiceLevelAgreementTests.java |    8 +-
 .../generated/ServicePropertiesTests.java     |   21 +-
 .../ServicesGetWithResponseMockTests.java     |   39 +-
 .../generated/ServicesListMockTests.java      |   37 +-
 .../generated/ServicesListResultTests.java    |   34 +-
 .../generated/SupportEngineerTests.java       |    2 +-
 ...anscriptsNoSubscriptionsListMockTests.java |   69 -
 ...nicationsNoSubscriptionsListMockTests.java |   72 -
 .../SupportTicketDetailsInnerTests.java       |  172 +-
 .../SupportTicketDetailsPropertiesTests.java  |  170 +-
 ...NameAvailabilityWithResponseMockTests.java |   46 +-
 .../SupportTicketsCreateMockTests.java        |  156 +-
 ...upportTicketsGetWithResponseMockTests.java |   88 +-
 .../SupportTicketsListMockTests.java          |  106 +-
 .../SupportTicketsListResultTests.java        |  328 +-
 ...NameAvailabilityWithResponseMockTests.java |   45 +-
 ...TicketsNoSubscriptionsCreateMockTests.java |  156 +-
 ...SubscriptionsGetWithResponseMockTests.java |   86 +-
 ...rtTicketsNoSubscriptionsListMockTests.java |  108 +-
 ...scriptionsUpdateWithResponseMockTests.java |  128 +-
 .../TechnicalTicketDetailsTests.java          |   10 +-
 .../generated/UpdateContactProfileTests.java  |   55 +-
 .../generated/UpdateSupportTicketTests.java   |   83 +-
 .../support/generated/UploadFileTests.java    |   14 +-
 .../pom.xml                                   |    2 +-
 .../azure-analytics-synapse-artifacts/pom.xml |    2 +-
 .../pom.xml                                   |    2 +-
 .../pom.xml                                   |    2 +-
 .../azure-analytics-synapse-spark/pom.xml     |    2 +-
 .../azure-resourcemanager-synapse/pom.xml     |    6 +-
 sdk/tables/azure-data-tables-perf/pom.xml     |    2 +-
 sdk/tables/azure-data-tables/CHANGELOG.md     |   30 +-
 sdk/tables/azure-data-tables/README.md        |    2 +-
 sdk/tables/azure-data-tables/pom.xml          |   10 +-
 .../azure/data/tables/TableServiceClient.java |    7 +-
 .../tables/implementation/TableUtils.java     |    4 -
 sdk/tables/platform-matrix.json               |   12 +-
 sdk/tables/tests.yml                          |   38 +-
 sdk/template/azure-template-stress/pom.xml    |   18 +-
 sdk/template/ci.yml                           |    2 +-
 sdk/template/tests.yml                        |   24 +-
 .../azure-ai-textanalytics-perf/pom.xml       |    2 +-
 .../azure-ai-textanalytics/CHANGELOG.md       |   11 +
 .../azure-ai-textanalytics/pom.xml            |    2 +-
 sdk/textanalytics/ci.yml                      |    2 +
 sdk/textanalytics/tests.native.yml            |   46 +-
 sdk/textanalytics/tests.yml                   |   46 +-
 .../pom.xml                                   |    6 +-
 .../azure-ai-documenttranslator/pom.xml       |    2 +-
 .../azure-ai-translation-text/pom.xml         |    2 +-
 .../azure-ai-vision-imageanalysis/pom.xml     |    6 +-
 .../pom.xml                                   |    6 +-
 .../pom.xml                                   |    6 +-
 .../CHANGELOG.md                              |   10 +
 .../azure-messaging-webpubsub-client/pom.xml  |    8 +-
 .../azure-messaging-webpubsub/CHANGELOG.md    |   12 +
 .../azure-messaging-webpubsub/pom.xml         |    4 +-
 .../azure-resourcemanager-webpubsub/pom.xml   |    8 +-
 sdk/webpubsub/tests.mgmt.yml                  |   24 +-
 sdk/webpubsub/tests.yml                       |   38 +-
 .../azure-resourcemanager-workloads/pom.xml   |   10 +-
 sdk/workloads/tests.mgmt.yml                  |   24 +-
 .../CHANGELOG.md                              |   13 +-
 .../SAMPLE.md                                 |    6 +-
 .../pom.xml                                   |    2 +-
 .../fluent/OperationsClient.java              |    4 +-
 .../implementation/OperationsClientImpl.java  |   12 +-
 .../models/Operations.java                    |    4 +-
 .../generated/OperationsListSamples.java      |    6 +-
 7577 files changed, 349671 insertions(+), 195282 deletions(-)
 create mode 100644 eng/common/scripts/check-for-git-changes.ps1
 create mode 100644 eng/common/testproxy/test-proxy-tool-shutdown.yml
 delete mode 100644 eng/pipelines/latest-jdk.yml
 create mode 100644 eng/pipelines/scripts/Get-Heap-Dump-Hprofs.ps1
 create mode 100644 eng/pipelines/scripts/Get-Test-Logs.ps1
 create mode 100644 eng/pipelines/scripts/Invoke-Sparse-Checkout.ps1
 create mode 100644 eng/pipelines/scripts/Set-Dev-Versioning.ps1
 create mode 100644 eng/pipelines/scripts/generate-project-list.ps1
 create mode 100644 eng/pipelines/templates/stages/1es-redirect.yml
 create mode 100644 eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml
 rename eng/pipelines/templates/steps/{upload-repository-on-failure.yml => clean-maven-local-cache.yml} (51%)
 create mode 100644 eng/pipelines/templates/steps/download-credscan-suppressions.yml
 create mode 100644 eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
 create mode 100644 eng/pipelines/templates/variables/image.yml
 create mode 100644 eng/scripts/setup-http-fault-injector.ps1
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AppResilienciesClient.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildAuthTokensClient.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildersClient.java
 rename sdk/{support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/SupportTicketChatTranscriptsNoSubscriptionsClient.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsByBuilderResourcesClient.java} (53%)
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountBackupsClient.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsClient.java} (51%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentResiliencyPoliciesClient.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprSubscriptionsClient.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupVaultsClient.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DotNetComponentsClient.java} (56%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JavaComponentsClient.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentUsagesClient.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/UsagesClient.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildResourceInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildTokenInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceUpdateProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HeaderMatchMatch.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyMatches.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyRetryBackOff.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/UsageInner.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResiliencyImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildResourceImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildTokenImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuilderResourceImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesImpl.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsClientImpl.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsClientImpl.java} (53%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPolicyImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentImpl.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsClientImpl.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsClientImpl.java} (56%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceManagerUtils.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsageImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesClientImpl.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesImpl.java
 delete mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/Utils.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppInsightsConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencies.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliency.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencyCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BlobStorageTokenStore.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildAuthTokens.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildProvisioningState.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildResource.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildStatus.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildToken.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderProvisioningState.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResource.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResourceUpdate.java
 rename sdk/{communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionLists.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builders.java} (55%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builds.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderAccounts.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildsByBuilderResources.java} (53%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateKeyVaultProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateType.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CircuitBreakerPolicy.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistry.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistryWithCustomImage.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicies.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPoliciesCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicy.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyTimeoutPolicyConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentServiceBinding.java
 rename sdk/{redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Capability.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprServiceBindMetadata.java} (51%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscription.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionBulkSubscribeOptions.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRouteRule.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRoutes.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptions.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionsCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DataDogConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DestinationsConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponent.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentConfigurationProperty.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentProvisioningState.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentServiceBind.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentType.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaults.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponents.java} (62%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentsCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EncryptionSettings.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVariable.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Header.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HeaderMatch.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpConnectionPool.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpGet.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpRetryPolicy.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressPortMapping.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTargetPortHttpScheme.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponent.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentConfigurationProperty.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentProvisioningState.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentServiceBind.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentType.java
 rename sdk/{communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddresses.java => appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponents.java} (51%)
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentsCollection.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ListUsagesResult.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogsConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentUsages.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/MetricsConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/NfsAzureFileProperties.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenTelemetryConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OtlpConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/PreBuildStep.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpConnectionPool.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpRetryPolicy.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TimeoutPolicy.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TokenStore.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TracesConfiguration.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usage.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UsageName.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usages.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/proxy-config.json
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/reflect-config.json
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildAuthTokenListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsByBuilderResourceListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsDeleteSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsUpdateSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsProxyGetSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderGetCustomDomainVerificationIdSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/UsagesListSamples.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppInsightsConfigurationTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteWithResponseMockTests.java => appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppResilienciesDeleteWithResponseMockTests.java} (82%)
 delete mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthConfigCollectionTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BlobStorageTokenStoreTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderCollectionTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderPropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdatePropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdateTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CircuitBreakerPolicyTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryWithCustomImageTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCollectionTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryBackOffConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryPolicyConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyPropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyTimeoutPolicyConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentServiceBindingTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprServiceBindMetadataTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionBulkSubscribeOptionsTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionPropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRouteRuleTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRoutesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCollectionTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentConfigurationPropertyTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentPropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentServiceBindTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCollectionTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/EnvironmentVariableTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchMatchTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpConnectionPoolTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpGetTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpRetryPolicyRetryBackOffTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressPortMappingTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentConfigurationPropertyTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentPropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentServiceBindTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCollectionTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsWithResponseMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ListUsagesResultTests.java
 delete mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LogsConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListMockTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/MetricsConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NfsAzureFilePropertiesTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/PreBuildStepTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersGetCustomDomainVerificationIdWithResponseMockTests.java
 delete mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceBindTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpConnectionPoolTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpRetryPolicyTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TimeoutPolicyTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TracesConfigurationTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageInnerTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageNameTests.java
 create mode 100644 sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsagesListMockTests.java
 rename sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/{Utils.java => ResourceManagerUtils.java} (90%)
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/Error.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationOptions.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequest.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequestOptions.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationResult.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchResultError.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorDetails.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformation.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationOptions.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationResult.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorNumberType.java
 create mode 100644 sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResultError.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListAddressesClient.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListsClient.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressProperties.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressResourceInner.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListProperties.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListResourceInner.java
 rename sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/{Utils.java => ResourceManagerUtils.java} (99%)
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressResourceImpl.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesClientImpl.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesImpl.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListResourceImpl.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsClientImpl.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsImpl.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResource.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResourceCollection.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResource.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResourceCollection.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesGetSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesListSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsCreateOrUpdateSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainSamples.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressPropertiesTests.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceCollectionTests.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceInnerTests.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListPropertiesTests.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceCollectionTests.java
 delete mode 100644 sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceInnerTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessRoleBindingNameListSuccessResponseInner.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyProperties.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyRecordInner.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ClusterProperties.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/EnvironmentProperties.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ListRegionsSuccessResponseInner.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/RegionProperties.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCClusterRecordInner.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCEnvironmentRecordInner.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterProperties.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterRecordInner.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessRoleBindingNameListSuccessResponseImpl.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ApiKeyRecordImpl.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ListRegionsSuccessResponseImpl.java
 rename sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/{Utils.java => ResourceManagerUtils.java} (99%)
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCClusterRecordImpl.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCEnvironmentRecordImpl.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SchemaRegistryClusterRecordImpl.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessCreateRoleBindingRequestModel.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessRoleBindingNameListSuccessResponse.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyOwnerEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyRecord.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyResourceEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeySpecEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/CreateApiKeyModel.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/GetEnvironmentsResponse.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListClustersSuccessResponse.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListRegionsSuccessResponse.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListSchemaRegistryClustersResponse.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionRecord.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionSpecEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterByokEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterNetworkEnvironmentEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterRecord.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterSpecEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCEnvironmentRecord.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCMetadataEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterEnvironmentRegionEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterRecord.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterSpecEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterStatusEntity.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateApiKeySamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteClusterApiKeySamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterApiKeySamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterByIdSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetEnvironmentByIdSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetSchemaRegistryClusterByIdSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListClustersSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListEnvironmentsSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListRegionsSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSchemaRegistryClustersSamples.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingRequestModelTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingWithResponseMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingWithResponseMockTests.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateMockTests.java => confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListWithResponseMockTests.java} (58%)
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessRoleBindingNameListSuccessResponseInnerTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyOwnerEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyResourceEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ClusterPropertiesTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/CreateApiKeyModelTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/EnvironmentPropertiesTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/GetEnvironmentsResponseTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListClustersSuccessResponseTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListRegionsSuccessResponseInnerTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListSchemaRegistryClustersResponseTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteClusterApiKeyWithResponseMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetClusterByIdWithResponseMockTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateWithResponseMockTests.java => confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetEnvironmentByIdWithResponseMockTests.java} (55%)
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetSchemaRegistryClusterByIdWithResponseMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListClustersMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListEnvironmentsMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListRegionsWithResponseMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListSchemaRegistryClustersMockTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionPropertiesTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionRecordTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionSpecEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterByokEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterNetworkEnvironmentEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterRecordInnerTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterSpecEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCEnvironmentRecordInnerTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCMetadataEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterEnvironmentRegionEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterPropertiesTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterRecordInnerTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterSpecEntityTests.java
 create mode 100644 sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterStatusEntityTests.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosUtils.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/Strings.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/ArrayUtils.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/CharSequenceUtils.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/StringUtils.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/Validate.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/Builder.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/CompareToBuilder.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/ImmutablePair.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/Pair.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Absent.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/AbstractIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ascii.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CaseFormat.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CharMatcher.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Charsets.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonMatcher.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonPattern.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Converter.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Defaults.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Enums.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Equivalence.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/ExtraObjectsMethodsForWeb.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizablePhantomReference.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReference.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReferenceQueue.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableSoftReference.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableWeakReference.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Function.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FunctionalEquivalence.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Functions.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/JdkPattern.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Joiner.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/MoreObjects.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Objects.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Optional.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PairwiseEquivalence.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PatternCompiler.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Platform.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Preconditions.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicate.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicates.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Present.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/SmallCharMatcher.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Splitter.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/StandardSystemProperty.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Stopwatch.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Strings.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Supplier.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Suppliers.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Throwables.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ticker.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Utf8.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Verify.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/VerifyException.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/internal/Finalizer.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/package-info.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIndexedListIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapEntry.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractNavigableMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractRangeSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSequentialIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedKeySortedSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AllEqualOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimapGwtSerializationDependencies.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BoundType.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ByFunctionOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CartesianList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ClassToInstanceMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectCollectors.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectPreconditions.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectSpliterators.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Collections2.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparatorOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Comparators.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparisonChain.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompoundOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComputationException.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConcurrentHashMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConsumingQueueIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ContiguousSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Count.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Cut.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DiscreteDomain.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyContiguousSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumHashBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EvictingQueue.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ExplicitOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntryMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntrySetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeySetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimapValues.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FluentIterable.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingCollection.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingConcurrentMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableCollection.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMapEntry.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingObject.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingQueue.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GeneralRange.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GwtTransient.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimapGwtSerializationDependencies.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Hashing.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableAsList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMapFauxverideShim.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableClassToInstanceMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableCollection.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEntry.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntry.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntrySet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapKeySet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapValues.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultisetGwtSerializationDependencies.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedAsList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMapFauxverideShim.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultisetFauxverideShim.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSetFauxverideShim.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/IndexedImmutableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interner.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interners.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterables.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterators.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LexicographicalOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimapGwtSerializationDependencies.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ListMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Lists.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapDifference.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMaker.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMakerInternalMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Maps.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MinMaxPriorityQueue.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MoreCollectors.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MultimapBuilder.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimaps.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multisets.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MutableClassToInstanceMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NaturalOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsFirstOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsLastOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ObjectArrays.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Ordering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/PeekingIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Platform.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Range.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeGwtSerializationDependencies.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularContiguousSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableAsList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseNaturalOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Serialization.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Sets.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableBiMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableList.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterable.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterables.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedLists.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMapDifference.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisetBridge.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisets.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedSetMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Streams.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Synchronized.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Table.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TopKSelector.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedListIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultimap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeSet.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableListIterator.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableSortedMultiset.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UsingToStringOrdering.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/WellBehavedMap.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/package-info.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/IntMath.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/LongMath.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/MathPreconditions.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/package-info.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Booleans.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Doubles.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Ints.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Longs.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/ParseRequest.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Primitives.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/UnsignedLongs.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/package-info.java
 create mode 100644 sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/KafkaCosmosChangeFeedState.java
 create mode 100644 sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ImplementationBridgeHelpersTest.java
 create mode 100644 sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ChannelAcquisitionException.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RolePropertiesExternalIdentity.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ResourceManagerUtils.java
 delete mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/Utils.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ActiveDirectoryAuth.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/AuthConfig.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PasswordAuth.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrincipalType.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PromoteRequest.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleType.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/proxy-config.json
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/reflect-config.json
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PromoteRequestTests.java
 create mode 100644 sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolePropertiesExternalIdentityTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2DatasetTypeProperties.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2LinkedServiceTypeProperties.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2LinkedServiceTypeProperties.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2TableDatasetTypeProperties.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/ServiceNowV2LinkedServiceTypeProperties.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2Type.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2AuthenticationType.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2LinkedService.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2ObjectDataset.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2Source.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2LinkedService.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2Source.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2TableDataset.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2AuthenticationType.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2LinkedService.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2ObjectDataset.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2Source.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExpressionV2Tests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2DatasetTypePropertiesTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2ObjectDatasetTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2SourceTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2SourceTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTypePropertiesTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2ObjectDatasetTests.java
 create mode 100644 sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2SourceTests.java
 create mode 100644 sdk/eventgrid/azure-resourcemanager-eventgrid/ci.yml
 create mode 100644 sdk/identity/azure-identity-extensions/ci.yml
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/BinaryHardeningsClient.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoCertificatesClient.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoKeysClient.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CvesClient.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/Fist.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/IoTFirmwareDefense.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/PasswordHashesClient.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SbomComponentsClient.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SummariesClient.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResourceInner.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/{BinaryHardeningInner.java => BinaryHardeningResult.java} (65%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/ComponentInner.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/{CryptoCertificateInner.java => CryptoCertificate.java} (73%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateResourceInner.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/{CryptoKeyInner.java => CryptoKey.java} (75%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyResourceInner.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResourceInner.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/{CveInner.java => CveResult.java} (54%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveSummaryInner.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareSummaryInner.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/{PasswordHashInner.java => PasswordHash.java} (78%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashResourceInner.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponent.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponentResourceInner.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SummaryResourceInner.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{BinaryHardeningImpl.java => BinaryHardeningResourceImpl.java} (63%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningSummaryImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsImpl.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{CryptoCertificateImpl.java => CryptoCertificateResourceImpl.java} (77%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateSummaryImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesImpl.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{CryptoKeyImpl.java => CryptoKeyResourceImpl.java} (74%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeySummaryImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysImpl.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{CveImpl.java => CveResourceImpl.java} (67%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveSummaryImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesImpl.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareSummaryImpl.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{FistBuilder.java => IoTFirmwareDefenseBuilder.java} (56%)
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{FistImpl.java => IoTFirmwareDefenseImpl.java} (60%)
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{PasswordHashImpl.java => PasswordHashResourceImpl.java} (68%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ResourceManagerUtils.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/{ComponentImpl.java => SbomComponentResourceImpl.java} (62%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesClientImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesImpl.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummaryResourceImpl.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/Utils.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardening.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{BinaryHardeningList.java => BinaryHardeningListResult.java} (70%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningResource.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummary.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/{fluent/models/BinaryHardeningSummaryInner.java => models/BinaryHardeningSummaryResource.java} (67%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardenings.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CanaryFlag.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Component.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateListResult.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{CryptoCertificate.java => CryptoCertificateResource.java} (69%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummary.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/{fluent/models/CryptoCertificateSummaryInner.java => models/CryptoCertificateSummaryResource.java} (72%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificates.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{CryptoKeyList.java => CryptoKeyListResult.java} (67%)
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{CryptoKey.java => CryptoKeyResource.java} (56%)
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummary.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/{fluent/models/CryptoKeySummaryInner.java => models/CryptoKeySummaryResource.java} (72%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeys.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cve.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveComponent.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{CveList.java => CveListResult.java} (73%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveResource.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cves.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsExpired.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsSelfSigned.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsShortKeySize.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsUpdateAvailable.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsWeakSignature.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/NxFlag.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashList.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashListResult.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{PasswordHash.java => PasswordHashResource.java} (58%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashes.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PieFlag.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/RelroFlag.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{ComponentList.java => SbomComponentListResult.java} (66%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentResource.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponents.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StatusMessage.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StrippedFlag.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Summaries.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/{CryptoCertificateList.java => SummaryListResult.java} (68%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryName.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResource.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResourceProperties.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryType.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/proxy-config.json
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/reflect-config.json
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListByFirmwareSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificatesListByFirmwareSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoKeysListByFirmwareSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareCreateSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareDeleteSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningDetailsSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningSummarySamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateComponentDetailsSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoCertificateSummarySamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoKeySummarySamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCveSummarySamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateDownloadUrlSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateFilesystemDownloadUrlSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateSummarySamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGetSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateBinaryHardeningListSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateComponentListSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoCertificateListSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoKeyListSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCveListSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGeneratePasswordHashListSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetSamples.java
 rename sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/{FirmwareListByWorkspaceSamples.java => FirmwaresListByWorkspaceSamples.java} (62%)
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresUpdateSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PasswordHashesListByFirmwareSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetSamples.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareSamples.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListResultTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResourceInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResultTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryResourceTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningsListByFirmwareMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentInnerTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentListTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveComponentTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListResultTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResourceInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResultTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareInnerTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwarePropertiesTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateDefinitionTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningDetailsWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningSummaryWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateComponentDetailsWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateCveSummaryWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateSummaryWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetWithResponseMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateBinaryHardeningListMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateComponentListMockTests.java
 delete mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateCveListMockTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentListResultTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentResourceInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareMockTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetWithResponseMockTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareMockTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryListResultTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourceInnerTests.java
 create mode 100644 sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourcePropertiesTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ExtendedUeInformationsClient.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/UeInformationsClient.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ExtendedUeInfoInner.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/UeInfoInner.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInfoImpl.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsClientImpl.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsImpl.java
 rename sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/{Utils.java => ResourceManagerUtils.java} (90%)
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInfoImpl.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsClientImpl.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsImpl.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AmfId.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DnnIpPair.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfo.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfoProperties.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInformations.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GNbId.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GlobalRanNodeId.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti4G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti5G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioning.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioningState.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPublicKey.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MmeId.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PdnType.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetwork.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetworkHomeNetworkPublicKeys.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RatType.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RrcEstablishmentCause.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo4G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo5G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4GProperties.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5GProperties.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoList.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoPropertiesFormat.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInformations.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeIpAddress.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeLocationInfo.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeQosFlow.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo4G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo5G.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeState.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeUsageSetting.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationGetSamples.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationListSamples.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmfIdTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DnnIpPairTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoInnerTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoPropertiesTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetWithResponseMockTests.java => mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationsGetWithResponseMockTests.java} (70%)
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GNbIdTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GlobalRanNodeIdTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/Guti5GTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPrivateKeysProvisioningTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPublicKeyTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneInnerTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneListResultTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanePropertiesFormatTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateMockTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupWithResponseMockTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupMockTests.java
 delete mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListMockTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PublicLandMobileNetworkHomeNetworkPublicKeysTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo4GTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo5GTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GPropertiesTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoInnerTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoListTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoPropertiesFormatTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationsListMockTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeIpAddressTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeLocationInfoTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeQosFlowTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo4GTests.java
 create mode 100644 sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo5GTests.java
 rename sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/{MetricsBatchQueryOptions.java => MetricsQueryResourcesOptions.java} (88%)
 rename sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/{MetricsBatchQueryResult.java => MetricsQueryResourcesResult.java} (85%)
 rename sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/{MetricsBatchQuerySample.java => MetricsSample.java} (83%)
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderAccountsClient.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderBackupVaultsClient.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderVolumesClient.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupInner.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupPatchProperties.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupProperties.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupStatusInner.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultInner.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultProperties.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/RegionInfoResourceInner.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupStatusImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsClientImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsClientImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesClientImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosClientImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosImpl.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/RegionInfoResourceImpl.java
 rename sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/{Utils.java => ResourceManagerUtils.java} (99%)
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backup.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupPatch.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupRestoreFiles.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupStatus.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupType.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVault.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultPatch.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsMigrationRequest.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderBackupVaults.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderVolumes.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/EncryptionMigrationRequest.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfoResource.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RemotePath.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumeBackupProperties.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsDeleteSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsGetSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsListByNetAppAccountSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsMigrateEncryptionKeySamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsDeleteSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsGetSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListByNetAppAccountSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsUpdateSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsCreateSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsDeleteSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsListByVaultSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderAccountMigrateBackupsSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderBackupVaultRestoreFilesSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderVolumeMigrateBackupsSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUpdateSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosListSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesSplitCloneFromParentSamples.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupInnerTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchPropertiesTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPropertiesTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupRestoreFilesTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupStatusInnerTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultInnerTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPatchTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPropertiesTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsListTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsMigrationRequestTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionMigrationRequestTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetWithResponseMockTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoResourceInnerTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfosListTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RemotePathTests.java
 delete mode 100644 sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupPropertiesTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/BillingInfoesClient.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/ConnectedPartnerResourcesClient.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitoredSubscriptionsClient.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/BillingInfoResponseInner.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/ConnectedPartnerResourcesListFormatInner.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/LinkedResourceInner.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionInner.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionPropertiesInner.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoResponseImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesClientImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesClientImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesListFormatImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/LinkedResourceImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionPropertiesImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsClientImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsImpl.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ResourceManagerUtils.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/Utils.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoResponse.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoes.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConfigurationName.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourceProperties.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResources.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListFormat.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListResponse.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResource.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResourceListResponse.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSaaSInfo.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscription.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionProperties.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionPropertiesList.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptions.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PartnerBillingEntity.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PatchOperation.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Status.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SubscriptionList.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/proxy-config.json
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/reflect-config.json
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoGetSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsDeleteSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsUpdateSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesSamples.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoResponseInnerTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoesGetWithResponseMockTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcePropertiesTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListFormatInnerTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListMockTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListResponseTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceInnerTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceListResponseTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MarketplaceSaaSInfoTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionInnerTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesInnerTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesListTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateMockTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetWithResponseMockTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListMockTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateMockTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupWithResponseMockTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupMockTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesMockTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMockTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingWithResponseMockTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceInnerTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceListResultTests.java
 delete mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceUpdateTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PartnerBillingEntityTests.java
 create mode 100644 sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SubscriptionListTests.java
 rename sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/{NotificationHubsManagementClient.java => NotificationHubsRPClient.java} (74%)
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/PrivateEndpointConnectionsClient.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BrowserCredentialProperties.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/FcmV1CredentialProperties.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentials.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsProperties.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateEndpointConnectionResourceInner.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateLinkResourceInner.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/XiaomiCredentialProperties.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientBuilder.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientBuilder.java
 rename sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/{NotificationHubsManagementClientImpl.java => NotificationHubsRPClientImpl.java} (73%)
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionResourceImpl.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsClientImpl.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsImpl.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateLinkResourceImpl.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceManagerUtils.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/Utils.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Availability.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BrowserCredential.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/FcmV1Credential.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/IpRule.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/LogSpecification.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MetricSpecification.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceCreateOrUpdateParameters.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceStatus.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NetworkAcls.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubCreateOrUpdateParameters.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProperties.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProvisioningState.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyResource.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyType.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicykeyResource.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProperties.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProvisioningState.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResource.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResourceListResult.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnections.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkConnectionStatus.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResource.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceListResult.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceProperties.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicInternetAuthorizationRule.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicNetworkAccess.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RegistrationResult.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateEndpointConnection.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateLinkServiceConnectionState.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ReplicationRegion.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ServiceSpecification.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleCreateOrUpdateParameters.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/XiaomiCredential.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ZoneRedundancyPreference.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/proxy-config.json
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/reflect-config.json
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetPnsCredentialsSamples.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesPatchSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesUpdateSamples.java
 delete mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsPatchSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsUpdateSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsDeleteSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateSamples.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/AvailabilityTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityParametersTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityResultInnerTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResponseInnerTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResultTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/IpRuleTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/LogSpecificationTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/MetricSpecificationTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilityWithResponseMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleWithResponseMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteByResourceGroupWithResponseMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NetworkAclsTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilityWithResponseMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendWithResponseMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleWithResponseMockTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteWithResponseMockTests.java => notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteWithResponseMockTests.java} (80%)
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationDisplayTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationInnerTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationListResultTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationPropertiesTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionPropertiesTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceInnerTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceListResultTests.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusWithResponseMockTests.java => notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdWithResponseMockTests.java} (66%)
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainMockTests.java => notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsMockTests.java} (66%)
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateMockTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceInnerTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceListResultTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourcePropertiesTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PublicInternetAuthorizationRuleTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RegistrationResultTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateEndpointConnectionTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateLinkServiceConnectionStateTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/ServiceSpecificationTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/SkuTests.java
 create mode 100644 sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ListKeysResultInner.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ListKeysResultImpl.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ResourceManagerUtils.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/Utils.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKey.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKeys.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/KeyType.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ListKeysResult.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/proxy-config.json
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/reflect-config.json
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationListKeysSamples.java
 create mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationRegenerateKeysSamples.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceInnerTests.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceListResultTests.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceResourcePropertiesTests.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateMockTests.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupMockTests.java
 delete mode 100644 sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/SkusClient.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/RegionSkuDetailInner.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RegionSkuDetailImpl.java
 create mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ResourceManagerUtils.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusClientImpl.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusImpl.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/Utils.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LocationInfo.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegionSkuDetail.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuDetail.java
 create mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/proxy-config.json
 create mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/reflect-config.json
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/SkusListSamples.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/CapabilityTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterInnerTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterListTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterUpdateTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LocationInfoTests.java
 create mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteWithResponseMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesCreateMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesGetByResourceGroupWithResponseMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListByResourceGroupMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListMockTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailInnerTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailsTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuDetailTests.java
 delete mode 100644 sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkusListMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOffboardingsClient.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOnboardingsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsOrgsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsProjectsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsReposClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DefenderForStoragesClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsConfigurationsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsOperationResultsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubOwnersClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubReposClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabGroupsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabProjectsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabSubgroupsClient.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IngestionSettingsClient.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppResourceRegionInfosClient.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ResourceProvidersClient.java} (51%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SensitivitySettingsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsSettingsClient.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionInner.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionResponseInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgInner.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultsList.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgListResponseInner.java} (52%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsProjectInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsRepositoryInner.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectionStringsInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DevOpsConfigurationInner.java
 rename sdk/{redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegionSkuDetails.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GetSensitivitySettingsListResponseInner.java} (50%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GetSensitivitySettingsResponseInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerInner.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsList.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerListResponseInner.java} (53%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubRepositoryInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupInner.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfosList.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupListResponseInner.java} (53%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabProjectInner.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingInner.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingTokenInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MalwareScanningProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OnUploadProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationStatusResultInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SensitiveDataDiscoveryProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ServerVulnerabilityAssessmentsAzureSettingProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ServerVulnerabilityAssessmentsSettingInner.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsClientImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsClientImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionResponseImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgListResponseImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsRepositoryImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectionStringsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStorageSettingImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsListResponseImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsResponseImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerListResponseImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubRepositoryImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupListResponseImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsClientImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingTokenImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsClientImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationStatusResultImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceManagerUtils.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsClientImpl.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsImpl.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/Utils.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediation.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediationState.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AnnotateDefaultBranchState.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollection.java
 rename sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/{ApiCollectionResponseList.java => ApiCollectionList.java} (77%)
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOffboardings.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOnboardings.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Authorization.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoDiscovery.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationUpdateModel.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrg.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgs.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProject.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjects.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepos.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepository.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureServersSetting.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BuiltInInfoType.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CategoryConfiguration.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionStrings.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiem.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemDiscovery.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemOidc.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersImageAssessment.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmGcpOfferingCiemDiscovery.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmGcpOfferingDataSensitivityDiscovery.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmGcpOfferingMdcContainersAgentlessDiscoveryK8S.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmGcpOfferingMdcContainersImageAssessment.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmGcpOfferingVmScanners.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmGcpOfferingVmScannersConfiguration.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersImageAssessment.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersImageAssessment.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorageSetting.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorages.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfiguration.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurations.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsOperationResults.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsProvisioningState.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Enforce.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponseProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponsePropertiesMipInformation.java
 rename sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/{ApiCollectionResponse.java => GitHubOwner.java} (52%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubOwnerListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubOwnerProperties.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/AccountBackups.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubOwners.java} (56%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubRepos.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubRepository.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubRepositoryListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitHubRepositoryProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabGroup.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabGroupListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabGroupProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabGroups.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabProject.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabProjectListResponse.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabProjectProperties.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabProjects.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GitLabSubgroups.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/InfoType.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/IngestionConnectionString.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/IngestionSetting.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/IngestionSettingToken.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/IngestionSettings.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/InheritFromParentState.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Inherited.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Label.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/MinimalRiskLevel.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/MipIntegrationStatus.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/NotificationsSource.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/NotificationsSourceAlert.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/NotificationsSourceAttackPath.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/OnboardingState.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/OperationStatusAutoGenerated.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/OperationStatusResult.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppResourceRegionInfos.java => security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ResourceProviders.java} (50%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ResourcesCoverageStatus.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Roles.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/RuleCategory.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ScopeName.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/SecurityContactName.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/SecurityContactPropertiesAlertNotifications.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/SecurityContactRole.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/SensitivitySettings.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ServerVulnerabilityAssessmentsAzureSettingSelectedProvider.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ServerVulnerabilityAssessmentsSetting.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ServerVulnerabilityAssessmentsSettingKind.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ServerVulnerabilityAssessmentsSettingKindName.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ServerVulnerabilityAssessmentsSettings.java
 rename sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/{IngestionSettingList.java => ServerVulnerabilityAssessmentsSettingsList.java} (58%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/SettingNameAutoGenerated.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/SourceType.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/TargetBranchConfiguration.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/UpdateSensitivitySettingsRequest.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-security/proxy-config.json
 create mode 100644 sdk/security/azure-resourcemanager-security/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-security/reflect-config.json
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionGetSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionListSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionOffboardingDeleteSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionOnboardingCreateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionsGetByAzureApiManagementServiceSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionsListByAzureApiManagementServiceSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionsListByResourceGroupSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionsOffboardAzureApiManagementApiSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ApiCollectionsOnboardAzureApiManagementApiSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AutomationsUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsCreateOrUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsListAvailableSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsCreateOrUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposCreateOrUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DefenderForStorageCreateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DefenderForStorageGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DevOpsConfigurationsCreateOrUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DevOpsConfigurationsDeleteSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DevOpsConfigurationsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DevOpsConfigurationsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DevOpsConfigurationsUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/DevOpsOperationResultsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitHubOwnersGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitHubOwnersListAvailableSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitHubOwnersListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitHubReposGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitHubReposListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitLabGroupsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitLabGroupsListAvailableSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitLabGroupsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitLabProjectsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitLabProjectsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/GitLabSubgroupsListSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/HealthReportOperationGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/HealthReportsGetSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/IngestionSettingsCreateSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/IngestionSettingsDeleteSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/IngestionSettingsGetSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/IngestionSettingsListConnectionStringsSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/IngestionSettingsListSamples.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/IngestionSettingsListTokensSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/PricingsDeleteSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ResourceProviderGetSensitivitySettingsSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ResourceProviderUpdateSensitivitySettingsSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/SensitivitySettingsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsCreateOrUpdateSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsDeleteSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsGetSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/samples/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsListSamples.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ActionableRemediationTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionListTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionOffboardingsDeleteWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionOnboardingsCreateWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionResponseInnerTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionResponseListTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionsGetByAzureApiManagementServiceWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionsGetWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionsListByAzureApiManagementServiceMockTests.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListByNetAppAccountMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionsListByResourceGroupMockTests.java} (65%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionsOffboardAzureApiManagementApiWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ApiCollectionsOnboardAzureApiManagementApiMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AutomationUpdateModelTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgListResponseInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsCreateOrUpdateMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsGetWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsListAvailableWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsOrgsListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectListResponseTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsCreateOrUpdateMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsGetWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsProjectsListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposCreateOrUpdateMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposGetWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsReposListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsRepositoryInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsRepositoryListResponseTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureDevOpsRepositoryPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/AzureServersSettingTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/BuiltInInfoTypeTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/CategoryConfigurationTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ConnectionStringsInnerTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ConnectorSettingInnerTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ConnectorSettingListTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ConnectorsCreateOrUpdateWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ConnectorsGetWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ConnectorsListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmAwsOfferingCiemDiscoveryTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmAwsOfferingCiemOidcTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmAwsOfferingCiemTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8STests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmAwsOfferingMdcContainersImageAssessmentTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmGcpOfferingCiemDiscoveryTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmGcpOfferingDataSensitivityDiscoveryTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmGcpOfferingMdcContainersAgentlessDiscoveryK8STests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmGcpOfferingMdcContainersImageAssessmentTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmGcpOfferingVmScannersConfigurationTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderCspmGcpOfferingVmScannersTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8STests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderForContainersAwsOfferingMdcContainersImageAssessmentTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8STests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DefenderForContainersGcpOfferingMdcContainersImageAssessmentTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DevOpsConfigurationsDeleteMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/DevOpsOperationResultsGetWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GetSensitivitySettingsListResponseInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GetSensitivitySettingsResponseInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GetSensitivitySettingsResponsePropertiesMipInformationTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GetSensitivitySettingsResponsePropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubOwnerInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubOwnerListResponseInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubOwnerPropertiesTests.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetWithResponseMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubOwnersGetWithResponseMockTests.java} (62%)
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsCreateMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubOwnersListAvailableWithResponseMockTests.java} (51%)
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosListMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubOwnersListMockTests.java} (59%)
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AccountBackupsGetWithResponseMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubReposGetWithResponseMockTests.java} (59%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubReposListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubRepositoryInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubRepositoryListResponseTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitHubRepositoryPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabGroupInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabGroupListResponseInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabGroupPropertiesTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesGetWithResponseMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabGroupsGetWithResponseMockTests.java} (61%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabGroupsListAvailableWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabGroupsListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabProjectInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabProjectListResponseTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabProjectPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabProjectsGetWithResponseMockTests.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsListByVaultMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabProjectsListMockTests.java} (58%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/GitLabSubgroupsListWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/HealthReportsListTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/InfoTypeTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/InformationProtectionPoliciesCreateOrUpdateWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/InformationProtectionPoliciesGetWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/InformationProtectionPoliciesListMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/InformationProtectionPolicyListTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionConnectionStringTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingInnerTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingListTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingsCreateWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingsDeleteWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingsGetWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingsListConnectionStringsWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/IngestionSettingsListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/LabelTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/NotificationsSourceAlertTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/NotificationsSourceAttackPathTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/NotificationsSourceTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/OnUploadPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/OperationStatusResultInnerTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/PricingInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/PricingsDeleteByResourceGroupWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/PricingsGetWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/PricingsUpdateWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ResourceProvidersGetSensitivitySettingsWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ResourceProvidersUpdateSensitivitySettingsWithResponseMockTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/SecurityAssessmentListTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/SecurityAssessmentMetadataResponseListTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/SecurityContactPropertiesAlertNotificationsTests.java
 delete mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/SecuritySubAssessmentListTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/SensitivitySettingsListWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsAzureSettingPropertiesTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingInnerTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsCreateOrUpdateWithResponseMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsDeleteWithResponseMockTests.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsGetWithResponseMockTests.java => security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsGetWithResponseMockTests.java} (68%)
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsListMockTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/ServerVulnerabilityAssessmentsSettingsListTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/TargetBranchConfigurationTests.java
 create mode 100644 sdk/security/azure-resourcemanager-security/src/test/java/com/azure/resourcemanager/security/generated/UpdateSensitivitySettingsRequestTests.java
 delete mode 100644 sdk/serialization/azure-xml/src/main/java/com/azure/xml/XmlProvider.java
 delete mode 100644 sdk/serialization/azure-xml/src/main/java/com/azure/xml/XmlProviders.java
 delete mode 100644 sdk/serialization/azure-xml/src/main/java/com/azure/xml/implementation/DefaultXmlReader.java
 delete mode 100644 sdk/serialization/azure-xml/src/main/java/com/azure/xml/implementation/DefaultXmlWriter.java
 delete mode 100644 sdk/serialization/azure-xml/src/main/java/com/azure/xml/implementation/package-info.java
 create mode 100644 sdk/spring/spring-cloud-azure-stream-binder-eventhubs/src/main/java/com/azure/spring/cloud/stream/binder/eventhubs/config/ExtendedBindingHandlerMappingsProviderConfiguration.java
 create mode 100644 sdk/spring/spring-cloud-azure-stream-binder-eventhubs/src/main/resources/META-INF/spring.factories
 create mode 100644 sdk/spring/spring-cloud-azure-stream-binder-eventhubs/src/test/java/com/azure/spring/cloud/stream/binder/eventhubs/config/ExtendedBindingHandlerMappingsProviderAutoConfigurationTests.java
 create mode 100644 sdk/spring/spring-cloud-azure-stream-binder-servicebus/src/main/java/com/azure/spring/cloud/stream/binder/servicebus/config/ExtendedBindingHandlerMappingsProviderConfiguration.java
 create mode 100644 sdk/spring/spring-cloud-azure-stream-binder-servicebus/src/main/resources/META-INF/spring.factories
 create mode 100644 sdk/spring/spring-cloud-azure-stream-binder-servicebus/src/test/java/com/azure/spring/cloud/stream/binder/servicebus/config/ExtendedBindingHandlerMappingsProviderAutoConfigurationTests.java
 create mode 100644 sdk/storage/azure-storage-blob-batch/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-blob-batch/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-blob-changefeed/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-blob-changefeed/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-blob-changefeed/src/main/java/com/azure/storage/blob/changefeed/implementation/models/package-info.java
 create mode 100644 sdk/storage/azure-storage-blob-changefeed/src/main/java/com/azure/storage/blob/changefeed/implementation/util/package-info.java
 create mode 100644 sdk/storage/azure-storage-blob-cryptography/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-blob-cryptography/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-blob-nio/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-blob-nio/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-blob/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-blob/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-blob/src/main/java/com/azure/storage/blob/implementation/accesshelpers/package-info.java
 create mode 100644 sdk/storage/azure-storage-common/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-common/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-common/src/main/java/com/azure/storage/common/implementation/connectionstring/package-info.java
 create mode 100644 sdk/storage/azure-storage-file-datalake/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-file-datalake/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-file-share/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-file-share/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-file-share/src/main/java/com/azure/storage/file/share/implementation/accesshelpers/package-info.java
 create mode 100644 sdk/storage/azure-storage-internal-avro/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-internal-avro/spotbugs-exclude.xml
 create mode 100644 sdk/storage/azure-storage-internal-avro/src/main/java/com/azure/storage/internal/avro/implementation/schema/complex/package-info.java
 create mode 100644 sdk/storage/azure-storage-internal-avro/src/main/java/com/azure/storage/internal/avro/implementation/schema/file/package-info.java
 create mode 100644 sdk/storage/azure-storage-internal-avro/src/main/java/com/azure/storage/internal/avro/implementation/schema/package-info.java
 create mode 100644 sdk/storage/azure-storage-internal-avro/src/main/java/com/azure/storage/internal/avro/implementation/schema/primitive/package-info.java
 create mode 100644 sdk/storage/azure-storage-queue/checkstyle-suppressions.xml
 create mode 100644 sdk/storage/azure-storage-queue/spotbugs-exclude.xml
 rename sdk/{security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/HealthReportOperationsClient.java => support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/LookUpResourceIdsClient.java} (52%)
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/ProblemClassificationsNoSubscriptionsClient.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/ServiceClassificationsClient.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/ServiceClassificationsNoSubscriptionsClient.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/SupportTicketCommunicationsNoSubscriptionsClient.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/models/ClassificationService.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/models/LookUpResourceIdResponseInner.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/models/ProblemClassificationProperties.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/models/ProblemClassificationPropertiesInner.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/models/ProblemClassificationsClassificationOutputInner.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/models/ServiceClassificationOutputInner.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/LookUpResourceIdResponseImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/LookUpResourceIdsClientImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/LookUpResourceIdsImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ProblemClassificationPropertiesImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ProblemClassificationsClassificationOutputImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ProblemClassificationsNoSubscriptionsClientImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ProblemClassificationsNoSubscriptionsImpl.java
 rename sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/{Utils.java => ResourceManagerUtils.java} (90%)
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ServiceClassificationOutputImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ServiceClassificationsClientImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ServiceClassificationsImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ServiceClassificationsNoSubscriptionsClientImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/ServiceClassificationsNoSubscriptionsImpl.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/SupportTicketChatTranscriptsNoSubscriptionsClientImpl.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/SupportTicketChatTranscriptsNoSubscriptionsImpl.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/SupportTicketCommunicationsNoSubscriptionsClientImpl.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/implementation/SupportTicketCommunicationsNoSubscriptionsImpl.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/IsTemporaryTicket.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/LookUpResourceIdRequest.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/LookUpResourceIdResponse.java
 rename sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/{SupportTicketChatTranscriptsNoSubscriptions.java => LookUpResourceIds.java} (54%)
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ProblemClassificationProperties.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ProblemClassificationsClassificationInput.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ProblemClassificationsClassificationOutput.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ProblemClassificationsClassificationResult.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ProblemClassificationsNoSubscriptions.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ResourceType.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ServiceClassificationAnswer.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ServiceClassificationOutput.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ServiceClassificationRequest.java
 rename sdk/{security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/HealthReportOperations.java => support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ServiceClassifications.java} (53%)
 rename sdk/{redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Skus.java => support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/ServiceClassificationsNoSubscriptions.java} (50%)
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/models/SupportTicketCommunicationsNoSubscriptions.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/ChatTranscriptsNoSubscriptionListSamples.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/CommunicationsNoSubscriptionListSamples.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/LookUpResourceIdPostSamples.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/ProblemClassificationsClassifyProblemsSamples.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/ProblemClassificationsNoSubscriptionClassifyProblemsSamples.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/ServiceClassificationsClassifyServicesSamples.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/ServiceClassificationsNoSubscriptionClassifyServicesSamples.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/SupportTicketChatTranscriptsNoSubscriptionListSamples.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/samples/java/com/azure/resourcemanager/support/generated/SupportTicketCommunicationsNoSubscriptionListSamples.java
 rename sdk/{netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AccountBackupsListByNetAppAccountMockTests.java => support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ChatTranscriptsNoSubscriptionsListMockTests.java} (57%)
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ClassificationServiceTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesListMockTests.java => support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/CommunicationsNoSubscriptionsListMockTests.java} (62%)
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/LookUpResourceIdRequestTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/LookUpResourceIdResponseInnerTests.java
 rename sdk/{communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsCreateOrUpdateWithResponseMockTests.java => support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/LookUpResourceIdsPostWithResponseMockTests.java} (64%)
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationPropertiesInnerTests.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationPropertiesTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationsClassificationInputTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationsClassificationOutputInnerTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationsClassificationResultTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationsClassifyProblemsWithResponseMockTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ProblemClassificationsNoSubscriptionsClassifyProblemsWithResponseMockTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ServiceClassificationAnswerTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ServiceClassificationOutputInnerTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ServiceClassificationRequestTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ServiceClassificationsClassifyServicesWithResponseMockTests.java
 create mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/ServiceClassificationsNoSubscriptionsClassifyServicesWithResponseMockTests.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/SupportTicketChatTranscriptsNoSubscriptionsListMockTests.java
 delete mode 100644 sdk/support/azure-resourcemanager-support/src/test/java/com/azure/resourcemanager/support/generated/SupportTicketCommunicationsNoSubscriptionsListMockTests.java

diff --git a/.github/event-processor.config b/.github/event-processor.config
index 3cecb54ae77d1..52e731644b0fd 100644
--- a/.github/event-processor.config
+++ b/.github/event-processor.config
@@ -22,5 +22,5 @@
   "IdentifyStalePullRequests": "On",
   "CloseAddressedIssues": "On",
   "LockClosedIssues": "On",
-  "EnforceMaxLifeOfIssues": "Off"
+  "EnforceMaxLifeOfIssues": "On"
 }
diff --git a/.gitignore b/.gitignore
index 61bbb8d8b2512..01be8561c4ec0 100644
--- a/.gitignore
+++ b/.gitignore
@@ -1,5 +1,6 @@
 # Default Assets restore directory
 .assets
+test-proxy.log
 
 *.class
 
@@ -118,4 +119,4 @@ stress-test-resources.json
 stress-test-addons*
 
 # Temp typespec files
-TempTypeSpecFiles/
\ No newline at end of file
+TempTypeSpecFiles/
diff --git a/.vscode/cspell.json b/.vscode/cspell.json
index 0bd6a639cdd46..32f0621d9435b 100644
--- a/.vscode/cspell.json
+++ b/.vscode/cspell.json
@@ -16,6 +16,7 @@
     "**/sdk/**/target/**",
     "**/session-records/**",
     "**/sdk/**/**/assets.json",
+    "**/sdk/**/*-matrix.json",
     "**/chromedriver",
     ".gitignore",
     "*.jar",
diff --git a/SUPPORT.md b/SUPPORT.md
index 6fb53d05f85d2..ffc0c3c67db93 100644
--- a/SUPPORT.md
+++ b/SUPPORT.md
@@ -12,6 +12,8 @@ Customers with an [Azure support plan](https://azure.microsoft.com/support/optio
 We use [GitHub Issues](https://github.com/Azure/azure-sdk-for-java/issues/new/choose) to track bugs, questions, and feature requests.
 GitHub issues are free, but **response time is not guaranteed.** See [GitHub issues support process](https://devblogs.microsoft.com/azure-sdk/github-issue-support-process/) for more details.
 
+To ensure the relevance and manageability of our issue queue, we have an automated process that will close issues that are over two years old and have not been updated in the last 30 days. This measure is designed to help us respond to and resolve current issues more efficiently. We appreciate your understanding and cooperation in maintaining a focused and up-to-date issue tracking system.
+
 ### Community resources
 - Take a look at the [Azure SDK blog](https://devblogs.microsoft.com/azure-sdk/).
 - Search for similar issues in [our GitHub repository](https://github.com/Azure/azure-sdk-for-java/issues)
diff --git a/common/perf-test-core/pom.xml b/common/perf-test-core/pom.xml
index a05f4b083af44..72b6683fc4c51 100644
--- a/common/perf-test-core/pom.xml
+++ b/common/perf-test-core/pom.xml
@@ -51,8 +51,8 @@
                 
                 com.beust:jcommander:[1.78] 
 
-                io.projectreactor:reactor-core:[3.4.34] 
-                io.vertx:vertx-codegen:[4.5.4] 
+                io.projectreactor:reactor-core:[3.4.36] 
+                io.vertx:vertx-codegen:[4.5.5] 
               
             
           
@@ -95,7 +95,7 @@
     
       io.vertx
       vertx-codegen
-      4.5.4 
+      4.5.5 
       provided
     
   
diff --git a/common/smoke-tests/pom.xml b/common/smoke-tests/pom.xml
index 9343a799ca1dc..2690fb8fbf7c2 100644
--- a/common/smoke-tests/pom.xml
+++ b/common/smoke-tests/pom.xml
@@ -118,37 +118,37 @@
     
       com.azure
       azure-cosmos
-      4.56.0 
+      4.57.0 
     
 
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
     
 
     
       com.azure
       azure-security-keyvault-secrets
-      4.8.0 
+      4.8.1 
     
 
     
       com.azure
       azure-messaging-eventhubs
-      5.18.1 
+      5.18.2 
     
 
     
       com.azure
       azure-messaging-eventhubs-checkpointstore-blob
-      1.19.1 
+      1.19.2 
     
 
     
       com.azure
       azure-storage-blob
-      12.25.2 
+      12.25.3 
     
 
     
diff --git a/eng/CredScanSuppression.json b/eng/CredScanSuppression.json
index 8c20f554ccc09..7402c236042db 100644
--- a/eng/CredScanSuppression.json
+++ b/eng/CredScanSuppression.json
@@ -35,9 +35,16 @@
             ],
             "_justification": "Javadoc in azure resourcemanagerhybrid"
         },
+        {
+            "file": [
+                "test-proxy.log"
+            ],
+            "_justification": "Transient test file that is locked by test-proxy and should not be scanned"
+        },
         {
             "file": [
                 "eng/common/testproxy/dotnet-devcert.pfx",
+                "sdk/core/azure-core-test/target/classes/keystore.jks",
                 "sdk/cosmos/azure-cosmos-tests/src/test/resources/server.jks",
                 "sdk/cosmos/azure-cosmos-tests/src/test/resources/client.jks",
                 "sdk/identity/azure-identity/src/test/resources/cert-chain.pem",
@@ -64,6 +71,9 @@
             "file": [
                 "sdk/communication/azure-communication-common-perf/src/main/java/com.azure.communication.common.perf/FakeCredentialInTest.java",
                 "sdk/keyvault/azure-security-keyvault-certificates/src/test/java/com/azure/security/keyvault/certificates/FakeCredentialInTest.java",
+                "sdk/keyvault/microsoft-azure-keyvault/src/test/resources/session-records/createCertificatePemForCertificateOperationsTest.json",
+                "sdk/keyvault/microsoft-azure-keyvault/src/test/resources/session-records/createSelfSignedCertificatePemForCertificateOperationsTest.json",
+                "sdk/resourcemanagerhybrid/azure-resourcemanager/src/test/resources/session-records/AzureResourceManagerTests.testContainerRegistry.json",
                 "sdk/spring/spring-cloud-azure-autoconfigure/src/test/java/com/azure/spring/cloud/autoconfigure/FakeCredentialInTest.java",
                 "sdk/spring/spring-cloud-azure-service/src/test/java/com/azure/spring/cloud/service/implementation/storage/FakeCredentialInTest.java",
                 "sdk/storage/azure-storage-blob/src/test/java/com/azure/storage/blob/FakeCredentialInTest.java",
diff --git a/eng/code-quality-reports/pom.xml b/eng/code-quality-reports/pom.xml
index 6779113c3f74f..88057a8c268d3 100755
--- a/eng/code-quality-reports/pom.xml
+++ b/eng/code-quality-reports/pom.xml
@@ -37,7 +37,7 @@
     
       com.google.guava
       guava
-      33.0.0-jre 
+      33.1.0-jre 
     
 
     
diff --git a/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle.xml b/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle.xml
index 3adb3bc7acbbd..363a6f00868fb 100755
--- a/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle.xml
+++ b/eng/code-quality-reports/src/main/resources/checkstyle/checkstyle.xml
@@ -9,6 +9,8 @@ what the following rules do, please see the checkstyle configuration
 page at http://checkstyle.sourceforge.net/config.html -->
 
 
+  
+
   
   
     
diff --git a/eng/common/pipelines/templates/steps/bypass-local-dns.yml b/eng/common/pipelines/templates/steps/bypass-local-dns.yml
index 922f58a8286c5..6e30c91fd2584 100644
--- a/eng/common/pipelines/templates/steps/bypass-local-dns.yml
+++ b/eng/common/pipelines/templates/steps/bypass-local-dns.yml
@@ -6,6 +6,6 @@ steps:
       condition: |
         and(
           succeededOrFailed(),
-          contains(variables['OSVmImage'], 'ubuntu'),
+          or(contains(variables['OSVmImage'], 'ubuntu'),contains(variables['OSVmImage'], 'linux')),
           eq(variables['Container'], '')
         )
diff --git a/eng/common/pipelines/templates/steps/create-apireview.yml b/eng/common/pipelines/templates/steps/create-apireview.yml
index e85006943772c..c69d05d5ae31d 100644
--- a/eng/common/pipelines/templates/steps/create-apireview.yml
+++ b/eng/common/pipelines/templates/steps/create-apireview.yml
@@ -2,28 +2,37 @@ parameters:
   ArtifactPath: $(Build.ArtifactStagingDirectory)
   Artifacts: []
   ConfigFileDir: $(Build.ArtifactStagingDirectory)/PackageInfo
+  MarkPackageAsShipped: false
+  GenerateApiReviewForManualOnly: false
+  ArtifactName: 'packages'
+  PackageName: ''
 
 steps:
   # ideally this should be done as initial step of a job in caller template
   # We can remove this step later once it is added in caller
   - template: /eng/common/pipelines/templates/steps/set-default-branch.yml
 
-  - ${{ each artifact in parameters.Artifacts }}:
+  # Automatic API review is generated for a package when pipeline runs irrespective of how pipeline gets triggered.
+  # Below condition ensures that API review is generated only for manual pipeline runs when flag GenerateApiReviewForManualOnly is set to true. 
+  - ${{ if or(ne(parameters.GenerateApiReviewForManualOnly, true), eq(variables['Build.Reason'], 'Manual')) }}:
     - task: Powershell@2
       inputs:
         filePath: $(Build.SourcesDirectory)/eng/common/scripts/Create-APIReview.ps1
         arguments: >
+          -ArtifactList ('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object Name)
           -ArtifactPath ${{parameters.ArtifactPath}}
-          -APIViewUri $(azuresdk-apiview-uri)
+          -ArtifactName ${{ parameters.ArtifactName }}
           -APIKey $(azuresdk-apiview-apikey)
-          -APILabel "Auto Review - $(Build.SourceVersion)"
-          -PackageName ${{artifact.name}}
+          -PackageName '${{parameters.PackageName}}'
           -SourceBranch $(Build.SourceBranchName)
           -DefaultBranch $(DefaultBranch)
           -ConfigFileDir '${{parameters.ConfigFileDir}}'
+          -BuildId $(Build.BuildId)
+          -RepoName '$(Build.Repository.Name)'    
+          -MarkPackageAsShipped $${{parameters.MarkPackageAsShipped}}
         pwsh: true
         workingDirectory: $(Pipeline.Workspace)
-      displayName: Create API Review for ${{ artifact.name}}
+      displayName: Create API Review
       condition: >-
         and(
           succeededOrFailed(),
diff --git a/eng/common/pipelines/templates/steps/git-push-changes.yml b/eng/common/pipelines/templates/steps/git-push-changes.yml
index a922b203a9b10..c8fbeaa9769ca 100644
--- a/eng/common/pipelines/templates/steps/git-push-changes.yml
+++ b/eng/common/pipelines/templates/steps/git-push-changes.yml
@@ -10,25 +10,14 @@ parameters:
   SkipCheckingForChanges: false
 
 steps:
-- pwsh: |
-    echo "git add -A"
-    git add -A
-
-    echo "git diff --name-status --cached --exit-code"
-    git diff --name-status --cached --exit-code
-
-    if ($LastExitCode -ne 0) {
-      echo "##vso[task.setvariable variable=HasChanges]$true"
-      echo "Changes detected so setting HasChanges=true"
-    }
-    else {
-      echo "##vso[task.setvariable variable=HasChanges]$false"
-      echo "No changes so skipping code push"
-    }
+- task: PowerShell@2
   displayName: Check for changes
   condition: and(succeeded(), eq(${{ parameters.SkipCheckingForChanges }}, false))
-  workingDirectory: ${{ parameters.WorkingDirectory }}
-  ignoreLASTEXITCODE: true
+  inputs:
+    pwsh: true
+    workingDirectory: ${{ parameters.WorkingDirectory }}
+    filePath: ${{ parameters.ScriptDirectory }}/check-for-git-changes.ps1
+    ignoreLASTEXITCODE: true
 
 - pwsh: |
     # Remove the repo owner from the front of the repo name if it exists there
diff --git a/eng/common/scripts/Create-APIReview.ps1 b/eng/common/scripts/Create-APIReview.ps1
index c3c8f4a46dc67..cdf4671802696 100644
--- a/eng/common/scripts/Create-APIReview.ps1
+++ b/eng/common/scripts/Create-APIReview.ps1
@@ -1,27 +1,37 @@
 [CmdletBinding()]
 Param (
   [Parameter(Mandatory=$True)]
-  [string] $ArtifactPath,
+  [array] $ArtifactList,
   [Parameter(Mandatory=$True)]
-  [string] $APIViewUri,
+  [string] $ArtifactPath,  
   [Parameter(Mandatory=$True)]
-  [string] $APIKey,
-  [Parameter(Mandatory=$True)]
-  [string] $APILabel,
-  [string] $PackageName,
+  [string] $APIKey,  
   [string] $SourceBranch,
   [string] $DefaultBranch,
-  [string] $ConfigFileDir = ""
+  [string] $RepoName,
+  [string] $BuildId,
+  [string] $PackageName = "",
+  [string] $ConfigFileDir = "",
+  [string] $APIViewUri = "https://apiview.dev/AutoReview",
+  [string] $ArtifactName = "packages",
+  [bool] $MarkPackageAsShipped = $false
 )
 
+Set-StrictMode -Version 3
+. (Join-Path $PSScriptRoot common.ps1)
+. (Join-Path $PSScriptRoot Helpers ApiView-Helpers.ps1)
+
 # Submit API review request and return status whether current revision is approved or pending or failed to create review
-function Submit-APIReview($packagename, $filePath, $uri, $apiKey, $apiLabel, $releaseStatus)
+function Upload-SourceArtifact($filePath, $apiLabel, $releaseStatus, $packageVersion)
 {
+    Write-Host "File path: $filePath"
+    $fileName = Split-Path -Leaf $filePath
+    Write-Host "File name: $fileName"
     $multipartContent = [System.Net.Http.MultipartFormDataContent]::new()
     $FileStream = [System.IO.FileStream]::new($filePath, [System.IO.FileMode]::Open)
     $fileHeader = [System.Net.Http.Headers.ContentDispositionHeaderValue]::new("form-data")
     $fileHeader.Name = "file"
-    $fileHeader.FileName = $packagename
+    $fileHeader.FileName = $fileName
     $fileContent = [System.Net.Http.StreamContent]::new($FileStream)
     $fileContent.Headers.ContentDisposition = $fileHeader
     $fileContent.Headers.ContentType = [System.Net.Http.Headers.MediaTypeHeaderValue]::Parse("application/octet-stream")
@@ -35,6 +45,20 @@ function Submit-APIReview($packagename, $filePath, $uri, $apiKey, $apiLabel, $re
     $multipartContent.Add($stringContent)
     Write-Host "Request param, label: $apiLabel"
 
+    $versionParam = [System.Net.Http.Headers.ContentDispositionHeaderValue]::new("form-data")
+    $versionParam.Name = "packageVersion"
+    $versionContent = [System.Net.Http.StringContent]::new($packageVersion)
+    $versionContent.Headers.ContentDisposition = $versionParam
+    $multipartContent.Add($versionContent)
+    Write-Host "Request param, packageVersion: $packageVersion"
+    
+    $releaseTagParam = [System.Net.Http.Headers.ContentDispositionHeaderValue]::new("form-data")
+    $releaseTagParam.Name = "setReleaseTag"
+    $releaseTagParamContent = [System.Net.Http.StringContent]::new($MarkPackageAsShipped)
+    $releaseTagParamContent.Headers.ContentDisposition = $releaseTagParam
+    $multipartContent.Add($releaseTagParamContent)
+    Write-Host "Request param, setReleaseTag: $MarkPackageAsShipped"
+
     if ($releaseStatus -and ($releaseStatus -ne "Unreleased"))
     {
         $compareAllParam = [System.Net.Http.Headers.ContentDispositionHeaderValue]::new("form-data")
@@ -45,6 +69,7 @@ function Submit-APIReview($packagename, $filePath, $uri, $apiKey, $apiLabel, $re
         Write-Host "Request param, compareAllRevisions: true"
     }
 
+    $uri = "${APIViewUri}/UploadAutoReview"
     $headers = @{
         "ApiKey" = $apiKey;
         "content-type" = "multipart/form-data"
@@ -53,7 +78,6 @@ function Submit-APIReview($packagename, $filePath, $uri, $apiKey, $apiLabel, $re
     try
     {
         $Response = Invoke-WebRequest -Method 'POST' -Uri $uri -Body $multipartContent -Headers $headers
-        Write-Host "API Review URL: $($Response.Content)"
         $StatusCode = $Response.StatusCode
     }
     catch
@@ -65,118 +89,225 @@ function Submit-APIReview($packagename, $filePath, $uri, $apiKey, $apiLabel, $re
     return $StatusCode
 }
 
+function Upload-ReviewTokenFile($packageName, $apiLabel, $releaseStatus, $reviewFileName, $packageVersion)
+{
+    $params = "buildId=${BuildId}&artifactName=${ArtifactName}&originalFilePath=${packageName}&reviewFilePath=${reviewFileName}"    
+    $params += "&label=${apiLabel}&repoName=${RepoName}&packageName=${packageName}&project=internal&packageVersion=${packageVersion}"
+    if($MarkPackageAsShipped) {
+        $params += "&setReleaseTag=true"
+    }
+    $uri = "${APIViewUri}/CreateApiReview?${params}"
+    if ($releaseStatus -and ($releaseStatus -ne "Unreleased"))
+    {
+        $uri += "&compareAllRevisions=true"
+    }
 
-. (Join-Path $PSScriptRoot common.ps1)
+    Write-Host "Request to APIView: $uri"
+    $headers = @{
+        "ApiKey" = $APIKey;
+    }
 
-Write-Host "Artifact path: $($ArtifactPath)"
-Write-Host "Package Name: $($PackageName)"
-Write-Host "Source branch: $($SourceBranch)"
-Write-Host "Config File directory: $($ConfigFileDir)"
+    try
+    {
+        $Response = Invoke-WebRequest -Method 'GET' -Uri $uri -Headers $headers
+        $StatusCode = $Response.StatusCode
+    }
+    catch
+    {
+        Write-Host "Exception details: $($_.Exception)"
+        $StatusCode = $_.Exception.Response.StatusCode
+    }
 
-$packages = @{}
-if ($FindArtifactForApiReviewFn -and (Test-Path "Function:$FindArtifactForApiReviewFn"))
-{
-    $packages = &$FindArtifactForApiReviewFn $ArtifactPath $PackageName
+    return $StatusCode
 }
-else
+
+function Get-APITokenFileName($packageName)
 {
-    Write-Host "The function for 'FindArtifactForApiReviewFn' was not found.`
-    Make sure it is present in eng/scripts/Language-Settings.ps1 and referenced in eng/common/scripts/common.ps1.`
-    See https://github.com/Azure/azure-sdk-tools/blob/main/doc/common/common_engsys.md#code-structure"
-    exit(1)
+    $reviewTokenFileName = "${packageName}_${LanguageShort}.json"
+    $tokenFilePath = Join-Path $ArtifactPath $packageName $reviewTokenFileName
+    if (Test-Path $tokenFilePath) {
+        Write-Host "Review token file is present at $tokenFilePath"
+        return $reviewTokenFileName
+    }
+    else {
+        Write-Host "Review token file is not present at $tokenFilePath"
+        return $null
+    }
 }
 
-# Check if package config file is present. This file has package version, SDK type etc info.
-if (-not $ConfigFileDir)
+function Submit-APIReview($packageInfo, $packagePath)
 {
-    $ConfigFileDir = Join-Path -Path $ArtifactPath "PackageInfo"
+    $packageName = $packageInfo.Name    
+    $apiLabel = "Source Branch:${SourceBranch}"
+
+    # Get generated review token file if present
+    # APIView processes request using different API if token file is already generated
+    $reviewTokenFileName =  Get-APITokenFileName $packageName
+    if ($reviewTokenFileName) {
+        Write-Host "Uploading review token file $reviewTokenFileName to APIView."
+        return Upload-ReviewTokenFile $packageName $apiLabel $packageInfo.ReleaseStatus $reviewTokenFileName $packageInfo.Version
+    }
+    else {
+        Write-Host "Uploading $packagePath to APIView."
+        return Upload-SourceArtifact $packagePath $apiLabel $packageInfo.ReleaseStatus $packageInfo.Version
+    }
 }
 
-if ($packages)
+
+function ProcessPackage($packageName)
 {
-    foreach($pkgPath in $packages.Values)
+    $packages = @{}
+    if ($FindArtifactForApiReviewFn -and (Test-Path "Function:$FindArtifactForApiReviewFn"))
     {
-        $pkg = Split-Path -Leaf $pkgPath
-        $pkgPropPath = Join-Path -Path $ConfigFileDir "$PackageName.json"
-        if (-Not (Test-Path $pkgPropPath))
-        {
-            Write-Host " Package property file path $($pkgPropPath) is invalid."
-            continue
-        }
-        # Get package info from json file created before updating version to daily dev
-        $pkgInfo = Get-Content $pkgPropPath | ConvertFrom-Json
-        $version = [AzureEngSemanticVersion]::ParseVersionString($pkgInfo.Version)
-        if ($version -eq $null)
-        {
-            Write-Host "Version info is not available for package $PackageName, because version '$(pkgInfo.Version)' is invalid. Please check if the version follows Azure SDK package versioning guidelines."
-            exit 1
-        }
-
-        Write-Host "Version: $($version)"
-        Write-Host "SDK Type: $($pkgInfo.SdkType)"
-        Write-Host "Release Status: $($pkgInfo.ReleaseStatus)"
+        $packages = &$FindArtifactForApiReviewFn $ArtifactPath $packageName
+    }
+    else
+    {
+        Write-Host "The function for 'FindArtifactForApiReviewFn' was not found.`
+        Make sure it is present in eng/scripts/Language-Settings.ps1 and referenced in eng/common/scripts/common.ps1.`
+        See https://github.com/Azure/azure-sdk-tools/blob/main/doc/common/common_engsys.md#code-structure"
+        return 1
+    }
 
-        # Run create review step only if build is triggered from main branch or if version is GA.
-        # This is to avoid invalidating review status by a build triggered from feature branch
-        if ( ($SourceBranch -eq $DefaultBranch) -or (-not $version.IsPrerelease))
+    if ($packages)
+    {
+        foreach($pkgPath in $packages.Values)
         {
-            Write-Host "Submitting API Review for package $($pkg)"
-            $respCode = Submit-APIReview -packagename $pkg -filePath $pkgPath -uri $APIViewUri -apiKey $APIKey -apiLabel $APILabel -releaseStatus $pkgInfo.ReleaseStatus
-            Write-Host "HTTP Response code: $($respCode)"
-            # HTTP status 200 means API is in approved status
-            if ($respCode -eq '200')
-            {
-                Write-Host "API review is in approved status."
-            }
-            elseif ($version.IsPrerelease)
+            $pkg = Split-Path -Leaf $pkgPath
+            $pkgPropPath = Join-Path -Path $ConfigFileDir "$packageName.json"
+            if (-Not (Test-Path $pkgPropPath))
             {
-                # Check if package name is approved. Preview version cannot be released without package name approval
-                if ($respCode -eq '202' -and $pkgInfo.ReleaseStatus -and $pkgInfo.ReleaseStatus -ne "Unreleased")
-                {
-                    Write-Host "Package name is not yet approved on APIView for $($PackageName). Package name must be approved by an API approver for a beta release if it was never released a stable version."
-                    Write-Host "You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on package name approval."
-                    exit 1
-                }
-                # Ignore API review status for prerelease version
-                Write-Host "Package version is not GA. Ignoring API view approval status"
+                Write-Host " Package property file path $($pkgPropPath) is invalid."
+                continue
             }
-            elseif (!$pkgInfo.ReleaseStatus -or $pkgInfo.ReleaseStatus -eq "Unreleased")
+            # Get package info from json file created before updating version to daily dev
+            $pkgInfo = Get-Content $pkgPropPath | ConvertFrom-Json
+            $version = [AzureEngSemanticVersion]::ParseVersionString($pkgInfo.Version)
+            if ($version -eq $null)
             {
-                Write-Host "Release date is not set for current version in change log file for package. Ignoring API review approval status since package is not yet ready for release."
+                Write-Host "Version info is not available for package $packageName, because version '$(pkgInfo.Version)' is invalid. Please check if the version follows Azure SDK package versioning guidelines."
+                return 1
             }
-            else
+            
+            Write-Host "Version: $($version)"
+            Write-Host "SDK Type: $($pkgInfo.SdkType)"
+            Write-Host "Release Status: $($pkgInfo.ReleaseStatus)"
+
+            # Run create review step only if build is triggered from main branch or if version is GA.
+            # This is to avoid invalidating review status by a build triggered from feature branch
+            if ( ($SourceBranch -eq $DefaultBranch) -or (-not $version.IsPrerelease) -or $MarkPackageAsShipped)
             {
-                # Return error code if status code is 201 for new data plane package
-                # Temporarily enable API review for spring SDK types. Ideally this should be done be using 'IsReviewRequired' method in language side
-                # to override default check of SDK type client
-                if (($pkgInfo.SdkType -eq "client" -or $pkgInfo.SdkType -eq "spring") -and $pkgInfo.IsNewSdk)
+                Write-Host "Submitting API Review request for package $($pkg), File path: $($pkgPath)"
+                $respCode = Submit-APIReview $pkgInfo $pkgPath
+                Write-Host "HTTP Response code: $($respCode)"
+
+                # no need to check API review status when marking a package as shipped
+                if ($MarkPackageAsShipped)
                 {
-                    if ($respCode -eq '201')
+                    if ($respCode -eq '500')
                     {
-                        Write-Host "Package version $($version) is GA and automatic API Review is not yet approved for package $($PackageName)."
-                        Write-Host "Build and release is not allowed for GA package without API review approval."
-                        Write-Host "You will need to queue another build to proceed further after API review is approved"
-                        Write-Host "You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on API Approval."
+                        Write-Host "Failed to mark package ${packageName} as released. Please reach out to Azure SDK engineering systems on teams channel."   
+                        return 1
                     }
-                    else
+                    Write-Host "Package ${packageName} is marked as released."   
+                    return 0
+                }
+
+                $apiStatus = [PSCustomObject]@{
+                    IsApproved = $false
+                    Details = ""
+                }
+                $pkgNameStatus = [PSCustomObject]@{
+                    IsApproved = $false
+                    Details = ""
+                }
+                Process-ReviewStatusCode $respCode $packageName $apiStatus $pkgNameStatus
+
+                if ($apiStatus.IsApproved) {
+                    Write-Host "API status: $($apiStatus.Details)"
+                }
+                elseif (!$pkgInfo.ReleaseStatus -or $pkgInfo.ReleaseStatus -eq "Unreleased") {
+                    Write-Host "Release date is not set for current version in change log file for package. Ignoring API review approval status since package is not yet ready for release."
+                }
+                elseif ($version.IsPrerelease)
+                {
+                    # Check if package name is approved. Preview version cannot be released without package name approval
+                    if (!$pkgNameStatus.IsApproved) 
                     {
-                        Write-Host "Failed to create API Review for package $($PackageName). Please reach out to Azure SDK engineering systems on teams channel and share this build details."
+                        Write-Error $($pkgNameStatus.Details)
+                        return 1
                     }
-                    exit 1
-                }
+                    # Ignore API review status for prerelease version
+                    Write-Host "Package version is not GA. Ignoring API view approval status"
+                }                
                 else
                 {
-                    Write-Host "API review is not approved for package $($PackageName), however it is not required for this package type so it can still be released without API review approval."
+                    # Return error code if status code is 201 for new data plane package
+                    # Temporarily enable API review for spring SDK types. Ideally this should be done be using 'IsReviewRequired' method in language side
+                    # to override default check of SDK type client
+                    if (($pkgInfo.SdkType -eq "client" -or $pkgInfo.SdkType -eq "spring") -and $pkgInfo.IsNewSdk)
+                    {
+                        if (!$apiStatus.IsApproved)
+                        {
+                            Write-Host "Package version $($version) is GA and automatic API Review is not yet approved for package $($packageName)."
+                            Write-Host "Build and release is not allowed for GA package without API review approval."
+                            Write-Host "You will need to queue another build to proceed further after API review is approved"
+                            Write-Host "You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on API Approval."
+                        }
+                        return 1
+                    }
+                    else {
+                        Write-Host "API review is not approved for package $($packageName), however it is not required for this package type so it can still be released without API review approval."
+                    }
                 }
             }
+            else {
+                Write-Host "Build is triggered from $($SourceBranch) with prerelease version. Skipping API review status check."
+            }
         }
-        else
-        {
-            Write-Host "Build is triggered from $($SourceBranch) with prerelease version. Skipping API review status check."
-        }
     }
+    else {
+        Write-Host "No package is found in artifact path to submit review request"
+    }
+    return 0
+}
+
+$responses = @{}
+# Check if package config file is present. This file has package version, SDK type etc info.
+if (-not $ConfigFileDir)
+{
+    $ConfigFileDir = Join-Path -Path $ArtifactPath "PackageInfo"
+}
+
+Write-Host "Artifact path: $($ArtifactPath)"
+Write-Host "Source branch: $($SourceBranch)"
+Write-Host "Config File directory: $($ConfigFileDir)"
+
+# if package name param is not empty then process only that package
+if ($PackageName)
+{
+    Write-Host "Processing $($PackageName)"
+    $result = ProcessPackage -packageName $PackageName
+    $responses[$PackageName] = $result 
 }
 else
 {
-    Write-Host "No package is found in artifact path to submit review request"
+    # process all packages in the artifact
+    foreach ($artifact in $ArtifactList)
+    {
+        Write-Host "Processing $($artifact.name)"
+        $result = ProcessPackage -packageName $artifact.name
+        $responses[$artifact.name] = $result
+    }
+}
+
+$exitCode = 0
+foreach($pkg in $responses.keys)
+{    
+    if ($responses[$pkg] -eq 1)
+    {
+        Write-Host "API changes are not approved for $($pkg)"
+        $exitCode = 1
+    }
 }
+exit $exitCode
\ No newline at end of file
diff --git a/eng/common/scripts/Helpers/ApiView-Helpers.ps1 b/eng/common/scripts/Helpers/ApiView-Helpers.ps1
index 73144204f4ecb..bf8b16a99e021 100644
--- a/eng/common/scripts/Helpers/ApiView-Helpers.ps1
+++ b/eng/common/scripts/Helpers/ApiView-Helpers.ps1
@@ -20,7 +20,7 @@ function MapLanguageName($language)
     return $lang
 }
 
-function Check-ApiReviewStatus($packageName, $packageVersion, $language, $url, $apiKey)
+function Check-ApiReviewStatus($packageName, $packageVersion, $language, $url, $apiKey, $apiApprovalStatus = $null, $packageNameStatus = $null)
 {
   # Get API view URL and API Key to check status
   Write-Host "Checking API review status"
@@ -35,31 +35,86 @@ function Check-ApiReviewStatus($packageName, $packageVersion, $language, $url, $
     packageVersion = $packageVersion
   }
 
+  if (!$apiApprovalStatus) {
+    $apiApprovalStatus = [PSCustomObject]@{
+      IsApproved = $false
+      Details = ""
+    }
+  }
+
+  if (!$packageNameStatus) {
+    $packageNameStatus = [PSCustomObject]@{
+      IsApproved = $false
+      Details = ""
+    }
+  }
+
   try
   {
     $response = Invoke-WebRequest $url -Method 'GET' -Headers $headers -Body $body
-    if ($response.StatusCode -eq '200')
-    {
-      Write-Host "API Review is approved for package $($packageName)"
+    Process-ReviewStatusCode -statusCode $response.StatusCode -packageName $packageName -apiApprovalStatus $apiApprovalStatus -packageNameStatus $packageNameStatus
+    if ($apiApprovalStatus.IsApproved) {
+      Write-Host $($apiApprovalStatus.Details)
     }
-    elseif ($response.StatusCode -eq '202')
-    {
-      Write-Host "Package name $($packageName) is not yet approved by an SDK API approver. Package name must be approved to release a beta version if $($packageName) was never released a stable version."
-      Write-Host "You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on package name Approval."
+    else {
+      Write-warning $($apiApprovalStatus.Details)
     }
-    elseif ($response.StatusCode -eq '201')
-    {
-      Write-Warning "API Review is not approved for package $($packageName). Release pipeline will fail if API review is not approved for a stable version release."
-      Write-Host "You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on API Approval."
+    if ($packageNameStatus.IsApproved) {
+      Write-Host $($packageNameStatus.Details)
     }
-    else
-    {
-      Write-Warning "API review status check returned unexpected response. $($response)"
-      Write-Host "You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on API Approval."
+    else {
+      Write-warning $($packageNameStatus.Details)
     }
   }
   catch
   {
     Write-Warning "Failed to check API review status for package $($PackageName). You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on API Approval."
   }
+}
+
+function Process-ReviewStatusCode($statusCode, $packageName, $apiApprovalStatus, $packageNameStatus)
+{
+  $apiApproved = $false
+  $apiApprovalDetails = "API Review is not approved for package $($packageName). Release pipeline will fail if API review is not approved for a GA version release. You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on API Approval."
+
+  $packageNameApproved = $false
+  $packageNameApprovalDetails = ""
+
+  # 200 API approved and Package name approved
+  # 201 API review is not approved, Package name is approved
+  # 202 API review is not approved, Package name is not approved
+
+  switch ($statusCode)
+  {
+    200
+    {
+      $apiApprovalDetails = "API Review is approved for package $($packageName)"
+      $apiApproved = $true
+
+      $packageNameApproved = $true
+      $packageNameApprovalDetails = "Package name is approved for package $($packageName)"
+    }
+    201
+    {
+      $packageNameApproved = $true
+      $packageNameApprovalDetails = "Package name is approved for package $($packageName)"
+    }
+    202
+    {
+      $packageNameApprovalDetails = "Package name $($packageName) is not yet approved by an SDK API approver. Package name must be approved to release a beta version if $($packageName) was never released as a stable version."
+      $packageNameApprovalDetails += " You can check http://aka.ms/azsdk/engsys/apireview/faq for more details on package name Approval."
+    }
+    default
+    {
+      $apiApprovalDetails = "Invalid status code from APIView. status code $($statusCode)"
+      $packageNameApprovalDetails = "Invalid status code from APIView. status code $($statusCode)"
+      Write-Error "Failed to process API Review status for for package $($PackageName). Please reach out to Azure SDK engineering systems on teams channel."
+    }
+  }
+
+  $apiApprovalStatus.IsApproved = $apiApproved
+  $apiApprovalStatus.Details = $apiApprovalDetails
+
+  $packageNameStatus.IsApproved = $packageNameApproved
+  $packageNameStatus.Details = $packageNameApprovalDetails
 }
\ No newline at end of file
diff --git a/eng/common/scripts/Update-DocsMsMetadata.ps1 b/eng/common/scripts/Update-DocsMsMetadata.ps1
index 817407f4bc184..8b7b777d503cb 100644
--- a/eng/common/scripts/Update-DocsMsMetadata.ps1
+++ b/eng/common/scripts/Update-DocsMsMetadata.ps1
@@ -111,18 +111,13 @@ function GetPackageInfoJson ($packageInfoJsonLocation) {
 
   $packageInfoJson = Get-Content $packageInfoJsonLocation -Raw
   $packageInfo = ConvertFrom-Json $packageInfoJson
+  if ($GetDocsMsDevLanguageSpecificPackageInfoFn -and (Test-Path "Function:$GetDocsMsDevLanguageSpecificPackageInfoFn")) {
+    $packageInfo = &$GetDocsMsDevLanguageSpecificPackageInfoFn $packageInfo $PackageSourceOverride
+  }
+  # Default: use the dev version from package info as the version for
+  # downstream processes
   if ($packageInfo.DevVersion) {
-    # If the package is of a dev version there may be language-specific needs to
-    # specify the appropriate version. For example, in the case of JS, the dev
-    # version is always 'dev' when interacting with NPM.
-    if ($GetDocsMsDevLanguageSpecificPackageInfoFn -and (Test-Path "Function:$GetDocsMsDevLanguageSpecificPackageInfoFn")) {
-      $packageInfo = &$GetDocsMsDevLanguageSpecificPackageInfoFn $packageInfo
-    }
-    else {
-      # Default: use the dev version from package info as the version for
-      # downstream processes
-      $packageInfo.Version = $packageInfo.DevVersion
-    }
+    $packageInfo.Version = $packageInfo.DevVersion
   }
   return $packageInfo
 }
@@ -202,7 +197,7 @@ foreach ($packageInfoLocation in $PackageInfoJsonLocations) {
     Write-Host "Validating the packages..."
 
     $packageInfo =  GetPackageInfoJson $packageInfoLocation
-    # This calls a function named "Validate-${Language}-DocMsPackages" 
+    # This calls a function named "Validate-${Language}-DocMsPackages"
     # declared in common.ps1, implemented in Language-Settings.ps1
     $isValid = &$ValidateDocsMsPackagesFn `
       -PackageInfos $packageInfo `
@@ -214,7 +209,7 @@ foreach ($packageInfoLocation in $PackageInfoJsonLocations) {
       Write-Host "Package validation failed for package: $packageInfoLocation"
       $allSucceeded = $false
 
-      # Skip the later call to UpdateDocsMsMetadataForPackage because this 
+      # Skip the later call to UpdateDocsMsMetadataForPackage because this
       # package has not passed validation
       continue
     }
@@ -229,6 +224,6 @@ foreach ($packageInfoLocation in $PackageInfoJsonLocations) {
 # any packages failed validation
 if ($allSucceeded) {
   Write-Host "##vso[task.setvariable variable=DocsMsPackagesAllValid;]$true"
-} else { 
+} else {
   Write-Host "##vso[task.setvariable variable=DocsMsPackagesAllValid;]$false"
 }
diff --git a/eng/common/scripts/check-for-git-changes.ps1 b/eng/common/scripts/check-for-git-changes.ps1
new file mode 100644
index 0000000000000..2c1186ab0b3f8
--- /dev/null
+++ b/eng/common/scripts/check-for-git-changes.ps1
@@ -0,0 +1,14 @@
+echo "git add -A"
+git add -A
+
+echo "git diff --name-status --cached --exit-code"
+git diff --name-status --cached --exit-code
+
+if ($LastExitCode -ne 0) {
+  echo "##vso[task.setvariable variable=HasChanges]$true"
+  echo "Changes detected so setting HasChanges=true"
+}
+else {
+  echo "##vso[task.setvariable variable=HasChanges]$false"
+  echo "No changes so skipping code push"
+}
diff --git a/eng/common/testproxy/onboarding/common-asset-functions.ps1 b/eng/common/testproxy/onboarding/common-asset-functions.ps1
index 3d7bcf6055844..2f7c1b37c3c79 100644
--- a/eng/common/testproxy/onboarding/common-asset-functions.ps1
+++ b/eng/common/testproxy/onboarding/common-asset-functions.ps1
@@ -207,41 +207,6 @@ Function Invoke-ProxyCommand {
   )
   $updatedDirectory = $TargetDirectory.Replace("`\", "/")
 
-  # CommandString just a string indicating the proxy arguments. In the default case of running against the proxy tool, can just be used directly.
-  # However, in the case of docker, we need to append a bunch more arguments to the string.
-  if ($TestProxyExe -eq "docker" -or $TestProxyExe -eq "podman"){
-    $token = $env:GIT_TOKEN
-    $committer = $env:GIT_COMMIT_OWNER
-    $email = $env:GIT_COMMIT_EMAIL
-
-    if (-not $committer) {
-      $committer = & git config --global user.name
-    }
-
-    if (-not $email) {
-      $email = & git config --global user.email
-    }
-
-    if(-not $token -or -not $committer -or -not $email){
-      Write-Error ("When running this transition script in `"docker`" or `"podman`" mode, " `
-        + "the environment variables GIT_TOKEN, GIT_COMMIT_OWNER, and GIT_COMMIT_EMAIL must be set to reflect the appropriate user. ")
-        exit 1
-    }
-
-    $targetImage = if ($env:TRANSITION_SCRIPT_DOCKER_TAG) { $env:TRANSITION_SCRIPT_DOCKER_TAG } else { "azsdkengsys.azurecr.io/engsys/test-proxy:latest" }
-
-    $CommandString = @(
-      "run --rm --name transition.test.proxy",
-      "-v `"${updatedDirectory}:/srv/testproxy`"",
-      "-e `"GIT_TOKEN=${token}`"",
-      "-e `"GIT_COMMIT_OWNER=${committer}`"",
-      "-e `"GIT_COMMIT_EMAIL=${email}`"",
-      $targetImage,
-      "test-proxy",
-      $CommandString
-    ) -join " "
-  }
-
   Write-Host "$TestProxyExe $CommandString"
   [array] $output = & "$TestProxyExe" $CommandString.Split(" ") --storage-location="$updatedDirectory"
   # echo the command output
diff --git a/eng/common/testproxy/onboarding/generate-assets-json.ps1 b/eng/common/testproxy/onboarding/generate-assets-json.ps1
index bd825eebbe141..3576fd3c6bd6b 100644
--- a/eng/common/testproxy/onboarding/generate-assets-json.ps1
+++ b/eng/common/testproxy/onboarding/generate-assets-json.ps1
@@ -22,9 +22,9 @@ Generated assets.json file contents
 If flag InitialPush is set, recordings will be automatically pushed to the assets repo and the Tag property updated.
 
 .PARAMETER TestProxyExe
-The executable used during the "InitialPush" action. Defaults to the dotnet tool test-proxy, but also supports "docker" or "podman".
+The executable used during the "InitialPush" action. Defaults to the dotnet tool test-proxy, but also supports custom executables as well.
 
-If the user provides their own value that doesn't match options "test-proxy", "docker", or "podman", the script will use this input as the test-proxy exe
+If the user provides their own value that doesn't match options "test-proxy" the script will use this input as the test-proxy exe
 when invoking commands. EG "$TestProxyExe push -a sdk/keyvault/azure-keyvault-keys/assets.json."
 
 .PARAMETER InitialPush
diff --git a/eng/common/testproxy/publish-proxy-logs.yml b/eng/common/testproxy/publish-proxy-logs.yml
index 543186edd353a..4f4d3d7f548f2 100644
--- a/eng/common/testproxy/publish-proxy-logs.yml
+++ b/eng/common/testproxy/publish-proxy-logs.yml
@@ -3,16 +3,17 @@ parameters:
 
 steps:
   - pwsh: |
-      Copy-Item -Path "${{ parameters.rootFolder }}/test-proxy.log" -Destination "${{ parameters.rootFolder }}/proxy.log"
+      New-Item -ItemType Directory -Force "${{ parameters.rootFolder }}/proxy-logs"
+      Copy-Item -Path "${{ parameters.rootFolder }}/test-proxy.log" -Destination "${{ parameters.rootFolder }}/proxy-logs/proxy.log"
     displayName: Copy Log File
     condition: succeededOrFailed()
 
   - template: ../pipelines/templates/steps/publish-artifact.yml
     parameters:
       ArtifactName: "$(System.StageName)-$(System.JobName)-$(System.JobAttempt)-proxy-logs"
-      ArtifactPath: "${{ parameters.rootFolder }}/proxy.log"
+      ArtifactPath: "${{ parameters.rootFolder }}/proxy-logs"
 
   - pwsh: |
-      Remove-Item -Force ${{ parameters.rootFolder }}/proxy.log
+      Remove-Item -Force ${{ parameters.rootFolder }}/proxy-logs/proxy.log
     displayName: Cleanup Copied Log File
     condition: succeededOrFailed()
diff --git a/eng/common/testproxy/target_version.txt b/eng/common/testproxy/target_version.txt
index 1dbfeeb059f34..34e1ab134d427 100644
--- a/eng/common/testproxy/target_version.txt
+++ b/eng/common/testproxy/target_version.txt
@@ -1 +1 @@
-1.0.0-dev.20231030.1
+1.0.0-dev.20240314.1
diff --git a/eng/common/testproxy/test-proxy-tool-shutdown.yml b/eng/common/testproxy/test-proxy-tool-shutdown.yml
new file mode 100644
index 0000000000000..20e24e70a0aa4
--- /dev/null
+++ b/eng/common/testproxy/test-proxy-tool-shutdown.yml
@@ -0,0 +1,10 @@
+steps:
+  - pwsh: |
+      Stop-Process -Id $(PROXY_PID)
+    displayName: 'Shut down the testproxy - windows'
+    condition: and(succeeded(), eq(variables['Agent.OS'],'Windows_NT'))
+
+  - bash: |
+      kill -9 $(PROXY_PID)
+    displayName: "Shut down the testproxy - linux/mac"
+    condition: and(succeeded(), ne(variables['Agent.OS'],'Windows_NT'))
diff --git a/eng/common/testproxy/test-proxy-tool.yml b/eng/common/testproxy/test-proxy-tool.yml
index 7aea55d472d22..d9a1668419263 100644
--- a/eng/common/testproxy/test-proxy-tool.yml
+++ b/eng/common/testproxy/test-proxy-tool.yml
@@ -1,3 +1,4 @@
+# This template sets variable PROXY_PID to be used for shutdown later.
 parameters:
   rootFolder: '$(Build.SourcesDirectory)'
   runProxy: true
@@ -42,15 +43,20 @@ steps:
       condition: and(succeeded(), ${{ parameters.condition }})
 
     - pwsh: |
-        Start-Process $(Build.BinariesDirectory)/test-proxy/test-proxy.exe `
+        $Process = Start-Process $(Build.BinariesDirectory)/test-proxy/test-proxy.exe `
           -ArgumentList "start --storage-location ${{ parameters.rootFolder }} -U" `
           -NoNewWindow -PassThru -RedirectStandardOutput ${{ parameters.rootFolder }}/test-proxy.log
+
+        Write-Host "##vso[task.setvariable variable=PROXY_PID]$($Process.Id)"
       displayName: 'Run the testproxy - windows'
       condition: and(succeeded(), eq(variables['Agent.OS'],'Windows_NT'), ${{ parameters.condition }})
 
     # nohup does NOT continue beyond the current session if you use it within powershell
     - bash: |
         nohup $(Build.BinariesDirectory)/test-proxy/test-proxy &>$(Build.SourcesDirectory)/test-proxy.log &
+        
+        echo $! > $(Build.SourcesDirectory)/test-proxy.pid
+        echo "##vso[task.setvariable variable=PROXY_PID]$(cat $(Build.SourcesDirectory)/test-proxy.pid)"
       displayName: "Run the testproxy - linux/mac"
       condition: and(succeeded(), ne(variables['Agent.OS'],'Windows_NT'), ${{ parameters.condition }})
       workingDirectory: "${{ parameters.rootFolder }}"
diff --git a/eng/pipelines/aggregate-reports.yml b/eng/pipelines/aggregate-reports.yml
index d3715620b9527..e96822458d55c 100644
--- a/eng/pipelines/aggregate-reports.yml
+++ b/eng/pipelines/aggregate-reports.yml
@@ -117,10 +117,11 @@ stages:
           displayName: 'Copy BOM report to artifact staging'
           condition: succeededOrFailed()
 
-        - publish: $(Build.ArtifactStagingDirectory)
-          condition: succeededOrFailed()
-          displayName: 'Publish Report Artifacts'
-          artifact: reports
+        - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+          parameters:
+            DisplayName: 'Publish Report Artifacts'
+            ArtifactPath: '$(Build.ArtifactStagingDirectory)'
+            ArtifactName: 'reports'
 
         - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
 
@@ -166,8 +167,9 @@ stages:
         timeoutInMinutes: 120
 
         pool:
-          name: azsdk-pool-mms-win-2022-general
-          vmImage: windows-2022
+          name: $(WINDOWSPOOL)
+          vmImage: $(WINDOWSVMIMAGE)
+          os: windows
 
         steps:
           - template: /eng/common/pipelines/templates/steps/policheck.yml
diff --git a/eng/pipelines/bompreparation.yml b/eng/pipelines/bompreparation.yml
index a5a2f6dfecf8a..5d5ad2c27dc83 100644
--- a/eng/pipelines/bompreparation.yml
+++ b/eng/pipelines/bompreparation.yml
@@ -19,6 +19,8 @@ stages:
             Copy-Item eng/scripts/bompom.html $(Build.ArtifactStagingDirectory)/staging
           displayName: 'Analyze and generate patches'
 
-        - publish: $(Build.ArtifactStagingDirectory)
-          displayName: 'Publish Report Artifacts'
-          artifact: patchreport
\ No newline at end of file
+        - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+          parameters:
+            DisplayName: 'Publish Report Artifacts'
+            ArtifactPath: '$(Build.ArtifactStagingDirectory)'
+            ArtifactName: 'patchreport'
diff --git a/eng/pipelines/code-quality-reports.yml b/eng/pipelines/code-quality-reports.yml
index 545e77940d516..5c4c0cce333f6 100644
--- a/eng/pipelines/code-quality-reports.yml
+++ b/eng/pipelines/code-quality-reports.yml
@@ -7,7 +7,9 @@ jobs:
       - template: /eng/pipelines/templates/variables/globals.yml
 
     pool:
-      name: azsdk-pool-mms-ubuntu-2004-general
+      name: $(LINUXPOOL)
+      vmImage: $(LINUXVMIMAGE)
+      os: linux
 
     steps:
       # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
@@ -53,7 +55,7 @@ jobs:
         displayName: 'Generate FromSource POM and directories for sparse checkout'
         inputs:
           scriptPath: 'eng/scripts/generate_from_source_pom.py'
-          arguments: '--set-skip-linting-projects SkipLintingProjects --project-list $(ProjectList)'
+          arguments: '--set-skip-linting-projects SkipLintingProjects --artifacts-list $(ArtifactsList) --additional-modules-list $(AdditionalModulesList)'
           workingDirectory: '$(System.DefaultWorkingDirectory)'
 
       - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
@@ -105,16 +107,27 @@ jobs:
       - task: PowerShell@2
         inputs:
           filePath: $(System.DefaultWorkingDirectory)/eng/pipelines/scripts/Get-Linting-Reports.ps1
-          arguments: -OutputDirectory $(Build.ArtifactStagingDirectory)/linting-reports
+          arguments: -StagingDirectory $(Build.ArtifactStagingDirectory)
           pwsh: true
           workingDirectory: $(Pipeline.Workspace)
         displayName: 'Generate Linting Reports'
         condition: and(always(), ne(variables['LintingGoals'], ''))
 
-      - publish: $(Build.ArtifactStagingDirectory)/linting-reports.zip
-        displayName: 'Publish Linting Reports'
-        artifact: linting-reports-$(System.StageName)-$(System.JobName)-$(System.JobAttempt)
-        condition: and(always(), ne(variables['LintingGoals'], ''))
+      - pwsh: |
+          $artifactZip = "$(Build.ArtifactStagingDirectory)/linting-reports.zip"
+          $destinationDir = "$(Build.ArtifactStagingDirectory)/zipped-linting-reports"
+          if (Test-Path $artifactZip) {
+            New-Item -ItemType Directory -Path "$destinationDir"
+            Move-Item -Path $artifactZip -Destination "$destinationDir"
+          }
+        displayName: Move linting report zip to individual directory for artifact upload
+        condition: succeededOrFailed()
+
+      - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+        parameters:
+          DisplayName: 'Publish Linting Reports'
+          ArtifactPath: '$(Build.ArtifactStagingDirectory)/zipped-linting-reports'
+          ArtifactName: 'linting-reports'
 
       - pwsh: |
           if ((Get-ChildItem -Path $(Build.ArtifactStagingDirectory)/linting-reports -Directory | Measure-Object).Count -gt 0) {
diff --git a/eng/pipelines/docindex.yml b/eng/pipelines/docindex.yml
index 5b69beaaa6e17..7201749f76b23 100644
--- a/eng/pipelines/docindex.yml
+++ b/eng/pipelines/docindex.yml
@@ -27,6 +27,8 @@ jobs:
           Repositories:
             - Name: $(DocRepoOwner)/$(DocRepoName)
               WorkingDirectory: $(DocRepoLocation)
+            - Name: azure-sdk/$(DocRepoName)
+              WorkingDirectory: $(DailyDocRepoLocation)
       # Pull and build the docker image.
       - template: /eng/common/pipelines/templates/steps/docker-pull-image.yml
         parameters:
@@ -94,17 +96,6 @@ jobs:
       - template: /eng/common/pipelines/templates/steps/set-daily-docs-branch-name.yml
         parameters:
           DailyBranchVariableName: DailyDocsBranchName
-      - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-        parameters:
-          SkipCheckoutNone: true
-          Paths:
-            - package.json
-            - metadata/
-            - docs-ref-mapping/reference-unified.yml
-            - docs-ref-services/
-          Repositories:
-            - Name: azure-sdk/$(DocRepoName)
-              WorkingDirectory: $(DailyDocRepoLocation)
 
       # Docs daily updates is supposed to download packages from public feed repository, so we have to specify additional repositories in a POM or the profile.
       # Here is maven documentation: https://maven.apache.org/guides/mini/guide-multiple-repositories.html
diff --git a/eng/pipelines/latest-jdk.yml b/eng/pipelines/latest-jdk.yml
deleted file mode 100644
index 332318a352880..0000000000000
--- a/eng/pipelines/latest-jdk.yml
+++ /dev/null
@@ -1,90 +0,0 @@
-trigger: none
-
-pr:
-  branches:
-    include:
-      - main
-  paths:
-    include:
-      - eng/pipelines/latest-jdk.yml
-      - eng/scrips/Install-Latest-JDK.ps1
-
-variables:
-  - template: /eng/pipelines/templates/variables/globals.yml
-
-stages:
-  - stage: LatestJdk
-    displayName: Latest JDK
-
-    jobs:
-      - job: RunLatestJdk
-        displayName: Run Latest JDK
-        timeoutInMinutes: 180
-
-        pool:
-          name: 'azsdk-pool-mms-ubuntu-2004-general'
-          vmImage: 'MMSUbuntu20.04'
-
-        steps:
-          # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
-          # as we require the GitHub service connection to be loaded.
-          - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-              - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-                parameters:
-                  Paths:
-                    - '**/*.xml'
-
-          - task: UsePythonVersion@0
-            displayName: 'Use Python $(PythonVersion)'
-            inputs:
-              versionSpec: $(PythonVersion)
-
-          - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
-            parameters:
-              JobType: 'LatestJdk'
-              Artifacts:
-                - name: latest-jdk # fake artifact name
-                  groupId: run # fake group id
-
-          - task: PythonScript@0
-            displayName: 'Generate FromSource POM and directories for sparse checkout'
-            inputs:
-              scriptPath: 'eng/scripts/generate_from_source_pom.py'
-              arguments: '--match-any-version --set-skip-linting-projects SkipLintingProjects --project-list com.azure:azure-core'
-              workingDirectory: '$(System.DefaultWorkingDirectory)'
-
-          - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-            parameters:
-              SkipCheckoutNone: true
-              Paths: $(SparseCheckoutDirectories)
-
-          - template: /eng/pipelines/templates/steps/install-reporting-tools.yml
-
-          - pwsh: |
-              Write-Host "##vso[task.setvariable variable=IsLatestNonLtsJdk;]true"
-
-          - template: /eng/pipelines/templates/steps/install-latest-jdk.yml
-            parameters:
-              LatestJdkFeatureVersion: $(LatestNonLtsJdkFeatureVersion)
-
-          - task: Maven@3
-            displayName: 'Run All Libraries Latest JDK'
-            inputs:
-              mavenPomFile: ClientFromSourcePom.xml
-              options: '$(DefaultOptions) -T 1C -Dgpg.skip'
-              mavenOptions: '$(MemoryOptions) $(LoggingOptions)'
-              javaHomeOption: 'JDKVersion'
-              jdkVersionOption: $(LatestNonLtsJavaVersion)
-              jdkArchitectureOption: 'x64'
-              publishJUnitResults: false
-              goals: 'clean install'
-            continueOnError: true
-
-          - task: PublishTestResults@2
-            condition: always()
-            inputs:
-              testResultsFormat: 'JUnit'
-              testResultsFiles: '**/TEST-*.xml'
-              searchFolder: '$(System.DefaultWorkingDirectory)/sdk'
-              mergeTestResults: true
-              testRunTitle: '$(Agent.JobName)_attempt_$(System.JobAttempt)'
diff --git a/eng/pipelines/partner-release.yml b/eng/pipelines/partner-release.yml
index f696d6504a609..8e6e5e544a6fd 100644
--- a/eng/pipelines/partner-release.yml
+++ b/eng/pipelines/partner-release.yml
@@ -11,71 +11,80 @@ parameters:
   type: boolean
   default: true
 
-resources:
-  repositories:
-    - repository: azure-sdk-build-tools
-      type: git
-      name: internal/azure-sdk-build-tools
-      ref: refs/tags/azure-sdk-build-tools_20230829.1
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - stage:
+        displayName: 'Partner Release'
+        variables:
+          - name: BuildToolScripts
+            value: $(Pipeline.Workspace)/azure-sdk-build-tools/scripts
+          - name: Artifacts
+            value: $(Pipeline.Workspace)/artifacts
+          - name: EsrpArtifacts
+            value: $(Build.SourcesDirectory)/packages-esrp
+          - template: /eng/pipelines/templates/variables/image.yml
+        jobs:
+        - job: Signing
+          pool:
+            name: $(WINDOWSPOOL)
+            image: $(WINDOWSVMIMAGE)
+            os: windows
+          steps:
+          - checkout: azure-sdk-build-tools
+            path: azure-sdk-build-tools
 
-variables:
-  BuildToolScripts: $(Pipeline.Workspace)/azure-sdk-build-tools/scripts
-  Artifacts: $(Pipeline.Workspace)/artifacts
-  EsrpArtifacts: $(Pipeline.Workspace)/packages-esrp
+          - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
 
-jobs:
-- job: Signing
-  pool:
-    name: azsdk-pool-mms-win-2022-general
-    vmImage: windows-2022
-  steps:
-  - checkout: azure-sdk-build-tools
-    path: azure-sdk-build-tools
+          - task: PowerShell@2
+            displayName: 'Download packages from blob storage'
+            inputs:
+              targetType: filePath
+              filePath: '$(BuildToolScripts)/copy-from-azuresdkpartnerdrops.ps1'
+              arguments: '$(Artifacts) ${{ parameters.BlobPath }} $(azuresdkpartnerdrops-access-key)'
 
-  - task: PowerShell@2
-    displayName: 'Download packages from blob storage'
-    inputs:
-      targetType: filePath
-      filePath: '$(BuildToolScripts)/copy-from-azuresdkpartnerdrops.ps1'
-      arguments: '$(Artifacts) ${{ parameters.BlobPath }} $(azuresdkpartnerdrops-access-key)'
+          - template: tools/java-esrp-signing/java-esrp-signing.yml@azure-sdk-build-tools
+            parameters:
+              ArtifactDirectory: $(Artifacts)
 
-  - template: tools/java-esrp-signing/java-esrp-signing.yml@azure-sdk-build-tools
-    parameters:
-      ArtifactDirectory: $(Artifacts)
+          - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+            parameters:
+              ArtifactName: packages-signed
+              ArtifactPath: $(Artifacts)
 
-  - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-    parameters:
-      ArtifactName: packages-signed
-      ArtifactPath: $(Artifacts)
-- job: Release
-  dependsOn: Signing
-  pool:
-    name: azsdk-pool-mms-win-2022-general
-    vmImage: windows-2022
-  steps:
-  - checkout: self
-    path: azure-sdk-for-java
+        - job: Release
+          dependsOn: Signing
+          pool:
+            name: $(WINDOWSPOOL)
+            image: $(WINDOWSVMIMAGE)
+            os: windows
+          steps:
+          - checkout: self
+            path: azure-sdk-for-java
 
-  - checkout: azure-sdk-build-tools
-    path: azure-sdk-build-tools
+          - checkout: azure-sdk-build-tools
+            path: azure-sdk-build-tools
 
-  - download: current
-    displayName: Download Signed Artifacts
-    artifact: packages-signed
+          - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
 
-  - template: tools/gpg/gpg.yml@azure-sdk-build-tools
+          - download: current
+            displayName: Download Signed Artifacts
+            artifact: packages-signed
 
-  - template: /eng/pipelines/templates/steps/java-publishing.yml
-    parameters:
-      ArtifactDirectory: $(Pipeline.Workspace)/packages-signed
-      OutputDirectory: $(EsrpArtifacts)
-      Target: EsrpRelease
-      BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
-      JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-      ShouldPublish: ${{ parameters.ShouldPublish }}
-      StageOnly: false
+          - template: tools/gpg/gpg.yml@azure-sdk-build-tools
 
-  - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-    parameters:
-      ArtifactName: packages-esrp-$(System.JobAttempt)
-      ArtifactPath: $(EsrpArtifacts)
+          - template: /eng/pipelines/templates/steps/java-publishing.yml
+            parameters:
+              ArtifactDirectory: $(Pipeline.Workspace)/packages-signed
+              OutputDirectory: $(EsrpArtifacts)
+              Target: EsrpRelease
+              BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
+              JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
+              ShouldPublish: ${{ parameters.ShouldPublish }}
+              StageOnly: false
+
+          - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+            parameters:
+              ArtifactName: packages-esrp-$(System.JobAttempt)
+              ArtifactPath: $(EsrpArtifacts)
diff --git a/eng/pipelines/patch_release_client.txt b/eng/pipelines/patch_release_client.txt
index 71b587f0a1683..6b7e827bfab8c 100644
--- a/eng/pipelines/patch_release_client.txt
+++ b/eng/pipelines/patch_release_client.txt
@@ -19,7 +19,7 @@
 # ensuring that all tests (recorded, live, etc.) are in a good state to ensure releases maintain quality and avoid
 # performance regressions or bugs. They are also responsible for smoke testing their library after every release.
 
-com.azure:azure-ai-content-safety # Tests owner: zhaiyutong
+com.azure:azure-ai-contentsafety # Tests owner: zhaiyutong
 com.azure:azure-ai-formrecognizer # Tests owner: samvaity
 com.azure:azure-ai-metricsadvisor # Tests owner: samvaity
 com.azure:azure-ai-textanalytics # Tests owner: mssfang
@@ -29,6 +29,7 @@ com.azure:azure-communication-common # Tests owner: AikoBB, maximrytych-ms, mjaf
 com.azure:azure-communication-email # Tests owner: yogeshmo, kagbakpem
 com.azure:azure-communication-identity # Tests owner: AikoBB, maximrytych-ms, mjafferi-msft
 com.azure:azure-communication-jobrouter # Tests owner: cparisineti
+com.azure:azure-communication-messages # Tests owner: arifibrahim4, juancamilor
 com.azure:azure-communication-phonenumbers # Tests owner: whisper6284, miguhern, danielortega-msft
 com.azure:azure-communication-sms # Tests owner: besh2014, DimaKolomiiets
 com.azure:azure-communication-rooms # Tests owner: minnieliu, Mrayyan
diff --git a/eng/pipelines/scripts/Get-Heap-Dump-Hprofs.ps1 b/eng/pipelines/scripts/Get-Heap-Dump-Hprofs.ps1
new file mode 100644
index 0000000000000..638cfad369dcc
--- /dev/null
+++ b/eng/pipelines/scripts/Get-Heap-Dump-Hprofs.ps1
@@ -0,0 +1,32 @@
+<#
+.SYNOPSIS
+Captures any .hprof files in the build directory and moves them to a staging directory for artifact publishing.
+
+.DESCRIPTION
+This script is used to capture any .hprof files in the build directory and move them to a staging directory for
+artifact publishing. It also sets a pipeline variable to indicate whether any .hprof files were found.
+
+.PARAMETER StagingDirectory
+The directory where the .hprof files will be moved to.
+
+.PARAMETER OomArtifactName
+The name of the artifact to be created.
+#>
+
+param(
+    [Parameter(Mandatory = $true)]
+    [string]$StagingDirectory,
+
+    [Parameter(Mandatory = $true)]
+    [string]$OomArtifactName
+)
+
+$hrpofs = Get-ChildItem -Path . -Recurse -Filter *.hprof -File
+
+if ($hrpofs.Count -gt 0) {
+    if (-not (Test-Path "$StagingDirectory/troubleshooting")) {
+        New-Item -ItemType Directory -Path "$StagingDirectory/troubleshooting" | Out-Null
+    }
+    Compress-Archive -Path $hrpofs -DestinationPath "$StagingDirectory/troubleshooting/$OomArtifactName.zip"
+    Write-Host "##vso[task.setvariable variable=HAS_TROUBLESHOOTING]true"
+}
diff --git a/eng/pipelines/scripts/Get-Linting-Reports.ps1 b/eng/pipelines/scripts/Get-Linting-Reports.ps1
index e642e9c68b882..1e68d67c9096a 100644
--- a/eng/pipelines/scripts/Get-Linting-Reports.ps1
+++ b/eng/pipelines/scripts/Get-Linting-Reports.ps1
@@ -19,8 +19,8 @@ For Checkstyle the XML report will have the file name, line number, column numbe
 error reported. For Spotbugs the XML will have the file name, line number, linting error message, and linting error
 reported, Spotbugs doesn't report the column number.
 
-.PARAMETER OutputDirectory
-Where the linting report retrieval and processing will be output.
+.PARAMETER StagingDirectory
+The directory where the linting reports will be output.
 
 .PARAMETER SdkDirectory
 An optional SDK directory, such as core, that scopes the linting report searching and processing to the specific SDK
@@ -29,7 +29,7 @@ directory. If this isn't passed all SDK directories will be searched and process
 
 param(
   [Parameter(Mandatory = $true)]
-  [string]$OutputDirectory,
+  [string]$StagingDirectory,
 
   [Parameter(Mandatory = $false)]
   [string]$SdkDirectory
@@ -106,7 +106,7 @@ function WriteSpotbugsProcessedReport($SpotbugsXmlReport, $ReportOutputFolder) {
 Set-Location -ErrorAction Stop -LiteralPath (Join-Path $PSScriptRoot "../../../")
 
 # Always create the output directory
-$OutputDirectory = $ExecutionContext.SessionState.Path.GetUnresolvedProviderPathFromPSPath($OutputDirectory)
+$OutputDirectory = $ExecutionContext.SessionState.Path.GetUnresolvedProviderPathFromPSPath("$StagingDirectory/linting-report")
 New-Item -Path $OutputDirectory -ItemType Directory | Out-Null
 
 $path = "sdk/*/"
@@ -144,4 +144,9 @@ foreach ($targetFolder in (Get-ChildItem -Path $path -Filter "target" -Directory
   }
 }
 
-[System.IO.Compression.ZipFile]::CreateFromDirectory($OutputDirectory, $OutputDirectory + ".zip")
+if (-not (Test-Path "$StagingDirectory/troubleshooting")) {
+  New-Item -ItemType Directory -Path "$StagingDirectory/troubleshooting" | Out-Null
+}
+
+Compress-Archive -Path $OutputDirectory -DestinationPath "$StagingDirectory/troubleshooting/linting-report.zip"
+Write-Host "##vso[task.setvariable variable=HAS_TROUBLESHOOTING]true"
diff --git a/eng/pipelines/scripts/Get-Test-Logs.ps1 b/eng/pipelines/scripts/Get-Test-Logs.ps1
new file mode 100644
index 0000000000000..53d2effe4aa45
--- /dev/null
+++ b/eng/pipelines/scripts/Get-Test-Logs.ps1
@@ -0,0 +1,32 @@
+<#
+.SYNOPSIS
+Captures any test.log files in the build directory and moves them to a staging directory for artifact publishing.
+
+.DESCRIPTION
+This script is used to capture any test.log files in the build directory and move them to a staging directory for
+artifact publishing. It also sets a pipeline variable to indicate whether any test.log files were found.
+
+.PARAMETER StagingDirectory
+The directory where the test.log files will be moved to.
+
+.PARAMETER TestLogsArtifactName
+The name of the artifact to be created.
+#>
+
+param(
+    [Parameter(Mandatory = $true)]
+    [string]$StagingDirectory,
+
+    [Parameter(Mandatory = $true)]
+    [string]$TestLogsArtifactName
+)
+
+$testLogs = Get-ChildItem -Path . -Recurse -Filter test.log -File -Depth 4
+
+if ($testLogs.Count -gt 0) {
+    if (-not (Test-Path "$StagingDirectory/troubleshooting")) {
+        New-Item -ItemType Directory -Path "$StagingDirectory/troubleshooting" | Out-Null
+    }
+    Write-Host "##vso[task.setvariable variable=HAS_TROUBLESHOOTING]true"
+    Compress-Archive -Path $testLogs -DestinationPath "$StagingDirectory/troubleshooting/$TestLogsArtifactName.zip"
+}
diff --git a/eng/pipelines/scripts/Invoke-Sparse-Checkout.ps1 b/eng/pipelines/scripts/Invoke-Sparse-Checkout.ps1
new file mode 100644
index 0000000000000..db43a1fe21555
--- /dev/null
+++ b/eng/pipelines/scripts/Invoke-Sparse-Checkout.ps1
@@ -0,0 +1,104 @@
+<#
+.SYNOPSIS
+Invokes sparse checkout on the specified repositories.
+
+.DESCRIPTION
+Invokes sparse checkout on the specified repositories.
+
+This script is special to Java as it uses layered sparse checkout to reduce the amount of code to checkout.
+The first run of sparse checkout is inlined into YAML as there is a chicken and egg problem where the script
+to perform sparse checkout won't be available until after the checkout step has completed.
+
+This script is used to reduce the size of YAML files as this is only called when the initial checkout has
+already been completed.
+
+.PARAMETER PathsJson
+JSON representation of the paths to checkout.
+
+.PARAMETER RepositoriesJson
+JSON representation of the repositories to checkout from.
+#>
+
+param(
+  [Parameter(Mandatory = $true)]
+  [string]$PathsJson,
+
+  [Parameter(Mandatory = $true)]
+  [string]$RepositoriesJson
+)
+
+# Setting $PSNativeCommandArgumentPassing to 'Legacy' to use PowerShell
+# 7.2 behavior for command argument passing. Newer behaviors will result
+# in errors from git.exe.
+$PSNativeCommandArgumentPassing = 'Legacy'
+
+function SparseCheckout([Array]$paths, [Hashtable]$repository)
+{
+    $dir = $repository.WorkingDirectory
+    if (!$dir) {
+        $dir = "./$($repository.Name)"
+    }
+    New-Item $dir -ItemType Directory -Force | Out-Null
+    Push-Location $dir
+
+    if (Test-Path .git/info/sparse-checkout) {
+        $hasInitialized = $true
+        Write-Host "Repository $($repository.Name) has already been initialized. Skipping this step."
+    } else {
+        Write-Host "Repository $($repository.Name) is being initialized."
+
+        if ($repository.Commitish -match '^refs/pull/\d+/merge$') {
+            Write-Host "git clone --no-checkout --filter=tree:0 -c remote.origin.fetch='+$($repository.Commitish):refs/remotes/origin/$($repository.Commitish)' https://github.com/$($repository.Name) ."
+            git clone --no-checkout --filter=tree:0 -c remote.origin.fetch=''+$($repository.Commitish):refs/remotes/origin/$($repository.Commitish)'' https://github.com/$($repository.Name) .
+        } else {
+            Write-Host "git clone --no-checkout --filter=tree:0 https://github.com/$($repository.Name) ."
+            git clone --no-checkout --filter=tree:0 https://github.com/$($repository.Name) .
+        }
+
+        # Turn off git GC for sparse checkout. Note: The devops checkout task does this by default
+        Write-Host "git config gc.auto 0"
+        git config gc.auto 0
+
+        Write-Host "git sparse-checkout init"
+        git sparse-checkout init
+
+        # Set non-cone mode otherwise path filters will not work in git >= 2.37.0
+        # See https://github.blog/2022-06-27-highlights-from-git-2-37/#tidbits
+        Write-Host "git sparse-checkout set --no-cone '/*' '!/*/' '/eng'"
+        git sparse-checkout set --no-cone '/*' '!/*/' '/eng'
+    }
+
+    # Prevent wildcard expansion in Invoke-Expression (e.g. for checkout path '/*')
+    $quotedPaths = $paths | ForEach-Object { "'$_'" }
+    $gitsparsecmd = "git sparse-checkout add $quotedPaths"
+    Write-Host $gitsparsecmd
+    Invoke-Expression -Command $gitsparsecmd
+
+    Write-Host "Set sparse checkout paths to:"
+    Get-Content .git/info/sparse-checkout
+
+    # sparse-checkout commands after initial checkout will auto-checkout again
+    if (!$hasInitialized) {
+        # Remove refs/heads/ prefix from branch names
+        $commitish = $repository.Commitish -replace '^refs/heads/', ''
+
+        # use -- to prevent git from interpreting the commitish as a path
+        Write-Host "git -c advice.detachedHead=false checkout $commitish --"
+
+        # This will use the default branch if repo.Commitish is empty
+        git -c advice.detachedHead=false checkout $commitish --
+    } else {
+        Write-Host "Skipping checkout as repo has already been initialized"
+    }
+
+    Pop-Location
+}
+
+# Paths may be sourced as a yaml object literal OR a dynamically generated variable json string.
+# If the latter, convertToJson will wrap the 'string' in quotes, so remove them.
+$paths = $PathsJson.Trim('"') | ConvertFrom-Json
+# Replace windows backslash paths, as Azure Pipelines default directories are sometimes formatted like 'D:\a\1\s'
+$repositories = $RepositoriesJson -replace '\\', '/' | ConvertFrom-Json -AsHashtable
+foreach ($repo in $repositories) {
+    SparseCheckout $paths $repo
+}
diff --git a/eng/pipelines/scripts/Set-Dev-Versioning.ps1 b/eng/pipelines/scripts/Set-Dev-Versioning.ps1
new file mode 100644
index 0000000000000..fde1b25de967c
--- /dev/null
+++ b/eng/pipelines/scripts/Set-Dev-Versioning.ps1
@@ -0,0 +1,37 @@
+<#
+.SYNOPSIS
+Sets the dev version for the SDKs and updates the repository with the new version.
+
+.DESCRIPTION
+Sets the dev version for the SDKs and updates the repository with the new version.
+
+The dev version is based on the date and the build number. The dev version is then published to the Azure Artifacts feed.
+
+.PARAMETER BuildNumber
+The build number to use for the dev version.
+
+.PARAMETER SdkType
+The type of SDK to set the dev version for.
+#>
+
+param(
+  [Parameter(Mandatory = $true)]
+  [string]$BuildNumber,
+
+  [Parameter(Mandatory = $true)]
+  [string]$SdkType
+)
+
+$artifacts = $env:ARTIFACTSJSON | ConvertFrom-Json
+python3 --version
+
+# Append dev package version suffix for each artifact
+foreach ($artifact in $artifacts) {
+    python3 "$PSScriptRoot/../../versioning/set_versions.py" --build-type $SDKType --build-qualifier "alpha.$BuildNumber" --artifact-id $artifact.name --group-id $artifact.groupId
+}
+
+# Set zero-dev-version for packages
+python3 "$PSScriptRoot/../../versioning/set_versions.py" --set-dev-zero-version --build-type $SDKType --build-qualifier "alpha.$BuildNumber"
+
+# Apply version settings to repository
+python3 "$PSScriptRoot/../../versioning/update_versions.py" --update-type library --build-type $SDKType --sr
diff --git a/eng/pipelines/scripts/generate-project-list.ps1 b/eng/pipelines/scripts/generate-project-list.ps1
new file mode 100644
index 0000000000000..72dded99fd8cc
--- /dev/null
+++ b/eng/pipelines/scripts/generate-project-list.ps1
@@ -0,0 +1,42 @@
+$projectList = @()
+$artifactsList = @()
+$additionalModulesList = @()
+
+if ($env:ARTIFACTSJSON -and $env:ARTIFACTSJSON -notlike '*ArtifactsJson*') {
+  $artifacts = $env:ARTIFACTSJSON | ConvertFrom-Json
+  foreach ($artifact in $artifacts) {
+    $projectList += "$($artifact.groupId):$($artifact.name)"
+    $artifactsList += "$($artifact.groupId):$($artifact.name)"
+  }
+}
+
+# Check if empty or still a literal devops variable reference $()
+if ($env:ADDITIONALMODULESJSON -and $env:ADDITIONALMODULESJSON -notlike '*AdditionalModulesJson*') {
+  $additionalModules = $env:ADDITIONALMODULESJSON | ConvertFrom-Json
+  foreach ($artifact in $additionalModules) {
+    $projectList += "$($artifact.groupId):$($artifact.name)"
+    $additionalModulesList += "$($artifact.groupId):$($artifact.name)"
+  }
+}
+
+$projects = $projectList -join ','
+if (!$projects) {
+    throw "parameters.Artifacts cannot be empty"
+}
+
+$artifactsString = $artifactsList -join ','
+Write-Host "ArtifactsList = $artifactsString"
+Write-Host "##vso[task.setvariable variable=ArtifactsList;]$artifactsString"
+
+$additionalModulesString = $additionalModulesList -join ','
+Write-Host "AdditionalModulesList = $additionalModulesString"
+Write-Host "##vso[task.setvariable variable=AdditionalModulesList;]$additionalModulesString"
+
+Write-Host "ProjectList = $projects"
+Write-Host "##vso[task.setvariable variable=ProjectList;]$projects"
+
+$sha256 = new-object -TypeName System.Security.Cryptography.SHA256Managed
+$utf8 = new-object -TypeName System.Text.UTF8Encoding
+
+$projectListSha256 = [Convert]::ToBase64String($sha256.ComputeHash($utf8.GetBytes($projects)))
+Write-Host "##vso[task.setvariable variable=ProjectListSha256;]$projectListSha256"
diff --git a/eng/pipelines/templates/jobs/build-validate-pom.yml b/eng/pipelines/templates/jobs/build-validate-pom.yml
index 07d80b3463a0d..01f5021e04b9d 100644
--- a/eng/pipelines/templates/jobs/build-validate-pom.yml
+++ b/eng/pipelines/templates/jobs/build-validate-pom.yml
@@ -13,12 +13,18 @@ jobs:
   - job: 'BuildAndValidate'
     displayName: 'Build and Validate'
 
-    variables:
-      - template: /eng/pipelines/templates/variables/globals.yml
+    templateContext:
+      outputs:
+        - output: pipelineArtifact
+          displayName: 'Capture repo state for analysis'
+          targetPath: '$(System.DefaultWorkingDirectory)'
+          artifactName: 'repository'
+          sbomEnabled: false
 
     pool:
-      name: azsdk-pool-mms-ubuntu-2004-general
-      vmImage: MMSUbuntu20.04
+      name: $(LINUXPOOL)
+      image: $(LINUXVMIMAGE)
+      os: linux
 
     strategy:
       matrix:
@@ -54,7 +60,6 @@ jobs:
       - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
         parameters:
           JobType: 'Build and Validate'
-          Artifacts: ${{ parameters.Artifacts }}
 
       # We `install` the code quality reports tooling into our local m2 cache separately from building the Maven project
       # reports. This means it is available as part of that, but also so that this is not documented in the project report.
@@ -85,11 +90,7 @@ jobs:
             -Artifacts ('${{ replace(convertToJson(parameters.Artifacts), '''', '`''') }}' | ConvertFrom-Json)
             -InformationAction Continue
 
-      - publish: $(Build.ArtifactStagingDirectory)
-        displayName: 'Publish Artifacts $(ArtifactName)'
-        artifact: '$(ArtifactName)'
-
-      - publish: $(System.DefaultWorkingDirectory)
-        condition: and(eq(variables['CaptureRepositoryOnFailure'], 'true'), failed())
-        displayName: 'Capture repo state for analysis'
-        artifact: repository
+      - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+        parameters:
+          ArtifactPath: '$(Build.ArtifactStagingDirectory)'
+          ArtifactName: '$(ArtifactName)'
diff --git a/eng/pipelines/templates/jobs/ci.tests.yml b/eng/pipelines/templates/jobs/ci.tests.yml
index 8096ae2e2382a..1ae88d231a82f 100644
--- a/eng/pipelines/templates/jobs/ci.tests.yml
+++ b/eng/pipelines/templates/jobs/ci.tests.yml
@@ -8,23 +8,17 @@ parameters:
   - name: TestPipeline
     type: boolean
     default: false
-  - name: Artifacts
-    type: object
-    default: []
-  - name: AdditionalModules
-    type: object
-    default: []
   - name: PreTestSteps
     type: object
     default: []
   - name: Matrix
-    type: object
+    type: string
   - name: CloudConfig
     type: object
     default: {}
   - name: DependsOn
-    type: string
-    default: ''
+    type: object
+    default: []
   - name: UsePlatformContainer
     type: boolean
     default: false
@@ -46,18 +40,31 @@ parameters:
   - name: AdditionalLintingOptions
     type: string
     default: ''
+  - name: OSName
+    type: string
+  - name: TimeoutInMinutes
+    type: number
+    default: 60
 
 jobs:
-  - job: 'Test'
-    dependsOn: ${{ parameters.DependsOn }}
-    condition: and(succeeded(), ne(variables['Skip.Test'], 'true'))
+  - job:
+    displayName: 'Test'
+    dependsOn:
+      - ${{ parameters.DependsOn }}
+    condition: and(succeeded(), ne(variables['Skip.Test'], 'true'), ne(${{ parameters.Matrix }}, '{}'))
 
     strategy:
       matrix: $[ ${{ parameters.Matrix }} ]
 
     pool:
+      # 1es pipeline templates converts `image` to demands: ImageOverride under the hood
+      # which is incompatible with image selection in the default non-1es hosted pools
+      ${{ if eq(parameters.OSName, 'macOS') }}:
+        vmImage: $(OSVmImage)
+      ${{ else }}:
+        image: $(OSVmImage)
       name: $(Pool)
-      vmImage: $(OSVmImage)
+      os: ${{ parameters.OSName }}
 
     ${{ if eq(parameters.UsePlatformContainer, 'true') }}:
       # Add a default so the job doesn't fail when the matrix is empty
@@ -66,14 +73,32 @@ jobs:
     variables:
       - template: /eng/pipelines/templates/variables/globals.yml
 
+    timeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
+
+    templateContext:
+      outputParentDirectory: '$(System.DefaultWorkingDirectory)'
+      outputs:
+        # Troubleshooting artifacts are creating in the staging directory under the folder 'troubleshooting'.
+        # This will contain things such as heap dumps hprofs if testing hit OutOfMemory errors, log files captured
+        # during testing if tests failed, and linting reports.
+        - output: pipelineArtifact
+          artifactName: troubleshooting-$(System.StageName)-$(System.JobName)
+          targetPath: $(System.DefaultWorkingDirectory)/troubleshooting
+          displayName: 'Capture troubleshooting artifacts'
+          condition: and(not(succeeded()), eq(variables['HAS_TROUBLESHOOTING'], 'true'))
+          sbomEnabled: false
+
+        - output: pipelineArtifact
+          artifactName: 'repository-$(System.JobName)'
+          targetPath: '$(System.DefaultWorkingDirectory)'
+          displayName: 'Capture repo state for analysis'
+          condition: or(eq(variables['CaptureRepositoryOnFailure'], 'true'), and(failed(), eq(variables['Build.Reason'],'Schedule')))
+          sbomEnabled: false
+
     steps:
       - template: /eng/pipelines/templates/steps/initialize-test-environment.yml
         parameters:
-          Artifacts: ${{ parameters.Artifacts }}
-          AdditionalModules: ${{ parameters.AdditionalModules }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
           CheckoutRecordings: true
-          SDKType: ${{ parameters.SDKType }}
 
       - template: /eng/common/testproxy/test-proxy-tool.yml
         parameters:
@@ -117,14 +142,14 @@ jobs:
 
       - template: /eng/pipelines/templates/steps/build-and-test.yml
         parameters:
-          SDKType: ${{ parameters.SDKType }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
           ParallelTestPlayback: 'true'
           TestGoals: ${{ parameters.TestGoals }}
           TestOptions: ${{ parameters.TestOptions }}
           TestParallelization: ${{ parameters.TestParallelization }}
           BuildParallelization: ${{ parameters.BuildParallelization }}
 
-      - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
+      - template: /eng/pipelines/templates/steps/clean-maven-local-cache.yml
 
-      - template: /eng/common/testproxy/publish-proxy-logs.yml
+      # Shut down proxy to prevent file locks on the log file for auto-injected credscan
+      # steps on windows
+      - template: /eng/common/testproxy/test-proxy-tool-shutdown.yml
diff --git a/eng/pipelines/templates/jobs/ci.versions.tests.yml b/eng/pipelines/templates/jobs/ci.versions.tests.yml
index 5fc0caeb8ab84..c822d5e69e6ad 100644
--- a/eng/pipelines/templates/jobs/ci.versions.tests.yml
+++ b/eng/pipelines/templates/jobs/ci.versions.tests.yml
@@ -2,18 +2,12 @@ parameters:
   - name: SDKType
     type: string
     default: client
-  - name: ServiceDirectory
-    type: string
-    default: 'not-specified' # Set a default that breaks in obvious ways.
-  - name: TestPipeline
-    type: boolean
-    default: false
   - name: Artifacts
     type: object
     default: []
-  - name: AdditionalModules
-    type: object
-    default: []
+  - name: TestPipeline
+    type: boolean
+    default: false
   - name: PreTestSteps
     type: object
     default: []
@@ -43,12 +37,40 @@ jobs:
   - job:
     condition: ne(variables['Skip.Test'], 'true')
 
-    pool:
-      name: 'azsdk-pool-mms-ubuntu-2004-general'
-      vmImage: 'MMSUbuntu20.04'
-
     variables:
       - template: /eng/pipelines/templates/variables/globals.yml
+      - template: /eng/pipelines/templates/variables/image.yml
+      # Convert artifact parameter objects to json and set them as variables to be used in
+      # pipeline jobs/stages. By setting these as a variable once we save thousands of lines
+      # of yaml in extreme cases. This helps us stay under the devops max yaml size limits.
+      - name: ArtifactsJson
+        value: '${{ convertToJson(parameters.Artifacts) }}'
+
+    pool:
+      name: $(LINUXPOOL)
+      image: $(LINUXVMIMAGE)
+      os: linux
+
+    templateContext:
+      outputParentDirectory: '$(System.DefaultWorkingDirectory)'
+      outputs:
+        - output: pipelineArtifact
+          artifactName: 'repository-$(System.JobName)'
+          targetPath: '$(System.DefaultWorkingDirectory)'
+          displayName: 'Capture repo state for analysis'
+          condition: or(eq(variables['CaptureRepositoryOnFailure'], 'true'), and(failed(), eq(variables['Build.Reason'],'Schedule')))
+          sbomEnabled: false
+
+        - ${{ each versionOverride in parameters.VersionOverrides }}:
+          # Troubleshooting artifacts are creating in the staging directory under the folder 'troubleshooting'.
+          # This will contain things such as heap dumps hprofs if testing hit OutOfMemory errors, log files captured
+          # during testing if tests failed, and linting reports.
+          - output: pipelineArtifact
+            artifactName: 'troubleshooting-vo-${{ versionOverride }}-troubleshooting-$(System.StageName)-$(System.JobName)'
+            targetPath: $(System.DefaultWorkingDirectory)/troubleshooting
+            displayName: 'Capture troubleshooting artifacts for version override ${{ versionOverride }}'
+            condition: and(not(succeeded()), eq(variables['HAS_TROUBLESHOOTING'], 'true'))
+            sbomEnabled: false
 
     steps:
       - pwsh: |
@@ -58,12 +80,8 @@ jobs:
 
       - template: /eng/pipelines/templates/steps/initialize-test-environment.yml
         parameters:
-          Artifacts: ${{ parameters.Artifacts }}
-          AdditionalModules: ${{ parameters.AdditionalModules }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
           CheckoutRecordings: true
-          SDKType: ${{ parameters.SDKType }}
-          OSVmImage: 'MMSUbuntu20.04'
+          OS: $(LINUXVMIMAGE)
 
       - template: /eng/common/testproxy/test-proxy-tool.yml
         parameters:
@@ -90,8 +108,6 @@ jobs:
       - ${{ each versionOverride in parameters.VersionOverrides }}:
         - template: /eng/pipelines/templates/steps/build-and-test.yml
           parameters:
-            SDKType: ${{ parameters.SDKType }}
-            ServiceDirectory: ${{ parameters.ServiceDirectory }}
             ParallelTestPlayback: 'true'
             TestGoals: ${{ parameters.TestGoals }}
             TestOptions: ${{ parameters.TestOptions }}
@@ -104,6 +120,4 @@ jobs:
             Write-Host "##vso[task.setvariable variable=FirstTestOverrideRan]true"
           displayName: "Reset version override changes"
 
-      - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
-
-      - template: /eng/common/testproxy/publish-proxy-logs.yml
+      - template: /eng/pipelines/templates/steps/clean-maven-local-cache.yml
diff --git a/eng/pipelines/templates/jobs/ci.yml b/eng/pipelines/templates/jobs/ci.yml
index 06d86d4cd4d63..f06e26b9e2b04 100644
--- a/eng/pipelines/templates/jobs/ci.yml
+++ b/eng/pipelines/templates/jobs/ci.yml
@@ -14,9 +14,6 @@ parameters:
   - name: ReleaseArtifacts
     type: object
     default: []
-  - name: AdditionalModules
-    type: object
-    default: []
   - name: PreTestSteps
     type: object
     default: []
@@ -58,6 +55,9 @@ parameters:
   - name: IgnoreVerifyTypeSpecCodeGenerationError
     type: boolean
     default: false
+  - name: TimeoutInMinutes
+    type: number
+    default: 60
 
 jobs:
   - job: 'Build'
@@ -69,7 +69,32 @@ jobs:
       Codeql.SkipTaskAutoInjection: false
 
     pool:
-      name: azsdk-pool-mms-ubuntu-2004-general
+      name: $(LINUXPOOL)
+      image: $(LINUXVMIMAGE)
+      os: linux
+
+    templateContext:
+      outputParentDirectory: '$(System.DefaultWorkingDirectory)'
+      outputs:
+        - output: pipelineArtifact
+          artifactName: '$(ArtifactName)'
+          targetPath: $(Build.ArtifactStagingDirectory)
+        # Troubleshooting artifacts are creating in the staging directory under the folder 'troubleshooting'.
+        # This will contain things such as heap dumps hprofs if testing hit OutOfMemory errors, log files captured
+        # during testing if tests failed, and linting reports.
+        - output: pipelineArtifact
+          artifactName: troubleshooting-$(System.StageName)-$(System.JobName)
+          targetPath: $(System.DefaultWorkingDirectory)/troubleshooting
+          displayName: 'Capture troubleshooting artifacts'
+          condition: and(not(succeeded()), eq(variables['HAS_TROUBLESHOOTING'], 'true'))
+          sbomEnabled: false
+
+        - output: pipelineArtifact
+          artifactName: 'repository-$(System.JobName)'
+          targetPath: '$(System.DefaultWorkingDirectory)'
+          displayName: 'Capture repo state for analysis'
+          condition: or(eq(variables['CaptureRepositoryOnFailure'], 'true'), and(failed(), eq(variables['Build.Reason'],'Schedule')))
+          sbomEnabled: false
 
     steps:
       # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
@@ -85,8 +110,6 @@ jobs:
 
       - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
         parameters:
-          Artifacts: ${{parameters.Artifacts}}
-          AdditionalModules: ${{parameters.AdditionalModules}}
           JobType: 'Build'
 
       - task: UsePythonVersion@0
@@ -108,7 +131,7 @@ jobs:
       # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
       # as we require the GitHub service connection to be loaded.
       - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
+        - template: /eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
           parameters:
             Paths: $(SparseCheckoutDirectories)
             SkipCheckoutNone: true
@@ -130,22 +153,17 @@ jobs:
         parameters:
           ServiceDirectory: ${{parameters.ServiceDirectory}}
 
-      - pwsh: |
-          $artifacts = '${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json
-          python3 --version
-
-          # Append dev package version suffix for each artifact
-          foreach ($artifact in $artifacts) {
-            python3 eng/versioning/set_versions.py --build-type ${{parameters.SDKType}} --build-qualifier alpha.$(Build.BuildNumber) --artifact-id $artifact.name --group-id $artifact.groupId
-          }
-
-          # Set zero-dev-version for packages
-          python3 eng/versioning/set_versions.py --set-dev-zero-version --build-type ${{parameters.SDKType}} --build-qualifier alpha.$(Build.BuildNumber)
-
-          # Apply version settings to repository
-          python3 eng/versioning/update_versions.py --update-type library --build-type ${{parameters.SDKType}} --sr
-        condition: and(succeeded(), eq(variables['SetDevVersion'],'true'))
+      - task: PowerShell@2
         displayName: Setup Dev versioning
+        condition: and(succeeded(), eq(variables['SetDevVersion'],'true'))
+        inputs:
+          pwsh: true
+          filePath: eng/pipelines/scripts/Set-Dev-Versioning.ps1
+          arguments: >
+            -BuildNumber $(Build.BuildNumber)
+            -SdkType ${{parameters.SDKType}}
+        env:
+          ARTIFACTSJSON: $(ArtifactsJson)
 
       - ${{if ne(parameters.ServiceDirectory, '')}}:
         - task: Powershell@2
@@ -184,7 +202,7 @@ jobs:
           filePath: eng/scripts/Verify-Required-Maven-Artifacts.ps1
           arguments: >
             -BuildOutputDirectory $(System.DefaultWorkingDirectory)/build
-            -ArtifactsList ('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
+            -ArtifactsList ('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId)
             -InformationAction Continue
 
       - task: PowerShell@2
@@ -203,35 +221,39 @@ jobs:
         parameters:
           Artifacts: ${{parameters.ReleaseArtifacts}}
 
-      - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-        parameters:
-          ArtifactPath: $(Build.ArtifactStagingDirectory)
-          ArtifactName: '$(ArtifactName)'
-
-      - ${{if eq(variables['System.TeamProject'], 'internal') }}:
-        - task: AzureArtifacts.manifest-generator-task.manifest-generator-task.ManifestGeneratorTask@0
-          displayName: 'Generate SBOM'
-          condition: succeededOrFailed()
-          inputs:
-            BuildDropPath: $(Build.ArtifactStagingDirectory)
-
-        - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-          parameters:
-            ArtifactPath: '$(Build.ArtifactStagingDirectory)/_manifest'
-            ArtifactName: 'manifest'
-
       - template: /eng/common/pipelines/templates/steps/detect-api-changes.yml
         parameters:
           Artifacts: ${{parameters.Artifacts}}
 
-      - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
+      - template: /eng/pipelines/templates/steps/clean-maven-local-cache.yml
 
   - job: 'Analyze'
     condition: and(succeeded(), ne(variables['Skip.Analyze'], 'true'))
 
     pool:
-      name: azsdk-pool-mms-ubuntu-2004-general
-      vmImage: MMSUbuntu20.04
+      name: $(LINUXPOOL)
+      image: $(LINUXVMIMAGE)
+      os: linux
+
+    templateContext:
+      outputParentDirectory: '$(System.DefaultWorkingDirectory)'
+      outputs:
+        - output: pipelineArtifact
+          artifactName: 'repository-$(System.JobName)'
+          targetPath: '$(System.DefaultWorkingDirectory)'
+          displayName: 'Capture repo state for analysis'
+          condition: or(eq(variables['CaptureRepositoryOnFailure'], 'true'), and(failed(), eq(variables['Build.Reason'],'Schedule')))
+          sbomEnabled: false
+
+        # Troubleshooting artifacts are creating in the staging directory under the folder 'troubleshooting'.
+        # This will contain things such as heap dumps hprofs if testing hit OutOfMemory errors, log files captured
+        # during testing if tests failed, and linting reports.
+        - output: pipelineArtifact
+          artifactName: troubleshooting-$(System.StageName)-$(System.JobName)
+          targetPath: $(System.DefaultWorkingDirectory)/troubleshooting
+          displayName: 'Capture troubleshooting artifacts'
+          condition: and(not(succeeded()), eq(variables['HAS_TROUBLESHOOTING'], 'true'))
+          sbomEnabled: false
 
     steps:
       - task: UsePythonVersion@0
@@ -261,8 +283,6 @@ jobs:
 
       - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
         parameters:
-          Artifacts: ${{ parameters.Artifacts }}
-          AdditionalModules: ${{ parameters.AdditionalModules }}
           JobType: 'Analyze'
 
       - task: PowerShell@2
@@ -277,7 +297,7 @@ jobs:
       # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
       # as we require the GitHub service connection to be loaded.
       - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
+        - template: /eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
           parameters:
             Paths: $(SparseCheckoutDirectories)
             SkipCheckoutNone: true
@@ -357,68 +377,16 @@ jobs:
           RunLinting: true
           SDKType: ${{ parameters.SDKType }}
 
-      - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
+      - template: /eng/pipelines/templates/steps/clean-maven-local-cache.yml
 
       - template: /eng/common/pipelines/templates/steps/eng-common-workflow-enforcer.yml
 
-  - job: Compliance
-    variables:
-    - template: /eng/pipelines/templates/variables/globals.yml
-
-    pool:
-      name: azsdk-pool-mms-win-2022-general
-      vmImage: windows-2022
-
-    steps:
-      # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
-      # as we require the GitHub service connection to be loaded.
-      - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-          parameters:
-            Paths:
-              - '**/*.xml'
-
-      # For PullRequests CredScan will be run against the files changed in the PR.
-      # For non-pull requests CredScan
-      - ${{if eq(variables['Build.Reason'], 'PullRequest')}}:
-        - task: PowerShell@2
-          displayName: 'Determine files changed in PR for sparse-checkout'
-          inputs:
-            pwsh: true
-            filePath: eng/pipelines/scripts/Set-CredScanCheckoutDirectories.ps1
-      - ${{else}}:
-        - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
-          parameters:
-            Artifacts: ${{ parameters.Artifacts }}
-            AdditionalModules: ${{ parameters.AdditionalModules }}
-            JobType: 'Compliance'
-            UseCache: false
-
-        - task: PowerShell@2
-          displayName: Generate directories variable for sparse checkout
-          inputs:
-            pwsh: true
-            filePath: $(Build.SourcesDirectory)/eng/scripts/Generate-ServiceDirectories-From-Project-List.ps1
-            arguments: >
-              -SourcesDirectory $(Build.SourcesDirectory)
-              -ProjectList $(ProjectList)
-
-      # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
-      # as we require the GitHub service connection to be loaded.
-      - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-          parameters:
-            SkipCheckoutNone: true
-            Paths: $(SparseCheckoutDirectories)
-
-      - template: /eng/common/pipelines/templates/steps/credscan.yml
-        parameters:
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          BaselineFilePath: $(Build.SourcesDirectory)\eng\java.gdnbaselines
-
-  - template: /eng/common/pipelines/templates/jobs/archetype-sdk-tests-generate.yml
+  - template: /eng/common/pipelines/templates/jobs/generate-job-matrix.yml
     parameters:
       JobTemplatePath: /eng/pipelines/templates/jobs/ci.tests.yml
+      OsVmImage: $(LINUXVMIMAGE)
+      Pool: $(LINUXPOOL)
+      Os: linux
       MatrixConfigs: ${{ parameters.MatrixConfigs }}
       MatrixFilters: ${{ parameters.MatrixFilters }}
       MatrixReplace: ${{ parameters.MatrixReplace }}
@@ -428,8 +396,6 @@ jobs:
         SDKType: ${{ parameters.SDKType }}
         ServiceDirectory: ${{ parameters.ServiceDirectory }}
         TestPipeline: ${{ parameters.TestPipeline }}
-        Artifacts: ${{ parameters.Artifacts }}
-        AdditionalModules: ${{ parameters.AdditionalModules }}
         PreTestSteps: ${{ parameters.PreTestSteps }}
         PreBuildSteps: ${{ parameters.PreBuildSteps }}
         TestGoals: ${{ parameters.TestGoals }}
@@ -437,3 +403,4 @@ jobs:
         TestParallelization: ${{ parameters.TestParallelization }}
         BuildParallelization: ${{ parameters.BuildParallelization }}
         AdditionalLintingOptions: ${{ parameters.AdditionalLintingOptions }}
+        TimeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
diff --git a/eng/pipelines/templates/jobs/live.tests.yml b/eng/pipelines/templates/jobs/live.tests.yml
index 912801af8aa80..d7c9009131b16 100644
--- a/eng/pipelines/templates/jobs/live.tests.yml
+++ b/eng/pipelines/templates/jobs/live.tests.yml
@@ -1,8 +1,6 @@
 parameters:
   ServiceDirectory: ''
   TestResourceDirectories:
-  Artifacts: []
-  AdditionalModules: []
   EnvVars: {}
   MaxParallel: 0
   PreSteps: []
@@ -23,6 +21,7 @@ parameters:
   TestParallelization: '1'
   TestVersionSupport: $(TestVersionSupport)
   UseHttpFaultInjector: false
+  OSName:
 
 jobs:
   - job:
@@ -33,27 +32,49 @@ jobs:
       matrix: $[ ${{ parameters.Matrix }} ]
 
     variables:
-      - template: /eng/pipelines/templates/variables/globals.yml
       - name: ArmTemplateParameters
         value: '@{}'
 
     timeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
 
     pool:
+      # 1es pipeline templates converts `image` to demands: ImageOverride under the hood
+      # which is incompatible with image selection in the default non-1es hosted pools
+      ${{ if eq(parameters.OSName, 'macOS') }}:
+        vmImage: $(OSVmImage)
+      ${{ else }}:
+        image: $(OSVmImage)
       name: $(Pool)
-      vmImage: $(OSVmImage)
+      os: ${{ parameters.OSName }}
 
     ${{ if eq(parameters.UsePlatformContainer, 'true') }}:
       # Add a default so the job doesn't fail when the matrix is empty
       container: $[ variables['Container'] ]
 
+    templateContext:
+      outputParentDirectory: '$(System.DefaultWorkingDirectory)'
+      outputs:
+        # Troubleshooting artifacts are creating in the staging directory under the folder 'troubleshooting'.
+        # This will contain things such as heap dumps hprofs if testing hit OutOfMemory errors, log files captured
+        # during testing if tests failed, and linting reports.
+        - output: pipelineArtifact
+          artifactName: troubleshooting-$(System.StageName)-$(System.JobName)
+          targetPath: $(System.DefaultWorkingDirectory)/troubleshooting
+          displayName: 'Capture troubleshooting artifacts'
+          condition: and(not(succeeded()), eq(variables['HAS_TROUBLESHOOTING'], 'true'))
+          sbomEnabled: false
+
+        - output: pipelineArtifact
+          artifactName: 'repository-$(System.JobName)'
+          targetPath: '$(System.DefaultWorkingDirectory)'
+          displayName: 'Capture repo state for analysis'
+          condition: or(eq(variables['CaptureRepositoryOnFailure'], 'true'), and(failed(), eq(variables['Build.Reason'],'Schedule')))
+          sbomEnabled: false
+
     steps:
       - template: /eng/pipelines/templates/steps/initialize-test-environment.yml
         parameters:
-          Artifacts: ${{ parameters.Artifacts }}
-          AdditionalModules: ${{ parameters.AdditionalModules }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          SDKType: ${{ parameters.SDKType }}
+          OS: ${{ parameters.OSName }}
 
       # macOS has known issues running HTTP fault injector, change this once
       # https://github.com/Azure/azure-sdk-tools/pull/6216 is resolved
@@ -100,8 +121,6 @@ jobs:
 
       - template: /eng/pipelines/templates/steps/build-and-test.yml
         parameters:
-          SDKType: ${{ parameters.SDKType }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
           ParallelTestPlayback: 'false'
           BuildParallelization: ${{ parameters.BuildParallelization }}
           IsLiveTest: true
@@ -115,7 +134,7 @@ jobs:
             ${{ each var in parameters.EnvVars }}:
               ${{ var.key }}: ${{ var.value }}
 
-      - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
+      - template: /eng/pipelines/templates/steps/clean-maven-local-cache.yml
 
       # macOS has known issues running HTTP fault injector, change this once
       # https://github.com/Azure/azure-sdk-tools/pull/6216 is resolved
diff --git a/eng/pipelines/templates/jobs/native.live.tests.yml b/eng/pipelines/templates/jobs/native.live.tests.yml
index d081a9522b78b..1660e94d174ed 100644
--- a/eng/pipelines/templates/jobs/native.live.tests.yml
+++ b/eng/pipelines/templates/jobs/native.live.tests.yml
@@ -1,27 +1,69 @@
 parameters:
-  ServiceDirectory: ''
-  TestResourceDirectories:
-  Artifacts: []
-  AdditionalModules: []
-  EnvVars: {}
-  MaxParallel: 0
-  PreSteps: []
-  PostSteps: []
-  TimeoutInMinutes: 60
-  TestMode: 'LIVE'
-  TestResultsFiles: ''
-  CloudConfig: {}
-  ArmTemplateParameters: '@{}'
-  Location: ''
-  Matrix: ''
-  DependsOn: ''
-  UsePlatformContainer: false
-  DisableAzureResourceCreation: false
-  BuildParallelization: '2C'
-  TestGoals: $(TestGoals)
-  TestOptions: $(TestOptions)
-  TestParallelization: '1'
-  TestVersionSupport: $(TestVersionSupport)
+  - name: ServiceDirectory
+    type: string
+    default: ''
+  - name: TestResourceDirectories
+    type: object
+    default: null
+  - name: EnvVars
+    type: object
+    default: {}
+  - name: MaxParallel
+    type: number
+    default: 0
+  - name: PreSteps
+    type: object
+    default: []
+  - name: PostSteps
+    type: object
+    default: []
+  - name: TimeoutInMinutes
+    type: number
+    default: 60
+  - name: TestMode
+    type: string
+    default: 'LIVE'
+  - name: TestResultsFiles
+    type: string
+    default: ''
+  - name: CloudConfig
+    type: object
+    default: {}
+  - name: ArmTemplateParameters
+    type: string
+    default: '@{}'
+  - name: Location
+    type: string
+    default: ''
+  - name: Matrix
+    type: string
+    default: ''
+  - name: DependsOn
+    type: string
+    default: ''
+  - name: UsePlatformContainer
+    type: boolean
+    default: false
+  - name: DisableAzureResourceCreation
+    type: boolean
+    default: false
+  - name: BuildParallelization
+    type: string
+    default: '2C'
+  - name: TestGoals
+    type: string
+    default: $(TestGoals)
+  - name: TestOptions
+    type: string
+    default: $(TestOptions)
+  - name: TestParallelization
+    type: string
+    default: '1'
+  - name: TestVersionSupport
+    type: string
+    default: $(TestVersionSupport)
+  - name: OSName
+    type: string
 
 jobs:
   - job:
@@ -32,27 +74,42 @@ jobs:
       matrix: $[ ${{ parameters.Matrix }} ]
 
     variables:
-      - template: /eng/pipelines/templates/variables/globals.yml
       - name: ArmTemplateParameters
         value: '@{}'
 
     timeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
 
     pool:
+      # 1es pipeline templates converts `image` to demands: ImageOverride under the hood
+      # which is incompatible with image selection in the default non-1es hosted pools
+      ${{ if eq(parameters.OSName, 'macOS') }}:
+        vmImage: $(OSVmImage)
+      ${{ else }}:
+        image: $(OSVmImage)
       name: $(Pool)
-      vmImage: $(OSVmImage)
+      os: ${{ parameters.OSName }}
 
     ${{ if eq(parameters.UsePlatformContainer, 'true') }}:
       # Add a default so the job doesn't fail when the matrix is empty
       container: $[ variables['Container'] ]
 
+    templateContext:
+      outputParentDirectory: '$(System.DefaultWorkingDirectory)'
+      outputs:
+        # Troubleshooting artifacts are creating in the staging directory under the folder 'troubleshooting'.
+        # This will contain things such as heap dumps hprofs if testing hit OutOfMemory errors, log files captured
+        # during testing if tests failed, and linting reports.
+        - output: pipelineArtifact
+          artifactName: troubleshooting-$(System.StageName)-$(System.JobName)
+          targetPath: $(System.DefaultWorkingDirectory)/troubleshooting
+          displayName: 'Capture troubleshooting artifacts'
+          condition: and(not(succeeded()), eq(variables['HAS_TROUBLESHOOTING'], 'true'))
+          sbomEnabled: false
+
     steps:
       - template: /eng/pipelines/templates/steps/initialize-test-environment.yml
         parameters:
-          Artifacts: ${{ parameters.Artifacts }}
-          AdditionalModules: ${{ parameters.AdditionalModules }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          SDKType: ${{ parameters.SDKType }}
+          OS: ${{ parameters.OSName }}
 
       - template: /eng/pipelines/templates/steps/install-reporting-tools.yml
         parameters:
@@ -91,8 +148,6 @@ jobs:
 
       - template: /eng/pipelines/templates/steps/build-and-test-native.yml
         parameters:
-          SDKType: ${{ parameters.SDKType }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
           ParallelTestPlayback: 'false'
           BuildParallelization: ${{ parameters.BuildParallelization }}
           IsLiveTest: true
diff --git a/eng/pipelines/templates/stages/1es-redirect.yml b/eng/pipelines/templates/stages/1es-redirect.yml
new file mode 100644
index 0000000000000..a51d503d24b12
--- /dev/null
+++ b/eng/pipelines/templates/stages/1es-redirect.yml
@@ -0,0 +1,60 @@
+resources:
+  repositories:
+    - repository: 1ESPipelineTemplates
+      type: git
+      name: 1ESPipelineTemplates/1ESPipelineTemplates
+      ref: refs/tags/release
+    - repository: azure-sdk-build-tools
+      type: git
+      name: internal/azure-sdk-build-tools
+      ref: refs/tags/azure-sdk-build-tools_20240320.1
+
+parameters:
+- name: stages
+  type: stageList
+  default: []
+- name: UseOfficial
+  type: boolean
+  default: true
+
+extends:
+  ${{ if and(parameters.UseOfficial, eq(variables['System.TeamProject'], 'internal')) }}:
+    template: v1/1ES.Official.PipelineTemplate.yml@1ESPipelineTemplates
+  ${{ else }}:
+    template: v1/1ES.Unofficial.PipelineTemplate.yml@1ESPipelineTemplates
+  parameters:
+    settings:
+      skipBuildTagsForGitHubPullRequests: true
+    sdl:
+      sourceAnalysisPool:
+        name: azsdk-pool-mms-win-2022-general
+        image: azsdk-pool-mms-win-2022-1espt
+        os: windows
+      sourceRepositoriesToScan:
+        exclude:
+          - repository: azure-sdk-build-tools
+      # TODO: migrate analyze spot bugs configuration here from steps/run-and-validate-linting.yml
+      spotBugs:
+        enabled: false
+      binskim:
+        # Exclude imported azure-sdk-build-tools gpg/azcopy binaries
+        # See https://dev.azure.com/securitytools/SecurityIntegration/_wiki/wikis/Guardian/1378/Glob-Format
+        analyzeTargetGlob: +:file|**/*.jar;+:file|**/*.exe;-:f|**/tools/gpg/**/*.dll;-:f|**/tools/gpg/**/*.exe;-:f|**/tools/azcopy/**/*.exe;-:f|**/tools/azcopy/**/*.dll
+      eslint:
+        enabled: false
+        justificationForDisabling: 'ESLint injected task has failures because it uses an old version of mkdirp. We should not fail for tools not controlled by the repo. See: https://dev.azure.com/azure-sdk/internal/_build/results?buildId=3499746'
+      codeql:
+        compiled:
+          enabled: false
+          justificationForDisabling: CodeQL times our pipelines out by running for 2+ hours before being force canceled.
+      psscriptanalyzer:
+        compiled: true
+        break: true
+      policy: M365
+      credscan:
+        suppressionsFile: $(Build.SourcesDirectory)/eng/CredScanSuppression.json
+        scanFolder: $(Build.SourcesDirectory)/credscan.tsv
+        toolVersion: 2.3.12.23
+        baselineFiles: $(Build.SourcesDirectory)/eng/java.gdnbaselines
+
+    stages: ${{ parameters.stages }}
diff --git a/eng/pipelines/templates/stages/archetype-java-release-batch.yml b/eng/pipelines/templates/stages/archetype-java-release-batch.yml
index 9b2f1c0006dbc..de255797bf329 100644
--- a/eng/pipelines/templates/stages/archetype-java-release-batch.yml
+++ b/eng/pipelines/templates/stages/archetype-java-release-batch.yml
@@ -41,14 +41,17 @@ stages:
         timeoutInMinutes: 20
         variables:
           - template: /eng/pipelines/templates/variables/globals.yml
+          - template: /eng/pipelines/templates/variables/image.yml
         pool:
           name: azsdk-pool-mms-ubuntu-2004-general
-          vmImage: MMSUbuntu20.04
+          image: azsdk-pool-mms-ubuntu-2004-1espt
+          os: linux
         strategy:
           runOnce:
             deploy:
               steps:
                 - checkout: none
+                - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
 
                 - download: current
                   displayName: 'Download Artifact: ${{parameters.ArtifactName}}'
@@ -59,9 +62,10 @@ stages:
                     Artifacts: ${{parameters.Artifacts}}
                     ArtifactDirectory: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
 
-                - publish: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
-                  artifact: ${{parameters.ArtifactName}}-signed
-                  displayName: 'Store signed packages in ${{parameters.ArtifactName}}-signed artifact'
+                - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+                  parameters:
+                    ArtifactPath: '$(Pipeline.Workspace)/${{parameters.ArtifactName}}'
+                    ArtifactName: '${{parameters.ArtifactName}}-signed'
 
   # We generate two interdependent stages for each artifact listed in the ci.yml file, creates the release
   # in GitHub. The Release stage publishes to Maven Central. Both stages require approval since they
@@ -73,15 +77,21 @@ stages:
       displayName: 'Releasing: ${{ length(parameters.Artifacts) }} libraries'
       dependsOn: Signing
       condition: and(succeeded(), ne(variables['SetDevVersion'], 'true'), ne(variables['Skip.Release'], 'true'), ne(variables['Build.Repository.Name'], 'Azure/azure-sdk-for-java-pr'))
+      variables:
+        - template: /eng/pipelines/templates/variables/globals.yml
+        # Convert artifact parameter objects to json and set them as variables to be used in
+        # pipeline jobs/stages. By setting these as a variable once we save thousands of lines
+        # of yaml in extreme cases. This helps us stay under the devops max yaml size limits.
+        - name: ArtifactsJson
+          value: '${{ convertToJson(parameters.Artifacts) }}'
       jobs:
         - job: VerifyReleaseVersion
           displayName: "Verify release versions"
           condition: ne(variables['Skip.VersionVerification'], 'true')
-          variables:
-            - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-ubuntu-2004-general
-            vmImage: MMSUbuntu20.04
+            image: azsdk-pool-mms-ubuntu-2004-1espt
+            os: linux
 
           steps:
             - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
@@ -91,7 +101,7 @@ stages:
                   - '**/*.xml'
                   - '!sdk/**/test-recordings'
                   - '!sdk/**/session-records'
-            
+
             - download: current
               displayName: 'Download Artifact: ${{parameters.ArtifactName}}'
               artifact: ${{parameters.ArtifactName}}-signed
@@ -108,7 +118,7 @@ stages:
 
             # Do all the verifications for all the artifacts after the sparse-checkout
             - pwsh: |
-                $ArtifactsList = $('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
+                $ArtifactsList = $('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId)
                 $ArtifactsList | Format-Table -Property GroupId, Name | Out-String | Write-Host
                 python --version
                 foreach($artifact in $ArtifactsList) {
@@ -120,7 +130,7 @@ stages:
             # need to list dependencies in the POM. The flattened POM is generated and won't comply with our versioning
             # validation, so omit uberJars from the artifacts list to verify.
             - pwsh: |
-                $ArtifactsList = $('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId, uberJar | Where-Object -Not "uberJar")
+                $ArtifactsList = $('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId, uberJar | Where-Object -Not "uberJar")
                 $ArtifactsList | Format-Table -Property GroupId, Name | Out-String | Write-Host
                 foreach($artifact in $ArtifactsList) {
                   $(Build.SourcesDirectory)/eng/versioning/scan_for_unreleased_dependencies.ps1 $($artifact.groupId) $($artifact.name) $(Build.SourcesDirectory)/sdk/${{parameters.ServiceDirectory}}
@@ -151,18 +161,17 @@ stages:
                 filePath: $(Build.SourcesDirectory)/eng/versioning/verify_release_set.ps1
                 arguments: >
                   -ServiceDirectory $(Build.SourcesDirectory)/sdk/${{parameters.ServiceDirectory}}
-                  -ArtifactsList ('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId, uberJar)
+                  -ArtifactsList ('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId, uberJar)
 
         - deployment: TagRepository
           displayName: "Create release tag"
           condition: and(succeeded(), ne(variables['Skip.TagRepository'], 'true'))
           environment: ${{ parameters.Environment }}
           dependsOn: VerifyReleaseVersion
-          variables:
-            - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -188,11 +197,10 @@ stages:
           timeoutInMinutes: 120
           environment: ${{ parameters.Environment }}
           dependsOn: TagRepository
-          variables:
-            - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -206,6 +214,7 @@ stages:
                         - Name: Azure/azure-sdk-for-java
                           Commitish: $(Build.SourceVersion)
                           WorkingDirectory: $(Pipeline.Workspace)/azure-sdk-for-java
+                  - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
                   - download: current
                     displayName: 'Download Artifacts'
                     artifact: ${{parameters.ArtifactName}}-signed
@@ -218,7 +227,7 @@ stages:
                         Target: EsrpRelease
                         BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                         JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                    - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                    - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                       parameters:
                         ArtifactName: ${{parameters.ArtifactName}}-esrp-$(System.JobAttempt)
                         ArtifactPath: $(Pipeline.Workspace)/EsrpPackages
@@ -229,7 +238,7 @@ stages:
                         Target: MavenCentral
                         BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                         JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                    - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                    - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                       parameters:
                         ArtifactName: ${{parameters.ArtifactName}}-mavencentral-$(System.JobAttempt)
                         ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -243,7 +252,7 @@ stages:
                       Target: JavaDevFeed
                       BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                       JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                  - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                  - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                     parameters:
                       ArtifactName: ${{parameters.ArtifactName}}-javadevfeed-$(System.JobAttempt)
                       ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -253,11 +262,10 @@ stages:
           condition: and(succeeded(), ne(variables['Skip.UpdatePackageVersion'], 'true'))
           environment: ${{ parameters.Environment }}
           dependsOn: PublishPackage
-          variables:
-            - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -280,7 +288,7 @@ stages:
                   # Apply the version increment to each library, which only updates the version_client.txt file,
                   # and then call to update_versions will then apply all of the updates.
                   - pwsh: |
-                      $ArtifactsList = $('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
+                      $ArtifactsList = $('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId)
                       $ArtifactsList | Format-Table -Property GroupId, Name | Out-String | Write-Host
                       python --version
                       foreach($artifact in $ArtifactsList) {
@@ -308,10 +316,10 @@ stages:
 
           pool:
             name: azsdk-pool-mms-ubuntu-2004-general
-            vmImage: MMSUbuntu20.04
+            image: azsdk-pool-mms-ubuntu-2004-1espt
+            os: linux
 
           variables:
-            - template: /eng/pipelines/templates/variables/globals.yml
             - name: DocValidationImageId
               value: azuresdkimages.azurecr.io/javarefautocr:latest
 
@@ -361,11 +369,10 @@ stages:
           condition: and(succeeded(), ne(variables['Skip.PublishDocs'], 'true'))
           environment: ${{ parameters.Environment }}
           dependsOn: PublishPackage
-          variables:
-            - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -393,18 +400,29 @@ stages:
   - ${{if ne(parameters.EnableIntegrationStage, false)}}:
     - stage: Integration
       dependsOn: Signing
+      variables:
+        - template: /eng/pipelines/templates/variables/globals.yml
+        - template: /eng/pipelines/templates/variables/image.yml
       jobs:
       - job: PublishPackages
         condition: or(eq(variables['SetDevVersion'], 'true'), and(eq(variables['Build.Reason'],'Schedule'), eq(variables['System.TeamProject'], 'internal')))
         displayName: Publish package to daily feed
         variables:
-          - template: /eng/pipelines/templates/variables/globals.yml
           - name: RepositoryUrl
             value: https://pkgs.dev.azure.com/azure-sdk/internal/_packaging/azure-sdk-for-java-pr/maven/v1
 
+        templateContext:
+          outputs:
+            - ${{ each artifact in parameters.Artifacts }}:
+              - ${{if ne(artifact.skipPublishDevFeed, 'true')}}:
+                - output: pipelineArtifact
+                  artifactName: integration-${{parameters.ArtifactName}}-${{artifact.name}}-javadevfeed
+                  targetPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
+
         pool:
           name: azsdk-pool-mms-win-2022-general
-          vmImage: windows-2022
+          image: azsdk-pool-mms-win-2022-1espt
+          os: windows
         steps:
         - checkout: azure-sdk-build-tools
           path: azure-sdk-build-tools
@@ -416,6 +434,8 @@ stages:
                 Commitish: $(Build.SourceVersion)
                 WorkingDirectory: $(Pipeline.Workspace)/azure-sdk-for-java
 
+        - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
+
         - download: current
           displayName: 'Download Artifact: ${{parameters.ArtifactName}}-signed'
           artifact: ${{parameters.ArtifactName}}-signed
@@ -444,19 +464,15 @@ stages:
                 RepositoryUrl: $(RepositoryUrl)
                 Target: JavaDevFeed
                 JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-            - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-              parameters:
-                ArtifactName: integration-${{parameters.ArtifactName}}-${{artifact.name}}-javadevfeed-$(System.JobAttempt)
-                ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
 
       - job: PublishDocsToNightlyBranch
         dependsOn: PublishPackages
         condition: or(eq(variables['SetDevVersion'], 'true'), and(eq(variables['Build.Reason'],'Schedule'), eq(variables['System.TeamProject'], 'internal')))
         pool:
           name: azsdk-pool-mms-ubuntu-2004-general
-          vmImage: MMSUbuntu20.04
+          image: azsdk-pool-mms-ubuntu-2004-1espt
+          os: linux
         variables:
-          - template: /eng/pipelines/templates/variables/globals.yml
           - name: DocValidationImageId
             value: azuresdkimages.azurecr.io/javarefautocr:latest
         steps:
diff --git a/eng/pipelines/templates/stages/archetype-java-release-patch.yml b/eng/pipelines/templates/stages/archetype-java-release-patch.yml
index 229d30a163445..79755180ea23a 100644
--- a/eng/pipelines/templates/stages/archetype-java-release-patch.yml
+++ b/eng/pipelines/templates/stages/archetype-java-release-patch.yml
@@ -38,12 +38,14 @@ stages:
           - template: /eng/pipelines/templates/variables/globals.yml
         pool:
           name: azsdk-pool-mms-ubuntu-2004-general
-          vmImage: MMSUbuntu20.04
+          image: azsdk-pool-mms-ubuntu-2004-1espt
+          os: linux
         strategy:
           runOnce:
             deploy:
               steps:
                 - checkout: none
+                - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
 
                 - download: current
                   displayName: 'Download Artifact: ${{parameters.ArtifactName}}'
@@ -54,9 +56,10 @@ stages:
                     Artifacts: ${{parameters.Artifacts}}
                     ArtifactDirectory: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
 
-                - publish: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
-                  artifact: ${{parameters.ArtifactName}}-signed
-                  displayName: 'Store signed packages in ${{parameters.ArtifactName}}-signed artifact'
+                - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+                  parameters:
+                    ArtifactPath: '$(Pipeline.Workspace)/${{parameters.ArtifactName}}'
+                    ArtifactName: '${{parameters.ArtifactName}}-signed'
 
   # We generate two interdependent stages for each artifact listed in the ci.yml file, creates the release
   # in GitHub. The Release stage publishes to Maven Central. Both stages require approval since they
@@ -77,7 +80,8 @@ stages:
             - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -108,7 +112,8 @@ stages:
             - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -122,6 +127,7 @@ stages:
                         - Name: Azure/azure-sdk-for-java
                           Commitish: $(Build.SourceVersion)
                           WorkingDirectory: $(Pipeline.Workspace)/azure-sdk-for-java
+                  - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
                   - download: current
                     displayName: 'Download Artifacts'
                     artifact: ${{parameters.ArtifactName}}-signed
@@ -134,7 +140,7 @@ stages:
                         Target: EsrpRelease
                         BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                         JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                    - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                    - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                       parameters:
                         ArtifactName: ${{parameters.ArtifactName}}-esrp-$(System.JobAttempt)
                         ArtifactPath: $(Pipeline.Workspace)/EsrpPackages
@@ -145,7 +151,7 @@ stages:
                         Target: MavenCentral
                         BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                         JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                    - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                    - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                       parameters:
                         ArtifactName: ${{parameters.ArtifactName}}-mavencentral-$(System.JobAttempt)
                         ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -159,7 +165,7 @@ stages:
                       Target: JavaDevFeed
                       BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                       JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                  - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                  - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                     parameters:
                       ArtifactName: ${{parameters.ArtifactName}}-javadevfeed-$(System.JobAttempt)
                       ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -171,9 +177,12 @@ stages:
           dependsOn: PublishPackage
           variables:
             - template: /eng/pipelines/templates/variables/globals.yml
+            - name: ArtifactsJson
+              value: '${{ convertToJson(parameters.Artifacts) }}'
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
@@ -196,7 +205,7 @@ stages:
                   # Apply the version increment to each library, which only updates the version_client.txt file,
                   # and then call to update_versions will then apply all of the updates.
                   - pwsh: |
-                      $ArtifactsList = $('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
+                      $ArtifactsList = $('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId)
                       $ArtifactsList | Format-Table -Property GroupId, Name | Out-String | Write-Host
                       python --version
                       foreach($artifact in $ArtifactsList) {
@@ -225,7 +234,8 @@ stages:
 
           pool:
             name: azsdk-pool-mms-ubuntu-2004-general
-            vmImage: MMSUbuntu20.04
+            image: azsdk-pool-mms-ubuntu-2004-1espt
+            os: linux
 
           variables:
             - template: /eng/pipelines/templates/variables/globals.yml
@@ -284,7 +294,8 @@ stages:
             - template: /eng/pipelines/templates/variables/globals.yml
           pool:
             name: azsdk-pool-mms-win-2022-general
-            vmImage: windows-2022
+            image: azsdk-pool-mms-win-2022-1espt
+            os: windows
           strategy:
             runOnce:
               deploy:
diff --git a/eng/pipelines/templates/stages/archetype-java-release-pom-only.yml b/eng/pipelines/templates/stages/archetype-java-release-pom-only.yml
index 3ff256ae66d3d..bb19e357b7735 100644
--- a/eng/pipelines/templates/stages/archetype-java-release-pom-only.yml
+++ b/eng/pipelines/templates/stages/archetype-java-release-pom-only.yml
@@ -20,12 +20,14 @@ stages:
           - template: ../variables/globals.yml
         pool:
           name: azsdk-pool-mms-ubuntu-2004-general
-          vmImage: MMSUbuntu20.04
+          image: azsdk-pool-mms-ubuntu-2004-1espt
+          os: linux
         strategy:
           runOnce:
             deploy:
               steps:
                 - checkout: none
+                - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
 
                 - download: current
                   displayName: 'Download Artifact: ${{parameters.ArtifactName}}'
@@ -36,9 +38,10 @@ stages:
                     Artifacts: ${{parameters.Artifacts}}
                     ArtifactDirectory: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
 
-                - publish: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
-                  artifact: ${{parameters.ArtifactName}}-signed
-                  displayName: 'Store signed packages in ${{parameters.ArtifactName}}-signed artifact'
+                - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+                  parameters:
+                    ArtifactPath: '$(Pipeline.Workspace)/${{parameters.ArtifactName}}'
+                    ArtifactName: '${{parameters.ArtifactName}}-signed'
 
   # We generate two interdepdent stages for each artifact listed in the ci.yml file, creates the release
   # in GitHub. The Release stage publishes to Maven Central. Both stages require approval since they
@@ -60,7 +63,8 @@ stages:
               - template: ../variables/globals.yml
             pool:
               name: azsdk-pool-mms-win-2022-general
-              vmImage: windows-2022
+              image: azsdk-pool-mms-win-2022-1espt
+              os: windows
             strategy:
               runOnce:
                 deploy:
@@ -87,7 +91,8 @@ stages:
                 - template: ../variables/globals.yml
               pool:
                 name: azsdk-pool-mms-win-2022-general
-                vmImage: windows-2022
+                image: azsdk-pool-mms-win-2022-1espt
+                os: windows
               strategy:
                 runOnce:
                   deploy:
@@ -115,7 +120,7 @@ stages:
                           Target: EsrpRelease
                           BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                           JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                      - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                      - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                         parameters:
                           ArtifactName: ${{parameters.ArtifactName}}-${{artifact.name}}-esrp-$(System.JobAttempt)
                           ArtifactPath: $(Pipeline.Workspace)/EsrpPackages
@@ -131,7 +136,7 @@ stages:
                           Target: JavaDevFeed
                           BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                           JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                      - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                      - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                         parameters:
                           ArtifactName: ${{parameters.ArtifactName}}-${{artifact.name}}-javadevfeed-$(System.JobAttempt)
                           ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
diff --git a/eng/pipelines/templates/stages/archetype-java-release.yml b/eng/pipelines/templates/stages/archetype-java-release.yml
index 0ee8e13ab342b..996fcba1bca75 100644
--- a/eng/pipelines/templates/stages/archetype-java-release.yml
+++ b/eng/pipelines/templates/stages/archetype-java-release.yml
@@ -38,12 +38,14 @@ stages:
           - template: /eng/pipelines/templates/variables/globals.yml
         pool:
           name: azsdk-pool-mms-ubuntu-2004-general
-          vmImage: MMSUbuntu20.04
+          image: azsdk-pool-mms-ubuntu-2004-1espt
+          os: linux
         strategy:
           runOnce:
             deploy:
               steps:
                 - checkout: none
+                - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
 
                 - download: current
                   displayName: 'Download Artifact: ${{parameters.ArtifactName}}'
@@ -54,9 +56,10 @@ stages:
                     Artifacts: ${{parameters.Artifacts}}
                     ArtifactDirectory: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
 
-                - publish: $(Pipeline.Workspace)/${{parameters.ArtifactName}}
-                  artifact: ${{parameters.ArtifactName}}-signed
-                  displayName: 'Store signed packages in ${{parameters.ArtifactName}}-signed artifact'
+                - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+                  parameters:
+                    ArtifactPath: '$(Pipeline.Workspace)/${{parameters.ArtifactName}}'
+                    ArtifactName: '${{parameters.ArtifactName}}-signed'
 
   # We generate two interdependent stages for each artifact listed in the ci.yml file, creates the release
   # in GitHub. The Release stage publishes to Maven Central. Both stages require approval since they
@@ -77,7 +80,8 @@ stages:
               - template: /eng/pipelines/templates/variables/globals.yml
             pool:
               name: azsdk-pool-mms-ubuntu-2004-general
-              vmImage: MMSUbuntu20.04
+              image: azsdk-pool-mms-ubuntu-2004-1espt
+              os: linux
 
             steps:
               - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
@@ -146,7 +150,8 @@ stages:
               - template: /eng/pipelines/templates/variables/globals.yml
             pool:
               name: azsdk-pool-mms-win-2022-general
-              vmImage: windows-2022
+              image: azsdk-pool-mms-win-2022-1espt
+              os: windows
             strategy:
               runOnce:
                 deploy:
@@ -173,7 +178,8 @@ stages:
                 - template: /eng/pipelines/templates/variables/globals.yml
               pool:
                 name: azsdk-pool-mms-win-2022-general
-                vmImage: windows-2022
+                image: azsdk-pool-mms-win-2022-1espt
+                os: windows
               strategy:
                 runOnce:
                   deploy:
@@ -202,7 +208,7 @@ stages:
                             Target: EsrpRelease
                             BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                             JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                        - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                        - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                           parameters:
                             ArtifactName: ${{parameters.ArtifactName}}-${{artifact.name}}-esrp-$(System.JobAttempt)
                             ArtifactPath: $(Pipeline.Workspace)/EsrpPackages
@@ -215,7 +221,7 @@ stages:
                             Target: MavenCentral
                             BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                             JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                        - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                        - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                           parameters:
                             ArtifactName: ${{parameters.ArtifactName}}-${{artifact.name}}-mavencentral-$(System.JobAttempt)
                             ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -231,7 +237,7 @@ stages:
                           Target: JavaDevFeed
                           BuildToolsPath: $(Pipeline.Workspace)/azure-sdk-build-tools
                           JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-                      - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+                      - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
                         parameters:
                           ArtifactName: ${{parameters.ArtifactName}}-${{artifact.name}}-javadevfeed-$(System.JobAttempt)
                           ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -245,7 +251,8 @@ stages:
 
               pool:
                 name: azsdk-pool-mms-ubuntu-2004-general
-                vmImage: MMSUbuntu20.04
+                image: azsdk-pool-mms-ubuntu-2004-1espt
+                os: linux
 
               variables:
                 - template: /eng/pipelines/templates/variables/globals.yml
@@ -299,7 +306,8 @@ stages:
                 - template: /eng/pipelines/templates/variables/globals.yml
               pool:
                 name: azsdk-pool-mms-win-2022-general
-                vmImage: windows-2022
+                image: azsdk-pool-mms-win-2022-1espt
+                os: windows
               strategy:
                 runOnce:
                   deploy:
@@ -333,7 +341,8 @@ stages:
                 - template: /eng/pipelines/templates/variables/globals.yml
               pool:
                 name: azsdk-pool-mms-win-2022-general
-                vmImage: windows-2022
+                image: azsdk-pool-mms-win-2022-1espt
+                os: windows
               strategy:
                 runOnce:
                   deploy:
@@ -384,7 +393,8 @@ stages:
 
         pool:
           name: azsdk-pool-mms-win-2022-general
-          vmImage: windows-2022
+          image: azsdk-pool-mms-win-2022-1espt
+          os: windows
         steps:
         - checkout: azure-sdk-build-tools
           path: azure-sdk-build-tools
@@ -396,6 +406,8 @@ stages:
                 Commitish: $(Build.SourceVersion)
                 WorkingDirectory: $(Pipeline.Workspace)/azure-sdk-for-java
 
+        - template: /eng/pipelines/templates/steps/download-credscan-suppressions.yml
+
         - download: current
           displayName: 'Download Artifact: ${{parameters.ArtifactName}}-signed'
           artifact: ${{parameters.ArtifactName}}-signed
@@ -424,7 +436,7 @@ stages:
                 RepositoryUrl: $(RepositoryUrl)
                 Target: JavaDevFeed
                 JavaRepoRoot: $(Pipeline.Workspace)/azure-sdk-for-java
-            - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
+            - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
               parameters:
                 ArtifactName: integration-${{parameters.ArtifactName}}-${{artifact.name}}-javadevfeed-$(System.JobAttempt)
                 ArtifactPath: $(Pipeline.Workspace)/${{parameters.ArtifactName}}-signed
@@ -434,7 +446,8 @@ stages:
         condition: or(eq(variables['SetDevVersion'], 'true'), and(eq(variables['Build.Reason'],'Schedule'), eq(variables['System.TeamProject'], 'internal')))
         pool:
           name: azsdk-pool-mms-ubuntu-2004-general
-          vmImage: MMSUbuntu20.04
+          image: azsdk-pool-mms-ubuntu-2004-1espt
+          os: linux
         variables:
           - template: /eng/pipelines/templates/variables/globals.yml
           - name: DocValidationImageId
diff --git a/eng/pipelines/templates/stages/archetype-sdk-client-patch.yml b/eng/pipelines/templates/stages/archetype-sdk-client-patch.yml
index 961b4cbb086ac..d44bc3a2759d6 100644
--- a/eng/pipelines/templates/stages/archetype-sdk-client-patch.yml
+++ b/eng/pipelines/templates/stages/archetype-sdk-client-patch.yml
@@ -1,10 +1,3 @@
-resources:
-  repositories:
-    - repository: azure-sdk-build-tools
-      type: git
-      name: internal/azure-sdk-build-tools
-      ref: refs/tags/azure-sdk-build-tools_20230829.1
-
 parameters:
 - name: Artifacts
   type: object
@@ -34,290 +27,296 @@ parameters:
   type: boolean
   default: false
 
-stages:
-  - stage: Build
-    jobs:
-    - job: 'Build'
-
-      variables:
-        - template: /eng/pipelines/templates/variables/globals.yml
-        - name: ArtifactName
-          value: 'packages'
-
-      pool:
-        name: azsdk-pool-mms-ubuntu-2004-general
-
-      steps:
-        # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories as we require the GitHub
-        # service connection to be loaded.
-        - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-          - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-            parameters:
-              Paths:
-                - '**/*.xml'
-                - '**/*.md'
-                - '!sdk/**/test-recordings/*'
-                - '!sdk/**/session-records/*'
-
-        - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
-          parameters:
-            Artifacts: ${{parameters.Artifacts}}
-            AdditionalModules: ${{parameters.AdditionalModules}}
-            JobType: 'Build'
-
-        - template: /eng/common/pipelines/templates/steps/set-test-pipeline-version.yml
-          parameters:
-            PackageNames: "azure-sdk-template,azure-sdk-template-two,azure-sdk-template-three"
-            ServiceDirectory: "template"
-            TestPipeline: ${{parameters.TestPipeline}}
-
-        - task: UsePythonVersion@0
-          displayName: 'Use Python $(PythonVersion)'
-          inputs:
-            versionSpec: $(PythonVersion)
-
-        - task: PythonScript@0
-          displayName: 'Set versions for source build'
-          inputs:
-            scriptPath: 'eng/versioning/set_versions.py'
-            arguments: '--build-type client --pst --project-list $(ProjectList)'
-
-        - task: PythonScript@0
-          displayName: 'Update versions for source build'
-          inputs:
-            scriptPath: 'eng/versioning/update_versions.py'
-            arguments: '--update-type library --build-type client --sr'
-
-        - task: PythonScript@0
-          displayName: 'Generate FromSource POM and directories for sparse checkout'
-          inputs:
-            scriptPath: 'eng/scripts/generate_from_source_pom.py'
-            arguments: '--project-list $(ProjectList)'
-            workingDirectory: '$(System.DefaultWorkingDirectory)'
-
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-          parameters:
-            Paths: $(SparseCheckoutDirectories)
-            SkipCheckoutNone: true
-
-        - script: |
-            python -m pip install markdown2 BeautifulSoup4
-          displayName: 'pip install markdown2 and BeautifulSoup4'
-
-        # Save the Package Properties
-        #
-        # ServiceDirectories variable is a plain, comma separated list of ServiceDirectories that is output by the
-        # generate_from_source_pom script above.
-        - pwsh: |
-            $ServiceDirs = '$(ServiceDirectories)'
-            foreach($ServiceDir in $ServiceDirs.Split(',')) {
-              $(Build.SourcesDirectory)/eng/common/scripts/Save-Package-Properties.ps1 -ServiceDirectory $ServiceDir -OutDirectory $(Build.ArtifactStagingDirectory)/PackageInfo
-            }
-          workingDirectory: $(Pipeline.Workspace)
-          displayName: "Save Package Properties"
-
-        # Why is this necessary? Normal build and release jobs don't install reporting tools prior to building however
-        # they're still able to find the build tools through the root aggregate pom file. The ClientFromSourcePom only
-        # contains what is necessary for the list of libraries and needs to have them installed explicitly otherwise the
-        # checkstyle plugin will fail to find com.azure:sdk-build-tools which is a dependency. It does this in spite of
-        # the fact that we're not running actually running checkstyle, it loads it to skip it...well done.
-        - template: ../steps/install-reporting-tools.yml
-          parameters:
-            JdkVersion: ${{ parameters.JavaBuildVersion }}
-
-        - task: Maven@3
-          displayName: 'Build From Source and Package, JDK Version: ${{ parameters.JavadocSafeJavaBuildVersion }}'
-          inputs:
-            mavenPomFile: ClientFromSourcePom.xml
-            goals: 'deploy'
-            options: '$(DefaultOptions) -T ${{parameters.BuildParallelization}} -DskipTests -Dgenerate-overview -Dspotbugs.skip=true -Dcheckstyle.skip=true -Drevapi.skip=true -Dspotless.apply.skip=true -pl $(ProjectList) -am -DaltDeploymentRepository=id::default::file://$(System.DefaultWorkingDirectory)/build' # We include template-module, so we ensure it always builds in CI.
-            mavenOptions: '$(MemoryOptions) $(LoggingOptions)'
-            javaHomeOption: 'JDKVersion'
-            jdkVersionOption: ${{ parameters.JavadocSafeJavaBuildVersion }}
-            jdkArchitectureOption: 'x64'
-            publishJUnitResults: false
-
-        - task: PowerShell@2
-          displayName: 'Verify Required Maven Artifacts'
-          inputs:
-            pwsh: true
-            workingDirectory: $(Agent.BuildDirectory)
-            filePath: eng/scripts/Verify-Required-Maven-Artifacts.ps1
-            arguments: >
-              -BuildOutputDirectory $(System.DefaultWorkingDirectory)/build
-              -ArtifactsList ('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
-              -InformationAction Continue
-
-        - task: PowerShell@2
-          displayName: 'Copy artifacts to staging'
-          inputs:
-            pwsh: true
-            workingDirectory: $(Agent.BuildDirectory)
-            filePath: eng/scripts/Stage-MavenPackageArtifacts.ps1
-            arguments: >
-              -SourceDirectory $(System.DefaultWorkingDirectory)/build
-              -TargetDirectory $(Build.ArtifactStagingDirectory)
-              -Artifacts ('${{ replace(convertToJson(parameters.Artifacts), '''', '`''') }}' | ConvertFrom-Json | Where-Object -Not skipPublishPackage)
-              -InformationAction Continue
-
-        - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-          parameters:
-            ArtifactPath: $(Build.ArtifactStagingDirectory)
-            ArtifactName: '$(ArtifactName)'
-
-        - task: AzureArtifacts.manifest-generator-task.manifest-generator-task.ManifestGeneratorTask@0
-          displayName: 'Generate SBOM'
-          condition: succeededOrFailed()
-          inputs:
-            BuildDropPath: $(Build.ArtifactStagingDirectory)
-
-        - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-          parameters:
-            ArtifactPath: '$(Build.ArtifactStagingDirectory)/_manifest'
-            ArtifactName: 'manifest'
-
-        - template: ../steps/upload-repository-on-failure.yml
-
-    - job: 'AnalyzeAndVerify'
-      condition: and(succeeded(), ne(variables['Skip.Analyze'], 'true'))
-
-      variables:
-        - template: /eng/pipelines/templates/variables/globals.yml
-
-      pool:
-        name: azsdk-pool-mms-ubuntu-2004-general
-        vmImage: MMSUbuntu20.04
-
-      steps:
-        - task: UsePythonVersion@0
-          displayName: 'Use Python $(PythonVersion)'
-          inputs:
-            versionSpec: $(PythonVersion)
-
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-          parameters:
-            Paths:
-              - '**/*.xml'
-              - '**/*.md'
-              - '.vscode/cspell.json'
-              - '!sdk/**/test-recordings/*'
-              - '!sdk/**/session-records/*'
-
-        - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - stage: Build
+        jobs:
+        - job: 'Build'
+
+          variables:
+            - template: /eng/pipelines/templates/variables/globals.yml
+            - name: ArtifactName
+              value: 'packages'
+            - name: ArtifactsJson
+              value: '${{ convertToJson(parameters.Artifacts) }}'
+            - name: ArtifactsJsonEscaped
+              value: "${{ replace(convertToJson(parameters.Artifacts), '''', '`''') }}"
+            - name: AdditionalModulesJson
+              value: '${{ convertToJson(parameters.AdditionalModules) }}'
+
+          pool:
+            name: azsdk-pool-mms-ubuntu-2004-general
+            image: azsdk-pool-mms-ubuntu-2004-1espt
+            os: linux
+
+          steps:
+            # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories as we require the GitHub
+            # service connection to be loaded.
+            - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
+              - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
+                parameters:
+                  Paths:
+                    - '**/*.xml'
+                    - '**/*.md'
+                    - '!sdk/**/test-recordings/*'
+                    - '!sdk/**/session-records/*'
+
+            - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
+              parameters:
+                JobType: 'Build'
+
+            - template: /eng/common/pipelines/templates/steps/set-test-pipeline-version.yml
+              parameters:
+                PackageNames: "azure-sdk-template,azure-sdk-template-two,azure-sdk-template-three"
+                ServiceDirectory: "template"
+                TestPipeline: ${{parameters.TestPipeline}}
+
+            - task: UsePythonVersion@0
+              displayName: 'Use Python $(PythonVersion)'
+              inputs:
+                versionSpec: $(PythonVersion)
+
+            - task: PythonScript@0
+              displayName: 'Set versions for source build'
+              inputs:
+                scriptPath: 'eng/versioning/set_versions.py'
+                arguments: '--build-type client --pst --project-list $(ProjectList)'
+
+            - task: PythonScript@0
+              displayName: 'Update versions for source build'
+              inputs:
+                scriptPath: 'eng/versioning/update_versions.py'
+                arguments: '--update-type library --build-type client --sr'
+
+            - task: PythonScript@0
+              displayName: 'Generate FromSource POM and directories for sparse checkout'
+              inputs:
+                scriptPath: 'eng/scripts/generate_from_source_pom.py'
+                arguments: '--artifacts-list $(ArtifactsList) --additional-modules-list $(AdditionalModulesList)'
+                workingDirectory: '$(System.DefaultWorkingDirectory)'
+
+            - template: /eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
+              parameters:
+                Paths: $(SparseCheckoutDirectories)
+                SkipCheckoutNone: true
+
+            - script: |
+                python -m pip install markdown2 BeautifulSoup4
+              displayName: 'pip install markdown2 and BeautifulSoup4'
+
+            # Save the Package Properties
+            #
+            # ServiceDirectories variable is a plain, comma separated list of ServiceDirectories that is output by the
+            # generate_from_source_pom script above.
+            - pwsh: |
+                $ServiceDirs = '$(ServiceDirectories)'
+                foreach($ServiceDir in $ServiceDirs.Split(',')) {
+                  $(Build.SourcesDirectory)/eng/common/scripts/Save-Package-Properties.ps1 -ServiceDirectory $ServiceDir -OutDirectory $(Build.ArtifactStagingDirectory)/PackageInfo
+                }
+              workingDirectory: $(Pipeline.Workspace)
+              displayName: "Save Package Properties"
+
+            # Why is this necessary? Normal build and release jobs don't install reporting tools prior to building however
+            # they're still able to find the build tools through the root aggregate pom file. The ClientFromSourcePom only
+            # contains what is necessary for the list of libraries and needs to have them installed explicitly otherwise the
+            # checkstyle plugin will fail to find com.azure:sdk-build-tools which is a dependency. It does this in spite of
+            # the fact that we're not running actually running checkstyle, it loads it to skip it...well done.
+            - template: ../steps/install-reporting-tools.yml
+              parameters:
+                JdkVersion: ${{ parameters.JavaBuildVersion }}
+
+            - task: Maven@3
+              displayName: 'Build From Source and Package, JDK Version: ${{ parameters.JavadocSafeJavaBuildVersion }}'
+              inputs:
+                mavenPomFile: ClientFromSourcePom.xml
+                goals: 'deploy'
+                options: '$(DefaultOptions) -T ${{parameters.BuildParallelization}} -DskipTests -Dgenerate-overview -Dspotbugs.skip=true -Dcheckstyle.skip=true -Drevapi.skip=true -Dspotless.apply.skip=true -pl $(ProjectList) -am -DaltDeploymentRepository=id::default::file://$(System.DefaultWorkingDirectory)/build' # We include template-module, so we ensure it always builds in CI.
+                mavenOptions: '$(MemoryOptions) $(LoggingOptions)'
+                javaHomeOption: 'JDKVersion'
+                jdkVersionOption: ${{ parameters.JavadocSafeJavaBuildVersion }}
+                jdkArchitectureOption: 'x64'
+                publishJUnitResults: false
+
+            - task: PowerShell@2
+              displayName: 'Verify Required Maven Artifacts'
+              inputs:
+                pwsh: true
+                workingDirectory: $(Agent.BuildDirectory)
+                filePath: eng/scripts/Verify-Required-Maven-Artifacts.ps1
+                arguments: >
+                  -BuildOutputDirectory $(System.DefaultWorkingDirectory)/build
+                  -ArtifactsList ('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId)
+                  -InformationAction Continue
+
+            - task: PowerShell@2
+              displayName: 'Copy artifacts to staging'
+              inputs:
+                pwsh: true
+                workingDirectory: $(Agent.BuildDirectory)
+                filePath: eng/scripts/Stage-MavenPackageArtifacts.ps1
+                arguments: >
+                  -SourceDirectory $(System.DefaultWorkingDirectory)/build
+                  -TargetDirectory $(Build.ArtifactStagingDirectory)
+                  -Artifacts ('$(ArtifactsJsonEscaped)' | ConvertFrom-Json | Where-Object -Not skipPublishPackage)
+                  -InformationAction Continue
+
+            - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+              parameters:
+                ArtifactPath: $(Build.ArtifactStagingDirectory)
+                ArtifactName: '$(ArtifactName)'
+
+            - template: /eng/common/pipelines/templates/steps/publish-1es-artifact.yml
+              parameters:
+                ArtifactPath: '$(Build.ArtifactStagingDirectory)/_manifest'
+                ArtifactName: 'manifest'
+
+            - template: /eng/pipelines/templates/steps/clean-maven-local-cache.yml
+
+        - job: 'AnalyzeAndVerify'
+          condition: and(succeeded(), ne(variables['Skip.Analyze'], 'true'))
+
+          variables:
+            - template: /eng/pipelines/templates/variables/globals.yml
+            - name: ArtifactsJson
+              value: '${{ convertToJson(parameters.Artifacts) }}'
+            - name: AdditionalModulesJson
+              value: '${{ convertToJson(parameters.AdditionalModules) }}'
+
+          pool:
+            name: azsdk-pool-mms-ubuntu-2004-general
+            image: azsdk-pool-mms-ubuntu-2004-1espt
+            os: linux
+
+          steps:
+            - task: UsePythonVersion@0
+              displayName: 'Use Python $(PythonVersion)'
+              inputs:
+                versionSpec: $(PythonVersion)
+
+            - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
+              parameters:
+                Paths:
+                  - '**/*.xml'
+                  - '**/*.md'
+                  - '.vscode/cspell.json'
+                  - '!sdk/**/test-recordings/*'
+                  - '!sdk/**/session-records/*'
+
+            - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
+              parameters:
+                JobType: 'AnalyzeAndVerify'
+                UseCache: false
+
+            - template: /eng/common/pipelines/templates/steps/set-test-pipeline-version.yml
+              parameters:
+                PackageNames: "azure-sdk-template,azure-sdk-template-two,azure-sdk-template-three"
+                ServiceDirectory: "template"
+                TestPipeline: ${{parameters.TestPipeline}}
+
+            - task: PythonScript@0
+              displayName: 'Set versions for source build'
+              inputs:
+                scriptPath: 'eng/versioning/set_versions.py'
+                arguments: '--build-type client --pst --project-list $(ProjectList)'
+
+            - task: PythonScript@0
+              displayName: 'Update versions for source build'
+              inputs:
+                scriptPath: 'eng/versioning/update_versions.py'
+                arguments: '--update-type library --build-type client --sr'
+
+            - task: PythonScript@0
+              displayName: 'Generate FromSource POM and directories for sparse checkout'
+              inputs:
+                scriptPath: 'eng/scripts/generate_from_source_pom.py'
+                arguments: '--artifacts-list $(ArtifactsList) --additional-modules-list $(AdditionalModulesList)'
+                workingDirectory: '$(System.DefaultWorkingDirectory)'
+
+            - template: /eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
+              parameters:
+                Paths: $(SparseCheckoutDirectories)
+                SkipCheckoutNone: true
+
+            - template: /eng/common/pipelines/templates/steps/check-spelling.yml
+              parameters:
+                CspellConfigPath: .vscode/cspell.json
+
+            - task: PowerShell@2
+              displayName: 'Verify versions in POM files'
+              inputs:
+                pwsh: true
+                workingDirectory: $(Agent.BuildDirectory)
+                filePath: eng/versioning/pom_file_version_scanner.ps1
+                arguments: -Debug:$$(IsDebug)
+
+            # Do all the verifications for all the artifacts after the sparse-checkout.
+            - pwsh: |
+                $ArtifactsList = $('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId)
+                $ArtifactsList | Format-Table -Property GroupId, Name | Out-String | Write-Host
+                python --version
+                foreach($artifact in $ArtifactsList) {
+                  python eng/versioning/set_versions.py --build-type ${{parameters.SDKType}} --vv --ai $($artifact.name) --gi $($artifact.groupId)
+                }
+              displayName: 'Verify artifact versions for release'
+
+            # Uber JARs release a flattened POM, as dependencies are incorporated into the JAR circumnavigating the need to
+            # list dependencies in the POM. The flattened POM is generated and won't comply with our versioning validation,
+            # so omit uberJars from the artifacts list to verify.
+            #
+            # Note: This is different from the Batch release verification step because the ServiceDirectory, for this
+            # pipeline, is attached to each Artifact. It needs to be in the query, so it can be passed into the script.
+            - pwsh: |
+                $ArtifactsList = $('$(ArtifactsJson)' | ConvertFrom-Json | Select-Object name, groupId, ServiceDirectory, uberJar | Where-Object -Not "uberJar")
+                $ArtifactsList | Format-Table -Property GroupId, Name, ServiceDirectory | Out-String | Write-Host
+                foreach($artifact in $ArtifactsList) {
+                  $(Build.SourcesDirectory)/eng/versioning/scan_for_unreleased_dependencies.ps1 $($artifact.groupId) $($artifact.name) $(Build.SourcesDirectory)/sdk/$($artifact.ServiceDirectory)
+                }
+              displayName: "Verify no unreleased dependencies or beta dependencies for GA libraries"
+
+            - template: /eng/common/pipelines/templates/steps/verify-readme.yml
+              parameters:
+                # For the moment, ServiceDirectories is JSON which may change to a comma delimited list instead of what's
+                # going on below
+                ScanPaths: $(ServiceDirectoriesFullPath)
+
+            - template: /eng/common/pipelines/templates/steps/verify-samples.yml
+              parameters:
+                ServiceDirectories: $(ServiceDirectories)
+
+            # Use BasePathLength of 38 instead of the default 49 as some released files fail when the number is higher.
+            - template: /eng/common/pipelines/templates/steps/verify-path-length.yml
+              parameters:
+                SourceDirectory: $(Build.SourcesDirectory)
+                BasePathLength: 38
+
+            # Verify changelog, the way it currently works would require a complete overhaul in order to be able to deal
+            # with multiple artifacts and multiple service directories. Even ignoring this, there's still the fact that each
+            # artifact's metadata can cause this step to be skipped, so we still need to loop over this.
+            - ${{if ne(parameters.SDKType, 'data')}}:
+              - ${{ each artifact in parameters.Artifacts }}:
+                - ${{if ne(artifact.skipVerifyChangelog, 'true')}}:
+                  - template: /eng/common/pipelines/templates/steps/verify-changelog.yml
+                    parameters:
+                      PackageName: ${{artifact.name}}
+                      ServiceName: ${{artifact.ServiceDirectory}}
+                      # Note: For the patch release pipeline there's no point in verifying the non-release changelog since
+                      # the only time this pipeline should run is for a patch release.
+                      ForRelease: true
+
+            - template: /eng/common/pipelines/templates/steps/eng-common-workflow-enforcer.yml
+
+      # This pipeline should never get kicked off outside an internal, manual run. Put the checks here so the pipeline never
+      # gets to the signing/release portions unless this is the case.
+      - ${{if and(in(variables['Build.Reason'], 'Manual', ''), eq(variables['System.TeamProject'], 'internal'))}}:
+        - template: archetype-java-release-patch.yml
           parameters:
+            DependsOn:
+              - Build
+            SDKType: ${{ parameters.SDKType }}
             Artifacts: ${{ parameters.Artifacts }}
-            AdditionalModules: ${{ parameters.AdditionalModules }}
-            JobType: 'AnalyzeAndVerify'
-            UseCache: false
-
-        - template: /eng/common/pipelines/templates/steps/set-test-pipeline-version.yml
-          parameters:
-            PackageNames: "azure-sdk-template,azure-sdk-template-two,azure-sdk-template-three"
-            ServiceDirectory: "template"
-            TestPipeline: ${{parameters.TestPipeline}}
-
-        - task: PythonScript@0
-          displayName: 'Set versions for source build'
-          inputs:
-            scriptPath: 'eng/versioning/set_versions.py'
-            arguments: '--build-type client --pst --project-list $(ProjectList)'
-
-        - task: PythonScript@0
-          displayName: 'Update versions for source build'
-          inputs:
-            scriptPath: 'eng/versioning/update_versions.py'
-            arguments: '--update-type library --build-type client --sr'
-
-        - task: PythonScript@0
-          displayName: 'Generate FromSource POM and directories for sparse checkout'
-          inputs:
-            scriptPath: 'eng/scripts/generate_from_source_pom.py'
-            arguments: '--project-list $(ProjectList)'
-            workingDirectory: '$(System.DefaultWorkingDirectory)'
-
-        - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
-          parameters:
-            Paths: $(SparseCheckoutDirectories)
-            SkipCheckoutNone: true
-
-        - template: /eng/common/pipelines/templates/steps/check-spelling.yml
-          parameters:
-            CspellConfigPath: .vscode/cspell.json
-
-        - task: PowerShell@2
-          displayName: 'Verify versions in POM files'
-          inputs:
-            pwsh: true
-            workingDirectory: $(Agent.BuildDirectory)
-            filePath: eng/versioning/pom_file_version_scanner.ps1
-            arguments: -Debug:$$(IsDebug)
-
-        # Do all the verifications for all the artifacts after the sparse-checkout.
-        - pwsh: |
-            $ArtifactsList = $('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
-            $ArtifactsList | Format-Table -Property GroupId, Name | Out-String | Write-Host
-            python --version
-            foreach($artifact in $ArtifactsList) {
-              python eng/versioning/set_versions.py --build-type ${{parameters.SDKType}} --vv --ai $($artifact.name) --gi $($artifact.groupId)
-            }
-          displayName: 'Verify artifact versions for release'
-
-        # Uber JARs release a flattened POM, as dependencies are incorporated into the JAR circumnavigating the need to
-        # list dependencies in the POM. The flattened POM is generated and won't comply with our versioning validation,
-        # so omit uberJars from the artifacts list to verify.
-        #
-        # Note: This is different from the Batch release verification step because the ServiceDirectory, for this
-        # pipeline, is attached to each Artifact. It needs to be in the query, so it can be passed into the script.
-        - pwsh: |
-            $ArtifactsList = $('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId, ServiceDirectory, uberJar | Where-Object -Not "uberJar")
-            $ArtifactsList | Format-Table -Property GroupId, Name, ServiceDirectory | Out-String | Write-Host
-            foreach($artifact in $ArtifactsList) {
-              $(Build.SourcesDirectory)/eng/versioning/scan_for_unreleased_dependencies.ps1 $($artifact.groupId) $($artifact.name) $(Build.SourcesDirectory)/sdk/$($artifact.ServiceDirectory)
-            }
-          displayName: "Verify no unreleased dependencies or beta dependencies for GA libraries"
-
-        - template: /eng/common/pipelines/templates/steps/verify-readme.yml
-          parameters:
-            # For the moment, ServiceDirectories is JSON which may change to a comma delimited list instead of what's
-            # going on below
-            ScanPaths: $(ServiceDirectoriesFullPath)
-
-        - template: /eng/common/pipelines/templates/steps/verify-samples.yml
-          parameters:
-            ServiceDirectories: $(ServiceDirectories)
-
-        # Use BasePathLength of 38 instead of the default 49 as some released files fail when the number is higher.
-        - template: /eng/common/pipelines/templates/steps/verify-path-length.yml
-          parameters:
-            SourceDirectory: $(Build.SourcesDirectory)
-            BasePathLength: 38
-
-        # Verify changelog, the way it currently works would require a complete overhaul in order to be able to deal
-        # with multiple artifacts and multiple service directories. Even ignoring this, there's still the fact that each
-        # artifact's metadata can cause this step to be skipped, so we still need to loop over this.
-        - ${{if ne(parameters.SDKType, 'data')}}:
-          - ${{ each artifact in parameters.Artifacts }}:
-            - ${{if ne(artifact.skipVerifyChangelog, 'true')}}:
-              - template: /eng/common/pipelines/templates/steps/verify-changelog.yml
-                parameters:
-                  PackageName: ${{artifact.name}}
-                  ServiceName: ${{artifact.ServiceDirectory}}
-                  # Note: For the patch release pipeline there's no point in verifying the non-release changelog since
-                  # the only time this pipeline should run is for a patch release.
-                  ForRelease: true
-
-        - template: /eng/common/pipelines/templates/steps/eng-common-workflow-enforcer.yml
-
-  # This pipeline should never get kicked off outside an internal, manual run. Put the checks here so the pipeline never
-  # gets to the signing/release portions unless this is the case.
-  - ${{if and(in(variables['Build.Reason'], 'Manual', ''), eq(variables['System.TeamProject'], 'internal'))}}:
-    - template: archetype-java-release-patch.yml
-      parameters:
-        DependsOn:
-          - Build
-        SDKType: ${{ parameters.SDKType }}
-        Artifacts: ${{ parameters.Artifacts }}
-        TestPipeline: ${{ parameters.TestPipeline }}
-        ArtifactName: packages
-        TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
-        UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
+            TestPipeline: ${{ parameters.TestPipeline }}
+            ArtifactName: packages
+            TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
+            UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
diff --git a/eng/pipelines/templates/stages/archetype-sdk-client.yml b/eng/pipelines/templates/stages/archetype-sdk-client.yml
index 939951bd4cd63..7bb86de837a24 100644
--- a/eng/pipelines/templates/stages/archetype-sdk-client.yml
+++ b/eng/pipelines/templates/stages/archetype-sdk-client.yml
@@ -1,171 +1,178 @@
-resources:
-  repositories:
-    - repository: azure-sdk-build-tools
-      type: git
-      name: internal/azure-sdk-build-tools
-      ref: refs/tags/azure-sdk-build-tools_20230829.1
-
 parameters:
-- name: Artifacts
-  type: object
-  default: []
-- name: TestPipeline
-  type: boolean
-  default: false
-- name: AdditionalModules
-  type: object
-  default: []
-- name: SDKType
-  type: string
-  default: client
-- name: ServiceDirectory
-  type: string
-  default: not-specified
-- name: TargetDocRepoName
-  type: string
-  default: azure-docs-sdk-java
-- name: MatrixConfigs
-  type: object
-  default:
-    - Name: Java_ci_test_base
-      Path: eng/pipelines/templates/stages/platform-matrix.json
-      Selection: sparse
-      NonSparseParameters: Agent
-      GenerateVMJobs: true
-- name: AdditionalMatrixConfigs
-  type: object
-  default: []
-- name: MatrixFilters
-  type: object
-  default: []
-- name: MatrixReplace
-  type: object
-  default: []
-- name: PreBuildSteps
-  type: object
-  default: []
-- name: AdditionalLintingOptions
-  type: string
-  default: ''
-- name: AdditionalStagesAfterBuild
-  type: object
-  default: []
-- name: EnableBatchRelease
-  type: boolean
-  default: false
-- name: UseEsrpRelease
-  type: boolean
-  default: true
-- name: JavaBuildVersion
-  type: string
-  default: $(JavaBuildVersion)
-- name: JavadocSafeJavaBuildVersion
-  type: string
-  default: $(JavadocSafeJavaBuildVersion)
-- name: LiveTestStages
-  type: stageList
-  default: []
-- name: ReleaseDependsOnLiveTests
-  type: string
-  default: not-specified
-- name: IgnoreVerifyTypeSpecCodeGenerationError
-  type: boolean
-  default: false
-
-variables:
-  - template: /eng/pipelines/templates/variables/globals.yml
+  - name: Artifacts
+    type: object
+    default: []
+  - name: TestPipeline
+    type: boolean
+    default: false
+  - name: AdditionalModules
+    type: object
+    default: []
+  - name: SDKType
+    type: string
+    default: client
+  - name: ServiceDirectory
+    type: string
+    default: not-specified
+  - name: TargetDocRepoName
+    type: string
+    default: azure-docs-sdk-java
+  - name: MatrixConfigs
+    type: object
+    default:
+      - Name: Java_ci_test_base
+        Path: eng/pipelines/templates/stages/platform-matrix.json
+        Selection: sparse
+        NonSparseParameters: Agent
+        GenerateVMJobs: true
+  - name: AdditionalMatrixConfigs
+    type: object
+    default: []
+  - name: MatrixFilters
+    type: object
+    default: []
+  - name: MatrixReplace
+    type: object
+    default: []
+  - name: PreBuildSteps
+    type: object
+    default: []
+  - name: AdditionalLintingOptions
+    type: string
+    default: ''
+  - name: AdditionalStagesAfterBuild
+    type: object
+    default: []
+  - name: EnableBatchRelease
+    type: boolean
+    default: false
+  - name: UseEsrpRelease
+    type: boolean
+    default: true
+  - name: JavaBuildVersion
+    type: string
+    default: $(JavaBuildVersion)
+  - name: JavadocSafeJavaBuildVersion
+    type: string
+    default: $(JavadocSafeJavaBuildVersion)
+  - name: LiveTestStages
+    type: stageList
+    default: []
+  - name: ReleaseDependsOnLiveTests
+    type: string
+    default: not-specified
+  - name: IgnoreVerifyTypeSpecCodeGenerationError
+    type: boolean
+    default: false
+  - name: TimeoutInMinutes
+    type: number
+    default: 60
 
-stages:
-  - stage: Build
-    jobs:
-    - template: /eng/pipelines/templates/jobs/ci.yml
-      parameters:
-        ServiceDirectory: ${{ parameters.ServiceDirectory }}
-        TestPipeline: ${{ parameters.TestPipeline }}
-        SDKType: ${{ parameters.SDKType }}
-        Artifacts: ${{ parameters.Artifacts }}
-        IgnoreVerifyTypeSpecCodeGenerationError: ${{ parameters.IgnoreVerifyTypeSpecCodeGenerationError }}
-        ReleaseArtifacts:
-        - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
-          - ${{ each artifact in parameters.Artifacts }}:
-            - ${{ if ne(artifact.releaseInBatch, 'false') }}:
-              - ${{ artifact }}
-        - ${{ else }}:
-          - ${{ parameters.Artifacts }}
-        AdditionalModules: ${{ parameters.AdditionalModules }}
-        MatrixConfigs:
-          - ${{ each config in parameters.MatrixConfigs }}:
-            -  ${{ config }}
-          - ${{ each config in parameters.AdditionalMatrixConfigs }}:
-            -  ${{ config }}
-        MatrixFilters:
-          - ${{ each filter in parameters.MatrixFilters }}:
-            - ${{ filter }}
-          # Skip TestFromSource jobs for SDKType data
-          - ${{ if eq(parameters.SDKType, 'data') }}:
-            - TestFromSource=^(?!true).*
-        MatrixReplace:
-          - ${{ each replacement in parameters.MatrixReplace }}:
-            - ${{ replacement }}
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - stage: Build
+        variables:
+          - template: /eng/pipelines/templates/variables/globals.yml
+          - template: /eng/pipelines/templates/variables/image.yml
+          # Convert artifact parameter objects to json and set them as variables to be used in
+          # pipeline jobs/stages. By setting these as a variable once we save thousands of lines
+          # of yaml in extreme cases. This helps us stay under the devops max yaml size limits.
+          - name: ArtifactsJson
+            value: '${{ convertToJson(parameters.Artifacts) }}'
+          - name: AdditionalModulesJson
+            value: '${{ convertToJson(parameters.AdditionalModules) }}'
+        jobs:
+          - template: /eng/pipelines/templates/jobs/ci.yml
+            parameters:
+              ServiceDirectory: ${{ parameters.ServiceDirectory }}
+              TestPipeline: ${{ parameters.TestPipeline }}
+              SDKType: ${{ parameters.SDKType }}
+              Artifacts: ${{ parameters.Artifacts }}
+              TimeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
+              IgnoreVerifyTypeSpecCodeGenerationError: ${{ parameters.IgnoreVerifyTypeSpecCodeGenerationError }}
+              ReleaseArtifacts:
+                - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
+                    - ${{ each artifact in parameters.Artifacts }}:
+                        - ${{ if ne(artifact.releaseInBatch, 'false') }}:
+                            - ${{ artifact }}
+                - ${{ else }}:
+                  - ${{ parameters.Artifacts }}
+              MatrixConfigs:
+                - ${{ each config in parameters.MatrixConfigs }}:
+                  - ${{ config }}
+                - ${{ each config in parameters.AdditionalMatrixConfigs }}:
+                  - ${{ config }}
+              MatrixFilters:
+                - ${{ each filter in parameters.MatrixFilters }}:
+                  - ${{ filter }}
+                # Skip TestFromSource jobs for SDKType data
+                - ${{ if eq(parameters.SDKType, 'data') }}:
+                  - TestFromSource=^(?!true).*
+              MatrixReplace:
+                - ${{ each replacement in parameters.MatrixReplace }}:
+                  - ${{ replacement }}
 
-          - AZURE_TEST.*=.*/
-          - ${{ if eq(parameters.SDKType, 'data') }}:
-            - JavaTestVersion=(.*1)\.\d{2}(.*)/$1.11$2
-        PreBuildSteps: ${{ parameters.PreBuildSteps }}
-        AdditionalLintingOptions: ${{ parameters.AdditionalLintingOptions }}
-        ${{ if eq(parameters.SDKType, 'data') }}:
-          TestGoals: 'verify'
-          TestOptions: '-am'
-          JavaBuildVersion: '1.11'
-        ${{ else }}:
-          JavaBuildVersion: ${{ parameters.JavaBuildVersion }}
+                - AZURE_TEST.*=.*/
+                - ${{ if eq(parameters.SDKType, 'data') }}:
+                  - JavaTestVersion=(.*1)\.\d{2}(.*)/$1.11$2
+              PreBuildSteps: ${{ parameters.PreBuildSteps }}
+              AdditionalLintingOptions: ${{ parameters.AdditionalLintingOptions }}
+              ${{ if eq(parameters.SDKType, 'data') }}:
+                TestGoals: 'verify'
+                TestOptions: '-am'
+                JavaBuildVersion: '1.11'
+              ${{ else }}:
+                JavaBuildVersion: ${{ parameters.JavaBuildVersion }}
 
-  - ${{ parameters.AdditionalStagesAfterBuild }}
+      - ${{ if parameters.AdditionalStagesAfterBuild }}:
+          - ${{ parameters.AdditionalStagesAfterBuild }}
 
-  - ${{ if and(eq(variables['System.TeamProject'], 'internal'), ne(variables['Build.Reason'],'IndividualCI')) }}:
-    - ${{ parameters.LiveTestStages }}
+      - ${{ if and(eq(variables['System.TeamProject'], 'internal'), ne(variables['Build.Reason'],'IndividualCI')) }}:
+          - ${{ parameters.LiveTestStages }}
 
-  # The Prerelease and Release stages are conditioned on whether we are building a pull request and the branch.
-  # The tests-weekly check needs to be done so we don't create signing/release tasks for manual weekly-test runs
-  - ${{if and(ne(variables['Build.Reason'], 'PullRequest'), eq(variables['System.TeamProject'], 'internal'), not(contains(variables['Build.DefinitionName'], 'tests-weekly'))) }}:
-    - ${{ if eq(parameters.EnableBatchRelease, 'true') }}:
-      - template: archetype-java-release-batch.yml
-        parameters:
-          DependsOn:
-            - Build
-            # Only depend on live test stages if the build depends on live tests and the build reason is manual.
-            # This prevents check-in builds and scheduled builds from having a requirement on live test stages.
-            - ${{ if and(eq(parameters.ReleaseDependsOnLiveTests, 'true'), eq(variables['Build.Reason'],'Manual')) }}:
-              - ${{ each liveTestStage in parameters.LiveTestStages }}:
-                - ${{ liveTestStage.stage }}
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          SDKType: ${{ parameters.SDKType }}
-          # By default, the Artifacts list will contain everything as we won't skip an artifact unless it,
-          # specifically, has releaseInBatch set to false. releaseInBatch should only be set libraries,
-          # not ready for release, in service directories that have batch release enabled. The if statement
-          # below, looking for releaseInBatch not equal false has to be set this way because if the library's
-          # metadata in the ci.yml doesn't have this variable then it defaults to being released.
-          Artifacts:
-          - ${{ each artifact in parameters.Artifacts }}:
-            - ${{ if ne(artifact.releaseInBatch, 'false') }}:
-              - ${{ artifact }}
-          TestPipeline: ${{ parameters.TestPipeline }}
-          ArtifactName: packages
-          TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
-          UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
-          # This where the MavenBreakGlassApprovers is set for the release tasks.
-          ${{ if and(in(variables['Build.Reason'], 'Manual', ''), eq(parameters.ReleaseDependsOnLiveTests, 'false')) }}:
-            Environment: 'MavenBreakGlassApprovers'
-    - ${{ else }}:
-      - template: archetype-java-release.yml
-        parameters:
-          DependsOn:
-            - Build
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          SDKType: ${{ parameters.SDKType }}
-          Artifacts: ${{ parameters.Artifacts }}
-          TestPipeline: ${{ parameters.TestPipeline }}
-          ArtifactName: packages
-          TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
-          UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
+      # The Prerelease and Release stages are conditioned on whether we are building a pull request and the branch.
+      # The tests-weekly check needs to be done so we don't create signing/release tasks for manual weekly-test runs
+      - ${{if and(ne(variables['Build.Reason'], 'PullRequest'), eq(variables['System.TeamProject'], 'internal'), not(contains(variables['Build.DefinitionName'], 'tests-weekly'))) }}:
+          - ${{ if eq(parameters.EnableBatchRelease, 'true') }}:
+              - template: archetype-java-release-batch.yml
+                parameters:
+                  DependsOn:
+                    - Build
+                    # Only depend on live test stages if the build depends on live tests and the build reason is manual.
+                    # This prevents check-in builds and scheduled builds from having a requirement on live test stages.
+                    - ${{ if and(eq(parameters.ReleaseDependsOnLiveTests, 'true'), eq(variables['Build.Reason'],'Manual')) }}:
+                        - ${{ each liveTestStage in parameters.LiveTestStages }}:
+                            - ${{ liveTestStage.stage }}
+                  ServiceDirectory: ${{ parameters.ServiceDirectory }}
+                  SDKType: ${{ parameters.SDKType }}
+                  # By default, the Artifacts list will contain everything as we won't skip an artifact unless it,
+                  # specifically, has releaseInBatch set to false. releaseInBatch should only be set libraries,
+                  # not ready for release, in service directories that have batch release enabled. The if statement
+                  # below, looking for releaseInBatch not equal false has to be set this way because if the library's
+                  # metadata in the ci.yml doesn't have this variable then it defaults to being released.
+                  Artifacts:
+                    - ${{ each artifact in parameters.Artifacts }}:
+                        - ${{ if ne(artifact.releaseInBatch, 'false') }}:
+                            - ${{ artifact }}
+                  TestPipeline: ${{ parameters.TestPipeline }}
+                  ArtifactName: packages
+                  TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
+                  UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
+                  # This where the MavenBreakGlassApprovers is set for the release tasks.
+                  ${{ if and(in(variables['Build.Reason'], 'Manual', ''), eq(parameters.ReleaseDependsOnLiveTests, 'false')) }}:
+                    Environment: 'MavenBreakGlassApprovers'
+          - ${{ else }}:
+            - template: archetype-java-release.yml
+              parameters:
+                DependsOn:
+                  - Build
+                ServiceDirectory: ${{ parameters.ServiceDirectory }}
+                SDKType: ${{ parameters.SDKType }}
+                Artifacts: ${{ parameters.Artifacts }}
+                TestPipeline: ${{ parameters.TestPipeline }}
+                ArtifactName: packages
+                TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
+                UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
 
diff --git a/eng/pipelines/templates/stages/archetype-sdk-native-tests.yml b/eng/pipelines/templates/stages/archetype-sdk-native-tests.yml
index 20cf37fe72465..8bdc02615114d 100644
--- a/eng/pipelines/templates/stages/archetype-sdk-native-tests.yml
+++ b/eng/pipelines/templates/stages/archetype-sdk-native-tests.yml
@@ -38,9 +38,6 @@ parameters:
   - name: MaxParallel
     type: number
     default: 0
-  - name: BuildInParallel
-    type: boolean
-    default: true
   - name: TimeoutInMinutes
     type: number
     default: 60
@@ -105,14 +102,27 @@ parameters:
     type: string
     default: $(TestOptions)
 
-stages:
-  - ${{ each cloud in parameters.CloudConfig }}:
-      - ${{ if or(contains(parameters.Clouds, cloud.key), and(contains(variables['Build.DefinitionName'], 'tests-weekly'), contains(parameters.SupportedClouds, cloud.key))) }}:
-          - ${{ if not(contains(parameters.UnsupportedClouds, cloud.key)) }}:
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - ${{ each cloud in parameters.CloudConfig }}:
+        - ${{ if or(contains(parameters.Clouds, cloud.key), and(contains(variables['Build.DefinitionName'], 'tests-weekly'), contains(parameters.SupportedClouds, cloud.key))) }}:
+            - ${{ if not(contains(parameters.UnsupportedClouds, cloud.key)) }}:
               - stage: ${{ cloud.key }}_${{ parameters.TestName }}
                 dependsOn: []
+                variables:
+                  - template: /eng/pipelines/templates/variables/globals.yml
+                  - template: /eng/pipelines/templates/variables/image.yml
+                  # Convert artifact parameter objects to json and set them as variables to be used in
+                  # pipeline jobs/stages. By setting these as a variable once we save thousands of lines
+                  # of yaml in extreme cases. This helps us stay under the devops max yaml size limits.
+                  - name: ArtifactsJson
+                    value: '${{ convertToJson(parameters.Artifacts) }}'
+                  - name: AdditionalModulesJson
+                    value: '${{ convertToJson(parameters.AdditionalModules) }}'
                 jobs:
-                  - template: /eng/common/pipelines/templates/jobs/archetype-sdk-tests-generate.yml
+                  - template: /eng/common/pipelines/templates/jobs/generate-job-matrix.yml
                     parameters:
                       SparseCheckoutPaths:
                         - "sdk/${{ parameters.ServiceDirectory }}/**/*.json"
@@ -128,16 +138,12 @@ stages:
                           - ${{ parameters.PostSteps }}
                         EnvVars: ${{ parameters.EnvVars }}
                         MaxParallel: ${{ parameters.MaxParallel }}
-                        BuildInParallel: ${{ parameters.BuildInParallel }}
                         TimeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
                         Location: ${{ parameters.Location }}
                         ServiceDirectory: ${{ parameters.ServiceDirectory }}
                         TestResourceDirectories: ${{ parameters.TestResourceDirectories }}
-                        TestSetupSteps: ${{ parameters.TestSetupSteps }}
                         TestMode: ${{ parameters.TestMode }}
                         TestVersionSupport: ${{ parameters.TestVersionSupport }}
-                        Artifacts: ${{ parameters.Artifacts }}
-                        AdditionalModules: ${{ parameters.AdditionalModules }}
                         TestResultsFiles: ${{ parameters.TestResultsFiles }}
                         DisableAzureResourceCreation: ${{ parameters.DisableAzureResourceCreation }}
                         TestGoals: ${{ parameters.TestGoals }}
diff --git a/eng/pipelines/templates/stages/archetype-sdk-pom-only.yml b/eng/pipelines/templates/stages/archetype-sdk-pom-only.yml
index a130bd9ffc738..3300ecc99ed60 100644
--- a/eng/pipelines/templates/stages/archetype-sdk-pom-only.yml
+++ b/eng/pipelines/templates/stages/archetype-sdk-pom-only.yml
@@ -1,10 +1,3 @@
-resources:
-  repositories:
-    - repository: azure-sdk-build-tools
-      type: git
-      name: internal/azure-sdk-build-tools
-      ref: refs/tags/azure-sdk-build-tools_20230829.1
-
 parameters:
 - name: Artifacts
   type: object
@@ -28,38 +21,46 @@ parameters:
   type: string
   default: azure-docs-sdk-java
 
-stages:
-  - stage: Build
-    jobs:
-      - template: /eng/pipelines/templates/jobs/build-validate-pom.yml
-        parameters:
-          ServiceDirectory: ${{parameters.ServiceDirectory}}
-          SDKType: ${{parameters.SDKType}}
-          Artifacts:
-          - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
-            - ${{ each artifact in parameters.Artifacts }}:
-              - ${{ if ne(artifact.releaseInBatch, 'false') }}:
-                - ${{ artifact }}
-          - ${{ else }}:
-            - ${{ parameters.Artifacts }}
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - stage: Build
+        variables:
+          - template: /eng/pipelines/templates/variables/globals.yml
+          - template: /eng/pipelines/templates/variables/image.yml
+          - name: ArtifactsJson
+            value: '${{ convertToJson(parameters.Artifacts) }}'
+        jobs:
+          - template: /eng/pipelines/templates/jobs/build-validate-pom.yml
+            parameters:
+              ServiceDirectory: ${{parameters.ServiceDirectory}}
+              SDKType: ${{parameters.SDKType}}
+              Artifacts:
+              - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
+                - ${{ each artifact in parameters.Artifacts }}:
+                  - ${{ if ne(artifact.releaseInBatch, 'false') }}:
+                    - ${{ artifact }}
+              - ${{ else }}:
+                - ${{ parameters.Artifacts }}
 
-  # The Prerelease and Release stages are conditioned on whether we are building a pull request and the branch.
-  - ${{if and(ne(variables['Build.Reason'], 'PullRequest'), eq(variables['System.TeamProject'], 'internal'))}}:
-      - template: archetype-java-release-pom-only.yml
-        parameters:
-          DependsOn: Build
-          ServiceDirectory: ${{parameters.ServiceDirectory}}
-          RunIntegrationStage: false
-          SDKType: ${{parameters.SDKType}}
-          Artifacts:
-          - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
-            - ${{ each artifact in parameters.Artifacts }}:
-              - ${{ if ne(artifact.releaseInBatch, 'false') }}:
-                - ${{ artifact }}
-          - ${{ else }}:
-            - ${{ parameters.Artifacts }}
-          ArtifactName: packages
-          TargetDocRepoOwner: ${{parameters.TargetDocRepoOwner}}
-          TargetDocRepoName: ${{parameters.TargetDocRepoName}}
-          EnableIntegrationStage: false
+      # The Prerelease and Release stages are conditioned on whether we are building a pull request and the branch.
+      - ${{if and(ne(variables['Build.Reason'], 'PullRequest'), eq(variables['System.TeamProject'], 'internal'))}}:
+        - template: archetype-java-release-pom-only.yml
+          parameters:
+            DependsOn: Build
+            ServiceDirectory: ${{parameters.ServiceDirectory}}
+            RunIntegrationStage: false
+            SDKType: ${{parameters.SDKType}}
+            Artifacts:
+            - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
+              - ${{ each artifact in parameters.Artifacts }}:
+                - ${{ if ne(artifact.releaseInBatch, 'false') }}:
+                  - ${{ artifact }}
+            - ${{ else }}:
+              - ${{ parameters.Artifacts }}
+            ArtifactName: packages
+            TargetDocRepoOwner: ${{parameters.TargetDocRepoOwner}}
+            TargetDocRepoName: ${{parameters.TargetDocRepoName}}
+            EnableIntegrationStage: false
 
diff --git a/eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml b/eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml
new file mode 100644
index 0000000000000..bc91d23f976a9
--- /dev/null
+++ b/eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml
@@ -0,0 +1,175 @@
+parameters:
+- name: TestName
+  type: string
+  default: 'LiveTest'
+- name: AdditionalPlatforms
+  type: object
+  default: {}
+- name: Clouds
+  type: string
+  default: 'Public'
+- name: SupportedClouds
+  type: string
+  default: 'Public'
+- name: UnsupportedClouds
+  type: string
+  default: ''
+- name: Artifacts
+  type: object
+  default: []
+- name: AdditionalModules
+  type: object
+  default: []
+- name: TestResultsFiles
+  type: object
+  default: ''
+- name: DisableAzureResourceCreation
+  type: boolean
+  default: false
+- name: PreSteps
+  type: object
+  default: []
+- name: PostSteps
+  type: object
+  default: []
+- name: EnvVars
+  type: object
+  default: {}
+- name: MaxParallel
+  type: number
+  default: 0
+- name: TimeoutInMinutes
+  type: number
+  default: 60
+- name: Location
+  type: string
+  default: ''
+- name: ServiceDirectory
+  type: string
+  default: not-specified
+- name: TestResourceDirectories
+  type: object
+  default:
+- name: TestMode
+  type: string
+  default: 'LIVE'
+- name: TestVersionSupport
+  type: boolean
+  default: false
+- name: CloudConfig
+  type: object
+  default:
+    Public:
+      SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources)
+    Preview:
+      SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources-preview)
+    Canary:
+      SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources)
+      Location: 'centraluseuap'
+    UsGov:
+      SubscriptionConfiguration: $(sub-config-gov-test-resources)
+    China:
+      SubscriptionConfiguration: $(sub-config-cn-test-resources)
+- name: MatrixConfigs
+  type: object
+  default:
+    - Name: Java_live_test_base
+      Path: eng/pipelines/templates/stages/platform-matrix.json
+      Selection: sparse
+      GenerateVMJobs: true
+- name: AdditionalMatrixConfigs
+  type: object
+  default: []
+- name: MatrixFilters
+  type: object
+  default: []
+- name: MatrixReplace
+  type: object
+  default: []
+- name: PlatformPreSteps
+  type: object
+  default: []
+- name: PlatformPostSteps
+  type: object
+  default: []
+- name: TestGoals
+  type: string
+  default: $(TestGoals)
+- name: TestOptions
+  type: string
+  default: $(TestOptions)
+- name: UseHttpFaultInjector
+  type: boolean
+  default: false
+- name: CalledFromClient
+  type: boolean
+  default: false
+- name: AdditionalVariables
+  type: object
+  default: []
+
+stages:
+- ${{ each cloud in parameters.CloudConfig }}:
+  - ${{ if or(contains(parameters.Clouds, cloud.key), and(contains(variables['Build.DefinitionName'], 'tests-weekly'), contains(parameters.SupportedClouds, cloud.key))) }}:
+    - ${{ if not(contains(parameters.UnsupportedClouds, cloud.key)) }}:
+      - stage: ${{ cloud.key }}_${{ parameters.TestName }}
+        dependsOn: []
+        variables:
+          - template: /eng/pipelines/templates/variables/globals.yml
+          - template: /eng/pipelines/templates/variables/image.yml
+          # Convert artifact parameter objects to json and set them as variables to be used in
+          # pipeline jobs/stages. By setting these as a variable once we save thousands of lines
+          # of yaml in extreme cases. This helps us stay under the devops max yaml size limits.
+          - name: ArtifactsJson
+            value: '${{ convertToJson(parameters.Artifacts) }}'
+          - name: AdditionalModulesJson
+            value: '${{ convertToJson(parameters.AdditionalModules) }}'
+          - ${{ parameters.AdditionalVariables }}
+        jobs:
+        - template: /eng/common/pipelines/templates/jobs/generate-job-matrix.yml
+          parameters:
+            SparseCheckoutPaths:
+              - "sdk/${{ parameters.ServiceDirectory }}/**/*.json"
+              - '!sdk/**/test-recordings/*'
+              - '!sdk/**/session-records/*'
+            JobTemplatePath: /eng/pipelines/templates/jobs/live.tests.yml
+            AdditionalParameters:
+              PreSteps:
+                - ${{ parameters.PlatformPreSteps }}
+                - ${{ parameters.PreSteps }}
+              PostSteps:
+                - ${{ parameters.PlatformPostSteps }}
+                - ${{ parameters.PostSteps }}
+              EnvVars: ${{ parameters.EnvVars }}
+              MaxParallel: ${{ parameters.MaxParallel }}
+              TimeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
+              Location: ${{ parameters.Location }}
+              ServiceDirectory: ${{ parameters.ServiceDirectory }}
+              TestResourceDirectories: ${{ parameters.TestResourceDirectories }}
+              TestMode: ${{ parameters.TestMode }}
+              TestVersionSupport: ${{ parameters.TestVersionSupport }}
+              TestResultsFiles: ${{ parameters.TestResultsFiles }}
+              DisableAzureResourceCreation: ${{ parameters.DisableAzureResourceCreation }}
+              TestGoals: ${{ parameters.TestGoals }}
+              TestOptions: ${{ parameters.TestOptions }}
+              UseHttpFaultInjector: ${{ parameters.UseHttpFaultInjector }}
+
+            MatrixConfigs:
+              # Enumerate platforms and additional platforms based on supported clouds (sparse platform<-->cloud matrix).
+              - ${{ each config in parameters.MatrixConfigs }}:
+                -  ${{ config }}
+              - ${{ each config in parameters.AdditionalMatrixConfigs }}:
+                -  ${{ config }}
+            MatrixFilters:
+              - ${{ each cloudFilter in cloud.value.MatrixFilters }}:
+                - ${{ cloudFilter }}
+              - ${{ parameters.MatrixFilters }}
+            MatrixReplace:
+              - ${{ each cloudReplace in cloud.value.MatrixReplace }}:
+                - ${{ cloudReplace }}
+              - ${{ parameters.MatrixReplace }}
+            CloudConfig:
+              SubscriptionConfiguration: ${{ cloud.value.SubscriptionConfiguration }}
+              SubscriptionConfigurations: ${{ cloud.value.SubscriptionConfigurations }}
+              Location: ${{ coalesce(parameters.Location, cloud.value.Location) }}
+              Cloud: ${{ cloud.key }}
diff --git a/eng/pipelines/templates/stages/archetype-sdk-tests.yml b/eng/pipelines/templates/stages/archetype-sdk-tests.yml
index 987a910f77c99..dc2887900b970 100644
--- a/eng/pipelines/templates/stages/archetype-sdk-tests.yml
+++ b/eng/pipelines/templates/stages/archetype-sdk-tests.yml
@@ -1,168 +1,116 @@
 parameters:
-- name: TestName
-  type: string
-  default: 'LiveTest'
-- name: AdditionalPlatforms
-  type: object
-  default: {}
-- name: Clouds
-  type: string
-  default: 'Public'
-- name: SupportedClouds
-  type: string
-  default: 'Public'
-- name: UnsupportedClouds
-  type: string
-  default: ''
-- name: Artifacts
-  type: object
-  default: []
-- name: AdditionalModules
-  type: object
-  default: []
-- name: TestResultsFiles
-  type: object
-  default: ''
-- name: DisableAzureResourceCreation
-  type: boolean
-  default: false
-- name: PreSteps
-  type: object
-  default: []
-- name: PostSteps
-  type: object
-  default: []
-- name: EnvVars
-  type: object
-  default: {}
-- name: MaxParallel
-  type: number
-  default: 0
-- name: BuildInParallel
-  type: boolean
-  default: true
-- name: TimeoutInMinutes
-  type: number
-  default: 60
-- name: Location
-  type: string
-  default: ''
-- name: ServiceDirectory
-  type: string
-  default: not-specified
-- name: TestResourceDirectories
-  type: object
-  default:
-- name: TestSetupSteps
-  type: stepList
-  default: []
-- name: TestMode
-  type: string
-  default: 'LIVE'
-- name: TestVersionSupport
-  type: boolean
-  default: false
-- name: CloudConfig
-  type: object
-  default:
-    Public:
-      SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources)
-    Preview:
-      SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources-preview)
-    Canary:
-      SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources)
-      Location: 'centraluseuap'
-    UsGov:
-      SubscriptionConfiguration: $(sub-config-gov-test-resources)
-    China:
-      SubscriptionConfiguration: $(sub-config-cn-test-resources)
-- name: MatrixConfigs
-  type: object
-  default:
-    - Name: Java_live_test_base
-      Path: eng/pipelines/templates/stages/platform-matrix.json
-      Selection: sparse
-      GenerateVMJobs: true
-- name: AdditionalMatrixConfigs
-  type: object
-  default: []
-- name: MatrixFilters
-  type: object
-  default: []
-- name: MatrixReplace
-  type: object
-  default: []
-- name: PlatformPreSteps
-  type: object
-  default: []
-- name: PlatformPostSteps
-  type: object
-  default: []
-- name: TestGoals
-  type: string
-  default: $(TestGoals)
-- name: TestOptions
-  type: string
-  default: $(TestOptions)
-- name: UseHttpFaultInjector
-  type: boolean
-  default: false
+  - name: TestName
+    type: string
+    default: 'LiveTest'
+  - name: AdditionalPlatforms
+    type: object
+    default: {}
+  - name: Clouds
+    type: string
+    default: 'Public'
+  - name: SupportedClouds
+    type: string
+    default: 'Public'
+  - name: UnsupportedClouds
+    type: string
+    default: ''
+  - name: Artifacts
+    type: object
+    default: []
+  - name: AdditionalModules
+    type: object
+    default: []
+  - name: TestResultsFiles
+    type: object
+    default: ''
+  - name: DisableAzureResourceCreation
+    type: boolean
+    default: false
+  - name: PreSteps
+    type: object
+    default: []
+  - name: PostSteps
+    type: object
+    default: []
+  - name: EnvVars
+    type: object
+    default: {}
+  - name: MaxParallel
+    type: number
+    default: 0
+  - name: TimeoutInMinutes
+    type: number
+    default: 60
+  - name: Location
+    type: string
+    default: ''
+  - name: ServiceDirectory
+    type: string
+    default: not-specified
+  - name: TestResourceDirectories
+    type: object
+    default:
+  - name: TestMode
+    type: string
+    default: 'LIVE'
+  - name: TestVersionSupport
+    type: boolean
+    default: false
+  - name: CloudConfig
+    type: object
+    default:
+      Public:
+        SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources)
+      Preview:
+        SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources-preview)
+      Canary:
+        SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources)
+        Location: 'centraluseuap'
+      UsGov:
+        SubscriptionConfiguration: $(sub-config-gov-test-resources)
+      China:
+        SubscriptionConfiguration: $(sub-config-cn-test-resources)
+  - name: MatrixConfigs
+    type: object
+    default:
+      - Name: Java_live_test_base
+        Path: eng/pipelines/templates/stages/platform-matrix.json
+        Selection: sparse
+        GenerateVMJobs: true
+  - name: AdditionalMatrixConfigs
+    type: object
+    default: []
+  - name: MatrixFilters
+    type: object
+    default: []
+  - name: MatrixReplace
+    type: object
+    default: []
+  - name: PlatformPreSteps
+    type: object
+    default: []
+  - name: PlatformPostSteps
+    type: object
+    default: []
+  - name: TestGoals
+    type: string
+    default: $(TestGoals)
+  - name: TestOptions
+    type: string
+    default: $(TestOptions)
+  - name: UseHttpFaultInjector
+    type: boolean
+    default: false
+  - name: CalledFromClient
+    type: boolean
+    default: false
+  - name: AdditionalVariables
+    type: object
+    default: []
 
-stages:
-- ${{ each cloud in parameters.CloudConfig }}:
-  - ${{ if or(contains(parameters.Clouds, cloud.key), and(contains(variables['Build.DefinitionName'], 'tests-weekly'), contains(parameters.SupportedClouds, cloud.key))) }}:
-    - ${{ if not(contains(parameters.UnsupportedClouds, cloud.key)) }}:
-      - stage: ${{ cloud.key }}_${{ parameters.TestName }}
-        dependsOn: []
-        jobs:
-        - template: /eng/common/pipelines/templates/jobs/archetype-sdk-tests-generate.yml
-          parameters:
-            SparseCheckoutPaths:
-              - "sdk/${{ parameters.ServiceDirectory }}/**/*.json"
-              - '!sdk/**/test-recordings/*'
-              - '!sdk/**/session-records/*'
-            JobTemplatePath: /eng/pipelines/templates/jobs/live.tests.yml
-            AdditionalParameters:
-              PreSteps:
-                - ${{ parameters.PlatformPreSteps }}
-                - ${{ parameters.PreSteps }}
-              PostSteps:
-                - ${{ parameters.PlatformPostSteps }}
-                - ${{ parameters.PostSteps }}
-              EnvVars: ${{ parameters.EnvVars }}
-              MaxParallel: ${{ parameters.MaxParallel }}
-              BuildInParallel: ${{ parameters.BuildInParallel }}
-              TimeoutInMinutes: ${{ parameters.TimeoutInMinutes }}
-              Location: ${{ parameters.Location }}
-              ServiceDirectory: ${{ parameters.ServiceDirectory }}
-              TestResourceDirectories: ${{ parameters.TestResourceDirectories }}
-              TestSetupSteps: ${{ parameters.TestSetupSteps }}
-              TestMode: ${{ parameters.TestMode }}
-              TestVersionSupport: ${{ parameters.TestVersionSupport }}
-              Artifacts: ${{ parameters.Artifacts }}
-              AdditionalModules: ${{ parameters.AdditionalModules }}
-              TestResultsFiles: ${{ parameters.TestResultsFiles }}
-              DisableAzureResourceCreation: ${{ parameters.DisableAzureResourceCreation }}
-              TestGoals: ${{ parameters.TestGoals }}
-              TestOptions: ${{ parameters.TestOptions }}
-              UseHttpFaultInjector: ${{ parameters.UseHttpFaultInjector }}
-
-            MatrixConfigs:
-              # Enumerate platforms and additional platforms based on supported clouds (sparse platform<-->cloud matrix).
-              - ${{ each config in parameters.MatrixConfigs }}:
-                -  ${{ config }}
-              - ${{ each config in parameters.AdditionalMatrixConfigs }}:
-                -  ${{ config }}
-            MatrixFilters:
-              - ${{ each cloudFilter in cloud.value.MatrixFilters }}:
-                - ${{ cloudFilter }}
-              - ${{ parameters.MatrixFilters }}
-            MatrixReplace:
-              - ${{ each cloudReplace in cloud.value.MatrixReplace }}:
-                - ${{ cloudReplace }}
-              - ${{ parameters.MatrixReplace }}
-            CloudConfig:
-              SubscriptionConfiguration: ${{ cloud.value.SubscriptionConfiguration }}
-              SubscriptionConfigurations: ${{ cloud.value.SubscriptionConfigurations }}
-              Location: ${{ coalesce(parameters.Location, cloud.value.Location) }}
-              Cloud: ${{ cloud.key }}
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - template: /eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml
+        parameters: ${{ parameters }}
diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json
index aa6190a49d3c4..6b4b78e3f8bc7 100644
--- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json
+++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json
@@ -1,7 +1,7 @@
 {
   "matrix": {
     "Agent": {
-      "windows-2022": { "OSVmImage": "windows-2022", "Pool": "azsdk-pool-mms-win-2022-general" }
+      "windows-2022": { "OSVmImage": "env:WINDOWSVMIMAGE", "Pool": "env:WINDOWSPOOL" }
     },
     "EmulatorConfig": {
       "Emulator Only Integration Tests Tcp - Java 8": {
diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml
index 4017cab872364..bd296d74e53b5 100644
--- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml
+++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml
@@ -1,10 +1,3 @@
-resources:
-  repositories:
-    - repository: azure-sdk-build-tools
-      type: git
-      name: internal/azure-sdk-build-tools
-      ref: refs/tags/azure-sdk-build-tools_20230829.1
-
 parameters:
 - name: Artifacts
   type: object
@@ -34,129 +27,141 @@ parameters:
   type: object
   default: []
 
-variables:
-  - template: /eng/pipelines/templates/variables/globals.yml
+extends:
+  template: /eng/pipelines/templates/stages/1es-redirect.yml
+  parameters:
+    stages:
+      - stage: Build
+        variables:
+          - template: /eng/pipelines/templates/variables/globals.yml
+          - template: /eng/pipelines/templates/variables/image.yml
+          # Convert artifact parameter objects to json and set them as variables to be used in
+          # pipeline jobs/stages. By setting these as a variable once we save thousands of lines
+          # of yaml in extreme cases. This helps us stay under the devops max yaml size limits.
+          - name: ArtifactsJson
+            value: '${{ convertToJson(parameters.Artifacts) }}'
+          - name: AdditionalModulesJson
+            value: '${{ convertToJson(parameters.AdditionalModules) }}'
+        jobs:
+        - template: /eng/pipelines/templates/jobs/ci.yml
+          parameters:
+            JavaBuildVersion: '1.17'
+            ServiceDirectory: ${{parameters.ServiceDirectory}}
+            Artifacts: ${{ parameters.Artifacts }}
+            ReleaseArtifacts:
+            - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
+              - ${{ each artifact in parameters.Artifacts }}:
+                - ${{ if ne(artifact.releaseInBatch, 'false') }}:
+                  - ${{ artifact }}
+            - ${{ else }}:
+              - ${{ parameters.Artifacts }}
+            SDKType: ${{parameters.SDKType}}
+            MatrixConfigs:
+              - Name: Cosmos_ci_test
+                Path: eng/pipelines/templates/stages/platform-matrix.json
+                Selection: sparse
+                NonSparseParameters: Agent
+                GenerateVMJobs: true
+            MatrixFilters:
+              - TestFromSource=^$|false
+              - ${{ parameters.MatrixFilters }}
+            MatrixReplace:
+              - AZURE_TEST.*=.*/
+              - .*Version=1.21/1.17
+            BuildParallelization: 1
+            TestOptions: '-Punit'
 
-stages:
-  - stage: Build
-    jobs:
-    - template: /eng/pipelines/templates/jobs/ci.yml
-      parameters:
-        JavaBuildVersion: '1.17'
-        ServiceDirectory: ${{parameters.ServiceDirectory}}
-        Artifacts: ${{ parameters.Artifacts }}
-        ReleaseArtifacts:
-        - ${{ if and(eq(variables['System.TeamProject'], 'internal'), eq(variables['Build.Reason'], 'Manual')) }}:
-          - ${{ each artifact in parameters.Artifacts }}:
-            - ${{ if ne(artifact.releaseInBatch, 'false') }}:
-              - ${{ artifact }}
-        - ${{ else }}:
-          - ${{ parameters.Artifacts }}
-        AdditionalModules: ${{ parameters.AdditionalModules }}
-        SDKType: ${{parameters.SDKType}}
-        MatrixConfigs:
-          - Name: Cosmos_ci_test
-            Path: eng/pipelines/templates/stages/platform-matrix.json
-            Selection: sparse
-            NonSparseParameters: Agent
-            GenerateVMJobs: true
-        MatrixFilters:
-          - TestFromSource=^$|false
-          - ${{ parameters.MatrixFilters }}
-        MatrixReplace:
-          - AZURE_TEST.*=.*/
-          - .*Version=1.21/1.17
-        BuildParallelization: 1
-        TestOptions: '-Punit'
+      - stage: TestEmulator
+        displayName: Test Emulator
+        dependsOn: []
+        variables:
+          - template: /eng/pipelines/templates/variables/globals.yml
+          - template: /eng/pipelines/templates/variables/image.yml
+          - name: ArtifactsJson
+            value: '${{ convertToJson(parameters.Artifacts) }}'
+        jobs:
+        - template: /eng/common/pipelines/templates/jobs/generate-job-matrix.yml
+          parameters:
+            JobTemplatePath: /eng/pipelines/templates/jobs/live.tests.yml
+            MatrixConfigs:
+              - Name: Cosmos_emulator_integration
+                Path: eng/pipelines/templates/stages/cosmos-emulator-matrix.json
+                Selection: all
+                GenerateVMJobs: true
+            MatrixFilters:
+              - ${{ parameters.LanguageFilter }}
+              - ${{ parameters.MatrixFilters }}
+            AdditionalParameters:
+              BuildParallelization: 1
+              DisableAzureResourceCreation: true
+              ServiceDirectory: cosmos
+              ACCOUNT_HOST: 'https://localhost:8081/'
+              ACCOUNT_KEY: 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=='
+              SECONDARY_ACCOUNT_KEY: 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=='
+              # Increased timeout to 90 because of cosmos emulator taking 25-30 mins to download emulator
+              # Issue filed to improve download speed: https://github.com/Azure/azure-sdk-for-java/issues/12970
+              TimeoutInMinutes: 90
+              TestGoals: 'clean verify'
+              TestOptions: '$(ProfileFlag) $(AdditionalArgs)'
 
-  - stage: TestEmulator
-    displayName: Test Emulator
-    dependsOn: []
-    jobs:
-    - template: /eng/common/pipelines/templates/jobs/archetype-sdk-tests-generate.yml
-      parameters:
-        JobTemplatePath: /eng/pipelines/templates/jobs/live.tests.yml
-        MatrixConfigs:
-          - Name: Cosmos_emulator_integration
-            Path: eng/pipelines/templates/stages/cosmos-emulator-matrix.json
-            Selection: all
-            GenerateVMJobs: true
-        MatrixFilters:
-          - ${{ parameters.LanguageFilter }}
-          - ${{ parameters.MatrixFilters }}
-        AdditionalParameters:
-          BuildParallelization: 1
-          DisableAzureResourceCreation: true
-          ServiceDirectory: cosmos
-          Artifacts: ${{ parameters.Artifacts }}
-          AdditionalModules: ${{ parameters.AdditionalModules }}
-          ACCOUNT_HOST: 'https://localhost:8081/'
-          ACCOUNT_KEY: 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=='
-          SECONDARY_ACCOUNT_KEY: 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=='
-          # Increased timeout to 90 because of cosmos emulator taking 25-30 mins to download emulator
-          # Issue filed to improve download speed: https://github.com/Azure/azure-sdk-for-java/issues/12970
-          TimeoutInMinutes: 90
-          TestGoals: 'clean verify'
-          TestOptions: '$(ProfileFlag) $(AdditionalArgs)'
+              PreSteps:
+              - template: /eng/common/pipelines/templates/steps/cosmos-emulator.yml
+                parameters:
+                  StartParameters: '/noexplorer /noui /enablepreview /EnableSqlComputeEndpoint /SqlComputePort=9999 /disableratelimiting /partitioncount=50 /consistency=Strong'
+              - powershell: |
+                  $Key = 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=='
+                  $password = ConvertTo-SecureString -String $Key -Force -AsPlainText
+                  $cert = Get-ChildItem cert:\LocalMachine\My | Where-Object { $_.FriendlyName -eq "DocumentDbEmulatorCertificate" }
+                  Export-PfxCertificate -Cert $cert -FilePath ".\CosmosDbEmulatorCert.pfx" -Password $password | Out-Null
+                  $cert = New-Object System.Security.Cryptography.X509Certificates.X509Certificate2
+                  $cert.Import(".\CosmosDbEmulatorCert.pfx", $Key, "DefaultKeySet")
+                  $cert | Export-Certificate -FilePath "$env:temp\CosmosDbEmulatorCert.cer" -Type CERT
+                displayName: 'Export Cosmos DB Emulator Certificate'
+              - powershell: |
+                  if (Test-Path $env:JAVA_HOME\jre\lib\security) {
+                    cd $env:JAVA_HOME\jre\lib\security
+                  } elseif (Test-Path $env:JAVA_HOME\lib\security) {
+                    cd $env:JAVA_HOME\lib\security
+                  } else {
+                    Write-Error "JDK directory structure is unknown and unsupported. JAVA_HOME is '$env:JAVA_HOME'"
+                    exit 1
+                  }
+                  cp $env:temp\CosmosDbEmulatorCert.cer .
+                  keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias CosmosDbEmulatorCert -file CosmosDbEmulatorCert.cer -storepass changeit
+                displayName: 'Create Java TrustStore'
 
-          PreSteps:
-          - template: /eng/common/pipelines/templates/steps/cosmos-emulator.yml
+      # The Prerelease and Release stages are conditioned on whether we are building a pull request and the branch.
+      - ${{if and(ne(variables['Build.Reason'], 'PullRequest'), eq(variables['System.TeamProject'], 'internal'))}}:
+        - ${{ if eq(parameters.EnableBatchRelease, 'true') }}:
+          - template: archetype-java-release-batch.yml
             parameters:
-              StartParameters: '/noexplorer /noui /enablepreview /EnableSqlComputeEndpoint /SqlComputePort=9999 /disableratelimiting /partitioncount=50 /consistency=Strong'
-          - powershell: |
-              $Key = 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw=='
-              $password = ConvertTo-SecureString -String $Key -Force -AsPlainText
-              $cert = Get-ChildItem cert:\LocalMachine\My | Where-Object { $_.FriendlyName -eq "DocumentDbEmulatorCertificate" }
-              Export-PfxCertificate -Cert $cert -FilePath ".\CosmosDbEmulatorCert.pfx" -Password $password | Out-Null
-              $cert = New-Object System.Security.Cryptography.X509Certificates.X509Certificate2
-              $cert.Import(".\CosmosDbEmulatorCert.pfx", $Key, "DefaultKeySet")
-              $cert | Export-Certificate -FilePath "$env:temp\CosmosDbEmulatorCert.cer" -Type CERT
-            displayName: 'Export Cosmos DB Emulator Certificate'
-          - powershell: |
-              if (Test-Path $env:JAVA_HOME\jre\lib\security) {
-                cd $env:JAVA_HOME\jre\lib\security
-              } elseif (Test-Path $env:JAVA_HOME\lib\security) {
-                cd $env:JAVA_HOME\lib\security
-              } else {
-                Write-Error "JDK directory structure is unknown and unsupported. JAVA_HOME is '$env:JAVA_HOME'"
-                exit 1
-              }
-              cp $env:temp\CosmosDbEmulatorCert.cer .
-              keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias CosmosDbEmulatorCert -file CosmosDbEmulatorCert.cer -storepass changeit
-            displayName: 'Create Java TrustStore'
-
-  # The Prerelease and Release stages are conditioned on whether we are building a pull request and the branch.
-  - ${{if and(ne(variables['Build.Reason'], 'PullRequest'), eq(variables['System.TeamProject'], 'internal'))}}:
-    - ${{ if eq(parameters.EnableBatchRelease, 'true') }}:
-      - template: archetype-java-release-batch.yml
-        parameters:
-          DependsOn:
-            - Build
-            - TestEmulator
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          SDKType: ${{ parameters.SDKType }}
-          # By default, the Artifacts list will contain everything as we won't skip an artifact unless it,
-          # specifically, has releaseInBatch set to false. releaseInBatch should only be set libraries,
-          # not ready for release, in service directories that have batch release enabled. The if statement
-          # below, looking for releaseInBatch not equal false has to be set this way because if the library's
-          # metadata in the ci.yml doesn't have this variable then it defaults to being released.
-          Artifacts:
-            - ${{ each artifact in parameters.Artifacts }}:
-              - ${{ if ne(artifact.releaseInBatch, 'false') }}:
-                - ${{ artifact }}
-          TestPipeline: false
-          ArtifactName: packages
-          TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
-          UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
-    - ${{ else }}:
-      - template: archetype-java-release.yml
-        parameters:
-          DependsOn:
-            - Build
-            - TestEmulator
-          ServiceDirectory: ${{ parameters.ServiceDirectory }}
-          SDKType: ${{ parameters.SDKType }}
-          Artifacts: ${{ parameters.Artifacts }}
-          ArtifactName: packages
-          TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
-          UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
+              DependsOn:
+                - Build
+                - TestEmulator
+              ServiceDirectory: ${{ parameters.ServiceDirectory }}
+              SDKType: ${{ parameters.SDKType }}
+              # By default, the Artifacts list will contain everything as we won't skip an artifact unless it,
+              # specifically, has releaseInBatch set to false. releaseInBatch should only be set libraries,
+              # not ready for release, in service directories that have batch release enabled. The if statement
+              # below, looking for releaseInBatch not equal false has to be set this way because if the library's
+              # metadata in the ci.yml doesn't have this variable then it defaults to being released.
+              Artifacts:
+                - ${{ each artifact in parameters.Artifacts }}:
+                  - ${{ if ne(artifact.releaseInBatch, 'false') }}:
+                    - ${{ artifact }}
+              TestPipeline: false
+              ArtifactName: packages
+              TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
+              UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
+        - ${{ else }}:
+          - template: archetype-java-release.yml
+            parameters:
+              DependsOn:
+                - Build
+                - TestEmulator
+              ServiceDirectory: ${{ parameters.ServiceDirectory }}
+              SDKType: ${{ parameters.SDKType }}
+              Artifacts: ${{ parameters.Artifacts }}
+              ArtifactName: packages
+              TargetDocRepoName: ${{ parameters.TargetDocRepoName }}
+              UseEsrpRelease: ${{ parameters.UseEsrpRelease }}
diff --git a/eng/pipelines/templates/stages/native-platform-matrix.json b/eng/pipelines/templates/stages/native-platform-matrix.json
index e6a62f87079ce..4752c7e38a5fd 100644
--- a/eng/pipelines/templates/stages/native-platform-matrix.json
+++ b/eng/pipelines/templates/stages/native-platform-matrix.json
@@ -4,7 +4,7 @@
   },
   "matrix": {
     "Agent": {
-      "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" }
+      "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" }
     },
     "JavaTestVersion": [ "1.11" ],
     "AZURE_TEST_HTTP_CLIENTS": [ "netty" ],
@@ -16,7 +16,7 @@
   "include": [
     {
       "Agent": {
-        "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" }
+        "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" }
       },
       "JavaTestVersion": "1.11",
       "AZURE_TEST_HTTP_CLIENTS": "netty",
diff --git a/eng/pipelines/templates/stages/platform-matrix.json b/eng/pipelines/templates/stages/platform-matrix.json
index 3a59e1c2bf562..672b982e45c24 100644
--- a/eng/pipelines/templates/stages/platform-matrix.json
+++ b/eng/pipelines/templates/stages/platform-matrix.json
@@ -4,9 +4,9 @@
   },
   "matrix": {
     "Agent": {
-      "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" },
-      "windows-2022": { "OSVmImage": "windows-2022", "Pool": "azsdk-pool-mms-win-2022-general" },
-      "macos-11": { "OSVmImage": "macos-11", "Pool": "Azure Pipelines" }
+      "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" },
+      "windows-2022": { "OSVmImage": "env:WINDOWSVMIMAGE", "Pool": "env:WINDOWSPOOL" },
+      "macos-11": { "OSVmImage": "env:MACVMIMAGE", "Pool": "env:MACPOOL" }
     },
     "JavaTestVersion": [ "1.8", "1.21" ],
     "AZURE_TEST_HTTP_CLIENTS": [ "okhttp", "netty" ],
@@ -17,14 +17,14 @@
   },
   "exclude": [
     {
-      "Pool": "azsdk-pool-mms-win-2022-general",
+      "Pool": "env:WINDOWSPOOL",
       "JavaTestVersion": "1.21"
     }
   ],
   "include": [
     {
       "Agent": {
-          "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" }
+          "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" }
       },
       "JavaTestVersion": "1.21",
       "AZURE_TEST_HTTP_CLIENTS": "netty",
@@ -35,7 +35,7 @@
     },
     {
       "Agent": {
-        "windows-2022": { "OSVmImage": "windows-2022", "Pool": "azsdk-pool-mms-win-2022-general" }
+        "windows-2022": { "OSVmImage": "env:WINDOWSVMIMAGE", "Pool": "env:WINDOWSPOOL" }
       },
       "JavaTestVersion": "1.21",
       "AZURE_TEST_HTTP_CLIENTS": "netty",
@@ -46,7 +46,7 @@
     },
     {
       "Agent": {
-        "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" }
+        "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" }
       },
       "JavaTestVersion": "1.11",
       "AZURE_TEST_HTTP_CLIENTS": "netty",
@@ -57,7 +57,7 @@
     },
     {
       "Agent": {
-        "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" }
+        "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" }
       },
       "JavaTestVersion": "1.17",
       "AZURE_TEST_HTTP_CLIENTS": "netty",
diff --git a/eng/pipelines/templates/steps/build-and-test-native.yml b/eng/pipelines/templates/steps/build-and-test-native.yml
index 88f0a8f8414b8..cc744ec479c0c 100644
--- a/eng/pipelines/templates/steps/build-and-test-native.yml
+++ b/eng/pipelines/templates/steps/build-and-test-native.yml
@@ -1,10 +1,4 @@
 parameters:
-  - name: SDKType
-    type: string
-    default: client
-  - name: ServiceDirectory
-    type: string
-    default: 'not-specified' # Set a default that breaks in obvious ways.
   - name: BuildParallelization
     type: string
     default: '2C'
@@ -124,8 +118,6 @@ steps:
     env: ${{ parameters.TestEnvVars }}
     condition: and(succeeded(), eq(variables['TestFromSource'], 'true'))
 
-  - template: /eng/pipelines/templates/steps/upload-repository-on-failure.yml
-
   - template: /eng/pipelines/templates/steps/retain-test-logs.yml
 
   - template: /eng/pipelines/templates/steps/retain-heap-dump-hprofs.yml
diff --git a/eng/pipelines/templates/steps/build-and-test.yml b/eng/pipelines/templates/steps/build-and-test.yml
index 8556d492ac6d4..b17275dbd8991 100644
--- a/eng/pipelines/templates/steps/build-and-test.yml
+++ b/eng/pipelines/templates/steps/build-and-test.yml
@@ -1,10 +1,4 @@
 parameters:
-  - name: SDKType
-    type: string
-    default: client
-  - name: ServiceDirectory
-    type: string
-    default: 'not-specified' # Set a default that breaks in obvious ways.
   - name: BuildParallelization
     type: string
     default: '2C'
diff --git a/eng/pipelines/templates/steps/upload-repository-on-failure.yml b/eng/pipelines/templates/steps/clean-maven-local-cache.yml
similarity index 51%
rename from eng/pipelines/templates/steps/upload-repository-on-failure.yml
rename to eng/pipelines/templates/steps/clean-maven-local-cache.yml
index 825f85f45dc3c..ed5702c22c98e 100644
--- a/eng/pipelines/templates/steps/upload-repository-on-failure.yml
+++ b/eng/pipelines/templates/steps/clean-maven-local-cache.yml
@@ -1,9 +1,4 @@
 steps:
-  - publish: $(System.DefaultWorkingDirectory)
-    condition: or(eq(variables['CaptureRepositoryOnFailure'], 'true'), and(failed(), eq(variables['Build.Reason'],'Schedule')))
-    displayName: 'Capture repo state for analysis'
-    artifact: repository-$(System.JobName)
-
   - task: PowerShell@2
     displayName: Clean up maven local cache
     inputs:
diff --git a/eng/pipelines/templates/steps/download-credscan-suppressions.yml b/eng/pipelines/templates/steps/download-credscan-suppressions.yml
new file mode 100644
index 0000000000000..bb56a50e30f4b
--- /dev/null
+++ b/eng/pipelines/templates/steps/download-credscan-suppressions.yml
@@ -0,0 +1,14 @@
+# 1es PT credscan step will fail if the suppression file path we use in other stages does not exist here.
+# Checking out the repo via sparse checkout changes the devops path structure so that can't be used instead.
+steps:
+  - pwsh: |
+      $ErrorActionPreference = 'Stop'
+      New-Item -ItemType Directory -Force eng
+
+      $req = Invoke-WebRequest https://raw.githubusercontent.com/Azure/azure-sdk-for-java/main/eng/CredScanSuppression.json
+      $req.RawContent | Out-File eng/CredScanSuppression.json
+
+      $req2 = Invoke-WebRequest https://raw.githubusercontent.com/Azure/azure-sdk-for-java/main/eng/java.gdnbaselines
+      $req2.RawContent | Out-File eng/java.gdnbaselines
+    workingDirectory: $(Build.SourcesDirectory)
+    displayName: Set credscan suppression/baseline file
diff --git a/eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml b/eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
index 57cbbc261b0b1..c9eaf3ca6c87d 100644
--- a/eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
+++ b/eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
@@ -9,12 +9,6 @@
 
 # The project list is also being used as a cache key.
 parameters:
-  - name: Artifacts
-    type: object
-    default: []
-  - name: AdditionalModules
-    type: object
-    default: []
   - name: JobType
     type: string
     default: test
@@ -27,28 +21,14 @@ steps:
   # The ProjectList is used by subsequent steps and tasks to run CI.
   # The ProjectListSha256 is used by the cache key, previously the cache key was using ProjectList but it was found
   # that long cache keys cause caching to fail.
-- pwsh: |
-    $artifacts = '${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json
-    $additionalModules = '${{ convertToJson(parameters.AdditionalModules) }}' | ConvertFrom-Json
-
-    $projectList = @()
-    foreach ($artifact in $artifacts) {
-      $projectList += "$($artifact.groupId):$($artifact.name)"
-    }
-    foreach ($artifact in $additionalModules) {
-      $projectList += "$($artifact.groupId):$($artifact.name)"
-    }
-    $projects = $projectList -join ','
-
-    Write-Host "ProjectList = $projects"
-    Write-Host "##vso[task.setvariable variable=ProjectList;]$projects"
-
-    $sha256 = new-object -TypeName System.Security.Cryptography.SHA256Managed
-    $utf8 = new-object -TypeName System.Text.UTF8Encoding
-
-    $projectListSha256 = [Convert]::ToBase64String($sha256.ComputeHash($utf8.GetBytes($projects)))
-    Write-Host "##vso[task.setvariable variable=ProjectListSha256;]$projectListSha256"
+- task: PowerShell@2
   displayName: Initialize project list variable
+  inputs:
+    pwsh: true
+    filePath: eng/pipelines/scripts/generate-project-list.ps1
+  env:
+    ARTIFACTSJSON: $(ArtifactsJson)
+    ADDITIONALMODULESJSON: $(AdditionalModulesJson)
 
 # The Cache task, when restoring, can have a miss and still pass but, if nothing is built, the
 # Post-Job Cache will fail because there's no .m2/repository
diff --git a/eng/pipelines/templates/steps/http-fault-injector.yml b/eng/pipelines/templates/steps/http-fault-injector.yml
index b6f351b3c90ca..2886b3cbf78a8 100644
--- a/eng/pipelines/templates/steps/http-fault-injector.yml
+++ b/eng/pipelines/templates/steps/http-fault-injector.yml
@@ -1,45 +1,10 @@
 steps:
-  - powershell: |
-      dotnet tool install azure.sdk.tools.httpfaultinjector --global --prerelease --add-source https://pkgs.dev.azure.com/azure-sdk/public/_packaging/azure-sdk-for-net/nuget/v3/index.json
-      dotnet tool update azure.sdk.tools.httpfaultinjector --global --prerelease --add-source https://pkgs.dev.azure.com/azure-sdk/public/_packaging/azure-sdk-for-net/nuget/v3/index.json
-      http-fault-injector --version
-    displayName: 'Install http-fault-injector'
-
-  - pwsh: |
-      Write-Host "##vso[task.setvariable variable=ASPNETCORE_Kestrel__Certificates__Default__Path]$(Build.SourcesDirectory)/eng/common/testproxy/dotnet-devcert.pfx"
-      Write-Host "##vso[task.setvariable variable=ASPNETCORE_Kestrel__Certificates__Default__Password]password"
-      Write-Host "##vso[task.setvariable variable=PROXY_MANUAL_START]true"
-    displayName: 'Configure Kestrel and PROXY_MANUAL_START Variables'
-
-  - powershell: |
-      if (Test-Path $env:JAVA_HOME\jre\lib\security) {
-        cd $env:JAVA_HOME\jre\lib\security
-      } elseif (Test-Path $env:JAVA_HOME\lib\security) {
-        cd $env:JAVA_HOME\lib\security
-      } else {
-        Write-Error "JDK directory structure is unknown and unsupported. JAVA_HOME is '$env:JAVA_HOME'"
-        exit 1
-      }
-
-      dotnet dev-certs https --export-path http-fault-injector.pfx
-      keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias HttpFaultInject -file http-fault-injector.pfx -storepass changeit
-    displayName: 'Trust http-fault-injector self-signed certificate - windows'
-    condition: and(succeeded(), eq(variables['Agent.OS'],'Windows_NT'))
-
-  - bash: |
-      if [ -d ${JAVA_HOME}/jre/lib/security ]; then
-        cd ${JAVA_HOME}/jre/lib/security
-      elif [ -d ${JAVA_HOME}/lib/security ]; then
-        cd ${JAVA_HOME}/lib/security
-      else
-        echo "JDK directory structure is unknown and unsupported. JAVA_HOME is '${JAVA_HOME}'"
-        exit 1
-      fi
-
-      dotnet dev-certs https --export-path http-fault-injector.pfx
-      sudo keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias HttpFaultInject -file http-fault-injector.pfx -storepass changeit
-    displayName: 'Trust http-fault-injector self-signed certificate - linux/mac'
-    condition: and(succeeded(), ne(variables['Agent.OS'],'Windows_NT'))
+  - task: PowerShell@2
+    displayName: Setup http fault injector
+    inputs:
+      pwsh: true
+      filePath: $(Build.SourcesDirectory)/eng/scripts/setup-http-fault-injector.ps1
+      arguments: -SourcesDirectory $(Build.SourcesDirectory)
 
   - pwsh: |
       Start-Process http-fault-injector -NoNewWindow -PassThru -RedirectStandardOutput "$(Build.SourcesDirectory)/http-fault-injector.log"
diff --git a/eng/pipelines/templates/steps/initialize-test-environment.yml b/eng/pipelines/templates/steps/initialize-test-environment.yml
index d1b89c2400d3c..62a437b7f87b0 100644
--- a/eng/pipelines/templates/steps/initialize-test-environment.yml
+++ b/eng/pipelines/templates/steps/initialize-test-environment.yml
@@ -1,19 +1,8 @@
 parameters:
-  - name: ServiceDirectory
-    type: string
-  - name: Artifacts
-    type: object
-    default: []
-  - name: AdditionalModules
-    type: object
-    default: []
   - name: CheckoutRecordings
     type: boolean
     default: false
-  - name: SDKType
-    type: string
-    default: client
-  - name: OSVmImage
+  - name: OS
     type: string
     default: $(OSVmImage)
 
@@ -36,7 +25,7 @@ steps:
 
   - template: /eng/common/pipelines/templates/steps/verify-agent-os.yml
     parameters:
-      AgentImage: ${{ parameters.OSVmImage }}
+      AgentImage: ${{ parameters.OS }}
 
   - task: PythonScript@0
     displayName: 'Set versions for source build'
@@ -54,8 +43,6 @@ steps:
 
   - template: /eng/pipelines/templates/steps/generate-project-list-and-cache-maven-repository.yml
     parameters:
-      Artifacts: ${{ parameters.Artifacts }}
-      AdditionalModules: ${{ parameters.AdditionalModules }}
       JobType: 'test'
 
   # This call is used generate the sparse checkout variables for non-FromSource runs.
@@ -77,14 +64,14 @@ steps:
     displayName: 'Generate FromSource POM and directories for sparse checkout'
     inputs:
       scriptPath: 'eng/scripts/generate_from_source_pom.py'
-      arguments: '--project-list $(ProjectList)'
+      arguments: '--artifacts-list $(ArtifactsList) --additional-modules-list $(AdditionalModulesList)'
       workingDirectory: '$(System.DefaultWorkingDirectory)'
     condition: and(succeeded(), eq(variables['TestFromSource'], 'true'))
 
   # Skip sparse checkout for the `azure-sdk-for--pr` private mirrored repositories
   # as we require the GitHub service connection to be loaded.
   - ${{ if not(contains(variables['Build.DefinitionName'], 'java-pr')) }}:
-    - template: /eng/common/pipelines/templates/steps/sparse-checkout.yml
+    - template: /eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
       parameters:
         SkipCheckoutNone: true
         Paths: $(SparseCheckoutDirectories)
diff --git a/eng/pipelines/templates/steps/publish-http-fault-injector-logs.yml b/eng/pipelines/templates/steps/publish-http-fault-injector-logs.yml
index 18ae4af62975e..a7ecdaa7a7eee 100644
--- a/eng/pipelines/templates/steps/publish-http-fault-injector-logs.yml
+++ b/eng/pipelines/templates/steps/publish-http-fault-injector-logs.yml
@@ -1,24 +1,11 @@
 steps:
   - pwsh: |
       if (Test-Path -Path "$(Build.SourcesDirectory)/http-fault-injector.log") {
-        Write-Host "##vso[task.setvariable variable=HAS_HTTP_FAULT_INJECTOR_LOGS]true"
-      } else {
-        Write-Host "##vso[task.setvariable variable=HAS_HTTP_FAULT_INJECTOR_LOGS]false"
+        if (-not (Test-Path "$(System.DefaultWorkingDirectory)/troubleshooting")) {
+          New-Item -ItemType Directory -Path "$(System.DefaultWorkingDirectory)/troubleshooting" | Out-Null
+        }
+        Compress-Archive -Path "$(Build.SourcesDirectory)/http-fault-injector.log" -DestinationPath "$(System.DefaultWorkingDirectory)/troubleshooting/fault-$(System.JobName).zip"
+        Write-Host "##vso[task.setvariable variable=HAS_TROUBLESHOOTING]true"
       }
-    displayName: 'Check if http-fault-injector log file exists'
+    displayName: 'Check for and copy http-fault-injector log file'
     condition: failed()
-
-  - pwsh: |
-      Copy-Item -Path "$(Build.SourcesDirectory)/http-fault-injector.log" -Destination "$(Build.SourcesDirectory)/fault.log"
-    condition: and(failed(), eq(variables['HAS_HTTP_FAULT_INJECTOR_LOGS'], 'true'))
-
-  - template: /eng/common/pipelines/templates/steps/publish-artifact.yml
-    parameters:
-      ArtifactName: "$(System.JobName)-http-fault-injector-logs"
-      ArtifactPath: "$(Build.SourcesDirectory)/fault.log"
-      CustomCondition: and(failed(), eq(variables['HAS_HTTP_FAULT_INJECTOR_LOGS'], 'true'))
-
-  - pwsh: |
-      Remove-Item -Force $(Build.SourcesDirectory)/fault.log
-    displayName: 'Cleanup http-fault-injector log file'
-    condition: and(failed(), eq(variables['HAS_HTTP_FAULT_INJECTOR_LOGS'], 'true'))
diff --git a/eng/pipelines/templates/steps/retain-heap-dump-hprofs.yml b/eng/pipelines/templates/steps/retain-heap-dump-hprofs.yml
index 9d125ce7b0110..2c27c3f37701d 100644
--- a/eng/pipelines/templates/steps/retain-heap-dump-hprofs.yml
+++ b/eng/pipelines/templates/steps/retain-heap-dump-hprofs.yml
@@ -4,27 +4,12 @@ parameters:
     default: 'oom-hprofs'
 
 steps:
-  - pwsh: |
-      if ((Get-ChildItem -Path . -Recurse -Filter *.hprof -File).Count -eq 0) {
-        Write-Host "##vso[task.setvariable variable=HAS_OOM_PROFS]false"
-      } else {
-        Write-Host "##vso[task.setvariable variable=HAS_OOM_PROFS]true"
-      }
-    displayName: 'Check for OOM hprofs'
+  - task: PowerShell@2
+    displayName: 'Check for and copy OOM hprofs'
+    inputs:
+      pwsh: true
+      filePath: eng/pipelines/scripts/Get-Heap-Dump-Hprofs.ps1
+      arguments: >
+        -StagingDirectory $(System.DefaultWorkingDirectory)
+        -OomArtifactName ${{ parameters.OomArtifactName }}
     condition: always()
-
-  - pwsh: |
-      New-Item $(Build.ArtifactStagingDirectory)/${{ parameters.OomArtifactName }} -ItemType directory
-      foreach($hprofFile in (Get-ChildItem -Path . -Recurse -Filter *.hprof -File)) {
-        $fileFullName = $hprofFile.FullName
-        $fileName = $hprofFile.Name
-        Move-Item -Path $fileFullName -Destination $(Build.ArtifactStagingDirectory)/${{ parameters.OomArtifactName }}/$fileName -ErrorAction SilentlyContinue
-      }
-      [System.IO.Compression.ZipFile]::CreateFromDirectory("$(Build.ArtifactStagingDirectory)/${{ parameters.OomArtifactName }}","$(Build.ArtifactStagingDirectory)/${{ parameters.OomArtifactName }}.zip")
-    displayName: 'Copy OOM hprofs'
-    condition: eq(variables['HAS_OOM_PROFS'], 'true')
-
-  - publish: $(Build.ArtifactStagingDirectory)/${{ parameters.OomArtifactName }}.zip
-    displayName: 'Publish OOM hprofs'
-    artifact: ${{ parameters.OomArtifactName }}-$(System.StageName)-$(System.JobName)-$(System.JobAttempt)
-    condition: eq(variables['HAS_OOM_PROFS'], 'true')
diff --git a/eng/pipelines/templates/steps/retain-test-logs.yml b/eng/pipelines/templates/steps/retain-test-logs.yml
index a777408c9cc9c..c644de203e194 100644
--- a/eng/pipelines/templates/steps/retain-test-logs.yml
+++ b/eng/pipelines/templates/steps/retain-test-logs.yml
@@ -4,24 +4,12 @@ parameters:
     default: 'test-logs'
 
 steps:
-  - pwsh: |
-      $testLogs = Get-ChildItem -Path . -Recurse -Filter test.log -File -Depth 4
-      if ($testLogs.Count -gt 0) {
-        Write-Host "##vso[task.setvariable variable=HAS_TEST_LOGS]true"
-        New-Item $(Build.ArtifactStagingDirectory)/${{ parameters.TestLogsArtifactName }} -ItemType directory
-        foreach($testLog in $testLogs) {
-          $fileFullName = $testLog.FullName
-          $fileName = "$($testLog.Directory.Parent.Parent.Name)-$($testLog.Directory.Parent.Name)-$($testLog.Name)"
-          Move-Item -Path $fileFullName -Destination $(Build.ArtifactStagingDirectory)/${{ parameters.TestLogsArtifactName }}/$fileName -ErrorAction SilentlyContinue
-        }
-        [System.IO.Compression.ZipFile]::CreateFromDirectory("$(Build.ArtifactStagingDirectory)/${{ parameters.TestLogsArtifactName }}","$(Build.ArtifactStagingDirectory)/${{ parameters.TestLogsArtifactName }}.zip")
-      } else {
-        Write-Host "##vso[task.setvariable variable=HAS_TEST_LOGS]false"
-      }
+  - task: PowerShell@2
     displayName: 'Check for and copy test logs'
+    inputs:
+      pwsh: true
+      filePath: eng/pipelines/scripts/Get-Test-Logs.ps1
+      arguments: >
+        -StagingDirectory $(System.DefaultWorkingDirectory)
+        -TestLogsArtifactName ${{ parameters.TestLogsArtifactName }}
     condition: always()
-
-  - publish: $(Build.ArtifactStagingDirectory)/${{ parameters.TestLogsArtifactName }}.zip
-    displayName: 'Publish test logs'
-    artifact: ${{ parameters.TestLogsArtifactName }}-$(System.StageName)-$(System.JobName)-$(System.JobAttempt)
-    condition: and(not(succeeded()), eq(variables['HAS_TEST_LOGS'], 'true'))
diff --git a/eng/pipelines/templates/steps/run-and-validate-linting.yml b/eng/pipelines/templates/steps/run-and-validate-linting.yml
index 34f7a1a4fcfc6..c9e456df5bdd0 100644
--- a/eng/pipelines/templates/steps/run-and-validate-linting.yml
+++ b/eng/pipelines/templates/steps/run-and-validate-linting.yml
@@ -61,22 +61,16 @@ steps:
   - task: PowerShell@2
     inputs:
       filePath: $(System.DefaultWorkingDirectory)/eng/pipelines/scripts/Get-Linting-Reports.ps1
-      arguments: -OutputDirectory $(Build.ArtifactStagingDirectory)/linting-reports
+      arguments: -StagingDirectory $(System.DefaultWorkingDirectory)
       pwsh: true
       workingDirectory: $(Pipeline.Workspace)
     displayName: 'Generate Linting Reports'
     continueOnError: ${{ parameters.ContinueOnError }}
     condition: or(eq(coalesce(variables['IsLatestNonLtsJdk'], true), true), and(${{ parameters.RunLinting }}, succeeded()))
 
-  - publish: $(Build.ArtifactStagingDirectory)/linting-reports.zip
-    displayName: 'Publish Linting Reports'
-    artifact: linting-reports-$(System.StageName)-$(System.JobName)-$(System.JobAttempt)
-    continueOnError: ${{ parameters.ContinueOnError }}
-    condition: or(eq(coalesce(variables['IsLatestNonLtsJdk'], true), true), and(${{ parameters.RunLinting }}, succeeded()))
-
   - pwsh: |
-      if ((Get-ChildItem -Path $(Build.ArtifactStagingDirectory)/linting-reports -Directory | Measure-Object).Count -gt 0) {
-        Write-Host "Linting errors have been found, download the 'linting-reports.zip' artifact to see what was found."
+      if (Test-Path -Path "$(System.DefaultWorkingDirectory)/troubleshooting/linting-report.zip") {
+        Write-Host "Linting errors have been found, download the 'linting-report.zip' artifact to see what was found."
         exit 1
       }
     displayName: 'Fail Pipeline if Linting Reports Exist'
diff --git a/eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml b/eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
new file mode 100644
index 0000000000000..07c75f94ff9e7
--- /dev/null
+++ b/eng/pipelines/templates/steps/sparse-checkout-repo-initialized.yml
@@ -0,0 +1,30 @@
+parameters:
+  - name: Paths
+    type: object
+    default: []
+  - name: Repositories
+    type: object
+    default:
+      - Name: $(Build.Repository.Name)
+        Commitish: $(Build.SourceVersion)
+        WorkingDirectory: $(System.DefaultWorkingDirectory)
+  - name: SkipCheckoutNone
+    type: boolean
+    default: false
+
+steps:
+  - ${{ if not(parameters.SkipCheckoutNone) }}:
+      - checkout: none
+
+  - task: PowerShell@2
+    ${{ if eq(length(parameters.Repositories), 1) }}:
+      displayName: 'Sparse checkout ${{ parameters.Repositories[0].Name }}'
+    ${{ else }}:
+      displayName: 'Sparse checkout repositories'
+    inputs:
+      pwsh: true
+      filePath: eng/pipelines/scripts/Invoke-Sparse-Checkout.ps1
+      arguments: >
+        -PathsJson '${{ convertToJson(parameters.Paths) }}'
+        -RepositoriesJson '${{ convertToJson(parameters.Repositories) }}'
+      workingDirectory: $(System.DefaultWorkingDirectory)
diff --git a/eng/pipelines/templates/variables/globals.yml b/eng/pipelines/templates/variables/globals.yml
index 83f38787a3627..64c8cdcf81039 100644
--- a/eng/pipelines/templates/variables/globals.yml
+++ b/eng/pipelines/templates/variables/globals.yml
@@ -55,3 +55,5 @@ variables:
 
   # Disable CodeQL injections except for where we specifically enable it
   Codeql.SkipTaskAutoInjection: true
+
+  ComponentDetection.Timeout: 1200
diff --git a/eng/pipelines/templates/variables/image.yml b/eng/pipelines/templates/variables/image.yml
new file mode 100644
index 0000000000000..322e3875f38ef
--- /dev/null
+++ b/eng/pipelines/templates/variables/image.yml
@@ -0,0 +1,26 @@
+# Default pool image selection. Set as variable so we can override at pipeline level
+
+variables:
+  - name: LINUXPOOL
+    value: azsdk-pool-mms-ubuntu-2004-general
+  - name: WINDOWSPOOL
+    value: azsdk-pool-mms-win-2022-general
+  - name: MACPOOL
+    value: Azure Pipelines
+
+  - name: LINUXVMIMAGE
+    value: azsdk-pool-mms-ubuntu-2004-1espt
+  - name: LINUXNEXTVMIMAGE
+    value: azsdk-pool-mms-ubuntu-2204-1espt
+  - name: WINDOWSVMIMAGE
+    value: azsdk-pool-mms-win-2022-1espt
+  - name: MACVMIMAGE
+    value: macos-11
+
+  # Values required for pool.os field in 1es pipeline templates
+  - name: LINUXOS
+    value: linux
+  - name: WINDOWSOS
+    value: windows
+  - name: MACOS
+    value: macOS
diff --git a/eng/scripts/Generate-ServiceDirectories-From-Project-List.ps1 b/eng/scripts/Generate-ServiceDirectories-From-Project-List.ps1
index b6d4a07971219..4e4734faa8ce3 100644
--- a/eng/scripts/Generate-ServiceDirectories-From-Project-List.ps1
+++ b/eng/scripts/Generate-ServiceDirectories-From-Project-List.ps1
@@ -10,9 +10,12 @@
 # prior to calling this script.
 param(
   # $(Build.SourcesDirectory) - root of the repository
-  [Parameter(Mandatory=$true)][string]$SourcesDirectory,
-  # ArtifactsList will be using ('${{ convertToJson(parameters.Artifacts) }}' | ConvertFrom-Json | Select-Object name, groupId)
-  [Parameter(Mandatory=$true)][array] $ProjectList
+  [Parameter(Mandatory=$true)]
+  [string]$SourcesDirectory,
+
+  [Parameter(Mandatory=$true)]
+  [AllowEmptyCollection()]
+  [array] $ProjectList
 )
 
 $StartTime = $(get-date)
diff --git a/eng/scripts/Language-Settings.ps1 b/eng/scripts/Language-Settings.ps1
index 43b53c526ca63..50fe7d1a90bf3 100644
--- a/eng/scripts/Language-Settings.ps1
+++ b/eng/scripts/Language-Settings.ps1
@@ -655,7 +655,7 @@ function Find-java-Artifacts-For-Apireview($artifactDir, $pkgName)
   # Filter for package in "com.azure*" groupid.
   $artifactPath = Join-Path $artifactDir "com.azure*" $pkgName
   Write-Host "Checking for source jar in artifact path $($artifactPath)"
-  $files = Get-ChildItem -Recurse "${artifactPath}" | Where-Object -FilterScript {$_.Name.EndsWith("sources.jar")}
+  $files = @(Get-ChildItem -Recurse "${artifactPath}" | Where-Object -FilterScript {$_.Name.EndsWith("sources.jar")})
   if (!$files)
   {
     Write-Host "$($artifactPath) does not have any package"
diff --git a/eng/scripts/Publish-MavenPackages.ps1 b/eng/scripts/Publish-MavenPackages.ps1
index 4973e8fce742a..2501556d15d7d 100644
--- a/eng/scripts/Publish-MavenPackages.ps1
+++ b/eng/scripts/Publish-MavenPackages.ps1
@@ -161,6 +161,15 @@ foreach ($packageDetail in $packageDetails) {
   $gpgexeOption = "-Dgpgexe=$GPGExecutablePath"
   Write-Information "GPG Executable Option is: $gpgexeOption"
 
+  $gpgPluginVersion = . $PSScriptRoot\Get-ExternalDependencyVersion.ps1 -GroupId 'org.apache.maven.plugins' -ArtifactId 'maven-gpg-plugin'
+  if ($LASTEXITCODE) {
+    Write-Information "##vso[task.logissue type=error]Unable to resolve version of external dependency 'org.apache.maven.plugins:maven-gpg-plugin'"
+    exit $LASTEXITCODE
+  }
+
+  $gpgSignAndDeployWithVer = "org.apache.maven.plugins:maven-gpg-plugin:$gpgPluginVersion`:sign-and-deploy-file"
+
+
   if ($requiresLocalGpg) {
     $localRepositoryDirectory = Get-RandomRepositoryDirectory
     $localRepositoryDirectoryUri = $([Uri]$localRepositoryDirectory.FullName).AbsoluteUri
@@ -170,8 +179,8 @@ foreach ($packageDetail in $packageDetails) {
     Write-Information "URL Option is: $urlOption"
 
     Write-Information "Signing and deploying package to $localRepositoryDirectoryUri"
-    Write-Information "mvn gpg:sign-and-deploy-file `"--batch-mode`" `"$pomOption`" `"$fileOption`" `"$javadocOption`" `"$sourcesOption`" `"$filesOption`" $classifiersOption `"$typesOption`" `"$urlOption`" `"$gpgexeOption`" `"-DrepositoryId=target-repo`" `"--settings=$PSScriptRoot\..\maven.publish.settings.xml`""
-    mvn gpg:sign-and-deploy-file "--batch-mode" "$pomOption" "$fileOption" "$javadocOption" "$sourcesOption" "$filesOption" $classifiersOption "$typesOption" "$urlOption" "$gpgexeOption" "-DrepositoryId=target-repo" "--settings=$PSScriptRoot\..\maven.publish.settings.xml"
+    Write-Information "mvn $gpgSignAndDeployWithVer `"--batch-mode`" `"$pomOption`" `"$fileOption`" `"$javadocOption`" `"$sourcesOption`" `"$filesOption`" $classifiersOption `"$typesOption`" `"$urlOption`" `"$gpgexeOption`" `"-DrepositoryId=target-repo`" `"--settings=$PSScriptRoot\..\maven.publish.settings.xml`""
+    mvn $gpgSignAndDeployWithVer "--batch-mode" "$pomOption" "$fileOption" "$javadocOption" "$sourcesOption" "$filesOption" $classifiersOption "$typesOption" "$urlOption" "$gpgexeOption" "-DrepositoryId=target-repo" "--settings=$PSScriptRoot\..\maven.publish.settings.xml"
     if ($LASTEXITCODE) { exit $LASTEXITCODE }
   }
 
@@ -183,8 +192,8 @@ foreach ($packageDetail in $packageDetails) {
 
   if ($releaseType -eq 'AzureDevOps') {
     Write-Information "GPG Signing and deploying package in one step to devops feed: $packageReposityUrl"
-    Write-Information "mvn gpg:sign-and-deploy-file `"--batch-mode`" `"$pomOption`" `"$fileOption`" `"$javadocOption`" `"$sourcesOption`" `"$filesOption`" $classifiersOption `"$typesOption`" `"-Durl=$packageReposityUrl`" `"$gpgexeOption`" `"-DrepositoryId=target-repo`" `"-Drepo.password=[redacted]`" `"--settings=$PSScriptRoot\..\maven.publish.settings.xml`""
-    mvn gpg:sign-and-deploy-file "--batch-mode" "$pomOption" "$fileOption" "$javadocOption" "$sourcesOption" "$filesOption" $classifiersOption "$typesOption" "-Durl=$packageReposityUrl" "$gpgexeOption" "-DrepositoryId=target-repo" "-Drepo.password=$RepositoryPassword" "--settings=$PSScriptRoot\..\maven.publish.settings.xml"
+    Write-Information "mvn $gpgSignAndDeployWithVer `"--batch-mode`" `"$pomOption`" `"$fileOption`" `"$javadocOption`" `"$sourcesOption`" `"$filesOption`" $classifiersOption `"$typesOption`" `"-Durl=$packageReposityUrl`" `"$gpgexeOption`" `"-DrepositoryId=target-repo`" `"-Drepo.password=[redacted]`" `"--settings=$PSScriptRoot\..\maven.publish.settings.xml`""
+    mvn $gpgSignAndDeployWithVer "--batch-mode" "$pomOption" "$fileOption" "$javadocOption" "$sourcesOption" "$filesOption" $classifiersOption "$typesOption" "-Durl=$packageReposityUrl" "$gpgexeOption" "-DrepositoryId=target-repo" "-Drepo.password=$RepositoryPassword" "--settings=$PSScriptRoot\..\maven.publish.settings.xml"
 
     if ($LASTEXITCODE -eq 0) {
       Write-Information "Package $($packageDetail.FullyQualifiedName) deployed"
@@ -202,8 +211,8 @@ foreach ($packageDetail in $packageDetails) {
   }
   elseif ($releaseType -eq 'MavenCentralSnapshot') {
     Write-Information "GPG Signing and deploying package in one step to Sonatype snapshots: $packageReposityUrl"
-    Write-Information "mvn gpg:sign-and-deploy-file `"--batch-mode`" `"$pomOption`" `"$fileOption`" `"$javadocOption`" `"$sourcesOption`" `"$filesOption`" $classifiersOption `"$typesOption`" `"-Durl=$packageReposityUrl`" `"$gpgexeOption`" `"-DrepositoryId=target-repo`" `"-Drepo.username=`"`"$RepositoryUsername`"`"`" `"-Drepo.password=[redacted]`" `"--settings=$PSScriptRoot\..\maven.publish.settings.xml`""
-    mvn gpg:sign-and-deploy-file "--batch-mode" "$pomOption" "$fileOption" "$javadocOption" "$sourcesOption" "$filesOption" $classifiersOption "$typesOption" "-Durl=$packageReposityUrl" "$gpgexeOption" "-DrepositoryId=target-repo" "-Drepo.username=""$RepositoryUsername""" "-Drepo.password=""$RepositoryPassword""" "--settings=$PSScriptRoot\..\maven.publish.settings.xml"
+    Write-Information "mvn $gpgSignAndDeployWithVer `"--batch-mode`" `"$pomOption`" `"$fileOption`" `"$javadocOption`" `"$sourcesOption`" `"$filesOption`" $classifiersOption `"$typesOption`" `"-Durl=$packageReposityUrl`" `"$gpgexeOption`" `"-DrepositoryId=target-repo`" `"-Drepo.username=`"`"$RepositoryUsername`"`"`" `"-Drepo.password=[redacted]`" `"--settings=$PSScriptRoot\..\maven.publish.settings.xml`""
+    mvn $gpgSignAndDeployWithVer "--batch-mode" "$pomOption" "$fileOption" "$javadocOption" "$sourcesOption" "$filesOption" $classifiersOption "$typesOption" "-Durl=$packageReposityUrl" "$gpgexeOption" "-DrepositoryId=target-repo" "-Drepo.username=""$RepositoryUsername""" "-Drepo.password=""$RepositoryPassword""" "--settings=$PSScriptRoot\..\maven.publish.settings.xml"
     if ($LASTEXITCODE) { exit $LASTEXITCODE }
   }
   else {
diff --git a/eng/scripts/generate_from_source_pom.py b/eng/scripts/generate_from_source_pom.py
index 0d39e4d9123c2..660c479980dc5 100644
--- a/eng/scripts/generate_from_source_pom.py
+++ b/eng/scripts/generate_from_source_pom.py
@@ -53,24 +53,28 @@
 sdk_string = "/sdk/"
 
 # Function that creates the aggregate POM.
-def create_from_source_pom(project_list: str, set_skip_linting_projects: str, match_any_version: bool):
-    project_list_identifiers = project_list.split(',')
+def create_from_source_pom(artifacts_list: str, additional_modules_list: str, set_skip_linting_projects: str, match_any_version: bool):
+    artifacts_list_identifiers = artifacts_list.split(',')
+    
+    additional_modules_identifiers = []
+    if additional_modules_list is not None:
+        additional_modules_identifiers = additional_modules_list.split(',')
 
     # Get the artifact identifiers from client_versions.txt to act as our source of truth.
     artifact_identifier_to_version = load_client_artifact_identifiers()
 
-    projects = create_projects(project_list_identifiers, artifact_identifier_to_version, match_any_version)
+    projects = create_projects(artifacts_list_identifiers, artifact_identifier_to_version, match_any_version)
 
     dependent_modules: Set[str] = set()
 
     # Resolve all projects, including transitively, that are dependent on the projects in the project list.
-    for project_identifier in project_list_identifiers:
+    for project_identifier in artifacts_list_identifiers:
         dependent_modules = resolve_dependent_project(project_identifier, dependent_modules, projects)
 
     dependency_modules: Set[str] = set()
 
     # Resolve all dependencies of the projects in the project list and of the dependent modules.
-    for project_identifier in project_list_identifiers:
+    for project_identifier in artifacts_list_identifiers:
         dependency_modules = resolve_project_dependencies(project_identifier, dependency_modules, projects)
     for project_identifier in dependent_modules:
         dependency_modules = resolve_project_dependencies(project_identifier, dependency_modules, projects)
@@ -78,7 +82,8 @@ def create_from_source_pom(project_list: str, set_skip_linting_projects: str, ma
     source_projects: Set[Project] = set()
 
     # Finally map the project identifiers to projects.
-    add_source_projects(source_projects, project_list_identifiers, projects)
+    add_source_projects(source_projects, artifacts_list_identifiers, projects)
+    add_source_projects(source_projects, additional_modules_identifiers, projects)
     add_source_projects(source_projects, dependent_modules, projects)
     add_source_projects(source_projects, dependency_modules, projects)
 
@@ -105,7 +110,7 @@ def create_from_source_pom(project_list: str, set_skip_linting_projects: str, ma
         sparse_checkout_directories.add(sparse_checkout_directory)
         # The ServiceDirectories list should only ever contain the list of service
         # directories for the project list and nothing else.
-        if p.identifier in project_list_identifiers:
+        if p.identifier in artifacts_list_identifiers:
             # Sparse checkout directories can contain directories that aren't service directories.
             # (aka. /common). Any service directory will start with "/sdk/", everything else is
             # would be attributed to supporting libraries (ex. perf-test-core).
@@ -181,7 +186,7 @@ def load_client_artifact_identifiers() -> Dict[str, ArtifactVersion]:
 
 # Function that creates the Projects within the repository.
 # Projects contain a Maven identifier, module path, parent POM, its dependency Maven identifiers, and Maven identifiers for projects dependent on it.
-def create_projects(project_list_identifiers: list, artifact_identifier_to_version: Dict[str, ArtifactVersion], match_any_version: bool) -> Dict[str, Project]:
+def create_projects(artifacts_list_identifiers: list, artifact_identifier_to_version: Dict[str, ArtifactVersion], match_any_version: bool) -> Dict[str, Project]:
     projects: Dict[str, Project] = {}
 
     for root, _, files in os.walk(root_path):
@@ -198,7 +203,7 @@ def create_projects(project_list_identifiers: list, artifact_identifier_to_versi
 
             # Only parse files that are pom.xml files.
             if (file_name.startswith('pom') and file_name.endswith('.xml')):
-                project = create_project_for_pom(file_path, project_list_identifiers, artifact_identifier_to_version, match_any_version)
+                project = create_project_for_pom(file_path, artifacts_list_identifiers, artifact_identifier_to_version, match_any_version)
                 if project is not None:
                     projects[project.identifier] = project
 
@@ -210,7 +215,7 @@ def create_projects(project_list_identifiers: list, artifact_identifier_to_versi
 
     return projects
 
-def create_project_for_pom(pom_path: str, project_list_identifiers: list, artifact_identifier_to_version: Dict[str, ArtifactVersion], match_any_version: bool):
+def create_project_for_pom(pom_path: str, artifacts_list_identifiers: list, artifact_identifier_to_version: Dict[str, ArtifactVersion], match_any_version: bool):
     if 'eng' in pom_path.split(os.sep):
         return
 
@@ -227,7 +232,7 @@ def create_project_for_pom(pom_path: str, project_list_identifiers: list, artifa
         return Project(project_identifier, directory_path, module_path, parent_pom)
 
     # If the project isn't a track 2 POM skip it and not one of the project list identifiers.
-    if not project_identifier in project_list_identifiers and not is_spring_child_pom(tree_root) and not parent_pom in valid_parents: # Spring pom's parent can be empty.
+    if not project_identifier in artifacts_list_identifiers and not is_spring_child_pom(tree_root) and not parent_pom in valid_parents: # Spring pom's parent can be empty.
         return
 
     project = Project(project_identifier, directory_path, module_path, parent_pom)
@@ -303,14 +308,15 @@ def project_uses_client_parent(project: Project, projects: Dict[str, Project]) -
 
 def main():
     parser = argparse.ArgumentParser(description='Generated an aggregate POM for a From Source run.')
-    parser.add_argument('--project-list', '--pl', type=str)
+    parser.add_argument('--artifacts-list', '--al', type=str)
+    parser.add_argument('--additional-modules-list', '--aml', type=str, required=False, default=None, nargs='?')
     parser.add_argument('--set-skip-linting-projects', type=str)
     parser.add_argument('--match-any-version', action='store_true')
     args = parser.parse_args()
-    if args.project_list == None:
-        raise ValueError('Missing project list.')
+    if args.artifacts_list == None:
+        raise ValueError('Missing artifacts list.')
     start_time = time.time()
-    create_from_source_pom(args.project_list, args.set_skip_linting_projects, args.match_any_version)
+    create_from_source_pom(args.artifacts_list, args.additional_modules_list, args.set_skip_linting_projects, args.match_any_version)
     elapsed_time = time.time() - start_time
 
     print('Effective From Source POM File')
diff --git a/eng/scripts/setup-http-fault-injector.ps1 b/eng/scripts/setup-http-fault-injector.ps1
new file mode 100644
index 0000000000000..3f5bc523a99a5
--- /dev/null
+++ b/eng/scripts/setup-http-fault-injector.ps1
@@ -0,0 +1,30 @@
+param(
+  [Parameter(Mandatory=$false)]
+  [string]$SourcesDirectory = $env:BUILD_SOURCESDIRECTORY
+)
+
+dotnet tool install azure.sdk.tools.httpfaultinjector --global --prerelease --add-source https://pkgs.dev.azure.com/azure-sdk/public/_packaging/azure-sdk-for-net/nuget/v3/index.json
+dotnet tool update azure.sdk.tools.httpfaultinjector --global --prerelease --add-source https://pkgs.dev.azure.com/azure-sdk/public/_packaging/azure-sdk-for-net/nuget/v3/index.json
+http-fault-injector --version
+
+Write-Host "##vso[task.setvariable variable=ASPNETCORE_Kestrel__Certificates__Default__Path]$SourcesDirectory/eng/common/testproxy/dotnet-devcert.pfx"
+Write-Host "##vso[task.setvariable variable=ASPNETCORE_Kestrel__Certificates__Default__Password]password"
+Write-Host "##vso[task.setvariable variable=PROXY_MANUAL_START]true"
+
+# Trust http-fault-injector self-siigned certificate
+if (Test-Path $env:JAVA_HOME/jre/lib/security) {
+    Set-Location $env:JAVA_HOME/jre/lib/security
+} elseif (Test-Path $env:JAVA_HOME/lib/security) {
+    Set-Location $env:JAVA_HOME/lib/security
+} else {
+    Write-Error "JDK directory structure is unknown and unsupported. JAVA_HOME is '$env:JAVA_HOME'"
+    exit 1
+}
+
+dotnet dev-certs https --export-path http-fault-injector.pfx
+
+if ($IsWindows) {
+    keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias HttpFaultInject -file http-fault-injector.pfx -storepass changeit
+} else {
+    sudo keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias HttpFaultInject -file http-fault-injector.pfx -storepass changeit
+}
diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt
index ff96fa9a23923..e879fca40aafd 100644
--- a/eng/versioning/external_dependencies.txt
+++ b/eng/versioning/external_dependencies.txt
@@ -7,7 +7,7 @@
 
 # Format;
 # groupId:artifactId;dependency-version
-ch.qos.logback:logback-classic;1.3.12
+ch.qos.logback:logback-classic;1.3.14
 com.azure.tools:azure-autorest-customization;1.0.0-beta.8
 com.fasterxml.jackson.core:jackson-annotations;2.13.5
 com.fasterxml.jackson.core:jackson-core;2.13.5
@@ -22,8 +22,8 @@ com.fasterxml.woodstox:woodstox-core;6.4.0
 com.github.spotbugs:spotbugs;4.8.3
 com.github.spotbugs:spotbugs-annotations;4.8.3
 com.github.spotbugs:spotbugs-maven-plugin;4.8.2.0
-com.google.code.gson:gson;2.10
-com.google.guava:guava;33.0.0-jre
+com.google.code.gson:gson;2.10.1
+com.google.guava:guava;33.1.0-jre
 com.h2database:h2;2.2.220
 com.h3xstream.findsecbugs:findsecbugs-plugin;1.9.0
 com.knuddels:jtokkit;1.0.0
@@ -34,6 +34,7 @@ com.microsoft.azure:azure-client-authentication;1.7.14
 com.microsoft.azure:azure-client-runtime;1.7.14
 com.microsoft.azure:azure-core;0.9.8
 com.microsoft.azure:azure-cosmos;3.7.3
+com.microsoft.azure:azure-eventhubs-eph;3.3.0
 com.microsoft.azure:azure-keyvault-cryptography;1.2.2
 com.microsoft.azure:azure-media;0.9.8
 com.microsoft.azure:azure-servicebus;3.6.1
@@ -55,25 +56,25 @@ io.fabric8:kubernetes-client;5.12.3
 io.micrometer:micrometer-core;1.9.17
 io.micrometer:micrometer-registry-azure-monitor;1.9.17
 io.micrometer:micrometer-registry-graphite;1.9.17
-io.netty:netty-buffer;4.1.101.Final
-io.netty:netty-common;4.1.101.Final
-io.netty:netty-codec;4.1.101.Final
-io.netty:netty-codec-http;4.1.101.Final
-io.netty:netty-codec-http2;4.1.101.Final
-io.netty:netty-transport;4.1.101.Final
-io.netty:netty-handler;4.1.101.Final
-io.netty:netty-handler-proxy;4.1.101.Final
-io.netty:netty-tcnative;2.0.62.Final
-io.netty:netty-tcnative-boringssl-static;2.0.62.Final
-io.netty:netty-transport-native-epoll;4.1.101.Final
-io.netty:netty-transport-native-unix-common;4.1.101.Final
-io.netty:netty-transport-native-kqueue;4.1.101.Final
-io.projectreactor.netty:reactor-netty;1.0.40
-io.projectreactor.netty:reactor-netty-http;1.0.40
-io.projectreactor:reactor-core;3.4.34
-io.vertx:vertx-codegen;4.5.4
-io.vertx:vertx-core;4.5.4
-io.vertx:vertx-reactive-streams;4.5.4
+io.netty:netty-buffer;4.1.107.Final
+io.netty:netty-common;4.1.107.Final
+io.netty:netty-codec;4.1.107.Final
+io.netty:netty-codec-http;4.1.107.Final
+io.netty:netty-codec-http2;4.1.107.Final
+io.netty:netty-transport;4.1.107.Final
+io.netty:netty-handler;4.1.107.Final
+io.netty:netty-handler-proxy;4.1.107.Final
+io.netty:netty-tcnative;2.0.65.Final
+io.netty:netty-tcnative-boringssl-static;2.0.65.Final
+io.netty:netty-transport-native-epoll;4.1.107.Final
+io.netty:netty-transport-native-unix-common;4.1.107.Final
+io.netty:netty-transport-native-kqueue;4.1.107.Final
+io.projectreactor.netty:reactor-netty;1.0.43
+io.projectreactor.netty:reactor-netty-http;1.0.43
+io.projectreactor:reactor-core;3.4.36
+io.vertx:vertx-codegen;4.5.5
+io.vertx:vertx-core;4.5.5
+io.vertx:vertx-reactive-streams;4.5.5
 jakarta.validation:jakarta.validation-api;2.0.2
 javax.annotation:javax.annotation-api;1.3.2
 javax.json:javax.json-api;1.1.4
@@ -92,6 +93,7 @@ org.apache.logging.log4j:log4j-slf4j-impl;2.17.2
 org.apache.maven:maven-core;3.5.4
 org.apache.maven.archetype:archetype-packaging;3.2.0
 org.apache.maven.plugins:maven-archetype-plugin;3.2.0
+org.apache.maven.plugins:maven-gpg-plugin;3.1.0
 org.apache.qpid:proton-j;0.33.8
 org.apache.qpid:qpid-jms-client;0.53.0
 org.apache.tinkerpop:gremlin-driver;3.2.4
@@ -115,11 +117,11 @@ org.slf4j:slf4j-api;1.7.36
 org.slf4j:slf4j-nop;1.7.36
 org.slf4j:slf4j-simple;1.7.36
 redis.clients:jedis;5.1.0
-io.lettuce:lettuce-core;6.3.1.RELEASE
-org.redisson:redisson;3.27.0
+io.lettuce:lettuce-core;6.3.2.RELEASE
+org.redisson:redisson;3.27.2
 net.bytebuddy:byte-buddy;1.12.23
-testdep_net.bytebuddy:byte-buddy;1.14.8
-testdep_net.bytebuddy:byte-buddy-agent;1.14.8
+testdep_net.bytebuddy:byte-buddy;1.14.12
+testdep_net.bytebuddy:byte-buddy-agent;1.14.12
 
 ## Spring boot dependency versions
 org.springframework.boot:spring-boot-dependencies;2.7.18
@@ -157,31 +159,31 @@ org.springframework.security:spring-security-oauth2-resource-server;5.7.11
 org.springframework.security:spring-security-oauth2-core;5.7.11
 org.springframework.security:spring-security-oauth2-jose;5.7.11
 org.springframework.security:spring-security-web;5.7.11
-org.springframework:spring-beans;5.3.31
-org.springframework:spring-context-support;5.3.31
-org.springframework:spring-context;5.3.31
-org.springframework:spring-core;5.3.31
-org.springframework:spring-expression;5.3.31
-org.springframework:spring-jdbc;5.3.31
-org.springframework:spring-jms;5.3.31
-org.springframework:spring-messaging;5.3.31
-org.springframework:spring-tx;5.3.31
-org.springframework:spring-web;5.3.31
-org.springframework:spring-webmvc;5.3.31
-org.springframework:spring-test;5.3.31
+org.springframework:spring-beans;5.3.32
+org.springframework:spring-context-support;5.3.32
+org.springframework:spring-context;5.3.32
+org.springframework:spring-core;5.3.32
+org.springframework:spring-expression;5.3.32
+org.springframework:spring-jdbc;5.3.32
+org.springframework:spring-jms;5.3.32
+org.springframework:spring-messaging;5.3.32
+org.springframework:spring-tx;5.3.32
+org.springframework:spring-web;5.3.32
+org.springframework:spring-webmvc;5.3.32
+org.springframework:spring-test;5.3.32
 org.springframework.retry:spring-retry;1.3.4
 org.springframework.boot:spring-boot-starter-parent;2.7.18
 
 ## Spring cloud dependency versions
-org.springframework.cloud:spring-cloud-dependencies;2021.0.8
-org.springframework.cloud:spring-cloud-starter-bootstrap;3.1.7
-org.springframework.cloud:spring-cloud-bus;3.1.2
-org.springframework.cloud:spring-cloud-context;3.1.7
-org.springframework.cloud:spring-cloud-sleuth-api;3.1.9
-org.springframework.cloud:spring-cloud-starter-sleuth;3.1.9
-org.springframework.cloud:spring-cloud-starter-stream-kafka;3.2.9
-org.springframework.cloud:spring-cloud-stream-binder-test;3.2.9
-org.springframework.cloud:spring-cloud-stream;3.2.9
+org.springframework.cloud:spring-cloud-dependencies;2021.0.9
+org.springframework.cloud:spring-cloud-starter-bootstrap;3.1.8
+org.springframework.cloud:spring-cloud-bus;3.1.3
+org.springframework.cloud:spring-cloud-context;3.1.8
+org.springframework.cloud:spring-cloud-sleuth-api;3.1.10
+org.springframework.cloud:spring-cloud-starter-sleuth;3.1.10
+org.springframework.cloud:spring-cloud-starter-stream-kafka;3.2.10
+org.springframework.cloud:spring-cloud-stream-binder-test;3.2.10
+org.springframework.cloud:spring-cloud-stream;3.2.10
 
 ## Test dependency versions
 org.objenesis:objenesis;3.1
@@ -198,30 +200,30 @@ com.microsoft.azure:azure-mgmt-resources;1.3.0
 com.microsoft.azure:azure-mgmt-search;1.24.1
 com.microsoft.azure:azure-mgmt-storage;1.3.0
 com.microsoft.azure:azure-storage;8.0.0
-com.microsoft.azure:msal4j;1.14.0
+com.microsoft.azure:msal4j;1.14.3
 com.microsoft.azure:msal4j-brokers;1.0.0
 com.microsoft.azure:msal4j-persistence-extension;1.2.0
 com.sun.activation:jakarta.activation;1.2.2
-io.opentelemetry:opentelemetry-api;1.35.0
-io.opentelemetry:opentelemetry-sdk;1.35.0
-io.opentelemetry:opentelemetry-sdk-metrics;1.35.0
-io.opentelemetry:opentelemetry-sdk-logs;1.35.0
-io.opentelemetry:opentelemetry-exporter-logging;1.35.0
-io.opentelemetry:opentelemetry-exporter-otlp;1.35.0
-io.opentelemetry:opentelemetry-sdk-testing;1.35.0
-io.opentelemetry:opentelemetry-sdk-extension-autoconfigure;1.35.0
-io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi;1.35.0
-io.opentelemetry:opentelemetry-bom;1.35.0
-io.opentelemetry.instrumentation:opentelemetry-instrumentation-bom-alpha;2.1.0-alpha
-io.opentelemetry.instrumentation:opentelemetry-runtime-telemetry-java8;2.1.0-alpha
-io.opentelemetry.instrumentation:opentelemetry-spring-boot-starter;2.1.0-alpha
-io.opentelemetry.instrumentation:opentelemetry-jdbc;2.1.0-alpha
-io.opentelemetry.instrumentation:opentelemetry-logback-appender-1.0;2.1.0-alpha
+io.opentelemetry:opentelemetry-api;1.36.0
+io.opentelemetry:opentelemetry-sdk;1.36.0
+io.opentelemetry:opentelemetry-sdk-metrics;1.36.0
+io.opentelemetry:opentelemetry-sdk-logs;1.36.0
+io.opentelemetry:opentelemetry-exporter-logging;1.36.0
+io.opentelemetry:opentelemetry-exporter-otlp;1.36.0
+io.opentelemetry:opentelemetry-sdk-testing;1.36.0
+io.opentelemetry:opentelemetry-sdk-extension-autoconfigure;1.36.0
+io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi;1.36.0
+io.opentelemetry:opentelemetry-bom;1.36.0
+io.opentelemetry.instrumentation:opentelemetry-instrumentation-bom-alpha;2.2.0-alpha
+io.opentelemetry.instrumentation:opentelemetry-runtime-telemetry-java8;2.2.0-alpha
+io.opentelemetry.instrumentation:opentelemetry-spring-boot-starter;2.2.0-alpha
+io.opentelemetry.instrumentation:opentelemetry-jdbc;2.2.0-alpha
+io.opentelemetry.instrumentation:opentelemetry-logback-appender-1.0;2.2.0-alpha
 
-io.projectreactor:reactor-test;3.4.34
+io.projectreactor:reactor-test;3.4.36
 io.github.hakky54:logcaptor;2.9.0
-com.squareup.okio:okio;3.6.0
-com.squareup.okio:okio-jvm;3.6.0
+com.squareup.okio:okio;3.9.0
+com.squareup.okio:okio-jvm;3.9.0
 junit:junit;4.13.2
 commons-cli:commons-cli;1.3
 org.assertj:assertj-core;3.22.0
@@ -429,7 +431,7 @@ media_com.microsoft.azure:adal4j;1.2.0
 resourcemanager_com.jcraft:jsch;0.1.55
 
 # sdk\storage\azure-storage-blob-cryptography\pom.xml
-storage_com.microsoft.azure:azure-storage;8.4.0
+storage_com.microsoft.azure:azure-storage;8.6.6
 
 # sdk\appconfiguration\azure-spring-cloud-test-appconfiguration-config\pom.xml
 spring_com.microsoft.azure:azure;1.34.0
diff --git a/eng/versioning/pom_file_version_scanner.ps1 b/eng/versioning/pom_file_version_scanner.ps1
index 101baf77ea553..bad7ebb0a1652 100644
--- a/eng/versioning/pom_file_version_scanner.ps1
+++ b/eng/versioning/pom_file_version_scanner.ps1
@@ -42,8 +42,10 @@ $ValidParents = ("azure-sdk-parent", "azure-client-sdk-parent", "azure-data-sdk-
 # which means these files have to be "sort of" scanned.
 $SpringSampleParents = ("spring-boot-starter-parent", "azure-spring-boot-test-parent")
 
+. "${PSScriptRoot}/../common/scripts/Helpers/PSModule-Helpers.ps1"
 $Path = Resolve-Path ($PSScriptRoot + "/../../")
 $SamplesPath = Resolve-Path ($PSScriptRoot + "/../../samples")
+$SdkRoot = Resolve-Path ($PSScriptRoot + "/../../sdk")
 
 # Not all POM files have a parent entry
 $PomFilesIgnoreParent = ("$($Path)\parent\pom.xml")
@@ -54,6 +56,9 @@ $DependencyTypeExternal = "external_dependency"
 $DependencyTypeForError = "$($DependencyTypeCurrent)|$($DependencyTypeDependency)|$($DependencyTypeExternal)"
 $UpdateTagFormat = "{x-version-update;:;$($DependencyTypeForError)}"
 $UseVerboseLogging = $PSBoundParameters['Debug'] -or $PSBoundParameters['Verbose']
+
+Install-ModuleIfNotInstalled "powershell-yaml" "0.4.1" | Import-Module
+
 $StartTime = $(get-date)
 
 # This is the for the bannedDependencies include exceptions. All  entries need to be of the
@@ -126,6 +131,95 @@ class ExternalDependency {
     }
 }
 
+# Create a dictionary of libraries per sdk/. The Key will be the directory
+# and the Value will be a HashSet of the libraries produced. Note: This looks at ci*.yml meaning
+# that it picks it'll pick up both track 1 and track 2 libraries. This is okay to join
+# since the libraries produced by both tracks have different GroupIds and even then, have different
+# ArtifactIds between tracks. Further, there are no interdependencies between track 1 and track 2
+# libraries.
+# The purpose of collecting this data is to verify that all interdependencies within an ServiceDirectory
+# are using current versions of other libraries built and released from the same ServiceDirectory.
+# Similarly, this can also be used to ensure that any dependencies not part of a given ServiceDirectory
+# only use the dependency versions.
+# It's also worth noting that the artifacts list is pulled from the ci.yml files. The search goes
+# depth 3 due to the fact that there are some libraries whose pipelines yml files are one level deeper
+# than the sdk/. They're in the sdk///ci.yml and
+# sdk/communication is an example of this as each library has its own pipeline.
+function Get-ArtifactsList-Per-Service-Directory {
+    param([System.Collections.Specialized.OrderedDictionary]$ArtifactsPerSD)
+
+    # Get all of the yml files under sdk/
+    $ymlFiles = Get-ChildItem -Path $SdkRoot -Recurse -Depth 3 -File -Filter "ci*yml"
+    foreach ($ymlFile in $ymlFiles) {
+        # The ci.cosmos.yml lives in spring and is used to test the cosmos spring library. Its exception
+        # will be moved once things are corrected.
+        if ($ymlFile.FullName.Split([IO.Path]::DirectorySeparatorChar) -contains "resourcemanagerhybrid" -or
+            $ymlFile.Name -eq "ci.cosmos.yml") {
+            continue
+        }
+        # The path is going to be the key. Since there can be multiple yml files for a single path,
+        # (ci.yml and ci.mgmt.yml),
+        $ymlPath = Split-Path $ymlFile
+        if (-not $ArtifactsPerSD.Contains($ymlPath)) {
+            $artifactHashSet = New-Object 'System.Collections.Generic.HashSet[String]'
+            $ArtifactsPerSD.Add($ymlPath, $artifactHashSet)
+        }
+
+        $ymlContent = Get-Content $ymlFile.FullName -Raw
+        $ymlObject = ConvertFrom-Yaml $ymlContent -Ordered
+        # Get each artifact that is built/released as part of this yml file
+        foreach ($artifact in $ymlObject["extends"]["parameters"]["artifacts"]) {
+            $libFullName = $artifact["groupId"] + ":" + $artifact["name"]
+            # The same artifact in different yml files or twice in the same file is bad.
+            if (-not $ArtifactsPerSD[$ymlPath].Add($libFullName)) {
+                Write-Error "Processing yml file $($ymlFile.FullName)"
+                Write-Error "$ymlPath already contains an Artifact entry for $libFullName"
+            }
+        }
+        # These list of modules per sdk/ has to be verified to be using the
+        # latest version of other modules in the same ServiceDirectory which includes AdditionModules.
+        # The groupIds will ensure that track1 and track2 libraries won't collide in here. In the
+        # case of AdditionalModules, com.azure:perf-test-core is ubiquitous and can appear in the
+        # AdditionalModules of ci.yml and ci.mgmt.yml. As such, it's the only library we'll ignore
+        # a dupe of.
+        foreach ($artifact in $ymlObject["extends"]["parameters"]["AdditionalModules"]) {
+            $libFullName = $artifact["groupId"] + ":" + $artifact["name"]
+            if (-not $ArtifactsPerSD[$ymlPath].Add($libFullName)) {
+                if ($libFullName -ne "com.azure:perf-test-core") {
+                    Write-Error "Processing yml file $($ymlFile.FullName)"
+                    Write-Error "$ymlPath already contains an AdditionalModule entry for $libFullName"
+                }
+            }
+        }
+    }
+}
+
+# Given the full path to a POM file, return the HashSet containing the list
+# of libraries built and released from that service directory, if there is
+# one, null otherwise.
+function Get-Artifacts-Built-In-Service-Directory {
+    param(
+        [System.Collections.Specialized.OrderedDictionary]$ArtifactsPerSD,
+        [string]$SdkRoot,
+        [string]$PomFileFullPath
+        )
+
+        # There are POM files that aren't in under under /sdk. Those
+        # aren't verified for current/dependency since they don't release
+        # and won't have entries in the ArtifactsPerSD
+        if ($PomFileFullPath.StartsWith($SdkRoot)) {
+            $tmpFile = $PomFileFullPath
+            while ($tmpFile -ne $SdkRoot) {
+                if ($ArtifactsPerSD.Contains($tmpFile)) {
+                    return $ArtifactsPerSD[$tmpFile]
+                } else {
+                    $tmpFile = Split-Path $tmpFile -Parent
+                }
+            }
+        }
+        return $null
+}
+
 function Build-Dependency-Hash-From-File {
     param(
         [hashtable]$depHash,
@@ -177,7 +271,8 @@ function Test-Dependency-Tag-And-Version {
         [hashtable]$libHash,
         [hashtable]$extDepHash,
         [string]$versionString,
-        [string]$versionUpdateString)
+        [string]$versionUpdateString,
+        [System.Collections.Generic.HashSet[string]]$libPerSDHash = $null)
 
     # This is the format of the versionUpdateString and there should be 3 parts:
     # 1. The update tag, itself eg. x-version-update
@@ -226,6 +321,14 @@ function Test-Dependency-Tag-And-Version {
         {
             if ($depType -eq $DependencyTypeDependency)
             {
+                # Any libraries built as part of the same pipeline need to use the current
+                # version of other libraries within the same pipeline. Verify that this
+                # dependency, of type dependency, shouldn't be current
+                if ($libPerSDHash) {
+                    if ($libPerSDHash.Contains($depKey)) {
+                        return "Error: $($versionUpdateString.Trim()) is incorrectly set to $DependencyTypeDependency. Libraries building and releasing as part of the same pipeline need to be using the $DependencyTypeCurrent versions of each other."
+                    }
+                }
                 if ($versionString -ne $libHash[$depKey].depVer)
                 {
                     return "Error: $($depKey)'s  is '$($versionString)' but the dependency version is listed as $($libHash[$depKey].depVer)"
@@ -236,7 +339,12 @@ function Test-Dependency-Tag-And-Version {
                 # Verify that none of the 'current' dependencies are using a groupId that starts with 'unreleased_' or 'beta_'
                 if ($depKey.StartsWith('unreleased_') -or $depKey.StartsWith('beta_'))
                 {
-                    return "Error: $($versionUpdateString) is using an unreleased_ or beta_ dependency and trying to set current value. Only dependency versions can be set with an unreleased or beta dependency."
+                    return "Error: $($versionUpdateString.Trim()) is using an unreleased_ or beta_ dependency and trying to set current value. Only dependency versions can be set with an unreleased or beta dependency."
+                }
+                if ($libPerSDHash) {
+                    if (-not $libPerSDHash.Contains($depKey)) {
+                        return "Error: $($versionUpdateString.Trim()) is incorrectly set to $DependencyTypeCurrent. Only libraries building and releasing as part of the same pipeline should be using the $DependencyTypeCurrent versions of each other."
+                    }
                 }
                 if ($versionString -ne $libHash[$depKey].curVer)
                 {
@@ -381,6 +489,9 @@ function Assert-Spring-Sample-Version-Tags {
     Write-Log-Message $potentialLogMessage $hasError
 }
 
+$ArtifactsPerSD = [ordered]@{};
+Get-ArtifactsList-Per-Service-Directory $ArtifactsPerSD
+
 # Create one dependency hashtable for libraries we build (the groupIds will make the entries unique) and
 # one hash for external dependencies
 $libHash = @{}
@@ -550,6 +661,8 @@ Get-ChildItem -Path $Path -Filter pom*.xml -Recurse -File | ForEach-Object {
         }
     }
 
+    $artifactsPerSDHashSet = Get-Artifacts-Built-In-Service-Directory $ArtifactsPerSD $SdkRoot $pomFile
+
     # Verify every dependency as a group, artifact and version
     # GetElementsByTagName should get all dependencies including dependencies under plugins
     foreach($dependencyNode in $xmlPomFile.GetElementsByTagName("dependency"))
@@ -589,7 +702,7 @@ Get-ChildItem -Path $Path -Filter pom*.xml -Recurse -File | ForEach-Object {
             else
             {
                 # verify the version tag and version are correct
-                $retVal = Test-Dependency-Tag-And-Version $libHash $extDepHash $versionNode.InnerText.Trim() $versionNode.NextSibling.Value
+                $retVal = Test-Dependency-Tag-And-Version $libHash $extDepHash $versionNode.InnerText.Trim() $versionNode.NextSibling.Value $artifactsPerSDHashSet
                 if ($retVal)
                 {
                     $hasError = $true
diff --git a/eng/versioning/update_versions.py b/eng/versioning/update_versions.py
index 73072995fabc1..f223ffc06c379 100644
--- a/eng/versioning/update_versions.py
+++ b/eng/versioning/update_versions.py
@@ -235,11 +235,13 @@ def update_versions_all(update_type, build_type, target_file, skip_readme, auto_
     # Load the version and/or external dependency file for the given UpdateType
     # into the verion_map. If UpdateType.all is selected then versions for both
     # the libraries and external dependencies are being updated.
-    version_file = os.path.normpath('eng/versioning/version_' + build_type.name + '.txt')
-    load_version_map_from_file(version_file, version_map)
+    if update_type == UpdateType.library or update_type == UpdateType.all:
+        version_file = os.path.normpath('eng/versioning/version_' + build_type.name + '.txt')
+        load_version_map_from_file(version_file, version_map)
 
-    dependency_file = os.path.normpath('eng/versioning/external_dependencies.txt')
-    load_version_map_from_file(dependency_file, ext_dep_map)
+    if update_type == UpdateType.external_dependency or update_type == UpdateType.all:
+        dependency_file = os.path.normpath('eng/versioning/external_dependencies.txt')
+        load_version_map_from_file(dependency_file, ext_dep_map)
 
     if version_overrides and not version_overrides.startswith('$'):
         # Azure DevOps passes '$(VersionOverrides)' when the variable value is not set
diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt
index c1e0233b53456..4b793ee964b19 100644
--- a/eng/versioning/version_client.txt
+++ b/eng/versioning/version_client.txt
@@ -37,17 +37,17 @@ com.azure:azure-sdk-all;1.0.0;1.0.0
 com.azure:azure-sdk-parent;1.6.0;1.6.0
 com.azure:azure-client-sdk-parent;1.7.0;1.7.0
 com.azure:azure-ai-anomalydetector;3.0.0-beta.5;3.0.0-beta.6
-com.azure:azure-ai-contentsafety;1.0.0;1.1.0-beta.1
-com.azure:azure-ai-documentintelligence;1.0.0-beta.1;1.0.0-beta.2
+com.azure:azure-ai-contentsafety;1.0.1;1.1.0-beta.1
+com.azure:azure-ai-documentintelligence;1.0.0-beta.2;1.0.0-beta.3
 com.azure:azure-ai-documenttranslator;1.0.0-beta.1;1.0.0-beta.2
-com.azure:azure-ai-formrecognizer;4.1.5;4.2.0-beta.1
+com.azure:azure-ai-formrecognizer;4.1.6;4.2.0-beta.1
 com.azure:azure-ai-formrecognizer-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-ai-metricsadvisor;1.1.22;1.2.0-beta.1
+com.azure:azure-ai-metricsadvisor;1.1.23;1.2.0-beta.1
 com.azure:azure-ai-metricsadvisor-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-ai-openai;1.0.0-beta.7;1.0.0-beta.8
 com.azure:azure-ai-openai-assistants;1.0.0-beta.2;1.0.0-beta.3
 com.azure:azure-ai-personalizer;1.0.0-beta.1;1.0.0-beta.2
-com.azure:azure-ai-textanalytics;5.4.3;5.5.0-beta.1
+com.azure:azure-ai-textanalytics;5.4.4;5.5.0-beta.1
 com.azure:azure-ai-textanalytics-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-ai-translation-text;1.0.0-beta.1;1.0.0-beta.2
 com.azure:azure-analytics-defender-easm;1.0.0-beta.1;1.0.0-beta.2
@@ -67,20 +67,20 @@ com.azure:azure-aot-graalvm-support-netty;1.0.0-beta.3;1.0.0-beta.4
 com.azure:azure-aot-graalvm-samples;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-aot-graalvm-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-code-customization-parent;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-communication-callautomation;1.1.2;1.2.0-beta.1
+com.azure:azure-communication-callautomation;1.1.3;1.2.0-beta.1
 com.azure:azure-communication-callingserver;1.0.0-beta.4;1.0.0-beta.5
-com.azure:azure-communication-chat;1.4.1;1.5.0-beta.1
-com.azure:azure-communication-common;1.3.0;1.4.0-beta.1
+com.azure:azure-communication-chat;1.4.2;1.5.0-beta.1
+com.azure:azure-communication-common;1.3.1;1.4.0-beta.1
 com.azure:azure-communication-common-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-communication-email;1.0.10;1.1.0-beta.1
-com.azure:azure-communication-identity;1.5.2;1.6.0-beta.1
-com.azure:azure-communication-jobrouter;1.1.1;1.2.0-beta.1
-com.azure:azure-communication-messages;1.0.0;1.1.0-beta.1
+com.azure:azure-communication-email;1.0.11;1.1.0-beta.1
+com.azure:azure-communication-identity;1.5.3;1.6.0-beta.1
+com.azure:azure-communication-jobrouter;1.1.2;1.2.0-beta.1
+com.azure:azure-communication-messages;1.0.1;1.1.0-beta.1
 com.azure:azure-communication-networktraversal;1.1.0-beta.2;1.1.0-beta.3
-com.azure:azure-communication-phonenumbers;1.1.10;1.2.0-beta.1
-com.azure:azure-communication-rooms;1.0.8;1.1.0-beta.2
-com.azure:azure-communication-sms;1.1.21;1.2.0-beta.1
-com.azure:azure-containers-containerregistry;1.2.5;1.3.0-beta.1
+com.azure:azure-communication-phonenumbers;1.1.11;1.2.0-beta.4
+com.azure:azure-communication-rooms;1.0.9;1.1.0-beta.2
+com.azure:azure-communication-sms;1.1.22;1.2.0-beta.1
+com.azure:azure-containers-containerregistry;1.2.6;1.3.0-beta.1
 com.azure:azure-containers-containerregistry-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-core;1.47.0;1.48.0-beta.1
 com.azure:azure-core-amqp;2.9.2;2.10.0-beta.1
@@ -101,38 +101,38 @@ com.azure:azure-core-test;1.24.0;1.25.0-beta.1
 com.azure:azure-core-tracing-opentelemetry;1.0.0-beta.44;1.0.0-beta.45
 com.azure:azure-core-tracing-opentelemetry-samples;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-core-version-tests;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-cosmos;4.56.0;4.57.0-beta.1
+com.azure:azure-cosmos;4.57.0;4.58.0-beta.1
 com.azure:azure-cosmos-benchmark;4.0.1-beta.1;4.0.1-beta.1
 com.azure:azure-cosmos-dotnet-benchmark;4.0.1-beta.1;4.0.1-beta.1
 com.azure.cosmos.spark:azure-cosmos-spark_3_2-12;1.0.0-beta.1;1.0.0-beta.1
-com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12;4.28.2;4.28.3
-com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12;4.28.2;4.28.3
-com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12;4.28.2;4.28.3
-com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12;4.28.2;4.28.3
-com.azure:azure-cosmos-encryption;2.8.0;2.9.0-beta.1
+com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12;4.28.4;4.29.0-beta.1
+com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12;4.28.4;4.29.0-beta.1
+com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12;4.28.4;4.29.0-beta.1
+com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12;4.28.4;4.29.0-beta.1
+com.azure:azure-cosmos-encryption;2.8.0;2.9.0
 com.azure:azure-cosmos-test;1.0.0-beta.6;1.0.0-beta.7
 com.azure:azure-cosmos-tests;1.0.0-beta.1;1.0.0-beta.1
 com.azure.cosmos.kafka:azure-cosmos-kafka-connect;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-data-appconfiguration;1.5.2;1.6.0-beta.2
+com.azure:azure-data-appconfiguration;1.5.3;1.6.0-beta.2
 com.azure:azure-data-appconfiguration-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-data-schemaregistry;1.4.3;1.5.0-beta.1
-com.azure:azure-data-schemaregistry-apacheavro;1.1.14;1.2.0-beta.2
+com.azure:azure-data-schemaregistry;1.4.4;1.5.0-beta.1
+com.azure:azure-data-schemaregistry-apacheavro;1.1.15;1.2.0-beta.2
 com.azure:azure-data-schemaregistry-jsonschema;1.0.0-beta.1;1.0.0-beta.2
-com.azure:azure-data-tables;12.3.19;12.4.0-beta.1
+com.azure:azure-data-tables;12.3.20;12.4.0-beta.2
 com.azure:azure-data-tables-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-developer-devcenter;1.0.0-beta.3;1.0.0-beta.4
-com.azure:azure-developer-loadtesting;1.0.10;1.1.0-beta.1
-com.azure:azure-digitaltwins-core;1.3.17;1.4.0-beta.1
+com.azure:azure-developer-loadtesting;1.0.11;1.1.0-beta.1
+com.azure:azure-digitaltwins-core;1.3.18;1.4.0-beta.1
 com.azure:azure-e2e;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-health-insights-clinicalmatching;1.0.0-beta.1;1.0.0-beta.2
 com.azure:azure-health-insights-cancerprofiling;1.0.0-beta.1;1.0.0-beta.2
 com.azure:azure-health-insights-radiologyinsights;1.0.0-beta.1;1.0.0-beta.2
-com.azure:azure-identity;1.11.3;1.12.0-beta.2
-com.azure:azure-identity-extensions;1.1.13;1.2.0-beta.2
-com.azure:azure-identity-broker;1.0.3;1.1.0-beta.1
+com.azure:azure-identity;1.11.4;1.12.0-beta.2
+com.azure:azure-identity-extensions;1.1.14;1.2.0-beta.2
+com.azure:azure-identity-broker;1.0.4;1.1.0-beta.1
 com.azure:azure-identity-broker-samples;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-identity-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-iot-deviceupdate;1.0.15;1.1.0-beta.1
+com.azure:azure-iot-deviceupdate;1.0.16;1.1.0-beta.1
 com.azure:azure-iot-modelsrepository;1.0.0-beta.1;1.0.0-beta.2
 com.azure:azure-json;1.1.0;1.2.0-beta.1
 com.azure:azure-json-gson;1.0.0-beta.3;1.0.0-beta.4
@@ -146,55 +146,55 @@ com.azure:azure-maps-render;1.0.0-beta.2;1.0.0-beta.3
 com.azure:azure-maps-route;1.0.0-beta.1;1.0.0-beta.2
 com.azure:azure-maps-search;1.0.0-beta.1;1.0.0-beta.2
 com.azure:azure-media-videoanalyzer-edge;1.0.0-beta.6;1.0.0-beta.7
-com.azure:azure-messaging-eventgrid;4.21.0;4.22.0
+com.azure:azure-messaging-eventgrid;4.22.0;4.23.0-beta.1
 com.azure:azure-messaging-eventgrid-cloudnative-cloudevents;1.0.0-beta.1;1.0.0-beta.2
-com.azure:azure-messaging-eventhubs;5.18.1;5.19.0-beta.1
-com.azure:azure-messaging-eventhubs-checkpointstore-blob;1.19.1;1.20.0-beta.1
+com.azure:azure-messaging-eventhubs;5.18.2;5.19.0-beta.1
+com.azure:azure-messaging-eventhubs-checkpointstore-blob;1.19.2;1.20.0-beta.1
 com.azure:azure-messaging-eventhubs-checkpointstore-jedis;1.0.0-beta.2;1.0.0-beta.3
 com.azure:azure-messaging-eventhubs-stress;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-messaging-eventhubs-track1-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-messaging-eventhubs-track2-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-messaging-servicebus;7.15.1;7.16.0-beta.1
+com.azure:azure-messaging-servicebus;7.15.2;7.16.0-beta.2
 com.azure:azure-messaging-servicebus-stress;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-messaging-servicebus-track1-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-messaging-servicebus-track2-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-messaging-webpubsub;1.2.12;1.3.0-beta.1
-com.azure:azure-messaging-webpubsub-client;1.0.0;1.1.0-beta.1
-com.azure:azure-mixedreality-authentication;1.2.21;1.3.0-beta.1
-com.azure:azure-mixedreality-remoterendering;1.1.26;1.2.0-beta.1
-com.azure:azure-monitor-opentelemetry-exporter;1.0.0-beta.20;1.0.0-beta.21
-com.azure:azure-monitor-ingestion;1.1.4;1.1.5
+com.azure:azure-messaging-webpubsub;1.2.13;1.3.0-beta.1
+com.azure:azure-messaging-webpubsub-client;1.0.1;1.1.0-beta.1
+com.azure:azure-mixedreality-authentication;1.2.22;1.3.0-beta.1
+com.azure:azure-mixedreality-remoterendering;1.1.27;1.2.0-beta.1
+com.azure:azure-monitor-opentelemetry-exporter;1.0.0-beta.21;1.0.0-beta.22
+com.azure:azure-monitor-ingestion;1.1.5;1.2.0-beta.1
 com.azure:azure-monitor-ingestion-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-monitor-query;1.2.9;1.3.0-beta.3
+com.azure:azure-monitor-query;1.2.10;1.3.0-beta.3
 com.azure:azure-monitor-query-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-perf-test-parent;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-quantum-jobs;1.0.0-beta.1;1.0.0-beta.2
-com.azure:azure-search-documents;11.6.2;11.7.0-beta.2
+com.azure:azure-search-documents;11.6.3;11.7.0-beta.3
 com.azure:azure-search-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-security-attestation;1.1.21;1.2.0-beta.1
-com.azure:azure-security-confidentialledger;1.0.17;1.1.0-beta.1
-com.azure:azure-security-keyvault-administration;4.5.0;4.6.0-beta.1
-com.azure:azure-security-keyvault-certificates;4.6.0;4.7.0-beta.1
+com.azure:azure-security-attestation;1.1.22;1.2.0-beta.1
+com.azure:azure-security-confidentialledger;1.0.18;1.1.0-beta.1
+com.azure:azure-security-keyvault-administration;4.5.1;4.6.0-beta.1
+com.azure:azure-security-keyvault-certificates;4.6.1;4.7.0-beta.1
 com.azure:azure-security-keyvault-jca;2.8.1;2.9.0-beta.1
 com.azure:azure-security-test-keyvault-jca;1.0.0;1.0.0
-com.azure:azure-security-keyvault-keys;4.8.0;4.9.0-beta.1
-com.azure:azure-security-keyvault-secrets;4.8.0;4.9.0-beta.1
+com.azure:azure-security-keyvault-keys;4.8.1;4.9.0-beta.1
+com.azure:azure-security-keyvault-secrets;4.8.1;4.9.0-beta.1
 com.azure:azure-security-keyvault-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-sdk-template;1.1.1234;1.2.2-beta.1
 com.azure:azure-sdk-template-two;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-sdk-template-three;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-spring-data-cosmos;3.43.0;3.44.0-beta.1
-com.azure:azure-storage-blob;12.25.2;12.26.0-beta.1
-com.azure:azure-storage-blob-batch;12.21.2;12.22.0-beta.1
+com.azure:azure-storage-blob;12.25.3;12.26.0-beta.1
+com.azure:azure-storage-blob-batch;12.21.3;12.22.0-beta.1
 com.azure:azure-storage-blob-changefeed;12.0.0-beta.19;12.0.0-beta.20
-com.azure:azure-storage-blob-cryptography;12.24.2;12.25.0-beta.1
+com.azure:azure-storage-blob-cryptography;12.24.3;12.25.0-beta.1
 com.azure:azure-storage-blob-nio;12.0.0-beta.20;12.0.0-beta.21
-com.azure:azure-storage-common;12.24.2;12.25.0-beta.1
-com.azure:azure-storage-file-share;12.21.2;12.22.0-beta.1
-com.azure:azure-storage-file-datalake;12.18.2;12.19.0-beta.1
-com.azure:azure-storage-internal-avro;12.10.2;12.11.0-beta.1
+com.azure:azure-storage-common;12.24.3;12.25.0-beta.1
+com.azure:azure-storage-file-share;12.21.3;12.22.0-beta.1
+com.azure:azure-storage-file-datalake;12.18.3;12.19.0-beta.1
+com.azure:azure-storage-internal-avro;12.10.3;12.11.0-beta.1
 com.azure:azure-storage-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure:azure-storage-queue;12.20.2;12.21.0-beta.1
+com.azure:azure-storage-queue;12.20.3;12.21.0-beta.1
 com.azure:azure-template-perf;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-template-stress;1.0.0-beta.1;1.0.0-beta.1
 com.azure:azure-verticals-agrifood-farming;1.0.0-beta.3;1.0.0-beta.4
@@ -255,32 +255,32 @@ com.azure.spring:spring-cloud-azure-stream-binder-eventhubs;4.16.0;4.17.0-beta.1
 com.azure.spring:spring-cloud-azure-stream-binder-servicebus-core;4.16.0;4.17.0-beta.1
 com.azure.spring:spring-cloud-azure-stream-binder-servicebus;4.16.0;4.17.0-beta.1
 com.azure.spring:spring-cloud-azure-trace-sleuth;4.16.0;4.17.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-appplatform;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-appservice;2.36.1;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-authorization;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-cdn;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-compute;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-containerinstance;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-containerregistry;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-containerservice;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-cosmos;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-dns;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-eventhubs;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-keyvault;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-monitor;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-msi;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-network;2.36.0;2.37.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-appplatform;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-appservice;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-authorization;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-cdn;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-compute;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-containerinstance;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-containerregistry;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-containerservice;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-cosmos;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-dns;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-eventhubs;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-keyvault;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-monitor;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-msi;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-network;2.37.0;2.38.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-perf;1.0.0-beta.1;1.0.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-privatedns;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-resources;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-redis;2.36.0;2.37.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-privatedns;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-resources;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-redis;2.37.0;2.38.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-samples;2.0.0-beta.1;2.0.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-search;2.36.0;2.37.0-beta.2
-com.azure.resourcemanager:azure-resourcemanager-servicebus;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-sql;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-storage;2.36.0;2.37.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-trafficmanager;2.36.0;2.37.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-search;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-servicebus;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-sql;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-storage;2.37.0;2.38.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-trafficmanager;2.37.0;2.38.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-test;2.0.0-beta.1;2.0.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-mediaservices;2.3.0;2.4.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-mysql;1.0.2;1.1.0-beta.1
@@ -294,15 +294,15 @@ com.azure.resourcemanager:azure-resourcemanager-kusto;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-loganalytics;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-eventgrid;1.1.0;1.2.0-beta.6
 com.azure.resourcemanager:azure-resourcemanager-healthbot;1.0.0-beta.2;1.0.0-beta.3
-com.azure.resourcemanager:azure-resourcemanager-confluent;1.0.0;1.1.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-confluent;1.1.0;1.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-digitaltwins;1.2.0;1.3.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-netapp;1.0.0;1.1.0-beta.2
+com.azure.resourcemanager:azure-resourcemanager-netapp;1.1.0;1.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-storagecache;1.0.0-beta.10;1.0.0-beta.11
-com.azure.resourcemanager:azure-resourcemanager-redisenterprise;1.0.0;1.1.0-beta.4
+com.azure.resourcemanager:azure-resourcemanager-redisenterprise;2.0.0;2.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-hybridkubernetes;1.0.0-beta.3;1.0.0-beta.4
 com.azure.resourcemanager:azure-resourcemanager-iothub;1.2.0;1.3.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-datadog;1.0.0;1.1.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-communication;2.0.0;2.1.0-beta.3
+com.azure.resourcemanager:azure-resourcemanager-communication;2.1.0;2.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-apimanagement;1.0.0-beta.4;1.0.0-beta.5
 com.azure.resourcemanager:azure-resourcemanager-kubernetesconfiguration;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-resourcegraph;1.0.0;1.1.0-beta.1
@@ -316,7 +316,7 @@ com.azure.resourcemanager:azure-resourcemanager-frontdoor;1.0.0-beta.3;1.0.0-bet
 com.azure.resourcemanager:azure-resourcemanager-mixedreality;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-automation;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-resourcemover;1.1.0;1.2.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-datafactory;1.0.0-beta.26;1.0.0-beta.27
+com.azure.resourcemanager:azure-resourcemanager-datafactory;1.0.0-beta.27;1.0.0-beta.28
 com.azure.resourcemanager:azure-resourcemanager-advisor;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-appconfiguration;1.0.0-beta.7;1.0.0-beta.8
 com.azure.resourcemanager:azure-resourcemanager-attestation;1.0.0-beta.2;1.0.0-beta.3
@@ -344,13 +344,13 @@ com.azure.resourcemanager:azure-resourcemanager-mariadb;1.0.0-beta.2;1.0.0-beta.
 com.azure.resourcemanager:azure-resourcemanager-servicefabric;1.0.0-beta.4;1.0.0-beta.5
 com.azure.resourcemanager:azure-resourcemanager-peering;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-policyinsights;1.0.0-beta.3;1.0.0-beta.4
-com.azure.resourcemanager:azure-resourcemanager-support;1.0.0-beta.3;1.0.0-beta.4
+com.azure.resourcemanager:azure-resourcemanager-support;1.0.0-beta.4;1.0.0-beta.5
 com.azure.resourcemanager:azure-resourcemanager-subscription;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-powerbidedicated;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-storageimportexport;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-applicationinsights;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-deploymentmanager;1.0.0-beta.2;1.0.0-beta.3
-com.azure.resourcemanager:azure-resourcemanager-notificationhubs;1.0.0-beta.3;1.0.0-beta.4
+com.azure.resourcemanager:azure-resourcemanager-notificationhubs;1.0.0-beta.4;1.0.0-beta.5
 com.azure.resourcemanager:azure-resourcemanager-marketplaceordering;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-timeseriesinsights;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-streamanalytics;1.0.0-beta.3;1.0.0-beta.4
@@ -371,7 +371,7 @@ com.azure.resourcemanager:azure-resourcemanager-deviceprovisioningservices;1.0.0
 com.azure.resourcemanager:azure-resourcemanager-postgresqlflexibleserver;1.0.0;1.1.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-elastic;1.0.0-beta.4;1.0.0-beta.5
 com.azure.resourcemanager:azure-resourcemanager-webpubsub;1.0.0;1.1.0-beta.3
-com.azure.resourcemanager:azure-resourcemanager-security;1.0.0-beta.5;1.0.0-beta.6
+com.azure.resourcemanager:azure-resourcemanager-security;1.0.0-beta.6;1.0.0-beta.7
 com.azure.resourcemanager:azure-resourcemanager-azurearcdata;1.0.0-beta.3;1.0.0-beta.4
 com.azure.resourcemanager:azure-resourcemanager-hybridnetwork;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-purview;1.0.0-beta.2;1.0.0-beta.3
@@ -389,11 +389,11 @@ com.azure.resourcemanager:azure-resourcemanager-edgeorder;1.0.0-beta.1;1.0.0-bet
 com.azure.resourcemanager:azure-resourcemanager-securityinsights;1.0.0-beta.4;1.0.0-beta.5
 com.azure.resourcemanager:azure-resourcemanager-oep;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-dnsresolver;1.0.0-beta.2;1.0.0-beta.3
-com.azure.resourcemanager:azure-resourcemanager-mobilenetwork;1.0.0;1.1.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-mobilenetwork;1.1.0;1.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-azureadexternalidentities;1.0.0-beta.1;1.0.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-dashboard;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-servicelinker;1.0.0-beta.2;1.0.0-beta.3
-com.azure.resourcemanager:azure-resourcemanager-appcontainers;1.0.0-beta.6;1.0.0-beta.7
+com.azure.resourcemanager:azure-resourcemanager-appcontainers;1.0.0-beta.7;1.0.0-beta.8
 com.azure.resourcemanager:azure-resourcemanager-scvmm;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-hardwaresecuritymodules;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-workloads;1.0.0;1.1.0-beta.1
@@ -426,17 +426,17 @@ com.azure.resourcemanager:azure-resourcemanager-storagemover;1.1.0;1.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-voiceservices;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-graphservices;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-paloaltonetworks-ngfw;1.1.0;1.2.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-newrelicobservability;1.0.0;1.1.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-newrelicobservability;1.1.0;1.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-qumulo;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-selfhelp;1.0.0;1.1.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-networkcloud;1.0.0;1.1.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-cosmosdbforpostgresql;1.0.0;1.1.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-cosmosdbforpostgresql;1.0.0;1.1.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-managementgroups;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-managednetworkfabric;1.0.0;1.1.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-iotfirmwaredefense;1.0.0-beta.1;1.0.0-beta.2
-com.azure.resourcemanager:azure-resourcemanager-quantum;1.0.0-beta.1;1.0.0-beta.2
+com.azure.resourcemanager:azure-resourcemanager-iotfirmwaredefense;1.0.0;1.1.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-quantum;1.0.0-beta.2;1.0.0-beta.3
 com.azure.resourcemanager:azure-resourcemanager-sphere;1.0.0-beta.1;1.0.0-beta.2
-com.azure.resourcemanager:azure-resourcemanager-chaos;1.0.0;1.1.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-chaos;1.1.0;1.2.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-defendereasm;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-hdinsight-containers;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-apicenter;1.0.0;1.1.0-beta.1
@@ -446,10 +446,10 @@ com.azure.resourcemanager:azure-resourcemanager-recoveryservicesdatareplication;
 com.azure.resourcemanager:azure-resourcemanager-networkanalytics;1.0.0;1.1.0-beta.1
 com.azure.resourcemanager:azure-resourcemanager-springappdiscovery;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-astro;1.0.0-beta.1;1.0.0-beta.2
-com.azure.resourcemanager:azure-resourcemanager-workloadssapvirtualinstance;1.0.0-beta.1;1.0.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-workloadssapvirtualinstance;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-largeinstance;1.0.0-beta.1;1.0.0-beta.2
 com.azure.resourcemanager:azure-resourcemanager-migrationdiscoverysap;1.0.0-beta.1;1.0.0-beta.1
-com.azure.resourcemanager:azure-resourcemanager-storageactions;1.0.0-beta.1;1.0.0-beta.1
+com.azure.resourcemanager:azure-resourcemanager-storageactions;1.0.0-beta.1;1.0.0-beta.2
 com.azure.tools:azure-sdk-archetype;1.0.0;1.2.0-beta.1
 com.azure.tools:azure-sdk-build-tool;1.0.0;1.1.0-beta.1
 
@@ -461,6 +461,8 @@ com.azure.tools:azure-sdk-build-tool;1.0.0;1.1.0-beta.1
 # In the pom, the version update tag after the version should name the unreleased package and the dependency version:
 # 
 
+unreleased_com.azure:azure-xml;1.0.0-beta.4
+unreleased_com.azure:azure-core;1.48.0-beta.1
 
 # Released Beta dependencies: Copy the entry from above, prepend "beta_", remove the current
 # version and set the version to the released beta. Released beta dependencies are only valid
diff --git a/eng/versioning/version_data.txt b/eng/versioning/version_data.txt
index 589f49857f73f..7c4f07c79a22e 100644
--- a/eng/versioning/version_data.txt
+++ b/eng/versioning/version_data.txt
@@ -35,7 +35,6 @@ com.microsoft.azure:azure-keyvault-extensions;1.2.6;1.3.0-beta.1
 com.microsoft.azure:azure-keyvault-test;1.2.3;1.2.6
 com.microsoft.azure:azure-keyvault-webkey;1.2.6;1.3.0-beta.1
 com.microsoft.azure:azure-servicebus;3.6.7;3.7.0-beta.1
-com.microsoft.azure:azure-storage;8.6.5;8.6.5
 com.microsoft.azure:azure-storage-blob;11.0.2;11.0.2
 com.microsoft.azure:azure-eventgrid;1.4.0;1.5.0-beta.1
 com.microsoft.azure:azure-loganalytics;1.0.0-beta-2;1.0.0-beta.2
diff --git a/sdk/advisor/azure-resourcemanager-advisor/pom.xml b/sdk/advisor/azure-resourcemanager-advisor/pom.xml
index 17ea19a1dcc96..ebaeb96a2ed04 100644
--- a/sdk/advisor/azure-resourcemanager-advisor/pom.xml
+++ b/sdk/advisor/azure-resourcemanager-advisor/pom.xml
@@ -60,7 +60,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -86,13 +86,13 @@
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
diff --git a/sdk/agrifood/azure-resourcemanager-agrifood/pom.xml b/sdk/agrifood/azure-resourcemanager-agrifood/pom.xml
index e7612d545fcdf..c435df190f3d0 100644
--- a/sdk/agrifood/azure-resourcemanager-agrifood/pom.xml
+++ b/sdk/agrifood/azure-resourcemanager-agrifood/pom.xml
@@ -60,7 +60,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -80,13 +80,13 @@
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
diff --git a/sdk/agrifood/azure-verticals-agrifood-farming/pom.xml b/sdk/agrifood/azure-verticals-agrifood-farming/pom.xml
index aff45b81ca67f..cb0c8a48eb82f 100644
--- a/sdk/agrifood/azure-verticals-agrifood-farming/pom.xml
+++ b/sdk/agrifood/azure-verticals-agrifood-farming/pom.xml
@@ -84,7 +84,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
diff --git a/sdk/anomalydetector/azure-ai-anomalydetector/pom.xml b/sdk/anomalydetector/azure-ai-anomalydetector/pom.xml
index 898455e7da928..7d40c89884010 100644
--- a/sdk/anomalydetector/azure-ai-anomalydetector/pom.xml
+++ b/sdk/anomalydetector/azure-ai-anomalydetector/pom.xml
@@ -81,7 +81,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
diff --git a/sdk/anomalydetector/azure-ai-anomalydetector/tests.yml b/sdk/anomalydetector/azure-ai-anomalydetector/tests.yml
index 4eb82410ad2c6..fe448ee40a2b9 100644
--- a/sdk/anomalydetector/azure-ai-anomalydetector/tests.yml
+++ b/sdk/anomalydetector/azure-ai-anomalydetector/tests.yml
@@ -1,10 +1,10 @@
 trigger: none
 
-stages:
-  - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml
-    parameters:
-      ServiceDirectory: anomalydetector
-      Artifacts:
-        - name: azure-ai-anomalydetector
-          groupId: com.azure
-          safeName: azureaianomalydetector
+extends:
+  template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml
+  parameters:
+    ServiceDirectory: anomalydetector
+    Artifacts:
+      - name: azure-ai-anomalydetector
+        groupId: com.azure
+        safeName: azureaianomalydetector
diff --git a/sdk/aot/azure-aot-graalvm-samples/pom.xml b/sdk/aot/azure-aot-graalvm-samples/pom.xml
index 13b441e0c399c..1e74700b569b4 100644
--- a/sdk/aot/azure-aot-graalvm-samples/pom.xml
+++ b/sdk/aot/azure-aot-graalvm-samples/pom.xml
@@ -61,54 +61,54 @@
         
             com.azure
             azure-data-appconfiguration
-            1.5.2 
+            1.5.3 
         
         
             com.azure
             azure-identity
-            1.11.3 
+            1.11.4 
         
         
             com.azure
             azure-security-keyvault-keys
-            4.8.0 
+            4.8.1 
         
         
             com.azure
             azure-storage-blob
-            12.25.2 
+            12.25.3 
         
 
         
             com.azure
             azure-security-keyvault-secrets
-            4.8.0 
+            4.8.1 
         
         
             com.azure
             azure-security-keyvault-certificates
-            4.6.0 
+            4.6.1 
         
 
         
             com.azure
             azure-messaging-eventhubs
-            5.18.1 
+            5.18.2 
         
         
             com.azure
             azure-cosmos
-            4.56.0 
+            4.57.0 
         
         
             com.azure
             azure-ai-formrecognizer
-            4.1.5 
+            4.1.6 
         
         
             com.azure
             azure-ai-textanalytics
-            5.4.3 
+            5.4.4 
         
 
 
diff --git a/sdk/aot/azure-aot-graalvm-support-netty/pom.xml b/sdk/aot/azure-aot-graalvm-support-netty/pom.xml
index dca396554fcb3..ee9b3d5f45347 100644
--- a/sdk/aot/azure-aot-graalvm-support-netty/pom.xml
+++ b/sdk/aot/azure-aot-graalvm-support-netty/pom.xml
@@ -66,44 +66,44 @@
         
             io.netty
             netty-handler
-            4.1.101.Final 
+            4.1.107.Final 
         
         
             io.netty
             netty-handler-proxy
-            4.1.101.Final 
+            4.1.107.Final 
         
         
             io.netty
             netty-buffer
-            4.1.101.Final 
+            4.1.107.Final 
         
         
             io.netty
             netty-codec-http
-            4.1.101.Final 
+            4.1.107.Final 
         
 
         
             io.netty
             netty-codec-http2
-            4.1.101.Final 
+            4.1.107.Final 
         
         
             io.netty
             netty-transport-native-unix-common
-            4.1.101.Final 
+            4.1.107.Final 
         
         
             io.netty
             netty-transport-native-epoll
-            4.1.101.Final 
+            4.1.107.Final 
             linux-x86_64
         
         
             io.netty
             netty-transport-native-kqueue
-            4.1.101.Final 
+            4.1.107.Final 
             osx-x86_64
         
     
@@ -118,14 +118,14 @@
                     
                         
                             
-                                io.netty:netty-buffer:[4.1.101.Final] 
-                                io.netty:netty-codec-http:[4.1.101.Final] 
-                                io.netty:netty-codec-http2:[4.1.101.Final] 
-                                io.netty:netty-handler:[4.1.101.Final] 
-                                io.netty:netty-handler-proxy:[4.1.101.Final] 
-                                io.netty:netty-transport-native-unix-common:[4.1.101.Final] 
-                                io.netty:netty-transport-native-epoll:[4.1.101.Final] 
-                                io.netty:netty-transport-native-kqueue:[4.1.101.Final] 
+                                io.netty:netty-buffer:[4.1.107.Final] 
+                                io.netty:netty-codec-http:[4.1.107.Final] 
+                                io.netty:netty-codec-http2:[4.1.107.Final] 
+                                io.netty:netty-handler:[4.1.107.Final] 
+                                io.netty:netty-handler-proxy:[4.1.107.Final] 
+                                io.netty:netty-transport-native-unix-common:[4.1.107.Final] 
+                                io.netty:netty-transport-native-epoll:[4.1.107.Final] 
+                                io.netty:netty-transport-native-kqueue:[4.1.107.Final] 
                             
                         
                     
diff --git a/sdk/aot/ci.yml b/sdk/aot/ci.yml
index 93cd28a0ba58e..0d882bd40ee4f 100644
--- a/sdk/aot/ci.yml
+++ b/sdk/aot/ci.yml
@@ -56,7 +56,5 @@ extends:
         releaseInBatch: ${{ parameters.release_azureaotgraalvmsupportnetty }}
         skipPublishDocMs: true
     AdditionalModules:
-      - name: azure-aot-graalvm-samples
-        groupId: com.azure
       - name: azure-aot-graalvm-perf
         groupId: com.azure
diff --git a/sdk/apicenter/azure-resourcemanager-apicenter/pom.xml b/sdk/apicenter/azure-resourcemanager-apicenter/pom.xml
index 6a6fb2c95b034..7a3f0c9c33f48 100644
--- a/sdk/apicenter/azure-resourcemanager-apicenter/pom.xml
+++ b/sdk/apicenter/azure-resourcemanager-apicenter/pom.xml
@@ -66,7 +66,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -90,13 +90,13 @@
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
diff --git a/sdk/apimanagement/azure-resourcemanager-apimanagement/pom.xml b/sdk/apimanagement/azure-resourcemanager-apimanagement/pom.xml
index 0fbc0fcf52e65..8c8f61121bf1b 100644
--- a/sdk/apimanagement/azure-resourcemanager-apimanagement/pom.xml
+++ b/sdk/apimanagement/azure-resourcemanager-apimanagement/pom.xml
@@ -67,7 +67,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -93,13 +93,13 @@
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
@@ -111,7 +111,7 @@
     
       com.azure.resourcemanager
       azure-resourcemanager-resources
-      2.36.0 
+      2.37.0 
       test
     
   
diff --git a/sdk/apimanagement/tests.mgmt.yml b/sdk/apimanagement/tests.mgmt.yml
index bd0dde5b5e985..bb8cfbf2ab5c5 100644
--- a/sdk/apimanagement/tests.mgmt.yml
+++ b/sdk/apimanagement/tests.mgmt.yml
@@ -2,15 +2,15 @@ trigger: none
 
 pr: none
 
-stages:
-  - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml
-    parameters:
-      ServiceDirectory: apimanagement
-      Artifacts:
-        - name: azure-resourcemanager-apimanagement
-          groupId: com.azure.resourcemanager
-          safeName: azureresourcemanagerapimanagement
-      Clouds: 'Public'
-      # Only run tests on Windows to save cost.
-      MatrixFilters:
-        - pool=.*(win).*
+extends:
+  template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml
+  parameters:
+    ServiceDirectory: apimanagement
+    Artifacts:
+      - name: azure-resourcemanager-apimanagement
+        groupId: com.azure.resourcemanager
+        safeName: azureresourcemanagerapimanagement
+    Clouds: 'Public'
+    # Only run tests on Windows to save cost.
+    MatrixFilters:
+      - pool=.*(win).*
diff --git a/sdk/appcomplianceautomation/azure-resourcemanager-appcomplianceautomation/pom.xml b/sdk/appcomplianceautomation/azure-resourcemanager-appcomplianceautomation/pom.xml
index 4a888b0e51581..1ce46588b5592 100644
--- a/sdk/appcomplianceautomation/azure-resourcemanager-appcomplianceautomation/pom.xml
+++ b/sdk/appcomplianceautomation/azure-resourcemanager-appcomplianceautomation/pom.xml
@@ -60,7 +60,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -86,13 +86,13 @@
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
diff --git a/sdk/appconfiguration/azure-data-appconfiguration-perf/pom.xml b/sdk/appconfiguration/azure-data-appconfiguration-perf/pom.xml
index 0ad8d0262ffc9..63cbb6f53e1ee 100644
--- a/sdk/appconfiguration/azure-data-appconfiguration-perf/pom.xml
+++ b/sdk/appconfiguration/azure-data-appconfiguration-perf/pom.xml
@@ -97,7 +97,7 @@
         
           com.azure
           azure-identity
-          1.11.3  
+          1.11.4  
         
       
     
diff --git a/sdk/appconfiguration/azure-data-appconfiguration/CHANGELOG.md b/sdk/appconfiguration/azure-data-appconfiguration/CHANGELOG.md
index 167897077616b..3346d6f435cac 100644
--- a/sdk/appconfiguration/azure-data-appconfiguration/CHANGELOG.md
+++ b/sdk/appconfiguration/azure-data-appconfiguration/CHANGELOG.md
@@ -10,6 +10,17 @@
 
 ### Other Changes
 
+
+## 1.5.3 (2024-03-20)
+
+### Other Changes
+
+#### Dependency Updates
+
+- Upgraded `azure-core` from `1.46.0` to version `1.47.0`.
+- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`.
+
+
 ## 1.6.0-beta.1 (2024-03-04)
 
 ### Features Added
diff --git a/sdk/appconfiguration/azure-data-appconfiguration/pom.xml b/sdk/appconfiguration/azure-data-appconfiguration/pom.xml
index c2540cee2deaf..1df91c64a5176 100644
--- a/sdk/appconfiguration/azure-data-appconfiguration/pom.xml
+++ b/sdk/appconfiguration/azure-data-appconfiguration/pom.xml
@@ -109,13 +109,13 @@
     
       io.projectreactor
       reactor-test
-      3.4.34 
+      3.4.36 
       test
     
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
   
diff --git a/sdk/appconfiguration/azure-resourcemanager-appconfiguration/pom.xml b/sdk/appconfiguration/azure-resourcemanager-appconfiguration/pom.xml
index 767b7b2ffcb99..0d2899216756e 100644
--- a/sdk/appconfiguration/azure-resourcemanager-appconfiguration/pom.xml
+++ b/sdk/appconfiguration/azure-resourcemanager-appconfiguration/pom.xml
@@ -66,7 +66,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -92,13 +92,13 @@
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
diff --git a/sdk/appconfiguration/ci.yml b/sdk/appconfiguration/ci.yml
index 373875154104b..1b7afb1c43e37 100644
--- a/sdk/appconfiguration/ci.yml
+++ b/sdk/appconfiguration/ci.yml
@@ -62,3 +62,8 @@ extends:
         groupId: com.azure.resourcemanager
         safeName: azureresourcemanagerappconfiguration
         releaseInBatch: ${{ parameters.release_azureresourcemanagerappconfiguration }}
+    AdditionalModules:
+      - name: perf-test-core
+        groupId: com.azure
+      - name: azure-aot-graalvm-perf
+        groupId: com.azure
diff --git a/sdk/appconfiguration/tests.native.yml b/sdk/appconfiguration/tests.native.yml
index 00ba3ca33a0bd..734347461fe5f 100644
--- a/sdk/appconfiguration/tests.native.yml
+++ b/sdk/appconfiguration/tests.native.yml
@@ -9,22 +9,19 @@ pr:
       - eng/pipelines/native-image-live-tests.yml
       - eng/scrips/Install-GraalVM-JDK.ps1
 
-variables:
-  - template: /eng/pipelines/templates/variables/globals.yml
-
-stages:
-  - template: /eng/pipelines/templates/stages/archetype-sdk-native-tests.yml
-    parameters:
-      ServiceDirectory: appconfiguration
-      Artifacts:
-      - name: azure-data-appconfiguration
-        groupId: com.azure
-        safeName: azuredataappconfiguration
-      TimeoutInMinutes: 90
-      SupportedClouds: 'Public'
-      EnvVars:
-        AZURE_CLIENT_ID: $(APPCONFIGURATION_CLIENT_ID)
-        AZURE_CLIENT_SECRET: $(APPCONFIGURATION_CLIENT_SECRET)
-        AZURE_TENANT_ID: $(APPCONFIGURATION_TENANT_ID)
-      TestGoals: 'verify'
-      TestOptions: '-DskipSpringITs=false'
+extends:
+  template: /eng/pipelines/templates/stages/archetype-sdk-native-tests.yml
+  parameters:
+    ServiceDirectory: appconfiguration
+    Artifacts:
+    - name: azure-data-appconfiguration
+      groupId: com.azure
+      safeName: azuredataappconfiguration
+    TimeoutInMinutes: 90
+    SupportedClouds: 'Public'
+    EnvVars:
+      AZURE_CLIENT_ID: $(APPCONFIGURATION_CLIENT_ID)
+      AZURE_CLIENT_SECRET: $(APPCONFIGURATION_CLIENT_SECRET)
+      AZURE_TENANT_ID: $(APPCONFIGURATION_TENANT_ID)
+    TestGoals: 'verify'
+    TestOptions: '-DskipSpringITs=false'
diff --git a/sdk/appconfiguration/tests.yml b/sdk/appconfiguration/tests.yml
index 5eebe05d80dec..fb162ada4dfcd 100644
--- a/sdk/appconfiguration/tests.yml
+++ b/sdk/appconfiguration/tests.yml
@@ -1,16 +1,16 @@
 trigger: none
 
-stages:
-  - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml
-    parameters:
-      ServiceDirectory: appconfiguration
-      Artifacts:
-        - name: azure-data-appconfiguration
-          groupId: com.azure
-          safeName: azuredataappconfiguration
-      TimeoutInMinutes: 90
-      SupportedClouds: "Public,UsGov,China"
-      EnvVars:
-        AZURE_CLIENT_ID: $(APPCONFIGURATION_CLIENT_ID)
-        AZURE_CLIENT_SECRET: $(APPCONFIGURATION_CLIENT_SECRET)
-        AZURE_TENANT_ID: $(APPCONFIGURATION_TENANT_ID)
+extends:
+  template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml
+  parameters:
+    ServiceDirectory: appconfiguration
+    Artifacts:
+      - name: azure-data-appconfiguration
+        groupId: com.azure
+        safeName: azuredataappconfiguration
+    TimeoutInMinutes: 90
+    SupportedClouds: "Public,UsGov,China"
+    EnvVars:
+      AZURE_CLIENT_ID: $(APPCONFIGURATION_CLIENT_ID)
+      AZURE_CLIENT_SECRET: $(APPCONFIGURATION_CLIENT_SECRET)
+      AZURE_TENANT_ID: $(APPCONFIGURATION_TENANT_ID)
diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/CHANGELOG.md b/sdk/appcontainers/azure-resourcemanager-appcontainers/CHANGELOG.md
index 92cf4d5397681..2d8837938fff7 100644
--- a/sdk/appcontainers/azure-resourcemanager-appcontainers/CHANGELOG.md
+++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/CHANGELOG.md
@@ -1,6 +1,6 @@
 # Release History
 
-## 1.0.0-beta.7 (Unreleased)
+## 1.0.0-beta.8 (Unreleased)
 
 ### Features Added
 
@@ -10,6 +10,405 @@
 
 ### Other Changes
 
+## 1.0.0-beta.7 (2024-03-20)
+
+- Azure Resource Manager ContainerAppsApi client library for Java. This package contains Microsoft Azure SDK for ContainerAppsApi Management SDK.  Package tag package-preview-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt).
+
+### Features Added
+
+* `models.DotNetComponent$UpdateStages` was added
+
+* `models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration` was added
+
+* `models.HttpConnectionPool` was added
+
+* `models.Usage` was added
+
+* `models.BuildResource$UpdateStages` was added
+
+* `models.DotNetComponents` was added
+
+* `models.BuildProvisioningState` was added
+
+* `models.AppResiliency` was added
+
+* `models.DaprSubscriptions` was added
+
+* `models.BuilderResource$Update` was added
+
+* `models.BuildConfiguration` was added
+
+* `models.BuildResource$Update` was added
+
+* `models.CertificateKeyVaultProperties` was added
+
+* `models.EnvironmentVariable` was added
+
+* `models.DotNetComponentServiceBind` was added
+
+* `models.AppResiliencies` was added
+
+* `models.JavaComponentServiceBind` was added
+
+* `models.LogsConfiguration` was added
+
+* `models.Usages` was added
+
+* `models.DotNetComponent$Update` was added
+
+* `models.UsageName` was added
+
+* `models.DaprSubscription$UpdateStages` was added
+
+* `models.DotNetComponent$DefinitionStages` was added
+
+* `models.HttpGet` was added
+
+* `models.JavaComponentConfigurationProperty` was added
+
+* `models.ListUsagesResult` was added
+
+* `models.DotNetComponentProvisioningState` was added
+
+* `models.DotNetComponentType` was added
+
+* `models.BuilderResourceUpdate` was added
+
+* `models.AppResiliency$DefinitionStages` was added
+
+* `models.DaprComponentResiliencyPoliciesCollection` was added
+
+* `models.BuildAuthTokens` was added
+
+* `models.ContainerRegistry` was added
+
+* `models.IngressTargetPortHttpScheme` was added
+
+* `models.AppResiliency$Definition` was added
+
+* `models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration` was added
+
+* `models.DaprSubscription$DefinitionStages` was added
+
+* `models.OtlpConfiguration` was added
+
+* `models.MetricsConfiguration` was added
+
+* `models.DaprServiceBindMetadata` was added
+
+* `models.TracesConfiguration` was added
+
+* `models.DaprSubscriptionBulkSubscribeOptions` was added
+
+* `models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration` was added
+
+* `models.JavaComponent$Definition` was added
+
+* `models.BuildsByBuilderResources` was added
+
+* `models.BuildResource` was added
+
+* `models.TimeoutPolicy` was added
+
+* `models.DotNetComponentConfigurationProperty` was added
+
+* `models.DaprSubscriptionsCollection` was added
+
+* `models.TokenStore` was added
+
+* `models.AppInsightsConfiguration` was added
+
+* `models.DaprSubscriptionRoutes` was added
+
+* `models.DaprComponentResiliencyPolicy$Definition` was added
+
+* `models.HttpRetryPolicy` was added
+
+* `models.NfsAzureFileProperties` was added
+
+* `models.CircuitBreakerPolicy` was added
+
+* `models.DaprComponentResiliencyPolicy$DefinitionStages` was added
+
+* `models.TcpRetryPolicy` was added
+
+* `models.DaprComponentResiliencyPolicies` was added
+
+* `models.HeaderMatch` was added
+
+* `models.AppResiliencyCollection` was added
+
+* `models.DotNetComponent` was added
+
+* `models.BuilderCollection` was added
+
+* `models.BuildStatus` was added
+
+* `models.DaprSubscription` was added
+
+* `models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration` was added
+
+* `models.BuildToken` was added
+
+* `models.DaprComponentResiliencyPolicyConfiguration` was added
+
+* `models.BuilderResource$DefinitionStages` was added
+
+* `models.JavaComponentsCollection` was added
+
+* `models.OpenTelemetryConfiguration` was added
+
+* `models.Builders` was added
+
+* `models.DaprComponentResiliencyPolicy$Update` was added
+
+* `models.BuildResource$DefinitionStages` was added
+
+* `models.JavaComponent` was added
+
+* `models.AppResiliency$Update` was added
+
+* `models.DaprSubscription$Update` was added
+
+* `models.DestinationsConfiguration` was added
+
+* `models.CertificateType` was added
+
+* `models.DataDogConfiguration` was added
+
+* `models.DotNetComponentsCollection` was added
+
+* `models.ManagedEnvironmentUsages` was added
+
+* `models.DotNetComponent$Definition` was added
+
+* `models.EncryptionSettings` was added
+
+* `models.DaprComponentResiliencyPolicy$UpdateStages` was added
+
+* `models.BuildCollection` was added
+
+* `models.JavaComponent$UpdateStages` was added
+
+* `models.JavaComponentProvisioningState` was added
+
+* `models.PreBuildStep` was added
+
+* `models.DaprComponentServiceBinding` was added
+
+* `models.BuilderResource$UpdateStages` was added
+
+* `models.JavaComponent$DefinitionStages` was added
+
+* `models.DaprComponentResiliencyPolicy` was added
+
+* `models.IngressPortMapping` was added
+
+* `models.ContainerRegistryWithCustomImage` was added
+
+* `models.BlobStorageTokenStore` was added
+
+* `models.JavaComponentType` was added
+
+* `models.AppResiliency$UpdateStages` was added
+
+* `models.BuilderResource` was added
+
+* `models.BuildResource$Definition` was added
+
+* `models.Builds` was added
+
+* `models.BuilderResource$Definition` was added
+
+* `models.JavaComponent$Update` was added
+
+* `models.Header` was added
+
+* `models.TcpConnectionPool` was added
+
+* `models.JavaComponents` was added
+
+* `models.DaprSubscriptionRouteRule` was added
+
+* `models.DaprSubscription$Definition` was added
+
+* `models.BuilderProvisioningState` was added
+
+#### `models.Replica` was modified
+
+* `systemData()` was added
+
+#### `models.ServiceBind` was modified
+
+* `withCustomizedKeys(java.util.Map)` was added
+* `clientType()` was added
+* `customizedKeys()` was added
+* `withClientType(java.lang.String)` was added
+
+#### `models.Job` was modified
+
+* `extendedLocation()` was added
+
+#### `models.DaprComponent$Definition` was modified
+
+* `withServiceComponentBind(java.util.List)` was added
+
+#### `models.ManagedEnvironment$Update` was modified
+
+* `withOpenTelemetryConfiguration(models.OpenTelemetryConfiguration)` was added
+* `withAppInsightsConfiguration(models.AppInsightsConfiguration)` was added
+* `withIdentity(models.ManagedServiceIdentity)` was added
+
+#### `models.Revision` was modified
+
+* `systemData()` was added
+
+#### `models.AvailableWorkloadProfileProperties` was modified
+
+* `withGpus(java.lang.Integer)` was added
+* `gpus()` was added
+
+#### `models.Job$Definition` was modified
+
+* `withExtendedLocation(models.ExtendedLocation)` was added
+
+#### `models.Jobs` was modified
+
+* `listDetectors(java.lang.String,java.lang.String)` was added
+* `listDetectorsWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added
+* `proxyGet(java.lang.String,java.lang.String)` was added
+* `getDetectorWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added
+* `proxyGetWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added
+* `getDetector(java.lang.String,java.lang.String,java.lang.String)` was added
+
+#### `models.DaprComponent$Update` was modified
+
+* `withServiceComponentBind(java.util.List)` was added
+
+#### `models.ManagedEnvironment$Definition` was modified
+
+* `withOpenTelemetryConfiguration(models.OpenTelemetryConfiguration)` was added
+* `withIdentity(models.ManagedServiceIdentity)` was added
+* `withAppInsightsConfiguration(models.AppInsightsConfiguration)` was added
+
+#### `models.ManagedEnvironmentStorageProperties` was modified
+
+* `nfsAzureFile()` was added
+* `withNfsAzureFile(models.NfsAzureFileProperties)` was added
+
+#### `models.BillingMeter` was modified
+
+* `systemData()` was added
+
+#### `models.GithubActionConfiguration` was modified
+
+* `buildEnvironmentVariables()` was added
+* `withBuildEnvironmentVariables(java.util.List)` was added
+
+#### `models.DaprComponent` was modified
+
+* `serviceComponentBind()` was added
+* `systemData()` was added
+
+#### `models.ManagedEnvironmentStorage` was modified
+
+* `systemData()` was added
+
+#### `models.LogAnalyticsConfiguration` was modified
+
+* `dynamicJsonColumns()` was added
+* `withDynamicJsonColumns(java.lang.Boolean)` was added
+
+#### `models.Job$Update` was modified
+
+* `withExtendedLocation(models.ExtendedLocation)` was added
+
+#### `models.ManagedEnvironment` was modified
+
+* `appInsightsConfiguration()` was added
+* `identity()` was added
+* `openTelemetryConfiguration()` was added
+
+#### `models.WorkloadProfileStates` was modified
+
+* `systemData()` was added
+
+#### `models.Login` was modified
+
+* `tokenStore()` was added
+* `withTokenStore(models.TokenStore)` was added
+
+#### `models.ResourceProviders` was modified
+
+* `getCustomDomainVerificationId()` was added
+* `getCustomDomainVerificationIdWithResponse(com.azure.core.util.Context)` was added
+
+#### `models.JobPatchProperties` was modified
+
+* `withExtendedLocation(models.ExtendedLocation)` was added
+* `extendedLocation()` was added
+
+#### `models.SourceControl` was modified
+
+* `systemData()` was added
+
+#### `models.CertificateProperties` was modified
+
+* `withCertificateType(models.CertificateType)` was added
+* `certificateType()` was added
+* `withCertificateKeyVaultProperties(models.CertificateKeyVaultProperties)` was added
+* `certificateKeyVaultProperties()` was added
+
+#### `models.Diagnostics` was modified
+
+* `systemData()` was added
+
+#### `models.AvailableWorkloadProfile` was modified
+
+* `systemData()` was added
+
+#### `models.AuthConfig` was modified
+
+* `encryptionSettings()` was added
+
+#### `ContainerAppsApiManager` was modified
+
+* `dotNetComponents()` was added
+* `builds()` was added
+* `buildsByBuilderResources()` was added
+* `javaComponents()` was added
+* `usages()` was added
+* `daprSubscriptions()` was added
+* `daprComponentResiliencyPolicies()` was added
+* `buildAuthTokens()` was added
+* `appResiliencies()` was added
+* `managedEnvironmentUsages()` was added
+* `builders()` was added
+
+#### `models.ConnectedEnvironmentStorage` was modified
+
+* `systemData()` was added
+
+#### `models.AuthConfig$Update` was modified
+
+* `withEncryptionSettings(models.EncryptionSettings)` was added
+
+#### `models.Ingress` was modified
+
+* `targetPortHttpScheme()` was added
+* `additionalPortMappings()` was added
+* `withAdditionalPortMappings(java.util.List)` was added
+* `withTargetPortHttpScheme(models.IngressTargetPortHttpScheme)` was added
+
+#### `models.CustomDomainConfiguration` was modified
+
+* `certificateKeyVaultProperties()` was added
+* `withCertificateKeyVaultProperties(models.CertificateKeyVaultProperties)` was added
+
+#### `models.AuthConfig$Definition` was modified
+
+* `withEncryptionSettings(models.EncryptionSettings)` was added
+
 ## 1.0.0-beta.6 (2023-08-21)
 
 - Azure Resource Manager ContainerAppsApi client library for Java. This package contains Microsoft Azure SDK for ContainerAppsApi Management SDK.  Package tag package-2023-05. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt).
diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/README.md b/sdk/appcontainers/azure-resourcemanager-appcontainers/README.md
index 7e58ce2f26aff..59a801fc4d049 100644
--- a/sdk/appcontainers/azure-resourcemanager-appcontainers/README.md
+++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/README.md
@@ -2,7 +2,7 @@
 
 Azure Resource Manager ContainerAppsApi client library for Java.
 
-This package contains Microsoft Azure SDK for ContainerAppsApi Management SDK.  Package tag package-2023-05. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt).
+This package contains Microsoft Azure SDK for ContainerAppsApi Management SDK.  Package tag package-preview-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt).
 
 ## We'd love to hear your feedback
 
@@ -32,7 +32,7 @@ Various documentation is available to help you get started
 
     com.azure.resourcemanager
     azure-resourcemanager-appcontainers
-    1.0.0-beta.6
+    1.0.0-beta.7
 
 ```
 [//]: # ({x-version-update-end})
@@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen
 
 ### Authentication
 
-By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables.
+By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables.
 
 - `AZURE_CLIENT_ID` for Azure client ID.
 - `AZURE_TENANT_ID` for Azure tenant ID.
@@ -94,7 +94,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m
 
 [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS
 [docs]: https://azure.github.io/azure-sdk-for-java/
-[jdk]: https://docs.microsoft.com/java/azure/jdk/
+[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/
 [azure_subscription]: https://azure.microsoft.com/free/
 [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity
 [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty
diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/SAMPLE.md b/sdk/appcontainers/azure-resourcemanager-appcontainers/SAMPLE.md
index 961f0002cf980..bb9e2e0bf3b20 100644
--- a/sdk/appcontainers/azure-resourcemanager-appcontainers/SAMPLE.md
+++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/SAMPLE.md
@@ -1,6 +1,14 @@
 # Code snippets and samples
 
 
+## AppResiliency
+
+- [CreateOrUpdate](#appresiliency_createorupdate)
+- [Delete](#appresiliency_delete)
+- [Get](#appresiliency_get)
+- [List](#appresiliency_list)
+- [Update](#appresiliency_update)
+
 ## AvailableWorkloadProfiles
 
 - [Get](#availableworkloadprofiles_get)
@@ -9,6 +17,29 @@
 
 - [Get](#billingmeters_get)
 
+## BuildAuthToken
+
+- [List](#buildauthtoken_list)
+
+## Builders
+
+- [CreateOrUpdate](#builders_createorupdate)
+- [Delete](#builders_delete)
+- [GetByResourceGroup](#builders_getbyresourcegroup)
+- [List](#builders_list)
+- [ListByResourceGroup](#builders_listbyresourcegroup)
+- [Update](#builders_update)
+
+## Builds
+
+- [CreateOrUpdate](#builds_createorupdate)
+- [Delete](#builds_delete)
+- [Get](#builds_get)
+
+## BuildsByBuilderResource
+
+- [List](#buildsbybuilderresource_list)
+
 ## Certificates
 
 - [CreateOrUpdate](#certificates_createorupdate)
@@ -99,6 +130,13 @@
 - [Get](#containerappssourcecontrols_get)
 - [ListByContainerApp](#containerappssourcecontrols_listbycontainerapp)
 
+## DaprComponentResiliencyPolicies
+
+- [CreateOrUpdate](#daprcomponentresiliencypolicies_createorupdate)
+- [Delete](#daprcomponentresiliencypolicies_delete)
+- [Get](#daprcomponentresiliencypolicies_get)
+- [List](#daprcomponentresiliencypolicies_list)
+
 ## DaprComponents
 
 - [CreateOrUpdate](#daprcomponents_createorupdate)
@@ -107,14 +145,40 @@
 - [List](#daprcomponents_list)
 - [ListSecrets](#daprcomponents_listsecrets)
 
+## DaprSubscriptions
+
+- [CreateOrUpdate](#daprsubscriptions_createorupdate)
+- [Delete](#daprsubscriptions_delete)
+- [Get](#daprsubscriptions_get)
+- [List](#daprsubscriptions_list)
+
+## DotNetComponents
+
+- [CreateOrUpdate](#dotnetcomponents_createorupdate)
+- [Delete](#dotnetcomponents_delete)
+- [Get](#dotnetcomponents_get)
+- [List](#dotnetcomponents_list)
+- [Update](#dotnetcomponents_update)
+
+## JavaComponents
+
+- [CreateOrUpdate](#javacomponents_createorupdate)
+- [Delete](#javacomponents_delete)
+- [Get](#javacomponents_get)
+- [List](#javacomponents_list)
+- [Update](#javacomponents_update)
+
 ## Jobs
 
 - [CreateOrUpdate](#jobs_createorupdate)
 - [Delete](#jobs_delete)
 - [GetByResourceGroup](#jobs_getbyresourcegroup)
+- [GetDetector](#jobs_getdetector)
 - [List](#jobs_list)
 - [ListByResourceGroup](#jobs_listbyresourcegroup)
+- [ListDetectors](#jobs_listdetectors)
 - [ListSecrets](#jobs_listsecrets)
+- [ProxyGet](#jobs_proxyget)
 - [Start](#jobs_start)
 - [StopExecution](#jobs_stopexecution)
 - [StopMultipleExecutions](#jobs_stopmultipleexecutions)
@@ -137,6 +201,10 @@
 - [GetDetector](#managedenvironmentdiagnostics_getdetector)
 - [ListDetectors](#managedenvironmentdiagnostics_listdetectors)
 
+## ManagedEnvironmentUsages
+
+- [List](#managedenvironmentusages_list)
+
 ## ManagedEnvironments
 
 - [CreateOrUpdate](#managedenvironments_createorupdate)
@@ -169,170 +237,263 @@
 
 ## ResourceProvider
 
+- [GetCustomDomainVerificationId](#resourceprovider_getcustomdomainverificationid)
 - [JobExecution](#resourceprovider_jobexecution)
-### AvailableWorkloadProfiles_Get
+
+## Usages
+
+- [List](#usages_list)
+### AppResiliency_CreateOrUpdate
 
 ```java
-/** Samples for AvailableWorkloadProfiles Get. */
-public final class AvailableWorkloadProfilesGetSamples {
+import com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy;
+import com.azure.resourcemanager.appcontainers.models.HeaderMatch;
+import com.azure.resourcemanager.appcontainers.models.HttpConnectionPool;
+import com.azure.resourcemanager.appcontainers.models.HttpRetryPolicy;
+import com.azure.resourcemanager.appcontainers.models.TcpConnectionPool;
+import com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy;
+import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy;
+import java.util.Arrays;
+
+/**
+ * Samples for AppResiliency CreateOrUpdate.
+ */
+public final class AppResiliencyCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AvailableWorkloadProfiles_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_CreateOrUpdate
+     * .json
      */
     /**
-     * Sample code: BillingMeters_Get.
-     *
+     * Sample code: Create or Update App Resiliency.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void billingMetersGet(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.availableWorkloadProfiles().get("East US", com.azure.core.util.Context.NONE);
+    public static void
+        createOrUpdateAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.appResiliencies().define("resiliency-policy-1").withExistingContainerApp("rg", "testcontainerApp0")
+            .withTimeoutPolicy(new TimeoutPolicy().withResponseTimeoutInSeconds(15).withConnectionTimeoutInSeconds(5))
+            .withHttpRetryPolicy(new HttpRetryPolicy().withMaxRetries(5).withInitialDelayInMilliseconds(1000L)
+                .withMaxIntervalInMilliseconds(10000L)
+                .withHeaders(
+                    Arrays.asList(new HeaderMatch().withHeaderProperty("X-Content-Type").withPrefixMatch("GOATS")))
+                .withHttpStatusCodes(Arrays.asList(502, 503)).withErrors(
+                    Arrays.asList("5xx", "connect-failure", "reset", "retriable-headers", "retriable-status-codes")))
+            .withTcpRetryPolicy(new TcpRetryPolicy().withMaxConnectAttempts(3))
+            .withCircuitBreakerPolicy(new CircuitBreakerPolicy().withConsecutiveErrors(5).withIntervalInSeconds(10)
+                .withMaxEjectionPercent(50))
+            .withHttpConnectionPool(
+                new HttpConnectionPool().withHttp1MaxPendingRequests(1024).withHttp2MaxRequests(1024))
+            .withTcpConnectionPool(new TcpConnectionPool().withMaxConnections(100)).create();
     }
 }
 ```
 
-### BillingMeters_Get
+### AppResiliency_Delete
 
 ```java
-/** Samples for BillingMeters Get. */
-public final class BillingMetersGetSamples {
+/**
+ * Samples for AppResiliency Delete.
+ */
+public final class AppResiliencyDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/BillingMeters_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_Delete.json
      */
     /**
-     * Sample code: BillingMeters_Get.
-     *
+     * Sample code: Delete App Resiliency.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void billingMetersGet(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.billingMeters().getWithResponse("East US", com.azure.core.util.Context.NONE);
+    public static void deleteAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.appResiliencies().deleteWithResponse("rg", "testcontainerApp0", "resiliency-policy-1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### Certificates_CreateOrUpdate
+### AppResiliency_Get
 
 ```java
-import com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner;
-import com.azure.resourcemanager.appcontainers.models.CertificateProperties;
+/**
+ * Samples for AppResiliency Get.
+ */
+public final class AppResiliencyGetSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_Get.json
+     */
+    /**
+     * Sample code: Get App Resiliency.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.appResiliencies().getWithResponse("rg", "testcontainerApp0", "resiliency-policy-1",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
 
-/** Samples for Certificates CreateOrUpdate. */
-public final class CertificatesCreateOrUpdateSamples {
+### AppResiliency_List
+
+```java
+/**
+ * Samples for AppResiliency List.
+ */
+public final class AppResiliencyListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificate_CreateOrUpdate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_List.json
      */
     /**
-     * Sample code: Create or Update Certificate.
-     *
+     * Sample code: List App Resiliency.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateCertificate(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .certificates()
-            .createOrUpdateWithResponse(
-                "examplerg",
-                "testcontainerenv",
-                "certificate-firendly-name",
-                new CertificateInner()
-                    .withLocation("East US")
-                    .withProperties(
-                        new CertificateProperties()
-                            .withPassword("fakeTokenPlaceholder")
-                            .withValue("Y2VydA==".getBytes())),
-                com.azure.core.util.Context.NONE);
+    public static void listAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.appResiliencies().list("rg", "testcontainerApp0", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### Certificates_Delete
+### AppResiliency_Update
 
 ```java
-/** Samples for Certificates Delete. */
-public final class CertificatesDeleteSamples {
+import com.azure.resourcemanager.appcontainers.models.AppResiliency;
+import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy;
+
+/**
+ * Samples for AppResiliency Update.
+ */
+public final class AppResiliencyUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificate_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_Patch.json
      */
     /**
-     * Sample code: Delete Certificate.
-     *
+     * Sample code: Update App Resiliency.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .certificates()
-            .deleteWithResponse(
-                "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE);
+    public static void updateAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        AppResiliency resource = manager.appResiliencies()
+            .getWithResponse("rg", "testcontainerApp0", "resiliency-policy-1", com.azure.core.util.Context.NONE)
+            .getValue();
+        resource.update()
+            .withTimeoutPolicy(new TimeoutPolicy().withResponseTimeoutInSeconds(30).withConnectionTimeoutInSeconds(40))
+            .apply();
     }
 }
 ```
 
-### Certificates_Get
+### AvailableWorkloadProfiles_Get
 
 ```java
-/** Samples for Certificates Get. */
-public final class CertificatesGetSamples {
+/**
+ * Samples for AvailableWorkloadProfiles Get.
+ */
+public final class AvailableWorkloadProfilesGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificate_Get.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * AvailableWorkloadProfiles_Get.json
      */
     /**
-     * Sample code: Get Certificate.
-     *
+     * Sample code: BillingMeters_Get.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .certificates()
-            .getWithResponse(
-                "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE);
+    public static void billingMetersGet(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.availableWorkloadProfiles().get("East US", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### Certificates_List
+### BillingMeters_Get
 
 ```java
-/** Samples for Certificates List. */
-public final class CertificatesListSamples {
+/**
+ * Samples for BillingMeters Get.
+ */
+public final class BillingMetersGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificates_ListByManagedEnvironment.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/BillingMeters_Get.json
      */
     /**
-     * Sample code: List Certificates by Managed Environment.
-     *
+     * Sample code: BillingMeters_Get.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listCertificatesByManagedEnvironment(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.certificates().list("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE);
+    public static void billingMetersGet(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.billingMeters().getWithResponse("East US", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### Certificates_Update
+### BuildAuthToken_List
 
 ```java
-import com.azure.resourcemanager.appcontainers.models.CertificatePatch;
+/**
+ * Samples for BuildAuthToken List.
+ */
+public final class BuildAuthTokenListSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_ListAuthToken.json
+     */
+    /**
+     * Sample code: Get Build Auth Token.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getBuildAuthToken(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.buildAuthTokens().listWithResponse("rg", "testBuilder", "testBuild", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### Builders_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.ContainerRegistry;
+import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity;
+import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType;
+import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
-/** Samples for Certificates Update. */
-public final class CertificatesUpdateSamples {
+/**
+ * Samples for Builders CreateOrUpdate.
+ */
+public final class BuildersCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificates_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_CreateOrUpdate.json
      */
     /**
-     * Sample code: Patch Certificate.
-     *
+     * Sample code: Builders_CreateOrUpdate_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void patchCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .certificates()
-            .updateWithResponse(
-                "examplerg",
-                "testcontainerenv",
-                "certificate-firendly-name",
-                new CertificatePatch().withTags(mapOf("tag1", "value1", "tag2", "value2")),
-                com.azure.core.util.Context.NONE);
+    public static void
+        buildersCreateOrUpdate0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builders().define("testBuilder").withRegion("eastus").withExistingResourceGroup("rg")
+            .withTags(mapOf("company", "Microsoft"))
+            .withIdentity(new ManagedServiceIdentity()
+                .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED)
+                .withUserAssignedIdentities(mapOf(
+                    "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/identity1",
+                    new UserAssignedIdentity())))
+            .withEnvironmentId(
+                "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg/providers/Microsoft.App/managedEnvironments/testEnv")
+            .withContainerRegistries(Arrays.asList(
+                new ContainerRegistry().withContainerRegistryServer("test.azurecr.io").withIdentityResourceId(
+                    "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/identity1"),
+                new ContainerRegistry().withContainerRegistryServer("test2.azurecr.io").withIdentityResourceId(
+                    "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/identity1")))
+            .create();
     }
 
     // Use "Map.of" if available
@@ -349,292 +510,414 @@ public final class CertificatesUpdateSamples {
 }
 ```
 
-### ConnectedEnvironments_CheckNameAvailability
+### Builders_Delete
 
 ```java
-import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest;
+/**
+ * Samples for Builders Delete.
+ */
+public final class BuildersDeleteSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_Delete.json
+     */
+    /**
+     * Sample code: Builders_Delete_0.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void buildersDelete0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builders().delete("rg", "testBuilder", com.azure.core.util.Context.NONE);
+    }
+}
+```
 
-/** Samples for ConnectedEnvironments CheckNameAvailability. */
-public final class ConnectedEnvironmentsCheckNameAvailabilitySamples {
+### Builders_GetByResourceGroup
+
+```java
+/**
+ * Samples for Builders GetByResourceGroup.
+ */
+public final class BuildersGetByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificates_CheckNameAvailability.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_Get.json
      */
     /**
-     * Sample code: Certificates_CheckNameAvailability.
-     *
+     * Sample code: Builders_Get_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void certificatesCheckNameAvailability(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironments()
-            .checkNameAvailabilityWithResponse(
-                "examplerg",
-                "testcontainerenv",
-                new CheckNameAvailabilityRequest()
-                    .withName("testcertificatename")
-                    .withType("Microsoft.App/connectedEnvironments/certificates"),
-                com.azure.core.util.Context.NONE);
+    public static void buildersGet0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builders().getByResourceGroupWithResponse("rg", "testBuilder", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironments_CreateOrUpdate
+### Builders_List
 
 ```java
-import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration;
+/**
+ * Samples for Builders List.
+ */
+public final class BuildersListSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_ListBySubscription.
+     * json
+     */
+    /**
+     * Sample code: Builders_ListBySubscription_0.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        buildersListBySubscription0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builders().list(com.azure.core.util.Context.NONE);
+    }
+}
+```
 
-/** Samples for ConnectedEnvironments CreateOrUpdate. */
-public final class ConnectedEnvironmentsCreateOrUpdateSamples {
+### Builders_ListByResourceGroup
+
+```java
+/**
+ * Samples for Builders ListByResourceGroup.
+ */
+public final class BuildersListByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_CreateOrUpdate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_ListByResourceGroup
+     * .json
      */
     /**
-     * Sample code: Create kube environments.
-     *
+     * Sample code: Builders_ListByResourceGroup_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createKubeEnvironments(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironments()
-            .define("testenv")
-            .withRegion("East US")
-            .withExistingResourceGroup("examplerg")
-            .withStaticIp("1.2.3.4")
-            .withDaprAIConnectionString(
-                "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/")
-            .withCustomDomainConfiguration(
-                new CustomDomainConfiguration()
-                    .withDnsSuffix("www.my-name.com")
-                    .withCertificateValue("Y2VydA==".getBytes())
-                    .withCertificatePassword("fakeTokenPlaceholder"))
-            .create();
+    public static void
+        buildersListByResourceGroup0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builders().listByResourceGroup("rg", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironments_Delete
+### Builders_Update
 
 ```java
-/** Samples for ConnectedEnvironments Delete. */
-public final class ConnectedEnvironmentsDeleteSamples {
+import com.azure.resourcemanager.appcontainers.models.BuilderResource;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Samples for Builders Update.
+ */
+public final class BuildersUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_Update.json
      */
     /**
-     * Sample code: Delete connected environment by connectedEnvironmentName.
-     *
+     * Sample code: Builders_Update_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteConnectedEnvironmentByConnectedEnvironmentName(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.connectedEnvironments().delete("examplerg", "examplekenv", com.azure.core.util.Context.NONE);
+    public static void buildersUpdate0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        BuilderResource resource = manager.builders()
+            .getByResourceGroupWithResponse("rg", "testBuilder", com.azure.core.util.Context.NONE).getValue();
+        resource.update().withTags(mapOf("mytag1", "myvalue1")).apply();
+    }
+
+    // Use "Map.of" if available
+    @SuppressWarnings("unchecked")
+    private static  Map mapOf(Object... inputs) {
+        Map map = new HashMap<>();
+        for (int i = 0; i < inputs.length; i += 2) {
+            String key = (String) inputs[i];
+            T value = (T) inputs[i + 1];
+            map.put(key, value);
+        }
+        return map;
     }
 }
 ```
 
-### ConnectedEnvironments_GetByResourceGroup
+### Builds_CreateOrUpdate
 
 ```java
-/** Samples for ConnectedEnvironments GetByResourceGroup. */
-public final class ConnectedEnvironmentsGetByResourceGroupSamples {
+import com.azure.resourcemanager.appcontainers.models.BuildConfiguration;
+import com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage;
+import com.azure.resourcemanager.appcontainers.models.EnvironmentVariable;
+import com.azure.resourcemanager.appcontainers.models.HttpGet;
+import com.azure.resourcemanager.appcontainers.models.PreBuildStep;
+import java.util.Arrays;
+
+/**
+ * Samples for Builds CreateOrUpdate.
+ */
+public final class BuildsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_CreateOrUpdate.json
      */
     /**
-     * Sample code: Get connected environment by connectedEnvironmentName.
-     *
+     * Sample code: Builds_CreateOrUpdate_WithConfig.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getConnectedEnvironmentByConnectedEnvironmentName(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironments()
-            .getByResourceGroupWithResponse("examplerg", "examplekenv", com.azure.core.util.Context.NONE);
+    public static void
+        buildsCreateOrUpdateWithConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builds().define("testBuild-123456789az").withExistingBuilder("rg", "testBuilder")
+            .withDestinationContainerRegistry(new ContainerRegistryWithCustomImage().withServer("test.azurecr.io")
+                .withImage("test.azurecr.io/repo:tag"))
+            .withConfiguration(
+                new BuildConfiguration().withBaseOs("DebianBullseye").withPlatform("dotnetcore")
+                    .withPlatformVersion("7.0")
+                    .withEnvironmentVariables(
+                        Arrays.asList(new EnvironmentVariable().withName("foo1").withValue("bar1"),
+                            new EnvironmentVariable().withName("foo2").withValue("bar2")))
+                    .withPreBuildSteps(Arrays.asList(
+                        new PreBuildStep().withDescription("First pre build step.")
+                            .withScripts(Arrays.asList("echo 'hello'", "echo 'world'"))
+                            .withHttpGet(new HttpGet().withUrl("https://microsoft.com").withFileName("output.txt")
+                                .withHeaders(Arrays.asList("foo", "bar"))),
+                        new PreBuildStep().withDescription("Second pre build step.")
+                            .withScripts(Arrays.asList("echo 'hello'", "echo 'again'"))
+                            .withHttpGet(new HttpGet().withUrl("https://microsoft.com").withFileName("output.txt")
+                                .withHeaders(Arrays.asList("foo"))))))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Builds_CreateOrUpdate_NoConfig.json
+     */
+    /**
+     * Sample code: Builds_CreateOrUpdate_NoConfig.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        buildsCreateOrUpdateNoConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builds().define("testBuild").withExistingBuilder("rg", "testBuilder").create();
     }
 }
 ```
 
-### ConnectedEnvironments_List
+### Builds_Delete
 
 ```java
-/** Samples for ConnectedEnvironments List. */
-public final class ConnectedEnvironmentsListSamples {
+/**
+ * Samples for Builds Delete.
+ */
+public final class BuildsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_ListBySubscription.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_Delete.json
      */
     /**
-     * Sample code: List connected environments by subscription.
-     *
+     * Sample code: Builds_Delete_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listConnectedEnvironmentsBySubscription(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.connectedEnvironments().list(com.azure.core.util.Context.NONE);
+    public static void buildsDelete0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builds().delete("rg", "testBuilder", "testBuild", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironments_ListByResourceGroup
+### Builds_Get
 
 ```java
-/** Samples for ConnectedEnvironments ListByResourceGroup. */
-public final class ConnectedEnvironmentsListByResourceGroupSamples {
+/**
+ * Samples for Builds Get.
+ */
+public final class BuildsGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_ListByResourceGroup.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_Get.json
      */
     /**
-     * Sample code: List environments by resource group.
-     *
+     * Sample code: Builds_Get_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listEnvironmentsByResourceGroup(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.connectedEnvironments().listByResourceGroup("examplerg", com.azure.core.util.Context.NONE);
+    public static void buildsGet0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.builds().getWithResponse("rg", "testBuilder", "testBuild", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironments_Update
+### BuildsByBuilderResource_List
 
 ```java
-/** Samples for ConnectedEnvironments Update. */
-public final class ConnectedEnvironmentsUpdateSamples {
+/**
+ * Samples for BuildsByBuilderResource List.
+ */
+public final class BuildsByBuilderResourceListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_ListByBuilderResource
+     * .json
      */
     /**
-     * Sample code: Patch Managed Environment.
-     *
+     * Sample code: Builds_ListByBuilderResource_0.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void patchManagedEnvironment(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.connectedEnvironments().updateWithResponse("examplerg", "testenv", com.azure.core.util.Context.NONE);
+    public static void
+        buildsListByBuilderResource0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.buildsByBuilderResources().list("rg", "testBuilder", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironmentsCertificates_CreateOrUpdate
+### Certificates_CreateOrUpdate
 
 ```java
+import com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner;
+import com.azure.resourcemanager.appcontainers.models.CertificateKeyVaultProperties;
 import com.azure.resourcemanager.appcontainers.models.CertificateProperties;
+import com.azure.resourcemanager.appcontainers.models.CertificateType;
 
-/** Samples for ConnectedEnvironmentsCertificates CreateOrUpdate. */
-public final class ConnectedEnvironmentsCertificatesCreateOrUpdateSamples {
+/**
+ * Samples for Certificates CreateOrUpdate.
+ */
+public final class CertificatesCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificate_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Certificate_CreateOrUpdate_FromKeyVault.json
      */
     /**
-     * Sample code: Create or Update Certificate.
-     *
+     * Sample code: Create or Update Certificate using Managed Identity.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateCertificate(
+    public static void createOrUpdateCertificateUsingManagedIdentity(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsCertificates()
-            .define("certificate-firendly-name")
-            .withRegion("East US")
-            .withExistingConnectedEnvironment("examplerg", "testcontainerenv")
-            .withProperties(
-                new CertificateProperties().withPassword("fakeTokenPlaceholder").withValue("Y2VydA==".getBytes()))
-            .create();
+        manager.certificates().createOrUpdateWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            new CertificateInner().withLocation("East US").withProperties(new CertificateProperties()
+                .withCertificateKeyVaultProperties(new CertificateKeyVaultProperties().withIdentity(
+                    "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/test-rg/providers/microsoft.managedidentity/userassignedidentities/test-user-mi")
+                    .withKeyVaultUrl("fakeTokenPlaceholder"))
+                .withCertificateType(CertificateType.SERVER_SSLCERTIFICATE)),
+            com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificate_CreateOrUpdate.
+     * json
+     */
+    /**
+     * Sample code: Create or Update Certificate.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        createOrUpdateCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.certificates().createOrUpdateWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            new CertificateInner().withLocation("East US")
+                .withProperties(new CertificateProperties().withPassword("fakeTokenPlaceholder")
+                    .withValue("Y2VydA==".getBytes()).withCertificateType(CertificateType.IMAGE_PULL_TRUSTED_CA)),
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironmentsCertificates_Delete
+### Certificates_Delete
 
 ```java
-/** Samples for ConnectedEnvironmentsCertificates Delete. */
-public final class ConnectedEnvironmentsCertificatesDeleteSamples {
+/**
+ * Samples for Certificates Delete.
+ */
+public final class CertificatesDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificate_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificate_Delete.json
      */
     /**
      * Sample code: Delete Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsCertificates()
-            .deleteWithResponse(
-                "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE);
+        manager.certificates().deleteWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironmentsCertificates_Get
+### Certificates_Get
 
 ```java
-/** Samples for ConnectedEnvironmentsCertificates Get. */
-public final class ConnectedEnvironmentsCertificatesGetSamples {
+/**
+ * Samples for Certificates Get.
+ */
+public final class CertificatesGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificate_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificate_Get.json
      */
     /**
      * Sample code: Get Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsCertificates()
-            .getWithResponse(
-                "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE);
+        manager.certificates().getWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironmentsCertificates_List
+### Certificates_List
 
 ```java
-/** Samples for ConnectedEnvironmentsCertificates List. */
-public final class ConnectedEnvironmentsCertificatesListSamples {
+/**
+ * Samples for Certificates List.
+ */
+public final class CertificatesListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificates_ListByConnectedEnvironment.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Certificates_ListByManagedEnvironment.json
      */
     /**
-     * Sample code: List Certificates by Connected Environment.
-     *
+     * Sample code: List Certificates by Managed Environment.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listCertificatesByConnectedEnvironment(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsCertificates()
-            .list("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE);
+    public static void
+        listCertificatesByManagedEnvironment(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.certificates().list("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironmentsCertificates_Update
+### Certificates_Update
 
 ```java
-import com.azure.resourcemanager.appcontainers.models.Certificate;
+import com.azure.resourcemanager.appcontainers.models.CertificatePatch;
 import java.util.HashMap;
 import java.util.Map;
 
-/** Samples for ConnectedEnvironmentsCertificates Update. */
-public final class ConnectedEnvironmentsCertificatesUpdateSamples {
+/**
+ * Samples for Certificates Update.
+ */
+public final class CertificatesUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificates_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificates_Patch.json
      */
     /**
      * Sample code: Patch Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void patchCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        Certificate resource =
-            manager
-                .connectedEnvironmentsCertificates()
-                .getWithResponse(
-                    "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE)
-                .getValue();
-        resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply();
+        manager.certificates().updateWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            new CertificatePatch().withTags(mapOf("tag1", "value1", "tag2", "value2")),
+            com.azure.core.util.Context.NONE);
     }
 
     // Use "Map.of" if available
@@ -651,64 +934,383 @@ public final class ConnectedEnvironmentsCertificatesUpdateSamples {
 }
 ```
 
-### ConnectedEnvironmentsDaprComponents_CreateOrUpdate
+### ConnectedEnvironments_CheckNameAvailability
 
 ```java
-import com.azure.resourcemanager.appcontainers.models.DaprMetadata;
-import com.azure.resourcemanager.appcontainers.models.Secret;
-import java.util.Arrays;
+import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest;
 
-/** Samples for ConnectedEnvironmentsDaprComponents CreateOrUpdate. */
-public final class ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples {
+/**
+ * Samples for ConnectedEnvironments CheckNameAvailability.
+ */
+public final class ConnectedEnvironmentsCheckNameAvailabilitySamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsCertificates_CheckNameAvailability.json
      */
     /**
-     * Sample code: Create or update dapr component.
-     *
+     * Sample code: Certificates_CheckNameAvailability.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateDaprComponent(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsDaprComponents()
-            .define("reddog")
-            .withExistingConnectedEnvironment("examplerg", "myenvironment")
-            .withComponentType("state.azure.cosmosdb")
-            .withVersion("v1")
-            .withIgnoreErrors(false)
-            .withInitTimeout("50s")
-            .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue")))
-            .withMetadata(
-                Arrays
-                    .asList(
-                        new DaprMetadata().withName("url").withValue(""),
-                        new DaprMetadata().withName("database").withValue("itemsDB"),
-                        new DaprMetadata().withName("collection").withValue("items"),
-                        new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder")))
-            .withScopes(Arrays.asList("container-app-1", "container-app-2"))
-            .create();
+    public static void
+        certificatesCheckNameAvailability(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments()
+            .checkNameAvailabilityWithResponse("examplerg", "testcontainerenv", new CheckNameAvailabilityRequest()
+                .withName("testcertificatename").withType("Microsoft.App/connectedEnvironments/certificates"),
+                com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ConnectedEnvironmentsDaprComponents_Delete
+### ConnectedEnvironments_CreateOrUpdate
 
 ```java
-/** Samples for ConnectedEnvironmentsDaprComponents Delete. */
-public final class ConnectedEnvironmentsDaprComponentsDeleteSamples {
+import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration;
+
+/**
+ * Samples for ConnectedEnvironments CreateOrUpdate.
+ */
+public final class ConnectedEnvironmentsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_Delete.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironments_CreateOrUpdate.json
      */
     /**
-     * Sample code: Delete dapr component.
-     *
+     * Sample code: Create kube environments.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsDaprComponents()
-            .deleteWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+    public static void createKubeEnvironments(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments().define("testenv").withRegion("East US").withExistingResourceGroup("examplerg")
+            .withStaticIp("1.2.3.4")
+            .withDaprAIConnectionString(
+                "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/")
+            .withCustomDomainConfiguration(new CustomDomainConfiguration().withDnsSuffix("www.my-name.com")
+                .withCertificateValue("Y2VydA==".getBytes()).withCertificatePassword("fakeTokenPlaceholder"))
+            .create();
+    }
+}
+```
+
+### ConnectedEnvironments_Delete
+
+```java
+/**
+ * Samples for ConnectedEnvironments Delete.
+ */
+public final class ConnectedEnvironmentsDeleteSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ConnectedEnvironments_Delete
+     * .json
+     */
+    /**
+     * Sample code: Delete connected environment by connectedEnvironmentName.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void deleteConnectedEnvironmentByConnectedEnvironmentName(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments().delete("examplerg", "examplekenv", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironments_GetByResourceGroup
+
+```java
+/**
+ * Samples for ConnectedEnvironments GetByResourceGroup.
+ */
+public final class ConnectedEnvironmentsGetByResourceGroupSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ConnectedEnvironments_Get.
+     * json
+     */
+    /**
+     * Sample code: Get connected environment by connectedEnvironmentName.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getConnectedEnvironmentByConnectedEnvironmentName(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments().getByResourceGroupWithResponse("examplerg", "examplekenv",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironments_List
+
+```java
+/**
+ * Samples for ConnectedEnvironments List.
+ */
+public final class ConnectedEnvironmentsListSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironments_ListBySubscription.json
+     */
+    /**
+     * Sample code: List connected environments by subscription.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listConnectedEnvironmentsBySubscription(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments().list(com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironments_ListByResourceGroup
+
+```java
+/**
+ * Samples for ConnectedEnvironments ListByResourceGroup.
+ */
+public final class ConnectedEnvironmentsListByResourceGroupSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironments_ListByResourceGroup.json
+     */
+    /**
+     * Sample code: List environments by resource group.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        listEnvironmentsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments().listByResourceGroup("examplerg", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironments_Update
+
+```java
+/**
+ * Samples for ConnectedEnvironments Update.
+ */
+public final class ConnectedEnvironmentsUpdateSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ConnectedEnvironments_Patch.
+     * json
+     */
+    /**
+     * Sample code: Patch Managed Environment.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        patchManagedEnvironment(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironments().updateWithResponse("examplerg", "testenv", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironmentsCertificates_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.CertificateProperties;
+
+/**
+ * Samples for ConnectedEnvironmentsCertificates CreateOrUpdate.
+ */
+public final class ConnectedEnvironmentsCertificatesCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsCertificate_CreateOrUpdate.json
+     */
+    /**
+     * Sample code: Create or Update Certificate.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        createOrUpdateCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsCertificates().define("certificate-firendly-name").withRegion("East US")
+            .withExistingConnectedEnvironment("examplerg", "testcontainerenv")
+            .withProperties(
+                new CertificateProperties().withPassword("fakeTokenPlaceholder").withValue("Y2VydA==".getBytes()))
+            .create();
+    }
+}
+```
+
+### ConnectedEnvironmentsCertificates_Delete
+
+```java
+/**
+ * Samples for ConnectedEnvironmentsCertificates Delete.
+ */
+public final class ConnectedEnvironmentsCertificatesDeleteSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsCertificate_Delete.json
+     */
+    /**
+     * Sample code: Delete Certificate.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsCertificates().deleteWithResponse("examplerg", "testcontainerenv",
+            "certificate-firendly-name", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironmentsCertificates_Get
+
+```java
+/**
+ * Samples for ConnectedEnvironmentsCertificates Get.
+ */
+public final class ConnectedEnvironmentsCertificatesGetSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsCertificate_Get.json
+     */
+    /**
+     * Sample code: Get Certificate.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsCertificates().getWithResponse("examplerg", "testcontainerenv",
+            "certificate-firendly-name", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironmentsCertificates_List
+
+```java
+/**
+ * Samples for ConnectedEnvironmentsCertificates List.
+ */
+public final class ConnectedEnvironmentsCertificatesListSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsCertificates_ListByConnectedEnvironment.json
+     */
+    /**
+     * Sample code: List Certificates by Connected Environment.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listCertificatesByConnectedEnvironment(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsCertificates().list("examplerg", "testcontainerenv",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ConnectedEnvironmentsCertificates_Update
+
+```java
+import com.azure.resourcemanager.appcontainers.models.Certificate;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Samples for ConnectedEnvironmentsCertificates Update.
+ */
+public final class ConnectedEnvironmentsCertificatesUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsCertificates_Patch.json
+     */
+    /**
+     * Sample code: Patch Certificate.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void patchCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        Certificate resource = manager.connectedEnvironmentsCertificates().getWithResponse("examplerg",
+            "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE).getValue();
+        resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply();
+    }
+
+    // Use "Map.of" if available
+    @SuppressWarnings("unchecked")
+    private static  Map mapOf(Object... inputs) {
+        Map map = new HashMap<>();
+        for (int i = 0; i < inputs.length; i += 2) {
+            String key = (String) inputs[i];
+            T value = (T) inputs[i + 1];
+            map.put(key, value);
+        }
+        return map;
+    }
+}
+```
+
+### ConnectedEnvironmentsDaprComponents_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding;
+import com.azure.resourcemanager.appcontainers.models.DaprMetadata;
+import com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata;
+import com.azure.resourcemanager.appcontainers.models.Secret;
+import java.util.Arrays;
+
+/**
+ * Samples for ConnectedEnvironmentsDaprComponents CreateOrUpdate.
+ */
+public final class ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsDaprComponents_CreateOrUpdate.json
+     */
+    /**
+     * Sample code: Create or update dapr component.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        createOrUpdateDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsDaprComponents().define("reddog")
+            .withExistingConnectedEnvironment("examplerg", "myenvironment").withComponentType("state.azure.cosmosdb")
+            .withVersion("v1").withIgnoreErrors(false).withInitTimeout("50s")
+            .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue")))
+            .withMetadata(Arrays.asList(new DaprMetadata().withName("url").withValue(""),
+                new DaprMetadata().withName("database").withValue("itemsDB"),
+                new DaprMetadata().withName("collection").withValue("items"),
+                new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder")))
+            .withScopes(Arrays.asList("container-app-1", "container-app-2"))
+            .withServiceComponentBind(Arrays.asList(new DaprComponentServiceBinding().withName("statestore")
+                .withServiceId(
+                    "/subscriptions/9f7371f1-b593-4c3c-84e2-9167806ad358/resourceGroups/ca-syn2-group/providers/Microsoft.App/containerapps/cappredis")
+                .withMetadata(new DaprServiceBindMetadata().withName("daprcomponentBind").withValue("redis-bind"))))
+            .create();
+    }
+}
+```
+
+### ConnectedEnvironmentsDaprComponents_Delete
+
+```java
+/**
+ * Samples for ConnectedEnvironmentsDaprComponents Delete.
+ */
+public final class ConnectedEnvironmentsDaprComponentsDeleteSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsDaprComponents_Delete.json
+     */
+    /**
+     * Sample code: Delete dapr component.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void deleteDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsDaprComponents().deleteWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -716,20 +1318,22 @@ public final class ConnectedEnvironmentsDaprComponentsDeleteSamples {
 ### ConnectedEnvironmentsDaprComponents_Get
 
 ```java
-/** Samples for ConnectedEnvironmentsDaprComponents Get. */
+/**
+ * Samples for ConnectedEnvironmentsDaprComponents Get.
+ */
 public final class ConnectedEnvironmentsDaprComponentsGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_Get.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsDaprComponents_Get.json
      */
     /**
      * Sample code: Get Dapr Component.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsDaprComponents()
-            .getWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+        manager.connectedEnvironmentsDaprComponents().getWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -737,20 +1341,22 @@ public final class ConnectedEnvironmentsDaprComponentsGetSamples {
 ### ConnectedEnvironmentsDaprComponents_List
 
 ```java
-/** Samples for ConnectedEnvironmentsDaprComponents List. */
+/**
+ * Samples for ConnectedEnvironmentsDaprComponents List.
+ */
 public final class ConnectedEnvironmentsDaprComponentsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_List.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsDaprComponents_List.json
      */
     /**
      * Sample code: List Dapr Components.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listDaprComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsDaprComponents()
-            .list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
+        manager.connectedEnvironmentsDaprComponents().list("examplerg", "myenvironment",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -758,21 +1364,23 @@ public final class ConnectedEnvironmentsDaprComponentsListSamples {
 ### ConnectedEnvironmentsDaprComponents_ListSecrets
 
 ```java
-/** Samples for ConnectedEnvironmentsDaprComponents ListSecrets. */
+/**
+ * Samples for ConnectedEnvironmentsDaprComponents ListSecrets.
+ */
 public final class ConnectedEnvironmentsDaprComponentsListSecretsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_ListSecrets.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsDaprComponents_ListSecrets.json
      */
     /**
      * Sample code: List Container Apps Secrets.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsSecrets(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsDaprComponents()
-            .listSecretsWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+    public static void
+        listContainerAppsSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsDaprComponents().listSecretsWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -784,30 +1392,26 @@ import com.azure.resourcemanager.appcontainers.models.AccessMode;
 import com.azure.resourcemanager.appcontainers.models.AzureFileProperties;
 import com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentStorageProperties;
 
-/** Samples for ConnectedEnvironmentsStorages CreateOrUpdate. */
+/**
+ * Samples for ConnectedEnvironmentsStorages CreateOrUpdate.
+ */
 public final class ConnectedEnvironmentsStoragesCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsStorages_CreateOrUpdate.json
      */
     /**
      * Sample code: Create or update environments storage.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateEnvironmentsStorage(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsStorages()
-            .define("jlaw-demo1")
+    public static void
+        createOrUpdateEnvironmentsStorage(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.connectedEnvironmentsStorages().define("jlaw-demo1")
             .withExistingConnectedEnvironment("examplerg", "env")
-            .withProperties(
-                new ConnectedEnvironmentStorageProperties()
-                    .withAzureFile(
-                        new AzureFileProperties()
-                            .withAccountName("account1")
-                            .withAccountKey("fakeTokenPlaceholder")
-                            .withAccessMode(AccessMode.READ_ONLY)
-                            .withShareName("share1")))
+            .withProperties(new ConnectedEnvironmentStorageProperties().withAzureFile(
+                new AzureFileProperties().withAccountName("account1").withAccountKey("fakeTokenPlaceholder")
+                    .withAccessMode(AccessMode.READ_ONLY).withShareName("share1")))
             .create();
     }
 }
@@ -816,21 +1420,23 @@ public final class ConnectedEnvironmentsStoragesCreateOrUpdateSamples {
 ### ConnectedEnvironmentsStorages_Delete
 
 ```java
-/** Samples for ConnectedEnvironmentsStorages Delete. */
+/**
+ * Samples for ConnectedEnvironmentsStorages Delete.
+ */
 public final class ConnectedEnvironmentsStoragesDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_Delete.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsStorages_Delete.json
      */
     /**
      * Sample code: List environments storages by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listEnvironmentsStoragesBySubscription(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsStorages()
-            .deleteWithResponse("examplerg", "env", "jlaw-demo1", com.azure.core.util.Context.NONE);
+        manager.connectedEnvironmentsStorages().deleteWithResponse("examplerg", "env", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -838,21 +1444,23 @@ public final class ConnectedEnvironmentsStoragesDeleteSamples {
 ### ConnectedEnvironmentsStorages_Get
 
 ```java
-/** Samples for ConnectedEnvironmentsStorages Get. */
+/**
+ * Samples for ConnectedEnvironmentsStorages Get.
+ */
 public final class ConnectedEnvironmentsStoragesGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_Get.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsStorages_Get.json
      */
     /**
      * Sample code: get a environments storage properties by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getAEnvironmentsStoragePropertiesBySubscription(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsStorages()
-            .getWithResponse("examplerg", "env", "jlaw-demo1", com.azure.core.util.Context.NONE);
+        manager.connectedEnvironmentsStorages().getWithResponse("examplerg", "env", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -860,21 +1468,23 @@ public final class ConnectedEnvironmentsStoragesGetSamples {
 ### ConnectedEnvironmentsStorages_List
 
 ```java
-/** Samples for ConnectedEnvironmentsStorages List. */
+/**
+ * Samples for ConnectedEnvironmentsStorages List.
+ */
 public final class ConnectedEnvironmentsStoragesListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_List.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ConnectedEnvironmentsStorages_List.json
      */
     /**
      * Sample code: List environments storages by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listEnvironmentsStoragesBySubscription(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .connectedEnvironmentsStorages()
-            .listWithResponse("examplerg", "managedEnv", com.azure.core.util.Context.NONE);
+        manager.connectedEnvironmentsStorages().listWithResponse("examplerg", "managedEnv",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -897,8 +1507,11 @@ import com.azure.resourcemanager.appcontainers.models.CorsPolicy;
 import com.azure.resourcemanager.appcontainers.models.CustomDomain;
 import com.azure.resourcemanager.appcontainers.models.CustomScaleRule;
 import com.azure.resourcemanager.appcontainers.models.Dapr;
+import com.azure.resourcemanager.appcontainers.models.ExtendedLocation;
+import com.azure.resourcemanager.appcontainers.models.ExtendedLocationTypes;
 import com.azure.resourcemanager.appcontainers.models.Ingress;
 import com.azure.resourcemanager.appcontainers.models.IngressClientCertificateMode;
+import com.azure.resourcemanager.appcontainers.models.IngressPortMapping;
 import com.azure.resourcemanager.appcontainers.models.IngressStickySessions;
 import com.azure.resourcemanager.appcontainers.models.IngressTransportMethod;
 import com.azure.resourcemanager.appcontainers.models.InitContainer;
@@ -908,228 +1521,219 @@ import com.azure.resourcemanager.appcontainers.models.Scale;
 import com.azure.resourcemanager.appcontainers.models.ScaleRule;
 import com.azure.resourcemanager.appcontainers.models.Service;
 import com.azure.resourcemanager.appcontainers.models.ServiceBind;
+import com.azure.resourcemanager.appcontainers.models.StorageType;
 import com.azure.resourcemanager.appcontainers.models.TcpScaleRule;
 import com.azure.resourcemanager.appcontainers.models.Template;
 import com.azure.resourcemanager.appcontainers.models.TrafficWeight;
 import com.azure.resourcemanager.appcontainers.models.Type;
+import com.azure.resourcemanager.appcontainers.models.Volume;
+import com.azure.resourcemanager.appcontainers.models.VolumeMount;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
-/** Samples for ContainerApps CreateOrUpdate. */
+/**
+ * Samples for ContainerApps CreateOrUpdate.
+ */
 public final class ContainerAppsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_CreateOrUpdate_ConnectedEnvironment.json
      */
     /**
-     * Sample code: Create or Update Container App.
-     *
+     * Sample code: Create or Update App On A Connected Environment.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateContainerApp(
+    public static void createOrUpdateAppOnAConnectedEnvironment(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerApps()
-            .define("testcontainerApp0")
-            .withRegion("East US")
-            .withExistingResourceGroup("rg")
+        manager.containerApps().define("testcontainerApp0").withRegion("East US").withExistingResourceGroup("rg")
+            .withExtendedLocation(new ExtendedLocation().withName(
+                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.ExtendedLocation/customLocations/testcustomlocation")
+                .withType(ExtendedLocationTypes.CUSTOM_LOCATION))
+            .withEnvironmentId(
+                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube")
+            .withConfiguration(new Configuration().withIngress(new Ingress().withExternal(true).withTargetPort(3000)
+                .withTraffic(Arrays.asList(new TrafficWeight().withRevisionName("testcontainerApp0-ab1234")
+                    .withWeight(100).withLabel("production")))
+                .withCustomDomains(Arrays.asList(new CustomDomain().withName("www.my-name.com")
+                    .withBindingType(BindingType.SNI_ENABLED).withCertificateId(
+                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"),
+                    new CustomDomain().withName("www.my-other-name.com").withBindingType(BindingType.SNI_ENABLED)
+                        .withCertificateId(
+                            "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com")))
+                .withIpSecurityRestrictions(Arrays.asList(
+                    new IpSecurityRestrictionRule().withName("Allow work IP A subnet")
+                        .withDescription("Allowing all IP's within the subnet below to access containerapp")
+                        .withIpAddressRange("192.168.1.1/32").withAction(Action.ALLOW),
+                    new IpSecurityRestrictionRule().withName("Allow work IP B subnet")
+                        .withDescription("Allowing all IP's within the subnet below to access containerapp")
+                        .withIpAddressRange("192.168.1.1/8").withAction(Action.ALLOW)))
+                .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY))
+                .withClientCertificateMode(IngressClientCertificateMode.ACCEPT)
+                .withCorsPolicy(new CorsPolicy()
+                    .withAllowedOrigins(Arrays.asList("https://a.test.com", "https://b.test.com"))
+                    .withAllowedMethods(Arrays.asList("GET", "POST"))
+                    .withAllowedHeaders(Arrays.asList("HEADER1", "HEADER2"))
+                    .withExposeHeaders(Arrays.asList("HEADER3", "HEADER4")).withMaxAge(1234).withAllowCredentials(true))
+                .withAdditionalPortMappings(
+                    Arrays.asList(new IngressPortMapping().withExternal(true).withTargetPort(1234),
+                        new IngressPortMapping().withExternal(false).withTargetPort(2345).withExposedPort(3456))))
+                .withDapr(
+                    new Dapr().withEnabled(true).withAppProtocol(AppProtocol.HTTP).withAppPort(3000)
+                        .withHttpReadBufferSize(30).withHttpMaxRequestSize(10).withLogLevel(
+                            LogLevel.DEBUG)
+                        .withEnableApiLogging(true))
+                .withMaxInactiveRevisions(10))
+            .withTemplate(new Template()
+                .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerApp0:v4")
+                    .withName("testinitcontainerApp0").withCommand(Arrays.asList("/bin/sh"))
+                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                .withContainers(
+                    Arrays
+                        .asList(new Container().withImage("repo/testcontainerApp0:v1").withName("testcontainerApp0")
+                            .withProbes(Arrays.asList(new ContainerAppProbe()
+                                .withHttpGet(new ContainerAppProbeHttpGet()
+                                    .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem()
+                                        .withName("Custom-Header").withValue("Awesome")))
+                                    .withPath("/health").withPort(8080))
+                                .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS)))))
+                .withScale(new Scale().withMinReplicas(1).withMaxReplicas(5)
+                    .withRules(Arrays.asList(new ScaleRule().withName("httpscalingrule").withCustom(
+                        new CustomScaleRule().withType("http").withMetadata(mapOf("concurrentRequests", "50")))))))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_CreateOrUpdate
+     * .json
+     */
+    /**
+     * Sample code: Create or Update Container App.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        createOrUpdateContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerApps().define("testcontainerApp0").withRegion("East US").withExistingResourceGroup("rg")
             .withEnvironmentId(
                 "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube")
-            .withWorkloadProfileName("My-GP-01")
-            .withConfiguration(
-                new Configuration()
-                    .withIngress(
-                        new Ingress()
-                            .withExternal(true)
-                            .withTargetPort(3000)
-                            .withTraffic(
-                                Arrays
-                                    .asList(
-                                        new TrafficWeight()
-                                            .withRevisionName("testcontainerApp0-ab1234")
-                                            .withWeight(100)
-                                            .withLabel("production")))
-                            .withCustomDomains(
-                                Arrays
-                                    .asList(
-                                        new CustomDomain()
-                                            .withName("www.my-name.com")
-                                            .withBindingType(BindingType.SNI_ENABLED)
-                                            .withCertificateId(
-                                                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"),
-                                        new CustomDomain()
-                                            .withName("www.my-other-name.com")
-                                            .withBindingType(BindingType.SNI_ENABLED)
-                                            .withCertificateId(
-                                                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com")))
-                            .withIpSecurityRestrictions(
-                                Arrays
-                                    .asList(
-                                        new IpSecurityRestrictionRule()
-                                            .withName("Allow work IP A subnet")
-                                            .withDescription(
-                                                "Allowing all IP's within the subnet below to access containerapp")
-                                            .withIpAddressRange("192.168.1.1/32")
-                                            .withAction(Action.ALLOW),
-                                        new IpSecurityRestrictionRule()
-                                            .withName("Allow work IP B subnet")
-                                            .withDescription(
-                                                "Allowing all IP's within the subnet below to access containerapp")
-                                            .withIpAddressRange("192.168.1.1/8")
-                                            .withAction(Action.ALLOW)))
-                            .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY))
-                            .withClientCertificateMode(IngressClientCertificateMode.ACCEPT)
-                            .withCorsPolicy(
-                                new CorsPolicy()
-                                    .withAllowedOrigins(Arrays.asList("https://a.test.com", "https://b.test.com"))
-                                    .withAllowedMethods(Arrays.asList("GET", "POST"))
-                                    .withAllowedHeaders(Arrays.asList("HEADER1", "HEADER2"))
-                                    .withExposeHeaders(Arrays.asList("HEADER3", "HEADER4"))
-                                    .withMaxAge(1234)
-                                    .withAllowCredentials(true)))
-                    .withDapr(
-                        new Dapr()
-                            .withEnabled(true)
-                            .withAppProtocol(AppProtocol.HTTP)
-                            .withAppPort(3000)
-                            .withHttpReadBufferSize(30)
-                            .withHttpMaxRequestSize(10)
-                            .withLogLevel(LogLevel.DEBUG)
-                            .withEnableApiLogging(true))
-                    .withMaxInactiveRevisions(10)
-                    .withService(new Service().withType("redis")))
-            .withTemplate(
-                new Template()
-                    .withInitContainers(
-                        Arrays
-                            .asList(
-                                new InitContainer()
-                                    .withImage("repo/testcontainerApp0:v4")
-                                    .withName("testinitcontainerApp0")
-                                    .withCommand(Arrays.asList("/bin/sh"))
-                                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
-                                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new Container()
-                                    .withImage("repo/testcontainerApp0:v1")
-                                    .withName("testcontainerApp0")
-                                    .withProbes(
-                                        Arrays
-                                            .asList(
-                                                new ContainerAppProbe()
-                                                    .withHttpGet(
-                                                        new ContainerAppProbeHttpGet()
-                                                            .withHttpHeaders(
-                                                                Arrays
-                                                                    .asList(
-                                                                        new ContainerAppProbeHttpGetHttpHeadersItem()
-                                                                            .withName("Custom-Header")
-                                                                            .withValue("Awesome")))
-                                                            .withPath("/health")
-                                                            .withPort(8080))
-                                                    .withInitialDelaySeconds(3)
-                                                    .withPeriodSeconds(3)
-                                                    .withType(Type.LIVENESS)))))
-                    .withScale(
-                        new Scale()
-                            .withMinReplicas(1)
-                            .withMaxReplicas(5)
-                            .withRules(
-                                Arrays
-                                    .asList(
-                                        new ScaleRule()
-                                            .withName("httpscalingrule")
-                                            .withCustom(
-                                                new CustomScaleRule()
-                                                    .withType("http")
-                                                    .withMetadata(mapOf("concurrentRequests", "50"))))))
-                    .withServiceBinds(
-                        Arrays
-                            .asList(
-                                new ServiceBind()
-                                    .withServiceId(
-                                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/redisService")
-                                    .withName("redisService"))))
+            .withWorkloadProfileName(
+                "My-GP-01")
+            .withConfiguration(new Configuration().withIngress(new Ingress().withExternal(true).withTargetPort(3000)
+                .withTraffic(Arrays.asList(new TrafficWeight().withRevisionName("testcontainerApp0-ab1234")
+                    .withWeight(100).withLabel("production")))
+                .withCustomDomains(Arrays.asList(new CustomDomain().withName("www.my-name.com")
+                    .withBindingType(BindingType.SNI_ENABLED).withCertificateId(
+                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"),
+                    new CustomDomain().withName("www.my-other-name.com").withBindingType(BindingType.SNI_ENABLED)
+                        .withCertificateId(
+                            "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com")))
+                .withIpSecurityRestrictions(Arrays.asList(
+                    new IpSecurityRestrictionRule().withName("Allow work IP A subnet")
+                        .withDescription("Allowing all IP's within the subnet below to access containerapp")
+                        .withIpAddressRange("192.168.1.1/32").withAction(Action.ALLOW),
+                    new IpSecurityRestrictionRule().withName("Allow work IP B subnet")
+                        .withDescription("Allowing all IP's within the subnet below to access containerapp")
+                        .withIpAddressRange("192.168.1.1/8").withAction(Action.ALLOW)))
+                .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY))
+                .withClientCertificateMode(IngressClientCertificateMode.ACCEPT)
+                .withCorsPolicy(new CorsPolicy()
+                    .withAllowedOrigins(Arrays.asList("https://a.test.com", "https://b.test.com"))
+                    .withAllowedMethods(Arrays.asList("GET", "POST"))
+                    .withAllowedHeaders(Arrays.asList("HEADER1", "HEADER2"))
+                    .withExposeHeaders(Arrays.asList("HEADER3", "HEADER4")).withMaxAge(1234).withAllowCredentials(true))
+                .withAdditionalPortMappings(
+                    Arrays.asList(new IngressPortMapping().withExternal(true).withTargetPort(1234),
+                        new IngressPortMapping().withExternal(false).withTargetPort(2345).withExposedPort(3456))))
+                .withDapr(new Dapr().withEnabled(true).withAppProtocol(AppProtocol.HTTP).withAppPort(3000)
+                    .withHttpReadBufferSize(30).withHttpMaxRequestSize(10).withLogLevel(LogLevel.DEBUG)
+                    .withEnableApiLogging(true))
+                .withMaxInactiveRevisions(10).withService(new Service().withType("redis")))
+            .withTemplate(new Template()
+                .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerApp0:v4")
+                    .withName("testinitcontainerApp0").withCommand(Arrays.asList("/bin/sh"))
+                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                .withContainers(
+                    Arrays.asList(new Container().withImage("repo/testcontainerApp0:v1").withName("testcontainerApp0")
+                        .withVolumeMounts(Arrays.asList(
+                            new VolumeMount().withVolumeName("azurefile").withMountPath("/mnt/path1")
+                                .withSubPath("subPath1"),
+                            new VolumeMount().withVolumeName("nfsazurefile").withMountPath("/mnt/path2")
+                                .withSubPath("subPath2")))
+                        .withProbes(Arrays.asList(new ContainerAppProbe()
+                            .withHttpGet(new ContainerAppProbeHttpGet()
+                                .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem()
+                                    .withName("Custom-Header").withValue("Awesome")))
+                                .withPath("/health").withPort(8080))
+                            .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS)))))
+                .withScale(new Scale().withMinReplicas(1).withMaxReplicas(5)
+                    .withRules(Arrays.asList(new ScaleRule().withName("httpscalingrule").withCustom(
+                        new CustomScaleRule().withType("http").withMetadata(mapOf("concurrentRequests", "50"))))))
+                .withVolumes(Arrays.asList(
+                    new Volume().withName("azurefile").withStorageType(StorageType.AZURE_FILE)
+                        .withStorageName("storage"),
+                    new Volume().withName("nfsazurefile").withStorageType(StorageType.NFS_AZURE_FILE)
+                        .withStorageName("nfsStorage")))
+                .withServiceBinds(Arrays.asList(new ServiceBind().withServiceId(
+                    "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/redisService")
+                    .withName("redisService").withClientType("dotnet")
+                    .withCustomizedKeys(mapOf("DesiredKey", "fakeTokenPlaceholder")))))
             .create();
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_TcpApp_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_TcpApp_CreateOrUpdate.json
      */
     /**
      * Sample code: Create or Update Tcp App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void createOrUpdateTcpApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerApps()
-            .define("testcontainerAppTcp")
-            .withRegion("East US")
-            .withExistingResourceGroup("rg")
+        manager.containerApps().define("testcontainerAppTcp").withRegion("East US").withExistingResourceGroup("rg")
             .withEnvironmentId(
                 "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube")
             .withConfiguration(
                 new Configuration()
                     .withIngress(
-                        new Ingress()
-                            .withExternal(true)
-                            .withTargetPort(3000)
-                            .withExposedPort(4000)
-                            .withTransport(IngressTransportMethod.TCP)
+                        new Ingress().withExternal(true).withTargetPort(3000).withExposedPort(4000)
+                            .withTransport(
+                                IngressTransportMethod.TCP)
                             .withTraffic(
                                 Arrays
                                     .asList(
-                                        new TrafficWeight()
-                                            .withRevisionName("testcontainerAppTcp-ab1234")
-                                            .withWeight(100)))))
-            .withTemplate(
-                new Template()
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new Container()
-                                    .withImage("repo/testcontainerAppTcp:v1")
-                                    .withName("testcontainerAppTcp")
-                                    .withProbes(
-                                        Arrays
-                                            .asList(
-                                                new ContainerAppProbe()
-                                                    .withInitialDelaySeconds(3)
-                                                    .withPeriodSeconds(3)
-                                                    .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080))
-                                                    .withType(Type.LIVENESS)))))
-                    .withScale(
-                        new Scale()
-                            .withMinReplicas(1)
-                            .withMaxReplicas(5)
-                            .withRules(
-                                Arrays
-                                    .asList(
-                                        new ScaleRule()
-                                            .withName("tcpscalingrule")
-                                            .withTcp(
-                                                new TcpScaleRule()
-                                                    .withMetadata(mapOf("concurrentConnections", "50")))))))
+                                        new TrafficWeight().withRevisionName(
+                                            "testcontainerAppTcp-ab1234").withWeight(
+                                                100)))))
+            .withTemplate(new Template()
+                .withContainers(Arrays.asList(new Container().withImage("repo/testcontainerAppTcp:v1")
+                    .withName("testcontainerAppTcp")
+                    .withProbes(Arrays.asList(new ContainerAppProbe().withInitialDelaySeconds(3).withPeriodSeconds(3)
+                        .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080)).withType(Type.LIVENESS)))))
+                .withScale(
+                    new Scale().withMinReplicas(1).withMaxReplicas(5)
+                        .withRules(Arrays.asList(new ScaleRule().withName("tcpscalingrule")
+                            .withTcp(new TcpScaleRule().withMetadata(mapOf("concurrentConnections", "50")))))))
             .create();
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ManagedBy_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_ManagedBy_CreateOrUpdate.json
      */
     /**
      * Sample code: Create or Update ManagedBy App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateManagedByApp(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerApps()
-            .define("testcontainerAppManagedBy")
-            .withRegion("East US")
+    public static void
+        createOrUpdateManagedByApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerApps().define("testcontainerAppManagedBy").withRegion("East US")
             .withExistingResourceGroup("rg")
             .withManagedBy(
                 "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.AppPlatform/Spring/springapp")
@@ -1138,45 +1742,21 @@ public final class ContainerAppsCreateOrUpdateSamples {
             .withConfiguration(
                 new Configuration()
                     .withIngress(
-                        new Ingress()
-                            .withExternal(true)
-                            .withTargetPort(3000)
-                            .withExposedPort(4000)
-                            .withTransport(IngressTransportMethod.TCP)
-                            .withTraffic(
+                        new Ingress().withExternal(true).withTargetPort(3000).withExposedPort(4000)
+                            .withTransport(IngressTransportMethod.TCP).withTraffic(
                                 Arrays
                                     .asList(
-                                        new TrafficWeight()
-                                            .withRevisionName("testcontainerAppManagedBy-ab1234")
+                                        new TrafficWeight().withRevisionName("testcontainerAppManagedBy-ab1234")
                                             .withWeight(100)))))
-            .withTemplate(
-                new Template()
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new Container()
-                                    .withImage("repo/testcontainerAppManagedBy:v1")
-                                    .withName("testcontainerAppManagedBy")
-                                    .withProbes(
-                                        Arrays
-                                            .asList(
-                                                new ContainerAppProbe()
-                                                    .withInitialDelaySeconds(3)
-                                                    .withPeriodSeconds(3)
-                                                    .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080))
-                                                    .withType(Type.LIVENESS)))))
-                    .withScale(
-                        new Scale()
-                            .withMinReplicas(1)
-                            .withMaxReplicas(5)
-                            .withRules(
-                                Arrays
-                                    .asList(
-                                        new ScaleRule()
-                                            .withName("tcpscalingrule")
-                                            .withTcp(
-                                                new TcpScaleRule()
-                                                    .withMetadata(mapOf("concurrentConnections", "50")))))))
+            .withTemplate(new Template()
+                .withContainers(Arrays.asList(new Container().withImage("repo/testcontainerAppManagedBy:v1")
+                    .withName("testcontainerAppManagedBy")
+                    .withProbes(Arrays.asList(new ContainerAppProbe().withInitialDelaySeconds(3).withPeriodSeconds(3)
+                        .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080)).withType(Type.LIVENESS)))))
+                .withScale(
+                    new Scale().withMinReplicas(1).withMaxReplicas(5)
+                        .withRules(Arrays.asList(new ScaleRule().withName("tcpscalingrule")
+                            .withTcp(new TcpScaleRule().withMetadata(mapOf("concurrentConnections", "50")))))))
             .create();
     }
 
@@ -1197,14 +1777,17 @@ public final class ContainerAppsCreateOrUpdateSamples {
 ### ContainerApps_Delete
 
 ```java
-/** Samples for ContainerApps Delete. */
+/**
+ * Samples for ContainerApps Delete.
+ */
 public final class ContainerAppsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Delete.json
      */
     /**
      * Sample code: Delete Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void deleteContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
@@ -1216,18 +1799,22 @@ public final class ContainerAppsDeleteSamples {
 ### ContainerApps_GetAuthToken
 
 ```java
-/** Samples for ContainerApps GetAuthToken. */
+/**
+ * Samples for ContainerApps GetAuthToken.
+ */
 public final class ContainerAppsGetAuthTokenSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_GetAuthToken.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_GetAuthToken.
+     * json
      */
     /**
      * Sample code: Get Container App Auth Token.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppAuthToken(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        getContainerAppAuthToken(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.containerApps().getAuthTokenWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE);
     }
 }
@@ -1236,20 +1823,22 @@ public final class ContainerAppsGetAuthTokenSamples {
 ### ContainerApps_GetByResourceGroup
 
 ```java
-/** Samples for ContainerApps GetByResourceGroup. */
+/**
+ * Samples for ContainerApps GetByResourceGroup.
+ */
 public final class ContainerAppsGetByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Get.json
      */
     /**
      * Sample code: Get Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerApps()
-            .getByResourceGroupWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE);
+        manager.containerApps().getByResourceGroupWithResponse("rg", "testcontainerApp0",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1257,18 +1846,21 @@ public final class ContainerAppsGetByResourceGroupSamples {
 ### ContainerApps_List
 
 ```java
-/** Samples for ContainerApps List. */
+/**
+ * Samples for ContainerApps List.
+ */
 public final class ContainerAppsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListBySubscription.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_ListBySubscription.json
      */
     /**
      * Sample code: List Container Apps by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsBySubscription(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listContainerAppsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.containerApps().list(com.azure.core.util.Context.NONE);
     }
 }
@@ -1277,18 +1869,21 @@ public final class ContainerAppsListSamples {
 ### ContainerApps_ListByResourceGroup
 
 ```java
-/** Samples for ContainerApps ListByResourceGroup. */
+/**
+ * Samples for ContainerApps ListByResourceGroup.
+ */
 public final class ContainerAppsListByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListByResourceGroup.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_ListByResourceGroup.json
      */
     /**
      * Sample code: List Container Apps by resource group.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsByResourceGroup(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listContainerAppsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.containerApps().listByResourceGroup("rg", com.azure.core.util.Context.NONE);
     }
 }
@@ -1297,21 +1892,22 @@ public final class ContainerAppsListByResourceGroupSamples {
 ### ContainerApps_ListCustomHostnameAnalysis
 
 ```java
-/** Samples for ContainerApps ListCustomHostnameAnalysis. */
+/**
+ * Samples for ContainerApps ListCustomHostnameAnalysis.
+ */
 public final class ContainerAppsListCustomHostnameAnalysisSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListCustomHostNameAnalysis.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_ListCustomHostNameAnalysis.json
      */
     /**
      * Sample code: Analyze Custom Hostname.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void analyzeCustomHostname(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerApps()
-            .listCustomHostnameAnalysisWithResponse(
-                "rg", "testcontainerApp0", "my.name.corp", com.azure.core.util.Context.NONE);
+        manager.containerApps().listCustomHostnameAnalysisWithResponse("rg", "testcontainerApp0", "my.name.corp",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1319,18 +1915,22 @@ public final class ContainerAppsListCustomHostnameAnalysisSamples {
 ### ContainerApps_ListSecrets
 
 ```java
-/** Samples for ContainerApps ListSecrets. */
+/**
+ * Samples for ContainerApps ListSecrets.
+ */
 public final class ContainerAppsListSecretsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListSecrets.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_ListSecrets.
+     * json
      */
     /**
      * Sample code: List Container Apps Secrets.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsSecrets(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listContainerAppsSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.containerApps().listSecretsWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE);
     }
 }
@@ -1339,14 +1939,17 @@ public final class ContainerAppsListSecretsSamples {
 ### ContainerApps_Start
 
 ```java
-/** Samples for ContainerApps Start. */
+/**
+ * Samples for ContainerApps Start.
+ */
 public final class ContainerAppsStartSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Start.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Start.json
      */
     /**
      * Sample code: Start Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void startContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
@@ -1358,14 +1961,17 @@ public final class ContainerAppsStartSamples {
 ### ContainerApps_Stop
 
 ```java
-/** Samples for ContainerApps Stop. */
+/**
+ * Samples for ContainerApps Stop.
+ */
 public final class ContainerAppsStopSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Stop.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Stop.json
      */
     /**
      * Sample code: Stop Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void stopContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
@@ -1407,130 +2013,65 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 
-/** Samples for ContainerApps Update. */
+/**
+ * Samples for ContainerApps Update.
+ */
 public final class ContainerAppsUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Patch.json
      */
     /**
      * Sample code: Patch Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void patchContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        ContainerApp resource =
-            manager
-                .containerApps()
-                .getByResourceGroupWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE)
-                .getValue();
+        ContainerApp resource = manager.containerApps()
+            .getByResourceGroupWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE).getValue();
         resource
-            .update()
-            .withTags(mapOf("tag1", "value1", "tag2", "value2"))
-            .withConfiguration(
-                new Configuration()
-                    .withIngress(
-                        new Ingress()
-                            .withExternal(true)
-                            .withTargetPort(3000)
-                            .withTraffic(
-                                Arrays
-                                    .asList(
-                                        new TrafficWeight()
-                                            .withRevisionName("testcontainerApp0-ab1234")
-                                            .withWeight(100)
-                                            .withLabel("production")))
-                            .withCustomDomains(
-                                Arrays
-                                    .asList(
-                                        new CustomDomain()
-                                            .withName("www.my-name.com")
-                                            .withBindingType(BindingType.SNI_ENABLED)
-                                            .withCertificateId(
-                                                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"),
-                                        new CustomDomain()
-                                            .withName("www.my-other-name.com")
-                                            .withBindingType(BindingType.SNI_ENABLED)
-                                            .withCertificateId(
-                                                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com")))
-                            .withIpSecurityRestrictions(
-                                Arrays
-                                    .asList(
-                                        new IpSecurityRestrictionRule()
-                                            .withName("Allow work IP A subnet")
-                                            .withDescription(
-                                                "Allowing all IP's within the subnet below to access containerapp")
-                                            .withIpAddressRange("192.168.1.1/32")
-                                            .withAction(Action.ALLOW),
-                                        new IpSecurityRestrictionRule()
-                                            .withName("Allow work IP B subnet")
-                                            .withDescription(
-                                                "Allowing all IP's within the subnet below to access containerapp")
-                                            .withIpAddressRange("192.168.1.1/8")
-                                            .withAction(Action.ALLOW)))
-                            .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY)))
-                    .withDapr(
-                        new Dapr()
-                            .withEnabled(true)
-                            .withAppProtocol(AppProtocol.HTTP)
-                            .withAppPort(3000)
-                            .withHttpReadBufferSize(30)
-                            .withHttpMaxRequestSize(10)
-                            .withLogLevel(LogLevel.DEBUG)
-                            .withEnableApiLogging(true))
-                    .withMaxInactiveRevisions(10)
-                    .withService(new Service().withType("redis")))
+            .update().withTags(
+                mapOf("tag1", "value1", "tag2", "value2"))
+            .withConfiguration(new Configuration().withIngress(new Ingress().withExternal(true).withTargetPort(3000)
+                .withTraffic(Arrays.asList(new TrafficWeight().withRevisionName("testcontainerApp0-ab1234")
+                    .withWeight(100).withLabel("production")))
+                .withCustomDomains(Arrays.asList(new CustomDomain().withName("www.my-name.com")
+                    .withBindingType(BindingType.SNI_ENABLED).withCertificateId(
+                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"),
+                    new CustomDomain().withName("www.my-other-name.com").withBindingType(BindingType.SNI_ENABLED)
+                        .withCertificateId(
+                            "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com")))
+                .withIpSecurityRestrictions(Arrays.asList(
+                    new IpSecurityRestrictionRule().withName("Allow work IP A subnet")
+                        .withDescription("Allowing all IP's within the subnet below to access containerapp")
+                        .withIpAddressRange("192.168.1.1/32").withAction(Action.ALLOW),
+                    new IpSecurityRestrictionRule().withName("Allow work IP B subnet")
+                        .withDescription("Allowing all IP's within the subnet below to access containerapp")
+                        .withIpAddressRange("192.168.1.1/8").withAction(Action.ALLOW)))
+                .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY)))
+                .withDapr(new Dapr().withEnabled(true).withAppProtocol(AppProtocol.HTTP).withAppPort(3000)
+                    .withHttpReadBufferSize(30).withHttpMaxRequestSize(10).withLogLevel(LogLevel.DEBUG)
+                    .withEnableApiLogging(true))
+                .withMaxInactiveRevisions(10).withService(new Service().withType("redis")))
             .withTemplate(
                 new Template()
-                    .withInitContainers(
-                        Arrays
-                            .asList(
-                                new InitContainer()
-                                    .withImage("repo/testcontainerApp0:v4")
-                                    .withName("testinitcontainerApp0")
-                                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new Container()
-                                    .withImage("repo/testcontainerApp0:v1")
-                                    .withName("testcontainerApp0")
-                                    .withProbes(
-                                        Arrays
-                                            .asList(
-                                                new ContainerAppProbe()
-                                                    .withHttpGet(
-                                                        new ContainerAppProbeHttpGet()
-                                                            .withHttpHeaders(
-                                                                Arrays
-                                                                    .asList(
-                                                                        new ContainerAppProbeHttpGetHttpHeadersItem()
-                                                                            .withName("Custom-Header")
-                                                                            .withValue("Awesome")))
-                                                            .withPath("/health")
-                                                            .withPort(8080))
-                                                    .withInitialDelaySeconds(3)
-                                                    .withPeriodSeconds(3)
-                                                    .withType(Type.LIVENESS)))))
-                    .withScale(
-                        new Scale()
-                            .withMinReplicas(1)
-                            .withMaxReplicas(5)
-                            .withRules(
-                                Arrays
-                                    .asList(
-                                        new ScaleRule()
-                                            .withName("httpscalingrule")
-                                            .withCustom(
-                                                new CustomScaleRule()
-                                                    .withType("http")
-                                                    .withMetadata(mapOf("concurrentRequests", "50"))))))
-                    .withServiceBinds(
-                        Arrays
-                            .asList(
-                                new ServiceBind()
-                                    .withServiceId(
-                                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/service")
-                                    .withName("service"))))
+                    .withInitContainers(Arrays.asList(
+                        new InitContainer().withImage("repo/testcontainerApp0:v4").withName("testinitcontainerApp0")
+                            .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                    .withContainers(Arrays.asList(new Container().withImage("repo/testcontainerApp0:v1")
+                        .withName("testcontainerApp0").withProbes(Arrays.asList(new ContainerAppProbe()
+                            .withHttpGet(new ContainerAppProbeHttpGet()
+                                .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem()
+                                    .withName("Custom-Header").withValue("Awesome")))
+                                .withPath("/health").withPort(8080))
+                            .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS)))))
+                    .withScale(new Scale().withMinReplicas(1).withMaxReplicas(5)
+                        .withRules(Arrays.asList(new ScaleRule().withName("httpscalingrule").withCustom(
+                            new CustomScaleRule().withType("http").withMetadata(mapOf("concurrentRequests", "50"))))))
+                    .withServiceBinds(Arrays.asList(new ServiceBind().withServiceId(
+                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/service")
+                        .withName("service").withClientType("dotnet")
+                        .withCustomizedKeys(mapOf("DesiredKey", "fakeTokenPlaceholder")))))
             .apply();
     }
 
@@ -1553,38 +2094,38 @@ public final class ContainerAppsUpdateSamples {
 ```java
 import com.azure.resourcemanager.appcontainers.models.AppRegistration;
 import com.azure.resourcemanager.appcontainers.models.AuthPlatform;
+import com.azure.resourcemanager.appcontainers.models.EncryptionSettings;
 import com.azure.resourcemanager.appcontainers.models.Facebook;
 import com.azure.resourcemanager.appcontainers.models.GlobalValidation;
 import com.azure.resourcemanager.appcontainers.models.IdentityProviders;
 import com.azure.resourcemanager.appcontainers.models.UnauthenticatedClientActionV2;
 
-/** Samples for ContainerAppsAuthConfigs CreateOrUpdate. */
+/**
+ * Samples for ContainerAppsAuthConfigs CreateOrUpdate.
+ */
 public final class ContainerAppsAuthConfigsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_CreateOrUpdate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_CreateOrUpdate.
+     * json
      */
     /**
      * Sample code: Create or Update Container App AuthConfig.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateContainerAppAuthConfig(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsAuthConfigs()
-            .define("current")
+    public static void
+        createOrUpdateContainerAppAuthConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsAuthConfigs().define("current")
             .withExistingContainerApp("workerapps-rg-xj", "testcanadacentral")
             .withPlatform(new AuthPlatform().withEnabled(true))
             .withGlobalValidation(
                 new GlobalValidation().withUnauthenticatedClientAction(UnauthenticatedClientActionV2.ALLOW_ANONYMOUS))
-            .withIdentityProviders(
-                new IdentityProviders()
-                    .withFacebook(
-                        new Facebook()
-                            .withRegistration(
-                                new AppRegistration()
-                                    .withAppId("123")
-                                    .withAppSecretSettingName("fakeTokenPlaceholder"))))
+            .withIdentityProviders(new IdentityProviders().withFacebook(new Facebook().withRegistration(
+                new AppRegistration().withAppId("123").withAppSecretSettingName("fakeTokenPlaceholder"))))
+            .withEncryptionSettings(
+                new EncryptionSettings().withContainerAppAuthEncryptionSecretName("fakeTokenPlaceholder")
+                    .withContainerAppAuthSigningSecretName("fakeTokenPlaceholder"))
             .create();
     }
 }
@@ -1593,21 +2134,23 @@ public final class ContainerAppsAuthConfigsCreateOrUpdateSamples {
 ### ContainerAppsAuthConfigs_Delete
 
 ```java
-/** Samples for ContainerAppsAuthConfigs Delete. */
+/**
+ * Samples for ContainerAppsAuthConfigs Delete.
+ */
 public final class ContainerAppsAuthConfigsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_Delete.json
      */
     /**
      * Sample code: Delete Container App AuthConfig.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteContainerAppAuthConfig(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsAuthConfigs()
-            .deleteWithResponse("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE);
+    public static void
+        deleteContainerAppAuthConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsAuthConfigs().deleteWithResponse("workerapps-rg-xj", "testcanadacentral", "current",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1615,21 +2158,23 @@ public final class ContainerAppsAuthConfigsDeleteSamples {
 ### ContainerAppsAuthConfigs_Get
 
 ```java
-/** Samples for ContainerAppsAuthConfigs Get. */
+/**
+ * Samples for ContainerAppsAuthConfigs Get.
+ */
 public final class ContainerAppsAuthConfigsGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_Get.json
      */
     /**
      * Sample code: Get Container App's AuthConfig.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppSAuthConfig(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsAuthConfigs()
-            .getWithResponse("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE);
+    public static void
+        getContainerAppSAuthConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsAuthConfigs().getWithResponse("workerapps-rg-xj", "testcanadacentral", "current",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1637,21 +2182,24 @@ public final class ContainerAppsAuthConfigsGetSamples {
 ### ContainerAppsAuthConfigs_ListByContainerApp
 
 ```java
-/** Samples for ContainerAppsAuthConfigs ListByContainerApp. */
+/**
+ * Samples for ContainerAppsAuthConfigs ListByContainerApp.
+ */
 public final class ContainerAppsAuthConfigsListByContainerAppSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_ListByContainer.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_ListByContainer.
+     * json
      */
     /**
      * Sample code: List Auth Configs by Container Apps.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listAuthConfigsByContainerApps(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsAuthConfigs()
-            .listByContainerApp("workerapps-rg-xj", "testcanadacentral", com.azure.core.util.Context.NONE);
+    public static void
+        listAuthConfigsByContainerApps(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsAuthConfigs().listByContainerApp("workerapps-rg-xj", "testcanadacentral",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1659,25 +2207,24 @@ public final class ContainerAppsAuthConfigsListByContainerAppSamples {
 ### ContainerAppsDiagnostics_GetDetector
 
 ```java
-/** Samples for ContainerAppsDiagnostics GetDetector. */
+/**
+ * Samples for ContainerAppsDiagnostics GetDetector.
+ */
 public final class ContainerAppsDiagnosticsGetDetectorSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerAppsDiagnostics_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerAppsDiagnostics_Get
+     * .json
      */
     /**
      * Sample code: Get Container App's diagnostics info.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppSDiagnosticsInfo(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsDiagnostics()
-            .getDetectorWithResponse(
-                "mikono-workerapp-test-rg",
-                "mikono-capp-stage1",
-                "cappcontainerappnetworkIO",
-                com.azure.core.util.Context.NONE);
+    public static void
+        getContainerAppSDiagnosticsInfo(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsDiagnostics().getDetectorWithResponse("mikono-workerapp-test-rg", "mikono-capp-stage1",
+            "cappcontainerappnetworkIO", com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1685,22 +2232,23 @@ public final class ContainerAppsDiagnosticsGetDetectorSamples {
 ### ContainerAppsDiagnostics_GetRevision
 
 ```java
-/** Samples for ContainerAppsDiagnostics GetRevision. */
+/**
+ * Samples for ContainerAppsDiagnostics GetRevision.
+ */
 public final class ContainerAppsDiagnosticsGetRevisionSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Get.json
      */
     /**
      * Sample code: Get Container App's revision.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppSRevision(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsDiagnostics()
-            .getRevisionWithResponse(
-                "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+    public static void
+        getContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsDiagnostics().getRevisionWithResponse("rg", "testcontainerApp0",
+            "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1708,20 +2256,22 @@ public final class ContainerAppsDiagnosticsGetRevisionSamples {
 ### ContainerAppsDiagnostics_GetRoot
 
 ```java
-/** Samples for ContainerAppsDiagnostics GetRoot. */
+/**
+ * Samples for ContainerAppsDiagnostics GetRoot.
+ */
 public final class ContainerAppsDiagnosticsGetRootSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Get.json
      */
     /**
      * Sample code: Get Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsDiagnostics()
-            .getRootWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE);
+        manager.containerAppsDiagnostics().getRootWithResponse("rg", "testcontainerApp0",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1729,21 +2279,23 @@ public final class ContainerAppsDiagnosticsGetRootSamples {
 ### ContainerAppsDiagnostics_ListDetectors
 
 ```java
-/** Samples for ContainerAppsDiagnostics ListDetectors. */
+/**
+ * Samples for ContainerAppsDiagnostics ListDetectors.
+ */
 public final class ContainerAppsDiagnosticsListDetectorsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerAppsDiagnostics_List.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerAppsDiagnostics_List.json
      */
     /**
      * Sample code: Get the list of available diagnostics for a given Container App.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getTheListOfAvailableDiagnosticsForAGivenContainerApp(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsDiagnostics()
-            .listDetectors("mikono-workerapp-test-rg", "mikono-capp-stage1", com.azure.core.util.Context.NONE);
+        manager.containerAppsDiagnostics().listDetectors("mikono-workerapp-test-rg", "mikono-capp-stage1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1751,21 +2303,23 @@ public final class ContainerAppsDiagnosticsListDetectorsSamples {
 ### ContainerAppsDiagnostics_ListRevisions
 
 ```java
-/** Samples for ContainerAppsDiagnostics ListRevisions. */
+/**
+ * Samples for ContainerAppsDiagnostics ListRevisions.
+ */
 public final class ContainerAppsDiagnosticsListRevisionsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_List.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_List.json
      */
     /**
      * Sample code: List Container App's revisions.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppSRevisions(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsDiagnostics()
-            .listRevisions("rg", "testcontainerApp0", null, com.azure.core.util.Context.NONE);
+    public static void
+        listContainerAppSRevisions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsDiagnostics().listRevisions("rg", "testcontainerApp0", null,
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1773,26 +2327,23 @@ public final class ContainerAppsDiagnosticsListRevisionsSamples {
 ### ContainerAppsRevisionReplicas_GetReplica
 
 ```java
-/** Samples for ContainerAppsRevisionReplicas GetReplica. */
+/**
+ * Samples for ContainerAppsRevisionReplicas GetReplica.
+ */
 public final class ContainerAppsRevisionReplicasGetReplicaSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Replicas_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Replicas_Get.json
      */
     /**
      * Sample code: Get Container App's revision replica.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppSRevisionReplica(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisionReplicas()
-            .getReplicaWithResponse(
-                "workerapps-rg-xj",
-                "myapp",
-                "myapp--0wlqy09",
-                "myapp--0wlqy09-5d9774cff-5wnd8",
-                com.azure.core.util.Context.NONE);
+    public static void
+        getContainerAppSRevisionReplica(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisionReplicas().getReplicaWithResponse("workerapps-rg-xj", "myapp", "myapp--0wlqy09",
+            "myapp--0wlqy09-5d9774cff-5wnd8", com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1800,21 +2351,23 @@ public final class ContainerAppsRevisionReplicasGetReplicaSamples {
 ### ContainerAppsRevisionReplicas_ListReplicas
 
 ```java
-/** Samples for ContainerAppsRevisionReplicas ListReplicas. */
+/**
+ * Samples for ContainerAppsRevisionReplicas ListReplicas.
+ */
 public final class ContainerAppsRevisionReplicasListReplicasSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Replicas_List.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Replicas_List.json
      */
     /**
      * Sample code: List Container App's replicas.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppSReplicas(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisionReplicas()
-            .listReplicasWithResponse("workerapps-rg-xj", "myapp", "myapp--0wlqy09", com.azure.core.util.Context.NONE);
+    public static void
+        listContainerAppSReplicas(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisionReplicas().listReplicasWithResponse("workerapps-rg-xj", "myapp", "myapp--0wlqy09",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1822,22 +2375,23 @@ public final class ContainerAppsRevisionReplicasListReplicasSamples {
 ### ContainerAppsRevisions_ActivateRevision
 
 ```java
-/** Samples for ContainerAppsRevisions ActivateRevision. */
+/**
+ * Samples for ContainerAppsRevisions ActivateRevision.
+ */
 public final class ContainerAppsRevisionsActivateRevisionSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Activate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Activate.json
      */
     /**
      * Sample code: Activate Container App's revision.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void activateContainerAppSRevision(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisions()
-            .activateRevisionWithResponse(
-                "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+    public static void
+        activateContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisions().activateRevisionWithResponse("rg", "testcontainerApp0",
+            "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -1845,378 +2399,1165 @@ public final class ContainerAppsRevisionsActivateRevisionSamples {
 ### ContainerAppsRevisions_DeactivateRevision
 
 ```java
-/** Samples for ContainerAppsRevisions DeactivateRevision. */
+/**
+ * Samples for ContainerAppsRevisions DeactivateRevision.
+ */
 public final class ContainerAppsRevisionsDeactivateRevisionSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Deactivate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Deactivate.json
+     */
+    /**
+     * Sample code: Deactivate Container App's revision.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        deactivateContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisions().deactivateRevisionWithResponse("rg", "testcontainerApp0",
+            "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ContainerAppsRevisions_GetRevision
+
+```java
+/**
+ * Samples for ContainerAppsRevisions GetRevision.
+ */
+public final class ContainerAppsRevisionsGetRevisionSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Get.json
+     */
+    /**
+     * Sample code: Get Container App's revision.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        getContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisions().getRevisionWithResponse("rg", "testcontainerApp0", "testcontainerApp0-pjxhsye",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ContainerAppsRevisions_ListRevisions
+
+```java
+/**
+ * Samples for ContainerAppsRevisions ListRevisions.
+ */
+public final class ContainerAppsRevisionsListRevisionsSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_List.json
+     */
+    /**
+     * Sample code: List Container App's revisions.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        listContainerAppSRevisions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisions().listRevisions("rg", "testcontainerApp0", null,
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ContainerAppsRevisions_RestartRevision
+
+```java
+/**
+ * Samples for ContainerAppsRevisions RestartRevision.
+ */
+public final class ContainerAppsRevisionsRestartRevisionSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Restart.json
+     */
+    /**
+     * Sample code: Restart Container App's revision.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        restartContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsRevisions().restartRevisionWithResponse("rg", "testStaticSite0",
+            "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ContainerAppsSourceControls_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.AzureCredentials;
+import com.azure.resourcemanager.appcontainers.models.EnvironmentVariable;
+import com.azure.resourcemanager.appcontainers.models.GithubActionConfiguration;
+import com.azure.resourcemanager.appcontainers.models.RegistryInfo;
+import java.util.Arrays;
+
+/**
+ * Samples for ContainerAppsSourceControls CreateOrUpdate.
+ */
+public final class ContainerAppsSourceControlsCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * SourceControls_CreateOrUpdate.json
+     */
+    /**
+     * Sample code: Create or Update Container App SourceControl.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateContainerAppSourceControl(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsSourceControls().define("current")
+            .withExistingContainerApp("workerapps-rg-xj", "testcanadacentral")
+            .withRepoUrl("https://github.com/xwang971/ghatest").withBranch("master")
+            .withGithubActionConfiguration(new GithubActionConfiguration()
+                .withRegistryInfo(new RegistryInfo().withRegistryUrl("test-registry.azurecr.io")
+                    .withRegistryUsername("test-registry").withRegistryPassword("fakeTokenPlaceholder"))
+                .withAzureCredentials(new AzureCredentials().withClientId("")
+                    .withClientSecret("fakeTokenPlaceholder").withTenantId("").withKind("feaderated"))
+                .withContextPath("./").withGithubPersonalAccessToken("fakeTokenPlaceholder").withImage("image/tag")
+                .withBuildEnvironmentVariables(
+                    Arrays.asList(new EnvironmentVariable().withName("foo1").withValue("bar1"),
+                        new EnvironmentVariable().withName("foo2").withValue("bar2"))))
+            .create();
+    }
+}
+```
+
+### ContainerAppsSourceControls_Delete
+
+```java
+/**
+ * Samples for ContainerAppsSourceControls Delete.
+ */
+public final class ContainerAppsSourceControlsDeleteSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/SourceControls_Delete.json
+     */
+    /**
+     * Sample code: Delete Container App SourceControl.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        deleteContainerAppSourceControl(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsSourceControls().delete("workerapps-rg-xj", "testcanadacentral", "current",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ContainerAppsSourceControls_Get
+
+```java
+/**
+ * Samples for ContainerAppsSourceControls Get.
+ */
+public final class ContainerAppsSourceControlsGetSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/SourceControls_Get.json
+     */
+    /**
+     * Sample code: Get Container App's SourceControl.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        getContainerAppSSourceControl(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsSourceControls().getWithResponse("workerapps-rg-xj", "testcanadacentral", "current",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ContainerAppsSourceControls_ListByContainerApp
+
+```java
+/**
+ * Samples for ContainerAppsSourceControls ListByContainerApp.
+ */
+public final class ContainerAppsSourceControlsListByContainerAppSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * SourceControls_ListByContainer.json
+     */
+    /**
+     * Sample code: List App's Source Controls.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listAppSSourceControls(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.containerAppsSourceControls().listByContainerApp("workerapps-rg-xj", "testcanadacentral",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponentResiliencyPolicies_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration;
+import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration;
+import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration;
+import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration;
+import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration;
+
+/**
+ * Samples for DaprComponentResiliencyPolicies CreateOrUpdate.
+ */
+public final class DaprComponentResiliencyPoliciesCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponentResiliencyPolicy_CreateOrUpdate_OutboundOnly.json
+     */
+    /**
+     * Sample code: Create or update dapr component resiliency policy with outbound policy only.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprComponentResiliencyPolicyWithOutboundPolicyOnly(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponentResiliencyPolicies().define("myresiliencypolicy")
+            .withExistingDaprComponent("examplerg", "myenvironment", "mydaprcomponent")
+            .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration()
+                .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(5)
+                    .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration()
+                        .withInitialDelayInMilliseconds(100).withMaxIntervalInMilliseconds(30000)))
+                .withTimeoutPolicy(
+                    new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(12))
+                .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration()
+                    .withConsecutiveErrors(3).withTimeoutInSeconds(20).withIntervalInSeconds(60)))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponentResiliencyPolicy_CreateOrUpdate_AllOptions.json
+     */
+    /**
+     * Sample code: Create or update dapr component resiliency policy with all options.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprComponentResiliencyPolicyWithAllOptions(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponentResiliencyPolicies().define("myresiliencypolicy")
+            .withExistingDaprComponent("examplerg", "myenvironment", "mydaprcomponent")
+            .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration()
+                .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(15)
+                    .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration()
+                        .withInitialDelayInMilliseconds(2000).withMaxIntervalInMilliseconds(5500)))
+                .withTimeoutPolicy(
+                    new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(30))
+                .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration()
+                    .withConsecutiveErrors(5).withTimeoutInSeconds(10).withIntervalInSeconds(4)))
+            .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration()
+                .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(5)
+                    .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration()
+                        .withInitialDelayInMilliseconds(100).withMaxIntervalInMilliseconds(30000)))
+                .withTimeoutPolicy(
+                    new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(12))
+                .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration()
+                    .withConsecutiveErrors(3).withTimeoutInSeconds(20).withIntervalInSeconds(60)))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponentResiliencyPolicy_CreateOrUpdate_SparseOptions.json
+     */
+    /**
+     * Sample code: Create or update dapr component resiliency policy with sparse options.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprComponentResiliencyPolicyWithSparseOptions(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponentResiliencyPolicies().define("myresiliencypolicy")
+            .withExistingDaprComponent("examplerg", "myenvironment", "mydaprcomponent")
+            .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration()
+                .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(5)
+                    .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration()
+                        .withInitialDelayInMilliseconds(2000).withMaxIntervalInMilliseconds(5500)))
+                .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration()
+                    .withConsecutiveErrors(3).withTimeoutInSeconds(20)))
+            .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration().withTimeoutPolicy(
+                new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(12)))
+            .create();
+    }
+}
+```
+
+### DaprComponentResiliencyPolicies_Delete
+
+```java
+/**
+ * Samples for DaprComponentResiliencyPolicies Delete.
+ */
+public final class DaprComponentResiliencyPoliciesDeleteSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponentResiliencyPolicies_Delete.json
+     */
+    /**
+     * Sample code: Delete dapr component resiliency policy.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        deleteDaprComponentResiliencyPolicy(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponentResiliencyPolicies().deleteWithResponse("examplerg", "myenvironment", "mydaprcomponent",
+            "myresiliencypolicy", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponentResiliencyPolicies_Get
+
+```java
+/**
+ * Samples for DaprComponentResiliencyPolicies Get.
+ */
+public final class DaprComponentResiliencyPoliciesGetSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponentResiliencyPolicies_Get.json
+     */
+    /**
+     * Sample code: Get Dapr component resiliency policy.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        getDaprComponentResiliencyPolicy(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponentResiliencyPolicies().getWithResponse("examplerg", "myenvironment", "mydaprcomponent",
+            "myresiliencypolicy", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponentResiliencyPolicies_List
+
+```java
+/**
+ * Samples for DaprComponentResiliencyPolicies List.
+ */
+public final class DaprComponentResiliencyPoliciesListSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponentResiliencyPolicies_List.json
+     */
+    /**
+     * Sample code: List Dapr component resiliency policies.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        listDaprComponentResiliencyPolicies(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponentResiliencyPolicies().list("examplerg", "myenvironment", "mydaprcomponent",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponents_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner;
+import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding;
+import com.azure.resourcemanager.appcontainers.models.DaprMetadata;
+import com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata;
+import com.azure.resourcemanager.appcontainers.models.Secret;
+import java.util.Arrays;
+
+/**
+ * Samples for DaprComponents CreateOrUpdate.
+ */
+public final class DaprComponentsCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponents_CreateOrUpdate_Secrets.json
+     */
+    /**
+     * Sample code: Create or update dapr component with secrets.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprComponentWithSecrets(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().createOrUpdateWithResponse("examplerg", "myenvironment", "reddog",
+            new DaprComponentInner().withComponentType("state.azure.cosmosdb").withVersion("v1").withIgnoreErrors(false)
+                .withInitTimeout("50s")
+                .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue")))
+                .withMetadata(Arrays.asList(new DaprMetadata().withName("url").withValue(""),
+                    new DaprMetadata().withName("database").withValue("itemsDB"),
+                    new DaprMetadata().withName("collection").withValue("items"),
+                    new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder")))
+                .withScopes(Arrays.asList("container-app-1", "container-app-2"))
+                .withServiceComponentBind(Arrays.asList(new DaprComponentServiceBinding().withName("statestore")
+                    .withServiceId(
+                        "/subscriptions/9f7371f1-b593-4c3c-84e2-9167806ad358/resourceGroups/ca-syn2-group/providers/Microsoft.App/containerapps/cappredis")
+                    .withMetadata(
+                        new DaprServiceBindMetadata().withName("daprcomponentBind").withValue("redis-bind")))),
+            com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponents_CreateOrUpdate_SecretStoreComponent.json
+     */
+    /**
+     * Sample code: Create or update dapr component with secret store component.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprComponentWithSecretStoreComponent(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().createOrUpdateWithResponse("examplerg", "myenvironment", "reddog",
+            new DaprComponentInner().withComponentType("state.azure.cosmosdb").withVersion("v1").withIgnoreErrors(false)
+                .withInitTimeout("50s").withSecretStoreComponent("fakeTokenPlaceholder")
+                .withMetadata(Arrays.asList(new DaprMetadata().withName("url").withValue(""),
+                    new DaprMetadata().withName("database").withValue("itemsDB"),
+                    new DaprMetadata().withName("collection").withValue("items"),
+                    new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder")))
+                .withScopes(Arrays.asList("container-app-1", "container-app-2"))
+                .withServiceComponentBind(Arrays.asList(new DaprComponentServiceBinding().withName("statestore")
+                    .withServiceId(
+                        "/subscriptions/9f7371f1-b593-4c3c-84e2-9167806ad358/resourceGroups/ca-syn2-group/providers/Microsoft.App/containerapps/cappredis")
+                    .withMetadata(
+                        new DaprServiceBindMetadata().withName("daprcomponentBind").withValue("redis-bind")))),
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponents_Delete
+
+```java
+/**
+ * Samples for DaprComponents Delete.
+ */
+public final class DaprComponentsDeleteSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_Delete.json
+     */
+    /**
+     * Sample code: Delete dapr component.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void deleteDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().deleteWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponents_Get
+
+```java
+/**
+ * Samples for DaprComponents Get.
+ */
+public final class DaprComponentsGetSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprComponents_Get_SecretStoreComponent.json
+     */
+    /**
+     * Sample code: Get Dapr Component with secret store component.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getDaprComponentWithSecretStoreComponent(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().getWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_Get_Secrets.
+     * json
+     */
+    /**
+     * Sample code: Get Dapr Component with secrets.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        getDaprComponentWithSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().getWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponents_List
+
+```java
+/**
+ * Samples for DaprComponents List.
+ */
+public final class DaprComponentsListSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_List.json
+     */
+    /**
+     * Sample code: List Dapr Components.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listDaprComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprComponents_ListSecrets
+
+```java
+/**
+ * Samples for DaprComponents ListSecrets.
+ */
+public final class DaprComponentsListSecretsSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_ListSecrets.
+     * json
+     */
+    /**
+     * Sample code: List Container Apps Secrets.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        listContainerAppsSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprComponents().listSecretsWithResponse("examplerg", "myenvironment", "reddog",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprSubscriptions_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions;
+import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule;
+import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Samples for DaprSubscriptions CreateOrUpdate.
+ */
+public final class DaprSubscriptionsCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprSubscriptions_CreateOrUpdate_RouteRulesAndMetadata.json
+     */
+    /**
+     * Sample code: Create or update dapr subscription with route rules and metadata.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprSubscriptionWithRouteRulesAndMetadata(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().define("mysubscription")
+            .withExistingManagedEnvironment("examplerg", "myenvironment").withPubsubName("mypubsubcomponent")
+            .withTopic("inventory")
+            .withRoutes(new DaprSubscriptionRoutes()
+                .withRules(Arrays.asList(
+                    new DaprSubscriptionRouteRule().withMatch("event.type == 'widget'").withPath("/widgets"),
+                    new DaprSubscriptionRouteRule().withMatch("event.type == 'gadget'").withPath("/gadgets")))
+                .withDefaultProperty("/products"))
+            .withMetadata(mapOf("foo", "bar", "hello", "world")).create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprSubscriptions_CreateOrUpdate_DefaultRoute.json
+     */
+    /**
+     * Sample code: Create or update dapr subscription with default route only.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprSubscriptionWithDefaultRouteOnly(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().define("mysubscription")
+            .withExistingManagedEnvironment("examplerg", "myenvironment").withPubsubName("mypubsubcomponent")
+            .withTopic("inventory").withRoutes(new DaprSubscriptionRoutes().withDefaultProperty("/products")).create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprSubscriptions_CreateOrUpdate_BulkSubscribeAndScopes.json
+     */
+    /**
+     * Sample code: Create or update dapr subscription with bulk subscribe configuration and scopes.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateDaprSubscriptionWithBulkSubscribeConfigurationAndScopes(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().define("mysubscription")
+            .withExistingManagedEnvironment("examplerg", "myenvironment").withPubsubName("mypubsubcomponent")
+            .withTopic("inventory").withRoutes(new DaprSubscriptionRoutes().withDefaultProperty("/products"))
+            .withScopes(Arrays.asList("warehouseapp", "customersupportapp"))
+            .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(true).withMaxMessagesCount(123)
+                .withMaxAwaitDurationMs(500))
+            .create();
+    }
+
+    // Use "Map.of" if available
+    @SuppressWarnings("unchecked")
+    private static  Map mapOf(Object... inputs) {
+        Map map = new HashMap<>();
+        for (int i = 0; i < inputs.length; i += 2) {
+            String key = (String) inputs[i];
+            T value = (T) inputs[i + 1];
+            map.put(key, value);
+        }
+        return map;
+    }
+}
+```
+
+### DaprSubscriptions_Delete
+
+```java
+/**
+ * Samples for DaprSubscriptions Delete.
+ */
+public final class DaprSubscriptionsDeleteSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprSubscriptions_Delete.
+     * json
+     */
+    /**
+     * Sample code: Delete dapr subscription.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void deleteDaprSubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().deleteWithResponse("examplerg", "myenvironment", "mysubscription",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprSubscriptions_Get
+
+```java
+/**
+ * Samples for DaprSubscriptions Get.
+ */
+public final class DaprSubscriptionsGetSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprSubscriptions_Get_DefaultRoute.json
+     */
+    /**
+     * Sample code: Get Dapr subscription with bulk subscribe configuration and scopes.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getDaprSubscriptionWithBulkSubscribeConfigurationAndScopes(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().getWithResponse("examplerg", "myenvironment", "mypubsubcomponent",
+            com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprSubscriptions_Get_BulkSubscribeAndScopes.json
+     */
+    /**
+     * Sample code: Get Dapr subscription with default route only.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getDaprSubscriptionWithDefaultRouteOnly(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().getWithResponse("examplerg", "myenvironment", "mypubsubcomponent",
+            com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DaprSubscriptions_Get_RouteRulesAndMetadata.json
+     */
+    /**
+     * Sample code: GetDapr subscription with route rules and metadata.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getDaprSubscriptionWithRouteRulesAndMetadata(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().getWithResponse("examplerg", "myenvironment", "mypubsubcomponent",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DaprSubscriptions_List
+
+```java
+/**
+ * Samples for DaprSubscriptions List.
+ */
+public final class DaprSubscriptionsListSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprSubscriptions_List.json
+     */
+    /**
+     * Sample code: List Dapr subscriptions.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listDaprSubscriptions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.daprSubscriptions().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### DotNetComponents_CreateOrUpdate
+
+```java
+import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty;
+import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind;
+import com.azure.resourcemanager.appcontainers.models.DotNetComponentType;
+import java.util.Arrays;
+
+/**
+ * Samples for DotNetComponents CreateOrUpdate.
+ */
+public final class DotNetComponentsCreateOrUpdateSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DotNetComponents_CreateOrUpdate.json
+     */
+    /**
+     * Sample code: Create or Update .NET Component.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        createOrUpdateNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.dotNetComponents().define("mydotnetcomponent")
+            .withExistingManagedEnvironment("examplerg", "myenvironment")
+            .withComponentType(DotNetComponentType.ASPIRE_DASHBOARD)
+            .withConfigurations(Arrays.asList(
+                new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark")))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DotNetComponents_CreateOrUpdate_ServiceBind.json
      */
     /**
-     * Sample code: Deactivate Container App's revision.
-     *
+     * Sample code: Create or Update .NET Component with ServiceBinds.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deactivateContainerAppSRevision(
+    public static void createOrUpdateNETComponentWithServiceBinds(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisions()
-            .deactivateRevisionWithResponse(
-                "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+        manager.dotNetComponents().define("mydotnetcomponent")
+            .withExistingManagedEnvironment("examplerg", "myenvironment")
+            .withComponentType(DotNetComponentType.ASPIRE_DASHBOARD)
+            .withConfigurations(Arrays.asList(
+                new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark")))
+            .withServiceBinds(Arrays.asList(new DotNetComponentServiceBind().withName("yellowcat").withServiceId(
+                "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/dotNetComponents/yellowcat")))
+            .create();
     }
 }
 ```
 
-### ContainerAppsRevisions_GetRevision
+### DotNetComponents_Delete
 
 ```java
-/** Samples for ContainerAppsRevisions GetRevision. */
-public final class ContainerAppsRevisionsGetRevisionSamples {
+/**
+ * Samples for DotNetComponents Delete.
+ */
+public final class DotNetComponentsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_Delete.json
      */
     /**
-     * Sample code: Get Container App's revision.
-     *
+     * Sample code: Delete .NET Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppSRevision(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisions()
-            .getRevisionWithResponse(
-                "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+    public static void deleteNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.dotNetComponents().delete("examplerg", "myenvironment", "mydotnetcomponent",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ContainerAppsRevisions_ListRevisions
+### DotNetComponents_Get
 
 ```java
-/** Samples for ContainerAppsRevisions ListRevisions. */
-public final class ContainerAppsRevisionsListRevisionsSamples {
+/**
+ * Samples for DotNetComponents Get.
+ */
+public final class DotNetComponentsGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_List.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DotNetComponents_Get_ServiceBind.json
      */
     /**
-     * Sample code: List Container App's revisions.
-     *
+     * Sample code: Get .NET Component with ServiceBinds.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppSRevisions(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisions()
-            .listRevisions("rg", "testcontainerApp0", null, com.azure.core.util.Context.NONE);
+    public static void
+        getNETComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.dotNetComponents().getWithResponse("examplerg", "myenvironment", "mydotnetcomponent",
+            com.azure.core.util.Context.NONE);
     }
-}
-```
-
-### ContainerAppsRevisions_RestartRevision
 
-```java
-/** Samples for ContainerAppsRevisions RestartRevision. */
-public final class ContainerAppsRevisionsRestartRevisionSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Restart.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_Get.json
      */
     /**
-     * Sample code: Restart Container App's revision.
-     *
+     * Sample code: Get .NET Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void restartContainerAppSRevision(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsRevisions()
-            .restartRevisionWithResponse(
-                "rg", "testStaticSite0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE);
+    public static void getNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.dotNetComponents().getWithResponse("examplerg", "myenvironment", "mydotnetcomponent",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ContainerAppsSourceControls_CreateOrUpdate
+### DotNetComponents_List
 
 ```java
-import com.azure.resourcemanager.appcontainers.models.AzureCredentials;
-import com.azure.resourcemanager.appcontainers.models.GithubActionConfiguration;
-import com.azure.resourcemanager.appcontainers.models.RegistryInfo;
-
-/** Samples for ContainerAppsSourceControls CreateOrUpdate. */
-public final class ContainerAppsSourceControlsCreateOrUpdateSamples {
+/**
+ * Samples for DotNetComponents List.
+ */
+public final class DotNetComponentsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_CreateOrUpdate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_List.json
      */
     /**
-     * Sample code: Create or Update Container App SourceControl.
-     *
+     * Sample code: List .NET Components.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateContainerAppSourceControl(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsSourceControls()
-            .define("current")
-            .withExistingContainerApp("workerapps-rg-xj", "testcanadacentral")
-            .withRepoUrl("https://github.com/xwang971/ghatest")
-            .withBranch("master")
-            .withGithubActionConfiguration(
-                new GithubActionConfiguration()
-                    .withRegistryInfo(
-                        new RegistryInfo()
-                            .withRegistryUrl("test-registry.azurecr.io")
-                            .withRegistryUsername("test-registry")
-                            .withRegistryPassword("fakeTokenPlaceholder"))
-                    .withAzureCredentials(
-                        new AzureCredentials()
-                            .withClientId("")
-                            .withClientSecret("fakeTokenPlaceholder")
-                            .withTenantId("")
-                            .withKind("feaderated"))
-                    .withContextPath("./")
-                    .withGithubPersonalAccessToken("fakeTokenPlaceholder")
-                    .withImage("image/tag"))
-            .create();
+    public static void listNETComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.dotNetComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
     }
-}
-```
 
-### ContainerAppsSourceControls_Delete
-
-```java
-/** Samples for ContainerAppsSourceControls Delete. */
-public final class ContainerAppsSourceControlsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_Delete.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DotNetComponents_List_ServiceBind.json
      */
     /**
-     * Sample code: Delete Container App SourceControl.
-     *
+     * Sample code: List .NET Components with ServiceBinds.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteContainerAppSourceControl(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsSourceControls()
-            .delete("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE);
+    public static void
+        listNETComponentsWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.dotNetComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### ContainerAppsSourceControls_Get
+### DotNetComponents_Update
 
 ```java
-/** Samples for ContainerAppsSourceControls Get. */
-public final class ContainerAppsSourceControlsGetSamples {
+import com.azure.resourcemanager.appcontainers.models.DotNetComponent;
+import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty;
+import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind;
+import com.azure.resourcemanager.appcontainers.models.DotNetComponentType;
+import java.util.Arrays;
+
+/**
+ * Samples for DotNetComponents Update.
+ */
+public final class DotNetComponentsUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_Get.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * DotNetComponents_Patch_ServiceBind.json
      */
     /**
-     * Sample code: Get Container App's SourceControl.
-     *
+     * Sample code: Patch .NET Component with ServiceBinds.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getContainerAppSSourceControl(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsSourceControls()
-            .getWithResponse("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE);
+    public static void
+        patchNETComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        DotNetComponent resource = manager.dotNetComponents()
+            .getWithResponse("examplerg", "myenvironment", "mydotnetcomponent", com.azure.core.util.Context.NONE)
+            .getValue();
+        resource.update().withComponentType(DotNetComponentType.ASPIRE_DASHBOARD)
+            .withConfigurations(Arrays.asList(
+                new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark")))
+            .withServiceBinds(Arrays.asList(new DotNetComponentServiceBind().withName("yellowcat").withServiceId(
+                "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/dotNetComponents/yellowcat")))
+            .apply();
     }
-}
-```
-
-### ContainerAppsSourceControls_ListByContainerApp
 
-```java
-/** Samples for ContainerAppsSourceControls ListByContainerApp. */
-public final class ContainerAppsSourceControlsListByContainerAppSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_ListByContainer.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_Patch.json
      */
     /**
-     * Sample code: List App's Source Controls.
-     *
+     * Sample code: Patch .NET Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listAppSSourceControls(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .containerAppsSourceControls()
-            .listByContainerApp("workerapps-rg-xj", "testcanadacentral", com.azure.core.util.Context.NONE);
+    public static void patchNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        DotNetComponent resource = manager.dotNetComponents()
+            .getWithResponse("examplerg", "myenvironment", "mydotnetcomponent", com.azure.core.util.Context.NONE)
+            .getValue();
+        resource.update().withComponentType(DotNetComponentType.ASPIRE_DASHBOARD)
+            .withConfigurations(Arrays.asList(
+                new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark")))
+            .apply();
     }
 }
 ```
 
-### DaprComponents_CreateOrUpdate
+### JavaComponents_CreateOrUpdate
 
 ```java
-import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner;
-import com.azure.resourcemanager.appcontainers.models.DaprMetadata;
-import com.azure.resourcemanager.appcontainers.models.Secret;
+import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty;
+import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind;
+import com.azure.resourcemanager.appcontainers.models.JavaComponentType;
 import java.util.Arrays;
 
-/** Samples for DaprComponents CreateOrUpdate. */
-public final class DaprComponentsCreateOrUpdateSamples {
+/**
+ * Samples for JavaComponents CreateOrUpdate.
+ */
+public final class JavaComponentsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_CreateOrUpdate_Secrets.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * JavaComponents_CreateOrUpdate_ServiceBind.json
      */
     /**
-     * Sample code: Create or update dapr component with secrets.
-     *
+     * Sample code: Create or Update Java Component with ServiceBinds.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateDaprComponentWithSecrets(
+    public static void createOrUpdateJavaComponentWithServiceBinds(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .daprComponents()
-            .createOrUpdateWithResponse(
-                "examplerg",
-                "myenvironment",
-                "reddog",
-                new DaprComponentInner()
-                    .withComponentType("state.azure.cosmosdb")
-                    .withVersion("v1")
-                    .withIgnoreErrors(false)
-                    .withInitTimeout("50s")
-                    .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue")))
-                    .withMetadata(
-                        Arrays
-                            .asList(
-                                new DaprMetadata().withName("url").withValue(""),
-                                new DaprMetadata().withName("database").withValue("itemsDB"),
-                                new DaprMetadata().withName("collection").withValue("items"),
-                                new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder")))
-                    .withScopes(Arrays.asList("container-app-1", "container-app-2")),
-                com.azure.core.util.Context.NONE);
+        manager.javaComponents().define("myjavacomponent").withExistingManagedEnvironment("examplerg", "myenvironment")
+            .withComponentType(JavaComponentType.SPRING_BOOT_ADMIN)
+            .withConfigurations(Arrays.asList(
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts")
+                    .withValue("true"),
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval")
+                    .withValue("10000ms")))
+            .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("yellowcat").withServiceId(
+                "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/javaComponents/yellowcat")))
+            .create();
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_CreateOrUpdate_SecretStoreComponent.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * JavaComponents_CreateOrUpdate.json
      */
     /**
-     * Sample code: Create or update dapr component with secret store component.
-     *
+     * Sample code: Create or Update Java Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateDaprComponentWithSecretStoreComponent(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .daprComponents()
-            .createOrUpdateWithResponse(
-                "examplerg",
-                "myenvironment",
-                "reddog",
-                new DaprComponentInner()
-                    .withComponentType("state.azure.cosmosdb")
-                    .withVersion("v1")
-                    .withIgnoreErrors(false)
-                    .withInitTimeout("50s")
-                    .withSecretStoreComponent("fakeTokenPlaceholder")
-                    .withMetadata(
-                        Arrays
-                            .asList(
-                                new DaprMetadata().withName("url").withValue(""),
-                                new DaprMetadata().withName("database").withValue("itemsDB"),
-                                new DaprMetadata().withName("collection").withValue("items"),
-                                new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder")))
-                    .withScopes(Arrays.asList("container-app-1", "container-app-2")),
-                com.azure.core.util.Context.NONE);
+    public static void
+        createOrUpdateJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.javaComponents().define("myjavacomponent").withExistingManagedEnvironment("examplerg", "myenvironment")
+            .withComponentType(JavaComponentType.SPRING_BOOT_ADMIN)
+            .withConfigurations(Arrays.asList(
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts")
+                    .withValue("true"),
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval")
+                    .withValue("10000ms")))
+            .create();
     }
 }
 ```
 
-### DaprComponents_Delete
+### JavaComponents_Delete
 
 ```java
-/** Samples for DaprComponents Delete. */
-public final class DaprComponentsDeleteSamples {
+/**
+ * Samples for JavaComponents Delete.
+ */
+public final class JavaComponentsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_Delete.json
      */
     /**
-     * Sample code: Delete dapr component.
-     *
+     * Sample code: Delete Java Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .daprComponents()
-            .deleteWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+    public static void deleteJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.javaComponents().delete("examplerg", "myenvironment", "myjavacomponent",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### DaprComponents_Get
+### JavaComponents_Get
 
 ```java
-/** Samples for DaprComponents Get. */
-public final class DaprComponentsGetSamples {
+/**
+ * Samples for JavaComponents Get.
+ */
+public final class JavaComponentsGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_Get_SecretStoreComponent.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_Get.json
      */
     /**
-     * Sample code: Get Dapr Component with secret store component.
-     *
+     * Sample code: Get Java Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getDaprComponentWithSecretStoreComponent(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .daprComponents()
-            .getWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+    public static void getJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.javaComponents().getWithResponse("examplerg", "myenvironment", "myjavacomponent",
+            com.azure.core.util.Context.NONE);
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_Get_Secrets.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * JavaComponents_Get_ServiceBind.json
      */
     /**
-     * Sample code: Get Dapr Component with secrets.
-     *
+     * Sample code: Get Java Component with ServiceBinds.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getDaprComponentWithSecrets(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .daprComponents()
-            .getWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+    public static void
+        getJavaComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.javaComponents().getWithResponse("examplerg", "myenvironment", "myjavacomponent",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### DaprComponents_List
+### JavaComponents_List
 
 ```java
-/** Samples for DaprComponents List. */
-public final class DaprComponentsListSamples {
+/**
+ * Samples for JavaComponents List.
+ */
+public final class JavaComponentsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_List.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_List.json
      */
     /**
-     * Sample code: List Dapr Components.
-     *
+     * Sample code: List Java Components.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listDaprComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager.daprComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
+    public static void listJavaComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.javaComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * JavaComponents_List_ServiceBind.json
+     */
+    /**
+     * Sample code: List Java Components with ServiceBinds.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        listJavaComponentsWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.javaComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
-### DaprComponents_ListSecrets
+### JavaComponents_Update
 
 ```java
-/** Samples for DaprComponents ListSecrets. */
-public final class DaprComponentsListSecretsSamples {
+import com.azure.resourcemanager.appcontainers.models.JavaComponent;
+import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty;
+import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind;
+import com.azure.resourcemanager.appcontainers.models.JavaComponentType;
+import java.util.Arrays;
+
+/**
+ * Samples for JavaComponents Update.
+ */
+public final class JavaComponentsUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_ListSecrets.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_Patch.json
      */
     /**
-     * Sample code: List Container Apps Secrets.
-     *
+     * Sample code: Patch Java Component.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsSecrets(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .daprComponents()
-            .listSecretsWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE);
+    public static void patchJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        JavaComponent resource = manager.javaComponents()
+            .getWithResponse("examplerg", "myenvironment", "myjavacomponent", com.azure.core.util.Context.NONE)
+            .getValue();
+        resource.update().withComponentType(JavaComponentType.SPRING_BOOT_ADMIN)
+            .withConfigurations(Arrays.asList(
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts")
+                    .withValue("true"),
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval")
+                    .withValue("10000ms")))
+            .apply();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * JavaComponents_Patch_ServiceBind.json
+     */
+    /**
+     * Sample code: Patch Java Component with ServiceBinds.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        patchJavaComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        JavaComponent resource = manager.javaComponents()
+            .getWithResponse("examplerg", "myenvironment", "myjavacomponent", com.azure.core.util.Context.NONE)
+            .getValue();
+        resource.update().withComponentType(JavaComponentType.SPRING_BOOT_ADMIN)
+            .withConfigurations(Arrays.asList(
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts")
+                    .withValue("true"),
+                new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval")
+                    .withValue("10000ms")))
+            .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("yellowcat").withServiceId(
+                "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/javaComponents/yellowcat")))
+            .apply();
     }
 }
 ```
@@ -2231,6 +3572,8 @@ import com.azure.resourcemanager.appcontainers.models.ContainerAppProbe;
 import com.azure.resourcemanager.appcontainers.models.ContainerAppProbeHttpGet;
 import com.azure.resourcemanager.appcontainers.models.ContainerAppProbeHttpGetHttpHeadersItem;
 import com.azure.resourcemanager.appcontainers.models.ContainerResources;
+import com.azure.resourcemanager.appcontainers.models.ExtendedLocation;
+import com.azure.resourcemanager.appcontainers.models.ExtendedLocationTypes;
 import com.azure.resourcemanager.appcontainers.models.InitContainer;
 import com.azure.resourcemanager.appcontainers.models.JobConfiguration;
 import com.azure.resourcemanager.appcontainers.models.JobConfigurationEventTriggerConfig;
@@ -2240,133 +3583,118 @@ import com.azure.resourcemanager.appcontainers.models.JobScaleRule;
 import com.azure.resourcemanager.appcontainers.models.JobTemplate;
 import com.azure.resourcemanager.appcontainers.models.TriggerType;
 import com.azure.resourcemanager.appcontainers.models.Type;
+import com.azure.resourcemanager.appcontainers.models.VolumeMount;
 import java.io.IOException;
 import java.util.Arrays;
 
-/** Samples for Jobs CreateOrUpdate. */
+/**
+ * Samples for Jobs CreateOrUpdate.
+ */
 public final class JobsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_CreateorUpdate_EventTrigger.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Job_CreateorUpdate_ConnectedEnvironment.json
+     */
+    /**
+     * Sample code: Create or Update Container Apps Job On A Connected Environment.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void createOrUpdateContainerAppsJobOnAConnectedEnvironment(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.jobs().define("testcontainerAppsJob0").withRegion("East US").withExistingResourceGroup("rg")
+            .withExtendedLocation(new ExtendedLocation().withName(
+                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.ExtendedLocation/customLocations/testcustomlocation")
+                .withType(ExtendedLocationTypes.CUSTOM_LOCATION))
+            .withEnvironmentId(
+                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube")
+            .withConfiguration(new JobConfiguration().withTriggerType(TriggerType.MANUAL).withReplicaTimeout(10)
+                .withReplicaRetryLimit(10).withManualTriggerConfig(
+                    new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(1).withParallelism(4)))
+            .withTemplate(new JobTemplate()
+                .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4")
+                    .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh"))
+                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                .withContainers(Arrays
+                    .asList(new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0")
+                        .withProbes(Arrays.asList(new ContainerAppProbe()
+                            .withHttpGet(new ContainerAppProbeHttpGet()
+                                .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem()
+                                    .withName("Custom-Header").withValue("Awesome")))
+                                .withPath("/health").withPort(8080))
+                            .withInitialDelaySeconds(5).withPeriodSeconds(3).withType(Type.LIVENESS))))))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Job_CreateorUpdate_EventTrigger.json
      */
     /**
      * Sample code: Create or Update Container Apps Job With Event Driven Trigger.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void createOrUpdateContainerAppsJobWithEventDrivenTrigger(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) throws IOException {
-        manager
-            .jobs()
-            .define("testcontainerAppsJob0")
-            .withRegion("East US")
-            .withExistingResourceGroup("rg")
+        manager.jobs().define("testcontainerAppsJob0").withRegion("East US").withExistingResourceGroup("rg")
             .withEnvironmentId(
                 "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube")
-            .withConfiguration(
-                new JobConfiguration()
-                    .withTriggerType(TriggerType.EVENT)
-                    .withReplicaTimeout(10)
-                    .withReplicaRetryLimit(10)
-                    .withEventTriggerConfig(
-                        new JobConfigurationEventTriggerConfig()
-                            .withReplicaCompletionCount(1)
-                            .withParallelism(4)
-                            .withScale(
-                                new JobScale()
-                                    .withPollingInterval(40)
-                                    .withMinExecutions(1)
-                                    .withMaxExecutions(5)
-                                    .withRules(
-                                        Arrays
-                                            .asList(
-                                                new JobScaleRule()
-                                                    .withName("servicebuscalingrule")
-                                                    .withType("azure-servicebus")
-                                                    .withMetadata(
-                                                        SerializerFactory
-                                                            .createDefaultManagementSerializerAdapter()
-                                                            .deserialize(
-                                                                "{\"topicName\":\"my-topic\"}",
-                                                                Object.class,
-                                                                SerializerEncoding.JSON)))))))
-            .withTemplate(
-                new JobTemplate()
-                    .withInitContainers(
-                        Arrays
-                            .asList(
-                                new InitContainer()
-                                    .withImage("repo/testcontainerAppsJob0:v4")
-                                    .withName("testinitcontainerAppsJob0")
-                                    .withCommand(Arrays.asList("/bin/sh"))
-                                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
-                                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new Container()
-                                    .withImage("repo/testcontainerAppsJob0:v1")
-                                    .withName("testcontainerAppsJob0"))))
+            .withConfiguration(new JobConfiguration().withTriggerType(TriggerType.EVENT).withReplicaTimeout(10)
+                .withReplicaRetryLimit(10).withEventTriggerConfig(
+                    new JobConfigurationEventTriggerConfig().withReplicaCompletionCount(1).withParallelism(4)
+                        .withScale(new JobScale().withPollingInterval(40).withMinExecutions(1).withMaxExecutions(5)
+                            .withRules(Arrays.asList(new JobScaleRule().withName("servicebuscalingrule")
+                                .withType("azure-servicebus")
+                                .withMetadata(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize(
+                                    "{\"topicName\":\"my-topic\"}", Object.class, SerializerEncoding.JSON)))))))
+            .withTemplate(new JobTemplate()
+                .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4")
+                    .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh"))
+                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                .withContainers(Arrays.asList(
+                    new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0"))))
             .create();
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_CreateorUpdate.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_CreateorUpdate.json
      */
     /**
      * Sample code: Create or Update Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateContainerAppsJob(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .jobs()
-            .define("testcontainerAppsJob0")
-            .withRegion("East US")
-            .withExistingResourceGroup("rg")
+    public static void
+        createOrUpdateContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.jobs().define("testcontainerAppsJob0").withRegion("East US").withExistingResourceGroup("rg")
             .withEnvironmentId(
                 "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube")
-            .withConfiguration(
-                new JobConfiguration()
-                    .withTriggerType(TriggerType.MANUAL)
-                    .withReplicaTimeout(10)
-                    .withReplicaRetryLimit(10)
-                    .withManualTriggerConfig(
-                        new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(1).withParallelism(4)))
+            .withConfiguration(new JobConfiguration().withTriggerType(TriggerType.MANUAL).withReplicaTimeout(10)
+                .withReplicaRetryLimit(10).withManualTriggerConfig(
+                    new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(1).withParallelism(4)))
             .withTemplate(
                 new JobTemplate()
-                    .withInitContainers(
-                        Arrays
-                            .asList(
-                                new InitContainer()
-                                    .withImage("repo/testcontainerAppsJob0:v4")
-                                    .withName("testinitcontainerAppsJob0")
-                                    .withCommand(Arrays.asList("/bin/sh"))
-                                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
-                                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new Container()
-                                    .withImage("repo/testcontainerAppsJob0:v1")
-                                    .withName("testcontainerAppsJob0")
-                                    .withProbes(
-                                        Arrays
-                                            .asList(
-                                                new ContainerAppProbe()
-                                                    .withHttpGet(
-                                                        new ContainerAppProbeHttpGet()
-                                                            .withHttpHeaders(
-                                                                Arrays
-                                                                    .asList(
-                                                                        new ContainerAppProbeHttpGetHttpHeadersItem()
-                                                                            .withName("Custom-Header")
-                                                                            .withValue("Awesome")))
-                                                            .withPath("/health")
-                                                            .withPort(8080))
-                                                    .withInitialDelaySeconds(5)
-                                                    .withPeriodSeconds(3)
-                                                    .withType(Type.LIVENESS))))))
+                    .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4")
+                        .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh"))
+                        .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                        .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                    .withContainers(Arrays.asList(
+                        new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0")
+                            .withVolumeMounts(Arrays.asList(
+                                new VolumeMount().withVolumeName("azurefile").withMountPath("/mnt/path1")
+                                    .withSubPath("subPath1"),
+                                new VolumeMount().withVolumeName("nfsazurefile").withMountPath("/mnt/path2")
+                                    .withSubPath("subPath2")))
+                            .withProbes(Arrays.asList(new ContainerAppProbe()
+                                .withHttpGet(new ContainerAppProbeHttpGet()
+                                    .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem()
+                                        .withName("Custom-Header").withValue("Awesome")))
+                                    .withPath("/health").withPort(8080))
+                                .withInitialDelaySeconds(5).withPeriodSeconds(3).withType(Type.LIVENESS))))))
             .create();
     }
 }
@@ -2375,14 +3703,17 @@ public final class JobsCreateOrUpdateSamples {
 ### Jobs_Delete
 
 ```java
-/** Samples for Jobs Delete. */
+/**
+ * Samples for Jobs Delete.
+ */
 public final class JobsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Delete.json
      */
     /**
      * Sample code: Delete Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void deleteContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
@@ -2394,14 +3725,17 @@ public final class JobsDeleteSamples {
 ### Jobs_GetByResourceGroup
 
 ```java
-/** Samples for Jobs GetByResourceGroup. */
+/**
+ * Samples for Jobs GetByResourceGroup.
+ */
 public final class JobsGetByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Get.json
      */
     /**
      * Sample code: Get Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
@@ -2410,21 +3744,48 @@ public final class JobsGetByResourceGroupSamples {
 }
 ```
 
+### Jobs_GetDetector
+
+```java
+/**
+ * Samples for Jobs GetDetector.
+ */
+public final class JobsGetDetectorSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_GetDetector.json
+     */
+    /**
+     * Sample code: Get diagnostic data for a Container App Job.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        getDiagnosticDataForAContainerAppJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.jobs().getDetectorWithResponse("mikono-workerapp-test-rg", "mikonojob1", "containerappjobnetworkIO",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
 ### Jobs_List
 
 ```java
-/** Samples for Jobs List. */
+/**
+ * Samples for Jobs List.
+ */
 public final class JobsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Jobs_ListBySubscription.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Jobs_ListBySubscription.json
      */
     /**
      * Sample code: List Container Apps Jobs by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsJobsBySubscription(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listContainerAppsJobsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.jobs().list(com.azure.core.util.Context.NONE);
     }
 }
@@ -2433,43 +3794,97 @@ public final class JobsListSamples {
 ### Jobs_ListByResourceGroup
 
 ```java
-/** Samples for Jobs ListByResourceGroup. */
+/**
+ * Samples for Jobs ListByResourceGroup.
+ */
 public final class JobsListByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Jobs_ListByResourceGroup.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Jobs_ListByResourceGroup.
+     * json
      */
     /**
      * Sample code: List Container Apps Jobs by resource group.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsJobsByResourceGroup(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listContainerAppsJobsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.jobs().listByResourceGroup("rg", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
+### Jobs_ListDetectors
+
+```java
+/**
+ * Samples for Jobs ListDetectors.
+ */
+public final class JobsListDetectorsSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_ListDetectors.json
+     */
+    /**
+     * Sample code: Get the list of available diagnostic data for a Container App Job.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getTheListOfAvailableDiagnosticDataForAContainerAppJob(
+        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.jobs().listDetectorsWithResponse("mikono-workerapp-test-rg", "mikonojob1",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
 ### Jobs_ListSecrets
 
 ```java
-/** Samples for Jobs ListSecrets. */
+/**
+ * Samples for Jobs ListSecrets.
+ */
 public final class JobsListSecretsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_ListSecrets.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_ListSecrets.json
      */
     /**
      * Sample code: List Container Apps Job Secrets.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listContainerAppsJobSecrets(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listContainerAppsJobSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.jobs().listSecretsWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE);
     }
 }
 ```
 
+### Jobs_ProxyGet
+
+```java
+/**
+ * Samples for Jobs ProxyGet.
+ */
+public final class JobsProxyGetSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_ProxyGet.json
+     */
+    /**
+     * Sample code: Get Container App Job by name.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        getContainerAppJobByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.jobs().proxyGetWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE);
+    }
+}
+```
+
 ### Jobs_Start
 
 ```java
@@ -2478,40 +3893,30 @@ import com.azure.resourcemanager.appcontainers.models.JobExecutionContainer;
 import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate;
 import java.util.Arrays;
 
-/** Samples for Jobs Start. */
+/**
+ * Samples for Jobs Start.
+ */
 public final class JobsStartSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Start.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Start.json
      */
     /**
      * Sample code: Run a Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void runAContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .jobs()
-            .start(
-                "rg",
-                "testcontainerAppsJob0",
-                new JobExecutionTemplate()
-                    .withContainers(
-                        Arrays
-                            .asList(
-                                new JobExecutionContainer()
-                                    .withImage("repo/testcontainerAppsJob0:v4")
-                                    .withName("testcontainerAppsJob0")
-                                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
-                    .withInitContainers(
-                        Arrays
-                            .asList(
-                                new JobExecutionContainer()
-                                    .withImage("repo/testcontainerAppsJob0:v4")
-                                    .withName("testinitcontainerAppsJob0")
-                                    .withCommand(Arrays.asList("/bin/sh"))
-                                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
-                                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))),
-                com.azure.core.util.Context.NONE);
+        manager.jobs().start("rg", "testcontainerAppsJob0",
+            new JobExecutionTemplate()
+                .withContainers(Arrays.asList(new JobExecutionContainer().withImage("repo/testcontainerAppsJob0:v4")
+                    .withName("testcontainerAppsJob0")
+                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                .withInitContainers(Arrays.asList(new JobExecutionContainer().withImage("repo/testcontainerAppsJob0:v4")
+                    .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh"))
+                    .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                    .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))),
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -2519,18 +3924,21 @@ public final class JobsStartSamples {
 ### Jobs_StopExecution
 
 ```java
-/** Samples for Jobs StopExecution. */
+/**
+ * Samples for Jobs StopExecution.
+ */
 public final class JobsStopExecutionSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Stop_Execution.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Stop_Execution.json
      */
     /**
      * Sample code: Terminate a Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void terminateAContainerAppsJob(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        terminateAContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.jobs().stopExecution("rg", "testcontainerAppsJob0", "jobExecution1", com.azure.core.util.Context.NONE);
     }
 }
@@ -2539,18 +3947,21 @@ public final class JobsStopExecutionSamples {
 ### Jobs_StopMultipleExecutions
 
 ```java
-/** Samples for Jobs StopMultipleExecutions. */
+/**
+ * Samples for Jobs StopMultipleExecutions.
+ */
 public final class JobsStopMultipleExecutionsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Stop_Multiple.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Stop_Multiple.json
      */
     /**
      * Sample code: Terminate Multiple Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void terminateMultipleContainerAppsJob(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        terminateMultipleContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.jobs().stopMultipleExecutions("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE);
     }
 }
@@ -2574,69 +3985,43 @@ import com.azure.resourcemanager.appcontainers.models.TriggerType;
 import com.azure.resourcemanager.appcontainers.models.Type;
 import java.util.Arrays;
 
-/** Samples for Jobs Update. */
+/**
+ * Samples for Jobs Update.
+ */
 public final class JobsUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Patch.json
      */
     /**
      * Sample code: Patch Container Apps Job.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void patchContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        Job resource =
-            manager
-                .jobs()
-                .getByResourceGroupWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE)
-                .getValue();
-        resource
-            .update()
+        Job resource = manager.jobs()
+            .getByResourceGroupWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE).getValue();
+        resource.update()
             .withProperties(
                 new JobPatchPropertiesProperties()
                     .withConfiguration(
-                        new JobConfiguration()
-                            .withTriggerType(TriggerType.MANUAL)
-                            .withReplicaTimeout(10)
+                        new JobConfiguration().withTriggerType(TriggerType.MANUAL).withReplicaTimeout(10)
                             .withReplicaRetryLimit(10)
-                            .withManualTriggerConfig(
-                                new JobConfigurationManualTriggerConfig()
-                                    .withReplicaCompletionCount(1)
-                                    .withParallelism(4)))
-                    .withTemplate(
-                        new JobTemplate()
-                            .withInitContainers(
-                                Arrays
-                                    .asList(
-                                        new InitContainer()
-                                            .withImage("repo/testcontainerAppsJob0:v4")
-                                            .withName("testinitcontainerAppsJob0")
-                                            .withCommand(Arrays.asList("/bin/sh"))
-                                            .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
-                                            .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
-                            .withContainers(
-                                Arrays
-                                    .asList(
-                                        new Container()
-                                            .withImage("repo/testcontainerAppsJob0:v1")
-                                            .withName("testcontainerAppsJob0")
-                                            .withProbes(
-                                                Arrays
-                                                    .asList(
-                                                        new ContainerAppProbe()
-                                                            .withHttpGet(
-                                                                new ContainerAppProbeHttpGet()
-                                                                    .withHttpHeaders(
-                                                                        Arrays
-                                                                            .asList(
-                                                                                new ContainerAppProbeHttpGetHttpHeadersItem()
-                                                                                    .withName("Custom-Header")
-                                                                                    .withValue("Awesome")))
-                                                                    .withPath("/health")
-                                                                    .withPort(8080))
-                                                            .withInitialDelaySeconds(3)
-                                                            .withPeriodSeconds(3)
-                                                            .withType(Type.LIVENESS)))))))
+                            .withManualTriggerConfig(new JobConfigurationManualTriggerConfig()
+                                .withReplicaCompletionCount(1).withParallelism(4)))
+                    .withTemplate(new JobTemplate()
+                        .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4")
+                            .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh"))
+                            .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done"))
+                            .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi"))))
+                        .withContainers(Arrays.asList(
+                            new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0")
+                                .withProbes(Arrays.asList(new ContainerAppProbe()
+                                    .withHttpGet(new ContainerAppProbeHttpGet()
+                                        .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem()
+                                            .withName("Custom-Header").withValue("Awesome")))
+                                        .withPath("/health").withPort(8080))
+                                    .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS)))))))
             .apply();
     }
 }
@@ -2645,18 +4030,21 @@ public final class JobsUpdateSamples {
 ### JobsExecutions_List
 
 ```java
-/** Samples for JobsExecutions List. */
+/**
+ * Samples for JobsExecutions List.
+ */
 public final class JobsExecutionsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Executions_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Executions_Get.json
      */
     /**
      * Sample code: Get a Container Apps Job Executions.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getAContainerAppsJobExecutions(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        getAContainerAppsJobExecutions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.jobsExecutions().list("rg", "testcontainerAppsJob0", null, com.azure.core.util.Context.NONE);
     }
 }
@@ -2668,27 +4056,25 @@ public final class JobsExecutionsListSamples {
 import com.azure.resourcemanager.appcontainers.models.ManagedCertificateDomainControlValidation;
 import com.azure.resourcemanager.appcontainers.models.ManagedCertificateProperties;
 
-/** Samples for ManagedCertificates CreateOrUpdate. */
+/**
+ * Samples for ManagedCertificates CreateOrUpdate.
+ */
 public final class ManagedCertificatesCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificate_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedCertificate_CreateOrUpdate.json
      */
     /**
      * Sample code: Create or Update Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateCertificate(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedCertificates()
-            .define("certificate-firendly-name")
-            .withRegion("East US")
+    public static void
+        createOrUpdateCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.managedCertificates().define("certificate-firendly-name").withRegion("East US")
             .withExistingManagedEnvironment("examplerg", "testcontainerenv")
-            .withProperties(
-                new ManagedCertificateProperties()
-                    .withSubjectName("my-subject-name.company.country.net")
-                    .withDomainControlValidation(ManagedCertificateDomainControlValidation.CNAME))
+            .withProperties(new ManagedCertificateProperties().withSubjectName("my-subject-name.company.country.net")
+                .withDomainControlValidation(ManagedCertificateDomainControlValidation.CNAME))
             .create();
     }
 }
@@ -2697,21 +4083,23 @@ public final class ManagedCertificatesCreateOrUpdateSamples {
 ### ManagedCertificates_Delete
 
 ```java
-/** Samples for ManagedCertificates Delete. */
+/**
+ * Samples for ManagedCertificates Delete.
+ */
 public final class ManagedCertificatesDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificate_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedCertificate_Delete.
+     * json
      */
     /**
      * Sample code: Delete Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedCertificates()
-            .deleteWithResponse(
-                "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE);
+        manager.managedCertificates().deleteWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -2719,21 +4107,22 @@ public final class ManagedCertificatesDeleteSamples {
 ### ManagedCertificates_Get
 
 ```java
-/** Samples for ManagedCertificates Get. */
+/**
+ * Samples for ManagedCertificates Get.
+ */
 public final class ManagedCertificatesGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificate_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedCertificate_Get.json
      */
     /**
      * Sample code: Get Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedCertificates()
-            .getWithResponse(
-                "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE);
+        manager.managedCertificates().getWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -2741,14 +4130,17 @@ public final class ManagedCertificatesGetSamples {
 ### ManagedCertificates_List
 
 ```java
-/** Samples for ManagedCertificates List. */
+/**
+ * Samples for ManagedCertificates List.
+ */
 public final class ManagedCertificatesListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificates_ListByManagedEnvironment.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedCertificates_ListByManagedEnvironment.json
      */
     /**
      * Sample code: List Managed Certificates by Managed Environment.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listManagedCertificatesByManagedEnvironment(
@@ -2765,24 +4157,24 @@ import com.azure.resourcemanager.appcontainers.models.ManagedCertificate;
 import java.util.HashMap;
 import java.util.Map;
 
-/** Samples for ManagedCertificates Update. */
+/**
+ * Samples for ManagedCertificates Update.
+ */
 public final class ManagedCertificatesUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificates_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedCertificates_Patch.
+     * json
      */
     /**
      * Sample code: Patch Managed Certificate.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void patchManagedCertificate(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        ManagedCertificate resource =
-            manager
-                .managedCertificates()
-                .getWithResponse(
-                    "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE)
-                .getValue();
+    public static void
+        patchManagedCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        ManagedCertificate resource = manager.managedCertificates().getWithResponse("examplerg", "testcontainerenv",
+            "certificate-firendly-name", com.azure.core.util.Context.NONE).getValue();
         resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply();
     }
 
@@ -2803,25 +4195,23 @@ public final class ManagedCertificatesUpdateSamples {
 ### ManagedEnvironmentDiagnostics_GetDetector
 
 ```java
-/** Samples for ManagedEnvironmentDiagnostics GetDetector. */
+/**
+ * Samples for ManagedEnvironmentDiagnostics GetDetector.
+ */
 public final class ManagedEnvironmentDiagnosticsGetDetectorSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentDiagnostics_Get.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentDiagnostics_Get.json
      */
     /**
      * Sample code: Get diagnostic data for a managed environments.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getDiagnosticDataForAManagedEnvironments(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentDiagnostics()
-            .getDetectorWithResponse(
-                "mikono-workerapp-test-rg",
-                "mikonokubeenv",
-                "ManagedEnvAvailabilityMetrics",
-                com.azure.core.util.Context.NONE);
+        manager.managedEnvironmentDiagnostics().getDetectorWithResponse("mikono-workerapp-test-rg", "mikonokubeenv",
+            "ManagedEnvAvailabilityMetrics", com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -2829,21 +4219,46 @@ public final class ManagedEnvironmentDiagnosticsGetDetectorSamples {
 ### ManagedEnvironmentDiagnostics_ListDetectors
 
 ```java
-/** Samples for ManagedEnvironmentDiagnostics ListDetectors. */
+/**
+ * Samples for ManagedEnvironmentDiagnostics ListDetectors.
+ */
 public final class ManagedEnvironmentDiagnosticsListDetectorsSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentDiagnostics_List.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentDiagnostics_List.json
      */
     /**
      * Sample code: Get the list of available diagnostic data for a managed environments.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getTheListOfAvailableDiagnosticDataForAManagedEnvironments(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentDiagnostics()
-            .listDetectorsWithResponse("mikono-workerapp-test-rg", "mikonokubeenv", com.azure.core.util.Context.NONE);
+        manager.managedEnvironmentDiagnostics().listDetectorsWithResponse("mikono-workerapp-test-rg", "mikonokubeenv",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### ManagedEnvironmentUsages_List
+
+```java
+/**
+ * Samples for ManagedEnvironmentUsages List.
+ */
+public final class ManagedEnvironmentUsagesListSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentUsages_List.json
+     */
+    /**
+     * Sample code: List managed environment usages.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        listManagedEnvironmentUsages(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.managedEnvironmentUsages().list("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -2851,142 +4266,154 @@ public final class ManagedEnvironmentDiagnosticsListDetectorsSamples {
 ### ManagedEnvironments_CreateOrUpdate
 
 ```java
+import com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration;
 import com.azure.resourcemanager.appcontainers.models.AppLogsConfiguration;
 import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration;
+import com.azure.resourcemanager.appcontainers.models.DataDogConfiguration;
+import com.azure.resourcemanager.appcontainers.models.DestinationsConfiguration;
+import com.azure.resourcemanager.appcontainers.models.Header;
 import com.azure.resourcemanager.appcontainers.models.LogAnalyticsConfiguration;
+import com.azure.resourcemanager.appcontainers.models.LogsConfiguration;
 import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentPropertiesPeerAuthentication;
+import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity;
+import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType;
+import com.azure.resourcemanager.appcontainers.models.MetricsConfiguration;
 import com.azure.resourcemanager.appcontainers.models.Mtls;
+import com.azure.resourcemanager.appcontainers.models.OpenTelemetryConfiguration;
+import com.azure.resourcemanager.appcontainers.models.OtlpConfiguration;
+import com.azure.resourcemanager.appcontainers.models.TracesConfiguration;
+import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity;
 import com.azure.resourcemanager.appcontainers.models.VnetConfiguration;
 import com.azure.resourcemanager.appcontainers.models.WorkloadProfile;
 import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 
-/** Samples for ManagedEnvironments CreateOrUpdate. */
+/**
+ * Samples for ManagedEnvironments CreateOrUpdate.
+ */
 public final class ManagedEnvironmentsCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_CustomInfrastructureResourceGroup_Create.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironments_CustomInfrastructureResourceGroup_Create.json
      */
     /**
      * Sample code: Create environment with custom infrastructureResourceGroup.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void createEnvironmentWithCustomInfrastructureResourceGroup(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironments()
-            .define("testcontainerenv")
-            .withRegion("East US")
+        manager.managedEnvironments().define("testcontainerenv").withRegion("East US")
             .withExistingResourceGroup("examplerg")
             .withDaprAIConnectionString(
                 "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/")
-            .withVnetConfiguration(
-                new VnetConfiguration()
-                    .withInfrastructureSubnetId(
-                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1"))
-            .withAppLogsConfiguration(
-                new AppLogsConfiguration()
-                    .withLogAnalyticsConfiguration(
-                        new LogAnalyticsConfiguration().withCustomerId("string").withSharedKey("fakeTokenPlaceholder")))
+            .withVnetConfiguration(new VnetConfiguration().withInfrastructureSubnetId(
+                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1"))
+            .withAppLogsConfiguration(new AppLogsConfiguration().withLogAnalyticsConfiguration(
+                new LogAnalyticsConfiguration().withCustomerId("string").withSharedKey("fakeTokenPlaceholder")))
             .withZoneRedundant(true)
-            .withCustomDomainConfiguration(
-                new CustomDomainConfiguration()
-                    .withDnsSuffix("www.my-name.com")
-                    .withCertificateValue("Y2VydA==".getBytes())
-                    .withCertificatePassword("fakeTokenPlaceholder"))
-            .withWorkloadProfiles(
-                Arrays
-                    .asList(
-                        new WorkloadProfile()
-                            .withName("My-GP-01")
-                            .withWorkloadProfileType("GeneralPurpose")
-                            .withMinimumCount(3)
-                            .withMaximumCount(12),
-                        new WorkloadProfile()
-                            .withName("My-MO-01")
-                            .withWorkloadProfileType("MemoryOptimized")
-                            .withMinimumCount(3)
-                            .withMaximumCount(6),
-                        new WorkloadProfile()
-                            .withName("My-CO-01")
-                            .withWorkloadProfileType("ComputeOptimized")
-                            .withMinimumCount(3)
-                            .withMaximumCount(6),
-                        new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption")))
-            .withInfrastructureResourceGroup("myInfrastructureRgName")
-            .create();
+            .withCustomDomainConfiguration(new CustomDomainConfiguration().withDnsSuffix("www.my-name.com")
+                .withCertificateValue("Y2VydA==".getBytes()).withCertificatePassword("fakeTokenPlaceholder"))
+            .withWorkloadProfiles(Arrays.asList(
+                new WorkloadProfile().withName("My-GP-01").withWorkloadProfileType("GeneralPurpose").withMinimumCount(3)
+                    .withMaximumCount(12),
+                new WorkloadProfile().withName("My-MO-01").withWorkloadProfileType("MemoryOptimized")
+                    .withMinimumCount(3).withMaximumCount(6),
+                new WorkloadProfile().withName("My-CO-01").withWorkloadProfileType("ComputeOptimized")
+                    .withMinimumCount(3).withMaximumCount(6),
+                new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption")))
+            .withInfrastructureResourceGroup("myInfrastructureRgName").create();
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironments_CreateOrUpdate.json
      */
     /**
      * Sample code: Create environments.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void createEnvironments(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironments()
-            .define("testcontainerenv")
-            .withRegion("East US")
+        manager.managedEnvironments().define("testcontainerenv").withRegion("East US")
             .withExistingResourceGroup("examplerg")
+            .withIdentity(new ManagedServiceIdentity()
+                .withType(ManagedServiceIdentityType.fromString("SystemAssigned, UserAssigned"))
+                .withUserAssignedIdentities(mapOf(
+                    "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/contoso-resources/providers/Microsoft.ManagedIdentity/userAssignedIdentities/contoso-identity",
+                    new UserAssignedIdentity())))
             .withDaprAIConnectionString(
                 "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/")
-            .withVnetConfiguration(
-                new VnetConfiguration()
-                    .withInfrastructureSubnetId(
-                        "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1"))
+            .withVnetConfiguration(new VnetConfiguration().withInfrastructureSubnetId(
+                "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1"))
             .withAppLogsConfiguration(
-                new AppLogsConfiguration()
-                    .withLogAnalyticsConfiguration(
-                        new LogAnalyticsConfiguration().withCustomerId("string").withSharedKey("fakeTokenPlaceholder")))
+                new AppLogsConfiguration().withLogAnalyticsConfiguration(new LogAnalyticsConfiguration()
+                    .withCustomerId("string").withSharedKey("fakeTokenPlaceholder").withDynamicJsonColumns(true)))
+            .withAppInsightsConfiguration(new AppInsightsConfiguration().withConnectionString(
+                "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://eastus-8.in.applicationinsights.azure.com/;LiveEndpoint=https://eastus.livediagnostics.monitor.azure.com/"))
+            .withOpenTelemetryConfiguration(
+                new OpenTelemetryConfiguration()
+                    .withDestinationsConfiguration(
+                        new DestinationsConfiguration()
+                            .withDataDogConfiguration(
+                                new DataDogConfiguration().withSite("string").withKey("fakeTokenPlaceholder"))
+                            .withOtlpConfigurations(Arrays.asList(new OtlpConfiguration().withName("dashboard")
+                                .withEndpoint("dashboard.k8s.region.azurecontainerapps.io:80").withInsecure(true)
+                                .withHeaders(Arrays
+                                    .asList(new Header().withKey("fakeTokenPlaceholder").withValue("xxxxxxxxxxx"))))))
+                    .withTracesConfiguration(new TracesConfiguration().withDestinations(Arrays.asList("appInsights")))
+                    .withLogsConfiguration(new LogsConfiguration().withDestinations(Arrays.asList("appInsights")))
+                    .withMetricsConfiguration(new MetricsConfiguration().withDestinations(Arrays.asList("dataDog"))))
             .withZoneRedundant(true)
-            .withCustomDomainConfiguration(
-                new CustomDomainConfiguration()
-                    .withDnsSuffix("www.my-name.com")
-                    .withCertificateValue("Y2VydA==".getBytes())
-                    .withCertificatePassword("fakeTokenPlaceholder"))
-            .withWorkloadProfiles(
-                Arrays
-                    .asList(
-                        new WorkloadProfile()
-                            .withName("My-GP-01")
-                            .withWorkloadProfileType("GeneralPurpose")
-                            .withMinimumCount(3)
-                            .withMaximumCount(12),
-                        new WorkloadProfile()
-                            .withName("My-MO-01")
-                            .withWorkloadProfileType("MemoryOptimized")
-                            .withMinimumCount(3)
-                            .withMaximumCount(6),
-                        new WorkloadProfile()
-                            .withName("My-CO-01")
-                            .withWorkloadProfileType("ComputeOptimized")
-                            .withMinimumCount(3)
-                            .withMaximumCount(6),
-                        new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption")))
+            .withCustomDomainConfiguration(new CustomDomainConfiguration().withDnsSuffix("www.my-name.com")
+                .withCertificateValue("Y2VydA==".getBytes()).withCertificatePassword("fakeTokenPlaceholder"))
+            .withWorkloadProfiles(Arrays.asList(
+                new WorkloadProfile().withName("My-GP-01").withWorkloadProfileType("GeneralPurpose").withMinimumCount(3)
+                    .withMaximumCount(12),
+                new WorkloadProfile().withName("My-MO-01").withWorkloadProfileType("MemoryOptimized")
+                    .withMinimumCount(3).withMaximumCount(6),
+                new WorkloadProfile().withName("My-CO-01").withWorkloadProfileType("ComputeOptimized")
+                    .withMinimumCount(3).withMaximumCount(6),
+                new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption")))
             .withPeerAuthentication(
                 new ManagedEnvironmentPropertiesPeerAuthentication().withMtls(new Mtls().withEnabled(true)))
             .create();
     }
+
+    // Use "Map.of" if available
+    @SuppressWarnings("unchecked")
+    private static  Map mapOf(Object... inputs) {
+        Map map = new HashMap<>();
+        for (int i = 0; i < inputs.length; i += 2) {
+            String key = (String) inputs[i];
+            T value = (T) inputs[i + 1];
+            map.put(key, value);
+        }
+        return map;
+    }
 }
 ```
 
 ### ManagedEnvironments_Delete
 
 ```java
-/** Samples for ManagedEnvironments Delete. */
+/**
+ * Samples for ManagedEnvironments Delete.
+ */
 public final class ManagedEnvironmentsDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Delete.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Delete.
+     * json
      */
     /**
      * Sample code: Delete environment by name.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void deleteEnvironmentByName(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        deleteEnvironmentByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.managedEnvironments().delete("examplerg", "examplekenv", com.azure.core.util.Context.NONE);
     }
 }
@@ -2995,18 +4422,21 @@ public final class ManagedEnvironmentsDeleteSamples {
 ### ManagedEnvironments_GetAuthToken
 
 ```java
-/** Samples for ManagedEnvironments GetAuthToken. */
+/**
+ * Samples for ManagedEnvironments GetAuthToken.
+ */
 public final class ManagedEnvironmentsGetAuthTokenSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_GetAuthToken.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironments_GetAuthToken.json
      */
     /**
      * Sample code: Get Managed Environment Auth Token.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getManagedEnvironmentAuthToken(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        getManagedEnvironmentAuthToken(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.managedEnvironments().getAuthTokenWithResponse("rg", "testenv", com.azure.core.util.Context.NONE);
     }
 }
@@ -3015,20 +4445,22 @@ public final class ManagedEnvironmentsGetAuthTokenSamples {
 ### ManagedEnvironments_GetByResourceGroup
 
 ```java
-/** Samples for ManagedEnvironments GetByResourceGroup. */
+/**
+ * Samples for ManagedEnvironments GetByResourceGroup.
+ */
 public final class ManagedEnvironmentsGetByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Get.json
      */
     /**
      * Sample code: Get environments by name.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getEnvironmentsByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironments()
-            .getByResourceGroupWithResponse("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE);
+        manager.managedEnvironments().getByResourceGroupWithResponse("examplerg", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3036,18 +4468,21 @@ public final class ManagedEnvironmentsGetByResourceGroupSamples {
 ### ManagedEnvironments_List
 
 ```java
-/** Samples for ManagedEnvironments List. */
+/**
+ * Samples for ManagedEnvironments List.
+ */
 public final class ManagedEnvironmentsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_ListBySubscription.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironments_ListBySubscription.json
      */
     /**
      * Sample code: List environments by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listEnvironmentsBySubscription(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listEnvironmentsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.managedEnvironments().list(com.azure.core.util.Context.NONE);
     }
 }
@@ -3056,18 +4491,21 @@ public final class ManagedEnvironmentsListSamples {
 ### ManagedEnvironments_ListByResourceGroup
 
 ```java
-/** Samples for ManagedEnvironments ListByResourceGroup. */
+/**
+ * Samples for ManagedEnvironments ListByResourceGroup.
+ */
 public final class ManagedEnvironmentsListByResourceGroupSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_ListByResourceGroup.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironments_ListByResourceGroup.json
      */
     /**
      * Sample code: List environments by resource group.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listEnvironmentsByResourceGroup(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+    public static void
+        listEnvironmentsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
         manager.managedEnvironments().listByResourceGroup("examplerg", com.azure.core.util.Context.NONE);
     }
 }
@@ -3076,21 +4514,23 @@ public final class ManagedEnvironmentsListByResourceGroupSamples {
 ### ManagedEnvironments_ListWorkloadProfileStates
 
 ```java
-/** Samples for ManagedEnvironments ListWorkloadProfileStates. */
+/**
+ * Samples for ManagedEnvironments ListWorkloadProfileStates.
+ */
 public final class ManagedEnvironmentsListWorkloadProfileStatesSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_ListWorkloadProfileStates.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironments_ListWorkloadProfileStates.json
      */
     /**
      * Sample code: List environments by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void listEnvironmentsBySubscription(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironments()
-            .listWorkloadProfileStates("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE);
+    public static void
+        listEnvironmentsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.managedEnvironments().listWorkloadProfileStates("examplerg", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3102,23 +4542,25 @@ import com.azure.resourcemanager.appcontainers.models.ManagedEnvironment;
 import java.util.HashMap;
 import java.util.Map;
 
-/** Samples for ManagedEnvironments Update. */
+/**
+ * Samples for ManagedEnvironments Update.
+ */
 public final class ManagedEnvironmentsUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Patch.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Patch.
+     * json
      */
     /**
      * Sample code: Patch Managed Environment.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void patchManagedEnvironment(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        ManagedEnvironment resource =
-            manager
-                .managedEnvironments()
-                .getByResourceGroupWithResponse("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE)
-                .getValue();
+    public static void
+        patchManagedEnvironment(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        ManagedEnvironment resource = manager.managedEnvironments()
+            .getByResourceGroupWithResponse("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE)
+            .getValue();
         resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply();
     }
 
@@ -3139,20 +4581,22 @@ public final class ManagedEnvironmentsUpdateSamples {
 ### ManagedEnvironmentsDiagnostics_GetRoot
 
 ```java
-/** Samples for ManagedEnvironmentsDiagnostics GetRoot. */
+/**
+ * Samples for ManagedEnvironmentsDiagnostics GetRoot.
+ */
 public final class ManagedEnvironmentsDiagnosticsGetRootSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Get.json
      */
     /**
      * Sample code: Get environments by name.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getEnvironmentsByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentsDiagnostics()
-            .getRootWithResponse("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE);
+        manager.managedEnvironmentsDiagnostics().getRootWithResponse("examplerg", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3163,31 +4607,46 @@ public final class ManagedEnvironmentsDiagnosticsGetRootSamples {
 import com.azure.resourcemanager.appcontainers.models.AccessMode;
 import com.azure.resourcemanager.appcontainers.models.AzureFileProperties;
 import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentStorageProperties;
+import com.azure.resourcemanager.appcontainers.models.NfsAzureFileProperties;
 
-/** Samples for ManagedEnvironmentsStorages CreateOrUpdate. */
+/**
+ * Samples for ManagedEnvironmentsStorages CreateOrUpdate.
+ */
 public final class ManagedEnvironmentsStoragesCreateOrUpdateSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_CreateOrUpdate.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentsStorages_CreateOrUpdate_NfsAzureFile.json
      */
     /**
-     * Sample code: Create or update environments storage.
-     *
+     * Sample code: Create or update environments storage for NFS Azure file.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void createOrUpdateEnvironmentsStorage(
+    public static void createOrUpdateEnvironmentsStorageForNFSAzureFile(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentsStorages()
-            .define("jlaw-demo1")
+        manager.managedEnvironmentsStorages().define("jlaw-demo1")
             .withExistingManagedEnvironment("examplerg", "managedEnv")
-            .withProperties(
-                new ManagedEnvironmentStorageProperties()
-                    .withAzureFile(
-                        new AzureFileProperties()
-                            .withAccountName("account1")
-                            .withAccountKey("fakeTokenPlaceholder")
-                            .withAccessMode(AccessMode.READ_ONLY)
-                            .withShareName("share1")))
+            .withProperties(new ManagedEnvironmentStorageProperties().withNfsAzureFile(new NfsAzureFileProperties()
+                .withServer("server1").withAccessMode(AccessMode.READ_ONLY).withShareName("share1")))
+            .create();
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentsStorages_CreateOrUpdate.json
+     */
+    /**
+     * Sample code: Create or update environments storage.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void
+        createOrUpdateEnvironmentsStorage(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.managedEnvironmentsStorages().define("jlaw-demo1")
+            .withExistingManagedEnvironment("examplerg", "managedEnv")
+            .withProperties(new ManagedEnvironmentStorageProperties().withAzureFile(
+                new AzureFileProperties().withAccountName("account1").withAccountKey("fakeTokenPlaceholder")
+                    .withAccessMode(AccessMode.READ_ONLY).withShareName("share1")))
             .create();
     }
 }
@@ -3196,21 +4655,23 @@ public final class ManagedEnvironmentsStoragesCreateOrUpdateSamples {
 ### ManagedEnvironmentsStorages_Delete
 
 ```java
-/** Samples for ManagedEnvironmentsStorages Delete. */
+/**
+ * Samples for ManagedEnvironmentsStorages Delete.
+ */
 public final class ManagedEnvironmentsStoragesDeleteSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_Delete.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentsStorages_Delete.json
      */
     /**
      * Sample code: List environments storages by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listEnvironmentsStoragesBySubscription(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentsStorages()
-            .deleteWithResponse("examplerg", "managedEnv", "jlaw-demo1", com.azure.core.util.Context.NONE);
+        manager.managedEnvironmentsStorages().deleteWithResponse("examplerg", "managedEnv", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3218,21 +4679,38 @@ public final class ManagedEnvironmentsStoragesDeleteSamples {
 ### ManagedEnvironmentsStorages_Get
 
 ```java
-/** Samples for ManagedEnvironmentsStorages Get. */
+/**
+ * Samples for ManagedEnvironmentsStorages Get.
+ */
 public final class ManagedEnvironmentsStoragesGetSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_Get.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentsStorages_Get.json
      */
     /**
-     * Sample code: get a environments storage properties by subscription.
-     *
+     * Sample code: get a environments storage.
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void getAEnvironmentsStoragePropertiesBySubscription(
+    public static void
+        getAEnvironmentsStorage(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.managedEnvironmentsStorages().getWithResponse("examplerg", "managedEnv", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
+    }
+
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentsStorages_Get_NfsAzureFile.json
+     */
+    /**
+     * Sample code: get a environments storage for NFS Azure file.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void getAEnvironmentsStorageForNFSAzureFile(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentsStorages()
-            .getWithResponse("examplerg", "managedEnv", "jlaw-demo1", com.azure.core.util.Context.NONE);
+        manager.managedEnvironmentsStorages().getWithResponse("examplerg", "managedEnv", "jlaw-demo1",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3240,21 +4718,23 @@ public final class ManagedEnvironmentsStoragesGetSamples {
 ### ManagedEnvironmentsStorages_List
 
 ```java
-/** Samples for ManagedEnvironmentsStorages List. */
+/**
+ * Samples for ManagedEnvironmentsStorages List.
+ */
 public final class ManagedEnvironmentsStoragesListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_List.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ManagedEnvironmentsStorages_List.json
      */
     /**
      * Sample code: List environments storages by subscription.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listEnvironmentsStoragesBySubscription(
         com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .managedEnvironmentsStorages()
-            .listWithResponse("examplerg", "managedEnv", com.azure.core.util.Context.NONE);
+        manager.managedEnvironmentsStorages().listWithResponse("examplerg", "managedEnv",
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3264,46 +4744,41 @@ public final class ManagedEnvironmentsStoragesListSamples {
 ```java
 import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest;
 
-/** Samples for Namespaces CheckNameAvailability. */
+/**
+ * Samples for Namespaces CheckNameAvailability.
+ */
 public final class NamespacesCheckNameAvailabilitySamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificates_CheckNameAvailability.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Certificates_CheckNameAvailability.json
      */
     /**
      * Sample code: Certificates_CheckNameAvailability.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void certificatesCheckNameAvailability(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .namespaces()
-            .checkNameAvailabilityWithResponse(
-                "examplerg",
-                "testcontainerenv",
-                new CheckNameAvailabilityRequest()
-                    .withName("testcertificatename")
-                    .withType("Microsoft.App/managedEnvironments/certificates"),
+    public static void
+        certificatesCheckNameAvailability(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.namespaces()
+            .checkNameAvailabilityWithResponse("examplerg", "testcontainerenv", new CheckNameAvailabilityRequest()
+                .withName("testcertificatename").withType("Microsoft.App/managedEnvironments/certificates"),
                 com.azure.core.util.Context.NONE);
     }
 
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_CheckNameAvailability.json
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * ContainerApps_CheckNameAvailability.json
      */
     /**
      * Sample code: ContainerApps_CheckNameAvailability.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
-    public static void containerAppsCheckNameAvailability(
-        com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .namespaces()
-            .checkNameAvailabilityWithResponse(
-                "examplerg",
-                "testcontainerenv",
-                new CheckNameAvailabilityRequest().withName("testcappname").withType("Microsoft.App/containerApps"),
-                com.azure.core.util.Context.NONE);
+    public static void
+        containerAppsCheckNameAvailability(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.namespaces().checkNameAvailabilityWithResponse("examplerg", "testcontainerenv",
+            new CheckNameAvailabilityRequest().withName("testcappname").withType("Microsoft.App/containerApps"),
+            com.azure.core.util.Context.NONE);
     }
 }
 ```
@@ -3311,14 +4786,17 @@ public final class NamespacesCheckNameAvailabilitySamples {
 ### Operations_List
 
 ```java
-/** Samples for Operations List. */
+/**
+ * Samples for Operations List.
+ */
 public final class OperationsListSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Operations_List.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Operations_List.json
      */
     /**
      * Sample code: List all operations.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void listAllOperations(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
@@ -3327,23 +4805,69 @@ public final class OperationsListSamples {
 }
 ```
 
+### ResourceProvider_GetCustomDomainVerificationId
+
+```java
+/**
+ * Samples for ResourceProvider GetCustomDomainVerificationId.
+ */
+public final class ResourceProviderGetCustomDomainVerificationIdSamples {
+    /*
+     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/
+     * Subscriptions_GetCustomDomainVerificationId.json
+     */
+    /**
+     * Sample code: List all operations.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listAllOperations(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.resourceProviders().getCustomDomainVerificationIdWithResponse(com.azure.core.util.Context.NONE);
+    }
+}
+```
+
 ### ResourceProvider_JobExecution
 
 ```java
-/** Samples for ResourceProvider JobExecution. */
+/**
+ * Samples for ResourceProvider JobExecution.
+ */
 public final class ResourceProviderJobExecutionSamples {
     /*
-     * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Execution_Get.json
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Execution_Get.json
      */
     /**
      * Sample code: Get a single Job Execution.
-     *
+     * 
      * @param manager Entry point to ContainerAppsApiManager.
      */
     public static void getASingleJobExecution(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
-        manager
-            .resourceProviders()
-            .jobExecutionWithResponse("rg", "testcontainerAppsJob0", "jobExecution1", com.azure.core.util.Context.NONE);
+        manager.resourceProviders().jobExecutionWithResponse("rg", "testcontainerAppsJob0", "jobExecution1",
+            com.azure.core.util.Context.NONE);
+    }
+}
+```
+
+### Usages_List
+
+```java
+/**
+ * Samples for Usages List.
+ */
+public final class UsagesListSamples {
+    /*
+     * x-ms-original-file:
+     * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Usages_List.json
+     */
+    /**
+     * Sample code: List usages.
+     * 
+     * @param manager Entry point to ContainerAppsApiManager.
+     */
+    public static void listUsages(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) {
+        manager.usages().list("westus", com.azure.core.util.Context.NONE);
     }
 }
 ```
diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/pom.xml b/sdk/appcontainers/azure-resourcemanager-appcontainers/pom.xml
index b48996d53a6a3..24bd120872a92 100644
--- a/sdk/appcontainers/azure-resourcemanager-appcontainers/pom.xml
+++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/pom.xml
@@ -14,11 +14,11 @@
 
   com.azure.resourcemanager
   azure-resourcemanager-appcontainers
-  1.0.0-beta.7 
+  1.0.0-beta.8 
   jar
 
   Microsoft Azure SDK for ContainerAppsApi Management
-  This package contains Microsoft Azure SDK for ContainerAppsApi Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt.  Package tag package-2023-05.
+  This package contains Microsoft Azure SDK for ContainerAppsApi Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt.  Package tag package-preview-2023-11.
   https://github.com/Azure/azure-sdk-for-java
 
   
@@ -67,7 +67,7 @@
     
       com.azure
       azure-identity
-      1.11.3 
+      1.11.4 
       test
     
     
@@ -88,18 +88,16 @@
       4.11.0 
       test
     
-    
-    
     
       net.bytebuddy
       byte-buddy
-      1.14.8 
+      1.14.12 
       test
     
     
       net.bytebuddy
       byte-buddy-agent
-      1.14.8 
+      1.14.12 
       test
     
     
diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/ContainerAppsApiManager.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/ContainerAppsApiManager.java
index a9b1dd45c81e4..087cc2b9ecaed 100644
--- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/ContainerAppsApiManager.java
+++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/ContainerAppsApiManager.java
@@ -24,8 +24,13 @@
 import com.azure.core.util.Configuration;
 import com.azure.core.util.logging.ClientLogger;
 import com.azure.resourcemanager.appcontainers.fluent.ContainerAppsApiClient;
+import com.azure.resourcemanager.appcontainers.implementation.AppResilienciesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.AvailableWorkloadProfilesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.BillingMetersImpl;
+import com.azure.resourcemanager.appcontainers.implementation.BuildAuthTokensImpl;
+import com.azure.resourcemanager.appcontainers.implementation.BuildersImpl;
+import com.azure.resourcemanager.appcontainers.implementation.BuildsByBuilderResourcesImpl;
+import com.azure.resourcemanager.appcontainers.implementation.BuildsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.CertificatesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ConnectedEnvironmentsCertificatesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ConnectedEnvironmentsDaprComponentsImpl;
@@ -38,19 +43,30 @@
 import com.azure.resourcemanager.appcontainers.implementation.ContainerAppsRevisionReplicasImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ContainerAppsRevisionsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ContainerAppsSourceControlsImpl;
+import com.azure.resourcemanager.appcontainers.implementation.DaprComponentResiliencyPoliciesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.DaprComponentsImpl;
+import com.azure.resourcemanager.appcontainers.implementation.DaprSubscriptionsImpl;
+import com.azure.resourcemanager.appcontainers.implementation.DotNetComponentsImpl;
+import com.azure.resourcemanager.appcontainers.implementation.JavaComponentsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.JobsExecutionsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.JobsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ManagedCertificatesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentDiagnosticsImpl;
+import com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentUsagesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentsDiagnosticsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentsStoragesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.NamespacesImpl;
 import com.azure.resourcemanager.appcontainers.implementation.OperationsImpl;
 import com.azure.resourcemanager.appcontainers.implementation.ResourceProvidersImpl;
+import com.azure.resourcemanager.appcontainers.implementation.UsagesImpl;
+import com.azure.resourcemanager.appcontainers.models.AppResiliencies;
 import com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfiles;
 import com.azure.resourcemanager.appcontainers.models.BillingMeters;
+import com.azure.resourcemanager.appcontainers.models.BuildAuthTokens;
+import com.azure.resourcemanager.appcontainers.models.Builders;
+import com.azure.resourcemanager.appcontainers.models.Builds;
+import com.azure.resourcemanager.appcontainers.models.BuildsByBuilderResources;
 import com.azure.resourcemanager.appcontainers.models.Certificates;
 import com.azure.resourcemanager.appcontainers.models.ConnectedEnvironments;
 import com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentsCertificates;
@@ -62,17 +78,23 @@
 import com.azure.resourcemanager.appcontainers.models.ContainerAppsRevisionReplicas;
 import com.azure.resourcemanager.appcontainers.models.ContainerAppsRevisions;
 import com.azure.resourcemanager.appcontainers.models.ContainerAppsSourceControls;
+import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicies;
 import com.azure.resourcemanager.appcontainers.models.DaprComponents;
+import com.azure.resourcemanager.appcontainers.models.DaprSubscriptions;
+import com.azure.resourcemanager.appcontainers.models.DotNetComponents;
+import com.azure.resourcemanager.appcontainers.models.JavaComponents;
 import com.azure.resourcemanager.appcontainers.models.Jobs;
 import com.azure.resourcemanager.appcontainers.models.JobsExecutions;
 import com.azure.resourcemanager.appcontainers.models.ManagedCertificates;
 import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentDiagnostics;
+import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentUsages;
 import com.azure.resourcemanager.appcontainers.models.ManagedEnvironments;
 import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentsDiagnostics;
 import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentsStorages;
 import com.azure.resourcemanager.appcontainers.models.Namespaces;
 import com.azure.resourcemanager.appcontainers.models.Operations;
 import com.azure.resourcemanager.appcontainers.models.ResourceProviders;
+import com.azure.resourcemanager.appcontainers.models.Usages;
 import java.time.Duration;
 import java.time.temporal.ChronoUnit;
 import java.util.ArrayList;
@@ -80,14 +102,26 @@
 import java.util.Objects;
 import java.util.stream.Collectors;
 
-/** Entry point to ContainerAppsApiManager. */
+/**
+ * Entry point to ContainerAppsApiManager.
+ */
 public final class ContainerAppsApiManager {
+    private AppResiliencies appResiliencies;
+
     private ContainerAppsAuthConfigs containerAppsAuthConfigs;
 
     private AvailableWorkloadProfiles availableWorkloadProfiles;
 
     private BillingMeters billingMeters;
 
+    private Builders builders;
+
+    private BuildsByBuilderResources buildsByBuilderResources;
+
+    private Builds builds;
+
+    private BuildAuthTokens buildAuthTokens;
+
     private ConnectedEnvironments connectedEnvironments;
 
     private ConnectedEnvironmentsCertificates connectedEnvironmentsCertificates;
@@ -108,10 +142,10 @@ public final class ContainerAppsApiManager {
 
     private ManagedEnvironmentsDiagnostics managedEnvironmentsDiagnostics;
 
-    private Operations operations;
-
     private Jobs jobs;
 
+    private Operations operations;
+
     private JobsExecutions jobsExecutions;
 
     private ResourceProviders resourceProviders;
@@ -124,29 +158,37 @@ public final class ContainerAppsApiManager {
 
     private Namespaces namespaces;
 
+    private DaprComponentResiliencyPolicies daprComponentResiliencyPolicies;
+
     private DaprComponents daprComponents;
 
+    private DaprSubscriptions daprSubscriptions;
+
     private ManagedEnvironmentsStorages managedEnvironmentsStorages;
 
     private ContainerAppsSourceControls containerAppsSourceControls;
 
+    private Usages usages;
+
+    private ManagedEnvironmentUsages managedEnvironmentUsages;
+
+    private JavaComponents javaComponents;
+
+    private DotNetComponents dotNetComponents;
+
     private final ContainerAppsApiClient clientObject;
 
     private ContainerAppsApiManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) {
         Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null.");
         Objects.requireNonNull(profile, "'profile' cannot be null.");
-        this.clientObject =
-            new ContainerAppsApiClientBuilder()
-                .pipeline(httpPipeline)
-                .endpoint(profile.getEnvironment().getResourceManagerEndpoint())
-                .subscriptionId(profile.getSubscriptionId())
-                .defaultPollInterval(defaultPollInterval)
-                .buildClient();
+        this.clientObject = new ContainerAppsApiClientBuilder().pipeline(httpPipeline)
+            .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId())
+            .defaultPollInterval(defaultPollInterval).buildClient();
     }
 
     /**
      * Creates an instance of ContainerAppsApi service API entry point.
-     *
+     * 
      * @param credential the credential to use.
      * @param profile the Azure profile for client.
      * @return the ContainerAppsApi service API instance.
@@ -159,7 +201,7 @@ public static ContainerAppsApiManager authenticate(TokenCredential credential, A
 
     /**
      * Creates an instance of ContainerAppsApi service API entry point.
-     *
+     * 
      * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential.
      * @param profile the Azure profile for client.
      * @return the ContainerAppsApi service API instance.
@@ -172,14 +214,16 @@ public static ContainerAppsApiManager authenticate(HttpPipeline httpPipeline, Az
 
     /**
      * Gets a Configurable instance that can be used to create ContainerAppsApiManager with optional configuration.
-     *
+     * 
      * @return the Configurable instance allowing configurations.
      */
     public static Configurable configure() {
         return new ContainerAppsApiManager.Configurable();
     }
 
-    /** The Configurable allowing configurations to be set. */
+    /**
+     * The Configurable allowing configurations to be set.
+     */
     public static final class Configurable {
         private static final ClientLogger LOGGER = new ClientLogger(Configurable.class);
 
@@ -251,8 +295,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) {
 
         /**
          * Sets the retry options for the HTTP pipeline retry policy.
-         *
-         * 

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -269,8 +313,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -290,21 +334,12 @@ public ContainerAppsApiManager authenticate(TokenCredential credential, AzurePro Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.appcontainers") - .append("/") - .append("1.0.0-beta.6"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.appcontainers") + .append("/").append("1.0.0-beta.7"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -323,64 +358,63 @@ public ContainerAppsApiManager authenticate(TokenCredential credential, AzurePro policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new ContainerAppsApiManager(httpPipeline, profile, defaultPollInterval); } } + /** + * Gets the resource collection API of AppResiliencies. It manages AppResiliency. + * + * @return Resource collection API of AppResiliencies. + */ + public AppResiliencies appResiliencies() { + if (this.appResiliencies == null) { + this.appResiliencies = new AppResilienciesImpl(clientObject.getAppResiliencies(), this); + } + return appResiliencies; + } + /** * Gets the resource collection API of ContainerAppsAuthConfigs. It manages AuthConfig. - * + * * @return Resource collection API of ContainerAppsAuthConfigs. */ public ContainerAppsAuthConfigs containerAppsAuthConfigs() { if (this.containerAppsAuthConfigs == null) { - this.containerAppsAuthConfigs = - new ContainerAppsAuthConfigsImpl(clientObject.getContainerAppsAuthConfigs(), this); + this.containerAppsAuthConfigs + = new ContainerAppsAuthConfigsImpl(clientObject.getContainerAppsAuthConfigs(), this); } return containerAppsAuthConfigs; } /** * Gets the resource collection API of AvailableWorkloadProfiles. - * + * * @return Resource collection API of AvailableWorkloadProfiles. */ public AvailableWorkloadProfiles availableWorkloadProfiles() { if (this.availableWorkloadProfiles == null) { - this.availableWorkloadProfiles = - new AvailableWorkloadProfilesImpl(clientObject.getAvailableWorkloadProfiles(), this); + this.availableWorkloadProfiles + = new AvailableWorkloadProfilesImpl(clientObject.getAvailableWorkloadProfiles(), this); } return availableWorkloadProfiles; } /** * Gets the resource collection API of BillingMeters. - * + * * @return Resource collection API of BillingMeters. */ public BillingMeters billingMeters() { @@ -390,9 +424,58 @@ public BillingMeters billingMeters() { return billingMeters; } + /** + * Gets the resource collection API of Builders. It manages BuilderResource. + * + * @return Resource collection API of Builders. + */ + public Builders builders() { + if (this.builders == null) { + this.builders = new BuildersImpl(clientObject.getBuilders(), this); + } + return builders; + } + + /** + * Gets the resource collection API of BuildsByBuilderResources. + * + * @return Resource collection API of BuildsByBuilderResources. + */ + public BuildsByBuilderResources buildsByBuilderResources() { + if (this.buildsByBuilderResources == null) { + this.buildsByBuilderResources + = new BuildsByBuilderResourcesImpl(clientObject.getBuildsByBuilderResources(), this); + } + return buildsByBuilderResources; + } + + /** + * Gets the resource collection API of Builds. It manages BuildResource. + * + * @return Resource collection API of Builds. + */ + public Builds builds() { + if (this.builds == null) { + this.builds = new BuildsImpl(clientObject.getBuilds(), this); + } + return builds; + } + + /** + * Gets the resource collection API of BuildAuthTokens. + * + * @return Resource collection API of BuildAuthTokens. + */ + public BuildAuthTokens buildAuthTokens() { + if (this.buildAuthTokens == null) { + this.buildAuthTokens = new BuildAuthTokensImpl(clientObject.getBuildAuthTokens(), this); + } + return buildAuthTokens; + } + /** * Gets the resource collection API of ConnectedEnvironments. It manages ConnectedEnvironment. - * + * * @return Resource collection API of ConnectedEnvironments. */ public ConnectedEnvironments connectedEnvironments() { @@ -404,47 +487,46 @@ public ConnectedEnvironments connectedEnvironments() { /** * Gets the resource collection API of ConnectedEnvironmentsCertificates. It manages Certificate. - * + * * @return Resource collection API of ConnectedEnvironmentsCertificates. */ public ConnectedEnvironmentsCertificates connectedEnvironmentsCertificates() { if (this.connectedEnvironmentsCertificates == null) { - this.connectedEnvironmentsCertificates = - new ConnectedEnvironmentsCertificatesImpl(clientObject.getConnectedEnvironmentsCertificates(), this); + this.connectedEnvironmentsCertificates + = new ConnectedEnvironmentsCertificatesImpl(clientObject.getConnectedEnvironmentsCertificates(), this); } return connectedEnvironmentsCertificates; } /** * Gets the resource collection API of ConnectedEnvironmentsDaprComponents. It manages DaprComponent. - * + * * @return Resource collection API of ConnectedEnvironmentsDaprComponents. */ public ConnectedEnvironmentsDaprComponents connectedEnvironmentsDaprComponents() { if (this.connectedEnvironmentsDaprComponents == null) { - this.connectedEnvironmentsDaprComponents = - new ConnectedEnvironmentsDaprComponentsImpl( - clientObject.getConnectedEnvironmentsDaprComponents(), this); + this.connectedEnvironmentsDaprComponents = new ConnectedEnvironmentsDaprComponentsImpl( + clientObject.getConnectedEnvironmentsDaprComponents(), this); } return connectedEnvironmentsDaprComponents; } /** * Gets the resource collection API of ConnectedEnvironmentsStorages. It manages ConnectedEnvironmentStorage. - * + * * @return Resource collection API of ConnectedEnvironmentsStorages. */ public ConnectedEnvironmentsStorages connectedEnvironmentsStorages() { if (this.connectedEnvironmentsStorages == null) { - this.connectedEnvironmentsStorages = - new ConnectedEnvironmentsStoragesImpl(clientObject.getConnectedEnvironmentsStorages(), this); + this.connectedEnvironmentsStorages + = new ConnectedEnvironmentsStoragesImpl(clientObject.getConnectedEnvironmentsStorages(), this); } return connectedEnvironmentsStorages; } /** * Gets the resource collection API of ContainerApps. It manages ContainerApp. - * + * * @return Resource collection API of ContainerApps. */ public ContainerApps containerApps() { @@ -456,84 +538,72 @@ public ContainerApps containerApps() { /** * Gets the resource collection API of ContainerAppsRevisions. - * + * * @return Resource collection API of ContainerAppsRevisions. */ public ContainerAppsRevisions containerAppsRevisions() { if (this.containerAppsRevisions == null) { - this.containerAppsRevisions = - new ContainerAppsRevisionsImpl(clientObject.getContainerAppsRevisions(), this); + this.containerAppsRevisions + = new ContainerAppsRevisionsImpl(clientObject.getContainerAppsRevisions(), this); } return containerAppsRevisions; } /** * Gets the resource collection API of ContainerAppsRevisionReplicas. - * + * * @return Resource collection API of ContainerAppsRevisionReplicas. */ public ContainerAppsRevisionReplicas containerAppsRevisionReplicas() { if (this.containerAppsRevisionReplicas == null) { - this.containerAppsRevisionReplicas = - new ContainerAppsRevisionReplicasImpl(clientObject.getContainerAppsRevisionReplicas(), this); + this.containerAppsRevisionReplicas + = new ContainerAppsRevisionReplicasImpl(clientObject.getContainerAppsRevisionReplicas(), this); } return containerAppsRevisionReplicas; } /** * Gets the resource collection API of ContainerAppsDiagnostics. - * + * * @return Resource collection API of ContainerAppsDiagnostics. */ public ContainerAppsDiagnostics containerAppsDiagnostics() { if (this.containerAppsDiagnostics == null) { - this.containerAppsDiagnostics = - new ContainerAppsDiagnosticsImpl(clientObject.getContainerAppsDiagnostics(), this); + this.containerAppsDiagnostics + = new ContainerAppsDiagnosticsImpl(clientObject.getContainerAppsDiagnostics(), this); } return containerAppsDiagnostics; } /** * Gets the resource collection API of ManagedEnvironmentDiagnostics. - * + * * @return Resource collection API of ManagedEnvironmentDiagnostics. */ public ManagedEnvironmentDiagnostics managedEnvironmentDiagnostics() { if (this.managedEnvironmentDiagnostics == null) { - this.managedEnvironmentDiagnostics = - new ManagedEnvironmentDiagnosticsImpl(clientObject.getManagedEnvironmentDiagnostics(), this); + this.managedEnvironmentDiagnostics + = new ManagedEnvironmentDiagnosticsImpl(clientObject.getManagedEnvironmentDiagnostics(), this); } return managedEnvironmentDiagnostics; } /** * Gets the resource collection API of ManagedEnvironmentsDiagnostics. - * + * * @return Resource collection API of ManagedEnvironmentsDiagnostics. */ public ManagedEnvironmentsDiagnostics managedEnvironmentsDiagnostics() { if (this.managedEnvironmentsDiagnostics == null) { - this.managedEnvironmentsDiagnostics = - new ManagedEnvironmentsDiagnosticsImpl(clientObject.getManagedEnvironmentsDiagnostics(), this); + this.managedEnvironmentsDiagnostics + = new ManagedEnvironmentsDiagnosticsImpl(clientObject.getManagedEnvironmentsDiagnostics(), this); } return managedEnvironmentsDiagnostics; } - /** - * Gets the resource collection API of Operations. - * - * @return Resource collection API of Operations. - */ - public Operations operations() { - if (this.operations == null) { - this.operations = new OperationsImpl(clientObject.getOperations(), this); - } - return operations; - } - /** * Gets the resource collection API of Jobs. It manages Job. - * + * * @return Resource collection API of Jobs. */ public Jobs jobs() { @@ -543,9 +613,21 @@ public Jobs jobs() { return jobs; } + /** + * Gets the resource collection API of Operations. + * + * @return Resource collection API of Operations. + */ + public Operations operations() { + if (this.operations == null) { + this.operations = new OperationsImpl(clientObject.getOperations(), this); + } + return operations; + } + /** * Gets the resource collection API of JobsExecutions. - * + * * @return Resource collection API of JobsExecutions. */ public JobsExecutions jobsExecutions() { @@ -557,7 +639,7 @@ public JobsExecutions jobsExecutions() { /** * Gets the resource collection API of ResourceProviders. - * + * * @return Resource collection API of ResourceProviders. */ public ResourceProviders resourceProviders() { @@ -569,7 +651,7 @@ public ResourceProviders resourceProviders() { /** * Gets the resource collection API of ManagedEnvironments. It manages ManagedEnvironment. - * + * * @return Resource collection API of ManagedEnvironments. */ public ManagedEnvironments managedEnvironments() { @@ -581,7 +663,7 @@ public ManagedEnvironments managedEnvironments() { /** * Gets the resource collection API of Certificates. - * + * * @return Resource collection API of Certificates. */ public Certificates certificates() { @@ -593,7 +675,7 @@ public Certificates certificates() { /** * Gets the resource collection API of ManagedCertificates. It manages ManagedCertificate. - * + * * @return Resource collection API of ManagedCertificates. */ public ManagedCertificates managedCertificates() { @@ -605,7 +687,7 @@ public ManagedCertificates managedCertificates() { /** * Gets the resource collection API of Namespaces. - * + * * @return Resource collection API of Namespaces. */ public Namespaces namespaces() { @@ -615,9 +697,22 @@ public Namespaces namespaces() { return namespaces; } + /** + * Gets the resource collection API of DaprComponentResiliencyPolicies. It manages DaprComponentResiliencyPolicy. + * + * @return Resource collection API of DaprComponentResiliencyPolicies. + */ + public DaprComponentResiliencyPolicies daprComponentResiliencyPolicies() { + if (this.daprComponentResiliencyPolicies == null) { + this.daprComponentResiliencyPolicies + = new DaprComponentResiliencyPoliciesImpl(clientObject.getDaprComponentResiliencyPolicies(), this); + } + return daprComponentResiliencyPolicies; + } + /** * Gets the resource collection API of DaprComponents. - * + * * @return Resource collection API of DaprComponents. */ public DaprComponents daprComponents() { @@ -627,36 +722,97 @@ public DaprComponents daprComponents() { return daprComponents; } + /** + * Gets the resource collection API of DaprSubscriptions. It manages DaprSubscription. + * + * @return Resource collection API of DaprSubscriptions. + */ + public DaprSubscriptions daprSubscriptions() { + if (this.daprSubscriptions == null) { + this.daprSubscriptions = new DaprSubscriptionsImpl(clientObject.getDaprSubscriptions(), this); + } + return daprSubscriptions; + } + /** * Gets the resource collection API of ManagedEnvironmentsStorages. It manages ManagedEnvironmentStorage. - * + * * @return Resource collection API of ManagedEnvironmentsStorages. */ public ManagedEnvironmentsStorages managedEnvironmentsStorages() { if (this.managedEnvironmentsStorages == null) { - this.managedEnvironmentsStorages = - new ManagedEnvironmentsStoragesImpl(clientObject.getManagedEnvironmentsStorages(), this); + this.managedEnvironmentsStorages + = new ManagedEnvironmentsStoragesImpl(clientObject.getManagedEnvironmentsStorages(), this); } return managedEnvironmentsStorages; } /** * Gets the resource collection API of ContainerAppsSourceControls. It manages SourceControl. - * + * * @return Resource collection API of ContainerAppsSourceControls. */ public ContainerAppsSourceControls containerAppsSourceControls() { if (this.containerAppsSourceControls == null) { - this.containerAppsSourceControls = - new ContainerAppsSourceControlsImpl(clientObject.getContainerAppsSourceControls(), this); + this.containerAppsSourceControls + = new ContainerAppsSourceControlsImpl(clientObject.getContainerAppsSourceControls(), this); } return containerAppsSourceControls; } + /** + * Gets the resource collection API of Usages. + * + * @return Resource collection API of Usages. + */ + public Usages usages() { + if (this.usages == null) { + this.usages = new UsagesImpl(clientObject.getUsages(), this); + } + return usages; + } + + /** + * Gets the resource collection API of ManagedEnvironmentUsages. + * + * @return Resource collection API of ManagedEnvironmentUsages. + */ + public ManagedEnvironmentUsages managedEnvironmentUsages() { + if (this.managedEnvironmentUsages == null) { + this.managedEnvironmentUsages + = new ManagedEnvironmentUsagesImpl(clientObject.getManagedEnvironmentUsages(), this); + } + return managedEnvironmentUsages; + } + + /** + * Gets the resource collection API of JavaComponents. It manages JavaComponent. + * + * @return Resource collection API of JavaComponents. + */ + public JavaComponents javaComponents() { + if (this.javaComponents == null) { + this.javaComponents = new JavaComponentsImpl(clientObject.getJavaComponents(), this); + } + return javaComponents; + } + + /** + * Gets the resource collection API of DotNetComponents. It manages DotNetComponent. + * + * @return Resource collection API of DotNetComponents. + */ + public DotNetComponents dotNetComponents() { + if (this.dotNetComponents == null) { + this.dotNetComponents = new DotNetComponentsImpl(clientObject.getDotNetComponents(), this); + } + return dotNetComponents; + } + /** * Gets wrapped service client ContainerAppsApiClient providing direct access to the underlying auto-generated API * implementation, based on Azure REST API. - * + * * @return Wrapped service client ContainerAppsApiClient. */ public ContainerAppsApiClient serviceClient() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AppResilienciesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AppResilienciesClient.java new file mode 100644 index 0000000000000..1a40d64209978 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AppResilienciesClient.java @@ -0,0 +1,198 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner; + +/** + * An instance of this class provides access to all the operations defined in AppResilienciesClient. + */ +public interface AppResilienciesClient { + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createOrUpdateWithResponse(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope, Context context); + + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AppResiliencyInner createOrUpdate(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope); + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response updateWithResponse(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope, Context context); + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AppResiliencyInner update(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope); + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response deleteWithResponse(String resourceGroupName, String appName, String name, Context context); + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String appName, String name); + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String appName, String name, + Context context); + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AppResiliencyInner get(String resourceGroupName, String appName, String name); + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String appName); + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String appName, Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AvailableWorkloadProfilesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AvailableWorkloadProfilesClient.java index 1c4ae5157f864..07b54b046db8a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AvailableWorkloadProfilesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/AvailableWorkloadProfilesClient.java @@ -10,13 +10,15 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.AvailableWorkloadProfileInner; -/** An instance of this class provides access to all the operations defined in AvailableWorkloadProfilesClient. */ +/** + * An instance of this class provides access to all the operations defined in AvailableWorkloadProfilesClient. + */ public interface AvailableWorkloadProfilesClient { /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -28,9 +30,9 @@ public interface AvailableWorkloadProfilesClient { /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BillingMetersClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BillingMetersClient.java index b6877f2b5e11c..114cdeccd72fb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BillingMetersClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BillingMetersClient.java @@ -10,18 +10,20 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.BillingMeterCollectionInner; -/** An instance of this class provides access to all the operations defined in BillingMetersClient. */ +/** + * An instance of this class provides access to all the operations defined in BillingMetersClient. + */ public interface BillingMetersClient { /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all billingMeters for a location along with {@link Response}. */ @@ -30,13 +32,13 @@ public interface BillingMetersClient { /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all billingMeters for a location. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildAuthTokensClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildAuthTokensClient.java new file mode 100644 index 0000000000000..c794c5f7277a6 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildAuthTokensClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner; + +/** + * An instance of this class provides access to all the operations defined in BuildAuthTokensClient. + */ +public interface BuildAuthTokensClient { + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build along with + * {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listWithResponse(String resourceGroupName, String builderName, String buildName, + Context context); + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BuildTokenInner list(String resourceGroupName, String builderName, String buildName); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildersClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildersClient.java new file mode 100644 index 0000000000000..65cd73fe23a40 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildersClient.java @@ -0,0 +1,270 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuilderResourceUpdate; + +/** + * An instance of this class provides access to all the operations defined in BuildersClient. + */ +public interface BuildersClient { + /** + * List BuilderResource resources by subscription ID. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(); + + /** + * List BuilderResource resources by subscription ID. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(Context context); + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByResourceGroup(String resourceGroupName); + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByResourceGroup(String resourceGroupName, Context context); + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getByResourceGroupWithResponse(String resourceGroupName, String builderName, + Context context); + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BuilderResourceInner getByResourceGroup(String resourceGroupName, String builderName); + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, BuilderResourceInner> beginCreateOrUpdate(String resourceGroupName, + String builderName, BuilderResourceInner builderEnvelope); + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, BuilderResourceInner> beginCreateOrUpdate(String resourceGroupName, + String builderName, BuilderResourceInner builderEnvelope, Context context); + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BuilderResourceInner createOrUpdate(String resourceGroupName, String builderName, + BuilderResourceInner builderEnvelope); + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BuilderResourceInner createOrUpdate(String resourceGroupName, String builderName, + BuilderResourceInner builderEnvelope, Context context); + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, BuilderResourceInner> beginUpdate(String resourceGroupName, + String builderName, BuilderResourceUpdate builderEnvelope); + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, BuilderResourceInner> beginUpdate(String resourceGroupName, + String builderName, BuilderResourceUpdate builderEnvelope, Context context); + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BuilderResourceInner update(String resourceGroupName, String builderName, BuilderResourceUpdate builderEnvelope); + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BuilderResourceInner update(String resourceGroupName, String builderName, BuilderResourceUpdate builderEnvelope, + Context context); + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String builderName); + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String builderName, Context context); + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String builderName); + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String builderName, Context context); +} diff --git a/sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/SupportTicketChatTranscriptsNoSubscriptionsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsByBuilderResourcesClient.java similarity index 53% rename from sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/SupportTicketChatTranscriptsNoSubscriptionsClient.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsByBuilderResourcesClient.java index d76c14546f06f..a957f2df2405a 100644 --- a/sdk/support/azure-resourcemanager-support/src/main/java/com/azure/resourcemanager/support/fluent/SupportTicketChatTranscriptsNoSubscriptionsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsByBuilderResourcesClient.java @@ -2,41 +2,42 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.support.fluent; +package com.azure.resourcemanager.appcontainers.fluent; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceMethod; import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -import com.azure.resourcemanager.support.fluent.models.ChatTranscriptDetailsInner; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; /** - * An instance of this class provides access to all the operations defined in - * SupportTicketChatTranscriptsNoSubscriptionsClient. + * An instance of this class provides access to all the operations defined in BuildsByBuilderResourcesClient. */ -public interface SupportTicketChatTranscriptsNoSubscriptionsClient { +public interface BuildsByBuilderResourcesClient { /** - * Lists all chat transcripts for a support ticket. - * - * @param supportTicketName Support ticket name. + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of Chat Transcripts resources as paginated response with {@link PagedIterable}. + * @return the response of a BuildResource list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String supportTicketName); + PagedIterable list(String resourceGroupName, String builderName); /** - * Lists all chat transcripts for a support ticket. - * - * @param supportTicketName Support ticket name. + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of Chat Transcripts resources as paginated response with {@link PagedIterable}. + * @return the response of a BuildResource list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String supportTicketName, Context context); + PagedIterable list(String resourceGroupName, String builderName, Context context); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountBackupsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsClient.java similarity index 51% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountBackupsClient.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsClient.java index 0210a2d5ac51f..5cce1f950e181 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountBackupsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/BuildsClient.java @@ -2,113 +2,136 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.fluent; +package com.azure.resourcemanager.appcontainers.fluent; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; /** - * An instance of this class provides access to all the operations defined in AccountBackupsClient. + * An instance of this class provides access to all the operations defined in BuildsClient. */ -public interface AccountBackupsClient { +public interface BuildsClient { /** - * List Backups for a Netapp Account + * Get a BuildResource. * - * List all Backups for a Netapp Account. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuildResource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String builderName, String buildName, + Context context); + + /** + * Get a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. + * @return a BuildResource. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByNetAppAccount(String resourceGroupName, String accountName); + @ServiceMethod(returns = ReturnType.SINGLE) + BuildResourceInner get(String resourceGroupName, String builderName, String buildName); /** - * List Backups for a Netapp Account + * Create a BuildResource. * - * List all Backups for a Netapp Account. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information pertaining to an individual build. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, BuildResourceInner> beginCreateOrUpdate(String resourceGroupName, + String builderName, String buildName, BuildResourceInner buildEnvelope); + + /** + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param includeOnlyBackupsFromDeletedVolumes An option to specify whether to return backups only from deleted - * volumes. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. + * @return the {@link SyncPoller} for polling of information pertaining to an individual build. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, - String includeOnlyBackupsFromDeletedVolumes, Context context); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, BuildResourceInner> beginCreateOrUpdate(String resourceGroupName, + String builderName, String buildName, BuildResourceInner buildEnvelope, Context context); /** - * Get Backup for a Netapp Account - * - * Gets the specified backup for a Netapp Account. + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account along with {@link Response}. + * @return information pertaining to an individual build. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String resourceGroupName, String accountName, String backupName, - Context context); + BuildResourceInner createOrUpdate(String resourceGroupName, String builderName, String buildName, + BuildResourceInner buildEnvelope); /** - * Get Backup for a Netapp Account - * - * Gets the specified backup for a Netapp Account. + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account. + * @return information pertaining to an individual build. */ @ServiceMethod(returns = ReturnType.SINGLE) - BackupInner get(String resourceGroupName, String accountName, String backupName); + BuildResourceInner createOrUpdate(String resourceGroupName, String builderName, String buildName, + BuildResourceInner buildEnvelope, Context context); /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, String backupName); + SyncPoller, Void> beginDelete(String resourceGroupName, String builderName, String buildName); /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -116,37 +139,33 @@ Response getWithResponse(String resourceGroupName, String accountNa * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, String backupName, + SyncPoller, Void> beginDelete(String resourceGroupName, String builderName, String buildName, Context context); /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String accountName, String backupName); + void delete(String resourceGroupName, String builderName, String buildName); /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String accountName, String backupName, Context context); + void delete(String resourceGroupName, String builderName, String buildName, Context context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/CertificatesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/CertificatesClient.java index 1e66f7dd0665c..5963e2aaa823a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/CertificatesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/CertificatesClient.java @@ -12,16 +12,18 @@ import com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner; import com.azure.resourcemanager.appcontainers.models.CertificatePatch; -/** An instance of this class provides access to all the operations defined in CertificatesClient. */ +/** + * An instance of this class provides access to all the operations defined in CertificatesClient. + */ public interface CertificatesClient { /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -30,13 +32,13 @@ public interface CertificatesClient { /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -45,30 +47,30 @@ public interface CertificatesClient { /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, String certificateName, + Context context); /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate. */ @@ -77,7 +79,7 @@ Response getWithResponse( /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -85,28 +87,24 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificateInner certificateEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificateInner certificateEnvelope, Context context); /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @@ -115,30 +113,30 @@ Response createOrUpdateWithResponse( /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String certificateName, + Context context); /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -146,9 +144,9 @@ Response deleteWithResponse( /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -156,35 +154,31 @@ Response deleteWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context); + Response updateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificatePatch certificateEnvelope, Context context); /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @param certificateEnvelope Properties of a certificate that need to be updated. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - CertificateInner update( - String resourceGroupName, String environmentName, String certificateName, CertificatePatch certificateEnvelope); + CertificateInner update(String resourceGroupName, String environmentName, String certificateName, + CertificatePatch certificateEnvelope); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsCertificatesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsCertificatesClient.java index 3924eac328c37..b40e6779b5018 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsCertificatesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsCertificatesClient.java @@ -18,12 +18,12 @@ public interface ConnectedEnvironmentsCertificatesClient { /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given connected environment as paginated response with {@link PagedIterable}. */ @@ -32,13 +32,13 @@ public interface ConnectedEnvironmentsCertificatesClient { /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given connected environment as paginated response with {@link PagedIterable}. */ @@ -47,30 +47,30 @@ public interface ConnectedEnvironmentsCertificatesClient { /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context); + Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context); /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate. */ @@ -79,7 +79,7 @@ Response getWithResponse( /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -87,28 +87,24 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificateInner certificateEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, CertificateInner certificateEnvelope, Context context); /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @@ -117,30 +113,30 @@ Response createOrUpdateWithResponse( /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context); + Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, String certificateName, + Context context); /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -148,9 +144,9 @@ Response deleteWithResponse( /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -158,38 +154,31 @@ Response deleteWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context); + Response updateWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, CertificatePatch certificateEnvelope, Context context); /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @param certificateEnvelope Properties of a certificate that need to be updated. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - CertificateInner update( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, + CertificateInner update(String resourceGroupName, String connectedEnvironmentName, String certificateName, CertificatePatch certificateEnvelope); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsClient.java index efa280c81bfa1..bf40789c97718 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsClient.java @@ -15,13 +15,15 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentInner; import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest; -/** An instance of this class provides access to all the operations defined in ConnectedEnvironmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsClient. + */ public interface ConnectedEnvironmentsClient { /** * Get all connectedEnvironments for a subscription. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription as paginated response with {@link PagedIterable}. */ @@ -30,11 +32,11 @@ public interface ConnectedEnvironmentsClient { /** * Get all connectedEnvironments for a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription as paginated response with {@link PagedIterable}. */ @@ -43,11 +45,11 @@ public interface ConnectedEnvironmentsClient { /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments in a resource group as paginated response with {@link PagedIterable}. */ @@ -56,12 +58,12 @@ public interface ConnectedEnvironmentsClient { /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments in a resource group as paginated response with {@link PagedIterable}. */ @@ -70,28 +72,28 @@ public interface ConnectedEnvironmentsClient { /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of an connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context); /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of an connectedEnvironment. */ @@ -100,16 +102,16 @@ Response getByResourceGroupWithResponse( /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for Kubernetes cluster specialized for web workloads - * by Azure App Service. + * by Azure App Service. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, ConnectedEnvironmentInner> beginCreateOrUpdate( @@ -117,69 +119,64 @@ SyncPoller, ConnectedEnvironmentInner> beg /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for Kubernetes cluster specialized for web workloads - * by Azure App Service. + * by Azure App Service. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, ConnectedEnvironmentInner> beginCreateOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, + String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope, Context context); /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ @ServiceMethod(returns = ReturnType.SINGLE) - ConnectedEnvironmentInner createOrUpdate( - String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope); + ConnectedEnvironmentInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + ConnectedEnvironmentInner environmentEnvelope); /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ @ServiceMethod(returns = ReturnType.SINGLE) - ConnectedEnvironmentInner createOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, - Context context); + ConnectedEnvironmentInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + ConnectedEnvironmentInner environmentEnvelope, Context context); /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -188,28 +185,28 @@ ConnectedEnvironmentInner createOrUpdate( /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String connectedEnvironmentName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String connectedEnvironmentName, + Context context); /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -217,13 +214,13 @@ SyncPoller, Void> beginDelete( /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -231,33 +228,33 @@ SyncPoller, Void> beginDelete( /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context); + Response updateWithResponse(String resourceGroupName, String connectedEnvironmentName, + Context context); /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ @@ -266,43 +263,38 @@ Response updateWithResponse( /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context); + Response checkNameAvailabilityWithResponse(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context); /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result. */ @ServiceMethod(returns = ReturnType.SINGLE) - CheckNameAvailabilityResponseInner checkNameAvailability( - String resourceGroupName, - String connectedEnvironmentName, + CheckNameAvailabilityResponseInner checkNameAvailability(String resourceGroupName, String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsDaprComponentsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsDaprComponentsClient.java index f137979cc9f88..b0839ec314635 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsDaprComponentsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsDaprComponentsClient.java @@ -13,17 +13,18 @@ import com.azure.resourcemanager.appcontainers.fluent.models.DaprSecretsCollectionInner; /** - * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsDaprComponentsClient. + * An instance of this class provides access to all the operations defined in + * ConnectedEnvironmentsDaprComponentsClient. */ public interface ConnectedEnvironmentsDaprComponentsClient { /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a connected environment as paginated response with {@link PagedIterable}. */ @@ -32,13 +33,13 @@ public interface ConnectedEnvironmentsDaprComponentsClient { /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a connected environment as paginated response with {@link PagedIterable}. */ @@ -47,30 +48,30 @@ public interface ConnectedEnvironmentsDaprComponentsClient { /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context); + Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context); /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component. */ @@ -79,9 +80,9 @@ Response getWithResponse( /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -89,70 +90,63 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr Component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, DaprComponentInner daprComponentEnvelope, Context context); /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param daprComponentEnvelope Configuration details of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - DaprComponentInner createOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, + DaprComponentInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, String componentName, DaprComponentInner daprComponentEnvelope); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context); + Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, String componentName, + Context context); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -160,34 +154,34 @@ Response deleteWithResponse( /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listSecretsWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context); + Response listSecretsWithResponse(String resourceGroupName, + String connectedEnvironmentName, String componentName, Context context); /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action. */ @ServiceMethod(returns = ReturnType.SINGLE) - DaprSecretsCollectionInner listSecrets( - String resourceGroupName, String connectedEnvironmentName, String componentName); + DaprSecretsCollectionInner listSecrets(String resourceGroupName, String connectedEnvironmentName, + String componentName); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsStoragesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsStoragesClient.java index 2718921f835dd..877de407b477f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsStoragesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ConnectedEnvironmentsStoragesClient.java @@ -11,32 +11,34 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStorageInner; import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStoragesCollectionInner; -/** An instance of this class provides access to all the operations defined in ConnectedEnvironmentsStoragesClient. */ +/** + * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsStoragesClient. + */ public interface ConnectedEnvironmentsStoragesClient { /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context); + Response listWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context); /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a connectedEnvironment. */ @@ -45,30 +47,30 @@ Response listWithResponse( /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context); + Response getWithResponse(String resourceGroupName, + String connectedEnvironmentName, String storageName, Context context); /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a connectedEnvironment. */ @@ -77,7 +79,7 @@ Response getWithResponse( /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -85,64 +87,58 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage resource for connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope, + Response createOrUpdateWithResponse(String resourceGroupName, + String connectedEnvironmentName, String storageName, ConnectedEnvironmentStorageInner storageEnvelope, Context context); /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @param storageEnvelope Configuration details of storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage resource for connectedEnvironment. */ @ServiceMethod(returns = ReturnType.SINGLE) - ConnectedEnvironmentStorageInner createOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope); + ConnectedEnvironmentStorageInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + String storageName, ConnectedEnvironmentStorageInner storageEnvelope); /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context); + Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, String storageName, + Context context); /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsApiClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsApiClient.java index 03bef3cfa47b8..51855da3e2ffa 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsApiClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsApiClient.java @@ -7,208 +7,287 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for ContainerAppsApiClient class. */ +/** + * The interface for ContainerAppsApiClient class. + */ public interface ContainerAppsApiClient { /** - * Gets The ID of the target subscription. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); + /** + * Gets the AppResilienciesClient object to access its operations. + * + * @return the AppResilienciesClient object. + */ + AppResilienciesClient getAppResiliencies(); + /** * Gets the ContainerAppsAuthConfigsClient object to access its operations. - * + * * @return the ContainerAppsAuthConfigsClient object. */ ContainerAppsAuthConfigsClient getContainerAppsAuthConfigs(); /** * Gets the AvailableWorkloadProfilesClient object to access its operations. - * + * * @return the AvailableWorkloadProfilesClient object. */ AvailableWorkloadProfilesClient getAvailableWorkloadProfiles(); /** * Gets the BillingMetersClient object to access its operations. - * + * * @return the BillingMetersClient object. */ BillingMetersClient getBillingMeters(); + /** + * Gets the BuildersClient object to access its operations. + * + * @return the BuildersClient object. + */ + BuildersClient getBuilders(); + + /** + * Gets the BuildsByBuilderResourcesClient object to access its operations. + * + * @return the BuildsByBuilderResourcesClient object. + */ + BuildsByBuilderResourcesClient getBuildsByBuilderResources(); + + /** + * Gets the BuildsClient object to access its operations. + * + * @return the BuildsClient object. + */ + BuildsClient getBuilds(); + + /** + * Gets the BuildAuthTokensClient object to access its operations. + * + * @return the BuildAuthTokensClient object. + */ + BuildAuthTokensClient getBuildAuthTokens(); + /** * Gets the ConnectedEnvironmentsClient object to access its operations. - * + * * @return the ConnectedEnvironmentsClient object. */ ConnectedEnvironmentsClient getConnectedEnvironments(); /** * Gets the ConnectedEnvironmentsCertificatesClient object to access its operations. - * + * * @return the ConnectedEnvironmentsCertificatesClient object. */ ConnectedEnvironmentsCertificatesClient getConnectedEnvironmentsCertificates(); /** * Gets the ConnectedEnvironmentsDaprComponentsClient object to access its operations. - * + * * @return the ConnectedEnvironmentsDaprComponentsClient object. */ ConnectedEnvironmentsDaprComponentsClient getConnectedEnvironmentsDaprComponents(); /** * Gets the ConnectedEnvironmentsStoragesClient object to access its operations. - * + * * @return the ConnectedEnvironmentsStoragesClient object. */ ConnectedEnvironmentsStoragesClient getConnectedEnvironmentsStorages(); /** * Gets the ContainerAppsClient object to access its operations. - * + * * @return the ContainerAppsClient object. */ ContainerAppsClient getContainerApps(); /** * Gets the ContainerAppsRevisionsClient object to access its operations. - * + * * @return the ContainerAppsRevisionsClient object. */ ContainerAppsRevisionsClient getContainerAppsRevisions(); /** * Gets the ContainerAppsRevisionReplicasClient object to access its operations. - * + * * @return the ContainerAppsRevisionReplicasClient object. */ ContainerAppsRevisionReplicasClient getContainerAppsRevisionReplicas(); /** * Gets the ContainerAppsDiagnosticsClient object to access its operations. - * + * * @return the ContainerAppsDiagnosticsClient object. */ ContainerAppsDiagnosticsClient getContainerAppsDiagnostics(); /** * Gets the ManagedEnvironmentDiagnosticsClient object to access its operations. - * + * * @return the ManagedEnvironmentDiagnosticsClient object. */ ManagedEnvironmentDiagnosticsClient getManagedEnvironmentDiagnostics(); /** * Gets the ManagedEnvironmentsDiagnosticsClient object to access its operations. - * + * * @return the ManagedEnvironmentsDiagnosticsClient object. */ ManagedEnvironmentsDiagnosticsClient getManagedEnvironmentsDiagnostics(); - /** - * Gets the OperationsClient object to access its operations. - * - * @return the OperationsClient object. - */ - OperationsClient getOperations(); - /** * Gets the JobsClient object to access its operations. - * + * * @return the JobsClient object. */ JobsClient getJobs(); + /** + * Gets the OperationsClient object to access its operations. + * + * @return the OperationsClient object. + */ + OperationsClient getOperations(); + /** * Gets the JobsExecutionsClient object to access its operations. - * + * * @return the JobsExecutionsClient object. */ JobsExecutionsClient getJobsExecutions(); /** * Gets the ResourceProvidersClient object to access its operations. - * + * * @return the ResourceProvidersClient object. */ ResourceProvidersClient getResourceProviders(); /** * Gets the ManagedEnvironmentsClient object to access its operations. - * + * * @return the ManagedEnvironmentsClient object. */ ManagedEnvironmentsClient getManagedEnvironments(); /** * Gets the CertificatesClient object to access its operations. - * + * * @return the CertificatesClient object. */ CertificatesClient getCertificates(); /** * Gets the ManagedCertificatesClient object to access its operations. - * + * * @return the ManagedCertificatesClient object. */ ManagedCertificatesClient getManagedCertificates(); /** * Gets the NamespacesClient object to access its operations. - * + * * @return the NamespacesClient object. */ NamespacesClient getNamespaces(); + /** + * Gets the DaprComponentResiliencyPoliciesClient object to access its operations. + * + * @return the DaprComponentResiliencyPoliciesClient object. + */ + DaprComponentResiliencyPoliciesClient getDaprComponentResiliencyPolicies(); + /** * Gets the DaprComponentsClient object to access its operations. - * + * * @return the DaprComponentsClient object. */ DaprComponentsClient getDaprComponents(); + /** + * Gets the DaprSubscriptionsClient object to access its operations. + * + * @return the DaprSubscriptionsClient object. + */ + DaprSubscriptionsClient getDaprSubscriptions(); + /** * Gets the ManagedEnvironmentsStoragesClient object to access its operations. - * + * * @return the ManagedEnvironmentsStoragesClient object. */ ManagedEnvironmentsStoragesClient getManagedEnvironmentsStorages(); /** * Gets the ContainerAppsSourceControlsClient object to access its operations. - * + * * @return the ContainerAppsSourceControlsClient object. */ ContainerAppsSourceControlsClient getContainerAppsSourceControls(); + + /** + * Gets the UsagesClient object to access its operations. + * + * @return the UsagesClient object. + */ + UsagesClient getUsages(); + + /** + * Gets the ManagedEnvironmentUsagesClient object to access its operations. + * + * @return the ManagedEnvironmentUsagesClient object. + */ + ManagedEnvironmentUsagesClient getManagedEnvironmentUsages(); + + /** + * Gets the JavaComponentsClient object to access its operations. + * + * @return the JavaComponentsClient object. + */ + JavaComponentsClient getJavaComponents(); + + /** + * Gets the DotNetComponentsClient object to access its operations. + * + * @return the DotNetComponentsClient object. + */ + DotNetComponentsClient getDotNetComponents(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsAuthConfigsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsAuthConfigsClient.java index 9b742d9945c5b..c28af17f98d36 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsAuthConfigsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsAuthConfigsClient.java @@ -11,16 +11,18 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigInner; -/** An instance of this class provides access to all the operations defined in ContainerAppsAuthConfigsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsAuthConfigsClient. + */ public interface ContainerAppsAuthConfigsClient { /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App AuthConfigs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -29,46 +31,46 @@ public interface ContainerAppsAuthConfigsClient { /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App AuthConfigs in a given resource group as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByContainerApp( - String resourceGroupName, String containerAppName, Context context); + PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, + Context context); /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a AuthConfig of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context); + Response getWithResponse(String resourceGroupName, String containerAppName, String authConfigName, + Context context); /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a AuthConfig of a Container App. */ @@ -77,7 +79,7 @@ Response getWithResponse( /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -85,62 +87,58 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature along - * with {@link Response}. + * with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String containerAppName, - String authConfigName, - AuthConfigInner authConfigEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String containerAppName, + String authConfigName, AuthConfigInner authConfigEnvelope, Context context); /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @param authConfigEnvelope Properties used to create a Container App AuthConfig. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature. */ @ServiceMethod(returns = ReturnType.SINGLE) - AuthConfigInner createOrUpdate( - String resourceGroupName, String containerAppName, String authConfigName, AuthConfigInner authConfigEnvelope); + AuthConfigInner createOrUpdate(String resourceGroupName, String containerAppName, String authConfigName, + AuthConfigInner authConfigEnvelope); /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context); + Response deleteWithResponse(String resourceGroupName, String containerAppName, String authConfigName, + Context context); /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsClient.java index 42624e5ff627c..88af33e45a937 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsClient.java @@ -16,13 +16,15 @@ import com.azure.resourcemanager.appcontainers.fluent.models.CustomHostnameAnalysisResultInner; import com.azure.resourcemanager.appcontainers.fluent.models.SecretsCollectionInner; -/** An instance of this class provides access to all the operations defined in ContainerAppsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsClient. + */ public interface ContainerAppsClient { /** * Get the Container Apps in a given subscription. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription as paginated response with {@link PagedIterable}. */ @@ -31,11 +33,11 @@ public interface ContainerAppsClient { /** * Get the Container Apps in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription as paginated response with {@link PagedIterable}. */ @@ -44,11 +46,11 @@ public interface ContainerAppsClient { /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given resource group as paginated response with {@link PagedIterable}. */ @@ -57,12 +59,12 @@ public interface ContainerAppsClient { /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given resource group as paginated response with {@link PagedIterable}. */ @@ -71,32 +73,32 @@ public interface ContainerAppsClient { /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String containerAppName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String containerAppName, + Context context); /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App. */ @@ -105,78 +107,78 @@ Response getByResourceGroupWithResponse( /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginCreateOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope); + SyncPoller, ContainerAppInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope); /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginCreateOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context); + SyncPoller, ContainerAppInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope, Context context); /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - ContainerAppInner createOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope); + ContainerAppInner createOrUpdate(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope); /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - ContainerAppInner createOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context); + ContainerAppInner createOrUpdate(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope, Context context); /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -185,13 +187,13 @@ ContainerAppInner createOrUpdate( /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -200,12 +202,12 @@ ContainerAppInner createOrUpdate( /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -213,13 +215,13 @@ ContainerAppInner createOrUpdate( /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -227,52 +229,52 @@ ContainerAppInner createOrUpdate( /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope); + SyncPoller, ContainerAppInner> beginUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope); /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context); + SyncPoller, ContainerAppInner> beginUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope, Context context); /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -281,48 +283,48 @@ SyncPoller, ContainerAppInner> beginUpdate( /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - ContainerAppInner update( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context); + ContainerAppInner update(String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, + Context context); /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param customHostname Custom hostname. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return custom domain analysis along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listCustomHostnameAnalysisWithResponse( - String resourceGroupName, String containerAppName, String customHostname, Context context); + Response listCustomHostnameAnalysisWithResponse(String resourceGroupName, + String containerAppName, String customHostname, Context context); /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return custom domain analysis. */ @@ -331,28 +333,28 @@ Response listCustomHostnameAnalysisWithRespon /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listSecretsWithResponse( - String resourceGroupName, String containerAppName, Context context); + Response listSecretsWithResponse(String resourceGroupName, String containerAppName, + Context context); /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource. */ @@ -361,32 +363,32 @@ Response listSecretsWithResponse( /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return auth token for a container app along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getAuthTokenWithResponse( - String resourceGroupName, String containerAppName, Context context); + Response getAuthTokenWithResponse(String resourceGroupName, String containerAppName, + Context context); /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return auth token for a container app. */ @@ -395,43 +397,43 @@ Response getAuthTokenWithResponse( /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginStart( - String resourceGroupName, String containerAppName); + SyncPoller, ContainerAppInner> beginStart(String resourceGroupName, + String containerAppName); /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginStart( - String resourceGroupName, String containerAppName, Context context); + SyncPoller, ContainerAppInner> beginStart(String resourceGroupName, + String containerAppName, Context context); /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -440,13 +442,13 @@ SyncPoller, ContainerAppInner> beginStart( /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -455,43 +457,43 @@ SyncPoller, ContainerAppInner> beginStart( /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginStop( - String resourceGroupName, String containerAppName); + SyncPoller, ContainerAppInner> beginStop(String resourceGroupName, + String containerAppName); /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppInner> beginStop( - String resourceGroupName, String containerAppName, Context context); + SyncPoller, ContainerAppInner> beginStop(String resourceGroupName, + String containerAppName, Context context); /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -500,13 +502,13 @@ SyncPoller, ContainerAppInner> beginStop( /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsDiagnosticsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsDiagnosticsClient.java index ed4321095d887..6460fb251e7a0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsDiagnosticsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsDiagnosticsClient.java @@ -13,16 +13,18 @@ import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; import com.azure.resourcemanager.appcontainers.fluent.models.RevisionInner; -/** An instance of this class provides access to all the operations defined in ContainerAppsDiagnosticsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsDiagnosticsClient. + */ public interface ContainerAppsDiagnosticsClient { /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a given Container App as paginated response with {@link PagedIterable}. */ @@ -31,13 +33,13 @@ public interface ContainerAppsDiagnosticsClient { /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a given Container App as paginated response with {@link PagedIterable}. */ @@ -46,30 +48,30 @@ public interface ContainerAppsDiagnosticsClient { /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a diagnostics result of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getDetectorWithResponse( - String resourceGroupName, String containerAppName, String detectorName, Context context); + Response getDetectorWithResponse(String resourceGroupName, String containerAppName, + String detectorName, Context context); /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a diagnostics result of a Container App. */ @@ -78,12 +80,12 @@ Response getDetectorWithResponse( /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @@ -92,47 +94,47 @@ Response getDetectorWithResponse( /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context); + PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response getRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App. */ @@ -141,15 +143,15 @@ Response getRevisionWithResponse( /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App along with {@link Response}. */ @@ -158,14 +160,14 @@ Response getRevisionWithResponse( /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionReplicasClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionReplicasClient.java index 79ad308766229..05a7e08c569a9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionReplicasClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionReplicasClient.java @@ -11,11 +11,13 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ReplicaCollectionInner; import com.azure.resourcemanager.appcontainers.fluent.models.ReplicaInner; -/** An instance of this class provides access to all the operations defined in ContainerAppsRevisionReplicasClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsRevisionReplicasClient. + */ public interface ContainerAppsRevisionReplicasClient { /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -23,24 +25,24 @@ public interface ContainerAppsRevisionReplicasClient { * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a replica for a Container App Revision along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getReplicaWithResponse( - String resourceGroupName, String containerAppName, String revisionName, String replicaName, Context context); + Response getReplicaWithResponse(String resourceGroupName, String containerAppName, + String revisionName, String replicaName, Context context); /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param replicaName Name of the Container App Revision Replica. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a replica for a Container App Revision. */ @@ -49,30 +51,30 @@ Response getReplicaWithResponse( /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revision Replicas collection ARM resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listReplicasWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response listReplicasWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context); /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revision Replicas collection ARM resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionsClient.java index fc706efbed30c..c8079e70967d9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsRevisionsClient.java @@ -11,16 +11,18 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.RevisionInner; -/** An instance of this class provides access to all the operations defined in ContainerAppsRevisionsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsRevisionsClient. + */ public interface ContainerAppsRevisionsClient { /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @@ -29,47 +31,47 @@ public interface ContainerAppsRevisionsClient { /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context); + PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response getRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App. */ @@ -78,30 +80,30 @@ Response getRevisionWithResponse( /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response activateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response activateRevisionWithResponse(String resourceGroupName, String containerAppName, String revisionName, + Context context); /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -109,30 +111,30 @@ Response activateRevisionWithResponse( /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deactivateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response deactivateRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context); /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -140,30 +142,30 @@ Response deactivateRevisionWithResponse( /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response restartRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response restartRevisionWithResponse(String resourceGroupName, String containerAppName, String revisionName, + Context context); /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsSourceControlsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsSourceControlsClient.java index 52444e4f6782a..1d972a0620042 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsSourceControlsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ContainerAppsSourceControlsClient.java @@ -13,66 +13,68 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.appcontainers.fluent.models.SourceControlInner; -/** An instance of this class provides access to all the operations defined in ContainerAppsSourceControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsSourceControlsClient. + */ public interface ContainerAppsSourceControlsClient { /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Container App SourceControls in a given resource group as paginated response with {@link - * PagedIterable}. + * @return the Container App SourceControls in a given resource group as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByContainerApp(String resourceGroupName, String containerAppName); /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Container App SourceControls in a given resource group as paginated response with {@link - * PagedIterable}. + * @return the Container App SourceControls in a given resource group as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByContainerApp( - String resourceGroupName, String containerAppName, Context context); + PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, + Context context); /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a SourceControl of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String containerAppName, String sourceControlName, Context context); + Response getWithResponse(String resourceGroupName, String containerAppName, + String sourceControlName, Context context); /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a SourceControl of a Container App. */ @@ -81,27 +83,24 @@ Response getWithResponse( /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param sourceControlEnvelope Properties used to create a Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App SourceControl. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SourceControlInner> beginCreateOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope); + SyncPoller, SourceControlInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope); /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -109,41 +108,34 @@ SyncPoller, SourceControlInner> beginCreateOrUpda * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App SourceControl. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SourceControlInner> beginCreateOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context); + SyncPoller, SourceControlInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope, Context context); /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param sourceControlEnvelope Properties used to create a Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App SourceControl. */ @ServiceMethod(returns = ReturnType.SINGLE) - SourceControlInner createOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, + SourceControlInner createOrUpdate(String resourceGroupName, String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope); /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -151,60 +143,56 @@ SourceControlInner createOrUpdate( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App SourceControl. */ @ServiceMethod(returns = ReturnType.SINGLE) - SourceControlInner createOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context); + SourceControlInner createOrUpdate(String resourceGroupName, String containerAppName, String sourceControlName, + SourceControlInner sourceControlEnvelope, Context context); /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String containerAppName, String sourceControlName); + SyncPoller, Void> beginDelete(String resourceGroupName, String containerAppName, + String sourceControlName); /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String containerAppName, String sourceControlName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String containerAppName, + String sourceControlName, Context context); /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -212,14 +200,14 @@ SyncPoller, Void> beginDelete( /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentResiliencyPoliciesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentResiliencyPoliciesClient.java new file mode 100644 index 0000000000000..c057e69e3e88f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentResiliencyPoliciesClient.java @@ -0,0 +1,164 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; + +/** + * An instance of this class provides access to all the operations defined in DaprComponentResiliencyPoliciesClient. + */ +public interface DaprComponentResiliencyPoliciesClient { + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName, + String componentName); + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName, + String componentName, Context context); + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String environmentName, + String componentName, String name, Context context); + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DaprComponentResiliencyPolicyInner get(String resourceGroupName, String environmentName, String componentName, + String name); + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createOrUpdateWithResponse(String resourceGroupName, + String environmentName, String componentName, String name, + DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope, Context context); + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DaprComponentResiliencyPolicyInner createOrUpdate(String resourceGroupName, String environmentName, + String componentName, String name, DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope); + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + String name, Context context); + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String environmentName, String componentName, String name); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentsClient.java index 62358a5fc62a4..cac5b28a2004f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprComponentsClient.java @@ -12,16 +12,18 @@ import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner; import com.azure.resourcemanager.appcontainers.fluent.models.DaprSecretsCollectionInner; -/** An instance of this class provides access to all the operations defined in DaprComponentsClient. */ +/** + * An instance of this class provides access to all the operations defined in DaprComponentsClient. + */ public interface DaprComponentsClient { /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a managed environment as paginated response with {@link PagedIterable}. */ @@ -30,13 +32,13 @@ public interface DaprComponentsClient { /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a managed environment as paginated response with {@link PagedIterable}. */ @@ -45,30 +47,30 @@ public interface DaprComponentsClient { /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, String componentName, + Context context); /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component. */ @@ -77,9 +79,9 @@ Response getWithResponse( /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -87,70 +89,63 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr Component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String componentName, DaprComponentInner daprComponentEnvelope, Context context); /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param daprComponentEnvelope Configuration details of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - DaprComponentInner createOrUpdate( - String resourceGroupName, - String environmentName, - String componentName, + DaprComponentInner createOrUpdate(String resourceGroupName, String environmentName, String componentName, DaprComponentInner daprComponentEnvelope); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + Context context); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -158,30 +153,30 @@ Response deleteWithResponse( /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listSecretsWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context); + Response listSecretsWithResponse(String resourceGroupName, String environmentName, + String componentName, Context context); /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprSubscriptionsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprSubscriptionsClient.java new file mode 100644 index 0000000000000..363e068ef7e3e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DaprSubscriptionsClient.java @@ -0,0 +1,151 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; + +/** + * An instance of this class provides access to all the operations defined in DaprSubscriptionsClient. + */ +public interface DaprSubscriptionsClient { + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName); + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName, Context context); + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context); + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DaprSubscriptionInner get(String resourceGroupName, String environmentName, String name); + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String name, DaprSubscriptionInner daprSubscriptionEnvelope, Context context); + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DaprSubscriptionInner createOrUpdate(String resourceGroupName, String environmentName, String name, + DaprSubscriptionInner daprSubscriptionEnvelope); + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response deleteWithResponse(String resourceGroupName, String environmentName, String name, Context context); + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String environmentName, String name); +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupVaultsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DotNetComponentsClient.java similarity index 56% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupVaultsClient.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DotNetComponentsClient.java index 0abfd494ddcdd..f321e882b169b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupVaultsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/DotNetComponentsClient.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.fluent; +package com.azure.resourcemanager.appcontainers.fluent; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceMethod; @@ -11,251 +11,237 @@ import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import com.azure.resourcemanager.netapp.models.BackupVaultPatch; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; /** - * An instance of this class provides access to all the operations defined in BackupVaultsClient. + * An instance of this class provides access to all the operations defined in DotNetComponentsClient. */ -public interface BackupVaultsClient { +public interface DotNetComponentsClient { /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the .NET Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedIterable}. + * @return the .NET Components for a managed environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByNetAppAccount(String resourceGroupName, String accountName); + PagedIterable list(String resourceGroupName, String environmentName); /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the .NET Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedIterable}. + * @return the .NET Components for a managed environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, Context context); + PagedIterable list(String resourceGroupName, String environmentName, Context context); /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response}. + * @return a .NET Component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, + Response getWithResponse(String resourceGroupName, String environmentName, String name, Context context); /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault. + * @return a .NET Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - BackupVaultInner get(String resourceGroupName, String accountName, String backupVaultName); + DotNetComponentInner get(String resourceGroupName, String environmentName, String name); /** - * Create or Update a Backup Vault + * Creates or updates a .NET Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a .NET Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of .NET Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupVaultInner> beginCreateOrUpdate(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultInner body); + SyncPoller, DotNetComponentInner> beginCreateOrUpdate(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope); /** - * Create or Update a Backup Vault + * Creates or updates a .NET Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a .NET Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of .NET Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupVaultInner> beginCreateOrUpdate(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultInner body, Context context); + SyncPoller, DotNetComponentInner> beginCreateOrUpdate(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, Context context); /** - * Create or Update a Backup Vault + * Creates or updates a .NET Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a .NET Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return .NET Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - BackupVaultInner createOrUpdate(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultInner body); + DotNetComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope); /** - * Create or Update a Backup Vault + * Creates or updates a .NET Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a .NET Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return .NET Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - BackupVaultInner createOrUpdate(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultInner body, Context context); + DotNetComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope, Context context); /** - * Update NetApp Backup Vault + * Update properties of a .NET Component * - * Patch the specified NetApp Backup Vault. + * Patches a .NET Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of .NET Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupVaultInner> beginUpdate(String resourceGroupName, String accountName, - String backupVaultName, BackupVaultPatch body); + SyncPoller, DotNetComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope); /** - * Update NetApp Backup Vault + * Update properties of a .NET Component * - * Patch the specified NetApp Backup Vault. + * Patches a .NET Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of .NET Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupVaultInner> beginUpdate(String resourceGroupName, String accountName, - String backupVaultName, BackupVaultPatch body, Context context); + SyncPoller, DotNetComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, Context context); /** - * Update NetApp Backup Vault + * Update properties of a .NET Component * - * Patch the specified NetApp Backup Vault. + * Patches a .NET Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return .NET Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - BackupVaultInner update(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultPatch body); + DotNetComponentInner update(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope); /** - * Update NetApp Backup Vault + * Update properties of a .NET Component * - * Patch the specified NetApp Backup Vault. + * Patches a .NET Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return .NET Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - BackupVaultInner update(String resourceGroupName, String accountName, String backupVaultName, BackupVaultPatch body, - Context context); + DotNetComponentInner update(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope, Context context); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupVaultName); + SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, String name); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -263,37 +249,33 @@ SyncPoller, Void> beginDelete(String resourceGroupName, String * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, String backupVaultName, + SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, String name, Context context); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String accountName, String backupVaultName); + void delete(String resourceGroupName, String environmentName, String name); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String accountName, String backupVaultName, Context context); + void delete(String resourceGroupName, String environmentName, String name, Context context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JavaComponentsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JavaComponentsClient.java new file mode 100644 index 0000000000000..160ff1772826e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JavaComponentsClient.java @@ -0,0 +1,281 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; + +/** + * An instance of this class provides access to all the operations defined in JavaComponentsClient. + */ +public interface JavaComponentsClient { + /** + * Get the Java Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Java Components for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName); + + /** + * Get the Java Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Java Components for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName, Context context); + + /** + * Get a Java Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Java Component along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context); + + /** + * Get a Java Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Java Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + JavaComponentInner get(String resourceGroupName, String environmentName, String name); + + /** + * Creates or updates a Java Component. + * + * Creates or updates a Java Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of java Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, JavaComponentInner> beginCreateOrUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope); + + /** + * Creates or updates a Java Component. + * + * Creates or updates a Java Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of java Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, JavaComponentInner> beginCreateOrUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope, Context context); + + /** + * Creates or updates a Java Component. + * + * Creates or updates a Java Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return java Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + JavaComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope); + + /** + * Creates or updates a Java Component. + * + * Creates or updates a Java Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return java Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + JavaComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope, Context context); + + /** + * Update properties of a Java Component + * + * Patches a Java Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of java Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, JavaComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope); + + /** + * Update properties of a Java Component + * + * Patches a Java Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of java Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, JavaComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope, Context context); + + /** + * Update properties of a Java Component + * + * Patches a Java Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return java Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + JavaComponentInner update(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope); + + /** + * Update properties of a Java Component + * + * Patches a Java Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return java Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + JavaComponentInner update(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope, Context context); + + /** + * Delete a Java Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, String name); + + /** + * Delete a Java Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, String name, + Context context); + + /** + * Delete a Java Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String environmentName, String name); + + /** + * Delete a Java Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String environmentName, String name, Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsClient.java index 934466bbe8ec4..a0b37059995f3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsClient.java @@ -12,19 +12,126 @@ import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner; +import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner; +import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionBaseInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobSecretsCollectionInner; import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate; import com.azure.resourcemanager.appcontainers.models.JobPatchProperties; -/** An instance of this class provides access to all the operations defined in JobsClient. */ +/** + * An instance of this class provides access to all the operations defined in JobsClient. + */ public interface JobsClient { + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listDetectorsWithResponse(String resourceGroupName, String jobName, + Context context); + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DiagnosticsCollectionInner listDetectors(String resourceGroupName, String jobName); + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getDetectorWithResponse(String resourceGroupName, String jobName, String detectorName, + Context context); + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DiagnosticsInner getDetector(String resourceGroupName, String jobName, String detectorName); + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response proxyGetWithResponse(String resourceGroupName, String jobName, Context context); + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + JobInner proxyGet(String resourceGroupName, String jobName); + /** * Get the Container Apps Jobs in a given subscription. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription as paginated response with {@link PagedIterable}. */ @@ -33,11 +140,11 @@ public interface JobsClient { /** * Get the Container Apps Jobs in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription as paginated response with {@link PagedIterable}. */ @@ -46,11 +153,11 @@ public interface JobsClient { /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -59,12 +166,12 @@ public interface JobsClient { /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -73,13 +180,13 @@ public interface JobsClient { /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container Apps Job along with {@link Response}. */ @@ -88,12 +195,12 @@ public interface JobsClient { /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container Apps Job. */ @@ -102,46 +209,46 @@ public interface JobsClient { /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, JobInner> beginCreateOrUpdate( - String resourceGroupName, String jobName, JobInner jobEnvelope); + SyncPoller, JobInner> beginCreateOrUpdate(String resourceGroupName, String jobName, + JobInner jobEnvelope); /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, JobInner> beginCreateOrUpdate( - String resourceGroupName, String jobName, JobInner jobEnvelope, Context context); + SyncPoller, JobInner> beginCreateOrUpdate(String resourceGroupName, String jobName, + JobInner jobEnvelope, Context context); /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Job. */ @@ -150,14 +257,14 @@ SyncPoller, JobInner> beginCreateOrUpdate( /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Job. */ @@ -166,12 +273,12 @@ SyncPoller, JobInner> beginCreateOrUpdate( /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -180,13 +287,13 @@ SyncPoller, JobInner> beginCreateOrUpdate( /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -195,12 +302,12 @@ SyncPoller, JobInner> beginCreateOrUpdate( /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -208,13 +315,13 @@ SyncPoller, JobInner> beginCreateOrUpdate( /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -222,52 +329,52 @@ SyncPoller, JobInner> beginCreateOrUpdate( /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, JobInner> beginUpdate( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope); + SyncPoller, JobInner> beginUpdate(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope); /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, JobInner> beginUpdate( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope, Context context); + SyncPoller, JobInner> beginUpdate(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope, Context context); /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Job. */ @@ -276,16 +383,16 @@ SyncPoller, JobInner> beginUpdate( /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Job. */ @@ -294,44 +401,44 @@ SyncPoller, JobInner> beginUpdate( /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, JobExecutionBaseInner> beginStart( - String resourceGroupName, String jobName); + SyncPoller, JobExecutionBaseInner> beginStart(String resourceGroupName, + String jobName); /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, JobExecutionBaseInner> beginStart( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context); + SyncPoller, JobExecutionBaseInner> beginStart(String resourceGroupName, + String jobName, JobExecutionTemplate template, Context context); /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App's Job execution name. */ @@ -340,63 +447,63 @@ SyncPoller, JobExecutionBaseInner> beginStart( /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App's Job execution name. */ @ServiceMethod(returns = ReturnType.SINGLE) - JobExecutionBaseInner start( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context); + JobExecutionBaseInner start(String resourceGroupName, String jobName, JobExecutionTemplate template, + Context context); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginStopExecution( - String resourceGroupName, String jobName, String jobExecutionName); + SyncPoller, Void> beginStopExecution(String resourceGroupName, String jobName, + String jobExecutionName); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginStopExecution( - String resourceGroupName, String jobName, String jobExecutionName, Context context); + SyncPoller, Void> beginStopExecution(String resourceGroupName, String jobName, + String jobExecutionName, Context context); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -404,14 +511,14 @@ SyncPoller, Void> beginStopExecution( /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -419,43 +526,43 @@ SyncPoller, Void> beginStopExecution( /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App executions collection ARM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppJobExecutionsInner> beginStopMultipleExecutions( - String resourceGroupName, String jobName); + SyncPoller, ContainerAppJobExecutionsInner> + beginStopMultipleExecutions(String resourceGroupName, String jobName); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of container App executions collection ARM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ContainerAppJobExecutionsInner> beginStopMultipleExecutions( - String resourceGroupName, String jobName, Context context); + SyncPoller, ContainerAppJobExecutionsInner> + beginStopMultipleExecutions(String resourceGroupName, String jobName, Context context); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource. */ @@ -464,13 +571,13 @@ SyncPoller, ContainerAppJobExecutions /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource. */ @@ -479,28 +586,28 @@ SyncPoller, ContainerAppJobExecutions /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listSecretsWithResponse( - String resourceGroupName, String jobName, Context context); + Response listSecretsWithResponse(String resourceGroupName, String jobName, + Context context); /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsExecutionsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsExecutionsClient.java index 402658e9be846..d9c95e6f4a71d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsExecutionsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/JobsExecutionsClient.java @@ -10,16 +10,18 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner; -/** An instance of this class provides access to all the operations defined in JobsExecutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in JobsExecutionsClient. + */ public interface JobsExecutionsClient { /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Container Apps Job's executions as paginated response with {@link PagedIterable}. */ @@ -28,14 +30,14 @@ public interface JobsExecutionsClient { /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Container Apps Job's executions as paginated response with {@link PagedIterable}. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedCertificatesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedCertificatesClient.java index 30113c809f0dc..37406c086470e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedCertificatesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedCertificatesClient.java @@ -14,34 +14,36 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ManagedCertificateInner; import com.azure.resourcemanager.appcontainers.models.ManagedCertificatePatch; -/** An instance of this class provides access to all the operations defined in ManagedCertificatesClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedCertificatesClient. + */ public interface ManagedCertificatesClient { /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Managed Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, Context context); /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Managed Certificate. */ @@ -50,26 +52,26 @@ Response getWithResponse( /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 400. + * status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ManagedCertificateInner> beginCreateOrUpdate( - String resourceGroupName, String environmentName, String managedCertificateName); + SyncPoller, ManagedCertificateInner> + beginCreateOrUpdate(String resourceGroupName, String environmentName, String managedCertificateName); /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -77,42 +79,39 @@ SyncPoller, ManagedCertificateInner> beginCr * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 400. + * status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, ManagedCertificateInner> beginCreateOrUpdate( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, - Context context); + String resourceGroupName, String environmentName, String managedCertificateName, + ManagedCertificateInner managedCertificateEnvelope, Context context); /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 400. + * status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedCertificateInner createOrUpdate( - String resourceGroupName, String environmentName, String managedCertificateName); + ManagedCertificateInner createOrUpdate(String resourceGroupName, String environmentName, + String managedCertificateName); /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -120,46 +119,42 @@ ManagedCertificateInner createOrUpdate( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 400. + * status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedCertificateInner createOrUpdate( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, - Context context); + ManagedCertificateInner createOrUpdate(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope, Context context); /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String managedCertificateName, + Context context); /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -167,9 +162,9 @@ Response deleteWithResponse( /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -177,49 +172,42 @@ Response deleteWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment along - * with {@link Response}. + * with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificatePatch managedCertificateEnvelope, - Context context); + Response updateWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope, Context context); /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @param managedCertificateEnvelope Properties of a managed certificate that need to be updated. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedCertificateInner update( - String resourceGroupName, - String environmentName, - String managedCertificateName, + ManagedCertificateInner update(String resourceGroupName, String environmentName, String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope); /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Managed Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -228,13 +216,13 @@ ManagedCertificateInner update( /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Managed Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentDiagnosticsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentDiagnosticsClient.java index e42f15fa36a03..9969602d43537 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentDiagnosticsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentDiagnosticsClient.java @@ -11,37 +11,39 @@ import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner; import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentDiagnosticsClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentDiagnosticsClient. + */ public interface ManagedEnvironmentDiagnosticsClient { /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of diagnostics for a Managed Environment used to host container apps along with {@link - * Response}. + * @return the list of diagnostics for a Managed Environment used to host container apps along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listDetectorsWithResponse( - String resourceGroupName, String environmentName, Context context); + Response listDetectorsWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a Managed Environment used to host container apps. */ @@ -50,34 +52,34 @@ Response listDetectorsWithResponse( /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getDetectorWithResponse( - String resourceGroupName, String environmentName, String detectorName, Context context); + Response getDetectorWithResponse(String resourceGroupName, String environmentName, + String detectorName, Context context); /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentUsagesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentUsagesClient.java new file mode 100644 index 0000000000000..3d9e05a007a28 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentUsagesClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; + +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentUsagesClient. + */ +public interface ManagedEnvironmentUsagesClient { + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName); + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String environmentName, Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsClient.java index 9631d1323e226..115d58555ac3a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsClient.java @@ -15,15 +15,17 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentInner; import com.azure.resourcemanager.appcontainers.fluent.models.WorkloadProfileStatesInner; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentsClient. + */ public interface ManagedEnvironmentsClient { /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription as paginated response with {@link PagedIterable}. */ @@ -32,13 +34,13 @@ public interface ManagedEnvironmentsClient { /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription as paginated response with {@link PagedIterable}. */ @@ -47,13 +49,13 @@ public interface ManagedEnvironmentsClient { /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the Managed Environments in a resource group as paginated response with {@link PagedIterable}. */ @@ -62,14 +64,14 @@ public interface ManagedEnvironmentsClient { /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the Managed Environments in a resource group as paginated response with {@link PagedIterable}. */ @@ -78,32 +80,32 @@ public interface ManagedEnvironmentsClient { /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String environmentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps. */ @@ -112,15 +114,15 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for hosting container apps. */ @@ -130,16 +132,16 @@ SyncPoller, ManagedEnvironmentInner> beginCr /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for hosting container apps. */ @@ -149,51 +151,51 @@ SyncPoller, ManagedEnvironmentInner> beginCr /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedEnvironmentInner createOrUpdate( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope); + ManagedEnvironmentInner createOrUpdate(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope); /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedEnvironmentInner createOrUpdate( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context); + ManagedEnvironmentInner createOrUpdate(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context); /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -202,15 +204,15 @@ ManagedEnvironmentInner createOrUpdate( /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @@ -219,14 +221,14 @@ ManagedEnvironmentInner createOrUpdate( /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -234,15 +236,15 @@ ManagedEnvironmentInner createOrUpdate( /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) @@ -250,106 +252,106 @@ ManagedEnvironmentInner createOrUpdate( /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ManagedEnvironmentInner> beginUpdate( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope); + SyncPoller, ManagedEnvironmentInner> beginUpdate(String resourceGroupName, + String environmentName, ManagedEnvironmentInner environmentEnvelope); /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ManagedEnvironmentInner> beginUpdate( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context); + SyncPoller, ManagedEnvironmentInner> beginUpdate(String resourceGroupName, + String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context); /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedEnvironmentInner update( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope); + ManagedEnvironmentInner update(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope); /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedEnvironmentInner update( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context); + ManagedEnvironmentInner update(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context); /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return environment Auth Token along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getAuthTokenWithResponse( - String resourceGroupName, String environmentName, Context context); + Response getAuthTokenWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return environment Auth Token. */ @@ -358,36 +360,36 @@ Response getAuthTokenWithResponse( /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName); + PagedIterable listWorkloadProfileStates(String resourceGroupName, + String environmentName); /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName, Context context); + PagedIterable listWorkloadProfileStates(String resourceGroupName, + String environmentName, Context context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsDiagnosticsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsDiagnosticsClient.java index c6580d78fdf51..1b4c03b2060dd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsDiagnosticsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsDiagnosticsClient.java @@ -10,36 +10,38 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentInner; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentsDiagnosticsClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentsDiagnosticsClient. + */ public interface ManagedEnvironmentsDiagnosticsClient { /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getRootWithResponse( - String resourceGroupName, String environmentName, Context context); + Response getRootWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsStoragesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsStoragesClient.java index f4a809933d52e..d3f52ead4a63e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsStoragesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ManagedEnvironmentsStoragesClient.java @@ -11,32 +11,34 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStorageInner; import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStoragesCollectionInner; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentsStoragesClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentsStoragesClient. + */ public interface ManagedEnvironmentsStoragesClient { /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a managedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listWithResponse( - String resourceGroupName, String environmentName, Context context); + Response listWithResponse(String resourceGroupName, + String environmentName, Context context); /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a managedEnvironment. */ @@ -45,30 +47,30 @@ Response listWithResponse( /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a managedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, + String storageName, Context context); /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a managedEnvironment. */ @@ -77,7 +79,7 @@ Response getWithResponse( /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -85,64 +87,57 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage resource for managedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String storageName, - ManagedEnvironmentStorageInner storageEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, + String environmentName, String storageName, ManagedEnvironmentStorageInner storageEnvelope, Context context); /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @param storageEnvelope Configuration details of storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage resource for managedEnvironment. */ @ServiceMethod(returns = ReturnType.SINGLE) - ManagedEnvironmentStorageInner createOrUpdate( - String resourceGroupName, - String environmentName, - String storageName, + ManagedEnvironmentStorageInner createOrUpdate(String resourceGroupName, String environmentName, String storageName, ManagedEnvironmentStorageInner storageEnvelope); /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String storageName, + Context context); /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/NamespacesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/NamespacesClient.java index 27878d6ce998c..df0db6b1e37e7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/NamespacesClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/NamespacesClient.java @@ -11,45 +11,44 @@ import com.azure.resourcemanager.appcontainers.fluent.models.CheckNameAvailabilityResponseInner; import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest; -/** An instance of this class provides access to all the operations defined in NamespacesClient. */ +/** + * An instance of this class provides access to all the operations defined in NamespacesClient. + */ public interface NamespacesClient { /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String environmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context); + Response checkNameAvailabilityWithResponse(String resourceGroupName, + String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context); /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result. */ @ServiceMethod(returns = ReturnType.SINGLE) - CheckNameAvailabilityResponseInner checkNameAvailability( - String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest); + CheckNameAvailabilityResponseInner checkNameAvailability(String resourceGroupName, String environmentName, + CheckNameAvailabilityRequest checkNameAvailabilityRequest); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/OperationsClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/OperationsClient.java index a9e17d64326ed..e421fd4a6c0a8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/OperationsClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/OperationsClient.java @@ -10,13 +10,15 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.OperationDetailInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Lists all of the available RP operations. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return available operations of the service as paginated response with {@link PagedIterable}. */ @@ -25,11 +27,11 @@ public interface OperationsClient { /** * Lists all of the available RP operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return available operations of the service as paginated response with {@link PagedIterable}. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ResourceProvidersClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ResourceProvidersClient.java index 9b59268f6d084..752dab23418df 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ResourceProvidersClient.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/ResourceProvidersClient.java @@ -10,37 +10,63 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner; -/** An instance of this class provides access to all the operations defined in ResourceProvidersClient. */ +/** + * An instance of this class provides access to all the operations defined in ResourceProvidersClient. + */ public interface ResourceProvidersClient { /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return details of a single job execution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response jobExecutionWithResponse( - String resourceGroupName, String jobName, String jobExecutionName, Context context); + Response jobExecutionWithResponse(String resourceGroupName, String jobName, + String jobExecutionName, Context context); /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return details of a single job execution. */ @ServiceMethod(returns = ReturnType.SINGLE) JobExecutionInner jobExecution(String resourceGroupName, String jobName, String jobExecutionName); + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getCustomDomainVerificationIdWithResponse(Context context); + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + String getCustomDomainVerificationId(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/UsagesClient.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/UsagesClient.java new file mode 100644 index 0000000000000..d57572384b323 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/UsagesClient.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; + +/** + * An instance of this class provides access to all the operations defined in UsagesClient. + */ +public interface UsagesClient { + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String location); + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String location, Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyInner.java new file mode 100644 index 0000000000000..8c8854f07d7f0 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyInner.java @@ -0,0 +1,207 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy; +import com.azure.resourcemanager.appcontainers.models.HttpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.HttpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TcpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration to setup App Resiliency. + */ +@Fluent +public final class AppResiliencyInner extends ProxyResource { + /* + * App Resiliency resource specific properties + */ + @JsonProperty(value = "properties") + private AppResiliencyProperties innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of AppResiliencyInner class. + */ + public AppResiliencyInner() { + } + + /** + * Get the innerProperties property: App Resiliency resource specific properties. + * + * @return the innerProperties value. + */ + private AppResiliencyProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the timeoutPolicy property: Policy to set request timeouts. + * + * @return the timeoutPolicy value. + */ + public TimeoutPolicy timeoutPolicy() { + return this.innerProperties() == null ? null : this.innerProperties().timeoutPolicy(); + } + + /** + * Set the timeoutPolicy property: Policy to set request timeouts. + * + * @param timeoutPolicy the timeoutPolicy value to set. + * @return the AppResiliencyInner object itself. + */ + public AppResiliencyInner withTimeoutPolicy(TimeoutPolicy timeoutPolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new AppResiliencyProperties(); + } + this.innerProperties().withTimeoutPolicy(timeoutPolicy); + return this; + } + + /** + * Get the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @return the httpRetryPolicy value. + */ + public HttpRetryPolicy httpRetryPolicy() { + return this.innerProperties() == null ? null : this.innerProperties().httpRetryPolicy(); + } + + /** + * Set the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @param httpRetryPolicy the httpRetryPolicy value to set. + * @return the AppResiliencyInner object itself. + */ + public AppResiliencyInner withHttpRetryPolicy(HttpRetryPolicy httpRetryPolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new AppResiliencyProperties(); + } + this.innerProperties().withHttpRetryPolicy(httpRetryPolicy); + return this; + } + + /** + * Get the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @return the tcpRetryPolicy value. + */ + public TcpRetryPolicy tcpRetryPolicy() { + return this.innerProperties() == null ? null : this.innerProperties().tcpRetryPolicy(); + } + + /** + * Set the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @param tcpRetryPolicy the tcpRetryPolicy value to set. + * @return the AppResiliencyInner object itself. + */ + public AppResiliencyInner withTcpRetryPolicy(TcpRetryPolicy tcpRetryPolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new AppResiliencyProperties(); + } + this.innerProperties().withTcpRetryPolicy(tcpRetryPolicy); + return this; + } + + /** + * Get the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @return the circuitBreakerPolicy value. + */ + public CircuitBreakerPolicy circuitBreakerPolicy() { + return this.innerProperties() == null ? null : this.innerProperties().circuitBreakerPolicy(); + } + + /** + * Set the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @param circuitBreakerPolicy the circuitBreakerPolicy value to set. + * @return the AppResiliencyInner object itself. + */ + public AppResiliencyInner withCircuitBreakerPolicy(CircuitBreakerPolicy circuitBreakerPolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new AppResiliencyProperties(); + } + this.innerProperties().withCircuitBreakerPolicy(circuitBreakerPolicy); + return this; + } + + /** + * Get the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @return the httpConnectionPool value. + */ + public HttpConnectionPool httpConnectionPool() { + return this.innerProperties() == null ? null : this.innerProperties().httpConnectionPool(); + } + + /** + * Set the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @param httpConnectionPool the httpConnectionPool value to set. + * @return the AppResiliencyInner object itself. + */ + public AppResiliencyInner withHttpConnectionPool(HttpConnectionPool httpConnectionPool) { + if (this.innerProperties() == null) { + this.innerProperties = new AppResiliencyProperties(); + } + this.innerProperties().withHttpConnectionPool(httpConnectionPool); + return this; + } + + /** + * Get the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @return the tcpConnectionPool value. + */ + public TcpConnectionPool tcpConnectionPool() { + return this.innerProperties() == null ? null : this.innerProperties().tcpConnectionPool(); + } + + /** + * Set the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @param tcpConnectionPool the tcpConnectionPool value to set. + * @return the AppResiliencyInner object itself. + */ + public AppResiliencyInner withTcpConnectionPool(TcpConnectionPool tcpConnectionPool) { + if (this.innerProperties() == null) { + this.innerProperties = new AppResiliencyProperties(); + } + this.innerProperties().withTcpConnectionPool(tcpConnectionPool); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyProperties.java new file mode 100644 index 0000000000000..aa0dd302ad097 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AppResiliencyProperties.java @@ -0,0 +1,208 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy; +import com.azure.resourcemanager.appcontainers.models.HttpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.HttpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TcpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * App Resiliency resource specific properties. + */ +@Fluent +public final class AppResiliencyProperties { + /* + * Policy to set request timeouts + */ + @JsonProperty(value = "timeoutPolicy") + private TimeoutPolicy timeoutPolicy; + + /* + * Policy that defines http request retry conditions + */ + @JsonProperty(value = "httpRetryPolicy") + private HttpRetryPolicy httpRetryPolicy; + + /* + * Policy that defines tcp request retry conditions + */ + @JsonProperty(value = "tcpRetryPolicy") + private TcpRetryPolicy tcpRetryPolicy; + + /* + * Policy that defines circuit breaker conditions + */ + @JsonProperty(value = "circuitBreakerPolicy") + private CircuitBreakerPolicy circuitBreakerPolicy; + + /* + * Defines parameters for http connection pooling + */ + @JsonProperty(value = "httpConnectionPool") + private HttpConnectionPool httpConnectionPool; + + /* + * Defines parameters for tcp connection pooling + */ + @JsonProperty(value = "tcpConnectionPool") + private TcpConnectionPool tcpConnectionPool; + + /** + * Creates an instance of AppResiliencyProperties class. + */ + public AppResiliencyProperties() { + } + + /** + * Get the timeoutPolicy property: Policy to set request timeouts. + * + * @return the timeoutPolicy value. + */ + public TimeoutPolicy timeoutPolicy() { + return this.timeoutPolicy; + } + + /** + * Set the timeoutPolicy property: Policy to set request timeouts. + * + * @param timeoutPolicy the timeoutPolicy value to set. + * @return the AppResiliencyProperties object itself. + */ + public AppResiliencyProperties withTimeoutPolicy(TimeoutPolicy timeoutPolicy) { + this.timeoutPolicy = timeoutPolicy; + return this; + } + + /** + * Get the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @return the httpRetryPolicy value. + */ + public HttpRetryPolicy httpRetryPolicy() { + return this.httpRetryPolicy; + } + + /** + * Set the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @param httpRetryPolicy the httpRetryPolicy value to set. + * @return the AppResiliencyProperties object itself. + */ + public AppResiliencyProperties withHttpRetryPolicy(HttpRetryPolicy httpRetryPolicy) { + this.httpRetryPolicy = httpRetryPolicy; + return this; + } + + /** + * Get the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @return the tcpRetryPolicy value. + */ + public TcpRetryPolicy tcpRetryPolicy() { + return this.tcpRetryPolicy; + } + + /** + * Set the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @param tcpRetryPolicy the tcpRetryPolicy value to set. + * @return the AppResiliencyProperties object itself. + */ + public AppResiliencyProperties withTcpRetryPolicy(TcpRetryPolicy tcpRetryPolicy) { + this.tcpRetryPolicy = tcpRetryPolicy; + return this; + } + + /** + * Get the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @return the circuitBreakerPolicy value. + */ + public CircuitBreakerPolicy circuitBreakerPolicy() { + return this.circuitBreakerPolicy; + } + + /** + * Set the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @param circuitBreakerPolicy the circuitBreakerPolicy value to set. + * @return the AppResiliencyProperties object itself. + */ + public AppResiliencyProperties withCircuitBreakerPolicy(CircuitBreakerPolicy circuitBreakerPolicy) { + this.circuitBreakerPolicy = circuitBreakerPolicy; + return this; + } + + /** + * Get the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @return the httpConnectionPool value. + */ + public HttpConnectionPool httpConnectionPool() { + return this.httpConnectionPool; + } + + /** + * Set the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @param httpConnectionPool the httpConnectionPool value to set. + * @return the AppResiliencyProperties object itself. + */ + public AppResiliencyProperties withHttpConnectionPool(HttpConnectionPool httpConnectionPool) { + this.httpConnectionPool = httpConnectionPool; + return this; + } + + /** + * Get the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @return the tcpConnectionPool value. + */ + public TcpConnectionPool tcpConnectionPool() { + return this.tcpConnectionPool; + } + + /** + * Set the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @param tcpConnectionPool the tcpConnectionPool value to set. + * @return the AppResiliencyProperties object itself. + */ + public AppResiliencyProperties withTcpConnectionPool(TcpConnectionPool tcpConnectionPool) { + this.tcpConnectionPool = tcpConnectionPool; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (timeoutPolicy() != null) { + timeoutPolicy().validate(); + } + if (httpRetryPolicy() != null) { + httpRetryPolicy().validate(); + } + if (tcpRetryPolicy() != null) { + tcpRetryPolicy().validate(); + } + if (circuitBreakerPolicy() != null) { + circuitBreakerPolicy().validate(); + } + if (httpConnectionPool() != null) { + httpConnectionPool().validate(); + } + if (tcpConnectionPool() != null) { + tcpConnectionPool().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigInner.java index 7a3c141ee7e55..0d159392fafbd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigInner.java @@ -8,13 +8,16 @@ import com.azure.core.management.ProxyResource; import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.AuthPlatform; +import com.azure.resourcemanager.appcontainers.models.EncryptionSettings; import com.azure.resourcemanager.appcontainers.models.GlobalValidation; import com.azure.resourcemanager.appcontainers.models.HttpSettings; import com.azure.resourcemanager.appcontainers.models.IdentityProviders; import com.azure.resourcemanager.appcontainers.models.Login; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration settings for the Azure ContainerApp Service Authentication / Authorization feature. */ +/** + * Configuration settings for the Azure ContainerApp Service Authentication / Authorization feature. + */ @Fluent public final class AuthConfigInner extends ProxyResource { /* @@ -29,13 +32,15 @@ public final class AuthConfigInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of AuthConfigInner class. */ + /** + * Creates an instance of AuthConfigInner class. + */ public AuthConfigInner() { } /** * Get the innerProperties property: AuthConfig resource specific properties. - * + * * @return the innerProperties value. */ private AuthConfigProperties innerProperties() { @@ -44,7 +49,7 @@ private AuthConfigProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -54,7 +59,7 @@ public SystemData systemData() { /** * Get the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization. - * + * * @return the platform value. */ public AuthPlatform platform() { @@ -64,7 +69,7 @@ public AuthPlatform platform() { /** * Set the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization. - * + * * @param platform the platform value to set. * @return the AuthConfigInner object itself. */ @@ -79,7 +84,7 @@ public AuthConfigInner withPlatform(AuthPlatform platform) { /** * Get the globalValidation property: The configuration settings that determines the validation flow of users using * Service Authentication/Authorization. - * + * * @return the globalValidation value. */ public GlobalValidation globalValidation() { @@ -89,7 +94,7 @@ public GlobalValidation globalValidation() { /** * Set the globalValidation property: The configuration settings that determines the validation flow of users using * Service Authentication/Authorization. - * + * * @param globalValidation the globalValidation value to set. * @return the AuthConfigInner object itself. */ @@ -104,7 +109,7 @@ public AuthConfigInner withGlobalValidation(GlobalValidation globalValidation) { /** * Get the identityProviders property: The configuration settings of each of the identity providers used to * configure ContainerApp Service Authentication/Authorization. - * + * * @return the identityProviders value. */ public IdentityProviders identityProviders() { @@ -114,7 +119,7 @@ public IdentityProviders identityProviders() { /** * Set the identityProviders property: The configuration settings of each of the identity providers used to * configure ContainerApp Service Authentication/Authorization. - * + * * @param identityProviders the identityProviders value to set. * @return the AuthConfigInner object itself. */ @@ -129,7 +134,7 @@ public AuthConfigInner withIdentityProviders(IdentityProviders identityProviders /** * Get the login property: The configuration settings of the login flow of users using ContainerApp Service * Authentication/Authorization. - * + * * @return the login value. */ public Login login() { @@ -139,7 +144,7 @@ public Login login() { /** * Set the login property: The configuration settings of the login flow of users using ContainerApp Service * Authentication/Authorization. - * + * * @param login the login value to set. * @return the AuthConfigInner object itself. */ @@ -154,7 +159,7 @@ public AuthConfigInner withLogin(Login login) { /** * Get the httpSettings property: The configuration settings of the HTTP requests for authentication and * authorization requests made against ContainerApp Service Authentication/Authorization. - * + * * @return the httpSettings value. */ public HttpSettings httpSettings() { @@ -164,7 +169,7 @@ public HttpSettings httpSettings() { /** * Set the httpSettings property: The configuration settings of the HTTP requests for authentication and * authorization requests made against ContainerApp Service Authentication/Authorization. - * + * * @param httpSettings the httpSettings value to set. * @return the AuthConfigInner object itself. */ @@ -176,9 +181,34 @@ public AuthConfigInner withHttpSettings(HttpSettings httpSettings) { return this; } + /** + * Get the encryptionSettings property: The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * + * @return the encryptionSettings value. + */ + public EncryptionSettings encryptionSettings() { + return this.innerProperties() == null ? null : this.innerProperties().encryptionSettings(); + } + + /** + * Set the encryptionSettings property: The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * + * @param encryptionSettings the encryptionSettings value to set. + * @return the AuthConfigInner object itself. + */ + public AuthConfigInner withEncryptionSettings(EncryptionSettings encryptionSettings) { + if (this.innerProperties() == null) { + this.innerProperties = new AuthConfigProperties(); + } + this.innerProperties().withEncryptionSettings(encryptionSettings); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigProperties.java index 9562abcf1c1de..2dd5af9254bd3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AuthConfigProperties.java @@ -6,13 +6,16 @@ import com.azure.core.annotation.Fluent; import com.azure.resourcemanager.appcontainers.models.AuthPlatform; +import com.azure.resourcemanager.appcontainers.models.EncryptionSettings; import com.azure.resourcemanager.appcontainers.models.GlobalValidation; import com.azure.resourcemanager.appcontainers.models.HttpSettings; import com.azure.resourcemanager.appcontainers.models.IdentityProviders; import com.azure.resourcemanager.appcontainers.models.Login; import com.fasterxml.jackson.annotation.JsonProperty; -/** AuthConfig resource specific properties. */ +/** + * AuthConfig resource specific properties. + */ @Fluent public final class AuthConfigProperties { /* @@ -22,7 +25,7 @@ public final class AuthConfigProperties { private AuthPlatform platform; /* - * The configuration settings that determines the validation flow of users using Service + * The configuration settings that determines the validation flow of users using Service * Authentication/Authorization. */ @JsonProperty(value = "globalValidation") @@ -48,14 +51,23 @@ public final class AuthConfigProperties { @JsonProperty(value = "httpSettings") private HttpSettings httpSettings; - /** Creates an instance of AuthConfigProperties class. */ + /* + * The configuration settings of the secrets references of encryption key and signing key for ContainerApp Service + * Authentication/Authorization. + */ + @JsonProperty(value = "encryptionSettings") + private EncryptionSettings encryptionSettings; + + /** + * Creates an instance of AuthConfigProperties class. + */ public AuthConfigProperties() { } /** * Get the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization. - * + * * @return the platform value. */ public AuthPlatform platform() { @@ -65,7 +77,7 @@ public AuthPlatform platform() { /** * Set the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization. - * + * * @param platform the platform value to set. * @return the AuthConfigProperties object itself. */ @@ -77,7 +89,7 @@ public AuthConfigProperties withPlatform(AuthPlatform platform) { /** * Get the globalValidation property: The configuration settings that determines the validation flow of users using * Service Authentication/Authorization. - * + * * @return the globalValidation value. */ public GlobalValidation globalValidation() { @@ -87,7 +99,7 @@ public GlobalValidation globalValidation() { /** * Set the globalValidation property: The configuration settings that determines the validation flow of users using * Service Authentication/Authorization. - * + * * @param globalValidation the globalValidation value to set. * @return the AuthConfigProperties object itself. */ @@ -99,7 +111,7 @@ public AuthConfigProperties withGlobalValidation(GlobalValidation globalValidati /** * Get the identityProviders property: The configuration settings of each of the identity providers used to * configure ContainerApp Service Authentication/Authorization. - * + * * @return the identityProviders value. */ public IdentityProviders identityProviders() { @@ -109,7 +121,7 @@ public IdentityProviders identityProviders() { /** * Set the identityProviders property: The configuration settings of each of the identity providers used to * configure ContainerApp Service Authentication/Authorization. - * + * * @param identityProviders the identityProviders value to set. * @return the AuthConfigProperties object itself. */ @@ -121,7 +133,7 @@ public AuthConfigProperties withIdentityProviders(IdentityProviders identityProv /** * Get the login property: The configuration settings of the login flow of users using ContainerApp Service * Authentication/Authorization. - * + * * @return the login value. */ public Login login() { @@ -131,7 +143,7 @@ public Login login() { /** * Set the login property: The configuration settings of the login flow of users using ContainerApp Service * Authentication/Authorization. - * + * * @param login the login value to set. * @return the AuthConfigProperties object itself. */ @@ -143,7 +155,7 @@ public AuthConfigProperties withLogin(Login login) { /** * Get the httpSettings property: The configuration settings of the HTTP requests for authentication and * authorization requests made against ContainerApp Service Authentication/Authorization. - * + * * @return the httpSettings value. */ public HttpSettings httpSettings() { @@ -153,7 +165,7 @@ public HttpSettings httpSettings() { /** * Set the httpSettings property: The configuration settings of the HTTP requests for authentication and * authorization requests made against ContainerApp Service Authentication/Authorization. - * + * * @param httpSettings the httpSettings value to set. * @return the AuthConfigProperties object itself. */ @@ -162,9 +174,31 @@ public AuthConfigProperties withHttpSettings(HttpSettings httpSettings) { return this; } + /** + * Get the encryptionSettings property: The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * + * @return the encryptionSettings value. + */ + public EncryptionSettings encryptionSettings() { + return this.encryptionSettings; + } + + /** + * Set the encryptionSettings property: The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * + * @param encryptionSettings the encryptionSettings value to set. + * @return the AuthConfigProperties object itself. + */ + public AuthConfigProperties withEncryptionSettings(EncryptionSettings encryptionSettings) { + this.encryptionSettings = encryptionSettings; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -183,5 +217,8 @@ public void validate() { if (httpSettings() != null) { httpSettings().validate(); } + if (encryptionSettings() != null) { + encryptionSettings().validate(); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AvailableWorkloadProfileInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AvailableWorkloadProfileInner.java index ad62df22ee74b..6128ac4509914 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AvailableWorkloadProfileInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/AvailableWorkloadProfileInner.java @@ -6,10 +6,13 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfileProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** A workload profile with specific hardware configure to run container apps. */ +/** + * A workload profile with specific hardware configure to run container apps. + */ @Fluent public final class AvailableWorkloadProfileInner extends ProxyResource { /* @@ -24,13 +27,21 @@ public final class AvailableWorkloadProfileInner extends ProxyResource { @JsonProperty(value = "properties") private AvailableWorkloadProfileProperties properties; - /** Creates an instance of AvailableWorkloadProfileInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of AvailableWorkloadProfileInner class. + */ public AvailableWorkloadProfileInner() { } /** * Get the location property: Region of the workload profile. - * + * * @return the location value. */ public String location() { @@ -39,7 +50,7 @@ public String location() { /** * Set the location property: Region of the workload profile. - * + * * @param location the location value to set. * @return the AvailableWorkloadProfileInner object itself. */ @@ -50,7 +61,7 @@ public AvailableWorkloadProfileInner withLocation(String location) { /** * Get the properties property: Revision resource specific properties. - * + * * @return the properties value. */ public AvailableWorkloadProfileProperties properties() { @@ -59,7 +70,7 @@ public AvailableWorkloadProfileProperties properties() { /** * Set the properties property: Revision resource specific properties. - * + * * @param properties the properties value to set. * @return the AvailableWorkloadProfileInner object itself. */ @@ -68,9 +79,18 @@ public AvailableWorkloadProfileInner withProperties(AvailableWorkloadProfileProp return this; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BillingMeterCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BillingMeterCollectionInner.java index a6bb8d5efcdcc..9508a6498f6e9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BillingMeterCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BillingMeterCollectionInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of billing meters. */ +/** + * Collection of billing meters. + */ @Fluent public final class BillingMeterCollectionInner { /* @@ -19,13 +21,15 @@ public final class BillingMeterCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of BillingMeterCollectionInner class. */ + /** + * Creates an instance of BillingMeterCollectionInner class. + */ public BillingMeterCollectionInner() { } /** * Get the value property: Collection of billing meters. - * + * * @return the value value. */ public List value() { @@ -34,7 +38,7 @@ public List value() { /** * Set the value property: Collection of billing meters. - * + * * @param value the value value to set. * @return the BillingMeterCollectionInner object itself. */ @@ -45,15 +49,13 @@ public BillingMeterCollectionInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model BillingMeterCollectionInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model BillingMeterCollectionInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildProperties.java new file mode 100644 index 0000000000000..3c9bb1a04b351 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildProperties.java @@ -0,0 +1,167 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.BuildConfiguration; +import com.azure.resourcemanager.appcontainers.models.BuildProvisioningState; +import com.azure.resourcemanager.appcontainers.models.BuildStatus; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The build properties. + */ +@Fluent +public final class BuildProperties { + /* + * Build provisioning state. + */ + @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) + private BuildProvisioningState provisioningState; + + /* + * Status of the build once it has been provisioned. + */ + @JsonProperty(value = "buildStatus", access = JsonProperty.Access.WRITE_ONLY) + private BuildStatus buildStatus; + + /* + * Container registry that the final image will be uploaded to. + */ + @JsonProperty(value = "destinationContainerRegistry") + private ContainerRegistryWithCustomImage destinationContainerRegistry; + + /* + * Configuration of the build. + */ + @JsonProperty(value = "configuration") + private BuildConfiguration configuration; + + /* + * Endpoint to which the source code should be uploaded. + */ + @JsonProperty(value = "uploadEndpoint", access = JsonProperty.Access.WRITE_ONLY) + private String uploadEndpoint; + + /* + * Endpoint from which the build logs can be streamed. + */ + @JsonProperty(value = "logStreamEndpoint", access = JsonProperty.Access.WRITE_ONLY) + private String logStreamEndpoint; + + /* + * Endpoint to use to retrieve an authentication token for log streaming and uploading source code. + */ + @JsonProperty(value = "tokenEndpoint", access = JsonProperty.Access.WRITE_ONLY) + private String tokenEndpoint; + + /** + * Creates an instance of BuildProperties class. + */ + public BuildProperties() { + } + + /** + * Get the provisioningState property: Build provisioning state. + * + * @return the provisioningState value. + */ + public BuildProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Get the buildStatus property: Status of the build once it has been provisioned. + * + * @return the buildStatus value. + */ + public BuildStatus buildStatus() { + return this.buildStatus; + } + + /** + * Get the destinationContainerRegistry property: Container registry that the final image will be uploaded to. + * + * @return the destinationContainerRegistry value. + */ + public ContainerRegistryWithCustomImage destinationContainerRegistry() { + return this.destinationContainerRegistry; + } + + /** + * Set the destinationContainerRegistry property: Container registry that the final image will be uploaded to. + * + * @param destinationContainerRegistry the destinationContainerRegistry value to set. + * @return the BuildProperties object itself. + */ + public BuildProperties + withDestinationContainerRegistry(ContainerRegistryWithCustomImage destinationContainerRegistry) { + this.destinationContainerRegistry = destinationContainerRegistry; + return this; + } + + /** + * Get the configuration property: Configuration of the build. + * + * @return the configuration value. + */ + public BuildConfiguration configuration() { + return this.configuration; + } + + /** + * Set the configuration property: Configuration of the build. + * + * @param configuration the configuration value to set. + * @return the BuildProperties object itself. + */ + public BuildProperties withConfiguration(BuildConfiguration configuration) { + this.configuration = configuration; + return this; + } + + /** + * Get the uploadEndpoint property: Endpoint to which the source code should be uploaded. + * + * @return the uploadEndpoint value. + */ + public String uploadEndpoint() { + return this.uploadEndpoint; + } + + /** + * Get the logStreamEndpoint property: Endpoint from which the build logs can be streamed. + * + * @return the logStreamEndpoint value. + */ + public String logStreamEndpoint() { + return this.logStreamEndpoint; + } + + /** + * Get the tokenEndpoint property: Endpoint to use to retrieve an authentication token for log streaming and + * uploading source code. + * + * @return the tokenEndpoint value. + */ + public String tokenEndpoint() { + return this.tokenEndpoint; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (destinationContainerRegistry() != null) { + destinationContainerRegistry().validate(); + } + if (configuration() != null) { + configuration().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildResourceInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildResourceInner.java new file mode 100644 index 0000000000000..8ce6df5eb20d1 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildResourceInner.java @@ -0,0 +1,160 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.BuildConfiguration; +import com.azure.resourcemanager.appcontainers.models.BuildProvisioningState; +import com.azure.resourcemanager.appcontainers.models.BuildStatus; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Information pertaining to an individual build. + */ +@Fluent +public final class BuildResourceInner extends ProxyResource { + /* + * The resource-specific properties for this resource. + */ + @JsonProperty(value = "properties") + private BuildProperties innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of BuildResourceInner class. + */ + public BuildResourceInner() { + } + + /** + * Get the innerProperties property: The resource-specific properties for this resource. + * + * @return the innerProperties value. + */ + private BuildProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the provisioningState property: Build provisioning state. + * + * @return the provisioningState value. + */ + public BuildProvisioningState provisioningState() { + return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + } + + /** + * Get the buildStatus property: Status of the build once it has been provisioned. + * + * @return the buildStatus value. + */ + public BuildStatus buildStatus() { + return this.innerProperties() == null ? null : this.innerProperties().buildStatus(); + } + + /** + * Get the destinationContainerRegistry property: Container registry that the final image will be uploaded to. + * + * @return the destinationContainerRegistry value. + */ + public ContainerRegistryWithCustomImage destinationContainerRegistry() { + return this.innerProperties() == null ? null : this.innerProperties().destinationContainerRegistry(); + } + + /** + * Set the destinationContainerRegistry property: Container registry that the final image will be uploaded to. + * + * @param destinationContainerRegistry the destinationContainerRegistry value to set. + * @return the BuildResourceInner object itself. + */ + public BuildResourceInner + withDestinationContainerRegistry(ContainerRegistryWithCustomImage destinationContainerRegistry) { + if (this.innerProperties() == null) { + this.innerProperties = new BuildProperties(); + } + this.innerProperties().withDestinationContainerRegistry(destinationContainerRegistry); + return this; + } + + /** + * Get the configuration property: Configuration of the build. + * + * @return the configuration value. + */ + public BuildConfiguration configuration() { + return this.innerProperties() == null ? null : this.innerProperties().configuration(); + } + + /** + * Set the configuration property: Configuration of the build. + * + * @param configuration the configuration value to set. + * @return the BuildResourceInner object itself. + */ + public BuildResourceInner withConfiguration(BuildConfiguration configuration) { + if (this.innerProperties() == null) { + this.innerProperties = new BuildProperties(); + } + this.innerProperties().withConfiguration(configuration); + return this; + } + + /** + * Get the uploadEndpoint property: Endpoint to which the source code should be uploaded. + * + * @return the uploadEndpoint value. + */ + public String uploadEndpoint() { + return this.innerProperties() == null ? null : this.innerProperties().uploadEndpoint(); + } + + /** + * Get the logStreamEndpoint property: Endpoint from which the build logs can be streamed. + * + * @return the logStreamEndpoint value. + */ + public String logStreamEndpoint() { + return this.innerProperties() == null ? null : this.innerProperties().logStreamEndpoint(); + } + + /** + * Get the tokenEndpoint property: Endpoint to use to retrieve an authentication token for log streaming and + * uploading source code. + * + * @return the tokenEndpoint value. + */ + public String tokenEndpoint() { + return this.innerProperties() == null ? null : this.innerProperties().tokenEndpoint(); + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildTokenInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildTokenInner.java new file mode 100644 index 0000000000000..8596ee9730610 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuildTokenInner.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * Build Auth Token. + */ +@Immutable +public final class BuildTokenInner { + /* + * Authentication token. + */ + @JsonProperty(value = "token", access = JsonProperty.Access.WRITE_ONLY) + private String token; + + /* + * Token expiration date. + */ + @JsonProperty(value = "expires", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime expires; + + /** + * Creates an instance of BuildTokenInner class. + */ + public BuildTokenInner() { + } + + /** + * Get the token property: Authentication token. + * + * @return the token value. + */ + public String token() { + return this.token; + } + + /** + * Get the expires property: Token expiration date. + * + * @return the expires value. + */ + public OffsetDateTime expires() { + return this.expires; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderProperties.java new file mode 100644 index 0000000000000..4004bc3451605 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderProperties.java @@ -0,0 +1,112 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.models.BuilderProvisioningState; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The builder properties. + */ +@Fluent +public final class BuilderProperties { + /* + * Provisioning state of a builder resource. + */ + @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) + private BuilderProvisioningState provisioningState; + + /* + * Resource ID of the container apps environment that the builder is associated with. + */ + @JsonProperty(value = "environmentId", required = true) + private String environmentId; + + /* + * List of mappings of container registries and the managed identity used to connect to it. + */ + @JsonProperty(value = "containerRegistries") + private List containerRegistries; + + /** + * Creates an instance of BuilderProperties class. + */ + public BuilderProperties() { + } + + /** + * Get the provisioningState property: Provisioning state of a builder resource. + * + * @return the provisioningState value. + */ + public BuilderProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Get the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @return the environmentId value. + */ + public String environmentId() { + return this.environmentId; + } + + /** + * Set the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @param environmentId the environmentId value to set. + * @return the BuilderProperties object itself. + */ + public BuilderProperties withEnvironmentId(String environmentId) { + this.environmentId = environmentId; + return this; + } + + /** + * Get the containerRegistries property: List of mappings of container registries and the managed identity used to + * connect to it. + * + * @return the containerRegistries value. + */ + public List containerRegistries() { + return this.containerRegistries; + } + + /** + * Set the containerRegistries property: List of mappings of container registries and the managed identity used to + * connect to it. + * + * @param containerRegistries the containerRegistries value to set. + * @return the BuilderProperties object itself. + */ + public BuilderProperties withContainerRegistries(List containerRegistries) { + this.containerRegistries = containerRegistries; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (environmentId() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property environmentId in model BuilderProperties")); + } + if (containerRegistries() != null) { + containerRegistries().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(BuilderProperties.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceInner.java new file mode 100644 index 0000000000000..2d0f6e77c3e08 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceInner.java @@ -0,0 +1,174 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.Resource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.BuilderProvisioningState; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + +/** + * Information about the SourceToCloud builder resource. + */ +@Fluent +public final class BuilderResourceInner extends Resource { + /* + * The resource-specific properties for this resource. + */ + @JsonProperty(value = "properties") + private BuilderProperties innerProperties; + + /* + * The managed service identities assigned to this resource. + */ + @JsonProperty(value = "identity") + private ManagedServiceIdentity identity; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of BuilderResourceInner class. + */ + public BuilderResourceInner() { + } + + /** + * Get the innerProperties property: The resource-specific properties for this resource. + * + * @return the innerProperties value. + */ + private BuilderProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the identity property: The managed service identities assigned to this resource. + * + * @return the identity value. + */ + public ManagedServiceIdentity identity() { + return this.identity; + } + + /** + * Set the identity property: The managed service identities assigned to this resource. + * + * @param identity the identity value to set. + * @return the BuilderResourceInner object itself. + */ + public BuilderResourceInner withIdentity(ManagedServiceIdentity identity) { + this.identity = identity; + return this; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * {@inheritDoc} + */ + @Override + public BuilderResourceInner withLocation(String location) { + super.withLocation(location); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public BuilderResourceInner withTags(Map tags) { + super.withTags(tags); + return this; + } + + /** + * Get the provisioningState property: Provisioning state of a builder resource. + * + * @return the provisioningState value. + */ + public BuilderProvisioningState provisioningState() { + return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + } + + /** + * Get the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @return the environmentId value. + */ + public String environmentId() { + return this.innerProperties() == null ? null : this.innerProperties().environmentId(); + } + + /** + * Set the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @param environmentId the environmentId value to set. + * @return the BuilderResourceInner object itself. + */ + public BuilderResourceInner withEnvironmentId(String environmentId) { + if (this.innerProperties() == null) { + this.innerProperties = new BuilderProperties(); + } + this.innerProperties().withEnvironmentId(environmentId); + return this; + } + + /** + * Get the containerRegistries property: List of mappings of container registries and the managed identity used to + * connect to it. + * + * @return the containerRegistries value. + */ + public List containerRegistries() { + return this.innerProperties() == null ? null : this.innerProperties().containerRegistries(); + } + + /** + * Set the containerRegistries property: List of mappings of container registries and the managed identity used to + * connect to it. + * + * @param containerRegistries the containerRegistries value to set. + * @return the BuilderResourceInner object itself. + */ + public BuilderResourceInner withContainerRegistries(List containerRegistries) { + if (this.innerProperties() == null) { + this.innerProperties = new BuilderProperties(); + } + this.innerProperties().withContainerRegistries(containerRegistries); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + if (identity() != null) { + identity().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceUpdateProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceUpdateProperties.java new file mode 100644 index 0000000000000..8c94193aa04f3 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/BuilderResourceUpdateProperties.java @@ -0,0 +1,56 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The updatable properties of the BuilderResource. + */ +@Fluent +public final class BuilderResourceUpdateProperties { + /* + * Resource ID of the container apps environment that the builder is associated with. + */ + @JsonProperty(value = "environmentId") + private String environmentId; + + /** + * Creates an instance of BuilderResourceUpdateProperties class. + */ + public BuilderResourceUpdateProperties() { + } + + /** + * Get the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @return the environmentId value. + */ + public String environmentId() { + return this.environmentId; + } + + /** + * Set the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @param environmentId the environmentId value to set. + * @return the BuilderResourceUpdateProperties object itself. + */ + public BuilderResourceUpdateProperties withEnvironmentId(String environmentId) { + this.environmentId = environmentId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CertificateInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CertificateInner.java index 49d78985bdcb6..99659aa96409f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CertificateInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CertificateInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ +/** + * Certificate used for Custom Domain bindings of Container Apps in a Managed Environment. + */ @Fluent public final class CertificateInner extends Resource { /* @@ -26,13 +28,15 @@ public final class CertificateInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of CertificateInner class. */ + /** + * Creates an instance of CertificateInner class. + */ public CertificateInner() { } /** * Get the properties property: Certificate resource specific properties. - * + * * @return the properties value. */ public CertificateProperties properties() { @@ -41,7 +45,7 @@ public CertificateProperties properties() { /** * Set the properties property: Certificate resource specific properties. - * + * * @param properties the properties value to set. * @return the CertificateInner object itself. */ @@ -52,21 +56,25 @@ public CertificateInner withProperties(CertificateProperties properties) { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CertificateInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CertificateInner withTags(Map tags) { super.withTags(tags); @@ -75,7 +83,7 @@ public CertificateInner withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CheckNameAvailabilityResponseInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CheckNameAvailabilityResponseInner.java index 1d13506867f28..6e67b71c3fc54 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CheckNameAvailabilityResponseInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CheckNameAvailabilityResponseInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityReason; import com.fasterxml.jackson.annotation.JsonProperty; -/** The check availability result. */ +/** + * The check availability result. + */ @Fluent public final class CheckNameAvailabilityResponseInner { /* @@ -29,13 +31,15 @@ public final class CheckNameAvailabilityResponseInner { @JsonProperty(value = "message") private String message; - /** Creates an instance of CheckNameAvailabilityResponseInner class. */ + /** + * Creates an instance of CheckNameAvailabilityResponseInner class. + */ public CheckNameAvailabilityResponseInner() { } /** * Get the nameAvailable property: Indicates if the resource name is available. - * + * * @return the nameAvailable value. */ public Boolean nameAvailable() { @@ -44,7 +48,7 @@ public Boolean nameAvailable() { /** * Set the nameAvailable property: Indicates if the resource name is available. - * + * * @param nameAvailable the nameAvailable value to set. * @return the CheckNameAvailabilityResponseInner object itself. */ @@ -55,7 +59,7 @@ public CheckNameAvailabilityResponseInner withNameAvailable(Boolean nameAvailabl /** * Get the reason property: The reason why the given name is not available. - * + * * @return the reason value. */ public CheckNameAvailabilityReason reason() { @@ -64,7 +68,7 @@ public CheckNameAvailabilityReason reason() { /** * Set the reason property: The reason why the given name is not available. - * + * * @param reason the reason value to set. * @return the CheckNameAvailabilityResponseInner object itself. */ @@ -75,7 +79,7 @@ public CheckNameAvailabilityResponseInner withReason(CheckNameAvailabilityReason /** * Get the message property: Detailed reason why the given name is available. - * + * * @return the message value. */ public String message() { @@ -84,7 +88,7 @@ public String message() { /** * Set the message property: Detailed reason why the given name is available. - * + * * @param message the message value to set. * @return the CheckNameAvailabilityResponseInner object itself. */ @@ -95,7 +99,7 @@ public CheckNameAvailabilityResponseInner withMessage(String message) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentInner.java index e4892a69f389b..79a0b8bbeb9d7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentInner.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** An environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ +/** + * An environment for Kubernetes cluster specialized for web workloads by Azure App Service. + */ @Fluent public final class ConnectedEnvironmentInner extends Resource { /* @@ -34,13 +36,15 @@ public final class ConnectedEnvironmentInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ConnectedEnvironmentInner class. */ + /** + * Creates an instance of ConnectedEnvironmentInner class. + */ public ConnectedEnvironmentInner() { } /** * Get the extendedLocation property: The complex type of the extended location. - * + * * @return the extendedLocation value. */ public ExtendedLocation extendedLocation() { @@ -49,7 +53,7 @@ public ExtendedLocation extendedLocation() { /** * Set the extendedLocation property: The complex type of the extended location. - * + * * @param extendedLocation the extendedLocation value to set. * @return the ConnectedEnvironmentInner object itself. */ @@ -60,7 +64,7 @@ public ConnectedEnvironmentInner withExtendedLocation(ExtendedLocation extendedL /** * Get the innerProperties property: ConnectedEnvironment resource specific properties. - * + * * @return the innerProperties value. */ private ConnectedEnvironmentProperties innerProperties() { @@ -69,21 +73,25 @@ private ConnectedEnvironmentProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ConnectedEnvironmentInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ConnectedEnvironmentInner withTags(Map tags) { super.withTags(tags); @@ -92,7 +100,7 @@ public ConnectedEnvironmentInner withTags(Map tags) { /** * Get the provisioningState property: Provisioning state of the Kubernetes Environment. - * + * * @return the provisioningState value. */ public ConnectedEnvironmentProvisioningState provisioningState() { @@ -101,7 +109,7 @@ public ConnectedEnvironmentProvisioningState provisioningState() { /** * Get the deploymentErrors property: Any errors that occurred during deployment or deployment validation. - * + * * @return the deploymentErrors value. */ public String deploymentErrors() { @@ -110,7 +118,7 @@ public String deploymentErrors() { /** * Get the defaultDomain property: Default Domain Name for the cluster. - * + * * @return the defaultDomain value. */ public String defaultDomain() { @@ -119,7 +127,7 @@ public String defaultDomain() { /** * Get the staticIp property: Static IP of the connectedEnvironment. - * + * * @return the staticIp value. */ public String staticIp() { @@ -128,7 +136,7 @@ public String staticIp() { /** * Set the staticIp property: Static IP of the connectedEnvironment. - * + * * @param staticIp the staticIp value to set. * @return the ConnectedEnvironmentInner object itself. */ @@ -141,9 +149,9 @@ public ConnectedEnvironmentInner withStaticIp(String staticIp) { } /** - * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @return the daprAIConnectionString value. */ public String daprAIConnectionString() { @@ -151,9 +159,9 @@ public String daprAIConnectionString() { } /** - * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @param daprAIConnectionString the daprAIConnectionString value to set. * @return the ConnectedEnvironmentInner object itself. */ @@ -167,7 +175,7 @@ public ConnectedEnvironmentInner withDaprAIConnectionString(String daprAIConnect /** * Get the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @return the customDomainConfiguration value. */ public CustomDomainConfiguration customDomainConfiguration() { @@ -176,12 +184,12 @@ public CustomDomainConfiguration customDomainConfiguration() { /** * Set the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration the customDomainConfiguration value to set. * @return the ConnectedEnvironmentInner object itself. */ - public ConnectedEnvironmentInner withCustomDomainConfiguration( - CustomDomainConfiguration customDomainConfiguration) { + public ConnectedEnvironmentInner + withCustomDomainConfiguration(CustomDomainConfiguration customDomainConfiguration) { if (this.innerProperties() == null) { this.innerProperties = new ConnectedEnvironmentProperties(); } @@ -191,7 +199,7 @@ public ConnectedEnvironmentInner withCustomDomainConfiguration( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentProperties.java index 28adb95807b1c..f9eb45032dfc5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration; import com.fasterxml.jackson.annotation.JsonProperty; -/** ConnectedEnvironment resource specific properties. */ +/** + * ConnectedEnvironment resource specific properties. + */ @Fluent public final class ConnectedEnvironmentProperties { /* @@ -48,13 +50,15 @@ public final class ConnectedEnvironmentProperties { @JsonProperty(value = "customDomainConfiguration") private CustomDomainConfiguration customDomainConfiguration; - /** Creates an instance of ConnectedEnvironmentProperties class. */ + /** + * Creates an instance of ConnectedEnvironmentProperties class. + */ public ConnectedEnvironmentProperties() { } /** * Get the provisioningState property: Provisioning state of the Kubernetes Environment. - * + * * @return the provisioningState value. */ public ConnectedEnvironmentProvisioningState provisioningState() { @@ -63,7 +67,7 @@ public ConnectedEnvironmentProvisioningState provisioningState() { /** * Get the deploymentErrors property: Any errors that occurred during deployment or deployment validation. - * + * * @return the deploymentErrors value. */ public String deploymentErrors() { @@ -72,7 +76,7 @@ public String deploymentErrors() { /** * Get the defaultDomain property: Default Domain Name for the cluster. - * + * * @return the defaultDomain value. */ public String defaultDomain() { @@ -81,7 +85,7 @@ public String defaultDomain() { /** * Get the staticIp property: Static IP of the connectedEnvironment. - * + * * @return the staticIp value. */ public String staticIp() { @@ -90,7 +94,7 @@ public String staticIp() { /** * Set the staticIp property: Static IP of the connectedEnvironment. - * + * * @param staticIp the staticIp value to set. * @return the ConnectedEnvironmentProperties object itself. */ @@ -100,9 +104,9 @@ public ConnectedEnvironmentProperties withStaticIp(String staticIp) { } /** - * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @return the daprAIConnectionString value. */ public String daprAIConnectionString() { @@ -110,9 +114,9 @@ public String daprAIConnectionString() { } /** - * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @param daprAIConnectionString the daprAIConnectionString value to set. * @return the ConnectedEnvironmentProperties object itself. */ @@ -123,7 +127,7 @@ public ConnectedEnvironmentProperties withDaprAIConnectionString(String daprAICo /** * Get the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @return the customDomainConfiguration value. */ public CustomDomainConfiguration customDomainConfiguration() { @@ -132,19 +136,19 @@ public CustomDomainConfiguration customDomainConfiguration() { /** * Set the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration the customDomainConfiguration value to set. * @return the ConnectedEnvironmentProperties object itself. */ - public ConnectedEnvironmentProperties withCustomDomainConfiguration( - CustomDomainConfiguration customDomainConfiguration) { + public ConnectedEnvironmentProperties + withCustomDomainConfiguration(CustomDomainConfiguration customDomainConfiguration) { this.customDomainConfiguration = customDomainConfiguration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStorageInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStorageInner.java index 4114a1b4e7322..0e97178c9c54f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStorageInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStorageInner.java @@ -6,10 +6,13 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentStorageProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Storage resource for connectedEnvironment. */ +/** + * Storage resource for connectedEnvironment. + */ @Fluent public final class ConnectedEnvironmentStorageInner extends ProxyResource { /* @@ -18,13 +21,21 @@ public final class ConnectedEnvironmentStorageInner extends ProxyResource { @JsonProperty(value = "properties") private ConnectedEnvironmentStorageProperties properties; - /** Creates an instance of ConnectedEnvironmentStorageInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of ConnectedEnvironmentStorageInner class. + */ public ConnectedEnvironmentStorageInner() { } /** * Get the properties property: Storage properties. - * + * * @return the properties value. */ public ConnectedEnvironmentStorageProperties properties() { @@ -33,7 +44,7 @@ public ConnectedEnvironmentStorageProperties properties() { /** * Set the properties property: Storage properties. - * + * * @param properties the properties value to set. * @return the ConnectedEnvironmentStorageInner object itself. */ @@ -42,9 +53,18 @@ public ConnectedEnvironmentStorageInner withProperties(ConnectedEnvironmentStora return this; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStoragesCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStoragesCollectionInner.java index f3fd1e14bfd0d..53bca84c6d613 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStoragesCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ConnectedEnvironmentStoragesCollectionInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of Storage for Environments. */ +/** + * Collection of Storage for Environments. + */ @Fluent public final class ConnectedEnvironmentStoragesCollectionInner { /* @@ -18,13 +20,15 @@ public final class ConnectedEnvironmentStoragesCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of ConnectedEnvironmentStoragesCollectionInner class. */ + /** + * Creates an instance of ConnectedEnvironmentStoragesCollectionInner class. + */ public ConnectedEnvironmentStoragesCollectionInner() { } /** * Get the value property: Collection of storage resources. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Collection of storage resources. - * + * * @param value the value value to set. * @return the ConnectedEnvironmentStoragesCollectionInner object itself. */ @@ -44,15 +48,13 @@ public ConnectedEnvironmentStoragesCollectionInner withValue(List e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenInner.java index 494939f8f2e60..e2661b263dd21 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenInner.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.Map; -/** Container App Auth Token. */ +/** + * Container App Auth Token. + */ @Fluent public final class ContainerAppAuthTokenInner extends Resource { /* @@ -26,13 +28,15 @@ public final class ContainerAppAuthTokenInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ContainerAppAuthTokenInner class. */ + /** + * Creates an instance of ContainerAppAuthTokenInner class. + */ public ContainerAppAuthTokenInner() { } /** * Get the innerProperties property: Container App auth token resource specific properties. - * + * * @return the innerProperties value. */ private ContainerAppAuthTokenProperties innerProperties() { @@ -41,21 +45,25 @@ private ContainerAppAuthTokenProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ContainerAppAuthTokenInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ContainerAppAuthTokenInner withTags(Map tags) { super.withTags(tags); @@ -64,7 +72,7 @@ public ContainerAppAuthTokenInner withTags(Map tags) { /** * Get the token property: Auth token value. - * + * * @return the token value. */ public String token() { @@ -73,7 +81,7 @@ public String token() { /** * Get the expires property: Token expiration date. - * + * * @return the expires value. */ public OffsetDateTime expires() { @@ -82,7 +90,7 @@ public OffsetDateTime expires() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenProperties.java index 18d85126be084..d37fd54241012 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppAuthTokenProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Container App auth token resource specific properties. */ +/** + * Container App auth token resource specific properties. + */ @Immutable public final class ContainerAppAuthTokenProperties { /* @@ -23,13 +25,15 @@ public final class ContainerAppAuthTokenProperties { @JsonProperty(value = "expires", access = JsonProperty.Access.WRITE_ONLY) private OffsetDateTime expires; - /** Creates an instance of ContainerAppAuthTokenProperties class. */ + /** + * Creates an instance of ContainerAppAuthTokenProperties class. + */ public ContainerAppAuthTokenProperties() { } /** * Get the token property: Auth token value. - * + * * @return the token value. */ public String token() { @@ -38,7 +42,7 @@ public String token() { /** * Get the expires property: Token expiration date. - * + * * @return the expires value. */ public OffsetDateTime expires() { @@ -47,7 +51,7 @@ public OffsetDateTime expires() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppInner.java index aa0e8dae7598f..a51478dfb1d99 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppInner.java @@ -16,7 +16,9 @@ import java.util.List; import java.util.Map; -/** Container App. */ +/** + * Container App. + */ @Fluent public final class ContainerAppInner extends Resource { /* @@ -52,13 +54,15 @@ public final class ContainerAppInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ContainerAppInner class. */ + /** + * Creates an instance of ContainerAppInner class. + */ public ContainerAppInner() { } /** * Get the extendedLocation property: The complex type of the extended location. - * + * * @return the extendedLocation value. */ public ExtendedLocation extendedLocation() { @@ -67,7 +71,7 @@ public ExtendedLocation extendedLocation() { /** * Set the extendedLocation property: The complex type of the extended location. - * + * * @param extendedLocation the extendedLocation value to set. * @return the ContainerAppInner object itself. */ @@ -77,9 +81,9 @@ public ContainerAppInner withExtendedLocation(ExtendedLocation extendedLocation) } /** - * Get the identity property: managed identities for the Container App to interact with other Azure services without - * maintaining any secrets or credentials in code. - * + * Get the identity property: managed identities for the Container App to interact with other Azure services + * without maintaining any secrets or credentials in code. + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -87,9 +91,9 @@ public ManagedServiceIdentity identity() { } /** - * Set the identity property: managed identities for the Container App to interact with other Azure services without - * maintaining any secrets or credentials in code. - * + * Set the identity property: managed identities for the Container App to interact with other Azure services + * without maintaining any secrets or credentials in code. + * * @param identity the identity value to set. * @return the ContainerAppInner object itself. */ @@ -99,10 +103,10 @@ public ContainerAppInner withIdentity(ManagedServiceIdentity identity) { } /** - * Get the managedBy property: The fully qualified resource ID of the resource that manages this resource. Indicates - * if this resource is managed by another Azure resource. If this is present, complete mode deployment will not - * delete the resource if it is removed from the template since it is managed by another resource. - * + * Get the managedBy property: The fully qualified resource ID of the resource that manages this resource. + * Indicates if this resource is managed by another Azure resource. If this is present, complete mode deployment + * will not delete the resource if it is removed from the template since it is managed by another resource. + * * @return the managedBy value. */ public String managedBy() { @@ -110,10 +114,10 @@ public String managedBy() { } /** - * Set the managedBy property: The fully qualified resource ID of the resource that manages this resource. Indicates - * if this resource is managed by another Azure resource. If this is present, complete mode deployment will not - * delete the resource if it is removed from the template since it is managed by another resource. - * + * Set the managedBy property: The fully qualified resource ID of the resource that manages this resource. + * Indicates if this resource is managed by another Azure resource. If this is present, complete mode deployment + * will not delete the resource if it is removed from the template since it is managed by another resource. + * * @param managedBy the managedBy value to set. * @return the ContainerAppInner object itself. */ @@ -124,7 +128,7 @@ public ContainerAppInner withManagedBy(String managedBy) { /** * Get the innerProperties property: ContainerApp resource specific properties. - * + * * @return the innerProperties value. */ private ContainerAppProperties innerProperties() { @@ -133,21 +137,25 @@ private ContainerAppProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ContainerAppInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ContainerAppInner withTags(Map tags) { super.withTags(tags); @@ -156,7 +164,7 @@ public ContainerAppInner withTags(Map tags) { /** * Get the provisioningState property: Provisioning state of the Container App. - * + * * @return the provisioningState value. */ public ContainerAppProvisioningState provisioningState() { @@ -165,7 +173,7 @@ public ContainerAppProvisioningState provisioningState() { /** * Get the managedEnvironmentId property: Deprecated. Resource ID of the Container App's environment. - * + * * @return the managedEnvironmentId value. */ public String managedEnvironmentId() { @@ -174,7 +182,7 @@ public String managedEnvironmentId() { /** * Set the managedEnvironmentId property: Deprecated. Resource ID of the Container App's environment. - * + * * @param managedEnvironmentId the managedEnvironmentId value to set. * @return the ContainerAppInner object itself. */ @@ -188,7 +196,7 @@ public ContainerAppInner withManagedEnvironmentId(String managedEnvironmentId) { /** * Get the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ public String environmentId() { @@ -197,7 +205,7 @@ public String environmentId() { /** * Set the environmentId property: Resource ID of environment. - * + * * @param environmentId the environmentId value to set. * @return the ContainerAppInner object itself. */ @@ -211,7 +219,7 @@ public ContainerAppInner withEnvironmentId(String environmentId) { /** * Get the workloadProfileName property: Workload profile name to pin for container app execution. - * + * * @return the workloadProfileName value. */ public String workloadProfileName() { @@ -220,7 +228,7 @@ public String workloadProfileName() { /** * Set the workloadProfileName property: Workload profile name to pin for container app execution. - * + * * @param workloadProfileName the workloadProfileName value to set. * @return the ContainerAppInner object itself. */ @@ -234,7 +242,7 @@ public ContainerAppInner withWorkloadProfileName(String workloadProfileName) { /** * Get the latestRevisionName property: Name of the latest revision of the Container App. - * + * * @return the latestRevisionName value. */ public String latestRevisionName() { @@ -243,7 +251,7 @@ public String latestRevisionName() { /** * Get the latestReadyRevisionName property: Name of the latest ready revision of the Container App. - * + * * @return the latestReadyRevisionName value. */ public String latestReadyRevisionName() { @@ -252,7 +260,7 @@ public String latestReadyRevisionName() { /** * Get the latestRevisionFqdn property: Fully Qualified Domain Name of the latest revision of the Container App. - * + * * @return the latestRevisionFqdn value. */ public String latestRevisionFqdn() { @@ -261,7 +269,7 @@ public String latestRevisionFqdn() { /** * Get the customDomainVerificationId property: Id used to verify domain name ownership. - * + * * @return the customDomainVerificationId value. */ public String customDomainVerificationId() { @@ -270,7 +278,7 @@ public String customDomainVerificationId() { /** * Get the configuration property: Non versioned Container App configuration properties. - * + * * @return the configuration value. */ public Configuration configuration() { @@ -279,7 +287,7 @@ public Configuration configuration() { /** * Set the configuration property: Non versioned Container App configuration properties. - * + * * @param configuration the configuration value to set. * @return the ContainerAppInner object itself. */ @@ -293,7 +301,7 @@ public ContainerAppInner withConfiguration(Configuration configuration) { /** * Get the template property: Container App versioned application definition. - * + * * @return the template value. */ public Template template() { @@ -302,7 +310,7 @@ public Template template() { /** * Set the template property: Container App versioned application definition. - * + * * @param template the template value to set. * @return the ContainerAppInner object itself. */ @@ -316,7 +324,7 @@ public ContainerAppInner withTemplate(Template template) { /** * Get the outboundIpAddresses property: Outbound IP Addresses for container app. - * + * * @return the outboundIpAddresses value. */ public List outboundIpAddresses() { @@ -325,7 +333,7 @@ public List outboundIpAddresses() { /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the container app. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -334,7 +342,7 @@ public String eventStreamEndpoint() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppJobExecutionsInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppJobExecutionsInner.java index 80a392e0d7c78..be02df97a4b80 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppJobExecutionsInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppJobExecutionsInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App executions collection ARM resource. */ +/** + * Container App executions collection ARM resource. + */ @Fluent public final class ContainerAppJobExecutionsInner { /* @@ -24,13 +26,15 @@ public final class ContainerAppJobExecutionsInner { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ContainerAppJobExecutionsInner class. */ + /** + * Creates an instance of ContainerAppJobExecutionsInner class. + */ public ContainerAppJobExecutionsInner() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the ContainerAppJobExecutionsInner object itself. */ @@ -50,7 +54,7 @@ public ContainerAppJobExecutionsInner withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,15 +63,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model ContainerAppJobExecutionsInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property value in model ContainerAppJobExecutionsInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppProperties.java index 1429f8f7e5f4c..312d98bd745c5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ContainerAppProperties.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** ContainerApp resource specific properties. */ +/** + * ContainerApp resource specific properties. + */ @Fluent public final class ContainerAppProperties { /* @@ -86,13 +88,15 @@ public final class ContainerAppProperties { @JsonProperty(value = "eventStreamEndpoint", access = JsonProperty.Access.WRITE_ONLY) private String eventStreamEndpoint; - /** Creates an instance of ContainerAppProperties class. */ + /** + * Creates an instance of ContainerAppProperties class. + */ public ContainerAppProperties() { } /** * Get the provisioningState property: Provisioning state of the Container App. - * + * * @return the provisioningState value. */ public ContainerAppProvisioningState provisioningState() { @@ -101,7 +105,7 @@ public ContainerAppProvisioningState provisioningState() { /** * Get the managedEnvironmentId property: Deprecated. Resource ID of the Container App's environment. - * + * * @return the managedEnvironmentId value. */ public String managedEnvironmentId() { @@ -110,7 +114,7 @@ public String managedEnvironmentId() { /** * Set the managedEnvironmentId property: Deprecated. Resource ID of the Container App's environment. - * + * * @param managedEnvironmentId the managedEnvironmentId value to set. * @return the ContainerAppProperties object itself. */ @@ -121,7 +125,7 @@ public ContainerAppProperties withManagedEnvironmentId(String managedEnvironment /** * Get the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ public String environmentId() { @@ -130,7 +134,7 @@ public String environmentId() { /** * Set the environmentId property: Resource ID of environment. - * + * * @param environmentId the environmentId value to set. * @return the ContainerAppProperties object itself. */ @@ -141,7 +145,7 @@ public ContainerAppProperties withEnvironmentId(String environmentId) { /** * Get the workloadProfileName property: Workload profile name to pin for container app execution. - * + * * @return the workloadProfileName value. */ public String workloadProfileName() { @@ -150,7 +154,7 @@ public String workloadProfileName() { /** * Set the workloadProfileName property: Workload profile name to pin for container app execution. - * + * * @param workloadProfileName the workloadProfileName value to set. * @return the ContainerAppProperties object itself. */ @@ -161,7 +165,7 @@ public ContainerAppProperties withWorkloadProfileName(String workloadProfileName /** * Get the latestRevisionName property: Name of the latest revision of the Container App. - * + * * @return the latestRevisionName value. */ public String latestRevisionName() { @@ -170,7 +174,7 @@ public String latestRevisionName() { /** * Get the latestReadyRevisionName property: Name of the latest ready revision of the Container App. - * + * * @return the latestReadyRevisionName value. */ public String latestReadyRevisionName() { @@ -179,7 +183,7 @@ public String latestReadyRevisionName() { /** * Get the latestRevisionFqdn property: Fully Qualified Domain Name of the latest revision of the Container App. - * + * * @return the latestRevisionFqdn value. */ public String latestRevisionFqdn() { @@ -188,7 +192,7 @@ public String latestRevisionFqdn() { /** * Get the customDomainVerificationId property: Id used to verify domain name ownership. - * + * * @return the customDomainVerificationId value. */ public String customDomainVerificationId() { @@ -197,7 +201,7 @@ public String customDomainVerificationId() { /** * Get the configuration property: Non versioned Container App configuration properties. - * + * * @return the configuration value. */ public Configuration configuration() { @@ -206,7 +210,7 @@ public Configuration configuration() { /** * Set the configuration property: Non versioned Container App configuration properties. - * + * * @param configuration the configuration value to set. * @return the ContainerAppProperties object itself. */ @@ -217,7 +221,7 @@ public ContainerAppProperties withConfiguration(Configuration configuration) { /** * Get the template property: Container App versioned application definition. - * + * * @return the template value. */ public Template template() { @@ -226,7 +230,7 @@ public Template template() { /** * Set the template property: Container App versioned application definition. - * + * * @param template the template value to set. * @return the ContainerAppProperties object itself. */ @@ -237,7 +241,7 @@ public ContainerAppProperties withTemplate(Template template) { /** * Get the outboundIpAddresses property: Outbound IP Addresses for container app. - * + * * @return the outboundIpAddresses value. */ public List outboundIpAddresses() { @@ -246,7 +250,7 @@ public List outboundIpAddresses() { /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the container app. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -255,7 +259,7 @@ public String eventStreamEndpoint() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CustomHostnameAnalysisResultInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CustomHostnameAnalysisResultInner.java index 58e0b8275ae2c..4f00be750b53c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CustomHostnameAnalysisResultInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/CustomHostnameAnalysisResultInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Custom domain analysis. */ +/** + * Custom domain analysis. + */ @Fluent public final class CustomHostnameAnalysisResultInner { /* @@ -87,13 +89,15 @@ public final class CustomHostnameAnalysisResultInner { @JsonProperty(value = "alternateTxtRecords") private List alternateTxtRecords; - /** Creates an instance of CustomHostnameAnalysisResultInner class. */ + /** + * Creates an instance of CustomHostnameAnalysisResultInner class. + */ public CustomHostnameAnalysisResultInner() { } /** * Get the hostname property: Host name that was analyzed. - * + * * @return the hostname value. */ public String hostname() { @@ -103,7 +107,7 @@ public String hostname() { /** * Get the isHostnameAlreadyVerified property: <code>true</code> if hostname is already verified; * otherwise, <code>false</code>. - * + * * @return the isHostnameAlreadyVerified value. */ public Boolean isHostnameAlreadyVerified() { @@ -112,7 +116,7 @@ public Boolean isHostnameAlreadyVerified() { /** * Get the customDomainVerificationTest property: DNS verification test result. - * + * * @return the customDomainVerificationTest value. */ public DnsVerificationTestResult customDomainVerificationTest() { @@ -121,7 +125,7 @@ public DnsVerificationTestResult customDomainVerificationTest() { /** * Get the customDomainVerificationFailureInfo property: Raw failure information if DNS verification fails. - * + * * @return the customDomainVerificationFailureInfo value. */ public CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo customDomainVerificationFailureInfo() { @@ -131,7 +135,7 @@ public CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo customDom /** * Get the hasConflictOnManagedEnvironment property: <code>true</code> if there is a conflict on the * Container App's managed environment; otherwise, <code>false</code>. - * + * * @return the hasConflictOnManagedEnvironment value. */ public Boolean hasConflictOnManagedEnvironment() { @@ -139,9 +143,9 @@ public Boolean hasConflictOnManagedEnvironment() { } /** - * Get the conflictWithEnvironmentCustomDomain property: <code>true</code> if there is a conflict on the - * Container App's managed environment level custom domain; otherwise, <code>false</code>. - * + * Get the conflictWithEnvironmentCustomDomain property: <code>true</code> if there is a conflict on + * the Container App's managed environment level custom domain; otherwise, <code>false</code>. + * * @return the conflictWithEnvironmentCustomDomain value. */ public Boolean conflictWithEnvironmentCustomDomain() { @@ -151,7 +155,7 @@ public Boolean conflictWithEnvironmentCustomDomain() { /** * Get the conflictingContainerAppResourceId property: Name of the conflicting Container App on the Managed * Environment if it's within the same subscription. - * + * * @return the conflictingContainerAppResourceId value. */ public String conflictingContainerAppResourceId() { @@ -160,7 +164,7 @@ public String conflictingContainerAppResourceId() { /** * Get the cNameRecords property: CName records visible for this hostname. - * + * * @return the cNameRecords value. */ public List cNameRecords() { @@ -169,7 +173,7 @@ public List cNameRecords() { /** * Set the cNameRecords property: CName records visible for this hostname. - * + * * @param cNameRecords the cNameRecords value to set. * @return the CustomHostnameAnalysisResultInner object itself. */ @@ -180,7 +184,7 @@ public CustomHostnameAnalysisResultInner withCNameRecords(List cNameReco /** * Get the txtRecords property: TXT records visible for this hostname. - * + * * @return the txtRecords value. */ public List txtRecords() { @@ -189,7 +193,7 @@ public List txtRecords() { /** * Set the txtRecords property: TXT records visible for this hostname. - * + * * @param txtRecords the txtRecords value to set. * @return the CustomHostnameAnalysisResultInner object itself. */ @@ -200,7 +204,7 @@ public CustomHostnameAnalysisResultInner withTxtRecords(List txtRecords) /** * Get the aRecords property: A records visible for this hostname. - * + * * @return the aRecords value. */ public List aRecords() { @@ -209,7 +213,7 @@ public List aRecords() { /** * Set the aRecords property: A records visible for this hostname. - * + * * @param aRecords the aRecords value to set. * @return the CustomHostnameAnalysisResultInner object itself. */ @@ -220,7 +224,7 @@ public CustomHostnameAnalysisResultInner withARecords(List aRecords) { /** * Get the alternateCNameRecords property: Alternate CName records visible for this hostname. - * + * * @return the alternateCNameRecords value. */ public List alternateCNameRecords() { @@ -229,7 +233,7 @@ public List alternateCNameRecords() { /** * Set the alternateCNameRecords property: Alternate CName records visible for this hostname. - * + * * @param alternateCNameRecords the alternateCNameRecords value to set. * @return the CustomHostnameAnalysisResultInner object itself. */ @@ -240,7 +244,7 @@ public CustomHostnameAnalysisResultInner withAlternateCNameRecords(List /** * Get the alternateTxtRecords property: Alternate TXT records visible for this hostname. - * + * * @return the alternateTxtRecords value. */ public List alternateTxtRecords() { @@ -249,7 +253,7 @@ public List alternateTxtRecords() { /** * Set the alternateTxtRecords property: Alternate TXT records visible for this hostname. - * + * * @param alternateTxtRecords the alternateTxtRecords value to set. * @return the CustomHostnameAnalysisResultInner object itself. */ @@ -260,7 +264,7 @@ public CustomHostnameAnalysisResultInner withAlternateTxtRecords(List al /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentInner.java index bb777312434d7..a9dbee9bcb878 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentInner.java @@ -6,12 +6,16 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding; import com.azure.resourcemanager.appcontainers.models.DaprMetadata; import com.azure.resourcemanager.appcontainers.models.Secret; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Dapr Component. */ +/** + * Dapr Component. + */ @Fluent public final class DaprComponentInner extends ProxyResource { /* @@ -20,22 +24,39 @@ public final class DaprComponentInner extends ProxyResource { @JsonProperty(value = "properties") private DaprComponentProperties innerProperties; - /** Creates an instance of DaprComponentInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of DaprComponentInner class. + */ public DaprComponentInner() { } /** * Get the innerProperties property: Dapr Component resource specific properties. - * + * * @return the innerProperties value. */ private DaprComponentProperties innerProperties() { return this.innerProperties; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Get the componentType property: Component type. - * + * * @return the componentType value. */ public String componentType() { @@ -44,7 +65,7 @@ public String componentType() { /** * Set the componentType property: Component type. - * + * * @param componentType the componentType value to set. * @return the DaprComponentInner object itself. */ @@ -58,7 +79,7 @@ public DaprComponentInner withComponentType(String componentType) { /** * Get the version property: Component version. - * + * * @return the version value. */ public String version() { @@ -67,7 +88,7 @@ public String version() { /** * Set the version property: Component version. - * + * * @param version the version value to set. * @return the DaprComponentInner object itself. */ @@ -81,7 +102,7 @@ public DaprComponentInner withVersion(String version) { /** * Get the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @return the ignoreErrors value. */ public Boolean ignoreErrors() { @@ -90,7 +111,7 @@ public Boolean ignoreErrors() { /** * Set the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @param ignoreErrors the ignoreErrors value to set. * @return the DaprComponentInner object itself. */ @@ -104,7 +125,7 @@ public DaprComponentInner withIgnoreErrors(Boolean ignoreErrors) { /** * Get the initTimeout property: Initialization timeout. - * + * * @return the initTimeout value. */ public String initTimeout() { @@ -113,7 +134,7 @@ public String initTimeout() { /** * Set the initTimeout property: Initialization timeout. - * + * * @param initTimeout the initTimeout value to set. * @return the DaprComponentInner object itself. */ @@ -127,7 +148,7 @@ public DaprComponentInner withInitTimeout(String initTimeout) { /** * Get the secrets property: Collection of secrets used by a Dapr component. - * + * * @return the secrets value. */ public List secrets() { @@ -136,7 +157,7 @@ public List secrets() { /** * Set the secrets property: Collection of secrets used by a Dapr component. - * + * * @param secrets the secrets value to set. * @return the DaprComponentInner object itself. */ @@ -150,7 +171,7 @@ public DaprComponentInner withSecrets(List secrets) { /** * Get the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @return the secretStoreComponent value. */ public String secretStoreComponent() { @@ -159,7 +180,7 @@ public String secretStoreComponent() { /** * Set the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @param secretStoreComponent the secretStoreComponent value to set. * @return the DaprComponentInner object itself. */ @@ -173,7 +194,7 @@ public DaprComponentInner withSecretStoreComponent(String secretStoreComponent) /** * Get the metadata property: Component metadata. - * + * * @return the metadata value. */ public List metadata() { @@ -182,7 +203,7 @@ public List metadata() { /** * Set the metadata property: Component metadata. - * + * * @param metadata the metadata value to set. * @return the DaprComponentInner object itself. */ @@ -196,7 +217,7 @@ public DaprComponentInner withMetadata(List metadata) { /** * Get the scopes property: Names of container apps that can use this Dapr component. - * + * * @return the scopes value. */ public List scopes() { @@ -205,7 +226,7 @@ public List scopes() { /** * Set the scopes property: Names of container apps that can use this Dapr component. - * + * * @param scopes the scopes value to set. * @return the DaprComponentInner object itself. */ @@ -217,9 +238,32 @@ public DaprComponentInner withScopes(List scopes) { return this; } + /** + * Get the serviceComponentBind property: List of container app services that are bound to the Dapr component. + * + * @return the serviceComponentBind value. + */ + public List serviceComponentBind() { + return this.innerProperties() == null ? null : this.innerProperties().serviceComponentBind(); + } + + /** + * Set the serviceComponentBind property: List of container app services that are bound to the Dapr component. + * + * @param serviceComponentBind the serviceComponentBind value to set. + * @return the DaprComponentInner object itself. + */ + public DaprComponentInner withServiceComponentBind(List serviceComponentBind) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprComponentProperties(); + } + this.innerProperties().withServiceComponentBind(serviceComponentBind); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentProperties.java index 5ea576bbf7406..dfe98a8c51d3b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentProperties.java @@ -5,12 +5,15 @@ package com.azure.resourcemanager.appcontainers.fluent.models; import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding; import com.azure.resourcemanager.appcontainers.models.DaprMetadata; import com.azure.resourcemanager.appcontainers.models.Secret; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Dapr Component resource specific properties. */ +/** + * Dapr Component resource specific properties. + */ @Fluent public final class DaprComponentProperties { /* @@ -61,13 +64,21 @@ public final class DaprComponentProperties { @JsonProperty(value = "scopes") private List scopes; - /** Creates an instance of DaprComponentProperties class. */ + /* + * List of container app services that are bound to the Dapr component + */ + @JsonProperty(value = "serviceComponentBind") + private List serviceComponentBind; + + /** + * Creates an instance of DaprComponentProperties class. + */ public DaprComponentProperties() { } /** * Get the componentType property: Component type. - * + * * @return the componentType value. */ public String componentType() { @@ -76,7 +87,7 @@ public String componentType() { /** * Set the componentType property: Component type. - * + * * @param componentType the componentType value to set. * @return the DaprComponentProperties object itself. */ @@ -87,7 +98,7 @@ public DaprComponentProperties withComponentType(String componentType) { /** * Get the version property: Component version. - * + * * @return the version value. */ public String version() { @@ -96,7 +107,7 @@ public String version() { /** * Set the version property: Component version. - * + * * @param version the version value to set. * @return the DaprComponentProperties object itself. */ @@ -107,7 +118,7 @@ public DaprComponentProperties withVersion(String version) { /** * Get the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @return the ignoreErrors value. */ public Boolean ignoreErrors() { @@ -116,7 +127,7 @@ public Boolean ignoreErrors() { /** * Set the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @param ignoreErrors the ignoreErrors value to set. * @return the DaprComponentProperties object itself. */ @@ -127,7 +138,7 @@ public DaprComponentProperties withIgnoreErrors(Boolean ignoreErrors) { /** * Get the initTimeout property: Initialization timeout. - * + * * @return the initTimeout value. */ public String initTimeout() { @@ -136,7 +147,7 @@ public String initTimeout() { /** * Set the initTimeout property: Initialization timeout. - * + * * @param initTimeout the initTimeout value to set. * @return the DaprComponentProperties object itself. */ @@ -147,7 +158,7 @@ public DaprComponentProperties withInitTimeout(String initTimeout) { /** * Get the secrets property: Collection of secrets used by a Dapr component. - * + * * @return the secrets value. */ public List secrets() { @@ -156,7 +167,7 @@ public List secrets() { /** * Set the secrets property: Collection of secrets used by a Dapr component. - * + * * @param secrets the secrets value to set. * @return the DaprComponentProperties object itself. */ @@ -167,7 +178,7 @@ public DaprComponentProperties withSecrets(List secrets) { /** * Get the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @return the secretStoreComponent value. */ public String secretStoreComponent() { @@ -176,7 +187,7 @@ public String secretStoreComponent() { /** * Set the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @param secretStoreComponent the secretStoreComponent value to set. * @return the DaprComponentProperties object itself. */ @@ -187,7 +198,7 @@ public DaprComponentProperties withSecretStoreComponent(String secretStoreCompon /** * Get the metadata property: Component metadata. - * + * * @return the metadata value. */ public List metadata() { @@ -196,7 +207,7 @@ public List metadata() { /** * Set the metadata property: Component metadata. - * + * * @param metadata the metadata value to set. * @return the DaprComponentProperties object itself. */ @@ -207,7 +218,7 @@ public DaprComponentProperties withMetadata(List metadata) { /** * Get the scopes property: Names of container apps that can use this Dapr component. - * + * * @return the scopes value. */ public List scopes() { @@ -216,7 +227,7 @@ public List scopes() { /** * Set the scopes property: Names of container apps that can use this Dapr component. - * + * * @param scopes the scopes value to set. * @return the DaprComponentProperties object itself. */ @@ -225,9 +236,29 @@ public DaprComponentProperties withScopes(List scopes) { return this; } + /** + * Get the serviceComponentBind property: List of container app services that are bound to the Dapr component. + * + * @return the serviceComponentBind value. + */ + public List serviceComponentBind() { + return this.serviceComponentBind; + } + + /** + * Set the serviceComponentBind property: List of container app services that are bound to the Dapr component. + * + * @param serviceComponentBind the serviceComponentBind value to set. + * @return the DaprComponentProperties object itself. + */ + public DaprComponentProperties withServiceComponentBind(List serviceComponentBind) { + this.serviceComponentBind = serviceComponentBind; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -237,5 +268,8 @@ public void validate() { if (metadata() != null) { metadata().forEach(e -> e.validate()); } + if (serviceComponentBind() != null) { + serviceComponentBind().forEach(e -> e.validate()); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyInner.java new file mode 100644 index 0000000000000..168e535807492 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyInner.java @@ -0,0 +1,112 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy. + */ +@Fluent +public final class DaprComponentResiliencyPolicyInner extends ProxyResource { + /* + * Dapr Component Resiliency Policy resource specific properties + */ + @JsonProperty(value = "properties") + private DaprComponentResiliencyPolicyProperties innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of DaprComponentResiliencyPolicyInner class. + */ + public DaprComponentResiliencyPolicyInner() { + } + + /** + * Get the innerProperties property: Dapr Component Resiliency Policy resource specific properties. + * + * @return the innerProperties value. + */ + private DaprComponentResiliencyPolicyProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @return the inboundPolicy value. + */ + public DaprComponentResiliencyPolicyConfiguration inboundPolicy() { + return this.innerProperties() == null ? null : this.innerProperties().inboundPolicy(); + } + + /** + * Set the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @param inboundPolicy the inboundPolicy value to set. + * @return the DaprComponentResiliencyPolicyInner object itself. + */ + public DaprComponentResiliencyPolicyInner + withInboundPolicy(DaprComponentResiliencyPolicyConfiguration inboundPolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprComponentResiliencyPolicyProperties(); + } + this.innerProperties().withInboundPolicy(inboundPolicy); + return this; + } + + /** + * Get the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @return the outboundPolicy value. + */ + public DaprComponentResiliencyPolicyConfiguration outboundPolicy() { + return this.innerProperties() == null ? null : this.innerProperties().outboundPolicy(); + } + + /** + * Set the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @param outboundPolicy the outboundPolicy value to set. + * @return the DaprComponentResiliencyPolicyInner object itself. + */ + public DaprComponentResiliencyPolicyInner + withOutboundPolicy(DaprComponentResiliencyPolicyConfiguration outboundPolicy) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprComponentResiliencyPolicyProperties(); + } + this.innerProperties().withOutboundPolicy(outboundPolicy); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyProperties.java new file mode 100644 index 0000000000000..5dbc7f9b16a03 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprComponentResiliencyPolicyProperties.java @@ -0,0 +1,89 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy resource specific properties. + */ +@Fluent +public final class DaprComponentResiliencyPolicyProperties { + /* + * The optional inbound component resiliency policy configuration + */ + @JsonProperty(value = "inboundPolicy") + private DaprComponentResiliencyPolicyConfiguration inboundPolicy; + + /* + * The optional outbound component resiliency policy configuration + */ + @JsonProperty(value = "outboundPolicy") + private DaprComponentResiliencyPolicyConfiguration outboundPolicy; + + /** + * Creates an instance of DaprComponentResiliencyPolicyProperties class. + */ + public DaprComponentResiliencyPolicyProperties() { + } + + /** + * Get the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @return the inboundPolicy value. + */ + public DaprComponentResiliencyPolicyConfiguration inboundPolicy() { + return this.inboundPolicy; + } + + /** + * Set the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @param inboundPolicy the inboundPolicy value to set. + * @return the DaprComponentResiliencyPolicyProperties object itself. + */ + public DaprComponentResiliencyPolicyProperties + withInboundPolicy(DaprComponentResiliencyPolicyConfiguration inboundPolicy) { + this.inboundPolicy = inboundPolicy; + return this; + } + + /** + * Get the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @return the outboundPolicy value. + */ + public DaprComponentResiliencyPolicyConfiguration outboundPolicy() { + return this.outboundPolicy; + } + + /** + * Set the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @param outboundPolicy the outboundPolicy value to set. + * @return the DaprComponentResiliencyPolicyProperties object itself. + */ + public DaprComponentResiliencyPolicyProperties + withOutboundPolicy(DaprComponentResiliencyPolicyConfiguration outboundPolicy) { + this.outboundPolicy = outboundPolicy; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (inboundPolicy() != null) { + inboundPolicy().validate(); + } + if (outboundPolicy() != null) { + outboundPolicy().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSecretsCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSecretsCollectionInner.java index 72cdffb688012..4f4b9a166bdeb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSecretsCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSecretsCollectionInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Dapr component Secrets Collection for ListSecrets Action. */ +/** + * Dapr component Secrets Collection for ListSecrets Action. + */ @Fluent public final class DaprSecretsCollectionInner { /* @@ -19,13 +21,15 @@ public final class DaprSecretsCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of DaprSecretsCollectionInner class. */ + /** + * Creates an instance of DaprSecretsCollectionInner class. + */ public DaprSecretsCollectionInner() { } /** * Get the value property: Collection of secrets used by a Dapr component. - * + * * @return the value value. */ public List value() { @@ -34,7 +38,7 @@ public List value() { /** * Set the value property: Collection of secrets used by a Dapr component. - * + * * @param value the value value to set. * @return the DaprSecretsCollectionInner object itself. */ @@ -45,15 +49,13 @@ public DaprSecretsCollectionInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model DaprSecretsCollectionInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DaprSecretsCollectionInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionInner.java new file mode 100644 index 0000000000000..0939471b441f2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionInner.java @@ -0,0 +1,228 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + +/** + * Dapr PubSub Event Subscription. + */ +@Fluent +public final class DaprSubscriptionInner extends ProxyResource { + /* + * Dapr PubSub Event Subscription resource specific properties + */ + @JsonProperty(value = "properties") + private DaprSubscriptionProperties innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of DaprSubscriptionInner class. + */ + public DaprSubscriptionInner() { + } + + /** + * Get the innerProperties property: Dapr PubSub Event Subscription resource specific properties. + * + * @return the innerProperties value. + */ + private DaprSubscriptionProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the pubsubName property: Dapr PubSub component name. + * + * @return the pubsubName value. + */ + public String pubsubName() { + return this.innerProperties() == null ? null : this.innerProperties().pubsubName(); + } + + /** + * Set the pubsubName property: Dapr PubSub component name. + * + * @param pubsubName the pubsubName value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withPubsubName(String pubsubName) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withPubsubName(pubsubName); + return this; + } + + /** + * Get the topic property: Topic name. + * + * @return the topic value. + */ + public String topic() { + return this.innerProperties() == null ? null : this.innerProperties().topic(); + } + + /** + * Set the topic property: Topic name. + * + * @param topic the topic value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withTopic(String topic) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withTopic(topic); + return this; + } + + /** + * Get the deadLetterTopic property: Deadletter topic name. + * + * @return the deadLetterTopic value. + */ + public String deadLetterTopic() { + return this.innerProperties() == null ? null : this.innerProperties().deadLetterTopic(); + } + + /** + * Set the deadLetterTopic property: Deadletter topic name. + * + * @param deadLetterTopic the deadLetterTopic value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withDeadLetterTopic(String deadLetterTopic) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withDeadLetterTopic(deadLetterTopic); + return this; + } + + /** + * Get the routes property: Subscription routes. + * + * @return the routes value. + */ + public DaprSubscriptionRoutes routes() { + return this.innerProperties() == null ? null : this.innerProperties().routes(); + } + + /** + * Set the routes property: Subscription routes. + * + * @param routes the routes value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withRoutes(DaprSubscriptionRoutes routes) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withRoutes(routes); + return this; + } + + /** + * Get the scopes property: Application scopes to restrict the subscription to specific apps. + * + * @return the scopes value. + */ + public List scopes() { + return this.innerProperties() == null ? null : this.innerProperties().scopes(); + } + + /** + * Set the scopes property: Application scopes to restrict the subscription to specific apps. + * + * @param scopes the scopes value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withScopes(List scopes) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withScopes(scopes); + return this; + } + + /** + * Get the metadata property: Subscription metadata. + * + * @return the metadata value. + */ + public Map metadata() { + return this.innerProperties() == null ? null : this.innerProperties().metadata(); + } + + /** + * Set the metadata property: Subscription metadata. + * + * @param metadata the metadata value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withMetadata(Map metadata) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withMetadata(metadata); + return this; + } + + /** + * Get the bulkSubscribe property: Bulk subscription options. + * + * @return the bulkSubscribe value. + */ + public DaprSubscriptionBulkSubscribeOptions bulkSubscribe() { + return this.innerProperties() == null ? null : this.innerProperties().bulkSubscribe(); + } + + /** + * Set the bulkSubscribe property: Bulk subscription options. + * + * @param bulkSubscribe the bulkSubscribe value to set. + * @return the DaprSubscriptionInner object itself. + */ + public DaprSubscriptionInner withBulkSubscribe(DaprSubscriptionBulkSubscribeOptions bulkSubscribe) { + if (this.innerProperties() == null) { + this.innerProperties = new DaprSubscriptionProperties(); + } + this.innerProperties().withBulkSubscribe(bulkSubscribe); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionProperties.java new file mode 100644 index 0000000000000..c3fbb8bc0aa60 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DaprSubscriptionProperties.java @@ -0,0 +1,222 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + +/** + * Dapr PubSub Event Subscription resource specific properties. + */ +@Fluent +public final class DaprSubscriptionProperties { + /* + * Dapr PubSub component name + */ + @JsonProperty(value = "pubsubName") + private String pubsubName; + + /* + * Topic name + */ + @JsonProperty(value = "topic") + private String topic; + + /* + * Deadletter topic name + */ + @JsonProperty(value = "deadLetterTopic") + private String deadLetterTopic; + + /* + * Subscription routes + */ + @JsonProperty(value = "routes") + private DaprSubscriptionRoutes routes; + + /* + * Application scopes to restrict the subscription to specific apps. + */ + @JsonProperty(value = "scopes") + private List scopes; + + /* + * Subscription metadata + */ + @JsonProperty(value = "metadata") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map metadata; + + /* + * Bulk subscription options + */ + @JsonProperty(value = "bulkSubscribe") + private DaprSubscriptionBulkSubscribeOptions bulkSubscribe; + + /** + * Creates an instance of DaprSubscriptionProperties class. + */ + public DaprSubscriptionProperties() { + } + + /** + * Get the pubsubName property: Dapr PubSub component name. + * + * @return the pubsubName value. + */ + public String pubsubName() { + return this.pubsubName; + } + + /** + * Set the pubsubName property: Dapr PubSub component name. + * + * @param pubsubName the pubsubName value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withPubsubName(String pubsubName) { + this.pubsubName = pubsubName; + return this; + } + + /** + * Get the topic property: Topic name. + * + * @return the topic value. + */ + public String topic() { + return this.topic; + } + + /** + * Set the topic property: Topic name. + * + * @param topic the topic value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withTopic(String topic) { + this.topic = topic; + return this; + } + + /** + * Get the deadLetterTopic property: Deadletter topic name. + * + * @return the deadLetterTopic value. + */ + public String deadLetterTopic() { + return this.deadLetterTopic; + } + + /** + * Set the deadLetterTopic property: Deadletter topic name. + * + * @param deadLetterTopic the deadLetterTopic value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withDeadLetterTopic(String deadLetterTopic) { + this.deadLetterTopic = deadLetterTopic; + return this; + } + + /** + * Get the routes property: Subscription routes. + * + * @return the routes value. + */ + public DaprSubscriptionRoutes routes() { + return this.routes; + } + + /** + * Set the routes property: Subscription routes. + * + * @param routes the routes value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withRoutes(DaprSubscriptionRoutes routes) { + this.routes = routes; + return this; + } + + /** + * Get the scopes property: Application scopes to restrict the subscription to specific apps. + * + * @return the scopes value. + */ + public List scopes() { + return this.scopes; + } + + /** + * Set the scopes property: Application scopes to restrict the subscription to specific apps. + * + * @param scopes the scopes value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withScopes(List scopes) { + this.scopes = scopes; + return this; + } + + /** + * Get the metadata property: Subscription metadata. + * + * @return the metadata value. + */ + public Map metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Subscription metadata. + * + * @param metadata the metadata value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withMetadata(Map metadata) { + this.metadata = metadata; + return this; + } + + /** + * Get the bulkSubscribe property: Bulk subscription options. + * + * @return the bulkSubscribe value. + */ + public DaprSubscriptionBulkSubscribeOptions bulkSubscribe() { + return this.bulkSubscribe; + } + + /** + * Set the bulkSubscribe property: Bulk subscription options. + * + * @param bulkSubscribe the bulkSubscribe value to set. + * @return the DaprSubscriptionProperties object itself. + */ + public DaprSubscriptionProperties withBulkSubscribe(DaprSubscriptionBulkSubscribeOptions bulkSubscribe) { + this.bulkSubscribe = bulkSubscribe; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (routes() != null) { + routes().validate(); + } + if (bulkSubscribe() != null) { + bulkSubscribe().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsCollectionInner.java index 88c3ba58530ad..3467b799bcbba 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsCollectionInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Diagnostics data collection for a resource. */ +/** + * Diagnostics data collection for a resource. + */ @Fluent public final class DiagnosticsCollectionInner { /* @@ -24,13 +26,15 @@ public final class DiagnosticsCollectionInner { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DiagnosticsCollectionInner class. */ + /** + * Creates an instance of DiagnosticsCollectionInner class. + */ public DiagnosticsCollectionInner() { } /** * Get the value property: Collection of diagnostic data. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: Collection of diagnostic data. - * + * * @param value the value value to set. * @return the DiagnosticsCollectionInner object itself. */ @@ -50,7 +54,7 @@ public DiagnosticsCollectionInner withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,15 +63,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model DiagnosticsCollectionInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DiagnosticsCollectionInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsInner.java index 8bd178e635bb0..7139bc06e6d9c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DiagnosticsInner.java @@ -6,10 +6,13 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.DiagnosticsProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Diagnostics data for a resource. */ +/** + * Diagnostics data for a resource. + */ @Fluent public final class DiagnosticsInner extends ProxyResource { /* @@ -18,13 +21,21 @@ public final class DiagnosticsInner extends ProxyResource { @JsonProperty(value = "properties") private DiagnosticsProperties properties; - /** Creates an instance of DiagnosticsInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of DiagnosticsInner class. + */ public DiagnosticsInner() { } /** * Get the properties property: Diagnostics resource specific properties. - * + * * @return the properties value. */ public DiagnosticsProperties properties() { @@ -33,7 +44,7 @@ public DiagnosticsProperties properties() { /** * Set the properties property: Diagnostics resource specific properties. - * + * * @param properties the properties value to set. * @return the DiagnosticsInner object itself. */ @@ -42,9 +53,18 @@ public DiagnosticsInner withProperties(DiagnosticsProperties properties) { return this; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentInner.java new file mode 100644 index 0000000000000..bd1d51f605340 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentInner.java @@ -0,0 +1,146 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentProvisioningState; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * .NET Component. + */ +@Fluent +public final class DotNetComponentInner extends ProxyResource { + /* + * .NET Component resource specific properties + */ + @JsonProperty(value = "properties") + private DotNetComponentProperties innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of DotNetComponentInner class. + */ + public DotNetComponentInner() { + } + + /** + * Get the innerProperties property: .NET Component resource specific properties. + * + * @return the innerProperties value. + */ + private DotNetComponentProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the componentType property: Type of the .NET Component. + * + * @return the componentType value. + */ + public DotNetComponentType componentType() { + return this.innerProperties() == null ? null : this.innerProperties().componentType(); + } + + /** + * Set the componentType property: Type of the .NET Component. + * + * @param componentType the componentType value to set. + * @return the DotNetComponentInner object itself. + */ + public DotNetComponentInner withComponentType(DotNetComponentType componentType) { + if (this.innerProperties() == null) { + this.innerProperties = new DotNetComponentProperties(); + } + this.innerProperties().withComponentType(componentType); + return this; + } + + /** + * Get the provisioningState property: Provisioning state of the .NET Component. + * + * @return the provisioningState value. + */ + public DotNetComponentProvisioningState provisioningState() { + return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + } + + /** + * Get the configurations property: List of .NET Components configuration properties. + * + * @return the configurations value. + */ + public List configurations() { + return this.innerProperties() == null ? null : this.innerProperties().configurations(); + } + + /** + * Set the configurations property: List of .NET Components configuration properties. + * + * @param configurations the configurations value to set. + * @return the DotNetComponentInner object itself. + */ + public DotNetComponentInner withConfigurations(List configurations) { + if (this.innerProperties() == null) { + this.innerProperties = new DotNetComponentProperties(); + } + this.innerProperties().withConfigurations(configurations); + return this; + } + + /** + * Get the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @return the serviceBinds value. + */ + public List serviceBinds() { + return this.innerProperties() == null ? null : this.innerProperties().serviceBinds(); + } + + /** + * Set the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @param serviceBinds the serviceBinds value to set. + * @return the DotNetComponentInner object itself. + */ + public DotNetComponentInner withServiceBinds(List serviceBinds) { + if (this.innerProperties() == null) { + this.innerProperties = new DotNetComponentProperties(); + } + this.innerProperties().withServiceBinds(serviceBinds); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentProperties.java new file mode 100644 index 0000000000000..ca41f342548f7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/DotNetComponentProperties.java @@ -0,0 +1,132 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentProvisioningState; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * .NET Component resource specific properties. + */ +@Fluent +public final class DotNetComponentProperties { + /* + * Type of the .NET Component. + */ + @JsonProperty(value = "componentType") + private DotNetComponentType componentType; + + /* + * Provisioning state of the .NET Component. + */ + @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) + private DotNetComponentProvisioningState provisioningState; + + /* + * List of .NET Components configuration properties + */ + @JsonProperty(value = "configurations") + private List configurations; + + /* + * List of .NET Components that are bound to the .NET component + */ + @JsonProperty(value = "serviceBinds") + private List serviceBinds; + + /** + * Creates an instance of DotNetComponentProperties class. + */ + public DotNetComponentProperties() { + } + + /** + * Get the componentType property: Type of the .NET Component. + * + * @return the componentType value. + */ + public DotNetComponentType componentType() { + return this.componentType; + } + + /** + * Set the componentType property: Type of the .NET Component. + * + * @param componentType the componentType value to set. + * @return the DotNetComponentProperties object itself. + */ + public DotNetComponentProperties withComponentType(DotNetComponentType componentType) { + this.componentType = componentType; + return this; + } + + /** + * Get the provisioningState property: Provisioning state of the .NET Component. + * + * @return the provisioningState value. + */ + public DotNetComponentProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Get the configurations property: List of .NET Components configuration properties. + * + * @return the configurations value. + */ + public List configurations() { + return this.configurations; + } + + /** + * Set the configurations property: List of .NET Components configuration properties. + * + * @param configurations the configurations value to set. + * @return the DotNetComponentProperties object itself. + */ + public DotNetComponentProperties withConfigurations(List configurations) { + this.configurations = configurations; + return this; + } + + /** + * Get the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @return the serviceBinds value. + */ + public List serviceBinds() { + return this.serviceBinds; + } + + /** + * Set the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @param serviceBinds the serviceBinds value to set. + * @return the DotNetComponentProperties object itself. + */ + public DotNetComponentProperties withServiceBinds(List serviceBinds) { + this.serviceBinds = serviceBinds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (configurations() != null) { + configurations().forEach(e -> e.validate()); + } + if (serviceBinds() != null) { + serviceBinds().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenInner.java index d756ff5d716b4..781f861022464 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenInner.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.Map; -/** Environment Auth Token. */ +/** + * Environment Auth Token. + */ @Fluent public final class EnvironmentAuthTokenInner extends Resource { /* @@ -26,13 +28,15 @@ public final class EnvironmentAuthTokenInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of EnvironmentAuthTokenInner class. */ + /** + * Creates an instance of EnvironmentAuthTokenInner class. + */ public EnvironmentAuthTokenInner() { } /** * Get the innerProperties property: Environment auth token resource specific properties. - * + * * @return the innerProperties value. */ private EnvironmentAuthTokenProperties innerProperties() { @@ -41,21 +45,25 @@ private EnvironmentAuthTokenProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public EnvironmentAuthTokenInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public EnvironmentAuthTokenInner withTags(Map tags) { super.withTags(tags); @@ -64,7 +72,7 @@ public EnvironmentAuthTokenInner withTags(Map tags) { /** * Get the token property: Auth token value. - * + * * @return the token value. */ public String token() { @@ -73,7 +81,7 @@ public String token() { /** * Get the expires property: Token expiration date. - * + * * @return the expires value. */ public OffsetDateTime expires() { @@ -82,7 +90,7 @@ public OffsetDateTime expires() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenProperties.java index 775e90da2e84a..59ca82a463d3f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/EnvironmentAuthTokenProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Environment auth token resource specific properties. */ +/** + * Environment auth token resource specific properties. + */ @Immutable public final class EnvironmentAuthTokenProperties { /* @@ -23,13 +25,15 @@ public final class EnvironmentAuthTokenProperties { @JsonProperty(value = "expires", access = JsonProperty.Access.WRITE_ONLY) private OffsetDateTime expires; - /** Creates an instance of EnvironmentAuthTokenProperties class. */ + /** + * Creates an instance of EnvironmentAuthTokenProperties class. + */ public EnvironmentAuthTokenProperties() { } /** * Get the token property: Auth token value. - * + * * @return the token value. */ public String token() { @@ -38,7 +42,7 @@ public String token() { /** * Get the expires property: Token expiration date. - * + * * @return the expires value. */ public OffsetDateTime expires() { @@ -47,7 +51,7 @@ public OffsetDateTime expires() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HeaderMatchMatch.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HeaderMatchMatch.java new file mode 100644 index 0000000000000..29d7763f51875 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HeaderMatchMatch.java @@ -0,0 +1,132 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Type of match to perform. + */ +@Fluent +public final class HeaderMatchMatch { + /* + * Exact value of the header + */ + @JsonProperty(value = "exactMatch") + private String exactMatch; + + /* + * Prefix value of the header + */ + @JsonProperty(value = "prefixMatch") + private String prefixMatch; + + /* + * Suffix value of the header + */ + @JsonProperty(value = "suffixMatch") + private String suffixMatch; + + /* + * Regex value of the header + */ + @JsonProperty(value = "regexMatch") + private String regexMatch; + + /** + * Creates an instance of HeaderMatchMatch class. + */ + public HeaderMatchMatch() { + } + + /** + * Get the exactMatch property: Exact value of the header. + * + * @return the exactMatch value. + */ + public String exactMatch() { + return this.exactMatch; + } + + /** + * Set the exactMatch property: Exact value of the header. + * + * @param exactMatch the exactMatch value to set. + * @return the HeaderMatchMatch object itself. + */ + public HeaderMatchMatch withExactMatch(String exactMatch) { + this.exactMatch = exactMatch; + return this; + } + + /** + * Get the prefixMatch property: Prefix value of the header. + * + * @return the prefixMatch value. + */ + public String prefixMatch() { + return this.prefixMatch; + } + + /** + * Set the prefixMatch property: Prefix value of the header. + * + * @param prefixMatch the prefixMatch value to set. + * @return the HeaderMatchMatch object itself. + */ + public HeaderMatchMatch withPrefixMatch(String prefixMatch) { + this.prefixMatch = prefixMatch; + return this; + } + + /** + * Get the suffixMatch property: Suffix value of the header. + * + * @return the suffixMatch value. + */ + public String suffixMatch() { + return this.suffixMatch; + } + + /** + * Set the suffixMatch property: Suffix value of the header. + * + * @param suffixMatch the suffixMatch value to set. + * @return the HeaderMatchMatch object itself. + */ + public HeaderMatchMatch withSuffixMatch(String suffixMatch) { + this.suffixMatch = suffixMatch; + return this; + } + + /** + * Get the regexMatch property: Regex value of the header. + * + * @return the regexMatch value. + */ + public String regexMatch() { + return this.regexMatch; + } + + /** + * Set the regexMatch property: Regex value of the header. + * + * @param regexMatch the regexMatch value to set. + * @return the HeaderMatchMatch object itself. + */ + public HeaderMatchMatch withRegexMatch(String regexMatch) { + this.regexMatch = regexMatch; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyMatches.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyMatches.java new file mode 100644 index 0000000000000..341adb4b7d7de --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyMatches.java @@ -0,0 +1,111 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.HeaderMatch; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Conditions that must be met for a request to be retried. + */ +@Fluent +public final class HttpRetryPolicyMatches { + /* + * Headers that must be present for a request to be retried + */ + @JsonProperty(value = "headers") + private List headers; + + /* + * Additional http status codes that can trigger a retry + */ + @JsonProperty(value = "httpStatusCodes") + private List httpStatusCodes; + + /* + * Errors that can trigger a retry + */ + @JsonProperty(value = "errors") + private List errors; + + /** + * Creates an instance of HttpRetryPolicyMatches class. + */ + public HttpRetryPolicyMatches() { + } + + /** + * Get the headers property: Headers that must be present for a request to be retried. + * + * @return the headers value. + */ + public List headers() { + return this.headers; + } + + /** + * Set the headers property: Headers that must be present for a request to be retried. + * + * @param headers the headers value to set. + * @return the HttpRetryPolicyMatches object itself. + */ + public HttpRetryPolicyMatches withHeaders(List headers) { + this.headers = headers; + return this; + } + + /** + * Get the httpStatusCodes property: Additional http status codes that can trigger a retry. + * + * @return the httpStatusCodes value. + */ + public List httpStatusCodes() { + return this.httpStatusCodes; + } + + /** + * Set the httpStatusCodes property: Additional http status codes that can trigger a retry. + * + * @param httpStatusCodes the httpStatusCodes value to set. + * @return the HttpRetryPolicyMatches object itself. + */ + public HttpRetryPolicyMatches withHttpStatusCodes(List httpStatusCodes) { + this.httpStatusCodes = httpStatusCodes; + return this; + } + + /** + * Get the errors property: Errors that can trigger a retry. + * + * @return the errors value. + */ + public List errors() { + return this.errors; + } + + /** + * Set the errors property: Errors that can trigger a retry. + * + * @param errors the errors value to set. + * @return the HttpRetryPolicyMatches object itself. + */ + public HttpRetryPolicyMatches withErrors(List errors) { + this.errors = errors; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (headers() != null) { + headers().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyRetryBackOff.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyRetryBackOff.java new file mode 100644 index 0000000000000..a56702256e359 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/HttpRetryPolicyRetryBackOff.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Settings for retry backoff characteristics. + */ +@Fluent +public final class HttpRetryPolicyRetryBackOff { + /* + * Initial delay, in milliseconds, before retrying a request + */ + @JsonProperty(value = "initialDelayInMilliseconds") + private Long initialDelayInMilliseconds; + + /* + * Maximum interval, in milliseconds, between retries + */ + @JsonProperty(value = "maxIntervalInMilliseconds") + private Long maxIntervalInMilliseconds; + + /** + * Creates an instance of HttpRetryPolicyRetryBackOff class. + */ + public HttpRetryPolicyRetryBackOff() { + } + + /** + * Get the initialDelayInMilliseconds property: Initial delay, in milliseconds, before retrying a request. + * + * @return the initialDelayInMilliseconds value. + */ + public Long initialDelayInMilliseconds() { + return this.initialDelayInMilliseconds; + } + + /** + * Set the initialDelayInMilliseconds property: Initial delay, in milliseconds, before retrying a request. + * + * @param initialDelayInMilliseconds the initialDelayInMilliseconds value to set. + * @return the HttpRetryPolicyRetryBackOff object itself. + */ + public HttpRetryPolicyRetryBackOff withInitialDelayInMilliseconds(Long initialDelayInMilliseconds) { + this.initialDelayInMilliseconds = initialDelayInMilliseconds; + return this; + } + + /** + * Get the maxIntervalInMilliseconds property: Maximum interval, in milliseconds, between retries. + * + * @return the maxIntervalInMilliseconds value. + */ + public Long maxIntervalInMilliseconds() { + return this.maxIntervalInMilliseconds; + } + + /** + * Set the maxIntervalInMilliseconds property: Maximum interval, in milliseconds, between retries. + * + * @param maxIntervalInMilliseconds the maxIntervalInMilliseconds value to set. + * @return the HttpRetryPolicyRetryBackOff object itself. + */ + public HttpRetryPolicyRetryBackOff withMaxIntervalInMilliseconds(Long maxIntervalInMilliseconds) { + this.maxIntervalInMilliseconds = maxIntervalInMilliseconds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentInner.java new file mode 100644 index 0000000000000..cedf0eaf85e1f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentInner.java @@ -0,0 +1,146 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentProvisioningState; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Java Component. + */ +@Fluent +public final class JavaComponentInner extends ProxyResource { + /* + * Java Component resource specific properties + */ + @JsonProperty(value = "properties") + private JavaComponentProperties innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of JavaComponentInner class. + */ + public JavaComponentInner() { + } + + /** + * Get the innerProperties property: Java Component resource specific properties. + * + * @return the innerProperties value. + */ + private JavaComponentProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the componentType property: Type of the Java Component. + * + * @return the componentType value. + */ + public JavaComponentType componentType() { + return this.innerProperties() == null ? null : this.innerProperties().componentType(); + } + + /** + * Set the componentType property: Type of the Java Component. + * + * @param componentType the componentType value to set. + * @return the JavaComponentInner object itself. + */ + public JavaComponentInner withComponentType(JavaComponentType componentType) { + if (this.innerProperties() == null) { + this.innerProperties = new JavaComponentProperties(); + } + this.innerProperties().withComponentType(componentType); + return this; + } + + /** + * Get the provisioningState property: Provisioning state of the Java Component. + * + * @return the provisioningState value. + */ + public JavaComponentProvisioningState provisioningState() { + return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + } + + /** + * Get the configurations property: List of Java Components configuration properties. + * + * @return the configurations value. + */ + public List configurations() { + return this.innerProperties() == null ? null : this.innerProperties().configurations(); + } + + /** + * Set the configurations property: List of Java Components configuration properties. + * + * @param configurations the configurations value to set. + * @return the JavaComponentInner object itself. + */ + public JavaComponentInner withConfigurations(List configurations) { + if (this.innerProperties() == null) { + this.innerProperties = new JavaComponentProperties(); + } + this.innerProperties().withConfigurations(configurations); + return this; + } + + /** + * Get the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @return the serviceBinds value. + */ + public List serviceBinds() { + return this.innerProperties() == null ? null : this.innerProperties().serviceBinds(); + } + + /** + * Set the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @param serviceBinds the serviceBinds value to set. + * @return the JavaComponentInner object itself. + */ + public JavaComponentInner withServiceBinds(List serviceBinds) { + if (this.innerProperties() == null) { + this.innerProperties = new JavaComponentProperties(); + } + this.innerProperties().withServiceBinds(serviceBinds); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentProperties.java new file mode 100644 index 0000000000000..7b0b5c56ad9e6 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JavaComponentProperties.java @@ -0,0 +1,132 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentProvisioningState; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Java Component resource specific properties. + */ +@Fluent +public final class JavaComponentProperties { + /* + * Type of the Java Component. + */ + @JsonProperty(value = "componentType") + private JavaComponentType componentType; + + /* + * Provisioning state of the Java Component. + */ + @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) + private JavaComponentProvisioningState provisioningState; + + /* + * List of Java Components configuration properties + */ + @JsonProperty(value = "configurations") + private List configurations; + + /* + * List of Java Components that are bound to the Java component + */ + @JsonProperty(value = "serviceBinds") + private List serviceBinds; + + /** + * Creates an instance of JavaComponentProperties class. + */ + public JavaComponentProperties() { + } + + /** + * Get the componentType property: Type of the Java Component. + * + * @return the componentType value. + */ + public JavaComponentType componentType() { + return this.componentType; + } + + /** + * Set the componentType property: Type of the Java Component. + * + * @param componentType the componentType value to set. + * @return the JavaComponentProperties object itself. + */ + public JavaComponentProperties withComponentType(JavaComponentType componentType) { + this.componentType = componentType; + return this; + } + + /** + * Get the provisioningState property: Provisioning state of the Java Component. + * + * @return the provisioningState value. + */ + public JavaComponentProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Get the configurations property: List of Java Components configuration properties. + * + * @return the configurations value. + */ + public List configurations() { + return this.configurations; + } + + /** + * Set the configurations property: List of Java Components configuration properties. + * + * @param configurations the configurations value to set. + * @return the JavaComponentProperties object itself. + */ + public JavaComponentProperties withConfigurations(List configurations) { + this.configurations = configurations; + return this; + } + + /** + * Get the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @return the serviceBinds value. + */ + public List serviceBinds() { + return this.serviceBinds; + } + + /** + * Set the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @param serviceBinds the serviceBinds value to set. + * @return the JavaComponentProperties object itself. + */ + public JavaComponentProperties withServiceBinds(List serviceBinds) { + this.serviceBinds = serviceBinds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (configurations() != null) { + configurations().forEach(e -> e.validate()); + } + if (serviceBinds() != null) { + serviceBinds().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionBaseInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionBaseInner.java index e8415f46c8706..53a7de1913f8f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionBaseInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionBaseInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App's Job execution name. */ +/** + * Container App's Job execution name. + */ @Fluent public final class JobExecutionBaseInner { /* @@ -22,13 +24,15 @@ public final class JobExecutionBaseInner { @JsonProperty(value = "id") private String id; - /** Creates an instance of JobExecutionBaseInner class. */ + /** + * Creates an instance of JobExecutionBaseInner class. + */ public JobExecutionBaseInner() { } /** * Get the name property: Job execution name. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Set the name property: Job execution name. - * + * * @param name the name value to set. * @return the JobExecutionBaseInner object itself. */ @@ -48,7 +52,7 @@ public JobExecutionBaseInner withName(String name) { /** * Get the id property: Job execution Id. - * + * * @return the id value. */ public String id() { @@ -57,7 +61,7 @@ public String id() { /** * Set the id property: Job execution Id. - * + * * @param id the id value to set. * @return the JobExecutionBaseInner object itself. */ @@ -68,7 +72,7 @@ public JobExecutionBaseInner withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionInner.java index dbebc66d46a42..d407d72fc16fb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Container Apps Job execution. */ +/** + * Container Apps Job execution. + */ @Fluent public final class JobExecutionInner { /* @@ -26,42 +28,26 @@ public final class JobExecutionInner { private String id; /* - * Job Type. + * Job execution type */ @JsonProperty(value = "type") private String type; /* - * Current running State of the job + * Container Apps Job execution specific properties. */ - @JsonProperty(value = "status", access = JsonProperty.Access.WRITE_ONLY) - private JobExecutionRunningState status; + @JsonProperty(value = "properties") + private JobExecutionProperties innerProperties; - /* - * Job execution start time. - */ - @JsonProperty(value = "startTime") - private OffsetDateTime startTime; - - /* - * Job execution end time. - */ - @JsonProperty(value = "endTime") - private OffsetDateTime endTime; - - /* - * Job's execution container. + /** + * Creates an instance of JobExecutionInner class. */ - @JsonProperty(value = "template") - private JobExecutionTemplate template; - - /** Creates an instance of JobExecutionInner class. */ public JobExecutionInner() { } /** * Get the name property: Job execution Name. - * + * * @return the name value. */ public String name() { @@ -70,7 +56,7 @@ public String name() { /** * Set the name property: Job execution Name. - * + * * @param name the name value to set. * @return the JobExecutionInner object itself. */ @@ -81,7 +67,7 @@ public JobExecutionInner withName(String name) { /** * Get the id property: Job execution Id. - * + * * @return the id value. */ public String id() { @@ -90,7 +76,7 @@ public String id() { /** * Set the id property: Job execution Id. - * + * * @param id the id value to set. * @return the JobExecutionInner object itself. */ @@ -100,8 +86,8 @@ public JobExecutionInner withId(String id) { } /** - * Get the type property: Job Type. - * + * Get the type property: Job execution type. + * * @return the type value. */ public String type() { @@ -109,8 +95,8 @@ public String type() { } /** - * Set the type property: Job Type. - * + * Set the type property: Job execution type. + * * @param type the type value to set. * @return the JobExecutionInner object itself. */ @@ -119,83 +105,101 @@ public JobExecutionInner withType(String type) { return this; } + /** + * Get the innerProperties property: Container Apps Job execution specific properties. + * + * @return the innerProperties value. + */ + private JobExecutionProperties innerProperties() { + return this.innerProperties; + } + /** * Get the status property: Current running State of the job. - * + * * @return the status value. */ public JobExecutionRunningState status() { - return this.status; + return this.innerProperties() == null ? null : this.innerProperties().status(); } /** * Get the startTime property: Job execution start time. - * + * * @return the startTime value. */ public OffsetDateTime startTime() { - return this.startTime; + return this.innerProperties() == null ? null : this.innerProperties().startTime(); } /** * Set the startTime property: Job execution start time. - * + * * @param startTime the startTime value to set. * @return the JobExecutionInner object itself. */ public JobExecutionInner withStartTime(OffsetDateTime startTime) { - this.startTime = startTime; + if (this.innerProperties() == null) { + this.innerProperties = new JobExecutionProperties(); + } + this.innerProperties().withStartTime(startTime); return this; } /** * Get the endTime property: Job execution end time. - * + * * @return the endTime value. */ public OffsetDateTime endTime() { - return this.endTime; + return this.innerProperties() == null ? null : this.innerProperties().endTime(); } /** * Set the endTime property: Job execution end time. - * + * * @param endTime the endTime value to set. * @return the JobExecutionInner object itself. */ public JobExecutionInner withEndTime(OffsetDateTime endTime) { - this.endTime = endTime; + if (this.innerProperties() == null) { + this.innerProperties = new JobExecutionProperties(); + } + this.innerProperties().withEndTime(endTime); return this; } /** * Get the template property: Job's execution container. - * + * * @return the template value. */ public JobExecutionTemplate template() { - return this.template; + return this.innerProperties() == null ? null : this.innerProperties().template(); } /** * Set the template property: Job's execution container. - * + * * @param template the template value to set. * @return the JobExecutionInner object itself. */ public JobExecutionInner withTemplate(JobExecutionTemplate template) { - this.template = template; + if (this.innerProperties() == null) { + this.innerProperties = new JobExecutionProperties(); + } + this.innerProperties().withTemplate(template); return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (template() != null) { - template().validate(); + if (innerProperties() != null) { + innerProperties().validate(); } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionProperties.java new file mode 100644 index 0000000000000..38038116fad5c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobExecutionProperties.java @@ -0,0 +1,127 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.JobExecutionRunningState; +import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * Container Apps Job execution specific properties. + */ +@Fluent +public final class JobExecutionProperties { + /* + * Current running State of the job + */ + @JsonProperty(value = "status", access = JsonProperty.Access.WRITE_ONLY) + private JobExecutionRunningState status; + + /* + * Job execution start time. + */ + @JsonProperty(value = "startTime") + private OffsetDateTime startTime; + + /* + * Job execution end time. + */ + @JsonProperty(value = "endTime") + private OffsetDateTime endTime; + + /* + * Job's execution container. + */ + @JsonProperty(value = "template") + private JobExecutionTemplate template; + + /** + * Creates an instance of JobExecutionProperties class. + */ + public JobExecutionProperties() { + } + + /** + * Get the status property: Current running State of the job. + * + * @return the status value. + */ + public JobExecutionRunningState status() { + return this.status; + } + + /** + * Get the startTime property: Job execution start time. + * + * @return the startTime value. + */ + public OffsetDateTime startTime() { + return this.startTime; + } + + /** + * Set the startTime property: Job execution start time. + * + * @param startTime the startTime value to set. + * @return the JobExecutionProperties object itself. + */ + public JobExecutionProperties withStartTime(OffsetDateTime startTime) { + this.startTime = startTime; + return this; + } + + /** + * Get the endTime property: Job execution end time. + * + * @return the endTime value. + */ + public OffsetDateTime endTime() { + return this.endTime; + } + + /** + * Set the endTime property: Job execution end time. + * + * @param endTime the endTime value to set. + * @return the JobExecutionProperties object itself. + */ + public JobExecutionProperties withEndTime(OffsetDateTime endTime) { + this.endTime = endTime; + return this; + } + + /** + * Get the template property: Job's execution container. + * + * @return the template value. + */ + public JobExecutionTemplate template() { + return this.template; + } + + /** + * Set the template property: Job's execution container. + * + * @param template the template value to set. + * @return the JobExecutionProperties object itself. + */ + public JobExecutionProperties withTemplate(JobExecutionTemplate template) { + this.template = template; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (template() != null) { + template().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobInner.java index c8aa364ef3f42..1f480d8be4250 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobInner.java @@ -7,6 +7,7 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.ExtendedLocation; import com.azure.resourcemanager.appcontainers.models.JobConfiguration; import com.azure.resourcemanager.appcontainers.models.JobProvisioningState; import com.azure.resourcemanager.appcontainers.models.JobTemplate; @@ -15,9 +16,17 @@ import java.util.List; import java.util.Map; -/** Container App Job. */ +/** + * Container App Job. + */ @Fluent public final class JobInner extends Resource { + /* + * The complex type of the extended location. + */ + @JsonProperty(value = "extendedLocation") + private ExtendedLocation extendedLocation; + /* * Managed identities needed by a container app job to interact with other Azure services to not maintain any * secrets or credentials in code. @@ -37,14 +46,36 @@ public final class JobInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of JobInner class. */ + /** + * Creates an instance of JobInner class. + */ public JobInner() { } /** - * Get the identity property: Managed identities needed by a container app job to interact with other Azure services - * to not maintain any secrets or credentials in code. - * + * Get the extendedLocation property: The complex type of the extended location. + * + * @return the extendedLocation value. + */ + public ExtendedLocation extendedLocation() { + return this.extendedLocation; + } + + /** + * Set the extendedLocation property: The complex type of the extended location. + * + * @param extendedLocation the extendedLocation value to set. + * @return the JobInner object itself. + */ + public JobInner withExtendedLocation(ExtendedLocation extendedLocation) { + this.extendedLocation = extendedLocation; + return this; + } + + /** + * Get the identity property: Managed identities needed by a container app job to interact with other Azure + * services to not maintain any secrets or credentials in code. + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -52,9 +83,9 @@ public ManagedServiceIdentity identity() { } /** - * Set the identity property: Managed identities needed by a container app job to interact with other Azure services - * to not maintain any secrets or credentials in code. - * + * Set the identity property: Managed identities needed by a container app job to interact with other Azure + * services to not maintain any secrets or credentials in code. + * * @param identity the identity value to set. * @return the JobInner object itself. */ @@ -65,7 +96,7 @@ public JobInner withIdentity(ManagedServiceIdentity identity) { /** * Get the innerProperties property: Container Apps Job resource specific properties. - * + * * @return the innerProperties value. */ private JobProperties innerProperties() { @@ -74,21 +105,25 @@ private JobProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public JobInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public JobInner withTags(Map tags) { super.withTags(tags); @@ -97,7 +132,7 @@ public JobInner withTags(Map tags) { /** * Get the provisioningState property: Provisioning state of the Container Apps Job. - * + * * @return the provisioningState value. */ public JobProvisioningState provisioningState() { @@ -106,7 +141,7 @@ public JobProvisioningState provisioningState() { /** * Get the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ public String environmentId() { @@ -115,7 +150,7 @@ public String environmentId() { /** * Set the environmentId property: Resource ID of environment. - * + * * @param environmentId the environmentId value to set. * @return the JobInner object itself. */ @@ -129,7 +164,7 @@ public JobInner withEnvironmentId(String environmentId) { /** * Get the workloadProfileName property: Workload profile name to pin for container apps job execution. - * + * * @return the workloadProfileName value. */ public String workloadProfileName() { @@ -138,7 +173,7 @@ public String workloadProfileName() { /** * Set the workloadProfileName property: Workload profile name to pin for container apps job execution. - * + * * @param workloadProfileName the workloadProfileName value to set. * @return the JobInner object itself. */ @@ -152,7 +187,7 @@ public JobInner withWorkloadProfileName(String workloadProfileName) { /** * Get the configuration property: Container Apps Job configuration properties. - * + * * @return the configuration value. */ public JobConfiguration configuration() { @@ -161,7 +196,7 @@ public JobConfiguration configuration() { /** * Set the configuration property: Container Apps Job configuration properties. - * + * * @param configuration the configuration value to set. * @return the JobInner object itself. */ @@ -175,7 +210,7 @@ public JobInner withConfiguration(JobConfiguration configuration) { /** * Get the template property: Container Apps job definition. - * + * * @return the template value. */ public JobTemplate template() { @@ -184,7 +219,7 @@ public JobTemplate template() { /** * Set the template property: Container Apps job definition. - * + * * @param template the template value to set. * @return the JobInner object itself. */ @@ -198,7 +233,7 @@ public JobInner withTemplate(JobTemplate template) { /** * Get the outboundIpAddresses property: Outbound IP Addresses of a container apps job. - * + * * @return the outboundIpAddresses value. */ public List outboundIpAddresses() { @@ -207,7 +242,7 @@ public List outboundIpAddresses() { /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the container apps job. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -216,10 +251,13 @@ public String eventStreamEndpoint() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (extendedLocation() != null) { + extendedLocation().validate(); + } if (identity() != null) { identity().validate(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobProperties.java index 7787e3d8147b6..f1e4e6838a0a4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobProperties.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container Apps Job resource specific properties. */ +/** + * Container Apps Job resource specific properties. + */ @Fluent public final class JobProperties { /* @@ -56,13 +58,15 @@ public final class JobProperties { @JsonProperty(value = "eventStreamEndpoint", access = JsonProperty.Access.WRITE_ONLY) private String eventStreamEndpoint; - /** Creates an instance of JobProperties class. */ + /** + * Creates an instance of JobProperties class. + */ public JobProperties() { } /** * Get the provisioningState property: Provisioning state of the Container Apps Job. - * + * * @return the provisioningState value. */ public JobProvisioningState provisioningState() { @@ -71,7 +75,7 @@ public JobProvisioningState provisioningState() { /** * Get the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ public String environmentId() { @@ -80,7 +84,7 @@ public String environmentId() { /** * Set the environmentId property: Resource ID of environment. - * + * * @param environmentId the environmentId value to set. * @return the JobProperties object itself. */ @@ -91,7 +95,7 @@ public JobProperties withEnvironmentId(String environmentId) { /** * Get the workloadProfileName property: Workload profile name to pin for container apps job execution. - * + * * @return the workloadProfileName value. */ public String workloadProfileName() { @@ -100,7 +104,7 @@ public String workloadProfileName() { /** * Set the workloadProfileName property: Workload profile name to pin for container apps job execution. - * + * * @param workloadProfileName the workloadProfileName value to set. * @return the JobProperties object itself. */ @@ -111,7 +115,7 @@ public JobProperties withWorkloadProfileName(String workloadProfileName) { /** * Get the configuration property: Container Apps Job configuration properties. - * + * * @return the configuration value. */ public JobConfiguration configuration() { @@ -120,7 +124,7 @@ public JobConfiguration configuration() { /** * Set the configuration property: Container Apps Job configuration properties. - * + * * @param configuration the configuration value to set. * @return the JobProperties object itself. */ @@ -131,7 +135,7 @@ public JobProperties withConfiguration(JobConfiguration configuration) { /** * Get the template property: Container Apps job definition. - * + * * @return the template value. */ public JobTemplate template() { @@ -140,7 +144,7 @@ public JobTemplate template() { /** * Set the template property: Container Apps job definition. - * + * * @param template the template value to set. * @return the JobProperties object itself. */ @@ -151,7 +155,7 @@ public JobProperties withTemplate(JobTemplate template) { /** * Get the outboundIpAddresses property: Outbound IP Addresses of a container apps job. - * + * * @return the outboundIpAddresses value. */ public List outboundIpAddresses() { @@ -160,7 +164,7 @@ public List outboundIpAddresses() { /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the container apps job. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -169,7 +173,7 @@ public String eventStreamEndpoint() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobSecretsCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobSecretsCollectionInner.java index 64d659c402ceb..47707c7fade7c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobSecretsCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/JobSecretsCollectionInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container Apps Job Secrets Collection ARM resource. */ +/** + * Container Apps Job Secrets Collection ARM resource. + */ @Fluent public final class JobSecretsCollectionInner { /* @@ -19,13 +21,15 @@ public final class JobSecretsCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of JobSecretsCollectionInner class. */ + /** + * Creates an instance of JobSecretsCollectionInner class. + */ public JobSecretsCollectionInner() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -34,7 +38,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the JobSecretsCollectionInner object itself. */ @@ -45,14 +49,13 @@ public JobSecretsCollectionInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model JobSecretsCollectionInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model JobSecretsCollectionInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedCertificateInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedCertificateInner.java index 4854ffb18f62f..d9284752cbc78 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedCertificateInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedCertificateInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ +/** + * Managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. + */ @Fluent public final class ManagedCertificateInner extends Resource { /* @@ -26,13 +28,15 @@ public final class ManagedCertificateInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ManagedCertificateInner class. */ + /** + * Creates an instance of ManagedCertificateInner class. + */ public ManagedCertificateInner() { } /** * Get the properties property: Certificate resource specific properties. - * + * * @return the properties value. */ public ManagedCertificateProperties properties() { @@ -41,7 +45,7 @@ public ManagedCertificateProperties properties() { /** * Set the properties property: Certificate resource specific properties. - * + * * @param properties the properties value to set. * @return the ManagedCertificateInner object itself. */ @@ -52,21 +56,25 @@ public ManagedCertificateInner withProperties(ManagedCertificateProperties prope /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ManagedCertificateInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ManagedCertificateInner withTags(Map tags) { super.withTags(tags); @@ -75,7 +83,7 @@ public ManagedCertificateInner withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentInner.java index 897faf511a692..c31e5a5ee4f04 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentInner.java @@ -7,19 +7,24 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; import com.azure.core.management.SystemData; +import com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration; import com.azure.resourcemanager.appcontainers.models.AppLogsConfiguration; import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration; import com.azure.resourcemanager.appcontainers.models.DaprConfiguration; import com.azure.resourcemanager.appcontainers.models.EnvironmentProvisioningState; import com.azure.resourcemanager.appcontainers.models.KedaConfiguration; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentPropertiesPeerAuthentication; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.OpenTelemetryConfiguration; import com.azure.resourcemanager.appcontainers.models.VnetConfiguration; import com.azure.resourcemanager.appcontainers.models.WorkloadProfile; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; import java.util.Map; -/** An environment for hosting container apps. */ +/** + * An environment for hosting container apps. + */ @Fluent public final class ManagedEnvironmentInner extends Resource { /* @@ -28,6 +33,13 @@ public final class ManagedEnvironmentInner extends Resource { @JsonProperty(value = "kind") private String kind; + /* + * Managed identities for the Managed Environment to interact with other Azure services without maintaining any + * secrets or credentials in code. + */ + @JsonProperty(value = "identity") + private ManagedServiceIdentity identity; + /* * Managed environment resource specific properties */ @@ -40,13 +52,15 @@ public final class ManagedEnvironmentInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ManagedEnvironmentInner class. */ + /** + * Creates an instance of ManagedEnvironmentInner class. + */ public ManagedEnvironmentInner() { } /** * Get the kind property: Kind of the Environment. - * + * * @return the kind value. */ public String kind() { @@ -55,7 +69,7 @@ public String kind() { /** * Set the kind property: Kind of the Environment. - * + * * @param kind the kind value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -64,9 +78,31 @@ public ManagedEnvironmentInner withKind(String kind) { return this; } + /** + * Get the identity property: Managed identities for the Managed Environment to interact with other Azure services + * without maintaining any secrets or credentials in code. + * + * @return the identity value. + */ + public ManagedServiceIdentity identity() { + return this.identity; + } + + /** + * Set the identity property: Managed identities for the Managed Environment to interact with other Azure services + * without maintaining any secrets or credentials in code. + * + * @param identity the identity value to set. + * @return the ManagedEnvironmentInner object itself. + */ + public ManagedEnvironmentInner withIdentity(ManagedServiceIdentity identity) { + this.identity = identity; + return this; + } + /** * Get the innerProperties property: Managed environment resource specific properties. - * + * * @return the innerProperties value. */ private ManagedEnvironmentProperties innerProperties() { @@ -75,21 +111,25 @@ private ManagedEnvironmentProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ManagedEnvironmentInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ManagedEnvironmentInner withTags(Map tags) { super.withTags(tags); @@ -98,7 +138,7 @@ public ManagedEnvironmentInner withTags(Map tags) { /** * Get the provisioningState property: Provisioning state of the Environment. - * + * * @return the provisioningState value. */ public EnvironmentProvisioningState provisioningState() { @@ -108,7 +148,7 @@ public EnvironmentProvisioningState provisioningState() { /** * Get the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export Service to * Service communication telemetry. - * + * * @return the daprAIInstrumentationKey value. */ public String daprAIInstrumentationKey() { @@ -118,7 +158,7 @@ public String daprAIInstrumentationKey() { /** * Set the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export Service to * Service communication telemetry. - * + * * @param daprAIInstrumentationKey the daprAIInstrumentationKey value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -131,9 +171,9 @@ public ManagedEnvironmentInner withDaprAIInstrumentationKey(String daprAIInstrum } /** - * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @return the daprAIConnectionString value. */ public String daprAIConnectionString() { @@ -141,9 +181,9 @@ public String daprAIConnectionString() { } /** - * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @param daprAIConnectionString the daprAIConnectionString value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -157,7 +197,7 @@ public ManagedEnvironmentInner withDaprAIConnectionString(String daprAIConnectio /** * Get the vnetConfiguration property: Vnet configuration for the environment. - * + * * @return the vnetConfiguration value. */ public VnetConfiguration vnetConfiguration() { @@ -166,7 +206,7 @@ public VnetConfiguration vnetConfiguration() { /** * Set the vnetConfiguration property: Vnet configuration for the environment. - * + * * @param vnetConfiguration the vnetConfiguration value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -180,7 +220,7 @@ public ManagedEnvironmentInner withVnetConfiguration(VnetConfiguration vnetConfi /** * Get the deploymentErrors property: Any errors that occurred during deployment or deployment validation. - * + * * @return the deploymentErrors value. */ public String deploymentErrors() { @@ -189,7 +229,7 @@ public String deploymentErrors() { /** * Get the defaultDomain property: Default Domain Name for the cluster. - * + * * @return the defaultDomain value. */ public String defaultDomain() { @@ -198,7 +238,7 @@ public String defaultDomain() { /** * Get the staticIp property: Static IP of the Environment. - * + * * @return the staticIp value. */ public String staticIp() { @@ -206,9 +246,10 @@ public String staticIp() { } /** - * Get the appLogsConfiguration property: Cluster configuration which enables the log daemon to export app logs to a - * destination. Currently only "log-analytics" is supported. - * + * Get the appLogsConfiguration property: Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. + * * @return the appLogsConfiguration value. */ public AppLogsConfiguration appLogsConfiguration() { @@ -216,9 +257,10 @@ public AppLogsConfiguration appLogsConfiguration() { } /** - * Set the appLogsConfiguration property: Cluster configuration which enables the log daemon to export app logs to a - * destination. Currently only "log-analytics" is supported. - * + * Set the appLogsConfiguration property: Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. + * * @param appLogsConfiguration the appLogsConfiguration value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -230,9 +272,56 @@ public ManagedEnvironmentInner withAppLogsConfiguration(AppLogsConfiguration app return this; } + /** + * Get the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @return the appInsightsConfiguration value. + */ + public AppInsightsConfiguration appInsightsConfiguration() { + return this.innerProperties() == null ? null : this.innerProperties().appInsightsConfiguration(); + } + + /** + * Set the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @param appInsightsConfiguration the appInsightsConfiguration value to set. + * @return the ManagedEnvironmentInner object itself. + */ + public ManagedEnvironmentInner withAppInsightsConfiguration(AppInsightsConfiguration appInsightsConfiguration) { + if (this.innerProperties() == null) { + this.innerProperties = new ManagedEnvironmentProperties(); + } + this.innerProperties().withAppInsightsConfiguration(appInsightsConfiguration); + return this; + } + + /** + * Get the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @return the openTelemetryConfiguration value. + */ + public OpenTelemetryConfiguration openTelemetryConfiguration() { + return this.innerProperties() == null ? null : this.innerProperties().openTelemetryConfiguration(); + } + + /** + * Set the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @param openTelemetryConfiguration the openTelemetryConfiguration value to set. + * @return the ManagedEnvironmentInner object itself. + */ + public ManagedEnvironmentInner + withOpenTelemetryConfiguration(OpenTelemetryConfiguration openTelemetryConfiguration) { + if (this.innerProperties() == null) { + this.innerProperties = new ManagedEnvironmentProperties(); + } + this.innerProperties().withOpenTelemetryConfiguration(openTelemetryConfiguration); + return this; + } + /** * Get the zoneRedundant property: Whether or not this Managed Environment is zone-redundant. - * + * * @return the zoneRedundant value. */ public Boolean zoneRedundant() { @@ -241,7 +330,7 @@ public Boolean zoneRedundant() { /** * Set the zoneRedundant property: Whether or not this Managed Environment is zone-redundant. - * + * * @param zoneRedundant the zoneRedundant value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -255,7 +344,7 @@ public ManagedEnvironmentInner withZoneRedundant(Boolean zoneRedundant) { /** * Get the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @return the customDomainConfiguration value. */ public CustomDomainConfiguration customDomainConfiguration() { @@ -264,7 +353,7 @@ public CustomDomainConfiguration customDomainConfiguration() { /** * Set the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration the customDomainConfiguration value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -278,7 +367,7 @@ public ManagedEnvironmentInner withCustomDomainConfiguration(CustomDomainConfigu /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the Environment. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -287,7 +376,7 @@ public String eventStreamEndpoint() { /** * Get the workloadProfiles property: Workload profiles configured for the Managed Environment. - * + * * @return the workloadProfiles value. */ public List workloadProfiles() { @@ -296,7 +385,7 @@ public List workloadProfiles() { /** * Set the workloadProfiles property: Workload profiles configured for the Managed Environment. - * + * * @param workloadProfiles the workloadProfiles value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -310,7 +399,7 @@ public ManagedEnvironmentInner withWorkloadProfiles(List worklo /** * Get the kedaConfiguration property: The configuration of Keda component. - * + * * @return the kedaConfiguration value. */ public KedaConfiguration kedaConfiguration() { @@ -319,7 +408,7 @@ public KedaConfiguration kedaConfiguration() { /** * Set the kedaConfiguration property: The configuration of Keda component. - * + * * @param kedaConfiguration the kedaConfiguration value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -333,7 +422,7 @@ public ManagedEnvironmentInner withKedaConfiguration(KedaConfiguration kedaConfi /** * Get the daprConfiguration property: The configuration of Dapr component. - * + * * @return the daprConfiguration value. */ public DaprConfiguration daprConfiguration() { @@ -342,7 +431,7 @@ public DaprConfiguration daprConfiguration() { /** * Set the daprConfiguration property: The configuration of Dapr component. - * + * * @param daprConfiguration the daprConfiguration value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -355,10 +444,10 @@ public ManagedEnvironmentInner withDaprConfiguration(DaprConfiguration daprConfi } /** - * Get the infrastructureResourceGroup property: Name of the platform-managed resource group created for the Managed - * Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be created in - * the same subscription as the subnet. - * + * Get the infrastructureResourceGroup property: Name of the platform-managed resource group created for the + * Managed Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be + * created in the same subscription as the subnet. + * * @return the infrastructureResourceGroup value. */ public String infrastructureResourceGroup() { @@ -366,10 +455,10 @@ public String infrastructureResourceGroup() { } /** - * Set the infrastructureResourceGroup property: Name of the platform-managed resource group created for the Managed - * Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be created in - * the same subscription as the subnet. - * + * Set the infrastructureResourceGroup property: Name of the platform-managed resource group created for the + * Managed Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be + * created in the same subscription as the subnet. + * * @param infrastructureResourceGroup the infrastructureResourceGroup value to set. * @return the ManagedEnvironmentInner object itself. */ @@ -383,7 +472,7 @@ public ManagedEnvironmentInner withInfrastructureResourceGroup(String infrastruc /** * Get the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @return the peerAuthentication value. */ public ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication() { @@ -392,12 +481,12 @@ public ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication() { /** * Set the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @param peerAuthentication the peerAuthentication value to set. * @return the ManagedEnvironmentInner object itself. */ - public ManagedEnvironmentInner withPeerAuthentication( - ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication) { + public ManagedEnvironmentInner + withPeerAuthentication(ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication) { if (this.innerProperties() == null) { this.innerProperties = new ManagedEnvironmentProperties(); } @@ -407,10 +496,13 @@ public ManagedEnvironmentInner withPeerAuthentication( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (identity() != null) { + identity().validate(); + } if (innerProperties() != null) { innerProperties().validate(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentProperties.java index f14479db3aa57..8cdc1ac40fbd0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentProperties.java @@ -5,18 +5,22 @@ package com.azure.resourcemanager.appcontainers.fluent.models; import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration; import com.azure.resourcemanager.appcontainers.models.AppLogsConfiguration; import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration; import com.azure.resourcemanager.appcontainers.models.DaprConfiguration; import com.azure.resourcemanager.appcontainers.models.EnvironmentProvisioningState; import com.azure.resourcemanager.appcontainers.models.KedaConfiguration; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentPropertiesPeerAuthentication; +import com.azure.resourcemanager.appcontainers.models.OpenTelemetryConfiguration; import com.azure.resourcemanager.appcontainers.models.VnetConfiguration; import com.azure.resourcemanager.appcontainers.models.WorkloadProfile; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Managed environment resource specific properties. */ +/** + * Managed environment resource specific properties. + */ @Fluent public final class ManagedEnvironmentProperties { /* @@ -69,6 +73,18 @@ public final class ManagedEnvironmentProperties { @JsonProperty(value = "appLogsConfiguration") private AppLogsConfiguration appLogsConfiguration; + /* + * Environment level Application Insights configuration + */ + @JsonProperty(value = "appInsightsConfiguration") + private AppInsightsConfiguration appInsightsConfiguration; + + /* + * Environment Open Telemetry configuration + */ + @JsonProperty(value = "openTelemetryConfiguration") + private OpenTelemetryConfiguration openTelemetryConfiguration; + /* * Whether or not this Managed Environment is zone-redundant. */ @@ -119,13 +135,15 @@ public final class ManagedEnvironmentProperties { @JsonProperty(value = "peerAuthentication") private ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication; - /** Creates an instance of ManagedEnvironmentProperties class. */ + /** + * Creates an instance of ManagedEnvironmentProperties class. + */ public ManagedEnvironmentProperties() { } /** * Get the provisioningState property: Provisioning state of the Environment. - * + * * @return the provisioningState value. */ public EnvironmentProvisioningState provisioningState() { @@ -135,7 +153,7 @@ public EnvironmentProvisioningState provisioningState() { /** * Get the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export Service to * Service communication telemetry. - * + * * @return the daprAIInstrumentationKey value. */ public String daprAIInstrumentationKey() { @@ -145,7 +163,7 @@ public String daprAIInstrumentationKey() { /** * Set the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export Service to * Service communication telemetry. - * + * * @param daprAIInstrumentationKey the daprAIInstrumentationKey value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -155,9 +173,9 @@ public ManagedEnvironmentProperties withDaprAIInstrumentationKey(String daprAIIn } /** - * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Get the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @return the daprAIConnectionString value. */ public String daprAIConnectionString() { @@ -165,9 +183,9 @@ public String daprAIConnectionString() { } /** - * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. - * + * Set the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service + * to Service communication telemetry. + * * @param daprAIConnectionString the daprAIConnectionString value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -178,7 +196,7 @@ public ManagedEnvironmentProperties withDaprAIConnectionString(String daprAIConn /** * Get the vnetConfiguration property: Vnet configuration for the environment. - * + * * @return the vnetConfiguration value. */ public VnetConfiguration vnetConfiguration() { @@ -187,7 +205,7 @@ public VnetConfiguration vnetConfiguration() { /** * Set the vnetConfiguration property: Vnet configuration for the environment. - * + * * @param vnetConfiguration the vnetConfiguration value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -198,7 +216,7 @@ public ManagedEnvironmentProperties withVnetConfiguration(VnetConfiguration vnet /** * Get the deploymentErrors property: Any errors that occurred during deployment or deployment validation. - * + * * @return the deploymentErrors value. */ public String deploymentErrors() { @@ -207,7 +225,7 @@ public String deploymentErrors() { /** * Get the defaultDomain property: Default Domain Name for the cluster. - * + * * @return the defaultDomain value. */ public String defaultDomain() { @@ -216,7 +234,7 @@ public String defaultDomain() { /** * Get the staticIp property: Static IP of the Environment. - * + * * @return the staticIp value. */ public String staticIp() { @@ -224,9 +242,10 @@ public String staticIp() { } /** - * Get the appLogsConfiguration property: Cluster configuration which enables the log daemon to export app logs to a - * destination. Currently only "log-analytics" is supported. - * + * Get the appLogsConfiguration property: Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. + * * @return the appLogsConfiguration value. */ public AppLogsConfiguration appLogsConfiguration() { @@ -234,9 +253,10 @@ public AppLogsConfiguration appLogsConfiguration() { } /** - * Set the appLogsConfiguration property: Cluster configuration which enables the log daemon to export app logs to a - * destination. Currently only "log-analytics" is supported. - * + * Set the appLogsConfiguration property: Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. + * * @param appLogsConfiguration the appLogsConfiguration value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -245,9 +265,51 @@ public ManagedEnvironmentProperties withAppLogsConfiguration(AppLogsConfiguratio return this; } + /** + * Get the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @return the appInsightsConfiguration value. + */ + public AppInsightsConfiguration appInsightsConfiguration() { + return this.appInsightsConfiguration; + } + + /** + * Set the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @param appInsightsConfiguration the appInsightsConfiguration value to set. + * @return the ManagedEnvironmentProperties object itself. + */ + public ManagedEnvironmentProperties + withAppInsightsConfiguration(AppInsightsConfiguration appInsightsConfiguration) { + this.appInsightsConfiguration = appInsightsConfiguration; + return this; + } + + /** + * Get the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @return the openTelemetryConfiguration value. + */ + public OpenTelemetryConfiguration openTelemetryConfiguration() { + return this.openTelemetryConfiguration; + } + + /** + * Set the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @param openTelemetryConfiguration the openTelemetryConfiguration value to set. + * @return the ManagedEnvironmentProperties object itself. + */ + public ManagedEnvironmentProperties + withOpenTelemetryConfiguration(OpenTelemetryConfiguration openTelemetryConfiguration) { + this.openTelemetryConfiguration = openTelemetryConfiguration; + return this; + } + /** * Get the zoneRedundant property: Whether or not this Managed Environment is zone-redundant. - * + * * @return the zoneRedundant value. */ public Boolean zoneRedundant() { @@ -256,7 +318,7 @@ public Boolean zoneRedundant() { /** * Set the zoneRedundant property: Whether or not this Managed Environment is zone-redundant. - * + * * @param zoneRedundant the zoneRedundant value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -267,7 +329,7 @@ public ManagedEnvironmentProperties withZoneRedundant(Boolean zoneRedundant) { /** * Get the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @return the customDomainConfiguration value. */ public CustomDomainConfiguration customDomainConfiguration() { @@ -276,19 +338,19 @@ public CustomDomainConfiguration customDomainConfiguration() { /** * Set the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration the customDomainConfiguration value to set. * @return the ManagedEnvironmentProperties object itself. */ - public ManagedEnvironmentProperties withCustomDomainConfiguration( - CustomDomainConfiguration customDomainConfiguration) { + public ManagedEnvironmentProperties + withCustomDomainConfiguration(CustomDomainConfiguration customDomainConfiguration) { this.customDomainConfiguration = customDomainConfiguration; return this; } /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the Environment. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -297,7 +359,7 @@ public String eventStreamEndpoint() { /** * Get the workloadProfiles property: Workload profiles configured for the Managed Environment. - * + * * @return the workloadProfiles value. */ public List workloadProfiles() { @@ -306,7 +368,7 @@ public List workloadProfiles() { /** * Set the workloadProfiles property: Workload profiles configured for the Managed Environment. - * + * * @param workloadProfiles the workloadProfiles value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -317,7 +379,7 @@ public ManagedEnvironmentProperties withWorkloadProfiles(List w /** * Get the kedaConfiguration property: The configuration of Keda component. - * + * * @return the kedaConfiguration value. */ public KedaConfiguration kedaConfiguration() { @@ -326,7 +388,7 @@ public KedaConfiguration kedaConfiguration() { /** * Set the kedaConfiguration property: The configuration of Keda component. - * + * * @param kedaConfiguration the kedaConfiguration value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -337,7 +399,7 @@ public ManagedEnvironmentProperties withKedaConfiguration(KedaConfiguration keda /** * Get the daprConfiguration property: The configuration of Dapr component. - * + * * @return the daprConfiguration value. */ public DaprConfiguration daprConfiguration() { @@ -346,7 +408,7 @@ public DaprConfiguration daprConfiguration() { /** * Set the daprConfiguration property: The configuration of Dapr component. - * + * * @param daprConfiguration the daprConfiguration value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -356,10 +418,10 @@ public ManagedEnvironmentProperties withDaprConfiguration(DaprConfiguration dapr } /** - * Get the infrastructureResourceGroup property: Name of the platform-managed resource group created for the Managed - * Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be created in - * the same subscription as the subnet. - * + * Get the infrastructureResourceGroup property: Name of the platform-managed resource group created for the + * Managed Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be + * created in the same subscription as the subnet. + * * @return the infrastructureResourceGroup value. */ public String infrastructureResourceGroup() { @@ -367,10 +429,10 @@ public String infrastructureResourceGroup() { } /** - * Set the infrastructureResourceGroup property: Name of the platform-managed resource group created for the Managed - * Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be created in - * the same subscription as the subnet. - * + * Set the infrastructureResourceGroup property: Name of the platform-managed resource group created for the + * Managed Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be + * created in the same subscription as the subnet. + * * @param infrastructureResourceGroup the infrastructureResourceGroup value to set. * @return the ManagedEnvironmentProperties object itself. */ @@ -381,7 +443,7 @@ public ManagedEnvironmentProperties withInfrastructureResourceGroup(String infra /** * Get the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @return the peerAuthentication value. */ public ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication() { @@ -390,19 +452,19 @@ public ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication() { /** * Set the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @param peerAuthentication the peerAuthentication value to set. * @return the ManagedEnvironmentProperties object itself. */ - public ManagedEnvironmentProperties withPeerAuthentication( - ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication) { + public ManagedEnvironmentProperties + withPeerAuthentication(ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication) { this.peerAuthentication = peerAuthentication; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -412,6 +474,12 @@ public void validate() { if (appLogsConfiguration() != null) { appLogsConfiguration().validate(); } + if (appInsightsConfiguration() != null) { + appInsightsConfiguration().validate(); + } + if (openTelemetryConfiguration() != null) { + openTelemetryConfiguration().validate(); + } if (customDomainConfiguration() != null) { customDomainConfiguration().validate(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStorageInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStorageInner.java index 5b01b9826d61f..57742145165c9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStorageInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStorageInner.java @@ -6,10 +6,13 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentStorageProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Storage resource for managedEnvironment. */ +/** + * Storage resource for managedEnvironment. + */ @Fluent public final class ManagedEnvironmentStorageInner extends ProxyResource { /* @@ -18,13 +21,21 @@ public final class ManagedEnvironmentStorageInner extends ProxyResource { @JsonProperty(value = "properties") private ManagedEnvironmentStorageProperties properties; - /** Creates an instance of ManagedEnvironmentStorageInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of ManagedEnvironmentStorageInner class. + */ public ManagedEnvironmentStorageInner() { } /** * Get the properties property: Storage properties. - * + * * @return the properties value. */ public ManagedEnvironmentStorageProperties properties() { @@ -33,7 +44,7 @@ public ManagedEnvironmentStorageProperties properties() { /** * Set the properties property: Storage properties. - * + * * @param properties the properties value to set. * @return the ManagedEnvironmentStorageInner object itself. */ @@ -42,9 +53,18 @@ public ManagedEnvironmentStorageInner withProperties(ManagedEnvironmentStoragePr return this; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStoragesCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStoragesCollectionInner.java index 620546ce1064e..58a26bebfd69a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStoragesCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ManagedEnvironmentStoragesCollectionInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of Storage for Environments. */ +/** + * Collection of Storage for Environments. + */ @Fluent public final class ManagedEnvironmentStoragesCollectionInner { /* @@ -18,13 +20,15 @@ public final class ManagedEnvironmentStoragesCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of ManagedEnvironmentStoragesCollectionInner class. */ + /** + * Creates an instance of ManagedEnvironmentStoragesCollectionInner class. + */ public ManagedEnvironmentStoragesCollectionInner() { } /** * Get the value property: Collection of storage resources. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Collection of storage resources. - * + * * @param value the value value to set. * @return the ManagedEnvironmentStoragesCollectionInner object itself. */ @@ -44,15 +48,13 @@ public ManagedEnvironmentStoragesCollectionInner withValue(List e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/OperationDetailInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/OperationDetailInner.java index f4a5897a8ff9b..da3d614ef1632 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/OperationDetailInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/OperationDetailInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.appcontainers.models.OperationDisplay; import com.fasterxml.jackson.annotation.JsonProperty; -/** Operation detail payload. */ +/** + * Operation detail payload. + */ @Fluent public final class OperationDetailInner { /* @@ -35,13 +37,15 @@ public final class OperationDetailInner { @JsonProperty(value = "origin") private String origin; - /** Creates an instance of OperationDetailInner class. */ + /** + * Creates an instance of OperationDetailInner class. + */ public OperationDetailInner() { } /** * Get the name property: Name of the operation. - * + * * @return the name value. */ public String name() { @@ -50,7 +54,7 @@ public String name() { /** * Set the name property: Name of the operation. - * + * * @param name the name value to set. * @return the OperationDetailInner object itself. */ @@ -61,7 +65,7 @@ public OperationDetailInner withName(String name) { /** * Get the isDataAction property: Indicates whether the operation is a data action. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -70,7 +74,7 @@ public Boolean isDataAction() { /** * Set the isDataAction property: Indicates whether the operation is a data action. - * + * * @param isDataAction the isDataAction value to set. * @return the OperationDetailInner object itself. */ @@ -81,7 +85,7 @@ public OperationDetailInner withIsDataAction(Boolean isDataAction) { /** * Get the display property: Display of the operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -90,7 +94,7 @@ public OperationDisplay display() { /** * Set the display property: Display of the operation. - * + * * @param display the display value to set. * @return the OperationDetailInner object itself. */ @@ -101,7 +105,7 @@ public OperationDetailInner withDisplay(OperationDisplay display) { /** * Get the origin property: Origin of the operation. - * + * * @return the origin value. */ public String origin() { @@ -110,7 +114,7 @@ public String origin() { /** * Set the origin property: Origin of the operation. - * + * * @param origin the origin value to set. * @return the OperationDetailInner object itself. */ @@ -121,7 +125,7 @@ public OperationDetailInner withOrigin(String origin) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaCollectionInner.java index 89aa52cf18ff5..ef472cec50786 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaCollectionInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App Revision Replicas collection ARM resource. */ +/** + * Container App Revision Replicas collection ARM resource. + */ @Fluent public final class ReplicaCollectionInner { /* @@ -18,13 +20,15 @@ public final class ReplicaCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of ReplicaCollectionInner class. */ + /** + * Creates an instance of ReplicaCollectionInner class. + */ public ReplicaCollectionInner() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the ReplicaCollectionInner object itself. */ @@ -44,14 +48,13 @@ public ReplicaCollectionInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model ReplicaCollectionInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ReplicaCollectionInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaInner.java index 503299ffa44f8..fb33c279dd3f4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaInner.java @@ -6,13 +6,16 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.ContainerAppReplicaRunningState; import com.azure.resourcemanager.appcontainers.models.ReplicaContainer; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; import java.util.List; -/** Container App Revision Replica. */ +/** + * Container App Revision Replica. + */ @Fluent public final class ReplicaInner extends ProxyResource { /* @@ -21,22 +24,39 @@ public final class ReplicaInner extends ProxyResource { @JsonProperty(value = "properties") private ReplicaProperties innerProperties; - /** Creates an instance of ReplicaInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of ReplicaInner class. + */ public ReplicaInner() { } /** * Get the innerProperties property: Replica resource specific properties. - * + * * @return the innerProperties value. */ private ReplicaProperties innerProperties() { return this.innerProperties; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Get the createdTime property: Timestamp describing when the pod was created by controller. - * + * * @return the createdTime value. */ public OffsetDateTime createdTime() { @@ -45,7 +65,7 @@ public OffsetDateTime createdTime() { /** * Get the runningState property: Current running state of the replica. - * + * * @return the runningState value. */ public ContainerAppReplicaRunningState runningState() { @@ -54,7 +74,7 @@ public ContainerAppReplicaRunningState runningState() { /** * Get the runningStateDetails property: The details of replica current running state. - * + * * @return the runningStateDetails value. */ public String runningStateDetails() { @@ -63,7 +83,7 @@ public String runningStateDetails() { /** * Get the containers property: The containers collection under a replica. - * + * * @return the containers value. */ public List containers() { @@ -72,7 +92,7 @@ public List containers() { /** * Set the containers property: The containers collection under a replica. - * + * * @param containers the containers value to set. * @return the ReplicaInner object itself. */ @@ -86,7 +106,7 @@ public ReplicaInner withContainers(List containers) { /** * Get the initContainers property: The init containers collection under a replica. - * + * * @return the initContainers value. */ public List initContainers() { @@ -95,7 +115,7 @@ public List initContainers() { /** * Set the initContainers property: The init containers collection under a replica. - * + * * @param initContainers the initContainers value to set. * @return the ReplicaInner object itself. */ @@ -109,7 +129,7 @@ public ReplicaInner withInitContainers(List initContainers) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaProperties.java index 0d3eed1784a79..68dfe31c056b7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/ReplicaProperties.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Replica resource specific properties. */ +/** + * Replica resource specific properties. + */ @Fluent public final class ReplicaProperties { /* @@ -44,13 +46,15 @@ public final class ReplicaProperties { @JsonProperty(value = "initContainers") private List initContainers; - /** Creates an instance of ReplicaProperties class. */ + /** + * Creates an instance of ReplicaProperties class. + */ public ReplicaProperties() { } /** * Get the createdTime property: Timestamp describing when the pod was created by controller. - * + * * @return the createdTime value. */ public OffsetDateTime createdTime() { @@ -59,7 +63,7 @@ public OffsetDateTime createdTime() { /** * Get the runningState property: Current running state of the replica. - * + * * @return the runningState value. */ public ContainerAppReplicaRunningState runningState() { @@ -68,7 +72,7 @@ public ContainerAppReplicaRunningState runningState() { /** * Get the runningStateDetails property: The details of replica current running state. - * + * * @return the runningStateDetails value. */ public String runningStateDetails() { @@ -77,7 +81,7 @@ public String runningStateDetails() { /** * Get the containers property: The containers collection under a replica. - * + * * @return the containers value. */ public List containers() { @@ -86,7 +90,7 @@ public List containers() { /** * Set the containers property: The containers collection under a replica. - * + * * @param containers the containers value to set. * @return the ReplicaProperties object itself. */ @@ -97,7 +101,7 @@ public ReplicaProperties withContainers(List containers) { /** * Get the initContainers property: The init containers collection under a replica. - * + * * @return the initContainers value. */ public List initContainers() { @@ -106,7 +110,7 @@ public List initContainers() { /** * Set the initContainers property: The init containers collection under a replica. - * + * * @param initContainers the initContainers value to set. * @return the ReplicaProperties object itself. */ @@ -117,7 +121,7 @@ public ReplicaProperties withInitContainers(List initContainer /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionInner.java index 178306f4e3f2e..a5635ffbc21b8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionInner.java @@ -6,6 +6,7 @@ import com.azure.core.annotation.Immutable; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.RevisionHealthState; import com.azure.resourcemanager.appcontainers.models.RevisionProvisioningState; import com.azure.resourcemanager.appcontainers.models.RevisionRunningState; @@ -13,7 +14,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Container App Revision. */ +/** + * Container App Revision. + */ @Immutable public final class RevisionInner extends ProxyResource { /* @@ -22,13 +25,21 @@ public final class RevisionInner extends ProxyResource { @JsonProperty(value = "properties") private RevisionProperties innerProperties; - /** Creates an instance of RevisionInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of RevisionInner class. + */ public RevisionInner() { } /** * Get the innerProperties property: Revision resource specific properties. - * + * * @return the innerProperties value. */ private RevisionProperties innerProperties() { @@ -36,8 +47,18 @@ private RevisionProperties innerProperties() { } /** - * Get the createdTime property: Timestamp describing when the revision was created by controller. - * + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the createdTime property: Timestamp describing when the revision was created + * by controller. + * * @return the createdTime value. */ public OffsetDateTime createdTime() { @@ -47,7 +68,7 @@ public OffsetDateTime createdTime() { /** * Get the lastActiveTime property: Timestamp describing when the revision was last active. Only meaningful when * revision is inactive. - * + * * @return the lastActiveTime value. */ public OffsetDateTime lastActiveTime() { @@ -56,7 +77,7 @@ public OffsetDateTime lastActiveTime() { /** * Get the fqdn property: Fully qualified domain name of the revision. - * + * * @return the fqdn value. */ public String fqdn() { @@ -64,9 +85,10 @@ public String fqdn() { } /** - * Get the template property: Container App Revision Template with all possible settings and the defaults if user - * did not provide them. The defaults are populated as they were at the creation time. - * + * Get the template property: Container App Revision Template with all possible settings and the + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @return the template value. */ public Template template() { @@ -75,7 +97,7 @@ public Template template() { /** * Get the active property: Boolean describing if the Revision is Active. - * + * * @return the active value. */ public Boolean active() { @@ -84,7 +106,7 @@ public Boolean active() { /** * Get the replicas property: Number of pods currently running for this revision. - * + * * @return the replicas value. */ public Integer replicas() { @@ -93,7 +115,7 @@ public Integer replicas() { /** * Get the trafficWeight property: Traffic weight assigned to this revision. - * + * * @return the trafficWeight value. */ public Integer trafficWeight() { @@ -102,7 +124,7 @@ public Integer trafficWeight() { /** * Get the provisioningError property: Optional Field - Platform Error Message. - * + * * @return the provisioningError value. */ public String provisioningError() { @@ -111,7 +133,7 @@ public String provisioningError() { /** * Get the healthState property: Current health State of the revision. - * + * * @return the healthState value. */ public RevisionHealthState healthState() { @@ -120,7 +142,7 @@ public RevisionHealthState healthState() { /** * Get the provisioningState property: Current provisioning State of the revision. - * + * * @return the provisioningState value. */ public RevisionProvisioningState provisioningState() { @@ -129,7 +151,7 @@ public RevisionProvisioningState provisioningState() { /** * Get the runningState property: Current running state of the revision. - * + * * @return the runningState value. */ public RevisionRunningState runningState() { @@ -138,7 +160,7 @@ public RevisionRunningState runningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionProperties.java index 64782de6f21c2..e05d58766764b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/RevisionProperties.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Revision resource specific properties. */ +/** + * Revision resource specific properties. + */ @Immutable public final class RevisionProperties { /* @@ -84,13 +86,16 @@ public final class RevisionProperties { @JsonProperty(value = "runningState", access = JsonProperty.Access.WRITE_ONLY) private RevisionRunningState runningState; - /** Creates an instance of RevisionProperties class. */ + /** + * Creates an instance of RevisionProperties class. + */ public RevisionProperties() { } /** - * Get the createdTime property: Timestamp describing when the revision was created by controller. - * + * Get the createdTime property: Timestamp describing when the revision was created + * by controller. + * * @return the createdTime value. */ public OffsetDateTime createdTime() { @@ -100,7 +105,7 @@ public OffsetDateTime createdTime() { /** * Get the lastActiveTime property: Timestamp describing when the revision was last active. Only meaningful when * revision is inactive. - * + * * @return the lastActiveTime value. */ public OffsetDateTime lastActiveTime() { @@ -109,7 +114,7 @@ public OffsetDateTime lastActiveTime() { /** * Get the fqdn property: Fully qualified domain name of the revision. - * + * * @return the fqdn value. */ public String fqdn() { @@ -117,9 +122,10 @@ public String fqdn() { } /** - * Get the template property: Container App Revision Template with all possible settings and the defaults if user - * did not provide them. The defaults are populated as they were at the creation time. - * + * Get the template property: Container App Revision Template with all possible settings and the + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @return the template value. */ public Template template() { @@ -128,7 +134,7 @@ public Template template() { /** * Get the active property: Boolean describing if the Revision is Active. - * + * * @return the active value. */ public Boolean active() { @@ -137,7 +143,7 @@ public Boolean active() { /** * Get the replicas property: Number of pods currently running for this revision. - * + * * @return the replicas value. */ public Integer replicas() { @@ -146,7 +152,7 @@ public Integer replicas() { /** * Get the trafficWeight property: Traffic weight assigned to this revision. - * + * * @return the trafficWeight value. */ public Integer trafficWeight() { @@ -155,7 +161,7 @@ public Integer trafficWeight() { /** * Get the provisioningError property: Optional Field - Platform Error Message. - * + * * @return the provisioningError value. */ public String provisioningError() { @@ -164,7 +170,7 @@ public String provisioningError() { /** * Get the healthState property: Current health State of the revision. - * + * * @return the healthState value. */ public RevisionHealthState healthState() { @@ -173,7 +179,7 @@ public RevisionHealthState healthState() { /** * Get the provisioningState property: Current provisioning State of the revision. - * + * * @return the provisioningState value. */ public RevisionProvisioningState provisioningState() { @@ -182,7 +188,7 @@ public RevisionProvisioningState provisioningState() { /** * Get the runningState property: Current running state of the revision. - * + * * @return the runningState value. */ public RevisionRunningState runningState() { @@ -191,7 +197,7 @@ public RevisionRunningState runningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SecretsCollectionInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SecretsCollectionInner.java index 4d9eaacf9c0d7..f3ee91d90586d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SecretsCollectionInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SecretsCollectionInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App Secrets Collection ARM resource. */ +/** + * Container App Secrets Collection ARM resource. + */ @Fluent public final class SecretsCollectionInner { /* @@ -19,13 +21,15 @@ public final class SecretsCollectionInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of SecretsCollectionInner class. */ + /** + * Creates an instance of SecretsCollectionInner class. + */ public SecretsCollectionInner() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -34,7 +38,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the SecretsCollectionInner object itself. */ @@ -45,14 +49,13 @@ public SecretsCollectionInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model SecretsCollectionInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model SecretsCollectionInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlInner.java index 74f9577011f61..434e069ed942b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlInner.java @@ -6,11 +6,14 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.GithubActionConfiguration; import com.azure.resourcemanager.appcontainers.models.SourceControlOperationState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App SourceControl. */ +/** + * Container App SourceControl. + */ @Fluent public final class SourceControlInner extends ProxyResource { /* @@ -19,22 +22,39 @@ public final class SourceControlInner extends ProxyResource { @JsonProperty(value = "properties") private SourceControlProperties innerProperties; - /** Creates an instance of SourceControlInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of SourceControlInner class. + */ public SourceControlInner() { } /** * Get the innerProperties property: SourceControl resource specific properties. - * + * * @return the innerProperties value. */ private SourceControlProperties innerProperties() { return this.innerProperties; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Get the operationState property: Current provisioning State of the operation. - * + * * @return the operationState value. */ public SourceControlOperationState operationState() { @@ -43,7 +63,7 @@ public SourceControlOperationState operationState() { /** * Get the repoUrl property: The repo url which will be integrated to ContainerApp. - * + * * @return the repoUrl value. */ public String repoUrl() { @@ -52,7 +72,7 @@ public String repoUrl() { /** * Set the repoUrl property: The repo url which will be integrated to ContainerApp. - * + * * @param repoUrl the repoUrl value to set. * @return the SourceControlInner object itself. */ @@ -66,7 +86,7 @@ public SourceControlInner withRepoUrl(String repoUrl) { /** * Get the branch property: The branch which will trigger the auto deployment. - * + * * @return the branch value. */ public String branch() { @@ -75,7 +95,7 @@ public String branch() { /** * Set the branch property: The branch which will trigger the auto deployment. - * + * * @param branch the branch value to set. * @return the SourceControlInner object itself. */ @@ -89,8 +109,9 @@ public SourceControlInner withBranch(String branch) { /** * Get the githubActionConfiguration property: Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. - * + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @return the githubActionConfiguration value. */ public GithubActionConfiguration githubActionConfiguration() { @@ -99,8 +120,9 @@ public GithubActionConfiguration githubActionConfiguration() { /** * Set the githubActionConfiguration property: Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. - * + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @param githubActionConfiguration the githubActionConfiguration value to set. * @return the SourceControlInner object itself. */ @@ -114,7 +136,7 @@ public SourceControlInner withGithubActionConfiguration(GithubActionConfiguratio /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlProperties.java index 94c3efcb7970d..c6111e7ed3566 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/SourceControlProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.appcontainers.models.SourceControlOperationState; import com.fasterxml.jackson.annotation.JsonProperty; -/** SourceControl resource specific properties. */ +/** + * SourceControl resource specific properties. + */ @Fluent public final class SourceControlProperties { /* @@ -38,13 +40,15 @@ public final class SourceControlProperties { @JsonProperty(value = "githubActionConfiguration") private GithubActionConfiguration githubActionConfiguration; - /** Creates an instance of SourceControlProperties class. */ + /** + * Creates an instance of SourceControlProperties class. + */ public SourceControlProperties() { } /** * Get the operationState property: Current provisioning State of the operation. - * + * * @return the operationState value. */ public SourceControlOperationState operationState() { @@ -53,7 +57,7 @@ public SourceControlOperationState operationState() { /** * Get the repoUrl property: The repo url which will be integrated to ContainerApp. - * + * * @return the repoUrl value. */ public String repoUrl() { @@ -62,7 +66,7 @@ public String repoUrl() { /** * Set the repoUrl property: The repo url which will be integrated to ContainerApp. - * + * * @param repoUrl the repoUrl value to set. * @return the SourceControlProperties object itself. */ @@ -73,7 +77,7 @@ public SourceControlProperties withRepoUrl(String repoUrl) { /** * Get the branch property: The branch which will trigger the auto deployment. - * + * * @return the branch value. */ public String branch() { @@ -82,7 +86,7 @@ public String branch() { /** * Set the branch property: The branch which will trigger the auto deployment. - * + * * @param branch the branch value to set. * @return the SourceControlProperties object itself. */ @@ -93,8 +97,9 @@ public SourceControlProperties withBranch(String branch) { /** * Get the githubActionConfiguration property: Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. - * + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @return the githubActionConfiguration value. */ public GithubActionConfiguration githubActionConfiguration() { @@ -103,8 +108,9 @@ public GithubActionConfiguration githubActionConfiguration() { /** * Set the githubActionConfiguration property: Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. - * + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @param githubActionConfiguration the githubActionConfiguration value to set. * @return the SourceControlProperties object itself. */ @@ -115,7 +121,7 @@ public SourceControlProperties withGithubActionConfiguration(GithubActionConfigu /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/UsageInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/UsageInner.java new file mode 100644 index 0000000000000..e37f5a12c5f96 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/UsageInner.java @@ -0,0 +1,142 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.models.UsageName; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Describes Compute Resource Usage. + */ +@Fluent +public final class UsageInner { + /* + * An enum describing the unit of usage measurement. + */ + @JsonProperty(value = "unit", required = true) + private String unit = "Count"; + + /* + * The current usage of the resource. + */ + @JsonProperty(value = "currentValue", required = true) + private float currentValue; + + /* + * The maximum permitted usage of the resource. + */ + @JsonProperty(value = "limit", required = true) + private float limit; + + /* + * The name of the type of usage. + */ + @JsonProperty(value = "name", required = true) + private UsageName name; + + /** + * Creates an instance of UsageInner class. + */ + public UsageInner() { + } + + /** + * Get the unit property: An enum describing the unit of usage measurement. + * + * @return the unit value. + */ + public String unit() { + return this.unit; + } + + /** + * Set the unit property: An enum describing the unit of usage measurement. + * + * @param unit the unit value to set. + * @return the UsageInner object itself. + */ + public UsageInner withUnit(String unit) { + this.unit = unit; + return this; + } + + /** + * Get the currentValue property: The current usage of the resource. + * + * @return the currentValue value. + */ + public float currentValue() { + return this.currentValue; + } + + /** + * Set the currentValue property: The current usage of the resource. + * + * @param currentValue the currentValue value to set. + * @return the UsageInner object itself. + */ + public UsageInner withCurrentValue(float currentValue) { + this.currentValue = currentValue; + return this; + } + + /** + * Get the limit property: The maximum permitted usage of the resource. + * + * @return the limit value. + */ + public float limit() { + return this.limit; + } + + /** + * Set the limit property: The maximum permitted usage of the resource. + * + * @param limit the limit value to set. + * @return the UsageInner object itself. + */ + public UsageInner withLimit(float limit) { + this.limit = limit; + return this; + } + + /** + * Get the name property: The name of the type of usage. + * + * @return the name value. + */ + public UsageName name() { + return this.name; + } + + /** + * Set the name property: The name of the type of usage. + * + * @param name the name value to set. + * @return the UsageInner object itself. + */ + public UsageInner withName(UsageName name) { + this.name = name; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (name() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model UsageInner")); + } else { + name().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UsageInner.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/WorkloadProfileStatesInner.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/WorkloadProfileStatesInner.java index 825cf626a1bfd..50c4bf0688d8f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/WorkloadProfileStatesInner.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/WorkloadProfileStatesInner.java @@ -6,10 +6,13 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.models.WorkloadProfileStatesProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Collection of all the workload Profile States for a Managed Environment.. */ +/** + * Collection of all the workload Profile States for a Managed Environment.. + */ @Fluent public final class WorkloadProfileStatesInner extends ProxyResource { /* @@ -18,13 +21,21 @@ public final class WorkloadProfileStatesInner extends ProxyResource { @JsonProperty(value = "properties") private WorkloadProfileStatesProperties properties; - /** Creates an instance of WorkloadProfileStatesInner class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of WorkloadProfileStatesInner class. + */ public WorkloadProfileStatesInner() { } /** * Get the properties property: Workload Profile resource specific properties. - * + * * @return the properties value. */ public WorkloadProfileStatesProperties properties() { @@ -33,7 +44,7 @@ public WorkloadProfileStatesProperties properties() { /** * Set the properties property: Workload Profile resource specific properties. - * + * * @param properties the properties value to set. * @return the WorkloadProfileStatesInner object itself. */ @@ -42,9 +53,18 @@ public WorkloadProfileStatesInner withProperties(WorkloadProfileStatesProperties return this; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/package-info.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/package-info.java index a0507c49fad09..9ea383c7aad70 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/package-info.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the inner data models for ContainerAppsApiClient. null. */ +/** + * Package containing the inner data models for ContainerAppsApiClient. + * null. + */ package com.azure.resourcemanager.appcontainers.fluent.models; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/package-info.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/package-info.java index 4d3731d374d35..131b0b619aad3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/package-info.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/fluent/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the service clients for ContainerAppsApiClient. null. */ +/** + * Package containing the service clients for ContainerAppsApiClient. + * null. + */ package com.azure.resourcemanager.appcontainers.fluent; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesClientImpl.java new file mode 100644 index 0000000000000..af7bcd069fbbf --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesClientImpl.java @@ -0,0 +1,930 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.AppResilienciesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner; +import com.azure.resourcemanager.appcontainers.models.AppResiliencyCollection; +import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in AppResilienciesClient. + */ +public final class AppResilienciesClientImpl implements AppResilienciesClient { + /** + * The proxy service used to perform REST calls. + */ + private final AppResilienciesService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of AppResilienciesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + AppResilienciesClientImpl(ContainerAppsApiClientImpl client) { + this.service + = RestProxy.create(AppResilienciesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientAppResiliencies to be used by the proxy + * service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface AppResilienciesService { + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{appName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("appName") String appName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AppResiliencyInner resiliencyEnvelope, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{appName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("appName") String appName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AppResiliencyInner resiliencyEnvelope, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{appName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("appName") String appName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{appName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("appName") String appName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{appName}/resiliencyPolicies") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("appName") String appName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, String appName, + String name, AppResiliencyInner resiliencyEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (resiliencyEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter resiliencyEnvelope is required and cannot be null.")); + } else { + resiliencyEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, appName, name, this.client.getApiVersion(), resiliencyEnvelope, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, String appName, + String name, AppResiliencyInner resiliencyEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (resiliencyEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter resiliencyEnvelope is required and cannot be null.")); + } else { + resiliencyEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + appName, name, this.client.getApiVersion(), resiliencyEnvelope, accept, context); + } + + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope) { + return createOrUpdateWithResponseAsync(resourceGroupName, appName, name, resiliencyEnvelope) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createOrUpdateWithResponse(String resourceGroupName, String appName, + String name, AppResiliencyInner resiliencyEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, appName, name, resiliencyEnvelope, context).block(); + } + + /** + * Create or update an application's resiliency policy. + * + * Create or update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to create or update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AppResiliencyInner createOrUpdate(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, appName, name, resiliencyEnvelope, Context.NONE) + .getValue(); + } + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> updateWithResponseAsync(String resourceGroupName, String appName, + String name, AppResiliencyInner resiliencyEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (resiliencyEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter resiliencyEnvelope is required and cannot be null.")); + } else { + resiliencyEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, appName, name, this.client.getApiVersion(), resiliencyEnvelope, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> updateWithResponseAsync(String resourceGroupName, String appName, + String name, AppResiliencyInner resiliencyEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (resiliencyEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter resiliencyEnvelope is required and cannot be null.")); + } else { + resiliencyEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, appName, + name, this.client.getApiVersion(), resiliencyEnvelope, accept, context); + } + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope) { + return updateWithResponseAsync(resourceGroupName, appName, name, resiliencyEnvelope) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response updateWithResponse(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope, Context context) { + return updateWithResponseAsync(resourceGroupName, appName, name, resiliencyEnvelope, context).block(); + } + + /** + * Update an application's resiliency policy. + * + * Update container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param resiliencyEnvelope The resiliency policy to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return configuration to setup App Resiliency. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AppResiliencyInner update(String resourceGroupName, String appName, String name, + AppResiliencyInner resiliencyEnvelope) { + return updateWithResponse(resourceGroupName, appName, name, resiliencyEnvelope, Context.NONE).getValue(); + } + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(String resourceGroupName, String appName, String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, appName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(String resourceGroupName, String appName, String name, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, appName, + name, this.client.getApiVersion(), accept, context); + } + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String appName, String name) { + return deleteWithResponseAsync(resourceGroupName, appName, name).flatMap(ignored -> Mono.empty()); + } + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteWithResponse(String resourceGroupName, String appName, String name, Context context) { + return deleteWithResponseAsync(resourceGroupName, appName, name, context).block(); + } + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String appName, String name) { + deleteWithResponse(resourceGroupName, appName, name, Context.NONE); + } + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String appName, + String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, appName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String appName, + String name, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, appName, name, + this.client.getApiVersion(), accept, context); + } + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String appName, String name) { + return getWithResponseAsync(resourceGroupName, appName, name).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String appName, String name, + Context context) { + return getWithResponseAsync(resourceGroupName, appName, name, context).block(); + } + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AppResiliencyInner get(String resourceGroupName, String appName, String name) { + return getWithResponse(resourceGroupName, appName, name, Context.NONE).getValue(); + } + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, String appName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, appName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, String appName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (appName == null) { + return Mono.error(new IllegalArgumentException("Parameter appName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, appName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String appName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, appName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String appName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, appName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String appName) { + return new PagedIterable<>(listAsync(resourceGroupName, appName)); + } + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String appName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, appName, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesImpl.java new file mode 100644 index 0000000000000..d1884526629d3 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResilienciesImpl.java @@ -0,0 +1,156 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.AppResilienciesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner; +import com.azure.resourcemanager.appcontainers.models.AppResiliencies; +import com.azure.resourcemanager.appcontainers.models.AppResiliency; + +public final class AppResilienciesImpl implements AppResiliencies { + private static final ClientLogger LOGGER = new ClientLogger(AppResilienciesImpl.class); + + private final AppResilienciesClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public AppResilienciesImpl(AppResilienciesClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response deleteWithResponse(String resourceGroupName, String appName, String name, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, appName, name, context); + } + + public void delete(String resourceGroupName, String appName, String name) { + this.serviceClient().delete(resourceGroupName, appName, name); + } + + public Response getWithResponse(String resourceGroupName, String appName, String name, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, appName, name, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new AppResiliencyImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public AppResiliency get(String resourceGroupName, String appName, String name) { + AppResiliencyInner inner = this.serviceClient().get(resourceGroupName, appName, name); + if (inner != null) { + return new AppResiliencyImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable list(String resourceGroupName, String appName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, appName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AppResiliencyImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String appName, Context context) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, appName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AppResiliencyImpl(inner1, this.manager())); + } + + public AppResiliency getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String appName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); + if (appName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + return this.getWithResponse(resourceGroupName, appName, name, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String appName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); + if (appName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + return this.getWithResponse(resourceGroupName, appName, name, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String appName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); + if (appName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + this.deleteWithResponse(resourceGroupName, appName, name, Context.NONE); + } + + public Response deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String appName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); + if (appName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + return this.deleteWithResponse(resourceGroupName, appName, name, context); + } + + private AppResilienciesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public AppResiliencyImpl define(String name) { + return new AppResiliencyImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResiliencyImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResiliencyImpl.java new file mode 100644 index 0000000000000..2d715f1928f4a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AppResiliencyImpl.java @@ -0,0 +1,171 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner; +import com.azure.resourcemanager.appcontainers.models.AppResiliency; +import com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy; +import com.azure.resourcemanager.appcontainers.models.HttpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.HttpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TcpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy; + +public final class AppResiliencyImpl implements AppResiliency, AppResiliency.Definition, AppResiliency.Update { + private AppResiliencyInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public TimeoutPolicy timeoutPolicy() { + return this.innerModel().timeoutPolicy(); + } + + public HttpRetryPolicy httpRetryPolicy() { + return this.innerModel().httpRetryPolicy(); + } + + public TcpRetryPolicy tcpRetryPolicy() { + return this.innerModel().tcpRetryPolicy(); + } + + public CircuitBreakerPolicy circuitBreakerPolicy() { + return this.innerModel().circuitBreakerPolicy(); + } + + public HttpConnectionPool httpConnectionPool() { + return this.innerModel().httpConnectionPool(); + } + + public TcpConnectionPool tcpConnectionPool() { + return this.innerModel().tcpConnectionPool(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public AppResiliencyInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String appName; + + private String name; + + public AppResiliencyImpl withExistingContainerApp(String resourceGroupName, String appName) { + this.resourceGroupName = resourceGroupName; + this.appName = appName; + return this; + } + + public AppResiliency create() { + this.innerObject = serviceManager.serviceClient().getAppResiliencies() + .createOrUpdateWithResponse(resourceGroupName, appName, name, this.innerModel(), Context.NONE).getValue(); + return this; + } + + public AppResiliency create(Context context) { + this.innerObject = serviceManager.serviceClient().getAppResiliencies() + .createOrUpdateWithResponse(resourceGroupName, appName, name, this.innerModel(), context).getValue(); + return this; + } + + AppResiliencyImpl(String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new AppResiliencyInner(); + this.serviceManager = serviceManager; + this.name = name; + } + + public AppResiliencyImpl update() { + return this; + } + + public AppResiliency apply() { + this.innerObject = serviceManager.serviceClient().getAppResiliencies() + .updateWithResponse(resourceGroupName, appName, name, this.innerModel(), Context.NONE).getValue(); + return this; + } + + public AppResiliency apply(Context context) { + this.innerObject = serviceManager.serviceClient().getAppResiliencies() + .updateWithResponse(resourceGroupName, appName, name, this.innerModel(), context).getValue(); + return this; + } + + AppResiliencyImpl(AppResiliencyInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.appName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "containerApps"); + this.name = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resiliencyPolicies"); + } + + public AppResiliency refresh() { + this.innerObject = serviceManager.serviceClient().getAppResiliencies() + .getWithResponse(resourceGroupName, appName, name, Context.NONE).getValue(); + return this; + } + + public AppResiliency refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getAppResiliencies() + .getWithResponse(resourceGroupName, appName, name, context).getValue(); + return this; + } + + public AppResiliencyImpl withTimeoutPolicy(TimeoutPolicy timeoutPolicy) { + this.innerModel().withTimeoutPolicy(timeoutPolicy); + return this; + } + + public AppResiliencyImpl withHttpRetryPolicy(HttpRetryPolicy httpRetryPolicy) { + this.innerModel().withHttpRetryPolicy(httpRetryPolicy); + return this; + } + + public AppResiliencyImpl withTcpRetryPolicy(TcpRetryPolicy tcpRetryPolicy) { + this.innerModel().withTcpRetryPolicy(tcpRetryPolicy); + return this; + } + + public AppResiliencyImpl withCircuitBreakerPolicy(CircuitBreakerPolicy circuitBreakerPolicy) { + this.innerModel().withCircuitBreakerPolicy(circuitBreakerPolicy); + return this; + } + + public AppResiliencyImpl withHttpConnectionPool(HttpConnectionPool httpConnectionPool) { + this.innerModel().withHttpConnectionPool(httpConnectionPool); + return this; + } + + public AppResiliencyImpl withTcpConnectionPool(TcpConnectionPool tcpConnectionPool) { + this.innerModel().withTcpConnectionPool(tcpConnectionPool); + return this; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AuthConfigImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AuthConfigImpl.java index 9db849c965eb3..579bbf3d7b604 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AuthConfigImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AuthConfigImpl.java @@ -9,6 +9,7 @@ import com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigInner; import com.azure.resourcemanager.appcontainers.models.AuthConfig; import com.azure.resourcemanager.appcontainers.models.AuthPlatform; +import com.azure.resourcemanager.appcontainers.models.EncryptionSettings; import com.azure.resourcemanager.appcontainers.models.GlobalValidation; import com.azure.resourcemanager.appcontainers.models.HttpSettings; import com.azure.resourcemanager.appcontainers.models.IdentityProviders; @@ -55,6 +56,10 @@ public HttpSettings httpSettings() { return this.innerModel().httpSettings(); } + public EncryptionSettings encryptionSettings() { + return this.innerModel().encryptionSettings(); + } + public String resourceGroupName() { return resourceGroupName; } @@ -80,24 +85,16 @@ public AuthConfigImpl withExistingContainerApp(String resourceGroupName, String } public AuthConfig create() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsAuthConfigs() - .createOrUpdateWithResponse( - resourceGroupName, containerAppName, authConfigName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getContainerAppsAuthConfigs().createOrUpdateWithResponse(resourceGroupName, + containerAppName, authConfigName, this.innerModel(), Context.NONE).getValue(); return this; } public AuthConfig create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsAuthConfigs() - .createOrUpdateWithResponse( - resourceGroupName, containerAppName, authConfigName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerAppsAuthConfigs() + .createOrUpdateWithResponse(resourceGroupName, containerAppName, authConfigName, this.innerModel(), context) + .getValue(); return this; } @@ -112,53 +109,37 @@ public AuthConfigImpl update() { } public AuthConfig apply() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsAuthConfigs() - .createOrUpdateWithResponse( - resourceGroupName, containerAppName, authConfigName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getContainerAppsAuthConfigs().createOrUpdateWithResponse(resourceGroupName, + containerAppName, authConfigName, this.innerModel(), Context.NONE).getValue(); return this; } public AuthConfig apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsAuthConfigs() - .createOrUpdateWithResponse( - resourceGroupName, containerAppName, authConfigName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerAppsAuthConfigs() + .createOrUpdateWithResponse(resourceGroupName, containerAppName, authConfigName, this.innerModel(), context) + .getValue(); return this; } - AuthConfigImpl( - AuthConfigInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + AuthConfigImpl(AuthConfigInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.containerAppName = Utils.getValueFromIdByName(innerObject.id(), "containerApps"); - this.authConfigName = Utils.getValueFromIdByName(innerObject.id(), "authConfigs"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.containerAppName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "containerApps"); + this.authConfigName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "authConfigs"); } public AuthConfig refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsAuthConfigs() - .getWithResponse(resourceGroupName, containerAppName, authConfigName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerAppsAuthConfigs() + .getWithResponse(resourceGroupName, containerAppName, authConfigName, Context.NONE).getValue(); return this; } public AuthConfig refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsAuthConfigs() - .getWithResponse(resourceGroupName, containerAppName, authConfigName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerAppsAuthConfigs() + .getWithResponse(resourceGroupName, containerAppName, authConfigName, context).getValue(); return this; } @@ -186,4 +167,9 @@ public AuthConfigImpl withHttpSettings(HttpSettings httpSettings) { this.innerModel().withHttpSettings(httpSettings); return this; } + + public AuthConfigImpl withEncryptionSettings(EncryptionSettings encryptionSettings) { + this.innerModel().withEncryptionSettings(encryptionSettings); + return this; + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfileImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfileImpl.java index c43cffd130a17..a4636a09d35b5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfileImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfileImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.AvailableWorkloadProfileInner; import com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfile; import com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfileProperties; @@ -13,8 +14,7 @@ public final class AvailableWorkloadProfileImpl implements AvailableWorkloadProf private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - AvailableWorkloadProfileImpl( - AvailableWorkloadProfileInner innerObject, + AvailableWorkloadProfileImpl(AvailableWorkloadProfileInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -40,6 +40,10 @@ public AvailableWorkloadProfileProperties properties() { return this.innerModel().properties(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public AvailableWorkloadProfileInner innerModel() { return this.innerObject; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesClientImpl.java index 15172ec96e9cd..8b2591bf07888 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesClientImpl.java @@ -30,24 +30,28 @@ import com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfilesCollection; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AvailableWorkloadProfilesClient. */ +/** + * An instance of this class provides access to all the operations defined in AvailableWorkloadProfilesClient. + */ public final class AvailableWorkloadProfilesClientImpl implements AvailableWorkloadProfilesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AvailableWorkloadProfilesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of AvailableWorkloadProfilesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AvailableWorkloadProfilesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - AvailableWorkloadProfilesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AvailableWorkloadProfilesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -58,109 +62,79 @@ public final class AvailableWorkloadProfilesClientImpl implements AvailableWorkl @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface AvailableWorkloadProfilesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.App/locations/{location}/availableManagedEnvironmentsWorkloadProfileTypes") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/locations/{location}/availableManagedEnvironmentsWorkloadProfileTypes") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("location") String location, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("location") String location, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> getNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all available workload profiles for a location along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getSinglePageAsync(String location) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - location, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), location, + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all available workload profiles for a location along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getSinglePageAsync(String location, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); @@ -168,29 +142,17 @@ private Mono> getSinglePageAsync(St final String accept = "application/json"; context = this.client.mergeContext(context); return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - location, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .get(this.client.getEndpoint(), this.client.getSubscriptionId(), location, this.client.getApiVersion(), + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -204,9 +166,9 @@ private PagedFlux getAsync(String location) { /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -216,15 +178,15 @@ private PagedFlux getAsync(String location) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux getAsync(String location, Context context) { - return new PagedFlux<>( - () -> getSinglePageAsync(location, context), nextLink -> getNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> getSinglePageAsync(location, context), + nextLink -> getNextSinglePageAsync(nextLink, context)); } /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -238,9 +200,9 @@ public PagedIterable get(String location) { /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -255,14 +217,15 @@ public PagedIterable get(String location, Context /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return collection of available workload profiles in the location along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getNextSinglePageAsync(String nextLink) { @@ -270,62 +233,43 @@ private Mono> getNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.getNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.getNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return collection of available workload profiles in the location along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getNextSinglePageAsync( - String nextLink, Context context) { + private Mono> getNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.getNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesImpl.java index 51a0cb9f72df7..de9fd579bec61 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/AvailableWorkloadProfilesImpl.java @@ -19,8 +19,7 @@ public final class AvailableWorkloadProfilesImpl implements AvailableWorkloadPro private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public AvailableWorkloadProfilesImpl( - AvailableWorkloadProfilesClient innerClient, + public AvailableWorkloadProfilesImpl(AvailableWorkloadProfilesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public AvailableWorkloadProfilesImpl( public PagedIterable get(String location) { PagedIterable inner = this.serviceClient().get(location); - return Utils.mapPage(inner, inner1 -> new AvailableWorkloadProfileImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AvailableWorkloadProfileImpl(inner1, this.manager())); } public PagedIterable get(String location, Context context) { PagedIterable inner = this.serviceClient().get(location, context); - return Utils.mapPage(inner, inner1 -> new AvailableWorkloadProfileImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AvailableWorkloadProfileImpl(inner1, this.manager())); } private AvailableWorkloadProfilesClient serviceClient() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMeterCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMeterCollectionImpl.java index 309f306eadbb2..b87bf3c7ed11a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMeterCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMeterCollectionImpl.java @@ -15,8 +15,7 @@ public final class BillingMeterCollectionImpl implements BillingMeterCollection private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - BillingMeterCollectionImpl( - BillingMeterCollectionInner innerObject, + BillingMeterCollectionImpl(BillingMeterCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersClientImpl.java index ca38979de6dc2..5d7a6a26b2298 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersClientImpl.java @@ -25,22 +25,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in BillingMetersClient. */ +/** + * An instance of this class provides access to all the operations defined in BillingMetersClient. + */ public final class BillingMetersClientImpl implements BillingMetersClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final BillingMetersService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of BillingMetersClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ BillingMetersClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(BillingMetersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(BillingMetersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -51,24 +57,20 @@ public final class BillingMetersClientImpl implements BillingMetersClient { @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface BillingMetersService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/locations/{location}/billingMeters") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("location") String location, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("location") String location, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -78,40 +80,28 @@ Mono> get( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String location) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - location, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), location, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -122,37 +112,27 @@ private Mono> getWithResponseAsync(String @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String location, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - location, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), location, + this.client.getApiVersion(), accept, context); } /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -166,9 +146,9 @@ private Mono getAsync(String location) { /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -183,9 +163,9 @@ public Response getWithResponse(String location, Co /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersImpl.java index 5032c704cca84..8b793dcb1ac00 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BillingMetersImpl.java @@ -20,8 +20,7 @@ public final class BillingMetersImpl implements BillingMeters { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public BillingMetersImpl( - BillingMetersClient innerClient, + public BillingMetersImpl(BillingMetersClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,10 +29,7 @@ public BillingMetersImpl( public Response getWithResponse(String location, Context context) { Response inner = this.serviceClient().getWithResponse(location, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new BillingMeterCollectionImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensClientImpl.java new file mode 100644 index 0000000000000..416f699371baa --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensClientImpl.java @@ -0,0 +1,202 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.BuildAuthTokensClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in BuildAuthTokensClient. + */ +public final class BuildAuthTokensClientImpl implements BuildAuthTokensClient { + /** + * The proxy service used to perform REST calls. + */ + private final BuildAuthTokensService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of BuildAuthTokensClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + BuildAuthTokensClientImpl(ContainerAppsApiClientImpl client) { + this.service + = RestProxy.create(BuildAuthTokensService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientBuildAuthTokens to be used by the proxy + * service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface BuildAuthTokensService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}/builds/{buildName}/listAuthToken") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @PathParam("buildName") String buildName, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(String resourceGroupName, String builderName, + String buildName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, buildName, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(String resourceGroupName, String builderName, + String buildName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, builderName, buildName, accept, context); + } + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAsync(String resourceGroupName, String builderName, String buildName) { + return listWithResponseAsync(resourceGroupName, builderName, buildName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build along with + * {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listWithResponse(String resourceGroupName, String builderName, String buildName, + Context context) { + return listWithResponseAsync(resourceGroupName, builderName, buildName, context).block(); + } + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuildTokenInner list(String resourceGroupName, String builderName, String buildName) { + return listWithResponse(resourceGroupName, builderName, buildName, Context.NONE).getValue(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensImpl.java new file mode 100644 index 0000000000000..be90dbf624920 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildAuthTokensImpl.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.BuildAuthTokensClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner; +import com.azure.resourcemanager.appcontainers.models.BuildAuthTokens; +import com.azure.resourcemanager.appcontainers.models.BuildToken; + +public final class BuildAuthTokensImpl implements BuildAuthTokens { + private static final ClientLogger LOGGER = new ClientLogger(BuildAuthTokensImpl.class); + + private final BuildAuthTokensClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public BuildAuthTokensImpl(BuildAuthTokensClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response listWithResponse(String resourceGroupName, String builderName, String buildName, + Context context) { + Response inner + = this.serviceClient().listWithResponse(resourceGroupName, builderName, buildName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new BuildTokenImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public BuildToken list(String resourceGroupName, String builderName, String buildName) { + BuildTokenInner inner = this.serviceClient().list(resourceGroupName, builderName, buildName); + if (inner != null) { + return new BuildTokenImpl(inner, this.manager()); + } else { + return null; + } + } + + private BuildAuthTokensClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildResourceImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildResourceImpl.java new file mode 100644 index 0000000000000..da4354a5db345 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildResourceImpl.java @@ -0,0 +1,154 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuildConfiguration; +import com.azure.resourcemanager.appcontainers.models.BuildProvisioningState; +import com.azure.resourcemanager.appcontainers.models.BuildResource; +import com.azure.resourcemanager.appcontainers.models.BuildStatus; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage; + +public final class BuildResourceImpl implements BuildResource, BuildResource.Definition, BuildResource.Update { + private BuildResourceInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public BuildProvisioningState provisioningState() { + return this.innerModel().provisioningState(); + } + + public BuildStatus buildStatus() { + return this.innerModel().buildStatus(); + } + + public ContainerRegistryWithCustomImage destinationContainerRegistry() { + return this.innerModel().destinationContainerRegistry(); + } + + public BuildConfiguration configuration() { + return this.innerModel().configuration(); + } + + public String uploadEndpoint() { + return this.innerModel().uploadEndpoint(); + } + + public String logStreamEndpoint() { + return this.innerModel().logStreamEndpoint(); + } + + public String tokenEndpoint() { + return this.innerModel().tokenEndpoint(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public BuildResourceInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String builderName; + + private String buildName; + + public BuildResourceImpl withExistingBuilder(String resourceGroupName, String builderName) { + this.resourceGroupName = resourceGroupName; + this.builderName = builderName; + return this; + } + + public BuildResource create() { + this.innerObject = serviceManager.serviceClient().getBuilds().createOrUpdate(resourceGroupName, builderName, + buildName, this.innerModel(), Context.NONE); + return this; + } + + public BuildResource create(Context context) { + this.innerObject = serviceManager.serviceClient().getBuilds().createOrUpdate(resourceGroupName, builderName, + buildName, this.innerModel(), context); + return this; + } + + BuildResourceImpl(String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new BuildResourceInner(); + this.serviceManager = serviceManager; + this.buildName = name; + } + + public BuildResourceImpl update() { + return this; + } + + public BuildResource apply() { + this.innerObject = serviceManager.serviceClient().getBuilds().createOrUpdate(resourceGroupName, builderName, + buildName, this.innerModel(), Context.NONE); + return this; + } + + public BuildResource apply(Context context) { + this.innerObject = serviceManager.serviceClient().getBuilds().createOrUpdate(resourceGroupName, builderName, + buildName, this.innerModel(), context); + return this; + } + + BuildResourceImpl(BuildResourceInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.builderName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "builders"); + this.buildName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "builds"); + } + + public BuildResource refresh() { + this.innerObject = serviceManager.serviceClient().getBuilds() + .getWithResponse(resourceGroupName, builderName, buildName, Context.NONE).getValue(); + return this; + } + + public BuildResource refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getBuilds() + .getWithResponse(resourceGroupName, builderName, buildName, context).getValue(); + return this; + } + + public BuildResourceImpl + withDestinationContainerRegistry(ContainerRegistryWithCustomImage destinationContainerRegistry) { + this.innerModel().withDestinationContainerRegistry(destinationContainerRegistry); + return this; + } + + public BuildResourceImpl withConfiguration(BuildConfiguration configuration) { + this.innerModel().withConfiguration(configuration); + return this; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildTokenImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildTokenImpl.java new file mode 100644 index 0000000000000..cd7d3b4913708 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildTokenImpl.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner; +import com.azure.resourcemanager.appcontainers.models.BuildToken; +import java.time.OffsetDateTime; + +public final class BuildTokenImpl implements BuildToken { + private BuildTokenInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + BuildTokenImpl(BuildTokenInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String token() { + return this.innerModel().token(); + } + + public OffsetDateTime expires() { + return this.innerModel().expires(); + } + + public BuildTokenInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuilderResourceImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuilderResourceImpl.java new file mode 100644 index 0000000000000..9926d63f18bcc --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuilderResourceImpl.java @@ -0,0 +1,209 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.Region; +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuilderProvisioningState; +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import com.azure.resourcemanager.appcontainers.models.BuilderResourceUpdate; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public final class BuilderResourceImpl implements BuilderResource, BuilderResource.Definition, BuilderResource.Update { + private BuilderResourceInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public String location() { + return this.innerModel().location(); + } + + public Map tags() { + Map inner = this.innerModel().tags(); + if (inner != null) { + return Collections.unmodifiableMap(inner); + } else { + return Collections.emptyMap(); + } + } + + public ManagedServiceIdentity identity() { + return this.innerModel().identity(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public BuilderProvisioningState provisioningState() { + return this.innerModel().provisioningState(); + } + + public String environmentId() { + return this.innerModel().environmentId(); + } + + public List containerRegistries() { + List inner = this.innerModel().containerRegistries(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public Region region() { + return Region.fromName(this.regionName()); + } + + public String regionName() { + return this.location(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public BuilderResourceInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String builderName; + + private BuilderResourceUpdate updateBuilderEnvelope; + + public BuilderResourceImpl withExistingResourceGroup(String resourceGroupName) { + this.resourceGroupName = resourceGroupName; + return this; + } + + public BuilderResource create() { + this.innerObject = serviceManager.serviceClient().getBuilders().createOrUpdate(resourceGroupName, builderName, + this.innerModel(), Context.NONE); + return this; + } + + public BuilderResource create(Context context) { + this.innerObject = serviceManager.serviceClient().getBuilders().createOrUpdate(resourceGroupName, builderName, + this.innerModel(), context); + return this; + } + + BuilderResourceImpl(String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new BuilderResourceInner(); + this.serviceManager = serviceManager; + this.builderName = name; + } + + public BuilderResourceImpl update() { + this.updateBuilderEnvelope = new BuilderResourceUpdate(); + return this; + } + + public BuilderResource apply() { + this.innerObject = serviceManager.serviceClient().getBuilders().update(resourceGroupName, builderName, + updateBuilderEnvelope, Context.NONE); + return this; + } + + public BuilderResource apply(Context context) { + this.innerObject = serviceManager.serviceClient().getBuilders().update(resourceGroupName, builderName, + updateBuilderEnvelope, context); + return this; + } + + BuilderResourceImpl(BuilderResourceInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.builderName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "builders"); + } + + public BuilderResource refresh() { + this.innerObject = serviceManager.serviceClient().getBuilders() + .getByResourceGroupWithResponse(resourceGroupName, builderName, Context.NONE).getValue(); + return this; + } + + public BuilderResource refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getBuilders() + .getByResourceGroupWithResponse(resourceGroupName, builderName, context).getValue(); + return this; + } + + public BuilderResourceImpl withRegion(Region location) { + this.innerModel().withLocation(location.toString()); + return this; + } + + public BuilderResourceImpl withRegion(String location) { + this.innerModel().withLocation(location); + return this; + } + + public BuilderResourceImpl withTags(Map tags) { + if (isInCreateMode()) { + this.innerModel().withTags(tags); + return this; + } else { + this.updateBuilderEnvelope.withTags(tags); + return this; + } + } + + public BuilderResourceImpl withIdentity(ManagedServiceIdentity identity) { + if (isInCreateMode()) { + this.innerModel().withIdentity(identity); + return this; + } else { + this.updateBuilderEnvelope.withIdentity(identity); + return this; + } + } + + public BuilderResourceImpl withEnvironmentId(String environmentId) { + if (isInCreateMode()) { + this.innerModel().withEnvironmentId(environmentId); + return this; + } else { + this.updateBuilderEnvelope.withEnvironmentId(environmentId); + return this; + } + } + + public BuilderResourceImpl withContainerRegistries(List containerRegistries) { + this.innerModel().withContainerRegistries(containerRegistries); + return this; + } + + private boolean isInCreateMode() { + return this.innerModel().id() == null; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersClientImpl.java new file mode 100644 index 0000000000000..9387f812cdd8f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersClientImpl.java @@ -0,0 +1,1290 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.appcontainers.fluent.BuildersClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuilderCollection; +import com.azure.resourcemanager.appcontainers.models.BuilderResourceUpdate; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in BuildersClient. + */ +public final class BuildersClientImpl implements BuildersClient { + /** + * The proxy service used to perform REST calls. + */ + private final BuildersService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of BuildersClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + BuildersClientImpl(ContainerAppsApiClientImpl client) { + this.service = RestProxy.create(BuildersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientBuilders to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface BuildersService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/builders") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @BodyParam("application/json") BuilderResourceInner builderEnvelope, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @BodyParam("application/json") BuilderResourceUpdate builderEnvelope, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}") + @ExpectedResponses({ 202, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listBySubscriptionNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByResourceGroupNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * List BuilderResource resources by subscription ID. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync() { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List BuilderResource resources by subscription ID. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, + context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List BuilderResource resources by subscription ID. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync() { + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + } + + /** + * List BuilderResource resources by subscription ID. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + } + + /** + * List BuilderResource resources by subscription ID. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list() { + return new PagedIterable<>(listAsync()); + } + + /** + * List BuilderResource resources by subscription ID. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(Context context) { + return new PagedIterable<>(listAsync(context)); + } + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), + nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); + } + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); + } + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByResourceGroup(String resourceGroupName) { + return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName)); + } + + /** + * List BuilderResource resources by resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { + return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, context)); + } + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String builderName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String builderName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, accept, context); + } + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getByResourceGroupAsync(String resourceGroupName, String builderName) { + return getByResourceGroupWithResponseAsync(resourceGroupName, builderName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getByResourceGroupWithResponse(String resourceGroupName, String builderName, + Context context) { + return getByResourceGroupWithResponseAsync(resourceGroupName, builderName, context).block(); + } + + /** + * Get a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuilderResource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuilderResourceInner getByResourceGroup(String resourceGroupName, String builderName) { + return getByResourceGroupWithResponse(resourceGroupName, builderName, Context.NONE).getValue(); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource along with {@link Response} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String builderName, BuilderResourceInner builderEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (builderEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter builderEnvelope is required and cannot be null.")); + } else { + builderEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, builderEnvelope, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource along with {@link Response} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String builderName, BuilderResourceInner builderEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (builderEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter builderEnvelope is required and cannot be null.")); + } else { + builderEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, builderEnvelope, accept, context); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, BuilderResourceInner> + beginCreateOrUpdateAsync(String resourceGroupName, String builderName, BuilderResourceInner builderEnvelope) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, builderName, builderEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + BuilderResourceInner.class, BuilderResourceInner.class, this.client.getContext()); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, BuilderResourceInner> beginCreateOrUpdateAsync( + String resourceGroupName, String builderName, BuilderResourceInner builderEnvelope, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, builderName, builderEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + BuilderResourceInner.class, BuilderResourceInner.class, context); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, BuilderResourceInner> + beginCreateOrUpdate(String resourceGroupName, String builderName, BuilderResourceInner builderEnvelope) { + return this.beginCreateOrUpdateAsync(resourceGroupName, builderName, builderEnvelope).getSyncPoller(); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, BuilderResourceInner> beginCreateOrUpdate( + String resourceGroupName, String builderName, BuilderResourceInner builderEnvelope, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, builderName, builderEnvelope, context).getSyncPoller(); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String builderName, + BuilderResourceInner builderEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, builderName, builderEnvelope).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String builderName, + BuilderResourceInner builderEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, builderName, builderEnvelope, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuilderResourceInner createOrUpdate(String resourceGroupName, String builderName, + BuilderResourceInner builderEnvelope) { + return createOrUpdateAsync(resourceGroupName, builderName, builderEnvelope).block(); + } + + /** + * Create or update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope Resource create parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuilderResourceInner createOrUpdate(String resourceGroupName, String builderName, + BuilderResourceInner builderEnvelope, Context context) { + return createOrUpdateAsync(resourceGroupName, builderName, builderEnvelope, context).block(); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource along with {@link Response} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String builderName, + BuilderResourceUpdate builderEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (builderEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter builderEnvelope is required and cannot be null.")); + } else { + builderEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, builderEnvelope, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource along with {@link Response} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String builderName, + BuilderResourceUpdate builderEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (builderEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter builderEnvelope is required and cannot be null.")); + } else { + builderEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, builderName, builderEnvelope, accept, context); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, BuilderResourceInner> + beginUpdateAsync(String resourceGroupName, String builderName, BuilderResourceUpdate builderEnvelope) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, builderName, builderEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + BuilderResourceInner.class, BuilderResourceInner.class, this.client.getContext()); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, BuilderResourceInner> beginUpdateAsync( + String resourceGroupName, String builderName, BuilderResourceUpdate builderEnvelope, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, builderName, builderEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + BuilderResourceInner.class, BuilderResourceInner.class, context); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, BuilderResourceInner> beginUpdate(String resourceGroupName, + String builderName, BuilderResourceUpdate builderEnvelope) { + return this.beginUpdateAsync(resourceGroupName, builderName, builderEnvelope).getSyncPoller(); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, BuilderResourceInner> beginUpdate(String resourceGroupName, + String builderName, BuilderResourceUpdate builderEnvelope, Context context) { + return this.beginUpdateAsync(resourceGroupName, builderName, builderEnvelope, context).getSyncPoller(); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String builderName, + BuilderResourceUpdate builderEnvelope) { + return beginUpdateAsync(resourceGroupName, builderName, builderEnvelope).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String builderName, + BuilderResourceUpdate builderEnvelope, Context context) { + return beginUpdateAsync(resourceGroupName, builderName, builderEnvelope, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuilderResourceInner update(String resourceGroupName, String builderName, + BuilderResourceUpdate builderEnvelope) { + return updateAsync(resourceGroupName, builderName, builderEnvelope).block(); + } + + /** + * Update a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param builderEnvelope The resource properties to be updated. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information about the SourceToCloud builder resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuilderResourceInner update(String resourceGroupName, String builderName, + BuilderResourceUpdate builderEnvelope, Context context) { + return updateAsync(resourceGroupName, builderName, builderEnvelope, context).block(); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String builderName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String builderName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, builderName, accept, context); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String builderName) { + Mono>> mono = deleteWithResponseAsync(resourceGroupName, builderName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String builderName, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono = deleteWithResponseAsync(resourceGroupName, builderName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String builderName) { + return this.beginDeleteAsync(resourceGroupName, builderName).getSyncPoller(); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String builderName, + Context context) { + return this.beginDeleteAsync(resourceGroupName, builderName, context).getSyncPoller(); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String builderName) { + return beginDeleteAsync(resourceGroupName, builderName).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String builderName, Context context) { + return beginDeleteAsync(resourceGroupName, builderName, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String builderName) { + deleteAsync(resourceGroupName, builderName).block(); + } + + /** + * Delete a BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String builderName, Context context) { + deleteAsync(resourceGroupName, builderName, context).block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersImpl.java new file mode 100644 index 0000000000000..afb724f88cafe --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildersImpl.java @@ -0,0 +1,147 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.BuildersClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import com.azure.resourcemanager.appcontainers.models.Builders; + +public final class BuildersImpl implements Builders { + private static final ClientLogger LOGGER = new ClientLogger(BuildersImpl.class); + + private final BuildersClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public BuildersImpl(BuildersClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list() { + PagedIterable inner = this.serviceClient().list(); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BuilderResourceImpl(inner1, this.manager())); + } + + public PagedIterable list(Context context) { + PagedIterable inner = this.serviceClient().list(context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BuilderResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByResourceGroup(String resourceGroupName) { + PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BuilderResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BuilderResourceImpl(inner1, this.manager())); + } + + public Response getByResourceGroupWithResponse(String resourceGroupName, String builderName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, builderName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new BuilderResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public BuilderResource getByResourceGroup(String resourceGroupName, String builderName) { + BuilderResourceInner inner = this.serviceClient().getByResourceGroup(resourceGroupName, builderName); + if (inner != null) { + return new BuilderResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + public void deleteByResourceGroup(String resourceGroupName, String builderName) { + this.serviceClient().delete(resourceGroupName, builderName); + } + + public void delete(String resourceGroupName, String builderName, Context context) { + this.serviceClient().delete(resourceGroupName, builderName, context); + } + + public BuilderResource getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + return this.getByResourceGroupWithResponse(resourceGroupName, builderName, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + return this.getByResourceGroupWithResponse(resourceGroupName, builderName, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + this.delete(resourceGroupName, builderName, Context.NONE); + } + + public void deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + this.delete(resourceGroupName, builderName, context); + } + + private BuildersClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public BuilderResourceImpl define(String name) { + return new BuilderResourceImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesClientImpl.java new file mode 100644 index 0000000000000..d1239550cfb15 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesClientImpl.java @@ -0,0 +1,278 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.BuildsByBuilderResourcesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuildCollection; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in BuildsByBuilderResourcesClient. + */ +public final class BuildsByBuilderResourcesClientImpl implements BuildsByBuilderResourcesClient { + /** + * The proxy service used to perform REST calls. + */ + private final BuildsByBuilderResourcesService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of BuildsByBuilderResourcesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + BuildsByBuilderResourcesClientImpl(ContainerAppsApiClientImpl client) { + this.service = RestProxy.create(BuildsByBuilderResourcesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientBuildsByBuilderResources to be used by the + * proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface BuildsByBuilderResourcesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}/builds") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + } + + /** + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, String builderName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, String builderName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, builderName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String builderName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, builderName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String builderName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, builderName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String builderName) { + return new PagedIterable<>(listAsync(resourceGroupName, builderName)); + } + + /** + * List BuildResource resources by BuilderResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String builderName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, builderName, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesImpl.java new file mode 100644 index 0000000000000..d1222c2dc55d9 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsByBuilderResourcesImpl.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.BuildsByBuilderResourcesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuildResource; +import com.azure.resourcemanager.appcontainers.models.BuildsByBuilderResources; + +public final class BuildsByBuilderResourcesImpl implements BuildsByBuilderResources { + private static final ClientLogger LOGGER = new ClientLogger(BuildsByBuilderResourcesImpl.class); + + private final BuildsByBuilderResourcesClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public BuildsByBuilderResourcesImpl(BuildsByBuilderResourcesClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String builderName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, builderName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BuildResourceImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String builderName, Context context) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, builderName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BuildResourceImpl(inner1, this.manager())); + } + + private BuildsByBuilderResourcesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsClientImpl.java similarity index 53% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsClientImpl.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsClientImpl.java index 2f31d271503b8..c297118316aca 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsClientImpl.java @@ -2,8 +2,9 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.implementation; +package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.annotation.BodyParam; import com.azure.core.annotation.Delete; import com.azure.core.annotation.ExpectedResponses; import com.azure.core.annotation.Get; @@ -12,15 +13,12 @@ import com.azure.core.annotation.Host; import com.azure.core.annotation.HostParam; import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; import com.azure.core.annotation.QueryParam; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceInterface; import com.azure.core.annotation.ServiceMethod; import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; import com.azure.core.http.rest.Response; import com.azure.core.http.rest.RestProxy; import com.azure.core.management.exception.ManagementException; @@ -29,94 +27,86 @@ import com.azure.core.util.FluxUtil; import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.AccountBackupsClient; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.models.BackupsList; +import com.azure.resourcemanager.appcontainers.fluent.BuildsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; import java.nio.ByteBuffer; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; /** - * An instance of this class provides access to all the operations defined in AccountBackupsClient. + * An instance of this class provides access to all the operations defined in BuildsClient. */ -public final class AccountBackupsClientImpl implements AccountBackupsClient { +public final class BuildsClientImpl implements BuildsClient { /** * The proxy service used to perform REST calls. */ - private final AccountBackupsService service; + private final BuildsService service; /** * The service client containing this operation class. */ - private final NetAppManagementClientImpl client; + private final ContainerAppsApiClientImpl client; /** - * Initializes an instance of AccountBackupsClientImpl. + * Initializes an instance of BuildsClientImpl. * * @param client the instance of the service client containing this operation class. */ - AccountBackupsClientImpl(NetAppManagementClientImpl client) { - this.service - = RestProxy.create(AccountBackupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + BuildsClientImpl(ContainerAppsApiClientImpl client) { + this.service = RestProxy.create(BuildsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for NetAppManagementClientAccountBackups to be used by the proxy service - * to perform REST calls. + * The interface defining all the services for ContainerAppsApiClientBuilds to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "NetAppManagementClie") - public interface AccountBackupsService { + @ServiceInterface(name = "ContainerAppsApiClie") + public interface BuildsService { @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/accountBackups") + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}/builds/{buildName}") @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByNetAppAccount(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @QueryParam("api-version") String apiVersion, - @QueryParam("includeOnlyBackupsFromDeletedVolumes") String includeOnlyBackupsFromDeletedVolumes, - @HeaderParam("Accept") String accept, Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @PathParam("buildName") String buildName, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/accountBackups/{backupName}") - @ExpectedResponses({ 200 }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}/builds/{buildName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupName") String backupName, @QueryParam("api-version") String apiVersion, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @PathParam("buildName") String buildName, @BodyParam("application/json") BuildResourceInner buildEnvelope, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) - @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/accountBackups/{backupName}") + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/builders/{builderName}/builds/{buildName}") @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> delete(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupName") String backupName, @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("builderName") String builderName, + @PathParam("buildName") String buildName, @HeaderParam("Accept") String accept, Context context); } /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. + * Get a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param includeOnlyBackupsFromDeletedVolumes An option to specify whether to return backups only from deleted - * volumes. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups along with {@link PagedResponse} on successful completion of {@link Mono}. + * @return a BuildResource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNetAppAccountSinglePageAsync(String resourceGroupName, - String accountName, String includeOnlyBackupsFromDeletedVolumes) { + private Mono> getWithResponseAsync(String resourceGroupName, String builderName, + String buildName) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -129,37 +119,34 @@ private Mono> listByNetAppAccountSinglePageAsync(Stri return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext(context -> service.listByNetAppAccount(this.client.getEndpoint(), - this.client.getSubscriptionId(), resourceGroupName, accountName, this.client.getApiVersion(), - includeOnlyBackupsFromDeletedVolumes, accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), - res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, buildName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. + * Get a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param includeOnlyBackupsFromDeletedVolumes An option to specify whether to return backups only from deleted - * volumes. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups along with {@link PagedResponse} on successful completion of {@link Mono}. + * @return a BuildResource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNetAppAccountSinglePageAsync(String resourceGroupName, - String accountName, String includeOnlyBackupsFromDeletedVolumes, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String builderName, + String buildName, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -172,138 +159,85 @@ private Mono> listByNetAppAccountSinglePageAsync(Stri return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByNetAppAccount(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, this.client.getApiVersion(), includeOnlyBackupsFromDeletedVolumes, accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), null, null)); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, builderName, buildName, accept, context); } /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. + * Get a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param includeOnlyBackupsFromDeletedVolumes An option to specify whether to return backups only from deleted - * volumes. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedFlux}. + * @return a BuildResource on successful completion of {@link Mono}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByNetAppAccountAsync(String resourceGroupName, String accountName, - String includeOnlyBackupsFromDeletedVolumes) { - return new PagedFlux<>(() -> listByNetAppAccountSinglePageAsync(resourceGroupName, accountName, - includeOnlyBackupsFromDeletedVolumes)); - } - - /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByNetAppAccountAsync(String resourceGroupName, String accountName) { - final String includeOnlyBackupsFromDeletedVolumes = null; - return new PagedFlux<>(() -> listByNetAppAccountSinglePageAsync(resourceGroupName, accountName, - includeOnlyBackupsFromDeletedVolumes)); + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String builderName, String buildName) { + return getWithResponseAsync(resourceGroupName, builderName, buildName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. + * Get a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param includeOnlyBackupsFromDeletedVolumes An option to specify whether to return backups only from deleted - * volumes. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByNetAppAccountAsync(String resourceGroupName, String accountName, - String includeOnlyBackupsFromDeletedVolumes, Context context) { - return new PagedFlux<>(() -> listByNetAppAccountSinglePageAsync(resourceGroupName, accountName, - includeOnlyBackupsFromDeletedVolumes, context)); - } - - /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. + * @return a BuildResource along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName) { - final String includeOnlyBackupsFromDeletedVolumes = null; - return new PagedIterable<>( - listByNetAppAccountAsync(resourceGroupName, accountName, includeOnlyBackupsFromDeletedVolumes)); + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String builderName, String buildName, + Context context) { + return getWithResponseAsync(resourceGroupName, builderName, buildName, context).block(); } /** - * List Backups for a Netapp Account - * - * List all Backups for a Netapp Account. + * Get a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param includeOnlyBackupsFromDeletedVolumes An option to specify whether to return backups only from deleted - * volumes. - * @param context The context to associate with this operation. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. + * @return a BuildResource. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, - String includeOnlyBackupsFromDeletedVolumes, Context context) { - return new PagedIterable<>( - listByNetAppAccountAsync(resourceGroupName, accountName, includeOnlyBackupsFromDeletedVolumes, context)); + @ServiceMethod(returns = ReturnType.SINGLE) + public BuildResourceInner get(String resourceGroupName, String builderName, String buildName) { + return getWithResponse(resourceGroupName, builderName, buildName, Context.NONE).getValue(); } /** - * Get Backup for a Netapp Account - * - * Gets the specified backup for a Netapp Account. + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account along with {@link Response} on successful completion of + * @return information pertaining to an individual build along with {@link Response} on successful completion of * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, String accountName, - String backupName) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String builderName, String buildName, BuildResourceInner buildEnvelope) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -316,37 +250,42 @@ private Mono> getWithResponseAsync(String resourceGroupNam return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); + if (buildEnvelope == null) { + return Mono.error(new IllegalArgumentException("Parameter buildEnvelope is required and cannot be null.")); + } else { + buildEnvelope.validate(); } final String accept = "application/json"; return FluxUtil - .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupName, this.client.getApiVersion(), accept, context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, buildName, buildEnvelope, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Get Backup for a Netapp Account - * - * Gets the specified backup for a Netapp Account. + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account along with {@link Response} on successful completion of + * @return information pertaining to an individual build along with {@link Response} on successful completion of * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, String accountName, - String backupName, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String builderName, String buildName, BuildResourceInner buildEnvelope, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -359,91 +298,196 @@ private Mono> getWithResponseAsync(String resourceGroupNam return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); + } + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); + if (buildEnvelope == null) { + return Mono.error(new IllegalArgumentException("Parameter buildEnvelope is required and cannot be null.")); + } else { + buildEnvelope.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, accountName, - backupName, this.client.getApiVersion(), accept, context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, buildName, buildEnvelope, accept, context); } /** - * Get Backup for a Netapp Account + * Create a BuildResource. * - * Gets the specified backup for a Netapp Account. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of information pertaining to an individual build. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, BuildResourceInner> beginCreateOrUpdateAsync( + String resourceGroupName, String builderName, String buildName, BuildResourceInner buildEnvelope) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, builderName, buildName, buildEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + BuildResourceInner.class, BuildResourceInner.class, this.client.getContext()); + } + + /** + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account on successful completion of {@link Mono}. + * @return the {@link PollerFlux} for polling of information pertaining to an individual build. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String resourceGroupName, String accountName, String backupName) { - return getWithResponseAsync(resourceGroupName, accountName, backupName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, BuildResourceInner> beginCreateOrUpdateAsync( + String resourceGroupName, String builderName, String buildName, BuildResourceInner buildEnvelope, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, builderName, buildName, buildEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + BuildResourceInner.class, BuildResourceInner.class, context); } /** - * Get Backup for a Netapp Account + * Create a BuildResource. * - * Gets the specified backup for a Netapp Account. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of information pertaining to an individual build. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, BuildResourceInner> beginCreateOrUpdate(String resourceGroupName, + String builderName, String buildName, BuildResourceInner buildEnvelope) { + return this.beginCreateOrUpdateAsync(resourceGroupName, builderName, buildName, buildEnvelope).getSyncPoller(); + } + + /** + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account along with {@link Response}. + * @return the {@link SyncPoller} for polling of information pertaining to an individual build. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, BuildResourceInner> beginCreateOrUpdate(String resourceGroupName, + String builderName, String buildName, BuildResourceInner buildEnvelope, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, builderName, buildName, buildEnvelope, context) + .getSyncPoller(); + } + + /** + * Create a BuildResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information pertaining to an individual build on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String resourceGroupName, String accountName, String backupName, - Context context) { - return getWithResponseAsync(resourceGroupName, accountName, backupName, context).block(); + private Mono createOrUpdateAsync(String resourceGroupName, String builderName, String buildName, + BuildResourceInner buildEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, builderName, buildName, buildEnvelope).last() + .flatMap(this.client::getLroFinalResultOrError); } /** - * Get Backup for a Netapp Account + * Create a BuildResource. * - * Gets the specified backup for a Netapp Account. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information pertaining to an individual build on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String builderName, String buildName, + BuildResourceInner buildEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, builderName, buildName, buildEnvelope, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Create a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified backup for a Netapp Account. + * @return information pertaining to an individual build. */ @ServiceMethod(returns = ReturnType.SINGLE) - public BackupInner get(String resourceGroupName, String accountName, String backupName) { - return getWithResponse(resourceGroupName, accountName, backupName, Context.NONE).getValue(); + public BuildResourceInner createOrUpdate(String resourceGroupName, String builderName, String buildName, + BuildResourceInner buildEnvelope) { + return createOrUpdateAsync(resourceGroupName, builderName, buildName, buildEnvelope).block(); } /** - * Delete Backup for a Netapp Account + * Create a BuildResource. * - * Delete the specified Backup for a Netapp Account. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param buildEnvelope Resource create or update parameters. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return information pertaining to an individual build. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BuildResourceInner createOrUpdate(String resourceGroupName, String builderName, String buildName, + BuildResourceInner buildEnvelope, Context context) { + return createOrUpdateAsync(resourceGroupName, builderName, buildName, buildEnvelope, context).block(); + } + + /** + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String accountName, - String backupName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String builderName, + String buildName) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -456,27 +500,25 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupName, this.client.getApiVersion(), accept, context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, builderName, buildName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -484,8 +526,8 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String accountName, - String backupName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String builderName, + String buildName, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -498,47 +540,43 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (builderName == null) { + return Mono.error(new IllegalArgumentException("Parameter builderName is required and cannot be null.")); } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); + if (buildName == null) { + return Mono.error(new IllegalArgumentException("Parameter buildName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupName, this.client.getApiVersion(), accept, context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, builderName, buildName, accept, context); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String accountName, - String backupName) { - Mono>> mono = deleteWithResponseAsync(resourceGroupName, accountName, backupName); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String builderName, + String buildName) { + Mono>> mono = deleteWithResponseAsync(resourceGroupName, builderName, buildName); return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -546,42 +584,38 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String accountName, - String backupName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String builderName, + String buildName, Context context) { context = this.client.mergeContext(context); Mono>> mono - = deleteWithResponseAsync(resourceGroupName, accountName, backupName, context); + = deleteWithResponseAsync(resourceGroupName, builderName, buildName, context); return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupName) { - return this.beginDeleteAsync(resourceGroupName, accountName, backupName).getSyncPoller(); + public SyncPoller, Void> beginDelete(String resourceGroupName, String builderName, + String buildName) { + return this.beginDeleteAsync(resourceGroupName, builderName, buildName).getSyncPoller(); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -589,38 +623,34 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupName, Context context) { - return this.beginDeleteAsync(resourceGroupName, accountName, backupName, context).getSyncPoller(); + public SyncPoller, Void> beginDelete(String resourceGroupName, String builderName, + String buildName, Context context) { + return this.beginDeleteAsync(resourceGroupName, builderName, buildName, context).getSyncPoller(); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String accountName, String backupName) { - return beginDeleteAsync(resourceGroupName, accountName, backupName).last() + private Mono deleteAsync(String resourceGroupName, String builderName, String buildName) { + return beginDeleteAsync(resourceGroupName, builderName, buildName).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -628,43 +658,39 @@ private Mono deleteAsync(String resourceGroupName, String accountName, Str * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String accountName, String backupName, Context context) { - return beginDeleteAsync(resourceGroupName, accountName, backupName, context).last() + private Mono deleteAsync(String resourceGroupName, String builderName, String buildName, Context context) { + return beginDeleteAsync(resourceGroupName, builderName, buildName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String accountName, String backupName) { - deleteAsync(resourceGroupName, accountName, backupName).block(); + public void delete(String resourceGroupName, String builderName, String buildName) { + deleteAsync(resourceGroupName, builderName, buildName).block(); } /** - * Delete Backup for a Netapp Account - * - * Delete the specified Backup for a Netapp Account. + * Delete a BuildResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupName The name of the backup. + * @param builderName The name of the builder. + * @param buildName The name of a build. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String accountName, String backupName, Context context) { - deleteAsync(resourceGroupName, accountName, backupName, context).block(); + public void delete(String resourceGroupName, String builderName, String buildName, Context context) { + deleteAsync(resourceGroupName, builderName, buildName, context).block(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsImpl.java new file mode 100644 index 0000000000000..7f29012d629e5 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/BuildsImpl.java @@ -0,0 +1,145 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.BuildsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuildResource; +import com.azure.resourcemanager.appcontainers.models.Builds; + +public final class BuildsImpl implements Builds { + private static final ClientLogger LOGGER = new ClientLogger(BuildsImpl.class); + + private final BuildsClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public BuildsImpl(BuildsClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response getWithResponse(String resourceGroupName, String builderName, String buildName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, builderName, buildName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new BuildResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public BuildResource get(String resourceGroupName, String builderName, String buildName) { + BuildResourceInner inner = this.serviceClient().get(resourceGroupName, builderName, buildName); + if (inner != null) { + return new BuildResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + public void delete(String resourceGroupName, String builderName, String buildName) { + this.serviceClient().delete(resourceGroupName, builderName, buildName); + } + + public void delete(String resourceGroupName, String builderName, String buildName, Context context) { + this.serviceClient().delete(resourceGroupName, builderName, buildName, context); + } + + public BuildResource getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + String buildName = ResourceManagerUtils.getValueFromIdByName(id, "builds"); + if (buildName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builds'.", id))); + } + return this.getWithResponse(resourceGroupName, builderName, buildName, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + String buildName = ResourceManagerUtils.getValueFromIdByName(id, "builds"); + if (buildName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builds'.", id))); + } + return this.getWithResponse(resourceGroupName, builderName, buildName, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + String buildName = ResourceManagerUtils.getValueFromIdByName(id, "builds"); + if (buildName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builds'.", id))); + } + this.delete(resourceGroupName, builderName, buildName, Context.NONE); + } + + public void deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String builderName = ResourceManagerUtils.getValueFromIdByName(id, "builders"); + if (builderName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builders'.", id))); + } + String buildName = ResourceManagerUtils.getValueFromIdByName(id, "builds"); + if (buildName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'builds'.", id))); + } + this.delete(resourceGroupName, builderName, buildName, context); + } + + private BuildsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public BuildResourceImpl define(String name) { + return new BuildResourceImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificateImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificateImpl.java index 2c083dfc0e7c1..237e1e0d2aa21 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificateImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificateImpl.java @@ -87,24 +87,17 @@ public CertificateImpl withExistingConnectedEnvironment(String resourceGroupName } public Certificate create() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsCertificates() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, certificateName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsCertificates() + .createOrUpdateWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, this.innerModel(), + Context.NONE) + .getValue(); return this; } public Certificate create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsCertificates() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, certificateName, this.innerModel(), context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectedEnvironmentsCertificates().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, certificateName, this.innerModel(), context).getValue(); return this; } @@ -120,57 +113,40 @@ public CertificateImpl update() { } public Certificate apply() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsCertificates() - .updateWithResponse( - resourceGroupName, - connectedEnvironmentName, - certificateName, - updateCertificateEnvelope, - Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsCertificates() + .updateWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, updateCertificateEnvelope, + Context.NONE) + .getValue(); return this; } public Certificate apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsCertificates() - .updateWithResponse( - resourceGroupName, connectedEnvironmentName, certificateName, updateCertificateEnvelope, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsCertificates() + .updateWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, updateCertificateEnvelope, + context) + .getValue(); return this; } - CertificateImpl( - CertificateInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + CertificateImpl(CertificateInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.connectedEnvironmentName = Utils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); - this.certificateName = Utils.getValueFromIdByName(innerObject.id(), "certificates"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.connectedEnvironmentName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); + this.certificateName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "certificates"); } public Certificate refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsCertificates() - .getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsCertificates() + .getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, Context.NONE).getValue(); return this; } public Certificate refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsCertificates() - .getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsCertificates() + .getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, context).getValue(); return this; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesClientImpl.java index 3e60f999c0ac5..3a3b7b71bd489 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesClientImpl.java @@ -35,144 +35,121 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CertificatesClient. */ +/** + * An instance of this class provides access to all the operations defined in CertificatesClient. + */ public final class CertificatesClientImpl implements CertificatesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CertificatesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of CertificatesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ CertificatesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(CertificatesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(CertificatesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for ContainerAppsApiClientCertificates to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for ContainerAppsApiClientCertificates to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface CertificatesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("certificateName") String certificateName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("certificateName") String certificateName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("certificateName") String certificateName, + @PathParam("environmentName") String environmentName, @PathParam("certificateName") String certificateName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") CertificateInner certificateEnvelope, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") CertificateInner certificateEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("certificateName") String certificateName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("certificateName") String certificateName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> update( - @HostParam("$host") String endpoint, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("certificateName") String certificateName, + @PathParam("environmentName") String environmentName, @PathParam("certificateName") String certificateName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") CertificatePatch certificateEnvelope, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") CertificatePatch certificateEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given managed environment along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String environmentName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -184,32 +161,16 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -217,22 +178,18 @@ private Mono> listSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given managed environment along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, String environmentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -245,28 +202,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -276,14 +220,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String environmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, environmentName), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -294,14 +237,13 @@ private PagedFlux listAsync(String resourceGroupName, String e */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String environmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, environmentName, context), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -316,7 +258,7 @@ public PagedIterable list(String resourceGroupName, String env /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -332,7 +274,7 @@ public PagedIterable list(String resourceGroupName, String env /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -342,19 +284,15 @@ public PagedIterable list(String resourceGroupName, String env * @return the specified Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String certificateName) { + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String certificateName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -370,24 +308,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, certificateName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -398,19 +326,15 @@ private Mono> getWithResponseAsync( * @return the specified Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String certificateName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String certificateName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -426,21 +350,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, certificateName, this.client.getApiVersion(), accept, context); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -457,7 +373,7 @@ private Mono getAsync(String resourceGroupName, String environ /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -468,14 +384,14 @@ private Mono getAsync(String resourceGroupName, String environ * @return the specified Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context) { + public Response getWithResponse(String resourceGroupName, String environmentName, + String certificateName, Context context) { return getWithResponseAsync(resourceGroupName, environmentName, certificateName, context).block(); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -491,7 +407,7 @@ public CertificateInner get(String resourceGroupName, String environmentName, St /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -499,26 +415,19 @@ public CertificateInner get(String resourceGroupName, String environmentName, St * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String certificateName, - CertificateInner certificateEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String certificateName, CertificateInner certificateEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -537,25 +446,15 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, certificateName, this.client.getApiVersion(), certificateEnvelope, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -564,27 +463,19 @@ private Mono> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String certificateName, - CertificateInner certificateEnvelope, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String certificateName, CertificateInner certificateEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -603,22 +494,13 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, certificateName, this.client.getApiVersion(), certificateEnvelope, accept, context); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -626,11 +508,11 @@ private Mono> createOrUpdateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String environmentName, String certificateName) { + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String certificateName) { final CertificateInner certificateEnvelope = null; return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, certificateName, certificateEnvelope) .flatMap(res -> Mono.justOrEmpty(res.getValue())); @@ -638,7 +520,7 @@ private Mono createOrUpdateAsync( /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -647,24 +529,19 @@ private Mono createOrUpdateAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificateInner certificateEnvelope, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, environmentName, certificateName, certificateEnvelope, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificateInner certificateEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, certificateName, certificateEnvelope, + context).block(); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -676,14 +553,13 @@ public Response createOrUpdateWithResponse( @ServiceMethod(returns = ReturnType.SINGLE) public CertificateInner createOrUpdate(String resourceGroupName, String environmentName, String certificateName) { final CertificateInner certificateEnvelope = null; - return createOrUpdateWithResponse( - resourceGroupName, environmentName, certificateName, certificateEnvelope, Context.NONE) - .getValue(); + return createOrUpdateWithResponse(resourceGroupName, environmentName, certificateName, certificateEnvelope, + Context.NONE).getValue(); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -693,19 +569,15 @@ public CertificateInner createOrUpdate(String resourceGroupName, String environm * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String certificateName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String certificateName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -721,24 +593,14 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, certificateName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -749,19 +611,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String certificateName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String certificateName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -777,21 +635,13 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, certificateName, this.client.getApiVersion(), accept, context); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -808,7 +658,7 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -819,14 +669,14 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, String certificateName, + Context context) { return deleteWithResponseAsync(resourceGroupName, environmentName, certificateName, context).block(); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -841,9 +691,9 @@ public void delete(String resourceGroupName, String environmentName, String cert /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -851,26 +701,19 @@ public void delete(String resourceGroupName, String environmentName, String cert * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String environmentName, - String certificateName, - CertificatePatch certificateEnvelope) { + private Mono> updateWithResponseAsync(String resourceGroupName, String environmentName, + String certificateName, CertificatePatch certificateEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -892,27 +735,17 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, certificateName, this.client.getApiVersion(), certificateEnvelope, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -921,27 +754,19 @@ private Mono> updateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String environmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, String environmentName, + String certificateName, CertificatePatch certificateEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -963,24 +788,15 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, certificateName, this.client.getApiVersion(), certificateEnvelope, accept, context); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -989,13 +805,10 @@ private Mono> updateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String environmentName, - String certificateName, + private Mono updateAsync(String resourceGroupName, String environmentName, String certificateName, CertificatePatch certificateEnvelope) { return updateWithResponseAsync(resourceGroupName, environmentName, certificateName, certificateEnvelope) .flatMap(res -> Mono.justOrEmpty(res.getValue())); @@ -1003,9 +816,9 @@ private Mono updateAsync( /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -1014,26 +827,21 @@ private Mono updateAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context) { - return updateWithResponseAsync( - resourceGroupName, environmentName, certificateName, certificateEnvelope, context) - .block(); + public Response updateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificatePatch certificateEnvelope, Context context) { + return updateWithResponseAsync(resourceGroupName, environmentName, certificateName, certificateEnvelope, + context).block(); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -1044,21 +852,18 @@ public Response updateWithResponse( * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CertificateInner update( - String resourceGroupName, - String environmentName, - String certificateName, + public CertificateInner update(String resourceGroupName, String environmentName, String certificateName, CertificatePatch certificateEnvelope) { - return updateWithResponse( - resourceGroupName, environmentName, certificateName, certificateEnvelope, Context.NONE) - .getValue(); + return updateWithResponse(resourceGroupName, environmentName, certificateName, certificateEnvelope, + Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1070,31 +875,22 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -1107,23 +903,13 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesImpl.java index 9a24cb875758a..bb48fdd2cdf8a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CertificatesImpl.java @@ -22,8 +22,7 @@ public final class CertificatesImpl implements Certificates { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public CertificatesImpl( - CertificatesClient innerClient, + public CertificatesImpl(CertificatesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -31,23 +30,20 @@ public CertificatesImpl( public PagedIterable list(String resourceGroupName, String environmentName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String environmentName, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName, context); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, environmentName, certificateName, context); + public Response getWithResponse(String resourceGroupName, String environmentName, + String certificateName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, certificateName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CertificateImpl(inner.getValue(), this.manager())); } else { return null; @@ -63,22 +59,12 @@ public Certificate get(String resourceGroupName, String environmentName, String } } - public Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificateInner certificateEnvelope, - Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateWithResponse( - resourceGroupName, environmentName, certificateName, certificateEnvelope, context); + public Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificateInner certificateEnvelope, Context context) { + Response inner = this.serviceClient().createOrUpdateWithResponse(resourceGroupName, + environmentName, certificateName, certificateEnvelope, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CertificateImpl(inner.getValue(), this.manager())); } else { return null; @@ -86,8 +72,8 @@ public Response createOrUpdateWithResponse( } public Certificate createOrUpdate(String resourceGroupName, String environmentName, String certificateName) { - CertificateInner inner = - this.serviceClient().createOrUpdate(resourceGroupName, environmentName, certificateName); + CertificateInner inner + = this.serviceClient().createOrUpdate(resourceGroupName, environmentName, certificateName); if (inner != null) { return new CertificateImpl(inner, this.manager()); } else { @@ -95,8 +81,8 @@ public Certificate createOrUpdate(String resourceGroupName, String environmentNa } } - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, String certificateName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, environmentName, certificateName, context); } @@ -104,34 +90,22 @@ public void delete(String resourceGroupName, String environmentName, String cert this.serviceClient().delete(resourceGroupName, environmentName, certificateName); } - public Response updateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context) { - Response inner = - this - .serviceClient() - .updateWithResponse(resourceGroupName, environmentName, certificateName, certificateEnvelope, context); + public Response updateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificatePatch certificateEnvelope, Context context) { + Response inner = this.serviceClient().updateWithResponse(resourceGroupName, environmentName, + certificateName, certificateEnvelope, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CertificateImpl(inner.getValue(), this.manager())); } else { return null; } } - public Certificate update( - String resourceGroupName, - String environmentName, - String certificateName, + public Certificate update(String resourceGroupName, String environmentName, String certificateName, CertificatePatch certificateEnvelope) { - CertificateInner inner = - this.serviceClient().update(resourceGroupName, environmentName, certificateName, certificateEnvelope); + CertificateInner inner + = this.serviceClient().update(resourceGroupName, environmentName, certificateName, certificateEnvelope); if (inner != null) { return new CertificateImpl(inner, this.manager()); } else { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CheckNameAvailabilityResponseImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CheckNameAvailabilityResponseImpl.java index 59b89b6157c12..fcc2727cde3dd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CheckNameAvailabilityResponseImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CheckNameAvailabilityResponseImpl.java @@ -13,8 +13,7 @@ public final class CheckNameAvailabilityResponseImpl implements CheckNameAvailab private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - CheckNameAvailabilityResponseImpl( - CheckNameAvailabilityResponseInner innerObject, + CheckNameAvailabilityResponseImpl(CheckNameAvailabilityResponseInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentImpl.java index 316012fa67ded..a733c450c07eb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentImpl.java @@ -111,25 +111,19 @@ public ConnectedEnvironmentImpl withExistingResourceGroup(String resourceGroupNa } public ConnectedEnvironment create() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironments() - .createOrUpdate(resourceGroupName, connectedEnvironmentName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironments().createOrUpdate(resourceGroupName, + connectedEnvironmentName, this.innerModel(), Context.NONE); return this; } public ConnectedEnvironment create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironments() - .createOrUpdate(resourceGroupName, connectedEnvironmentName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironments().createOrUpdate(resourceGroupName, + connectedEnvironmentName, this.innerModel(), context); return this; } - ConnectedEnvironmentImpl( - String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ConnectedEnvironmentImpl(String name, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = new ConnectedEnvironmentInner(); this.serviceManager = serviceManager; this.connectedEnvironmentName = name; @@ -140,65 +134,48 @@ public ConnectedEnvironmentImpl update() { } public ConnectedEnvironment apply() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironments() - .createOrUpdate(resourceGroupName, connectedEnvironmentName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironments().createOrUpdate(resourceGroupName, + connectedEnvironmentName, this.innerModel(), Context.NONE); return this; } public ConnectedEnvironment apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironments() - .createOrUpdate(resourceGroupName, connectedEnvironmentName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironments().createOrUpdate(resourceGroupName, + connectedEnvironmentName, this.innerModel(), context); return this; } - ConnectedEnvironmentImpl( - ConnectedEnvironmentInner innerObject, + ConnectedEnvironmentImpl(ConnectedEnvironmentInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.connectedEnvironmentName = Utils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.connectedEnvironmentName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); } public ConnectedEnvironment refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironments() - .getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironments() + .getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, Context.NONE).getValue(); return this; } public ConnectedEnvironment refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironments() - .getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironments() + .getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, context).getValue(); return this; } - public Response checkNameAvailabilityWithResponse( - CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { - return serviceManager - .connectedEnvironments() - .checkNameAvailabilityWithResponse( - resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest, context); + public Response + checkNameAvailabilityWithResponse(CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { + return serviceManager.connectedEnvironments().checkNameAvailabilityWithResponse(resourceGroupName, + connectedEnvironmentName, checkNameAvailabilityRequest, context); } - public CheckNameAvailabilityResponse checkNameAvailability( - CheckNameAvailabilityRequest checkNameAvailabilityRequest) { - return serviceManager - .connectedEnvironments() - .checkNameAvailability(resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest); + public CheckNameAvailabilityResponse + checkNameAvailability(CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + return serviceManager.connectedEnvironments().checkNameAvailability(resourceGroupName, connectedEnvironmentName, + checkNameAvailabilityRequest); } public ConnectedEnvironmentImpl withRegion(Region location) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStorageImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStorageImpl.java index 4e0adba8acb47..1dfa1dde464cf 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStorageImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStorageImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStorageInner; import com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentStorage; @@ -31,6 +32,10 @@ public ConnectedEnvironmentStorageProperties properties() { return this.innerModel().properties(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String resourceGroupName() { return resourceGroupName; } @@ -49,37 +54,27 @@ private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager( private String storageName; - public ConnectedEnvironmentStorageImpl withExistingConnectedEnvironment( - String resourceGroupName, String connectedEnvironmentName) { + public ConnectedEnvironmentStorageImpl withExistingConnectedEnvironment(String resourceGroupName, + String connectedEnvironmentName) { this.resourceGroupName = resourceGroupName; this.connectedEnvironmentName = connectedEnvironmentName; return this; } public ConnectedEnvironmentStorage create() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsStorages() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsStorages().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), Context.NONE).getValue(); return this; } public ConnectedEnvironmentStorage create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsStorages() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsStorages().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), context).getValue(); return this; } - ConnectedEnvironmentStorageImpl( - String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ConnectedEnvironmentStorageImpl(String name, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = new ConnectedEnvironmentStorageInner(); this.serviceManager = serviceManager; this.storageName = name; @@ -90,54 +85,36 @@ public ConnectedEnvironmentStorageImpl update() { } public ConnectedEnvironmentStorage apply() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsStorages() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsStorages().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), Context.NONE).getValue(); return this; } public ConnectedEnvironmentStorage apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsStorages() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsStorages().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, storageName, this.innerModel(), context).getValue(); return this; } - ConnectedEnvironmentStorageImpl( - ConnectedEnvironmentStorageInner innerObject, + ConnectedEnvironmentStorageImpl(ConnectedEnvironmentStorageInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.connectedEnvironmentName = Utils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); - this.storageName = Utils.getValueFromIdByName(innerObject.id(), "storages"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.connectedEnvironmentName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); + this.storageName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "storages"); } public ConnectedEnvironmentStorage refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsStorages() - .getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsStorages() + .getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, Context.NONE).getValue(); return this; } public ConnectedEnvironmentStorage refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsStorages() - .getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsStorages() + .getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context).getValue(); return this; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStoragesCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStoragesCollectionImpl.java index e3f77be64af18..98958a7e2466d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStoragesCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentStoragesCollectionImpl.java @@ -17,8 +17,7 @@ public final class ConnectedEnvironmentStoragesCollectionImpl implements Connect private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - ConnectedEnvironmentStoragesCollectionImpl( - ConnectedEnvironmentStoragesCollectionInner innerObject, + ConnectedEnvironmentStoragesCollectionImpl(ConnectedEnvironmentStoragesCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,9 @@ public final class ConnectedEnvironmentStoragesCollectionImpl implements Connect public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new ConnectedEnvironmentStorageImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new ConnectedEnvironmentStorageImpl(inner1, this.manager())) + .collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesClientImpl.java index 5e48209c01446..be1c333d2d808 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesClientImpl.java @@ -39,184 +39,140 @@ * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsCertificatesClient. */ public final class ConnectedEnvironmentsCertificatesClientImpl implements ConnectedEnvironmentsCertificatesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ConnectedEnvironmentsCertificatesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ConnectedEnvironmentsCertificatesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ConnectedEnvironmentsCertificatesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ConnectedEnvironmentsCertificatesService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ConnectedEnvironmentsCertificatesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for ContainerAppsApiClientConnectedEnvironmentsCertificates to be used by - * the proxy service to perform REST calls. + * The interface defining all the services for ContainerAppsApiClientConnectedEnvironmentsCertificates to be used + * by the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ConnectedEnvironmentsCertificatesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("certificateName") String certificateName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("certificateName") String certificateName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("certificateName") String certificateName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") CertificateInner certificateEnvelope, - @HeaderParam("Accept") String accept, + @PathParam("certificateName") String certificateName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") CertificateInner certificateEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("certificateName") String certificateName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("certificateName") String certificateName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/certificates/{certificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> update( - @HostParam("$host") String endpoint, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("certificateName") String certificateName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") CertificatePatch certificateEnvelope, - @HeaderParam("Accept") String accept, + @PathParam("certificateName") String certificateName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") CertificatePatch certificateEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given connected environment along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String connectedEnvironmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param context The context to associate with this operation. @@ -224,57 +180,39 @@ private Mono> listSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given connected environment along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -284,14 +222,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String connectedEnvironmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param context The context to associate with this operation. @@ -301,16 +238,15 @@ private PagedFlux listAsync(String resourceGroupName, String c * @return the Certificates in a given connected environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName, context), + private PagedFlux listAsync(String resourceGroupName, String connectedEnvironmentName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -325,7 +261,7 @@ public PagedIterable list(String resourceGroupName, String con /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param context The context to associate with this operation. @@ -335,14 +271,14 @@ public PagedIterable list(String resourceGroupName, String con * @return the Certificates in a given connected environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String connectedEnvironmentName, Context context) { + public PagedIterable list(String resourceGroupName, String connectedEnvironmentName, + Context context) { return new PagedIterable<>(listAsync(resourceGroupName, connectedEnvironmentName, context)); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -352,28 +288,23 @@ public PagedIterable list( * @return the specified Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String certificateName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String certificateName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -382,23 +313,14 @@ private Mono> getWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - accept, - context)) + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, certificateName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -409,28 +331,23 @@ private Mono> getWithResponseAsync( * @return the specified Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String certificateName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -438,21 +355,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, certificateName, this.client.getApiVersion(), accept, context); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -462,15 +371,15 @@ private Mono> getWithResponseAsync( * @return the specified Certificate on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String connectedEnvironmentName, String certificateName) { + private Mono getAsync(String resourceGroupName, String connectedEnvironmentName, + String certificateName) { return getWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -481,14 +390,14 @@ private Mono getAsync( * @return the specified Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context) { + public Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context) { return getWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName, context).block(); } /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -504,7 +413,7 @@ public CertificateInner get(String resourceGroupName, String connectedEnvironmen /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -512,35 +421,27 @@ public CertificateInner get(String resourceGroupName, String connectedEnvironmen * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificateInner certificateEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String certificateName, CertificateInner certificateEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -551,25 +452,15 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, certificateName, this.client.getApiVersion(), + certificateEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -578,36 +469,28 @@ private Mono> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificateInner certificateEnvelope, + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String certificateName, CertificateInner certificateEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -618,22 +501,14 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, certificateName, this.client.getApiVersion(), certificateEnvelope, accept, + context); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -641,20 +516,19 @@ private Mono> createOrUpdateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String connectedEnvironmentName, String certificateName) { + private Mono createOrUpdateAsync(String resourceGroupName, String connectedEnvironmentName, + String certificateName) { final CertificateInner certificateEnvelope = null; - return createOrUpdateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + return createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName, + certificateEnvelope).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -663,24 +537,20 @@ private Mono createOrUpdateAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificateInner certificateEnvelope, + public Response createOrUpdateWithResponse(String resourceGroupName, + String connectedEnvironmentName, String certificateName, CertificateInner certificateEnvelope, Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope, context) - .block(); + return createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName, + certificateEnvelope, context).block(); } /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -690,17 +560,16 @@ public Response createOrUpdateWithResponse( * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CertificateInner createOrUpdate( - String resourceGroupName, String connectedEnvironmentName, String certificateName) { + public CertificateInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + String certificateName) { final CertificateInner certificateEnvelope = null; - return createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope, Context.NONE) - .getValue(); + return createOrUpdateWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, + certificateEnvelope, Context.NONE).getValue(); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -710,28 +579,23 @@ public CertificateInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String certificateName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String connectedEnvironmentName, + String certificateName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -740,23 +604,14 @@ private Mono> deleteWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - accept, - context)) + context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, certificateName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -767,28 +622,23 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -796,21 +646,13 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, certificateName, this.client.getApiVersion(), accept, context); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -827,7 +669,7 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -838,14 +680,14 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context) { return deleteWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName, context).block(); } /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -860,9 +702,9 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, St /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -870,35 +712,27 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, St * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificatePatch certificateEnvelope) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String certificateName, CertificatePatch certificateEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -912,27 +746,17 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, certificateName, this.client.getApiVersion(), + certificateEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -941,36 +765,28 @@ private Mono> updateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response} on successful completion of {@link Mono}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificatePatch certificateEnvelope, + private Mono> updateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String certificateName, CertificatePatch certificateEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (certificateName == null) { return Mono @@ -984,24 +800,16 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - certificateName, - this.client.getApiVersion(), - certificateEnvelope, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, certificateName, this.client.getApiVersion(), certificateEnvelope, accept, + context); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -1010,24 +818,20 @@ private Mono> updateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificatePatch certificateEnvelope) { - return updateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono updateAsync(String resourceGroupName, String connectedEnvironmentName, + String certificateName, CertificatePatch certificateEnvelope) { + return updateWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName, + certificateEnvelope).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -1036,26 +840,21 @@ private Mono updateAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context) { - return updateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope, context) - .block(); + public Response updateWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, CertificatePatch certificateEnvelope, Context context) { + return updateWithResponseAsync(resourceGroupName, connectedEnvironmentName, certificateName, + certificateEnvelope, context).block(); } /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. @@ -1066,21 +865,18 @@ public Response updateWithResponse( * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CertificateInner update( - String resourceGroupName, - String connectedEnvironmentName, - String certificateName, + public CertificateInner update(String resourceGroupName, String connectedEnvironmentName, String certificateName, CertificatePatch certificateEnvelope) { - return updateWithResponse( - resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope, Context.NONE) - .getValue(); + return updateWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, certificateEnvelope, + Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1092,31 +888,22 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -1129,23 +916,13 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesImpl.java index 6c5a00ced3de9..599b752b8e5b5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsCertificatesImpl.java @@ -21,8 +21,7 @@ public final class ConnectedEnvironmentsCertificatesImpl implements ConnectedEnv private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ConnectedEnvironmentsCertificatesImpl( - ConnectedEnvironmentsCertificatesClient innerClient, + public ConnectedEnvironmentsCertificatesImpl(ConnectedEnvironmentsCertificatesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,24 +29,21 @@ public ConnectedEnvironmentsCertificatesImpl( public PagedIterable list(String resourceGroupName, String connectedEnvironmentName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, connectedEnvironmentName); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String connectedEnvironmentName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, connectedEnvironmentName, context); - return Utils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, connectedEnvironmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CertificateImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, context); + public Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + connectedEnvironmentName, certificateName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CertificateImpl(inner.getValue(), this.manager())); } else { return null; @@ -63,11 +59,10 @@ public Certificate get(String resourceGroupName, String connectedEnvironmentName } } - public Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, context); + public Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, + context); } public void delete(String resourceGroupName, String connectedEnvironmentName, String certificateName) { @@ -75,119 +70,78 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, St } public Certificate getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); } - return this - .getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, Context.NONE) + return this.getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); } return this.getWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); } this.deleteWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } - String certificateName = Utils.getValueFromIdByName(id, "certificates"); + String certificateName = ResourceManagerUtils.getValueFromIdByName(id, "certificates"); if (certificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'certificates'.", id))); } return this.deleteWithResponse(resourceGroupName, connectedEnvironmentName, certificateName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsClientImpl.java index e502ade54c632..ab3ab2053e3f9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsClientImpl.java @@ -42,23 +42,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ConnectedEnvironmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsClient. + */ public final class ConnectedEnvironmentsClientImpl implements ConnectedEnvironmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ConnectedEnvironmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ConnectedEnvironmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ConnectedEnvironmentsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create(ConnectedEnvironmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ConnectedEnvironmentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -69,228 +74,166 @@ public final class ConnectedEnvironmentsClientImpl implements ConnectedEnvironme @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ConnectedEnvironmentsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/connectedEnvironments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ConnectedEnvironmentInner environmentEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> update( - @HostParam("$host") String endpoint, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/checkNameAvailability") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/checkNameAvailability") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> checkNameAvailability( - @HostParam("$host") String endpoint, + Mono> checkNameAvailability(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") CheckNameAvailabilityRequest checkNameAvailabilityRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get all connectedEnvironments for a subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all connectedEnvironments for a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all connectedEnvironments for a subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Get all connectedEnvironments for a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -299,13 +242,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Get all connectedEnvironments for a subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription as paginated response with {@link PagedIterable}. @@ -317,7 +260,7 @@ public PagedIterable list() { /** * Get all connectedEnvironments for a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -331,28 +274,24 @@ public PagedIterable list(Context context) { /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments in a resource group along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -360,53 +299,34 @@ private Mono> listByResourceGroupSingle } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments in a resource group along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -415,27 +335,15 @@ private Mono> listByResourceGroupSingle final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -444,14 +352,13 @@ private Mono> listByResourceGroupSingle */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -461,14 +368,13 @@ private PagedFlux listByResourceGroupAsync(String res */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -482,7 +388,7 @@ public PagedIterable listByResourceGroup(String resou /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -497,107 +403,82 @@ public PagedIterable listByResourceGroup(String resou /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the properties of an connectedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return the properties of an connectedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the properties of an connectedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return the properties of an connectedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), accept, context); } /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -606,15 +487,15 @@ private Mono> getByResourceGroupWithResponse * @return the properties of an connectedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getByResourceGroupAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono getByResourceGroupAsync(String resourceGroupName, + String connectedEnvironmentName) { return getByResourceGroupWithResponseAsync(resourceGroupName, connectedEnvironmentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -624,14 +505,14 @@ private Mono getByResourceGroupAsync( * @return the properties of an connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, connectedEnvironmentName, context).block(); } /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -646,7 +527,7 @@ public ConnectedEnvironmentInner getByResourceGroup(String resourceGroupName, St /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -654,31 +535,26 @@ public ConnectedEnvironmentInner getByResourceGroup(String resourceGroupName, St * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (environmentEnvelope == null) { return Mono @@ -688,24 +564,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - environmentEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), environmentEnvelope, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -714,34 +581,26 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (environmentEnvelope == null) { return Mono @@ -751,21 +610,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - environmentEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), environmentEnvelope, accept, context); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -773,26 +624,21 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of an environment for Kubernetes cluster specialized for web workloads - * by Azure App Service. + * by Azure App Service. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ConnectedEnvironmentInner> beginCreateOrUpdateAsync( String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ConnectedEnvironmentInner.class, - ConnectedEnvironmentInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ConnectedEnvironmentInner.class, ConnectedEnvironmentInner.class, + this.client.getContext()); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -801,30 +647,22 @@ private PollerFlux, ConnectedEnvironmentIn * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of an environment for Kubernetes cluster specialized for web workloads - * by Azure App Service. + * by Azure App Service. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ConnectedEnvironmentInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, + String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ConnectedEnvironmentInner.class, - ConnectedEnvironmentInner.class, - context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + connectedEnvironmentName, environmentEnvelope, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ConnectedEnvironmentInner.class, ConnectedEnvironmentInner.class, context); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -832,19 +670,18 @@ private PollerFlux, ConnectedEnvironmentIn * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for Kubernetes cluster specialized for web workloads - * by Azure App Service. + * by Azure App Service. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ConnectedEnvironmentInner> beginCreateOrUpdate( String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope) + return this.beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope) .getSyncPoller(); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -853,22 +690,19 @@ public SyncPoller, ConnectedEnvironmentInn * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of an environment for Kubernetes cluster specialized for web workloads - * by Azure App Service. + * by Azure App Service. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ConnectedEnvironmentInner> beginCreateOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, + String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope, context) + return this.beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope, context) .getSyncPoller(); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -876,19 +710,18 @@ public SyncPoller, ConnectedEnvironmentInn * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { - return beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, + String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -897,22 +730,18 @@ private Mono createOrUpdateAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, - Context context) { + private Mono createOrUpdateAsync(String resourceGroupName, + String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope, Context context) { return beginCreateOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -922,14 +751,14 @@ private Mono createOrUpdateAsync( * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ConnectedEnvironmentInner createOrUpdate( - String resourceGroupName, String connectedEnvironmentName, ConnectedEnvironmentInner environmentEnvelope) { + public ConnectedEnvironmentInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + ConnectedEnvironmentInner environmentEnvelope) { return createOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope).block(); } /** * Creates or updates an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param environmentEnvelope Configuration details of the connectedEnvironment. @@ -940,17 +769,14 @@ public ConnectedEnvironmentInner createOrUpdate( * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ConnectedEnvironmentInner createOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - ConnectedEnvironmentInner environmentEnvelope, - Context context) { + public ConnectedEnvironmentInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + ConnectedEnvironmentInner environmentEnvelope, Context context) { return createOrUpdateAsync(resourceGroupName, connectedEnvironmentName, environmentEnvelope, context).block(); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -959,48 +785,34 @@ public ConnectedEnvironmentInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1010,45 +822,33 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), accept, context); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1057,18 +857,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String connectedEnvironmentName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String connectedEnvironmentName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, connectedEnvironmentName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1078,19 +876,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, connectedEnvironmentName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, connectedEnvironmentName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1105,7 +902,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1115,14 +912,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String connectedEnvironmentName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String connectedEnvironmentName, + Context context) { return this.beginDeleteAsync(resourceGroupName, connectedEnvironmentName, context).getSyncPoller(); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1132,14 +929,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String connectedEnvironmentName) { - return beginDeleteAsync(resourceGroupName, connectedEnvironmentName) - .last() + return beginDeleteAsync(resourceGroupName, connectedEnvironmentName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1150,14 +946,13 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String connectedEnvironmentName, Context context) { - return beginDeleteAsync(resourceGroupName, connectedEnvironmentName, context) - .last() + return beginDeleteAsync(resourceGroupName, connectedEnvironmentName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1171,7 +966,7 @@ public void delete(String resourceGroupName, String connectedEnvironmentName) { /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1186,62 +981,48 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, Co /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1249,57 +1030,45 @@ private Mono> updateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), accept, context); } /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono updateAsync(String resourceGroupName, String connectedEnvironmentName) { @@ -1309,9 +1078,9 @@ private Mono updateAsync(String resourceGroupName, St /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. @@ -1319,19 +1088,19 @@ private Mono updateAsync(String resourceGroupName, St * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context) { + public Response updateWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context) { return updateWithResponseAsync(resourceGroupName, connectedEnvironmentName, context).block(); } /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1346,9 +1115,9 @@ public ConnectedEnvironmentInner update(String resourceGroupName, String connect /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. @@ -1359,60 +1128,43 @@ public ConnectedEnvironmentInner update(String resourceGroupName, String connect */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> checkNameAvailabilityWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, + String resourceGroupName, String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (checkNameAvailabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter checkNameAvailabilityRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter checkNameAvailabilityRequest is required and cannot be null.")); } else { checkNameAvailabilityRequest.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - checkNameAvailabilityRequest, - accept, - context)) + .withContext(context -> service.checkNameAvailability(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, connectedEnvironmentName, + this.client.getApiVersion(), checkNameAvailabilityRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. @@ -1424,58 +1176,42 @@ private Mono> checkNameAvailability */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> checkNameAvailabilityWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context) { + String resourceGroupName, String connectedEnvironmentName, + CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (checkNameAvailabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter checkNameAvailabilityRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter checkNameAvailabilityRequest is required and cannot be null.")); } else { checkNameAvailabilityRequest.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - checkNameAvailabilityRequest, - accept, - context); + return service.checkNameAvailability(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), checkNameAvailabilityRequest, + accept, context); } /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. @@ -1485,20 +1221,17 @@ private Mono> checkNameAvailability * @return the check availability result on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono checkNameAvailabilityAsync( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest) { - return checkNameAvailabilityWithResponseAsync( - resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono checkNameAvailabilityAsync(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + return checkNameAvailabilityWithResponseAsync(resourceGroupName, connectedEnvironmentName, + checkNameAvailabilityRequest).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. @@ -1509,21 +1242,17 @@ private Mono checkNameAvailabilityAsync( * @return the check availability result along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context) { - return checkNameAvailabilityWithResponseAsync( - resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest, context) - .block(); + public Response checkNameAvailabilityWithResponse(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { + return checkNameAvailabilityWithResponseAsync(resourceGroupName, connectedEnvironmentName, + checkNameAvailabilityRequest, context).block(); } /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. @@ -1533,25 +1262,23 @@ public Response checkNameAvailabilityWithRes * @return the check availability result. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CheckNameAvailabilityResponseInner checkNameAvailability( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest) { - return checkNameAvailabilityWithResponse( - resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest, Context.NONE) - .getValue(); + public CheckNameAvailabilityResponseInner checkNameAvailability(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + return checkNameAvailabilityWithResponse(resourceGroupName, connectedEnvironmentName, + checkNameAvailabilityRequest, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -1559,76 +1286,59 @@ private Mono> listBySubscriptionNextSin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1636,63 +1346,45 @@ private Mono> listByResourceGroupNextSi return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of connectedEnvironments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsClientImpl.java index 7a8dd2baa954f..bc4defccdabf3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsClientImpl.java @@ -36,27 +36,28 @@ import reactor.core.publisher.Mono; /** - * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsDaprComponentsClient. + * An instance of this class provides access to all the operations defined in + * ConnectedEnvironmentsDaprComponentsClient. */ public final class ConnectedEnvironmentsDaprComponentsClientImpl implements ConnectedEnvironmentsDaprComponentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ConnectedEnvironmentsDaprComponentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ConnectedEnvironmentsDaprComponentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ConnectedEnvironmentsDaprComponentsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ConnectedEnvironmentsDaprComponentsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ConnectedEnvironmentsDaprComponentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -67,155 +68,112 @@ public final class ConnectedEnvironmentsDaprComponentsClientImpl implements Conn @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ConnectedEnvironmentsDaprComponentsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("componentName") String componentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, + @PathParam("componentName") String componentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") DaprComponentInner daprComponentEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("componentName") String componentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}/listSecrets") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/daprComponents/{componentName}/listSecrets") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listSecrets( - @HostParam("$host") String endpoint, + Mono> listSecrets(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("componentName") String componentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a connected environment along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String connectedEnvironmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param context The context to associate with this operation. @@ -223,57 +181,39 @@ private Mono> listSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a connected environment along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -283,14 +223,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String connectedEnvironmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param context The context to associate with this operation. @@ -300,16 +239,15 @@ private PagedFlux listAsync(String resourceGroupName, String * @return the Dapr Components for a connected environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName, context), + private PagedFlux listAsync(String resourceGroupName, String connectedEnvironmentName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, connectedEnvironmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -324,7 +262,7 @@ public PagedIterable list(String resourceGroupName, String c /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param context The context to associate with this operation. @@ -334,14 +272,14 @@ public PagedIterable list(String resourceGroupName, String c * @return the Dapr Components for a connected environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String connectedEnvironmentName, Context context) { + public PagedIterable list(String resourceGroupName, String connectedEnvironmentName, + Context context) { return new PagedIterable<>(listAsync(resourceGroupName, connectedEnvironmentName, context)); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -351,28 +289,23 @@ public PagedIterable list( * @return a dapr component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String componentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); @@ -380,23 +313,14 @@ private Mono> getWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - accept, - context)) + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, componentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -407,49 +331,36 @@ private Mono> getWithResponseAsync( * @return a dapr component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String componentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, componentName, this.client.getApiVersion(), accept, context); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -459,15 +370,15 @@ private Mono> getWithResponseAsync( * @return a dapr component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName) { + private Mono getAsync(String resourceGroupName, String connectedEnvironmentName, + String componentName) { return getWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -478,14 +389,14 @@ private Mono getAsync( * @return a dapr component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { + public Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context) { return getWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName, context).block(); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -501,9 +412,9 @@ public DaprComponentInner get(String resourceGroupName, String connectedEnvironm /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -514,31 +425,23 @@ public DaprComponentInner get(String resourceGroupName, String connectedEnvironm * @return dapr Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, - DaprComponentInner daprComponentEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String componentName, DaprComponentInner daprComponentEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); @@ -551,27 +454,17 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - daprComponentEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, componentName, this.client.getApiVersion(), + daprComponentEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -583,32 +476,24 @@ private Mono> createOrUpdateWithResponseAsync( * @return dapr Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String componentName, DaprComponentInner daprComponentEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); @@ -621,24 +506,16 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - daprComponentEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, componentName, this.client.getApiVersion(), daprComponentEnvelope, accept, + context); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -649,21 +526,17 @@ private Mono> createOrUpdateWithResponseAsync( * @return dapr Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, - DaprComponentInner daprComponentEnvelope) { - return createOrUpdateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, componentName, daprComponentEnvelope) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono createOrUpdateAsync(String resourceGroupName, String connectedEnvironmentName, + String componentName, DaprComponentInner daprComponentEnvelope) { + return createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName, + daprComponentEnvelope).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -675,22 +548,18 @@ private Mono createOrUpdateAsync( * @return dapr Component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, + public Response createOrUpdateWithResponse(String resourceGroupName, + String connectedEnvironmentName, String componentName, DaprComponentInner daprComponentEnvelope, Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, componentName, daprComponentEnvelope, context) - .block(); + return createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName, + daprComponentEnvelope, context).block(); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a connected environment. - * + * + * Creates or updates a Dapr Component in a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -701,21 +570,17 @@ public Response createOrUpdateWithResponse( * @return dapr Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DaprComponentInner createOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - String componentName, - DaprComponentInner daprComponentEnvelope) { - return createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, componentName, daprComponentEnvelope, Context.NONE) - .getValue(); + public DaprComponentInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + String componentName, DaprComponentInner daprComponentEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, connectedEnvironmentName, componentName, + daprComponentEnvelope, Context.NONE).getValue(); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -725,28 +590,23 @@ public DaprComponentInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String connectedEnvironmentName, + String componentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); @@ -754,25 +614,16 @@ private Mono> deleteWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - accept, - context)) + context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, componentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -783,51 +634,38 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, componentName, this.client.getApiVersion(), accept, context); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -844,9 +682,9 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -857,16 +695,16 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context) { return deleteWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName, context).block(); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -881,7 +719,7 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, St /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -889,55 +727,41 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, St * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String componentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, componentName, this.client.getApiVersion(), accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -946,52 +770,39 @@ private Mono> listSecretsWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String componentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (componentName == null) { return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - componentName, - this.client.getApiVersion(), - accept, - context); + return service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, componentName, this.client.getApiVersion(), accept, context); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -1001,15 +812,15 @@ private Mono> listSecretsWithResponseAsync( * @return dapr component Secrets Collection for ListSecrets Action on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listSecretsAsync( - String resourceGroupName, String connectedEnvironmentName, String componentName) { + private Mono listSecretsAsync(String resourceGroupName, String connectedEnvironmentName, + String componentName) { return listSecretsWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -1020,15 +831,15 @@ private Mono listSecretsAsync( * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listSecretsWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { + public Response listSecretsWithResponse(String resourceGroupName, + String connectedEnvironmentName, String componentName, Context context) { return listSecretsWithResponseAsync(resourceGroupName, connectedEnvironmentName, componentName, context) .block(); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. @@ -1038,17 +849,18 @@ public Response listSecretsWithResponse( * @return dapr component Secrets Collection for ListSecrets Action. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DaprSecretsCollectionInner listSecrets( - String resourceGroupName, String connectedEnvironmentName, String componentName) { + public DaprSecretsCollectionInner listSecrets(String resourceGroupName, String connectedEnvironmentName, + String componentName) { return listSecretsWithResponse(resourceGroupName, connectedEnvironmentName, componentName, Context.NONE) .getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1060,31 +872,22 @@ private Mono> listNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -1097,23 +900,13 @@ private Mono> listNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsImpl.java index a42732ed38b8c..eacdeaebf3bba 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsDaprComponentsImpl.java @@ -23,35 +23,31 @@ public final class ConnectedEnvironmentsDaprComponentsImpl implements ConnectedE private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ConnectedEnvironmentsDaprComponentsImpl( - ConnectedEnvironmentsDaprComponentsClient innerClient, + public ConnectedEnvironmentsDaprComponentsImpl(ConnectedEnvironmentsDaprComponentsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String resourceGroupName, String connectedEnvironmentName) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, connectedEnvironmentName); - return Utils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, connectedEnvironmentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); } - public PagedIterable list( - String resourceGroupName, String connectedEnvironmentName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, connectedEnvironmentName, context); - return Utils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String connectedEnvironmentName, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, connectedEnvironmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); + public Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DaprComponentImpl(inner.getValue(), this.manager())); } else { return null; @@ -67,38 +63,32 @@ public DaprComponent get(String resourceGroupName, String connectedEnvironmentNa } } - public Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); + public Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, connectedEnvironmentName, componentName, + context); } public void delete(String resourceGroupName, String connectedEnvironmentName, String componentName) { this.serviceClient().delete(resourceGroupName, connectedEnvironmentName, componentName); } - public Response listSecretsWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context) { - Response inner = - this - .serviceClient() - .listSecretsWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); + public Response listSecretsWithResponse(String resourceGroupName, + String connectedEnvironmentName, String componentName, Context context) { + Response inner = this.serviceClient().listSecretsWithResponse(resourceGroupName, + connectedEnvironmentName, componentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DaprSecretsCollectionImpl(inner.getValue(), this.manager())); } else { return null; } } - public DaprSecretsCollection listSecrets( - String resourceGroupName, String connectedEnvironmentName, String componentName) { - DaprSecretsCollectionInner inner = - this.serviceClient().listSecrets(resourceGroupName, connectedEnvironmentName, componentName); + public DaprSecretsCollection listSecrets(String resourceGroupName, String connectedEnvironmentName, + String componentName) { + DaprSecretsCollectionInner inner + = this.serviceClient().listSecrets(resourceGroupName, connectedEnvironmentName, componentName); if (inner != null) { return new DaprSecretsCollectionImpl(inner, this.manager()); } else { @@ -107,123 +97,78 @@ public DaprSecretsCollection listSecrets( } public DaprComponent getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String componentName = Utils.getValueFromIdByName(id, "daprComponents"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); if (componentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); - } - return this - .getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, Context.NONE) + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + } + return this.getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String componentName = Utils.getValueFromIdByName(id, "daprComponents"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); if (componentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); } return this.getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String componentName = Utils.getValueFromIdByName(id, "daprComponents"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); if (componentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); } this.deleteWithResponse(resourceGroupName, connectedEnvironmentName, componentName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String componentName = Utils.getValueFromIdByName(id, "daprComponents"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); if (componentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); } return this.deleteWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsImpl.java index 9bfc8b8b89fad..2919f62d27be6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsImpl.java @@ -24,8 +24,7 @@ public final class ConnectedEnvironmentsImpl implements ConnectedEnvironments { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ConnectedEnvironmentsImpl( - ConnectedEnvironmentsClient innerClient, + public ConnectedEnvironmentsImpl(ConnectedEnvironmentsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -33,34 +32,31 @@ public ConnectedEnvironmentsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConnectedEnvironmentImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ConnectedEnvironmentImpl(inner.getValue(), this.manager())); } else { return null; @@ -68,8 +64,8 @@ public Response getByResourceGroupWithResponse( } public ConnectedEnvironment getByResourceGroup(String resourceGroupName, String connectedEnvironmentName) { - ConnectedEnvironmentInner inner = - this.serviceClient().getByResourceGroup(resourceGroupName, connectedEnvironmentName); + ConnectedEnvironmentInner inner + = this.serviceClient().getByResourceGroup(resourceGroupName, connectedEnvironmentName); if (inner != null) { return new ConnectedEnvironmentImpl(inner, this.manager()); } else { @@ -85,15 +81,12 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, Co this.serviceClient().delete(resourceGroupName, connectedEnvironmentName, context); } - public Response updateWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context) { - Response inner = - this.serviceClient().updateWithResponse(resourceGroupName, connectedEnvironmentName, context); + public Response updateWithResponse(String resourceGroupName, String connectedEnvironmentName, + Context context) { + Response inner + = this.serviceClient().updateWithResponse(resourceGroupName, connectedEnvironmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ConnectedEnvironmentImpl(inner.getValue(), this.manager())); } else { return null; @@ -109,35 +102,22 @@ public ConnectedEnvironment update(String resourceGroupName, String connectedEnv } } - public Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context) { - Response inner = - this - .serviceClient() - .checkNameAvailabilityWithResponse( - resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest, context); + public Response checkNameAvailabilityWithResponse(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { + Response inner = this.serviceClient().checkNameAvailabilityWithResponse( + resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CheckNameAvailabilityResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public CheckNameAvailabilityResponse checkNameAvailability( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest) { - CheckNameAvailabilityResponseInner inner = - this - .serviceClient() - .checkNameAvailability(resourceGroupName, connectedEnvironmentName, checkNameAvailabilityRequest); + public CheckNameAvailabilityResponse checkNameAvailability(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + CheckNameAvailabilityResponseInner inner = this.serviceClient().checkNameAvailability(resourceGroupName, + connectedEnvironmentName, checkNameAvailabilityRequest); if (inner != null) { return new CheckNameAvailabilityResponseImpl(inner, this.manager()); } else { @@ -146,91 +126,58 @@ public CheckNameAvailabilityResponse checkNameAvailability( } public ConnectedEnvironment getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } - return this - .getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, Context.NONE) + return this.getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, connectedEnvironmentName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } this.delete(resourceGroupName, connectedEnvironmentName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); } this.delete(resourceGroupName, connectedEnvironmentName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesClientImpl.java index 511e12aa1d1b9..c44b673d95623 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesClientImpl.java @@ -29,200 +29,160 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ConnectedEnvironmentsStoragesClient. */ +/** + * An instance of this class provides access to all the operations defined in ConnectedEnvironmentsStoragesClient. + */ public final class ConnectedEnvironmentsStoragesClientImpl implements ConnectedEnvironmentsStoragesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ConnectedEnvironmentsStoragesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ConnectedEnvironmentsStoragesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ConnectedEnvironmentsStoragesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ConnectedEnvironmentsStoragesService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ConnectedEnvironmentsStoragesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for ContainerAppsApiClientConnectedEnvironmentsStorages to be used by the - * proxy service to perform REST calls. + * The interface defining all the services for ContainerAppsApiClientConnectedEnvironmentsStorages to be used by + * the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ConnectedEnvironmentsStoragesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages/{storageName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages/{storageName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("storageName") String storageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("storageName") String storageName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages/{storageName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages/{storageName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("storageName") String storageName, - @QueryParam("api-version") String apiVersion, + @PathParam("storageName") String storageName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ConnectedEnvironmentStorageInner storageEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages/{storageName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/connectedEnvironments/{connectedEnvironmentName}/storages/{storageName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("connectedEnvironmentName") String connectedEnvironmentName, - @PathParam("storageName") String storageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("storageName") String storageName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); } /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all storages for a connectedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return all storages for a connectedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono> listWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all storages for a connectedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return all storages for a connectedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, Context context) { + private Mono> listWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - this.client.getApiVersion(), - accept, - context); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, this.client.getApiVersion(), accept, context); } /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -231,15 +191,15 @@ private Mono> listWithResp * @return all storages for a connectedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listAsync( - String resourceGroupName, String connectedEnvironmentName) { + private Mono listAsync(String resourceGroupName, + String connectedEnvironmentName) { return listWithResponseAsync(resourceGroupName, connectedEnvironmentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param context The context to associate with this operation. @@ -249,14 +209,14 @@ private Mono listAsync( * @return all storages for a connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context) { + public Response listWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context) { return listWithResponseAsync(resourceGroupName, connectedEnvironmentName, context).block(); } /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -271,7 +231,7 @@ public ConnectedEnvironmentStoragesCollectionInner list(String resourceGroupName /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -281,52 +241,37 @@ public ConnectedEnvironmentStoragesCollectionInner list(String resourceGroupName * @return storage for a connectedEnvironment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String storageName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String storageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (storageName == null) { return Mono.error(new IllegalArgumentException("Parameter storageName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - storageName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, storageName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -337,49 +282,36 @@ private Mono> getWithResponseAsync( * @return storage for a connectedEnvironment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String storageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (storageName == null) { return Mono.error(new IllegalArgumentException("Parameter storageName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - storageName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, storageName, this.client.getApiVersion(), accept, context); } /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -389,15 +321,15 @@ private Mono> getWithResponseAsync( * @return storage for a connectedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String connectedEnvironmentName, String storageName) { + private Mono getAsync(String resourceGroupName, String connectedEnvironmentName, + String storageName) { return getWithResponseAsync(resourceGroupName, connectedEnvironmentName, storageName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -408,14 +340,14 @@ private Mono getAsync( * @return storage for a connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context) { + public Response getWithResponse(String resourceGroupName, + String connectedEnvironmentName, String storageName, Context context) { return getWithResponseAsync(resourceGroupName, connectedEnvironmentName, storageName, context).block(); } /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -425,14 +357,14 @@ public Response getWithResponse( * @return storage for a connectedEnvironment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ConnectedEnvironmentStorageInner get( - String resourceGroupName, String connectedEnvironmentName, String storageName) { + public ConnectedEnvironmentStorageInner get(String resourceGroupName, String connectedEnvironmentName, + String storageName) { return getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, Context.NONE).getValue(); } /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -440,35 +372,27 @@ public ConnectedEnvironmentStorageInner get( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return storage resource for connectedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return storage resource for connectedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String storageName, ConnectedEnvironmentStorageInner storageEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (storageName == null) { return Mono.error(new IllegalArgumentException("Parameter storageName is required and cannot be null.")); @@ -481,25 +405,15 @@ private Mono> createOrUpdateWithRespo } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - storageName, - this.client.getApiVersion(), - storageEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, storageName, this.client.getApiVersion(), storageEnvelope, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -508,36 +422,28 @@ private Mono> createOrUpdateWithRespo * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return storage resource for connectedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return storage resource for connectedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope, + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String connectedEnvironmentName, String storageName, ConnectedEnvironmentStorageInner storageEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (storageName == null) { return Mono.error(new IllegalArgumentException("Parameter storageName is required and cannot be null.")); @@ -550,22 +456,13 @@ private Mono> createOrUpdateWithRespo } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - storageName, - this.client.getApiVersion(), - storageEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, storageName, this.client.getApiVersion(), storageEnvelope, accept, context); } /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -576,19 +473,15 @@ private Mono> createOrUpdateWithRespo * @return storage resource for connectedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope) { - return createOrUpdateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, storageName, storageEnvelope) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono createOrUpdateAsync(String resourceGroupName, + String connectedEnvironmentName, String storageName, ConnectedEnvironmentStorageInner storageEnvelope) { + return createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, storageName, + storageEnvelope).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -600,20 +493,16 @@ private Mono createOrUpdateAsync( * @return storage resource for connectedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope, + public Response createOrUpdateWithResponse(String resourceGroupName, + String connectedEnvironmentName, String storageName, ConnectedEnvironmentStorageInner storageEnvelope, Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, connectedEnvironmentName, storageName, storageEnvelope, context) - .block(); + return createOrUpdateWithResponseAsync(resourceGroupName, connectedEnvironmentName, storageName, + storageEnvelope, context).block(); } /** * Create or update storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -624,19 +513,15 @@ public Response createOrUpdateWithResponse( * @return storage resource for connectedEnvironment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ConnectedEnvironmentStorageInner createOrUpdate( - String resourceGroupName, - String connectedEnvironmentName, - String storageName, - ConnectedEnvironmentStorageInner storageEnvelope) { - return createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, storageName, storageEnvelope, Context.NONE) - .getValue(); + public ConnectedEnvironmentStorageInner createOrUpdate(String resourceGroupName, String connectedEnvironmentName, + String storageName, ConnectedEnvironmentStorageInner storageEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, connectedEnvironmentName, storageName, storageEnvelope, + Context.NONE).getValue(); } /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -646,52 +531,37 @@ public ConnectedEnvironmentStorageInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String storageName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String connectedEnvironmentName, + String storageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (storageName == null) { return Mono.error(new IllegalArgumentException("Parameter storageName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - storageName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, connectedEnvironmentName, storageName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -702,49 +572,36 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String connectedEnvironmentName, + String storageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (connectedEnvironmentName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter connectedEnvironmentName is required and cannot be null.")); } if (storageName == null) { return Mono.error(new IllegalArgumentException("Parameter storageName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - connectedEnvironmentName, - storageName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + connectedEnvironmentName, storageName, this.client.getApiVersion(), accept, context); } /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -761,7 +618,7 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. @@ -772,14 +629,14 @@ private Mono deleteAsync(String resourceGroupName, String connectedEnviron * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, + String storageName, Context context) { return deleteWithResponseAsync(resourceGroupName, connectedEnvironmentName, storageName, context).block(); } /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesImpl.java index a6bcb614f45f6..3c5f66271c3b9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ConnectedEnvironmentsStoragesImpl.java @@ -22,22 +22,18 @@ public final class ConnectedEnvironmentsStoragesImpl implements ConnectedEnviron private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ConnectedEnvironmentsStoragesImpl( - ConnectedEnvironmentsStoragesClient innerClient, + public ConnectedEnvironmentsStoragesImpl(ConnectedEnvironmentsStoragesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response listWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context) { - Response inner = - this.serviceClient().listWithResponse(resourceGroupName, connectedEnvironmentName, context); + public Response listWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context) { + Response inner + = this.serviceClient().listWithResponse(resourceGroupName, connectedEnvironmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ConnectedEnvironmentStoragesCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -45,8 +41,8 @@ public Response listWithResponse( } public ConnectedEnvironmentStoragesCollection list(String resourceGroupName, String connectedEnvironmentName) { - ConnectedEnvironmentStoragesCollectionInner inner = - this.serviceClient().list(resourceGroupName, connectedEnvironmentName); + ConnectedEnvironmentStoragesCollectionInner inner + = this.serviceClient().list(resourceGroupName, connectedEnvironmentName); if (inner != null) { return new ConnectedEnvironmentStoragesCollectionImpl(inner, this.manager()); } else { @@ -54,25 +50,22 @@ public ConnectedEnvironmentStoragesCollection list(String resourceGroupName, Str } } - public Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context); + public Response getWithResponse(String resourceGroupName, + String connectedEnvironmentName, String storageName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ConnectedEnvironmentStorageImpl(inner.getValue(), this.manager())); } else { return null; } } - public ConnectedEnvironmentStorage get( - String resourceGroupName, String connectedEnvironmentName, String storageName) { - ConnectedEnvironmentStorageInner inner = - this.serviceClient().get(resourceGroupName, connectedEnvironmentName, storageName); + public ConnectedEnvironmentStorage get(String resourceGroupName, String connectedEnvironmentName, + String storageName) { + ConnectedEnvironmentStorageInner inner + = this.serviceClient().get(resourceGroupName, connectedEnvironmentName, storageName); if (inner != null) { return new ConnectedEnvironmentStorageImpl(inner, this.manager()); } else { @@ -80,11 +73,10 @@ public ConnectedEnvironmentStorage get( } } - public Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context); + public Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, + String storageName, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, connectedEnvironmentName, storageName, + context); } public void delete(String resourceGroupName, String connectedEnvironmentName, String storageName) { @@ -92,117 +84,77 @@ public void delete(String resourceGroupName, String connectedEnvironmentName, St } public ConnectedEnvironmentStorage getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } return this.getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } return this.getWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } this.deleteWithResponse(resourceGroupName, connectedEnvironmentName, storageName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String connectedEnvironmentName = Utils.getValueFromIdByName(id, "connectedEnvironments"); + String connectedEnvironmentName = ResourceManagerUtils.getValueFromIdByName(id, "connectedEnvironments"); if (connectedEnvironmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", - id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } return this.deleteWithResponse(resourceGroupName, connectedEnvironmentName, storageName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppAuthTokenImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppAuthTokenImpl.java index 186b92d42676a..c0b49215b5a4f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppAuthTokenImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppAuthTokenImpl.java @@ -16,8 +16,7 @@ public final class ContainerAppAuthTokenImpl implements ContainerAppAuthToken { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - ContainerAppAuthTokenImpl( - ContainerAppAuthTokenInner innerObject, + ContainerAppAuthTokenImpl(ContainerAppAuthTokenInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppImpl.java index a56a23c9a4394..1325dc07837e0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppImpl.java @@ -151,20 +151,14 @@ public ContainerAppImpl withExistingResourceGroup(String resourceGroupName) { } public ContainerApp create() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerApps() - .createOrUpdate(resourceGroupName, containerAppName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getContainerApps().createOrUpdate(resourceGroupName, + containerAppName, this.innerModel(), Context.NONE); return this; } public ContainerApp create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerApps() - .createOrUpdate(resourceGroupName, containerAppName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getContainerApps().createOrUpdate(resourceGroupName, + containerAppName, this.innerModel(), context); return this; } @@ -179,56 +173,41 @@ public ContainerAppImpl update() { } public ContainerApp apply() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerApps() - .update(resourceGroupName, containerAppName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getContainerApps().update(resourceGroupName, containerAppName, + this.innerModel(), Context.NONE); return this; } public ContainerApp apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerApps() - .update(resourceGroupName, containerAppName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getContainerApps().update(resourceGroupName, containerAppName, + this.innerModel(), context); return this; } - ContainerAppImpl( - ContainerAppInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ContainerAppImpl(ContainerAppInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.containerAppName = Utils.getValueFromIdByName(innerObject.id(), "containerApps"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.containerAppName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "containerApps"); } public ContainerApp refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerApps() - .getByResourceGroupWithResponse(resourceGroupName, containerAppName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerApps() + .getByResourceGroupWithResponse(resourceGroupName, containerAppName, Context.NONE).getValue(); return this; } public ContainerApp refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerApps() - .getByResourceGroupWithResponse(resourceGroupName, containerAppName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerApps() + .getByResourceGroupWithResponse(resourceGroupName, containerAppName, context).getValue(); return this; } - public Response listCustomHostnameAnalysisWithResponse( - String customHostname, Context context) { - return serviceManager - .containerApps() - .listCustomHostnameAnalysisWithResponse(resourceGroupName, containerAppName, customHostname, context); + public Response listCustomHostnameAnalysisWithResponse(String customHostname, + Context context) { + return serviceManager.containerApps().listCustomHostnameAnalysisWithResponse(resourceGroupName, + containerAppName, customHostname, context); } public CustomHostnameAnalysisResult listCustomHostnameAnalysis() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppJobExecutionsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppJobExecutionsImpl.java index 4fb1bf7119140..6f7c1e0d91d39 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppJobExecutionsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppJobExecutionsImpl.java @@ -17,8 +17,7 @@ public final class ContainerAppJobExecutionsImpl implements ContainerAppJobExecu private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - ContainerAppJobExecutionsImpl( - ContainerAppJobExecutionsInner innerObject, + ContainerAppJobExecutionsImpl(ContainerAppJobExecutionsInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,8 @@ public final class ContainerAppJobExecutionsImpl implements ContainerAppJobExecu public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new JobExecutionImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new JobExecutionImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientBuilder.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientBuilder.java index 67c65c220581c..49c7d2ab750ab 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientBuilder.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientBuilder.java @@ -14,17 +14,19 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the ContainerAppsApiClientImpl type. */ -@ServiceClientBuilder(serviceClients = {ContainerAppsApiClientImpl.class}) +/** + * A builder for creating a new instance of the ContainerAppsApiClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { ContainerAppsApiClientImpl.class }) public final class ContainerAppsApiClientBuilder { /* - * The ID of the target subscription. + * The ID of the target subscription. The value must be an UUID. */ private String subscriptionId; /** - * Sets The ID of the target subscription. - * + * Sets The ID of the target subscription. The value must be an UUID. + * * @param subscriptionId the subscriptionId value. * @return the ContainerAppsApiClientBuilder. */ @@ -40,7 +42,7 @@ public ContainerAppsApiClientBuilder subscriptionId(String subscriptionId) { /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the ContainerAppsApiClientBuilder. */ @@ -56,7 +58,7 @@ public ContainerAppsApiClientBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the ContainerAppsApiClientBuilder. */ @@ -72,7 +74,7 @@ public ContainerAppsApiClientBuilder environment(AzureEnvironment environment) { /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the ContainerAppsApiClientBuilder. */ @@ -88,7 +90,7 @@ public ContainerAppsApiClientBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the ContainerAppsApiClientBuilder. */ @@ -104,7 +106,7 @@ public ContainerAppsApiClientBuilder defaultPollInterval(Duration defaultPollInt /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the ContainerAppsApiClientBuilder. */ @@ -115,30 +117,20 @@ public ContainerAppsApiClientBuilder serializerAdapter(SerializerAdapter seriali /** * Builds an instance of ContainerAppsApiClientImpl with the provided parameters. - * + * * @return an instance of ContainerAppsApiClientImpl. */ public ContainerAppsApiClientImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - ContainerAppsApiClientImpl client = - new ContainerAppsApiClientImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - this.subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + ContainerAppsApiClientImpl client = new ContainerAppsApiClientImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientImpl.java index 3289bbba92736..1253a9deec72c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsApiClientImpl.java @@ -22,8 +22,13 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.serializer.SerializerAdapter; import com.azure.core.util.serializer.SerializerEncoding; +import com.azure.resourcemanager.appcontainers.fluent.AppResilienciesClient; import com.azure.resourcemanager.appcontainers.fluent.AvailableWorkloadProfilesClient; import com.azure.resourcemanager.appcontainers.fluent.BillingMetersClient; +import com.azure.resourcemanager.appcontainers.fluent.BuildAuthTokensClient; +import com.azure.resourcemanager.appcontainers.fluent.BuildersClient; +import com.azure.resourcemanager.appcontainers.fluent.BuildsByBuilderResourcesClient; +import com.azure.resourcemanager.appcontainers.fluent.BuildsClient; import com.azure.resourcemanager.appcontainers.fluent.CertificatesClient; import com.azure.resourcemanager.appcontainers.fluent.ConnectedEnvironmentsCertificatesClient; import com.azure.resourcemanager.appcontainers.fluent.ConnectedEnvironmentsClient; @@ -36,17 +41,23 @@ import com.azure.resourcemanager.appcontainers.fluent.ContainerAppsRevisionReplicasClient; import com.azure.resourcemanager.appcontainers.fluent.ContainerAppsRevisionsClient; import com.azure.resourcemanager.appcontainers.fluent.ContainerAppsSourceControlsClient; +import com.azure.resourcemanager.appcontainers.fluent.DaprComponentResiliencyPoliciesClient; import com.azure.resourcemanager.appcontainers.fluent.DaprComponentsClient; +import com.azure.resourcemanager.appcontainers.fluent.DaprSubscriptionsClient; +import com.azure.resourcemanager.appcontainers.fluent.DotNetComponentsClient; +import com.azure.resourcemanager.appcontainers.fluent.JavaComponentsClient; import com.azure.resourcemanager.appcontainers.fluent.JobsClient; import com.azure.resourcemanager.appcontainers.fluent.JobsExecutionsClient; import com.azure.resourcemanager.appcontainers.fluent.ManagedCertificatesClient; import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentDiagnosticsClient; +import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentUsagesClient; import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentsClient; import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentsDiagnosticsClient; import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentsStoragesClient; import com.azure.resourcemanager.appcontainers.fluent.NamespacesClient; import com.azure.resourcemanager.appcontainers.fluent.OperationsClient; import com.azure.resourcemanager.appcontainers.fluent.ResourceProvidersClient; +import com.azure.resourcemanager.appcontainers.fluent.UsagesClient; import java.io.IOException; import java.lang.reflect.Type; import java.nio.ByteBuffer; @@ -56,395 +67,611 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the ContainerAppsApiClientImpl type. */ +/** + * Initializes a new instance of the ContainerAppsApiClientImpl type. + */ @ServiceClient(builder = ContainerAppsApiClientBuilder.class) public final class ContainerAppsApiClientImpl implements ContainerAppsApiClient { - /** The ID of the target subscription. */ + /** + * The ID of the target subscription. The value must be an UUID. + */ private final String subscriptionId; /** - * Gets The ID of the target subscription. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The ContainerAppsAuthConfigsClient object to access its operations. */ + /** + * The AppResilienciesClient object to access its operations. + */ + private final AppResilienciesClient appResiliencies; + + /** + * Gets the AppResilienciesClient object to access its operations. + * + * @return the AppResilienciesClient object. + */ + public AppResilienciesClient getAppResiliencies() { + return this.appResiliencies; + } + + /** + * The ContainerAppsAuthConfigsClient object to access its operations. + */ private final ContainerAppsAuthConfigsClient containerAppsAuthConfigs; /** * Gets the ContainerAppsAuthConfigsClient object to access its operations. - * + * * @return the ContainerAppsAuthConfigsClient object. */ public ContainerAppsAuthConfigsClient getContainerAppsAuthConfigs() { return this.containerAppsAuthConfigs; } - /** The AvailableWorkloadProfilesClient object to access its operations. */ + /** + * The AvailableWorkloadProfilesClient object to access its operations. + */ private final AvailableWorkloadProfilesClient availableWorkloadProfiles; /** * Gets the AvailableWorkloadProfilesClient object to access its operations. - * + * * @return the AvailableWorkloadProfilesClient object. */ public AvailableWorkloadProfilesClient getAvailableWorkloadProfiles() { return this.availableWorkloadProfiles; } - /** The BillingMetersClient object to access its operations. */ + /** + * The BillingMetersClient object to access its operations. + */ private final BillingMetersClient billingMeters; /** * Gets the BillingMetersClient object to access its operations. - * + * * @return the BillingMetersClient object. */ public BillingMetersClient getBillingMeters() { return this.billingMeters; } - /** The ConnectedEnvironmentsClient object to access its operations. */ + /** + * The BuildersClient object to access its operations. + */ + private final BuildersClient builders; + + /** + * Gets the BuildersClient object to access its operations. + * + * @return the BuildersClient object. + */ + public BuildersClient getBuilders() { + return this.builders; + } + + /** + * The BuildsByBuilderResourcesClient object to access its operations. + */ + private final BuildsByBuilderResourcesClient buildsByBuilderResources; + + /** + * Gets the BuildsByBuilderResourcesClient object to access its operations. + * + * @return the BuildsByBuilderResourcesClient object. + */ + public BuildsByBuilderResourcesClient getBuildsByBuilderResources() { + return this.buildsByBuilderResources; + } + + /** + * The BuildsClient object to access its operations. + */ + private final BuildsClient builds; + + /** + * Gets the BuildsClient object to access its operations. + * + * @return the BuildsClient object. + */ + public BuildsClient getBuilds() { + return this.builds; + } + + /** + * The BuildAuthTokensClient object to access its operations. + */ + private final BuildAuthTokensClient buildAuthTokens; + + /** + * Gets the BuildAuthTokensClient object to access its operations. + * + * @return the BuildAuthTokensClient object. + */ + public BuildAuthTokensClient getBuildAuthTokens() { + return this.buildAuthTokens; + } + + /** + * The ConnectedEnvironmentsClient object to access its operations. + */ private final ConnectedEnvironmentsClient connectedEnvironments; /** * Gets the ConnectedEnvironmentsClient object to access its operations. - * + * * @return the ConnectedEnvironmentsClient object. */ public ConnectedEnvironmentsClient getConnectedEnvironments() { return this.connectedEnvironments; } - /** The ConnectedEnvironmentsCertificatesClient object to access its operations. */ + /** + * The ConnectedEnvironmentsCertificatesClient object to access its operations. + */ private final ConnectedEnvironmentsCertificatesClient connectedEnvironmentsCertificates; /** * Gets the ConnectedEnvironmentsCertificatesClient object to access its operations. - * + * * @return the ConnectedEnvironmentsCertificatesClient object. */ public ConnectedEnvironmentsCertificatesClient getConnectedEnvironmentsCertificates() { return this.connectedEnvironmentsCertificates; } - /** The ConnectedEnvironmentsDaprComponentsClient object to access its operations. */ + /** + * The ConnectedEnvironmentsDaprComponentsClient object to access its operations. + */ private final ConnectedEnvironmentsDaprComponentsClient connectedEnvironmentsDaprComponents; /** * Gets the ConnectedEnvironmentsDaprComponentsClient object to access its operations. - * + * * @return the ConnectedEnvironmentsDaprComponentsClient object. */ public ConnectedEnvironmentsDaprComponentsClient getConnectedEnvironmentsDaprComponents() { return this.connectedEnvironmentsDaprComponents; } - /** The ConnectedEnvironmentsStoragesClient object to access its operations. */ + /** + * The ConnectedEnvironmentsStoragesClient object to access its operations. + */ private final ConnectedEnvironmentsStoragesClient connectedEnvironmentsStorages; /** * Gets the ConnectedEnvironmentsStoragesClient object to access its operations. - * + * * @return the ConnectedEnvironmentsStoragesClient object. */ public ConnectedEnvironmentsStoragesClient getConnectedEnvironmentsStorages() { return this.connectedEnvironmentsStorages; } - /** The ContainerAppsClient object to access its operations. */ + /** + * The ContainerAppsClient object to access its operations. + */ private final ContainerAppsClient containerApps; /** * Gets the ContainerAppsClient object to access its operations. - * + * * @return the ContainerAppsClient object. */ public ContainerAppsClient getContainerApps() { return this.containerApps; } - /** The ContainerAppsRevisionsClient object to access its operations. */ + /** + * The ContainerAppsRevisionsClient object to access its operations. + */ private final ContainerAppsRevisionsClient containerAppsRevisions; /** * Gets the ContainerAppsRevisionsClient object to access its operations. - * + * * @return the ContainerAppsRevisionsClient object. */ public ContainerAppsRevisionsClient getContainerAppsRevisions() { return this.containerAppsRevisions; } - /** The ContainerAppsRevisionReplicasClient object to access its operations. */ + /** + * The ContainerAppsRevisionReplicasClient object to access its operations. + */ private final ContainerAppsRevisionReplicasClient containerAppsRevisionReplicas; /** * Gets the ContainerAppsRevisionReplicasClient object to access its operations. - * + * * @return the ContainerAppsRevisionReplicasClient object. */ public ContainerAppsRevisionReplicasClient getContainerAppsRevisionReplicas() { return this.containerAppsRevisionReplicas; } - /** The ContainerAppsDiagnosticsClient object to access its operations. */ + /** + * The ContainerAppsDiagnosticsClient object to access its operations. + */ private final ContainerAppsDiagnosticsClient containerAppsDiagnostics; /** * Gets the ContainerAppsDiagnosticsClient object to access its operations. - * + * * @return the ContainerAppsDiagnosticsClient object. */ public ContainerAppsDiagnosticsClient getContainerAppsDiagnostics() { return this.containerAppsDiagnostics; } - /** The ManagedEnvironmentDiagnosticsClient object to access its operations. */ + /** + * The ManagedEnvironmentDiagnosticsClient object to access its operations. + */ private final ManagedEnvironmentDiagnosticsClient managedEnvironmentDiagnostics; /** * Gets the ManagedEnvironmentDiagnosticsClient object to access its operations. - * + * * @return the ManagedEnvironmentDiagnosticsClient object. */ public ManagedEnvironmentDiagnosticsClient getManagedEnvironmentDiagnostics() { return this.managedEnvironmentDiagnostics; } - /** The ManagedEnvironmentsDiagnosticsClient object to access its operations. */ + /** + * The ManagedEnvironmentsDiagnosticsClient object to access its operations. + */ private final ManagedEnvironmentsDiagnosticsClient managedEnvironmentsDiagnostics; /** * Gets the ManagedEnvironmentsDiagnosticsClient object to access its operations. - * + * * @return the ManagedEnvironmentsDiagnosticsClient object. */ public ManagedEnvironmentsDiagnosticsClient getManagedEnvironmentsDiagnostics() { return this.managedEnvironmentsDiagnostics; } - /** The OperationsClient object to access its operations. */ - private final OperationsClient operations; - /** - * Gets the OperationsClient object to access its operations. - * - * @return the OperationsClient object. + * The JobsClient object to access its operations. */ - public OperationsClient getOperations() { - return this.operations; - } - - /** The JobsClient object to access its operations. */ private final JobsClient jobs; /** * Gets the JobsClient object to access its operations. - * + * * @return the JobsClient object. */ public JobsClient getJobs() { return this.jobs; } - /** The JobsExecutionsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ + private final OperationsClient operations; + + /** + * Gets the OperationsClient object to access its operations. + * + * @return the OperationsClient object. + */ + public OperationsClient getOperations() { + return this.operations; + } + + /** + * The JobsExecutionsClient object to access its operations. + */ private final JobsExecutionsClient jobsExecutions; /** * Gets the JobsExecutionsClient object to access its operations. - * + * * @return the JobsExecutionsClient object. */ public JobsExecutionsClient getJobsExecutions() { return this.jobsExecutions; } - /** The ResourceProvidersClient object to access its operations. */ + /** + * The ResourceProvidersClient object to access its operations. + */ private final ResourceProvidersClient resourceProviders; /** * Gets the ResourceProvidersClient object to access its operations. - * + * * @return the ResourceProvidersClient object. */ public ResourceProvidersClient getResourceProviders() { return this.resourceProviders; } - /** The ManagedEnvironmentsClient object to access its operations. */ + /** + * The ManagedEnvironmentsClient object to access its operations. + */ private final ManagedEnvironmentsClient managedEnvironments; /** * Gets the ManagedEnvironmentsClient object to access its operations. - * + * * @return the ManagedEnvironmentsClient object. */ public ManagedEnvironmentsClient getManagedEnvironments() { return this.managedEnvironments; } - /** The CertificatesClient object to access its operations. */ + /** + * The CertificatesClient object to access its operations. + */ private final CertificatesClient certificates; /** * Gets the CertificatesClient object to access its operations. - * + * * @return the CertificatesClient object. */ public CertificatesClient getCertificates() { return this.certificates; } - /** The ManagedCertificatesClient object to access its operations. */ + /** + * The ManagedCertificatesClient object to access its operations. + */ private final ManagedCertificatesClient managedCertificates; /** * Gets the ManagedCertificatesClient object to access its operations. - * + * * @return the ManagedCertificatesClient object. */ public ManagedCertificatesClient getManagedCertificates() { return this.managedCertificates; } - /** The NamespacesClient object to access its operations. */ + /** + * The NamespacesClient object to access its operations. + */ private final NamespacesClient namespaces; /** * Gets the NamespacesClient object to access its operations. - * + * * @return the NamespacesClient object. */ public NamespacesClient getNamespaces() { return this.namespaces; } - /** The DaprComponentsClient object to access its operations. */ + /** + * The DaprComponentResiliencyPoliciesClient object to access its operations. + */ + private final DaprComponentResiliencyPoliciesClient daprComponentResiliencyPolicies; + + /** + * Gets the DaprComponentResiliencyPoliciesClient object to access its operations. + * + * @return the DaprComponentResiliencyPoliciesClient object. + */ + public DaprComponentResiliencyPoliciesClient getDaprComponentResiliencyPolicies() { + return this.daprComponentResiliencyPolicies; + } + + /** + * The DaprComponentsClient object to access its operations. + */ private final DaprComponentsClient daprComponents; /** * Gets the DaprComponentsClient object to access its operations. - * + * * @return the DaprComponentsClient object. */ public DaprComponentsClient getDaprComponents() { return this.daprComponents; } - /** The ManagedEnvironmentsStoragesClient object to access its operations. */ + /** + * The DaprSubscriptionsClient object to access its operations. + */ + private final DaprSubscriptionsClient daprSubscriptions; + + /** + * Gets the DaprSubscriptionsClient object to access its operations. + * + * @return the DaprSubscriptionsClient object. + */ + public DaprSubscriptionsClient getDaprSubscriptions() { + return this.daprSubscriptions; + } + + /** + * The ManagedEnvironmentsStoragesClient object to access its operations. + */ private final ManagedEnvironmentsStoragesClient managedEnvironmentsStorages; /** * Gets the ManagedEnvironmentsStoragesClient object to access its operations. - * + * * @return the ManagedEnvironmentsStoragesClient object. */ public ManagedEnvironmentsStoragesClient getManagedEnvironmentsStorages() { return this.managedEnvironmentsStorages; } - /** The ContainerAppsSourceControlsClient object to access its operations. */ + /** + * The ContainerAppsSourceControlsClient object to access its operations. + */ private final ContainerAppsSourceControlsClient containerAppsSourceControls; /** * Gets the ContainerAppsSourceControlsClient object to access its operations. - * + * * @return the ContainerAppsSourceControlsClient object. */ public ContainerAppsSourceControlsClient getContainerAppsSourceControls() { return this.containerAppsSourceControls; } + /** + * The UsagesClient object to access its operations. + */ + private final UsagesClient usages; + + /** + * Gets the UsagesClient object to access its operations. + * + * @return the UsagesClient object. + */ + public UsagesClient getUsages() { + return this.usages; + } + + /** + * The ManagedEnvironmentUsagesClient object to access its operations. + */ + private final ManagedEnvironmentUsagesClient managedEnvironmentUsages; + + /** + * Gets the ManagedEnvironmentUsagesClient object to access its operations. + * + * @return the ManagedEnvironmentUsagesClient object. + */ + public ManagedEnvironmentUsagesClient getManagedEnvironmentUsages() { + return this.managedEnvironmentUsages; + } + + /** + * The JavaComponentsClient object to access its operations. + */ + private final JavaComponentsClient javaComponents; + + /** + * Gets the JavaComponentsClient object to access its operations. + * + * @return the JavaComponentsClient object. + */ + public JavaComponentsClient getJavaComponents() { + return this.javaComponents; + } + + /** + * The DotNetComponentsClient object to access its operations. + */ + private final DotNetComponentsClient dotNetComponents; + + /** + * Gets the DotNetComponentsClient object to access its operations. + * + * @return the DotNetComponentsClient object. + */ + public DotNetComponentsClient getDotNetComponents() { + return this.dotNetComponents; + } + /** * Initializes an instance of ContainerAppsApiClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. * @param environment The Azure environment. - * @param subscriptionId The ID of the target subscription. + * @param subscriptionId The ID of the target subscription. The value must be an UUID. * @param endpoint server parameter. */ - ContainerAppsApiClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + ContainerAppsApiClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-05-01"; + this.apiVersion = "2023-11-02-preview"; + this.appResiliencies = new AppResilienciesClientImpl(this); this.containerAppsAuthConfigs = new ContainerAppsAuthConfigsClientImpl(this); this.availableWorkloadProfiles = new AvailableWorkloadProfilesClientImpl(this); this.billingMeters = new BillingMetersClientImpl(this); + this.builders = new BuildersClientImpl(this); + this.buildsByBuilderResources = new BuildsByBuilderResourcesClientImpl(this); + this.builds = new BuildsClientImpl(this); + this.buildAuthTokens = new BuildAuthTokensClientImpl(this); this.connectedEnvironments = new ConnectedEnvironmentsClientImpl(this); this.connectedEnvironmentsCertificates = new ConnectedEnvironmentsCertificatesClientImpl(this); this.connectedEnvironmentsDaprComponents = new ConnectedEnvironmentsDaprComponentsClientImpl(this); @@ -455,22 +682,28 @@ public ContainerAppsSourceControlsClient getContainerAppsSourceControls() { this.containerAppsDiagnostics = new ContainerAppsDiagnosticsClientImpl(this); this.managedEnvironmentDiagnostics = new ManagedEnvironmentDiagnosticsClientImpl(this); this.managedEnvironmentsDiagnostics = new ManagedEnvironmentsDiagnosticsClientImpl(this); - this.operations = new OperationsClientImpl(this); this.jobs = new JobsClientImpl(this); + this.operations = new OperationsClientImpl(this); this.jobsExecutions = new JobsExecutionsClientImpl(this); this.resourceProviders = new ResourceProvidersClientImpl(this); this.managedEnvironments = new ManagedEnvironmentsClientImpl(this); this.certificates = new CertificatesClientImpl(this); this.managedCertificates = new ManagedCertificatesClientImpl(this); this.namespaces = new NamespacesClientImpl(this); + this.daprComponentResiliencyPolicies = new DaprComponentResiliencyPoliciesClientImpl(this); this.daprComponents = new DaprComponentsClientImpl(this); + this.daprSubscriptions = new DaprSubscriptionsClientImpl(this); this.managedEnvironmentsStorages = new ManagedEnvironmentsStoragesClientImpl(this); this.containerAppsSourceControls = new ContainerAppsSourceControlsClientImpl(this); + this.usages = new UsagesClientImpl(this); + this.managedEnvironmentUsages = new ManagedEnvironmentUsagesClientImpl(this); + this.javaComponents = new JavaComponentsClientImpl(this); + this.dotNetComponents = new DotNetComponentsClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -479,7 +712,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -489,7 +722,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -499,26 +732,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -531,19 +753,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsClientImpl.java index e52c6e91fd925..83d04f142fbfc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsClientImpl.java @@ -33,23 +33,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ContainerAppsAuthConfigsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsAuthConfigsClient. + */ public final class ContainerAppsAuthConfigsClientImpl implements ContainerAppsAuthConfigsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ContainerAppsAuthConfigsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ContainerAppsAuthConfigsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ContainerAppsAuthConfigsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create(ContainerAppsAuthConfigsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ContainerAppsAuthConfigsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,102 +65,78 @@ public final class ContainerAppsAuthConfigsClientImpl implements ContainerAppsAu @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ContainerAppsAuthConfigsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listByContainerApp( - @HostParam("$host") String endpoint, + Mono> listByContainerApp(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs/{authConfigName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs/{authConfigName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("authConfigName") String authConfigName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("authConfigName") String authConfigName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs/{authConfigName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs/{authConfigName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("authConfigName") String authConfigName, + @PathParam("containerAppName") String containerAppName, @PathParam("authConfigName") String authConfigName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") AuthConfigInner authConfigEnvelope, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") AuthConfigInner authConfigEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs/{authConfigName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/authConfigs/{authConfigName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("authConfigName") String authConfigName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("authConfigName") String authConfigName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listByContainerAppNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App AuthConfigs in a given resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByContainerAppSinglePageAsync( - String resourceGroupName, String containerAppName) { + private Mono> listByContainerAppSinglePageAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -168,31 +149,16 @@ private Mono> listByContainerAppSinglePageAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listByContainerApp( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listByContainerApp(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -200,22 +166,18 @@ private Mono> listByContainerAppSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App AuthConfigs in a given resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByContainerAppSinglePageAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> listByContainerAppSinglePageAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -228,28 +190,15 @@ private Mono> listByContainerAppSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByContainerApp( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByContainerApp(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -259,14 +208,13 @@ private Mono> listByContainerAppSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByContainerAppAsync(String resourceGroupName, String containerAppName) { - return new PagedFlux<>( - () -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName), + return new PagedFlux<>(() -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName), nextLink -> listByContainerAppNextSinglePageAsync(nextLink)); } /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -276,16 +224,15 @@ private PagedFlux listByContainerAppAsync(String resourceGroupN * @return the Container App AuthConfigs in a given resource group as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByContainerAppAsync( - String resourceGroupName, String containerAppName, Context context) { - return new PagedFlux<>( - () -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName, context), + private PagedFlux listByContainerAppAsync(String resourceGroupName, String containerAppName, + Context context) { + return new PagedFlux<>(() -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName, context), nextLink -> listByContainerAppNextSinglePageAsync(nextLink, context)); } /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -300,7 +247,7 @@ public PagedIterable listByContainerApp(String resourceGroupNam /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -310,14 +257,14 @@ public PagedIterable listByContainerApp(String resourceGroupNam * @return the Container App AuthConfigs in a given resource group as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByContainerApp( - String resourceGroupName, String containerAppName, Context context) { + public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, + Context context) { return new PagedIterable<>(listByContainerAppAsync(resourceGroupName, containerAppName, context)); } /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -327,19 +274,15 @@ public PagedIterable listByContainerApp( * @return a AuthConfig of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String containerAppName, String authConfigName) { + private Mono> getWithResponseAsync(String resourceGroupName, String containerAppName, + String authConfigName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -354,24 +297,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - authConfigName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, authConfigName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -382,19 +315,15 @@ private Mono> getWithResponseAsync( * @return a AuthConfig of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String containerAppName, String authConfigName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String containerAppName, + String authConfigName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -409,21 +338,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - authConfigName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, authConfigName, this.client.getApiVersion(), accept, context); } /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -440,7 +361,7 @@ private Mono getAsync(String resourceGroupName, String containe /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -451,14 +372,14 @@ private Mono getAsync(String resourceGroupName, String containe * @return a AuthConfig of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context) { + public Response getWithResponse(String resourceGroupName, String containerAppName, + String authConfigName, Context context) { return getWithResponseAsync(resourceGroupName, containerAppName, authConfigName, context).block(); } /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -474,7 +395,7 @@ public AuthConfigInner get(String resourceGroupName, String containerAppName, St /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -483,22 +404,18 @@ public AuthConfigInner get(String resourceGroupName, String containerAppName, St * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String containerAppName, String authConfigName, AuthConfigInner authConfigEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String containerAppName, String authConfigName, AuthConfigInner authConfigEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -519,25 +436,15 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - authConfigName, - this.client.getApiVersion(), - authConfigEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, authConfigName, this.client.getApiVersion(), authConfigEnvelope, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -547,26 +454,18 @@ private Mono> createOrUpdateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String containerAppName, - String authConfigName, - AuthConfigInner authConfigEnvelope, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String containerAppName, String authConfigName, AuthConfigInner authConfigEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -587,22 +486,13 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - authConfigName, - this.client.getApiVersion(), - authConfigEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, authConfigName, this.client.getApiVersion(), authConfigEnvelope, accept, context); } /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -611,18 +501,18 @@ private Mono> createOrUpdateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String containerAppName, String authConfigName, AuthConfigInner authConfigEnvelope) { + private Mono createOrUpdateAsync(String resourceGroupName, String containerAppName, + String authConfigName, AuthConfigInner authConfigEnvelope) { return createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, authConfigName, authConfigEnvelope) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -632,23 +522,18 @@ private Mono createOrUpdateAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature along - * with {@link Response}. + * with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String containerAppName, - String authConfigName, - AuthConfigInner authConfigEnvelope, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, containerAppName, authConfigName, authConfigEnvelope, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, String containerAppName, + String authConfigName, AuthConfigInner authConfigEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, authConfigName, authConfigEnvelope, + context).block(); } /** * Create or update the AuthConfig for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -659,16 +544,15 @@ public Response createOrUpdateWithResponse( * @return configuration settings for the Azure ContainerApp Service Authentication / Authorization feature. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AuthConfigInner createOrUpdate( - String resourceGroupName, String containerAppName, String authConfigName, AuthConfigInner authConfigEnvelope) { - return createOrUpdateWithResponse( - resourceGroupName, containerAppName, authConfigName, authConfigEnvelope, Context.NONE) - .getValue(); + public AuthConfigInner createOrUpdate(String resourceGroupName, String containerAppName, String authConfigName, + AuthConfigInner authConfigEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, containerAppName, authConfigName, authConfigEnvelope, + Context.NONE).getValue(); } /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -678,19 +562,15 @@ public AuthConfigInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String containerAppName, String authConfigName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String containerAppName, + String authConfigName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -705,24 +585,14 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - authConfigName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, authConfigName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -733,19 +603,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String containerAppName, String authConfigName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String containerAppName, + String authConfigName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -760,21 +626,13 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - authConfigName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, authConfigName, this.client.getApiVersion(), accept, context); } /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -791,7 +649,7 @@ private Mono deleteAsync(String resourceGroupName, String containerAppName /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -802,14 +660,14 @@ private Mono deleteAsync(String resourceGroupName, String containerAppName * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String containerAppName, String authConfigName, + Context context) { return deleteWithResponseAsync(resourceGroupName, containerAppName, authConfigName, context).block(); } /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. @@ -824,14 +682,15 @@ public void delete(String resourceGroupName, String containerAppName, String aut /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return authConfig collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return authConfig collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByContainerAppNextSinglePageAsync(String nextLink) { @@ -839,63 +698,45 @@ private Mono> listByContainerAppNextSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByContainerAppNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return authConfig collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return authConfig collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByContainerAppNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByContainerAppNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByContainerAppNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByContainerAppNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsImpl.java index b9c6591e44cab..f3d9f8b6c14ab 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsAuthConfigsImpl.java @@ -21,35 +21,31 @@ public final class ContainerAppsAuthConfigsImpl implements ContainerAppsAuthConf private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ContainerAppsAuthConfigsImpl( - ContainerAppsAuthConfigsClient innerClient, + public ContainerAppsAuthConfigsImpl(ContainerAppsAuthConfigsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName) { - PagedIterable inner = - this.serviceClient().listByContainerApp(resourceGroupName, containerAppName); - return Utils.mapPage(inner, inner1 -> new AuthConfigImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByContainerApp(resourceGroupName, containerAppName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AuthConfigImpl(inner1, this.manager())); } - public PagedIterable listByContainerApp( - String resourceGroupName, String containerAppName, Context context) { - PagedIterable inner = - this.serviceClient().listByContainerApp(resourceGroupName, containerAppName, context); - return Utils.mapPage(inner, inner1 -> new AuthConfigImpl(inner1, this.manager())); + public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, + Context context) { + PagedIterable inner + = this.serviceClient().listByContainerApp(resourceGroupName, containerAppName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AuthConfigImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, containerAppName, authConfigName, context); + public Response getWithResponse(String resourceGroupName, String containerAppName, + String authConfigName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, containerAppName, authConfigName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AuthConfigImpl(inner.getValue(), this.manager())); } else { return null; @@ -65,8 +61,8 @@ public AuthConfig get(String resourceGroupName, String containerAppName, String } } - public Response deleteWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String containerAppName, String authConfigName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, containerAppName, authConfigName, context); } @@ -75,105 +71,77 @@ public void delete(String resourceGroupName, String containerAppName, String aut } public AuthConfig getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String authConfigName = Utils.getValueFromIdByName(id, "authConfigs"); + String authConfigName = ResourceManagerUtils.getValueFromIdByName(id, "authConfigs"); if (authConfigName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); } return this.getWithResponse(resourceGroupName, containerAppName, authConfigName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String authConfigName = Utils.getValueFromIdByName(id, "authConfigs"); + String authConfigName = ResourceManagerUtils.getValueFromIdByName(id, "authConfigs"); if (authConfigName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); } return this.getWithResponse(resourceGroupName, containerAppName, authConfigName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String authConfigName = Utils.getValueFromIdByName(id, "authConfigs"); + String authConfigName = ResourceManagerUtils.getValueFromIdByName(id, "authConfigs"); if (authConfigName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); } this.deleteWithResponse(resourceGroupName, containerAppName, authConfigName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String authConfigName = Utils.getValueFromIdByName(id, "authConfigs"); + String authConfigName = ResourceManagerUtils.getValueFromIdByName(id, "authConfigs"); if (authConfigName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authConfigs'.", id))); } return this.deleteWithResponse(resourceGroupName, containerAppName, authConfigName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsClientImpl.java index 31d17e3abca06..fc84088e1a3bd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsClientImpl.java @@ -44,22 +44,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ContainerAppsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsClient. + */ public final class ContainerAppsClientImpl implements ContainerAppsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ContainerAppsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ContainerAppsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ContainerAppsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(ContainerAppsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ContainerAppsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -70,290 +76,206 @@ public final class ContainerAppsClientImpl implements ContainerAppsClient { @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ContainerAppsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/containerApps") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType( - value = ManagementException.class, - code = {404}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(value = ManagementException.class, code = { 404 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") ContainerAppInner containerAppEnvelope, - @HeaderParam("Accept") String accept, + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") ContainerAppInner containerAppEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> update( - @HostParam("$host") String endpoint, + Mono>> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") ContainerAppInner containerAppEnvelope, - @HeaderParam("Accept") String accept, + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") ContainerAppInner containerAppEnvelope, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/listCustomHostNameAnalysis") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/listCustomHostNameAnalysis") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listCustomHostnameAnalysis( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("customHostname") String customHostname, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("customHostname") String customHostname, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/listSecrets") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/listSecrets") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listSecrets( - @HostParam("$host") String endpoint, + Mono> listSecrets(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("containerAppName") String containerAppName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("containerAppName") String containerAppName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/getAuthtoken") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType( - value = ManagementException.class, - code = {404}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/getAuthtoken") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(value = ManagementException.class, code = { 404 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getAuthToken( - @HostParam("$host") String endpoint, + Mono> getAuthToken(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("containerAppName") String containerAppName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("containerAppName") String containerAppName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/start") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/start") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> start( - @HostParam("$host") String endpoint, + Mono>> start(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("containerAppName") String containerAppName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("containerAppName") String containerAppName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/stop") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/stop") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> stop( - @HostParam("$host") String endpoint, + Mono>> stop(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("containerAppName") String containerAppName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("containerAppName") String containerAppName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the Container Apps in a given subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Container Apps in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Container Apps in a given subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Get the Container Apps in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -362,13 +284,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Get the Container Apps in a given subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription as paginated response with {@link PagedIterable}. @@ -380,7 +302,7 @@ public PagedIterable list() { /** * Get the Container Apps in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -394,27 +316,23 @@ public PagedIterable list(Context context) { /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given resource group along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -422,53 +340,34 @@ private Mono> listByResourceGroupSinglePageAsyn } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given resource group along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -477,27 +376,15 @@ private Mono> listByResourceGroupSinglePageAsyn final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -506,14 +393,13 @@ private Mono> listByResourceGroupSinglePageAsyn */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -523,14 +409,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGro */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -544,7 +429,7 @@ public PagedIterable listByResourceGroup(String resourceGroup /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -559,7 +444,7 @@ public PagedIterable listByResourceGroup(String resourceGroup /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -569,19 +454,15 @@ public PagedIterable listByResourceGroup(String resourceGroup * @return the properties of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String containerAppName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -594,22 +475,14 @@ private Mono> getByResourceGroupWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -620,19 +493,15 @@ private Mono> getByResourceGroupWithResponseAsync( * @return the properties of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -644,20 +513,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), accept, context); } /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -674,7 +536,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -685,14 +547,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName * @return the properties of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String containerAppName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String containerAppName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, containerAppName, context).block(); } /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -708,7 +570,7 @@ public ContainerAppInner getByResourceGroup(String resourceGroupName, String con /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -718,19 +580,15 @@ public ContainerAppInner getByResourceGroup(String resourceGroupName, String con * @return container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -748,24 +606,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - containerAppEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), containerAppEnvelope, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -776,19 +625,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -806,21 +651,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - containerAppEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), containerAppEnvelope, accept, context); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -832,21 +669,15 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ContainerAppInner> beginCreateOrUpdateAsync( String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ContainerAppInner.class, - ContainerAppInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, this.client.getContext()); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -860,17 +691,15 @@ private PollerFlux, ContainerAppInner> beginCreate private PollerFlux, ContainerAppInner> beginCreateOrUpdateAsync( String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ContainerAppInner.class, ContainerAppInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, context); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -880,14 +709,14 @@ private PollerFlux, ContainerAppInner> beginCreate * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginCreateOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { + public SyncPoller, ContainerAppInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope) { return this.beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope).getSyncPoller(); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -898,16 +727,15 @@ public SyncPoller, ContainerAppInner> beginCreateO * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginCreateOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context) + public SyncPoller, ContainerAppInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context) .getSyncPoller(); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -917,16 +745,15 @@ public SyncPoller, ContainerAppInner> beginCreateO * @return container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { - return beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -937,16 +764,15 @@ private Mono createOrUpdateAsync( * @return container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -956,14 +782,14 @@ private Mono createOrUpdateAsync( * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ContainerAppInner createOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { + public ContainerAppInner createOrUpdate(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope) { return createOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope).block(); } /** * Create or update a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties used to create a container app. @@ -974,14 +800,14 @@ public ContainerAppInner createOrUpdate( * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ContainerAppInner createOrUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + public ContainerAppInner createOrUpdate(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope, Context context) { return createOrUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context).block(); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -990,19 +816,15 @@ public ContainerAppInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String containerAppName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1014,23 +836,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1040,19 +853,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String containerAppName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1064,20 +873,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), accept, context); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1088,15 +890,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String containerAppName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, containerAppName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1106,18 +906,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String containerAppName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String containerAppName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, containerAppName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1132,7 +931,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1142,14 +941,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String containerAppName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String containerAppName, + Context context) { return this.beginDeleteAsync(resourceGroupName, containerAppName, context).getSyncPoller(); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1159,14 +958,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String containerAppName) { - return beginDeleteAsync(resourceGroupName, containerAppName) - .last() + return beginDeleteAsync(resourceGroupName, containerAppName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1177,14 +975,13 @@ private Mono deleteAsync(String resourceGroupName, String containerAppName */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String containerAppName, Context context) { - return beginDeleteAsync(resourceGroupName, containerAppName, context) - .last() + return beginDeleteAsync(resourceGroupName, containerAppName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1198,7 +995,7 @@ public void delete(String resourceGroupName, String containerAppName) { /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1213,9 +1010,9 @@ public void delete(String resourceGroupName, String containerAppName, Context co /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1225,19 +1022,15 @@ public void delete(String resourceGroupName, String containerAppName, Context co * @return container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1256,25 +1049,16 @@ private Mono>> updateWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - containerAppEnvelope, - accept, - context)) + context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), containerAppEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1285,19 +1069,15 @@ private Mono>> updateWithResponseAsync( * @return container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1315,23 +1095,15 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - containerAppEnvelope, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), containerAppEnvelope, accept, context); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1341,25 +1113,19 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ContainerAppInner> beginUpdateAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { - Mono>> mono = - updateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ContainerAppInner.class, - ContainerAppInner.class, - this.client.getContext()); + private PollerFlux, ContainerAppInner> beginUpdateAsync(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, this.client.getContext()); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1370,22 +1136,20 @@ private PollerFlux, ContainerAppInner> beginUpdate * @return the {@link PollerFlux} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ContainerAppInner> beginUpdateAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + private PollerFlux, ContainerAppInner> beginUpdateAsync(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ContainerAppInner.class, ContainerAppInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, containerAppName, containerAppEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, context); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1395,16 +1159,16 @@ private PollerFlux, ContainerAppInner> beginUpdate * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { + public SyncPoller, ContainerAppInner> beginUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope) { return this.beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope).getSyncPoller(); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1415,18 +1179,17 @@ public SyncPoller, ContainerAppInner> beginUpdate( * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginUpdate( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { - return this - .beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context) + public SyncPoller, ContainerAppInner> beginUpdate(String resourceGroupName, + String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + return this.beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context) .getSyncPoller(); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1436,18 +1199,17 @@ public SyncPoller, ContainerAppInner> beginUpdate( * @return container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { - return beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope) - .last() + private Mono updateAsync(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope) { + return beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1458,18 +1220,17 @@ private Mono updateAsync( * @return container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { - return beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context) - .last() + private Mono updateAsync(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope, Context context) { + return beginUpdateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1479,16 +1240,16 @@ private Mono updateAsync( * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ContainerAppInner update( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope) { + public ContainerAppInner update(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope) { return updateAsync(resourceGroupName, containerAppName, containerAppEnvelope).block(); } /** * Update properties of a Container App - * - *

Patches a Container App using JSON Merge Patch. - * + * + * Patches a Container App using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param containerAppEnvelope Properties of a Container App that need to be updated. @@ -1499,14 +1260,14 @@ public ContainerAppInner update( * @return container App. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ContainerAppInner update( - String resourceGroupName, String containerAppName, ContainerAppInner containerAppEnvelope, Context context) { + public ContainerAppInner update(String resourceGroupName, String containerAppName, + ContainerAppInner containerAppEnvelope, Context context) { return updateAsync(resourceGroupName, containerAppName, containerAppEnvelope, context).block(); } /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param customHostname Custom hostname. @@ -1519,16 +1280,12 @@ public ContainerAppInner update( private Mono> listCustomHostnameAnalysisWithResponseAsync( String resourceGroupName, String containerAppName, String customHostname) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1540,24 +1297,15 @@ private Mono> listCustomHostnameAnal } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listCustomHostnameAnalysis( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - customHostname, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.listCustomHostnameAnalysis(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, containerAppName, customHostname, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param customHostname Custom hostname. @@ -1571,16 +1319,12 @@ private Mono> listCustomHostnameAnal private Mono> listCustomHostnameAnalysisWithResponseAsync( String resourceGroupName, String containerAppName, String customHostname, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1592,21 +1336,13 @@ private Mono> listCustomHostnameAnal } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listCustomHostnameAnalysis( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - customHostname, - this.client.getApiVersion(), - accept, - context); + return service.listCustomHostnameAnalysis(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, customHostname, this.client.getApiVersion(), accept, context); } /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1615,8 +1351,8 @@ private Mono> listCustomHostnameAnal * @return custom domain analysis on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listCustomHostnameAnalysisAsync( - String resourceGroupName, String containerAppName) { + private Mono listCustomHostnameAnalysisAsync(String resourceGroupName, + String containerAppName) { final String customHostname = null; return listCustomHostnameAnalysisWithResponseAsync(resourceGroupName, containerAppName, customHostname) .flatMap(res -> Mono.justOrEmpty(res.getValue())); @@ -1624,7 +1360,7 @@ private Mono listCustomHostnameAnalysisAsync( /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param customHostname Custom hostname. @@ -1635,15 +1371,15 @@ private Mono listCustomHostnameAnalysisAsync( * @return custom domain analysis along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listCustomHostnameAnalysisWithResponse( - String resourceGroupName, String containerAppName, String customHostname, Context context) { + public Response listCustomHostnameAnalysisWithResponse(String resourceGroupName, + String containerAppName, String customHostname, Context context) { return listCustomHostnameAnalysisWithResponseAsync(resourceGroupName, containerAppName, customHostname, context) .block(); } /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1652,8 +1388,8 @@ public Response listCustomHostnameAnalysisWit * @return custom domain analysis. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CustomHostnameAnalysisResultInner listCustomHostnameAnalysis( - String resourceGroupName, String containerAppName) { + public CustomHostnameAnalysisResultInner listCustomHostnameAnalysis(String resourceGroupName, + String containerAppName) { final String customHostname = null; return listCustomHostnameAnalysisWithResponse(resourceGroupName, containerAppName, customHostname, Context.NONE) .getValue(); @@ -1661,29 +1397,25 @@ public CustomHostnameAnalysisResultInner listCustomHostnameAnalysis( /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String containerAppName) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1695,23 +1427,14 @@ private Mono> listSecretsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context)) + .withContext(context -> service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), containerAppName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1719,22 +1442,18 @@ private Mono> listSecretsWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1746,20 +1465,13 @@ private Mono> listSecretsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context); + return service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), containerAppName, accept, context); } /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1775,7 +1487,7 @@ private Mono listSecretsAsync(String resourceGroupName, /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1785,14 +1497,14 @@ private Mono listSecretsAsync(String resourceGroupName, * @return container App Secrets Collection ARM resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listSecretsWithResponse( - String resourceGroupName, String containerAppName, Context context) { + public Response listSecretsWithResponse(String resourceGroupName, String containerAppName, + Context context) { return listSecretsWithResponseAsync(resourceGroupName, containerAppName, context).block(); } /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1807,7 +1519,7 @@ public SecretsCollectionInner listSecrets(String resourceGroupName, String conta /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1817,19 +1529,15 @@ public SecretsCollectionInner listSecrets(String resourceGroupName, String conta * @return auth token for a container app along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthTokenWithResponseAsync( - String resourceGroupName, String containerAppName) { + private Mono> getAuthTokenWithResponseAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1841,23 +1549,14 @@ private Mono> getAuthTokenWithResponseAsync } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getAuthToken( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context)) + .withContext(context -> service.getAuthToken(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), containerAppName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1868,19 +1567,15 @@ private Mono> getAuthTokenWithResponseAsync * @return auth token for a container app along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthTokenWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> getAuthTokenWithResponseAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1892,20 +1587,13 @@ private Mono> getAuthTokenWithResponseAsync } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getAuthToken( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context); + return service.getAuthToken(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), containerAppName, accept, context); } /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1922,7 +1610,7 @@ private Mono getAuthTokenAsync(String resourceGroupN /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -1933,14 +1621,14 @@ private Mono getAuthTokenAsync(String resourceGroupN * @return auth token for a container app along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getAuthTokenWithResponse( - String resourceGroupName, String containerAppName, Context context) { + public Response getAuthTokenWithResponse(String resourceGroupName, + String containerAppName, Context context) { return getAuthTokenWithResponseAsync(resourceGroupName, containerAppName, context).block(); } /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1956,7 +1644,7 @@ public ContainerAppAuthTokenInner getAuthToken(String resourceGroupName, String /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1967,16 +1655,12 @@ public ContainerAppAuthTokenInner getAuthToken(String resourceGroupName, String @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> startWithResponseAsync(String resourceGroupName, String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1988,23 +1672,14 @@ private Mono>> startWithResponseAsync(String resourceG } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .start( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context)) + .withContext(context -> service.start(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), containerAppName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2014,19 +1689,15 @@ private Mono>> startWithResponseAsync(String resourceG * @return container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> startWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono>> startWithResponseAsync(String resourceGroupName, String containerAppName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2038,20 +1709,13 @@ private Mono>> startWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .start( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context); + return service.start(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), containerAppName, accept, context); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2060,22 +1724,16 @@ private Mono>> startWithResponseAsync( * @return the {@link PollerFlux} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ContainerAppInner> beginStartAsync( - String resourceGroupName, String containerAppName) { + private PollerFlux, ContainerAppInner> beginStartAsync(String resourceGroupName, + String containerAppName) { Mono>> mono = startWithResponseAsync(resourceGroupName, containerAppName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ContainerAppInner.class, - ContainerAppInner.class, - this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, this.client.getContext()); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2085,19 +1743,17 @@ private PollerFlux, ContainerAppInner> beginStartA * @return the {@link PollerFlux} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ContainerAppInner> beginStartAsync( - String resourceGroupName, String containerAppName, Context context) { + private PollerFlux, ContainerAppInner> beginStartAsync(String resourceGroupName, + String containerAppName, Context context) { context = this.client.mergeContext(context); Mono>> mono = startWithResponseAsync(resourceGroupName, containerAppName, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ContainerAppInner.class, ContainerAppInner.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, context); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2106,14 +1762,14 @@ private PollerFlux, ContainerAppInner> beginStartA * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginStart( - String resourceGroupName, String containerAppName) { + public SyncPoller, ContainerAppInner> beginStart(String resourceGroupName, + String containerAppName) { return this.beginStartAsync(resourceGroupName, containerAppName).getSyncPoller(); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2123,14 +1779,14 @@ public SyncPoller, ContainerAppInner> beginStart( * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginStart( - String resourceGroupName, String containerAppName, Context context) { + public SyncPoller, ContainerAppInner> beginStart(String resourceGroupName, + String containerAppName, Context context) { return this.beginStartAsync(resourceGroupName, containerAppName, context).getSyncPoller(); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2140,14 +1796,13 @@ public SyncPoller, ContainerAppInner> beginStart( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono startAsync(String resourceGroupName, String containerAppName) { - return beginStartAsync(resourceGroupName, containerAppName) - .last() + return beginStartAsync(resourceGroupName, containerAppName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2158,14 +1813,13 @@ private Mono startAsync(String resourceGroupName, String cont */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono startAsync(String resourceGroupName, String containerAppName, Context context) { - return beginStartAsync(resourceGroupName, containerAppName, context) - .last() + return beginStartAsync(resourceGroupName, containerAppName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2180,7 +1834,7 @@ public ContainerAppInner start(String resourceGroupName, String containerAppName /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2196,7 +1850,7 @@ public ContainerAppInner start(String resourceGroupName, String containerAppName /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2207,16 +1861,12 @@ public ContainerAppInner start(String resourceGroupName, String containerAppName @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> stopWithResponseAsync(String resourceGroupName, String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2228,23 +1878,14 @@ private Mono>> stopWithResponseAsync(String resourceGr } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .stop( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context)) + .withContext(context -> service.stop(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), containerAppName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2254,19 +1895,15 @@ private Mono>> stopWithResponseAsync(String resourceGr * @return container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono>> stopWithResponseAsync(String resourceGroupName, String containerAppName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2278,20 +1915,13 @@ private Mono>> stopWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .stop( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - containerAppName, - accept, - context); + return service.stop(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), containerAppName, accept, context); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2300,22 +1930,16 @@ private Mono>> stopWithResponseAsync( * @return the {@link PollerFlux} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ContainerAppInner> beginStopAsync( - String resourceGroupName, String containerAppName) { + private PollerFlux, ContainerAppInner> beginStopAsync(String resourceGroupName, + String containerAppName) { Mono>> mono = stopWithResponseAsync(resourceGroupName, containerAppName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ContainerAppInner.class, - ContainerAppInner.class, - this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, this.client.getContext()); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2325,19 +1949,17 @@ private PollerFlux, ContainerAppInner> beginStopAs * @return the {@link PollerFlux} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ContainerAppInner> beginStopAsync( - String resourceGroupName, String containerAppName, Context context) { + private PollerFlux, ContainerAppInner> beginStopAsync(String resourceGroupName, + String containerAppName, Context context) { context = this.client.mergeContext(context); Mono>> mono = stopWithResponseAsync(resourceGroupName, containerAppName, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ContainerAppInner.class, ContainerAppInner.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ContainerAppInner.class, ContainerAppInner.class, context); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2346,14 +1968,14 @@ private PollerFlux, ContainerAppInner> beginStopAs * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginStop( - String resourceGroupName, String containerAppName) { + public SyncPoller, ContainerAppInner> beginStop(String resourceGroupName, + String containerAppName) { return this.beginStopAsync(resourceGroupName, containerAppName).getSyncPoller(); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2363,14 +1985,14 @@ public SyncPoller, ContainerAppInner> beginStop( * @return the {@link SyncPoller} for polling of container App. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ContainerAppInner> beginStop( - String resourceGroupName, String containerAppName, Context context) { + public SyncPoller, ContainerAppInner> beginStop(String resourceGroupName, + String containerAppName, Context context) { return this.beginStopAsync(resourceGroupName, containerAppName, context).getSyncPoller(); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2380,14 +2002,13 @@ public SyncPoller, ContainerAppInner> beginStop( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono stopAsync(String resourceGroupName, String containerAppName) { - return beginStopAsync(resourceGroupName, containerAppName) - .last() + return beginStopAsync(resourceGroupName, containerAppName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2398,14 +2019,13 @@ private Mono stopAsync(String resourceGroupName, String conta */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono stopAsync(String resourceGroupName, String containerAppName, Context context) { - return beginStopAsync(resourceGroupName, containerAppName, context) - .last() + return beginStopAsync(resourceGroupName, containerAppName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2420,7 +2040,7 @@ public ContainerAppInner stop(String resourceGroupName, String containerAppName) /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -2436,14 +2056,15 @@ public ContainerAppInner stop(String resourceGroupName, String containerAppName, /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -2451,76 +2072,59 @@ private Mono> listBySubscriptionNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -2528,63 +2132,45 @@ private Mono> listByResourceGroupNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return container App collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsClientImpl.java index f56a404c5e7fb..e2c922eaca47a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsClientImpl.java @@ -34,23 +34,28 @@ import com.azure.resourcemanager.appcontainers.models.RevisionCollection; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ContainerAppsDiagnosticsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsDiagnosticsClient. + */ public final class ContainerAppsDiagnosticsClientImpl implements ContainerAppsDiagnosticsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ContainerAppsDiagnosticsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ContainerAppsDiagnosticsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ContainerAppsDiagnosticsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create(ContainerAppsDiagnosticsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ContainerAppsDiagnosticsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -61,128 +66,95 @@ public final class ContainerAppsDiagnosticsClientImpl implements ContainerAppsDi @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ContainerAppsDiagnosticsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectors") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectors") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listDetectors( - @HostParam("$host") String endpoint, + Mono> listDetectors(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectors/{detectorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectors/{detectorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getDetector( - @HostParam("$host") String endpoint, + Mono> getDetector(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("detectorName") String detectorName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("detectorName") String detectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectorProperties/revisionsApi/revisions/") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectorProperties/revisionsApi/revisions/") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listRevisions( - @HostParam("$host") String endpoint, + Mono> listRevisions(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectorProperties/revisionsApi/revisions/{revisionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectorProperties/revisionsApi/revisions/{revisionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getRevision( - @HostParam("$host") String endpoint, + Mono> getRevision(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectorProperties/rootApi/") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType( - value = ManagementException.class, - code = {404}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/detectorProperties/rootApi/") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(value = ManagementException.class, code = { 404 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getRoot( - @HostParam("$host") String endpoint, + Mono> getRoot(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listDetectorsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listRevisionsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a given Container App along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDetectorsSinglePageAsync( - String resourceGroupName, String containerAppName) { + private Mono> listDetectorsSinglePageAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -194,32 +166,16 @@ private Mono> listDetectorsSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listDetectors( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listDetectors(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @param context The context to associate with this operation. @@ -227,22 +183,18 @@ private Mono> listDetectorsSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a given Container App along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDetectorsSinglePageAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> listDetectorsSinglePageAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -255,28 +207,15 @@ private Mono> listDetectorsSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listDetectors( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listDetectors(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -286,14 +225,13 @@ private Mono> listDetectorsSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listDetectorsAsync(String resourceGroupName, String containerAppName) { - return new PagedFlux<>( - () -> listDetectorsSinglePageAsync(resourceGroupName, containerAppName), + return new PagedFlux<>(() -> listDetectorsSinglePageAsync(resourceGroupName, containerAppName), nextLink -> listDetectorsNextSinglePageAsync(nextLink)); } /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @param context The context to associate with this operation. @@ -303,16 +241,15 @@ private PagedFlux listDetectorsAsync(String resourceGroupName, * @return the list of diagnostics for a given Container App as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listDetectorsAsync( - String resourceGroupName, String containerAppName, Context context) { - return new PagedFlux<>( - () -> listDetectorsSinglePageAsync(resourceGroupName, containerAppName, context), + private PagedFlux listDetectorsAsync(String resourceGroupName, String containerAppName, + Context context) { + return new PagedFlux<>(() -> listDetectorsSinglePageAsync(resourceGroupName, containerAppName, context), nextLink -> listDetectorsNextSinglePageAsync(nextLink, context)); } /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -327,7 +264,7 @@ public PagedIterable listDetectors(String resourceGroupName, S /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @param context The context to associate with this operation. @@ -337,37 +274,33 @@ public PagedIterable listDetectors(String resourceGroupName, S * @return the list of diagnostics for a given Container App as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listDetectors( - String resourceGroupName, String containerAppName, Context context) { + public PagedIterable listDetectors(String resourceGroupName, String containerAppName, + Context context) { return new PagedIterable<>(listDetectorsAsync(resourceGroupName, containerAppName, context)); } /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a diagnostics result of a Container App along with {@link Response} on successful completion of {@link - * Mono}. + * @return a diagnostics result of a Container App along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getDetectorWithResponseAsync( - String resourceGroupName, String containerAppName, String detectorName) { + private Mono> getDetectorWithResponseAsync(String resourceGroupName, + String containerAppName, String detectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -382,24 +315,14 @@ private Mono> getDetectorWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getDetector( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - detectorName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getDetector(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, detectorName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. @@ -407,23 +330,19 @@ private Mono> getDetectorWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a diagnostics result of a Container App along with {@link Response} on successful completion of {@link - * Mono}. + * @return a diagnostics result of a Container App along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getDetectorWithResponseAsync( - String resourceGroupName, String containerAppName, String detectorName, Context context) { + private Mono> getDetectorWithResponseAsync(String resourceGroupName, + String containerAppName, String detectorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -438,21 +357,13 @@ private Mono> getDetectorWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getDetector( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - detectorName, - this.client.getApiVersion(), - accept, - context); + return service.getDetector(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, detectorName, this.client.getApiVersion(), accept, context); } /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. @@ -462,15 +373,15 @@ private Mono> getDetectorWithResponseAsync( * @return a diagnostics result of a Container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getDetectorAsync( - String resourceGroupName, String containerAppName, String detectorName) { + private Mono getDetectorAsync(String resourceGroupName, String containerAppName, + String detectorName) { return getDetectorWithResponseAsync(resourceGroupName, containerAppName, detectorName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. @@ -481,14 +392,14 @@ private Mono getDetectorAsync( * @return a diagnostics result of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getDetectorWithResponse( - String resourceGroupName, String containerAppName, String detectorName, Context context) { + public Response getDetectorWithResponse(String resourceGroupName, String containerAppName, + String detectorName, Context context) { return getDetectorWithResponseAsync(resourceGroupName, containerAppName, detectorName, context).block(); } /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. @@ -504,7 +415,7 @@ public DiagnosticsInner getDetector(String resourceGroupName, String containerAp /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -512,22 +423,18 @@ public DiagnosticsInner getDetector(String resourceGroupName, String containerAp * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listRevisionsSinglePageAsync( - String resourceGroupName, String containerAppName, String filter) { + private Mono> listRevisionsSinglePageAsync(String resourceGroupName, + String containerAppName, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -539,33 +446,16 @@ private Mono> listRevisionsSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listRevisions( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listRevisions(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -574,22 +464,18 @@ private Mono> listRevisionsSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listRevisionsSinglePageAsync( - String resourceGroupName, String containerAppName, String filter, Context context) { + private Mono> listRevisionsSinglePageAsync(String resourceGroupName, + String containerAppName, String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -602,29 +488,15 @@ private Mono> listRevisionsSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listRevisions( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listRevisions(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -634,16 +506,15 @@ private Mono> listRevisionsSinglePageAsync( * @return the Revisions for a given Container App as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listRevisionsAsync( - String resourceGroupName, String containerAppName, String filter) { - return new PagedFlux<>( - () -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), + private PagedFlux listRevisionsAsync(String resourceGroupName, String containerAppName, + String filter) { + return new PagedFlux<>(() -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), nextLink -> listRevisionsNextSinglePageAsync(nextLink)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -654,14 +525,13 @@ private PagedFlux listRevisionsAsync( @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listRevisionsAsync(String resourceGroupName, String containerAppName) { final String filter = null; - return new PagedFlux<>( - () -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), + return new PagedFlux<>(() -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), nextLink -> listRevisionsNextSinglePageAsync(nextLink)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -672,16 +542,15 @@ private PagedFlux listRevisionsAsync(String resourceGroupName, St * @return the Revisions for a given Container App as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listRevisionsAsync( - String resourceGroupName, String containerAppName, String filter, Context context) { - return new PagedFlux<>( - () -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter, context), + private PagedFlux listRevisionsAsync(String resourceGroupName, String containerAppName, + String filter, Context context) { + return new PagedFlux<>(() -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter, context), nextLink -> listRevisionsNextSinglePageAsync(nextLink, context)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -697,7 +566,7 @@ public PagedIterable listRevisions(String resourceGroupName, Stri /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -708,14 +577,14 @@ public PagedIterable listRevisions(String resourceGroupName, Stri * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context) { + public PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context) { return new PagedIterable<>(listRevisionsAsync(resourceGroupName, containerAppName, filter, context)); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -725,19 +594,15 @@ public PagedIterable listRevisions( * @return a revision of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono> getRevisionWithResponseAsync(String resourceGroupName, + String containerAppName, String revisionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -752,24 +617,14 @@ private Mono> getRevisionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -780,19 +635,15 @@ private Mono> getRevisionWithResponseAsync( * @return a revision of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + private Mono> getRevisionWithResponseAsync(String resourceGroupName, + String containerAppName, String revisionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -807,21 +658,13 @@ private Mono> getRevisionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context); + return service.getRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, this.client.getApiVersion(), accept, context); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -831,15 +674,15 @@ private Mono> getRevisionWithResponseAsync( * @return a revision of a Container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getRevisionAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono getRevisionAsync(String resourceGroupName, String containerAppName, + String revisionName) { return getRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -850,14 +693,14 @@ private Mono getRevisionAsync( * @return a revision of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + public Response getRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { return getRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName, context).block(); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -873,7 +716,7 @@ public RevisionInner getRevision(String resourceGroupName, String containerAppNa /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -883,19 +726,15 @@ public RevisionInner getRevision(String resourceGroupName, String containerAppNa * @return the properties of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRootWithResponseAsync( - String resourceGroupName, String containerAppName) { + private Mono> getRootWithResponseAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -907,23 +746,14 @@ private Mono> getRootWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getRoot( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getRoot(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -934,19 +764,15 @@ private Mono> getRootWithResponseAsync( * @return the properties of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRootWithResponseAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> getRootWithResponseAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -958,20 +784,13 @@ private Mono> getRootWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getRoot( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context); + return service.getRoot(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), accept, context); } /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -988,7 +807,7 @@ private Mono getRootAsync(String resourceGroupName, String co /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -999,14 +818,14 @@ private Mono getRootAsync(String resourceGroupName, String co * @return the properties of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getRootWithResponse( - String resourceGroupName, String containerAppName, Context context) { + public Response getRootWithResponse(String resourceGroupName, String containerAppName, + Context context) { return getRootWithResponseAsync(resourceGroupName, containerAppName, context).block(); } /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1022,14 +841,15 @@ public ContainerAppInner getRoot(String resourceGroupName, String containerAppNa /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return diagnostics data collection for a resource along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDetectorsNextSinglePageAsync(String nextLink) { @@ -1037,37 +857,29 @@ private Mono> listDetectorsNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listDetectorsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return diagnostics data collection for a resource along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listDetectorsNextSinglePageAsync(String nextLink, Context context) { @@ -1075,36 +887,27 @@ private Mono> listDetectorsNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listDetectorsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listDetectorsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revisions collection ARM resource along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listRevisionsNextSinglePageAsync(String nextLink) { @@ -1112,37 +915,29 @@ private Mono> listRevisionsNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listRevisionsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revisions collection ARM resource along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listRevisionsNextSinglePageAsync(String nextLink, Context context) { @@ -1150,23 +945,13 @@ private Mono> listRevisionsNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listRevisionsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listRevisionsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsImpl.java index e640a600068b6..32d1d19b0fe1e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsDiagnosticsImpl.java @@ -25,8 +25,7 @@ public final class ContainerAppsDiagnosticsImpl implements ContainerAppsDiagnost private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ContainerAppsDiagnosticsImpl( - ContainerAppsDiagnosticsClient innerClient, + public ContainerAppsDiagnosticsImpl(ContainerAppsDiagnosticsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -34,25 +33,22 @@ public ContainerAppsDiagnosticsImpl( public PagedIterable listDetectors(String resourceGroupName, String containerAppName) { PagedIterable inner = this.serviceClient().listDetectors(resourceGroupName, containerAppName); - return Utils.mapPage(inner, inner1 -> new DiagnosticsImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DiagnosticsImpl(inner1, this.manager())); } - public PagedIterable listDetectors( - String resourceGroupName, String containerAppName, Context context) { - PagedIterable inner = - this.serviceClient().listDetectors(resourceGroupName, containerAppName, context); - return Utils.mapPage(inner, inner1 -> new DiagnosticsImpl(inner1, this.manager())); + public PagedIterable listDetectors(String resourceGroupName, String containerAppName, + Context context) { + PagedIterable inner + = this.serviceClient().listDetectors(resourceGroupName, containerAppName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DiagnosticsImpl(inner1, this.manager())); } - public Response getDetectorWithResponse( - String resourceGroupName, String containerAppName, String detectorName, Context context) { - Response inner = - this.serviceClient().getDetectorWithResponse(resourceGroupName, containerAppName, detectorName, context); + public Response getDetectorWithResponse(String resourceGroupName, String containerAppName, + String detectorName, Context context) { + Response inner + = this.serviceClient().getDetectorWithResponse(resourceGroupName, containerAppName, detectorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DiagnosticsImpl(inner.getValue(), this.manager())); } else { return null; @@ -70,25 +66,22 @@ public Diagnostics getDetector(String resourceGroupName, String containerAppName public PagedIterable listRevisions(String resourceGroupName, String containerAppName) { PagedIterable inner = this.serviceClient().listRevisions(resourceGroupName, containerAppName); - return Utils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); } - public PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context) { - PagedIterable inner = - this.serviceClient().listRevisions(resourceGroupName, containerAppName, filter, context); - return Utils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); + public PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context) { + PagedIterable inner + = this.serviceClient().listRevisions(resourceGroupName, containerAppName, filter, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); } - public Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { - Response inner = - this.serviceClient().getRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); + public Response getRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { + Response inner + = this.serviceClient().getRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RevisionImpl(inner.getValue(), this.manager())); } else { return null; @@ -104,15 +97,12 @@ public Revision getRevision(String resourceGroupName, String containerAppName, S } } - public Response getRootWithResponse( - String resourceGroupName, String containerAppName, Context context) { - Response inner = - this.serviceClient().getRootWithResponse(resourceGroupName, containerAppName, context); + public Response getRootWithResponse(String resourceGroupName, String containerAppName, + Context context) { + Response inner + = this.serviceClient().getRootWithResponse(resourceGroupName, containerAppName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ContainerAppImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsImpl.java index 19f3b1c802036..ca52900339afa 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsImpl.java @@ -27,8 +27,7 @@ public final class ContainerAppsImpl implements ContainerApps { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ContainerAppsImpl( - ContainerAppsClient innerClient, + public ContainerAppsImpl(ContainerAppsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -36,33 +35,30 @@ public ContainerAppsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ContainerAppImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String containerAppName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, containerAppName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String containerAppName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, containerAppName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ContainerAppImpl(inner.getValue(), this.manager())); } else { return null; @@ -86,17 +82,12 @@ public void delete(String resourceGroupName, String containerAppName, Context co this.serviceClient().delete(resourceGroupName, containerAppName, context); } - public Response listCustomHostnameAnalysisWithResponse( - String resourceGroupName, String containerAppName, String customHostname, Context context) { - Response inner = - this - .serviceClient() - .listCustomHostnameAnalysisWithResponse(resourceGroupName, containerAppName, customHostname, context); + public Response listCustomHostnameAnalysisWithResponse(String resourceGroupName, + String containerAppName, String customHostname, Context context) { + Response inner = this.serviceClient() + .listCustomHostnameAnalysisWithResponse(resourceGroupName, containerAppName, customHostname, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CustomHostnameAnalysisResultImpl(inner.getValue(), this.manager())); } else { return null; @@ -104,8 +95,8 @@ public Response listCustomHostnameAnalysisWithResp } public CustomHostnameAnalysisResult listCustomHostnameAnalysis(String resourceGroupName, String containerAppName) { - CustomHostnameAnalysisResultInner inner = - this.serviceClient().listCustomHostnameAnalysis(resourceGroupName, containerAppName); + CustomHostnameAnalysisResultInner inner + = this.serviceClient().listCustomHostnameAnalysis(resourceGroupName, containerAppName); if (inner != null) { return new CustomHostnameAnalysisResultImpl(inner, this.manager()); } else { @@ -113,15 +104,12 @@ public CustomHostnameAnalysisResult listCustomHostnameAnalysis(String resourceGr } } - public Response listSecretsWithResponse( - String resourceGroupName, String containerAppName, Context context) { - Response inner = - this.serviceClient().listSecretsWithResponse(resourceGroupName, containerAppName, context); + public Response listSecretsWithResponse(String resourceGroupName, String containerAppName, + Context context) { + Response inner + = this.serviceClient().listSecretsWithResponse(resourceGroupName, containerAppName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecretsCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -137,15 +125,12 @@ public SecretsCollection listSecrets(String resourceGroupName, String containerA } } - public Response getAuthTokenWithResponse( - String resourceGroupName, String containerAppName, Context context) { - Response inner = - this.serviceClient().getAuthTokenWithResponse(resourceGroupName, containerAppName, context); + public Response getAuthTokenWithResponse(String resourceGroupName, String containerAppName, + Context context) { + Response inner + = this.serviceClient().getAuthTokenWithResponse(resourceGroupName, containerAppName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ContainerAppAuthTokenImpl(inner.getValue(), this.manager())); } else { return null; @@ -198,77 +183,57 @@ public ContainerApp stop(String resourceGroupName, String containerAppName, Cont } public ContainerApp getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, containerAppName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, containerAppName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } this.delete(resourceGroupName, containerAppName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } this.delete(resourceGroupName, containerAppName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasClientImpl.java index 021d8be1c3e4d..1e8b67b9cbfa4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasClientImpl.java @@ -26,71 +26,63 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ContainerAppsRevisionReplicasClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsRevisionReplicasClient. + */ public final class ContainerAppsRevisionReplicasClientImpl implements ContainerAppsRevisionReplicasClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ContainerAppsRevisionReplicasService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ContainerAppsRevisionReplicasClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ContainerAppsRevisionReplicasClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ContainerAppsRevisionReplicasService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ContainerAppsRevisionReplicasService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for ContainerAppsApiClientContainerAppsRevisionReplicas to be used by the - * proxy service to perform REST calls. + * The interface defining all the services for ContainerAppsApiClientContainerAppsRevisionReplicas to be used by + * the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ContainerAppsRevisionReplicasService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/replicas/{replicaName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/replicas/{replicaName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getReplica( - @HostParam("$host") String endpoint, + Mono> getReplica(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @PathParam("replicaName") String replicaName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @PathParam("replicaName") String replicaName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/replicas") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/replicas") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listReplicas( - @HostParam("$host") String endpoint, + Mono> listReplicas(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -98,23 +90,19 @@ Mono> listReplicas( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a replica for a Container App Revision along with {@link Response} on successful completion of {@link - * Mono}. + * @return a replica for a Container App Revision along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getReplicaWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, String replicaName) { + private Mono> getReplicaWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName, String replicaName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -132,25 +120,15 @@ private Mono> getReplicaWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getReplica( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - replicaName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getReplica(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, replicaName, this.client.getApiVersion(), accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -159,23 +137,19 @@ private Mono> getReplicaWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a replica for a Container App Revision along with {@link Response} on successful completion of {@link - * Mono}. + * @return a replica for a Container App Revision along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getReplicaWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, String replicaName, Context context) { + private Mono> getReplicaWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName, String replicaName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -193,22 +167,13 @@ private Mono> getReplicaWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getReplica( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - replicaName, - this.client.getApiVersion(), - accept, - context); + return service.getReplica(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, replicaName, this.client.getApiVersion(), accept, context); } /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -219,15 +184,15 @@ private Mono> getReplicaWithResponseAsync( * @return a replica for a Container App Revision on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getReplicaAsync( - String resourceGroupName, String containerAppName, String revisionName, String replicaName) { + private Mono getReplicaAsync(String resourceGroupName, String containerAppName, String revisionName, + String replicaName) { return getReplicaWithResponseAsync(resourceGroupName, containerAppName, revisionName, replicaName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -239,15 +204,15 @@ private Mono getReplicaAsync( * @return a replica for a Container App Revision along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getReplicaWithResponse( - String resourceGroupName, String containerAppName, String revisionName, String replicaName, Context context) { + public Response getReplicaWithResponse(String resourceGroupName, String containerAppName, + String revisionName, String replicaName, Context context) { return getReplicaWithResponseAsync(resourceGroupName, containerAppName, revisionName, replicaName, context) .block(); } /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -258,15 +223,15 @@ public Response getReplicaWithResponse( * @return a replica for a Container App Revision. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ReplicaInner getReplica( - String resourceGroupName, String containerAppName, String revisionName, String replicaName) { + public ReplicaInner getReplica(String resourceGroupName, String containerAppName, String revisionName, + String replicaName) { return getReplicaWithResponse(resourceGroupName, containerAppName, revisionName, replicaName, Context.NONE) .getValue(); } /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -274,22 +239,18 @@ public ReplicaInner getReplica( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revision Replicas collection ARM resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listReplicasWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono> listReplicasWithResponseAsync(String resourceGroupName, + String containerAppName, String revisionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -304,24 +265,14 @@ private Mono> listReplicasWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listReplicas( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.listReplicas(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -330,22 +281,18 @@ private Mono> listReplicasWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revision Replicas collection ARM resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listReplicasWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + private Mono> listReplicasWithResponseAsync(String resourceGroupName, + String containerAppName, String revisionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -360,21 +307,13 @@ private Mono> listReplicasWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listReplicas( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context); + return service.listReplicas(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, this.client.getApiVersion(), accept, context); } /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -384,15 +323,15 @@ private Mono> listReplicasWithResponseAsync( * @return container App Revision Replicas collection ARM resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listReplicasAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono listReplicasAsync(String resourceGroupName, String containerAppName, + String revisionName) { return listReplicasWithResponseAsync(resourceGroupName, containerAppName, revisionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -403,14 +342,14 @@ private Mono listReplicasAsync( * @return container App Revision Replicas collection ARM resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listReplicasWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + public Response listReplicasWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { return listReplicasWithResponseAsync(resourceGroupName, containerAppName, revisionName, context).block(); } /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasImpl.java index 37db2d255db5a..d4af3bac90f63 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionReplicasImpl.java @@ -22,34 +22,28 @@ public final class ContainerAppsRevisionReplicasImpl implements ContainerAppsRev private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ContainerAppsRevisionReplicasImpl( - ContainerAppsRevisionReplicasClient innerClient, + public ContainerAppsRevisionReplicasImpl(ContainerAppsRevisionReplicasClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getReplicaWithResponse( - String resourceGroupName, String containerAppName, String revisionName, String replicaName, Context context) { - Response inner = - this - .serviceClient() - .getReplicaWithResponse(resourceGroupName, containerAppName, revisionName, replicaName, context); + public Response getReplicaWithResponse(String resourceGroupName, String containerAppName, + String revisionName, String replicaName, Context context) { + Response inner = this.serviceClient().getReplicaWithResponse(resourceGroupName, containerAppName, + revisionName, replicaName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ReplicaImpl(inner.getValue(), this.manager())); } else { return null; } } - public Replica getReplica( - String resourceGroupName, String containerAppName, String revisionName, String replicaName) { - ReplicaInner inner = - this.serviceClient().getReplica(resourceGroupName, containerAppName, revisionName, replicaName); + public Replica getReplica(String resourceGroupName, String containerAppName, String revisionName, + String replicaName) { + ReplicaInner inner + = this.serviceClient().getReplica(resourceGroupName, containerAppName, revisionName, replicaName); if (inner != null) { return new ReplicaImpl(inner, this.manager()); } else { @@ -57,15 +51,12 @@ public Replica getReplica( } } - public Response listReplicasWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { - Response inner = - this.serviceClient().listReplicasWithResponse(resourceGroupName, containerAppName, revisionName, context); + public Response listReplicasWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { + Response inner + = this.serviceClient().listReplicasWithResponse(resourceGroupName, containerAppName, revisionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ReplicaCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -73,8 +64,8 @@ public Response listReplicasWithResponse( } public ReplicaCollection listReplicas(String resourceGroupName, String containerAppName, String revisionName) { - ReplicaCollectionInner inner = - this.serviceClient().listReplicas(resourceGroupName, containerAppName, revisionName); + ReplicaCollectionInner inner + = this.serviceClient().listReplicas(resourceGroupName, containerAppName, revisionName); if (inner != null) { return new ReplicaCollectionImpl(inner, this.manager()); } else { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsClientImpl.java index b7b32bc2b31a5..9603ed5f754c3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsClientImpl.java @@ -31,23 +31,28 @@ import com.azure.resourcemanager.appcontainers.models.RevisionCollection; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ContainerAppsRevisionsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsRevisionsClient. + */ public final class ContainerAppsRevisionsClientImpl implements ContainerAppsRevisionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ContainerAppsRevisionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ContainerAppsRevisionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ContainerAppsRevisionsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create(ContainerAppsRevisionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ContainerAppsRevisionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -58,95 +63,68 @@ public final class ContainerAppsRevisionsClientImpl implements ContainerAppsRevi @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ContainerAppsRevisionsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listRevisions( - @HostParam("$host") String endpoint, + Mono> listRevisions(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getRevision( - @HostParam("$host") String endpoint, + Mono> getRevision(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/activate") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/activate") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> activateRevision( - @HostParam("$host") String endpoint, + Mono> activateRevision(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/deactivate") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/deactivate") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> deactivateRevision( - @HostParam("$host") String endpoint, + Mono> deactivateRevision(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/restart") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/revisions/{revisionName}/restart") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> restartRevision( - @HostParam("$host") String endpoint, + Mono> restartRevision(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @PathParam("revisionName") String revisionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @PathParam("revisionName") String revisionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listRevisionsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -154,22 +132,18 @@ Mono> listRevisionsNext( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listRevisionsSinglePageAsync( - String resourceGroupName, String containerAppName, String filter) { + private Mono> listRevisionsSinglePageAsync(String resourceGroupName, + String containerAppName, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -181,33 +155,16 @@ private Mono> listRevisionsSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listRevisions( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listRevisions(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -216,22 +173,18 @@ private Mono> listRevisionsSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listRevisionsSinglePageAsync( - String resourceGroupName, String containerAppName, String filter, Context context) { + private Mono> listRevisionsSinglePageAsync(String resourceGroupName, + String containerAppName, String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -244,29 +197,15 @@ private Mono> listRevisionsSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listRevisions( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listRevisions(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -276,16 +215,15 @@ private Mono> listRevisionsSinglePageAsync( * @return the Revisions for a given Container App as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listRevisionsAsync( - String resourceGroupName, String containerAppName, String filter) { - return new PagedFlux<>( - () -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), + private PagedFlux listRevisionsAsync(String resourceGroupName, String containerAppName, + String filter) { + return new PagedFlux<>(() -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), nextLink -> listRevisionsNextSinglePageAsync(nextLink)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -296,14 +234,13 @@ private PagedFlux listRevisionsAsync( @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listRevisionsAsync(String resourceGroupName, String containerAppName) { final String filter = null; - return new PagedFlux<>( - () -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), + return new PagedFlux<>(() -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter), nextLink -> listRevisionsNextSinglePageAsync(nextLink)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -314,16 +251,15 @@ private PagedFlux listRevisionsAsync(String resourceGroupName, St * @return the Revisions for a given Container App as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listRevisionsAsync( - String resourceGroupName, String containerAppName, String filter, Context context) { - return new PagedFlux<>( - () -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter, context), + private PagedFlux listRevisionsAsync(String resourceGroupName, String containerAppName, + String filter, Context context) { + return new PagedFlux<>(() -> listRevisionsSinglePageAsync(resourceGroupName, containerAppName, filter, context), nextLink -> listRevisionsNextSinglePageAsync(nextLink, context)); } /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -339,7 +275,7 @@ public PagedIterable listRevisions(String resourceGroupName, Stri /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. @@ -350,14 +286,14 @@ public PagedIterable listRevisions(String resourceGroupName, Stri * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context) { + public PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context) { return new PagedIterable<>(listRevisionsAsync(resourceGroupName, containerAppName, filter, context)); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -367,19 +303,15 @@ public PagedIterable listRevisions( * @return a revision of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono> getRevisionWithResponseAsync(String resourceGroupName, + String containerAppName, String revisionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -394,24 +326,14 @@ private Mono> getRevisionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -422,19 +344,15 @@ private Mono> getRevisionWithResponseAsync( * @return a revision of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + private Mono> getRevisionWithResponseAsync(String resourceGroupName, + String containerAppName, String revisionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -449,21 +367,13 @@ private Mono> getRevisionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context); + return service.getRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, this.client.getApiVersion(), accept, context); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -473,15 +383,15 @@ private Mono> getRevisionWithResponseAsync( * @return a revision of a Container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getRevisionAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono getRevisionAsync(String resourceGroupName, String containerAppName, + String revisionName) { return getRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -492,14 +402,14 @@ private Mono getRevisionAsync( * @return a revision of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + public Response getRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { return getRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName, context).block(); } /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -515,7 +425,7 @@ public RevisionInner getRevision(String resourceGroupName, String containerAppNa /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -525,19 +435,15 @@ public RevisionInner getRevision(String resourceGroupName, String containerAppNa * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> activateRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono> activateRevisionWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -552,24 +458,14 @@ private Mono> activateRevisionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .activateRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.activateRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -580,19 +476,15 @@ private Mono> activateRevisionWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> activateRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + private Mono> activateRevisionWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -607,21 +499,13 @@ private Mono> activateRevisionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .activateRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context); + return service.activateRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, this.client.getApiVersion(), accept, context); } /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -638,7 +522,7 @@ private Mono activateRevisionAsync(String resourceGroupName, String contai /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -649,14 +533,14 @@ private Mono activateRevisionAsync(String resourceGroupName, String contai * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response activateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + public Response activateRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { return activateRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName, context).block(); } /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -671,7 +555,7 @@ public void activateRevision(String resourceGroupName, String containerAppName, /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -681,19 +565,15 @@ public void activateRevision(String resourceGroupName, String containerAppName, * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deactivateRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono> deactivateRevisionWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -709,23 +589,14 @@ private Mono> deactivateRevisionWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .deactivateRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context)) + context -> service.deactivateRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -736,19 +607,15 @@ private Mono> deactivateRevisionWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deactivateRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + private Mono> deactivateRevisionWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -763,21 +630,13 @@ private Mono> deactivateRevisionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .deactivateRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context); + return service.deactivateRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, this.client.getApiVersion(), accept, context); } /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -794,7 +653,7 @@ private Mono deactivateRevisionAsync(String resourceGroupName, String cont /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -805,14 +664,14 @@ private Mono deactivateRevisionAsync(String resourceGroupName, String cont * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deactivateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + public Response deactivateRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { return deactivateRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName, context).block(); } /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -827,7 +686,7 @@ public void deactivateRevision(String resourceGroupName, String containerAppName /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -837,19 +696,15 @@ public void deactivateRevision(String resourceGroupName, String containerAppName * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> restartRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName) { + private Mono> restartRevisionWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -864,24 +719,14 @@ private Mono> restartRevisionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .restartRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.restartRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, revisionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -892,19 +737,15 @@ private Mono> restartRevisionWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> restartRevisionWithResponseAsync( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + private Mono> restartRevisionWithResponseAsync(String resourceGroupName, String containerAppName, + String revisionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -919,21 +760,13 @@ private Mono> restartRevisionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .restartRevision( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - revisionName, - this.client.getApiVersion(), - accept, - context); + return service.restartRevision(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, revisionName, this.client.getApiVersion(), accept, context); } /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -950,7 +783,7 @@ private Mono restartRevisionAsync(String resourceGroupName, String contain /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -961,14 +794,14 @@ private Mono restartRevisionAsync(String resourceGroupName, String contain * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response restartRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { + public Response restartRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { return restartRevisionWithResponseAsync(resourceGroupName, containerAppName, revisionName, context).block(); } /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -983,14 +816,15 @@ public void restartRevision(String resourceGroupName, String containerAppName, S /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revisions collection ARM resource along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listRevisionsNextSinglePageAsync(String nextLink) { @@ -998,37 +832,29 @@ private Mono> listRevisionsNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listRevisionsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revisions collection ARM resource along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listRevisionsNextSinglePageAsync(String nextLink, Context context) { @@ -1036,23 +862,13 @@ private Mono> listRevisionsNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listRevisionsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listRevisionsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsImpl.java index 8c4c17319b32f..2f77c0175db5e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsRevisionsImpl.java @@ -21,8 +21,7 @@ public final class ContainerAppsRevisionsImpl implements ContainerAppsRevisions private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ContainerAppsRevisionsImpl( - ContainerAppsRevisionsClient innerClient, + public ContainerAppsRevisionsImpl(ContainerAppsRevisionsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,25 +29,22 @@ public ContainerAppsRevisionsImpl( public PagedIterable listRevisions(String resourceGroupName, String containerAppName) { PagedIterable inner = this.serviceClient().listRevisions(resourceGroupName, containerAppName); - return Utils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); } - public PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context) { - PagedIterable inner = - this.serviceClient().listRevisions(resourceGroupName, containerAppName, filter, context); - return Utils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); + public PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context) { + PagedIterable inner + = this.serviceClient().listRevisions(resourceGroupName, containerAppName, filter, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new RevisionImpl(inner1, this.manager())); } - public Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { - Response inner = - this.serviceClient().getRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); + public Response getRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { + Response inner + = this.serviceClient().getRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RevisionImpl(inner.getValue(), this.manager())); } else { return null; @@ -64,33 +60,30 @@ public Revision getRevision(String resourceGroupName, String containerAppName, S } } - public Response activateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { - return this - .serviceClient() - .activateRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); + public Response activateRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { + return this.serviceClient().activateRevisionWithResponse(resourceGroupName, containerAppName, revisionName, + context); } public void activateRevision(String resourceGroupName, String containerAppName, String revisionName) { this.serviceClient().activateRevision(resourceGroupName, containerAppName, revisionName); } - public Response deactivateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { - return this - .serviceClient() - .deactivateRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); + public Response deactivateRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { + return this.serviceClient().deactivateRevisionWithResponse(resourceGroupName, containerAppName, revisionName, + context); } public void deactivateRevision(String resourceGroupName, String containerAppName, String revisionName) { this.serviceClient().deactivateRevision(resourceGroupName, containerAppName, revisionName); } - public Response restartRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context) { - return this - .serviceClient() - .restartRevisionWithResponse(resourceGroupName, containerAppName, revisionName, context); + public Response restartRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context) { + return this.serviceClient().restartRevisionWithResponse(resourceGroupName, containerAppName, revisionName, + context); } public void restartRevision(String resourceGroupName, String containerAppName, String revisionName) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsClientImpl.java index 68605a00c5567..f2afb95f62170 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsClientImpl.java @@ -38,24 +38,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ContainerAppsSourceControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in ContainerAppsSourceControlsClient. + */ public final class ContainerAppsSourceControlsClientImpl implements ContainerAppsSourceControlsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ContainerAppsSourceControlsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ContainerAppsSourceControlsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ContainerAppsSourceControlsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ContainerAppsSourceControlsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ContainerAppsSourceControlsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -66,102 +70,80 @@ public final class ContainerAppsSourceControlsClientImpl implements ContainerApp @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ContainerAppsSourceControlsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listByContainerApp( - @HostParam("$host") String endpoint, + Mono> listByContainerApp(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("containerAppName") String containerAppName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("containerAppName") String containerAppName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols/{sourceControlName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols/{sourceControlName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("containerAppName") String containerAppName, - @PathParam("sourceControlName") String sourceControlName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("sourceControlName") String sourceControlName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols/{sourceControlName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols/{sourceControlName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("containerAppName") String containerAppName, - @PathParam("sourceControlName") String sourceControlName, - @QueryParam("api-version") String apiVersion, + @PathParam("sourceControlName") String sourceControlName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") SourceControlInner sourceControlEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols/{sourceControlName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/containerApps/{containerAppName}/sourcecontrols/{sourceControlName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("containerAppName") String containerAppName, - @PathParam("sourceControlName") String sourceControlName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("sourceControlName") String sourceControlName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listByContainerAppNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App SourceControls in a given resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByContainerAppSinglePageAsync( - String resourceGroupName, String containerAppName) { + private Mono> listByContainerAppSinglePageAsync(String resourceGroupName, + String containerAppName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -174,31 +156,16 @@ private Mono> listByContainerAppSinglePageAsyn final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listByContainerApp( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listByContainerApp(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -206,22 +173,18 @@ private Mono> listByContainerAppSinglePageAsyn * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App SourceControls in a given resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByContainerAppSinglePageAsync( - String resourceGroupName, String containerAppName, Context context) { + private Mono> listByContainerAppSinglePageAsync(String resourceGroupName, + String containerAppName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -234,28 +197,15 @@ private Mono> listByContainerAppSinglePageAsyn final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByContainerApp( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByContainerApp(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -265,14 +215,13 @@ private Mono> listByContainerAppSinglePageAsyn */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByContainerAppAsync(String resourceGroupName, String containerAppName) { - return new PagedFlux<>( - () -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName), + return new PagedFlux<>(() -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName), nextLink -> listByContainerAppNextSinglePageAsync(nextLink)); } /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. @@ -282,23 +231,22 @@ private PagedFlux listByContainerAppAsync(String resourceGro * @return the Container App SourceControls in a given resource group as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByContainerAppAsync( - String resourceGroupName, String containerAppName, Context context) { - return new PagedFlux<>( - () -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName, context), + private PagedFlux listByContainerAppAsync(String resourceGroupName, String containerAppName, + Context context) { + return new PagedFlux<>(() -> listByContainerAppSinglePageAsync(resourceGroupName, containerAppName, context), nextLink -> listByContainerAppNextSinglePageAsync(nextLink, context)); } /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Container App SourceControls in a given resource group as paginated response with {@link - * PagedIterable}. + * @return the Container App SourceControls in a given resource group as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName) { @@ -307,25 +255,25 @@ public PagedIterable listByContainerApp(String resourceGroup /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Container App SourceControls in a given resource group as paginated response with {@link - * PagedIterable}. + * @return the Container App SourceControls in a given resource group as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByContainerApp( - String resourceGroupName, String containerAppName, Context context) { + public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, + Context context) { return new PagedIterable<>(listByContainerAppAsync(resourceGroupName, containerAppName, context)); } /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -335,19 +283,15 @@ public PagedIterable listByContainerApp( * @return a SourceControl of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String containerAppName, String sourceControlName) { + private Mono> getWithResponseAsync(String resourceGroupName, String containerAppName, + String sourceControlName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -363,24 +307,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - sourceControlName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, sourceControlName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -391,19 +325,15 @@ private Mono> getWithResponseAsync( * @return a SourceControl of a Container App along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String containerAppName, + String sourceControlName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -419,21 +349,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - sourceControlName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, sourceControlName, this.client.getApiVersion(), accept, context); } /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -443,15 +365,15 @@ private Mono> getWithResponseAsync( * @return a SourceControl of a Container App on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String containerAppName, String sourceControlName) { + private Mono getAsync(String resourceGroupName, String containerAppName, + String sourceControlName) { return getWithResponseAsync(resourceGroupName, containerAppName, sourceControlName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -462,14 +384,14 @@ private Mono getAsync( * @return a SourceControl of a Container App along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { + public Response getWithResponse(String resourceGroupName, String containerAppName, + String sourceControlName, Context context) { return getWithResponseAsync(resourceGroupName, containerAppName, sourceControlName, context).block(); } /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -485,7 +407,7 @@ public SourceControlInner get(String resourceGroupName, String containerAppName, /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -496,22 +418,15 @@ public SourceControlInner get(String resourceGroupName, String containerAppName, * @return container App SourceControl along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -533,25 +448,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - sourceControlName, - this.client.getApiVersion(), - sourceControlEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, sourceControlName, this.client.getApiVersion(), + sourceControlEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -563,23 +468,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return container App SourceControl along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -601,22 +498,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - sourceControlName, - this.client.getApiVersion(), - sourceControlEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, sourceControlName, this.client.getApiVersion(), sourceControlEnvelope, accept, context); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -628,26 +516,17 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, SourceControlInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String containerAppName, - String sourceControlName, + String resourceGroupName, String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - SourceControlInner.class, - SourceControlInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, + sourceControlName, sourceControlEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + SourceControlInner.class, SourceControlInner.class, this.client.getContext()); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -660,24 +539,18 @@ private PollerFlux, SourceControlInner> beginCrea */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, SourceControlInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context) { + String resourceGroupName, String containerAppName, String sourceControlName, + SourceControlInner sourceControlEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SourceControlInner.class, SourceControlInner.class, context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, containerAppName, + sourceControlName, sourceControlEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + SourceControlInner.class, SourceControlInner.class, context); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -688,11 +561,8 @@ private PollerFlux, SourceControlInner> beginCrea * @return the {@link SyncPoller} for polling of container App SourceControl. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SourceControlInner> beginCreateOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope) { + public SyncPoller, SourceControlInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope) { return this .beginCreateOrUpdateAsync(resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope) .getSyncPoller(); @@ -700,7 +570,7 @@ public SyncPoller, SourceControlInner> beginCreat /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -712,21 +582,15 @@ public SyncPoller, SourceControlInner> beginCreat * @return the {@link SyncPoller} for polling of container App SourceControl. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SourceControlInner> beginCreateOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope, context) - .getSyncPoller(); + public SyncPoller, SourceControlInner> beginCreateOrUpdate(String resourceGroupName, + String containerAppName, String sourceControlName, SourceControlInner sourceControlEnvelope, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, containerAppName, sourceControlName, + sourceControlEnvelope, context).getSyncPoller(); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -737,19 +601,15 @@ public SyncPoller, SourceControlInner> beginCreat * @return container App SourceControl on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope) { + private Mono createOrUpdateAsync(String resourceGroupName, String containerAppName, + String sourceControlName, SourceControlInner sourceControlEnvelope) { return beginCreateOrUpdateAsync(resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -761,21 +621,15 @@ private Mono createOrUpdateAsync( * @return container App SourceControl on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String containerAppName, + String sourceControlName, SourceControlInner sourceControlEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope, + context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -786,18 +640,15 @@ private Mono createOrUpdateAsync( * @return container App SourceControl. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SourceControlInner createOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope) { + public SourceControlInner createOrUpdate(String resourceGroupName, String containerAppName, + String sourceControlName, SourceControlInner sourceControlEnvelope) { return createOrUpdateAsync(resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope) .block(); } /** * Create or update the SourceControl for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -809,20 +660,15 @@ public SourceControlInner createOrUpdate( * @return container App SourceControl. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SourceControlInner createOrUpdate( - String resourceGroupName, - String containerAppName, - String sourceControlName, - SourceControlInner sourceControlEnvelope, - Context context) { - return createOrUpdateAsync( - resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope, context) - .block(); + public SourceControlInner createOrUpdate(String resourceGroupName, String containerAppName, + String sourceControlName, SourceControlInner sourceControlEnvelope, Context context) { + return createOrUpdateAsync(resourceGroupName, containerAppName, sourceControlName, sourceControlEnvelope, + context).block(); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -832,19 +678,15 @@ public SourceControlInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String containerAppName, String sourceControlName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String containerAppName, + String sourceControlName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -860,24 +702,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - sourceControlName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, containerAppName, sourceControlName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -888,19 +720,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String containerAppName, + String sourceControlName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -916,21 +744,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - containerAppName, - sourceControlName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + containerAppName, sourceControlName, this.client.getApiVersion(), accept, context); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -940,19 +760,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String containerAppName, String sourceControlName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, containerAppName, sourceControlName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String containerAppName, + String sourceControlName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, containerAppName, sourceControlName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -963,19 +781,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String containerAppName, + String sourceControlName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, containerAppName, sourceControlName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, containerAppName, sourceControlName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -985,14 +802,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String containerAppName, String sourceControlName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String containerAppName, + String sourceControlName) { return this.beginDeleteAsync(resourceGroupName, containerAppName, sourceControlName).getSyncPoller(); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -1003,14 +820,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String containerAppName, + String sourceControlName, Context context) { return this.beginDeleteAsync(resourceGroupName, containerAppName, sourceControlName, context).getSyncPoller(); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -1021,14 +838,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String containerAppName, String sourceControlName) { - return beginDeleteAsync(resourceGroupName, containerAppName, sourceControlName) - .last() + return beginDeleteAsync(resourceGroupName, containerAppName, sourceControlName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -1039,16 +855,15 @@ private Mono deleteAsync(String resourceGroupName, String containerAppName * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { - return beginDeleteAsync(resourceGroupName, containerAppName, sourceControlName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String containerAppName, String sourceControlName, + Context context) { + return beginDeleteAsync(resourceGroupName, containerAppName, sourceControlName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -1063,7 +878,7 @@ public void delete(String resourceGroupName, String containerAppName, String sou /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. @@ -1079,14 +894,15 @@ public void delete(String resourceGroupName, String containerAppName, String sou /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return sourceControl collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return sourceControl collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByContainerAppNextSinglePageAsync(String nextLink) { @@ -1094,63 +910,45 @@ private Mono> listByContainerAppNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByContainerAppNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return sourceControl collection ARM resource along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return sourceControl collection ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByContainerAppNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByContainerAppNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByContainerAppNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByContainerAppNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsImpl.java index c4b25c702c741..4f33cd3ab4779 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ContainerAppsSourceControlsImpl.java @@ -21,35 +21,31 @@ public final class ContainerAppsSourceControlsImpl implements ContainerAppsSourc private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ContainerAppsSourceControlsImpl( - ContainerAppsSourceControlsClient innerClient, + public ContainerAppsSourceControlsImpl(ContainerAppsSourceControlsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName) { - PagedIterable inner = - this.serviceClient().listByContainerApp(resourceGroupName, containerAppName); - return Utils.mapPage(inner, inner1 -> new SourceControlImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByContainerApp(resourceGroupName, containerAppName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SourceControlImpl(inner1, this.manager())); } - public PagedIterable listByContainerApp( - String resourceGroupName, String containerAppName, Context context) { - PagedIterable inner = - this.serviceClient().listByContainerApp(resourceGroupName, containerAppName, context); - return Utils.mapPage(inner, inner1 -> new SourceControlImpl(inner1, this.manager())); + public PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, + Context context) { + PagedIterable inner + = this.serviceClient().listByContainerApp(resourceGroupName, containerAppName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SourceControlImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String containerAppName, String sourceControlName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, containerAppName, sourceControlName, context); + public Response getWithResponse(String resourceGroupName, String containerAppName, + String sourceControlName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, containerAppName, sourceControlName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SourceControlImpl(inner.getValue(), this.manager())); } else { return null; @@ -74,109 +70,77 @@ public void delete(String resourceGroupName, String containerAppName, String sou } public SourceControl getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String sourceControlName = Utils.getValueFromIdByName(id, "sourcecontrols"); + String sourceControlName = ResourceManagerUtils.getValueFromIdByName(id, "sourcecontrols"); if (sourceControlName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); } return this.getWithResponse(resourceGroupName, containerAppName, sourceControlName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String sourceControlName = Utils.getValueFromIdByName(id, "sourcecontrols"); + String sourceControlName = ResourceManagerUtils.getValueFromIdByName(id, "sourcecontrols"); if (sourceControlName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); } return this.getWithResponse(resourceGroupName, containerAppName, sourceControlName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String sourceControlName = Utils.getValueFromIdByName(id, "sourcecontrols"); + String sourceControlName = ResourceManagerUtils.getValueFromIdByName(id, "sourcecontrols"); if (sourceControlName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); } this.delete(resourceGroupName, containerAppName, sourceControlName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String containerAppName = Utils.getValueFromIdByName(id, "containerApps"); + String containerAppName = ResourceManagerUtils.getValueFromIdByName(id, "containerApps"); if (containerAppName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'containerApps'.", id))); } - String sourceControlName = Utils.getValueFromIdByName(id, "sourcecontrols"); + String sourceControlName = ResourceManagerUtils.getValueFromIdByName(id, "sourcecontrols"); if (sourceControlName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sourcecontrols'.", id))); } this.delete(resourceGroupName, containerAppName, sourceControlName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CustomHostnameAnalysisResultImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CustomHostnameAnalysisResultImpl.java index 294b901837f9c..189b97144b22c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CustomHostnameAnalysisResultImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/CustomHostnameAnalysisResultImpl.java @@ -16,8 +16,7 @@ public final class CustomHostnameAnalysisResultImpl implements CustomHostnameAna private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - CustomHostnameAnalysisResultImpl( - CustomHostnameAnalysisResultInner innerObject, + CustomHostnameAnalysisResultImpl(CustomHostnameAnalysisResultInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentImpl.java index 9afafdafa4c73..d460f9e1331a4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentImpl.java @@ -5,9 +5,11 @@ package com.azure.resourcemanager.appcontainers.implementation; import com.azure.core.http.rest.Response; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner; import com.azure.resourcemanager.appcontainers.models.DaprComponent; +import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding; import com.azure.resourcemanager.appcontainers.models.DaprMetadata; import com.azure.resourcemanager.appcontainers.models.DaprSecretsCollection; import com.azure.resourcemanager.appcontainers.models.Secret; @@ -31,6 +33,10 @@ public String type() { return this.innerModel().type(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String componentType() { return this.innerModel().componentType(); } @@ -78,6 +84,15 @@ public List scopes() { } } + public List serviceComponentBind() { + List inner = this.innerModel().serviceComponentBind(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + public String resourceGroupName() { return resourceGroupName; } @@ -96,32 +111,24 @@ private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager( private String componentName; - public DaprComponentImpl withExistingConnectedEnvironment( - String resourceGroupName, String connectedEnvironmentName) { + public DaprComponentImpl withExistingConnectedEnvironment(String resourceGroupName, + String connectedEnvironmentName) { this.resourceGroupName = resourceGroupName; this.connectedEnvironmentName = connectedEnvironmentName; return this; } public DaprComponent create() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsDaprComponents() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectedEnvironmentsDaprComponents().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), Context.NONE).getValue(); return this; } public DaprComponent create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsDaprComponents() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectedEnvironmentsDaprComponents().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), context).getValue(); return this; } @@ -136,67 +143,49 @@ public DaprComponentImpl update() { } public DaprComponent apply() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsDaprComponents() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectedEnvironmentsDaprComponents().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), Context.NONE).getValue(); return this; } public DaprComponent apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsDaprComponents() - .createOrUpdateWithResponse( - resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectedEnvironmentsDaprComponents().createOrUpdateWithResponse( + resourceGroupName, connectedEnvironmentName, componentName, this.innerModel(), context).getValue(); return this; } - DaprComponentImpl( - DaprComponentInner innerObject, + DaprComponentImpl(DaprComponentInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.connectedEnvironmentName = Utils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); - this.componentName = Utils.getValueFromIdByName(innerObject.id(), "daprComponents"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.connectedEnvironmentName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "connectedEnvironments"); + this.componentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "daprComponents"); } public DaprComponent refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsDaprComponents() - .getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsDaprComponents() + .getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, Context.NONE).getValue(); return this; } public DaprComponent refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectedEnvironmentsDaprComponents() - .getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectedEnvironmentsDaprComponents() + .getWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context).getValue(); return this; } public Response listSecretsWithResponse(Context context) { - return serviceManager - .connectedEnvironmentsDaprComponents() - .listSecretsWithResponse(resourceGroupName, connectedEnvironmentName, componentName, context); + return serviceManager.connectedEnvironmentsDaprComponents().listSecretsWithResponse(resourceGroupName, + connectedEnvironmentName, componentName, context); } public DaprSecretsCollection listSecrets() { - return serviceManager - .connectedEnvironmentsDaprComponents() - .listSecrets(resourceGroupName, connectedEnvironmentName, componentName); + return serviceManager.connectedEnvironmentsDaprComponents().listSecrets(resourceGroupName, + connectedEnvironmentName, componentName); } public DaprComponentImpl withComponentType(String componentType) { @@ -238,4 +227,9 @@ public DaprComponentImpl withScopes(List scopes) { this.innerModel().withScopes(scopes); return this; } + + public DaprComponentImpl withServiceComponentBind(List serviceComponentBind) { + this.innerModel().withServiceComponentBind(serviceComponentBind); + return this; + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesClientImpl.java new file mode 100644 index 0000000000000..fbda276a308c8 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesClientImpl.java @@ -0,0 +1,812 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.DaprComponentResiliencyPoliciesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPoliciesCollection; +import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in DaprComponentResiliencyPoliciesClient. + */ +public final class DaprComponentResiliencyPoliciesClientImpl implements DaprComponentResiliencyPoliciesClient { + /** + * The proxy service used to perform REST calls. + */ + private final DaprComponentResiliencyPoliciesService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of DaprComponentResiliencyPoliciesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + DaprComponentResiliencyPoliciesClientImpl(ContainerAppsApiClientImpl client) { + this.service = RestProxy.create(DaprComponentResiliencyPoliciesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientDaprComponentResiliencyPolicies to be used by + * the proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface DaprComponentResiliencyPoliciesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}/resiliencyPolicies") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}/resiliencyPolicies/{name}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @PathParam("name") String name, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, String componentName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, String componentName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + componentName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName, + String componentName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, componentName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName, + String componentName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, componentName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName, + String componentName) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName, componentName)); + } + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName, + String componentName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName, componentName, context)); + } + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, String name, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, name, this.client.getApiVersion(), accept, context); + } + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String environmentName, + String componentName, String name) { + return getWithResponseAsync(resourceGroupName, environmentName, componentName, name) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, + String environmentName, String componentName, String name, Context context) { + return getWithResponseAsync(resourceGroupName, environmentName, componentName, name, context).block(); + } + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DaprComponentResiliencyPolicyInner get(String resourceGroupName, String environmentName, + String componentName, String name) { + return getWithResponse(resourceGroupName, environmentName, componentName, name, Context.NONE).getValue(); + } + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, String name, + DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (daprComponentResiliencyPolicyEnvelope == null) { + return Mono.error(new IllegalArgumentException( + "Parameter daprComponentResiliencyPolicyEnvelope is required and cannot be null.")); + } else { + daprComponentResiliencyPolicyEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, name, this.client.getApiVersion(), + daprComponentResiliencyPolicyEnvelope, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, String name, + DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (daprComponentResiliencyPolicyEnvelope == null) { + return Mono.error(new IllegalArgumentException( + "Parameter daprComponentResiliencyPolicyEnvelope is required and cannot be null.")); + } else { + daprComponentResiliencyPolicyEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, name, this.client.getApiVersion(), daprComponentResiliencyPolicyEnvelope, + accept, context); + } + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, + String environmentName, String componentName, String name, + DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, componentName, name, + daprComponentResiliencyPolicyEnvelope).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createOrUpdateWithResponse(String resourceGroupName, + String environmentName, String componentName, String name, + DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, componentName, name, + daprComponentResiliencyPolicyEnvelope, context).block(); + } + + /** + * Creates or updates a Dapr component resiliency policy. + * + * Creates or updates a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param daprComponentResiliencyPolicyEnvelope Configuration details of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policy. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DaprComponentResiliencyPolicyInner createOrUpdate(String resourceGroupName, String environmentName, + String componentName, String name, DaprComponentResiliencyPolicyInner daprComponentResiliencyPolicyEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, environmentName, componentName, name, + daprComponentResiliencyPolicyEnvelope, Context.NONE).getValue(); + } + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String componentName, String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String componentName, String name, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (componentName == null) { + return Mono.error(new IllegalArgumentException("Parameter componentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, name, this.client.getApiVersion(), accept, context); + } + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String environmentName, String componentName, + String name) { + return deleteWithResponseAsync(resourceGroupName, environmentName, componentName, name) + .flatMap(ignored -> Mono.empty()); + } + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + String name, Context context) { + return deleteWithResponseAsync(resourceGroupName, environmentName, componentName, name, context).block(); + } + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String environmentName, String componentName, String name) { + deleteWithResponse(resourceGroupName, environmentName, componentName, name, Context.NONE); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policies ARM resource along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Component Resiliency Policies ARM resource along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesImpl.java new file mode 100644 index 0000000000000..03620b7c46562 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPoliciesImpl.java @@ -0,0 +1,186 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.DaprComponentResiliencyPoliciesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicies; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicy; + +public final class DaprComponentResiliencyPoliciesImpl implements DaprComponentResiliencyPolicies { + private static final ClientLogger LOGGER = new ClientLogger(DaprComponentResiliencyPoliciesImpl.class); + + private final DaprComponentResiliencyPoliciesClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public DaprComponentResiliencyPoliciesImpl(DaprComponentResiliencyPoliciesClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String environmentName, + String componentName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, componentName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new DaprComponentResiliencyPolicyImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String environmentName, + String componentName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, componentName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new DaprComponentResiliencyPolicyImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String environmentName, + String componentName, String name, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, componentName, name, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DaprComponentResiliencyPolicyImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public DaprComponentResiliencyPolicy get(String resourceGroupName, String environmentName, String componentName, + String name) { + DaprComponentResiliencyPolicyInner inner + = this.serviceClient().get(resourceGroupName, environmentName, componentName, name); + if (inner != null) { + return new DaprComponentResiliencyPolicyImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + String name, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, environmentName, componentName, name, + context); + } + + public void delete(String resourceGroupName, String environmentName, String componentName, String name) { + this.serviceClient().delete(resourceGroupName, environmentName, componentName, name); + } + + public DaprComponentResiliencyPolicy getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); + if (componentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, componentName, name, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); + if (componentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, componentName, name, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); + if (componentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + this.deleteWithResponse(resourceGroupName, environmentName, componentName, name, Context.NONE); + } + + public Response deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String componentName = ResourceManagerUtils.getValueFromIdByName(id, "daprComponents"); + if (componentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprComponents'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "resiliencyPolicies"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resiliencyPolicies'.", id))); + } + return this.deleteWithResponse(resourceGroupName, environmentName, componentName, name, context); + } + + private DaprComponentResiliencyPoliciesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public DaprComponentResiliencyPolicyImpl define(String name) { + return new DaprComponentResiliencyPolicyImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPolicyImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPolicyImpl.java new file mode 100644 index 0000000000000..cbf400cbf2422 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentResiliencyPolicyImpl.java @@ -0,0 +1,143 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicy; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; + +public final class DaprComponentResiliencyPolicyImpl implements DaprComponentResiliencyPolicy, + DaprComponentResiliencyPolicy.Definition, DaprComponentResiliencyPolicy.Update { + private DaprComponentResiliencyPolicyInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public DaprComponentResiliencyPolicyConfiguration inboundPolicy() { + return this.innerModel().inboundPolicy(); + } + + public DaprComponentResiliencyPolicyConfiguration outboundPolicy() { + return this.innerModel().outboundPolicy(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public DaprComponentResiliencyPolicyInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String environmentName; + + private String componentName; + + private String name; + + public DaprComponentResiliencyPolicyImpl withExistingDaprComponent(String resourceGroupName, String environmentName, + String componentName) { + this.resourceGroupName = resourceGroupName; + this.environmentName = environmentName; + this.componentName = componentName; + return this; + } + + public DaprComponentResiliencyPolicy create() { + this.innerObject + = serviceManager.serviceClient().getDaprComponentResiliencyPolicies().createOrUpdateWithResponse( + resourceGroupName, environmentName, componentName, name, this.innerModel(), Context.NONE).getValue(); + return this; + } + + public DaprComponentResiliencyPolicy create(Context context) { + this.innerObject + = serviceManager.serviceClient().getDaprComponentResiliencyPolicies().createOrUpdateWithResponse( + resourceGroupName, environmentName, componentName, name, this.innerModel(), context).getValue(); + return this; + } + + DaprComponentResiliencyPolicyImpl(String name, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new DaprComponentResiliencyPolicyInner(); + this.serviceManager = serviceManager; + this.name = name; + } + + public DaprComponentResiliencyPolicyImpl update() { + return this; + } + + public DaprComponentResiliencyPolicy apply() { + this.innerObject + = serviceManager.serviceClient().getDaprComponentResiliencyPolicies().createOrUpdateWithResponse( + resourceGroupName, environmentName, componentName, name, this.innerModel(), Context.NONE).getValue(); + return this; + } + + public DaprComponentResiliencyPolicy apply(Context context) { + this.innerObject + = serviceManager.serviceClient().getDaprComponentResiliencyPolicies().createOrUpdateWithResponse( + resourceGroupName, environmentName, componentName, name, this.innerModel(), context).getValue(); + return this; + } + + DaprComponentResiliencyPolicyImpl(DaprComponentResiliencyPolicyInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.componentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "daprComponents"); + this.name = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resiliencyPolicies"); + } + + public DaprComponentResiliencyPolicy refresh() { + this.innerObject = serviceManager.serviceClient().getDaprComponentResiliencyPolicies() + .getWithResponse(resourceGroupName, environmentName, componentName, name, Context.NONE).getValue(); + return this; + } + + public DaprComponentResiliencyPolicy refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getDaprComponentResiliencyPolicies() + .getWithResponse(resourceGroupName, environmentName, componentName, name, context).getValue(); + return this; + } + + public DaprComponentResiliencyPolicyImpl + withInboundPolicy(DaprComponentResiliencyPolicyConfiguration inboundPolicy) { + this.innerModel().withInboundPolicy(inboundPolicy); + return this; + } + + public DaprComponentResiliencyPolicyImpl + withOutboundPolicy(DaprComponentResiliencyPolicyConfiguration outboundPolicy) { + this.innerModel().withOutboundPolicy(outboundPolicy); + return this; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsClientImpl.java index fde054b8f7f2e..0a9d6b7f95474 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsClientImpl.java @@ -35,22 +35,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DaprComponentsClient. */ +/** + * An instance of this class provides access to all the operations defined in DaprComponentsClient. + */ public final class DaprComponentsClientImpl implements DaprComponentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DaprComponentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of DaprComponentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ DaprComponentsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(DaprComponentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(DaprComponentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -61,117 +67,88 @@ public final class DaprComponentsClientImpl implements DaprComponentsClient { @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface DaprComponentsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("componentName") String componentName, + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") DaprComponentInner daprComponentEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}/listSecrets") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprComponents/{componentName}/listSecrets") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listSecrets( - @HostParam("$host") String endpoint, + Mono> listSecrets(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("componentName") String componentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("componentName") String componentName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a managed environment along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String environmentName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -183,32 +160,16 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -216,22 +177,18 @@ private Mono> listSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a managed environment along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -244,28 +201,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -275,14 +219,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String environmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, environmentName), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -293,14 +236,13 @@ private PagedFlux listAsync(String resourceGroupName, String */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String environmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, environmentName, context), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -315,7 +257,7 @@ public PagedIterable list(String resourceGroupName, String e /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -331,7 +273,7 @@ public PagedIterable list(String resourceGroupName, String e /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -341,19 +283,15 @@ public PagedIterable list(String resourceGroupName, String e * @return a dapr component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String componentName) { + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String componentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -368,24 +306,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -396,19 +324,15 @@ private Mono> getWithResponseAsync( * @return a dapr component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String componentName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String componentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -423,21 +347,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, this.client.getApiVersion(), accept, context); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -454,7 +370,7 @@ private Mono getAsync(String resourceGroupName, String envir /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -465,14 +381,14 @@ private Mono getAsync(String resourceGroupName, String envir * @return a dapr component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context) { + public Response getWithResponse(String resourceGroupName, String environmentName, + String componentName, Context context) { return getWithResponseAsync(resourceGroupName, environmentName, componentName, context).block(); } /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -488,9 +404,9 @@ public DaprComponentInner get(String resourceGroupName, String environmentName, /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -501,22 +417,15 @@ public DaprComponentInner get(String resourceGroupName, String environmentName, * @return dapr Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, DaprComponentInner daprComponentEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -537,27 +446,17 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - daprComponentEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, this.client.getApiVersion(), daprComponentEnvelope, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -569,23 +468,15 @@ private Mono> createOrUpdateWithResponseAsync( * @return dapr Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, DaprComponentInner daprComponentEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -606,24 +497,15 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - daprComponentEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, this.client.getApiVersion(), daprComponentEnvelope, accept, context); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -634,20 +516,17 @@ private Mono> createOrUpdateWithResponseAsync( * @return dapr Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope) { + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String componentName, DaprComponentInner daprComponentEnvelope) { return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, componentName, daprComponentEnvelope) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -659,22 +538,17 @@ private Mono createOrUpdateAsync( * @return dapr Component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, environmentName, componentName, daprComponentEnvelope, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String componentName, DaprComponentInner daprComponentEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, componentName, daprComponentEnvelope, + context).block(); } /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -685,21 +559,17 @@ public Response createOrUpdateWithResponse( * @return dapr Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DaprComponentInner createOrUpdate( - String resourceGroupName, - String environmentName, - String componentName, + public DaprComponentInner createOrUpdate(String resourceGroupName, String environmentName, String componentName, DaprComponentInner daprComponentEnvelope) { - return createOrUpdateWithResponse( - resourceGroupName, environmentName, componentName, daprComponentEnvelope, Context.NONE) - .getValue(); + return createOrUpdateWithResponse(resourceGroupName, environmentName, componentName, daprComponentEnvelope, + Context.NONE).getValue(); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -709,19 +579,15 @@ public DaprComponentInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String componentName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String componentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -736,26 +602,16 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -766,19 +622,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String componentName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String componentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -793,23 +645,15 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, this.client.getApiVersion(), accept, context); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -826,9 +670,9 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -839,16 +683,16 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + Context context) { return deleteWithResponseAsync(resourceGroupName, environmentName, componentName, context).block(); } /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -863,7 +707,7 @@ public void delete(String resourceGroupName, String environmentName, String comp /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -871,22 +715,18 @@ public void delete(String resourceGroupName, String environmentName, String comp * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String environmentName, String componentName) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String environmentName, String componentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -901,24 +741,14 @@ private Mono> listSecretsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, componentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -927,22 +757,18 @@ private Mono> listSecretsWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String environmentName, String componentName, Context context) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String environmentName, String componentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -957,21 +783,13 @@ private Mono> listSecretsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - componentName, - this.client.getApiVersion(), - accept, - context); + return service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, componentName, this.client.getApiVersion(), accept, context); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -981,15 +799,15 @@ private Mono> listSecretsWithResponseAsync( * @return dapr component Secrets Collection for ListSecrets Action on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listSecretsAsync( - String resourceGroupName, String environmentName, String componentName) { + private Mono listSecretsAsync(String resourceGroupName, String environmentName, + String componentName) { return listSecretsWithResponseAsync(resourceGroupName, environmentName, componentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -1000,14 +818,14 @@ private Mono listSecretsAsync( * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listSecretsWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context) { + public Response listSecretsWithResponse(String resourceGroupName, + String environmentName, String componentName, Context context) { return listSecretsWithResponseAsync(resourceGroupName, environmentName, componentName, context).block(); } /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -1017,16 +835,17 @@ public Response listSecretsWithResponse( * @return dapr component Secrets Collection for ListSecrets Action. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DaprSecretsCollectionInner listSecrets( - String resourceGroupName, String environmentName, String componentName) { + public DaprSecretsCollectionInner listSecrets(String resourceGroupName, String environmentName, + String componentName) { return listSecretsWithResponse(resourceGroupName, environmentName, componentName, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1038,31 +857,22 @@ private Mono> listNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -1075,23 +885,13 @@ private Mono> listNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsImpl.java index 516bc044c8fe6..1e94a6c8e32f2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprComponentsImpl.java @@ -23,8 +23,7 @@ public final class DaprComponentsImpl implements DaprComponents { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public DaprComponentsImpl( - DaprComponentsClient innerClient, + public DaprComponentsImpl(DaprComponentsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -32,24 +31,21 @@ public DaprComponentsImpl( public PagedIterable list(String resourceGroupName, String environmentName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); - return Utils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String environmentName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, environmentName, context); - return Utils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DaprComponentImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, environmentName, componentName, context); + public Response getWithResponse(String resourceGroupName, String environmentName, + String componentName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, componentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DaprComponentImpl(inner.getValue(), this.manager())); } else { return null; @@ -65,37 +61,22 @@ public DaprComponent get(String resourceGroupName, String environmentName, Strin } } - public Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, - Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateWithResponse( - resourceGroupName, environmentName, componentName, daprComponentEnvelope, context); + public Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String componentName, DaprComponentInner daprComponentEnvelope, Context context) { + Response inner = this.serviceClient().createOrUpdateWithResponse(resourceGroupName, + environmentName, componentName, daprComponentEnvelope, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DaprComponentImpl(inner.getValue(), this.manager())); } else { return null; } } - public DaprComponent createOrUpdate( - String resourceGroupName, - String environmentName, - String componentName, + public DaprComponent createOrUpdate(String resourceGroupName, String environmentName, String componentName, DaprComponentInner daprComponentEnvelope) { - DaprComponentInner inner = - this - .serviceClient() - .createOrUpdate(resourceGroupName, environmentName, componentName, daprComponentEnvelope); + DaprComponentInner inner = this.serviceClient().createOrUpdate(resourceGroupName, environmentName, + componentName, daprComponentEnvelope); if (inner != null) { return new DaprComponentImpl(inner, this.manager()); } else { @@ -103,8 +84,8 @@ public DaprComponent createOrUpdate( } } - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, environmentName, componentName, context); } @@ -112,15 +93,12 @@ public void delete(String resourceGroupName, String environmentName, String comp this.serviceClient().delete(resourceGroupName, environmentName, componentName); } - public Response listSecretsWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context) { - Response inner = - this.serviceClient().listSecretsWithResponse(resourceGroupName, environmentName, componentName, context); + public Response listSecretsWithResponse(String resourceGroupName, String environmentName, + String componentName, Context context) { + Response inner + = this.serviceClient().listSecretsWithResponse(resourceGroupName, environmentName, componentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DaprSecretsCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -128,8 +106,8 @@ public Response listSecretsWithResponse( } public DaprSecretsCollection listSecrets(String resourceGroupName, String environmentName, String componentName) { - DaprSecretsCollectionInner inner = - this.serviceClient().listSecrets(resourceGroupName, environmentName, componentName); + DaprSecretsCollectionInner inner + = this.serviceClient().listSecrets(resourceGroupName, environmentName, componentName); if (inner != null) { return new DaprSecretsCollectionImpl(inner, this.manager()); } else { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSecretsCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSecretsCollectionImpl.java index 444784c197fe4..2aa4eff3aa205 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSecretsCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSecretsCollectionImpl.java @@ -15,8 +15,7 @@ public final class DaprSecretsCollectionImpl implements DaprSecretsCollection { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - DaprSecretsCollectionImpl( - DaprSecretsCollectionInner innerObject, + DaprSecretsCollectionImpl(DaprSecretsCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionImpl.java new file mode 100644 index 0000000000000..b503f52d86b1b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionImpl.java @@ -0,0 +1,194 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import com.azure.resourcemanager.appcontainers.models.DaprSubscription; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public final class DaprSubscriptionImpl + implements DaprSubscription, DaprSubscription.Definition, DaprSubscription.Update { + private DaprSubscriptionInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public String pubsubName() { + return this.innerModel().pubsubName(); + } + + public String topic() { + return this.innerModel().topic(); + } + + public String deadLetterTopic() { + return this.innerModel().deadLetterTopic(); + } + + public DaprSubscriptionRoutes routes() { + return this.innerModel().routes(); + } + + public List scopes() { + List inner = this.innerModel().scopes(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public Map metadata() { + Map inner = this.innerModel().metadata(); + if (inner != null) { + return Collections.unmodifiableMap(inner); + } else { + return Collections.emptyMap(); + } + } + + public DaprSubscriptionBulkSubscribeOptions bulkSubscribe() { + return this.innerModel().bulkSubscribe(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public DaprSubscriptionInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String environmentName; + + private String name; + + public DaprSubscriptionImpl withExistingManagedEnvironment(String resourceGroupName, String environmentName) { + this.resourceGroupName = resourceGroupName; + this.environmentName = environmentName; + return this; + } + + public DaprSubscription create() { + this.innerObject = serviceManager.serviceClient().getDaprSubscriptions() + .createOrUpdateWithResponse(resourceGroupName, environmentName, name, this.innerModel(), Context.NONE) + .getValue(); + return this; + } + + public DaprSubscription create(Context context) { + this.innerObject = serviceManager.serviceClient().getDaprSubscriptions() + .createOrUpdateWithResponse(resourceGroupName, environmentName, name, this.innerModel(), context) + .getValue(); + return this; + } + + DaprSubscriptionImpl(String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new DaprSubscriptionInner(); + this.serviceManager = serviceManager; + this.name = name; + } + + public DaprSubscriptionImpl update() { + return this; + } + + public DaprSubscription apply() { + this.innerObject = serviceManager.serviceClient().getDaprSubscriptions() + .createOrUpdateWithResponse(resourceGroupName, environmentName, name, this.innerModel(), Context.NONE) + .getValue(); + return this; + } + + public DaprSubscription apply(Context context) { + this.innerObject = serviceManager.serviceClient().getDaprSubscriptions() + .createOrUpdateWithResponse(resourceGroupName, environmentName, name, this.innerModel(), context) + .getValue(); + return this; + } + + DaprSubscriptionImpl(DaprSubscriptionInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.name = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "daprSubscriptions"); + } + + public DaprSubscription refresh() { + this.innerObject = serviceManager.serviceClient().getDaprSubscriptions() + .getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + return this; + } + + public DaprSubscription refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getDaprSubscriptions() + .getWithResponse(resourceGroupName, environmentName, name, context).getValue(); + return this; + } + + public DaprSubscriptionImpl withPubsubName(String pubsubName) { + this.innerModel().withPubsubName(pubsubName); + return this; + } + + public DaprSubscriptionImpl withTopic(String topic) { + this.innerModel().withTopic(topic); + return this; + } + + public DaprSubscriptionImpl withDeadLetterTopic(String deadLetterTopic) { + this.innerModel().withDeadLetterTopic(deadLetterTopic); + return this; + } + + public DaprSubscriptionImpl withRoutes(DaprSubscriptionRoutes routes) { + this.innerModel().withRoutes(routes); + return this; + } + + public DaprSubscriptionImpl withScopes(List scopes) { + this.innerModel().withScopes(scopes); + return this; + } + + public DaprSubscriptionImpl withMetadata(Map metadata) { + this.innerModel().withMetadata(metadata); + return this; + } + + public DaprSubscriptionImpl withBulkSubscribe(DaprSubscriptionBulkSubscribeOptions bulkSubscribe) { + this.innerModel().withBulkSubscribe(bulkSubscribe); + return this; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsClientImpl.java new file mode 100644 index 0000000000000..d56c8586a2b93 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsClientImpl.java @@ -0,0 +1,753 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.DaprSubscriptionsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionsCollection; +import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in DaprSubscriptionsClient. + */ +public final class DaprSubscriptionsClientImpl implements DaprSubscriptionsClient { + /** + * The proxy service used to perform REST calls. + */ + private final DaprSubscriptionsService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of DaprSubscriptionsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + DaprSubscriptionsClientImpl(ContainerAppsApiClientImpl client) { + this.service + = RestProxy.create(DaprSubscriptionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientDaprSubscriptions to be used by the proxy + * service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface DaprSubscriptionsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprSubscriptions") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprSubscriptions/{name}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprSubscriptions/{name}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") DaprSubscriptionInner daprSubscriptionEnvelope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/daprSubscriptions/{name}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName)); + } + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName, context)); + } + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String name, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), accept, context); + } + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String environmentName, String name) { + return getWithResponseAsync(resourceGroupName, environmentName, name) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String environmentName, + String name, Context context) { + return getWithResponseAsync(resourceGroupName, environmentName, name, context).block(); + } + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DaprSubscriptionInner get(String resourceGroupName, String environmentName, String name) { + return getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + } + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String name, DaprSubscriptionInner daprSubscriptionEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (daprSubscriptionEnvelope == null) { + return Mono.error( + new IllegalArgumentException("Parameter daprSubscriptionEnvelope is required and cannot be null.")); + } else { + daprSubscriptionEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, name, this.client.getApiVersion(), daprSubscriptionEnvelope, accept, + context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String name, DaprSubscriptionInner daprSubscriptionEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (daprSubscriptionEnvelope == null) { + return Mono.error( + new IllegalArgumentException("Parameter daprSubscriptionEnvelope is required and cannot be null.")); + } else { + daprSubscriptionEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), daprSubscriptionEnvelope, accept, context); + } + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String name, DaprSubscriptionInner daprSubscriptionEnvelope) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, name, daprSubscriptionEnvelope) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String name, DaprSubscriptionInner daprSubscriptionEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, name, daprSubscriptionEnvelope, + context).block(); + } + + /** + * Creates or updates a Dapr subscription. + * + * Creates or updates a Dapr subscription in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param daprSubscriptionEnvelope Configuration details of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr PubSub Event Subscription. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DaprSubscriptionInner createOrUpdate(String resourceGroupName, String environmentName, String name, + DaprSubscriptionInner daprSubscriptionEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, environmentName, name, daprSubscriptionEnvelope, + Context.NONE).getValue(); + } + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, String name, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), accept, context); + } + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String environmentName, String name) { + return deleteWithResponseAsync(resourceGroupName, environmentName, name).flatMap(ignored -> Mono.empty()); + } + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + return deleteWithResponseAsync(resourceGroupName, environmentName, name, context).block(); + } + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String environmentName, String name) { + deleteWithResponse(resourceGroupName, environmentName, name, Context.NONE); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Subscriptions ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return dapr Subscriptions ARM resource along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsImpl.java new file mode 100644 index 0000000000000..168ba34fb53da --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DaprSubscriptionsImpl.java @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.DaprSubscriptionsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import com.azure.resourcemanager.appcontainers.models.DaprSubscription; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptions; + +public final class DaprSubscriptionsImpl implements DaprSubscriptions { + private static final ClientLogger LOGGER = new ClientLogger(DaprSubscriptionsImpl.class); + + private final DaprSubscriptionsClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public DaprSubscriptionsImpl(DaprSubscriptionsClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String environmentName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DaprSubscriptionImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DaprSubscriptionImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, name, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DaprSubscriptionImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public DaprSubscription get(String resourceGroupName, String environmentName, String name) { + DaprSubscriptionInner inner = this.serviceClient().get(resourceGroupName, environmentName, name); + if (inner != null) { + return new DaprSubscriptionImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response deleteWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, environmentName, name, context); + } + + public void delete(String resourceGroupName, String environmentName, String name) { + this.serviceClient().delete(resourceGroupName, environmentName, name); + } + + public DaprSubscription getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "daprSubscriptions"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprSubscriptions'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "daprSubscriptions"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprSubscriptions'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, name, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "daprSubscriptions"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprSubscriptions'.", id))); + } + this.deleteWithResponse(resourceGroupName, environmentName, name, Context.NONE); + } + + public Response deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "daprSubscriptions"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'daprSubscriptions'.", id))); + } + return this.deleteWithResponse(resourceGroupName, environmentName, name, context); + } + + private DaprSubscriptionsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public DaprSubscriptionImpl define(String name) { + return new DaprSubscriptionImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsCollectionImpl.java index 89edd269aaada..b9602fa4b9662 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsCollectionImpl.java @@ -17,8 +17,7 @@ public final class DiagnosticsCollectionImpl implements DiagnosticsCollection { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - DiagnosticsCollectionImpl( - DiagnosticsCollectionInner innerObject, + DiagnosticsCollectionImpl(DiagnosticsCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,8 @@ public final class DiagnosticsCollectionImpl implements DiagnosticsCollection { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new DiagnosticsImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new DiagnosticsImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsImpl.java index 40453e7de9471..30dee7a892ecb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DiagnosticsImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; import com.azure.resourcemanager.appcontainers.models.Diagnostics; import com.azure.resourcemanager.appcontainers.models.DiagnosticsProperties; @@ -13,8 +14,8 @@ public final class DiagnosticsImpl implements Diagnostics { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - DiagnosticsImpl( - DiagnosticsInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + DiagnosticsImpl(DiagnosticsInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -35,6 +36,10 @@ public DiagnosticsProperties properties() { return this.innerModel().properties(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public DiagnosticsInner innerModel() { return this.innerObject; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentImpl.java new file mode 100644 index 0000000000000..c575a5b205e3e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentImpl.java @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import com.azure.resourcemanager.appcontainers.models.DotNetComponent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentProvisioningState; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.util.Collections; +import java.util.List; + +public final class DotNetComponentImpl implements DotNetComponent, DotNetComponent.Definition, DotNetComponent.Update { + private DotNetComponentInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public DotNetComponentType componentType() { + return this.innerModel().componentType(); + } + + public DotNetComponentProvisioningState provisioningState() { + return this.innerModel().provisioningState(); + } + + public List configurations() { + List inner = this.innerModel().configurations(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public List serviceBinds() { + List inner = this.innerModel().serviceBinds(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public DotNetComponentInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String environmentName; + + private String name; + + public DotNetComponentImpl withExistingManagedEnvironment(String resourceGroupName, String environmentName) { + this.resourceGroupName = resourceGroupName; + this.environmentName = environmentName; + return this; + } + + public DotNetComponent create() { + this.innerObject = serviceManager.serviceClient().getDotNetComponents().createOrUpdate(resourceGroupName, + environmentName, name, this.innerModel(), Context.NONE); + return this; + } + + public DotNetComponent create(Context context) { + this.innerObject = serviceManager.serviceClient().getDotNetComponents().createOrUpdate(resourceGroupName, + environmentName, name, this.innerModel(), context); + return this; + } + + DotNetComponentImpl(String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new DotNetComponentInner(); + this.serviceManager = serviceManager; + this.name = name; + } + + public DotNetComponentImpl update() { + return this; + } + + public DotNetComponent apply() { + this.innerObject = serviceManager.serviceClient().getDotNetComponents().update(resourceGroupName, + environmentName, name, this.innerModel(), Context.NONE); + return this; + } + + public DotNetComponent apply(Context context) { + this.innerObject = serviceManager.serviceClient().getDotNetComponents().update(resourceGroupName, + environmentName, name, this.innerModel(), context); + return this; + } + + DotNetComponentImpl(DotNetComponentInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.name = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "dotNetComponents"); + } + + public DotNetComponent refresh() { + this.innerObject = serviceManager.serviceClient().getDotNetComponents() + .getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + return this; + } + + public DotNetComponent refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getDotNetComponents() + .getWithResponse(resourceGroupName, environmentName, name, context).getValue(); + return this; + } + + public DotNetComponentImpl withComponentType(DotNetComponentType componentType) { + this.innerModel().withComponentType(componentType); + return this; + } + + public DotNetComponentImpl withConfigurations(List configurations) { + this.innerModel().withConfigurations(configurations); + return this; + } + + public DotNetComponentImpl withServiceBinds(List serviceBinds) { + this.innerModel().withServiceBinds(serviceBinds); + return this; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsClientImpl.java new file mode 100644 index 0000000000000..33b02f2c79cfc --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsClientImpl.java @@ -0,0 +1,1238 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.appcontainers.fluent.DotNetComponentsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentsCollection; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in DotNetComponentsClient. + */ +public final class DotNetComponentsClientImpl implements DotNetComponentsClient { + /** + * The proxy service used to perform REST calls. + */ + private final DotNetComponentsService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of DotNetComponentsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + DotNetComponentsClientImpl(ContainerAppsApiClientImpl client) { + this.service + = RestProxy.create(DotNetComponentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientDotNetComponents to be used by the proxy + * service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface DotNetComponentsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/dotNetComponents") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/dotNetComponents/{name}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/dotNetComponents/{name}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") DotNetComponentInner dotNetComponentEnvelope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/dotNetComponents/{name}") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") DotNetComponentInner dotNetComponentEnvelope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/dotNetComponents/{name}") + @ExpectedResponses({ 202, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get the .NET Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the .NET Components for a managed environment along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the .NET Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the .NET Components for a managed environment along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the .NET Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the .NET Components for a managed environment as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Get the .NET Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the .NET Components for a managed environment as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Get the .NET Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the .NET Components for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName)); + } + + /** + * Get the .NET Components for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the .NET Components for a managed environment as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName, context)); + } + + /** + * Get a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a .NET Component along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a .NET Component along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String name, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), accept, context); + } + + /** + * Get a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a .NET Component on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String environmentName, String name) { + return getWithResponseAsync(resourceGroupName, environmentName, name) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a .NET Component along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + return getWithResponseAsync(resourceGroupName, environmentName, name, context).block(); + } + + /** + * Get a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a .NET Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DotNetComponentInner get(String resourceGroupName, String environmentName, String name) { + return getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (dotNetComponentEnvelope == null) { + return Mono.error( + new IllegalArgumentException("Parameter dotNetComponentEnvelope is required and cannot be null.")); + } else { + dotNetComponentEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, name, this.client.getApiVersion(), dotNetComponentEnvelope, accept, + context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (dotNetComponentEnvelope == null) { + return Mono.error( + new IllegalArgumentException("Parameter dotNetComponentEnvelope is required and cannot be null.")); + } else { + dotNetComponentEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), dotNetComponentEnvelope, accept, context); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DotNetComponentInner> beginCreateOrUpdateAsync( + String resourceGroupName, String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DotNetComponentInner.class, DotNetComponentInner.class, this.client.getContext()); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DotNetComponentInner> beginCreateOrUpdateAsync( + String resourceGroupName, String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, + name, dotNetComponentEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DotNetComponentInner.class, DotNetComponentInner.class, context); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DotNetComponentInner> beginCreateOrUpdate( + String resourceGroupName, String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope) { + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope) + .getSyncPoller(); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DotNetComponentInner> beginCreateOrUpdate( + String resourceGroupName, String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, + Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context) + .getSyncPoller(); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String name, DotNetComponentInner dotNetComponentEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String name, DotNetComponentInner dotNetComponentEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context) + .last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DotNetComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope) { + return createOrUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope).block(); + } + + /** + * Creates or updates a .NET Component. + * + * Creates or updates a .NET Component in a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DotNetComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope, Context context) { + return createOrUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context).block(); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String environmentName, + String name, DotNetComponentInner dotNetComponentEnvelope) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (dotNetComponentEnvelope == null) { + return Mono.error( + new IllegalArgumentException("Parameter dotNetComponentEnvelope is required and cannot be null.")); + } else { + dotNetComponentEnvelope.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), dotNetComponentEnvelope, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String environmentName, + String name, DotNetComponentInner dotNetComponentEnvelope, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (dotNetComponentEnvelope == null) { + return Mono.error( + new IllegalArgumentException("Parameter dotNetComponentEnvelope is required and cannot be null.")); + } else { + dotNetComponentEnvelope.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), dotNetComponentEnvelope, accept, context); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DotNetComponentInner> beginUpdateAsync( + String resourceGroupName, String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DotNetComponentInner.class, DotNetComponentInner.class, this.client.getContext()); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DotNetComponentInner> beginUpdateAsync( + String resourceGroupName, String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DotNetComponentInner.class, DotNetComponentInner.class, context); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DotNetComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope) { + return this.beginUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope).getSyncPoller(); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of .NET Component. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DotNetComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, DotNetComponentInner dotNetComponentEnvelope, Context context) { + return this.beginUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context) + .getSyncPoller(); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope) { + return beginUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope, Context context) { + return beginUpdateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DotNetComponentInner update(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope) { + return updateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope).block(); + } + + /** + * Update properties of a .NET Component + * + * Patches a .NET Component using JSON Merge Patch. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param dotNetComponentEnvelope Configuration details of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Component. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DotNetComponentInner update(String resourceGroupName, String environmentName, String name, + DotNetComponentInner dotNetComponentEnvelope, Context context) { + return updateAsync(resourceGroupName, environmentName, name, dotNetComponentEnvelope, context).block(); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String name) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, name, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String name, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), accept, context); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String environmentName, + String name) { + Mono>> mono = deleteWithResponseAsync(resourceGroupName, environmentName, name); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String environmentName, + String name, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, environmentName, name, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, + String name) { + return this.beginDeleteAsync(resourceGroupName, environmentName, name).getSyncPoller(); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, String name, + Context context) { + return this.beginDeleteAsync(resourceGroupName, environmentName, name, context).getSyncPoller(); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String environmentName, String name) { + return beginDeleteAsync(resourceGroupName, environmentName, name).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String environmentName, String name, Context context) { + return beginDeleteAsync(resourceGroupName, environmentName, name, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String environmentName, String name) { + deleteAsync(resourceGroupName, environmentName, name).block(); + } + + /** + * Delete a .NET Component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String environmentName, String name, Context context) { + deleteAsync(resourceGroupName, environmentName, name, context).block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Components ARM resource along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return .NET Components ARM resource along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsImpl.java new file mode 100644 index 0000000000000..82351e9b232fb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/DotNetComponentsImpl.java @@ -0,0 +1,157 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.DotNetComponentsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import com.azure.resourcemanager.appcontainers.models.DotNetComponent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponents; + +public final class DotNetComponentsImpl implements DotNetComponents { + private static final ClientLogger LOGGER = new ClientLogger(DotNetComponentsImpl.class); + + private final DotNetComponentsClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public DotNetComponentsImpl(DotNetComponentsClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String environmentName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DotNetComponentImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DotNetComponentImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, name, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DotNetComponentImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public DotNetComponent get(String resourceGroupName, String environmentName, String name) { + DotNetComponentInner inner = this.serviceClient().get(resourceGroupName, environmentName, name); + if (inner != null) { + return new DotNetComponentImpl(inner, this.manager()); + } else { + return null; + } + } + + public void delete(String resourceGroupName, String environmentName, String name) { + this.serviceClient().delete(resourceGroupName, environmentName, name); + } + + public void delete(String resourceGroupName, String environmentName, String name, Context context) { + this.serviceClient().delete(resourceGroupName, environmentName, name, context); + } + + public DotNetComponent getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "dotNetComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dotNetComponents'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "dotNetComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dotNetComponents'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, name, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "dotNetComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dotNetComponents'.", id))); + } + this.delete(resourceGroupName, environmentName, name, Context.NONE); + } + + public void deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "dotNetComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dotNetComponents'.", id))); + } + this.delete(resourceGroupName, environmentName, name, context); + } + + private DotNetComponentsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public DotNetComponentImpl define(String name) { + return new DotNetComponentImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/EnvironmentAuthTokenImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/EnvironmentAuthTokenImpl.java index c2a56e6d8305b..3e719b07d22a6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/EnvironmentAuthTokenImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/EnvironmentAuthTokenImpl.java @@ -16,8 +16,7 @@ public final class EnvironmentAuthTokenImpl implements EnvironmentAuthToken { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - EnvironmentAuthTokenImpl( - EnvironmentAuthTokenInner innerObject, + EnvironmentAuthTokenImpl(EnvironmentAuthTokenInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentImpl.java new file mode 100644 index 0000000000000..66398fe94215e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentImpl.java @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import com.azure.resourcemanager.appcontainers.models.JavaComponent; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentProvisioningState; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.util.Collections; +import java.util.List; + +public final class JavaComponentImpl implements JavaComponent, JavaComponent.Definition, JavaComponent.Update { + private JavaComponentInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public JavaComponentType componentType() { + return this.innerModel().componentType(); + } + + public JavaComponentProvisioningState provisioningState() { + return this.innerModel().provisioningState(); + } + + public List configurations() { + List inner = this.innerModel().configurations(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public List serviceBinds() { + List inner = this.innerModel().serviceBinds(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public JavaComponentInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String environmentName; + + private String name; + + public JavaComponentImpl withExistingManagedEnvironment(String resourceGroupName, String environmentName) { + this.resourceGroupName = resourceGroupName; + this.environmentName = environmentName; + return this; + } + + public JavaComponent create() { + this.innerObject = serviceManager.serviceClient().getJavaComponents().createOrUpdate(resourceGroupName, + environmentName, name, this.innerModel(), Context.NONE); + return this; + } + + public JavaComponent create(Context context) { + this.innerObject = serviceManager.serviceClient().getJavaComponents().createOrUpdate(resourceGroupName, + environmentName, name, this.innerModel(), context); + return this; + } + + JavaComponentImpl(String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = new JavaComponentInner(); + this.serviceManager = serviceManager; + this.name = name; + } + + public JavaComponentImpl update() { + return this; + } + + public JavaComponent apply() { + this.innerObject = serviceManager.serviceClient().getJavaComponents().update(resourceGroupName, environmentName, + name, this.innerModel(), Context.NONE); + return this; + } + + public JavaComponent apply(Context context) { + this.innerObject = serviceManager.serviceClient().getJavaComponents().update(resourceGroupName, environmentName, + name, this.innerModel(), context); + return this; + } + + JavaComponentImpl(JavaComponentInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.name = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "javaComponents"); + } + + public JavaComponent refresh() { + this.innerObject = serviceManager.serviceClient().getJavaComponents() + .getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + return this; + } + + public JavaComponent refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getJavaComponents() + .getWithResponse(resourceGroupName, environmentName, name, context).getValue(); + return this; + } + + public JavaComponentImpl withComponentType(JavaComponentType componentType) { + this.innerModel().withComponentType(componentType); + return this; + } + + public JavaComponentImpl withConfigurations(List configurations) { + this.innerModel().withConfigurations(configurations); + return this; + } + + public JavaComponentImpl withServiceBinds(List serviceBinds) { + this.innerModel().withServiceBinds(serviceBinds); + return this; + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsClientImpl.java similarity index 56% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsClientImpl.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsClientImpl.java index 522682119808d..60ad7023f66a3 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsClientImpl.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.implementation; +package com.azure.resourcemanager.appcontainers.implementation; import com.azure.core.annotation.BodyParam; import com.azure.core.annotation.Delete; @@ -32,121 +32,122 @@ import com.azure.core.util.FluxUtil; import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.BackupVaultsClient; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import com.azure.resourcemanager.netapp.models.BackupVaultPatch; -import com.azure.resourcemanager.netapp.models.BackupVaultsList; +import com.azure.resourcemanager.appcontainers.fluent.JavaComponentsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import com.azure.resourcemanager.appcontainers.models.JavaComponentsCollection; import java.nio.ByteBuffer; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; /** - * An instance of this class provides access to all the operations defined in BackupVaultsClient. + * An instance of this class provides access to all the operations defined in JavaComponentsClient. */ -public final class BackupVaultsClientImpl implements BackupVaultsClient { +public final class JavaComponentsClientImpl implements JavaComponentsClient { /** * The proxy service used to perform REST calls. */ - private final BackupVaultsService service; + private final JavaComponentsService service; /** * The service client containing this operation class. */ - private final NetAppManagementClientImpl client; + private final ContainerAppsApiClientImpl client; /** - * Initializes an instance of BackupVaultsClientImpl. + * Initializes an instance of JavaComponentsClientImpl. * * @param client the instance of the service client containing this operation class. */ - BackupVaultsClientImpl(NetAppManagementClientImpl client) { + JavaComponentsClientImpl(ContainerAppsApiClientImpl client) { this.service - = RestProxy.create(BackupVaultsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + = RestProxy.create(JavaComponentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for NetAppManagementClientBackupVaults to be used by the proxy service + * The interface defining all the services for ContainerAppsApiClientJavaComponents to be used by the proxy service * to perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "NetAppManagementClie") - public interface BackupVaultsService { + @ServiceInterface(name = "ContainerAppsApiClie") + public interface JavaComponentsService { @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults") + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/javaComponents") @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByNetAppAccount(@HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}") + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/javaComponents/{name}") @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get(@HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, Context context); + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) - @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}") + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/javaComponents/{name}") @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") BackupVaultInner body, @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") JavaComponentInner javaComponentEnvelope, + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) - @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}") + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/javaComponents/{name}") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") BackupVaultPatch body, @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") JavaComponentInner javaComponentEnvelope, + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) - @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}") + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/javaComponents/{name}") @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, Context context); + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @PathParam("name") String name, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByNetAppAccountNext( + Mono> listNext( @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults along with {@link PagedResponse} on successful completion of {@link Mono}. + * @return the Java Components for a managed environment along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNetAppAccountSinglePageAsync(String resourceGroupName, - String accountName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -159,35 +160,34 @@ private Mono> listByNetAppAccountSinglePageAsync return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> service.listByNetAppAccount(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, this.client.getApiVersion(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), - res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults along with {@link PagedResponse} on successful completion of {@link Mono}. + * @return the Java Components for a managed environment along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNetAppAccountSinglePageAsync(String resourceGroupName, - String accountName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -200,108 +200,97 @@ private Mono> listByNetAppAccountSinglePageAsync return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByNetAppAccount(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, this.client.getApiVersion(), accept, context) + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)); } /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedFlux}. + * @return the Java Components for a managed environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByNetAppAccountAsync(String resourceGroupName, String accountName) { - return new PagedFlux<>(() -> listByNetAppAccountSinglePageAsync(resourceGroupName, accountName), - nextLink -> listByNetAppAccountNextSinglePageAsync(nextLink)); + private PagedFlux listAsync(String resourceGroupName, String environmentName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), + nextLink -> listNextSinglePageAsync(nextLink)); } /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedFlux}. + * @return the Java Components for a managed environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByNetAppAccountAsync(String resourceGroupName, String accountName, - Context context) { - return new PagedFlux<>(() -> listByNetAppAccountSinglePageAsync(resourceGroupName, accountName, context), - nextLink -> listByNetAppAccountNextSinglePageAsync(nextLink, context)); + private PagedFlux listAsync(String resourceGroupName, String environmentName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedIterable}. + * @return the Java Components for a managed environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName) { - return new PagedIterable<>(listByNetAppAccountAsync(resourceGroupName, accountName)); + public PagedIterable list(String resourceGroupName, String environmentName) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName)); } /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedIterable}. + * @return the Java Components for a managed environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, - Context context) { - return new PagedIterable<>(listByNetAppAccountAsync(resourceGroupName, accountName, context)); + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName, context)); } /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response} on successful completion of {@link Mono}. + * @return a Java Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName) { + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String name) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -314,37 +303,35 @@ private Mono> getWithResponseAsync(String resourceGro return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupVaultName, this.client.getApiVersion(), accept, context)) + resourceGroupName, environmentName, name, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response} on successful completion of {@link Mono}. + * @return a Java Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String environmentName, + String name, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -357,93 +344,87 @@ private Mono> getWithResponseAsync(String resourceGro return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, accountName, - backupVaultName, this.client.getApiVersion(), accept, context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), accept, context); } /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault on successful completion of {@link Mono}. + * @return a Java Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String resourceGroupName, String accountName, String backupVaultName) { - return getWithResponseAsync(resourceGroupName, accountName, backupVaultName) + private Mono getAsync(String resourceGroupName, String environmentName, String name) { + return getWithResponseAsync(resourceGroupName, environmentName, name) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response}. + * @return a Java Component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String resourceGroupName, String accountName, - String backupVaultName, Context context) { - return getWithResponseAsync(resourceGroupName, accountName, backupVaultName, context).block(); + public Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + return getWithResponseAsync(resourceGroupName, environmentName, name, context).block(); } /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault. + * @return a Java Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public BackupVaultInner get(String resourceGroupName, String accountName, String backupVaultName) { - return getWithResponse(resourceGroupName, accountName, backupVaultName, Context.NONE).getValue(); + public JavaComponentInner get(String resourceGroupName, String environmentName, String name) { + return getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information along with {@link Response} on successful completion of {@link Mono}. + * @return java Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultInner body) { + String environmentName, String name, JavaComponentInner javaComponentEnvelope) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -456,43 +437,45 @@ private Mono>> createOrUpdateWithResponseAsync(String return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + if (javaComponentEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter javaComponentEnvelope is required and cannot be null.")); } else { - body.validate(); + javaComponentEnvelope.validate(); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupVaultName, this.client.getApiVersion(), body, accept, context)) + resourceGroupName, environmentName, name, this.client.getApiVersion(), javaComponentEnvelope, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information along with {@link Response} on successful completion of {@link Mono}. + * @return java Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultInner body, Context context) { + String environmentName, String name, JavaComponentInner javaComponentEnvelope, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -505,215 +488,218 @@ private Mono>> createOrUpdateWithResponseAsync(String return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); + } + if (javaComponentEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter javaComponentEnvelope is required and cannot be null.")); } else { - body.validate(); + javaComponentEnvelope.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, this.client.getApiVersion(), body, accept, context); + environmentName, name, this.client.getApiVersion(), javaComponentEnvelope, accept, context); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup Vault information. + * @return the {@link PollerFlux} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupVaultInner> beginCreateOrUpdateAsync( - String resourceGroupName, String accountName, String backupVaultName, BackupVaultInner body) { + private PollerFlux, JavaComponentInner> beginCreateOrUpdateAsync( + String resourceGroupName, String environmentName, String name, JavaComponentInner javaComponentEnvelope) { Mono>> mono - = createOrUpdateWithResponseAsync(resourceGroupName, accountName, backupVaultName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupVaultInner.class, BackupVaultInner.class, this.client.getContext()); + = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, name, javaComponentEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + JavaComponentInner.class, JavaComponentInner.class, this.client.getContext()); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup Vault information. + * @return the {@link PollerFlux} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupVaultInner> beginCreateOrUpdateAsync( - String resourceGroupName, String accountName, String backupVaultName, BackupVaultInner body, Context context) { + private PollerFlux, JavaComponentInner> beginCreateOrUpdateAsync( + String resourceGroupName, String environmentName, String name, JavaComponentInner javaComponentEnvelope, + Context context) { context = this.client.mergeContext(context); Mono>> mono - = createOrUpdateWithResponseAsync(resourceGroupName, accountName, backupVaultName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupVaultInner.class, BackupVaultInner.class, context); + = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + JavaComponentInner.class, JavaComponentInner.class, context); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupVaultInner> beginCreateOrUpdate(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultInner body) { - return this.beginCreateOrUpdateAsync(resourceGroupName, accountName, backupVaultName, body).getSyncPoller(); + public SyncPoller, JavaComponentInner> beginCreateOrUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope) { + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope) + .getSyncPoller(); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupVaultInner> beginCreateOrUpdate(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultInner body, Context context) { - return this.beginCreateOrUpdateAsync(resourceGroupName, accountName, backupVaultName, body, context) + public SyncPoller, JavaComponentInner> beginCreateOrUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context) .getSyncPoller(); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information on successful completion of {@link Mono}. + * @return java Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync(String resourceGroupName, String accountName, - String backupVaultName, BackupVaultInner body) { - return beginCreateOrUpdateAsync(resourceGroupName, accountName, backupVaultName, body).last() + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information on successful completion of {@link Mono}. + * @return java Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync(String resourceGroupName, String accountName, - String backupVaultName, BackupVaultInner body, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, accountName, backupVaultName, body, context).last() + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return java Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public BackupVaultInner createOrUpdate(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultInner body) { - return createOrUpdateAsync(resourceGroupName, accountName, backupVaultName, body).block(); + public JavaComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope) { + return createOrUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope).block(); } /** - * Create or Update a Backup Vault + * Creates or updates a Java Component. * - * Create or update the specified Backup Vault in the NetApp account. + * Creates or updates a Java Component in a Managed Environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body BackupVault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return java Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public BackupVaultInner createOrUpdate(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultInner body, Context context) { - return createOrUpdateAsync(resourceGroupName, accountName, backupVaultName, body, context).block(); + public JavaComponentInner createOrUpdate(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope, Context context) { + return createOrUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context).block(); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information along with {@link Response} on successful completion of {@link Mono}. + * @return java Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, BackupVaultPatch body) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String environmentName, + String name, JavaComponentInner javaComponentEnvelope) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -726,43 +712,45 @@ private Mono>> updateWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + if (javaComponentEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter javaComponentEnvelope is required and cannot be null.")); } else { - body.validate(); + javaComponentEnvelope.validate(); } final String accept = "application/json"; return FluxUtil - .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupVaultName, this.client.getApiVersion(), body, accept, context)) + .withContext( + context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, name, this.client.getApiVersion(), javaComponentEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information along with {@link Response} on successful completion of {@link Mono}. + * @return java Component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, BackupVaultPatch body, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String environmentName, + String name, JavaComponentInner javaComponentEnvelope, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -775,213 +763,213 @@ private Mono>> updateWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + if (javaComponentEnvelope == null) { + return Mono + .error(new IllegalArgumentException("Parameter javaComponentEnvelope is required and cannot be null.")); } else { - body.validate(); + javaComponentEnvelope.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, this.client.getApiVersion(), body, accept, context); + environmentName, name, this.client.getApiVersion(), javaComponentEnvelope, accept, context); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup Vault information. + * @return the {@link PollerFlux} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupVaultInner> beginUpdateAsync(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultPatch body) { + private PollerFlux, JavaComponentInner> beginUpdateAsync(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope) { Mono>> mono - = updateWithResponseAsync(resourceGroupName, accountName, backupVaultName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupVaultInner.class, BackupVaultInner.class, this.client.getContext()); + = updateWithResponseAsync(resourceGroupName, environmentName, name, javaComponentEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + JavaComponentInner.class, JavaComponentInner.class, this.client.getContext()); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup Vault information. + * @return the {@link PollerFlux} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupVaultInner> beginUpdateAsync(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultPatch body, Context context) { + private PollerFlux, JavaComponentInner> beginUpdateAsync(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope, Context context) { context = this.client.mergeContext(context); Mono>> mono - = updateWithResponseAsync(resourceGroupName, accountName, backupVaultName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupVaultInner.class, BackupVaultInner.class, context); + = updateWithResponseAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + JavaComponentInner.class, JavaComponentInner.class, context); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupVaultInner> beginUpdate(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultPatch body) { - return this.beginUpdateAsync(resourceGroupName, accountName, backupVaultName, body).getSyncPoller(); + public SyncPoller, JavaComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope) { + return this.beginUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope).getSyncPoller(); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup Vault information. + * @return the {@link SyncPoller} for polling of java Component. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupVaultInner> beginUpdate(String resourceGroupName, - String accountName, String backupVaultName, BackupVaultPatch body, Context context) { - return this.beginUpdateAsync(resourceGroupName, accountName, backupVaultName, body, context).getSyncPoller(); + public SyncPoller, JavaComponentInner> beginUpdate(String resourceGroupName, + String environmentName, String name, JavaComponentInner javaComponentEnvelope, Context context) { + return this.beginUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context) + .getSyncPoller(); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information on successful completion of {@link Mono}. + * @return java Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultPatch body) { - return beginUpdateAsync(resourceGroupName, accountName, backupVaultName, body).last() + private Mono updateAsync(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope) { + return beginUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information on successful completion of {@link Mono}. + * @return java Component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultPatch body, Context context) { - return beginUpdateAsync(resourceGroupName, accountName, backupVaultName, body, context).last() + private Mono updateAsync(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope, Context context) { + return beginUpdateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return java Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public BackupVaultInner update(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultPatch body) { - return updateAsync(resourceGroupName, accountName, backupVaultName, body).block(); + public JavaComponentInner update(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope) { + return updateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope).block(); } /** - * Update NetApp Backup Vault + * Update properties of a Java Component * - * Patch the specified NetApp Backup Vault. + * Patches a Java Component using JSON Merge Patch. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param body Backup Vault object supplied in the body of the operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param javaComponentEnvelope Configuration details of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup Vault information. + * @return java Component. */ @ServiceMethod(returns = ReturnType.SINGLE) - public BackupVaultInner update(String resourceGroupName, String accountName, String backupVaultName, - BackupVaultPatch body, Context context) { - return updateAsync(resourceGroupName, accountName, backupVaultName, body, context).block(); + public JavaComponentInner update(String resourceGroupName, String environmentName, String name, + JavaComponentInner javaComponentEnvelope, Context context) { + return updateAsync(resourceGroupName, environmentName, name, javaComponentEnvelope, context).block(); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String name) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -994,28 +982,26 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupVaultName, this.client.getApiVersion(), accept, context)) + resourceGroupName, environmentName, name, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1023,8 +1009,8 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String name, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); @@ -1037,49 +1023,44 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { + if (environmentName == null) { return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + if (name == null) { + return Mono.error(new IllegalArgumentException("Parameter name is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, this.client.getApiVersion(), accept, context); + environmentName, name, this.client.getApiVersion(), accept, context); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String accountName, - String backupVaultName) { - Mono>> mono - = deleteWithResponseAsync(resourceGroupName, accountName, backupVaultName); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String environmentName, + String name) { + Mono>> mono = deleteWithResponseAsync(resourceGroupName, environmentName, name); return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1087,42 +1068,38 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String accountName, - String backupVaultName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String environmentName, + String name, Context context) { context = this.client.mergeContext(context); Mono>> mono - = deleteWithResponseAsync(resourceGroupName, accountName, backupVaultName, context); + = deleteWithResponseAsync(resourceGroupName, environmentName, name, context); return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupVaultName) { - return this.beginDeleteAsync(resourceGroupName, accountName, backupVaultName).getSyncPoller(); + public SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, + String name) { + return this.beginDeleteAsync(resourceGroupName, environmentName, name).getSyncPoller(); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1130,38 +1107,34 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupVaultName, Context context) { - return this.beginDeleteAsync(resourceGroupName, accountName, backupVaultName, context).getSyncPoller(); + public SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, String name, + Context context) { + return this.beginDeleteAsync(resourceGroupName, environmentName, name, context).getSyncPoller(); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String accountName, String backupVaultName) { - return beginDeleteAsync(resourceGroupName, accountName, backupVaultName).last() + private Mono deleteAsync(String resourceGroupName, String environmentName, String name) { + return beginDeleteAsync(resourceGroupName, environmentName, name).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1169,45 +1142,40 @@ private Mono deleteAsync(String resourceGroupName, String accountName, Str * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String accountName, String backupVaultName, - Context context) { - return beginDeleteAsync(resourceGroupName, accountName, backupVaultName, context).last() + private Mono deleteAsync(String resourceGroupName, String environmentName, String name, Context context) { + return beginDeleteAsync(resourceGroupName, environmentName, name, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String accountName, String backupVaultName) { - deleteAsync(resourceGroupName, accountName, backupVaultName).block(); + public void delete(String resourceGroupName, String environmentName, String name) { + deleteAsync(resourceGroupName, environmentName, name).block(); } /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String accountName, String backupVaultName, Context context) { - deleteAsync(resourceGroupName, accountName, backupVaultName, context).block(); + public void delete(String resourceGroupName, String environmentName, String name, Context context) { + deleteAsync(resourceGroupName, environmentName, name, context).block(); } /** @@ -1219,10 +1187,10 @@ public void delete(String resourceGroupName, String accountName, String backupVa * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults along with {@link PagedResponse} on successful completion of {@link Mono}. + * @return java Components ARM resource along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNetAppAccountNextSinglePageAsync(String nextLink) { + private Mono> listNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } @@ -1231,11 +1199,9 @@ private Mono> listByNetAppAccountNextSinglePageA new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.listByNetAppAccountNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), - res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1249,11 +1215,10 @@ private Mono> listByNetAppAccountNextSinglePageA * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults along with {@link PagedResponse} on successful completion of {@link Mono}. + * @return java Components ARM resource along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNetAppAccountNextSinglePageAsync(String nextLink, - Context context) { + private Mono> listNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } @@ -1263,7 +1228,7 @@ private Mono> listByNetAppAccountNextSinglePageA } final String accept = "application/json"; context = this.client.mergeContext(context); - return service.listByNetAppAccountNext(nextLink, this.client.getEndpoint(), accept, context) + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsImpl.java new file mode 100644 index 0000000000000..1c820fa61d338 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JavaComponentsImpl.java @@ -0,0 +1,157 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.JavaComponentsClient; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import com.azure.resourcemanager.appcontainers.models.JavaComponent; +import com.azure.resourcemanager.appcontainers.models.JavaComponents; + +public final class JavaComponentsImpl implements JavaComponents { + private static final ClientLogger LOGGER = new ClientLogger(JavaComponentsImpl.class); + + private final JavaComponentsClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public JavaComponentsImpl(JavaComponentsClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String environmentName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JavaComponentImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JavaComponentImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, name, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new JavaComponentImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public JavaComponent get(String resourceGroupName, String environmentName, String name) { + JavaComponentInner inner = this.serviceClient().get(resourceGroupName, environmentName, name); + if (inner != null) { + return new JavaComponentImpl(inner, this.manager()); + } else { + return null; + } + } + + public void delete(String resourceGroupName, String environmentName, String name) { + this.serviceClient().delete(resourceGroupName, environmentName, name); + } + + public void delete(String resourceGroupName, String environmentName, String name, Context context) { + this.serviceClient().delete(resourceGroupName, environmentName, name, context); + } + + public JavaComponent getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "javaComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'javaComponents'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, name, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "javaComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'javaComponents'.", id))); + } + return this.getWithResponse(resourceGroupName, environmentName, name, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "javaComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'javaComponents'.", id))); + } + this.delete(resourceGroupName, environmentName, name, Context.NONE); + } + + public void deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); + if (environmentName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String name = ResourceManagerUtils.getValueFromIdByName(id, "javaComponents"); + if (name == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'javaComponents'.", id))); + } + this.delete(resourceGroupName, environmentName, name, context); + } + + private JavaComponentsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } + + public JavaComponentImpl define(String name) { + return new JavaComponentImpl(name, this.manager()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionBaseImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionBaseImpl.java index 78e2d345844d2..6f069589b1a45 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionBaseImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionBaseImpl.java @@ -12,8 +12,7 @@ public final class JobExecutionBaseImpl implements JobExecutionBase { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - JobExecutionBaseImpl( - JobExecutionBaseInner innerObject, + JobExecutionBaseImpl(JobExecutionBaseInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionImpl.java index 16c34c6f75562..a763cde3740e3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobExecutionImpl.java @@ -15,8 +15,8 @@ public final class JobExecutionImpl implements JobExecution { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - JobExecutionImpl( - JobExecutionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + JobExecutionImpl(JobExecutionInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobImpl.java index 0ff3f79f0cdf7..73af308a860d1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobImpl.java @@ -10,6 +10,7 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.JobInner; import com.azure.resourcemanager.appcontainers.models.ContainerAppJobExecutions; +import com.azure.resourcemanager.appcontainers.models.ExtendedLocation; import com.azure.resourcemanager.appcontainers.models.Job; import com.azure.resourcemanager.appcontainers.models.JobConfiguration; import com.azure.resourcemanager.appcontainers.models.JobExecutionBase; @@ -54,6 +55,10 @@ public Map tags() { } } + public ExtendedLocation extendedLocation() { + return this.innerModel().extendedLocation(); + } + public ManagedServiceIdentity identity() { return this.innerModel().identity(); } @@ -127,20 +132,14 @@ public JobImpl withExistingResourceGroup(String resourceGroupName) { } public Job create() { - this.innerObject = - serviceManager - .serviceClient() - .getJobs() - .createOrUpdate(resourceGroupName, jobName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getJobs().createOrUpdate(resourceGroupName, jobName, + this.innerModel(), Context.NONE); return this; } public Job create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getJobs() - .createOrUpdate(resourceGroupName, jobName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getJobs().createOrUpdate(resourceGroupName, jobName, + this.innerModel(), context); return this; } @@ -156,44 +155,33 @@ public JobImpl update() { } public Job apply() { - this.innerObject = - serviceManager - .serviceClient() - .getJobs() - .update(resourceGroupName, jobName, updateJobEnvelope, Context.NONE); + this.innerObject = serviceManager.serviceClient().getJobs().update(resourceGroupName, jobName, + updateJobEnvelope, Context.NONE); return this; } public Job apply(Context context) { - this.innerObject = - serviceManager.serviceClient().getJobs().update(resourceGroupName, jobName, updateJobEnvelope, context); + this.innerObject + = serviceManager.serviceClient().getJobs().update(resourceGroupName, jobName, updateJobEnvelope, context); return this; } JobImpl(JobInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.jobName = Utils.getValueFromIdByName(innerObject.id(), "jobs"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.jobName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "jobs"); } public Job refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getJobs() - .getByResourceGroupWithResponse(resourceGroupName, jobName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getJobs() + .getByResourceGroupWithResponse(resourceGroupName, jobName, Context.NONE).getValue(); return this; } public Job refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getJobs() - .getByResourceGroupWithResponse(resourceGroupName, jobName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getJobs() + .getByResourceGroupWithResponse(resourceGroupName, jobName, context).getValue(); return this; } @@ -241,6 +229,16 @@ public JobImpl withTags(Map tags) { } } + public JobImpl withExtendedLocation(ExtendedLocation extendedLocation) { + if (isInCreateMode()) { + this.innerModel().withExtendedLocation(extendedLocation); + return this; + } else { + this.updateJobEnvelope.withExtendedLocation(extendedLocation); + return this; + } + } + public JobImpl withIdentity(ManagedServiceIdentity identity) { if (isInCreateMode()) { this.innerModel().withIdentity(identity); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobSecretsCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobSecretsCollectionImpl.java index 552293d405e18..ebec3ac60e341 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobSecretsCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobSecretsCollectionImpl.java @@ -15,8 +15,7 @@ public final class JobSecretsCollectionImpl implements JobSecretsCollection { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - JobSecretsCollectionImpl( - JobSecretsCollectionInner innerObject, + JobSecretsCollectionImpl(JobSecretsCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsClientImpl.java index e39425f9ec659..e8a3e69f1368f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsClientImpl.java @@ -34,6 +34,8 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.appcontainers.fluent.JobsClient; import com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner; +import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner; +import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionBaseInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobSecretsCollectionInner; @@ -45,17 +47,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in JobsClient. */ +/** + * An instance of this class provides access to all the operations defined in JobsClient. + */ public final class JobsClientImpl implements JobsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final JobsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of JobsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ JobsClientImpl(ContainerAppsApiClientImpl client) { @@ -64,277 +72,623 @@ public final class JobsClientImpl implements JobsClient { } /** - * The interface defining all the services for ContainerAppsApiClientJobs to be used by the proxy service to perform - * REST calls. + * The interface defining all the services for ContainerAppsApiClientJobs to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface JobsService { - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/jobs") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/detectors") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> listDetectors(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/detectors/{detectorName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> getDetector(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @PathParam("detectorName") String detectorName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/detectorProperties/{apiName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> proxyGet(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @PathParam("apiName") String apiName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/jobs") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") - @ExpectedResponses({200}) + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") - @ExpectedResponses({200, 201}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") JobInner jobEnvelope, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") - @ExpectedResponses({200, 202, 204}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") JobInner jobEnvelope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") - @ExpectedResponses({200, 202}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> update( - @HostParam("$host") String endpoint, + Mono>> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") JobPatchProperties jobEnvelope, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/start") - @ExpectedResponses({200, 202}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") JobPatchProperties jobEnvelope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/start") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> start( - @HostParam("$host") String endpoint, + Mono>> start(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") JobExecutionTemplate template, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/executions/{jobExecutionName}/stop") - @ExpectedResponses({200, 202}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") JobExecutionTemplate template, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/executions/{jobExecutionName}/stop") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> stopExecution( - @HostParam("$host") String endpoint, + Mono>> stopExecution(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("jobName") String jobName, - @PathParam("jobExecutionName") String jobExecutionName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/stop") - @ExpectedResponses({200, 202}) + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("jobName") String jobName, @PathParam("jobExecutionName") String jobExecutionName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/stop") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> stopMultipleExecutions( - @HostParam("$host") String endpoint, + Mono>> stopMultipleExecutions(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("jobName") String jobName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/listSecrets") - @ExpectedResponses({200}) + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("jobName") String jobName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/listSecrets") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listSecrets( - @HostParam("$host") String endpoint, + Mono> listSecrets(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("jobName") String jobName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("jobName") String jobName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listDetectorsWithResponseAsync(String resourceGroupName, + String jobName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (jobName == null) { + return Mono.error(new IllegalArgumentException("Parameter jobName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listDetectors(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listDetectorsWithResponseAsync(String resourceGroupName, + String jobName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (jobName == null) { + return Mono.error(new IllegalArgumentException("Parameter jobName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listDetectors(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + jobName, this.client.getApiVersion(), accept, context); + } + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listDetectorsAsync(String resourceGroupName, String jobName) { + return listDetectorsWithResponseAsync(resourceGroupName, jobName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listDetectorsWithResponse(String resourceGroupName, String jobName, + Context context) { + return listDetectorsWithResponseAsync(resourceGroupName, jobName, context).block(); + } + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DiagnosticsCollectionInner listDetectors(String resourceGroupName, String jobName) { + return listDetectorsWithResponse(resourceGroupName, jobName, Context.NONE).getValue(); + } + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getDetectorWithResponseAsync(String resourceGroupName, String jobName, + String detectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (jobName == null) { + return Mono.error(new IllegalArgumentException("Parameter jobName is required and cannot be null.")); + } + if (detectorName == null) { + return Mono.error(new IllegalArgumentException("Parameter detectorName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getDetector(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, detectorName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getDetectorWithResponseAsync(String resourceGroupName, String jobName, + String detectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (jobName == null) { + return Mono.error(new IllegalArgumentException("Parameter jobName is required and cannot be null.")); + } + if (detectorName == null) { + return Mono.error(new IllegalArgumentException("Parameter detectorName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getDetector(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + jobName, detectorName, this.client.getApiVersion(), accept, context); + } + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getDetectorAsync(String resourceGroupName, String jobName, String detectorName) { + return getDetectorWithResponseAsync(resourceGroupName, jobName, detectorName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getDetectorWithResponse(String resourceGroupName, String jobName, + String detectorName, Context context) { + return getDetectorWithResponseAsync(resourceGroupName, jobName, detectorName, context).block(); + } + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DiagnosticsInner getDetector(String resourceGroupName, String jobName, String detectorName) { + return getDetectorWithResponse(resourceGroupName, jobName, detectorName, Context.NONE).getValue(); + } + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> proxyGetWithResponseAsync(String resourceGroupName, String jobName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (jobName == null) { + return Mono.error(new IllegalArgumentException("Parameter jobName is required and cannot be null.")); + } + final String apiName = "rootApi"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.proxyGet(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, apiName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> proxyGetWithResponseAsync(String resourceGroupName, String jobName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (jobName == null) { + return Mono.error(new IllegalArgumentException("Parameter jobName is required and cannot be null.")); + } + final String apiName = "rootApi"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.proxyGet(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, jobName, + apiName, this.client.getApiVersion(), accept, context); + } + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono proxyGetAsync(String resourceGroupName, String jobName) { + return proxyGetWithResponseAsync(resourceGroupName, jobName).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response proxyGetWithResponse(String resourceGroupName, String jobName, Context context) { + return proxyGetWithResponseAsync(resourceGroupName, jobName, context).block(); + } + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public JobInner proxyGet(String resourceGroupName, String jobName) { + return proxyGetWithResponse(resourceGroupName, jobName, Context.NONE).getValue(); } /** * Get the Container Apps Jobs in a given subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Container Apps Jobs in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Container Apps Jobs in a given subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Get the Container Apps Jobs in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -343,13 +697,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Get the Container Apps Jobs in a given subscription. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription as paginated response with {@link PagedIterable}. @@ -361,7 +715,7 @@ public PagedIterable list() { /** * Get the Container Apps Jobs in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -375,27 +729,23 @@ public PagedIterable list(Context context) { /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -403,53 +753,34 @@ private Mono> listByResourceGroupSinglePageAsync(String } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -458,27 +789,15 @@ private Mono> listByResourceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -487,14 +806,13 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -504,14 +822,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupName) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -525,7 +842,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName) { /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -540,28 +857,24 @@ public PagedIterable listByResourceGroup(String resourceGroupName, Con /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the properties of a Container Apps Job along with {@link Response} on successful completion of {@link - * Mono}. + * @return the properties of a Container Apps Job along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, String jobName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -573,45 +886,33 @@ private Mono> getByResourceGroupWithResponseAsync(String reso final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the properties of a Container Apps Job along with {@link Response} on successful completion of {@link - * Mono}. + * @return the properties of a Container Apps Job along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String jobName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, String jobName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -622,20 +923,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + jobName, this.client.getApiVersion(), accept, context); } /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -651,7 +945,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, String /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -661,14 +955,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName, String * @return the properties of a Container Apps Job along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String jobName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String jobName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, jobName, context).block(); } /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -683,7 +977,7 @@ public JobInner getByResourceGroup(String resourceGroupName, String jobName) { /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -693,19 +987,15 @@ public JobInner getByResourceGroup(String resourceGroupName, String jobName) { * @return container App Job along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String jobName, JobInner jobEnvelope) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, String jobName, + JobInner jobEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -721,24 +1011,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - jobEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), jobEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -749,19 +1029,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return container App Job along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String jobName, JobInner jobEnvelope, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, String jobName, + JobInner jobEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -777,21 +1053,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - jobEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + jobName, this.client.getApiVersion(), jobEnvelope, accept, context); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -801,19 +1069,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobInner> beginCreateOrUpdateAsync( - String resourceGroupName, String jobName, JobInner jobEnvelope) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, jobName, jobEnvelope); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), JobInner.class, JobInner.class, this.client.getContext()); + private PollerFlux, JobInner> beginCreateOrUpdateAsync(String resourceGroupName, + String jobName, JobInner jobEnvelope) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, jobName, jobEnvelope); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), JobInner.class, + JobInner.class, this.client.getContext()); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -824,20 +1090,18 @@ private PollerFlux, JobInner> beginCreateOrUpdateAsync( * @return the {@link PollerFlux} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobInner> beginCreateOrUpdateAsync( - String resourceGroupName, String jobName, JobInner jobEnvelope, Context context) { + private PollerFlux, JobInner> beginCreateOrUpdateAsync(String resourceGroupName, + String jobName, JobInner jobEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, jobName, jobEnvelope, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), JobInner.class, JobInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, jobName, jobEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), JobInner.class, + JobInner.class, context); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -847,14 +1111,14 @@ private PollerFlux, JobInner> beginCreateOrUpdateAsync( * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, JobInner> beginCreateOrUpdate( - String resourceGroupName, String jobName, JobInner jobEnvelope) { + public SyncPoller, JobInner> beginCreateOrUpdate(String resourceGroupName, String jobName, + JobInner jobEnvelope) { return this.beginCreateOrUpdateAsync(resourceGroupName, jobName, jobEnvelope).getSyncPoller(); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -865,14 +1129,14 @@ public SyncPoller, JobInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, JobInner> beginCreateOrUpdate( - String resourceGroupName, String jobName, JobInner jobEnvelope, Context context) { + public SyncPoller, JobInner> beginCreateOrUpdate(String resourceGroupName, String jobName, + JobInner jobEnvelope, Context context) { return this.beginCreateOrUpdateAsync(resourceGroupName, jobName, jobEnvelope, context).getSyncPoller(); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -883,14 +1147,13 @@ public SyncPoller, JobInner> beginCreateOrUpdate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono createOrUpdateAsync(String resourceGroupName, String jobName, JobInner jobEnvelope) { - return beginCreateOrUpdateAsync(resourceGroupName, jobName, jobEnvelope) - .last() + return beginCreateOrUpdateAsync(resourceGroupName, jobName, jobEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -901,16 +1164,15 @@ private Mono createOrUpdateAsync(String resourceGroupName, String jobN * @return container App Job on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String jobName, JobInner jobEnvelope, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, jobName, jobEnvelope, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String jobName, JobInner jobEnvelope, + Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, jobName, jobEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -926,7 +1188,7 @@ public JobInner createOrUpdate(String resourceGroupName, String jobName, JobInne /** * Create or Update a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -943,7 +1205,7 @@ public JobInner createOrUpdate(String resourceGroupName, String jobName, JobInne /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -954,16 +1216,12 @@ public JobInner createOrUpdate(String resourceGroupName, String jobName, JobInne @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String jobName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -974,23 +1232,14 @@ private Mono>> deleteWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -1000,19 +1249,15 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String jobName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String jobName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1023,20 +1268,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, jobName, + this.client.getApiVersion(), accept, context); } /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1047,15 +1285,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String jobName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, jobName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -1065,18 +1301,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String jobName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String jobName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, jobName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1091,7 +1326,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -1107,7 +1342,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1122,7 +1357,7 @@ private Mono deleteAsync(String resourceGroupName, String jobName) { /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -1133,14 +1368,13 @@ private Mono deleteAsync(String resourceGroupName, String jobName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String jobName, Context context) { - return beginDeleteAsync(resourceGroupName, jobName, context) - .last() + return beginDeleteAsync(resourceGroupName, jobName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1154,7 +1388,7 @@ public void delete(String resourceGroupName, String jobName) { /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -1169,9 +1403,9 @@ public void delete(String resourceGroupName, String jobName, Context context) { /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1181,19 +1415,15 @@ public void delete(String resourceGroupName, String jobName, Context context) { * @return container App Job along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1209,26 +1439,16 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - jobEnvelope, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), jobEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1239,19 +1459,15 @@ private Mono>> updateWithResponseAsync( * @return container App Job along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1267,23 +1483,15 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - jobEnvelope, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, jobName, + this.client.getApiVersion(), jobEnvelope, accept, context); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1293,20 +1501,18 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobInner> beginUpdateAsync( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope) { + private PollerFlux, JobInner> beginUpdateAsync(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope) { Mono>> mono = updateWithResponseAsync(resourceGroupName, jobName, jobEnvelope); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), JobInner.class, JobInner.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), JobInner.class, + JobInner.class, this.client.getContext()); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1317,22 +1523,20 @@ private PollerFlux, JobInner> beginUpdateAsync( * @return the {@link PollerFlux} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobInner> beginUpdateAsync( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope, Context context) { + private PollerFlux, JobInner> beginUpdateAsync(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, jobName, jobEnvelope, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), JobInner.class, JobInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, jobName, jobEnvelope, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), JobInner.class, + JobInner.class, context); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1342,16 +1546,16 @@ private PollerFlux, JobInner> beginUpdateAsync( * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, JobInner> beginUpdate( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope) { + public SyncPoller, JobInner> beginUpdate(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope) { return this.beginUpdateAsync(resourceGroupName, jobName, jobEnvelope).getSyncPoller(); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1362,16 +1566,16 @@ public SyncPoller, JobInner> beginUpdate( * @return the {@link SyncPoller} for polling of container App Job. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, JobInner> beginUpdate( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope, Context context) { + public SyncPoller, JobInner> beginUpdate(String resourceGroupName, String jobName, + JobPatchProperties jobEnvelope, Context context) { return this.beginUpdateAsync(resourceGroupName, jobName, jobEnvelope, context).getSyncPoller(); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1382,16 +1586,15 @@ public SyncPoller, JobInner> beginUpdate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono updateAsync(String resourceGroupName, String jobName, JobPatchProperties jobEnvelope) { - return beginUpdateAsync(resourceGroupName, jobName, jobEnvelope) - .last() + return beginUpdateAsync(resourceGroupName, jobName, jobEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1402,18 +1605,17 @@ private Mono updateAsync(String resourceGroupName, String jobName, Job * @return container App Job on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String jobName, JobPatchProperties jobEnvelope, Context context) { - return beginUpdateAsync(resourceGroupName, jobName, jobEnvelope, context) - .last() + private Mono updateAsync(String resourceGroupName, String jobName, JobPatchProperties jobEnvelope, + Context context) { + return beginUpdateAsync(resourceGroupName, jobName, jobEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1429,9 +1631,9 @@ public JobInner update(String resourceGroupName, String jobName, JobPatchPropert /** * Update properties of a Container Apps Job - * - *

Patches a Container Apps Job using JSON Merge Patch. - * + * + * Patches a Container Apps Job using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobEnvelope Properties used to create a container apps job. @@ -1448,7 +1650,7 @@ public JobInner update(String resourceGroupName, String jobName, JobPatchPropert /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1458,19 +1660,15 @@ public JobInner update(String resourceGroupName, String jobName, JobPatchPropert * @return container App's Job execution name along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> startWithResponseAsync( - String resourceGroupName, String jobName, JobExecutionTemplate template) { + private Mono>> startWithResponseAsync(String resourceGroupName, String jobName, + JobExecutionTemplate template) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1484,24 +1682,14 @@ private Mono>> startWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .start( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - template, - accept, - context)) + .withContext(context -> service.start(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), template, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1512,19 +1700,15 @@ private Mono>> startWithResponseAsync( * @return container App's Job execution name along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> startWithResponseAsync( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { + private Mono>> startWithResponseAsync(String resourceGroupName, String jobName, + JobExecutionTemplate template, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1538,21 +1722,13 @@ private Mono>> startWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .start( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - template, - accept, - context); + return service.start(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, jobName, + this.client.getApiVersion(), template, accept, context); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1562,22 +1738,17 @@ private Mono>> startWithResponseAsync( * @return the {@link PollerFlux} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobExecutionBaseInner> beginStartAsync( - String resourceGroupName, String jobName, JobExecutionTemplate template) { + private PollerFlux, JobExecutionBaseInner> + beginStartAsync(String resourceGroupName, String jobName, JobExecutionTemplate template) { Mono>> mono = startWithResponseAsync(resourceGroupName, jobName, template); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - JobExecutionBaseInner.class, - JobExecutionBaseInner.class, - this.client.getContext()); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), JobExecutionBaseInner.class, JobExecutionBaseInner.class, + this.client.getContext()); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1586,23 +1757,18 @@ private PollerFlux, JobExecutionBaseInner> beg * @return the {@link PollerFlux} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobExecutionBaseInner> beginStartAsync( - String resourceGroupName, String jobName) { + private PollerFlux, JobExecutionBaseInner> + beginStartAsync(String resourceGroupName, String jobName) { final JobExecutionTemplate template = null; Mono>> mono = startWithResponseAsync(resourceGroupName, jobName, template); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - JobExecutionBaseInner.class, - JobExecutionBaseInner.class, - this.client.getContext()); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), JobExecutionBaseInner.class, JobExecutionBaseInner.class, + this.client.getContext()); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1613,19 +1779,17 @@ private PollerFlux, JobExecutionBaseInner> beg * @return the {@link PollerFlux} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, JobExecutionBaseInner> beginStartAsync( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { + private PollerFlux, JobExecutionBaseInner> + beginStartAsync(String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { context = this.client.mergeContext(context); Mono>> mono = startWithResponseAsync(resourceGroupName, jobName, template, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), JobExecutionBaseInner.class, JobExecutionBaseInner.class, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), JobExecutionBaseInner.class, JobExecutionBaseInner.class, context); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1634,15 +1798,15 @@ private PollerFlux, JobExecutionBaseInner> beg * @return the {@link SyncPoller} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, JobExecutionBaseInner> beginStart( - String resourceGroupName, String jobName) { + public SyncPoller, JobExecutionBaseInner> beginStart(String resourceGroupName, + String jobName) { final JobExecutionTemplate template = null; return this.beginStartAsync(resourceGroupName, jobName, template).getSyncPoller(); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1653,14 +1817,14 @@ public SyncPoller, JobExecutionBaseInner> begi * @return the {@link SyncPoller} for polling of container App's Job execution name. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, JobExecutionBaseInner> beginStart( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { + public SyncPoller, JobExecutionBaseInner> beginStart(String resourceGroupName, + String jobName, JobExecutionTemplate template, Context context) { return this.beginStartAsync(resourceGroupName, jobName, template, context).getSyncPoller(); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1670,16 +1834,15 @@ public SyncPoller, JobExecutionBaseInner> begi * @return container App's Job execution name on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono startAsync( - String resourceGroupName, String jobName, JobExecutionTemplate template) { - return beginStartAsync(resourceGroupName, jobName, template) - .last() + private Mono startAsync(String resourceGroupName, String jobName, + JobExecutionTemplate template) { + return beginStartAsync(resourceGroupName, jobName, template).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1690,14 +1853,13 @@ private Mono startAsync( @ServiceMethod(returns = ReturnType.SINGLE) private Mono startAsync(String resourceGroupName, String jobName) { final JobExecutionTemplate template = null; - return beginStartAsync(resourceGroupName, jobName, template) - .last() + return beginStartAsync(resourceGroupName, jobName, template).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1708,16 +1870,15 @@ private Mono startAsync(String resourceGroupName, String * @return container App's Job execution name on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono startAsync( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { - return beginStartAsync(resourceGroupName, jobName, template, context) - .last() + private Mono startAsync(String resourceGroupName, String jobName, + JobExecutionTemplate template, Context context) { + return beginStartAsync(resourceGroupName, jobName, template, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1733,7 +1894,7 @@ public JobExecutionBaseInner start(String resourceGroupName, String jobName) { /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. @@ -1744,14 +1905,14 @@ public JobExecutionBaseInner start(String resourceGroupName, String jobName) { * @return container App's Job execution name. */ @ServiceMethod(returns = ReturnType.SINGLE) - public JobExecutionBaseInner start( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { + public JobExecutionBaseInner start(String resourceGroupName, String jobName, JobExecutionTemplate template, + Context context) { return startAsync(resourceGroupName, jobName, template, context).block(); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1761,19 +1922,15 @@ public JobExecutionBaseInner start( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopExecutionWithResponseAsync( - String resourceGroupName, String jobName, String jobExecutionName) { + private Mono>> stopExecutionWithResponseAsync(String resourceGroupName, String jobName, + String jobExecutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1788,24 +1945,14 @@ private Mono>> stopExecutionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .stopExecution( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - jobName, - jobExecutionName, - accept, - context)) + .withContext(context -> service.stopExecution(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), jobName, jobExecutionName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1816,19 +1963,15 @@ private Mono>> stopExecutionWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopExecutionWithResponseAsync( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { + private Mono>> stopExecutionWithResponseAsync(String resourceGroupName, String jobName, + String jobExecutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1843,21 +1986,13 @@ private Mono>> stopExecutionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .stopExecution( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - jobName, - jobExecutionName, - accept, - context); + return service.stopExecution(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), jobName, jobExecutionName, accept, context); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1867,19 +2002,17 @@ private Mono>> stopExecutionWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginStopExecutionAsync( - String resourceGroupName, String jobName, String jobExecutionName) { - Mono>> mono = - stopExecutionWithResponseAsync(resourceGroupName, jobName, jobExecutionName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginStopExecutionAsync(String resourceGroupName, String jobName, + String jobExecutionName) { + Mono>> mono + = stopExecutionWithResponseAsync(resourceGroupName, jobName, jobExecutionName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1890,19 +2023,18 @@ private PollerFlux, Void> beginStopExecutionAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginStopExecutionAsync( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { + private PollerFlux, Void> beginStopExecutionAsync(String resourceGroupName, String jobName, + String jobExecutionName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - stopExecutionWithResponseAsync(resourceGroupName, jobName, jobExecutionName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = stopExecutionWithResponseAsync(resourceGroupName, jobName, jobExecutionName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1912,14 +2044,14 @@ private PollerFlux, Void> beginStopExecutionAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginStopExecution( - String resourceGroupName, String jobName, String jobExecutionName) { + public SyncPoller, Void> beginStopExecution(String resourceGroupName, String jobName, + String jobExecutionName) { return this.beginStopExecutionAsync(resourceGroupName, jobName, jobExecutionName).getSyncPoller(); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1930,14 +2062,14 @@ public SyncPoller, Void> beginStopExecution( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginStopExecution( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { + public SyncPoller, Void> beginStopExecution(String resourceGroupName, String jobName, + String jobExecutionName, Context context) { return this.beginStopExecutionAsync(resourceGroupName, jobName, jobExecutionName, context).getSyncPoller(); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1948,14 +2080,13 @@ public SyncPoller, Void> beginStopExecution( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono stopExecutionAsync(String resourceGroupName, String jobName, String jobExecutionName) { - return beginStopExecutionAsync(resourceGroupName, jobName, jobExecutionName) - .last() + return beginStopExecutionAsync(resourceGroupName, jobName, jobExecutionName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1966,16 +2097,15 @@ private Mono stopExecutionAsync(String resourceGroupName, String jobName, * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono stopExecutionAsync( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { - return beginStopExecutionAsync(resourceGroupName, jobName, jobExecutionName, context) - .last() + private Mono stopExecutionAsync(String resourceGroupName, String jobName, String jobExecutionName, + Context context) { + return beginStopExecutionAsync(resourceGroupName, jobName, jobExecutionName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -1990,7 +2120,7 @@ public void stopExecution(String resourceGroupName, String jobName, String jobEx /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -2006,29 +2136,25 @@ public void stopExecution(String resourceGroupName, String jobName, String jobEx /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopMultipleExecutionsWithResponseAsync( - String resourceGroupName, String jobName) { + private Mono>> stopMultipleExecutionsWithResponseAsync(String resourceGroupName, + String jobName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2040,22 +2166,14 @@ private Mono>> stopMultipleExecutionsWithResponseAsync final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .stopMultipleExecutions( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - jobName, - accept, - context)) + context -> service.stopMultipleExecutions(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), jobName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2063,22 +2181,18 @@ private Mono>> stopMultipleExecutionsWithResponseAsync * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopMultipleExecutionsWithResponseAsync( - String resourceGroupName, String jobName, Context context) { + private Mono>> stopMultipleExecutionsWithResponseAsync(String resourceGroupName, + String jobName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2089,20 +2203,13 @@ private Mono>> stopMultipleExecutionsWithResponseAsync } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .stopMultipleExecutions( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - jobName, - accept, - context); + return service.stopMultipleExecutions(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), jobName, accept, context); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2114,19 +2221,14 @@ private Mono>> stopMultipleExecutionsWithResponseAsync private PollerFlux, ContainerAppJobExecutionsInner> beginStopMultipleExecutionsAsync(String resourceGroupName, String jobName) { Mono>> mono = stopMultipleExecutionsWithResponseAsync(resourceGroupName, jobName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ContainerAppJobExecutionsInner.class, - ContainerAppJobExecutionsInner.class, - this.client.getContext()); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ContainerAppJobExecutionsInner.class, ContainerAppJobExecutionsInner.class, + this.client.getContext()); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2139,21 +2241,16 @@ private Mono>> stopMultipleExecutionsWithResponseAsync private PollerFlux, ContainerAppJobExecutionsInner> beginStopMultipleExecutionsAsync(String resourceGroupName, String jobName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - stopMultipleExecutionsWithResponseAsync(resourceGroupName, jobName, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ContainerAppJobExecutionsInner.class, - ContainerAppJobExecutionsInner.class, - context); + Mono>> mono + = stopMultipleExecutionsWithResponseAsync(resourceGroupName, jobName, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ContainerAppJobExecutionsInner.class, ContainerAppJobExecutionsInner.class, + context); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2169,7 +2266,7 @@ private Mono>> stopMultipleExecutionsWithResponseAsync /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2186,7 +2283,7 @@ private Mono>> stopMultipleExecutionsWithResponseAsync /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2196,14 +2293,13 @@ private Mono>> stopMultipleExecutionsWithResponseAsync */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono stopMultipleExecutionsAsync(String resourceGroupName, String jobName) { - return beginStopMultipleExecutionsAsync(resourceGroupName, jobName) - .last() + return beginStopMultipleExecutionsAsync(resourceGroupName, jobName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2213,16 +2309,15 @@ private Mono stopMultipleExecutionsAsync(String * @return container App executions collection ARM resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono stopMultipleExecutionsAsync( - String resourceGroupName, String jobName, Context context) { - return beginStopMultipleExecutionsAsync(resourceGroupName, jobName, context) - .last() + private Mono stopMultipleExecutionsAsync(String resourceGroupName, String jobName, + Context context) { + return beginStopMultipleExecutionsAsync(resourceGroupName, jobName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2237,7 +2332,7 @@ public ContainerAppJobExecutionsInner stopMultipleExecutions(String resourceGrou /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2247,36 +2342,32 @@ public ContainerAppJobExecutionsInner stopMultipleExecutions(String resourceGrou * @return container App executions collection ARM resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ContainerAppJobExecutionsInner stopMultipleExecutions( - String resourceGroupName, String jobName, Context context) { + public ContainerAppJobExecutionsInner stopMultipleExecutions(String resourceGroupName, String jobName, + Context context) { return stopMultipleExecutionsAsync(resourceGroupName, jobName, context).block(); } /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String jobName) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String jobName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2287,23 +2378,14 @@ private Mono> listSecretsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - jobName, - accept, - context)) + .withContext(context -> service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), jobName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2311,22 +2393,18 @@ private Mono> listSecretsWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSecretsWithResponseAsync( - String resourceGroupName, String jobName, Context context) { + private Mono> listSecretsWithResponseAsync(String resourceGroupName, + String jobName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2337,20 +2415,13 @@ private Mono> listSecretsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listSecrets( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - jobName, - accept, - context); + return service.listSecrets(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), jobName, accept, context); } /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2366,7 +2437,7 @@ private Mono listSecretsAsync(String resourceGroupNam /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. @@ -2376,14 +2447,14 @@ private Mono listSecretsAsync(String resourceGroupNam * @return container Apps Job Secrets Collection ARM resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listSecretsWithResponse( - String resourceGroupName, String jobName, Context context) { + public Response listSecretsWithResponse(String resourceGroupName, String jobName, + Context context) { return listSecretsWithResponseAsync(resourceGroupName, jobName, context).block(); } /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2398,14 +2469,15 @@ public JobSecretsCollectionInner listSecrets(String resourceGroupName, String jo /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Jobs collection ARM resource along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -2413,38 +2485,30 @@ private Mono> listBySubscriptionNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Jobs collection ARM resource along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { @@ -2452,36 +2516,27 @@ private Mono> listBySubscriptionNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Jobs collection ARM resource along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -2489,38 +2544,30 @@ private Mono> listByResourceGroupNextSinglePageAsync(Str return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Jobs collection ARM resource along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, Context context) { @@ -2528,23 +2575,13 @@ private Mono> listByResourceGroupNextSinglePageAsync(Str return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsClientImpl.java index 6bf912ce7b4b7..4712b36b05141 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in JobsExecutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in JobsExecutionsClient. + */ public final class JobsExecutionsClientImpl implements JobsExecutionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final JobsExecutionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of JobsExecutionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ JobsExecutionsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(JobsExecutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(JobsExecutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,58 +62,47 @@ public final class JobsExecutionsClientImpl implements JobsExecutionsClient { @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface JobsExecutionsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/executions") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/executions") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @QueryParam("api-version") String apiVersion, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @QueryParam("api-version") String apiVersion, @QueryParam("$filter") String filter, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a Container Apps Job's executions along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a Container Apps Job's executions along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String jobName, String filter) { + private Mono> listSinglePageAsync(String resourceGroupName, String jobName, + String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -118,33 +113,16 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, this.client.getApiVersion(), filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. @@ -152,23 +130,19 @@ private Mono> listSinglePageAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a Container Apps Job's executions along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a Container Apps Job's executions along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String jobName, String filter, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, String jobName, + String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -180,29 +154,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - this.client.getApiVersion(), - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, jobName, + this.client.getApiVersion(), filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. @@ -213,14 +173,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String jobName, String filter) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, jobName, filter), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, jobName, filter), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -231,14 +190,13 @@ private PagedFlux listAsync(String resourceGroupName, String @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String jobName) { final String filter = null; - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, jobName, filter), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, jobName, filter), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. @@ -249,16 +207,15 @@ private PagedFlux listAsync(String resourceGroupName, String * @return a Container Apps Job's executions as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String jobName, String filter, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, jobName, filter, context), + private PagedFlux listAsync(String resourceGroupName, String jobName, String filter, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, jobName, filter, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -274,7 +231,7 @@ public PagedIterable list(String resourceGroupName, String jo /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. @@ -285,21 +242,22 @@ public PagedIterable list(String resourceGroupName, String jo * @return a Container Apps Job's executions as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String jobName, String filter, Context context) { + public PagedIterable list(String resourceGroupName, String jobName, String filter, + Context context) { return new PagedIterable<>(listAsync(resourceGroupName, jobName, filter, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -307,37 +265,28 @@ private Mono> listNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -345,23 +294,13 @@ private Mono> listNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsImpl.java index ec5d43fa745b5..eb3cb8fe223ab 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsExecutionsImpl.java @@ -19,8 +19,7 @@ public final class JobsExecutionsImpl implements JobsExecutions { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public JobsExecutionsImpl( - JobsExecutionsClient innerClient, + public JobsExecutionsImpl(JobsExecutionsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public JobsExecutionsImpl( public PagedIterable list(String resourceGroupName, String jobName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, jobName); - return Utils.mapPage(inner, inner1 -> new JobExecutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JobExecutionImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String jobName, String filter, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, jobName, filter, context); - return Utils.mapPage(inner, inner1 -> new JobExecutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JobExecutionImpl(inner1, this.manager())); } private JobsExecutionsClient serviceClient() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsImpl.java index f141ae9a6fa65..2ec3b5f9e57f0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/JobsImpl.java @@ -11,10 +11,14 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.appcontainers.fluent.JobsClient; import com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner; +import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner; +import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionBaseInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobSecretsCollectionInner; import com.azure.resourcemanager.appcontainers.models.ContainerAppJobExecutions; +import com.azure.resourcemanager.appcontainers.models.Diagnostics; +import com.azure.resourcemanager.appcontainers.models.DiagnosticsCollection; import com.azure.resourcemanager.appcontainers.models.Job; import com.azure.resourcemanager.appcontainers.models.JobExecutionBase; import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate; @@ -28,40 +32,98 @@ public final class JobsImpl implements Jobs { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public JobsImpl( - JobsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + public JobsImpl(JobsClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } + public Response listDetectorsWithResponse(String resourceGroupName, String jobName, + Context context) { + Response inner + = this.serviceClient().listDetectorsWithResponse(resourceGroupName, jobName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DiagnosticsCollectionImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public DiagnosticsCollection listDetectors(String resourceGroupName, String jobName) { + DiagnosticsCollectionInner inner = this.serviceClient().listDetectors(resourceGroupName, jobName); + if (inner != null) { + return new DiagnosticsCollectionImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response getDetectorWithResponse(String resourceGroupName, String jobName, String detectorName, + Context context) { + Response inner + = this.serviceClient().getDetectorWithResponse(resourceGroupName, jobName, detectorName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DiagnosticsImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public Diagnostics getDetector(String resourceGroupName, String jobName, String detectorName) { + DiagnosticsInner inner = this.serviceClient().getDetector(resourceGroupName, jobName, detectorName); + if (inner != null) { + return new DiagnosticsImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response proxyGetWithResponse(String resourceGroupName, String jobName, Context context) { + Response inner = this.serviceClient().proxyGetWithResponse(resourceGroupName, jobName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new JobImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public Job proxyGet(String resourceGroupName, String jobName) { + JobInner inner = this.serviceClient().proxyGet(resourceGroupName, jobName); + if (inner != null) { + return new JobImpl(inner, this.manager()); + } else { + return null; + } + } + public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JobImpl(inner1, this.manager())); } public Response getByResourceGroupWithResponse(String resourceGroupName, String jobName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, jobName, context); + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, jobName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new JobImpl(inner.getValue(), this.manager())); } else { return null; @@ -94,8 +156,8 @@ public JobExecutionBase start(String resourceGroupName, String jobName) { } } - public JobExecutionBase start( - String resourceGroupName, String jobName, JobExecutionTemplate template, Context context) { + public JobExecutionBase start(String resourceGroupName, String jobName, JobExecutionTemplate template, + Context context) { JobExecutionBaseInner inner = this.serviceClient().start(resourceGroupName, jobName, template, context); if (inner != null) { return new JobExecutionBaseImpl(inner, this.manager()); @@ -122,8 +184,8 @@ public ContainerAppJobExecutions stopMultipleExecutions(String resourceGroupName } public ContainerAppJobExecutions stopMultipleExecutions(String resourceGroupName, String jobName, Context context) { - ContainerAppJobExecutionsInner inner = - this.serviceClient().stopMultipleExecutions(resourceGroupName, jobName, context); + ContainerAppJobExecutionsInner inner + = this.serviceClient().stopMultipleExecutions(resourceGroupName, jobName, context); if (inner != null) { return new ContainerAppJobExecutionsImpl(inner, this.manager()); } else { @@ -131,15 +193,12 @@ public ContainerAppJobExecutions stopMultipleExecutions(String resourceGroupName } } - public Response listSecretsWithResponse( - String resourceGroupName, String jobName, Context context) { - Response inner = - this.serviceClient().listSecretsWithResponse(resourceGroupName, jobName, context); + public Response listSecretsWithResponse(String resourceGroupName, String jobName, + Context context) { + Response inner + = this.serviceClient().listSecretsWithResponse(resourceGroupName, jobName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new JobSecretsCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -156,77 +215,57 @@ public JobSecretsCollection listSecrets(String resourceGroupName, String jobName } public Job getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String jobName = Utils.getValueFromIdByName(id, "jobs"); + String jobName = ResourceManagerUtils.getValueFromIdByName(id, "jobs"); if (jobName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, jobName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String jobName = Utils.getValueFromIdByName(id, "jobs"); + String jobName = ResourceManagerUtils.getValueFromIdByName(id, "jobs"); if (jobName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, jobName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String jobName = Utils.getValueFromIdByName(id, "jobs"); + String jobName = ResourceManagerUtils.getValueFromIdByName(id, "jobs"); if (jobName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); } this.delete(resourceGroupName, jobName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String jobName = Utils.getValueFromIdByName(id, "jobs"); + String jobName = ResourceManagerUtils.getValueFromIdByName(id, "jobs"); if (jobName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'jobs'.", id))); } this.delete(resourceGroupName, jobName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificateImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificateImpl.java index 678fd7d6c7851..56dda00443796 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificateImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificateImpl.java @@ -88,26 +88,19 @@ public ManagedCertificateImpl withExistingManagedEnvironment(String resourceGrou } public ManagedCertificate create() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedCertificates() - .createOrUpdate( - resourceGroupName, environmentName, managedCertificateName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getManagedCertificates().createOrUpdate(resourceGroupName, + environmentName, managedCertificateName, this.innerModel(), Context.NONE); return this; } public ManagedCertificate create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedCertificates() - .createOrUpdate(resourceGroupName, environmentName, managedCertificateName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getManagedCertificates().createOrUpdate(resourceGroupName, + environmentName, managedCertificateName, this.innerModel(), context); return this; } - ManagedCertificateImpl( - String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ManagedCertificateImpl(String name, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = new ManagedCertificateInner(); this.serviceManager = serviceManager; this.managedCertificateName = name; @@ -119,62 +112,36 @@ public ManagedCertificateImpl update() { } public ManagedCertificate apply() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedCertificates() - .updateWithResponse( - resourceGroupName, - environmentName, - managedCertificateName, - updateManagedCertificateEnvelope, - Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedCertificates().updateWithResponse(resourceGroupName, + environmentName, managedCertificateName, updateManagedCertificateEnvelope, Context.NONE).getValue(); return this; } public ManagedCertificate apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedCertificates() - .updateWithResponse( - resourceGroupName, - environmentName, - managedCertificateName, - updateManagedCertificateEnvelope, - context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedCertificates().updateWithResponse(resourceGroupName, + environmentName, managedCertificateName, updateManagedCertificateEnvelope, context).getValue(); return this; } - ManagedCertificateImpl( - ManagedCertificateInner innerObject, + ManagedCertificateImpl(ManagedCertificateInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.environmentName = Utils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); - this.managedCertificateName = Utils.getValueFromIdByName(innerObject.id(), "managedCertificates"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.managedCertificateName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedCertificates"); } public ManagedCertificate refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedCertificates() - .getWithResponse(resourceGroupName, environmentName, managedCertificateName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedCertificates() + .getWithResponse(resourceGroupName, environmentName, managedCertificateName, Context.NONE).getValue(); return this; } public ManagedCertificate refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedCertificates() - .getWithResponse(resourceGroupName, environmentName, managedCertificateName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedCertificates() + .getWithResponse(resourceGroupName, environmentName, managedCertificateName, context).getValue(); return this; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesClientImpl.java index 73e22fb564193..2f82424fccd47 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesClientImpl.java @@ -41,22 +41,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ManagedCertificatesClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedCertificatesClient. + */ public final class ManagedCertificatesClientImpl implements ManagedCertificatesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ManagedCertificatesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ManagedCertificatesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ManagedCertificatesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(ManagedCertificatesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ManagedCertificatesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -67,99 +73,77 @@ public final class ManagedCertificatesClientImpl implements ManagedCertificatesC @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ManagedCertificatesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("environmentName") String environmentName, @PathParam("managedCertificateName") String managedCertificateName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") - @ExpectedResponses({200, 201}) - @UnexpectedResponseExceptionType( - value = ManagementException.class, - code = {400}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(value = ManagementException.class, code = { 400 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("environmentName") String environmentName, @PathParam("managedCertificateName") String managedCertificateName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ManagedCertificateInner managedCertificateEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("environmentName") String environmentName, @PathParam("managedCertificateName") String managedCertificateName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates/{managedCertificateName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> update( - @HostParam("$host") String endpoint, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("environmentName") String environmentName, @PathParam("managedCertificateName") String managedCertificateName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ManagedCertificatePatch managedCertificateEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/managedCertificates") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -169,19 +153,15 @@ Mono> listNext( * @return the specified Managed Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String managedCertificateName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String environmentName, String managedCertificateName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -192,30 +172,20 @@ private Mono> getWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - accept, - context)) + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, managedCertificateName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -226,19 +196,15 @@ private Mono> getWithResponseAsync( * @return the specified Managed Certificate along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String managedCertificateName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String environmentName, String managedCertificateName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -249,27 +215,18 @@ private Mono> getWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, managedCertificateName, this.client.getApiVersion(), accept, context); } /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -279,15 +236,15 @@ private Mono> getWithResponseAsync( * @return the specified Managed Certificate on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String environmentName, String managedCertificateName) { + private Mono getAsync(String resourceGroupName, String environmentName, + String managedCertificateName) { return getWithResponseAsync(resourceGroupName, environmentName, managedCertificateName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -298,14 +255,14 @@ private Mono getAsync( * @return the specified Managed Certificate along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context) { + public Response getWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, Context context) { return getWithResponseAsync(resourceGroupName, environmentName, managedCertificateName, context).block(); } /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -315,14 +272,14 @@ public Response getWithResponse( * @return the specified Managed Certificate. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedCertificateInner get( - String resourceGroupName, String environmentName, String managedCertificateName) { + public ManagedCertificateInner get(String resourceGroupName, String environmentName, + String managedCertificateName) { return getWithResponse(resourceGroupName, environmentName, managedCertificateName, Context.NONE).getValue(); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -332,25 +289,18 @@ public ManagedCertificateInner get( * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -361,34 +311,23 @@ private Mono>> createOrUpdateWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } if (managedCertificateEnvelope != null) { managedCertificateEnvelope.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - managedCertificateEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, managedCertificateName, this.client.getApiVersion(), + managedCertificateEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -399,26 +338,19 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -429,31 +361,22 @@ private Mono>> createOrUpdateWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } if (managedCertificateEnvelope != null) { managedCertificateEnvelope.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - managedCertificateEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, managedCertificateName, this.client.getApiVersion(), managedCertificateEnvelope, accept, + context); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -463,30 +386,22 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ManagedCertificateInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, + String resourceGroupName, String environmentName, String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedCertificateInner.class, - ManagedCertificateInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, + managedCertificateName, managedCertificateEnvelope); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedCertificateInner.class, ManagedCertificateInner.class, + this.client.getContext()); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -495,28 +410,22 @@ private PollerFlux, ManagedCertificateInner> * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ManagedCertificateInner> beginCreateOrUpdateAsync( - String resourceGroupName, String environmentName, String managedCertificateName) { + private PollerFlux, ManagedCertificateInner> + beginCreateOrUpdateAsync(String resourceGroupName, String environmentName, String managedCertificateName) { final ManagedCertificateInner managedCertificateEnvelope = null; - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedCertificateInner.class, - ManagedCertificateInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, + managedCertificateName, managedCertificateEnvelope); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedCertificateInner.class, ManagedCertificateInner.class, + this.client.getContext()); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -527,32 +436,22 @@ private PollerFlux, ManagedCertificateInner> * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ManagedCertificateInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, - Context context) { + String resourceGroupName, String environmentName, String managedCertificateName, + ManagedCertificateInner managedCertificateEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedCertificateInner.class, - ManagedCertificateInner.class, - context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, + managedCertificateName, managedCertificateEnvelope, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedCertificateInner.class, ManagedCertificateInner.class, context); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -561,21 +460,19 @@ private PollerFlux, ManagedCertificateInner> * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ManagedCertificateInner> beginCreateOrUpdate( - String resourceGroupName, String environmentName, String managedCertificateName) { + public SyncPoller, ManagedCertificateInner> + beginCreateOrUpdate(String resourceGroupName, String environmentName, String managedCertificateName) { final ManagedCertificateInner managedCertificateEnvelope = null; - return this - .beginCreateOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope) - .getSyncPoller(); + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope).getSyncPoller(); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -586,24 +483,19 @@ public SyncPoller, ManagedCertificateInner> * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of managed certificates used for Custom Domain bindings of Container - * Apps in a Managed Environment. + * Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ManagedCertificateInner> beginCreateOrUpdate( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, - Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope, context) - .getSyncPoller(); + String resourceGroupName, String environmentName, String managedCertificateName, + ManagedCertificateInner managedCertificateEnvelope, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope, context).getSyncPoller(); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -613,23 +505,18 @@ public SyncPoller, ManagedCertificateInner> * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope) { - return beginCreateOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -638,21 +525,19 @@ private Mono createOrUpdateAsync( * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String environmentName, String managedCertificateName) { + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String managedCertificateName) { final ManagedCertificateInner managedCertificateEnvelope = null; - return beginCreateOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope) - .last() - .flatMap(this.client::getLroFinalResultOrError); + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -663,24 +548,18 @@ private Mono createOrUpdateAsync( * @throws ManagementException thrown if the request is rejected by server on status code 400. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -691,17 +570,16 @@ private Mono createOrUpdateAsync( * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedCertificateInner createOrUpdate( - String resourceGroupName, String environmentName, String managedCertificateName) { + public ManagedCertificateInner createOrUpdate(String resourceGroupName, String environmentName, + String managedCertificateName) { final ManagedCertificateInner managedCertificateEnvelope = null; - return createOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope) - .block(); + return createOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope).block(); } /** * Create or Update a Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -714,20 +592,15 @@ public ManagedCertificateInner createOrUpdate( * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedCertificateInner createOrUpdate( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificateInner managedCertificateEnvelope, - Context context) { - return createOrUpdateAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope, context) - .block(); + public ManagedCertificateInner createOrUpdate(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificateInner managedCertificateEnvelope, Context context) { + return createOrUpdateAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope, context).block(); } /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -737,19 +610,15 @@ public ManagedCertificateInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String managedCertificateName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String managedCertificateName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -760,30 +629,20 @@ private Mono> deleteWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - accept, - context)) + context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, managedCertificateName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -794,19 +653,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String managedCertificateName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String managedCertificateName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -817,27 +672,18 @@ private Mono> deleteWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, managedCertificateName, this.client.getApiVersion(), accept, context); } /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -854,7 +700,7 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -865,14 +711,14 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, Context context) { return deleteWithResponseAsync(resourceGroupName, environmentName, managedCertificateName, context).block(); } /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -887,9 +733,9 @@ public void delete(String resourceGroupName, String environmentName, String mana /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -898,25 +744,18 @@ public void delete(String resourceGroupName, String environmentName, String mana * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificatePatch managedCertificateEnvelope) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String environmentName, String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -927,41 +766,28 @@ private Mono> updateWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } if (managedCertificateEnvelope == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter managedCertificateEnvelope is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateEnvelope is required and cannot be null.")); } else { managedCertificateEnvelope.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - managedCertificateEnvelope, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, managedCertificateName, this.client.getApiVersion(), + managedCertificateEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -971,26 +797,19 @@ private Mono> updateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificatePatch managedCertificateEnvelope, + private Mono> updateWithResponseAsync(String resourceGroupName, + String environmentName, String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1001,38 +820,27 @@ private Mono> updateWithResponseAsync( .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (managedCertificateName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateName is required and cannot be null.")); } if (managedCertificateEnvelope == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter managedCertificateEnvelope is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter managedCertificateEnvelope is required and cannot be null.")); } else { managedCertificateEnvelope.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - managedCertificateName, - this.client.getApiVersion(), - managedCertificateEnvelope, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, managedCertificateName, this.client.getApiVersion(), managedCertificateEnvelope, accept, + context); } /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -1041,24 +849,20 @@ private Mono> updateWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificatePatch managedCertificateEnvelope) { - return updateWithResponseAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono updateAsync(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope) { + return updateWithResponseAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -1068,25 +872,20 @@ private Mono updateAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment along - * with {@link Response}. + * with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificatePatch managedCertificateEnvelope, - Context context) { - return updateWithResponseAsync( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope, context) - .block(); + public Response updateWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope, Context context) { + return updateWithResponseAsync(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope, context).block(); } /** * Update tags of a managed certificate - * - *

Patches a managed certificate. Oly patching of tags is supported. - * + * + * Patches a managed certificate. Oly patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. @@ -1097,41 +896,33 @@ public Response updateWithResponse( * @return managed certificates used for Custom Domain bindings of Container Apps in a Managed Environment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedCertificateInner update( - String resourceGroupName, - String environmentName, - String managedCertificateName, - ManagedCertificatePatch managedCertificateEnvelope) { - return updateWithResponse( - resourceGroupName, environmentName, managedCertificateName, managedCertificateEnvelope, Context.NONE) - .getValue(); + public ManagedCertificateInner update(String resourceGroupName, String environmentName, + String managedCertificateName, ManagedCertificatePatch managedCertificateEnvelope) { + return updateWithResponse(resourceGroupName, environmentName, managedCertificateName, + managedCertificateEnvelope, Context.NONE).getValue(); } /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Managed Certificates in a given managed environment along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String environmentName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1143,32 +934,16 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1176,22 +951,18 @@ private Mono> listSinglePageAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Managed Certificates in a given managed environment along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1204,28 +975,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1235,14 +993,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String environmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, environmentName), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1252,16 +1009,15 @@ private PagedFlux listAsync(String resourceGroupName, S * @return the Managed Certificates in a given managed environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String environmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, environmentName, context), + private PagedFlux listAsync(String resourceGroupName, String environmentName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1276,7 +1032,7 @@ public PagedIterable list(String resourceGroupName, Str /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1286,21 +1042,22 @@ public PagedIterable list(String resourceGroupName, Str * @return the Managed Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String environmentName, Context context) { + public PagedIterable list(String resourceGroupName, String environmentName, + Context context) { return new PagedIterable<>(listAsync(resourceGroupName, environmentName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of Managed Certificates along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of Managed Certificates along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1308,37 +1065,28 @@ private Mono> listNextSinglePageAsync(Str return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of Managed Certificates along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of Managed Certificates along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1346,23 +1094,13 @@ private Mono> listNextSinglePageAsync(Str return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesImpl.java index d20f95e682ae3..cfdb8c2361cc5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedCertificatesImpl.java @@ -21,22 +21,18 @@ public final class ManagedCertificatesImpl implements ManagedCertificates { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ManagedCertificatesImpl( - ManagedCertificatesClient innerClient, + public ManagedCertificatesImpl(ManagedCertificatesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, environmentName, managedCertificateName, context); + public Response getWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, managedCertificateName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ManagedCertificateImpl(inner.getValue(), this.manager())); } else { return null; @@ -44,8 +40,8 @@ public Response getWithResponse( } public ManagedCertificate get(String resourceGroupName, String environmentName, String managedCertificateName) { - ManagedCertificateInner inner = - this.serviceClient().get(resourceGroupName, environmentName, managedCertificateName); + ManagedCertificateInner inner + = this.serviceClient().get(resourceGroupName, environmentName, managedCertificateName); if (inner != null) { return new ManagedCertificateImpl(inner, this.manager()); } else { @@ -53,11 +49,10 @@ public ManagedCertificate get(String resourceGroupName, String environmentName, } } - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, environmentName, managedCertificateName, context); + public Response deleteWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, environmentName, managedCertificateName, + context); } public void delete(String resourceGroupName, String environmentName, String managedCertificateName) { @@ -66,133 +61,88 @@ public void delete(String resourceGroupName, String environmentName, String mana public PagedIterable list(String resourceGroupName, String environmentName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); - return Utils.mapPage(inner, inner1 -> new ManagedCertificateImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ManagedCertificateImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String environmentName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, environmentName, context); - return Utils.mapPage(inner, inner1 -> new ManagedCertificateImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ManagedCertificateImpl(inner1, this.manager())); } public ManagedCertificate getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } - String managedCertificateName = Utils.getValueFromIdByName(id, "managedCertificates"); + String managedCertificateName = ResourceManagerUtils.getValueFromIdByName(id, "managedCertificates"); if (managedCertificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); } - return this - .getWithResponse(resourceGroupName, environmentName, managedCertificateName, Context.NONE) + return this.getWithResponse(resourceGroupName, environmentName, managedCertificateName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } - String managedCertificateName = Utils.getValueFromIdByName(id, "managedCertificates"); + String managedCertificateName = ResourceManagerUtils.getValueFromIdByName(id, "managedCertificates"); if (managedCertificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); } return this.getWithResponse(resourceGroupName, environmentName, managedCertificateName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } - String managedCertificateName = Utils.getValueFromIdByName(id, "managedCertificates"); + String managedCertificateName = ResourceManagerUtils.getValueFromIdByName(id, "managedCertificates"); if (managedCertificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); } this.deleteWithResponse(resourceGroupName, environmentName, managedCertificateName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } - String managedCertificateName = Utils.getValueFromIdByName(id, "managedCertificates"); + String managedCertificateName = ResourceManagerUtils.getValueFromIdByName(id, "managedCertificates"); if (managedCertificateName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedCertificates'.", id))); } return this.deleteWithResponse(resourceGroupName, environmentName, managedCertificateName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsClientImpl.java index 9609f44b6edd8..f0b8491e223a8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsClientImpl.java @@ -26,93 +26,82 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentDiagnosticsClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentDiagnosticsClient. + */ public final class ManagedEnvironmentDiagnosticsClientImpl implements ManagedEnvironmentDiagnosticsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ManagedEnvironmentDiagnosticsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ManagedEnvironmentDiagnosticsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ManagedEnvironmentDiagnosticsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ManagedEnvironmentDiagnosticsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ManagedEnvironmentDiagnosticsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for ContainerAppsApiClientManagedEnvironmentDiagnostics to be used by the - * proxy service to perform REST calls. + * The interface defining all the services for ContainerAppsApiClientManagedEnvironmentDiagnostics to be used by + * the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ManagedEnvironmentDiagnosticsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/detectors") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/detectors") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listDetectors( - @HostParam("$host") String endpoint, + Mono> listDetectors(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/detectors/{detectorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/detectors/{detectorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getDetector( - @HostParam("$host") String endpoint, + Mono> getDetector(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("detectorName") String detectorName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("detectorName") String detectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a Managed Environment used to host container apps along with {@link Response} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDetectorsWithResponseAsync( - String resourceGroupName, String environmentName) { + private Mono> listDetectorsWithResponseAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -124,25 +113,16 @@ private Mono> listDetectorsWithResponseAsyn } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listDetectors( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.listDetectors(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -150,22 +130,18 @@ private Mono> listDetectorsWithResponseAsyn * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a Managed Environment used to host container apps along with {@link Response} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listDetectorsWithResponseAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> listDetectorsWithResponseAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -177,29 +153,22 @@ private Mono> listDetectorsWithResponseAsyn } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listDetectors( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context); + return service.listDetectors(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), accept, context); } /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a Managed Environment used to host container apps on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono listDetectorsAsync(String resourceGroupName, String environmentName) { @@ -209,29 +178,29 @@ private Mono listDetectorsAsync(String resourceGroup /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of diagnostics for a Managed Environment used to host container apps along with {@link - * Response}. + * @return the list of diagnostics for a Managed Environment used to host container apps along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listDetectorsWithResponse( - String resourceGroupName, String environmentName, Context context) { + public Response listDetectorsWithResponse(String resourceGroupName, + String environmentName, Context context) { return listDetectorsWithResponseAsync(resourceGroupName, environmentName, context).block(); } /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -246,9 +215,9 @@ public DiagnosticsCollectionInner listDetectors(String resourceGroupName, String /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. @@ -256,22 +225,18 @@ public DiagnosticsCollectionInner listDetectors(String resourceGroupName, String * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getDetectorWithResponseAsync( - String resourceGroupName, String environmentName, String detectorName) { + private Mono> getDetectorWithResponseAsync(String resourceGroupName, + String environmentName, String detectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -286,26 +251,16 @@ private Mono> getDetectorWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getDetector( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - detectorName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getDetector(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, detectorName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. @@ -314,22 +269,18 @@ private Mono> getDetectorWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getDetectorWithResponseAsync( - String resourceGroupName, String environmentName, String detectorName, Context context) { + private Mono> getDetectorWithResponseAsync(String resourceGroupName, + String environmentName, String detectorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -344,23 +295,15 @@ private Mono> getDetectorWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getDetector( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - detectorName, - this.client.getApiVersion(), - accept, - context); + return service.getDetector(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, detectorName, this.client.getApiVersion(), accept, context); } /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. @@ -368,20 +311,20 @@ private Mono> getDetectorWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getDetectorAsync( - String resourceGroupName, String environmentName, String detectorName) { + private Mono getDetectorAsync(String resourceGroupName, String environmentName, + String detectorName) { return getDetectorWithResponseAsync(resourceGroupName, environmentName, detectorName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. @@ -392,16 +335,16 @@ private Mono getDetectorAsync( * @return the diagnostics data for a Managed Environment used to host container apps along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getDetectorWithResponse( - String resourceGroupName, String environmentName, String detectorName, Context context) { + public Response getDetectorWithResponse(String resourceGroupName, String environmentName, + String detectorName, Context context) { return getDetectorWithResponseAsync(resourceGroupName, environmentName, detectorName, context).block(); } /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsImpl.java index 6ab273c405566..0a191bfe668a0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentDiagnosticsImpl.java @@ -22,22 +22,18 @@ public final class ManagedEnvironmentDiagnosticsImpl implements ManagedEnvironme private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ManagedEnvironmentDiagnosticsImpl( - ManagedEnvironmentDiagnosticsClient innerClient, + public ManagedEnvironmentDiagnosticsImpl(ManagedEnvironmentDiagnosticsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response listDetectorsWithResponse( - String resourceGroupName, String environmentName, Context context) { - Response inner = - this.serviceClient().listDetectorsWithResponse(resourceGroupName, environmentName, context); + public Response listDetectorsWithResponse(String resourceGroupName, String environmentName, + Context context) { + Response inner + = this.serviceClient().listDetectorsWithResponse(resourceGroupName, environmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DiagnosticsCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -53,15 +49,12 @@ public DiagnosticsCollection listDetectors(String resourceGroupName, String envi } } - public Response getDetectorWithResponse( - String resourceGroupName, String environmentName, String detectorName, Context context) { - Response inner = - this.serviceClient().getDetectorWithResponse(resourceGroupName, environmentName, detectorName, context); + public Response getDetectorWithResponse(String resourceGroupName, String environmentName, + String detectorName, Context context) { + Response inner + = this.serviceClient().getDetectorWithResponse(resourceGroupName, environmentName, detectorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DiagnosticsImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentImpl.java index 1d0877490ef02..9a5213f6b0bf2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentImpl.java @@ -9,6 +9,7 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentInner; +import com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration; import com.azure.resourcemanager.appcontainers.models.AppLogsConfiguration; import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration; import com.azure.resourcemanager.appcontainers.models.DaprConfiguration; @@ -17,6 +18,8 @@ import com.azure.resourcemanager.appcontainers.models.KedaConfiguration; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironment; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentPropertiesPeerAuthentication; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.OpenTelemetryConfiguration; import com.azure.resourcemanager.appcontainers.models.VnetConfiguration; import com.azure.resourcemanager.appcontainers.models.WorkloadProfile; import java.util.Collections; @@ -58,6 +61,10 @@ public String kind() { return this.innerModel().kind(); } + public ManagedServiceIdentity identity() { + return this.innerModel().identity(); + } + public SystemData systemData() { return this.innerModel().systemData(); } @@ -94,6 +101,14 @@ public AppLogsConfiguration appLogsConfiguration() { return this.innerModel().appLogsConfiguration(); } + public AppInsightsConfiguration appInsightsConfiguration() { + return this.innerModel().appInsightsConfiguration(); + } + + public OpenTelemetryConfiguration openTelemetryConfiguration() { + return this.innerModel().openTelemetryConfiguration(); + } + public Boolean zoneRedundant() { return this.innerModel().zoneRedundant(); } @@ -161,25 +176,19 @@ public ManagedEnvironmentImpl withExistingResourceGroup(String resourceGroupName } public ManagedEnvironment create() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironments() - .createOrUpdate(resourceGroupName, environmentName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getManagedEnvironments().createOrUpdate(resourceGroupName, + environmentName, this.innerModel(), Context.NONE); return this; } public ManagedEnvironment create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironments() - .createOrUpdate(resourceGroupName, environmentName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getManagedEnvironments().createOrUpdate(resourceGroupName, + environmentName, this.innerModel(), context); return this; } - ManagedEnvironmentImpl( - String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ManagedEnvironmentImpl(String name, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = new ManagedEnvironmentInner(); this.serviceManager = serviceManager; this.environmentName = name; @@ -190,56 +199,40 @@ public ManagedEnvironmentImpl update() { } public ManagedEnvironment apply() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironments() - .update(resourceGroupName, environmentName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getManagedEnvironments().update(resourceGroupName, + environmentName, this.innerModel(), Context.NONE); return this; } public ManagedEnvironment apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironments() - .update(resourceGroupName, environmentName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getManagedEnvironments().update(resourceGroupName, + environmentName, this.innerModel(), context); return this; } - ManagedEnvironmentImpl( - ManagedEnvironmentInner innerObject, + ManagedEnvironmentImpl(ManagedEnvironmentInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.environmentName = Utils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); } public ManagedEnvironment refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironments() - .getByResourceGroupWithResponse(resourceGroupName, environmentName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironments() + .getByResourceGroupWithResponse(resourceGroupName, environmentName, Context.NONE).getValue(); return this; } public ManagedEnvironment refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironments() - .getByResourceGroupWithResponse(resourceGroupName, environmentName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironments() + .getByResourceGroupWithResponse(resourceGroupName, environmentName, context).getValue(); return this; } public Response getAuthTokenWithResponse(Context context) { - return serviceManager - .managedEnvironments() - .getAuthTokenWithResponse(resourceGroupName, environmentName, context); + return serviceManager.managedEnvironments().getAuthTokenWithResponse(resourceGroupName, environmentName, + context); } public EnvironmentAuthToken getAuthToken() { @@ -266,6 +259,11 @@ public ManagedEnvironmentImpl withKind(String kind) { return this; } + public ManagedEnvironmentImpl withIdentity(ManagedServiceIdentity identity) { + this.innerModel().withIdentity(identity); + return this; + } + public ManagedEnvironmentImpl withDaprAIInstrumentationKey(String daprAIInstrumentationKey) { this.innerModel().withDaprAIInstrumentationKey(daprAIInstrumentationKey); return this; @@ -286,6 +284,17 @@ public ManagedEnvironmentImpl withAppLogsConfiguration(AppLogsConfiguration appL return this; } + public ManagedEnvironmentImpl withAppInsightsConfiguration(AppInsightsConfiguration appInsightsConfiguration) { + this.innerModel().withAppInsightsConfiguration(appInsightsConfiguration); + return this; + } + + public ManagedEnvironmentImpl + withOpenTelemetryConfiguration(OpenTelemetryConfiguration openTelemetryConfiguration) { + this.innerModel().withOpenTelemetryConfiguration(openTelemetryConfiguration); + return this; + } + public ManagedEnvironmentImpl withZoneRedundant(Boolean zoneRedundant) { this.innerModel().withZoneRedundant(zoneRedundant); return this; @@ -316,8 +325,8 @@ public ManagedEnvironmentImpl withInfrastructureResourceGroup(String infrastruct return this; } - public ManagedEnvironmentImpl withPeerAuthentication( - ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication) { + public ManagedEnvironmentImpl + withPeerAuthentication(ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication) { this.innerModel().withPeerAuthentication(peerAuthentication); return this; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStorageImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStorageImpl.java index dc5d0b9e1ee45..fb765ed16d538 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStorageImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStorageImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStorageInner; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentStorage; @@ -31,6 +32,10 @@ public ManagedEnvironmentStorageProperties properties() { return this.innerModel().properties(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String resourceGroupName() { return resourceGroupName; } @@ -49,36 +54,28 @@ private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager( private String storageName; - public ManagedEnvironmentStorageImpl withExistingManagedEnvironment( - String resourceGroupName, String environmentName) { + public ManagedEnvironmentStorageImpl withExistingManagedEnvironment(String resourceGroupName, + String environmentName) { this.resourceGroupName = resourceGroupName; this.environmentName = environmentName; return this; } public ManagedEnvironmentStorage create() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironmentsStorages() - .createOrUpdateWithResponse( - resourceGroupName, environmentName, storageName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironmentsStorages().createOrUpdateWithResponse( + resourceGroupName, environmentName, storageName, this.innerModel(), Context.NONE).getValue(); return this; } public ManagedEnvironmentStorage create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironmentsStorages() - .createOrUpdateWithResponse(resourceGroupName, environmentName, storageName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironmentsStorages() + .createOrUpdateWithResponse(resourceGroupName, environmentName, storageName, this.innerModel(), context) + .getValue(); return this; } - ManagedEnvironmentStorageImpl( - String name, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ManagedEnvironmentStorageImpl(String name, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = new ManagedEnvironmentStorageInner(); this.serviceManager = serviceManager; this.storageName = name; @@ -89,53 +86,36 @@ public ManagedEnvironmentStorageImpl update() { } public ManagedEnvironmentStorage apply() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironmentsStorages() - .createOrUpdateWithResponse( - resourceGroupName, environmentName, storageName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironmentsStorages().createOrUpdateWithResponse( + resourceGroupName, environmentName, storageName, this.innerModel(), Context.NONE).getValue(); return this; } public ManagedEnvironmentStorage apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironmentsStorages() - .createOrUpdateWithResponse(resourceGroupName, environmentName, storageName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironmentsStorages() + .createOrUpdateWithResponse(resourceGroupName, environmentName, storageName, this.innerModel(), context) + .getValue(); return this; } - ManagedEnvironmentStorageImpl( - ManagedEnvironmentStorageInner innerObject, + ManagedEnvironmentStorageImpl(ManagedEnvironmentStorageInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.environmentName = Utils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); - this.storageName = Utils.getValueFromIdByName(innerObject.id(), "storages"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.environmentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "managedEnvironments"); + this.storageName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "storages"); } public ManagedEnvironmentStorage refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironmentsStorages() - .getWithResponse(resourceGroupName, environmentName, storageName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironmentsStorages() + .getWithResponse(resourceGroupName, environmentName, storageName, Context.NONE).getValue(); return this; } public ManagedEnvironmentStorage refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getManagedEnvironmentsStorages() - .getWithResponse(resourceGroupName, environmentName, storageName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getManagedEnvironmentsStorages() + .getWithResponse(resourceGroupName, environmentName, storageName, context).getValue(); return this; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStoragesCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStoragesCollectionImpl.java index 6fa6eecef739a..4a37a11ebfed9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStoragesCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentStoragesCollectionImpl.java @@ -17,8 +17,7 @@ public final class ManagedEnvironmentStoragesCollectionImpl implements ManagedEn private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - ManagedEnvironmentStoragesCollectionImpl( - ManagedEnvironmentStoragesCollectionInner innerObject, + ManagedEnvironmentStoragesCollectionImpl(ManagedEnvironmentStoragesCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,8 @@ public final class ManagedEnvironmentStoragesCollectionImpl implements ManagedEn public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new ManagedEnvironmentStorageImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new ManagedEnvironmentStorageImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesClientImpl.java new file mode 100644 index 0000000000000..c9e4fdceb1640 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesClientImpl.java @@ -0,0 +1,283 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentUsagesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; +import com.azure.resourcemanager.appcontainers.models.ListUsagesResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentUsagesClient. + */ +public final class ManagedEnvironmentUsagesClientImpl implements ManagedEnvironmentUsagesClient { + /** + * The proxy service used to perform REST calls. + */ + private final ManagedEnvironmentUsagesService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of ManagedEnvironmentUsagesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + ManagedEnvironmentUsagesClientImpl(ContainerAppsApiClientImpl client) { + this.service = RestProxy.create(ManagedEnvironmentUsagesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientManagedEnvironmentUsages to be used by the + * proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface ManagedEnvironmentUsagesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/usages") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment along with {@link PagedResponse} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, String environmentName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment along with {@link PagedResponse} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, String environmentName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (environmentName == null) { + return Mono + .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, environmentName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String environmentName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, environmentName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName)); + } + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, environmentName, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesImpl.java new file mode 100644 index 0000000000000..d50d07f8e2b2a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentUsagesImpl.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.ManagedEnvironmentUsagesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentUsages; +import com.azure.resourcemanager.appcontainers.models.Usage; + +public final class ManagedEnvironmentUsagesImpl implements ManagedEnvironmentUsages { + private static final ClientLogger LOGGER = new ClientLogger(ManagedEnvironmentUsagesImpl.class); + + private final ManagedEnvironmentUsagesClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public ManagedEnvironmentUsagesImpl(ManagedEnvironmentUsagesClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String environmentName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new UsageImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String environmentName, Context context) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new UsageImpl(inner1, this.manager())); + } + + private ManagedEnvironmentUsagesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsClientImpl.java index 17ca799512912..d1d0a06f943df 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsClientImpl.java @@ -43,22 +43,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentsClient. + */ public final class ManagedEnvironmentsClientImpl implements ManagedEnvironmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ManagedEnvironmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ManagedEnvironmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ManagedEnvironmentsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(ManagedEnvironmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ManagedEnvironmentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -69,260 +75,190 @@ public final class ManagedEnvironmentsClientImpl implements ManagedEnvironmentsC @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ManagedEnvironmentsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/managedEnvironments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ManagedEnvironmentInner environmentEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono>> update( - @HostParam("$host") String endpoint, + Mono>> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ManagedEnvironmentInner environmentEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/getAuthtoken") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/getAuthtoken") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getAuthToken( - @HostParam("$host") String endpoint, + Mono> getAuthToken(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/workloadProfileStates") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/workloadProfileStates") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listWorkloadProfileStates( - @HostParam("$host") String endpoint, + Mono> listWorkloadProfileStates(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("environmentName") String environmentName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("environmentName") String environmentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) Mono> listWorkloadProfileStatesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -331,15 +267,15 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription as paginated response with {@link PagedIterable}. @@ -351,9 +287,9 @@ public PagedIterable list() { /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -367,29 +303,25 @@ public PagedIterable list(Context context) { /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the Managed Environments in a resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -397,55 +329,36 @@ private Mono> listByResourceGroupSinglePa } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the Managed Environments in a resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -454,29 +367,17 @@ private Mono> listByResourceGroupSinglePa final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -485,16 +386,15 @@ private Mono> listByResourceGroupSinglePa */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -504,16 +404,15 @@ private PagedFlux listByResourceGroupAsync(String resou */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -527,9 +426,9 @@ public PagedIterable listByResourceGroup(String resourc /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -544,31 +443,27 @@ public PagedIterable listByResourceGroup(String resourc /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String environmentName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -581,24 +476,16 @@ private Mono> getByResourceGroupWithResponseAs final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -606,22 +493,18 @@ private Mono> getByResourceGroupWithResponseAs * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -633,29 +516,22 @@ private Mono> getByResourceGroupWithResponseAs } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), accept, context); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the properties of a Managed Environment used to host container apps on successful completion of {@link - * Mono}. + * @return the properties of a Managed Environment used to host container apps on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getByResourceGroupAsync(String resourceGroupName, String environmentName) { @@ -665,9 +541,9 @@ private Mono getByResourceGroupAsync(String resourceGro /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -677,16 +553,16 @@ private Mono getByResourceGroupAsync(String resourceGro * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String environmentName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String environmentName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, environmentName, context).block(); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -701,32 +577,28 @@ public ManagedEnvironmentInner getByResourceGroup(String resourceGroupName, Stri /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an environment for hosting container apps along with {@link Response} on successful completion of {@link - * Mono}. + * @return an environment for hosting container apps along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, ManagedEnvironmentInner environmentEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -744,26 +616,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - environmentEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), environmentEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -771,26 +633,19 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an environment for hosting container apps along with {@link Response} on successful completion of {@link - * Mono}. + * @return an environment for hosting container apps along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -808,23 +663,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - environmentEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), environmentEnvelope, accept, context); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -836,23 +683,18 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ManagedEnvironmentInner> beginCreateOrUpdateAsync( String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedEnvironmentInner.class, - ManagedEnvironmentInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedEnvironmentInner.class, ManagedEnvironmentInner.class, + this.client.getContext()); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -864,28 +706,20 @@ private PollerFlux, ManagedEnvironmentInner> */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ManagedEnvironmentInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, + String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedEnvironmentInner.class, - ManagedEnvironmentInner.class, - context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedEnvironmentInner.class, ManagedEnvironmentInner.class, context); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -902,9 +736,9 @@ public SyncPoller, ManagedEnvironmentInner> /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -916,20 +750,17 @@ public SyncPoller, ManagedEnvironmentInner> */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ManagedEnvironmentInner> beginCreateOrUpdate( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, + String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context) + return this.beginCreateOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context) .getSyncPoller(); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -939,18 +770,17 @@ public SyncPoller, ManagedEnvironmentInner> * @return an environment for hosting container apps on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { - return beginCreateOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -961,21 +791,17 @@ private Mono createOrUpdateAsync( * @return an environment for hosting container apps on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, - Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -985,16 +811,16 @@ private Mono createOrUpdateAsync( * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedEnvironmentInner createOrUpdate( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { + public ManagedEnvironmentInner createOrUpdate(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope) { return createOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope).block(); } /** * Creates or updates a Managed Environment. - * - *

Creates or updates a Managed Environment used to host container apps. - * + * + * Creates or updates a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1005,19 +831,16 @@ public ManagedEnvironmentInner createOrUpdate( * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedEnvironmentInner createOrUpdate( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, - Context context) { + public ManagedEnvironmentInner createOrUpdate(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context) { return createOrUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context).block(); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1028,16 +851,12 @@ public ManagedEnvironmentInner createOrUpdate( @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1049,25 +868,16 @@ private Mono>> deleteWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -1077,19 +887,15 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String environmentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1101,22 +907,15 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), accept, context); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1127,17 +926,15 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String environmentName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, environmentName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -1147,20 +944,19 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String environmentName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String environmentName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, environmentName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1175,9 +971,9 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -1187,16 +983,16 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String environmentName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String environmentName, + Context context) { return this.beginDeleteAsync(resourceGroupName, environmentName, context).getSyncPoller(); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1206,16 +1002,15 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String environmentName) { - return beginDeleteAsync(resourceGroupName, environmentName) - .last() + return beginDeleteAsync(resourceGroupName, environmentName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -1226,16 +1021,15 @@ private Mono deleteAsync(String resourceGroupName, String environmentName) */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String environmentName, Context context) { - return beginDeleteAsync(resourceGroupName, environmentName, context) - .last() + return beginDeleteAsync(resourceGroupName, environmentName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1249,9 +1043,9 @@ public void delete(String resourceGroupName, String environmentName) { /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -1266,32 +1060,28 @@ public void delete(String resourceGroupName, String environmentName, Context con /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an environment for hosting container apps along with {@link Response} on successful completion of {@link - * Mono}. + * @return an environment for hosting container apps along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1309,26 +1099,16 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - environmentEnvelope, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), environmentEnvelope, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1336,26 +1116,19 @@ private Mono>> updateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an environment for hosting container apps along with {@link Response} on successful completion of {@link - * Mono}. + * @return an environment for hosting container apps along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, - Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1373,23 +1146,15 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - environmentEnvelope, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), environmentEnvelope, accept, context); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1401,23 +1166,18 @@ private Mono>> updateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ManagedEnvironmentInner> beginUpdateAsync( String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { - Mono>> mono = - updateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedEnvironmentInner.class, - ManagedEnvironmentInner.class, - this.client.getContext()); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedEnvironmentInner.class, ManagedEnvironmentInner.class, + this.client.getContext()); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1429,28 +1189,20 @@ private PollerFlux, ManagedEnvironmentInner> */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ManagedEnvironmentInner> beginUpdateAsync( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, + String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ManagedEnvironmentInner.class, - ManagedEnvironmentInner.class, - context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, environmentName, environmentEnvelope, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ManagedEnvironmentInner.class, ManagedEnvironmentInner.class, context); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1460,16 +1212,16 @@ private PollerFlux, ManagedEnvironmentInner> * @return the {@link SyncPoller} for polling of an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ManagedEnvironmentInner> beginUpdate( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { + public SyncPoller, ManagedEnvironmentInner> + beginUpdate(String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { return this.beginUpdateAsync(resourceGroupName, environmentName, environmentEnvelope).getSyncPoller(); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1481,18 +1233,16 @@ public SyncPoller, ManagedEnvironmentInner> */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ManagedEnvironmentInner> beginUpdate( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, + String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope, Context context) { return this.beginUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context).getSyncPoller(); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1502,18 +1252,17 @@ public SyncPoller, ManagedEnvironmentInner> * @return an environment for hosting container apps on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { - return beginUpdateAsync(resourceGroupName, environmentName, environmentEnvelope) - .last() + private Mono updateAsync(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope) { + return beginUpdateAsync(resourceGroupName, environmentName, environmentEnvelope).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1524,21 +1273,17 @@ private Mono updateAsync( * @return an environment for hosting container apps on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, - Context context) { - return beginUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context) - .last() + private Mono updateAsync(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context) { + return beginUpdateAsync(resourceGroupName, environmentName, environmentEnvelope, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1548,16 +1293,16 @@ private Mono updateAsync( * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedEnvironmentInner update( - String resourceGroupName, String environmentName, ManagedEnvironmentInner environmentEnvelope) { + public ManagedEnvironmentInner update(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope) { return updateAsync(resourceGroupName, environmentName, environmentEnvelope).block(); } /** * Update Managed Environment's properties. - * - *

Patches a Managed Environment using JSON Merge Patch. - * + * + * Patches a Managed Environment using JSON Merge Patch. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param environmentEnvelope Configuration details of the Environment. @@ -1568,19 +1313,16 @@ public ManagedEnvironmentInner update( * @return an environment for hosting container apps. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedEnvironmentInner update( - String resourceGroupName, - String environmentName, - ManagedEnvironmentInner environmentEnvelope, - Context context) { + public ManagedEnvironmentInner update(String resourceGroupName, String environmentName, + ManagedEnvironmentInner environmentEnvelope, Context context) { return updateAsync(resourceGroupName, environmentName, environmentEnvelope, context).block(); } /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1589,19 +1331,15 @@ public ManagedEnvironmentInner update( * @return environment Auth Token along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthTokenWithResponseAsync( - String resourceGroupName, String environmentName) { + private Mono> getAuthTokenWithResponseAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1613,25 +1351,16 @@ private Mono> getAuthTokenWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getAuthToken( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getAuthToken(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1641,19 +1370,15 @@ private Mono> getAuthTokenWithResponseAsync( * @return environment Auth Token along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthTokenWithResponseAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> getAuthTokenWithResponseAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1665,22 +1390,15 @@ private Mono> getAuthTokenWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getAuthToken( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context); + return service.getAuthToken(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), accept, context); } /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1696,9 +1414,9 @@ private Mono getAuthTokenAsync(String resourceGroupNa /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1708,16 +1426,16 @@ private Mono getAuthTokenAsync(String resourceGroupNa * @return environment Auth Token along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getAuthTokenWithResponse( - String resourceGroupName, String environmentName, Context context) { + public Response getAuthTokenWithResponse(String resourceGroupName, + String environmentName, Context context) { return getAuthTokenWithResponseAsync(resourceGroupName, environmentName, context).block(); } /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1732,31 +1450,27 @@ public EnvironmentAuthTokenInner getAuthToken(String resourceGroupName, String e /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all workload Profile States for a Managed Environment along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWorkloadProfileStatesSinglePageAsync( - String resourceGroupName, String environmentName) { + private Mono> + listWorkloadProfileStatesSinglePageAsync(String resourceGroupName, String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1769,33 +1483,18 @@ private Mono> listWorkloadProfileState final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listWorkloadProfileStates( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - environmentName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listWorkloadProfileStates(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), environmentName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1803,22 +1502,18 @@ private Mono> listWorkloadProfileState * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all workload Profile States for a Managed Environment along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWorkloadProfileStatesSinglePageAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> + listWorkloadProfileStatesSinglePageAsync(String resourceGroupName, String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1831,30 +1526,17 @@ private Mono> listWorkloadProfileState final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listWorkloadProfileStates( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - environmentName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listWorkloadProfileStates(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), environmentName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1863,18 +1545,17 @@ private Mono> listWorkloadProfileState * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listWorkloadProfileStatesAsync( - String resourceGroupName, String environmentName) { - return new PagedFlux<>( - () -> listWorkloadProfileStatesSinglePageAsync(resourceGroupName, environmentName), + private PagedFlux listWorkloadProfileStatesAsync(String resourceGroupName, + String environmentName) { + return new PagedFlux<>(() -> listWorkloadProfileStatesSinglePageAsync(resourceGroupName, environmentName), nextLink -> listWorkloadProfileStatesNextSinglePageAsync(nextLink)); } /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1884,8 +1565,8 @@ private PagedFlux listWorkloadProfileStatesAsync( * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listWorkloadProfileStatesAsync( - String resourceGroupName, String environmentName, Context context) { + private PagedFlux listWorkloadProfileStatesAsync(String resourceGroupName, + String environmentName, Context context) { return new PagedFlux<>( () -> listWorkloadProfileStatesSinglePageAsync(resourceGroupName, environmentName, context), nextLink -> listWorkloadProfileStatesNextSinglePageAsync(nextLink, context)); @@ -1893,9 +1574,9 @@ private PagedFlux listWorkloadProfileStatesAsync( /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1904,16 +1585,16 @@ private PagedFlux listWorkloadProfileStatesAsync( * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName) { + public PagedIterable listWorkloadProfileStates(String resourceGroupName, + String environmentName) { return new PagedIterable<>(listWorkloadProfileStatesAsync(resourceGroupName, environmentName)); } /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. @@ -1923,16 +1604,17 @@ public PagedIterable listWorkloadProfileStates( * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName, Context context) { + public PagedIterable listWorkloadProfileStates(String resourceGroupName, + String environmentName, Context context) { return new PagedIterable<>(listWorkloadProfileStatesAsync(resourceGroupName, environmentName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1944,32 +1626,24 @@ private Mono> listBySubscriptionNextSingl return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -1977,37 +1651,28 @@ private Mono> listBySubscriptionNextSingl * @return collection of Environments along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -2019,32 +1684,24 @@ private Mono> listByResourceGroupNextSing return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -2052,107 +1709,80 @@ private Mono> listByResourceGroupNextSing * @return collection of Environments along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of workloadProfileStates along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of workloadProfileStates along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWorkloadProfileStatesNextSinglePageAsync( - String nextLink) { + private Mono> + listWorkloadProfileStatesNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listWorkloadProfileStatesNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of workloadProfileStates along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return collection of workloadProfileStates along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWorkloadProfileStatesNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listWorkloadProfileStatesNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listWorkloadProfileStatesNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listWorkloadProfileStatesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsClientImpl.java index e13f64a584cfb..deb3dae4382c6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsClientImpl.java @@ -25,26 +25,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentsDiagnosticsClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentsDiagnosticsClient. + */ public final class ManagedEnvironmentsDiagnosticsClientImpl implements ManagedEnvironmentsDiagnosticsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ManagedEnvironmentsDiagnosticsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ManagedEnvironmentsDiagnosticsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ManagedEnvironmentsDiagnosticsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ManagedEnvironmentsDiagnosticsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ManagedEnvironmentsDiagnosticsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -55,48 +57,40 @@ public final class ManagedEnvironmentsDiagnosticsClientImpl implements ManagedEn @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ManagedEnvironmentsDiagnosticsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/detectorProperties/rootApi/") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/detectorProperties/rootApi/") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> getRoot( - @HostParam("$host") String endpoint, + Mono> getRoot(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRootWithResponseAsync( - String resourceGroupName, String environmentName) { + private Mono> getRootWithResponseAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -108,25 +102,16 @@ private Mono> getRootWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getRoot( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getRoot(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -134,22 +119,18 @@ private Mono> getRootWithResponseAsync( * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getRootWithResponseAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> getRootWithResponseAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -161,29 +142,22 @@ private Mono> getRootWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getRoot( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context); + return service.getRoot(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), accept, context); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the properties of a Managed Environment used to host container apps on successful completion of {@link - * Mono}. + * @return the properties of a Managed Environment used to host container apps on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getRootAsync(String resourceGroupName, String environmentName) { @@ -193,9 +167,9 @@ private Mono getRootAsync(String resourceGroupName, Str /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -205,16 +179,16 @@ private Mono getRootAsync(String resourceGroupName, Str * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getRootWithResponse( - String resourceGroupName, String environmentName, Context context) { + public Response getRootWithResponse(String resourceGroupName, String environmentName, + Context context) { return getRootWithResponseAsync(resourceGroupName, environmentName, context).block(); } /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsImpl.java index 7a4187d4cd521..544062c66a9d5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsDiagnosticsImpl.java @@ -20,22 +20,18 @@ public final class ManagedEnvironmentsDiagnosticsImpl implements ManagedEnvironm private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ManagedEnvironmentsDiagnosticsImpl( - ManagedEnvironmentsDiagnosticsClient innerClient, + public ManagedEnvironmentsDiagnosticsImpl(ManagedEnvironmentsDiagnosticsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getRootWithResponse( - String resourceGroupName, String environmentName, Context context) { - Response inner = - this.serviceClient().getRootWithResponse(resourceGroupName, environmentName, context); + public Response getRootWithResponse(String resourceGroupName, String environmentName, + Context context) { + Response inner + = this.serviceClient().getRootWithResponse(resourceGroupName, environmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ManagedEnvironmentImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsImpl.java index 80f706e9a89c4..c140973da287c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsImpl.java @@ -25,8 +25,7 @@ public final class ManagedEnvironmentsImpl implements ManagedEnvironments { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ManagedEnvironmentsImpl( - ManagedEnvironmentsClient innerClient, + public ManagedEnvironmentsImpl(ManagedEnvironmentsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -34,34 +33,31 @@ public ManagedEnvironmentsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ManagedEnvironmentImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String environmentName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, environmentName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String environmentName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, environmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ManagedEnvironmentImpl(inner.getValue(), this.manager())); } else { return null; @@ -85,15 +81,12 @@ public void delete(String resourceGroupName, String environmentName, Context con this.serviceClient().delete(resourceGroupName, environmentName, context); } - public Response getAuthTokenWithResponse( - String resourceGroupName, String environmentName, Context context) { - Response inner = - this.serviceClient().getAuthTokenWithResponse(resourceGroupName, environmentName, context); + public Response getAuthTokenWithResponse(String resourceGroupName, String environmentName, + Context context) { + Response inner + = this.serviceClient().getAuthTokenWithResponse(resourceGroupName, environmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new EnvironmentAuthTokenImpl(inner.getValue(), this.manager())); } else { return null; @@ -109,100 +102,72 @@ public EnvironmentAuthToken getAuthToken(String resourceGroupName, String enviro } } - public PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName) { - PagedIterable inner = - this.serviceClient().listWorkloadProfileStates(resourceGroupName, environmentName); - return Utils.mapPage(inner, inner1 -> new WorkloadProfileStatesImpl(inner1, this.manager())); + public PagedIterable listWorkloadProfileStates(String resourceGroupName, + String environmentName) { + PagedIterable inner + = this.serviceClient().listWorkloadProfileStates(resourceGroupName, environmentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkloadProfileStatesImpl(inner1, this.manager())); } - public PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName, Context context) { - PagedIterable inner = - this.serviceClient().listWorkloadProfileStates(resourceGroupName, environmentName, context); - return Utils.mapPage(inner, inner1 -> new WorkloadProfileStatesImpl(inner1, this.manager())); + public PagedIterable listWorkloadProfileStates(String resourceGroupName, + String environmentName, Context context) { + PagedIterable inner + = this.serviceClient().listWorkloadProfileStates(resourceGroupName, environmentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkloadProfileStatesImpl(inner1, this.manager())); } public ManagedEnvironment getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, environmentName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, environmentName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } this.delete(resourceGroupName, environmentName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); } this.delete(resourceGroupName, environmentName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesClientImpl.java index c92a99ffc179a..7507b73c6cb5d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesClientImpl.java @@ -29,24 +29,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ManagedEnvironmentsStoragesClient. */ +/** + * An instance of this class provides access to all the operations defined in ManagedEnvironmentsStoragesClient. + */ public final class ManagedEnvironmentsStoragesClientImpl implements ManagedEnvironmentsStoragesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ManagedEnvironmentsStoragesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ManagedEnvironmentsStoragesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ManagedEnvironmentsStoragesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy - .create( - ManagedEnvironmentsStoragesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ManagedEnvironmentsStoragesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -57,92 +61,70 @@ public final class ManagedEnvironmentsStoragesClientImpl implements ManagedEnvir @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ManagedEnvironmentsStoragesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages/{storageName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages/{storageName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("storageName") String storageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("storageName") String storageName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages/{storageName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages/{storageName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("storageName") String storageName, + @PathParam("environmentName") String environmentName, @PathParam("storageName") String storageName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") ManagedEnvironmentStorageInner storageEnvelope, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages/{storageName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/storages/{storageName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @PathParam("storageName") String storageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("environmentName") String environmentName, @PathParam("storageName") String storageName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all storages for a managedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return all storages for a managedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String resourceGroupName, String environmentName) { + private Mono> listWithResponseAsync(String resourceGroupName, + String environmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -154,46 +136,33 @@ private Mono> listWithRespon } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all storages for a managedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return all storages for a managedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String resourceGroupName, String environmentName, Context context) { + private Mono> listWithResponseAsync(String resourceGroupName, + String environmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -205,20 +174,13 @@ private Mono> listWithRespon } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - accept, - context); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, this.client.getApiVersion(), accept, context); } /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -227,15 +189,15 @@ private Mono> listWithRespon * @return all storages for a managedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listAsync( - String resourceGroupName, String environmentName) { + private Mono listAsync(String resourceGroupName, + String environmentName) { return listWithResponseAsync(resourceGroupName, environmentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. @@ -245,14 +207,14 @@ private Mono listAsync( * @return all storages for a managedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listWithResponse( - String resourceGroupName, String environmentName, Context context) { + public Response listWithResponse(String resourceGroupName, + String environmentName, Context context) { return listWithResponseAsync(resourceGroupName, environmentName, context).block(); } /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -267,7 +229,7 @@ public ManagedEnvironmentStoragesCollectionInner list(String resourceGroupName, /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -277,19 +239,15 @@ public ManagedEnvironmentStoragesCollectionInner list(String resourceGroupName, * @return storage for a managedEnvironment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String storageName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String environmentName, String storageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -304,24 +262,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - storageName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, storageName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -332,19 +280,15 @@ private Mono> getWithResponseAsync( * @return storage for a managedEnvironment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String environmentName, String storageName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String environmentName, String storageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -359,21 +303,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - storageName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, storageName, this.client.getApiVersion(), accept, context); } /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -383,15 +319,15 @@ private Mono> getWithResponseAsync( * @return storage for a managedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String environmentName, String storageName) { + private Mono getAsync(String resourceGroupName, String environmentName, + String storageName) { return getWithResponseAsync(resourceGroupName, environmentName, storageName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -402,14 +338,14 @@ private Mono getAsync( * @return storage for a managedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context) { + public Response getWithResponse(String resourceGroupName, String environmentName, + String storageName, Context context) { return getWithResponseAsync(resourceGroupName, environmentName, storageName, context).block(); } /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -425,7 +361,7 @@ public ManagedEnvironmentStorageInner get(String resourceGroupName, String envir /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -433,26 +369,19 @@ public ManagedEnvironmentStorageInner get(String resourceGroupName, String envir * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return storage resource for managedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return storage resource for managedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String storageName, - ManagedEnvironmentStorageInner storageEnvelope) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String storageName, ManagedEnvironmentStorageInner storageEnvelope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -473,25 +402,15 @@ private Mono> createOrUpdateWithRespons } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - storageName, - this.client.getApiVersion(), - storageEnvelope, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, storageName, this.client.getApiVersion(), storageEnvelope, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -500,27 +419,19 @@ private Mono> createOrUpdateWithRespons * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return storage resource for managedEnvironment along with {@link Response} on successful completion of {@link - * Mono}. + * @return storage resource for managedEnvironment along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String environmentName, - String storageName, - ManagedEnvironmentStorageInner storageEnvelope, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String environmentName, String storageName, ManagedEnvironmentStorageInner storageEnvelope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -541,22 +452,13 @@ private Mono> createOrUpdateWithRespons } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - storageName, - this.client.getApiVersion(), - storageEnvelope, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, storageName, this.client.getApiVersion(), storageEnvelope, accept, context); } /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -567,18 +469,15 @@ private Mono> createOrUpdateWithRespons * @return storage resource for managedEnvironment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String environmentName, - String storageName, - ManagedEnvironmentStorageInner storageEnvelope) { + private Mono createOrUpdateAsync(String resourceGroupName, String environmentName, + String storageName, ManagedEnvironmentStorageInner storageEnvelope) { return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, storageName, storageEnvelope) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -590,20 +489,15 @@ private Mono createOrUpdateAsync( * @return storage resource for managedEnvironment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String storageName, - ManagedEnvironmentStorageInner storageEnvelope, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, environmentName, storageName, storageEnvelope, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, + String environmentName, String storageName, ManagedEnvironmentStorageInner storageEnvelope, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, environmentName, storageName, storageEnvelope, + context).block(); } /** * Create or update storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -614,19 +508,15 @@ public Response createOrUpdateWithResponse( * @return storage resource for managedEnvironment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ManagedEnvironmentStorageInner createOrUpdate( - String resourceGroupName, - String environmentName, - String storageName, - ManagedEnvironmentStorageInner storageEnvelope) { - return createOrUpdateWithResponse( - resourceGroupName, environmentName, storageName, storageEnvelope, Context.NONE) - .getValue(); + public ManagedEnvironmentStorageInner createOrUpdate(String resourceGroupName, String environmentName, + String storageName, ManagedEnvironmentStorageInner storageEnvelope) { + return createOrUpdateWithResponse(resourceGroupName, environmentName, storageName, storageEnvelope, + Context.NONE).getValue(); } /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -636,19 +526,15 @@ public ManagedEnvironmentStorageInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String storageName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String storageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -663,24 +549,14 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - storageName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, storageName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -691,19 +567,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String environmentName, String storageName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String environmentName, + String storageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -718,21 +590,13 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - storageName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + environmentName, storageName, this.client.getApiVersion(), accept, context); } /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -749,7 +613,7 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. @@ -760,14 +624,14 @@ private Mono deleteAsync(String resourceGroupName, String environmentName, * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, String storageName, + Context context) { return deleteWithResponseAsync(resourceGroupName, environmentName, storageName, context).block(); } /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesImpl.java index dfcc84bf5607f..0912a5938a1ba 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ManagedEnvironmentsStoragesImpl.java @@ -22,22 +22,18 @@ public final class ManagedEnvironmentsStoragesImpl implements ManagedEnvironment private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ManagedEnvironmentsStoragesImpl( - ManagedEnvironmentsStoragesClient innerClient, + public ManagedEnvironmentsStoragesImpl(ManagedEnvironmentsStoragesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response listWithResponse( - String resourceGroupName, String environmentName, Context context) { - Response inner = - this.serviceClient().listWithResponse(resourceGroupName, environmentName, context); + public Response listWithResponse(String resourceGroupName, + String environmentName, Context context) { + Response inner + = this.serviceClient().listWithResponse(resourceGroupName, environmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ManagedEnvironmentStoragesCollectionImpl(inner.getValue(), this.manager())); } else { return null; @@ -53,15 +49,12 @@ public ManagedEnvironmentStoragesCollection list(String resourceGroupName, Strin } } - public Response getWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, environmentName, storageName, context); + public Response getWithResponse(String resourceGroupName, String environmentName, + String storageName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, environmentName, storageName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ManagedEnvironmentStorageImpl(inner.getValue(), this.manager())); } else { return null; @@ -69,8 +62,8 @@ public Response getWithResponse( } public ManagedEnvironmentStorage get(String resourceGroupName, String environmentName, String storageName) { - ManagedEnvironmentStorageInner inner = - this.serviceClient().get(resourceGroupName, environmentName, storageName); + ManagedEnvironmentStorageInner inner + = this.serviceClient().get(resourceGroupName, environmentName, storageName); if (inner != null) { return new ManagedEnvironmentStorageImpl(inner, this.manager()); } else { @@ -78,8 +71,8 @@ public ManagedEnvironmentStorage get(String resourceGroupName, String environmen } } - public Response deleteWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String environmentName, String storageName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, environmentName, storageName, context); } @@ -88,113 +81,77 @@ public void delete(String resourceGroupName, String environmentName, String stor } public ManagedEnvironmentStorage getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } return this.getWithResponse(resourceGroupName, environmentName, storageName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } return this.getWithResponse(resourceGroupName, environmentName, storageName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } this.deleteWithResponse(resourceGroupName, environmentName, storageName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String environmentName = Utils.getValueFromIdByName(id, "managedEnvironments"); + String environmentName = ResourceManagerUtils.getValueFromIdByName(id, "managedEnvironments"); if (environmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); - } - String storageName = Utils.getValueFromIdByName(id, "storages"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'managedEnvironments'.", id))); + } + String storageName = ResourceManagerUtils.getValueFromIdByName(id, "storages"); if (storageName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'storages'.", id))); } return this.deleteWithResponse(resourceGroupName, environmentName, storageName, context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesClientImpl.java index 0a6fb3fc357bf..404efd5e29572 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesClientImpl.java @@ -27,22 +27,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in NamespacesClient. */ +/** + * An instance of this class provides access to all the operations defined in NamespacesClient. + */ public final class NamespacesClientImpl implements NamespacesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final NamespacesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of NamespacesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ NamespacesClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(NamespacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(NamespacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -53,27 +59,23 @@ public final class NamespacesClientImpl implements NamespacesClient { @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface NamespacesService { - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/checkNameAvailability") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/managedEnvironments/{environmentName}/checkNameAvailability") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> checkNameAvailability( - @HostParam("$host") String endpoint, + Mono> checkNameAvailability(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("environmentName") String environmentName, - @QueryParam("api-version") String apiVersion, + @PathParam("environmentName") String environmentName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") CheckNameAvailabilityRequest checkNameAvailabilityRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. @@ -86,16 +88,12 @@ Mono> checkNameAvailability( private Mono> checkNameAvailabilityWithResponseAsync( String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -106,35 +104,24 @@ private Mono> checkNameAvailability .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (checkNameAvailabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter checkNameAvailabilityRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter checkNameAvailabilityRequest is required and cannot be null.")); } else { checkNameAvailabilityRequest.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - checkNameAvailabilityRequest, - accept, - context)) + .withContext(context -> service.checkNameAvailability(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, environmentName, this.client.getApiVersion(), + checkNameAvailabilityRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. @@ -146,21 +133,15 @@ private Mono> checkNameAvailability */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> checkNameAvailabilityWithResponseAsync( - String resourceGroupName, - String environmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, + String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -171,32 +152,23 @@ private Mono> checkNameAvailability .error(new IllegalArgumentException("Parameter environmentName is required and cannot be null.")); } if (checkNameAvailabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter checkNameAvailabilityRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter checkNameAvailabilityRequest is required and cannot be null.")); } else { checkNameAvailabilityRequest.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - environmentName, - this.client.getApiVersion(), - checkNameAvailabilityRequest, - accept, - context); + return service.checkNameAvailability(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, environmentName, this.client.getApiVersion(), checkNameAvailabilityRequest, accept, + context); } /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. @@ -206,17 +178,17 @@ private Mono> checkNameAvailability * @return the check availability result on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono checkNameAvailabilityAsync( - String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + private Mono checkNameAvailabilityAsync(String resourceGroupName, + String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { return checkNameAvailabilityWithResponseAsync(resourceGroupName, environmentName, checkNameAvailabilityRequest) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. @@ -227,21 +199,17 @@ private Mono checkNameAvailabilityAsync( * @return the check availability result along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String environmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context) { - return checkNameAvailabilityWithResponseAsync( - resourceGroupName, environmentName, checkNameAvailabilityRequest, context) - .block(); + public Response checkNameAvailabilityWithResponse(String resourceGroupName, + String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { + return checkNameAvailabilityWithResponseAsync(resourceGroupName, environmentName, checkNameAvailabilityRequest, + context).block(); } /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. @@ -251,10 +219,9 @@ public Response checkNameAvailabilityWithRes * @return the check availability result. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CheckNameAvailabilityResponseInner checkNameAvailability( - String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { - return checkNameAvailabilityWithResponse( - resourceGroupName, environmentName, checkNameAvailabilityRequest, Context.NONE) - .getValue(); + public CheckNameAvailabilityResponseInner checkNameAvailability(String resourceGroupName, String environmentName, + CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + return checkNameAvailabilityWithResponse(resourceGroupName, environmentName, checkNameAvailabilityRequest, + Context.NONE).getValue(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesImpl.java index 167621f13759d..91183b68a218f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/NamespacesImpl.java @@ -21,39 +21,28 @@ public final class NamespacesImpl implements Namespaces { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public NamespacesImpl( - NamespacesClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + public NamespacesImpl(NamespacesClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String environmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context) { - Response inner = - this - .serviceClient() - .checkNameAvailabilityWithResponse( - resourceGroupName, environmentName, checkNameAvailabilityRequest, context); + public Response checkNameAvailabilityWithResponse(String resourceGroupName, + String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context) { + Response inner = this.serviceClient().checkNameAvailabilityWithResponse( + resourceGroupName, environmentName, checkNameAvailabilityRequest, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CheckNameAvailabilityResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public CheckNameAvailabilityResponse checkNameAvailability( - String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest) { - CheckNameAvailabilityResponseInner inner = - this - .serviceClient() - .checkNameAvailability(resourceGroupName, environmentName, checkNameAvailabilityRequest); + public CheckNameAvailabilityResponse checkNameAvailability(String resourceGroupName, String environmentName, + CheckNameAvailabilityRequest checkNameAvailabilityRequest) { + CheckNameAvailabilityResponseInner inner = this.serviceClient().checkNameAvailability(resourceGroupName, + environmentName, checkNameAvailabilityRequest); if (inner != null) { return new CheckNameAvailabilityResponseImpl(inner, this.manager()); } else { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationDetailImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationDetailImpl.java index 1d78896f2723f..cd2fca311b4b5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationDetailImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationDetailImpl.java @@ -13,8 +13,7 @@ public final class OperationDetailImpl implements OperationDetail { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - OperationDetailImpl( - OperationDetailInner innerObject, + OperationDetailImpl(OperationDetailInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsClientImpl.java index 6f299e42cfedc..6d5969c5049e4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,95 +62,70 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.App/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Lists all of the available RP operations. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return available operations of the service along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return available operations of the service along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all of the available RP operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return available operations of the service along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return available operations of the service along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all of the available RP operations. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return available operations of the service as paginated response with {@link PagedFlux}. @@ -156,7 +137,7 @@ private PagedFlux listAsync() { /** * Lists all of the available RP operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -165,13 +146,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists all of the available RP operations. - * + * * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return available operations of the service as paginated response with {@link PagedIterable}. @@ -183,7 +164,7 @@ public PagedIterable list() { /** * Lists all of the available RP operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. @@ -197,14 +178,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return available operations of the service along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return available operations of the service along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -212,37 +194,28 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return available operations of the service along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return available operations of the service along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -250,23 +223,13 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsImpl.java index fb3cfe7ac5857..7cbc6577e929e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/OperationsImpl.java @@ -19,20 +19,20 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + public OperationsImpl(OperationsClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationDetailImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationDetailImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationDetailImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationDetailImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaCollectionImpl.java index fcb6c32ab7899..e9bba45d900ef 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaCollectionImpl.java @@ -17,8 +17,7 @@ public final class ReplicaCollectionImpl implements ReplicaCollection { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - ReplicaCollectionImpl( - ReplicaCollectionInner innerObject, + ReplicaCollectionImpl(ReplicaCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,9 +26,8 @@ public final class ReplicaCollectionImpl implements ReplicaCollection { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner.stream().map(inner1 -> new ReplicaImpl(inner1, this.manager())).collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new ReplicaImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaImpl.java index 89ccdf824e2c5..927df01ea0d92 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ReplicaImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.ReplicaInner; import com.azure.resourcemanager.appcontainers.models.ContainerAppReplicaRunningState; import com.azure.resourcemanager.appcontainers.models.Replica; @@ -17,8 +18,8 @@ public final class ReplicaImpl implements Replica { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - ReplicaImpl( - ReplicaInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + ReplicaImpl(ReplicaInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -35,6 +36,10 @@ public String type() { return this.innerModel().type(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public OffsetDateTime createdTime() { return this.innerModel().createdTime(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceManagerUtils.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..94a651ce19fad --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersClientImpl.java index 02ade665b7195..22debf25d2728 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersClientImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersClientImpl.java @@ -11,6 +11,7 @@ import com.azure.core.annotation.Host; import com.azure.core.annotation.HostParam; import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; import com.azure.core.annotation.QueryParam; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceInterface; @@ -25,22 +26,28 @@ import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ResourceProvidersClient. */ +/** + * An instance of this class provides access to all the operations defined in ResourceProvidersClient. + */ public final class ResourceProvidersClientImpl implements ResourceProvidersClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ResourceProvidersService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ContainerAppsApiClientImpl client; /** * Initializes an instance of ResourceProvidersClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ResourceProvidersClientImpl(ContainerAppsApiClientImpl client) { - this.service = - RestProxy.create(ResourceProvidersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ResourceProvidersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -51,25 +58,28 @@ public final class ResourceProvidersClientImpl implements ResourceProvidersClien @Host("{$host}") @ServiceInterface(name = "ContainerAppsApiClie") public interface ResourceProvidersService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/executions/{jobExecutionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.App/jobs/{jobName}/executions/{jobExecutionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) - Mono> jobExecution( - @HostParam("$host") String endpoint, + Mono> jobExecution(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("jobName") String jobName, - @PathParam("jobExecutionName") String jobExecutionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("jobName") String jobName, + @PathParam("jobExecutionName") String jobExecutionName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.App/getCustomDomainVerificationId") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> getCustomDomainVerificationId(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); } /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -79,19 +89,15 @@ Mono> jobExecution( * @return details of a single job execution along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> jobExecutionWithResponseAsync( - String resourceGroupName, String jobName, String jobExecutionName) { + private Mono> jobExecutionWithResponseAsync(String resourceGroupName, String jobName, + String jobExecutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -106,24 +112,14 @@ private Mono> jobExecutionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .jobExecution( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - jobExecutionName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.jobExecution(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, jobName, jobExecutionName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -134,19 +130,15 @@ private Mono> jobExecutionWithResponseAsync( * @return details of a single job execution along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> jobExecutionWithResponseAsync( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { + private Mono> jobExecutionWithResponseAsync(String resourceGroupName, String jobName, + String jobExecutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -161,21 +153,13 @@ private Mono> jobExecutionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .jobExecution( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - jobName, - jobExecutionName, - this.client.getApiVersion(), - accept, - context); + return service.jobExecution(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + jobName, jobExecutionName, this.client.getApiVersion(), accept, context); } /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -185,15 +169,15 @@ private Mono> jobExecutionWithResponseAsync( * @return details of a single job execution on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono jobExecutionAsync( - String resourceGroupName, String jobName, String jobExecutionName) { + private Mono jobExecutionAsync(String resourceGroupName, String jobName, + String jobExecutionName) { return jobExecutionWithResponseAsync(resourceGroupName, jobName, jobExecutionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -204,14 +188,14 @@ private Mono jobExecutionAsync( * @return details of a single job execution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response jobExecutionWithResponse( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { + public Response jobExecutionWithResponse(String resourceGroupName, String jobName, + String jobExecutionName, Context context) { return jobExecutionWithResponseAsync(resourceGroupName, jobName, jobExecutionName, context).block(); } /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. @@ -224,4 +208,94 @@ public Response jobExecutionWithResponse( public JobExecutionInner jobExecution(String resourceGroupName, String jobName, String jobExecutionName) { return jobExecutionWithResponse(resourceGroupName, jobName, jobExecutionName, Context.NONE).getValue(); } + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getCustomDomainVerificationIdWithResponseAsync() { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getCustomDomainVerificationId(this.client.getEndpoint(), + this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getCustomDomainVerificationIdWithResponseAsync(Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getCustomDomainVerificationId(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context); + } + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getCustomDomainVerificationIdAsync() { + return getCustomDomainVerificationIdWithResponseAsync().flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getCustomDomainVerificationIdWithResponse(Context context) { + return getCustomDomainVerificationIdWithResponseAsync(context).block(); + } + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public String getCustomDomainVerificationId() { + return getCustomDomainVerificationIdWithResponse(Context.NONE).getValue(); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersImpl.java index fcec09b561bf8..cacbfd4555064 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/ResourceProvidersImpl.java @@ -20,22 +20,18 @@ public final class ResourceProvidersImpl implements ResourceProviders { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - public ResourceProvidersImpl( - ResourceProvidersClient innerClient, + public ResourceProvidersImpl(ResourceProvidersClient innerClient, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response jobExecutionWithResponse( - String resourceGroupName, String jobName, String jobExecutionName, Context context) { - Response inner = - this.serviceClient().jobExecutionWithResponse(resourceGroupName, jobName, jobExecutionName, context); + public Response jobExecutionWithResponse(String resourceGroupName, String jobName, + String jobExecutionName, Context context) { + Response inner + = this.serviceClient().jobExecutionWithResponse(resourceGroupName, jobName, jobExecutionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new JobExecutionImpl(inner.getValue(), this.manager())); } else { return null; @@ -51,6 +47,14 @@ public JobExecution jobExecution(String resourceGroupName, String jobName, Strin } } + public Response getCustomDomainVerificationIdWithResponse(Context context) { + return this.serviceClient().getCustomDomainVerificationIdWithResponse(context); + } + + public String getCustomDomainVerificationId() { + return this.serviceClient().getCustomDomainVerificationId(); + } + private ResourceProvidersClient serviceClient() { return this.innerClient; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/RevisionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/RevisionImpl.java index e56a7cd26d093..93deebbb0d1b9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/RevisionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/RevisionImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.RevisionInner; import com.azure.resourcemanager.appcontainers.models.Revision; import com.azure.resourcemanager.appcontainers.models.RevisionHealthState; @@ -17,8 +18,8 @@ public final class RevisionImpl implements Revision { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - RevisionImpl( - RevisionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + RevisionImpl(RevisionInner innerObject, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -35,6 +36,10 @@ public String type() { return this.innerModel().type(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public OffsetDateTime createdTime() { return this.innerModel().createdTime(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SecretsCollectionImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SecretsCollectionImpl.java index deeb159aabe97..e08baebd76bdc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SecretsCollectionImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SecretsCollectionImpl.java @@ -15,8 +15,7 @@ public final class SecretsCollectionImpl implements SecretsCollection { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - SecretsCollectionImpl( - SecretsCollectionInner innerObject, + SecretsCollectionImpl(SecretsCollectionInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SourceControlImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SourceControlImpl.java index 20efb9b6dd7fc..432fbc9bdb43b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SourceControlImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/SourceControlImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.SourceControlInner; import com.azure.resourcemanager.appcontainers.models.GithubActionConfiguration; @@ -27,6 +28,10 @@ public String type() { return this.innerModel().type(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public SourceControlOperationState operationState() { return this.innerModel().operationState(); } @@ -68,21 +73,14 @@ public SourceControlImpl withExistingContainerApp(String resourceGroupName, Stri } public SourceControl create() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsSourceControls() - .createOrUpdate( - resourceGroupName, containerAppName, sourceControlName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getContainerAppsSourceControls() + .createOrUpdate(resourceGroupName, containerAppName, sourceControlName, this.innerModel(), Context.NONE); return this; } public SourceControl create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsSourceControls() - .createOrUpdate(resourceGroupName, containerAppName, sourceControlName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getContainerAppsSourceControls() + .createOrUpdate(resourceGroupName, containerAppName, sourceControlName, this.innerModel(), context); return this; } @@ -97,51 +95,35 @@ public SourceControlImpl update() { } public SourceControl apply() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsSourceControls() - .createOrUpdate( - resourceGroupName, containerAppName, sourceControlName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getContainerAppsSourceControls() + .createOrUpdate(resourceGroupName, containerAppName, sourceControlName, this.innerModel(), Context.NONE); return this; } public SourceControl apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsSourceControls() - .createOrUpdate(resourceGroupName, containerAppName, sourceControlName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getContainerAppsSourceControls() + .createOrUpdate(resourceGroupName, containerAppName, sourceControlName, this.innerModel(), context); return this; } - SourceControlImpl( - SourceControlInner innerObject, + SourceControlImpl(SourceControlInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.containerAppName = Utils.getValueFromIdByName(innerObject.id(), "containerApps"); - this.sourceControlName = Utils.getValueFromIdByName(innerObject.id(), "sourcecontrols"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.containerAppName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "containerApps"); + this.sourceControlName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "sourcecontrols"); } public SourceControl refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsSourceControls() - .getWithResponse(resourceGroupName, containerAppName, sourceControlName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerAppsSourceControls() + .getWithResponse(resourceGroupName, containerAppName, sourceControlName, Context.NONE).getValue(); return this; } public SourceControl refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getContainerAppsSourceControls() - .getWithResponse(resourceGroupName, containerAppName, sourceControlName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getContainerAppsSourceControls() + .getWithResponse(resourceGroupName, containerAppName, sourceControlName, context).getValue(); return this; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsageImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsageImpl.java new file mode 100644 index 0000000000000..9d768b691cd4e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsageImpl.java @@ -0,0 +1,44 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.Usage; +import com.azure.resourcemanager.appcontainers.models.UsageName; + +public final class UsageImpl implements Usage { + private UsageInner innerObject; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + UsageImpl(UsageInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String unit() { + return this.innerModel().unit(); + } + + public float currentValue() { + return this.innerModel().currentValue(); + } + + public float limit() { + return this.innerModel().limit(); + } + + public UsageName name() { + return this.innerModel().name(); + } + + public UsageInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesClientImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesClientImpl.java new file mode 100644 index 0000000000000..e24dcfce41662 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesClientImpl.java @@ -0,0 +1,262 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.appcontainers.fluent.UsagesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException; +import com.azure.resourcemanager.appcontainers.models.ListUsagesResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in UsagesClient. + */ +public final class UsagesClientImpl implements UsagesClient { + /** + * The proxy service used to perform REST calls. + */ + private final UsagesService service; + + /** + * The service client containing this operation class. + */ + private final ContainerAppsApiClientImpl client; + + /** + * Initializes an instance of UsagesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + UsagesClientImpl(ContainerAppsApiClientImpl client) { + this.service = RestProxy.create(UsagesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for ContainerAppsApiClientUsages to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "ContainerAppsApiClie") + public interface UsagesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.App/locations/{location}/usages") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("location") String location, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(DefaultErrorResponseErrorException.class) + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String location) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (location == null) { + return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), location, this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String location, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (location == null) { + return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), location, this.client.getSubscriptionId(), this.client.getApiVersion(), + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String location) { + return new PagedFlux<>(() -> listSinglePageAsync(location), nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String location, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(location, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String location) { + return new PagedIterable<>(listAsync(location)); + } + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String location, Context context) { + return new PagedIterable<>(listAsync(location, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws DefaultErrorResponseErrorException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesImpl.java new file mode 100644 index 0000000000000..9606e538e86b8 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/UsagesImpl.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.UsagesClient; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.Usage; +import com.azure.resourcemanager.appcontainers.models.Usages; + +public final class UsagesImpl implements Usages { + private static final ClientLogger LOGGER = new ClientLogger(UsagesImpl.class); + + private final UsagesClient innerClient; + + private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; + + public UsagesImpl(UsagesClient innerClient, + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String location) { + PagedIterable inner = this.serviceClient().list(location); + return ResourceManagerUtils.mapPage(inner, inner1 -> new UsageImpl(inner1, this.manager())); + } + + public PagedIterable list(String location, Context context) { + PagedIterable inner = this.serviceClient().list(location, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new UsageImpl(inner1, this.manager())); + } + + private UsagesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/Utils.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/Utils.java deleted file mode 100644 index 0ea4b1a2e5c1c..0000000000000 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.appcontainers.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/WorkloadProfileStatesImpl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/WorkloadProfileStatesImpl.java index 8e91506dd4319..6bf062081f2c0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/WorkloadProfileStatesImpl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/WorkloadProfileStatesImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.appcontainers.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.WorkloadProfileStatesInner; import com.azure.resourcemanager.appcontainers.models.WorkloadProfileStates; import com.azure.resourcemanager.appcontainers.models.WorkloadProfileStatesProperties; @@ -13,8 +14,7 @@ public final class WorkloadProfileStatesImpl implements WorkloadProfileStates { private final com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager; - WorkloadProfileStatesImpl( - WorkloadProfileStatesInner innerObject, + WorkloadProfileStatesImpl(WorkloadProfileStatesInner innerObject, com.azure.resourcemanager.appcontainers.ContainerAppsApiManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -36,6 +36,10 @@ public WorkloadProfileStatesProperties properties() { return this.innerModel().properties(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public WorkloadProfileStatesInner innerModel() { return this.innerObject; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/package-info.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/package-info.java index 9eb807ce176ac..4afb39fb56527 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/package-info.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the implementations for ContainerAppsApiClient. null. */ +/** + * Package containing the implementations for ContainerAppsApiClient. + * null. + */ package com.azure.resourcemanager.appcontainers.implementation; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AccessMode.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AccessMode.java index 32fca9da968cb..852bec4dd599c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AccessMode.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AccessMode.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Access mode for storage. */ +/** + * Access mode for storage. + */ public final class AccessMode extends ExpandableStringEnum { - /** Static value ReadOnly for AccessMode. */ + /** + * Static value ReadOnly for AccessMode. + */ public static final AccessMode READ_ONLY = fromString("ReadOnly"); - /** Static value ReadWrite for AccessMode. */ + /** + * Static value ReadWrite for AccessMode. + */ public static final AccessMode READ_WRITE = fromString("ReadWrite"); /** * Creates a new instance of AccessMode value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AccessMode() { /** * Creates or finds a AccessMode from its string representation. - * + * * @param name a name to look for. * @return the corresponding AccessMode. */ @@ -38,7 +44,7 @@ public static AccessMode fromString(String name) { /** * Gets known AccessMode values. - * + * * @return known AccessMode values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Action.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Action.java index 03ba2ba32d115..a6e094ef55ee6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Action.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Action.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Allow or Deny rules to determine for incoming IP. Note: Rules can only consist of ALL Allow or ALL Deny. */ +/** + * Allow or Deny rules to determine for incoming IP. Note: Rules can only consist of ALL Allow or ALL Deny. + */ public final class Action extends ExpandableStringEnum { - /** Static value Allow for Action. */ + /** + * Static value Allow for Action. + */ public static final Action ALLOW = fromString("Allow"); - /** Static value Deny for Action. */ + /** + * Static value Deny for Action. + */ public static final Action DENY = fromString("Deny"); /** * Creates a new instance of Action value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public Action() { /** * Creates or finds a Action from its string representation. - * + * * @param name a name to look for. * @return the corresponding Action. */ @@ -38,7 +44,7 @@ public static Action fromString(String name) { /** * Gets known Action values. - * + * * @return known Action values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ActiveRevisionsMode.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ActiveRevisionsMode.java index 7084cbc19dc93..1423db539a9a1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ActiveRevisionsMode.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ActiveRevisionsMode.java @@ -15,15 +15,19 @@ * default.</item></list>. */ public final class ActiveRevisionsMode extends ExpandableStringEnum { - /** Static value Multiple for ActiveRevisionsMode. */ + /** + * Static value Multiple for ActiveRevisionsMode. + */ public static final ActiveRevisionsMode MULTIPLE = fromString("Multiple"); - /** Static value Single for ActiveRevisionsMode. */ + /** + * Static value Single for ActiveRevisionsMode. + */ public static final ActiveRevisionsMode SINGLE = fromString("Single"); /** * Creates a new instance of ActiveRevisionsMode value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -32,7 +36,7 @@ public ActiveRevisionsMode() { /** * Creates or finds a ActiveRevisionsMode from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActiveRevisionsMode. */ @@ -43,7 +47,7 @@ public static ActiveRevisionsMode fromString(String name) { /** * Gets known ActiveRevisionsMode values. - * + * * @return known ActiveRevisionsMode values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Affinity.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Affinity.java index c7493e9bb820e..cab5ae672ceb0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Affinity.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Affinity.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Sticky Session Affinity. */ +/** + * Sticky Session Affinity. + */ public final class Affinity extends ExpandableStringEnum { - /** Static value sticky for Affinity. */ + /** + * Static value sticky for Affinity. + */ public static final Affinity STICKY = fromString("sticky"); - /** Static value none for Affinity. */ + /** + * Static value none for Affinity. + */ public static final Affinity NONE = fromString("none"); /** * Creates a new instance of Affinity value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public Affinity() { /** * Creates or finds a Affinity from its string representation. - * + * * @param name a name to look for. * @return the corresponding Affinity. */ @@ -38,7 +44,7 @@ public static Affinity fromString(String name) { /** * Gets known Affinity values. - * + * * @return known Affinity values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedAudiencesValidation.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedAudiencesValidation.java index 3a63ecb42ca00..4f3646099c376 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedAudiencesValidation.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedAudiencesValidation.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the Allowed Audiences validation flow. */ +/** + * The configuration settings of the Allowed Audiences validation flow. + */ @Fluent public final class AllowedAudiencesValidation { /* @@ -17,14 +19,16 @@ public final class AllowedAudiencesValidation { @JsonProperty(value = "allowedAudiences") private List allowedAudiences; - /** Creates an instance of AllowedAudiencesValidation class. */ + /** + * Creates an instance of AllowedAudiencesValidation class. + */ public AllowedAudiencesValidation() { } /** * Get the allowedAudiences property: The configuration settings of the allowed list of audiences from which to * validate the JWT token. - * + * * @return the allowedAudiences value. */ public List allowedAudiences() { @@ -34,7 +38,7 @@ public List allowedAudiences() { /** * Set the allowedAudiences property: The configuration settings of the allowed list of audiences from which to * validate the JWT token. - * + * * @param allowedAudiences the allowedAudiences value to set. * @return the AllowedAudiencesValidation object itself. */ @@ -45,7 +49,7 @@ public AllowedAudiencesValidation withAllowedAudiences(List allowedAudie /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedPrincipals.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedPrincipals.java index 836c4bcc78160..608dbfd51e793 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedPrincipals.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AllowedPrincipals.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the Azure Active Directory allowed principals. */ +/** + * The configuration settings of the Azure Active Directory allowed principals. + */ @Fluent public final class AllowedPrincipals { /* @@ -23,13 +25,15 @@ public final class AllowedPrincipals { @JsonProperty(value = "identities") private List identities; - /** Creates an instance of AllowedPrincipals class. */ + /** + * Creates an instance of AllowedPrincipals class. + */ public AllowedPrincipals() { } /** * Get the groups property: The list of the allowed groups. - * + * * @return the groups value. */ public List groups() { @@ -38,7 +42,7 @@ public List groups() { /** * Set the groups property: The list of the allowed groups. - * + * * @param groups the groups value to set. * @return the AllowedPrincipals object itself. */ @@ -49,7 +53,7 @@ public AllowedPrincipals withGroups(List groups) { /** * Get the identities property: The list of the allowed identities. - * + * * @return the identities value. */ public List identities() { @@ -58,7 +62,7 @@ public List identities() { /** * Set the identities property: The list of the allowed identities. - * + * * @param identities the identities value to set. * @return the AllowedPrincipals object itself. */ @@ -69,7 +73,7 @@ public AllowedPrincipals withIdentities(List identities) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppInsightsConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppInsightsConfiguration.java new file mode 100644 index 0000000000000..7a1878f167234 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppInsightsConfiguration.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration of Application Insights. + */ +@Fluent +public final class AppInsightsConfiguration { + /* + * Application Insights connection string + */ + @JsonProperty(value = "connectionString") + private String connectionString; + + /** + * Creates an instance of AppInsightsConfiguration class. + */ + public AppInsightsConfiguration() { + } + + /** + * Get the connectionString property: Application Insights connection string. + * + * @return the connectionString value. + */ + public String connectionString() { + return this.connectionString; + } + + /** + * Set the connectionString property: Application Insights connection string. + * + * @param connectionString the connectionString value to set. + * @return the AppInsightsConfiguration object itself. + */ + public AppInsightsConfiguration withConnectionString(String connectionString) { + this.connectionString = connectionString; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppLogsConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppLogsConfiguration.java index b015b446de0f4..4e09302a68187 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppLogsConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppLogsConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration of application logs. */ +/** + * Configuration of application logs. + */ @Fluent public final class AppLogsConfiguration { /* @@ -22,13 +24,15 @@ public final class AppLogsConfiguration { @JsonProperty(value = "logAnalyticsConfiguration") private LogAnalyticsConfiguration logAnalyticsConfiguration; - /** Creates an instance of AppLogsConfiguration class. */ + /** + * Creates an instance of AppLogsConfiguration class. + */ public AppLogsConfiguration() { } /** * Get the destination property: Logs destination, can be 'log-analytics', 'azure-monitor' or 'none'. - * + * * @return the destination value. */ public String destination() { @@ -37,7 +41,7 @@ public String destination() { /** * Set the destination property: Logs destination, can be 'log-analytics', 'azure-monitor' or 'none'. - * + * * @param destination the destination value to set. * @return the AppLogsConfiguration object itself. */ @@ -49,7 +53,7 @@ public AppLogsConfiguration withDestination(String destination) { /** * Get the logAnalyticsConfiguration property: Log Analytics configuration, must only be provided when destination * is configured as 'log-analytics'. - * + * * @return the logAnalyticsConfiguration value. */ public LogAnalyticsConfiguration logAnalyticsConfiguration() { @@ -59,7 +63,7 @@ public LogAnalyticsConfiguration logAnalyticsConfiguration() { /** * Set the logAnalyticsConfiguration property: Log Analytics configuration, must only be provided when destination * is configured as 'log-analytics'. - * + * * @param logAnalyticsConfiguration the logAnalyticsConfiguration value to set. * @return the AppLogsConfiguration object itself. */ @@ -70,7 +74,7 @@ public AppLogsConfiguration withLogAnalyticsConfiguration(LogAnalyticsConfigurat /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppProtocol.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppProtocol.java index f4c1d5c4844bf..c4db82bce2a55 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppProtocol.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppProtocol.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Tells Dapr which protocol your application is using. Valid options are http and grpc. Default is http. */ +/** + * Tells Dapr which protocol your application is using. Valid options are http and grpc. Default is http. + */ public final class AppProtocol extends ExpandableStringEnum { - /** Static value http for AppProtocol. */ + /** + * Static value http for AppProtocol. + */ public static final AppProtocol HTTP = fromString("http"); - /** Static value grpc for AppProtocol. */ + /** + * Static value grpc for AppProtocol. + */ public static final AppProtocol GRPC = fromString("grpc"); /** * Creates a new instance of AppProtocol value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AppProtocol() { /** * Creates or finds a AppProtocol from its string representation. - * + * * @param name a name to look for. * @return the corresponding AppProtocol. */ @@ -38,7 +44,7 @@ public static AppProtocol fromString(String name) { /** * Gets known AppProtocol values. - * + * * @return known AppProtocol values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppRegistration.java index d3114a04a7e5e..103b3dc5d7838 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the app registration for providers that have app ids and app secrets. */ +/** + * The configuration settings of the app registration for providers that have app ids and app secrets. + */ @Fluent public final class AppRegistration { /* @@ -22,13 +24,15 @@ public final class AppRegistration { @JsonProperty(value = "appSecretSettingName") private String appSecretSettingName; - /** Creates an instance of AppRegistration class. */ + /** + * Creates an instance of AppRegistration class. + */ public AppRegistration() { } /** * Get the appId property: The App ID of the app used for login. - * + * * @return the appId value. */ public String appId() { @@ -37,7 +41,7 @@ public String appId() { /** * Set the appId property: The App ID of the app used for login. - * + * * @param appId the appId value to set. * @return the AppRegistration object itself. */ @@ -48,7 +52,7 @@ public AppRegistration withAppId(String appId) { /** * Get the appSecretSettingName property: The app setting name that contains the app secret. - * + * * @return the appSecretSettingName value. */ public String appSecretSettingName() { @@ -57,7 +61,7 @@ public String appSecretSettingName() { /** * Set the appSecretSettingName property: The app setting name that contains the app secret. - * + * * @param appSecretSettingName the appSecretSettingName value to set. * @return the AppRegistration object itself. */ @@ -68,7 +72,7 @@ public AppRegistration withAppSecretSettingName(String appSecretSettingName) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencies.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencies.java new file mode 100644 index 0000000000000..06fb0d166368c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencies.java @@ -0,0 +1,175 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of AppResiliencies. + */ +public interface AppResiliencies { + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteWithResponse(String resourceGroupName, String appName, String name, Context context); + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String appName, String name); + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String appName, String name, Context context); + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param name Name of the resiliency policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy. + */ + AppResiliency get(String resourceGroupName, String appName, String name); + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String appName); + + /** + * List an application's resiliency policies. + * + * List container app resiliency policies. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return collection of AppResiliency policies as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String appName, Context context); + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response}. + */ + AppResiliency getById(String id); + + /** + * Get an application's resiliency policy. + * + * Get container app resiliency policy. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return container app resiliency policy along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteById(String id); + + /** + * Delete an application's resiliency policy. + * + * Delete container app resiliency policy. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new AppResiliency resource. + * + * @param name resource name. + * @return the first stage of the new AppResiliency definition. + */ + AppResiliency.DefinitionStages.Blank define(String name); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliency.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliency.java new file mode 100644 index 0000000000000..2ddc37d1845c0 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliency.java @@ -0,0 +1,358 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner; + +/** + * An immutable client-side representation of AppResiliency. + */ +public interface AppResiliency { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the timeoutPolicy property: Policy to set request timeouts. + * + * @return the timeoutPolicy value. + */ + TimeoutPolicy timeoutPolicy(); + + /** + * Gets the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @return the httpRetryPolicy value. + */ + HttpRetryPolicy httpRetryPolicy(); + + /** + * Gets the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @return the tcpRetryPolicy value. + */ + TcpRetryPolicy tcpRetryPolicy(); + + /** + * Gets the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @return the circuitBreakerPolicy value. + */ + CircuitBreakerPolicy circuitBreakerPolicy(); + + /** + * Gets the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @return the httpConnectionPool value. + */ + HttpConnectionPool httpConnectionPool(); + + /** + * Gets the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @return the tcpConnectionPool value. + */ + TcpConnectionPool tcpConnectionPool(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner object. + * + * @return the inner object. + */ + AppResiliencyInner innerModel(); + + /** + * The entirety of the AppResiliency definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The AppResiliency definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the AppResiliency definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the AppResiliency definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, appName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param appName Name of the Container App. + * @return the next definition stage. + */ + WithCreate withExistingContainerApp(String resourceGroupName, String appName); + } + + /** + * The stage of the AppResiliency definition which contains all the minimum required properties for the resource + * to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithTimeoutPolicy, DefinitionStages.WithHttpRetryPolicy, + DefinitionStages.WithTcpRetryPolicy, DefinitionStages.WithCircuitBreakerPolicy, + DefinitionStages.WithHttpConnectionPool, DefinitionStages.WithTcpConnectionPool { + /** + * Executes the create request. + * + * @return the created resource. + */ + AppResiliency create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + AppResiliency create(Context context); + } + + /** + * The stage of the AppResiliency definition allowing to specify timeoutPolicy. + */ + interface WithTimeoutPolicy { + /** + * Specifies the timeoutPolicy property: Policy to set request timeouts. + * + * @param timeoutPolicy Policy to set request timeouts. + * @return the next definition stage. + */ + WithCreate withTimeoutPolicy(TimeoutPolicy timeoutPolicy); + } + + /** + * The stage of the AppResiliency definition allowing to specify httpRetryPolicy. + */ + interface WithHttpRetryPolicy { + /** + * Specifies the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @param httpRetryPolicy Policy that defines http request retry conditions. + * @return the next definition stage. + */ + WithCreate withHttpRetryPolicy(HttpRetryPolicy httpRetryPolicy); + } + + /** + * The stage of the AppResiliency definition allowing to specify tcpRetryPolicy. + */ + interface WithTcpRetryPolicy { + /** + * Specifies the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @param tcpRetryPolicy Policy that defines tcp request retry conditions. + * @return the next definition stage. + */ + WithCreate withTcpRetryPolicy(TcpRetryPolicy tcpRetryPolicy); + } + + /** + * The stage of the AppResiliency definition allowing to specify circuitBreakerPolicy. + */ + interface WithCircuitBreakerPolicy { + /** + * Specifies the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @param circuitBreakerPolicy Policy that defines circuit breaker conditions. + * @return the next definition stage. + */ + WithCreate withCircuitBreakerPolicy(CircuitBreakerPolicy circuitBreakerPolicy); + } + + /** + * The stage of the AppResiliency definition allowing to specify httpConnectionPool. + */ + interface WithHttpConnectionPool { + /** + * Specifies the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @param httpConnectionPool Defines parameters for http connection pooling. + * @return the next definition stage. + */ + WithCreate withHttpConnectionPool(HttpConnectionPool httpConnectionPool); + } + + /** + * The stage of the AppResiliency definition allowing to specify tcpConnectionPool. + */ + interface WithTcpConnectionPool { + /** + * Specifies the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @param tcpConnectionPool Defines parameters for tcp connection pooling. + * @return the next definition stage. + */ + WithCreate withTcpConnectionPool(TcpConnectionPool tcpConnectionPool); + } + } + + /** + * Begins update for the AppResiliency resource. + * + * @return the stage of resource update. + */ + AppResiliency.Update update(); + + /** + * The template for AppResiliency update. + */ + interface Update + extends UpdateStages.WithTimeoutPolicy, UpdateStages.WithHttpRetryPolicy, UpdateStages.WithTcpRetryPolicy, + UpdateStages.WithCircuitBreakerPolicy, UpdateStages.WithHttpConnectionPool, UpdateStages.WithTcpConnectionPool { + /** + * Executes the update request. + * + * @return the updated resource. + */ + AppResiliency apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + AppResiliency apply(Context context); + } + + /** + * The AppResiliency update stages. + */ + interface UpdateStages { + /** + * The stage of the AppResiliency update allowing to specify timeoutPolicy. + */ + interface WithTimeoutPolicy { + /** + * Specifies the timeoutPolicy property: Policy to set request timeouts. + * + * @param timeoutPolicy Policy to set request timeouts. + * @return the next definition stage. + */ + Update withTimeoutPolicy(TimeoutPolicy timeoutPolicy); + } + + /** + * The stage of the AppResiliency update allowing to specify httpRetryPolicy. + */ + interface WithHttpRetryPolicy { + /** + * Specifies the httpRetryPolicy property: Policy that defines http request retry conditions. + * + * @param httpRetryPolicy Policy that defines http request retry conditions. + * @return the next definition stage. + */ + Update withHttpRetryPolicy(HttpRetryPolicy httpRetryPolicy); + } + + /** + * The stage of the AppResiliency update allowing to specify tcpRetryPolicy. + */ + interface WithTcpRetryPolicy { + /** + * Specifies the tcpRetryPolicy property: Policy that defines tcp request retry conditions. + * + * @param tcpRetryPolicy Policy that defines tcp request retry conditions. + * @return the next definition stage. + */ + Update withTcpRetryPolicy(TcpRetryPolicy tcpRetryPolicy); + } + + /** + * The stage of the AppResiliency update allowing to specify circuitBreakerPolicy. + */ + interface WithCircuitBreakerPolicy { + /** + * Specifies the circuitBreakerPolicy property: Policy that defines circuit breaker conditions. + * + * @param circuitBreakerPolicy Policy that defines circuit breaker conditions. + * @return the next definition stage. + */ + Update withCircuitBreakerPolicy(CircuitBreakerPolicy circuitBreakerPolicy); + } + + /** + * The stage of the AppResiliency update allowing to specify httpConnectionPool. + */ + interface WithHttpConnectionPool { + /** + * Specifies the httpConnectionPool property: Defines parameters for http connection pooling. + * + * @param httpConnectionPool Defines parameters for http connection pooling. + * @return the next definition stage. + */ + Update withHttpConnectionPool(HttpConnectionPool httpConnectionPool); + } + + /** + * The stage of the AppResiliency update allowing to specify tcpConnectionPool. + */ + interface WithTcpConnectionPool { + /** + * Specifies the tcpConnectionPool property: Defines parameters for tcp connection pooling. + * + * @param tcpConnectionPool Defines parameters for tcp connection pooling. + * @return the next definition stage. + */ + Update withTcpConnectionPool(TcpConnectionPool tcpConnectionPool); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + AppResiliency refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + AppResiliency refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencyCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencyCollection.java new file mode 100644 index 0000000000000..89441a8d33bb9 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppResiliencyCollection.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Collection of AppResiliency policies. + */ +@Fluent +public final class AppResiliencyCollection { + /* + * Collection of resources. + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * Link to next page of resources. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of AppResiliencyCollection class. + */ + public AppResiliencyCollection() { + } + + /** + * Get the value property: Collection of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Collection of resources. + * + * @param value the value value to set. + * @return the AppResiliencyCollection object itself. + */ + public AppResiliencyCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model AppResiliencyCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(AppResiliencyCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Apple.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Apple.java index 34b83d23ca185..9d31e6c185185 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Apple.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Apple.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Apple provider. */ +/** + * The configuration settings of the Apple provider. + */ @Fluent public final class Apple { /* @@ -29,14 +31,16 @@ public final class Apple { @JsonProperty(value = "login") private LoginScopes login; - /** Creates an instance of Apple class. */ + /** + * Creates an instance of Apple class. + */ public Apple() { } /** * Get the enabled property: <code>false</code> if the Apple provider should not be enabled despite the * set registration; otherwise, <code>true</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -46,7 +50,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>false</code> if the Apple provider should not be enabled despite the * set registration; otherwise, <code>true</code>. - * + * * @param enabled the enabled value to set. * @return the Apple object itself. */ @@ -57,7 +61,7 @@ public Apple withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the Apple registration. - * + * * @return the registration value. */ public AppleRegistration registration() { @@ -66,7 +70,7 @@ public AppleRegistration registration() { /** * Set the registration property: The configuration settings of the Apple registration. - * + * * @param registration the registration value to set. * @return the Apple object itself. */ @@ -77,7 +81,7 @@ public Apple withRegistration(AppleRegistration registration) { /** * Get the login property: The configuration settings of the login flow. - * + * * @return the login value. */ public LoginScopes login() { @@ -86,7 +90,7 @@ public LoginScopes login() { /** * Set the login property: The configuration settings of the login flow. - * + * * @param login the login value to set. * @return the Apple object itself. */ @@ -97,7 +101,7 @@ public Apple withLogin(LoginScopes login) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppleRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppleRegistration.java index 19013329ef25b..2c98beff60286 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppleRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AppleRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the registration for the Apple provider. */ +/** + * The configuration settings of the registration for the Apple provider. + */ @Fluent public final class AppleRegistration { /* @@ -22,13 +24,15 @@ public final class AppleRegistration { @JsonProperty(value = "clientSecretSettingName") private String clientSecretSettingName; - /** Creates an instance of AppleRegistration class. */ + /** + * Creates an instance of AppleRegistration class. + */ public AppleRegistration() { } /** * Get the clientId property: The Client ID of the app used for login. - * + * * @return the clientId value. */ public String clientId() { @@ -37,7 +41,7 @@ public String clientId() { /** * Set the clientId property: The Client ID of the app used for login. - * + * * @param clientId the clientId value to set. * @return the AppleRegistration object itself. */ @@ -48,7 +52,7 @@ public AppleRegistration withClientId(String clientId) { /** * Get the clientSecretSettingName property: The app setting name that contains the client secret. - * + * * @return the clientSecretSettingName value. */ public String clientSecretSettingName() { @@ -57,7 +61,7 @@ public String clientSecretSettingName() { /** * Set the clientSecretSettingName property: The app setting name that contains the client secret. - * + * * @param clientSecretSettingName the clientSecretSettingName value to set. * @return the AppleRegistration object itself. */ @@ -68,7 +72,7 @@ public AppleRegistration withClientSecretSettingName(String clientSecretSettingN /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Applicability.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Applicability.java index 1eb2343792343..e11f3e143baaa 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Applicability.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Applicability.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** indicates whether the profile is default for the location. */ +/** + * indicates whether the profile is default for the location. + */ public final class Applicability extends ExpandableStringEnum { - /** Static value LocationDefault for Applicability. */ + /** + * Static value LocationDefault for Applicability. + */ public static final Applicability LOCATION_DEFAULT = fromString("LocationDefault"); - /** Static value Custom for Applicability. */ + /** + * Static value Custom for Applicability. + */ public static final Applicability CUSTOM = fromString("Custom"); /** * Creates a new instance of Applicability value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public Applicability() { /** * Creates or finds a Applicability from its string representation. - * + * * @param name a name to look for. * @return the corresponding Applicability. */ @@ -38,7 +44,7 @@ public static Applicability fromString(String name) { /** * Gets known Applicability values. - * + * * @return known Applicability values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfig.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfig.java index 9f83b649744db..0c8e2faee1f91 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfig.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfig.java @@ -8,32 +8,34 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigInner; -/** An immutable client-side representation of AuthConfig. */ +/** + * An immutable client-side representation of AuthConfig. + */ public interface AuthConfig { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); @@ -41,7 +43,7 @@ public interface AuthConfig { /** * Gets the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization. - * + * * @return the platform value. */ AuthPlatform platform(); @@ -49,7 +51,7 @@ public interface AuthConfig { /** * Gets the globalValidation property: The configuration settings that determines the validation flow of users using * Service Authentication/Authorization. - * + * * @return the globalValidation value. */ GlobalValidation globalValidation(); @@ -57,7 +59,7 @@ public interface AuthConfig { /** * Gets the identityProviders property: The configuration settings of each of the identity providers used to * configure ContainerApp Service Authentication/Authorization. - * + * * @return the identityProviders value. */ IdentityProviders identityProviders(); @@ -65,7 +67,7 @@ public interface AuthConfig { /** * Gets the login property: The configuration settings of the login flow of users using ContainerApp Service * Authentication/Authorization. - * + * * @return the login value. */ Login login(); @@ -73,41 +75,57 @@ public interface AuthConfig { /** * Gets the httpSettings property: The configuration settings of the HTTP requests for authentication and * authorization requests made against ContainerApp Service Authentication/Authorization. - * + * * @return the httpSettings value. */ HttpSettings httpSettings(); + /** + * Gets the encryptionSettings property: The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * + * @return the encryptionSettings value. + */ + EncryptionSettings encryptionSettings(); + /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigInner object. - * + * * @return the inner object. */ AuthConfigInner innerModel(); - /** The entirety of the AuthConfig definition. */ + /** + * The entirety of the AuthConfig definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The AuthConfig definition stages. */ + /** + * The AuthConfig definition stages. + */ interface DefinitionStages { - /** The first stage of the AuthConfig definition. */ + /** + * The first stage of the AuthConfig definition. + */ interface Blank extends WithParentResource { } - /** The stage of the AuthConfig definition allowing to specify parent resource. */ + /** + * The stage of the AuthConfig definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, containerAppName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @return the next definition stage. @@ -119,202 +137,250 @@ interface WithParentResource { * The stage of the AuthConfig definition which contains all the minimum required properties for the resource to * be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithPlatform, - DefinitionStages.WithGlobalValidation, - DefinitionStages.WithIdentityProviders, - DefinitionStages.WithLogin, - DefinitionStages.WithHttpSettings { + interface WithCreate extends DefinitionStages.WithPlatform, DefinitionStages.WithGlobalValidation, + DefinitionStages.WithIdentityProviders, DefinitionStages.WithLogin, DefinitionStages.WithHttpSettings, + DefinitionStages.WithEncryptionSettings { /** * Executes the create request. - * + * * @return the created resource. */ AuthConfig create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ AuthConfig create(Context context); } - /** The stage of the AuthConfig definition allowing to specify platform. */ + /** + * The stage of the AuthConfig definition allowing to specify platform. + */ interface WithPlatform { /** * Specifies the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization.. - * + * * @param platform The configuration settings of the platform of ContainerApp Service - * Authentication/Authorization. + * Authentication/Authorization. * @return the next definition stage. */ WithCreate withPlatform(AuthPlatform platform); } - /** The stage of the AuthConfig definition allowing to specify globalValidation. */ + /** + * The stage of the AuthConfig definition allowing to specify globalValidation. + */ interface WithGlobalValidation { /** * Specifies the globalValidation property: The configuration settings that determines the validation flow * of users using Service Authentication/Authorization.. - * + * * @param globalValidation The configuration settings that determines the validation flow of users using - * Service Authentication/Authorization. + * Service Authentication/Authorization. * @return the next definition stage. */ WithCreate withGlobalValidation(GlobalValidation globalValidation); } - /** The stage of the AuthConfig definition allowing to specify identityProviders. */ + /** + * The stage of the AuthConfig definition allowing to specify identityProviders. + */ interface WithIdentityProviders { /** * Specifies the identityProviders property: The configuration settings of each of the identity providers * used to configure ContainerApp Service Authentication/Authorization.. - * + * * @param identityProviders The configuration settings of each of the identity providers used to configure - * ContainerApp Service Authentication/Authorization. + * ContainerApp Service Authentication/Authorization. * @return the next definition stage. */ WithCreate withIdentityProviders(IdentityProviders identityProviders); } - /** The stage of the AuthConfig definition allowing to specify login. */ + /** + * The stage of the AuthConfig definition allowing to specify login. + */ interface WithLogin { /** * Specifies the login property: The configuration settings of the login flow of users using ContainerApp * Service Authentication/Authorization.. - * + * * @param login The configuration settings of the login flow of users using ContainerApp Service - * Authentication/Authorization. + * Authentication/Authorization. * @return the next definition stage. */ WithCreate withLogin(Login login); } - /** The stage of the AuthConfig definition allowing to specify httpSettings. */ + /** + * The stage of the AuthConfig definition allowing to specify httpSettings. + */ interface WithHttpSettings { /** * Specifies the httpSettings property: The configuration settings of the HTTP requests for authentication * and authorization requests made against ContainerApp Service Authentication/Authorization.. - * + * * @param httpSettings The configuration settings of the HTTP requests for authentication and authorization - * requests made against ContainerApp Service Authentication/Authorization. + * requests made against ContainerApp Service Authentication/Authorization. * @return the next definition stage. */ WithCreate withHttpSettings(HttpSettings httpSettings); } + + /** + * The stage of the AuthConfig definition allowing to specify encryptionSettings. + */ + interface WithEncryptionSettings { + /** + * Specifies the encryptionSettings property: The configuration settings of the secrets references of + * encryption key and signing key for ContainerApp Service Authentication/Authorization.. + * + * @param encryptionSettings The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * @return the next definition stage. + */ + WithCreate withEncryptionSettings(EncryptionSettings encryptionSettings); + } } /** * Begins update for the AuthConfig resource. - * + * * @return the stage of resource update. */ AuthConfig.Update update(); - /** The template for AuthConfig update. */ + /** + * The template for AuthConfig update. + */ interface Update - extends UpdateStages.WithPlatform, - UpdateStages.WithGlobalValidation, - UpdateStages.WithIdentityProviders, - UpdateStages.WithLogin, - UpdateStages.WithHttpSettings { + extends UpdateStages.WithPlatform, UpdateStages.WithGlobalValidation, UpdateStages.WithIdentityProviders, + UpdateStages.WithLogin, UpdateStages.WithHttpSettings, UpdateStages.WithEncryptionSettings { /** * Executes the update request. - * + * * @return the updated resource. */ AuthConfig apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ AuthConfig apply(Context context); } - /** The AuthConfig update stages. */ + /** + * The AuthConfig update stages. + */ interface UpdateStages { - /** The stage of the AuthConfig update allowing to specify platform. */ + /** + * The stage of the AuthConfig update allowing to specify platform. + */ interface WithPlatform { /** * Specifies the platform property: The configuration settings of the platform of ContainerApp Service * Authentication/Authorization.. - * + * * @param platform The configuration settings of the platform of ContainerApp Service - * Authentication/Authorization. + * Authentication/Authorization. * @return the next definition stage. */ Update withPlatform(AuthPlatform platform); } - /** The stage of the AuthConfig update allowing to specify globalValidation. */ + /** + * The stage of the AuthConfig update allowing to specify globalValidation. + */ interface WithGlobalValidation { /** * Specifies the globalValidation property: The configuration settings that determines the validation flow * of users using Service Authentication/Authorization.. - * + * * @param globalValidation The configuration settings that determines the validation flow of users using - * Service Authentication/Authorization. + * Service Authentication/Authorization. * @return the next definition stage. */ Update withGlobalValidation(GlobalValidation globalValidation); } - /** The stage of the AuthConfig update allowing to specify identityProviders. */ + /** + * The stage of the AuthConfig update allowing to specify identityProviders. + */ interface WithIdentityProviders { /** * Specifies the identityProviders property: The configuration settings of each of the identity providers * used to configure ContainerApp Service Authentication/Authorization.. - * + * * @param identityProviders The configuration settings of each of the identity providers used to configure - * ContainerApp Service Authentication/Authorization. + * ContainerApp Service Authentication/Authorization. * @return the next definition stage. */ Update withIdentityProviders(IdentityProviders identityProviders); } - /** The stage of the AuthConfig update allowing to specify login. */ + /** + * The stage of the AuthConfig update allowing to specify login. + */ interface WithLogin { /** * Specifies the login property: The configuration settings of the login flow of users using ContainerApp * Service Authentication/Authorization.. - * + * * @param login The configuration settings of the login flow of users using ContainerApp Service - * Authentication/Authorization. + * Authentication/Authorization. * @return the next definition stage. */ Update withLogin(Login login); } - /** The stage of the AuthConfig update allowing to specify httpSettings. */ + /** + * The stage of the AuthConfig update allowing to specify httpSettings. + */ interface WithHttpSettings { /** * Specifies the httpSettings property: The configuration settings of the HTTP requests for authentication * and authorization requests made against ContainerApp Service Authentication/Authorization.. - * + * * @param httpSettings The configuration settings of the HTTP requests for authentication and authorization - * requests made against ContainerApp Service Authentication/Authorization. + * requests made against ContainerApp Service Authentication/Authorization. * @return the next definition stage. */ Update withHttpSettings(HttpSettings httpSettings); } + + /** + * The stage of the AuthConfig update allowing to specify encryptionSettings. + */ + interface WithEncryptionSettings { + /** + * Specifies the encryptionSettings property: The configuration settings of the secrets references of + * encryption key and signing key for ContainerApp Service Authentication/Authorization.. + * + * @param encryptionSettings The configuration settings of the secrets references of encryption key and + * signing key for ContainerApp Service Authentication/Authorization. + * @return the next definition stage. + */ + Update withEncryptionSettings(EncryptionSettings encryptionSettings); + } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ AuthConfig refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfigCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfigCollection.java index cca73947e6dc6..cb6e133045e8c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfigCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthConfigCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** AuthConfig collection ARM resource. */ +/** + * AuthConfig collection ARM resource. + */ @Fluent public final class AuthConfigCollection { /* @@ -25,13 +27,15 @@ public final class AuthConfigCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AuthConfigCollection class. */ + /** + * Creates an instance of AuthConfigCollection class. + */ public AuthConfigCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the AuthConfigCollection object itself. */ @@ -51,7 +55,7 @@ public AuthConfigCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model AuthConfigCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model AuthConfigCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthPlatform.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthPlatform.java index 94143f340220b..0c477b52f3a7b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthPlatform.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AuthPlatform.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the platform of ContainerApp Service Authentication/Authorization. */ +/** + * The configuration settings of the platform of ContainerApp Service Authentication/Authorization. + */ @Fluent public final class AuthPlatform { /* @@ -25,14 +27,16 @@ public final class AuthPlatform { @JsonProperty(value = "runtimeVersion") private String runtimeVersion; - /** Creates an instance of AuthPlatform class. */ + /** + * Creates an instance of AuthPlatform class. + */ public AuthPlatform() { } /** * Get the enabled property: <code>true</code> if the Authentication / Authorization feature is enabled * for the current app; otherwise, <code>false</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -42,7 +46,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>true</code> if the Authentication / Authorization feature is enabled * for the current app; otherwise, <code>false</code>. - * + * * @param enabled the enabled value to set. * @return the AuthPlatform object itself. */ @@ -53,9 +57,10 @@ public AuthPlatform withEnabled(Boolean enabled) { /** * Get the runtimeVersion property: The RuntimeVersion of the Authentication / Authorization feature in use for the - * current app. The setting in this value can control the behavior of certain features in the Authentication / - * Authorization module. - * + * current app. + * The setting in this value can control the behavior of certain features in the Authentication / Authorization + * module. + * * @return the runtimeVersion value. */ public String runtimeVersion() { @@ -64,9 +69,10 @@ public String runtimeVersion() { /** * Set the runtimeVersion property: The RuntimeVersion of the Authentication / Authorization feature in use for the - * current app. The setting in this value can control the behavior of certain features in the Authentication / - * Authorization module. - * + * current app. + * The setting in this value can control the behavior of certain features in the Authentication / Authorization + * module. + * * @param runtimeVersion the runtimeVersion value to set. * @return the AuthPlatform object itself. */ @@ -77,7 +83,7 @@ public AuthPlatform withRuntimeVersion(String runtimeVersion) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableOperations.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableOperations.java index 61b0f8789edab..adf507e54f746 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableOperations.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableOperations.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Available operations of the service. */ +/** + * Available operations of the service. + */ @Fluent public final class AvailableOperations { /* @@ -25,13 +27,15 @@ public final class AvailableOperations { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of AvailableOperations class. */ + /** + * Creates an instance of AvailableOperations class. + */ public AvailableOperations() { } /** * Get the value property: Collection of available operation details. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of available operation details. - * + * * @param value the value value to set. * @return the AvailableOperations object itself. */ @@ -50,9 +54,9 @@ public AvailableOperations withValue(List value) { } /** - * Get the nextLink property: URL client should use to fetch the next page (per server side paging). It's null for - * now, added for future use. - * + * Get the nextLink property: URL client should use to fetch the next page (per server side paging). + * It's null for now, added for future use. + * * @return the nextLink value. */ public String nextLink() { @@ -60,9 +64,9 @@ public String nextLink() { } /** - * Set the nextLink property: URL client should use to fetch the next page (per server side paging). It's null for - * now, added for future use. - * + * Set the nextLink property: URL client should use to fetch the next page (per server side paging). + * It's null for now, added for future use. + * * @param nextLink the nextLink value to set. * @return the AvailableOperations object itself. */ @@ -73,7 +77,7 @@ public AvailableOperations withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfile.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfile.java index 9d2d444694b5d..453036211ae0c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfile.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfile.java @@ -4,48 +4,58 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.AvailableWorkloadProfileInner; -/** An immutable client-side representation of AvailableWorkloadProfile. */ +/** + * An immutable client-side representation of AvailableWorkloadProfile. + */ public interface AvailableWorkloadProfile { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: Region of the workload profile. - * + * * @return the location value. */ String location(); /** * Gets the properties property: Revision resource specific properties. - * + * * @return the properties value. */ AvailableWorkloadProfileProperties properties(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.AvailableWorkloadProfileInner object. - * + * * @return the inner object. */ AvailableWorkloadProfileInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfileProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfileProperties.java index 2960060ee3cfa..7985d9a2d6154 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfileProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfileProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Revision resource specific properties. */ +/** + * Revision resource specific properties. + */ @Fluent public final class AvailableWorkloadProfileProperties { /* @@ -34,19 +36,27 @@ public final class AvailableWorkloadProfileProperties { @JsonProperty(value = "memoryGiB") private Integer memoryGiB; + /* + * Number of GPUs. + */ + @JsonProperty(value = "gpus") + private Integer gpus; + /* * The everyday name of the workload profile. */ @JsonProperty(value = "displayName") private String displayName; - /** Creates an instance of AvailableWorkloadProfileProperties class. */ + /** + * Creates an instance of AvailableWorkloadProfileProperties class. + */ public AvailableWorkloadProfileProperties() { } /** * Get the category property: Used to categorize workload profiles. - * + * * @return the category value. */ public String category() { @@ -55,7 +65,7 @@ public String category() { /** * Set the category property: Used to categorize workload profiles. - * + * * @param category the category value to set. * @return the AvailableWorkloadProfileProperties object itself. */ @@ -66,7 +76,7 @@ public AvailableWorkloadProfileProperties withCategory(String category) { /** * Get the applicability property: indicates whether the profile is default for the location. - * + * * @return the applicability value. */ public Applicability applicability() { @@ -75,7 +85,7 @@ public Applicability applicability() { /** * Set the applicability property: indicates whether the profile is default for the location. - * + * * @param applicability the applicability value to set. * @return the AvailableWorkloadProfileProperties object itself. */ @@ -86,7 +96,7 @@ public AvailableWorkloadProfileProperties withApplicability(Applicability applic /** * Get the cores property: Number of cores in CPU. - * + * * @return the cores value. */ public Integer cores() { @@ -95,7 +105,7 @@ public Integer cores() { /** * Set the cores property: Number of cores in CPU. - * + * * @param cores the cores value to set. * @return the AvailableWorkloadProfileProperties object itself. */ @@ -106,7 +116,7 @@ public AvailableWorkloadProfileProperties withCores(Integer cores) { /** * Get the memoryGiB property: Memory in GiB. - * + * * @return the memoryGiB value. */ public Integer memoryGiB() { @@ -115,7 +125,7 @@ public Integer memoryGiB() { /** * Set the memoryGiB property: Memory in GiB. - * + * * @param memoryGiB the memoryGiB value to set. * @return the AvailableWorkloadProfileProperties object itself. */ @@ -124,9 +134,29 @@ public AvailableWorkloadProfileProperties withMemoryGiB(Integer memoryGiB) { return this; } + /** + * Get the gpus property: Number of GPUs. + * + * @return the gpus value. + */ + public Integer gpus() { + return this.gpus; + } + + /** + * Set the gpus property: Number of GPUs. + * + * @param gpus the gpus value to set. + * @return the AvailableWorkloadProfileProperties object itself. + */ + public AvailableWorkloadProfileProperties withGpus(Integer gpus) { + this.gpus = gpus; + return this; + } + /** * Get the displayName property: The everyday name of the workload profile. - * + * * @return the displayName value. */ public String displayName() { @@ -135,7 +165,7 @@ public String displayName() { /** * Set the displayName property: The everyday name of the workload profile. - * + * * @param displayName the displayName value to set. * @return the AvailableWorkloadProfileProperties object itself. */ @@ -146,7 +176,7 @@ public AvailableWorkloadProfileProperties withDisplayName(String displayName) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfiles.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfiles.java index 30eb8fd64ad8d..e31a39b02d696 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfiles.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfiles.java @@ -7,13 +7,15 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of AvailableWorkloadProfiles. */ +/** + * Resource collection API of AvailableWorkloadProfiles. + */ public interface AvailableWorkloadProfiles { /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -24,9 +26,9 @@ public interface AvailableWorkloadProfiles { /** * Get available workload profiles by location. - * - *

Get all available workload profiles for a location. - * + * + * Get all available workload profiles for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfilesCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfilesCollection.java index a21dfb315cd7e..78d73b05d5706 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfilesCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AvailableWorkloadProfilesCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of available workload profiles in the location. */ +/** + * Collection of available workload profiles in the location. + */ @Fluent public final class AvailableWorkloadProfilesCollection { /* @@ -25,13 +27,15 @@ public final class AvailableWorkloadProfilesCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AvailableWorkloadProfilesCollection class. */ + /** + * Creates an instance of AvailableWorkloadProfilesCollection class. + */ public AvailableWorkloadProfilesCollection() { } /** * Get the value property: Collection of workload profiles. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of workload profiles. - * + * * @param value the value value to set. * @return the AvailableWorkloadProfilesCollection object itself. */ @@ -51,7 +55,7 @@ public AvailableWorkloadProfilesCollection withValue(List e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectory.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectory.java index a75e724b450f1..4f9eaadb5f714 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectory.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectory.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Azure Active directory provider. */ +/** + * The configuration settings of the Azure Active directory provider. + */ @Fluent public final class AzureActiveDirectory { /* @@ -43,14 +45,16 @@ public final class AzureActiveDirectory { @JsonProperty(value = "isAutoProvisioned") private Boolean isAutoProvisioned; - /** Creates an instance of AzureActiveDirectory class. */ + /** + * Creates an instance of AzureActiveDirectory class. + */ public AzureActiveDirectory() { } /** * Get the enabled property: <code>false</code> if the Azure Active Directory provider should not be * enabled despite the set registration; otherwise, <code>true</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -60,7 +64,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>false</code> if the Azure Active Directory provider should not be * enabled despite the set registration; otherwise, <code>true</code>. - * + * * @param enabled the enabled value to set. * @return the AzureActiveDirectory object itself. */ @@ -71,7 +75,7 @@ public AzureActiveDirectory withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the Azure Active Directory app registration. - * + * * @return the registration value. */ public AzureActiveDirectoryRegistration registration() { @@ -80,7 +84,7 @@ public AzureActiveDirectoryRegistration registration() { /** * Set the registration property: The configuration settings of the Azure Active Directory app registration. - * + * * @param registration the registration value to set. * @return the AzureActiveDirectory object itself. */ @@ -91,7 +95,7 @@ public AzureActiveDirectory withRegistration(AzureActiveDirectoryRegistration re /** * Get the login property: The configuration settings of the Azure Active Directory login flow. - * + * * @return the login value. */ public AzureActiveDirectoryLogin login() { @@ -100,7 +104,7 @@ public AzureActiveDirectoryLogin login() { /** * Set the login property: The configuration settings of the Azure Active Directory login flow. - * + * * @param login the login value to set. * @return the AzureActiveDirectory object itself. */ @@ -111,7 +115,7 @@ public AzureActiveDirectory withLogin(AzureActiveDirectoryLogin login) { /** * Get the validation property: The configuration settings of the Azure Active Directory token validation flow. - * + * * @return the validation value. */ public AzureActiveDirectoryValidation validation() { @@ -120,7 +124,7 @@ public AzureActiveDirectoryValidation validation() { /** * Set the validation property: The configuration settings of the Azure Active Directory token validation flow. - * + * * @param validation the validation value to set. * @return the AzureActiveDirectory object itself. */ @@ -131,9 +135,10 @@ public AzureActiveDirectory withValidation(AzureActiveDirectoryValidation valida /** * Get the isAutoProvisioned property: Gets a value indicating whether the Azure AD configuration was - * auto-provisioned using 1st party tooling. This is an internal flag primarily intended to support the Azure - * Management Portal. Users should not read or write to this property. - * + * auto-provisioned using 1st party tooling. + * This is an internal flag primarily intended to support the Azure Management Portal. Users should not + * read or write to this property. + * * @return the isAutoProvisioned value. */ public Boolean isAutoProvisioned() { @@ -142,9 +147,10 @@ public Boolean isAutoProvisioned() { /** * Set the isAutoProvisioned property: Gets a value indicating whether the Azure AD configuration was - * auto-provisioned using 1st party tooling. This is an internal flag primarily intended to support the Azure - * Management Portal. Users should not read or write to this property. - * + * auto-provisioned using 1st party tooling. + * This is an internal flag primarily intended to support the Azure Management Portal. Users should not + * read or write to this property. + * * @param isAutoProvisioned the isAutoProvisioned value to set. * @return the AzureActiveDirectory object itself. */ @@ -155,7 +161,7 @@ public AzureActiveDirectory withIsAutoProvisioned(Boolean isAutoProvisioned) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryLogin.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryLogin.java index 8f56e461f975e..89bc07ad12162 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryLogin.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryLogin.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the Azure Active Directory login flow. */ +/** + * The configuration settings of the Azure Active Directory login flow. + */ @Fluent public final class AzureActiveDirectoryLogin { /* @@ -25,14 +27,16 @@ public final class AzureActiveDirectoryLogin { @JsonProperty(value = "disableWWWAuthenticate") private Boolean disableWwwAuthenticate; - /** Creates an instance of AzureActiveDirectoryLogin class. */ + /** + * Creates an instance of AzureActiveDirectoryLogin class. + */ public AzureActiveDirectoryLogin() { } /** - * Get the loginParameters property: Login parameters to send to the OpenID Connect authorization endpoint when a - * user logs in. Each parameter must be in the form "key=value". - * + * Get the loginParameters property: Login parameters to send to the OpenID Connect authorization endpoint when + * a user logs in. Each parameter must be in the form "key=value". + * * @return the loginParameters value. */ public List loginParameters() { @@ -40,9 +44,9 @@ public List loginParameters() { } /** - * Set the loginParameters property: Login parameters to send to the OpenID Connect authorization endpoint when a - * user logs in. Each parameter must be in the form "key=value". - * + * Set the loginParameters property: Login parameters to send to the OpenID Connect authorization endpoint when + * a user logs in. Each parameter must be in the form "key=value". + * * @param loginParameters the loginParameters value to set. * @return the AzureActiveDirectoryLogin object itself. */ @@ -52,9 +56,9 @@ public AzureActiveDirectoryLogin withLoginParameters(List loginParameter } /** - * Get the disableWwwAuthenticate property: <code>true</code> if the www-authenticate provider should be - * omitted from the request; otherwise, <code>false</code>. - * + * Get the disableWwwAuthenticate property: <code>true</code> if the www-authenticate provider should + * be omitted from the request; otherwise, <code>false</code>. + * * @return the disableWwwAuthenticate value. */ public Boolean disableWwwAuthenticate() { @@ -62,9 +66,9 @@ public Boolean disableWwwAuthenticate() { } /** - * Set the disableWwwAuthenticate property: <code>true</code> if the www-authenticate provider should be - * omitted from the request; otherwise, <code>false</code>. - * + * Set the disableWwwAuthenticate property: <code>true</code> if the www-authenticate provider should + * be omitted from the request; otherwise, <code>false</code>. + * * @param disableWwwAuthenticate the disableWwwAuthenticate value to set. * @return the AzureActiveDirectoryLogin object itself. */ @@ -75,7 +79,7 @@ public AzureActiveDirectoryLogin withDisableWwwAuthenticate(Boolean disableWwwAu /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryRegistration.java index 4ce29639d2574..12c0102dc6d8d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Azure Active Directory app registration. */ +/** + * The configuration settings of the Azure Active Directory app registration. + */ @Fluent public final class AzureActiveDirectoryRegistration { /* @@ -59,16 +61,20 @@ public final class AzureActiveDirectoryRegistration { @JsonProperty(value = "clientSecretCertificateIssuer") private String clientSecretCertificateIssuer; - /** Creates an instance of AzureActiveDirectoryRegistration class. */ + /** + * Creates an instance of AzureActiveDirectoryRegistration class. + */ public AzureActiveDirectoryRegistration() { } /** * Get the openIdIssuer property: The OpenID Connect Issuer URI that represents the entity which issues access - * tokens for this application. When using Azure Active Directory, this value is the URI of the directory tenant, - * e.g. https://login.microsoftonline.com/v2.0/{tenant-guid}/. This URI is a case-sensitive identifier for the token - * issuer. More information on OpenID Connect Discovery: http://openid.net/specs/openid-connect-discovery-1_0.html. - * + * tokens for this application. + * When using Azure Active Directory, this value is the URI of the directory tenant, e.g. + * https://login.microsoftonline.com/v2.0/{tenant-guid}/. + * This URI is a case-sensitive identifier for the token issuer. + * More information on OpenID Connect Discovery: http://openid.net/specs/openid-connect-discovery-1_0.html. + * * @return the openIdIssuer value. */ public String openIdIssuer() { @@ -77,10 +83,12 @@ public String openIdIssuer() { /** * Set the openIdIssuer property: The OpenID Connect Issuer URI that represents the entity which issues access - * tokens for this application. When using Azure Active Directory, this value is the URI of the directory tenant, - * e.g. https://login.microsoftonline.com/v2.0/{tenant-guid}/. This URI is a case-sensitive identifier for the token - * issuer. More information on OpenID Connect Discovery: http://openid.net/specs/openid-connect-discovery-1_0.html. - * + * tokens for this application. + * When using Azure Active Directory, this value is the URI of the directory tenant, e.g. + * https://login.microsoftonline.com/v2.0/{tenant-guid}/. + * This URI is a case-sensitive identifier for the token issuer. + * More information on OpenID Connect Discovery: http://openid.net/specs/openid-connect-discovery-1_0.html. + * * @param openIdIssuer the openIdIssuer value to set. * @return the AzureActiveDirectoryRegistration object itself. */ @@ -90,10 +98,11 @@ public AzureActiveDirectoryRegistration withOpenIdIssuer(String openIdIssuer) { } /** - * Get the clientId property: The Client ID of this relying party application, known as the client_id. This setting - * is required for enabling OpenID Connection authentication with Azure Active Directory or other 3rd party OpenID - * Connect providers. More information on OpenID Connect: http://openid.net/specs/openid-connect-core-1_0.html. - * + * Get the clientId property: The Client ID of this relying party application, known as the client_id. + * This setting is required for enabling OpenID Connection authentication with Azure Active Directory or + * other 3rd party OpenID Connect providers. + * More information on OpenID Connect: http://openid.net/specs/openid-connect-core-1_0.html. + * * @return the clientId value. */ public String clientId() { @@ -101,10 +110,11 @@ public String clientId() { } /** - * Set the clientId property: The Client ID of this relying party application, known as the client_id. This setting - * is required for enabling OpenID Connection authentication with Azure Active Directory or other 3rd party OpenID - * Connect providers. More information on OpenID Connect: http://openid.net/specs/openid-connect-core-1_0.html. - * + * Set the clientId property: The Client ID of this relying party application, known as the client_id. + * This setting is required for enabling OpenID Connection authentication with Azure Active Directory or + * other 3rd party OpenID Connect providers. + * More information on OpenID Connect: http://openid.net/specs/openid-connect-core-1_0.html. + * * @param clientId the clientId value to set. * @return the AzureActiveDirectoryRegistration object itself. */ @@ -116,7 +126,7 @@ public AzureActiveDirectoryRegistration withClientId(String clientId) { /** * Get the clientSecretSettingName property: The app setting name that contains the client secret of the relying * party application. - * + * * @return the clientSecretSettingName value. */ public String clientSecretSettingName() { @@ -126,7 +136,7 @@ public String clientSecretSettingName() { /** * Set the clientSecretSettingName property: The app setting name that contains the client secret of the relying * party application. - * + * * @param clientSecretSettingName the clientSecretSettingName value to set. * @return the AzureActiveDirectoryRegistration object itself. */ @@ -137,9 +147,9 @@ public AzureActiveDirectoryRegistration withClientSecretSettingName(String clien /** * Get the clientSecretCertificateThumbprint property: An alternative to the client secret, that is the thumbprint - * of a certificate used for signing purposes. This property acts as a replacement for the Client Secret. It is also - * optional. - * + * of a certificate used for signing purposes. This property acts as + * a replacement for the Client Secret. It is also optional. + * * @return the clientSecretCertificateThumbprint value. */ public String clientSecretCertificateThumbprint() { @@ -148,23 +158,23 @@ public String clientSecretCertificateThumbprint() { /** * Set the clientSecretCertificateThumbprint property: An alternative to the client secret, that is the thumbprint - * of a certificate used for signing purposes. This property acts as a replacement for the Client Secret. It is also - * optional. - * + * of a certificate used for signing purposes. This property acts as + * a replacement for the Client Secret. It is also optional. + * * @param clientSecretCertificateThumbprint the clientSecretCertificateThumbprint value to set. * @return the AzureActiveDirectoryRegistration object itself. */ - public AzureActiveDirectoryRegistration withClientSecretCertificateThumbprint( - String clientSecretCertificateThumbprint) { + public AzureActiveDirectoryRegistration + withClientSecretCertificateThumbprint(String clientSecretCertificateThumbprint) { this.clientSecretCertificateThumbprint = clientSecretCertificateThumbprint; return this; } /** * Get the clientSecretCertificateSubjectAlternativeName property: An alternative to the client secret thumbprint, - * that is the subject alternative name of a certificate used for signing purposes. This property acts as a - * replacement for the Client Secret Certificate Thumbprint. It is also optional. - * + * that is the subject alternative name of a certificate used for signing purposes. This property acts as + * a replacement for the Client Secret Certificate Thumbprint. It is also optional. + * * @return the clientSecretCertificateSubjectAlternativeName value. */ public String clientSecretCertificateSubjectAlternativeName() { @@ -173,24 +183,24 @@ public String clientSecretCertificateSubjectAlternativeName() { /** * Set the clientSecretCertificateSubjectAlternativeName property: An alternative to the client secret thumbprint, - * that is the subject alternative name of a certificate used for signing purposes. This property acts as a - * replacement for the Client Secret Certificate Thumbprint. It is also optional. - * + * that is the subject alternative name of a certificate used for signing purposes. This property acts as + * a replacement for the Client Secret Certificate Thumbprint. It is also optional. + * * @param clientSecretCertificateSubjectAlternativeName the clientSecretCertificateSubjectAlternativeName value to - * set. + * set. * @return the AzureActiveDirectoryRegistration object itself. */ - public AzureActiveDirectoryRegistration withClientSecretCertificateSubjectAlternativeName( - String clientSecretCertificateSubjectAlternativeName) { + public AzureActiveDirectoryRegistration + withClientSecretCertificateSubjectAlternativeName(String clientSecretCertificateSubjectAlternativeName) { this.clientSecretCertificateSubjectAlternativeName = clientSecretCertificateSubjectAlternativeName; return this; } /** * Get the clientSecretCertificateIssuer property: An alternative to the client secret thumbprint, that is the - * issuer of a certificate used for signing purposes. This property acts as a replacement for the Client Secret - * Certificate Thumbprint. It is also optional. - * + * issuer of a certificate used for signing purposes. This property acts as + * a replacement for the Client Secret Certificate Thumbprint. It is also optional. + * * @return the clientSecretCertificateIssuer value. */ public String clientSecretCertificateIssuer() { @@ -199,9 +209,9 @@ public String clientSecretCertificateIssuer() { /** * Set the clientSecretCertificateIssuer property: An alternative to the client secret thumbprint, that is the - * issuer of a certificate used for signing purposes. This property acts as a replacement for the Client Secret - * Certificate Thumbprint. It is also optional. - * + * issuer of a certificate used for signing purposes. This property acts as + * a replacement for the Client Secret Certificate Thumbprint. It is also optional. + * * @param clientSecretCertificateIssuer the clientSecretCertificateIssuer value to set. * @return the AzureActiveDirectoryRegistration object itself. */ @@ -212,7 +222,7 @@ public AzureActiveDirectoryRegistration withClientSecretCertificateIssuer(String /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryValidation.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryValidation.java index 8b67f4fdf7155..173ceeab4e9c7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryValidation.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureActiveDirectoryValidation.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the Azure Active Directory token validation flow. */ +/** + * The configuration settings of the Azure Active Directory token validation flow. + */ @Fluent public final class AzureActiveDirectoryValidation { /* @@ -29,14 +31,16 @@ public final class AzureActiveDirectoryValidation { @JsonProperty(value = "defaultAuthorizationPolicy") private DefaultAuthorizationPolicy defaultAuthorizationPolicy; - /** Creates an instance of AzureActiveDirectoryValidation class. */ + /** + * Creates an instance of AzureActiveDirectoryValidation class. + */ public AzureActiveDirectoryValidation() { } /** * Get the jwtClaimChecks property: The configuration settings of the checks that should be made while validating * the JWT Claims. - * + * * @return the jwtClaimChecks value. */ public JwtClaimChecks jwtClaimChecks() { @@ -46,7 +50,7 @@ public JwtClaimChecks jwtClaimChecks() { /** * Set the jwtClaimChecks property: The configuration settings of the checks that should be made while validating * the JWT Claims. - * + * * @param jwtClaimChecks the jwtClaimChecks value to set. * @return the AzureActiveDirectoryValidation object itself. */ @@ -58,7 +62,7 @@ public AzureActiveDirectoryValidation withJwtClaimChecks(JwtClaimChecks jwtClaim /** * Get the allowedAudiences property: The list of audiences that can make successful authentication/authorization * requests. - * + * * @return the allowedAudiences value. */ public List allowedAudiences() { @@ -68,7 +72,7 @@ public List allowedAudiences() { /** * Set the allowedAudiences property: The list of audiences that can make successful authentication/authorization * requests. - * + * * @param allowedAudiences the allowedAudiences value to set. * @return the AzureActiveDirectoryValidation object itself. */ @@ -79,7 +83,7 @@ public AzureActiveDirectoryValidation withAllowedAudiences(List allowedA /** * Get the defaultAuthorizationPolicy property: The configuration settings of the default authorization policy. - * + * * @return the defaultAuthorizationPolicy value. */ public DefaultAuthorizationPolicy defaultAuthorizationPolicy() { @@ -88,19 +92,19 @@ public DefaultAuthorizationPolicy defaultAuthorizationPolicy() { /** * Set the defaultAuthorizationPolicy property: The configuration settings of the default authorization policy. - * + * * @param defaultAuthorizationPolicy the defaultAuthorizationPolicy value to set. * @return the AzureActiveDirectoryValidation object itself. */ - public AzureActiveDirectoryValidation withDefaultAuthorizationPolicy( - DefaultAuthorizationPolicy defaultAuthorizationPolicy) { + public AzureActiveDirectoryValidation + withDefaultAuthorizationPolicy(DefaultAuthorizationPolicy defaultAuthorizationPolicy) { this.defaultAuthorizationPolicy = defaultAuthorizationPolicy; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureCredentials.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureCredentials.java index b2c689859bc01..6e6c7219119e0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureCredentials.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureCredentials.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App credentials. */ +/** + * Container App credentials. + */ @Fluent public final class AzureCredentials { /* @@ -40,13 +42,15 @@ public final class AzureCredentials { @JsonProperty(value = "subscriptionId") private String subscriptionId; - /** Creates an instance of AzureCredentials class. */ + /** + * Creates an instance of AzureCredentials class. + */ public AzureCredentials() { } /** * Get the clientId property: Client Id. - * + * * @return the clientId value. */ public String clientId() { @@ -55,7 +59,7 @@ public String clientId() { /** * Set the clientId property: Client Id. - * + * * @param clientId the clientId value to set. * @return the AzureCredentials object itself. */ @@ -66,7 +70,7 @@ public AzureCredentials withClientId(String clientId) { /** * Get the clientSecret property: Client Secret. - * + * * @return the clientSecret value. */ public String clientSecret() { @@ -75,7 +79,7 @@ public String clientSecret() { /** * Set the clientSecret property: Client Secret. - * + * * @param clientSecret the clientSecret value to set. * @return the AzureCredentials object itself. */ @@ -86,7 +90,7 @@ public AzureCredentials withClientSecret(String clientSecret) { /** * Get the tenantId property: Tenant Id. - * + * * @return the tenantId value. */ public String tenantId() { @@ -95,7 +99,7 @@ public String tenantId() { /** * Set the tenantId property: Tenant Id. - * + * * @param tenantId the tenantId value to set. * @return the AzureCredentials object itself. */ @@ -106,7 +110,7 @@ public AzureCredentials withTenantId(String tenantId) { /** * Get the kind property: Kind of auth github does for deploying the template. - * + * * @return the kind value. */ public String kind() { @@ -115,7 +119,7 @@ public String kind() { /** * Set the kind property: Kind of auth github does for deploying the template. - * + * * @param kind the kind value to set. * @return the AzureCredentials object itself. */ @@ -126,7 +130,7 @@ public AzureCredentials withKind(String kind) { /** * Get the subscriptionId property: Subscription Id. - * + * * @return the subscriptionId value. */ public String subscriptionId() { @@ -135,7 +139,7 @@ public String subscriptionId() { /** * Set the subscriptionId property: Subscription Id. - * + * * @param subscriptionId the subscriptionId value to set. * @return the AzureCredentials object itself. */ @@ -146,7 +150,7 @@ public AzureCredentials withSubscriptionId(String subscriptionId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureFileProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureFileProperties.java index a5aa32a2dc536..5e5551a792386 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureFileProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureFileProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Azure File Properties. */ +/** + * Azure File Properties. + */ @Fluent public final class AzureFileProperties { /* @@ -34,13 +36,15 @@ public final class AzureFileProperties { @JsonProperty(value = "shareName") private String shareName; - /** Creates an instance of AzureFileProperties class. */ + /** + * Creates an instance of AzureFileProperties class. + */ public AzureFileProperties() { } /** * Get the accountName property: Storage account name for azure file. - * + * * @return the accountName value. */ public String accountName() { @@ -49,7 +53,7 @@ public String accountName() { /** * Set the accountName property: Storage account name for azure file. - * + * * @param accountName the accountName value to set. * @return the AzureFileProperties object itself. */ @@ -60,7 +64,7 @@ public AzureFileProperties withAccountName(String accountName) { /** * Get the accountKey property: Storage account key for azure file. - * + * * @return the accountKey value. */ public String accountKey() { @@ -69,7 +73,7 @@ public String accountKey() { /** * Set the accountKey property: Storage account key for azure file. - * + * * @param accountKey the accountKey value to set. * @return the AzureFileProperties object itself. */ @@ -80,7 +84,7 @@ public AzureFileProperties withAccountKey(String accountKey) { /** * Get the accessMode property: Access mode for storage. - * + * * @return the accessMode value. */ public AccessMode accessMode() { @@ -89,7 +93,7 @@ public AccessMode accessMode() { /** * Set the accessMode property: Access mode for storage. - * + * * @param accessMode the accessMode value to set. * @return the AzureFileProperties object itself. */ @@ -100,7 +104,7 @@ public AzureFileProperties withAccessMode(AccessMode accessMode) { /** * Get the shareName property: Azure file share name. - * + * * @return the shareName value. */ public String shareName() { @@ -109,7 +113,7 @@ public String shareName() { /** * Set the shareName property: Azure file share name. - * + * * @param shareName the shareName value to set. * @return the AzureFileProperties object itself. */ @@ -120,7 +124,7 @@ public AzureFileProperties withShareName(String shareName) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebApps.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebApps.java index 8d4937e08dc48..42132f4fac687 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebApps.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebApps.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Azure Static Web Apps provider. */ +/** + * The configuration settings of the Azure Static Web Apps provider. + */ @Fluent public final class AzureStaticWebApps { /* @@ -23,14 +25,16 @@ public final class AzureStaticWebApps { @JsonProperty(value = "registration") private AzureStaticWebAppsRegistration registration; - /** Creates an instance of AzureStaticWebApps class. */ + /** + * Creates an instance of AzureStaticWebApps class. + */ public AzureStaticWebApps() { } /** * Get the enabled property: <code>false</code> if the Azure Static Web Apps provider should not be * enabled despite the set registration; otherwise, <code>true</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -40,7 +44,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>false</code> if the Azure Static Web Apps provider should not be * enabled despite the set registration; otherwise, <code>true</code>. - * + * * @param enabled the enabled value to set. * @return the AzureStaticWebApps object itself. */ @@ -51,7 +55,7 @@ public AzureStaticWebApps withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the Azure Static Web Apps registration. - * + * * @return the registration value. */ public AzureStaticWebAppsRegistration registration() { @@ -60,7 +64,7 @@ public AzureStaticWebAppsRegistration registration() { /** * Set the registration property: The configuration settings of the Azure Static Web Apps registration. - * + * * @param registration the registration value to set. * @return the AzureStaticWebApps object itself. */ @@ -71,7 +75,7 @@ public AzureStaticWebApps withRegistration(AzureStaticWebAppsRegistration regist /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebAppsRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebAppsRegistration.java index cdb01ec16a07d..1d8651ab2d1ee 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebAppsRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/AzureStaticWebAppsRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the registration for the Azure Static Web Apps provider. */ +/** + * The configuration settings of the registration for the Azure Static Web Apps provider. + */ @Fluent public final class AzureStaticWebAppsRegistration { /* @@ -16,13 +18,15 @@ public final class AzureStaticWebAppsRegistration { @JsonProperty(value = "clientId") private String clientId; - /** Creates an instance of AzureStaticWebAppsRegistration class. */ + /** + * Creates an instance of AzureStaticWebAppsRegistration class. + */ public AzureStaticWebAppsRegistration() { } /** * Get the clientId property: The Client ID of the app used for login. - * + * * @return the clientId value. */ public String clientId() { @@ -31,7 +35,7 @@ public String clientId() { /** * Set the clientId property: The Client ID of the app used for login. - * + * * @param clientId the clientId value to set. * @return the AzureStaticWebAppsRegistration object itself. */ @@ -42,7 +46,7 @@ public AzureStaticWebAppsRegistration withClientId(String clientId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BaseContainer.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BaseContainer.java index d82e18e74af14..541432d9b2af0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BaseContainer.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BaseContainer.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App base container definition. */ +/** + * Container App base container definition. + */ @Fluent public class BaseContainer { /* @@ -53,13 +55,15 @@ public class BaseContainer { @JsonProperty(value = "volumeMounts") private List volumeMounts; - /** Creates an instance of BaseContainer class. */ + /** + * Creates an instance of BaseContainer class. + */ public BaseContainer() { } /** * Get the image property: Container image tag. - * + * * @return the image value. */ public String image() { @@ -68,7 +72,7 @@ public String image() { /** * Set the image property: Container image tag. - * + * * @param image the image value to set. * @return the BaseContainer object itself. */ @@ -79,7 +83,7 @@ public BaseContainer withImage(String image) { /** * Get the name property: Custom container name. - * + * * @return the name value. */ public String name() { @@ -88,7 +92,7 @@ public String name() { /** * Set the name property: Custom container name. - * + * * @param name the name value to set. * @return the BaseContainer object itself. */ @@ -99,7 +103,7 @@ public BaseContainer withName(String name) { /** * Get the command property: Container start command. - * + * * @return the command value. */ public List command() { @@ -108,7 +112,7 @@ public List command() { /** * Set the command property: Container start command. - * + * * @param command the command value to set. * @return the BaseContainer object itself. */ @@ -119,7 +123,7 @@ public BaseContainer withCommand(List command) { /** * Get the args property: Container start command arguments. - * + * * @return the args value. */ public List args() { @@ -128,7 +132,7 @@ public List args() { /** * Set the args property: Container start command arguments. - * + * * @param args the args value to set. * @return the BaseContainer object itself. */ @@ -139,7 +143,7 @@ public BaseContainer withArgs(List args) { /** * Get the env property: Container environment variables. - * + * * @return the env value. */ public List env() { @@ -148,7 +152,7 @@ public List env() { /** * Set the env property: Container environment variables. - * + * * @param env the env value to set. * @return the BaseContainer object itself. */ @@ -159,7 +163,7 @@ public BaseContainer withEnv(List env) { /** * Get the resources property: Container resource requirements. - * + * * @return the resources value. */ public ContainerResources resources() { @@ -168,7 +172,7 @@ public ContainerResources resources() { /** * Set the resources property: Container resource requirements. - * + * * @param resources the resources value to set. * @return the BaseContainer object itself. */ @@ -179,7 +183,7 @@ public BaseContainer withResources(ContainerResources resources) { /** * Get the volumeMounts property: Container volume mounts. - * + * * @return the volumeMounts value. */ public List volumeMounts() { @@ -188,7 +192,7 @@ public List volumeMounts() { /** * Set the volumeMounts property: Container volume mounts. - * + * * @param volumeMounts the volumeMounts value to set. * @return the BaseContainer object itself. */ @@ -199,7 +203,7 @@ public BaseContainer withVolumeMounts(List volumeMounts) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeter.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeter.java index 6e78ff4210160..21e8e061d4b88 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeter.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeter.java @@ -6,9 +6,12 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.fasterxml.jackson.annotation.JsonProperty; -/** Billing meter. */ +/** + * Billing meter. + */ @Fluent public final class BillingMeter extends ProxyResource { /* @@ -23,13 +26,21 @@ public final class BillingMeter extends ProxyResource { @JsonProperty(value = "properties") private BillingMeterProperties properties; - /** Creates an instance of BillingMeter class. */ + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of BillingMeter class. + */ public BillingMeter() { } /** * Get the location property: Region for the billing meter. - * + * * @return the location value. */ public String location() { @@ -38,7 +49,7 @@ public String location() { /** * Set the location property: Region for the billing meter. - * + * * @param location the location value to set. * @return the BillingMeter object itself. */ @@ -49,7 +60,7 @@ public BillingMeter withLocation(String location) { /** * Get the properties property: Revision resource specific properties. - * + * * @return the properties value. */ public BillingMeterProperties properties() { @@ -58,7 +69,7 @@ public BillingMeterProperties properties() { /** * Set the properties property: Revision resource specific properties. - * + * * @param properties the properties value to set. * @return the BillingMeter object itself. */ @@ -67,9 +78,18 @@ public BillingMeter withProperties(BillingMeterProperties properties) { return this; } + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterCollection.java index 167deef302edd..7a5c5cf99169f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterCollection.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.appcontainers.fluent.models.BillingMeterCollectionInner; import java.util.List; -/** An immutable client-side representation of BillingMeterCollection. */ +/** + * An immutable client-side representation of BillingMeterCollection. + */ public interface BillingMeterCollection { /** * Gets the value property: Collection of billing meters. - * + * * @return the value value. */ List value(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.BillingMeterCollectionInner object. - * + * * @return the inner object. */ BillingMeterCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterProperties.java index 00e2c5d0fed52..e8ab0e0c1fcc4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeterProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Revision resource specific properties. */ +/** + * Revision resource specific properties. + */ @Fluent public final class BillingMeterProperties { /* @@ -28,13 +30,15 @@ public final class BillingMeterProperties { @JsonProperty(value = "displayName") private String displayName; - /** Creates an instance of BillingMeterProperties class. */ + /** + * Creates an instance of BillingMeterProperties class. + */ public BillingMeterProperties() { } /** * Get the category property: Used to categorize billing meters. - * + * * @return the category value. */ public String category() { @@ -43,7 +47,7 @@ public String category() { /** * Set the category property: Used to categorize billing meters. - * + * * @param category the category value to set. * @return the BillingMeterProperties object itself. */ @@ -54,7 +58,7 @@ public BillingMeterProperties withCategory(String category) { /** * Get the meterType property: Billing meter type. - * + * * @return the meterType value. */ public String meterType() { @@ -63,7 +67,7 @@ public String meterType() { /** * Set the meterType property: Billing meter type. - * + * * @param meterType the meterType value to set. * @return the BillingMeterProperties object itself. */ @@ -74,7 +78,7 @@ public BillingMeterProperties withMeterType(String meterType) { /** * Get the displayName property: The everyday name of the billing meter. - * + * * @return the displayName value. */ public String displayName() { @@ -83,7 +87,7 @@ public String displayName() { /** * Set the displayName property: The everyday name of the billing meter. - * + * * @param displayName the displayName value to set. * @return the BillingMeterProperties object itself. */ @@ -94,7 +98,7 @@ public BillingMeterProperties withDisplayName(String displayName) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeters.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeters.java index c67c45e0e69df..a993eeac3855d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeters.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BillingMeters.java @@ -7,18 +7,20 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of BillingMeters. */ +/** + * Resource collection API of BillingMeters. + */ public interface BillingMeters { /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all billingMeters for a location along with {@link Response}. */ @@ -26,13 +28,13 @@ public interface BillingMeters { /** * Get billing meters by location. - * - *

Get all billingMeters for a location. - * + * + * Get all billingMeters for a location. + * * @param location The name of Azure region. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all billingMeters for a location. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BindingType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BindingType.java index 8e207dccbd689..6683cd637112c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BindingType.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BindingType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Custom Domain binding type. */ +/** + * Custom Domain binding type. + */ public final class BindingType extends ExpandableStringEnum { - /** Static value Disabled for BindingType. */ + /** + * Static value Disabled for BindingType. + */ public static final BindingType DISABLED = fromString("Disabled"); - /** Static value SniEnabled for BindingType. */ + /** + * Static value SniEnabled for BindingType. + */ public static final BindingType SNI_ENABLED = fromString("SniEnabled"); /** * Creates a new instance of BindingType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public BindingType() { /** * Creates or finds a BindingType from its string representation. - * + * * @param name a name to look for. * @return the corresponding BindingType. */ @@ -38,7 +44,7 @@ public static BindingType fromString(String name) { /** * Gets known BindingType values. - * + * * @return known BindingType values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BlobStorageTokenStore.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BlobStorageTokenStore.java new file mode 100644 index 0000000000000..ff9b6b4d46bcd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BlobStorageTokenStore.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The configuration settings of the storage of the tokens if blob storage is used. + */ +@Fluent +public final class BlobStorageTokenStore { + /* + * The name of the app secrets containing the SAS URL of the blob storage containing the tokens. + */ + @JsonProperty(value = "sasUrlSettingName", required = true) + private String sasUrlSettingName; + + /** + * Creates an instance of BlobStorageTokenStore class. + */ + public BlobStorageTokenStore() { + } + + /** + * Get the sasUrlSettingName property: The name of the app secrets containing the SAS URL of the blob storage + * containing the tokens. + * + * @return the sasUrlSettingName value. + */ + public String sasUrlSettingName() { + return this.sasUrlSettingName; + } + + /** + * Set the sasUrlSettingName property: The name of the app secrets containing the SAS URL of the blob storage + * containing the tokens. + * + * @param sasUrlSettingName the sasUrlSettingName value to set. + * @return the BlobStorageTokenStore object itself. + */ + public BlobStorageTokenStore withSasUrlSettingName(String sasUrlSettingName) { + this.sasUrlSettingName = sasUrlSettingName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (sasUrlSettingName() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sasUrlSettingName in model BlobStorageTokenStore")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(BlobStorageTokenStore.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildAuthTokens.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildAuthTokens.java new file mode 100644 index 0000000000000..ac5dd37d1d4ba --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildAuthTokens.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of BuildAuthTokens. + */ +public interface BuildAuthTokens { + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build along with + * {@link Response}. + */ + Response listWithResponse(String resourceGroupName, String builderName, String buildName, + Context context); + + /** + * Gets the token used to connect to the endpoint where source code can be uploaded for a build. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the token used to connect to the endpoint where source code can be uploaded for a build. + */ + BuildToken list(String resourceGroupName, String builderName, String buildName); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildCollection.java new file mode 100644 index 0000000000000..bb7e4f45c623f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildCollection.java @@ -0,0 +1,91 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The response of a BuildResource list operation. + */ +@Fluent +public final class BuildCollection { + /* + * The BuildResource items on this page + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * The link to the next page of items + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of BuildCollection class. + */ + public BuildCollection() { + } + + /** + * Get the value property: The BuildResource items on this page. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: The BuildResource items on this page. + * + * @param value the value value to set. + * @return the BuildCollection object itself. + */ + public BuildCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: The link to the next page of items. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: The link to the next page of items. + * + * @param nextLink the nextLink value to set. + * @return the BuildCollection object itself. + */ + public BuildCollection withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model BuildCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(BuildCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildConfiguration.java new file mode 100644 index 0000000000000..20aa5a04a517c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildConfiguration.java @@ -0,0 +1,167 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration of the build. + */ +@Fluent +public final class BuildConfiguration { + /* + * Base OS used to build and run the app. + */ + @JsonProperty(value = "baseOs") + private String baseOs; + + /* + * Platform to be used to build and run the app. + */ + @JsonProperty(value = "platform") + private String platform; + + /* + * Platform version to be used to build and run the app. + */ + @JsonProperty(value = "platformVersion") + private String platformVersion; + + /* + * List of environment variables to be passed to the build, secrets should not be used in environment variable. + */ + @JsonProperty(value = "environmentVariables") + private List environmentVariables; + + /* + * List of steps to perform before the build. + */ + @JsonProperty(value = "preBuildSteps") + private List preBuildSteps; + + /** + * Creates an instance of BuildConfiguration class. + */ + public BuildConfiguration() { + } + + /** + * Get the baseOs property: Base OS used to build and run the app. + * + * @return the baseOs value. + */ + public String baseOs() { + return this.baseOs; + } + + /** + * Set the baseOs property: Base OS used to build and run the app. + * + * @param baseOs the baseOs value to set. + * @return the BuildConfiguration object itself. + */ + public BuildConfiguration withBaseOs(String baseOs) { + this.baseOs = baseOs; + return this; + } + + /** + * Get the platform property: Platform to be used to build and run the app. + * + * @return the platform value. + */ + public String platform() { + return this.platform; + } + + /** + * Set the platform property: Platform to be used to build and run the app. + * + * @param platform the platform value to set. + * @return the BuildConfiguration object itself. + */ + public BuildConfiguration withPlatform(String platform) { + this.platform = platform; + return this; + } + + /** + * Get the platformVersion property: Platform version to be used to build and run the app. + * + * @return the platformVersion value. + */ + public String platformVersion() { + return this.platformVersion; + } + + /** + * Set the platformVersion property: Platform version to be used to build and run the app. + * + * @param platformVersion the platformVersion value to set. + * @return the BuildConfiguration object itself. + */ + public BuildConfiguration withPlatformVersion(String platformVersion) { + this.platformVersion = platformVersion; + return this; + } + + /** + * Get the environmentVariables property: List of environment variables to be passed to the build, secrets should + * not be used in environment variable. + * + * @return the environmentVariables value. + */ + public List environmentVariables() { + return this.environmentVariables; + } + + /** + * Set the environmentVariables property: List of environment variables to be passed to the build, secrets should + * not be used in environment variable. + * + * @param environmentVariables the environmentVariables value to set. + * @return the BuildConfiguration object itself. + */ + public BuildConfiguration withEnvironmentVariables(List environmentVariables) { + this.environmentVariables = environmentVariables; + return this; + } + + /** + * Get the preBuildSteps property: List of steps to perform before the build. + * + * @return the preBuildSteps value. + */ + public List preBuildSteps() { + return this.preBuildSteps; + } + + /** + * Set the preBuildSteps property: List of steps to perform before the build. + * + * @param preBuildSteps the preBuildSteps value to set. + * @return the BuildConfiguration object itself. + */ + public BuildConfiguration withPreBuildSteps(List preBuildSteps) { + this.preBuildSteps = preBuildSteps; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (environmentVariables() != null) { + environmentVariables().forEach(e -> e.validate()); + } + if (preBuildSteps() != null) { + preBuildSteps().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildProvisioningState.java new file mode 100644 index 0000000000000..64fbc890e6760 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildProvisioningState.java @@ -0,0 +1,73 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Resource instance provisioning state. + */ +public final class BuildProvisioningState extends ExpandableStringEnum { + /** + * Static value Succeeded for BuildProvisioningState. + */ + public static final BuildProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Failed for BuildProvisioningState. + */ + public static final BuildProvisioningState FAILED = fromString("Failed"); + + /** + * Static value Canceled for BuildProvisioningState. + */ + public static final BuildProvisioningState CANCELED = fromString("Canceled"); + + /** + * Static value Creating for BuildProvisioningState. + */ + public static final BuildProvisioningState CREATING = fromString("Creating"); + + /** + * Static value Updating for BuildProvisioningState. + */ + public static final BuildProvisioningState UPDATING = fromString("Updating"); + + /** + * Static value Deleting for BuildProvisioningState. + */ + public static final BuildProvisioningState DELETING = fromString("Deleting"); + + /** + * Creates a new instance of BuildProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public BuildProvisioningState() { + } + + /** + * Creates or finds a BuildProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding BuildProvisioningState. + */ + @JsonCreator + public static BuildProvisioningState fromString(String name) { + return fromString(name, BuildProvisioningState.class); + } + + /** + * Gets known BuildProvisioningState values. + * + * @return known BuildProvisioningState values. + */ + public static Collection values() { + return values(BuildProvisioningState.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildResource.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildResource.java new file mode 100644 index 0000000000000..2917c702fa3f7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildResource.java @@ -0,0 +1,261 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner; + +/** + * An immutable client-side representation of BuildResource. + */ +public interface BuildResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the provisioningState property: Build provisioning state. + * + * @return the provisioningState value. + */ + BuildProvisioningState provisioningState(); + + /** + * Gets the buildStatus property: Status of the build once it has been provisioned. + * + * @return the buildStatus value. + */ + BuildStatus buildStatus(); + + /** + * Gets the destinationContainerRegistry property: Container registry that the final image will be uploaded to. + * + * @return the destinationContainerRegistry value. + */ + ContainerRegistryWithCustomImage destinationContainerRegistry(); + + /** + * Gets the configuration property: Configuration of the build. + * + * @return the configuration value. + */ + BuildConfiguration configuration(); + + /** + * Gets the uploadEndpoint property: Endpoint to which the source code should be uploaded. + * + * @return the uploadEndpoint value. + */ + String uploadEndpoint(); + + /** + * Gets the logStreamEndpoint property: Endpoint from which the build logs can be streamed. + * + * @return the logStreamEndpoint value. + */ + String logStreamEndpoint(); + + /** + * Gets the tokenEndpoint property: Endpoint to use to retrieve an authentication token for log streaming and + * uploading source code. + * + * @return the tokenEndpoint value. + */ + String tokenEndpoint(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner object. + * + * @return the inner object. + */ + BuildResourceInner innerModel(); + + /** + * The entirety of the BuildResource definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The BuildResource definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the BuildResource definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the BuildResource definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, builderName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @return the next definition stage. + */ + WithCreate withExistingBuilder(String resourceGroupName, String builderName); + } + + /** + * The stage of the BuildResource definition which contains all the minimum required properties for the resource + * to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate + extends DefinitionStages.WithDestinationContainerRegistry, DefinitionStages.WithConfiguration { + /** + * Executes the create request. + * + * @return the created resource. + */ + BuildResource create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + BuildResource create(Context context); + } + + /** + * The stage of the BuildResource definition allowing to specify destinationContainerRegistry. + */ + interface WithDestinationContainerRegistry { + /** + * Specifies the destinationContainerRegistry property: Container registry that the final image will be + * uploaded to.. + * + * @param destinationContainerRegistry Container registry that the final image will be uploaded to. + * @return the next definition stage. + */ + WithCreate withDestinationContainerRegistry(ContainerRegistryWithCustomImage destinationContainerRegistry); + } + + /** + * The stage of the BuildResource definition allowing to specify configuration. + */ + interface WithConfiguration { + /** + * Specifies the configuration property: Configuration of the build.. + * + * @param configuration Configuration of the build. + * @return the next definition stage. + */ + WithCreate withConfiguration(BuildConfiguration configuration); + } + } + + /** + * Begins update for the BuildResource resource. + * + * @return the stage of resource update. + */ + BuildResource.Update update(); + + /** + * The template for BuildResource update. + */ + interface Update extends UpdateStages.WithDestinationContainerRegistry, UpdateStages.WithConfiguration { + /** + * Executes the update request. + * + * @return the updated resource. + */ + BuildResource apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + BuildResource apply(Context context); + } + + /** + * The BuildResource update stages. + */ + interface UpdateStages { + /** + * The stage of the BuildResource update allowing to specify destinationContainerRegistry. + */ + interface WithDestinationContainerRegistry { + /** + * Specifies the destinationContainerRegistry property: Container registry that the final image will be + * uploaded to.. + * + * @param destinationContainerRegistry Container registry that the final image will be uploaded to. + * @return the next definition stage. + */ + Update withDestinationContainerRegistry(ContainerRegistryWithCustomImage destinationContainerRegistry); + } + + /** + * The stage of the BuildResource update allowing to specify configuration. + */ + interface WithConfiguration { + /** + * Specifies the configuration property: Configuration of the build.. + * + * @param configuration Configuration of the build. + * @return the next definition stage. + */ + Update withConfiguration(BuildConfiguration configuration); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + BuildResource refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + BuildResource refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildStatus.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildStatus.java new file mode 100644 index 0000000000000..77c849cce275b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildStatus.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Status of the build once it has been provisioned. + */ +public final class BuildStatus extends ExpandableStringEnum { + /** + * Static value NotStarted for BuildStatus. + */ + public static final BuildStatus NOT_STARTED = fromString("NotStarted"); + + /** + * Static value InProgress for BuildStatus. + */ + public static final BuildStatus IN_PROGRESS = fromString("InProgress"); + + /** + * Static value Succeeded for BuildStatus. + */ + public static final BuildStatus SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Canceled for BuildStatus. + */ + public static final BuildStatus CANCELED = fromString("Canceled"); + + /** + * Static value Failed for BuildStatus. + */ + public static final BuildStatus FAILED = fromString("Failed"); + + /** + * Creates a new instance of BuildStatus value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public BuildStatus() { + } + + /** + * Creates or finds a BuildStatus from its string representation. + * + * @param name a name to look for. + * @return the corresponding BuildStatus. + */ + @JsonCreator + public static BuildStatus fromString(String name) { + return fromString(name, BuildStatus.class); + } + + /** + * Gets known BuildStatus values. + * + * @return known BuildStatus values. + */ + public static Collection values() { + return values(BuildStatus.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildToken.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildToken.java new file mode 100644 index 0000000000000..31786178e9d9e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildToken.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner; +import java.time.OffsetDateTime; + +/** + * An immutable client-side representation of BuildToken. + */ +public interface BuildToken { + /** + * Gets the token property: Authentication token. + * + * @return the token value. + */ + String token(); + + /** + * Gets the expires property: Token expiration date. + * + * @return the expires value. + */ + OffsetDateTime expires(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner object. + * + * @return the inner object. + */ + BuildTokenInner innerModel(); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderCollection.java new file mode 100644 index 0000000000000..f82e7ac7e8b6b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderCollection.java @@ -0,0 +1,91 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The response of a BuilderResource list operation. + */ +@Fluent +public final class BuilderCollection { + /* + * The BuilderResource items on this page + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * The link to the next page of items + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of BuilderCollection class. + */ + public BuilderCollection() { + } + + /** + * Get the value property: The BuilderResource items on this page. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: The BuilderResource items on this page. + * + * @param value the value value to set. + * @return the BuilderCollection object itself. + */ + public BuilderCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: The link to the next page of items. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: The link to the next page of items. + * + * @param nextLink the nextLink value to set. + * @return the BuilderCollection object itself. + */ + public BuilderCollection withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model BuilderCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(BuilderCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderProvisioningState.java new file mode 100644 index 0000000000000..2a8293b79a564 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderProvisioningState.java @@ -0,0 +1,73 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Resource instance provisioning state. + */ +public final class BuilderProvisioningState extends ExpandableStringEnum { + /** + * Static value Succeeded for BuilderProvisioningState. + */ + public static final BuilderProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Failed for BuilderProvisioningState. + */ + public static final BuilderProvisioningState FAILED = fromString("Failed"); + + /** + * Static value Canceled for BuilderProvisioningState. + */ + public static final BuilderProvisioningState CANCELED = fromString("Canceled"); + + /** + * Static value Creating for BuilderProvisioningState. + */ + public static final BuilderProvisioningState CREATING = fromString("Creating"); + + /** + * Static value Updating for BuilderProvisioningState. + */ + public static final BuilderProvisioningState UPDATING = fromString("Updating"); + + /** + * Static value Deleting for BuilderProvisioningState. + */ + public static final BuilderProvisioningState DELETING = fromString("Deleting"); + + /** + * Creates a new instance of BuilderProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public BuilderProvisioningState() { + } + + /** + * Creates or finds a BuilderProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding BuilderProvisioningState. + */ + @JsonCreator + public static BuilderProvisioningState fromString(String name) { + return fromString(name, BuilderProvisioningState.class); + } + + /** + * Gets known BuilderProvisioningState values. + * + * @return known BuilderProvisioningState values. + */ + public static Collection values() { + return values(BuilderProvisioningState.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResource.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResource.java new file mode 100644 index 0000000000000..1d81dcc8f511e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResource.java @@ -0,0 +1,333 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.Region; +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import java.util.List; +import java.util.Map; + +/** + * An immutable client-side representation of BuilderResource. + */ +public interface BuilderResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the location property: The geo-location where the resource lives. + * + * @return the location value. + */ + String location(); + + /** + * Gets the tags property: Resource tags. + * + * @return the tags value. + */ + Map tags(); + + /** + * Gets the identity property: The managed service identities assigned to this resource. + * + * @return the identity value. + */ + ManagedServiceIdentity identity(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the provisioningState property: Provisioning state of a builder resource. + * + * @return the provisioningState value. + */ + BuilderProvisioningState provisioningState(); + + /** + * Gets the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @return the environmentId value. + */ + String environmentId(); + + /** + * Gets the containerRegistries property: List of mappings of container registries and the managed identity used to + * connect to it. + * + * @return the containerRegistries value. + */ + List containerRegistries(); + + /** + * Gets the region of the resource. + * + * @return the region of the resource. + */ + Region region(); + + /** + * Gets the name of the resource region. + * + * @return the name of the resource region. + */ + String regionName(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner object. + * + * @return the inner object. + */ + BuilderResourceInner innerModel(); + + /** + * The entirety of the BuilderResource definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { + } + + /** + * The BuilderResource definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the BuilderResource definition. + */ + interface Blank extends WithLocation { + } + + /** + * The stage of the BuilderResource definition allowing to specify location. + */ + interface WithLocation { + /** + * Specifies the region for the resource. + * + * @param location The geo-location where the resource lives. + * @return the next definition stage. + */ + WithResourceGroup withRegion(Region location); + + /** + * Specifies the region for the resource. + * + * @param location The geo-location where the resource lives. + * @return the next definition stage. + */ + WithResourceGroup withRegion(String location); + } + + /** + * The stage of the BuilderResource definition allowing to specify parent resource. + */ + interface WithResourceGroup { + /** + * Specifies resourceGroupName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @return the next definition stage. + */ + WithCreate withExistingResourceGroup(String resourceGroupName); + } + + /** + * The stage of the BuilderResource definition which contains all the minimum required properties for the + * resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithEnvironmentId, DefinitionStages.WithContainerRegistries { + /** + * Executes the create request. + * + * @return the created resource. + */ + BuilderResource create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + BuilderResource create(Context context); + } + + /** + * The stage of the BuilderResource definition allowing to specify tags. + */ + interface WithTags { + /** + * Specifies the tags property: Resource tags.. + * + * @param tags Resource tags. + * @return the next definition stage. + */ + WithCreate withTags(Map tags); + } + + /** + * The stage of the BuilderResource definition allowing to specify identity. + */ + interface WithIdentity { + /** + * Specifies the identity property: The managed service identities assigned to this resource.. + * + * @param identity The managed service identities assigned to this resource. + * @return the next definition stage. + */ + WithCreate withIdentity(ManagedServiceIdentity identity); + } + + /** + * The stage of the BuilderResource definition allowing to specify environmentId. + */ + interface WithEnvironmentId { + /** + * Specifies the environmentId property: Resource ID of the container apps environment that the builder is + * associated with.. + * + * @param environmentId Resource ID of the container apps environment that the builder is associated with. + * @return the next definition stage. + */ + WithCreate withEnvironmentId(String environmentId); + } + + /** + * The stage of the BuilderResource definition allowing to specify containerRegistries. + */ + interface WithContainerRegistries { + /** + * Specifies the containerRegistries property: List of mappings of container registries and the managed + * identity used to connect to it.. + * + * @param containerRegistries List of mappings of container registries and the managed identity used to + * connect to it. + * @return the next definition stage. + */ + WithCreate withContainerRegistries(List containerRegistries); + } + } + + /** + * Begins update for the BuilderResource resource. + * + * @return the stage of resource update. + */ + BuilderResource.Update update(); + + /** + * The template for BuilderResource update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity, UpdateStages.WithEnvironmentId { + /** + * Executes the update request. + * + * @return the updated resource. + */ + BuilderResource apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + BuilderResource apply(Context context); + } + + /** + * The BuilderResource update stages. + */ + interface UpdateStages { + /** + * The stage of the BuilderResource update allowing to specify tags. + */ + interface WithTags { + /** + * Specifies the tags property: Resource tags.. + * + * @param tags Resource tags. + * @return the next definition stage. + */ + Update withTags(Map tags); + } + + /** + * The stage of the BuilderResource update allowing to specify identity. + */ + interface WithIdentity { + /** + * Specifies the identity property: The managed service identities assigned to this resource.. + * + * @param identity The managed service identities assigned to this resource. + * @return the next definition stage. + */ + Update withIdentity(ManagedServiceIdentity identity); + } + + /** + * The stage of the BuilderResource update allowing to specify environmentId. + */ + interface WithEnvironmentId { + /** + * Specifies the environmentId property: Resource ID of the container apps environment that the builder is + * associated with.. + * + * @param environmentId Resource ID of the container apps environment that the builder is associated with. + * @return the next definition stage. + */ + Update withEnvironmentId(String environmentId); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + BuilderResource refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + BuilderResource refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResourceUpdate.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResourceUpdate.java new file mode 100644 index 0000000000000..32c3b8c07a3ee --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuilderResourceUpdate.java @@ -0,0 +1,130 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceUpdateProperties; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; + +/** + * The type used for update operations of the BuilderResource. + */ +@Fluent +public final class BuilderResourceUpdate { + /* + * The managed service identities assigned to this resource. + */ + @JsonProperty(value = "identity") + private ManagedServiceIdentity identity; + + /* + * Resource tags. + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; + + /* + * The updatable properties of the BuilderResource. + */ + @JsonProperty(value = "properties") + private BuilderResourceUpdateProperties innerProperties; + + /** + * Creates an instance of BuilderResourceUpdate class. + */ + public BuilderResourceUpdate() { + } + + /** + * Get the identity property: The managed service identities assigned to this resource. + * + * @return the identity value. + */ + public ManagedServiceIdentity identity() { + return this.identity; + } + + /** + * Set the identity property: The managed service identities assigned to this resource. + * + * @param identity the identity value to set. + * @return the BuilderResourceUpdate object itself. + */ + public BuilderResourceUpdate withIdentity(ManagedServiceIdentity identity) { + this.identity = identity; + return this; + } + + /** + * Get the tags property: Resource tags. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; + } + + /** + * Set the tags property: Resource tags. + * + * @param tags the tags value to set. + * @return the BuilderResourceUpdate object itself. + */ + public BuilderResourceUpdate withTags(Map tags) { + this.tags = tags; + return this; + } + + /** + * Get the innerProperties property: The updatable properties of the BuilderResource. + * + * @return the innerProperties value. + */ + private BuilderResourceUpdateProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @return the environmentId value. + */ + public String environmentId() { + return this.innerProperties() == null ? null : this.innerProperties().environmentId(); + } + + /** + * Set the environmentId property: Resource ID of the container apps environment that the builder is associated + * with. + * + * @param environmentId the environmentId value to set. + * @return the BuilderResourceUpdate object itself. + */ + public BuilderResourceUpdate withEnvironmentId(String environmentId) { + if (this.innerProperties() == null) { + this.innerProperties = new BuilderResourceUpdateProperties(); + } + this.innerProperties().withEnvironmentId(environmentId); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (identity() != null) { + identity().validate(); + } + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionLists.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builders.java similarity index 55% rename from sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionLists.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builders.java index bf22eb6235c95..d91007741556b 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionLists.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builders.java @@ -2,150 +2,133 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.models; +package com.azure.resourcemanager.appcontainers.models; import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; /** - * Resource collection API of SuppressionLists. + * Resource collection API of Builders. */ -public interface SuppressionLists { +public interface Builders { /** - * List + * List BuilderResource resources by subscription ID. * - * List all suppression lists for a domains resource. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + PagedIterable list(); + + /** + * List BuilderResource resources by subscription ID. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedIterable}. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName); + PagedIterable list(Context context); /** - * List + * List BuilderResource resources by resource group. * - * List all suppression lists for a domains resource. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. + */ + PagedIterable listByResourceGroup(String resourceGroupName); + + /** + * List BuilderResource resources by resource group. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedIterable}. + * @return the response of a BuilderResource list operation as paginated response with {@link PagedIterable}. */ - PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName, Context context); + PagedIterable listByResourceGroup(String resourceGroupName, Context context); /** - * Get - * - * Get a SuppressionList resource. + * Get a BuilderResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. + * @param builderName The name of the builder. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response}. + * @return a BuilderResource along with {@link Response}. */ - Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String builderName, + Context context); /** - * Get - * - * Get a SuppressionList resource. + * Get a BuilderResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. + * @param builderName The name of the builder. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource. + * @return a BuilderResource. */ - SuppressionListResource get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName); + BuilderResource getByResourceGroup(String resourceGroupName, String builderName); /** - * Delete - * - * Delete a SuppressionList. + * Delete a BuilderResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. + * @param builderName The name of the builder. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. */ - Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, Context context); + void deleteByResourceGroup(String resourceGroupName, String builderName); /** - * Delete - * - * Delete a SuppressionList. + * Delete a BuilderResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. + * @param builderName The name of the builder. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete(String resourceGroupName, String emailServiceName, String domainName, String suppressionListName); + void delete(String resourceGroupName, String builderName, Context context); /** - * Get - * - * Get a SuppressionList resource. + * Get a BuilderResource. * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response}. + * @return a BuilderResource along with {@link Response}. */ - SuppressionListResource getById(String id); + BuilderResource getById(String id); /** - * Get - * - * Get a SuppressionList resource. + * Get a BuilderResource. * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response}. + * @return a BuilderResource along with {@link Response}. */ - Response getByIdWithResponse(String id, Context context); + Response getByIdWithResponse(String id, Context context); /** - * Delete - * - * Delete a SuppressionList. + * Delete a BuilderResource. * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -155,24 +138,21 @@ Response deleteWithResponse(String resourceGroupName, String emailServiceN void deleteById(String id); /** - * Delete - * - * Delete a SuppressionList. + * Delete a BuilderResource. * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. */ - Response deleteByIdWithResponse(String id, Context context); + void deleteByIdWithResponse(String id, Context context); /** - * Begins definition for a new SuppressionListResource resource. + * Begins definition for a new BuilderResource resource. * * @param name resource name. - * @return the first stage of the new SuppressionListResource definition. + * @return the first stage of the new BuilderResource definition. */ - SuppressionListResource.DefinitionStages.Blank define(String name); + BuilderResource.DefinitionStages.Blank define(String name); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builds.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builds.java new file mode 100644 index 0000000000000..e00e51a76b575 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Builds.java @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of Builds. + */ +public interface Builds { + /** + * Get a BuildResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuildResource along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String builderName, String buildName, + Context context); + + /** + * Get a BuildResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuildResource. + */ + BuildResource get(String resourceGroupName, String builderName, String buildName); + + /** + * Delete a BuildResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String builderName, String buildName); + + /** + * Delete a BuildResource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param builderName The name of the builder. + * @param buildName The name of a build. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String builderName, String buildName, Context context); + + /** + * Get a BuildResource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuildResource along with {@link Response}. + */ + BuildResource getById(String id); + + /** + * Get a BuildResource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a BuildResource along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Delete a BuildResource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteById(String id); + + /** + * Delete a BuildResource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new BuildResource resource. + * + * @param name resource name. + * @return the first stage of the new BuildResource definition. + */ + BuildResource.DefinitionStages.Blank define(String name); +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderAccounts.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildsByBuilderResources.java similarity index 53% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderAccounts.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildsByBuilderResources.java index 2f0176a3561a1..f0ff7e81d7042 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderAccounts.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/BuildsByBuilderResources.java @@ -2,40 +2,37 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.models; +package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; /** - * Resource collection API of BackupsUnderAccounts. + * Resource collection API of BuildsByBuilderResources. */ -public interface BackupsUnderAccounts { +public interface BuildsByBuilderResources { /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. + * List BuildResource resources by BuilderResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. + * @param builderName The name of the builder. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation as paginated response with {@link PagedIterable}. */ - void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body); + PagedIterable list(String resourceGroupName, String builderName); /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. + * List BuildResource resources by BuilderResource. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. + * @param builderName The name of the builder. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a BuildResource list operation as paginated response with {@link PagedIterable}. */ - void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body, Context context); + PagedIterable list(String resourceGroupName, String builderName, Context context); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificate.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificate.java index e25331db625d7..1407efe1b2440 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificate.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificate.java @@ -10,104 +10,111 @@ import com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner; import java.util.Map; -/** An immutable client-side representation of Certificate. */ +/** + * An immutable client-side representation of Certificate. + */ public interface Certificate { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the properties property: Certificate resource specific properties. - * + * * @return the properties value. */ CertificateProperties properties(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner object. - * + * * @return the inner object. */ CertificateInner innerModel(); - /** The entirety of the Certificate definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithCreate { + /** + * The entirety of the Certificate definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Certificate definition stages. */ + /** + * The Certificate definition stages. + */ interface DefinitionStages { - /** The first stage of the Certificate definition. */ + /** + * The first stage of the Certificate definition. + */ interface Blank extends WithLocation { } - /** The stage of the Certificate definition allowing to specify location. */ + /** + * The stage of the Certificate definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -115,18 +122,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the Certificate definition allowing to specify parent resource. */ + /** + * The stage of the Certificate definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, connectedEnvironmentName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @return the next definition stage. @@ -141,36 +150,40 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ Certificate create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Certificate create(Context context); } - /** The stage of the Certificate definition allowing to specify tags. */ + /** + * The stage of the Certificate definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Certificate definition allowing to specify properties. */ + /** + * The stage of the Certificate definition allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: Certificate resource specific properties. - * + * * @param properties Certificate resource specific properties. * @return the next definition stage. */ @@ -180,36 +193,42 @@ interface WithProperties { /** * Begins update for the Certificate resource. - * + * * @return the stage of resource update. */ Certificate.Update update(); - /** The template for Certificate update. */ + /** + * The template for Certificate update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ Certificate apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Certificate apply(Context context); } - /** The Certificate update stages. */ + /** + * The Certificate update stages. + */ interface UpdateStages { - /** The stage of the Certificate update allowing to specify tags. */ + /** + * The stage of the Certificate update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Application-specific metadata in the form of key-value pairs.. - * + * * @param tags Application-specific metadata in the form of key-value pairs. * @return the next definition stage. */ @@ -219,14 +238,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Certificate refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateCollection.java index b2d35859d8520..4b2427f101cc8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of Certificates. */ +/** + * Collection of Certificates. + */ @Fluent public final class CertificateCollection { /* @@ -25,13 +27,15 @@ public final class CertificateCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of CertificateCollection class. */ + /** + * Creates an instance of CertificateCollection class. + */ public CertificateCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the CertificateCollection object itself. */ @@ -51,7 +55,7 @@ public CertificateCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model CertificateCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model CertificateCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateKeyVaultProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateKeyVaultProperties.java new file mode 100644 index 0000000000000..77c3a421c24d3 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateKeyVaultProperties.java @@ -0,0 +1,83 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Properties for a certificate stored in a Key Vault. + */ +@Fluent +public final class CertificateKeyVaultProperties { + /* + * Resource ID of a managed identity to authenticate with Azure Key Vault, or System to use a system-assigned + * identity. + */ + @JsonProperty(value = "identity") + private String identity; + + /* + * URL pointing to the Azure Key Vault secret that holds the certificate. + */ + @JsonProperty(value = "keyVaultUrl") + private String keyVaultUrl; + + /** + * Creates an instance of CertificateKeyVaultProperties class. + */ + public CertificateKeyVaultProperties() { + } + + /** + * Get the identity property: Resource ID of a managed identity to authenticate with Azure Key Vault, or System to + * use a system-assigned identity. + * + * @return the identity value. + */ + public String identity() { + return this.identity; + } + + /** + * Set the identity property: Resource ID of a managed identity to authenticate with Azure Key Vault, or System to + * use a system-assigned identity. + * + * @param identity the identity value to set. + * @return the CertificateKeyVaultProperties object itself. + */ + public CertificateKeyVaultProperties withIdentity(String identity) { + this.identity = identity; + return this; + } + + /** + * Get the keyVaultUrl property: URL pointing to the Azure Key Vault secret that holds the certificate. + * + * @return the keyVaultUrl value. + */ + public String keyVaultUrl() { + return this.keyVaultUrl; + } + + /** + * Set the keyVaultUrl property: URL pointing to the Azure Key Vault secret that holds the certificate. + * + * @param keyVaultUrl the keyVaultUrl value to set. + * @return the CertificateKeyVaultProperties object itself. + */ + public CertificateKeyVaultProperties withKeyVaultUrl(String keyVaultUrl) { + this.keyVaultUrl = keyVaultUrl; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificatePatch.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificatePatch.java index f567a5f4f7d8c..d1be80c2c81da 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificatePatch.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificatePatch.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** A certificate to update. */ +/** + * A certificate to update. + */ @Fluent public final class CertificatePatch { /* @@ -19,13 +21,15 @@ public final class CertificatePatch { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of CertificatePatch class. */ + /** + * Creates an instance of CertificatePatch class. + */ public CertificatePatch() { } /** * Get the tags property: Application-specific metadata in the form of key-value pairs. - * + * * @return the tags value. */ public Map tags() { @@ -34,7 +38,7 @@ public Map tags() { /** * Set the tags property: Application-specific metadata in the form of key-value pairs. - * + * * @param tags the tags value to set. * @return the CertificatePatch object itself. */ @@ -45,7 +49,7 @@ public CertificatePatch withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProperties.java index efc126b11fcb1..ef82a7bb28b5f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProperties.java @@ -10,7 +10,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Certificate resource specific properties. */ +/** + * Certificate resource specific properties. + */ @Fluent public final class CertificateProperties { /* @@ -19,6 +21,12 @@ public final class CertificateProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private CertificateProvisioningState provisioningState; + /* + * Properties for a certificate stored in a Key Vault. + */ + @JsonProperty(value = "certificateKeyVaultProperties") + private CertificateKeyVaultProperties certificateKeyVaultProperties; + /* * Certificate password. */ @@ -79,22 +87,51 @@ public final class CertificateProperties { @JsonProperty(value = "publicKeyHash", access = JsonProperty.Access.WRITE_ONLY) private String publicKeyHash; - /** Creates an instance of CertificateProperties class. */ + /* + * The type of the certificate. Allowed values are `ServerSSLCertificate` and `ImagePullTrustedCA` + */ + @JsonProperty(value = "certificateType") + private CertificateType certificateType; + + /** + * Creates an instance of CertificateProperties class. + */ public CertificateProperties() { } /** * Get the provisioningState property: Provisioning state of the certificate. - * + * * @return the provisioningState value. */ public CertificateProvisioningState provisioningState() { return this.provisioningState; } + /** + * Get the certificateKeyVaultProperties property: Properties for a certificate stored in a Key Vault. + * + * @return the certificateKeyVaultProperties value. + */ + public CertificateKeyVaultProperties certificateKeyVaultProperties() { + return this.certificateKeyVaultProperties; + } + + /** + * Set the certificateKeyVaultProperties property: Properties for a certificate stored in a Key Vault. + * + * @param certificateKeyVaultProperties the certificateKeyVaultProperties value to set. + * @return the CertificateProperties object itself. + */ + public CertificateProperties + withCertificateKeyVaultProperties(CertificateKeyVaultProperties certificateKeyVaultProperties) { + this.certificateKeyVaultProperties = certificateKeyVaultProperties; + return this; + } + /** * Get the password property: Certificate password. - * + * * @return the password value. */ public String password() { @@ -103,7 +140,7 @@ public String password() { /** * Set the password property: Certificate password. - * + * * @param password the password value to set. * @return the CertificateProperties object itself. */ @@ -114,7 +151,7 @@ public CertificateProperties withPassword(String password) { /** * Get the subjectName property: Subject name of the certificate. - * + * * @return the subjectName value. */ public String subjectName() { @@ -123,7 +160,7 @@ public String subjectName() { /** * Get the subjectAlternativeNames property: Subject alternative names the certificate applies to. - * + * * @return the subjectAlternativeNames value. */ public List subjectAlternativeNames() { @@ -132,7 +169,7 @@ public List subjectAlternativeNames() { /** * Get the value property: PFX or PEM blob. - * + * * @return the value value. */ public byte[] value() { @@ -141,7 +178,7 @@ public byte[] value() { /** * Set the value property: PFX or PEM blob. - * + * * @param value the value value to set. * @return the CertificateProperties object itself. */ @@ -152,7 +189,7 @@ public CertificateProperties withValue(byte[] value) { /** * Get the issuer property: Certificate issuer. - * + * * @return the issuer value. */ public String issuer() { @@ -161,7 +198,7 @@ public String issuer() { /** * Get the issueDate property: Certificate issue Date. - * + * * @return the issueDate value. */ public OffsetDateTime issueDate() { @@ -170,7 +207,7 @@ public OffsetDateTime issueDate() { /** * Get the expirationDate property: Certificate expiration date. - * + * * @return the expirationDate value. */ public OffsetDateTime expirationDate() { @@ -179,7 +216,7 @@ public OffsetDateTime expirationDate() { /** * Get the thumbprint property: Certificate thumbprint. - * + * * @return the thumbprint value. */ public String thumbprint() { @@ -188,7 +225,7 @@ public String thumbprint() { /** * Get the valid property: Is the certificate valid?. - * + * * @return the valid value. */ public Boolean valid() { @@ -197,18 +234,43 @@ public Boolean valid() { /** * Get the publicKeyHash property: Public key hash. - * + * * @return the publicKeyHash value. */ public String publicKeyHash() { return this.publicKeyHash; } + /** + * Get the certificateType property: The type of the certificate. Allowed values are `ServerSSLCertificate` and + * `ImagePullTrustedCA`. + * + * @return the certificateType value. + */ + public CertificateType certificateType() { + return this.certificateType; + } + + /** + * Set the certificateType property: The type of the certificate. Allowed values are `ServerSSLCertificate` and + * `ImagePullTrustedCA`. + * + * @param certificateType the certificateType value to set. + * @return the CertificateProperties object itself. + */ + public CertificateProperties withCertificateType(CertificateType certificateType) { + this.certificateType = certificateType; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (certificateKeyVaultProperties() != null) { + certificateKeyVaultProperties().validate(); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProvisioningState.java index 65e92aa12643f..74cae6f86cc9a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProvisioningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateProvisioningState.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning state of the certificate. */ +/** + * Provisioning state of the certificate. + */ public final class CertificateProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for CertificateProvisioningState. */ + /** + * Static value Succeeded for CertificateProvisioningState. + */ public static final CertificateProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for CertificateProvisioningState. */ + /** + * Static value Failed for CertificateProvisioningState. + */ public static final CertificateProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for CertificateProvisioningState. */ + /** + * Static value Canceled for CertificateProvisioningState. + */ public static final CertificateProvisioningState CANCELED = fromString("Canceled"); - /** Static value DeleteFailed for CertificateProvisioningState. */ + /** + * Static value DeleteFailed for CertificateProvisioningState. + */ public static final CertificateProvisioningState DELETE_FAILED = fromString("DeleteFailed"); - /** Static value Pending for CertificateProvisioningState. */ + /** + * Static value Pending for CertificateProvisioningState. + */ public static final CertificateProvisioningState PENDING = fromString("Pending"); /** * Creates a new instance of CertificateProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public CertificateProvisioningState() { /** * Creates or finds a CertificateProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding CertificateProvisioningState. */ @@ -47,7 +59,7 @@ public static CertificateProvisioningState fromString(String name) { /** * Gets known CertificateProvisioningState values. - * + * * @return known CertificateProvisioningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateType.java new file mode 100644 index 0000000000000..93d70629eeb92 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CertificateType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The type of the certificate. Allowed values are `ServerSSLCertificate` and `ImagePullTrustedCA`. + */ +public final class CertificateType extends ExpandableStringEnum { + /** + * Static value ServerSSLCertificate for CertificateType. + */ + public static final CertificateType SERVER_SSLCERTIFICATE = fromString("ServerSSLCertificate"); + + /** + * Static value ImagePullTrustedCA for CertificateType. + */ + public static final CertificateType IMAGE_PULL_TRUSTED_CA = fromString("ImagePullTrustedCA"); + + /** + * Creates a new instance of CertificateType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public CertificateType() { + } + + /** + * Creates or finds a CertificateType from its string representation. + * + * @param name a name to look for. + * @return the corresponding CertificateType. + */ + @JsonCreator + public static CertificateType fromString(String name) { + return fromString(name, CertificateType.class); + } + + /** + * Gets known CertificateType values. + * + * @return known CertificateType values. + */ + public static Collection values() { + return values(CertificateType.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificates.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificates.java index 3604ca25aca79..8f33ff0d5d1ca 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificates.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Certificates.java @@ -9,16 +9,18 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner; -/** Resource collection API of Certificates. */ +/** + * Resource collection API of Certificates. + */ public interface Certificates { /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -26,13 +28,13 @@ public interface Certificates { /** * Get the Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -40,29 +42,29 @@ public interface Certificates { /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, String certificateName, + Context context); /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate. */ @@ -70,7 +72,7 @@ Response getWithResponse( /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -78,27 +80,23 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ - Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificateInner certificateEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String certificateName, CertificateInner certificateEnvelope, Context context); /** * Create or Update a Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ @@ -106,38 +104,38 @@ Response createOrUpdateWithResponse( /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String environmentName, String certificateName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String certificateName, + Context context); /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String environmentName, String certificateName); /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. @@ -145,33 +143,29 @@ Response deleteWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with {@link - * Response}. + * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment along with + * {@link Response}. */ - Response updateWithResponse( - String resourceGroupName, - String environmentName, - String certificateName, - CertificatePatch certificateEnvelope, - Context context); + Response updateWithResponse(String resourceGroupName, String environmentName, String certificateName, + CertificatePatch certificateEnvelope, Context context); /** * Update properties of a certificate - * - *

Patches a certificate. Currently only patching of tags is supported. - * + * + * Patches a certificate. Currently only patching of tags is supported. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param certificateName Name of the Certificate. * @param certificateEnvelope Properties of a certificate that need to be updated. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return certificate used for Custom Domain bindings of Container Apps in a Managed Environment. */ - Certificate update( - String resourceGroupName, String environmentName, String certificateName, CertificatePatch certificateEnvelope); + Certificate update(String resourceGroupName, String environmentName, String certificateName, + CertificatePatch certificateEnvelope); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityReason.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityReason.java index 3913496c6504e..e499c606fcab6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityReason.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityReason.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The reason why the given name is not available. */ +/** + * The reason why the given name is not available. + */ public final class CheckNameAvailabilityReason extends ExpandableStringEnum { - /** Static value Invalid for CheckNameAvailabilityReason. */ + /** + * Static value Invalid for CheckNameAvailabilityReason. + */ public static final CheckNameAvailabilityReason INVALID = fromString("Invalid"); - /** Static value AlreadyExists for CheckNameAvailabilityReason. */ + /** + * Static value AlreadyExists for CheckNameAvailabilityReason. + */ public static final CheckNameAvailabilityReason ALREADY_EXISTS = fromString("AlreadyExists"); /** * Creates a new instance of CheckNameAvailabilityReason value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public CheckNameAvailabilityReason() { /** * Creates or finds a CheckNameAvailabilityReason from its string representation. - * + * * @param name a name to look for. * @return the corresponding CheckNameAvailabilityReason. */ @@ -38,7 +44,7 @@ public static CheckNameAvailabilityReason fromString(String name) { /** * Gets known CheckNameAvailabilityReason values. - * + * * @return known CheckNameAvailabilityReason values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityRequest.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityRequest.java index ebdac57ca9006..51361d97bdd90 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityRequest.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityRequest.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The check availability request body. */ +/** + * The check availability request body. + */ @Fluent public final class CheckNameAvailabilityRequest { /* @@ -22,13 +24,15 @@ public final class CheckNameAvailabilityRequest { @JsonProperty(value = "type") private String type; - /** Creates an instance of CheckNameAvailabilityRequest class. */ + /** + * Creates an instance of CheckNameAvailabilityRequest class. + */ public CheckNameAvailabilityRequest() { } /** * Get the name property: The name of the resource for which availability needs to be checked. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Set the name property: The name of the resource for which availability needs to be checked. - * + * * @param name the name value to set. * @return the CheckNameAvailabilityRequest object itself. */ @@ -48,7 +52,7 @@ public CheckNameAvailabilityRequest withName(String name) { /** * Get the type property: The resource type. - * + * * @return the type value. */ public String type() { @@ -57,7 +61,7 @@ public String type() { /** * Set the type property: The resource type. - * + * * @param type the type value to set. * @return the CheckNameAvailabilityRequest object itself. */ @@ -68,7 +72,7 @@ public CheckNameAvailabilityRequest withType(String type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityResponse.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityResponse.java index 7f61f59cc2a32..306acb1e1ff8e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityResponse.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CheckNameAvailabilityResponse.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.appcontainers.fluent.models.CheckNameAvailabilityResponseInner; -/** An immutable client-side representation of CheckNameAvailabilityResponse. */ +/** + * An immutable client-side representation of CheckNameAvailabilityResponse. + */ public interface CheckNameAvailabilityResponse { /** * Gets the nameAvailable property: Indicates if the resource name is available. - * + * * @return the nameAvailable value. */ Boolean nameAvailable(); /** * Gets the reason property: The reason why the given name is not available. - * + * * @return the reason value. */ CheckNameAvailabilityReason reason(); /** * Gets the message property: Detailed reason why the given name is available. - * + * * @return the message value. */ String message(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.CheckNameAvailabilityResponseInner object. - * + * * @return the inner object. */ CheckNameAvailabilityResponseInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CircuitBreakerPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CircuitBreakerPolicy.java new file mode 100644 index 0000000000000..15fa7600e32bc --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CircuitBreakerPolicy.java @@ -0,0 +1,113 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Policy that defines circuit breaker conditions. + */ +@Fluent +public final class CircuitBreakerPolicy { + /* + * Number of consecutive errors before the circuit breaker opens + */ + @JsonProperty(value = "consecutiveErrors") + private Integer consecutiveErrors; + + /* + * The time interval, in seconds, between endpoint checks. This can result in opening the circuit breaker if the + * check fails as well as closing the circuit breaker if the check succeeds. Defaults to 10s. + */ + @JsonProperty(value = "intervalInSeconds") + private Integer intervalInSeconds; + + /* + * Maximum percentage of hosts that will be ejected after failure threshold has been met + */ + @JsonProperty(value = "maxEjectionPercent") + private Integer maxEjectionPercent; + + /** + * Creates an instance of CircuitBreakerPolicy class. + */ + public CircuitBreakerPolicy() { + } + + /** + * Get the consecutiveErrors property: Number of consecutive errors before the circuit breaker opens. + * + * @return the consecutiveErrors value. + */ + public Integer consecutiveErrors() { + return this.consecutiveErrors; + } + + /** + * Set the consecutiveErrors property: Number of consecutive errors before the circuit breaker opens. + * + * @param consecutiveErrors the consecutiveErrors value to set. + * @return the CircuitBreakerPolicy object itself. + */ + public CircuitBreakerPolicy withConsecutiveErrors(Integer consecutiveErrors) { + this.consecutiveErrors = consecutiveErrors; + return this; + } + + /** + * Get the intervalInSeconds property: The time interval, in seconds, between endpoint checks. This can result in + * opening the circuit breaker if the check fails as well as closing the circuit breaker if the check succeeds. + * Defaults to 10s. + * + * @return the intervalInSeconds value. + */ + public Integer intervalInSeconds() { + return this.intervalInSeconds; + } + + /** + * Set the intervalInSeconds property: The time interval, in seconds, between endpoint checks. This can result in + * opening the circuit breaker if the check fails as well as closing the circuit breaker if the check succeeds. + * Defaults to 10s. + * + * @param intervalInSeconds the intervalInSeconds value to set. + * @return the CircuitBreakerPolicy object itself. + */ + public CircuitBreakerPolicy withIntervalInSeconds(Integer intervalInSeconds) { + this.intervalInSeconds = intervalInSeconds; + return this; + } + + /** + * Get the maxEjectionPercent property: Maximum percentage of hosts that will be ejected after failure threshold + * has been met. + * + * @return the maxEjectionPercent value. + */ + public Integer maxEjectionPercent() { + return this.maxEjectionPercent; + } + + /** + * Set the maxEjectionPercent property: Maximum percentage of hosts that will be ejected after failure threshold + * has been met. + * + * @param maxEjectionPercent the maxEjectionPercent value to set. + * @return the CircuitBreakerPolicy object itself. + */ + public CircuitBreakerPolicy withMaxEjectionPercent(Integer maxEjectionPercent) { + this.maxEjectionPercent = maxEjectionPercent; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientCredentialMethod.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientCredentialMethod.java index 9486ee8854e01..c2dc3e9ab0383 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientCredentialMethod.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientCredentialMethod.java @@ -7,12 +7,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** The method that should be used to authenticate the user. */ +/** + * The method that should be used to authenticate the user. + */ public enum ClientCredentialMethod { - /** Enum value ClientSecretPost. */ + /** + * Enum value ClientSecretPost. + */ CLIENT_SECRET_POST("ClientSecretPost"); - /** The actual serialized value for a ClientCredentialMethod instance. */ + /** + * The actual serialized value for a ClientCredentialMethod instance. + */ private final String value; ClientCredentialMethod(String value) { @@ -21,7 +27,7 @@ public enum ClientCredentialMethod { /** * Parses a serialized value to a ClientCredentialMethod instance. - * + * * @param value the serialized value to parse. * @return the parsed ClientCredentialMethod object, or null if unable to parse. */ @@ -39,7 +45,9 @@ public static ClientCredentialMethod fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientRegistration.java index d8d3cdfcf9bab..c016c58d4bc31 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ClientRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the app registration for providers that have client ids and client secrets. */ +/** + * The configuration settings of the app registration for providers that have client ids and client secrets. + */ @Fluent public final class ClientRegistration { /* @@ -22,13 +24,15 @@ public final class ClientRegistration { @JsonProperty(value = "clientSecretSettingName") private String clientSecretSettingName; - /** Creates an instance of ClientRegistration class. */ + /** + * Creates an instance of ClientRegistration class. + */ public ClientRegistration() { } /** * Get the clientId property: The Client ID of the app used for login. - * + * * @return the clientId value. */ public String clientId() { @@ -37,7 +41,7 @@ public String clientId() { /** * Set the clientId property: The Client ID of the app used for login. - * + * * @param clientId the clientId value to set. * @return the ClientRegistration object itself. */ @@ -48,7 +52,7 @@ public ClientRegistration withClientId(String clientId) { /** * Get the clientSecretSettingName property: The app setting name that contains the client secret. - * + * * @return the clientSecretSettingName value. */ public String clientSecretSettingName() { @@ -57,7 +61,7 @@ public String clientSecretSettingName() { /** * Set the clientSecretSettingName property: The app setting name that contains the client secret. - * + * * @param clientSecretSettingName the clientSecretSettingName value to set. * @return the ClientRegistration object itself. */ @@ -68,7 +72,7 @@ public ClientRegistration withClientSecretSettingName(String clientSecretSetting /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Configuration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Configuration.java index 98f0b1ad7b070..82755893d1128 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Configuration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Configuration.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Non versioned Container App configuration properties that define the mutable settings of a Container app. */ +/** + * Non versioned Container App configuration properties that define the mutable settings of a Container app. + */ @Fluent public final class Configuration { /* @@ -55,13 +57,15 @@ public final class Configuration { @JsonProperty(value = "service") private Service service; - /** Creates an instance of Configuration class. */ + /** + * Creates an instance of Configuration class. + */ public Configuration() { } /** * Get the secrets property: Collection of secrets used by a Container app. - * + * * @return the secrets value. */ public List secrets() { @@ -70,7 +74,7 @@ public List secrets() { /** * Set the secrets property: Collection of secrets used by a Container app. - * + * * @param secrets the secrets value to set. * @return the Configuration object itself. */ @@ -81,10 +85,11 @@ public Configuration withSecrets(List secrets) { /** * Get the activeRevisionsMode property: ActiveRevisionsMode controls how active revisions are handled for the - * Container app: <list><item>Multiple: multiple revisions can be - * active.</item><item>Single: Only one revision can be active at a time. Revision weights can not be - * used in this mode. If no value if provided, this is the default.</item></list>. - * + * Container app: + * <list><item>Multiple: multiple revisions can be active.</item><item>Single: Only one + * revision can be active at a time. Revision weights can not be used in this mode. If no value if provided, this + * is the default.</item></list>. + * * @return the activeRevisionsMode value. */ public ActiveRevisionsMode activeRevisionsMode() { @@ -93,10 +98,11 @@ public ActiveRevisionsMode activeRevisionsMode() { /** * Set the activeRevisionsMode property: ActiveRevisionsMode controls how active revisions are handled for the - * Container app: <list><item>Multiple: multiple revisions can be - * active.</item><item>Single: Only one revision can be active at a time. Revision weights can not be - * used in this mode. If no value if provided, this is the default.</item></list>. - * + * Container app: + * <list><item>Multiple: multiple revisions can be active.</item><item>Single: Only one + * revision can be active at a time. Revision weights can not be used in this mode. If no value if provided, this + * is the default.</item></list>. + * * @param activeRevisionsMode the activeRevisionsMode value to set. * @return the Configuration object itself. */ @@ -107,7 +113,7 @@ public Configuration withActiveRevisionsMode(ActiveRevisionsMode activeRevisions /** * Get the ingress property: Ingress configurations. - * + * * @return the ingress value. */ public Ingress ingress() { @@ -116,7 +122,7 @@ public Ingress ingress() { /** * Set the ingress property: Ingress configurations. - * + * * @param ingress the ingress value to set. * @return the Configuration object itself. */ @@ -128,7 +134,7 @@ public Configuration withIngress(Ingress ingress) { /** * Get the registries property: Collection of private container registry credentials for containers used by the * Container app. - * + * * @return the registries value. */ public List registries() { @@ -138,7 +144,7 @@ public List registries() { /** * Set the registries property: Collection of private container registry credentials for containers used by the * Container app. - * + * * @param registries the registries value to set. * @return the Configuration object itself. */ @@ -149,7 +155,7 @@ public Configuration withRegistries(List registries) { /** * Get the dapr property: Dapr configuration for the Container App. - * + * * @return the dapr value. */ public Dapr dapr() { @@ -158,7 +164,7 @@ public Dapr dapr() { /** * Set the dapr property: Dapr configuration for the Container App. - * + * * @param dapr the dapr value to set. * @return the Configuration object itself. */ @@ -169,7 +175,7 @@ public Configuration withDapr(Dapr dapr) { /** * Get the maxInactiveRevisions property: Optional. Max inactive revisions a Container App can have. - * + * * @return the maxInactiveRevisions value. */ public Integer maxInactiveRevisions() { @@ -178,7 +184,7 @@ public Integer maxInactiveRevisions() { /** * Set the maxInactiveRevisions property: Optional. Max inactive revisions a Container App can have. - * + * * @param maxInactiveRevisions the maxInactiveRevisions value to set. * @return the Configuration object itself. */ @@ -189,7 +195,7 @@ public Configuration withMaxInactiveRevisions(Integer maxInactiveRevisions) { /** * Get the service property: Container App to be a dev Container App Service. - * + * * @return the service value. */ public Service service() { @@ -198,7 +204,7 @@ public Service service() { /** * Set the service property: Container App to be a dev Container App Service. - * + * * @param service the service value to set. * @return the Configuration object itself. */ @@ -209,7 +215,7 @@ public Configuration withService(Service service) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironment.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironment.java index 7cd325e26c169..a389b68a39410 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironment.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironment.java @@ -11,81 +11,83 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentInner; import java.util.Map; -/** An immutable client-side representation of ConnectedEnvironment. */ +/** + * An immutable client-side representation of ConnectedEnvironment. + */ public interface ConnectedEnvironment { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the extendedLocation property: The complex type of the extended location. - * + * * @return the extendedLocation value. */ ExtendedLocation extendedLocation(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning state of the Kubernetes Environment. - * + * * @return the provisioningState value. */ ConnectedEnvironmentProvisioningState provisioningState(); /** * Gets the deploymentErrors property: Any errors that occurred during deployment or deployment validation. - * + * * @return the deploymentErrors value. */ String deploymentErrors(); /** * Gets the defaultDomain property: Default Domain Name for the cluster. - * + * * @return the defaultDomain value. */ String defaultDomain(); /** * Gets the staticIp property: Static IP of the connectedEnvironment. - * + * * @return the staticIp value. */ String staticIp(); @@ -93,65 +95,70 @@ public interface ConnectedEnvironment { /** * Gets the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service * to Service communication telemetry. - * + * * @return the daprAIConnectionString value. */ String daprAIConnectionString(); /** * Gets the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @return the customDomainConfiguration value. */ CustomDomainConfiguration customDomainConfiguration(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentInner object. - * + * * @return the inner object. */ ConnectedEnvironmentInner innerModel(); - /** The entirety of the ConnectedEnvironment definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the ConnectedEnvironment definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The ConnectedEnvironment definition stages. */ + /** + * The ConnectedEnvironment definition stages. + */ interface DefinitionStages { - /** The first stage of the ConnectedEnvironment definition. */ + /** + * The first stage of the ConnectedEnvironment definition. + */ interface Blank extends WithLocation { } - /** The stage of the ConnectedEnvironment definition allowing to specify location. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -159,18 +166,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the ConnectedEnvironment definition allowing to specify parent resource. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -182,78 +191,85 @@ interface WithResourceGroup { * resource to be created, but also allows for any other optional properties to be specified. */ interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithExtendedLocation, - DefinitionStages.WithStaticIp, - DefinitionStages.WithDaprAIConnectionString, - DefinitionStages.WithCustomDomainConfiguration { + extends DefinitionStages.WithTags, DefinitionStages.WithExtendedLocation, DefinitionStages.WithStaticIp, + DefinitionStages.WithDaprAIConnectionString, DefinitionStages.WithCustomDomainConfiguration { /** * Executes the create request. - * + * * @return the created resource. */ ConnectedEnvironment create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ConnectedEnvironment create(Context context); } - /** The stage of the ConnectedEnvironment definition allowing to specify tags. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the ConnectedEnvironment definition allowing to specify extendedLocation. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify extendedLocation. + */ interface WithExtendedLocation { /** * Specifies the extendedLocation property: The complex type of the extended location.. - * + * * @param extendedLocation The complex type of the extended location. * @return the next definition stage. */ WithCreate withExtendedLocation(ExtendedLocation extendedLocation); } - /** The stage of the ConnectedEnvironment definition allowing to specify staticIp. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify staticIp. + */ interface WithStaticIp { /** * Specifies the staticIp property: Static IP of the connectedEnvironment. - * + * * @param staticIp Static IP of the connectedEnvironment. * @return the next definition stage. */ WithCreate withStaticIp(String staticIp); } - /** The stage of the ConnectedEnvironment definition allowing to specify daprAIConnectionString. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify daprAIConnectionString. + */ interface WithDaprAIConnectionString { /** * Specifies the daprAIConnectionString property: Application Insights connection string used by Dapr to * export Service to Service communication telemetry. - * + * * @param daprAIConnectionString Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. + * Service communication telemetry. * @return the next definition stage. */ WithCreate withDaprAIConnectionString(String daprAIConnectionString); } - /** The stage of the ConnectedEnvironment definition allowing to specify customDomainConfiguration. */ + /** + * The stage of the ConnectedEnvironment definition allowing to specify customDomainConfiguration. + */ interface WithCustomDomainConfiguration { /** * Specifies the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration Custom domain configuration for the environment. * @return the next definition stage. */ @@ -263,87 +279,97 @@ interface WithCustomDomainConfiguration { /** * Begins update for the ConnectedEnvironment resource. - * + * * @return the stage of resource update. */ ConnectedEnvironment.Update update(); - /** The template for ConnectedEnvironment update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithExtendedLocation, - UpdateStages.WithStaticIp, - UpdateStages.WithDaprAIConnectionString, - UpdateStages.WithCustomDomainConfiguration { + /** + * The template for ConnectedEnvironment update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithExtendedLocation, UpdateStages.WithStaticIp, + UpdateStages.WithDaprAIConnectionString, UpdateStages.WithCustomDomainConfiguration { /** * Executes the update request. - * + * * @return the updated resource. */ ConnectedEnvironment apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ConnectedEnvironment apply(Context context); } - /** The ConnectedEnvironment update stages. */ + /** + * The ConnectedEnvironment update stages. + */ interface UpdateStages { - /** The stage of the ConnectedEnvironment update allowing to specify tags. */ + /** + * The stage of the ConnectedEnvironment update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the ConnectedEnvironment update allowing to specify extendedLocation. */ + /** + * The stage of the ConnectedEnvironment update allowing to specify extendedLocation. + */ interface WithExtendedLocation { /** * Specifies the extendedLocation property: The complex type of the extended location.. - * + * * @param extendedLocation The complex type of the extended location. * @return the next definition stage. */ Update withExtendedLocation(ExtendedLocation extendedLocation); } - /** The stage of the ConnectedEnvironment update allowing to specify staticIp. */ + /** + * The stage of the ConnectedEnvironment update allowing to specify staticIp. + */ interface WithStaticIp { /** * Specifies the staticIp property: Static IP of the connectedEnvironment. - * + * * @param staticIp Static IP of the connectedEnvironment. * @return the next definition stage. */ Update withStaticIp(String staticIp); } - /** The stage of the ConnectedEnvironment update allowing to specify daprAIConnectionString. */ + /** + * The stage of the ConnectedEnvironment update allowing to specify daprAIConnectionString. + */ interface WithDaprAIConnectionString { /** * Specifies the daprAIConnectionString property: Application Insights connection string used by Dapr to * export Service to Service communication telemetry. - * + * * @param daprAIConnectionString Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. + * Service communication telemetry. * @return the next definition stage. */ Update withDaprAIConnectionString(String daprAIConnectionString); } - /** The stage of the ConnectedEnvironment update allowing to specify customDomainConfiguration. */ + /** + * The stage of the ConnectedEnvironment update allowing to specify customDomainConfiguration. + */ interface WithCustomDomainConfiguration { /** * Specifies the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration Custom domain configuration for the environment. * @return the next definition stage. */ @@ -353,14 +379,14 @@ interface WithCustomDomainConfiguration { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ConnectedEnvironment refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -368,29 +394,29 @@ interface WithCustomDomainConfiguration { /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result along with {@link Response}. */ - Response checkNameAvailabilityWithResponse( - CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context); + Response + checkNameAvailabilityWithResponse(CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context); /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentCollection.java index 7b7e7fcb9144d..f95c515f5c0fd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentCollection.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of connectedEnvironments. */ +/** + * Collection of connectedEnvironments. + */ @Fluent public final class ConnectedEnvironmentCollection { /* @@ -24,13 +26,15 @@ public final class ConnectedEnvironmentCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ConnectedEnvironmentCollection class. */ + /** + * Creates an instance of ConnectedEnvironmentCollection class. + */ public ConnectedEnvironmentCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the ConnectedEnvironmentCollection object itself. */ @@ -50,7 +54,7 @@ public ConnectedEnvironmentCollection withValue(List /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentProvisioningState.java index 19364fb05e732..5655b3a8691f9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentProvisioningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentProvisioningState.java @@ -8,39 +8,57 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning state of the Kubernetes Environment. */ +/** + * Provisioning state of the Kubernetes Environment. + */ public final class ConnectedEnvironmentProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for ConnectedEnvironmentProvisioningState. */ + /** + * Static value Succeeded for ConnectedEnvironmentProvisioningState. + */ public static final ConnectedEnvironmentProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for ConnectedEnvironmentProvisioningState. */ + /** + * Static value Failed for ConnectedEnvironmentProvisioningState. + */ public static final ConnectedEnvironmentProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ConnectedEnvironmentProvisioningState. */ + /** + * Static value Canceled for ConnectedEnvironmentProvisioningState. + */ public static final ConnectedEnvironmentProvisioningState CANCELED = fromString("Canceled"); - /** Static value Waiting for ConnectedEnvironmentProvisioningState. */ + /** + * Static value Waiting for ConnectedEnvironmentProvisioningState. + */ public static final ConnectedEnvironmentProvisioningState WAITING = fromString("Waiting"); - /** Static value InitializationInProgress for ConnectedEnvironmentProvisioningState. */ - public static final ConnectedEnvironmentProvisioningState INITIALIZATION_IN_PROGRESS = - fromString("InitializationInProgress"); + /** + * Static value InitializationInProgress for ConnectedEnvironmentProvisioningState. + */ + public static final ConnectedEnvironmentProvisioningState INITIALIZATION_IN_PROGRESS + = fromString("InitializationInProgress"); - /** Static value InfrastructureSetupInProgress for ConnectedEnvironmentProvisioningState. */ - public static final ConnectedEnvironmentProvisioningState INFRASTRUCTURE_SETUP_IN_PROGRESS = - fromString("InfrastructureSetupInProgress"); + /** + * Static value InfrastructureSetupInProgress for ConnectedEnvironmentProvisioningState. + */ + public static final ConnectedEnvironmentProvisioningState INFRASTRUCTURE_SETUP_IN_PROGRESS + = fromString("InfrastructureSetupInProgress"); - /** Static value InfrastructureSetupComplete for ConnectedEnvironmentProvisioningState. */ - public static final ConnectedEnvironmentProvisioningState INFRASTRUCTURE_SETUP_COMPLETE = - fromString("InfrastructureSetupComplete"); + /** + * Static value InfrastructureSetupComplete for ConnectedEnvironmentProvisioningState. + */ + public static final ConnectedEnvironmentProvisioningState INFRASTRUCTURE_SETUP_COMPLETE + = fromString("InfrastructureSetupComplete"); - /** Static value ScheduledForDelete for ConnectedEnvironmentProvisioningState. */ + /** + * Static value ScheduledForDelete for ConnectedEnvironmentProvisioningState. + */ public static final ConnectedEnvironmentProvisioningState SCHEDULED_FOR_DELETE = fromString("ScheduledForDelete"); /** * Creates a new instance of ConnectedEnvironmentProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -49,7 +67,7 @@ public ConnectedEnvironmentProvisioningState() { /** * Creates or finds a ConnectedEnvironmentProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ConnectedEnvironmentProvisioningState. */ @@ -60,7 +78,7 @@ public static ConnectedEnvironmentProvisioningState fromString(String name) { /** * Gets known ConnectedEnvironmentProvisioningState values. - * + * * @return known ConnectedEnvironmentProvisioningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorage.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorage.java index c413cde7c5f13..861b0c8c0339d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorage.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorage.java @@ -4,69 +4,87 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStorageInner; -/** An immutable client-side representation of ConnectedEnvironmentStorage. */ +/** + * An immutable client-side representation of ConnectedEnvironmentStorage. + */ public interface ConnectedEnvironmentStorage { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the properties property: Storage properties. - * + * * @return the properties value. */ ConnectedEnvironmentStorageProperties properties(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStorageInner object. - * + * * @return the inner object. */ ConnectedEnvironmentStorageInner innerModel(); - /** The entirety of the ConnectedEnvironmentStorage definition. */ + /** + * The entirety of the ConnectedEnvironmentStorage definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The ConnectedEnvironmentStorage definition stages. */ + /** + * The ConnectedEnvironmentStorage definition stages. + */ interface DefinitionStages { - /** The first stage of the ConnectedEnvironmentStorage definition. */ + /** + * The first stage of the ConnectedEnvironmentStorage definition. + */ interface Blank extends WithParentResource { } - /** The stage of the ConnectedEnvironmentStorage definition allowing to specify parent resource. */ + /** + * The stage of the ConnectedEnvironmentStorage definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, connectedEnvironmentName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @return the next definition stage. @@ -81,25 +99,27 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ ConnectedEnvironmentStorage create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ConnectedEnvironmentStorage create(Context context); } - /** The stage of the ConnectedEnvironmentStorage definition allowing to specify properties. */ + /** + * The stage of the ConnectedEnvironmentStorage definition allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: Storage properties. - * + * * @param properties Storage properties. * @return the next definition stage. */ @@ -109,36 +129,42 @@ interface WithProperties { /** * Begins update for the ConnectedEnvironmentStorage resource. - * + * * @return the stage of resource update. */ ConnectedEnvironmentStorage.Update update(); - /** The template for ConnectedEnvironmentStorage update. */ + /** + * The template for ConnectedEnvironmentStorage update. + */ interface Update extends UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ ConnectedEnvironmentStorage apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ConnectedEnvironmentStorage apply(Context context); } - /** The ConnectedEnvironmentStorage update stages. */ + /** + * The ConnectedEnvironmentStorage update stages. + */ interface UpdateStages { - /** The stage of the ConnectedEnvironmentStorage update allowing to specify properties. */ + /** + * The stage of the ConnectedEnvironmentStorage update allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: Storage properties. - * + * * @param properties Storage properties. * @return the next definition stage. */ @@ -148,14 +174,14 @@ interface WithProperties { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ConnectedEnvironmentStorage refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorageProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorageProperties.java index 199d8dd7ab4f5..9715cc8bc3cba 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorageProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStorageProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Storage properties. */ +/** + * Storage properties. + */ @Fluent public final class ConnectedEnvironmentStorageProperties { /* @@ -16,13 +18,15 @@ public final class ConnectedEnvironmentStorageProperties { @JsonProperty(value = "azureFile") private AzureFileProperties azureFile; - /** Creates an instance of ConnectedEnvironmentStorageProperties class. */ + /** + * Creates an instance of ConnectedEnvironmentStorageProperties class. + */ public ConnectedEnvironmentStorageProperties() { } /** * Get the azureFile property: Azure file properties. - * + * * @return the azureFile value. */ public AzureFileProperties azureFile() { @@ -31,7 +35,7 @@ public AzureFileProperties azureFile() { /** * Set the azureFile property: Azure file properties. - * + * * @param azureFile the azureFile value to set. * @return the ConnectedEnvironmentStorageProperties object itself. */ @@ -42,7 +46,7 @@ public ConnectedEnvironmentStorageProperties withAzureFile(AzureFileProperties a /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStoragesCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStoragesCollection.java index 1177335e4438b..4493ee7e11dce 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStoragesCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentStoragesCollection.java @@ -7,11 +7,13 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStoragesCollectionInner; import java.util.List; -/** An immutable client-side representation of ConnectedEnvironmentStoragesCollection. */ +/** + * An immutable client-side representation of ConnectedEnvironmentStoragesCollection. + */ public interface ConnectedEnvironmentStoragesCollection { /** * Gets the value property: Collection of storage resources. - * + * * @return the value value. */ List value(); @@ -19,7 +21,7 @@ public interface ConnectedEnvironmentStoragesCollection { /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStoragesCollectionInner * object. - * + * * @return the inner object. */ ConnectedEnvironmentStoragesCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironments.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironments.java index 74729069dd720..e8cef8647f816 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironments.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironments.java @@ -8,13 +8,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ConnectedEnvironments. */ +/** + * Resource collection API of ConnectedEnvironments. + */ public interface ConnectedEnvironments { /** * Get all connectedEnvironments for a subscription. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription as paginated response with {@link PagedIterable}. */ @@ -22,11 +24,11 @@ public interface ConnectedEnvironments { /** * Get all connectedEnvironments for a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments for a subscription as paginated response with {@link PagedIterable}. */ @@ -34,11 +36,11 @@ public interface ConnectedEnvironments { /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments in a resource group as paginated response with {@link PagedIterable}. */ @@ -46,12 +48,12 @@ public interface ConnectedEnvironments { /** * Get all connectedEnvironments in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all connectedEnvironments in a resource group as paginated response with {@link PagedIterable}. */ @@ -59,27 +61,27 @@ public interface ConnectedEnvironments { /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of an connectedEnvironment along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context); /** * Get the properties of an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of an connectedEnvironment. */ @@ -87,57 +89,57 @@ Response getByResourceGroupWithResponse( /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByResourceGroup(String resourceGroupName, String connectedEnvironmentName); /** * Delete an connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String connectedEnvironmentName, Context context); /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service along with - * {@link Response}. + * {@link Response}. */ - Response updateWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context); + Response updateWithResponse(String resourceGroupName, String connectedEnvironmentName, + Context context); /** * Update connected Environment's properties. - * - *

Patches a Managed Environment. Only patching of tags is supported currently. - * + * + * Patches a Managed Environment. Only patching of tags is supported currently. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connectedEnvironment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an environment for Kubernetes cluster specialized for web workloads by Azure App Service. */ @@ -145,51 +147,46 @@ Response updateWithResponse( /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result along with {@link Response}. */ - Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String connectedEnvironmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context); + Response checkNameAvailabilityWithResponse(String resourceGroupName, + String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context); /** * Checks the resource connectedEnvironmentName availability. - * - *

Checks if resource connectedEnvironmentName is available. - * + * + * Checks if resource connectedEnvironmentName is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check connectedEnvironmentName availability request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result. */ - CheckNameAvailabilityResponse checkNameAvailability( - String resourceGroupName, - String connectedEnvironmentName, + CheckNameAvailabilityResponse checkNameAvailability(String resourceGroupName, String connectedEnvironmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest); /** * Get the properties of an connectedEnvironment. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of an connectedEnvironment along with {@link Response}. */ @@ -197,12 +194,12 @@ CheckNameAvailabilityResponse checkNameAvailability( /** * Get the properties of an connectedEnvironment. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of an connectedEnvironment along with {@link Response}. */ @@ -210,30 +207,30 @@ CheckNameAvailabilityResponse checkNameAvailability( /** * Delete an connectedEnvironment. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete an connectedEnvironment. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new ConnectedEnvironment resource. - * + * * @param name resource name. * @return the first stage of the new ConnectedEnvironment definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsCertificates.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsCertificates.java index 86b7664931662..58a536c24fb91 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsCertificates.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsCertificates.java @@ -8,16 +8,18 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ConnectedEnvironmentsCertificates. */ +/** + * Resource collection API of ConnectedEnvironmentsCertificates. + */ public interface ConnectedEnvironmentsCertificates { /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given connected environment as paginated response with {@link PagedIterable}. */ @@ -25,13 +27,13 @@ public interface ConnectedEnvironmentsCertificates { /** * Get the Certificates in a given connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Certificates in a given connected environment as paginated response with {@link PagedIterable}. */ @@ -39,29 +41,29 @@ public interface ConnectedEnvironmentsCertificates { /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context); + Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String certificateName, Context context); /** * Get the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate. */ @@ -69,40 +71,40 @@ Response getWithResponse( /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String certificateName, Context context); + Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, String certificateName, + Context context); /** * Deletes the specified Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Connected Environment. * @param certificateName Name of the Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String connectedEnvironmentName, String certificateName); /** * Get the specified Certificate. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate along with {@link Response}. */ @@ -110,12 +112,12 @@ Response deleteWithResponse( /** * Get the specified Certificate. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Certificate along with {@link Response}. */ @@ -123,23 +125,23 @@ Response deleteWithResponse( /** * Deletes the specified Certificate. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Deletes the specified Certificate. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @@ -147,7 +149,7 @@ Response deleteWithResponse( /** * Begins definition for a new Certificate resource. - * + * * @param name resource name. * @return the first stage of the new Certificate definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsDaprComponents.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsDaprComponents.java index 4e4afa80120f1..f02d74ed72266 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsDaprComponents.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsDaprComponents.java @@ -8,16 +8,18 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ConnectedEnvironmentsDaprComponents. */ +/** + * Resource collection API of ConnectedEnvironmentsDaprComponents. + */ public interface ConnectedEnvironmentsDaprComponents { /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a connected environment as paginated response with {@link PagedIterable}. */ @@ -25,13 +27,13 @@ public interface ConnectedEnvironmentsDaprComponents { /** * Get the Dapr Components for a connected environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a connected environment as paginated response with {@link PagedIterable}. */ @@ -39,29 +41,29 @@ public interface ConnectedEnvironmentsDaprComponents { /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context); + Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context); /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component. */ @@ -69,62 +71,62 @@ Response getWithResponse( /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context); + Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, String componentName, + Context context); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String connectedEnvironmentName, String componentName); /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ - Response listSecretsWithResponse( - String resourceGroupName, String connectedEnvironmentName, String componentName, Context context); + Response listSecretsWithResponse(String resourceGroupName, String connectedEnvironmentName, + String componentName, Context context); /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action. */ @@ -132,11 +134,11 @@ Response listSecretsWithResponse( /** * Get a dapr component. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component along with {@link Response}. */ @@ -144,12 +146,12 @@ Response listSecretsWithResponse( /** * Get a dapr component. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component along with {@link Response}. */ @@ -157,27 +159,27 @@ Response listSecretsWithResponse( /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a connected environment. - * + * + * Delete a Dapr Component from a connected environment. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @@ -185,7 +187,7 @@ Response listSecretsWithResponse( /** * Begins definition for a new DaprComponent resource. - * + * * @param name resource name. * @return the first stage of the new DaprComponent definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsStorages.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsStorages.java index 0a7d282219740..3b4ba9cde372c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsStorages.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ConnectedEnvironmentsStorages.java @@ -7,31 +7,33 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ConnectedEnvironmentsStorages. */ +/** + * Resource collection API of ConnectedEnvironmentsStorages. + */ public interface ConnectedEnvironmentsStorages { /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a connectedEnvironment along with {@link Response}. */ - Response listWithResponse( - String resourceGroupName, String connectedEnvironmentName, Context context); + Response listWithResponse(String resourceGroupName, + String connectedEnvironmentName, Context context); /** * Get all storages for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a connectedEnvironment. */ @@ -39,29 +41,29 @@ Response listWithResponse( /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a connectedEnvironment along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context); + Response getWithResponse(String resourceGroupName, String connectedEnvironmentName, + String storageName, Context context); /** * Get storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a connectedEnvironment. */ @@ -69,40 +71,40 @@ Response getWithResponse( /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String connectedEnvironmentName, String storageName, Context context); + Response deleteWithResponse(String resourceGroupName, String connectedEnvironmentName, String storageName, + Context context); /** * Delete storage for a connectedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String connectedEnvironmentName, String storageName); /** * Get storage for a connectedEnvironment. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a connectedEnvironment along with {@link Response}. */ @@ -110,12 +112,12 @@ Response deleteWithResponse( /** * Get storage for a connectedEnvironment. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a connectedEnvironment along with {@link Response}. */ @@ -123,23 +125,23 @@ Response deleteWithResponse( /** * Delete storage for a connectedEnvironment. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete storage for a connectedEnvironment. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @@ -147,7 +149,7 @@ Response deleteWithResponse( /** * Begins definition for a new ConnectedEnvironmentStorage resource. - * + * * @param name resource name. * @return the first stage of the new ConnectedEnvironmentStorage definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Container.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Container.java index 24556e20cbe2a..b1f4d676514cd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Container.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Container.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App container definition. */ +/** + * Container App container definition. + */ @Fluent public final class Container extends BaseContainer { /* @@ -17,13 +19,15 @@ public final class Container extends BaseContainer { @JsonProperty(value = "probes") private List probes; - /** Creates an instance of Container class. */ + /** + * Creates an instance of Container class. + */ public Container() { } /** * Get the probes property: List of probes for the container. - * + * * @return the probes value. */ public List probes() { @@ -32,7 +36,7 @@ public List probes() { /** * Set the probes property: List of probes for the container. - * + * * @param probes the probes value to set. * @return the Container object itself. */ @@ -41,49 +45,63 @@ public Container withProbes(List probes) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withImage(String image) { super.withImage(image); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withName(String name) { super.withName(name); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withCommand(List command) { super.withCommand(command); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withArgs(List args) { super.withArgs(args); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withEnv(List env) { super.withEnv(env); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withResources(ContainerResources resources) { super.withResources(resources); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public Container withVolumeMounts(List volumeMounts) { super.withVolumeMounts(volumeMounts); @@ -92,7 +110,7 @@ public Container withVolumeMounts(List volumeMounts) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApp.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApp.java index 6b2c89375c65a..45488a9f26c94 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApp.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApp.java @@ -12,46 +12,48 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of ContainerApp. */ +/** + * An immutable client-side representation of ContainerApp. + */ public interface ContainerApp { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the extendedLocation property: The complex type of the extended location. - * + * * @return the extendedLocation value. */ ExtendedLocation extendedLocation(); @@ -59,7 +61,7 @@ public interface ContainerApp { /** * Gets the identity property: managed identities for the Container App to interact with other Azure services * without maintaining any secrets or credentials in code. - * + * * @return the identity value. */ ManagedServiceIdentity identity(); @@ -68,149 +70,154 @@ public interface ContainerApp { * Gets the managedBy property: The fully qualified resource ID of the resource that manages this resource. * Indicates if this resource is managed by another Azure resource. If this is present, complete mode deployment * will not delete the resource if it is removed from the template since it is managed by another resource. - * + * * @return the managedBy value. */ String managedBy(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning state of the Container App. - * + * * @return the provisioningState value. */ ContainerAppProvisioningState provisioningState(); /** * Gets the managedEnvironmentId property: Deprecated. Resource ID of the Container App's environment. - * + * * @return the managedEnvironmentId value. */ String managedEnvironmentId(); /** * Gets the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ String environmentId(); /** * Gets the workloadProfileName property: Workload profile name to pin for container app execution. - * + * * @return the workloadProfileName value. */ String workloadProfileName(); /** * Gets the latestRevisionName property: Name of the latest revision of the Container App. - * + * * @return the latestRevisionName value. */ String latestRevisionName(); /** * Gets the latestReadyRevisionName property: Name of the latest ready revision of the Container App. - * + * * @return the latestReadyRevisionName value. */ String latestReadyRevisionName(); /** * Gets the latestRevisionFqdn property: Fully Qualified Domain Name of the latest revision of the Container App. - * + * * @return the latestRevisionFqdn value. */ String latestRevisionFqdn(); /** * Gets the customDomainVerificationId property: Id used to verify domain name ownership. - * + * * @return the customDomainVerificationId value. */ String customDomainVerificationId(); /** * Gets the configuration property: Non versioned Container App configuration properties. - * + * * @return the configuration value. */ Configuration configuration(); /** * Gets the template property: Container App versioned application definition. - * + * * @return the template value. */ Template template(); /** * Gets the outboundIpAddresses property: Outbound IP Addresses for container app. - * + * * @return the outboundIpAddresses value. */ List outboundIpAddresses(); /** * Gets the eventStreamEndpoint property: The endpoint of the eventstream of the container app. - * + * * @return the eventStreamEndpoint value. */ String eventStreamEndpoint(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppInner object. - * + * * @return the inner object. */ ContainerAppInner innerModel(); - /** The entirety of the ContainerApp definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the ContainerApp definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The ContainerApp definition stages. */ + /** + * The ContainerApp definition stages. + */ interface DefinitionStages { - /** The first stage of the ContainerApp definition. */ + /** + * The first stage of the ContainerApp definition. + */ interface Blank extends WithLocation { } - /** The stage of the ContainerApp definition allowing to specify location. */ + /** + * The stage of the ContainerApp definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -218,18 +225,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the ContainerApp definition allowing to specify parent resource. */ + /** + * The stage of the ContainerApp definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -240,132 +249,144 @@ interface WithResourceGroup { * The stage of the ContainerApp definition which contains all the minimum required properties for the resource * to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithExtendedLocation, - DefinitionStages.WithIdentity, - DefinitionStages.WithManagedBy, - DefinitionStages.WithManagedEnvironmentId, - DefinitionStages.WithEnvironmentId, - DefinitionStages.WithWorkloadProfileName, - DefinitionStages.WithConfiguration, - DefinitionStages.WithTemplate { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithExtendedLocation, + DefinitionStages.WithIdentity, DefinitionStages.WithManagedBy, DefinitionStages.WithManagedEnvironmentId, + DefinitionStages.WithEnvironmentId, DefinitionStages.WithWorkloadProfileName, + DefinitionStages.WithConfiguration, DefinitionStages.WithTemplate { /** * Executes the create request. - * + * * @return the created resource. */ ContainerApp create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ContainerApp create(Context context); } - /** The stage of the ContainerApp definition allowing to specify tags. */ + /** + * The stage of the ContainerApp definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the ContainerApp definition allowing to specify extendedLocation. */ + /** + * The stage of the ContainerApp definition allowing to specify extendedLocation. + */ interface WithExtendedLocation { /** * Specifies the extendedLocation property: The complex type of the extended location.. - * + * * @param extendedLocation The complex type of the extended location. * @return the next definition stage. */ WithCreate withExtendedLocation(ExtendedLocation extendedLocation); } - /** The stage of the ContainerApp definition allowing to specify identity. */ + /** + * The stage of the ContainerApp definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: managed identities for the Container App to interact with other Azure * services without maintaining any secrets or credentials in code.. - * + * * @param identity managed identities for the Container App to interact with other Azure services without - * maintaining any secrets or credentials in code. + * maintaining any secrets or credentials in code. * @return the next definition stage. */ WithCreate withIdentity(ManagedServiceIdentity identity); } - /** The stage of the ContainerApp definition allowing to specify managedBy. */ + /** + * The stage of the ContainerApp definition allowing to specify managedBy. + */ interface WithManagedBy { /** * Specifies the managedBy property: The fully qualified resource ID of the resource that manages this * resource. Indicates if this resource is managed by another Azure resource. If this is present, complete * mode deployment will not delete the resource if it is removed from the template since it is managed by * another resource.. - * + * * @param managedBy The fully qualified resource ID of the resource that manages this resource. Indicates if - * this resource is managed by another Azure resource. If this is present, complete mode deployment will - * not delete the resource if it is removed from the template since it is managed by another resource. + * this resource is managed by another Azure resource. If this is present, complete mode deployment will not + * delete the resource if it is removed from the template since it is managed by another resource. * @return the next definition stage. */ WithCreate withManagedBy(String managedBy); } - /** The stage of the ContainerApp definition allowing to specify managedEnvironmentId. */ + /** + * The stage of the ContainerApp definition allowing to specify managedEnvironmentId. + */ interface WithManagedEnvironmentId { /** * Specifies the managedEnvironmentId property: Deprecated. Resource ID of the Container App's environment.. - * + * * @param managedEnvironmentId Deprecated. Resource ID of the Container App's environment. * @return the next definition stage. */ WithCreate withManagedEnvironmentId(String managedEnvironmentId); } - /** The stage of the ContainerApp definition allowing to specify environmentId. */ + /** + * The stage of the ContainerApp definition allowing to specify environmentId. + */ interface WithEnvironmentId { /** * Specifies the environmentId property: Resource ID of environment.. - * + * * @param environmentId Resource ID of environment. * @return the next definition stage. */ WithCreate withEnvironmentId(String environmentId); } - /** The stage of the ContainerApp definition allowing to specify workloadProfileName. */ + /** + * The stage of the ContainerApp definition allowing to specify workloadProfileName. + */ interface WithWorkloadProfileName { /** * Specifies the workloadProfileName property: Workload profile name to pin for container app execution.. - * + * * @param workloadProfileName Workload profile name to pin for container app execution. * @return the next definition stage. */ WithCreate withWorkloadProfileName(String workloadProfileName); } - /** The stage of the ContainerApp definition allowing to specify configuration. */ + /** + * The stage of the ContainerApp definition allowing to specify configuration. + */ interface WithConfiguration { /** * Specifies the configuration property: Non versioned Container App configuration properties.. - * + * * @param configuration Non versioned Container App configuration properties. * @return the next definition stage. */ WithCreate withConfiguration(Configuration configuration); } - /** The stage of the ContainerApp definition allowing to specify template. */ + /** + * The stage of the ContainerApp definition allowing to specify template. + */ interface WithTemplate { /** * Specifies the template property: Container App versioned application definition.. - * + * * @param template Container App versioned application definition. * @return the next definition stage. */ @@ -375,116 +396,129 @@ interface WithTemplate { /** * Begins update for the ContainerApp resource. - * + * * @return the stage of resource update. */ ContainerApp.Update update(); - /** The template for ContainerApp update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithExtendedLocation, - UpdateStages.WithIdentity, - UpdateStages.WithManagedBy, - UpdateStages.WithWorkloadProfileName, - UpdateStages.WithConfiguration, - UpdateStages.WithTemplate { + /** + * The template for ContainerApp update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithExtendedLocation, UpdateStages.WithIdentity, + UpdateStages.WithManagedBy, UpdateStages.WithWorkloadProfileName, UpdateStages.WithConfiguration, + UpdateStages.WithTemplate { /** * Executes the update request. - * + * * @return the updated resource. */ ContainerApp apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ContainerApp apply(Context context); } - /** The ContainerApp update stages. */ + /** + * The ContainerApp update stages. + */ interface UpdateStages { - /** The stage of the ContainerApp update allowing to specify tags. */ + /** + * The stage of the ContainerApp update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the ContainerApp update allowing to specify extendedLocation. */ + /** + * The stage of the ContainerApp update allowing to specify extendedLocation. + */ interface WithExtendedLocation { /** * Specifies the extendedLocation property: The complex type of the extended location.. - * + * * @param extendedLocation The complex type of the extended location. * @return the next definition stage. */ Update withExtendedLocation(ExtendedLocation extendedLocation); } - /** The stage of the ContainerApp update allowing to specify identity. */ + /** + * The stage of the ContainerApp update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: managed identities for the Container App to interact with other Azure * services without maintaining any secrets or credentials in code.. - * + * * @param identity managed identities for the Container App to interact with other Azure services without - * maintaining any secrets or credentials in code. + * maintaining any secrets or credentials in code. * @return the next definition stage. */ Update withIdentity(ManagedServiceIdentity identity); } - /** The stage of the ContainerApp update allowing to specify managedBy. */ + /** + * The stage of the ContainerApp update allowing to specify managedBy. + */ interface WithManagedBy { /** * Specifies the managedBy property: The fully qualified resource ID of the resource that manages this * resource. Indicates if this resource is managed by another Azure resource. If this is present, complete * mode deployment will not delete the resource if it is removed from the template since it is managed by * another resource.. - * + * * @param managedBy The fully qualified resource ID of the resource that manages this resource. Indicates if - * this resource is managed by another Azure resource. If this is present, complete mode deployment will - * not delete the resource if it is removed from the template since it is managed by another resource. + * this resource is managed by another Azure resource. If this is present, complete mode deployment will not + * delete the resource if it is removed from the template since it is managed by another resource. * @return the next definition stage. */ Update withManagedBy(String managedBy); } - /** The stage of the ContainerApp update allowing to specify workloadProfileName. */ + /** + * The stage of the ContainerApp update allowing to specify workloadProfileName. + */ interface WithWorkloadProfileName { /** * Specifies the workloadProfileName property: Workload profile name to pin for container app execution.. - * + * * @param workloadProfileName Workload profile name to pin for container app execution. * @return the next definition stage. */ Update withWorkloadProfileName(String workloadProfileName); } - /** The stage of the ContainerApp update allowing to specify configuration. */ + /** + * The stage of the ContainerApp update allowing to specify configuration. + */ interface WithConfiguration { /** * Specifies the configuration property: Non versioned Container App configuration properties.. - * + * * @param configuration Non versioned Container App configuration properties. * @return the next definition stage. */ Update withConfiguration(Configuration configuration); } - /** The stage of the ContainerApp update allowing to specify template. */ + /** + * The stage of the ContainerApp update allowing to specify template. + */ interface WithTemplate { /** * Specifies the template property: Container App versioned application definition.. - * + * * @param template Container App versioned application definition. * @return the next definition stage. */ @@ -494,14 +528,14 @@ interface WithTemplate { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ContainerApp refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -509,23 +543,23 @@ interface WithTemplate { /** * Analyzes a custom hostname for a Container App. - * + * * @param customHostname Custom hostname. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return custom domain analysis along with {@link Response}. */ - Response listCustomHostnameAnalysisWithResponse( - String customHostname, Context context); + Response listCustomHostnameAnalysisWithResponse(String customHostname, + Context context); /** * Analyzes a custom hostname for a Container App. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return custom domain analysis. */ @@ -533,11 +567,11 @@ Response listCustomHostnameAnalysisWithResponse( /** * List secrets for a container app. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource along with {@link Response}. */ @@ -545,9 +579,9 @@ Response listCustomHostnameAnalysisWithResponse( /** * List secrets for a container app. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource. */ @@ -555,13 +589,13 @@ Response listCustomHostnameAnalysisWithResponse( /** * Get auth token for a container app. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return auth token for a container app along with {@link Response}. */ @@ -569,11 +603,11 @@ Response listCustomHostnameAnalysisWithResponse( /** * Get auth token for a container app. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return auth token for a container app. */ @@ -581,9 +615,9 @@ Response listCustomHostnameAnalysisWithResponse( /** * Start a container app. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -591,11 +625,11 @@ Response listCustomHostnameAnalysisWithResponse( /** * Start a container app. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -603,9 +637,9 @@ Response listCustomHostnameAnalysisWithResponse( /** * Stop a container app. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -613,11 +647,11 @@ Response listCustomHostnameAnalysisWithResponse( /** * Stop a container app. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppAuthToken.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppAuthToken.java index 8b00db3906e5c..48faf0c045db2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppAuthToken.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppAuthToken.java @@ -9,67 +9,69 @@ import java.time.OffsetDateTime; import java.util.Map; -/** An immutable client-side representation of ContainerAppAuthToken. */ +/** + * An immutable client-side representation of ContainerAppAuthToken. + */ public interface ContainerAppAuthToken { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the token property: Auth token value. - * + * * @return the token value. */ String token(); /** * Gets the expires property: Token expiration date. - * + * * @return the expires value. */ OffsetDateTime expires(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppAuthTokenInner object. - * + * * @return the inner object. */ ContainerAppAuthTokenInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppCollection.java index a84d9ec656764..5a225b40667cc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App collection ARM resource. */ +/** + * Container App collection ARM resource. + */ @Fluent public final class ContainerAppCollection { /* @@ -25,13 +27,15 @@ public final class ContainerAppCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ContainerAppCollection class. */ + /** + * Creates an instance of ContainerAppCollection class. + */ public ContainerAppCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the ContainerAppCollection object itself. */ @@ -51,7 +55,7 @@ public ContainerAppCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model ContainerAppCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ContainerAppCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppContainerRunningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppContainerRunningState.java index e70047c568736..861d6514b3b4a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppContainerRunningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppContainerRunningState.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current running state of the container. */ +/** + * Current running state of the container. + */ public final class ContainerAppContainerRunningState extends ExpandableStringEnum { - /** Static value Running for ContainerAppContainerRunningState. */ + /** + * Static value Running for ContainerAppContainerRunningState. + */ public static final ContainerAppContainerRunningState RUNNING = fromString("Running"); - /** Static value Terminated for ContainerAppContainerRunningState. */ + /** + * Static value Terminated for ContainerAppContainerRunningState. + */ public static final ContainerAppContainerRunningState TERMINATED = fromString("Terminated"); - /** Static value Waiting for ContainerAppContainerRunningState. */ + /** + * Static value Waiting for ContainerAppContainerRunningState. + */ public static final ContainerAppContainerRunningState WAITING = fromString("Waiting"); /** * Creates a new instance of ContainerAppContainerRunningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public ContainerAppContainerRunningState() { /** * Creates or finds a ContainerAppContainerRunningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ContainerAppContainerRunningState. */ @@ -41,7 +49,7 @@ public static ContainerAppContainerRunningState fromString(String name) { /** * Gets known ContainerAppContainerRunningState values. - * + * * @return known ContainerAppContainerRunningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppJobExecutions.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppJobExecutions.java index 126c8b0905436..2ff6428904194 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppJobExecutions.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppJobExecutions.java @@ -7,25 +7,27 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner; import java.util.List; -/** An immutable client-side representation of ContainerAppJobExecutions. */ +/** + * An immutable client-side representation of ContainerAppJobExecutions. + */ public interface ContainerAppJobExecutions { /** * Gets the value property: Collection of resources. - * + * * @return the value value. */ List value(); /** * Gets the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ String nextLink(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner object. - * + * * @return the inner object. */ ContainerAppJobExecutionsInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbe.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbe.java index 7b275d050e038..907cf9dd9aae5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbe.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbe.java @@ -77,14 +77,16 @@ public final class ContainerAppProbe { @JsonProperty(value = "type") private Type type; - /** Creates an instance of ContainerAppProbe class. */ + /** + * Creates an instance of ContainerAppProbe class. + */ public ContainerAppProbe() { } /** * Get the failureThreshold property: Minimum consecutive failures for the probe to be considered failed after * having succeeded. Defaults to 3. Minimum value is 1. Maximum value is 10. - * + * * @return the failureThreshold value. */ public Integer failureThreshold() { @@ -94,7 +96,7 @@ public Integer failureThreshold() { /** * Set the failureThreshold property: Minimum consecutive failures for the probe to be considered failed after * having succeeded. Defaults to 3. Minimum value is 1. Maximum value is 10. - * + * * @param failureThreshold the failureThreshold value to set. * @return the ContainerAppProbe object itself. */ @@ -105,7 +107,7 @@ public ContainerAppProbe withFailureThreshold(Integer failureThreshold) { /** * Get the httpGet property: HTTPGet specifies the http request to perform. - * + * * @return the httpGet value. */ public ContainerAppProbeHttpGet httpGet() { @@ -114,7 +116,7 @@ public ContainerAppProbeHttpGet httpGet() { /** * Set the httpGet property: HTTPGet specifies the http request to perform. - * + * * @param httpGet the httpGet value to set. * @return the ContainerAppProbe object itself. */ @@ -126,7 +128,7 @@ public ContainerAppProbe withHttpGet(ContainerAppProbeHttpGet httpGet) { /** * Get the initialDelaySeconds property: Number of seconds after the container has started before liveness probes * are initiated. Minimum value is 1. Maximum value is 60. - * + * * @return the initialDelaySeconds value. */ public Integer initialDelaySeconds() { @@ -136,7 +138,7 @@ public Integer initialDelaySeconds() { /** * Set the initialDelaySeconds property: Number of seconds after the container has started before liveness probes * are initiated. Minimum value is 1. Maximum value is 60. - * + * * @param initialDelaySeconds the initialDelaySeconds value to set. * @return the ContainerAppProbe object itself. */ @@ -146,9 +148,9 @@ public ContainerAppProbe withInitialDelaySeconds(Integer initialDelaySeconds) { } /** - * Get the periodSeconds property: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value - * is 1. Maximum value is 240. - * + * Get the periodSeconds property: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum + * value is 1. Maximum value is 240. + * * @return the periodSeconds value. */ public Integer periodSeconds() { @@ -156,9 +158,9 @@ public Integer periodSeconds() { } /** - * Set the periodSeconds property: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum value - * is 1. Maximum value is 240. - * + * Set the periodSeconds property: How often (in seconds) to perform the probe. Default to 10 seconds. Minimum + * value is 1. Maximum value is 240. + * * @param periodSeconds the periodSeconds value to set. * @return the ContainerAppProbe object itself. */ @@ -170,7 +172,7 @@ public ContainerAppProbe withPeriodSeconds(Integer periodSeconds) { /** * Get the successThreshold property: Minimum consecutive successes for the probe to be considered successful after * having failed. Defaults to 1. Must be 1 for liveness and startup. Minimum value is 1. Maximum value is 10. - * + * * @return the successThreshold value. */ public Integer successThreshold() { @@ -180,7 +182,7 @@ public Integer successThreshold() { /** * Set the successThreshold property: Minimum consecutive successes for the probe to be considered successful after * having failed. Defaults to 1. Must be 1 for liveness and startup. Minimum value is 1. Maximum value is 10. - * + * * @param successThreshold the successThreshold value to set. * @return the ContainerAppProbe object itself. */ @@ -191,7 +193,7 @@ public ContainerAppProbe withSuccessThreshold(Integer successThreshold) { /** * Get the tcpSocket property: TCPSocket specifies an action involving a TCP port. TCP hooks not yet supported. - * + * * @return the tcpSocket value. */ public ContainerAppProbeTcpSocket tcpSocket() { @@ -200,7 +202,7 @@ public ContainerAppProbeTcpSocket tcpSocket() { /** * Set the tcpSocket property: TCPSocket specifies an action involving a TCP port. TCP hooks not yet supported. - * + * * @param tcpSocket the tcpSocket value to set. * @return the ContainerAppProbe object itself. */ @@ -211,14 +213,14 @@ public ContainerAppProbe withTcpSocket(ContainerAppProbeTcpSocket tcpSocket) { /** * Get the terminationGracePeriodSeconds property: Optional duration in seconds the pod needs to terminate - * gracefully upon probe failure. The grace period is the duration in seconds after the processes running in the pod - * are sent a termination signal and the time when the processes are forcibly halted with a kill signal. Set this - * value longer than the expected cleanup time for your process. If this value is nil, the pod's + * gracefully upon probe failure. The grace period is the duration in seconds after the processes running in the + * pod are sent a termination signal and the time when the processes are forcibly halted with a kill signal. Set + * this value longer than the expected cleanup time for your process. If this value is nil, the pod's * terminationGracePeriodSeconds will be used. Otherwise, this value overrides the value provided by the pod spec. - * Value must be non-negative integer. The value zero indicates stop immediately via the kill signal (no opportunity - * to shut down). This is an alpha field and requires enabling ProbeTerminationGracePeriod feature gate. Maximum - * value is 3600 seconds (1 hour). - * + * Value must be non-negative integer. The value zero indicates stop immediately via the kill signal (no + * opportunity to shut down). This is an alpha field and requires enabling ProbeTerminationGracePeriod feature + * gate. Maximum value is 3600 seconds (1 hour). + * * @return the terminationGracePeriodSeconds value. */ public Long terminationGracePeriodSeconds() { @@ -227,14 +229,14 @@ public Long terminationGracePeriodSeconds() { /** * Set the terminationGracePeriodSeconds property: Optional duration in seconds the pod needs to terminate - * gracefully upon probe failure. The grace period is the duration in seconds after the processes running in the pod - * are sent a termination signal and the time when the processes are forcibly halted with a kill signal. Set this - * value longer than the expected cleanup time for your process. If this value is nil, the pod's + * gracefully upon probe failure. The grace period is the duration in seconds after the processes running in the + * pod are sent a termination signal and the time when the processes are forcibly halted with a kill signal. Set + * this value longer than the expected cleanup time for your process. If this value is nil, the pod's * terminationGracePeriodSeconds will be used. Otherwise, this value overrides the value provided by the pod spec. - * Value must be non-negative integer. The value zero indicates stop immediately via the kill signal (no opportunity - * to shut down). This is an alpha field and requires enabling ProbeTerminationGracePeriod feature gate. Maximum - * value is 3600 seconds (1 hour). - * + * Value must be non-negative integer. The value zero indicates stop immediately via the kill signal (no + * opportunity to shut down). This is an alpha field and requires enabling ProbeTerminationGracePeriod feature + * gate. Maximum value is 3600 seconds (1 hour). + * * @param terminationGracePeriodSeconds the terminationGracePeriodSeconds value to set. * @return the ContainerAppProbe object itself. */ @@ -244,9 +246,9 @@ public ContainerAppProbe withTerminationGracePeriodSeconds(Long terminationGrace } /** - * Get the timeoutSeconds property: Number of seconds after which the probe times out. Defaults to 1 second. Minimum - * value is 1. Maximum value is 240. - * + * Get the timeoutSeconds property: Number of seconds after which the probe times out. Defaults to 1 second. + * Minimum value is 1. Maximum value is 240. + * * @return the timeoutSeconds value. */ public Integer timeoutSeconds() { @@ -254,9 +256,9 @@ public Integer timeoutSeconds() { } /** - * Set the timeoutSeconds property: Number of seconds after which the probe times out. Defaults to 1 second. Minimum - * value is 1. Maximum value is 240. - * + * Set the timeoutSeconds property: Number of seconds after which the probe times out. Defaults to 1 second. + * Minimum value is 1. Maximum value is 240. + * * @param timeoutSeconds the timeoutSeconds value to set. * @return the ContainerAppProbe object itself. */ @@ -267,7 +269,7 @@ public ContainerAppProbe withTimeoutSeconds(Integer timeoutSeconds) { /** * Get the type property: The type of probe. - * + * * @return the type value. */ public Type type() { @@ -276,7 +278,7 @@ public Type type() { /** * Set the type property: The type of probe. - * + * * @param type the type value to set. * @return the ContainerAppProbe object itself. */ @@ -287,7 +289,7 @@ public ContainerAppProbe withType(Type type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbeHttpGet.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbeHttpGet.java index e02fc66557832..1de99cf72dbbe 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbeHttpGet.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppProbeHttpGet.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** HTTPGet specifies the http request to perform. */ +/** + * HTTPGet specifies the http request to perform. + */ @Fluent public final class ContainerAppProbeHttpGet { /* @@ -42,14 +44,16 @@ public final class ContainerAppProbeHttpGet { @JsonProperty(value = "scheme") private Scheme scheme; - /** Creates an instance of ContainerAppProbeHttpGet class. */ + /** + * Creates an instance of ContainerAppProbeHttpGet class. + */ public ContainerAppProbeHttpGet() { } /** * Get the host property: Host name to connect to, defaults to the pod IP. You probably want to set "Host" in * httpHeaders instead. - * + * * @return the host value. */ public String host() { @@ -59,7 +63,7 @@ public String host() { /** * Set the host property: Host name to connect to, defaults to the pod IP. You probably want to set "Host" in * httpHeaders instead. - * + * * @param host the host value to set. * @return the ContainerAppProbeHttpGet object itself. */ @@ -70,7 +74,7 @@ public ContainerAppProbeHttpGet withHost(String host) { /** * Get the httpHeaders property: Custom headers to set in the request. HTTP allows repeated headers. - * + * * @return the httpHeaders value. */ public List httpHeaders() { @@ -79,7 +83,7 @@ public List httpHeaders() { /** * Set the httpHeaders property: Custom headers to set in the request. HTTP allows repeated headers. - * + * * @param httpHeaders the httpHeaders value to set. * @return the ContainerAppProbeHttpGet object itself. */ @@ -90,7 +94,7 @@ public ContainerAppProbeHttpGet withHttpHeaders(List { - /** Static value InProgress for ContainerAppProvisioningState. */ + /** + * Static value InProgress for ContainerAppProvisioningState. + */ public static final ContainerAppProvisioningState IN_PROGRESS = fromString("InProgress"); - /** Static value Succeeded for ContainerAppProvisioningState. */ + /** + * Static value Succeeded for ContainerAppProvisioningState. + */ public static final ContainerAppProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for ContainerAppProvisioningState. */ + /** + * Static value Failed for ContainerAppProvisioningState. + */ public static final ContainerAppProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ContainerAppProvisioningState. */ + /** + * Static value Canceled for ContainerAppProvisioningState. + */ public static final ContainerAppProvisioningState CANCELED = fromString("Canceled"); - /** Static value Deleting for ContainerAppProvisioningState. */ + /** + * Static value Deleting for ContainerAppProvisioningState. + */ public static final ContainerAppProvisioningState DELETING = fromString("Deleting"); /** * Creates a new instance of ContainerAppProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public ContainerAppProvisioningState() { /** * Creates or finds a ContainerAppProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ContainerAppProvisioningState. */ @@ -47,7 +59,7 @@ public static ContainerAppProvisioningState fromString(String name) { /** * Gets known ContainerAppProvisioningState values. - * + * * @return known ContainerAppProvisioningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppReplicaRunningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppReplicaRunningState.java index a480ec43349e9..fa8058f0d086b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppReplicaRunningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppReplicaRunningState.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current running state of the replica. */ +/** + * Current running state of the replica. + */ public final class ContainerAppReplicaRunningState extends ExpandableStringEnum { - /** Static value Running for ContainerAppReplicaRunningState. */ + /** + * Static value Running for ContainerAppReplicaRunningState. + */ public static final ContainerAppReplicaRunningState RUNNING = fromString("Running"); - /** Static value NotRunning for ContainerAppReplicaRunningState. */ + /** + * Static value NotRunning for ContainerAppReplicaRunningState. + */ public static final ContainerAppReplicaRunningState NOT_RUNNING = fromString("NotRunning"); - /** Static value Unknown for ContainerAppReplicaRunningState. */ + /** + * Static value Unknown for ContainerAppReplicaRunningState. + */ public static final ContainerAppReplicaRunningState UNKNOWN = fromString("Unknown"); /** * Creates a new instance of ContainerAppReplicaRunningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public ContainerAppReplicaRunningState() { /** * Creates or finds a ContainerAppReplicaRunningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ContainerAppReplicaRunningState. */ @@ -41,7 +49,7 @@ public static ContainerAppReplicaRunningState fromString(String name) { /** * Gets known ContainerAppReplicaRunningState values. - * + * * @return known ContainerAppReplicaRunningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppSecret.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppSecret.java index 9c65dc9389fef..023aaea23ec4f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppSecret.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppSecret.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App Secret. */ +/** + * Container App Secret. + */ @Immutable public final class ContainerAppSecret { /* @@ -35,13 +37,15 @@ public final class ContainerAppSecret { @JsonProperty(value = "keyVaultUrl", access = JsonProperty.Access.WRITE_ONLY) private String keyVaultUrl; - /** Creates an instance of ContainerAppSecret class. */ + /** + * Creates an instance of ContainerAppSecret class. + */ public ContainerAppSecret() { } /** * Get the name property: Secret Name. - * + * * @return the name value. */ public String name() { @@ -50,7 +54,7 @@ public String name() { /** * Get the value property: Secret Value. - * + * * @return the value value. */ public String value() { @@ -60,7 +64,7 @@ public String value() { /** * Get the identity property: Resource ID of a managed identity to authenticate with Azure Key Vault, or System to * use a system-assigned identity. - * + * * @return the identity value. */ public String identity() { @@ -69,7 +73,7 @@ public String identity() { /** * Get the keyVaultUrl property: Azure Key Vault URL pointing to the secret referenced by the container app. - * + * * @return the keyVaultUrl value. */ public String keyVaultUrl() { @@ -78,7 +82,7 @@ public String keyVaultUrl() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApps.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApps.java index 2f770f1625746..c695056471b94 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApps.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerApps.java @@ -8,13 +8,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ContainerApps. */ +/** + * Resource collection API of ContainerApps. + */ public interface ContainerApps { /** * Get the Container Apps in a given subscription. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription as paginated response with {@link PagedIterable}. */ @@ -22,11 +24,11 @@ public interface ContainerApps { /** * Get the Container Apps in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given subscription as paginated response with {@link PagedIterable}. */ @@ -34,11 +36,11 @@ public interface ContainerApps { /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given resource group as paginated response with {@link PagedIterable}. */ @@ -46,12 +48,12 @@ public interface ContainerApps { /** * Get the Container Apps in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps in a given resource group as paginated response with {@link PagedIterable}. */ @@ -59,31 +61,31 @@ public interface ContainerApps { /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String containerAppName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String containerAppName, + Context context); /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App. */ @@ -91,53 +93,53 @@ Response getByResourceGroupWithResponse( /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByResourceGroup(String resourceGroupName, String containerAppName); /** * Delete a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String containerAppName, Context context); /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param customHostname Custom hostname. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return custom domain analysis along with {@link Response}. */ - Response listCustomHostnameAnalysisWithResponse( - String resourceGroupName, String containerAppName, String customHostname, Context context); + Response listCustomHostnameAnalysisWithResponse(String resourceGroupName, + String containerAppName, String customHostname, Context context); /** * Analyzes a custom hostname for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return custom domain analysis. */ @@ -145,27 +147,27 @@ Response listCustomHostnameAnalysisWithResponse( /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource along with {@link Response}. */ - Response listSecretsWithResponse( - String resourceGroupName, String containerAppName, Context context); + Response listSecretsWithResponse(String resourceGroupName, String containerAppName, + Context context); /** * List secrets for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Secrets Collection ARM resource. */ @@ -173,31 +175,31 @@ Response listSecretsWithResponse( /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return auth token for a container app along with {@link Response}. */ - Response getAuthTokenWithResponse( - String resourceGroupName, String containerAppName, Context context); + Response getAuthTokenWithResponse(String resourceGroupName, String containerAppName, + Context context); /** * Get auth token for a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return auth token for a container app. */ @@ -205,12 +207,12 @@ Response getAuthTokenWithResponse( /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -218,13 +220,13 @@ Response getAuthTokenWithResponse( /** * Start a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -232,12 +234,12 @@ Response getAuthTokenWithResponse( /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -245,13 +247,13 @@ Response getAuthTokenWithResponse( /** * Stop a container app. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App. */ @@ -259,13 +261,13 @@ Response getAuthTokenWithResponse( /** * Get the properties of a Container App. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App along with {@link Response}. */ @@ -273,14 +275,14 @@ Response getAuthTokenWithResponse( /** * Get the properties of a Container App. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App along with {@link Response}. */ @@ -288,30 +290,30 @@ Response getAuthTokenWithResponse( /** * Delete a Container App. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete a Container App. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new ContainerApp resource. - * + * * @param name resource name. * @return the first stage of the new ContainerApp definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsAuthConfigs.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsAuthConfigs.java index 09373e21dca18..1859a10571089 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsAuthConfigs.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsAuthConfigs.java @@ -8,16 +8,18 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ContainerAppsAuthConfigs. */ +/** + * Resource collection API of ContainerAppsAuthConfigs. + */ public interface ContainerAppsAuthConfigs { /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App AuthConfigs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -25,13 +27,13 @@ public interface ContainerAppsAuthConfigs { /** * Get the Container App AuthConfigs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container App AuthConfigs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -39,29 +41,29 @@ public interface ContainerAppsAuthConfigs { /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a AuthConfig of a Container App along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context); + Response getWithResponse(String resourceGroupName, String containerAppName, String authConfigName, + Context context); /** * Get a AuthConfig of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a AuthConfig of a Container App. */ @@ -69,40 +71,40 @@ Response getWithResponse( /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String containerAppName, String authConfigName, Context context); + Response deleteWithResponse(String resourceGroupName, String containerAppName, String authConfigName, + Context context); /** * Delete a Container App AuthConfig. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param authConfigName Name of the Container App AuthConfig. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String containerAppName, String authConfigName); /** * Get a AuthConfig of a Container App. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a AuthConfig of a Container App along with {@link Response}. */ @@ -110,12 +112,12 @@ Response deleteWithResponse( /** * Get a AuthConfig of a Container App. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a AuthConfig of a Container App along with {@link Response}. */ @@ -123,23 +125,23 @@ Response deleteWithResponse( /** * Delete a Container App AuthConfig. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete a Container App AuthConfig. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @@ -147,7 +149,7 @@ Response deleteWithResponse( /** * Begins definition for a new AuthConfig resource. - * + * * @param name resource name. * @return the first stage of the new AuthConfig definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsDiagnostics.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsDiagnostics.java index 5b4e2d2cbc3ac..74e0bd45eb2cc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsDiagnostics.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsDiagnostics.java @@ -8,16 +8,18 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ContainerAppsDiagnostics. */ +/** + * Resource collection API of ContainerAppsDiagnostics. + */ public interface ContainerAppsDiagnostics { /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a given Container App as paginated response with {@link PagedIterable}. */ @@ -25,13 +27,13 @@ public interface ContainerAppsDiagnostics { /** * Get the list of diagnostics for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which detector info is needed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a given Container App as paginated response with {@link PagedIterable}. */ @@ -39,29 +41,29 @@ public interface ContainerAppsDiagnostics { /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a diagnostics result of a Container App along with {@link Response}. */ - Response getDetectorWithResponse( - String resourceGroupName, String containerAppName, String detectorName, Context context); + Response getDetectorWithResponse(String resourceGroupName, String containerAppName, + String detectorName, Context context); /** * Get a diagnostics result of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param detectorName Name of the Container App Detector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a diagnostics result of a Container App. */ @@ -69,12 +71,12 @@ Response getDetectorWithResponse( /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @@ -82,45 +84,45 @@ Response getDetectorWithResponse( /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ - PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context); + PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App along with {@link Response}. */ - Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response getRevisionWithResponse(String resourceGroupName, String containerAppName, String revisionName, + Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App. */ @@ -128,15 +130,15 @@ Response getRevisionWithResponse( /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App along with {@link Response}. */ @@ -144,14 +146,14 @@ Response getRevisionWithResponse( /** * Get the properties of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server on - * status code 404. + * status code 404. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container App. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisionReplicas.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisionReplicas.java index 221ffd8e23ced..0bc8de8d41d7a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisionReplicas.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisionReplicas.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ContainerAppsRevisionReplicas. */ +/** + * Resource collection API of ContainerAppsRevisionReplicas. + */ public interface ContainerAppsRevisionReplicas { /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. @@ -19,23 +21,23 @@ public interface ContainerAppsRevisionReplicas { * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a replica for a Container App Revision along with {@link Response}. */ - Response getReplicaWithResponse( - String resourceGroupName, String containerAppName, String revisionName, String replicaName, Context context); + Response getReplicaWithResponse(String resourceGroupName, String containerAppName, String revisionName, + String replicaName, Context context); /** * Get a replica for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param replicaName Name of the Container App Revision Replica. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a replica for a Container App Revision. */ @@ -43,29 +45,29 @@ Response getReplicaWithResponse( /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revision Replicas collection ARM resource along with {@link Response}. */ - Response listReplicasWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response listReplicasWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context); /** * List replicas for a Container App Revision. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App Revision Replicas collection ARM resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisions.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisions.java index 4d6a333e6fb60..60cb3b6963f77 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisions.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsRevisions.java @@ -8,16 +8,18 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ContainerAppsRevisions. */ +/** + * Resource collection API of ContainerAppsRevisions. + */ public interface ContainerAppsRevisions { /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ @@ -25,45 +27,45 @@ public interface ContainerAppsRevisions { /** * Get the Revisions for a given Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App for which Revisions are needed. * @param filter The filter to apply on the operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Revisions for a given Container App as paginated response with {@link PagedIterable}. */ - PagedIterable listRevisions( - String resourceGroupName, String containerAppName, String filter, Context context); + PagedIterable listRevisions(String resourceGroupName, String containerAppName, String filter, + Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App along with {@link Response}. */ - Response getRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response getRevisionWithResponse(String resourceGroupName, String containerAppName, String revisionName, + Context context); /** * Get a revision of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a revision of a Container App. */ @@ -71,87 +73,87 @@ Response getRevisionWithResponse( /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response activateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response activateRevisionWithResponse(String resourceGroupName, String containerAppName, String revisionName, + Context context); /** * Activates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void activateRevision(String resourceGroupName, String containerAppName, String revisionName); /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deactivateRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response deactivateRevisionWithResponse(String resourceGroupName, String containerAppName, + String revisionName, Context context); /** * Deactivates a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deactivateRevision(String resourceGroupName, String containerAppName, String revisionName); /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response restartRevisionWithResponse( - String resourceGroupName, String containerAppName, String revisionName, Context context); + Response restartRevisionWithResponse(String resourceGroupName, String containerAppName, String revisionName, + Context context); /** * Restarts a revision for a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param revisionName Name of the Container App Revision. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void restartRevision(String resourceGroupName, String containerAppName, String revisionName); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsSourceControls.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsSourceControls.java index 1506120e596a9..41cc21d952a22 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsSourceControls.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerAppsSourceControls.java @@ -8,62 +8,64 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ContainerAppsSourceControls. */ +/** + * Resource collection API of ContainerAppsSourceControls. + */ public interface ContainerAppsSourceControls { /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Container App SourceControls in a given resource group as paginated response with {@link - * PagedIterable}. + * @return the Container App SourceControls in a given resource group as paginated response with + * {@link PagedIterable}. */ PagedIterable listByContainerApp(String resourceGroupName, String containerAppName); /** * Get the Container App SourceControls in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Container App SourceControls in a given resource group as paginated response with {@link - * PagedIterable}. + * @return the Container App SourceControls in a given resource group as paginated response with + * {@link PagedIterable}. */ PagedIterable listByContainerApp(String resourceGroupName, String containerAppName, Context context); /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a SourceControl of a Container App along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String containerAppName, String sourceControlName, Context context); + Response getWithResponse(String resourceGroupName, String containerAppName, String sourceControlName, + Context context); /** * Get a SourceControl of a Container App. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a SourceControl of a Container App. */ @@ -71,38 +73,38 @@ Response getWithResponse( /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String containerAppName, String sourceControlName); /** * Delete a Container App SourceControl. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @param sourceControlName Name of the Container App SourceControl. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String containerAppName, String sourceControlName, Context context); /** * Get a SourceControl of a Container App. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a SourceControl of a Container App along with {@link Response}. */ @@ -110,12 +112,12 @@ Response getWithResponse( /** * Get a SourceControl of a Container App. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a SourceControl of a Container App along with {@link Response}. */ @@ -123,30 +125,30 @@ Response getWithResponse( /** * Delete a Container App SourceControl. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete a Container App SourceControl. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new SourceControl resource. - * + * * @param name resource name. * @return the first stage of the new SourceControl definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistry.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistry.java new file mode 100644 index 0000000000000..8c1b99939147a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistry.java @@ -0,0 +1,91 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Model representing a mapping from a container registry to the identity used to connect to it. + */ +@Fluent +public final class ContainerRegistry { + /* + * Login server of the container registry. + */ + @JsonProperty(value = "containerRegistryServer", required = true) + private String containerRegistryServer; + + /* + * Resource ID of the managed identity. + */ + @JsonProperty(value = "identityResourceId", required = true) + private String identityResourceId; + + /** + * Creates an instance of ContainerRegistry class. + */ + public ContainerRegistry() { + } + + /** + * Get the containerRegistryServer property: Login server of the container registry. + * + * @return the containerRegistryServer value. + */ + public String containerRegistryServer() { + return this.containerRegistryServer; + } + + /** + * Set the containerRegistryServer property: Login server of the container registry. + * + * @param containerRegistryServer the containerRegistryServer value to set. + * @return the ContainerRegistry object itself. + */ + public ContainerRegistry withContainerRegistryServer(String containerRegistryServer) { + this.containerRegistryServer = containerRegistryServer; + return this; + } + + /** + * Get the identityResourceId property: Resource ID of the managed identity. + * + * @return the identityResourceId value. + */ + public String identityResourceId() { + return this.identityResourceId; + } + + /** + * Set the identityResourceId property: Resource ID of the managed identity. + * + * @param identityResourceId the identityResourceId value to set. + * @return the ContainerRegistry object itself. + */ + public ContainerRegistry withIdentityResourceId(String identityResourceId) { + this.identityResourceId = identityResourceId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (containerRegistryServer() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property containerRegistryServer in model ContainerRegistry")); + } + if (identityResourceId() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property identityResourceId in model ContainerRegistry")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(ContainerRegistry.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistryWithCustomImage.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistryWithCustomImage.java new file mode 100644 index 0000000000000..2682fc52cb1fb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerRegistryWithCustomImage.java @@ -0,0 +1,90 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Container registry that the final image will be uploaded to. + */ +@Fluent +public final class ContainerRegistryWithCustomImage { + /* + * Login server of the container registry that the final image should be uploaded to. Builder resource needs to + * have this container registry defined along with an identity to use to access it. + */ + @JsonProperty(value = "server", required = true) + private String server; + + /* + * Full name that the final image should be uploaded as, including both image name and tag. + */ + @JsonProperty(value = "image") + private String image; + + /** + * Creates an instance of ContainerRegistryWithCustomImage class. + */ + public ContainerRegistryWithCustomImage() { + } + + /** + * Get the server property: Login server of the container registry that the final image should be uploaded to. + * Builder resource needs to have this container registry defined along with an identity to use to access it. + * + * @return the server value. + */ + public String server() { + return this.server; + } + + /** + * Set the server property: Login server of the container registry that the final image should be uploaded to. + * Builder resource needs to have this container registry defined along with an identity to use to access it. + * + * @param server the server value to set. + * @return the ContainerRegistryWithCustomImage object itself. + */ + public ContainerRegistryWithCustomImage withServer(String server) { + this.server = server; + return this; + } + + /** + * Get the image property: Full name that the final image should be uploaded as, including both image name and tag. + * + * @return the image value. + */ + public String image() { + return this.image; + } + + /** + * Set the image property: Full name that the final image should be uploaded as, including both image name and tag. + * + * @param image the image value to set. + * @return the ContainerRegistryWithCustomImage object itself. + */ + public ContainerRegistryWithCustomImage withImage(String image) { + this.image = image; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (server() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property server in model ContainerRegistryWithCustomImage")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(ContainerRegistryWithCustomImage.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerResources.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerResources.java index 2daecab42dbee..eb5ba7e7e19d4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerResources.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ContainerResources.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App container resource requirements. */ +/** + * Container App container resource requirements. + */ @Fluent public final class ContainerResources { /* @@ -28,13 +30,15 @@ public final class ContainerResources { @JsonProperty(value = "ephemeralStorage", access = JsonProperty.Access.WRITE_ONLY) private String ephemeralStorage; - /** Creates an instance of ContainerResources class. */ + /** + * Creates an instance of ContainerResources class. + */ public ContainerResources() { } /** * Get the cpu property: Required CPU in cores, e.g. 0.5. - * + * * @return the cpu value. */ public Double cpu() { @@ -43,7 +47,7 @@ public Double cpu() { /** * Set the cpu property: Required CPU in cores, e.g. 0.5. - * + * * @param cpu the cpu value to set. * @return the ContainerResources object itself. */ @@ -54,7 +58,7 @@ public ContainerResources withCpu(Double cpu) { /** * Get the memory property: Required memory, e.g. "250Mb". - * + * * @return the memory value. */ public String memory() { @@ -63,7 +67,7 @@ public String memory() { /** * Set the memory property: Required memory, e.g. "250Mb". - * + * * @param memory the memory value to set. * @return the ContainerResources object itself. */ @@ -74,7 +78,7 @@ public ContainerResources withMemory(String memory) { /** * Get the ephemeralStorage property: Ephemeral Storage, e.g. "1Gi". - * + * * @return the ephemeralStorage value. */ public String ephemeralStorage() { @@ -83,7 +87,7 @@ public String ephemeralStorage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpiration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpiration.java index 3eb091b10ed5d..5b4454d8e747f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpiration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpiration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the session cookie's expiration. */ +/** + * The configuration settings of the session cookie's expiration. + */ @Fluent public final class CookieExpiration { /* @@ -22,13 +24,15 @@ public final class CookieExpiration { @JsonProperty(value = "timeToExpiration") private String timeToExpiration; - /** Creates an instance of CookieExpiration class. */ + /** + * Creates an instance of CookieExpiration class. + */ public CookieExpiration() { } /** * Get the convention property: The convention used when determining the session cookie's expiration. - * + * * @return the convention value. */ public CookieExpirationConvention convention() { @@ -37,7 +41,7 @@ public CookieExpirationConvention convention() { /** * Set the convention property: The convention used when determining the session cookie's expiration. - * + * * @param convention the convention value to set. * @return the CookieExpiration object itself. */ @@ -48,7 +52,7 @@ public CookieExpiration withConvention(CookieExpirationConvention convention) { /** * Get the timeToExpiration property: The time after the request is made when the session cookie should expire. - * + * * @return the timeToExpiration value. */ public String timeToExpiration() { @@ -57,7 +61,7 @@ public String timeToExpiration() { /** * Set the timeToExpiration property: The time after the request is made when the session cookie should expire. - * + * * @param timeToExpiration the timeToExpiration value to set. * @return the CookieExpiration object itself. */ @@ -68,7 +72,7 @@ public CookieExpiration withTimeToExpiration(String timeToExpiration) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpirationConvention.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpirationConvention.java index defc5483b6e0d..8680a3d10e9f2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpirationConvention.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CookieExpirationConvention.java @@ -7,15 +7,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** The convention used when determining the session cookie's expiration. */ +/** + * The convention used when determining the session cookie's expiration. + */ public enum CookieExpirationConvention { - /** Enum value FixedTime. */ + /** + * Enum value FixedTime. + */ FIXED_TIME("FixedTime"), - /** Enum value IdentityProviderDerived. */ + /** + * Enum value IdentityProviderDerived. + */ IDENTITY_PROVIDER_DERIVED("IdentityProviderDerived"); - /** The actual serialized value for a CookieExpirationConvention instance. */ + /** + * The actual serialized value for a CookieExpirationConvention instance. + */ private final String value; CookieExpirationConvention(String value) { @@ -24,7 +32,7 @@ public enum CookieExpirationConvention { /** * Parses a serialized value to a CookieExpirationConvention instance. - * + * * @param value the serialized value to parse. * @return the parsed CookieExpirationConvention object, or null if unable to parse. */ @@ -42,7 +50,9 @@ public static CookieExpirationConvention fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CorsPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CorsPolicy.java index c59afb15a75f6..9c0a944196c1e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CorsPolicy.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CorsPolicy.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Cross-Origin-Resource-Sharing policy. */ +/** + * Cross-Origin-Resource-Sharing policy. + */ @Fluent public final class CorsPolicy { /* @@ -48,13 +50,15 @@ public final class CorsPolicy { @JsonProperty(value = "allowCredentials") private Boolean allowCredentials; - /** Creates an instance of CorsPolicy class. */ + /** + * Creates an instance of CorsPolicy class. + */ public CorsPolicy() { } /** * Get the allowedOrigins property: Specifies the content for the access-control-allow-origins header. - * + * * @return the allowedOrigins value. */ public List allowedOrigins() { @@ -63,7 +67,7 @@ public List allowedOrigins() { /** * Set the allowedOrigins property: Specifies the content for the access-control-allow-origins header. - * + * * @param allowedOrigins the allowedOrigins value to set. * @return the CorsPolicy object itself. */ @@ -74,7 +78,7 @@ public CorsPolicy withAllowedOrigins(List allowedOrigins) { /** * Get the allowedMethods property: Specifies the content for the access-control-allow-methods header. - * + * * @return the allowedMethods value. */ public List allowedMethods() { @@ -83,7 +87,7 @@ public List allowedMethods() { /** * Set the allowedMethods property: Specifies the content for the access-control-allow-methods header. - * + * * @param allowedMethods the allowedMethods value to set. * @return the CorsPolicy object itself. */ @@ -94,7 +98,7 @@ public CorsPolicy withAllowedMethods(List allowedMethods) { /** * Get the allowedHeaders property: Specifies the content for the access-control-allow-headers header. - * + * * @return the allowedHeaders value. */ public List allowedHeaders() { @@ -103,7 +107,7 @@ public List allowedHeaders() { /** * Set the allowedHeaders property: Specifies the content for the access-control-allow-headers header. - * + * * @param allowedHeaders the allowedHeaders value to set. * @return the CorsPolicy object itself. */ @@ -114,7 +118,7 @@ public CorsPolicy withAllowedHeaders(List allowedHeaders) { /** * Get the exposeHeaders property: Specifies the content for the access-control-expose-headers header. - * + * * @return the exposeHeaders value. */ public List exposeHeaders() { @@ -123,7 +127,7 @@ public List exposeHeaders() { /** * Set the exposeHeaders property: Specifies the content for the access-control-expose-headers header. - * + * * @param exposeHeaders the exposeHeaders value to set. * @return the CorsPolicy object itself. */ @@ -134,7 +138,7 @@ public CorsPolicy withExposeHeaders(List exposeHeaders) { /** * Get the maxAge property: Specifies the content for the access-control-max-age header. - * + * * @return the maxAge value. */ public Integer maxAge() { @@ -143,7 +147,7 @@ public Integer maxAge() { /** * Set the maxAge property: Specifies the content for the access-control-max-age header. - * + * * @param maxAge the maxAge value to set. * @return the CorsPolicy object itself. */ @@ -154,7 +158,7 @@ public CorsPolicy withMaxAge(Integer maxAge) { /** * Get the allowCredentials property: Specifies whether the resource allows credentials. - * + * * @return the allowCredentials value. */ public Boolean allowCredentials() { @@ -163,7 +167,7 @@ public Boolean allowCredentials() { /** * Set the allowCredentials property: Specifies whether the resource allows credentials. - * + * * @param allowCredentials the allowCredentials value to set. * @return the CorsPolicy object itself. */ @@ -174,14 +178,13 @@ public CorsPolicy withAllowCredentials(Boolean allowCredentials) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (allowedOrigins() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property allowedOrigins in model CorsPolicy")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property allowedOrigins in model CorsPolicy")); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomain.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomain.java index a57a362a886de..298c01667aba0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomain.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomain.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Custom Domain of a Container App. */ +/** + * Custom Domain of a Container App. + */ @Fluent public final class CustomDomain { /* @@ -29,13 +31,15 @@ public final class CustomDomain { @JsonProperty(value = "certificateId") private String certificateId; - /** Creates an instance of CustomDomain class. */ + /** + * Creates an instance of CustomDomain class. + */ public CustomDomain() { } /** * Get the name property: Hostname. - * + * * @return the name value. */ public String name() { @@ -44,7 +48,7 @@ public String name() { /** * Set the name property: Hostname. - * + * * @param name the name value to set. * @return the CustomDomain object itself. */ @@ -55,7 +59,7 @@ public CustomDomain withName(String name) { /** * Get the bindingType property: Custom Domain binding type. - * + * * @return the bindingType value. */ public BindingType bindingType() { @@ -64,7 +68,7 @@ public BindingType bindingType() { /** * Set the bindingType property: Custom Domain binding type. - * + * * @param bindingType the bindingType value to set. * @return the CustomDomain object itself. */ @@ -76,7 +80,7 @@ public CustomDomain withBindingType(BindingType bindingType) { /** * Get the certificateId property: Resource Id of the Certificate to be bound to this hostname. Must exist in the * Managed Environment. - * + * * @return the certificateId value. */ public String certificateId() { @@ -86,7 +90,7 @@ public String certificateId() { /** * Set the certificateId property: Resource Id of the Certificate to be bound to this hostname. Must exist in the * Managed Environment. - * + * * @param certificateId the certificateId value to set. * @return the CustomDomain object itself. */ @@ -97,14 +101,13 @@ public CustomDomain withCertificateId(String certificateId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model CustomDomain")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model CustomDomain")); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomainConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomainConfiguration.java index f6d0f7c4eae57..ba53b980ff5e2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomainConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomDomainConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Configuration properties for apps environment custom domain. */ +/** + * Configuration properties for apps environment custom domain. + */ @Fluent public final class CustomDomainConfiguration { /* @@ -24,6 +26,12 @@ public final class CustomDomainConfiguration { @JsonProperty(value = "dnsSuffix") private String dnsSuffix; + /* + * Certificate stored in Azure Key Vault. + */ + @JsonProperty(value = "certificateKeyVaultProperties") + private CertificateKeyVaultProperties certificateKeyVaultProperties; + /* * PFX or PEM blob */ @@ -54,13 +62,15 @@ public final class CustomDomainConfiguration { @JsonProperty(value = "subjectName", access = JsonProperty.Access.WRITE_ONLY) private String subjectName; - /** Creates an instance of CustomDomainConfiguration class. */ + /** + * Creates an instance of CustomDomainConfiguration class. + */ public CustomDomainConfiguration() { } /** * Get the customDomainVerificationId property: Id used to verify domain name ownership. - * + * * @return the customDomainVerificationId value. */ public String customDomainVerificationId() { @@ -69,7 +79,7 @@ public String customDomainVerificationId() { /** * Get the dnsSuffix property: Dns suffix for the environment domain. - * + * * @return the dnsSuffix value. */ public String dnsSuffix() { @@ -78,7 +88,7 @@ public String dnsSuffix() { /** * Set the dnsSuffix property: Dns suffix for the environment domain. - * + * * @param dnsSuffix the dnsSuffix value to set. * @return the CustomDomainConfiguration object itself. */ @@ -87,9 +97,30 @@ public CustomDomainConfiguration withDnsSuffix(String dnsSuffix) { return this; } + /** + * Get the certificateKeyVaultProperties property: Certificate stored in Azure Key Vault. + * + * @return the certificateKeyVaultProperties value. + */ + public CertificateKeyVaultProperties certificateKeyVaultProperties() { + return this.certificateKeyVaultProperties; + } + + /** + * Set the certificateKeyVaultProperties property: Certificate stored in Azure Key Vault. + * + * @param certificateKeyVaultProperties the certificateKeyVaultProperties value to set. + * @return the CustomDomainConfiguration object itself. + */ + public CustomDomainConfiguration + withCertificateKeyVaultProperties(CertificateKeyVaultProperties certificateKeyVaultProperties) { + this.certificateKeyVaultProperties = certificateKeyVaultProperties; + return this; + } + /** * Get the certificateValue property: PFX or PEM blob. - * + * * @return the certificateValue value. */ public byte[] certificateValue() { @@ -98,7 +129,7 @@ public byte[] certificateValue() { /** * Set the certificateValue property: PFX or PEM blob. - * + * * @param certificateValue the certificateValue value to set. * @return the CustomDomainConfiguration object itself. */ @@ -109,7 +140,7 @@ public CustomDomainConfiguration withCertificateValue(byte[] certificateValue) { /** * Get the certificatePassword property: Certificate password. - * + * * @return the certificatePassword value. */ public String certificatePassword() { @@ -118,7 +149,7 @@ public String certificatePassword() { /** * Set the certificatePassword property: Certificate password. - * + * * @param certificatePassword the certificatePassword value to set. * @return the CustomDomainConfiguration object itself. */ @@ -129,7 +160,7 @@ public CustomDomainConfiguration withCertificatePassword(String certificatePassw /** * Get the expirationDate property: Certificate expiration date. - * + * * @return the expirationDate value. */ public OffsetDateTime expirationDate() { @@ -138,7 +169,7 @@ public OffsetDateTime expirationDate() { /** * Get the thumbprint property: Certificate thumbprint. - * + * * @return the thumbprint value. */ public String thumbprint() { @@ -147,7 +178,7 @@ public String thumbprint() { /** * Get the subjectName property: Subject name of the certificate. - * + * * @return the subjectName value. */ public String subjectName() { @@ -156,9 +187,12 @@ public String subjectName() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (certificateKeyVaultProperties() != null) { + certificateKeyVaultProperties().validate(); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResult.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResult.java index e80ffbb899cfc..27c48d9aa5974 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResult.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResult.java @@ -7,11 +7,13 @@ import com.azure.resourcemanager.appcontainers.fluent.models.CustomHostnameAnalysisResultInner; import java.util.List; -/** An immutable client-side representation of CustomHostnameAnalysisResult. */ +/** + * An immutable client-side representation of CustomHostnameAnalysisResult. + */ public interface CustomHostnameAnalysisResult { /** * Gets the hostname property: Host name that was analyzed. - * + * * @return the hostname value. */ String hostname(); @@ -19,21 +21,21 @@ public interface CustomHostnameAnalysisResult { /** * Gets the isHostnameAlreadyVerified property: <code>true</code> if hostname is already verified; * otherwise, <code>false</code>. - * + * * @return the isHostnameAlreadyVerified value. */ Boolean isHostnameAlreadyVerified(); /** * Gets the customDomainVerificationTest property: DNS verification test result. - * + * * @return the customDomainVerificationTest value. */ DnsVerificationTestResult customDomainVerificationTest(); /** * Gets the customDomainVerificationFailureInfo property: Raw failure information if DNS verification fails. - * + * * @return the customDomainVerificationFailureInfo value. */ CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo customDomainVerificationFailureInfo(); @@ -41,7 +43,7 @@ public interface CustomHostnameAnalysisResult { /** * Gets the hasConflictOnManagedEnvironment property: <code>true</code> if there is a conflict on the * Container App's managed environment; otherwise, <code>false</code>. - * + * * @return the hasConflictOnManagedEnvironment value. */ Boolean hasConflictOnManagedEnvironment(); @@ -49,7 +51,7 @@ public interface CustomHostnameAnalysisResult { /** * Gets the conflictWithEnvironmentCustomDomain property: <code>true</code> if there is a conflict on * the Container App's managed environment level custom domain; otherwise, <code>false</code>. - * + * * @return the conflictWithEnvironmentCustomDomain value. */ Boolean conflictWithEnvironmentCustomDomain(); @@ -57,49 +59,49 @@ public interface CustomHostnameAnalysisResult { /** * Gets the conflictingContainerAppResourceId property: Name of the conflicting Container App on the Managed * Environment if it's within the same subscription. - * + * * @return the conflictingContainerAppResourceId value. */ String conflictingContainerAppResourceId(); /** * Gets the cNameRecords property: CName records visible for this hostname. - * + * * @return the cNameRecords value. */ List cNameRecords(); /** * Gets the txtRecords property: TXT records visible for this hostname. - * + * * @return the txtRecords value. */ List txtRecords(); /** * Gets the aRecords property: A records visible for this hostname. - * + * * @return the aRecords value. */ List aRecords(); /** * Gets the alternateCNameRecords property: Alternate CName records visible for this hostname. - * + * * @return the alternateCNameRecords value. */ List alternateCNameRecords(); /** * Gets the alternateTxtRecords property: Alternate TXT records visible for this hostname. - * + * * @return the alternateTxtRecords value. */ List alternateTxtRecords(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.CustomHostnameAnalysisResultInner object. - * + * * @return the inner object. */ CustomHostnameAnalysisResultInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo.java index 981f2943541f2..45529ba9c4b53 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Raw failure information if DNS verification fails. */ +/** + * Raw failure information if DNS verification fails. + */ @Fluent public final class CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo { /* @@ -35,13 +37,15 @@ public final class CustomHostnameAnalysisResultCustomDomainVerificationFailureIn @JsonProperty(value = "details") private List details; - /** Creates an instance of CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo class. */ + /** + * Creates an instance of CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo class. + */ public CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo() { } /** * Get the code property: Standardized string to programmatically identify the error. - * + * * @return the code value. */ public String code() { @@ -50,7 +54,7 @@ public String code() { /** * Get the message property: Detailed error description and debugging information. - * + * * @return the message value. */ public String message() { @@ -59,7 +63,7 @@ public String message() { /** * Get the target property: Detailed error description and debugging information. - * + * * @return the target value. */ public String target() { @@ -68,7 +72,7 @@ public String target() { /** * Get the details property: Details or the error. - * + * * @return the details value. */ public List details() { @@ -77,19 +81,19 @@ public List details) { + public CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo + withDetails(List details) { this.details = details; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem.java index 8324c64c41afd..1f62e74e378d9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Detailed errors. */ +/** + * Detailed errors. + */ @Immutable public final class CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem { /* @@ -28,13 +30,15 @@ public final class CustomHostnameAnalysisResultCustomDomainVerificationFailureIn @JsonProperty(value = "target", access = JsonProperty.Access.WRITE_ONLY) private String target; - /** Creates an instance of CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem class. */ + /** + * Creates an instance of CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem class. + */ public CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem() { } /** * Get the code property: Standardized string to programmatically identify the error. - * + * * @return the code value. */ public String code() { @@ -43,7 +47,7 @@ public String code() { /** * Get the message property: Detailed error description and debugging information. - * + * * @return the message value. */ public String message() { @@ -52,7 +56,7 @@ public String message() { /** * Get the target property: Detailed error description and debugging information. - * + * * @return the target value. */ public String target() { @@ -61,7 +65,7 @@ public String target() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomOpenIdConnectProvider.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomOpenIdConnectProvider.java index 12afd42c402df..042d48d132332 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomOpenIdConnectProvider.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomOpenIdConnectProvider.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the custom Open ID Connect provider. */ +/** + * The configuration settings of the custom Open ID Connect provider. + */ @Fluent public final class CustomOpenIdConnectProvider { /* @@ -28,14 +30,16 @@ public final class CustomOpenIdConnectProvider { @JsonProperty(value = "login") private OpenIdConnectLogin login; - /** Creates an instance of CustomOpenIdConnectProvider class. */ + /** + * Creates an instance of CustomOpenIdConnectProvider class. + */ public CustomOpenIdConnectProvider() { } /** * Get the enabled property: <code>false</code> if the custom Open ID provider provider should not be * enabled; otherwise, <code>true</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -45,7 +49,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>false</code> if the custom Open ID provider provider should not be * enabled; otherwise, <code>true</code>. - * + * * @param enabled the enabled value to set. * @return the CustomOpenIdConnectProvider object itself. */ @@ -57,7 +61,7 @@ public CustomOpenIdConnectProvider withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the app registration for the custom Open ID Connect * provider. - * + * * @return the registration value. */ public OpenIdConnectRegistration registration() { @@ -67,7 +71,7 @@ public OpenIdConnectRegistration registration() { /** * Set the registration property: The configuration settings of the app registration for the custom Open ID Connect * provider. - * + * * @param registration the registration value to set. * @return the CustomOpenIdConnectProvider object itself. */ @@ -78,7 +82,7 @@ public CustomOpenIdConnectProvider withRegistration(OpenIdConnectRegistration re /** * Get the login property: The configuration settings of the login flow of the custom Open ID Connect provider. - * + * * @return the login value. */ public OpenIdConnectLogin login() { @@ -87,7 +91,7 @@ public OpenIdConnectLogin login() { /** * Set the login property: The configuration settings of the login flow of the custom Open ID Connect provider. - * + * * @param login the login value to set. * @return the CustomOpenIdConnectProvider object itself. */ @@ -98,7 +102,7 @@ public CustomOpenIdConnectProvider withLogin(OpenIdConnectLogin login) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomScaleRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomScaleRule.java index a3adaf102f250..c5b307384859a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomScaleRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/CustomScaleRule.java @@ -10,7 +10,9 @@ import java.util.List; import java.util.Map; -/** Container App container Custom scaling rule. */ +/** + * Container App container Custom scaling rule. + */ @Fluent public final class CustomScaleRule { /* @@ -33,13 +35,16 @@ public final class CustomScaleRule { @JsonProperty(value = "auth") private List auth; - /** Creates an instance of CustomScaleRule class. */ + /** + * Creates an instance of CustomScaleRule class. + */ public CustomScaleRule() { } /** - * Get the type property: Type of the custom scale rule eg: azure-servicebus, redis etc. - * + * Get the type property: Type of the custom scale rule + * eg: azure-servicebus, redis etc. + * * @return the type value. */ public String type() { @@ -47,8 +52,9 @@ public String type() { } /** - * Set the type property: Type of the custom scale rule eg: azure-servicebus, redis etc. - * + * Set the type property: Type of the custom scale rule + * eg: azure-servicebus, redis etc. + * * @param type the type value to set. * @return the CustomScaleRule object itself. */ @@ -59,7 +65,7 @@ public CustomScaleRule withType(String type) { /** * Get the metadata property: Metadata properties to describe custom scale rule. - * + * * @return the metadata value. */ public Map metadata() { @@ -68,7 +74,7 @@ public Map metadata() { /** * Set the metadata property: Metadata properties to describe custom scale rule. - * + * * @param metadata the metadata value to set. * @return the CustomScaleRule object itself. */ @@ -79,7 +85,7 @@ public CustomScaleRule withMetadata(Map metadata) { /** * Get the auth property: Authentication secrets for the custom scale rule. - * + * * @return the auth value. */ public List auth() { @@ -88,7 +94,7 @@ public List auth() { /** * Set the auth property: Authentication secrets for the custom scale rule. - * + * * @param auth the auth value to set. * @return the CustomScaleRule object itself. */ @@ -99,7 +105,7 @@ public CustomScaleRule withAuth(List auth) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Dapr.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Dapr.java index 9d378cd0ee739..bd279c80bfc67 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Dapr.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Dapr.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App Dapr configuration. */ +/** + * Container App Dapr configuration. + */ @Fluent public final class Dapr { /* @@ -59,13 +61,15 @@ public final class Dapr { @JsonProperty(value = "enableApiLogging") private Boolean enableApiLogging; - /** Creates an instance of Dapr class. */ + /** + * Creates an instance of Dapr class. + */ public Dapr() { } /** * Get the enabled property: Boolean indicating if the Dapr side car is enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -74,7 +78,7 @@ public Boolean enabled() { /** * Set the enabled property: Boolean indicating if the Dapr side car is enabled. - * + * * @param enabled the enabled value to set. * @return the Dapr object itself. */ @@ -85,7 +89,7 @@ public Dapr withEnabled(Boolean enabled) { /** * Get the appId property: Dapr application identifier. - * + * * @return the appId value. */ public String appId() { @@ -94,7 +98,7 @@ public String appId() { /** * Set the appId property: Dapr application identifier. - * + * * @param appId the appId value to set. * @return the Dapr object itself. */ @@ -106,7 +110,7 @@ public Dapr withAppId(String appId) { /** * Get the appProtocol property: Tells Dapr which protocol your application is using. Valid options are http and * grpc. Default is http. - * + * * @return the appProtocol value. */ public AppProtocol appProtocol() { @@ -116,7 +120,7 @@ public AppProtocol appProtocol() { /** * Set the appProtocol property: Tells Dapr which protocol your application is using. Valid options are http and * grpc. Default is http. - * + * * @param appProtocol the appProtocol value to set. * @return the Dapr object itself. */ @@ -127,7 +131,7 @@ public Dapr withAppProtocol(AppProtocol appProtocol) { /** * Get the appPort property: Tells Dapr which port your application is listening on. - * + * * @return the appPort value. */ public Integer appPort() { @@ -136,7 +140,7 @@ public Integer appPort() { /** * Set the appPort property: Tells Dapr which port your application is listening on. - * + * * @param appPort the appPort value to set. * @return the Dapr object itself. */ @@ -148,7 +152,7 @@ public Dapr withAppPort(Integer appPort) { /** * Get the httpReadBufferSize property: Dapr max size of http header read buffer in KB to handle when sending * multi-KB headers. Default is 65KB. - * + * * @return the httpReadBufferSize value. */ public Integer httpReadBufferSize() { @@ -158,7 +162,7 @@ public Integer httpReadBufferSize() { /** * Set the httpReadBufferSize property: Dapr max size of http header read buffer in KB to handle when sending * multi-KB headers. Default is 65KB. - * + * * @param httpReadBufferSize the httpReadBufferSize value to set. * @return the Dapr object itself. */ @@ -168,9 +172,9 @@ public Dapr withHttpReadBufferSize(Integer httpReadBufferSize) { } /** - * Get the httpMaxRequestSize property: Increasing max size of request body http and grpc servers parameter in MB to - * handle uploading of big files. Default is 4 MB. - * + * Get the httpMaxRequestSize property: Increasing max size of request body http and grpc servers parameter in MB + * to handle uploading of big files. Default is 4 MB. + * * @return the httpMaxRequestSize value. */ public Integer httpMaxRequestSize() { @@ -178,9 +182,9 @@ public Integer httpMaxRequestSize() { } /** - * Set the httpMaxRequestSize property: Increasing max size of request body http and grpc servers parameter in MB to - * handle uploading of big files. Default is 4 MB. - * + * Set the httpMaxRequestSize property: Increasing max size of request body http and grpc servers parameter in MB + * to handle uploading of big files. Default is 4 MB. + * * @param httpMaxRequestSize the httpMaxRequestSize value to set. * @return the Dapr object itself. */ @@ -192,7 +196,7 @@ public Dapr withHttpMaxRequestSize(Integer httpMaxRequestSize) { /** * Get the logLevel property: Sets the log level for the Dapr sidecar. Allowed values are debug, info, warn, error. * Default is info. - * + * * @return the logLevel value. */ public LogLevel logLevel() { @@ -202,7 +206,7 @@ public LogLevel logLevel() { /** * Set the logLevel property: Sets the log level for the Dapr sidecar. Allowed values are debug, info, warn, error. * Default is info. - * + * * @param logLevel the logLevel value to set. * @return the Dapr object itself. */ @@ -213,7 +217,7 @@ public Dapr withLogLevel(LogLevel logLevel) { /** * Get the enableApiLogging property: Enables API logging for the Dapr sidecar. - * + * * @return the enableApiLogging value. */ public Boolean enableApiLogging() { @@ -222,7 +226,7 @@ public Boolean enableApiLogging() { /** * Set the enableApiLogging property: Enables API logging for the Dapr sidecar. - * + * * @param enableApiLogging the enableApiLogging value to set. * @return the Dapr object itself. */ @@ -233,7 +237,7 @@ public Dapr withEnableApiLogging(Boolean enableApiLogging) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponent.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponent.java index 5b0052ba06f72..dc79196c99796 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponent.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponent.java @@ -5,119 +5,144 @@ package com.azure.resourcemanager.appcontainers.models; import com.azure.core.http.rest.Response; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner; import java.util.List; -/** An immutable client-side representation of DaprComponent. */ +/** + * An immutable client-side representation of DaprComponent. + */ public interface DaprComponent { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the componentType property: Component type. - * + * * @return the componentType value. */ String componentType(); /** * Gets the version property: Component version. - * + * * @return the version value. */ String version(); /** * Gets the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @return the ignoreErrors value. */ Boolean ignoreErrors(); /** * Gets the initTimeout property: Initialization timeout. - * + * * @return the initTimeout value. */ String initTimeout(); /** * Gets the secrets property: Collection of secrets used by a Dapr component. - * + * * @return the secrets value. */ List secrets(); /** * Gets the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @return the secretStoreComponent value. */ String secretStoreComponent(); /** * Gets the metadata property: Component metadata. - * + * * @return the metadata value. */ List metadata(); /** * Gets the scopes property: Names of container apps that can use this Dapr component. - * + * * @return the scopes value. */ List scopes(); + /** + * Gets the serviceComponentBind property: List of container app services that are bound to the Dapr component. + * + * @return the serviceComponentBind value. + */ + List serviceComponentBind(); + /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner object. - * + * * @return the inner object. */ DaprComponentInner innerModel(); - /** The entirety of the DaprComponent definition. */ + /** + * The entirety of the DaprComponent definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The DaprComponent definition stages. */ + /** + * The DaprComponent definition stages. + */ interface DefinitionStages { - /** The first stage of the DaprComponent definition. */ + /** + * The first stage of the DaprComponent definition. + */ interface Blank extends WithParentResource { } - /** The stage of the DaprComponent definition allowing to specify parent resource. */ + /** + * The stage of the DaprComponent definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, connectedEnvironmentName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param connectedEnvironmentName Name of the connected environment. * @return the next definition stage. @@ -130,253 +155,306 @@ interface WithParentResource { * to be created, but also allows for any other optional properties to be specified. */ interface WithCreate - extends DefinitionStages.WithComponentType, - DefinitionStages.WithVersion, - DefinitionStages.WithIgnoreErrors, - DefinitionStages.WithInitTimeout, - DefinitionStages.WithSecrets, - DefinitionStages.WithSecretStoreComponent, - DefinitionStages.WithMetadata, - DefinitionStages.WithScopes { + extends DefinitionStages.WithComponentType, DefinitionStages.WithVersion, DefinitionStages.WithIgnoreErrors, + DefinitionStages.WithInitTimeout, DefinitionStages.WithSecrets, DefinitionStages.WithSecretStoreComponent, + DefinitionStages.WithMetadata, DefinitionStages.WithScopes, DefinitionStages.WithServiceComponentBind { /** * Executes the create request. - * + * * @return the created resource. */ DaprComponent create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ DaprComponent create(Context context); } - /** The stage of the DaprComponent definition allowing to specify componentType. */ + /** + * The stage of the DaprComponent definition allowing to specify componentType. + */ interface WithComponentType { /** * Specifies the componentType property: Component type. - * + * * @param componentType Component type. * @return the next definition stage. */ WithCreate withComponentType(String componentType); } - /** The stage of the DaprComponent definition allowing to specify version. */ + /** + * The stage of the DaprComponent definition allowing to specify version. + */ interface WithVersion { /** * Specifies the version property: Component version. - * + * * @param version Component version. * @return the next definition stage. */ WithCreate withVersion(String version); } - /** The stage of the DaprComponent definition allowing to specify ignoreErrors. */ + /** + * The stage of the DaprComponent definition allowing to specify ignoreErrors. + */ interface WithIgnoreErrors { /** * Specifies the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @param ignoreErrors Boolean describing if the component errors are ignores. * @return the next definition stage. */ WithCreate withIgnoreErrors(Boolean ignoreErrors); } - /** The stage of the DaprComponent definition allowing to specify initTimeout. */ + /** + * The stage of the DaprComponent definition allowing to specify initTimeout. + */ interface WithInitTimeout { /** * Specifies the initTimeout property: Initialization timeout. - * + * * @param initTimeout Initialization timeout. * @return the next definition stage. */ WithCreate withInitTimeout(String initTimeout); } - /** The stage of the DaprComponent definition allowing to specify secrets. */ + /** + * The stage of the DaprComponent definition allowing to specify secrets. + */ interface WithSecrets { /** * Specifies the secrets property: Collection of secrets used by a Dapr component. - * + * * @param secrets Collection of secrets used by a Dapr component. * @return the next definition stage. */ WithCreate withSecrets(List secrets); } - /** The stage of the DaprComponent definition allowing to specify secretStoreComponent. */ + /** + * The stage of the DaprComponent definition allowing to specify secretStoreComponent. + */ interface WithSecretStoreComponent { /** * Specifies the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @param secretStoreComponent Name of a Dapr component to retrieve component secrets from. * @return the next definition stage. */ WithCreate withSecretStoreComponent(String secretStoreComponent); } - /** The stage of the DaprComponent definition allowing to specify metadata. */ + /** + * The stage of the DaprComponent definition allowing to specify metadata. + */ interface WithMetadata { /** * Specifies the metadata property: Component metadata. - * + * * @param metadata Component metadata. * @return the next definition stage. */ WithCreate withMetadata(List metadata); } - /** The stage of the DaprComponent definition allowing to specify scopes. */ + /** + * The stage of the DaprComponent definition allowing to specify scopes. + */ interface WithScopes { /** * Specifies the scopes property: Names of container apps that can use this Dapr component. - * + * * @param scopes Names of container apps that can use this Dapr component. * @return the next definition stage. */ WithCreate withScopes(List scopes); } + + /** + * The stage of the DaprComponent definition allowing to specify serviceComponentBind. + */ + interface WithServiceComponentBind { + /** + * Specifies the serviceComponentBind property: List of container app services that are bound to the Dapr + * component. + * + * @param serviceComponentBind List of container app services that are bound to the Dapr component. + * @return the next definition stage. + */ + WithCreate withServiceComponentBind(List serviceComponentBind); + } } /** * Begins update for the DaprComponent resource. - * + * * @return the stage of resource update. */ DaprComponent.Update update(); - /** The template for DaprComponent update. */ - interface Update - extends UpdateStages.WithComponentType, - UpdateStages.WithVersion, - UpdateStages.WithIgnoreErrors, - UpdateStages.WithInitTimeout, - UpdateStages.WithSecrets, - UpdateStages.WithSecretStoreComponent, - UpdateStages.WithMetadata, - UpdateStages.WithScopes { + /** + * The template for DaprComponent update. + */ + interface Update extends UpdateStages.WithComponentType, UpdateStages.WithVersion, UpdateStages.WithIgnoreErrors, + UpdateStages.WithInitTimeout, UpdateStages.WithSecrets, UpdateStages.WithSecretStoreComponent, + UpdateStages.WithMetadata, UpdateStages.WithScopes, UpdateStages.WithServiceComponentBind { /** * Executes the update request. - * + * * @return the updated resource. */ DaprComponent apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ DaprComponent apply(Context context); } - /** The DaprComponent update stages. */ + /** + * The DaprComponent update stages. + */ interface UpdateStages { - /** The stage of the DaprComponent update allowing to specify componentType. */ + /** + * The stage of the DaprComponent update allowing to specify componentType. + */ interface WithComponentType { /** * Specifies the componentType property: Component type. - * + * * @param componentType Component type. * @return the next definition stage. */ Update withComponentType(String componentType); } - /** The stage of the DaprComponent update allowing to specify version. */ + /** + * The stage of the DaprComponent update allowing to specify version. + */ interface WithVersion { /** * Specifies the version property: Component version. - * + * * @param version Component version. * @return the next definition stage. */ Update withVersion(String version); } - /** The stage of the DaprComponent update allowing to specify ignoreErrors. */ + /** + * The stage of the DaprComponent update allowing to specify ignoreErrors. + */ interface WithIgnoreErrors { /** * Specifies the ignoreErrors property: Boolean describing if the component errors are ignores. - * + * * @param ignoreErrors Boolean describing if the component errors are ignores. * @return the next definition stage. */ Update withIgnoreErrors(Boolean ignoreErrors); } - /** The stage of the DaprComponent update allowing to specify initTimeout. */ + /** + * The stage of the DaprComponent update allowing to specify initTimeout. + */ interface WithInitTimeout { /** * Specifies the initTimeout property: Initialization timeout. - * + * * @param initTimeout Initialization timeout. * @return the next definition stage. */ Update withInitTimeout(String initTimeout); } - /** The stage of the DaprComponent update allowing to specify secrets. */ + /** + * The stage of the DaprComponent update allowing to specify secrets. + */ interface WithSecrets { /** * Specifies the secrets property: Collection of secrets used by a Dapr component. - * + * * @param secrets Collection of secrets used by a Dapr component. * @return the next definition stage. */ Update withSecrets(List secrets); } - /** The stage of the DaprComponent update allowing to specify secretStoreComponent. */ + /** + * The stage of the DaprComponent update allowing to specify secretStoreComponent. + */ interface WithSecretStoreComponent { /** * Specifies the secretStoreComponent property: Name of a Dapr component to retrieve component secrets from. - * + * * @param secretStoreComponent Name of a Dapr component to retrieve component secrets from. * @return the next definition stage. */ Update withSecretStoreComponent(String secretStoreComponent); } - /** The stage of the DaprComponent update allowing to specify metadata. */ + /** + * The stage of the DaprComponent update allowing to specify metadata. + */ interface WithMetadata { /** * Specifies the metadata property: Component metadata. - * + * * @param metadata Component metadata. * @return the next definition stage. */ Update withMetadata(List metadata); } - /** The stage of the DaprComponent update allowing to specify scopes. */ + /** + * The stage of the DaprComponent update allowing to specify scopes. + */ interface WithScopes { /** * Specifies the scopes property: Names of container apps that can use this Dapr component. - * + * * @param scopes Names of container apps that can use this Dapr component. * @return the next definition stage. */ Update withScopes(List scopes); } + + /** + * The stage of the DaprComponent update allowing to specify serviceComponentBind. + */ + interface WithServiceComponentBind { + /** + * Specifies the serviceComponentBind property: List of container app services that are bound to the Dapr + * component. + * + * @param serviceComponentBind List of container app services that are bound to the Dapr component. + * @return the next definition stage. + */ + Update withServiceComponentBind(List serviceComponentBind); + } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ DaprComponent refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -384,11 +462,11 @@ interface WithScopes { /** * List secrets for a dapr component. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ @@ -396,9 +474,9 @@ interface WithScopes { /** * List secrets for a dapr component. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicies.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicies.java new file mode 100644 index 0000000000000..5658db8005ade --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicies.java @@ -0,0 +1,174 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of DaprComponentResiliencyPolicies. + */ +public interface DaprComponentResiliencyPolicies { + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String environmentName, + String componentName); + + /** + * Get the resiliency policies for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the resiliency policies for a Dapr component as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String environmentName, + String componentName, Context context); + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String environmentName, + String componentName, String name, Context context); + + /** + * Get a Dapr component resiliency policy. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy. + */ + DaprComponentResiliencyPolicy get(String resourceGroupName, String environmentName, String componentName, + String name); + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + String name, Context context); + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @param name Name of the Dapr Component Resiliency Policy. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String environmentName, String componentName, String name); + + /** + * Get a Dapr component resiliency policy. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response}. + */ + DaprComponentResiliencyPolicy getById(String id); + + /** + * Get a Dapr component resiliency policy. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Dapr component resiliency policy along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteById(String id); + + /** + * Delete a Dapr component resiliency policy. + * + * Delete a resiliency policy for a Dapr component. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new DaprComponentResiliencyPolicy resource. + * + * @param name resource name. + * @return the first stage of the new DaprComponentResiliencyPolicy definition. + */ + DaprComponentResiliencyPolicy.DefinitionStages.Blank define(String name); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPoliciesCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPoliciesCollection.java new file mode 100644 index 0000000000000..d2cd3641ede9b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPoliciesCollection.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Dapr Component Resiliency Policies ARM resource. + */ +@Fluent +public final class DaprComponentResiliencyPoliciesCollection { + /* + * Collection of resources. + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * Link to next page of resources. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of DaprComponentResiliencyPoliciesCollection class. + */ + public DaprComponentResiliencyPoliciesCollection() { + } + + /** + * Get the value property: Collection of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Collection of resources. + * + * @param value the value value to set. + * @return the DaprComponentResiliencyPoliciesCollection object itself. + */ + public DaprComponentResiliencyPoliciesCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property value in model DaprComponentResiliencyPoliciesCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(DaprComponentResiliencyPoliciesCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicy.java new file mode 100644 index 0000000000000..a8322cfe73ff2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicy.java @@ -0,0 +1,224 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; + +/** + * An immutable client-side representation of DaprComponentResiliencyPolicy. + */ +public interface DaprComponentResiliencyPolicy { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @return the inboundPolicy value. + */ + DaprComponentResiliencyPolicyConfiguration inboundPolicy(); + + /** + * Gets the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @return the outboundPolicy value. + */ + DaprComponentResiliencyPolicyConfiguration outboundPolicy(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner object. + * + * @return the inner object. + */ + DaprComponentResiliencyPolicyInner innerModel(); + + /** + * The entirety of the DaprComponentResiliencyPolicy definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The DaprComponentResiliencyPolicy definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the DaprComponentResiliencyPolicy definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the DaprComponentResiliencyPolicy definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, environmentName, componentName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param componentName Name of the Dapr Component. + * @return the next definition stage. + */ + WithCreate withExistingDaprComponent(String resourceGroupName, String environmentName, + String componentName); + } + + /** + * The stage of the DaprComponentResiliencyPolicy definition which contains all the minimum required properties + * for the resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithInboundPolicy, DefinitionStages.WithOutboundPolicy { + /** + * Executes the create request. + * + * @return the created resource. + */ + DaprComponentResiliencyPolicy create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + DaprComponentResiliencyPolicy create(Context context); + } + + /** + * The stage of the DaprComponentResiliencyPolicy definition allowing to specify inboundPolicy. + */ + interface WithInboundPolicy { + /** + * Specifies the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @param inboundPolicy The optional inbound component resiliency policy configuration. + * @return the next definition stage. + */ + WithCreate withInboundPolicy(DaprComponentResiliencyPolicyConfiguration inboundPolicy); + } + + /** + * The stage of the DaprComponentResiliencyPolicy definition allowing to specify outboundPolicy. + */ + interface WithOutboundPolicy { + /** + * Specifies the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @param outboundPolicy The optional outbound component resiliency policy configuration. + * @return the next definition stage. + */ + WithCreate withOutboundPolicy(DaprComponentResiliencyPolicyConfiguration outboundPolicy); + } + } + + /** + * Begins update for the DaprComponentResiliencyPolicy resource. + * + * @return the stage of resource update. + */ + DaprComponentResiliencyPolicy.Update update(); + + /** + * The template for DaprComponentResiliencyPolicy update. + */ + interface Update extends UpdateStages.WithInboundPolicy, UpdateStages.WithOutboundPolicy { + /** + * Executes the update request. + * + * @return the updated resource. + */ + DaprComponentResiliencyPolicy apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + DaprComponentResiliencyPolicy apply(Context context); + } + + /** + * The DaprComponentResiliencyPolicy update stages. + */ + interface UpdateStages { + /** + * The stage of the DaprComponentResiliencyPolicy update allowing to specify inboundPolicy. + */ + interface WithInboundPolicy { + /** + * Specifies the inboundPolicy property: The optional inbound component resiliency policy configuration. + * + * @param inboundPolicy The optional inbound component resiliency policy configuration. + * @return the next definition stage. + */ + Update withInboundPolicy(DaprComponentResiliencyPolicyConfiguration inboundPolicy); + } + + /** + * The stage of the DaprComponentResiliencyPolicy update allowing to specify outboundPolicy. + */ + interface WithOutboundPolicy { + /** + * Specifies the outboundPolicy property: The optional outbound component resiliency policy configuration. + * + * @param outboundPolicy The optional outbound component resiliency policy configuration. + * @return the next definition stage. + */ + Update withOutboundPolicy(DaprComponentResiliencyPolicyConfiguration outboundPolicy); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + DaprComponentResiliencyPolicy refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + DaprComponentResiliencyPolicy refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration.java new file mode 100644 index 0000000000000..00acaa9ac2ced --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration.java @@ -0,0 +1,114 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy Circuit Breaker Policy Configuration. + */ +@Fluent +public final class DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration { + /* + * The number of consecutive errors before the circuit is opened. + */ + @JsonProperty(value = "consecutiveErrors") + private Integer consecutiveErrors; + + /* + * The interval in seconds until a retry attempt is made after the circuit is opened. + */ + @JsonProperty(value = "timeoutInSeconds") + private Integer timeoutInSeconds; + + /* + * The optional interval in seconds after which the error count resets to 0. An interval of 0 will never reset. If + * not specified, the timeoutInSeconds value will be used. + */ + @JsonProperty(value = "intervalInSeconds") + private Integer intervalInSeconds; + + /** + * Creates an instance of DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration class. + */ + public DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() { + } + + /** + * Get the consecutiveErrors property: The number of consecutive errors before the circuit is opened. + * + * @return the consecutiveErrors value. + */ + public Integer consecutiveErrors() { + return this.consecutiveErrors; + } + + /** + * Set the consecutiveErrors property: The number of consecutive errors before the circuit is opened. + * + * @param consecutiveErrors the consecutiveErrors value to set. + * @return the DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration + withConsecutiveErrors(Integer consecutiveErrors) { + this.consecutiveErrors = consecutiveErrors; + return this; + } + + /** + * Get the timeoutInSeconds property: The interval in seconds until a retry attempt is made after the circuit is + * opened. + * + * @return the timeoutInSeconds value. + */ + public Integer timeoutInSeconds() { + return this.timeoutInSeconds; + } + + /** + * Set the timeoutInSeconds property: The interval in seconds until a retry attempt is made after the circuit is + * opened. + * + * @param timeoutInSeconds the timeoutInSeconds value to set. + * @return the DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration + withTimeoutInSeconds(Integer timeoutInSeconds) { + this.timeoutInSeconds = timeoutInSeconds; + return this; + } + + /** + * Get the intervalInSeconds property: The optional interval in seconds after which the error count resets to 0. An + * interval of 0 will never reset. If not specified, the timeoutInSeconds value will be used. + * + * @return the intervalInSeconds value. + */ + public Integer intervalInSeconds() { + return this.intervalInSeconds; + } + + /** + * Set the intervalInSeconds property: The optional interval in seconds after which the error count resets to 0. An + * interval of 0 will never reset. If not specified, the timeoutInSeconds value will be used. + * + * @param intervalInSeconds the intervalInSeconds value to set. + * @return the DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration + withIntervalInSeconds(Integer intervalInSeconds) { + this.intervalInSeconds = intervalInSeconds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyConfiguration.java new file mode 100644 index 0000000000000..de2ba311504e2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyConfiguration.java @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy Configuration. + */ +@Fluent +public final class DaprComponentResiliencyPolicyConfiguration { + /* + * The optional HTTP retry policy configuration + */ + @JsonProperty(value = "httpRetryPolicy") + private DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration httpRetryPolicy; + + /* + * The optional timeout policy configuration + */ + @JsonProperty(value = "timeoutPolicy") + private DaprComponentResiliencyPolicyTimeoutPolicyConfiguration timeoutPolicy; + + /* + * The optional circuit breaker policy configuration + */ + @JsonProperty(value = "circuitBreakerPolicy") + private DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration circuitBreakerPolicy; + + /** + * Creates an instance of DaprComponentResiliencyPolicyConfiguration class. + */ + public DaprComponentResiliencyPolicyConfiguration() { + } + + /** + * Get the httpRetryPolicy property: The optional HTTP retry policy configuration. + * + * @return the httpRetryPolicy value. + */ + public DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration httpRetryPolicy() { + return this.httpRetryPolicy; + } + + /** + * Set the httpRetryPolicy property: The optional HTTP retry policy configuration. + * + * @param httpRetryPolicy the httpRetryPolicy value to set. + * @return the DaprComponentResiliencyPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyConfiguration + withHttpRetryPolicy(DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration httpRetryPolicy) { + this.httpRetryPolicy = httpRetryPolicy; + return this; + } + + /** + * Get the timeoutPolicy property: The optional timeout policy configuration. + * + * @return the timeoutPolicy value. + */ + public DaprComponentResiliencyPolicyTimeoutPolicyConfiguration timeoutPolicy() { + return this.timeoutPolicy; + } + + /** + * Set the timeoutPolicy property: The optional timeout policy configuration. + * + * @param timeoutPolicy the timeoutPolicy value to set. + * @return the DaprComponentResiliencyPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyConfiguration + withTimeoutPolicy(DaprComponentResiliencyPolicyTimeoutPolicyConfiguration timeoutPolicy) { + this.timeoutPolicy = timeoutPolicy; + return this; + } + + /** + * Get the circuitBreakerPolicy property: The optional circuit breaker policy configuration. + * + * @return the circuitBreakerPolicy value. + */ + public DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration circuitBreakerPolicy() { + return this.circuitBreakerPolicy; + } + + /** + * Set the circuitBreakerPolicy property: The optional circuit breaker policy configuration. + * + * @param circuitBreakerPolicy the circuitBreakerPolicy value to set. + * @return the DaprComponentResiliencyPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyConfiguration + withCircuitBreakerPolicy(DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration circuitBreakerPolicy) { + this.circuitBreakerPolicy = circuitBreakerPolicy; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (httpRetryPolicy() != null) { + httpRetryPolicy().validate(); + } + if (timeoutPolicy() != null) { + timeoutPolicy().validate(); + } + if (circuitBreakerPolicy() != null) { + circuitBreakerPolicy().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration.java new file mode 100644 index 0000000000000..ade23b45f831f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy HTTP Retry Backoff Configuration. + */ +@Fluent +public final class DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration { + /* + * The optional initial delay in milliseconds before an operation is retried + */ + @JsonProperty(value = "initialDelayInMilliseconds") + private Integer initialDelayInMilliseconds; + + /* + * The optional maximum time interval in milliseconds between retry attempts + */ + @JsonProperty(value = "maxIntervalInMilliseconds") + private Integer maxIntervalInMilliseconds; + + /** + * Creates an instance of DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration class. + */ + public DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() { + } + + /** + * Get the initialDelayInMilliseconds property: The optional initial delay in milliseconds before an operation is + * retried. + * + * @return the initialDelayInMilliseconds value. + */ + public Integer initialDelayInMilliseconds() { + return this.initialDelayInMilliseconds; + } + + /** + * Set the initialDelayInMilliseconds property: The optional initial delay in milliseconds before an operation is + * retried. + * + * @param initialDelayInMilliseconds the initialDelayInMilliseconds value to set. + * @return the DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration object itself. + */ + public DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration + withInitialDelayInMilliseconds(Integer initialDelayInMilliseconds) { + this.initialDelayInMilliseconds = initialDelayInMilliseconds; + return this; + } + + /** + * Get the maxIntervalInMilliseconds property: The optional maximum time interval in milliseconds between retry + * attempts. + * + * @return the maxIntervalInMilliseconds value. + */ + public Integer maxIntervalInMilliseconds() { + return this.maxIntervalInMilliseconds; + } + + /** + * Set the maxIntervalInMilliseconds property: The optional maximum time interval in milliseconds between retry + * attempts. + * + * @param maxIntervalInMilliseconds the maxIntervalInMilliseconds value to set. + * @return the DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration object itself. + */ + public DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration + withMaxIntervalInMilliseconds(Integer maxIntervalInMilliseconds) { + this.maxIntervalInMilliseconds = maxIntervalInMilliseconds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration.java new file mode 100644 index 0000000000000..da8bf6a03fd4d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy HTTP Retry Policy Configuration. + */ +@Fluent +public final class DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration { + /* + * The optional maximum number of retries + */ + @JsonProperty(value = "maxRetries") + private Integer maxRetries; + + /* + * The optional retry backoff configuration + */ + @JsonProperty(value = "retryBackOff") + private DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration retryBackOff; + + /** + * Creates an instance of DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration class. + */ + public DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration() { + } + + /** + * Get the maxRetries property: The optional maximum number of retries. + * + * @return the maxRetries value. + */ + public Integer maxRetries() { + return this.maxRetries; + } + + /** + * Set the maxRetries property: The optional maximum number of retries. + * + * @param maxRetries the maxRetries value to set. + * @return the DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration withMaxRetries(Integer maxRetries) { + this.maxRetries = maxRetries; + return this; + } + + /** + * Get the retryBackOff property: The optional retry backoff configuration. + * + * @return the retryBackOff value. + */ + public DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration retryBackOff() { + return this.retryBackOff; + } + + /** + * Set the retryBackOff property: The optional retry backoff configuration. + * + * @param retryBackOff the retryBackOff value to set. + * @return the DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration + withRetryBackOff(DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration retryBackOff) { + this.retryBackOff = retryBackOff; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (retryBackOff() != null) { + retryBackOff().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyTimeoutPolicyConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyTimeoutPolicyConfiguration.java new file mode 100644 index 0000000000000..558f969164286 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentResiliencyPolicyTimeoutPolicyConfiguration.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Component Resiliency Policy Timeout Policy Configuration. + */ +@Fluent +public final class DaprComponentResiliencyPolicyTimeoutPolicyConfiguration { + /* + * The optional response timeout in seconds + */ + @JsonProperty(value = "responseTimeoutInSeconds") + private Integer responseTimeoutInSeconds; + + /** + * Creates an instance of DaprComponentResiliencyPolicyTimeoutPolicyConfiguration class. + */ + public DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() { + } + + /** + * Get the responseTimeoutInSeconds property: The optional response timeout in seconds. + * + * @return the responseTimeoutInSeconds value. + */ + public Integer responseTimeoutInSeconds() { + return this.responseTimeoutInSeconds; + } + + /** + * Set the responseTimeoutInSeconds property: The optional response timeout in seconds. + * + * @param responseTimeoutInSeconds the responseTimeoutInSeconds value to set. + * @return the DaprComponentResiliencyPolicyTimeoutPolicyConfiguration object itself. + */ + public DaprComponentResiliencyPolicyTimeoutPolicyConfiguration + withResponseTimeoutInSeconds(Integer responseTimeoutInSeconds) { + this.responseTimeoutInSeconds = responseTimeoutInSeconds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentServiceBinding.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentServiceBinding.java new file mode 100644 index 0000000000000..f2ab867136a41 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentServiceBinding.java @@ -0,0 +1,109 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration to bind a Dapr Component to a dev ContainerApp Service. + */ +@Fluent +public final class DaprComponentServiceBinding { + /* + * Name of the service bind + */ + @JsonProperty(value = "name") + private String name; + + /* + * Resource id of the target service + */ + @JsonProperty(value = "serviceId") + private String serviceId; + + /* + * Service bind metadata + */ + @JsonProperty(value = "metadata") + private DaprServiceBindMetadata metadata; + + /** + * Creates an instance of DaprComponentServiceBinding class. + */ + public DaprComponentServiceBinding() { + } + + /** + * Get the name property: Name of the service bind. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the service bind. + * + * @param name the name value to set. + * @return the DaprComponentServiceBinding object itself. + */ + public DaprComponentServiceBinding withName(String name) { + this.name = name; + return this; + } + + /** + * Get the serviceId property: Resource id of the target service. + * + * @return the serviceId value. + */ + public String serviceId() { + return this.serviceId; + } + + /** + * Set the serviceId property: Resource id of the target service. + * + * @param serviceId the serviceId value to set. + * @return the DaprComponentServiceBinding object itself. + */ + public DaprComponentServiceBinding withServiceId(String serviceId) { + this.serviceId = serviceId; + return this; + } + + /** + * Get the metadata property: Service bind metadata. + * + * @return the metadata value. + */ + public DaprServiceBindMetadata metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Service bind metadata. + * + * @param metadata the metadata value to set. + * @return the DaprComponentServiceBinding object itself. + */ + public DaprComponentServiceBinding withMetadata(DaprServiceBindMetadata metadata) { + this.metadata = metadata; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponents.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponents.java index decfd9343fba5..8834ca17b94b6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponents.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponents.java @@ -9,16 +9,18 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner; -/** Resource collection API of DaprComponents. */ +/** + * Resource collection API of DaprComponents. + */ public interface DaprComponents { /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a managed environment as paginated response with {@link PagedIterable}. */ @@ -26,13 +28,13 @@ public interface DaprComponents { /** * Get the Dapr Components for a managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Dapr Components for a managed environment as paginated response with {@link PagedIterable}. */ @@ -40,29 +42,29 @@ public interface DaprComponents { /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, String componentName, + Context context); /** * Get a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a dapr component. */ @@ -70,9 +72,9 @@ Response getWithResponse( /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. @@ -80,96 +82,89 @@ Response getWithResponse( * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr Component along with {@link Response}. */ - Response createOrUpdateWithResponse( - String resourceGroupName, - String environmentName, - String componentName, - DaprComponentInner daprComponentEnvelope, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String environmentName, + String componentName, DaprComponentInner daprComponentEnvelope, Context context); /** * Creates or updates a Dapr Component. - * - *

Creates or updates a Dapr Component in a Managed Environment. - * + * + * Creates or updates a Dapr Component in a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param daprComponentEnvelope Configuration details of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr Component. */ - DaprComponent createOrUpdate( - String resourceGroupName, - String environmentName, - String componentName, + DaprComponent createOrUpdate(String resourceGroupName, String environmentName, String componentName, DaprComponentInner daprComponentEnvelope); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String componentName, + Context context); /** * Delete a Dapr Component. - * - *

Delete a Dapr Component from a Managed Environment. - * + * + * Delete a Dapr Component from a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String environmentName, String componentName); /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action along with {@link Response}. */ - Response listSecretsWithResponse( - String resourceGroupName, String environmentName, String componentName, Context context); + Response listSecretsWithResponse(String resourceGroupName, String environmentName, + String componentName, Context context); /** * List secrets for a dapr component. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param componentName Name of the Dapr Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dapr component Secrets Collection for ListSecrets Action. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentsCollection.java index e3d61a519a551..a61699c74b2bf 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprComponentsCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Dapr Components ARM resource. */ +/** + * Dapr Components ARM resource. + */ @Fluent public final class DaprComponentsCollection { /* @@ -25,13 +27,15 @@ public final class DaprComponentsCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DaprComponentsCollection class. */ + /** + * Creates an instance of DaprComponentsCollection class. + */ public DaprComponentsCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the DaprComponentsCollection object itself. */ @@ -51,7 +55,7 @@ public DaprComponentsCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model DaprComponentsCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DaprComponentsCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprConfiguration.java index f0b9ce6544bef..fb4c30f8dfae1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration properties Dapr component. */ +/** + * Configuration properties Dapr component. + */ @Immutable public final class DaprConfiguration { /* @@ -16,13 +18,15 @@ public final class DaprConfiguration { @JsonProperty(value = "version", access = JsonProperty.Access.WRITE_ONLY) private String version; - /** Creates an instance of DaprConfiguration class. */ + /** + * Creates an instance of DaprConfiguration class. + */ public DaprConfiguration() { } /** * Get the version property: The version of Dapr. - * + * * @return the version value. */ public String version() { @@ -31,7 +35,7 @@ public String version() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprMetadata.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprMetadata.java index 41e6d4e7d8b30..4c1ba04e90353 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprMetadata.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprMetadata.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Dapr component metadata. */ +/** + * Dapr component metadata. + */ @Fluent public final class DaprMetadata { /* @@ -28,13 +30,15 @@ public final class DaprMetadata { @JsonProperty(value = "secretRef") private String secretRef; - /** Creates an instance of DaprMetadata class. */ + /** + * Creates an instance of DaprMetadata class. + */ public DaprMetadata() { } /** * Get the name property: Metadata property name. - * + * * @return the name value. */ public String name() { @@ -43,7 +47,7 @@ public String name() { /** * Set the name property: Metadata property name. - * + * * @param name the name value to set. * @return the DaprMetadata object itself. */ @@ -54,7 +58,7 @@ public DaprMetadata withName(String name) { /** * Get the value property: Metadata property value. - * + * * @return the value value. */ public String value() { @@ -63,7 +67,7 @@ public String value() { /** * Set the value property: Metadata property value. - * + * * @param value the value value to set. * @return the DaprMetadata object itself. */ @@ -74,7 +78,7 @@ public DaprMetadata withValue(String value) { /** * Get the secretRef property: Name of the Dapr Component secret from which to pull the metadata property value. - * + * * @return the secretRef value. */ public String secretRef() { @@ -83,7 +87,7 @@ public String secretRef() { /** * Set the secretRef property: Name of the Dapr Component secret from which to pull the metadata property value. - * + * * @param secretRef the secretRef value to set. * @return the DaprMetadata object itself. */ @@ -94,7 +98,7 @@ public DaprMetadata withSecretRef(String secretRef) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecret.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecret.java index 54d093bca047c..d30582091a869 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecret.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecret.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Dapr component Secret for ListSecrets Action. */ +/** + * Dapr component Secret for ListSecrets Action. + */ @Immutable public final class DaprSecret { /* @@ -22,13 +24,15 @@ public final class DaprSecret { @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) private String value; - /** Creates an instance of DaprSecret class. */ + /** + * Creates an instance of DaprSecret class. + */ public DaprSecret() { } /** * Get the name property: Secret Name. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Get the value property: Secret Value. - * + * * @return the value value. */ public String value() { @@ -46,7 +50,7 @@ public String value() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecretsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecretsCollection.java index de12dbb024343..7a8c0766d80ef 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecretsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSecretsCollection.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.appcontainers.fluent.models.DaprSecretsCollectionInner; import java.util.List; -/** An immutable client-side representation of DaprSecretsCollection. */ +/** + * An immutable client-side representation of DaprSecretsCollection. + */ public interface DaprSecretsCollection { /** * Gets the value property: Collection of secrets used by a Dapr component. - * + * * @return the value value. */ List value(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DaprSecretsCollectionInner object. - * + * * @return the inner object. */ DaprSecretsCollectionInner innerModel(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Capability.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprServiceBindMetadata.java similarity index 51% rename from sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Capability.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprServiceBindMetadata.java index 5eeb214d8498b..db50d0768187e 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Capability.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprServiceBindMetadata.java @@ -2,37 +2,37 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.redisenterprise.models; +package com.azure.resourcemanager.appcontainers.models; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; /** - * Capability - * - *

Information about the features the location supports. + * Dapr component metadata. */ @Fluent -public final class Capability { +public final class DaprServiceBindMetadata { /* - * Feature name + * Service bind metadata property name. */ @JsonProperty(value = "name") private String name; /* - * Indicates whether feature is supported or not + * Service bind metadata property value. */ @JsonProperty(value = "value") - private Boolean value; + private String value; - /** Creates an instance of Capability class. */ - public Capability() { + /** + * Creates an instance of DaprServiceBindMetadata class. + */ + public DaprServiceBindMetadata() { } /** - * Get the name property: Feature name. - * + * Get the name property: Service bind metadata property name. + * * @return the name value. */ public String name() { @@ -40,39 +40,39 @@ public String name() { } /** - * Set the name property: Feature name. - * + * Set the name property: Service bind metadata property name. + * * @param name the name value to set. - * @return the Capability object itself. + * @return the DaprServiceBindMetadata object itself. */ - public Capability withName(String name) { + public DaprServiceBindMetadata withName(String name) { this.name = name; return this; } /** - * Get the value property: Indicates whether feature is supported or not. - * + * Get the value property: Service bind metadata property value. + * * @return the value value. */ - public Boolean value() { + public String value() { return this.value; } /** - * Set the value property: Indicates whether feature is supported or not. - * + * Set the value property: Service bind metadata property value. + * * @param value the value value to set. - * @return the Capability object itself. + * @return the DaprServiceBindMetadata object itself. */ - public Capability withValue(Boolean value) { + public DaprServiceBindMetadata withValue(String value) { this.value = value; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscription.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscription.java new file mode 100644 index 0000000000000..f5283e5b9ec8f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscription.java @@ -0,0 +1,392 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import java.util.List; +import java.util.Map; + +/** + * An immutable client-side representation of DaprSubscription. + */ +public interface DaprSubscription { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the pubsubName property: Dapr PubSub component name. + * + * @return the pubsubName value. + */ + String pubsubName(); + + /** + * Gets the topic property: Topic name. + * + * @return the topic value. + */ + String topic(); + + /** + * Gets the deadLetterTopic property: Deadletter topic name. + * + * @return the deadLetterTopic value. + */ + String deadLetterTopic(); + + /** + * Gets the routes property: Subscription routes. + * + * @return the routes value. + */ + DaprSubscriptionRoutes routes(); + + /** + * Gets the scopes property: Application scopes to restrict the subscription to specific apps. + * + * @return the scopes value. + */ + List scopes(); + + /** + * Gets the metadata property: Subscription metadata. + * + * @return the metadata value. + */ + Map metadata(); + + /** + * Gets the bulkSubscribe property: Bulk subscription options. + * + * @return the bulkSubscribe value. + */ + DaprSubscriptionBulkSubscribeOptions bulkSubscribe(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner object. + * + * @return the inner object. + */ + DaprSubscriptionInner innerModel(); + + /** + * The entirety of the DaprSubscription definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The DaprSubscription definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the DaprSubscription definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the DaprSubscription definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, environmentName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @return the next definition stage. + */ + WithCreate withExistingManagedEnvironment(String resourceGroupName, String environmentName); + } + + /** + * The stage of the DaprSubscription definition which contains all the minimum required properties for the + * resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithPubsubName, DefinitionStages.WithTopic, + DefinitionStages.WithDeadLetterTopic, DefinitionStages.WithRoutes, DefinitionStages.WithScopes, + DefinitionStages.WithMetadata, DefinitionStages.WithBulkSubscribe { + /** + * Executes the create request. + * + * @return the created resource. + */ + DaprSubscription create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + DaprSubscription create(Context context); + } + + /** + * The stage of the DaprSubscription definition allowing to specify pubsubName. + */ + interface WithPubsubName { + /** + * Specifies the pubsubName property: Dapr PubSub component name. + * + * @param pubsubName Dapr PubSub component name. + * @return the next definition stage. + */ + WithCreate withPubsubName(String pubsubName); + } + + /** + * The stage of the DaprSubscription definition allowing to specify topic. + */ + interface WithTopic { + /** + * Specifies the topic property: Topic name. + * + * @param topic Topic name. + * @return the next definition stage. + */ + WithCreate withTopic(String topic); + } + + /** + * The stage of the DaprSubscription definition allowing to specify deadLetterTopic. + */ + interface WithDeadLetterTopic { + /** + * Specifies the deadLetterTopic property: Deadletter topic name. + * + * @param deadLetterTopic Deadletter topic name. + * @return the next definition stage. + */ + WithCreate withDeadLetterTopic(String deadLetterTopic); + } + + /** + * The stage of the DaprSubscription definition allowing to specify routes. + */ + interface WithRoutes { + /** + * Specifies the routes property: Subscription routes. + * + * @param routes Subscription routes. + * @return the next definition stage. + */ + WithCreate withRoutes(DaprSubscriptionRoutes routes); + } + + /** + * The stage of the DaprSubscription definition allowing to specify scopes. + */ + interface WithScopes { + /** + * Specifies the scopes property: Application scopes to restrict the subscription to specific apps.. + * + * @param scopes Application scopes to restrict the subscription to specific apps. + * @return the next definition stage. + */ + WithCreate withScopes(List scopes); + } + + /** + * The stage of the DaprSubscription definition allowing to specify metadata. + */ + interface WithMetadata { + /** + * Specifies the metadata property: Subscription metadata. + * + * @param metadata Subscription metadata. + * @return the next definition stage. + */ + WithCreate withMetadata(Map metadata); + } + + /** + * The stage of the DaprSubscription definition allowing to specify bulkSubscribe. + */ + interface WithBulkSubscribe { + /** + * Specifies the bulkSubscribe property: Bulk subscription options. + * + * @param bulkSubscribe Bulk subscription options. + * @return the next definition stage. + */ + WithCreate withBulkSubscribe(DaprSubscriptionBulkSubscribeOptions bulkSubscribe); + } + } + + /** + * Begins update for the DaprSubscription resource. + * + * @return the stage of resource update. + */ + DaprSubscription.Update update(); + + /** + * The template for DaprSubscription update. + */ + interface Update extends UpdateStages.WithPubsubName, UpdateStages.WithTopic, UpdateStages.WithDeadLetterTopic, + UpdateStages.WithRoutes, UpdateStages.WithScopes, UpdateStages.WithMetadata, UpdateStages.WithBulkSubscribe { + /** + * Executes the update request. + * + * @return the updated resource. + */ + DaprSubscription apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + DaprSubscription apply(Context context); + } + + /** + * The DaprSubscription update stages. + */ + interface UpdateStages { + /** + * The stage of the DaprSubscription update allowing to specify pubsubName. + */ + interface WithPubsubName { + /** + * Specifies the pubsubName property: Dapr PubSub component name. + * + * @param pubsubName Dapr PubSub component name. + * @return the next definition stage. + */ + Update withPubsubName(String pubsubName); + } + + /** + * The stage of the DaprSubscription update allowing to specify topic. + */ + interface WithTopic { + /** + * Specifies the topic property: Topic name. + * + * @param topic Topic name. + * @return the next definition stage. + */ + Update withTopic(String topic); + } + + /** + * The stage of the DaprSubscription update allowing to specify deadLetterTopic. + */ + interface WithDeadLetterTopic { + /** + * Specifies the deadLetterTopic property: Deadletter topic name. + * + * @param deadLetterTopic Deadletter topic name. + * @return the next definition stage. + */ + Update withDeadLetterTopic(String deadLetterTopic); + } + + /** + * The stage of the DaprSubscription update allowing to specify routes. + */ + interface WithRoutes { + /** + * Specifies the routes property: Subscription routes. + * + * @param routes Subscription routes. + * @return the next definition stage. + */ + Update withRoutes(DaprSubscriptionRoutes routes); + } + + /** + * The stage of the DaprSubscription update allowing to specify scopes. + */ + interface WithScopes { + /** + * Specifies the scopes property: Application scopes to restrict the subscription to specific apps.. + * + * @param scopes Application scopes to restrict the subscription to specific apps. + * @return the next definition stage. + */ + Update withScopes(List scopes); + } + + /** + * The stage of the DaprSubscription update allowing to specify metadata. + */ + interface WithMetadata { + /** + * Specifies the metadata property: Subscription metadata. + * + * @param metadata Subscription metadata. + * @return the next definition stage. + */ + Update withMetadata(Map metadata); + } + + /** + * The stage of the DaprSubscription update allowing to specify bulkSubscribe. + */ + interface WithBulkSubscribe { + /** + * Specifies the bulkSubscribe property: Bulk subscription options. + * + * @param bulkSubscribe Bulk subscription options. + * @return the next definition stage. + */ + Update withBulkSubscribe(DaprSubscriptionBulkSubscribeOptions bulkSubscribe); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + DaprSubscription refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + DaprSubscription refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionBulkSubscribeOptions.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionBulkSubscribeOptions.java new file mode 100644 index 0000000000000..75faad2981a7d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionBulkSubscribeOptions.java @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr PubSub Bulk Subscription Options. + */ +@Fluent +public final class DaprSubscriptionBulkSubscribeOptions { + /* + * Enable bulk subscription + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * Maximum number of messages to deliver in a bulk message. + */ + @JsonProperty(value = "maxMessagesCount") + private Integer maxMessagesCount; + + /* + * Maximum duration in milliseconds to wait before a bulk message is sent to the app. + */ + @JsonProperty(value = "maxAwaitDurationMs") + private Integer maxAwaitDurationMs; + + /** + * Creates an instance of DaprSubscriptionBulkSubscribeOptions class. + */ + public DaprSubscriptionBulkSubscribeOptions() { + } + + /** + * Get the enabled property: Enable bulk subscription. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Enable bulk subscription. + * + * @param enabled the enabled value to set. + * @return the DaprSubscriptionBulkSubscribeOptions object itself. + */ + public DaprSubscriptionBulkSubscribeOptions withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the maxMessagesCount property: Maximum number of messages to deliver in a bulk message. + * + * @return the maxMessagesCount value. + */ + public Integer maxMessagesCount() { + return this.maxMessagesCount; + } + + /** + * Set the maxMessagesCount property: Maximum number of messages to deliver in a bulk message. + * + * @param maxMessagesCount the maxMessagesCount value to set. + * @return the DaprSubscriptionBulkSubscribeOptions object itself. + */ + public DaprSubscriptionBulkSubscribeOptions withMaxMessagesCount(Integer maxMessagesCount) { + this.maxMessagesCount = maxMessagesCount; + return this; + } + + /** + * Get the maxAwaitDurationMs property: Maximum duration in milliseconds to wait before a bulk message is sent to + * the app. + * + * @return the maxAwaitDurationMs value. + */ + public Integer maxAwaitDurationMs() { + return this.maxAwaitDurationMs; + } + + /** + * Set the maxAwaitDurationMs property: Maximum duration in milliseconds to wait before a bulk message is sent to + * the app. + * + * @param maxAwaitDurationMs the maxAwaitDurationMs value to set. + * @return the DaprSubscriptionBulkSubscribeOptions object itself. + */ + public DaprSubscriptionBulkSubscribeOptions withMaxAwaitDurationMs(Integer maxAwaitDurationMs) { + this.maxAwaitDurationMs = maxAwaitDurationMs; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRouteRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRouteRule.java new file mode 100644 index 0000000000000..d6d2bdcf1338a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRouteRule.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Dapr Pubsub Event Subscription Route Rule is used to specify the condition for sending a message to a specific path. + */ +@Fluent +public final class DaprSubscriptionRouteRule { + /* + * The optional CEL expression used to match the event. If the match is not specified, then the route is considered + * the default. The rules are tested in the order specified, so they should be define from most-to-least specific. + * The default route should appear last in the list. + */ + @JsonProperty(value = "match") + private String match; + + /* + * The path for events that match this rule + */ + @JsonProperty(value = "path") + private String path; + + /** + * Creates an instance of DaprSubscriptionRouteRule class. + */ + public DaprSubscriptionRouteRule() { + } + + /** + * Get the match property: The optional CEL expression used to match the event. If the match is not specified, then + * the route is considered the default. The rules are tested in the order specified, so they should be define from + * most-to-least specific. The default route should appear last in the list. + * + * @return the match value. + */ + public String match() { + return this.match; + } + + /** + * Set the match property: The optional CEL expression used to match the event. If the match is not specified, then + * the route is considered the default. The rules are tested in the order specified, so they should be define from + * most-to-least specific. The default route should appear last in the list. + * + * @param match the match value to set. + * @return the DaprSubscriptionRouteRule object itself. + */ + public DaprSubscriptionRouteRule withMatch(String match) { + this.match = match; + return this; + } + + /** + * Get the path property: The path for events that match this rule. + * + * @return the path value. + */ + public String path() { + return this.path; + } + + /** + * Set the path property: The path for events that match this rule. + * + * @param path the path value to set. + * @return the DaprSubscriptionRouteRule object itself. + */ + public DaprSubscriptionRouteRule withPath(String path) { + this.path = path; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRoutes.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRoutes.java new file mode 100644 index 0000000000000..695bdbdb1e294 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionRoutes.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Dapr PubSub Event Subscription Routes configuration. + */ +@Fluent +public final class DaprSubscriptionRoutes { + /* + * The list of Dapr PubSub Event Subscription Route Rules. + */ + @JsonProperty(value = "rules") + private List rules; + + /* + * The default path to deliver events that do not match any of the rules. + */ + @JsonProperty(value = "default") + private String defaultProperty; + + /** + * Creates an instance of DaprSubscriptionRoutes class. + */ + public DaprSubscriptionRoutes() { + } + + /** + * Get the rules property: The list of Dapr PubSub Event Subscription Route Rules. + * + * @return the rules value. + */ + public List rules() { + return this.rules; + } + + /** + * Set the rules property: The list of Dapr PubSub Event Subscription Route Rules. + * + * @param rules the rules value to set. + * @return the DaprSubscriptionRoutes object itself. + */ + public DaprSubscriptionRoutes withRules(List rules) { + this.rules = rules; + return this; + } + + /** + * Get the defaultProperty property: The default path to deliver events that do not match any of the rules. + * + * @return the defaultProperty value. + */ + public String defaultProperty() { + return this.defaultProperty; + } + + /** + * Set the defaultProperty property: The default path to deliver events that do not match any of the rules. + * + * @param defaultProperty the defaultProperty value to set. + * @return the DaprSubscriptionRoutes object itself. + */ + public DaprSubscriptionRoutes withDefaultProperty(String defaultProperty) { + this.defaultProperty = defaultProperty; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (rules() != null) { + rules().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptions.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptions.java new file mode 100644 index 0000000000000..a7cc8cc87f40c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptions.java @@ -0,0 +1,164 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of DaprSubscriptions. + */ +public interface DaprSubscriptions { + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String environmentName); + + /** + * Get the Dapr subscriptions for a managed environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Dapr subscriptions for a managed environment as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String environmentName, Context context); + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context); + + /** + * Get a dapr subscription. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription. + */ + DaprSubscription get(String resourceGroupName, String environmentName, String name); + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteWithResponse(String resourceGroupName, String environmentName, String name, Context context); + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Dapr subscription. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String environmentName, String name); + + /** + * Get a dapr subscription. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response}. + */ + DaprSubscription getById(String id); + + /** + * Get a dapr subscription. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a dapr subscription along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteById(String id); + + /** + * Delete a Dapr subscription. + * + * Delete a Dapr subscription from a Managed Environment. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new DaprSubscription resource. + * + * @param name resource name. + * @return the first stage of the new DaprSubscription definition. + */ + DaprSubscription.DefinitionStages.Blank define(String name); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionsCollection.java new file mode 100644 index 0000000000000..e78b2ac2a46c7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DaprSubscriptionsCollection.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Dapr Subscriptions ARM resource. + */ +@Fluent +public final class DaprSubscriptionsCollection { + /* + * Collection of resources. + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * Link to next page of resources. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of DaprSubscriptionsCollection class. + */ + public DaprSubscriptionsCollection() { + } + + /** + * Get the value property: Collection of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Collection of resources. + * + * @param value the value value to set. + * @return the DaprSubscriptionsCollection object itself. + */ + public DaprSubscriptionsCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DaprSubscriptionsCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(DaprSubscriptionsCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DataDogConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DataDogConfiguration.java new file mode 100644 index 0000000000000..47880887d1991 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DataDogConfiguration.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration of datadog. + */ +@Fluent +public final class DataDogConfiguration { + /* + * The data dog site + */ + @JsonProperty(value = "site") + private String site; + + /* + * The data dog api key + */ + @JsonProperty(value = "key") + private String key; + + /** + * Creates an instance of DataDogConfiguration class. + */ + public DataDogConfiguration() { + } + + /** + * Get the site property: The data dog site. + * + * @return the site value. + */ + public String site() { + return this.site; + } + + /** + * Set the site property: The data dog site. + * + * @param site the site value to set. + * @return the DataDogConfiguration object itself. + */ + public DataDogConfiguration withSite(String site) { + this.site = site; + return this; + } + + /** + * Get the key property: The data dog api key. + * + * @return the key value. + */ + public String key() { + return this.key; + } + + /** + * Set the key property: The data dog api key. + * + * @param key the key value to set. + * @return the DataDogConfiguration object itself. + */ + public DataDogConfiguration withKey(String key) { + this.key = key; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultAuthorizationPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultAuthorizationPolicy.java index 701357e0e2177..49f7b2b12bbb5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultAuthorizationPolicy.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultAuthorizationPolicy.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the Azure Active Directory default authorization policy. */ +/** + * The configuration settings of the Azure Active Directory default authorization policy. + */ @Fluent public final class DefaultAuthorizationPolicy { /* @@ -23,13 +25,15 @@ public final class DefaultAuthorizationPolicy { @JsonProperty(value = "allowedApplications") private List allowedApplications; - /** Creates an instance of DefaultAuthorizationPolicy class. */ + /** + * Creates an instance of DefaultAuthorizationPolicy class. + */ public DefaultAuthorizationPolicy() { } /** * Get the allowedPrincipals property: The configuration settings of the Azure Active Directory allowed principals. - * + * * @return the allowedPrincipals value. */ public AllowedPrincipals allowedPrincipals() { @@ -38,7 +42,7 @@ public AllowedPrincipals allowedPrincipals() { /** * Set the allowedPrincipals property: The configuration settings of the Azure Active Directory allowed principals. - * + * * @param allowedPrincipals the allowedPrincipals value to set. * @return the DefaultAuthorizationPolicy object itself. */ @@ -50,7 +54,7 @@ public DefaultAuthorizationPolicy withAllowedPrincipals(AllowedPrincipals allowe /** * Get the allowedApplications property: The configuration settings of the Azure Active Directory allowed * applications. - * + * * @return the allowedApplications value. */ public List allowedApplications() { @@ -60,7 +64,7 @@ public List allowedApplications() { /** * Set the allowedApplications property: The configuration settings of the Azure Active Directory allowed * applications. - * + * * @param allowedApplications the allowedApplications value to set. * @return the DefaultAuthorizationPolicy object itself. */ @@ -71,7 +75,7 @@ public DefaultAuthorizationPolicy withAllowedApplications(List allowedAp /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseError.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseError.java index 0fd4fc04d3335..a0d8564a8449e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseError.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseError.java @@ -8,7 +8,9 @@ import com.azure.core.management.exception.ManagementError; import com.fasterxml.jackson.annotation.JsonProperty; -/** App Service error response. */ +/** + * App Service error response. + */ @Immutable public final class DefaultErrorResponseError extends ManagementError { /* @@ -17,13 +19,15 @@ public final class DefaultErrorResponseError extends ManagementError { @JsonProperty(value = "innererror", access = JsonProperty.Access.WRITE_ONLY) private String innererror; - /** Creates an instance of DefaultErrorResponseError class. */ + /** + * Creates an instance of DefaultErrorResponseError class. + */ public DefaultErrorResponseError() { } /** * Get the innererror property: More information to debug error. - * + * * @return the innererror value. */ public String getInnererror() { @@ -32,7 +36,7 @@ public String getInnererror() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseErrorException.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseErrorException.java index 9cfd6edb32583..9b49fecbf027a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseErrorException.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DefaultErrorResponseErrorException.java @@ -7,11 +7,13 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.exception.ManagementException; -/** Exception thrown for an invalid response with DefaultErrorResponseError information. */ +/** + * Exception thrown for an invalid response with DefaultErrorResponseError information. + */ public final class DefaultErrorResponseErrorException extends ManagementException { /** * Initializes a new instance of the DefaultErrorResponseErrorException class. - * + * * @param message the exception message or the response content if a message is not available. * @param response the HTTP response. */ @@ -21,7 +23,7 @@ public DefaultErrorResponseErrorException(String message, HttpResponse response) /** * Initializes a new instance of the DefaultErrorResponseErrorException class. - * + * * @param message the exception message or the response content if a message is not available. * @param response the HTTP response. * @param value the deserialized response value. @@ -30,7 +32,9 @@ public DefaultErrorResponseErrorException(String message, HttpResponse response, super(message, response, value); } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DefaultErrorResponseError getValue() { return (DefaultErrorResponseError) super.getValue(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DestinationsConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DestinationsConfiguration.java new file mode 100644 index 0000000000000..53fd75e4424b5 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DestinationsConfiguration.java @@ -0,0 +1,87 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration of Open Telemetry destinations. + */ +@Fluent +public final class DestinationsConfiguration { + /* + * Open telemetry datadog destination configuration + */ + @JsonProperty(value = "dataDogConfiguration") + private DataDogConfiguration dataDogConfiguration; + + /* + * Open telemetry otlp configurations + */ + @JsonProperty(value = "otlpConfigurations") + private List otlpConfigurations; + + /** + * Creates an instance of DestinationsConfiguration class. + */ + public DestinationsConfiguration() { + } + + /** + * Get the dataDogConfiguration property: Open telemetry datadog destination configuration. + * + * @return the dataDogConfiguration value. + */ + public DataDogConfiguration dataDogConfiguration() { + return this.dataDogConfiguration; + } + + /** + * Set the dataDogConfiguration property: Open telemetry datadog destination configuration. + * + * @param dataDogConfiguration the dataDogConfiguration value to set. + * @return the DestinationsConfiguration object itself. + */ + public DestinationsConfiguration withDataDogConfiguration(DataDogConfiguration dataDogConfiguration) { + this.dataDogConfiguration = dataDogConfiguration; + return this; + } + + /** + * Get the otlpConfigurations property: Open telemetry otlp configurations. + * + * @return the otlpConfigurations value. + */ + public List otlpConfigurations() { + return this.otlpConfigurations; + } + + /** + * Set the otlpConfigurations property: Open telemetry otlp configurations. + * + * @param otlpConfigurations the otlpConfigurations value to set. + * @return the DestinationsConfiguration object itself. + */ + public DestinationsConfiguration withOtlpConfigurations(List otlpConfigurations) { + this.otlpConfigurations = otlpConfigurations; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (dataDogConfiguration() != null) { + dataDogConfiguration().validate(); + } + if (otlpConfigurations() != null) { + otlpConfigurations().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadata.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadata.java index 70380a0296743..22ffbe86a162b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadata.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadata.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Details of a diagnostics data provider. */ +/** + * Details of a diagnostics data provider. + */ @Fluent public final class DiagnosticDataProviderMetadata { /* @@ -23,13 +25,15 @@ public final class DiagnosticDataProviderMetadata { @JsonProperty(value = "propertyBag") private List propertyBag; - /** Creates an instance of DiagnosticDataProviderMetadata class. */ + /** + * Creates an instance of DiagnosticDataProviderMetadata class. + */ public DiagnosticDataProviderMetadata() { } /** * Get the providerName property: Name of data provider. - * + * * @return the providerName value. */ public String providerName() { @@ -38,7 +42,7 @@ public String providerName() { /** * Set the providerName property: Name of data provider. - * + * * @param providerName the providerName value to set. * @return the DiagnosticDataProviderMetadata object itself. */ @@ -49,7 +53,7 @@ public DiagnosticDataProviderMetadata withProviderName(String providerName) { /** * Get the propertyBag property: Collection of properties. - * + * * @return the propertyBag value. */ public List propertyBag() { @@ -58,19 +62,19 @@ public List propertyBag() { /** * Set the propertyBag property: Collection of properties. - * + * * @param propertyBag the propertyBag value to set. * @return the DiagnosticDataProviderMetadata object itself. */ - public DiagnosticDataProviderMetadata withPropertyBag( - List propertyBag) { + public DiagnosticDataProviderMetadata + withPropertyBag(List propertyBag) { this.propertyBag = propertyBag; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadataPropertyBagItem.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadataPropertyBagItem.java index bc385080b743e..582e1a89341be 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadataPropertyBagItem.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataProviderMetadataPropertyBagItem.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Property details. */ +/** + * Property details. + */ @Fluent public final class DiagnosticDataProviderMetadataPropertyBagItem { /* @@ -22,13 +24,15 @@ public final class DiagnosticDataProviderMetadataPropertyBagItem { @JsonProperty(value = "value") private String value; - /** Creates an instance of DiagnosticDataProviderMetadataPropertyBagItem class. */ + /** + * Creates an instance of DiagnosticDataProviderMetadataPropertyBagItem class. + */ public DiagnosticDataProviderMetadataPropertyBagItem() { } /** * Get the name property: Property name. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Set the name property: Property name. - * + * * @param name the name value to set. * @return the DiagnosticDataProviderMetadataPropertyBagItem object itself. */ @@ -48,7 +52,7 @@ public DiagnosticDataProviderMetadataPropertyBagItem withName(String name) { /** * Get the value property: Property value. - * + * * @return the value value. */ public String value() { @@ -57,7 +61,7 @@ public String value() { /** * Set the value property: Property value. - * + * * @param value the value value to set. * @return the DiagnosticDataProviderMetadataPropertyBagItem object itself. */ @@ -68,7 +72,7 @@ public DiagnosticDataProviderMetadataPropertyBagItem withValue(String value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseColumn.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseColumn.java index 1379ab0e28000..4195cc5ea2f40 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseColumn.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseColumn.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Diagnostics data column. */ +/** + * Diagnostics data column. + */ @Fluent public final class DiagnosticDataTableResponseColumn { /* @@ -28,13 +30,15 @@ public final class DiagnosticDataTableResponseColumn { @JsonProperty(value = "columnType") private String columnType; - /** Creates an instance of DiagnosticDataTableResponseColumn class. */ + /** + * Creates an instance of DiagnosticDataTableResponseColumn class. + */ public DiagnosticDataTableResponseColumn() { } /** * Get the columnName property: Column name. - * + * * @return the columnName value. */ public String columnName() { @@ -43,7 +47,7 @@ public String columnName() { /** * Set the columnName property: Column name. - * + * * @param columnName the columnName value to set. * @return the DiagnosticDataTableResponseColumn object itself. */ @@ -54,7 +58,7 @@ public DiagnosticDataTableResponseColumn withColumnName(String columnName) { /** * Get the dataType property: Data type of the column. - * + * * @return the dataType value. */ public String dataType() { @@ -63,7 +67,7 @@ public String dataType() { /** * Set the dataType property: Data type of the column. - * + * * @param dataType the dataType value to set. * @return the DiagnosticDataTableResponseColumn object itself. */ @@ -74,7 +78,7 @@ public DiagnosticDataTableResponseColumn withDataType(String dataType) { /** * Get the columnType property: Column type. - * + * * @return the columnType value. */ public String columnType() { @@ -83,7 +87,7 @@ public String columnType() { /** * Set the columnType property: Column type. - * + * * @param columnType the columnType value to set. * @return the DiagnosticDataTableResponseColumn object itself. */ @@ -94,7 +98,7 @@ public DiagnosticDataTableResponseColumn withColumnType(String columnType) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseObject.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseObject.java index 6d338c3bc35bc..1f91ef838dccb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseObject.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticDataTableResponseObject.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Diagnostics data table. */ +/** + * Diagnostics data table. + */ @Fluent public final class DiagnosticDataTableResponseObject { /* @@ -29,13 +31,15 @@ public final class DiagnosticDataTableResponseObject { @JsonProperty(value = "rows") private List rows; - /** Creates an instance of DiagnosticDataTableResponseObject class. */ + /** + * Creates an instance of DiagnosticDataTableResponseObject class. + */ public DiagnosticDataTableResponseObject() { } /** * Get the tableName property: Table name. - * + * * @return the tableName value. */ public String tableName() { @@ -44,7 +48,7 @@ public String tableName() { /** * Set the tableName property: Table name. - * + * * @param tableName the tableName value to set. * @return the DiagnosticDataTableResponseObject object itself. */ @@ -55,7 +59,7 @@ public DiagnosticDataTableResponseObject withTableName(String tableName) { /** * Get the columns property: Columns in the table. - * + * * @return the columns value. */ public List columns() { @@ -64,7 +68,7 @@ public List columns() { /** * Set the columns property: Columns in the table. - * + * * @param columns the columns value to set. * @return the DiagnosticDataTableResponseObject object itself. */ @@ -75,7 +79,7 @@ public DiagnosticDataTableResponseObject withColumns(List rows() { @@ -84,7 +88,7 @@ public List rows() { /** * Set the rows property: Rows in the table. - * + * * @param rows the rows value to set. * @return the DiagnosticDataTableResponseObject object itself. */ @@ -95,7 +99,7 @@ public DiagnosticDataTableResponseObject withRows(List rows) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticRendering.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticRendering.java index b70b85b605002..3c400d9e5fa48 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticRendering.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticRendering.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Rendering details of a diagnostics table. */ +/** + * Rendering details of a diagnostics table. + */ @Fluent public final class DiagnosticRendering { /* @@ -34,13 +36,15 @@ public final class DiagnosticRendering { @JsonProperty(value = "isVisible") private Boolean isVisible; - /** Creates an instance of DiagnosticRendering class. */ + /** + * Creates an instance of DiagnosticRendering class. + */ public DiagnosticRendering() { } /** * Get the type property: Rendering type. - * + * * @return the type value. */ public Integer type() { @@ -49,7 +53,7 @@ public Integer type() { /** * Set the type property: Rendering type. - * + * * @param type the type value to set. * @return the DiagnosticRendering object itself. */ @@ -60,7 +64,7 @@ public DiagnosticRendering withType(Integer type) { /** * Get the title property: Title of the table. - * + * * @return the title value. */ public String title() { @@ -69,7 +73,7 @@ public String title() { /** * Set the title property: Title of the table. - * + * * @param title the title value to set. * @return the DiagnosticRendering object itself. */ @@ -80,7 +84,7 @@ public DiagnosticRendering withTitle(String title) { /** * Get the description property: Description of the table. - * + * * @return the description value. */ public String description() { @@ -89,7 +93,7 @@ public String description() { /** * Set the description property: Description of the table. - * + * * @param description the description value to set. * @return the DiagnosticRendering object itself. */ @@ -100,7 +104,7 @@ public DiagnosticRendering withDescription(String description) { /** * Get the isVisible property: Flag if the table should be rendered. - * + * * @return the isVisible value. */ public Boolean isVisible() { @@ -109,7 +113,7 @@ public Boolean isVisible() { /** * Set the isVisible property: Flag if the table should be rendered. - * + * * @param isVisible the isVisible value to set. * @return the DiagnosticRendering object itself. */ @@ -120,7 +124,7 @@ public DiagnosticRendering withIsVisible(Boolean isVisible) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticSupportTopic.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticSupportTopic.java index db92218e5f06d..ece2fd3fa51e6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticSupportTopic.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticSupportTopic.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Support topic information. */ +/** + * Support topic information. + */ @Immutable public final class DiagnosticSupportTopic { /* @@ -22,13 +24,15 @@ public final class DiagnosticSupportTopic { @JsonProperty(value = "pesId", access = JsonProperty.Access.WRITE_ONLY) private String pesId; - /** Creates an instance of DiagnosticSupportTopic class. */ + /** + * Creates an instance of DiagnosticSupportTopic class. + */ public DiagnosticSupportTopic() { } /** * Get the id property: Unique topic identifier. - * + * * @return the id value. */ public String id() { @@ -37,7 +41,7 @@ public String id() { /** * Get the pesId property: PES identifier. - * + * * @return the pesId value. */ public String pesId() { @@ -46,7 +50,7 @@ public String pesId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Diagnostics.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Diagnostics.java index 73bed89218f12..fb32bde4a6b74 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Diagnostics.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Diagnostics.java @@ -4,41 +4,51 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner; -/** An immutable client-side representation of Diagnostics. */ +/** + * An immutable client-side representation of Diagnostics. + */ public interface Diagnostics { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the properties property: Diagnostics resource specific properties. - * + * * @return the properties value. */ DiagnosticsProperties properties(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner object. - * + * * @return the inner object. */ DiagnosticsInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsCollection.java index 855f169b1bcaf..74cc7c4d5bd00 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsCollection.java @@ -7,25 +7,27 @@ import com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner; import java.util.List; -/** An immutable client-side representation of DiagnosticsCollection. */ +/** + * An immutable client-side representation of DiagnosticsCollection. + */ public interface DiagnosticsCollection { /** * Gets the value property: Collection of diagnostic data. - * + * * @return the value value. */ List value(); /** * Gets the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ String nextLink(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner object. - * + * * @return the inner object. */ DiagnosticsCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDataApiResponse.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDataApiResponse.java index dcfb30eb74e95..287125a29abdd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDataApiResponse.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDataApiResponse.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Diagnostics data returned from a detector. */ +/** + * Diagnostics data returned from a detector. + */ @Fluent public final class DiagnosticsDataApiResponse { /* @@ -22,13 +24,15 @@ public final class DiagnosticsDataApiResponse { @JsonProperty(value = "renderingProperties") private DiagnosticRendering renderingProperties; - /** Creates an instance of DiagnosticsDataApiResponse class. */ + /** + * Creates an instance of DiagnosticsDataApiResponse class. + */ public DiagnosticsDataApiResponse() { } /** * Get the table property: Table response. - * + * * @return the table value. */ public DiagnosticDataTableResponseObject table() { @@ -37,7 +41,7 @@ public DiagnosticDataTableResponseObject table() { /** * Set the table property: Table response. - * + * * @param table the table value to set. * @return the DiagnosticsDataApiResponse object itself. */ @@ -48,7 +52,7 @@ public DiagnosticsDataApiResponse withTable(DiagnosticDataTableResponseObject ta /** * Get the renderingProperties property: Details of the table response. - * + * * @return the renderingProperties value. */ public DiagnosticRendering renderingProperties() { @@ -57,7 +61,7 @@ public DiagnosticRendering renderingProperties() { /** * Set the renderingProperties property: Details of the table response. - * + * * @param renderingProperties the renderingProperties value to set. * @return the DiagnosticsDataApiResponse object itself. */ @@ -68,7 +72,7 @@ public DiagnosticsDataApiResponse withRenderingProperties(DiagnosticRendering re /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDefinition.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDefinition.java index 0be766b4afb67..8d3046762ba6a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDefinition.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsDefinition.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Metadata of the diagnostics response. */ +/** + * Metadata of the diagnostics response. + */ @Fluent public final class DiagnosticsDefinition { /* @@ -65,13 +67,15 @@ public final class DiagnosticsDefinition { @JsonProperty(value = "score", access = JsonProperty.Access.WRITE_ONLY) private Float score; - /** Creates an instance of DiagnosticsDefinition class. */ + /** + * Creates an instance of DiagnosticsDefinition class. + */ public DiagnosticsDefinition() { } /** * Get the id property: Unique detector name. - * + * * @return the id value. */ public String id() { @@ -80,7 +84,7 @@ public String id() { /** * Get the name property: Display Name of the detector. - * + * * @return the name value. */ public String name() { @@ -89,7 +93,7 @@ public String name() { /** * Get the description property: Details of the diagnostics info. - * + * * @return the description value. */ public String description() { @@ -98,7 +102,7 @@ public String description() { /** * Get the author property: Authors' names of the detector. - * + * * @return the author value. */ public String author() { @@ -107,7 +111,7 @@ public String author() { /** * Get the category property: Category of the detector. - * + * * @return the category value. */ public String category() { @@ -116,7 +120,7 @@ public String category() { /** * Get the supportTopicList property: List of support topics. - * + * * @return the supportTopicList value. */ public List supportTopicList() { @@ -125,7 +129,7 @@ public List supportTopicList() { /** * Set the supportTopicList property: List of support topics. - * + * * @param supportTopicList the supportTopicList value to set. * @return the DiagnosticsDefinition object itself. */ @@ -136,7 +140,7 @@ public DiagnosticsDefinition withSupportTopicList(List s /** * Get the analysisTypes property: List of analysis types. - * + * * @return the analysisTypes value. */ public List analysisTypes() { @@ -145,7 +149,7 @@ public List analysisTypes() { /** * Set the analysisTypes property: List of analysis types. - * + * * @param analysisTypes the analysisTypes value to set. * @return the DiagnosticsDefinition object itself. */ @@ -156,7 +160,7 @@ public DiagnosticsDefinition withAnalysisTypes(List analysisTypes) { /** * Get the type property: Authors' names of the detector. - * + * * @return the type value. */ public String type() { @@ -165,7 +169,7 @@ public String type() { /** * Get the score property: Authors' names of the detector. - * + * * @return the score value. */ public Float score() { @@ -174,7 +178,7 @@ public Float score() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsProperties.java index f0edd43a932b8..012915c98278f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Diagnostics resource specific properties. */ +/** + * Diagnostics resource specific properties. + */ @Fluent public final class DiagnosticsProperties { /* @@ -35,13 +37,15 @@ public final class DiagnosticsProperties { @JsonProperty(value = "dataProviderMetadata") private DiagnosticDataProviderMetadata dataProviderMetadata; - /** Creates an instance of DiagnosticsProperties class. */ + /** + * Creates an instance of DiagnosticsProperties class. + */ public DiagnosticsProperties() { } /** * Get the metadata property: Metadata of the diagnostics response. - * + * * @return the metadata value. */ public DiagnosticsDefinition metadata() { @@ -50,7 +54,7 @@ public DiagnosticsDefinition metadata() { /** * Set the metadata property: Metadata of the diagnostics response. - * + * * @param metadata the metadata value to set. * @return the DiagnosticsProperties object itself. */ @@ -61,7 +65,7 @@ public DiagnosticsProperties withMetadata(DiagnosticsDefinition metadata) { /** * Get the dataset property: Set of data collections associated with the response. - * + * * @return the dataset value. */ public List dataset() { @@ -70,7 +74,7 @@ public List dataset() { /** * Set the dataset property: Set of data collections associated with the response. - * + * * @param dataset the dataset value to set. * @return the DiagnosticsProperties object itself. */ @@ -81,7 +85,7 @@ public DiagnosticsProperties withDataset(List datase /** * Get the status property: Status of the diagnostics response. - * + * * @return the status value. */ public DiagnosticsStatus status() { @@ -90,7 +94,7 @@ public DiagnosticsStatus status() { /** * Set the status property: Status of the diagnostics response. - * + * * @param status the status value to set. * @return the DiagnosticsProperties object itself. */ @@ -101,7 +105,7 @@ public DiagnosticsProperties withStatus(DiagnosticsStatus status) { /** * Get the dataProviderMetadata property: List of data providers' metadata. - * + * * @return the dataProviderMetadata value. */ public DiagnosticDataProviderMetadata dataProviderMetadata() { @@ -110,7 +114,7 @@ public DiagnosticDataProviderMetadata dataProviderMetadata() { /** * Set the dataProviderMetadata property: List of data providers' metadata. - * + * * @param dataProviderMetadata the dataProviderMetadata value to set. * @return the DiagnosticsProperties object itself. */ @@ -121,7 +125,7 @@ public DiagnosticsProperties withDataProviderMetadata(DiagnosticDataProviderMeta /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsStatus.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsStatus.java index 899d1bffb3956..1379a4992d57f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsStatus.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DiagnosticsStatus.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Rendering details of a diagnostics table. */ +/** + * Rendering details of a diagnostics table. + */ @Fluent public final class DiagnosticsStatus { /* @@ -22,13 +24,15 @@ public final class DiagnosticsStatus { @JsonProperty(value = "statusId") private Integer statusId; - /** Creates an instance of DiagnosticsStatus class. */ + /** + * Creates an instance of DiagnosticsStatus class. + */ public DiagnosticsStatus() { } /** * Get the message property: Diagnostic message. - * + * * @return the message value. */ public String message() { @@ -37,7 +41,7 @@ public String message() { /** * Set the message property: Diagnostic message. - * + * * @param message the message value to set. * @return the DiagnosticsStatus object itself. */ @@ -48,7 +52,7 @@ public DiagnosticsStatus withMessage(String message) { /** * Get the statusId property: Status. - * + * * @return the statusId value. */ public Integer statusId() { @@ -57,7 +61,7 @@ public Integer statusId() { /** * Set the statusId property: Status. - * + * * @param statusId the statusId value to set. * @return the DiagnosticsStatus object itself. */ @@ -68,7 +72,7 @@ public DiagnosticsStatus withStatusId(Integer statusId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DnsVerificationTestResult.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DnsVerificationTestResult.java index be7d141905d1b..65a02dd71aeb7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DnsVerificationTestResult.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DnsVerificationTestResult.java @@ -7,18 +7,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** DNS verification test result. */ +/** + * DNS verification test result. + */ public enum DnsVerificationTestResult { - /** Enum value Passed. */ + /** + * Enum value Passed. + */ PASSED("Passed"), - /** Enum value Failed. */ + /** + * Enum value Failed. + */ FAILED("Failed"), - /** Enum value Skipped. */ + /** + * Enum value Skipped. + */ SKIPPED("Skipped"); - /** The actual serialized value for a DnsVerificationTestResult instance. */ + /** + * The actual serialized value for a DnsVerificationTestResult instance. + */ private final String value; DnsVerificationTestResult(String value) { @@ -27,7 +37,7 @@ public enum DnsVerificationTestResult { /** * Parses a serialized value to a DnsVerificationTestResult instance. - * + * * @param value the serialized value to parse. * @return the parsed DnsVerificationTestResult object, or null if unable to parse. */ @@ -45,7 +55,9 @@ public static DnsVerificationTestResult fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponent.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponent.java new file mode 100644 index 0000000000000..6b1f7daa57720 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponent.java @@ -0,0 +1,265 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import java.util.List; + +/** + * An immutable client-side representation of DotNetComponent. + */ +public interface DotNetComponent { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the componentType property: Type of the .NET Component. + * + * @return the componentType value. + */ + DotNetComponentType componentType(); + + /** + * Gets the provisioningState property: Provisioning state of the .NET Component. + * + * @return the provisioningState value. + */ + DotNetComponentProvisioningState provisioningState(); + + /** + * Gets the configurations property: List of .NET Components configuration properties. + * + * @return the configurations value. + */ + List configurations(); + + /** + * Gets the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @return the serviceBinds value. + */ + List serviceBinds(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner object. + * + * @return the inner object. + */ + DotNetComponentInner innerModel(); + + /** + * The entirety of the DotNetComponent definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The DotNetComponent definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the DotNetComponent definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the DotNetComponent definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, environmentName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @return the next definition stage. + */ + WithCreate withExistingManagedEnvironment(String resourceGroupName, String environmentName); + } + + /** + * The stage of the DotNetComponent definition which contains all the minimum required properties for the + * resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithComponentType, DefinitionStages.WithConfigurations, + DefinitionStages.WithServiceBinds { + /** + * Executes the create request. + * + * @return the created resource. + */ + DotNetComponent create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + DotNetComponent create(Context context); + } + + /** + * The stage of the DotNetComponent definition allowing to specify componentType. + */ + interface WithComponentType { + /** + * Specifies the componentType property: Type of the .NET Component.. + * + * @param componentType Type of the .NET Component. + * @return the next definition stage. + */ + WithCreate withComponentType(DotNetComponentType componentType); + } + + /** + * The stage of the DotNetComponent definition allowing to specify configurations. + */ + interface WithConfigurations { + /** + * Specifies the configurations property: List of .NET Components configuration properties. + * + * @param configurations List of .NET Components configuration properties. + * @return the next definition stage. + */ + WithCreate withConfigurations(List configurations); + } + + /** + * The stage of the DotNetComponent definition allowing to specify serviceBinds. + */ + interface WithServiceBinds { + /** + * Specifies the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @param serviceBinds List of .NET Components that are bound to the .NET component. + * @return the next definition stage. + */ + WithCreate withServiceBinds(List serviceBinds); + } + } + + /** + * Begins update for the DotNetComponent resource. + * + * @return the stage of resource update. + */ + DotNetComponent.Update update(); + + /** + * The template for DotNetComponent update. + */ + interface Update + extends UpdateStages.WithComponentType, UpdateStages.WithConfigurations, UpdateStages.WithServiceBinds { + /** + * Executes the update request. + * + * @return the updated resource. + */ + DotNetComponent apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + DotNetComponent apply(Context context); + } + + /** + * The DotNetComponent update stages. + */ + interface UpdateStages { + /** + * The stage of the DotNetComponent update allowing to specify componentType. + */ + interface WithComponentType { + /** + * Specifies the componentType property: Type of the .NET Component.. + * + * @param componentType Type of the .NET Component. + * @return the next definition stage. + */ + Update withComponentType(DotNetComponentType componentType); + } + + /** + * The stage of the DotNetComponent update allowing to specify configurations. + */ + interface WithConfigurations { + /** + * Specifies the configurations property: List of .NET Components configuration properties. + * + * @param configurations List of .NET Components configuration properties. + * @return the next definition stage. + */ + Update withConfigurations(List configurations); + } + + /** + * The stage of the DotNetComponent update allowing to specify serviceBinds. + */ + interface WithServiceBinds { + /** + * Specifies the serviceBinds property: List of .NET Components that are bound to the .NET component. + * + * @param serviceBinds List of .NET Components that are bound to the .NET component. + * @return the next definition stage. + */ + Update withServiceBinds(List serviceBinds); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + DotNetComponent refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + DotNetComponent refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentConfigurationProperty.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentConfigurationProperty.java new file mode 100644 index 0000000000000..9a529ae5766f6 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentConfigurationProperty.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration properties for a .NET Component. + */ +@Fluent +public final class DotNetComponentConfigurationProperty { + /* + * The name of the property + */ + @JsonProperty(value = "propertyName") + private String propertyName; + + /* + * The value of the property + */ + @JsonProperty(value = "value") + private String value; + + /** + * Creates an instance of DotNetComponentConfigurationProperty class. + */ + public DotNetComponentConfigurationProperty() { + } + + /** + * Get the propertyName property: The name of the property. + * + * @return the propertyName value. + */ + public String propertyName() { + return this.propertyName; + } + + /** + * Set the propertyName property: The name of the property. + * + * @param propertyName the propertyName value to set. + * @return the DotNetComponentConfigurationProperty object itself. + */ + public DotNetComponentConfigurationProperty withPropertyName(String propertyName) { + this.propertyName = propertyName; + return this; + } + + /** + * Get the value property: The value of the property. + * + * @return the value value. + */ + public String value() { + return this.value; + } + + /** + * Set the value property: The value of the property. + * + * @param value the value value to set. + * @return the DotNetComponentConfigurationProperty object itself. + */ + public DotNetComponentConfigurationProperty withValue(String value) { + this.value = value; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentProvisioningState.java new file mode 100644 index 0000000000000..eaffcc2516bc4 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentProvisioningState.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Provisioning state of the .NET Component. + */ +public final class DotNetComponentProvisioningState extends ExpandableStringEnum { + /** + * Static value Succeeded for DotNetComponentProvisioningState. + */ + public static final DotNetComponentProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Failed for DotNetComponentProvisioningState. + */ + public static final DotNetComponentProvisioningState FAILED = fromString("Failed"); + + /** + * Static value Canceled for DotNetComponentProvisioningState. + */ + public static final DotNetComponentProvisioningState CANCELED = fromString("Canceled"); + + /** + * Static value Deleting for DotNetComponentProvisioningState. + */ + public static final DotNetComponentProvisioningState DELETING = fromString("Deleting"); + + /** + * Static value InProgress for DotNetComponentProvisioningState. + */ + public static final DotNetComponentProvisioningState IN_PROGRESS = fromString("InProgress"); + + /** + * Creates a new instance of DotNetComponentProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public DotNetComponentProvisioningState() { + } + + /** + * Creates or finds a DotNetComponentProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding DotNetComponentProvisioningState. + */ + @JsonCreator + public static DotNetComponentProvisioningState fromString(String name) { + return fromString(name, DotNetComponentProvisioningState.class); + } + + /** + * Gets known DotNetComponentProvisioningState values. + * + * @return known DotNetComponentProvisioningState values. + */ + public static Collection values() { + return values(DotNetComponentProvisioningState.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentServiceBind.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentServiceBind.java new file mode 100644 index 0000000000000..88f33359f4834 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentServiceBind.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration to bind a .NET Component to another .NET Component. + */ +@Fluent +public final class DotNetComponentServiceBind { + /* + * Name of the service bind + */ + @JsonProperty(value = "name") + private String name; + + /* + * Resource id of the target service + */ + @JsonProperty(value = "serviceId") + private String serviceId; + + /** + * Creates an instance of DotNetComponentServiceBind class. + */ + public DotNetComponentServiceBind() { + } + + /** + * Get the name property: Name of the service bind. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the service bind. + * + * @param name the name value to set. + * @return the DotNetComponentServiceBind object itself. + */ + public DotNetComponentServiceBind withName(String name) { + this.name = name; + return this; + } + + /** + * Get the serviceId property: Resource id of the target service. + * + * @return the serviceId value. + */ + public String serviceId() { + return this.serviceId; + } + + /** + * Set the serviceId property: Resource id of the target service. + * + * @param serviceId the serviceId value to set. + * @return the DotNetComponentServiceBind object itself. + */ + public DotNetComponentServiceBind withServiceId(String serviceId) { + this.serviceId = serviceId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentType.java new file mode 100644 index 0000000000000..9d1346962f4fd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Type of the .NET Component. + */ +public final class DotNetComponentType extends ExpandableStringEnum { + /** + * Static value AspireDashboard for DotNetComponentType. + */ + public static final DotNetComponentType ASPIRE_DASHBOARD = fromString("AspireDashboard"); + + /** + * Static value AspireResourceServerApi for DotNetComponentType. + */ + public static final DotNetComponentType ASPIRE_RESOURCE_SERVER_API = fromString("AspireResourceServerApi"); + + /** + * Creates a new instance of DotNetComponentType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public DotNetComponentType() { + } + + /** + * Creates or finds a DotNetComponentType from its string representation. + * + * @param name a name to look for. + * @return the corresponding DotNetComponentType. + */ + @JsonCreator + public static DotNetComponentType fromString(String name) { + return fromString(name, DotNetComponentType.class); + } + + /** + * Gets known DotNetComponentType values. + * + * @return known DotNetComponentType values. + */ + public static Collection values() { + return values(DotNetComponentType.class); + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaults.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponents.java similarity index 62% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaults.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponents.java index bdf958fac0789..18177c70e5865 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaults.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponents.java @@ -2,137 +2,119 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.models; +package com.azure.resourcemanager.appcontainers.models; import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; /** - * Resource collection API of BackupVaults. + * Resource collection API of DotNetComponents. */ -public interface BackupVaults { +public interface DotNetComponents { /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the .NET Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedIterable}. + * @return the .NET Components for a managed environment as paginated response with {@link PagedIterable}. */ - PagedIterable listByNetAppAccount(String resourceGroupName, String accountName); + PagedIterable list(String resourceGroupName, String environmentName); /** - * Describe all Backup Vaults - * - * List and describe all Backup Vaults in the NetApp account. + * Get the .NET Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. + * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backup Vaults as paginated response with {@link PagedIterable}. + * @return the .NET Components for a managed environment as paginated response with {@link PagedIterable}. */ - PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, Context context); + PagedIterable list(String resourceGroupName, String environmentName, Context context); /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response}. + * @return a .NET Component along with {@link Response}. */ - Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, + Response getWithResponse(String resourceGroupName, String environmentName, String name, Context context); /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault. + * @return a .NET Component. */ - BackupVault get(String resourceGroupName, String accountName, String backupVaultName); + DotNetComponent get(String resourceGroupName, String environmentName, String name); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete(String resourceGroupName, String accountName, String backupVaultName); + void delete(String resourceGroupName, String environmentName, String name); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. + * @param environmentName Name of the Managed Environment. + * @param name Name of the .NET Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete(String resourceGroupName, String accountName, String backupVaultName, Context context); + void delete(String resourceGroupName, String environmentName, String name, Context context); /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a .NET Component. * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response}. + * @return a .NET Component along with {@link Response}. */ - BackupVault getById(String id); + DotNetComponent getById(String id); /** - * Describe the Backup Vault - * - * Get the Backup Vault. + * Get a .NET Component. * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Backup Vault along with {@link Response}. + * @return a .NET Component along with {@link Response}. */ - Response getByIdWithResponse(String id, Context context); + Response getByIdWithResponse(String id, Context context); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -142,9 +124,7 @@ Response getWithResponse(String resourceGroupName, String accountNa void deleteById(String id); /** - * Delete NetApp Backup Vault - * - * Delete the specified Backup Vault. + * Delete a .NET Component. * * @param id the resource ID. * @param context The context to associate with this operation. @@ -155,10 +135,10 @@ Response getWithResponse(String resourceGroupName, String accountNa void deleteByIdWithResponse(String id, Context context); /** - * Begins definition for a new BackupVault resource. + * Begins definition for a new DotNetComponent resource. * * @param name resource name. - * @return the first stage of the new BackupVault definition. + * @return the first stage of the new DotNetComponent definition. */ - BackupVault.DefinitionStages.Blank define(String name); + DotNetComponent.DefinitionStages.Blank define(String name); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentsCollection.java new file mode 100644 index 0000000000000..bd61c158ab4dc --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/DotNetComponentsCollection.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * .NET Components ARM resource. + */ +@Fluent +public final class DotNetComponentsCollection { + /* + * Collection of resources. + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * Link to next page of resources. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of DotNetComponentsCollection class. + */ + public DotNetComponentsCollection() { + } + + /** + * Get the value property: Collection of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Collection of resources. + * + * @param value the value value to set. + * @return the DotNetComponentsCollection object itself. + */ + public DotNetComponentsCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model DotNetComponentsCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(DotNetComponentsCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EncryptionSettings.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EncryptionSettings.java new file mode 100644 index 0000000000000..57068af0e0d4c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EncryptionSettings.java @@ -0,0 +1,81 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The configuration settings of the secrets references of encryption key and signing key for ContainerApp Service + * Authentication/Authorization. + */ +@Fluent +public final class EncryptionSettings { + /* + * The secret name which is referenced for EncryptionKey. + */ + @JsonProperty(value = "containerAppAuthEncryptionSecretName") + private String containerAppAuthEncryptionSecretName; + + /* + * The secret name which is referenced for SigningKey. + */ + @JsonProperty(value = "containerAppAuthSigningSecretName") + private String containerAppAuthSigningSecretName; + + /** + * Creates an instance of EncryptionSettings class. + */ + public EncryptionSettings() { + } + + /** + * Get the containerAppAuthEncryptionSecretName property: The secret name which is referenced for EncryptionKey. + * + * @return the containerAppAuthEncryptionSecretName value. + */ + public String containerAppAuthEncryptionSecretName() { + return this.containerAppAuthEncryptionSecretName; + } + + /** + * Set the containerAppAuthEncryptionSecretName property: The secret name which is referenced for EncryptionKey. + * + * @param containerAppAuthEncryptionSecretName the containerAppAuthEncryptionSecretName value to set. + * @return the EncryptionSettings object itself. + */ + public EncryptionSettings withContainerAppAuthEncryptionSecretName(String containerAppAuthEncryptionSecretName) { + this.containerAppAuthEncryptionSecretName = containerAppAuthEncryptionSecretName; + return this; + } + + /** + * Get the containerAppAuthSigningSecretName property: The secret name which is referenced for SigningKey. + * + * @return the containerAppAuthSigningSecretName value. + */ + public String containerAppAuthSigningSecretName() { + return this.containerAppAuthSigningSecretName; + } + + /** + * Set the containerAppAuthSigningSecretName property: The secret name which is referenced for SigningKey. + * + * @param containerAppAuthSigningSecretName the containerAppAuthSigningSecretName value to set. + * @return the EncryptionSettings object itself. + */ + public EncryptionSettings withContainerAppAuthSigningSecretName(String containerAppAuthSigningSecretName) { + this.containerAppAuthSigningSecretName = containerAppAuthSigningSecretName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentAuthToken.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentAuthToken.java index 10f4f38fd2fe2..46933f2d9ed06 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentAuthToken.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentAuthToken.java @@ -9,67 +9,69 @@ import java.time.OffsetDateTime; import java.util.Map; -/** An immutable client-side representation of EnvironmentAuthToken. */ +/** + * An immutable client-side representation of EnvironmentAuthToken. + */ public interface EnvironmentAuthToken { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the token property: Auth token value. - * + * * @return the token value. */ String token(); /** * Gets the expires property: Token expiration date. - * + * * @return the expires value. */ OffsetDateTime expires(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.EnvironmentAuthTokenInner object. - * + * * @return the inner object. */ EnvironmentAuthTokenInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentProvisioningState.java index b7b9a41cdc280..255c3a14cf5a0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentProvisioningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentProvisioningState.java @@ -8,44 +8,66 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning state of the Environment. */ +/** + * Provisioning state of the Environment. + */ public final class EnvironmentProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for EnvironmentProvisioningState. */ + /** + * Static value Succeeded for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for EnvironmentProvisioningState. */ + /** + * Static value Failed for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for EnvironmentProvisioningState. */ + /** + * Static value Canceled for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState CANCELED = fromString("Canceled"); - /** Static value Waiting for EnvironmentProvisioningState. */ + /** + * Static value Waiting for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState WAITING = fromString("Waiting"); - /** Static value InitializationInProgress for EnvironmentProvisioningState. */ - public static final EnvironmentProvisioningState INITIALIZATION_IN_PROGRESS = - fromString("InitializationInProgress"); + /** + * Static value InitializationInProgress for EnvironmentProvisioningState. + */ + public static final EnvironmentProvisioningState INITIALIZATION_IN_PROGRESS + = fromString("InitializationInProgress"); - /** Static value InfrastructureSetupInProgress for EnvironmentProvisioningState. */ - public static final EnvironmentProvisioningState INFRASTRUCTURE_SETUP_IN_PROGRESS = - fromString("InfrastructureSetupInProgress"); + /** + * Static value InfrastructureSetupInProgress for EnvironmentProvisioningState. + */ + public static final EnvironmentProvisioningState INFRASTRUCTURE_SETUP_IN_PROGRESS + = fromString("InfrastructureSetupInProgress"); - /** Static value InfrastructureSetupComplete for EnvironmentProvisioningState. */ - public static final EnvironmentProvisioningState INFRASTRUCTURE_SETUP_COMPLETE = - fromString("InfrastructureSetupComplete"); + /** + * Static value InfrastructureSetupComplete for EnvironmentProvisioningState. + */ + public static final EnvironmentProvisioningState INFRASTRUCTURE_SETUP_COMPLETE + = fromString("InfrastructureSetupComplete"); - /** Static value ScheduledForDelete for EnvironmentProvisioningState. */ + /** + * Static value ScheduledForDelete for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState SCHEDULED_FOR_DELETE = fromString("ScheduledForDelete"); - /** Static value UpgradeRequested for EnvironmentProvisioningState. */ + /** + * Static value UpgradeRequested for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState UPGRADE_REQUESTED = fromString("UpgradeRequested"); - /** Static value UpgradeFailed for EnvironmentProvisioningState. */ + /** + * Static value UpgradeFailed for EnvironmentProvisioningState. + */ public static final EnvironmentProvisioningState UPGRADE_FAILED = fromString("UpgradeFailed"); /** * Creates a new instance of EnvironmentProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -54,7 +76,7 @@ public EnvironmentProvisioningState() { /** * Creates or finds a EnvironmentProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding EnvironmentProvisioningState. */ @@ -65,7 +87,7 @@ public static EnvironmentProvisioningState fromString(String name) { /** * Gets known EnvironmentProvisioningState values. - * + * * @return known EnvironmentProvisioningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVar.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVar.java index 415303a593df5..4355fee30854c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVar.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVar.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App container environment variable. */ +/** + * Container App container environment variable. + */ @Fluent public final class EnvironmentVar { /* @@ -28,13 +30,15 @@ public final class EnvironmentVar { @JsonProperty(value = "secretRef") private String secretRef; - /** Creates an instance of EnvironmentVar class. */ + /** + * Creates an instance of EnvironmentVar class. + */ public EnvironmentVar() { } /** * Get the name property: Environment variable name. - * + * * @return the name value. */ public String name() { @@ -43,7 +47,7 @@ public String name() { /** * Set the name property: Environment variable name. - * + * * @param name the name value to set. * @return the EnvironmentVar object itself. */ @@ -54,7 +58,7 @@ public EnvironmentVar withName(String name) { /** * Get the value property: Non-secret environment variable value. - * + * * @return the value value. */ public String value() { @@ -63,7 +67,7 @@ public String value() { /** * Set the value property: Non-secret environment variable value. - * + * * @param value the value value to set. * @return the EnvironmentVar object itself. */ @@ -74,7 +78,7 @@ public EnvironmentVar withValue(String value) { /** * Get the secretRef property: Name of the Container App secret from which to pull the environment variable value. - * + * * @return the secretRef value. */ public String secretRef() { @@ -83,7 +87,7 @@ public String secretRef() { /** * Set the secretRef property: Name of the Container App secret from which to pull the environment variable value. - * + * * @param secretRef the secretRef value to set. * @return the EnvironmentVar object itself. */ @@ -94,7 +98,7 @@ public EnvironmentVar withSecretRef(String secretRef) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVariable.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVariable.java new file mode 100644 index 0000000000000..0e8c6c43c8f2f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/EnvironmentVariable.java @@ -0,0 +1,91 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Model representing an environment variable. + */ +@Fluent +public final class EnvironmentVariable { + /* + * Environment variable name. + */ + @JsonProperty(value = "name", required = true) + private String name; + + /* + * Environment variable value. + */ + @JsonProperty(value = "value", required = true) + private String value; + + /** + * Creates an instance of EnvironmentVariable class. + */ + public EnvironmentVariable() { + } + + /** + * Get the name property: Environment variable name. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Environment variable name. + * + * @param name the name value to set. + * @return the EnvironmentVariable object itself. + */ + public EnvironmentVariable withName(String name) { + this.name = name; + return this; + } + + /** + * Get the value property: Environment variable value. + * + * @return the value value. + */ + public String value() { + return this.value; + } + + /** + * Set the value property: Environment variable value. + * + * @param value the value value to set. + * @return the EnvironmentVariable object itself. + */ + public EnvironmentVariable withValue(String value) { + this.value = value; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (name() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model EnvironmentVariable")); + } + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model EnvironmentVariable")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(EnvironmentVariable.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocation.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocation.java index 01baaa9fa9f5d..8bb445ea86da1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocation.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocation.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The complex type of the extended location. */ +/** + * The complex type of the extended location. + */ @Fluent public final class ExtendedLocation { /* @@ -22,13 +24,15 @@ public final class ExtendedLocation { @JsonProperty(value = "type") private ExtendedLocationTypes type; - /** Creates an instance of ExtendedLocation class. */ + /** + * Creates an instance of ExtendedLocation class. + */ public ExtendedLocation() { } /** * Get the name property: The name of the extended location. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Set the name property: The name of the extended location. - * + * * @param name the name value to set. * @return the ExtendedLocation object itself. */ @@ -48,7 +52,7 @@ public ExtendedLocation withName(String name) { /** * Get the type property: The type of the extended location. - * + * * @return the type value. */ public ExtendedLocationTypes type() { @@ -57,7 +61,7 @@ public ExtendedLocationTypes type() { /** * Set the type property: The type of the extended location. - * + * * @param type the type value to set. * @return the ExtendedLocation object itself. */ @@ -68,7 +72,7 @@ public ExtendedLocation withType(ExtendedLocationTypes type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocationTypes.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocationTypes.java index cabf1439ca647..fc9716ab3040d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocationTypes.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ExtendedLocationTypes.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The type of extendedLocation. */ +/** + * The type of extendedLocation. + */ public final class ExtendedLocationTypes extends ExpandableStringEnum { - /** Static value CustomLocation for ExtendedLocationTypes. */ + /** + * Static value CustomLocation for ExtendedLocationTypes. + */ public static final ExtendedLocationTypes CUSTOM_LOCATION = fromString("CustomLocation"); /** * Creates a new instance of ExtendedLocationTypes value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ExtendedLocationTypes() { /** * Creates or finds a ExtendedLocationTypes from its string representation. - * + * * @param name a name to look for. * @return the corresponding ExtendedLocationTypes. */ @@ -35,7 +39,7 @@ public static ExtendedLocationTypes fromString(String name) { /** * Gets known ExtendedLocationTypes values. - * + * * @return known ExtendedLocationTypes values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Facebook.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Facebook.java index 217ee0f35166d..6eebc6093dd5c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Facebook.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Facebook.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Facebook provider. */ +/** + * The configuration settings of the Facebook provider. + */ @Fluent public final class Facebook { /* @@ -35,14 +37,16 @@ public final class Facebook { @JsonProperty(value = "login") private LoginScopes login; - /** Creates an instance of Facebook class. */ + /** + * Creates an instance of Facebook class. + */ public Facebook() { } /** * Get the enabled property: <code>false</code> if the Facebook provider should not be enabled despite * the set registration; otherwise, <code>true</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -52,7 +56,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>false</code> if the Facebook provider should not be enabled despite * the set registration; otherwise, <code>true</code>. - * + * * @param enabled the enabled value to set. * @return the Facebook object itself. */ @@ -63,7 +67,7 @@ public Facebook withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the app registration for the Facebook provider. - * + * * @return the registration value. */ public AppRegistration registration() { @@ -72,7 +76,7 @@ public AppRegistration registration() { /** * Set the registration property: The configuration settings of the app registration for the Facebook provider. - * + * * @param registration the registration value to set. * @return the Facebook object itself. */ @@ -83,7 +87,7 @@ public Facebook withRegistration(AppRegistration registration) { /** * Get the graphApiVersion property: The version of the Facebook api to be used while logging in. - * + * * @return the graphApiVersion value. */ public String graphApiVersion() { @@ -92,7 +96,7 @@ public String graphApiVersion() { /** * Set the graphApiVersion property: The version of the Facebook api to be used while logging in. - * + * * @param graphApiVersion the graphApiVersion value to set. * @return the Facebook object itself. */ @@ -103,7 +107,7 @@ public Facebook withGraphApiVersion(String graphApiVersion) { /** * Get the login property: The configuration settings of the login flow. - * + * * @return the login value. */ public LoginScopes login() { @@ -112,7 +116,7 @@ public LoginScopes login() { /** * Set the login property: The configuration settings of the login flow. - * + * * @param login the login value to set. * @return the Facebook object itself. */ @@ -123,7 +127,7 @@ public Facebook withLogin(LoginScopes login) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxy.java index f60cb05443c4a..4c1c94333cb40 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxy.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxy.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of a forward proxy used to make the requests. */ +/** + * The configuration settings of a forward proxy used to make the requests. + */ @Fluent public final class ForwardProxy { /* @@ -28,13 +30,15 @@ public final class ForwardProxy { @JsonProperty(value = "customProtoHeaderName") private String customProtoHeaderName; - /** Creates an instance of ForwardProxy class. */ + /** + * Creates an instance of ForwardProxy class. + */ public ForwardProxy() { } /** * Get the convention property: The convention used to determine the url of the request made. - * + * * @return the convention value. */ public ForwardProxyConvention convention() { @@ -43,7 +47,7 @@ public ForwardProxyConvention convention() { /** * Set the convention property: The convention used to determine the url of the request made. - * + * * @param convention the convention value to set. * @return the ForwardProxy object itself. */ @@ -54,7 +58,7 @@ public ForwardProxy withConvention(ForwardProxyConvention convention) { /** * Get the customHostHeaderName property: The name of the header containing the host of the request. - * + * * @return the customHostHeaderName value. */ public String customHostHeaderName() { @@ -63,7 +67,7 @@ public String customHostHeaderName() { /** * Set the customHostHeaderName property: The name of the header containing the host of the request. - * + * * @param customHostHeaderName the customHostHeaderName value to set. * @return the ForwardProxy object itself. */ @@ -74,7 +78,7 @@ public ForwardProxy withCustomHostHeaderName(String customHostHeaderName) { /** * Get the customProtoHeaderName property: The name of the header containing the scheme of the request. - * + * * @return the customProtoHeaderName value. */ public String customProtoHeaderName() { @@ -83,7 +87,7 @@ public String customProtoHeaderName() { /** * Set the customProtoHeaderName property: The name of the header containing the scheme of the request. - * + * * @param customProtoHeaderName the customProtoHeaderName value to set. * @return the ForwardProxy object itself. */ @@ -94,7 +98,7 @@ public ForwardProxy withCustomProtoHeaderName(String customProtoHeaderName) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxyConvention.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxyConvention.java index 67a969e08129c..987f429671959 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxyConvention.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ForwardProxyConvention.java @@ -7,18 +7,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** The convention used to determine the url of the request made. */ +/** + * The convention used to determine the url of the request made. + */ public enum ForwardProxyConvention { - /** Enum value NoProxy. */ + /** + * Enum value NoProxy. + */ NO_PROXY("NoProxy"), - /** Enum value Standard. */ + /** + * Enum value Standard. + */ STANDARD("Standard"), - /** Enum value Custom. */ + /** + * Enum value Custom. + */ CUSTOM("Custom"); - /** The actual serialized value for a ForwardProxyConvention instance. */ + /** + * The actual serialized value for a ForwardProxyConvention instance. + */ private final String value; ForwardProxyConvention(String value) { @@ -27,7 +37,7 @@ public enum ForwardProxyConvention { /** * Parses a serialized value to a ForwardProxyConvention instance. - * + * * @param value the serialized value to parse. * @return the parsed ForwardProxyConvention object, or null if unable to parse. */ @@ -45,7 +55,9 @@ public static ForwardProxyConvention fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GitHub.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GitHub.java index 3cbf2faf2f43a..f017ad4e55b7e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GitHub.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GitHub.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the GitHub provider. */ +/** + * The configuration settings of the GitHub provider. + */ @Fluent public final class GitHub { /* @@ -29,14 +31,16 @@ public final class GitHub { @JsonProperty(value = "login") private LoginScopes login; - /** Creates an instance of GitHub class. */ + /** + * Creates an instance of GitHub class. + */ public GitHub() { } /** - * Get the enabled property: <code>false</code> if the GitHub provider should not be enabled despite the - * set registration; otherwise, <code>true</code>. - * + * Get the enabled property: <code>false</code> if the GitHub provider should not be enabled despite + * the set registration; otherwise, <code>true</code>. + * * @return the enabled value. */ public Boolean enabled() { @@ -44,9 +48,9 @@ public Boolean enabled() { } /** - * Set the enabled property: <code>false</code> if the GitHub provider should not be enabled despite the - * set registration; otherwise, <code>true</code>. - * + * Set the enabled property: <code>false</code> if the GitHub provider should not be enabled despite + * the set registration; otherwise, <code>true</code>. + * * @param enabled the enabled value to set. * @return the GitHub object itself. */ @@ -57,7 +61,7 @@ public GitHub withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the app registration for the GitHub provider. - * + * * @return the registration value. */ public ClientRegistration registration() { @@ -66,7 +70,7 @@ public ClientRegistration registration() { /** * Set the registration property: The configuration settings of the app registration for the GitHub provider. - * + * * @param registration the registration value to set. * @return the GitHub object itself. */ @@ -77,7 +81,7 @@ public GitHub withRegistration(ClientRegistration registration) { /** * Get the login property: The configuration settings of the login flow. - * + * * @return the login value. */ public LoginScopes login() { @@ -86,7 +90,7 @@ public LoginScopes login() { /** * Set the login property: The configuration settings of the login flow. - * + * * @param login the login value to set. * @return the GitHub object itself. */ @@ -97,7 +101,7 @@ public GitHub withLogin(LoginScopes login) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GithubActionConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GithubActionConfiguration.java index 2c1cf3be67f4b..ea17571bff533 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GithubActionConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GithubActionConfiguration.java @@ -6,8 +6,11 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; -/** Configuration properties that define the mutable settings of a Container App SourceControl. */ +/** + * Configuration properties that define the mutable settings of a Container App SourceControl. + */ @Fluent public final class GithubActionConfiguration { /* @@ -64,13 +67,21 @@ public final class GithubActionConfiguration { @JsonProperty(value = "runtimeVersion") private String runtimeVersion; - /** Creates an instance of GithubActionConfiguration class. */ + /* + * List of environment variables to be passed to the build. + */ + @JsonProperty(value = "buildEnvironmentVariables") + private List buildEnvironmentVariables; + + /** + * Creates an instance of GithubActionConfiguration class. + */ public GithubActionConfiguration() { } /** * Get the registryInfo property: Registry configurations. - * + * * @return the registryInfo value. */ public RegistryInfo registryInfo() { @@ -79,7 +90,7 @@ public RegistryInfo registryInfo() { /** * Set the registryInfo property: Registry configurations. - * + * * @param registryInfo the registryInfo value to set. * @return the GithubActionConfiguration object itself. */ @@ -90,7 +101,7 @@ public GithubActionConfiguration withRegistryInfo(RegistryInfo registryInfo) { /** * Get the azureCredentials property: AzureCredentials configurations. - * + * * @return the azureCredentials value. */ public AzureCredentials azureCredentials() { @@ -99,7 +110,7 @@ public AzureCredentials azureCredentials() { /** * Set the azureCredentials property: AzureCredentials configurations. - * + * * @param azureCredentials the azureCredentials value to set. * @return the GithubActionConfiguration object itself. */ @@ -110,7 +121,7 @@ public GithubActionConfiguration withAzureCredentials(AzureCredentials azureCred /** * Get the contextPath property: Context path. - * + * * @return the contextPath value. */ public String contextPath() { @@ -119,7 +130,7 @@ public String contextPath() { /** * Set the contextPath property: Context path. - * + * * @param contextPath the contextPath value to set. * @return the GithubActionConfiguration object itself. */ @@ -130,7 +141,7 @@ public GithubActionConfiguration withContextPath(String contextPath) { /** * Get the githubPersonalAccessToken property: One time Github PAT to configure github environment. - * + * * @return the githubPersonalAccessToken value. */ public String githubPersonalAccessToken() { @@ -139,7 +150,7 @@ public String githubPersonalAccessToken() { /** * Set the githubPersonalAccessToken property: One time Github PAT to configure github environment. - * + * * @param githubPersonalAccessToken the githubPersonalAccessToken value to set. * @return the GithubActionConfiguration object itself. */ @@ -150,7 +161,7 @@ public GithubActionConfiguration withGithubPersonalAccessToken(String githubPers /** * Get the image property: Image name. - * + * * @return the image value. */ public String image() { @@ -159,7 +170,7 @@ public String image() { /** * Set the image property: Image name. - * + * * @param image the image value to set. * @return the GithubActionConfiguration object itself. */ @@ -170,7 +181,7 @@ public GithubActionConfiguration withImage(String image) { /** * Get the publishType property: Code or Image. - * + * * @return the publishType value. */ public String publishType() { @@ -179,7 +190,7 @@ public String publishType() { /** * Set the publishType property: Code or Image. - * + * * @param publishType the publishType value to set. * @return the GithubActionConfiguration object itself. */ @@ -190,7 +201,7 @@ public GithubActionConfiguration withPublishType(String publishType) { /** * Get the os property: Operation system. - * + * * @return the os value. */ public String os() { @@ -199,7 +210,7 @@ public String os() { /** * Set the os property: Operation system. - * + * * @param os the os value to set. * @return the GithubActionConfiguration object itself. */ @@ -210,7 +221,7 @@ public GithubActionConfiguration withOs(String os) { /** * Get the runtimeStack property: Runtime stack. - * + * * @return the runtimeStack value. */ public String runtimeStack() { @@ -219,7 +230,7 @@ public String runtimeStack() { /** * Set the runtimeStack property: Runtime stack. - * + * * @param runtimeStack the runtimeStack value to set. * @return the GithubActionConfiguration object itself. */ @@ -230,7 +241,7 @@ public GithubActionConfiguration withRuntimeStack(String runtimeStack) { /** * Get the runtimeVersion property: Runtime version. - * + * * @return the runtimeVersion value. */ public String runtimeVersion() { @@ -239,7 +250,7 @@ public String runtimeVersion() { /** * Set the runtimeVersion property: Runtime version. - * + * * @param runtimeVersion the runtimeVersion value to set. * @return the GithubActionConfiguration object itself. */ @@ -248,9 +259,30 @@ public GithubActionConfiguration withRuntimeVersion(String runtimeVersion) { return this; } + /** + * Get the buildEnvironmentVariables property: List of environment variables to be passed to the build. + * + * @return the buildEnvironmentVariables value. + */ + public List buildEnvironmentVariables() { + return this.buildEnvironmentVariables; + } + + /** + * Set the buildEnvironmentVariables property: List of environment variables to be passed to the build. + * + * @param buildEnvironmentVariables the buildEnvironmentVariables value to set. + * @return the GithubActionConfiguration object itself. + */ + public GithubActionConfiguration + withBuildEnvironmentVariables(List buildEnvironmentVariables) { + this.buildEnvironmentVariables = buildEnvironmentVariables; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -260,5 +292,8 @@ public void validate() { if (azureCredentials() != null) { azureCredentials().validate(); } + if (buildEnvironmentVariables() != null) { + buildEnvironmentVariables().forEach(e -> e.validate()); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GlobalValidation.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GlobalValidation.java index 9cc8c4a8f6254..133206ec91f31 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GlobalValidation.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/GlobalValidation.java @@ -34,14 +34,16 @@ public final class GlobalValidation { @JsonProperty(value = "excludedPaths") private List excludedPaths; - /** Creates an instance of GlobalValidation class. */ + /** + * Creates an instance of GlobalValidation class. + */ public GlobalValidation() { } /** * Get the unauthenticatedClientAction property: The action to take when an unauthenticated client attempts to * access the app. - * + * * @return the unauthenticatedClientAction value. */ public UnauthenticatedClientActionV2 unauthenticatedClientAction() { @@ -51,7 +53,7 @@ public UnauthenticatedClientActionV2 unauthenticatedClientAction() { /** * Set the unauthenticatedClientAction property: The action to take when an unauthenticated client attempts to * access the app. - * + * * @param unauthenticatedClientAction the unauthenticatedClientAction value to set. * @return the GlobalValidation object itself. */ @@ -62,9 +64,10 @@ public GlobalValidation withUnauthenticatedClientAction(UnauthenticatedClientAct /** * Get the redirectToProvider property: The default authentication provider to use when multiple providers are - * configured. This setting is only needed if multiple providers are configured and the unauthenticated client + * configured. + * This setting is only needed if multiple providers are configured and the unauthenticated client * action is set to "RedirectToLoginPage". - * + * * @return the redirectToProvider value. */ public String redirectToProvider() { @@ -73,9 +76,10 @@ public String redirectToProvider() { /** * Set the redirectToProvider property: The default authentication provider to use when multiple providers are - * configured. This setting is only needed if multiple providers are configured and the unauthenticated client + * configured. + * This setting is only needed if multiple providers are configured and the unauthenticated client * action is set to "RedirectToLoginPage". - * + * * @param redirectToProvider the redirectToProvider value to set. * @return the GlobalValidation object itself. */ @@ -87,7 +91,7 @@ public GlobalValidation withRedirectToProvider(String redirectToProvider) { /** * Get the excludedPaths property: The paths for which unauthenticated flow would not be redirected to the login * page. - * + * * @return the excludedPaths value. */ public List excludedPaths() { @@ -97,7 +101,7 @@ public List excludedPaths() { /** * Set the excludedPaths property: The paths for which unauthenticated flow would not be redirected to the login * page. - * + * * @param excludedPaths the excludedPaths value to set. * @return the GlobalValidation object itself. */ @@ -108,7 +112,7 @@ public GlobalValidation withExcludedPaths(List excludedPaths) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Google.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Google.java index 2d3054e4162dd..085f2d03acff2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Google.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Google.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Google provider. */ +/** + * The configuration settings of the Google provider. + */ @Fluent public final class Google { /* @@ -35,14 +37,16 @@ public final class Google { @JsonProperty(value = "validation") private AllowedAudiencesValidation validation; - /** Creates an instance of Google class. */ + /** + * Creates an instance of Google class. + */ public Google() { } /** - * Get the enabled property: <code>false</code> if the Google provider should not be enabled despite the - * set registration; otherwise, <code>true</code>. - * + * Get the enabled property: <code>false</code> if the Google provider should not be enabled despite + * the set registration; otherwise, <code>true</code>. + * * @return the enabled value. */ public Boolean enabled() { @@ -50,9 +54,9 @@ public Boolean enabled() { } /** - * Set the enabled property: <code>false</code> if the Google provider should not be enabled despite the - * set registration; otherwise, <code>true</code>. - * + * Set the enabled property: <code>false</code> if the Google provider should not be enabled despite + * the set registration; otherwise, <code>true</code>. + * * @param enabled the enabled value to set. * @return the Google object itself. */ @@ -63,7 +67,7 @@ public Google withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the app registration for the Google provider. - * + * * @return the registration value. */ public ClientRegistration registration() { @@ -72,7 +76,7 @@ public ClientRegistration registration() { /** * Set the registration property: The configuration settings of the app registration for the Google provider. - * + * * @param registration the registration value to set. * @return the Google object itself. */ @@ -83,7 +87,7 @@ public Google withRegistration(ClientRegistration registration) { /** * Get the login property: The configuration settings of the login flow. - * + * * @return the login value. */ public LoginScopes login() { @@ -92,7 +96,7 @@ public LoginScopes login() { /** * Set the login property: The configuration settings of the login flow. - * + * * @param login the login value to set. * @return the Google object itself. */ @@ -103,7 +107,7 @@ public Google withLogin(LoginScopes login) { /** * Get the validation property: The configuration settings of the Azure Active Directory token validation flow. - * + * * @return the validation value. */ public AllowedAudiencesValidation validation() { @@ -112,7 +116,7 @@ public AllowedAudiencesValidation validation() { /** * Set the validation property: The configuration settings of the Azure Active Directory token validation flow. - * + * * @param validation the validation value to set. * @return the Google object itself. */ @@ -123,7 +127,7 @@ public Google withValidation(AllowedAudiencesValidation validation) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Header.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Header.java new file mode 100644 index 0000000000000..d8364dae74577 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Header.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Header of otlp configuration. + */ +@Fluent +public final class Header { + /* + * The key of otlp configuration header + */ + @JsonProperty(value = "key") + private String key; + + /* + * The value of otlp configuration header + */ + @JsonProperty(value = "value") + private String value; + + /** + * Creates an instance of Header class. + */ + public Header() { + } + + /** + * Get the key property: The key of otlp configuration header. + * + * @return the key value. + */ + public String key() { + return this.key; + } + + /** + * Set the key property: The key of otlp configuration header. + * + * @param key the key value to set. + * @return the Header object itself. + */ + public Header withKey(String key) { + this.key = key; + return this; + } + + /** + * Get the value property: The value of otlp configuration header. + * + * @return the value value. + */ + public String value() { + return this.value; + } + + /** + * Set the value property: The value of otlp configuration header. + * + * @param value the value value to set. + * @return the Header object itself. + */ + public Header withValue(String value) { + this.value = value; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HeaderMatch.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HeaderMatch.java new file mode 100644 index 0000000000000..c1d1d4b55426d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HeaderMatch.java @@ -0,0 +1,165 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.fluent.models.HeaderMatchMatch; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Conditions required to match a header. + */ +@Fluent +public final class HeaderMatch { + /* + * Name of the header + */ + @JsonProperty(value = "header") + private String headerProperty; + + /* + * Type of match to perform + */ + @JsonProperty(value = "match") + private HeaderMatchMatch innerMatch; + + /** + * Creates an instance of HeaderMatch class. + */ + public HeaderMatch() { + } + + /** + * Get the headerProperty property: Name of the header. + * + * @return the headerProperty value. + */ + public String headerProperty() { + return this.headerProperty; + } + + /** + * Set the headerProperty property: Name of the header. + * + * @param headerProperty the headerProperty value to set. + * @return the HeaderMatch object itself. + */ + public HeaderMatch withHeaderProperty(String headerProperty) { + this.headerProperty = headerProperty; + return this; + } + + /** + * Get the innerMatch property: Type of match to perform. + * + * @return the innerMatch value. + */ + private HeaderMatchMatch innerMatch() { + return this.innerMatch; + } + + /** + * Get the exactMatch property: Exact value of the header. + * + * @return the exactMatch value. + */ + public String exactMatch() { + return this.innerMatch() == null ? null : this.innerMatch().exactMatch(); + } + + /** + * Set the exactMatch property: Exact value of the header. + * + * @param exactMatch the exactMatch value to set. + * @return the HeaderMatch object itself. + */ + public HeaderMatch withExactMatch(String exactMatch) { + if (this.innerMatch() == null) { + this.innerMatch = new HeaderMatchMatch(); + } + this.innerMatch().withExactMatch(exactMatch); + return this; + } + + /** + * Get the prefixMatch property: Prefix value of the header. + * + * @return the prefixMatch value. + */ + public String prefixMatch() { + return this.innerMatch() == null ? null : this.innerMatch().prefixMatch(); + } + + /** + * Set the prefixMatch property: Prefix value of the header. + * + * @param prefixMatch the prefixMatch value to set. + * @return the HeaderMatch object itself. + */ + public HeaderMatch withPrefixMatch(String prefixMatch) { + if (this.innerMatch() == null) { + this.innerMatch = new HeaderMatchMatch(); + } + this.innerMatch().withPrefixMatch(prefixMatch); + return this; + } + + /** + * Get the suffixMatch property: Suffix value of the header. + * + * @return the suffixMatch value. + */ + public String suffixMatch() { + return this.innerMatch() == null ? null : this.innerMatch().suffixMatch(); + } + + /** + * Set the suffixMatch property: Suffix value of the header. + * + * @param suffixMatch the suffixMatch value to set. + * @return the HeaderMatch object itself. + */ + public HeaderMatch withSuffixMatch(String suffixMatch) { + if (this.innerMatch() == null) { + this.innerMatch = new HeaderMatchMatch(); + } + this.innerMatch().withSuffixMatch(suffixMatch); + return this; + } + + /** + * Get the regexMatch property: Regex value of the header. + * + * @return the regexMatch value. + */ + public String regexMatch() { + return this.innerMatch() == null ? null : this.innerMatch().regexMatch(); + } + + /** + * Set the regexMatch property: Regex value of the header. + * + * @param regexMatch the regexMatch value to set. + * @return the HeaderMatch object itself. + */ + public HeaderMatch withRegexMatch(String regexMatch) { + if (this.innerMatch() == null) { + this.innerMatch = new HeaderMatchMatch(); + } + this.innerMatch().withRegexMatch(regexMatch); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerMatch() != null) { + innerMatch().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpConnectionPool.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpConnectionPool.java new file mode 100644 index 0000000000000..1b1502dcce298 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpConnectionPool.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Defines parameters for http connection pooling. + */ +@Fluent +public final class HttpConnectionPool { + /* + * Maximum number of pending http1 requests allowed + */ + @JsonProperty(value = "http1MaxPendingRequests") + private Integer http1MaxPendingRequests; + + /* + * Maximum number of http2 requests allowed + */ + @JsonProperty(value = "http2MaxRequests") + private Integer http2MaxRequests; + + /** + * Creates an instance of HttpConnectionPool class. + */ + public HttpConnectionPool() { + } + + /** + * Get the http1MaxPendingRequests property: Maximum number of pending http1 requests allowed. + * + * @return the http1MaxPendingRequests value. + */ + public Integer http1MaxPendingRequests() { + return this.http1MaxPendingRequests; + } + + /** + * Set the http1MaxPendingRequests property: Maximum number of pending http1 requests allowed. + * + * @param http1MaxPendingRequests the http1MaxPendingRequests value to set. + * @return the HttpConnectionPool object itself. + */ + public HttpConnectionPool withHttp1MaxPendingRequests(Integer http1MaxPendingRequests) { + this.http1MaxPendingRequests = http1MaxPendingRequests; + return this; + } + + /** + * Get the http2MaxRequests property: Maximum number of http2 requests allowed. + * + * @return the http2MaxRequests value. + */ + public Integer http2MaxRequests() { + return this.http2MaxRequests; + } + + /** + * Set the http2MaxRequests property: Maximum number of http2 requests allowed. + * + * @param http2MaxRequests the http2MaxRequests value to set. + * @return the HttpConnectionPool object itself. + */ + public HttpConnectionPool withHttp2MaxRequests(Integer http2MaxRequests) { + this.http2MaxRequests = http2MaxRequests; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpGet.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpGet.java new file mode 100644 index 0000000000000..bd154edb34233 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpGet.java @@ -0,0 +1,114 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Model representing a http get request. + */ +@Fluent +public final class HttpGet { + /* + * URL to make HTTP GET request against. + */ + @JsonProperty(value = "url", required = true) + private String url; + + /* + * Name of the file that the request should be saved to. + */ + @JsonProperty(value = "fileName") + private String fileName; + + /* + * List of headers to send with the request. + */ + @JsonProperty(value = "headers") + private List headers; + + /** + * Creates an instance of HttpGet class. + */ + public HttpGet() { + } + + /** + * Get the url property: URL to make HTTP GET request against. + * + * @return the url value. + */ + public String url() { + return this.url; + } + + /** + * Set the url property: URL to make HTTP GET request against. + * + * @param url the url value to set. + * @return the HttpGet object itself. + */ + public HttpGet withUrl(String url) { + this.url = url; + return this; + } + + /** + * Get the fileName property: Name of the file that the request should be saved to. + * + * @return the fileName value. + */ + public String fileName() { + return this.fileName; + } + + /** + * Set the fileName property: Name of the file that the request should be saved to. + * + * @param fileName the fileName value to set. + * @return the HttpGet object itself. + */ + public HttpGet withFileName(String fileName) { + this.fileName = fileName; + return this; + } + + /** + * Get the headers property: List of headers to send with the request. + * + * @return the headers value. + */ + public List headers() { + return this.headers; + } + + /** + * Set the headers property: List of headers to send with the request. + * + * @param headers the headers value to set. + * @return the HttpGet object itself. + */ + public HttpGet withHeaders(List headers) { + this.headers = headers; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (url() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property url in model HttpGet")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(HttpGet.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpRetryPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpRetryPolicy.java new file mode 100644 index 0000000000000..e80b222f5f1e8 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpRetryPolicy.java @@ -0,0 +1,208 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.fluent.models.HttpRetryPolicyMatches; +import com.azure.resourcemanager.appcontainers.fluent.models.HttpRetryPolicyRetryBackOff; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Policy that defines http request retry conditions. + */ +@Fluent +public final class HttpRetryPolicy { + /* + * Maximum number of times a request will retry + */ + @JsonProperty(value = "maxRetries") + private Integer maxRetries; + + /* + * Settings for retry backoff characteristics + */ + @JsonProperty(value = "retryBackOff") + private HttpRetryPolicyRetryBackOff innerRetryBackOff; + + /* + * Conditions that must be met for a request to be retried + */ + @JsonProperty(value = "matches") + private HttpRetryPolicyMatches innerMatches; + + /** + * Creates an instance of HttpRetryPolicy class. + */ + public HttpRetryPolicy() { + } + + /** + * Get the maxRetries property: Maximum number of times a request will retry. + * + * @return the maxRetries value. + */ + public Integer maxRetries() { + return this.maxRetries; + } + + /** + * Set the maxRetries property: Maximum number of times a request will retry. + * + * @param maxRetries the maxRetries value to set. + * @return the HttpRetryPolicy object itself. + */ + public HttpRetryPolicy withMaxRetries(Integer maxRetries) { + this.maxRetries = maxRetries; + return this; + } + + /** + * Get the innerRetryBackOff property: Settings for retry backoff characteristics. + * + * @return the innerRetryBackOff value. + */ + private HttpRetryPolicyRetryBackOff innerRetryBackOff() { + return this.innerRetryBackOff; + } + + /** + * Get the innerMatches property: Conditions that must be met for a request to be retried. + * + * @return the innerMatches value. + */ + private HttpRetryPolicyMatches innerMatches() { + return this.innerMatches; + } + + /** + * Get the initialDelayInMilliseconds property: Initial delay, in milliseconds, before retrying a request. + * + * @return the initialDelayInMilliseconds value. + */ + public Long initialDelayInMilliseconds() { + return this.innerRetryBackOff() == null ? null : this.innerRetryBackOff().initialDelayInMilliseconds(); + } + + /** + * Set the initialDelayInMilliseconds property: Initial delay, in milliseconds, before retrying a request. + * + * @param initialDelayInMilliseconds the initialDelayInMilliseconds value to set. + * @return the HttpRetryPolicy object itself. + */ + public HttpRetryPolicy withInitialDelayInMilliseconds(Long initialDelayInMilliseconds) { + if (this.innerRetryBackOff() == null) { + this.innerRetryBackOff = new HttpRetryPolicyRetryBackOff(); + } + this.innerRetryBackOff().withInitialDelayInMilliseconds(initialDelayInMilliseconds); + return this; + } + + /** + * Get the maxIntervalInMilliseconds property: Maximum interval, in milliseconds, between retries. + * + * @return the maxIntervalInMilliseconds value. + */ + public Long maxIntervalInMilliseconds() { + return this.innerRetryBackOff() == null ? null : this.innerRetryBackOff().maxIntervalInMilliseconds(); + } + + /** + * Set the maxIntervalInMilliseconds property: Maximum interval, in milliseconds, between retries. + * + * @param maxIntervalInMilliseconds the maxIntervalInMilliseconds value to set. + * @return the HttpRetryPolicy object itself. + */ + public HttpRetryPolicy withMaxIntervalInMilliseconds(Long maxIntervalInMilliseconds) { + if (this.innerRetryBackOff() == null) { + this.innerRetryBackOff = new HttpRetryPolicyRetryBackOff(); + } + this.innerRetryBackOff().withMaxIntervalInMilliseconds(maxIntervalInMilliseconds); + return this; + } + + /** + * Get the headers property: Headers that must be present for a request to be retried. + * + * @return the headers value. + */ + public List headers() { + return this.innerMatches() == null ? null : this.innerMatches().headers(); + } + + /** + * Set the headers property: Headers that must be present for a request to be retried. + * + * @param headers the headers value to set. + * @return the HttpRetryPolicy object itself. + */ + public HttpRetryPolicy withHeaders(List headers) { + if (this.innerMatches() == null) { + this.innerMatches = new HttpRetryPolicyMatches(); + } + this.innerMatches().withHeaders(headers); + return this; + } + + /** + * Get the httpStatusCodes property: Additional http status codes that can trigger a retry. + * + * @return the httpStatusCodes value. + */ + public List httpStatusCodes() { + return this.innerMatches() == null ? null : this.innerMatches().httpStatusCodes(); + } + + /** + * Set the httpStatusCodes property: Additional http status codes that can trigger a retry. + * + * @param httpStatusCodes the httpStatusCodes value to set. + * @return the HttpRetryPolicy object itself. + */ + public HttpRetryPolicy withHttpStatusCodes(List httpStatusCodes) { + if (this.innerMatches() == null) { + this.innerMatches = new HttpRetryPolicyMatches(); + } + this.innerMatches().withHttpStatusCodes(httpStatusCodes); + return this; + } + + /** + * Get the errors property: Errors that can trigger a retry. + * + * @return the errors value. + */ + public List errors() { + return this.innerMatches() == null ? null : this.innerMatches().errors(); + } + + /** + * Set the errors property: Errors that can trigger a retry. + * + * @param errors the errors value to set. + * @return the HttpRetryPolicy object itself. + */ + public HttpRetryPolicy withErrors(List errors) { + if (this.innerMatches() == null) { + this.innerMatches = new HttpRetryPolicyMatches(); + } + this.innerMatches().withErrors(errors); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerRetryBackOff() != null) { + innerRetryBackOff().validate(); + } + if (innerMatches() != null) { + innerMatches().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpScaleRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpScaleRule.java index 1e42b8eb3e4f0..0ecbbf8ea5f2d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpScaleRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpScaleRule.java @@ -10,7 +10,9 @@ import java.util.List; import java.util.Map; -/** Container App container Http scaling rule. */ +/** + * Container App container Http scaling rule. + */ @Fluent public final class HttpScaleRule { /* @@ -26,13 +28,15 @@ public final class HttpScaleRule { @JsonProperty(value = "auth") private List auth; - /** Creates an instance of HttpScaleRule class. */ + /** + * Creates an instance of HttpScaleRule class. + */ public HttpScaleRule() { } /** * Get the metadata property: Metadata properties to describe http scale rule. - * + * * @return the metadata value. */ public Map metadata() { @@ -41,7 +45,7 @@ public Map metadata() { /** * Set the metadata property: Metadata properties to describe http scale rule. - * + * * @param metadata the metadata value to set. * @return the HttpScaleRule object itself. */ @@ -52,7 +56,7 @@ public HttpScaleRule withMetadata(Map metadata) { /** * Get the auth property: Authentication secrets for the custom scale rule. - * + * * @return the auth value. */ public List auth() { @@ -61,7 +65,7 @@ public List auth() { /** * Set the auth property: Authentication secrets for the custom scale rule. - * + * * @param auth the auth value to set. * @return the HttpScaleRule object itself. */ @@ -72,7 +76,7 @@ public HttpScaleRule withAuth(List auth) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettings.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettings.java index a6b61f3c5d21f..59a984e74eb48 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettings.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettings.java @@ -32,14 +32,16 @@ public final class HttpSettings { @JsonProperty(value = "forwardProxy") private ForwardProxy forwardProxy; - /** Creates an instance of HttpSettings class. */ + /** + * Creates an instance of HttpSettings class. + */ public HttpSettings() { } /** * Get the requireHttps property: <code>false</code> if the authentication/authorization responses not * having the HTTPS scheme are permissible; otherwise, <code>true</code>. - * + * * @return the requireHttps value. */ public Boolean requireHttps() { @@ -49,7 +51,7 @@ public Boolean requireHttps() { /** * Set the requireHttps property: <code>false</code> if the authentication/authorization responses not * having the HTTPS scheme are permissible; otherwise, <code>true</code>. - * + * * @param requireHttps the requireHttps value to set. * @return the HttpSettings object itself. */ @@ -60,7 +62,7 @@ public HttpSettings withRequireHttps(Boolean requireHttps) { /** * Get the routes property: The configuration settings of the paths HTTP requests. - * + * * @return the routes value. */ public HttpSettingsRoutes routes() { @@ -69,7 +71,7 @@ public HttpSettingsRoutes routes() { /** * Set the routes property: The configuration settings of the paths HTTP requests. - * + * * @param routes the routes value to set. * @return the HttpSettings object itself. */ @@ -80,7 +82,7 @@ public HttpSettings withRoutes(HttpSettingsRoutes routes) { /** * Get the forwardProxy property: The configuration settings of a forward proxy used to make the requests. - * + * * @return the forwardProxy value. */ public ForwardProxy forwardProxy() { @@ -89,7 +91,7 @@ public ForwardProxy forwardProxy() { /** * Set the forwardProxy property: The configuration settings of a forward proxy used to make the requests. - * + * * @param forwardProxy the forwardProxy value to set. * @return the HttpSettings object itself. */ @@ -100,7 +102,7 @@ public HttpSettings withForwardProxy(ForwardProxy forwardProxy) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettingsRoutes.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettingsRoutes.java index 44e17ed42b3df..53d5558dba0f2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettingsRoutes.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/HttpSettingsRoutes.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the paths HTTP requests. */ +/** + * The configuration settings of the paths HTTP requests. + */ @Fluent public final class HttpSettingsRoutes { /* @@ -16,13 +18,15 @@ public final class HttpSettingsRoutes { @JsonProperty(value = "apiPrefix") private String apiPrefix; - /** Creates an instance of HttpSettingsRoutes class. */ + /** + * Creates an instance of HttpSettingsRoutes class. + */ public HttpSettingsRoutes() { } /** * Get the apiPrefix property: The prefix that should precede all the authentication/authorization paths. - * + * * @return the apiPrefix value. */ public String apiPrefix() { @@ -31,7 +35,7 @@ public String apiPrefix() { /** * Set the apiPrefix property: The prefix that should precede all the authentication/authorization paths. - * + * * @param apiPrefix the apiPrefix value to set. * @return the HttpSettingsRoutes object itself. */ @@ -42,7 +46,7 @@ public HttpSettingsRoutes withApiPrefix(String apiPrefix) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IdentityProviders.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IdentityProviders.java index 923d51db32d8f..7620168080379 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IdentityProviders.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IdentityProviders.java @@ -65,13 +65,15 @@ public final class IdentityProviders { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map customOpenIdConnectProviders; - /** Creates an instance of IdentityProviders class. */ + /** + * Creates an instance of IdentityProviders class. + */ public IdentityProviders() { } /** * Get the azureActiveDirectory property: The configuration settings of the Azure Active directory provider. - * + * * @return the azureActiveDirectory value. */ public AzureActiveDirectory azureActiveDirectory() { @@ -80,7 +82,7 @@ public AzureActiveDirectory azureActiveDirectory() { /** * Set the azureActiveDirectory property: The configuration settings of the Azure Active directory provider. - * + * * @param azureActiveDirectory the azureActiveDirectory value to set. * @return the IdentityProviders object itself. */ @@ -91,7 +93,7 @@ public IdentityProviders withAzureActiveDirectory(AzureActiveDirectory azureActi /** * Get the facebook property: The configuration settings of the Facebook provider. - * + * * @return the facebook value. */ public Facebook facebook() { @@ -100,7 +102,7 @@ public Facebook facebook() { /** * Set the facebook property: The configuration settings of the Facebook provider. - * + * * @param facebook the facebook value to set. * @return the IdentityProviders object itself. */ @@ -111,7 +113,7 @@ public IdentityProviders withFacebook(Facebook facebook) { /** * Get the gitHub property: The configuration settings of the GitHub provider. - * + * * @return the gitHub value. */ public GitHub gitHub() { @@ -120,7 +122,7 @@ public GitHub gitHub() { /** * Set the gitHub property: The configuration settings of the GitHub provider. - * + * * @param gitHub the gitHub value to set. * @return the IdentityProviders object itself. */ @@ -131,7 +133,7 @@ public IdentityProviders withGitHub(GitHub gitHub) { /** * Get the google property: The configuration settings of the Google provider. - * + * * @return the google value. */ public Google google() { @@ -140,7 +142,7 @@ public Google google() { /** * Set the google property: The configuration settings of the Google provider. - * + * * @param google the google value to set. * @return the IdentityProviders object itself. */ @@ -151,7 +153,7 @@ public IdentityProviders withGoogle(Google google) { /** * Get the twitter property: The configuration settings of the Twitter provider. - * + * * @return the twitter value. */ public Twitter twitter() { @@ -160,7 +162,7 @@ public Twitter twitter() { /** * Set the twitter property: The configuration settings of the Twitter provider. - * + * * @param twitter the twitter value to set. * @return the IdentityProviders object itself. */ @@ -171,7 +173,7 @@ public IdentityProviders withTwitter(Twitter twitter) { /** * Get the apple property: The configuration settings of the Apple provider. - * + * * @return the apple value. */ public Apple apple() { @@ -180,7 +182,7 @@ public Apple apple() { /** * Set the apple property: The configuration settings of the Apple provider. - * + * * @param apple the apple value to set. * @return the IdentityProviders object itself. */ @@ -191,7 +193,7 @@ public IdentityProviders withApple(Apple apple) { /** * Get the azureStaticWebApps property: The configuration settings of the Azure Static Web Apps provider. - * + * * @return the azureStaticWebApps value. */ public AzureStaticWebApps azureStaticWebApps() { @@ -200,7 +202,7 @@ public AzureStaticWebApps azureStaticWebApps() { /** * Set the azureStaticWebApps property: The configuration settings of the Azure Static Web Apps provider. - * + * * @param azureStaticWebApps the azureStaticWebApps value to set. * @return the IdentityProviders object itself. */ @@ -211,8 +213,9 @@ public IdentityProviders withAzureStaticWebApps(AzureStaticWebApps azureStaticWe /** * Get the customOpenIdConnectProviders property: The map of the name of the alias of each custom Open ID Connect - * provider to the configuration settings of the custom Open ID Connect provider. - * + * provider to the + * configuration settings of the custom Open ID Connect provider. + * * @return the customOpenIdConnectProviders value. */ public Map customOpenIdConnectProviders() { @@ -221,20 +224,21 @@ public Map customOpenIdConnectProviders() { /** * Set the customOpenIdConnectProviders property: The map of the name of the alias of each custom Open ID Connect - * provider to the configuration settings of the custom Open ID Connect provider. - * + * provider to the + * configuration settings of the custom Open ID Connect provider. + * * @param customOpenIdConnectProviders the customOpenIdConnectProviders value to set. * @return the IdentityProviders object itself. */ - public IdentityProviders withCustomOpenIdConnectProviders( - Map customOpenIdConnectProviders) { + public IdentityProviders + withCustomOpenIdConnectProviders(Map customOpenIdConnectProviders) { this.customOpenIdConnectProviders = customOpenIdConnectProviders; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -260,14 +264,11 @@ public void validate() { azureStaticWebApps().validate(); } if (customOpenIdConnectProviders() != null) { - customOpenIdConnectProviders() - .values() - .forEach( - e -> { - if (e != null) { - e.validate(); - } - }); + customOpenIdConnectProviders().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Ingress.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Ingress.java index affb86e66066c..115157470d1b9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Ingress.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Ingress.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App Ingress configuration. */ +/** + * Container App Ingress configuration. + */ @Fluent public final class Ingress { /* @@ -86,13 +88,27 @@ public final class Ingress { @JsonProperty(value = "corsPolicy") private CorsPolicy corsPolicy; - /** Creates an instance of Ingress class. */ + /* + * Settings to expose additional ports on container app + */ + @JsonProperty(value = "additionalPortMappings") + private List additionalPortMappings; + + /* + * Whether an http app listens on http or https + */ + @JsonProperty(value = "targetPortHttpScheme") + private IngressTargetPortHttpScheme targetPortHttpScheme; + + /** + * Creates an instance of Ingress class. + */ public Ingress() { } /** * Get the fqdn property: Hostname. - * + * * @return the fqdn value. */ public String fqdn() { @@ -101,7 +117,7 @@ public String fqdn() { /** * Get the external property: Bool indicating if app exposes an external http endpoint. - * + * * @return the external value. */ public Boolean external() { @@ -110,7 +126,7 @@ public Boolean external() { /** * Set the external property: Bool indicating if app exposes an external http endpoint. - * + * * @param external the external value to set. * @return the Ingress object itself. */ @@ -121,7 +137,7 @@ public Ingress withExternal(Boolean external) { /** * Get the targetPort property: Target Port in containers for traffic from ingress. - * + * * @return the targetPort value. */ public Integer targetPort() { @@ -130,7 +146,7 @@ public Integer targetPort() { /** * Set the targetPort property: Target Port in containers for traffic from ingress. - * + * * @param targetPort the targetPort value to set. * @return the Ingress object itself. */ @@ -141,7 +157,7 @@ public Ingress withTargetPort(Integer targetPort) { /** * Get the exposedPort property: Exposed Port in containers for TCP traffic from ingress. - * + * * @return the exposedPort value. */ public Integer exposedPort() { @@ -150,7 +166,7 @@ public Integer exposedPort() { /** * Set the exposedPort property: Exposed Port in containers for TCP traffic from ingress. - * + * * @param exposedPort the exposedPort value to set. * @return the Ingress object itself. */ @@ -161,7 +177,7 @@ public Ingress withExposedPort(Integer exposedPort) { /** * Get the transport property: Ingress transport protocol. - * + * * @return the transport value. */ public IngressTransportMethod transport() { @@ -170,7 +186,7 @@ public IngressTransportMethod transport() { /** * Set the transport property: Ingress transport protocol. - * + * * @param transport the transport value to set. * @return the Ingress object itself. */ @@ -181,7 +197,7 @@ public Ingress withTransport(IngressTransportMethod transport) { /** * Get the traffic property: Traffic weights for app's revisions. - * + * * @return the traffic value. */ public List traffic() { @@ -190,7 +206,7 @@ public List traffic() { /** * Set the traffic property: Traffic weights for app's revisions. - * + * * @param traffic the traffic value to set. * @return the Ingress object itself. */ @@ -201,7 +217,7 @@ public Ingress withTraffic(List traffic) { /** * Get the customDomains property: custom domain bindings for Container Apps' hostnames. - * + * * @return the customDomains value. */ public List customDomains() { @@ -210,7 +226,7 @@ public List customDomains() { /** * Set the customDomains property: custom domain bindings for Container Apps' hostnames. - * + * * @param customDomains the customDomains value to set. * @return the Ingress object itself. */ @@ -222,7 +238,7 @@ public Ingress withCustomDomains(List customDomains) { /** * Get the allowInsecure property: Bool indicating if HTTP connections to is allowed. If set to false HTTP * connections are automatically redirected to HTTPS connections. - * + * * @return the allowInsecure value. */ public Boolean allowInsecure() { @@ -232,7 +248,7 @@ public Boolean allowInsecure() { /** * Set the allowInsecure property: Bool indicating if HTTP connections to is allowed. If set to false HTTP * connections are automatically redirected to HTTPS connections. - * + * * @param allowInsecure the allowInsecure value to set. * @return the Ingress object itself. */ @@ -243,7 +259,7 @@ public Ingress withAllowInsecure(Boolean allowInsecure) { /** * Get the ipSecurityRestrictions property: Rules to restrict incoming IP address. - * + * * @return the ipSecurityRestrictions value. */ public List ipSecurityRestrictions() { @@ -252,7 +268,7 @@ public List ipSecurityRestrictions() { /** * Set the ipSecurityRestrictions property: Rules to restrict incoming IP address. - * + * * @param ipSecurityRestrictions the ipSecurityRestrictions value to set. * @return the Ingress object itself. */ @@ -263,7 +279,7 @@ public Ingress withIpSecurityRestrictions(List ipSecu /** * Get the stickySessions property: Sticky Sessions for Single Revision Mode. - * + * * @return the stickySessions value. */ public IngressStickySessions stickySessions() { @@ -272,7 +288,7 @@ public IngressStickySessions stickySessions() { /** * Set the stickySessions property: Sticky Sessions for Single Revision Mode. - * + * * @param stickySessions the stickySessions value to set. * @return the Ingress object itself. */ @@ -285,7 +301,7 @@ public Ingress withStickySessions(IngressStickySessions stickySessions) { * Get the clientCertificateMode property: Client certificate mode for mTLS authentication. Ignore indicates server * drops client certificate on forwarding. Accept indicates server forwards client certificate but does not require * a client certificate. Require indicates server requires a client certificate. - * + * * @return the clientCertificateMode value. */ public IngressClientCertificateMode clientCertificateMode() { @@ -296,7 +312,7 @@ public IngressClientCertificateMode clientCertificateMode() { * Set the clientCertificateMode property: Client certificate mode for mTLS authentication. Ignore indicates server * drops client certificate on forwarding. Accept indicates server forwards client certificate but does not require * a client certificate. Require indicates server requires a client certificate. - * + * * @param clientCertificateMode the clientCertificateMode value to set. * @return the Ingress object itself. */ @@ -307,7 +323,7 @@ public Ingress withClientCertificateMode(IngressClientCertificateMode clientCert /** * Get the corsPolicy property: CORS policy for container app. - * + * * @return the corsPolicy value. */ public CorsPolicy corsPolicy() { @@ -316,7 +332,7 @@ public CorsPolicy corsPolicy() { /** * Set the corsPolicy property: CORS policy for container app. - * + * * @param corsPolicy the corsPolicy value to set. * @return the Ingress object itself. */ @@ -325,9 +341,49 @@ public Ingress withCorsPolicy(CorsPolicy corsPolicy) { return this; } + /** + * Get the additionalPortMappings property: Settings to expose additional ports on container app. + * + * @return the additionalPortMappings value. + */ + public List additionalPortMappings() { + return this.additionalPortMappings; + } + + /** + * Set the additionalPortMappings property: Settings to expose additional ports on container app. + * + * @param additionalPortMappings the additionalPortMappings value to set. + * @return the Ingress object itself. + */ + public Ingress withAdditionalPortMappings(List additionalPortMappings) { + this.additionalPortMappings = additionalPortMappings; + return this; + } + + /** + * Get the targetPortHttpScheme property: Whether an http app listens on http or https. + * + * @return the targetPortHttpScheme value. + */ + public IngressTargetPortHttpScheme targetPortHttpScheme() { + return this.targetPortHttpScheme; + } + + /** + * Set the targetPortHttpScheme property: Whether an http app listens on http or https. + * + * @param targetPortHttpScheme the targetPortHttpScheme value to set. + * @return the Ingress object itself. + */ + public Ingress withTargetPortHttpScheme(IngressTargetPortHttpScheme targetPortHttpScheme) { + this.targetPortHttpScheme = targetPortHttpScheme; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -346,5 +402,8 @@ public void validate() { if (corsPolicy() != null) { corsPolicy().validate(); } + if (additionalPortMappings() != null) { + additionalPortMappings().forEach(e -> e.validate()); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressClientCertificateMode.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressClientCertificateMode.java index a30b69413bb61..2600873124dff 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressClientCertificateMode.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressClientCertificateMode.java @@ -14,18 +14,24 @@ * server requires a client certificate. */ public final class IngressClientCertificateMode extends ExpandableStringEnum { - /** Static value ignore for IngressClientCertificateMode. */ + /** + * Static value ignore for IngressClientCertificateMode. + */ public static final IngressClientCertificateMode IGNORE = fromString("ignore"); - /** Static value accept for IngressClientCertificateMode. */ + /** + * Static value accept for IngressClientCertificateMode. + */ public static final IngressClientCertificateMode ACCEPT = fromString("accept"); - /** Static value require for IngressClientCertificateMode. */ + /** + * Static value require for IngressClientCertificateMode. + */ public static final IngressClientCertificateMode REQUIRE = fromString("require"); /** * Creates a new instance of IngressClientCertificateMode value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -34,7 +40,7 @@ public IngressClientCertificateMode() { /** * Creates or finds a IngressClientCertificateMode from its string representation. - * + * * @param name a name to look for. * @return the corresponding IngressClientCertificateMode. */ @@ -45,7 +51,7 @@ public static IngressClientCertificateMode fromString(String name) { /** * Gets known IngressClientCertificateMode values. - * + * * @return known IngressClientCertificateMode values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressPortMapping.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressPortMapping.java new file mode 100644 index 0000000000000..204a36953b79a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressPortMapping.java @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Port mappings of container app ingress. + */ +@Fluent +public final class IngressPortMapping { + /* + * Specifies whether the app port is accessible outside of the environment + */ + @JsonProperty(value = "external", required = true) + private boolean external; + + /* + * Specifies the port user's container listens on + */ + @JsonProperty(value = "targetPort", required = true) + private int targetPort; + + /* + * Specifies the exposed port for the target port. If not specified, it defaults to target port + */ + @JsonProperty(value = "exposedPort") + private Integer exposedPort; + + /** + * Creates an instance of IngressPortMapping class. + */ + public IngressPortMapping() { + } + + /** + * Get the external property: Specifies whether the app port is accessible outside of the environment. + * + * @return the external value. + */ + public boolean external() { + return this.external; + } + + /** + * Set the external property: Specifies whether the app port is accessible outside of the environment. + * + * @param external the external value to set. + * @return the IngressPortMapping object itself. + */ + public IngressPortMapping withExternal(boolean external) { + this.external = external; + return this; + } + + /** + * Get the targetPort property: Specifies the port user's container listens on. + * + * @return the targetPort value. + */ + public int targetPort() { + return this.targetPort; + } + + /** + * Set the targetPort property: Specifies the port user's container listens on. + * + * @param targetPort the targetPort value to set. + * @return the IngressPortMapping object itself. + */ + public IngressPortMapping withTargetPort(int targetPort) { + this.targetPort = targetPort; + return this; + } + + /** + * Get the exposedPort property: Specifies the exposed port for the target port. If not specified, it defaults to + * target port. + * + * @return the exposedPort value. + */ + public Integer exposedPort() { + return this.exposedPort; + } + + /** + * Set the exposedPort property: Specifies the exposed port for the target port. If not specified, it defaults to + * target port. + * + * @param exposedPort the exposedPort value to set. + * @return the IngressPortMapping object itself. + */ + public IngressPortMapping withExposedPort(Integer exposedPort) { + this.exposedPort = exposedPort; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressStickySessions.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressStickySessions.java index 8c4de6b75dfa1..b6f60418efd70 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressStickySessions.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressStickySessions.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Sticky Sessions for Single Revision Mode. */ +/** + * Sticky Sessions for Single Revision Mode. + */ @Fluent public final class IngressStickySessions { /* @@ -16,13 +18,15 @@ public final class IngressStickySessions { @JsonProperty(value = "affinity") private Affinity affinity; - /** Creates an instance of IngressStickySessions class. */ + /** + * Creates an instance of IngressStickySessions class. + */ public IngressStickySessions() { } /** * Get the affinity property: Sticky Session Affinity. - * + * * @return the affinity value. */ public Affinity affinity() { @@ -31,7 +35,7 @@ public Affinity affinity() { /** * Set the affinity property: Sticky Session Affinity. - * + * * @param affinity the affinity value to set. * @return the IngressStickySessions object itself. */ @@ -42,7 +46,7 @@ public IngressStickySessions withAffinity(Affinity affinity) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTargetPortHttpScheme.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTargetPortHttpScheme.java new file mode 100644 index 0000000000000..479288c1707bd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTargetPortHttpScheme.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Whether an http app listens on http or https. + */ +public final class IngressTargetPortHttpScheme extends ExpandableStringEnum { + /** + * Static value http for IngressTargetPortHttpScheme. + */ + public static final IngressTargetPortHttpScheme HTTP = fromString("http"); + + /** + * Static value https for IngressTargetPortHttpScheme. + */ + public static final IngressTargetPortHttpScheme HTTPS = fromString("https"); + + /** + * Creates a new instance of IngressTargetPortHttpScheme value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public IngressTargetPortHttpScheme() { + } + + /** + * Creates or finds a IngressTargetPortHttpScheme from its string representation. + * + * @param name a name to look for. + * @return the corresponding IngressTargetPortHttpScheme. + */ + @JsonCreator + public static IngressTargetPortHttpScheme fromString(String name) { + return fromString(name, IngressTargetPortHttpScheme.class); + } + + /** + * Gets known IngressTargetPortHttpScheme values. + * + * @return known IngressTargetPortHttpScheme values. + */ + public static Collection values() { + return values(IngressTargetPortHttpScheme.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTransportMethod.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTransportMethod.java index 15eab472d3d20..e7f0f64fc6a03 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTransportMethod.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IngressTransportMethod.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Ingress transport protocol. */ +/** + * Ingress transport protocol. + */ public final class IngressTransportMethod extends ExpandableStringEnum { - /** Static value auto for IngressTransportMethod. */ + /** + * Static value auto for IngressTransportMethod. + */ public static final IngressTransportMethod AUTO = fromString("auto"); - /** Static value http for IngressTransportMethod. */ + /** + * Static value http for IngressTransportMethod. + */ public static final IngressTransportMethod HTTP = fromString("http"); - /** Static value http2 for IngressTransportMethod. */ + /** + * Static value http2 for IngressTransportMethod. + */ public static final IngressTransportMethod HTTP2 = fromString("http2"); - /** Static value tcp for IngressTransportMethod. */ + /** + * Static value tcp for IngressTransportMethod. + */ public static final IngressTransportMethod TCP = fromString("tcp"); /** * Creates a new instance of IngressTransportMethod value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public IngressTransportMethod() { /** * Creates or finds a IngressTransportMethod from its string representation. - * + * * @param name a name to look for. * @return the corresponding IngressTransportMethod. */ @@ -44,7 +54,7 @@ public static IngressTransportMethod fromString(String name) { /** * Gets known IngressTransportMethod values. - * + * * @return known IngressTransportMethod values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/InitContainer.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/InitContainer.java index f1bee6364d895..dc57c485c5b8d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/InitContainer.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/InitContainer.java @@ -7,56 +7,74 @@ import com.azure.core.annotation.Fluent; import java.util.List; -/** Container App init container definition. */ +/** + * Container App init container definition. + */ @Fluent public final class InitContainer extends BaseContainer { - /** Creates an instance of InitContainer class. */ + /** + * Creates an instance of InitContainer class. + */ public InitContainer() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withImage(String image) { super.withImage(image); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withName(String name) { super.withName(name); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withCommand(List command) { super.withCommand(command); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withArgs(List args) { super.withArgs(args); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withEnv(List env) { super.withEnv(env); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withResources(ContainerResources resources) { super.withResources(resources); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public InitContainer withVolumeMounts(List volumeMounts) { super.withVolumeMounts(volumeMounts); @@ -65,7 +83,7 @@ public InitContainer withVolumeMounts(List volumeMounts) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IpSecurityRestrictionRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IpSecurityRestrictionRule.java index 4c27320473b0f..78fa22df8a373 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IpSecurityRestrictionRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/IpSecurityRestrictionRule.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Rule to restrict incoming IP address. */ +/** + * Rule to restrict incoming IP address. + */ @Fluent public final class IpSecurityRestrictionRule { /* @@ -35,13 +37,15 @@ public final class IpSecurityRestrictionRule { @JsonProperty(value = "action", required = true) private Action action; - /** Creates an instance of IpSecurityRestrictionRule class. */ + /** + * Creates an instance of IpSecurityRestrictionRule class. + */ public IpSecurityRestrictionRule() { } /** * Get the name property: Name for the IP restriction rule. - * + * * @return the name value. */ public String name() { @@ -50,7 +54,7 @@ public String name() { /** * Set the name property: Name for the IP restriction rule. - * + * * @param name the name value to set. * @return the IpSecurityRestrictionRule object itself. */ @@ -62,7 +66,7 @@ public IpSecurityRestrictionRule withName(String name) { /** * Get the description property: Describe the IP restriction rule that is being sent to the container-app. This is * an optional field. - * + * * @return the description value. */ public String description() { @@ -72,7 +76,7 @@ public String description() { /** * Set the description property: Describe the IP restriction rule that is being sent to the container-app. This is * an optional field. - * + * * @param description the description value to set. * @return the IpSecurityRestrictionRule object itself. */ @@ -83,7 +87,7 @@ public IpSecurityRestrictionRule withDescription(String description) { /** * Get the ipAddressRange property: CIDR notation to match incoming IP address. - * + * * @return the ipAddressRange value. */ public String ipAddressRange() { @@ -92,7 +96,7 @@ public String ipAddressRange() { /** * Set the ipAddressRange property: CIDR notation to match incoming IP address. - * + * * @param ipAddressRange the ipAddressRange value to set. * @return the IpSecurityRestrictionRule object itself. */ @@ -104,7 +108,7 @@ public IpSecurityRestrictionRule withIpAddressRange(String ipAddressRange) { /** * Get the action property: Allow or Deny rules to determine for incoming IP. Note: Rules can only consist of ALL * Allow or ALL Deny. - * + * * @return the action value. */ public Action action() { @@ -114,7 +118,7 @@ public Action action() { /** * Set the action property: Allow or Deny rules to determine for incoming IP. Note: Rules can only consist of ALL * Allow or ALL Deny. - * + * * @param action the action value to set. * @return the IpSecurityRestrictionRule object itself. */ @@ -125,26 +129,21 @@ public IpSecurityRestrictionRule withAction(Action action) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model IpSecurityRestrictionRule")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model IpSecurityRestrictionRule")); } if (ipAddressRange() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property ipAddressRange in model IpSecurityRestrictionRule")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property ipAddressRange in model IpSecurityRestrictionRule")); } if (action() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property action in model IpSecurityRestrictionRule")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property action in model IpSecurityRestrictionRule")); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponent.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponent.java new file mode 100644 index 0000000000000..bf42b2379e15a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponent.java @@ -0,0 +1,265 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import java.util.List; + +/** + * An immutable client-side representation of JavaComponent. + */ +public interface JavaComponent { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the componentType property: Type of the Java Component. + * + * @return the componentType value. + */ + JavaComponentType componentType(); + + /** + * Gets the provisioningState property: Provisioning state of the Java Component. + * + * @return the provisioningState value. + */ + JavaComponentProvisioningState provisioningState(); + + /** + * Gets the configurations property: List of Java Components configuration properties. + * + * @return the configurations value. + */ + List configurations(); + + /** + * Gets the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @return the serviceBinds value. + */ + List serviceBinds(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner object. + * + * @return the inner object. + */ + JavaComponentInner innerModel(); + + /** + * The entirety of the JavaComponent definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The JavaComponent definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the JavaComponent definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the JavaComponent definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, environmentName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Managed Environment. + * @return the next definition stage. + */ + WithCreate withExistingManagedEnvironment(String resourceGroupName, String environmentName); + } + + /** + * The stage of the JavaComponent definition which contains all the minimum required properties for the resource + * to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithComponentType, DefinitionStages.WithConfigurations, + DefinitionStages.WithServiceBinds { + /** + * Executes the create request. + * + * @return the created resource. + */ + JavaComponent create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + JavaComponent create(Context context); + } + + /** + * The stage of the JavaComponent definition allowing to specify componentType. + */ + interface WithComponentType { + /** + * Specifies the componentType property: Type of the Java Component.. + * + * @param componentType Type of the Java Component. + * @return the next definition stage. + */ + WithCreate withComponentType(JavaComponentType componentType); + } + + /** + * The stage of the JavaComponent definition allowing to specify configurations. + */ + interface WithConfigurations { + /** + * Specifies the configurations property: List of Java Components configuration properties. + * + * @param configurations List of Java Components configuration properties. + * @return the next definition stage. + */ + WithCreate withConfigurations(List configurations); + } + + /** + * The stage of the JavaComponent definition allowing to specify serviceBinds. + */ + interface WithServiceBinds { + /** + * Specifies the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @param serviceBinds List of Java Components that are bound to the Java component. + * @return the next definition stage. + */ + WithCreate withServiceBinds(List serviceBinds); + } + } + + /** + * Begins update for the JavaComponent resource. + * + * @return the stage of resource update. + */ + JavaComponent.Update update(); + + /** + * The template for JavaComponent update. + */ + interface Update + extends UpdateStages.WithComponentType, UpdateStages.WithConfigurations, UpdateStages.WithServiceBinds { + /** + * Executes the update request. + * + * @return the updated resource. + */ + JavaComponent apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + JavaComponent apply(Context context); + } + + /** + * The JavaComponent update stages. + */ + interface UpdateStages { + /** + * The stage of the JavaComponent update allowing to specify componentType. + */ + interface WithComponentType { + /** + * Specifies the componentType property: Type of the Java Component.. + * + * @param componentType Type of the Java Component. + * @return the next definition stage. + */ + Update withComponentType(JavaComponentType componentType); + } + + /** + * The stage of the JavaComponent update allowing to specify configurations. + */ + interface WithConfigurations { + /** + * Specifies the configurations property: List of Java Components configuration properties. + * + * @param configurations List of Java Components configuration properties. + * @return the next definition stage. + */ + Update withConfigurations(List configurations); + } + + /** + * The stage of the JavaComponent update allowing to specify serviceBinds. + */ + interface WithServiceBinds { + /** + * Specifies the serviceBinds property: List of Java Components that are bound to the Java component. + * + * @param serviceBinds List of Java Components that are bound to the Java component. + * @return the next definition stage. + */ + Update withServiceBinds(List serviceBinds); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + JavaComponent refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + JavaComponent refresh(Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentConfigurationProperty.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentConfigurationProperty.java new file mode 100644 index 0000000000000..03843583c021c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentConfigurationProperty.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration properties for a Java Component. + */ +@Fluent +public final class JavaComponentConfigurationProperty { + /* + * The name of the property + */ + @JsonProperty(value = "propertyName") + private String propertyName; + + /* + * The value of the property + */ + @JsonProperty(value = "value") + private String value; + + /** + * Creates an instance of JavaComponentConfigurationProperty class. + */ + public JavaComponentConfigurationProperty() { + } + + /** + * Get the propertyName property: The name of the property. + * + * @return the propertyName value. + */ + public String propertyName() { + return this.propertyName; + } + + /** + * Set the propertyName property: The name of the property. + * + * @param propertyName the propertyName value to set. + * @return the JavaComponentConfigurationProperty object itself. + */ + public JavaComponentConfigurationProperty withPropertyName(String propertyName) { + this.propertyName = propertyName; + return this; + } + + /** + * Get the value property: The value of the property. + * + * @return the value value. + */ + public String value() { + return this.value; + } + + /** + * Set the value property: The value of the property. + * + * @param value the value value to set. + * @return the JavaComponentConfigurationProperty object itself. + */ + public JavaComponentConfigurationProperty withValue(String value) { + this.value = value; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentProvisioningState.java new file mode 100644 index 0000000000000..40caabccda700 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentProvisioningState.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Provisioning state of the Java Component. + */ +public final class JavaComponentProvisioningState extends ExpandableStringEnum { + /** + * Static value Succeeded for JavaComponentProvisioningState. + */ + public static final JavaComponentProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Failed for JavaComponentProvisioningState. + */ + public static final JavaComponentProvisioningState FAILED = fromString("Failed"); + + /** + * Static value Canceled for JavaComponentProvisioningState. + */ + public static final JavaComponentProvisioningState CANCELED = fromString("Canceled"); + + /** + * Static value Deleting for JavaComponentProvisioningState. + */ + public static final JavaComponentProvisioningState DELETING = fromString("Deleting"); + + /** + * Static value InProgress for JavaComponentProvisioningState. + */ + public static final JavaComponentProvisioningState IN_PROGRESS = fromString("InProgress"); + + /** + * Creates a new instance of JavaComponentProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public JavaComponentProvisioningState() { + } + + /** + * Creates or finds a JavaComponentProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding JavaComponentProvisioningState. + */ + @JsonCreator + public static JavaComponentProvisioningState fromString(String name) { + return fromString(name, JavaComponentProvisioningState.class); + } + + /** + * Gets known JavaComponentProvisioningState values. + * + * @return known JavaComponentProvisioningState values. + */ + public static Collection values() { + return values(JavaComponentProvisioningState.class); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentServiceBind.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentServiceBind.java new file mode 100644 index 0000000000000..382c10132618b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentServiceBind.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration to bind a Java Component to another Java Component. + */ +@Fluent +public final class JavaComponentServiceBind { + /* + * Name of the service bind + */ + @JsonProperty(value = "name") + private String name; + + /* + * Resource id of the target service + */ + @JsonProperty(value = "serviceId") + private String serviceId; + + /** + * Creates an instance of JavaComponentServiceBind class. + */ + public JavaComponentServiceBind() { + } + + /** + * Get the name property: Name of the service bind. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the service bind. + * + * @param name the name value to set. + * @return the JavaComponentServiceBind object itself. + */ + public JavaComponentServiceBind withName(String name) { + this.name = name; + return this; + } + + /** + * Get the serviceId property: Resource id of the target service. + * + * @return the serviceId value. + */ + public String serviceId() { + return this.serviceId; + } + + /** + * Set the serviceId property: Resource id of the target service. + * + * @param serviceId the serviceId value to set. + * @return the JavaComponentServiceBind object itself. + */ + public JavaComponentServiceBind withServiceId(String serviceId) { + this.serviceId = serviceId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentType.java new file mode 100644 index 0000000000000..60c54e06369d9 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentType.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Type of the Java Component. + */ +public final class JavaComponentType extends ExpandableStringEnum { + /** + * Static value SpringBootAdmin for JavaComponentType. + */ + public static final JavaComponentType SPRING_BOOT_ADMIN = fromString("SpringBootAdmin"); + + /** + * Static value SpringCloudEureka for JavaComponentType. + */ + public static final JavaComponentType SPRING_CLOUD_EUREKA = fromString("SpringCloudEureka"); + + /** + * Static value SpringCloudConfig for JavaComponentType. + */ + public static final JavaComponentType SPRING_CLOUD_CONFIG = fromString("SpringCloudConfig"); + + /** + * Creates a new instance of JavaComponentType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public JavaComponentType() { + } + + /** + * Creates or finds a JavaComponentType from its string representation. + * + * @param name a name to look for. + * @return the corresponding JavaComponentType. + */ + @JsonCreator + public static JavaComponentType fromString(String name) { + return fromString(name, JavaComponentType.class); + } + + /** + * Gets known JavaComponentType values. + * + * @return known JavaComponentType values. + */ + public static Collection values() { + return values(JavaComponentType.class); + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddresses.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponents.java similarity index 51% rename from sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddresses.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponents.java index 141fa327d1f0a..3f27736d5a732 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddresses.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponents.java @@ -2,155 +2,119 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.models; +package com.azure.resourcemanager.appcontainers.models; import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; /** - * Resource collection API of SuppressionListAddresses. + * Resource collection API of JavaComponents. */ -public interface SuppressionListAddresses { +public interface JavaComponents { /** - * Get - * - * Get all the addresses in a suppression list. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. + * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedIterable}. + * @return the Java Components for a managed environment as paginated response with {@link PagedIterable}. */ - PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName); + PagedIterable list(String resourceGroupName, String environmentName); /** - * Get - * - * Get all the addresses in a suppression list. + * Get the Java Components for a managed environment. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. + * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedIterable}. + * @return the Java Components for a managed environment as paginated response with {@link PagedIterable}. */ - PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context); + PagedIterable list(String resourceGroupName, String environmentName, Context context); /** - * Get - * - * Get a SuppressionListAddress. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response}. + * @return a Java Component along with {@link Response}. */ - Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, String name, + Context context); /** - * Get - * - * Get a SuppressionListAddress. + * Get a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress. + * @return a Java Component. */ - SuppressionListAddressResource get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId); + JavaComponent get(String resourceGroupName, String environmentName, String name); /** - * Delete - * - * Operation to delete a single address from a suppression list. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. */ - Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId, Context context); + void delete(String resourceGroupName, String environmentName, String name); /** - * Delete - * - * Operation to delete a single address from a suppression list. + * Delete a Java Component. * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. + * @param environmentName Name of the Managed Environment. + * @param name Name of the Java Component. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete(String resourceGroupName, String emailServiceName, String domainName, String suppressionListName, - String addressId); + void delete(String resourceGroupName, String environmentName, String name, Context context); /** - * Get - * - * Get a SuppressionListAddress. + * Get a Java Component. * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response}. + * @return a Java Component along with {@link Response}. */ - SuppressionListAddressResource getById(String id); + JavaComponent getById(String id); /** - * Get - * - * Get a SuppressionListAddress. + * Get a Java Component. * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response}. + * @return a Java Component along with {@link Response}. */ - Response getByIdWithResponse(String id, Context context); + Response getByIdWithResponse(String id, Context context); /** - * Delete - * - * Operation to delete a single address from a suppression list. + * Delete a Java Component. * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -160,24 +124,21 @@ void delete(String resourceGroupName, String emailServiceName, String domainName void deleteById(String id); /** - * Delete - * - * Operation to delete a single address from a suppression list. + * Delete a Java Component. * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. */ - Response deleteByIdWithResponse(String id, Context context); + void deleteByIdWithResponse(String id, Context context); /** - * Begins definition for a new SuppressionListAddressResource resource. + * Begins definition for a new JavaComponent resource. * * @param name resource name. - * @return the first stage of the new SuppressionListAddressResource definition. + * @return the first stage of the new JavaComponent definition. */ - SuppressionListAddressResource.DefinitionStages.Blank define(String name); + JavaComponent.DefinitionStages.Blank define(String name); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentsCollection.java new file mode 100644 index 0000000000000..595151e50511f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JavaComponentsCollection.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Java Components ARM resource. + */ +@Fluent +public final class JavaComponentsCollection { + /* + * Collection of resources. + */ + @JsonProperty(value = "value", required = true) + private List value; + + /* + * Link to next page of resources. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of JavaComponentsCollection class. + */ + public JavaComponentsCollection() { + } + + /** + * Get the value property: Collection of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Collection of resources. + * + * @param value the value value to set. + * @return the JavaComponentsCollection object itself. + */ + public JavaComponentsCollection withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model JavaComponentsCollection")); + } else { + value().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(JavaComponentsCollection.class); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Job.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Job.java index 1f8e936f0514e..1834712523b23 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Job.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Job.java @@ -12,154 +12,168 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Job. */ +/** + * An immutable client-side representation of Job. + */ public interface Job { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); + /** + * Gets the extendedLocation property: The complex type of the extended location. + * + * @return the extendedLocation value. + */ + ExtendedLocation extendedLocation(); + /** * Gets the identity property: Managed identities needed by a container app job to interact with other Azure * services to not maintain any secrets or credentials in code. - * + * * @return the identity value. */ ManagedServiceIdentity identity(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning state of the Container Apps Job. - * + * * @return the provisioningState value. */ JobProvisioningState provisioningState(); /** * Gets the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ String environmentId(); /** * Gets the workloadProfileName property: Workload profile name to pin for container apps job execution. - * + * * @return the workloadProfileName value. */ String workloadProfileName(); /** * Gets the configuration property: Container Apps Job configuration properties. - * + * * @return the configuration value. */ JobConfiguration configuration(); /** * Gets the template property: Container Apps job definition. - * + * * @return the template value. */ JobTemplate template(); /** * Gets the outboundIpAddresses property: Outbound IP Addresses of a container apps job. - * + * * @return the outboundIpAddresses value. */ List outboundIpAddresses(); /** * Gets the eventStreamEndpoint property: The endpoint of the eventstream of the container apps job. - * + * * @return the eventStreamEndpoint value. */ String eventStreamEndpoint(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.JobInner object. - * + * * @return the inner object. */ JobInner innerModel(); - /** The entirety of the Job definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the Job definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The Job definition stages. */ + /** + * The Job definition stages. + */ interface DefinitionStages { - /** The first stage of the Job definition. */ + /** + * The first stage of the Job definition. + */ interface Blank extends WithLocation { } - /** The stage of the Job definition allowing to specify location. */ + /** + * The stage of the Job definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -167,18 +181,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Job definition allowing to specify parent resource. */ + /** + * The stage of the Job definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -189,92 +205,113 @@ interface WithResourceGroup { * The stage of the Job definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithIdentity, - DefinitionStages.WithEnvironmentId, - DefinitionStages.WithWorkloadProfileName, - DefinitionStages.WithConfiguration, - DefinitionStages.WithTemplate { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithExtendedLocation, + DefinitionStages.WithIdentity, DefinitionStages.WithEnvironmentId, DefinitionStages.WithWorkloadProfileName, + DefinitionStages.WithConfiguration, DefinitionStages.WithTemplate { /** * Executes the create request. - * + * * @return the created resource. */ Job create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Job create(Context context); } - /** The stage of the Job definition allowing to specify tags. */ + /** + * The stage of the Job definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Job definition allowing to specify identity. */ + /** + * The stage of the Job definition allowing to specify extendedLocation. + */ + interface WithExtendedLocation { + /** + * Specifies the extendedLocation property: The complex type of the extended location.. + * + * @param extendedLocation The complex type of the extended location. + * @return the next definition stage. + */ + WithCreate withExtendedLocation(ExtendedLocation extendedLocation); + } + + /** + * The stage of the Job definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: Managed identities needed by a container app job to interact with other * Azure services to not maintain any secrets or credentials in code.. - * + * * @param identity Managed identities needed by a container app job to interact with other Azure services to - * not maintain any secrets or credentials in code. + * not maintain any secrets or credentials in code. * @return the next definition stage. */ WithCreate withIdentity(ManagedServiceIdentity identity); } - /** The stage of the Job definition allowing to specify environmentId. */ + /** + * The stage of the Job definition allowing to specify environmentId. + */ interface WithEnvironmentId { /** * Specifies the environmentId property: Resource ID of environment.. - * + * * @param environmentId Resource ID of environment. * @return the next definition stage. */ WithCreate withEnvironmentId(String environmentId); } - /** The stage of the Job definition allowing to specify workloadProfileName. */ + /** + * The stage of the Job definition allowing to specify workloadProfileName. + */ interface WithWorkloadProfileName { /** * Specifies the workloadProfileName property: Workload profile name to pin for container apps job * execution.. - * + * * @param workloadProfileName Workload profile name to pin for container apps job execution. * @return the next definition stage. */ WithCreate withWorkloadProfileName(String workloadProfileName); } - /** The stage of the Job definition allowing to specify configuration. */ + /** + * The stage of the Job definition allowing to specify configuration. + */ interface WithConfiguration { /** * Specifies the configuration property: Container Apps Job configuration properties.. - * + * * @param configuration Container Apps Job configuration properties. * @return the next definition stage. */ WithCreate withConfiguration(JobConfiguration configuration); } - /** The stage of the Job definition allowing to specify template. */ + /** + * The stage of the Job definition allowing to specify template. + */ interface WithTemplate { /** * Specifies the template property: Container Apps job definition.. - * + * * @param template Container Apps job definition. * @return the next definition stage. */ @@ -284,60 +321,84 @@ interface WithTemplate { /** * Begins update for the Job resource. - * + * * @return the stage of resource update. */ Job.Update update(); - /** The template for Job update. */ - interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity, UpdateStages.WithProperties { + /** + * The template for Job update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithExtendedLocation, UpdateStages.WithIdentity, + UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ Job apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Job apply(Context context); } - /** The Job update stages. */ + /** + * The Job update stages. + */ interface UpdateStages { - /** The stage of the Job update allowing to specify tags. */ + /** + * The stage of the Job update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the Job update allowing to specify identity. */ + /** + * The stage of the Job update allowing to specify extendedLocation. + */ + interface WithExtendedLocation { + /** + * Specifies the extendedLocation property: The complex type of the extended location.. + * + * @param extendedLocation The complex type of the extended location. + * @return the next definition stage. + */ + Update withExtendedLocation(ExtendedLocation extendedLocation); + } + + /** + * The stage of the Job update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: Managed identities needed by a container app job to interact with other * Azure services to not maintain any secrets or credentials in code.. - * + * * @param identity Managed identities needed by a container app job to interact with other Azure services to - * not maintain any secrets or credentials in code. + * not maintain any secrets or credentials in code. * @return the next definition stage. */ Update withIdentity(ManagedServiceIdentity identity); } - /** The stage of the Job update allowing to specify properties. */ + /** + * The stage of the Job update allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: The properties property.. - * + * * @param properties The properties property. * @return the next definition stage. */ @@ -347,14 +408,14 @@ interface WithProperties { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Job refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -362,9 +423,9 @@ interface WithProperties { /** * Start a Container Apps Job. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App's Job execution name. */ @@ -372,12 +433,12 @@ interface WithProperties { /** * Start a Container Apps Job. - * + * * @param template Properties used to start a job execution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App's Job execution name. */ @@ -385,9 +446,9 @@ interface WithProperties { /** * Terminates execution of a running container apps job. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource. */ @@ -395,11 +456,11 @@ interface WithProperties { /** * Terminates execution of a running container apps job. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource. */ @@ -407,11 +468,11 @@ interface WithProperties { /** * List secrets for a container apps job. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource along with {@link Response}. */ @@ -419,9 +480,9 @@ interface WithProperties { /** * List secrets for a container apps job. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfiguration.java index 647c2dadb87e7..12c36c4401d1a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Non versioned Container Apps Job configuration properties. */ +/** + * Non versioned Container Apps Job configuration properties. + */ @Fluent public final class JobConfiguration { /* @@ -62,13 +64,15 @@ public final class JobConfiguration { @JsonProperty(value = "registries") private List registries; - /** Creates an instance of JobConfiguration class. */ + /** + * Creates an instance of JobConfiguration class. + */ public JobConfiguration() { } /** * Get the secrets property: Collection of secrets used by a Container Apps Job. - * + * * @return the secrets value. */ public List secrets() { @@ -77,7 +81,7 @@ public List secrets() { /** * Set the secrets property: Collection of secrets used by a Container Apps Job. - * + * * @param secrets the secrets value to set. * @return the JobConfiguration object itself. */ @@ -88,7 +92,7 @@ public JobConfiguration withSecrets(List secrets) { /** * Get the triggerType property: Trigger type of the job. - * + * * @return the triggerType value. */ public TriggerType triggerType() { @@ -97,7 +101,7 @@ public TriggerType triggerType() { /** * Set the triggerType property: Trigger type of the job. - * + * * @param triggerType the triggerType value to set. * @return the JobConfiguration object itself. */ @@ -108,7 +112,7 @@ public JobConfiguration withTriggerType(TriggerType triggerType) { /** * Get the replicaTimeout property: Maximum number of seconds a replica is allowed to run. - * + * * @return the replicaTimeout value. */ public int replicaTimeout() { @@ -117,7 +121,7 @@ public int replicaTimeout() { /** * Set the replicaTimeout property: Maximum number of seconds a replica is allowed to run. - * + * * @param replicaTimeout the replicaTimeout value to set. * @return the JobConfiguration object itself. */ @@ -128,7 +132,7 @@ public JobConfiguration withReplicaTimeout(int replicaTimeout) { /** * Get the replicaRetryLimit property: Maximum number of retries before failing the job. - * + * * @return the replicaRetryLimit value. */ public Integer replicaRetryLimit() { @@ -137,7 +141,7 @@ public Integer replicaRetryLimit() { /** * Set the replicaRetryLimit property: Maximum number of retries before failing the job. - * + * * @param replicaRetryLimit the replicaRetryLimit value to set. * @return the JobConfiguration object itself. */ @@ -149,7 +153,7 @@ public JobConfiguration withReplicaRetryLimit(Integer replicaRetryLimit) { /** * Get the manualTriggerConfig property: Manual trigger configuration for a single execution job. Properties * replicaCompletionCount and parallelism would be set to 1 by default. - * + * * @return the manualTriggerConfig value. */ public JobConfigurationManualTriggerConfig manualTriggerConfig() { @@ -159,7 +163,7 @@ public JobConfigurationManualTriggerConfig manualTriggerConfig() { /** * Set the manualTriggerConfig property: Manual trigger configuration for a single execution job. Properties * replicaCompletionCount and parallelism would be set to 1 by default. - * + * * @param manualTriggerConfig the manualTriggerConfig value to set. * @return the JobConfiguration object itself. */ @@ -171,7 +175,7 @@ public JobConfiguration withManualTriggerConfig(JobConfigurationManualTriggerCon /** * Get the scheduleTriggerConfig property: Cron formatted repeating trigger schedule ("* * * * *") for cronjobs. * Properties completions and parallelism would be set to 1 by default. - * + * * @return the scheduleTriggerConfig value. */ public JobConfigurationScheduleTriggerConfig scheduleTriggerConfig() { @@ -181,7 +185,7 @@ public JobConfigurationScheduleTriggerConfig scheduleTriggerConfig() { /** * Set the scheduleTriggerConfig property: Cron formatted repeating trigger schedule ("* * * * *") for cronjobs. * Properties completions and parallelism would be set to 1 by default. - * + * * @param scheduleTriggerConfig the scheduleTriggerConfig value to set. * @return the JobConfiguration object itself. */ @@ -192,7 +196,7 @@ public JobConfiguration withScheduleTriggerConfig(JobConfigurationScheduleTrigge /** * Get the eventTriggerConfig property: Trigger configuration of an event driven job. - * + * * @return the eventTriggerConfig value. */ public JobConfigurationEventTriggerConfig eventTriggerConfig() { @@ -201,7 +205,7 @@ public JobConfigurationEventTriggerConfig eventTriggerConfig() { /** * Set the eventTriggerConfig property: Trigger configuration of an event driven job. - * + * * @param eventTriggerConfig the eventTriggerConfig value to set. * @return the JobConfiguration object itself. */ @@ -212,7 +216,7 @@ public JobConfiguration withEventTriggerConfig(JobConfigurationEventTriggerConfi /** * Get the registries property: Collection of private container registry credentials used by a Container apps job. - * + * * @return the registries value. */ public List registries() { @@ -221,7 +225,7 @@ public List registries() { /** * Set the registries property: Collection of private container registry credentials used by a Container apps job. - * + * * @param registries the registries value to set. * @return the JobConfiguration object itself. */ @@ -232,7 +236,7 @@ public JobConfiguration withRegistries(List registries) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -240,9 +244,8 @@ public void validate() { secrets().forEach(e -> e.validate()); } if (triggerType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property triggerType in model JobConfiguration")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property triggerType in model JobConfiguration")); } if (manualTriggerConfig() != null) { manualTriggerConfig().validate(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationEventTriggerConfig.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationEventTriggerConfig.java index d2a7e3053a938..fb10042c2e63b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationEventTriggerConfig.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationEventTriggerConfig.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Trigger configuration of an event driven job. */ +/** + * Trigger configuration of an event driven job. + */ @Fluent public final class JobConfigurationEventTriggerConfig { /* @@ -28,14 +30,16 @@ public final class JobConfigurationEventTriggerConfig { @JsonProperty(value = "scale") private JobScale scale; - /** Creates an instance of JobConfigurationEventTriggerConfig class. */ + /** + * Creates an instance of JobConfigurationEventTriggerConfig class. + */ public JobConfigurationEventTriggerConfig() { } /** * Get the replicaCompletionCount property: Minimum number of successful replica completions before overall job * completion. - * + * * @return the replicaCompletionCount value. */ public Integer replicaCompletionCount() { @@ -45,7 +49,7 @@ public Integer replicaCompletionCount() { /** * Set the replicaCompletionCount property: Minimum number of successful replica completions before overall job * completion. - * + * * @param replicaCompletionCount the replicaCompletionCount value to set. * @return the JobConfigurationEventTriggerConfig object itself. */ @@ -56,7 +60,7 @@ public JobConfigurationEventTriggerConfig withReplicaCompletionCount(Integer rep /** * Get the parallelism property: Number of parallel replicas of a job that can run at a given time. - * + * * @return the parallelism value. */ public Integer parallelism() { @@ -65,7 +69,7 @@ public Integer parallelism() { /** * Set the parallelism property: Number of parallel replicas of a job that can run at a given time. - * + * * @param parallelism the parallelism value to set. * @return the JobConfigurationEventTriggerConfig object itself. */ @@ -76,7 +80,7 @@ public JobConfigurationEventTriggerConfig withParallelism(Integer parallelism) { /** * Get the scale property: Scaling configurations for event driven jobs. - * + * * @return the scale value. */ public JobScale scale() { @@ -85,7 +89,7 @@ public JobScale scale() { /** * Set the scale property: Scaling configurations for event driven jobs. - * + * * @param scale the scale value to set. * @return the JobConfigurationEventTriggerConfig object itself. */ @@ -96,7 +100,7 @@ public JobConfigurationEventTriggerConfig withScale(JobScale scale) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationManualTriggerConfig.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationManualTriggerConfig.java index 64bd76c836273..dbc0e1127a7bf 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationManualTriggerConfig.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationManualTriggerConfig.java @@ -25,14 +25,16 @@ public final class JobConfigurationManualTriggerConfig { @JsonProperty(value = "parallelism") private Integer parallelism; - /** Creates an instance of JobConfigurationManualTriggerConfig class. */ + /** + * Creates an instance of JobConfigurationManualTriggerConfig class. + */ public JobConfigurationManualTriggerConfig() { } /** * Get the replicaCompletionCount property: Minimum number of successful replica completions before overall job * completion. - * + * * @return the replicaCompletionCount value. */ public Integer replicaCompletionCount() { @@ -42,7 +44,7 @@ public Integer replicaCompletionCount() { /** * Set the replicaCompletionCount property: Minimum number of successful replica completions before overall job * completion. - * + * * @param replicaCompletionCount the replicaCompletionCount value to set. * @return the JobConfigurationManualTriggerConfig object itself. */ @@ -53,7 +55,7 @@ public JobConfigurationManualTriggerConfig withReplicaCompletionCount(Integer re /** * Get the parallelism property: Number of parallel replicas of a job that can run at a given time. - * + * * @return the parallelism value. */ public Integer parallelism() { @@ -62,7 +64,7 @@ public Integer parallelism() { /** * Set the parallelism property: Number of parallel replicas of a job that can run at a given time. - * + * * @param parallelism the parallelism value to set. * @return the JobConfigurationManualTriggerConfig object itself. */ @@ -73,7 +75,7 @@ public JobConfigurationManualTriggerConfig withParallelism(Integer parallelism) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationScheduleTriggerConfig.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationScheduleTriggerConfig.java index 69435efd237d0..4ce57dc75d706 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationScheduleTriggerConfig.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobConfigurationScheduleTriggerConfig.java @@ -9,8 +9,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * Cron formatted repeating trigger schedule ("* * * * *") for cronjobs. Properties completions and parallelism would be - * set to 1 by default. + * Cron formatted repeating trigger schedule ("* * * * *") for cronjobs. Properties completions and parallelism would + * be set to 1 by default. */ @Fluent public final class JobConfigurationScheduleTriggerConfig { @@ -32,14 +32,16 @@ public final class JobConfigurationScheduleTriggerConfig { @JsonProperty(value = "parallelism") private Integer parallelism; - /** Creates an instance of JobConfigurationScheduleTriggerConfig class. */ + /** + * Creates an instance of JobConfigurationScheduleTriggerConfig class. + */ public JobConfigurationScheduleTriggerConfig() { } /** * Get the replicaCompletionCount property: Minimum number of successful replica completions before overall job * completion. - * + * * @return the replicaCompletionCount value. */ public Integer replicaCompletionCount() { @@ -49,7 +51,7 @@ public Integer replicaCompletionCount() { /** * Set the replicaCompletionCount property: Minimum number of successful replica completions before overall job * completion. - * + * * @param replicaCompletionCount the replicaCompletionCount value to set. * @return the JobConfigurationScheduleTriggerConfig object itself. */ @@ -60,7 +62,7 @@ public JobConfigurationScheduleTriggerConfig withReplicaCompletionCount(Integer /** * Get the cronExpression property: Cron formatted repeating schedule ("* * * * *") of a Cron Job. - * + * * @return the cronExpression value. */ public String cronExpression() { @@ -69,7 +71,7 @@ public String cronExpression() { /** * Set the cronExpression property: Cron formatted repeating schedule ("* * * * *") of a Cron Job. - * + * * @param cronExpression the cronExpression value to set. * @return the JobConfigurationScheduleTriggerConfig object itself. */ @@ -80,7 +82,7 @@ public JobConfigurationScheduleTriggerConfig withCronExpression(String cronExpre /** * Get the parallelism property: Number of parallel replicas of a job that can run at a given time. - * + * * @return the parallelism value. */ public Integer parallelism() { @@ -89,7 +91,7 @@ public Integer parallelism() { /** * Set the parallelism property: Number of parallel replicas of a job that can run at a given time. - * + * * @param parallelism the parallelism value to set. * @return the JobConfigurationScheduleTriggerConfig object itself. */ @@ -100,15 +102,13 @@ public JobConfigurationScheduleTriggerConfig withParallelism(Integer parallelism /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (cronExpression() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property cronExpression in model JobConfigurationScheduleTriggerConfig")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property cronExpression in model JobConfigurationScheduleTriggerConfig")); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecution.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecution.java index 94a3a816e3c81..cf83584a43866 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecution.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecution.java @@ -7,60 +7,62 @@ import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of JobExecution. */ +/** + * An immutable client-side representation of JobExecution. + */ public interface JobExecution { /** * Gets the name property: Job execution Name. - * + * * @return the name value. */ String name(); /** * Gets the id property: Job execution Id. - * + * * @return the id value. */ String id(); /** - * Gets the type property: Job Type. - * + * Gets the type property: Job execution type. + * * @return the type value. */ String type(); /** * Gets the status property: Current running State of the job. - * + * * @return the status value. */ JobExecutionRunningState status(); /** * Gets the startTime property: Job execution start time. - * + * * @return the startTime value. */ OffsetDateTime startTime(); /** * Gets the endTime property: Job execution end time. - * + * * @return the endTime value. */ OffsetDateTime endTime(); /** * Gets the template property: Job's execution container. - * + * * @return the template value. */ JobExecutionTemplate template(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner object. - * + * * @return the inner object. */ JobExecutionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionBase.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionBase.java index fb4caf902bb2f..9a987dc30d609 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionBase.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionBase.java @@ -6,25 +6,27 @@ import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionBaseInner; -/** An immutable client-side representation of JobExecutionBase. */ +/** + * An immutable client-side representation of JobExecutionBase. + */ public interface JobExecutionBase { /** * Gets the name property: Job execution name. - * + * * @return the name value. */ String name(); /** * Gets the id property: Job execution Id. - * + * * @return the id value. */ String id(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionBaseInner object. - * + * * @return the inner object. */ JobExecutionBaseInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionContainer.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionContainer.java index dea126fc397cc..254e4f869bc84 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionContainer.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionContainer.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container Apps Jobs execution container definition. */ +/** + * Container Apps Jobs execution container definition. + */ @Fluent public final class JobExecutionContainer { /* @@ -47,13 +49,15 @@ public final class JobExecutionContainer { @JsonProperty(value = "resources") private ContainerResources resources; - /** Creates an instance of JobExecutionContainer class. */ + /** + * Creates an instance of JobExecutionContainer class. + */ public JobExecutionContainer() { } /** * Get the image property: Container image tag. - * + * * @return the image value. */ public String image() { @@ -62,7 +66,7 @@ public String image() { /** * Set the image property: Container image tag. - * + * * @param image the image value to set. * @return the JobExecutionContainer object itself. */ @@ -73,7 +77,7 @@ public JobExecutionContainer withImage(String image) { /** * Get the name property: Custom container name. - * + * * @return the name value. */ public String name() { @@ -82,7 +86,7 @@ public String name() { /** * Set the name property: Custom container name. - * + * * @param name the name value to set. * @return the JobExecutionContainer object itself. */ @@ -93,7 +97,7 @@ public JobExecutionContainer withName(String name) { /** * Get the command property: Container start command. - * + * * @return the command value. */ public List command() { @@ -102,7 +106,7 @@ public List command() { /** * Set the command property: Container start command. - * + * * @param command the command value to set. * @return the JobExecutionContainer object itself. */ @@ -113,7 +117,7 @@ public JobExecutionContainer withCommand(List command) { /** * Get the args property: Container start command arguments. - * + * * @return the args value. */ public List args() { @@ -122,7 +126,7 @@ public List args() { /** * Set the args property: Container start command arguments. - * + * * @param args the args value to set. * @return the JobExecutionContainer object itself. */ @@ -133,7 +137,7 @@ public JobExecutionContainer withArgs(List args) { /** * Get the env property: Container environment variables. - * + * * @return the env value. */ public List env() { @@ -142,7 +146,7 @@ public List env() { /** * Set the env property: Container environment variables. - * + * * @param env the env value to set. * @return the JobExecutionContainer object itself. */ @@ -153,7 +157,7 @@ public JobExecutionContainer withEnv(List env) { /** * Get the resources property: Container resource requirements. - * + * * @return the resources value. */ public ContainerResources resources() { @@ -162,7 +166,7 @@ public ContainerResources resources() { /** * Set the resources property: Container resource requirements. - * + * * @param resources the resources value to set. * @return the JobExecutionContainer object itself. */ @@ -173,7 +177,7 @@ public JobExecutionContainer withResources(ContainerResources resources) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionRunningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionRunningState.java index d10e8261d16c5..aed929aec8619 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionRunningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionRunningState.java @@ -8,32 +8,48 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current running State of the job. */ +/** + * Current running State of the job. + */ public final class JobExecutionRunningState extends ExpandableStringEnum { - /** Static value Running for JobExecutionRunningState. */ + /** + * Static value Running for JobExecutionRunningState. + */ public static final JobExecutionRunningState RUNNING = fromString("Running"); - /** Static value Processing for JobExecutionRunningState. */ + /** + * Static value Processing for JobExecutionRunningState. + */ public static final JobExecutionRunningState PROCESSING = fromString("Processing"); - /** Static value Stopped for JobExecutionRunningState. */ + /** + * Static value Stopped for JobExecutionRunningState. + */ public static final JobExecutionRunningState STOPPED = fromString("Stopped"); - /** Static value Degraded for JobExecutionRunningState. */ + /** + * Static value Degraded for JobExecutionRunningState. + */ public static final JobExecutionRunningState DEGRADED = fromString("Degraded"); - /** Static value Failed for JobExecutionRunningState. */ + /** + * Static value Failed for JobExecutionRunningState. + */ public static final JobExecutionRunningState FAILED = fromString("Failed"); - /** Static value Unknown for JobExecutionRunningState. */ + /** + * Static value Unknown for JobExecutionRunningState. + */ public static final JobExecutionRunningState UNKNOWN = fromString("Unknown"); - /** Static value Succeeded for JobExecutionRunningState. */ + /** + * Static value Succeeded for JobExecutionRunningState. + */ public static final JobExecutionRunningState SUCCEEDED = fromString("Succeeded"); /** * Creates a new instance of JobExecutionRunningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -42,7 +58,7 @@ public JobExecutionRunningState() { /** * Creates or finds a JobExecutionRunningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding JobExecutionRunningState. */ @@ -53,7 +69,7 @@ public static JobExecutionRunningState fromString(String name) { /** * Gets known JobExecutionRunningState values. - * + * * @return known JobExecutionRunningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionTemplate.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionTemplate.java index eec1fda1387a2..c9305f3f20c0e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionTemplate.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobExecutionTemplate.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Job's execution template, containing container configuration for a job's execution. */ +/** + * Job's execution template, containing container configuration for a job's execution. + */ @Fluent public final class JobExecutionTemplate { /* @@ -23,13 +25,15 @@ public final class JobExecutionTemplate { @JsonProperty(value = "initContainers") private List initContainers; - /** Creates an instance of JobExecutionTemplate class. */ + /** + * Creates an instance of JobExecutionTemplate class. + */ public JobExecutionTemplate() { } /** * Get the containers property: List of container definitions for the Container Apps Job. - * + * * @return the containers value. */ public List containers() { @@ -38,7 +42,7 @@ public List containers() { /** * Set the containers property: List of container definitions for the Container Apps Job. - * + * * @param containers the containers value to set. * @return the JobExecutionTemplate object itself. */ @@ -49,7 +53,7 @@ public JobExecutionTemplate withContainers(List container /** * Get the initContainers property: List of specialized containers that run before job containers. - * + * * @return the initContainers value. */ public List initContainers() { @@ -58,7 +62,7 @@ public List initContainers() { /** * Set the initContainers property: List of specialized containers that run before job containers. - * + * * @param initContainers the initContainers value to set. * @return the JobExecutionTemplate object itself. */ @@ -69,7 +73,7 @@ public JobExecutionTemplate withInitContainers(List initC /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchProperties.java index dddbf57dd6ac1..d09f18fc692a7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchProperties.java @@ -9,9 +9,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Container Apps Job resource specific properties. */ +/** + * Container Apps Job resource specific properties. + */ @Fluent public final class JobPatchProperties { + /* + * The complex type of the extended location. + */ + @JsonProperty(value = "extendedLocation") + private ExtendedLocation extendedLocation; + /* * Managed identities needed by a container app job to interact with other Azure services to not maintain any * secrets or credentials in code. @@ -32,14 +40,36 @@ public final class JobPatchProperties { @JsonProperty(value = "properties") private JobPatchPropertiesProperties properties; - /** Creates an instance of JobPatchProperties class. */ + /** + * Creates an instance of JobPatchProperties class. + */ public JobPatchProperties() { } /** - * Get the identity property: Managed identities needed by a container app job to interact with other Azure services - * to not maintain any secrets or credentials in code. - * + * Get the extendedLocation property: The complex type of the extended location. + * + * @return the extendedLocation value. + */ + public ExtendedLocation extendedLocation() { + return this.extendedLocation; + } + + /** + * Set the extendedLocation property: The complex type of the extended location. + * + * @param extendedLocation the extendedLocation value to set. + * @return the JobPatchProperties object itself. + */ + public JobPatchProperties withExtendedLocation(ExtendedLocation extendedLocation) { + this.extendedLocation = extendedLocation; + return this; + } + + /** + * Get the identity property: Managed identities needed by a container app job to interact with other Azure + * services to not maintain any secrets or credentials in code. + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -47,9 +77,9 @@ public ManagedServiceIdentity identity() { } /** - * Set the identity property: Managed identities needed by a container app job to interact with other Azure services - * to not maintain any secrets or credentials in code. - * + * Set the identity property: Managed identities needed by a container app job to interact with other Azure + * services to not maintain any secrets or credentials in code. + * * @param identity the identity value to set. * @return the JobPatchProperties object itself. */ @@ -60,7 +90,7 @@ public JobPatchProperties withIdentity(ManagedServiceIdentity identity) { /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -69,7 +99,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the JobPatchProperties object itself. */ @@ -80,7 +110,7 @@ public JobPatchProperties withTags(Map tags) { /** * Get the properties property: The properties property. - * + * * @return the properties value. */ public JobPatchPropertiesProperties properties() { @@ -89,7 +119,7 @@ public JobPatchPropertiesProperties properties() { /** * Set the properties property: The properties property. - * + * * @param properties the properties value to set. * @return the JobPatchProperties object itself. */ @@ -100,10 +130,13 @@ public JobPatchProperties withProperties(JobPatchPropertiesProperties properties /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (extendedLocation() != null) { + extendedLocation().validate(); + } if (identity() != null) { identity().validate(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchPropertiesProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchPropertiesProperties.java index 246e0a88c0d0b..38474193d99fb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchPropertiesProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobPatchPropertiesProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The JobPatchPropertiesProperties model. */ +/** + * The JobPatchPropertiesProperties model. + */ @Fluent public final class JobPatchPropertiesProperties { /* @@ -41,13 +43,15 @@ public final class JobPatchPropertiesProperties { @JsonProperty(value = "eventStreamEndpoint") private String eventStreamEndpoint; - /** Creates an instance of JobPatchPropertiesProperties class. */ + /** + * Creates an instance of JobPatchPropertiesProperties class. + */ public JobPatchPropertiesProperties() { } /** * Get the environmentId property: Resource ID of environment. - * + * * @return the environmentId value. */ public String environmentId() { @@ -56,7 +60,7 @@ public String environmentId() { /** * Set the environmentId property: Resource ID of environment. - * + * * @param environmentId the environmentId value to set. * @return the JobPatchPropertiesProperties object itself. */ @@ -67,7 +71,7 @@ public JobPatchPropertiesProperties withEnvironmentId(String environmentId) { /** * Get the configuration property: Container Apps Job configuration properties. - * + * * @return the configuration value. */ public JobConfiguration configuration() { @@ -76,7 +80,7 @@ public JobConfiguration configuration() { /** * Set the configuration property: Container Apps Job configuration properties. - * + * * @param configuration the configuration value to set. * @return the JobPatchPropertiesProperties object itself. */ @@ -87,7 +91,7 @@ public JobPatchPropertiesProperties withConfiguration(JobConfiguration configura /** * Get the template property: Container Apps job definition. - * + * * @return the template value. */ public JobTemplate template() { @@ -96,7 +100,7 @@ public JobTemplate template() { /** * Set the template property: Container Apps job definition. - * + * * @param template the template value to set. * @return the JobPatchPropertiesProperties object itself. */ @@ -107,7 +111,7 @@ public JobPatchPropertiesProperties withTemplate(JobTemplate template) { /** * Get the outboundIpAddresses property: Outbound IP Addresses of a container apps job. - * + * * @return the outboundIpAddresses value. */ public List outboundIpAddresses() { @@ -116,7 +120,7 @@ public List outboundIpAddresses() { /** * Set the outboundIpAddresses property: Outbound IP Addresses of a container apps job. - * + * * @param outboundIpAddresses the outboundIpAddresses value to set. * @return the JobPatchPropertiesProperties object itself. */ @@ -127,7 +131,7 @@ public JobPatchPropertiesProperties withOutboundIpAddresses(List outboun /** * Get the eventStreamEndpoint property: The endpoint of the eventstream of the container apps job. - * + * * @return the eventStreamEndpoint value. */ public String eventStreamEndpoint() { @@ -136,7 +140,7 @@ public String eventStreamEndpoint() { /** * Set the eventStreamEndpoint property: The endpoint of the eventstream of the container apps job. - * + * * @param eventStreamEndpoint the eventStreamEndpoint value to set. * @return the JobPatchPropertiesProperties object itself. */ @@ -147,7 +151,7 @@ public JobPatchPropertiesProperties withEventStreamEndpoint(String eventStreamEn /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobProvisioningState.java index 36f82edc4b8cf..adbef85d8a2e8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobProvisioningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobProvisioningState.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning state of the Container Apps Job. */ +/** + * Provisioning state of the Container Apps Job. + */ public final class JobProvisioningState extends ExpandableStringEnum { - /** Static value InProgress for JobProvisioningState. */ + /** + * Static value InProgress for JobProvisioningState. + */ public static final JobProvisioningState IN_PROGRESS = fromString("InProgress"); - /** Static value Succeeded for JobProvisioningState. */ + /** + * Static value Succeeded for JobProvisioningState. + */ public static final JobProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for JobProvisioningState. */ + /** + * Static value Failed for JobProvisioningState. + */ public static final JobProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for JobProvisioningState. */ + /** + * Static value Canceled for JobProvisioningState. + */ public static final JobProvisioningState CANCELED = fromString("Canceled"); - /** Static value Deleting for JobProvisioningState. */ + /** + * Static value Deleting for JobProvisioningState. + */ public static final JobProvisioningState DELETING = fromString("Deleting"); /** * Creates a new instance of JobProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public JobProvisioningState() { /** * Creates or finds a JobProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding JobProvisioningState. */ @@ -47,7 +59,7 @@ public static JobProvisioningState fromString(String name) { /** * Gets known JobProvisioningState values. - * + * * @return known JobProvisioningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScale.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScale.java index efaf34a130321..a0047df76dd4d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScale.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScale.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Scaling configurations for event driven jobs. */ +/** + * Scaling configurations for event driven jobs. + */ @Fluent public final class JobScale { /* @@ -35,13 +37,15 @@ public final class JobScale { @JsonProperty(value = "rules") private List rules; - /** Creates an instance of JobScale class. */ + /** + * Creates an instance of JobScale class. + */ public JobScale() { } /** * Get the pollingInterval property: Interval to check each event source in seconds. Defaults to 30s. - * + * * @return the pollingInterval value. */ public Integer pollingInterval() { @@ -50,7 +54,7 @@ public Integer pollingInterval() { /** * Set the pollingInterval property: Interval to check each event source in seconds. Defaults to 30s. - * + * * @param pollingInterval the pollingInterval value to set. * @return the JobScale object itself. */ @@ -61,7 +65,7 @@ public JobScale withPollingInterval(Integer pollingInterval) { /** * Get the minExecutions property: Minimum number of job executions that are created for a trigger, default 0. - * + * * @return the minExecutions value. */ public Integer minExecutions() { @@ -70,7 +74,7 @@ public Integer minExecutions() { /** * Set the minExecutions property: Minimum number of job executions that are created for a trigger, default 0. - * + * * @param minExecutions the minExecutions value to set. * @return the JobScale object itself. */ @@ -81,7 +85,7 @@ public JobScale withMinExecutions(Integer minExecutions) { /** * Get the maxExecutions property: Maximum number of job executions that are created for a trigger, default 100. - * + * * @return the maxExecutions value. */ public Integer maxExecutions() { @@ -90,7 +94,7 @@ public Integer maxExecutions() { /** * Set the maxExecutions property: Maximum number of job executions that are created for a trigger, default 100. - * + * * @param maxExecutions the maxExecutions value to set. * @return the JobScale object itself. */ @@ -101,7 +105,7 @@ public JobScale withMaxExecutions(Integer maxExecutions) { /** * Get the rules property: Scaling rules. - * + * * @return the rules value. */ public List rules() { @@ -110,7 +114,7 @@ public List rules() { /** * Set the rules property: Scaling rules. - * + * * @param rules the rules value to set. * @return the JobScale object itself. */ @@ -121,7 +125,7 @@ public JobScale withRules(List rules) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScaleRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScaleRule.java index 98412e5d0c3e2..a78b1c2b38b70 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScaleRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobScaleRule.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Scaling rule. */ +/** + * Scaling rule. + */ @Fluent public final class JobScaleRule { /* @@ -36,13 +38,15 @@ public final class JobScaleRule { @JsonProperty(value = "auth") private List auth; - /** Creates an instance of JobScaleRule class. */ + /** + * Creates an instance of JobScaleRule class. + */ public JobScaleRule() { } /** * Get the name property: Scale Rule Name. - * + * * @return the name value. */ public String name() { @@ -51,7 +55,7 @@ public String name() { /** * Set the name property: Scale Rule Name. - * + * * @param name the name value to set. * @return the JobScaleRule object itself. */ @@ -61,8 +65,9 @@ public JobScaleRule withName(String name) { } /** - * Get the type property: Type of the scale rule eg: azure-servicebus, redis etc. - * + * Get the type property: Type of the scale rule + * eg: azure-servicebus, redis etc. + * * @return the type value. */ public String type() { @@ -70,8 +75,9 @@ public String type() { } /** - * Set the type property: Type of the scale rule eg: azure-servicebus, redis etc. - * + * Set the type property: Type of the scale rule + * eg: azure-servicebus, redis etc. + * * @param type the type value to set. * @return the JobScaleRule object itself. */ @@ -82,7 +88,7 @@ public JobScaleRule withType(String type) { /** * Get the metadata property: Metadata properties to describe the scale rule. - * + * * @return the metadata value. */ public Object metadata() { @@ -91,7 +97,7 @@ public Object metadata() { /** * Set the metadata property: Metadata properties to describe the scale rule. - * + * * @param metadata the metadata value to set. * @return the JobScaleRule object itself. */ @@ -102,7 +108,7 @@ public JobScaleRule withMetadata(Object metadata) { /** * Get the auth property: Authentication secrets for the scale rule. - * + * * @return the auth value. */ public List auth() { @@ -111,7 +117,7 @@ public List auth() { /** * Set the auth property: Authentication secrets for the scale rule. - * + * * @param auth the auth value to set. * @return the JobScaleRule object itself. */ @@ -122,7 +128,7 @@ public JobScaleRule withAuth(List auth) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobSecretsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobSecretsCollection.java index 89a553df4015b..d7abd9c403d63 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobSecretsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobSecretsCollection.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.appcontainers.fluent.models.JobSecretsCollectionInner; import java.util.List; -/** An immutable client-side representation of JobSecretsCollection. */ +/** + * An immutable client-side representation of JobSecretsCollection. + */ public interface JobSecretsCollection { /** * Gets the value property: Collection of resources. - * + * * @return the value value. */ List value(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.JobSecretsCollectionInner object. - * + * * @return the inner object. */ JobSecretsCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobTemplate.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobTemplate.java index ca61cdb3f88be..753c254e3b020 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobTemplate.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobTemplate.java @@ -32,13 +32,15 @@ public final class JobTemplate { @JsonProperty(value = "volumes") private List volumes; - /** Creates an instance of JobTemplate class. */ + /** + * Creates an instance of JobTemplate class. + */ public JobTemplate() { } /** * Get the initContainers property: List of specialized containers that run before app containers. - * + * * @return the initContainers value. */ public List initContainers() { @@ -47,7 +49,7 @@ public List initContainers() { /** * Set the initContainers property: List of specialized containers that run before app containers. - * + * * @param initContainers the initContainers value to set. * @return the JobTemplate object itself. */ @@ -58,7 +60,7 @@ public JobTemplate withInitContainers(List initContainers) { /** * Get the containers property: List of container definitions for the Container App. - * + * * @return the containers value. */ public List containers() { @@ -67,7 +69,7 @@ public List containers() { /** * Set the containers property: List of container definitions for the Container App. - * + * * @param containers the containers value to set. * @return the JobTemplate object itself. */ @@ -78,7 +80,7 @@ public JobTemplate withContainers(List containers) { /** * Get the volumes property: List of volume definitions for the Container App. - * + * * @return the volumes value. */ public List volumes() { @@ -87,7 +89,7 @@ public List volumes() { /** * Set the volumes property: List of volume definitions for the Container App. - * + * * @param volumes the volumes value to set. * @return the JobTemplate object itself. */ @@ -98,7 +100,7 @@ public JobTemplate withVolumes(List volumes) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Jobs.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Jobs.java index bc45c11c5424c..0cee44f1bba71 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Jobs.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Jobs.java @@ -8,13 +8,112 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Jobs. */ +/** + * Resource collection API of Jobs. + */ public interface Jobs { + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job along with {@link Response}. + */ + Response listDetectorsWithResponse(String resourceGroupName, String jobName, + Context context); + + /** + * Get the list of diagnostics for a given Container App Job. + * + * Get the list of diagnostics for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the list of diagnostics for a Container App Job. + */ + DiagnosticsCollection listDetectors(String resourceGroupName, String jobName); + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job along with {@link Response}. + */ + Response getDetectorWithResponse(String resourceGroupName, String jobName, String detectorName, + Context context); + + /** + * Get the diagnostics data for a given Container App Job. + * + * Get the diagnostics data for a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param detectorName Name of the Container App Job detector. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the diagnostics data for a Container App Job. + */ + Diagnostics getDetector(String resourceGroupName, String jobName, String detectorName); + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job along with {@link Response}. + */ + Response proxyGetWithResponse(String resourceGroupName, String jobName, Context context); + + /** + * Get the properties for a given Container App Job. + * + * Get the properties of a Container App Job. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param jobName Job Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the properties of a Container App Job. + */ + Job proxyGet(String resourceGroupName, String jobName); + /** * Get the Container Apps Jobs in a given subscription. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription as paginated response with {@link PagedIterable}. */ @@ -22,11 +121,11 @@ public interface Jobs { /** * Get the Container Apps Jobs in a given subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given subscription as paginated response with {@link PagedIterable}. */ @@ -34,11 +133,11 @@ public interface Jobs { /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -46,12 +145,12 @@ public interface Jobs { /** * Get the Container Apps Jobs in a given resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Container Apps Jobs in a given resource group as paginated response with {@link PagedIterable}. */ @@ -59,13 +158,13 @@ public interface Jobs { /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container Apps Job along with {@link Response}. */ @@ -73,12 +172,12 @@ public interface Jobs { /** * Get the properties of a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container Apps Job. */ @@ -86,37 +185,37 @@ public interface Jobs { /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByResourceGroup(String resourceGroupName, String jobName); /** * Delete a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String jobName, Context context); /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App's Job execution name. */ @@ -124,14 +223,14 @@ public interface Jobs { /** * Start a Container Apps Job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param template Properties used to start a job execution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App's Job execution name. */ @@ -139,39 +238,39 @@ public interface Jobs { /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void stopExecution(String resourceGroupName, String jobName, String jobExecutionName); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void stopExecution(String resourceGroupName, String jobName, String jobExecutionName, Context context); /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource. */ @@ -179,13 +278,13 @@ public interface Jobs { /** * Terminates execution of a running container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container App executions collection ARM resource. */ @@ -193,13 +292,13 @@ public interface Jobs { /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource along with {@link Response}. */ @@ -207,12 +306,12 @@ public interface Jobs { /** * List secrets for a container apps job. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return container Apps Job Secrets Collection ARM resource. */ @@ -220,11 +319,11 @@ public interface Jobs { /** * Get the properties of a Container Apps Job. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container Apps Job along with {@link Response}. */ @@ -232,12 +331,12 @@ public interface Jobs { /** * Get the properties of a Container Apps Job. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Container Apps Job along with {@link Response}. */ @@ -245,30 +344,30 @@ public interface Jobs { /** * Delete a Container Apps Job. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete a Container Apps Job. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new Job resource. - * + * * @param name resource name. * @return the first stage of the new Job definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsCollection.java index 40f2582ca10ed..7d0d995b3a42b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container Apps Jobs collection ARM resource. */ +/** + * Container Apps Jobs collection ARM resource. + */ @Fluent public final class JobsCollection { /* @@ -25,13 +27,15 @@ public final class JobsCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of JobsCollection class. */ + /** + * Creates an instance of JobsCollection class. + */ public JobsCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the JobsCollection object itself. */ @@ -51,7 +55,7 @@ public JobsCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model JobsCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model JobsCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsExecutions.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsExecutions.java index 250d6e5c3961f..d97086ea85499 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsExecutions.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JobsExecutions.java @@ -7,16 +7,18 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of JobsExecutions. */ +/** + * Resource collection API of JobsExecutions. + */ public interface JobsExecutions { /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Container Apps Job's executions as paginated response with {@link PagedIterable}. */ @@ -24,14 +26,14 @@ public interface JobsExecutions { /** * Get a Container Apps Job's executions. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param filter The filter to apply on the operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Container Apps Job's executions as paginated response with {@link PagedIterable}. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JwtClaimChecks.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JwtClaimChecks.java index 987739c9a1853..b4086c851a89c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JwtClaimChecks.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/JwtClaimChecks.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the checks that should be made while validating the JWT Claims. */ +/** + * The configuration settings of the checks that should be made while validating the JWT Claims. + */ @Fluent public final class JwtClaimChecks { /* @@ -23,13 +25,15 @@ public final class JwtClaimChecks { @JsonProperty(value = "allowedClientApplications") private List allowedClientApplications; - /** Creates an instance of JwtClaimChecks class. */ + /** + * Creates an instance of JwtClaimChecks class. + */ public JwtClaimChecks() { } /** * Get the allowedGroups property: The list of the allowed groups. - * + * * @return the allowedGroups value. */ public List allowedGroups() { @@ -38,7 +42,7 @@ public List allowedGroups() { /** * Set the allowedGroups property: The list of the allowed groups. - * + * * @param allowedGroups the allowedGroups value to set. * @return the JwtClaimChecks object itself. */ @@ -49,7 +53,7 @@ public JwtClaimChecks withAllowedGroups(List allowedGroups) { /** * Get the allowedClientApplications property: The list of the allowed client applications. - * + * * @return the allowedClientApplications value. */ public List allowedClientApplications() { @@ -58,7 +62,7 @@ public List allowedClientApplications() { /** * Set the allowedClientApplications property: The list of the allowed client applications. - * + * * @param allowedClientApplications the allowedClientApplications value to set. * @return the JwtClaimChecks object itself. */ @@ -69,7 +73,7 @@ public JwtClaimChecks withAllowedClientApplications(List allowedClientAp /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/KedaConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/KedaConfiguration.java index 0641d83247184..af0054b064a13 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/KedaConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/KedaConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration properties Keda component. */ +/** + * Configuration properties Keda component. + */ @Immutable public final class KedaConfiguration { /* @@ -16,13 +18,15 @@ public final class KedaConfiguration { @JsonProperty(value = "version", access = JsonProperty.Access.WRITE_ONLY) private String version; - /** Creates an instance of KedaConfiguration class. */ + /** + * Creates an instance of KedaConfiguration class. + */ public KedaConfiguration() { } /** * Get the version property: The version of Keda. - * + * * @return the version value. */ public String version() { @@ -31,7 +35,7 @@ public String version() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ListUsagesResult.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ListUsagesResult.java new file mode 100644 index 0000000000000..d4a6e587bc769 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ListUsagesResult.java @@ -0,0 +1,88 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The ListUsagesResult model. + */ +@Fluent +public final class ListUsagesResult { + /* + * The list of compute resource usages. + */ + @JsonProperty(value = "value") + private List value; + + /* + * The URI to fetch the next page of compute resource usage information. Call ListNext() with this to fetch the + * next page of compute resource usage information. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of ListUsagesResult class. + */ + public ListUsagesResult() { + } + + /** + * Get the value property: The list of compute resource usages. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: The list of compute resource usages. + * + * @param value the value value to set. + * @return the ListUsagesResult object itself. + */ + public ListUsagesResult withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: The URI to fetch the next page of compute resource usage information. Call ListNext() + * with this to fetch the next page of compute resource usage information. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: The URI to fetch the next page of compute resource usage information. Call ListNext() + * with this to fetch the next page of compute resource usage information. + * + * @param nextLink the nextLink value to set. + * @return the ListUsagesResult object itself. + */ + public ListUsagesResult withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogAnalyticsConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogAnalyticsConfiguration.java index 56aa6fc181030..aab153d857850 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogAnalyticsConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogAnalyticsConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Log Analytics configuration, must only be provided when destination is configured as 'log-analytics'. */ +/** + * Log Analytics configuration, must only be provided when destination is configured as 'log-analytics'. + */ @Fluent public final class LogAnalyticsConfiguration { /* @@ -22,13 +24,21 @@ public final class LogAnalyticsConfiguration { @JsonProperty(value = "sharedKey") private String sharedKey; - /** Creates an instance of LogAnalyticsConfiguration class. */ + /* + * Boolean indicating whether to parse json string log into dynamic json columns + */ + @JsonProperty(value = "dynamicJsonColumns") + private Boolean dynamicJsonColumns; + + /** + * Creates an instance of LogAnalyticsConfiguration class. + */ public LogAnalyticsConfiguration() { } /** * Get the customerId property: Log analytics customer id. - * + * * @return the customerId value. */ public String customerId() { @@ -37,7 +47,7 @@ public String customerId() { /** * Set the customerId property: Log analytics customer id. - * + * * @param customerId the customerId value to set. * @return the LogAnalyticsConfiguration object itself. */ @@ -48,7 +58,7 @@ public LogAnalyticsConfiguration withCustomerId(String customerId) { /** * Get the sharedKey property: Log analytics customer key. - * + * * @return the sharedKey value. */ public String sharedKey() { @@ -57,7 +67,7 @@ public String sharedKey() { /** * Set the sharedKey property: Log analytics customer key. - * + * * @param sharedKey the sharedKey value to set. * @return the LogAnalyticsConfiguration object itself. */ @@ -66,9 +76,31 @@ public LogAnalyticsConfiguration withSharedKey(String sharedKey) { return this; } + /** + * Get the dynamicJsonColumns property: Boolean indicating whether to parse json string log into dynamic json + * columns. + * + * @return the dynamicJsonColumns value. + */ + public Boolean dynamicJsonColumns() { + return this.dynamicJsonColumns; + } + + /** + * Set the dynamicJsonColumns property: Boolean indicating whether to parse json string log into dynamic json + * columns. + * + * @param dynamicJsonColumns the dynamicJsonColumns value to set. + * @return the LogAnalyticsConfiguration object itself. + */ + public LogAnalyticsConfiguration withDynamicJsonColumns(Boolean dynamicJsonColumns) { + this.dynamicJsonColumns = dynamicJsonColumns; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogLevel.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogLevel.java index 10288186ff842..18536747ff1b2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogLevel.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogLevel.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Sets the log level for the Dapr sidecar. Allowed values are debug, info, warn, error. Default is info. */ +/** + * Sets the log level for the Dapr sidecar. Allowed values are debug, info, warn, error. Default is info. + */ public final class LogLevel extends ExpandableStringEnum { - /** Static value info for LogLevel. */ + /** + * Static value info for LogLevel. + */ public static final LogLevel INFO = fromString("info"); - /** Static value debug for LogLevel. */ + /** + * Static value debug for LogLevel. + */ public static final LogLevel DEBUG = fromString("debug"); - /** Static value warn for LogLevel. */ + /** + * Static value warn for LogLevel. + */ public static final LogLevel WARN = fromString("warn"); - /** Static value error for LogLevel. */ + /** + * Static value error for LogLevel. + */ public static final LogLevel ERROR = fromString("error"); /** * Creates a new instance of LogLevel value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public LogLevel() { /** * Creates or finds a LogLevel from its string representation. - * + * * @param name a name to look for. * @return the corresponding LogLevel. */ @@ -44,7 +54,7 @@ public static LogLevel fromString(String name) { /** * Gets known LogLevel values. - * + * * @return known LogLevel values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Login.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Login.java index 2457b6865eb28..d26582e52661a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Login.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Login.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the login flow of users using ContainerApp Service Authentication/Authorization. */ +/** + * The configuration settings of the login flow of users using ContainerApp Service Authentication/Authorization. + */ @Fluent public final class Login { /* @@ -17,6 +19,12 @@ public final class Login { @JsonProperty(value = "routes") private LoginRoutes routes; + /* + * The configuration settings of the token store. + */ + @JsonProperty(value = "tokenStore") + private TokenStore tokenStore; + /* * true if the fragments from the request are preserved after the login request is made; otherwise, * false. @@ -45,13 +53,15 @@ public final class Login { @JsonProperty(value = "nonce") private Nonce nonce; - /** Creates an instance of Login class. */ + /** + * Creates an instance of Login class. + */ public Login() { } /** * Get the routes property: The routes that specify the endpoints used for login and logout requests. - * + * * @return the routes value. */ public LoginRoutes routes() { @@ -60,7 +70,7 @@ public LoginRoutes routes() { /** * Set the routes property: The routes that specify the endpoints used for login and logout requests. - * + * * @param routes the routes value to set. * @return the Login object itself. */ @@ -69,10 +79,30 @@ public Login withRoutes(LoginRoutes routes) { return this; } + /** + * Get the tokenStore property: The configuration settings of the token store. + * + * @return the tokenStore value. + */ + public TokenStore tokenStore() { + return this.tokenStore; + } + + /** + * Set the tokenStore property: The configuration settings of the token store. + * + * @param tokenStore the tokenStore value to set. + * @return the Login object itself. + */ + public Login withTokenStore(TokenStore tokenStore) { + this.tokenStore = tokenStore; + return this; + } + /** * Get the preserveUrlFragmentsForLogins property: <code>true</code> if the fragments from the request * are preserved after the login request is made; otherwise, <code>false</code>. - * + * * @return the preserveUrlFragmentsForLogins value. */ public Boolean preserveUrlFragmentsForLogins() { @@ -82,7 +112,7 @@ public Boolean preserveUrlFragmentsForLogins() { /** * Set the preserveUrlFragmentsForLogins property: <code>true</code> if the fragments from the request * are preserved after the login request is made; otherwise, <code>false</code>. - * + * * @param preserveUrlFragmentsForLogins the preserveUrlFragmentsForLogins value to set. * @return the Login object itself. */ @@ -93,10 +123,10 @@ public Login withPreserveUrlFragmentsForLogins(Boolean preserveUrlFragmentsForLo /** * Get the allowedExternalRedirectUrls property: External URLs that can be redirected to as part of logging in or - * logging out of the app. Note that the query string part of the URL is ignored. This is an advanced setting - * typically only needed by Windows Store application backends. Note that URLs within the current domain are always - * implicitly allowed. - * + * logging out of the app. Note that the query string part of the URL is ignored. + * This is an advanced setting typically only needed by Windows Store application backends. + * Note that URLs within the current domain are always implicitly allowed. + * * @return the allowedExternalRedirectUrls value. */ public List allowedExternalRedirectUrls() { @@ -105,10 +135,10 @@ public List allowedExternalRedirectUrls() { /** * Set the allowedExternalRedirectUrls property: External URLs that can be redirected to as part of logging in or - * logging out of the app. Note that the query string part of the URL is ignored. This is an advanced setting - * typically only needed by Windows Store application backends. Note that URLs within the current domain are always - * implicitly allowed. - * + * logging out of the app. Note that the query string part of the URL is ignored. + * This is an advanced setting typically only needed by Windows Store application backends. + * Note that URLs within the current domain are always implicitly allowed. + * * @param allowedExternalRedirectUrls the allowedExternalRedirectUrls value to set. * @return the Login object itself. */ @@ -119,7 +149,7 @@ public Login withAllowedExternalRedirectUrls(List allowedExternalRedirec /** * Get the cookieExpiration property: The configuration settings of the session cookie's expiration. - * + * * @return the cookieExpiration value. */ public CookieExpiration cookieExpiration() { @@ -128,7 +158,7 @@ public CookieExpiration cookieExpiration() { /** * Set the cookieExpiration property: The configuration settings of the session cookie's expiration. - * + * * @param cookieExpiration the cookieExpiration value to set. * @return the Login object itself. */ @@ -139,7 +169,7 @@ public Login withCookieExpiration(CookieExpiration cookieExpiration) { /** * Get the nonce property: The configuration settings of the nonce used in the login flow. - * + * * @return the nonce value. */ public Nonce nonce() { @@ -148,7 +178,7 @@ public Nonce nonce() { /** * Set the nonce property: The configuration settings of the nonce used in the login flow. - * + * * @param nonce the nonce value to set. * @return the Login object itself. */ @@ -159,13 +189,16 @@ public Login withNonce(Nonce nonce) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (routes() != null) { routes().validate(); } + if (tokenStore() != null) { + tokenStore().validate(); + } if (cookieExpiration() != null) { cookieExpiration().validate(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginRoutes.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginRoutes.java index b46f5243d9704..367f861eef9d1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginRoutes.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginRoutes.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The routes that specify the endpoints used for login and logout requests. */ +/** + * The routes that specify the endpoints used for login and logout requests. + */ @Fluent public final class LoginRoutes { /* @@ -16,13 +18,15 @@ public final class LoginRoutes { @JsonProperty(value = "logoutEndpoint") private String logoutEndpoint; - /** Creates an instance of LoginRoutes class. */ + /** + * Creates an instance of LoginRoutes class. + */ public LoginRoutes() { } /** * Get the logoutEndpoint property: The endpoint at which a logout request should be made. - * + * * @return the logoutEndpoint value. */ public String logoutEndpoint() { @@ -31,7 +35,7 @@ public String logoutEndpoint() { /** * Set the logoutEndpoint property: The endpoint at which a logout request should be made. - * + * * @param logoutEndpoint the logoutEndpoint value to set. * @return the LoginRoutes object itself. */ @@ -42,7 +46,7 @@ public LoginRoutes withLogoutEndpoint(String logoutEndpoint) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginScopes.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginScopes.java index 017270ea4dd16..13755be204d09 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginScopes.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LoginScopes.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the login flow, including the scopes that should be requested. */ +/** + * The configuration settings of the login flow, including the scopes that should be requested. + */ @Fluent public final class LoginScopes { /* @@ -17,13 +19,15 @@ public final class LoginScopes { @JsonProperty(value = "scopes") private List scopes; - /** Creates an instance of LoginScopes class. */ + /** + * Creates an instance of LoginScopes class. + */ public LoginScopes() { } /** * Get the scopes property: A list of the scopes that should be requested while authenticating. - * + * * @return the scopes value. */ public List scopes() { @@ -32,7 +36,7 @@ public List scopes() { /** * Set the scopes property: A list of the scopes that should be requested while authenticating. - * + * * @param scopes the scopes value to set. * @return the LoginScopes object itself. */ @@ -43,7 +47,7 @@ public LoginScopes withScopes(List scopes) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogsConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogsConfiguration.java new file mode 100644 index 0000000000000..0f25acc6ec6b2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/LogsConfiguration.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration of Open Telemetry logs. + */ +@Fluent +public final class LogsConfiguration { + /* + * Open telemetry logs destinations + */ + @JsonProperty(value = "destinations") + private List destinations; + + /** + * Creates an instance of LogsConfiguration class. + */ + public LogsConfiguration() { + } + + /** + * Get the destinations property: Open telemetry logs destinations. + * + * @return the destinations value. + */ + public List destinations() { + return this.destinations; + } + + /** + * Set the destinations property: Open telemetry logs destinations. + * + * @param destinations the destinations value to set. + * @return the LogsConfiguration object itself. + */ + public LogsConfiguration withDestinations(List destinations) { + this.destinations = destinations; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificate.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificate.java index 40f38dd19c370..d3863e678425b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificate.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificate.java @@ -10,104 +10,111 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ManagedCertificateInner; import java.util.Map; -/** An immutable client-side representation of ManagedCertificate. */ +/** + * An immutable client-side representation of ManagedCertificate. + */ public interface ManagedCertificate { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the properties property: Certificate resource specific properties. - * + * * @return the properties value. */ ManagedCertificateProperties properties(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ManagedCertificateInner object. - * + * * @return the inner object. */ ManagedCertificateInner innerModel(); - /** The entirety of the ManagedCertificate definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithCreate { + /** + * The entirety of the ManagedCertificate definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The ManagedCertificate definition stages. */ + /** + * The ManagedCertificate definition stages. + */ interface DefinitionStages { - /** The first stage of the ManagedCertificate definition. */ + /** + * The first stage of the ManagedCertificate definition. + */ interface Blank extends WithLocation { } - /** The stage of the ManagedCertificate definition allowing to specify location. */ + /** + * The stage of the ManagedCertificate definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -115,18 +122,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the ManagedCertificate definition allowing to specify parent resource. */ + /** + * The stage of the ManagedCertificate definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, environmentName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @return the next definition stage. @@ -141,36 +150,40 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ ManagedCertificate create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ManagedCertificate create(Context context); } - /** The stage of the ManagedCertificate definition allowing to specify tags. */ + /** + * The stage of the ManagedCertificate definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the ManagedCertificate definition allowing to specify properties. */ + /** + * The stage of the ManagedCertificate definition allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: Certificate resource specific properties. - * + * * @param properties Certificate resource specific properties. * @return the next definition stage. */ @@ -180,36 +193,42 @@ interface WithProperties { /** * Begins update for the ManagedCertificate resource. - * + * * @return the stage of resource update. */ ManagedCertificate.Update update(); - /** The template for ManagedCertificate update. */ + /** + * The template for ManagedCertificate update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ ManagedCertificate apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ManagedCertificate apply(Context context); } - /** The ManagedCertificate update stages. */ + /** + * The ManagedCertificate update stages. + */ interface UpdateStages { - /** The stage of the ManagedCertificate update allowing to specify tags. */ + /** + * The stage of the ManagedCertificate update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Application-specific metadata in the form of key-value pairs.. - * + * * @param tags Application-specific metadata in the form of key-value pairs. * @return the next definition stage. */ @@ -219,14 +238,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ManagedCertificate refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateCollection.java index 5f331c81666df..c086402c97060 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of Managed Certificates. */ +/** + * Collection of Managed Certificates. + */ @Fluent public final class ManagedCertificateCollection { /* @@ -25,13 +27,15 @@ public final class ManagedCertificateCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ManagedCertificateCollection class. */ + /** + * Creates an instance of ManagedCertificateCollection class. + */ public ManagedCertificateCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the ManagedCertificateCollection object itself. */ @@ -51,7 +55,7 @@ public ManagedCertificateCollection withValue(List valu /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,15 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model ManagedCertificateCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ManagedCertificateCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateDomainControlValidation.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateDomainControlValidation.java index 291c8a1efe6f2..5b63fa644a274 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateDomainControlValidation.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateDomainControlValidation.java @@ -8,21 +8,29 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Selected type of domain control validation for managed certificates. */ +/** + * Selected type of domain control validation for managed certificates. + */ public final class ManagedCertificateDomainControlValidation extends ExpandableStringEnum { - /** Static value CNAME for ManagedCertificateDomainControlValidation. */ + /** + * Static value CNAME for ManagedCertificateDomainControlValidation. + */ public static final ManagedCertificateDomainControlValidation CNAME = fromString("CNAME"); - /** Static value HTTP for ManagedCertificateDomainControlValidation. */ + /** + * Static value HTTP for ManagedCertificateDomainControlValidation. + */ public static final ManagedCertificateDomainControlValidation HTTP = fromString("HTTP"); - /** Static value TXT for ManagedCertificateDomainControlValidation. */ + /** + * Static value TXT for ManagedCertificateDomainControlValidation. + */ public static final ManagedCertificateDomainControlValidation TXT = fromString("TXT"); /** * Creates a new instance of ManagedCertificateDomainControlValidation value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -31,7 +39,7 @@ public ManagedCertificateDomainControlValidation() { /** * Creates or finds a ManagedCertificateDomainControlValidation from its string representation. - * + * * @param name a name to look for. * @return the corresponding ManagedCertificateDomainControlValidation. */ @@ -42,7 +50,7 @@ public static ManagedCertificateDomainControlValidation fromString(String name) /** * Gets known ManagedCertificateDomainControlValidation values. - * + * * @return known ManagedCertificateDomainControlValidation values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificatePatch.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificatePatch.java index b13c65047e596..ccd81e9339061 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificatePatch.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificatePatch.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** A managed certificate to update. */ +/** + * A managed certificate to update. + */ @Fluent public final class ManagedCertificatePatch { /* @@ -19,13 +21,15 @@ public final class ManagedCertificatePatch { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of ManagedCertificatePatch class. */ + /** + * Creates an instance of ManagedCertificatePatch class. + */ public ManagedCertificatePatch() { } /** * Get the tags property: Application-specific metadata in the form of key-value pairs. - * + * * @return the tags value. */ public Map tags() { @@ -34,7 +38,7 @@ public Map tags() { /** * Set the tags property: Application-specific metadata in the form of key-value pairs. - * + * * @param tags the tags value to set. * @return the ManagedCertificatePatch object itself. */ @@ -45,7 +49,7 @@ public ManagedCertificatePatch withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateProperties.java index cafc6a8482430..23f1a397b074f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificateProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Certificate resource specific properties. */ +/** + * Certificate resource specific properties. + */ @Fluent public final class ManagedCertificateProperties { /* @@ -40,13 +42,15 @@ public final class ManagedCertificateProperties { @JsonProperty(value = "validationToken", access = JsonProperty.Access.WRITE_ONLY) private String validationToken; - /** Creates an instance of ManagedCertificateProperties class. */ + /** + * Creates an instance of ManagedCertificateProperties class. + */ public ManagedCertificateProperties() { } /** * Get the provisioningState property: Provisioning state of the certificate. - * + * * @return the provisioningState value. */ public CertificateProvisioningState provisioningState() { @@ -55,7 +59,7 @@ public CertificateProvisioningState provisioningState() { /** * Get the subjectName property: Subject name of the certificate. - * + * * @return the subjectName value. */ public String subjectName() { @@ -64,7 +68,7 @@ public String subjectName() { /** * Set the subjectName property: Subject name of the certificate. - * + * * @param subjectName the subjectName value to set. * @return the ManagedCertificateProperties object itself. */ @@ -75,7 +79,7 @@ public ManagedCertificateProperties withSubjectName(String subjectName) { /** * Get the error property: Any error occurred during the certificate provision. - * + * * @return the error value. */ public String error() { @@ -84,7 +88,7 @@ public String error() { /** * Get the domainControlValidation property: Selected type of domain control validation for managed certificates. - * + * * @return the domainControlValidation value. */ public ManagedCertificateDomainControlValidation domainControlValidation() { @@ -93,12 +97,12 @@ public ManagedCertificateDomainControlValidation domainControlValidation() { /** * Set the domainControlValidation property: Selected type of domain control validation for managed certificates. - * + * * @param domainControlValidation the domainControlValidation value to set. * @return the ManagedCertificateProperties object itself. */ - public ManagedCertificateProperties withDomainControlValidation( - ManagedCertificateDomainControlValidation domainControlValidation) { + public ManagedCertificateProperties + withDomainControlValidation(ManagedCertificateDomainControlValidation domainControlValidation) { this.domainControlValidation = domainControlValidation; return this; } @@ -106,7 +110,7 @@ public ManagedCertificateProperties withDomainControlValidation( /** * Get the validationToken property: A TXT token used for DNS TXT domain control validation when issuing this type * of managed certificates. - * + * * @return the validationToken value. */ public String validationToken() { @@ -115,7 +119,7 @@ public String validationToken() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificates.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificates.java index b8dddf1b849fe..6a41e1d6758cc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificates.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedCertificates.java @@ -8,33 +8,35 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ManagedCertificates. */ +/** + * Resource collection API of ManagedCertificates. + */ public interface ManagedCertificates { /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Managed Certificate along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, + String managedCertificateName, Context context); /** * Get the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Managed Certificate. */ @@ -42,41 +44,41 @@ Response getWithResponse( /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String environmentName, String managedCertificateName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String managedCertificateName, + Context context); /** * Deletes the specified Managed Certificate. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param managedCertificateName Name of the Managed Certificate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String environmentName, String managedCertificateName); /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Managed Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -84,13 +86,13 @@ Response deleteWithResponse( /** * Get the Managed Certificates in a given managed environment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Managed Certificates in a given managed environment as paginated response with {@link PagedIterable}. */ @@ -98,11 +100,11 @@ Response deleteWithResponse( /** * Get the specified Managed Certificate. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Managed Certificate along with {@link Response}. */ @@ -110,12 +112,12 @@ Response deleteWithResponse( /** * Get the specified Managed Certificate. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified Managed Certificate along with {@link Response}. */ @@ -123,23 +125,23 @@ Response deleteWithResponse( /** * Deletes the specified Managed Certificate. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Deletes the specified Managed Certificate. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @@ -147,7 +149,7 @@ Response deleteWithResponse( /** * Begins definition for a new ManagedCertificate resource. - * + * * @param name resource name. * @return the first stage of the new ManagedCertificate definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironment.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironment.java index b4064c98a6bbe..1999293a62837 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironment.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironment.java @@ -12,60 +12,70 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of ManagedEnvironment. */ +/** + * An immutable client-side representation of ManagedEnvironment. + */ public interface ManagedEnvironment { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the kind property: Kind of the Environment. - * + * * @return the kind value. */ String kind(); + /** + * Gets the identity property: Managed identities for the Managed Environment to interact with other Azure services + * without maintaining any secrets or credentials in code. + * + * @return the identity value. + */ + ManagedServiceIdentity identity(); + /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning state of the Environment. - * + * * @return the provisioningState value. */ EnvironmentProvisioningState provisioningState(); @@ -73,7 +83,7 @@ public interface ManagedEnvironment { /** * Gets the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export Service to * Service communication telemetry. - * + * * @return the daprAIInstrumentationKey value. */ String daprAIInstrumentationKey(); @@ -81,85 +91,100 @@ public interface ManagedEnvironment { /** * Gets the daprAIConnectionString property: Application Insights connection string used by Dapr to export Service * to Service communication telemetry. - * + * * @return the daprAIConnectionString value. */ String daprAIConnectionString(); /** * Gets the vnetConfiguration property: Vnet configuration for the environment. - * + * * @return the vnetConfiguration value. */ VnetConfiguration vnetConfiguration(); /** * Gets the deploymentErrors property: Any errors that occurred during deployment or deployment validation. - * + * * @return the deploymentErrors value. */ String deploymentErrors(); /** * Gets the defaultDomain property: Default Domain Name for the cluster. - * + * * @return the defaultDomain value. */ String defaultDomain(); /** * Gets the staticIp property: Static IP of the Environment. - * + * * @return the staticIp value. */ String staticIp(); /** - * Gets the appLogsConfiguration property: Cluster configuration which enables the log daemon to export app logs to - * a destination. Currently only "log-analytics" is supported. - * + * Gets the appLogsConfiguration property: Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. + * * @return the appLogsConfiguration value. */ AppLogsConfiguration appLogsConfiguration(); + /** + * Gets the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @return the appInsightsConfiguration value. + */ + AppInsightsConfiguration appInsightsConfiguration(); + + /** + * Gets the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @return the openTelemetryConfiguration value. + */ + OpenTelemetryConfiguration openTelemetryConfiguration(); + /** * Gets the zoneRedundant property: Whether or not this Managed Environment is zone-redundant. - * + * * @return the zoneRedundant value. */ Boolean zoneRedundant(); /** * Gets the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @return the customDomainConfiguration value. */ CustomDomainConfiguration customDomainConfiguration(); /** * Gets the eventStreamEndpoint property: The endpoint of the eventstream of the Environment. - * + * * @return the eventStreamEndpoint value. */ String eventStreamEndpoint(); /** * Gets the workloadProfiles property: Workload profiles configured for the Managed Environment. - * + * * @return the workloadProfiles value. */ List workloadProfiles(); /** * Gets the kedaConfiguration property: The configuration of Keda component. - * + * * @return the kedaConfiguration value. */ KedaConfiguration kedaConfiguration(); /** * Gets the daprConfiguration property: The configuration of Dapr component. - * + * * @return the daprConfiguration value. */ DaprConfiguration daprConfiguration(); @@ -168,65 +193,70 @@ public interface ManagedEnvironment { * Gets the infrastructureResourceGroup property: Name of the platform-managed resource group created for the * Managed Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be * created in the same subscription as the subnet. - * + * * @return the infrastructureResourceGroup value. */ String infrastructureResourceGroup(); /** * Gets the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @return the peerAuthentication value. */ ManagedEnvironmentPropertiesPeerAuthentication peerAuthentication(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentInner object. - * + * * @return the inner object. */ ManagedEnvironmentInner innerModel(); - /** The entirety of the ManagedEnvironment definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the ManagedEnvironment definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The ManagedEnvironment definition stages. */ + /** + * The ManagedEnvironment definition stages. + */ interface DefinitionStages { - /** The first stage of the ManagedEnvironment definition. */ + /** + * The first stage of the ManagedEnvironment definition. + */ interface Blank extends WithLocation { } - /** The stage of the ManagedEnvironment definition allowing to specify location. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -234,18 +264,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the ManagedEnvironment definition allowing to specify parent resource. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -256,183 +288,246 @@ interface WithResourceGroup { * The stage of the ManagedEnvironment definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithKind, - DefinitionStages.WithDaprAIInstrumentationKey, - DefinitionStages.WithDaprAIConnectionString, - DefinitionStages.WithVnetConfiguration, - DefinitionStages.WithAppLogsConfiguration, - DefinitionStages.WithZoneRedundant, - DefinitionStages.WithCustomDomainConfiguration, - DefinitionStages.WithWorkloadProfiles, - DefinitionStages.WithKedaConfiguration, - DefinitionStages.WithDaprConfiguration, - DefinitionStages.WithInfrastructureResourceGroup, - DefinitionStages.WithPeerAuthentication { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithKind, + DefinitionStages.WithIdentity, DefinitionStages.WithDaprAIInstrumentationKey, + DefinitionStages.WithDaprAIConnectionString, DefinitionStages.WithVnetConfiguration, + DefinitionStages.WithAppLogsConfiguration, DefinitionStages.WithAppInsightsConfiguration, + DefinitionStages.WithOpenTelemetryConfiguration, DefinitionStages.WithZoneRedundant, + DefinitionStages.WithCustomDomainConfiguration, DefinitionStages.WithWorkloadProfiles, + DefinitionStages.WithKedaConfiguration, DefinitionStages.WithDaprConfiguration, + DefinitionStages.WithInfrastructureResourceGroup, DefinitionStages.WithPeerAuthentication { /** * Executes the create request. - * + * * @return the created resource. */ ManagedEnvironment create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ManagedEnvironment create(Context context); } - /** The stage of the ManagedEnvironment definition allowing to specify tags. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the ManagedEnvironment definition allowing to specify kind. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify kind. + */ interface WithKind { /** * Specifies the kind property: Kind of the Environment.. - * + * * @param kind Kind of the Environment. * @return the next definition stage. */ WithCreate withKind(String kind); } - /** The stage of the ManagedEnvironment definition allowing to specify daprAIInstrumentationKey. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify identity. + */ + interface WithIdentity { + /** + * Specifies the identity property: Managed identities for the Managed Environment to interact with other + * Azure services without maintaining any secrets or credentials in code.. + * + * @param identity Managed identities for the Managed Environment to interact with other Azure services + * without maintaining any secrets or credentials in code. + * @return the next definition stage. + */ + WithCreate withIdentity(ManagedServiceIdentity identity); + } + + /** + * The stage of the ManagedEnvironment definition allowing to specify daprAIInstrumentationKey. + */ interface WithDaprAIInstrumentationKey { /** * Specifies the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export * Service to Service communication telemetry. - * + * * @param daprAIInstrumentationKey Azure Monitor instrumentation key used by Dapr to export Service to - * Service communication telemetry. + * Service communication telemetry. * @return the next definition stage. */ WithCreate withDaprAIInstrumentationKey(String daprAIInstrumentationKey); } - /** The stage of the ManagedEnvironment definition allowing to specify daprAIConnectionString. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify daprAIConnectionString. + */ interface WithDaprAIConnectionString { /** * Specifies the daprAIConnectionString property: Application Insights connection string used by Dapr to * export Service to Service communication telemetry. - * + * * @param daprAIConnectionString Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. + * Service communication telemetry. * @return the next definition stage. */ WithCreate withDaprAIConnectionString(String daprAIConnectionString); } - /** The stage of the ManagedEnvironment definition allowing to specify vnetConfiguration. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify vnetConfiguration. + */ interface WithVnetConfiguration { /** * Specifies the vnetConfiguration property: Vnet configuration for the environment. - * + * * @param vnetConfiguration Vnet configuration for the environment. * @return the next definition stage. */ WithCreate withVnetConfiguration(VnetConfiguration vnetConfiguration); } - /** The stage of the ManagedEnvironment definition allowing to specify appLogsConfiguration. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify appLogsConfiguration. + */ interface WithAppLogsConfiguration { /** * Specifies the appLogsConfiguration property: Cluster configuration which enables the log daemon to export - * app logs to a destination. Currently only "log-analytics" is supported. - * - * @param appLogsConfiguration Cluster configuration which enables the log daemon to export app logs to a - * destination. Currently only "log-analytics" is supported. + * app logs to a destination. Currently only "log-analytics" is + * supported. + * + * @param appLogsConfiguration Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. * @return the next definition stage. */ WithCreate withAppLogsConfiguration(AppLogsConfiguration appLogsConfiguration); } - /** The stage of the ManagedEnvironment definition allowing to specify zoneRedundant. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify appInsightsConfiguration. + */ + interface WithAppInsightsConfiguration { + /** + * Specifies the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @param appInsightsConfiguration Environment level Application Insights configuration. + * @return the next definition stage. + */ + WithCreate withAppInsightsConfiguration(AppInsightsConfiguration appInsightsConfiguration); + } + + /** + * The stage of the ManagedEnvironment definition allowing to specify openTelemetryConfiguration. + */ + interface WithOpenTelemetryConfiguration { + /** + * Specifies the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @param openTelemetryConfiguration Environment Open Telemetry configuration. + * @return the next definition stage. + */ + WithCreate withOpenTelemetryConfiguration(OpenTelemetryConfiguration openTelemetryConfiguration); + } + + /** + * The stage of the ManagedEnvironment definition allowing to specify zoneRedundant. + */ interface WithZoneRedundant { /** * Specifies the zoneRedundant property: Whether or not this Managed Environment is zone-redundant.. - * + * * @param zoneRedundant Whether or not this Managed Environment is zone-redundant. * @return the next definition stage. */ WithCreate withZoneRedundant(Boolean zoneRedundant); } - /** The stage of the ManagedEnvironment definition allowing to specify customDomainConfiguration. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify customDomainConfiguration. + */ interface WithCustomDomainConfiguration { /** * Specifies the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration Custom domain configuration for the environment. * @return the next definition stage. */ WithCreate withCustomDomainConfiguration(CustomDomainConfiguration customDomainConfiguration); } - /** The stage of the ManagedEnvironment definition allowing to specify workloadProfiles. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify workloadProfiles. + */ interface WithWorkloadProfiles { /** * Specifies the workloadProfiles property: Workload profiles configured for the Managed Environment.. - * + * * @param workloadProfiles Workload profiles configured for the Managed Environment. * @return the next definition stage. */ WithCreate withWorkloadProfiles(List workloadProfiles); } - /** The stage of the ManagedEnvironment definition allowing to specify kedaConfiguration. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify kedaConfiguration. + */ interface WithKedaConfiguration { /** * Specifies the kedaConfiguration property: The configuration of Keda component.. - * + * * @param kedaConfiguration The configuration of Keda component. * @return the next definition stage. */ WithCreate withKedaConfiguration(KedaConfiguration kedaConfiguration); } - /** The stage of the ManagedEnvironment definition allowing to specify daprConfiguration. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify daprConfiguration. + */ interface WithDaprConfiguration { /** * Specifies the daprConfiguration property: The configuration of Dapr component.. - * + * * @param daprConfiguration The configuration of Dapr component. * @return the next definition stage. */ WithCreate withDaprConfiguration(DaprConfiguration daprConfiguration); } - /** The stage of the ManagedEnvironment definition allowing to specify infrastructureResourceGroup. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify infrastructureResourceGroup. + */ interface WithInfrastructureResourceGroup { /** * Specifies the infrastructureResourceGroup property: Name of the platform-managed resource group created * for the Managed Environment to host infrastructure resources. If a subnet ID is provided, this resource * group will be created in the same subscription as the subnet.. - * + * * @param infrastructureResourceGroup Name of the platform-managed resource group created for the Managed - * Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be - * created in the same subscription as the subnet. + * Environment to host infrastructure resources. If a subnet ID is provided, this resource group will be + * created in the same subscription as the subnet. * @return the next definition stage. */ WithCreate withInfrastructureResourceGroup(String infrastructureResourceGroup); } - /** The stage of the ManagedEnvironment definition allowing to specify peerAuthentication. */ + /** + * The stage of the ManagedEnvironment definition allowing to specify peerAuthentication. + */ interface WithPeerAuthentication { /** * Specifies the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @param peerAuthentication Peer authentication settings for the Managed Environment. * @return the next definition stage. */ @@ -442,163 +537,226 @@ interface WithPeerAuthentication { /** * Begins update for the ManagedEnvironment resource. - * + * * @return the stage of resource update. */ ManagedEnvironment.Update update(); - /** The template for ManagedEnvironment update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithKind, - UpdateStages.WithDaprAIInstrumentationKey, - UpdateStages.WithDaprAIConnectionString, - UpdateStages.WithVnetConfiguration, - UpdateStages.WithAppLogsConfiguration, - UpdateStages.WithCustomDomainConfiguration, - UpdateStages.WithWorkloadProfiles, - UpdateStages.WithKedaConfiguration, - UpdateStages.WithDaprConfiguration, - UpdateStages.WithPeerAuthentication { + /** + * The template for ManagedEnvironment update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithKind, UpdateStages.WithIdentity, + UpdateStages.WithDaprAIInstrumentationKey, UpdateStages.WithDaprAIConnectionString, + UpdateStages.WithVnetConfiguration, UpdateStages.WithAppLogsConfiguration, + UpdateStages.WithAppInsightsConfiguration, UpdateStages.WithOpenTelemetryConfiguration, + UpdateStages.WithCustomDomainConfiguration, UpdateStages.WithWorkloadProfiles, + UpdateStages.WithKedaConfiguration, UpdateStages.WithDaprConfiguration, UpdateStages.WithPeerAuthentication { /** * Executes the update request. - * + * * @return the updated resource. */ ManagedEnvironment apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ManagedEnvironment apply(Context context); } - /** The ManagedEnvironment update stages. */ + /** + * The ManagedEnvironment update stages. + */ interface UpdateStages { - /** The stage of the ManagedEnvironment update allowing to specify tags. */ + /** + * The stage of the ManagedEnvironment update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the ManagedEnvironment update allowing to specify kind. */ + /** + * The stage of the ManagedEnvironment update allowing to specify kind. + */ interface WithKind { /** * Specifies the kind property: Kind of the Environment.. - * + * * @param kind Kind of the Environment. * @return the next definition stage. */ Update withKind(String kind); } - /** The stage of the ManagedEnvironment update allowing to specify daprAIInstrumentationKey. */ + /** + * The stage of the ManagedEnvironment update allowing to specify identity. + */ + interface WithIdentity { + /** + * Specifies the identity property: Managed identities for the Managed Environment to interact with other + * Azure services without maintaining any secrets or credentials in code.. + * + * @param identity Managed identities for the Managed Environment to interact with other Azure services + * without maintaining any secrets or credentials in code. + * @return the next definition stage. + */ + Update withIdentity(ManagedServiceIdentity identity); + } + + /** + * The stage of the ManagedEnvironment update allowing to specify daprAIInstrumentationKey. + */ interface WithDaprAIInstrumentationKey { /** * Specifies the daprAIInstrumentationKey property: Azure Monitor instrumentation key used by Dapr to export * Service to Service communication telemetry. - * + * * @param daprAIInstrumentationKey Azure Monitor instrumentation key used by Dapr to export Service to - * Service communication telemetry. + * Service communication telemetry. * @return the next definition stage. */ Update withDaprAIInstrumentationKey(String daprAIInstrumentationKey); } - /** The stage of the ManagedEnvironment update allowing to specify daprAIConnectionString. */ + /** + * The stage of the ManagedEnvironment update allowing to specify daprAIConnectionString. + */ interface WithDaprAIConnectionString { /** * Specifies the daprAIConnectionString property: Application Insights connection string used by Dapr to * export Service to Service communication telemetry. - * + * * @param daprAIConnectionString Application Insights connection string used by Dapr to export Service to - * Service communication telemetry. + * Service communication telemetry. * @return the next definition stage. */ Update withDaprAIConnectionString(String daprAIConnectionString); } - /** The stage of the ManagedEnvironment update allowing to specify vnetConfiguration. */ + /** + * The stage of the ManagedEnvironment update allowing to specify vnetConfiguration. + */ interface WithVnetConfiguration { /** * Specifies the vnetConfiguration property: Vnet configuration for the environment. - * + * * @param vnetConfiguration Vnet configuration for the environment. * @return the next definition stage. */ Update withVnetConfiguration(VnetConfiguration vnetConfiguration); } - /** The stage of the ManagedEnvironment update allowing to specify appLogsConfiguration. */ + /** + * The stage of the ManagedEnvironment update allowing to specify appLogsConfiguration. + */ interface WithAppLogsConfiguration { /** * Specifies the appLogsConfiguration property: Cluster configuration which enables the log daemon to export - * app logs to a destination. Currently only "log-analytics" is supported. - * - * @param appLogsConfiguration Cluster configuration which enables the log daemon to export app logs to a - * destination. Currently only "log-analytics" is supported. + * app logs to a destination. Currently only "log-analytics" is + * supported. + * + * @param appLogsConfiguration Cluster configuration which enables the log daemon to export + * app logs to a destination. Currently only "log-analytics" is + * supported. * @return the next definition stage. */ Update withAppLogsConfiguration(AppLogsConfiguration appLogsConfiguration); } - /** The stage of the ManagedEnvironment update allowing to specify customDomainConfiguration. */ + /** + * The stage of the ManagedEnvironment update allowing to specify appInsightsConfiguration. + */ + interface WithAppInsightsConfiguration { + /** + * Specifies the appInsightsConfiguration property: Environment level Application Insights configuration. + * + * @param appInsightsConfiguration Environment level Application Insights configuration. + * @return the next definition stage. + */ + Update withAppInsightsConfiguration(AppInsightsConfiguration appInsightsConfiguration); + } + + /** + * The stage of the ManagedEnvironment update allowing to specify openTelemetryConfiguration. + */ + interface WithOpenTelemetryConfiguration { + /** + * Specifies the openTelemetryConfiguration property: Environment Open Telemetry configuration. + * + * @param openTelemetryConfiguration Environment Open Telemetry configuration. + * @return the next definition stage. + */ + Update withOpenTelemetryConfiguration(OpenTelemetryConfiguration openTelemetryConfiguration); + } + + /** + * The stage of the ManagedEnvironment update allowing to specify customDomainConfiguration. + */ interface WithCustomDomainConfiguration { /** * Specifies the customDomainConfiguration property: Custom domain configuration for the environment. - * + * * @param customDomainConfiguration Custom domain configuration for the environment. * @return the next definition stage. */ Update withCustomDomainConfiguration(CustomDomainConfiguration customDomainConfiguration); } - /** The stage of the ManagedEnvironment update allowing to specify workloadProfiles. */ + /** + * The stage of the ManagedEnvironment update allowing to specify workloadProfiles. + */ interface WithWorkloadProfiles { /** * Specifies the workloadProfiles property: Workload profiles configured for the Managed Environment.. - * + * * @param workloadProfiles Workload profiles configured for the Managed Environment. * @return the next definition stage. */ Update withWorkloadProfiles(List workloadProfiles); } - /** The stage of the ManagedEnvironment update allowing to specify kedaConfiguration. */ + /** + * The stage of the ManagedEnvironment update allowing to specify kedaConfiguration. + */ interface WithKedaConfiguration { /** * Specifies the kedaConfiguration property: The configuration of Keda component.. - * + * * @param kedaConfiguration The configuration of Keda component. * @return the next definition stage. */ Update withKedaConfiguration(KedaConfiguration kedaConfiguration); } - /** The stage of the ManagedEnvironment update allowing to specify daprConfiguration. */ + /** + * The stage of the ManagedEnvironment update allowing to specify daprConfiguration. + */ interface WithDaprConfiguration { /** * Specifies the daprConfiguration property: The configuration of Dapr component.. - * + * * @param daprConfiguration The configuration of Dapr component. * @return the next definition stage. */ Update withDaprConfiguration(DaprConfiguration daprConfiguration); } - /** The stage of the ManagedEnvironment update allowing to specify peerAuthentication. */ + /** + * The stage of the ManagedEnvironment update allowing to specify peerAuthentication. + */ interface WithPeerAuthentication { /** * Specifies the peerAuthentication property: Peer authentication settings for the Managed Environment. - * + * * @param peerAuthentication Peer authentication settings for the Managed Environment. * @return the next definition stage. */ @@ -608,14 +766,14 @@ interface WithPeerAuthentication { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ManagedEnvironment refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -623,13 +781,13 @@ interface WithPeerAuthentication { /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return environment Auth Token along with {@link Response}. */ @@ -637,11 +795,11 @@ interface WithPeerAuthentication { /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return environment Auth Token. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentDiagnostics.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentDiagnostics.java index b3e3c06ab8793..03a0cb082365d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentDiagnostics.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentDiagnostics.java @@ -7,36 +7,38 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ManagedEnvironmentDiagnostics. */ +/** + * Resource collection API of ManagedEnvironmentDiagnostics. + */ public interface ManagedEnvironmentDiagnostics { /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of diagnostics for a Managed Environment used to host container apps along with {@link - * Response}. + * @return the list of diagnostics for a Managed Environment used to host container apps along with + * {@link Response}. */ - Response listDetectorsWithResponse( - String resourceGroupName, String environmentName, Context context); + Response listDetectorsWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get the list of diagnostics for a given Managed Environment. - * - *

Get the list of diagnostics for a Managed Environment used to host container apps. - * + * + * Get the list of diagnostics for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of diagnostics for a Managed Environment used to host container apps. */ @@ -44,33 +46,33 @@ Response listDetectorsWithResponse( /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps along with {@link Response}. */ - Response getDetectorWithResponse( - String resourceGroupName, String environmentName, String detectorName, Context context); + Response getDetectorWithResponse(String resourceGroupName, String environmentName, String detectorName, + Context context); /** * Get the diagnostics data for a given Managed Environment. - * - *

Get the diagnostics data for a Managed Environment used to host container apps. - * + * + * Get the diagnostics data for a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param detectorName Name of the Managed Environment detector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the diagnostics data for a Managed Environment used to host container apps. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentPropertiesPeerAuthentication.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentPropertiesPeerAuthentication.java index ce20f1532b009..53f2f9a2a4caa 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentPropertiesPeerAuthentication.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentPropertiesPeerAuthentication.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Peer authentication settings for the Managed Environment. */ +/** + * Peer authentication settings for the Managed Environment. + */ @Fluent public final class ManagedEnvironmentPropertiesPeerAuthentication { /* @@ -16,13 +18,15 @@ public final class ManagedEnvironmentPropertiesPeerAuthentication { @JsonProperty(value = "mtls") private Mtls mtls; - /** Creates an instance of ManagedEnvironmentPropertiesPeerAuthentication class. */ + /** + * Creates an instance of ManagedEnvironmentPropertiesPeerAuthentication class. + */ public ManagedEnvironmentPropertiesPeerAuthentication() { } /** * Get the mtls property: Mutual TLS authentication settings for the Managed Environment. - * + * * @return the mtls value. */ public Mtls mtls() { @@ -31,7 +35,7 @@ public Mtls mtls() { /** * Set the mtls property: Mutual TLS authentication settings for the Managed Environment. - * + * * @param mtls the mtls value to set. * @return the ManagedEnvironmentPropertiesPeerAuthentication object itself. */ @@ -42,7 +46,7 @@ public ManagedEnvironmentPropertiesPeerAuthentication withMtls(Mtls mtls) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorage.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorage.java index 3e565811a2106..2e1a0fcd4b129 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorage.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorage.java @@ -4,69 +4,87 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStorageInner; -/** An immutable client-side representation of ManagedEnvironmentStorage. */ +/** + * An immutable client-side representation of ManagedEnvironmentStorage. + */ public interface ManagedEnvironmentStorage { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the properties property: Storage properties. - * + * * @return the properties value. */ ManagedEnvironmentStorageProperties properties(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStorageInner object. - * + * * @return the inner object. */ ManagedEnvironmentStorageInner innerModel(); - /** The entirety of the ManagedEnvironmentStorage definition. */ + /** + * The entirety of the ManagedEnvironmentStorage definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The ManagedEnvironmentStorage definition stages. */ + /** + * The ManagedEnvironmentStorage definition stages. + */ interface DefinitionStages { - /** The first stage of the ManagedEnvironmentStorage definition. */ + /** + * The first stage of the ManagedEnvironmentStorage definition. + */ interface Blank extends WithParentResource { } - /** The stage of the ManagedEnvironmentStorage definition allowing to specify parent resource. */ + /** + * The stage of the ManagedEnvironmentStorage definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, environmentName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @return the next definition stage. @@ -81,25 +99,27 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithProperties { /** * Executes the create request. - * + * * @return the created resource. */ ManagedEnvironmentStorage create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ManagedEnvironmentStorage create(Context context); } - /** The stage of the ManagedEnvironmentStorage definition allowing to specify properties. */ + /** + * The stage of the ManagedEnvironmentStorage definition allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: Storage properties. - * + * * @param properties Storage properties. * @return the next definition stage. */ @@ -109,36 +129,42 @@ interface WithProperties { /** * Begins update for the ManagedEnvironmentStorage resource. - * + * * @return the stage of resource update. */ ManagedEnvironmentStorage.Update update(); - /** The template for ManagedEnvironmentStorage update. */ + /** + * The template for ManagedEnvironmentStorage update. + */ interface Update extends UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ ManagedEnvironmentStorage apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ManagedEnvironmentStorage apply(Context context); } - /** The ManagedEnvironmentStorage update stages. */ + /** + * The ManagedEnvironmentStorage update stages. + */ interface UpdateStages { - /** The stage of the ManagedEnvironmentStorage update allowing to specify properties. */ + /** + * The stage of the ManagedEnvironmentStorage update allowing to specify properties. + */ interface WithProperties { /** * Specifies the properties property: Storage properties. - * + * * @param properties Storage properties. * @return the next definition stage. */ @@ -148,14 +174,14 @@ interface WithProperties { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ManagedEnvironmentStorage refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorageProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorageProperties.java index 047e29ebf282a..ca7eaef51f421 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorageProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStorageProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Storage properties. */ +/** + * Storage properties. + */ @Fluent public final class ManagedEnvironmentStorageProperties { /* @@ -16,13 +18,21 @@ public final class ManagedEnvironmentStorageProperties { @JsonProperty(value = "azureFile") private AzureFileProperties azureFile; - /** Creates an instance of ManagedEnvironmentStorageProperties class. */ + /* + * NFS Azure file properties + */ + @JsonProperty(value = "nfsAzureFile") + private NfsAzureFileProperties nfsAzureFile; + + /** + * Creates an instance of ManagedEnvironmentStorageProperties class. + */ public ManagedEnvironmentStorageProperties() { } /** * Get the azureFile property: Azure file properties. - * + * * @return the azureFile value. */ public AzureFileProperties azureFile() { @@ -31,7 +41,7 @@ public AzureFileProperties azureFile() { /** * Set the azureFile property: Azure file properties. - * + * * @param azureFile the azureFile value to set. * @return the ManagedEnvironmentStorageProperties object itself. */ @@ -40,14 +50,37 @@ public ManagedEnvironmentStorageProperties withAzureFile(AzureFileProperties azu return this; } + /** + * Get the nfsAzureFile property: NFS Azure file properties. + * + * @return the nfsAzureFile value. + */ + public NfsAzureFileProperties nfsAzureFile() { + return this.nfsAzureFile; + } + + /** + * Set the nfsAzureFile property: NFS Azure file properties. + * + * @param nfsAzureFile the nfsAzureFile value to set. + * @return the ManagedEnvironmentStorageProperties object itself. + */ + public ManagedEnvironmentStorageProperties withNfsAzureFile(NfsAzureFileProperties nfsAzureFile) { + this.nfsAzureFile = nfsAzureFile; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (azureFile() != null) { azureFile().validate(); } + if (nfsAzureFile() != null) { + nfsAzureFile().validate(); + } } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStoragesCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStoragesCollection.java index c6572a452c8b7..2e891809bbce1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStoragesCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentStoragesCollection.java @@ -7,11 +7,13 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStoragesCollectionInner; import java.util.List; -/** An immutable client-side representation of ManagedEnvironmentStoragesCollection. */ +/** + * An immutable client-side representation of ManagedEnvironmentStoragesCollection. + */ public interface ManagedEnvironmentStoragesCollection { /** * Gets the value property: Collection of storage resources. - * + * * @return the value value. */ List value(); @@ -19,7 +21,7 @@ public interface ManagedEnvironmentStoragesCollection { /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStoragesCollectionInner * object. - * + * * @return the inner object. */ ManagedEnvironmentStoragesCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentUsages.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentUsages.java new file mode 100644 index 0000000000000..c9c1d12477fe9 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentUsages.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of ManagedEnvironmentUsages. + */ +public interface ManagedEnvironmentUsages { + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String environmentName); + + /** + * Gets the current usage information as well as the limits for environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param environmentName Name of the Environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the current usage information as well as the limits for environment as paginated response with + * {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String environmentName, Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironments.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironments.java index 982eb2b0c3f5e..cf2ffc7fc4f83 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironments.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironments.java @@ -8,15 +8,17 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ManagedEnvironments. */ +/** + * Resource collection API of ManagedEnvironments. + */ public interface ManagedEnvironments { /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription as paginated response with {@link PagedIterable}. */ @@ -24,13 +26,13 @@ public interface ManagedEnvironments { /** * Get all Environments for a subscription. - * - *

Get all Managed Environments for a subscription. - * + * + * Get all Managed Environments for a subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all Managed Environments for a subscription as paginated response with {@link PagedIterable}. */ @@ -38,13 +40,13 @@ public interface ManagedEnvironments { /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the Managed Environments in a resource group as paginated response with {@link PagedIterable}. */ @@ -52,14 +54,14 @@ public interface ManagedEnvironments { /** * Get all the Environments in a resource group. - * - *

Get all the Managed Environments in a resource group. - * + * + * Get all the Managed Environments in a resource group. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the Managed Environments in a resource group as paginated response with {@link PagedIterable}. */ @@ -67,31 +69,31 @@ public interface ManagedEnvironments { /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String environmentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps. */ @@ -99,60 +101,60 @@ Response getByResourceGroupWithResponse( /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByResourceGroup(String resourceGroupName, String environmentName); /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String environmentName, Context context); /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return environment Auth Token along with {@link Response}. */ - Response getAuthTokenWithResponse( - String resourceGroupName, String environmentName, Context context); + Response getAuthTokenWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get auth token for a managed environment - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return environment Auth Token. */ @@ -160,14 +162,14 @@ Response getAuthTokenWithResponse( /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedIterable}. */ @@ -175,30 +177,30 @@ Response getAuthTokenWithResponse( /** * Get all workload Profile States for a Managed Environment.. - * - *

Get all workload Profile States for a Managed Environment. - * + * + * Get all workload Profile States for a Managed Environment. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all workload Profile States for a Managed Environment as paginated response with {@link PagedIterable}. */ - PagedIterable listWorkloadProfileStates( - String resourceGroupName, String environmentName, Context context); + PagedIterable listWorkloadProfileStates(String resourceGroupName, String environmentName, + Context context); /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @@ -206,14 +208,14 @@ PagedIterable listWorkloadProfileStates( /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @@ -221,34 +223,34 @@ PagedIterable listWorkloadProfileStates( /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete a Managed Environment. - * - *

Delete a Managed Environment if it does not have any container apps. - * + * + * Delete a Managed Environment if it does not have any container apps. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new ManagedEnvironment resource. - * + * * @param name resource name. * @return the first stage of the new ManagedEnvironment definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsCollection.java index a57a3e082679b..3ef87ab8253b7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of Environments. */ +/** + * Collection of Environments. + */ @Fluent public final class ManagedEnvironmentsCollection { /* @@ -25,13 +27,15 @@ public final class ManagedEnvironmentsCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ManagedEnvironmentsCollection class. */ + /** + * Creates an instance of ManagedEnvironmentsCollection class. + */ public ManagedEnvironmentsCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the ManagedEnvironmentsCollection object itself. */ @@ -51,7 +55,7 @@ public ManagedEnvironmentsCollection withValue(List val /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,15 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model ManagedEnvironmentsCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ManagedEnvironmentsCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsDiagnostics.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsDiagnostics.java index 516893bd6ae09..f4c6fb165f768 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsDiagnostics.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsDiagnostics.java @@ -7,19 +7,21 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ManagedEnvironmentsDiagnostics. */ +/** + * Resource collection API of ManagedEnvironmentsDiagnostics. + */ public interface ManagedEnvironmentsDiagnostics { /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps along with {@link Response}. */ @@ -27,14 +29,14 @@ public interface ManagedEnvironmentsDiagnostics { /** * Get the properties of a Managed Environment. - * - *

Get the properties of a Managed Environment used to host container apps. - * + * + * Get the properties of a Managed Environment used to host container apps. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the properties of a Managed Environment used to host container apps. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsStorages.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsStorages.java index 4ee96342baeeb..a1be60b7e13a8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsStorages.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedEnvironmentsStorages.java @@ -7,31 +7,33 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ManagedEnvironmentsStorages. */ +/** + * Resource collection API of ManagedEnvironmentsStorages. + */ public interface ManagedEnvironmentsStorages { /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a managedEnvironment along with {@link Response}. */ - Response listWithResponse( - String resourceGroupName, String environmentName, Context context); + Response listWithResponse(String resourceGroupName, String environmentName, + Context context); /** * Get all storages for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all storages for a managedEnvironment. */ @@ -39,29 +41,29 @@ Response listWithResponse( /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a managedEnvironment along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context); + Response getWithResponse(String resourceGroupName, String environmentName, + String storageName, Context context); /** * Get storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a managedEnvironment. */ @@ -69,40 +71,40 @@ Response getWithResponse( /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String environmentName, String storageName, Context context); + Response deleteWithResponse(String resourceGroupName, String environmentName, String storageName, + Context context); /** * Delete storage for a managedEnvironment. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Environment. * @param storageName Name of the storage. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void delete(String resourceGroupName, String environmentName, String storageName); /** * Get storage for a managedEnvironment. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a managedEnvironment along with {@link Response}. */ @@ -110,12 +112,12 @@ Response deleteWithResponse( /** * Get storage for a managedEnvironment. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return storage for a managedEnvironment along with {@link Response}. */ @@ -123,23 +125,23 @@ Response deleteWithResponse( /** * Delete storage for a managedEnvironment. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void deleteById(String id); /** * Delete storage for a managedEnvironment. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ @@ -147,7 +149,7 @@ Response deleteWithResponse( /** * Begins definition for a new ManagedEnvironmentStorage resource. - * + * * @param name resource name. * @return the first stage of the new ManagedEnvironmentStorage definition. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentity.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentity.java index 0b21b71ef2e41..d4f2c257c0156 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentity.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentity.java @@ -11,7 +11,9 @@ import java.util.Map; import java.util.UUID; -/** Managed service identity (system assigned and/or user assigned identities). */ +/** + * Managed service identity (system assigned and/or user assigned identities). + */ @Fluent public final class ManagedServiceIdentity { /* @@ -37,21 +39,24 @@ public final class ManagedServiceIdentity { /* * The set of user assigned identities associated with the resource. The userAssignedIdentities dictionary keys * will be ARM resource ids in the form: - * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. + * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/ + * userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. */ @JsonProperty(value = "userAssignedIdentities") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map userAssignedIdentities; - /** Creates an instance of ManagedServiceIdentity class. */ + /** + * Creates an instance of ManagedServiceIdentity class. + */ public ManagedServiceIdentity() { } /** * Get the principalId property: The service principal ID of the system assigned identity. This property will only * be provided for a system assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -59,9 +64,9 @@ public UUID principalId() { } /** - * Get the tenantId property: The tenant ID of the system assigned identity. This property will only be provided for - * a system assigned identity. - * + * Get the tenantId property: The tenant ID of the system assigned identity. This property will only be provided + * for a system assigned identity. + * * @return the tenantId value. */ public UUID tenantId() { @@ -71,7 +76,7 @@ public UUID tenantId() { /** * Get the type property: Type of managed service identity (where both SystemAssigned and UserAssigned types are * allowed). - * + * * @return the type value. */ public ManagedServiceIdentityType type() { @@ -81,7 +86,7 @@ public ManagedServiceIdentityType type() { /** * Set the type property: Type of managed service identity (where both SystemAssigned and UserAssigned types are * allowed). - * + * * @param type the type value to set. * @return the ManagedServiceIdentity object itself. */ @@ -95,7 +100,7 @@ public ManagedServiceIdentity withType(ManagedServiceIdentityType type) { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @return the userAssignedIdentities value. */ public Map userAssignedIdentities() { @@ -107,7 +112,7 @@ public Map userAssignedIdentities() { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @param userAssignedIdentities the userAssignedIdentities value to set. * @return the ManagedServiceIdentity object itself. */ @@ -118,24 +123,20 @@ public ManagedServiceIdentity withUserAssignedIdentities(Map { - if (e != null) { - e.validate(); - } - }); + userAssignedIdentities().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentityType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentityType.java index 26bdab801b0b7..5b9996316a961 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentityType.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ManagedServiceIdentityType.java @@ -8,24 +8,34 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Type of managed service identity (where both SystemAssigned and UserAssigned types are allowed). */ +/** + * Type of managed service identity (where both SystemAssigned and UserAssigned types are allowed). + */ public final class ManagedServiceIdentityType extends ExpandableStringEnum { - /** Static value None for ManagedServiceIdentityType. */ + /** + * Static value None for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType NONE = fromString("None"); - /** Static value SystemAssigned for ManagedServiceIdentityType. */ + /** + * Static value SystemAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType SYSTEM_ASSIGNED = fromString("SystemAssigned"); - /** Static value UserAssigned for ManagedServiceIdentityType. */ + /** + * Static value UserAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType USER_ASSIGNED = fromString("UserAssigned"); - /** Static value SystemAssigned,UserAssigned for ManagedServiceIdentityType. */ - public static final ManagedServiceIdentityType SYSTEM_ASSIGNED_USER_ASSIGNED = - fromString("SystemAssigned,UserAssigned"); + /** + * Static value SystemAssigned,UserAssigned for ManagedServiceIdentityType. + */ + public static final ManagedServiceIdentityType SYSTEM_ASSIGNED_USER_ASSIGNED + = fromString("SystemAssigned,UserAssigned"); /** * Creates a new instance of ManagedServiceIdentityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -34,7 +44,7 @@ public ManagedServiceIdentityType() { /** * Creates or finds a ManagedServiceIdentityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ManagedServiceIdentityType. */ @@ -45,7 +55,7 @@ public static ManagedServiceIdentityType fromString(String name) { /** * Gets known ManagedServiceIdentityType values. - * + * * @return known ManagedServiceIdentityType values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/MetricsConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/MetricsConfiguration.java new file mode 100644 index 0000000000000..8087166e8689e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/MetricsConfiguration.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration of Open Telemetry metrics. + */ +@Fluent +public final class MetricsConfiguration { + /* + * Open telemetry metrics destinations + */ + @JsonProperty(value = "destinations") + private List destinations; + + /** + * Creates an instance of MetricsConfiguration class. + */ + public MetricsConfiguration() { + } + + /** + * Get the destinations property: Open telemetry metrics destinations. + * + * @return the destinations value. + */ + public List destinations() { + return this.destinations; + } + + /** + * Set the destinations property: Open telemetry metrics destinations. + * + * @param destinations the destinations value to set. + * @return the MetricsConfiguration object itself. + */ + public MetricsConfiguration withDestinations(List destinations) { + this.destinations = destinations; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Mtls.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Mtls.java index 8c66d5f1c720d..cc7e238c927cd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Mtls.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Mtls.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration properties for mutual TLS authentication. */ +/** + * Configuration properties for mutual TLS authentication. + */ @Fluent public final class Mtls { /* @@ -16,13 +18,15 @@ public final class Mtls { @JsonProperty(value = "enabled") private Boolean enabled; - /** Creates an instance of Mtls class. */ + /** + * Creates an instance of Mtls class. + */ public Mtls() { } /** * Get the enabled property: Boolean indicating whether the mutual TLS authentication is enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -31,7 +35,7 @@ public Boolean enabled() { /** * Set the enabled property: Boolean indicating whether the mutual TLS authentication is enabled. - * + * * @param enabled the enabled value to set. * @return the Mtls object itself. */ @@ -42,7 +46,7 @@ public Mtls withEnabled(Boolean enabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Namespaces.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Namespaces.java index bf527d3027b7b..48776943b50c2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Namespaces.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Namespaces.java @@ -7,43 +7,42 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Namespaces. */ +/** + * Resource collection API of Namespaces. + */ public interface Namespaces { /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result along with {@link Response}. */ - Response checkNameAvailabilityWithResponse( - String resourceGroupName, - String environmentName, - CheckNameAvailabilityRequest checkNameAvailabilityRequest, - Context context); + Response checkNameAvailabilityWithResponse(String resourceGroupName, + String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest, Context context); /** * Checks the resource name availability. - * - *

Checks if resource name is available. - * + * + * Checks if resource name is available. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param environmentName Name of the Managed Environment. * @param checkNameAvailabilityRequest The check name availability request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the check availability result. */ - CheckNameAvailabilityResponse checkNameAvailability( - String resourceGroupName, String environmentName, CheckNameAvailabilityRequest checkNameAvailabilityRequest); + CheckNameAvailabilityResponse checkNameAvailability(String resourceGroupName, String environmentName, + CheckNameAvailabilityRequest checkNameAvailabilityRequest); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/NfsAzureFileProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/NfsAzureFileProperties.java new file mode 100644 index 0000000000000..269e2d0149a49 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/NfsAzureFileProperties.java @@ -0,0 +1,106 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * NFS Azure File Properties. + */ +@Fluent +public final class NfsAzureFileProperties { + /* + * Server for NFS azure file. + */ + @JsonProperty(value = "server") + private String server; + + /* + * Access mode for storage + */ + @JsonProperty(value = "accessMode") + private AccessMode accessMode; + + /* + * NFS Azure file share name. + */ + @JsonProperty(value = "shareName") + private String shareName; + + /** + * Creates an instance of NfsAzureFileProperties class. + */ + public NfsAzureFileProperties() { + } + + /** + * Get the server property: Server for NFS azure file. + * + * @return the server value. + */ + public String server() { + return this.server; + } + + /** + * Set the server property: Server for NFS azure file. + * + * @param server the server value to set. + * @return the NfsAzureFileProperties object itself. + */ + public NfsAzureFileProperties withServer(String server) { + this.server = server; + return this; + } + + /** + * Get the accessMode property: Access mode for storage. + * + * @return the accessMode value. + */ + public AccessMode accessMode() { + return this.accessMode; + } + + /** + * Set the accessMode property: Access mode for storage. + * + * @param accessMode the accessMode value to set. + * @return the NfsAzureFileProperties object itself. + */ + public NfsAzureFileProperties withAccessMode(AccessMode accessMode) { + this.accessMode = accessMode; + return this; + } + + /** + * Get the shareName property: NFS Azure file share name. + * + * @return the shareName value. + */ + public String shareName() { + return this.shareName; + } + + /** + * Set the shareName property: NFS Azure file share name. + * + * @param shareName the shareName value to set. + * @return the NfsAzureFileProperties object itself. + */ + public NfsAzureFileProperties withShareName(String shareName) { + this.shareName = shareName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Nonce.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Nonce.java index 92dfa219b508e..804cdd2e45559 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Nonce.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Nonce.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the nonce used in the login flow. */ +/** + * The configuration settings of the nonce used in the login flow. + */ @Fluent public final class Nonce { /* @@ -23,14 +25,16 @@ public final class Nonce { @JsonProperty(value = "nonceExpirationInterval") private String nonceExpirationInterval; - /** Creates an instance of Nonce class. */ + /** + * Creates an instance of Nonce class. + */ public Nonce() { } /** * Get the validateNonce property: <code>false</code> if the nonce should not be validated while * completing the login flow; otherwise, <code>true</code>. - * + * * @return the validateNonce value. */ public Boolean validateNonce() { @@ -40,7 +44,7 @@ public Boolean validateNonce() { /** * Set the validateNonce property: <code>false</code> if the nonce should not be validated while * completing the login flow; otherwise, <code>true</code>. - * + * * @param validateNonce the validateNonce value to set. * @return the Nonce object itself. */ @@ -51,7 +55,7 @@ public Nonce withValidateNonce(Boolean validateNonce) { /** * Get the nonceExpirationInterval property: The time after the request is made when the nonce should expire. - * + * * @return the nonceExpirationInterval value. */ public String nonceExpirationInterval() { @@ -60,7 +64,7 @@ public String nonceExpirationInterval() { /** * Set the nonceExpirationInterval property: The time after the request is made when the nonce should expire. - * + * * @param nonceExpirationInterval the nonceExpirationInterval value to set. * @return the Nonce object itself. */ @@ -71,7 +75,7 @@ public Nonce withNonceExpirationInterval(String nonceExpirationInterval) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectClientCredential.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectClientCredential.java index 2331e42f625ec..9bfacd2229f58 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectClientCredential.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectClientCredential.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The authentication client credentials of the custom Open ID Connect provider. */ +/** + * The authentication client credentials of the custom Open ID Connect provider. + */ @Fluent public final class OpenIdConnectClientCredential { /* @@ -22,13 +24,15 @@ public final class OpenIdConnectClientCredential { @JsonProperty(value = "clientSecretSettingName") private String clientSecretSettingName; - /** Creates an instance of OpenIdConnectClientCredential class. */ + /** + * Creates an instance of OpenIdConnectClientCredential class. + */ public OpenIdConnectClientCredential() { } /** * Get the method property: The method that should be used to authenticate the user. - * + * * @return the method value. */ public ClientCredentialMethod method() { @@ -37,7 +41,7 @@ public ClientCredentialMethod method() { /** * Set the method property: The method that should be used to authenticate the user. - * + * * @param method the method value to set. * @return the OpenIdConnectClientCredential object itself. */ @@ -49,7 +53,7 @@ public OpenIdConnectClientCredential withMethod(ClientCredentialMethod method) { /** * Get the clientSecretSettingName property: The app setting that contains the client secret for the custom Open ID * Connect provider. - * + * * @return the clientSecretSettingName value. */ public String clientSecretSettingName() { @@ -59,7 +63,7 @@ public String clientSecretSettingName() { /** * Set the clientSecretSettingName property: The app setting that contains the client secret for the custom Open ID * Connect provider. - * + * * @param clientSecretSettingName the clientSecretSettingName value to set. * @return the OpenIdConnectClientCredential object itself. */ @@ -70,7 +74,7 @@ public OpenIdConnectClientCredential withClientSecretSettingName(String clientSe /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectConfig.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectConfig.java index 08d9bc0e41fdb..c35be9b8c3fee 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectConfig.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectConfig.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the endpoints used for the custom Open ID Connect provider. */ +/** + * The configuration settings of the endpoints used for the custom Open ID Connect provider. + */ @Fluent public final class OpenIdConnectConfig { /* @@ -40,13 +42,15 @@ public final class OpenIdConnectConfig { @JsonProperty(value = "wellKnownOpenIdConfiguration") private String wellKnownOpenIdConfiguration; - /** Creates an instance of OpenIdConnectConfig class. */ + /** + * Creates an instance of OpenIdConnectConfig class. + */ public OpenIdConnectConfig() { } /** * Get the authorizationEndpoint property: The endpoint to be used to make an authorization request. - * + * * @return the authorizationEndpoint value. */ public String authorizationEndpoint() { @@ -55,7 +59,7 @@ public String authorizationEndpoint() { /** * Set the authorizationEndpoint property: The endpoint to be used to make an authorization request. - * + * * @param authorizationEndpoint the authorizationEndpoint value to set. * @return the OpenIdConnectConfig object itself. */ @@ -66,7 +70,7 @@ public OpenIdConnectConfig withAuthorizationEndpoint(String authorizationEndpoin /** * Get the tokenEndpoint property: The endpoint to be used to request a token. - * + * * @return the tokenEndpoint value. */ public String tokenEndpoint() { @@ -75,7 +79,7 @@ public String tokenEndpoint() { /** * Set the tokenEndpoint property: The endpoint to be used to request a token. - * + * * @param tokenEndpoint the tokenEndpoint value to set. * @return the OpenIdConnectConfig object itself. */ @@ -86,7 +90,7 @@ public OpenIdConnectConfig withTokenEndpoint(String tokenEndpoint) { /** * Get the issuer property: The endpoint that issues the token. - * + * * @return the issuer value. */ public String issuer() { @@ -95,7 +99,7 @@ public String issuer() { /** * Set the issuer property: The endpoint that issues the token. - * + * * @param issuer the issuer value to set. * @return the OpenIdConnectConfig object itself. */ @@ -106,7 +110,7 @@ public OpenIdConnectConfig withIssuer(String issuer) { /** * Get the certificationUri property: The endpoint that provides the keys necessary to validate the token. - * + * * @return the certificationUri value. */ public String certificationUri() { @@ -115,7 +119,7 @@ public String certificationUri() { /** * Set the certificationUri property: The endpoint that provides the keys necessary to validate the token. - * + * * @param certificationUri the certificationUri value to set. * @return the OpenIdConnectConfig object itself. */ @@ -125,9 +129,9 @@ public OpenIdConnectConfig withCertificationUri(String certificationUri) { } /** - * Get the wellKnownOpenIdConfiguration property: The endpoint that contains all the configuration endpoints for the - * provider. - * + * Get the wellKnownOpenIdConfiguration property: The endpoint that contains all the configuration endpoints for + * the provider. + * * @return the wellKnownOpenIdConfiguration value. */ public String wellKnownOpenIdConfiguration() { @@ -135,9 +139,9 @@ public String wellKnownOpenIdConfiguration() { } /** - * Set the wellKnownOpenIdConfiguration property: The endpoint that contains all the configuration endpoints for the - * provider. - * + * Set the wellKnownOpenIdConfiguration property: The endpoint that contains all the configuration endpoints for + * the provider. + * * @param wellKnownOpenIdConfiguration the wellKnownOpenIdConfiguration value to set. * @return the OpenIdConnectConfig object itself. */ @@ -148,7 +152,7 @@ public OpenIdConnectConfig withWellKnownOpenIdConfiguration(String wellKnownOpen /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectLogin.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectLogin.java index 2d5eeaf03614a..48e465c4d0cc3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectLogin.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectLogin.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The configuration settings of the login flow of the custom Open ID Connect provider. */ +/** + * The configuration settings of the login flow of the custom Open ID Connect provider. + */ @Fluent public final class OpenIdConnectLogin { /* @@ -23,13 +25,15 @@ public final class OpenIdConnectLogin { @JsonProperty(value = "scopes") private List scopes; - /** Creates an instance of OpenIdConnectLogin class. */ + /** + * Creates an instance of OpenIdConnectLogin class. + */ public OpenIdConnectLogin() { } /** * Get the nameClaimType property: The name of the claim that contains the users name. - * + * * @return the nameClaimType value. */ public String nameClaimType() { @@ -38,7 +42,7 @@ public String nameClaimType() { /** * Set the nameClaimType property: The name of the claim that contains the users name. - * + * * @param nameClaimType the nameClaimType value to set. * @return the OpenIdConnectLogin object itself. */ @@ -49,7 +53,7 @@ public OpenIdConnectLogin withNameClaimType(String nameClaimType) { /** * Get the scopes property: A list of the scopes that should be requested while authenticating. - * + * * @return the scopes value. */ public List scopes() { @@ -58,7 +62,7 @@ public List scopes() { /** * Set the scopes property: A list of the scopes that should be requested while authenticating. - * + * * @param scopes the scopes value to set. * @return the OpenIdConnectLogin object itself. */ @@ -69,7 +73,7 @@ public OpenIdConnectLogin withScopes(List scopes) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectRegistration.java index 85683990f70f5..626b6c1fd21cb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenIdConnectRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the app registration for the custom Open ID Connect provider. */ +/** + * The configuration settings of the app registration for the custom Open ID Connect provider. + */ @Fluent public final class OpenIdConnectRegistration { /* @@ -28,13 +30,15 @@ public final class OpenIdConnectRegistration { @JsonProperty(value = "openIdConnectConfiguration") private OpenIdConnectConfig openIdConnectConfiguration; - /** Creates an instance of OpenIdConnectRegistration class. */ + /** + * Creates an instance of OpenIdConnectRegistration class. + */ public OpenIdConnectRegistration() { } /** * Get the clientId property: The client id of the custom Open ID Connect provider. - * + * * @return the clientId value. */ public String clientId() { @@ -43,7 +47,7 @@ public String clientId() { /** * Set the clientId property: The client id of the custom Open ID Connect provider. - * + * * @param clientId the clientId value to set. * @return the OpenIdConnectRegistration object itself. */ @@ -54,7 +58,7 @@ public OpenIdConnectRegistration withClientId(String clientId) { /** * Get the clientCredential property: The authentication credentials of the custom Open ID Connect provider. - * + * * @return the clientCredential value. */ public OpenIdConnectClientCredential clientCredential() { @@ -63,7 +67,7 @@ public OpenIdConnectClientCredential clientCredential() { /** * Set the clientCredential property: The authentication credentials of the custom Open ID Connect provider. - * + * * @param clientCredential the clientCredential value to set. * @return the OpenIdConnectRegistration object itself. */ @@ -73,9 +77,9 @@ public OpenIdConnectRegistration withClientCredential(OpenIdConnectClientCredent } /** - * Get the openIdConnectConfiguration property: The configuration settings of the endpoints used for the custom Open - * ID Connect provider. - * + * Get the openIdConnectConfiguration property: The configuration settings of the endpoints used for the custom + * Open ID Connect provider. + * * @return the openIdConnectConfiguration value. */ public OpenIdConnectConfig openIdConnectConfiguration() { @@ -83,9 +87,9 @@ public OpenIdConnectConfig openIdConnectConfiguration() { } /** - * Set the openIdConnectConfiguration property: The configuration settings of the endpoints used for the custom Open - * ID Connect provider. - * + * Set the openIdConnectConfiguration property: The configuration settings of the endpoints used for the custom + * Open ID Connect provider. + * * @param openIdConnectConfiguration the openIdConnectConfiguration value to set. * @return the OpenIdConnectRegistration object itself. */ @@ -96,7 +100,7 @@ public OpenIdConnectRegistration withOpenIdConnectConfiguration(OpenIdConnectCon /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenTelemetryConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenTelemetryConfiguration.java new file mode 100644 index 0000000000000..54c6df96b74bb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OpenTelemetryConfiguration.java @@ -0,0 +1,145 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration of Open Telemetry. + */ +@Fluent +public final class OpenTelemetryConfiguration { + /* + * Open telemetry destinations configuration + */ + @JsonProperty(value = "destinationsConfiguration") + private DestinationsConfiguration destinationsConfiguration; + + /* + * Open telemetry trace configuration + */ + @JsonProperty(value = "tracesConfiguration") + private TracesConfiguration tracesConfiguration; + + /* + * Open telemetry logs configuration + */ + @JsonProperty(value = "logsConfiguration") + private LogsConfiguration logsConfiguration; + + /* + * Open telemetry metrics configuration + */ + @JsonProperty(value = "metricsConfiguration") + private MetricsConfiguration metricsConfiguration; + + /** + * Creates an instance of OpenTelemetryConfiguration class. + */ + public OpenTelemetryConfiguration() { + } + + /** + * Get the destinationsConfiguration property: Open telemetry destinations configuration. + * + * @return the destinationsConfiguration value. + */ + public DestinationsConfiguration destinationsConfiguration() { + return this.destinationsConfiguration; + } + + /** + * Set the destinationsConfiguration property: Open telemetry destinations configuration. + * + * @param destinationsConfiguration the destinationsConfiguration value to set. + * @return the OpenTelemetryConfiguration object itself. + */ + public OpenTelemetryConfiguration + withDestinationsConfiguration(DestinationsConfiguration destinationsConfiguration) { + this.destinationsConfiguration = destinationsConfiguration; + return this; + } + + /** + * Get the tracesConfiguration property: Open telemetry trace configuration. + * + * @return the tracesConfiguration value. + */ + public TracesConfiguration tracesConfiguration() { + return this.tracesConfiguration; + } + + /** + * Set the tracesConfiguration property: Open telemetry trace configuration. + * + * @param tracesConfiguration the tracesConfiguration value to set. + * @return the OpenTelemetryConfiguration object itself. + */ + public OpenTelemetryConfiguration withTracesConfiguration(TracesConfiguration tracesConfiguration) { + this.tracesConfiguration = tracesConfiguration; + return this; + } + + /** + * Get the logsConfiguration property: Open telemetry logs configuration. + * + * @return the logsConfiguration value. + */ + public LogsConfiguration logsConfiguration() { + return this.logsConfiguration; + } + + /** + * Set the logsConfiguration property: Open telemetry logs configuration. + * + * @param logsConfiguration the logsConfiguration value to set. + * @return the OpenTelemetryConfiguration object itself. + */ + public OpenTelemetryConfiguration withLogsConfiguration(LogsConfiguration logsConfiguration) { + this.logsConfiguration = logsConfiguration; + return this; + } + + /** + * Get the metricsConfiguration property: Open telemetry metrics configuration. + * + * @return the metricsConfiguration value. + */ + public MetricsConfiguration metricsConfiguration() { + return this.metricsConfiguration; + } + + /** + * Set the metricsConfiguration property: Open telemetry metrics configuration. + * + * @param metricsConfiguration the metricsConfiguration value to set. + * @return the OpenTelemetryConfiguration object itself. + */ + public OpenTelemetryConfiguration withMetricsConfiguration(MetricsConfiguration metricsConfiguration) { + this.metricsConfiguration = metricsConfiguration; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (destinationsConfiguration() != null) { + destinationsConfiguration().validate(); + } + if (tracesConfiguration() != null) { + tracesConfiguration().validate(); + } + if (logsConfiguration() != null) { + logsConfiguration().validate(); + } + if (metricsConfiguration() != null) { + metricsConfiguration().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDetail.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDetail.java index 1feb34ba83292..e3796d6840cc8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDetail.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDetail.java @@ -6,39 +6,41 @@ import com.azure.resourcemanager.appcontainers.fluent.models.OperationDetailInner; -/** An immutable client-side representation of OperationDetail. */ +/** + * An immutable client-side representation of OperationDetail. + */ public interface OperationDetail { /** * Gets the name property: Name of the operation. - * + * * @return the name value. */ String name(); /** * Gets the isDataAction property: Indicates whether the operation is a data action. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Display of the operation. - * + * * @return the display value. */ OperationDisplay display(); /** * Gets the origin property: Origin of the operation. - * + * * @return the origin value. */ String origin(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.OperationDetailInner object. - * + * * @return the inner object. */ OperationDetailInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDisplay.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDisplay.java index 2233a26203b56..a9e2cc05d789d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDisplay.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Operation display payload. */ +/** + * Operation display payload. + */ @Fluent public final class OperationDisplay { /* @@ -34,13 +36,15 @@ public final class OperationDisplay { @JsonProperty(value = "description") private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: Resource provider of the operation. - * + * * @return the provider value. */ public String provider() { @@ -49,7 +53,7 @@ public String provider() { /** * Set the provider property: Resource provider of the operation. - * + * * @param provider the provider value to set. * @return the OperationDisplay object itself. */ @@ -60,7 +64,7 @@ public OperationDisplay withProvider(String provider) { /** * Get the resource property: Resource of the operation. - * + * * @return the resource value. */ public String resource() { @@ -69,7 +73,7 @@ public String resource() { /** * Set the resource property: Resource of the operation. - * + * * @param resource the resource value to set. * @return the OperationDisplay object itself. */ @@ -80,7 +84,7 @@ public OperationDisplay withResource(String resource) { /** * Get the operation property: Localized friendly name for the operation. - * + * * @return the operation value. */ public String operation() { @@ -89,7 +93,7 @@ public String operation() { /** * Set the operation property: Localized friendly name for the operation. - * + * * @param operation the operation value to set. * @return the OperationDisplay object itself. */ @@ -100,7 +104,7 @@ public OperationDisplay withOperation(String operation) { /** * Get the description property: Localized friendly description for the operation. - * + * * @return the description value. */ public String description() { @@ -109,7 +113,7 @@ public String description() { /** * Set the description property: Localized friendly description for the operation. - * + * * @param description the description value to set. * @return the OperationDisplay object itself. */ @@ -120,7 +124,7 @@ public OperationDisplay withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Operations.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Operations.java index d9cf37c1461de..387a605db0d28 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Operations.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Operations.java @@ -7,13 +7,15 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Lists all of the available RP operations. - * + * * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return available operations of the service as paginated response with {@link PagedIterable}. */ @@ -21,11 +23,11 @@ public interface Operations { /** * Lists all of the available RP operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return available operations of the service as paginated response with {@link PagedIterable}. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OtlpConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OtlpConfiguration.java new file mode 100644 index 0000000000000..019b028484f28 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/OtlpConfiguration.java @@ -0,0 +1,136 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration of otlp. + */ +@Fluent +public final class OtlpConfiguration { + /* + * The name of otlp configuration + */ + @JsonProperty(value = "name") + private String name; + + /* + * The endpoint of otlp configuration + */ + @JsonProperty(value = "endpoint") + private String endpoint; + + /* + * Boolean indicating if otlp configuration is insecure + */ + @JsonProperty(value = "insecure") + private Boolean insecure; + + /* + * Headers of otlp configurations + */ + @JsonProperty(value = "headers") + private List

headers; + + /** + * Creates an instance of OtlpConfiguration class. + */ + public OtlpConfiguration() { + } + + /** + * Get the name property: The name of otlp configuration. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: The name of otlp configuration. + * + * @param name the name value to set. + * @return the OtlpConfiguration object itself. + */ + public OtlpConfiguration withName(String name) { + this.name = name; + return this; + } + + /** + * Get the endpoint property: The endpoint of otlp configuration. + * + * @return the endpoint value. + */ + public String endpoint() { + return this.endpoint; + } + + /** + * Set the endpoint property: The endpoint of otlp configuration. + * + * @param endpoint the endpoint value to set. + * @return the OtlpConfiguration object itself. + */ + public OtlpConfiguration withEndpoint(String endpoint) { + this.endpoint = endpoint; + return this; + } + + /** + * Get the insecure property: Boolean indicating if otlp configuration is insecure. + * + * @return the insecure value. + */ + public Boolean insecure() { + return this.insecure; + } + + /** + * Set the insecure property: Boolean indicating if otlp configuration is insecure. + * + * @param insecure the insecure value to set. + * @return the OtlpConfiguration object itself. + */ + public OtlpConfiguration withInsecure(Boolean insecure) { + this.insecure = insecure; + return this; + } + + /** + * Get the headers property: Headers of otlp configurations. + * + * @return the headers value. + */ + public List
headers() { + return this.headers; + } + + /** + * Set the headers property: Headers of otlp configurations. + * + * @param headers the headers value to set. + * @return the OtlpConfiguration object itself. + */ + public OtlpConfiguration withHeaders(List
headers) { + this.headers = headers; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (headers() != null) { + headers().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/PreBuildStep.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/PreBuildStep.java new file mode 100644 index 0000000000000..8f67420509b32 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/PreBuildStep.java @@ -0,0 +1,110 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Model representing a pre-build step. + */ +@Fluent +public final class PreBuildStep { + /* + * Description of the pre-build step. + */ + @JsonProperty(value = "description") + private String description; + + /* + * List of custom commands to run. + */ + @JsonProperty(value = "scripts") + private List scripts; + + /* + * Http get request to send before the build. + */ + @JsonProperty(value = "httpGet") + private HttpGet httpGet; + + /** + * Creates an instance of PreBuildStep class. + */ + public PreBuildStep() { + } + + /** + * Get the description property: Description of the pre-build step. + * + * @return the description value. + */ + public String description() { + return this.description; + } + + /** + * Set the description property: Description of the pre-build step. + * + * @param description the description value to set. + * @return the PreBuildStep object itself. + */ + public PreBuildStep withDescription(String description) { + this.description = description; + return this; + } + + /** + * Get the scripts property: List of custom commands to run. + * + * @return the scripts value. + */ + public List scripts() { + return this.scripts; + } + + /** + * Set the scripts property: List of custom commands to run. + * + * @param scripts the scripts value to set. + * @return the PreBuildStep object itself. + */ + public PreBuildStep withScripts(List scripts) { + this.scripts = scripts; + return this; + } + + /** + * Get the httpGet property: Http get request to send before the build. + * + * @return the httpGet value. + */ + public HttpGet httpGet() { + return this.httpGet; + } + + /** + * Set the httpGet property: Http get request to send before the build. + * + * @param httpGet the httpGet value to set. + * @return the PreBuildStep object itself. + */ + public PreBuildStep withHttpGet(HttpGet httpGet) { + this.httpGet = httpGet; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (httpGet() != null) { + httpGet().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/QueueScaleRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/QueueScaleRule.java index cfbffd1cecb8e..aef1fb3523f6a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/QueueScaleRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/QueueScaleRule.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App container Azure Queue based scaling rule. */ +/** + * Container App container Azure Queue based scaling rule. + */ @Fluent public final class QueueScaleRule { /* @@ -29,13 +31,15 @@ public final class QueueScaleRule { @JsonProperty(value = "auth") private List auth; - /** Creates an instance of QueueScaleRule class. */ + /** + * Creates an instance of QueueScaleRule class. + */ public QueueScaleRule() { } /** * Get the queueName property: Queue name. - * + * * @return the queueName value. */ public String queueName() { @@ -44,7 +48,7 @@ public String queueName() { /** * Set the queueName property: Queue name. - * + * * @param queueName the queueName value to set. * @return the QueueScaleRule object itself. */ @@ -55,7 +59,7 @@ public QueueScaleRule withQueueName(String queueName) { /** * Get the queueLength property: Queue length. - * + * * @return the queueLength value. */ public Integer queueLength() { @@ -64,7 +68,7 @@ public Integer queueLength() { /** * Set the queueLength property: Queue length. - * + * * @param queueLength the queueLength value to set. * @return the QueueScaleRule object itself. */ @@ -75,7 +79,7 @@ public QueueScaleRule withQueueLength(Integer queueLength) { /** * Get the auth property: Authentication secrets for the queue scale rule. - * + * * @return the auth value. */ public List auth() { @@ -84,7 +88,7 @@ public List auth() { /** * Set the auth property: Authentication secrets for the queue scale rule. - * + * * @param auth the auth value to set. * @return the QueueScaleRule object itself. */ @@ -95,7 +99,7 @@ public QueueScaleRule withAuth(List auth) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryCredentials.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryCredentials.java index bb5b53a159fae..a20caa49d9995 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryCredentials.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryCredentials.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App Private Registry. */ +/** + * Container App Private Registry. + */ @Fluent public final class RegistryCredentials { /* @@ -35,13 +37,15 @@ public final class RegistryCredentials { @JsonProperty(value = "identity") private String identity; - /** Creates an instance of RegistryCredentials class. */ + /** + * Creates an instance of RegistryCredentials class. + */ public RegistryCredentials() { } /** * Get the server property: Container Registry Server. - * + * * @return the server value. */ public String server() { @@ -50,7 +54,7 @@ public String server() { /** * Set the server property: Container Registry Server. - * + * * @param server the server value to set. * @return the RegistryCredentials object itself. */ @@ -61,7 +65,7 @@ public RegistryCredentials withServer(String server) { /** * Get the username property: Container Registry Username. - * + * * @return the username value. */ public String username() { @@ -70,7 +74,7 @@ public String username() { /** * Set the username property: Container Registry Username. - * + * * @param username the username value to set. * @return the RegistryCredentials object itself. */ @@ -81,7 +85,7 @@ public RegistryCredentials withUsername(String username) { /** * Get the passwordSecretRef property: The name of the Secret that contains the registry login password. - * + * * @return the passwordSecretRef value. */ public String passwordSecretRef() { @@ -90,7 +94,7 @@ public String passwordSecretRef() { /** * Set the passwordSecretRef property: The name of the Secret that contains the registry login password. - * + * * @param passwordSecretRef the passwordSecretRef value to set. * @return the RegistryCredentials object itself. */ @@ -103,7 +107,7 @@ public RegistryCredentials withPasswordSecretRef(String passwordSecretRef) { * Get the identity property: A Managed Identity to use to authenticate with Azure Container Registry. For * user-assigned identities, use the full user-assigned identity Resource ID. For system-assigned identities, use * 'system'. - * + * * @return the identity value. */ public String identity() { @@ -114,7 +118,7 @@ public String identity() { * Set the identity property: A Managed Identity to use to authenticate with Azure Container Registry. For * user-assigned identities, use the full user-assigned identity Resource ID. For system-assigned identities, use * 'system'. - * + * * @param identity the identity value to set. * @return the RegistryCredentials object itself. */ @@ -125,7 +129,7 @@ public RegistryCredentials withIdentity(String identity) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryInfo.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryInfo.java index 719abbb378b5a..ba74fd1702705 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryInfo.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RegistryInfo.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App registry information. */ +/** + * Container App registry information. + */ @Fluent public final class RegistryInfo { /* @@ -28,13 +30,15 @@ public final class RegistryInfo { @JsonProperty(value = "registryPassword") private String registryPassword; - /** Creates an instance of RegistryInfo class. */ + /** + * Creates an instance of RegistryInfo class. + */ public RegistryInfo() { } /** * Get the registryUrl property: registry server Url. - * + * * @return the registryUrl value. */ public String registryUrl() { @@ -43,7 +47,7 @@ public String registryUrl() { /** * Set the registryUrl property: registry server Url. - * + * * @param registryUrl the registryUrl value to set. * @return the RegistryInfo object itself. */ @@ -54,7 +58,7 @@ public RegistryInfo withRegistryUrl(String registryUrl) { /** * Get the registryUsername property: registry username. - * + * * @return the registryUsername value. */ public String registryUsername() { @@ -63,7 +67,7 @@ public String registryUsername() { /** * Set the registryUsername property: registry username. - * + * * @param registryUsername the registryUsername value to set. * @return the RegistryInfo object itself. */ @@ -74,7 +78,7 @@ public RegistryInfo withRegistryUsername(String registryUsername) { /** * Get the registryPassword property: registry secret. - * + * * @return the registryPassword value. */ public String registryPassword() { @@ -83,7 +87,7 @@ public String registryPassword() { /** * Set the registryPassword property: registry secret. - * + * * @param registryPassword the registryPassword value to set. * @return the RegistryInfo object itself. */ @@ -94,7 +98,7 @@ public RegistryInfo withRegistryPassword(String registryPassword) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Replica.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Replica.java index f3df8cdf13fe8..7c9a93485af97 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Replica.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Replica.java @@ -4,71 +4,81 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.ReplicaInner; import java.time.OffsetDateTime; import java.util.List; -/** An immutable client-side representation of Replica. */ +/** + * An immutable client-side representation of Replica. + */ public interface Replica { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the createdTime property: Timestamp describing when the pod was created by controller. - * + * * @return the createdTime value. */ OffsetDateTime createdTime(); /** * Gets the runningState property: Current running state of the replica. - * + * * @return the runningState value. */ ContainerAppReplicaRunningState runningState(); /** * Gets the runningStateDetails property: The details of replica current running state. - * + * * @return the runningStateDetails value. */ String runningStateDetails(); /** * Gets the containers property: The containers collection under a replica. - * + * * @return the containers value. */ List containers(); /** * Gets the initContainers property: The init containers collection under a replica. - * + * * @return the initContainers value. */ List initContainers(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ReplicaInner object. - * + * * @return the inner object. */ ReplicaInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaCollection.java index a0ccd84658a1f..5f224d043a2a9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaCollection.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.appcontainers.fluent.models.ReplicaCollectionInner; import java.util.List; -/** An immutable client-side representation of ReplicaCollection. */ +/** + * An immutable client-side representation of ReplicaCollection. + */ public interface ReplicaCollection { /** * Gets the value property: Collection of resources. - * + * * @return the value value. */ List value(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.ReplicaCollectionInner object. - * + * * @return the inner object. */ ReplicaCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaContainer.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaContainer.java index 129d1e0ff42b9..8f05e41336c63 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaContainer.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ReplicaContainer.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container object under Container App Revision Replica. */ +/** + * Container object under Container App Revision Replica. + */ @Fluent public final class ReplicaContainer { /* @@ -64,13 +66,15 @@ public final class ReplicaContainer { @JsonProperty(value = "execEndpoint", access = JsonProperty.Access.WRITE_ONLY) private String execEndpoint; - /** Creates an instance of ReplicaContainer class. */ + /** + * Creates an instance of ReplicaContainer class. + */ public ReplicaContainer() { } /** * Get the name property: The Name of the Container. - * + * * @return the name value. */ public String name() { @@ -79,7 +83,7 @@ public String name() { /** * Set the name property: The Name of the Container. - * + * * @param name the name value to set. * @return the ReplicaContainer object itself. */ @@ -90,7 +94,7 @@ public ReplicaContainer withName(String name) { /** * Get the containerId property: The Id of the Container. - * + * * @return the containerId value. */ public String containerId() { @@ -99,7 +103,7 @@ public String containerId() { /** * Set the containerId property: The Id of the Container. - * + * * @param containerId the containerId value to set. * @return the ReplicaContainer object itself. */ @@ -110,7 +114,7 @@ public ReplicaContainer withContainerId(String containerId) { /** * Get the ready property: The container ready status. - * + * * @return the ready value. */ public Boolean ready() { @@ -119,7 +123,7 @@ public Boolean ready() { /** * Set the ready property: The container ready status. - * + * * @param ready the ready value to set. * @return the ReplicaContainer object itself. */ @@ -130,7 +134,7 @@ public ReplicaContainer withReady(Boolean ready) { /** * Get the started property: The container start status. - * + * * @return the started value. */ public Boolean started() { @@ -139,7 +143,7 @@ public Boolean started() { /** * Set the started property: The container start status. - * + * * @param started the started value to set. * @return the ReplicaContainer object itself. */ @@ -150,7 +154,7 @@ public ReplicaContainer withStarted(Boolean started) { /** * Get the restartCount property: The container restart count. - * + * * @return the restartCount value. */ public Integer restartCount() { @@ -159,7 +163,7 @@ public Integer restartCount() { /** * Set the restartCount property: The container restart count. - * + * * @param restartCount the restartCount value to set. * @return the ReplicaContainer object itself. */ @@ -170,7 +174,7 @@ public ReplicaContainer withRestartCount(Integer restartCount) { /** * Get the runningState property: Current running state of the container. - * + * * @return the runningState value. */ public ContainerAppContainerRunningState runningState() { @@ -179,7 +183,7 @@ public ContainerAppContainerRunningState runningState() { /** * Get the runningStateDetails property: The details of container current running state. - * + * * @return the runningStateDetails value. */ public String runningStateDetails() { @@ -188,7 +192,7 @@ public String runningStateDetails() { /** * Get the logStreamEndpoint property: Log Stream endpoint. - * + * * @return the logStreamEndpoint value. */ public String logStreamEndpoint() { @@ -197,7 +201,7 @@ public String logStreamEndpoint() { /** * Get the execEndpoint property: Container exec endpoint. - * + * * @return the execEndpoint value. */ public String execEndpoint() { @@ -206,7 +210,7 @@ public String execEndpoint() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ResourceProviders.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ResourceProviders.java index 548ce75182790..11d4944e98808 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ResourceProviders.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ResourceProviders.java @@ -7,35 +7,59 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ResourceProviders. */ +/** + * Resource collection API of ResourceProviders. + */ public interface ResourceProviders { /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return details of a single job execution along with {@link Response}. */ - Response jobExecutionWithResponse( - String resourceGroupName, String jobName, String jobExecutionName, Context context); + Response jobExecutionWithResponse(String resourceGroupName, String jobName, String jobExecutionName, + Context context); /** * Get details of a single job execution. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param jobName Job Name. * @param jobExecutionName Job execution name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request - * is rejected by server. + * is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return details of a single job execution. */ JobExecution jobExecution(String resourceGroupName, String jobName, String jobExecutionName); + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains along with {@link Response}. + */ + Response getCustomDomainVerificationIdWithResponse(Context context); + + /** + * Get the verification id of a subscription used for verifying custom domains. + * + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the verification id of a subscription used for verifying custom domains. + */ + String getCustomDomainVerificationId(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Revision.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Revision.java index 5e2eeb58945e8..5f83c296e50c7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Revision.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Revision.java @@ -4,35 +4,46 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.RevisionInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of Revision. */ +/** + * An immutable client-side representation of Revision. + */ public interface Revision { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the createdTime property: Timestamp describing when the revision was created by controller. - * + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the createdTime property: Timestamp describing when the revision was created + * by controller. + * * @return the createdTime value. */ OffsetDateTime createdTime(); @@ -40,78 +51,79 @@ public interface Revision { /** * Gets the lastActiveTime property: Timestamp describing when the revision was last active. Only meaningful when * revision is inactive. - * + * * @return the lastActiveTime value. */ OffsetDateTime lastActiveTime(); /** * Gets the fqdn property: Fully qualified domain name of the revision. - * + * * @return the fqdn value. */ String fqdn(); /** - * Gets the template property: Container App Revision Template with all possible settings and the defaults if user - * did not provide them. The defaults are populated as they were at the creation time. - * + * Gets the template property: Container App Revision Template with all possible settings and the + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @return the template value. */ Template template(); /** * Gets the active property: Boolean describing if the Revision is Active. - * + * * @return the active value. */ Boolean active(); /** * Gets the replicas property: Number of pods currently running for this revision. - * + * * @return the replicas value. */ Integer replicas(); /** * Gets the trafficWeight property: Traffic weight assigned to this revision. - * + * * @return the trafficWeight value. */ Integer trafficWeight(); /** * Gets the provisioningError property: Optional Field - Platform Error Message. - * + * * @return the provisioningError value. */ String provisioningError(); /** * Gets the healthState property: Current health State of the revision. - * + * * @return the healthState value. */ RevisionHealthState healthState(); /** * Gets the provisioningState property: Current provisioning State of the revision. - * + * * @return the provisioningState value. */ RevisionProvisioningState provisioningState(); /** * Gets the runningState property: Current running state of the revision. - * + * * @return the runningState value. */ RevisionRunningState runningState(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.RevisionInner object. - * + * * @return the inner object. */ RevisionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionCollection.java index 18b8b90d9908d..a9efc54b85d54 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App Revisions collection ARM resource. */ +/** + * Container App Revisions collection ARM resource. + */ @Fluent public final class RevisionCollection { /* @@ -25,13 +27,15 @@ public final class RevisionCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of RevisionCollection class. */ + /** + * Creates an instance of RevisionCollection class. + */ public RevisionCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the RevisionCollection object itself. */ @@ -51,7 +55,7 @@ public RevisionCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model RevisionCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model RevisionCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionHealthState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionHealthState.java index 788dab697cd45..9c7e078b68376 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionHealthState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionHealthState.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current health State of the revision. */ +/** + * Current health State of the revision. + */ public final class RevisionHealthState extends ExpandableStringEnum { - /** Static value Healthy for RevisionHealthState. */ + /** + * Static value Healthy for RevisionHealthState. + */ public static final RevisionHealthState HEALTHY = fromString("Healthy"); - /** Static value Unhealthy for RevisionHealthState. */ + /** + * Static value Unhealthy for RevisionHealthState. + */ public static final RevisionHealthState UNHEALTHY = fromString("Unhealthy"); - /** Static value None for RevisionHealthState. */ + /** + * Static value None for RevisionHealthState. + */ public static final RevisionHealthState NONE = fromString("None"); /** * Creates a new instance of RevisionHealthState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public RevisionHealthState() { /** * Creates or finds a RevisionHealthState from its string representation. - * + * * @param name a name to look for. * @return the corresponding RevisionHealthState. */ @@ -41,7 +49,7 @@ public static RevisionHealthState fromString(String name) { /** * Gets known RevisionHealthState values. - * + * * @return known RevisionHealthState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionProvisioningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionProvisioningState.java index 28f51f9d0fbfd..48a5bb3264569 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionProvisioningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionProvisioningState.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current provisioning State of the revision. */ +/** + * Current provisioning State of the revision. + */ public final class RevisionProvisioningState extends ExpandableStringEnum { - /** Static value Provisioning for RevisionProvisioningState. */ + /** + * Static value Provisioning for RevisionProvisioningState. + */ public static final RevisionProvisioningState PROVISIONING = fromString("Provisioning"); - /** Static value Provisioned for RevisionProvisioningState. */ + /** + * Static value Provisioned for RevisionProvisioningState. + */ public static final RevisionProvisioningState PROVISIONED = fromString("Provisioned"); - /** Static value Failed for RevisionProvisioningState. */ + /** + * Static value Failed for RevisionProvisioningState. + */ public static final RevisionProvisioningState FAILED = fromString("Failed"); - /** Static value Deprovisioning for RevisionProvisioningState. */ + /** + * Static value Deprovisioning for RevisionProvisioningState. + */ public static final RevisionProvisioningState DEPROVISIONING = fromString("Deprovisioning"); - /** Static value Deprovisioned for RevisionProvisioningState. */ + /** + * Static value Deprovisioned for RevisionProvisioningState. + */ public static final RevisionProvisioningState DEPROVISIONED = fromString("Deprovisioned"); /** * Creates a new instance of RevisionProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public RevisionProvisioningState() { /** * Creates or finds a RevisionProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding RevisionProvisioningState. */ @@ -47,7 +59,7 @@ public static RevisionProvisioningState fromString(String name) { /** * Gets known RevisionProvisioningState values. - * + * * @return known RevisionProvisioningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionRunningState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionRunningState.java index 347a898eb5966..517991e24dc79 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionRunningState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/RevisionRunningState.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current running state of the revision. */ +/** + * Current running state of the revision. + */ public final class RevisionRunningState extends ExpandableStringEnum { - /** Static value Running for RevisionRunningState. */ + /** + * Static value Running for RevisionRunningState. + */ public static final RevisionRunningState RUNNING = fromString("Running"); - /** Static value Processing for RevisionRunningState. */ + /** + * Static value Processing for RevisionRunningState. + */ public static final RevisionRunningState PROCESSING = fromString("Processing"); - /** Static value Stopped for RevisionRunningState. */ + /** + * Static value Stopped for RevisionRunningState. + */ public static final RevisionRunningState STOPPED = fromString("Stopped"); - /** Static value Degraded for RevisionRunningState. */ + /** + * Static value Degraded for RevisionRunningState. + */ public static final RevisionRunningState DEGRADED = fromString("Degraded"); - /** Static value Failed for RevisionRunningState. */ + /** + * Static value Failed for RevisionRunningState. + */ public static final RevisionRunningState FAILED = fromString("Failed"); - /** Static value Unknown for RevisionRunningState. */ + /** + * Static value Unknown for RevisionRunningState. + */ public static final RevisionRunningState UNKNOWN = fromString("Unknown"); /** * Creates a new instance of RevisionRunningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public RevisionRunningState() { /** * Creates or finds a RevisionRunningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding RevisionRunningState. */ @@ -50,7 +64,7 @@ public static RevisionRunningState fromString(String name) { /** * Gets known RevisionRunningState values. - * + * * @return known RevisionRunningState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scale.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scale.java index 4880d53682887..99a5cfe707b40 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scale.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scale.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Container App scaling configurations. */ +/** + * Container App scaling configurations. + */ @Fluent public final class Scale { /* @@ -29,13 +31,15 @@ public final class Scale { @JsonProperty(value = "rules") private List rules; - /** Creates an instance of Scale class. */ + /** + * Creates an instance of Scale class. + */ public Scale() { } /** * Get the minReplicas property: Optional. Minimum number of container replicas. - * + * * @return the minReplicas value. */ public Integer minReplicas() { @@ -44,7 +48,7 @@ public Integer minReplicas() { /** * Set the minReplicas property: Optional. Minimum number of container replicas. - * + * * @param minReplicas the minReplicas value to set. * @return the Scale object itself. */ @@ -55,7 +59,7 @@ public Scale withMinReplicas(Integer minReplicas) { /** * Get the maxReplicas property: Optional. Maximum number of container replicas. Defaults to 10 if not set. - * + * * @return the maxReplicas value. */ public Integer maxReplicas() { @@ -64,7 +68,7 @@ public Integer maxReplicas() { /** * Set the maxReplicas property: Optional. Maximum number of container replicas. Defaults to 10 if not set. - * + * * @param maxReplicas the maxReplicas value to set. * @return the Scale object itself. */ @@ -75,7 +79,7 @@ public Scale withMaxReplicas(Integer maxReplicas) { /** * Get the rules property: Scaling rules. - * + * * @return the rules value. */ public List rules() { @@ -84,7 +88,7 @@ public List rules() { /** * Set the rules property: Scaling rules. - * + * * @param rules the rules value to set. * @return the Scale object itself. */ @@ -95,7 +99,7 @@ public Scale withRules(List rules) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRule.java index 7072b43fc40ba..84dbae8b7efc0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRule.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App container scaling rule. */ +/** + * Container App container scaling rule. + */ @Fluent public final class ScaleRule { /* @@ -40,13 +42,15 @@ public final class ScaleRule { @JsonProperty(value = "tcp") private TcpScaleRule tcp; - /** Creates an instance of ScaleRule class. */ + /** + * Creates an instance of ScaleRule class. + */ public ScaleRule() { } /** * Get the name property: Scale Rule Name. - * + * * @return the name value. */ public String name() { @@ -55,7 +59,7 @@ public String name() { /** * Set the name property: Scale Rule Name. - * + * * @param name the name value to set. * @return the ScaleRule object itself. */ @@ -66,7 +70,7 @@ public ScaleRule withName(String name) { /** * Get the azureQueue property: Azure Queue based scaling. - * + * * @return the azureQueue value. */ public QueueScaleRule azureQueue() { @@ -75,7 +79,7 @@ public QueueScaleRule azureQueue() { /** * Set the azureQueue property: Azure Queue based scaling. - * + * * @param azureQueue the azureQueue value to set. * @return the ScaleRule object itself. */ @@ -86,7 +90,7 @@ public ScaleRule withAzureQueue(QueueScaleRule azureQueue) { /** * Get the custom property: Custom scale rule. - * + * * @return the custom value. */ public CustomScaleRule custom() { @@ -95,7 +99,7 @@ public CustomScaleRule custom() { /** * Set the custom property: Custom scale rule. - * + * * @param custom the custom value to set. * @return the ScaleRule object itself. */ @@ -106,7 +110,7 @@ public ScaleRule withCustom(CustomScaleRule custom) { /** * Get the http property: HTTP requests based scaling. - * + * * @return the http value. */ public HttpScaleRule http() { @@ -115,7 +119,7 @@ public HttpScaleRule http() { /** * Set the http property: HTTP requests based scaling. - * + * * @param http the http value to set. * @return the ScaleRule object itself. */ @@ -126,7 +130,7 @@ public ScaleRule withHttp(HttpScaleRule http) { /** * Get the tcp property: Tcp requests based scaling. - * + * * @return the tcp value. */ public TcpScaleRule tcp() { @@ -135,7 +139,7 @@ public TcpScaleRule tcp() { /** * Set the tcp property: Tcp requests based scaling. - * + * * @param tcp the tcp value to set. * @return the ScaleRule object itself. */ @@ -146,7 +150,7 @@ public ScaleRule withTcp(TcpScaleRule tcp) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRuleAuth.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRuleAuth.java index 08b69ad41a061..d957279f9c950 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRuleAuth.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ScaleRuleAuth.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Auth Secrets for Scale Rule. */ +/** + * Auth Secrets for Scale Rule. + */ @Fluent public final class ScaleRuleAuth { /* @@ -22,13 +24,15 @@ public final class ScaleRuleAuth { @JsonProperty(value = "triggerParameter") private String triggerParameter; - /** Creates an instance of ScaleRuleAuth class. */ + /** + * Creates an instance of ScaleRuleAuth class. + */ public ScaleRuleAuth() { } /** * Get the secretRef property: Name of the secret from which to pull the auth params. - * + * * @return the secretRef value. */ public String secretRef() { @@ -37,7 +41,7 @@ public String secretRef() { /** * Set the secretRef property: Name of the secret from which to pull the auth params. - * + * * @param secretRef the secretRef value to set. * @return the ScaleRuleAuth object itself. */ @@ -48,7 +52,7 @@ public ScaleRuleAuth withSecretRef(String secretRef) { /** * Get the triggerParameter property: Trigger Parameter that uses the secret. - * + * * @return the triggerParameter value. */ public String triggerParameter() { @@ -57,7 +61,7 @@ public String triggerParameter() { /** * Set the triggerParameter property: Trigger Parameter that uses the secret. - * + * * @param triggerParameter the triggerParameter value to set. * @return the ScaleRuleAuth object itself. */ @@ -68,7 +72,7 @@ public ScaleRuleAuth withTriggerParameter(String triggerParameter) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scheme.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scheme.java index 5ea43b744e228..84f4c144aa00f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scheme.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Scheme.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Scheme to use for connecting to the host. Defaults to HTTP. */ +/** + * Scheme to use for connecting to the host. Defaults to HTTP. + */ public final class Scheme extends ExpandableStringEnum { - /** Static value HTTP for Scheme. */ + /** + * Static value HTTP for Scheme. + */ public static final Scheme HTTP = fromString("HTTP"); - /** Static value HTTPS for Scheme. */ + /** + * Static value HTTPS for Scheme. + */ public static final Scheme HTTPS = fromString("HTTPS"); /** * Creates a new instance of Scheme value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public Scheme() { /** * Creates or finds a Scheme from its string representation. - * + * * @param name a name to look for. * @return the corresponding Scheme. */ @@ -38,7 +44,7 @@ public static Scheme fromString(String name) { /** * Gets known Scheme values. - * + * * @return known Scheme values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Secret.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Secret.java index 219d53c773e9a..984042a638afd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Secret.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Secret.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Secret definition. */ +/** + * Secret definition. + */ @Fluent public final class Secret { /* @@ -35,13 +37,15 @@ public final class Secret { @JsonProperty(value = "keyVaultUrl") private String keyVaultUrl; - /** Creates an instance of Secret class. */ + /** + * Creates an instance of Secret class. + */ public Secret() { } /** * Get the name property: Secret Name. - * + * * @return the name value. */ public String name() { @@ -50,7 +54,7 @@ public String name() { /** * Set the name property: Secret Name. - * + * * @param name the name value to set. * @return the Secret object itself. */ @@ -61,7 +65,7 @@ public Secret withName(String name) { /** * Get the value property: Secret Value. - * + * * @return the value value. */ public String value() { @@ -70,7 +74,7 @@ public String value() { /** * Set the value property: Secret Value. - * + * * @param value the value value to set. * @return the Secret object itself. */ @@ -82,7 +86,7 @@ public Secret withValue(String value) { /** * Get the identity property: Resource ID of a managed identity to authenticate with Azure Key Vault, or System to * use a system-assigned identity. - * + * * @return the identity value. */ public String identity() { @@ -92,7 +96,7 @@ public String identity() { /** * Set the identity property: Resource ID of a managed identity to authenticate with Azure Key Vault, or System to * use a system-assigned identity. - * + * * @param identity the identity value to set. * @return the Secret object itself. */ @@ -103,7 +107,7 @@ public Secret withIdentity(String identity) { /** * Get the keyVaultUrl property: Azure Key Vault URL pointing to the secret referenced by the container app. - * + * * @return the keyVaultUrl value. */ public String keyVaultUrl() { @@ -112,7 +116,7 @@ public String keyVaultUrl() { /** * Set the keyVaultUrl property: Azure Key Vault URL pointing to the secret referenced by the container app. - * + * * @param keyVaultUrl the keyVaultUrl value to set. * @return the Secret object itself. */ @@ -123,7 +127,7 @@ public Secret withKeyVaultUrl(String keyVaultUrl) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretVolumeItem.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretVolumeItem.java index 9d4e2ddaa5a22..55d91e2015a8b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretVolumeItem.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretVolumeItem.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Secret to be added to volume. */ +/** + * Secret to be added to volume. + */ @Fluent public final class SecretVolumeItem { /* @@ -22,13 +24,15 @@ public final class SecretVolumeItem { @JsonProperty(value = "path") private String path; - /** Creates an instance of SecretVolumeItem class. */ + /** + * Creates an instance of SecretVolumeItem class. + */ public SecretVolumeItem() { } /** * Get the secretRef property: Name of the Container App secret from which to pull the secret value. - * + * * @return the secretRef value. */ public String secretRef() { @@ -37,7 +41,7 @@ public String secretRef() { /** * Set the secretRef property: Name of the Container App secret from which to pull the secret value. - * + * * @param secretRef the secretRef value to set. * @return the SecretVolumeItem object itself. */ @@ -49,7 +53,7 @@ public SecretVolumeItem withSecretRef(String secretRef) { /** * Get the path property: Path to project secret to. If no path is provided, path defaults to name of secret listed * in secretRef. - * + * * @return the path value. */ public String path() { @@ -59,7 +63,7 @@ public String path() { /** * Set the path property: Path to project secret to. If no path is provided, path defaults to name of secret listed * in secretRef. - * + * * @param path the path value to set. * @return the SecretVolumeItem object itself. */ @@ -70,7 +74,7 @@ public SecretVolumeItem withPath(String path) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretsCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretsCollection.java index cb2a5d815cb42..a7c46963e8e3e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretsCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SecretsCollection.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.appcontainers.fluent.models.SecretsCollectionInner; import java.util.List; -/** An immutable client-side representation of SecretsCollection. */ +/** + * An immutable client-side representation of SecretsCollection. + */ public interface SecretsCollection { /** * Gets the value property: Collection of resources. - * + * * @return the value value. */ List value(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.SecretsCollectionInner object. - * + * * @return the inner object. */ SecretsCollectionInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Service.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Service.java index 5673a1e1ce38a..af26a69c5fe24 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Service.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Service.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Container App to be a dev service. */ +/** + * Container App to be a dev service. + */ @Fluent public final class Service { /* @@ -17,13 +19,15 @@ public final class Service { @JsonProperty(value = "type", required = true) private String type; - /** Creates an instance of Service class. */ + /** + * Creates an instance of Service class. + */ public Service() { } /** * Get the type property: Dev ContainerApp service type. - * + * * @return the type value. */ public String type() { @@ -32,7 +36,7 @@ public String type() { /** * Set the type property: Dev ContainerApp service type. - * + * * @param type the type value to set. * @return the Service object itself. */ @@ -43,7 +47,7 @@ public Service withType(String type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ServiceBind.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ServiceBind.java index 85afd1e257ecb..f78914381442c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ServiceBind.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/ServiceBind.java @@ -5,9 +5,13 @@ package com.azure.resourcemanager.appcontainers.models; import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; -/** Configuration to bind a ContainerApp to a dev ContainerApp Service. */ +/** + * Configuration to bind a ContainerApp to a dev ContainerApp Service. + */ @Fluent public final class ServiceBind { /* @@ -22,13 +26,28 @@ public final class ServiceBind { @JsonProperty(value = "name") private String name; - /** Creates an instance of ServiceBind class. */ + /* + * Type of the client to be used to connect to the service + */ + @JsonProperty(value = "clientType") + private String clientType; + + /* + * Customized keys for customizing injected values to the app + */ + @JsonProperty(value = "customizedKeys") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map customizedKeys; + + /** + * Creates an instance of ServiceBind class. + */ public ServiceBind() { } /** * Get the serviceId property: Resource id of the target service. - * + * * @return the serviceId value. */ public String serviceId() { @@ -37,7 +56,7 @@ public String serviceId() { /** * Set the serviceId property: Resource id of the target service. - * + * * @param serviceId the serviceId value to set. * @return the ServiceBind object itself. */ @@ -48,7 +67,7 @@ public ServiceBind withServiceId(String serviceId) { /** * Get the name property: Name of the service bind. - * + * * @return the name value. */ public String name() { @@ -57,7 +76,7 @@ public String name() { /** * Set the name property: Name of the service bind. - * + * * @param name the name value to set. * @return the ServiceBind object itself. */ @@ -66,9 +85,49 @@ public ServiceBind withName(String name) { return this; } + /** + * Get the clientType property: Type of the client to be used to connect to the service. + * + * @return the clientType value. + */ + public String clientType() { + return this.clientType; + } + + /** + * Set the clientType property: Type of the client to be used to connect to the service. + * + * @param clientType the clientType value to set. + * @return the ServiceBind object itself. + */ + public ServiceBind withClientType(String clientType) { + this.clientType = clientType; + return this; + } + + /** + * Get the customizedKeys property: Customized keys for customizing injected values to the app. + * + * @return the customizedKeys value. + */ + public Map customizedKeys() { + return this.customizedKeys; + } + + /** + * Set the customizedKeys property: Customized keys for customizing injected values to the app. + * + * @param customizedKeys the customizedKeys value to set. + * @return the ServiceBind object itself. + */ + public ServiceBind withCustomizedKeys(Map customizedKeys) { + this.customizedKeys = customizedKeys; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControl.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControl.java index e1e9cf9f49351..14349f87c7ce2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControl.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControl.java @@ -4,91 +4,110 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.appcontainers.fluent.models.SourceControlInner; -/** An immutable client-side representation of SourceControl. */ +/** + * An immutable client-side representation of SourceControl. + */ public interface SourceControl { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the operationState property: Current provisioning State of the operation. - * + * * @return the operationState value. */ SourceControlOperationState operationState(); /** * Gets the repoUrl property: The repo url which will be integrated to ContainerApp. - * + * * @return the repoUrl value. */ String repoUrl(); /** * Gets the branch property: The branch which will trigger the auto deployment. - * + * * @return the branch value. */ String branch(); /** * Gets the githubActionConfiguration property: Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. - * + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @return the githubActionConfiguration value. */ GithubActionConfiguration githubActionConfiguration(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.SourceControlInner object. - * + * * @return the inner object. */ SourceControlInner innerModel(); - /** The entirety of the SourceControl definition. */ + /** + * The entirety of the SourceControl definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The SourceControl definition stages. */ + /** + * The SourceControl definition stages. + */ interface DefinitionStages { - /** The first stage of the SourceControl definition. */ + /** + * The first stage of the SourceControl definition. + */ interface Blank extends WithParentResource { } - /** The stage of the SourceControl definition allowing to specify parent resource. */ + /** + * The stage of the SourceControl definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, containerAppName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param containerAppName Name of the Container App. * @return the next definition stage. @@ -100,57 +119,63 @@ interface WithParentResource { * The stage of the SourceControl definition which contains all the minimum required properties for the resource * to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithRepoUrl, - DefinitionStages.WithBranch, - DefinitionStages.WithGithubActionConfiguration { + interface WithCreate extends DefinitionStages.WithRepoUrl, DefinitionStages.WithBranch, + DefinitionStages.WithGithubActionConfiguration { /** * Executes the create request. - * + * * @return the created resource. */ SourceControl create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ SourceControl create(Context context); } - /** The stage of the SourceControl definition allowing to specify repoUrl. */ + /** + * The stage of the SourceControl definition allowing to specify repoUrl. + */ interface WithRepoUrl { /** * Specifies the repoUrl property: The repo url which will be integrated to ContainerApp.. - * + * * @param repoUrl The repo url which will be integrated to ContainerApp. * @return the next definition stage. */ WithCreate withRepoUrl(String repoUrl); } - /** The stage of the SourceControl definition allowing to specify branch. */ + /** + * The stage of the SourceControl definition allowing to specify branch. + */ interface WithBranch { /** * Specifies the branch property: The branch which will trigger the auto deployment. - * + * * @param branch The branch which will trigger the auto deployment. * @return the next definition stage. */ WithCreate withBranch(String branch); } - /** The stage of the SourceControl definition allowing to specify githubActionConfiguration. */ + /** + * The stage of the SourceControl definition allowing to specify githubActionConfiguration. + */ interface WithGithubActionConfiguration { /** * Specifies the githubActionConfiguration property: Container App Revision Template with all possible - * settings and the defaults if user did not provide them. The defaults are populated as they were at the - * creation time. - * + * settings and the + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @param githubActionConfiguration Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. * @return the next definition stage. */ WithCreate withGithubActionConfiguration(GithubActionConfiguration githubActionConfiguration); @@ -159,63 +184,75 @@ interface WithGithubActionConfiguration { /** * Begins update for the SourceControl resource. - * + * * @return the stage of resource update. */ SourceControl.Update update(); - /** The template for SourceControl update. */ + /** + * The template for SourceControl update. + */ interface Update extends UpdateStages.WithRepoUrl, UpdateStages.WithBranch, UpdateStages.WithGithubActionConfiguration { /** * Executes the update request. - * + * * @return the updated resource. */ SourceControl apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ SourceControl apply(Context context); } - /** The SourceControl update stages. */ + /** + * The SourceControl update stages. + */ interface UpdateStages { - /** The stage of the SourceControl update allowing to specify repoUrl. */ + /** + * The stage of the SourceControl update allowing to specify repoUrl. + */ interface WithRepoUrl { /** * Specifies the repoUrl property: The repo url which will be integrated to ContainerApp.. - * + * * @param repoUrl The repo url which will be integrated to ContainerApp. * @return the next definition stage. */ Update withRepoUrl(String repoUrl); } - /** The stage of the SourceControl update allowing to specify branch. */ + /** + * The stage of the SourceControl update allowing to specify branch. + */ interface WithBranch { /** * Specifies the branch property: The branch which will trigger the auto deployment. - * + * * @param branch The branch which will trigger the auto deployment. * @return the next definition stage. */ Update withBranch(String branch); } - /** The stage of the SourceControl update allowing to specify githubActionConfiguration. */ + /** + * The stage of the SourceControl update allowing to specify githubActionConfiguration. + */ interface WithGithubActionConfiguration { /** * Specifies the githubActionConfiguration property: Container App Revision Template with all possible - * settings and the defaults if user did not provide them. The defaults are populated as they were at the - * creation time. - * + * settings and the + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. + * * @param githubActionConfiguration Container App Revision Template with all possible settings and the - * defaults if user did not provide them. The defaults are populated as they were at the creation time. + * defaults if user did not provide them. The defaults are populated + * as they were at the creation time. * @return the next definition stage. */ Update withGithubActionConfiguration(GithubActionConfiguration githubActionConfiguration); @@ -224,14 +261,14 @@ interface WithGithubActionConfiguration { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ SourceControl refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlCollection.java index ba48d5e1579d6..652bfc58a8357 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** SourceControl collection ARM resource. */ +/** + * SourceControl collection ARM resource. + */ @Fluent public final class SourceControlCollection { /* @@ -25,13 +27,15 @@ public final class SourceControlCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of SourceControlCollection class. */ + /** + * Creates an instance of SourceControlCollection class. + */ public SourceControlCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the SourceControlCollection object itself. */ @@ -51,7 +55,7 @@ public SourceControlCollection withValue(List value) { /** * Get the nextLink property: Link to next page of resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model SourceControlCollection")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model SourceControlCollection")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlOperationState.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlOperationState.java index 5c1f3d14d013f..ce792cdf36abb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlOperationState.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/SourceControlOperationState.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current provisioning State of the operation. */ +/** + * Current provisioning State of the operation. + */ public final class SourceControlOperationState extends ExpandableStringEnum { - /** Static value InProgress for SourceControlOperationState. */ + /** + * Static value InProgress for SourceControlOperationState. + */ public static final SourceControlOperationState IN_PROGRESS = fromString("InProgress"); - /** Static value Succeeded for SourceControlOperationState. */ + /** + * Static value Succeeded for SourceControlOperationState. + */ public static final SourceControlOperationState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for SourceControlOperationState. */ + /** + * Static value Failed for SourceControlOperationState. + */ public static final SourceControlOperationState FAILED = fromString("Failed"); - /** Static value Canceled for SourceControlOperationState. */ + /** + * Static value Canceled for SourceControlOperationState. + */ public static final SourceControlOperationState CANCELED = fromString("Canceled"); /** * Creates a new instance of SourceControlOperationState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public SourceControlOperationState() { /** * Creates or finds a SourceControlOperationState from its string representation. - * + * * @param name a name to look for. * @return the corresponding SourceControlOperationState. */ @@ -44,7 +54,7 @@ public static SourceControlOperationState fromString(String name) { /** * Gets known SourceControlOperationState values. - * + * * @return known SourceControlOperationState values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/StorageType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/StorageType.java index e62d9cffc4895..07da6d736766e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/StorageType.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/StorageType.java @@ -8,20 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Storage type for the volume. If not provided, use EmptyDir. */ +/** + * Storage type for the volume. If not provided, use EmptyDir. + */ public final class StorageType extends ExpandableStringEnum { - /** Static value AzureFile for StorageType. */ + /** + * Static value AzureFile for StorageType. + */ public static final StorageType AZURE_FILE = fromString("AzureFile"); - /** Static value EmptyDir for StorageType. */ + /** + * Static value EmptyDir for StorageType. + */ public static final StorageType EMPTY_DIR = fromString("EmptyDir"); - /** Static value Secret for StorageType. */ + /** + * Static value Secret for StorageType. + */ public static final StorageType SECRET = fromString("Secret"); + /** + * Static value NfsAzureFile for StorageType. + */ + public static final StorageType NFS_AZURE_FILE = fromString("NfsAzureFile"); + /** * Creates a new instance of StorageType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +43,7 @@ public StorageType() { /** * Creates or finds a StorageType from its string representation. - * + * * @param name a name to look for. * @return the corresponding StorageType. */ @@ -41,7 +54,7 @@ public static StorageType fromString(String name) { /** * Gets known StorageType values. - * + * * @return known StorageType values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpConnectionPool.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpConnectionPool.java new file mode 100644 index 0000000000000..041eb1c8dde50 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpConnectionPool.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Defines parameters for tcp connection pooling. + */ +@Fluent +public final class TcpConnectionPool { + /* + * Maximum number of tcp connections allowed + */ + @JsonProperty(value = "maxConnections") + private Integer maxConnections; + + /** + * Creates an instance of TcpConnectionPool class. + */ + public TcpConnectionPool() { + } + + /** + * Get the maxConnections property: Maximum number of tcp connections allowed. + * + * @return the maxConnections value. + */ + public Integer maxConnections() { + return this.maxConnections; + } + + /** + * Set the maxConnections property: Maximum number of tcp connections allowed. + * + * @param maxConnections the maxConnections value to set. + * @return the TcpConnectionPool object itself. + */ + public TcpConnectionPool withMaxConnections(Integer maxConnections) { + this.maxConnections = maxConnections; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpRetryPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpRetryPolicy.java new file mode 100644 index 0000000000000..885dd643742c0 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpRetryPolicy.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Policy that defines tcp request retry conditions. + */ +@Fluent +public final class TcpRetryPolicy { + /* + * Maximum number of attempts to connect to the tcp service + */ + @JsonProperty(value = "maxConnectAttempts") + private Integer maxConnectAttempts; + + /** + * Creates an instance of TcpRetryPolicy class. + */ + public TcpRetryPolicy() { + } + + /** + * Get the maxConnectAttempts property: Maximum number of attempts to connect to the tcp service. + * + * @return the maxConnectAttempts value. + */ + public Integer maxConnectAttempts() { + return this.maxConnectAttempts; + } + + /** + * Set the maxConnectAttempts property: Maximum number of attempts to connect to the tcp service. + * + * @param maxConnectAttempts the maxConnectAttempts value to set. + * @return the TcpRetryPolicy object itself. + */ + public TcpRetryPolicy withMaxConnectAttempts(Integer maxConnectAttempts) { + this.maxConnectAttempts = maxConnectAttempts; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpScaleRule.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpScaleRule.java index f297d3d3bf85c..d8b26124d95c1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpScaleRule.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TcpScaleRule.java @@ -10,7 +10,9 @@ import java.util.List; import java.util.Map; -/** Container App container Tcp scaling rule. */ +/** + * Container App container Tcp scaling rule. + */ @Fluent public final class TcpScaleRule { /* @@ -26,13 +28,15 @@ public final class TcpScaleRule { @JsonProperty(value = "auth") private List auth; - /** Creates an instance of TcpScaleRule class. */ + /** + * Creates an instance of TcpScaleRule class. + */ public TcpScaleRule() { } /** * Get the metadata property: Metadata properties to describe tcp scale rule. - * + * * @return the metadata value. */ public Map metadata() { @@ -41,7 +45,7 @@ public Map metadata() { /** * Set the metadata property: Metadata properties to describe tcp scale rule. - * + * * @param metadata the metadata value to set. * @return the TcpScaleRule object itself. */ @@ -52,7 +56,7 @@ public TcpScaleRule withMetadata(Map metadata) { /** * Get the auth property: Authentication secrets for the tcp scale rule. - * + * * @return the auth value. */ public List auth() { @@ -61,7 +65,7 @@ public List auth() { /** * Set the auth property: Authentication secrets for the tcp scale rule. - * + * * @param auth the auth value to set. * @return the TcpScaleRule object itself. */ @@ -72,7 +76,7 @@ public TcpScaleRule withAuth(List auth) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Template.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Template.java index a2ac125099189..a5b91a04bc387 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Template.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Template.java @@ -9,8 +9,9 @@ import java.util.List; /** - * Container App versioned application definition. Defines the desired state of an immutable revision. Any changes to - * this section Will result in a new revision being created. + * Container App versioned application definition. + * Defines the desired state of an immutable revision. + * Any changes to this section Will result in a new revision being created. */ @Fluent public final class Template { @@ -59,13 +60,15 @@ public final class Template { @JsonProperty(value = "serviceBinds") private List serviceBinds; - /** Creates an instance of Template class. */ + /** + * Creates an instance of Template class. + */ public Template() { } /** * Get the revisionSuffix property: User friendly suffix that is appended to the revision name. - * + * * @return the revisionSuffix value. */ public String revisionSuffix() { @@ -74,7 +77,7 @@ public String revisionSuffix() { /** * Set the revisionSuffix property: User friendly suffix that is appended to the revision name. - * + * * @param revisionSuffix the revisionSuffix value to set. * @return the Template object itself. */ @@ -88,7 +91,7 @@ public Template withRevisionSuffix(String revisionSuffix) { * terminate gracefully. Value must be non-negative integer. The value zero indicates stop immediately via the kill * signal (no opportunity to shut down). If this value is nil, the default grace period will be used instead. Set * this value longer than the expected cleanup time for your process. Defaults to 30 seconds. - * + * * @return the terminationGracePeriodSeconds value. */ public Long terminationGracePeriodSeconds() { @@ -100,7 +103,7 @@ public Long terminationGracePeriodSeconds() { * terminate gracefully. Value must be non-negative integer. The value zero indicates stop immediately via the kill * signal (no opportunity to shut down). If this value is nil, the default grace period will be used instead. Set * this value longer than the expected cleanup time for your process. Defaults to 30 seconds. - * + * * @param terminationGracePeriodSeconds the terminationGracePeriodSeconds value to set. * @return the Template object itself. */ @@ -111,7 +114,7 @@ public Template withTerminationGracePeriodSeconds(Long terminationGracePeriodSec /** * Get the initContainers property: List of specialized containers that run before app containers. - * + * * @return the initContainers value. */ public List initContainers() { @@ -120,7 +123,7 @@ public List initContainers() { /** * Set the initContainers property: List of specialized containers that run before app containers. - * + * * @param initContainers the initContainers value to set. * @return the Template object itself. */ @@ -131,7 +134,7 @@ public Template withInitContainers(List initContainers) { /** * Get the containers property: List of container definitions for the Container App. - * + * * @return the containers value. */ public List containers() { @@ -140,7 +143,7 @@ public List containers() { /** * Set the containers property: List of container definitions for the Container App. - * + * * @param containers the containers value to set. * @return the Template object itself. */ @@ -151,7 +154,7 @@ public Template withContainers(List containers) { /** * Get the scale property: Scaling properties for the Container App. - * + * * @return the scale value. */ public Scale scale() { @@ -160,7 +163,7 @@ public Scale scale() { /** * Set the scale property: Scaling properties for the Container App. - * + * * @param scale the scale value to set. * @return the Template object itself. */ @@ -171,7 +174,7 @@ public Template withScale(Scale scale) { /** * Get the volumes property: List of volume definitions for the Container App. - * + * * @return the volumes value. */ public List volumes() { @@ -180,7 +183,7 @@ public List volumes() { /** * Set the volumes property: List of volume definitions for the Container App. - * + * * @param volumes the volumes value to set. * @return the Template object itself. */ @@ -191,7 +194,7 @@ public Template withVolumes(List volumes) { /** * Get the serviceBinds property: List of container app services bound to the app. - * + * * @return the serviceBinds value. */ public List serviceBinds() { @@ -200,7 +203,7 @@ public List serviceBinds() { /** * Set the serviceBinds property: List of container app services bound to the app. - * + * * @param serviceBinds the serviceBinds value to set. * @return the Template object itself. */ @@ -211,7 +214,7 @@ public Template withServiceBinds(List serviceBinds) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TimeoutPolicy.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TimeoutPolicy.java new file mode 100644 index 0000000000000..1b529b5b065cd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TimeoutPolicy.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Policy to set request timeouts. + */ +@Fluent +public final class TimeoutPolicy { + /* + * Timeout, in seconds, for a request to respond + */ + @JsonProperty(value = "responseTimeoutInSeconds") + private Integer responseTimeoutInSeconds; + + /* + * Timeout, in seconds, for a request to initiate a connection + */ + @JsonProperty(value = "connectionTimeoutInSeconds") + private Integer connectionTimeoutInSeconds; + + /** + * Creates an instance of TimeoutPolicy class. + */ + public TimeoutPolicy() { + } + + /** + * Get the responseTimeoutInSeconds property: Timeout, in seconds, for a request to respond. + * + * @return the responseTimeoutInSeconds value. + */ + public Integer responseTimeoutInSeconds() { + return this.responseTimeoutInSeconds; + } + + /** + * Set the responseTimeoutInSeconds property: Timeout, in seconds, for a request to respond. + * + * @param responseTimeoutInSeconds the responseTimeoutInSeconds value to set. + * @return the TimeoutPolicy object itself. + */ + public TimeoutPolicy withResponseTimeoutInSeconds(Integer responseTimeoutInSeconds) { + this.responseTimeoutInSeconds = responseTimeoutInSeconds; + return this; + } + + /** + * Get the connectionTimeoutInSeconds property: Timeout, in seconds, for a request to initiate a connection. + * + * @return the connectionTimeoutInSeconds value. + */ + public Integer connectionTimeoutInSeconds() { + return this.connectionTimeoutInSeconds; + } + + /** + * Set the connectionTimeoutInSeconds property: Timeout, in seconds, for a request to initiate a connection. + * + * @param connectionTimeoutInSeconds the connectionTimeoutInSeconds value to set. + * @return the TimeoutPolicy object itself. + */ + public TimeoutPolicy withConnectionTimeoutInSeconds(Integer connectionTimeoutInSeconds) { + this.connectionTimeoutInSeconds = connectionTimeoutInSeconds; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TokenStore.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TokenStore.java new file mode 100644 index 0000000000000..3214e366a176a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TokenStore.java @@ -0,0 +1,122 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The configuration settings of the token store. + */ +@Fluent +public final class TokenStore { + /* + * true to durably store platform-specific security tokens that are obtained during login flows; + * otherwise, false. + * The default is false. + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The number of hours after session token expiration that a session token can be used to + * call the token refresh API. The default is 72 hours. + */ + @JsonProperty(value = "tokenRefreshExtensionHours") + private Double tokenRefreshExtensionHours; + + /* + * The configuration settings of the storage of the tokens if blob storage is used. + */ + @JsonProperty(value = "azureBlobStorage") + private BlobStorageTokenStore azureBlobStorage; + + /** + * Creates an instance of TokenStore class. + */ + public TokenStore() { + } + + /** + * Get the enabled property: <code>true</code> to durably store platform-specific security tokens that + * are obtained during login flows; otherwise, <code>false</code>. + * The default is <code>false</code>. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: <code>true</code> to durably store platform-specific security tokens that + * are obtained during login flows; otherwise, <code>false</code>. + * The default is <code>false</code>. + * + * @param enabled the enabled value to set. + * @return the TokenStore object itself. + */ + public TokenStore withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the tokenRefreshExtensionHours property: The number of hours after session token expiration that a session + * token can be used to + * call the token refresh API. The default is 72 hours. + * + * @return the tokenRefreshExtensionHours value. + */ + public Double tokenRefreshExtensionHours() { + return this.tokenRefreshExtensionHours; + } + + /** + * Set the tokenRefreshExtensionHours property: The number of hours after session token expiration that a session + * token can be used to + * call the token refresh API. The default is 72 hours. + * + * @param tokenRefreshExtensionHours the tokenRefreshExtensionHours value to set. + * @return the TokenStore object itself. + */ + public TokenStore withTokenRefreshExtensionHours(Double tokenRefreshExtensionHours) { + this.tokenRefreshExtensionHours = tokenRefreshExtensionHours; + return this; + } + + /** + * Get the azureBlobStorage property: The configuration settings of the storage of the tokens if blob storage is + * used. + * + * @return the azureBlobStorage value. + */ + public BlobStorageTokenStore azureBlobStorage() { + return this.azureBlobStorage; + } + + /** + * Set the azureBlobStorage property: The configuration settings of the storage of the tokens if blob storage is + * used. + * + * @param azureBlobStorage the azureBlobStorage value to set. + * @return the TokenStore object itself. + */ + public TokenStore withAzureBlobStorage(BlobStorageTokenStore azureBlobStorage) { + this.azureBlobStorage = azureBlobStorage; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (azureBlobStorage() != null) { + azureBlobStorage().validate(); + } + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TracesConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TracesConfiguration.java new file mode 100644 index 0000000000000..e84da4bc067fd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TracesConfiguration.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration of Open Telemetry traces. + */ +@Fluent +public final class TracesConfiguration { + /* + * Open telemetry traces destinations + */ + @JsonProperty(value = "destinations") + private List destinations; + + /** + * Creates an instance of TracesConfiguration class. + */ + public TracesConfiguration() { + } + + /** + * Get the destinations property: Open telemetry traces destinations. + * + * @return the destinations value. + */ + public List destinations() { + return this.destinations; + } + + /** + * Set the destinations property: Open telemetry traces destinations. + * + * @param destinations the destinations value to set. + * @return the TracesConfiguration object itself. + */ + public TracesConfiguration withDestinations(List destinations) { + this.destinations = destinations; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TrafficWeight.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TrafficWeight.java index bf509ea5c3ce6..06240037edf00 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TrafficWeight.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TrafficWeight.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Traffic weight assigned to a revision. */ +/** + * Traffic weight assigned to a revision. + */ @Fluent public final class TrafficWeight { /* @@ -34,13 +36,15 @@ public final class TrafficWeight { @JsonProperty(value = "label") private String label; - /** Creates an instance of TrafficWeight class. */ + /** + * Creates an instance of TrafficWeight class. + */ public TrafficWeight() { } /** * Get the revisionName property: Name of a revision. - * + * * @return the revisionName value. */ public String revisionName() { @@ -49,7 +53,7 @@ public String revisionName() { /** * Set the revisionName property: Name of a revision. - * + * * @param revisionName the revisionName value to set. * @return the TrafficWeight object itself. */ @@ -60,7 +64,7 @@ public TrafficWeight withRevisionName(String revisionName) { /** * Get the weight property: Traffic weight assigned to a revision. - * + * * @return the weight value. */ public Integer weight() { @@ -69,7 +73,7 @@ public Integer weight() { /** * Set the weight property: Traffic weight assigned to a revision. - * + * * @param weight the weight value to set. * @return the TrafficWeight object itself. */ @@ -80,7 +84,7 @@ public TrafficWeight withWeight(Integer weight) { /** * Get the latestRevision property: Indicates that the traffic weight belongs to a latest stable revision. - * + * * @return the latestRevision value. */ public Boolean latestRevision() { @@ -89,7 +93,7 @@ public Boolean latestRevision() { /** * Set the latestRevision property: Indicates that the traffic weight belongs to a latest stable revision. - * + * * @param latestRevision the latestRevision value to set. * @return the TrafficWeight object itself. */ @@ -100,7 +104,7 @@ public TrafficWeight withLatestRevision(Boolean latestRevision) { /** * Get the label property: Associates a traffic label with a revision. - * + * * @return the label value. */ public String label() { @@ -109,7 +113,7 @@ public String label() { /** * Set the label property: Associates a traffic label with a revision. - * + * * @param label the label value to set. * @return the TrafficWeight object itself. */ @@ -120,7 +124,7 @@ public TrafficWeight withLabel(String label) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TriggerType.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TriggerType.java index 9bffe6ca49bea..c30ffffb4ae93 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TriggerType.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TriggerType.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Trigger type of the job. */ +/** + * Trigger type of the job. + */ public final class TriggerType extends ExpandableStringEnum { - /** Static value Schedule for TriggerType. */ + /** + * Static value Schedule for TriggerType. + */ public static final TriggerType SCHEDULE = fromString("Schedule"); - /** Static value Event for TriggerType. */ + /** + * Static value Event for TriggerType. + */ public static final TriggerType EVENT = fromString("Event"); - /** Static value Manual for TriggerType. */ + /** + * Static value Manual for TriggerType. + */ public static final TriggerType MANUAL = fromString("Manual"); /** * Creates a new instance of TriggerType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public TriggerType() { /** * Creates or finds a TriggerType from its string representation. - * + * * @param name a name to look for. * @return the corresponding TriggerType. */ @@ -41,7 +49,7 @@ public static TriggerType fromString(String name) { /** * Gets known TriggerType values. - * + * * @return known TriggerType values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Twitter.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Twitter.java index 2cd7a0a5a6352..a23d0f0b96ff1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Twitter.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Twitter.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the Twitter provider. */ +/** + * The configuration settings of the Twitter provider. + */ @Fluent public final class Twitter { /* @@ -23,14 +25,16 @@ public final class Twitter { @JsonProperty(value = "registration") private TwitterRegistration registration; - /** Creates an instance of Twitter class. */ + /** + * Creates an instance of Twitter class. + */ public Twitter() { } /** * Get the enabled property: <code>false</code> if the Twitter provider should not be enabled despite * the set registration; otherwise, <code>true</code>. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -40,7 +44,7 @@ public Boolean enabled() { /** * Set the enabled property: <code>false</code> if the Twitter provider should not be enabled despite * the set registration; otherwise, <code>true</code>. - * + * * @param enabled the enabled value to set. * @return the Twitter object itself. */ @@ -51,7 +55,7 @@ public Twitter withEnabled(Boolean enabled) { /** * Get the registration property: The configuration settings of the app registration for the Twitter provider. - * + * * @return the registration value. */ public TwitterRegistration registration() { @@ -60,7 +64,7 @@ public TwitterRegistration registration() { /** * Set the registration property: The configuration settings of the app registration for the Twitter provider. - * + * * @param registration the registration value to set. * @return the Twitter object itself. */ @@ -71,7 +75,7 @@ public Twitter withRegistration(TwitterRegistration registration) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TwitterRegistration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TwitterRegistration.java index 26f1599daa13e..293488655d990 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TwitterRegistration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/TwitterRegistration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The configuration settings of the app registration for the Twitter provider. */ +/** + * The configuration settings of the app registration for the Twitter provider. + */ @Fluent public final class TwitterRegistration { /* @@ -25,15 +27,17 @@ public final class TwitterRegistration { @JsonProperty(value = "consumerSecretSettingName") private String consumerSecretSettingName; - /** Creates an instance of TwitterRegistration class. */ + /** + * Creates an instance of TwitterRegistration class. + */ public TwitterRegistration() { } /** - * Get the consumerKey property: The OAuth 1.0a consumer key of the Twitter application used for sign-in. This - * setting is required for enabling Twitter Sign-In. Twitter Sign-In documentation: - * https://dev.twitter.com/web/sign-in. - * + * Get the consumerKey property: The OAuth 1.0a consumer key of the Twitter application used for sign-in. + * This setting is required for enabling Twitter Sign-In. + * Twitter Sign-In documentation: https://dev.twitter.com/web/sign-in. + * * @return the consumerKey value. */ public String consumerKey() { @@ -41,10 +45,10 @@ public String consumerKey() { } /** - * Set the consumerKey property: The OAuth 1.0a consumer key of the Twitter application used for sign-in. This - * setting is required for enabling Twitter Sign-In. Twitter Sign-In documentation: - * https://dev.twitter.com/web/sign-in. - * + * Set the consumerKey property: The OAuth 1.0a consumer key of the Twitter application used for sign-in. + * This setting is required for enabling Twitter Sign-In. + * Twitter Sign-In documentation: https://dev.twitter.com/web/sign-in. + * * @param consumerKey the consumerKey value to set. * @return the TwitterRegistration object itself. */ @@ -55,8 +59,9 @@ public TwitterRegistration withConsumerKey(String consumerKey) { /** * Get the consumerSecretSettingName property: The app setting name that contains the OAuth 1.0a consumer secret of - * the Twitter application used for sign-in. - * + * the Twitter + * application used for sign-in. + * * @return the consumerSecretSettingName value. */ public String consumerSecretSettingName() { @@ -65,8 +70,9 @@ public String consumerSecretSettingName() { /** * Set the consumerSecretSettingName property: The app setting name that contains the OAuth 1.0a consumer secret of - * the Twitter application used for sign-in. - * + * the Twitter + * application used for sign-in. + * * @param consumerSecretSettingName the consumerSecretSettingName value to set. * @return the TwitterRegistration object itself. */ @@ -77,7 +83,7 @@ public TwitterRegistration withConsumerSecretSettingName(String consumerSecretSe /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Type.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Type.java index d7c75a6a0542b..82941e75731d8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Type.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Type.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The type of probe. */ +/** + * The type of probe. + */ public final class Type extends ExpandableStringEnum { - /** Static value Liveness for Type. */ + /** + * Static value Liveness for Type. + */ public static final Type LIVENESS = fromString("Liveness"); - /** Static value Readiness for Type. */ + /** + * Static value Readiness for Type. + */ public static final Type READINESS = fromString("Readiness"); - /** Static value Startup for Type. */ + /** + * Static value Startup for Type. + */ public static final Type STARTUP = fromString("Startup"); /** * Creates a new instance of Type value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public Type() { /** * Creates or finds a Type from its string representation. - * + * * @param name a name to look for. * @return the corresponding Type. */ @@ -41,7 +49,7 @@ public static Type fromString(String name) { /** * Gets known Type values. - * + * * @return known Type values. */ public static Collection values() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UnauthenticatedClientActionV2.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UnauthenticatedClientActionV2.java index 0cc5de4ca5213..5490543935333 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UnauthenticatedClientActionV2.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UnauthenticatedClientActionV2.java @@ -7,21 +7,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** The action to take when an unauthenticated client attempts to access the app. */ +/** + * The action to take when an unauthenticated client attempts to access the app. + */ public enum UnauthenticatedClientActionV2 { - /** Enum value RedirectToLoginPage. */ + /** + * Enum value RedirectToLoginPage. + */ REDIRECT_TO_LOGIN_PAGE("RedirectToLoginPage"), - /** Enum value AllowAnonymous. */ + /** + * Enum value AllowAnonymous. + */ ALLOW_ANONYMOUS("AllowAnonymous"), - /** Enum value Return401. */ + /** + * Enum value Return401. + */ RETURN401("Return401"), - /** Enum value Return403. */ + /** + * Enum value Return403. + */ RETURN403("Return403"); - /** The actual serialized value for a UnauthenticatedClientActionV2 instance. */ + /** + * The actual serialized value for a UnauthenticatedClientActionV2 instance. + */ private final String value; UnauthenticatedClientActionV2(String value) { @@ -30,7 +42,7 @@ public enum UnauthenticatedClientActionV2 { /** * Parses a serialized value to a UnauthenticatedClientActionV2 instance. - * + * * @param value the serialized value to parse. * @return the parsed UnauthenticatedClientActionV2 object, or null if unable to parse. */ @@ -48,7 +60,9 @@ public static UnauthenticatedClientActionV2 fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usage.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usage.java new file mode 100644 index 0000000000000..a47794e3061bc --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usage.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; + +/** + * An immutable client-side representation of Usage. + */ +public interface Usage { + /** + * Gets the unit property: An enum describing the unit of usage measurement. + * + * @return the unit value. + */ + String unit(); + + /** + * Gets the currentValue property: The current usage of the resource. + * + * @return the currentValue value. + */ + float currentValue(); + + /** + * Gets the limit property: The maximum permitted usage of the resource. + * + * @return the limit value. + */ + float limit(); + + /** + * Gets the name property: The name of the type of usage. + * + * @return the name value. + */ + UsageName name(); + + /** + * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.UsageInner object. + * + * @return the inner object. + */ + UsageInner innerModel(); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UsageName.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UsageName.java new file mode 100644 index 0000000000000..aab56b1c68b05 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UsageName.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Usage Names. + */ +@Fluent +public final class UsageName { + /* + * The name of the resource. + */ + @JsonProperty(value = "value") + private String value; + + /* + * The localized name of the resource. + */ + @JsonProperty(value = "localizedValue") + private String localizedValue; + + /** + * Creates an instance of UsageName class. + */ + public UsageName() { + } + + /** + * Get the value property: The name of the resource. + * + * @return the value value. + */ + public String value() { + return this.value; + } + + /** + * Set the value property: The name of the resource. + * + * @param value the value value to set. + * @return the UsageName object itself. + */ + public UsageName withValue(String value) { + this.value = value; + return this; + } + + /** + * Get the localizedValue property: The localized name of the resource. + * + * @return the localizedValue value. + */ + public String localizedValue() { + return this.localizedValue; + } + + /** + * Set the localizedValue property: The localized name of the resource. + * + * @param localizedValue the localizedValue value to set. + * @return the UsageName object itself. + */ + public UsageName withLocalizedValue(String localizedValue) { + this.localizedValue = localizedValue; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usages.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usages.java new file mode 100644 index 0000000000000..8121f34a32486 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Usages.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of Usages. + */ +public interface Usages { + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + PagedIterable list(String location); + + /** + * Gets, for the specified location, the current resource usage information as well as the limits under the + * subscription. + * + * @param location The location for which resource usage is queried. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException thrown if the request + * is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + PagedIterable list(String location, Context context); +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UserAssignedIdentity.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UserAssignedIdentity.java index 5d590ce71821b..0ab010bee6579 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UserAssignedIdentity.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/UserAssignedIdentity.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.UUID; -/** User assigned identity properties. */ +/** + * User assigned identity properties. + */ @Immutable public final class UserAssignedIdentity { /* @@ -23,13 +25,15 @@ public final class UserAssignedIdentity { @JsonProperty(value = "clientId", access = JsonProperty.Access.WRITE_ONLY) private UUID clientId; - /** Creates an instance of UserAssignedIdentity class. */ + /** + * Creates an instance of UserAssignedIdentity class. + */ public UserAssignedIdentity() { } /** * Get the principalId property: The principal ID of the assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -38,7 +42,7 @@ public UUID principalId() { /** * Get the clientId property: The client ID of the assigned identity. - * + * * @return the clientId value. */ public UUID clientId() { @@ -47,7 +51,7 @@ public UUID clientId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VnetConfiguration.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VnetConfiguration.java index 9ab43cd344391..acc516dd80935 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VnetConfiguration.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VnetConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration properties for apps environment to join a Virtual Network. */ +/** + * Configuration properties for apps environment to join a Virtual Network. + */ @Fluent public final class VnetConfiguration { /* @@ -44,7 +46,9 @@ public final class VnetConfiguration { @JsonProperty(value = "platformReservedDnsIP") private String platformReservedDnsIp; - /** Creates an instance of VnetConfiguration class. */ + /** + * Creates an instance of VnetConfiguration class. + */ public VnetConfiguration() { } @@ -52,7 +56,7 @@ public VnetConfiguration() { * Get the internal property: Boolean indicating the environment only has an internal load balancer. These * environments do not have a public static IP resource. They must provide infrastructureSubnetId if enabling this * property. - * + * * @return the internal value. */ public Boolean internal() { @@ -63,7 +67,7 @@ public Boolean internal() { * Set the internal property: Boolean indicating the environment only has an internal load balancer. These * environments do not have a public static IP resource. They must provide infrastructureSubnetId if enabling this * property. - * + * * @param internal the internal value to set. * @return the VnetConfiguration object itself. */ @@ -75,7 +79,7 @@ public VnetConfiguration withInternal(Boolean internal) { /** * Get the infrastructureSubnetId property: Resource ID of a subnet for infrastructure components. Must not overlap * with any other provided IP ranges. - * + * * @return the infrastructureSubnetId value. */ public String infrastructureSubnetId() { @@ -85,7 +89,7 @@ public String infrastructureSubnetId() { /** * Set the infrastructureSubnetId property: Resource ID of a subnet for infrastructure components. Must not overlap * with any other provided IP ranges. - * + * * @param infrastructureSubnetId the infrastructureSubnetId value to set. * @return the VnetConfiguration object itself. */ @@ -97,7 +101,7 @@ public VnetConfiguration withInfrastructureSubnetId(String infrastructureSubnetI /** * Get the dockerBridgeCidr property: CIDR notation IP range assigned to the Docker bridge, network. Must not * overlap with any other provided IP ranges. - * + * * @return the dockerBridgeCidr value. */ public String dockerBridgeCidr() { @@ -107,7 +111,7 @@ public String dockerBridgeCidr() { /** * Set the dockerBridgeCidr property: CIDR notation IP range assigned to the Docker bridge, network. Must not * overlap with any other provided IP ranges. - * + * * @param dockerBridgeCidr the dockerBridgeCidr value to set. * @return the VnetConfiguration object itself. */ @@ -119,7 +123,7 @@ public VnetConfiguration withDockerBridgeCidr(String dockerBridgeCidr) { /** * Get the platformReservedCidr property: IP range in CIDR notation that can be reserved for environment * infrastructure IP addresses. Must not overlap with any other provided IP ranges. - * + * * @return the platformReservedCidr value. */ public String platformReservedCidr() { @@ -129,7 +133,7 @@ public String platformReservedCidr() { /** * Set the platformReservedCidr property: IP range in CIDR notation that can be reserved for environment * infrastructure IP addresses. Must not overlap with any other provided IP ranges. - * + * * @param platformReservedCidr the platformReservedCidr value to set. * @return the VnetConfiguration object itself. */ @@ -139,9 +143,9 @@ public VnetConfiguration withPlatformReservedCidr(String platformReservedCidr) { } /** - * Get the platformReservedDnsIp property: An IP address from the IP range defined by platformReservedCidr that will - * be reserved for the internal DNS server. - * + * Get the platformReservedDnsIp property: An IP address from the IP range defined by platformReservedCidr that + * will be reserved for the internal DNS server. + * * @return the platformReservedDnsIp value. */ public String platformReservedDnsIp() { @@ -149,9 +153,9 @@ public String platformReservedDnsIp() { } /** - * Set the platformReservedDnsIp property: An IP address from the IP range defined by platformReservedCidr that will - * be reserved for the internal DNS server. - * + * Set the platformReservedDnsIp property: An IP address from the IP range defined by platformReservedCidr that + * will be reserved for the internal DNS server. + * * @param platformReservedDnsIp the platformReservedDnsIp value to set. * @return the VnetConfiguration object itself. */ @@ -162,7 +166,7 @@ public VnetConfiguration withPlatformReservedDnsIp(String platformReservedDnsIp) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Volume.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Volume.java index c14298b76ec7f..5f0f95c517a11 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Volume.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/Volume.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Volume definitions for the Container App. */ +/** + * Volume definitions for the Container App. + */ @Fluent public final class Volume { /* @@ -37,18 +39,21 @@ public final class Volume { private List secrets; /* - * Mount options used while mounting the AzureFile. Must be a comma-separated string. + * Mount options used while mounting the Azure file share or NFS Azure file share. Must be a comma-separated + * string. */ @JsonProperty(value = "mountOptions") private String mountOptions; - /** Creates an instance of Volume class. */ + /** + * Creates an instance of Volume class. + */ public Volume() { } /** * Get the name property: Volume name. - * + * * @return the name value. */ public String name() { @@ -57,7 +62,7 @@ public String name() { /** * Set the name property: Volume name. - * + * * @param name the name value to set. * @return the Volume object itself. */ @@ -68,7 +73,7 @@ public Volume withName(String name) { /** * Get the storageType property: Storage type for the volume. If not provided, use EmptyDir. - * + * * @return the storageType value. */ public StorageType storageType() { @@ -77,7 +82,7 @@ public StorageType storageType() { /** * Set the storageType property: Storage type for the volume. If not provided, use EmptyDir. - * + * * @param storageType the storageType value to set. * @return the Volume object itself. */ @@ -88,7 +93,7 @@ public Volume withStorageType(StorageType storageType) { /** * Get the storageName property: Name of storage resource. No need to provide for EmptyDir and Secret. - * + * * @return the storageName value. */ public String storageName() { @@ -97,7 +102,7 @@ public String storageName() { /** * Set the storageName property: Name of storage resource. No need to provide for EmptyDir and Secret. - * + * * @param storageName the storageName value to set. * @return the Volume object itself. */ @@ -109,7 +114,7 @@ public Volume withStorageName(String storageName) { /** * Get the secrets property: List of secrets to be added in volume. If no secrets are provided, all secrets in * collection will be added to volume. - * + * * @return the secrets value. */ public List secrets() { @@ -119,7 +124,7 @@ public List secrets() { /** * Set the secrets property: List of secrets to be added in volume. If no secrets are provided, all secrets in * collection will be added to volume. - * + * * @param secrets the secrets value to set. * @return the Volume object itself. */ @@ -129,8 +134,9 @@ public Volume withSecrets(List secrets) { } /** - * Get the mountOptions property: Mount options used while mounting the AzureFile. Must be a comma-separated string. - * + * Get the mountOptions property: Mount options used while mounting the Azure file share or NFS Azure file share. + * Must be a comma-separated string. + * * @return the mountOptions value. */ public String mountOptions() { @@ -138,8 +144,9 @@ public String mountOptions() { } /** - * Set the mountOptions property: Mount options used while mounting the AzureFile. Must be a comma-separated string. - * + * Set the mountOptions property: Mount options used while mounting the Azure file share or NFS Azure file share. + * Must be a comma-separated string. + * * @param mountOptions the mountOptions value to set. * @return the Volume object itself. */ @@ -150,7 +157,7 @@ public Volume withMountOptions(String mountOptions) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VolumeMount.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VolumeMount.java index ad15f4101e738..8e5dd84d45d8e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VolumeMount.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/VolumeMount.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Volume mount for the Container App. */ +/** + * Volume mount for the Container App. + */ @Fluent public final class VolumeMount { /* @@ -28,13 +30,15 @@ public final class VolumeMount { @JsonProperty(value = "subPath") private String subPath; - /** Creates an instance of VolumeMount class. */ + /** + * Creates an instance of VolumeMount class. + */ public VolumeMount() { } /** * Get the volumeName property: This must match the Name of a Volume. - * + * * @return the volumeName value. */ public String volumeName() { @@ -43,7 +47,7 @@ public String volumeName() { /** * Set the volumeName property: This must match the Name of a Volume. - * + * * @param volumeName the volumeName value to set. * @return the VolumeMount object itself. */ @@ -53,8 +57,9 @@ public VolumeMount withVolumeName(String volumeName) { } /** - * Get the mountPath property: Path within the container at which the volume should be mounted.Must not contain ':'. - * + * Get the mountPath property: Path within the container at which the volume should be mounted.Must not contain + * ':'. + * * @return the mountPath value. */ public String mountPath() { @@ -62,8 +67,9 @@ public String mountPath() { } /** - * Set the mountPath property: Path within the container at which the volume should be mounted.Must not contain ':'. - * + * Set the mountPath property: Path within the container at which the volume should be mounted.Must not contain + * ':'. + * * @param mountPath the mountPath value to set. * @return the VolumeMount object itself. */ @@ -73,9 +79,9 @@ public VolumeMount withMountPath(String mountPath) { } /** - * Get the subPath property: Path within the volume from which the container's volume should be mounted. Defaults to - * "" (volume's root). - * + * Get the subPath property: Path within the volume from which the container's volume should be mounted. Defaults + * to "" (volume's root). + * * @return the subPath value. */ public String subPath() { @@ -83,9 +89,9 @@ public String subPath() { } /** - * Set the subPath property: Path within the volume from which the container's volume should be mounted. Defaults to - * "" (volume's root). - * + * Set the subPath property: Path within the volume from which the container's volume should be mounted. Defaults + * to "" (volume's root). + * * @param subPath the subPath value to set. * @return the VolumeMount object itself. */ @@ -96,7 +102,7 @@ public VolumeMount withSubPath(String subPath) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfile.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfile.java index 045be9814561c..066d2ac72069c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfile.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfile.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Workload profile to scope container app execution. */ +/** + * Workload profile to scope container app execution. + */ @Fluent public final class WorkloadProfile { /* @@ -35,13 +37,15 @@ public final class WorkloadProfile { @JsonProperty(value = "maximumCount") private Integer maximumCount; - /** Creates an instance of WorkloadProfile class. */ + /** + * Creates an instance of WorkloadProfile class. + */ public WorkloadProfile() { } /** * Get the name property: Workload profile type for the workloads to run on. - * + * * @return the name value. */ public String name() { @@ -50,7 +54,7 @@ public String name() { /** * Set the name property: Workload profile type for the workloads to run on. - * + * * @param name the name value to set. * @return the WorkloadProfile object itself. */ @@ -61,7 +65,7 @@ public WorkloadProfile withName(String name) { /** * Get the workloadProfileType property: Workload profile type for the workloads to run on. - * + * * @return the workloadProfileType value. */ public String workloadProfileType() { @@ -70,7 +74,7 @@ public String workloadProfileType() { /** * Set the workloadProfileType property: Workload profile type for the workloads to run on. - * + * * @param workloadProfileType the workloadProfileType value to set. * @return the WorkloadProfile object itself. */ @@ -81,7 +85,7 @@ public WorkloadProfile withWorkloadProfileType(String workloadProfileType) { /** * Get the minimumCount property: The minimum capacity. - * + * * @return the minimumCount value. */ public Integer minimumCount() { @@ -90,7 +94,7 @@ public Integer minimumCount() { /** * Set the minimumCount property: The minimum capacity. - * + * * @param minimumCount the minimumCount value to set. * @return the WorkloadProfile object itself. */ @@ -101,7 +105,7 @@ public WorkloadProfile withMinimumCount(Integer minimumCount) { /** * Get the maximumCount property: The maximum capacity. - * + * * @return the maximumCount value. */ public Integer maximumCount() { @@ -110,7 +114,7 @@ public Integer maximumCount() { /** * Set the maximumCount property: The maximum capacity. - * + * * @param maximumCount the maximumCount value to set. * @return the WorkloadProfile object itself. */ @@ -121,20 +125,17 @@ public WorkloadProfile withMaximumCount(Integer maximumCount) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model WorkloadProfile")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model WorkloadProfile")); } if (workloadProfileType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property workloadProfileType in model WorkloadProfile")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property workloadProfileType in model WorkloadProfile")); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStates.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStates.java index db01b9b5bb5a5..68703405f0124 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStates.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStates.java @@ -4,41 +4,51 @@ package com.azure.resourcemanager.appcontainers.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.appcontainers.fluent.models.WorkloadProfileStatesInner; -/** An immutable client-side representation of WorkloadProfileStates. */ +/** + * An immutable client-side representation of WorkloadProfileStates. + */ public interface WorkloadProfileStates { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the properties property: Workload Profile resource specific properties. - * + * * @return the properties value. */ WorkloadProfileStatesProperties properties(); + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + /** * Gets the inner com.azure.resourcemanager.appcontainers.fluent.models.WorkloadProfileStatesInner object. - * + * * @return the inner object. */ WorkloadProfileStatesInner innerModel(); diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesCollection.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesCollection.java index b863517afddd2..d8e70bc82c768 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesCollection.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesCollection.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Collection of workloadProfileStates. */ +/** + * Collection of workloadProfileStates. + */ @Fluent public final class WorkloadProfileStatesCollection { /* @@ -25,13 +27,15 @@ public final class WorkloadProfileStatesCollection { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of WorkloadProfileStatesCollection class. */ + /** + * Creates an instance of WorkloadProfileStatesCollection class. + */ public WorkloadProfileStatesCollection() { } /** * Get the value property: Collection of resources. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of resources. - * + * * @param value the value value to set. * @return the WorkloadProfileStatesCollection object itself. */ @@ -51,7 +55,7 @@ public WorkloadProfileStatesCollection withValue(List e.validate()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesProperties.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesProperties.java index f385197880887..1c4937e687ff5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesProperties.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/WorkloadProfileStatesProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Workload Profile resource specific properties. */ +/** + * Workload Profile resource specific properties. + */ @Fluent public final class WorkloadProfileStatesProperties { /* @@ -28,13 +30,15 @@ public final class WorkloadProfileStatesProperties { @JsonProperty(value = "currentCount") private Integer currentCount; - /** Creates an instance of WorkloadProfileStatesProperties class. */ + /** + * Creates an instance of WorkloadProfileStatesProperties class. + */ public WorkloadProfileStatesProperties() { } /** * Get the minimumCount property: Minimum count of instances. - * + * * @return the minimumCount value. */ public Integer minimumCount() { @@ -43,7 +47,7 @@ public Integer minimumCount() { /** * Set the minimumCount property: Minimum count of instances. - * + * * @param minimumCount the minimumCount value to set. * @return the WorkloadProfileStatesProperties object itself. */ @@ -54,7 +58,7 @@ public WorkloadProfileStatesProperties withMinimumCount(Integer minimumCount) { /** * Get the maximumCount property: Maximum count of nodes. - * + * * @return the maximumCount value. */ public Integer maximumCount() { @@ -63,7 +67,7 @@ public Integer maximumCount() { /** * Set the maximumCount property: Maximum count of nodes. - * + * * @param maximumCount the maximumCount value to set. * @return the WorkloadProfileStatesProperties object itself. */ @@ -74,7 +78,7 @@ public WorkloadProfileStatesProperties withMaximumCount(Integer maximumCount) { /** * Get the currentCount property: Current count of nodes. - * + * * @return the currentCount value. */ public Integer currentCount() { @@ -83,7 +87,7 @@ public Integer currentCount() { /** * Set the currentCount property: Current count of nodes. - * + * * @param currentCount the currentCount value to set. * @return the WorkloadProfileStatesProperties object itself. */ @@ -94,7 +98,7 @@ public WorkloadProfileStatesProperties withCurrentCount(Integer currentCount) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/package-info.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/package-info.java index 8fa357d7560c2..a5d836af42d9f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/package-info.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the data models for ContainerAppsApiClient. null. */ +/** + * Package containing the data models for ContainerAppsApiClient. + * null. + */ package com.azure.resourcemanager.appcontainers.models; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/package-info.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/package-info.java index ed78d26488435..82dc538e0b9f7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/package-info.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/com/azure/resourcemanager/appcontainers/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for ContainerAppsApiClient. null. */ +/** + * Package containing the classes for ContainerAppsApiClient. + * null. + */ package com.azure.resourcemanager.appcontainers; diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/module-info.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/module-info.java index 465278559546b..e63ceaf542d12 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/module-info.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.appcontainers { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.appcontainers; exports com.azure.resourcemanager.appcontainers.fluent; exports com.azure.resourcemanager.appcontainers.fluent.models; exports com.azure.resourcemanager.appcontainers.models; - - opens com.azure.resourcemanager.appcontainers.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.appcontainers.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.appcontainers.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.appcontainers.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/proxy-config.json b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/proxy-config.json new file mode 100644 index 0000000000000..8de18d9e34b17 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.appcontainers.implementation.AppResilienciesClientImpl$AppResilienciesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ContainerAppsAuthConfigsClientImpl$ContainerAppsAuthConfigsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.AvailableWorkloadProfilesClientImpl$AvailableWorkloadProfilesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.BillingMetersClientImpl$BillingMetersService" ], [ "com.azure.resourcemanager.appcontainers.implementation.BuildersClientImpl$BuildersService" ], [ "com.azure.resourcemanager.appcontainers.implementation.BuildsByBuilderResourcesClientImpl$BuildsByBuilderResourcesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.BuildsClientImpl$BuildsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.BuildAuthTokensClientImpl$BuildAuthTokensService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ConnectedEnvironmentsClientImpl$ConnectedEnvironmentsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ConnectedEnvironmentsCertificatesClientImpl$ConnectedEnvironmentsCertificatesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ConnectedEnvironmentsDaprComponentsClientImpl$ConnectedEnvironmentsDaprComponentsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ConnectedEnvironmentsStoragesClientImpl$ConnectedEnvironmentsStoragesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ContainerAppsClientImpl$ContainerAppsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ContainerAppsRevisionsClientImpl$ContainerAppsRevisionsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ContainerAppsRevisionReplicasClientImpl$ContainerAppsRevisionReplicasService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ContainerAppsDiagnosticsClientImpl$ContainerAppsDiagnosticsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentDiagnosticsClientImpl$ManagedEnvironmentDiagnosticsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentsDiagnosticsClientImpl$ManagedEnvironmentsDiagnosticsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.JobsClientImpl$JobsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.JobsExecutionsClientImpl$JobsExecutionsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ResourceProvidersClientImpl$ResourceProvidersService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentsClientImpl$ManagedEnvironmentsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.CertificatesClientImpl$CertificatesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ManagedCertificatesClientImpl$ManagedCertificatesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.NamespacesClientImpl$NamespacesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.DaprComponentResiliencyPoliciesClientImpl$DaprComponentResiliencyPoliciesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.DaprComponentsClientImpl$DaprComponentsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.DaprSubscriptionsClientImpl$DaprSubscriptionsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentsStoragesClientImpl$ManagedEnvironmentsStoragesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ContainerAppsSourceControlsClientImpl$ContainerAppsSourceControlsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.UsagesClientImpl$UsagesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.ManagedEnvironmentUsagesClientImpl$ManagedEnvironmentUsagesService" ], [ "com.azure.resourcemanager.appcontainers.implementation.JavaComponentsClientImpl$JavaComponentsService" ], [ "com.azure.resourcemanager.appcontainers.implementation.DotNetComponentsClientImpl$DotNetComponentsService" ] ] \ No newline at end of file diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/reflect-config.json b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/reflect-config.json new file mode 100644 index 0000000000000..daaeffa7283ae --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-appcontainers/reflect-config.json @@ -0,0 +1,1486 @@ +[ { + "name" : "com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseErrorException", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.AppResiliencyProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TimeoutPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HttpRetryPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.HttpRetryPolicyRetryBackOff", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.HttpRetryPolicyMatches", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HeaderMatch", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.HeaderMatchMatch", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HttpConnectionPool", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TcpConnectionPool", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DefaultErrorResponseError", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AppResiliencyCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AuthConfigCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AuthPlatform", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.GlobalValidation", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IdentityProviders", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureActiveDirectory", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureActiveDirectoryRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureActiveDirectoryLogin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureActiveDirectoryValidation", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JwtClaimChecks", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DefaultAuthorizationPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AllowedPrincipals", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Facebook", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AppRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.LoginScopes", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.GitHub", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ClientRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Google", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AllowedAudiencesValidation", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Twitter", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TwitterRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Apple", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AppleRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureStaticWebApps", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureStaticWebAppsRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CustomOpenIdConnectProvider", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OpenIdConnectRegistration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OpenIdConnectClientCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OpenIdConnectConfig", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OpenIdConnectLogin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Login", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.LoginRoutes", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TokenStore", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BlobStorageTokenStore", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CookieExpiration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Nonce", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HttpSettings", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HttpSettingsRoutes", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ForwardProxy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.EncryptionSettings", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfilesCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.AvailableWorkloadProfileInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AvailableWorkloadProfileProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BillingMeterCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BillingMeter", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BillingMeterProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuilderCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BuilderProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerRegistry", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuilderResourceUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceUpdateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuildCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BuildResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BuildProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuildConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.EnvironmentVariable", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.PreBuildStep", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HttpGet", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.BuildTokenInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ExtendedLocation", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CertificateKeyVaultProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.CheckNameAvailabilityResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CertificateCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CertificateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CertificatePatch", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentsCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Secret", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprMetadata", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprSecretsCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprSecret", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStoragesCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ConnectedEnvironmentStorageInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentStorageProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureFileProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Configuration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Ingress", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TrafficWeight", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CustomDomain", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IpSecurityRestrictionRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IngressStickySessions", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CorsPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IngressPortMapping", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.RegistryCredentials", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Dapr", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Service", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Template", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.InitContainer", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BaseContainer", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.EnvironmentVar", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerResources", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.VolumeMount", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Container", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppProbe", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppProbeHttpGet", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppProbeHttpGetHttpHeadersItem", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppProbeTcpSocket", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Scale", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ScaleRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.QueueScaleRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ScaleRuleAuth", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CustomScaleRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.HttpScaleRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TcpScaleRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Volume", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.SecretVolumeItem", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ServiceBind", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.CustomHostnameAnalysisResultInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CustomHostnameAnalysisResultCustomDomainVerificationFailureInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CustomHostnameAnalysisResultCustomDomainVerificationFailureInfoDetailsItem", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.SecretsCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppSecret", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppAuthTokenInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppAuthTokenProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.RevisionCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.RevisionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.RevisionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ReplicaInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ReplicaProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ReplicaContainer", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ReplicaCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DiagnosticsInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticsProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticsDefinition", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticSupportTopic", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticsDataApiResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticDataTableResponseObject", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticDataTableResponseColumn", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticRendering", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticDataProviderMetadata", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DiagnosticDataProviderMetadataPropertyBagItem", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.VnetConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AppLogsConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.LogAnalyticsConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OpenTelemetryConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DestinationsConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DataDogConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OtlpConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Header", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TracesConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.LogsConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.MetricsConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.WorkloadProfile", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.KedaConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentPropertiesPeerAuthentication", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Mtls", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JobInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JobProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobConfigurationManualTriggerConfig", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobConfigurationScheduleTriggerConfig", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobConfigurationEventTriggerConfig", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobScale", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobScaleRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobTemplate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AvailableOperations", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.OperationDetailInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobsCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobPatchProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobPatchPropertiesProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobExecutionContainer", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionBaseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JobSecretsCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentsCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ManagedCertificateInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedCertificateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedCertificatePatch", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedCertificateCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.EnvironmentAuthTokenInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.EnvironmentAuthTokenProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.WorkloadProfileStatesCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.WorkloadProfileStatesInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.WorkloadProfileStatesProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPoliciesCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprSubscriptionsCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStoragesCollectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.ManagedEnvironmentStorageInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentStorageProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.NfsAzureFileProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.SourceControlCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.SourceControlInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.SourceControlProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.GithubActionConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.RegistryInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AzureCredentials", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ListUsagesResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.UsageInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.UsageName", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JavaComponentsCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DotNetComponentsCollection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Applicability", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuilderProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuildProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BuildStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ExtendedLocationTypes", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityReason", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CertificateProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CertificateType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AccessMode", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ActiveRevisionsMode", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IngressTransportMethod", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.BindingType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Action", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Affinity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IngressClientCertificateMode", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.IngressTargetPortHttpScheme", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.AppProtocol", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.LogLevel", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Scheme", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.Type", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.StorageType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.RevisionHealthState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.RevisionProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.RevisionRunningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppReplicaRunningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ContainerAppContainerRunningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.EnvironmentProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.TriggerType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JobExecutionRunningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ManagedCertificateDomainControlValidation", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.SourceControlOperationState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JavaComponentType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.JavaComponentProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DotNetComponentType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DotNetComponentProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.UnauthenticatedClientActionV2", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.CookieExpirationConvention", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ForwardProxyConvention", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.DnsVerificationTestResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.appcontainers.models.ClientCredentialMethod", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..f2dcd1e3fd982 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyCreateOrUpdateSamples.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy; +import com.azure.resourcemanager.appcontainers.models.HeaderMatch; +import com.azure.resourcemanager.appcontainers.models.HttpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.HttpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TcpConnectionPool; +import com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy; +import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy; +import java.util.Arrays; + +/** + * Samples for AppResiliency CreateOrUpdate. + */ +public final class AppResiliencyCreateOrUpdateSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_CreateOrUpdate + * .json + */ + /** + * Sample code: Create or Update App Resiliency. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + createOrUpdateAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.appResiliencies().define("resiliency-policy-1").withExistingContainerApp("rg", "testcontainerApp0") + .withTimeoutPolicy(new TimeoutPolicy().withResponseTimeoutInSeconds(15).withConnectionTimeoutInSeconds(5)) + .withHttpRetryPolicy(new HttpRetryPolicy().withMaxRetries(5).withInitialDelayInMilliseconds(1000L) + .withMaxIntervalInMilliseconds(10000L) + .withHeaders( + Arrays.asList(new HeaderMatch().withHeaderProperty("X-Content-Type").withPrefixMatch("GOATS"))) + .withHttpStatusCodes(Arrays.asList(502, 503)).withErrors( + Arrays.asList("5xx", "connect-failure", "reset", "retriable-headers", "retriable-status-codes"))) + .withTcpRetryPolicy(new TcpRetryPolicy().withMaxConnectAttempts(3)) + .withCircuitBreakerPolicy(new CircuitBreakerPolicy().withConsecutiveErrors(5).withIntervalInSeconds(10) + .withMaxEjectionPercent(50)) + .withHttpConnectionPool( + new HttpConnectionPool().withHttp1MaxPendingRequests(1024).withHttp2MaxRequests(1024)) + .withTcpConnectionPool(new TcpConnectionPool().withMaxConnections(100)).create(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyDeleteSamples.java new file mode 100644 index 0000000000000..ff432a623f649 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyDeleteSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for AppResiliency Delete. + */ +public final class AppResiliencyDeleteSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_Delete.json + */ + /** + * Sample code: Delete App Resiliency. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void deleteAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.appResiliencies().deleteWithResponse("rg", "testcontainerApp0", "resiliency-policy-1", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyGetSamples.java new file mode 100644 index 0000000000000..a0a5bd608f42c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyGetSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for AppResiliency Get. + */ +public final class AppResiliencyGetSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_Get.json + */ + /** + * Sample code: Get App Resiliency. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.appResiliencies().getWithResponse("rg", "testcontainerApp0", "resiliency-policy-1", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyListSamples.java new file mode 100644 index 0000000000000..dd4f9d77ee940 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyListSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for AppResiliency List. + */ +public final class AppResiliencyListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_List.json + */ + /** + * Sample code: List App Resiliency. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void listAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.appResiliencies().list("rg", "testcontainerApp0", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyUpdateSamples.java new file mode 100644 index 0000000000000..e0d26b251899d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AppResiliencyUpdateSamples.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.AppResiliency; +import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy; + +/** + * Samples for AppResiliency Update. + */ +public final class AppResiliencyUpdateSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AppResiliency_Patch.json + */ + /** + * Sample code: Update App Resiliency. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void updateAppResiliency(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + AppResiliency resource = manager.appResiliencies() + .getWithResponse("rg", "testcontainerApp0", "resiliency-policy-1", com.azure.core.util.Context.NONE) + .getValue(); + resource.update() + .withTimeoutPolicy(new TimeoutPolicy().withResponseTimeoutInSeconds(30).withConnectionTimeoutInSeconds(40)) + .apply(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetSamples.java index fc26f7c84d77e..628041decb5c8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for AvailableWorkloadProfiles Get. */ +/** + * Samples for AvailableWorkloadProfiles Get. + */ public final class AvailableWorkloadProfilesGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AvailableWorkloadProfiles_Get.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * AvailableWorkloadProfiles_Get.json */ /** * Sample code: BillingMeters_Get. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void billingMetersGet(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetSamples.java index 2d96fe6be1492..506e65cca4d18 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for BillingMeters Get. */ +/** + * Samples for BillingMeters Get. + */ public final class BillingMetersGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/BillingMeters_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/BillingMeters_Get.json */ /** * Sample code: BillingMeters_Get. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void billingMetersGet(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildAuthTokenListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildAuthTokenListSamples.java new file mode 100644 index 0000000000000..c364ecfe64ff1 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildAuthTokenListSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for BuildAuthToken List. + */ +public final class BuildAuthTokenListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_ListAuthToken.json + */ + /** + * Sample code: Get Build Auth Token. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getBuildAuthToken(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.buildAuthTokens().listWithResponse("rg", "testBuilder", "testBuild", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..227168e3fe8fe --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateSamples.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Builders CreateOrUpdate. + */ +public final class BuildersCreateOrUpdateSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_CreateOrUpdate.json + */ + /** + * Sample code: Builders_CreateOrUpdate_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + buildersCreateOrUpdate0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builders().define("testBuilder").withRegion("eastus").withExistingResourceGroup("rg") + .withTags(mapOf("company", "Microsoft")) + .withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/identity1", + new UserAssignedIdentity()))) + .withEnvironmentId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg/providers/Microsoft.App/managedEnvironments/testEnv") + .withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("test.azurecr.io").withIdentityResourceId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/identity1"), + new ContainerRegistry().withContainerRegistryServer("test2.azurecr.io").withIdentityResourceId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/identity1"))) + .create(); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersDeleteSamples.java new file mode 100644 index 0000000000000..00990722478d2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersDeleteSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Builders Delete. + */ +public final class BuildersDeleteSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_Delete.json + */ + /** + * Sample code: Builders_Delete_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void buildersDelete0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builders().delete("rg", "testBuilder", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupSamples.java new file mode 100644 index 0000000000000..9d1b5ef1a93bd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Builders GetByResourceGroup. + */ +public final class BuildersGetByResourceGroupSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_Get.json + */ + /** + * Sample code: Builders_Get_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void buildersGet0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builders().getByResourceGroupWithResponse("rg", "testBuilder", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupSamples.java new file mode 100644 index 0000000000000..96827840fea88 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Builders ListByResourceGroup. + */ +public final class BuildersListByResourceGroupSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_ListByResourceGroup + * .json + */ + /** + * Sample code: Builders_ListByResourceGroup_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + buildersListByResourceGroup0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builders().listByResourceGroup("rg", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListSamples.java new file mode 100644 index 0000000000000..cd9a5c76be8a6 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersListSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Builders List. + */ +public final class BuildersListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_ListBySubscription. + * json + */ + /** + * Sample code: Builders_ListBySubscription_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + buildersListBySubscription0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builders().list(com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersUpdateSamples.java new file mode 100644 index 0000000000000..db6b95d59959a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildersUpdateSamples.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Builders Update. + */ +public final class BuildersUpdateSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builders_Update.json + */ + /** + * Sample code: Builders_Update_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void buildersUpdate0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + BuilderResource resource = manager.builders() + .getByResourceGroupWithResponse("rg", "testBuilder", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("mytag1", "myvalue1")).apply(); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsByBuilderResourceListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsByBuilderResourceListSamples.java new file mode 100644 index 0000000000000..0234c29a09b2f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsByBuilderResourceListSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for BuildsByBuilderResource List. + */ +public final class BuildsByBuilderResourceListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_ListByBuilderResource + * .json + */ + /** + * Sample code: Builds_ListByBuilderResource_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + buildsListByBuilderResource0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.buildsByBuilderResources().list("rg", "testBuilder", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..1b1cf25bd03d5 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsCreateOrUpdateSamples.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.BuildConfiguration; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage; +import com.azure.resourcemanager.appcontainers.models.EnvironmentVariable; +import com.azure.resourcemanager.appcontainers.models.HttpGet; +import com.azure.resourcemanager.appcontainers.models.PreBuildStep; +import java.util.Arrays; + +/** + * Samples for Builds CreateOrUpdate. + */ +public final class BuildsCreateOrUpdateSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_CreateOrUpdate.json + */ + /** + * Sample code: Builds_CreateOrUpdate_WithConfig. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + buildsCreateOrUpdateWithConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builds().define("testBuild-123456789az").withExistingBuilder("rg", "testBuilder") + .withDestinationContainerRegistry(new ContainerRegistryWithCustomImage().withServer("test.azurecr.io") + .withImage("test.azurecr.io/repo:tag")) + .withConfiguration( + new BuildConfiguration().withBaseOs("DebianBullseye").withPlatform("dotnetcore") + .withPlatformVersion("7.0") + .withEnvironmentVariables( + Arrays.asList(new EnvironmentVariable().withName("foo1").withValue("bar1"), + new EnvironmentVariable().withName("foo2").withValue("bar2"))) + .withPreBuildSteps(Arrays.asList( + new PreBuildStep().withDescription("First pre build step.") + .withScripts(Arrays.asList("echo 'hello'", "echo 'world'")) + .withHttpGet(new HttpGet().withUrl("https://microsoft.com").withFileName("output.txt") + .withHeaders(Arrays.asList("foo", "bar"))), + new PreBuildStep().withDescription("Second pre build step.") + .withScripts(Arrays.asList("echo 'hello'", "echo 'again'")) + .withHttpGet(new HttpGet().withUrl("https://microsoft.com").withFileName("output.txt") + .withHeaders(Arrays.asList("foo")))))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Builds_CreateOrUpdate_NoConfig.json + */ + /** + * Sample code: Builds_CreateOrUpdate_NoConfig. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + buildsCreateOrUpdateNoConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builds().define("testBuild").withExistingBuilder("rg", "testBuilder").create(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsDeleteSamples.java new file mode 100644 index 0000000000000..85b1819a1ed1e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsDeleteSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Builds Delete. + */ +public final class BuildsDeleteSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_Delete.json + */ + /** + * Sample code: Builds_Delete_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void buildsDelete0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builds().delete("rg", "testBuilder", "testBuild", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsGetSamples.java new file mode 100644 index 0000000000000..274c47086205f --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/BuildsGetSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Builds Get. + */ +public final class BuildsGetSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Builds_Get.json + */ + /** + * Sample code: Builds_Get_0. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void buildsGet0(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.builds().getWithResponse("rg", "testBuilder", "testBuild", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesCreateOrUpdateSamples.java index 7d3ada5340872..0679d0bd5325c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesCreateOrUpdateSamples.java @@ -5,32 +5,50 @@ package com.azure.resourcemanager.appcontainers.generated; import com.azure.resourcemanager.appcontainers.fluent.models.CertificateInner; +import com.azure.resourcemanager.appcontainers.models.CertificateKeyVaultProperties; import com.azure.resourcemanager.appcontainers.models.CertificateProperties; +import com.azure.resourcemanager.appcontainers.models.CertificateType; -/** Samples for Certificates CreateOrUpdate. */ +/** + * Samples for Certificates CreateOrUpdate. + */ public final class CertificatesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificate_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Certificate_CreateOrUpdate_FromKeyVault.json */ /** - * Sample code: Create or Update Certificate. - * + * Sample code: Create or Update Certificate using Managed Identity. + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateCertificate( + public static void createOrUpdateCertificateUsingManagedIdentity( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .certificates() - .createOrUpdateWithResponse( - "examplerg", - "testcontainerenv", - "certificate-firendly-name", - new CertificateInner() - .withLocation("East US") - .withProperties( - new CertificateProperties() - .withPassword("fakeTokenPlaceholder") - .withValue("Y2VydA==".getBytes())), - com.azure.core.util.Context.NONE); + manager.certificates().createOrUpdateWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + new CertificateInner().withLocation("East US").withProperties(new CertificateProperties() + .withCertificateKeyVaultProperties(new CertificateKeyVaultProperties().withIdentity( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/test-rg/providers/microsoft.managedidentity/userassignedidentities/test-user-mi") + .withKeyVaultUrl("fakeTokenPlaceholder")) + .withCertificateType(CertificateType.SERVER_SSLCERTIFICATE)), + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificate_CreateOrUpdate. + * json + */ + /** + * Sample code: Create or Update Certificate. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + createOrUpdateCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.certificates().createOrUpdateWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + new CertificateInner().withLocation("East US") + .withProperties(new CertificateProperties().withPassword("fakeTokenPlaceholder") + .withValue("Y2VydA==".getBytes()).withCertificateType(CertificateType.IMAGE_PULL_TRUSTED_CA)), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteSamples.java index cc39bc23942ec..51fd5931b8c76 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Certificates Delete. */ +/** + * Samples for Certificates Delete. + */ public final class CertificatesDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificate_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificate_Delete.json */ /** * Sample code: Delete Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .certificates() - .deleteWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE); + manager.certificates().deleteWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesGetSamples.java index cd2c70dc99a4d..89b21554efc59 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesGetSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Certificates Get. */ +/** + * Samples for Certificates Get. + */ public final class CertificatesGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificate_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificate_Get.json */ /** * Sample code: Get Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .certificates() - .getWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE); + manager.certificates().getWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesListSamples.java index b22269ab3bdbc..ed4587d1ba6a1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesListSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Certificates List. */ +/** + * Samples for Certificates List. + */ public final class CertificatesListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificates_ListByManagedEnvironment.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Certificates_ListByManagedEnvironment.json */ /** * Sample code: List Certificates by Managed Environment. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listCertificatesByManagedEnvironment( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listCertificatesByManagedEnvironment(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.certificates().list("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesUpdateSamples.java index 709fe08925a65..293cf822faea5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/CertificatesUpdateSamples.java @@ -8,25 +8,23 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Certificates Update. */ +/** + * Samples for Certificates Update. + */ public final class CertificatesUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificates_Patch.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Certificates_Patch.json */ /** * Sample code: Patch Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void patchCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .certificates() - .updateWithResponse( - "examplerg", - "testcontainerenv", - "certificate-firendly-name", - new CertificatePatch().withTags(mapOf("tag1", "value1", "tag2", "value2")), - com.azure.core.util.Context.NONE); + manager.certificates().updateWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + new CertificatePatch().withTags(mapOf("tag1", "value1", "tag2", "value2")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesCreateOrUpdateSamples.java index 34426b12ab877..a6e0e0999cd04 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesCreateOrUpdateSamples.java @@ -6,22 +6,22 @@ import com.azure.resourcemanager.appcontainers.models.CertificateProperties; -/** Samples for ConnectedEnvironmentsCertificates CreateOrUpdate. */ +/** + * Samples for ConnectedEnvironmentsCertificates CreateOrUpdate. + */ public final class ConnectedEnvironmentsCertificatesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificate_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsCertificate_CreateOrUpdate.json */ /** * Sample code: Create or Update Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateCertificate( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsCertificates() - .define("certificate-firendly-name") - .withRegion("East US") + public static void + createOrUpdateCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.connectedEnvironmentsCertificates().define("certificate-firendly-name").withRegion("East US") .withExistingConnectedEnvironment("examplerg", "testcontainerenv") .withProperties( new CertificateProperties().withPassword("fakeTokenPlaceholder").withValue("Y2VydA==".getBytes())) diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteSamples.java index d61ec9dbec42b..8ba3c58c59a45 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsCertificates Delete. */ +/** + * Samples for ConnectedEnvironmentsCertificates Delete. + */ public final class ConnectedEnvironmentsCertificatesDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificate_Delete.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsCertificate_Delete.json */ /** * Sample code: Delete Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsCertificates() - .deleteWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsCertificates().deleteWithResponse("examplerg", "testcontainerenv", + "certificate-firendly-name", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesGetSamples.java index e6631de95ac40..55e10d25a7251 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesGetSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsCertificates Get. */ +/** + * Samples for ConnectedEnvironmentsCertificates Get. + */ public final class ConnectedEnvironmentsCertificatesGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificate_Get.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsCertificate_Get.json */ /** * Sample code: Get Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsCertificates() - .getWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsCertificates().getWithResponse("examplerg", "testcontainerenv", + "certificate-firendly-name", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesListSamples.java index 2c2a8fd6e499a..8c1d0632f8c6d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesListSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsCertificates List. */ +/** + * Samples for ConnectedEnvironmentsCertificates List. + */ public final class ConnectedEnvironmentsCertificatesListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificates_ListByConnectedEnvironment.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsCertificates_ListByConnectedEnvironment.json */ /** * Sample code: List Certificates by Connected Environment. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listCertificatesByConnectedEnvironment( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsCertificates() - .list("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsCertificates().list("examplerg", "testcontainerenv", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesUpdateSamples.java index 2ad18da66708d..3204b307a089d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesUpdateSamples.java @@ -8,23 +8,22 @@ import java.util.HashMap; import java.util.Map; -/** Samples for ConnectedEnvironmentsCertificates Update. */ +/** + * Samples for ConnectedEnvironmentsCertificates Update. + */ public final class ConnectedEnvironmentsCertificatesUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificates_Patch.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsCertificates_Patch.json */ /** * Sample code: Patch Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void patchCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - Certificate resource = - manager - .connectedEnvironmentsCertificates() - .getWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE) - .getValue(); + Certificate resource = manager.connectedEnvironmentsCertificates().getWithResponse("examplerg", + "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilitySamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilitySamples.java index 0d86087d57820..c5609981289cc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilitySamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilitySamples.java @@ -6,26 +6,24 @@ import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest; -/** Samples for ConnectedEnvironments CheckNameAvailability. */ +/** + * Samples for ConnectedEnvironments CheckNameAvailability. + */ public final class ConnectedEnvironmentsCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsCertificates_CheckNameAvailability.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsCertificates_CheckNameAvailability.json */ /** * Sample code: Certificates_CheckNameAvailability. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void certificatesCheckNameAvailability( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironments() - .checkNameAvailabilityWithResponse( - "examplerg", - "testcontainerenv", - new CheckNameAvailabilityRequest() - .withName("testcertificatename") - .withType("Microsoft.App/connectedEnvironments/certificates"), + public static void + certificatesCheckNameAvailability(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.connectedEnvironments() + .checkNameAvailabilityWithResponse("examplerg", "testcontainerenv", new CheckNameAvailabilityRequest() + .withName("testcertificatename").withType("Microsoft.App/connectedEnvironments/certificates"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCreateOrUpdateSamples.java index e6faa26d9ac2a..132aecd42e86f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCreateOrUpdateSamples.java @@ -6,30 +6,26 @@ import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration; -/** Samples for ConnectedEnvironments CreateOrUpdate. */ +/** + * Samples for ConnectedEnvironments CreateOrUpdate. + */ public final class ConnectedEnvironmentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironments_CreateOrUpdate.json */ /** * Sample code: Create kube environments. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createKubeEnvironments(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironments() - .define("testenv") - .withRegion("East US") - .withExistingResourceGroup("examplerg") + manager.connectedEnvironments().define("testenv").withRegion("East US").withExistingResourceGroup("examplerg") .withStaticIp("1.2.3.4") .withDaprAIConnectionString( "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/") - .withCustomDomainConfiguration( - new CustomDomainConfiguration() - .withDnsSuffix("www.my-name.com") - .withCertificateValue("Y2VydA==".getBytes()) - .withCertificatePassword("fakeTokenPlaceholder")) + .withCustomDomainConfiguration(new CustomDomainConfiguration().withDnsSuffix("www.my-name.com") + .withCertificateValue("Y2VydA==".getBytes()).withCertificatePassword("fakeTokenPlaceholder")) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples.java index 0f41b4e6ae475..c80b32b2c3d53 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples.java @@ -4,39 +4,40 @@ package com.azure.resourcemanager.appcontainers.generated; +import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding; import com.azure.resourcemanager.appcontainers.models.DaprMetadata; +import com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata; import com.azure.resourcemanager.appcontainers.models.Secret; import java.util.Arrays; -/** Samples for ConnectedEnvironmentsDaprComponents CreateOrUpdate. */ +/** + * Samples for ConnectedEnvironmentsDaprComponents CreateOrUpdate. + */ public final class ConnectedEnvironmentsDaprComponentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsDaprComponents_CreateOrUpdate.json */ /** * Sample code: Create or update dapr component. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateDaprComponent( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsDaprComponents() - .define("reddog") - .withExistingConnectedEnvironment("examplerg", "myenvironment") - .withComponentType("state.azure.cosmosdb") - .withVersion("v1") - .withIgnoreErrors(false) - .withInitTimeout("50s") + public static void + createOrUpdateDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.connectedEnvironmentsDaprComponents().define("reddog") + .withExistingConnectedEnvironment("examplerg", "myenvironment").withComponentType("state.azure.cosmosdb") + .withVersion("v1").withIgnoreErrors(false).withInitTimeout("50s") .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue"))) - .withMetadata( - Arrays - .asList( - new DaprMetadata().withName("url").withValue(""), - new DaprMetadata().withName("database").withValue("itemsDB"), - new DaprMetadata().withName("collection").withValue("items"), - new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder"))) + .withMetadata(Arrays.asList(new DaprMetadata().withName("url").withValue(""), + new DaprMetadata().withName("database").withValue("itemsDB"), + new DaprMetadata().withName("collection").withValue("items"), + new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder"))) .withScopes(Arrays.asList("container-app-1", "container-app-2")) + .withServiceComponentBind(Arrays.asList(new DaprComponentServiceBinding().withName("statestore") + .withServiceId( + "/subscriptions/9f7371f1-b593-4c3c-84e2-9167806ad358/resourceGroups/ca-syn2-group/providers/Microsoft.App/containerapps/cappredis") + .withMetadata(new DaprServiceBindMetadata().withName("daprcomponentBind").withValue("redis-bind")))) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteSamples.java index 08213cdf047d3..0d09c50557ede 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsDaprComponents Delete. */ +/** + * Samples for ConnectedEnvironmentsDaprComponents Delete. + */ public final class ConnectedEnvironmentsDaprComponentsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_Delete.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsDaprComponents_Delete.json */ /** * Sample code: Delete dapr component. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsDaprComponents() - .deleteWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsDaprComponents().deleteWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsGetSamples.java index c315b1ce87e1e..46f5e23495eed 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsDaprComponents Get. */ +/** + * Samples for ConnectedEnvironmentsDaprComponents Get. + */ public final class ConnectedEnvironmentsDaprComponentsGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_Get.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsDaprComponents_Get.json */ /** * Sample code: Get Dapr Component. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsDaprComponents() - .getWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsDaprComponents().getWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSamples.java index 48092b3d64ff2..ab17db519628f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsDaprComponents List. */ +/** + * Samples for ConnectedEnvironmentsDaprComponents List. + */ public final class ConnectedEnvironmentsDaprComponentsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_List.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsDaprComponents_List.json */ /** * Sample code: List Dapr Components. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listDaprComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsDaprComponents() - .list("examplerg", "myenvironment", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsDaprComponents().list("examplerg", "myenvironment", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsSamples.java index 905baafb61987..c28a23c8bcc3a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsDaprComponents ListSecrets. */ +/** + * Samples for ConnectedEnvironmentsDaprComponents ListSecrets. + */ public final class ConnectedEnvironmentsDaprComponentsListSecretsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsDaprComponents_ListSecrets.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsDaprComponents_ListSecrets.json */ /** * Sample code: List Container Apps Secrets. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsSecrets( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsDaprComponents() - .listSecretsWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + public static void + listContainerAppsSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.connectedEnvironmentsDaprComponents().listSecretsWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteSamples.java index a2ab7dbff31a1..8b303c72d1cb3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironments Delete. */ +/** + * Samples for ConnectedEnvironments Delete. + */ public final class ConnectedEnvironmentsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ConnectedEnvironments_Delete + * .json */ /** * Sample code: Delete connected environment by connectedEnvironmentName. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteConnectedEnvironmentByConnectedEnvironmentName( diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsGetByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsGetByResourceGroupSamples.java index 87c6e351dc178..4e2b5a153c388 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsGetByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsGetByResourceGroupSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironments GetByResourceGroup. */ +/** + * Samples for ConnectedEnvironments GetByResourceGroup. + */ public final class ConnectedEnvironmentsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ConnectedEnvironments_Get. + * json */ /** * Sample code: Get connected environment by connectedEnvironmentName. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getConnectedEnvironmentByConnectedEnvironmentName( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironments() - .getByResourceGroupWithResponse("examplerg", "examplekenv", com.azure.core.util.Context.NONE); + manager.connectedEnvironments().getByResourceGroupWithResponse("examplerg", "examplekenv", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListByResourceGroupSamples.java index 24e8daf8315be..46055391c2695 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListByResourceGroupSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironments ListByResourceGroup. */ +/** + * Samples for ConnectedEnvironments ListByResourceGroup. + */ public final class ConnectedEnvironmentsListByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_ListByResourceGroup.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironments_ListByResourceGroup.json */ /** * Sample code: List environments by resource group. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listEnvironmentsByResourceGroup( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listEnvironmentsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.connectedEnvironments().listByResourceGroup("examplerg", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListSamples.java index c902a717b52be..0249dc9cd9c1c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironments List. */ +/** + * Samples for ConnectedEnvironments List. + */ public final class ConnectedEnvironmentsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_ListBySubscription.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironments_ListBySubscription.json */ /** * Sample code: List connected environments by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listConnectedEnvironmentsBySubscription( diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesCreateOrUpdateSamples.java index 3139a930bc9f1..efdbcb1dd3b72 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesCreateOrUpdateSamples.java @@ -8,30 +8,26 @@ import com.azure.resourcemanager.appcontainers.models.AzureFileProperties; import com.azure.resourcemanager.appcontainers.models.ConnectedEnvironmentStorageProperties; -/** Samples for ConnectedEnvironmentsStorages CreateOrUpdate. */ +/** + * Samples for ConnectedEnvironmentsStorages CreateOrUpdate. + */ public final class ConnectedEnvironmentsStoragesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsStorages_CreateOrUpdate.json */ /** * Sample code: Create or update environments storage. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateEnvironmentsStorage( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsStorages() - .define("jlaw-demo1") + public static void + createOrUpdateEnvironmentsStorage(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.connectedEnvironmentsStorages().define("jlaw-demo1") .withExistingConnectedEnvironment("examplerg", "env") - .withProperties( - new ConnectedEnvironmentStorageProperties() - .withAzureFile( - new AzureFileProperties() - .withAccountName("account1") - .withAccountKey("fakeTokenPlaceholder") - .withAccessMode(AccessMode.READ_ONLY) - .withShareName("share1"))) + .withProperties(new ConnectedEnvironmentStorageProperties().withAzureFile( + new AzureFileProperties().withAccountName("account1").withAccountKey("fakeTokenPlaceholder") + .withAccessMode(AccessMode.READ_ONLY).withShareName("share1"))) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteSamples.java index b6501e6c33f91..1fa1aae1cd262 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsStorages Delete. */ +/** + * Samples for ConnectedEnvironmentsStorages Delete. + */ public final class ConnectedEnvironmentsStoragesDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_Delete.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsStorages_Delete.json */ /** * Sample code: List environments storages by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listEnvironmentsStoragesBySubscription( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsStorages() - .deleteWithResponse("examplerg", "env", "jlaw-demo1", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsStorages().deleteWithResponse("examplerg", "env", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesGetSamples.java index 890dbec721347..6766bd976d04a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesGetSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsStorages Get. */ +/** + * Samples for ConnectedEnvironmentsStorages Get. + */ public final class ConnectedEnvironmentsStoragesGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_Get.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsStorages_Get.json */ /** * Sample code: get a environments storage properties by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getAEnvironmentsStoragePropertiesBySubscription( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsStorages() - .getWithResponse("examplerg", "env", "jlaw-demo1", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsStorages().getWithResponse("examplerg", "env", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesListSamples.java index e0c78e04ddf88..ae861ce3634f8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesListSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironmentsStorages List. */ +/** + * Samples for ConnectedEnvironmentsStorages List. + */ public final class ConnectedEnvironmentsStoragesListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironmentsStorages_List.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ConnectedEnvironmentsStorages_List.json */ /** * Sample code: List environments storages by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listEnvironmentsStoragesBySubscription( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .connectedEnvironmentsStorages() - .listWithResponse("examplerg", "managedEnv", com.azure.core.util.Context.NONE); + manager.connectedEnvironmentsStorages().listWithResponse("examplerg", "managedEnv", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsUpdateSamples.java index da3c24d3deccb..c7b698b3a9f2e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsUpdateSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ConnectedEnvironments Update. */ +/** + * Samples for ConnectedEnvironments Update. + */ public final class ConnectedEnvironmentsUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ConnectedEnvironments_Patch.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ConnectedEnvironments_Patch. + * json */ /** * Sample code: Patch Managed Environment. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void patchManagedEnvironment( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + patchManagedEnvironment(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.connectedEnvironments().updateWithResponse("examplerg", "testenv", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsCreateOrUpdateSamples.java index 9241aee1c4365..ba742e7bcea30 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsCreateOrUpdateSamples.java @@ -6,38 +6,38 @@ import com.azure.resourcemanager.appcontainers.models.AppRegistration; import com.azure.resourcemanager.appcontainers.models.AuthPlatform; +import com.azure.resourcemanager.appcontainers.models.EncryptionSettings; import com.azure.resourcemanager.appcontainers.models.Facebook; import com.azure.resourcemanager.appcontainers.models.GlobalValidation; import com.azure.resourcemanager.appcontainers.models.IdentityProviders; import com.azure.resourcemanager.appcontainers.models.UnauthenticatedClientActionV2; -/** Samples for ContainerAppsAuthConfigs CreateOrUpdate. */ +/** + * Samples for ContainerAppsAuthConfigs CreateOrUpdate. + */ public final class ContainerAppsAuthConfigsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_CreateOrUpdate.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_CreateOrUpdate. + * json */ /** * Sample code: Create or Update Container App AuthConfig. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateContainerAppAuthConfig( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsAuthConfigs() - .define("current") + public static void + createOrUpdateContainerAppAuthConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsAuthConfigs().define("current") .withExistingContainerApp("workerapps-rg-xj", "testcanadacentral") .withPlatform(new AuthPlatform().withEnabled(true)) .withGlobalValidation( new GlobalValidation().withUnauthenticatedClientAction(UnauthenticatedClientActionV2.ALLOW_ANONYMOUS)) - .withIdentityProviders( - new IdentityProviders() - .withFacebook( - new Facebook() - .withRegistration( - new AppRegistration() - .withAppId("123") - .withAppSecretSettingName("fakeTokenPlaceholder")))) + .withIdentityProviders(new IdentityProviders().withFacebook(new Facebook().withRegistration( + new AppRegistration().withAppId("123").withAppSecretSettingName("fakeTokenPlaceholder")))) + .withEncryptionSettings( + new EncryptionSettings().withContainerAppAuthEncryptionSecretName("fakeTokenPlaceholder") + .withContainerAppAuthSigningSecretName("fakeTokenPlaceholder")) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteSamples.java index 8cff519593580..2c949bcda3652 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsAuthConfigs Delete. */ +/** + * Samples for ContainerAppsAuthConfigs Delete. + */ public final class ContainerAppsAuthConfigsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_Delete.json */ /** * Sample code: Delete Container App AuthConfig. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void deleteContainerAppAuthConfig( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsAuthConfigs() - .deleteWithResponse("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE); + public static void + deleteContainerAppAuthConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsAuthConfigs().deleteWithResponse("workerapps-rg-xj", "testcanadacentral", "current", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsGetSamples.java index c496745171404..4ef9610059c36 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsGetSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsAuthConfigs Get. */ +/** + * Samples for ContainerAppsAuthConfigs Get. + */ public final class ContainerAppsAuthConfigsGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_Get.json */ /** * Sample code: Get Container App's AuthConfig. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppSAuthConfig( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsAuthConfigs() - .getWithResponse("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE); + public static void + getContainerAppSAuthConfig(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsAuthConfigs().getWithResponse("workerapps-rg-xj", "testcanadacentral", "current", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsListByContainerAppSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsListByContainerAppSamples.java index 2f6c9020681d8..6735d790759c1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsListByContainerAppSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsListByContainerAppSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsAuthConfigs ListByContainerApp. */ +/** + * Samples for ContainerAppsAuthConfigs ListByContainerApp. + */ public final class ContainerAppsAuthConfigsListByContainerAppSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/AuthConfigs_ListByContainer.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/AuthConfigs_ListByContainer. + * json */ /** * Sample code: List Auth Configs by Container Apps. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listAuthConfigsByContainerApps( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsAuthConfigs() - .listByContainerApp("workerapps-rg-xj", "testcanadacentral", com.azure.core.util.Context.NONE); + public static void + listAuthConfigsByContainerApps(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsAuthConfigs().listByContainerApp("workerapps-rg-xj", "testcanadacentral", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsCreateOrUpdateSamples.java index 9e67a629cb81b..5472e0cc9b7e7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsCreateOrUpdateSamples.java @@ -19,8 +19,11 @@ import com.azure.resourcemanager.appcontainers.models.CustomDomain; import com.azure.resourcemanager.appcontainers.models.CustomScaleRule; import com.azure.resourcemanager.appcontainers.models.Dapr; +import com.azure.resourcemanager.appcontainers.models.ExtendedLocation; +import com.azure.resourcemanager.appcontainers.models.ExtendedLocationTypes; import com.azure.resourcemanager.appcontainers.models.Ingress; import com.azure.resourcemanager.appcontainers.models.IngressClientCertificateMode; +import com.azure.resourcemanager.appcontainers.models.IngressPortMapping; import com.azure.resourcemanager.appcontainers.models.IngressStickySessions; import com.azure.resourcemanager.appcontainers.models.IngressTransportMethod; import com.azure.resourcemanager.appcontainers.models.InitContainer; @@ -30,228 +33,219 @@ import com.azure.resourcemanager.appcontainers.models.ScaleRule; import com.azure.resourcemanager.appcontainers.models.Service; import com.azure.resourcemanager.appcontainers.models.ServiceBind; +import com.azure.resourcemanager.appcontainers.models.StorageType; import com.azure.resourcemanager.appcontainers.models.TcpScaleRule; import com.azure.resourcemanager.appcontainers.models.Template; import com.azure.resourcemanager.appcontainers.models.TrafficWeight; import com.azure.resourcemanager.appcontainers.models.Type; +import com.azure.resourcemanager.appcontainers.models.Volume; +import com.azure.resourcemanager.appcontainers.models.VolumeMount; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for ContainerApps CreateOrUpdate. */ +/** + * Samples for ContainerApps CreateOrUpdate. + */ public final class ContainerAppsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_CreateOrUpdate_ConnectedEnvironment.json */ /** - * Sample code: Create or Update Container App. - * + * Sample code: Create or Update App On A Connected Environment. + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateContainerApp( + public static void createOrUpdateAppOnAConnectedEnvironment( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerApps() - .define("testcontainerApp0") - .withRegion("East US") - .withExistingResourceGroup("rg") + manager.containerApps().define("testcontainerApp0").withRegion("East US").withExistingResourceGroup("rg") + .withExtendedLocation(new ExtendedLocation().withName( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.ExtendedLocation/customLocations/testcustomlocation") + .withType(ExtendedLocationTypes.CUSTOM_LOCATION)) + .withEnvironmentId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube") + .withConfiguration(new Configuration().withIngress(new Ingress().withExternal(true).withTargetPort(3000) + .withTraffic(Arrays.asList(new TrafficWeight().withRevisionName("testcontainerApp0-ab1234") + .withWeight(100).withLabel("production"))) + .withCustomDomains(Arrays.asList(new CustomDomain().withName("www.my-name.com") + .withBindingType(BindingType.SNI_ENABLED).withCertificateId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"), + new CustomDomain().withName("www.my-other-name.com").withBindingType(BindingType.SNI_ENABLED) + .withCertificateId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com"))) + .withIpSecurityRestrictions(Arrays.asList( + new IpSecurityRestrictionRule().withName("Allow work IP A subnet") + .withDescription("Allowing all IP's within the subnet below to access containerapp") + .withIpAddressRange("192.168.1.1/32").withAction(Action.ALLOW), + new IpSecurityRestrictionRule().withName("Allow work IP B subnet") + .withDescription("Allowing all IP's within the subnet below to access containerapp") + .withIpAddressRange("192.168.1.1/8").withAction(Action.ALLOW))) + .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY)) + .withClientCertificateMode(IngressClientCertificateMode.ACCEPT) + .withCorsPolicy(new CorsPolicy() + .withAllowedOrigins(Arrays.asList("https://a.test.com", "https://b.test.com")) + .withAllowedMethods(Arrays.asList("GET", "POST")) + .withAllowedHeaders(Arrays.asList("HEADER1", "HEADER2")) + .withExposeHeaders(Arrays.asList("HEADER3", "HEADER4")).withMaxAge(1234).withAllowCredentials(true)) + .withAdditionalPortMappings( + Arrays.asList(new IngressPortMapping().withExternal(true).withTargetPort(1234), + new IngressPortMapping().withExternal(false).withTargetPort(2345).withExposedPort(3456)))) + .withDapr( + new Dapr().withEnabled(true).withAppProtocol(AppProtocol.HTTP).withAppPort(3000) + .withHttpReadBufferSize(30).withHttpMaxRequestSize(10).withLogLevel( + LogLevel.DEBUG) + .withEnableApiLogging(true)) + .withMaxInactiveRevisions(10)) + .withTemplate(new Template() + .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerApp0:v4") + .withName("testinitcontainerApp0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers( + Arrays + .asList(new Container().withImage("repo/testcontainerApp0:v1").withName("testcontainerApp0") + .withProbes(Arrays.asList(new ContainerAppProbe() + .withHttpGet(new ContainerAppProbeHttpGet() + .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem() + .withName("Custom-Header").withValue("Awesome"))) + .withPath("/health").withPort(8080)) + .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS))))) + .withScale(new Scale().withMinReplicas(1).withMaxReplicas(5) + .withRules(Arrays.asList(new ScaleRule().withName("httpscalingrule").withCustom( + new CustomScaleRule().withType("http").withMetadata(mapOf("concurrentRequests", "50"))))))) + .create(); + } + + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_CreateOrUpdate + * .json + */ + /** + * Sample code: Create or Update Container App. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + createOrUpdateContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerApps().define("testcontainerApp0").withRegion("East US").withExistingResourceGroup("rg") .withEnvironmentId( "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube") - .withWorkloadProfileName("My-GP-01") - .withConfiguration( - new Configuration() - .withIngress( - new Ingress() - .withExternal(true) - .withTargetPort(3000) - .withTraffic( - Arrays - .asList( - new TrafficWeight() - .withRevisionName("testcontainerApp0-ab1234") - .withWeight(100) - .withLabel("production"))) - .withCustomDomains( - Arrays - .asList( - new CustomDomain() - .withName("www.my-name.com") - .withBindingType(BindingType.SNI_ENABLED) - .withCertificateId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"), - new CustomDomain() - .withName("www.my-other-name.com") - .withBindingType(BindingType.SNI_ENABLED) - .withCertificateId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com"))) - .withIpSecurityRestrictions( - Arrays - .asList( - new IpSecurityRestrictionRule() - .withName("Allow work IP A subnet") - .withDescription( - "Allowing all IP's within the subnet below to access containerapp") - .withIpAddressRange("192.168.1.1/32") - .withAction(Action.ALLOW), - new IpSecurityRestrictionRule() - .withName("Allow work IP B subnet") - .withDescription( - "Allowing all IP's within the subnet below to access containerapp") - .withIpAddressRange("192.168.1.1/8") - .withAction(Action.ALLOW))) - .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY)) - .withClientCertificateMode(IngressClientCertificateMode.ACCEPT) - .withCorsPolicy( - new CorsPolicy() - .withAllowedOrigins(Arrays.asList("https://a.test.com", "https://b.test.com")) - .withAllowedMethods(Arrays.asList("GET", "POST")) - .withAllowedHeaders(Arrays.asList("HEADER1", "HEADER2")) - .withExposeHeaders(Arrays.asList("HEADER3", "HEADER4")) - .withMaxAge(1234) - .withAllowCredentials(true))) - .withDapr( - new Dapr() - .withEnabled(true) - .withAppProtocol(AppProtocol.HTTP) - .withAppPort(3000) - .withHttpReadBufferSize(30) - .withHttpMaxRequestSize(10) - .withLogLevel(LogLevel.DEBUG) - .withEnableApiLogging(true)) - .withMaxInactiveRevisions(10) - .withService(new Service().withType("redis"))) - .withTemplate( - new Template() - .withInitContainers( - Arrays - .asList( - new InitContainer() - .withImage("repo/testcontainerApp0:v4") - .withName("testinitcontainerApp0") - .withCommand(Arrays.asList("/bin/sh")) - .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerApp0:v1") - .withName("testcontainerApp0") - .withProbes( - Arrays - .asList( - new ContainerAppProbe() - .withHttpGet( - new ContainerAppProbeHttpGet() - .withHttpHeaders( - Arrays - .asList( - new ContainerAppProbeHttpGetHttpHeadersItem() - .withName("Custom-Header") - .withValue("Awesome"))) - .withPath("/health") - .withPort(8080)) - .withInitialDelaySeconds(3) - .withPeriodSeconds(3) - .withType(Type.LIVENESS))))) - .withScale( - new Scale() - .withMinReplicas(1) - .withMaxReplicas(5) - .withRules( - Arrays - .asList( - new ScaleRule() - .withName("httpscalingrule") - .withCustom( - new CustomScaleRule() - .withType("http") - .withMetadata(mapOf("concurrentRequests", "50")))))) - .withServiceBinds( - Arrays - .asList( - new ServiceBind() - .withServiceId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/redisService") - .withName("redisService")))) + .withWorkloadProfileName( + "My-GP-01") + .withConfiguration(new Configuration().withIngress(new Ingress().withExternal(true).withTargetPort(3000) + .withTraffic(Arrays.asList(new TrafficWeight().withRevisionName("testcontainerApp0-ab1234") + .withWeight(100).withLabel("production"))) + .withCustomDomains(Arrays.asList(new CustomDomain().withName("www.my-name.com") + .withBindingType(BindingType.SNI_ENABLED).withCertificateId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"), + new CustomDomain().withName("www.my-other-name.com").withBindingType(BindingType.SNI_ENABLED) + .withCertificateId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com"))) + .withIpSecurityRestrictions(Arrays.asList( + new IpSecurityRestrictionRule().withName("Allow work IP A subnet") + .withDescription("Allowing all IP's within the subnet below to access containerapp") + .withIpAddressRange("192.168.1.1/32").withAction(Action.ALLOW), + new IpSecurityRestrictionRule().withName("Allow work IP B subnet") + .withDescription("Allowing all IP's within the subnet below to access containerapp") + .withIpAddressRange("192.168.1.1/8").withAction(Action.ALLOW))) + .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY)) + .withClientCertificateMode(IngressClientCertificateMode.ACCEPT) + .withCorsPolicy(new CorsPolicy() + .withAllowedOrigins(Arrays.asList("https://a.test.com", "https://b.test.com")) + .withAllowedMethods(Arrays.asList("GET", "POST")) + .withAllowedHeaders(Arrays.asList("HEADER1", "HEADER2")) + .withExposeHeaders(Arrays.asList("HEADER3", "HEADER4")).withMaxAge(1234).withAllowCredentials(true)) + .withAdditionalPortMappings( + Arrays.asList(new IngressPortMapping().withExternal(true).withTargetPort(1234), + new IngressPortMapping().withExternal(false).withTargetPort(2345).withExposedPort(3456)))) + .withDapr(new Dapr().withEnabled(true).withAppProtocol(AppProtocol.HTTP).withAppPort(3000) + .withHttpReadBufferSize(30).withHttpMaxRequestSize(10).withLogLevel(LogLevel.DEBUG) + .withEnableApiLogging(true)) + .withMaxInactiveRevisions(10).withService(new Service().withType("redis"))) + .withTemplate(new Template() + .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerApp0:v4") + .withName("testinitcontainerApp0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers( + Arrays.asList(new Container().withImage("repo/testcontainerApp0:v1").withName("testcontainerApp0") + .withVolumeMounts(Arrays.asList( + new VolumeMount().withVolumeName("azurefile").withMountPath("/mnt/path1") + .withSubPath("subPath1"), + new VolumeMount().withVolumeName("nfsazurefile").withMountPath("/mnt/path2") + .withSubPath("subPath2"))) + .withProbes(Arrays.asList(new ContainerAppProbe() + .withHttpGet(new ContainerAppProbeHttpGet() + .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem() + .withName("Custom-Header").withValue("Awesome"))) + .withPath("/health").withPort(8080)) + .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS))))) + .withScale(new Scale().withMinReplicas(1).withMaxReplicas(5) + .withRules(Arrays.asList(new ScaleRule().withName("httpscalingrule").withCustom( + new CustomScaleRule().withType("http").withMetadata(mapOf("concurrentRequests", "50")))))) + .withVolumes(Arrays.asList( + new Volume().withName("azurefile").withStorageType(StorageType.AZURE_FILE) + .withStorageName("storage"), + new Volume().withName("nfsazurefile").withStorageType(StorageType.NFS_AZURE_FILE) + .withStorageName("nfsStorage"))) + .withServiceBinds(Arrays.asList(new ServiceBind().withServiceId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/redisService") + .withName("redisService").withClientType("dotnet") + .withCustomizedKeys(mapOf("DesiredKey", "fakeTokenPlaceholder"))))) .create(); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_TcpApp_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_TcpApp_CreateOrUpdate.json */ /** * Sample code: Create or Update Tcp App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createOrUpdateTcpApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerApps() - .define("testcontainerAppTcp") - .withRegion("East US") - .withExistingResourceGroup("rg") + manager.containerApps().define("testcontainerAppTcp").withRegion("East US").withExistingResourceGroup("rg") .withEnvironmentId( "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube") .withConfiguration( new Configuration() .withIngress( - new Ingress() - .withExternal(true) - .withTargetPort(3000) - .withExposedPort(4000) - .withTransport(IngressTransportMethod.TCP) + new Ingress().withExternal(true).withTargetPort(3000).withExposedPort(4000) + .withTransport( + IngressTransportMethod.TCP) .withTraffic( Arrays .asList( - new TrafficWeight() - .withRevisionName("testcontainerAppTcp-ab1234") - .withWeight(100))))) - .withTemplate( - new Template() - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerAppTcp:v1") - .withName("testcontainerAppTcp") - .withProbes( - Arrays - .asList( - new ContainerAppProbe() - .withInitialDelaySeconds(3) - .withPeriodSeconds(3) - .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080)) - .withType(Type.LIVENESS))))) - .withScale( - new Scale() - .withMinReplicas(1) - .withMaxReplicas(5) - .withRules( - Arrays - .asList( - new ScaleRule() - .withName("tcpscalingrule") - .withTcp( - new TcpScaleRule() - .withMetadata(mapOf("concurrentConnections", "50"))))))) + new TrafficWeight().withRevisionName( + "testcontainerAppTcp-ab1234").withWeight( + 100))))) + .withTemplate(new Template() + .withContainers(Arrays.asList(new Container().withImage("repo/testcontainerAppTcp:v1") + .withName("testcontainerAppTcp") + .withProbes(Arrays.asList(new ContainerAppProbe().withInitialDelaySeconds(3).withPeriodSeconds(3) + .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080)).withType(Type.LIVENESS))))) + .withScale( + new Scale().withMinReplicas(1).withMaxReplicas(5) + .withRules(Arrays.asList(new ScaleRule().withName("tcpscalingrule") + .withTcp(new TcpScaleRule().withMetadata(mapOf("concurrentConnections", "50"))))))) .create(); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ManagedBy_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_ManagedBy_CreateOrUpdate.json */ /** * Sample code: Create or Update ManagedBy App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateManagedByApp( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerApps() - .define("testcontainerAppManagedBy") - .withRegion("East US") + public static void + createOrUpdateManagedByApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerApps().define("testcontainerAppManagedBy").withRegion("East US") .withExistingResourceGroup("rg") .withManagedBy( "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.AppPlatform/Spring/springapp") @@ -260,45 +254,21 @@ public static void createOrUpdateManagedByApp( .withConfiguration( new Configuration() .withIngress( - new Ingress() - .withExternal(true) - .withTargetPort(3000) - .withExposedPort(4000) - .withTransport(IngressTransportMethod.TCP) - .withTraffic( + new Ingress().withExternal(true).withTargetPort(3000).withExposedPort(4000) + .withTransport(IngressTransportMethod.TCP).withTraffic( Arrays .asList( - new TrafficWeight() - .withRevisionName("testcontainerAppManagedBy-ab1234") + new TrafficWeight().withRevisionName("testcontainerAppManagedBy-ab1234") .withWeight(100))))) - .withTemplate( - new Template() - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerAppManagedBy:v1") - .withName("testcontainerAppManagedBy") - .withProbes( - Arrays - .asList( - new ContainerAppProbe() - .withInitialDelaySeconds(3) - .withPeriodSeconds(3) - .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080)) - .withType(Type.LIVENESS))))) - .withScale( - new Scale() - .withMinReplicas(1) - .withMaxReplicas(5) - .withRules( - Arrays - .asList( - new ScaleRule() - .withName("tcpscalingrule") - .withTcp( - new TcpScaleRule() - .withMetadata(mapOf("concurrentConnections", "50"))))))) + .withTemplate(new Template() + .withContainers(Arrays.asList(new Container().withImage("repo/testcontainerAppManagedBy:v1") + .withName("testcontainerAppManagedBy") + .withProbes(Arrays.asList(new ContainerAppProbe().withInitialDelaySeconds(3).withPeriodSeconds(3) + .withTcpSocket(new ContainerAppProbeTcpSocket().withPort(8080)).withType(Type.LIVENESS))))) + .withScale( + new Scale().withMinReplicas(1).withMaxReplicas(5) + .withRules(Arrays.asList(new ScaleRule().withName("tcpscalingrule") + .withTcp(new TcpScaleRule().withMetadata(mapOf("concurrentConnections", "50"))))))) .create(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteSamples.java index d7b8bc4e1f7ac..47048d97bd883 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps Delete. */ +/** + * Samples for ContainerApps Delete. + */ public final class ContainerAppsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Delete.json */ /** * Sample code: Delete Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorSamples.java index 992e7aa9bb034..a0d8c8dc108fc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorSamples.java @@ -4,24 +4,23 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsDiagnostics GetDetector. */ +/** + * Samples for ContainerAppsDiagnostics GetDetector. + */ public final class ContainerAppsDiagnosticsGetDetectorSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerAppsDiagnostics_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerAppsDiagnostics_Get + * .json */ /** * Sample code: Get Container App's diagnostics info. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppSDiagnosticsInfo( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsDiagnostics() - .getDetectorWithResponse( - "mikono-workerapp-test-rg", - "mikono-capp-stage1", - "cappcontainerappnetworkIO", - com.azure.core.util.Context.NONE); + public static void + getContainerAppSDiagnosticsInfo(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsDiagnostics().getDetectorWithResponse("mikono-workerapp-test-rg", "mikono-capp-stage1", + "cappcontainerappnetworkIO", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRevisionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRevisionSamples.java index bd0c74d90e31c..aa1614e9b831e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRevisionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRevisionSamples.java @@ -4,21 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsDiagnostics GetRevision. */ +/** + * Samples for ContainerAppsDiagnostics GetRevision. + */ public final class ContainerAppsDiagnosticsGetRevisionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Get.json */ /** * Sample code: Get Container App's revision. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppSRevision( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsDiagnostics() - .getRevisionWithResponse( - "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); + public static void + getContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsDiagnostics().getRevisionWithResponse("rg", "testcontainerApp0", + "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRootSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRootSamples.java index 937fbe5d46b82..a6264f1afb5f5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRootSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetRootSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsDiagnostics GetRoot. */ +/** + * Samples for ContainerAppsDiagnostics GetRoot. + */ public final class ContainerAppsDiagnosticsGetRootSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Get.json */ /** * Sample code: Get Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsDiagnostics() - .getRootWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE); + manager.containerAppsDiagnostics().getRootWithResponse("rg", "testcontainerApp0", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsSamples.java index 390c12f486780..6bda0b6f50dee 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsDiagnostics ListDetectors. */ +/** + * Samples for ContainerAppsDiagnostics ListDetectors. + */ public final class ContainerAppsDiagnosticsListDetectorsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerAppsDiagnostics_List.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerAppsDiagnostics_List.json */ /** * Sample code: Get the list of available diagnostics for a given Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getTheListOfAvailableDiagnosticsForAGivenContainerApp( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsDiagnostics() - .listDetectors("mikono-workerapp-test-rg", "mikono-capp-stage1", com.azure.core.util.Context.NONE); + manager.containerAppsDiagnostics().listDetectors("mikono-workerapp-test-rg", "mikono-capp-stage1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListRevisionsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListRevisionsSamples.java index 20d47c4b4c604..3a04c7903251b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListRevisionsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListRevisionsSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsDiagnostics ListRevisions. */ +/** + * Samples for ContainerAppsDiagnostics ListRevisions. + */ public final class ContainerAppsDiagnosticsListRevisionsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_List.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_List.json */ /** * Sample code: List Container App's revisions. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppSRevisions( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsDiagnostics() - .listRevisions("rg", "testcontainerApp0", null, com.azure.core.util.Context.NONE); + public static void + listContainerAppSRevisions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsDiagnostics().listRevisions("rg", "testcontainerApp0", null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetAuthTokenSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetAuthTokenSamples.java index 8b1ba6a70851e..7c041c9d923c2 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetAuthTokenSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetAuthTokenSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps GetAuthToken. */ +/** + * Samples for ContainerApps GetAuthToken. + */ public final class ContainerAppsGetAuthTokenSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_GetAuthToken.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_GetAuthToken. + * json */ /** * Sample code: Get Container App Auth Token. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppAuthToken( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + getContainerAppAuthToken(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.containerApps().getAuthTokenWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetByResourceGroupSamples.java index 9edea0317e640..ff0cf63b25b67 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsGetByResourceGroupSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps GetByResourceGroup. */ +/** + * Samples for ContainerApps GetByResourceGroup. + */ public final class ContainerAppsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Get.json */ /** * Sample code: Get Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerApps() - .getByResourceGroupWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE); + manager.containerApps().getByResourceGroupWithResponse("rg", "testcontainerApp0", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListByResourceGroupSamples.java index 7bcf0aa375b08..44027ab39d2b8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListByResourceGroupSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps ListByResourceGroup. */ +/** + * Samples for ContainerApps ListByResourceGroup. + */ public final class ContainerAppsListByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListByResourceGroup.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_ListByResourceGroup.json */ /** * Sample code: List Container Apps by resource group. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsByResourceGroup( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listContainerAppsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.containerApps().listByResourceGroup("rg", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListCustomHostnameAnalysisSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListCustomHostnameAnalysisSamples.java index 52c21100e52a0..174e04e654e21 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListCustomHostnameAnalysisSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListCustomHostnameAnalysisSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps ListCustomHostnameAnalysis. */ +/** + * Samples for ContainerApps ListCustomHostnameAnalysis. + */ public final class ContainerAppsListCustomHostnameAnalysisSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListCustomHostNameAnalysis.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_ListCustomHostNameAnalysis.json */ /** * Sample code: Analyze Custom Hostname. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void analyzeCustomHostname(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerApps() - .listCustomHostnameAnalysisWithResponse( - "rg", "testcontainerApp0", "my.name.corp", com.azure.core.util.Context.NONE); + manager.containerApps().listCustomHostnameAnalysisWithResponse("rg", "testcontainerApp0", "my.name.corp", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSamples.java index 18ab90f2c3182..8610a6fd759e4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps List. */ +/** + * Samples for ContainerApps List. + */ public final class ContainerAppsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListBySubscription.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_ListBySubscription.json */ /** * Sample code: List Container Apps by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsBySubscription( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listContainerAppsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.containerApps().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSecretsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSecretsSamples.java index aeb01c17975dc..ef9b743107ffa 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSecretsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsListSecretsSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps ListSecrets. */ +/** + * Samples for ContainerApps ListSecrets. + */ public final class ContainerAppsListSecretsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_ListSecrets.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_ListSecrets. + * json */ /** * Sample code: List Container Apps Secrets. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsSecrets( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listContainerAppsSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.containerApps().listSecretsWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaSamples.java index 40ee7800d55b7..4a723e1189c1f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaSamples.java @@ -4,25 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisionReplicas GetReplica. */ +/** + * Samples for ContainerAppsRevisionReplicas GetReplica. + */ public final class ContainerAppsRevisionReplicasGetReplicaSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Replicas_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Replicas_Get.json */ /** * Sample code: Get Container App's revision replica. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppSRevisionReplica( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisionReplicas() - .getReplicaWithResponse( - "workerapps-rg-xj", - "myapp", - "myapp--0wlqy09", - "myapp--0wlqy09-5d9774cff-5wnd8", - com.azure.core.util.Context.NONE); + public static void + getContainerAppSRevisionReplica(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisionReplicas().getReplicaWithResponse("workerapps-rg-xj", "myapp", "myapp--0wlqy09", + "myapp--0wlqy09-5d9774cff-5wnd8", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasSamples.java index 7e7f5f482af56..05dc2154609e0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisionReplicas ListReplicas. */ +/** + * Samples for ContainerAppsRevisionReplicas ListReplicas. + */ public final class ContainerAppsRevisionReplicasListReplicasSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Replicas_List.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Replicas_List.json */ /** * Sample code: List Container App's replicas. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppSReplicas( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisionReplicas() - .listReplicasWithResponse("workerapps-rg-xj", "myapp", "myapp--0wlqy09", com.azure.core.util.Context.NONE); + public static void + listContainerAppSReplicas(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisionReplicas().listReplicasWithResponse("workerapps-rg-xj", "myapp", "myapp--0wlqy09", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionSamples.java index 39ec41f8b7678..718323d35fcb8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionSamples.java @@ -4,21 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisions ActivateRevision. */ +/** + * Samples for ContainerAppsRevisions ActivateRevision. + */ public final class ContainerAppsRevisionsActivateRevisionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Activate.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Activate.json */ /** * Sample code: Activate Container App's revision. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void activateContainerAppSRevision( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisions() - .activateRevisionWithResponse( - "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); + public static void + activateContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisions().activateRevisionWithResponse("rg", "testcontainerApp0", + "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionSamples.java index 5cb88570661f3..02575b8d6abfa 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionSamples.java @@ -4,21 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisions DeactivateRevision. */ +/** + * Samples for ContainerAppsRevisions DeactivateRevision. + */ public final class ContainerAppsRevisionsDeactivateRevisionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Deactivate.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Deactivate.json */ /** * Sample code: Deactivate Container App's revision. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void deactivateContainerAppSRevision( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisions() - .deactivateRevisionWithResponse( - "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); + public static void + deactivateContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisions().deactivateRevisionWithResponse("rg", "testcontainerApp0", + "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsGetRevisionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsGetRevisionSamples.java index 97956a6143760..9d602a61b191a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsGetRevisionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsGetRevisionSamples.java @@ -4,21 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisions GetRevision. */ +/** + * Samples for ContainerAppsRevisions GetRevision. + */ public final class ContainerAppsRevisionsGetRevisionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Get.json */ /** * Sample code: Get Container App's revision. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppSRevision( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisions() - .getRevisionWithResponse( - "rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); + public static void + getContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisions().getRevisionWithResponse("rg", "testcontainerApp0", "testcontainerApp0-pjxhsye", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsListRevisionsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsListRevisionsSamples.java index 985067a2c45da..310c3fb98a894 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsListRevisionsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsListRevisionsSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisions ListRevisions. */ +/** + * Samples for ContainerAppsRevisions ListRevisions. + */ public final class ContainerAppsRevisionsListRevisionsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_List.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_List.json */ /** * Sample code: List Container App's revisions. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppSRevisions( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisions() - .listRevisions("rg", "testcontainerApp0", null, com.azure.core.util.Context.NONE); + public static void + listContainerAppSRevisions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisions().listRevisions("rg", "testcontainerApp0", null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionSamples.java index bce649a169019..548cd13d8829d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionSamples.java @@ -4,21 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsRevisions RestartRevision. */ +/** + * Samples for ContainerAppsRevisions RestartRevision. + */ public final class ContainerAppsRevisionsRestartRevisionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Revisions_Restart.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Revisions_Restart.json */ /** * Sample code: Restart Container App's revision. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void restartContainerAppSRevision( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsRevisions() - .restartRevisionWithResponse( - "rg", "testStaticSite0", "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); + public static void + restartContainerAppSRevision(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsRevisions().restartRevisionWithResponse("rg", "testStaticSite0", + "testcontainerApp0-pjxhsye", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsCreateOrUpdateSamples.java index 8c03624e124b4..a95df9fdde095 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsCreateOrUpdateSamples.java @@ -5,43 +5,38 @@ package com.azure.resourcemanager.appcontainers.generated; import com.azure.resourcemanager.appcontainers.models.AzureCredentials; +import com.azure.resourcemanager.appcontainers.models.EnvironmentVariable; import com.azure.resourcemanager.appcontainers.models.GithubActionConfiguration; import com.azure.resourcemanager.appcontainers.models.RegistryInfo; +import java.util.Arrays; -/** Samples for ContainerAppsSourceControls CreateOrUpdate. */ +/** + * Samples for ContainerAppsSourceControls CreateOrUpdate. + */ public final class ContainerAppsSourceControlsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * SourceControls_CreateOrUpdate.json */ /** * Sample code: Create or Update Container App SourceControl. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createOrUpdateContainerAppSourceControl( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsSourceControls() - .define("current") + manager.containerAppsSourceControls().define("current") .withExistingContainerApp("workerapps-rg-xj", "testcanadacentral") - .withRepoUrl("https://github.com/xwang971/ghatest") - .withBranch("master") - .withGithubActionConfiguration( - new GithubActionConfiguration() - .withRegistryInfo( - new RegistryInfo() - .withRegistryUrl("test-registry.azurecr.io") - .withRegistryUsername("test-registry") - .withRegistryPassword("fakeTokenPlaceholder")) - .withAzureCredentials( - new AzureCredentials() - .withClientId("") - .withClientSecret("fakeTokenPlaceholder") - .withTenantId("") - .withKind("feaderated")) - .withContextPath("./") - .withGithubPersonalAccessToken("fakeTokenPlaceholder") - .withImage("image/tag")) + .withRepoUrl("https://github.com/xwang971/ghatest").withBranch("master") + .withGithubActionConfiguration(new GithubActionConfiguration() + .withRegistryInfo(new RegistryInfo().withRegistryUrl("test-registry.azurecr.io") + .withRegistryUsername("test-registry").withRegistryPassword("fakeTokenPlaceholder")) + .withAzureCredentials(new AzureCredentials().withClientId("") + .withClientSecret("fakeTokenPlaceholder").withTenantId("").withKind("feaderated")) + .withContextPath("./").withGithubPersonalAccessToken("fakeTokenPlaceholder").withImage("image/tag") + .withBuildEnvironmentVariables( + Arrays.asList(new EnvironmentVariable().withName("foo1").withValue("bar1"), + new EnvironmentVariable().withName("foo2").withValue("bar2")))) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteSamples.java index 54d5428861c18..58c3fd76c7fef 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsSourceControls Delete. */ +/** + * Samples for ContainerAppsSourceControls Delete. + */ public final class ContainerAppsSourceControlsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/SourceControls_Delete.json */ /** * Sample code: Delete Container App SourceControl. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void deleteContainerAppSourceControl( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsSourceControls() - .delete("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE); + public static void + deleteContainerAppSourceControl(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsSourceControls().delete("workerapps-rg-xj", "testcanadacentral", "current", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsGetSamples.java index f4e6fa68ec2c4..4a72f7738a85c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsGetSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsSourceControls Get. */ +/** + * Samples for ContainerAppsSourceControls Get. + */ public final class ContainerAppsSourceControlsGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/SourceControls_Get.json */ /** * Sample code: Get Container App's SourceControl. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getContainerAppSSourceControl( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsSourceControls() - .getWithResponse("workerapps-rg-xj", "testcanadacentral", "current", com.azure.core.util.Context.NONE); + public static void + getContainerAppSSourceControl(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.containerAppsSourceControls().getWithResponse("workerapps-rg-xj", "testcanadacentral", "current", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsListByContainerAppSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsListByContainerAppSamples.java index 883a72b2f8503..568c72b952ba0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsListByContainerAppSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsListByContainerAppSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerAppsSourceControls ListByContainerApp. */ +/** + * Samples for ContainerAppsSourceControls ListByContainerApp. + */ public final class ContainerAppsSourceControlsListByContainerAppSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/SourceControls_ListByContainer.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * SourceControls_ListByContainer.json */ /** * Sample code: List App's Source Controls. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listAppSSourceControls(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .containerAppsSourceControls() - .listByContainerApp("workerapps-rg-xj", "testcanadacentral", com.azure.core.util.Context.NONE); + manager.containerAppsSourceControls().listByContainerApp("workerapps-rg-xj", "testcanadacentral", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStartSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStartSamples.java index b1f49ae8f313f..fc183d69d6649 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStartSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStartSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps Start. */ +/** + * Samples for ContainerApps Start. + */ public final class ContainerAppsStartSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Start.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Start.json */ /** * Sample code: Start Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void startContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStopSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStopSamples.java index ae4ab7a00a884..fcde0beb24fbd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStopSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsStopSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ContainerApps Stop. */ +/** + * Samples for ContainerApps Stop. + */ public final class ContainerAppsStopSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Stop.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Stop.json */ /** * Sample code: Stop Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void stopContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsUpdateSamples.java index dcdce3b8d1186..0adee1573e3a7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsUpdateSamples.java @@ -34,130 +34,65 @@ import java.util.HashMap; import java.util.Map; -/** Samples for ContainerApps Update. */ +/** + * Samples for ContainerApps Update. + */ public final class ContainerAppsUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_Patch.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ContainerApps_Patch.json */ /** * Sample code: Patch Container App. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void patchContainerApp(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - ContainerApp resource = - manager - .containerApps() - .getByResourceGroupWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE) - .getValue(); + ContainerApp resource = manager.containerApps() + .getByResourceGroupWithResponse("rg", "testcontainerApp0", com.azure.core.util.Context.NONE).getValue(); resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withConfiguration( - new Configuration() - .withIngress( - new Ingress() - .withExternal(true) - .withTargetPort(3000) - .withTraffic( - Arrays - .asList( - new TrafficWeight() - .withRevisionName("testcontainerApp0-ab1234") - .withWeight(100) - .withLabel("production"))) - .withCustomDomains( - Arrays - .asList( - new CustomDomain() - .withName("www.my-name.com") - .withBindingType(BindingType.SNI_ENABLED) - .withCertificateId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"), - new CustomDomain() - .withName("www.my-other-name.com") - .withBindingType(BindingType.SNI_ENABLED) - .withCertificateId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com"))) - .withIpSecurityRestrictions( - Arrays - .asList( - new IpSecurityRestrictionRule() - .withName("Allow work IP A subnet") - .withDescription( - "Allowing all IP's within the subnet below to access containerapp") - .withIpAddressRange("192.168.1.1/32") - .withAction(Action.ALLOW), - new IpSecurityRestrictionRule() - .withName("Allow work IP B subnet") - .withDescription( - "Allowing all IP's within the subnet below to access containerapp") - .withIpAddressRange("192.168.1.1/8") - .withAction(Action.ALLOW))) - .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY))) - .withDapr( - new Dapr() - .withEnabled(true) - .withAppProtocol(AppProtocol.HTTP) - .withAppPort(3000) - .withHttpReadBufferSize(30) - .withHttpMaxRequestSize(10) - .withLogLevel(LogLevel.DEBUG) - .withEnableApiLogging(true)) - .withMaxInactiveRevisions(10) - .withService(new Service().withType("redis"))) + .update().withTags( + mapOf("tag1", "value1", "tag2", "value2")) + .withConfiguration(new Configuration().withIngress(new Ingress().withExternal(true).withTargetPort(3000) + .withTraffic(Arrays.asList(new TrafficWeight().withRevisionName("testcontainerApp0-ab1234") + .withWeight(100).withLabel("production"))) + .withCustomDomains(Arrays.asList(new CustomDomain().withName("www.my-name.com") + .withBindingType(BindingType.SNI_ENABLED).withCertificateId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-name-dot-com"), + new CustomDomain().withName("www.my-other-name.com").withBindingType(BindingType.SNI_ENABLED) + .withCertificateId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube/certificates/my-certificate-for-my-other-name-dot-com"))) + .withIpSecurityRestrictions(Arrays.asList( + new IpSecurityRestrictionRule().withName("Allow work IP A subnet") + .withDescription("Allowing all IP's within the subnet below to access containerapp") + .withIpAddressRange("192.168.1.1/32").withAction(Action.ALLOW), + new IpSecurityRestrictionRule().withName("Allow work IP B subnet") + .withDescription("Allowing all IP's within the subnet below to access containerapp") + .withIpAddressRange("192.168.1.1/8").withAction(Action.ALLOW))) + .withStickySessions(new IngressStickySessions().withAffinity(Affinity.STICKY))) + .withDapr(new Dapr().withEnabled(true).withAppProtocol(AppProtocol.HTTP).withAppPort(3000) + .withHttpReadBufferSize(30).withHttpMaxRequestSize(10).withLogLevel(LogLevel.DEBUG) + .withEnableApiLogging(true)) + .withMaxInactiveRevisions(10).withService(new Service().withType("redis"))) .withTemplate( new Template() - .withInitContainers( - Arrays - .asList( - new InitContainer() - .withImage("repo/testcontainerApp0:v4") - .withName("testinitcontainerApp0") - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerApp0:v1") - .withName("testcontainerApp0") - .withProbes( - Arrays - .asList( - new ContainerAppProbe() - .withHttpGet( - new ContainerAppProbeHttpGet() - .withHttpHeaders( - Arrays - .asList( - new ContainerAppProbeHttpGetHttpHeadersItem() - .withName("Custom-Header") - .withValue("Awesome"))) - .withPath("/health") - .withPort(8080)) - .withInitialDelaySeconds(3) - .withPeriodSeconds(3) - .withType(Type.LIVENESS))))) - .withScale( - new Scale() - .withMinReplicas(1) - .withMaxReplicas(5) - .withRules( - Arrays - .asList( - new ScaleRule() - .withName("httpscalingrule") - .withCustom( - new CustomScaleRule() - .withType("http") - .withMetadata(mapOf("concurrentRequests", "50")))))) - .withServiceBinds( - Arrays - .asList( - new ServiceBind() - .withServiceId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/service") - .withName("service")))) + .withInitContainers(Arrays.asList( + new InitContainer().withImage("repo/testcontainerApp0:v4").withName("testinitcontainerApp0") + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers(Arrays.asList(new Container().withImage("repo/testcontainerApp0:v1") + .withName("testcontainerApp0").withProbes(Arrays.asList(new ContainerAppProbe() + .withHttpGet(new ContainerAppProbeHttpGet() + .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem() + .withName("Custom-Header").withValue("Awesome"))) + .withPath("/health").withPort(8080)) + .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS))))) + .withScale(new Scale().withMinReplicas(1).withMaxReplicas(5) + .withRules(Arrays.asList(new ScaleRule().withName("httpscalingrule").withCustom( + new CustomScaleRule().withType("http").withMetadata(mapOf("concurrentRequests", "50")))))) + .withServiceBinds(Arrays.asList(new ServiceBind().withServiceId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/containerApps/service") + .withName("service").withClientType("dotnet") + .withCustomizedKeys(mapOf("DesiredKey", "fakeTokenPlaceholder"))))) .apply(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..b9fee2733760c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateSamples.java @@ -0,0 +1,96 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; + +/** + * Samples for DaprComponentResiliencyPolicies CreateOrUpdate. + */ +public final class DaprComponentResiliencyPoliciesCreateOrUpdateSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponentResiliencyPolicy_CreateOrUpdate_OutboundOnly.json + */ + /** + * Sample code: Create or update dapr component resiliency policy with outbound policy only. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateDaprComponentResiliencyPolicyWithOutboundPolicyOnly( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponentResiliencyPolicies().define("myresiliencypolicy") + .withExistingDaprComponent("examplerg", "myenvironment", "mydaprcomponent") + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(5) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(100).withMaxIntervalInMilliseconds(30000))) + .withTimeoutPolicy( + new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(12)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(3).withTimeoutInSeconds(20).withIntervalInSeconds(60))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponentResiliencyPolicy_CreateOrUpdate_AllOptions.json + */ + /** + * Sample code: Create or update dapr component resiliency policy with all options. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateDaprComponentResiliencyPolicyWithAllOptions( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponentResiliencyPolicies().define("myresiliencypolicy") + .withExistingDaprComponent("examplerg", "myenvironment", "mydaprcomponent") + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(15) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(2000).withMaxIntervalInMilliseconds(5500))) + .withTimeoutPolicy( + new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(30)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(5).withTimeoutInSeconds(10).withIntervalInSeconds(4))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(5) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(100).withMaxIntervalInMilliseconds(30000))) + .withTimeoutPolicy( + new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(12)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(3).withTimeoutInSeconds(20).withIntervalInSeconds(60))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponentResiliencyPolicy_CreateOrUpdate_SparseOptions.json + */ + /** + * Sample code: Create or update dapr component resiliency policy with sparse options. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateDaprComponentResiliencyPolicyWithSparseOptions( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponentResiliencyPolicies().define("myresiliencypolicy") + .withExistingDaprComponent("examplerg", "myenvironment", "mydaprcomponent") + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy(new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(5) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(2000).withMaxIntervalInMilliseconds(5500))) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(3).withTimeoutInSeconds(20))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration().withTimeoutPolicy( + new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(12))) + .create(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteSamples.java new file mode 100644 index 0000000000000..5028bc277fa78 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DaprComponentResiliencyPolicies Delete. + */ +public final class DaprComponentResiliencyPoliciesDeleteSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponentResiliencyPolicies_Delete.json + */ + /** + * Sample code: Delete dapr component resiliency policy. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + deleteDaprComponentResiliencyPolicy(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponentResiliencyPolicies().deleteWithResponse("examplerg", "myenvironment", "mydaprcomponent", + "myresiliencypolicy", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetSamples.java new file mode 100644 index 0000000000000..b61495084f830 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DaprComponentResiliencyPolicies Get. + */ +public final class DaprComponentResiliencyPoliciesGetSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponentResiliencyPolicies_Get.json + */ + /** + * Sample code: Get Dapr component resiliency policy. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + getDaprComponentResiliencyPolicy(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponentResiliencyPolicies().getWithResponse("examplerg", "myenvironment", "mydaprcomponent", + "myresiliencypolicy", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListSamples.java new file mode 100644 index 0000000000000..3958c1b87824b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DaprComponentResiliencyPolicies List. + */ +public final class DaprComponentResiliencyPoliciesListSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponentResiliencyPolicies_List.json + */ + /** + * Sample code: List Dapr component resiliency policies. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + listDaprComponentResiliencyPolicies(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponentResiliencyPolicies().list("examplerg", "myenvironment", "mydaprcomponent", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsCreateOrUpdateSamples.java index 30857b58e6c47..a5f7ec35e4aca 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsCreateOrUpdateSamples.java @@ -5,75 +5,68 @@ package com.azure.resourcemanager.appcontainers.generated; import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentInner; +import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding; import com.azure.resourcemanager.appcontainers.models.DaprMetadata; +import com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata; import com.azure.resourcemanager.appcontainers.models.Secret; import java.util.Arrays; -/** Samples for DaprComponents CreateOrUpdate. */ +/** + * Samples for DaprComponents CreateOrUpdate. + */ public final class DaprComponentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_CreateOrUpdate_Secrets.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponents_CreateOrUpdate_Secrets.json */ /** * Sample code: Create or update dapr component with secrets. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createOrUpdateDaprComponentWithSecrets( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .daprComponents() - .createOrUpdateWithResponse( - "examplerg", - "myenvironment", - "reddog", - new DaprComponentInner() - .withComponentType("state.azure.cosmosdb") - .withVersion("v1") - .withIgnoreErrors(false) - .withInitTimeout("50s") - .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue"))) + manager.daprComponents().createOrUpdateWithResponse("examplerg", "myenvironment", "reddog", + new DaprComponentInner().withComponentType("state.azure.cosmosdb").withVersion("v1").withIgnoreErrors(false) + .withInitTimeout("50s") + .withSecrets(Arrays.asList(new Secret().withName("masterkey").withValue("keyvalue"))) + .withMetadata(Arrays.asList(new DaprMetadata().withName("url").withValue(""), + new DaprMetadata().withName("database").withValue("itemsDB"), + new DaprMetadata().withName("collection").withValue("items"), + new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder"))) + .withScopes(Arrays.asList("container-app-1", "container-app-2")) + .withServiceComponentBind(Arrays.asList(new DaprComponentServiceBinding().withName("statestore") + .withServiceId( + "/subscriptions/9f7371f1-b593-4c3c-84e2-9167806ad358/resourceGroups/ca-syn2-group/providers/Microsoft.App/containerapps/cappredis") .withMetadata( - Arrays - .asList( - new DaprMetadata().withName("url").withValue(""), - new DaprMetadata().withName("database").withValue("itemsDB"), - new DaprMetadata().withName("collection").withValue("items"), - new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder"))) - .withScopes(Arrays.asList("container-app-1", "container-app-2")), - com.azure.core.util.Context.NONE); + new DaprServiceBindMetadata().withName("daprcomponentBind").withValue("redis-bind")))), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_CreateOrUpdate_SecretStoreComponent.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponents_CreateOrUpdate_SecretStoreComponent.json */ /** * Sample code: Create or update dapr component with secret store component. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createOrUpdateDaprComponentWithSecretStoreComponent( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .daprComponents() - .createOrUpdateWithResponse( - "examplerg", - "myenvironment", - "reddog", - new DaprComponentInner() - .withComponentType("state.azure.cosmosdb") - .withVersion("v1") - .withIgnoreErrors(false) - .withInitTimeout("50s") - .withSecretStoreComponent("fakeTokenPlaceholder") + manager.daprComponents().createOrUpdateWithResponse("examplerg", "myenvironment", "reddog", + new DaprComponentInner().withComponentType("state.azure.cosmosdb").withVersion("v1").withIgnoreErrors(false) + .withInitTimeout("50s").withSecretStoreComponent("fakeTokenPlaceholder") + .withMetadata(Arrays.asList(new DaprMetadata().withName("url").withValue(""), + new DaprMetadata().withName("database").withValue("itemsDB"), + new DaprMetadata().withName("collection").withValue("items"), + new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder"))) + .withScopes(Arrays.asList("container-app-1", "container-app-2")) + .withServiceComponentBind(Arrays.asList(new DaprComponentServiceBinding().withName("statestore") + .withServiceId( + "/subscriptions/9f7371f1-b593-4c3c-84e2-9167806ad358/resourceGroups/ca-syn2-group/providers/Microsoft.App/containerapps/cappredis") .withMetadata( - Arrays - .asList( - new DaprMetadata().withName("url").withValue(""), - new DaprMetadata().withName("database").withValue("itemsDB"), - new DaprMetadata().withName("collection").withValue("items"), - new DaprMetadata().withName("masterkey").withSecretRef("fakeTokenPlaceholder"))) - .withScopes(Arrays.asList("container-app-1", "container-app-2")), - com.azure.core.util.Context.NONE); + new DaprServiceBindMetadata().withName("daprcomponentBind").withValue("redis-bind")))), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteSamples.java index b43543b25ff92..0ca2552786b8e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for DaprComponents Delete. */ +/** + * Samples for DaprComponents Delete. + */ public final class DaprComponentsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_Delete.json */ /** * Sample code: Delete dapr component. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteDaprComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .daprComponents() - .deleteWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + manager.daprComponents().deleteWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsGetSamples.java index 9d78dfd7338a1..4d7160ec58e32 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsGetSamples.java @@ -4,35 +4,38 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for DaprComponents Get. */ +/** + * Samples for DaprComponents Get. + */ public final class DaprComponentsGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_Get_SecretStoreComponent.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprComponents_Get_SecretStoreComponent.json */ /** * Sample code: Get Dapr Component with secret store component. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getDaprComponentWithSecretStoreComponent( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .daprComponents() - .getWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + manager.daprComponents().getWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_Get_Secrets.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_Get_Secrets. + * json */ /** * Sample code: Get Dapr Component with secrets. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getDaprComponentWithSecrets( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .daprComponents() - .getWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + public static void + getDaprComponentWithSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponents().getWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSamples.java index 42a07bc738808..485c421cb4881 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for DaprComponents List. */ +/** + * Samples for DaprComponents List. + */ public final class DaprComponentsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_List.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_List.json */ /** * Sample code: List Dapr Components. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listDaprComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsSamples.java index a08f7ca6bfce5..f20d1b9d59d68 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for DaprComponents ListSecrets. */ +/** + * Samples for DaprComponents ListSecrets. + */ public final class DaprComponentsListSecretsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/DaprComponents_ListSecrets.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprComponents_ListSecrets. + * json */ /** * Sample code: List Container Apps Secrets. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsSecrets( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .daprComponents() - .listSecretsWithResponse("examplerg", "myenvironment", "reddog", com.azure.core.util.Context.NONE); + public static void + listContainerAppsSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprComponents().listSecretsWithResponse("examplerg", "myenvironment", "reddog", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..9c85aa97a89ed --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateSamples.java @@ -0,0 +1,87 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for DaprSubscriptions CreateOrUpdate. + */ +public final class DaprSubscriptionsCreateOrUpdateSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprSubscriptions_CreateOrUpdate_RouteRulesAndMetadata.json + */ + /** + * Sample code: Create or update dapr subscription with route rules and metadata. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateDaprSubscriptionWithRouteRulesAndMetadata( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().define("mysubscription") + .withExistingManagedEnvironment("examplerg", "myenvironment").withPubsubName("mypubsubcomponent") + .withTopic("inventory") + .withRoutes(new DaprSubscriptionRoutes() + .withRules(Arrays.asList( + new DaprSubscriptionRouteRule().withMatch("event.type == 'widget'").withPath("/widgets"), + new DaprSubscriptionRouteRule().withMatch("event.type == 'gadget'").withPath("/gadgets"))) + .withDefaultProperty("/products")) + .withMetadata(mapOf("foo", "bar", "hello", "world")).create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprSubscriptions_CreateOrUpdate_DefaultRoute.json + */ + /** + * Sample code: Create or update dapr subscription with default route only. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateDaprSubscriptionWithDefaultRouteOnly( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().define("mysubscription") + .withExistingManagedEnvironment("examplerg", "myenvironment").withPubsubName("mypubsubcomponent") + .withTopic("inventory").withRoutes(new DaprSubscriptionRoutes().withDefaultProperty("/products")).create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprSubscriptions_CreateOrUpdate_BulkSubscribeAndScopes.json + */ + /** + * Sample code: Create or update dapr subscription with bulk subscribe configuration and scopes. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateDaprSubscriptionWithBulkSubscribeConfigurationAndScopes( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().define("mysubscription") + .withExistingManagedEnvironment("examplerg", "myenvironment").withPubsubName("mypubsubcomponent") + .withTopic("inventory").withRoutes(new DaprSubscriptionRoutes().withDefaultProperty("/products")) + .withScopes(Arrays.asList("warehouseapp", "customersupportapp")) + .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(true).withMaxMessagesCount(123) + .withMaxAwaitDurationMs(500)) + .create(); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteSamples.java new file mode 100644 index 0000000000000..140aede4ee32b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DaprSubscriptions Delete. + */ +public final class DaprSubscriptionsDeleteSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprSubscriptions_Delete. + * json + */ + /** + * Sample code: Delete dapr subscription. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void deleteDaprSubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().deleteWithResponse("examplerg", "myenvironment", "mysubscription", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetSamples.java new file mode 100644 index 0000000000000..e6c044b425c33 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetSamples.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DaprSubscriptions Get. + */ +public final class DaprSubscriptionsGetSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprSubscriptions_Get_DefaultRoute.json + */ + /** + * Sample code: Get Dapr subscription with bulk subscribe configuration and scopes. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getDaprSubscriptionWithBulkSubscribeConfigurationAndScopes( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().getWithResponse("examplerg", "myenvironment", "mypubsubcomponent", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprSubscriptions_Get_BulkSubscribeAndScopes.json + */ + /** + * Sample code: Get Dapr subscription with default route only. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getDaprSubscriptionWithDefaultRouteOnly( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().getWithResponse("examplerg", "myenvironment", "mypubsubcomponent", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DaprSubscriptions_Get_RouteRulesAndMetadata.json + */ + /** + * Sample code: GetDapr subscription with route rules and metadata. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getDaprSubscriptionWithRouteRulesAndMetadata( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().getWithResponse("examplerg", "myenvironment", "mypubsubcomponent", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListSamples.java new file mode 100644 index 0000000000000..c94c7a32d307d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DaprSubscriptions List. + */ +public final class DaprSubscriptionsListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DaprSubscriptions_List.json + */ + /** + * Sample code: List Dapr subscriptions. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void listDaprSubscriptions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.daprSubscriptions().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..e89663f1e9755 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateSamples.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.util.Arrays; + +/** + * Samples for DotNetComponents CreateOrUpdate. + */ +public final class DotNetComponentsCreateOrUpdateSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DotNetComponents_CreateOrUpdate.json + */ + /** + * Sample code: Create or Update .NET Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + createOrUpdateNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().define("mydotnetcomponent") + .withExistingManagedEnvironment("examplerg", "myenvironment") + .withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark"))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DotNetComponents_CreateOrUpdate_ServiceBind.json + */ + /** + * Sample code: Create or Update .NET Component with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateNETComponentWithServiceBinds( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().define("mydotnetcomponent") + .withExistingManagedEnvironment("examplerg", "myenvironment") + .withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark"))) + .withServiceBinds(Arrays.asList(new DotNetComponentServiceBind().withName("yellowcat").withServiceId( + "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/dotNetComponents/yellowcat"))) + .create(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsDeleteSamples.java new file mode 100644 index 0000000000000..9725b60a6de91 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsDeleteSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DotNetComponents Delete. + */ +public final class DotNetComponentsDeleteSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_Delete.json + */ + /** + * Sample code: Delete .NET Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void deleteNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().delete("examplerg", "myenvironment", "mydotnetcomponent", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetSamples.java new file mode 100644 index 0000000000000..1fbd1fab504cb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetSamples.java @@ -0,0 +1,39 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DotNetComponents Get. + */ +public final class DotNetComponentsGetSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DotNetComponents_Get_ServiceBind.json + */ + /** + * Sample code: Get .NET Component with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + getNETComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().getWithResponse("examplerg", "myenvironment", "mydotnetcomponent", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_Get.json + */ + /** + * Sample code: Get .NET Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().getWithResponse("examplerg", "myenvironment", "mydotnetcomponent", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListSamples.java new file mode 100644 index 0000000000000..9ec1b69c8d0c5 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListSamples.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for DotNetComponents List. + */ +public final class DotNetComponentsListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_List.json + */ + /** + * Sample code: List .NET Components. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void listNETComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DotNetComponents_List_ServiceBind.json + */ + /** + * Sample code: List .NET Components with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + listNETComponentsWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.dotNetComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsUpdateSamples.java new file mode 100644 index 0000000000000..a2bc69652dd2d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsUpdateSamples.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.DotNetComponent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.util.Arrays; + +/** + * Samples for DotNetComponents Update. + */ +public final class DotNetComponentsUpdateSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * DotNetComponents_Patch_ServiceBind.json + */ + /** + * Sample code: Patch .NET Component with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + patchNETComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + DotNetComponent resource = manager.dotNetComponents() + .getWithResponse("examplerg", "myenvironment", "mydotnetcomponent", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark"))) + .withServiceBinds(Arrays.asList(new DotNetComponentServiceBind().withName("yellowcat").withServiceId( + "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/dotNetComponents/yellowcat"))) + .apply(); + } + + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/DotNetComponents_Patch.json + */ + /** + * Sample code: Patch .NET Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void patchNETComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + DotNetComponent resource = manager.dotNetComponents() + .getWithResponse("examplerg", "myenvironment", "mydotnetcomponent", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("dashboard-theme").withValue("dark"))) + .apply(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateSamples.java new file mode 100644 index 0000000000000..2f881bdca227a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateSamples.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.util.Arrays; + +/** + * Samples for JavaComponents CreateOrUpdate. + */ +public final class JavaComponentsCreateOrUpdateSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * JavaComponents_CreateOrUpdate_ServiceBind.json + */ + /** + * Sample code: Create or Update Java Component with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateJavaComponentWithServiceBinds( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().define("myjavacomponent").withExistingManagedEnvironment("examplerg", "myenvironment") + .withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations(Arrays.asList( + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts") + .withValue("true"), + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval") + .withValue("10000ms"))) + .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("yellowcat").withServiceId( + "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/javaComponents/yellowcat"))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * JavaComponents_CreateOrUpdate.json + */ + /** + * Sample code: Create or Update Java Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + createOrUpdateJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().define("myjavacomponent").withExistingManagedEnvironment("examplerg", "myenvironment") + .withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations(Arrays.asList( + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts") + .withValue("true"), + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval") + .withValue("10000ms"))) + .create(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsDeleteSamples.java new file mode 100644 index 0000000000000..5b5824e89d4ab --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsDeleteSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for JavaComponents Delete. + */ +public final class JavaComponentsDeleteSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_Delete.json + */ + /** + * Sample code: Delete Java Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void deleteJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().delete("examplerg", "myenvironment", "myjavacomponent", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetSamples.java new file mode 100644 index 0000000000000..85ff835b9743a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetSamples.java @@ -0,0 +1,39 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for JavaComponents Get. + */ +public final class JavaComponentsGetSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_Get.json + */ + /** + * Sample code: Get Java Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().getWithResponse("examplerg", "myenvironment", "myjavacomponent", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * JavaComponents_Get_ServiceBind.json + */ + /** + * Sample code: Get Java Component with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + getJavaComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().getWithResponse("examplerg", "myenvironment", "myjavacomponent", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListSamples.java new file mode 100644 index 0000000000000..6f3508a78f3c7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListSamples.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for JavaComponents List. + */ +public final class JavaComponentsListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_List.json + */ + /** + * Sample code: List Java Components. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void listJavaComponents(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * JavaComponents_List_ServiceBind.json + */ + /** + * Sample code: List Java Components with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + listJavaComponentsWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.javaComponents().list("examplerg", "myenvironment", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsUpdateSamples.java new file mode 100644 index 0000000000000..7164653c6720d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsUpdateSamples.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.resourcemanager.appcontainers.models.JavaComponent; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.util.Arrays; + +/** + * Samples for JavaComponents Update. + */ +public final class JavaComponentsUpdateSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/JavaComponents_Patch.json + */ + /** + * Sample code: Patch Java Component. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void patchJavaComponent(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + JavaComponent resource = manager.javaComponents() + .getWithResponse("examplerg", "myenvironment", "myjavacomponent", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations(Arrays.asList( + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts") + .withValue("true"), + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval") + .withValue("10000ms"))) + .apply(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * JavaComponents_Patch_ServiceBind.json + */ + /** + * Sample code: Patch Java Component with ServiceBinds. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + patchJavaComponentWithServiceBinds(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + JavaComponent resource = manager.javaComponents() + .getWithResponse("examplerg", "myenvironment", "myjavacomponent", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations(Arrays.asList( + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.ui.enable-toasts") + .withValue("true"), + new JavaComponentConfigurationProperty().withPropertyName("spring.boot.admin.monitor.status-interval") + .withValue("10000ms"))) + .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("yellowcat").withServiceId( + "/subscriptions/8efdecc5-919e-44eb-b179-915dca89ebf9/resourceGroups/examplerg/providers/Microsoft.App/managedEnvironments/myenvironment/javaComponents/yellowcat"))) + .apply(); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsCreateOrUpdateSamples.java index 016f291626e5f..844b388741699 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsCreateOrUpdateSamples.java @@ -11,6 +11,8 @@ import com.azure.resourcemanager.appcontainers.models.ContainerAppProbeHttpGet; import com.azure.resourcemanager.appcontainers.models.ContainerAppProbeHttpGetHttpHeadersItem; import com.azure.resourcemanager.appcontainers.models.ContainerResources; +import com.azure.resourcemanager.appcontainers.models.ExtendedLocation; +import com.azure.resourcemanager.appcontainers.models.ExtendedLocationTypes; import com.azure.resourcemanager.appcontainers.models.InitContainer; import com.azure.resourcemanager.appcontainers.models.JobConfiguration; import com.azure.resourcemanager.appcontainers.models.JobConfigurationEventTriggerConfig; @@ -20,133 +22,118 @@ import com.azure.resourcemanager.appcontainers.models.JobTemplate; import com.azure.resourcemanager.appcontainers.models.TriggerType; import com.azure.resourcemanager.appcontainers.models.Type; +import com.azure.resourcemanager.appcontainers.models.VolumeMount; import java.io.IOException; import java.util.Arrays; -/** Samples for Jobs CreateOrUpdate. */ +/** + * Samples for Jobs CreateOrUpdate. + */ public final class JobsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_CreateorUpdate_EventTrigger.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Job_CreateorUpdate_ConnectedEnvironment.json + */ + /** + * Sample code: Create or Update Container Apps Job On A Connected Environment. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void createOrUpdateContainerAppsJobOnAConnectedEnvironment( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.jobs().define("testcontainerAppsJob0").withRegion("East US").withExistingResourceGroup("rg") + .withExtendedLocation(new ExtendedLocation().withName( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.ExtendedLocation/customLocations/testcustomlocation") + .withType(ExtendedLocationTypes.CUSTOM_LOCATION)) + .withEnvironmentId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/connectedEnvironments/demokube") + .withConfiguration(new JobConfiguration().withTriggerType(TriggerType.MANUAL).withReplicaTimeout(10) + .withReplicaRetryLimit(10).withManualTriggerConfig( + new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(1).withParallelism(4))) + .withTemplate(new JobTemplate() + .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4") + .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers(Arrays + .asList(new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0") + .withProbes(Arrays.asList(new ContainerAppProbe() + .withHttpGet(new ContainerAppProbeHttpGet() + .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem() + .withName("Custom-Header").withValue("Awesome"))) + .withPath("/health").withPort(8080)) + .withInitialDelaySeconds(5).withPeriodSeconds(3).withType(Type.LIVENESS)))))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Job_CreateorUpdate_EventTrigger.json */ /** * Sample code: Create or Update Container Apps Job With Event Driven Trigger. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createOrUpdateContainerAppsJobWithEventDrivenTrigger( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) throws IOException { - manager - .jobs() - .define("testcontainerAppsJob0") - .withRegion("East US") - .withExistingResourceGroup("rg") + manager.jobs().define("testcontainerAppsJob0").withRegion("East US").withExistingResourceGroup("rg") .withEnvironmentId( "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube") - .withConfiguration( - new JobConfiguration() - .withTriggerType(TriggerType.EVENT) - .withReplicaTimeout(10) - .withReplicaRetryLimit(10) - .withEventTriggerConfig( - new JobConfigurationEventTriggerConfig() - .withReplicaCompletionCount(1) - .withParallelism(4) - .withScale( - new JobScale() - .withPollingInterval(40) - .withMinExecutions(1) - .withMaxExecutions(5) - .withRules( - Arrays - .asList( - new JobScaleRule() - .withName("servicebuscalingrule") - .withType("azure-servicebus") - .withMetadata( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"topicName\":\"my-topic\"}", - Object.class, - SerializerEncoding.JSON))))))) - .withTemplate( - new JobTemplate() - .withInitContainers( - Arrays - .asList( - new InitContainer() - .withImage("repo/testcontainerAppsJob0:v4") - .withName("testinitcontainerAppsJob0") - .withCommand(Arrays.asList("/bin/sh")) - .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerAppsJob0:v1") - .withName("testcontainerAppsJob0")))) + .withConfiguration(new JobConfiguration().withTriggerType(TriggerType.EVENT).withReplicaTimeout(10) + .withReplicaRetryLimit(10).withEventTriggerConfig( + new JobConfigurationEventTriggerConfig().withReplicaCompletionCount(1).withParallelism(4) + .withScale(new JobScale().withPollingInterval(40).withMinExecutions(1).withMaxExecutions(5) + .withRules(Arrays.asList(new JobScaleRule().withName("servicebuscalingrule") + .withType("azure-servicebus") + .withMetadata(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"topicName\":\"my-topic\"}", Object.class, SerializerEncoding.JSON))))))) + .withTemplate(new JobTemplate() + .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4") + .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers(Arrays.asList( + new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0")))) .create(); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_CreateorUpdate.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_CreateorUpdate.json */ /** * Sample code: Create or Update Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateContainerAppsJob( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .jobs() - .define("testcontainerAppsJob0") - .withRegion("East US") - .withExistingResourceGroup("rg") + public static void + createOrUpdateContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.jobs().define("testcontainerAppsJob0").withRegion("East US").withExistingResourceGroup("rg") .withEnvironmentId( "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/rg/providers/Microsoft.App/managedEnvironments/demokube") - .withConfiguration( - new JobConfiguration() - .withTriggerType(TriggerType.MANUAL) - .withReplicaTimeout(10) - .withReplicaRetryLimit(10) - .withManualTriggerConfig( - new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(1).withParallelism(4))) + .withConfiguration(new JobConfiguration().withTriggerType(TriggerType.MANUAL).withReplicaTimeout(10) + .withReplicaRetryLimit(10).withManualTriggerConfig( + new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(1).withParallelism(4))) .withTemplate( new JobTemplate() - .withInitContainers( - Arrays - .asList( - new InitContainer() - .withImage("repo/testcontainerAppsJob0:v4") - .withName("testinitcontainerAppsJob0") - .withCommand(Arrays.asList("/bin/sh")) - .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerAppsJob0:v1") - .withName("testcontainerAppsJob0") - .withProbes( - Arrays - .asList( - new ContainerAppProbe() - .withHttpGet( - new ContainerAppProbeHttpGet() - .withHttpHeaders( - Arrays - .asList( - new ContainerAppProbeHttpGetHttpHeadersItem() - .withName("Custom-Header") - .withValue("Awesome"))) - .withPath("/health") - .withPort(8080)) - .withInitialDelaySeconds(5) - .withPeriodSeconds(3) - .withType(Type.LIVENESS)))))) + .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4") + .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers(Arrays.asList( + new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0") + .withVolumeMounts(Arrays.asList( + new VolumeMount().withVolumeName("azurefile").withMountPath("/mnt/path1") + .withSubPath("subPath1"), + new VolumeMount().withVolumeName("nfsazurefile").withMountPath("/mnt/path2") + .withSubPath("subPath2"))) + .withProbes(Arrays.asList(new ContainerAppProbe() + .withHttpGet(new ContainerAppProbeHttpGet() + .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem() + .withName("Custom-Header").withValue("Awesome"))) + .withPath("/health").withPort(8080)) + .withInitialDelaySeconds(5).withPeriodSeconds(3).withType(Type.LIVENESS)))))) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteSamples.java index 871bcabeac07d..6db91f788162c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs Delete. */ +/** + * Samples for Jobs Delete. + */ public final class JobsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Delete.json */ /** * Sample code: Delete Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListSamples.java index e4c7ee3ca564a..f10bd3a921c87 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for JobsExecutions List. */ +/** + * Samples for JobsExecutions List. + */ public final class JobsExecutionsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Executions_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Executions_Get.json */ /** * Sample code: Get a Container Apps Job Executions. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getAContainerAppsJobExecutions( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + getAContainerAppsJobExecutions(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.jobsExecutions().list("rg", "testcontainerAppsJob0", null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetByResourceGroupSamples.java index a874e18846807..b0ff6f85a8cd0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetByResourceGroupSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs GetByResourceGroup. */ +/** + * Samples for Jobs GetByResourceGroup. + */ public final class JobsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Get.json */ /** * Sample code: Get Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorSamples.java new file mode 100644 index 0000000000000..1fe953f78b702 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Jobs GetDetector. + */ +public final class JobsGetDetectorSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_GetDetector.json + */ + /** + * Sample code: Get diagnostic data for a Container App Job. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + getDiagnosticDataForAContainerAppJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.jobs().getDetectorWithResponse("mikono-workerapp-test-rg", "mikonojob1", "containerappjobnetworkIO", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListByResourceGroupSamples.java index cc7de8ef684de..be43a18ec9850 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListByResourceGroupSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs ListByResourceGroup. */ +/** + * Samples for Jobs ListByResourceGroup. + */ public final class JobsListByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Jobs_ListByResourceGroup.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Jobs_ListByResourceGroup. + * json */ /** * Sample code: List Container Apps Jobs by resource group. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsJobsByResourceGroup( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listContainerAppsJobsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.jobs().listByResourceGroup("rg", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsSamples.java new file mode 100644 index 0000000000000..f15907dd2ff95 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Jobs ListDetectors. + */ +public final class JobsListDetectorsSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_ListDetectors.json + */ + /** + * Sample code: Get the list of available diagnostic data for a Container App Job. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getTheListOfAvailableDiagnosticDataForAContainerAppJob( + com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.jobs().listDetectorsWithResponse("mikono-workerapp-test-rg", "mikonojob1", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSamples.java index 0e97b2fc3b37a..adad6deaae4b9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs List. */ +/** + * Samples for Jobs List. + */ public final class JobsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Jobs_ListBySubscription.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Jobs_ListBySubscription.json */ /** * Sample code: List Container Apps Jobs by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsJobsBySubscription( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listContainerAppsJobsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.jobs().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSecretsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSecretsSamples.java index 6c143787f249d..c4af654141345 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSecretsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsListSecretsSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs ListSecrets. */ +/** + * Samples for Jobs ListSecrets. + */ public final class JobsListSecretsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_ListSecrets.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_ListSecrets.json */ /** * Sample code: List Container Apps Job Secrets. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listContainerAppsJobSecrets( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listContainerAppsJobSecrets(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.jobs().listSecretsWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsProxyGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsProxyGetSamples.java new file mode 100644 index 0000000000000..a93625f93ef09 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsProxyGetSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Jobs ProxyGet. + */ +public final class JobsProxyGetSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_ProxyGet.json + */ + /** + * Sample code: Get Container App Job by name. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + getContainerAppJobByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.jobs().proxyGetWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStartSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStartSamples.java index f816938a3c82d..58cfd9bc200b1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStartSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStartSamples.java @@ -9,39 +9,29 @@ import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate; import java.util.Arrays; -/** Samples for Jobs Start. */ +/** + * Samples for Jobs Start. + */ public final class JobsStartSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Start.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Start.json */ /** * Sample code: Run a Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void runAContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .jobs() - .start( - "rg", - "testcontainerAppsJob0", - new JobExecutionTemplate() - .withContainers( - Arrays - .asList( - new JobExecutionContainer() - .withImage("repo/testcontainerAppsJob0:v4") - .withName("testcontainerAppsJob0") - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) - .withInitContainers( - Arrays - .asList( - new JobExecutionContainer() - .withImage("repo/testcontainerAppsJob0:v4") - .withName("testinitcontainerAppsJob0") - .withCommand(Arrays.asList("/bin/sh")) - .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))), - com.azure.core.util.Context.NONE); + manager.jobs().start("rg", "testcontainerAppsJob0", + new JobExecutionTemplate() + .withContainers(Arrays.asList(new JobExecutionContainer().withImage("repo/testcontainerAppsJob0:v4") + .withName("testcontainerAppsJob0") + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withInitContainers(Arrays.asList(new JobExecutionContainer().withImage("repo/testcontainerAppsJob0:v4") + .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionSamples.java index bdc43bc3fe758..1afbf7e09fda1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs StopExecution. */ +/** + * Samples for Jobs StopExecution. + */ public final class JobsStopExecutionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Stop_Execution.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Stop_Execution.json */ /** * Sample code: Terminate a Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void terminateAContainerAppsJob( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + terminateAContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.jobs().stopExecution("rg", "testcontainerAppsJob0", "jobExecution1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsSamples.java index cdd0821a101d2..cb8f119105be5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Jobs StopMultipleExecutions. */ +/** + * Samples for Jobs StopMultipleExecutions. + */ public final class JobsStopMultipleExecutionsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Stop_Multiple.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Stop_Multiple.json */ /** * Sample code: Terminate Multiple Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void terminateMultipleContainerAppsJob( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + terminateMultipleContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.jobs().stopMultipleExecutions("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsUpdateSamples.java index 6d27fb4dbcd33..504050178133d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/JobsUpdateSamples.java @@ -19,69 +19,43 @@ import com.azure.resourcemanager.appcontainers.models.Type; import java.util.Arrays; -/** Samples for Jobs Update. */ +/** + * Samples for Jobs Update. + */ public final class JobsUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Patch.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Patch.json */ /** * Sample code: Patch Container Apps Job. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void patchContainerAppsJob(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - Job resource = - manager - .jobs() - .getByResourceGroupWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() + Job resource = manager.jobs() + .getByResourceGroupWithResponse("rg", "testcontainerAppsJob0", com.azure.core.util.Context.NONE).getValue(); + resource.update() .withProperties( new JobPatchPropertiesProperties() .withConfiguration( - new JobConfiguration() - .withTriggerType(TriggerType.MANUAL) - .withReplicaTimeout(10) + new JobConfiguration().withTriggerType(TriggerType.MANUAL).withReplicaTimeout(10) .withReplicaRetryLimit(10) - .withManualTriggerConfig( - new JobConfigurationManualTriggerConfig() - .withReplicaCompletionCount(1) - .withParallelism(4))) - .withTemplate( - new JobTemplate() - .withInitContainers( - Arrays - .asList( - new InitContainer() - .withImage("repo/testcontainerAppsJob0:v4") - .withName("testinitcontainerAppsJob0") - .withCommand(Arrays.asList("/bin/sh")) - .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) - .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) - .withContainers( - Arrays - .asList( - new Container() - .withImage("repo/testcontainerAppsJob0:v1") - .withName("testcontainerAppsJob0") - .withProbes( - Arrays - .asList( - new ContainerAppProbe() - .withHttpGet( - new ContainerAppProbeHttpGet() - .withHttpHeaders( - Arrays - .asList( - new ContainerAppProbeHttpGetHttpHeadersItem() - .withName("Custom-Header") - .withValue("Awesome"))) - .withPath("/health") - .withPort(8080)) - .withInitialDelaySeconds(3) - .withPeriodSeconds(3) - .withType(Type.LIVENESS))))))) + .withManualTriggerConfig(new JobConfigurationManualTriggerConfig() + .withReplicaCompletionCount(1).withParallelism(4))) + .withTemplate(new JobTemplate() + .withInitContainers(Arrays.asList(new InitContainer().withImage("repo/testcontainerAppsJob0:v4") + .withName("testinitcontainerAppsJob0").withCommand(Arrays.asList("/bin/sh")) + .withArgs(Arrays.asList("-c", "while true; do echo hello; sleep 10;done")) + .withResources(new ContainerResources().withCpu(0.2D).withMemory("100Mi")))) + .withContainers(Arrays.asList( + new Container().withImage("repo/testcontainerAppsJob0:v1").withName("testcontainerAppsJob0") + .withProbes(Arrays.asList(new ContainerAppProbe() + .withHttpGet(new ContainerAppProbeHttpGet() + .withHttpHeaders(Arrays.asList(new ContainerAppProbeHttpGetHttpHeadersItem() + .withName("Custom-Header").withValue("Awesome"))) + .withPath("/health").withPort(8080)) + .withInitialDelaySeconds(3).withPeriodSeconds(3).withType(Type.LIVENESS))))))) .apply(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesCreateOrUpdateSamples.java index 4edc4b3c8106b..fde470aa26cb8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesCreateOrUpdateSamples.java @@ -7,27 +7,25 @@ import com.azure.resourcemanager.appcontainers.models.ManagedCertificateDomainControlValidation; import com.azure.resourcemanager.appcontainers.models.ManagedCertificateProperties; -/** Samples for ManagedCertificates CreateOrUpdate. */ +/** + * Samples for ManagedCertificates CreateOrUpdate. + */ public final class ManagedCertificatesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificate_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedCertificate_CreateOrUpdate.json */ /** * Sample code: Create or Update Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateCertificate( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedCertificates() - .define("certificate-firendly-name") - .withRegion("East US") + public static void + createOrUpdateCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.managedCertificates().define("certificate-firendly-name").withRegion("East US") .withExistingManagedEnvironment("examplerg", "testcontainerenv") - .withProperties( - new ManagedCertificateProperties() - .withSubjectName("my-subject-name.company.country.net") - .withDomainControlValidation(ManagedCertificateDomainControlValidation.CNAME)) + .withProperties(new ManagedCertificateProperties().withSubjectName("my-subject-name.company.country.net") + .withDomainControlValidation(ManagedCertificateDomainControlValidation.CNAME)) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteSamples.java index ca06075abb5ab..0208a9f10167b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedCertificates Delete. */ +/** + * Samples for ManagedCertificates Delete. + */ public final class ManagedCertificatesDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificate_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedCertificate_Delete. + * json */ /** * Sample code: Delete Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void deleteCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedCertificates() - .deleteWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE); + manager.managedCertificates().deleteWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesGetSamples.java index 103bf88d56713..aa76bca21eaff 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesGetSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedCertificates Get. */ +/** + * Samples for ManagedCertificates Get. + */ public final class ManagedCertificatesGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificate_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedCertificate_Get.json */ /** * Sample code: Get Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedCertificates() - .getWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE); + manager.managedCertificates().getWithResponse("examplerg", "testcontainerenv", "certificate-firendly-name", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesListSamples.java index 41123accf3f86..a0acb8ae2b9dd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedCertificates List. */ +/** + * Samples for ManagedCertificates List. + */ public final class ManagedCertificatesListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificates_ListByManagedEnvironment.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedCertificates_ListByManagedEnvironment.json */ /** * Sample code: List Managed Certificates by Managed Environment. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listManagedCertificatesByManagedEnvironment( diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesUpdateSamples.java index f2a73091a95c8..2c4059828efa7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesUpdateSamples.java @@ -8,24 +8,24 @@ import java.util.HashMap; import java.util.Map; -/** Samples for ManagedCertificates Update. */ +/** + * Samples for ManagedCertificates Update. + */ public final class ManagedCertificatesUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedCertificates_Patch.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedCertificates_Patch. + * json */ /** * Sample code: Patch Managed Certificate. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void patchManagedCertificate( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - ManagedCertificate resource = - manager - .managedCertificates() - .getWithResponse( - "examplerg", "testcontainerenv", "certificate-firendly-name", com.azure.core.util.Context.NONE) - .getValue(); + public static void + patchManagedCertificate(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + ManagedCertificate resource = manager.managedCertificates().getWithResponse("examplerg", "testcontainerenv", + "certificate-firendly-name", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorSamples.java index 87b75b6d922e1..a8284ec063a85 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorSamples.java @@ -4,24 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironmentDiagnostics GetDetector. */ +/** + * Samples for ManagedEnvironmentDiagnostics GetDetector. + */ public final class ManagedEnvironmentDiagnosticsGetDetectorSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentDiagnostics_Get.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentDiagnostics_Get.json */ /** * Sample code: Get diagnostic data for a managed environments. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getDiagnosticDataForAManagedEnvironments( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentDiagnostics() - .getDetectorWithResponse( - "mikono-workerapp-test-rg", - "mikonokubeenv", - "ManagedEnvAvailabilityMetrics", - com.azure.core.util.Context.NONE); + manager.managedEnvironmentDiagnostics().getDetectorWithResponse("mikono-workerapp-test-rg", "mikonokubeenv", + "ManagedEnvAvailabilityMetrics", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsSamples.java index f7704984eafd1..b1e0cd1a73571 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironmentDiagnostics ListDetectors. */ +/** + * Samples for ManagedEnvironmentDiagnostics ListDetectors. + */ public final class ManagedEnvironmentDiagnosticsListDetectorsSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentDiagnostics_List.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentDiagnostics_List.json */ /** * Sample code: Get the list of available diagnostic data for a managed environments. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getTheListOfAvailableDiagnosticDataForAManagedEnvironments( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentDiagnostics() - .listDetectorsWithResponse("mikono-workerapp-test-rg", "mikonokubeenv", com.azure.core.util.Context.NONE); + manager.managedEnvironmentDiagnostics().listDetectorsWithResponse("mikono-workerapp-test-rg", "mikonokubeenv", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListSamples.java new file mode 100644 index 0000000000000..3f0fd18501838 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for ManagedEnvironmentUsages List. + */ +public final class ManagedEnvironmentUsagesListSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentUsages_List.json + */ + /** + * Sample code: List managed environment usages. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + listManagedEnvironmentUsages(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.managedEnvironmentUsages().list("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsCreateOrUpdateSamples.java index 05df7f2c4a9c3..de579eae42f2d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsCreateOrUpdateSamples.java @@ -4,122 +4,130 @@ package com.azure.resourcemanager.appcontainers.generated; +import com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration; import com.azure.resourcemanager.appcontainers.models.AppLogsConfiguration; import com.azure.resourcemanager.appcontainers.models.CustomDomainConfiguration; +import com.azure.resourcemanager.appcontainers.models.DataDogConfiguration; +import com.azure.resourcemanager.appcontainers.models.DestinationsConfiguration; +import com.azure.resourcemanager.appcontainers.models.Header; import com.azure.resourcemanager.appcontainers.models.LogAnalyticsConfiguration; +import com.azure.resourcemanager.appcontainers.models.LogsConfiguration; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentPropertiesPeerAuthentication; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import com.azure.resourcemanager.appcontainers.models.MetricsConfiguration; import com.azure.resourcemanager.appcontainers.models.Mtls; +import com.azure.resourcemanager.appcontainers.models.OpenTelemetryConfiguration; +import com.azure.resourcemanager.appcontainers.models.OtlpConfiguration; +import com.azure.resourcemanager.appcontainers.models.TracesConfiguration; +import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity; import com.azure.resourcemanager.appcontainers.models.VnetConfiguration; import com.azure.resourcemanager.appcontainers.models.WorkloadProfile; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; -/** Samples for ManagedEnvironments CreateOrUpdate. */ +/** + * Samples for ManagedEnvironments CreateOrUpdate. + */ public final class ManagedEnvironmentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_CustomInfrastructureResourceGroup_Create.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironments_CustomInfrastructureResourceGroup_Create.json */ /** * Sample code: Create environment with custom infrastructureResourceGroup. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createEnvironmentWithCustomInfrastructureResourceGroup( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironments() - .define("testcontainerenv") - .withRegion("East US") + manager.managedEnvironments().define("testcontainerenv").withRegion("East US") .withExistingResourceGroup("examplerg") .withDaprAIConnectionString( "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/") - .withVnetConfiguration( - new VnetConfiguration() - .withInfrastructureSubnetId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1")) - .withAppLogsConfiguration( - new AppLogsConfiguration() - .withLogAnalyticsConfiguration( - new LogAnalyticsConfiguration().withCustomerId("string").withSharedKey("fakeTokenPlaceholder"))) + .withVnetConfiguration(new VnetConfiguration().withInfrastructureSubnetId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1")) + .withAppLogsConfiguration(new AppLogsConfiguration().withLogAnalyticsConfiguration( + new LogAnalyticsConfiguration().withCustomerId("string").withSharedKey("fakeTokenPlaceholder"))) .withZoneRedundant(true) - .withCustomDomainConfiguration( - new CustomDomainConfiguration() - .withDnsSuffix("www.my-name.com") - .withCertificateValue("Y2VydA==".getBytes()) - .withCertificatePassword("fakeTokenPlaceholder")) - .withWorkloadProfiles( - Arrays - .asList( - new WorkloadProfile() - .withName("My-GP-01") - .withWorkloadProfileType("GeneralPurpose") - .withMinimumCount(3) - .withMaximumCount(12), - new WorkloadProfile() - .withName("My-MO-01") - .withWorkloadProfileType("MemoryOptimized") - .withMinimumCount(3) - .withMaximumCount(6), - new WorkloadProfile() - .withName("My-CO-01") - .withWorkloadProfileType("ComputeOptimized") - .withMinimumCount(3) - .withMaximumCount(6), - new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption"))) - .withInfrastructureResourceGroup("myInfrastructureRgName") - .create(); + .withCustomDomainConfiguration(new CustomDomainConfiguration().withDnsSuffix("www.my-name.com") + .withCertificateValue("Y2VydA==".getBytes()).withCertificatePassword("fakeTokenPlaceholder")) + .withWorkloadProfiles(Arrays.asList( + new WorkloadProfile().withName("My-GP-01").withWorkloadProfileType("GeneralPurpose").withMinimumCount(3) + .withMaximumCount(12), + new WorkloadProfile().withName("My-MO-01").withWorkloadProfileType("MemoryOptimized") + .withMinimumCount(3).withMaximumCount(6), + new WorkloadProfile().withName("My-CO-01").withWorkloadProfileType("ComputeOptimized") + .withMinimumCount(3).withMaximumCount(6), + new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption"))) + .withInfrastructureResourceGroup("myInfrastructureRgName").create(); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironments_CreateOrUpdate.json */ /** * Sample code: Create environments. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void createEnvironments(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironments() - .define("testcontainerenv") - .withRegion("East US") + manager.managedEnvironments().define("testcontainerenv").withRegion("East US") .withExistingResourceGroup("examplerg") + .withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.fromString("SystemAssigned, UserAssigned")) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/contoso-resources/providers/Microsoft.ManagedIdentity/userAssignedIdentities/contoso-identity", + new UserAssignedIdentity()))) .withDaprAIConnectionString( "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://northcentralus-0.in.applicationinsights.azure.com/") - .withVnetConfiguration( - new VnetConfiguration() - .withInfrastructureSubnetId( - "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1")) + .withVnetConfiguration(new VnetConfiguration().withInfrastructureSubnetId( + "/subscriptions/34adfa4f-cedf-4dc0-ba29-b6d1a69ab345/resourceGroups/RGName/providers/Microsoft.Network/virtualNetworks/VNetName/subnets/subnetName1")) .withAppLogsConfiguration( - new AppLogsConfiguration() - .withLogAnalyticsConfiguration( - new LogAnalyticsConfiguration().withCustomerId("string").withSharedKey("fakeTokenPlaceholder"))) + new AppLogsConfiguration().withLogAnalyticsConfiguration(new LogAnalyticsConfiguration() + .withCustomerId("string").withSharedKey("fakeTokenPlaceholder").withDynamicJsonColumns(true))) + .withAppInsightsConfiguration(new AppInsightsConfiguration().withConnectionString( + "InstrumentationKey=00000000-0000-0000-0000-000000000000;IngestionEndpoint=https://eastus-8.in.applicationinsights.azure.com/;LiveEndpoint=https://eastus.livediagnostics.monitor.azure.com/")) + .withOpenTelemetryConfiguration( + new OpenTelemetryConfiguration() + .withDestinationsConfiguration( + new DestinationsConfiguration() + .withDataDogConfiguration( + new DataDogConfiguration().withSite("string").withKey("fakeTokenPlaceholder")) + .withOtlpConfigurations(Arrays.asList(new OtlpConfiguration().withName("dashboard") + .withEndpoint("dashboard.k8s.region.azurecontainerapps.io:80").withInsecure(true) + .withHeaders(Arrays + .asList(new Header().withKey("fakeTokenPlaceholder").withValue("xxxxxxxxxxx")))))) + .withTracesConfiguration(new TracesConfiguration().withDestinations(Arrays.asList("appInsights"))) + .withLogsConfiguration(new LogsConfiguration().withDestinations(Arrays.asList("appInsights"))) + .withMetricsConfiguration(new MetricsConfiguration().withDestinations(Arrays.asList("dataDog")))) .withZoneRedundant(true) - .withCustomDomainConfiguration( - new CustomDomainConfiguration() - .withDnsSuffix("www.my-name.com") - .withCertificateValue("Y2VydA==".getBytes()) - .withCertificatePassword("fakeTokenPlaceholder")) - .withWorkloadProfiles( - Arrays - .asList( - new WorkloadProfile() - .withName("My-GP-01") - .withWorkloadProfileType("GeneralPurpose") - .withMinimumCount(3) - .withMaximumCount(12), - new WorkloadProfile() - .withName("My-MO-01") - .withWorkloadProfileType("MemoryOptimized") - .withMinimumCount(3) - .withMaximumCount(6), - new WorkloadProfile() - .withName("My-CO-01") - .withWorkloadProfileType("ComputeOptimized") - .withMinimumCount(3) - .withMaximumCount(6), - new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption"))) + .withCustomDomainConfiguration(new CustomDomainConfiguration().withDnsSuffix("www.my-name.com") + .withCertificateValue("Y2VydA==".getBytes()).withCertificatePassword("fakeTokenPlaceholder")) + .withWorkloadProfiles(Arrays.asList( + new WorkloadProfile().withName("My-GP-01").withWorkloadProfileType("GeneralPurpose").withMinimumCount(3) + .withMaximumCount(12), + new WorkloadProfile().withName("My-MO-01").withWorkloadProfileType("MemoryOptimized") + .withMinimumCount(3).withMaximumCount(6), + new WorkloadProfile().withName("My-CO-01").withWorkloadProfileType("ComputeOptimized") + .withMinimumCount(3).withMaximumCount(6), + new WorkloadProfile().withName("My-consumption-01").withWorkloadProfileType("Consumption"))) .withPeerAuthentication( new ManagedEnvironmentPropertiesPeerAuthentication().withMtls(new Mtls().withEnabled(true))) .create(); } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteSamples.java index 054f73203210b..457641d21825c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironments Delete. */ +/** + * Samples for ManagedEnvironments Delete. + */ public final class ManagedEnvironmentsDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Delete.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Delete. + * json */ /** * Sample code: Delete environment by name. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void deleteEnvironmentByName( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + deleteEnvironmentByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.managedEnvironments().delete("examplerg", "examplekenv", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDiagnosticsGetRootSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDiagnosticsGetRootSamples.java index ce30d1f231370..b60a91d55c768 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDiagnosticsGetRootSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDiagnosticsGetRootSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironmentsDiagnostics GetRoot. */ +/** + * Samples for ManagedEnvironmentsDiagnostics GetRoot. + */ public final class ManagedEnvironmentsDiagnosticsGetRootSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Get.json */ /** * Sample code: Get environments by name. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getEnvironmentsByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentsDiagnostics() - .getRootWithResponse("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE); + manager.managedEnvironmentsDiagnostics().getRootWithResponse("examplerg", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetAuthTokenSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetAuthTokenSamples.java index 75360fb7b838d..7241d5d183be6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetAuthTokenSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetAuthTokenSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironments GetAuthToken. */ +/** + * Samples for ManagedEnvironments GetAuthToken. + */ public final class ManagedEnvironmentsGetAuthTokenSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_GetAuthToken.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironments_GetAuthToken.json */ /** * Sample code: Get Managed Environment Auth Token. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getManagedEnvironmentAuthToken( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + getManagedEnvironmentAuthToken(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.managedEnvironments().getAuthTokenWithResponse("rg", "testenv", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetByResourceGroupSamples.java index 92cc0793165c3..042e57fb442cc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsGetByResourceGroupSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironments GetByResourceGroup. */ +/** + * Samples for ManagedEnvironments GetByResourceGroup. + */ public final class ManagedEnvironmentsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Get.json */ /** * Sample code: Get environments by name. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getEnvironmentsByName(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironments() - .getByResourceGroupWithResponse("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE); + manager.managedEnvironments().getByResourceGroupWithResponse("examplerg", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListByResourceGroupSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListByResourceGroupSamples.java index 393cc7c3e5b0b..0d2ddc69d7490 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListByResourceGroupSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListByResourceGroupSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironments ListByResourceGroup. */ +/** + * Samples for ManagedEnvironments ListByResourceGroup. + */ public final class ManagedEnvironmentsListByResourceGroupSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_ListByResourceGroup.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironments_ListByResourceGroup.json */ /** * Sample code: List environments by resource group. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listEnvironmentsByResourceGroup( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listEnvironmentsByResourceGroup(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.managedEnvironments().listByResourceGroup("examplerg", com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListSamples.java index f7411924aff87..8d741e4aa717c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironments List. */ +/** + * Samples for ManagedEnvironments List. + */ public final class ManagedEnvironmentsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_ListBySubscription.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironments_ListBySubscription.json */ /** * Sample code: List environments by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listEnvironmentsBySubscription( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + public static void + listEnvironmentsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { manager.managedEnvironments().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesSamples.java index 307209c681a17..6ceae8d251fc1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironments ListWorkloadProfileStates. */ +/** + * Samples for ManagedEnvironments ListWorkloadProfileStates. + */ public final class ManagedEnvironmentsListWorkloadProfileStatesSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_ListWorkloadProfileStates.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironments_ListWorkloadProfileStates.json */ /** * Sample code: List environments by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void listEnvironmentsBySubscription( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironments() - .listWorkloadProfileStates("examplerg", "jlaw-demo1", com.azure.core.util.Context.NONE); + public static void + listEnvironmentsBySubscription(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.managedEnvironments().listWorkloadProfileStates("examplerg", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesCreateOrUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesCreateOrUpdateSamples.java index a9312d5fa604d..26b1d0f897f7f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesCreateOrUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesCreateOrUpdateSamples.java @@ -7,31 +7,46 @@ import com.azure.resourcemanager.appcontainers.models.AccessMode; import com.azure.resourcemanager.appcontainers.models.AzureFileProperties; import com.azure.resourcemanager.appcontainers.models.ManagedEnvironmentStorageProperties; +import com.azure.resourcemanager.appcontainers.models.NfsAzureFileProperties; -/** Samples for ManagedEnvironmentsStorages CreateOrUpdate. */ +/** + * Samples for ManagedEnvironmentsStorages CreateOrUpdate. + */ public final class ManagedEnvironmentsStoragesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_CreateOrUpdate.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentsStorages_CreateOrUpdate_NfsAzureFile.json */ /** - * Sample code: Create or update environments storage. - * + * Sample code: Create or update environments storage for NFS Azure file. + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void createOrUpdateEnvironmentsStorage( + public static void createOrUpdateEnvironmentsStorageForNFSAzureFile( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentsStorages() - .define("jlaw-demo1") + manager.managedEnvironmentsStorages().define("jlaw-demo1") + .withExistingManagedEnvironment("examplerg", "managedEnv") + .withProperties(new ManagedEnvironmentStorageProperties().withNfsAzureFile(new NfsAzureFileProperties() + .withServer("server1").withAccessMode(AccessMode.READ_ONLY).withShareName("share1"))) + .create(); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentsStorages_CreateOrUpdate.json + */ + /** + * Sample code: Create or update environments storage. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void + createOrUpdateEnvironmentsStorage(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.managedEnvironmentsStorages().define("jlaw-demo1") .withExistingManagedEnvironment("examplerg", "managedEnv") - .withProperties( - new ManagedEnvironmentStorageProperties() - .withAzureFile( - new AzureFileProperties() - .withAccountName("account1") - .withAccountKey("fakeTokenPlaceholder") - .withAccessMode(AccessMode.READ_ONLY) - .withShareName("share1"))) + .withProperties(new ManagedEnvironmentStorageProperties().withAzureFile( + new AzureFileProperties().withAccountName("account1").withAccountKey("fakeTokenPlaceholder") + .withAccessMode(AccessMode.READ_ONLY).withShareName("share1"))) .create(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteSamples.java index dd6296422fc5a..03400192ca158 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironmentsStorages Delete. */ +/** + * Samples for ManagedEnvironmentsStorages Delete. + */ public final class ManagedEnvironmentsStoragesDeleteSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_Delete.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentsStorages_Delete.json */ /** * Sample code: List environments storages by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listEnvironmentsStoragesBySubscription( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentsStorages() - .deleteWithResponse("examplerg", "managedEnv", "jlaw-demo1", com.azure.core.util.Context.NONE); + manager.managedEnvironmentsStorages().deleteWithResponse("examplerg", "managedEnv", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesGetSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesGetSamples.java index 5e0eeb477f66c..210f199df9fdd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesGetSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesGetSamples.java @@ -4,20 +4,37 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironmentsStorages Get. */ +/** + * Samples for ManagedEnvironmentsStorages Get. + */ public final class ManagedEnvironmentsStoragesGetSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_Get.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentsStorages_Get.json */ /** - * Sample code: get a environments storage properties by subscription. - * + * Sample code: get a environments storage. + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void getAEnvironmentsStoragePropertiesBySubscription( + public static void + getAEnvironmentsStorage(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.managedEnvironmentsStorages().getWithResponse("examplerg", "managedEnv", "jlaw-demo1", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentsStorages_Get_NfsAzureFile.json + */ + /** + * Sample code: get a environments storage for NFS Azure file. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void getAEnvironmentsStorageForNFSAzureFile( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentsStorages() - .getWithResponse("examplerg", "managedEnv", "jlaw-demo1", com.azure.core.util.Context.NONE); + manager.managedEnvironmentsStorages().getWithResponse("examplerg", "managedEnv", "jlaw-demo1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesListSamples.java index 077dbf57477ae..af95ae5d86fd4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesListSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ManagedEnvironmentsStorages List. */ +/** + * Samples for ManagedEnvironmentsStorages List. + */ public final class ManagedEnvironmentsStoragesListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironmentsStorages_List.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ManagedEnvironmentsStorages_List.json */ /** * Sample code: List environments storages by subscription. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listEnvironmentsStoragesBySubscription( com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .managedEnvironmentsStorages() - .listWithResponse("examplerg", "managedEnv", com.azure.core.util.Context.NONE); + manager.managedEnvironmentsStorages().listWithResponse("examplerg", "managedEnv", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsUpdateSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsUpdateSamples.java index 473995a4eb266..965bdf450755a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsUpdateSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsUpdateSamples.java @@ -8,23 +8,25 @@ import java.util.HashMap; import java.util.Map; -/** Samples for ManagedEnvironments Update. */ +/** + * Samples for ManagedEnvironments Update. + */ public final class ManagedEnvironmentsUpdateSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ManagedEnvironments_Patch.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ManagedEnvironments_Patch. + * json */ /** * Sample code: Patch Managed Environment. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void patchManagedEnvironment( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - ManagedEnvironment resource = - manager - .managedEnvironments() - .getByResourceGroupWithResponse("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE) - .getValue(); + public static void + patchManagedEnvironment(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + ManagedEnvironment resource = manager.managedEnvironments() + .getByResourceGroupWithResponse("examplerg", "testcontainerenv", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilitySamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilitySamples.java index 8a68bc86390ca..f721c1e7e9da1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilitySamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilitySamples.java @@ -6,45 +6,40 @@ import com.azure.resourcemanager.appcontainers.models.CheckNameAvailabilityRequest; -/** Samples for Namespaces CheckNameAvailability. */ +/** + * Samples for Namespaces CheckNameAvailability. + */ public final class NamespacesCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Certificates_CheckNameAvailability.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Certificates_CheckNameAvailability.json */ /** * Sample code: Certificates_CheckNameAvailability. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void certificatesCheckNameAvailability( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .namespaces() - .checkNameAvailabilityWithResponse( - "examplerg", - "testcontainerenv", - new CheckNameAvailabilityRequest() - .withName("testcertificatename") - .withType("Microsoft.App/managedEnvironments/certificates"), + public static void + certificatesCheckNameAvailability(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.namespaces() + .checkNameAvailabilityWithResponse("examplerg", "testcontainerenv", new CheckNameAvailabilityRequest() + .withName("testcertificatename").withType("Microsoft.App/managedEnvironments/certificates"), com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/ContainerApps_CheckNameAvailability.json + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * ContainerApps_CheckNameAvailability.json */ /** * Sample code: ContainerApps_CheckNameAvailability. - * + * * @param manager Entry point to ContainerAppsApiManager. */ - public static void containerAppsCheckNameAvailability( - com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .namespaces() - .checkNameAvailabilityWithResponse( - "examplerg", - "testcontainerenv", - new CheckNameAvailabilityRequest().withName("testcappname").withType("Microsoft.App/containerApps"), - com.azure.core.util.Context.NONE); + public static void + containerAppsCheckNameAvailability(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.namespaces().checkNameAvailabilityWithResponse("examplerg", "testcontainerenv", + new CheckNameAvailabilityRequest().withName("testcappname").withType("Microsoft.App/containerApps"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/OperationsListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/OperationsListSamples.java index 6453506f85a34..ba1100264a9b8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/OperationsListSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/OperationsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Operations_List.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Operations_List.json */ /** * Sample code: List all operations. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void listAllOperations(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderGetCustomDomainVerificationIdSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderGetCustomDomainVerificationIdSamples.java new file mode 100644 index 0000000000000..d9fcf15a6a33e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderGetCustomDomainVerificationIdSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for ResourceProvider GetCustomDomainVerificationId. + */ +public final class ResourceProviderGetCustomDomainVerificationIdSamples { + /* + * x-ms-original-file: specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/ + * Subscriptions_GetCustomDomainVerificationId.json + */ + /** + * Sample code: List all operations. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void listAllOperations(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.resourceProviders().getCustomDomainVerificationIdWithResponse(com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderJobExecutionSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderJobExecutionSamples.java index 1a03ed51d5e3a..8c74bfa78cc46 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderJobExecutionSamples.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/ResourceProviderJobExecutionSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.appcontainers.generated; -/** Samples for ResourceProvider JobExecution. */ +/** + * Samples for ResourceProvider JobExecution. + */ public final class ResourceProviderJobExecutionSamples { /* - * x-ms-original-file: specification/app/resource-manager/Microsoft.App/stable/2023-05-01/examples/Job_Execution_Get.json + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Job_Execution_Get.json */ /** * Sample code: Get a single Job Execution. - * + * * @param manager Entry point to ContainerAppsApiManager. */ public static void getASingleJobExecution(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { - manager - .resourceProviders() - .jobExecutionWithResponse("rg", "testcontainerAppsJob0", "jobExecution1", com.azure.core.util.Context.NONE); + manager.resourceProviders().jobExecutionWithResponse("rg", "testcontainerAppsJob0", "jobExecution1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/UsagesListSamples.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/UsagesListSamples.java new file mode 100644 index 0000000000000..149d43f6e59ee --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/samples/java/com/azure/resourcemanager/appcontainers/generated/UsagesListSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +/** + * Samples for Usages List. + */ +public final class UsagesListSamples { + /* + * x-ms-original-file: + * specification/app/resource-manager/Microsoft.App/preview/2023-11-02-preview/examples/Usages_List.json + */ + /** + * Sample code: List usages. + * + * @param manager Entry point to ContainerAppsApiManager. + */ + public static void listUsages(com.azure.resourcemanager.appcontainers.ContainerAppsApiManager manager) { + manager.usages().list("westus", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedAudiencesValidationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedAudiencesValidationTests.java index 5f3f4f48f8680..4a32859e2cf65 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedAudiencesValidationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedAudiencesValidationTests.java @@ -12,18 +12,17 @@ public final class AllowedAudiencesValidationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AllowedAudiencesValidation model = - BinaryData - .fromString("{\"allowedAudiences\":[\"wzrlovmclwhij\",\"oejctbzaqsqsy\"]}") + AllowedAudiencesValidation model + = BinaryData.fromString("{\"allowedAudiences\":[\"igdtopbob\",\"og\",\"m\",\"w\"]}") .toObject(AllowedAudiencesValidation.class); - Assertions.assertEquals("wzrlovmclwhij", model.allowedAudiences().get(0)); + Assertions.assertEquals("igdtopbob", model.allowedAudiences().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AllowedAudiencesValidation model = - new AllowedAudiencesValidation().withAllowedAudiences(Arrays.asList("wzrlovmclwhij", "oejctbzaqsqsy")); + AllowedAudiencesValidation model + = new AllowedAudiencesValidation().withAllowedAudiences(Arrays.asList("igdtopbob", "og", "m", "w")); model = BinaryData.fromObject(model).toObject(AllowedAudiencesValidation.class); - Assertions.assertEquals("wzrlovmclwhij", model.allowedAudiences().get(0)); + Assertions.assertEquals("igdtopbob", model.allowedAudiences().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedPrincipalsTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedPrincipalsTests.java index 72c21b159380a..89f4d08ea3692 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedPrincipalsTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AllowedPrincipalsTests.java @@ -12,22 +12,19 @@ public final class AllowedPrincipalsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AllowedPrincipals model = - BinaryData - .fromString("{\"groups\":[\"xhcr\",\"nohjt\"],\"identities\":[\"h\",\"soifiyipjxsqw\",\"gr\"]}") - .toObject(AllowedPrincipals.class); - Assertions.assertEquals("xhcr", model.groups().get(0)); - Assertions.assertEquals("h", model.identities().get(0)); + AllowedPrincipals model = BinaryData.fromString( + "{\"groups\":[\"ajionpimexgstxg\",\"po\",\"gmaajrm\"],\"identities\":[\"wzrlovmclwhij\",\"oejctbzaqsqsy\"]}") + .toObject(AllowedPrincipals.class); + Assertions.assertEquals("ajionpimexgstxg", model.groups().get(0)); + Assertions.assertEquals("wzrlovmclwhij", model.identities().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AllowedPrincipals model = - new AllowedPrincipals() - .withGroups(Arrays.asList("xhcr", "nohjt")) - .withIdentities(Arrays.asList("h", "soifiyipjxsqw", "gr")); + AllowedPrincipals model = new AllowedPrincipals().withGroups(Arrays.asList("ajionpimexgstxg", "po", "gmaajrm")) + .withIdentities(Arrays.asList("wzrlovmclwhij", "oejctbzaqsqsy")); model = BinaryData.fromObject(model).toObject(AllowedPrincipals.class); - Assertions.assertEquals("xhcr", model.groups().get(0)); - Assertions.assertEquals("h", model.identities().get(0)); + Assertions.assertEquals("ajionpimexgstxg", model.groups().get(0)); + Assertions.assertEquals("wzrlovmclwhij", model.identities().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppInsightsConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppInsightsConfigurationTests.java new file mode 100644 index 0000000000000..496c1ba150756 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppInsightsConfigurationTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.AppInsightsConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class AppInsightsConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + AppInsightsConfiguration model = BinaryData.fromString("{\"connectionString\":\"vgsgzwywakoihkn\"}") + .toObject(AppInsightsConfiguration.class); + Assertions.assertEquals("vgsgzwywakoihkn", model.connectionString()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + AppInsightsConfiguration model = new AppInsightsConfiguration().withConnectionString("vgsgzwywakoihkn"); + model = BinaryData.fromObject(model).toObject(AppInsightsConfiguration.class); + Assertions.assertEquals("vgsgzwywakoihkn", model.connectionString()); + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppResilienciesDeleteWithResponseMockTests.java similarity index 82% rename from sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteWithResponseMockTests.java rename to sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppResilienciesDeleteWithResponseMockTests.java index 1bbdb598438da..f694ba25e6411 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AppResilienciesDeleteWithResponseMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.generated; +package com.azure.resourcemanager.appcontainers.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -11,7 +11,7 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.communication.CommunicationManager; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -21,7 +21,7 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SuppressionListsDeleteWithResponseMockTests { +public final class AppResilienciesDeleteWithResponseMockTests { @Test public void testDeleteWithResponse() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); @@ -41,11 +41,11 @@ public void testDeleteWithResponse() throws Exception { return Mono.just(httpResponse); })); - CommunicationManager manager = CommunicationManager.configure().withHttpClient(httpClient).authenticate( + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.suppressionLists().deleteWithResponse("ubeddg", "sofwqmzqalkrmnji", "pxacqqudfn", "yxbaaabjyvayf", + manager.appResiliencies().deleteWithResponse("fulopmjnlexwhcb", "pibkephuu", "erctatoyin", com.azure.core.util.Context.NONE); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthConfigCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthConfigCollectionTests.java deleted file mode 100644 index 3f774f15d4666..0000000000000 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthConfigCollectionTests.java +++ /dev/null @@ -1,364 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.appcontainers.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.appcontainers.fluent.models.AuthConfigInner; -import com.azure.resourcemanager.appcontainers.models.AllowedAudiencesValidation; -import com.azure.resourcemanager.appcontainers.models.AppRegistration; -import com.azure.resourcemanager.appcontainers.models.Apple; -import com.azure.resourcemanager.appcontainers.models.AppleRegistration; -import com.azure.resourcemanager.appcontainers.models.AuthConfigCollection; -import com.azure.resourcemanager.appcontainers.models.AuthPlatform; -import com.azure.resourcemanager.appcontainers.models.AzureActiveDirectory; -import com.azure.resourcemanager.appcontainers.models.AzureActiveDirectoryLogin; -import com.azure.resourcemanager.appcontainers.models.AzureActiveDirectoryRegistration; -import com.azure.resourcemanager.appcontainers.models.AzureActiveDirectoryValidation; -import com.azure.resourcemanager.appcontainers.models.AzureStaticWebApps; -import com.azure.resourcemanager.appcontainers.models.AzureStaticWebAppsRegistration; -import com.azure.resourcemanager.appcontainers.models.ClientRegistration; -import com.azure.resourcemanager.appcontainers.models.CookieExpiration; -import com.azure.resourcemanager.appcontainers.models.CookieExpirationConvention; -import com.azure.resourcemanager.appcontainers.models.CustomOpenIdConnectProvider; -import com.azure.resourcemanager.appcontainers.models.Facebook; -import com.azure.resourcemanager.appcontainers.models.ForwardProxy; -import com.azure.resourcemanager.appcontainers.models.ForwardProxyConvention; -import com.azure.resourcemanager.appcontainers.models.GitHub; -import com.azure.resourcemanager.appcontainers.models.GlobalValidation; -import com.azure.resourcemanager.appcontainers.models.Google; -import com.azure.resourcemanager.appcontainers.models.HttpSettings; -import com.azure.resourcemanager.appcontainers.models.HttpSettingsRoutes; -import com.azure.resourcemanager.appcontainers.models.IdentityProviders; -import com.azure.resourcemanager.appcontainers.models.Login; -import com.azure.resourcemanager.appcontainers.models.LoginRoutes; -import com.azure.resourcemanager.appcontainers.models.LoginScopes; -import com.azure.resourcemanager.appcontainers.models.Nonce; -import com.azure.resourcemanager.appcontainers.models.Twitter; -import com.azure.resourcemanager.appcontainers.models.TwitterRegistration; -import com.azure.resourcemanager.appcontainers.models.UnauthenticatedClientActionV2; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class AuthConfigCollectionTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - AuthConfigCollection model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"platform\":{\"enabled\":false,\"runtimeVersion\":\"gjxpybczmehmt\"},\"globalValidation\":{\"unauthenticatedClientAction\":\"Return403\",\"redirectToProvider\":\"phrupidgsybbejhp\",\"excludedPaths\":[\"cmsxaobhdxbm\"]},\"identityProviders\":{\"azureActiveDirectory\":{\"enabled\":false,\"registration\":{},\"login\":{},\"validation\":{},\"isAutoProvisioned\":false},\"facebook\":{\"enabled\":false,\"registration\":{},\"graphApiVersion\":\"ownoizhw\",\"login\":{}},\"gitHub\":{\"enabled\":false,\"registration\":{},\"login\":{}},\"google\":{\"enabled\":false,\"registration\":{},\"login\":{},\"validation\":{}},\"twitter\":{\"enabled\":true,\"registration\":{}},\"apple\":{\"enabled\":true,\"registration\":{},\"login\":{}},\"azureStaticWebApps\":{\"enabled\":false,\"registration\":{}},\"customOpenIdConnectProviders\":{\"nrbtcqqjnq\":{}}},\"login\":{\"routes\":{\"logoutEndpoint\":\"gnufoooj\"},\"preserveUrlFragmentsForLogins\":true,\"allowedExternalRedirectUrls\":[\"qes\"],\"cookieExpiration\":{\"convention\":\"IdentityProviderDerived\",\"timeToExpiration\":\"mglzlhj\"},\"nonce\":{\"validateNonce\":false,\"nonceExpirationInterval\":\"wmrvktsizntocipa\"}},\"httpSettings\":{\"requireHttps\":false,\"routes\":{\"apiPrefix\":\"qucmpo\"},\"forwardProxy\":{\"convention\":\"Standard\",\"customHostHeaderName\":\"ogknygjofjdd\",\"customProtoHeaderName\":\"s\"}}},\"id\":\"eupewnwreitjz\",\"name\":\"flusarhmof\",\"type\":\"qhsmyurkdtml\"},{\"properties\":{\"platform\":{\"enabled\":true,\"runtimeVersion\":\"sjtxukcdmp\"},\"globalValidation\":{\"unauthenticatedClientAction\":\"Return403\",\"redirectToProvider\":\"uanzwuxzdx\",\"excludedPaths\":[\"rlhm\",\"hfpmrqobmtukknr\",\"rtihfxtijbpz\",\"gnwzsymglzufc\"]},\"identityProviders\":{\"azureActiveDirectory\":{\"enabled\":false,\"registration\":{},\"login\":{},\"validation\":{},\"isAutoProvisioned\":true},\"facebook\":{\"enabled\":false,\"registration\":{},\"graphApiVersion\":\"cbjy\",\"login\":{}},\"gitHub\":{\"enabled\":false,\"registration\":{},\"login\":{}},\"google\":{\"enabled\":true,\"registration\":{},\"login\":{},\"validation\":{}},\"twitter\":{\"enabled\":true,\"registration\":{}},\"apple\":{\"enabled\":false,\"registration\":{},\"login\":{}},\"azureStaticWebApps\":{\"enabled\":false,\"registration\":{}},\"customOpenIdConnectProviders\":{\"uhivyqniw\":{}}},\"login\":{\"routes\":{\"logoutEndpoint\":\"k\"},\"preserveUrlFragmentsForLogins\":true,\"allowedExternalRedirectUrls\":[\"jgrtfwvukxga\"],\"cookieExpiration\":{\"convention\":\"FixedTime\",\"timeToExpiration\":\"nhsjcnyej\"},\"nonce\":{\"validateNonce\":true,\"nonceExpirationInterval\":\"tnapczwlokjyemkk\"}},\"httpSettings\":{\"requireHttps\":true,\"routes\":{\"apiPrefix\":\"xzjnchgejs\"},\"forwardProxy\":{\"convention\":\"Custom\",\"customHostHeaderName\":\"ilzyd\",\"customProtoHeaderName\":\"o\"}}},\"id\":\"yahux\",\"name\":\"npmqnjaqwixjspro\",\"type\":\"vcputegj\"},{\"properties\":{\"platform\":{\"enabled\":false,\"runtimeVersion\":\"t\"},\"globalValidation\":{\"unauthenticatedClientAction\":\"Return403\",\"redirectToProvider\":\"pjhulsuuvmkj\",\"excludedPaths\":[\"rwfndiod\"]},\"identityProviders\":{\"azureActiveDirectory\":{\"enabled\":false,\"registration\":{},\"login\":{},\"validation\":{},\"isAutoProvisioned\":true},\"facebook\":{\"enabled\":true,\"registration\":{},\"graphApiVersion\":\"psoacctazakljl\",\"login\":{}},\"gitHub\":{\"enabled\":true,\"registration\":{},\"login\":{}},\"google\":{\"enabled\":false,\"registration\":{},\"login\":{},\"validation\":{}},\"twitter\":{\"enabled\":false,\"registration\":{}},\"apple\":{\"enabled\":true,\"registration\":{},\"login\":{}},\"azureStaticWebApps\":{\"enabled\":true,\"registration\":{}},\"customOpenIdConnectProviders\":{\"jhcrz\":{},\"vdphlxaolthqtr\":{},\"qjbpfzfsin\":{},\"gvfcj\":{}}},\"login\":{\"routes\":{\"logoutEndpoint\":\"xxjtfe\"},\"preserveUrlFragmentsForLogins\":true,\"allowedExternalRedirectUrls\":[\"zitonpeqfpjkjl\",\"ofpdvh\"],\"cookieExpiration\":{\"convention\":\"IdentityProviderDerived\",\"timeToExpiration\":\"pini\"},\"nonce\":{\"validateNonce\":true,\"nonceExpirationInterval\":\"uybbkpodep\"}},\"httpSettings\":{\"requireHttps\":true,\"routes\":{\"apiPrefix\":\"vamih\"},\"forwardProxy\":{\"convention\":\"Custom\",\"customHostHeaderName\":\"rxzxtheo\",\"customProtoHeaderName\":\"si\"}}},\"id\":\"evcciqihnhun\",\"name\":\"bwjzr\",\"type\":\"fygxgispemvtzfk\"}],\"nextLink\":\"ubljofxqe\"}") - .toObject(AuthConfigCollection.class); - Assertions.assertEquals(false, model.value().get(0).platform().enabled()); - Assertions.assertEquals("gjxpybczmehmt", model.value().get(0).platform().runtimeVersion()); - Assertions - .assertEquals( - UnauthenticatedClientActionV2.RETURN403, - model.value().get(0).globalValidation().unauthenticatedClientAction()); - Assertions.assertEquals("phrupidgsybbejhp", model.value().get(0).globalValidation().redirectToProvider()); - Assertions.assertEquals("cmsxaobhdxbm", model.value().get(0).globalValidation().excludedPaths().get(0)); - Assertions.assertEquals(false, model.value().get(0).identityProviders().azureActiveDirectory().enabled()); - Assertions - .assertEquals(false, model.value().get(0).identityProviders().azureActiveDirectory().isAutoProvisioned()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().facebook().enabled()); - Assertions.assertEquals("ownoizhw", model.value().get(0).identityProviders().facebook().graphApiVersion()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().gitHub().enabled()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().google().enabled()); - Assertions.assertEquals(true, model.value().get(0).identityProviders().twitter().enabled()); - Assertions.assertEquals(true, model.value().get(0).identityProviders().apple().enabled()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().azureStaticWebApps().enabled()); - Assertions.assertEquals("gnufoooj", model.value().get(0).login().routes().logoutEndpoint()); - Assertions.assertEquals(true, model.value().get(0).login().preserveUrlFragmentsForLogins()); - Assertions.assertEquals("qes", model.value().get(0).login().allowedExternalRedirectUrls().get(0)); - Assertions - .assertEquals( - CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED, - model.value().get(0).login().cookieExpiration().convention()); - Assertions.assertEquals("mglzlhj", model.value().get(0).login().cookieExpiration().timeToExpiration()); - Assertions.assertEquals(false, model.value().get(0).login().nonce().validateNonce()); - Assertions.assertEquals("wmrvktsizntocipa", model.value().get(0).login().nonce().nonceExpirationInterval()); - Assertions.assertEquals(false, model.value().get(0).httpSettings().requireHttps()); - Assertions.assertEquals("qucmpo", model.value().get(0).httpSettings().routes().apiPrefix()); - Assertions - .assertEquals( - ForwardProxyConvention.STANDARD, model.value().get(0).httpSettings().forwardProxy().convention()); - Assertions - .assertEquals("ogknygjofjdd", model.value().get(0).httpSettings().forwardProxy().customHostHeaderName()); - Assertions.assertEquals("s", model.value().get(0).httpSettings().forwardProxy().customProtoHeaderName()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - AuthConfigCollection model = - new AuthConfigCollection() - .withValue( - Arrays - .asList( - new AuthConfigInner() - .withPlatform(new AuthPlatform().withEnabled(false).withRuntimeVersion("gjxpybczmehmt")) - .withGlobalValidation( - new GlobalValidation() - .withUnauthenticatedClientAction(UnauthenticatedClientActionV2.RETURN403) - .withRedirectToProvider("phrupidgsybbejhp") - .withExcludedPaths(Arrays.asList("cmsxaobhdxbm"))) - .withIdentityProviders( - new IdentityProviders() - .withAzureActiveDirectory( - new AzureActiveDirectory() - .withEnabled(false) - .withRegistration(new AzureActiveDirectoryRegistration()) - .withLogin(new AzureActiveDirectoryLogin()) - .withValidation(new AzureActiveDirectoryValidation()) - .withIsAutoProvisioned(false)) - .withFacebook( - new Facebook() - .withEnabled(false) - .withRegistration(new AppRegistration()) - .withGraphApiVersion("ownoizhw") - .withLogin(new LoginScopes())) - .withGitHub( - new GitHub() - .withEnabled(false) - .withRegistration(new ClientRegistration()) - .withLogin(new LoginScopes())) - .withGoogle( - new Google() - .withEnabled(false) - .withRegistration(new ClientRegistration()) - .withLogin(new LoginScopes()) - .withValidation(new AllowedAudiencesValidation())) - .withTwitter( - new Twitter().withEnabled(true).withRegistration(new TwitterRegistration())) - .withApple( - new Apple() - .withEnabled(true) - .withRegistration(new AppleRegistration()) - .withLogin(new LoginScopes())) - .withAzureStaticWebApps( - new AzureStaticWebApps() - .withEnabled(false) - .withRegistration(new AzureStaticWebAppsRegistration())) - .withCustomOpenIdConnectProviders( - mapOf("nrbtcqqjnq", new CustomOpenIdConnectProvider()))) - .withLogin( - new Login() - .withRoutes(new LoginRoutes().withLogoutEndpoint("gnufoooj")) - .withPreserveUrlFragmentsForLogins(true) - .withAllowedExternalRedirectUrls(Arrays.asList("qes")) - .withCookieExpiration( - new CookieExpiration() - .withConvention(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED) - .withTimeToExpiration("mglzlhj")) - .withNonce( - new Nonce() - .withValidateNonce(false) - .withNonceExpirationInterval("wmrvktsizntocipa"))) - .withHttpSettings( - new HttpSettings() - .withRequireHttps(false) - .withRoutes(new HttpSettingsRoutes().withApiPrefix("qucmpo")) - .withForwardProxy( - new ForwardProxy() - .withConvention(ForwardProxyConvention.STANDARD) - .withCustomHostHeaderName("ogknygjofjdd") - .withCustomProtoHeaderName("s"))), - new AuthConfigInner() - .withPlatform(new AuthPlatform().withEnabled(true).withRuntimeVersion("sjtxukcdmp")) - .withGlobalValidation( - new GlobalValidation() - .withUnauthenticatedClientAction(UnauthenticatedClientActionV2.RETURN403) - .withRedirectToProvider("uanzwuxzdx") - .withExcludedPaths( - Arrays.asList("rlhm", "hfpmrqobmtukknr", "rtihfxtijbpz", "gnwzsymglzufc"))) - .withIdentityProviders( - new IdentityProviders() - .withAzureActiveDirectory( - new AzureActiveDirectory() - .withEnabled(false) - .withRegistration(new AzureActiveDirectoryRegistration()) - .withLogin(new AzureActiveDirectoryLogin()) - .withValidation(new AzureActiveDirectoryValidation()) - .withIsAutoProvisioned(true)) - .withFacebook( - new Facebook() - .withEnabled(false) - .withRegistration(new AppRegistration()) - .withGraphApiVersion("cbjy") - .withLogin(new LoginScopes())) - .withGitHub( - new GitHub() - .withEnabled(false) - .withRegistration(new ClientRegistration()) - .withLogin(new LoginScopes())) - .withGoogle( - new Google() - .withEnabled(true) - .withRegistration(new ClientRegistration()) - .withLogin(new LoginScopes()) - .withValidation(new AllowedAudiencesValidation())) - .withTwitter( - new Twitter().withEnabled(true).withRegistration(new TwitterRegistration())) - .withApple( - new Apple() - .withEnabled(false) - .withRegistration(new AppleRegistration()) - .withLogin(new LoginScopes())) - .withAzureStaticWebApps( - new AzureStaticWebApps() - .withEnabled(false) - .withRegistration(new AzureStaticWebAppsRegistration())) - .withCustomOpenIdConnectProviders( - mapOf("uhivyqniw", new CustomOpenIdConnectProvider()))) - .withLogin( - new Login() - .withRoutes(new LoginRoutes().withLogoutEndpoint("k")) - .withPreserveUrlFragmentsForLogins(true) - .withAllowedExternalRedirectUrls(Arrays.asList("jgrtfwvukxga")) - .withCookieExpiration( - new CookieExpiration() - .withConvention(CookieExpirationConvention.FIXED_TIME) - .withTimeToExpiration("nhsjcnyej")) - .withNonce( - new Nonce() - .withValidateNonce(true) - .withNonceExpirationInterval("tnapczwlokjyemkk"))) - .withHttpSettings( - new HttpSettings() - .withRequireHttps(true) - .withRoutes(new HttpSettingsRoutes().withApiPrefix("xzjnchgejs")) - .withForwardProxy( - new ForwardProxy() - .withConvention(ForwardProxyConvention.CUSTOM) - .withCustomHostHeaderName("ilzyd") - .withCustomProtoHeaderName("o"))), - new AuthConfigInner() - .withPlatform(new AuthPlatform().withEnabled(false).withRuntimeVersion("t")) - .withGlobalValidation( - new GlobalValidation() - .withUnauthenticatedClientAction(UnauthenticatedClientActionV2.RETURN403) - .withRedirectToProvider("pjhulsuuvmkj") - .withExcludedPaths(Arrays.asList("rwfndiod"))) - .withIdentityProviders( - new IdentityProviders() - .withAzureActiveDirectory( - new AzureActiveDirectory() - .withEnabled(false) - .withRegistration(new AzureActiveDirectoryRegistration()) - .withLogin(new AzureActiveDirectoryLogin()) - .withValidation(new AzureActiveDirectoryValidation()) - .withIsAutoProvisioned(true)) - .withFacebook( - new Facebook() - .withEnabled(true) - .withRegistration(new AppRegistration()) - .withGraphApiVersion("psoacctazakljl") - .withLogin(new LoginScopes())) - .withGitHub( - new GitHub() - .withEnabled(true) - .withRegistration(new ClientRegistration()) - .withLogin(new LoginScopes())) - .withGoogle( - new Google() - .withEnabled(false) - .withRegistration(new ClientRegistration()) - .withLogin(new LoginScopes()) - .withValidation(new AllowedAudiencesValidation())) - .withTwitter( - new Twitter() - .withEnabled(false) - .withRegistration(new TwitterRegistration())) - .withApple( - new Apple() - .withEnabled(true) - .withRegistration(new AppleRegistration()) - .withLogin(new LoginScopes())) - .withAzureStaticWebApps( - new AzureStaticWebApps() - .withEnabled(true) - .withRegistration(new AzureStaticWebAppsRegistration())) - .withCustomOpenIdConnectProviders( - mapOf( - "jhcrz", - new CustomOpenIdConnectProvider(), - "vdphlxaolthqtr", - new CustomOpenIdConnectProvider(), - "qjbpfzfsin", - new CustomOpenIdConnectProvider(), - "gvfcj", - new CustomOpenIdConnectProvider()))) - .withLogin( - new Login() - .withRoutes(new LoginRoutes().withLogoutEndpoint("xxjtfe")) - .withPreserveUrlFragmentsForLogins(true) - .withAllowedExternalRedirectUrls(Arrays.asList("zitonpeqfpjkjl", "ofpdvh")) - .withCookieExpiration( - new CookieExpiration() - .withConvention(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED) - .withTimeToExpiration("pini")) - .withNonce( - new Nonce() - .withValidateNonce(true) - .withNonceExpirationInterval("uybbkpodep"))) - .withHttpSettings( - new HttpSettings() - .withRequireHttps(true) - .withRoutes(new HttpSettingsRoutes().withApiPrefix("vamih")) - .withForwardProxy( - new ForwardProxy() - .withConvention(ForwardProxyConvention.CUSTOM) - .withCustomHostHeaderName("rxzxtheo") - .withCustomProtoHeaderName("si"))))); - model = BinaryData.fromObject(model).toObject(AuthConfigCollection.class); - Assertions.assertEquals(false, model.value().get(0).platform().enabled()); - Assertions.assertEquals("gjxpybczmehmt", model.value().get(0).platform().runtimeVersion()); - Assertions - .assertEquals( - UnauthenticatedClientActionV2.RETURN403, - model.value().get(0).globalValidation().unauthenticatedClientAction()); - Assertions.assertEquals("phrupidgsybbejhp", model.value().get(0).globalValidation().redirectToProvider()); - Assertions.assertEquals("cmsxaobhdxbm", model.value().get(0).globalValidation().excludedPaths().get(0)); - Assertions.assertEquals(false, model.value().get(0).identityProviders().azureActiveDirectory().enabled()); - Assertions - .assertEquals(false, model.value().get(0).identityProviders().azureActiveDirectory().isAutoProvisioned()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().facebook().enabled()); - Assertions.assertEquals("ownoizhw", model.value().get(0).identityProviders().facebook().graphApiVersion()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().gitHub().enabled()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().google().enabled()); - Assertions.assertEquals(true, model.value().get(0).identityProviders().twitter().enabled()); - Assertions.assertEquals(true, model.value().get(0).identityProviders().apple().enabled()); - Assertions.assertEquals(false, model.value().get(0).identityProviders().azureStaticWebApps().enabled()); - Assertions.assertEquals("gnufoooj", model.value().get(0).login().routes().logoutEndpoint()); - Assertions.assertEquals(true, model.value().get(0).login().preserveUrlFragmentsForLogins()); - Assertions.assertEquals("qes", model.value().get(0).login().allowedExternalRedirectUrls().get(0)); - Assertions - .assertEquals( - CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED, - model.value().get(0).login().cookieExpiration().convention()); - Assertions.assertEquals("mglzlhj", model.value().get(0).login().cookieExpiration().timeToExpiration()); - Assertions.assertEquals(false, model.value().get(0).login().nonce().validateNonce()); - Assertions.assertEquals("wmrvktsizntocipa", model.value().get(0).login().nonce().nonceExpirationInterval()); - Assertions.assertEquals(false, model.value().get(0).httpSettings().requireHttps()); - Assertions.assertEquals("qucmpo", model.value().get(0).httpSettings().routes().apiPrefix()); - Assertions - .assertEquals( - ForwardProxyConvention.STANDARD, model.value().get(0).httpSettings().forwardProxy().convention()); - Assertions - .assertEquals("ogknygjofjdd", model.value().get(0).httpSettings().forwardProxy().customHostHeaderName()); - Assertions.assertEquals("s", model.value().get(0).httpSettings().forwardProxy().customProtoHeaderName()); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthPlatformTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthPlatformTests.java index bd6d1c94b9cd0..7a96afa744ab0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthPlatformTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AuthPlatformTests.java @@ -11,17 +11,17 @@ public final class AuthPlatformTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AuthPlatform model = - BinaryData.fromString("{\"enabled\":true,\"runtimeVersion\":\"dvkaozw\"}").toObject(AuthPlatform.class); + AuthPlatform model + = BinaryData.fromString("{\"enabled\":true,\"runtimeVersion\":\"jaoyfhrtx\"}").toObject(AuthPlatform.class); Assertions.assertEquals(true, model.enabled()); - Assertions.assertEquals("dvkaozw", model.runtimeVersion()); + Assertions.assertEquals("jaoyfhrtx", model.runtimeVersion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AuthPlatform model = new AuthPlatform().withEnabled(true).withRuntimeVersion("dvkaozw"); + AuthPlatform model = new AuthPlatform().withEnabled(true).withRuntimeVersion("jaoyfhrtx"); model = BinaryData.fromObject(model).toObject(AuthPlatform.class); Assertions.assertEquals(true, model.enabled()); - Assertions.assertEquals("dvkaozw", model.runtimeVersion()); + Assertions.assertEquals("jaoyfhrtx", model.runtimeVersion()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableOperationsTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableOperationsTests.java index 5f5385f1bf94e..140b8f8ffc45e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableOperationsTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableOperationsTests.java @@ -14,47 +14,44 @@ public final class AvailableOperationsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AvailableOperations model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"pfhoqcaaewdao\",\"isDataAction\":true,\"display\":{\"provider\":\"pjxxkzb\",\"resource\":\"sgeivsiy\",\"operation\":\"kdncj\",\"description\":\"onbzoggculapzwy\"},\"origin\":\"gogtqxepnylbf\"}],\"nextLink\":\"jlyjtlvofq\"}") - .toObject(AvailableOperations.class); - Assertions.assertEquals("pfhoqcaaewdao", model.value().get(0).name()); + AvailableOperations model = BinaryData.fromString( + "{\"value\":[{\"name\":\"rtltla\",\"isDataAction\":true,\"display\":{\"provider\":\"katbhjm\",\"resource\":\"nbsoqeqalarv\",\"operation\":\"gunbtgfebwlnbm\",\"description\":\"reeudzqavb\"},\"origin\":\"qmjxlyyzglgouwtl\"},{\"name\":\"jyuojqtobaxkjeyt\",\"isDataAction\":false,\"display\":{\"provider\":\"jkwrusnkq\",\"resource\":\"syrq\",\"operation\":\"jqhden\",\"description\":\"ulkpakd\"},\"origin\":\"fmjnnawtqa\"},{\"name\":\"xuckpggqoweyir\",\"isDataAction\":false,\"display\":{\"provider\":\"ngwflqqmpizruwn\",\"resource\":\"xpxiwfcngjs\",\"operation\":\"sii\",\"description\":\"mkzjvkviir\"},\"origin\":\"fgrwsdpgratzvz\"}],\"nextLink\":\"lbyvictctbrxkjzw\"}") + .toObject(AvailableOperations.class); + Assertions.assertEquals("rtltla", model.value().get(0).name()); Assertions.assertEquals(true, model.value().get(0).isDataAction()); - Assertions.assertEquals("pjxxkzb", model.value().get(0).display().provider()); - Assertions.assertEquals("sgeivsiy", model.value().get(0).display().resource()); - Assertions.assertEquals("kdncj", model.value().get(0).display().operation()); - Assertions.assertEquals("onbzoggculapzwy", model.value().get(0).display().description()); - Assertions.assertEquals("gogtqxepnylbf", model.value().get(0).origin()); - Assertions.assertEquals("jlyjtlvofq", model.nextLink()); + Assertions.assertEquals("katbhjm", model.value().get(0).display().provider()); + Assertions.assertEquals("nbsoqeqalarv", model.value().get(0).display().resource()); + Assertions.assertEquals("gunbtgfebwlnbm", model.value().get(0).display().operation()); + Assertions.assertEquals("reeudzqavb", model.value().get(0).display().description()); + Assertions.assertEquals("qmjxlyyzglgouwtl", model.value().get(0).origin()); + Assertions.assertEquals("lbyvictctbrxkjzw", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvailableOperations model = - new AvailableOperations() - .withValue( - Arrays - .asList( - new OperationDetailInner() - .withName("pfhoqcaaewdao") - .withIsDataAction(true) - .withDisplay( - new OperationDisplay() - .withProvider("pjxxkzb") - .withResource("sgeivsiy") - .withOperation("kdncj") - .withDescription("onbzoggculapzwy")) - .withOrigin("gogtqxepnylbf"))) - .withNextLink("jlyjtlvofq"); + AvailableOperations model = new AvailableOperations() + .withValue(Arrays.asList( + new OperationDetailInner().withName("rtltla").withIsDataAction(true) + .withDisplay(new OperationDisplay().withProvider("katbhjm").withResource("nbsoqeqalarv") + .withOperation("gunbtgfebwlnbm").withDescription("reeudzqavb")) + .withOrigin("qmjxlyyzglgouwtl"), + new OperationDetailInner().withName("jyuojqtobaxkjeyt").withIsDataAction(false) + .withDisplay(new OperationDisplay().withProvider("jkwrusnkq").withResource("syrq") + .withOperation("jqhden").withDescription("ulkpakd")) + .withOrigin("fmjnnawtqa"), + new OperationDetailInner().withName("xuckpggqoweyir").withIsDataAction(false) + .withDisplay(new OperationDisplay().withProvider("ngwflqqmpizruwn").withResource("xpxiwfcngjs") + .withOperation("sii").withDescription("mkzjvkviir")) + .withOrigin("fgrwsdpgratzvz"))) + .withNextLink("lbyvictctbrxkjzw"); model = BinaryData.fromObject(model).toObject(AvailableOperations.class); - Assertions.assertEquals("pfhoqcaaewdao", model.value().get(0).name()); + Assertions.assertEquals("rtltla", model.value().get(0).name()); Assertions.assertEquals(true, model.value().get(0).isDataAction()); - Assertions.assertEquals("pjxxkzb", model.value().get(0).display().provider()); - Assertions.assertEquals("sgeivsiy", model.value().get(0).display().resource()); - Assertions.assertEquals("kdncj", model.value().get(0).display().operation()); - Assertions.assertEquals("onbzoggculapzwy", model.value().get(0).display().description()); - Assertions.assertEquals("gogtqxepnylbf", model.value().get(0).origin()); - Assertions.assertEquals("jlyjtlvofq", model.nextLink()); + Assertions.assertEquals("katbhjm", model.value().get(0).display().provider()); + Assertions.assertEquals("nbsoqeqalarv", model.value().get(0).display().resource()); + Assertions.assertEquals("gunbtgfebwlnbm", model.value().get(0).display().operation()); + Assertions.assertEquals("reeudzqavb", model.value().get(0).display().description()); + Assertions.assertEquals("qmjxlyyzglgouwtl", model.value().get(0).origin()); + Assertions.assertEquals("lbyvictctbrxkjzw", model.nextLink()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfileInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfileInnerTests.java index 9cf8caa44ee7e..917ad9d8d6cea 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfileInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfileInnerTests.java @@ -13,37 +13,30 @@ public final class AvailableWorkloadProfileInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AvailableWorkloadProfileInner model = - BinaryData - .fromString( - "{\"location\":\"efovgmk\",\"properties\":{\"category\":\"eyyvxyqjpkcat\",\"applicability\":\"LocationDefault\",\"cores\":1698610201,\"memoryGiB\":1035421848,\"displayName\":\"czsqpjhvm\"},\"id\":\"ajvnysounqe\",\"name\":\"a\",\"type\":\"oaeupfhyhltrpmo\"}") - .toObject(AvailableWorkloadProfileInner.class); - Assertions.assertEquals("efovgmk", model.location()); - Assertions.assertEquals("eyyvxyqjpkcat", model.properties().category()); - Assertions.assertEquals(Applicability.LOCATION_DEFAULT, model.properties().applicability()); - Assertions.assertEquals(1698610201, model.properties().cores()); - Assertions.assertEquals(1035421848, model.properties().memoryGiB()); - Assertions.assertEquals("czsqpjhvm", model.properties().displayName()); + AvailableWorkloadProfileInner model = BinaryData.fromString( + "{\"location\":\"yocf\",\"properties\":{\"category\":\"s\",\"applicability\":\"Custom\",\"cores\":376493890,\"memoryGiB\":666664165,\"gpus\":30564685,\"displayName\":\"uxh\"},\"id\":\"udxorrqn\",\"name\":\"poczvyifqrvkdvjs\",\"type\":\"lrmv\"}") + .toObject(AvailableWorkloadProfileInner.class); + Assertions.assertEquals("yocf", model.location()); + Assertions.assertEquals("s", model.properties().category()); + Assertions.assertEquals(Applicability.CUSTOM, model.properties().applicability()); + Assertions.assertEquals(376493890, model.properties().cores()); + Assertions.assertEquals(666664165, model.properties().memoryGiB()); + Assertions.assertEquals(30564685, model.properties().gpus()); + Assertions.assertEquals("uxh", model.properties().displayName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvailableWorkloadProfileInner model = - new AvailableWorkloadProfileInner() - .withLocation("efovgmk") - .withProperties( - new AvailableWorkloadProfileProperties() - .withCategory("eyyvxyqjpkcat") - .withApplicability(Applicability.LOCATION_DEFAULT) - .withCores(1698610201) - .withMemoryGiB(1035421848) - .withDisplayName("czsqpjhvm")); + AvailableWorkloadProfileInner model = new AvailableWorkloadProfileInner().withLocation("yocf").withProperties( + new AvailableWorkloadProfileProperties().withCategory("s").withApplicability(Applicability.CUSTOM) + .withCores(376493890).withMemoryGiB(666664165).withGpus(30564685).withDisplayName("uxh")); model = BinaryData.fromObject(model).toObject(AvailableWorkloadProfileInner.class); - Assertions.assertEquals("efovgmk", model.location()); - Assertions.assertEquals("eyyvxyqjpkcat", model.properties().category()); - Assertions.assertEquals(Applicability.LOCATION_DEFAULT, model.properties().applicability()); - Assertions.assertEquals(1698610201, model.properties().cores()); - Assertions.assertEquals(1035421848, model.properties().memoryGiB()); - Assertions.assertEquals("czsqpjhvm", model.properties().displayName()); + Assertions.assertEquals("yocf", model.location()); + Assertions.assertEquals("s", model.properties().category()); + Assertions.assertEquals(Applicability.CUSTOM, model.properties().applicability()); + Assertions.assertEquals(376493890, model.properties().cores()); + Assertions.assertEquals(666664165, model.properties().memoryGiB()); + Assertions.assertEquals(30564685, model.properties().gpus()); + Assertions.assertEquals("uxh", model.properties().displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilePropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilePropertiesTests.java index 41f6a1b78e406..615d01b41854a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilePropertiesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilePropertiesTests.java @@ -12,32 +12,28 @@ public final class AvailableWorkloadProfilePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AvailableWorkloadProfileProperties model = - BinaryData - .fromString( - "{\"category\":\"mcmatuokthfuiu\",\"applicability\":\"Custom\",\"cores\":680550088,\"memoryGiB\":397485144,\"displayName\":\"vxodpu\"}") - .toObject(AvailableWorkloadProfileProperties.class); - Assertions.assertEquals("mcmatuokthfuiu", model.category()); - Assertions.assertEquals(Applicability.CUSTOM, model.applicability()); - Assertions.assertEquals(680550088, model.cores()); - Assertions.assertEquals(397485144, model.memoryGiB()); - Assertions.assertEquals("vxodpu", model.displayName()); + AvailableWorkloadProfileProperties model = BinaryData.fromString( + "{\"category\":\"f\",\"applicability\":\"LocationDefault\",\"cores\":1825333220,\"memoryGiB\":500448783,\"gpus\":608072417,\"displayName\":\"xxbczwtr\"}") + .toObject(AvailableWorkloadProfileProperties.class); + Assertions.assertEquals("f", model.category()); + Assertions.assertEquals(Applicability.LOCATION_DEFAULT, model.applicability()); + Assertions.assertEquals(1825333220, model.cores()); + Assertions.assertEquals(500448783, model.memoryGiB()); + Assertions.assertEquals(608072417, model.gpus()); + Assertions.assertEquals("xxbczwtr", model.displayName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvailableWorkloadProfileProperties model = - new AvailableWorkloadProfileProperties() - .withCategory("mcmatuokthfuiu") - .withApplicability(Applicability.CUSTOM) - .withCores(680550088) - .withMemoryGiB(397485144) - .withDisplayName("vxodpu"); + AvailableWorkloadProfileProperties model = new AvailableWorkloadProfileProperties().withCategory("f") + .withApplicability(Applicability.LOCATION_DEFAULT).withCores(1825333220).withMemoryGiB(500448783) + .withGpus(608072417).withDisplayName("xxbczwtr"); model = BinaryData.fromObject(model).toObject(AvailableWorkloadProfileProperties.class); - Assertions.assertEquals("mcmatuokthfuiu", model.category()); - Assertions.assertEquals(Applicability.CUSTOM, model.applicability()); - Assertions.assertEquals(680550088, model.cores()); - Assertions.assertEquals(397485144, model.memoryGiB()); - Assertions.assertEquals("vxodpu", model.displayName()); + Assertions.assertEquals("f", model.category()); + Assertions.assertEquals(Applicability.LOCATION_DEFAULT, model.applicability()); + Assertions.assertEquals(1825333220, model.cores()); + Assertions.assertEquals(500448783, model.memoryGiB()); + Assertions.assertEquals(608072417, model.gpus()); + Assertions.assertEquals("xxbczwtr", model.displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesCollectionTests.java index 1fb36c62a1ee2..fa4a65a0e272b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesCollectionTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesCollectionTests.java @@ -15,59 +15,40 @@ public final class AvailableWorkloadProfilesCollectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AvailableWorkloadProfilesCollection model = - BinaryData - .fromString( - "{\"value\":[{\"location\":\"jbdlwtgrhpdjpju\",\"properties\":{\"category\":\"xazjpqyegual\",\"applicability\":\"Custom\",\"cores\":1163750271,\"memoryGiB\":344964368,\"displayName\":\"zzvdudgwds\"},\"id\":\"fhotw\",\"name\":\"cynpwlbjnp\",\"type\":\"acfta\"},{\"location\":\"h\",\"properties\":{\"category\":\"tyfsoppusuesn\",\"applicability\":\"LocationDefault\",\"cores\":317621756,\"memoryGiB\":600821678,\"displayName\":\"orxzdmohctbqvud\"},\"id\":\"xdn\",\"name\":\"nvowgujju\",\"type\":\"wdkcglhsl\"},{\"location\":\"jdyggdtji\",\"properties\":{\"category\":\"kuofqweykhme\",\"applicability\":\"LocationDefault\",\"cores\":1463310275,\"memoryGiB\":669612849,\"displayName\":\"whybcib\"},\"id\":\"yvdcsitynnaa\",\"name\":\"dectehfiqsc\",\"type\":\"eypvhezrkg\"}],\"nextLink\":\"c\"}") - .toObject(AvailableWorkloadProfilesCollection.class); - Assertions.assertEquals("jbdlwtgrhpdjpju", model.value().get(0).location()); - Assertions.assertEquals("xazjpqyegual", model.value().get(0).properties().category()); - Assertions.assertEquals(Applicability.CUSTOM, model.value().get(0).properties().applicability()); - Assertions.assertEquals(1163750271, model.value().get(0).properties().cores()); - Assertions.assertEquals(344964368, model.value().get(0).properties().memoryGiB()); - Assertions.assertEquals("zzvdudgwds", model.value().get(0).properties().displayName()); + AvailableWorkloadProfilesCollection model = BinaryData.fromString( + "{\"value\":[{\"location\":\"dmoh\",\"properties\":{\"category\":\"qvudwxdndnvowgu\",\"applicability\":\"LocationDefault\",\"cores\":247911704,\"memoryGiB\":175400299,\"gpus\":2105007169,\"displayName\":\"hslazjdyggdtj\"},\"id\":\"hbkuofqwey\",\"name\":\"hmenevfyexfwhybc\",\"type\":\"bvyvdcsity\"},{\"location\":\"aamdect\",\"properties\":{\"category\":\"iqscjeypv\",\"applicability\":\"Custom\",\"cores\":902064602,\"memoryGiB\":177141497,\"gpus\":61185346,\"displayName\":\"refovgmkqsleyyvx\"},\"id\":\"jpkcattpng\",\"name\":\"cr\",\"type\":\"czsqpjhvm\"},{\"location\":\"jvnysounqe\",\"properties\":{\"category\":\"oaeupfhyhltrpmo\",\"applicability\":\"Custom\",\"cores\":1490971149,\"memoryGiB\":557848148,\"gpus\":860799201,\"displayName\":\"thfuiuaodsfcpkvx\"},\"id\":\"puozmyzydag\",\"name\":\"uaxbezyiuokkt\",\"type\":\"hrdxwzywqsmbs\"}],\"nextLink\":\"exim\"}") + .toObject(AvailableWorkloadProfilesCollection.class); + Assertions.assertEquals("dmoh", model.value().get(0).location()); + Assertions.assertEquals("qvudwxdndnvowgu", model.value().get(0).properties().category()); + Assertions.assertEquals(Applicability.LOCATION_DEFAULT, model.value().get(0).properties().applicability()); + Assertions.assertEquals(247911704, model.value().get(0).properties().cores()); + Assertions.assertEquals(175400299, model.value().get(0).properties().memoryGiB()); + Assertions.assertEquals(2105007169, model.value().get(0).properties().gpus()); + Assertions.assertEquals("hslazjdyggdtj", model.value().get(0).properties().displayName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvailableWorkloadProfilesCollection model = - new AvailableWorkloadProfilesCollection() - .withValue( - Arrays - .asList( - new AvailableWorkloadProfileInner() - .withLocation("jbdlwtgrhpdjpju") - .withProperties( - new AvailableWorkloadProfileProperties() - .withCategory("xazjpqyegual") - .withApplicability(Applicability.CUSTOM) - .withCores(1163750271) - .withMemoryGiB(344964368) - .withDisplayName("zzvdudgwds")), - new AvailableWorkloadProfileInner() - .withLocation("h") - .withProperties( - new AvailableWorkloadProfileProperties() - .withCategory("tyfsoppusuesn") - .withApplicability(Applicability.LOCATION_DEFAULT) - .withCores(317621756) - .withMemoryGiB(600821678) - .withDisplayName("orxzdmohctbqvud")), - new AvailableWorkloadProfileInner() - .withLocation("jdyggdtji") - .withProperties( - new AvailableWorkloadProfileProperties() - .withCategory("kuofqweykhme") - .withApplicability(Applicability.LOCATION_DEFAULT) - .withCores(1463310275) - .withMemoryGiB(669612849) - .withDisplayName("whybcib")))); + AvailableWorkloadProfilesCollection model = new AvailableWorkloadProfilesCollection().withValue(Arrays.asList( + new AvailableWorkloadProfileInner().withLocation("dmoh") + .withProperties(new AvailableWorkloadProfileProperties().withCategory("qvudwxdndnvowgu") + .withApplicability(Applicability.LOCATION_DEFAULT).withCores(247911704).withMemoryGiB(175400299) + .withGpus(2105007169).withDisplayName("hslazjdyggdtj")), + new AvailableWorkloadProfileInner().withLocation("aamdect") + .withProperties(new AvailableWorkloadProfileProperties().withCategory("iqscjeypv") + .withApplicability(Applicability.CUSTOM).withCores(902064602).withMemoryGiB(177141497) + .withGpus(61185346).withDisplayName("refovgmkqsleyyvx")), + new AvailableWorkloadProfileInner().withLocation("jvnysounqe") + .withProperties(new AvailableWorkloadProfileProperties().withCategory("oaeupfhyhltrpmo") + .withApplicability(Applicability.CUSTOM).withCores(1490971149).withMemoryGiB(557848148) + .withGpus(860799201).withDisplayName("thfuiuaodsfcpkvx")))); model = BinaryData.fromObject(model).toObject(AvailableWorkloadProfilesCollection.class); - Assertions.assertEquals("jbdlwtgrhpdjpju", model.value().get(0).location()); - Assertions.assertEquals("xazjpqyegual", model.value().get(0).properties().category()); - Assertions.assertEquals(Applicability.CUSTOM, model.value().get(0).properties().applicability()); - Assertions.assertEquals(1163750271, model.value().get(0).properties().cores()); - Assertions.assertEquals(344964368, model.value().get(0).properties().memoryGiB()); - Assertions.assertEquals("zzvdudgwds", model.value().get(0).properties().displayName()); + Assertions.assertEquals("dmoh", model.value().get(0).location()); + Assertions.assertEquals("qvudwxdndnvowgu", model.value().get(0).properties().category()); + Assertions.assertEquals(Applicability.LOCATION_DEFAULT, model.value().get(0).properties().applicability()); + Assertions.assertEquals(247911704, model.value().get(0).properties().cores()); + Assertions.assertEquals(175400299, model.value().get(0).properties().memoryGiB()); + Assertions.assertEquals(2105007169, model.value().get(0).properties().gpus()); + Assertions.assertEquals("hslazjdyggdtj", model.value().get(0).properties().displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetMockTests.java index 996c49a12b5b9..d709a86c04934 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AvailableWorkloadProfilesGetMockTests.java @@ -32,44 +32,33 @@ public void testGet() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"location\":\"uxtyasiibmi\",\"properties\":{\"category\":\"nustgnljh\",\"applicability\":\"LocationDefault\",\"cores\":1287166393,\"memoryGiB\":1773419326,\"displayName\":\"avmqfoudor\"},\"id\":\"cgyypro\",\"name\":\"wy\",\"type\":\"undmbx\"}]}"; + String responseStr + = "{\"value\":[{\"location\":\"uflgbhgauacdixm\",\"properties\":{\"category\":\"rs\",\"applicability\":\"Custom\",\"cores\":1835241804,\"memoryGiB\":1032355307,\"gpus\":1636214642,\"displayName\":\"zoeo\"},\"id\":\"vjhvefgwbm\",\"name\":\"jchnta\",\"type\":\"faymxbulpz\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.availableWorkloadProfiles().get("nuciqdsmexiit", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.availableWorkloadProfiles().get("hl", com.azure.core.util.Context.NONE); - Assertions.assertEquals("uxtyasiibmi", response.iterator().next().location()); - Assertions.assertEquals("nustgnljh", response.iterator().next().properties().category()); - Assertions - .assertEquals(Applicability.LOCATION_DEFAULT, response.iterator().next().properties().applicability()); - Assertions.assertEquals(1287166393, response.iterator().next().properties().cores()); - Assertions.assertEquals(1773419326, response.iterator().next().properties().memoryGiB()); - Assertions.assertEquals("avmqfoudor", response.iterator().next().properties().displayName()); + Assertions.assertEquals("uflgbhgauacdixm", response.iterator().next().location()); + Assertions.assertEquals("rs", response.iterator().next().properties().category()); + Assertions.assertEquals(Applicability.CUSTOM, response.iterator().next().properties().applicability()); + Assertions.assertEquals(1835241804, response.iterator().next().properties().cores()); + Assertions.assertEquals(1032355307, response.iterator().next().properties().memoryGiB()); + Assertions.assertEquals(1636214642, response.iterator().next().properties().gpus()); + Assertions.assertEquals("zoeo", response.iterator().next().properties().displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureActiveDirectoryLoginTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureActiveDirectoryLoginTests.java index b16765ab966f9..cb1420efd4970 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureActiveDirectoryLoginTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureActiveDirectoryLoginTests.java @@ -12,23 +12,19 @@ public final class AzureActiveDirectoryLoginTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzureActiveDirectoryLogin model = - BinaryData - .fromString( - "{\"loginParameters\":[\"kzsmodm\",\"lougpbkw\",\"mutduqktaps\"],\"disableWWWAuthenticate\":false}") - .toObject(AzureActiveDirectoryLogin.class); - Assertions.assertEquals("kzsmodm", model.loginParameters().get(0)); + AzureActiveDirectoryLogin model = BinaryData + .fromString("{\"loginParameters\":[\"isdkfthwxmnteiw\",\"opvkmijcm\"],\"disableWWWAuthenticate\":false}") + .toObject(AzureActiveDirectoryLogin.class); + Assertions.assertEquals("isdkfthwxmnteiw", model.loginParameters().get(0)); Assertions.assertEquals(false, model.disableWwwAuthenticate()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureActiveDirectoryLogin model = - new AzureActiveDirectoryLogin() - .withLoginParameters(Arrays.asList("kzsmodm", "lougpbkw", "mutduqktaps")) - .withDisableWwwAuthenticate(false); + AzureActiveDirectoryLogin model = new AzureActiveDirectoryLogin() + .withLoginParameters(Arrays.asList("isdkfthwxmnteiw", "opvkmijcm")).withDisableWwwAuthenticate(false); model = BinaryData.fromObject(model).toObject(AzureActiveDirectoryLogin.class); - Assertions.assertEquals("kzsmodm", model.loginParameters().get(0)); + Assertions.assertEquals("isdkfthwxmnteiw", model.loginParameters().get(0)); Assertions.assertEquals(false, model.disableWwwAuthenticate()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsRegistrationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsRegistrationTests.java index f4180d4eee4ca..070a4441d95d4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsRegistrationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsRegistrationTests.java @@ -11,15 +11,15 @@ public final class AzureStaticWebAppsRegistrationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzureStaticWebAppsRegistration model = - BinaryData.fromString("{\"clientId\":\"uhrhcffcyddgl\"}").toObject(AzureStaticWebAppsRegistration.class); - Assertions.assertEquals("uhrhcffcyddgl", model.clientId()); + AzureStaticWebAppsRegistration model + = BinaryData.fromString("{\"clientId\":\"wgndrvynhzgpp\"}").toObject(AzureStaticWebAppsRegistration.class); + Assertions.assertEquals("wgndrvynhzgpp", model.clientId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureStaticWebAppsRegistration model = new AzureStaticWebAppsRegistration().withClientId("uhrhcffcyddgl"); + AzureStaticWebAppsRegistration model = new AzureStaticWebAppsRegistration().withClientId("wgndrvynhzgpp"); model = BinaryData.fromObject(model).toObject(AzureStaticWebAppsRegistration.class); - Assertions.assertEquals("uhrhcffcyddgl", model.clientId()); + Assertions.assertEquals("wgndrvynhzgpp", model.clientId()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsTests.java index 82093c81aa911..796a36ccea52e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/AzureStaticWebAppsTests.java @@ -12,22 +12,19 @@ public final class AzureStaticWebAppsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzureStaticWebApps model = - BinaryData - .fromString("{\"enabled\":false,\"registration\":{\"clientId\":\"icbtwnpzao\"}}") + AzureStaticWebApps model + = BinaryData.fromString("{\"enabled\":true,\"registration\":{\"clientId\":\"ktzlcuiywg\"}}") .toObject(AzureStaticWebApps.class); - Assertions.assertEquals(false, model.enabled()); - Assertions.assertEquals("icbtwnpzao", model.registration().clientId()); + Assertions.assertEquals(true, model.enabled()); + Assertions.assertEquals("ktzlcuiywg", model.registration().clientId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureStaticWebApps model = - new AzureStaticWebApps() - .withEnabled(false) - .withRegistration(new AzureStaticWebAppsRegistration().withClientId("icbtwnpzao")); + AzureStaticWebApps model = new AzureStaticWebApps().withEnabled(true) + .withRegistration(new AzureStaticWebAppsRegistration().withClientId("ktzlcuiywg")); model = BinaryData.fromObject(model).toObject(AzureStaticWebApps.class); - Assertions.assertEquals(false, model.enabled()); - Assertions.assertEquals("icbtwnpzao", model.registration().clientId()); + Assertions.assertEquals(true, model.enabled()); + Assertions.assertEquals("ktzlcuiywg", model.registration().clientId()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterCollectionInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterCollectionInnerTests.java index 7cc2ffd7f104e..6a849d6d1681e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterCollectionInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterCollectionInnerTests.java @@ -14,49 +14,33 @@ public final class BillingMeterCollectionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - BillingMeterCollectionInner model = - BinaryData - .fromString( - "{\"value\":[{\"location\":\"yzydagfuaxbezyi\",\"properties\":{\"category\":\"ktwh\",\"meterType\":\"xw\",\"displayName\":\"wqsmbsur\"},\"id\":\"xim\",\"name\":\"ryocfsfksymdd\",\"type\":\"stkiiuxhqyud\"},{\"location\":\"rrqnbpoczvyifqrv\",\"properties\":{\"category\":\"jsllrmv\",\"meterType\":\"f\",\"displayName\":\"t\"},\"id\":\"pnpulexxbczwtru\",\"name\":\"iqzbq\",\"type\":\"vsovmyokac\"},{\"location\":\"kwlhzdo\",\"properties\":{\"category\":\"jmflbvvnch\",\"meterType\":\"cciw\",\"displayName\":\"juqk\"},\"id\":\"rsa\",\"name\":\"iwkuofos\",\"type\":\"ghsauuimjmvxied\"}]}") - .toObject(BillingMeterCollectionInner.class); - Assertions.assertEquals("yzydagfuaxbezyi", model.value().get(0).location()); - Assertions.assertEquals("ktwh", model.value().get(0).properties().category()); - Assertions.assertEquals("xw", model.value().get(0).properties().meterType()); - Assertions.assertEquals("wqsmbsur", model.value().get(0).properties().displayName()); + BillingMeterCollectionInner model = BinaryData.fromString( + "{\"value\":[{\"location\":\"qzbqjvsov\",\"properties\":{\"category\":\"kacspkw\",\"meterType\":\"zdobpxjmflbvvnch\",\"displayName\":\"cciw\"},\"id\":\"juqk\",\"name\":\"rsa\",\"type\":\"iwkuofos\"},{\"location\":\"hsauuimjmvxied\",\"properties\":{\"category\":\"idyjrrfbyaosvexc\",\"meterType\":\"npc\",\"displayName\":\"ocohslkevleg\"},\"id\":\"fbuhfmvfaxkffe\",\"name\":\"ithlvmezyvshxm\",\"type\":\"sbbzo\"},{\"location\":\"igrxwburvjxxjn\",\"properties\":{\"category\":\"dptkoenkouk\",\"meterType\":\"udwtiukbl\",\"displayName\":\"gkpocipazyxoe\"},\"id\":\"kgjn\",\"name\":\"iucgygevqzn\",\"type\":\"yp\"},{\"location\":\"bpizcdrqjsdpydn\",\"properties\":{\"category\":\"xdeoejzic\",\"meterType\":\"fsj\",\"displayName\":\"gzfbishcbk\"},\"id\":\"jdeyeamdpha\",\"name\":\"alpbuxwgipwhon\",\"type\":\"wkgshwa\"}]}") + .toObject(BillingMeterCollectionInner.class); + Assertions.assertEquals("qzbqjvsov", model.value().get(0).location()); + Assertions.assertEquals("kacspkw", model.value().get(0).properties().category()); + Assertions.assertEquals("zdobpxjmflbvvnch", model.value().get(0).properties().meterType()); + Assertions.assertEquals("cciw", model.value().get(0).properties().displayName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BillingMeterCollectionInner model = - new BillingMeterCollectionInner() - .withValue( - Arrays - .asList( - new BillingMeter() - .withLocation("yzydagfuaxbezyi") - .withProperties( - new BillingMeterProperties() - .withCategory("ktwh") - .withMeterType("xw") - .withDisplayName("wqsmbsur")), - new BillingMeter() - .withLocation("rrqnbpoczvyifqrv") - .withProperties( - new BillingMeterProperties() - .withCategory("jsllrmv") - .withMeterType("f") - .withDisplayName("t")), - new BillingMeter() - .withLocation("kwlhzdo") - .withProperties( - new BillingMeterProperties() - .withCategory("jmflbvvnch") - .withMeterType("cciw") - .withDisplayName("juqk")))); + BillingMeterCollectionInner model = new BillingMeterCollectionInner().withValue(Arrays.asList( + new BillingMeter().withLocation("qzbqjvsov") + .withProperties(new BillingMeterProperties().withCategory("kacspkw").withMeterType("zdobpxjmflbvvnch") + .withDisplayName("cciw")), + new BillingMeter().withLocation("hsauuimjmvxied") + .withProperties(new BillingMeterProperties().withCategory("idyjrrfbyaosvexc").withMeterType("npc") + .withDisplayName("ocohslkevleg")), + new BillingMeter().withLocation("igrxwburvjxxjn") + .withProperties(new BillingMeterProperties().withCategory("dptkoenkouk").withMeterType("udwtiukbl") + .withDisplayName("gkpocipazyxoe")), + new BillingMeter().withLocation("bpizcdrqjsdpydn").withProperties(new BillingMeterProperties() + .withCategory("xdeoejzic").withMeterType("fsj").withDisplayName("gzfbishcbk")))); model = BinaryData.fromObject(model).toObject(BillingMeterCollectionInner.class); - Assertions.assertEquals("yzydagfuaxbezyi", model.value().get(0).location()); - Assertions.assertEquals("ktwh", model.value().get(0).properties().category()); - Assertions.assertEquals("xw", model.value().get(0).properties().meterType()); - Assertions.assertEquals("wqsmbsur", model.value().get(0).properties().displayName()); + Assertions.assertEquals("qzbqjvsov", model.value().get(0).location()); + Assertions.assertEquals("kacspkw", model.value().get(0).properties().category()); + Assertions.assertEquals("zdobpxjmflbvvnch", model.value().get(0).properties().meterType()); + Assertions.assertEquals("cciw", model.value().get(0).properties().displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterPropertiesTests.java index dffbe371cd092..346ab94b64669 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterPropertiesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterPropertiesTests.java @@ -11,22 +11,21 @@ public final class BillingMeterPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - BillingMeterProperties model = - BinaryData - .fromString("{\"category\":\"ur\",\"meterType\":\"xxjnspydptk\",\"displayName\":\"nkoukn\"}") - .toObject(BillingMeterProperties.class); - Assertions.assertEquals("ur", model.category()); - Assertions.assertEquals("xxjnspydptk", model.meterType()); - Assertions.assertEquals("nkoukn", model.displayName()); + BillingMeterProperties model = BinaryData + .fromString("{\"category\":\"fkbey\",\"meterType\":\"wrmjmwvvjektc\",\"displayName\":\"enhwlrs\"}") + .toObject(BillingMeterProperties.class); + Assertions.assertEquals("fkbey", model.category()); + Assertions.assertEquals("wrmjmwvvjektc", model.meterType()); + Assertions.assertEquals("enhwlrs", model.displayName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BillingMeterProperties model = - new BillingMeterProperties().withCategory("ur").withMeterType("xxjnspydptk").withDisplayName("nkoukn"); + BillingMeterProperties model = new BillingMeterProperties().withCategory("fkbey").withMeterType("wrmjmwvvjektc") + .withDisplayName("enhwlrs"); model = BinaryData.fromObject(model).toObject(BillingMeterProperties.class); - Assertions.assertEquals("ur", model.category()); - Assertions.assertEquals("xxjnspydptk", model.meterType()); - Assertions.assertEquals("nkoukn", model.displayName()); + Assertions.assertEquals("fkbey", model.category()); + Assertions.assertEquals("wrmjmwvvjektc", model.meterType()); + Assertions.assertEquals("enhwlrs", model.displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterTests.java index 79cbea3b4c3f1..bfd70b49381db 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMeterTests.java @@ -12,31 +12,24 @@ public final class BillingMeterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - BillingMeter model = - BinaryData - .fromString( - "{\"location\":\"gidyjrrf\",\"properties\":{\"category\":\"osvexcsonpclhoc\",\"meterType\":\"slkevle\",\"displayName\":\"zfbuhf\"},\"id\":\"vfaxkffeiith\",\"name\":\"vmezy\",\"type\":\"shxmzsbbzoggigrx\"}") - .toObject(BillingMeter.class); - Assertions.assertEquals("gidyjrrf", model.location()); - Assertions.assertEquals("osvexcsonpclhoc", model.properties().category()); - Assertions.assertEquals("slkevle", model.properties().meterType()); - Assertions.assertEquals("zfbuhf", model.properties().displayName()); + BillingMeter model = BinaryData.fromString( + "{\"location\":\"ixzbinjeputtmryw\",\"properties\":{\"category\":\"oqftiyqzrnkcq\",\"meterType\":\"xlwhzlsicoh\",\"displayName\":\"qnwvlrya\"},\"id\":\"hheunmmqhgyx\",\"name\":\"konocu\",\"type\":\"oklyaxuconuq\"}") + .toObject(BillingMeter.class); + Assertions.assertEquals("ixzbinjeputtmryw", model.location()); + Assertions.assertEquals("oqftiyqzrnkcq", model.properties().category()); + Assertions.assertEquals("xlwhzlsicoh", model.properties().meterType()); + Assertions.assertEquals("qnwvlrya", model.properties().displayName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BillingMeter model = - new BillingMeter() - .withLocation("gidyjrrf") - .withProperties( - new BillingMeterProperties() - .withCategory("osvexcsonpclhoc") - .withMeterType("slkevle") - .withDisplayName("zfbuhf")); + BillingMeter model + = new BillingMeter().withLocation("ixzbinjeputtmryw").withProperties(new BillingMeterProperties() + .withCategory("oqftiyqzrnkcq").withMeterType("xlwhzlsicoh").withDisplayName("qnwvlrya")); model = BinaryData.fromObject(model).toObject(BillingMeter.class); - Assertions.assertEquals("gidyjrrf", model.location()); - Assertions.assertEquals("osvexcsonpclhoc", model.properties().category()); - Assertions.assertEquals("slkevle", model.properties().meterType()); - Assertions.assertEquals("zfbuhf", model.properties().displayName()); + Assertions.assertEquals("ixzbinjeputtmryw", model.location()); + Assertions.assertEquals("oqftiyqzrnkcq", model.properties().category()); + Assertions.assertEquals("xlwhzlsicoh", model.properties().meterType()); + Assertions.assertEquals("qnwvlrya", model.properties().displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetWithResponseMockTests.java index 67a8fd8e1b530..553b35b0c8873 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BillingMetersGetWithResponseMockTests.java @@ -30,41 +30,30 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"location\":\"pmdtz\",\"properties\":{\"category\":\"tfvnz\",\"meterType\":\"jtotpvopvpbd\",\"displayName\":\"qgqqihedsvqwthmk\"},\"id\":\"ibcysihsgqc\",\"name\":\"dhohsdtmcdzsuf\",\"type\":\"ohdxbzlmcmu\"},{\"location\":\"cvhd\",\"properties\":{\"category\":\"wqqxeysko\",\"meterType\":\"zinkfkbgbzbowxeq\",\"displayName\":\"ljmygvkzqkjjeokb\"},\"id\":\"efezrxcczurtlei\",\"name\":\"q\",\"type\":\"bkwvzg\"},{\"location\":\"v\",\"properties\":{\"category\":\"zdix\",\"meterType\":\"q\",\"displayName\":\"odawopqhewjptmcg\"},\"id\":\"bostzel\",\"name\":\"dlat\",\"type\":\"tmzlbiojlv\"},{\"location\":\"rbbpneqvcwwyy\",\"properties\":{\"category\":\"ochpprpr\",\"meterType\":\"mo\",\"displayName\":\"yzejnhlbk\"},\"id\":\"bzpcpiljhahzvec\",\"name\":\"ndbnwieh\",\"type\":\"lewjwiuubwef\"}]}"; + String responseStr + = "{\"value\":[{\"location\":\"suah\",\"properties\":{\"category\":\"xjcmmzrrsc\",\"meterType\":\"i\",\"displayName\":\"drnpxqwodiff\"},\"id\":\"cjrmmua\",\"name\":\"wibvjogj\",\"type\":\"nmc\"},{\"location\":\"foyzbamwineof\",\"properties\":{\"category\":\"akpoldtvevbo\",\"meterType\":\"zhzjknyuxg\",\"displayName\":\"txpnrupza\"},\"id\":\"rdixt\",\"name\":\"ekidswyskb\",\"type\":\"uffgllukkutvlx\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - BillingMeterCollection response = - manager.billingMeters().getWithResponse("ugcmjkavlgorb", com.azure.core.util.Context.NONE).getValue(); - - Assertions.assertEquals("pmdtz", response.value().get(0).location()); - Assertions.assertEquals("tfvnz", response.value().get(0).properties().category()); - Assertions.assertEquals("jtotpvopvpbd", response.value().get(0).properties().meterType()); - Assertions.assertEquals("qgqqihedsvqwthmk", response.value().get(0).properties().displayName()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + BillingMeterCollection response + = manager.billingMeters().getWithResponse("albmqkyojwyvfk", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("suah", response.value().get(0).location()); + Assertions.assertEquals("xjcmmzrrsc", response.value().get(0).properties().category()); + Assertions.assertEquals("i", response.value().get(0).properties().meterType()); + Assertions.assertEquals("drnpxqwodiff", response.value().get(0).properties().displayName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BlobStorageTokenStoreTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BlobStorageTokenStoreTests.java new file mode 100644 index 0000000000000..4ba0dc297ed01 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BlobStorageTokenStoreTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.BlobStorageTokenStore; +import org.junit.jupiter.api.Assertions; + +public final class BlobStorageTokenStoreTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BlobStorageTokenStore model + = BinaryData.fromString("{\"sasUrlSettingName\":\"keyyi\"}").toObject(BlobStorageTokenStore.class); + Assertions.assertEquals("keyyi", model.sasUrlSettingName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BlobStorageTokenStore model = new BlobStorageTokenStore().withSasUrlSettingName("keyyi"); + model = BinaryData.fromObject(model).toObject(BlobStorageTokenStore.class); + Assertions.assertEquals("keyyi", model.sasUrlSettingName()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildConfigurationTests.java new file mode 100644 index 0000000000000..3bd395011233a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildConfigurationTests.java @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.BuildConfiguration; +import com.azure.resourcemanager.appcontainers.models.EnvironmentVariable; +import com.azure.resourcemanager.appcontainers.models.HttpGet; +import com.azure.resourcemanager.appcontainers.models.PreBuildStep; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class BuildConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BuildConfiguration model = BinaryData.fromString( + "{\"baseOs\":\"qtkoievs\",\"platform\":\"tgqr\",\"platformVersion\":\"tmuwlauwzi\",\"environmentVariables\":[{\"name\":\"mpgcjefuzmuvpbt\",\"value\":\"d\"}],\"preBuildSteps\":[{\"description\":\"p\",\"scripts\":[\"bmnzbtbhjpgl\",\"fgohdneuelfphs\",\"yhtozfikdowwqu\",\"v\"],\"httpGet\":{\"url\":\"xclvit\",\"fileName\":\"qzonosggbhcohf\",\"headers\":[\"jnkaljutiiswacff\",\"dkzzewkfvhqcrail\"]}},{\"description\":\"n\",\"scripts\":[\"uflrwd\",\"hdlxyjrxsagafcn\",\"hgw\"],\"httpGet\":{\"url\":\"pnedgf\",\"fileName\":\"vkcvqvpkeqd\",\"headers\":[\"rhvoods\",\"tbobz\",\"opcjwvnhd\"]}},{\"description\":\"wmgxcxrsl\",\"scripts\":[\"twuoegrpkhjwni\"],\"httpGet\":{\"url\":\"sluicpdggkzz\",\"fileName\":\"mbmpaxmodfvuefy\",\"headers\":[\"pfvmwyhrfou\",\"ft\"]}},{\"description\":\"kcpwiy\",\"scripts\":[\"tmnubexkpzksmon\",\"jmquxvypomgk\",\"pkwhojvpa\"],\"httpGet\":{\"url\":\"gxysmocmbqfqvm\",\"fileName\":\"xozap\",\"headers\":[\"lxprglyatddckcbc\",\"ejrjxgciqibrho\",\"xsdqrhzoymibmrqy\",\"bahwfl\"]}}]}") + .toObject(BuildConfiguration.class); + Assertions.assertEquals("qtkoievs", model.baseOs()); + Assertions.assertEquals("tgqr", model.platform()); + Assertions.assertEquals("tmuwlauwzi", model.platformVersion()); + Assertions.assertEquals("mpgcjefuzmuvpbt", model.environmentVariables().get(0).name()); + Assertions.assertEquals("d", model.environmentVariables().get(0).value()); + Assertions.assertEquals("p", model.preBuildSteps().get(0).description()); + Assertions.assertEquals("bmnzbtbhjpgl", model.preBuildSteps().get(0).scripts().get(0)); + Assertions.assertEquals("xclvit", model.preBuildSteps().get(0).httpGet().url()); + Assertions.assertEquals("qzonosggbhcohf", model.preBuildSteps().get(0).httpGet().fileName()); + Assertions.assertEquals("jnkaljutiiswacff", model.preBuildSteps().get(0).httpGet().headers().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BuildConfiguration model + = new BuildConfiguration().withBaseOs("qtkoievs").withPlatform("tgqr").withPlatformVersion("tmuwlauwzi") + .withEnvironmentVariables( + Arrays.asList(new EnvironmentVariable().withName("mpgcjefuzmuvpbt").withValue("d"))) + .withPreBuildSteps(Arrays.asList( + new PreBuildStep().withDescription("p") + .withScripts(Arrays.asList("bmnzbtbhjpgl", "fgohdneuelfphs", "yhtozfikdowwqu", "v")) + .withHttpGet(new HttpGet().withUrl("xclvit").withFileName("qzonosggbhcohf") + .withHeaders(Arrays.asList("jnkaljutiiswacff", "dkzzewkfvhqcrail"))), + new PreBuildStep().withDescription("n") + .withScripts(Arrays.asList("uflrwd", "hdlxyjrxsagafcn", "hgw")) + .withHttpGet(new HttpGet().withUrl("pnedgf").withFileName("vkcvqvpkeqd") + .withHeaders(Arrays.asList("rhvoods", "tbobz", "opcjwvnhd"))), + new PreBuildStep().withDescription("wmgxcxrsl").withScripts(Arrays.asList("twuoegrpkhjwni")) + .withHttpGet(new HttpGet().withUrl("sluicpdggkzz").withFileName("mbmpaxmodfvuefy") + .withHeaders(Arrays.asList("pfvmwyhrfou", "ft"))), + new PreBuildStep().withDescription("kcpwiy") + .withScripts(Arrays.asList("tmnubexkpzksmon", "jmquxvypomgk", "pkwhojvpa")) + .withHttpGet(new HttpGet().withUrl("gxysmocmbqfqvm").withFileName("xozap").withHeaders( + Arrays.asList("lxprglyatddckcbc", "ejrjxgciqibrho", "xsdqrhzoymibmrqy", "bahwfl"))))); + model = BinaryData.fromObject(model).toObject(BuildConfiguration.class); + Assertions.assertEquals("qtkoievs", model.baseOs()); + Assertions.assertEquals("tgqr", model.platform()); + Assertions.assertEquals("tmuwlauwzi", model.platformVersion()); + Assertions.assertEquals("mpgcjefuzmuvpbt", model.environmentVariables().get(0).name()); + Assertions.assertEquals("d", model.environmentVariables().get(0).value()); + Assertions.assertEquals("p", model.preBuildSteps().get(0).description()); + Assertions.assertEquals("bmnzbtbhjpgl", model.preBuildSteps().get(0).scripts().get(0)); + Assertions.assertEquals("xclvit", model.preBuildSteps().get(0).httpGet().url()); + Assertions.assertEquals("qzonosggbhcohf", model.preBuildSteps().get(0).httpGet().fileName()); + Assertions.assertEquals("jnkaljutiiswacff", model.preBuildSteps().get(0).httpGet().headers().get(0)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderCollectionTests.java new file mode 100644 index 0000000000000..19ee9362b5ac8 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderCollectionTests.java @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.azure.resourcemanager.appcontainers.models.BuilderCollection; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class BuilderCollectionTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BuilderCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"environmentId\":\"wvlqdqgb\",\"containerRegistries\":[{\"containerRegistryServer\":\"lihkaetcktvfc\",\"identityResourceId\":\"vf\"},{\"containerRegistryServer\":\"nkymuctqhjfbebrj\",\"identityResourceId\":\"xerf\"},{\"containerRegistryServer\":\"wutttxfvjrbi\",\"identityResourceId\":\"phxepcyvahf\"}]},\"identity\":{\"principalId\":\"847b4663-757a-41a8-a56a-bef3059e77aa\",\"tenantId\":\"df774d78-50b5-4a1d-adac-338d71865837\",\"type\":\"None\",\"userAssignedIdentities\":{\"uujqgidokgjljyo\":{\"principalId\":\"a7fee7c7-290a-4bae-94d9-fb07083e5965\",\"clientId\":\"0889cb62-fb77-4fdb-b1c8-2b173c8fc2be\"},\"cltbgsncghkjesz\":{\"principalId\":\"164d1a09-5a25-463b-b558-f2efc8364f6b\",\"clientId\":\"2ffaa11d-2317-4fec-86d8-1d93bf776f8a\"},\"ijhtxf\":{\"principalId\":\"3951829e-34d2-4e98-bf40-5a758a45d0e8\",\"clientId\":\"96279c01-3539-4642-84b5-ebccf5e05ff1\"},\"bfs\":{\"principalId\":\"728f80c3-ede8-4470-87e4-fe11de510296\",\"clientId\":\"8b1c2b8d-f2f2-4209-8f41-3f8c230b516b\"}}},\"location\":\"nehmpvecx\",\"tags\":{\"ukgri\":\"ebfqkkrbm\",\"fbxzpuzycisp\":\"flz\",\"y\":\"qzahmgkbrp\",\"rgvtqag\":\"hibnuqqkpika\"},\"id\":\"buynhijggm\",\"name\":\"bfs\",\"type\":\"arbu\"},{\"properties\":{\"provisioningState\":\"Succeeded\",\"environmentId\":\"pnazzm\",\"containerRegistries\":[{\"containerRegistryServer\":\"unmpxttd\",\"identityResourceId\":\"hrbnlankxmyskpbh\"},{\"containerRegistryServer\":\"nbtkcxywnytnr\",\"identityResourceId\":\"yn\"}]},\"identity\":{\"principalId\":\"99dac99b-6165-4414-8858-081afadffaee\",\"tenantId\":\"9e9e4442-9e55-454f-9121-dfd87c468189\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"zfcl\":{\"principalId\":\"97be5077-94a1-473d-baa0-da7633d0826c\",\"clientId\":\"2099c3f0-e4d9-420c-9d5b-85bace4a931d\"},\"xdbabphlwr\":{\"principalId\":\"f0472d36-d7b8-4ded-a1c3-7b4b6ccccb2a\",\"clientId\":\"e147659a-84e8-472b-80fd-0d82d0663c31\"}}},\"location\":\"fkts\",\"tags\":{\"nyyazttbtwwrqpue\":\"ucoc\",\"xibxujwbhqwalm\":\"ckzywbiexzfeyue\",\"ux\":\"zyoxaepdkzjan\",\"zt\":\"hdwbavxbniwdjs\"},\"id\":\"dbpgnxytxhp\",\"name\":\"xbzpfzab\",\"type\":\"lcuhxwtctyqiklb\"},{\"properties\":{\"provisioningState\":\"Creating\",\"environmentId\":\"lwzbhvgyugu\",\"containerRegistries\":[{\"containerRegistryServer\":\"mkfssxqukkfplgm\",\"identityResourceId\":\"sxnkjzkdeslpvlo\"},{\"containerRegistryServer\":\"wiyighxpkdw\",\"identityResourceId\":\"baiuebbaumny\"},{\"containerRegistryServer\":\"upedeojnabckhs\",\"identityResourceId\":\"txp\"}]},\"identity\":{\"principalId\":\"f84befd7-75fa-49a4-abee-cfbabdc853d1\",\"tenantId\":\"85643d46-bb48-41b8-ad99-9bc5fea1ab84\",\"type\":\"SystemAssigned,UserAssigned\",\"userAssignedIdentities\":{\"esap\":{\"principalId\":\"a98a3a00-f04d-4aec-bdad-24e8cea65ea8\",\"clientId\":\"5a3b38f1-76b5-4b45-ba9e-61ef317d599a\"},\"dqmh\":{\"principalId\":\"656b7cd5-d2b0-4edf-b482-37215064e168\",\"clientId\":\"c8bb6d30-8937-448a-97a2-c4ff7d398651\"},\"htldwk\":{\"principalId\":\"dd5064b4-3820-4423-834c-29b659c662b6\",\"clientId\":\"d75b71f7-777c-4b86-9fa1-82c28fab3b0d\"},\"uutkncw\":{\"principalId\":\"c059546f-c78c-47cf-9cfa-5cafa7a16def\",\"clientId\":\"7de0cf22-8535-493e-86f6-03a477389d01\"}}},\"location\":\"wsvlxotogtwrupqs\",\"tags\":{\"notyfjfcnjbkcn\":\"micykvceoveilo\"},\"id\":\"dhbt\",\"name\":\"kphywpnvjto\",\"type\":\"nermcl\"}],\"nextLink\":\"lphox\"}") + .toObject(BuilderCollection.class); + Assertions.assertEquals("nehmpvecx", model.value().get(0).location()); + Assertions.assertEquals("ebfqkkrbm", model.value().get(0).tags().get("ukgri")); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.value().get(0).identity().type()); + Assertions.assertEquals("wvlqdqgb", model.value().get(0).environmentId()); + Assertions.assertEquals("lihkaetcktvfc", + model.value().get(0).containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("vf", model.value().get(0).containerRegistries().get(0).identityResourceId()); + Assertions.assertEquals("lphox", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BuilderCollection model + = new BuilderCollection() + .withValue( + Arrays + .asList( + new BuilderResourceInner().withLocation("nehmpvecx") + .withTags(mapOf("ukgri", "ebfqkkrbm", "fbxzpuzycisp", "flz", "y", "qzahmgkbrp", + "rgvtqag", "hibnuqqkpika")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("uujqgidokgjljyo", new UserAssignedIdentity(), + "cltbgsncghkjesz", new UserAssignedIdentity(), "ijhtxf", + new UserAssignedIdentity(), "bfs", new UserAssignedIdentity()))) + .withEnvironmentId("wvlqdqgb") + .withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("lihkaetcktvfc") + .withIdentityResourceId("vf"), + new ContainerRegistry().withContainerRegistryServer("nkymuctqhjfbebrj") + .withIdentityResourceId("xerf"), + new ContainerRegistry().withContainerRegistryServer("wutttxfvjrbi") + .withIdentityResourceId("phxepcyvahf"))), + new BuilderResourceInner().withLocation("fkts") + .withTags(mapOf("nyyazttbtwwrqpue", "ucoc", "xibxujwbhqwalm", "ckzywbiexzfeyue", "ux", + "zyoxaepdkzjan", "zt", "hdwbavxbniwdjs")) + .withIdentity( + new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED) + .withUserAssignedIdentities(mapOf("zfcl", new UserAssignedIdentity(), + "xdbabphlwr", new UserAssignedIdentity()))) + .withEnvironmentId("pnazzm") + .withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("unmpxttd") + .withIdentityResourceId("hrbnlankxmyskpbh"), + new ContainerRegistry().withContainerRegistryServer("nbtkcxywnytnr") + .withIdentityResourceId("yn"))), + new BuilderResourceInner().withLocation("wsvlxotogtwrupqs") + .withTags(mapOf("notyfjfcnjbkcn", "micykvceoveilo")) + .withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("esap", new UserAssignedIdentity(), "dqmh", + new UserAssignedIdentity(), "htldwk", new UserAssignedIdentity(), "uutkncw", + new UserAssignedIdentity()))) + .withEnvironmentId("lwzbhvgyugu") + .withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("mkfssxqukkfplgm") + .withIdentityResourceId("sxnkjzkdeslpvlo"), + new ContainerRegistry().withContainerRegistryServer("wiyighxpkdw") + .withIdentityResourceId("baiuebbaumny"), + new ContainerRegistry().withContainerRegistryServer("upedeojnabckhs") + .withIdentityResourceId("txp"))))) + .withNextLink("lphox"); + model = BinaryData.fromObject(model).toObject(BuilderCollection.class); + Assertions.assertEquals("nehmpvecx", model.value().get(0).location()); + Assertions.assertEquals("ebfqkkrbm", model.value().get(0).tags().get("ukgri")); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.value().get(0).identity().type()); + Assertions.assertEquals("wvlqdqgb", model.value().get(0).environmentId()); + Assertions.assertEquals("lihkaetcktvfc", + model.value().get(0).containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("vf", model.value().get(0).containerRegistries().get(0).identityResourceId()); + Assertions.assertEquals("lphox", model.nextLink()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderPropertiesTests.java new file mode 100644 index 0000000000000..0005ed2ef5073 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderPropertiesTests.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderProperties; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class BuilderPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BuilderProperties model = BinaryData.fromString( + "{\"provisioningState\":\"Deleting\",\"environmentId\":\"b\",\"containerRegistries\":[{\"containerRegistryServer\":\"dawkzbali\",\"identityResourceId\":\"urqhaka\"},{\"containerRegistryServer\":\"hashsfwxosow\",\"identityResourceId\":\"xcug\"},{\"containerRegistryServer\":\"cjooxdjebwpucwwf\",\"identityResourceId\":\"ovbvmeueciv\"},{\"containerRegistryServer\":\"hzceuojgjrwjue\",\"identityResourceId\":\"otwmcdyt\"}]}") + .toObject(BuilderProperties.class); + Assertions.assertEquals("b", model.environmentId()); + Assertions.assertEquals("dawkzbali", model.containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("urqhaka", model.containerRegistries().get(0).identityResourceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BuilderProperties model = new BuilderProperties().withEnvironmentId("b").withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("dawkzbali").withIdentityResourceId("urqhaka"), + new ContainerRegistry().withContainerRegistryServer("hashsfwxosow").withIdentityResourceId("xcug"), + new ContainerRegistry().withContainerRegistryServer("cjooxdjebwpucwwf") + .withIdentityResourceId("ovbvmeueciv"), + new ContainerRegistry().withContainerRegistryServer("hzceuojgjrwjue").withIdentityResourceId("otwmcdyt"))); + model = BinaryData.fromObject(model).toObject(BuilderProperties.class); + Assertions.assertEquals("b", model.environmentId()); + Assertions.assertEquals("dawkzbali", model.containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("urqhaka", model.containerRegistries().get(0).identityResourceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceInnerTests.java new file mode 100644 index 0000000000000..80d64973eabaa --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceInnerTests.java @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceInner; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class BuilderResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BuilderResourceInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Canceled\",\"environmentId\":\"pabgyeps\",\"containerRegistries\":[{\"containerRegistryServer\":\"azqugxywpmueefj\",\"identityResourceId\":\"wfqkquj\"},{\"containerRegistryServer\":\"dsuyonobgla\",\"identityResourceId\":\"cq\"},{\"containerRegistryServer\":\"tcc\",\"identityResourceId\":\"g\"},{\"containerRegistryServer\":\"udxytlmoyrx\",\"identityResourceId\":\"wfudwpzntxhdzhl\"}]},\"identity\":{\"principalId\":\"d283a77e-dc68-4ae8-9a29-91c6c96dc2b0\",\"tenantId\":\"b5f6c785-2c4d-4e2f-a0e8-4d64408cfd27\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"lhrxsbkyvpyc\":{\"principalId\":\"2490eb9b-07eb-4e06-966e-9cf27322e46a\",\"clientId\":\"fa5994e2-52a1-4105-b133-71ed814de3d2\"},\"z\":{\"principalId\":\"fd046a24-727e-48e9-9aa4-f098807f38bb\",\"clientId\":\"ab7a4713-0fc0-41a8-9005-ac1ab2176387\"},\"kafkuwbcrnwbm\":{\"principalId\":\"306b92b0-9b5e-4a2f-90ff-9628fb77bc1c\",\"clientId\":\"63448741-c49e-4827-94f1-a0095ba114de\"}}},\"location\":\"hseyvju\",\"tags\":{\"kdeemaofmxagkvtm\":\"slhs\",\"ahaquh\":\"lmqkrhahvlj\",\"aex\":\"dhmdua\"},\"id\":\"pvfadmwsrcr\",\"name\":\"vxpvgomz\",\"type\":\"fmisg\"}") + .toObject(BuilderResourceInner.class); + Assertions.assertEquals("hseyvju", model.location()); + Assertions.assertEquals("slhs", model.tags().get("kdeemaofmxagkvtm")); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.identity().type()); + Assertions.assertEquals("pabgyeps", model.environmentId()); + Assertions.assertEquals("azqugxywpmueefj", model.containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("wfqkquj", model.containerRegistries().get(0).identityResourceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BuilderResourceInner model = new BuilderResourceInner().withLocation("hseyvju") + .withTags(mapOf("kdeemaofmxagkvtm", "slhs", "ahaquh", "lmqkrhahvlj", "aex", "dhmdua")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("lhrxsbkyvpyc", new UserAssignedIdentity(), "z", + new UserAssignedIdentity(), "kafkuwbcrnwbm", new UserAssignedIdentity()))) + .withEnvironmentId("pabgyeps") + .withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("azqugxywpmueefj") + .withIdentityResourceId("wfqkquj"), + new ContainerRegistry().withContainerRegistryServer("dsuyonobgla").withIdentityResourceId("cq"), + new ContainerRegistry().withContainerRegistryServer("tcc").withIdentityResourceId("g"), + new ContainerRegistry().withContainerRegistryServer("udxytlmoyrx") + .withIdentityResourceId("wfudwpzntxhdzhl"))); + model = BinaryData.fromObject(model).toObject(BuilderResourceInner.class); + Assertions.assertEquals("hseyvju", model.location()); + Assertions.assertEquals("slhs", model.tags().get("kdeemaofmxagkvtm")); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.identity().type()); + Assertions.assertEquals("pabgyeps", model.environmentId()); + Assertions.assertEquals("azqugxywpmueefj", model.containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("wfqkquj", model.containerRegistries().get(0).identityResourceId()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdatePropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdatePropertiesTests.java new file mode 100644 index 0000000000000..cca471d917c80 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdatePropertiesTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.BuilderResourceUpdateProperties; +import org.junit.jupiter.api.Assertions; + +public final class BuilderResourceUpdatePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BuilderResourceUpdateProperties model + = BinaryData.fromString("{\"environmentId\":\"zjuzgwyz\"}").toObject(BuilderResourceUpdateProperties.class); + Assertions.assertEquals("zjuzgwyz", model.environmentId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BuilderResourceUpdateProperties model = new BuilderResourceUpdateProperties().withEnvironmentId("zjuzgwyz"); + model = BinaryData.fromObject(model).toObject(BuilderResourceUpdateProperties.class); + Assertions.assertEquals("zjuzgwyz", model.environmentId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdateTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdateTests.java new file mode 100644 index 0000000000000..53627eaca33b1 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuilderResourceUpdateTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.BuilderResourceUpdate; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class BuilderResourceUpdateTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BuilderResourceUpdate model = BinaryData.fromString( + "{\"identity\":{\"principalId\":\"84d35291-d118-4f28-95b0-c1dcc3c3a910\",\"tenantId\":\"263ffff3-91e3-4316-90cc-1884f071a054\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"dnhjdauwhvylw\":{\"principalId\":\"b77a149b-c006-4798-9e9d-df5a35b59301\",\"clientId\":\"39ba2083-5071-4bde-9214-424f126d40a2\"}}},\"tags\":{\"uwprzql\":\"dhxujznbmpo\",\"obbc\":\"eualupjmkhf\",\"tghfgblcgwxzvl\":\"wsrtjriplrbpbe\",\"egibtnmxiebww\":\"qhjk\"},\"properties\":{\"environmentId\":\"ayqcgw\"}}") + .toObject(BuilderResourceUpdate.class); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, model.identity().type()); + Assertions.assertEquals("dhxujznbmpo", model.tags().get("uwprzql")); + Assertions.assertEquals("ayqcgw", model.environmentId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BuilderResourceUpdate model = new BuilderResourceUpdate() + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED) + .withUserAssignedIdentities(mapOf("dnhjdauwhvylw", new UserAssignedIdentity()))) + .withTags(mapOf("uwprzql", "dhxujznbmpo", "obbc", "eualupjmkhf", "tghfgblcgwxzvl", "wsrtjriplrbpbe", + "egibtnmxiebww", "qhjk")) + .withEnvironmentId("ayqcgw"); + model = BinaryData.fromObject(model).toObject(BuilderResourceUpdate.class); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, model.identity().type()); + Assertions.assertEquals("dhxujznbmpo", model.tags().get("uwprzql")); + Assertions.assertEquals("ayqcgw", model.environmentId()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateMockTests.java new file mode 100644 index 0000000000000..906b03fa1bd0d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersCreateOrUpdateMockTests.java @@ -0,0 +1,88 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentity; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import com.azure.resourcemanager.appcontainers.models.UserAssignedIdentity; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class BuildersCreateOrUpdateMockTests { + @Test + public void testCreateOrUpdate() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"environmentId\":\"atmqaqkuea\",\"containerRegistries\":[{\"containerRegistryServer\":\"oeshoygzcbyfq\",\"identityResourceId\":\"kfaoytehq\"},{\"containerRegistryServer\":\"uvjmv\",\"identityResourceId\":\"mtdwcky\"},{\"containerRegistryServer\":\"roejnndl\",\"identityResourceId\":\"djus\"}]},\"identity\":{\"principalId\":\"29dd6fc0-5b2c-4cc8-9cb1-9501230a97a8\",\"tenantId\":\"5b1e8839-d481-4cb6-90ec-e581fdf6f956\",\"type\":\"None\",\"userAssignedIdentities\":{\"ceysfaqeg\":{\"principalId\":\"6cd39565-3324-42f6-98f0-583a9e522072\",\"clientId\":\"7ff600fa-8860-4231-b8b5-4d6dc00eafc2\"},\"ryshwddkvbxgk\":{\"principalId\":\"0c22e148-22ce-455b-bffb-2d4bbec3cd57\",\"clientId\":\"8c461e97-f8b7-4554-84f9-0649f58baf04\"},\"ybwptda\":{\"principalId\":\"af01ff2e-9a5d-499a-b7e8-0bf72c2ab883\",\"clientId\":\"6db5edb8-83ab-4cf7-82c0-3800f26995e0\"},\"vvlfntymtp\":{\"principalId\":\"20a56fda-56ca-42b7-85fb-ccf6bca27205\",\"clientId\":\"e465b82e-bfa2-44c4-9a2f-d84fc0233ac5\"}}},\"location\":\"wenaz\",\"tags\":{\"qalsxkd\":\"hzr\",\"vessm\":\"wqapfgsdp\",\"dqq\":\"hhkuuip\"},\"id\":\"tekva\",\"name\":\"blhtjq\",\"type\":\"qyv\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + BuilderResource response = manager.builders().define("jbuscg").withRegion("yqrhvyeld") + .withExistingResourceGroup("bapxkiyfjjkb").withTags(mapOf("kwiswskukjtas", "v")) + .withIdentity( + new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("ovwmbjlzqsczpgvd", new UserAssignedIdentity(), "pfdqwowftptnu", + new UserAssignedIdentity(), "kschgcgqyhle", new UserAssignedIdentity()))) + .withEnvironmentId("ioycbl") + .withContainerRegistries(Arrays.asList( + new ContainerRegistry().withContainerRegistryServer("mclujyxkyxlzgs").withIdentityResourceId("gkzz"))) + .create(); + + Assertions.assertEquals("wenaz", response.location()); + Assertions.assertEquals("hzr", response.tags().get("qalsxkd")); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.identity().type()); + Assertions.assertEquals("atmqaqkuea", response.environmentId()); + Assertions.assertEquals("oeshoygzcbyfq", response.containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("kfaoytehq", response.containerRegistries().get(0).identityResourceId()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupWithResponseMockTests.java new file mode 100644 index 0000000000000..1773ed82e36c9 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersGetByResourceGroupWithResponseMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class BuildersGetByResourceGroupWithResponseMockTests { + @Test + public void testGetByResourceGroupWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"provisioningState\":\"Creating\",\"environmentId\":\"fwxudgnhg\",\"containerRegistries\":[{\"containerRegistryServer\":\"rtalvnbw\",\"identityResourceId\":\"pbeme\"},{\"containerRegistryServer\":\"uclvdjj\",\"identityResourceId\":\"kyrdnqodx\"}]},\"identity\":{\"principalId\":\"bd9b5d66-1857-47d4-96d2-da4b481c35dd\",\"tenantId\":\"84cd9d11-7fea-4e15-ac57-bda25c53edac\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"nvzoqgyipemch\":{\"principalId\":\"16827f64-00c8-404f-aeea-4da9cde3c89b\",\"clientId\":\"c5ce0006-6d24-445d-92b4-833a2c7a7490\"},\"sczuejdtxptlghwz\":{\"principalId\":\"b8d8b831-a0f2-47f3-8a51-4a61f7514391\",\"clientId\":\"cc54e4b5-cc4c-4b96-8585-53dc03fcb8e8\"}}},\"location\":\"mewjjstliuhq\",\"tags\":{\"blxydkxr\":\"oaiancznvodrrs\",\"hychocokuleh\":\"vvbxiwkgfbqljnq\"},\"id\":\"rqlrqffawe\",\"name\":\"urkphyjdxravju\",\"type\":\"dbrxmrgc\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + BuilderResource response = manager.builders() + .getByResourceGroupWithResponse("ftabenbbklqp", "zucafeddww", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("mewjjstliuhq", response.location()); + Assertions.assertEquals("oaiancznvodrrs", response.tags().get("blxydkxr")); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); + Assertions.assertEquals("fwxudgnhg", response.environmentId()); + Assertions.assertEquals("rtalvnbw", response.containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("pbeme", response.containerRegistries().get(0).identityResourceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupMockTests.java new file mode 100644 index 0000000000000..9b78786f059b2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListByResourceGroupMockTests.java @@ -0,0 +1,65 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class BuildersListByResourceGroupMockTests { + @Test + public void testListByResourceGroup() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"environmentId\":\"jzcfyjzptwr\",\"containerRegistries\":[{\"containerRegistryServer\":\"apqinf\",\"identityResourceId\":\"zpyglqdhmrj\"}]},\"identity\":{\"principalId\":\"c109c21e-0de6-4ec9-9e01-b23c08f50998\",\"tenantId\":\"05125a64-3f48-49a4-8407-c0dbd090d728\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"yypsjoqc\":{\"principalId\":\"9eb3f752-d5b7-4d78-b818-4f329432f5b6\",\"clientId\":\"a410d029-5e56-4090-83f9-84d3bb9258c0\"}}},\"location\":\"nkyhf\",\"tags\":{\"jhhhqxu\":\"sqxfxjelgcmpz\",\"oyvivbsiz\":\"yvca\",\"bscm\":\"sjsz\",\"v\":\"lzijiufehgmvflnw\"},\"id\":\"kxrerlniylylyfwx\",\"name\":\"utgqztwh\",\"type\":\"hmupgxyjtcdxabbu\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.builders().listByResourceGroup("hcoeocnhzq", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("nkyhf", response.iterator().next().location()); + Assertions.assertEquals("sqxfxjelgcmpz", response.iterator().next().tags().get("jhhhqxu")); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.iterator().next().identity().type()); + Assertions.assertEquals("jzcfyjzptwr", response.iterator().next().environmentId()); + Assertions.assertEquals("apqinf", + response.iterator().next().containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("zpyglqdhmrj", + response.iterator().next().containerRegistries().get(0).identityResourceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListMockTests.java new file mode 100644 index 0000000000000..14576179a9bf9 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/BuildersListMockTests.java @@ -0,0 +1,64 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.BuilderResource; +import com.azure.resourcemanager.appcontainers.models.ManagedServiceIdentityType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class BuildersListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"environmentId\":\"hvmblcouqehbhbc\",\"containerRegistries\":[{\"containerRegistryServer\":\"iryr\",\"identityResourceId\":\"ndo\"},{\"containerRegistryServer\":\"pmbltoormkfql\",\"identityResourceId\":\"xldykalsygaolnjp\"},{\"containerRegistryServer\":\"nbmjksibjgsjjxxa\",\"identityResourceId\":\"mr\"}]},\"identity\":{\"principalId\":\"1c0c7461-3309-4da8-9a13-57a29f20a97a\",\"tenantId\":\"f4d0c201-f27f-4a61-9ce1-ee809d63cbc9\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"yi\":{\"principalId\":\"8c44f5f7-1e24-45c6-8ed8-7777f6b38434\",\"clientId\":\"996c6a94-136d-48bd-826a-dfa5b00f46e1\"},\"nbm\":{\"principalId\":\"15efa945-7dba-47ff-9a63-7af495e5bce0\",\"clientId\":\"87bb6b63-12d1-4dc5-9198-4d1337032c68\"},\"jijkgqxnhmbke\":{\"principalId\":\"c3b09876-f7ea-4464-8c81-489961014d00\",\"clientId\":\"f3afd130-f471-4f6e-bccb-6be5ee9517d3\"},\"aujvaa\":{\"principalId\":\"aea57d92-76b1-4aa4-95da-0a7b69363c60\",\"clientId\":\"58f73390-0e22-4fce-a34a-5b5e0fbf1c13\"}}},\"location\":\"ggiycwkdtaawxwf\",\"tags\":{\"kratbnxwbj\":\"umrrqmbzm\",\"dgo\":\"idbirkfpkso\",\"zkye\":\"ewijymrhbguz\",\"o\":\"nfnzhhh\"},\"id\":\"mffjkutycyarn\",\"name\":\"oohguabzoghkt\",\"type\":\"pyc\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.builders().list(com.azure.core.util.Context.NONE); + + Assertions.assertEquals("ggiycwkdtaawxwf", response.iterator().next().location()); + Assertions.assertEquals("umrrqmbzm", response.iterator().next().tags().get("kratbnxwbj")); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, + response.iterator().next().identity().type()); + Assertions.assertEquals("hvmblcouqehbhbc", response.iterator().next().environmentId()); + Assertions.assertEquals("iryr", + response.iterator().next().containerRegistries().get(0).containerRegistryServer()); + Assertions.assertEquals("ndo", response.iterator().next().containerRegistries().get(0).identityResourceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatePatchTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatePatchTests.java index f35aba2d25b3c..52739b2c49ca0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatePatchTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatePatchTests.java @@ -13,22 +13,16 @@ public final class CertificatePatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CertificatePatch model = - BinaryData - .fromString( - "{\"tags\":{\"flz\":\"ukgri\",\"qzahmgkbrp\":\"fbxzpuzycisp\",\"hibnuqqkpika\":\"y\",\"buynhijggm\":\"rgvtqag\"}}") - .toObject(CertificatePatch.class); - Assertions.assertEquals("ukgri", model.tags().get("flz")); + CertificatePatch model + = BinaryData.fromString("{\"tags\":{\"ekkezzikhlyfjh\":\"kfcktqum\"}}").toObject(CertificatePatch.class); + Assertions.assertEquals("kfcktqum", model.tags().get("ekkezzikhlyfjh")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CertificatePatch model = - new CertificatePatch() - .withTags( - mapOf("flz", "ukgri", "qzahmgkbrp", "fbxzpuzycisp", "hibnuqqkpika", "y", "buynhijggm", "rgvtqag")); + CertificatePatch model = new CertificatePatch().withTags(mapOf("ekkezzikhlyfjh", "kfcktqum")); model = BinaryData.fromObject(model).toObject(CertificatePatch.class); - Assertions.assertEquals("ukgri", model.tags().get("flz")); + Assertions.assertEquals("kfcktqum", model.tags().get("ekkezzikhlyfjh")); } // Use "Map.of" if available diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteWithResponseMockTests.java index 5b585901c86bf..cbeb7511b7d9f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CertificatesDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .certificates() - .deleteWithResponse("afjrqpjiyrqjcrg", "xwmzwdfkbnrz", "rpdltbq", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.certificates().deleteWithResponse("ymjzpwdlvwtiws", "osaonhqnamppu", "tassaekewna", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityRequestTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityRequestTests.java index 692d3ffcf8f7b..54fee892b5dad 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityRequestTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityRequestTests.java @@ -11,19 +11,17 @@ public final class CheckNameAvailabilityRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CheckNameAvailabilityRequest model = - BinaryData - .fromString("{\"name\":\"idokgjlj\",\"type\":\"xgvcl\"}") - .toObject(CheckNameAvailabilityRequest.class); - Assertions.assertEquals("idokgjlj", model.name()); - Assertions.assertEquals("xgvcl", model.type()); + CheckNameAvailabilityRequest model = BinaryData.fromString("{\"name\":\"i\",\"type\":\"blxgwimf\"}") + .toObject(CheckNameAvailabilityRequest.class); + Assertions.assertEquals("i", model.name()); + Assertions.assertEquals("blxgwimf", model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CheckNameAvailabilityRequest model = new CheckNameAvailabilityRequest().withName("idokgjlj").withType("xgvcl"); + CheckNameAvailabilityRequest model = new CheckNameAvailabilityRequest().withName("i").withType("blxgwimf"); model = BinaryData.fromObject(model).toObject(CheckNameAvailabilityRequest.class); - Assertions.assertEquals("idokgjlj", model.name()); - Assertions.assertEquals("xgvcl", model.type()); + Assertions.assertEquals("i", model.name()); + Assertions.assertEquals("blxgwimf", model.type()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityResponseInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityResponseInnerTests.java index 747841faabc2a..6d25db82465be 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityResponseInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CheckNameAvailabilityResponseInnerTests.java @@ -12,25 +12,21 @@ public final class CheckNameAvailabilityResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CheckNameAvailabilityResponseInner model = - BinaryData - .fromString("{\"nameAvailable\":true,\"reason\":\"Invalid\",\"message\":\"ghkjeszzhbi\"}") + CheckNameAvailabilityResponseInner model + = BinaryData.fromString("{\"nameAvailable\":false,\"reason\":\"Invalid\",\"message\":\"wmszkk\"}") .toObject(CheckNameAvailabilityResponseInner.class); - Assertions.assertEquals(true, model.nameAvailable()); + Assertions.assertEquals(false, model.nameAvailable()); Assertions.assertEquals(CheckNameAvailabilityReason.INVALID, model.reason()); - Assertions.assertEquals("ghkjeszzhbi", model.message()); + Assertions.assertEquals("wmszkk", model.message()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CheckNameAvailabilityResponseInner model = - new CheckNameAvailabilityResponseInner() - .withNameAvailable(true) - .withReason(CheckNameAvailabilityReason.INVALID) - .withMessage("ghkjeszzhbi"); + CheckNameAvailabilityResponseInner model = new CheckNameAvailabilityResponseInner().withNameAvailable(false) + .withReason(CheckNameAvailabilityReason.INVALID).withMessage("wmszkk"); model = BinaryData.fromObject(model).toObject(CheckNameAvailabilityResponseInner.class); - Assertions.assertEquals(true, model.nameAvailable()); + Assertions.assertEquals(false, model.nameAvailable()); Assertions.assertEquals(CheckNameAvailabilityReason.INVALID, model.reason()); - Assertions.assertEquals("ghkjeszzhbi", model.message()); + Assertions.assertEquals("wmszkk", model.message()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CircuitBreakerPolicyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CircuitBreakerPolicyTests.java new file mode 100644 index 0000000000000..552002cd6f222 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CircuitBreakerPolicyTests.java @@ -0,0 +1,32 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.CircuitBreakerPolicy; +import org.junit.jupiter.api.Assertions; + +public final class CircuitBreakerPolicyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CircuitBreakerPolicy model = BinaryData + .fromString( + "{\"consecutiveErrors\":1722437661,\"intervalInSeconds\":1946289710,\"maxEjectionPercent\":158340850}") + .toObject(CircuitBreakerPolicy.class); + Assertions.assertEquals(1722437661, model.consecutiveErrors()); + Assertions.assertEquals(1946289710, model.intervalInSeconds()); + Assertions.assertEquals(158340850, model.maxEjectionPercent()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CircuitBreakerPolicy model = new CircuitBreakerPolicy().withConsecutiveErrors(1722437661) + .withIntervalInSeconds(1946289710).withMaxEjectionPercent(158340850); + model = BinaryData.fromObject(model).toObject(CircuitBreakerPolicy.class); + Assertions.assertEquals(1722437661, model.consecutiveErrors()); + Assertions.assertEquals(1946289710, model.intervalInSeconds()); + Assertions.assertEquals(158340850, model.maxEjectionPercent()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteWithResponseMockTests.java index eb4d7837e45a8..aeaabdb8f0804 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCertificatesDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .connectedEnvironmentsCertificates() - .deleteWithResponse("sqqw", "tcmwqkchc", "waxfewzjkj", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.connectedEnvironmentsCertificates().deleteWithResponse("rcyrucpcunnu", "dqumoenodnai", "nhq", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilityWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilityWithResponseMockTests.java index 64f135a178207..03908e4869acc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsCheckNameAvailabilityWithResponseMockTests.java @@ -32,46 +32,30 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"nameAvailable\":true,\"reason\":\"AlreadyExists\",\"message\":\"mbnkb\"}"; + String responseStr = "{\"nameAvailable\":false,\"reason\":\"Invalid\",\"message\":\"jtfvpndpmiljpn\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - CheckNameAvailabilityResponse response = - manager - .connectedEnvironments() - .checkNameAvailabilityWithResponse( - "yqo", - "mpqoxw", - new CheckNameAvailabilityRequest().withName("fdbxiqxeiiqbim").withType("mwwinhehfqpofv"), - com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals(true, response.nameAvailable()); - Assertions.assertEquals(CheckNameAvailabilityReason.ALREADY_EXISTS, response.reason()); - Assertions.assertEquals("mbnkb", response.message()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + CheckNameAvailabilityResponse response = manager.connectedEnvironments() + .checkNameAvailabilityWithResponse("zregzgyufutrwpw", "ryekzkd", + new CheckNameAvailabilityRequest().withName("eotta").withType("yos"), com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals(false, response.nameAvailable()); + Assertions.assertEquals(CheckNameAvailabilityReason.INVALID, response.reason()); + Assertions.assertEquals("jtfvpndpmiljpn", response.message()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteWithResponseMockTests.java index b32238da4cebb..8036deaffa7f0 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .connectedEnvironmentsDaprComponents() - .deleteWithResponse("tsxoatftgz", "npbs", "vefloccsrmozihmi", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.connectedEnvironmentsDaprComponents().deleteWithResponse("rhunlp", "rykycndzfqivjr", "uykbbmn", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsWithResponseMockTests.java index f40867f2a5d18..e8e274a5ebe64 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDaprComponentsListSecretsWithResponseMockTests.java @@ -29,39 +29,26 @@ public void testListSecretsWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"ilrmcaykggnox\",\"value\":\"t\"},{\"name\":\"sxwpndfcpfnznthj\",\"value\":\"kjaosrxuzv\"},{\"name\":\"mktcqiosmgbza\",\"value\":\"xqdlyrtltlapr\"},{\"name\":\"zkatb\",\"value\":\"mznnbsoqeqalarvl\"}]}"; + String responseStr + = "{\"value\":[{\"name\":\"myymvqdbpbhfckdv\",\"value\":\"crcss\"},{\"name\":\"hddubbnqfbl\",\"value\":\"alehpav\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DaprSecretsCollection response = manager.connectedEnvironmentsDaprComponents() + .listSecretsWithResponse("aglt", "xoe", "onql", com.azure.core.util.Context.NONE).getValue(); - DaprSecretsCollection response = - manager - .connectedEnvironmentsDaprComponents() - .listSecretsWithResponse("g", "wtxxpkyjcx", "jxgrytfmp", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteMockTests.java index e17158608086c..24f9db539107a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.connectedEnvironments().delete("lnwcltyjed", "xxmlfmkqscazua", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.connectedEnvironments().delete("hfw", "v", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteWithResponseMockTests.java index 9ab93659458b8..29790bd95607b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ConnectedEnvironmentsStoragesDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .connectedEnvironmentsStorages() - .deleteWithResponse("xff", "shkwfbkgozxwopd", "yd", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.connectedEnvironmentsStorages().deleteWithResponse("xeyg", "qigijiitnspxlzde", "ygr", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppJobExecutionsInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppJobExecutionsInnerTests.java index 739ced9b872b0..99c65f203b0ca 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppJobExecutionsInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppJobExecutionsInnerTests.java @@ -7,8 +7,6 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.appcontainers.fluent.models.ContainerAppJobExecutionsInner; import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner; -import com.azure.resourcemanager.appcontainers.models.ContainerResources; -import com.azure.resourcemanager.appcontainers.models.EnvironmentVar; import com.azure.resourcemanager.appcontainers.models.JobExecutionContainer; import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate; import java.time.OffsetDateTime; @@ -18,182 +16,36 @@ public final class ContainerAppJobExecutionsInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ContainerAppJobExecutionsInner model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"djhlimm\",\"id\":\"x\",\"type\":\"bcporxvxcjzhqizx\",\"status\":\"Degraded\",\"startTime\":\"2021-01-04T20:39:53Z\",\"endTime\":\"2021-03-07T05:50:04Z\",\"template\":{\"containers\":[{\"image\":\"ftjuh\",\"name\":\"azkmtgguwp\",\"command\":[\"ajc\"],\"args\":[\"mghfcfiwrxgkne\"],\"env\":[{},{},{}],\"resources\":{}}],\"initContainers\":[{\"image\":\"o\",\"name\":\"vpgshoxgsgbp\",\"command\":[\"djtxvzflbq\",\"gaqvlgafcqusrd\",\"etnwsdtutnw\"],\"args\":[\"ycvuzhyrmewipmv\",\"k\",\"xukuqgsj\",\"xundxgk\"],\"env\":[{},{},{},{}],\"resources\":{}},{\"image\":\"hzjhf\",\"name\":\"hvvmuvgpmun\",\"command\":[\"xvmhf\"],\"args\":[\"jyihsasbhudypo\"],\"env\":[{},{}],\"resources\":{}},{\"image\":\"slynsqyrpfoo\",\"name\":\"lttymsjn\",\"command\":[\"dnfwqzdzg\",\"ilaxhn\",\"hqlyvijo\"],\"args\":[\"vk\"],\"env\":[{},{},{},{}],\"resources\":{}}]}},{\"name\":\"nbixxrti\",\"id\":\"cpwpg\",\"type\":\"rc\",\"status\":\"Degraded\",\"startTime\":\"2021-01-12T06:52:04Z\",\"endTime\":\"2020-12-22T20:01:29Z\",\"template\":{\"containers\":[{\"image\":\"xpmyyefrpmpdnq\",\"name\":\"ka\",\"command\":[\"qvmmbn\"],\"args\":[\"rtql\"],\"env\":[{},{}],\"resources\":{}},{\"image\":\"nitgvkxlz\",\"name\":\"drf\",\"command\":[\"ealzxwhcansymoyq\",\"lwigdivbkbx\"],\"args\":[\"fajuwas\"],\"env\":[{},{}],\"resources\":{}},{\"image\":\"y\",\"name\":\"uxakjsqzhzbezk\",\"command\":[\"sidxasicdd\",\"vvjskgfmocwahp\",\"gat\",\"eaahhvjhhn\"],\"args\":[\"ybbjjidjksyx\",\"yxvxevblbjed\"],\"env\":[{},{}],\"resources\":{}}],\"initContainers\":[{\"image\":\"uaulxunsm\",\"name\":\"nkppxynen\",\"command\":[\"xeizzg\"],\"args\":[\"nsrmffeycx\",\"ktp\"],\"env\":[{},{},{},{}],\"resources\":{}},{\"image\":\"teeammxqiekkkzd\",\"name\":\"tkgdojbmxva\",\"command\":[\"fdeesvecuijpxtx\",\"uwprtujwsawd\"],\"args\":[\"babxvitit\",\"tzeexav\",\"xtfglecdmdqb\",\"pypqtgsfj\"],\"env\":[{}],\"resources\":{}}]}}],\"nextLink\":\"hhxud\"}") - .toObject(ContainerAppJobExecutionsInner.class); - Assertions.assertEquals("djhlimm", model.value().get(0).name()); - Assertions.assertEquals("x", model.value().get(0).id()); - Assertions.assertEquals("bcporxvxcjzhqizx", model.value().get(0).type()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-04T20:39:53Z"), model.value().get(0).startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-03-07T05:50:04Z"), model.value().get(0).endTime()); - Assertions.assertEquals("ftjuh", model.value().get(0).template().containers().get(0).image()); - Assertions.assertEquals("azkmtgguwp", model.value().get(0).template().containers().get(0).name()); - Assertions.assertEquals("ajc", model.value().get(0).template().containers().get(0).command().get(0)); - Assertions.assertEquals("mghfcfiwrxgkne", model.value().get(0).template().containers().get(0).args().get(0)); - Assertions.assertEquals("o", model.value().get(0).template().initContainers().get(0).image()); - Assertions.assertEquals("vpgshoxgsgbp", model.value().get(0).template().initContainers().get(0).name()); - Assertions.assertEquals("djtxvzflbq", model.value().get(0).template().initContainers().get(0).command().get(0)); - Assertions - .assertEquals("ycvuzhyrmewipmv", model.value().get(0).template().initContainers().get(0).args().get(0)); + ContainerAppJobExecutionsInner model = BinaryData.fromString( + "{\"value\":[{\"name\":\"uartvti\",\"id\":\"yefchnm\",\"type\":\"hmnxhkxjqi\",\"properties\":{\"status\":\"Degraded\",\"startTime\":\"2021-02-25T22:43:28Z\",\"endTime\":\"2021-08-15T18:00:46Z\",\"template\":{\"containers\":[{}],\"initContainers\":[{}]}}},{\"name\":\"wrsnewmozqvbu\",\"id\":\"ma\",\"type\":\"sycxhxzgaz\",\"properties\":{\"status\":\"Running\",\"startTime\":\"2021-09-16T19:45:24Z\",\"endTime\":\"2021-01-19T00:16:09Z\",\"template\":{\"containers\":[{},{},{},{}],\"initContainers\":[{}]}}}],\"nextLink\":\"ubowsepdfgkm\"}") + .toObject(ContainerAppJobExecutionsInner.class); + Assertions.assertEquals("uartvti", model.value().get(0).name()); + Assertions.assertEquals("yefchnm", model.value().get(0).id()); + Assertions.assertEquals("hmnxhkxjqi", model.value().get(0).type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-25T22:43:28Z"), model.value().get(0).startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-15T18:00:46Z"), model.value().get(0).endTime()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ContainerAppJobExecutionsInner model = - new ContainerAppJobExecutionsInner() - .withValue( - Arrays - .asList( - new JobExecutionInner() - .withName("djhlimm") - .withId("x") - .withType("bcporxvxcjzhqizx") - .withStartTime(OffsetDateTime.parse("2021-01-04T20:39:53Z")) - .withEndTime(OffsetDateTime.parse("2021-03-07T05:50:04Z")) - .withTemplate( - new JobExecutionTemplate() - .withContainers( - Arrays - .asList( - new JobExecutionContainer() - .withImage("ftjuh") - .withName("azkmtgguwp") - .withCommand(Arrays.asList("ajc")) - .withArgs(Arrays.asList("mghfcfiwrxgkne")) - .withEnv( - Arrays - .asList( - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar())) - .withResources(new ContainerResources()))) - .withInitContainers( - Arrays - .asList( - new JobExecutionContainer() - .withImage("o") - .withName("vpgshoxgsgbp") - .withCommand( - Arrays - .asList("djtxvzflbq", "gaqvlgafcqusrd", "etnwsdtutnw")) - .withArgs( - Arrays - .asList("ycvuzhyrmewipmv", "k", "xukuqgsj", "xundxgk")) - .withEnv( - Arrays - .asList( - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar())) - .withResources(new ContainerResources()), - new JobExecutionContainer() - .withImage("hzjhf") - .withName("hvvmuvgpmun") - .withCommand(Arrays.asList("xvmhf")) - .withArgs(Arrays.asList("jyihsasbhudypo")) - .withEnv( - Arrays.asList(new EnvironmentVar(), new EnvironmentVar())) - .withResources(new ContainerResources()), - new JobExecutionContainer() - .withImage("slynsqyrpfoo") - .withName("lttymsjn") - .withCommand(Arrays.asList("dnfwqzdzg", "ilaxhn", "hqlyvijo")) - .withArgs(Arrays.asList("vk")) - .withEnv( - Arrays - .asList( - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar())) - .withResources(new ContainerResources())))), - new JobExecutionInner() - .withName("nbixxrti") - .withId("cpwpg") - .withType("rc") - .withStartTime(OffsetDateTime.parse("2021-01-12T06:52:04Z")) - .withEndTime(OffsetDateTime.parse("2020-12-22T20:01:29Z")) - .withTemplate( - new JobExecutionTemplate() - .withContainers( - Arrays - .asList( - new JobExecutionContainer() - .withImage("xpmyyefrpmpdnq") - .withName("ka") - .withCommand(Arrays.asList("qvmmbn")) - .withArgs(Arrays.asList("rtql")) - .withEnv( - Arrays.asList(new EnvironmentVar(), new EnvironmentVar())) - .withResources(new ContainerResources()), - new JobExecutionContainer() - .withImage("nitgvkxlz") - .withName("drf") - .withCommand(Arrays.asList("ealzxwhcansymoyq", "lwigdivbkbx")) - .withArgs(Arrays.asList("fajuwas")) - .withEnv( - Arrays.asList(new EnvironmentVar(), new EnvironmentVar())) - .withResources(new ContainerResources()), - new JobExecutionContainer() - .withImage("y") - .withName("uxakjsqzhzbezk") - .withCommand( - Arrays - .asList( - "sidxasicdd", - "vvjskgfmocwahp", - "gat", - "eaahhvjhhn")) - .withArgs(Arrays.asList("ybbjjidjksyx", "yxvxevblbjed")) - .withEnv( - Arrays.asList(new EnvironmentVar(), new EnvironmentVar())) - .withResources(new ContainerResources()))) - .withInitContainers( - Arrays - .asList( - new JobExecutionContainer() - .withImage("uaulxunsm") - .withName("nkppxynen") - .withCommand(Arrays.asList("xeizzg")) - .withArgs(Arrays.asList("nsrmffeycx", "ktp")) - .withEnv( - Arrays - .asList( - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar(), - new EnvironmentVar())) - .withResources(new ContainerResources()), - new JobExecutionContainer() - .withImage("teeammxqiekkkzd") - .withName("tkgdojbmxva") - .withCommand(Arrays.asList("fdeesvecuijpxtx", "uwprtujwsawd")) - .withArgs( - Arrays - .asList( - "babxvitit", - "tzeexav", - "xtfglecdmdqb", - "pypqtgsfj")) - .withEnv(Arrays.asList(new EnvironmentVar())) - .withResources(new ContainerResources())))))); + ContainerAppJobExecutionsInner model = new ContainerAppJobExecutionsInner().withValue(Arrays.asList( + new JobExecutionInner().withName("uartvti").withId("yefchnm").withType("hmnxhkxjqi") + .withStartTime(OffsetDateTime.parse("2021-02-25T22:43:28Z")) + .withEndTime(OffsetDateTime.parse("2021-08-15T18:00:46Z")) + .withTemplate(new JobExecutionTemplate().withContainers(Arrays.asList(new JobExecutionContainer())) + .withInitContainers(Arrays.asList(new JobExecutionContainer()))), + new JobExecutionInner().withName("wrsnewmozqvbu").withId("ma").withType("sycxhxzgaz") + .withStartTime(OffsetDateTime.parse("2021-09-16T19:45:24Z")) + .withEndTime(OffsetDateTime.parse("2021-01-19T00:16:09Z")) + .withTemplate(new JobExecutionTemplate() + .withContainers(Arrays.asList(new JobExecutionContainer(), new JobExecutionContainer(), + new JobExecutionContainer(), new JobExecutionContainer())) + .withInitContainers(Arrays.asList(new JobExecutionContainer()))))); model = BinaryData.fromObject(model).toObject(ContainerAppJobExecutionsInner.class); - Assertions.assertEquals("djhlimm", model.value().get(0).name()); - Assertions.assertEquals("x", model.value().get(0).id()); - Assertions.assertEquals("bcporxvxcjzhqizx", model.value().get(0).type()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-04T20:39:53Z"), model.value().get(0).startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-03-07T05:50:04Z"), model.value().get(0).endTime()); - Assertions.assertEquals("ftjuh", model.value().get(0).template().containers().get(0).image()); - Assertions.assertEquals("azkmtgguwp", model.value().get(0).template().containers().get(0).name()); - Assertions.assertEquals("ajc", model.value().get(0).template().containers().get(0).command().get(0)); - Assertions.assertEquals("mghfcfiwrxgkne", model.value().get(0).template().containers().get(0).args().get(0)); - Assertions.assertEquals("o", model.value().get(0).template().initContainers().get(0).image()); - Assertions.assertEquals("vpgshoxgsgbp", model.value().get(0).template().initContainers().get(0).name()); - Assertions.assertEquals("djtxvzflbq", model.value().get(0).template().initContainers().get(0).command().get(0)); - Assertions - .assertEquals("ycvuzhyrmewipmv", model.value().get(0).template().initContainers().get(0).args().get(0)); + Assertions.assertEquals("uartvti", model.value().get(0).name()); + Assertions.assertEquals("yefchnm", model.value().get(0).id()); + Assertions.assertEquals("hmnxhkxjqi", model.value().get(0).type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-25T22:43:28Z"), model.value().get(0).startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-15T18:00:46Z"), model.value().get(0).endTime()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetHttpHeadersItemTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetHttpHeadersItemTests.java index 80a9d6dc75b84..546d6b8d32bd3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetHttpHeadersItemTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetHttpHeadersItemTests.java @@ -11,20 +11,19 @@ public final class ContainerAppProbeHttpGetHttpHeadersItemTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ContainerAppProbeHttpGetHttpHeadersItem model = - BinaryData - .fromString("{\"name\":\"hzoymibmrqy\",\"value\":\"bahwfl\"}") + ContainerAppProbeHttpGetHttpHeadersItem model + = BinaryData.fromString("{\"name\":\"lihhyuspskasdvlm\",\"value\":\"wdgzxulucv\"}") .toObject(ContainerAppProbeHttpGetHttpHeadersItem.class); - Assertions.assertEquals("hzoymibmrqy", model.name()); - Assertions.assertEquals("bahwfl", model.value()); + Assertions.assertEquals("lihhyuspskasdvlm", model.name()); + Assertions.assertEquals("wdgzxulucv", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ContainerAppProbeHttpGetHttpHeadersItem model = - new ContainerAppProbeHttpGetHttpHeadersItem().withName("hzoymibmrqy").withValue("bahwfl"); + ContainerAppProbeHttpGetHttpHeadersItem model + = new ContainerAppProbeHttpGetHttpHeadersItem().withName("lihhyuspskasdvlm").withValue("wdgzxulucv"); model = BinaryData.fromObject(model).toObject(ContainerAppProbeHttpGetHttpHeadersItem.class); - Assertions.assertEquals("hzoymibmrqy", model.name()); - Assertions.assertEquals("bahwfl", model.value()); + Assertions.assertEquals("lihhyuspskasdvlm", model.name()); + Assertions.assertEquals("wdgzxulucv", model.value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetTests.java index 6053fe33e51cc..0a450bfe15d95 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeHttpGetTests.java @@ -14,40 +14,31 @@ public final class ContainerAppProbeHttpGetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ContainerAppProbeHttpGet model = - BinaryData - .fromString( - "{\"host\":\"quxvypomgkop\",\"httpHeaders\":[{\"name\":\"ojvpa\",\"value\":\"qgxy\"},{\"name\":\"mocmbqfqvmk\",\"value\":\"xozap\"},{\"name\":\"helxprglya\",\"value\":\"dd\"},{\"name\":\"kcbcue\",\"value\":\"rjxgciqib\"}],\"path\":\"osx\",\"port\":341443712,\"scheme\":\"HTTP\"}") - .toObject(ContainerAppProbeHttpGet.class); - Assertions.assertEquals("quxvypomgkop", model.host()); - Assertions.assertEquals("ojvpa", model.httpHeaders().get(0).name()); - Assertions.assertEquals("qgxy", model.httpHeaders().get(0).value()); - Assertions.assertEquals("osx", model.path()); - Assertions.assertEquals(341443712, model.port()); + ContainerAppProbeHttpGet model = BinaryData.fromString( + "{\"host\":\"shfssnrbgyef\",\"httpHeaders\":[{\"name\":\"sgaojfmwncot\",\"value\":\"rfh\"},{\"name\":\"rctym\",\"value\":\"xoftpipiwyczu\"}],\"path\":\"a\",\"port\":850210589,\"scheme\":\"HTTP\"}") + .toObject(ContainerAppProbeHttpGet.class); + Assertions.assertEquals("shfssnrbgyef", model.host()); + Assertions.assertEquals("sgaojfmwncot", model.httpHeaders().get(0).name()); + Assertions.assertEquals("rfh", model.httpHeaders().get(0).value()); + Assertions.assertEquals("a", model.path()); + Assertions.assertEquals(850210589, model.port()); Assertions.assertEquals(Scheme.HTTP, model.scheme()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ContainerAppProbeHttpGet model = - new ContainerAppProbeHttpGet() - .withHost("quxvypomgkop") - .withHttpHeaders( - Arrays - .asList( - new ContainerAppProbeHttpGetHttpHeadersItem().withName("ojvpa").withValue("qgxy"), - new ContainerAppProbeHttpGetHttpHeadersItem().withName("mocmbqfqvmk").withValue("xozap"), - new ContainerAppProbeHttpGetHttpHeadersItem().withName("helxprglya").withValue("dd"), - new ContainerAppProbeHttpGetHttpHeadersItem().withName("kcbcue").withValue("rjxgciqib"))) - .withPath("osx") - .withPort(341443712) - .withScheme(Scheme.HTTP); + ContainerAppProbeHttpGet model + = new ContainerAppProbeHttpGet().withHost("shfssnrbgyef") + .withHttpHeaders(Arrays.asList( + new ContainerAppProbeHttpGetHttpHeadersItem().withName("sgaojfmwncot").withValue("rfh"), + new ContainerAppProbeHttpGetHttpHeadersItem().withName("rctym").withValue("xoftpipiwyczu"))) + .withPath("a").withPort(850210589).withScheme(Scheme.HTTP); model = BinaryData.fromObject(model).toObject(ContainerAppProbeHttpGet.class); - Assertions.assertEquals("quxvypomgkop", model.host()); - Assertions.assertEquals("ojvpa", model.httpHeaders().get(0).name()); - Assertions.assertEquals("qgxy", model.httpHeaders().get(0).value()); - Assertions.assertEquals("osx", model.path()); - Assertions.assertEquals(341443712, model.port()); + Assertions.assertEquals("shfssnrbgyef", model.host()); + Assertions.assertEquals("sgaojfmwncot", model.httpHeaders().get(0).name()); + Assertions.assertEquals("rfh", model.httpHeaders().get(0).value()); + Assertions.assertEquals("a", model.path()); + Assertions.assertEquals(850210589, model.port()); Assertions.assertEquals(Scheme.HTTP, model.scheme()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTcpSocketTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTcpSocketTests.java index d35547dd69c04..9acc700164488 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTcpSocketTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTcpSocketTests.java @@ -11,20 +11,18 @@ public final class ContainerAppProbeTcpSocketTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ContainerAppProbeTcpSocket model = - BinaryData - .fromString("{\"host\":\"zdtmhrkwofy\",\"port\":1026914225}") - .toObject(ContainerAppProbeTcpSocket.class); - Assertions.assertEquals("zdtmhrkwofy", model.host()); - Assertions.assertEquals(1026914225, model.port()); + ContainerAppProbeTcpSocket model = BinaryData.fromString("{\"host\":\"mrsreuzvxurisjnh\",\"port\":1189641650}") + .toObject(ContainerAppProbeTcpSocket.class); + Assertions.assertEquals("mrsreuzvxurisjnh", model.host()); + Assertions.assertEquals(1189641650, model.port()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ContainerAppProbeTcpSocket model = - new ContainerAppProbeTcpSocket().withHost("zdtmhrkwofy").withPort(1026914225); + ContainerAppProbeTcpSocket model + = new ContainerAppProbeTcpSocket().withHost("mrsreuzvxurisjnh").withPort(1189641650); model = BinaryData.fromObject(model).toObject(ContainerAppProbeTcpSocket.class); - Assertions.assertEquals("zdtmhrkwofy", model.host()); - Assertions.assertEquals(1026914225, model.port()); + Assertions.assertEquals("mrsreuzvxurisjnh", model.host()); + Assertions.assertEquals(1189641650, model.port()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTests.java index f945085b6144f..61093c41d2fdc 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppProbeTests.java @@ -17,67 +17,55 @@ public final class ContainerAppProbeTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ContainerAppProbe model = - BinaryData - .fromString( - "{\"failureThreshold\":1317619791,\"httpGet\":{\"host\":\"kzzlvmbmpaxmodf\",\"httpHeaders\":[{\"name\":\"fy\",\"value\":\"sbpfvmwyhr\"}],\"path\":\"uyfta\",\"port\":1378791233,\"scheme\":\"HTTPS\"},\"initialDelaySeconds\":369562825,\"periodSeconds\":1040923285,\"successThreshold\":842537299,\"tcpSocket\":{\"host\":\"nubexk\",\"port\":1989438546},\"terminationGracePeriodSeconds\":5387832879483701613,\"timeoutSeconds\":487764269,\"type\":\"Startup\"}") - .toObject(ContainerAppProbe.class); - Assertions.assertEquals(1317619791, model.failureThreshold()); - Assertions.assertEquals("kzzlvmbmpaxmodf", model.httpGet().host()); - Assertions.assertEquals("fy", model.httpGet().httpHeaders().get(0).name()); - Assertions.assertEquals("sbpfvmwyhr", model.httpGet().httpHeaders().get(0).value()); - Assertions.assertEquals("uyfta", model.httpGet().path()); - Assertions.assertEquals(1378791233, model.httpGet().port()); + ContainerAppProbe model = BinaryData.fromString( + "{\"failureThreshold\":1290029311,\"httpGet\":{\"host\":\"ycnunvjsrtk\",\"httpHeaders\":[{\"name\":\"nopqgikyzirtx\",\"value\":\"yuxzejntpsewgi\"},{\"name\":\"ilqu\",\"value\":\"rydxtqm\"},{\"name\":\"eoxorggufhyao\",\"value\":\"tbghhavgrvkf\"},{\"name\":\"ovjzhpjbibgjmfx\",\"value\":\"mv\"}],\"path\":\"luyovwxnbkfezzx\",\"port\":357757853,\"scheme\":\"HTTPS\"},\"initialDelaySeconds\":1673017156,\"periodSeconds\":215949715,\"successThreshold\":1640359024,\"tcpSocket\":{\"host\":\"bzbomvzzbtdcqvpn\",\"port\":1622829877},\"terminationGracePeriodSeconds\":2588743107415946739,\"timeoutSeconds\":277819924,\"type\":\"Startup\"}") + .toObject(ContainerAppProbe.class); + Assertions.assertEquals(1290029311, model.failureThreshold()); + Assertions.assertEquals("ycnunvjsrtk", model.httpGet().host()); + Assertions.assertEquals("nopqgikyzirtx", model.httpGet().httpHeaders().get(0).name()); + Assertions.assertEquals("yuxzejntpsewgi", model.httpGet().httpHeaders().get(0).value()); + Assertions.assertEquals("luyovwxnbkfezzx", model.httpGet().path()); + Assertions.assertEquals(357757853, model.httpGet().port()); Assertions.assertEquals(Scheme.HTTPS, model.httpGet().scheme()); - Assertions.assertEquals(369562825, model.initialDelaySeconds()); - Assertions.assertEquals(1040923285, model.periodSeconds()); - Assertions.assertEquals(842537299, model.successThreshold()); - Assertions.assertEquals("nubexk", model.tcpSocket().host()); - Assertions.assertEquals(1989438546, model.tcpSocket().port()); - Assertions.assertEquals(5387832879483701613L, model.terminationGracePeriodSeconds()); - Assertions.assertEquals(487764269, model.timeoutSeconds()); + Assertions.assertEquals(1673017156, model.initialDelaySeconds()); + Assertions.assertEquals(215949715, model.periodSeconds()); + Assertions.assertEquals(1640359024, model.successThreshold()); + Assertions.assertEquals("bzbomvzzbtdcqvpn", model.tcpSocket().host()); + Assertions.assertEquals(1622829877, model.tcpSocket().port()); + Assertions.assertEquals(2588743107415946739L, model.terminationGracePeriodSeconds()); + Assertions.assertEquals(277819924, model.timeoutSeconds()); Assertions.assertEquals(Type.STARTUP, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ContainerAppProbe model = - new ContainerAppProbe() - .withFailureThreshold(1317619791) - .withHttpGet( - new ContainerAppProbeHttpGet() - .withHost("kzzlvmbmpaxmodf") - .withHttpHeaders( - Arrays - .asList( - new ContainerAppProbeHttpGetHttpHeadersItem() - .withName("fy") - .withValue("sbpfvmwyhr"))) - .withPath("uyfta") - .withPort(1378791233) - .withScheme(Scheme.HTTPS)) - .withInitialDelaySeconds(369562825) - .withPeriodSeconds(1040923285) - .withSuccessThreshold(842537299) - .withTcpSocket(new ContainerAppProbeTcpSocket().withHost("nubexk").withPort(1989438546)) - .withTerminationGracePeriodSeconds(5387832879483701613L) - .withTimeoutSeconds(487764269) - .withType(Type.STARTUP); + ContainerAppProbe model = new ContainerAppProbe().withFailureThreshold(1290029311) + .withHttpGet(new ContainerAppProbeHttpGet().withHost("ycnunvjsrtk") + .withHttpHeaders(Arrays.asList( + new ContainerAppProbeHttpGetHttpHeadersItem().withName("nopqgikyzirtx").withValue("yuxzejntpsewgi"), + new ContainerAppProbeHttpGetHttpHeadersItem().withName("ilqu").withValue("rydxtqm"), + new ContainerAppProbeHttpGetHttpHeadersItem().withName("eoxorggufhyao").withValue("tbghhavgrvkf"), + new ContainerAppProbeHttpGetHttpHeadersItem().withName("ovjzhpjbibgjmfx").withValue("mv"))) + .withPath("luyovwxnbkfezzx").withPort(357757853).withScheme(Scheme.HTTPS)) + .withInitialDelaySeconds(1673017156).withPeriodSeconds(215949715).withSuccessThreshold(1640359024) + .withTcpSocket(new ContainerAppProbeTcpSocket().withHost("bzbomvzzbtdcqvpn").withPort(1622829877)) + .withTerminationGracePeriodSeconds(2588743107415946739L).withTimeoutSeconds(277819924) + .withType(Type.STARTUP); model = BinaryData.fromObject(model).toObject(ContainerAppProbe.class); - Assertions.assertEquals(1317619791, model.failureThreshold()); - Assertions.assertEquals("kzzlvmbmpaxmodf", model.httpGet().host()); - Assertions.assertEquals("fy", model.httpGet().httpHeaders().get(0).name()); - Assertions.assertEquals("sbpfvmwyhr", model.httpGet().httpHeaders().get(0).value()); - Assertions.assertEquals("uyfta", model.httpGet().path()); - Assertions.assertEquals(1378791233, model.httpGet().port()); + Assertions.assertEquals(1290029311, model.failureThreshold()); + Assertions.assertEquals("ycnunvjsrtk", model.httpGet().host()); + Assertions.assertEquals("nopqgikyzirtx", model.httpGet().httpHeaders().get(0).name()); + Assertions.assertEquals("yuxzejntpsewgi", model.httpGet().httpHeaders().get(0).value()); + Assertions.assertEquals("luyovwxnbkfezzx", model.httpGet().path()); + Assertions.assertEquals(357757853, model.httpGet().port()); Assertions.assertEquals(Scheme.HTTPS, model.httpGet().scheme()); - Assertions.assertEquals(369562825, model.initialDelaySeconds()); - Assertions.assertEquals(1040923285, model.periodSeconds()); - Assertions.assertEquals(842537299, model.successThreshold()); - Assertions.assertEquals("nubexk", model.tcpSocket().host()); - Assertions.assertEquals(1989438546, model.tcpSocket().port()); - Assertions.assertEquals(5387832879483701613L, model.terminationGracePeriodSeconds()); - Assertions.assertEquals(487764269, model.timeoutSeconds()); + Assertions.assertEquals(1673017156, model.initialDelaySeconds()); + Assertions.assertEquals(215949715, model.periodSeconds()); + Assertions.assertEquals(1640359024, model.successThreshold()); + Assertions.assertEquals("bzbomvzzbtdcqvpn", model.tcpSocket().host()); + Assertions.assertEquals(1622829877, model.tcpSocket().port()); + Assertions.assertEquals(2588743107415946739L, model.terminationGracePeriodSeconds()); + Assertions.assertEquals(277819924, model.timeoutSeconds()); Assertions.assertEquals(Type.STARTUP, model.type()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteWithResponseMockTests.java index 80c20b9b23f16..3bbb3e8099b16 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsAuthConfigsDeleteWithResponseMockTests.java @@ -32,32 +32,20 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .containerAppsAuthConfigs() - .deleteWithResponse("mfp", "hojeevy", "yhsgz", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.containerAppsAuthConfigs().deleteWithResponse("d", "kkyihzt", "eq", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteMockTests.java index 75f04d9e9f246..1801b268202bd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.containerApps().delete("ilkmk", "holvdn", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.containerApps().delete("o", "sxxh", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorWithResponseMockTests.java index 579b6aa1dd416..75ea345b4e869 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsGetDetectorWithResponseMockTests.java @@ -30,53 +30,40 @@ public void testGetDetectorWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"metadata\":{\"id\":\"omzisglrrcz\",\"name\":\"k\",\"description\":\"ltn\",\"author\":\"dhqoawj\",\"category\":\"yueayfbpcmsp\",\"supportTopicList\":[{\"id\":\"rueqthwm\",\"pesId\":\"mbscbbx\"},{\"id\":\"dhxiidlopedbwd\",\"pesId\":\"qyybxubmdnafcbq\"}],\"analysisTypes\":[\"mjel\"],\"type\":\"acigel\",\"score\":74.37067},\"dataset\":[{\"table\":{\"tableName\":\"vwzkj\",\"columns\":[{},{},{},{}],\"rows\":[\"dataonrlkwzdqybxceak\",\"datacptsoqfyiase\",\"datachkrttzr\",\"datazisgykiuemvanb\"]},\"renderingProperties\":{\"type\":974039641,\"title\":\"nrxxbsojklin\",\"description\":\"dptysprqs\",\"isVisible\":true}},{\"table\":{\"tableName\":\"jp\",\"columns\":[{},{},{},{}],\"rows\":[\"datag\",\"dataliufiqwoyxq\"]},\"renderingProperties\":{\"type\":711463850,\"title\":\"hhoucq\",\"description\":\"oj\",\"isVisible\":false}},{\"table\":{\"tableName\":\"zdcgdzbenr\",\"columns\":[{},{},{}],\"rows\":[\"dataetzqd\",\"datatjwfljhznamtua\",\"datamzwcjjncqt\",\"dataz\"]},\"renderingProperties\":{\"type\":1502507838,\"title\":\"bgatzu\",\"description\":\"bxn\",\"isVisible\":false}},{\"table\":{\"tableName\":\"ggahtt\",\"columns\":[{},{}],\"rows\":[\"dataajqfutlx\"]},\"renderingProperties\":{\"type\":1442149444,\"title\":\"sunwqrjzfrgqhaoh\",\"description\":\"buocnjr\",\"isVisible\":true}}],\"status\":{\"message\":\"yryxameblydyv\",\"statusId\":2057319587},\"dataProviderMetadata\":{\"providerName\":\"ocxnehvsmtodl\",\"propertyBag\":[{\"name\":\"pucygvoavyunss\",\"value\":\"ghiee\"},{\"name\":\"lgvvpaseksgbu\",\"value\":\"ntuygdhgaqipir\"},{\"name\":\"wrq\",\"value\":\"ulopmjnlexwhcb\"}]}},\"id\":\"pibkephuu\",\"name\":\"erctatoyin\",\"type\":\"qpbrlc\"}"; + String responseStr + = "{\"properties\":{\"metadata\":{\"id\":\"xxgfbbmtlpqagyn\",\"name\":\"prnzc\",\"description\":\"incryqxzx\",\"author\":\"zi\",\"category\":\"qimiymqr\",\"supportTopicList\":[{\"id\":\"hfupetasvvoqsbpk\",\"pesId\":\"anfkgxsyaowuzow\"},{\"id\":\"ohdkcprgukx\",\"pesId\":\"tiochlutixmqr\"}],\"analysisTypes\":[\"izcbfzmcrunfhiuc\",\"mfbcpaqktkrum\",\"u\",\"dkyzbfvxov\"],\"type\":\"xiuxqggvqrnhy\",\"score\":87.82275},\"dataset\":[{\"table\":{\"tableName\":\"ggjh\",\"columns\":[{},{},{}],\"rows\":[\"dataqrkijpeuq\",\"datasdxeqztvxwmwwmjs\",\"dataen\",\"datawwa\"]},\"renderingProperties\":{\"type\":308965260,\"title\":\"qioulndhzyoeojht\",\"description\":\"lhsvidmytzlnglxp\",\"isVisible\":true}}],\"status\":{\"message\":\"anfbc\",\"statusId\":536650888},\"dataProviderMetadata\":{\"providerName\":\"ywv\",\"propertyBag\":[{\"name\":\"vjrktpgaeukyawoh\",\"value\":\"w\"},{\"name\":\"nucsk\",\"value\":\"sidsjtd\"}]}},\"id\":\"bninjgazlsvbzfc\",\"name\":\"uoeedwjcci\",\"type\":\"lhsyekrdrenxolr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Diagnostics response = - manager - .containerAppsDiagnostics() - .getDetectorWithResponse("iowxi", "spnxwqagnepzw", "klsbsbqqqagw", com.azure.core.util.Context.NONE) - .getValue(); + Diagnostics response = manager.containerAppsDiagnostics() + .getDetectorWithResponse("kkld", "rcwfcmfcnrjajq", "atxjtiel", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("mjel", response.properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("vwzkj", response.properties().dataset().get(0).table().tableName()); - Assertions.assertEquals(974039641, response.properties().dataset().get(0).renderingProperties().type()); - Assertions.assertEquals("nrxxbsojklin", response.properties().dataset().get(0).renderingProperties().title()); - Assertions - .assertEquals("dptysprqs", response.properties().dataset().get(0).renderingProperties().description()); + Assertions.assertEquals("izcbfzmcrunfhiuc", response.properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("ggjh", response.properties().dataset().get(0).table().tableName()); + Assertions.assertEquals(308965260, response.properties().dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("qioulndhzyoeojht", + response.properties().dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("lhsvidmytzlnglxp", + response.properties().dataset().get(0).renderingProperties().description()); Assertions.assertEquals(true, response.properties().dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("yryxameblydyv", response.properties().status().message()); - Assertions.assertEquals(2057319587, response.properties().status().statusId()); - Assertions.assertEquals("ocxnehvsmtodl", response.properties().dataProviderMetadata().providerName()); - Assertions - .assertEquals("pucygvoavyunss", response.properties().dataProviderMetadata().propertyBag().get(0).name()); - Assertions.assertEquals("ghiee", response.properties().dataProviderMetadata().propertyBag().get(0).value()); + Assertions.assertEquals("anfbc", response.properties().status().message()); + Assertions.assertEquals(536650888, response.properties().status().statusId()); + Assertions.assertEquals("ywv", response.properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("vjrktpgaeukyawoh", + response.properties().dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("w", response.properties().dataProviderMetadata().propertyBag().get(0).value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsMockTests.java index 3736d16590e06..fc0c158dadd8f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsDiagnosticsListDetectorsMockTests.java @@ -31,63 +31,44 @@ public void testListDetectors() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"gvgovpbbttefjo\",\"name\":\"ssqyzqedik\",\"description\":\"rdbiqmrjge\",\"author\":\"fqlggw\",\"category\":\"wzcxmjpbyephmgt\",\"supportTopicList\":[{\"id\":\"rc\",\"pesId\":\"fqip\"}],\"analysisTypes\":[\"npomy\"],\"type\":\"cabvnuil\",\"score\":64.833626},\"dataset\":[{\"table\":{\"tableName\":\"paugmrmfjlrxwt\",\"columns\":[{},{}],\"rows\":[\"datafkvcisi\",\"datamoaedsxj\",\"datauivedwcgyeewxeiq\",\"datapsmgo\"]},\"renderingProperties\":{\"type\":1932184921,\"title\":\"ljdlrgmspl\",\"description\":\"auf\",\"isVisible\":true}},{\"table\":{\"tableName\":\"new\",\"columns\":[{}],\"rows\":[\"datampqanxrjkixtwbt\",\"dataoypnyg\",\"datashxcylhkg\",\"datansghp\"]},\"renderingProperties\":{\"type\":1324259721,\"title\":\"drwjjkhvyom\",\"description\":\"luzvxnq\",\"isVisible\":false}},{\"table\":{\"tableName\":\"pd\",\"columns\":[{},{},{}],\"rows\":[\"dataisqcssffxuifmcs\"]},\"renderingProperties\":{\"type\":861169012,\"title\":\"dqzrdzsyloll\",\"description\":\"rc\",\"isVisible\":false}}],\"status\":{\"message\":\"xzji\",\"statusId\":1590781119},\"dataProviderMetadata\":{\"providerName\":\"urkihci\",\"propertyBag\":[{\"name\":\"fx\",\"value\":\"c\"},{\"name\":\"nbkkjanurnnq\",\"value\":\"q\"}]}},\"id\":\"pizxqltgrdogyp\",\"name\":\"rxvbfihwuh\",\"type\":\"ctafsrbxrblm\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"zarhzvqnsqktcmbj\",\"name\":\"zoslpkybtgl\",\"description\":\"zp\",\"author\":\"jsqjcemqbm\",\"category\":\"vqar\",\"supportTopicList\":[{\"id\":\"qreblui\",\"pesId\":\"bwxsfgtdm\"},{\"id\":\"xekr\",\"pesId\":\"k\"}],\"analysisTypes\":[\"ud\",\"twaokb\"],\"type\":\"lyttaaknwfr\",\"score\":69.09618},\"dataset\":[{\"table\":{\"tableName\":\"dujdigatoleksc\",\"columns\":[{},{},{}],\"rows\":[\"datanqimwbzxpdcldp\",\"dataawnsnlaimou\",\"dataw\",\"datasqmu\"]},\"renderingProperties\":{\"type\":2118230542,\"title\":\"ibicz\",\"description\":\"swswjrkbq\",\"isVisible\":true}},{\"table\":{\"tableName\":\"qqvyfscyrfw\",\"columns\":[{},{}],\"rows\":[\"dataogfuyzwvbhlim\",\"datayqe\",\"datar\"]},\"renderingProperties\":{\"type\":1779759130,\"title\":\"kcdrdaasaxxobsm\",\"description\":\"wiyjvzuko\",\"isVisible\":false}}],\"status\":{\"message\":\"nvzmlnkoywsx\",\"statusId\":1909708055},\"dataProviderMetadata\":{\"providerName\":\"qqax\",\"propertyBag\":[{\"name\":\"mc\",\"value\":\"dndoabhjxw\"}]}},\"id\":\"weuipmpvksmi\",\"name\":\"nsqxtltc\",\"type\":\"krdpqgfhyrfr\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.containerAppsDiagnostics().listDetectors("xxkubvphavp", "h", com.azure.core.util.Context.NONE); + PagedIterable response = manager.containerAppsDiagnostics().listDetectors("gmihzpimcqr", + "nxtminklog", com.azure.core.util.Context.NONE); - Assertions.assertEquals("npomy", response.iterator().next().properties().metadata().analysisTypes().get(0)); - Assertions - .assertEquals( - "paugmrmfjlrxwt", response.iterator().next().properties().dataset().get(0).table().tableName()); - Assertions - .assertEquals( - 1932184921, response.iterator().next().properties().dataset().get(0).renderingProperties().type()); - Assertions - .assertEquals( - "ljdlrgmspl", response.iterator().next().properties().dataset().get(0).renderingProperties().title()); - Assertions - .assertEquals( - "auf", response.iterator().next().properties().dataset().get(0).renderingProperties().description()); - Assertions - .assertEquals( - true, response.iterator().next().properties().dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("xzji", response.iterator().next().properties().status().message()); - Assertions.assertEquals(1590781119, response.iterator().next().properties().status().statusId()); - Assertions - .assertEquals("urkihci", response.iterator().next().properties().dataProviderMetadata().providerName()); - Assertions - .assertEquals( - "fx", response.iterator().next().properties().dataProviderMetadata().propertyBag().get(0).name()); - Assertions - .assertEquals( - "c", response.iterator().next().properties().dataProviderMetadata().propertyBag().get(0).value()); + Assertions.assertEquals("ud", response.iterator().next().properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("dujdigatoleksc", + response.iterator().next().properties().dataset().get(0).table().tableName()); + Assertions.assertEquals(2118230542, + response.iterator().next().properties().dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("ibicz", + response.iterator().next().properties().dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("swswjrkbq", + response.iterator().next().properties().dataset().get(0).renderingProperties().description()); + Assertions.assertEquals(true, + response.iterator().next().properties().dataset().get(0).renderingProperties().isVisible()); + Assertions.assertEquals("nvzmlnkoywsx", response.iterator().next().properties().status().message()); + Assertions.assertEquals(1909708055, response.iterator().next().properties().status().statusId()); + Assertions.assertEquals("qqax", response.iterator().next().properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("mc", + response.iterator().next().properties().dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("dndoabhjxw", + response.iterator().next().properties().dataProviderMetadata().propertyBag().get(0).value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaWithResponseMockTests.java index 9adf3b9355043..a6042c4543bc3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasGetReplicaWithResponseMockTests.java @@ -30,50 +30,36 @@ public void testGetReplicaWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"createdTime\":\"2021-04-22T19:07:29Z\",\"runningState\":\"Running\",\"runningStateDetails\":\"nsq\",\"containers\":[{\"name\":\"comlikytwvczc\",\"containerId\":\"k\",\"ready\":false,\"started\":false,\"restartCount\":789888473,\"runningState\":\"Terminated\",\"runningStateDetails\":\"vhb\",\"logStreamEndpoint\":\"nfxtgdd\",\"execEndpoint\":\"th\"}],\"initContainers\":[{\"name\":\"naoyank\",\"containerId\":\"eqswanklty\",\"ready\":false,\"started\":true,\"restartCount\":1174469585,\"runningState\":\"Terminated\",\"runningStateDetails\":\"drlktg\",\"logStreamEndpoint\":\"sggux\",\"execEndpoint\":\"mlwywaeeczg\"}]},\"id\":\"bukklels\",\"name\":\"xblycsxzuj\",\"type\":\"srlsmd\"}"; + String responseStr + = "{\"properties\":{\"createdTime\":\"2021-11-27T21:15:44Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"epdjxqeskoyn\",\"containers\":[{\"name\":\"pckaewse\",\"containerId\":\"eskwxeg\",\"ready\":false,\"started\":false,\"restartCount\":665890978,\"runningState\":\"Terminated\",\"runningStateDetails\":\"tmjtsghp\",\"logStreamEndpoint\":\"bcpzarpzeqacdldt\",\"execEndpoint\":\"pypefcpczshnuq\"},{\"name\":\"aizu\",\"containerId\":\"kh\",\"ready\":false,\"started\":true,\"restartCount\":1048467054,\"runningState\":\"Running\",\"runningStateDetails\":\"vtvegwqiukvzw\",\"logStreamEndpoint\":\"wtthaokgksk\",\"execEndpoint\":\"vb\"},{\"name\":\"hajqfuk\",\"containerId\":\"expgeumilhwu\",\"ready\":false,\"started\":true,\"restartCount\":1014412406,\"runningState\":\"Terminated\",\"runningStateDetails\":\"ofninbdbz\",\"logStreamEndpoint\":\"cw\",\"execEndpoint\":\"rsmpcbbprtuga\"}],\"initContainers\":[{\"name\":\"cyksivmfo\",\"containerId\":\"rt\",\"ready\":true,\"started\":true,\"restartCount\":1811675094,\"runningState\":\"Running\",\"runningStateDetails\":\"cwjjxsgmbawvif\",\"logStreamEndpoint\":\"kecifhocjxwklo\",\"execEndpoint\":\"rvtxvcmufunlc\"},{\"name\":\"xvi\",\"containerId\":\"eyngjg\",\"ready\":false,\"started\":false,\"restartCount\":563018487,\"runningState\":\"Running\",\"runningStateDetails\":\"pmcrdcuelj\",\"logStreamEndpoint\":\"ahxm\",\"execEndpoint\":\"ryarvsxzqb\"}]},\"id\":\"cjkay\",\"name\":\"pth\",\"type\":\"odubtlmjt\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Replica response = - manager - .containerAppsRevisionReplicas() - .getReplicaWithResponse("rhpw", "gddeimaw", "o", "gkkumuikjcj", com.azure.core.util.Context.NONE) - .getValue(); + Replica response = manager.containerAppsRevisionReplicas().getReplicaWithResponse("cvumepjpbi", "nzpphepife", + "leqirccjclykcgxv", "pjlvczuoda", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("comlikytwvczc", response.containers().get(0).name()); - Assertions.assertEquals("k", response.containers().get(0).containerId()); + Assertions.assertEquals("pckaewse", response.containers().get(0).name()); + Assertions.assertEquals("eskwxeg", response.containers().get(0).containerId()); Assertions.assertEquals(false, response.containers().get(0).ready()); Assertions.assertEquals(false, response.containers().get(0).started()); - Assertions.assertEquals(789888473, response.containers().get(0).restartCount()); - Assertions.assertEquals("naoyank", response.initContainers().get(0).name()); - Assertions.assertEquals("eqswanklty", response.initContainers().get(0).containerId()); - Assertions.assertEquals(false, response.initContainers().get(0).ready()); + Assertions.assertEquals(665890978, response.containers().get(0).restartCount()); + Assertions.assertEquals("cyksivmfo", response.initContainers().get(0).name()); + Assertions.assertEquals("rt", response.initContainers().get(0).containerId()); + Assertions.assertEquals(true, response.initContainers().get(0).ready()); Assertions.assertEquals(true, response.initContainers().get(0).started()); - Assertions.assertEquals(1174469585, response.initContainers().get(0).restartCount()); + Assertions.assertEquals(1811675094, response.initContainers().get(0).restartCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasWithResponseMockTests.java index 188a36db323b3..28d4a830b2301 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionReplicasListReplicasWithResponseMockTests.java @@ -29,40 +29,27 @@ public void testListReplicasWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"createdTime\":\"2021-08-20T05:45:20Z\",\"runningState\":\"Unknown\",\"runningStateDetails\":\"vuhx\",\"containers\":[{},{},{},{}],\"initContainers\":[{},{},{}]},\"id\":\"tznabao\",\"name\":\"nslujd\",\"type\":\"ltymkmvguihywart\"},{\"properties\":{\"createdTime\":\"2021-01-15T23:22:51Z\",\"runningState\":\"Running\",\"runningStateDetails\":\"ky\",\"containers\":[{}],\"initContainers\":[{}]},\"id\":\"j\",\"name\":\"emmucfxh\",\"type\":\"kkflrmymyincqlhr\"},{\"properties\":{\"createdTime\":\"2021-10-10T16:10:58Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"iiovgqcgxu\",\"containers\":[{},{},{},{}],\"initContainers\":[{},{},{}]},\"id\":\"totiowlxteqdptjg\",\"name\":\"dtguk\",\"type\":\"anblwphqlkccu\"},{\"properties\":{\"createdTime\":\"2021-06-12T18:54:42Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"hoi\",\"containers\":[{},{}],\"initContainers\":[{},{},{}]},\"id\":\"iip\",\"name\":\"glvawuwzdufypivl\",\"type\":\"bbjpmcubkmif\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"createdTime\":\"2021-01-16T05:14:58Z\",\"runningState\":\"Running\",\"runningStateDetails\":\"vjnzdpvocojhpcna\",\"containers\":[{}],\"initContainers\":[{},{},{},{}]},\"id\":\"ggytexvzilmhivz\",\"name\":\"wwwncknr\",\"type\":\"dajlskzpt\"},{\"properties\":{\"createdTime\":\"2021-09-18T03:15:46Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"cyrthxqlehm\",\"containers\":[{}],\"initContainers\":[{},{},{}]},\"id\":\"n\",\"name\":\"ehokamvfej\",\"type\":\"qnttmbq\"},{\"properties\":{\"createdTime\":\"2021-02-03T14:51:39Z\",\"runningState\":\"Unknown\",\"runningStateDetails\":\"okpysthhzagj\",\"containers\":[{},{},{}],\"initContainers\":[{},{},{}]},\"id\":\"hgenuzejgvkveb\",\"name\":\"qszllrz\",\"type\":\"smmd\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + ReplicaCollection response = manager.containerAppsRevisionReplicas() + .listReplicasWithResponse("blioskkfmkmf", "jxyxgb", "kqvjcteoedlr", com.azure.core.util.Context.NONE) + .getValue(); - ReplicaCollection response = - manager - .containerAppsRevisionReplicas() - .listReplicasWithResponse( - "sqplpvmjcd", "ewbidyvteowxv", "piudeugfsxzecpa", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionWithResponseMockTests.java index cbeef4941d5ad..382a79bd13792 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsActivateRevisionWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testActivateRevisionWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .containerAppsRevisions() - .activateRevisionWithResponse("obbpihehc", "cy", "mrqbrjbbmpxdlv", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.containerAppsRevisions().activateRevisionWithResponse("htkbtnq", "rngl", "mbiipsnawwlqk", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionWithResponseMockTests.java index abbcebd7f7cd8..9ac9853466e61 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsDeactivateRevisionWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeactivateRevisionWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .containerAppsRevisions() - .deactivateRevisionWithResponse("kfrexcrseqwjks", "hud", "zhxogjggsvo", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.containerAppsRevisions().deactivateRevisionWithResponse("nxhhl", "xricctkwmuqq", "ajxeiygle", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionWithResponseMockTests.java index adfd8cfe9a9b4..623f6e937a241 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsRevisionsRestartRevisionWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testRestartRevisionWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .containerAppsRevisions() - .restartRevisionWithResponse("jkxibda", "hrkmdyomkxfbvfbh", "y", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.containerAppsRevisions().restartRevisionWithResponse("rwvaexhdc", "rceqnkbrupob", "hdmljz", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteMockTests.java index 2f123cd1fd8a9..be01b831fcb98 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerAppsSourceControlsDeleteMockTests.java @@ -32,32 +32,21 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .containerAppsSourceControls() - .delete("efgnaavuagnt", "taoutnpdct", "hspfefyihd", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.containerAppsSourceControls().delete("lpfliwoyn", "uuzhw", "ladpcmhjha", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryTests.java new file mode 100644 index 0000000000000..04215d89f660c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistry; +import org.junit.jupiter.api.Assertions; + +public final class ContainerRegistryTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ContainerRegistry model + = BinaryData.fromString("{\"containerRegistryServer\":\"x\",\"identityResourceId\":\"it\"}") + .toObject(ContainerRegistry.class); + Assertions.assertEquals("x", model.containerRegistryServer()); + Assertions.assertEquals("it", model.identityResourceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ContainerRegistry model = new ContainerRegistry().withContainerRegistryServer("x").withIdentityResourceId("it"); + model = BinaryData.fromObject(model).toObject(ContainerRegistry.class); + Assertions.assertEquals("x", model.containerRegistryServer()); + Assertions.assertEquals("it", model.identityResourceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryWithCustomImageTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryWithCustomImageTests.java new file mode 100644 index 0000000000000..afff40faacf6d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerRegistryWithCustomImageTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.ContainerRegistryWithCustomImage; +import org.junit.jupiter.api.Assertions; + +public final class ContainerRegistryWithCustomImageTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ContainerRegistryWithCustomImage model + = BinaryData.fromString("{\"server\":\"bpodxunkbebxm\",\"image\":\"yyntwl\"}") + .toObject(ContainerRegistryWithCustomImage.class); + Assertions.assertEquals("bpodxunkbebxm", model.server()); + Assertions.assertEquals("yyntwl", model.image()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ContainerRegistryWithCustomImage model + = new ContainerRegistryWithCustomImage().withServer("bpodxunkbebxm").withImage("yyntwl"); + model = BinaryData.fromObject(model).toObject(ContainerRegistryWithCustomImage.class); + Assertions.assertEquals("bpodxunkbebxm", model.server()); + Assertions.assertEquals("yyntwl", model.image()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerResourcesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerResourcesTests.java index 421a48460236b..650b5f062a0a5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerResourcesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ContainerResourcesTests.java @@ -11,20 +11,18 @@ public final class ContainerResourcesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ContainerResources model = - BinaryData - .fromString( - "{\"cpu\":4.886296094520515,\"memory\":\"muwlauwzizxbm\",\"ephemeralStorage\":\"cjefuzmu\"}") - .toObject(ContainerResources.class); - Assertions.assertEquals(4.886296094520515D, model.cpu()); - Assertions.assertEquals("muwlauwzizxbm", model.memory()); + ContainerResources model = BinaryData + .fromString("{\"cpu\":95.55475275329178,\"memory\":\"hcexdrrvqa\",\"ephemeralStorage\":\"kghtpwijnh\"}") + .toObject(ContainerResources.class); + Assertions.assertEquals(95.55475275329178D, model.cpu()); + Assertions.assertEquals("hcexdrrvqa", model.memory()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ContainerResources model = new ContainerResources().withCpu(4.886296094520515D).withMemory("muwlauwzizxbm"); + ContainerResources model = new ContainerResources().withCpu(95.55475275329178D).withMemory("hcexdrrvqa"); model = BinaryData.fromObject(model).toObject(ContainerResources.class); - Assertions.assertEquals(4.886296094520515D, model.cpu()); - Assertions.assertEquals("muwlauwzizxbm", model.memory()); + Assertions.assertEquals(95.55475275329178D, model.cpu()); + Assertions.assertEquals("hcexdrrvqa", model.memory()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CookieExpirationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CookieExpirationTests.java index a520ea88850d8..84c98c134e832 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CookieExpirationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CookieExpirationTests.java @@ -12,20 +12,19 @@ public final class CookieExpirationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CookieExpiration model = - BinaryData - .fromString("{\"convention\":\"FixedTime\",\"timeToExpiration\":\"y\"}") + CookieExpiration model + = BinaryData.fromString("{\"convention\":\"IdentityProviderDerived\",\"timeToExpiration\":\"dlwtgrhpdj\"}") .toObject(CookieExpiration.class); - Assertions.assertEquals(CookieExpirationConvention.FIXED_TIME, model.convention()); - Assertions.assertEquals("y", model.timeToExpiration()); + Assertions.assertEquals(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED, model.convention()); + Assertions.assertEquals("dlwtgrhpdj", model.timeToExpiration()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CookieExpiration model = - new CookieExpiration().withConvention(CookieExpirationConvention.FIXED_TIME).withTimeToExpiration("y"); + CookieExpiration model = new CookieExpiration() + .withConvention(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED).withTimeToExpiration("dlwtgrhpdj"); model = BinaryData.fromObject(model).toObject(CookieExpiration.class); - Assertions.assertEquals(CookieExpirationConvention.FIXED_TIME, model.convention()); - Assertions.assertEquals("y", model.timeToExpiration()); + Assertions.assertEquals(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED, model.convention()); + Assertions.assertEquals("dlwtgrhpdj", model.timeToExpiration()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CustomDomainTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CustomDomainTests.java index 8d6738b7f3ca9..df779637ad7da 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CustomDomainTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/CustomDomainTests.java @@ -12,25 +12,21 @@ public final class CustomDomainTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CustomDomain model = - BinaryData - .fromString("{\"name\":\"ac\",\"bindingType\":\"SniEnabled\",\"certificateId\":\"fqrhhuaopppc\"}") - .toObject(CustomDomain.class); - Assertions.assertEquals("ac", model.name()); - Assertions.assertEquals(BindingType.SNI_ENABLED, model.bindingType()); - Assertions.assertEquals("fqrhhuaopppc", model.certificateId()); + CustomDomain model = BinaryData + .fromString("{\"name\":\"owqkdwytisi\",\"bindingType\":\"Disabled\",\"certificateId\":\"gpikpzimejza\"}") + .toObject(CustomDomain.class); + Assertions.assertEquals("owqkdwytisi", model.name()); + Assertions.assertEquals(BindingType.DISABLED, model.bindingType()); + Assertions.assertEquals("gpikpzimejza", model.certificateId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CustomDomain model = - new CustomDomain() - .withName("ac") - .withBindingType(BindingType.SNI_ENABLED) - .withCertificateId("fqrhhuaopppc"); + CustomDomain model = new CustomDomain().withName("owqkdwytisi").withBindingType(BindingType.DISABLED) + .withCertificateId("gpikpzimejza"); model = BinaryData.fromObject(model).toObject(CustomDomain.class); - Assertions.assertEquals("ac", model.name()); - Assertions.assertEquals(BindingType.SNI_ENABLED, model.bindingType()); - Assertions.assertEquals("fqrhhuaopppc", model.certificateId()); + Assertions.assertEquals("owqkdwytisi", model.name()); + Assertions.assertEquals(BindingType.DISABLED, model.bindingType()); + Assertions.assertEquals("gpikpzimejza", model.certificateId()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCollectionTests.java new file mode 100644 index 0000000000000..30423923b704d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCollectionTests.java @@ -0,0 +1,144 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPoliciesCollection; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPoliciesCollectionTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPoliciesCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":1701983547,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1998431294},\"circuitBreakerPolicy\":{\"consecutiveErrors\":672713653,\"timeoutInSeconds\":1300202975,\"intervalInSeconds\":64736512}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":499319607,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":972204015},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1265182150,\"timeoutInSeconds\":112594464,\"intervalInSeconds\":9633412}}},\"id\":\"kkhminqcymczngn\",\"name\":\"dxxewuninv\",\"type\":\"db\"},{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":709462942,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1900039321},\"circuitBreakerPolicy\":{\"consecutiveErrors\":313351461,\"timeoutInSeconds\":720636191,\"intervalInSeconds\":1857605955}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":217232947,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":485950763},\"circuitBreakerPolicy\":{\"consecutiveErrors\":334709748,\"timeoutInSeconds\":297911426,\"intervalInSeconds\":1148214254}}},\"id\":\"napxbannovv\",\"name\":\"xc\",\"type\":\"ytprwnwvroev\"},{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":1307302388,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":67510907},\"circuitBreakerPolicy\":{\"consecutiveErrors\":491107905,\"timeoutInSeconds\":725083050,\"intervalInSeconds\":1312539760}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":810647712,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1031456845},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1458275962,\"timeoutInSeconds\":952405971,\"intervalInSeconds\":1220621790}}},\"id\":\"d\",\"name\":\"nazpmk\",\"type\":\"lmv\"},{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":343155487,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1939182119},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1553253878,\"timeoutInSeconds\":1749137224,\"intervalInSeconds\":1704256415}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":632448787,\"retryBackOff\":{}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1848154538},\"circuitBreakerPolicy\":{\"consecutiveErrors\":530417864,\"timeoutInSeconds\":540353819,\"intervalInSeconds\":796010051}}},\"id\":\"zqcyknap\",\"name\":\"ofyuicd\",\"type\":\"zb\"}],\"nextLink\":\"bwwg\"}") + .toObject(DaprComponentResiliencyPoliciesCollection.class); + Assertions.assertEquals(1701983547, model.value().get(0).inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1998431294, + model.value().get(0).inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(672713653, + model.value().get(0).inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1300202975, + model.value().get(0).inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(64736512, + model.value().get(0).inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(499319607, model.value().get(0).outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(972204015, + model.value().get(0).outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1265182150, + model.value().get(0).outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(112594464, + model.value().get(0).outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(9633412, + model.value().get(0).outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPoliciesCollection model + = new DaprComponentResiliencyPoliciesCollection().withValue(Arrays.asList( + new DaprComponentResiliencyPolicyInner() + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(1701983547) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1998431294)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(672713653).withTimeoutInSeconds(1300202975) + .withIntervalInSeconds(64736512))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(499319607) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(972204015)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(1265182150).withTimeoutInSeconds(112594464) + .withIntervalInSeconds(9633412))), + new DaprComponentResiliencyPolicyInner() + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(709462942) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1900039321)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(313351461).withTimeoutInSeconds(720636191) + .withIntervalInSeconds(1857605955))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(217232947) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(485950763)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(334709748).withTimeoutInSeconds(297911426) + .withIntervalInSeconds(1148214254))), + new DaprComponentResiliencyPolicyInner() + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(1307302388) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(67510907)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(491107905).withTimeoutInSeconds(725083050) + .withIntervalInSeconds(1312539760))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(810647712) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1031456845)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(1458275962).withTimeoutInSeconds(952405971) + .withIntervalInSeconds(1220621790))), + new DaprComponentResiliencyPolicyInner() + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(343155487) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1939182119)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(1553253878).withTimeoutInSeconds(1749137224) + .withIntervalInSeconds(1704256415))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(632448787) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration())) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1848154538)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(530417864).withTimeoutInSeconds(540353819) + .withIntervalInSeconds(796010051))))); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPoliciesCollection.class); + Assertions.assertEquals(1701983547, model.value().get(0).inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1998431294, + model.value().get(0).inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(672713653, + model.value().get(0).inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1300202975, + model.value().get(0).inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(64736512, + model.value().get(0).inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(499319607, model.value().get(0).outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(972204015, + model.value().get(0).outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1265182150, + model.value().get(0).outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(112594464, + model.value().get(0).outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(9633412, + model.value().get(0).outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateWithResponseMockTests.java new file mode 100644 index 0000000000000..4fefee0af7071 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesCreateOrUpdateWithResponseMockTests.java @@ -0,0 +1,97 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicy; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprComponentResiliencyPoliciesCreateOrUpdateWithResponseMockTests { + @Test + public void testCreateOrUpdateWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":2061685284,\"retryBackOff\":{\"initialDelayInMilliseconds\":76340315,\"maxIntervalInMilliseconds\":1313860717}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1186020907},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1701022036,\"timeoutInSeconds\":2062030350,\"intervalInSeconds\":700693046}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":562282561,\"retryBackOff\":{\"initialDelayInMilliseconds\":644167930,\"maxIntervalInMilliseconds\":1581599883}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":285262475},\"circuitBreakerPolicy\":{\"consecutiveErrors\":620670470,\"timeoutInSeconds\":1906572737,\"intervalInSeconds\":445835887}}},\"id\":\"jod\",\"name\":\"cbjqqwmtqsm\",\"type\":\"xsazuxejgw\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DaprComponentResiliencyPolicy response = manager.daprComponentResiliencyPolicies().define("vctkbbx") + .withExistingDaprComponent("wetjtdrhutf", "oadtxopgehpadkmd", "gssz") + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(989359747) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(1895023582).withMaxIntervalInMilliseconds(1275197408))) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1486832256)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(18849329).withTimeoutInSeconds(946198346).withIntervalInSeconds(758730321))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(380797824) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(187800600).withMaxIntervalInMilliseconds(1759604216))) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1843031385)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(221407560).withTimeoutInSeconds(2145155757).withIntervalInSeconds(20348867))) + .create(); + + Assertions.assertEquals(2061685284, response.inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(76340315, + response.inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1313860717, + response.inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1186020907, response.inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1701022036, response.inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(2062030350, response.inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(700693046, response.inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(562282561, response.outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(644167930, + response.outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1581599883, + response.outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(285262475, response.outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(620670470, response.outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1906572737, response.outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(445835887, response.outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteWithResponseMockTests.java new file mode 100644 index 0000000000000..ac65275659d9a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesDeleteWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprComponentResiliencyPoliciesDeleteWithResponseMockTests { + @Test + public void testDeleteWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.daprComponentResiliencyPolicies().deleteWithResponse("itbfjtdy", "tnp", "facqo", "cqrq", + com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetWithResponseMockTests.java new file mode 100644 index 0000000000000..16792a762d38b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesGetWithResponseMockTests.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicy; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprComponentResiliencyPoliciesGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":2069524060,\"retryBackOff\":{\"initialDelayInMilliseconds\":104080595,\"maxIntervalInMilliseconds\":336537069}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":52257926},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1868056089,\"timeoutInSeconds\":702156768,\"intervalInSeconds\":1966996701}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":1592793613,\"retryBackOff\":{\"initialDelayInMilliseconds\":964008937,\"maxIntervalInMilliseconds\":1025699882}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":190570552},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1736402043,\"timeoutInSeconds\":1964383171,\"intervalInSeconds\":822001690}}},\"id\":\"proqk\",\"name\":\"mfxm\",\"type\":\"vprstv\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DaprComponentResiliencyPolicy response + = manager.daprComponentResiliencyPolicies().getWithResponse("ajpxecxqnwhscoza", "mvgxsmpknpwir", "ljfewxqo", + "oxudnmckap", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals(2069524060, response.inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(104080595, + response.inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(336537069, + response.inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(52257926, response.inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1868056089, response.inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(702156768, response.inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(1966996701, response.inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(1592793613, response.outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(964008937, + response.outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1025699882, + response.outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(190570552, response.outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1736402043, response.outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1964383171, response.outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(822001690, response.outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListMockTests.java new file mode 100644 index 0000000000000..762bc4efbb243 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPoliciesListMockTests.java @@ -0,0 +1,82 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicy; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprComponentResiliencyPoliciesListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":741089432,\"retryBackOff\":{\"initialDelayInMilliseconds\":1379724649,\"maxIntervalInMilliseconds\":433842501}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":2087912773},\"circuitBreakerPolicy\":{\"consecutiveErrors\":206815663,\"timeoutInSeconds\":495264872,\"intervalInSeconds\":910596652}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":529500532,\"retryBackOff\":{\"initialDelayInMilliseconds\":330883602,\"maxIntervalInMilliseconds\":1649946980}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":2137785308},\"circuitBreakerPolicy\":{\"consecutiveErrors\":550103154,\"timeoutInSeconds\":1960286905,\"intervalInSeconds\":781281199}}},\"id\":\"naoaqymhccto\",\"name\":\"uowyrnskbyhqu\",\"type\":\"czygxv\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.daprComponentResiliencyPolicies() + .list("qdnzyza", "atuwqkokbc", "othymgobl", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(741089432, response.iterator().next().inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1379724649, + response.iterator().next().inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(433842501, + response.iterator().next().inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(2087912773, + response.iterator().next().inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(206815663, + response.iterator().next().inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(495264872, + response.iterator().next().inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(910596652, + response.iterator().next().inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(529500532, response.iterator().next().outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(330883602, + response.iterator().next().outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1649946980, + response.iterator().next().outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(2137785308, + response.iterator().next().outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(550103154, + response.iterator().next().outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1960286905, + response.iterator().next().outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(781281199, + response.iterator().next().outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfigurationTests.java new file mode 100644 index 0000000000000..86047c5217ae6 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyCircuitBreakerPolicyConfigurationTests.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyCircuitBreakerPolicyConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration model = BinaryData + .fromString( + "{\"consecutiveErrors\":1053497499,\"timeoutInSeconds\":729007822,\"intervalInSeconds\":367318900}") + .toObject(DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration.class); + Assertions.assertEquals(1053497499, model.consecutiveErrors()); + Assertions.assertEquals(729007822, model.timeoutInSeconds()); + Assertions.assertEquals(367318900, model.intervalInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration model + = new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration().withConsecutiveErrors(1053497499) + .withTimeoutInSeconds(729007822).withIntervalInSeconds(367318900); + model = BinaryData.fromObject(model) + .toObject(DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration.class); + Assertions.assertEquals(1053497499, model.consecutiveErrors()); + Assertions.assertEquals(729007822, model.timeoutInSeconds()); + Assertions.assertEquals(367318900, model.intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyConfigurationTests.java new file mode 100644 index 0000000000000..6e0eae0eb5255 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyConfigurationTests.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyConfiguration model = BinaryData.fromString( + "{\"httpRetryPolicy\":{\"maxRetries\":36314711,\"retryBackOff\":{\"initialDelayInMilliseconds\":715935859,\"maxIntervalInMilliseconds\":2047901283}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1840957180},\"circuitBreakerPolicy\":{\"consecutiveErrors\":304822445,\"timeoutInSeconds\":1524765987,\"intervalInSeconds\":1267462998}}") + .toObject(DaprComponentResiliencyPolicyConfiguration.class); + Assertions.assertEquals(36314711, model.httpRetryPolicy().maxRetries()); + Assertions.assertEquals(715935859, model.httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(2047901283, model.httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1840957180, model.timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(304822445, model.circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1524765987, model.circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(1267462998, model.circuitBreakerPolicy().intervalInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyConfiguration model = new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(36314711) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(715935859).withMaxIntervalInMilliseconds(2047901283))) + .withTimeoutPolicy( + new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(1840957180)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(304822445).withTimeoutInSeconds(1524765987).withIntervalInSeconds(1267462998)); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPolicyConfiguration.class); + Assertions.assertEquals(36314711, model.httpRetryPolicy().maxRetries()); + Assertions.assertEquals(715935859, model.httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(2047901283, model.httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1840957180, model.timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(304822445, model.circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1524765987, model.circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(1267462998, model.circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryBackOffConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryBackOffConfigurationTests.java new file mode 100644 index 0000000000000..8febc17720aa1 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryBackOffConfigurationTests.java @@ -0,0 +1,30 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyHttpRetryBackOffConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration model = BinaryData + .fromString("{\"initialDelayInMilliseconds\":1297889112,\"maxIntervalInMilliseconds\":1150159087}") + .toObject(DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration.class); + Assertions.assertEquals(1297889112, model.initialDelayInMilliseconds()); + Assertions.assertEquals(1150159087, model.maxIntervalInMilliseconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration model + = new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(1297889112).withMaxIntervalInMilliseconds(1150159087); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration.class); + Assertions.assertEquals(1297889112, model.initialDelayInMilliseconds()); + Assertions.assertEquals(1150159087, model.maxIntervalInMilliseconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryPolicyConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryPolicyConfigurationTests.java new file mode 100644 index 0000000000000..0c2cdc232246d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyHttpRetryPolicyConfigurationTests.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyHttpRetryPolicyConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration model = BinaryData.fromString( + "{\"maxRetries\":1923932304,\"retryBackOff\":{\"initialDelayInMilliseconds\":366697291,\"maxIntervalInMilliseconds\":1294543038}}") + .toObject(DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration.class); + Assertions.assertEquals(1923932304, model.maxRetries()); + Assertions.assertEquals(366697291, model.retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1294543038, model.retryBackOff().maxIntervalInMilliseconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration model + = new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(1923932304) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(366697291).withMaxIntervalInMilliseconds(1294543038)); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration.class); + Assertions.assertEquals(1923932304, model.maxRetries()); + Assertions.assertEquals(366697291, model.retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1294543038, model.retryBackOff().maxIntervalInMilliseconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyInnerTests.java new file mode 100644 index 0000000000000..5a1d972a017f7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyInnerTests.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyInner; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyInner model = BinaryData.fromString( + "{\"properties\":{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":1801029414,\"retryBackOff\":{\"initialDelayInMilliseconds\":728731016,\"maxIntervalInMilliseconds\":1056355405}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":2137652282},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1436158527,\"timeoutInSeconds\":1159295504,\"intervalInSeconds\":881600617}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":2065822308,\"retryBackOff\":{\"initialDelayInMilliseconds\":643996998,\"maxIntervalInMilliseconds\":1253377600}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":798651388},\"circuitBreakerPolicy\":{\"consecutiveErrors\":1601507048,\"timeoutInSeconds\":64161749,\"intervalInSeconds\":919572814}}},\"id\":\"immoiroqboshbrag\",\"name\":\"pyyrmfs\",\"type\":\"bpav\"}") + .toObject(DaprComponentResiliencyPolicyInner.class); + Assertions.assertEquals(1801029414, model.inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(728731016, + model.inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1056355405, + model.inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(2137652282, model.inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1436158527, model.inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1159295504, model.inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(881600617, model.inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(2065822308, model.outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(643996998, + model.outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1253377600, + model.outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(798651388, model.outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1601507048, model.outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(64161749, model.outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(919572814, model.outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyInner model = new DaprComponentResiliencyPolicyInner() + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(1801029414) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(728731016).withMaxIntervalInMilliseconds(1056355405))) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(2137652282)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(1436158527).withTimeoutInSeconds(1159295504) + .withIntervalInSeconds(881600617))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(2065822308) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(643996998).withMaxIntervalInMilliseconds(1253377600))) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(798651388)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(1601507048).withTimeoutInSeconds(64161749) + .withIntervalInSeconds(919572814))); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPolicyInner.class); + Assertions.assertEquals(1801029414, model.inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(728731016, + model.inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1056355405, + model.inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(2137652282, model.inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1436158527, model.inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1159295504, model.inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(881600617, model.inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(2065822308, model.outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(643996998, + model.outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1253377600, + model.outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(798651388, model.outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(1601507048, model.outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(64161749, model.outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(919572814, model.outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyPropertiesTests.java new file mode 100644 index 0000000000000..e6b3dbf066a86 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyPropertiesTests.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprComponentResiliencyPolicyProperties; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyProperties model = BinaryData.fromString( + "{\"inboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":1818158355,\"retryBackOff\":{\"initialDelayInMilliseconds\":1487017150,\"maxIntervalInMilliseconds\":267572624}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1237918766},\"circuitBreakerPolicy\":{\"consecutiveErrors\":745209495,\"timeoutInSeconds\":353739432,\"intervalInSeconds\":1569477780}},\"outboundPolicy\":{\"httpRetryPolicy\":{\"maxRetries\":1931546764,\"retryBackOff\":{\"initialDelayInMilliseconds\":1973714604,\"maxIntervalInMilliseconds\":1422758538}},\"timeoutPolicy\":{\"responseTimeoutInSeconds\":1903849017},\"circuitBreakerPolicy\":{\"consecutiveErrors\":344097144,\"timeoutInSeconds\":1836074224,\"intervalInSeconds\":913638656}}}") + .toObject(DaprComponentResiliencyPolicyProperties.class); + Assertions.assertEquals(1818158355, model.inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1487017150, + model.inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(267572624, + model.inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1237918766, model.inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(745209495, model.inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(353739432, model.inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(1569477780, model.inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(1931546764, model.outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1973714604, + model.outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1422758538, + model.outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1903849017, model.outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(344097144, model.outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1836074224, model.outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(913638656, model.outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyProperties model = new DaprComponentResiliencyPolicyProperties() + .withInboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(1818158355) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(1487017150).withMaxIntervalInMilliseconds(267572624))) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1237918766)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(745209495).withTimeoutInSeconds(353739432) + .withIntervalInSeconds(1569477780))) + .withOutboundPolicy(new DaprComponentResiliencyPolicyConfiguration() + .withHttpRetryPolicy( + new DaprComponentResiliencyPolicyHttpRetryPolicyConfiguration().withMaxRetries(1931546764) + .withRetryBackOff(new DaprComponentResiliencyPolicyHttpRetryBackOffConfiguration() + .withInitialDelayInMilliseconds(1973714604).withMaxIntervalInMilliseconds(1422758538))) + .withTimeoutPolicy(new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration() + .withResponseTimeoutInSeconds(1903849017)) + .withCircuitBreakerPolicy(new DaprComponentResiliencyPolicyCircuitBreakerPolicyConfiguration() + .withConsecutiveErrors(344097144).withTimeoutInSeconds(1836074224) + .withIntervalInSeconds(913638656))); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPolicyProperties.class); + Assertions.assertEquals(1818158355, model.inboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1487017150, + model.inboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(267572624, + model.inboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1237918766, model.inboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(745209495, model.inboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(353739432, model.inboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(1569477780, model.inboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + Assertions.assertEquals(1931546764, model.outboundPolicy().httpRetryPolicy().maxRetries()); + Assertions.assertEquals(1973714604, + model.outboundPolicy().httpRetryPolicy().retryBackOff().initialDelayInMilliseconds()); + Assertions.assertEquals(1422758538, + model.outboundPolicy().httpRetryPolicy().retryBackOff().maxIntervalInMilliseconds()); + Assertions.assertEquals(1903849017, model.outboundPolicy().timeoutPolicy().responseTimeoutInSeconds()); + Assertions.assertEquals(344097144, model.outboundPolicy().circuitBreakerPolicy().consecutiveErrors()); + Assertions.assertEquals(1836074224, model.outboundPolicy().circuitBreakerPolicy().timeoutInSeconds()); + Assertions.assertEquals(913638656, model.outboundPolicy().circuitBreakerPolicy().intervalInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyTimeoutPolicyConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyTimeoutPolicyConfigurationTests.java new file mode 100644 index 0000000000000..d5c553bd706e7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentResiliencyPolicyTimeoutPolicyConfigurationTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentResiliencyPolicyTimeoutPolicyConfiguration; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentResiliencyPolicyTimeoutPolicyConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentResiliencyPolicyTimeoutPolicyConfiguration model + = BinaryData.fromString("{\"responseTimeoutInSeconds\":417226641}") + .toObject(DaprComponentResiliencyPolicyTimeoutPolicyConfiguration.class); + Assertions.assertEquals(417226641, model.responseTimeoutInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentResiliencyPolicyTimeoutPolicyConfiguration model + = new DaprComponentResiliencyPolicyTimeoutPolicyConfiguration().withResponseTimeoutInSeconds(417226641); + model = BinaryData.fromObject(model).toObject(DaprComponentResiliencyPolicyTimeoutPolicyConfiguration.class); + Assertions.assertEquals(417226641, model.responseTimeoutInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentServiceBindingTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentServiceBindingTests.java new file mode 100644 index 0000000000000..493189dba9d6d --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentServiceBindingTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprComponentServiceBinding; +import com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata; +import org.junit.jupiter.api.Assertions; + +public final class DaprComponentServiceBindingTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprComponentServiceBinding model = BinaryData.fromString( + "{\"name\":\"owpulpq\",\"serviceId\":\"ylsyxkqjnsje\",\"metadata\":{\"name\":\"iagxsdszuempsbz\",\"value\":\"z\"}}") + .toObject(DaprComponentServiceBinding.class); + Assertions.assertEquals("owpulpq", model.name()); + Assertions.assertEquals("ylsyxkqjnsje", model.serviceId()); + Assertions.assertEquals("iagxsdszuempsbz", model.metadata().name()); + Assertions.assertEquals("z", model.metadata().value()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprComponentServiceBinding model + = new DaprComponentServiceBinding().withName("owpulpq").withServiceId("ylsyxkqjnsje") + .withMetadata(new DaprServiceBindMetadata().withName("iagxsdszuempsbz").withValue("z")); + model = BinaryData.fromObject(model).toObject(DaprComponentServiceBinding.class); + Assertions.assertEquals("owpulpq", model.name()); + Assertions.assertEquals("ylsyxkqjnsje", model.serviceId()); + Assertions.assertEquals("iagxsdszuempsbz", model.metadata().name()); + Assertions.assertEquals("z", model.metadata().value()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteWithResponseMockTests.java index 5758d5a08d1c7..1e715b47922f3 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .daprComponents() - .deleteWithResponse("zjiihjr", "ybmrzoepnxwd", "wnjkgvfn", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.daprComponents().deleteWithResponse("abalfdxaglz", "ytlbtlqhopxouvm", "siflikyypzkgxf", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsWithResponseMockTests.java index a748b5bfa896d..61dcbf5a6ff11 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprComponentsListSecretsWithResponseMockTests.java @@ -29,39 +29,25 @@ public void testListSecretsWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"sbede\",\"value\":\"exkxbhx\"},{\"name\":\"cnulgmnhjevdy\",\"value\":\"fajs\"},{\"name\":\"skmqozzkivyhj\",\"value\":\"iizjixlqfhefkwa\"},{\"name\":\"olro\",\"value\":\"qlmgnlqxsjxt\"}]}"; + String responseStr = "{\"value\":[{\"name\":\"h\",\"value\":\"pter\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DaprSecretsCollection response = manager.daprComponents() + .listSecretsWithResponse("fmy", "qsdb", "oksz", com.azure.core.util.Context.NONE).getValue(); - DaprSecretsCollection response = - manager - .daprComponents() - .listSecretsWithResponse("xaurs", "ftibtyibuyvpirfq", "pnqn", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprConfigurationTests.java index 7007877b3e7a3..342b528c00a37 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprConfigurationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprConfigurationTests.java @@ -10,8 +10,7 @@ public final class DaprConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DaprConfiguration model = - BinaryData.fromString("{\"version\":\"idfcxsspuunnoxyh\"}").toObject(DaprConfiguration.class); + DaprConfiguration model = BinaryData.fromString("{\"version\":\"jed\"}").toObject(DaprConfiguration.class); } @org.junit.jupiter.api.Test diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretTests.java index 27ed39a23269f..2b872446772dd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretTests.java @@ -10,8 +10,8 @@ public final class DaprSecretTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DaprSecret model = - BinaryData.fromString("{\"name\":\"eil\",\"value\":\"notyfjfcnjbkcn\"}").toObject(DaprSecret.class); + DaprSecret model + = BinaryData.fromString("{\"name\":\"bqwcsdbnwdcf\",\"value\":\"cqdpfuv\"}").toObject(DaprSecret.class); } @org.junit.jupiter.api.Test diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretsCollectionInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretsCollectionInnerTests.java index a270b848efff6..c4dcd23aa1430 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretsCollectionInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSecretsCollectionInnerTests.java @@ -12,18 +12,15 @@ public final class DaprSecretsCollectionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DaprSecretsCollectionInner model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"vpesapskrdqmhjjd\",\"value\":\"ldwkyzxuutkn\"},{\"name\":\"scwsv\",\"value\":\"otogtwrupqs\"},{\"name\":\"nmic\",\"value\":\"vce\"}]}") - .toObject(DaprSecretsCollectionInner.class); + DaprSecretsCollectionInner model = BinaryData.fromString( + "{\"value\":[{\"name\":\"lw\",\"value\":\"ztzp\"},{\"name\":\"ncckw\",\"value\":\"zqwhxxbuyqaxzfeq\"},{\"name\":\"ppriol\",\"value\":\"rjaltolmncw\"}]}") + .toObject(DaprSecretsCollectionInner.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DaprSecretsCollectionInner model = - new DaprSecretsCollectionInner() - .withValue(Arrays.asList(new DaprSecret(), new DaprSecret(), new DaprSecret())); + DaprSecretsCollectionInner model = new DaprSecretsCollectionInner() + .withValue(Arrays.asList(new DaprSecret(), new DaprSecret(), new DaprSecret())); model = BinaryData.fromObject(model).toObject(DaprSecretsCollectionInner.class); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprServiceBindMetadataTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprServiceBindMetadataTests.java new file mode 100644 index 0000000000000..6a5ef80460915 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprServiceBindMetadataTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprServiceBindMetadata; +import org.junit.jupiter.api.Assertions; + +public final class DaprServiceBindMetadataTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprServiceBindMetadata model = BinaryData.fromString("{\"name\":\"yvpnqicvinvkjj\",\"value\":\"xrbuukzclew\"}") + .toObject(DaprServiceBindMetadata.class); + Assertions.assertEquals("yvpnqicvinvkjj", model.name()); + Assertions.assertEquals("xrbuukzclew", model.value()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprServiceBindMetadata model + = new DaprServiceBindMetadata().withName("yvpnqicvinvkjj").withValue("xrbuukzclew"); + model = BinaryData.fromObject(model).toObject(DaprServiceBindMetadata.class); + Assertions.assertEquals("yvpnqicvinvkjj", model.name()); + Assertions.assertEquals("xrbuukzclew", model.value()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionBulkSubscribeOptionsTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionBulkSubscribeOptionsTests.java new file mode 100644 index 0000000000000..9b28d6282e109 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionBulkSubscribeOptionsTests.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import org.junit.jupiter.api.Assertions; + +public final class DaprSubscriptionBulkSubscribeOptionsTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprSubscriptionBulkSubscribeOptions model = BinaryData + .fromString("{\"enabled\":false,\"maxMessagesCount\":2076319922,\"maxAwaitDurationMs\":1797276638}") + .toObject(DaprSubscriptionBulkSubscribeOptions.class); + Assertions.assertEquals(false, model.enabled()); + Assertions.assertEquals(2076319922, model.maxMessagesCount()); + Assertions.assertEquals(1797276638, model.maxAwaitDurationMs()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprSubscriptionBulkSubscribeOptions model = new DaprSubscriptionBulkSubscribeOptions().withEnabled(false) + .withMaxMessagesCount(2076319922).withMaxAwaitDurationMs(1797276638); + model = BinaryData.fromObject(model).toObject(DaprSubscriptionBulkSubscribeOptions.class); + Assertions.assertEquals(false, model.enabled()); + Assertions.assertEquals(2076319922, model.maxMessagesCount()); + Assertions.assertEquals(1797276638, model.maxAwaitDurationMs()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionInnerTests.java new file mode 100644 index 0000000000000..9fc23bde2ddc4 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionInnerTests.java @@ -0,0 +1,72 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class DaprSubscriptionInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprSubscriptionInner model = BinaryData.fromString( + "{\"properties\":{\"pubsubName\":\"zvlnsnnjz\",\"topic\":\"afolpymwamxqzrag\",\"deadLetterTopic\":\"dphtv\",\"routes\":{\"rules\":[{\"match\":\"vl\",\"path\":\"c\"}],\"default\":\"srlzknmzlan\"},\"scopes\":[\"dwv\",\"phcnzqtpj\",\"mqrhvthl\"],\"metadata\":{\"hzdtxetlgydlh\":\"dcxsmlz\",\"b\":\"vlnnpx\",\"gjekglklby\":\"fiqgeaar\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":2131258889,\"maxAwaitDurationMs\":1571149104}},\"id\":\"zegjonfhjirwgdn\",\"name\":\"z\",\"type\":\"rfkspzhz\"}") + .toObject(DaprSubscriptionInner.class); + Assertions.assertEquals("zvlnsnnjz", model.pubsubName()); + Assertions.assertEquals("afolpymwamxqzrag", model.topic()); + Assertions.assertEquals("dphtv", model.deadLetterTopic()); + Assertions.assertEquals("vl", model.routes().rules().get(0).match()); + Assertions.assertEquals("c", model.routes().rules().get(0).path()); + Assertions.assertEquals("srlzknmzlan", model.routes().defaultProperty()); + Assertions.assertEquals("dwv", model.scopes().get(0)); + Assertions.assertEquals("dcxsmlz", model.metadata().get("hzdtxetlgydlh")); + Assertions.assertEquals(false, model.bulkSubscribe().enabled()); + Assertions.assertEquals(2131258889, model.bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(1571149104, model.bulkSubscribe().maxAwaitDurationMs()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprSubscriptionInner model = new DaprSubscriptionInner().withPubsubName("zvlnsnnjz") + .withTopic("afolpymwamxqzrag").withDeadLetterTopic("dphtv") + .withRoutes(new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule().withMatch("vl").withPath("c"))) + .withDefaultProperty("srlzknmzlan")) + .withScopes(Arrays.asList("dwv", "phcnzqtpj", "mqrhvthl")) + .withMetadata(mapOf("hzdtxetlgydlh", "dcxsmlz", "b", "vlnnpx", "gjekglklby", "fiqgeaar")) + .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(false) + .withMaxMessagesCount(2131258889).withMaxAwaitDurationMs(1571149104)); + model = BinaryData.fromObject(model).toObject(DaprSubscriptionInner.class); + Assertions.assertEquals("zvlnsnnjz", model.pubsubName()); + Assertions.assertEquals("afolpymwamxqzrag", model.topic()); + Assertions.assertEquals("dphtv", model.deadLetterTopic()); + Assertions.assertEquals("vl", model.routes().rules().get(0).match()); + Assertions.assertEquals("c", model.routes().rules().get(0).path()); + Assertions.assertEquals("srlzknmzlan", model.routes().defaultProperty()); + Assertions.assertEquals("dwv", model.scopes().get(0)); + Assertions.assertEquals("dcxsmlz", model.metadata().get("hzdtxetlgydlh")); + Assertions.assertEquals(false, model.bulkSubscribe().enabled()); + Assertions.assertEquals(2131258889, model.bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(1571149104, model.bulkSubscribe().maxAwaitDurationMs()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionPropertiesTests.java new file mode 100644 index 0000000000000..b6c8a1b4883ed --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionPropertiesTests.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionProperties; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class DaprSubscriptionPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprSubscriptionProperties model = BinaryData.fromString( + "{\"pubsubName\":\"ksjcitdigs\",\"topic\":\"dglj\",\"deadLetterTopic\":\"keuachtomf\",\"routes\":{\"rules\":[{\"match\":\"wfpfmdgycx\",\"path\":\"skw\"},{\"match\":\"jjyslurlps\",\"path\":\"kvp\"},{\"match\":\"wqslsrhmpqvw\",\"path\":\"kondcb\"},{\"match\":\"imuvqejosovyrrl\",\"path\":\"esi\"}],\"default\":\"qtljqobbpihehc\"},\"scopes\":[\"bmrqbrjbbmp\",\"dlvykfrex\",\"rseqwjksghudgz\"],\"metadata\":{\"u\":\"gjggsv\",\"kmdyomkxfbvfbh\":\"kxibdafh\",\"rhpw\":\"y\",\"o\":\"gddeimaw\"},\"bulkSubscribe\":{\"enabled\":true,\"maxMessagesCount\":1582308433,\"maxAwaitDurationMs\":166613008}}") + .toObject(DaprSubscriptionProperties.class); + Assertions.assertEquals("ksjcitdigs", model.pubsubName()); + Assertions.assertEquals("dglj", model.topic()); + Assertions.assertEquals("keuachtomf", model.deadLetterTopic()); + Assertions.assertEquals("wfpfmdgycx", model.routes().rules().get(0).match()); + Assertions.assertEquals("skw", model.routes().rules().get(0).path()); + Assertions.assertEquals("qtljqobbpihehc", model.routes().defaultProperty()); + Assertions.assertEquals("bmrqbrjbbmp", model.scopes().get(0)); + Assertions.assertEquals("gjggsv", model.metadata().get("u")); + Assertions.assertEquals(true, model.bulkSubscribe().enabled()); + Assertions.assertEquals(1582308433, model.bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(166613008, model.bulkSubscribe().maxAwaitDurationMs()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprSubscriptionProperties model = new DaprSubscriptionProperties().withPubsubName("ksjcitdigs") + .withTopic("dglj").withDeadLetterTopic("keuachtomf") + .withRoutes(new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule().withMatch("wfpfmdgycx").withPath("skw"), + new DaprSubscriptionRouteRule().withMatch("jjyslurlps").withPath("kvp"), + new DaprSubscriptionRouteRule().withMatch("wqslsrhmpqvw").withPath("kondcb"), + new DaprSubscriptionRouteRule().withMatch("imuvqejosovyrrl").withPath("esi"))) + .withDefaultProperty("qtljqobbpihehc")) + .withScopes(Arrays.asList("bmrqbrjbbmp", "dlvykfrex", "rseqwjksghudgz")) + .withMetadata(mapOf("u", "gjggsv", "kmdyomkxfbvfbh", "kxibdafh", "rhpw", "y", "o", "gddeimaw")) + .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(true) + .withMaxMessagesCount(1582308433).withMaxAwaitDurationMs(166613008)); + model = BinaryData.fromObject(model).toObject(DaprSubscriptionProperties.class); + Assertions.assertEquals("ksjcitdigs", model.pubsubName()); + Assertions.assertEquals("dglj", model.topic()); + Assertions.assertEquals("keuachtomf", model.deadLetterTopic()); + Assertions.assertEquals("wfpfmdgycx", model.routes().rules().get(0).match()); + Assertions.assertEquals("skw", model.routes().rules().get(0).path()); + Assertions.assertEquals("qtljqobbpihehc", model.routes().defaultProperty()); + Assertions.assertEquals("bmrqbrjbbmp", model.scopes().get(0)); + Assertions.assertEquals("gjggsv", model.metadata().get("u")); + Assertions.assertEquals(true, model.bulkSubscribe().enabled()); + Assertions.assertEquals(1582308433, model.bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(166613008, model.bulkSubscribe().maxAwaitDurationMs()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRouteRuleTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRouteRuleTests.java new file mode 100644 index 0000000000000..af04eb16faf56 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRouteRuleTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import org.junit.jupiter.api.Assertions; + +public final class DaprSubscriptionRouteRuleTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprSubscriptionRouteRule model = BinaryData.fromString("{\"match\":\"n\",\"path\":\"aoyankcoeqswa\"}") + .toObject(DaprSubscriptionRouteRule.class); + Assertions.assertEquals("n", model.match()); + Assertions.assertEquals("aoyankcoeqswa", model.path()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprSubscriptionRouteRule model = new DaprSubscriptionRouteRule().withMatch("n").withPath("aoyankcoeqswa"); + model = BinaryData.fromObject(model).toObject(DaprSubscriptionRouteRule.class); + Assertions.assertEquals("n", model.match()); + Assertions.assertEquals("aoyankcoeqswa", model.path()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRoutesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRoutesTests.java new file mode 100644 index 0000000000000..3bedba6279c6a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionRoutesTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class DaprSubscriptionRoutesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprSubscriptionRoutes model = BinaryData.fromString( + "{\"rules\":[{\"match\":\"jcazt\",\"path\":\"snsqowxwcoml\"},{\"match\":\"ytwvczcswkacve\",\"path\":\"fdv\"}],\"default\":\"hbwrnfxtgddpqth\"}") + .toObject(DaprSubscriptionRoutes.class); + Assertions.assertEquals("jcazt", model.rules().get(0).match()); + Assertions.assertEquals("snsqowxwcoml", model.rules().get(0).path()); + Assertions.assertEquals("hbwrnfxtgddpqth", model.defaultProperty()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprSubscriptionRoutes model = new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule().withMatch("jcazt").withPath("snsqowxwcoml"), + new DaprSubscriptionRouteRule().withMatch("ytwvczcswkacve").withPath("fdv"))) + .withDefaultProperty("hbwrnfxtgddpqth"); + model = BinaryData.fromObject(model).toObject(DaprSubscriptionRoutes.class); + Assertions.assertEquals("jcazt", model.rules().get(0).match()); + Assertions.assertEquals("snsqowxwcoml", model.rules().get(0).path()); + Assertions.assertEquals("hbwrnfxtgddpqth", model.defaultProperty()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCollectionTests.java new file mode 100644 index 0000000000000..42ef1ef68013a --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCollectionTests.java @@ -0,0 +1,107 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DaprSubscriptionInner; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionsCollection; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class DaprSubscriptionsCollectionTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DaprSubscriptionsCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"pubsubName\":\"rmoucsofl\",\"topic\":\"uviyfcaabeo\",\"deadLetterTopic\":\"bh\",\"routes\":{\"rules\":[{}],\"default\":\"uqibsxtkcu\"},\"scopes\":[\"sfar\"],\"metadata\":{\"gf\":\"owlkjxnqpv\"},\"bulkSubscribe\":{\"enabled\":true,\"maxMessagesCount\":889840232,\"maxAwaitDurationMs\":1864942425}},\"id\":\"mdksaoa\",\"name\":\"cluqvo\",\"type\":\"mycjimryvwgcwwp\"},{\"properties\":{\"pubsubName\":\"gwe\",\"topic\":\"dsxwefoh\",\"deadLetterTopic\":\"bvopwndyqle\",\"routes\":{\"rules\":[{},{},{}],\"default\":\"mtkhlowkxxpvbr\"},\"scopes\":[\"mzsyzfhotl\",\"ikcyyc\"],\"metadata\":{\"uic\":\"sjlpjrtwszhv\",\"ubhvj\":\"hvtrrmhwrbfdpyf\",\"memhooclutnpq\":\"lrocuyzlwh\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":339637531,\"maxAwaitDurationMs\":1132519096}},\"id\":\"ykyujxsg\",\"name\":\"hsrrryejylmbkz\",\"type\":\"dnigrfihot\"},{\"properties\":{\"pubsubName\":\"lpxuzzjgnrefq\",\"topic\":\"qotoihiqakydiwfb\",\"deadLetterTopic\":\"wpzdqtvhcspo\",\"routes\":{\"rules\":[{},{},{}],\"default\":\"sipi\"},\"scopes\":[\"bebjfu\",\"bmoichd\"],\"metadata\":{\"iqsowsaaelc\":\"fpubntnbatz\",\"mjvlgfgg\":\"ttcjuhplrvkmjc\",\"sfuztlvtmv\":\"vkyylizrzbjpsf\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":891535208,\"maxAwaitDurationMs\":563395516}},\"id\":\"hukoveofizrv\",\"name\":\"f\",\"type\":\"mjm\"},{\"properties\":{\"pubsubName\":\"yzgib\",\"topic\":\"ujr\",\"deadLetterTopic\":\"fojuidjpuuyj\",\"routes\":{\"rules\":[{},{},{},{}],\"default\":\"kzoeovvtzejetj\"},\"scopes\":[\"tikyj\"],\"metadata\":{\"rhtgvgzpcrrkol\":\"dbqzolxrzvhqjw\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":817860491,\"maxAwaitDurationMs\":660756224}},\"id\":\"okcdxfzzzwyjaf\",\"name\":\"tlhguynuchl\",\"type\":\"mltx\"}],\"nextLink\":\"hmoz\"}") + .toObject(DaprSubscriptionsCollection.class); + Assertions.assertEquals("rmoucsofl", model.value().get(0).pubsubName()); + Assertions.assertEquals("uviyfcaabeo", model.value().get(0).topic()); + Assertions.assertEquals("bh", model.value().get(0).deadLetterTopic()); + Assertions.assertEquals("uqibsxtkcu", model.value().get(0).routes().defaultProperty()); + Assertions.assertEquals("sfar", model.value().get(0).scopes().get(0)); + Assertions.assertEquals("owlkjxnqpv", model.value().get(0).metadata().get("gf")); + Assertions.assertEquals(true, model.value().get(0).bulkSubscribe().enabled()); + Assertions.assertEquals(889840232, model.value().get(0).bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(1864942425, model.value().get(0).bulkSubscribe().maxAwaitDurationMs()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DaprSubscriptionsCollection model + = new DaprSubscriptionsCollection() + .withValue(Arrays.asList( + new DaprSubscriptionInner().withPubsubName("rmoucsofl").withTopic("uviyfcaabeo") + .withDeadLetterTopic("bh") + .withRoutes( + new DaprSubscriptionRoutes().withRules(Arrays.asList(new DaprSubscriptionRouteRule())) + .withDefaultProperty("uqibsxtkcu")) + .withScopes(Arrays.asList("sfar")).withMetadata( + mapOf("gf", "owlkjxnqpv")) + .withBulkSubscribe( + new DaprSubscriptionBulkSubscribeOptions().withEnabled(true).withMaxMessagesCount(889840232) + .withMaxAwaitDurationMs(1864942425)), + new DaprSubscriptionInner().withPubsubName("gwe").withTopic("dsxwefoh") + .withDeadLetterTopic("bvopwndyqle") + .withRoutes( + new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule(), + new DaprSubscriptionRouteRule(), new DaprSubscriptionRouteRule())) + .withDefaultProperty("mtkhlowkxxpvbr")) + .withScopes(Arrays.asList("mzsyzfhotl", "ikcyyc")) + .withMetadata( + mapOf("uic", "sjlpjrtwszhv", "ubhvj", "hvtrrmhwrbfdpyf", "memhooclutnpq", "lrocuyzlwh")) + .withBulkSubscribe( + new DaprSubscriptionBulkSubscribeOptions().withEnabled(false).withMaxMessagesCount( + 339637531).withMaxAwaitDurationMs(1132519096)), + new DaprSubscriptionInner().withPubsubName("lpxuzzjgnrefq").withTopic("qotoihiqakydiwfb") + .withDeadLetterTopic("wpzdqtvhcspo") + .withRoutes( + new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule(), + new DaprSubscriptionRouteRule(), new DaprSubscriptionRouteRule())) + .withDefaultProperty("sipi")) + .withScopes(Arrays.asList("bebjfu", "bmoichd")) + .withMetadata(mapOf("iqsowsaaelc", "fpubntnbatz", "mjvlgfgg", "ttcjuhplrvkmjc", "sfuztlvtmv", + "vkyylizrzbjpsf")) + .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(false) + .withMaxMessagesCount(891535208).withMaxAwaitDurationMs(563395516)), + new DaprSubscriptionInner().withPubsubName("yzgib").withTopic("ujr") + .withDeadLetterTopic("fojuidjpuuyj") + .withRoutes(new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule(), new DaprSubscriptionRouteRule(), + new DaprSubscriptionRouteRule(), new DaprSubscriptionRouteRule())) + .withDefaultProperty("kzoeovvtzejetj")) + .withScopes(Arrays.asList("tikyj")).withMetadata(mapOf("rhtgvgzpcrrkol", "dbqzolxrzvhqjw")) + .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(false) + .withMaxMessagesCount(817860491).withMaxAwaitDurationMs(660756224)))); + model = BinaryData.fromObject(model).toObject(DaprSubscriptionsCollection.class); + Assertions.assertEquals("rmoucsofl", model.value().get(0).pubsubName()); + Assertions.assertEquals("uviyfcaabeo", model.value().get(0).topic()); + Assertions.assertEquals("bh", model.value().get(0).deadLetterTopic()); + Assertions.assertEquals("uqibsxtkcu", model.value().get(0).routes().defaultProperty()); + Assertions.assertEquals("sfar", model.value().get(0).scopes().get(0)); + Assertions.assertEquals("owlkjxnqpv", model.value().get(0).metadata().get("gf")); + Assertions.assertEquals(true, model.value().get(0).bulkSubscribe().enabled()); + Assertions.assertEquals(889840232, model.value().get(0).bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(1864942425, model.value().get(0).bulkSubscribe().maxAwaitDurationMs()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateWithResponseMockTests.java new file mode 100644 index 0000000000000..6c1b443c66f28 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsCreateOrUpdateWithResponseMockTests.java @@ -0,0 +1,94 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DaprSubscription; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionBulkSubscribeOptions; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRouteRule; +import com.azure.resourcemanager.appcontainers.models.DaprSubscriptionRoutes; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprSubscriptionsCreateOrUpdateWithResponseMockTests { + @Test + public void testCreateOrUpdateWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"pubsubName\":\"jiotif\",\"topic\":\"bcngkegxc\",\"deadLetterTopic\":\"xbbfetwil\",\"routes\":{\"rules\":[{\"match\":\"pdxqlfrolqownkiu\",\"path\":\"ewnahwkxjjmzt\"},{\"match\":\"mso\",\"path\":\"tmvecdhdyswcrptv\"}],\"default\":\"jczxvlgsrg\"},\"scopes\":[\"izrpywlptyuqhrl\",\"cs\",\"ykpfu\",\"fixcnpcfykkpy\"],\"metadata\":{\"wrpcfpcfjfwzlgza\":\"awmpjprd\",\"amnni\":\"kgyepe\",\"z\":\"mdiawpzx\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":335893529,\"maxAwaitDurationMs\":825702294}},\"id\":\"uos\",\"name\":\"wqpsqaz\",\"type\":\"hqodv\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DaprSubscription response + = manager.daprSubscriptions().define("wennin").withExistingManagedEnvironment("xquddrw", "cljbrhlhpvzad") + .withPubsubName("xrzfrmvztiucwvi").withTopic("lukh").withDeadLetterTopic("cqx") + .withRoutes(new DaprSubscriptionRoutes() + .withRules(Arrays.asList(new DaprSubscriptionRouteRule().withMatch("a").withPath("qcbpokstsinv"))) + .withDefaultProperty("ovjyhd")) + .withScopes(Arrays.asList("rdvcehqwhit", "nmxgnm")) + .withMetadata(mapOf("kjnbkbdhlltqstq", "buwvorba", "vagzkheuba", "qsygxiynec", "pqcckqiawzlz", "lxu", + "r", "laslgacizux")) + .withBulkSubscribe(new DaprSubscriptionBulkSubscribeOptions().withEnabled(false) + .withMaxMessagesCount(1749126168).withMaxAwaitDurationMs(1501943232)) + .create(); + + Assertions.assertEquals("jiotif", response.pubsubName()); + Assertions.assertEquals("bcngkegxc", response.topic()); + Assertions.assertEquals("xbbfetwil", response.deadLetterTopic()); + Assertions.assertEquals("pdxqlfrolqownkiu", response.routes().rules().get(0).match()); + Assertions.assertEquals("ewnahwkxjjmzt", response.routes().rules().get(0).path()); + Assertions.assertEquals("jczxvlgsrg", response.routes().defaultProperty()); + Assertions.assertEquals("izrpywlptyuqhrl", response.scopes().get(0)); + Assertions.assertEquals("awmpjprd", response.metadata().get("wrpcfpcfjfwzlgza")); + Assertions.assertEquals(false, response.bulkSubscribe().enabled()); + Assertions.assertEquals(335893529, response.bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(825702294, response.bulkSubscribe().maxAwaitDurationMs()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteWithResponseMockTests.java new file mode 100644 index 0000000000000..cbdf3dd6e62be --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsDeleteWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprSubscriptionsDeleteWithResponseMockTests { + @Test + public void testDeleteWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.daprSubscriptions().deleteWithResponse("c", "wnujvqynvav", "tmdmuqohhihr", + com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetWithResponseMockTests.java new file mode 100644 index 0000000000000..68ef1a7b24711 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsGetWithResponseMockTests.java @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DaprSubscription; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprSubscriptionsGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"pubsubName\":\"suaawj\",\"topic\":\"xwjnfcz\",\"deadLetterTopic\":\"nii\",\"routes\":{\"rules\":[{\"match\":\"ba\",\"path\":\"sjtgirnbgmgmddo\"},{\"match\":\"mynltwmpftmfoeaj\",\"path\":\"syxwetamfdd\"}],\"default\":\"lkpzwbhnrecchd\"},\"scopes\":[\"muhkahm\",\"edbiucvkhhwmjpjb\",\"eunxcqrrihufoi\",\"ppiybxv\"],\"metadata\":{\"zvkunhdi\":\"uzpbgkzcscpi\",\"cfqp\":\"juktirzkaugpucd\",\"rjlvzklkvbgi\":\"wgofmhx\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":86596689,\"maxAwaitDurationMs\":1162128652}},\"id\":\"bishjvpzaptuo\",\"name\":\"kaoizjixw\",\"type\":\"gcdiyk\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DaprSubscription response = manager.daprSubscriptions() + .getWithResponse("btimpkjblor", "s", "hqhud", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("suaawj", response.pubsubName()); + Assertions.assertEquals("xwjnfcz", response.topic()); + Assertions.assertEquals("nii", response.deadLetterTopic()); + Assertions.assertEquals("ba", response.routes().rules().get(0).match()); + Assertions.assertEquals("sjtgirnbgmgmddo", response.routes().rules().get(0).path()); + Assertions.assertEquals("lkpzwbhnrecchd", response.routes().defaultProperty()); + Assertions.assertEquals("muhkahm", response.scopes().get(0)); + Assertions.assertEquals("uzpbgkzcscpi", response.metadata().get("zvkunhdi")); + Assertions.assertEquals(false, response.bulkSubscribe().enabled()); + Assertions.assertEquals(86596689, response.bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(1162128652, response.bulkSubscribe().maxAwaitDurationMs()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListMockTests.java new file mode 100644 index 0000000000000..a26bc89bdf8dd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprSubscriptionsListMockTests.java @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DaprSubscription; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DaprSubscriptionsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"pubsubName\":\"sdgzyy\",\"topic\":\"oxlvocptvdxxheig\",\"deadLetterTopic\":\"ilwzghj\",\"routes\":{\"rules\":[{\"match\":\"bzzbwayb\",\"path\":\"d\"}],\"default\":\"bgymqt\"},\"scopes\":[\"reojxrjnbsc\",\"nx\",\"vi\",\"neychbjizq\"],\"metadata\":{\"ipmlnfyzavf\":\"nwdxzedpqlrfbor\",\"aq\":\"bypicdbkpdj\"},\"bulkSubscribe\":{\"enabled\":false,\"maxMessagesCount\":2041682008,\"maxAwaitDurationMs\":1356872563}},\"id\":\"onjhxshthmgpczqu\",\"name\":\"ptkbvcpxtzhi\",\"type\":\"q\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.daprSubscriptions().list("iuwkirkskwzts", "etjygowifcqpolz", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("sdgzyy", response.iterator().next().pubsubName()); + Assertions.assertEquals("oxlvocptvdxxheig", response.iterator().next().topic()); + Assertions.assertEquals("ilwzghj", response.iterator().next().deadLetterTopic()); + Assertions.assertEquals("bzzbwayb", response.iterator().next().routes().rules().get(0).match()); + Assertions.assertEquals("d", response.iterator().next().routes().rules().get(0).path()); + Assertions.assertEquals("bgymqt", response.iterator().next().routes().defaultProperty()); + Assertions.assertEquals("reojxrjnbsc", response.iterator().next().scopes().get(0)); + Assertions.assertEquals("nwdxzedpqlrfbor", response.iterator().next().metadata().get("ipmlnfyzavf")); + Assertions.assertEquals(false, response.iterator().next().bulkSubscribe().enabled()); + Assertions.assertEquals(2041682008, response.iterator().next().bulkSubscribe().maxMessagesCount()); + Assertions.assertEquals(1356872563, response.iterator().next().bulkSubscribe().maxAwaitDurationMs()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprTests.java index 96f6f41ecfe6c..0feb7db9d9333 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DaprTests.java @@ -13,41 +13,32 @@ public final class DaprTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Dapr model = - BinaryData - .fromString( - "{\"enabled\":false,\"appId\":\"rfbjf\",\"appProtocol\":\"http\",\"appPort\":2146010865,\"httpReadBufferSize\":22179172,\"httpMaxRequestSize\":2101794539,\"logLevel\":\"info\",\"enableApiLogging\":true}") - .toObject(Dapr.class); - Assertions.assertEquals(false, model.enabled()); - Assertions.assertEquals("rfbjf", model.appId()); - Assertions.assertEquals(AppProtocol.HTTP, model.appProtocol()); - Assertions.assertEquals(2146010865, model.appPort()); - Assertions.assertEquals(22179172, model.httpReadBufferSize()); - Assertions.assertEquals(2101794539, model.httpMaxRequestSize()); - Assertions.assertEquals(LogLevel.INFO, model.logLevel()); - Assertions.assertEquals(true, model.enableApiLogging()); + Dapr model = BinaryData.fromString( + "{\"enabled\":true,\"appId\":\"kfwynw\",\"appProtocol\":\"grpc\",\"appPort\":885107915,\"httpReadBufferSize\":1353261360,\"httpMaxRequestSize\":1479821561,\"logLevel\":\"warn\",\"enableApiLogging\":false}") + .toObject(Dapr.class); + Assertions.assertEquals(true, model.enabled()); + Assertions.assertEquals("kfwynw", model.appId()); + Assertions.assertEquals(AppProtocol.GRPC, model.appProtocol()); + Assertions.assertEquals(885107915, model.appPort()); + Assertions.assertEquals(1353261360, model.httpReadBufferSize()); + Assertions.assertEquals(1479821561, model.httpMaxRequestSize()); + Assertions.assertEquals(LogLevel.WARN, model.logLevel()); + Assertions.assertEquals(false, model.enableApiLogging()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Dapr model = - new Dapr() - .withEnabled(false) - .withAppId("rfbjf") - .withAppProtocol(AppProtocol.HTTP) - .withAppPort(2146010865) - .withHttpReadBufferSize(22179172) - .withHttpMaxRequestSize(2101794539) - .withLogLevel(LogLevel.INFO) - .withEnableApiLogging(true); + Dapr model = new Dapr().withEnabled(true).withAppId("kfwynw").withAppProtocol(AppProtocol.GRPC) + .withAppPort(885107915).withHttpReadBufferSize(1353261360).withHttpMaxRequestSize(1479821561) + .withLogLevel(LogLevel.WARN).withEnableApiLogging(false); model = BinaryData.fromObject(model).toObject(Dapr.class); - Assertions.assertEquals(false, model.enabled()); - Assertions.assertEquals("rfbjf", model.appId()); - Assertions.assertEquals(AppProtocol.HTTP, model.appProtocol()); - Assertions.assertEquals(2146010865, model.appPort()); - Assertions.assertEquals(22179172, model.httpReadBufferSize()); - Assertions.assertEquals(2101794539, model.httpMaxRequestSize()); - Assertions.assertEquals(LogLevel.INFO, model.logLevel()); - Assertions.assertEquals(true, model.enableApiLogging()); + Assertions.assertEquals(true, model.enabled()); + Assertions.assertEquals("kfwynw", model.appId()); + Assertions.assertEquals(AppProtocol.GRPC, model.appProtocol()); + Assertions.assertEquals(885107915, model.appPort()); + Assertions.assertEquals(1353261360, model.httpReadBufferSize()); + Assertions.assertEquals(1479821561, model.httpMaxRequestSize()); + Assertions.assertEquals(LogLevel.WARN, model.logLevel()); + Assertions.assertEquals(false, model.enableApiLogging()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultAuthorizationPolicyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultAuthorizationPolicyTests.java index 1ce9050dcdfd6..1ca655a966286 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultAuthorizationPolicyTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultAuthorizationPolicyTests.java @@ -13,28 +13,25 @@ public final class DefaultAuthorizationPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DefaultAuthorizationPolicy model = - BinaryData - .fromString( - "{\"allowedPrincipals\":{\"groups\":[\"mxdcufufsrp\",\"mzidnsezcxtb\"],\"identities\":[\"fycc\",\"newmdwzjeiachbo\"]},\"allowedApplications\":[\"lnrosfqp\",\"eeh\",\"zvypyqrimzinp\",\"swjdkirso\"]}") - .toObject(DefaultAuthorizationPolicy.class); - Assertions.assertEquals("mxdcufufsrp", model.allowedPrincipals().groups().get(0)); - Assertions.assertEquals("fycc", model.allowedPrincipals().identities().get(0)); - Assertions.assertEquals("lnrosfqp", model.allowedApplications().get(0)); + DefaultAuthorizationPolicy model = BinaryData.fromString( + "{\"allowedPrincipals\":{\"groups\":[\"hbzhfepg\",\"gqexzlocxs\",\"paierh\"],\"identities\":[\"sglumma\",\"tjaodxobnb\"]},\"allowedApplications\":[\"qp\"]}") + .toObject(DefaultAuthorizationPolicy.class); + Assertions.assertEquals("hbzhfepg", model.allowedPrincipals().groups().get(0)); + Assertions.assertEquals("sglumma", model.allowedPrincipals().identities().get(0)); + Assertions.assertEquals("qp", model.allowedApplications().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DefaultAuthorizationPolicy model = - new DefaultAuthorizationPolicy() + DefaultAuthorizationPolicy model + = new DefaultAuthorizationPolicy() .withAllowedPrincipals( - new AllowedPrincipals() - .withGroups(Arrays.asList("mxdcufufsrp", "mzidnsezcxtb")) - .withIdentities(Arrays.asList("fycc", "newmdwzjeiachbo"))) - .withAllowedApplications(Arrays.asList("lnrosfqp", "eeh", "zvypyqrimzinp", "swjdkirso")); + new AllowedPrincipals().withGroups(Arrays.asList("hbzhfepg", "gqexzlocxs", "paierh")) + .withIdentities(Arrays.asList("sglumma", "tjaodxobnb"))) + .withAllowedApplications(Arrays.asList("qp")); model = BinaryData.fromObject(model).toObject(DefaultAuthorizationPolicy.class); - Assertions.assertEquals("mxdcufufsrp", model.allowedPrincipals().groups().get(0)); - Assertions.assertEquals("fycc", model.allowedPrincipals().identities().get(0)); - Assertions.assertEquals("lnrosfqp", model.allowedApplications().get(0)); + Assertions.assertEquals("hbzhfepg", model.allowedPrincipals().groups().get(0)); + Assertions.assertEquals("sglumma", model.allowedPrincipals().identities().get(0)); + Assertions.assertEquals("qp", model.allowedApplications().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultErrorResponseErrorTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultErrorResponseErrorTests.java index de7c60517d5c5..1b9c62c0f6b78 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultErrorResponseErrorTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DefaultErrorResponseErrorTests.java @@ -10,8 +10,8 @@ public final class DefaultErrorResponseErrorTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DefaultErrorResponseError model = - BinaryData.fromString("{\"innererror\":\"lokeyy\"}").toObject(DefaultErrorResponseError.class); + DefaultErrorResponseError model + = BinaryData.fromString("{\"innererror\":\"qulngsntnbybkzgc\"}").toObject(DefaultErrorResponseError.class); } @org.junit.jupiter.api.Test diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataPropertyBagItemTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataPropertyBagItemTests.java index f4da5fb60726d..b7337a343a0de 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataPropertyBagItemTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataPropertyBagItemTests.java @@ -11,20 +11,19 @@ public final class DiagnosticDataProviderMetadataPropertyBagItemTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticDataProviderMetadataPropertyBagItem model = - BinaryData - .fromString("{\"name\":\"r\",\"value\":\"lmywwtkgkxnyed\"}") + DiagnosticDataProviderMetadataPropertyBagItem model + = BinaryData.fromString("{\"name\":\"ifrygzn\",\"value\":\"axri\"}") .toObject(DiagnosticDataProviderMetadataPropertyBagItem.class); - Assertions.assertEquals("r", model.name()); - Assertions.assertEquals("lmywwtkgkxnyed", model.value()); + Assertions.assertEquals("ifrygzn", model.name()); + Assertions.assertEquals("axri", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticDataProviderMetadataPropertyBagItem model = - new DiagnosticDataProviderMetadataPropertyBagItem().withName("r").withValue("lmywwtkgkxnyed"); + DiagnosticDataProviderMetadataPropertyBagItem model + = new DiagnosticDataProviderMetadataPropertyBagItem().withName("ifrygzn").withValue("axri"); model = BinaryData.fromObject(model).toObject(DiagnosticDataProviderMetadataPropertyBagItem.class); - Assertions.assertEquals("r", model.name()); - Assertions.assertEquals("lmywwtkgkxnyed", model.value()); + Assertions.assertEquals("ifrygzn", model.name()); + Assertions.assertEquals("axri", model.value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataTests.java index 2cbf11413ca00..b0e24a5ad71b7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataProviderMetadataTests.java @@ -13,34 +13,23 @@ public final class DiagnosticDataProviderMetadataTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticDataProviderMetadata model = - BinaryData - .fromString( - "{\"providerName\":\"vazivjlfr\",\"propertyBag\":[{\"name\":\"ajlkatnw\",\"value\":\"iopid\"},{\"name\":\"qfkuvscxkdmli\",\"value\":\"vibrxkpmloazuruo\"},{\"name\":\"goorbteo\",\"value\":\"fhjxakvvjgs\"}]}") - .toObject(DiagnosticDataProviderMetadata.class); - Assertions.assertEquals("vazivjlfr", model.providerName()); - Assertions.assertEquals("ajlkatnw", model.propertyBag().get(0).name()); - Assertions.assertEquals("iopid", model.propertyBag().get(0).value()); + DiagnosticDataProviderMetadata model = BinaryData.fromString( + "{\"providerName\":\"b\",\"propertyBag\":[{\"name\":\"hv\",\"value\":\"lkvn\"},{\"name\":\"lrigjkskyri\",\"value\":\"vzidsxwaab\"}]}") + .toObject(DiagnosticDataProviderMetadata.class); + Assertions.assertEquals("b", model.providerName()); + Assertions.assertEquals("hv", model.propertyBag().get(0).name()); + Assertions.assertEquals("lkvn", model.propertyBag().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticDataProviderMetadata model = - new DiagnosticDataProviderMetadata() - .withProviderName("vazivjlfr") - .withPropertyBag( - Arrays - .asList( - new DiagnosticDataProviderMetadataPropertyBagItem().withName("ajlkatnw").withValue("iopid"), - new DiagnosticDataProviderMetadataPropertyBagItem() - .withName("qfkuvscxkdmli") - .withValue("vibrxkpmloazuruo"), - new DiagnosticDataProviderMetadataPropertyBagItem() - .withName("goorbteo") - .withValue("fhjxakvvjgs"))); + DiagnosticDataProviderMetadata model = new DiagnosticDataProviderMetadata().withProviderName("b") + .withPropertyBag(Arrays.asList( + new DiagnosticDataProviderMetadataPropertyBagItem().withName("hv").withValue("lkvn"), + new DiagnosticDataProviderMetadataPropertyBagItem().withName("lrigjkskyri").withValue("vzidsxwaab"))); model = BinaryData.fromObject(model).toObject(DiagnosticDataProviderMetadata.class); - Assertions.assertEquals("vazivjlfr", model.providerName()); - Assertions.assertEquals("ajlkatnw", model.propertyBag().get(0).name()); - Assertions.assertEquals("iopid", model.propertyBag().get(0).value()); + Assertions.assertEquals("b", model.providerName()); + Assertions.assertEquals("hv", model.propertyBag().get(0).name()); + Assertions.assertEquals("lkvn", model.propertyBag().get(0).value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseColumnTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseColumnTests.java index 30ee263cca401..a7c905a0bf2ea 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseColumnTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseColumnTests.java @@ -11,25 +11,21 @@ public final class DiagnosticDataTableResponseColumnTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticDataTableResponseColumn model = - BinaryData - .fromString("{\"columnName\":\"ubdyhgk\",\"dataType\":\"in\",\"columnType\":\"owzfttsttkt\"}") + DiagnosticDataTableResponseColumn model + = BinaryData.fromString("{\"columnName\":\"waz\",\"dataType\":\"qnmcjngzqdqx\",\"columnType\":\"jw\"}") .toObject(DiagnosticDataTableResponseColumn.class); - Assertions.assertEquals("ubdyhgk", model.columnName()); - Assertions.assertEquals("in", model.dataType()); - Assertions.assertEquals("owzfttsttkt", model.columnType()); + Assertions.assertEquals("waz", model.columnName()); + Assertions.assertEquals("qnmcjngzqdqx", model.dataType()); + Assertions.assertEquals("jw", model.columnType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticDataTableResponseColumn model = - new DiagnosticDataTableResponseColumn() - .withColumnName("ubdyhgk") - .withDataType("in") - .withColumnType("owzfttsttkt"); + DiagnosticDataTableResponseColumn model = new DiagnosticDataTableResponseColumn().withColumnName("waz") + .withDataType("qnmcjngzqdqx").withColumnType("jw"); model = BinaryData.fromObject(model).toObject(DiagnosticDataTableResponseColumn.class); - Assertions.assertEquals("ubdyhgk", model.columnName()); - Assertions.assertEquals("in", model.dataType()); - Assertions.assertEquals("owzfttsttkt", model.columnType()); + Assertions.assertEquals("waz", model.columnName()); + Assertions.assertEquals("qnmcjngzqdqx", model.dataType()); + Assertions.assertEquals("jw", model.columnType()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseObjectTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseObjectTests.java index cd4989b292ae6..8efb2fbdfcb68 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseObjectTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticDataTableResponseObjectTests.java @@ -13,46 +13,32 @@ public final class DiagnosticDataTableResponseObjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticDataTableResponseObject model = - BinaryData - .fromString( - "{\"tableName\":\"gdufiqnd\",\"columns\":[{\"columnName\":\"ao\",\"dataType\":\"chvcyyysfgdo\",\"columnType\":\"ubiipuipwoqonma\"},{\"columnName\":\"ekni\",\"dataType\":\"hqvcimpevfgmblr\",\"columnType\":\"lbywdxsm\"},{\"columnName\":\"cwrwfs\",\"dataType\":\"fnynszqujizdvoqy\",\"columnType\":\"byowbblgyavutp\"},{\"columnName\":\"joxoism\",\"dataType\":\"sbpimlq\",\"columnType\":\"jxkcgxxlxsff\"}],\"rows\":[\"dataizqzdwlvwlyou\",\"datafgfb\"]}") - .toObject(DiagnosticDataTableResponseObject.class); - Assertions.assertEquals("gdufiqnd", model.tableName()); - Assertions.assertEquals("ao", model.columns().get(0).columnName()); - Assertions.assertEquals("chvcyyysfgdo", model.columns().get(0).dataType()); - Assertions.assertEquals("ubiipuipwoqonma", model.columns().get(0).columnType()); + DiagnosticDataTableResponseObject model = BinaryData.fromString( + "{\"tableName\":\"tmzlbiojlv\",\"columns\":[{\"columnName\":\"bpneqvcwwyyurm\",\"dataType\":\"hppr\",\"columnType\":\"snmokayzej\"},{\"columnName\":\"lbkpb\",\"dataType\":\"cpilj\",\"columnType\":\"hzvechndbnwieho\"},{\"columnName\":\"wjwiuub\",\"dataType\":\"fqsfa\",\"columnType\":\"qtferrqwexjkmf\"},{\"columnName\":\"pjwogqqno\",\"dataType\":\"udcdabtqwpwyawb\",\"columnType\":\"sqbuc\"}],\"rows\":[\"datakyexaoguyaipi\",\"datasdaultxij\",\"dataum\"]}") + .toObject(DiagnosticDataTableResponseObject.class); + Assertions.assertEquals("tmzlbiojlv", model.tableName()); + Assertions.assertEquals("bpneqvcwwyyurm", model.columns().get(0).columnName()); + Assertions.assertEquals("hppr", model.columns().get(0).dataType()); + Assertions.assertEquals("snmokayzej", model.columns().get(0).columnType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticDataTableResponseObject model = - new DiagnosticDataTableResponseObject() - .withTableName("gdufiqnd") - .withColumns( - Arrays - .asList( - new DiagnosticDataTableResponseColumn() - .withColumnName("ao") - .withDataType("chvcyyysfgdo") - .withColumnType("ubiipuipwoqonma"), - new DiagnosticDataTableResponseColumn() - .withColumnName("ekni") - .withDataType("hqvcimpevfgmblr") - .withColumnType("lbywdxsm"), - new DiagnosticDataTableResponseColumn() - .withColumnName("cwrwfs") - .withDataType("fnynszqujizdvoqy") - .withColumnType("byowbblgyavutp"), - new DiagnosticDataTableResponseColumn() - .withColumnName("joxoism") - .withDataType("sbpimlq") - .withColumnType("jxkcgxxlxsff"))) - .withRows(Arrays.asList("dataizqzdwlvwlyou", "datafgfb")); + DiagnosticDataTableResponseObject model = new DiagnosticDataTableResponseObject().withTableName("tmzlbiojlv") + .withColumns(Arrays.asList( + new DiagnosticDataTableResponseColumn().withColumnName("bpneqvcwwyyurm").withDataType("hppr") + .withColumnType("snmokayzej"), + new DiagnosticDataTableResponseColumn().withColumnName("lbkpb").withDataType("cpilj") + .withColumnType("hzvechndbnwieho"), + new DiagnosticDataTableResponseColumn().withColumnName("wjwiuub").withDataType("fqsfa") + .withColumnType("qtferrqwexjkmf"), + new DiagnosticDataTableResponseColumn().withColumnName("pjwogqqno").withDataType("udcdabtqwpwyawb") + .withColumnType("sqbuc"))) + .withRows(Arrays.asList("datakyexaoguyaipi", "datasdaultxij", "dataum")); model = BinaryData.fromObject(model).toObject(DiagnosticDataTableResponseObject.class); - Assertions.assertEquals("gdufiqnd", model.tableName()); - Assertions.assertEquals("ao", model.columns().get(0).columnName()); - Assertions.assertEquals("chvcyyysfgdo", model.columns().get(0).dataType()); - Assertions.assertEquals("ubiipuipwoqonma", model.columns().get(0).columnType()); + Assertions.assertEquals("tmzlbiojlv", model.tableName()); + Assertions.assertEquals("bpneqvcwwyyurm", model.columns().get(0).columnName()); + Assertions.assertEquals("hppr", model.columns().get(0).dataType()); + Assertions.assertEquals("snmokayzej", model.columns().get(0).columnType()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticRenderingTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticRenderingTests.java index e4a2e054676ef..ed630e867c94d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticRenderingTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticRenderingTests.java @@ -11,29 +11,23 @@ public final class DiagnosticRenderingTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticRendering model = - BinaryData - .fromString( - "{\"type\":631157488,\"title\":\"qactxtgzukxitmmq\",\"description\":\"qqqxhrnxrx\",\"isVisible\":true}") - .toObject(DiagnosticRendering.class); - Assertions.assertEquals(631157488, model.type()); - Assertions.assertEquals("qactxtgzukxitmmq", model.title()); - Assertions.assertEquals("qqqxhrnxrx", model.description()); + DiagnosticRendering model = BinaryData.fromString( + "{\"type\":673657894,\"title\":\"usfzsvtuikzha\",\"description\":\"glcfhmlrqryxyn\",\"isVisible\":true}") + .toObject(DiagnosticRendering.class); + Assertions.assertEquals(673657894, model.type()); + Assertions.assertEquals("usfzsvtuikzha", model.title()); + Assertions.assertEquals("glcfhmlrqryxyn", model.description()); Assertions.assertEquals(true, model.isVisible()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticRendering model = - new DiagnosticRendering() - .withType(631157488) - .withTitle("qactxtgzukxitmmq") - .withDescription("qqqxhrnxrx") - .withIsVisible(true); + DiagnosticRendering model = new DiagnosticRendering().withType(673657894).withTitle("usfzsvtuikzha") + .withDescription("glcfhmlrqryxyn").withIsVisible(true); model = BinaryData.fromObject(model).toObject(DiagnosticRendering.class); - Assertions.assertEquals(631157488, model.type()); - Assertions.assertEquals("qactxtgzukxitmmq", model.title()); - Assertions.assertEquals("qqqxhrnxrx", model.description()); + Assertions.assertEquals(673657894, model.type()); + Assertions.assertEquals("usfzsvtuikzha", model.title()); + Assertions.assertEquals("glcfhmlrqryxyn", model.description()); Assertions.assertEquals(true, model.isVisible()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticSupportTopicTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticSupportTopicTests.java index 54b164330df7b..f3775dd9fd2a5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticSupportTopicTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticSupportTopicTests.java @@ -10,10 +10,8 @@ public final class DiagnosticSupportTopicTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticSupportTopic model = - BinaryData - .fromString("{\"id\":\"bkwdlenrds\",\"pesId\":\"ujbazpjuohminyfl\"}") - .toObject(DiagnosticSupportTopic.class); + DiagnosticSupportTopic model = BinaryData.fromString("{\"id\":\"opv\",\"pesId\":\"dbzqgqqihed\"}") + .toObject(DiagnosticSupportTopic.class); } @org.junit.jupiter.api.Test diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsCollectionInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsCollectionInnerTests.java index 4c91459bd2412..c7738824b28f1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsCollectionInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsCollectionInnerTests.java @@ -22,87 +22,98 @@ public final class DiagnosticsCollectionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsCollectionInner model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"bqaays\",\"name\":\"ixqtn\",\"description\":\"tezlwff\",\"author\":\"akpjpqqmtedlt\",\"category\":\"jihy\",\"supportTopicList\":[{}],\"analysisTypes\":[\"vwau\",\"qncygupkvi\",\"mdscwxqupev\"],\"type\":\"f\",\"score\":56.80255},\"dataset\":[{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}}],\"status\":{\"message\":\"ypelmcu\",\"statusId\":1157427195},\"dataProviderMetadata\":{\"providerName\":\"jxyfwnylrcoolstt\",\"propertyBag\":[{},{}]}},\"id\":\"w\",\"name\":\"kbnujr\",\"type\":\"wvtylbfpncurdo\"},{\"properties\":{\"metadata\":{\"id\":\"thtywub\",\"name\":\"bihwqknfdnt\",\"description\":\"chrdgoihxumwcto\",\"author\":\"zj\",\"category\":\"udfdlwgg\",\"supportTopicList\":[{}],\"analysisTypes\":[\"tov\",\"tgseinqfiufxqkn\",\"irgne\"],\"type\":\"twqmsniffcdmqnr\",\"score\":65.80664},\"dataset\":[{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}}],\"status\":{\"message\":\"dhcrati\",\"statusId\":142407954},\"dataProviderMetadata\":{\"providerName\":\"asxifto\",\"propertyBag\":[{},{},{},{}]}},\"id\":\"zh\",\"name\":\"tw\",\"type\":\"sgogczhonnxk\"}],\"nextLink\":\"gnyhmossxkkg\"}") - .toObject(DiagnosticsCollectionInner.class); - Assertions.assertEquals("vwau", model.value().get(0).properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("ypelmcu", model.value().get(0).properties().status().message()); - Assertions.assertEquals(1157427195, model.value().get(0).properties().status().statusId()); - Assertions - .assertEquals("jxyfwnylrcoolstt", model.value().get(0).properties().dataProviderMetadata().providerName()); + DiagnosticsCollectionInner model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"nzqodfvpg\",\"name\":\"oxgsgbpfgzdjtx\",\"description\":\"flbqvgaq\",\"author\":\"gafcqu\",\"category\":\"dvetnws\",\"supportTopicList\":[{}],\"analysisTypes\":[\"wlduyc\",\"uzhyrmewipmvekdx\",\"kuqgsjjxundxgket\",\"zhhzjhfjmhvvmu\"],\"type\":\"pmuneqsx\",\"score\":97.42455},\"dataset\":[{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}}],\"status\":{\"message\":\"sasbhu\",\"statusId\":1639205644},\"dataProviderMetadata\":{\"providerName\":\"yue\",\"propertyBag\":[{},{},{}]}},\"id\":\"nsqyrpfoobrltt\",\"name\":\"msjnygqdnfw\",\"type\":\"zdzgtilaxhnfhqly\"},{\"properties\":{\"metadata\":{\"id\":\"uwivkxoy\",\"name\":\"nbixxrti\",\"description\":\"cpwpg\",\"author\":\"rc\",\"category\":\"tso\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"nxpmyyefrpmpdnq\",\"skawaoqvmmb\",\"pqfrtqlkz\"],\"type\":\"gnitgvkxlzyq\",\"score\":34.082},\"dataset\":[{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}}],\"status\":{\"message\":\"cansymoyqhlwigd\",\"statusId\":247334898},\"dataProviderMetadata\":{\"providerName\":\"xgomfajuwa\",\"propertyBag\":[{},{},{},{}]}},\"id\":\"aeyyg\",\"name\":\"xakjsqzhzb\",\"type\":\"zkgimsid\"},{\"properties\":{\"metadata\":{\"id\":\"cddyvvjsk\",\"name\":\"mocwa\",\"description\":\"qgatjeaahhvjhhn\",\"author\":\"zybbj\",\"category\":\"dj\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"yxvxevblbjed\",\"ljlageuaulxu\",\"smjbnkppxyn\",\"nlsvxeiz\"],\"type\":\"wklnsrmffeyc\",\"score\":18.504536},\"dataset\":[{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}},{\"table\":{},\"renderingProperties\":{}}],\"status\":{\"message\":\"ammxqi\",\"statusId\":1876834385},\"dataProviderMetadata\":{\"providerName\":\"ddrtkgdojb\",\"propertyBag\":[{},{},{}]}},\"id\":\"vrefdeesv\",\"name\":\"cuijpxt\",\"type\":\"s\"},{\"properties\":{\"metadata\":{\"id\":\"tujwsawdd\",\"name\":\"babxvitit\",\"description\":\"zeexavoxtfgle\",\"author\":\"m\",\"category\":\"bwpypqtgsfjacb\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"udbxv\"],\"type\":\"htnsi\",\"score\":69.404366},\"dataset\":[{\"table\":{},\"renderingProperties\":{}}],\"status\":{\"message\":\"ckdlpag\",\"statusId\":1885722966},\"dataProviderMetadata\":{\"providerName\":\"a\",\"propertyBag\":[{},{},{}]}},\"id\":\"xwmdboxd\",\"name\":\"gsftufqobrjlnacg\",\"type\":\"ckknhxkizvy\"}],\"nextLink\":\"rzvul\"}") + .toObject(DiagnosticsCollectionInner.class); + Assertions.assertEquals("wlduyc", model.value().get(0).properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("sasbhu", model.value().get(0).properties().status().message()); + Assertions.assertEquals(1639205644, model.value().get(0).properties().status().statusId()); + Assertions.assertEquals("yue", model.value().get(0).properties().dataProviderMetadata().providerName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsCollectionInner model = - new DiagnosticsCollectionInner() - .withValue( - Arrays - .asList( - new DiagnosticsInner() - .withProperties( - new DiagnosticsProperties() - .withMetadata( - new DiagnosticsDefinition() - .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) - .withAnalysisTypes(Arrays.asList("vwau", "qncygupkvi", "mdscwxqupev"))) - .withDataset( - Arrays - .asList( - new DiagnosticsDataApiResponse() - .withTable(new DiagnosticDataTableResponseObject()) - .withRenderingProperties(new DiagnosticRendering()), - new DiagnosticsDataApiResponse() - .withTable(new DiagnosticDataTableResponseObject()) - .withRenderingProperties(new DiagnosticRendering()))) - .withStatus( - new DiagnosticsStatus().withMessage("ypelmcu").withStatusId(1157427195)) - .withDataProviderMetadata( - new DiagnosticDataProviderMetadata() - .withProviderName("jxyfwnylrcoolstt") - .withPropertyBag( - Arrays - .asList( - new DiagnosticDataProviderMetadataPropertyBagItem(), - new DiagnosticDataProviderMetadataPropertyBagItem())))), - new DiagnosticsInner() - .withProperties( - new DiagnosticsProperties() - .withMetadata( - new DiagnosticsDefinition() - .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) - .withAnalysisTypes(Arrays.asList("tov", "tgseinqfiufxqkn", "irgne"))) - .withDataset( - Arrays - .asList( - new DiagnosticsDataApiResponse() - .withTable(new DiagnosticDataTableResponseObject()) - .withRenderingProperties(new DiagnosticRendering()), - new DiagnosticsDataApiResponse() - .withTable(new DiagnosticDataTableResponseObject()) - .withRenderingProperties(new DiagnosticRendering()), - new DiagnosticsDataApiResponse() - .withTable(new DiagnosticDataTableResponseObject()) - .withRenderingProperties(new DiagnosticRendering()))) - .withStatus( - new DiagnosticsStatus().withMessage("dhcrati").withStatusId(142407954)) - .withDataProviderMetadata( - new DiagnosticDataProviderMetadata() - .withProviderName("asxifto") - .withPropertyBag( - Arrays - .asList( - new DiagnosticDataProviderMetadataPropertyBagItem(), - new DiagnosticDataProviderMetadataPropertyBagItem(), - new DiagnosticDataProviderMetadataPropertyBagItem(), - new DiagnosticDataProviderMetadataPropertyBagItem())))))); + DiagnosticsCollectionInner model + = new DiagnosticsCollectionInner() + .withValue(Arrays.asList( + new DiagnosticsInner() + .withProperties( + new DiagnosticsProperties() + .withMetadata(new DiagnosticsDefinition() + .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) + .withAnalysisTypes(Arrays.asList("wlduyc", "uzhyrmewipmvekdx", "kuqgsjjxundxgket", + "zhhzjhfjmhvvmu"))) + .withDataset(Arrays.asList( + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()), + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()))) + .withStatus(new DiagnosticsStatus().withMessage("sasbhu").withStatusId(1639205644)) + .withDataProviderMetadata( + new DiagnosticDataProviderMetadata().withProviderName( + "yue").withPropertyBag( + Arrays.asList(new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem())))), + new DiagnosticsInner() + .withProperties( + new DiagnosticsProperties() + .withMetadata(new DiagnosticsDefinition() + .withSupportTopicList( + Arrays.asList(new DiagnosticSupportTopic(), new DiagnosticSupportTopic())) + .withAnalysisTypes(Arrays.asList("nxpmyyefrpmpdnq", "skawaoqvmmb", "pqfrtqlkz"))) + .withDataset(Arrays.asList( + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()), + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()), + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()))) + .withStatus(new DiagnosticsStatus().withMessage("cansymoyqhlwigd").withStatusId( + 247334898)) + .withDataProviderMetadata(new DiagnosticDataProviderMetadata() + .withProviderName("xgomfajuwa") + .withPropertyBag(Arrays.asList(new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem())))), + new DiagnosticsInner() + .withProperties(new DiagnosticsProperties() + .withMetadata(new DiagnosticsDefinition() + .withSupportTopicList( + Arrays.asList(new DiagnosticSupportTopic(), new DiagnosticSupportTopic())) + .withAnalysisTypes( + Arrays.asList("yxvxevblbjed", "ljlageuaulxu", "smjbnkppxyn", "nlsvxeiz"))) + .withDataset(Arrays.asList( + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()), + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()), + new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()))) + .withStatus(new DiagnosticsStatus().withMessage("ammxqi").withStatusId(1876834385)) + .withDataProviderMetadata( + new DiagnosticDataProviderMetadata().withProviderName("ddrtkgdojb") + .withPropertyBag(Arrays.asList(new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem())))), + new DiagnosticsInner().withProperties(new DiagnosticsProperties() + .withMetadata(new DiagnosticsDefinition() + .withSupportTopicList( + Arrays.asList(new DiagnosticSupportTopic(), new DiagnosticSupportTopic())) + .withAnalysisTypes(Arrays.asList("udbxv"))) + .withDataset(Arrays + .asList(new DiagnosticsDataApiResponse().withTable(new DiagnosticDataTableResponseObject()) + .withRenderingProperties(new DiagnosticRendering()))) + .withStatus(new DiagnosticsStatus().withMessage("ckdlpag").withStatusId(1885722966)) + .withDataProviderMetadata(new DiagnosticDataProviderMetadata().withProviderName("a") + .withPropertyBag(Arrays.asList(new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem(), + new DiagnosticDataProviderMetadataPropertyBagItem())))))); model = BinaryData.fromObject(model).toObject(DiagnosticsCollectionInner.class); - Assertions.assertEquals("vwau", model.value().get(0).properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("ypelmcu", model.value().get(0).properties().status().message()); - Assertions.assertEquals(1157427195, model.value().get(0).properties().status().statusId()); - Assertions - .assertEquals("jxyfwnylrcoolstt", model.value().get(0).properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("wlduyc", model.value().get(0).properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("sasbhu", model.value().get(0).properties().status().message()); + Assertions.assertEquals(1639205644, model.value().get(0).properties().status().statusId()); + Assertions.assertEquals("yue", model.value().get(0).properties().dataProviderMetadata().providerName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDataApiResponseTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDataApiResponseTests.java index 76de19a59494a..030f1d800655b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDataApiResponseTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDataApiResponseTests.java @@ -15,54 +15,43 @@ public final class DiagnosticsDataApiResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsDataApiResponse model = - BinaryData - .fromString( - "{\"table\":{\"tableName\":\"wmd\",\"columns\":[{\"columnName\":\"klvxwmyg\",\"dataType\":\"pgpqchiszepnnb\",\"columnType\":\"rxgibbd\"},{\"columnName\":\"confozauors\",\"dataType\":\"okwbqplh\",\"columnType\":\"nuuepzlrp\"}],\"rows\":[\"datasoldweyuqdunv\",\"datannrwrbiork\"]},\"renderingProperties\":{\"type\":476504183,\"title\":\"jhhgdnhxmsi\",\"description\":\"omi\",\"isVisible\":false}}") - .toObject(DiagnosticsDataApiResponse.class); - Assertions.assertEquals("wmd", model.table().tableName()); - Assertions.assertEquals("klvxwmyg", model.table().columns().get(0).columnName()); - Assertions.assertEquals("pgpqchiszepnnb", model.table().columns().get(0).dataType()); - Assertions.assertEquals("rxgibbd", model.table().columns().get(0).columnType()); - Assertions.assertEquals(476504183, model.renderingProperties().type()); - Assertions.assertEquals("jhhgdnhxmsi", model.renderingProperties().title()); - Assertions.assertEquals("omi", model.renderingProperties().description()); + DiagnosticsDataApiResponse model = BinaryData.fromString( + "{\"table\":{\"tableName\":\"wthmkyib\",\"columns\":[{\"columnName\":\"hsgqc\",\"dataType\":\"hohsd\",\"columnType\":\"cdzsu\"},{\"columnName\":\"ohdxbzlmcmu\",\"dataType\":\"cvhd\",\"columnType\":\"v\"},{\"columnName\":\"qxeysko\",\"dataType\":\"zinkfkbgbzbowxeq\",\"columnType\":\"ljmygvkzqkjjeokb\"},{\"columnName\":\"fezrx\",\"dataType\":\"zurtleipqxbkwvz\",\"columnType\":\"zvd\"}],\"rows\":[\"datad\"]},\"renderingProperties\":{\"type\":1814010794,\"title\":\"pnodawopqhe\",\"description\":\"ptmcgsbostzelnd\",\"isVisible\":false}}") + .toObject(DiagnosticsDataApiResponse.class); + Assertions.assertEquals("wthmkyib", model.table().tableName()); + Assertions.assertEquals("hsgqc", model.table().columns().get(0).columnName()); + Assertions.assertEquals("hohsd", model.table().columns().get(0).dataType()); + Assertions.assertEquals("cdzsu", model.table().columns().get(0).columnType()); + Assertions.assertEquals(1814010794, model.renderingProperties().type()); + Assertions.assertEquals("pnodawopqhe", model.renderingProperties().title()); + Assertions.assertEquals("ptmcgsbostzelnd", model.renderingProperties().description()); Assertions.assertEquals(false, model.renderingProperties().isVisible()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsDataApiResponse model = - new DiagnosticsDataApiResponse() - .withTable( - new DiagnosticDataTableResponseObject() - .withTableName("wmd") - .withColumns( - Arrays - .asList( - new DiagnosticDataTableResponseColumn() - .withColumnName("klvxwmyg") - .withDataType("pgpqchiszepnnb") - .withColumnType("rxgibbd"), - new DiagnosticDataTableResponseColumn() - .withColumnName("confozauors") - .withDataType("okwbqplh") - .withColumnType("nuuepzlrp"))) - .withRows(Arrays.asList("datasoldweyuqdunv", "datannrwrbiork"))) - .withRenderingProperties( - new DiagnosticRendering() - .withType(476504183) - .withTitle("jhhgdnhxmsi") - .withDescription("omi") - .withIsVisible(false)); + DiagnosticsDataApiResponse model = new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("wthmkyib") + .withColumns(Arrays.asList( + new DiagnosticDataTableResponseColumn().withColumnName("hsgqc").withDataType("hohsd") + .withColumnType("cdzsu"), + new DiagnosticDataTableResponseColumn().withColumnName("ohdxbzlmcmu").withDataType("cvhd") + .withColumnType("v"), + new DiagnosticDataTableResponseColumn().withColumnName("qxeysko").withDataType("zinkfkbgbzbowxeq") + .withColumnType("ljmygvkzqkjjeokb"), + new DiagnosticDataTableResponseColumn().withColumnName("fezrx").withDataType("zurtleipqxbkwvz") + .withColumnType("zvd"))) + .withRows(Arrays.asList("datad"))) + .withRenderingProperties(new DiagnosticRendering().withType(1814010794).withTitle("pnodawopqhe") + .withDescription("ptmcgsbostzelnd").withIsVisible(false)); model = BinaryData.fromObject(model).toObject(DiagnosticsDataApiResponse.class); - Assertions.assertEquals("wmd", model.table().tableName()); - Assertions.assertEquals("klvxwmyg", model.table().columns().get(0).columnName()); - Assertions.assertEquals("pgpqchiszepnnb", model.table().columns().get(0).dataType()); - Assertions.assertEquals("rxgibbd", model.table().columns().get(0).columnType()); - Assertions.assertEquals(476504183, model.renderingProperties().type()); - Assertions.assertEquals("jhhgdnhxmsi", model.renderingProperties().title()); - Assertions.assertEquals("omi", model.renderingProperties().description()); + Assertions.assertEquals("wthmkyib", model.table().tableName()); + Assertions.assertEquals("hsgqc", model.table().columns().get(0).columnName()); + Assertions.assertEquals("hohsd", model.table().columns().get(0).dataType()); + Assertions.assertEquals("cdzsu", model.table().columns().get(0).columnType()); + Assertions.assertEquals(1814010794, model.renderingProperties().type()); + Assertions.assertEquals("pnodawopqhe", model.renderingProperties().title()); + Assertions.assertEquals("ptmcgsbostzelnd", model.renderingProperties().description()); Assertions.assertEquals(false, model.renderingProperties().isVisible()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDefinitionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDefinitionTests.java index 3fc7910422a19..d4271ef45f599 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDefinitionTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsDefinitionTests.java @@ -13,21 +13,18 @@ public final class DiagnosticsDefinitionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsDefinition model = - BinaryData - .fromString( - "{\"id\":\"yjkqabqgzslesjcb\",\"name\":\"rnntiewdjcv\",\"description\":\"uwrbehwagoh\",\"author\":\"f\",\"category\":\"mrqemvvhmx\",\"supportTopicList\":[{\"id\":\"futacoebjvewzc\",\"pesId\":\"nmwcpmgu\"},{\"id\":\"draufactkah\",\"pesId\":\"v\"}],\"analysisTypes\":[\"ziuxxpshnee\"],\"type\":\"lfg\",\"score\":26.94627}") - .toObject(DiagnosticsDefinition.class); - Assertions.assertEquals("ziuxxpshnee", model.analysisTypes().get(0)); + DiagnosticsDefinition model = BinaryData.fromString( + "{\"id\":\"hlqhykprlpy\",\"name\":\"uciqdsme\",\"description\":\"itdfuxtyasiib\",\"author\":\"ybnnustg\",\"category\":\"jh\",\"supportTopicList\":[{\"id\":\"xhcmavmqfoudo\",\"pesId\":\"cgyypro\"}],\"analysisTypes\":[\"pundmbxhug\",\"mjkavlgorbmft\",\"mdtzfjltfvnzc\"],\"type\":\"t\",\"score\":1.86252}") + .toObject(DiagnosticsDefinition.class); + Assertions.assertEquals("pundmbxhug", model.analysisTypes().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsDefinition model = - new DiagnosticsDefinition() - .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic(), new DiagnosticSupportTopic())) - .withAnalysisTypes(Arrays.asList("ziuxxpshnee")); + DiagnosticsDefinition model + = new DiagnosticsDefinition().withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) + .withAnalysisTypes(Arrays.asList("pundmbxhug", "mjkavlgorbmft", "mdtzfjltfvnzc")); model = BinaryData.fromObject(model).toObject(DiagnosticsDefinition.class); - Assertions.assertEquals("ziuxxpshnee", model.analysisTypes().get(0)); + Assertions.assertEquals("pundmbxhug", model.analysisTypes().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsInnerTests.java index ab8eed7c778f8..e6d2d26a529b1 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsInnerTests.java @@ -22,137 +22,73 @@ public final class DiagnosticsInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsInner model = - BinaryData - .fromString( - "{\"properties\":{\"metadata\":{\"id\":\"gh\",\"name\":\"bdhqxvcxgf\",\"description\":\"dsofbshrns\",\"author\":\"uswdv\",\"category\":\"ybycnunvj\",\"supportTopicList\":[{\"id\":\"f\",\"pesId\":\"nopqgikyzirtx\"}],\"analysisTypes\":[\"x\",\"ejnt\",\"sewgioilqukr\"],\"type\":\"xtqmieoxor\",\"score\":64.62172},\"dataset\":[{\"table\":{\"tableName\":\"omtbghhavgrvkff\",\"columns\":[{},{},{},{}],\"rows\":[\"datapjbi\",\"datagjmfxumvfcl\"]},\"renderingProperties\":{\"type\":2107633088,\"title\":\"xnbkfezzxscyhwzd\",\"description\":\"rujbzbomvzzbtdc\",\"isVisible\":true}},{\"table\":{\"tableName\":\"yujviylwdshfssn\",\"columns\":[{}],\"rows\":[\"datafrymsgaojfmwnc\"]},\"renderingProperties\":{\"type\":700953056,\"title\":\"hirctymoxoftpipi\",\"description\":\"czuhxacpqjlihh\",\"isVisible\":true}},{\"table\":{\"tableName\":\"kasdvlm\",\"columns\":[{},{},{},{}],\"rows\":[\"datax\",\"datalucvpam\",\"datasreuzvxurisjnh\",\"dataytxifqjzgxmrh\"]},\"renderingProperties\":{\"type\":403382899,\"title\":\"cesutrgjupauut\",\"description\":\"oqh\",\"isVisible\":false}},{\"table\":{\"tableName\":\"g\",\"columns\":[{},{},{}],\"rows\":[\"dataqntcypsxjvfoimwk\",\"datalirc\"]},\"renderingProperties\":{\"type\":1956254866,\"title\":\"ydfce\",\"description\":\"vlhv\",\"isVisible\":true}}],\"status\":{\"message\":\"t\",\"statusId\":642737212},\"dataProviderMetadata\":{\"providerName\":\"nawjslbiwkojgcy\",\"propertyBag\":[{\"name\":\"mznbaeqphch\",\"value\":\"rn\"},{\"name\":\"x\",\"value\":\"uwrykqgaifmvikl\"},{\"name\":\"dvk\",\"value\":\"ejd\"}]}},\"id\":\"nxcvds\",\"name\":\"hnjivo\",\"type\":\"v\"}") - .toObject(DiagnosticsInner.class); - Assertions.assertEquals("x", model.properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("omtbghhavgrvkff", model.properties().dataset().get(0).table().tableName()); - Assertions.assertEquals(2107633088, model.properties().dataset().get(0).renderingProperties().type()); - Assertions.assertEquals("xnbkfezzxscyhwzd", model.properties().dataset().get(0).renderingProperties().title()); - Assertions - .assertEquals("rujbzbomvzzbtdc", model.properties().dataset().get(0).renderingProperties().description()); + DiagnosticsInner model = BinaryData.fromString( + "{\"properties\":{\"metadata\":{\"id\":\"eranokqgukkjqnv\",\"name\":\"oylaxxul\",\"description\":\"isdos\",\"author\":\"b\",\"category\":\"vgjrwhr\",\"supportTopicList\":[{\"id\":\"ytdc\",\"pesId\":\"gc\"}],\"analysisTypes\":[\"fnwmbtmvpdvj\"],\"type\":\"ttzaefed\",\"score\":85.42932},\"dataset\":[{\"table\":{\"tableName\":\"hk\",\"columns\":[{},{},{}],\"rows\":[\"dataqnsdfzpbgtgky\"]},\"renderingProperties\":{\"type\":50734303,\"title\":\"rjeuut\",\"description\":\"xezw\",\"isVisible\":true}},{\"table\":{\"tableName\":\"bwnhhtql\",\"columns\":[{},{},{},{}],\"rows\":[\"datapipifh\",\"dataf\",\"dataoajvgcxtxjcs\"]},\"renderingProperties\":{\"type\":1665237289,\"title\":\"dltug\",\"description\":\"esmkssjhoiftxfkf\",\"isVisible\":false}},{\"table\":{\"tableName\":\"hpt\",\"columns\":[{},{},{},{}],\"rows\":[\"databiqtgdq\",\"datahm\",\"datawsldrizetpwbr\"]},\"renderingProperties\":{\"type\":1155046174,\"title\":\"bphbqzmizakakank\",\"description\":\"dnjzh\",\"isVisible\":true}}],\"status\":{\"message\":\"hjlmu\",\"statusId\":29560725},\"dataProviderMetadata\":{\"providerName\":\"imrsopteecjmei\",\"propertyBag\":[{\"name\":\"vasylwxdzau\",\"value\":\"eoohguufuzboyj\"},{\"name\":\"hwtzolb\",\"value\":\"mwmdxmebwjscjpa\"},{\"name\":\"xveabf\",\"value\":\"nmwmqtibx\"},{\"name\":\"jddtvqct\",\"value\":\"dija\"}]}},\"id\":\"kmr\",\"name\":\"ieekpndzaa\",\"type\":\"mudqmeq\"}") + .toObject(DiagnosticsInner.class); + Assertions.assertEquals("fnwmbtmvpdvj", model.properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("hk", model.properties().dataset().get(0).table().tableName()); + Assertions.assertEquals(50734303, model.properties().dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("rjeuut", model.properties().dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("xezw", model.properties().dataset().get(0).renderingProperties().description()); Assertions.assertEquals(true, model.properties().dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("t", model.properties().status().message()); - Assertions.assertEquals(642737212, model.properties().status().statusId()); - Assertions.assertEquals("nawjslbiwkojgcy", model.properties().dataProviderMetadata().providerName()); - Assertions.assertEquals("mznbaeqphch", model.properties().dataProviderMetadata().propertyBag().get(0).name()); - Assertions.assertEquals("rn", model.properties().dataProviderMetadata().propertyBag().get(0).value()); + Assertions.assertEquals("hjlmu", model.properties().status().message()); + Assertions.assertEquals(29560725, model.properties().status().statusId()); + Assertions.assertEquals("imrsopteecjmei", model.properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("vasylwxdzau", model.properties().dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("eoohguufuzboyj", + model.properties().dataProviderMetadata().propertyBag().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsInner model = - new DiagnosticsInner() - .withProperties( - new DiagnosticsProperties() - .withMetadata( - new DiagnosticsDefinition() - .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) - .withAnalysisTypes(Arrays.asList("x", "ejnt", "sewgioilqukr"))) - .withDataset( - Arrays - .asList( - new DiagnosticsDataApiResponse() - .withTable( - new DiagnosticDataTableResponseObject() - .withTableName("omtbghhavgrvkff") - .withColumns( - Arrays - .asList( - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn())) - .withRows(Arrays.asList("datapjbi", "datagjmfxumvfcl"))) - .withRenderingProperties( - new DiagnosticRendering() - .withType(2107633088) - .withTitle("xnbkfezzxscyhwzd") - .withDescription("rujbzbomvzzbtdc") - .withIsVisible(true)), - new DiagnosticsDataApiResponse() - .withTable( - new DiagnosticDataTableResponseObject() - .withTableName("yujviylwdshfssn") - .withColumns(Arrays.asList(new DiagnosticDataTableResponseColumn())) - .withRows(Arrays.asList("datafrymsgaojfmwnc"))) - .withRenderingProperties( - new DiagnosticRendering() - .withType(700953056) - .withTitle("hirctymoxoftpipi") - .withDescription("czuhxacpqjlihh") - .withIsVisible(true)), - new DiagnosticsDataApiResponse() - .withTable( - new DiagnosticDataTableResponseObject() - .withTableName("kasdvlm") - .withColumns( - Arrays - .asList( - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn())) - .withRows( - Arrays - .asList( - "datax", - "datalucvpam", - "datasreuzvxurisjnh", - "dataytxifqjzgxmrh"))) - .withRenderingProperties( - new DiagnosticRendering() - .withType(403382899) - .withTitle("cesutrgjupauut") - .withDescription("oqh") - .withIsVisible(false)), - new DiagnosticsDataApiResponse() - .withTable( - new DiagnosticDataTableResponseObject() - .withTableName("g") - .withColumns( - Arrays - .asList( - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn(), - new DiagnosticDataTableResponseColumn())) - .withRows(Arrays.asList("dataqntcypsxjvfoimwk", "datalirc"))) - .withRenderingProperties( - new DiagnosticRendering() - .withType(1956254866) - .withTitle("ydfce") - .withDescription("vlhv") - .withIsVisible(true)))) - .withStatus(new DiagnosticsStatus().withMessage("t").withStatusId(642737212)) - .withDataProviderMetadata( - new DiagnosticDataProviderMetadata() - .withProviderName("nawjslbiwkojgcy") - .withPropertyBag( - Arrays - .asList( - new DiagnosticDataProviderMetadataPropertyBagItem() - .withName("mznbaeqphch") - .withValue("rn"), - new DiagnosticDataProviderMetadataPropertyBagItem() - .withName("x") - .withValue("uwrykqgaifmvikl"), - new DiagnosticDataProviderMetadataPropertyBagItem() - .withName("dvk") - .withValue("ejd"))))); + DiagnosticsInner model = new DiagnosticsInner().withProperties(new DiagnosticsProperties() + .withMetadata(new DiagnosticsDefinition().withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) + .withAnalysisTypes(Arrays.asList("fnwmbtmvpdvj"))) + .withDataset(Arrays.asList( + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("hk") + .withColumns(Arrays.asList(new DiagnosticDataTableResponseColumn(), + new DiagnosticDataTableResponseColumn(), new DiagnosticDataTableResponseColumn())) + .withRows(Arrays.asList("dataqnsdfzpbgtgky"))) + .withRenderingProperties(new DiagnosticRendering().withType(50734303).withTitle("rjeuut") + .withDescription("xezw").withIsVisible(true)), + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("bwnhhtql") + .withColumns(Arrays.asList(new DiagnosticDataTableResponseColumn(), + new DiagnosticDataTableResponseColumn(), new DiagnosticDataTableResponseColumn(), + new DiagnosticDataTableResponseColumn())) + .withRows(Arrays.asList("datapipifh", "dataf", "dataoajvgcxtxjcs"))) + .withRenderingProperties(new DiagnosticRendering().withType(1665237289).withTitle("dltug") + .withDescription("esmkssjhoiftxfkf").withIsVisible(false)), + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("hpt") + .withColumns(Arrays.asList(new DiagnosticDataTableResponseColumn(), + new DiagnosticDataTableResponseColumn(), new DiagnosticDataTableResponseColumn(), + new DiagnosticDataTableResponseColumn())) + .withRows(Arrays.asList("databiqtgdq", "datahm", "datawsldrizetpwbr"))) + .withRenderingProperties(new DiagnosticRendering().withType(1155046174) + .withTitle("bphbqzmizakakank").withDescription("dnjzh").withIsVisible(true)))) + .withStatus(new DiagnosticsStatus().withMessage("hjlmu").withStatusId(29560725)) + .withDataProviderMetadata(new DiagnosticDataProviderMetadata().withProviderName("imrsopteecjmei") + .withPropertyBag(Arrays.asList( + new DiagnosticDataProviderMetadataPropertyBagItem().withName("vasylwxdzau") + .withValue("eoohguufuzboyj"), + new DiagnosticDataProviderMetadataPropertyBagItem().withName("hwtzolb") + .withValue("mwmdxmebwjscjpa"), + new DiagnosticDataProviderMetadataPropertyBagItem().withName("xveabf").withValue("nmwmqtibx"), + new DiagnosticDataProviderMetadataPropertyBagItem().withName("jddtvqct").withValue("dija"))))); model = BinaryData.fromObject(model).toObject(DiagnosticsInner.class); - Assertions.assertEquals("x", model.properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("omtbghhavgrvkff", model.properties().dataset().get(0).table().tableName()); - Assertions.assertEquals(2107633088, model.properties().dataset().get(0).renderingProperties().type()); - Assertions.assertEquals("xnbkfezzxscyhwzd", model.properties().dataset().get(0).renderingProperties().title()); - Assertions - .assertEquals("rujbzbomvzzbtdc", model.properties().dataset().get(0).renderingProperties().description()); + Assertions.assertEquals("fnwmbtmvpdvj", model.properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("hk", model.properties().dataset().get(0).table().tableName()); + Assertions.assertEquals(50734303, model.properties().dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("rjeuut", model.properties().dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("xezw", model.properties().dataset().get(0).renderingProperties().description()); Assertions.assertEquals(true, model.properties().dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("t", model.properties().status().message()); - Assertions.assertEquals(642737212, model.properties().status().statusId()); - Assertions.assertEquals("nawjslbiwkojgcy", model.properties().dataProviderMetadata().providerName()); - Assertions.assertEquals("mznbaeqphch", model.properties().dataProviderMetadata().propertyBag().get(0).name()); - Assertions.assertEquals("rn", model.properties().dataProviderMetadata().propertyBag().get(0).value()); + Assertions.assertEquals("hjlmu", model.properties().status().message()); + Assertions.assertEquals(29560725, model.properties().status().statusId()); + Assertions.assertEquals("imrsopteecjmei", model.properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("vasylwxdzau", model.properties().dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("eoohguufuzboyj", + model.properties().dataProviderMetadata().propertyBag().get(0).value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsPropertiesTests.java index 61ca916d10c3c..3f5416c74a0ff 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsPropertiesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsPropertiesTests.java @@ -21,81 +21,87 @@ public final class DiagnosticsPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsProperties model = - BinaryData - .fromString( - "{\"metadata\":{\"id\":\"v\",\"name\":\"zgemjdftuljlt\",\"description\":\"ceamtm\",\"author\":\"u\",\"category\":\"ejwcwwqiok\",\"supportTopicList\":[{\"id\":\"mojmsvpkjprvkw\",\"pesId\":\"zqljyxgtczh\"}],\"analysisTypes\":[\"bsdshmkxmaehvbbx\",\"ripltf\"],\"type\":\"tbaxk\",\"score\":35.67025},\"dataset\":[{\"table\":{\"tableName\":\"pyklyhpluodpvru\",\"columns\":[{\"columnName\":\"zibt\",\"dataType\":\"stgktst\",\"columnType\":\"xeclzedqbcvhzlhp\"}],\"rows\":[\"dataqkdlw\",\"dataqfbumlkxtrqjf\",\"datamlmbtxhwgfwsrta\"]},\"renderingProperties\":{\"type\":337783528,\"title\":\"brhu\",\"description\":\"kh\",\"isVisible\":false}}],\"status\":{\"message\":\"ookk\",\"statusId\":870237510},\"dataProviderMetadata\":{\"providerName\":\"vleo\",\"propertyBag\":[{\"name\":\"uiqtqzfavy\",\"value\":\"qqybarye\"}]}}") - .toObject(DiagnosticsProperties.class); - Assertions.assertEquals("bsdshmkxmaehvbbx", model.metadata().analysisTypes().get(0)); - Assertions.assertEquals("pyklyhpluodpvru", model.dataset().get(0).table().tableName()); - Assertions.assertEquals("zibt", model.dataset().get(0).table().columns().get(0).columnName()); - Assertions.assertEquals("stgktst", model.dataset().get(0).table().columns().get(0).dataType()); - Assertions.assertEquals("xeclzedqbcvhzlhp", model.dataset().get(0).table().columns().get(0).columnType()); - Assertions.assertEquals(337783528, model.dataset().get(0).renderingProperties().type()); - Assertions.assertEquals("brhu", model.dataset().get(0).renderingProperties().title()); - Assertions.assertEquals("kh", model.dataset().get(0).renderingProperties().description()); - Assertions.assertEquals(false, model.dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("ookk", model.status().message()); - Assertions.assertEquals(870237510, model.status().statusId()); - Assertions.assertEquals("vleo", model.dataProviderMetadata().providerName()); - Assertions.assertEquals("uiqtqzfavy", model.dataProviderMetadata().propertyBag().get(0).name()); - Assertions.assertEquals("qqybarye", model.dataProviderMetadata().propertyBag().get(0).value()); + DiagnosticsProperties model = BinaryData.fromString( + "{\"metadata\":{\"id\":\"pibudqwyxebeybpm\",\"name\":\"nrtffyaqi\",\"description\":\"hheioqaqhvseuf\",\"author\":\"yrxpdlcgqls\",\"category\":\"mjqfrddgamquhio\",\"supportTopicList\":[{\"id\":\"uivfcdis\",\"pesId\":\"rnxzh\"},{\"id\":\"exrxzbujrtrhq\",\"pesId\":\"revkhgnlnzo\"},{\"id\":\"lrpiqywnc\",\"pesId\":\"tszcofizehtdhgb\"},{\"id\":\"vreljea\",\"pesId\":\"rvzmlovuana\"}],\"analysisTypes\":[\"xlpm\",\"erbdk\",\"lvidizozs\"],\"type\":\"ccxjm\",\"score\":89.4738},\"dataset\":[{\"table\":{\"tableName\":\"n\",\"columns\":[{\"columnName\":\"uwwltvuqjctz\",\"dataType\":\"keifzzhmkdasv\",\"columnType\":\"yhbxcudchxgs\"}],\"rows\":[\"dataldforobwj\",\"datavizbfhfo\",\"datavacqpbtuodxesz\"]},\"renderingProperties\":{\"type\":1654037219,\"title\":\"awumuaslzkwrrwo\",\"description\":\"qucwyhahnom\",\"isVisible\":true}},{\"table\":{\"tableName\":\"uh\",\"columns\":[{\"columnName\":\"uurutlwexxwlalni\",\"dataType\":\"zsrzpgepq\",\"columnType\":\"bb\"},{\"columnName\":\"pgdakchzyvli\",\"dataType\":\"nrkcxkj\",\"columnType\":\"nxm\"}],\"rows\":[\"dataxs\",\"dataqrntv\",\"datawijpsttexoqqpwc\"]},\"renderingProperties\":{\"type\":402556977,\"title\":\"hruncuwmq\",\"description\":\"kcdqzhlct\",\"isVisible\":true}},{\"table\":{\"tableName\":\"ndy\",\"columns\":[{\"columnName\":\"rqbnjjrc\",\"dataType\":\"gydcw\",\"columnType\":\"xjumvq\"},{\"columnName\":\"lihrraiouaubr\",\"dataType\":\"loqxfuojrngif\",\"columnType\":\"z\"}],\"rows\":[\"dataccbiuimzdlyjdfq\",\"datamkyoqufdvruzsl\",\"dataojhp\"]},\"renderingProperties\":{\"type\":320738899,\"title\":\"dxotng\",\"description\":\"gugey\",\"isVisible\":false}},{\"table\":{\"tableName\":\"kyuizabsnmfpp\",\"columns\":[{\"columnName\":\"evy\",\"dataType\":\"hsgz\",\"columnType\":\"zbgomfgbeg\"}],\"rows\":[\"dataleohibetnluankr\",\"datafxeeebtijvacvbm\"]},\"renderingProperties\":{\"type\":1802665958,\"title\":\"xl\",\"description\":\"rnwxacevehjkuyxo\",\"isVisible\":false}}],\"status\":{\"message\":\"qltfaey\",\"statusId\":277162772},\"dataProviderMetadata\":{\"providerName\":\"gv\",\"propertyBag\":[{\"name\":\"gh\",\"value\":\"ypoq\"}]}}") + .toObject(DiagnosticsProperties.class); + Assertions.assertEquals("xlpm", model.metadata().analysisTypes().get(0)); + Assertions.assertEquals("n", model.dataset().get(0).table().tableName()); + Assertions.assertEquals("uwwltvuqjctz", model.dataset().get(0).table().columns().get(0).columnName()); + Assertions.assertEquals("keifzzhmkdasv", model.dataset().get(0).table().columns().get(0).dataType()); + Assertions.assertEquals("yhbxcudchxgs", model.dataset().get(0).table().columns().get(0).columnType()); + Assertions.assertEquals(1654037219, model.dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("awumuaslzkwrrwo", model.dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("qucwyhahnom", model.dataset().get(0).renderingProperties().description()); + Assertions.assertEquals(true, model.dataset().get(0).renderingProperties().isVisible()); + Assertions.assertEquals("qltfaey", model.status().message()); + Assertions.assertEquals(277162772, model.status().statusId()); + Assertions.assertEquals("gv", model.dataProviderMetadata().providerName()); + Assertions.assertEquals("gh", model.dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("ypoq", model.dataProviderMetadata().propertyBag().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsProperties model = - new DiagnosticsProperties() - .withMetadata( - new DiagnosticsDefinition() - .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic())) - .withAnalysisTypes(Arrays.asList("bsdshmkxmaehvbbx", "ripltf"))) - .withDataset( - Arrays - .asList( - new DiagnosticsDataApiResponse() - .withTable( - new DiagnosticDataTableResponseObject() - .withTableName("pyklyhpluodpvru") - .withColumns( - Arrays - .asList( - new DiagnosticDataTableResponseColumn() - .withColumnName("zibt") - .withDataType("stgktst") - .withColumnType("xeclzedqbcvhzlhp"))) - .withRows( - Arrays.asList("dataqkdlw", "dataqfbumlkxtrqjf", "datamlmbtxhwgfwsrta"))) - .withRenderingProperties( - new DiagnosticRendering() - .withType(337783528) - .withTitle("brhu") - .withDescription("kh") - .withIsVisible(false)))) - .withStatus(new DiagnosticsStatus().withMessage("ookk").withStatusId(870237510)) + DiagnosticsProperties model + = new DiagnosticsProperties() + .withMetadata(new DiagnosticsDefinition() + .withSupportTopicList(Arrays.asList(new DiagnosticSupportTopic(), new DiagnosticSupportTopic(), + new DiagnosticSupportTopic(), new DiagnosticSupportTopic())) + .withAnalysisTypes(Arrays.asList("xlpm", "erbdk", "lvidizozs"))) + .withDataset(Arrays.asList( + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("n") + .withColumns( + Arrays.asList(new DiagnosticDataTableResponseColumn().withColumnName("uwwltvuqjctz") + .withDataType("keifzzhmkdasv").withColumnType("yhbxcudchxgs"))) + .withRows(Arrays.asList("dataldforobwj", "datavizbfhfo", "datavacqpbtuodxesz"))) + .withRenderingProperties(new DiagnosticRendering().withType(1654037219) + .withTitle("awumuaslzkwrrwo").withDescription("qucwyhahnom").withIsVisible(true)), + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("uh") + .withColumns(Arrays.asList( + new DiagnosticDataTableResponseColumn().withColumnName("uurutlwexxwlalni") + .withDataType("zsrzpgepq").withColumnType("bb"), + new DiagnosticDataTableResponseColumn().withColumnName("pgdakchzyvli") + .withDataType("nrkcxkj").withColumnType("nxm"))) + .withRows(Arrays.asList("dataxs", "dataqrntv", "datawijpsttexoqqpwc"))) + .withRenderingProperties(new DiagnosticRendering().withType(402556977).withTitle("hruncuwmq") + .withDescription("kcdqzhlct").withIsVisible(true)), + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("ndy") + .withColumns(Arrays.asList( + new DiagnosticDataTableResponseColumn().withColumnName("rqbnjjrc").withDataType("gydcw") + .withColumnType("xjumvq"), + new DiagnosticDataTableResponseColumn().withColumnName("lihrraiouaubr") + .withDataType("loqxfuojrngif").withColumnType("z"))) + .withRows(Arrays.asList("dataccbiuimzdlyjdfq", "datamkyoqufdvruzsl", "dataojhp"))) + .withRenderingProperties(new DiagnosticRendering().withType(320738899).withTitle("dxotng") + .withDescription("gugey").withIsVisible(false)), + new DiagnosticsDataApiResponse() + .withTable(new DiagnosticDataTableResponseObject().withTableName("kyuizabsnmfpp") + .withColumns(Arrays.asList(new DiagnosticDataTableResponseColumn().withColumnName("evy") + .withDataType("hsgz").withColumnType("zbgomfgbeg"))) + .withRows(Arrays.asList("dataleohibetnluankr", "datafxeeebtijvacvbm"))) + .withRenderingProperties(new DiagnosticRendering().withType(1802665958).withTitle("xl") + .withDescription("rnwxacevehjkuyxo").withIsVisible(false)))) + .withStatus(new DiagnosticsStatus().withMessage("qltfaey").withStatusId(277162772)) .withDataProviderMetadata( - new DiagnosticDataProviderMetadata() - .withProviderName("vleo") - .withPropertyBag( - Arrays - .asList( - new DiagnosticDataProviderMetadataPropertyBagItem() - .withName("uiqtqzfavy") - .withValue("qqybarye")))); + new DiagnosticDataProviderMetadata().withProviderName("gv").withPropertyBag(Arrays + .asList(new DiagnosticDataProviderMetadataPropertyBagItem().withName("gh").withValue("ypoq")))); model = BinaryData.fromObject(model).toObject(DiagnosticsProperties.class); - Assertions.assertEquals("bsdshmkxmaehvbbx", model.metadata().analysisTypes().get(0)); - Assertions.assertEquals("pyklyhpluodpvru", model.dataset().get(0).table().tableName()); - Assertions.assertEquals("zibt", model.dataset().get(0).table().columns().get(0).columnName()); - Assertions.assertEquals("stgktst", model.dataset().get(0).table().columns().get(0).dataType()); - Assertions.assertEquals("xeclzedqbcvhzlhp", model.dataset().get(0).table().columns().get(0).columnType()); - Assertions.assertEquals(337783528, model.dataset().get(0).renderingProperties().type()); - Assertions.assertEquals("brhu", model.dataset().get(0).renderingProperties().title()); - Assertions.assertEquals("kh", model.dataset().get(0).renderingProperties().description()); - Assertions.assertEquals(false, model.dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("ookk", model.status().message()); - Assertions.assertEquals(870237510, model.status().statusId()); - Assertions.assertEquals("vleo", model.dataProviderMetadata().providerName()); - Assertions.assertEquals("uiqtqzfavy", model.dataProviderMetadata().propertyBag().get(0).name()); - Assertions.assertEquals("qqybarye", model.dataProviderMetadata().propertyBag().get(0).value()); + Assertions.assertEquals("xlpm", model.metadata().analysisTypes().get(0)); + Assertions.assertEquals("n", model.dataset().get(0).table().tableName()); + Assertions.assertEquals("uwwltvuqjctz", model.dataset().get(0).table().columns().get(0).columnName()); + Assertions.assertEquals("keifzzhmkdasv", model.dataset().get(0).table().columns().get(0).dataType()); + Assertions.assertEquals("yhbxcudchxgs", model.dataset().get(0).table().columns().get(0).columnType()); + Assertions.assertEquals(1654037219, model.dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("awumuaslzkwrrwo", model.dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("qucwyhahnom", model.dataset().get(0).renderingProperties().description()); + Assertions.assertEquals(true, model.dataset().get(0).renderingProperties().isVisible()); + Assertions.assertEquals("qltfaey", model.status().message()); + Assertions.assertEquals(277162772, model.status().statusId()); + Assertions.assertEquals("gv", model.dataProviderMetadata().providerName()); + Assertions.assertEquals("gh", model.dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("ypoq", model.dataProviderMetadata().propertyBag().get(0).value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsStatusTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsStatusTests.java index a7b7b21c68dc5..8f497b3cec3ec 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsStatusTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DiagnosticsStatusTests.java @@ -11,17 +11,17 @@ public final class DiagnosticsStatusTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsStatus model = - BinaryData.fromString("{\"message\":\"isavok\",\"statusId\":1487761087}").toObject(DiagnosticsStatus.class); - Assertions.assertEquals("isavok", model.message()); - Assertions.assertEquals(1487761087, model.statusId()); + DiagnosticsStatus model = BinaryData.fromString("{\"message\":\"dpsovwxznptgo\",\"statusId\":935021144}") + .toObject(DiagnosticsStatus.class); + Assertions.assertEquals("dpsovwxznptgo", model.message()); + Assertions.assertEquals(935021144, model.statusId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsStatus model = new DiagnosticsStatus().withMessage("isavok").withStatusId(1487761087); + DiagnosticsStatus model = new DiagnosticsStatus().withMessage("dpsovwxznptgo").withStatusId(935021144); model = BinaryData.fromObject(model).toObject(DiagnosticsStatus.class); - Assertions.assertEquals("isavok", model.message()); - Assertions.assertEquals(1487761087, model.statusId()); + Assertions.assertEquals("dpsovwxznptgo", model.message()); + Assertions.assertEquals(935021144, model.statusId()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentConfigurationPropertyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentConfigurationPropertyTests.java new file mode 100644 index 0000000000000..f29915ca3f90e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentConfigurationPropertyTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import org.junit.jupiter.api.Assertions; + +public final class DotNetComponentConfigurationPropertyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DotNetComponentConfigurationProperty model + = BinaryData.fromString("{\"propertyName\":\"gjlgvvpase\",\"value\":\"gbuxantuygdh\"}") + .toObject(DotNetComponentConfigurationProperty.class); + Assertions.assertEquals("gjlgvvpase", model.propertyName()); + Assertions.assertEquals("gbuxantuygdh", model.value()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DotNetComponentConfigurationProperty model + = new DotNetComponentConfigurationProperty().withPropertyName("gjlgvvpase").withValue("gbuxantuygdh"); + model = BinaryData.fromObject(model).toObject(DotNetComponentConfigurationProperty.class); + Assertions.assertEquals("gjlgvvpase", model.propertyName()); + Assertions.assertEquals("gbuxantuygdh", model.value()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentInnerTests.java new file mode 100644 index 0000000000000..e27a5336ff69c --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentInnerTests.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class DotNetComponentInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DotNetComponentInner model = BinaryData.fromString( + "{\"properties\":{\"componentType\":\"AspireDashboard\",\"provisioningState\":\"Deleting\",\"configurations\":[{\"propertyName\":\"rx\",\"value\":\"sojklinhmdptys\"},{\"propertyName\":\"qsgnzxojpsl\",\"value\":\"jgpliuf\"},{\"propertyName\":\"woyxqvapcohhou\",\"value\":\"pqojxcx\"}],\"serviceBinds\":[{\"name\":\"cgdz\",\"serviceId\":\"nr\"},{\"name\":\"cawetzqddt\",\"serviceId\":\"fljhznamtua\"},{\"name\":\"zwcjjncqtj\",\"serviceId\":\"izvg\"}]},\"id\":\"atzuuv\",\"name\":\"xngre\",\"type\":\"wggahttzlswvaj\"}") + .toObject(DotNetComponentInner.class); + Assertions.assertEquals(DotNetComponentType.ASPIRE_DASHBOARD, model.componentType()); + Assertions.assertEquals("rx", model.configurations().get(0).propertyName()); + Assertions.assertEquals("sojklinhmdptys", model.configurations().get(0).value()); + Assertions.assertEquals("cgdz", model.serviceBinds().get(0).name()); + Assertions.assertEquals("nr", model.serviceBinds().get(0).serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DotNetComponentInner model = new DotNetComponentInner().withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("rx").withValue("sojklinhmdptys"), + new DotNetComponentConfigurationProperty().withPropertyName("qsgnzxojpsl").withValue("jgpliuf"), + new DotNetComponentConfigurationProperty().withPropertyName("woyxqvapcohhou").withValue("pqojxcx"))) + .withServiceBinds(Arrays.asList(new DotNetComponentServiceBind().withName("cgdz").withServiceId("nr"), + new DotNetComponentServiceBind().withName("cawetzqddt").withServiceId("fljhznamtua"), + new DotNetComponentServiceBind().withName("zwcjjncqtj").withServiceId("izvg"))); + model = BinaryData.fromObject(model).toObject(DotNetComponentInner.class); + Assertions.assertEquals(DotNetComponentType.ASPIRE_DASHBOARD, model.componentType()); + Assertions.assertEquals("rx", model.configurations().get(0).propertyName()); + Assertions.assertEquals("sojklinhmdptys", model.configurations().get(0).value()); + Assertions.assertEquals("cgdz", model.serviceBinds().get(0).name()); + Assertions.assertEquals("nr", model.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentPropertiesTests.java new file mode 100644 index 0000000000000..813190824ddfd --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentPropertiesTests.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentProperties; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class DotNetComponentPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DotNetComponentProperties model = BinaryData.fromString( + "{\"componentType\":\"AspireResourceServerApi\",\"provisioningState\":\"Succeeded\",\"configurations\":[{\"propertyName\":\"qzasunwqrjzfrgqh\",\"value\":\"hcmbuocnjrohmbp\"},{\"propertyName\":\"yx\",\"value\":\"eblydyv\"}],\"serviceBinds\":[{\"name\":\"rocxnehvs\",\"serviceId\":\"odlfpyapucygvo\"},{\"name\":\"yunssx\",\"serviceId\":\"hi\"}]}") + .toObject(DotNetComponentProperties.class); + Assertions.assertEquals(DotNetComponentType.ASPIRE_RESOURCE_SERVER_API, model.componentType()); + Assertions.assertEquals("qzasunwqrjzfrgqh", model.configurations().get(0).propertyName()); + Assertions.assertEquals("hcmbuocnjrohmbp", model.configurations().get(0).value()); + Assertions.assertEquals("rocxnehvs", model.serviceBinds().get(0).name()); + Assertions.assertEquals("odlfpyapucygvo", model.serviceBinds().get(0).serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DotNetComponentProperties model = new DotNetComponentProperties() + .withComponentType(DotNetComponentType.ASPIRE_RESOURCE_SERVER_API) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("qzasunwqrjzfrgqh") + .withValue("hcmbuocnjrohmbp"), + new DotNetComponentConfigurationProperty().withPropertyName("yx").withValue("eblydyv"))) + .withServiceBinds( + Arrays.asList(new DotNetComponentServiceBind().withName("rocxnehvs").withServiceId("odlfpyapucygvo"), + new DotNetComponentServiceBind().withName("yunssx").withServiceId("hi"))); + model = BinaryData.fromObject(model).toObject(DotNetComponentProperties.class); + Assertions.assertEquals(DotNetComponentType.ASPIRE_RESOURCE_SERVER_API, model.componentType()); + Assertions.assertEquals("qzasunwqrjzfrgqh", model.configurations().get(0).propertyName()); + Assertions.assertEquals("hcmbuocnjrohmbp", model.configurations().get(0).value()); + Assertions.assertEquals("rocxnehvs", model.serviceBinds().get(0).name()); + Assertions.assertEquals("odlfpyapucygvo", model.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentServiceBindTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentServiceBindTests.java new file mode 100644 index 0000000000000..c988c2e532679 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentServiceBindTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import org.junit.jupiter.api.Assertions; + +public final class DotNetComponentServiceBindTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DotNetComponentServiceBind model = BinaryData.fromString("{\"name\":\"qipir\",\"serviceId\":\"wrq\"}") + .toObject(DotNetComponentServiceBind.class); + Assertions.assertEquals("qipir", model.name()); + Assertions.assertEquals("wrq", model.serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DotNetComponentServiceBind model = new DotNetComponentServiceBind().withName("qipir").withServiceId("wrq"); + model = BinaryData.fromObject(model).toObject(DotNetComponentServiceBind.class); + Assertions.assertEquals("qipir", model.name()); + Assertions.assertEquals("wrq", model.serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCollectionTests.java new file mode 100644 index 0000000000000..b7bb2687921e8 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCollectionTests.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.DotNetComponentInner; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentsCollection; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class DotNetComponentsCollectionTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DotNetComponentsCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"componentType\":\"AspireDashboard\",\"provisioningState\":\"Succeeded\",\"configurations\":[{\"propertyName\":\"xigdhxiidlope\",\"value\":\"wdpyq\"}],\"serviceBinds\":[{\"name\":\"ubmdnafcbqwre\",\"serviceId\":\"ela\"},{\"name\":\"cigeleohdbvqvw\",\"serviceId\":\"jopwbeonrlkwz\"}]},\"id\":\"ybxc\",\"name\":\"akxcptsoqfyiaseq\",\"type\":\"hkrttzrazis\"}],\"nextLink\":\"kiuemv\"}") + .toObject(DotNetComponentsCollection.class); + Assertions.assertEquals(DotNetComponentType.ASPIRE_DASHBOARD, model.value().get(0).componentType()); + Assertions.assertEquals("xigdhxiidlope", model.value().get(0).configurations().get(0).propertyName()); + Assertions.assertEquals("wdpyq", model.value().get(0).configurations().get(0).value()); + Assertions.assertEquals("ubmdnafcbqwre", model.value().get(0).serviceBinds().get(0).name()); + Assertions.assertEquals("ela", model.value().get(0).serviceBinds().get(0).serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DotNetComponentsCollection model = new DotNetComponentsCollection() + .withValue(Arrays.asList(new DotNetComponentInner().withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("xigdhxiidlope").withValue("wdpyq"))) + .withServiceBinds( + Arrays.asList(new DotNetComponentServiceBind().withName("ubmdnafcbqwre").withServiceId("ela"), + new DotNetComponentServiceBind().withName("cigeleohdbvqvw").withServiceId("jopwbeonrlkwz"))))); + model = BinaryData.fromObject(model).toObject(DotNetComponentsCollection.class); + Assertions.assertEquals(DotNetComponentType.ASPIRE_DASHBOARD, model.value().get(0).componentType()); + Assertions.assertEquals("xigdhxiidlope", model.value().get(0).configurations().get(0).propertyName()); + Assertions.assertEquals("wdpyq", model.value().get(0).configurations().get(0).value()); + Assertions.assertEquals("ubmdnafcbqwre", model.value().get(0).serviceBinds().get(0).name()); + Assertions.assertEquals("ela", model.value().get(0).serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateMockTests.java new file mode 100644 index 0000000000000..eaca56b8abc30 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsCreateOrUpdateMockTests.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DotNetComponent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DotNetComponentsCreateOrUpdateMockTests { + @Test + public void testCreateOrUpdate() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"componentType\":\"AspireDashboard\",\"provisioningState\":\"Succeeded\",\"configurations\":[{\"propertyName\":\"qbbewfcuqfpyyxm\",\"value\":\"mtmvwituwey\"}],\"serviceBinds\":[{\"name\":\"cybwfuppoxprcm\",\"serviceId\":\"uujxdii\"},{\"name\":\"oxrezsvavlrxik\",\"serviceId\":\"oywlunpipcwyb\"},{\"name\":\"fncn\",\"serviceId\":\"tpf\"}]},\"id\":\"aebwfwhxorpwa\",\"name\":\"tzwugexojfccy\",\"type\":\"h\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DotNetComponent response = manager.dotNetComponents().define("bibiwks") + .withExistingManagedEnvironment("ajqzj", "alec").withComponentType(DotNetComponentType.ASPIRE_DASHBOARD) + .withConfigurations(Arrays.asList( + new DotNetComponentConfigurationProperty().withPropertyName("zvoikvntwczf").withValue("ushlcxpblalh"), + new DotNetComponentConfigurationProperty().withPropertyName("zpfkis").withValue("idqzsaaoqdsgpto"), + new DotNetComponentConfigurationProperty().withPropertyName("jq").withValue("afcnlrtb"), + new DotNetComponentConfigurationProperty().withPropertyName("jzzca").withValue("jolb"))) + .withServiceBinds( + Arrays.asList(new DotNetComponentServiceBind().withName("twieopexelxqdwr").withServiceId("yil"), + new DotNetComponentServiceBind().withName("ibkgxyxyau").withServiceId("eddobmcnltm"), + new DotNetComponentServiceBind().withName("tkujsqycm").withServiceId("xfabl"), + new DotNetComponentServiceBind().withName("pwbmwhria").withServiceId("wrycgnwpl"))) + .create(); + + Assertions.assertEquals(DotNetComponentType.ASPIRE_DASHBOARD, response.componentType()); + Assertions.assertEquals("qbbewfcuqfpyyxm", response.configurations().get(0).propertyName()); + Assertions.assertEquals("mtmvwituwey", response.configurations().get(0).value()); + Assertions.assertEquals("cybwfuppoxprcm", response.serviceBinds().get(0).name()); + Assertions.assertEquals("uujxdii", response.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetWithResponseMockTests.java new file mode 100644 index 0000000000000..61d5b7692e8aa --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsGetWithResponseMockTests.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DotNetComponent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DotNetComponentsGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"componentType\":\"AspireResourceServerApi\",\"provisioningState\":\"Deleting\",\"configurations\":[{\"propertyName\":\"oxinunjlzkdr\",\"value\":\"qsxytqqtcmiw\"},{\"propertyName\":\"is\",\"value\":\"mey\"}],\"serviceBinds\":[{\"name\":\"m\",\"serviceId\":\"jyh\"},{\"name\":\"p\",\"serviceId\":\"rryklleynqan\"}]},\"id\":\"igglclwalhvub\",\"name\":\"gzphetxd\",\"type\":\"cmyc\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DotNetComponent response = manager.dotNetComponents() + .getWithResponse("zlhhfix", "cfculzj", "mhpfywvy", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals(DotNetComponentType.ASPIRE_RESOURCE_SERVER_API, response.componentType()); + Assertions.assertEquals("oxinunjlzkdr", response.configurations().get(0).propertyName()); + Assertions.assertEquals("qsxytqqtcmiw", response.configurations().get(0).value()); + Assertions.assertEquals("m", response.serviceBinds().get(0).name()); + Assertions.assertEquals("jyh", response.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListMockTests.java new file mode 100644 index 0000000000000..63366437f67a4 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/DotNetComponentsListMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DotNetComponent; +import com.azure.resourcemanager.appcontainers.models.DotNetComponentType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class DotNetComponentsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"componentType\":\"AspireDashboard\",\"provisioningState\":\"Canceled\",\"configurations\":[{\"propertyName\":\"ixyqhfnkvy\",\"value\":\"qqdseipnquwzxhrp\"},{\"propertyName\":\"odlhkfktl\",\"value\":\"dsobjopnouhbq\"},{\"propertyName\":\"kqxs\",\"value\":\"uzyigfcvcewbwqhd\"}],\"serviceBinds\":[{\"name\":\"atm\",\"serviceId\":\"c\"},{\"name\":\"zdfsqxhyqmr\",\"serviceId\":\"parn\"}]},\"id\":\"grszrbwtdrcwg\",\"name\":\"w\",\"type\":\"l\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.dotNetComponents().list("ceov", "gzwhsxyrujmtik", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(DotNetComponentType.ASPIRE_DASHBOARD, response.iterator().next().componentType()); + Assertions.assertEquals("ixyqhfnkvy", response.iterator().next().configurations().get(0).propertyName()); + Assertions.assertEquals("qqdseipnquwzxhrp", response.iterator().next().configurations().get(0).value()); + Assertions.assertEquals("atm", response.iterator().next().serviceBinds().get(0).name()); + Assertions.assertEquals("c", response.iterator().next().serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/EnvironmentVariableTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/EnvironmentVariableTests.java new file mode 100644 index 0000000000000..e1434bf5a7d49 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/EnvironmentVariableTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.EnvironmentVariable; +import org.junit.jupiter.api.Assertions; + +public final class EnvironmentVariableTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + EnvironmentVariable model = BinaryData.fromString("{\"name\":\"szdtmhrkwof\",\"value\":\"yvoqa\"}") + .toObject(EnvironmentVariable.class); + Assertions.assertEquals("szdtmhrkwof", model.name()); + Assertions.assertEquals("yvoqa", model.value()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + EnvironmentVariable model = new EnvironmentVariable().withName("szdtmhrkwof").withValue("yvoqa"); + model = BinaryData.fromObject(model).toObject(EnvironmentVariable.class); + Assertions.assertEquals("szdtmhrkwof", model.name()); + Assertions.assertEquals("yvoqa", model.value()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ExtendedLocationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ExtendedLocationTests.java index af1e855911c3c..2ef2ffb9678a8 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ExtendedLocationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ExtendedLocationTests.java @@ -12,17 +12,18 @@ public final class ExtendedLocationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExtendedLocation model = - BinaryData.fromString("{\"name\":\"zf\",\"type\":\"CustomLocation\"}").toObject(ExtendedLocation.class); - Assertions.assertEquals("zf", model.name()); + ExtendedLocation model = BinaryData.fromString("{\"name\":\"gsyocogj\",\"type\":\"CustomLocation\"}") + .toObject(ExtendedLocation.class); + Assertions.assertEquals("gsyocogj", model.name()); Assertions.assertEquals(ExtendedLocationTypes.CUSTOM_LOCATION, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExtendedLocation model = new ExtendedLocation().withName("zf").withType(ExtendedLocationTypes.CUSTOM_LOCATION); + ExtendedLocation model + = new ExtendedLocation().withName("gsyocogj").withType(ExtendedLocationTypes.CUSTOM_LOCATION); model = BinaryData.fromObject(model).toObject(ExtendedLocation.class); - Assertions.assertEquals("zf", model.name()); + Assertions.assertEquals("gsyocogj", model.name()); Assertions.assertEquals(ExtendedLocationTypes.CUSTOM_LOCATION, model.type()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ForwardProxyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ForwardProxyTests.java index 0574ddd04012a..f181e726332bd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ForwardProxyTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ForwardProxyTests.java @@ -12,26 +12,21 @@ public final class ForwardProxyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ForwardProxy model = - BinaryData - .fromString( - "{\"convention\":\"NoProxy\",\"customHostHeaderName\":\"chcbonqvpkvlrxnj\",\"customProtoHeaderName\":\"seiphe\"}") - .toObject(ForwardProxy.class); - Assertions.assertEquals(ForwardProxyConvention.NO_PROXY, model.convention()); - Assertions.assertEquals("chcbonqvpkvlrxnj", model.customHostHeaderName()); - Assertions.assertEquals("seiphe", model.customProtoHeaderName()); + ForwardProxy model = BinaryData.fromString( + "{\"convention\":\"Custom\",\"customHostHeaderName\":\"yfsoppu\",\"customProtoHeaderName\":\"esnzwde\"}") + .toObject(ForwardProxy.class); + Assertions.assertEquals(ForwardProxyConvention.CUSTOM, model.convention()); + Assertions.assertEquals("yfsoppu", model.customHostHeaderName()); + Assertions.assertEquals("esnzwde", model.customProtoHeaderName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ForwardProxy model = - new ForwardProxy() - .withConvention(ForwardProxyConvention.NO_PROXY) - .withCustomHostHeaderName("chcbonqvpkvlrxnj") - .withCustomProtoHeaderName("seiphe"); + ForwardProxy model = new ForwardProxy().withConvention(ForwardProxyConvention.CUSTOM) + .withCustomHostHeaderName("yfsoppu").withCustomProtoHeaderName("esnzwde"); model = BinaryData.fromObject(model).toObject(ForwardProxy.class); - Assertions.assertEquals(ForwardProxyConvention.NO_PROXY, model.convention()); - Assertions.assertEquals("chcbonqvpkvlrxnj", model.customHostHeaderName()); - Assertions.assertEquals("seiphe", model.customProtoHeaderName()); + Assertions.assertEquals(ForwardProxyConvention.CUSTOM, model.convention()); + Assertions.assertEquals("yfsoppu", model.customHostHeaderName()); + Assertions.assertEquals("esnzwde", model.customProtoHeaderName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/GlobalValidationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/GlobalValidationTests.java index 691e954f7a667..110f45d9daeeb 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/GlobalValidationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/GlobalValidationTests.java @@ -13,26 +13,24 @@ public final class GlobalValidationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GlobalValidation model = - BinaryData - .fromString( - "{\"unauthenticatedClientAction\":\"Return401\",\"redirectToProvider\":\"yhxhu\",\"excludedPaths\":[\"ftyxolniw\"]}") - .toObject(GlobalValidation.class); - Assertions.assertEquals(UnauthenticatedClientActionV2.RETURN401, model.unauthenticatedClientAction()); - Assertions.assertEquals("yhxhu", model.redirectToProvider()); - Assertions.assertEquals("ftyxolniw", model.excludedPaths().get(0)); + GlobalValidation model = BinaryData.fromString( + "{\"unauthenticatedClientAction\":\"RedirectToLoginPage\",\"redirectToProvider\":\"rkujy\",\"excludedPaths\":[\"eju\"]}") + .toObject(GlobalValidation.class); + Assertions.assertEquals(UnauthenticatedClientActionV2.REDIRECT_TO_LOGIN_PAGE, + model.unauthenticatedClientAction()); + Assertions.assertEquals("rkujy", model.redirectToProvider()); + Assertions.assertEquals("eju", model.excludedPaths().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GlobalValidation model = - new GlobalValidation() - .withUnauthenticatedClientAction(UnauthenticatedClientActionV2.RETURN401) - .withRedirectToProvider("yhxhu") - .withExcludedPaths(Arrays.asList("ftyxolniw")); + GlobalValidation model = new GlobalValidation() + .withUnauthenticatedClientAction(UnauthenticatedClientActionV2.REDIRECT_TO_LOGIN_PAGE) + .withRedirectToProvider("rkujy").withExcludedPaths(Arrays.asList("eju")); model = BinaryData.fromObject(model).toObject(GlobalValidation.class); - Assertions.assertEquals(UnauthenticatedClientActionV2.RETURN401, model.unauthenticatedClientAction()); - Assertions.assertEquals("yhxhu", model.redirectToProvider()); - Assertions.assertEquals("ftyxolniw", model.excludedPaths().get(0)); + Assertions.assertEquals(UnauthenticatedClientActionV2.REDIRECT_TO_LOGIN_PAGE, + model.unauthenticatedClientAction()); + Assertions.assertEquals("rkujy", model.redirectToProvider()); + Assertions.assertEquals("eju", model.excludedPaths().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchMatchTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchMatchTests.java new file mode 100644 index 0000000000000..515a6c0da5289 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchMatchTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.HeaderMatchMatch; +import org.junit.jupiter.api.Assertions; + +public final class HeaderMatchMatchTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HeaderMatchMatch model = BinaryData.fromString( + "{\"exactMatch\":\"wjzrnfygxgisp\",\"prefixMatch\":\"vtz\",\"suffixMatch\":\"ufubl\",\"regexMatch\":\"fxqeof\"}") + .toObject(HeaderMatchMatch.class); + Assertions.assertEquals("wjzrnfygxgisp", model.exactMatch()); + Assertions.assertEquals("vtz", model.prefixMatch()); + Assertions.assertEquals("ufubl", model.suffixMatch()); + Assertions.assertEquals("fxqeof", model.regexMatch()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HeaderMatchMatch model = new HeaderMatchMatch().withExactMatch("wjzrnfygxgisp").withPrefixMatch("vtz") + .withSuffixMatch("ufubl").withRegexMatch("fxqeof"); + model = BinaryData.fromObject(model).toObject(HeaderMatchMatch.class); + Assertions.assertEquals("wjzrnfygxgisp", model.exactMatch()); + Assertions.assertEquals("vtz", model.prefixMatch()); + Assertions.assertEquals("ufubl", model.suffixMatch()); + Assertions.assertEquals("fxqeof", model.regexMatch()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchTests.java new file mode 100644 index 0000000000000..4e1a115aee8d2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HeaderMatchTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.HeaderMatch; +import org.junit.jupiter.api.Assertions; + +public final class HeaderMatchTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HeaderMatch model = BinaryData.fromString( + "{\"header\":\"uybbkpodep\",\"match\":{\"exactMatch\":\"inuvamiheogn\",\"prefixMatch\":\"xzxtheo\",\"suffixMatch\":\"si\",\"regexMatch\":\"evcciqihnhun\"}}") + .toObject(HeaderMatch.class); + Assertions.assertEquals("uybbkpodep", model.headerProperty()); + Assertions.assertEquals("inuvamiheogn", model.exactMatch()); + Assertions.assertEquals("xzxtheo", model.prefixMatch()); + Assertions.assertEquals("si", model.suffixMatch()); + Assertions.assertEquals("evcciqihnhun", model.regexMatch()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HeaderMatch model = new HeaderMatch().withHeaderProperty("uybbkpodep").withExactMatch("inuvamiheogn") + .withPrefixMatch("xzxtheo").withSuffixMatch("si").withRegexMatch("evcciqihnhun"); + model = BinaryData.fromObject(model).toObject(HeaderMatch.class); + Assertions.assertEquals("uybbkpodep", model.headerProperty()); + Assertions.assertEquals("inuvamiheogn", model.exactMatch()); + Assertions.assertEquals("xzxtheo", model.prefixMatch()); + Assertions.assertEquals("si", model.suffixMatch()); + Assertions.assertEquals("evcciqihnhun", model.regexMatch()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpConnectionPoolTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpConnectionPoolTests.java new file mode 100644 index 0000000000000..5662a4ebc19ca --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpConnectionPoolTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.HttpConnectionPool; +import org.junit.jupiter.api.Assertions; + +public final class HttpConnectionPoolTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HttpConnectionPool model + = BinaryData.fromString("{\"http1MaxPendingRequests\":5903092,\"http2MaxRequests\":1154409847}") + .toObject(HttpConnectionPool.class); + Assertions.assertEquals(5903092, model.http1MaxPendingRequests()); + Assertions.assertEquals(1154409847, model.http2MaxRequests()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HttpConnectionPool model + = new HttpConnectionPool().withHttp1MaxPendingRequests(5903092).withHttp2MaxRequests(1154409847); + model = BinaryData.fromObject(model).toObject(HttpConnectionPool.class); + Assertions.assertEquals(5903092, model.http1MaxPendingRequests()); + Assertions.assertEquals(1154409847, model.http2MaxRequests()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpGetTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpGetTests.java new file mode 100644 index 0000000000000..a6cf972503e00 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpGetTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.HttpGet; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class HttpGetTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HttpGet model = BinaryData + .fromString( + "{\"url\":\"auvfbtkuwhhmhyk\",\"fileName\":\"oxafn\",\"headers\":[\"pichkoymkcdy\",\"bpkkpwdre\"]}") + .toObject(HttpGet.class); + Assertions.assertEquals("auvfbtkuwhhmhyk", model.url()); + Assertions.assertEquals("oxafn", model.fileName()); + Assertions.assertEquals("pichkoymkcdy", model.headers().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HttpGet model = new HttpGet().withUrl("auvfbtkuwhhmhyk").withFileName("oxafn") + .withHeaders(Arrays.asList("pichkoymkcdy", "bpkkpwdre")); + model = BinaryData.fromObject(model).toObject(HttpGet.class); + Assertions.assertEquals("auvfbtkuwhhmhyk", model.url()); + Assertions.assertEquals("oxafn", model.fileName()); + Assertions.assertEquals("pichkoymkcdy", model.headers().get(0)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpRetryPolicyRetryBackOffTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpRetryPolicyRetryBackOffTests.java new file mode 100644 index 0000000000000..42af511c7a7a7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpRetryPolicyRetryBackOffTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.HttpRetryPolicyRetryBackOff; +import org.junit.jupiter.api.Assertions; + +public final class HttpRetryPolicyRetryBackOffTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HttpRetryPolicyRetryBackOff model = BinaryData.fromString( + "{\"initialDelayInMilliseconds\":2232295730634911769,\"maxIntervalInMilliseconds\":8368000594620652423}") + .toObject(HttpRetryPolicyRetryBackOff.class); + Assertions.assertEquals(2232295730634911769L, model.initialDelayInMilliseconds()); + Assertions.assertEquals(8368000594620652423L, model.maxIntervalInMilliseconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HttpRetryPolicyRetryBackOff model = new HttpRetryPolicyRetryBackOff() + .withInitialDelayInMilliseconds(2232295730634911769L).withMaxIntervalInMilliseconds(8368000594620652423L); + model = BinaryData.fromObject(model).toObject(HttpRetryPolicyRetryBackOff.class); + Assertions.assertEquals(2232295730634911769L, model.initialDelayInMilliseconds()); + Assertions.assertEquals(8368000594620652423L, model.maxIntervalInMilliseconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsRoutesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsRoutesTests.java index 2760f3932dc15..f55fbf7cff924 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsRoutesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsRoutesTests.java @@ -11,15 +11,15 @@ public final class HttpSettingsRoutesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - HttpSettingsRoutes model = - BinaryData.fromString("{\"apiPrefix\":\"nzscxa\"}").toObject(HttpSettingsRoutes.class); - Assertions.assertEquals("nzscxa", model.apiPrefix()); + HttpSettingsRoutes model + = BinaryData.fromString("{\"apiPrefix\":\"gacftadeh\"}").toObject(HttpSettingsRoutes.class); + Assertions.assertEquals("gacftadeh", model.apiPrefix()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpSettingsRoutes model = new HttpSettingsRoutes().withApiPrefix("nzscxa"); + HttpSettingsRoutes model = new HttpSettingsRoutes().withApiPrefix("gacftadeh"); model = BinaryData.fromObject(model).toObject(HttpSettingsRoutes.class); - Assertions.assertEquals("nzscxa", model.apiPrefix()); + Assertions.assertEquals("gacftadeh", model.apiPrefix()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsTests.java index 57936f392b446..7d131968d9379 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/HttpSettingsTests.java @@ -14,34 +14,27 @@ public final class HttpSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - HttpSettings model = - BinaryData - .fromString( - "{\"requireHttps\":false,\"routes\":{\"apiPrefix\":\"onlebxetqgtzxdpn\"},\"forwardProxy\":{\"convention\":\"NoProxy\",\"customHostHeaderName\":\"xrjfeallnwsub\",\"customProtoHeaderName\":\"njampm\"}}") - .toObject(HttpSettings.class); + HttpSettings model = BinaryData.fromString( + "{\"requireHttps\":false,\"routes\":{\"apiPrefix\":\"jzzvdud\"},\"forwardProxy\":{\"convention\":\"NoProxy\",\"customHostHeaderName\":\"fhotw\",\"customProtoHeaderName\":\"ynpwlbj\"}}") + .toObject(HttpSettings.class); Assertions.assertEquals(false, model.requireHttps()); - Assertions.assertEquals("onlebxetqgtzxdpn", model.routes().apiPrefix()); + Assertions.assertEquals("jzzvdud", model.routes().apiPrefix()); Assertions.assertEquals(ForwardProxyConvention.NO_PROXY, model.forwardProxy().convention()); - Assertions.assertEquals("xrjfeallnwsub", model.forwardProxy().customHostHeaderName()); - Assertions.assertEquals("njampm", model.forwardProxy().customProtoHeaderName()); + Assertions.assertEquals("fhotw", model.forwardProxy().customHostHeaderName()); + Assertions.assertEquals("ynpwlbj", model.forwardProxy().customProtoHeaderName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpSettings model = - new HttpSettings() - .withRequireHttps(false) - .withRoutes(new HttpSettingsRoutes().withApiPrefix("onlebxetqgtzxdpn")) - .withForwardProxy( - new ForwardProxy() - .withConvention(ForwardProxyConvention.NO_PROXY) - .withCustomHostHeaderName("xrjfeallnwsub") - .withCustomProtoHeaderName("njampm")); + HttpSettings model + = new HttpSettings().withRequireHttps(false).withRoutes(new HttpSettingsRoutes().withApiPrefix("jzzvdud")) + .withForwardProxy(new ForwardProxy().withConvention(ForwardProxyConvention.NO_PROXY) + .withCustomHostHeaderName("fhotw").withCustomProtoHeaderName("ynpwlbj")); model = BinaryData.fromObject(model).toObject(HttpSettings.class); Assertions.assertEquals(false, model.requireHttps()); - Assertions.assertEquals("onlebxetqgtzxdpn", model.routes().apiPrefix()); + Assertions.assertEquals("jzzvdud", model.routes().apiPrefix()); Assertions.assertEquals(ForwardProxyConvention.NO_PROXY, model.forwardProxy().convention()); - Assertions.assertEquals("xrjfeallnwsub", model.forwardProxy().customHostHeaderName()); - Assertions.assertEquals("njampm", model.forwardProxy().customProtoHeaderName()); + Assertions.assertEquals("fhotw", model.forwardProxy().customHostHeaderName()); + Assertions.assertEquals("ynpwlbj", model.forwardProxy().customProtoHeaderName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressPortMappingTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressPortMappingTests.java new file mode 100644 index 0000000000000..71de6b53f3872 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressPortMappingTests.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.IngressPortMapping; +import org.junit.jupiter.api.Assertions; + +public final class IngressPortMappingTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + IngressPortMapping model + = BinaryData.fromString("{\"external\":true,\"targetPort\":2060980520,\"exposedPort\":200260047}") + .toObject(IngressPortMapping.class); + Assertions.assertEquals(true, model.external()); + Assertions.assertEquals(2060980520, model.targetPort()); + Assertions.assertEquals(200260047, model.exposedPort()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + IngressPortMapping model + = new IngressPortMapping().withExternal(true).withTargetPort(2060980520).withExposedPort(200260047); + model = BinaryData.fromObject(model).toObject(IngressPortMapping.class); + Assertions.assertEquals(true, model.external()); + Assertions.assertEquals(2060980520, model.targetPort()); + Assertions.assertEquals(200260047, model.exposedPort()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressStickySessionsTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressStickySessionsTests.java index 7bd342097aa1a..f5793c8ace6bd 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressStickySessionsTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IngressStickySessionsTests.java @@ -12,15 +12,15 @@ public final class IngressStickySessionsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - IngressStickySessions model = - BinaryData.fromString("{\"affinity\":\"sticky\"}").toObject(IngressStickySessions.class); - Assertions.assertEquals(Affinity.STICKY, model.affinity()); + IngressStickySessions model + = BinaryData.fromString("{\"affinity\":\"none\"}").toObject(IngressStickySessions.class); + Assertions.assertEquals(Affinity.NONE, model.affinity()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IngressStickySessions model = new IngressStickySessions().withAffinity(Affinity.STICKY); + IngressStickySessions model = new IngressStickySessions().withAffinity(Affinity.NONE); model = BinaryData.fromObject(model).toObject(IngressStickySessions.class); - Assertions.assertEquals(Affinity.STICKY, model.affinity()); + Assertions.assertEquals(Affinity.NONE, model.affinity()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IpSecurityRestrictionRuleTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IpSecurityRestrictionRuleTests.java index 0763fa532f194..468f27b140ff5 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IpSecurityRestrictionRuleTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/IpSecurityRestrictionRuleTests.java @@ -12,29 +12,23 @@ public final class IpSecurityRestrictionRuleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - IpSecurityRestrictionRule model = - BinaryData - .fromString( - "{\"name\":\"eqx\",\"description\":\"z\",\"ipAddressRange\":\"ahzxctobgbk\",\"action\":\"Allow\"}") - .toObject(IpSecurityRestrictionRule.class); - Assertions.assertEquals("eqx", model.name()); - Assertions.assertEquals("z", model.description()); - Assertions.assertEquals("ahzxctobgbk", model.ipAddressRange()); - Assertions.assertEquals(Action.ALLOW, model.action()); + IpSecurityRestrictionRule model = BinaryData.fromString( + "{\"name\":\"lfzxiavrmbzonoki\",\"description\":\"jq\",\"ipAddressRange\":\"irgzp\",\"action\":\"Deny\"}") + .toObject(IpSecurityRestrictionRule.class); + Assertions.assertEquals("lfzxiavrmbzonoki", model.name()); + Assertions.assertEquals("jq", model.description()); + Assertions.assertEquals("irgzp", model.ipAddressRange()); + Assertions.assertEquals(Action.DENY, model.action()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IpSecurityRestrictionRule model = - new IpSecurityRestrictionRule() - .withName("eqx") - .withDescription("z") - .withIpAddressRange("ahzxctobgbk") - .withAction(Action.ALLOW); + IpSecurityRestrictionRule model = new IpSecurityRestrictionRule().withName("lfzxiavrmbzonoki") + .withDescription("jq").withIpAddressRange("irgzp").withAction(Action.DENY); model = BinaryData.fromObject(model).toObject(IpSecurityRestrictionRule.class); - Assertions.assertEquals("eqx", model.name()); - Assertions.assertEquals("z", model.description()); - Assertions.assertEquals("ahzxctobgbk", model.ipAddressRange()); - Assertions.assertEquals(Action.ALLOW, model.action()); + Assertions.assertEquals("lfzxiavrmbzonoki", model.name()); + Assertions.assertEquals("jq", model.description()); + Assertions.assertEquals("irgzp", model.ipAddressRange()); + Assertions.assertEquals(Action.DENY, model.action()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentConfigurationPropertyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentConfigurationPropertyTests.java new file mode 100644 index 0000000000000..08a4978c304c7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentConfigurationPropertyTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import org.junit.jupiter.api.Assertions; + +public final class JavaComponentConfigurationPropertyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + JavaComponentConfigurationProperty model + = BinaryData.fromString("{\"propertyName\":\"ltn\",\"value\":\"dhqoawj\"}") + .toObject(JavaComponentConfigurationProperty.class); + Assertions.assertEquals("ltn", model.propertyName()); + Assertions.assertEquals("dhqoawj", model.value()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + JavaComponentConfigurationProperty model + = new JavaComponentConfigurationProperty().withPropertyName("ltn").withValue("dhqoawj"); + model = BinaryData.fromObject(model).toObject(JavaComponentConfigurationProperty.class); + Assertions.assertEquals("ltn", model.propertyName()); + Assertions.assertEquals("dhqoawj", model.value()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentInnerTests.java new file mode 100644 index 0000000000000..3a1c3198ceca2 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentInnerTests.java @@ -0,0 +1,44 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class JavaComponentInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + JavaComponentInner model = BinaryData.fromString( + "{\"properties\":{\"componentType\":\"SpringBootAdmin\",\"provisioningState\":\"Canceled\",\"configurations\":[{\"propertyName\":\"p\",\"value\":\"kdqzrdzsylo\"}],\"serviceBinds\":[{\"name\":\"rc\",\"serviceId\":\"ydmxzj\"},{\"name\":\"p\",\"serviceId\":\"aurkihcirlde\"},{\"name\":\"rdcoxnbkkj\",\"serviceId\":\"urnnqbnqbpiz\"},{\"name\":\"ltgrdogypxrxv\",\"serviceId\":\"ihwuhvctafsrbxrb\"}]},\"id\":\"li\",\"name\":\"wxihs\",\"type\":\"nxw\"}") + .toObject(JavaComponentInner.class); + Assertions.assertEquals(JavaComponentType.SPRING_BOOT_ADMIN, model.componentType()); + Assertions.assertEquals("p", model.configurations().get(0).propertyName()); + Assertions.assertEquals("kdqzrdzsylo", model.configurations().get(0).value()); + Assertions.assertEquals("rc", model.serviceBinds().get(0).name()); + Assertions.assertEquals("ydmxzj", model.serviceBinds().get(0).serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + JavaComponentInner model = new JavaComponentInner().withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations( + Arrays.asList(new JavaComponentConfigurationProperty().withPropertyName("p").withValue("kdqzrdzsylo"))) + .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("rc").withServiceId("ydmxzj"), + new JavaComponentServiceBind().withName("p").withServiceId("aurkihcirlde"), + new JavaComponentServiceBind().withName("rdcoxnbkkj").withServiceId("urnnqbnqbpiz"), + new JavaComponentServiceBind().withName("ltgrdogypxrxv").withServiceId("ihwuhvctafsrbxrb"))); + model = BinaryData.fromObject(model).toObject(JavaComponentInner.class); + Assertions.assertEquals(JavaComponentType.SPRING_BOOT_ADMIN, model.componentType()); + Assertions.assertEquals("p", model.configurations().get(0).propertyName()); + Assertions.assertEquals("kdqzrdzsylo", model.configurations().get(0).value()); + Assertions.assertEquals("rc", model.serviceBinds().get(0).name()); + Assertions.assertEquals("ydmxzj", model.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentPropertiesTests.java new file mode 100644 index 0000000000000..a72e3d59057fb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentPropertiesTests.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentProperties; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class JavaComponentPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + JavaComponentProperties model = BinaryData.fromString( + "{\"componentType\":\"SpringCloudConfig\",\"provisioningState\":\"Succeeded\",\"configurations\":[{\"propertyName\":\"aklsbsbq\",\"value\":\"agwwrxaom\"}],\"serviceBinds\":[{\"name\":\"lrrcz\",\"serviceId\":\"k\"}]}") + .toObject(JavaComponentProperties.class); + Assertions.assertEquals(JavaComponentType.SPRING_CLOUD_CONFIG, model.componentType()); + Assertions.assertEquals("aklsbsbq", model.configurations().get(0).propertyName()); + Assertions.assertEquals("agwwrxaom", model.configurations().get(0).value()); + Assertions.assertEquals("lrrcz", model.serviceBinds().get(0).name()); + Assertions.assertEquals("k", model.serviceBinds().get(0).serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + JavaComponentProperties model + = new JavaComponentProperties().withComponentType(JavaComponentType.SPRING_CLOUD_CONFIG) + .withConfigurations(Arrays.asList( + new JavaComponentConfigurationProperty().withPropertyName("aklsbsbq").withValue("agwwrxaom"))) + .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("lrrcz").withServiceId("k"))); + model = BinaryData.fromObject(model).toObject(JavaComponentProperties.class); + Assertions.assertEquals(JavaComponentType.SPRING_CLOUD_CONFIG, model.componentType()); + Assertions.assertEquals("aklsbsbq", model.configurations().get(0).propertyName()); + Assertions.assertEquals("agwwrxaom", model.configurations().get(0).value()); + Assertions.assertEquals("lrrcz", model.serviceBinds().get(0).name()); + Assertions.assertEquals("k", model.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentServiceBindTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentServiceBindTests.java new file mode 100644 index 0000000000000..218d76e5d6958 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentServiceBindTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import org.junit.jupiter.api.Assertions; + +public final class JavaComponentServiceBindTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + JavaComponentServiceBind model = BinaryData.fromString("{\"name\":\"yueayfbpcmsp\",\"serviceId\":\"yrrueqth\"}") + .toObject(JavaComponentServiceBind.class); + Assertions.assertEquals("yueayfbpcmsp", model.name()); + Assertions.assertEquals("yrrueqth", model.serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + JavaComponentServiceBind model + = new JavaComponentServiceBind().withName("yueayfbpcmsp").withServiceId("yrrueqth"); + model = BinaryData.fromObject(model).toObject(JavaComponentServiceBind.class); + Assertions.assertEquals("yueayfbpcmsp", model.name()); + Assertions.assertEquals("yrrueqth", model.serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCollectionTests.java new file mode 100644 index 0000000000000..3b76ca16ca1bb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCollectionTests.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.JavaComponentInner; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import com.azure.resourcemanager.appcontainers.models.JavaComponentsCollection; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class JavaComponentsCollectionTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + JavaComponentsCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"componentType\":\"SpringBootAdmin\",\"provisioningState\":\"Failed\",\"configurations\":[{\"propertyName\":\"jwuive\",\"value\":\"cgyee\"}],\"serviceBinds\":[{\"name\":\"qbpsmg\",\"serviceId\":\"guamlj\"}]},\"id\":\"rgmsplzga\",\"name\":\"fcshh\",\"type\":\"new\"},{\"properties\":{\"componentType\":\"SpringCloudConfig\",\"provisioningState\":\"Succeeded\",\"configurations\":[{\"propertyName\":\"nxrjkixtwbtaoy\",\"value\":\"yghs\"}],\"serviceBinds\":[{\"name\":\"lhkgmnsghp\",\"serviceId\":\"cphdrwjjkhvyo\"}]},\"id\":\"c\",\"name\":\"u\",\"type\":\"vxnqmhrpqpd\"}],\"nextLink\":\"mkoisqcssf\"}") + .toObject(JavaComponentsCollection.class); + Assertions.assertEquals(JavaComponentType.SPRING_BOOT_ADMIN, model.value().get(0).componentType()); + Assertions.assertEquals("jwuive", model.value().get(0).configurations().get(0).propertyName()); + Assertions.assertEquals("cgyee", model.value().get(0).configurations().get(0).value()); + Assertions.assertEquals("qbpsmg", model.value().get(0).serviceBinds().get(0).name()); + Assertions.assertEquals("guamlj", model.value().get(0).serviceBinds().get(0).serviceId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + JavaComponentsCollection model + = new JavaComponentsCollection() + .withValue( + Arrays + .asList( + new JavaComponentInner().withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations(Arrays.asList(new JavaComponentConfigurationProperty() + .withPropertyName("jwuive").withValue("cgyee"))) + .withServiceBinds(Arrays + .asList(new JavaComponentServiceBind().withName("qbpsmg").withServiceId("guamlj"))), + new JavaComponentInner().withComponentType(JavaComponentType.SPRING_CLOUD_CONFIG) + .withConfigurations(Arrays.asList(new JavaComponentConfigurationProperty() + .withPropertyName("nxrjkixtwbtaoy").withValue("yghs"))) + .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("lhkgmnsghp") + .withServiceId("cphdrwjjkhvyo"))))); + model = BinaryData.fromObject(model).toObject(JavaComponentsCollection.class); + Assertions.assertEquals(JavaComponentType.SPRING_BOOT_ADMIN, model.value().get(0).componentType()); + Assertions.assertEquals("jwuive", model.value().get(0).configurations().get(0).propertyName()); + Assertions.assertEquals("cgyee", model.value().get(0).configurations().get(0).value()); + Assertions.assertEquals("qbpsmg", model.value().get(0).serviceBinds().get(0).name()); + Assertions.assertEquals("guamlj", model.value().get(0).serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateMockTests.java new file mode 100644 index 0000000000000..746ef359e6b32 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsCreateOrUpdateMockTests.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.JavaComponent; +import com.azure.resourcemanager.appcontainers.models.JavaComponentConfigurationProperty; +import com.azure.resourcemanager.appcontainers.models.JavaComponentServiceBind; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class JavaComponentsCreateOrUpdateMockTests { + @Test + public void testCreateOrUpdate() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"componentType\":\"SpringCloudEureka\",\"provisioningState\":\"Succeeded\",\"configurations\":[{\"propertyName\":\"wmmkfq\",\"value\":\"iqulwwtrjmeq\"},{\"propertyName\":\"yhzok\",\"value\":\"yuoh\"},{\"propertyName\":\"nsnaajphmp\",\"value\":\"jnglpwsad\"},{\"propertyName\":\"jsumxpezcoio\",\"value\":\"rmfqzwq\"}],\"serviceBinds\":[{\"name\":\"eedcnwmywx\",\"serviceId\":\"zkvemy\"},{\"name\":\"pczaqpqif\",\"serviceId\":\"m\"},{\"name\":\"rwtx\",\"serviceId\":\"isamonatniz\"}]},\"id\":\"r\",\"name\":\"qsqjghrmthsplwst\",\"type\":\"xsrgxfqpan\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + JavaComponent response = manager.javaComponents().define("tpjfojiunrlshxuk") + .withExistingManagedEnvironment("misvpztd", "vykpxkqe") + .withComponentType(JavaComponentType.SPRING_BOOT_ADMIN) + .withConfigurations( + Arrays.asList(new JavaComponentConfigurationProperty().withPropertyName("oancdrco").withValue("vx"), + new JavaComponentConfigurationProperty().withPropertyName("dxonckbn").withValue("lfxlupibaqzizxzp"), + new JavaComponentConfigurationProperty().withPropertyName("eghlw").withValue("ogvgfklqiyn"), + new JavaComponentConfigurationProperty().withPropertyName("eqelsb").withValue("dstrkzxsgtz"))) + .withServiceBinds(Arrays.asList(new JavaComponentServiceBind().withName("rds").withServiceId("vpimyndnoxa"), + new JavaComponentServiceBind().withName("rqaqotn").withServiceId("xolousdv"), + new JavaComponentServiceBind().withName("ptqmawzjdrpiz").withServiceId("l"))) + .create(); + + Assertions.assertEquals(JavaComponentType.SPRING_CLOUD_EUREKA, response.componentType()); + Assertions.assertEquals("wmmkfq", response.configurations().get(0).propertyName()); + Assertions.assertEquals("iqulwwtrjmeq", response.configurations().get(0).value()); + Assertions.assertEquals("eedcnwmywx", response.serviceBinds().get(0).name()); + Assertions.assertEquals("zkvemy", response.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetWithResponseMockTests.java new file mode 100644 index 0000000000000..bf495efd54f72 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsGetWithResponseMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.JavaComponent; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class JavaComponentsGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"componentType\":\"SpringCloudEureka\",\"provisioningState\":\"Canceled\",\"configurations\":[{\"propertyName\":\"sc\",\"value\":\"ydzjemexm\"},{\"propertyName\":\"kvmuwrxlniwmcp\",\"value\":\"rdlhvdvmiphbe\"},{\"propertyName\":\"qjzmhkdcl\",\"value\":\"roczfmuner\"}],\"serviceBinds\":[{\"name\":\"xzs\",\"serviceId\":\"zezbzu\"},{\"name\":\"dlevzskejcgw\",\"serviceId\":\"gqkstyecupyu\"},{\"name\":\"p\",\"serviceId\":\"davsjcfmazpz\"},{\"name\":\"wuzvcmcokxize\",\"serviceId\":\"vfrjwu\"}]},\"id\":\"o\",\"name\":\"zvajbvbnkrdem\",\"type\":\"idackzidgzwdyd\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + JavaComponent response = manager.javaComponents() + .getWithResponse("zqazvb", "arkptgongruatsyi", "sjqhenigbeqngu", com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals(JavaComponentType.SPRING_CLOUD_EUREKA, response.componentType()); + Assertions.assertEquals("sc", response.configurations().get(0).propertyName()); + Assertions.assertEquals("ydzjemexm", response.configurations().get(0).value()); + Assertions.assertEquals("xzs", response.serviceBinds().get(0).name()); + Assertions.assertEquals("zezbzu", response.serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListMockTests.java new file mode 100644 index 0000000000000..b1d025f1f7e87 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JavaComponentsListMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.JavaComponent; +import com.azure.resourcemanager.appcontainers.models.JavaComponentType; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class JavaComponentsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"componentType\":\"SpringCloudConfig\",\"provisioningState\":\"InProgress\",\"configurations\":[{\"propertyName\":\"rshzzbgullcxiqqz\",\"value\":\"oxdu\"},{\"propertyName\":\"amglrouig\",\"value\":\"fivjqterdqqigdy\"}],\"serviceBinds\":[{\"name\":\"p\",\"serviceId\":\"rwqirvtktyhhmvf\"},{\"name\":\"apj\",\"serviceId\":\"odmkrrwepgqv\"},{\"name\":\"kqlujqgira\",\"serviceId\":\"lyvxchp\"}]},\"id\":\"ctsfaeuhwwsknst\",\"name\":\"zuzhasup\",\"type\":\"lppdpgz\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.javaComponents().list("oq", "dedecfiwh", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(JavaComponentType.SPRING_CLOUD_CONFIG, response.iterator().next().componentType()); + Assertions.assertEquals("rshzzbgullcxiqqz", response.iterator().next().configurations().get(0).propertyName()); + Assertions.assertEquals("oxdu", response.iterator().next().configurations().get(0).value()); + Assertions.assertEquals("p", response.iterator().next().serviceBinds().get(0).name()); + Assertions.assertEquals("rwqirvtktyhhmvf", response.iterator().next().serviceBinds().get(0).serviceId()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationManualTriggerConfigTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationManualTriggerConfigTests.java index b01beabffa0f5..d038c4d341ef9 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationManualTriggerConfigTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationManualTriggerConfigTests.java @@ -11,20 +11,19 @@ public final class JobConfigurationManualTriggerConfigTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - JobConfigurationManualTriggerConfig model = - BinaryData - .fromString("{\"replicaCompletionCount\":598099932,\"parallelism\":1189347509}") + JobConfigurationManualTriggerConfig model + = BinaryData.fromString("{\"replicaCompletionCount\":666781891,\"parallelism\":385989453}") .toObject(JobConfigurationManualTriggerConfig.class); - Assertions.assertEquals(598099932, model.replicaCompletionCount()); - Assertions.assertEquals(1189347509, model.parallelism()); + Assertions.assertEquals(666781891, model.replicaCompletionCount()); + Assertions.assertEquals(385989453, model.parallelism()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JobConfigurationManualTriggerConfig model = - new JobConfigurationManualTriggerConfig().withReplicaCompletionCount(598099932).withParallelism(1189347509); + JobConfigurationManualTriggerConfig model = new JobConfigurationManualTriggerConfig() + .withReplicaCompletionCount(666781891).withParallelism(385989453); model = BinaryData.fromObject(model).toObject(JobConfigurationManualTriggerConfig.class); - Assertions.assertEquals(598099932, model.replicaCompletionCount()); - Assertions.assertEquals(1189347509, model.parallelism()); + Assertions.assertEquals(666781891, model.replicaCompletionCount()); + Assertions.assertEquals(385989453, model.parallelism()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationScheduleTriggerConfigTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationScheduleTriggerConfigTests.java index 2f72bfaaa101e..b510e6f962dbe 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationScheduleTriggerConfigTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobConfigurationScheduleTriggerConfigTests.java @@ -11,26 +11,21 @@ public final class JobConfigurationScheduleTriggerConfigTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - JobConfigurationScheduleTriggerConfig model = - BinaryData - .fromString( - "{\"replicaCompletionCount\":540645493,\"cronExpression\":\"liqhzvhx\",\"parallelism\":1549555977}") - .toObject(JobConfigurationScheduleTriggerConfig.class); - Assertions.assertEquals(540645493, model.replicaCompletionCount()); - Assertions.assertEquals("liqhzvhx", model.cronExpression()); - Assertions.assertEquals(1549555977, model.parallelism()); + JobConfigurationScheduleTriggerConfig model = BinaryData + .fromString("{\"replicaCompletionCount\":530591598,\"cronExpression\":\"rds\",\"parallelism\":1544209565}") + .toObject(JobConfigurationScheduleTriggerConfig.class); + Assertions.assertEquals(530591598, model.replicaCompletionCount()); + Assertions.assertEquals("rds", model.cronExpression()); + Assertions.assertEquals(1544209565, model.parallelism()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JobConfigurationScheduleTriggerConfig model = - new JobConfigurationScheduleTriggerConfig() - .withReplicaCompletionCount(540645493) - .withCronExpression("liqhzvhx") - .withParallelism(1549555977); + JobConfigurationScheduleTriggerConfig model = new JobConfigurationScheduleTriggerConfig() + .withReplicaCompletionCount(530591598).withCronExpression("rds").withParallelism(1544209565); model = BinaryData.fromObject(model).toObject(JobConfigurationScheduleTriggerConfig.class); - Assertions.assertEquals(540645493, model.replicaCompletionCount()); - Assertions.assertEquals("liqhzvhx", model.cronExpression()); - Assertions.assertEquals(1549555977, model.parallelism()); + Assertions.assertEquals(530591598, model.replicaCompletionCount()); + Assertions.assertEquals("rds", model.cronExpression()); + Assertions.assertEquals(1544209565, model.parallelism()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionBaseInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionBaseInnerTests.java index 1d1d65256ecc5..a21e9edc97749 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionBaseInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionBaseInnerTests.java @@ -11,17 +11,17 @@ public final class JobExecutionBaseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - JobExecutionBaseInner model = - BinaryData.fromString("{\"name\":\"pgfewetwlyx\",\"id\":\"cxy\"}").toObject(JobExecutionBaseInner.class); - Assertions.assertEquals("pgfewetwlyx", model.name()); - Assertions.assertEquals("cxy", model.id()); + JobExecutionBaseInner model + = BinaryData.fromString("{\"name\":\"olvdnd\",\"id\":\"auo\"}").toObject(JobExecutionBaseInner.class); + Assertions.assertEquals("olvdnd", model.name()); + Assertions.assertEquals("auo", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JobExecutionBaseInner model = new JobExecutionBaseInner().withName("pgfewetwlyx").withId("cxy"); + JobExecutionBaseInner model = new JobExecutionBaseInner().withName("olvdnd").withId("auo"); model = BinaryData.fromObject(model).toObject(JobExecutionBaseInner.class); - Assertions.assertEquals("pgfewetwlyx", model.name()); - Assertions.assertEquals("cxy", model.id()); + Assertions.assertEquals("olvdnd", model.name()); + Assertions.assertEquals("auo", model.id()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionInnerTests.java new file mode 100644 index 0000000000000..6dd1b13a2971b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobExecutionInnerTests.java @@ -0,0 +1,92 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.JobExecutionInner; +import com.azure.resourcemanager.appcontainers.models.ContainerResources; +import com.azure.resourcemanager.appcontainers.models.EnvironmentVar; +import com.azure.resourcemanager.appcontainers.models.JobExecutionContainer; +import com.azure.resourcemanager.appcontainers.models.JobExecutionTemplate; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class JobExecutionInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + JobExecutionInner model = BinaryData.fromString( + "{\"name\":\"herngb\",\"id\":\"juahokqto\",\"type\":\"auxofshfph\",\"properties\":{\"status\":\"Unknown\",\"startTime\":\"2021-02-08T14:31:19Z\",\"endTime\":\"2021-11-07T16:30:06Z\",\"template\":{\"containers\":[{\"image\":\"ywhslwkojpllndnp\",\"name\":\"rpqaf\",\"command\":[\"gsnnf\"],\"args\":[\"tefypococtfjgti\"],\"env\":[{},{}],\"resources\":{\"cpu\":43.561702255466116,\"memory\":\"urmlmuo\",\"ephemeralStorage\":\"lbau\"}}],\"initContainers\":[{\"image\":\"ionszonwp\",\"name\":\"ajinnixjawrtmjfj\",\"command\":[\"cxlzhcoxovnekh\",\"nlusfnrd\",\"jxtxrdc\",\"tjvidt\"],\"args\":[\"puslvyj\"],\"env\":[{},{},{},{}],\"resources\":{\"cpu\":39.025050688905914,\"memory\":\"iziesfuughtuq\",\"ephemeralStorage\":\"cjxeygt\"}},{\"image\":\"xu\",\"name\":\"buew\",\"command\":[\"wnjlxu\"],\"args\":[\"wpusxjbaqehg\",\"dohzjq\"],\"env\":[{},{}],\"resources\":{\"cpu\":77.02751783750413,\"memory\":\"bxncnwfepbnw\",\"ephemeralStorage\":\"m\"}},{\"image\":\"gcgbjb\",\"name\":\"lfgtdysnaquflqbc\",\"command\":[\"amz\",\"rwd\",\"qzeqyjleziunjxdf\"],\"args\":[\"tkw\",\"eg\"],\"env\":[{},{}],\"resources\":{\"cpu\":57.4131663862715,\"memory\":\"qa\",\"ephemeralStorage\":\"jvpilguooqja\"}}]}}}") + .toObject(JobExecutionInner.class); + Assertions.assertEquals("herngb", model.name()); + Assertions.assertEquals("juahokqto", model.id()); + Assertions.assertEquals("auxofshfph", model.type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-08T14:31:19Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-07T16:30:06Z"), model.endTime()); + Assertions.assertEquals("ywhslwkojpllndnp", model.template().containers().get(0).image()); + Assertions.assertEquals("rpqaf", model.template().containers().get(0).name()); + Assertions.assertEquals("gsnnf", model.template().containers().get(0).command().get(0)); + Assertions.assertEquals("tefypococtfjgti", model.template().containers().get(0).args().get(0)); + Assertions.assertEquals(43.561702255466116D, model.template().containers().get(0).resources().cpu()); + Assertions.assertEquals("urmlmuo", model.template().containers().get(0).resources().memory()); + Assertions.assertEquals("ionszonwp", model.template().initContainers().get(0).image()); + Assertions.assertEquals("ajinnixjawrtmjfj", model.template().initContainers().get(0).name()); + Assertions.assertEquals("cxlzhcoxovnekh", model.template().initContainers().get(0).command().get(0)); + Assertions.assertEquals("puslvyj", model.template().initContainers().get(0).args().get(0)); + Assertions.assertEquals(39.025050688905914D, model.template().initContainers().get(0).resources().cpu()); + Assertions.assertEquals("iziesfuughtuq", model.template().initContainers().get(0).resources().memory()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + JobExecutionInner model + = new JobExecutionInner().withName("herngb").withId("juahokqto").withType("auxofshfph") + .withStartTime(OffsetDateTime.parse("2021-02-08T14:31:19Z")).withEndTime( + OffsetDateTime.parse("2021-11-07T16:30:06Z")) + .withTemplate( + new JobExecutionTemplate() + .withContainers( + Arrays.asList(new JobExecutionContainer().withImage("ywhslwkojpllndnp").withName("rpqaf") + .withCommand(Arrays.asList("gsnnf")).withArgs(Arrays.asList("tefypococtfjgti")) + .withEnv(Arrays.asList(new EnvironmentVar(), new EnvironmentVar())) + .withResources(new ContainerResources().withCpu(43.561702255466116D) + .withMemory("urmlmuo")))) + .withInitContainers(Arrays.asList( + new JobExecutionContainer().withImage("ionszonwp").withName("ajinnixjawrtmjfj") + .withCommand(Arrays.asList("cxlzhcoxovnekh", "nlusfnrd", "jxtxrdc", "tjvidt")) + .withArgs(Arrays.asList("puslvyj")) + .withEnv(Arrays.asList(new EnvironmentVar(), new EnvironmentVar(), new EnvironmentVar(), + new EnvironmentVar())) + .withResources( + new ContainerResources().withCpu(39.025050688905914D).withMemory("iziesfuughtuq")), + new JobExecutionContainer().withImage("xu").withName("buew") + .withCommand(Arrays.asList("wnjlxu")).withArgs(Arrays.asList("wpusxjbaqehg", "dohzjq")) + .withEnv(Arrays.asList(new EnvironmentVar(), new EnvironmentVar())).withResources( + new ContainerResources().withCpu(77.02751783750413D).withMemory("bxncnwfepbnw")), + new JobExecutionContainer().withImage("gcgbjb").withName("lfgtdysnaquflqbc") + .withCommand(Arrays.asList("amz", "rwd", "qzeqyjleziunjxdf")) + .withArgs(Arrays.asList("tkw", "eg")) + .withEnv(Arrays.asList(new EnvironmentVar(), new EnvironmentVar())) + .withResources(new ContainerResources().withCpu(57.4131663862715D).withMemory("qa"))))); + model = BinaryData.fromObject(model).toObject(JobExecutionInner.class); + Assertions.assertEquals("herngb", model.name()); + Assertions.assertEquals("juahokqto", model.id()); + Assertions.assertEquals("auxofshfph", model.type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-08T14:31:19Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-07T16:30:06Z"), model.endTime()); + Assertions.assertEquals("ywhslwkojpllndnp", model.template().containers().get(0).image()); + Assertions.assertEquals("rpqaf", model.template().containers().get(0).name()); + Assertions.assertEquals("gsnnf", model.template().containers().get(0).command().get(0)); + Assertions.assertEquals("tefypococtfjgti", model.template().containers().get(0).args().get(0)); + Assertions.assertEquals(43.561702255466116D, model.template().containers().get(0).resources().cpu()); + Assertions.assertEquals("urmlmuo", model.template().containers().get(0).resources().memory()); + Assertions.assertEquals("ionszonwp", model.template().initContainers().get(0).image()); + Assertions.assertEquals("ajinnixjawrtmjfj", model.template().initContainers().get(0).name()); + Assertions.assertEquals("cxlzhcoxovnekh", model.template().initContainers().get(0).command().get(0)); + Assertions.assertEquals("puslvyj", model.template().initContainers().get(0).args().get(0)); + Assertions.assertEquals(39.025050688905914D, model.template().initContainers().get(0).resources().cpu()); + Assertions.assertEquals("iziesfuughtuq", model.template().initContainers().get(0).resources().memory()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteMockTests.java index 59c80438c7ad0..7c1ebb65d5d75 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.jobs().delete("alwcjgckbb", "ccgzpraoxnyu", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.jobs().delete("reyxelyicghf", "rufssjyg", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListMockTests.java index 21c67aab759a2..973a22a951541 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsExecutionsListMockTests.java @@ -31,62 +31,42 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"lfg\",\"id\":\"nnnoytz\",\"type\":\"sewxigpxvkqm\",\"status\":\"Running\",\"startTime\":\"2021-07-26T17:50:31Z\",\"endTime\":\"2021-01-19T22:27:27Z\",\"template\":{\"containers\":[{\"image\":\"ify\",\"name\":\"zeyuubeidszl\",\"command\":[\"oithg\"],\"args\":[\"fltgvdiho\",\"nkrxwetwkdrcy\"],\"env\":[{},{},{}],\"resources\":{\"cpu\":90.34799204771788,\"memory\":\"uzdqumoenod\",\"ephemeralStorage\":\"ienhqhskndnelq\"}}],\"initContainers\":[{\"image\":\"lknwfoanniyop\",\"name\":\"xivcnrlyxnu\",\"command\":[\"p\",\"blkwqpatvbqs\",\"tcjb\",\"tvivuzqym\"],\"args\":[\"wogtgitsq\"],\"env\":[{},{}],\"resources\":{\"cpu\":25.130681898907213,\"memory\":\"banf\",\"ephemeralStorage\":\"ds\"}},{\"image\":\"mxeatkdbmwnrdjy\",\"name\":\"q\",\"command\":[\"omhjrmkuhm\",\"xljalfihc\",\"mobcan\",\"de\"],\"args\":[\"cwg\",\"xf\",\"vaknokzwjj\",\"r\"],\"env\":[{},{},{}],\"resources\":{\"cpu\":55.14214526010209,\"memory\":\"yfytpq\",\"ephemeralStorage\":\"x\"}},{\"image\":\"m\",\"name\":\"jivyqlkjuv\",\"command\":[\"msl\",\"oyovwz\",\"bpqvybefg\"],\"args\":[\"nokcv\"],\"env\":[{},{},{},{}],\"resources\":{\"cpu\":75.35469796285636,\"memory\":\"vcuartrhun\",\"ephemeralStorage\":\"iryky\"}}]}}]}"; + String responseStr + = "{\"value\":[{\"name\":\"gnnbzrtf\",\"id\":\"dzuubjtvgjsxm\",\"type\":\"jjvavdpww\",\"properties\":{\"status\":\"Degraded\",\"startTime\":\"2021-08-09T18:08:13Z\",\"endTime\":\"2021-10-21T08:59:09Z\",\"template\":{\"containers\":[{\"image\":\"ofw\",\"name\":\"m\",\"command\":[\"cauwazcgwdfriwg\",\"bjpozokscvgllixd\",\"byfg\",\"ewqkjvxprwpxs\"],\"args\":[\"utxlcskltezu\"],\"env\":[{},{}],\"resources\":{}},{\"image\":\"lfb\",\"name\":\"dc\",\"command\":[\"srtmdylperpiltt\"],\"args\":[\"czfcmfpfbod\",\"tresr\"],\"env\":[{},{},{}],\"resources\":{}}],\"initContainers\":[{\"image\":\"ft\",\"name\":\"ivmuqkevzgjyp\",\"command\":[\"xmpdxxzetww\",\"jwotnxlkfhglhrf\"],\"args\":[\"wecr\"],\"env\":[{},{},{}],\"resources\":{}},{\"image\":\"selqxov\",\"name\":\"qibukklvzrl\",\"command\":[\"ccmetjsczivfqb\"],\"args\":[\"sdsyenzsie\",\"scplhyvd\",\"xlyzkxit\"],\"env\":[{},{},{},{}],\"resources\":{}},{\"image\":\"svkolru\",\"name\":\"ovmozsayebraz\",\"command\":[\"pzbtzuyk\",\"kipfsdyepf\",\"ocmbezacf\",\"ztgaz\"],\"args\":[\"ejgaaokct\"],\"env\":[{},{}],\"resources\":{}}]}}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.jobsExecutions().list("sykvwjtqpke", "myltj", "rspxklur", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.jobsExecutions().list("cy", "yzlwhbwzjnufzrf", "m", com.azure.core.util.Context.NONE); - Assertions.assertEquals("lfg", response.iterator().next().name()); - Assertions.assertEquals("nnnoytz", response.iterator().next().id()); - Assertions.assertEquals("sewxigpxvkqm", response.iterator().next().type()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-26T17:50:31Z"), response.iterator().next().startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-19T22:27:27Z"), response.iterator().next().endTime()); - Assertions.assertEquals("ify", response.iterator().next().template().containers().get(0).image()); - Assertions.assertEquals("zeyuubeidszl", response.iterator().next().template().containers().get(0).name()); - Assertions.assertEquals("oithg", response.iterator().next().template().containers().get(0).command().get(0)); - Assertions.assertEquals("fltgvdiho", response.iterator().next().template().containers().get(0).args().get(0)); - Assertions - .assertEquals( - 90.34799204771788D, response.iterator().next().template().containers().get(0).resources().cpu()); - Assertions - .assertEquals( - "uzdqumoenod", response.iterator().next().template().containers().get(0).resources().memory()); - Assertions.assertEquals("lknwfoanniyop", response.iterator().next().template().initContainers().get(0).image()); - Assertions.assertEquals("xivcnrlyxnu", response.iterator().next().template().initContainers().get(0).name()); - Assertions.assertEquals("p", response.iterator().next().template().initContainers().get(0).command().get(0)); - Assertions - .assertEquals("wogtgitsq", response.iterator().next().template().initContainers().get(0).args().get(0)); - Assertions - .assertEquals( - 25.130681898907213D, response.iterator().next().template().initContainers().get(0).resources().cpu()); - Assertions - .assertEquals("banf", response.iterator().next().template().initContainers().get(0).resources().memory()); + Assertions.assertEquals("gnnbzrtf", response.iterator().next().name()); + Assertions.assertEquals("dzuubjtvgjsxm", response.iterator().next().id()); + Assertions.assertEquals("jjvavdpww", response.iterator().next().type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-09T18:08:13Z"), response.iterator().next().startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-21T08:59:09Z"), response.iterator().next().endTime()); + Assertions.assertEquals("ofw", response.iterator().next().template().containers().get(0).image()); + Assertions.assertEquals("m", response.iterator().next().template().containers().get(0).name()); + Assertions.assertEquals("cauwazcgwdfriwg", + response.iterator().next().template().containers().get(0).command().get(0)); + Assertions.assertEquals("utxlcskltezu", + response.iterator().next().template().containers().get(0).args().get(0)); + Assertions.assertEquals("ft", response.iterator().next().template().initContainers().get(0).image()); + Assertions.assertEquals("ivmuqkevzgjyp", response.iterator().next().template().initContainers().get(0).name()); + Assertions.assertEquals("xmpdxxzetww", + response.iterator().next().template().initContainers().get(0).command().get(0)); + Assertions.assertEquals("wecr", response.iterator().next().template().initContainers().get(0).args().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorWithResponseMockTests.java new file mode 100644 index 0000000000000..b6bcb58964d04 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsGetDetectorWithResponseMockTests.java @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.Diagnostics; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class JobsGetDetectorWithResponseMockTests { + @Test + public void testGetDetectorWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"metadata\":{\"id\":\"hbuubpy\",\"name\":\"wtjoxz\",\"description\":\"wfqchvcz\",\"author\":\"j\",\"category\":\"cta\",\"supportTopicList\":[{\"id\":\"rtpqpemhz\",\"pesId\":\"krepdqhqyhwqwem\"},{\"id\":\"qabckmzeoxin\",\"pesId\":\"re\"}],\"analysisTypes\":[\"whlpuzjpceezn\",\"angp\",\"bfaxyxzlbc\"],\"type\":\"hm\",\"score\":3.0061007},\"dataset\":[{\"table\":{\"tableName\":\"ndktxfv\",\"columns\":[{},{},{},{}],\"rows\":[\"dataqgpkrie\"]},\"renderingProperties\":{\"type\":1132775916,\"title\":\"xx\",\"description\":\"wzkyf\",\"isVisible\":true}},{\"table\":{\"tableName\":\"wxeiicrmpepk\",\"columns\":[{},{},{},{}],\"rows\":[\"dataxijvskwsdgkjgyac\"]},\"renderingProperties\":{\"type\":1775384365,\"title\":\"kwefc\",\"description\":\"inwoqartwyxq\",\"isVisible\":false}},{\"table\":{\"tableName\":\"vatdavuqmcbymsf\",\"columns\":[{},{}],\"rows\":[\"datauvjezcjumvps\",\"datamioyo\"]},\"renderingProperties\":{\"type\":671724843,\"title\":\"iqwnnraclibbfq\",\"description\":\"pkl\",\"isVisible\":false}},{\"table\":{\"tableName\":\"nhautwukexzgpmnm\",\"columns\":[{},{},{},{}],\"rows\":[\"dataqilwgdfpfqfpcvs\",\"dataclg\",\"datarvwerfwxbsmtb\"]},\"renderingProperties\":{\"type\":273842551,\"title\":\"hci\",\"description\":\"wdv\",\"isVisible\":true}}],\"status\":{\"message\":\"kqhs\",\"statusId\":811287506},\"dataProviderMetadata\":{\"providerName\":\"wpq\",\"propertyBag\":[{\"name\":\"uwyqwdq\",\"value\":\"mghg\"},{\"name\":\"z\",\"value\":\"lujkhn\"},{\"name\":\"mrnkfmkhcqtwml\",\"value\":\"jnqtqe\"},{\"name\":\"j\",\"value\":\"vragpokddx\"}]}},\"id\":\"hhkvguavtptbk\",\"name\":\"wkqyns\",\"type\":\"gbvoffbkk\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Diagnostics response = manager.jobs() + .getDetectorWithResponse("qayfl", "iyu", "snuudtelvhyibdr", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("whlpuzjpceezn", response.properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("ndktxfv", response.properties().dataset().get(0).table().tableName()); + Assertions.assertEquals(1132775916, response.properties().dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("xx", response.properties().dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("wzkyf", response.properties().dataset().get(0).renderingProperties().description()); + Assertions.assertEquals(true, response.properties().dataset().get(0).renderingProperties().isVisible()); + Assertions.assertEquals("kqhs", response.properties().status().message()); + Assertions.assertEquals(811287506, response.properties().status().statusId()); + Assertions.assertEquals("wpq", response.properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("uwyqwdq", response.properties().dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("mghg", response.properties().dataProviderMetadata().propertyBag().get(0).value()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsWithResponseMockTests.java new file mode 100644 index 0000000000000..9efe3b6bc09fc --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsListDetectorsWithResponseMockTests.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.DiagnosticsCollection; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class JobsListDetectorsWithResponseMockTests { + @Test + public void testListDetectorsWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"hdklmvetatl\",\"name\":\"fqoixwgiksbbvt\",\"description\":\"xrpo\",\"author\":\"phchgjtnhtukfaci\",\"category\":\"bfntumeezbxvq\",\"supportTopicList\":[{},{},{},{}],\"analysisTypes\":[\"wcgasgom\",\"mjzwx\"],\"type\":\"govsxpwwzt\",\"score\":71.73476},\"dataset\":[{},{},{},{}],\"status\":{\"message\":\"gfredmlscg\",\"statusId\":836747859},\"dataProviderMetadata\":{\"providerName\":\"na\",\"propertyBag\":[{},{},{}]}},\"id\":\"wazhpabaco\",\"name\":\"lyotg\",\"type\":\"wsxnsrqor\"},{\"properties\":{\"metadata\":{\"id\":\"mv\",\"name\":\"bxeetqujxcxxqn\",\"description\":\"qjkedwqurc\",\"author\":\"jmrvvxwjongz\",\"category\":\"hqqrsilcch\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"ansbvriaqg\",\"tojrulfucte\",\"rthcfjzh\",\"lyubqjrostvrjeq\"],\"type\":\"z\",\"score\":80.793976},\"dataset\":[{},{},{},{}],\"status\":{\"message\":\"rxalxrdha\",\"statusId\":1408690933},\"dataProviderMetadata\":{\"providerName\":\"sn\",\"propertyBag\":[{},{},{},{}]}},\"id\":\"a\",\"name\":\"iwkkvya\",\"type\":\"xkvvcs\"},{\"properties\":{\"metadata\":{\"id\":\"uvdjkqxetqm\",\"name\":\"ivrjjxnwxdc\",\"description\":\"ojxl\",\"author\":\"z\",\"category\":\"pgfquwzpwiibelwc\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"bpjxljtxbusqtb\"],\"type\":\"niuisdz\",\"score\":80.91173},\"dataset\":[{},{}],\"status\":{\"message\":\"agsecnadbuw\",\"statusId\":1642935539},\"dataProviderMetadata\":{\"providerName\":\"llmqiyne\",\"propertyBag\":[{},{}]}},\"id\":\"lnkkiiwvmtumxpym\",\"name\":\"jfuaxroqvqpilrgu\",\"type\":\"canlduwzorxs\"},{\"properties\":{\"metadata\":{\"id\":\"qk\",\"name\":\"ymxkqvfqepdx\",\"description\":\"tuubwyvpjb\",\"author\":\"cpj\",\"category\":\"uqgixex\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"wljavhuerkjddvrg\",\"iegftc\",\"biiftksdwgdnk\"],\"type\":\"gmwdh\",\"score\":87.856026},\"dataset\":[{},{}],\"status\":{\"message\":\"ldbglzout\",\"statusId\":1388435389},\"dataProviderMetadata\":{\"providerName\":\"zeka\",\"propertyBag\":[{},{}]}},\"id\":\"zg\",\"name\":\"norbjgmnzjot\",\"type\":\"mrxkhlobvv\"}],\"nextLink\":\"hvhd\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DiagnosticsCollection response + = manager.jobs().listDetectorsWithResponse("w", "ae", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("wcgasgom", response.value().get(0).properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("gfredmlscg", response.value().get(0).properties().status().message()); + Assertions.assertEquals(836747859, response.value().get(0).properties().status().statusId()); + Assertions.assertEquals("na", response.value().get(0).properties().dataProviderMetadata().providerName()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionMockTests.java index f0a5d576c078d..c392a95255ff6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopExecutionMockTests.java @@ -32,30 +32,20 @@ public void testStopExecution() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.jobs().stopExecution("aqylkjztji", "azjcgmxitpfin", "cpdltkrlg", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.jobs().stopExecution("jsvlpg", "dnw", "ehaqidoyzltgio", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsMockTests.java index 6370a6fe15519..a5924215bfa87 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JobsStopMultipleExecutionsMockTests.java @@ -30,42 +30,31 @@ public void testStopMultipleExecutions() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"elyuj\",\"id\":\"youmpc\",\"type\":\"eclcdigptajbrzm\",\"status\":\"Processing\",\"startTime\":\"2021-01-20T00:05:16Z\",\"endTime\":\"2021-10-11T16:23:08Z\",\"template\":{\"containers\":[{},{}],\"initContainers\":[{},{},{}]}},{\"name\":\"utgjcyz\",\"id\":\"jdnrqjbt\",\"type\":\"eaoqaqbzgyhf\",\"status\":\"Failed\",\"startTime\":\"2021-09-21T19:14:48Z\",\"endTime\":\"2021-03-20T20:58:19Z\",\"template\":{\"containers\":[{},{},{}],\"initContainers\":[{},{},{}]}},{\"name\":\"e\",\"id\":\"iyslpkcvmwfaux\",\"type\":\"pmywbormcq\",\"status\":\"Succeeded\",\"startTime\":\"2021-02-28T18:13:56Z\",\"endTime\":\"2021-09-01T08:07:43Z\",\"template\":{\"containers\":[{},{}],\"initContainers\":[{},{},{}]}},{\"name\":\"jxjmcsmyqw\",\"id\":\"vcpwnkwy\",\"type\":\"wofali\",\"status\":\"Failed\",\"startTime\":\"2021-10-16T22:49:42Z\",\"endTime\":\"2021-04-13T10:57:52Z\",\"template\":{\"containers\":[{}],\"initContainers\":[{}]}}],\"nextLink\":\"sknxrwzawnvsbcf\"}"; + String responseStr + = "{\"value\":[{\"name\":\"cyheqwbpqqncj\",\"id\":\"khjoz\",\"type\":\"mcwmbup\",\"properties\":{\"status\":\"Failed\",\"startTime\":\"2021-04-23T03:13:25Z\",\"endTime\":\"2021-08-31T23:28:22Z\",\"template\":{\"containers\":[{},{},{},{}],\"initContainers\":[{},{},{}]}}},{\"name\":\"bsvs\",\"id\":\"ieswhddzydisn\",\"type\":\"pywyjlnld\",\"properties\":{\"status\":\"Succeeded\",\"startTime\":\"2021-02-22T08:36:21Z\",\"endTime\":\"2021-08-08T07:22:04Z\",\"template\":{\"containers\":[{},{}],\"initContainers\":[{}]}}},{\"name\":\"zjihweebiphr\",\"id\":\"cjwqw\",\"type\":\"sratjhdhzyb\",\"properties\":{\"status\":\"Stopped\",\"startTime\":\"2021-04-15T18:17:15Z\",\"endTime\":\"2021-09-02T11:19:23Z\",\"template\":{\"containers\":[{},{},{},{}],\"initContainers\":[{},{}]}}}],\"nextLink\":\"v\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - ContainerAppJobExecutions response = - manager.jobs().stopMultipleExecutions("mtbdrvcqgu", "fzhompheq", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("elyuj", response.value().get(0).name()); - Assertions.assertEquals("youmpc", response.value().get(0).id()); - Assertions.assertEquals("eclcdigptajbrzm", response.value().get(0).type()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-20T00:05:16Z"), response.value().get(0).startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-11T16:23:08Z"), response.value().get(0).endTime()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + ContainerAppJobExecutions response + = manager.jobs().stopMultipleExecutions("qoqpepiaeap", "sergdtpe", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("cyheqwbpqqncj", response.value().get(0).name()); + Assertions.assertEquals("khjoz", response.value().get(0).id()); + Assertions.assertEquals("mcwmbup", response.value().get(0).type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-23T03:13:25Z"), response.value().get(0).startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-31T23:28:22Z"), response.value().get(0).endTime()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JwtClaimChecksTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JwtClaimChecksTests.java index 38ac66bf51f7d..4b359efc6ac89 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JwtClaimChecksTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/JwtClaimChecksTests.java @@ -12,23 +12,19 @@ public final class JwtClaimChecksTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - JwtClaimChecks model = - BinaryData - .fromString( - "{\"allowedGroups\":[\"htymw\"],\"allowedClientApplications\":[\"kfthwxmntei\",\"aop\",\"km\"]}") - .toObject(JwtClaimChecks.class); - Assertions.assertEquals("htymw", model.allowedGroups().get(0)); - Assertions.assertEquals("kfthwxmntei", model.allowedClientApplications().get(0)); + JwtClaimChecks model = BinaryData + .fromString("{\"allowedGroups\":[\"klj\",\"vbqid\"],\"allowedClientApplications\":[\"jzyulpk\",\"dj\"]}") + .toObject(JwtClaimChecks.class); + Assertions.assertEquals("klj", model.allowedGroups().get(0)); + Assertions.assertEquals("jzyulpk", model.allowedClientApplications().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JwtClaimChecks model = - new JwtClaimChecks() - .withAllowedGroups(Arrays.asList("htymw")) - .withAllowedClientApplications(Arrays.asList("kfthwxmntei", "aop", "km")); + JwtClaimChecks model = new JwtClaimChecks().withAllowedGroups(Arrays.asList("klj", "vbqid")) + .withAllowedClientApplications(Arrays.asList("jzyulpk", "dj")); model = BinaryData.fromObject(model).toObject(JwtClaimChecks.class); - Assertions.assertEquals("htymw", model.allowedGroups().get(0)); - Assertions.assertEquals("kfthwxmntei", model.allowedClientApplications().get(0)); + Assertions.assertEquals("klj", model.allowedGroups().get(0)); + Assertions.assertEquals("jzyulpk", model.allowedClientApplications().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/KedaConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/KedaConfigurationTests.java index 08831dd55f681..7e5a7723aab20 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/KedaConfigurationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/KedaConfigurationTests.java @@ -10,8 +10,8 @@ public final class KedaConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - KedaConfiguration model = - BinaryData.fromString("{\"version\":\"chpqbmfpjba\"}").toObject(KedaConfiguration.class); + KedaConfiguration model + = BinaryData.fromString("{\"version\":\"rylniofrzg\"}").toObject(KedaConfiguration.class); } @org.junit.jupiter.api.Test diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ListUsagesResultTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ListUsagesResultTests.java new file mode 100644 index 0000000000000..b17b79c25492b --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ListUsagesResultTests.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.ListUsagesResult; +import com.azure.resourcemanager.appcontainers.models.UsageName; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ListUsagesResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ListUsagesResult model = BinaryData.fromString( + "{\"value\":[{\"currentValue\":1.2442172,\"limit\":34.942795,\"name\":{\"value\":\"j\",\"localizedValue\":\"cmyfqipgxhnpo\"}}],\"nextLink\":\"qwcabvnui\"}") + .toObject(ListUsagesResult.class); + Assertions.assertEquals(1.2442172f, model.value().get(0).currentValue()); + Assertions.assertEquals(34.942795f, model.value().get(0).limit()); + Assertions.assertEquals("j", model.value().get(0).name().value()); + Assertions.assertEquals("cmyfqipgxhnpo", model.value().get(0).name().localizedValue()); + Assertions.assertEquals("qwcabvnui", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ListUsagesResult model = new ListUsagesResult() + .withValue(Arrays.asList(new UsageInner().withCurrentValue(1.2442172f).withLimit(34.942795f) + .withName(new UsageName().withValue("j").withLocalizedValue("cmyfqipgxhnpo")))) + .withNextLink("qwcabvnui"); + model = BinaryData.fromObject(model).toObject(ListUsagesResult.class); + Assertions.assertEquals(1.2442172f, model.value().get(0).currentValue()); + Assertions.assertEquals(34.942795f, model.value().get(0).limit()); + Assertions.assertEquals("j", model.value().get(0).name().value()); + Assertions.assertEquals("cmyfqipgxhnpo", model.value().get(0).name().localizedValue()); + Assertions.assertEquals("qwcabvnui", model.nextLink()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginRoutesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginRoutesTests.java index 218ab502f5476..c33d27beef8e4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginRoutesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginRoutesTests.java @@ -11,14 +11,15 @@ public final class LoginRoutesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - LoginRoutes model = BinaryData.fromString("{\"logoutEndpoint\":\"sxlzevgbmqj\"}").toObject(LoginRoutes.class); - Assertions.assertEquals("sxlzevgbmqj", model.logoutEndpoint()); + LoginRoutes model + = BinaryData.fromString("{\"logoutEndpoint\":\"qvpkvlrxnjeaseip\"}").toObject(LoginRoutes.class); + Assertions.assertEquals("qvpkvlrxnjeaseip", model.logoutEndpoint()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LoginRoutes model = new LoginRoutes().withLogoutEndpoint("sxlzevgbmqj"); + LoginRoutes model = new LoginRoutes().withLogoutEndpoint("qvpkvlrxnjeaseip"); model = BinaryData.fromObject(model).toObject(LoginRoutes.class); - Assertions.assertEquals("sxlzevgbmqj", model.logoutEndpoint()); + Assertions.assertEquals("qvpkvlrxnjeaseip", model.logoutEndpoint()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginScopesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginScopesTests.java index ec4e43ea7c7a1..5a453060b9460 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginScopesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginScopesTests.java @@ -12,15 +12,14 @@ public final class LoginScopesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - LoginScopes model = - BinaryData.fromString("{\"scopes\":[\"bqidtqaj\",\"yulpkudjkr\"]}").toObject(LoginScopes.class); - Assertions.assertEquals("bqidtqaj", model.scopes().get(0)); + LoginScopes model = BinaryData.fromString("{\"scopes\":[\"exhd\",\"xibqeojnx\"]}").toObject(LoginScopes.class); + Assertions.assertEquals("exhd", model.scopes().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LoginScopes model = new LoginScopes().withScopes(Arrays.asList("bqidtqaj", "yulpkudjkr")); + LoginScopes model = new LoginScopes().withScopes(Arrays.asList("exhd", "xibqeojnx")); model = BinaryData.fromObject(model).toObject(LoginScopes.class); - Assertions.assertEquals("bqidtqaj", model.scopes().get(0)); + Assertions.assertEquals("exhd", model.scopes().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginTests.java deleted file mode 100644 index b7c04a276f9d0..0000000000000 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LoginTests.java +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.appcontainers.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.appcontainers.models.CookieExpiration; -import com.azure.resourcemanager.appcontainers.models.CookieExpirationConvention; -import com.azure.resourcemanager.appcontainers.models.Login; -import com.azure.resourcemanager.appcontainers.models.LoginRoutes; -import com.azure.resourcemanager.appcontainers.models.Nonce; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class LoginTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - Login model = - BinaryData - .fromString( - "{\"routes\":{\"logoutEndpoint\":\"oqfbowskanyk\"},\"preserveUrlFragmentsForLogins\":true,\"allowedExternalRedirectUrls\":[\"iywgqywgndrvynh\"],\"cookieExpiration\":{\"convention\":\"IdentityProviderDerived\",\"timeToExpiration\":\"rcgyn\"},\"nonce\":{\"validateNonce\":false,\"nonceExpirationInterval\":\"cfvmmco\"}}") - .toObject(Login.class); - Assertions.assertEquals("oqfbowskanyk", model.routes().logoutEndpoint()); - Assertions.assertEquals(true, model.preserveUrlFragmentsForLogins()); - Assertions.assertEquals("iywgqywgndrvynh", model.allowedExternalRedirectUrls().get(0)); - Assertions - .assertEquals(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED, model.cookieExpiration().convention()); - Assertions.assertEquals("rcgyn", model.cookieExpiration().timeToExpiration()); - Assertions.assertEquals(false, model.nonce().validateNonce()); - Assertions.assertEquals("cfvmmco", model.nonce().nonceExpirationInterval()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - Login model = - new Login() - .withRoutes(new LoginRoutes().withLogoutEndpoint("oqfbowskanyk")) - .withPreserveUrlFragmentsForLogins(true) - .withAllowedExternalRedirectUrls(Arrays.asList("iywgqywgndrvynh")) - .withCookieExpiration( - new CookieExpiration() - .withConvention(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED) - .withTimeToExpiration("rcgyn")) - .withNonce(new Nonce().withValidateNonce(false).withNonceExpirationInterval("cfvmmco")); - model = BinaryData.fromObject(model).toObject(Login.class); - Assertions.assertEquals("oqfbowskanyk", model.routes().logoutEndpoint()); - Assertions.assertEquals(true, model.preserveUrlFragmentsForLogins()); - Assertions.assertEquals("iywgqywgndrvynh", model.allowedExternalRedirectUrls().get(0)); - Assertions - .assertEquals(CookieExpirationConvention.IDENTITY_PROVIDER_DERIVED, model.cookieExpiration().convention()); - Assertions.assertEquals("rcgyn", model.cookieExpiration().timeToExpiration()); - Assertions.assertEquals(false, model.nonce().validateNonce()); - Assertions.assertEquals("cfvmmco", model.nonce().nonceExpirationInterval()); - } -} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LogsConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LogsConfigurationTests.java new file mode 100644 index 0000000000000..8398ff1f9b157 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/LogsConfigurationTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.LogsConfiguration; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class LogsConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + LogsConfiguration model + = BinaryData.fromString("{\"destinations\":[\"ebwtswb\"]}").toObject(LogsConfiguration.class); + Assertions.assertEquals("ebwtswb", model.destinations().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + LogsConfiguration model = new LogsConfiguration().withDestinations(Arrays.asList("ebwtswb")); + model = BinaryData.fromObject(model).toObject(LogsConfiguration.class); + Assertions.assertEquals("ebwtswb", model.destinations().get(0)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatePatchTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatePatchTests.java index 00a143b450727..6bfd34f590366 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatePatchTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatePatchTests.java @@ -13,16 +13,16 @@ public final class ManagedCertificatePatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ManagedCertificatePatch model = - BinaryData.fromString("{\"tags\":{\"wegprh\":\"kssjhoiftxfk\"}}").toObject(ManagedCertificatePatch.class); - Assertions.assertEquals("kssjhoiftxfk", model.tags().get("wegprh")); + ManagedCertificatePatch model + = BinaryData.fromString("{\"tags\":{\"jmucftby\":\"mtrwah\"}}").toObject(ManagedCertificatePatch.class); + Assertions.assertEquals("mtrwah", model.tags().get("jmucftby")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedCertificatePatch model = new ManagedCertificatePatch().withTags(mapOf("wegprh", "kssjhoiftxfk")); + ManagedCertificatePatch model = new ManagedCertificatePatch().withTags(mapOf("jmucftby", "mtrwah")); model = BinaryData.fromObject(model).toObject(ManagedCertificatePatch.class); - Assertions.assertEquals("kssjhoiftxfk", model.tags().get("wegprh")); + Assertions.assertEquals("mtrwah", model.tags().get("jmucftby")); } // Use "Map.of" if available diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteWithResponseMockTests.java index 3d049202a7744..73c1963e95919 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedCertificatesDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .managedCertificates() - .deleteWithResponse("gno", "ikkgqo", "jwpindedvabbxbh", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.managedCertificates().deleteWithResponse("zaahzbhuroolkoli", "hhmo", "usuzgfjzcvaaxo", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorWithResponseMockTests.java index 134f43887399d..6f878ae0bd15e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsGetDetectorWithResponseMockTests.java @@ -30,52 +30,39 @@ public void testGetDetectorWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"metadata\":{\"id\":\"hzzysevusxiv\",\"name\":\"rryveimipskdy\",\"description\":\"tv\",\"author\":\"zkaftjvvrux\",\"category\":\"gsyeipqdsmjtg\",\"supportTopicList\":[{\"id\":\"gkkileplkcsmkn\",\"pesId\":\"tbbaedorvvmqf\"},{\"id\":\"ygbdgwumgxdgdhpa\",\"pesId\":\"dexjddvjs\"}],\"analysisTypes\":[\"otmmw\",\"lcol\",\"rsxaptefh\",\"xcgjokjljnhvlq\"],\"type\":\"ek\",\"score\":83.19882},\"dataset\":[{\"table\":{\"tableName\":\"ksdqhjvy\",\"columns\":[{},{},{},{}],\"rows\":[\"datalkhhu\"]},\"renderingProperties\":{\"type\":550059489,\"title\":\"qmavnw\",\"description\":\"wgoknlejjjkx\",\"isVisible\":false}},{\"table\":{\"tableName\":\"bkjbz\",\"columns\":[{}],\"rows\":[\"datakzykjtjk\",\"datasxfwushcdp\",\"dataupnqrmgjfb\",\"datakuwxeoiojfizfavk\"]},\"renderingProperties\":{\"type\":143517524,\"title\":\"cyayk\",\"description\":\"fz\",\"isVisible\":true}},{\"table\":{\"tableName\":\"rzx\",\"columns\":[{},{},{},{}],\"rows\":[\"datarsxkr\"]},\"renderingProperties\":{\"type\":1331396954,\"title\":\"zej\",\"description\":\"viyoypsuhbrnnhjx\",\"isVisible\":false}}],\"status\":{\"message\":\"qkbiwet\",\"statusId\":321648657},\"dataProviderMetadata\":{\"providerName\":\"yqiq\",\"propertyBag\":[{\"name\":\"se\",\"value\":\"lexbsf\"},{\"name\":\"dynojpziuwfb\",\"value\":\"kdtnhqsyclj\"},{\"name\":\"lpkpbafvafhl\",\"value\":\"lc\"},{\"name\":\"evxrhyz\",\"value\":\"wrso\"}]}},\"id\":\"pl\",\"name\":\"dbmairrhvhfnr\",\"type\":\"cwnpqigtuujwouhd\"}"; + String responseStr + = "{\"properties\":{\"metadata\":{\"id\":\"nazku\",\"name\":\"drey\",\"description\":\"whsetwwjwzzqs\",\"author\":\"zuukykcyqhyqq\",\"category\":\"dcykeyst\",\"supportTopicList\":[{\"id\":\"pazdazgbsqgp\",\"pesId\":\"q\"}],\"analysisTypes\":[\"tmdpvozglqjbknlz\",\"lctzeyowmndcovd\",\"zqauxzan\"],\"type\":\"kvfruwkudr\",\"score\":66.552605},\"dataset\":[{\"table\":{\"tableName\":\"dqyemebunaucm\",\"columns\":[{},{},{}],\"rows\":[\"dataeemmjauwcgx\"]},\"renderingProperties\":{\"type\":1976462334,\"title\":\"aitranizerw\",\"description\":\"dasmxu\",\"isVisible\":true}},{\"table\":{\"tableName\":\"gfcoc\",\"columns\":[{},{}],\"rows\":[\"dataiylfmpztrau\",\"datasvhl\",\"datadculregp\"]},\"renderingProperties\":{\"type\":1371553792,\"title\":\"hvrztnvg\",\"description\":\"hqrdgrtwmewjzlpy\",\"isVisible\":false}},{\"table\":{\"tableName\":\"zwjcaye\",\"columns\":[{},{}],\"rows\":[\"datansyby\",\"datapolwzrghsrlei\"]},\"renderingProperties\":{\"type\":1125068139,\"title\":\"jfncjwvuagfqw\",\"description\":\"tngvmreuptrklz\",\"isVisible\":true}}],\"status\":{\"message\":\"wo\",\"statusId\":1419456371},\"dataProviderMetadata\":{\"providerName\":\"aghm\",\"propertyBag\":[{\"name\":\"lslrcigtzjc\",\"value\":\"xqlaps\"},{\"name\":\"sovyxpavidnievw\",\"value\":\"cvvy\"},{\"name\":\"slpuxgcbdsva\",\"value\":\"nptw\"},{\"name\":\"kx\",\"value\":\"azwu\"}]}},\"id\":\"yqvnjobfe\",\"name\":\"hldiuhzzgqlm\",\"type\":\"aewzgiudjp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Diagnostics response = - manager - .managedEnvironmentDiagnostics() - .getDetectorWithResponse("ctekval", "l", "tjqvqyvweht", com.azure.core.util.Context.NONE) - .getValue(); + Diagnostics response = manager.managedEnvironmentDiagnostics() + .getDetectorWithResponse("ygnxcgjtfrnqukt", "fnslnlrxsmy", "trwntfmtbgw", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("otmmw", response.properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("ksdqhjvy", response.properties().dataset().get(0).table().tableName()); - Assertions.assertEquals(550059489, response.properties().dataset().get(0).renderingProperties().type()); - Assertions.assertEquals("qmavnw", response.properties().dataset().get(0).renderingProperties().title()); - Assertions - .assertEquals("wgoknlejjjkx", response.properties().dataset().get(0).renderingProperties().description()); - Assertions.assertEquals(false, response.properties().dataset().get(0).renderingProperties().isVisible()); - Assertions.assertEquals("qkbiwet", response.properties().status().message()); - Assertions.assertEquals(321648657, response.properties().status().statusId()); - Assertions.assertEquals("yqiq", response.properties().dataProviderMetadata().providerName()); - Assertions.assertEquals("se", response.properties().dataProviderMetadata().propertyBag().get(0).name()); - Assertions.assertEquals("lexbsf", response.properties().dataProviderMetadata().propertyBag().get(0).value()); + Assertions.assertEquals("tmdpvozglqjbknlz", response.properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("dqyemebunaucm", response.properties().dataset().get(0).table().tableName()); + Assertions.assertEquals(1976462334, response.properties().dataset().get(0).renderingProperties().type()); + Assertions.assertEquals("aitranizerw", response.properties().dataset().get(0).renderingProperties().title()); + Assertions.assertEquals("dasmxu", response.properties().dataset().get(0).renderingProperties().description()); + Assertions.assertEquals(true, response.properties().dataset().get(0).renderingProperties().isVisible()); + Assertions.assertEquals("wo", response.properties().status().message()); + Assertions.assertEquals(1419456371, response.properties().status().statusId()); + Assertions.assertEquals("aghm", response.properties().dataProviderMetadata().providerName()); + Assertions.assertEquals("lslrcigtzjc", + response.properties().dataProviderMetadata().propertyBag().get(0).name()); + Assertions.assertEquals("xqlaps", response.properties().dataProviderMetadata().propertyBag().get(0).value()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsWithResponseMockTests.java index 1616e0fc5eed9..975abe0dd0c4d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentDiagnosticsListDetectorsWithResponseMockTests.java @@ -30,45 +30,30 @@ public void testListDetectorsWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"zusjsz\",\"name\":\"scmnlziji\",\"description\":\"ehgmvflnwyv\",\"author\":\"xrerlniylylyf\",\"category\":\"zutgqztwhghmupg\",\"supportTopicList\":[{},{},{},{}],\"analysisTypes\":[\"dxabbujfta\"],\"type\":\"nbbklqpxzucafed\",\"score\":51.97972},\"dataset\":[{},{}],\"status\":{\"message\":\"fwxudgnhg\",\"statusId\":1717815556},\"dataProviderMetadata\":{\"providerName\":\"alvnbwgpb\",\"propertyBag\":[{},{},{}]}},\"id\":\"luclvdjjuk\",\"name\":\"rdnqodxahhxhqf\",\"type\":\"qnvzoqgyipemch\"},{\"properties\":{\"metadata\":{\"id\":\"czuejdtxptl\",\"name\":\"wzhomewjjstl\",\"description\":\"hqawmo\",\"author\":\"ancz\",\"category\":\"odrrslblxyd\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"vbxiwkgfbqlj\",\"qkhychocok\",\"lehu\",\"qlrqffaweyurk\"],\"type\":\"y\",\"score\":15.436524},\"dataset\":[{}],\"status\":{\"message\":\"uqd\",\"statusId\":1079970798},\"dataProviderMetadata\":{\"providerName\":\"gchbapxkiy\",\"propertyBag\":[{}]}},\"id\":\"kbajbu\",\"name\":\"cgduusio\",\"type\":\"cblevpmc\"},{\"properties\":{\"metadata\":{\"id\":\"xkyxlzgs\",\"name\":\"kzzltafhbzf\",\"description\":\"vwmbjlzqsczpg\",\"author\":\"wnapfdq\",\"category\":\"wf\",\"supportTopicList\":[{},{},{}],\"analysisTypes\":[\"wjtkschgcgqyhl\",\"seyqrhvyeldotjv\"],\"type\":\"wiswskukjtasbvw\",\"score\":54.18455},\"dataset\":[{},{},{},{}],\"status\":{\"message\":\"txfkndlqvt\",\"statusId\":926574477},\"dataProviderMetadata\":{\"providerName\":\"mmbugtywatmqaq\",\"propertyBag\":[{},{},{}]}},\"id\":\"atgr\",\"name\":\"eshoygzcb\",\"type\":\"fqxkfaoyteh\"},{\"properties\":{\"metadata\":{\"id\":\"jmvqmtd\",\"name\":\"kygroejnndljdj\",\"description\":\"kb\",\"author\":\"eqy\",\"category\":\"ceysfaqeg\",\"supportTopicList\":[{},{}],\"analysisTypes\":[\"shwddkvbxgk\"],\"type\":\"sybwptdaca\",\"score\":12.566},\"dataset\":[{}],\"status\":{\"message\":\"ymtpo\",\"statusId\":1736120485},\"dataProviderMetadata\":{\"providerName\":\"zerohzrsqalsxk\",\"propertyBag\":[{},{},{}]}},\"id\":\"qapfgsdpc\",\"name\":\"essmzhhku\",\"type\":\"ip\"}],\"nextLink\":\"q\"}"; + String responseStr + = "{\"value\":[{\"properties\":{\"metadata\":{\"id\":\"qpmbhyqg\",\"name\":\"rmmttjxop\",\"description\":\"erhsmvgohtw\",\"author\":\"qilrixysfn\",\"category\":\"sqywwwmhkru\",\"supportTopicList\":[{},{},{}],\"analysisTypes\":[\"ympmlqoin\",\"zduewihapfjii\"],\"type\":\"jdiq\",\"score\":12.614834},\"dataset\":[{}],\"status\":{\"message\":\"cl\",\"statusId\":1765036901},\"dataProviderMetadata\":{\"providerName\":\"sbw\",\"propertyBag\":[{},{}]}},\"id\":\"bv\",\"name\":\"ipbwxgooo\",\"type\":\"zp\"},{\"properties\":{\"metadata\":{\"id\":\"s\",\"name\":\"knpdgzigjsu\",\"description\":\"whgsaodkww\",\"author\":\"afoctohz\",\"category\":\"quvwsxbgn\",\"supportTopicList\":[{},{},{},{}],\"analysisTypes\":[\"qchoadhrsxqv\",\"vspabdsrgfajgl\",\"rsubklrxhjnl\"],\"type\":\"etjdvqydieqqkwa\",\"score\":70.96507},\"dataset\":[{}],\"status\":{\"message\":\"zxoebwgjxb\",\"statusId\":315596375},\"dataProviderMetadata\":{\"providerName\":\"aupwtzvpaklo\",\"propertyBag\":[{},{},{},{}]}},\"id\":\"zrpejpl\",\"name\":\"s\",\"type\":\"nbtttkgsuxu\"},{\"properties\":{\"metadata\":{\"id\":\"gkp\",\"name\":\"boy\",\"description\":\"ebhuhkslgwlokhu\",\"author\":\"ijyzcqypzqzufgsy\",\"category\":\"jyvdwtfxptpqayam\",\"supportTopicList\":[{}],\"analysisTypes\":[\"ybmx\"],\"type\":\"xocuullojkpoyhgw\",\"score\":2.738905},\"dataset\":[{},{},{},{}],\"status\":{\"message\":\"dljzgdyrc\",\"statusId\":1586962716},\"dataProviderMetadata\":{\"providerName\":\"gzlrqhbj\",\"propertyBag\":[{},{},{},{}]}},\"id\":\"dxwbsfpyxxtj\",\"name\":\"flecominxojjl\",\"type\":\"xxdhilzzdzzqjm\"},{\"properties\":{\"metadata\":{\"id\":\"y\",\"name\":\"ribqlotokh\",\"description\":\"wtaznkcqw\",\"author\":\"wjyofgwhnkbtl\",\"category\":\"jssmctsnldkpwo\",\"supportTopicList\":[{},{},{}],\"analysisTypes\":[\"bxbteogfgfiijry\",\"wlefksxqceazfpxg\",\"m\",\"vzvluyq\"],\"type\":\"ios\",\"score\":85.90183},\"dataset\":[{},{},{}],\"status\":{\"message\":\"fppuacvfye\",\"statusId\":970206060},\"dataProviderMetadata\":{\"providerName\":\"x\",\"propertyBag\":[{},{},{}]}},\"id\":\"soy\",\"name\":\"hpvtyqftteh\",\"type\":\"pboujs\"}],\"nextLink\":\"fvvdshxcdedsue\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - DiagnosticsCollection response = - manager - .managedEnvironmentDiagnostics() - .listDetectorsWithResponse("w", "vcacoyv", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("dxabbujfta", response.value().get(0).properties().metadata().analysisTypes().get(0)); - Assertions.assertEquals("fwxudgnhg", response.value().get(0).properties().status().message()); - Assertions.assertEquals(1717815556, response.value().get(0).properties().status().statusId()); - Assertions - .assertEquals("alvnbwgpb", response.value().get(0).properties().dataProviderMetadata().providerName()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DiagnosticsCollection response = manager.managedEnvironmentDiagnostics() + .listDetectorsWithResponse("sqwudohzilfmnli", "psimsf", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("ympmlqoin", response.value().get(0).properties().metadata().analysisTypes().get(0)); + Assertions.assertEquals("cl", response.value().get(0).properties().status().message()); + Assertions.assertEquals(1765036901, response.value().get(0).properties().status().statusId()); + Assertions.assertEquals("sbw", response.value().get(0).properties().dataProviderMetadata().providerName()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentPropertiesPeerAuthenticationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentPropertiesPeerAuthenticationTests.java index 78bbd4d0b2955..5e7b39d3ab48b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentPropertiesPeerAuthenticationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentPropertiesPeerAuthenticationTests.java @@ -12,17 +12,15 @@ public final class ManagedEnvironmentPropertiesPeerAuthenticationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ManagedEnvironmentPropertiesPeerAuthentication model = - BinaryData - .fromString("{\"mtls\":{\"enabled\":true}}") - .toObject(ManagedEnvironmentPropertiesPeerAuthentication.class); + ManagedEnvironmentPropertiesPeerAuthentication model = BinaryData.fromString("{\"mtls\":{\"enabled\":true}}") + .toObject(ManagedEnvironmentPropertiesPeerAuthentication.class); Assertions.assertEquals(true, model.mtls().enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedEnvironmentPropertiesPeerAuthentication model = - new ManagedEnvironmentPropertiesPeerAuthentication().withMtls(new Mtls().withEnabled(true)); + ManagedEnvironmentPropertiesPeerAuthentication model + = new ManagedEnvironmentPropertiesPeerAuthentication().withMtls(new Mtls().withEnabled(true)); model = BinaryData.fromObject(model).toObject(ManagedEnvironmentPropertiesPeerAuthentication.class); Assertions.assertEquals(true, model.mtls().enabled()); } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListMockTests.java new file mode 100644 index 0000000000000..a43d612fdc730 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentUsagesListMockTests.java @@ -0,0 +1,60 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.Usage; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class ManagedEnvironmentUsagesListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"currentValue\":74.240906,\"limit\":50.552414,\"name\":{\"value\":\"wki\",\"localizedValue\":\"dgfhbssdpje\"}}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.managedEnvironmentUsages().list("zdnckidbj", "glhzqp", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(74.240906f, response.iterator().next().currentValue()); + Assertions.assertEquals(50.552414f, response.iterator().next().limit()); + Assertions.assertEquals("wki", response.iterator().next().name().value()); + Assertions.assertEquals("dgfhbssdpje", response.iterator().next().name().localizedValue()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteMockTests.java index 66b60c1ee3a64..3c7163e40cc6e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.managedEnvironments().delete("bhtmeplvuk", "obrl", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.managedEnvironments().delete("yimyccgrvk", "xzznnui", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesMockTests.java index 102f93f99355f..12409f1e9082c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsListWorkloadProfileStatesMockTests.java @@ -31,42 +31,29 @@ public void testListWorkloadProfileStates() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"minimumCount\":1521065056,\"maximumCount\":1869442572,\"currentCount\":1373287799},\"id\":\"ycnlbvgjcodk\",\"name\":\"gjiiytssiki\",\"type\":\"bcufqbvntn\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"minimumCount\":1874096637,\"maximumCount\":1667611376,\"currentCount\":883857471},\"id\":\"eumexmjbxc\",\"name\":\"ccwkqmtx\",\"type\":\"p\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .managedEnvironments() - .listWorkloadProfileStates("cgqtag", "rclsso", com.azure.core.util.Context.NONE); + PagedIterable response = manager.managedEnvironments() + .listWorkloadProfileStates("dekotjgxieqfkyf", "iwvjaqupbyyn", com.azure.core.util.Context.NONE); - Assertions.assertEquals(1521065056, response.iterator().next().properties().minimumCount()); - Assertions.assertEquals(1869442572, response.iterator().next().properties().maximumCount()); - Assertions.assertEquals(1373287799, response.iterator().next().properties().currentCount()); + Assertions.assertEquals(1874096637, response.iterator().next().properties().minimumCount()); + Assertions.assertEquals(1667611376, response.iterator().next().properties().maximumCount()); + Assertions.assertEquals(883857471, response.iterator().next().properties().currentCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteWithResponseMockTests.java index ee1f14bc21c8a..d4138b04f453f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedEnvironmentsStoragesDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .managedEnvironmentsStorages() - .deleteWithResponse("ykcrraue", "kcsueho", "ddacbcbgydlqidy", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.managedEnvironmentsStorages().deleteWithResponse("mqgisnion", "tb", "dr", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedServiceIdentityTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedServiceIdentityTests.java index 404c2c533ee79..e805a0ba86c58 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedServiceIdentityTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ManagedServiceIdentityTests.java @@ -15,23 +15,18 @@ public final class ManagedServiceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ManagedServiceIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"d073bf54-c2cd-4b62-9f3b-090a75b5196b\",\"tenantId\":\"635985ec-52b2-47ea-99d6-f20863ffe688\",\"type\":\"None\",\"userAssignedIdentities\":{\"meue\":{\"principalId\":\"4f4bbb93-ec01-481a-aec4-7b4a443e6ae6\",\"clientId\":\"d092a881-698b-4ef0-9294-c894ef9f35bf\"},\"yhz\":{\"principalId\":\"5657ddc3-3e07-41e8-aaab-9381cc5eced8\",\"clientId\":\"a7b371f6-e488-41c8-848c-3924ddba6d58\"}}}") - .toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); + ManagedServiceIdentity model = BinaryData.fromString( + "{\"principalId\":\"1bfb3b12-d3c4-4b35-8f47-4176368e6e5c\",\"tenantId\":\"ed1e4bcc-472b-48f8-9f19-4d0991ab0086\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"wgxhn\":{\"principalId\":\"028db3d9-1340-449d-bdff-b678a2ee278d\",\"clientId\":\"51e4a008-5b03-4c2d-a0c0-31398ef4c4c6\"},\"x\":{\"principalId\":\"202c75aa-8c8e-4276-b9e6-e3384332734e\",\"clientId\":\"b9b508e5-b6db-42b9-bdce-4f77c2ba9a4d\"}}}") + .toObject(ManagedServiceIdentity.class); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedServiceIdentity model = - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities( - mapOf("meue", new UserAssignedIdentity(), "yhz", new UserAssignedIdentity())); + ManagedServiceIdentity model = new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED) + .withUserAssignedIdentities(mapOf("wgxhn", new UserAssignedIdentity(), "x", new UserAssignedIdentity())); model = BinaryData.fromObject(model).toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, model.type()); } // Use "Map.of" if available diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/MetricsConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/MetricsConfigurationTests.java new file mode 100644 index 0000000000000..6a5c0030e0bb3 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/MetricsConfigurationTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.MetricsConfiguration; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class MetricsConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + MetricsConfiguration model + = BinaryData.fromString("{\"destinations\":[\"fmd\",\"ragegi\",\"vcjfelisdjubggb\"]}") + .toObject(MetricsConfiguration.class); + Assertions.assertEquals("fmd", model.destinations().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + MetricsConfiguration model + = new MetricsConfiguration().withDestinations(Arrays.asList("fmd", "ragegi", "vcjfelisdjubggb")); + model = BinaryData.fromObject(model).toObject(MetricsConfiguration.class); + Assertions.assertEquals("fmd", model.destinations().get(0)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilityWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilityWithResponseMockTests.java index b0eb6026c75ce..42a8af4b2451f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NamespacesCheckNameAvailabilityWithResponseMockTests.java @@ -32,46 +32,31 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"nameAvailable\":false,\"reason\":\"Invalid\",\"message\":\"avn\"}"; + String responseStr = "{\"nameAvailable\":false,\"reason\":\"AlreadyExists\",\"message\":\"htvsnvl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - CheckNameAvailabilityResponse response = - manager - .namespaces() - .checkNameAvailabilityWithResponse( - "ashxgonoyjfqi", - "ubyzn", - new CheckNameAvailabilityRequest().withName("k").withType("eebgvopemtuoqu"), - com.azure.core.util.Context.NONE) - .getValue(); + CheckNameAvailabilityResponse response = manager.namespaces() + .checkNameAvailabilityWithResponse("isjm", "kkhmwdmdlgyqixok", + new CheckNameAvailabilityRequest().withName("jawh").withType("gnqfqqdlcvmyol"), + com.azure.core.util.Context.NONE) + .getValue(); Assertions.assertEquals(false, response.nameAvailable()); - Assertions.assertEquals(CheckNameAvailabilityReason.INVALID, response.reason()); - Assertions.assertEquals("avn", response.message()); + Assertions.assertEquals(CheckNameAvailabilityReason.ALREADY_EXISTS, response.reason()); + Assertions.assertEquals("htvsnvl", response.message()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NfsAzureFilePropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NfsAzureFilePropertiesTests.java new file mode 100644 index 0000000000000..ca4bf2448f51e --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NfsAzureFilePropertiesTests.java @@ -0,0 +1,32 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.AccessMode; +import com.azure.resourcemanager.appcontainers.models.NfsAzureFileProperties; +import org.junit.jupiter.api.Assertions; + +public final class NfsAzureFilePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + NfsAzureFileProperties model + = BinaryData.fromString("{\"server\":\"qplpvmjc\",\"accessMode\":\"ReadWrite\",\"shareName\":\"bidyv\"}") + .toObject(NfsAzureFileProperties.class); + Assertions.assertEquals("qplpvmjc", model.server()); + Assertions.assertEquals(AccessMode.READ_WRITE, model.accessMode()); + Assertions.assertEquals("bidyv", model.shareName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + NfsAzureFileProperties model = new NfsAzureFileProperties().withServer("qplpvmjc") + .withAccessMode(AccessMode.READ_WRITE).withShareName("bidyv"); + model = BinaryData.fromObject(model).toObject(NfsAzureFileProperties.class); + Assertions.assertEquals("qplpvmjc", model.server()); + Assertions.assertEquals(AccessMode.READ_WRITE, model.accessMode()); + Assertions.assertEquals("bidyv", model.shareName()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NonceTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NonceTests.java index e3aed94e1a5b7..1dd4cbc2e2c6e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NonceTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/NonceTests.java @@ -11,19 +11,17 @@ public final class NonceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Nonce model = - BinaryData - .fromString("{\"validateNonce\":true,\"nonceExpirationInterval\":\"kwlzuvccfwnfn\"}") - .toObject(Nonce.class); + Nonce model = BinaryData.fromString("{\"validateNonce\":true,\"nonceExpirationInterval\":\"asxazjpqyegualhb\"}") + .toObject(Nonce.class); Assertions.assertEquals(true, model.validateNonce()); - Assertions.assertEquals("kwlzuvccfwnfn", model.nonceExpirationInterval()); + Assertions.assertEquals("asxazjpqyegualhb", model.nonceExpirationInterval()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Nonce model = new Nonce().withValidateNonce(true).withNonceExpirationInterval("kwlzuvccfwnfn"); + Nonce model = new Nonce().withValidateNonce(true).withNonceExpirationInterval("asxazjpqyegualhb"); model = BinaryData.fromObject(model).toObject(Nonce.class); Assertions.assertEquals(true, model.validateNonce()); - Assertions.assertEquals("kwlzuvccfwnfn", model.nonceExpirationInterval()); + Assertions.assertEquals("asxazjpqyegualhb", model.nonceExpirationInterval()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OpenIdConnectLoginTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OpenIdConnectLoginTests.java index 8f4dce32f8565..1364879d004a6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OpenIdConnectLoginTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OpenIdConnectLoginTests.java @@ -12,23 +12,19 @@ public final class OpenIdConnectLoginTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OpenIdConnectLogin model = - BinaryData - .fromString( - "{\"nameClaimType\":\"otkftutqxlngx\",\"scopes\":[\"gug\",\"xkrxdqmi\",\"tthzrvqd\",\"abhjybi\"]}") - .toObject(OpenIdConnectLogin.class); - Assertions.assertEquals("otkftutqxlngx", model.nameClaimType()); - Assertions.assertEquals("gug", model.scopes().get(0)); + OpenIdConnectLogin model = BinaryData + .fromString("{\"nameClaimType\":\"nfnbacfionlebxe\",\"scopes\":[\"tzxdpnqbqqwx\",\"jfeallnwsub\"]}") + .toObject(OpenIdConnectLogin.class); + Assertions.assertEquals("nfnbacfionlebxe", model.nameClaimType()); + Assertions.assertEquals("tzxdpnqbqqwx", model.scopes().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OpenIdConnectLogin model = - new OpenIdConnectLogin() - .withNameClaimType("otkftutqxlngx") - .withScopes(Arrays.asList("gug", "xkrxdqmi", "tthzrvqd", "abhjybi")); + OpenIdConnectLogin model = new OpenIdConnectLogin().withNameClaimType("nfnbacfionlebxe") + .withScopes(Arrays.asList("tzxdpnqbqqwx", "jfeallnwsub")); model = BinaryData.fromObject(model).toObject(OpenIdConnectLogin.class); - Assertions.assertEquals("otkftutqxlngx", model.nameClaimType()); - Assertions.assertEquals("gug", model.scopes().get(0)); + Assertions.assertEquals("nfnbacfionlebxe", model.nameClaimType()); + Assertions.assertEquals("tzxdpnqbqqwx", model.scopes().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDetailInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDetailInnerTests.java index b64c3024cb9c0..6550d7f15bdc7 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDetailInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDetailInnerTests.java @@ -12,40 +12,31 @@ public final class OperationDetailInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDetailInner model = - BinaryData - .fromString( - "{\"name\":\"vfcibyfmowux\",\"isDataAction\":false,\"display\":{\"provider\":\"dwxf\",\"resource\":\"iivwzjbhyzsxjrka\",\"operation\":\"trnegvmnvuqeqvld\",\"description\":\"astjbkkdmflvestm\"},\"origin\":\"xrrilozapee\"}") - .toObject(OperationDetailInner.class); - Assertions.assertEquals("vfcibyfmowux", model.name()); - Assertions.assertEquals(false, model.isDataAction()); - Assertions.assertEquals("dwxf", model.display().provider()); - Assertions.assertEquals("iivwzjbhyzsxjrka", model.display().resource()); - Assertions.assertEquals("trnegvmnvuqeqvld", model.display().operation()); - Assertions.assertEquals("astjbkkdmflvestm", model.display().description()); - Assertions.assertEquals("xrrilozapee", model.origin()); + OperationDetailInner model = BinaryData.fromString( + "{\"name\":\"xff\",\"isDataAction\":true,\"display\":{\"provider\":\"fbkgozxwopdby\",\"resource\":\"izqaclnapxbiyg\",\"operation\":\"gjkn\",\"description\":\"mfcttux\"},\"origin\":\"yilflqoiquvrehmr\"}") + .toObject(OperationDetailInner.class); + Assertions.assertEquals("xff", model.name()); + Assertions.assertEquals(true, model.isDataAction()); + Assertions.assertEquals("fbkgozxwopdby", model.display().provider()); + Assertions.assertEquals("izqaclnapxbiyg", model.display().resource()); + Assertions.assertEquals("gjkn", model.display().operation()); + Assertions.assertEquals("mfcttux", model.display().description()); + Assertions.assertEquals("yilflqoiquvrehmr", model.origin()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationDetailInner model = - new OperationDetailInner() - .withName("vfcibyfmowux") - .withIsDataAction(false) - .withDisplay( - new OperationDisplay() - .withProvider("dwxf") - .withResource("iivwzjbhyzsxjrka") - .withOperation("trnegvmnvuqeqvld") - .withDescription("astjbkkdmflvestm")) - .withOrigin("xrrilozapee"); + OperationDetailInner model = new OperationDetailInner() + .withName("xff").withIsDataAction(true).withDisplay(new OperationDisplay().withProvider("fbkgozxwopdby") + .withResource("izqaclnapxbiyg").withOperation("gjkn").withDescription("mfcttux")) + .withOrigin("yilflqoiquvrehmr"); model = BinaryData.fromObject(model).toObject(OperationDetailInner.class); - Assertions.assertEquals("vfcibyfmowux", model.name()); - Assertions.assertEquals(false, model.isDataAction()); - Assertions.assertEquals("dwxf", model.display().provider()); - Assertions.assertEquals("iivwzjbhyzsxjrka", model.display().resource()); - Assertions.assertEquals("trnegvmnvuqeqvld", model.display().operation()); - Assertions.assertEquals("astjbkkdmflvestm", model.display().description()); - Assertions.assertEquals("xrrilozapee", model.origin()); + Assertions.assertEquals("xff", model.name()); + Assertions.assertEquals(true, model.isDataAction()); + Assertions.assertEquals("fbkgozxwopdby", model.display().provider()); + Assertions.assertEquals("izqaclnapxbiyg", model.display().resource()); + Assertions.assertEquals("gjkn", model.display().operation()); + Assertions.assertEquals("mfcttux", model.display().description()); + Assertions.assertEquals("yilflqoiquvrehmr", model.origin()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDisplayTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDisplayTests.java index 8668298979b36..3e8cd630a35ec 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDisplayTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationDisplayTests.java @@ -11,29 +11,23 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"hpxlktwkuziycs\",\"resource\":\"vu\",\"operation\":\"ztcktyh\",\"description\":\"qedcgzulwm\"}") - .toObject(OperationDisplay.class); - Assertions.assertEquals("hpxlktwkuziycs", model.provider()); - Assertions.assertEquals("vu", model.resource()); - Assertions.assertEquals("ztcktyh", model.operation()); - Assertions.assertEquals("qedcgzulwm", model.description()); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"hvsujztc\",\"resource\":\"tqjtwhauu\",\"operation\":\"prnjletlxsmr\",\"description\":\"do\"}") + .toObject(OperationDisplay.class); + Assertions.assertEquals("hvsujztc", model.provider()); + Assertions.assertEquals("tqjtwhauu", model.resource()); + Assertions.assertEquals("prnjletlxsmr", model.operation()); + Assertions.assertEquals("do", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationDisplay model = - new OperationDisplay() - .withProvider("hpxlktwkuziycs") - .withResource("vu") - .withOperation("ztcktyh") - .withDescription("qedcgzulwm"); + OperationDisplay model = new OperationDisplay().withProvider("hvsujztc").withResource("tqjtwhauu") + .withOperation("prnjletlxsmr").withDescription("do"); model = BinaryData.fromObject(model).toObject(OperationDisplay.class); - Assertions.assertEquals("hpxlktwkuziycs", model.provider()); - Assertions.assertEquals("vu", model.resource()); - Assertions.assertEquals("ztcktyh", model.operation()); - Assertions.assertEquals("qedcgzulwm", model.description()); + Assertions.assertEquals("hvsujztc", model.provider()); + Assertions.assertEquals("tqjtwhauu", model.resource()); + Assertions.assertEquals("prnjletlxsmr", model.operation()); + Assertions.assertEquals("do", model.description()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationsListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationsListMockTests.java index f8e0b6e17f7b6..24e344e5df60b 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationsListMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/OperationsListMockTests.java @@ -31,43 +31,32 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"oakizvoai\",\"isDataAction\":false,\"display\":{\"provider\":\"nuwiguyxlykwph\",\"resource\":\"zc\",\"operation\":\"hmpejtlkexaonw\",\"description\":\"kcq\"},\"origin\":\"xhxknlc\"}]}"; + String responseStr + = "{\"value\":[{\"name\":\"b\",\"isDataAction\":true,\"display\":{\"provider\":\"dylytcovq\",\"resource\":\"usrf\",\"operation\":\"d\",\"description\":\"fxnxml\"},\"origin\":\"uoswkjmdih\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("oakizvoai", response.iterator().next().name()); - Assertions.assertEquals(false, response.iterator().next().isDataAction()); - Assertions.assertEquals("nuwiguyxlykwph", response.iterator().next().display().provider()); - Assertions.assertEquals("zc", response.iterator().next().display().resource()); - Assertions.assertEquals("hmpejtlkexaonw", response.iterator().next().display().operation()); - Assertions.assertEquals("kcq", response.iterator().next().display().description()); - Assertions.assertEquals("xhxknlc", response.iterator().next().origin()); + Assertions.assertEquals("b", response.iterator().next().name()); + Assertions.assertEquals(true, response.iterator().next().isDataAction()); + Assertions.assertEquals("dylytcovq", response.iterator().next().display().provider()); + Assertions.assertEquals("usrf", response.iterator().next().display().resource()); + Assertions.assertEquals("d", response.iterator().next().display().operation()); + Assertions.assertEquals("fxnxml", response.iterator().next().display().description()); + Assertions.assertEquals("uoswkjmdih", response.iterator().next().origin()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/PreBuildStepTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/PreBuildStepTests.java new file mode 100644 index 0000000000000..e49fc44d4e063 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/PreBuildStepTests.java @@ -0,0 +1,38 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.HttpGet; +import com.azure.resourcemanager.appcontainers.models.PreBuildStep; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class PreBuildStepTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PreBuildStep model = BinaryData.fromString( + "{\"description\":\"iexpbtgiwbwo\",\"scripts\":[\"ashrt\",\"tkcnqxwb\"],\"httpGet\":{\"url\":\"kulpiujwaasi\",\"fileName\":\"i\",\"headers\":[\"yuq\",\"rpqlp\",\"wcciuqgbdbu\"]}}") + .toObject(PreBuildStep.class); + Assertions.assertEquals("iexpbtgiwbwo", model.description()); + Assertions.assertEquals("ashrt", model.scripts().get(0)); + Assertions.assertEquals("kulpiujwaasi", model.httpGet().url()); + Assertions.assertEquals("i", model.httpGet().fileName()); + Assertions.assertEquals("yuq", model.httpGet().headers().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PreBuildStep model = new PreBuildStep().withDescription("iexpbtgiwbwo") + .withScripts(Arrays.asList("ashrt", "tkcnqxwb")).withHttpGet(new HttpGet().withUrl("kulpiujwaasi") + .withFileName("i").withHeaders(Arrays.asList("yuq", "rpqlp", "wcciuqgbdbu"))); + model = BinaryData.fromObject(model).toObject(PreBuildStep.class); + Assertions.assertEquals("iexpbtgiwbwo", model.description()); + Assertions.assertEquals("ashrt", model.scripts().get(0)); + Assertions.assertEquals("kulpiujwaasi", model.httpGet().url()); + Assertions.assertEquals("i", model.httpGet().fileName()); + Assertions.assertEquals("yuq", model.httpGet().headers().get(0)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaCollectionInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaCollectionInnerTests.java index b0a1b8b5d9c86..ed980d566da13 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaCollectionInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaCollectionInnerTests.java @@ -14,114 +14,48 @@ public final class ReplicaCollectionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ReplicaCollectionInner model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"createdTime\":\"2021-07-29T01:36:25Z\",\"runningState\":\"Unknown\",\"runningStateDetails\":\"indfpwpjyl\",\"containers\":[{\"name\":\"h\",\"containerId\":\"sj\",\"ready\":false,\"started\":true,\"restartCount\":554003670,\"runningState\":\"Waiting\",\"runningStateDetails\":\"gofel\",\"logStreamEndpoint\":\"grqmqhldvrii\",\"execEndpoint\":\"jnalghf\"},{\"name\":\"tvsexsowuel\",\"containerId\":\"hhahhxvrhmzkwpjg\",\"ready\":true,\"started\":false,\"restartCount\":1806327837,\"runningState\":\"Running\",\"runningStateDetails\":\"sxhqxujx\",\"logStreamEndpoint\":\"ndxdigrjguufzdm\",\"execEndpoint\":\"qtfihwhbotzinga\"},{\"name\":\"pph\",\"containerId\":\"zqzudph\",\"ready\":false,\"started\":false,\"restartCount\":1182689505,\"runningState\":\"Running\",\"runningStateDetails\":\"wcvtbvkayhmtnvyq\",\"logStreamEndpoint\":\"tkzwpcnpwzc\",\"execEndpoint\":\"esgvvsccyaj\"},{\"name\":\"qfhwyg\",\"containerId\":\"vdnkfxusem\",\"ready\":true,\"started\":true,\"restartCount\":704204401,\"runningState\":\"Running\",\"runningStateDetails\":\"cqdpsqxqvpsvuoym\",\"logStreamEndpoint\":\"celve\",\"execEndpoint\":\"ypql\"}],\"initContainers\":[{\"name\":\"kerqwkyh\",\"containerId\":\"bopgxedkowepbqp\",\"ready\":false,\"started\":true,\"restartCount\":1217910065,\"runningState\":\"Waiting\",\"runningStateDetails\":\"jvcdwxlpqekf\",\"logStreamEndpoint\":\"khtj\",\"execEndpoint\":\"i\"},{\"name\":\"wfqatmtd\",\"containerId\":\"mdvy\",\"ready\":true,\"started\":true,\"restartCount\":624157964,\"runningState\":\"Running\",\"runningStateDetails\":\"kbir\",\"logStreamEndpoint\":\"uzhlhkjoqrv\",\"execEndpoint\":\"aatjinrvgoupmfi\"},{\"name\":\"fggjioolvr\",\"containerId\":\"kvtkkg\",\"ready\":false,\"started\":true,\"restartCount\":423959041,\"runningState\":\"Waiting\",\"runningStateDetails\":\"yvblmhvkzu\",\"logStreamEndpoint\":\"xvvy\",\"execEndpoint\":\"s\"},{\"name\":\"byrqufeg\",\"containerId\":\"vwz\",\"ready\":true,\"started\":false,\"restartCount\":882366629,\"runningState\":\"Running\",\"runningStateDetails\":\"dn\",\"logStreamEndpoint\":\"tvgbmhrixkwmy\",\"execEndpoint\":\"ejvegrhbpnaixex\"}]},\"id\":\"cbdreaxhcexd\",\"name\":\"rvqahqkghtpwi\",\"type\":\"nhyjsv\"},{\"properties\":{\"createdTime\":\"2021-01-05T11:11:10Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"voowvr\",\"containers\":[{\"name\":\"jqppyostronzmy\",\"containerId\":\"fipns\",\"ready\":true,\"started\":true,\"restartCount\":1488259864,\"runningState\":\"Waiting\",\"runningStateDetails\":\"jreafxtsgum\",\"logStreamEndpoint\":\"glikkxwslolb\",\"execEndpoint\":\"vuzlm\"}],\"initContainers\":[{\"name\":\"fktgplc\",\"containerId\":\"wjxeznoigbr\",\"ready\":false,\"started\":false,\"restartCount\":490713776,\"runningState\":\"Running\",\"runningStateDetails\":\"azej\",\"logStreamEndpoint\":\"qkagfhsxt\",\"execEndpoint\":\"ugzxnf\"}]},\"id\":\"azpxdtnkdmkqjjl\",\"name\":\"uenvrkp\",\"type\":\"ou\"}]}") - .toObject(ReplicaCollectionInner.class); - Assertions.assertEquals("h", model.value().get(0).containers().get(0).name()); - Assertions.assertEquals("sj", model.value().get(0).containers().get(0).containerId()); - Assertions.assertEquals(false, model.value().get(0).containers().get(0).ready()); + ReplicaCollectionInner model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"createdTime\":\"2021-02-06T14:48:02Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"qovekqvgqou\",\"containers\":[{\"name\":\"mpjw\",\"containerId\":\"vqikfxcvhrfsphu\",\"ready\":true,\"started\":true,\"restartCount\":95880088,\"runningState\":\"Terminated\",\"runningStateDetails\":\"sqczkvyklxubyj\",\"logStreamEndpoint\":\"fmmfblcqcu\",\"execEndpoint\":\"gqibrtalmetttw\"},{\"name\":\"slqxi\",\"containerId\":\"rmooizqse\",\"ready\":false,\"started\":true,\"restartCount\":719616130,\"runningState\":\"Terminated\",\"runningStateDetails\":\"zhyrpeto\",\"logStreamEndpoint\":\"bjoxs\",\"execEndpoint\":\"vnh\"},{\"name\":\"brqnkkzjcjb\",\"containerId\":\"gaehvvibrxjjst\",\"ready\":false,\"started\":false,\"restartCount\":774286462,\"runningState\":\"Waiting\",\"runningStateDetails\":\"tmo\",\"logStreamEndpoint\":\"klf\",\"execEndpoint\":\"dgfcwqmp\"}],\"initContainers\":[{\"name\":\"xzhemjyh\",\"containerId\":\"uj\",\"ready\":false,\"started\":true,\"restartCount\":1064188609,\"runningState\":\"Running\",\"runningStateDetails\":\"ulkb\",\"logStreamEndpoint\":\"pfajnjwltlwtjj\",\"execEndpoint\":\"ktalhsnvkcdmxz\"},{\"name\":\"oaimlnw\",\"containerId\":\"aomylwea\",\"ready\":true,\"started\":false,\"restartCount\":1226042956,\"runningState\":\"Running\",\"runningStateDetails\":\"npjhlfzswpchwahf\",\"logStreamEndpoint\":\"usnfepgfewet\",\"execEndpoint\":\"yxgncxykxhdjhli\"},{\"name\":\"bcxf\",\"containerId\":\"cporxvxcjz\",\"ready\":false,\"started\":true,\"restartCount\":444626662,\"runningState\":\"Waiting\",\"runningStateDetails\":\"qscjavftjuh\",\"logStreamEndpoint\":\"azkmtgguwp\",\"execEndpoint\":\"r\"}]},\"id\":\"civmmg\",\"name\":\"f\",\"type\":\"fiwrxgkn\"}]}") + .toObject(ReplicaCollectionInner.class); + Assertions.assertEquals("mpjw", model.value().get(0).containers().get(0).name()); + Assertions.assertEquals("vqikfxcvhrfsphu", model.value().get(0).containers().get(0).containerId()); + Assertions.assertEquals(true, model.value().get(0).containers().get(0).ready()); Assertions.assertEquals(true, model.value().get(0).containers().get(0).started()); - Assertions.assertEquals(554003670, model.value().get(0).containers().get(0).restartCount()); - Assertions.assertEquals("kerqwkyh", model.value().get(0).initContainers().get(0).name()); - Assertions.assertEquals("bopgxedkowepbqp", model.value().get(0).initContainers().get(0).containerId()); + Assertions.assertEquals(95880088, model.value().get(0).containers().get(0).restartCount()); + Assertions.assertEquals("xzhemjyh", model.value().get(0).initContainers().get(0).name()); + Assertions.assertEquals("uj", model.value().get(0).initContainers().get(0).containerId()); Assertions.assertEquals(false, model.value().get(0).initContainers().get(0).ready()); Assertions.assertEquals(true, model.value().get(0).initContainers().get(0).started()); - Assertions.assertEquals(1217910065, model.value().get(0).initContainers().get(0).restartCount()); + Assertions.assertEquals(1064188609, model.value().get(0).initContainers().get(0).restartCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReplicaCollectionInner model = - new ReplicaCollectionInner() - .withValue( - Arrays - .asList( - new ReplicaInner() - .withContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("h") - .withContainerId("sj") - .withReady(false) - .withStarted(true) - .withRestartCount(554003670), - new ReplicaContainer() - .withName("tvsexsowuel") - .withContainerId("hhahhxvrhmzkwpjg") - .withReady(true) - .withStarted(false) - .withRestartCount(1806327837), - new ReplicaContainer() - .withName("pph") - .withContainerId("zqzudph") - .withReady(false) - .withStarted(false) - .withRestartCount(1182689505), - new ReplicaContainer() - .withName("qfhwyg") - .withContainerId("vdnkfxusem") - .withReady(true) - .withStarted(true) - .withRestartCount(704204401))) - .withInitContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("kerqwkyh") - .withContainerId("bopgxedkowepbqp") - .withReady(false) - .withStarted(true) - .withRestartCount(1217910065), - new ReplicaContainer() - .withName("wfqatmtd") - .withContainerId("mdvy") - .withReady(true) - .withStarted(true) - .withRestartCount(624157964), - new ReplicaContainer() - .withName("fggjioolvr") - .withContainerId("kvtkkg") - .withReady(false) - .withStarted(true) - .withRestartCount(423959041), - new ReplicaContainer() - .withName("byrqufeg") - .withContainerId("vwz") - .withReady(true) - .withStarted(false) - .withRestartCount(882366629))), - new ReplicaInner() - .withContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("jqppyostronzmy") - .withContainerId("fipns") - .withReady(true) - .withStarted(true) - .withRestartCount(1488259864))) - .withInitContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("fktgplc") - .withContainerId("wjxeznoigbr") - .withReady(false) - .withStarted(false) - .withRestartCount(490713776))))); + ReplicaCollectionInner model = new ReplicaCollectionInner().withValue(Arrays.asList(new ReplicaInner() + .withContainers(Arrays.asList( + new ReplicaContainer().withName("mpjw").withContainerId("vqikfxcvhrfsphu").withReady(true) + .withStarted(true).withRestartCount(95880088), + new ReplicaContainer().withName("slqxi").withContainerId("rmooizqse").withReady(false).withStarted(true) + .withRestartCount(719616130), + new ReplicaContainer().withName("brqnkkzjcjb").withContainerId("gaehvvibrxjjst").withReady(false) + .withStarted(false).withRestartCount(774286462))) + .withInitContainers(Arrays.asList( + new ReplicaContainer().withName("xzhemjyh").withContainerId("uj").withReady(false).withStarted(true) + .withRestartCount(1064188609), + new ReplicaContainer().withName("oaimlnw").withContainerId("aomylwea").withReady(true) + .withStarted(false).withRestartCount(1226042956), + new ReplicaContainer().withName("bcxf").withContainerId("cporxvxcjz").withReady(false).withStarted(true) + .withRestartCount(444626662))))); model = BinaryData.fromObject(model).toObject(ReplicaCollectionInner.class); - Assertions.assertEquals("h", model.value().get(0).containers().get(0).name()); - Assertions.assertEquals("sj", model.value().get(0).containers().get(0).containerId()); - Assertions.assertEquals(false, model.value().get(0).containers().get(0).ready()); + Assertions.assertEquals("mpjw", model.value().get(0).containers().get(0).name()); + Assertions.assertEquals("vqikfxcvhrfsphu", model.value().get(0).containers().get(0).containerId()); + Assertions.assertEquals(true, model.value().get(0).containers().get(0).ready()); Assertions.assertEquals(true, model.value().get(0).containers().get(0).started()); - Assertions.assertEquals(554003670, model.value().get(0).containers().get(0).restartCount()); - Assertions.assertEquals("kerqwkyh", model.value().get(0).initContainers().get(0).name()); - Assertions.assertEquals("bopgxedkowepbqp", model.value().get(0).initContainers().get(0).containerId()); + Assertions.assertEquals(95880088, model.value().get(0).containers().get(0).restartCount()); + Assertions.assertEquals("xzhemjyh", model.value().get(0).initContainers().get(0).name()); + Assertions.assertEquals("uj", model.value().get(0).initContainers().get(0).containerId()); Assertions.assertEquals(false, model.value().get(0).initContainers().get(0).ready()); Assertions.assertEquals(true, model.value().get(0).initContainers().get(0).started()); - Assertions.assertEquals(1217910065, model.value().get(0).initContainers().get(0).restartCount()); + Assertions.assertEquals(1064188609, model.value().get(0).initContainers().get(0).restartCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaContainerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaContainerTests.java index ac54ebd0cd276..6a77fa5d344a6 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaContainerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaContainerTests.java @@ -11,32 +11,25 @@ public final class ReplicaContainerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ReplicaContainer model = - BinaryData - .fromString( - "{\"name\":\"cgpik\",\"containerId\":\"imejzanl\",\"ready\":false,\"started\":false,\"restartCount\":1297052787,\"runningState\":\"Terminated\",\"runningStateDetails\":\"onok\",\"logStreamEndpoint\":\"rjqc\",\"execEndpoint\":\"gzpfrla\"}") - .toObject(ReplicaContainer.class); - Assertions.assertEquals("cgpik", model.name()); - Assertions.assertEquals("imejzanl", model.containerId()); + ReplicaContainer model = BinaryData.fromString( + "{\"name\":\"isofieypefojyqd\",\"containerId\":\"u\",\"ready\":false,\"started\":true,\"restartCount\":1554312221,\"runningState\":\"Terminated\",\"runningStateDetails\":\"hihlhzdsqtzbs\",\"logStreamEndpoint\":\"nowc\",\"execEndpoint\":\"fgmvecactxmwo\"}") + .toObject(ReplicaContainer.class); + Assertions.assertEquals("isofieypefojyqd", model.name()); + Assertions.assertEquals("u", model.containerId()); Assertions.assertEquals(false, model.ready()); - Assertions.assertEquals(false, model.started()); - Assertions.assertEquals(1297052787, model.restartCount()); + Assertions.assertEquals(true, model.started()); + Assertions.assertEquals(1554312221, model.restartCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReplicaContainer model = - new ReplicaContainer() - .withName("cgpik") - .withContainerId("imejzanl") - .withReady(false) - .withStarted(false) - .withRestartCount(1297052787); + ReplicaContainer model = new ReplicaContainer().withName("isofieypefojyqd").withContainerId("u") + .withReady(false).withStarted(true).withRestartCount(1554312221); model = BinaryData.fromObject(model).toObject(ReplicaContainer.class); - Assertions.assertEquals("cgpik", model.name()); - Assertions.assertEquals("imejzanl", model.containerId()); + Assertions.assertEquals("isofieypefojyqd", model.name()); + Assertions.assertEquals("u", model.containerId()); Assertions.assertEquals(false, model.ready()); - Assertions.assertEquals(false, model.started()); - Assertions.assertEquals(1297052787, model.restartCount()); + Assertions.assertEquals(true, model.started()); + Assertions.assertEquals(1554312221, model.restartCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaInnerTests.java index eec5323681c29..55bccfb6fde7f 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaInnerTests.java @@ -13,67 +13,38 @@ public final class ReplicaInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ReplicaInner model = - BinaryData - .fromString( - "{\"properties\":{\"createdTime\":\"2021-07-06T19:00:01Z\",\"runningState\":\"Running\",\"runningStateDetails\":\"ux\",\"containers\":[{\"name\":\"ldsyuuximerqfob\",\"containerId\":\"znkbykutwpfhpagm\",\"ready\":true,\"started\":true,\"restartCount\":803916344,\"runningState\":\"Running\",\"runningStateDetails\":\"doakgtdlmkkzevdl\",\"logStreamEndpoint\":\"wpusdsttwvogv\",\"execEndpoint\":\"ejdcngqqmoakuf\"}],\"initContainers\":[{\"name\":\"rwr\",\"containerId\":\"rtwaenuuzko\",\"ready\":false,\"started\":false,\"restartCount\":2010767026,\"runningState\":\"Waiting\",\"runningStateDetails\":\"yuhhziu\",\"logStreamEndpoint\":\"fozbhdmsmlmzqhof\",\"execEndpoint\":\"maequiahxicslfa\"},{\"name\":\"z\",\"containerId\":\"yylhalnswhccsp\",\"ready\":true,\"started\":true,\"restartCount\":2011273129,\"runningState\":\"Terminated\",\"runningStateDetails\":\"cywuggwol\",\"logStreamEndpoint\":\"czbwemhairsbr\",\"execEndpoint\":\"dwmsweypqwd\"},{\"name\":\"gicccnxqhuex\",\"containerId\":\"ttlstvlzywemhz\",\"ready\":true,\"started\":true,\"restartCount\":1411475506,\"runningState\":\"Terminated\",\"runningStateDetails\":\"iypbsfgytgusl\",\"logStreamEndpoint\":\"adcy\",\"execEndpoint\":\"ukyhejhzis\"}]},\"id\":\"gfpelolppvksrpqv\",\"name\":\"jzraehtwdwrf\",\"type\":\"swibyr\"}") - .toObject(ReplicaInner.class); - Assertions.assertEquals("ldsyuuximerqfob", model.containers().get(0).name()); - Assertions.assertEquals("znkbykutwpfhpagm", model.containers().get(0).containerId()); - Assertions.assertEquals(true, model.containers().get(0).ready()); - Assertions.assertEquals(true, model.containers().get(0).started()); - Assertions.assertEquals(803916344, model.containers().get(0).restartCount()); - Assertions.assertEquals("rwr", model.initContainers().get(0).name()); - Assertions.assertEquals("rtwaenuuzko", model.initContainers().get(0).containerId()); + ReplicaInner model = BinaryData.fromString( + "{\"properties\":{\"createdTime\":\"2021-08-12T09:55:05Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"iglaecx\",\"containers\":[{\"name\":\"cokpv\",\"containerId\":\"lqtmldgxob\",\"ready\":false,\"started\":false,\"restartCount\":563418466,\"runningState\":\"Waiting\",\"runningStateDetails\":\"ayzri\",\"logStreamEndpoint\":\"hya\",\"execEndpoint\":\"vjlboxqvk\"}],\"initContainers\":[{\"name\":\"ho\",\"containerId\":\"ynhdwdigum\",\"ready\":false,\"started\":true,\"restartCount\":745752113,\"runningState\":\"Waiting\",\"runningStateDetails\":\"jazysdzhezwwvaiq\",\"logStreamEndpoint\":\"vv\",\"execEndpoint\":\"nk\"}]},\"id\":\"hqyikvy\",\"name\":\"auy\",\"type\":\"vluwmncsttij\"}") + .toObject(ReplicaInner.class); + Assertions.assertEquals("cokpv", model.containers().get(0).name()); + Assertions.assertEquals("lqtmldgxob", model.containers().get(0).containerId()); + Assertions.assertEquals(false, model.containers().get(0).ready()); + Assertions.assertEquals(false, model.containers().get(0).started()); + Assertions.assertEquals(563418466, model.containers().get(0).restartCount()); + Assertions.assertEquals("ho", model.initContainers().get(0).name()); + Assertions.assertEquals("ynhdwdigum", model.initContainers().get(0).containerId()); Assertions.assertEquals(false, model.initContainers().get(0).ready()); - Assertions.assertEquals(false, model.initContainers().get(0).started()); - Assertions.assertEquals(2010767026, model.initContainers().get(0).restartCount()); + Assertions.assertEquals(true, model.initContainers().get(0).started()); + Assertions.assertEquals(745752113, model.initContainers().get(0).restartCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReplicaInner model = - new ReplicaInner() - .withContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("ldsyuuximerqfob") - .withContainerId("znkbykutwpfhpagm") - .withReady(true) - .withStarted(true) - .withRestartCount(803916344))) - .withInitContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("rwr") - .withContainerId("rtwaenuuzko") - .withReady(false) - .withStarted(false) - .withRestartCount(2010767026), - new ReplicaContainer() - .withName("z") - .withContainerId("yylhalnswhccsp") - .withReady(true) - .withStarted(true) - .withRestartCount(2011273129), - new ReplicaContainer() - .withName("gicccnxqhuex") - .withContainerId("ttlstvlzywemhz") - .withReady(true) - .withStarted(true) - .withRestartCount(1411475506))); + ReplicaInner model = new ReplicaInner() + .withContainers(Arrays.asList(new ReplicaContainer().withName("cokpv").withContainerId("lqtmldgxob") + .withReady(false).withStarted(false).withRestartCount(563418466))) + .withInitContainers(Arrays.asList(new ReplicaContainer().withName("ho").withContainerId("ynhdwdigum") + .withReady(false).withStarted(true).withRestartCount(745752113))); model = BinaryData.fromObject(model).toObject(ReplicaInner.class); - Assertions.assertEquals("ldsyuuximerqfob", model.containers().get(0).name()); - Assertions.assertEquals("znkbykutwpfhpagm", model.containers().get(0).containerId()); - Assertions.assertEquals(true, model.containers().get(0).ready()); - Assertions.assertEquals(true, model.containers().get(0).started()); - Assertions.assertEquals(803916344, model.containers().get(0).restartCount()); - Assertions.assertEquals("rwr", model.initContainers().get(0).name()); - Assertions.assertEquals("rtwaenuuzko", model.initContainers().get(0).containerId()); + Assertions.assertEquals("cokpv", model.containers().get(0).name()); + Assertions.assertEquals("lqtmldgxob", model.containers().get(0).containerId()); + Assertions.assertEquals(false, model.containers().get(0).ready()); + Assertions.assertEquals(false, model.containers().get(0).started()); + Assertions.assertEquals(563418466, model.containers().get(0).restartCount()); + Assertions.assertEquals("ho", model.initContainers().get(0).name()); + Assertions.assertEquals("ynhdwdigum", model.initContainers().get(0).containerId()); Assertions.assertEquals(false, model.initContainers().get(0).ready()); - Assertions.assertEquals(false, model.initContainers().get(0).started()); - Assertions.assertEquals(2010767026, model.initContainers().get(0).restartCount()); + Assertions.assertEquals(true, model.initContainers().get(0).started()); + Assertions.assertEquals(745752113, model.initContainers().get(0).restartCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaPropertiesTests.java index a6fd7ba8b76f7..38f70a8c368ae 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaPropertiesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ReplicaPropertiesTests.java @@ -13,73 +13,48 @@ public final class ReplicaPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ReplicaProperties model = - BinaryData - .fromString( - "{\"createdTime\":\"2021-01-18T20:33:29Z\",\"runningState\":\"NotRunning\",\"runningStateDetails\":\"hfwpracstwit\",\"containers\":[{\"name\":\"vxccedcp\",\"containerId\":\"dyodnwzxltj\",\"ready\":true,\"started\":true,\"restartCount\":823307226,\"runningState\":\"Waiting\",\"runningStateDetails\":\"xnavvwxq\",\"logStreamEndpoint\":\"y\",\"execEndpoint\":\"nyowxwlmdjrkvfg\"},{\"name\":\"fvpdbo\",\"containerId\":\"cizsjqlhkrribdei\",\"ready\":true,\"started\":false,\"restartCount\":1566726508,\"runningState\":\"Terminated\",\"runningStateDetails\":\"ndzwmkrefa\",\"logStreamEndpoint\":\"jorwkqnyhgbij\",\"execEndpoint\":\"ivfxzsjabibsyst\"}],\"initContainers\":[{\"name\":\"djpvkvpbjxbkz\",\"containerId\":\"kd\",\"ready\":true,\"started\":false,\"restartCount\":1067469550,\"runningState\":\"Running\",\"runningStateDetails\":\"gkakmokzhjjklff\",\"logStreamEndpoint\":\"ouw\",\"execEndpoint\":\"gzrf\"},{\"name\":\"eyebizikayuhql\",\"containerId\":\"bs\",\"ready\":true,\"started\":false,\"restartCount\":1463319790,\"runningState\":\"Waiting\",\"runningStateDetails\":\"gmfpgvmp\",\"logStreamEndpoint\":\"as\",\"execEndpoint\":\"haq\"},{\"name\":\"ss\",\"containerId\":\"u\",\"ready\":false,\"started\":false,\"restartCount\":213351747,\"runningState\":\"Terminated\",\"runningStateDetails\":\"rhneuyowq\",\"logStreamEndpoint\":\"wyt\",\"execEndpoint\":\"ib\"}]}") - .toObject(ReplicaProperties.class); - Assertions.assertEquals("vxccedcp", model.containers().get(0).name()); - Assertions.assertEquals("dyodnwzxltj", model.containers().get(0).containerId()); - Assertions.assertEquals(true, model.containers().get(0).ready()); - Assertions.assertEquals(true, model.containers().get(0).started()); - Assertions.assertEquals(823307226, model.containers().get(0).restartCount()); - Assertions.assertEquals("djpvkvpbjxbkz", model.initContainers().get(0).name()); - Assertions.assertEquals("kd", model.initContainers().get(0).containerId()); + ReplicaProperties model = BinaryData.fromString( + "{\"createdTime\":\"2021-04-05T16:52:36Z\",\"runningState\":\"Unknown\",\"runningStateDetails\":\"ekrsgs\",\"containers\":[{\"name\":\"uzqgnjdgkynsc\",\"containerId\":\"qhzvhxnkomt\",\"ready\":false,\"started\":false,\"restartCount\":1449469351,\"runningState\":\"Running\",\"runningStateDetails\":\"xz\",\"logStreamEndpoint\":\"ihfrbbcevqa\",\"execEndpoint\":\"ltd\"},{\"name\":\"fkqojpy\",\"containerId\":\"gtrd\",\"ready\":false,\"started\":false,\"restartCount\":2043504503,\"runningState\":\"Terminated\",\"runningStateDetails\":\"m\",\"logStreamEndpoint\":\"nysuxmprafwgckh\",\"execEndpoint\":\"xvd\"},{\"name\":\"fwafqrouda\",\"containerId\":\"avehhrvkbunzo\",\"ready\":true,\"started\":true,\"restartCount\":2097383102,\"runningState\":\"Terminated\",\"runningStateDetails\":\"yxcdyuibhmfdnbzy\",\"logStreamEndpoint\":\"f\",\"execEndpoint\":\"cjnaeoisrvhmgor\"}],\"initContainers\":[{\"name\":\"is\",\"containerId\":\"w\",\"ready\":true,\"started\":false,\"restartCount\":157711526,\"runningState\":\"Waiting\",\"runningStateDetails\":\"vxilcbt\",\"logStreamEndpoint\":\"hnze\",\"execEndpoint\":\"xtjjfzqlqhycav\"},{\"name\":\"ggxdb\",\"containerId\":\"smieknlra\",\"ready\":true,\"started\":true,\"restartCount\":1541326221,\"runningState\":\"Waiting\",\"runningStateDetails\":\"dwqf\",\"logStreamEndpoint\":\"lyr\",\"execEndpoint\":\"iagtc\"},{\"name\":\"ocqwogfnzjvus\",\"containerId\":\"ld\",\"ready\":true,\"started\":false,\"restartCount\":1126537021,\"runningState\":\"Waiting\",\"runningStateDetails\":\"tkad\",\"logStreamEndpoint\":\"s\",\"execEndpoint\":\"nbtgkbugrjqctoj\"}]}") + .toObject(ReplicaProperties.class); + Assertions.assertEquals("uzqgnjdgkynsc", model.containers().get(0).name()); + Assertions.assertEquals("qhzvhxnkomt", model.containers().get(0).containerId()); + Assertions.assertEquals(false, model.containers().get(0).ready()); + Assertions.assertEquals(false, model.containers().get(0).started()); + Assertions.assertEquals(1449469351, model.containers().get(0).restartCount()); + Assertions.assertEquals("is", model.initContainers().get(0).name()); + Assertions.assertEquals("w", model.initContainers().get(0).containerId()); Assertions.assertEquals(true, model.initContainers().get(0).ready()); Assertions.assertEquals(false, model.initContainers().get(0).started()); - Assertions.assertEquals(1067469550, model.initContainers().get(0).restartCount()); + Assertions.assertEquals(157711526, model.initContainers().get(0).restartCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReplicaProperties model = - new ReplicaProperties() - .withContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("vxccedcp") - .withContainerId("dyodnwzxltj") - .withReady(true) - .withStarted(true) - .withRestartCount(823307226), - new ReplicaContainer() - .withName("fvpdbo") - .withContainerId("cizsjqlhkrribdei") - .withReady(true) - .withStarted(false) - .withRestartCount(1566726508))) - .withInitContainers( - Arrays - .asList( - new ReplicaContainer() - .withName("djpvkvpbjxbkz") - .withContainerId("kd") - .withReady(true) - .withStarted(false) - .withRestartCount(1067469550), - new ReplicaContainer() - .withName("eyebizikayuhql") - .withContainerId("bs") - .withReady(true) - .withStarted(false) - .withRestartCount(1463319790), - new ReplicaContainer() - .withName("ss") - .withContainerId("u") - .withReady(false) - .withStarted(false) - .withRestartCount(213351747))); + ReplicaProperties model = new ReplicaProperties() + .withContainers(Arrays.asList( + new ReplicaContainer().withName("uzqgnjdgkynsc").withContainerId("qhzvhxnkomt").withReady(false) + .withStarted(false).withRestartCount(1449469351), + new ReplicaContainer().withName("fkqojpy").withContainerId("gtrd").withReady(false).withStarted(false) + .withRestartCount(2043504503), + new ReplicaContainer().withName("fwafqrouda").withContainerId("avehhrvkbunzo").withReady(true) + .withStarted(true).withRestartCount(2097383102))) + .withInitContainers(Arrays.asList( + new ReplicaContainer().withName("is").withContainerId("w").withReady(true).withStarted(false) + .withRestartCount(157711526), + new ReplicaContainer().withName("ggxdb").withContainerId("smieknlra").withReady(true).withStarted(true) + .withRestartCount(1541326221), + new ReplicaContainer().withName("ocqwogfnzjvus").withContainerId("ld").withReady(true) + .withStarted(false).withRestartCount(1126537021))); model = BinaryData.fromObject(model).toObject(ReplicaProperties.class); - Assertions.assertEquals("vxccedcp", model.containers().get(0).name()); - Assertions.assertEquals("dyodnwzxltj", model.containers().get(0).containerId()); - Assertions.assertEquals(true, model.containers().get(0).ready()); - Assertions.assertEquals(true, model.containers().get(0).started()); - Assertions.assertEquals(823307226, model.containers().get(0).restartCount()); - Assertions.assertEquals("djpvkvpbjxbkz", model.initContainers().get(0).name()); - Assertions.assertEquals("kd", model.initContainers().get(0).containerId()); + Assertions.assertEquals("uzqgnjdgkynsc", model.containers().get(0).name()); + Assertions.assertEquals("qhzvhxnkomt", model.containers().get(0).containerId()); + Assertions.assertEquals(false, model.containers().get(0).ready()); + Assertions.assertEquals(false, model.containers().get(0).started()); + Assertions.assertEquals(1449469351, model.containers().get(0).restartCount()); + Assertions.assertEquals("is", model.initContainers().get(0).name()); + Assertions.assertEquals("w", model.initContainers().get(0).containerId()); Assertions.assertEquals(true, model.initContainers().get(0).ready()); Assertions.assertEquals(false, model.initContainers().get(0).started()); - Assertions.assertEquals(1067469550, model.initContainers().get(0).restartCount()); + Assertions.assertEquals(157711526, model.initContainers().get(0).restartCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersGetCustomDomainVerificationIdWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersGetCustomDomainVerificationIdWithResponseMockTests.java new file mode 100644 index 0000000000000..d9d0c35ff1ee0 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersGetCustomDomainVerificationIdWithResponseMockTests.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class ResourceProvidersGetCustomDomainVerificationIdWithResponseMockTests { + @Test + public void testGetCustomDomainVerificationIdWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "\"zolgjzmicuydocc\""; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + String response = manager.resourceProviders() + .getCustomDomainVerificationIdWithResponse(com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("zolgjzmicuydocc", response); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersJobExecutionWithResponseMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersJobExecutionWithResponseMockTests.java index 8d4df9e9d1eab..ca30f71ae269c 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersJobExecutionWithResponseMockTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ResourceProvidersJobExecutionWithResponseMockTests.java @@ -30,57 +30,40 @@ public void testJobExecutionWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"name\":\"ymvqdbpbhfckdvez\",\"id\":\"cssbzhddu\",\"type\":\"nqfblhkalehpava\",\"status\":\"Running\",\"startTime\":\"2021-07-31T19:35:54Z\",\"endTime\":\"2021-02-25T01:40:45Z\",\"template\":{\"containers\":[{\"image\":\"gdmin\",\"name\":\"tteajohiygs\",\"command\":[\"onhpcz\",\"kmktpvwxqcseh\",\"hkhufm\",\"qumq\"],\"args\":[\"ydzulodsaeuzan\"],\"env\":[{},{},{}],\"resources\":{\"cpu\":51.25538357259788,\"memory\":\"wphpzfngqj\",\"ephemeralStorage\":\"idftujwjjufwbe\"}},{\"image\":\"k\",\"name\":\"rhtssr\",\"command\":[\"mdvhazcvjy\",\"iqswbqer\"],\"args\":[\"iytxt\",\"gukvlbpkt\",\"dstyouam\"],\"env\":[{},{},{}],\"resources\":{\"cpu\":89.09465324456659,\"memory\":\"owegmmutey\",\"ephemeralStorage\":\"yguqigijiitns\"}},{\"image\":\"lz\",\"name\":\"sygrijwa\",\"command\":[\"anra\",\"bfu\",\"qfrojsydgrhyd\"],\"args\":[\"ywezskiecafyg\",\"m\",\"ieqvds\"],\"env\":[{},{},{},{}],\"resources\":{\"cpu\":9.705369980968115,\"memory\":\"ah\",\"ephemeralStorage\":\"xalybxawoijpo\"}},{\"image\":\"blxpkkwjdjodq\",\"name\":\"kincnremeh\",\"command\":[\"zhceumoqodk\",\"dppyibngqlady\",\"rxwhydtlu\",\"vadswzs\"],\"args\":[\"emlowuowhl\",\"lnwy\",\"mouvbl\",\"mo\"],\"env\":[{},{}],\"resources\":{\"cpu\":85.24180497890332,\"memory\":\"wtdvrfmvli\",\"ephemeralStorage\":\"vjdrqcrjidhftuk\"}}],\"initContainers\":[{\"image\":\"lwyojbfqzdkfnj\",\"name\":\"xhafratqx\",\"command\":[\"roumzznva\",\"qjrhuzgfxonj\"],\"args\":[\"sllywpvtio\",\"zbpdbo\",\"lgry\",\"qiuasigrows\"],\"env\":[{},{}],\"resources\":{\"cpu\":56.766450145521155,\"memory\":\"djboqgrmtq\",\"ephemeralStorage\":\"qevadrmmw\"}},{\"image\":\"awvcmjzkxiid\",\"name\":\"czskoswoqiqazu\",\"command\":[\"xzkrrcoi\",\"sbamnppcc\"],\"args\":[\"ztdsbeza\"],\"env\":[{}],\"resources\":{\"cpu\":87.2973677793008,\"memory\":\"izhyhnep\",\"ephemeralStorage\":\"etiarxqiubxdukec\"}},{\"image\":\"dazvdhctm\",\"name\":\"os\",\"command\":[\"blnsntrpcaqk\",\"ofkbtf\",\"hklbnldpvcbh\"],\"args\":[\"yq\"],\"env\":[{},{},{}],\"resources\":{\"cpu\":15.430525461055334,\"memory\":\"rp\",\"ephemeralStorage\":\"r\"}}]}}"; + String responseStr + = "{\"name\":\"iutg\",\"id\":\"mkahpqha\",\"type\":\"ntacih\",\"properties\":{\"status\":\"Unknown\",\"startTime\":\"2021-02-15T03:50:34Z\",\"endTime\":\"2021-11-24T08:50:55Z\",\"template\":{\"containers\":[{\"image\":\"mvlbhikeaqgrv\",\"name\":\"omxp\",\"command\":[\"tsdfjyieso\",\"wiqbuoutm\"],\"args\":[\"gleofjsbgbw\",\"zvdajfwnncf\",\"aciq\"],\"env\":[{},{},{},{}],\"resources\":{}}],\"initContainers\":[{\"image\":\"qlwixvtbougu\",\"name\":\"n\",\"command\":[\"gjttbasualapdlnd\"],\"args\":[\"qb\",\"ixv\",\"lwynpbbfqvzfj\",\"spugzfeuzjljmph\"],\"env\":[{},{},{}],\"resources\":{}}]}}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ContainerAppsApiManager manager = - ContainerAppsApiManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - JobExecution response = - manager - .resourceProviders() - .jobExecutionWithResponse("ndzfqivjreuyk", "bmnwa", "ltbxoeeonqlnfw", com.azure.core.util.Context.NONE) - .getValue(); + JobExecution response = manager.resourceProviders() + .jobExecutionWithResponse("gkqzkcyzmffngdyf", "ixrhl", "qvhoej", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("ymvqdbpbhfckdvez", response.name()); - Assertions.assertEquals("cssbzhddu", response.id()); - Assertions.assertEquals("nqfblhkalehpava", response.type()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-31T19:35:54Z"), response.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-02-25T01:40:45Z"), response.endTime()); - Assertions.assertEquals("gdmin", response.template().containers().get(0).image()); - Assertions.assertEquals("tteajohiygs", response.template().containers().get(0).name()); - Assertions.assertEquals("onhpcz", response.template().containers().get(0).command().get(0)); - Assertions.assertEquals("ydzulodsaeuzan", response.template().containers().get(0).args().get(0)); - Assertions.assertEquals(51.25538357259788D, response.template().containers().get(0).resources().cpu()); - Assertions.assertEquals("wphpzfngqj", response.template().containers().get(0).resources().memory()); - Assertions.assertEquals("lwyojbfqzdkfnj", response.template().initContainers().get(0).image()); - Assertions.assertEquals("xhafratqx", response.template().initContainers().get(0).name()); - Assertions.assertEquals("roumzznva", response.template().initContainers().get(0).command().get(0)); - Assertions.assertEquals("sllywpvtio", response.template().initContainers().get(0).args().get(0)); - Assertions.assertEquals(56.766450145521155D, response.template().initContainers().get(0).resources().cpu()); - Assertions.assertEquals("djboqgrmtq", response.template().initContainers().get(0).resources().memory()); + Assertions.assertEquals("iutg", response.name()); + Assertions.assertEquals("mkahpqha", response.id()); + Assertions.assertEquals("ntacih", response.type()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-15T03:50:34Z"), response.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-24T08:50:55Z"), response.endTime()); + Assertions.assertEquals("mvlbhikeaqgrv", response.template().containers().get(0).image()); + Assertions.assertEquals("omxp", response.template().containers().get(0).name()); + Assertions.assertEquals("tsdfjyieso", response.template().containers().get(0).command().get(0)); + Assertions.assertEquals("gleofjsbgbw", response.template().containers().get(0).args().get(0)); + Assertions.assertEquals("qlwixvtbougu", response.template().initContainers().get(0).image()); + Assertions.assertEquals("n", response.template().initContainers().get(0).name()); + Assertions.assertEquals("gjttbasualapdlnd", response.template().initContainers().get(0).command().get(0)); + Assertions.assertEquals("qb", response.template().initContainers().get(0).args().get(0)); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/RevisionCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/RevisionCollectionTests.java index bb4e8bc6c920c..6bd5dcca4d05a 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/RevisionCollectionTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/RevisionCollectionTests.java @@ -12,17 +12,15 @@ public final class RevisionCollectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - RevisionCollection model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"createdTime\":\"2021-02-22T10:12:12Z\",\"lastActiveTime\":\"2021-10-17T14:02:40Z\",\"fqdn\":\"vxwc\",\"template\":{\"revisionSuffix\":\"pcirelsfeaen\",\"terminationGracePeriodSeconds\":6089881890484118319,\"initContainers\":[{}],\"containers\":[{},{},{}],\"scale\":{\"minReplicas\":1993086714,\"maxReplicas\":863801878,\"rules\":[{},{}]},\"volumes\":[{},{},{},{}],\"serviceBinds\":[{}]},\"active\":false,\"replicas\":2046993019,\"trafficWeight\":443965325,\"provisioningError\":\"youlp\",\"healthState\":\"None\",\"provisioningState\":\"Deprovisioning\",\"runningState\":\"Stopped\"},\"id\":\"rvimjwosytxitcsk\",\"name\":\"cktqumiekkezzi\",\"type\":\"hlyfjhdgqgg\"},{\"properties\":{\"createdTime\":\"2021-11-21T10:43:47Z\",\"lastActiveTime\":\"2021-04-09T06:20:49Z\",\"fqdn\":\"aeqidbqfatpxll\",\"template\":{\"revisionSuffix\":\"yjmoadsu\",\"terminationGracePeriodSeconds\":1027634010219173638,\"initContainers\":[{},{},{},{}],\"containers\":[{},{}],\"scale\":{\"minReplicas\":1651100345,\"maxReplicas\":1483853053,\"rules\":[{},{}]},\"volumes\":[{},{},{}],\"serviceBinds\":[{},{}]},\"active\":true,\"replicas\":288329779,\"trafficWeight\":1115314881,\"provisioningError\":\"uhpkxkgymar\",\"healthState\":\"Unhealthy\",\"provisioningState\":\"Deprovisioning\",\"runningState\":\"Processing\"},\"id\":\"ugjhky\",\"name\":\"ubeddg\",\"type\":\"sofwqmzqalkrmnji\"}],\"nextLink\":\"xacqqudfnbyx\"}") - .toObject(RevisionCollection.class); + RevisionCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"createdTime\":\"2021-08-08T16:05:05Z\",\"lastActiveTime\":\"2021-11-22T09:46:17Z\",\"fqdn\":\"rjfut\",\"template\":{\"revisionSuffix\":\"ebjvewzcjzn\",\"terminationGracePeriodSeconds\":7534826156141275141,\"initContainers\":[{},{},{},{}],\"containers\":[{},{},{},{}],\"scale\":{\"minReplicas\":1437675806,\"maxReplicas\":1356605173,\"rules\":[{},{},{}]},\"volumes\":[{},{}],\"serviceBinds\":[{},{},{},{}]},\"active\":false,\"replicas\":240161872,\"trafficWeight\":279681197,\"provisioningError\":\"ziuxxpshnee\",\"healthState\":\"Healthy\",\"provisioningState\":\"Provisioning\",\"runningState\":\"Failed\"},\"id\":\"ubkwdle\",\"name\":\"rds\",\"type\":\"tujbazpju\"},{\"properties\":{\"createdTime\":\"2021-04-13T05:15:14Z\",\"lastActiveTime\":\"2021-04-06T17:16:51Z\",\"fqdn\":\"lnorwmdu\",\"template\":{\"revisionSuffix\":\"klvxwmyg\",\"terminationGracePeriodSeconds\":6535644343996923938,\"initContainers\":[{},{}],\"containers\":[{},{},{},{}],\"scale\":{\"minReplicas\":1044665061,\"maxReplicas\":1565134020,\"rules\":[{}]},\"volumes\":[{},{},{},{}],\"serviceBinds\":[{},{}]},\"active\":false,\"replicas\":1689782344,\"trafficWeight\":1959517954,\"provisioningError\":\"xconfozauors\",\"healthState\":\"Healthy\",\"provisioningState\":\"Provisioning\",\"runningState\":\"Degraded\"},\"id\":\"lhlv\",\"name\":\"uuepzlrphwzsoldw\",\"type\":\"yuqdu\"},{\"properties\":{\"createdTime\":\"2020-12-22T02:15:52Z\",\"lastActiveTime\":\"2021-05-15T23:33:47Z\",\"fqdn\":\"rbior\",\"template\":{\"revisionSuffix\":\"lywjhh\",\"terminationGracePeriodSeconds\":6322848214394065275,\"initContainers\":[{}],\"containers\":[{},{},{}],\"scale\":{\"minReplicas\":759547890,\"maxReplicas\":601340793,\"rules\":[{}]},\"volumes\":[{},{},{}],\"serviceBinds\":[{},{}]},\"active\":false,\"replicas\":75081100,\"trafficWeight\":133601107,\"provisioningError\":\"euzaof\",\"healthState\":\"Healthy\",\"provisioningState\":\"Failed\",\"runningState\":\"Stopped\"},\"id\":\"s\",\"name\":\"gdotcubiipuipwo\",\"type\":\"onmacjekniz\"}],\"nextLink\":\"qvci\"}") + .toObject(RevisionCollection.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RevisionCollection model = - new RevisionCollection().withValue(Arrays.asList(new RevisionInner(), new RevisionInner())); + RevisionCollection model = new RevisionCollection() + .withValue(Arrays.asList(new RevisionInner(), new RevisionInner(), new RevisionInner())); model = BinaryData.fromObject(model).toObject(RevisionCollection.class); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceBindTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceBindTests.java deleted file mode 100644 index e1b616b9d96c2..0000000000000 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceBindTests.java +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.appcontainers.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.appcontainers.models.ServiceBind; -import org.junit.jupiter.api.Assertions; - -public final class ServiceBindTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - ServiceBind model = - BinaryData.fromString("{\"serviceId\":\"mqg\",\"name\":\"queziky\"}").toObject(ServiceBind.class); - Assertions.assertEquals("mqg", model.serviceId()); - Assertions.assertEquals("queziky", model.name()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - ServiceBind model = new ServiceBind().withServiceId("mqg").withName("queziky"); - model = BinaryData.fromObject(model).toObject(ServiceBind.class); - Assertions.assertEquals("mqg", model.serviceId()); - Assertions.assertEquals("queziky", model.name()); - } -} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceTests.java index 8bafb8e5381bb..be1ebe2aee332 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/ServiceTests.java @@ -11,14 +11,14 @@ public final class ServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Service model = BinaryData.fromString("{\"type\":\"bexilzznfqqnv\"}").toObject(Service.class); - Assertions.assertEquals("bexilzznfqqnv", model.type()); + Service model = BinaryData.fromString("{\"type\":\"yqiatkzwp\"}").toObject(Service.class); + Assertions.assertEquals("yqiatkzwp", model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Service model = new Service().withType("bexilzznfqqnv"); + Service model = new Service().withType("yqiatkzwp"); model = BinaryData.fromObject(model).toObject(Service.class); - Assertions.assertEquals("bexilzznfqqnv", model.type()); + Assertions.assertEquals("yqiatkzwp", model.type()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpConnectionPoolTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpConnectionPoolTests.java new file mode 100644 index 0000000000000..525f953e5e6e7 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpConnectionPoolTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.TcpConnectionPool; +import org.junit.jupiter.api.Assertions; + +public final class TcpConnectionPoolTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + TcpConnectionPool model + = BinaryData.fromString("{\"maxConnections\":171596164}").toObject(TcpConnectionPool.class); + Assertions.assertEquals(171596164, model.maxConnections()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + TcpConnectionPool model = new TcpConnectionPool().withMaxConnections(171596164); + model = BinaryData.fromObject(model).toObject(TcpConnectionPool.class); + Assertions.assertEquals(171596164, model.maxConnections()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpRetryPolicyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpRetryPolicyTests.java new file mode 100644 index 0000000000000..aa0e1f0e20192 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TcpRetryPolicyTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.TcpRetryPolicy; +import org.junit.jupiter.api.Assertions; + +public final class TcpRetryPolicyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + TcpRetryPolicy model + = BinaryData.fromString("{\"maxConnectAttempts\":56034680}").toObject(TcpRetryPolicy.class); + Assertions.assertEquals(56034680, model.maxConnectAttempts()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + TcpRetryPolicy model = new TcpRetryPolicy().withMaxConnectAttempts(56034680); + model = BinaryData.fromObject(model).toObject(TcpRetryPolicy.class); + Assertions.assertEquals(56034680, model.maxConnectAttempts()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TimeoutPolicyTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TimeoutPolicyTests.java new file mode 100644 index 0000000000000..22186455269fb --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TimeoutPolicyTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.TimeoutPolicy; +import org.junit.jupiter.api.Assertions; + +public final class TimeoutPolicyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + TimeoutPolicy model = BinaryData + .fromString("{\"responseTimeoutInSeconds\":2068674986,\"connectionTimeoutInSeconds\":974822937}") + .toObject(TimeoutPolicy.class); + Assertions.assertEquals(2068674986, model.responseTimeoutInSeconds()); + Assertions.assertEquals(974822937, model.connectionTimeoutInSeconds()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + TimeoutPolicy model + = new TimeoutPolicy().withResponseTimeoutInSeconds(2068674986).withConnectionTimeoutInSeconds(974822937); + model = BinaryData.fromObject(model).toObject(TimeoutPolicy.class); + Assertions.assertEquals(2068674986, model.responseTimeoutInSeconds()); + Assertions.assertEquals(974822937, model.connectionTimeoutInSeconds()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TracesConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TracesConfigurationTests.java new file mode 100644 index 0000000000000..8d85d6589f163 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TracesConfigurationTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.TracesConfiguration; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class TracesConfigurationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + TracesConfiguration model + = BinaryData.fromString("{\"destinations\":[\"b\",\"kbwvqvxkdiv\"]}").toObject(TracesConfiguration.class); + Assertions.assertEquals("b", model.destinations().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + TracesConfiguration model = new TracesConfiguration().withDestinations(Arrays.asList("b", "kbwvqvxkdiv")); + model = BinaryData.fromObject(model).toObject(TracesConfiguration.class); + Assertions.assertEquals("b", model.destinations().get(0)); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TrafficWeightTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TrafficWeightTests.java index 6fb2e500cde44..df9886348ed7e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TrafficWeightTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/TrafficWeightTests.java @@ -11,29 +11,23 @@ public final class TrafficWeightTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TrafficWeight model = - BinaryData - .fromString( - "{\"revisionName\":\"trmgucnapkte\",\"weight\":1950925712,\"latestRevision\":false,\"label\":\"tfdygpfqb\"}") - .toObject(TrafficWeight.class); - Assertions.assertEquals("trmgucnapkte", model.revisionName()); - Assertions.assertEquals(1950925712, model.weight()); + TrafficWeight model = BinaryData.fromString( + "{\"revisionName\":\"ipaslthaqfxssmwu\",\"weight\":1363759065,\"latestRevision\":false,\"label\":\"ezpdrhneu\"}") + .toObject(TrafficWeight.class); + Assertions.assertEquals("ipaslthaqfxssmwu", model.revisionName()); + Assertions.assertEquals(1363759065, model.weight()); Assertions.assertEquals(false, model.latestRevision()); - Assertions.assertEquals("tfdygpfqb", model.label()); + Assertions.assertEquals("ezpdrhneu", model.label()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TrafficWeight model = - new TrafficWeight() - .withRevisionName("trmgucnapkte") - .withWeight(1950925712) - .withLatestRevision(false) - .withLabel("tfdygpfqb"); + TrafficWeight model = new TrafficWeight().withRevisionName("ipaslthaqfxssmwu").withWeight(1363759065) + .withLatestRevision(false).withLabel("ezpdrhneu"); model = BinaryData.fromObject(model).toObject(TrafficWeight.class); - Assertions.assertEquals("trmgucnapkte", model.revisionName()); - Assertions.assertEquals(1950925712, model.weight()); + Assertions.assertEquals("ipaslthaqfxssmwu", model.revisionName()); + Assertions.assertEquals(1363759065, model.weight()); Assertions.assertEquals(false, model.latestRevision()); - Assertions.assertEquals("tfdygpfqb", model.label()); + Assertions.assertEquals("ezpdrhneu", model.label()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageInnerTests.java new file mode 100644 index 0000000000000..250fabac5c5c1 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageInnerTests.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.fluent.models.UsageInner; +import com.azure.resourcemanager.appcontainers.models.UsageName; +import org.junit.jupiter.api.Assertions; + +public final class UsageInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UsageInner model = BinaryData.fromString( + "{\"currentValue\":41.70743,\"limit\":12.357182,\"name\":{\"value\":\"aswlp\",\"localizedValue\":\"gm\"}}") + .toObject(UsageInner.class); + Assertions.assertEquals(41.70743f, model.currentValue()); + Assertions.assertEquals(12.357182f, model.limit()); + Assertions.assertEquals("aswlp", model.name().value()); + Assertions.assertEquals("gm", model.name().localizedValue()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UsageInner model = new UsageInner().withCurrentValue(41.70743f).withLimit(12.357182f) + .withName(new UsageName().withValue("aswlp").withLocalizedValue("gm")); + model = BinaryData.fromObject(model).toObject(UsageInner.class); + Assertions.assertEquals(41.70743f, model.currentValue()); + Assertions.assertEquals(12.357182f, model.limit()); + Assertions.assertEquals("aswlp", model.name().value()); + Assertions.assertEquals("gm", model.name().localizedValue()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageNameTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageNameTests.java new file mode 100644 index 0000000000000..5ee591e9b9146 --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsageNameTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.appcontainers.models.UsageName; +import org.junit.jupiter.api.Assertions; + +public final class UsageNameTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UsageName model = BinaryData.fromString("{\"value\":\"fjlrxwtoauk\",\"localizedValue\":\"kvci\"}") + .toObject(UsageName.class); + Assertions.assertEquals("fjlrxwtoauk", model.value()); + Assertions.assertEquals("kvci", model.localizedValue()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UsageName model = new UsageName().withValue("fjlrxwtoauk").withLocalizedValue("kvci"); + model = BinaryData.fromObject(model).toObject(UsageName.class); + Assertions.assertEquals("fjlrxwtoauk", model.value()); + Assertions.assertEquals("kvci", model.localizedValue()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsagesListMockTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsagesListMockTests.java new file mode 100644 index 0000000000000..be338c6aa16ae --- /dev/null +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UsagesListMockTests.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.appcontainers.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.appcontainers.ContainerAppsApiManager; +import com.azure.resourcemanager.appcontainers.models.Usage; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class UsagesListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"currentValue\":38.23954,\"limit\":32.95426,\"name\":{\"value\":\"zmuhbcak\",\"localizedValue\":\"hokhoitwhrjsdmm\"}}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ContainerAppsApiManager manager = ContainerAppsApiManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.usages().list("dyhi", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(38.23954f, response.iterator().next().currentValue()); + Assertions.assertEquals(32.95426f, response.iterator().next().limit()); + Assertions.assertEquals("zmuhbcak", response.iterator().next().name().value()); + Assertions.assertEquals("hokhoitwhrjsdmm", response.iterator().next().name().localizedValue()); + } +} diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UserAssignedIdentityTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UserAssignedIdentityTests.java index fae761209d08c..5da71dfdc5129 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UserAssignedIdentityTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/UserAssignedIdentityTests.java @@ -10,11 +10,9 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UserAssignedIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"d7bc9138-ce8b-4d64-b68f-6892f3278db3\",\"clientId\":\"8b444edc-73a4-4717-972d-bc7f7edae1b7\"}") - .toObject(UserAssignedIdentity.class); + UserAssignedIdentity model = BinaryData.fromString( + "{\"principalId\":\"be8f935b-e984-46cf-8757-d39bf7c7af37\",\"clientId\":\"92f0cebe-72c5-4e0f-b455-000fbc84bb04\"}") + .toObject(UserAssignedIdentity.class); } @org.junit.jupiter.api.Test diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VnetConfigurationTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VnetConfigurationTests.java index 4f1c3fb06af6a..12418ae3f3650 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VnetConfigurationTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VnetConfigurationTests.java @@ -11,32 +11,25 @@ public final class VnetConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VnetConfiguration model = - BinaryData - .fromString( - "{\"internal\":true,\"infrastructureSubnetId\":\"lma\",\"dockerBridgeCidr\":\"tcyohpfkyrk\",\"platformReservedCidr\":\"dg\",\"platformReservedDnsIP\":\"gsj\"}") - .toObject(VnetConfiguration.class); + VnetConfiguration model = BinaryData.fromString( + "{\"internal\":true,\"infrastructureSubnetId\":\"n\",\"dockerBridgeCidr\":\"ujeickp\",\"platformReservedCidr\":\"cpopmxel\",\"platformReservedDnsIP\":\"clt\"}") + .toObject(VnetConfiguration.class); Assertions.assertEquals(true, model.internal()); - Assertions.assertEquals("lma", model.infrastructureSubnetId()); - Assertions.assertEquals("tcyohpfkyrk", model.dockerBridgeCidr()); - Assertions.assertEquals("dg", model.platformReservedCidr()); - Assertions.assertEquals("gsj", model.platformReservedDnsIp()); + Assertions.assertEquals("n", model.infrastructureSubnetId()); + Assertions.assertEquals("ujeickp", model.dockerBridgeCidr()); + Assertions.assertEquals("cpopmxel", model.platformReservedCidr()); + Assertions.assertEquals("clt", model.platformReservedDnsIp()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VnetConfiguration model = - new VnetConfiguration() - .withInternal(true) - .withInfrastructureSubnetId("lma") - .withDockerBridgeCidr("tcyohpfkyrk") - .withPlatformReservedCidr("dg") - .withPlatformReservedDnsIp("gsj"); + VnetConfiguration model = new VnetConfiguration().withInternal(true).withInfrastructureSubnetId("n") + .withDockerBridgeCidr("ujeickp").withPlatformReservedCidr("cpopmxel").withPlatformReservedDnsIp("clt"); model = BinaryData.fromObject(model).toObject(VnetConfiguration.class); Assertions.assertEquals(true, model.internal()); - Assertions.assertEquals("lma", model.infrastructureSubnetId()); - Assertions.assertEquals("tcyohpfkyrk", model.dockerBridgeCidr()); - Assertions.assertEquals("dg", model.platformReservedCidr()); - Assertions.assertEquals("gsj", model.platformReservedDnsIp()); + Assertions.assertEquals("n", model.infrastructureSubnetId()); + Assertions.assertEquals("ujeickp", model.dockerBridgeCidr()); + Assertions.assertEquals("cpopmxel", model.platformReservedCidr()); + Assertions.assertEquals("clt", model.platformReservedDnsIp()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VolumeMountTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VolumeMountTests.java index 7406025c692c2..abae444926e6d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VolumeMountTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/VolumeMountTests.java @@ -11,22 +11,21 @@ public final class VolumeMountTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VolumeMount model = - BinaryData - .fromString("{\"volumeName\":\"bttdumorppxe\",\"mountPath\":\"nzbtbhj\",\"subPath\":\"lkfg\"}") - .toObject(VolumeMount.class); - Assertions.assertEquals("bttdumorppxe", model.volumeName()); - Assertions.assertEquals("nzbtbhj", model.mountPath()); - Assertions.assertEquals("lkfg", model.subPath()); + VolumeMount model = BinaryData + .fromString("{\"volumeName\":\"svfycxzbfv\",\"mountPath\":\"wvrvmtg\",\"subPath\":\"ppyostronzmyhgf\"}") + .toObject(VolumeMount.class); + Assertions.assertEquals("svfycxzbfv", model.volumeName()); + Assertions.assertEquals("wvrvmtg", model.mountPath()); + Assertions.assertEquals("ppyostronzmyhgf", model.subPath()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeMount model = - new VolumeMount().withVolumeName("bttdumorppxe").withMountPath("nzbtbhj").withSubPath("lkfg"); + VolumeMount model + = new VolumeMount().withVolumeName("svfycxzbfv").withMountPath("wvrvmtg").withSubPath("ppyostronzmyhgf"); model = BinaryData.fromObject(model).toObject(VolumeMount.class); - Assertions.assertEquals("bttdumorppxe", model.volumeName()); - Assertions.assertEquals("nzbtbhj", model.mountPath()); - Assertions.assertEquals("lkfg", model.subPath()); + Assertions.assertEquals("svfycxzbfv", model.volumeName()); + Assertions.assertEquals("wvrvmtg", model.mountPath()); + Assertions.assertEquals("ppyostronzmyhgf", model.subPath()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesCollectionTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesCollectionTests.java index fa8900b458c86..f10ba612b21b4 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesCollectionTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesCollectionTests.java @@ -14,38 +14,22 @@ public final class WorkloadProfileStatesCollectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkloadProfileStatesCollection model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"minimumCount\":1486999564,\"maximumCount\":2052287318,\"currentCount\":426763582},\"id\":\"uo\",\"name\":\"xprimrsop\",\"type\":\"eecjmeis\"},{\"properties\":{\"minimumCount\":749085962,\"maximumCount\":1672946681,\"currentCount\":2018313749},\"id\":\"wxdzaumweoohgu\",\"name\":\"fuzboyjathwtzolb\",\"type\":\"emwmdxmebwjs\"}],\"nextLink\":\"p\"}") - .toObject(WorkloadProfileStatesCollection.class); - Assertions.assertEquals(1486999564, model.value().get(0).properties().minimumCount()); - Assertions.assertEquals(2052287318, model.value().get(0).properties().maximumCount()); - Assertions.assertEquals(426763582, model.value().get(0).properties().currentCount()); + WorkloadProfileStatesCollection model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"minimumCount\":792139303,\"maximumCount\":1099107522,\"currentCount\":381107449},\"id\":\"yrneizjcpeo\",\"name\":\"khnmgbrou\",\"type\":\"ddbhf\"}],\"nextLink\":\"fpazjzoywjxhpd\"}") + .toObject(WorkloadProfileStatesCollection.class); + Assertions.assertEquals(792139303, model.value().get(0).properties().minimumCount()); + Assertions.assertEquals(1099107522, model.value().get(0).properties().maximumCount()); + Assertions.assertEquals(381107449, model.value().get(0).properties().currentCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WorkloadProfileStatesCollection model = - new WorkloadProfileStatesCollection() - .withValue( - Arrays - .asList( - new WorkloadProfileStatesInner() - .withProperties( - new WorkloadProfileStatesProperties() - .withMinimumCount(1486999564) - .withMaximumCount(2052287318) - .withCurrentCount(426763582)), - new WorkloadProfileStatesInner() - .withProperties( - new WorkloadProfileStatesProperties() - .withMinimumCount(749085962) - .withMaximumCount(1672946681) - .withCurrentCount(2018313749)))); + WorkloadProfileStatesCollection model = new WorkloadProfileStatesCollection().withValue( + Arrays.asList(new WorkloadProfileStatesInner().withProperties(new WorkloadProfileStatesProperties() + .withMinimumCount(792139303).withMaximumCount(1099107522).withCurrentCount(381107449)))); model = BinaryData.fromObject(model).toObject(WorkloadProfileStatesCollection.class); - Assertions.assertEquals(1486999564, model.value().get(0).properties().minimumCount()); - Assertions.assertEquals(2052287318, model.value().get(0).properties().maximumCount()); - Assertions.assertEquals(426763582, model.value().get(0).properties().currentCount()); + Assertions.assertEquals(792139303, model.value().get(0).properties().minimumCount()); + Assertions.assertEquals(1099107522, model.value().get(0).properties().maximumCount()); + Assertions.assertEquals(381107449, model.value().get(0).properties().currentCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesInnerTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesInnerTests.java index 11c3cb2319eca..59f33bc2b768e 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesInnerTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesInnerTests.java @@ -12,28 +12,22 @@ public final class WorkloadProfileStatesInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkloadProfileStatesInner model = - BinaryData - .fromString( - "{\"properties\":{\"minimumCount\":1591085258,\"maximumCount\":2130021286,\"currentCount\":1715283989},\"id\":\"f\",\"name\":\"xnmwmqtibxyijddt\",\"type\":\"qcttadijaeukmrsi\"}") - .toObject(WorkloadProfileStatesInner.class); - Assertions.assertEquals(1591085258, model.properties().minimumCount()); - Assertions.assertEquals(2130021286, model.properties().maximumCount()); - Assertions.assertEquals(1715283989, model.properties().currentCount()); + WorkloadProfileStatesInner model = BinaryData.fromString( + "{\"properties\":{\"minimumCount\":1181873760,\"maximumCount\":796854695,\"currentCount\":314959325},\"id\":\"qwtehtuevr\",\"name\":\"rljyoogwxhnsdu\",\"type\":\"gwbsreurf\"}") + .toObject(WorkloadProfileStatesInner.class); + Assertions.assertEquals(1181873760, model.properties().minimumCount()); + Assertions.assertEquals(796854695, model.properties().maximumCount()); + Assertions.assertEquals(314959325, model.properties().currentCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WorkloadProfileStatesInner model = - new WorkloadProfileStatesInner() - .withProperties( - new WorkloadProfileStatesProperties() - .withMinimumCount(1591085258) - .withMaximumCount(2130021286) - .withCurrentCount(1715283989)); + WorkloadProfileStatesInner model + = new WorkloadProfileStatesInner().withProperties(new WorkloadProfileStatesProperties() + .withMinimumCount(1181873760).withMaximumCount(796854695).withCurrentCount(314959325)); model = BinaryData.fromObject(model).toObject(WorkloadProfileStatesInner.class); - Assertions.assertEquals(1591085258, model.properties().minimumCount()); - Assertions.assertEquals(2130021286, model.properties().maximumCount()); - Assertions.assertEquals(1715283989, model.properties().currentCount()); + Assertions.assertEquals(1181873760, model.properties().minimumCount()); + Assertions.assertEquals(796854695, model.properties().maximumCount()); + Assertions.assertEquals(314959325, model.properties().currentCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesPropertiesTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesPropertiesTests.java index 1630f9f1def3a..224a756c1c66d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesPropertiesTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileStatesPropertiesTests.java @@ -11,25 +11,21 @@ public final class WorkloadProfileStatesPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkloadProfileStatesProperties model = - BinaryData - .fromString("{\"minimumCount\":647971020,\"maximumCount\":513622258,\"currentCount\":1321937950}") - .toObject(WorkloadProfileStatesProperties.class); - Assertions.assertEquals(647971020, model.minimumCount()); - Assertions.assertEquals(513622258, model.maximumCount()); - Assertions.assertEquals(1321937950, model.currentCount()); + WorkloadProfileStatesProperties model = BinaryData + .fromString("{\"minimumCount\":669436660,\"maximumCount\":2008330964,\"currentCount\":571851575}") + .toObject(WorkloadProfileStatesProperties.class); + Assertions.assertEquals(669436660, model.minimumCount()); + Assertions.assertEquals(2008330964, model.maximumCount()); + Assertions.assertEquals(571851575, model.currentCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WorkloadProfileStatesProperties model = - new WorkloadProfileStatesProperties() - .withMinimumCount(647971020) - .withMaximumCount(513622258) - .withCurrentCount(1321937950); + WorkloadProfileStatesProperties model = new WorkloadProfileStatesProperties().withMinimumCount(669436660) + .withMaximumCount(2008330964).withCurrentCount(571851575); model = BinaryData.fromObject(model).toObject(WorkloadProfileStatesProperties.class); - Assertions.assertEquals(647971020, model.minimumCount()); - Assertions.assertEquals(513622258, model.maximumCount()); - Assertions.assertEquals(1321937950, model.currentCount()); + Assertions.assertEquals(669436660, model.minimumCount()); + Assertions.assertEquals(2008330964, model.maximumCount()); + Assertions.assertEquals(571851575, model.currentCount()); } } diff --git a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileTests.java b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileTests.java index e93e8f410ca6a..6e315d4788a6d 100644 --- a/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileTests.java +++ b/sdk/appcontainers/azure-resourcemanager-appcontainers/src/test/java/com/azure/resourcemanager/appcontainers/generated/WorkloadProfileTests.java @@ -11,29 +11,23 @@ public final class WorkloadProfileTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkloadProfile model = - BinaryData - .fromString( - "{\"name\":\"rdnhfuku\",\"workloadProfileType\":\"sjcswsmystuluqyp\",\"minimumCount\":1783589840,\"maximumCount\":291266426}") - .toObject(WorkloadProfile.class); - Assertions.assertEquals("rdnhfuku", model.name()); - Assertions.assertEquals("sjcswsmystuluqyp", model.workloadProfileType()); - Assertions.assertEquals(1783589840, model.minimumCount()); - Assertions.assertEquals(291266426, model.maximumCount()); + WorkloadProfile model = BinaryData.fromString( + "{\"name\":\"igkxkbsazga\",\"workloadProfileType\":\"gacyrcmjdmspo\",\"minimumCount\":361382788,\"maximumCount\":283706298}") + .toObject(WorkloadProfile.class); + Assertions.assertEquals("igkxkbsazga", model.name()); + Assertions.assertEquals("gacyrcmjdmspo", model.workloadProfileType()); + Assertions.assertEquals(361382788, model.minimumCount()); + Assertions.assertEquals(283706298, model.maximumCount()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WorkloadProfile model = - new WorkloadProfile() - .withName("rdnhfuku") - .withWorkloadProfileType("sjcswsmystuluqyp") - .withMinimumCount(1783589840) - .withMaximumCount(291266426); + WorkloadProfile model = new WorkloadProfile().withName("igkxkbsazga").withWorkloadProfileType("gacyrcmjdmspo") + .withMinimumCount(361382788).withMaximumCount(283706298); model = BinaryData.fromObject(model).toObject(WorkloadProfile.class); - Assertions.assertEquals("rdnhfuku", model.name()); - Assertions.assertEquals("sjcswsmystuluqyp", model.workloadProfileType()); - Assertions.assertEquals(1783589840, model.minimumCount()); - Assertions.assertEquals(291266426, model.maximumCount()); + Assertions.assertEquals("igkxkbsazga", model.name()); + Assertions.assertEquals("gacyrcmjdmspo", model.workloadProfileType()); + Assertions.assertEquals(361382788, model.minimumCount()); + Assertions.assertEquals(283706298, model.maximumCount()); } } diff --git a/sdk/applicationinsights/azure-resourcemanager-applicationinsights/pom.xml b/sdk/applicationinsights/azure-resourcemanager-applicationinsights/pom.xml index 68f79687f54e8..4d3caf8938ad0 100644 --- a/sdk/applicationinsights/azure-resourcemanager-applicationinsights/pom.xml +++ b/sdk/applicationinsights/azure-resourcemanager-applicationinsights/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/applicationinsights/tests.mgmt.yml b/sdk/applicationinsights/tests.mgmt.yml index f9ba29025dd7a..71ac5d2ee43de 100644 --- a/sdk/applicationinsights/tests.mgmt.yml +++ b/sdk/applicationinsights/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: applicationinsights - Artifacts: - - name: azure-resourcemanager-applicationinsights - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerapplicationinsights - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: applicationinsights + Artifacts: + - name: azure-resourcemanager-applicationinsights + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerapplicationinsights + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/astro/azure-resourcemanager-astro/pom.xml b/sdk/astro/azure-resourcemanager-astro/pom.xml index 47243d312865f..10d85fb5afb15 100644 --- a/sdk/astro/azure-resourcemanager-astro/pom.xml +++ b/sdk/astro/azure-resourcemanager-astro/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/attestation/azure-resourcemanager-attestation/pom.xml b/sdk/attestation/azure-resourcemanager-attestation/pom.xml index 8fcf14d0ddf13..221be185c5a79 100644 --- a/sdk/attestation/azure-resourcemanager-attestation/pom.xml +++ b/sdk/attestation/azure-resourcemanager-attestation/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/attestation/azure-security-attestation/CHANGELOG.md b/sdk/attestation/azure-security-attestation/CHANGELOG.md index 8f52314c567ae..aae32de21d50f 100644 --- a/sdk/attestation/azure-security-attestation/CHANGELOG.md +++ b/sdk/attestation/azure-security-attestation/CHANGELOG.md @@ -13,6 +13,17 @@ ### Other Changes + +## 1.1.22 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `nimbus-jose-jwt` from `9.31` to version `9.37.3`. + + ## 1.1.21 (2024-02-20) ### Other Changes diff --git a/sdk/attestation/azure-security-attestation/pom.xml b/sdk/attestation/azure-security-attestation/pom.xml index 838a080479a0b..8b57ecce91ebc 100644 --- a/sdk/attestation/azure-security-attestation/pom.xml +++ b/sdk/attestation/azure-security-attestation/pom.xml @@ -77,19 +77,19 @@ io.opentelemetry opentelemetry-api - 1.35.0 + 1.36.0 test io.opentelemetry opentelemetry-exporter-logging - 1.35.0 + 1.36.0 test io.opentelemetry opentelemetry-sdk - 1.35.0 + 1.36.0 test @@ -119,7 +119,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -149,9 +149,9 @@ com.nimbusds:nimbus-jose-jwt:[9.37.3] - io.opentelemetry:opentelemetry-api:[1.35.0] - io.opentelemetry:opentelemetry-sdk:[1.35.0] - io.opentelemetry:opentelemetry-exporter-logging:[1.35.0] + io.opentelemetry:opentelemetry-api:[1.36.0] + io.opentelemetry:opentelemetry-sdk:[1.36.0] + io.opentelemetry:opentelemetry-exporter-logging:[1.36.0] diff --git a/sdk/attestation/tests.yml b/sdk/attestation/tests.yml index 64a6092ddfea0..af31f22cbdd03 100644 --- a/sdk/attestation/tests.yml +++ b/sdk/attestation/tests.yml @@ -1,15 +1,15 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: attestation - Location: westus - Artifacts: - - name: azure-security-attestation - groupId: com.azure - safeName: azuresecurityattestation - EnvVars: - AZURE_CLIENT_ID: $(ATTESTATION_CLIENT_ID) - AZURE_TENANT_ID: $(ATTESTATION_TENANT_ID) - AZURE_CLIENT_SECRET: $(ATTESTATION_CLIENT_SECRET) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: attestation + Location: westus + Artifacts: + - name: azure-security-attestation + groupId: com.azure + safeName: azuresecurityattestation + EnvVars: + AZURE_CLIENT_ID: $(ATTESTATION_CLIENT_ID) + AZURE_TENANT_ID: $(ATTESTATION_TENANT_ID) + AZURE_CLIENT_SECRET: $(ATTESTATION_CLIENT_SECRET) diff --git a/sdk/avs/azure-resourcemanager-avs/pom.xml b/sdk/avs/azure-resourcemanager-avs/pom.xml index a111f454c9046..0f8d434fb0df1 100644 --- a/sdk/avs/azure-resourcemanager-avs/pom.xml +++ b/sdk/avs/azure-resourcemanager-avs/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/avs/tests.mgmt.yml b/sdk/avs/tests.mgmt.yml index 4b43a5c5355e5..3ee04cbe35ab5 100644 --- a/sdk/avs/tests.mgmt.yml +++ b/sdk/avs/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: avs - Artifacts: - - name: azure-resourcemanager-avs - groupId: com.azure.resourcemanager - safeName: azureresourcemanageravs - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: avs + Artifacts: + - name: azure-resourcemanager-avs + groupId: com.azure.resourcemanager + safeName: azureresourcemanageravs + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/azurearcdata/azure-resourcemanager-azurearcdata/pom.xml b/sdk/azurearcdata/azure-resourcemanager-azurearcdata/pom.xml index f1ef639c6b347..d0519fb2d26d3 100644 --- a/sdk/azurearcdata/azure-resourcemanager-azurearcdata/pom.xml +++ b/sdk/azurearcdata/azure-resourcemanager-azurearcdata/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/azurestack/azure-resourcemanager-azurestack/pom.xml b/sdk/azurestack/azure-resourcemanager-azurestack/pom.xml index 6030777647d2a..c0a50598d2c48 100644 --- a/sdk/azurestack/azure-resourcemanager-azurestack/pom.xml +++ b/sdk/azurestack/azure-resourcemanager-azurestack/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/azurestackhci/azure-resourcemanager-azurestackhci/pom.xml b/sdk/azurestackhci/azure-resourcemanager-azurestackhci/pom.xml index 0d8f26557b11e..bcc8626514762 100644 --- a/sdk/azurestackhci/azure-resourcemanager-azurestackhci/pom.xml +++ b/sdk/azurestackhci/azure-resourcemanager-azurestackhci/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/baremetalinfrastructure/azure-resourcemanager-baremetalinfrastructure/pom.xml b/sdk/baremetalinfrastructure/azure-resourcemanager-baremetalinfrastructure/pom.xml index 4ab5cedcae825..86f7f9dec1a45 100644 --- a/sdk/baremetalinfrastructure/azure-resourcemanager-baremetalinfrastructure/pom.xml +++ b/sdk/baremetalinfrastructure/azure-resourcemanager-baremetalinfrastructure/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/batch/azure-resourcemanager-batch/pom.xml b/sdk/batch/azure-resourcemanager-batch/pom.xml index 5f655b94e3789..1dd3584e5f069 100644 --- a/sdk/batch/azure-resourcemanager-batch/pom.xml +++ b/sdk/batch/azure-resourcemanager-batch/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 test diff --git a/sdk/batch/tests.mgmt.yml b/sdk/batch/tests.mgmt.yml index 52caa2d85686b..0d88557f9d2a4 100644 --- a/sdk/batch/tests.mgmt.yml +++ b/sdk/batch/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: batch - Artifacts: - - name: azure-resourcemanager-batch - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerbatch - Clouds: 'Public' - # Batch account has regional quota limit on test subscription. Only run tests on Windows to avoid exceeding. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: batch + Artifacts: + - name: azure-resourcemanager-batch + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerbatch + Clouds: 'Public' + # Batch account has regional quota limit on test subscription. Only run tests on Windows to avoid exceeding. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/batch/tests.yml b/sdk/batch/tests.yml index 73449e3fafc33..df11ea64671df 100644 --- a/sdk/batch/tests.yml +++ b/sdk/batch/tests.yml @@ -1,16 +1,16 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: batch - Artifacts: - - name: azure-batch - groupId: com.microsoft.azure - safeName: azurebatch - EnvVars: - AZURE_BATCH_ACCOUNT: $(java-batch-test-account-name) - AZURE_BATCH_ACCESS_KEY: $(java-batch-test-account-access-key) - STORAGE_ACCOUNT_KEY: $(java-batch-test-storage-account-key) - STORAGE_ACCOUNT_NAME: $(java-batch-test-storage-account-name) - AZURE_BATCH_ENDPOINT: https://servbatch.centralus.batch.azure.com +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: batch + Artifacts: + - name: azure-batch + groupId: com.microsoft.azure + safeName: azurebatch + EnvVars: + AZURE_BATCH_ACCOUNT: $(java-batch-test-account-name) + AZURE_BATCH_ACCESS_KEY: $(java-batch-test-account-access-key) + STORAGE_ACCOUNT_KEY: $(java-batch-test-storage-account-key) + STORAGE_ACCOUNT_NAME: $(java-batch-test-storage-account-name) + AZURE_BATCH_ENDPOINT: https://servbatch.centralus.batch.azure.com diff --git a/sdk/billing/azure-resourcemanager-billing/pom.xml b/sdk/billing/azure-resourcemanager-billing/pom.xml index d043ab4c3821c..6f43baa395fa3 100644 --- a/sdk/billing/azure-resourcemanager-billing/pom.xml +++ b/sdk/billing/azure-resourcemanager-billing/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/billingbenefits/azure-resourcemanager-billingbenefits/pom.xml b/sdk/billingbenefits/azure-resourcemanager-billingbenefits/pom.xml index 69654e67e6bf9..05cf47cf0e510 100644 --- a/sdk/billingbenefits/azure-resourcemanager-billingbenefits/pom.xml +++ b/sdk/billingbenefits/azure-resourcemanager-billingbenefits/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/boms/azure-sdk-bom/CHANGELOG.md b/sdk/boms/azure-sdk-bom/CHANGELOG.md index 9ecdb2fe35153..4b2e160f6c55f 100644 --- a/sdk/boms/azure-sdk-bom/CHANGELOG.md +++ b/sdk/boms/azure-sdk-bom/CHANGELOG.md @@ -1,5 +1,11 @@ # Release History +## 1.2.22 (2024-03-26) + +### Dependency Updates + +- Updated Azure SDK dependency versions to the latest releases. + ## 1.2.21 (2024-02-26) ### Dependency Updates diff --git a/sdk/boms/azure-sdk-bom/README.md b/sdk/boms/azure-sdk-bom/README.md index a1e43c88544a8..6c8d1ac8284fa 100644 --- a/sdk/boms/azure-sdk-bom/README.md +++ b/sdk/boms/azure-sdk-bom/README.md @@ -21,7 +21,7 @@ result in all dependencies being included in your project. com.azure azure-sdk-bom - 1.2.21 + 1.2.22 pom import diff --git a/sdk/boms/azure-sdk-bom/pom.xml b/sdk/boms/azure-sdk-bom/pom.xml index 56b11c20c485f..04dde96cf0467 100644 --- a/sdk/boms/azure-sdk-bom/pom.xml +++ b/sdk/boms/azure-sdk-bom/pom.xml @@ -38,167 +38,172 @@ com.azure azure-ai-contentsafety - 1.0.0 + 1.0.1 com.azure azure-ai-formrecognizer - 4.1.5 + 4.1.6 com.azure azure-ai-metricsadvisor - 1.1.22 + 1.1.23 com.azure azure-ai-textanalytics - 5.4.3 + 5.4.4 com.azure azure-communication-callautomation - 1.1.2 + 1.1.3 com.azure azure-communication-chat - 1.4.1 + 1.4.2 com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure azure-communication-email - 1.0.10 + 1.0.11 com.azure azure-communication-identity - 1.5.2 + 1.5.3 com.azure azure-communication-jobrouter - 1.1.1 + 1.1.2 + + + com.azure + azure-communication-messages + 1.0.1 com.azure azure-communication-phonenumbers - 1.1.10 + 1.1.11 com.azure azure-communication-rooms - 1.0.8 + 1.0.9 com.azure azure-communication-sms - 1.1.21 + 1.1.22 com.azure azure-containers-containerregistry - 1.2.5 + 1.2.6 com.azure azure-core - 1.46.0 + 1.47.0 com.azure azure-core-amqp - 2.9.1 + 2.9.2 com.azure azure-core-http-netty - 1.14.0 + 1.14.1 com.azure azure-core-http-okhttp - 1.11.18 + 1.11.19 com.azure azure-core-management - 1.11.10 + 1.12.0 com.azure azure-core-serializer-json-gson - 1.2.9 + 1.2.10 com.azure azure-core-serializer-json-jackson - 1.4.9 + 1.4.10 com.azure azure-cosmos - 4.56.0 + 4.57.0 com.azure azure-cosmos-encryption - 2.8.0 + 2.9.0 com.azure azure-data-appconfiguration - 1.5.2 + 1.5.3 com.azure azure-data-schemaregistry - 1.4.3 + 1.4.4 com.azure azure-data-schemaregistry-apacheavro - 1.1.14 + 1.1.15 com.azure azure-data-tables - 12.3.19 + 12.3.20 com.azure azure-developer-loadtesting - 1.0.10 + 1.0.11 com.azure azure-digitaltwins-core - 1.3.17 + 1.3.18 com.azure azure-identity - 1.11.2 + 1.11.4 com.azure azure-identity-broker - 1.0.2 + 1.0.4 com.azure azure-identity-extensions - 1.1.13 + 1.1.14 com.azure azure-iot-deviceupdate - 1.0.15 + 1.0.16 com.azure @@ -208,77 +213,77 @@ com.azure azure-messaging-eventgrid - 4.21.0 + 4.22.0 com.azure azure-messaging-eventhubs - 5.18.1 + 5.18.2 com.azure azure-messaging-eventhubs-checkpointstore-blob - 1.19.1 + 1.19.2 com.azure azure-messaging-servicebus - 7.15.1 + 7.15.2 com.azure azure-messaging-webpubsub - 1.2.12 + 1.2.13 com.azure azure-messaging-webpubsub-client - 1.0.0 + 1.0.1 com.azure azure-mixedreality-authentication - 1.2.21 + 1.2.22 com.azure azure-mixedreality-remoterendering - 1.1.26 + 1.1.27 com.azure azure-monitor-ingestion - 1.1.4 + 1.1.5 com.azure azure-monitor-query - 1.2.9 + 1.2.10 com.azure azure-search-documents - 11.6.2 + 11.6.3 com.azure azure-security-attestation - 1.1.21 + 1.1.22 com.azure azure-security-confidentialledger - 1.0.17 + 1.0.18 com.azure azure-security-keyvault-administration - 4.5.0 + 4.5.1 com.azure azure-security-keyvault-certificates - 4.6.0 + 4.6.1 com.azure @@ -288,52 +293,52 @@ com.azure azure-security-keyvault-keys - 4.8.0 + 4.8.1 com.azure azure-security-keyvault-secrets - 4.8.0 + 4.8.1 com.azure azure-storage-blob - 12.25.2 + 12.25.3 com.azure azure-storage-blob-batch - 12.21.2 + 12.21.3 com.azure azure-storage-blob-cryptography - 12.24.2 + 12.24.3 com.azure azure-storage-common - 12.24.2 + 12.24.3 com.azure azure-storage-file-datalake - 12.18.2 + 12.18.3 com.azure azure-storage-file-share - 12.21.2 + 12.21.3 com.azure azure-storage-internal-avro - 12.10.2 + 12.10.3 com.azure azure-storage-queue - 12.20.2 + 12.20.3 @@ -397,4 +402,4 @@ - \ No newline at end of file + diff --git a/sdk/boms/spring-cloud-azure-dependencies/pom.xml b/sdk/boms/spring-cloud-azure-dependencies/pom.xml index a1597675af494..a92707f2a6cd1 100644 --- a/sdk/boms/spring-cloud-azure-dependencies/pom.xml +++ b/sdk/boms/spring-cloud-azure-dependencies/pom.xml @@ -63,7 +63,7 @@ com.azure.resourcemanager azure-resourcemanager - 2.36.0 + 2.37.0 diff --git a/sdk/changeanalysis/azure-resourcemanager-changeanalysis/pom.xml b/sdk/changeanalysis/azure-resourcemanager-changeanalysis/pom.xml index 19ada632bb805..0feb8cc4ce4e8 100644 --- a/sdk/changeanalysis/azure-resourcemanager-changeanalysis/pom.xml +++ b/sdk/changeanalysis/azure-resourcemanager-changeanalysis/pom.xml @@ -55,7 +55,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/changeanalysis/tests.mgmt.yml b/sdk/changeanalysis/tests.mgmt.yml index 679bd822628ac..1fe15915cde10 100644 --- a/sdk/changeanalysis/tests.mgmt.yml +++ b/sdk/changeanalysis/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: changeanalysis - Artifacts: - - name: azure-resourcemanager-changeanalysis - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerchangeanalysis - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: changeanalysis + Artifacts: + - name: azure-resourcemanager-changeanalysis + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerchangeanalysis + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/chaos/azure-resourcemanager-chaos/CHANGELOG.md b/sdk/chaos/azure-resourcemanager-chaos/CHANGELOG.md index 9965d26918be8..ce03fbe3ff670 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/CHANGELOG.md +++ b/sdk/chaos/azure-resourcemanager-chaos/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.1 (Unreleased) +## 1.2.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,21 @@ ### Other Changes +## 1.1.0 (2024-03-15) + +- Azure Resource Manager Chaos client library for Java. This package contains Microsoft Azure SDK for Chaos Management SDK. Chaos Management Client. Package tag package-2024-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +#### `models.ExperimentUpdate` was modified + +* `tags()` was added +* `withTags(java.util.Map)` was added + +#### `models.Experiment$Update` was modified + +* `withTags(java.util.Map)` was added + ## 1.0.0 (2023-11-16) - Azure Resource Manager Chaos client library for Java. This package contains Microsoft Azure SDK for Chaos Management SDK. Chaos Management Client. Package tag package-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/chaos/azure-resourcemanager-chaos/README.md b/sdk/chaos/azure-resourcemanager-chaos/README.md index ca6b629cdb0c6..4aaf374a2027c 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/README.md +++ b/sdk/chaos/azure-resourcemanager-chaos/README.md @@ -2,7 +2,7 @@ Azure Resource Manager Chaos client library for Java. -This package contains Microsoft Azure SDK for Chaos Management SDK. Chaos Management Client. Package tag package-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for Chaos Management SDK. Chaos Management Client. Package tag package-2024-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-chaos - 1.0.0 + 1.1.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/chaos/azure-resourcemanager-chaos/SAMPLE.md b/sdk/chaos/azure-resourcemanager-chaos/SAMPLE.md index d0e54ab142e3b..fa5e6756ef177 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/SAMPLE.md +++ b/sdk/chaos/azure-resourcemanager-chaos/SAMPLE.md @@ -47,29 +47,24 @@ ```java import com.azure.resourcemanager.chaos.fluent.models.CapabilityInner; -/** Samples for Capabilities CreateOrUpdate. */ +/** + * Samples for Capabilities CreateOrUpdate. + */ public final class CapabilitiesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CreateUpdateCapability.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CreateUpdateCapability.json */ /** * Sample code: Create/update a Capability that extends a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ public static void createUpdateACapabilityThatExtendsAVirtualMachineTargetResource( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .createOrUpdateWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - "Shutdown-1.0", - new CapabilityInner(), - com.azure.core.util.Context.NONE); + manager.capabilities().createOrUpdateWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", + "exampleVM", "Microsoft-VirtualMachine", "Shutdown-1.0", new CapabilityInner(), + com.azure.core.util.Context.NONE); } } ``` @@ -77,28 +72,23 @@ public final class CapabilitiesCreateOrUpdateSamples { ### Capabilities_Delete ```java -/** Samples for Capabilities Delete. */ +/** + * Samples for Capabilities Delete. + */ public final class CapabilitiesDeleteSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DeleteCapability.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DeleteCapability.json */ /** * Sample code: Delete a Capability that extends a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ public static void deleteACapabilityThatExtendsAVirtualMachineTargetResource( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .deleteWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - "Shutdown-1.0", - com.azure.core.util.Context.NONE); + manager.capabilities().deleteWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-VirtualMachine", "Shutdown-1.0", com.azure.core.util.Context.NONE); } } ``` @@ -106,28 +96,23 @@ public final class CapabilitiesDeleteSamples { ### Capabilities_Get ```java -/** Samples for Capabilities Get. */ +/** + * Samples for Capabilities Get. + */ public final class CapabilitiesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetCapability.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetCapability.json */ /** * Sample code: Get a Capability that extends a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void getACapabilityThatExtendsAVirtualMachineTargetResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .getWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - "Shutdown-1.0", - com.azure.core.util.Context.NONE); + public static void + getACapabilityThatExtendsAVirtualMachineTargetResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.capabilities().getWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-VirtualMachine", "Shutdown-1.0", com.azure.core.util.Context.NONE); } } ``` @@ -135,28 +120,23 @@ public final class CapabilitiesGetSamples { ### Capabilities_List ```java -/** Samples for Capabilities List. */ +/** + * Samples for Capabilities List. + */ public final class CapabilitiesListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListCapabilities.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListCapabilities.json */ /** * Sample code: List all Capabilities that extend a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllCapabilitiesThatExtendAVirtualMachineTargetResource( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .list( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - null, - com.azure.core.util.Context.NONE); + manager.capabilities().list("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-VirtualMachine", null, com.azure.core.util.Context.NONE); } } ``` @@ -164,21 +144,23 @@ public final class CapabilitiesListSamples { ### CapabilityTypes_Get ```java -/** Samples for CapabilityTypes Get. */ +/** + * Samples for CapabilityTypes Get. + */ public final class CapabilityTypesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetCapabilityType.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetCapabilityType.json */ /** * Sample code: Get a Capability Type for a virtual machine Target resource on westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void getACapabilityTypeForAVirtualMachineTargetResourceOnWestus2Location( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilityTypes() - .getWithResponse("westus2", "Microsoft-VirtualMachine", "Shutdown-1.0", com.azure.core.util.Context.NONE); + manager.capabilityTypes().getWithResponse("westus2", "Microsoft-VirtualMachine", "Shutdown-1.0", + com.azure.core.util.Context.NONE); } } ``` @@ -186,14 +168,17 @@ public final class CapabilityTypesGetSamples { ### CapabilityTypes_List ```java -/** Samples for CapabilityTypes List. */ +/** + * Samples for CapabilityTypes List. + */ public final class CapabilityTypesListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListCapabilityTypes.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListCapabilityTypes.json */ /** * Sample code: List all Capability Types for a virtual machine Target resource on westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllCapabilityTypesForAVirtualMachineTargetResourceOnWestus2Location( @@ -206,14 +191,17 @@ public final class CapabilityTypesListSamples { ### Experiments_Cancel ```java -/** Samples for Experiments Cancel. */ +/** + * Samples for Experiments Cancel. + */ public final class ExperimentsCancelSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CancelExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CancelExperiment.json */ /** * Sample code: Cancel a running Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void cancelARunningExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -237,59 +225,35 @@ import com.azure.resourcemanager.chaos.models.TargetReferenceType; import java.time.Duration; import java.util.Arrays; -/** Samples for Experiments CreateOrUpdate. */ +/** + * Samples for Experiments CreateOrUpdate. + */ public final class ExperimentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CreateUpdateExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CreateUpdateExperiment.json */ /** * Sample code: Create/update a Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void createUpdateAExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .define("exampleExperiment") - .withRegion("eastus2euap") + manager.experiments().define("exampleExperiment").withRegion("eastus2euap") .withExistingResourceGroup("exampleRG") .withSteps( - Arrays - .asList( - new ChaosExperimentStep() - .withName("step1") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("branch1") - .withActions( - Arrays - .asList( - new ContinuousAction() - .withName("urn:csci:microsoft:virtualMachine:shutdown/1.0") - .withDuration(Duration.parse("PT10M")) - .withParameters( - Arrays - .asList( - new KeyValuePair() - .withKey("fakeTokenPlaceholder") - .withValue("false"))) - .withSelectorId("selector1"))))))) - .withSelectors( - Arrays - .asList( - new ChaosTargetListSelector() - .withId("selector1") - .withTargets( - Arrays - .asList( - new TargetReference() - .withType(TargetReferenceType.CHAOS_TARGET) - .withId( - "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.Compute/virtualMachines/exampleVM/providers/Microsoft.Chaos/targets/Microsoft-VirtualMachine"))))) - .withIdentity(new ResourceIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)) - .create(); + Arrays.asList(new ChaosExperimentStep().withName("step1") + .withBranches(Arrays.asList(new ChaosExperimentBranch().withName("branch1") + .withActions(Arrays.asList(new ContinuousAction() + .withName("urn:csci:microsoft:virtualMachine:shutdown/1.0") + .withDuration(Duration.parse("PT10M")) + .withParameters( + Arrays.asList(new KeyValuePair().withKey("fakeTokenPlaceholder").withValue("false"))) + .withSelectorId("selector1"))))))) + .withSelectors(Arrays.asList(new ChaosTargetListSelector().withId("selector1") + .withTargets(Arrays.asList(new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId( + "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.Compute/virtualMachines/exampleVM/providers/Microsoft.Chaos/targets/Microsoft-VirtualMachine"))))) + .withIdentity(new ResourceIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)).create(); } } ``` @@ -297,14 +261,17 @@ public final class ExperimentsCreateOrUpdateSamples { ### Experiments_Delete ```java -/** Samples for Experiments Delete. */ +/** + * Samples for Experiments Delete. + */ public final class ExperimentsDeleteSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DeleteExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DeleteExperiment.json */ /** * Sample code: Delete a Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void deleteAExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -316,24 +283,22 @@ public final class ExperimentsDeleteSamples { ### Experiments_ExecutionDetails ```java -/** Samples for Experiments ExecutionDetails. */ +/** + * Samples for Experiments ExecutionDetails. + */ public final class ExperimentsExecutionDetailsSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DetailsExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DetailsExperiment.json */ /** * Sample code: Get experiment execution details. - * + * * @param manager Entry point to ChaosManager. */ public static void getExperimentExecutionDetails(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .executionDetailsWithResponse( - "exampleRG", - "exampleExperiment", - "f24500ad-744e-4a26-864b-b76199eac333", - com.azure.core.util.Context.NONE); + manager.experiments().executionDetailsWithResponse("exampleRG", "exampleExperiment", + "f24500ad-744e-4a26-864b-b76199eac333", com.azure.core.util.Context.NONE); } } ``` @@ -341,20 +306,22 @@ public final class ExperimentsExecutionDetailsSamples { ### Experiments_GetByResourceGroup ```java -/** Samples for Experiments GetByResourceGroup. */ +/** + * Samples for Experiments GetByResourceGroup. + */ public final class ExperimentsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetExperiment.json */ /** * Sample code: Get a Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void getAExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .getByResourceGroupWithResponse("exampleRG", "exampleExperiment", com.azure.core.util.Context.NONE); + manager.experiments().getByResourceGroupWithResponse("exampleRG", "exampleExperiment", + com.azure.core.util.Context.NONE); } } ``` @@ -362,24 +329,22 @@ public final class ExperimentsGetByResourceGroupSamples { ### Experiments_GetExecution ```java -/** Samples for Experiments GetExecution. */ +/** + * Samples for Experiments GetExecution. + */ public final class ExperimentsGetExecutionSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetExperimentExecution.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetExperimentExecution.json */ /** * Sample code: Get the execution of a Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void getTheExecutionOfAExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .getExecutionWithResponse( - "exampleRG", - "exampleExperiment", - "f24500ad-744e-4a26-864b-b76199eac333", - com.azure.core.util.Context.NONE); + manager.experiments().getExecutionWithResponse("exampleRG", "exampleExperiment", + "f24500ad-744e-4a26-864b-b76199eac333", com.azure.core.util.Context.NONE); } } ``` @@ -387,14 +352,18 @@ public final class ExperimentsGetExecutionSamples { ### Experiments_List ```java -/** Samples for Experiments List. */ +/** + * Samples for Experiments List. + */ public final class ExperimentsListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListExperimentsInASubscription.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListExperimentsInASubscription. + * json */ /** * Sample code: List all Experiments in a subscription. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllExperimentsInASubscription(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -406,14 +375,17 @@ public final class ExperimentsListSamples { ### Experiments_ListAllExecutions ```java -/** Samples for Experiments ListAllExecutions. */ +/** + * Samples for Experiments ListAllExecutions. + */ public final class ExperimentsListAllExecutionsSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListExperimentExecutions.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListExperimentExecutions.json */ /** * Sample code: List all executions of an Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllExecutionsOfAnExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -425,14 +397,18 @@ public final class ExperimentsListAllExecutionsSamples { ### Experiments_ListByResourceGroup ```java -/** Samples for Experiments ListByResourceGroup. */ +/** + * Samples for Experiments ListByResourceGroup. + */ public final class ExperimentsListByResourceGroupSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListExperimentsInAResourceGroup.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListExperimentsInAResourceGroup. + * json */ /** * Sample code: List all Experiments in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllExperimentsInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -444,14 +420,17 @@ public final class ExperimentsListByResourceGroupSamples { ### Experiments_Start ```java -/** Samples for Experiments Start. */ +/** + * Samples for Experiments Start. + */ public final class ExperimentsStartSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/StartExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/StartExperiment.json */ /** * Sample code: Start a Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void startAExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -470,31 +449,27 @@ import com.azure.resourcemanager.chaos.models.UserAssignedIdentity; import java.util.HashMap; import java.util.Map; -/** Samples for Experiments Update. */ +/** + * Samples for Experiments Update. + */ public final class ExperimentsUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/UpdateExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/UpdateExperiment.json */ /** * Sample code: Update an Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void updateAnExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { - Experiment resource = - manager - .experiments() - .getByResourceGroupWithResponse("exampleRG", "exampleExperiment", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withIdentity( - new ResourceIdentity() - .withType(ResourceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.ManagedIdentity/userAssignedIdentity/exampleUMI", - new UserAssignedIdentity()))) + Experiment resource = manager.experiments() + .getByResourceGroupWithResponse("exampleRG", "exampleExperiment", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withTags(mapOf("key1", "fakeTokenPlaceholder", "key2", "fakeTokenPlaceholder")).withIdentity( + new ResourceIdentity().withType(ResourceIdentityType.USER_ASSIGNED).withUserAssignedIdentities(mapOf( + "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.ManagedIdentity/userAssignedIdentity/exampleUMI", + new UserAssignedIdentity()))) .apply(); } @@ -515,20 +490,22 @@ public final class ExperimentsUpdateSamples { ### OperationStatuses_Get ```java -/** Samples for OperationStatuses Get. */ +/** + * Samples for OperationStatuses Get. + */ public final class OperationStatusesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetOperationStatus.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetOperationStatus.json */ /** * Sample code: Get specific operation status. - * + * * @param manager Entry point to ChaosManager. */ public static void getSpecificOperationStatus(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .operationStatuses() - .getWithResponse("West US", "713192d7-503f-477a-9cfe-4efc3ee2bd11", com.azure.core.util.Context.NONE); + manager.operationStatuses().getWithResponse("West US", "713192d7-503f-477a-9cfe-4efc3ee2bd11", + com.azure.core.util.Context.NONE); } } ``` @@ -536,14 +513,17 @@ public final class OperationStatusesGetSamples { ### TargetTypes_Get ```java -/** Samples for TargetTypes Get. */ +/** + * Samples for TargetTypes Get. + */ public final class TargetTypesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetTargetType.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetTargetType.json */ /** * Sample code: Get a Target Type for westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void getATargetTypeForWestus2Location(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -555,14 +535,17 @@ public final class TargetTypesGetSamples { ### TargetTypes_List ```java -/** Samples for TargetTypes List. */ +/** + * Samples for TargetTypes List. + */ public final class TargetTypesListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListTargetTypes.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListTargetTypes.json */ /** * Sample code: List all Target Types for westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllTargetTypesForWestus2Location(com.azure.resourcemanager.chaos.ChaosManager manager) { @@ -581,37 +564,28 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; -/** Samples for Targets CreateOrUpdate. */ +/** + * Samples for Targets CreateOrUpdate. + */ public final class TargetsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CreateUpdateTarget.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CreateUpdateTarget.json */ /** * Sample code: Create/update a Target that extends a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ public static void createUpdateATargetThatExtendsAVirtualMachineResource( com.azure.resourcemanager.chaos.ChaosManager manager) throws IOException { - manager - .targets() - .createOrUpdateWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-Agent", - new TargetInner() - .withProperties( - mapOf( - "identities", - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "[{\"type\":\"CertificateSubjectIssuer\",\"subject\":\"CN=example.subject\"}]", - Object.class, - SerializerEncoding.JSON))), - com.azure.core.util.Context.NONE); + manager.targets().createOrUpdateWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-Agent", + new TargetInner().withProperties(mapOf("identities", + SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "[{\"type\":\"CertificateSubjectIssuer\",\"subject\":\"CN=example.subject\"}]", Object.class, + SerializerEncoding.JSON))), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -631,27 +605,23 @@ public final class TargetsCreateOrUpdateSamples { ### Targets_Delete ```java -/** Samples for Targets Delete. */ +/** + * Samples for Targets Delete. + */ public final class TargetsDeleteSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DeleteTarget.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DeleteTarget.json */ /** * Sample code: Delete a Target that extends a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void deleteATargetThatExtendsAVirtualMachineResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .targets() - .deleteWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-Agent", - com.azure.core.util.Context.NONE); + public static void + deleteATargetThatExtendsAVirtualMachineResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.targets().deleteWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-Agent", com.azure.core.util.Context.NONE); } } ``` @@ -659,27 +629,23 @@ public final class TargetsDeleteSamples { ### Targets_Get ```java -/** Samples for Targets Get. */ +/** + * Samples for Targets Get. + */ public final class TargetsGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetTarget.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetTarget.json */ /** * Sample code: Get a Target that extends a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void getATargetThatExtendsAVirtualMachineResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .targets() - .getWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-Agent", - com.azure.core.util.Context.NONE); + public static void + getATargetThatExtendsAVirtualMachineResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.targets().getWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-Agent", com.azure.core.util.Context.NONE); } } ``` @@ -687,27 +653,23 @@ public final class TargetsGetSamples { ### Targets_List ```java -/** Samples for Targets List. */ +/** + * Samples for Targets List. + */ public final class TargetsListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListTargets.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListTargets.json */ /** * Sample code: List all Targets that extend a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void listAllTargetsThatExtendAVirtualMachineResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .targets() - .list( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - null, - com.azure.core.util.Context.NONE); + public static void + listAllTargetsThatExtendAVirtualMachineResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.targets().list("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", null, + com.azure.core.util.Context.NONE); } } ``` diff --git a/sdk/chaos/azure-resourcemanager-chaos/pom.xml b/sdk/chaos/azure-resourcemanager-chaos/pom.xml index d0e05378b7a10..d61e4692f13c6 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/pom.xml +++ b/sdk/chaos/azure-resourcemanager-chaos/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-chaos - 1.1.0-beta.1 + 1.2.0-beta.1 jar Microsoft Azure SDK for Chaos Management - This package contains Microsoft Azure SDK for Chaos Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Chaos Management Client. Package tag package-2023-11. + This package contains Microsoft Azure SDK for Chaos Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Chaos Management Client. Package tag package-2024-01. https://github.com/Azure/azure-sdk-for-java @@ -66,19 +66,7 @@ com.azure azure-identity - 1.11.3 - test - - - com.azure.resourcemanager - azure-resourcemanager-resources - 2.36.0 - test - - - com.azure.resourcemanager - azure-resourcemanager-keyvault - 2.36.0 + 1.11.4 test @@ -99,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -119,5 +105,17 @@ 1.7.36 test + + com.azure.resourcemanager + azure-resourcemanager-resources + 2.37.0 + test + + + com.azure.resourcemanager + azure-resourcemanager-keyvault + 2.37.0 + test + diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/ChaosManager.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/ChaosManager.java index 753d6894f87de..07250547306cc 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/ChaosManager.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/ChaosManager.java @@ -46,7 +46,10 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to ChaosManager. Chaos Management Client. */ +/** + * Entry point to ChaosManager. + * Chaos Management Client. + */ public final class ChaosManager { private Capabilities capabilities; @@ -67,18 +70,14 @@ public final class ChaosManager { private ChaosManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new ChaosManagementClientBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new ChaosManagementClientBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of Chaos service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the Chaos service API instance. @@ -91,7 +90,7 @@ public static ChaosManager authenticate(TokenCredential credential, AzureProfile /** * Creates an instance of Chaos service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the Chaos service API instance. @@ -104,14 +103,16 @@ public static ChaosManager authenticate(HttpPipeline httpPipeline, AzureProfile /** * Gets a Configurable instance that can be used to create ChaosManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new ChaosManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -183,8 +184,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -201,8 +202,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -222,21 +223,12 @@ public ChaosManager authenticate(TokenCredential credential, AzureProfile profil Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.chaos") - .append("/") - .append("1.0.0"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.chaos").append("/") + .append("1.1.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -255,38 +247,25 @@ public ChaosManager authenticate(TokenCredential credential, AzureProfile profil policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new ChaosManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of Capabilities. - * + * * @return Resource collection API of Capabilities. */ public Capabilities capabilities() { @@ -298,7 +277,7 @@ public Capabilities capabilities() { /** * Gets the resource collection API of CapabilityTypes. - * + * * @return Resource collection API of CapabilityTypes. */ public CapabilityTypes capabilityTypes() { @@ -310,7 +289,7 @@ public CapabilityTypes capabilityTypes() { /** * Gets the resource collection API of Experiments. It manages Experiment. - * + * * @return Resource collection API of Experiments. */ public Experiments experiments() { @@ -322,7 +301,7 @@ public Experiments experiments() { /** * Gets the resource collection API of OperationStatuses. - * + * * @return Resource collection API of OperationStatuses. */ public OperationStatuses operationStatuses() { @@ -334,7 +313,7 @@ public OperationStatuses operationStatuses() { /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -346,7 +325,7 @@ public Operations operations() { /** * Gets the resource collection API of TargetTypes. - * + * * @return Resource collection API of TargetTypes. */ public TargetTypes targetTypes() { @@ -358,7 +337,7 @@ public TargetTypes targetTypes() { /** * Gets the resource collection API of Targets. - * + * * @return Resource collection API of Targets. */ public Targets targets() { @@ -371,7 +350,7 @@ public Targets targets() { /** * Gets wrapped service client ChaosManagementClient providing direct access to the underlying auto-generated API * implementation, based on Azure REST API. - * + * * @return Wrapped service client ChaosManagementClient. */ public ChaosManagementClient serviceClient() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilitiesClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilitiesClient.java index 9de7900898eea..711876aab678d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilitiesClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilitiesClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.CapabilityInner; -/** An instance of this class provides access to all the operations defined in CapabilitiesClient. */ +/** + * An instance of this class provides access to all the operations defined in CapabilitiesClient. + */ public interface CapabilitiesClient { /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -24,20 +26,16 @@ public interface CapabilitiesClient { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedIterable}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName); /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -48,22 +46,17 @@ PagedIterable list( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedIterable}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken, + PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String continuationToken, Context context); /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -77,18 +70,13 @@ PagedIterable list( * @return a Capability resource that extends a Target resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context); /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -101,17 +89,12 @@ Response getWithResponse( * @return a Capability resource that extends a Target resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - CapabilityInner get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName); + CapabilityInner get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName); /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -125,18 +108,13 @@ CapabilityInner get( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context); /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -148,17 +126,12 @@ Response deleteWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName); + void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName); /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -173,19 +146,13 @@ void delete( * @return model that represents a Capability resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, + CapabilityInner capability, Context context); /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -199,12 +166,6 @@ Response createOrUpdateWithResponse( * @return model that represents a Capability resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - CapabilityInner createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability); + CapabilityInner createOrUpdate(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName, CapabilityInner capability); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilityTypesClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilityTypesClient.java index 1bf7ffdeaf694..723d735439ac3 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilityTypesClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/CapabilityTypesClient.java @@ -11,25 +11,27 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.CapabilityTypeInner; -/** An instance of this class provides access to all the operations defined in CapabilityTypesClient. */ +/** + * An instance of this class provides access to all the operations defined in CapabilityTypesClient. + */ public interface CapabilityTypesClient { /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedIterable}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String locationName, String targetTypeName); /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. @@ -37,16 +39,16 @@ public interface CapabilityTypesClient { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedIterable}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String locationName, String targetTypeName, String continuationToken, Context context); + PagedIterable list(String locationName, String targetTypeName, String continuationToken, + Context context); /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -57,12 +59,12 @@ PagedIterable list( * @return a Capability Type resource for given Target Type and location along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String locationName, String targetTypeName, String capabilityTypeName, Context context); + Response getWithResponse(String locationName, String targetTypeName, String capabilityTypeName, + Context context); /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ChaosManagementClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ChaosManagementClient.java index 2aaac5167d493..0957af884ce84 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ChaosManagementClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ChaosManagementClient.java @@ -7,88 +7,90 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for ChaosManagementClient class. */ +/** + * The interface for ChaosManagementClient class. + */ public interface ChaosManagementClient { /** * Gets GUID that represents an Azure subscription ID. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the CapabilitiesClient object to access its operations. - * + * * @return the CapabilitiesClient object. */ CapabilitiesClient getCapabilities(); /** * Gets the CapabilityTypesClient object to access its operations. - * + * * @return the CapabilityTypesClient object. */ CapabilityTypesClient getCapabilityTypes(); /** * Gets the ExperimentsClient object to access its operations. - * + * * @return the ExperimentsClient object. */ ExperimentsClient getExperiments(); /** * Gets the OperationStatusesClient object to access its operations. - * + * * @return the OperationStatusesClient object. */ OperationStatusesClient getOperationStatuses(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the TargetTypesClient object to access its operations. - * + * * @return the TargetTypesClient object. */ TargetTypesClient getTargetTypes(); /** * Gets the TargetsClient object to access its operations. - * + * * @return the TargetsClient object. */ TargetsClient getTargets(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ExperimentsClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ExperimentsClient.java index ba81f716b0782..266e6c4ba7ea1 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ExperimentsClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/ExperimentsClient.java @@ -16,11 +16,13 @@ import com.azure.resourcemanager.chaos.fluent.models.ExperimentInner; import com.azure.resourcemanager.chaos.models.ExperimentUpdate; -/** An instance of this class provides access to all the operations defined in ExperimentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ExperimentsClient. + */ public interface ExperimentsClient { /** * Get a list of Experiment resources in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a subscription as paginated response with {@link PagedIterable}. @@ -30,9 +32,9 @@ public interface ExperimentsClient { /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -45,7 +47,7 @@ public interface ExperimentsClient { /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -57,10 +59,10 @@ public interface ExperimentsClient { /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -69,12 +71,12 @@ public interface ExperimentsClient { * @return a list of Experiment resources in a resource group as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroup( - String resourceGroupName, Boolean running, String continuationToken, Context context); + PagedIterable listByResourceGroup(String resourceGroupName, Boolean running, + String continuationToken, Context context); /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -87,7 +89,7 @@ PagedIterable listByResourceGroup( /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -101,7 +103,7 @@ PagedIterable listByResourceGroup( /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,7 +115,7 @@ PagedIterable listByResourceGroup( /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -126,7 +128,7 @@ PagedIterable listByResourceGroup( /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -136,12 +138,12 @@ PagedIterable listByResourceGroup( * @return a Experiment resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String experimentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String experimentName, + Context context); /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -154,7 +156,7 @@ Response getByResourceGroupWithResponse( /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -164,12 +166,12 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ExperimentInner> beginCreateOrUpdate( - String resourceGroupName, String experimentName, ExperimentInner experiment); + SyncPoller, ExperimentInner> beginCreateOrUpdate(String resourceGroupName, + String experimentName, ExperimentInner experiment); /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -180,12 +182,12 @@ SyncPoller, ExperimentInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ExperimentInner> beginCreateOrUpdate( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context); + SyncPoller, ExperimentInner> beginCreateOrUpdate(String resourceGroupName, + String experimentName, ExperimentInner experiment, Context context); /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -199,7 +201,7 @@ SyncPoller, ExperimentInner> beginCreateOrUpdate( /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -210,12 +212,12 @@ SyncPoller, ExperimentInner> beginCreateOrUpdate( * @return model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ExperimentInner createOrUpdate( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context); + ExperimentInner createOrUpdate(String resourceGroupName, String experimentName, ExperimentInner experiment, + Context context); /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -225,12 +227,12 @@ ExperimentInner createOrUpdate( * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ExperimentInner> beginUpdate( - String resourceGroupName, String experimentName, ExperimentUpdate experiment); + SyncPoller, ExperimentInner> beginUpdate(String resourceGroupName, + String experimentName, ExperimentUpdate experiment); /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -241,12 +243,12 @@ SyncPoller, ExperimentInner> beginUpdate( * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ExperimentInner> beginUpdate( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context); + SyncPoller, ExperimentInner> beginUpdate(String resourceGroupName, + String experimentName, ExperimentUpdate experiment, Context context); /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -260,7 +262,7 @@ SyncPoller, ExperimentInner> beginUpdate( /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -271,12 +273,12 @@ SyncPoller, ExperimentInner> beginUpdate( * @return model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ExperimentInner update( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context); + ExperimentInner update(String resourceGroupName, String experimentName, ExperimentUpdate experiment, + Context context); /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -289,7 +291,7 @@ ExperimentInner update( /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -303,7 +305,7 @@ ExperimentInner update( /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -315,7 +317,7 @@ ExperimentInner update( /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -328,7 +330,7 @@ ExperimentInner update( /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -341,7 +343,7 @@ ExperimentInner update( /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -355,7 +357,7 @@ ExperimentInner update( /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -367,7 +369,7 @@ ExperimentInner update( /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -380,7 +382,7 @@ ExperimentInner update( /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -393,7 +395,7 @@ ExperimentInner update( /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -403,12 +405,12 @@ ExperimentInner update( * @return a list of executions of an Experiment resource as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAllExecutions( - String resourceGroupName, String experimentName, Context context); + PagedIterable listAllExecutions(String resourceGroupName, String experimentName, + Context context); /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -419,12 +421,12 @@ PagedIterable listAllExecutions( * @return an execution of an Experiment resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getExecutionWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context); + Response getExecutionWithResponse(String resourceGroupName, String experimentName, + String executionId, Context context); /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -438,7 +440,7 @@ Response getExecutionWithResponse( /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -449,12 +451,12 @@ Response getExecutionWithResponse( * @return model that represents the execution details of an Experiment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response executionDetailsWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context); + Response executionDetailsWithResponse(String resourceGroupName, + String experimentName, String executionId, Context context); /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -464,6 +466,6 @@ Response executionDetailsWithResponse( * @return model that represents the execution details of an Experiment. */ @ServiceMethod(returns = ReturnType.SINGLE) - ExperimentExecutionDetailsInner executionDetails( - String resourceGroupName, String experimentName, String executionId); + ExperimentExecutionDetailsInner executionDetails(String resourceGroupName, String experimentName, + String executionId); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationStatusesClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationStatusesClient.java index b50d0814f55fa..87083b88e1ee8 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationStatusesClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationStatusesClient.java @@ -10,12 +10,14 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.OperationStatusInner; -/** An instance of this class provides access to all the operations defined in OperationStatusesClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationStatusesClient. + */ public interface OperationStatusesClient { /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -28,8 +30,8 @@ public interface OperationStatusesClient { /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationsClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationsClient.java index fdd25c05ff159..6be2ec3d60b24 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationsClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/OperationsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Get a list all available Operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list all available Operations as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface OperationsClient { /** * Get a list all available Operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetTypesClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetTypesClient.java index 46f1a4177d1fe..92e8fb223c8f3 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetTypesClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetTypesClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.TargetTypeInner; -/** An instance of this class provides access to all the operations defined in TargetTypesClient. */ +/** + * An instance of this class provides access to all the operations defined in TargetTypesClient. + */ public interface TargetTypesClient { /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -27,7 +29,7 @@ public interface TargetTypesClient { /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. @@ -41,7 +43,7 @@ public interface TargetTypesClient { /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param context The context to associate with this operation. @@ -55,7 +57,7 @@ public interface TargetTypesClient { /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetsClient.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetsClient.java index a3ec48e28f4b0..0caf92afda5df 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetsClient.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/TargetsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.TargetInner; -/** An instance of this class provides access to all the operations defined in TargetsClient. */ +/** + * An instance of this class provides access to all the operations defined in TargetsClient. + */ public interface TargetsClient { /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -23,16 +25,16 @@ public interface TargetsClient { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedIterable}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String resourceGroupName, String parentProviderNamespace, String parentResourceType, String parentResourceName); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName); /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -42,21 +44,16 @@ PagedIterable list( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedIterable}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken, - Context context); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String continuationToken, Context context); /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -69,17 +66,12 @@ PagedIterable list( * @return a Target resource that extends a tracked regional resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context); + Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context); /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -91,16 +83,12 @@ Response getWithResponse( * @return a Target resource that extends a tracked regional resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - TargetInner get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName); + TargetInner get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName); /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -113,17 +101,12 @@ TargetInner get( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context); + Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context); /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -134,16 +117,12 @@ Response deleteWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName); + void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName); /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -157,18 +136,12 @@ void delete( * @return model that represents a Target resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, TargetInner target, Context context); /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -181,11 +154,6 @@ Response createOrUpdateWithResponse( * @return model that represents a Target resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - TargetInner createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target); + TargetInner createOrUpdate(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, TargetInner target); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityInner.java index 32126fd4a0729..77d0fabdd71f2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityInner.java @@ -9,7 +9,9 @@ import com.azure.core.management.SystemData; import com.fasterxml.jackson.annotation.JsonProperty; -/** Model that represents a Capability resource. */ +/** + * Model that represents a Capability resource. + */ @Immutable public final class CapabilityInner extends ProxyResource { /* @@ -24,13 +26,15 @@ public final class CapabilityInner extends ProxyResource { @JsonProperty(value = "properties", access = JsonProperty.Access.WRITE_ONLY) private CapabilityProperties innerProperties; - /** Creates an instance of CapabilityInner class. */ + /** + * Creates an instance of CapabilityInner class. + */ public CapabilityInner() { } /** * Get the systemData property: The standard system metadata of a resource type. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -39,7 +43,7 @@ public SystemData systemData() { /** * Get the innerProperties property: The properties of a capability resource. - * + * * @return the innerProperties value. */ private CapabilityProperties innerProperties() { @@ -48,7 +52,7 @@ private CapabilityProperties innerProperties() { /** * Get the publisher property: String of the Publisher that this Capability extends. - * + * * @return the publisher value. */ public String publisher() { @@ -57,7 +61,7 @@ public String publisher() { /** * Get the targetType property: String of the Target Type that this Capability extends. - * + * * @return the targetType value. */ public String targetType() { @@ -66,7 +70,7 @@ public String targetType() { /** * Get the description property: Localized string of the description. - * + * * @return the description value. */ public String description() { @@ -75,7 +79,7 @@ public String description() { /** * Get the parametersSchema property: URL to retrieve JSON schema of the Capability parameters. - * + * * @return the parametersSchema value. */ public String parametersSchema() { @@ -84,7 +88,7 @@ public String parametersSchema() { /** * Get the urn property: String of the URN for this Capability Type. - * + * * @return the urn value. */ public String urn() { @@ -93,7 +97,7 @@ public String urn() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityProperties.java index 4970fe21a6cac..2380ddf8be90a 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Model that represents the Capability properties model. */ +/** + * Model that represents the Capability properties model. + */ @Immutable public final class CapabilityProperties { /* @@ -40,13 +42,15 @@ public final class CapabilityProperties { @JsonProperty(value = "urn", access = JsonProperty.Access.WRITE_ONLY) private String urn; - /** Creates an instance of CapabilityProperties class. */ + /** + * Creates an instance of CapabilityProperties class. + */ public CapabilityProperties() { } /** * Get the publisher property: String of the Publisher that this Capability extends. - * + * * @return the publisher value. */ public String publisher() { @@ -55,7 +59,7 @@ public String publisher() { /** * Get the targetType property: String of the Target Type that this Capability extends. - * + * * @return the targetType value. */ public String targetType() { @@ -64,7 +68,7 @@ public String targetType() { /** * Get the description property: Localized string of the description. - * + * * @return the description value. */ public String description() { @@ -73,7 +77,7 @@ public String description() { /** * Get the parametersSchema property: URL to retrieve JSON schema of the Capability parameters. - * + * * @return the parametersSchema value. */ public String parametersSchema() { @@ -82,7 +86,7 @@ public String parametersSchema() { /** * Get the urn property: String of the URN for this Capability Type. - * + * * @return the urn value. */ public String urn() { @@ -91,7 +95,7 @@ public String urn() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeInner.java index 5fa57575bc576..ffea085d3935f 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a Capability Type resource. */ +/** + * Model that represents a Capability Type resource. + */ @Fluent public final class CapabilityTypeInner extends ProxyResource { /* @@ -32,13 +34,15 @@ public final class CapabilityTypeInner extends ProxyResource { @JsonProperty(value = "properties", access = JsonProperty.Access.WRITE_ONLY) private CapabilityTypeProperties innerProperties; - /** Creates an instance of CapabilityTypeInner class. */ + /** + * Creates an instance of CapabilityTypeInner class. + */ public CapabilityTypeInner() { } /** * Get the systemData property: The system metadata properties of the capability type resource. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -47,7 +51,7 @@ public SystemData systemData() { /** * Get the location property: Location of the Capability Type resource. - * + * * @return the location value. */ public String location() { @@ -56,7 +60,7 @@ public String location() { /** * Set the location property: Location of the Capability Type resource. - * + * * @param location the location value to set. * @return the CapabilityTypeInner object itself. */ @@ -67,7 +71,7 @@ public CapabilityTypeInner withLocation(String location) { /** * Get the innerProperties property: The properties of the capability type resource. - * + * * @return the innerProperties value. */ private CapabilityTypeProperties innerProperties() { @@ -76,7 +80,7 @@ private CapabilityTypeProperties innerProperties() { /** * Get the publisher property: String of the Publisher that this Capability Type extends. - * + * * @return the publisher value. */ public String publisher() { @@ -85,7 +89,7 @@ public String publisher() { /** * Get the targetType property: String of the Target Type that this Capability Type extends. - * + * * @return the targetType value. */ public String targetType() { @@ -94,7 +98,7 @@ public String targetType() { /** * Get the displayName property: Localized string of the display name. - * + * * @return the displayName value. */ public String displayName() { @@ -103,7 +107,7 @@ public String displayName() { /** * Get the description property: Localized string of the description. - * + * * @return the description value. */ public String description() { @@ -112,7 +116,7 @@ public String description() { /** * Get the parametersSchema property: URL to retrieve JSON schema of the Capability Type parameters. - * + * * @return the parametersSchema value. */ public String parametersSchema() { @@ -121,7 +125,7 @@ public String parametersSchema() { /** * Get the urn property: String of the URN for this Capability Type. - * + * * @return the urn value. */ public String urn() { @@ -130,7 +134,7 @@ public String urn() { /** * Get the kind property: String of the kind of this Capability Type. - * + * * @return the kind value. */ public String kind() { @@ -139,7 +143,7 @@ public String kind() { /** * Get the azureRbacActions property: Control plane actions necessary to execute capability type. - * + * * @return the azureRbacActions value. */ public List azureRbacActions() { @@ -148,7 +152,7 @@ public List azureRbacActions() { /** * Set the azureRbacActions property: Control plane actions necessary to execute capability type. - * + * * @param azureRbacActions the azureRbacActions value to set. * @return the CapabilityTypeInner object itself. */ @@ -162,7 +166,7 @@ public CapabilityTypeInner withAzureRbacActions(List azureRbacActions) { /** * Get the azureRbacDataActions property: Data plane actions necessary to execute capability type. - * + * * @return the azureRbacDataActions value. */ public List azureRbacDataActions() { @@ -171,7 +175,7 @@ public List azureRbacDataActions() { /** * Set the azureRbacDataActions property: Data plane actions necessary to execute capability type. - * + * * @param azureRbacDataActions the azureRbacDataActions value to set. * @return the CapabilityTypeInner object itself. */ @@ -185,7 +189,7 @@ public CapabilityTypeInner withAzureRbacDataActions(List azureRbacDataAc /** * Get the runtimeProperties property: Runtime properties of this Capability Type. - * + * * @return the runtimeProperties value. */ public CapabilityTypePropertiesRuntimeProperties runtimeProperties() { @@ -194,7 +198,7 @@ public CapabilityTypePropertiesRuntimeProperties runtimeProperties() { /** * Set the runtimeProperties property: Runtime properties of this Capability Type. - * + * * @param runtimeProperties the runtimeProperties value to set. * @return the CapabilityTypeInner object itself. */ @@ -208,7 +212,7 @@ public CapabilityTypeInner withRuntimeProperties(CapabilityTypePropertiesRuntime /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeProperties.java index 5767a0f53322a..823c0f6fffdcf 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/CapabilityTypeProperties.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents the Capability Type properties model. */ +/** + * Model that represents the Capability Type properties model. + */ @Fluent public final class CapabilityTypeProperties { /* @@ -72,13 +74,15 @@ public final class CapabilityTypeProperties { @JsonProperty(value = "runtimeProperties") private CapabilityTypePropertiesRuntimeProperties runtimeProperties; - /** Creates an instance of CapabilityTypeProperties class. */ + /** + * Creates an instance of CapabilityTypeProperties class. + */ public CapabilityTypeProperties() { } /** * Get the publisher property: String of the Publisher that this Capability Type extends. - * + * * @return the publisher value. */ public String publisher() { @@ -87,7 +91,7 @@ public String publisher() { /** * Get the targetType property: String of the Target Type that this Capability Type extends. - * + * * @return the targetType value. */ public String targetType() { @@ -96,7 +100,7 @@ public String targetType() { /** * Get the displayName property: Localized string of the display name. - * + * * @return the displayName value. */ public String displayName() { @@ -105,7 +109,7 @@ public String displayName() { /** * Get the description property: Localized string of the description. - * + * * @return the description value. */ public String description() { @@ -114,7 +118,7 @@ public String description() { /** * Get the parametersSchema property: URL to retrieve JSON schema of the Capability Type parameters. - * + * * @return the parametersSchema value. */ public String parametersSchema() { @@ -123,7 +127,7 @@ public String parametersSchema() { /** * Get the urn property: String of the URN for this Capability Type. - * + * * @return the urn value. */ public String urn() { @@ -132,7 +136,7 @@ public String urn() { /** * Get the kind property: String of the kind of this Capability Type. - * + * * @return the kind value. */ public String kind() { @@ -141,7 +145,7 @@ public String kind() { /** * Get the azureRbacActions property: Control plane actions necessary to execute capability type. - * + * * @return the azureRbacActions value. */ public List azureRbacActions() { @@ -150,7 +154,7 @@ public List azureRbacActions() { /** * Set the azureRbacActions property: Control plane actions necessary to execute capability type. - * + * * @param azureRbacActions the azureRbacActions value to set. * @return the CapabilityTypeProperties object itself. */ @@ -161,7 +165,7 @@ public CapabilityTypeProperties withAzureRbacActions(List azureRbacActio /** * Get the azureRbacDataActions property: Data plane actions necessary to execute capability type. - * + * * @return the azureRbacDataActions value. */ public List azureRbacDataActions() { @@ -170,7 +174,7 @@ public List azureRbacDataActions() { /** * Set the azureRbacDataActions property: Data plane actions necessary to execute capability type. - * + * * @param azureRbacDataActions the azureRbacDataActions value to set. * @return the CapabilityTypeProperties object itself. */ @@ -181,7 +185,7 @@ public CapabilityTypeProperties withAzureRbacDataActions(List azureRbacD /** * Get the runtimeProperties property: Runtime properties of this Capability Type. - * + * * @return the runtimeProperties value. */ public CapabilityTypePropertiesRuntimeProperties runtimeProperties() { @@ -190,7 +194,7 @@ public CapabilityTypePropertiesRuntimeProperties runtimeProperties() { /** * Set the runtimeProperties property: Runtime properties of this Capability Type. - * + * * @param runtimeProperties the runtimeProperties value to set. * @return the CapabilityTypeProperties object itself. */ @@ -201,7 +205,7 @@ public CapabilityTypeProperties withRuntimeProperties(CapabilityTypePropertiesRu /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsInner.java index c8b4c2738e167..a9206b47ee9e4 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Model that represents the execution details of an Experiment. */ +/** + * Model that represents the execution details of an Experiment. + */ @Immutable public final class ExperimentExecutionDetailsInner { /* @@ -36,13 +38,15 @@ public final class ExperimentExecutionDetailsInner { @JsonProperty(value = "properties", access = JsonProperty.Access.WRITE_ONLY) private ExperimentExecutionDetailsProperties innerProperties; - /** Creates an instance of ExperimentExecutionDetailsInner class. */ + /** + * Creates an instance of ExperimentExecutionDetailsInner class. + */ public ExperimentExecutionDetailsInner() { } /** * Get the type property: String of the resource type. - * + * * @return the type value. */ public String type() { @@ -51,7 +55,7 @@ public String type() { /** * Get the id property: String of the fully qualified resource ID. - * + * * @return the id value. */ public String id() { @@ -60,7 +64,7 @@ public String id() { /** * Get the name property: String of the resource name. - * + * * @return the name value. */ public String name() { @@ -69,7 +73,7 @@ public String name() { /** * Get the innerProperties property: The properties of the experiment execution details. - * + * * @return the innerProperties value. */ private ExperimentExecutionDetailsProperties innerProperties() { @@ -78,7 +82,7 @@ private ExperimentExecutionDetailsProperties innerProperties() { /** * Get the failureReason property: The reason why the execution failed. - * + * * @return the failureReason value. */ public String failureReason() { @@ -87,7 +91,7 @@ public String failureReason() { /** * Get the lastActionAt property: String that represents the last action date time. - * + * * @return the lastActionAt value. */ public OffsetDateTime lastActionAt() { @@ -96,7 +100,7 @@ public OffsetDateTime lastActionAt() { /** * Get the runInformation property: The information of the experiment run. - * + * * @return the runInformation value. */ public ExperimentExecutionDetailsPropertiesRunInformation runInformation() { @@ -105,7 +109,7 @@ public ExperimentExecutionDetailsPropertiesRunInformation runInformation() { /** * Get the status property: The status of the execution. - * + * * @return the status value. */ public String status() { @@ -114,7 +118,7 @@ public String status() { /** * Get the startedAt property: String that represents the start date time. - * + * * @return the startedAt value. */ public OffsetDateTime startedAt() { @@ -123,7 +127,7 @@ public OffsetDateTime startedAt() { /** * Get the stoppedAt property: String that represents the stop date time. - * + * * @return the stoppedAt value. */ public OffsetDateTime stoppedAt() { @@ -132,7 +136,7 @@ public OffsetDateTime stoppedAt() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsProperties.java index 785f78ecb7432..d8094e07050fb 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionDetailsProperties.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Model that represents the extended properties of an experiment execution. */ +/** + * Model that represents the extended properties of an experiment execution. + */ @Immutable public final class ExperimentExecutionDetailsProperties extends ExperimentExecutionProperties { /* @@ -30,13 +32,15 @@ public final class ExperimentExecutionDetailsProperties extends ExperimentExecut @JsonProperty(value = "runInformation", access = JsonProperty.Access.WRITE_ONLY) private ExperimentExecutionDetailsPropertiesRunInformation runInformation; - /** Creates an instance of ExperimentExecutionDetailsProperties class. */ + /** + * Creates an instance of ExperimentExecutionDetailsProperties class. + */ public ExperimentExecutionDetailsProperties() { } /** * Get the failureReason property: The reason why the execution failed. - * + * * @return the failureReason value. */ public String failureReason() { @@ -45,7 +49,7 @@ public String failureReason() { /** * Get the lastActionAt property: String that represents the last action date time. - * + * * @return the lastActionAt value. */ public OffsetDateTime lastActionAt() { @@ -54,7 +58,7 @@ public OffsetDateTime lastActionAt() { /** * Get the runInformation property: The information of the experiment run. - * + * * @return the runInformation value. */ public ExperimentExecutionDetailsPropertiesRunInformation runInformation() { @@ -63,7 +67,7 @@ public ExperimentExecutionDetailsPropertiesRunInformation runInformation() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionInner.java index 59ad9f95d129f..420ffd70a2dfe 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionInner.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Model that represents the execution of a Experiment. */ +/** + * Model that represents the execution of a Experiment. + */ @Immutable public final class ExperimentExecutionInner { /* @@ -35,13 +37,15 @@ public final class ExperimentExecutionInner { @JsonProperty(value = "properties") private ExperimentExecutionProperties innerProperties; - /** Creates an instance of ExperimentExecutionInner class. */ + /** + * Creates an instance of ExperimentExecutionInner class. + */ public ExperimentExecutionInner() { } /** * Get the type property: String of the resource type. - * + * * @return the type value. */ public String type() { @@ -50,7 +54,7 @@ public String type() { /** * Get the id property: String of the fully qualified resource ID. - * + * * @return the id value. */ public String id() { @@ -59,7 +63,7 @@ public String id() { /** * Get the name property: String of the resource name. - * + * * @return the name value. */ public String name() { @@ -68,7 +72,7 @@ public String name() { /** * Get the innerProperties property: The properties of experiment execution status. - * + * * @return the innerProperties value. */ private ExperimentExecutionProperties innerProperties() { @@ -77,7 +81,7 @@ private ExperimentExecutionProperties innerProperties() { /** * Get the status property: The status of the execution. - * + * * @return the status value. */ public String status() { @@ -86,7 +90,7 @@ public String status() { /** * Get the startedAt property: String that represents the start date time. - * + * * @return the startedAt value. */ public OffsetDateTime startedAt() { @@ -95,7 +99,7 @@ public OffsetDateTime startedAt() { /** * Get the stoppedAt property: String that represents the stop date time. - * + * * @return the stoppedAt value. */ public OffsetDateTime stoppedAt() { @@ -104,7 +108,7 @@ public OffsetDateTime stoppedAt() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionProperties.java index 2a71fd31e3417..0338e23a53bbf 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentExecutionProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Model that represents the execution properties of an Experiment. */ +/** + * Model that represents the execution properties of an Experiment. + */ @Immutable public class ExperimentExecutionProperties { /* @@ -29,13 +31,15 @@ public class ExperimentExecutionProperties { @JsonProperty(value = "stoppedAt", access = JsonProperty.Access.WRITE_ONLY) private OffsetDateTime stoppedAt; - /** Creates an instance of ExperimentExecutionProperties class. */ + /** + * Creates an instance of ExperimentExecutionProperties class. + */ public ExperimentExecutionProperties() { } /** * Get the status property: The status of the execution. - * + * * @return the status value. */ public String status() { @@ -44,7 +48,7 @@ public String status() { /** * Get the startedAt property: String that represents the start date time. - * + * * @return the startedAt value. */ public OffsetDateTime startedAt() { @@ -53,7 +57,7 @@ public OffsetDateTime startedAt() { /** * Get the stoppedAt property: String that represents the stop date time. - * + * * @return the stoppedAt value. */ public OffsetDateTime stoppedAt() { @@ -62,7 +66,7 @@ public OffsetDateTime stoppedAt() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentInner.java index e2fc798864492..70656ef6bef37 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentInner.java @@ -16,7 +16,9 @@ import java.util.List; import java.util.Map; -/** Model that represents a Experiment resource. */ +/** + * Model that represents a Experiment resource. + */ @Fluent public final class ExperimentInner extends Resource { /* @@ -37,13 +39,15 @@ public final class ExperimentInner extends Resource { @JsonProperty(value = "properties", required = true) private ExperimentProperties innerProperties = new ExperimentProperties(); - /** Creates an instance of ExperimentInner class. */ + /** + * Creates an instance of ExperimentInner class. + */ public ExperimentInner() { } /** * Get the systemData property: The system metadata of the experiment resource. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -52,7 +56,7 @@ public SystemData systemData() { /** * Get the identity property: The identity of the experiment resource. - * + * * @return the identity value. */ public ResourceIdentity identity() { @@ -61,7 +65,7 @@ public ResourceIdentity identity() { /** * Set the identity property: The identity of the experiment resource. - * + * * @param identity the identity value to set. * @return the ExperimentInner object itself. */ @@ -72,21 +76,25 @@ public ExperimentInner withIdentity(ResourceIdentity identity) { /** * Get the innerProperties property: The properties of the experiment resource. - * + * * @return the innerProperties value. */ private ExperimentProperties innerProperties() { return this.innerProperties; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ExperimentInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ExperimentInner withTags(Map tags) { super.withTags(tags); @@ -95,7 +103,7 @@ public ExperimentInner withTags(Map tags) { /** * Get the provisioningState property: Most recent provisioning state for the given experiment resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -104,7 +112,7 @@ public ProvisioningState provisioningState() { /** * Get the steps property: List of steps. - * + * * @return the steps value. */ public List steps() { @@ -113,7 +121,7 @@ public List steps() { /** * Set the steps property: List of steps. - * + * * @param steps the steps value to set. * @return the ExperimentInner object itself. */ @@ -127,7 +135,7 @@ public ExperimentInner withSteps(List steps) { /** * Get the selectors property: List of selectors. - * + * * @return the selectors value. */ public List selectors() { @@ -136,7 +144,7 @@ public List selectors() { /** * Set the selectors property: List of selectors. - * + * * @param selectors the selectors value to set. * @return the ExperimentInner object itself. */ @@ -150,7 +158,7 @@ public ExperimentInner withSelectors(List selectors) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -158,9 +166,8 @@ public void validate() { identity().validate(); } if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model ExperimentInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model ExperimentInner")); } else { innerProperties().validate(); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentProperties.java index ddcd079377968..bb1445fe5dc11 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/ExperimentProperties.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents the Experiment properties model. */ +/** + * Model that represents the Experiment properties model. + */ @Fluent public final class ExperimentProperties { /* @@ -33,13 +35,15 @@ public final class ExperimentProperties { @JsonProperty(value = "selectors", required = true) private List selectors; - /** Creates an instance of ExperimentProperties class. */ + /** + * Creates an instance of ExperimentProperties class. + */ public ExperimentProperties() { } /** * Get the provisioningState property: Most recent provisioning state for the given experiment resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -48,7 +52,7 @@ public ProvisioningState provisioningState() { /** * Get the steps property: List of steps. - * + * * @return the steps value. */ public List steps() { @@ -57,7 +61,7 @@ public List steps() { /** * Set the steps property: List of steps. - * + * * @param steps the steps value to set. * @return the ExperimentProperties object itself. */ @@ -68,7 +72,7 @@ public ExperimentProperties withSteps(List steps) { /** * Get the selectors property: List of selectors. - * + * * @return the selectors value. */ public List selectors() { @@ -77,7 +81,7 @@ public List selectors() { /** * Set the selectors property: List of selectors. - * + * * @param selectors the selectors value to set. * @return the ExperimentProperties object itself. */ @@ -88,21 +92,19 @@ public ExperimentProperties withSelectors(List selectors) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (steps() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property steps in model ExperimentProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property steps in model ExperimentProperties")); } else { steps().forEach(e -> e.validate()); } if (selectors() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property selectors in model ExperimentProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property selectors in model ExperimentProperties")); } else { selectors().forEach(e -> e.validate()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationInner.java index e710daaf81ae3..d71cf2dbad10e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationInner.java @@ -12,8 +12,8 @@ /** * REST API Operation - * - *

Details of a REST API operation, returned from the Resource Provider Operations API. + * + * Details of a REST API operation, returned from the Resource Provider Operations API. */ @Fluent public final class OperationInner { @@ -50,14 +50,16 @@ public final class OperationInner { @JsonProperty(value = "actionType", access = JsonProperty.Access.WRITE_ONLY) private ActionType actionType; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ public String name() { @@ -67,7 +69,7 @@ public String name() { /** * Get the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -76,7 +78,7 @@ public Boolean isDataAction() { /** * Get the display property: Localized display information for this particular operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -85,7 +87,7 @@ public OperationDisplay display() { /** * Set the display property: Localized display information for this particular operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -97,7 +99,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Get the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ public Origin origin() { @@ -107,7 +109,7 @@ public Origin origin() { /** * Get the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ public ActionType actionType() { @@ -116,7 +118,7 @@ public ActionType actionType() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationStatusInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationStatusInner.java index c6be5e917a602..24456ee1cf196 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationStatusInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/OperationStatusInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.exception.ManagementError; import com.fasterxml.jackson.annotation.JsonProperty; -/** The status of operation. */ +/** + * The status of operation. + */ @Fluent public final class OperationStatusInner { /* @@ -47,13 +49,15 @@ public final class OperationStatusInner { @JsonProperty(value = "error") private ManagementError error; - /** Creates an instance of OperationStatusInner class. */ + /** + * Creates an instance of OperationStatusInner class. + */ public OperationStatusInner() { } /** * Get the id property: The operation Id. - * + * * @return the id value. */ public String id() { @@ -62,7 +66,7 @@ public String id() { /** * Set the id property: The operation Id. - * + * * @param id the id value to set. * @return the OperationStatusInner object itself. */ @@ -73,7 +77,7 @@ public OperationStatusInner withId(String id) { /** * Get the name property: The operation name. - * + * * @return the name value. */ public String name() { @@ -82,7 +86,7 @@ public String name() { /** * Set the name property: The operation name. - * + * * @param name the name value to set. * @return the OperationStatusInner object itself. */ @@ -93,7 +97,7 @@ public OperationStatusInner withName(String name) { /** * Get the startTime property: The start time of the operation. - * + * * @return the startTime value. */ public String startTime() { @@ -102,7 +106,7 @@ public String startTime() { /** * Set the startTime property: The start time of the operation. - * + * * @param startTime the startTime value to set. * @return the OperationStatusInner object itself. */ @@ -113,7 +117,7 @@ public OperationStatusInner withStartTime(String startTime) { /** * Get the endTime property: The end time of the operation. - * + * * @return the endTime value. */ public String endTime() { @@ -122,7 +126,7 @@ public String endTime() { /** * Set the endTime property: The end time of the operation. - * + * * @param endTime the endTime value to set. * @return the OperationStatusInner object itself. */ @@ -133,7 +137,7 @@ public OperationStatusInner withEndTime(String endTime) { /** * Get the status property: The status of the operation. - * + * * @return the status value. */ public String status() { @@ -142,7 +146,7 @@ public String status() { /** * Set the status property: The status of the operation. - * + * * @param status the status value to set. * @return the OperationStatusInner object itself. */ @@ -153,7 +157,7 @@ public OperationStatusInner withStatus(String status) { /** * Get the error property: The error object. - * + * * @return the error value. */ public ManagementError error() { @@ -162,7 +166,7 @@ public ManagementError error() { /** * Set the error property: The error object. - * + * * @param error the error value to set. * @return the OperationStatusInner object itself. */ @@ -173,7 +177,7 @@ public OperationStatusInner withError(ManagementError error) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetInner.java index 7145cd04d0680..2700e264d225d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetInner.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Model that represents a Target resource. */ +/** + * Model that represents a Target resource. + */ @Fluent public final class TargetInner extends ProxyResource { /* @@ -34,13 +36,15 @@ public final class TargetInner extends ProxyResource { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map properties; - /** Creates an instance of TargetInner class. */ + /** + * Creates an instance of TargetInner class. + */ public TargetInner() { } /** * Get the systemData property: The system metadata of the target resource. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -49,7 +53,7 @@ public SystemData systemData() { /** * Get the location property: Location of the target resource. - * + * * @return the location value. */ public String location() { @@ -58,7 +62,7 @@ public String location() { /** * Set the location property: Location of the target resource. - * + * * @param location the location value to set. * @return the TargetInner object itself. */ @@ -69,7 +73,7 @@ public TargetInner withLocation(String location) { /** * Get the properties property: The properties of the target resource. - * + * * @return the properties value. */ public Map properties() { @@ -78,7 +82,7 @@ public Map properties() { /** * Set the properties property: The properties of the target resource. - * + * * @param properties the properties value to set. * @return the TargetInner object itself. */ @@ -89,14 +93,13 @@ public TargetInner withProperties(Map properties) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (properties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property properties in model TargetInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property properties in model TargetInner")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeInner.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeInner.java index d96b2afce917e..7977d4e0b5d8b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeInner.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a Target Type resource. */ +/** + * Model that represents a Target Type resource. + */ @Fluent public final class TargetTypeInner extends ProxyResource { /* @@ -32,13 +34,15 @@ public final class TargetTypeInner extends ProxyResource { @JsonProperty(value = "properties", required = true) private TargetTypeProperties innerProperties = new TargetTypeProperties(); - /** Creates an instance of TargetTypeInner class. */ + /** + * Creates an instance of TargetTypeInner class. + */ public TargetTypeInner() { } /** * Get the systemData property: The system metadata properties of the target type resource. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -47,7 +51,7 @@ public SystemData systemData() { /** * Get the location property: Location of the Target Type resource. - * + * * @return the location value. */ public String location() { @@ -56,7 +60,7 @@ public String location() { /** * Set the location property: Location of the Target Type resource. - * + * * @param location the location value to set. * @return the TargetTypeInner object itself. */ @@ -67,7 +71,7 @@ public TargetTypeInner withLocation(String location) { /** * Get the innerProperties property: The properties of the target type resource. - * + * * @return the innerProperties value. */ private TargetTypeProperties innerProperties() { @@ -76,7 +80,7 @@ private TargetTypeProperties innerProperties() { /** * Get the displayName property: Localized string of the display name. - * + * * @return the displayName value. */ public String displayName() { @@ -85,7 +89,7 @@ public String displayName() { /** * Get the description property: Localized string of the description. - * + * * @return the description value. */ public String description() { @@ -94,7 +98,7 @@ public String description() { /** * Get the propertiesSchema property: URL to retrieve JSON schema of the Target Type properties. - * + * * @return the propertiesSchema value. */ public String propertiesSchema() { @@ -103,7 +107,7 @@ public String propertiesSchema() { /** * Get the resourceTypes property: List of resource types this Target Type can extend. - * + * * @return the resourceTypes value. */ public List resourceTypes() { @@ -112,14 +116,13 @@ public List resourceTypes() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model TargetTypeInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model TargetTypeInner")); } else { innerProperties().validate(); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeProperties.java index 9bc624dedb5c0..56be5f6e63786 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/TargetTypeProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents the base Target Type properties model. */ +/** + * Model that represents the base Target Type properties model. + */ @Immutable public final class TargetTypeProperties { /* @@ -35,13 +37,15 @@ public final class TargetTypeProperties { @JsonProperty(value = "resourceTypes", access = JsonProperty.Access.WRITE_ONLY) private List resourceTypes; - /** Creates an instance of TargetTypeProperties class. */ + /** + * Creates an instance of TargetTypeProperties class. + */ public TargetTypeProperties() { } /** * Get the displayName property: Localized string of the display name. - * + * * @return the displayName value. */ public String displayName() { @@ -50,7 +54,7 @@ public String displayName() { /** * Get the description property: Localized string of the description. - * + * * @return the description value. */ public String description() { @@ -59,7 +63,7 @@ public String description() { /** * Get the propertiesSchema property: URL to retrieve JSON schema of the Target Type properties. - * + * * @return the propertiesSchema value. */ public String propertiesSchema() { @@ -68,7 +72,7 @@ public String propertiesSchema() { /** * Get the resourceTypes property: List of resource types this Target Type can extend. - * + * * @return the resourceTypes value. */ public List resourceTypes() { @@ -77,7 +81,7 @@ public List resourceTypes() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/package-info.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/package-info.java index 528b3b0bc9607..197d8d2de364b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/package-info.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the inner data models for ChaosManagementClient. Chaos Management Client. */ +/** + * Package containing the inner data models for ChaosManagementClient. + * Chaos Management Client. + */ package com.azure.resourcemanager.chaos.fluent.models; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/package-info.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/package-info.java index 80b2eb1aca618..28b65cdb44829 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/package-info.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/fluent/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the service clients for ChaosManagementClient. Chaos Management Client. */ +/** + * Package containing the service clients for ChaosManagementClient. + * Chaos Management Client. + */ package com.azure.resourcemanager.chaos.fluent; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesClientImpl.java index d9f1e9571315e..0c362fb7a8851 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesClientImpl.java @@ -33,22 +33,28 @@ import com.azure.resourcemanager.chaos.models.CapabilityListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CapabilitiesClient. */ +/** + * An instance of this class provides access to all the operations defined in CapabilitiesClient. + */ public final class CapabilitiesClientImpl implements CapabilitiesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CapabilitiesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of CapabilitiesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ CapabilitiesClientImpl(ChaosManagementClientImpl client) { - this.service = - RestProxy.create(CapabilitiesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(CapabilitiesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -59,93 +65,68 @@ public final class CapabilitiesClientImpl implements CapabilitiesClient { @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface CapabilitiesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, - @QueryParam("continuationToken") String continuationToken, - @HeaderParam("Accept") String accept, + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, + @QueryParam("continuationToken") String continuationToken, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities/{capabilityName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities/{capabilityName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, - @PathParam("capabilityName") String capabilityName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, + @PathParam("capabilityName") String capabilityName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities/{capabilityName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities/{capabilityName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, - @PathParam("capabilityName") String capabilityName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, + @PathParam("capabilityName") String capabilityName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities/{capabilityName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}/capabilities/{capabilityName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, @PathParam("capabilityName") String capabilityName, - @BodyParam("application/json") CapabilityInner capability, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") CapabilityInner capability, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -156,36 +137,27 @@ Mono> listNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Capability resources that extend a Target resource. along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, + private Mono> listSinglePageAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, String continuationToken) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -200,36 +172,17 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, continuationToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -241,37 +194,27 @@ private Mono> listSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Capability resources that extend a Target resource. along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken, - Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, + String continuationToken, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -287,32 +230,16 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, + continuationToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -322,32 +249,19 @@ private Mono> listSinglePageAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedFlux}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken) { - return new PagedFlux<>( - () -> - listSinglePageAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken), - nextLink -> listNextSinglePageAsync(nextLink)); + private PagedFlux listAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String continuationToken) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -356,32 +270,20 @@ private PagedFlux listAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedFlux}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { + private PagedFlux listAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName) { final String continuationToken = null; - return new PagedFlux<>( - () -> - listSinglePageAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken), - nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -392,34 +294,22 @@ private PagedFlux listAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedFlux}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken, + private PagedFlux listAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String continuationToken, Context context) { return new PagedFlux<>( - () -> - listSinglePageAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken, - context), + () -> listSinglePageAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, continuationToken, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -428,30 +318,20 @@ private PagedFlux listAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedIterable}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName) { final String continuationToken = null; - return new PagedIterable<>( - listAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken)); + return new PagedIterable<>(listAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, continuationToken)); } /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -462,32 +342,20 @@ public PagedIterable list( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedIterable}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken, + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String continuationToken, Context context) { - return new PagedIterable<>( - listAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken, - context)); + return new PagedIterable<>(listAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, continuationToken, context)); } /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -498,36 +366,27 @@ public PagedIterable list( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability resource that extends a Target resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, + private Mono> getWithResponseAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, String capabilityName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -545,27 +404,15 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -577,37 +424,27 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability resource that extends a Target resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, + String capabilityName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -625,24 +462,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, + capabilityName, accept, context); } /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -655,26 +482,15 @@ private Mono> getWithResponseAsync( * @return a Capability resource that extends a Target resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { - return getWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono getAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName) { + return getWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, capabilityName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -688,28 +504,16 @@ private Mono getAsync( * @return a Capability resource that extends a Target resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + public Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context) { - return getWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - context) - .block(); + return getWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, capabilityName, context).block(); } /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -722,27 +526,15 @@ public Response getWithResponse( * @return a Capability resource that extends a Target resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CapabilityInner get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { - return getWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - Context.NONE) - .getValue(); + public CapabilityInner get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName) { + return getWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, capabilityName, Context.NONE).getValue(); } /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -755,33 +547,23 @@ public CapabilityInner get( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -799,27 +581,15 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -833,34 +603,24 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + private Mono> deleteWithResponseAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -878,24 +638,14 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, + capabilityName, accept, context); } /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -908,26 +658,15 @@ private Mono> deleteWithResponseAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { - return deleteWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName) - .flatMap(ignored -> Mono.empty()); + private Mono deleteAsync(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName) { + return deleteWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName).flatMap(ignored -> Mono.empty()); } /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -941,28 +680,16 @@ private Mono deleteAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + public Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context) { - return deleteWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - context) - .block(); + return deleteWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, context).block(); } /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -974,26 +701,15 @@ public Response deleteWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { - deleteWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - Context.NONE); + public void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName) { + deleteWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, capabilityName, Context.NONE); } /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1005,37 +721,27 @@ public void delete( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Capability resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, + String capabilityName, CapabilityInner capability) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -1058,28 +764,15 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, capability, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1092,38 +785,27 @@ private Mono> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Capability resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, + String capabilityName, CapabilityInner capability, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -1146,25 +828,14 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, capability, accept, context); } /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1178,28 +849,17 @@ private Mono> createOrUpdateWithResponseAsync( * @return model that represents a Capability resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + private Mono createOrUpdateAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, CapabilityInner capability) { - return createOrUpdateWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability) + return createOrUpdateWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, capability) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1214,30 +874,16 @@ private Mono createOrUpdateAsync( * @return model that represents a Capability resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability, - context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, + String capabilityName, CapabilityInner capability, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, capability, context).block(); } /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1251,36 +897,24 @@ public Response createOrUpdateWithResponse( * @return model that represents a Capability resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CapabilityInner createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + public CapabilityInner createOrUpdate(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, CapabilityInner capability) { - return createOrUpdateWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability, - Context.NONE) - .getValue(); + return createOrUpdateWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, capability, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Capability resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Capability resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1288,37 +922,28 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Capability resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Capability resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1326,23 +951,13 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesImpl.java index 524e853d9a4c6..c21aa6590ca61 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilitiesImpl.java @@ -21,94 +21,44 @@ public final class CapabilitiesImpl implements Capabilities { private final com.azure.resourcemanager.chaos.ChaosManager serviceManager; - public CapabilitiesImpl( - CapabilitiesClient innerClient, com.azure.resourcemanager.chaos.ChaosManager serviceManager) { + public CapabilitiesImpl(CapabilitiesClient innerClient, + com.azure.resourcemanager.chaos.ChaosManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - PagedIterable inner = - this - .serviceClient() - .list(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName); - return Utils.mapPage(inner, inner1 -> new CapabilityImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, targetName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CapabilityImpl(inner1, this.manager())); } - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken, + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String continuationToken, Context context) { - PagedIterable inner = - this - .serviceClient() - .list( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - continuationToken, - context); - return Utils.mapPage(inner, inner1 -> new CapabilityImpl(inner1, this.manager())); + PagedIterable inner = this.serviceClient().list(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, targetName, continuationToken, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CapabilityImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + public Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - context); + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + parentProviderNamespace, parentResourceType, parentResourceName, targetName, capabilityName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CapabilityImpl(inner.getValue(), this.manager())); } else { return null; } } - public Capability get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { - CapabilityInner inner = - this - .serviceClient() - .get( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName); + public Capability get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName) { + CapabilityInner inner = this.serviceClient().get(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName); if (inner != null) { return new CapabilityImpl(inner, this.manager()); } else { @@ -116,95 +66,38 @@ public Capability get( } } - public Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + public Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context) { - return this - .serviceClient() - .deleteWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - context); + return this.serviceClient().deleteWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, capabilityName, context); } - public void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName) { - this - .serviceClient() - .delete( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName); + public void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName) { + this.serviceClient().delete(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, capabilityName); } - public Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability, - Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability, - context); + public Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, + CapabilityInner capability, Context context) { + Response inner + = this.serviceClient().createOrUpdateWithResponse(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, targetName, capabilityName, capability, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CapabilityImpl(inner.getValue(), this.manager())); } else { return null; } } - public Capability createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + public Capability createOrUpdate(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, CapabilityInner capability) { - CapabilityInner inner = - this - .serviceClient() - .createOrUpdate( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - capabilityName, - capability); + CapabilityInner inner = this.serviceClient().createOrUpdate(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, targetName, capabilityName, capability); if (inner != null) { return new CapabilityImpl(inner, this.manager()); } else { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesClientImpl.java index fb5359984b2c1..b49dfa127debb 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.chaos.models.CapabilityTypeListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CapabilityTypesClient. */ +/** + * An instance of this class provides access to all the operations defined in CapabilityTypesClient. + */ public final class CapabilityTypesClientImpl implements CapabilityTypesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CapabilityTypesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of CapabilityTypesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ CapabilityTypesClientImpl(ChaosManagementClientImpl client) { - this.service = - RestProxy.create(CapabilityTypesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(CapabilityTypesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,50 +62,38 @@ public final class CapabilityTypesClientImpl implements CapabilityTypesClient { @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface CapabilityTypesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes/{targetTypeName}/capabilityTypes") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes/{targetTypeName}/capabilityTypes") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("locationName") String locationName, - @PathParam("targetTypeName") String targetTypeName, - @QueryParam("continuationToken") String continuationToken, - @HeaderParam("Accept") String accept, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("locationName") String locationName, @PathParam("targetTypeName") String targetTypeName, + @QueryParam("continuationToken") String continuationToken, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes/{targetTypeName}/capabilityTypes/{capabilityTypeName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes/{targetTypeName}/capabilityTypes/{capabilityTypeName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("locationName") String locationName, - @PathParam("targetTypeName") String targetTypeName, - @PathParam("capabilityTypeName") String capabilityTypeName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("locationName") String locationName, @PathParam("targetTypeName") String targetTypeName, + @PathParam("capabilityTypeName") String capabilityTypeName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. @@ -107,22 +101,18 @@ Mono> listNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Capability Type resources for given Target Type and location along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String locationName, String targetTypeName, String continuationToken) { + private Mono> listSinglePageAsync(String locationName, String targetTypeName, + String continuationToken) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -132,33 +122,16 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - targetTypeName, - continuationToken, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), locationName, targetTypeName, continuationToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. @@ -167,22 +140,18 @@ private Mono> listSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Capability Type resources for given Target Type and location along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String locationName, String targetTypeName, String continuationToken, Context context) { + private Mono> listSinglePageAsync(String locationName, String targetTypeName, + String continuationToken, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -193,68 +162,52 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - targetTypeName, - continuationToken, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), locationName, + targetTypeName, continuationToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedFlux}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String locationName, String targetTypeName, String continuationToken) { - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, targetTypeName, continuationToken), + private PagedFlux listAsync(String locationName, String targetTypeName, + String continuationToken) { + return new PagedFlux<>(() -> listSinglePageAsync(locationName, targetTypeName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedFlux}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String locationName, String targetTypeName) { final String continuationToken = null; - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, targetTypeName, continuationToken), + return new PagedFlux<>(() -> listSinglePageAsync(locationName, targetTypeName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. @@ -262,27 +215,26 @@ private PagedFlux listAsync(String locationName, String tar * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedFlux}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String locationName, String targetTypeName, String continuationToken, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, targetTypeName, continuationToken, context), + private PagedFlux listAsync(String locationName, String targetTypeName, + String continuationToken, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(locationName, targetTypeName, continuationToken, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedIterable}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String locationName, String targetTypeName) { @@ -292,7 +244,7 @@ public PagedIterable list(String locationName, String targe /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. @@ -300,18 +252,18 @@ public PagedIterable list(String locationName, String targe * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedIterable}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String locationName, String targetTypeName, String continuationToken, Context context) { + public PagedIterable list(String locationName, String targetTypeName, String continuationToken, + Context context) { return new PagedIterable<>(listAsync(locationName, targetTypeName, continuationToken, context)); } /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -319,22 +271,18 @@ public PagedIterable list( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability Type resource for given Target Type and location along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String locationName, String targetTypeName, String capabilityTypeName) { + private Mono> getWithResponseAsync(String locationName, String targetTypeName, + String capabilityTypeName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -348,24 +296,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - targetTypeName, - capabilityTypeName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), locationName, targetTypeName, capabilityTypeName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -374,22 +312,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability Type resource for given Target Type and location along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String locationName, String targetTypeName, String capabilityTypeName, Context context) { + private Mono> getWithResponseAsync(String locationName, String targetTypeName, + String capabilityTypeName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -403,21 +337,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - targetTypeName, - capabilityTypeName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + locationName, targetTypeName, capabilityTypeName, accept, context); } /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -434,7 +360,7 @@ private Mono getAsync(String locationName, String targetTyp /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -445,14 +371,14 @@ private Mono getAsync(String locationName, String targetTyp * @return a Capability Type resource for given Target Type and location along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String locationName, String targetTypeName, String capabilityTypeName, Context context) { + public Response getWithResponse(String locationName, String targetTypeName, + String capabilityTypeName, Context context) { return getWithResponseAsync(locationName, targetTypeName, capabilityTypeName, context).block(); } /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -468,14 +394,15 @@ public CapabilityTypeInner get(String locationName, String targetTypeName, Strin /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Capability Type resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Capability Type resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -483,37 +410,28 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Capability Type resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Capability Type resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -521,23 +439,13 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesImpl.java index d846aab402ea6..6a206147b5583 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/CapabilityTypesImpl.java @@ -21,33 +21,30 @@ public final class CapabilityTypesImpl implements CapabilityTypes { private final com.azure.resourcemanager.chaos.ChaosManager serviceManager; - public CapabilityTypesImpl( - CapabilityTypesClient innerClient, com.azure.resourcemanager.chaos.ChaosManager serviceManager) { + public CapabilityTypesImpl(CapabilityTypesClient innerClient, + com.azure.resourcemanager.chaos.ChaosManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String locationName, String targetTypeName) { PagedIterable inner = this.serviceClient().list(locationName, targetTypeName); - return Utils.mapPage(inner, inner1 -> new CapabilityTypeImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CapabilityTypeImpl(inner1, this.manager())); } - public PagedIterable list( - String locationName, String targetTypeName, String continuationToken, Context context) { - PagedIterable inner = - this.serviceClient().list(locationName, targetTypeName, continuationToken, context); - return Utils.mapPage(inner, inner1 -> new CapabilityTypeImpl(inner1, this.manager())); + public PagedIterable list(String locationName, String targetTypeName, String continuationToken, + Context context) { + PagedIterable inner + = this.serviceClient().list(locationName, targetTypeName, continuationToken, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CapabilityTypeImpl(inner1, this.manager())); } - public Response getWithResponse( - String locationName, String targetTypeName, String capabilityTypeName, Context context) { - Response inner = - this.serviceClient().getWithResponse(locationName, targetTypeName, capabilityTypeName, context); + public Response getWithResponse(String locationName, String targetTypeName, + String capabilityTypeName, Context context) { + Response inner + = this.serviceClient().getWithResponse(locationName, targetTypeName, capabilityTypeName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CapabilityTypeImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientBuilder.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientBuilder.java index 866e2f6186acd..0fe0b181ab68e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientBuilder.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientBuilder.java @@ -14,8 +14,10 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the ChaosManagementClientImpl type. */ -@ServiceClientBuilder(serviceClients = {ChaosManagementClientImpl.class}) +/** + * A builder for creating a new instance of the ChaosManagementClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { ChaosManagementClientImpl.class }) public final class ChaosManagementClientBuilder { /* * GUID that represents an Azure subscription ID. @@ -24,7 +26,7 @@ public final class ChaosManagementClientBuilder { /** * Sets GUID that represents an Azure subscription ID. - * + * * @param subscriptionId the subscriptionId value. * @return the ChaosManagementClientBuilder. */ @@ -40,7 +42,7 @@ public ChaosManagementClientBuilder subscriptionId(String subscriptionId) { /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the ChaosManagementClientBuilder. */ @@ -56,7 +58,7 @@ public ChaosManagementClientBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the ChaosManagementClientBuilder. */ @@ -72,7 +74,7 @@ public ChaosManagementClientBuilder environment(AzureEnvironment environment) { /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the ChaosManagementClientBuilder. */ @@ -88,7 +90,7 @@ public ChaosManagementClientBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the ChaosManagementClientBuilder. */ @@ -104,7 +106,7 @@ public ChaosManagementClientBuilder defaultPollInterval(Duration defaultPollInte /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the ChaosManagementClientBuilder. */ @@ -115,30 +117,20 @@ public ChaosManagementClientBuilder serializerAdapter(SerializerAdapter serializ /** * Builds an instance of ChaosManagementClientImpl with the provided parameters. - * + * * @return an instance of ChaosManagementClientImpl. */ public ChaosManagementClientImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - ChaosManagementClientImpl client = - new ChaosManagementClientImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - this.subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + ChaosManagementClientImpl client = new ChaosManagementClientImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientImpl.java index cffdad3b65792..d975b8c1dac07 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ChaosManagementClientImpl.java @@ -39,159 +39,187 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the ChaosManagementClientImpl type. */ +/** + * Initializes a new instance of the ChaosManagementClientImpl type. + */ @ServiceClient(builder = ChaosManagementClientBuilder.class) public final class ChaosManagementClientImpl implements ChaosManagementClient { - /** GUID that represents an Azure subscription ID. */ + /** + * GUID that represents an Azure subscription ID. + */ private final String subscriptionId; /** * Gets GUID that represents an Azure subscription ID. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The CapabilitiesClient object to access its operations. */ + /** + * The CapabilitiesClient object to access its operations. + */ private final CapabilitiesClient capabilities; /** * Gets the CapabilitiesClient object to access its operations. - * + * * @return the CapabilitiesClient object. */ public CapabilitiesClient getCapabilities() { return this.capabilities; } - /** The CapabilityTypesClient object to access its operations. */ + /** + * The CapabilityTypesClient object to access its operations. + */ private final CapabilityTypesClient capabilityTypes; /** * Gets the CapabilityTypesClient object to access its operations. - * + * * @return the CapabilityTypesClient object. */ public CapabilityTypesClient getCapabilityTypes() { return this.capabilityTypes; } - /** The ExperimentsClient object to access its operations. */ + /** + * The ExperimentsClient object to access its operations. + */ private final ExperimentsClient experiments; /** * Gets the ExperimentsClient object to access its operations. - * + * * @return the ExperimentsClient object. */ public ExperimentsClient getExperiments() { return this.experiments; } - /** The OperationStatusesClient object to access its operations. */ + /** + * The OperationStatusesClient object to access its operations. + */ private final OperationStatusesClient operationStatuses; /** * Gets the OperationStatusesClient object to access its operations. - * + * * @return the OperationStatusesClient object. */ public OperationStatusesClient getOperationStatuses() { return this.operationStatuses; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The TargetTypesClient object to access its operations. */ + /** + * The TargetTypesClient object to access its operations. + */ private final TargetTypesClient targetTypes; /** * Gets the TargetTypesClient object to access its operations. - * + * * @return the TargetTypesClient object. */ public TargetTypesClient getTargetTypes() { return this.targetTypes; } - /** The TargetsClient object to access its operations. */ + /** + * The TargetsClient object to access its operations. + */ private final TargetsClient targets; /** * Gets the TargetsClient object to access its operations. - * + * * @return the TargetsClient object. */ public TargetsClient getTargets() { @@ -200,7 +228,7 @@ public TargetsClient getTargets() { /** * Initializes an instance of ChaosManagementClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -208,19 +236,14 @@ public TargetsClient getTargets() { * @param subscriptionId GUID that represents an Azure subscription ID. * @param endpoint server parameter. */ - ChaosManagementClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + ChaosManagementClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-11-01"; + this.apiVersion = "2024-01-01"; this.capabilities = new CapabilitiesClientImpl(this); this.capabilityTypes = new CapabilityTypesClientImpl(this); this.experiments = new ExperimentsClientImpl(this); @@ -232,7 +255,7 @@ public TargetsClient getTargets() { /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -241,7 +264,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -251,7 +274,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -261,26 +284,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -293,19 +305,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionDetailsImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionDetailsImpl.java index 9e53868c14023..f8574189f0640 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionDetailsImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionDetailsImpl.java @@ -14,8 +14,8 @@ public final class ExperimentExecutionDetailsImpl implements ExperimentExecution private final com.azure.resourcemanager.chaos.ChaosManager serviceManager; - ExperimentExecutionDetailsImpl( - ExperimentExecutionDetailsInner innerObject, com.azure.resourcemanager.chaos.ChaosManager serviceManager) { + ExperimentExecutionDetailsImpl(ExperimentExecutionDetailsInner innerObject, + com.azure.resourcemanager.chaos.ChaosManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionImpl.java index 6b494ead16fa5..5e188019d810a 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentExecutionImpl.java @@ -13,8 +13,8 @@ public final class ExperimentExecutionImpl implements ExperimentExecution { private final com.azure.resourcemanager.chaos.ChaosManager serviceManager; - ExperimentExecutionImpl( - ExperimentExecutionInner innerObject, com.azure.resourcemanager.chaos.ChaosManager serviceManager) { + ExperimentExecutionImpl(ExperimentExecutionInner innerObject, + com.azure.resourcemanager.chaos.ChaosManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentImpl.java index 6681415751dd6..6960c54c54c2f 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentImpl.java @@ -110,20 +110,14 @@ public ExperimentImpl withExistingResourceGroup(String resourceGroupName) { } public Experiment create() { - this.innerObject = - serviceManager - .serviceClient() - .getExperiments() - .createOrUpdate(resourceGroupName, experimentName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getExperiments().createOrUpdate(resourceGroupName, + experimentName, this.innerModel(), Context.NONE); return this; } public Experiment create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getExperiments() - .createOrUpdate(resourceGroupName, experimentName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getExperiments().createOrUpdate(resourceGroupName, + experimentName, this.innerModel(), context); return this; } @@ -139,47 +133,33 @@ public ExperimentImpl update() { } public Experiment apply() { - this.innerObject = - serviceManager - .serviceClient() - .getExperiments() - .update(resourceGroupName, experimentName, updateExperiment, Context.NONE); + this.innerObject = serviceManager.serviceClient().getExperiments().update(resourceGroupName, experimentName, + updateExperiment, Context.NONE); return this; } public Experiment apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getExperiments() - .update(resourceGroupName, experimentName, updateExperiment, context); + this.innerObject = serviceManager.serviceClient().getExperiments().update(resourceGroupName, experimentName, + updateExperiment, context); return this; } ExperimentImpl(ExperimentInner innerObject, com.azure.resourcemanager.chaos.ChaosManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.experimentName = Utils.getValueFromIdByName(innerObject.id(), "experiments"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.experimentName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "experiments"); } public Experiment refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getExperiments() - .getByResourceGroupWithResponse(resourceGroupName, experimentName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getExperiments() + .getByResourceGroupWithResponse(resourceGroupName, experimentName, Context.NONE).getValue(); return this; } public Experiment refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getExperiments() - .getByResourceGroupWithResponse(resourceGroupName, experimentName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getExperiments() + .getByResourceGroupWithResponse(resourceGroupName, experimentName, context).getValue(); return this; } @@ -220,8 +200,13 @@ public ExperimentImpl withSelectors(List selectors) { } public ExperimentImpl withTags(Map tags) { - this.innerModel().withTags(tags); - return this; + if (isInCreateMode()) { + this.innerModel().withTags(tags); + return this; + } else { + this.updateExperiment.withTags(tags); + return this; + } } public ExperimentImpl withIdentity(ResourceIdentity identity) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsClientImpl.java index 467c951c9313c..91bb521156901 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsClientImpl.java @@ -44,22 +44,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ExperimentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ExperimentsClient. + */ public final class ExperimentsClientImpl implements ExperimentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ExperimentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of ExperimentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ExperimentsClientImpl(ChaosManagementClientImpl client) { - this.service = - RestProxy.create(ExperimentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ExperimentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -70,300 +76,204 @@ public final class ExperimentsClientImpl implements ExperimentsClient { @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface ExperimentsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/experiments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("running") Boolean running, - @QueryParam("continuationToken") String continuationToken, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("running") Boolean running, @QueryParam("continuationToken") String continuationToken, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("running") Boolean running, - @QueryParam("continuationToken") String continuationToken, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("running") Boolean running, + @QueryParam("continuationToken") String continuationToken, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") - @ExpectedResponses({202}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("experimentName") String experimentName, - @BodyParam("application/json") ExperimentInner experiment, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ExperimentInner experiment, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") - @ExpectedResponses({202}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("experimentName") String experimentName, - @BodyParam("application/json") ExperimentUpdate experiment, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ExperimentUpdate experiment, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/cancel") - @ExpectedResponses({202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/cancel") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> cancel( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> cancel(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/start") - @ExpectedResponses({202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/start") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> start( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> start(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/executions") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/executions") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAllExecutions( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> listAllExecutions(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/executions/{executionId}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/executions/{executionId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getExecution( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> getExecution(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @PathParam("executionId") String executionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @PathParam("executionId") String executionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/executions/{executionId}/getExecutionDetails") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Chaos/experiments/{experimentName}/executions/{executionId}/getExecutionDetails") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> executionDetails( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> executionDetails(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("experimentName") String experimentName, - @PathParam("executionId") String executionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("experimentName") String experimentName, @PathParam("executionId") String executionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAllNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listAllNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listAllExecutionsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Boolean running, String continuationToken) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - running, - continuationToken, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), running, continuationToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - Boolean running, String continuationToken, Context context) { + private Mono> listSinglePageAsync(Boolean running, String continuationToken, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - running, - continuationToken, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), running, + continuationToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -372,13 +282,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Boolean running, String continuationToken) { - return new PagedFlux<>( - () -> listSinglePageAsync(running, continuationToken), nextLink -> listAllNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(running, continuationToken), + nextLink -> listAllNextSinglePageAsync(nextLink)); } /** * Get a list of Experiment resources in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a subscription as paginated response with {@link PagedFlux}. @@ -387,15 +297,15 @@ private PagedFlux listAsync(Boolean running, String continuatio private PagedFlux listAsync() { final Boolean running = null; final String continuationToken = null; - return new PagedFlux<>( - () -> listSinglePageAsync(running, continuationToken), nextLink -> listAllNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(running, continuationToken), + nextLink -> listAllNextSinglePageAsync(nextLink)); } /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -405,14 +315,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Boolean running, String continuationToken, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(running, continuationToken, context), + return new PagedFlux<>(() -> listSinglePageAsync(running, continuationToken, context), nextLink -> listAllNextSinglePageAsync(nextLink, context)); } /** * Get a list of Experiment resources in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a subscription as paginated response with {@link PagedIterable}. @@ -426,9 +335,9 @@ public PagedIterable list() { /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -443,31 +352,27 @@ public PagedIterable list(Boolean running, String continuationT /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Boolean running, String continuationToken) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Boolean running, String continuationToken) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -475,58 +380,37 @@ private Mono> listByResourceGroupSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - running, - continuationToken, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, running, continuationToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a resource group along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Boolean running, String continuationToken, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Boolean running, String continuationToken, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -535,32 +419,18 @@ private Mono> listByResourceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - running, - continuationToken, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, running, continuationToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -568,16 +438,15 @@ private Mono> listByResourceGroupSinglePageAsync( * @return a list of Experiment resources in a resource group as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, Boolean running, String continuationToken) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, running, continuationToken), + private PagedFlux listByResourceGroupAsync(String resourceGroupName, Boolean running, + String continuationToken) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, running, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -588,17 +457,16 @@ private PagedFlux listByResourceGroupAsync( private PagedFlux listByResourceGroupAsync(String resourceGroupName) { final Boolean running = null; final String continuationToken = null; - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, running, continuationToken), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, running, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -607,8 +475,8 @@ private PagedFlux listByResourceGroupAsync(String resourceGroup * @return a list of Experiment resources in a resource group as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, Boolean running, String continuationToken, Context context) { + private PagedFlux listByResourceGroupAsync(String resourceGroupName, Boolean running, + String continuationToken, Context context) { return new PagedFlux<>( () -> listByResourceGroupSinglePageAsync(resourceGroupName, running, continuationToken, context), nextLink -> listNextSinglePageAsync(nextLink, context)); @@ -616,7 +484,7 @@ private PagedFlux listByResourceGroupAsync( /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -632,10 +500,10 @@ public PagedIterable listByResourceGroup(String resourceGroupNa /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -644,14 +512,14 @@ public PagedIterable listByResourceGroup(String resourceGroupNa * @return a list of Experiment resources in a resource group as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup( - String resourceGroupName, Boolean running, String continuationToken, Context context) { + public PagedIterable listByResourceGroup(String resourceGroupName, Boolean running, + String continuationToken, Context context) { return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, running, continuationToken, context)); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -662,16 +530,12 @@ public PagedIterable listByResourceGroup( @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String experimentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -682,23 +546,14 @@ private Mono>> deleteWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -708,19 +563,15 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String experimentName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String experimentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -731,20 +582,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, experimentName, accept, context); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -755,15 +599,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String experimentName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, experimentName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -773,18 +615,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String experimentName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String experimentName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, experimentName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -799,7 +640,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -809,14 +650,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String experimentName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String experimentName, + Context context) { return this.beginDeleteAsync(resourceGroupName, experimentName, context).getSyncPoller(); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -826,14 +667,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String experimentName) { - return beginDeleteAsync(resourceGroupName, experimentName) - .last() + return beginDeleteAsync(resourceGroupName, experimentName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -844,14 +684,13 @@ private Mono deleteAsync(String resourceGroupName, String experimentName) */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String experimentName, Context context) { - return beginDeleteAsync(resourceGroupName, experimentName, context) - .last() + return beginDeleteAsync(resourceGroupName, experimentName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -865,7 +704,7 @@ public void delete(String resourceGroupName, String experimentName) { /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -880,7 +719,7 @@ public void delete(String resourceGroupName, String experimentName, Context cont /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -889,19 +728,15 @@ public void delete(String resourceGroupName, String experimentName, Context cont * @return a Experiment resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String experimentName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String experimentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -912,23 +747,14 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -938,19 +764,15 @@ private Mono> getByResourceGroupWithResponseAsync( * @return a Experiment resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String experimentName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String experimentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -961,20 +783,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, accept, context); } /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -990,7 +805,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1000,14 +815,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName, * @return a Experiment resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String experimentName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String experimentName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, experimentName, context).block(); } /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1022,7 +837,7 @@ public ExperimentInner getByResourceGroup(String resourceGroupName, String exper /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1030,22 +845,18 @@ public ExperimentInner getByResourceGroup(String resourceGroupName, String exper * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Experiment resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String experimentName, ExperimentInner experiment) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String experimentName, ExperimentInner experiment) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1061,24 +872,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - experiment, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, experiment, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1087,22 +888,18 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Experiment resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String experimentName, ExperimentInner experiment, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1118,21 +915,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - experiment, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, experiment, accept, context); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1142,23 +931,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ExperimentInner> beginCreateOrUpdateAsync( - String resourceGroupName, String experimentName, ExperimentInner experiment) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, experimentName, experiment); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ExperimentInner.class, - ExperimentInner.class, - this.client.getContext()); + private PollerFlux, ExperimentInner> beginCreateOrUpdateAsync(String resourceGroupName, + String experimentName, ExperimentInner experiment) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, experimentName, experiment); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ExperimentInner.class, ExperimentInner.class, this.client.getContext()); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1169,20 +952,18 @@ private PollerFlux, ExperimentInner> beginCreateOrUp * @return the {@link PollerFlux} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ExperimentInner> beginCreateOrUpdateAsync( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context) { + private PollerFlux, ExperimentInner> beginCreateOrUpdateAsync(String resourceGroupName, + String experimentName, ExperimentInner experiment, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, experimentName, experiment, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ExperimentInner.class, ExperimentInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, experimentName, experiment, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ExperimentInner.class, ExperimentInner.class, context); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1192,14 +973,14 @@ private PollerFlux, ExperimentInner> beginCreateOrUp * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ExperimentInner> beginCreateOrUpdate( - String resourceGroupName, String experimentName, ExperimentInner experiment) { + public SyncPoller, ExperimentInner> beginCreateOrUpdate(String resourceGroupName, + String experimentName, ExperimentInner experiment) { return this.beginCreateOrUpdateAsync(resourceGroupName, experimentName, experiment).getSyncPoller(); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1210,14 +991,14 @@ public SyncPoller, ExperimentInner> beginCreateOrUpd * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ExperimentInner> beginCreateOrUpdate( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context) { + public SyncPoller, ExperimentInner> beginCreateOrUpdate(String resourceGroupName, + String experimentName, ExperimentInner experiment, Context context) { return this.beginCreateOrUpdateAsync(resourceGroupName, experimentName, experiment, context).getSyncPoller(); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1227,16 +1008,15 @@ public SyncPoller, ExperimentInner> beginCreateOrUpd * @return model that represents a Experiment resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String experimentName, ExperimentInner experiment) { - return beginCreateOrUpdateAsync(resourceGroupName, experimentName, experiment) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String experimentName, + ExperimentInner experiment) { + return beginCreateOrUpdateAsync(resourceGroupName, experimentName, experiment).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1247,16 +1027,15 @@ private Mono createOrUpdateAsync( * @return model that represents a Experiment resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, experimentName, experiment, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String experimentName, + ExperimentInner experiment, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, experimentName, experiment, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1272,7 +1051,7 @@ public ExperimentInner createOrUpdate(String resourceGroupName, String experimen /** * Create or update a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Experiment resource to be created or updated. @@ -1283,14 +1062,14 @@ public ExperimentInner createOrUpdate(String resourceGroupName, String experimen * @return model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ExperimentInner createOrUpdate( - String resourceGroupName, String experimentName, ExperimentInner experiment, Context context) { + public ExperimentInner createOrUpdate(String resourceGroupName, String experimentName, ExperimentInner experiment, + Context context) { return createOrUpdateAsync(resourceGroupName, experimentName, experiment, context).block(); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1298,22 +1077,18 @@ public ExperimentInner createOrUpdate( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Experiment resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String experimentName, ExperimentUpdate experiment) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String experimentName, + ExperimentUpdate experiment) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1329,24 +1104,14 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - experiment, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, experiment, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1355,22 +1120,18 @@ private Mono>> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Experiment resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String experimentName, + ExperimentUpdate experiment, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1386,21 +1147,13 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - experiment, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, experimentName, experiment, accept, context); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1410,22 +1163,16 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ExperimentInner> beginUpdateAsync( - String resourceGroupName, String experimentName, ExperimentUpdate experiment) { + private PollerFlux, ExperimentInner> beginUpdateAsync(String resourceGroupName, + String experimentName, ExperimentUpdate experiment) { Mono>> mono = updateWithResponseAsync(resourceGroupName, experimentName, experiment); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ExperimentInner.class, - ExperimentInner.class, - this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ExperimentInner.class, ExperimentInner.class, this.client.getContext()); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1436,20 +1183,18 @@ private PollerFlux, ExperimentInner> beginUpdateAsyn * @return the {@link PollerFlux} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ExperimentInner> beginUpdateAsync( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context) { + private PollerFlux, ExperimentInner> beginUpdateAsync(String resourceGroupName, + String experimentName, ExperimentUpdate experiment, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, experimentName, experiment, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ExperimentInner.class, ExperimentInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, experimentName, experiment, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ExperimentInner.class, ExperimentInner.class, context); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1459,14 +1204,14 @@ private PollerFlux, ExperimentInner> beginUpdateAsyn * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ExperimentInner> beginUpdate( - String resourceGroupName, String experimentName, ExperimentUpdate experiment) { + public SyncPoller, ExperimentInner> beginUpdate(String resourceGroupName, + String experimentName, ExperimentUpdate experiment) { return this.beginUpdateAsync(resourceGroupName, experimentName, experiment).getSyncPoller(); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1477,14 +1222,14 @@ public SyncPoller, ExperimentInner> beginUpdate( * @return the {@link SyncPoller} for polling of model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ExperimentInner> beginUpdate( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context) { + public SyncPoller, ExperimentInner> beginUpdate(String resourceGroupName, + String experimentName, ExperimentUpdate experiment, Context context) { return this.beginUpdateAsync(resourceGroupName, experimentName, experiment, context).getSyncPoller(); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1494,16 +1239,15 @@ public SyncPoller, ExperimentInner> beginUpdate( * @return model that represents a Experiment resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String experimentName, ExperimentUpdate experiment) { - return beginUpdateAsync(resourceGroupName, experimentName, experiment) - .last() + private Mono updateAsync(String resourceGroupName, String experimentName, + ExperimentUpdate experiment) { + return beginUpdateAsync(resourceGroupName, experimentName, experiment).last() .flatMap(this.client::getLroFinalResultOrError); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1514,16 +1258,15 @@ private Mono updateAsync( * @return model that represents a Experiment resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context) { - return beginUpdateAsync(resourceGroupName, experimentName, experiment, context) - .last() + private Mono updateAsync(String resourceGroupName, String experimentName, + ExperimentUpdate experiment, Context context) { + return beginUpdateAsync(resourceGroupName, experimentName, experiment, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1539,7 +1282,7 @@ public ExperimentInner update(String resourceGroupName, String experimentName, E /** * The operation to update an experiment. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param experiment Parameters supplied to the Update experiment operation. @@ -1550,14 +1293,14 @@ public ExperimentInner update(String resourceGroupName, String experimentName, E * @return model that represents a Experiment resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ExperimentInner update( - String resourceGroupName, String experimentName, ExperimentUpdate experiment, Context context) { + public ExperimentInner update(String resourceGroupName, String experimentName, ExperimentUpdate experiment, + Context context) { return updateAsync(resourceGroupName, experimentName, experiment, context).block(); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1568,16 +1311,12 @@ public ExperimentInner update( @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> cancelWithResponseAsync(String resourceGroupName, String experimentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1588,23 +1327,14 @@ private Mono>> cancelWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .cancel( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context)) + .withContext(context -> service.cancel(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1614,19 +1344,15 @@ private Mono>> cancelWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> cancelWithResponseAsync( - String resourceGroupName, String experimentName, Context context) { + private Mono>> cancelWithResponseAsync(String resourceGroupName, String experimentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1637,20 +1363,13 @@ private Mono>> cancelWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .cancel( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context); + return service.cancel(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, experimentName, accept, context); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1661,15 +1380,13 @@ private Mono>> cancelWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginCancelAsync(String resourceGroupName, String experimentName) { Mono>> mono = cancelWithResponseAsync(resourceGroupName, experimentName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1679,18 +1396,17 @@ private PollerFlux, Void> beginCancelAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginCancelAsync( - String resourceGroupName, String experimentName, Context context) { + private PollerFlux, Void> beginCancelAsync(String resourceGroupName, String experimentName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = cancelWithResponseAsync(resourceGroupName, experimentName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1705,7 +1421,7 @@ public SyncPoller, Void> beginCancel(String resourceGroupName, /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1715,14 +1431,14 @@ public SyncPoller, Void> beginCancel(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginCancel( - String resourceGroupName, String experimentName, Context context) { + public SyncPoller, Void> beginCancel(String resourceGroupName, String experimentName, + Context context) { return this.beginCancelAsync(resourceGroupName, experimentName, context).getSyncPoller(); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1732,14 +1448,13 @@ public SyncPoller, Void> beginCancel( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono cancelAsync(String resourceGroupName, String experimentName) { - return beginCancelAsync(resourceGroupName, experimentName) - .last() + return beginCancelAsync(resourceGroupName, experimentName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1750,14 +1465,13 @@ private Mono cancelAsync(String resourceGroupName, String experimentName) */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono cancelAsync(String resourceGroupName, String experimentName, Context context) { - return beginCancelAsync(resourceGroupName, experimentName, context) - .last() + return beginCancelAsync(resourceGroupName, experimentName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1771,7 +1485,7 @@ public void cancel(String resourceGroupName, String experimentName) { /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1786,7 +1500,7 @@ public void cancel(String resourceGroupName, String experimentName, Context cont /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1797,16 +1511,12 @@ public void cancel(String resourceGroupName, String experimentName, Context cont @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> startWithResponseAsync(String resourceGroupName, String experimentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1817,23 +1527,14 @@ private Mono>> startWithResponseAsync(String resourceG } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .start( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context)) + .withContext(context -> service.start(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1843,19 +1544,15 @@ private Mono>> startWithResponseAsync(String resourceG * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> startWithResponseAsync( - String resourceGroupName, String experimentName, Context context) { + private Mono>> startWithResponseAsync(String resourceGroupName, String experimentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1866,20 +1563,13 @@ private Mono>> startWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .start( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context); + return service.start(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, experimentName, accept, context); } /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1890,15 +1580,13 @@ private Mono>> startWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginStartAsync(String resourceGroupName, String experimentName) { Mono>> mono = startWithResponseAsync(resourceGroupName, experimentName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1908,18 +1596,17 @@ private PollerFlux, Void> beginStartAsync(String resourceGroupN * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginStartAsync( - String resourceGroupName, String experimentName, Context context) { + private PollerFlux, Void> beginStartAsync(String resourceGroupName, String experimentName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = startWithResponseAsync(resourceGroupName, experimentName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1934,7 +1621,7 @@ public SyncPoller, Void> beginStart(String resourceGroupName, S /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1944,14 +1631,14 @@ public SyncPoller, Void> beginStart(String resourceGroupName, S * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginStart( - String resourceGroupName, String experimentName, Context context) { + public SyncPoller, Void> beginStart(String resourceGroupName, String experimentName, + Context context) { return this.beginStartAsync(resourceGroupName, experimentName, context).getSyncPoller(); } /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1966,7 +1653,7 @@ private Mono startAsync(String resourceGroupName, String experimentName) { /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -1977,14 +1664,13 @@ private Mono startAsync(String resourceGroupName, String experimentName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono startAsync(String resourceGroupName, String experimentName, Context context) { - return beginStartAsync(resourceGroupName, experimentName, context) - .last() + return beginStartAsync(resourceGroupName, experimentName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1998,7 +1684,7 @@ public void start(String resourceGroupName, String experimentName) { /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -2013,29 +1699,25 @@ public void start(String resourceGroupName, String experimentName, Context conte /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of executions of an Experiment resource along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAllExecutionsSinglePageAsync( - String resourceGroupName, String experimentName) { + private Mono> listAllExecutionsSinglePageAsync(String resourceGroupName, + String experimentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2046,32 +1728,16 @@ private Mono> listAllExecutionsSinglePag } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listAllExecutions( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listAllExecutions(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -2079,22 +1745,18 @@ private Mono> listAllExecutionsSinglePag * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of executions of an Experiment resource along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAllExecutionsSinglePageAsync( - String resourceGroupName, String experimentName, Context context) { + private Mono> listAllExecutionsSinglePageAsync(String resourceGroupName, + String experimentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2106,28 +1768,15 @@ private Mono> listAllExecutionsSinglePag final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listAllExecutions( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listAllExecutions(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, experimentName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2136,16 +1785,15 @@ private Mono> listAllExecutionsSinglePag * @return a list of executions of an Experiment resource as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAllExecutionsAsync( - String resourceGroupName, String experimentName) { - return new PagedFlux<>( - () -> listAllExecutionsSinglePageAsync(resourceGroupName, experimentName), + private PagedFlux listAllExecutionsAsync(String resourceGroupName, + String experimentName) { + return new PagedFlux<>(() -> listAllExecutionsSinglePageAsync(resourceGroupName, experimentName), nextLink -> listAllExecutionsNextSinglePageAsync(nextLink)); } /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -2155,16 +1803,15 @@ private PagedFlux listAllExecutionsAsync( * @return a list of executions of an Experiment resource as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAllExecutionsAsync( - String resourceGroupName, String experimentName, Context context) { - return new PagedFlux<>( - () -> listAllExecutionsSinglePageAsync(resourceGroupName, experimentName, context), + private PagedFlux listAllExecutionsAsync(String resourceGroupName, String experimentName, + Context context) { + return new PagedFlux<>(() -> listAllExecutionsSinglePageAsync(resourceGroupName, experimentName, context), nextLink -> listAllExecutionsNextSinglePageAsync(nextLink, context)); } /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2179,7 +1826,7 @@ public PagedIterable listAllExecutions(String resource /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -2189,37 +1836,33 @@ public PagedIterable listAllExecutions(String resource * @return a list of executions of an Experiment resource as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAllExecutions( - String resourceGroupName, String experimentName, Context context) { + public PagedIterable listAllExecutions(String resourceGroupName, String experimentName, + Context context) { return new PagedIterable<>(listAllExecutionsAsync(resourceGroupName, experimentName, context)); } /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an execution of an Experiment resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return an execution of an Experiment resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getExecutionWithResponseAsync( - String resourceGroupName, String experimentName, String executionId) { + private Mono> getExecutionWithResponseAsync(String resourceGroupName, + String experimentName, String executionId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2233,24 +1876,14 @@ private Mono> getExecutionWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getExecution( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - executionId, - accept, - context)) + .withContext(context -> service.getExecution(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, executionId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2258,23 +1891,19 @@ private Mono> getExecutionWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an execution of an Experiment resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return an execution of an Experiment resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getExecutionWithResponseAsync( - String resourceGroupName, String experimentName, String executionId, Context context) { + private Mono> getExecutionWithResponseAsync(String resourceGroupName, + String experimentName, String executionId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2288,21 +1917,13 @@ private Mono> getExecutionWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getExecution( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - executionId, - accept, - context); + return service.getExecution(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, executionId, accept, context); } /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2312,15 +1933,15 @@ private Mono> getExecutionWithResponseAsync( * @return an execution of an Experiment resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getExecutionAsync( - String resourceGroupName, String experimentName, String executionId) { + private Mono getExecutionAsync(String resourceGroupName, String experimentName, + String executionId) { return getExecutionWithResponseAsync(resourceGroupName, experimentName, executionId) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2331,14 +1952,14 @@ private Mono getExecutionAsync( * @return an execution of an Experiment resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getExecutionWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context) { + public Response getExecutionWithResponse(String resourceGroupName, String experimentName, + String executionId, Context context) { return getExecutionWithResponseAsync(resourceGroupName, experimentName, executionId, context).block(); } /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2354,7 +1975,7 @@ public ExperimentExecutionInner getExecution(String resourceGroupName, String ex /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2362,22 +1983,18 @@ public ExperimentExecutionInner getExecution(String resourceGroupName, String ex * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents the execution details of an Experiment along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> executionDetailsWithResponseAsync( - String resourceGroupName, String experimentName, String executionId) { + private Mono> executionDetailsWithResponseAsync(String resourceGroupName, + String experimentName, String executionId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2391,24 +2008,14 @@ private Mono> executionDetailsWithResp } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .executionDetails( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - executionId, - accept, - context)) + .withContext(context -> service.executionDetails(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, executionId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2417,22 +2024,18 @@ private Mono> executionDetailsWithResp * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents the execution details of an Experiment along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> executionDetailsWithResponseAsync( - String resourceGroupName, String experimentName, String executionId, Context context) { + private Mono> executionDetailsWithResponseAsync(String resourceGroupName, + String experimentName, String executionId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2446,21 +2049,13 @@ private Mono> executionDetailsWithResp } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .executionDetails( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - experimentName, - executionId, - accept, - context); + return service.executionDetails(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, experimentName, executionId, accept, context); } /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2470,15 +2065,15 @@ private Mono> executionDetailsWithResp * @return model that represents the execution details of an Experiment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono executionDetailsAsync( - String resourceGroupName, String experimentName, String executionId) { + private Mono executionDetailsAsync(String resourceGroupName, String experimentName, + String executionId) { return executionDetailsWithResponseAsync(resourceGroupName, experimentName, executionId) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2489,14 +2084,14 @@ private Mono executionDetailsAsync( * @return model that represents the execution details of an Experiment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response executionDetailsWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context) { + public Response executionDetailsWithResponse(String resourceGroupName, + String experimentName, String executionId, Context context) { return executionDetailsWithResponseAsync(resourceGroupName, experimentName, executionId, context).block(); } /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -2506,21 +2101,22 @@ public Response executionDetailsWithResponse( * @return model that represents the execution details of an Experiment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ExperimentExecutionDetailsInner executionDetails( - String resourceGroupName, String experimentName, String executionId) { + public ExperimentExecutionDetailsInner executionDetails(String resourceGroupName, String experimentName, + String executionId) { return executionDetailsWithResponse(resourceGroupName, experimentName, executionId, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Experiment resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Experiment resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllNextSinglePageAsync(String nextLink) { @@ -2528,37 +2124,29 @@ private Mono> listAllNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listAllNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Experiment resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Experiment resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllNextSinglePageAsync(String nextLink, Context context) { @@ -2566,36 +2154,27 @@ private Mono> listAllNextSinglePageAsync(String n return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAllNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAllNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Experiment resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Experiment resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -2603,37 +2182,28 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Experiment resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Experiment resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -2641,36 +2211,27 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Experiment executions and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Experiment executions and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllExecutionsNextSinglePageAsync(String nextLink) { @@ -2678,62 +2239,44 @@ private Mono> listAllExecutionsNextSingl return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listAllExecutionsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Experiment executions and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Experiment executions and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAllExecutionsNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listAllExecutionsNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAllExecutionsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAllExecutionsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsImpl.java index cc81637e23b15..86e0c55e9cafe 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ExperimentsImpl.java @@ -32,24 +32,24 @@ public ExperimentsImpl(ExperimentsClient innerClient, com.azure.resourcemanager. public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); } public PagedIterable list(Boolean running, String continuationToken, Context context) { PagedIterable inner = this.serviceClient().list(running, continuationToken, context); - return Utils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); } - public PagedIterable listByResourceGroup( - String resourceGroupName, Boolean running, String continuationToken, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, running, continuationToken, context); - return Utils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); + public PagedIterable listByResourceGroup(String resourceGroupName, Boolean running, + String continuationToken, Context context) { + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, running, continuationToken, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExperimentImpl(inner1, this.manager())); } public void deleteByResourceGroup(String resourceGroupName, String experimentName) { @@ -60,15 +60,12 @@ public void delete(String resourceGroupName, String experimentName, Context cont this.serviceClient().delete(resourceGroupName, experimentName, context); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String experimentName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, experimentName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String experimentName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, experimentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ExperimentImpl(inner.getValue(), this.manager())); } else { return null; @@ -101,27 +98,24 @@ public void start(String resourceGroupName, String experimentName, Context conte } public PagedIterable listAllExecutions(String resourceGroupName, String experimentName) { - PagedIterable inner = - this.serviceClient().listAllExecutions(resourceGroupName, experimentName); - return Utils.mapPage(inner, inner1 -> new ExperimentExecutionImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listAllExecutions(resourceGroupName, experimentName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExperimentExecutionImpl(inner1, this.manager())); } - public PagedIterable listAllExecutions( - String resourceGroupName, String experimentName, Context context) { - PagedIterable inner = - this.serviceClient().listAllExecutions(resourceGroupName, experimentName, context); - return Utils.mapPage(inner, inner1 -> new ExperimentExecutionImpl(inner1, this.manager())); + public PagedIterable listAllExecutions(String resourceGroupName, String experimentName, + Context context) { + PagedIterable inner + = this.serviceClient().listAllExecutions(resourceGroupName, experimentName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExperimentExecutionImpl(inner1, this.manager())); } - public Response getExecutionWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context) { - Response inner = - this.serviceClient().getExecutionWithResponse(resourceGroupName, experimentName, executionId, context); + public Response getExecutionWithResponse(String resourceGroupName, String experimentName, + String executionId, Context context) { + Response inner + = this.serviceClient().getExecutionWithResponse(resourceGroupName, experimentName, executionId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ExperimentExecutionImpl(inner.getValue(), this.manager())); } else { return null; @@ -129,8 +123,8 @@ public Response getExecutionWithResponse( } public ExperimentExecution getExecution(String resourceGroupName, String experimentName, String executionId) { - ExperimentExecutionInner inner = - this.serviceClient().getExecution(resourceGroupName, experimentName, executionId); + ExperimentExecutionInner inner + = this.serviceClient().getExecution(resourceGroupName, experimentName, executionId); if (inner != null) { return new ExperimentExecutionImpl(inner, this.manager()); } else { @@ -138,25 +132,22 @@ public ExperimentExecution getExecution(String resourceGroupName, String experim } } - public Response executionDetailsWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context) { - Response inner = - this.serviceClient().executionDetailsWithResponse(resourceGroupName, experimentName, executionId, context); + public Response executionDetailsWithResponse(String resourceGroupName, + String experimentName, String executionId, Context context) { + Response inner = this.serviceClient() + .executionDetailsWithResponse(resourceGroupName, experimentName, executionId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ExperimentExecutionDetailsImpl(inner.getValue(), this.manager())); } else { return null; } } - public ExperimentExecutionDetails executionDetails( - String resourceGroupName, String experimentName, String executionId) { - ExperimentExecutionDetailsInner inner = - this.serviceClient().executionDetails(resourceGroupName, experimentName, executionId); + public ExperimentExecutionDetails executionDetails(String resourceGroupName, String experimentName, + String executionId) { + ExperimentExecutionDetailsInner inner + = this.serviceClient().executionDetails(resourceGroupName, experimentName, executionId); if (inner != null) { return new ExperimentExecutionDetailsImpl(inner, this.manager()); } else { @@ -165,77 +156,57 @@ public ExperimentExecutionDetails executionDetails( } public Experiment getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String experimentName = Utils.getValueFromIdByName(id, "experiments"); + String experimentName = ResourceManagerUtils.getValueFromIdByName(id, "experiments"); if (experimentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, experimentName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String experimentName = Utils.getValueFromIdByName(id, "experiments"); + String experimentName = ResourceManagerUtils.getValueFromIdByName(id, "experiments"); if (experimentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, experimentName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String experimentName = Utils.getValueFromIdByName(id, "experiments"); + String experimentName = ResourceManagerUtils.getValueFromIdByName(id, "experiments"); if (experimentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); } this.delete(resourceGroupName, experimentName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String experimentName = Utils.getValueFromIdByName(id, "experiments"); + String experimentName = ResourceManagerUtils.getValueFromIdByName(id, "experiments"); if (experimentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'experiments'.", id))); } this.delete(resourceGroupName, experimentName, context); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesClientImpl.java index 9fd0d252fbaa7..a163e34ece0fc 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesClientImpl.java @@ -25,22 +25,28 @@ import com.azure.resourcemanager.chaos.fluent.models.OperationStatusInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationStatusesClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationStatusesClient. + */ public final class OperationStatusesClientImpl implements OperationStatusesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationStatusesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of OperationStatusesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationStatusesClientImpl(ChaosManagementClientImpl client) { - this.service = - RestProxy.create(OperationStatusesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationStatusesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -51,39 +57,32 @@ public final class OperationStatusesClientImpl implements OperationStatusesClien @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface OperationStatusesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{location}/operationsStatuses/{asyncOperationId}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{location}/operationStatuses/{asyncOperationId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("location") String location, - @PathParam("asyncOperationId") String asyncOperationId, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @PathParam("location") String location, @PathParam("asyncOperationId") String asyncOperationId, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); } /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the status of a long running azure asynchronous operation along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String location, String asyncOperationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); @@ -93,47 +92,34 @@ private Mono> getWithResponseAsync(String locatio .error(new IllegalArgumentException("Parameter asyncOperationId is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - location, - asyncOperationId, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), location, asyncOperationId, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the status of a long running azure asynchronous operation along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String location, String asyncOperationId, Context context) { + private Mono> getWithResponseAsync(String location, String asyncOperationId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); @@ -143,28 +129,19 @@ private Mono> getWithResponseAsync( .error(new IllegalArgumentException("Parameter asyncOperationId is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - location, - asyncOperationId, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), location, asyncOperationId, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context); } /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -178,8 +155,8 @@ private Mono getAsync(String location, String asyncOperati /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -194,8 +171,8 @@ public Response getWithResponse(String location, String as /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesImpl.java index 8ae63088b7b05..af94c1a0556e8 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationStatusesImpl.java @@ -20,20 +20,17 @@ public final class OperationStatusesImpl implements OperationStatuses { private final com.azure.resourcemanager.chaos.ChaosManager serviceManager; - public OperationStatusesImpl( - OperationStatusesClient innerClient, com.azure.resourcemanager.chaos.ChaosManager serviceManager) { + public OperationStatusesImpl(OperationStatusesClient innerClient, + com.azure.resourcemanager.chaos.ChaosManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public Response getWithResponse(String location, String asyncOperationId, Context context) { - Response inner = - this.serviceClient().getWithResponse(location, asyncOperationId, context); + Response inner + = this.serviceClient().getWithResponse(location, asyncOperationId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new OperationStatusImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsClientImpl.java index 886c955a9fccc..3f14079d2aab9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.chaos.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(ChaosManagementClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,95 +62,70 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Chaos/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAll( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listAll(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAllNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listAllNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list all available Operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list all available Operations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list all available Operations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listAll(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list all available Operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list all available Operations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list all available Operations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAll(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAll(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list all available Operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list all available Operations as paginated response with {@link PagedFlux}. @@ -156,7 +137,7 @@ private PagedFlux listAllAsync() { /** * Get a list all available Operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -165,13 +146,13 @@ private PagedFlux listAllAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAllAsync(Context context) { - return new PagedFlux<>( - () -> listAllSinglePageAsync(context), nextLink -> listAllNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listAllSinglePageAsync(context), + nextLink -> listAllNextSinglePageAsync(nextLink, context)); } /** * Get a list all available Operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list all available Operations as paginated response with {@link PagedIterable}. @@ -183,7 +164,7 @@ public PagedIterable listAll() { /** * Get a list all available Operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -197,14 +178,15 @@ public PagedIterable listAll(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllNextSinglePageAsync(String nextLink) { @@ -212,37 +194,29 @@ private Mono> listAllNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listAllNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllNextSinglePageAsync(String nextLink, Context context) { @@ -250,23 +224,13 @@ private Mono> listAllNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAllNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAllNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsImpl.java index 25e7075255985..39da64444c055 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/OperationsImpl.java @@ -26,12 +26,12 @@ public OperationsImpl(OperationsClient innerClient, com.azure.resourcemanager.ch public PagedIterable listAll() { PagedIterable inner = this.serviceClient().listAll(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable listAll(Context context) { PagedIterable inner = this.serviceClient().listAll(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/Utils.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ResourceManagerUtils.java similarity index 90% rename from sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/Utils.java rename to sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ResourceManagerUtils.java index 2ea1ef1f8be78..04312bfa92609 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/Utils.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/ResourceManagerUtils.java @@ -19,8 +19,8 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { - private Utils() { +final class ResourceManagerUtils { + private ResourceManagerUtils() { } static String getValueFromIdByName(String id, String name) { @@ -41,6 +41,7 @@ static String getValueFromIdByName(String id, String name) { } } return null; + } static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { @@ -74,6 +75,7 @@ static String getValueFromIdByParameterName(String id, String pathTemplate, Stri } } return null; + } static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { @@ -87,26 +89,17 @@ private static final class PagedIterableImpl extends PagedIterable { private final Function, PagedResponse> pageMapper; private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); this.pagedIterable = pagedIterable; this.mapper = mapper; this.pageMapper = getPageMapper(mapper); } private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); } @Override diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesClientImpl.java index ce7407df6a7c3..1611957b7d981 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.chaos.models.TargetTypeListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in TargetTypesClient. */ +/** + * An instance of this class provides access to all the operations defined in TargetTypesClient. + */ public final class TargetTypesClientImpl implements TargetTypesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final TargetTypesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of TargetTypesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ TargetTypesClientImpl(ChaosManagementClientImpl client) { - this.service = - RestProxy.create(TargetTypesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(TargetTypesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,100 +62,68 @@ public final class TargetTypesClientImpl implements TargetTypesClient { @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface TargetTypesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("locationName") String locationName, - @QueryParam("continuationToken") String continuationToken, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("locationName") String locationName, @QueryParam("continuationToken") String continuationToken, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes/{targetTypeName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Chaos/locations/{locationName}/targetTypes/{targetTypeName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("locationName") String locationName, - @PathParam("targetTypeName") String targetTypeName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("locationName") String locationName, @PathParam("targetTypeName") String targetTypeName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Target Type resources for given location along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String locationName, String continuationToken) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - continuationToken, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), locationName, continuationToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. @@ -157,22 +131,18 @@ private Mono> listSinglePageAsync(String location * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Target Type resources for given location along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String locationName, String continuationToken, Context context) { + private Mono> listSinglePageAsync(String locationName, String continuationToken, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -180,28 +150,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - continuationToken, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), locationName, + continuationToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -211,13 +168,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String locationName, String continuationToken) { - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(locationName, continuationToken), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -227,13 +184,13 @@ private PagedFlux listAsync(String locationName, String continu @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String locationName) { final String continuationToken = null; - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(locationName, continuationToken), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. @@ -244,14 +201,13 @@ private PagedFlux listAsync(String locationName) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String locationName, String continuationToken, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, continuationToken, context), + return new PagedFlux<>(() -> listSinglePageAsync(locationName, continuationToken, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -266,7 +222,7 @@ public PagedIterable list(String locationName) { /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. @@ -282,28 +238,24 @@ public PagedIterable list(String locationName, String continuat /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a Target Type resources for given location along with {@link Response} on successful completion of {@link - * Mono}. + * @return a Target Type resources for given location along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String locationName, String targetTypeName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -313,46 +265,33 @@ private Mono> getWithResponseAsync(String locationName } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - targetTypeName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), locationName, targetTypeName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a Target Type resources for given location along with {@link Response} on successful completion of {@link - * Mono}. + * @return a Target Type resources for given location along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String locationName, String targetTypeName, Context context) { + private Mono> getWithResponseAsync(String locationName, String targetTypeName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -362,20 +301,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - locationName, - targetTypeName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + locationName, targetTypeName, accept, context); } /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -390,7 +322,7 @@ private Mono getAsync(String locationName, String targetTypeNam /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param context The context to associate with this operation. @@ -406,7 +338,7 @@ public Response getWithResponse(String locationName, String tar /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -421,14 +353,15 @@ public TargetTypeInner get(String locationName, String targetTypeName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Target Type resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Target Type resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -436,37 +369,28 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Target Type resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Target Type resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -474,23 +398,13 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesImpl.java index f3b78d87c6c6f..81dc5edb19063 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetTypesImpl.java @@ -28,21 +28,18 @@ public TargetTypesImpl(TargetTypesClient innerClient, com.azure.resourcemanager. public PagedIterable list(String locationName) { PagedIterable inner = this.serviceClient().list(locationName); - return Utils.mapPage(inner, inner1 -> new TargetTypeImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TargetTypeImpl(inner1, this.manager())); } public PagedIterable list(String locationName, String continuationToken, Context context) { PagedIterable inner = this.serviceClient().list(locationName, continuationToken, context); - return Utils.mapPage(inner, inner1 -> new TargetTypeImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TargetTypeImpl(inner1, this.manager())); } public Response getWithResponse(String locationName, String targetTypeName, Context context) { Response inner = this.serviceClient().getWithResponse(locationName, targetTypeName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new TargetTypeImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsClientImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsClientImpl.java index 316f3813e73ac..f3de7fe94bafd 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsClientImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsClientImpl.java @@ -33,17 +33,23 @@ import com.azure.resourcemanager.chaos.models.TargetListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in TargetsClient. */ +/** + * An instance of this class provides access to all the operations defined in TargetsClient. + */ public final class TargetsClientImpl implements TargetsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final TargetsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final ChaosManagementClientImpl client; /** * Initializes an instance of TargetsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ TargetsClientImpl(ChaosManagementClientImpl client) { @@ -58,89 +64,66 @@ public final class TargetsClientImpl implements TargetsClient { @Host("{$host}") @ServiceInterface(name = "ChaosManagementClien") public interface TargetsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, @PathParam("parentResourceName") String parentResourceName, - @QueryParam("continuationToken") String continuationToken, - @HeaderParam("Accept") String accept, + @QueryParam("continuationToken") String continuationToken, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{parentProviderNamespace}/{parentResourceType}/{parentResourceName}/providers/Microsoft.Chaos/targets/{targetName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("parentProviderNamespace") String parentProviderNamespace, @PathParam("parentResourceType") String parentResourceType, - @PathParam("parentResourceName") String parentResourceName, - @PathParam("targetName") String targetName, - @BodyParam("application/json") TargetInner target, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("parentResourceName") String parentResourceName, @PathParam("targetName") String targetName, + @BodyParam("application/json") TargetInner target, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -150,35 +133,27 @@ Mono> listNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Target resources that extend a tracked regional resource along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, + private Mono> listSinglePageAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String continuationToken) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -190,35 +165,17 @@ private Mono> listSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, continuationToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -229,36 +186,27 @@ private Mono> listSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Target resources that extend a tracked regional resource along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken, + private Mono> listSinglePageAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String continuationToken, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -271,31 +219,16 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, continuationToken, + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -304,30 +237,19 @@ private Mono> listSinglePageAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedFlux}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken) { - return new PagedFlux<>( - () -> - listSinglePageAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken), - nextLink -> listNextSinglePageAsync(nextLink)); + private PagedFlux listAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String continuationToken) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -335,30 +257,20 @@ private PagedFlux listAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedFlux}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName) { + private PagedFlux listAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName) { final String continuationToken = null; - return new PagedFlux<>( - () -> - listSinglePageAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken), - nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, continuationToken), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -368,32 +280,21 @@ private PagedFlux listAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedFlux}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken, - Context context) { + private PagedFlux listAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String continuationToken, Context context) { return new PagedFlux<>( - () -> - listSinglePageAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken, - context), + () -> listSinglePageAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, continuationToken, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -401,24 +302,20 @@ private PagedFlux listAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedIterable}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName) { + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName) { final String continuationToken = null; - return new PagedIterable<>( - listAsync( - resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, continuationToken)); + return new PagedIterable<>(listAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, continuationToken)); } /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -428,30 +325,19 @@ public PagedIterable list( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedIterable}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken, - Context context) { - return new PagedIterable<>( - listAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken, - context)); + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String continuationToken, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, continuationToken, context)); } /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -461,35 +347,26 @@ public PagedIterable list( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Target resource that extends a tracked regional resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { + private Mono> getWithResponseAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -504,26 +381,15 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -534,36 +400,26 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Target resource that extends a tracked regional resource along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -578,23 +434,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, accept, + context); } /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -606,20 +453,15 @@ private Mono> getWithResponseAsync( * @return a Target resource that extends a tracked regional resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - return getWithResponseAsync( - resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono getAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName) { + return getWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -632,21 +474,15 @@ private Mono getAsync( * @return a Target resource that extends a tracked regional resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context) { - return getWithResponseAsync( - resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, context) - .block(); + public Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context) { + return getWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, context).block(); } /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -658,25 +494,15 @@ public Response getWithResponse( * @return a Target resource that extends a tracked regional resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public TargetInner get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - return getWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - Context.NONE) - .getValue(); + public TargetInner get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName) { + return getWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, Context.NONE).getValue(); } /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -688,32 +514,23 @@ public TargetInner get( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -728,26 +545,15 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -760,33 +566,23 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -801,23 +597,14 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, accept, + context); } /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -829,20 +616,15 @@ private Mono> deleteWithResponseAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - return deleteWithResponseAsync( - resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName) - .flatMap(ignored -> Mono.empty()); + private Mono deleteAsync(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName) { + return deleteWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName).flatMap(ignored -> Mono.empty()); } /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -855,21 +637,15 @@ private Mono deleteAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context) { - return deleteWithResponseAsync( - resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, context) - .block(); + public Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context) { + return deleteWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, context).block(); } /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -880,24 +656,15 @@ public Response deleteWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - deleteWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - Context.NONE); + public void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName) { + deleteWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName, Context.NONE); } /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -907,37 +674,28 @@ public void delete( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a Target resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return model that represents a Target resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, TargetInner target) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -957,27 +715,15 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - target, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, target, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -988,38 +734,28 @@ private Mono> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a Target resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return model that represents a Target resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String parentProviderNamespace, String parentResourceType, String parentResourceName, String targetName, + TargetInner target, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (parentProviderNamespace == null) { - return Mono - .error( - new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter parentProviderNamespace is required and cannot be null.")); } if (parentResourceType == null) { return Mono @@ -1039,24 +775,14 @@ private Mono> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - target, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, target, accept, context); } /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1069,21 +795,15 @@ private Mono> createOrUpdateWithResponseAsync( * @return model that represents a Target resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target) { - return createOrUpdateWithResponseAsync( - resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName, target) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono createOrUpdateAsync(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, TargetInner target) { + return createOrUpdateWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, target).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1097,28 +817,15 @@ private Mono createOrUpdateAsync( * @return model that represents a Target resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - target, - context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, TargetInner target, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, target, context).block(); } /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -1131,34 +838,23 @@ public Response createOrUpdateWithResponse( * @return model that represents a Target resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public TargetInner createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target) { - return createOrUpdateWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - target, - Context.NONE) - .getValue(); + public TargetInner createOrUpdate(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, TargetInner target) { + return createOrUpdateWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, target, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Target resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Target resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1166,37 +862,28 @@ private Mono> listNextSinglePageAsync(String nextLink return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return model that represents a list of Target resources and a link for pagination along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return model that represents a list of Target resources and a link for pagination along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1204,23 +891,13 @@ private Mono> listNextSinglePageAsync(String nextLink return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsImpl.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsImpl.java index 7bcbde028239f..1fa90685eea4d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsImpl.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/TargetsImpl.java @@ -26,76 +26,36 @@ public TargetsImpl(TargetsClient innerClient, com.azure.resourcemanager.chaos.Ch this.serviceManager = serviceManager; } - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName) { - PagedIterable inner = - this - .serviceClient() - .list(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName); - return Utils.mapPage(inner, inner1 -> new TargetImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TargetImpl(inner1, this.manager())); } - public PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken, - Context context) { - PagedIterable inner = - this - .serviceClient() - .list( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - continuationToken, - context); - return Utils.mapPage(inner, inner1 -> new TargetImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String continuationToken, Context context) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, continuationToken, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TargetImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - context); + public Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, targetName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new TargetImpl(inner.getValue(), this.manager())); } else { return null; } } - public Target get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - TargetInner inner = - this - .serviceClient() - .get(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName); + public Target get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName) { + TargetInner inner = this.serviceClient().get(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName); if (inner != null) { return new TargetImpl(inner, this.manager()); } else { @@ -103,82 +63,34 @@ public Target get( } } - public Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context) { - return this - .serviceClient() - .deleteWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - context); + public Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, parentProviderNamespace, parentResourceType, + parentResourceName, targetName, context); } - public void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName) { - this - .serviceClient() - .delete(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, targetName); + public void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName) { + this.serviceClient().delete(resourceGroupName, parentProviderNamespace, parentResourceType, parentResourceName, + targetName); } - public Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target, - Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateWithResponse( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - target, - context); + public Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, TargetInner target, Context context) { + Response inner = this.serviceClient().createOrUpdateWithResponse(resourceGroupName, + parentProviderNamespace, parentResourceType, parentResourceName, targetName, target, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new TargetImpl(inner.getValue(), this.manager())); } else { return null; } } - public Target createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target) { - TargetInner inner = - this - .serviceClient() - .createOrUpdate( - resourceGroupName, - parentProviderNamespace, - parentResourceType, - parentResourceName, - targetName, - target); + public Target createOrUpdate(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, TargetInner target) { + TargetInner inner = this.serviceClient().createOrUpdate(resourceGroupName, parentProviderNamespace, + parentResourceType, parentResourceName, targetName, target); if (inner != null) { return new TargetImpl(inner, this.manager()); } else { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/package-info.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/package-info.java index a6fc0cab942d5..49f8284e0865d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/package-info.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the implementations for ChaosManagementClient. Chaos Management Client. */ +/** + * Package containing the implementations for ChaosManagementClient. + * Chaos Management Client. + */ package com.azure.resourcemanager.chaos.implementation; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionStatus.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionStatus.java index 4307ffdb739a8..758773bc359f9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionStatus.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionStatus.java @@ -9,7 +9,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Model that represents the an action and its status. */ +/** + * Model that represents the an action and its status. + */ @Immutable public final class ActionStatus { /* @@ -48,13 +50,15 @@ public final class ActionStatus { @JsonProperty(value = "targets", access = JsonProperty.Access.WRITE_ONLY) private List targets; - /** Creates an instance of ActionStatus class. */ + /** + * Creates an instance of ActionStatus class. + */ public ActionStatus() { } /** * Get the actionName property: The name of the action status. - * + * * @return the actionName value. */ public String actionName() { @@ -63,7 +67,7 @@ public String actionName() { /** * Get the actionId property: The id of the action status. - * + * * @return the actionId value. */ public String actionId() { @@ -72,7 +76,7 @@ public String actionId() { /** * Get the status property: The status of the action. - * + * * @return the status value. */ public String status() { @@ -81,7 +85,7 @@ public String status() { /** * Get the startTime property: String that represents the start time of the action. - * + * * @return the startTime value. */ public OffsetDateTime startTime() { @@ -90,7 +94,7 @@ public OffsetDateTime startTime() { /** * Get the endTime property: String that represents the end time of the action. - * + * * @return the endTime value. */ public OffsetDateTime endTime() { @@ -99,7 +103,7 @@ public OffsetDateTime endTime() { /** * Get the targets property: The array of targets. - * + * * @return the targets value. */ public List targets() { @@ -108,7 +112,7 @@ public List targets() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionType.java index 83a87be8eca45..b0e71ff86b99e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ActionType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. */ +/** + * Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. + */ public final class ActionType extends ExpandableStringEnum { - /** Static value Internal for ActionType. */ + /** + * Static value Internal for ActionType. + */ public static final ActionType INTERNAL = fromString("Internal"); /** * Creates a new instance of ActionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ActionType() { /** * Creates or finds a ActionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActionType. */ @@ -35,7 +39,7 @@ public static ActionType fromString(String name) { /** * Gets known ActionType values. - * + * * @return known ActionType values. */ public static Collection values() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/BranchStatus.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/BranchStatus.java index af57124c70813..46752e6e415b2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/BranchStatus.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/BranchStatus.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents the a list of actions and action statuses. */ +/** + * Model that represents the a list of actions and action statuses. + */ @Immutable public final class BranchStatus { /* @@ -35,13 +37,15 @@ public final class BranchStatus { @JsonProperty(value = "actions", access = JsonProperty.Access.WRITE_ONLY) private List actions; - /** Creates an instance of BranchStatus class. */ + /** + * Creates an instance of BranchStatus class. + */ public BranchStatus() { } /** * Get the branchName property: The name of the branch status. - * + * * @return the branchName value. */ public String branchName() { @@ -50,7 +54,7 @@ public String branchName() { /** * Get the branchId property: The id of the branch status. - * + * * @return the branchId value. */ public String branchId() { @@ -59,7 +63,7 @@ public String branchId() { /** * Get the status property: The status of the branch. - * + * * @return the status value. */ public String status() { @@ -68,7 +72,7 @@ public String status() { /** * Get the actions property: The array of actions. - * + * * @return the actions value. */ public List actions() { @@ -77,7 +81,7 @@ public List actions() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capabilities.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capabilities.java index e071ff39ef619..86ae2c843c9e5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capabilities.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capabilities.java @@ -9,11 +9,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.CapabilityInner; -/** Resource collection API of Capabilities. */ +/** + * Resource collection API of Capabilities. + */ public interface Capabilities { /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -22,19 +24,15 @@ public interface Capabilities { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedIterable}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedIterable}. */ - PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName); /** * Get a list of Capability resources that extend a Target resource.. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -45,21 +43,15 @@ PagedIterable list( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability resources that extend a Target resource. as paginated response with {@link - * PagedIterable}. + * @return a list of Capability resources that extend a Target resource. as paginated response with + * {@link PagedIterable}. */ - PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String continuationToken, - Context context); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String continuationToken, Context context); /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -72,18 +64,13 @@ PagedIterable list( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability resource that extends a Target resource along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context); /** * Get a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -95,17 +82,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability resource that extends a Target resource. */ - Capability get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName); + Capability get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName); /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -118,18 +100,13 @@ Capability get( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, + Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, Context context); /** * Delete a Capability that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -140,17 +117,12 @@ Response deleteWithResponse( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName); + void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName); /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -164,19 +136,13 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Capability resource along with {@link Response}. */ - Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, String capabilityName, + CapabilityInner capability, Context context); /** * Create or update a Capability resource that extends a Target resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -189,12 +155,6 @@ Response createOrUpdateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Capability resource. */ - Capability createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - String capabilityName, - CapabilityInner capability); + Capability createOrUpdate(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, String capabilityName, CapabilityInner capability); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capability.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capability.java index 26837dec35b4a..c5e258f3cbdb3 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capability.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Capability.java @@ -7,74 +7,76 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.chaos.fluent.models.CapabilityInner; -/** An immutable client-side representation of Capability. */ +/** + * An immutable client-side representation of Capability. + */ public interface Capability { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: The standard system metadata of a resource type. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the publisher property: String of the Publisher that this Capability extends. - * + * * @return the publisher value. */ String publisher(); /** * Gets the targetType property: String of the Target Type that this Capability extends. - * + * * @return the targetType value. */ String targetType(); /** * Gets the description property: Localized string of the description. - * + * * @return the description value. */ String description(); /** * Gets the parametersSchema property: URL to retrieve JSON schema of the Capability parameters. - * + * * @return the parametersSchema value. */ String parametersSchema(); /** * Gets the urn property: String of the URN for this Capability Type. - * + * * @return the urn value. */ String urn(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.CapabilityInner object. - * + * * @return the inner object. */ CapabilityInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityListResult.java index e1fd74729bef0..5a8f1a78f705e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a list of Capability resources and a link for pagination. */ +/** + * Model that represents a list of Capability resources and a link for pagination. + */ @Immutable public final class CapabilityListResult { /* @@ -24,13 +26,15 @@ public final class CapabilityListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of CapabilityListResult class. */ + /** + * Creates an instance of CapabilityListResult class. + */ public CapabilityListResult() { } /** * Get the value property: List of Capability resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: URL to retrieve the next page of Capability resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityType.java index a06b99e80baa2..cc8626daf5295 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityType.java @@ -8,116 +8,118 @@ import com.azure.resourcemanager.chaos.fluent.models.CapabilityTypeInner; import java.util.List; -/** An immutable client-side representation of CapabilityType. */ +/** + * An immutable client-side representation of CapabilityType. + */ public interface CapabilityType { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: The system metadata properties of the capability type resource. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the location property: Location of the Capability Type resource. - * + * * @return the location value. */ String location(); /** * Gets the publisher property: String of the Publisher that this Capability Type extends. - * + * * @return the publisher value. */ String publisher(); /** * Gets the targetType property: String of the Target Type that this Capability Type extends. - * + * * @return the targetType value. */ String targetType(); /** * Gets the displayName property: Localized string of the display name. - * + * * @return the displayName value. */ String displayName(); /** * Gets the description property: Localized string of the description. - * + * * @return the description value. */ String description(); /** * Gets the parametersSchema property: URL to retrieve JSON schema of the Capability Type parameters. - * + * * @return the parametersSchema value. */ String parametersSchema(); /** * Gets the urn property: String of the URN for this Capability Type. - * + * * @return the urn value. */ String urn(); /** * Gets the kind property: String of the kind of this Capability Type. - * + * * @return the kind value. */ String kind(); /** * Gets the azureRbacActions property: Control plane actions necessary to execute capability type. - * + * * @return the azureRbacActions value. */ List azureRbacActions(); /** * Gets the azureRbacDataActions property: Data plane actions necessary to execute capability type. - * + * * @return the azureRbacDataActions value. */ List azureRbacDataActions(); /** * Gets the runtimeProperties property: Runtime properties of this Capability Type. - * + * * @return the runtimeProperties value. */ CapabilityTypePropertiesRuntimeProperties runtimeProperties(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.CapabilityTypeInner object. - * + * * @return the inner object. */ CapabilityTypeInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypeListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypeListResult.java index 103292b6fa7cd..39627dcd75350 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypeListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypeListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a list of Capability Type resources and a link for pagination. */ +/** + * Model that represents a list of Capability Type resources and a link for pagination. + */ @Immutable public final class CapabilityTypeListResult { /* @@ -24,13 +26,15 @@ public final class CapabilityTypeListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of CapabilityTypeListResult class. */ + /** + * Creates an instance of CapabilityTypeListResult class. + */ public CapabilityTypeListResult() { } /** * Get the value property: List of Capability Type resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: URL to retrieve the next page of Capability Type resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypePropertiesRuntimeProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypePropertiesRuntimeProperties.java index 4aea503b42c3d..75b79cfeff2e3 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypePropertiesRuntimeProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypePropertiesRuntimeProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Runtime properties of this Capability Type. */ +/** + * Runtime properties of this Capability Type. + */ @Immutable public final class CapabilityTypePropertiesRuntimeProperties { /* @@ -16,13 +18,15 @@ public final class CapabilityTypePropertiesRuntimeProperties { @JsonProperty(value = "kind", access = JsonProperty.Access.WRITE_ONLY) private String kind; - /** Creates an instance of CapabilityTypePropertiesRuntimeProperties class. */ + /** + * Creates an instance of CapabilityTypePropertiesRuntimeProperties class. + */ public CapabilityTypePropertiesRuntimeProperties() { } /** * Get the kind property: String of the kind of the resource's action type (continuous or discrete). - * + * * @return the kind value. */ public String kind() { @@ -31,7 +35,7 @@ public String kind() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypes.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypes.java index b04d648c21f51..f6625900b0084 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypes.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/CapabilityTypes.java @@ -8,24 +8,26 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of CapabilityTypes. */ +/** + * Resource collection API of CapabilityTypes. + */ public interface CapabilityTypes { /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedIterable}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedIterable}. */ PagedIterable list(String locationName, String targetTypeName); /** * Get a list of Capability Type resources for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param continuationToken String that sets the continuation token. @@ -33,15 +35,15 @@ public interface CapabilityTypes { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Capability Type resources for given Target Type and location as paginated response with {@link - * PagedIterable}. + * @return a list of Capability Type resources for given Target Type and location as paginated response with + * {@link PagedIterable}. */ - PagedIterable list( - String locationName, String targetTypeName, String continuationToken, Context context); + PagedIterable list(String locationName, String targetTypeName, String continuationToken, + Context context); /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. @@ -51,12 +53,12 @@ PagedIterable list( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Capability Type resource for given Target Type and location along with {@link Response}. */ - Response getWithResponse( - String locationName, String targetTypeName, String capabilityTypeName, Context context); + Response getWithResponse(String locationName, String targetTypeName, String capabilityTypeName, + Context context); /** * Get a Capability Type resource for given Target Type and location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param capabilityTypeName String that represents a Capability Type resource name. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentAction.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentAction.java index f550e46428c6e..ca8fb8ed777b0 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentAction.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentAction.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Model that represents the base action model. 9 total per experiment. */ +/** + * Model that represents the base action model. 9 total per experiment. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -21,8 +23,7 @@ @JsonSubTypes({ @JsonSubTypes.Type(name = "delay", value = DelayAction.class), @JsonSubTypes.Type(name = "discrete", value = DiscreteAction.class), - @JsonSubTypes.Type(name = "continuous", value = ContinuousAction.class) -}) + @JsonSubTypes.Type(name = "continuous", value = ContinuousAction.class) }) @Fluent public class ChaosExperimentAction { /* @@ -31,13 +32,15 @@ public class ChaosExperimentAction { @JsonProperty(value = "name", required = true) private String name; - /** Creates an instance of ChaosExperimentAction class. */ + /** + * Creates an instance of ChaosExperimentAction class. + */ public ChaosExperimentAction() { } /** * Get the name property: String that represents a Capability URN. - * + * * @return the name value. */ public String name() { @@ -46,7 +49,7 @@ public String name() { /** * Set the name property: String that represents a Capability URN. - * + * * @param name the name value to set. * @return the ChaosExperimentAction object itself. */ @@ -57,14 +60,13 @@ public ChaosExperimentAction withName(String name) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model ChaosExperimentAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model ChaosExperimentAction")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentBranch.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentBranch.java index 72f28884f9e29..03176b9cee2d5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentBranch.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentBranch.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a branch in the step. 9 total per experiment. */ +/** + * Model that represents a branch in the step. 9 total per experiment. + */ @Fluent public final class ChaosExperimentBranch { /* @@ -24,13 +26,15 @@ public final class ChaosExperimentBranch { @JsonProperty(value = "actions", required = true) private List actions; - /** Creates an instance of ChaosExperimentBranch class. */ + /** + * Creates an instance of ChaosExperimentBranch class. + */ public ChaosExperimentBranch() { } /** * Get the name property: String of the branch name. - * + * * @return the name value. */ public String name() { @@ -39,7 +43,7 @@ public String name() { /** * Set the name property: String of the branch name. - * + * * @param name the name value to set. * @return the ChaosExperimentBranch object itself. */ @@ -50,7 +54,7 @@ public ChaosExperimentBranch withName(String name) { /** * Get the actions property: List of actions. - * + * * @return the actions value. */ public List actions() { @@ -59,7 +63,7 @@ public List actions() { /** * Set the actions property: List of actions. - * + * * @param actions the actions value to set. * @return the ChaosExperimentBranch object itself. */ @@ -70,19 +74,17 @@ public ChaosExperimentBranch withActions(List actions) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model ChaosExperimentBranch")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model ChaosExperimentBranch")); } if (actions() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property actions in model ChaosExperimentBranch")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property actions in model ChaosExperimentBranch")); } else { actions().forEach(e -> e.validate()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentStep.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentStep.java index f8b09bda8fc63..a217c09c74cdc 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentStep.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosExperimentStep.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a step in the Experiment resource. */ +/** + * Model that represents a step in the Experiment resource. + */ @Fluent public final class ChaosExperimentStep { /* @@ -24,13 +26,15 @@ public final class ChaosExperimentStep { @JsonProperty(value = "branches", required = true) private List branches; - /** Creates an instance of ChaosExperimentStep class. */ + /** + * Creates an instance of ChaosExperimentStep class. + */ public ChaosExperimentStep() { } /** * Get the name property: String of the step name. - * + * * @return the name value. */ public String name() { @@ -39,7 +43,7 @@ public String name() { /** * Set the name property: String of the step name. - * + * * @param name the name value to set. * @return the ChaosExperimentStep object itself. */ @@ -50,7 +54,7 @@ public ChaosExperimentStep withName(String name) { /** * Get the branches property: List of branches. - * + * * @return the branches value. */ public List branches() { @@ -59,7 +63,7 @@ public List branches() { /** * Set the branches property: List of branches. - * + * * @param branches the branches value to set. * @return the ChaosExperimentStep object itself. */ @@ -70,19 +74,17 @@ public ChaosExperimentStep withBranches(List branches) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model ChaosExperimentStep")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model ChaosExperimentStep")); } if (branches() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property branches in model ChaosExperimentStep")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property branches in model ChaosExperimentStep")); } else { branches().forEach(e -> e.validate()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetFilter.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetFilter.java index 725dd624f89dd..e40af79c50930 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetFilter.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetFilter.java @@ -9,23 +9,27 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Model that represents available filter types that can be applied to a targets list. */ +/** + * Model that represents available filter types that can be applied to a targets list. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type", defaultImpl = ChaosTargetFilter.class) @JsonTypeName("ChaosTargetFilter") -@JsonSubTypes({@JsonSubTypes.Type(name = "Simple", value = ChaosTargetSimpleFilter.class)}) +@JsonSubTypes({ @JsonSubTypes.Type(name = "Simple", value = ChaosTargetSimpleFilter.class) }) @Immutable public class ChaosTargetFilter { - /** Creates an instance of ChaosTargetFilter class. */ + /** + * Creates an instance of ChaosTargetFilter class. + */ public ChaosTargetFilter() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetListSelector.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetListSelector.java index 86d992203dfce..9a724295ecfc1 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetListSelector.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetListSelector.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Model that represents a list selector. */ +/** + * Model that represents a list selector. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("List") @Fluent @@ -22,13 +24,15 @@ public final class ChaosTargetListSelector extends ChaosTargetSelector { @JsonProperty(value = "targets", required = true) private List targets; - /** Creates an instance of ChaosTargetListSelector class. */ + /** + * Creates an instance of ChaosTargetListSelector class. + */ public ChaosTargetListSelector() { } /** * Get the targets property: List of Target references. - * + * * @return the targets value. */ public List targets() { @@ -37,7 +41,7 @@ public List targets() { /** * Set the targets property: List of Target references. - * + * * @param targets the targets value to set. * @return the ChaosTargetListSelector object itself. */ @@ -46,14 +50,18 @@ public ChaosTargetListSelector withTargets(List targets) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ChaosTargetListSelector withId(String id) { super.withId(id); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ChaosTargetListSelector withFilter(ChaosTargetFilter filter) { super.withFilter(filter); @@ -62,16 +70,15 @@ public ChaosTargetListSelector withFilter(ChaosTargetFilter filter) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (targets() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property targets in model ChaosTargetListSelector")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property targets in model ChaosTargetListSelector")); } else { targets().forEach(e -> e.validate()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetQuerySelector.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetQuerySelector.java index be471aaf91b9b..5e0aa3765518a 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetQuerySelector.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetQuerySelector.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Model that represents a query selector. */ +/** + * Model that represents a query selector. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("Query") @Fluent @@ -28,13 +30,15 @@ public final class ChaosTargetQuerySelector extends ChaosTargetSelector { @JsonProperty(value = "subscriptionIds", required = true) private List subscriptionIds; - /** Creates an instance of ChaosTargetQuerySelector class. */ + /** + * Creates an instance of ChaosTargetQuerySelector class. + */ public ChaosTargetQuerySelector() { } /** * Get the queryString property: Azure Resource Graph (ARG) Query Language query for target resources. - * + * * @return the queryString value. */ public String queryString() { @@ -43,7 +47,7 @@ public String queryString() { /** * Set the queryString property: Azure Resource Graph (ARG) Query Language query for target resources. - * + * * @param queryString the queryString value to set. * @return the ChaosTargetQuerySelector object itself. */ @@ -54,7 +58,7 @@ public ChaosTargetQuerySelector withQueryString(String queryString) { /** * Get the subscriptionIds property: Subscription id list to scope resource query. - * + * * @return the subscriptionIds value. */ public List subscriptionIds() { @@ -63,7 +67,7 @@ public List subscriptionIds() { /** * Set the subscriptionIds property: Subscription id list to scope resource query. - * + * * @param subscriptionIds the subscriptionIds value to set. * @return the ChaosTargetQuerySelector object itself. */ @@ -72,14 +76,18 @@ public ChaosTargetQuerySelector withSubscriptionIds(List subscriptionIds return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ChaosTargetQuerySelector withId(String id) { super.withId(id); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ChaosTargetQuerySelector withFilter(ChaosTargetFilter filter) { super.withFilter(filter); @@ -88,23 +96,19 @@ public ChaosTargetQuerySelector withFilter(ChaosTargetFilter filter) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (queryString() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property queryString in model ChaosTargetQuerySelector")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property queryString in model ChaosTargetQuerySelector")); } if (subscriptionIds() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property subscriptionIds in model ChaosTargetQuerySelector")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property subscriptionIds in model ChaosTargetQuerySelector")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSelector.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSelector.java index 422350c2e38ac..4d30f8cc29ad9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSelector.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSelector.java @@ -16,7 +16,9 @@ import java.util.HashMap; import java.util.Map; -/** Model that represents a selector in the Experiment resource. */ +/** + * Model that represents a selector in the Experiment resource. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -25,8 +27,7 @@ @JsonTypeName("ChaosTargetSelector") @JsonSubTypes({ @JsonSubTypes.Type(name = "List", value = ChaosTargetListSelector.class), - @JsonSubTypes.Type(name = "Query", value = ChaosTargetQuerySelector.class) -}) + @JsonSubTypes.Type(name = "Query", value = ChaosTargetQuerySelector.class) }) @Fluent public class ChaosTargetSelector { /* @@ -44,15 +45,18 @@ public class ChaosTargetSelector { /* * Model that represents a selector in the Experiment resource. */ - @JsonIgnore private Map additionalProperties; + @JsonIgnore + private Map additionalProperties; - /** Creates an instance of ChaosTargetSelector class. */ + /** + * Creates an instance of ChaosTargetSelector class. + */ public ChaosTargetSelector() { } /** * Get the id property: String of the selector ID. - * + * * @return the id value. */ public String id() { @@ -61,7 +65,7 @@ public String id() { /** * Set the id property: String of the selector ID. - * + * * @param id the id value to set. * @return the ChaosTargetSelector object itself. */ @@ -72,7 +76,7 @@ public ChaosTargetSelector withId(String id) { /** * Get the filter property: Model that represents available filter types that can be applied to a targets list. - * + * * @return the filter value. */ public ChaosTargetFilter filter() { @@ -81,7 +85,7 @@ public ChaosTargetFilter filter() { /** * Set the filter property: Model that represents available filter types that can be applied to a targets list. - * + * * @param filter the filter value to set. * @return the ChaosTargetSelector object itself. */ @@ -92,7 +96,7 @@ public ChaosTargetSelector withFilter(ChaosTargetFilter filter) { /** * Get the additionalProperties property: Model that represents a selector in the Experiment resource. - * + * * @return the additionalProperties value. */ @JsonAnyGetter @@ -102,7 +106,7 @@ public Map additionalProperties() { /** * Set the additionalProperties property: Model that represents a selector in the Experiment resource. - * + * * @param additionalProperties the additionalProperties value to set. * @return the ChaosTargetSelector object itself. */ @@ -121,14 +125,13 @@ void withAdditionalProperties(String key, Object value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model ChaosTargetSelector")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model ChaosTargetSelector")); } if (filter() != null) { filter().validate(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilter.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilter.java index 6f122cddcad79..d2a3e18b37d5a 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilter.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilter.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Model that represents a simple target filter. */ +/** + * Model that represents a simple target filter. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("Simple") @Fluent @@ -20,13 +22,15 @@ public final class ChaosTargetSimpleFilter extends ChaosTargetFilter { @JsonProperty(value = "parameters") private ChaosTargetSimpleFilterParameters parameters; - /** Creates an instance of ChaosTargetSimpleFilter class. */ + /** + * Creates an instance of ChaosTargetSimpleFilter class. + */ public ChaosTargetSimpleFilter() { } /** * Get the parameters property: Model that represents the Simple filter parameters. - * + * * @return the parameters value. */ public ChaosTargetSimpleFilterParameters parameters() { @@ -35,7 +39,7 @@ public ChaosTargetSimpleFilterParameters parameters() { /** * Set the parameters property: Model that represents the Simple filter parameters. - * + * * @param parameters the parameters value to set. * @return the ChaosTargetSimpleFilter object itself. */ @@ -46,7 +50,7 @@ public ChaosTargetSimpleFilter withParameters(ChaosTargetSimpleFilterParameters /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilterParameters.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilterParameters.java index 73e4ed26ad56c..d0725941ad414 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilterParameters.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ChaosTargetSimpleFilterParameters.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents the Simple filter parameters. */ +/** + * Model that represents the Simple filter parameters. + */ @Fluent public final class ChaosTargetSimpleFilterParameters { /* @@ -17,13 +19,15 @@ public final class ChaosTargetSimpleFilterParameters { @JsonProperty(value = "zones") private List zones; - /** Creates an instance of ChaosTargetSimpleFilterParameters class. */ + /** + * Creates an instance of ChaosTargetSimpleFilterParameters class. + */ public ChaosTargetSimpleFilterParameters() { } /** * Get the zones property: List of Azure availability zones to filter targets by. - * + * * @return the zones value. */ public List zones() { @@ -32,7 +36,7 @@ public List zones() { /** * Set the zones property: List of Azure availability zones to filter targets by. - * + * * @param zones the zones value to set. * @return the ChaosTargetSimpleFilterParameters object itself. */ @@ -43,7 +47,7 @@ public ChaosTargetSimpleFilterParameters withZones(List zones) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ContinuousAction.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ContinuousAction.java index a8abd196a3ae4..3a2d7ed30bebb 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ContinuousAction.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ContinuousAction.java @@ -12,7 +12,9 @@ import java.time.Duration; import java.util.List; -/** Model that represents a continuous action. */ +/** + * Model that represents a continuous action. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("continuous") @Fluent @@ -35,13 +37,15 @@ public final class ContinuousAction extends ChaosExperimentAction { @JsonProperty(value = "selectorId", required = true) private String selectorId; - /** Creates an instance of ContinuousAction class. */ + /** + * Creates an instance of ContinuousAction class. + */ public ContinuousAction() { } /** * Get the duration property: ISO8601 formatted string that represents a duration. - * + * * @return the duration value. */ public Duration duration() { @@ -50,7 +54,7 @@ public Duration duration() { /** * Set the duration property: ISO8601 formatted string that represents a duration. - * + * * @param duration the duration value to set. * @return the ContinuousAction object itself. */ @@ -61,7 +65,7 @@ public ContinuousAction withDuration(Duration duration) { /** * Get the parameters property: List of key value pairs. - * + * * @return the parameters value. */ public List parameters() { @@ -70,7 +74,7 @@ public List parameters() { /** * Set the parameters property: List of key value pairs. - * + * * @param parameters the parameters value to set. * @return the ContinuousAction object itself. */ @@ -81,7 +85,7 @@ public ContinuousAction withParameters(List parameters) { /** * Get the selectorId property: String that represents a selector. - * + * * @return the selectorId value. */ public String selectorId() { @@ -90,7 +94,7 @@ public String selectorId() { /** * Set the selectorId property: String that represents a selector. - * + * * @param selectorId the selectorId value to set. * @return the ContinuousAction object itself. */ @@ -99,7 +103,9 @@ public ContinuousAction withSelectorId(String selectorId) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ContinuousAction withName(String name) { super.withName(name); @@ -108,28 +114,25 @@ public ContinuousAction withName(String name) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (duration() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property duration in model ContinuousAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property duration in model ContinuousAction")); } if (parameters() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property parameters in model ContinuousAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property parameters in model ContinuousAction")); } else { parameters().forEach(e -> e.validate()); } if (selectorId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property selectorId in model ContinuousAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property selectorId in model ContinuousAction")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DelayAction.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DelayAction.java index 5a6ac12c212b7..93a670931b6de 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DelayAction.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DelayAction.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Model that represents a delay action. */ +/** + * Model that represents a delay action. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("delay") @Fluent @@ -22,13 +24,15 @@ public final class DelayAction extends ChaosExperimentAction { @JsonProperty(value = "duration", required = true) private Duration duration; - /** Creates an instance of DelayAction class. */ + /** + * Creates an instance of DelayAction class. + */ public DelayAction() { } /** * Get the duration property: ISO8601 formatted string that represents a duration. - * + * * @return the duration value. */ public Duration duration() { @@ -37,7 +41,7 @@ public Duration duration() { /** * Set the duration property: ISO8601 formatted string that represents a duration. - * + * * @param duration the duration value to set. * @return the DelayAction object itself. */ @@ -46,7 +50,9 @@ public DelayAction withDuration(Duration duration) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DelayAction withName(String name) { super.withName(name); @@ -55,16 +61,15 @@ public DelayAction withName(String name) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (duration() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property duration in model DelayAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property duration in model DelayAction")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DiscreteAction.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DiscreteAction.java index 6eda3392ddab2..ba456679176d0 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DiscreteAction.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/DiscreteAction.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Model that represents a discrete action. */ +/** + * Model that represents a discrete action. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("discrete") @Fluent @@ -28,13 +30,15 @@ public final class DiscreteAction extends ChaosExperimentAction { @JsonProperty(value = "selectorId", required = true) private String selectorId; - /** Creates an instance of DiscreteAction class. */ + /** + * Creates an instance of DiscreteAction class. + */ public DiscreteAction() { } /** * Get the parameters property: List of key value pairs. - * + * * @return the parameters value. */ public List parameters() { @@ -43,7 +47,7 @@ public List parameters() { /** * Set the parameters property: List of key value pairs. - * + * * @param parameters the parameters value to set. * @return the DiscreteAction object itself. */ @@ -54,7 +58,7 @@ public DiscreteAction withParameters(List parameters) { /** * Get the selectorId property: String that represents a selector. - * + * * @return the selectorId value. */ public String selectorId() { @@ -63,7 +67,7 @@ public String selectorId() { /** * Set the selectorId property: String that represents a selector. - * + * * @param selectorId the selectorId value to set. * @return the DiscreteAction object itself. */ @@ -72,7 +76,9 @@ public DiscreteAction withSelectorId(String selectorId) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DiscreteAction withName(String name) { super.withName(name); @@ -81,23 +87,21 @@ public DiscreteAction withName(String name) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (parameters() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property parameters in model DiscreteAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property parameters in model DiscreteAction")); } else { parameters().forEach(e -> e.validate()); } if (selectorId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property selectorId in model DiscreteAction")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property selectorId in model DiscreteAction")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiment.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiment.java index 13394954bf23c..31e719c87678c 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiment.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiment.java @@ -11,127 +11,133 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Experiment. */ +/** + * An immutable client-side representation of Experiment. + */ public interface Experiment { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: The system metadata of the experiment resource. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the identity property: The identity of the experiment resource. - * + * * @return the identity value. */ ResourceIdentity identity(); /** * Gets the provisioningState property: Most recent provisioning state for the given experiment resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the steps property: List of steps. - * + * * @return the steps value. */ List steps(); /** * Gets the selectors property: List of selectors. - * + * * @return the selectors value. */ List selectors(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.ExperimentInner object. - * + * * @return the inner object. */ ExperimentInner innerModel(); - /** The entirety of the Experiment definition. */ + /** + * The entirety of the Experiment definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithSteps, - DefinitionStages.WithSelectors, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithLocation, DefinitionStages.WithResourceGroup, + DefinitionStages.WithSteps, DefinitionStages.WithSelectors, DefinitionStages.WithCreate { } - /** The Experiment definition stages. */ + /** + * The Experiment definition stages. + */ interface DefinitionStages { - /** The first stage of the Experiment definition. */ + /** + * The first stage of the Experiment definition. + */ interface Blank extends WithLocation { } - /** The stage of the Experiment definition allowing to specify location. */ + /** + * The stage of the Experiment definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -139,40 +145,46 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Experiment definition allowing to specify parent resource. */ + /** + * The stage of the Experiment definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName String that represents an Azure resource group. * @return the next definition stage. */ WithSteps withExistingResourceGroup(String resourceGroupName); } - /** The stage of the Experiment definition allowing to specify steps. */ + /** + * The stage of the Experiment definition allowing to specify steps. + */ interface WithSteps { /** * Specifies the steps property: List of steps.. - * + * * @param steps List of steps. * @return the next definition stage. */ WithSelectors withSteps(List steps); } - /** The stage of the Experiment definition allowing to specify selectors. */ + /** + * The stage of the Experiment definition allowing to specify selectors. + */ interface WithSelectors { /** * Specifies the selectors property: List of selectors.. - * + * * @param selectors List of selectors. * @return the next definition stage. */ @@ -186,36 +198,40 @@ interface WithSelectors { interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity { /** * Executes the create request. - * + * * @return the created resource. */ Experiment create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Experiment create(Context context); } - /** The stage of the Experiment definition allowing to specify tags. */ + /** + * The stage of the Experiment definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Experiment definition allowing to specify identity. */ + /** + * The stage of the Experiment definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The identity of the experiment resource.. - * + * * @param identity The identity of the experiment resource. * @return the next definition stage. */ @@ -225,36 +241,55 @@ interface WithIdentity { /** * Begins update for the Experiment resource. - * + * * @return the stage of resource update. */ Experiment.Update update(); - /** The template for Experiment update. */ - interface Update extends UpdateStages.WithIdentity { + /** + * The template for Experiment update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity { /** * Executes the update request. - * + * * @return the updated resource. */ Experiment apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Experiment apply(Context context); } - /** The Experiment update stages. */ + /** + * The Experiment update stages. + */ interface UpdateStages { - /** The stage of the Experiment update allowing to specify identity. */ + /** + * The stage of the Experiment update allowing to specify tags. + */ + interface WithTags { + /** + * Specifies the tags property: The tags of the experiment resource.. + * + * @param tags The tags of the experiment resource. + * @return the next definition stage. + */ + Update withTags(Map tags); + } + + /** + * The stage of the Experiment update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The identity of the experiment resource.. - * + * * @param identity The identity of the experiment resource. * @return the next definition stage. */ @@ -264,14 +299,14 @@ interface WithIdentity { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Experiment refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -279,7 +314,7 @@ interface WithIdentity { /** * Cancel a running Experiment resource. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @@ -287,7 +322,7 @@ interface WithIdentity { /** * Cancel a running Experiment resource. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -297,7 +332,7 @@ interface WithIdentity { /** * Start a Experiment resource. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @@ -305,7 +340,7 @@ interface WithIdentity { /** * Start a Experiment resource. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecution.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecution.java index 4ddba48804ac2..84c51ef155c60 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecution.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecution.java @@ -7,53 +7,55 @@ import com.azure.resourcemanager.chaos.fluent.models.ExperimentExecutionInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of ExperimentExecution. */ +/** + * An immutable client-side representation of ExperimentExecution. + */ public interface ExperimentExecution { /** * Gets the type property: String of the resource type. - * + * * @return the type value. */ String type(); /** * Gets the id property: String of the fully qualified resource ID. - * + * * @return the id value. */ String id(); /** * Gets the name property: String of the resource name. - * + * * @return the name value. */ String name(); /** * Gets the status property: The status of the execution. - * + * * @return the status value. */ String status(); /** * Gets the startedAt property: String that represents the start date time. - * + * * @return the startedAt value. */ OffsetDateTime startedAt(); /** * Gets the stoppedAt property: String that represents the stop date time. - * + * * @return the stoppedAt value. */ OffsetDateTime stoppedAt(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.ExperimentExecutionInner object. - * + * * @return the inner object. */ ExperimentExecutionInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsError.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsError.java index 39adeaad1e032..7dbf1ab4fd96f 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsError.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsError.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Model that represents the Experiment action target details error model. */ +/** + * Model that represents the Experiment action target details error model. + */ @Immutable public final class ExperimentExecutionActionTargetDetailsError { /* @@ -22,13 +24,15 @@ public final class ExperimentExecutionActionTargetDetailsError { @JsonProperty(value = "message", access = JsonProperty.Access.WRITE_ONLY) private String message; - /** Creates an instance of ExperimentExecutionActionTargetDetailsError class. */ + /** + * Creates an instance of ExperimentExecutionActionTargetDetailsError class. + */ public ExperimentExecutionActionTargetDetailsError() { } /** * Get the code property: The error code. - * + * * @return the code value. */ public String code() { @@ -37,7 +41,7 @@ public String code() { /** * Get the message property: The error message. - * + * * @return the message value. */ public String message() { @@ -46,7 +50,7 @@ public String message() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsProperties.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsProperties.java index b238ea3c146e8..9d46d9bd9c4e9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsProperties.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionActionTargetDetailsProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Model that represents the Experiment action target details properties model. */ +/** + * Model that represents the Experiment action target details properties model. + */ @Immutable public final class ExperimentExecutionActionTargetDetailsProperties { /* @@ -41,13 +43,15 @@ public final class ExperimentExecutionActionTargetDetailsProperties { @JsonProperty(value = "error", access = JsonProperty.Access.WRITE_ONLY) private ExperimentExecutionActionTargetDetailsError error; - /** Creates an instance of ExperimentExecutionActionTargetDetailsProperties class. */ + /** + * Creates an instance of ExperimentExecutionActionTargetDetailsProperties class. + */ public ExperimentExecutionActionTargetDetailsProperties() { } /** * Get the status property: The status of the execution. - * + * * @return the status value. */ public String status() { @@ -56,7 +60,7 @@ public String status() { /** * Get the target property: The target for the action. - * + * * @return the target value. */ public String target() { @@ -65,7 +69,7 @@ public String target() { /** * Get the targetFailedTime property: String that represents the failed date time. - * + * * @return the targetFailedTime value. */ public OffsetDateTime targetFailedTime() { @@ -74,7 +78,7 @@ public OffsetDateTime targetFailedTime() { /** * Get the targetCompletedTime property: String that represents the completed date time. - * + * * @return the targetCompletedTime value. */ public OffsetDateTime targetCompletedTime() { @@ -83,7 +87,7 @@ public OffsetDateTime targetCompletedTime() { /** * Get the error property: The error of the action. - * + * * @return the error value. */ public ExperimentExecutionActionTargetDetailsError error() { @@ -92,7 +96,7 @@ public ExperimentExecutionActionTargetDetailsError error() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetails.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetails.java index 67895a8e836da..a3389a3da0900 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetails.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetails.java @@ -7,74 +7,76 @@ import com.azure.resourcemanager.chaos.fluent.models.ExperimentExecutionDetailsInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of ExperimentExecutionDetails. */ +/** + * An immutable client-side representation of ExperimentExecutionDetails. + */ public interface ExperimentExecutionDetails { /** * Gets the type property: String of the resource type. - * + * * @return the type value. */ String type(); /** * Gets the id property: String of the fully qualified resource ID. - * + * * @return the id value. */ String id(); /** * Gets the name property: String of the resource name. - * + * * @return the name value. */ String name(); /** * Gets the failureReason property: The reason why the execution failed. - * + * * @return the failureReason value. */ String failureReason(); /** * Gets the lastActionAt property: String that represents the last action date time. - * + * * @return the lastActionAt value. */ OffsetDateTime lastActionAt(); /** * Gets the runInformation property: The information of the experiment run. - * + * * @return the runInformation value. */ ExperimentExecutionDetailsPropertiesRunInformation runInformation(); /** * Gets the status property: The status of the execution. - * + * * @return the status value. */ String status(); /** * Gets the startedAt property: String that represents the start date time. - * + * * @return the startedAt value. */ OffsetDateTime startedAt(); /** * Gets the stoppedAt property: String that represents the stop date time. - * + * * @return the stoppedAt value. */ OffsetDateTime stoppedAt(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.ExperimentExecutionDetailsInner object. - * + * * @return the inner object. */ ExperimentExecutionDetailsInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetailsPropertiesRunInformation.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetailsPropertiesRunInformation.java index 11f4e69ec52cc..9dea24cb514fa 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetailsPropertiesRunInformation.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionDetailsPropertiesRunInformation.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The information of the experiment run. */ +/** + * The information of the experiment run. + */ @Immutable public final class ExperimentExecutionDetailsPropertiesRunInformation { /* @@ -17,13 +19,15 @@ public final class ExperimentExecutionDetailsPropertiesRunInformation { @JsonProperty(value = "steps", access = JsonProperty.Access.WRITE_ONLY) private List steps; - /** Creates an instance of ExperimentExecutionDetailsPropertiesRunInformation class. */ + /** + * Creates an instance of ExperimentExecutionDetailsPropertiesRunInformation class. + */ public ExperimentExecutionDetailsPropertiesRunInformation() { } /** * Get the steps property: The steps of the experiment run. - * + * * @return the steps value. */ public List steps() { @@ -32,7 +36,7 @@ public List steps() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionListResult.java index 69997a14cbb45..d5a187706a501 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentExecutionListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a list of Experiment executions and a link for pagination. */ +/** + * Model that represents a list of Experiment executions and a link for pagination. + */ @Immutable public final class ExperimentExecutionListResult { /* @@ -24,13 +26,15 @@ public final class ExperimentExecutionListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ExperimentExecutionListResult class. */ + /** + * Creates an instance of ExperimentExecutionListResult class. + */ public ExperimentExecutionListResult() { } /** * Get the value property: List of Experiment executions. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: URL to retrieve the next page of Experiment executions. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentListResult.java index 3c55db51087b0..4865337f29486 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a list of Experiment resources and a link for pagination. */ +/** + * Model that represents a list of Experiment resources and a link for pagination. + */ @Immutable public final class ExperimentListResult { /* @@ -24,13 +26,15 @@ public final class ExperimentListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ExperimentListResult class. */ + /** + * Creates an instance of ExperimentListResult class. + */ public ExperimentListResult() { } /** * Get the value property: List of Experiment resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: URL to retrieve the next page of Experiment resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentUpdate.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentUpdate.java index 9f36b57da5487..a58e286236650 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentUpdate.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ExperimentUpdate.java @@ -5,9 +5,13 @@ package com.azure.resourcemanager.chaos.models; import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.Map; -/** Describes an experiment update. */ +/** + * Describes an experiment update. + */ @Fluent public final class ExperimentUpdate { /* @@ -16,13 +20,22 @@ public final class ExperimentUpdate { @JsonProperty(value = "identity") private ResourceIdentity identity; - /** Creates an instance of ExperimentUpdate class. */ + /* + * The tags of the experiment resource. + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; + + /** + * Creates an instance of ExperimentUpdate class. + */ public ExperimentUpdate() { } /** * Get the identity property: The identity of the experiment resource. - * + * * @return the identity value. */ public ResourceIdentity identity() { @@ -31,7 +44,7 @@ public ResourceIdentity identity() { /** * Set the identity property: The identity of the experiment resource. - * + * * @param identity the identity value to set. * @return the ExperimentUpdate object itself. */ @@ -40,9 +53,29 @@ public ExperimentUpdate withIdentity(ResourceIdentity identity) { return this; } + /** + * Get the tags property: The tags of the experiment resource. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; + } + + /** + * Set the tags property: The tags of the experiment resource. + * + * @param tags the tags value to set. + * @return the ExperimentUpdate object itself. + */ + public ExperimentUpdate withTags(Map tags) { + this.tags = tags; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiments.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiments.java index 8505040f5fbd6..60f1f37688616 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiments.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Experiments.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Experiments. */ +/** + * Resource collection API of Experiments. + */ public interface Experiments { /** * Get a list of Experiment resources in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a subscription as paginated response with {@link PagedIterable}. @@ -21,9 +23,9 @@ public interface Experiments { /** * Get a list of Experiment resources in a subscription. - * + * * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -35,7 +37,7 @@ public interface Experiments { /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -46,10 +48,10 @@ public interface Experiments { /** * Get a list of Experiment resources in a resource group. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param running Optional value that indicates whether to filter results based on if the Experiment is currently - * running. If null, then the results will not be filtered. + * running. If null, then the results will not be filtered. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -57,12 +59,12 @@ public interface Experiments { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Experiment resources in a resource group as paginated response with {@link PagedIterable}. */ - PagedIterable listByResourceGroup( - String resourceGroupName, Boolean running, String continuationToken, Context context); + PagedIterable listByResourceGroup(String resourceGroupName, Boolean running, String continuationToken, + Context context); /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -73,7 +75,7 @@ PagedIterable listByResourceGroup( /** * Delete a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -85,7 +87,7 @@ PagedIterable listByResourceGroup( /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -94,12 +96,12 @@ PagedIterable listByResourceGroup( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Experiment resource along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String experimentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String experimentName, + Context context); /** * Get a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -111,7 +113,7 @@ Response getByResourceGroupWithResponse( /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -122,7 +124,7 @@ Response getByResourceGroupWithResponse( /** * Cancel a running Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -134,7 +136,7 @@ Response getByResourceGroupWithResponse( /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -145,7 +147,7 @@ Response getByResourceGroupWithResponse( /** * Start a Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -157,7 +159,7 @@ Response getByResourceGroupWithResponse( /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -169,7 +171,7 @@ Response getByResourceGroupWithResponse( /** * Get a list of executions of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param context The context to associate with this operation. @@ -178,12 +180,12 @@ Response getByResourceGroupWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of executions of an Experiment resource as paginated response with {@link PagedIterable}. */ - PagedIterable listAllExecutions( - String resourceGroupName, String experimentName, Context context); + PagedIterable listAllExecutions(String resourceGroupName, String experimentName, + Context context); /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -193,12 +195,12 @@ PagedIterable listAllExecutions( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an execution of an Experiment resource along with {@link Response}. */ - Response getExecutionWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context); + Response getExecutionWithResponse(String resourceGroupName, String experimentName, + String executionId, Context context); /** * Get an execution of an Experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -211,7 +213,7 @@ Response getExecutionWithResponse( /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -221,12 +223,12 @@ Response getExecutionWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents the execution details of an Experiment along with {@link Response}. */ - Response executionDetailsWithResponse( - String resourceGroupName, String experimentName, String executionId, Context context); + Response executionDetailsWithResponse(String resourceGroupName, String experimentName, + String executionId, Context context); /** * Execution details of an experiment resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param experimentName String that represents a Experiment resource name. * @param executionId GUID that represents a Experiment execution detail. @@ -239,7 +241,7 @@ Response executionDetailsWithResponse( /** * Get a Experiment resource. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -250,7 +252,7 @@ Response executionDetailsWithResponse( /** * Get a Experiment resource. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -262,7 +264,7 @@ Response executionDetailsWithResponse( /** * Delete a Experiment resource. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -272,7 +274,7 @@ Response executionDetailsWithResponse( /** * Delete a Experiment resource. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -283,7 +285,7 @@ Response executionDetailsWithResponse( /** * Begins definition for a new Experiment resource. - * + * * @param name resource name. * @return the first stage of the new Experiment definition. */ diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/FilterType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/FilterType.java index 53ee9caa8191a..403376e735372 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/FilterType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/FilterType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum that discriminates between filter types. Currently only `Simple` type is supported. */ +/** + * Enum that discriminates between filter types. Currently only `Simple` type is supported. + */ public final class FilterType extends ExpandableStringEnum { - /** Static value Simple for FilterType. */ + /** + * Static value Simple for FilterType. + */ public static final FilterType SIMPLE = fromString("Simple"); /** * Creates a new instance of FilterType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public FilterType() { /** * Creates or finds a FilterType from its string representation. - * + * * @param name a name to look for. * @return the corresponding FilterType. */ @@ -35,7 +39,7 @@ public static FilterType fromString(String name) { /** * Gets known FilterType values. - * + * * @return known FilterType values. */ public static Collection values() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/KeyValuePair.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/KeyValuePair.java index 8b63f9050d981..2f235f21b6a61 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/KeyValuePair.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/KeyValuePair.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** A map to describe the settings of an action. */ +/** + * A map to describe the settings of an action. + */ @Fluent public final class KeyValuePair { /* @@ -23,13 +25,15 @@ public final class KeyValuePair { @JsonProperty(value = "value", required = true) private String value; - /** Creates an instance of KeyValuePair class. */ + /** + * Creates an instance of KeyValuePair class. + */ public KeyValuePair() { } /** * Get the key property: The name of the setting for the action. - * + * * @return the key value. */ public String key() { @@ -38,7 +42,7 @@ public String key() { /** * Set the key property: The name of the setting for the action. - * + * * @param key the key value to set. * @return the KeyValuePair object itself. */ @@ -49,7 +53,7 @@ public KeyValuePair withKey(String key) { /** * Get the value property: The value of the setting for the action. - * + * * @return the value value. */ public String value() { @@ -58,7 +62,7 @@ public String value() { /** * Set the value property: The value of the setting for the action. - * + * * @param value the value value to set. * @return the KeyValuePair object itself. */ @@ -69,19 +73,17 @@ public KeyValuePair withValue(String value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (key() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property key in model KeyValuePair")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property key in model KeyValuePair")); } if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model KeyValuePair")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model KeyValuePair")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operation.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operation.java index b5c9d7060c9d1..c1f1253969710 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operation.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operation.java @@ -6,12 +6,14 @@ import com.azure.resourcemanager.chaos.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ String name(); @@ -19,14 +21,14 @@ public interface Operation { /** * Gets the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Localized display information for this particular operation. - * + * * @return the display value. */ OperationDisplay display(); @@ -34,7 +36,7 @@ public interface Operation { /** * Gets the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ Origin origin(); @@ -42,14 +44,14 @@ public interface Operation { /** * Gets the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ ActionType actionType(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationDisplay.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationDisplay.java index c94f5b20ac746..fb7918a6873ac 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationDisplay.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Localized display information for this particular operation. */ +/** + * Localized display information for this particular operation. + */ @Immutable public final class OperationDisplay { /* @@ -37,14 +39,16 @@ public final class OperationDisplay { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: The localized friendly form of the resource provider name, e.g. "Microsoft Monitoring * Insights" or "Microsoft Compute". - * + * * @return the provider value. */ public String provider() { @@ -54,7 +58,7 @@ public String provider() { /** * Get the resource property: The localized friendly name of the resource type related to this operation. E.g. * "Virtual Machines" or "Job Schedule Collections". - * + * * @return the resource value. */ public String resource() { @@ -64,7 +68,7 @@ public String resource() { /** * Get the operation property: The concise, localized friendly name for the operation; suitable for dropdowns. E.g. * "Create or Update Virtual Machine", "Restart Virtual Machine". - * + * * @return the operation value. */ public String operation() { @@ -74,7 +78,7 @@ public String operation() { /** * Get the description property: The short, localized friendly description of the operation; suitable for tool tips * and detailed views. - * + * * @return the description value. */ public String description() { @@ -83,7 +87,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationListResult.java index 62f3cb4b30bf7..098f1446216e8 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationListResult.java @@ -10,8 +10,8 @@ import java.util.List; /** - * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set of - * results. + * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set + * of results. */ @Immutable public final class OperationListResult { @@ -27,13 +27,15 @@ public final class OperationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationListResult class. */ + /** + * Creates an instance of OperationListResult class. + */ public OperationListResult() { } /** * Get the value property: List of operations supported by the resource provider. - * + * * @return the value value. */ public List value() { @@ -42,7 +44,7 @@ public List value() { /** * Get the nextLink property: URL to get the next set of operation list results (if there are any). - * + * * @return the nextLink value. */ public String nextLink() { @@ -51,7 +53,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatus.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatus.java index 471fff86a4e12..b1e094af9f5e2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatus.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatus.java @@ -7,53 +7,55 @@ import com.azure.core.management.exception.ManagementError; import com.azure.resourcemanager.chaos.fluent.models.OperationStatusInner; -/** An immutable client-side representation of OperationStatus. */ +/** + * An immutable client-side representation of OperationStatus. + */ public interface OperationStatus { /** * Gets the id property: The operation Id. - * + * * @return the id value. */ String id(); /** * Gets the name property: The operation name. - * + * * @return the name value. */ String name(); /** * Gets the startTime property: The start time of the operation. - * + * * @return the startTime value. */ String startTime(); /** * Gets the endTime property: The end time of the operation. - * + * * @return the endTime value. */ String endTime(); /** * Gets the status property: The status of the operation. - * + * * @return the status value. */ String status(); /** * Gets the error property: The error object. - * + * * @return the error value. */ ManagementError error(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.OperationStatusInner object. - * + * * @return the inner object. */ OperationStatusInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatuses.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatuses.java index 3858ebd24b3c6..54ddb12e295ab 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatuses.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/OperationStatuses.java @@ -7,12 +7,14 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of OperationStatuses. */ +/** + * Resource collection API of OperationStatuses. + */ public interface OperationStatuses { /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,8 +26,8 @@ public interface OperationStatuses { /** * Get the status of a long running azure asynchronous operation. - * - * @param location The region name of operation. + * + * @param location The name of the Azure region. * @param asyncOperationId The operation Id. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operations.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operations.java index a0fb76ffe3943..31ed96d56468b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operations.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Operations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Get a list all available Operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list all available Operations as paginated response with {@link PagedIterable}. @@ -20,7 +22,7 @@ public interface Operations { /** * Get a list all available Operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Origin.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Origin.java index daccb78e7d022..41a6ec69321b5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Origin.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Origin.java @@ -13,18 +13,24 @@ * is "user,system". */ public final class Origin extends ExpandableStringEnum { - /** Static value user for Origin. */ + /** + * Static value user for Origin. + */ public static final Origin USER = fromString("user"); - /** Static value system for Origin. */ + /** + * Static value system for Origin. + */ public static final Origin SYSTEM = fromString("system"); - /** Static value user,system for Origin. */ + /** + * Static value user,system for Origin. + */ public static final Origin USER_SYSTEM = fromString("user,system"); /** * Creates a new instance of Origin value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +39,7 @@ public Origin() { /** * Creates or finds a Origin from its string representation. - * + * * @param name a name to look for. * @return the corresponding Origin. */ @@ -44,7 +50,7 @@ public static Origin fromString(String name) { /** * Gets known Origin values. - * + * * @return known Origin values. */ public static Collection values() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ProvisioningState.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ProvisioningState.java index 845d724d149aa..bd11aa2f6ec41 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ProvisioningState.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ProvisioningState.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current provisioning state for a given Azure Chaos resource. */ +/** + * Current provisioning state for a given Azure Chaos resource. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value Creating for ProvisioningState. */ + /** + * Static value Creating for ProvisioningState. + */ public static final ProvisioningState CREATING = fromString("Creating"); - /** Static value Updating for ProvisioningState. */ + /** + * Static value Updating for ProvisioningState. + */ public static final ProvisioningState UPDATING = fromString("Updating"); - /** Static value Deleting for ProvisioningState. */ + /** + * Static value Deleting for ProvisioningState. + */ public static final ProvisioningState DELETING = fromString("Deleting"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -50,7 +64,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentity.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentity.java index 7dcd66abfac37..4a2268f6a8cdc 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentity.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentity.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** The identity of a resource. */ +/** + * The identity of a resource. + */ @Fluent public final class ResourceIdentity { /* @@ -22,7 +24,8 @@ public final class ResourceIdentity { /* * The list of user identities associated with the Experiment. The user identity dictionary key references will be * ARM resource ids in the form: - * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}'. + * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/ + * userAssignedIdentities/{identityName}'. */ @JsonProperty(value = "userAssignedIdentities") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) @@ -40,13 +43,15 @@ public final class ResourceIdentity { @JsonProperty(value = "tenantId", access = JsonProperty.Access.WRITE_ONLY) private String tenantId; - /** Creates an instance of ResourceIdentity class. */ + /** + * Creates an instance of ResourceIdentity class. + */ public ResourceIdentity() { } /** * Get the type property: String of the resource identity type. - * + * * @return the type value. */ public ResourceIdentityType type() { @@ -55,7 +60,7 @@ public ResourceIdentityType type() { /** * Set the type property: String of the resource identity type. - * + * * @param type the type value to set. * @return the ResourceIdentity object itself. */ @@ -68,7 +73,7 @@ public ResourceIdentity withType(ResourceIdentityType type) { * Get the userAssignedIdentities property: The list of user identities associated with the Experiment. The user * identity dictionary key references will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}'. - * + * * @return the userAssignedIdentities value. */ public Map userAssignedIdentities() { @@ -79,7 +84,7 @@ public Map userAssignedIdentities() { * Set the userAssignedIdentities property: The list of user identities associated with the Experiment. The user * identity dictionary key references will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}'. - * + * * @param userAssignedIdentities the userAssignedIdentities value to set. * @return the ResourceIdentity object itself. */ @@ -90,7 +95,7 @@ public ResourceIdentity withUserAssignedIdentities(Map { - if (e != null) { - e.validate(); - } - }); + userAssignedIdentities().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentityType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentityType.java index 682743cc21364..f8f9bb37b4c29 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentityType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/ResourceIdentityType.java @@ -7,18 +7,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** String of the resource identity type. */ +/** + * String of the resource identity type. + */ public enum ResourceIdentityType { - /** Enum value None. */ + /** + * Enum value None. + */ NONE("None"), - /** Enum value SystemAssigned. */ + /** + * Enum value SystemAssigned. + */ SYSTEM_ASSIGNED("SystemAssigned"), - /** Enum value UserAssigned. */ + /** + * Enum value UserAssigned. + */ USER_ASSIGNED("UserAssigned"); - /** The actual serialized value for a ResourceIdentityType instance. */ + /** + * The actual serialized value for a ResourceIdentityType instance. + */ private final String value; ResourceIdentityType(String value) { @@ -27,7 +37,7 @@ public enum ResourceIdentityType { /** * Parses a serialized value to a ResourceIdentityType instance. - * + * * @param value the serialized value to parse. * @return the parsed ResourceIdentityType object, or null if unable to parse. */ @@ -45,7 +55,9 @@ public static ResourceIdentityType fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/SelectorType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/SelectorType.java index 1ea697825d764..1776fa6c7b616 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/SelectorType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/SelectorType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum of the selector type. */ +/** + * Enum of the selector type. + */ public final class SelectorType extends ExpandableStringEnum { - /** Static value List for SelectorType. */ + /** + * Static value List for SelectorType. + */ public static final SelectorType LIST = fromString("List"); - /** Static value Query for SelectorType. */ + /** + * Static value Query for SelectorType. + */ public static final SelectorType QUERY = fromString("Query"); /** * Creates a new instance of SelectorType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SelectorType() { /** * Creates or finds a SelectorType from its string representation. - * + * * @param name a name to look for. * @return the corresponding SelectorType. */ @@ -38,7 +44,7 @@ public static SelectorType fromString(String name) { /** * Gets known SelectorType values. - * + * * @return known SelectorType values. */ public static Collection values() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/StepStatus.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/StepStatus.java index c4670d561f380..e615c194cedac 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/StepStatus.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/StepStatus.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents the a list of branches and branch statuses. */ +/** + * Model that represents the a list of branches and branch statuses. + */ @Immutable public final class StepStatus { /* @@ -35,13 +37,15 @@ public final class StepStatus { @JsonProperty(value = "branches", access = JsonProperty.Access.WRITE_ONLY) private List branches; - /** Creates an instance of StepStatus class. */ + /** + * Creates an instance of StepStatus class. + */ public StepStatus() { } /** * Get the stepName property: The name of the step. - * + * * @return the stepName value. */ public String stepName() { @@ -50,7 +54,7 @@ public String stepName() { /** * Get the stepId property: The id of the step. - * + * * @return the stepId value. */ public String stepId() { @@ -59,7 +63,7 @@ public String stepId() { /** * Get the status property: The value of the status of the step. - * + * * @return the status value. */ public String status() { @@ -68,7 +72,7 @@ public String status() { /** * Get the branches property: The array of branches. - * + * * @return the branches value. */ public List branches() { @@ -77,7 +81,7 @@ public List branches() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Target.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Target.java index 064f0d74239b0..a4ada3a72e4c7 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Target.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Target.java @@ -8,53 +8,55 @@ import com.azure.resourcemanager.chaos.fluent.models.TargetInner; import java.util.Map; -/** An immutable client-side representation of Target. */ +/** + * An immutable client-side representation of Target. + */ public interface Target { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: The system metadata of the target resource. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the location property: Location of the target resource. - * + * * @return the location value. */ String location(); /** * Gets the properties property: The properties of the target resource. - * + * * @return the properties value. */ Map properties(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.TargetInner object. - * + * * @return the inner object. */ TargetInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetListResult.java index de6d488fdf68b..6fe46f644dd77 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a list of Target resources and a link for pagination. */ +/** + * Model that represents a list of Target resources and a link for pagination. + */ @Immutable public final class TargetListResult { /* @@ -24,13 +26,15 @@ public final class TargetListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of TargetListResult class. */ + /** + * Creates an instance of TargetListResult class. + */ public TargetListResult() { } /** * Get the value property: List of Target resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: URL to retrieve the next page of Target resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReference.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReference.java index d6d40028cf072..dfda7e9f14915 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReference.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReference.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Model that represents a reference to a Target in the selector. */ +/** + * Model that represents a reference to a Target in the selector. + */ @Fluent public final class TargetReference { /* @@ -23,13 +25,15 @@ public final class TargetReference { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of TargetReference class. */ + /** + * Creates an instance of TargetReference class. + */ public TargetReference() { } /** * Get the type property: Enum of the Target reference type. - * + * * @return the type value. */ public TargetReferenceType type() { @@ -38,7 +42,7 @@ public TargetReferenceType type() { /** * Set the type property: Enum of the Target reference type. - * + * * @param type the type value to set. * @return the TargetReference object itself. */ @@ -49,7 +53,7 @@ public TargetReference withType(TargetReferenceType type) { /** * Get the id property: String of the resource ID of a Target resource. - * + * * @return the id value. */ public String id() { @@ -58,7 +62,7 @@ public String id() { /** * Set the id property: String of the resource ID of a Target resource. - * + * * @param id the id value to set. * @return the TargetReference object itself. */ @@ -69,19 +73,17 @@ public TargetReference withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (type() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property type in model TargetReference")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property type in model TargetReference")); } if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model TargetReference")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model TargetReference")); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReferenceType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReferenceType.java index 81ed19e0d1235..6e196ebcd7f12 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReferenceType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetReferenceType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum of the Target reference type. */ +/** + * Enum of the Target reference type. + */ public final class TargetReferenceType extends ExpandableStringEnum { - /** Static value ChaosTarget for TargetReferenceType. */ + /** + * Static value ChaosTarget for TargetReferenceType. + */ public static final TargetReferenceType CHAOS_TARGET = fromString("ChaosTarget"); /** * Creates a new instance of TargetReferenceType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public TargetReferenceType() { /** * Creates or finds a TargetReferenceType from its string representation. - * + * * @param name a name to look for. * @return the corresponding TargetReferenceType. */ @@ -35,7 +39,7 @@ public static TargetReferenceType fromString(String name) { /** * Gets known TargetReferenceType values. - * + * * @return known TargetReferenceType values. */ public static Collection values() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetType.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetType.java index ebc7a5164cf01..f464af6c8f3cb 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetType.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetType.java @@ -8,74 +8,76 @@ import com.azure.resourcemanager.chaos.fluent.models.TargetTypeInner; import java.util.List; -/** An immutable client-side representation of TargetType. */ +/** + * An immutable client-side representation of TargetType. + */ public interface TargetType { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: The system metadata properties of the target type resource. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the location property: Location of the Target Type resource. - * + * * @return the location value. */ String location(); /** * Gets the displayName property: Localized string of the display name. - * + * * @return the displayName value. */ String displayName(); /** * Gets the description property: Localized string of the description. - * + * * @return the description value. */ String description(); /** * Gets the propertiesSchema property: URL to retrieve JSON schema of the Target Type properties. - * + * * @return the propertiesSchema value. */ String propertiesSchema(); /** * Gets the resourceTypes property: List of resource types this Target Type can extend. - * + * * @return the resourceTypes value. */ List resourceTypes(); /** * Gets the inner com.azure.resourcemanager.chaos.fluent.models.TargetTypeInner object. - * + * * @return the inner object. */ TargetTypeInner innerModel(); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypeListResult.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypeListResult.java index ffb021af968c5..1eca9e0228510 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypeListResult.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypeListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Model that represents a list of Target Type resources and a link for pagination. */ +/** + * Model that represents a list of Target Type resources and a link for pagination. + */ @Immutable public final class TargetTypeListResult { /* @@ -24,13 +26,15 @@ public final class TargetTypeListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of TargetTypeListResult class. */ + /** + * Creates an instance of TargetTypeListResult class. + */ public TargetTypeListResult() { } /** * Get the value property: List of Target Type resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: URL to retrieve the next page of Target Type resources. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypes.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypes.java index 8ff793a6ccfaf..15b8a39602422 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypes.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/TargetTypes.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of TargetTypes. */ +/** + * Resource collection API of TargetTypes. + */ public interface TargetTypes { /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -23,7 +25,7 @@ public interface TargetTypes { /** * Get a list of Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param continuationToken String that sets the continuation token. * @param context The context to associate with this operation. @@ -36,7 +38,7 @@ public interface TargetTypes { /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @param context The context to associate with this operation. @@ -49,7 +51,7 @@ public interface TargetTypes { /** * Get a Target Type resources for given location. - * + * * @param locationName String that represents a Location resource name. * @param targetTypeName String that represents a Target Type resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Targets.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Targets.java index 6187753e73276..12dc0dcad5a50 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Targets.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/Targets.java @@ -9,11 +9,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.chaos.fluent.models.TargetInner; -/** Resource collection API of Targets. */ +/** + * Resource collection API of Targets. + */ public interface Targets { /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -21,15 +23,15 @@ public interface Targets { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedIterable}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedIterable}. */ - PagedIterable list( - String resourceGroupName, String parentProviderNamespace, String parentResourceType, String parentResourceName); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName); /** * Get a list of Target resources that extend a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -39,20 +41,15 @@ PagedIterable list( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Target resources that extend a tracked regional resource as paginated response with {@link - * PagedIterable}. + * @return a list of Target resources that extend a tracked regional resource as paginated response with + * {@link PagedIterable}. */ - PagedIterable list( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String continuationToken, - Context context); + PagedIterable list(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String continuationToken, Context context); /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -64,17 +61,12 @@ PagedIterable list( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Target resource that extends a tracked regional resource along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context); + Response getWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context); /** * Get a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -85,16 +77,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Target resource that extends a tracked regional resource. */ - Target get( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName); + Target get(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName); /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -106,17 +94,12 @@ Target get( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - Context context); + Response deleteWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, Context context); /** * Delete a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -126,16 +109,12 @@ Response deleteWithResponse( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName); + void delete(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName); /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -148,18 +127,12 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Target resource along with {@link Response}. */ - Response createOrUpdateWithResponse( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String parentProviderNamespace, + String parentResourceType, String parentResourceName, String targetName, TargetInner target, Context context); /** * Create or update a Target resource that extends a tracked regional resource. - * + * * @param resourceGroupName String that represents an Azure resource group. * @param parentProviderNamespace String that represents a resource provider namespace. * @param parentResourceType String that represents a resource type. @@ -171,11 +144,6 @@ Response createOrUpdateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return model that represents a Target resource. */ - Target createOrUpdate( - String resourceGroupName, - String parentProviderNamespace, - String parentResourceType, - String parentResourceName, - String targetName, - TargetInner target); + Target createOrUpdate(String resourceGroupName, String parentProviderNamespace, String parentResourceType, + String parentResourceName, String targetName, TargetInner target); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/UserAssignedIdentity.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/UserAssignedIdentity.java index e83eda8f0bcbf..7f8c9eba71ef9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/UserAssignedIdentity.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/UserAssignedIdentity.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.UUID; -/** User assigned identity properties. */ +/** + * User assigned identity properties. + */ @Immutable public final class UserAssignedIdentity { /* @@ -23,13 +25,15 @@ public final class UserAssignedIdentity { @JsonProperty(value = "clientId", access = JsonProperty.Access.WRITE_ONLY) private UUID clientId; - /** Creates an instance of UserAssignedIdentity class. */ + /** + * Creates an instance of UserAssignedIdentity class. + */ public UserAssignedIdentity() { } /** * Get the principalId property: The principal ID of the assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -38,7 +42,7 @@ public UUID principalId() { /** * Get the clientId property: The client ID of the assigned identity. - * + * * @return the clientId value. */ public UUID clientId() { @@ -47,7 +51,7 @@ public UUID clientId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/package-info.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/package-info.java index 081adb707661f..1c102fc54e047 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/package-info.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the data models for ChaosManagementClient. Chaos Management Client. */ +/** + * Package containing the data models for ChaosManagementClient. + * Chaos Management Client. + */ package com.azure.resourcemanager.chaos.models; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/package-info.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/package-info.java index 9da2e994a2c79..e51a56207ae28 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/package-info.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/com/azure/resourcemanager/chaos/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for ChaosManagementClient. Chaos Management Client. */ +/** + * Package containing the classes for ChaosManagementClient. + * Chaos Management Client. + */ package com.azure.resourcemanager.chaos; diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/module-info.java b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/module-info.java index 0a9080fade2a8..ec75a9114c7d2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/main/java/module-info.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.chaos { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.chaos; exports com.azure.resourcemanager.chaos.fluent; exports com.azure.resourcemanager.chaos.fluent.models; exports com.azure.resourcemanager.chaos.models; - - opens com.azure.resourcemanager.chaos.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.chaos.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.chaos.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.chaos.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateSamples.java index bff7d610edc17..a1e390fbe238d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateSamples.java @@ -6,28 +6,23 @@ import com.azure.resourcemanager.chaos.fluent.models.CapabilityInner; -/** Samples for Capabilities CreateOrUpdate. */ +/** + * Samples for Capabilities CreateOrUpdate. + */ public final class CapabilitiesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CreateUpdateCapability.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CreateUpdateCapability.json */ /** * Sample code: Create/update a Capability that extends a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ public static void createUpdateACapabilityThatExtendsAVirtualMachineTargetResource( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .createOrUpdateWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - "Shutdown-1.0", - new CapabilityInner(), - com.azure.core.util.Context.NONE); + manager.capabilities().createOrUpdateWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", + "exampleVM", "Microsoft-VirtualMachine", "Shutdown-1.0", new CapabilityInner(), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteSamples.java index c0711247e7786..e50d25508504d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteSamples.java @@ -4,27 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Capabilities Delete. */ +/** + * Samples for Capabilities Delete. + */ public final class CapabilitiesDeleteSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DeleteCapability.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DeleteCapability.json */ /** * Sample code: Delete a Capability that extends a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ public static void deleteACapabilityThatExtendsAVirtualMachineTargetResource( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .deleteWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - "Shutdown-1.0", - com.azure.core.util.Context.NONE); + manager.capabilities().deleteWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-VirtualMachine", "Shutdown-1.0", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetSamples.java index c9fcb72d1c84c..f551931ddaf07 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetSamples.java @@ -4,27 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Capabilities Get. */ +/** + * Samples for Capabilities Get. + */ public final class CapabilitiesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetCapability.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetCapability.json */ /** * Sample code: Get a Capability that extends a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void getACapabilityThatExtendsAVirtualMachineTargetResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .getWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - "Shutdown-1.0", - com.azure.core.util.Context.NONE); + public static void + getACapabilityThatExtendsAVirtualMachineTargetResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.capabilities().getWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-VirtualMachine", "Shutdown-1.0", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListSamples.java index e9877ec3e950f..8df30c20895ff 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListSamples.java @@ -4,27 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Capabilities List. */ +/** + * Samples for Capabilities List. + */ public final class CapabilitiesListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListCapabilities.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListCapabilities.json */ /** * Sample code: List all Capabilities that extend a virtual machine Target resource. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllCapabilitiesThatExtendAVirtualMachineTargetResource( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilities() - .list( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-VirtualMachine", - null, - com.azure.core.util.Context.NONE); + manager.capabilities().list("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-VirtualMachine", null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetSamples.java index f6ccf64f5fe37..2dfe563f27d7e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for CapabilityTypes Get. */ +/** + * Samples for CapabilityTypes Get. + */ public final class CapabilityTypesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetCapabilityType.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetCapabilityType.json */ /** * Sample code: Get a Capability Type for a virtual machine Target resource on westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void getACapabilityTypeForAVirtualMachineTargetResourceOnWestus2Location( com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .capabilityTypes() - .getWithResponse("westus2", "Microsoft-VirtualMachine", "Shutdown-1.0", com.azure.core.util.Context.NONE); + manager.capabilityTypes().getWithResponse("westus2", "Microsoft-VirtualMachine", "Shutdown-1.0", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListSamples.java index 3d6fb9271308d..41d3af07ff147 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for CapabilityTypes List. */ +/** + * Samples for CapabilityTypes List. + */ public final class CapabilityTypesListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListCapabilityTypes.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListCapabilityTypes.json */ /** * Sample code: List all Capability Types for a virtual machine Target resource on westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllCapabilityTypesForAVirtualMachineTargetResourceOnWestus2Location( diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCancelSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCancelSamples.java index 11882ad7b0021..990b2f16fdb8e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCancelSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCancelSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments Cancel. */ +/** + * Samples for Experiments Cancel. + */ public final class ExperimentsCancelSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CancelExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CancelExperiment.json */ /** * Sample code: Cancel a running Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void cancelARunningExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateSamples.java index 7dd384185243e..a1e01b8f4864b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateSamples.java @@ -16,58 +16,34 @@ import java.time.Duration; import java.util.Arrays; -/** Samples for Experiments CreateOrUpdate. */ +/** + * Samples for Experiments CreateOrUpdate. + */ public final class ExperimentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CreateUpdateExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CreateUpdateExperiment.json */ /** * Sample code: Create/update a Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void createUpdateAExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .define("exampleExperiment") - .withRegion("eastus2euap") + manager.experiments().define("exampleExperiment").withRegion("eastus2euap") .withExistingResourceGroup("exampleRG") .withSteps( - Arrays - .asList( - new ChaosExperimentStep() - .withName("step1") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("branch1") - .withActions( - Arrays - .asList( - new ContinuousAction() - .withName("urn:csci:microsoft:virtualMachine:shutdown/1.0") - .withDuration(Duration.parse("PT10M")) - .withParameters( - Arrays - .asList( - new KeyValuePair() - .withKey("fakeTokenPlaceholder") - .withValue("false"))) - .withSelectorId("selector1"))))))) - .withSelectors( - Arrays - .asList( - new ChaosTargetListSelector() - .withId("selector1") - .withTargets( - Arrays - .asList( - new TargetReference() - .withType(TargetReferenceType.CHAOS_TARGET) - .withId( - "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.Compute/virtualMachines/exampleVM/providers/Microsoft.Chaos/targets/Microsoft-VirtualMachine"))))) - .withIdentity(new ResourceIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)) - .create(); + Arrays.asList(new ChaosExperimentStep().withName("step1") + .withBranches(Arrays.asList(new ChaosExperimentBranch().withName("branch1") + .withActions(Arrays.asList(new ContinuousAction() + .withName("urn:csci:microsoft:virtualMachine:shutdown/1.0") + .withDuration(Duration.parse("PT10M")) + .withParameters( + Arrays.asList(new KeyValuePair().withKey("fakeTokenPlaceholder").withValue("false"))) + .withSelectorId("selector1"))))))) + .withSelectors(Arrays.asList(new ChaosTargetListSelector().withId("selector1") + .withTargets(Arrays.asList(new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId( + "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.Compute/virtualMachines/exampleVM/providers/Microsoft.Chaos/targets/Microsoft-VirtualMachine"))))) + .withIdentity(new ResourceIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)).create(); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsDeleteSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsDeleteSamples.java index bfc219fab7b09..d71ed6f1c2f9e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsDeleteSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments Delete. */ +/** + * Samples for Experiments Delete. + */ public final class ExperimentsDeleteSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DeleteExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DeleteExperiment.json */ /** * Sample code: Delete a Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void deleteAExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsSamples.java index 5fef1f0dba905..b4b1018aeedaf 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsSamples.java @@ -4,23 +4,21 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments ExecutionDetails. */ +/** + * Samples for Experiments ExecutionDetails. + */ public final class ExperimentsExecutionDetailsSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DetailsExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DetailsExperiment.json */ /** * Sample code: Get experiment execution details. - * + * * @param manager Entry point to ChaosManager. */ public static void getExperimentExecutionDetails(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .executionDetailsWithResponse( - "exampleRG", - "exampleExperiment", - "f24500ad-744e-4a26-864b-b76199eac333", - com.azure.core.util.Context.NONE); + manager.experiments().executionDetailsWithResponse("exampleRG", "exampleExperiment", + "f24500ad-744e-4a26-864b-b76199eac333", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupSamples.java index 890dfb7f8d88a..bcff08a197cb5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments GetByResourceGroup. */ +/** + * Samples for Experiments GetByResourceGroup. + */ public final class ExperimentsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetExperiment.json */ /** * Sample code: Get a Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void getAExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .getByResourceGroupWithResponse("exampleRG", "exampleExperiment", com.azure.core.util.Context.NONE); + manager.experiments().getByResourceGroupWithResponse("exampleRG", "exampleExperiment", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionSamples.java index 28f28dd3c4e90..6c17f0dddf124 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionSamples.java @@ -4,23 +4,21 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments GetExecution. */ +/** + * Samples for Experiments GetExecution. + */ public final class ExperimentsGetExecutionSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetExperimentExecution.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetExperimentExecution.json */ /** * Sample code: Get the execution of a Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void getTheExecutionOfAExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .experiments() - .getExecutionWithResponse( - "exampleRG", - "exampleExperiment", - "f24500ad-744e-4a26-864b-b76199eac333", - com.azure.core.util.Context.NONE); + manager.experiments().getExecutionWithResponse("exampleRG", "exampleExperiment", + "f24500ad-744e-4a26-864b-b76199eac333", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsSamples.java index e7f7cb140fcfe..27038b5ac78d6 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments ListAllExecutions. */ +/** + * Samples for Experiments ListAllExecutions. + */ public final class ExperimentsListAllExecutionsSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListExperimentExecutions.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListExperimentExecutions.json */ /** * Sample code: List all executions of an Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllExecutionsOfAnExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupSamples.java index 2f36631b04eb6..15eebd7e5b8c3 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments ListByResourceGroup. */ +/** + * Samples for Experiments ListByResourceGroup. + */ public final class ExperimentsListByResourceGroupSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListExperimentsInAResourceGroup.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListExperimentsInAResourceGroup. + * json */ /** * Sample code: List all Experiments in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllExperimentsInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListSamples.java index 46c0dd24b7cf2..d9367565bfb47 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsListSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments List. */ +/** + * Samples for Experiments List. + */ public final class ExperimentsListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListExperimentsInASubscription.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListExperimentsInASubscription. + * json */ /** * Sample code: List all Experiments in a subscription. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllExperimentsInASubscription(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsStartSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsStartSamples.java index f1624202c0f89..e032207008d2b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsStartSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsStartSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Experiments Start. */ +/** + * Samples for Experiments Start. + */ public final class ExperimentsStartSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/StartExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/StartExperiment.json */ /** * Sample code: Start a Experiment. - * + * * @param manager Entry point to ChaosManager. */ public static void startAExperiment(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsUpdateSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsUpdateSamples.java index f5127f1918eb6..f53dce2d1c8ee 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsUpdateSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/ExperimentsUpdateSamples.java @@ -11,31 +11,27 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Experiments Update. */ +/** + * Samples for Experiments Update. + */ public final class ExperimentsUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/UpdateExperiment.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/UpdateExperiment.json */ /** * Sample code: Update an Experiment in a resource group. - * + * * @param manager Entry point to ChaosManager. */ public static void updateAnExperimentInAResourceGroup(com.azure.resourcemanager.chaos.ChaosManager manager) { - Experiment resource = - manager - .experiments() - .getByResourceGroupWithResponse("exampleRG", "exampleExperiment", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withIdentity( - new ResourceIdentity() - .withType(ResourceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.ManagedIdentity/userAssignedIdentity/exampleUMI", - new UserAssignedIdentity()))) + Experiment resource = manager.experiments() + .getByResourceGroupWithResponse("exampleRG", "exampleExperiment", com.azure.core.util.Context.NONE) + .getValue(); + resource.update().withTags(mapOf("key1", "fakeTokenPlaceholder", "key2", "fakeTokenPlaceholder")).withIdentity( + new ResourceIdentity().withType(ResourceIdentityType.USER_ASSIGNED).withUserAssignedIdentities(mapOf( + "/subscriptions/6b052e15-03d3-4f17-b2e1-be7f07588291/resourceGroups/exampleRG/providers/Microsoft.ManagedIdentity/userAssignedIdentity/exampleUMI", + new UserAssignedIdentity()))) .apply(); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetSamples.java index a9b80b0a21670..29c02fa4fa086 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for OperationStatuses Get. */ +/** + * Samples for OperationStatuses Get. + */ public final class OperationStatusesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetOperationStatus.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetOperationStatus.json */ /** * Sample code: Get specific operation status. - * + * * @param manager Entry point to ChaosManager. */ public static void getSpecificOperationStatus(com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .operationStatuses() - .getWithResponse("West US", "713192d7-503f-477a-9cfe-4efc3ee2bd11", com.azure.core.util.Context.NONE); + manager.operationStatuses().getWithResponse("West US", "713192d7-503f-477a-9cfe-4efc3ee2bd11", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetSamples.java index 6486c6f7d0e7c..8dcf3c0cc1d23 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for TargetTypes Get. */ +/** + * Samples for TargetTypes Get. + */ public final class TargetTypesGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetTargetType.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetTargetType.json */ /** * Sample code: Get a Target Type for westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void getATargetTypeForWestus2Location(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesListSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesListSamples.java index 4a09189e47ec3..206536f875f70 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesListSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetTypesListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for TargetTypes List. */ +/** + * Samples for TargetTypes List. + */ public final class TargetTypesListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListTargetTypes.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListTargetTypes.json */ /** * Sample code: List all Target Types for westus2 location. - * + * * @param manager Entry point to ChaosManager. */ public static void listAllTargetTypesForWestus2Location(com.azure.resourcemanager.chaos.ChaosManager manager) { diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateSamples.java index afc02f4e79eb4..8dd1ca5cff4c9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateSamples.java @@ -11,37 +11,28 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Targets CreateOrUpdate. */ +/** + * Samples for Targets CreateOrUpdate. + */ public final class TargetsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/CreateUpdateTarget.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/CreateUpdateTarget.json */ /** * Sample code: Create/update a Target that extends a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ public static void createUpdateATargetThatExtendsAVirtualMachineResource( com.azure.resourcemanager.chaos.ChaosManager manager) throws IOException { - manager - .targets() - .createOrUpdateWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-Agent", - new TargetInner() - .withProperties( - mapOf( - "identities", - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "[{\"type\":\"CertificateSubjectIssuer\",\"subject\":\"CN=example.subject\"}]", - Object.class, - SerializerEncoding.JSON))), - com.azure.core.util.Context.NONE); + manager.targets().createOrUpdateWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-Agent", + new TargetInner().withProperties(mapOf("identities", + SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "[{\"type\":\"CertificateSubjectIssuer\",\"subject\":\"CN=example.subject\"}]", Object.class, + SerializerEncoding.JSON))), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteSamples.java index 27de7a2fc06be..ebce9379ebfcf 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteSamples.java @@ -4,26 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Targets Delete. */ +/** + * Samples for Targets Delete. + */ public final class TargetsDeleteSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/DeleteTarget.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/DeleteTarget.json */ /** * Sample code: Delete a Target that extends a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void deleteATargetThatExtendsAVirtualMachineResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .targets() - .deleteWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-Agent", - com.azure.core.util.Context.NONE); + public static void + deleteATargetThatExtendsAVirtualMachineResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.targets().deleteWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-Agent", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsGetSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsGetSamples.java index 1866fd6292961..2a2a0ec5e3f5c 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsGetSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsGetSamples.java @@ -4,26 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Targets Get. */ +/** + * Samples for Targets Get. + */ public final class TargetsGetSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/GetTarget.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/GetTarget.json */ /** * Sample code: Get a Target that extends a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void getATargetThatExtendsAVirtualMachineResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .targets() - .getWithResponse( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - "Microsoft-Agent", - com.azure.core.util.Context.NONE); + public static void + getATargetThatExtendsAVirtualMachineResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.targets().getWithResponse("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", + "Microsoft-Agent", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsListSamples.java b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsListSamples.java index 464c96b1aa65c..0bc677e2566ed 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsListSamples.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/samples/java/com/azure/resourcemanager/chaos/generated/TargetsListSamples.java @@ -4,26 +4,22 @@ package com.azure.resourcemanager.chaos.generated; -/** Samples for Targets List. */ +/** + * Samples for Targets List. + */ public final class TargetsListSamples { /* - * x-ms-original-file: specification/chaos/resource-manager/Microsoft.Chaos/stable/2023-11-01/examples/ListTargets.json + * x-ms-original-file: + * specification/chaos/resource-manager/Microsoft.Chaos/stable/2024-01-01/examples/ListTargets.json */ /** * Sample code: List all Targets that extend a virtual machine resource. - * + * * @param manager Entry point to ChaosManager. */ - public static void listAllTargetsThatExtendAVirtualMachineResource( - com.azure.resourcemanager.chaos.ChaosManager manager) { - manager - .targets() - .list( - "exampleRG", - "Microsoft.Compute", - "virtualMachines", - "exampleVM", - null, - com.azure.core.util.Context.NONE); + public static void + listAllTargetsThatExtendAVirtualMachineResource(com.azure.resourcemanager.chaos.ChaosManager manager) { + manager.targets().list("exampleRG", "Microsoft.Compute", "virtualMachines", "exampleVM", null, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateWithResponseMockTests.java index 85b971cbe5d77..7329296999f0a 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesCreateOrUpdateWithResponseMockTests.java @@ -30,47 +30,26 @@ public void testCreateOrUpdateWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"publisher\":\"fxzsjabibsyst\",\"targetType\":\"fsdjpvkvp\",\"description\":\"xbkzbzkdvncj\",\"parametersSchema\":\"udurgkakmokz\",\"urn\":\"jk\"},\"id\":\"ffhmouwqlgzr\",\"name\":\"zeeyebi\",\"type\":\"ikayuhqlbjbsybb\"}"; + String responseStr + = "{\"properties\":{\"publisher\":\"kpzi\",\"targetType\":\"j\",\"description\":\"nlfzxiavrmbz\",\"parametersSchema\":\"okixrjqcir\",\"urn\":\"pfrlazsz\"},\"id\":\"nwoiind\",\"name\":\"pwp\",\"type\":\"ylwbtlhflsjcdhsz\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Capability response = manager.capabilities().createOrUpdateWithResponse("sjabibs", "stawfsdjpvkv", "bjxbkzbzk", + "vncjabudurgk", "kmokz", "jjklff", new CapabilityInner(), com.azure.core.util.Context.NONE).getValue(); - Capability response = - manager - .capabilities() - .createOrUpdateWithResponse( - "h", - "hfwpracstwit", - "khevxccedc", - "nmdyodnwzxl", - "jc", - "nhltiugcxn", - new CapabilityInner(), - com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteWithResponseMockTests.java index adee4ba2841ac..7bad4cd3df7d5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesDeleteWithResponseMockTests.java @@ -32,32 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .capabilities() - .deleteWithResponse("lolp", "vk", "r", "qvujzraehtwdwrf", "swibyr", "dl", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.capabilities().deleteWithResponse("vfvpdbodaciz", "j", "lhkrribdeibqipqk", "hvxndzwmkrefajpj", "rwkq", + "yhgbijtjivfx", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetWithResponseMockTests.java index 780efbe470cce..2e02342c21b5b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesGetWithResponseMockTests.java @@ -29,46 +29,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"publisher\":\"mhairsbrgzdwmsw\",\"targetType\":\"pqwd\",\"description\":\"gicccnxqhuex\",\"parametersSchema\":\"ttlstvlzywemhz\",\"urn\":\"csdtclusiypbs\"},\"id\":\"gytguslfead\",\"name\":\"ygqukyhejh\",\"type\":\"isxgfp\"}"; + String responseStr + = "{\"properties\":{\"publisher\":\"fwpracstwi\",\"targetType\":\"khevxccedc\",\"description\":\"md\",\"parametersSchema\":\"dnwzxltjcvnhltiu\",\"urn\":\"xnavvwxq\"},\"id\":\"byqunyow\",\"name\":\"wlmdjrkv\",\"type\":\"g\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Capability response = manager.capabilities() + .getWithResponse("lolp", "vk", "r", "qvujzraehtwdwrf", "swibyr", "dl", com.azure.core.util.Context.NONE) + .getValue(); - Capability response = - manager - .capabilities() - .getWithResponse( - "rmaequ", - "ah", - "icslfaoq", - "piyylhalnswhccsp", - "kaivwit", - "scywuggwoluhc", - com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListMockTests.java index e696394b2d835..8d4b60f080e83 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilitiesListMockTests.java @@ -30,45 +30,26 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"publisher\":\"ttwvogvbbe\",\"targetType\":\"cngqqmoakufgmjz\",\"description\":\"rdgrtw\",\"parametersSchema\":\"nuuzkopbm\",\"urn\":\"rfdwoyu\"},\"id\":\"hziuiefozbhdms\",\"name\":\"l\",\"type\":\"zqhof\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"publisher\":\"ggicccnxqhue\",\"targetType\":\"ktt\",\"description\":\"tvlz\",\"parametersSchema\":\"emhzrncsdtc\",\"urn\":\"siypbs\"},\"id\":\"gytguslfead\",\"name\":\"ygqukyhejh\",\"type\":\"isxgfp\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.capabilities().list("hxicslfaoqz", "iyylhalnswhccsp", "kaivwit", + "scywuggwoluhc", "bwemhairs", "rgzdwmsweyp", com.azure.core.util.Context.NONE); - PagedIterable response = - manager - .capabilities() - .list( - "uximerqfobw", - "znkbykutwpfhpagm", - "r", - "kdsnfdsdoakgtdl", - "kkze", - "dlhewp", - com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityInnerTests.java index 46b73882dea59..70b8bc2b67fb9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityInnerTests.java @@ -10,11 +10,9 @@ public final class CapabilityInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityInner model = - BinaryData - .fromString( - "{\"properties\":{\"publisher\":\"mparcryuanzw\",\"targetType\":\"zdxtayrlhmwhf\",\"description\":\"rqobmtuk\",\"parametersSchema\":\"ryrtihfxtijbpzv\",\"urn\":\"wzsymglzufcy\"},\"id\":\"kohdbiha\",\"name\":\"ufhfcbjysa\",\"type\":\"ithxqhabifpi\"}") - .toObject(CapabilityInner.class); + CapabilityInner model = BinaryData.fromString( + "{\"properties\":{\"publisher\":\"mparcryuanzw\",\"targetType\":\"zdxtayrlhmwhf\",\"description\":\"rqobmtuk\",\"parametersSchema\":\"ryrtihfxtijbpzv\",\"urn\":\"wzsymglzufcy\"},\"id\":\"kohdbiha\",\"name\":\"ufhfcbjysa\",\"type\":\"ithxqhabifpi\"}") + .toObject(CapabilityInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityListResultTests.java index c81a42399e57d..67964f884b7e5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityListResultTests.java @@ -10,11 +10,9 @@ public final class CapabilityListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"publisher\":\"vgjxpybczm\",\"targetType\":\"mtz\",\"description\":\"bsphrupidgsybbe\",\"parametersSchema\":\"ph\",\"urn\":\"cmsxaobhdxbm\"},\"id\":\"qioqjzehtbmu\",\"name\":\"p\",\"type\":\"wnoi\"},{\"properties\":{\"publisher\":\"rxybqsoq\",\"targetType\":\"gkdmb\",\"description\":\"zlobcufpd\",\"parametersSchema\":\"rbt\",\"urn\":\"qjnqglhqgnufoooj\"},\"id\":\"wifsq\",\"name\":\"saagdf\",\"type\":\"glzlhjxrifkwmrv\"},{\"properties\":{\"publisher\":\"zntocipaouajps\",\"targetType\":\"cmpoyfdkfogkny\",\"description\":\"ofjdde\",\"parametersSchema\":\"rd\",\"urn\":\"pewnw\"},\"id\":\"eitjz\",\"name\":\"flusarhmof\",\"type\":\"qhsmyurkdtml\"}],\"nextLink\":\"ekuksjtx\"}") - .toObject(CapabilityListResult.class); + CapabilityListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"publisher\":\"vgjxpybczm\",\"targetType\":\"mtz\",\"description\":\"bsphrupidgsybbe\",\"parametersSchema\":\"ph\",\"urn\":\"cmsxaobhdxbm\"},\"id\":\"qioqjzehtbmu\",\"name\":\"p\",\"type\":\"wnoi\"},{\"properties\":{\"publisher\":\"rxybqsoq\",\"targetType\":\"gkdmb\",\"description\":\"zlobcufpd\",\"parametersSchema\":\"rbt\",\"urn\":\"qjnqglhqgnufoooj\"},\"id\":\"wifsq\",\"name\":\"saagdf\",\"type\":\"glzlhjxrifkwmrv\"},{\"properties\":{\"publisher\":\"zntocipaouajps\",\"targetType\":\"cmpoyfdkfogkny\",\"description\":\"ofjdde\",\"parametersSchema\":\"rd\",\"urn\":\"pewnw\"},\"id\":\"eitjz\",\"name\":\"flusarhmof\",\"type\":\"qhsmyurkdtml\"}],\"nextLink\":\"ekuksjtx\"}") + .toObject(CapabilityListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityPropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityPropertiesTests.java index a40c0f86e689a..0fcdc6236f37c 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityPropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityPropertiesTests.java @@ -10,11 +10,9 @@ public final class CapabilityPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityProperties model = - BinaryData - .fromString( - "{\"publisher\":\"wczbys\",\"targetType\":\"pqxu\",\"description\":\"vyq\",\"parametersSchema\":\"wby\",\"urn\":\"k\"}") - .toObject(CapabilityProperties.class); + CapabilityProperties model = BinaryData.fromString( + "{\"publisher\":\"wczbys\",\"targetType\":\"pqxu\",\"description\":\"vyq\",\"parametersSchema\":\"wby\",\"urn\":\"k\"}") + .toObject(CapabilityProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeInnerTests.java index 35b9529de72b4..86ef79b9b2544 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeInnerTests.java @@ -11,11 +11,9 @@ public final class CapabilityTypeInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityTypeInner model = - BinaryData - .fromString( - "{\"location\":\"huticndvkao\",\"properties\":{\"publisher\":\"i\",\"targetType\":\"yhxhu\",\"displayName\":\"k\",\"description\":\"yxolniwp\",\"parametersSchema\":\"ukjfkgiawxklr\",\"urn\":\"lwckbasyypnddhs\",\"kind\":\"bacphejko\",\"azureRbacActions\":[\"qgoulznd\",\"i\",\"wyqkgfgibm\"],\"azureRbacDataActions\":[\"akeqs\",\"xybz\"],\"runtimeProperties\":{\"kind\":\"dqytbciqfouflmm\"}},\"id\":\"kzsmodm\",\"name\":\"lougpbkw\",\"type\":\"mutduqktaps\"}") - .toObject(CapabilityTypeInner.class); + CapabilityTypeInner model = BinaryData.fromString( + "{\"location\":\"huticndvkao\",\"properties\":{\"publisher\":\"i\",\"targetType\":\"yhxhu\",\"displayName\":\"k\",\"description\":\"yxolniwp\",\"parametersSchema\":\"ukjfkgiawxklr\",\"urn\":\"lwckbasyypnddhs\",\"kind\":\"bacphejko\",\"azureRbacActions\":[\"qgoulznd\",\"i\",\"wyqkgfgibm\"],\"azureRbacDataActions\":[\"akeqs\",\"xybz\"],\"runtimeProperties\":{\"kind\":\"dqytbciqfouflmm\"}},\"id\":\"kzsmodm\",\"name\":\"lougpbkw\",\"type\":\"mutduqktaps\"}") + .toObject(CapabilityTypeInner.class); Assertions.assertEquals("huticndvkao", model.location()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeListResultTests.java index e5e3f3b7faf8c..f217bc572f4c5 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypeListResultTests.java @@ -10,11 +10,9 @@ public final class CapabilityTypeListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityTypeListResult model = - BinaryData - .fromString( - "{\"value\":[{\"location\":\"jgrtfwvukxga\",\"properties\":{\"publisher\":\"cs\",\"targetType\":\"s\",\"displayName\":\"nyejhkryhtnap\",\"description\":\"wlokjyem\",\"parametersSchema\":\"vnipjox\",\"urn\":\"nchgej\",\"kind\":\"odmailzyd\",\"azureRbacActions\":[\"jwyahuxinpmqnja\"],\"azureRbacDataActions\":[\"xj\",\"prozvcputegjvwmf\",\"atscmd\",\"pjhulsuuvmkj\"],\"runtimeProperties\":{\"kind\":\"rwfndiod\"}},\"id\":\"pslwejdpvw\",\"name\":\"yoqpsoaccta\",\"type\":\"akl\"},{\"location\":\"hbcryffdfdosyge\",\"properties\":{\"publisher\":\"ojakhmsbzjhcrze\",\"targetType\":\"phlxa\",\"displayName\":\"thqt\",\"description\":\"qjbpfzfsin\",\"parametersSchema\":\"v\",\"urn\":\"jrwzox\",\"kind\":\"tfell\",\"azureRbacActions\":[\"zitonpeqfpjkjl\",\"ofpdvh\"],\"azureRbacDataActions\":[\"xypininmayhuybbk\",\"odepoogin\",\"vamih\"],\"runtimeProperties\":{\"kind\":\"narxzxtheotus\"}},\"id\":\"vyevcciqi\",\"name\":\"nhungbw\",\"type\":\"zrnf\"},{\"location\":\"gispemvtzfkufubl\",\"properties\":{\"publisher\":\"xqeofjaeqjhqjba\",\"targetType\":\"msmjqulngsntn\",\"displayName\":\"bkzgcwrwclx\",\"description\":\"rljdouskcqv\",\"parametersSchema\":\"cr\",\"urn\":\"dkwt\",\"kind\":\"xbnjbiksq\",\"azureRbacActions\":[\"ssainqpjwnzll\",\"fmppe\"],\"azureRbacDataActions\":[\"mgxsab\",\"yqduujit\"],\"runtimeProperties\":{\"kind\":\"zdzevndh\"}},\"id\":\"rwpdappdsbdkvwrw\",\"name\":\"feusnhut\",\"type\":\"eltmrldhugjzzdat\"}],\"nextLink\":\"hocdgeab\"}") - .toObject(CapabilityTypeListResult.class); + CapabilityTypeListResult model = BinaryData.fromString( + "{\"value\":[{\"location\":\"jgrtfwvukxga\",\"properties\":{\"publisher\":\"cs\",\"targetType\":\"s\",\"displayName\":\"nyejhkryhtnap\",\"description\":\"wlokjyem\",\"parametersSchema\":\"vnipjox\",\"urn\":\"nchgej\",\"kind\":\"odmailzyd\",\"azureRbacActions\":[\"jwyahuxinpmqnja\"],\"azureRbacDataActions\":[\"xj\",\"prozvcputegjvwmf\",\"atscmd\",\"pjhulsuuvmkj\"],\"runtimeProperties\":{\"kind\":\"rwfndiod\"}},\"id\":\"pslwejdpvw\",\"name\":\"yoqpsoaccta\",\"type\":\"akl\"},{\"location\":\"hbcryffdfdosyge\",\"properties\":{\"publisher\":\"ojakhmsbzjhcrze\",\"targetType\":\"phlxa\",\"displayName\":\"thqt\",\"description\":\"qjbpfzfsin\",\"parametersSchema\":\"v\",\"urn\":\"jrwzox\",\"kind\":\"tfell\",\"azureRbacActions\":[\"zitonpeqfpjkjl\",\"ofpdvh\"],\"azureRbacDataActions\":[\"xypininmayhuybbk\",\"odepoogin\",\"vamih\"],\"runtimeProperties\":{\"kind\":\"narxzxtheotus\"}},\"id\":\"vyevcciqi\",\"name\":\"nhungbw\",\"type\":\"zrnf\"},{\"location\":\"gispemvtzfkufubl\",\"properties\":{\"publisher\":\"xqeofjaeqjhqjba\",\"targetType\":\"msmjqulngsntn\",\"displayName\":\"bkzgcwrwclx\",\"description\":\"rljdouskcqv\",\"parametersSchema\":\"cr\",\"urn\":\"dkwt\",\"kind\":\"xbnjbiksq\",\"azureRbacActions\":[\"ssainqpjwnzll\",\"fmppe\"],\"azureRbacDataActions\":[\"mgxsab\",\"yqduujit\"],\"runtimeProperties\":{\"kind\":\"zdzevndh\"}},\"id\":\"rwpdappdsbdkvwrw\",\"name\":\"feusnhut\",\"type\":\"eltmrldhugjzzdat\"}],\"nextLink\":\"hocdgeab\"}") + .toObject(CapabilityTypeListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesRuntimePropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesRuntimePropertiesTests.java index eea6de5793390..4d7aa0103811e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesRuntimePropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesRuntimePropertiesTests.java @@ -10,10 +10,8 @@ public final class CapabilityTypePropertiesRuntimePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityTypePropertiesRuntimeProperties model = - BinaryData - .fromString("{\"kind\":\"uhmuouqfprwzwbn\"}") - .toObject(CapabilityTypePropertiesRuntimeProperties.class); + CapabilityTypePropertiesRuntimeProperties model = BinaryData.fromString("{\"kind\":\"uhmuouqfprwzwbn\"}") + .toObject(CapabilityTypePropertiesRuntimeProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesTests.java index b71a9ea9af7f4..5d7b16915c7b2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypePropertiesTests.java @@ -13,20 +13,17 @@ public final class CapabilityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CapabilityTypeProperties model = - BinaryData - .fromString( - "{\"publisher\":\"gcue\",\"targetType\":\"umkdosvqwhbmd\",\"displayName\":\"bjf\",\"description\":\"gmbmbexppbh\",\"parametersSchema\":\"qrolfpf\",\"urn\":\"algbquxigjyjg\",\"kind\":\"aoyfhrtxilnerkuj\",\"azureRbacActions\":[\"l\",\"juvf\",\"awrlyx\"],\"azureRbacDataActions\":[\"cpr\",\"nwbxgjvtbvpyssz\"],\"runtimeProperties\":{\"kind\":\"uj\"}}") - .toObject(CapabilityTypeProperties.class); + CapabilityTypeProperties model = BinaryData.fromString( + "{\"publisher\":\"gcue\",\"targetType\":\"umkdosvqwhbmd\",\"displayName\":\"bjf\",\"description\":\"gmbmbexppbh\",\"parametersSchema\":\"qrolfpf\",\"urn\":\"algbquxigjyjg\",\"kind\":\"aoyfhrtxilnerkuj\",\"azureRbacActions\":[\"l\",\"juvf\",\"awrlyx\"],\"azureRbacDataActions\":[\"cpr\",\"nwbxgjvtbvpyssz\"],\"runtimeProperties\":{\"kind\":\"uj\"}}") + .toObject(CapabilityTypeProperties.class); Assertions.assertEquals("l", model.azureRbacActions().get(0)); Assertions.assertEquals("cpr", model.azureRbacDataActions().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CapabilityTypeProperties model = - new CapabilityTypeProperties() - .withAzureRbacActions(Arrays.asList("l", "juvf", "awrlyx")) + CapabilityTypeProperties model + = new CapabilityTypeProperties().withAzureRbacActions(Arrays.asList("l", "juvf", "awrlyx")) .withAzureRbacDataActions(Arrays.asList("cpr", "nwbxgjvtbvpyssz")) .withRuntimeProperties(new CapabilityTypePropertiesRuntimeProperties()); model = BinaryData.fromObject(model).toObject(CapabilityTypeProperties.class); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetWithResponseMockTests.java index a6097005c5ab5..d5e7a7184b750 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesGetWithResponseMockTests.java @@ -30,41 +30,28 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"location\":\"hftqsxhqxujxukn\",\"properties\":{\"publisher\":\"igrjguufzdm\",\"targetType\":\"qtfihwhbotzinga\",\"displayName\":\"pph\",\"description\":\"zqzudph\",\"parametersSchema\":\"mvdk\",\"urn\":\"ynwcvtbv\",\"kind\":\"yhmtnvyqiat\",\"azureRbacActions\":[\"pcnp\"],\"azureRbacDataActions\":[\"jaesgvvsccya\",\"g\",\"qfhwyg\"],\"runtimeProperties\":{\"kind\":\"dnkfx\"}},\"id\":\"semdwzrmu\",\"name\":\"apfcqdpsq\",\"type\":\"qvpsvuoymg\"}"; + String responseStr + = "{\"location\":\"csnjvcdwxlpqekft\",\"properties\":{\"publisher\":\"tjsyin\",\"targetType\":\"fq\",\"displayName\":\"mtdh\",\"description\":\"dvypgikdgsz\",\"parametersSchema\":\"kbir\",\"urn\":\"uzhlhkjoqrv\",\"kind\":\"aatjinrvgoupmfi\",\"azureRbacActions\":[\"ggjioolvr\",\"x\",\"v\"],\"azureRbacDataActions\":[\"gllqwjy\"],\"runtimeProperties\":{\"kind\":\"ayvblmhvkzuhbx\"}},\"id\":\"vyhgs\",\"name\":\"pbyrqufegxu\",\"type\":\"wz\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - CapabilityType response = - manager - .capabilityTypes() - .getWithResponse("hfkvtvsexsowuel", "qhhahhxvrhmzkwpj", "wws", com.azure.core.util.Context.NONE) - .getValue(); + CapabilityType response = manager.capabilityTypes() + .getWithResponse("vezrypqlmfeo", "erqwkyhkobopg", "edkowepbqpcrfk", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("hftqsxhqxujxukn", response.location()); + Assertions.assertEquals("csnjvcdwxlpqekft", response.location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListMockTests.java index 46f910742c573..44f7ccf5e7059 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/CapabilityTypesListMockTests.java @@ -31,38 +31,27 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"location\":\"smwutwbdsrezpd\",\"properties\":{\"publisher\":\"euyowqkd\",\"targetType\":\"t\",\"displayName\":\"ib\",\"description\":\"cgpik\",\"parametersSchema\":\"imejzanl\",\"urn\":\"xi\",\"kind\":\"rmbzo\",\"azureRbacActions\":[\"i\",\"rjqc\"],\"azureRbacDataActions\":[\"zpfrla\",\"szrnwo\"],\"runtimeProperties\":{\"kind\":\"dfpwpjylwbtlhfls\"}},\"id\":\"cdhszf\",\"name\":\"vfbgofeljagrqmqh\",\"type\":\"dvriiiojnal\"}]}"; + String responseStr + = "{\"value\":[{\"location\":\"sexso\",\"properties\":{\"publisher\":\"luqhhahhxv\",\"targetType\":\"mzkwpjg\",\"displayName\":\"spughftqsxhq\",\"description\":\"j\",\"parametersSchema\":\"kndxdigrjgu\",\"urn\":\"zdmsyqtfi\",\"kind\":\"hbotzingamvppho\",\"azureRbacActions\":[\"zudphqamvdkfw\",\"nwcvtbvkayhmtnv\"],\"azureRbacDataActions\":[\"atkzwpcnpw\",\"cjaesgvvs\",\"cyajguqf\"],\"runtimeProperties\":{\"kind\":\"gzlvdnkfxu\"}},\"id\":\"emdwzrmuhapfc\",\"name\":\"dpsqx\",\"type\":\"vpsvuoymgcce\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.capabilityTypes().list("wrv", "ldgmfpgvmpip", "slthaq", com.azure.core.util.Context.NONE); + PagedIterable response = manager.capabilityTypes().list("jvfbgofelja", "rqmq", + "ldvriiiojnalghfk", com.azure.core.util.Context.NONE); - Assertions.assertEquals("smwutwbdsrezpd", response.iterator().next().location()); + Assertions.assertEquals("sexso", response.iterator().next().location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentActionTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentActionTests.java index f9a3e0c713c12..b68fefea7b58d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentActionTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentActionTests.java @@ -11,9 +11,8 @@ public final class ChaosExperimentActionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosExperimentAction model = - BinaryData - .fromString("{\"type\":\"ChaosExperimentAction\",\"name\":\"nrjawgqwg\"}") + ChaosExperimentAction model + = BinaryData.fromString("{\"type\":\"ChaosExperimentAction\",\"name\":\"nrjawgqwg\"}") .toObject(ChaosExperimentAction.class); Assertions.assertEquals("nrjawgqwg", model.name()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentBranchTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentBranchTests.java index d45e9d95d3fa7..5cfee040ba421 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentBranchTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentBranchTests.java @@ -13,27 +13,19 @@ public final class ChaosExperimentBranchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosExperimentBranch model = - BinaryData - .fromString( - "{\"name\":\"ovbvmeueciv\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"zceuojgjrw\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"ueiotwmcdyt\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"x\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"it\"}]}") - .toObject(ChaosExperimentBranch.class); + ChaosExperimentBranch model = BinaryData.fromString( + "{\"name\":\"ovbvmeueciv\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"zceuojgjrw\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"ueiotwmcdyt\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"x\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"it\"}]}") + .toObject(ChaosExperimentBranch.class); Assertions.assertEquals("ovbvmeueciv", model.name()); Assertions.assertEquals("zceuojgjrw", model.actions().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosExperimentBranch model = - new ChaosExperimentBranch() - .withName("ovbvmeueciv") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("zceuojgjrw"), - new ChaosExperimentAction().withName("ueiotwmcdyt"), - new ChaosExperimentAction().withName("x"), - new ChaosExperimentAction().withName("it"))); + ChaosExperimentBranch model = new ChaosExperimentBranch().withName("ovbvmeueciv") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("zceuojgjrw"), + new ChaosExperimentAction().withName("ueiotwmcdyt"), new ChaosExperimentAction().withName("x"), + new ChaosExperimentAction().withName("it"))); model = BinaryData.fromObject(model).toObject(ChaosExperimentBranch.class); Assertions.assertEquals("ovbvmeueciv", model.name()); Assertions.assertEquals("zceuojgjrw", model.actions().get(0).name()); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentStepTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentStepTests.java index fad7bd7628330..d90372a6000fb 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentStepTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosExperimentStepTests.java @@ -14,11 +14,9 @@ public final class ChaosExperimentStepTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosExperimentStep model = - BinaryData - .fromString( - "{\"name\":\"bnbbeldawkz\",\"branches\":[{\"name\":\"liourqhak\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"hashsfwxosow\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"xcug\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"cjooxdjebwpucwwf\"}]}]}") - .toObject(ChaosExperimentStep.class); + ChaosExperimentStep model = BinaryData.fromString( + "{\"name\":\"bnbbeldawkz\",\"branches\":[{\"name\":\"liourqhak\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"hashsfwxosow\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"xcug\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"cjooxdjebwpucwwf\"}]}]}") + .toObject(ChaosExperimentStep.class); Assertions.assertEquals("bnbbeldawkz", model.name()); Assertions.assertEquals("liourqhak", model.branches().get(0).name()); Assertions.assertEquals("hashsfwxosow", model.branches().get(0).actions().get(0).name()); @@ -26,20 +24,11 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosExperimentStep model = - new ChaosExperimentStep() - .withName("bnbbeldawkz") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("liourqhak") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("hashsfwxosow"), - new ChaosExperimentAction().withName("xcug"), - new ChaosExperimentAction().withName("cjooxdjebwpucwwf"))))); + ChaosExperimentStep model = new ChaosExperimentStep().withName("bnbbeldawkz") + .withBranches(Arrays.asList(new ChaosExperimentBranch().withName("liourqhak") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("hashsfwxosow"), + new ChaosExperimentAction().withName("xcug"), + new ChaosExperimentAction().withName("cjooxdjebwpucwwf"))))); model = BinaryData.fromObject(model).toObject(ChaosExperimentStep.class); Assertions.assertEquals("bnbbeldawkz", model.name()); Assertions.assertEquals("liourqhak", model.branches().get(0).name()); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetFilterTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetFilterTests.java index 8a16b9eaa2be7..38099d620262d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetFilterTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetFilterTests.java @@ -10,8 +10,8 @@ public final class ChaosTargetFilterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosTargetFilter model = - BinaryData.fromString("{\"type\":\"ChaosTargetFilter\"}").toObject(ChaosTargetFilter.class); + ChaosTargetFilter model + = BinaryData.fromString("{\"type\":\"ChaosTargetFilter\"}").toObject(ChaosTargetFilter.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetListSelectorTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetListSelectorTests.java index afb7463345bfc..586faac4cc5f1 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetListSelectorTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetListSelectorTests.java @@ -15,29 +15,25 @@ public final class ChaosTargetListSelectorTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosTargetListSelector model = - BinaryData - .fromString( - "{\"type\":\"List\",\"targets\":[{\"type\":\"ChaosTarget\",\"id\":\"grauwjuetaebur\"}],\"id\":\"vdmovsmzlxwabm\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"vtpuqujmqlgk\":\"datakif\",\"ongbjcnt\":\"databtndo\"}}") - .toObject(ChaosTargetListSelector.class); - Assertions.assertEquals("vdmovsmzlxwabm", model.id()); + ChaosTargetListSelector model = BinaryData.fromString( + "{\"type\":\"List\",\"targets\":[{\"type\":\"ChaosTarget\",\"id\":\"puqujmqlgkfbtn\"},{\"type\":\"ChaosTarget\",\"id\":\"aongbj\"},{\"type\":\"ChaosTarget\",\"id\":\"tujitcjedft\"}],\"id\":\"waezkojvd\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"oxciqopidoamcio\":\"dataoqouicybxarzgszu\",\"zxkhnzbonlwnto\":\"datahkh\",\"zcmrvexztvb\":\"datagokdwbwhks\"}}") + .toObject(ChaosTargetListSelector.class); + Assertions.assertEquals("waezkojvd", model.id()); Assertions.assertEquals(TargetReferenceType.CHAOS_TARGET, model.targets().get(0).type()); - Assertions.assertEquals("grauwjuetaebur", model.targets().get(0).id()); + Assertions.assertEquals("puqujmqlgkfbtn", model.targets().get(0).id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosTargetListSelector model = - new ChaosTargetListSelector() - .withId("vdmovsmzlxwabm") - .withFilter(new ChaosTargetFilter()) - .withTargets( - Arrays - .asList( - new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId("grauwjuetaebur"))); + ChaosTargetListSelector model + = new ChaosTargetListSelector().withId("waezkojvd").withFilter(new ChaosTargetFilter()) + .withTargets(Arrays.asList( + new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId("puqujmqlgkfbtn"), + new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId("aongbj"), + new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId("tujitcjedft"))); model = BinaryData.fromObject(model).toObject(ChaosTargetListSelector.class); - Assertions.assertEquals("vdmovsmzlxwabm", model.id()); + Assertions.assertEquals("waezkojvd", model.id()); Assertions.assertEquals(TargetReferenceType.CHAOS_TARGET, model.targets().get(0).type()); - Assertions.assertEquals("grauwjuetaebur", model.targets().get(0).id()); + Assertions.assertEquals("puqujmqlgkfbtn", model.targets().get(0).id()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetQuerySelectorTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetQuerySelectorTests.java index ba178421b2269..f16e0d69e92f3 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetQuerySelectorTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetQuerySelectorTests.java @@ -13,27 +13,22 @@ public final class ChaosTargetQuerySelectorTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosTargetQuerySelector model = - BinaryData - .fromString( - "{\"type\":\"Query\",\"queryString\":\"c\",\"subscriptionIds\":[\"df\"],\"id\":\"wwa\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"bxarzgszufoxci\":\"datajvdcpzfoqouic\",\"haz\":\"dataopidoamciodh\",\"toego\":\"datakhnzbonlw\"}}") - .toObject(ChaosTargetQuerySelector.class); - Assertions.assertEquals("wwa", model.id()); - Assertions.assertEquals("c", model.queryString()); - Assertions.assertEquals("df", model.subscriptionIds().get(0)); + ChaosTargetQuerySelector model = BinaryData.fromString( + "{\"type\":\"Query\",\"queryString\":\"lmnguxaw\",\"subscriptionIds\":[\"ldsyuuximerqfob\",\"yznkby\"],\"id\":\"utwpfhp\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"kdsnfdsdoakgtdl\":\"datar\",\"dlhewp\":\"datakkze\",\"bbejdcngqqm\":\"datasdsttwvog\"}}") + .toObject(ChaosTargetQuerySelector.class); + Assertions.assertEquals("utwpfhp", model.id()); + Assertions.assertEquals("lmnguxaw", model.queryString()); + Assertions.assertEquals("ldsyuuximerqfob", model.subscriptionIds().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosTargetQuerySelector model = - new ChaosTargetQuerySelector() - .withId("wwa") - .withFilter(new ChaosTargetFilter()) - .withQueryString("c") - .withSubscriptionIds(Arrays.asList("df")); + ChaosTargetQuerySelector model + = new ChaosTargetQuerySelector().withId("utwpfhp").withFilter(new ChaosTargetFilter()) + .withQueryString("lmnguxaw").withSubscriptionIds(Arrays.asList("ldsyuuximerqfob", "yznkby")); model = BinaryData.fromObject(model).toObject(ChaosTargetQuerySelector.class); - Assertions.assertEquals("wwa", model.id()); - Assertions.assertEquals("c", model.queryString()); - Assertions.assertEquals("df", model.subscriptionIds().get(0)); + Assertions.assertEquals("utwpfhp", model.id()); + Assertions.assertEquals("lmnguxaw", model.queryString()); + Assertions.assertEquals("ldsyuuximerqfob", model.subscriptionIds().get(0)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSelectorTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSelectorTests.java index f366ad1f40cd5..b8422912999c6 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSelectorTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSelectorTests.java @@ -14,21 +14,16 @@ public final class ChaosTargetSelectorTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosTargetSelector model = - BinaryData - .fromString( - "{\"type\":\"ChaosTargetSelector\",\"id\":\"hniskxfbkpyc\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"auwhvylwzbtdhx\":\"datandnhj\",\"pow\":\"datajznb\"}}") - .toObject(ChaosTargetSelector.class); + ChaosTargetSelector model = BinaryData.fromString( + "{\"type\":\"ChaosTargetSelector\",\"id\":\"hniskxfbkpyc\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"auwhvylwzbtdhx\":\"datandnhj\",\"pow\":\"datajznb\"}}") + .toObject(ChaosTargetSelector.class); Assertions.assertEquals("hniskxfbkpyc", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosTargetSelector model = - new ChaosTargetSelector() - .withId("hniskxfbkpyc") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")); + ChaosTargetSelector model = new ChaosTargetSelector().withId("hniskxfbkpyc").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")); model = BinaryData.fromObject(model).toObject(ChaosTargetSelector.class); Assertions.assertEquals("hniskxfbkpyc", model.id()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterParametersTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterParametersTests.java index 9312f6aceb551..4b83faaefc242 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterParametersTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterParametersTests.java @@ -12,18 +12,17 @@ public final class ChaosTargetSimpleFilterParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosTargetSimpleFilterParameters model = - BinaryData - .fromString("{\"zones\":[\"sfraoyzko\",\"wtl\",\"nguxawqaldsy\"]}") + ChaosTargetSimpleFilterParameters model + = BinaryData.fromString("{\"zones\":[\"hhziuief\",\"zbhd\",\"smlmzqhoftrm\",\"equi\"]}") .toObject(ChaosTargetSimpleFilterParameters.class); - Assertions.assertEquals("sfraoyzko", model.zones().get(0)); + Assertions.assertEquals("hhziuief", model.zones().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosTargetSimpleFilterParameters model = - new ChaosTargetSimpleFilterParameters().withZones(Arrays.asList("sfraoyzko", "wtl", "nguxawqaldsy")); + ChaosTargetSimpleFilterParameters model = new ChaosTargetSimpleFilterParameters() + .withZones(Arrays.asList("hhziuief", "zbhd", "smlmzqhoftrm", "equi")); model = BinaryData.fromObject(model).toObject(ChaosTargetSimpleFilterParameters.class); - Assertions.assertEquals("sfraoyzko", model.zones().get(0)); + Assertions.assertEquals("hhziuief", model.zones().get(0)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterTests.java index 064067fa02428..2362ff719dd6a 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ChaosTargetSimpleFilterTests.java @@ -13,19 +13,17 @@ public final class ChaosTargetSimpleFilterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChaosTargetSimpleFilter model = - BinaryData - .fromString("{\"type\":\"Simple\",\"parameters\":{\"zones\":[\"whkszzcmrvexztvb\"]}}") - .toObject(ChaosTargetSimpleFilter.class); - Assertions.assertEquals("whkszzcmrvexztvb", model.parameters().zones().get(0)); + ChaosTargetSimpleFilter model = BinaryData + .fromString("{\"type\":\"Simple\",\"parameters\":{\"zones\":[\"fgmjzrwrdgrt\",\"aenuuz\",\"opbminrfdw\"]}}") + .toObject(ChaosTargetSimpleFilter.class); + Assertions.assertEquals("fgmjzrwrdgrt", model.parameters().zones().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ChaosTargetSimpleFilter model = - new ChaosTargetSimpleFilter() - .withParameters(new ChaosTargetSimpleFilterParameters().withZones(Arrays.asList("whkszzcmrvexztvb"))); + ChaosTargetSimpleFilter model = new ChaosTargetSimpleFilter().withParameters( + new ChaosTargetSimpleFilterParameters().withZones(Arrays.asList("fgmjzrwrdgrt", "aenuuz", "opbminrfdw"))); model = BinaryData.fromObject(model).toObject(ChaosTargetSimpleFilter.class); - Assertions.assertEquals("whkszzcmrvexztvb", model.parameters().zones().get(0)); + Assertions.assertEquals("fgmjzrwrdgrt", model.parameters().zones().get(0)); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/DelayActionTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/DelayActionTests.java index 7744e7062441f..f2a274939e8af 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/DelayActionTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/DelayActionTests.java @@ -12,19 +12,17 @@ public final class DelayActionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DelayAction model = - BinaryData - .fromString("{\"type\":\"delay\",\"duration\":\"PT221H14M7S\",\"name\":\"ormrlxqtvcofudfl\"}") - .toObject(DelayAction.class); - Assertions.assertEquals("ormrlxqtvcofudfl", model.name()); - Assertions.assertEquals(Duration.parse("PT221H14M7S"), model.duration()); + DelayAction model = BinaryData.fromString("{\"type\":\"delay\",\"duration\":\"PT185H37M50S\",\"name\":\"l\"}") + .toObject(DelayAction.class); + Assertions.assertEquals("l", model.name()); + Assertions.assertEquals(Duration.parse("PT185H37M50S"), model.duration()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DelayAction model = new DelayAction().withName("ormrlxqtvcofudfl").withDuration(Duration.parse("PT221H14M7S")); + DelayAction model = new DelayAction().withName("l").withDuration(Duration.parse("PT185H37M50S")); model = BinaryData.fromObject(model).toObject(DelayAction.class); - Assertions.assertEquals("ormrlxqtvcofudfl", model.name()); - Assertions.assertEquals(Duration.parse("PT221H14M7S"), model.duration()); + Assertions.assertEquals("l", model.name()); + Assertions.assertEquals(Duration.parse("PT185H37M50S"), model.duration()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsInnerTests.java index 5db8e88ce721e..670c59416742e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsInnerTests.java @@ -10,11 +10,9 @@ public final class ExperimentExecutionDetailsInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentExecutionDetailsInner model = - BinaryData - .fromString( - "{\"type\":\"xznelixhnrztf\",\"id\":\"hb\",\"name\":\"knalaulppg\",\"properties\":{\"failureReason\":\"pnapnyiropuh\",\"lastActionAt\":\"2020-12-20T18:46:56Z\",\"runInformation\":{\"steps\":[{\"stepName\":\"lgqg\",\"stepId\":\"xmedjvcsly\",\"status\":\"wwncwzzhxgk\",\"branches\":[{},{}]},{\"stepName\":\"ucnapkteoellwp\",\"stepId\":\"d\",\"status\":\"pfqbuaceopzf\",\"branches\":[{},{}]},{\"stepName\":\"uaopppcqeq\",\"stepId\":\"lzdahzxctobgbkdm\",\"status\":\"zpostmgrcfbu\",\"branches\":[{},{},{}]},{\"stepName\":\"qjhhkxbpv\",\"stepId\":\"mjh\",\"status\":\"j\",\"branches\":[{},{},{},{}]}]},\"status\":\"divkrt\",\"startedAt\":\"2021-11-23T03:43:54Z\",\"stoppedAt\":\"2021-05-30T10:12:47Z\"}}") - .toObject(ExperimentExecutionDetailsInner.class); + ExperimentExecutionDetailsInner model = BinaryData.fromString( + "{\"type\":\"tfdygpfqb\",\"id\":\"ceopzfqrhhuaopp\",\"name\":\"qeqxo\",\"properties\":{\"failureReason\":\"ahzxctobgbk\",\"lastActionAt\":\"2021-03-02T22:54:24Z\",\"runInformation\":{\"steps\":[{\"stepName\":\"stmgrcfbunrmfqjh\",\"stepId\":\"xbpvjymjhx\",\"status\":\"yngudivk\",\"branches\":[{},{}]},{\"stepName\":\"bxqz\",\"stepId\":\"zjf\",\"status\":\"vjfdx\",\"branches\":[{},{}]},{\"stepName\":\"tvtc\",\"stepId\":\"qtdo\",\"status\":\"cbxvwvxyslqbh\",\"branches\":[{},{},{}]},{\"stepName\":\"blytk\",\"stepId\":\"mpew\",\"status\":\"fbkrvrnsvs\",\"branches\":[{},{}]}]},\"status\":\"hxcr\",\"startedAt\":\"2020-12-28T12:25:09Z\",\"stoppedAt\":\"2021-11-28T02:47:56Z\"}}") + .toObject(ExperimentExecutionDetailsInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesRunInformationTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesRunInformationTests.java index 2b2ea3580c899..f87544138491b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesRunInformationTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesRunInformationTests.java @@ -10,17 +10,15 @@ public final class ExperimentExecutionDetailsPropertiesRunInformationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentExecutionDetailsPropertiesRunInformation model = - BinaryData - .fromString( - "{\"steps\":[{\"stepName\":\"tdss\",\"stepId\":\"wtmwerio\",\"status\":\"pyqs\",\"branches\":[{\"branchName\":\"bnetshh\",\"branchId\":\"h\",\"status\":\"plvwiwubmwmbes\",\"actions\":[{}]}]},{\"stepName\":\"wwtppj\",\"stepId\":\"cxogaokonzm\",\"status\":\"ikvmkqzeqqk\",\"branches\":[{\"branchName\":\"zxmhhvhgu\",\"branchId\":\"odkwobd\",\"status\":\"xtibqdxbxwakbog\",\"actions\":[{},{},{}]}]},{\"stepName\":\"lkzgxhuriplbp\",\"stepId\":\"xunkbebxmubyynt\",\"status\":\"rbqtkoie\",\"branches\":[{\"branchName\":\"tgqr\",\"branchId\":\"tmuwlauwzi\",\"status\":\"bm\",\"actions\":[{},{}]}]}]}") - .toObject(ExperimentExecutionDetailsPropertiesRunInformation.class); + ExperimentExecutionDetailsPropertiesRunInformation model = BinaryData.fromString( + "{\"steps\":[{\"stepName\":\"efuzmuvpbttd\",\"stepId\":\"orppxebmnzbtb\",\"status\":\"pglkf\",\"branches\":[{\"branchName\":\"neuelfphsdyhtoz\",\"branchId\":\"kd\",\"status\":\"wq\",\"actions\":[{}]},{\"branchName\":\"zx\",\"branchId\":\"vithh\",\"status\":\"o\",\"actions\":[{}]},{\"branchName\":\"gbhcohfwdsj\",\"branchId\":\"aljutiiswac\",\"status\":\"gdkz\",\"actions\":[{},{},{},{}]}]},{\"stepName\":\"fvhqc\",\"stepId\":\"ilvpnppfuflrwd\",\"status\":\"dlxyjrxs\",\"branches\":[{\"branchName\":\"cnihgwqapnedgfbc\",\"branchId\":\"cvqvpkeqdcv\",\"status\":\"hvoodsotbobzd\",\"actions\":[{},{}]},{\"branchName\":\"wvnhdldwmgx\",\"branchId\":\"rslpmutwuoeg\",\"status\":\"khjwn\",\"actions\":[{},{}]}]}]}") + .toObject(ExperimentExecutionDetailsPropertiesRunInformation.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExperimentExecutionDetailsPropertiesRunInformation model = - new ExperimentExecutionDetailsPropertiesRunInformation(); + ExperimentExecutionDetailsPropertiesRunInformation model + = new ExperimentExecutionDetailsPropertiesRunInformation(); model = BinaryData.fromObject(model).toObject(ExperimentExecutionDetailsPropertiesRunInformation.class); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesTests.java index 0aad76abe6273..a4e513fabb733 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionDetailsPropertiesTests.java @@ -10,11 +10,9 @@ public final class ExperimentExecutionDetailsPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentExecutionDetailsProperties model = - BinaryData - .fromString( - "{\"failureReason\":\"vszjfauvjfdxxi\",\"lastActionAt\":\"2021-01-09T05:59Z\",\"runInformation\":{\"steps\":[{\"stepName\":\"aqtdoqmcbx\",\"stepId\":\"vxysl\",\"status\":\"hsfxoblytkb\",\"branches\":[{\"branchName\":\"wwwfbkr\",\"branchId\":\"nsvs\",\"status\":\"johxcrsb\",\"actions\":[{},{},{}]}]},{\"stepName\":\"srruvwbhsqfsubcg\",\"stepId\":\"irx\",\"status\":\"ybsrfbjfdtwss\",\"branches\":[{\"branchName\":\"pvjzbe\",\"branchId\":\"l\",\"status\":\"nfqqnvwp\",\"actions\":[{},{},{}]},{\"branchName\":\"ruoujmk\",\"branchId\":\"hwqytj\",\"status\":\"bnw\",\"actions\":[{},{},{},{}]},{\"branchName\":\"drjervnaenqpehin\",\"branchId\":\"ygmi\",\"status\":\"hnzdndslgnayqi\",\"actions\":[{},{},{},{}]}]},{\"stepName\":\"uhavhql\",\"stepId\":\"humaqolbgyc\",\"status\":\"iertgccymvaolp\",\"branches\":[{\"branchName\":\"lfmmdnbbglzpswi\",\"branchId\":\"mcwyhzdxssadb\",\"status\":\"nvdfznuda\",\"actions\":[{},{}]},{\"branchName\":\"zbn\",\"branchId\":\"lylpstdb\",\"status\":\"xsrz\",\"actions\":[{},{},{},{}]}]}]},\"status\":\"erscdntne\",\"startedAt\":\"2021-04-19T04:18:41Z\",\"stoppedAt\":\"2021-05-10T05:26:40Z\"}") - .toObject(ExperimentExecutionDetailsProperties.class); + ExperimentExecutionDetailsProperties model = BinaryData.fromString( + "{\"failureReason\":\"srruvwbhsqfsubcg\",\"lastActionAt\":\"2021-02-04T22:06:27Z\",\"runInformation\":{\"steps\":[{\"stepName\":\"ybsrfbjfdtwss\",\"stepId\":\"ftpvjzbexil\",\"status\":\"nfqqnvwp\",\"branches\":[{\"branchName\":\"ruoujmk\",\"branchId\":\"hwqytj\",\"status\":\"bnw\",\"actions\":[{},{},{},{}]},{\"branchName\":\"drjervnaenqpehin\",\"branchId\":\"ygmi\",\"status\":\"hnzdndslgnayqi\",\"actions\":[{},{},{},{}]},{\"branchName\":\"uhavhql\",\"branchId\":\"humaqolbgyc\",\"status\":\"iertgccymvaolp\",\"actions\":[{},{}]}]},{\"stepName\":\"lfmmdnbbglzpswi\",\"stepId\":\"mcwyhzdxssadb\",\"status\":\"nvdfznuda\",\"branches\":[{\"branchName\":\"zbn\",\"branchId\":\"lylpstdb\",\"status\":\"xsrz\",\"actions\":[{},{},{},{}]},{\"branchName\":\"erscdntne\",\"branchId\":\"iwjmygtdssls\",\"status\":\"mweriofzpy\",\"actions\":[{},{}]}]},{\"stepName\":\"wab\",\"stepId\":\"tshhszhedp\",\"status\":\"wiwubm\",\"branches\":[{\"branchName\":\"sldnkwwtppjflcxo\",\"branchId\":\"okonzmnsikvmkqz\",\"status\":\"qkdltfz\",\"actions\":[{}]},{\"branchName\":\"v\",\"branchId\":\"ur\",\"status\":\"dkwobdagx\",\"actions\":[{},{}]},{\"branchName\":\"dxbx\",\"branchId\":\"kbogqxndlkzgx\",\"status\":\"ripl\",\"actions\":[{},{}]},{\"branchName\":\"xunkbebxmubyynt\",\"branchId\":\"rbqtkoie\",\"status\":\"eotg\",\"actions\":[{}]}]}]},\"status\":\"tmuwlauwzi\",\"startedAt\":\"2021-01-24T11:46:23Z\",\"stoppedAt\":\"2021-03-14T00:48:06Z\"}") + .toObject(ExperimentExecutionDetailsProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionInnerTests.java index 6188858cc8c95..bf0ce793e2861 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionInnerTests.java @@ -10,11 +10,9 @@ public final class ExperimentExecutionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentExecutionInner model = - BinaryData - .fromString( - "{\"type\":\"ttouwaboekqvkel\",\"id\":\"mvb\",\"name\":\"yjsflhhcaalnji\",\"properties\":{\"status\":\"xyawj\",\"startedAt\":\"2021-08-07T05:01:18Z\",\"stoppedAt\":\"2021-10-05T03:46:28Z\"}}") - .toObject(ExperimentExecutionInner.class); + ExperimentExecutionInner model = BinaryData.fromString( + "{\"type\":\"tpnapnyiropuhpig\",\"id\":\"gylgqgitxmedjvcs\",\"name\":\"n\",\"properties\":{\"status\":\"ncw\",\"startedAt\":\"2021-09-15T03:16:23Z\",\"stoppedAt\":\"2021-03-30T03:40:31Z\"}}") + .toObject(ExperimentExecutionInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionListResultTests.java index ae27b76dd691f..beee7877c53cf 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionListResultTests.java @@ -10,11 +10,9 @@ public final class ExperimentExecutionListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentExecutionListResult model = - BinaryData - .fromString( - "{\"value\":[{\"type\":\"xiebwwaloayqcg\",\"id\":\"tzjuzgwyzmhtxo\",\"name\":\"mtsavjcbpwxqp\",\"properties\":{\"status\":\"nftguvriuhpr\",\"startedAt\":\"2021-02-04T08:02:16Z\",\"stoppedAt\":\"2021-06-28T04:39:12Z\"}},{\"type\":\"qtayri\",\"id\":\"ro\",\"name\":\"bexrmcq\",\"properties\":{\"status\":\"cnojvknmefqsg\",\"startedAt\":\"2021-11-18T15:54:51Z\",\"stoppedAt\":\"2021-11-22T14:09:07Z\"}},{\"type\":\"jyzhpvgq\",\"id\":\"j\",\"name\":\"xdjzlmwlxk\",\"properties\":{\"status\":\"fhzovawjvzunluth\",\"startedAt\":\"2021-01-21T11:08:17Z\",\"stoppedAt\":\"2021-03-11T05:06:45Z\"}},{\"type\":\"i\",\"id\":\"ilpjzuaejxdult\",\"name\":\"zbbtdzumveek\",\"properties\":{\"status\":\"ozuhkfp\",\"startedAt\":\"2021-02-19T22:50:40Z\",\"stoppedAt\":\"2021-08-24T12:02:24Z\"}}],\"nextLink\":\"dxluu\"}") - .toObject(ExperimentExecutionListResult.class); + ExperimentExecutionListResult model = BinaryData.fromString( + "{\"value\":[{\"type\":\"nojvknmefqsg\",\"id\":\"ah\",\"name\":\"jyzhpvgq\",\"properties\":{\"status\":\"rvxdjzlmw\",\"startedAt\":\"2021-07-11T22:18:52Z\",\"stoppedAt\":\"2021-11-03T14:53:04Z\"}},{\"type\":\"hzovawjvzunlut\",\"id\":\"n\",\"name\":\"nxipeil\",\"properties\":{\"status\":\"uaejxdultsk\",\"startedAt\":\"2021-11-26T08:31:17Z\",\"stoppedAt\":\"2021-05-16T15:29:03Z\"}},{\"type\":\"umveekgpwozuhkf\",\"id\":\"sjyofdx\",\"name\":\"us\",\"properties\":{\"status\":\"ouwaboekqvkeln\",\"startedAt\":\"2021-02-02T03:25:54Z\",\"stoppedAt\":\"2020-12-28T23:28:21Z\"}},{\"type\":\"yjsflhhcaalnji\",\"id\":\"sxyawjoyaqcs\",\"name\":\"jpkiidzyexznelix\",\"properties\":{\"status\":\"ztfolhbnxk\",\"startedAt\":\"2021-03-24T05:03:40Z\",\"stoppedAt\":\"2021-09-05T19:55:42Z\"}}],\"nextLink\":\"ppg\"}") + .toObject(ExperimentExecutionListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionPropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionPropertiesTests.java index ce4b3e0d82cab..437ca93c39f9f 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionPropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentExecutionPropertiesTests.java @@ -10,11 +10,9 @@ public final class ExperimentExecutionPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentExecutionProperties model = - BinaryData - .fromString( - "{\"status\":\"lyjpk\",\"startedAt\":\"2021-04-24T14:25:44Z\",\"stoppedAt\":\"2021-12-04T03:35:31Z\"}") - .toObject(ExperimentExecutionProperties.class); + ExperimentExecutionProperties model = BinaryData.fromString( + "{\"status\":\"trmgucnapkte\",\"startedAt\":\"2021-08-31T01:54:16Z\",\"stoppedAt\":\"2021-11-12T01:15:19Z\"}") + .toObject(ExperimentExecutionProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentInnerTests.java index e7ac877f73766..775cafcd68fdf 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentInnerTests.java @@ -22,11 +22,9 @@ public final class ExperimentInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentInner model = - BinaryData - .fromString( - "{\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"hagalpbuxwgipwh\":{\"principalId\":\"9b0db9d2-9556-4d54-973c-006d52ba9b4e\",\"clientId\":\"37bbce60-3515-4461-9385-a696b18becec\"}},\"principalId\":\"ow\",\"tenantId\":\"shwankixzbinje\"},\"properties\":{\"provisioningState\":\"Succeeded\",\"steps\":[{\"name\":\"mryw\",\"branches\":[{\"name\":\"zoqftiyqzrnkcqvy\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"whzlsicohoq\"}]},{\"name\":\"nwvlryavwhheunmm\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"gyxzk\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"noc\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"koklya\"}]}]},{\"name\":\"uconuqszfkbey\",\"branches\":[{\"name\":\"wrmjmwvvjektc\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"enhwlrs\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"frzpwvlqdqgb\"}]},{\"name\":\"qylihkaetckt\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"civfsnkymuctq\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"jf\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"ebrjcxe\"}]},{\"name\":\"fuwutttxf\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"rbirphxe\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"c\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"vahfn\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"jky\"}]},{\"name\":\"xjvuujqgidokg\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"jyoxgvclt\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"gsncghkjeszz\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"bijhtxfvgxbf\"}]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"xnehmpvec\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ukgri\":\"dataebfqkkrbm\",\"fbxzpuzycisp\":\"dataflz\",\"y\":\"dataqzahmgkbrp\",\"rgvtqag\":\"datahibnuqqkpika\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"buynhijggm\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"zmhjrunmp\":\"dataiarbutrcvpna\",\"nkxmyskpbhenbtk\":\"datattdbhrbnl\",\"yxczfclh\":\"dataxywnytnrsynlqidy\"}}]},\"location\":\"axdbabph\",\"tags\":{\"cocmnyyaztt\":\"qlfktsths\",\"edckzywbiexzfey\":\"twwrqp\",\"ujwb\":\"eaxib\",\"zjancuxr\":\"qwalmuzyoxaepd\"},\"id\":\"d\",\"name\":\"bavxbniwdjswzt\",\"type\":\"dbpgnxytxhp\"}") - .toObject(ExperimentInner.class); + ExperimentInner model = BinaryData.fromString( + "{\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"hagalpbuxwgipwh\":{\"principalId\":\"ab062eec-ed37-4f5c-aaa7-69e1f3d4edf6\",\"clientId\":\"6597e0c3-0aaa-46e2-a4cb-b9c75f83f580\"}},\"principalId\":\"ow\",\"tenantId\":\"shwankixzbinje\"},\"properties\":{\"provisioningState\":\"Succeeded\",\"steps\":[{\"name\":\"mryw\",\"branches\":[{\"name\":\"zoqftiyqzrnkcqvy\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"whzlsicohoq\"}]},{\"name\":\"nwvlryavwhheunmm\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"gyxzk\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"noc\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"koklya\"}]}]},{\"name\":\"uconuqszfkbey\",\"branches\":[{\"name\":\"wrmjmwvvjektc\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"enhwlrs\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"frzpwvlqdqgb\"}]},{\"name\":\"qylihkaetckt\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"civfsnkymuctq\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"jf\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"ebrjcxe\"}]},{\"name\":\"fuwutttxf\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"rbirphxe\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"c\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"vahfn\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"jky\"}]},{\"name\":\"xjvuujqgidokg\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"jyoxgvclt\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"gsncghkjeszz\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"bijhtxfvgxbf\"}]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"xnehmpvec\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ukgri\":\"dataebfqkkrbm\",\"fbxzpuzycisp\":\"dataflz\",\"y\":\"dataqzahmgkbrp\",\"rgvtqag\":\"datahibnuqqkpika\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"buynhijggm\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"zmhjrunmp\":\"dataiarbutrcvpna\",\"nkxmyskpbhenbtk\":\"datattdbhrbnl\",\"yxczfclh\":\"dataxywnytnrsynlqidy\"}}]},\"location\":\"axdbabph\",\"tags\":{\"cocmnyyaztt\":\"qlfktsths\",\"edckzywbiexzfey\":\"twwrqp\",\"ujwb\":\"eaxib\",\"zjancuxr\":\"qwalmuzyoxaepd\"},\"id\":\"d\",\"name\":\"bavxbniwdjswzt\",\"type\":\"dbpgnxytxhp\"}") + .toObject(ExperimentInner.class); Assertions.assertEquals("axdbabph", model.location()); Assertions.assertEquals("qlfktsths", model.tags().get("cocmnyyaztt")); Assertions.assertEquals(ResourceIdentityType.USER_ASSIGNED, model.identity().type()); @@ -38,91 +36,45 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExperimentInner model = - new ExperimentInner() - .withLocation("axdbabph") + ExperimentInner model + = new ExperimentInner().withLocation("axdbabph") .withTags( - mapOf( - "cocmnyyaztt", - "qlfktsths", - "edckzywbiexzfey", - "twwrqp", - "ujwb", - "eaxib", - "zjancuxr", + mapOf("cocmnyyaztt", "qlfktsths", "edckzywbiexzfey", "twwrqp", "ujwb", "eaxib", "zjancuxr", "qwalmuzyoxaepd")) .withIdentity( new ResourceIdentity() - .withType(ResourceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities(mapOf("hagalpbuxwgipwh", new UserAssignedIdentity()))) - .withSteps( - Arrays - .asList( - new ChaosExperimentStep() - .withName("mryw") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("zoqftiyqzrnkcqvy") - .withActions( - Arrays.asList(new ChaosExperimentAction().withName("whzlsicohoq"))), - new ChaosExperimentBranch() - .withName("nwvlryavwhheunmm") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("gyxzk"), - new ChaosExperimentAction().withName("noc"), - new ChaosExperimentAction().withName("koklya"))))), - new ChaosExperimentStep() - .withName("uconuqszfkbey") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("wrmjmwvvjektc") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("enhwlrs"), - new ChaosExperimentAction().withName("frzpwvlqdqgb"))), - new ChaosExperimentBranch() - .withName("qylihkaetckt") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("civfsnkymuctq"), - new ChaosExperimentAction().withName("jf"), - new ChaosExperimentAction().withName("ebrjcxe"))), - new ChaosExperimentBranch() - .withName("fuwutttxf") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("rbirphxe"), - new ChaosExperimentAction().withName("c"), - new ChaosExperimentAction().withName("vahfn"), - new ChaosExperimentAction().withName("jky"))), - new ChaosExperimentBranch() - .withName("xjvuujqgidokg") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("jyoxgvclt"), - new ChaosExperimentAction().withName("gsncghkjeszz"), - new ChaosExperimentAction().withName("bijhtxfvgxbf"))))))) - .withSelectors( - Arrays - .asList( - new ChaosTargetSelector() - .withId("xnehmpvec") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), - new ChaosTargetSelector() - .withId("buynhijggm") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")))); + .withType(ResourceIdentityType.USER_ASSIGNED).withUserAssignedIdentities( + mapOf("hagalpbuxwgipwh", new UserAssignedIdentity()))) + .withSteps(Arrays.asList( + new ChaosExperimentStep().withName("mryw").withBranches(Arrays.asList( + new ChaosExperimentBranch().withName("zoqftiyqzrnkcqvy") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("whzlsicohoq"))), + new ChaosExperimentBranch().withName("nwvlryavwhheunmm") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("gyxzk"), + new ChaosExperimentAction().withName("noc"), + new ChaosExperimentAction().withName("koklya"))))), + new ChaosExperimentStep().withName("uconuqszfkbey").withBranches(Arrays.asList( + new ChaosExperimentBranch().withName("wrmjmwvvjektc") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("enhwlrs"), + new ChaosExperimentAction().withName("frzpwvlqdqgb"))), + new ChaosExperimentBranch().withName("qylihkaetckt") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("civfsnkymuctq"), + new ChaosExperimentAction().withName("jf"), + new ChaosExperimentAction().withName("ebrjcxe"))), + new ChaosExperimentBranch().withName("fuwutttxf") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("rbirphxe"), + new ChaosExperimentAction().withName("c"), + new ChaosExperimentAction().withName("vahfn"), + new ChaosExperimentAction().withName("jky"))), + new ChaosExperimentBranch().withName("xjvuujqgidokg") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("jyoxgvclt"), + new ChaosExperimentAction().withName("gsncghkjeszz"), + new ChaosExperimentAction().withName("bijhtxfvgxbf"))))))) + .withSelectors(Arrays.asList( + new ChaosTargetSelector().withId("xnehmpvec").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), + new ChaosTargetSelector().withId("buynhijggm").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")))); model = BinaryData.fromObject(model).toObject(ExperimentInner.class); Assertions.assertEquals("axdbabph", model.location()); Assertions.assertEquals("qlfktsths", model.tags().get("cocmnyyaztt")); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentListResultTests.java index 71973cdd0f770..0b4a18d4d4ec0 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentListResultTests.java @@ -10,11 +10,9 @@ public final class ExperimentListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentListResult model = - BinaryData - .fromString( - "{\"value\":[{\"identity\":{\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"a\":{\"principalId\":\"cd1ba453-abce-4337-8cd6-f1dbead03fc3\",\"clientId\":\"bc064b79-8b4f-45f4-88ae-b771b3f9b09e\"},\"fizuckyf\":{\"principalId\":\"a9f58721-949a-4641-ba81-dd3219bb3361\",\"clientId\":\"5eae2532-43d7-4815-b11b-faf9031cae80\"}},\"principalId\":\"rfidfvzwdz\",\"tenantId\":\"tymw\"},\"properties\":{\"provisioningState\":\"Updating\",\"steps\":[{\"name\":\"fthwxmnteiwa\",\"branches\":[{\"name\":\"vkmijcmmxdcuf\",\"actions\":[]},{\"name\":\"fsrpymzidnse\",\"actions\":[]},{\"name\":\"cxtbzsg\",\"actions\":[]},{\"name\":\"yc\",\"actions\":[]}]},{\"name\":\"sne\",\"branches\":[{\"name\":\"dwzjeiach\",\"actions\":[]},{\"name\":\"oosflnr\",\"actions\":[]},{\"name\":\"sfqpteehz\",\"actions\":[]},{\"name\":\"vypyqrimzinpv\",\"actions\":[]}]},{\"name\":\"wjdk\",\"branches\":[{\"name\":\"soodqxhcrmnoh\",\"actions\":[]},{\"name\":\"t\",\"actions\":[]},{\"name\":\"kwh\",\"actions\":[]},{\"name\":\"soifiyipjxsqw\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"rjb\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"mocpc\":\"datacjxvsnbyxqab\",\"c\":\"datashurzafbljjgpbto\",\"idtqajzyu\":\"datamkljavb\",\"jkrlkhbzhfepg\":\"datapku\"}}]},\"location\":\"gqexzlocxs\",\"tags\":{\"j\":\"ierhhbcsglummaj\",\"jionpimexgstxgc\":\"odxobnbdxkqpxok\"},\"id\":\"odgmaajrmvdjwz\",\"name\":\"lovmclwhijcoe\",\"type\":\"ctbzaq\"},{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"fkgukdkexxppof\":{\"principalId\":\"73c6842a-4ce6-4276-a088-bbd4ffc9151b\",\"clientId\":\"59ba3843-3d81-4aa7-84a0-8767620db3ac\"},\"x\":{\"principalId\":\"c42fd434-765f-4f95-8480-fcb4e7fefb72\",\"clientId\":\"2ded50ac-ffd3-48fc-a71a-59a86940060b\"},\"pg\":{\"principalId\":\"c52f9d33-7f20-4ae2-a258-ff6945c7afba\",\"clientId\":\"9ab61bea-4229-4ba4-a0c4-23ceb66c02fc\"}},\"principalId\":\"toc\",\"tenantId\":\"xhvpmoue\"},\"properties\":{\"provisioningState\":\"Failed\",\"steps\":[{\"name\":\"xibqeojnx\",\"branches\":[{\"name\":\"zvddntwndeicbtwn\",\"actions\":[]}]},{\"name\":\"zao\",\"branches\":[{\"name\":\"uhrhcffcyddgl\",\"actions\":[]},{\"name\":\"jthjqkwpyei\",\"actions\":[]},{\"name\":\"xmqci\",\"actions\":[]},{\"name\":\"q\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"khixuigdtopbo\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"w\":\"datahm\",\"a\":\"dataa\"}}]},\"location\":\"hrzayvvtpgvdf\",\"tags\":{\"efgugnxk\":\"tkftutqxlngx\",\"hjybigehoqfbo\":\"xdqmidtthzrvqdra\",\"zlcuiywgqywgndrv\":\"skanyk\",\"ocpecfvmmco\":\"nhzgpphrcgyn\"},\"id\":\"fsxlzevgbmqjqa\",\"name\":\"c\",\"type\":\"pmivkwlzu\"},{\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"bacfionlebxetq\":{\"principalId\":\"4fbd05f7-80cf-4c3d-bf0f-9eb454da4ca5\",\"clientId\":\"316b8493-82f0-4011-9efa-c118f420cd16\"},\"xdpnqbqq\":{\"principalId\":\"73fc0ce4-c7ba-4215-893d-e6e7d7ab685f\",\"clientId\":\"6e6ba8ce-3da5-4d4c-9850-7242e872ce4f\"},\"jfeallnwsub\":{\"principalId\":\"cbfa5e8e-e711-44f5-b881-4e8c2a65ba32\",\"clientId\":\"3569f45c-d910-4755-b823-f8862d17432c\"},\"jampmngnzscxaqw\":{\"principalId\":\"7d945f06-cdc5-4076-9c82-3ef3b6463cb4\",\"clientId\":\"09e5bf68-1f27-444a-bc0c-053c6ef7f367\"}},\"principalId\":\"chcbonqvpkvlrxnj\",\"tenantId\":\"seiphe\"},\"properties\":{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"keyyi\",\"branches\":[{\"name\":\"jbdlwtgrhpdjpju\",\"actions\":[]}]},{\"name\":\"asxazjpqyegualhb\",\"branches\":[{\"name\":\"hejjz\",\"actions\":[]},{\"name\":\"v\",\"actions\":[]},{\"name\":\"udgwdslfho\",\"actions\":[]}]},{\"name\":\"wmc\",\"branches\":[{\"name\":\"pwlbjnpg\",\"actions\":[]},{\"name\":\"cftadeh\",\"actions\":[]},{\"name\":\"nltyfsoppusuesnz\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"ej\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ndnvo\":\"datarxzdmohctbqvudwx\",\"lazjdyggdtjixhbk\":\"datagujjugwdkcglh\",\"fwhybcibvy\":\"dataofqweykhmenevfye\",\"ynnaam\":\"datadcsi\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"ectehf\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"hezrkgq\":\"datajeyp\"}}]},\"location\":\"c\",\"tags\":{\"jpkcattpng\":\"fovgmkqsleyyvxy\",\"czsqpjhvm\":\"cr\"},\"id\":\"ajvnysounqe\",\"name\":\"a\",\"type\":\"oaeupfhyhltrpmo\"},{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"okth\":{\"principalId\":\"3fb5098a-2666-4598-8453-0c5afcdda8b7\",\"clientId\":\"4807f25b-29f9-44a4-bd61-e5f622fb01b4\"},\"uaodsfcpk\":{\"principalId\":\"658efceb-7d9d-4bd0-988b-fc779abd01f4\",\"clientId\":\"6fb50596-8811-4c6e-a0b7-b49556a3a7d9\"},\"dpuozmyz\":{\"principalId\":\"d9165f61-b80d-43d3-9bab-3d3dcde8d89a\",\"clientId\":\"445dc8e7-6013-4af4-9fde-43cd6280218c\"}},\"principalId\":\"agfuaxbezyiu\",\"tenantId\":\"ktwh\"},\"properties\":{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"zywqsmbsu\",\"branches\":[{\"name\":\"xim\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"yocf\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"kiiuxhqyudxor\":\"dataymddys\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"qn\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"llr\":\"datazvyifqrvkdvj\",\"xxbczwtr\":\"datavvdfwatkpnpul\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"wiqzbqjvsovmyo\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"obpxjmflbvvn\":\"datapkwlhz\",\"ciwwzjuqkhr\":\"datahrk\",\"oskg\":\"dataajiwkuo\",\"vxieduugidyj\":\"datasauuimj\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"rfbyaosvexcso\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"eggzfb\":\"datahocohslkev\",\"ithlvmezyvshxm\":\"datahfmvfaxkffe\",\"gigr\":\"datasbbzo\"}}]},\"location\":\"wburvjxxjnspydpt\",\"tags\":{\"udwtiukbl\":\"nkoukn\",\"o\":\"ngkpocipazy\",\"ntypmrbpizcdrqj\":\"gukgjnpiucgygevq\"},\"id\":\"dpydn\",\"name\":\"yhxdeoejzicwi\",\"type\":\"sjttgzfbish\"}],\"nextLink\":\"khaj\"}") - .toObject(ExperimentListResult.class); + ExperimentListResult model = BinaryData.fromString( + "{\"value\":[{\"identity\":{\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"a\":{\"principalId\":\"255ed4c6-34ae-4c9b-a3f5-c479d3905fe3\",\"clientId\":\"149bf398-a7e2-4742-b0d3-79870042685b\"},\"fizuckyf\":{\"principalId\":\"d2cb8d1f-ac96-45cd-9a7b-324d79a7af90\",\"clientId\":\"0304c61b-354c-4833-be16-114eadb9bff5\"}},\"principalId\":\"rfidfvzwdz\",\"tenantId\":\"tymw\"},\"properties\":{\"provisioningState\":\"Updating\",\"steps\":[{\"name\":\"fthwxmnteiwa\",\"branches\":[{\"name\":\"vkmijcmmxdcuf\",\"actions\":[]},{\"name\":\"fsrpymzidnse\",\"actions\":[]},{\"name\":\"cxtbzsg\",\"actions\":[]},{\"name\":\"yc\",\"actions\":[]}]},{\"name\":\"sne\",\"branches\":[{\"name\":\"dwzjeiach\",\"actions\":[]},{\"name\":\"oosflnr\",\"actions\":[]},{\"name\":\"sfqpteehz\",\"actions\":[]},{\"name\":\"vypyqrimzinpv\",\"actions\":[]}]},{\"name\":\"wjdk\",\"branches\":[{\"name\":\"soodqxhcrmnoh\",\"actions\":[]},{\"name\":\"t\",\"actions\":[]},{\"name\":\"kwh\",\"actions\":[]},{\"name\":\"soifiyipjxsqw\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"rjb\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"mocpc\":\"datacjxvsnbyxqab\",\"c\":\"datashurzafbljjgpbto\",\"idtqajzyu\":\"datamkljavb\",\"jkrlkhbzhfepg\":\"datapku\"}}]},\"location\":\"gqexzlocxs\",\"tags\":{\"j\":\"ierhhbcsglummaj\",\"jionpimexgstxgc\":\"odxobnbdxkqpxok\"},\"id\":\"odgmaajrmvdjwz\",\"name\":\"lovmclwhijcoe\",\"type\":\"ctbzaq\"},{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"fkgukdkexxppof\":{\"principalId\":\"59e13f26-36e1-4cbc-a8ba-0b141b28818d\",\"clientId\":\"628bbccf-25bc-45c3-b1e3-8af954a8f5a6\"},\"x\":{\"principalId\":\"5916b84d-cba8-488e-a8c7-42ef55e6d785\",\"clientId\":\"1fc26ffd-d13b-40f9-be22-45a429a4cae3\"},\"pg\":{\"principalId\":\"caf2753f-940f-4586-978d-1e94d28fd415\",\"clientId\":\"a9c34bb9-854e-48be-a3b6-863dd74dc2a5\"}},\"principalId\":\"toc\",\"tenantId\":\"xhvpmoue\"},\"properties\":{\"provisioningState\":\"Failed\",\"steps\":[{\"name\":\"xibqeojnx\",\"branches\":[{\"name\":\"zvddntwndeicbtwn\",\"actions\":[]}]},{\"name\":\"zao\",\"branches\":[{\"name\":\"uhrhcffcyddgl\",\"actions\":[]},{\"name\":\"jthjqkwpyei\",\"actions\":[]},{\"name\":\"xmqci\",\"actions\":[]},{\"name\":\"q\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"khixuigdtopbo\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"w\":\"datahm\",\"a\":\"dataa\"}}]},\"location\":\"hrzayvvtpgvdf\",\"tags\":{\"efgugnxk\":\"tkftutqxlngx\",\"hjybigehoqfbo\":\"xdqmidtthzrvqdra\",\"zlcuiywgqywgndrv\":\"skanyk\",\"ocpecfvmmco\":\"nhzgpphrcgyn\"},\"id\":\"fsxlzevgbmqjqa\",\"name\":\"c\",\"type\":\"pmivkwlzu\"},{\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"bacfionlebxetq\":{\"principalId\":\"7204d12f-5595-4a88-b5db-9d9ab62fc0b9\",\"clientId\":\"2f311a43-9660-4aa3-a9fb-d68e86572abd\"},\"xdpnqbqq\":{\"principalId\":\"54c91147-345a-4eef-8bbc-d8bbe6cd851d\",\"clientId\":\"27b9ee2d-21d3-4cb5-8213-3f515f3c4753\"},\"jfeallnwsub\":{\"principalId\":\"29d4915a-f7f4-4cca-8bfc-0599d98b75b3\",\"clientId\":\"bb083cf2-faa4-4e3f-bdcc-c6d19f24c6d7\"},\"jampmngnzscxaqw\":{\"principalId\":\"a07187c6-2083-4892-b8ff-9a573449a2a3\",\"clientId\":\"a2efa42d-9caa-4e0d-a837-3512dae0695e\"}},\"principalId\":\"chcbonqvpkvlrxnj\",\"tenantId\":\"seiphe\"},\"properties\":{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"keyyi\",\"branches\":[{\"name\":\"jbdlwtgrhpdjpju\",\"actions\":[]}]},{\"name\":\"asxazjpqyegualhb\",\"branches\":[{\"name\":\"hejjz\",\"actions\":[]},{\"name\":\"v\",\"actions\":[]},{\"name\":\"udgwdslfho\",\"actions\":[]}]},{\"name\":\"wmc\",\"branches\":[{\"name\":\"pwlbjnpg\",\"actions\":[]},{\"name\":\"cftadeh\",\"actions\":[]},{\"name\":\"nltyfsoppusuesnz\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"ej\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ndnvo\":\"datarxzdmohctbqvudwx\",\"lazjdyggdtjixhbk\":\"datagujjugwdkcglh\",\"fwhybcibvy\":\"dataofqweykhmenevfye\",\"ynnaam\":\"datadcsi\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"ectehf\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"hezrkgq\":\"datajeyp\"}}]},\"location\":\"c\",\"tags\":{\"jpkcattpng\":\"fovgmkqsleyyvxy\",\"czsqpjhvm\":\"cr\"},\"id\":\"ajvnysounqe\",\"name\":\"a\",\"type\":\"oaeupfhyhltrpmo\"},{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"okth\":{\"principalId\":\"862b509f-e184-48e4-af87-86e9bfa46293\",\"clientId\":\"6687bec8-fcdb-421b-9ca8-75b8bec619c5\"},\"uaodsfcpk\":{\"principalId\":\"7227b047-d517-408e-a6f6-cfbfa1b4312f\",\"clientId\":\"7d3fd070-8fbf-4bdc-b01e-3fd2e27429b6\"},\"dpuozmyz\":{\"principalId\":\"533c94a5-73fd-4e78-9a83-bc7d98547436\",\"clientId\":\"9733cd6b-18a5-4bb6-ad2e-6adcfd916aa8\"}},\"principalId\":\"agfuaxbezyiu\",\"tenantId\":\"ktwh\"},\"properties\":{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"zywqsmbsu\",\"branches\":[{\"name\":\"xim\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"yocf\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"kiiuxhqyudxor\":\"dataymddys\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"qn\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"llr\":\"datazvyifqrvkdvj\",\"xxbczwtr\":\"datavvdfwatkpnpul\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"wiqzbqjvsovmyo\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"obpxjmflbvvn\":\"datapkwlhz\",\"ciwwzjuqkhr\":\"datahrk\",\"oskg\":\"dataajiwkuo\",\"vxieduugidyj\":\"datasauuimj\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"rfbyaosvexcso\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"eggzfb\":\"datahocohslkev\",\"ithlvmezyvshxm\":\"datahfmvfaxkffe\",\"gigr\":\"datasbbzo\"}}]},\"location\":\"wburvjxxjnspydpt\",\"tags\":{\"udwtiukbl\":\"nkoukn\",\"o\":\"ngkpocipazy\",\"ntypmrbpizcdrqj\":\"gukgjnpiucgygevq\"},\"id\":\"dpydn\",\"name\":\"yhxdeoejzicwi\",\"type\":\"sjttgzfbish\"}],\"nextLink\":\"khaj\"}") + .toObject(ExperimentListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentPropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentPropertiesTests.java index e5ac2ab611fb8..7568078d455a6 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentPropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentPropertiesTests.java @@ -19,11 +19,9 @@ public final class ExperimentPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentProperties model = - BinaryData - .fromString( - "{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"mg\",\"branches\":[{\"name\":\"nkjzkdeslpvlop\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"yighxpk\"}]},{\"name\":\"wzbaiue\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"a\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"m\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"yqupedeojnabckh\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"mtxpsiebtfh\"}]},{\"name\":\"pesapskrdqmhjj\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"tldwkyzxuutk\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"cwscwsvlx\"}]}]},{\"name\":\"togt\",\"branches\":[{\"name\":\"upqsx\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"micykvceoveilo\"}]},{\"name\":\"notyfjfcnjbkcn\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"hbttkphyw\"}]},{\"name\":\"nv\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"oqnermclfpl\"}]}]},{\"name\":\"hoxus\",\"branches\":[{\"name\":\"pabgyeps\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"tazqugxywpmueefj\"}]},{\"name\":\"wfqkquj\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"suyonobglaocq\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"tcc\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"g\"}]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"dxyt\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"fudwpznt\":\"datarxv\",\"hckfrlhrx\":\"datahdzhlrqj\",\"ca\":\"databkyvp\",\"b\":\"datauzbpzkafku\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"rnwb\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"srtslhspkdeem\":\"dataseyvj\",\"ahvljuaha\":\"dataofmxagkvtmelmqkr\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"uhcdhm\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"pvfadmwsrcr\":\"dataaex\",\"fmisg\":\"datavxpvgomz\"}}]}") - .toObject(ExperimentProperties.class); + ExperimentProperties model = BinaryData.fromString( + "{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"mg\",\"branches\":[{\"name\":\"nkjzkdeslpvlop\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"yighxpk\"}]},{\"name\":\"wzbaiue\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"a\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"m\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"yqupedeojnabckh\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"mtxpsiebtfh\"}]},{\"name\":\"pesapskrdqmhjj\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"tldwkyzxuutk\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"cwscwsvlx\"}]}]},{\"name\":\"togt\",\"branches\":[{\"name\":\"upqsx\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"micykvceoveilo\"}]},{\"name\":\"notyfjfcnjbkcn\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"hbttkphyw\"}]},{\"name\":\"nv\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"oqnermclfpl\"}]}]},{\"name\":\"hoxus\",\"branches\":[{\"name\":\"pabgyeps\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"tazqugxywpmueefj\"}]},{\"name\":\"wfqkquj\",\"actions\":[{\"type\":\"ChaosExperimentAction\",\"name\":\"suyonobglaocq\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"tcc\"},{\"type\":\"ChaosExperimentAction\",\"name\":\"g\"}]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"dxyt\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"fudwpznt\":\"datarxv\",\"hckfrlhrx\":\"datahdzhlrqj\",\"ca\":\"databkyvp\",\"b\":\"datauzbpzkafku\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"rnwb\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"srtslhspkdeem\":\"dataseyvj\",\"ahvljuaha\":\"dataofmxagkvtmelmqkr\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"uhcdhm\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"pvfadmwsrcr\":\"dataaex\",\"fmisg\":\"datavxpvgomz\"}}]}") + .toObject(ExperimentProperties.class); Assertions.assertEquals("mg", model.steps().get(0).name()); Assertions.assertEquals("nkjzkdeslpvlop", model.steps().get(0).branches().get(0).name()); Assertions.assertEquals("yighxpk", model.steps().get(0).branches().get(0).actions().get(0).name()); @@ -32,90 +30,42 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExperimentProperties model = - new ExperimentProperties() - .withSteps( - Arrays - .asList( - new ChaosExperimentStep() - .withName("mg") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("nkjzkdeslpvlop") - .withActions( - Arrays.asList(new ChaosExperimentAction().withName("yighxpk"))), - new ChaosExperimentBranch() - .withName("wzbaiue") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("a"), - new ChaosExperimentAction().withName("m"), - new ChaosExperimentAction().withName("yqupedeojnabckh"), - new ChaosExperimentAction().withName("mtxpsiebtfh"))), - new ChaosExperimentBranch() - .withName("pesapskrdqmhjj") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("tldwkyzxuutk"), - new ChaosExperimentAction().withName("cwscwsvlx"))))), - new ChaosExperimentStep() - .withName("togt") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("upqsx") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("micykvceoveilo"))), - new ChaosExperimentBranch() - .withName("notyfjfcnjbkcn") - .withActions( - Arrays.asList(new ChaosExperimentAction().withName("hbttkphyw"))), - new ChaosExperimentBranch() - .withName("nv") - .withActions( - Arrays - .asList(new ChaosExperimentAction().withName("oqnermclfpl"))))), - new ChaosExperimentStep() - .withName("hoxus") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("pabgyeps") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("tazqugxywpmueefj"))), - new ChaosExperimentBranch() - .withName("wfqkquj") - .withActions( - Arrays - .asList( - new ChaosExperimentAction().withName("suyonobglaocq"), - new ChaosExperimentAction().withName("tcc"), - new ChaosExperimentAction().withName("g"))))))) - .withSelectors( - Arrays - .asList( - new ChaosTargetSelector() - .withId("dxyt") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), - new ChaosTargetSelector() - .withId("rnwb") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), - new ChaosTargetSelector() - .withId("uhcdhm") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")))); + ExperimentProperties model = new ExperimentProperties() + .withSteps(Arrays.asList( + new ChaosExperimentStep().withName("mg") + .withBranches(Arrays.asList( + new ChaosExperimentBranch().withName("nkjzkdeslpvlop") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("yighxpk"))), + new ChaosExperimentBranch().withName("wzbaiue") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("a"), + new ChaosExperimentAction().withName("m"), + new ChaosExperimentAction().withName("yqupedeojnabckh"), + new ChaosExperimentAction().withName("mtxpsiebtfh"))), + new ChaosExperimentBranch().withName("pesapskrdqmhjj") + .withActions( + Arrays.asList(new ChaosExperimentAction().withName("tldwkyzxuutk"), + new ChaosExperimentAction().withName("cwscwsvlx"))))), + new ChaosExperimentStep().withName("togt") + .withBranches(Arrays.asList( + new ChaosExperimentBranch().withName("upqsx").withActions( + Arrays.asList(new ChaosExperimentAction().withName("micykvceoveilo"))), + new ChaosExperimentBranch().withName("notyfjfcnjbkcn") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("hbttkphyw"))), + new ChaosExperimentBranch().withName("nv") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("oqnermclfpl"))))), + new ChaosExperimentStep().withName("hoxus").withBranches(Arrays.asList( + new ChaosExperimentBranch().withName("pabgyeps") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("tazqugxywpmueefj"))), + new ChaosExperimentBranch().withName("wfqkquj") + .withActions(Arrays.asList(new ChaosExperimentAction().withName("suyonobglaocq"), + new ChaosExperimentAction().withName("tcc"), new ChaosExperimentAction().withName("g"))))))) + .withSelectors(Arrays.asList( + new ChaosTargetSelector().withId("dxyt").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), + new ChaosTargetSelector().withId("rnwb").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), + new ChaosTargetSelector().withId("uhcdhm").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")))); model = BinaryData.fromObject(model).toObject(ExperimentProperties.class); Assertions.assertEquals("mg", model.steps().get(0).name()); Assertions.assertEquals("nkjzkdeslpvlop", model.steps().get(0).branches().get(0).name()); diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentUpdateTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentUpdateTests.java index 624a4bbfa455f..b900d67b16d4f 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentUpdateTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentUpdateTests.java @@ -16,33 +16,24 @@ public final class ExperimentUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExperimentUpdate model = - BinaryData - .fromString( - "{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"veual\":{\"principalId\":\"722f6d5d-a4be-49ef-859d-8e0961238354\",\"clientId\":\"a4541266-741f-439c-981a-c1cf2b17ae98\"},\"mkh\":{\"principalId\":\"54b509c6-afca-41df-a2ea-819d552cfcf1\",\"clientId\":\"9a36844d-8db6-4d0e-b5f9-c495c195c8a6\"},\"bbcswsrtjri\":{\"principalId\":\"804d3b9b-f5ae-4616-9331-d242e48040b4\",\"clientId\":\"63b3ce36-c7db-4e64-bd93-42681aab08ca\"},\"bpbewtghfgb\":{\"principalId\":\"a21c1186-61a2-47c5-b1c6-e74d4d4d9ccf\",\"clientId\":\"19babc93-1b64-4e6c-8513-919d7f32cae6\"}},\"principalId\":\"gw\",\"tenantId\":\"vlvqhjkbegi\"}}") - .toObject(ExperimentUpdate.class); + ExperimentUpdate model = BinaryData.fromString( + "{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"veual\":{\"principalId\":\"2ab296ac-e81e-4690-b2e9-e8ca33e2c7f3\",\"clientId\":\"e2c93b86-0c6b-4a76-b205-8abd8405e73c\"},\"mkh\":{\"principalId\":\"a488b418-a1eb-433c-a43f-5fae44a0b576\",\"clientId\":\"6b698d05-0238-4471-9218-dbbad59580b9\"},\"bbcswsrtjri\":{\"principalId\":\"8116ceee-1e63-4d79-ad88-a3b1cf189c9d\",\"clientId\":\"7893fbc9-a3b8-4f3d-9909-b5051e6de38e\"},\"bpbewtghfgb\":{\"principalId\":\"d1b6d058-d12a-45b2-b1f0-0a6e04a0d8ff\",\"clientId\":\"2cb9b1c9-46cf-4f32-b1c0-888374c5f9e6\"}},\"principalId\":\"gw\",\"tenantId\":\"vlvqhjkbegi\"},\"tags\":{\"aloayqcgwrtzju\":\"mxiebw\",\"txon\":\"gwyzm\",\"rknftguvriuhprwm\":\"mtsavjcbpwxqp\",\"bexrmcq\":\"yvxqtayriwwroy\"}}") + .toObject(ExperimentUpdate.class); Assertions.assertEquals(ResourceIdentityType.NONE, model.identity().type()); + Assertions.assertEquals("mxiebw", model.tags().get("aloayqcgwrtzju")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExperimentUpdate model = - new ExperimentUpdate() - .withIdentity( - new ResourceIdentity() - .withType(ResourceIdentityType.NONE) - .withUserAssignedIdentities( - mapOf( - "veual", - new UserAssignedIdentity(), - "mkh", - new UserAssignedIdentity(), - "bbcswsrtjri", - new UserAssignedIdentity(), - "bpbewtghfgb", - new UserAssignedIdentity()))); + ExperimentUpdate model = new ExperimentUpdate() + .withIdentity(new ResourceIdentity().withType(ResourceIdentityType.NONE).withUserAssignedIdentities( + mapOf("veual", new UserAssignedIdentity(), "mkh", new UserAssignedIdentity(), "bbcswsrtjri", + new UserAssignedIdentity(), "bpbewtghfgb", new UserAssignedIdentity()))) + .withTags(mapOf("aloayqcgwrtzju", "mxiebw", "txon", "gwyzm", "rknftguvriuhprwm", "mtsavjcbpwxqp", "bexrmcq", + "yvxqtayriwwroy")); model = BinaryData.fromObject(model).toObject(ExperimentUpdate.class); Assertions.assertEquals(ResourceIdentityType.NONE, model.identity().type()); + Assertions.assertEquals("mxiebw", model.tags().get("aloayqcgwrtzju")); } // Use "Map.of" if available diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateMockTests.java index c47606a7fe3a5..afcf5ef2888b8 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsCreateOrUpdateMockTests.java @@ -40,102 +40,49 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"thjoxoism\":{\"principalId\":\"e83eb8bb-c87a-4a10-8644-8e10e6424e65\",\"clientId\":\"a82cee1b-b08f-4952-bdb9-ca138978dcfb\"}},\"principalId\":\"sbpimlq\",\"tenantId\":\"jxkcgxxlxsff\"},\"properties\":{\"provisioningState\":\"Succeeded\",\"steps\":[{\"name\":\"zqzdw\",\"branches\":[{\"name\":\"w\",\"actions\":[]},{\"name\":\"youpfgfbkj\",\"actions\":[]}]},{\"name\":\"bdyhgkfminsgowz\",\"branches\":[{\"name\":\"tsttktlahbq\",\"actions\":[]},{\"name\":\"ctxtgzukxi\",\"actions\":[]},{\"name\":\"mmqtgqqqxhr\",\"actions\":[]},{\"name\":\"xrxc\",\"actions\":[]}]},{\"name\":\"juisavokqdzf\",\"branches\":[{\"name\":\"zivj\",\"actions\":[]},{\"name\":\"frqttbajlkatnw\",\"actions\":[]},{\"name\":\"yiopi\",\"actions\":[]},{\"name\":\"kqqfk\",\"actions\":[]}]},{\"name\":\"vscx\",\"branches\":[{\"name\":\"mligov\",\"actions\":[]},{\"name\":\"brxk\",\"actions\":[]},{\"name\":\"mloazuru\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"bgo\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ordilmywwtkgkxny\":\"dataeoybfhjxakvvjgs\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"dabg\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"uwhcjyxccybv\":\"datatjuewbcihx\"}}]},\"location\":\"ayakkudzpx\",\"tags\":{\"tcyohpfkyrk\":\"plmag\",\"nwqjnoba\":\"bdgiogsjk\",\"egfnmntfpmvmemfn\":\"yhddvia\"},\"id\":\"zdwvvbalxl\",\"name\":\"lchpodbzevwrdn\",\"type\":\"fukuvsjcswsmystu\"}"; + String responseStr + = "{\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"ev\":{\"principalId\":\"6900a581-a089-4451-b294-0dccce9a7181\",\"clientId\":\"c0aab7de-0ad5-4a09-ad58-31e9defd8dd4\"},\"b\":{\"principalId\":\"25a877c7-39ff-46be-8058-0c1f8c6955e9\",\"clientId\":\"2986140d-b1b7-4329-a3f5-cb5f12d81712\"}},\"principalId\":\"rilbywdx\",\"tenantId\":\"icc\"},\"properties\":{\"provisioningState\":\"Succeeded\",\"steps\":[{\"name\":\"scjfnyns\",\"branches\":[{\"name\":\"ujiz\",\"actions\":[]},{\"name\":\"voqyt\",\"actions\":[]}]},{\"name\":\"byowbblgyavutp\",\"branches\":[{\"name\":\"joxoism\",\"actions\":[]},{\"name\":\"ksbpimlqoljx\",\"actions\":[]}]},{\"name\":\"cgxxlxs\",\"branches\":[{\"name\":\"gcvizqzdwlvwlyou\",\"actions\":[]},{\"name\":\"fgfb\",\"actions\":[]}]},{\"name\":\"jub\",\"branches\":[{\"name\":\"hgkfmin\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"owzfttsttkt\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"gqqqxh\":\"dataqactxtgzukxitmmq\",\"rxcpjuisavo\":\"datan\",\"ajlkatnw\":\"dataqdzfvazivjlfrqtt\"}}]},\"location\":\"yiopi\",\"tags\":{\"ovibrxkp\":\"qfkuvscxkdmli\",\"bteoybf\":\"loazuruocbgoo\"},\"id\":\"jxakv\",\"name\":\"jgslordilmyww\",\"type\":\"kgkxn\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Experiment response = - manager - .experiments() - .define("mvvhmxtdrjfuta") - .withRegion("s") - .withExistingResourceGroup("gohbuffkmrq") - .withSteps( - Arrays - .asList( - new ChaosExperimentStep() - .withName("kwdlenrdsutujba") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("juohminyflnorw") - .withActions(Arrays.asList()), - new ChaosExperimentBranch() - .withName("duvwpklvxwmygd") - .withActions(Arrays.asList()), - new ChaosExperimentBranch() - .withName("pgpqchiszepnnb") - .withActions(Arrays.asList()))), - new ChaosExperimentStep() - .withName("crxgibb") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch() - .withName("xconfozauors") - .withActions(Arrays.asList()), - new ChaosExperimentBranch() - .withName("kokwbqplhlvnu") - .withActions(Arrays.asList()), - new ChaosExperimentBranch().withName("epzl").withActions(Arrays.asList()))), - new ChaosExperimentStep() - .withName("phwzsoldweyuqdu") - .withBranches( - Arrays - .asList( - new ChaosExperimentBranch().withName("mnnrwr").withActions(Arrays.asList()), - new ChaosExperimentBranch().withName("i").withActions(Arrays.asList()))))) - .withSelectors( - Arrays - .asList( - new ChaosTargetSelector() - .withId("ktalywjhhgdnhxms") - .withFilter(new ChaosTargetFilter()) - .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")))) - .withTags(mapOf("p", "otcubi", "izsh", "ipwoqonmacjek", "gmblrri", "vcimpev")) - .withIdentity( - new ResourceIdentity() - .withType(ResourceIdentityType.SYSTEM_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "zcjznmwcpmgua", - new UserAssignedIdentity(), - "aufactkahzovajjz", - new UserAssignedIdentity()))) - .create(); + Experiment response = manager.experiments().define("yjkqabqgzslesjcb").withRegion("axconfozauo") + .withExistingResourceGroup("aryeu") + .withSteps(Arrays.asList(new ChaosExperimentStep().withName("ebjvewzcjzn").withBranches( + Arrays.asList(new ChaosExperimentBranch().withName("cpmguaadraufact").withActions(Arrays.asList()), + new ChaosExperimentBranch().withName("ahzovajjziuxxp").withActions(Arrays.asList()), + new ChaosExperimentBranch().withName("hneekul").withActions(Arrays.asList()), + new ChaosExperimentBranch().withName("gs").withActions(Arrays.asList()))))) + .withSelectors(Arrays.asList( + new ChaosTargetSelector().withId("ubkwdle").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")), + new ChaosTargetSelector().withId("ygdxpgpqchis").withFilter(new ChaosTargetFilter()) + .withAdditionalProperties(mapOf("type", "ChaosTargetSelector")))) + .withTags(mapOf("epzl", "kokwbqplhlvnu", "vmnnrw", "phwzsoldweyuqdu", "wjhhgdnhxmsivf", "biorktal", + "zaofjchvcyy", "miloxggdufiqndie")) + .withIdentity( + new ResourceIdentity().withType(ResourceIdentityType.USER_ASSIGNED).withUserAssignedIdentities( + mapOf("w", new UserAssignedIdentity(), "vbquwr", new UserAssignedIdentity()))) + .create(); - Assertions.assertEquals("ayakkudzpx", response.location()); - Assertions.assertEquals("plmag", response.tags().get("tcyohpfkyrk")); - Assertions.assertEquals(ResourceIdentityType.NONE, response.identity().type()); - Assertions.assertEquals("zqzdw", response.steps().get(0).name()); - Assertions.assertEquals("w", response.steps().get(0).branches().get(0).name()); - Assertions.assertEquals("bgo", response.selectors().get(0).id()); + Assertions.assertEquals("yiopi", response.location()); + Assertions.assertEquals("qfkuvscxkdmli", response.tags().get("ovibrxkp")); + Assertions.assertEquals(ResourceIdentityType.USER_ASSIGNED, response.identity().type()); + Assertions.assertEquals("scjfnyns", response.steps().get(0).name()); + Assertions.assertEquals("ujiz", response.steps().get(0).branches().get(0).name()); + Assertions.assertEquals("owzfttsttkt", response.selectors().get(0).id()); } // Use "Map.of" if available diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsWithResponseMockTests.java index 88bd45ed00e0e..64bb13a10cd3c 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsExecutionDetailsWithResponseMockTests.java @@ -29,40 +29,27 @@ public void testExecutionDetailsWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"type\":\"eclze\",\"id\":\"bcvhzlhpl\",\"name\":\"qkdlw\",\"properties\":{\"failureReason\":\"bum\",\"lastActionAt\":\"2021-01-09T13:22:40Z\",\"runInformation\":{\"steps\":[{\"stepName\":\"fsmlmbtxhwgfw\",\"stepId\":\"tawc\",\"status\":\"zbrhubskhudyg\",\"branches\":[{},{},{}]},{\"stepName\":\"kqfqjbvl\",\"stepId\":\"rfmluiqtq\",\"status\":\"avyvnqqyba\",\"branches\":[{},{},{},{}]},{\"stepName\":\"ayjkqa\",\"stepId\":\"gzslesjcbhernnti\",\"status\":\"djc\",\"branches\":[{},{},{}]}]},\"status\":\"wr\",\"startedAt\":\"2021-03-14T01:37:10Z\",\"stoppedAt\":\"2021-08-03T09:19:04Z\"}}"; + String responseStr + = "{\"type\":\"qkdlw\",\"id\":\"fbumlkx\",\"name\":\"qjfsmlmbtxhw\",\"properties\":{\"failureReason\":\"srtawcoezbr\",\"lastActionAt\":\"2021-02-21T07:03:26Z\",\"runInformation\":{\"steps\":[{\"stepName\":\"dyg\",\"stepId\":\"okkqfqjbvleo\",\"status\":\"ml\",\"branches\":[{},{},{}]}]},\"status\":\"qzfavyv\",\"startedAt\":\"2021-03-05T04:35Z\",\"stoppedAt\":\"2021-05-23T06:21:06Z\"}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + ExperimentExecutionDetails response = manager.experiments() + .executionDetailsWithResponse("stgktst", "dxeclzedqbcvh", "lhpl", com.azure.core.util.Context.NONE) + .getValue(); - ExperimentExecutionDetails response = - manager - .experiments() - .executionDetailsWithResponse( - "wrck", "yklyhpluodpvruud", "gzibthostgktstv", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupWithResponseMockTests.java index aff8965c5c4d2..213b95e71cae9 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetByResourceGroupWithResponseMockTests.java @@ -31,46 +31,32 @@ public void testGetByResourceGroupWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"ssxkkgthr\":{\"principalId\":\"27e8664d-1a55-442e-bd7a-eca97866fd5c\",\"clientId\":\"a2befb8a-4066-4897-a8ee-13fc8280fb3c\"},\"xjb\":{\"principalId\":\"7b4d97a8-974b-40f3-a098-31ee065cdd01\",\"clientId\":\"1058d915-f56e-4840-8e9e-27b22db6cf3a\"},\"xvcxgfrpdsofbshr\":{\"principalId\":\"7cd92f3c-757b-4ffc-9146-2a180e9ab0ba\",\"clientId\":\"c3f02a99-3408-4c2c-bbcd-72323665917a\"}},\"principalId\":\"vbuswd\",\"tenantId\":\"yybyc\"},\"properties\":{\"provisioningState\":\"Canceled\",\"steps\":[{\"name\":\"jsrtk\",\"branches\":[{\"name\":\"wnopqgikyzirtx\",\"actions\":[]},{\"name\":\"yuxzejntpsewgi\",\"actions\":[]},{\"name\":\"ilqu\",\"actions\":[]},{\"name\":\"rydxtqm\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"ox\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"avgrvkffovjz\":\"dataufhyaomtbgh\",\"gjmfxumvfcl\":\"datapjbi\",\"wxnb\":\"datayo\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"fezzxscyhwzdg\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"yujviylwdshfssn\":\"databzbomvzzbtdcqvpn\",\"rymsgaojfmw\":\"databgye\",\"hirctymoxoftpipi\":\"datacotmr\",\"lihhyuspskasdvlm\":\"datayczuhxacpq\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"wdgzxulucv\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ytxifqjzgxmrh\":\"datasreuzvxurisjnh\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"blwpcesutrgj\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"pnfqntcyp\":\"datautpwoqhihejqgw\",\"rcizjxvyd\":\"dataxjvfoimwksl\",\"t\":\"dataceacvlhvygdy\"}}]},\"location\":\"mrtwna\",\"tags\":{\"ojgcyzt\":\"lbiw\"},\"id\":\"fmznba\",\"name\":\"qphchqnrnrpxehuw\",\"type\":\"ykqgaifmvik\"}"; + String responseStr + = "{\"identity\":{\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"opqgikyzirtxdyux\":{\"principalId\":\"ed9e294c-ba78-4a01-9758-19761ba750b0\",\"clientId\":\"a2543718-8588-4dfb-8eaf-4fea9e1ab6b7\"},\"ntps\":{\"principalId\":\"a1f707cd-b841-41f1-bc3c-af8593d08837\",\"clientId\":\"6f1bac67-d21a-4434-a85e-68a7413d937d\"},\"ioilqukrydxtq\":{\"principalId\":\"a97a125d-157d-4bae-a88a-69b9f437a6fc\",\"clientId\":\"1d2845f8-8e7b-4f78-a4d7-47a82dc40e67\"},\"ox\":{\"principalId\":\"c0b79486-4535-400a-af65-889b887410a8\",\"clientId\":\"2ed7a83d-db34-4f38-9790-6877bd60d579\"}},\"principalId\":\"ggufhyaomtb\",\"tenantId\":\"havgrvk\"},\"properties\":{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"jzhpjbibgjmfx\",\"branches\":[{\"name\":\"vfcluyovwxnbkfe\",\"actions\":[]}]},{\"name\":\"zxscyhwzdgirujb\",\"branches\":[{\"name\":\"omvzzbtd\",\"actions\":[]},{\"name\":\"qvpn\",\"actions\":[]},{\"name\":\"yujviylwdshfssn\",\"actions\":[]}]},{\"name\":\"bgye\",\"branches\":[{\"name\":\"ymsgaojfmwnc\",\"actions\":[]},{\"name\":\"tmr\",\"actions\":[]},{\"name\":\"hirctymoxoftpipi\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"czuhxacpqjlihh\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ucvpamrs\":\"dataskasdvlmfwdgzxu\",\"risjnhnytxifqjz\":\"dataeuzvx\",\"lw\":\"dataxmrhu\",\"woqhihe\":\"datacesutrgjupauut\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"qg\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"xjvfoimwksl\":\"datafqntcyp\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"rcizjxvyd\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"gdyftumrtwna\":\"datacvlhv\",\"wkojgcyztsfmzn\":\"datajslb\"}}]},\"location\":\"aeqphchqnr\",\"tags\":{\"huwrykqgaifm\":\"x\",\"jdz\":\"iklbydvkhb\",\"srhnjivo\":\"xcv\"},\"id\":\"v\",\"name\":\"novqfzge\",\"type\":\"jdftuljltd\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Experiment response = - manager - .experiments() - .getByResourceGroupWithResponse("tw", "sgogczhonnxk", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("mrtwna", response.location()); - Assertions.assertEquals("lbiw", response.tags().get("ojgcyzt")); - Assertions.assertEquals(ResourceIdentityType.NONE, response.identity().type()); - Assertions.assertEquals("jsrtk", response.steps().get(0).name()); - Assertions.assertEquals("wnopqgikyzirtx", response.steps().get(0).branches().get(0).name()); - Assertions.assertEquals("ox", response.selectors().get(0).id()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Experiment response = manager.experiments() + .getByResourceGroupWithResponse("z", "ybycnunvj", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("aeqphchqnr", response.location()); + Assertions.assertEquals("x", response.tags().get("huwrykqgaifm")); + Assertions.assertEquals(ResourceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); + Assertions.assertEquals("jzhpjbibgjmfx", response.steps().get(0).name()); + Assertions.assertEquals("vfcluyovwxnbkfe", response.steps().get(0).branches().get(0).name()); + Assertions.assertEquals("czuhxacpqjlihh", response.selectors().get(0).id()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionWithResponseMockTests.java index 24a1640fbfff1..e65f713632da2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsGetExecutionWithResponseMockTests.java @@ -29,39 +29,27 @@ public void testGetExecutionWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"type\":\"wcfzqljyxgt\",\"id\":\"heyd\",\"name\":\"dshmkxmaehvbbx\",\"properties\":{\"status\":\"pltfnhtba\",\"startedAt\":\"2021-07-30T22:05:34Z\",\"stoppedAt\":\"2021-11-06T21:21:21Z\"}}"; + String responseStr + = "{\"type\":\"ripltf\",\"id\":\"tbaxk\",\"name\":\"ywrckp\",\"properties\":{\"status\":\"yhpluodpvruudlgz\",\"startedAt\":\"2021-07-04T20:01:35Z\",\"stoppedAt\":\"2021-04-27T04:41:25Z\"}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + ExperimentExecution response = manager.experiments() + .getExecutionWithResponse("zqljyxgtczh", "ydbsd", "hmkxmaehvbb", com.azure.core.util.Context.NONE) + .getValue(); - ExperimentExecution response = - manager - .experiments() - .getExecutionWithResponse("w", "qioknssxmojm", "vpkjpr", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsMockTests.java index 97adc679f4751..07808338511b2 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListAllExecutionsMockTests.java @@ -30,36 +30,26 @@ public void testListAllExecutions() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"type\":\"rhnj\",\"id\":\"olvtnovqfzge\",\"name\":\"dftuljltduce\",\"properties\":{\"status\":\"mczuo\",\"startedAt\":\"2021-10-02T15:37:14Z\",\"stoppedAt\":\"2021-01-27T20:06:14Z\"}}]}"; + String responseStr + = "{\"value\":[{\"type\":\"jw\",\"id\":\"wqiok\",\"name\":\"sx\",\"properties\":{\"status\":\"msvpkjpr\",\"startedAt\":\"2021-06-29T04:33:46Z\",\"stoppedAt\":\"2021-06-26T08:15:06Z\"}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.experiments().listAllExecutions("ceamtm", "zuo", com.azure.core.util.Context.NONE); - PagedIterable response = - manager.experiments().listAllExecutions("bydvkhbejdz", "xcv", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupMockTests.java index ea1b65d06e754..7ebe66262142b 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListByResourceGroupMockTests.java @@ -32,45 +32,32 @@ public void testListByResourceGroup() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"azpxdtnkdmkqjjl\":{\"principalId\":\"d3811f85-895d-4f46-8859-2b0865e18fa5\",\"clientId\":\"8d0cdc82-fea6-4658-9bca-584d67b7dfd3\"},\"nvrk\":{\"principalId\":\"ec49a718-d51c-4e71-9106-7f7fed1cc9a0\",\"clientId\":\"a5740e1d-13d0-40cf-8b7c-7a74babb059c\"},\"uaibrebqaaysj\":{\"principalId\":\"f3d93ac3-ea28-4d91-984a-26ab3c46ecc7\",\"clientId\":\"5cf08346-017d-4041-be9d-7d2d19f2286f\"},\"qtnqtt\":{\"principalId\":\"38ad80f7-babd-43ac-bfec-5774032a538f\",\"clientId\":\"1acda986-2887-48ef-a438-2307d03e4195\"}},\"principalId\":\"lwfffi\",\"tenantId\":\"pjpqqmtedltmmji\"},\"properties\":{\"provisioningState\":\"Updating\",\"steps\":[{\"name\":\"zphv\",\"branches\":[{\"name\":\"uyqncygupkvipmd\",\"actions\":[]},{\"name\":\"cwxqu\",\"actions\":[]},{\"name\":\"evzhfsto\",\"actions\":[]}]},{\"name\":\"xhojuj\",\"branches\":[{\"name\":\"pelmcuvhixbjxyf\",\"actions\":[]},{\"name\":\"n\",\"actions\":[]}]},{\"name\":\"lrcoolsttpki\",\"branches\":[{\"name\":\"kbnujr\",\"actions\":[]},{\"name\":\"wvtylbfpncurdo\",\"actions\":[]}]},{\"name\":\"wiithtywub\",\"branches\":[{\"name\":\"bihwqknfdnt\",\"actions\":[]},{\"name\":\"jchrdgoihxumw\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"ond\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"tsbwtovvtgse\":\"dataudfdlwgg\"}}]},\"location\":\"nqfiufxqknpi\",\"tags\":{\"dmqnrojlpij\":\"epttwqmsniff\",\"xfrdd\":\"k\",\"atiz\":\"c\"},\"id\":\"ronasxift\",\"name\":\"zq\",\"type\":\"zh\"}]}"; + String responseStr + = "{\"value\":[{\"identity\":{\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"qmt\":{\"principalId\":\"09cc1a83-7967-4a00-9444-c8273185c704\",\"clientId\":\"7dbf2abf-1c99-4e0d-a363-474476929b40\"},\"tmmjihyeozph\":{\"principalId\":\"80362830-f6b2-45e6-b2e9-67f951c7e6a6\",\"clientId\":\"ed3d11fa-d978-4743-a66f-061725d1cf92\"}},\"principalId\":\"auyqncygupkv\",\"tenantId\":\"mdscwxqupev\"},\"properties\":{\"provisioningState\":\"Deleting\",\"steps\":[{\"name\":\"totxhojujb\",\"branches\":[{\"name\":\"elmcuvhixbjxyfw\",\"actions\":[]},{\"name\":\"yl\",\"actions\":[]},{\"name\":\"coolsttpkiwkkb\",\"actions\":[]},{\"name\":\"ujrywvtyl\",\"actions\":[]}]},{\"name\":\"fpncurdo\",\"branches\":[{\"name\":\"iithtywu\",\"actions\":[]},{\"name\":\"xcbihw\",\"actions\":[]},{\"name\":\"knfd\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"wjchrdg\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"luudfdlwggytsb\":\"dataumwctondz\"}},{\"type\":\"ChaosTargetSelector\",\"id\":\"tov\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"ufxqknpirgnepttw\":\"dataeinqf\",\"mqnrojlpijnkr\":\"datamsniffc\",\"zronasxift\":\"datafrddhcrati\"}}]},\"location\":\"zq\",\"tags\":{\"wesgogczh\":\"f\",\"lgnyhmo\":\"nnxk\",\"h\":\"sxkkg\"},\"id\":\"rghxjb\",\"name\":\"hqxvcxgfrpdsofbs\",\"type\":\"rnsvbuswd\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .experiments() - .listByResourceGroup("njwmwkpnbsazejj", true, "kagfhsxtt", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("nqfiufxqknpi", response.iterator().next().location()); - Assertions.assertEquals("epttwqmsniff", response.iterator().next().tags().get("dmqnrojlpij")); - Assertions.assertEquals(ResourceIdentityType.USER_ASSIGNED, response.iterator().next().identity().type()); - Assertions.assertEquals("zphv", response.iterator().next().steps().get(0).name()); - Assertions.assertEquals("uyqncygupkvipmd", response.iterator().next().steps().get(0).branches().get(0).name()); - Assertions.assertEquals("ond", response.iterator().next().selectors().get(0).id()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.experiments().listByResourceGroup("ixqtn", false, "tezlwff", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("zq", response.iterator().next().location()); + Assertions.assertEquals("f", response.iterator().next().tags().get("wesgogczh")); + Assertions.assertEquals(ResourceIdentityType.SYSTEM_ASSIGNED, response.iterator().next().identity().type()); + Assertions.assertEquals("totxhojujb", response.iterator().next().steps().get(0).name()); + Assertions.assertEquals("elmcuvhixbjxyfw", response.iterator().next().steps().get(0).branches().get(0).name()); + Assertions.assertEquals("wjchrdg", response.iterator().next().selectors().get(0).id()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListMockTests.java index 2ff75593f1cf9..1b203f8577b92 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ExperimentsListMockTests.java @@ -32,43 +32,32 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"identity\":{\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"qwkyhkobop\":{\"principalId\":\"9afaec88-e2cb-4e51-8957-3ac88c1a8455\",\"clientId\":\"57e284ba-2b78-4605-9676-7005b0d63933\"},\"dkow\":{\"principalId\":\"4f41ab29-13a3-4ed0-bc0f-ef5f6f9a59f3\",\"clientId\":\"b83fdc62-696d-4f02-9fa4-53ac6555da27\"}},\"principalId\":\"bqpc\",\"tenantId\":\"kbwcc\"},\"properties\":{\"provisioningState\":\"Deleting\",\"steps\":[{\"name\":\"cdwxlpq\",\"branches\":[{\"name\":\"ftnkhtj\",\"actions\":[]},{\"name\":\"y\",\"actions\":[]},{\"name\":\"ngwfqatm\",\"actions\":[]}]},{\"name\":\"dhtmdvypgikd\",\"branches\":[{\"name\":\"zywkb\",\"actions\":[]},{\"name\":\"rryuzhlhkjo\",\"actions\":[]},{\"name\":\"rvqqaatj\",\"actions\":[]}]},{\"name\":\"nrvgoupmfiibfgg\",\"branches\":[{\"name\":\"ool\",\"actions\":[]},{\"name\":\"rwxkvtkkgl\",\"actions\":[]},{\"name\":\"qwjygvja\",\"actions\":[]},{\"name\":\"vblm\",\"actions\":[]}]},{\"name\":\"vkzuhbxvvyhgso\",\"branches\":[{\"name\":\"yrqufegxuvwz\",\"actions\":[]},{\"name\":\"bnhlmc\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"p\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"jejveg\":\"datatvgbmhrixkwmy\",\"eaxhcexdrrvqahqk\":\"datahbpnaixexccbd\",\"hyjsvfycx\":\"datahtpwij\",\"t\":\"databfvoowvrv\"}}]},\"location\":\"jqppyostronzmy\",\"tags\":{\"xkmcwaekrrjre\":\"ipn\",\"jglikkxwslolb\":\"fxtsgum\"},\"id\":\"pvuzlmv\",\"name\":\"elfk\",\"type\":\"gplcrpwjxeznoigb\"}]}"; + String responseStr + = "{\"value\":[{\"identity\":{\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"ixkwmyijejveg\":{\"principalId\":\"cc1b7169-3ea9-4e3e-8e9c-9da325fbd6d6\",\"clientId\":\"283a5759-690d-4a2f-81b0-cf54f7f0038c\"},\"pna\":{\"principalId\":\"19de67e8-5fb5-4422-bcf4-f04b0ff13c34\",\"clientId\":\"f470ab26-71a1-4133-8067-4882dab06c7e\"},\"xccbdreaxhcex\":{\"principalId\":\"8f92162f-760d-47a9-9ab6-ffa73bfe34f0\",\"clientId\":\"f3d81987-4331-499d-9ad6-a7ab24671ed0\"},\"vqahqkghtpwi\":{\"principalId\":\"7c4bf370-3cff-4a7e-bfb7-df7ec7a5b8b9\",\"clientId\":\"58a5a2a5-a4c4-4336-a6d0-df2207854d28\"}},\"principalId\":\"hyjsvfycx\",\"tenantId\":\"fvoow\"},\"properties\":{\"provisioningState\":\"Creating\",\"steps\":[{\"name\":\"t\",\"branches\":[{\"name\":\"qp\",\"actions\":[]},{\"name\":\"y\",\"actions\":[]},{\"name\":\"s\",\"actions\":[]},{\"name\":\"ronzmyhgfip\",\"actions\":[]}]}],\"selectors\":[{\"type\":\"ChaosTargetSelector\",\"id\":\"xkmcwaekrrjre\",\"filter\":{\"type\":\"ChaosTargetFilter\"},\"\":{\"lbqpvuzlmvfelf\":\"datasgumhjglikkxwsl\",\"crpw\":\"datatgp\"}}]},\"location\":\"xeznoi\",\"tags\":{\"qkagfhsxt\":\"njwmwkpnbsazejj\",\"dm\":\"augzxnfaazpxdtn\"},\"id\":\"qjjlwuen\",\"name\":\"rkpyouaibrebqaay\",\"type\":\"j\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.experiments().list(false, "elvezrypq", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.experiments().list(false, "nhlmctlpdng", com.azure.core.util.Context.NONE); - Assertions.assertEquals("jqppyostronzmy", response.iterator().next().location()); - Assertions.assertEquals("ipn", response.iterator().next().tags().get("xkmcwaekrrjre")); + Assertions.assertEquals("xeznoi", response.iterator().next().location()); + Assertions.assertEquals("njwmwkpnbsazejj", response.iterator().next().tags().get("qkagfhsxt")); Assertions.assertEquals(ResourceIdentityType.SYSTEM_ASSIGNED, response.iterator().next().identity().type()); - Assertions.assertEquals("cdwxlpq", response.iterator().next().steps().get(0).name()); - Assertions.assertEquals("ftnkhtj", response.iterator().next().steps().get(0).branches().get(0).name()); - Assertions.assertEquals("p", response.iterator().next().selectors().get(0).id()); + Assertions.assertEquals("t", response.iterator().next().steps().get(0).name()); + Assertions.assertEquals("qp", response.iterator().next().steps().get(0).branches().get(0).name()); + Assertions.assertEquals("xkmcwaekrrjre", response.iterator().next().selectors().get(0).id()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationDisplayTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationDisplayTests.java index 5223d22a1ff91..75d0dc13da38c 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationDisplayTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationDisplayTests.java @@ -10,11 +10,10 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"d\",\"resource\":\"ygdvwv\",\"operation\":\"iohgwxrtfud\",\"description\":\"pxgy\"}") - .toObject(OperationDisplay.class); + OperationDisplay model = BinaryData + .fromString( + "{\"provider\":\"k\",\"resource\":\"umiekkezzi\",\"operation\":\"ly\",\"description\":\"hdgqggeb\"}") + .toObject(OperationDisplay.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationInnerTests.java index f913ee8cc5a99..266bd595cc873 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationInnerTests.java @@ -11,11 +11,9 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"gxk\",\"isDataAction\":false,\"display\":{\"provider\":\"melwuipiccjz\",\"resource\":\"ivgvvcna\",\"operation\":\"hyrnxxmu\",\"description\":\"dndrdvstkwqqtche\"},\"origin\":\"user\",\"actionType\":\"Internal\"}") - .toObject(OperationInner.class); + OperationInner model = BinaryData.fromString( + "{\"name\":\"bfatklddxbjhwu\",\"isDataAction\":false,\"display\":{\"provider\":\"jos\",\"resource\":\"youlp\",\"operation\":\"v\",\"description\":\"glrvimjwosytxi\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") + .toObject(OperationInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationListResultTests.java index 0b2bfd880ad94..11837150b953f 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationListResultTests.java @@ -10,11 +10,9 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"d\",\"isDataAction\":false,\"display\":{\"provider\":\"ocogj\",\"resource\":\"dtbnnha\",\"operation\":\"ocrkvcikh\",\"description\":\"p\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"queziky\"}") - .toObject(OperationListResult.class); + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"z\",\"isDataAction\":true,\"display\":{\"provider\":\"c\",\"resource\":\"y\",\"operation\":\"yrnxxmueedn\",\"description\":\"dvstkw\"},\"origin\":\"user\",\"actionType\":\"Internal\"},{\"name\":\"alm\",\"isDataAction\":true,\"display\":{\"provider\":\"aygdvwvgpioh\",\"resource\":\"xrtfudxep\",\"operation\":\"yqagvrvm\",\"description\":\"k\"},\"origin\":\"user\",\"actionType\":\"Internal\"},{\"name\":\"dblx\",\"isDataAction\":true,\"display\":{\"provider\":\"njhf\",\"resource\":\"wmszkk\",\"operation\":\"qreyfkzi\",\"description\":\"jawneaiv\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"pcirelsfeaen\"}") + .toObject(OperationListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusInnerTests.java index fffe94a447c50..848054746acde 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusInnerTests.java @@ -11,32 +11,25 @@ public final class OperationStatusInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationStatusInner model = - BinaryData - .fromString( - "{\"id\":\"tsmypyynpcdp\",\"name\":\"nzgmwznmabik\",\"startTime\":\"orgjhxbldt\",\"endTime\":\"wrlkdmtn\",\"status\":\"ok\"}") - .toObject(OperationStatusInner.class); - Assertions.assertEquals("tsmypyynpcdp", model.id()); - Assertions.assertEquals("nzgmwznmabik", model.name()); - Assertions.assertEquals("orgjhxbldt", model.startTime()); - Assertions.assertEquals("wrlkdmtn", model.endTime()); - Assertions.assertEquals("ok", model.status()); + OperationStatusInner model = BinaryData.fromString( + "{\"id\":\"ocrkvcikh\",\"name\":\"p\",\"startTime\":\"qgxqquezikyw\",\"endTime\":\"xkalla\",\"status\":\"elwuipi\"}") + .toObject(OperationStatusInner.class); + Assertions.assertEquals("ocrkvcikh", model.id()); + Assertions.assertEquals("p", model.name()); + Assertions.assertEquals("qgxqquezikyw", model.startTime()); + Assertions.assertEquals("xkalla", model.endTime()); + Assertions.assertEquals("elwuipi", model.status()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationStatusInner model = - new OperationStatusInner() - .withId("tsmypyynpcdp") - .withName("nzgmwznmabik") - .withStartTime("orgjhxbldt") - .withEndTime("wrlkdmtn") - .withStatus("ok"); + OperationStatusInner model = new OperationStatusInner().withId("ocrkvcikh").withName("p") + .withStartTime("qgxqquezikyw").withEndTime("xkalla").withStatus("elwuipi"); model = BinaryData.fromObject(model).toObject(OperationStatusInner.class); - Assertions.assertEquals("tsmypyynpcdp", model.id()); - Assertions.assertEquals("nzgmwznmabik", model.name()); - Assertions.assertEquals("orgjhxbldt", model.startTime()); - Assertions.assertEquals("wrlkdmtn", model.endTime()); - Assertions.assertEquals("ok", model.status()); + Assertions.assertEquals("ocrkvcikh", model.id()); + Assertions.assertEquals("p", model.name()); + Assertions.assertEquals("qgxqquezikyw", model.startTime()); + Assertions.assertEquals("xkalla", model.endTime()); + Assertions.assertEquals("elwuipi", model.status()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetWithResponseMockTests.java index 1ae947bc5fc43..5a108f0be9d44 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/OperationStatusesGetWithResponseMockTests.java @@ -30,45 +30,31 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"idfcxsspuunnoxyh\",\"name\":\"g\",\"startTime\":\"drihpfhoqcaaewda\",\"endTime\":\"djvlpj\",\"status\":\"kzbrmsgeivsiy\"}"; + String responseStr + = "{\"id\":\"xccybvpa\",\"name\":\"kkudzp\",\"startTime\":\"wjplma\",\"endTime\":\"tcyohpfkyrk\",\"status\":\"dg\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - OperationStatus response = - manager - .operationStatuses() - .getWithResponse("uqypfcvle", "chpqbmfpjba", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("idfcxsspuunnoxyh", response.id()); - Assertions.assertEquals("g", response.name()); - Assertions.assertEquals("drihpfhoqcaaewda", response.startTime()); - Assertions.assertEquals("djvlpj", response.endTime()); - Assertions.assertEquals("kzbrmsgeivsiy", response.status()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + OperationStatus response = manager.operationStatuses() + .getWithResponse("edabgyvudtjue", "bcihxuuwhc", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("xccybvpa", response.id()); + Assertions.assertEquals("kkudzp", response.name()); + Assertions.assertEquals("wjplma", response.startTime()); + Assertions.assertEquals("tcyohpfkyrk", response.endTime()); + Assertions.assertEquals("dg", response.status()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ResourceIdentityTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ResourceIdentityTests.java index 403c5b8d79c2f..9a55512823f94 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ResourceIdentityTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/ResourceIdentityTests.java @@ -15,27 +15,17 @@ public final class ResourceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ResourceIdentity model = - BinaryData - .fromString( - "{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"zabglcuhxwt\":{\"principalId\":\"636e32c1-82cf-4032-bd14-59f4ad3d0032\",\"clientId\":\"3ee8071b-e657-4e3d-831a-4027e711a01e\"},\"qik\":{\"principalId\":\"ab6859fb-fab8-4827-96a8-801d4dda4ca9\",\"clientId\":\"f450bb8c-9e9e-431f-958e-f49dfe1d5a5f\"},\"ovplw\":{\"principalId\":\"28a08b74-30ee-4366-891c-399d804b0e21\",\"clientId\":\"2613cf04-8f03-4160-978d-493ebae52a11\"}},\"principalId\":\"hvgyuguosvmk\",\"tenantId\":\"sxqu\"}") - .toObject(ResourceIdentity.class); + ResourceIdentity model = BinaryData.fromString( + "{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"zabglcuhxwt\":{\"principalId\":\"b6530509-94c7-4e0d-95ac-1e4e78448f71\",\"clientId\":\"c48be21d-ce83-40e4-a49c-ae0060c973e9\"},\"qik\":{\"principalId\":\"64799503-0174-4da8-9b23-9124b8c6506d\",\"clientId\":\"23a7338f-5f5b-4819-9023-acec91c1bd3f\"},\"ovplw\":{\"principalId\":\"6826292f-a310-47eb-8590-89dd0506f502\",\"clientId\":\"3146ab84-69f6-46e1-af4c-2125972011a4\"}},\"principalId\":\"hvgyuguosvmk\",\"tenantId\":\"sxqu\"}") + .toObject(ResourceIdentity.class); Assertions.assertEquals(ResourceIdentityType.USER_ASSIGNED, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ResourceIdentity model = - new ResourceIdentity() - .withType(ResourceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "zabglcuhxwt", - new UserAssignedIdentity(), - "qik", - new UserAssignedIdentity(), - "ovplw", - new UserAssignedIdentity())); + ResourceIdentity model = new ResourceIdentity().withType(ResourceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("zabglcuhxwt", new UserAssignedIdentity(), "qik", + new UserAssignedIdentity(), "ovplw", new UserAssignedIdentity())); model = BinaryData.fromObject(model).toObject(ResourceIdentity.class); Assertions.assertEquals(ResourceIdentityType.USER_ASSIGNED, model.type()); } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/StepStatusTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/StepStatusTests.java index aa65a797c4ae8..1b9539c392814 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/StepStatusTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/StepStatusTests.java @@ -10,11 +10,9 @@ public final class StepStatusTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - StepStatus model = - BinaryData - .fromString( - "{\"stepName\":\"efuzmuvpbttd\",\"stepId\":\"orppxebmnzbtb\",\"status\":\"pglkf\",\"branches\":[{\"branchName\":\"neuelfphsdyhtoz\",\"branchId\":\"kd\",\"status\":\"wq\",\"actions\":[{\"actionName\":\"zx\",\"actionId\":\"vithh\",\"status\":\"o\",\"startTime\":\"2021-11-01T18:47:46Z\",\"endTime\":\"2021-03-28T21:26:02Z\",\"targets\":[{},{},{}]}]},{\"branchName\":\"ohfwds\",\"branchId\":\"ka\",\"status\":\"utiiswacf\",\"actions\":[{\"actionName\":\"zzewkfvhqcrai\",\"actionId\":\"pnppfuf\",\"status\":\"wdmhdlxyjrxs\",\"startTime\":\"2021-10-27T14:22:38Z\",\"endTime\":\"2020-12-27T20:37:24Z\",\"targets\":[{}]},{\"actionName\":\"wqapnedgfbcvk\",\"actionId\":\"q\",\"status\":\"keqdcvdrhvoods\",\"startTime\":\"2021-11-02T01:53:07Z\",\"endTime\":\"2021-03-29T00:37:52Z\",\"targets\":[{},{},{}]},{\"actionName\":\"pcjwv\",\"actionId\":\"dldwmgxc\",\"status\":\"slpmutwuo\",\"startTime\":\"2021-07-03T15:36:28Z\",\"endTime\":\"2021-08-11T07:57:19Z\",\"targets\":[{}]},{\"actionName\":\"niyqslui\",\"actionId\":\"dggkzzlvmbmpa\",\"status\":\"odfvuefywsbp\",\"startTime\":\"2021-09-23T19:46:07Z\",\"endTime\":\"2021-11-18T13:44:32Z\",\"targets\":[{},{},{},{}]}]},{\"branchName\":\"ouyftaakc\",\"branchId\":\"iyzvqtmnub\",\"status\":\"kpzksmondjmq\",\"actions\":[{\"actionName\":\"pomgkopkwhojvp\",\"actionId\":\"qgxy\",\"status\":\"ocmbqfqvmkcxoza\",\"startTime\":\"2021-08-25T20:55:15Z\",\"endTime\":\"2021-06-19T06:02:18Z\",\"targets\":[{},{}]},{\"actionName\":\"glyatddckcbcuej\",\"actionId\":\"xgc\",\"status\":\"ibrhosxsdqr\",\"startTime\":\"2021-07-06T01:32:33Z\",\"endTime\":\"2021-06-02T05:04:16Z\",\"targets\":[{},{},{}]},{\"actionName\":\"rq\",\"actionId\":\"bahwfl\",\"status\":\"zdtmhrkwofy\",\"startTime\":\"2021-02-28T16:21:21Z\",\"endTime\":\"2021-09-15T04:25:08Z\",\"targets\":[{},{},{}]},{\"actionName\":\"expbtg\",\"actionId\":\"bwoenwashrt\",\"status\":\"kcnqxwbpo\",\"startTime\":\"2021-09-01T09:08:06Z\",\"endTime\":\"2021-07-21T21:11:01Z\",\"targets\":[{}]}]}]}") - .toObject(StepStatus.class); + StepStatus model = BinaryData.fromString( + "{\"stepName\":\"luicpdggkzzlvmbm\",\"stepId\":\"xmodf\",\"status\":\"efyw\",\"branches\":[{\"branchName\":\"vmwy\",\"branchId\":\"fouyf\",\"status\":\"akcp\",\"actions\":[{\"actionName\":\"vqtmnub\",\"actionId\":\"kpzksmondjmq\",\"status\":\"vypomgkopkwho\",\"startTime\":\"2021-08-09T17:57:45Z\",\"endTime\":\"2021-12-07T09:07:14Z\",\"targets\":[{},{},{},{}]},{\"actionName\":\"ysmocmbqfqvmkcxo\",\"actionId\":\"pvhelxprg\",\"status\":\"atddc\",\"startTime\":\"2021-09-13T16:27:08Z\",\"endTime\":\"2021-12-04T18:39:42Z\",\"targets\":[{},{},{},{}]}]},{\"branchName\":\"jxgciqibrh\",\"branchId\":\"xsdqrhzoymibmrqy\",\"status\":\"ahwfluszdtmhrk\",\"actions\":[{\"actionName\":\"yvoqa\",\"actionId\":\"iexpbtgiwbwo\",\"status\":\"washr\",\"startTime\":\"2021-08-01T13:56:15Z\",\"endTime\":\"2021-10-17T07:53:24Z\",\"targets\":[{},{},{}]},{\"actionName\":\"wbpokulpiujwaasi\",\"actionId\":\"i\",\"status\":\"byuqerpqlp\",\"startTime\":\"2021-10-05T15:21:02Z\",\"endTime\":\"2020-12-29T00:03:32Z\",\"targets\":[{},{}]}]},{\"branchName\":\"bdbutauvf\",\"branchId\":\"kuwhh\",\"status\":\"ykojoxafnndlpic\",\"actions\":[{\"actionName\":\"mkcdyhbpkkpwdre\",\"actionId\":\"ovvqfovljxywsu\",\"status\":\"yrs\",\"startTime\":\"2021-03-16T15:34:55Z\",\"endTime\":\"2021-05-20T04:26:58Z\",\"targets\":[{},{},{}]}]}]}") + .toObject(StepStatus.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetInnerTests.java index 2434a88591a6c..5b2a5d7669efa 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetInnerTests.java @@ -13,23 +13,18 @@ public final class TargetInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetInner model = - BinaryData - .fromString( - "{\"location\":\"xrbuukzclew\",\"properties\":{\"pofncck\":\"datamlwpazt\",\"qa\":\"datayfzqwhxxbu\",\"lxorjaltolmncws\":\"datazfeqztppri\"},\"id\":\"bqwcsdbnwdcf\",\"name\":\"ucqdpfuvglsb\",\"type\":\"jcanvxbvtvudut\"}") - .toObject(TargetInner.class); - Assertions.assertEquals("xrbuukzclew", model.location()); + TargetInner model = BinaryData.fromString( + "{\"location\":\"knnqvsaznq\",\"properties\":{\"mkycgra\":\"dataorudsgsa\"},\"id\":\"wjue\",\"name\":\"aeburuvdmo\",\"type\":\"s\"}") + .toObject(TargetInner.class); + Assertions.assertEquals("knnqvsaznq", model.location()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TargetInner model = - new TargetInner() - .withLocation("xrbuukzclew") - .withProperties( - mapOf("pofncck", "datamlwpazt", "qa", "datayfzqwhxxbu", "lxorjaltolmncws", "datazfeqztppri")); + TargetInner model + = new TargetInner().withLocation("knnqvsaznq").withProperties(mapOf("mkycgra", "dataorudsgsa")); model = BinaryData.fromObject(model).toObject(TargetInner.class); - Assertions.assertEquals("xrbuukzclew", model.location()); + Assertions.assertEquals("knnqvsaznq", model.location()); } // Use "Map.of" if available diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetListResultTests.java index bd6e3ec46e3b1..503cdf7830640 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetListResultTests.java @@ -10,11 +10,9 @@ public final class TargetListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetListResult model = - BinaryData - .fromString( - "{\"value\":[{\"location\":\"dgssofwqmzqal\",\"properties\":{\"byxbaaabjy\":\"datamnjijpxacqqudf\",\"zrtuzq\":\"dataayffim\"},\"id\":\"gsexne\",\"name\":\"fdnw\",\"type\":\"wmewzsyy\"},{\"location\":\"zsoibjudpfrxtr\",\"properties\":{\"ubpaxhe\":\"datazvaytdwkqbr\"},\"id\":\"iilivpdtiirqtd\",\"name\":\"oaxoruzfgsqu\",\"type\":\"fxrxxle\"},{\"location\":\"amxjezwlw\",\"properties\":{\"ooaojkniodkooebw\":\"dataxuqlcvydypat\",\"infwjlfltkacjve\":\"dataujhemmsbvdkcrodt\",\"ggkfpagaowpul\":\"datakdlfoa\",\"sjervti\":\"dataqblylsyxkqj\"},\"id\":\"gxsds\",\"name\":\"uem\",\"type\":\"sbzkf\"}],\"nextLink\":\"eyvpnqicvinvkj\"}") - .toObject(TargetListResult.class); + TargetListResult model = BinaryData.fromString( + "{\"value\":[{\"location\":\"nuj\",\"properties\":{\"wj\":\"datammsbvdkcrodtjin\",\"f\":\"datafltkacjv\",\"gaowpulpqblylsyx\":\"datadlfoakggkfp\"},\"id\":\"qjnsjervtia\",\"name\":\"xsdszuempsb\",\"type\":\"kfzbeyvpnqicvi\"},{\"location\":\"jjxd\",\"properties\":{\"paztzpofncck\":\"databuukzclewyhml\",\"qa\":\"datayfzqwhxxbu\",\"lxorjaltolmncws\":\"datazfeqztppri\",\"ucqdpfuvglsb\":\"databqwcsdbnwdcf\"},\"id\":\"jcanvxbvtvudut\",\"name\":\"cormr\",\"type\":\"xqtvcofu\"}],\"nextLink\":\"lvkgju\"}") + .toObject(TargetListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetReferenceTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetReferenceTests.java index 7db09e563b4f4..d233e4c6b5094 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetReferenceTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetReferenceTests.java @@ -12,17 +12,17 @@ public final class TargetReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetReference model = - BinaryData.fromString("{\"type\":\"ChaosTarget\",\"id\":\"i\"}").toObject(TargetReference.class); + TargetReference model = BinaryData.fromString("{\"type\":\"ChaosTarget\",\"id\":\"gsfraoyzkoow\"}") + .toObject(TargetReference.class); Assertions.assertEquals(TargetReferenceType.CHAOS_TARGET, model.type()); - Assertions.assertEquals("i", model.id()); + Assertions.assertEquals("gsfraoyzkoow", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TargetReference model = new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId("i"); + TargetReference model = new TargetReference().withType(TargetReferenceType.CHAOS_TARGET).withId("gsfraoyzkoow"); model = BinaryData.fromObject(model).toObject(TargetReference.class); Assertions.assertEquals(TargetReferenceType.CHAOS_TARGET, model.type()); - Assertions.assertEquals("i", model.id()); + Assertions.assertEquals("gsfraoyzkoow", model.id()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeInnerTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeInnerTests.java index 1a5844d275777..585b05d7d3758 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeInnerTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeInnerTests.java @@ -11,18 +11,16 @@ public final class TargetTypeInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetTypeInner model = - BinaryData - .fromString( - "{\"location\":\"cktqumiekkezzi\",\"properties\":{\"displayName\":\"ly\",\"description\":\"hdgqggeb\",\"propertiesSchema\":\"nyga\",\"resourceTypes\":[\"db\"]},\"id\":\"fatpxllrxcyjmoa\",\"name\":\"su\",\"type\":\"arm\"}") - .toObject(TargetTypeInner.class); - Assertions.assertEquals("cktqumiekkezzi", model.location()); + TargetTypeInner model = BinaryData.fromString( + "{\"location\":\"fimrzrtuzqogse\",\"properties\":{\"displayName\":\"evfdnwnwm\",\"description\":\"zsyyceuzso\",\"propertiesSchema\":\"judpfrxt\",\"resourceTypes\":[\"zvaytdwkqbr\"]},\"id\":\"ubpaxhe\",\"name\":\"iilivpdtiirqtd\",\"type\":\"oaxoruzfgsqu\"}") + .toObject(TargetTypeInner.class); + Assertions.assertEquals("fimrzrtuzqogse", model.location()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TargetTypeInner model = new TargetTypeInner().withLocation("cktqumiekkezzi"); + TargetTypeInner model = new TargetTypeInner().withLocation("fimrzrtuzqogse"); model = BinaryData.fromObject(model).toObject(TargetTypeInner.class); - Assertions.assertEquals("cktqumiekkezzi", model.location()); + Assertions.assertEquals("fimrzrtuzqogse", model.location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeListResultTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeListResultTests.java index 6330c9b78de72..dc526578822a7 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeListResultTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypeListResultTests.java @@ -10,11 +10,9 @@ public final class TargetTypeListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetTypeListResult model = - BinaryData - .fromString( - "{\"value\":[{\"location\":\"vmnpkukghimdblx\",\"properties\":{\"displayName\":\"imfnjhfjx\",\"description\":\"szkkfoqre\",\"propertiesSchema\":\"kzikfjawneaivxwc\",\"resourceTypes\":[\"pcirelsfeaen\",\"abfatkl\",\"dxbjhwuaanozj\"]},\"id\":\"sphyoulpjrvxa\",\"name\":\"l\",\"type\":\"vimjwos\"}],\"nextLink\":\"xitc\"}") - .toObject(TargetTypeListResult.class); + TargetTypeListResult model = BinaryData.fromString( + "{\"value\":[{\"location\":\"aeqidbqfatpxll\",\"properties\":{\"displayName\":\"cyjmoadsuvarmy\",\"description\":\"mjsjqb\",\"propertiesSchema\":\"hyxxrwlycoduhpk\",\"resourceTypes\":[\"ymareqnajxqugj\",\"ky\"]},\"id\":\"ubeddg\",\"name\":\"sofwqmzqalkrmnji\",\"type\":\"pxacqqudfn\"}],\"nextLink\":\"xbaaabjyv\"}") + .toObject(TargetTypeListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypePropertiesTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypePropertiesTests.java index 8257131ab7c3d..7b5100b2d9329 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypePropertiesTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypePropertiesTests.java @@ -10,11 +10,9 @@ public final class TargetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetTypeProperties model = - BinaryData - .fromString( - "{\"displayName\":\"dmjsjqb\",\"description\":\"hyxxrwlycoduhpk\",\"propertiesSchema\":\"gymare\",\"resourceTypes\":[\"jxqugjhky\"]}") - .toObject(TargetTypeProperties.class); + TargetTypeProperties model = BinaryData.fromString( + "{\"displayName\":\"xrxxlep\",\"description\":\"amxjezwlw\",\"propertiesSchema\":\"xuqlcvydypat\",\"resourceTypes\":[\"aojkniodk\"]}") + .toObject(TargetTypeProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetWithResponseMockTests.java index 993012ec0523b..82f08b9242a2d 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesGetWithResponseMockTests.java @@ -30,41 +30,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"location\":\"vu\",\"properties\":{\"displayName\":\"ztcktyh\",\"description\":\"qedcgzulwm\",\"propertiesSchema\":\"qzz\",\"resourceTypes\":[\"vpglydz\",\"krvq\",\"ev\"]},\"id\":\"oepry\",\"name\":\"t\",\"type\":\"wytpzdmovz\"}"; + String responseStr + = "{\"location\":\"ddrihpf\",\"properties\":{\"displayName\":\"qcaaewdaomdjvl\",\"description\":\"x\",\"propertiesSchema\":\"zb\",\"resourceTypes\":[\"geivsiykzkdncj\",\"xonbzoggculapz\"]},\"id\":\"y\",\"name\":\"pgogtqxepny\",\"type\":\"b\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - TargetType response = - manager - .targetTypes() - .getWithResponse("zapeewchpx", "ktwkuziyc", com.azure.core.util.Context.NONE) - .getValue(); + TargetType response = manager.targetTypes() + .getWithResponse("qbmfpjbabwidf", "xsspuunnoxyhk", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("vu", response.location()); + Assertions.assertEquals("ddrihpf", response.location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesListMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesListMockTests.java index 7a8bee51a7079..d66b166f6e6e0 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesListMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetTypesListMockTests.java @@ -31,38 +31,27 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"location\":\"pgogtqxepny\",\"properties\":{\"displayName\":\"fuajly\",\"description\":\"lvofqzhvfcibyfmo\",\"propertiesSchema\":\"xrkjpvdw\",\"resourceTypes\":[\"wiivwzjbhyzsx\",\"rkambt\",\"negvmnvuqe\",\"vldspa\"]},\"id\":\"tjb\",\"name\":\"kdmflvestmjlx\",\"type\":\"ril\"}]}"; + String responseStr + = "{\"value\":[{\"location\":\"aiy\",\"properties\":{\"displayName\":\"d\",\"description\":\"acegfnmntf\",\"propertiesSchema\":\"vm\",\"resourceTypes\":[\"nczdwvv\",\"alxlllchp\",\"db\"]},\"id\":\"evwrdnhfuk\",\"name\":\"vsjcswsmystuluqy\",\"type\":\"fcvlerch\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.targetTypes().list("kdncj", "xonbzoggculapz", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.targetTypes().list("gsj", "mnwqj", com.azure.core.util.Context.NONE); - Assertions.assertEquals("pgogtqxepny", response.iterator().next().location()); + Assertions.assertEquals("aiy", response.iterator().next().location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateWithResponseMockTests.java index 169963a3008f4..c8002eaa7d36e 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsCreateOrUpdateWithResponseMockTests.java @@ -33,60 +33,32 @@ public void testCreateOrUpdateWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"location\":\"qwogfnzjvus\",\"properties\":{\"ozuxylfsbtkadpys\":\"datald\",\"tgkbugrjqctojc\":\"datawn\",\"cuplcplcwkhih\":\"dataisofieypefojyqd\"},\"id\":\"hlhzdsqtzbsrgno\",\"name\":\"cjhfgmvecactxmw\",\"type\":\"teyowclu\"}"; + String responseStr + = "{\"location\":\"ocxvdfffwafqr\",\"properties\":{\"unzo\":\"datadaspavehhrvk\",\"cxgkmoyxcdyui\":\"dataud\",\"aeoisrvh\":\"datahmfdnbzydvfvfcj\"},\"id\":\"gorf\",\"name\":\"ukiscvwmzhw\",\"type\":\"lefaxvxilcbtgn\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Target response = - manager - .targets() - .createOrUpdateWithResponse( - "spave", - "hrv", - "bunzozudh", - "xg", - "moy", - new TargetInner() - .withLocation("yuibhm") - .withProperties( - mapOf( - "aeoisrvh", - "datanbzydvfvfcj", - "ukiscvwmzhw", - "datagorf", - "nzeyqxtjj", - "datalefaxvxilcbtgn", - "vodggxdbee", - "datazqlqhyc")), - com.azure.core.util.Context.NONE) - .getValue(); + Target response = manager.targets() + .createOrUpdateWithResponse("vluwmncsttij", "y", "vpo", "krsgsgb", "huzqgn", + new TargetInner().withLocation("kynscliqhzv") + .withProperties(mapOf("mtk", "datank", "ppnvdxz", "databo", "hlfkqojpy", "datahihfrbbcevqagtlt")), + com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("qwogfnzjvus", response.location()); + Assertions.assertEquals("ocxvdfffwafqr", response.location()); } // Use "Map.of" if available diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteWithResponseMockTests.java index 5764427905809..498dbd9598525 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsDeleteWithResponseMockTests.java @@ -32,33 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.targets().deleteWithResponse("auzzptjazysd", "hezwwvaiq", "uvvfonkp", "hqyikvy", "auy", + com.azure.core.util.Context.NONE); - manager - .targets() - .deleteWithResponse( - "vgtrdcnifmzzs", "ymbrnysuxmpraf", "g", "khocxvdfffwaf", "roud", com.azure.core.util.Context.NONE); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsGetWithResponseMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsGetWithResponseMockTests.java index 23e33ab2ad433..492cd5efbe7b7 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsGetWithResponseMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsGetWithResponseMockTests.java @@ -30,41 +30,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"location\":\"uwmncs\",\"properties\":{\"bvpoekrsgsgbdhu\":\"dataijf\",\"hzvhxnkomtkubo\":\"dataqgnjdgkynscli\"},\"id\":\"ppnvdxz\",\"name\":\"hihfrbbcevqagtlt\",\"type\":\"hlfkqojpy\"}"; + String responseStr + = "{\"location\":\"mldgxobfirc\",\"properties\":{\"ayzri\":\"datapkc\"},\"id\":\"khyawfvjlboxqv\",\"name\":\"jlmxhomdynhd\",\"type\":\"digumbnr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Target response = - manager - .targets() - .getWithResponse("ysdzhez", "wva", "qyuvvfonkp", "hqyikvy", "auy", com.azure.core.util.Context.NONE) - .getValue(); + Target response = manager.targets().getWithResponse("vtoepryutnw", "tpzdmovzvfvaawzq", "dflgzuri", + "laecxndticok", "vzm", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("uwmncs", response.location()); + Assertions.assertEquals("mldgxobfirc", response.location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsListMockTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsListMockTests.java index a67d7d5a0410c..4e4c05305c878 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsListMockTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/TargetsListMockTests.java @@ -31,40 +31,27 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"location\":\"clnpkci\",\"properties\":{\"wf\":\"datazriykhy\",\"xqvkjlmxhomdyn\":\"datajlb\"},\"id\":\"dwdigumb\",\"name\":\"raauzzpt\",\"type\":\"a\"}]}"; + String responseStr + = "{\"value\":[{\"location\":\"ambtrnegvm\",\"properties\":{\"dspastjbkkdmfl\":\"datauqeqv\",\"jlxr\":\"dataest\",\"wk\":\"datailozapeewchpxlk\",\"yhjtqedcgzu\":\"dataziycslevufuztck\"},\"id\":\"wmmrq\",\"name\":\"zrrjvpgly\",\"type\":\"zgkrvqe\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - ChaosManager manager = - ChaosManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + ChaosManager manager = ChaosManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .targets() - .list("fvaawzqa", "f", "gzuriglaecxndt", "cokpv", "mlqtmldgxob", com.azure.core.util.Context.NONE); + PagedIterable response = manager.targets().list("uajlyj", "lvofqzhvfcibyfmo", "uxrkjp", "dwxf", + "wiivwzjbhyzsx", com.azure.core.util.Context.NONE); - Assertions.assertEquals("clnpkci", response.iterator().next().location()); + Assertions.assertEquals("ambtrnegvm", response.iterator().next().location()); } } diff --git a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/UserAssignedIdentityTests.java b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/UserAssignedIdentityTests.java index 830b9fc5e197f..b1fe4b0049d73 100644 --- a/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/UserAssignedIdentityTests.java +++ b/sdk/chaos/azure-resourcemanager-chaos/src/test/java/com/azure/resourcemanager/chaos/generated/UserAssignedIdentityTests.java @@ -10,11 +10,9 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UserAssignedIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"15998785-2c8d-4c6e-b801-9dfeb5ab338f\",\"clientId\":\"978c9dd2-fcdd-466e-9c2e-6a409875c785\"}") - .toObject(UserAssignedIdentity.class); + UserAssignedIdentity model = BinaryData.fromString( + "{\"principalId\":\"928a78b2-4f92-47c1-a5ee-1b78249ce92d\",\"clientId\":\"ff7e318d-8f6e-4d3f-b755-1092969d4109\"}") + .toObject(UserAssignedIdentity.class); } @org.junit.jupiter.api.Test diff --git a/sdk/chaos/tests.mgmt.yml b/sdk/chaos/tests.mgmt.yml index 82a61825b4232..4b0b3dcc33c29 100644 --- a/sdk/chaos/tests.mgmt.yml +++ b/sdk/chaos/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: chaos - Artifacts: - - name: azure-resourcemanager-chaos - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerchaos - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: chaos + Artifacts: + - name: azure-resourcemanager-chaos + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerchaos + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/cognitiveservices/azure-resourcemanager-cognitiveservices/pom.xml b/sdk/cognitiveservices/azure-resourcemanager-cognitiveservices/pom.xml index b2de2f7c2c956..fa4898c03f226 100644 --- a/sdk/cognitiveservices/azure-resourcemanager-cognitiveservices/pom.xml +++ b/sdk/cognitiveservices/azure-resourcemanager-cognitiveservices/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/cognitiveservices/tests.mgmt.yml b/sdk/cognitiveservices/tests.mgmt.yml index ee398b6c64733..04096a679849c 100644 --- a/sdk/cognitiveservices/tests.mgmt.yml +++ b/sdk/cognitiveservices/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: cognitiveservices - Artifacts: - - name: azure-resourcemanager-cognitiveservices - groupId: com.azure.resourcemanager - safeName: azureresourcemanagercognitiveservices - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: cognitiveservices + Artifacts: + - name: azure-resourcemanager-cognitiveservices + groupId: com.azure.resourcemanager + safeName: azureresourcemanagercognitiveservices + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/commerce/azure-resourcemanager-commerce/pom.xml b/sdk/commerce/azure-resourcemanager-commerce/pom.xml index 1b75cf51685af..dd36f982f73de 100644 --- a/sdk/commerce/azure-resourcemanager-commerce/pom.xml +++ b/sdk/commerce/azure-resourcemanager-commerce/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/communication/azure-communication-callautomation/CHANGELOG.md b/sdk/communication/azure-communication-callautomation/CHANGELOG.md index fe50694896b3f..f545d11da0696 100644 --- a/sdk/communication/azure-communication-callautomation/CHANGELOG.md +++ b/sdk/communication/azure-communication-callautomation/CHANGELOG.md @@ -12,6 +12,18 @@ ### Other Changes + +## 1.1.3 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.1.2 (2024-02-20) ### Other Changes diff --git a/sdk/communication/azure-communication-callautomation/pom.xml b/sdk/communication/azure-communication-callautomation/pom.xml index 33684caeb2fca..83e5ffa4c8509 100644 --- a/sdk/communication/azure-communication-callautomation/pom.xml +++ b/sdk/communication/azure-communication-callautomation/pom.xml @@ -60,18 +60,18 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure azure-communication-identity - 1.5.2 + 1.5.3 test com.azure azure-messaging-servicebus - 7.15.1 + 7.15.2 test @@ -113,7 +113,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -125,13 +125,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure azure-communication-phonenumbers - 1.1.10 + 1.1.11 test diff --git a/sdk/communication/azure-communication-callautomation/tests.yml b/sdk/communication/azure-communication-callautomation/tests.yml index 02949212d53a7..6faa4c10981ee 100644 --- a/sdk/communication/azure-communication-callautomation/tests.yml +++ b/sdk/communication/azure-communication-callautomation/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: -- template: /sdk/communication/communication-tests-template.yml +extends: + template: /sdk/communication/communication-tests-template.yml parameters: PackageName: azure-communication-callautomation SafeName: azurecommunicationcallautomation diff --git a/sdk/communication/azure-communication-callingserver/pom.xml b/sdk/communication/azure-communication-callingserver/pom.xml index 90bfd5aa83aa5..2c25935c88baf 100644 --- a/sdk/communication/azure-communication-callingserver/pom.xml +++ b/sdk/communication/azure-communication-callingserver/pom.xml @@ -59,12 +59,12 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure azure-communication-identity - 1.5.2 + 1.5.3 test @@ -114,19 +114,19 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -138,7 +138,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/communication/azure-communication-callingserver/tests.yml b/sdk/communication/azure-communication-callingserver/tests.yml index 27671325fa64a..5eb85f52d279a 100644 --- a/sdk/communication/azure-communication-callingserver/tests.yml +++ b/sdk/communication/azure-communication-callingserver/tests.yml @@ -1,9 +1,9 @@ trigger: none -stages: -- template: /sdk/communication/communication-tests-template.yml +extends: + template: /sdk/communication/communication-tests-template.yml parameters: PackageName: azure-communication-callingserver - SafeName: azurecommunicationcallingserver + SafeName: azurecommunicationcallingserver Clouds: 'Public' TestMode: 'LIVE' diff --git a/sdk/communication/azure-communication-chat/CHANGELOG.md b/sdk/communication/azure-communication-chat/CHANGELOG.md index 666536674973e..f01e6e5f4ea5f 100644 --- a/sdk/communication/azure-communication-chat/CHANGELOG.md +++ b/sdk/communication/azure-communication-chat/CHANGELOG.md @@ -10,6 +10,18 @@ ### Other Changes + +## 1.4.2 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.4.1 (2024-02-20) ### Other Changes diff --git a/sdk/communication/azure-communication-chat/pom.xml b/sdk/communication/azure-communication-chat/pom.xml index 95c6f039b6c5f..ac30c723665d8 100644 --- a/sdk/communication/azure-communication-chat/pom.xml +++ b/sdk/communication/azure-communication-chat/pom.xml @@ -55,12 +55,12 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure azure-communication-identity - 1.5.2 + 1.5.3 test diff --git a/sdk/communication/azure-communication-chat/tests.yml b/sdk/communication/azure-communication-chat/tests.yml index 94c2431991fa9..17f1106290674 100644 --- a/sdk/communication/azure-communication-chat/tests.yml +++ b/sdk/communication/azure-communication-chat/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-chat - SafeName: azurecommunicationchat +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-chat + SafeName: azurecommunicationchat diff --git a/sdk/communication/azure-communication-common/CHANGELOG.md b/sdk/communication/azure-communication-common/CHANGELOG.md index e647f7e2849d0..5ccf56f90dd58 100644 --- a/sdk/communication/azure-communication-common/CHANGELOG.md +++ b/sdk/communication/azure-communication-common/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.3.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.3.0 (2024-02-12) ### Features Added diff --git a/sdk/communication/azure-communication-common/pom.xml b/sdk/communication/azure-communication-common/pom.xml index a423becb6eb4e..102731b28d22f 100644 --- a/sdk/communication/azure-communication-common/pom.xml +++ b/sdk/communication/azure-communication-common/pom.xml @@ -81,7 +81,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/communication/azure-communication-email/CHANGELOG.md b/sdk/communication/azure-communication-email/CHANGELOG.md index 788492fed3a6b..97ce01341f05a 100644 --- a/sdk/communication/azure-communication-email/CHANGELOG.md +++ b/sdk/communication/azure-communication-email/CHANGELOG.md @@ -11,6 +11,18 @@ - An `EmailMessage` with null recipient addresses can no longer be sent. ### Other Changes + +## 1.0.11 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.0.10 (2024-02-20) ### Other Changes diff --git a/sdk/communication/azure-communication-email/pom.xml b/sdk/communication/azure-communication-email/pom.xml index 4983e96a017c6..2ef1796d32882 100644 --- a/sdk/communication/azure-communication-email/pom.xml +++ b/sdk/communication/azure-communication-email/pom.xml @@ -65,7 +65,7 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 @@ -84,7 +84,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/communication/azure-communication-email/tests.yml b/sdk/communication/azure-communication-email/tests.yml index 02b3f882b2eb8..05fc0c00a69ef 100644 --- a/sdk/communication/azure-communication-email/tests.yml +++ b/sdk/communication/azure-communication-email/tests.yml @@ -6,28 +6,28 @@ parameters: type: boolean default: false -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-email - SafeName: azurecommunicationemail - ${{ if eq(parameters.runOnlyPPE, true) }}: - Clouds: 'PPE' - ${{ if eq(parameters.runOnlyPPE, false) }}: - Clouds: 'Public,PPE,Int' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-communication-services-cloud-test-resources-common) - - $(sub-config-communication-services-cloud-test-resources-java) - Int: - SubscriptionConfigurations: - - $(sub-config-communication-int-test-resources-common) - - $(sub-config-communication-int-test-resources-java) - PPE: - SubscriptionConfigurations: - - $(sub-config-communication-ppe-test-resources-common) - - $(sub-config-communication-ppe-test-resources-java) - TestResourceDirectories: - - communication/azure-communication-email/ +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-email + SafeName: azurecommunicationemail + ${{ if eq(parameters.runOnlyPPE, true) }}: + Clouds: 'PPE' + ${{ if eq(parameters.runOnlyPPE, false) }}: + Clouds: 'Public,PPE,Int' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-communication-services-cloud-test-resources-common) + - $(sub-config-communication-services-cloud-test-resources-java) + Int: + SubscriptionConfigurations: + - $(sub-config-communication-int-test-resources-common) + - $(sub-config-communication-int-test-resources-java) + PPE: + SubscriptionConfigurations: + - $(sub-config-communication-ppe-test-resources-common) + - $(sub-config-communication-ppe-test-resources-java) + TestResourceDirectories: + - communication/azure-communication-email/ diff --git a/sdk/communication/azure-communication-identity/CHANGELOG.md b/sdk/communication/azure-communication-identity/CHANGELOG.md index 05d5b95aa9c38..d09421af6d319 100644 --- a/sdk/communication/azure-communication-identity/CHANGELOG.md +++ b/sdk/communication/azure-communication-identity/CHANGELOG.md @@ -10,6 +10,18 @@ ### Other Changes + +## 1.5.3 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.5.2 (2024-02-20) ### Other Changes diff --git a/sdk/communication/azure-communication-identity/pom.xml b/sdk/communication/azure-communication-identity/pom.xml index ffc02a77f7b4b..d606a2c640ab9 100644 --- a/sdk/communication/azure-communication-identity/pom.xml +++ b/sdk/communication/azure-communication-identity/pom.xml @@ -68,7 +68,7 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 org.junit.jupiter @@ -91,7 +91,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -115,7 +115,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/communication/azure-communication-identity/tests.yml b/sdk/communication/azure-communication-identity/tests.yml index 2d5244b67d96f..958d8192c86bc 100644 --- a/sdk/communication/azure-communication-identity/tests.yml +++ b/sdk/communication/azure-communication-identity/tests.yml @@ -1,22 +1,22 @@ trigger: none -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-identity - SafeName: azurecommunicationidentity - Clouds: 'Public,PPE,Int' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-communication-services-cloud-test-resources-common) - - $(sub-config-communication-services-cloud-test-resources-java) - Int: - SubscriptionConfigurations: - - $(sub-config-communication-int-test-resources-common) - - $(sub-config-communication-int-test-resources-java) - PPE: - SubscriptionConfigurations: - - $(sub-config-communication-ppe-test-resources-common) - - $(sub-config-communication-ppe-test-resources-java) +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-identity + SafeName: azurecommunicationidentity + Clouds: 'Public,PPE,Int' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-communication-services-cloud-test-resources-common) + - $(sub-config-communication-services-cloud-test-resources-java) + Int: + SubscriptionConfigurations: + - $(sub-config-communication-int-test-resources-common) + - $(sub-config-communication-int-test-resources-java) + PPE: + SubscriptionConfigurations: + - $(sub-config-communication-ppe-test-resources-common) + - $(sub-config-communication-ppe-test-resources-java) diff --git a/sdk/communication/azure-communication-jobrouter/CHANGELOG.md b/sdk/communication/azure-communication-jobrouter/CHANGELOG.md index 4c05cc12d7924..9a02fc2d54657 100644 --- a/sdk/communication/azure-communication-jobrouter/CHANGELOG.md +++ b/sdk/communication/azure-communication-jobrouter/CHANGELOG.md @@ -13,6 +13,18 @@ ### Other Changes + +## 1.1.2 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.1.1 (2024-02-13) ### Other Changes diff --git a/sdk/communication/azure-communication-jobrouter/pom.xml b/sdk/communication/azure-communication-jobrouter/pom.xml index fc5cd8337b205..7cb1654e6c257 100644 --- a/sdk/communication/azure-communication-jobrouter/pom.xml +++ b/sdk/communication/azure-communication-jobrouter/pom.xml @@ -56,12 +56,12 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/communication/azure-communication-jobrouter/tests.yml b/sdk/communication/azure-communication-jobrouter/tests.yml index 71e0a12ea8e46..7426940a8985a 100644 --- a/sdk/communication/azure-communication-jobrouter/tests.yml +++ b/sdk/communication/azure-communication-jobrouter/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-jobrouter - SafeName: azurecommunicationjobrouter +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-jobrouter + SafeName: azurecommunicationjobrouter diff --git a/sdk/communication/azure-communication-messages/CHANGELOG.md b/sdk/communication/azure-communication-messages/CHANGELOG.md index d32e45bbfce0b..2c131959783a6 100644 --- a/sdk/communication/azure-communication-messages/CHANGELOG.md +++ b/sdk/communication/azure-communication-messages/CHANGELOG.md @@ -10,6 +10,18 @@ ### Other Changes + +## 1.0.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.0.0 (2024-02-29) ### Features Added diff --git a/sdk/communication/azure-communication-messages/pom.xml b/sdk/communication/azure-communication-messages/pom.xml index 7254fbfef446b..9b4bfaaf56892 100644 --- a/sdk/communication/azure-communication-messages/pom.xml +++ b/sdk/communication/azure-communication-messages/pom.xml @@ -56,7 +56,7 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure @@ -84,7 +84,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -102,7 +102,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/communication/azure-communication-messages/tests.yml b/sdk/communication/azure-communication-messages/tests.yml index a9ffca231eddf..95cc9020d4a43 100644 --- a/sdk/communication/azure-communication-messages/tests.yml +++ b/sdk/communication/azure-communication-messages/tests.yml @@ -6,24 +6,24 @@ parameters: type: boolean default: false -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-messages - SafeName: azurecommunicationmessages - ${{ if eq(parameters.runOnlyPPE, true) }}: - Clouds: 'PPE' - ${{ if eq(parameters.runOnlyPPE, false) }}: - Clouds: 'Public,PPE' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-communication-services-cloud-test-resources-common) - - $(sub-config-communication-services-cloud-test-resources-java) - PPE: - SubscriptionConfigurations: - - $(sub-config-communication-ppe-test-resources-common) - - $(sub-config-communication-ppe-test-resources-java) - TestResourceDirectories: - - communication/azure-communication-messages/ +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-messages + SafeName: azurecommunicationmessages + ${{ if eq(parameters.runOnlyPPE, true) }}: + Clouds: 'PPE' + ${{ if eq(parameters.runOnlyPPE, false) }}: + Clouds: 'Public,PPE' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-communication-services-cloud-test-resources-common) + - $(sub-config-communication-services-cloud-test-resources-java) + PPE: + SubscriptionConfigurations: + - $(sub-config-communication-ppe-test-resources-common) + - $(sub-config-communication-ppe-test-resources-java) + TestResourceDirectories: + - communication/azure-communication-messages/ diff --git a/sdk/communication/azure-communication-networktraversal/pom.xml b/sdk/communication/azure-communication-networktraversal/pom.xml index 7f1b70a580c7d..5ab12c911f69e 100644 --- a/sdk/communication/azure-communication-networktraversal/pom.xml +++ b/sdk/communication/azure-communication-networktraversal/pom.xml @@ -66,7 +66,7 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 org.junit.jupiter @@ -89,7 +89,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -107,13 +107,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure azure-communication-identity - 1.5.2 + 1.5.3 test diff --git a/sdk/communication/azure-communication-networktraversal/tests.yml b/sdk/communication/azure-communication-networktraversal/tests.yml index aab91a89141d5..407ac8b620b13 100644 --- a/sdk/communication/azure-communication-networktraversal/tests.yml +++ b/sdk/communication/azure-communication-networktraversal/tests.yml @@ -1,8 +1,8 @@ trigger: none -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-networktraversal - SafeName: azurecommunicationnetworktraversal - Clouds: 'Public' +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-networktraversal + SafeName: azurecommunicationnetworktraversal + Clouds: 'Public' diff --git a/sdk/communication/azure-communication-phonenumbers/CHANGELOG.md b/sdk/communication/azure-communication-phonenumbers/CHANGELOG.md index 76530a5a67485..5cd810aee64c1 100644 --- a/sdk/communication/azure-communication-phonenumbers/CHANGELOG.md +++ b/sdk/communication/azure-communication-phonenumbers/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.2.0-beta.1 (Unreleased) +## 1.2.0-beta.4 (Unreleased) ### Features Added @@ -10,6 +10,46 @@ ### Other Changes + +## 1.1.11 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + +## 1.2.0-beta.3 (2024-03-15) + +### Features Added + +- Add support for number lookup + - Format only can be returned for no cost + - Additional number details can be returned for a cost + + +## 1.1.10 (2024-02-22) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.45.1` to version `1.46.0`. +- Upgraded `azure-communication-common` from `1.2.14` to version `1.3.0`. + + +## 1.2.0-beta.2 (2024-01-30) + +### Features Added + +- Added support for API version `2023-10-01-preview`, This gives us functionality for Do Not Resell Consent Changes. +- Users will now be able to purchase from countries like `IT`, `AT`, `FR`, `NO`, `PT`. They can do so by agreeing not to resell phone numbers by setting consentToNotResellNumbers to true. +- Added support for 10DLC changes. + + ## 1.1.9 (2023-12-04) ### Other Changes @@ -46,6 +86,12 @@ - Upgraded `azure-core` from `1.42.0` to version `1.43.0`. - Upgraded `azure-communication-common` from `1.2.11` to version `1.2.12`. +## 1.2.0-beta.1 (2023-09-07) + +### Features Added +- Number Lookup API public preview +- API version `2023-05-01-preview` is the default + ## 1.1.5 (2023-08-18) ### Other Changes diff --git a/sdk/communication/azure-communication-phonenumbers/assets.json b/sdk/communication/azure-communication-phonenumbers/assets.json index 3787aed70683d..407d713a01737 100644 --- a/sdk/communication/azure-communication-phonenumbers/assets.json +++ b/sdk/communication/azure-communication-phonenumbers/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/communication/azure-communication-phonenumbers", - "Tag": "java/communication/azure-communication-phonenumbers_154a5685d1" + "Tag": "java/communication/azure-communication-phonenumbers_737e64b52b" } diff --git a/sdk/communication/azure-communication-phonenumbers/phone-numbers-livetest-matrix.json b/sdk/communication/azure-communication-phonenumbers/phone-numbers-livetest-matrix.json index 245909fe9d6df..46a5386a8f96e 100644 --- a/sdk/communication/azure-communication-phonenumbers/phone-numbers-livetest-matrix.json +++ b/sdk/communication/azure-communication-phonenumbers/phone-numbers-livetest-matrix.json @@ -5,15 +5,15 @@ "matrix": { "Agent": { "ubuntu-20.04": { - "OSVmImage": "MMSUbuntu20.04", - "Pool": "azsdk-pool-mms-ubuntu-2004-general", + "OSVmImage": "env:LINUXVMIMAGE", + "Pool": "env:LINUXPOOL", "JavaTestVersion": "1.8", "AZURE_TEST_HTTP_CLIENTS": "okhttp", "AZURE_TEST_AGENT": "UBUNTU_2004_JAVA8" }, "macos-11": { - "OSVmImage": "macos-11", - "Pool": "Azure Pipelines", + "OSVmImage": "env:MACVMIMAGE", + "Pool": "env:MACPOOL", "JavaTestVersion": "1.21", "AZURE_TEST_HTTP_CLIENTS": "netty", "AZURE_TEST_AGENT": "MACOS_1015_JAVA11" @@ -30,8 +30,8 @@ { "Agent": { "ubuntu-20.04": { - "OSVmImage": "MMSUbuntu20.04", - "Pool": "azsdk-pool-mms-ubuntu-2004-general" + "OSVmImage": "env:LINUXVMIMAGE", + "Pool": "env:LINUXPOOL" } }, "JavaTestVersion": "1.21", @@ -47,8 +47,8 @@ { "Agent": { "windows-2022": { - "OSVmImage": "windows-2022", - "Pool": "azsdk-pool-mms-win-2022-general" + "OSVmImage": "env:WINDOWSVMIMAGE", + "Pool": "env:WINDOWSPOOL" } }, "JavaTestVersion": "1.21", diff --git a/sdk/communication/azure-communication-phonenumbers/pom.xml b/sdk/communication/azure-communication-phonenumbers/pom.xml index bad841ea884ef..cbf7389c435ff 100644 --- a/sdk/communication/azure-communication-phonenumbers/pom.xml +++ b/sdk/communication/azure-communication-phonenumbers/pom.xml @@ -16,7 +16,7 @@ com.azure azure-communication-phonenumbers jar - 1.2.0-beta.1 + 1.2.0-beta.4 Microsoft Azure client phone numbers library for communication @@ -68,7 +68,7 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 org.junit.jupiter @@ -91,7 +91,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -117,13 +117,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -135,7 +135,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClient.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClient.java index db9390db88a32..d772c29147de2 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClient.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClient.java @@ -14,6 +14,9 @@ import com.azure.communication.phonenumbers.implementation.models.PhoneNumbersReleasePhoneNumberResponse; import com.azure.communication.phonenumbers.implementation.models.PhoneNumberCapabilitiesRequest; import com.azure.communication.phonenumbers.implementation.models.PhoneNumbersUpdateCapabilitiesResponse; +import com.azure.communication.phonenumbers.implementation.models.OperatorInformationRequest; +import com.azure.communication.phonenumbers.models.OperatorInformationResult; +import com.azure.communication.phonenumbers.models.OperatorInformationOptions; import com.azure.communication.phonenumbers.models.PhoneNumberAreaCode; import com.azure.communication.phonenumbers.models.PurchasedPhoneNumber; import com.azure.communication.phonenumbers.models.ReleasePhoneNumberResult; @@ -45,6 +48,7 @@ import reactor.core.publisher.Mono; import java.time.Duration; +import java.util.List; import java.util.Objects; import java.util.function.BiFunction; import java.util.function.Function; @@ -61,7 +65,7 @@ * * - * + * *

  * PhoneNumbersAsyncClient phoneNumberAsyncClient = new PhoneNumbersClientBuilder()
  *         .endpoint(endpoint)
@@ -69,7 +73,7 @@
  *         .httpClient(httpClient)
  *         .buildAsyncClient();
  * 
- * + * * * * @see PhoneNumbersClientBuilder @@ -99,13 +103,13 @@ public final class PhoneNumbersAsyncClient { * * - * + * *
      * PurchasedPhoneNumber phoneNumber = phoneNumberAsyncClient.getPurchasedPhoneNumber("+18001234567").block();
      * System.out.println("Phone Number Value: " + phoneNumber.getPhoneNumber());
      * System.out.println("Phone Number Country Code: " + phoneNumber.getCountryCode());
      * 
- * + * * * * @param phoneNumber The phone number id in E.164 format. The leading plus can @@ -133,7 +137,7 @@ public Mono getPurchasedPhoneNumber(String phoneNumber) { * * - * + * *
      * Response<PurchasedPhoneNumber> response = phoneNumberAsyncClient
      *         .getPurchasedPhoneNumberWithResponse("+18001234567").block();
@@ -141,7 +145,7 @@ public Mono getPurchasedPhoneNumber(String phoneNumber) {
      * System.out.println("Phone Number Value: " + phoneNumber.getPhoneNumber());
      * System.out.println("Phone Number Country Code: " + phoneNumber.getCountryCode());
      * 
- * + * * * @@ -170,14 +174,14 @@ public Mono> getPurchasedPhoneNumberWithResponse( * * - * + * *
      * PagedFlux<PurchasedPhoneNumber> phoneNumbers = phoneNumberAsyncClient.listPurchasedPhoneNumbers();
      * PurchasedPhoneNumber phoneNumber = phoneNumbers.blockFirst();
      * System.out.println("Phone Number Value: " + phoneNumber.getPhoneNumber());
      * System.out.println("Phone Number Country Code: " + phoneNumber.getCountryCode());
      * 
- * + * * * * @return A {@link PagedFlux} of {@link PurchasedPhoneNumber} instances @@ -201,7 +205,7 @@ public PagedFlux listPurchasedPhoneNumbers() { * * - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities()
      *         .setCalling(PhoneNumberCapabilityType.INBOUND)
@@ -221,7 +225,7 @@ public PagedFlux listPurchasedPhoneNumbers() {
      *     System.out.println("Phone number costs:" + searchResult.getCost().getAmount());
      * }
      * 
- * + * * * @@ -252,7 +256,7 @@ public PollerFlux beginSearchAvai * - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities()
      *         .setCalling(PhoneNumberCapabilityType.INBOUND)
@@ -272,7 +276,7 @@ public PollerFlux beginSearchAvai
      *     System.out.println("Phone number costs:" + searchResult.getCost().getAmount());
      * }
      * 
- * + * * @@ -406,13 +410,13 @@ private Function, Mono - * + * *
      * AsyncPollResponse<PhoneNumberOperation, PurchasePhoneNumbersResult> purchaseResponse = phoneNumberAsyncClient
      *         .beginPurchasePhoneNumbers(searchId).blockFirst();
      * System.out.println("Purchase phone numbers is complete: " + purchaseResponse.getStatus());
      * 
- * + * * * * @param searchId ID of the search. @@ -472,13 +476,13 @@ private Function, Mono - * + * *
      * AsyncPollResponse<PhoneNumberOperation, ReleasePhoneNumberResult> releaseResponse = phoneNumberAsyncClient
      *         .beginReleasePhoneNumber("+18001234567").blockFirst();
      * System.out.println("Release phone number is complete: " + releaseResponse.getStatus());
      * 
- * + * * * * @param phoneNumber The phone number id in E.164 format. The leading plus can @@ -534,7 +538,7 @@ private Function, Mono - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities();
      * capabilities
@@ -551,7 +555,7 @@ private Function, Mono
-     * 
+     *
      * 
      *
@@ -715,6 +719,40 @@ public PagedFlux listAvailableOfferings(String countryCode,
         }
     }
 
+    /**
+     * Searches for operator information for a given list of phone numbers.
+     *
+     * @param phoneNumbers The phone number(s) whose operator information should be searched.
+     *
+     * @return A {@link OperatorInformationResult} which contains the results of the search.
+     */
+    @ServiceMethod(returns = ReturnType.SINGLE)
+    public Mono searchOperatorInformation(List phoneNumbers) {
+        OperatorInformationRequest request = new OperatorInformationRequest();
+        request.setPhoneNumbers(phoneNumbers);
+        request.setOptions(new OperatorInformationOptions().setIncludeAdditionalOperatorDetails(false));
+        return client.operatorInformationSearchAsync(request)
+                .onErrorMap(CommunicationErrorResponseException.class, e -> translateException(e));
+    }
+
+    /**
+     * Searches for operator information for a given list of phone numbers.
+     *
+     * @param phoneNumbers The phone number(s) whose operator information should be searched.
+     * @param requestOptions Modifies the search to include additional fields in the response.
+     *                  Please note: use of options will affect the cost of the search.
+     *
+     * @return A {@link OperatorInformationResult} which contains the results of the search.
+     */
+    @ServiceMethod(returns = ReturnType.SINGLE)
+    public Mono> searchOperatorInformationWithResponse(List phoneNumbers, OperatorInformationOptions requestOptions) {
+        OperatorInformationRequest request = new OperatorInformationRequest();
+        request.setPhoneNumbers(phoneNumbers);
+        request.setOptions(requestOptions);
+        return client.operatorInformationSearchWithResponseAsync(request)
+                .onErrorMap(CommunicationErrorResponseException.class, e -> translateException(e));
+    }
+
     private Mono getOperation(String operationId) {
         return client.getOperationAsync(operationId)
                 .onErrorMap(CommunicationErrorResponseException.class, e -> translateException(e))
diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersClient.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersClient.java
index 87d302a15e665..a1b43dac22d37 100644
--- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersClient.java
+++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersClient.java
@@ -2,10 +2,14 @@
 // Licensed under the MIT License.
 package com.azure.communication.phonenumbers;
 
+import java.util.List;
 import java.util.Objects;
 
 import com.azure.communication.phonenumbers.implementation.PhoneNumberAdminClientImpl;
 import com.azure.communication.phonenumbers.implementation.PhoneNumbersImpl;
+import com.azure.communication.phonenumbers.implementation.models.OperatorInformationRequest;
+import com.azure.communication.phonenumbers.models.OperatorInformationResult;
+import com.azure.communication.phonenumbers.models.OperatorInformationOptions;
 import com.azure.communication.phonenumbers.models.PhoneNumberAreaCode;
 import com.azure.communication.phonenumbers.models.PhoneNumberAssignmentType;
 import com.azure.communication.phonenumbers.models.PhoneNumberCapabilities;
@@ -104,7 +108,7 @@ public PurchasedPhoneNumber getPurchasedPhoneNumber(String phoneNumber) {
      *
      * 
-     * 
+     *
      * 
      * Response<PurchasedPhoneNumber> response = phoneNumberClient
      *         .getPurchasedPhoneNumberWithResponse("+18001234567", Context.NONE);
@@ -112,7 +116,7 @@ public PurchasedPhoneNumber getPurchasedPhoneNumber(String phoneNumber) {
      * System.out.println("Phone Number Value: " + phoneNumber.getPhoneNumber());
      * System.out.println("Phone Number Country Code: " + phoneNumber.getCountryCode());
      * 
- * + * * * @@ -164,14 +168,14 @@ public PagedIterable listPurchasedPhoneNumbers() { * * - * + * *
      * PagedIterable<PurchasedPhoneNumber> phoneNumbers = phoneNumberClient.listPurchasedPhoneNumbers(Context.NONE);
      * PurchasedPhoneNumber phoneNumber = phoneNumbers.iterator().next();
      * System.out.println("Phone Number Value: " + phoneNumber.getPhoneNumber());
      * System.out.println("Phone Number Country Code: " + phoneNumber.getCountryCode());
      * 
- * + * * * @@ -198,7 +202,7 @@ public PagedIterable listPurchasedPhoneNumbers(Context con * * - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities()
      *         .setCalling(PhoneNumberCapabilityType.INBOUND)
@@ -218,7 +222,7 @@ public PagedIterable listPurchasedPhoneNumbers(Context con
      *     System.out.println("Phone number costs:" + searchResult.getCost().getAmount());
      * }
      * 
- * + * * * * @param countryCode The ISO 3166-2 country code. @@ -253,7 +257,7 @@ public SyncPoller beginSearchAvai * - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities()
      *         .setCalling(PhoneNumberCapabilityType.INBOUND)
@@ -274,7 +278,7 @@ public SyncPoller beginSearchAvai
      *     System.out.println("Phone number costs:" + searchResult.getCost().getAmount());
      * }
      * 
- * + * * @@ -342,13 +346,13 @@ public SyncPoller beginPurchas * * - * + * *
      * PollResponse<PhoneNumberOperation> purchaseResponse = phoneNumberClient
      *         .beginPurchasePhoneNumbers(searchId, Context.NONE).waitForCompletion();
      * System.out.println("Purchase phone numbers is complete: " + purchaseResponse.getStatus());
      * 
- * + * * * @@ -406,13 +410,13 @@ public SyncPoller beginReleasePh * * - * + * *
      * PollResponse<PhoneNumberOperation> releaseResponse = phoneNumberClient
      *         .beginReleasePhoneNumber("+18001234567", Context.NONE).waitForCompletion();
      * System.out.println("Release phone number is complete: " + releaseResponse.getStatus());
      * 
- * + * * * @@ -442,7 +446,7 @@ public SyncPoller beginReleasePh * * - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities();
      * capabilities
@@ -459,7 +463,7 @@ public SyncPoller beginReleasePh
      *     System.out.println("Phone Number SMS capabilities: " + phoneNumber.getCapabilities().getSms());
      * }
      * 
- * + * * * @@ -491,7 +495,7 @@ public SyncPoller beginUpdatePhoneNu * - * + * *
      * PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities();
      * capabilities
@@ -508,7 +512,7 @@ public SyncPoller beginUpdatePhoneNu
      *     System.out.println("Phone Number SMS capabilities: " + phoneNumber.getCapabilities().getSms());
      * }
      * 
- * + * * @@ -695,4 +699,40 @@ public PagedIterable listAvailableOfferings(String countryC context = context == null ? Context.NONE : context; return client.listOfferings(countryCode, null, null, phoneNumberType, assignmentType, acceptLanguage, context); } + + /** + * Searches for operator information for a given list of phone numbers. + * + * @param phoneNumbers The phone number(s) whose operator information should be searched. + * + * @return A {@link OperatorInformationResult} which contains the results of the search. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public OperatorInformationResult searchOperatorInformation(List phoneNumbers) { + OperatorInformationRequest request = new OperatorInformationRequest(); + request.setPhoneNumbers(phoneNumbers); + request.setOptions(new OperatorInformationOptions().setIncludeAdditionalOperatorDetails(false)); + return client.operatorInformationSearch(request); + } + + /** + * Searches for operator information for a given list of phone numbers. + * + * @param phoneNumbers The phone number(s) whose operator information should be searched. + * @param requestOptions Modifies the search to include additional fields in the response. + * Please note: use of options will affect the cost of the search. + * @param context A {@link Context} representing the request context. + * + * @return A {@link OperatorInformationResult} which contains the results of the search. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response searchOperatorInformationWithResponse(List phoneNumbers, + OperatorInformationOptions requestOptions, + Context context) { + context = context == null ? Context.NONE : context; + OperatorInformationRequest request = new OperatorInformationRequest(); + request.setPhoneNumbers(phoneNumbers); + request.setOptions(requestOptions); + return client.operatorInformationSearchWithResponse(request, context); + } } diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersServiceVersion.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersServiceVersion.java index 31e2a47be2465..ac4b94ed4bc36 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersServiceVersion.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/PhoneNumbersServiceVersion.java @@ -12,7 +12,12 @@ public enum PhoneNumbersServiceVersion implements ServiceVersion { /** * Service version {@code 2021-03-07}. */ - V2021_03_07("2021-03-07"); + V2021_03_07("2021-03-07"), + + /** + * Number Lookup update to public preview {@code 2024-03-01-preview} + */ + V2024_03_01_PREVIEW("2024-03-01-preview"); private final String version; @@ -37,6 +42,6 @@ public String getVersion() { */ public static PhoneNumbersServiceVersion getLatest() { - return V2021_03_07; + return V2024_03_01_PREVIEW; } } diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImpl.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImpl.java index 6513b87288d54..0e399024be738 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImpl.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImpl.java @@ -74,23 +74,31 @@ public PhoneNumbersImpl getPhoneNumbers() { return this.phoneNumbers; } - /** Initializes an instance of PhoneNumberAdminClient client. */ - PhoneNumberAdminClientImpl(String endpoint) { + /** + * Initializes an instance of PhoneNumberAdminClient client. + * + * @param endpoint The communication resource, for example https://resourcename.communication.azure.com. + * @param apiVersion Api Version. + */ + PhoneNumberAdminClientImpl(String endpoint, String apiVersion) { this( new HttpPipelineBuilder() .policies(new UserAgentPolicy(), new RetryPolicy(), new CookiePolicy()) .build(), JacksonAdapter.createDefaultSerializerAdapter(), - endpoint); + endpoint, + apiVersion); } /** * Initializes an instance of PhoneNumberAdminClient client. * * @param httpPipeline The HTTP pipeline to send requests through. + * @param endpoint The communication resource, for example https://resourcename.communication.azure.com. + * @param apiVersion Api Version. */ - PhoneNumberAdminClientImpl(HttpPipeline httpPipeline, String endpoint) { - this(httpPipeline, JacksonAdapter.createDefaultSerializerAdapter(), endpoint); + PhoneNumberAdminClientImpl(HttpPipeline httpPipeline, String endpoint, String apiVersion) { + this(httpPipeline, JacksonAdapter.createDefaultSerializerAdapter(), endpoint, apiVersion); } /** @@ -98,12 +106,15 @@ public PhoneNumbersImpl getPhoneNumbers() { * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. + * @param endpoint The communication resource, for example https://resourcename.communication.azure.com. + * @param apiVersion Api Version. */ - PhoneNumberAdminClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, String endpoint) { + PhoneNumberAdminClientImpl( + HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, String endpoint, String apiVersion) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.endpoint = endpoint; - this.apiVersion = "2022-12-01"; + this.apiVersion = apiVersion; this.phoneNumbers = new PhoneNumbersImpl(this); } } diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImplBuilder.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImplBuilder.java index 425704ad03a39..3df8f784d61da 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImplBuilder.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumberAdminClientImplBuilder.java @@ -4,55 +4,182 @@ package com.azure.communication.phonenumbers.implementation; +import com.azure.core.annotation.Generated; import com.azure.core.annotation.ServiceClientBuilder; +import com.azure.core.client.traits.ConfigurationTrait; +import com.azure.core.client.traits.EndpointTrait; +import com.azure.core.client.traits.HttpTrait; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; import com.azure.core.http.HttpPipeline; import com.azure.core.http.HttpPipelineBuilder; +import com.azure.core.http.HttpPipelinePosition; +import com.azure.core.http.policy.AddDatePolicy; +import com.azure.core.http.policy.AddHeadersFromContextPolicy; +import com.azure.core.http.policy.AddHeadersPolicy; import com.azure.core.http.policy.CookiePolicy; +import com.azure.core.http.policy.HttpLogOptions; +import com.azure.core.http.policy.HttpLoggingPolicy; +import com.azure.core.http.policy.HttpPipelinePolicy; +import com.azure.core.http.policy.HttpPolicyProviders; +import com.azure.core.http.policy.RequestIdPolicy; +import com.azure.core.http.policy.RetryOptions; import com.azure.core.http.policy.RetryPolicy; import com.azure.core.http.policy.UserAgentPolicy; +import com.azure.core.util.ClientOptions; +import com.azure.core.util.Configuration; +import com.azure.core.util.CoreUtils; +import com.azure.core.util.builder.ClientBuilderUtil; import com.azure.core.util.serializer.JacksonAdapter; import com.azure.core.util.serializer.SerializerAdapter; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; /** A builder for creating a new instance of the PhoneNumberAdminClient type. */ @ServiceClientBuilder(serviceClients = {PhoneNumberAdminClientImpl.class}) -public final class PhoneNumberAdminClientImplBuilder { +public final class PhoneNumberAdminClientImplBuilder + implements HttpTrait, + ConfigurationTrait, + EndpointTrait { + @Generated private static final String SDK_NAME = "name"; + + @Generated private static final String SDK_VERSION = "version"; + + @Generated private final Map properties = new HashMap<>(); + + @Generated private final List pipelinePolicies; + + /** Create an instance of the PhoneNumberAdminClientImplBuilder. */ + @Generated + public PhoneNumberAdminClientImplBuilder() { + this.pipelinePolicies = new ArrayList<>(); + } + /* - * The communication resource, for example - * https://resourcename.communication.azure.com + * The HTTP pipeline to send requests through. */ - private String endpoint; + @Generated private HttpPipeline pipeline; - /** - * Sets The communication resource, for example https://resourcename.communication.azure.com. - * - * @param endpoint the endpoint value. - * @return the PhoneNumberAdminClientImplBuilder. + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder pipeline(HttpPipeline pipeline) { + this.pipeline = pipeline; + return this; + } + + /* + * The HTTP client used to send the request. + */ + @Generated private HttpClient httpClient; + + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder httpClient(HttpClient httpClient) { + this.httpClient = httpClient; + return this; + } + + /* + * The logging configuration for HTTP requests and responses. */ + @Generated private HttpLogOptions httpLogOptions; + + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder httpLogOptions(HttpLogOptions httpLogOptions) { + this.httpLogOptions = httpLogOptions; + return this; + } + + /* + * The client options such as application ID and custom headers to set on a + * request. + */ + @Generated private ClientOptions clientOptions; + + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder clientOptions(ClientOptions clientOptions) { + this.clientOptions = clientOptions; + return this; + } + + /* + * The retry options to configure retry policy for failed requests. + */ + @Generated private RetryOptions retryOptions; + + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder retryOptions(RetryOptions retryOptions) { + this.retryOptions = retryOptions; + return this; + } + + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder addPolicy(HttpPipelinePolicy customPolicy) { + pipelinePolicies.add(customPolicy); + return this; + } + + /* + * The configuration store that is used during construction of the service + * client. + */ + @Generated private Configuration configuration; + + /** {@inheritDoc}. */ + @Generated + @Override + public PhoneNumberAdminClientImplBuilder configuration(Configuration configuration) { + this.configuration = configuration; + return this; + } + + /* + * The service endpoint + */ + @Generated private String endpoint; + + /** {@inheritDoc}. */ + @Generated + @Override public PhoneNumberAdminClientImplBuilder endpoint(String endpoint) { this.endpoint = endpoint; return this; } /* - * The HTTP pipeline to send requests through + * Api Version */ - private HttpPipeline pipeline; + @Generated private String apiVersion; /** - * Sets The HTTP pipeline to send requests through. + * Sets Api Version. * - * @param pipeline the pipeline value. + * @param apiVersion the apiVersion value. * @return the PhoneNumberAdminClientImplBuilder. */ - public PhoneNumberAdminClientImplBuilder pipeline(HttpPipeline pipeline) { - this.pipeline = pipeline; + @Generated + public PhoneNumberAdminClientImplBuilder apiVersion(String apiVersion) { + this.apiVersion = apiVersion; return this; } /* * The serializer to serialize an object into a string */ - private SerializerAdapter serializerAdapter; + @Generated private SerializerAdapter serializerAdapter; /** * Sets The serializer to serialize an object into a string. @@ -60,27 +187,93 @@ public PhoneNumberAdminClientImplBuilder pipeline(HttpPipeline pipeline) { * @param serializerAdapter the serializerAdapter value. * @return the PhoneNumberAdminClientImplBuilder. */ + @Generated public PhoneNumberAdminClientImplBuilder serializerAdapter(SerializerAdapter serializerAdapter) { this.serializerAdapter = serializerAdapter; return this; } + /* + * The retry policy that will attempt to retry failed requests, if + * applicable. + */ + @Generated private RetryPolicy retryPolicy; + + /** + * Sets The retry policy that will attempt to retry failed requests, if applicable. + * + * @param retryPolicy the retryPolicy value. + * @return the PhoneNumberAdminClientImplBuilder. + */ + @Generated + public PhoneNumberAdminClientImplBuilder retryPolicy(RetryPolicy retryPolicy) { + this.retryPolicy = retryPolicy; + return this; + } + /** * Builds an instance of PhoneNumberAdminClientImpl with the provided parameters. * * @return an instance of PhoneNumberAdminClientImpl. */ + @Generated public PhoneNumberAdminClientImpl buildClient() { if (pipeline == null) { - this.pipeline = - new HttpPipelineBuilder() - .policies(new UserAgentPolicy(), new RetryPolicy(), new CookiePolicy()) - .build(); + this.pipeline = createHttpPipeline(); + } + if (apiVersion == null) { + this.apiVersion = "2024-03-01-preview"; } if (serializerAdapter == null) { this.serializerAdapter = JacksonAdapter.createDefaultSerializerAdapter(); } - PhoneNumberAdminClientImpl client = new PhoneNumberAdminClientImpl(pipeline, serializerAdapter, endpoint); + PhoneNumberAdminClientImpl client = + new PhoneNumberAdminClientImpl(pipeline, serializerAdapter, endpoint, apiVersion); return client; } + + @Generated + private HttpPipeline createHttpPipeline() { + Configuration buildConfiguration = + (configuration == null) ? Configuration.getGlobalConfiguration() : configuration; + if (httpLogOptions == null) { + httpLogOptions = new HttpLogOptions(); + } + if (clientOptions == null) { + clientOptions = new ClientOptions(); + } + List policies = new ArrayList<>(); + String clientName = properties.getOrDefault(SDK_NAME, "UnknownName"); + String clientVersion = properties.getOrDefault(SDK_VERSION, "UnknownVersion"); + String applicationId = CoreUtils.getApplicationId(clientOptions, httpLogOptions); + policies.add(new UserAgentPolicy(applicationId, clientName, clientVersion, buildConfiguration)); + policies.add(new RequestIdPolicy()); + policies.add(new AddHeadersFromContextPolicy()); + HttpHeaders headers = new HttpHeaders(); + clientOptions.getHeaders().forEach(header -> headers.set(header.getName(), header.getValue())); + if (headers.getSize() > 0) { + policies.add(new AddHeadersPolicy(headers)); + } + policies.addAll( + this.pipelinePolicies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); + HttpPolicyProviders.addBeforeRetryPolicies(policies); + policies.add(ClientBuilderUtil.validateAndGetRetryPolicy(retryPolicy, retryOptions, new RetryPolicy())); + policies.add(new AddDatePolicy()); + policies.add(new CookiePolicy()); + policies.addAll( + this.pipelinePolicies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) + .collect(Collectors.toList())); + HttpPolicyProviders.addAfterRetryPolicies(policies); + policies.add(new HttpLoggingPolicy(httpLogOptions)); + HttpPipeline httpPipeline = + new HttpPipelineBuilder() + .policies(policies.toArray(new HttpPipelinePolicy[0])) + .httpClient(httpClient) + .clientOptions(clientOptions) + .build(); + return httpPipeline; + } } diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumbersImpl.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumbersImpl.java index 8f37bf040525d..dd2e8c5c0f011 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumbersImpl.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/PhoneNumbersImpl.java @@ -6,6 +6,7 @@ import com.azure.communication.phonenumbers.implementation.models.CommunicationErrorResponseException; import com.azure.communication.phonenumbers.implementation.models.OfferingsResponse; +import com.azure.communication.phonenumbers.implementation.models.OperatorInformationRequest; import com.azure.communication.phonenumbers.implementation.models.PhoneNumberAreaCodes; import com.azure.communication.phonenumbers.implementation.models.PhoneNumberCapabilitiesRequest; import com.azure.communication.phonenumbers.implementation.models.PhoneNumberCountries; @@ -19,6 +20,7 @@ import com.azure.communication.phonenumbers.implementation.models.PhoneNumbersSearchAvailablePhoneNumbersResponse; import com.azure.communication.phonenumbers.implementation.models.PhoneNumbersUpdateCapabilitiesResponse; import com.azure.communication.phonenumbers.implementation.models.PurchasedPhoneNumbers; +import com.azure.communication.phonenumbers.models.OperatorInformationResult; import com.azure.communication.phonenumbers.models.PhoneNumberAreaCode; import com.azure.communication.phonenumbers.models.PhoneNumberAssignmentType; import com.azure.communication.phonenumbers.models.PhoneNumberCountry; @@ -92,6 +94,7 @@ Mono> listAreaCodes( @QueryParam("administrativeDivision") String administrativeDivision, @QueryParam("api-version") String apiVersion, @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, Context context); @Get("/availablePhoneNumbers/countries") @@ -103,6 +106,7 @@ Mono> listAvailableCountries( @QueryParam("maxPageSize") Integer maxPageSize, @QueryParam("api-version") String apiVersion, @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, Context context); @Get("/availablePhoneNumbers/countries/{countryCode}/localities") @@ -116,6 +120,7 @@ Mono> listAvailableLocalities( @QueryParam("administrativeDivision") String administrativeDivision, @QueryParam("api-version") String apiVersion, @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, Context context); @Get("/availablePhoneNumbers/countries/{countryCode}/offerings") @@ -130,6 +135,7 @@ Mono> listOfferings( @QueryParam("assignmentType") PhoneNumberAssignmentType assignmentType, @QueryParam("api-version") String apiVersion, @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, Context context); @Post("/availablePhoneNumbers/countries/{countryCode}/:search") @@ -140,6 +146,7 @@ Mono searchAvailablePhoneNumber @PathParam("countryCode") String countryCode, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") PhoneNumberSearchRequest body, + @HeaderParam("Accept") String accept, Context context); @Get("/availablePhoneNumbers/searchResults/{searchId}") @@ -149,6 +156,7 @@ Mono> getSearchResult( @HostParam("endpoint") String endpoint, @PathParam("searchId") String searchId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Post("/availablePhoneNumbers/:purchase") @@ -158,6 +166,7 @@ Mono purchasePhoneNumbers( @HostParam("endpoint") String endpoint, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") PhoneNumberPurchaseRequest body, + @HeaderParam("Accept") String accept, Context context); @Get("/phoneNumbers/operations/{operationId}") @@ -167,6 +176,7 @@ Mono getOperation( @HostParam("endpoint") String endpoint, @PathParam("operationId") String operationId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Delete("/phoneNumbers/operations/{operationId}") @@ -176,6 +186,7 @@ Mono> cancelOperation( @HostParam("endpoint") String endpoint, @PathParam("operationId") String operationId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Patch("/phoneNumbers/{phoneNumber}/capabilities") @@ -186,6 +197,7 @@ Mono updateCapabilities( @PathParam("phoneNumber") String phoneNumber, @QueryParam("api-version") String apiVersion, @BodyParam("application/merge-patch+json") PhoneNumberCapabilitiesRequest body, + @HeaderParam("Accept") String accept, Context context); @Get("/phoneNumbers/{phoneNumber}") @@ -195,6 +207,7 @@ Mono> getByNumber( @HostParam("endpoint") String endpoint, @PathParam("phoneNumber") String phoneNumber, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Delete("/phoneNumbers/{phoneNumber}") @@ -204,6 +217,7 @@ Mono releasePhoneNumber( @HostParam("endpoint") String endpoint, @PathParam("phoneNumber") String phoneNumber, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Get("/phoneNumbers") @@ -214,48 +228,78 @@ Mono> listPhoneNumbers( @QueryParam("skip") Integer skip, @QueryParam("top") Integer top, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, + Context context); + + @Post("/operatorInformation/:search") + @ExpectedResponses({200}) + @UnexpectedResponseExceptionType(CommunicationErrorResponseException.class) + Mono> operatorInformationSearch( + @HostParam("endpoint") String endpoint, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") OperatorInformationRequest body, + @HeaderParam("Accept") String accept, Context context); @Get("{nextLink}") @ExpectedResponses({200}) @UnexpectedResponseExceptionType(CommunicationErrorResponseException.class) Mono> listAreaCodesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("endpoint") String endpoint, + @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, + Context context); @Get("{nextLink}") @ExpectedResponses({200}) @UnexpectedResponseExceptionType(CommunicationErrorResponseException.class) Mono> listAvailableCountriesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("endpoint") String endpoint, + @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, + Context context); @Get("{nextLink}") @ExpectedResponses({200}) @UnexpectedResponseExceptionType(CommunicationErrorResponseException.class) Mono> listAvailableLocalitiesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("endpoint") String endpoint, + @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, + Context context); @Get("{nextLink}") @ExpectedResponses({200}) @UnexpectedResponseExceptionType(CommunicationErrorResponseException.class) Mono> listOfferingsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("endpoint") String endpoint, + @HeaderParam("accept-language") String acceptLanguage, + @HeaderParam("Accept") String accept, + Context context); @Get("{nextLink}") @ExpectedResponses({200}) @UnexpectedResponseExceptionType(CommunicationErrorResponseException.class) Mono> listPhoneNumbersNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("endpoint") String endpoint, + @HeaderParam("Accept") String accept, + Context context); } /** * Gets the list of available area codes. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param phoneNumberType Represents the number type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param assignmentType Represents the assignment type of the offering. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param locality The name of locality or town in which to search for the area code. This is required if the number * type is Geographic. * @param administrativeDivision The name of the state or province in which to search for the area code. @@ -263,7 +307,8 @@ Mono> listPhoneNumbersNext( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAreaCodesSinglePageAsync( @@ -275,6 +320,7 @@ public Mono> listAreaCodesSinglePageAsync( String locality, String administrativeDivision, String acceptLanguage) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.listAreaCodes( @@ -288,6 +334,7 @@ public Mono> listAreaCodesSinglePageAsync( administrativeDivision, this.client.getApiVersion(), acceptLanguage, + accept, context)) .map( res -> @@ -304,11 +351,11 @@ public Mono> listAreaCodesSinglePageAsync( * Gets the list of available area codes. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param phoneNumberType Represents the number type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param assignmentType Represents the assignment type of the offering. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param locality The name of locality or town in which to search for the area code. This is required if the number * type is Geographic. * @param administrativeDivision The name of the state or province in which to search for the area code. @@ -317,7 +364,8 @@ public Mono> listAreaCodesSinglePageAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAreaCodesSinglePageAsync( @@ -330,6 +378,7 @@ public Mono> listAreaCodesSinglePageAsync( String administrativeDivision, String acceptLanguage, Context context) { + final String accept = "application/json"; return service.listAreaCodes( this.client.getEndpoint(), countryCode, @@ -341,6 +390,7 @@ public Mono> listAreaCodesSinglePageAsync( administrativeDivision, this.client.getApiVersion(), acceptLanguage, + accept, context) .map( res -> @@ -357,11 +407,11 @@ public Mono> listAreaCodesSinglePageAsync( * Gets the list of available area codes. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param phoneNumberType Represents the number type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param assignmentType Represents the assignment type of the offering. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param locality The name of locality or town in which to search for the area code. This is required if the number * type is Geographic. * @param administrativeDivision The name of the state or province in which to search for the area code. @@ -369,7 +419,7 @@ public Mono> listAreaCodesSinglePageAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listAreaCodesAsync( @@ -392,18 +442,18 @@ public PagedFlux listAreaCodesAsync( locality, administrativeDivision, acceptLanguage), - nextLink -> listAreaCodesNextSinglePageAsync(nextLink)); + nextLink -> listAreaCodesNextSinglePageAsync(nextLink, acceptLanguage)); } /** * Gets the list of available area codes. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param phoneNumberType Represents the number type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param assignmentType Represents the assignment type of the offering. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param locality The name of locality or town in which to search for the area code. This is required if the number * type is Geographic. * @param administrativeDivision The name of the state or province in which to search for the area code. @@ -412,7 +462,7 @@ public PagedFlux listAreaCodesAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listAreaCodesAsync( @@ -437,18 +487,18 @@ public PagedFlux listAreaCodesAsync( administrativeDivision, acceptLanguage, context), - nextLink -> listAreaCodesNextSinglePageAsync(nextLink, context)); + nextLink -> listAreaCodesNextSinglePageAsync(nextLink, acceptLanguage, context)); } /** * Gets the list of available area codes. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param phoneNumberType Represents the number type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param assignmentType Represents the assignment type of the offering. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param locality The name of locality or town in which to search for the area code. This is required if the number * type is Geographic. * @param administrativeDivision The name of the state or province in which to search for the area code. @@ -456,7 +506,7 @@ public PagedFlux listAreaCodesAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAreaCodes( @@ -484,11 +534,11 @@ public PagedIterable listAreaCodes( * Gets the list of available area codes. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param phoneNumberType Represents the number type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param assignmentType Represents the assignment type of the offering. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param locality The name of locality or town in which to search for the area code. This is required if the number * type is Geographic. * @param administrativeDivision The name of the state or province in which to search for the area code. @@ -497,7 +547,7 @@ public PagedIterable listAreaCodes( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAreaCodes( @@ -533,11 +583,13 @@ public PagedIterable listAreaCodes( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of supported countries. + * @return the list of supported countries along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAvailableCountriesSinglePageAsync( Integer skip, Integer maxPageSize, String acceptLanguage) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.listAvailableCountries( @@ -546,6 +598,7 @@ public Mono> listAvailableCountriesSinglePageA maxPageSize, this.client.getApiVersion(), acceptLanguage, + accept, context)) .map( res -> @@ -569,17 +622,20 @@ public Mono> listAvailableCountriesSinglePageA * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of supported countries. + * @return the list of supported countries along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAvailableCountriesSinglePageAsync( Integer skip, Integer maxPageSize, String acceptLanguage, Context context) { + final String accept = "application/json"; return service.listAvailableCountries( this.client.getEndpoint(), skip, maxPageSize, this.client.getApiVersion(), acceptLanguage, + accept, context) .map( res -> @@ -602,14 +658,14 @@ public Mono> listAvailableCountriesSinglePageA * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of supported countries. + * @return the list of supported countries as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listAvailableCountriesAsync( Integer skip, Integer maxPageSize, String acceptLanguage) { return new PagedFlux<>( () -> listAvailableCountriesSinglePageAsync(skip, maxPageSize, acceptLanguage), - nextLink -> listAvailableCountriesNextSinglePageAsync(nextLink)); + nextLink -> listAvailableCountriesNextSinglePageAsync(nextLink, acceptLanguage)); } /** @@ -623,14 +679,14 @@ public PagedFlux listAvailableCountriesAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of supported countries. + * @return the list of supported countries as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listAvailableCountriesAsync( Integer skip, Integer maxPageSize, String acceptLanguage, Context context) { return new PagedFlux<>( () -> listAvailableCountriesSinglePageAsync(skip, maxPageSize, acceptLanguage, context), - nextLink -> listAvailableCountriesNextSinglePageAsync(nextLink, context)); + nextLink -> listAvailableCountriesNextSinglePageAsync(nextLink, acceptLanguage, context)); } /** @@ -643,7 +699,7 @@ public PagedFlux listAvailableCountriesAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of supported countries. + * @return the list of supported countries as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAvailableCountries( @@ -662,7 +718,7 @@ public PagedIterable listAvailableCountries( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of supported countries. + * @return the list of supported countries as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAvailableCountries( @@ -683,7 +739,8 @@ public PagedIterable listAvailableCountries( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of cities or towns with available phone numbers. + * @return the list of cities or towns with available phone numbers along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAvailableLocalitiesSinglePageAsync( @@ -692,6 +749,7 @@ public Mono> listAvailableLocalitiesSinglePag Integer maxPageSize, String administrativeDivision, String acceptLanguage) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.listAvailableLocalities( @@ -702,6 +760,7 @@ public Mono> listAvailableLocalitiesSinglePag administrativeDivision, this.client.getApiVersion(), acceptLanguage, + accept, context)) .map( res -> @@ -728,7 +787,8 @@ public Mono> listAvailableLocalitiesSinglePag * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of cities or towns with available phone numbers. + * @return the list of cities or towns with available phone numbers along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAvailableLocalitiesSinglePageAsync( @@ -738,6 +798,7 @@ public Mono> listAvailableLocalitiesSinglePag String administrativeDivision, String acceptLanguage, Context context) { + final String accept = "application/json"; return service.listAvailableLocalities( this.client.getEndpoint(), countryCode, @@ -746,6 +807,7 @@ public Mono> listAvailableLocalitiesSinglePag administrativeDivision, this.client.getApiVersion(), acceptLanguage, + accept, context) .map( res -> @@ -771,7 +833,7 @@ public Mono> listAvailableLocalitiesSinglePag * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of cities or towns with available phone numbers. + * @return the list of cities or towns with available phone numbers as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listAvailableLocalitiesAsync( @@ -784,7 +846,7 @@ public PagedFlux listAvailableLocalitiesAsync( () -> listAvailableLocalitiesSinglePageAsync( countryCode, skip, maxPageSize, administrativeDivision, acceptLanguage), - nextLink -> listAvailableLocalitiesNextSinglePageAsync(nextLink)); + nextLink -> listAvailableLocalitiesNextSinglePageAsync(nextLink, acceptLanguage)); } /** @@ -801,7 +863,7 @@ public PagedFlux listAvailableLocalitiesAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of cities or towns with available phone numbers. + * @return the list of cities or towns with available phone numbers as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listAvailableLocalitiesAsync( @@ -815,7 +877,7 @@ public PagedFlux listAvailableLocalitiesAsync( () -> listAvailableLocalitiesSinglePageAsync( countryCode, skip, maxPageSize, administrativeDivision, acceptLanguage, context), - nextLink -> listAvailableLocalitiesNextSinglePageAsync(nextLink, context)); + nextLink -> listAvailableLocalitiesNextSinglePageAsync(nextLink, acceptLanguage, context)); } /** @@ -831,7 +893,8 @@ public PagedFlux listAvailableLocalitiesAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of cities or towns with available phone numbers. + * @return the list of cities or towns with available phone numbers as paginated response with {@link + * PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAvailableLocalities( @@ -858,7 +921,8 @@ public PagedIterable listAvailableLocalities( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of cities or towns with available phone numbers. + * @return the list of cities or towns with available phone numbers as paginated response with {@link + * PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAvailableLocalities( @@ -880,13 +944,14 @@ public PagedIterable listAvailableLocalities( * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param phoneNumberType Represents the number type of the offering. - * @param assignmentType Represents the assignment type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listOfferingsSinglePageAsync( @@ -896,6 +961,7 @@ public Mono> listOfferingsSinglePageAsync( PhoneNumberType phoneNumberType, PhoneNumberAssignmentType assignmentType, String acceptLanguage) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.listOfferings( @@ -907,6 +973,7 @@ public Mono> listOfferingsSinglePageAsync( assignmentType, this.client.getApiVersion(), acceptLanguage, + accept, context)) .map( res -> @@ -926,14 +993,15 @@ public Mono> listOfferingsSinglePageAsync( * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param phoneNumberType Represents the number type of the offering. - * @param assignmentType Represents the assignment type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listOfferingsSinglePageAsync( @@ -944,6 +1012,7 @@ public Mono> listOfferingsSinglePageAsync( PhoneNumberAssignmentType assignmentType, String acceptLanguage, Context context) { + final String accept = "application/json"; return service.listOfferings( this.client.getEndpoint(), countryCode, @@ -953,6 +1022,7 @@ public Mono> listOfferingsSinglePageAsync( assignmentType, this.client.getApiVersion(), acceptLanguage, + accept, context) .map( res -> @@ -972,13 +1042,13 @@ public Mono> listOfferingsSinglePageAsync( * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param phoneNumberType Represents the number type of the offering. - * @param assignmentType Represents the assignment type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listOfferingsAsync( @@ -992,7 +1062,7 @@ public PagedFlux listOfferingsAsync( () -> listOfferingsSinglePageAsync( countryCode, skip, maxPageSize, phoneNumberType, assignmentType, acceptLanguage), - nextLink -> listOfferingsNextSinglePageAsync(nextLink)); + nextLink -> listOfferingsNextSinglePageAsync(nextLink, acceptLanguage)); } /** @@ -1002,14 +1072,14 @@ public PagedFlux listOfferingsAsync( * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param phoneNumberType Represents the number type of the offering. - * @param assignmentType Represents the assignment type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listOfferingsAsync( @@ -1030,7 +1100,7 @@ public PagedFlux listOfferingsAsync( assignmentType, acceptLanguage, context), - nextLink -> listOfferingsNextSinglePageAsync(nextLink, context)); + nextLink -> listOfferingsNextSinglePageAsync(nextLink, acceptLanguage, context)); } /** @@ -1040,13 +1110,13 @@ public PagedFlux listOfferingsAsync( * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param phoneNumberType Represents the number type of the offering. - * @param assignmentType Represents the assignment type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listOfferings( @@ -1067,14 +1137,14 @@ public PagedIterable listOfferings( * @param skip An optional parameter for how many entries to skip, for pagination purposes. The default value is 0. * @param maxPageSize An optional parameter for how many entries to return, for pagination purposes. The default * value is 100. - * @param phoneNumberType Represents the number type of the offering. - * @param assignmentType Represents the assignment type of the offering. + * @param phoneNumberType Filter by numberType, e.g. Geographic, TollFree. + * @param assignmentType Filter by assignmentType, e.g. Person, Application. * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listOfferings( @@ -1094,98 +1164,86 @@ public PagedIterable listOfferings( * Search for available phone numbers to purchase. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param body Represents a phone number search request to find phone numbers. Found phone numbers are temporarily - * held for a following purchase. + * @param body The phone number search request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the result of a phone number search operation. + * @return the result of a phone number search operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono searchAvailablePhoneNumbersWithResponseAsync( String countryCode, PhoneNumberSearchRequest body) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.searchAvailablePhoneNumbers( - this.client.getEndpoint(), countryCode, this.client.getApiVersion(), body, context)); + this.client.getEndpoint(), + countryCode, + this.client.getApiVersion(), + body, + accept, + context)); } /** * Search for available phone numbers to purchase. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param body Represents a phone number search request to find phone numbers. Found phone numbers are temporarily - * held for a following purchase. + * @param body The phone number search request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the result of a phone number search operation. + * @return the result of a phone number search operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono searchAvailablePhoneNumbersWithResponseAsync( String countryCode, PhoneNumberSearchRequest body, Context context) { + final String accept = "application/json"; return service.searchAvailablePhoneNumbers( - this.client.getEndpoint(), countryCode, this.client.getApiVersion(), body, context); + this.client.getEndpoint(), countryCode, this.client.getApiVersion(), body, accept, context); } /** * Search for available phone numbers to purchase. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param body Represents a phone number search request to find phone numbers. Found phone numbers are temporarily - * held for a following purchase. + * @param body The phone number search request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the result of a phone number search operation. + * @return the result of a phone number search operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono searchAvailablePhoneNumbersAsync( String countryCode, PhoneNumberSearchRequest body) { return searchAvailablePhoneNumbersWithResponseAsync(countryCode, body) - .flatMap( - (PhoneNumbersSearchAvailablePhoneNumbersResponse res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Search for available phone numbers to purchase. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param body Represents a phone number search request to find phone numbers. Found phone numbers are temporarily - * held for a following purchase. + * @param body The phone number search request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the result of a phone number search operation. + * @return the result of a phone number search operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono searchAvailablePhoneNumbersAsync( String countryCode, PhoneNumberSearchRequest body, Context context) { return searchAvailablePhoneNumbersWithResponseAsync(countryCode, body, context) - .flatMap( - (PhoneNumbersSearchAvailablePhoneNumbersResponse res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Search for available phone numbers to purchase. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param body Represents a phone number search request to find phone numbers. Found phone numbers are temporarily - * held for a following purchase. + * @param body The phone number search request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1200,8 +1258,7 @@ public PhoneNumberSearchResult searchAvailablePhoneNumbers(String countryCode, P * Search for available phone numbers to purchase. * * @param countryCode The ISO 3166-2 country code, e.g. US. - * @param body Represents a phone number search request to find phone numbers. Found phone numbers are temporarily - * held for a following purchase. + * @param body The phone number search request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. @@ -1209,9 +1266,9 @@ public PhoneNumberSearchResult searchAvailablePhoneNumbers(String countryCode, P * @return the result of a phone number search operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PhoneNumberSearchResult searchAvailablePhoneNumbers( + public PhoneNumbersSearchAvailablePhoneNumbersResponse searchAvailablePhoneNumbersWithResponse( String countryCode, PhoneNumberSearchRequest body, Context context) { - return searchAvailablePhoneNumbersAsync(countryCode, body, context).block(); + return searchAvailablePhoneNumbersWithResponseAsync(countryCode, body, context).block(); } /** @@ -1221,14 +1278,16 @@ public PhoneNumberSearchResult searchAvailablePhoneNumbers( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a phone number search result by search id. + * @return a phone number search result by search id along with {@link Response} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> getSearchResultWithResponseAsync(String searchId) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.getSearchResult( - this.client.getEndpoint(), searchId, this.client.getApiVersion(), context)); + this.client.getEndpoint(), searchId, this.client.getApiVersion(), accept, context)); } /** @@ -1239,11 +1298,14 @@ public Mono> getSearchResultWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a phone number search result by search id. + * @return a phone number search result by search id along with {@link Response} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> getSearchResultWithResponseAsync(String searchId, Context context) { - return service.getSearchResult(this.client.getEndpoint(), searchId, this.client.getApiVersion(), context); + final String accept = "application/json"; + return service.getSearchResult( + this.client.getEndpoint(), searchId, this.client.getApiVersion(), accept, context); } /** @@ -1253,19 +1315,11 @@ public Mono> getSearchResultWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a phone number search result by search id. + * @return a phone number search result by search id on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getSearchResultAsync(String searchId) { - return getSearchResultWithResponseAsync(searchId) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return getSearchResultWithResponseAsync(searchId).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1276,19 +1330,11 @@ public Mono getSearchResultAsync(String searchId) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a phone number search result by search id. + * @return a phone number search result by search id on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getSearchResultAsync(String searchId, Context context) { - return getSearchResultWithResponseAsync(searchId, context) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return getSearchResultWithResponseAsync(searchId, context).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1313,82 +1359,83 @@ public PhoneNumberSearchResult getSearchResult(String searchId) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a phone number search result by search id. + * @return a phone number search result by search id along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PhoneNumberSearchResult getSearchResult(String searchId, Context context) { - return getSearchResultAsync(searchId, context).block(); + public Response getSearchResultWithResponse(String searchId, Context context) { + return getSearchResultWithResponseAsync(searchId, context).block(); } /** * Purchases phone numbers. * - * @param body The phone number search purchase request. + * @param body The phone number purchase request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono purchasePhoneNumbersWithResponseAsync( PhoneNumberPurchaseRequest body) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.purchasePhoneNumbers( - this.client.getEndpoint(), this.client.getApiVersion(), body, context)); + this.client.getEndpoint(), this.client.getApiVersion(), body, accept, context)); } /** * Purchases phone numbers. * - * @param body The phone number search purchase request. + * @param body The phone number purchase request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono purchasePhoneNumbersWithResponseAsync( PhoneNumberPurchaseRequest body, Context context) { - return service.purchasePhoneNumbers(this.client.getEndpoint(), this.client.getApiVersion(), body, context); + final String accept = "application/json"; + return service.purchasePhoneNumbers( + this.client.getEndpoint(), this.client.getApiVersion(), body, accept, context); } /** * Purchases phone numbers. * - * @param body The phone number search purchase request. + * @param body The phone number purchase request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono purchasePhoneNumbersAsync(PhoneNumberPurchaseRequest body) { - return purchasePhoneNumbersWithResponseAsync(body) - .flatMap((PhoneNumbersPurchasePhoneNumbersResponse res) -> Mono.empty()); + return purchasePhoneNumbersWithResponseAsync(body).flatMap(ignored -> Mono.empty()); } /** * Purchases phone numbers. * - * @param body The phone number search purchase request. + * @param body The phone number purchase request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono purchasePhoneNumbersAsync(PhoneNumberPurchaseRequest body, Context context) { - return purchasePhoneNumbersWithResponseAsync(body, context) - .flatMap((PhoneNumbersPurchasePhoneNumbersResponse res) -> Mono.empty()); + return purchasePhoneNumbersWithResponseAsync(body, context).flatMap(ignored -> Mono.empty()); } /** * Purchases phone numbers. * - * @param body The phone number search purchase request. + * @param body The phone number purchase request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1401,15 +1448,17 @@ public void purchasePhoneNumbers(PhoneNumberPurchaseRequest body) { /** * Purchases phone numbers. * - * @param body The phone number search purchase request. + * @param body The phone number purchase request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void purchasePhoneNumbers(PhoneNumberPurchaseRequest body, Context context) { - purchasePhoneNumbersAsync(body, context).block(); + public PhoneNumbersPurchasePhoneNumbersResponse purchasePhoneNumbersWithResponse( + PhoneNumberPurchaseRequest body, Context context) { + return purchasePhoneNumbersWithResponseAsync(body, context).block(); } /** @@ -1419,14 +1468,15 @@ public void purchasePhoneNumbers(PhoneNumberPurchaseRequest body, Context contex * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an operation by its id. + * @return an operation by its id on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getOperationWithResponseAsync(String operationId) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.getOperation( - this.client.getEndpoint(), operationId, this.client.getApiVersion(), context)); + this.client.getEndpoint(), operationId, this.client.getApiVersion(), accept, context)); } /** @@ -1437,11 +1487,13 @@ public Mono getOperationWithResponseAsync(Stri * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an operation by its id. + * @return an operation by its id on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getOperationWithResponseAsync(String operationId, Context context) { - return service.getOperation(this.client.getEndpoint(), operationId, this.client.getApiVersion(), context); + final String accept = "application/json"; + return service.getOperation( + this.client.getEndpoint(), operationId, this.client.getApiVersion(), accept, context); } /** @@ -1451,19 +1503,11 @@ public Mono getOperationWithResponseAsync(Stri * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an operation by its id. + * @return an operation by its id on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getOperationAsync(String operationId) { - return getOperationWithResponseAsync(operationId) - .flatMap( - (PhoneNumbersGetOperationResponse res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return getOperationWithResponseAsync(operationId).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1474,19 +1518,11 @@ public Mono getOperationAsync(String operationId) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an operation by its id. + * @return an operation by its id on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getOperationAsync(String operationId, Context context) { - return getOperationWithResponseAsync(operationId, context) - .flatMap( - (PhoneNumbersGetOperationResponse res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return getOperationWithResponseAsync(operationId, context).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1514,8 +1550,8 @@ public PhoneNumberRawOperation getOperation(String operationId) { * @return an operation by its id. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PhoneNumberRawOperation getOperation(String operationId, Context context) { - return getOperationAsync(operationId, context).block(); + public PhoneNumbersGetOperationResponse getOperationWithResponse(String operationId, Context context) { + return getOperationWithResponseAsync(operationId, context).block(); } /** @@ -1525,14 +1561,15 @@ public PhoneNumberRawOperation getOperation(String operationId, Context context) * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> cancelOperationWithResponseAsync(String operationId) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.cancelOperation( - this.client.getEndpoint(), operationId, this.client.getApiVersion(), context)); + this.client.getEndpoint(), operationId, this.client.getApiVersion(), accept, context)); } /** @@ -1543,11 +1580,13 @@ public Mono> cancelOperationWithResponseAsync(String operationId) * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> cancelOperationWithResponseAsync(String operationId, Context context) { - return service.cancelOperation(this.client.getEndpoint(), operationId, this.client.getApiVersion(), context); + final String accept = "application/json"; + return service.cancelOperation( + this.client.getEndpoint(), operationId, this.client.getApiVersion(), accept, context); } /** @@ -1557,11 +1596,11 @@ public Mono> cancelOperationWithResponseAsync(String operationId, * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono cancelOperationAsync(String operationId) { - return cancelOperationWithResponseAsync(operationId).flatMap((Response res) -> Mono.empty()); + return cancelOperationWithResponseAsync(operationId).flatMap(ignored -> Mono.empty()); } /** @@ -1572,11 +1611,11 @@ public Mono cancelOperationAsync(String operationId) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono cancelOperationAsync(String operationId, Context context) { - return cancelOperationWithResponseAsync(operationId, context).flatMap((Response res) -> Mono.empty()); + return cancelOperationWithResponseAsync(operationId, context).flatMap(ignored -> Mono.empty()); } /** @@ -1600,10 +1639,11 @@ public void cancelOperation(String operationId) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void cancelOperation(String operationId, Context context) { - cancelOperationAsync(operationId, context).block(); + public Response cancelOperationWithResponse(String operationId, Context context) { + return cancelOperationWithResponseAsync(operationId, context).block(); } /** @@ -1611,19 +1651,25 @@ public void cancelOperation(String operationId, Context context) { * * @param phoneNumber The phone number id in E.164 format. The leading plus can be either + or encoded as %2B, e.g. * +11234567890. - * @param body Capabilities of a phone number. + * @param body Defines the update capabilities request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a purchased phone number. + * @return represents a purchased phone number on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono updateCapabilitiesWithResponseAsync( String phoneNumber, PhoneNumberCapabilitiesRequest body) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.updateCapabilities( - this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), body, context)); + this.client.getEndpoint(), + phoneNumber, + this.client.getApiVersion(), + body, + accept, + context)); } /** @@ -1631,18 +1677,19 @@ public Mono updateCapabilitiesWithRespon * * @param phoneNumber The phone number id in E.164 format. The leading plus can be either + or encoded as %2B, e.g. * +11234567890. - * @param body Capabilities of a phone number. + * @param body Defines the update capabilities request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a purchased phone number. + * @return represents a purchased phone number on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono updateCapabilitiesWithResponseAsync( String phoneNumber, PhoneNumberCapabilitiesRequest body, Context context) { + final String accept = "application/json"; return service.updateCapabilities( - this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), body, context); + this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), body, accept, context); } /** @@ -1650,23 +1697,15 @@ public Mono updateCapabilitiesWithRespon * * @param phoneNumber The phone number id in E.164 format. The leading plus can be either + or encoded as %2B, e.g. * +11234567890. - * @param body Capabilities of a phone number. + * @param body Defines the update capabilities request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a purchased phone number. + * @return represents a purchased phone number on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono updateCapabilitiesAsync(String phoneNumber, PhoneNumberCapabilitiesRequest body) { - return updateCapabilitiesWithResponseAsync(phoneNumber, body) - .flatMap( - (PhoneNumbersUpdateCapabilitiesResponse res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return updateCapabilitiesWithResponseAsync(phoneNumber, body).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1674,25 +1713,18 @@ public Mono updateCapabilitiesAsync(String phoneNumber, Ph * * @param phoneNumber The phone number id in E.164 format. The leading plus can be either + or encoded as %2B, e.g. * +11234567890. - * @param body Capabilities of a phone number. + * @param body Defines the update capabilities request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a purchased phone number. + * @return represents a purchased phone number on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono updateCapabilitiesAsync( String phoneNumber, PhoneNumberCapabilitiesRequest body, Context context) { return updateCapabilitiesWithResponseAsync(phoneNumber, body, context) - .flatMap( - (PhoneNumbersUpdateCapabilitiesResponse res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1700,7 +1732,7 @@ public Mono updateCapabilitiesAsync( * * @param phoneNumber The phone number id in E.164 format. The leading plus can be either + or encoded as %2B, e.g. * +11234567890. - * @param body Capabilities of a phone number. + * @param body Defines the update capabilities request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1716,7 +1748,7 @@ public PurchasedPhoneNumber updateCapabilities(String phoneNumber, PhoneNumberCa * * @param phoneNumber The phone number id in E.164 format. The leading plus can be either + or encoded as %2B, e.g. * +11234567890. - * @param body Capabilities of a phone number. + * @param body Defines the update capabilities request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. @@ -1724,9 +1756,9 @@ public PurchasedPhoneNumber updateCapabilities(String phoneNumber, PhoneNumberCa * @return represents a purchased phone number. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PurchasedPhoneNumber updateCapabilities( + public PhoneNumbersUpdateCapabilitiesResponse updateCapabilitiesWithResponse( String phoneNumber, PhoneNumberCapabilitiesRequest body, Context context) { - return updateCapabilitiesAsync(phoneNumber, body, context).block(); + return updateCapabilitiesWithResponseAsync(phoneNumber, body, context).block(); } /** @@ -1736,14 +1768,16 @@ public PurchasedPhoneNumber updateCapabilities( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the details of the given purchased phone number. + * @return the details of the given purchased phone number along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> getByNumberWithResponseAsync(String phoneNumber) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.getByNumber( - this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), context)); + this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), accept, context)); } /** @@ -1754,11 +1788,14 @@ public Mono> getByNumberWithResponseAsync(String * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the details of the given purchased phone number. + * @return the details of the given purchased phone number along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> getByNumberWithResponseAsync(String phoneNumber, Context context) { - return service.getByNumber(this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), context); + final String accept = "application/json"; + return service.getByNumber( + this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), accept, context); } /** @@ -1768,19 +1805,11 @@ public Mono> getByNumberWithResponseAsync(String * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the details of the given purchased phone number. + * @return the details of the given purchased phone number on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getByNumberAsync(String phoneNumber) { - return getByNumberWithResponseAsync(phoneNumber) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return getByNumberWithResponseAsync(phoneNumber).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1791,19 +1820,11 @@ public Mono getByNumberAsync(String phoneNumber) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the details of the given purchased phone number. + * @return the details of the given purchased phone number on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono getByNumberAsync(String phoneNumber, Context context) { - return getByNumberWithResponseAsync(phoneNumber, context) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return getByNumberWithResponseAsync(phoneNumber, context).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** @@ -1828,11 +1849,11 @@ public PurchasedPhoneNumber getByNumber(String phoneNumber) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the details of the given purchased phone number. + * @return the details of the given purchased phone number along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PurchasedPhoneNumber getByNumber(String phoneNumber, Context context) { - return getByNumberAsync(phoneNumber, context).block(); + public Response getByNumberWithResponse(String phoneNumber, Context context) { + return getByNumberWithResponseAsync(phoneNumber, context).block(); } /** @@ -1842,14 +1863,15 @@ public PurchasedPhoneNumber getByNumber(String phoneNumber, Context context) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono releasePhoneNumberWithResponseAsync(String phoneNumber) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.releasePhoneNumber( - this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), context)); + this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), accept, context)); } /** @@ -1860,12 +1882,14 @@ public Mono releasePhoneNumberWithRespon * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono releasePhoneNumberWithResponseAsync( String phoneNumber, Context context) { - return service.releasePhoneNumber(this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), context); + final String accept = "application/json"; + return service.releasePhoneNumber( + this.client.getEndpoint(), phoneNumber, this.client.getApiVersion(), accept, context); } /** @@ -1875,12 +1899,11 @@ public Mono releasePhoneNumberWithRespon * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono releasePhoneNumberAsync(String phoneNumber) { - return releasePhoneNumberWithResponseAsync(phoneNumber) - .flatMap((PhoneNumbersReleasePhoneNumberResponse res) -> Mono.empty()); + return releasePhoneNumberWithResponseAsync(phoneNumber).flatMap(ignored -> Mono.empty()); } /** @@ -1891,12 +1914,11 @@ public Mono releasePhoneNumberAsync(String phoneNumber) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono releasePhoneNumberAsync(String phoneNumber, Context context) { - return releasePhoneNumberWithResponseAsync(phoneNumber, context) - .flatMap((PhoneNumbersReleasePhoneNumberResponse res) -> Mono.empty()); + return releasePhoneNumberWithResponseAsync(phoneNumber, context).flatMap(ignored -> Mono.empty()); } /** @@ -1920,10 +1942,11 @@ public void releasePhoneNumber(String phoneNumber) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void releasePhoneNumber(String phoneNumber, Context context) { - releasePhoneNumberAsync(phoneNumber, context).block(); + public PhoneNumbersReleasePhoneNumberResponse releasePhoneNumberWithResponse(String phoneNumber, Context context) { + return releasePhoneNumberWithResponseAsync(phoneNumber, context).block(); } /** @@ -1935,14 +1958,21 @@ public void releasePhoneNumber(String phoneNumber, Context context) { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all purchased phone numbers. + * @return the list of all purchased phone numbers along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listPhoneNumbersSinglePageAsync(Integer skip, Integer top) { + final String accept = "application/json"; return FluxUtil.withContext( context -> service.listPhoneNumbers( - this.client.getEndpoint(), skip, top, this.client.getApiVersion(), context)) + this.client.getEndpoint(), + skip, + top, + this.client.getApiVersion(), + accept, + context)) .map( res -> new PagedResponseBase<>( @@ -1964,12 +1994,15 @@ public Mono> listPhoneNumbersSinglePageAsync * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all purchased phone numbers. + * @return the list of all purchased phone numbers along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listPhoneNumbersSinglePageAsync( Integer skip, Integer top, Context context) { - return service.listPhoneNumbers(this.client.getEndpoint(), skip, top, this.client.getApiVersion(), context) + final String accept = "application/json"; + return service.listPhoneNumbers( + this.client.getEndpoint(), skip, top, this.client.getApiVersion(), accept, context) .map( res -> new PagedResponseBase<>( @@ -1990,7 +2023,7 @@ public Mono> listPhoneNumbersSinglePageAsync * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all purchased phone numbers. + * @return the list of all purchased phone numbers as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listPhoneNumbersAsync(Integer skip, Integer top) { @@ -2009,7 +2042,7 @@ public PagedFlux listPhoneNumbersAsync(Integer skip, Integ * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all purchased phone numbers. + * @return the list of all purchased phone numbers as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedFlux listPhoneNumbersAsync(Integer skip, Integer top, Context context) { @@ -2027,7 +2060,7 @@ public PagedFlux listPhoneNumbersAsync(Integer skip, Integ * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all purchased phone numbers. + * @return the list of all purchased phone numbers as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listPhoneNumbers(Integer skip, Integer top) { @@ -2044,25 +2077,136 @@ public PagedIterable listPhoneNumbers(Integer skip, Intege * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all purchased phone numbers. + * @return the list of all purchased phone numbers as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listPhoneNumbers(Integer skip, Integer top, Context context) { return new PagedIterable<>(listPhoneNumbersAsync(skip, top, context)); } + /** + * Searches for number format and operator information for a given list of phone numbers. + * + * @param body The phone number(s) whose number format and operator information should be searched. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws CommunicationErrorResponseException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a search result containing format and operator information associated with the requested phone + * numbers along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Mono> operatorInformationSearchWithResponseAsync( + OperatorInformationRequest body) { + final String accept = "application/json"; + return FluxUtil.withContext( + context -> + service.operatorInformationSearch( + this.client.getEndpoint(), this.client.getApiVersion(), body, accept, context)); + } + + /** + * Searches for number format and operator information for a given list of phone numbers. + * + * @param body The phone number(s) whose number format and operator information should be searched. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws CommunicationErrorResponseException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a search result containing format and operator information associated with the requested phone + * numbers along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Mono> operatorInformationSearchWithResponseAsync( + OperatorInformationRequest body, Context context) { + final String accept = "application/json"; + return service.operatorInformationSearch( + this.client.getEndpoint(), this.client.getApiVersion(), body, accept, context); + } + + /** + * Searches for number format and operator information for a given list of phone numbers. + * + * @param body The phone number(s) whose number format and operator information should be searched. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws CommunicationErrorResponseException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a search result containing format and operator information associated with the requested phone + * numbers on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Mono operatorInformationSearchAsync(OperatorInformationRequest body) { + return operatorInformationSearchWithResponseAsync(body).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Searches for number format and operator information for a given list of phone numbers. + * + * @param body The phone number(s) whose number format and operator information should be searched. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws CommunicationErrorResponseException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a search result containing format and operator information associated with the requested phone + * numbers on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Mono operatorInformationSearchAsync( + OperatorInformationRequest body, Context context) { + return operatorInformationSearchWithResponseAsync(body, context) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Searches for number format and operator information for a given list of phone numbers. + * + * @param body The phone number(s) whose number format and operator information should be searched. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws CommunicationErrorResponseException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a search result containing format and operator information associated with the requested phone + * numbers. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public OperatorInformationResult operatorInformationSearch(OperatorInformationRequest body) { + return operatorInformationSearchAsync(body).block(); + } + + /** + * Searches for number format and operator information for a given list of phone numbers. + * + * @param body The phone number(s) whose number format and operator information should be searched. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws CommunicationErrorResponseException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a search result containing format and operator information associated with the requested phone + * numbers along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response operatorInformationSearchWithResponse( + OperatorInformationRequest body, Context context) { + return operatorInformationSearchWithResponseAsync(body, context).block(); + } + /** * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> listAreaCodesNextSinglePageAsync(String nextLink) { - return FluxUtil.withContext(context -> service.listAreaCodesNext(nextLink, context)) + public Mono> listAreaCodesNextSinglePageAsync( + String nextLink, String acceptLanguage) { + final String accept = "application/json"; + return FluxUtil.withContext( + context -> + service.listAreaCodesNext( + nextLink, this.client.getEndpoint(), acceptLanguage, accept, context)) .map( res -> new PagedResponseBase<>( @@ -2078,15 +2222,19 @@ public Mono> listAreaCodesNextSinglePageAsync * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of available area codes. + * @return the list of available area codes along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> listAreaCodesNextSinglePageAsync(String nextLink, Context context) { - return service.listAreaCodesNext(nextLink, context) + public Mono> listAreaCodesNextSinglePageAsync( + String nextLink, String acceptLanguage, Context context) { + final String accept = "application/json"; + return service.listAreaCodesNext(nextLink, this.client.getEndpoint(), acceptLanguage, accept, context) .map( res -> new PagedResponseBase<>( @@ -2102,14 +2250,21 @@ public Mono> listAreaCodesNextSinglePageAsync * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of countries. + * @return represents a wrapper around a list of countries along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> listAvailableCountriesNextSinglePageAsync(String nextLink) { - return FluxUtil.withContext(context -> service.listAvailableCountriesNext(nextLink, context)) + public Mono> listAvailableCountriesNextSinglePageAsync( + String nextLink, String acceptLanguage) { + final String accept = "application/json"; + return FluxUtil.withContext( + context -> + service.listAvailableCountriesNext( + nextLink, this.client.getEndpoint(), acceptLanguage, accept, context)) .map( res -> new PagedResponseBase<>( @@ -2125,16 +2280,19 @@ public Mono> listAvailableCountriesNextSingleP * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of countries. + * @return represents a wrapper around a list of countries along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAvailableCountriesNextSinglePageAsync( - String nextLink, Context context) { - return service.listAvailableCountriesNext(nextLink, context) + String nextLink, String acceptLanguage, Context context) { + final String accept = "application/json"; + return service.listAvailableCountriesNext(nextLink, this.client.getEndpoint(), acceptLanguage, accept, context) .map( res -> new PagedResponseBase<>( @@ -2150,14 +2308,21 @@ public Mono> listAvailableCountriesNextSingleP * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of cities or towns. + * @return represents a wrapper around a list of cities or towns along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> listAvailableLocalitiesNextSinglePageAsync(String nextLink) { - return FluxUtil.withContext(context -> service.listAvailableLocalitiesNext(nextLink, context)) + public Mono> listAvailableLocalitiesNextSinglePageAsync( + String nextLink, String acceptLanguage) { + final String accept = "application/json"; + return FluxUtil.withContext( + context -> + service.listAvailableLocalitiesNext( + nextLink, this.client.getEndpoint(), acceptLanguage, accept, context)) .map( res -> new PagedResponseBase<>( @@ -2173,16 +2338,19 @@ public Mono> listAvailableLocalitiesNextSingl * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of cities or towns. + * @return represents a wrapper around a list of cities or towns along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listAvailableLocalitiesNextSinglePageAsync( - String nextLink, Context context) { - return service.listAvailableLocalitiesNext(nextLink, context) + String nextLink, String acceptLanguage, Context context) { + final String accept = "application/json"; + return service.listAvailableLocalitiesNext(nextLink, this.client.getEndpoint(), acceptLanguage, accept, context) .map( res -> new PagedResponseBase<>( @@ -2198,14 +2366,21 @@ public Mono> listAvailableLocalitiesNextSingl * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> listOfferingsNextSinglePageAsync(String nextLink) { - return FluxUtil.withContext(context -> service.listOfferingsNext(nextLink, context)) + public Mono> listOfferingsNextSinglePageAsync( + String nextLink, String acceptLanguage) { + final String accept = "application/json"; + return FluxUtil.withContext( + context -> + service.listOfferingsNext( + nextLink, this.client.getEndpoint(), acceptLanguage, accept, context)) .map( res -> new PagedResponseBase<>( @@ -2221,15 +2396,19 @@ public Mono> listOfferingsNextSinglePageAsync * Get the next page of items. * * @param nextLink The nextLink parameter. + * @param acceptLanguage The locale to display in the localized fields in the response. e.g. 'en-US'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents a wrapper around a list of offerings. + * @return represents a wrapper around a list of offerings along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> listOfferingsNextSinglePageAsync(String nextLink, Context context) { - return service.listOfferingsNext(nextLink, context) + public Mono> listOfferingsNextSinglePageAsync( + String nextLink, String acceptLanguage, Context context) { + final String accept = "application/json"; + return service.listOfferingsNext(nextLink, this.client.getEndpoint(), acceptLanguage, accept, context) .map( res -> new PagedResponseBase<>( @@ -2248,11 +2427,14 @@ public Mono> listOfferingsNextSinglePageAsync * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of purchased phone numbers. + * @return the list of purchased phone numbers along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listPhoneNumbersNextSinglePageAsync(String nextLink) { - return FluxUtil.withContext(context -> service.listPhoneNumbersNext(nextLink, context)) + final String accept = "application/json"; + return FluxUtil.withContext( + context -> service.listPhoneNumbersNext(nextLink, this.client.getEndpoint(), accept, context)) .map( res -> new PagedResponseBase<>( @@ -2272,12 +2454,14 @@ public Mono> listPhoneNumbersNextSinglePageA * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws CommunicationErrorResponseException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of purchased phone numbers. + * @return the list of purchased phone numbers along with {@link PagedResponse} on successful completion of {@link + * Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Mono> listPhoneNumbersNextSinglePageAsync( String nextLink, Context context) { - return service.listPhoneNumbersNext(nextLink, context) + final String accept = "application/json"; + return service.listPhoneNumbersNext(nextLink, this.client.getEndpoint(), accept, context) .map( res -> new PagedResponseBase<>( diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationError.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationError.java index 8af08537303ef..6faa242aeb4cf 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationError.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationError.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The CommunicationError model. */ -@Generated +/** The Communication Services error. */ @Fluent public final class CommunicationError { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponse.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponse.java index 3d99b21e80f29..43cf593dca001 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponse.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponse.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The CommunicationErrorResponse model. */ +/** The Communication Services error. */ @Fluent -@Generated public final class CommunicationErrorResponse { /* * The Communication Services error. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponseException.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponseException.java index 3fc15d3aa72ca..246a19981b61b 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponseException.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/CommunicationErrorResponseException.java @@ -4,12 +4,10 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.exception.HttpResponseException; import com.azure.core.http.HttpResponse; /** Exception thrown for an invalid response with CommunicationErrorResponse information. */ -@Generated public final class CommunicationErrorResponseException extends HttpResponseException { /** * Initializes a new instance of the CommunicationErrorResponseException class. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/Error.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/Error.java new file mode 100644 index 0000000000000..8ffdcac479bf7 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/Error.java @@ -0,0 +1,83 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.implementation.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** Defines values for Error. */ +public final class Error extends ExpandableStringEnum { + /** Static value NoError for Error. */ + public static final Error NO_ERROR = fromString("NoError"); + + /** Static value UnknownErrorCode for Error. */ + public static final Error UNKNOWN_ERROR_CODE = fromString("UnknownErrorCode"); + + /** Static value OutOfStock for Error. */ + public static final Error OUT_OF_STOCK = fromString("OutOfStock"); + + /** Static value AuthorizationDenied for Error. */ + public static final Error AUTHORIZATION_DENIED = fromString("AuthorizationDenied"); + + /** Static value MissingAddress for Error. */ + public static final Error MISSING_ADDRESS = fromString("MissingAddress"); + + /** Static value InvalidAddress for Error. */ + public static final Error INVALID_ADDRESS = fromString("InvalidAddress"); + + /** Static value InvalidOfferModel for Error. */ + public static final Error INVALID_OFFER_MODEL = fromString("InvalidOfferModel"); + + /** Static value NotEnoughLicenses for Error. */ + public static final Error NOT_ENOUGH_LICENSES = fromString("NotEnoughLicenses"); + + /** Static value NoWallet for Error. */ + public static final Error NO_WALLET = fromString("NoWallet"); + + /** Static value NotEnoughCredit for Error. */ + public static final Error NOT_ENOUGH_CREDIT = fromString("NotEnoughCredit"); + + /** Static value NumbersPartiallyAcquired for Error. */ + public static final Error NUMBERS_PARTIALLY_ACQUIRED = fromString("NumbersPartiallyAcquired"); + + /** Static value AllNumbersNotAcquired for Error. */ + public static final Error ALL_NUMBERS_NOT_ACQUIRED = fromString("AllNumbersNotAcquired"); + + /** Static value ReservationExpired for Error. */ + public static final Error RESERVATION_EXPIRED = fromString("ReservationExpired"); + + /** Static value PurchaseFailed for Error. */ + public static final Error PURCHASE_FAILED = fromString("PurchaseFailed"); + + /** Static value BillingUnavailable for Error. */ + public static final Error BILLING_UNAVAILABLE = fromString("BillingUnavailable"); + + /** Static value ProvisioningFailed for Error. */ + public static final Error PROVISIONING_FAILED = fromString("ProvisioningFailed"); + + /** Static value UnknownSearchError for Error. */ + public static final Error UNKNOWN_SEARCH_ERROR = fromString("UnknownSearchError"); + + /** + * Creates or finds a Error from its string representation. + * + * @param name a name to look for. + * @return the corresponding Error. + */ + @JsonCreator + public static Error fromString(String name) { + return fromString(name, Error.class); + } + + /** + * Gets known Error values. + * + * @return known Error values. + */ + public static Collection values() { + return values(Error.class); + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OfferingsResponse.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OfferingsResponse.java index bca1348a8d91f..3d1f1d3c1cf15 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OfferingsResponse.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OfferingsResponse.java @@ -4,15 +4,13 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberOffering; import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The OfferingsResponse model. */ +/** Represents a wrapper around a list of offerings. */ @Immutable -@Generated public final class OfferingsResponse { /* * Represents the underlying list of offerings. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationOptions.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationOptions.java new file mode 100644 index 0000000000000..bb4982e04af0d --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationOptions.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.implementation.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** Represents options to modify a search request for operator information. */ +@Fluent +public final class OperatorInformationOptions { + /* + * Includes the fields operatorDetails, numberType, and isoCountryCode in + * the response. Please note: use of this option will result in additional + * costs + */ + @JsonProperty(value = "includeAdditionalOperatorDetails") + private Boolean includeAdditionalOperatorDetails; + + /** + * Get the includeAdditionalOperatorDetails property: Includes the fields operatorDetails, numberType, and + * isoCountryCode in the response. Please note: use of this option will result in additional costs. + * + * @return the includeAdditionalOperatorDetails value. + */ + public Boolean isIncludeAdditionalOperatorDetails() { + return this.includeAdditionalOperatorDetails; + } + + /** + * Set the includeAdditionalOperatorDetails property: Includes the fields operatorDetails, numberType, and + * isoCountryCode in the response. Please note: use of this option will result in additional costs. + * + * @param includeAdditionalOperatorDetails the includeAdditionalOperatorDetails value to set. + * @return the OperatorInformationOptions object itself. + */ + public OperatorInformationOptions setIncludeAdditionalOperatorDetails(Boolean includeAdditionalOperatorDetails) { + this.includeAdditionalOperatorDetails = includeAdditionalOperatorDetails; + return this; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequest.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequest.java new file mode 100644 index 0000000000000..e53765d39836a --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequest.java @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.implementation.models; + +import com.azure.communication.phonenumbers.models.OperatorInformationOptions; +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** Represents a search request for operator information for the given phone numbers. */ +@Fluent +public final class OperatorInformationRequest { + /* + * Phone number(s) whose operator information is being requested + */ + @JsonProperty(value = "phoneNumbers", required = true) + private List phoneNumbers; + + /* + * Represents options to modify a search request for operator information + */ + @JsonProperty(value = "options") + private OperatorInformationOptions options; + + /** + * Get the phoneNumbers property: Phone number(s) whose operator information is being requested. + * + * @return the phoneNumbers value. + */ + public List getPhoneNumbers() { + return this.phoneNumbers; + } + + /** + * Set the phoneNumbers property: Phone number(s) whose operator information is being requested. + * + * @param phoneNumbers the phoneNumbers value to set. + * @return the OperatorInformationRequest object itself. + */ + public OperatorInformationRequest setPhoneNumbers(List phoneNumbers) { + this.phoneNumbers = phoneNumbers; + return this; + } + + /** + * Get the options property: Represents options to modify a search request for operator information. + * + * @return the options value. + */ + public OperatorInformationOptions getOptions() { + return this.options; + } + + /** + * Set the options property: Represents options to modify a search request for operator information. + * + * @param options the options value to set. + * @return the OperatorInformationRequest object itself. + */ + public OperatorInformationRequest setOptions(OperatorInformationOptions options) { + this.options = options; + return this; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequestOptions.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequestOptions.java new file mode 100644 index 0000000000000..c9cdbcf84a982 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationRequestOptions.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.implementation.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** Represents options to modify a search request for operator information. */ +@Fluent +public final class OperatorInformationRequestOptions { + /* + * Includes the fields operatorDetails, numberType, and isoCountryCode in + * the response. Please note: use of this option will result in additional + * costs + */ + @JsonProperty(value = "includeAdditionalPhoneAndOperatorDetails") + private Boolean includeAdditionalPhoneAndOperatorDetails; + + /** + * Get the includeAdditionalPhoneAndOperatorDetails property: Includes the fields operatorDetails, numberType, and + * isoCountryCode in the response. Please note: use of this option will result in additional costs. + * + * @return the includeAdditionalPhoneAndOperatorDetails value. + */ + public Boolean isIncludeAdditionalPhoneAndOperatorDetails() { + return this.includeAdditionalPhoneAndOperatorDetails; + } + + /** + * Set the includeAdditionalPhoneAndOperatorDetails property: Includes the fields operatorDetails, numberType, and + * isoCountryCode in the response. Please note: use of this option will result in additional costs. + * + * @param includeAdditionalPhoneAndOperatorDetails the includeAdditionalPhoneAndOperatorDetails value to set. + * @return the OperatorInformationRequestOptions object itself. + */ + public OperatorInformationRequestOptions setIncludeAdditionalPhoneAndOperatorDetails( + Boolean includeAdditionalPhoneAndOperatorDetails) { + this.includeAdditionalPhoneAndOperatorDetails = includeAdditionalPhoneAndOperatorDetails; + return this; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationResult.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationResult.java new file mode 100644 index 0000000000000..a59e2f9c70416 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/OperatorInformationResult.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.implementation.models; + +import com.azure.communication.phonenumbers.models.OperatorInformation; +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Represents a search result containing format and operator information associated with the requested phone numbers. + */ +@Immutable +public final class OperatorInformationResult { + /* + * Results of a search. + * This array will have one entry per requested phone number which will + * contain the relevant operator information. + */ + @JsonProperty(value = "values", access = JsonProperty.Access.WRITE_ONLY) + private List values; + + /** + * Get the values property: Results of a search. This array will have one entry per requested phone number which + * will contain the relevant operator information. + * + * @return the values value. + */ + public List getValues() { + return this.values; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberAreaCodes.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberAreaCodes.java index e698325d511de..6ceaebb5b9bd0 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberAreaCodes.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberAreaCodes.java @@ -4,15 +4,13 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberAreaCode; import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The PhoneNumberAreaCodes model. */ +/** The list of available area codes. */ @Immutable -@Generated public final class PhoneNumberAreaCodes { /* * Represents a list of available toll-free area codes. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCapabilitiesRequest.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCapabilitiesRequest.java index 64c1aa6477e76..97da4316da66d 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCapabilitiesRequest.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCapabilitiesRequest.java @@ -4,14 +4,12 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberCapabilityType; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberCapabilitiesRequest model. */ +/** Capabilities of a phone number. */ @Fluent -@Generated public final class PhoneNumberCapabilitiesRequest { /* * Capability value for calling. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCountries.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCountries.java index feb74b0171ab8..33eef7b2fbe27 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCountries.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberCountries.java @@ -4,15 +4,13 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberCountry; import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The PhoneNumberCountries model. */ +/** Represents a wrapper around a list of countries. */ @Immutable -@Generated public final class PhoneNumberCountries { /* * Represents the underlying list of countries. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberLocalities.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberLocalities.java index 345ea3e369cc2..eff6de721cdbf 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberLocalities.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberLocalities.java @@ -4,15 +4,13 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberLocality; import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The PhoneNumberLocalities model. */ +/** Represents a wrapper around a list of cities or towns. */ @Immutable -@Generated public final class PhoneNumberLocalities { /* * Represents the underlying list of localities, e.g. cities or town. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberPurchaseRequest.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberPurchaseRequest.java index a0a14467f64be..f9eca676f6b4a 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberPurchaseRequest.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberPurchaseRequest.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberPurchaseRequest model. */ +/** The phone number search purchase request. */ @Fluent -@Generated public final class PhoneNumberPurchaseRequest { /* * The search id. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberRawOperation.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberRawOperation.java index 9861f11c85ec9..880f519aab0ba 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberRawOperation.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberRawOperation.java @@ -4,7 +4,6 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberOperationStatus; import com.azure.communication.phonenumbers.models.PhoneNumberOperationType; import com.azure.core.annotation.Fluent; @@ -13,7 +12,6 @@ /** The PhoneNumberRawOperation model. */ @Fluent -@Generated public final class PhoneNumberRawOperation { /* * The type of operation, e.g. Search diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchRequest.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchRequest.java index 85b73cb1fa3b3..b83dff58033ff 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchRequest.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchRequest.java @@ -4,16 +4,17 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PhoneNumberAssignmentType; import com.azure.communication.phonenumbers.models.PhoneNumberCapabilities; import com.azure.communication.phonenumbers.models.PhoneNumberType; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberSearchRequest model. */ +/** + * Represents a phone number search request to find phone numbers. Found phone numbers are temporarily held for a + * following purchase. + */ @Fluent -@Generated public final class PhoneNumberSearchRequest { /* * The type of phone numbers to search for, e.g. geographic, or tollFree. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchResultError.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchResultError.java new file mode 100644 index 0000000000000..9313a8bf41399 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumberSearchResultError.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.implementation.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** Defines values for PhoneNumberSearchResultError. */ +public final class PhoneNumberSearchResultError extends ExpandableStringEnum { + /** Static value NoError for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NO_ERROR = fromString("NoError"); + + /** Static value UnknownErrorCode for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError UNKNOWN_ERROR_CODE = fromString("UnknownErrorCode"); + + /** Static value OutOfStock for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError OUT_OF_STOCK = fromString("OutOfStock"); + + /** Static value AuthorizationDenied for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError AUTHORIZATION_DENIED = fromString("AuthorizationDenied"); + + /** Static value MissingAddress for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError MISSING_ADDRESS = fromString("MissingAddress"); + + /** Static value InvalidAddress for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError INVALID_ADDRESS = fromString("InvalidAddress"); + + /** Static value InvalidOfferModel for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError INVALID_OFFER_MODEL = fromString("InvalidOfferModel"); + + /** Static value NotEnoughLicenses for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NOT_ENOUGH_LICENSES = fromString("NotEnoughLicenses"); + + /** Static value NoWallet for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NO_WALLET = fromString("NoWallet"); + + /** Static value NotEnoughCredit for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NOT_ENOUGH_CREDIT = fromString("NotEnoughCredit"); + + /** Static value NumbersPartiallyAcquired for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NUMBERS_PARTIALLY_ACQUIRED = + fromString("NumbersPartiallyAcquired"); + + /** Static value AllNumbersNotAcquired for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError ALL_NUMBERS_NOT_ACQUIRED = fromString("AllNumbersNotAcquired"); + + /** Static value ReservationExpired for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError RESERVATION_EXPIRED = fromString("ReservationExpired"); + + /** Static value PurchaseFailed for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError PURCHASE_FAILED = fromString("PurchaseFailed"); + + /** Static value BillingUnavailable for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError BILLING_UNAVAILABLE = fromString("BillingUnavailable"); + + /** Static value ProvisioningFailed for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError PROVISIONING_FAILED = fromString("ProvisioningFailed"); + + /** Static value UnknownSearchError for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError UNKNOWN_SEARCH_ERROR = fromString("UnknownSearchError"); + + /** + * Creates or finds a PhoneNumberSearchResultError from its string representation. + * + * @param name a name to look for. + * @return the corresponding PhoneNumberSearchResultError. + */ + @JsonCreator + public static PhoneNumberSearchResultError fromString(String name) { + return fromString(name, PhoneNumberSearchResultError.class); + } + + /** + * Gets known PhoneNumberSearchResultError values. + * + * @return known PhoneNumberSearchResultError values. + */ + public static Collection values() { + return values(PhoneNumberSearchResultError.class); + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationHeaders.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationHeaders.java index 5b6d2a109b08f..750735aec94ae 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationHeaders.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationHeaders.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; /** The PhoneNumbersGetOperationHeaders model. */ @Fluent -@Generated public final class PhoneNumbersGetOperationHeaders { /* * The Location property. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationResponse.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationResponse.java index 5155754aed561..ab10ce0b9de7a 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationResponse.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersGetOperationResponse.java @@ -29,7 +29,11 @@ public PhoneNumbersGetOperationResponse( super(request, statusCode, rawHeaders, value, headers); } - /** @return the deserialized response body. */ + /** + * Gets the deserialized response body. + * + * @return the deserialized response body. + */ @Override public PhoneNumberRawOperation getValue() { return super.getValue(); diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersPurchasePhoneNumbersHeaders.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersPurchasePhoneNumbersHeaders.java index 04e19ebdafae0..d1839773ea2d1 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersPurchasePhoneNumbersHeaders.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersPurchasePhoneNumbersHeaders.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; /** The PhoneNumbersPurchasePhoneNumbersHeaders model. */ @Fluent -@Generated public final class PhoneNumbersPurchasePhoneNumbersHeaders { /* * The operation-id property. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersReleasePhoneNumberHeaders.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersReleasePhoneNumberHeaders.java index 79a04996bfcc0..ed6778ee2fd3d 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersReleasePhoneNumberHeaders.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersReleasePhoneNumberHeaders.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; /** The PhoneNumbersReleasePhoneNumberHeaders model. */ @Fluent -@Generated public final class PhoneNumbersReleasePhoneNumberHeaders { /* * The release-id property. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersHeaders.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersHeaders.java index cd821f23c7b84..1c0c5d37b9d8d 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersHeaders.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersHeaders.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; /** The PhoneNumbersSearchAvailablePhoneNumbersHeaders model. */ @Fluent -@Generated public final class PhoneNumbersSearchAvailablePhoneNumbersHeaders { /* * The operation-id property. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersResponse.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersResponse.java index 885ad0f71db4e..41e0b46b111fa 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersResponse.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersSearchAvailablePhoneNumbersResponse.java @@ -30,7 +30,11 @@ public PhoneNumbersSearchAvailablePhoneNumbersResponse( super(request, statusCode, rawHeaders, value, headers); } - /** @return the deserialized response body. */ + /** + * Gets the deserialized response body. + * + * @return the deserialized response body. + */ @Override public PhoneNumberSearchResult getValue() { return super.getValue(); diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesHeaders.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesHeaders.java index 233623fd2d944..a24924527770f 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesHeaders.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesHeaders.java @@ -4,13 +4,11 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; /** The PhoneNumbersUpdateCapabilitiesHeaders model. */ @Fluent -@Generated public final class PhoneNumbersUpdateCapabilitiesHeaders { /* * The capabilities-id property. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesResponse.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesResponse.java index e8df2c4afc4e8..4514062064b2b 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesResponse.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PhoneNumbersUpdateCapabilitiesResponse.java @@ -30,7 +30,11 @@ public PhoneNumbersUpdateCapabilitiesResponse( super(request, statusCode, rawHeaders, value, headers); } - /** @return the deserialized response body. */ + /** + * Gets the deserialized response body. + * + * @return the deserialized response body. + */ @Override public PurchasedPhoneNumber getValue() { return super.getValue(); diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PurchasedPhoneNumbers.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PurchasedPhoneNumbers.java index bece66deae9fb..22b42f45753e7 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PurchasedPhoneNumbers.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/models/PurchasedPhoneNumbers.java @@ -4,15 +4,13 @@ package com.azure.communication.phonenumbers.implementation.models; -import com.azure.communication.phonenumbers.CodeCoverageAnnotation.Generated; import com.azure.communication.phonenumbers.models.PurchasedPhoneNumber; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The PurchasedPhoneNumbers model. */ +/** The list of purchased phone numbers. */ @Fluent -@Generated public final class PurchasedPhoneNumbers { /* * Represents a list of phone numbers. diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/package-info.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/package-info.java index ca2fb8168ac6d..407db3a212ab4 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/package-info.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/implementation/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the implementations and inner classes for PhoneNumberAdminClient. The phone numbers client uses - * Azure Communication Services to purchase and manage phone numbers. + * Package containing the implementations for PhoneNumberAdminClient. The phone numbers client uses Azure Communication + * Services to purchase and manage phone numbers. */ package com.azure.communication.phonenumbers.implementation; diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/BillingFrequency.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/BillingFrequency.java index 2d799d007c969..4a5e7ba00c62c 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/BillingFrequency.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/BillingFrequency.java @@ -3,6 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. package com.azure.communication.phonenumbers.models; + import com.azure.core.util.ExpandableStringEnum; import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; @@ -23,10 +24,10 @@ public static BillingFrequency fromString(String name) { return fromString(name, BillingFrequency.class); } - /** - * Gives a Collection of BillingFrequency Values - * - * @return known BillingFrequency values. + /** + * Gets known BillingFrequency values. + * + * @return known BillingFrequency values. */ public static Collection values() { return values(BillingFrequency.class); diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorDetails.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorDetails.java new file mode 100644 index 0000000000000..d1c40ee27acc6 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorDetails.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** Represents metadata describing the operator of a phone number. */ +@Immutable +public final class OperatorDetails { + /* + * Name of the phone operator + */ + @JsonProperty(value = "name", required = true, access = JsonProperty.Access.WRITE_ONLY) + private String name; + + /* + * Mobile Network Code + */ + @JsonProperty(value = "mobileNetworkCode", access = JsonProperty.Access.WRITE_ONLY) + private String mobileNetworkCode; + + /* + * Mobile Country Code + */ + @JsonProperty(value = "mobileCountryCode", access = JsonProperty.Access.WRITE_ONLY) + private String mobileCountryCode; + + /** + * Get the name property: Name of the phone operator. + * + * @return the name value. + */ + public String getName() { + return this.name; + } + + /** + * Get the mobileNetworkCode property: Mobile Network Code. + * + * @return the mobileNetworkCode value. + */ + public String getMobileNetworkCode() { + return this.mobileNetworkCode; + } + + /** + * Get the mobileCountryCode property: Mobile Country Code. + * + * @return the mobileCountryCode value. + */ + public String getMobileCountryCode() { + return this.mobileCountryCode; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformation.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformation.java new file mode 100644 index 0000000000000..d3bbe11c89a89 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformation.java @@ -0,0 +1,103 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** Represents metadata about a phone number that is controlled/provided by that phone number's operator. */ +@Immutable +public final class OperatorInformation { + /* + * E.164 formatted string representation of the phone number + */ + @JsonProperty(value = "phoneNumber", required = true, access = JsonProperty.Access.WRITE_ONLY) + private String phoneNumber; + + /* + * National format of the phone number + */ + @JsonProperty(value = "nationalFormat", access = JsonProperty.Access.WRITE_ONLY) + private String nationalFormat; + + /* + * International format of the phone number + */ + @JsonProperty(value = "internationalFormat", access = JsonProperty.Access.WRITE_ONLY) + private String internationalFormat; + + /* + * ISO 3166-1 two character ('alpha-2') code associated with the phone + * number. + */ + @JsonProperty(value = "isoCountryCode", access = JsonProperty.Access.WRITE_ONLY) + private String isoCountryCode; + + /* + * Type of service associated with the phone number + */ + @JsonProperty(value = "numberType", access = JsonProperty.Access.WRITE_ONLY) + private OperatorNumberType numberType; + + /* + * Represents metadata describing the operator of a phone number + */ + @JsonProperty(value = "operatorDetails", access = JsonProperty.Access.WRITE_ONLY) + private OperatorDetails operatorDetails; + + /** + * Get the phoneNumber property: E.164 formatted string representation of the phone number. + * + * @return the phoneNumber value. + */ + public String getPhoneNumber() { + return this.phoneNumber; + } + + /** + * Get the nationalFormat property: National format of the phone number. + * + * @return the nationalFormat value. + */ + public String getNationalFormat() { + return this.nationalFormat; + } + + /** + * Get the internationalFormat property: International format of the phone number. + * + * @return the internationalFormat value. + */ + public String getInternationalFormat() { + return this.internationalFormat; + } + + /** + * Get the isoCountryCode property: ISO 3166-1 two character ('alpha-2') code associated with the phone number. + * + * @return the isoCountryCode value. + */ + public String getIsoCountryCode() { + return this.isoCountryCode; + } + + /** + * Get the numberType property: Type of service associated with the phone number. + * + * @return the numberType value. + */ + public OperatorNumberType getNumberType() { + return this.numberType; + } + + /** + * Get the operatorDetails property: Represents metadata describing the operator of a phone number. + * + * @return the operatorDetails value. + */ + public OperatorDetails getOperatorDetails() { + return this.operatorDetails; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationOptions.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationOptions.java new file mode 100644 index 0000000000000..beb3cde0cc08c --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationOptions.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** Represents options to modify a search request for operator information. */ +@Fluent +public final class OperatorInformationOptions { + /* + * Includes the fields operatorDetails, numberType, and isoCountryCode in + * the response. Please note: use of this option will result in additional + * costs + */ + @JsonProperty(value = "includeAdditionalOperatorDetails") + private Boolean includeAdditionalOperatorDetails; + + /** + * Get the includeAdditionalOperatorDetails property: Includes the fields operatorDetails, numberType, and + * isoCountryCode in the response. Please note: use of this option will result in additional costs. + * + * @return the includeAdditionalOperatorDetails value. + */ + public Boolean isIncludeAdditionalOperatorDetails() { + return this.includeAdditionalOperatorDetails; + } + + /** + * Set the includeAdditionalOperatorDetails property: Includes the fields operatorDetails, numberType, and + * isoCountryCode in the response. Please note: use of this option will result in additional costs. + * + * @param includeAdditionalOperatorDetails the includeAdditionalOperatorDetails value to set. + * @return the OperatorInformationOptions object itself. + */ + public OperatorInformationOptions setIncludeAdditionalOperatorDetails(Boolean includeAdditionalOperatorDetails) { + this.includeAdditionalOperatorDetails = includeAdditionalOperatorDetails; + return this; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationResult.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationResult.java new file mode 100644 index 0000000000000..3f8c271d5b740 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorInformationResult.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Represents a search result containing format and operator information associated with the requested phone numbers. + */ +@Immutable +public final class OperatorInformationResult { + /* + * Results of a search. + * This array will have one entry per requested phone number which will + * contain the relevant operator information. + */ + @JsonProperty(value = "values", access = JsonProperty.Access.WRITE_ONLY) + private List values; + + /** + * Get the values property: Results of a search. This array will have one entry per requested phone number which + * will contain the relevant operator information. + * + * @return the values value. + */ + public List getValues() { + return this.values; + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorNumberType.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorNumberType.java new file mode 100644 index 0000000000000..9f86c3654121b --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/OperatorNumberType.java @@ -0,0 +1,44 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** Defines values for OperatorNumberType. */ +public final class OperatorNumberType extends ExpandableStringEnum { + /** Static value unknown for OperatorNumberType. */ + public static final OperatorNumberType UNKNOWN = fromString("unknown"); + + /** Static value other for OperatorNumberType. */ + public static final OperatorNumberType OTHER = fromString("other"); + + /** Static value geographic for OperatorNumberType. */ + public static final OperatorNumberType GEOGRAPHIC = fromString("geographic"); + + /** Static value mobile for OperatorNumberType. */ + public static final OperatorNumberType MOBILE = fromString("mobile"); + + /** + * Creates or finds a OperatorNumberType from its string representation. + * + * @param name a name to look for. + * @return the corresponding OperatorNumberType. + */ + @JsonCreator + public static OperatorNumberType fromString(String name) { + return fromString(name, OperatorNumberType.class); + } + + /** + * Gets known OperatorNumberType values. + * + * @return known OperatorNumberType values. + */ + public static Collection values() { + return values(OperatorNumberType.class); + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAdministrativeDivision.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAdministrativeDivision.java index 61dce61a3501b..d8c6f9245fe75 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAdministrativeDivision.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAdministrativeDivision.java @@ -7,7 +7,7 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberAdministrativeDivision model. */ +/** Represents an administrative division. e.g. state or province. */ @Immutable public final class PhoneNumberAdministrativeDivision { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAreaCode.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAreaCode.java index 20f0f4cdebb17..c82d3211fe93f 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAreaCode.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAreaCode.java @@ -7,9 +7,7 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** - * The PhoneNumberAreaCode model. - */ +/** Represents an Area Code. */ @Immutable public final class PhoneNumberAreaCode { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAssignmentType.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAssignmentType.java index 394760625eead..243e0860a2f9f 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAssignmentType.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberAssignmentType.java @@ -27,9 +27,9 @@ public static PhoneNumberAssignmentType fromString(String name) { return fromString(name, PhoneNumberAssignmentType.class); } - /** - * Gives a collestion ov PhoneNumberAssignmentType - * + /** + * Gets known PhoneNumberAssignmentType values. + * * @return known PhoneNumberAssignmentType values. */ public static Collection values() { diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilities.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilities.java index c6f809cc7ac2f..c31e1683e2560 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilities.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilities.java @@ -7,7 +7,7 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberCapabilities model. */ +/** Capabilities of a phone number. */ @Fluent public final class PhoneNumberCapabilities { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilityType.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilityType.java index cb7f07c39d713..8e52991b3d4e8 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilityType.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCapabilityType.java @@ -33,9 +33,9 @@ public static PhoneNumberCapabilityType fromString(String name) { return fromString(name, PhoneNumberCapabilityType.class); } - /** - * Gives a Collection of PhoneNumberCapabilityType - * + /** + * Gets known PhoneNumberCapabilityType values. + * * @return known PhoneNumberCapabilityType values. */ public static Collection values() { diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCost.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCost.java index a66a2f4b552fe..cd5635b330d4d 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCost.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCost.java @@ -7,7 +7,7 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberCost model. */ +/** The incurred cost for a single phone number. */ @Immutable public final class PhoneNumberCost { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCountry.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCountry.java index 525593379bc1e..675f0181b7e13 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCountry.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberCountry.java @@ -7,7 +7,7 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberCountry model. */ +/** Represents a country. */ @Immutable public final class PhoneNumberCountry { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberLocality.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberLocality.java index 470546892fb30..99136bca717c2 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberLocality.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberLocality.java @@ -7,7 +7,7 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberLocality model. */ +/** Represents a locality. */ @Immutable public final class PhoneNumberLocality { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOffering.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOffering.java index 59d80cc5fbeba..7272cb5191055 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOffering.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOffering.java @@ -7,7 +7,7 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The PhoneNumberOffering model. */ +/** Represents a phone number capability offering. */ @Immutable public final class PhoneNumberOffering { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationStatus.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationStatus.java index 30c5feac9a037..d77a8de272016 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationStatus.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationStatus.java @@ -33,7 +33,11 @@ public static PhoneNumberOperationStatus fromString(String name) { return fromString(name, PhoneNumberOperationStatus.class); } - /** @return known PhoneNumberOperationStatus values. */ + /** + * Gets known PhoneNumberOperationStatus values. + * + * @return known PhoneNumberOperationStatus values. + */ public static Collection values() { return values(PhoneNumberOperationStatus.class); } diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationType.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationType.java index 1fbb70b421e7b..cb15a0b290114 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationType.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberOperationType.java @@ -34,10 +34,10 @@ public static PhoneNumberOperationType fromString(String name) { return fromString(name, PhoneNumberOperationType.class); } - /** - * Gives a collection of PhoneNumberOperationType values - * - * @return known PhoneNumberOperationType values + /** + * Gets known PhoneNumberOperationType values. + * + * @return known PhoneNumberOperationType values. */ public static Collection values() { return values(PhoneNumberOperationType.class); diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResult.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResult.java index 5214db68b4b62..81df3852bf707 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResult.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResult.java @@ -9,7 +9,7 @@ import java.time.OffsetDateTime; import java.util.List; -/** The PhoneNumberSearchResult model. */ +/** The result of a phone number search operation. */ @Immutable public final class PhoneNumberSearchResult { /* @@ -57,6 +57,18 @@ public final class PhoneNumberSearchResult { @JsonProperty(value = "searchExpiresBy", required = true, access = JsonProperty.Access.WRITE_ONLY) private OffsetDateTime searchExpiresBy; + /* + * The error code of the search. + */ + @JsonProperty(value = "errorCode", access = JsonProperty.Access.WRITE_ONLY) + private Integer errorCode; + + /* + * Mapping Error Messages to Codes + */ + @JsonProperty(value = "error", access = JsonProperty.Access.WRITE_ONLY) + private PhoneNumberSearchResultError error; + /** * Get the searchId property: The search id. * @@ -121,4 +133,22 @@ public PhoneNumberCost getCost() { public OffsetDateTime getSearchExpiresBy() { return this.searchExpiresBy; } + + /** + * Get the errorCode property: The error code of the search. + * + * @return the errorCode value. + */ + public Integer getErrorCode() { + return this.errorCode; + } + + /** + * Get the error property: Mapping Error Messages to Codes. + * + * @return the error value. + */ + public PhoneNumberSearchResultError getError() { + return this.error; + } } diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResultError.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResultError.java new file mode 100644 index 0000000000000..47db8912beb93 --- /dev/null +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberSearchResultError.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.communication.phonenumbers.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** Defines values for PhoneNumberSearchResultError. */ +public final class PhoneNumberSearchResultError extends ExpandableStringEnum { + /** Static value NoError for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NO_ERROR = fromString("NoError"); + + /** Static value UnknownErrorCode for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError UNKNOWN_ERROR_CODE = fromString("UnknownErrorCode"); + + /** Static value OutOfStock for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError OUT_OF_STOCK = fromString("OutOfStock"); + + /** Static value AuthorizationDenied for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError AUTHORIZATION_DENIED = fromString("AuthorizationDenied"); + + /** Static value MissingAddress for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError MISSING_ADDRESS = fromString("MissingAddress"); + + /** Static value InvalidAddress for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError INVALID_ADDRESS = fromString("InvalidAddress"); + + /** Static value InvalidOfferModel for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError INVALID_OFFER_MODEL = fromString("InvalidOfferModel"); + + /** Static value NotEnoughLicenses for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NOT_ENOUGH_LICENSES = fromString("NotEnoughLicenses"); + + /** Static value NoWallet for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NO_WALLET = fromString("NoWallet"); + + /** Static value NotEnoughCredit for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NOT_ENOUGH_CREDIT = fromString("NotEnoughCredit"); + + /** Static value NumbersPartiallyAcquired for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError NUMBERS_PARTIALLY_ACQUIRED = + fromString("NumbersPartiallyAcquired"); + + /** Static value AllNumbersNotAcquired for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError ALL_NUMBERS_NOT_ACQUIRED = fromString("AllNumbersNotAcquired"); + + /** Static value ReservationExpired for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError RESERVATION_EXPIRED = fromString("ReservationExpired"); + + /** Static value PurchaseFailed for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError PURCHASE_FAILED = fromString("PurchaseFailed"); + + /** Static value BillingUnavailable for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError BILLING_UNAVAILABLE = fromString("BillingUnavailable"); + + /** Static value ProvisioningFailed for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError PROVISIONING_FAILED = fromString("ProvisioningFailed"); + + /** Static value UnknownSearchError for PhoneNumberSearchResultError. */ + public static final PhoneNumberSearchResultError UNKNOWN_SEARCH_ERROR = fromString("UnknownSearchError"); + + /** + * Creates or finds a PhoneNumberSearchResultError from its string representation. + * + * @param name a name to look for. + * @return the corresponding PhoneNumberSearchResultError. + */ + @JsonCreator + public static PhoneNumberSearchResultError fromString(String name) { + return fromString(name, PhoneNumberSearchResultError.class); + } + + /** + * Gets known PhoneNumberSearchResultError values. + * + * @return known PhoneNumberSearchResultError values. + */ + public static Collection values() { + return values(PhoneNumberSearchResultError.class); + } +} diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberType.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberType.java index 91c09ef7c16f8..e4537fd9c6910 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberType.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PhoneNumberType.java @@ -27,9 +27,9 @@ public static PhoneNumberType fromString(String name) { return fromString(name, PhoneNumberType.class); } - /** - * Gives a collection of PhoneNumberType values - * + /** + * Gets known PhoneNumberType values. + * * @return known PhoneNumberType values. */ public static Collection values() { diff --git a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PurchasedPhoneNumber.java b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PurchasedPhoneNumber.java index 6f05ff05fb16f..b24bd52c4f29f 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PurchasedPhoneNumber.java +++ b/sdk/communication/azure-communication-phonenumbers/src/main/java/com/azure/communication/phonenumbers/models/PurchasedPhoneNumber.java @@ -8,7 +8,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** The PurchasedPhoneNumber model. */ +/** Represents a purchased phone number. */ @Immutable public final class PurchasedPhoneNumber { /* diff --git a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClientIntegrationTest.java b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClientIntegrationTest.java index 3932f7eeba8c6..ddf3ea567afb9 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClientIntegrationTest.java +++ b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersAsyncClientIntegrationTest.java @@ -6,6 +6,8 @@ import com.azure.communication.phonenumbers.implementation.models.CommunicationError; import com.azure.communication.phonenumbers.models.BillingFrequency; import com.azure.communication.phonenumbers.models.PhoneNumberAdministrativeDivision; +import com.azure.communication.phonenumbers.models.OperatorInformationResult; +import com.azure.communication.phonenumbers.models.OperatorInformationOptions; import com.azure.communication.phonenumbers.models.PhoneNumberAreaCode; import com.azure.communication.phonenumbers.models.PhoneNumberAssignmentType; import com.azure.communication.phonenumbers.models.PhoneNumberCapabilities; @@ -480,6 +482,67 @@ public void convertCommunicationErrorWithNull(HttpClient httpClient) { assertEquals(null, error); } + @ParameterizedTest + @MethodSource("com.azure.core.test.TestBase#getHttpClients") + public void searchOperatorInformationSucceeds(HttpClient httpClient) { + List phoneNumbers = new ArrayList(); + phoneNumbers.add(redactIfPlaybackMode(getTestPhoneNumber())); + StepVerifier.create( + this.getClientWithConnectionString(httpClient, "searchOperatorInformation") + .searchOperatorInformation(phoneNumbers)) + .assertNext((OperatorInformationResult result) -> { + assertEquals(phoneNumbers.get(0), result.getValues().get(0).getPhoneNumber()); + }) + .verifyComplete(); + } + + @ParameterizedTest + @MethodSource("com.azure.core.test.TestBase#getHttpClients") + public void searchOperatorInformationOnlyAcceptsOnePhoneNumber(HttpClient httpClient) { + List phoneNumbers = new ArrayList(); + phoneNumbers.add(redactIfPlaybackMode(getTestPhoneNumber())); + phoneNumbers.add(redactIfPlaybackMode(getTestPhoneNumber())); + StepVerifier.create( + this.getClientWithConnectionString(httpClient, "searchOperatorInformationOnlyAcceptsOnePhoneNumber") + .searchOperatorInformation(phoneNumbers)) + .verifyError(); + } + + @ParameterizedTest + @MethodSource("com.azure.core.test.TestBase#getHttpClients") + public void searchOperatorInformationRespectsSearchOptions(HttpClient httpClient) { + List phoneNumbers = new ArrayList(); + phoneNumbers.add(redactIfPlaybackMode(getTestPhoneNumber())); + OperatorInformationOptions requestOptions = new OperatorInformationOptions(); + requestOptions.setIncludeAdditionalOperatorDetails(false); + StepVerifier.create( + this.getClientWithConnectionString(httpClient, "searchOperatorInformation") + .searchOperatorInformationWithResponse(phoneNumbers, requestOptions)) + .assertNext((Response result) -> { + assertEquals(phoneNumbers.get(0), result.getValue().getValues().get(0).getPhoneNumber()); + assertNotNull(result.getValue().getValues().get(0).getNationalFormat()); + assertNotNull(result.getValue().getValues().get(0).getInternationalFormat()); + assertEquals(null, result.getValue().getValues().get(0).getNumberType()); + assertEquals(null, result.getValue().getValues().get(0).getIsoCountryCode()); + assertEquals(null, result.getValue().getValues().get(0).getOperatorDetails()); + }) + .verifyComplete(); + + requestOptions.setIncludeAdditionalOperatorDetails(true); + StepVerifier.create( + this.getClientWithConnectionString(httpClient, "searchOperatorInformation") + .searchOperatorInformationWithResponse(phoneNumbers, requestOptions)) + .assertNext((Response result) -> { + assertEquals(phoneNumbers.get(0), result.getValue().getValues().get(0).getPhoneNumber()); + assertNotNull(result.getValue().getValues().get(0).getNationalFormat()); + assertNotNull(result.getValue().getValues().get(0).getInternationalFormat()); + assertNotNull(result.getValue().getValues().get(0).getNumberType()); + assertNotNull(result.getValue().getValues().get(0).getIsoCountryCode()); + assertNotNull(result.getValue().getValues().get(0).getOperatorDetails()); + }) + .verifyComplete(); + } + private PollerFlux beginSearchAvailablePhoneNumbersHelper( PhoneNumbersAsyncClient client, boolean withOptions) { PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities(); diff --git a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientBuilderTest.java b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientBuilderTest.java index 6a05ee7a5ea51..be3f6be43f765 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientBuilderTest.java +++ b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientBuilderTest.java @@ -110,7 +110,7 @@ public void buildClientWithConfiguration() { public void buildClientWithServiceVersion() { // Build client with required settings and mock configuration PhoneNumbersClient phoneNumberClient = this.setupBuilderWithHttpClientWithCredential(this.clientBuilder) - .serviceVersion(PhoneNumbersServiceVersion.V2021_03_07) + .serviceVersion(PhoneNumbersServiceVersion.V2024_03_01_PREVIEW) .buildClient(); // Validate client created with expected settings @@ -227,7 +227,7 @@ public void buildAsyncClientWithServiceVersion() { // Build client with required settings and mock configuration PhoneNumbersAsyncClient phoneNumberAsyncClient = this .setupBuilderWithHttpClientWithCredential(this.clientBuilder) - .serviceVersion(PhoneNumbersServiceVersion.V2021_03_07) + .serviceVersion(PhoneNumbersServiceVersion.V2024_03_01_PREVIEW) .buildAsyncClient(); // Validate client created with expected settings diff --git a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientIntegrationTest.java b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientIntegrationTest.java index 1679ab3ab2fcf..230c107c84888 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientIntegrationTest.java +++ b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersClientIntegrationTest.java @@ -6,6 +6,7 @@ import com.azure.communication.phonenumbers.implementation.models.CommunicationError; import com.azure.communication.phonenumbers.implementation.models.CommunicationErrorResponseException; import com.azure.communication.phonenumbers.models.PhoneNumberAdministrativeDivision; +import com.azure.communication.phonenumbers.models.OperatorInformationResult; import com.azure.communication.phonenumbers.models.PhoneNumberAreaCode; import com.azure.communication.phonenumbers.models.PhoneNumberAssignmentType; import com.azure.communication.phonenumbers.models.PhoneNumberCapabilities; @@ -449,6 +450,16 @@ public void convertCommunicationErrorWithNull(HttpClient httpClient) { assertEquals(null, error); } + @ParameterizedTest + @MethodSource("com.azure.core.test.TestBase#getHttpClients") + public void searchOperatorInformationSucceeds(HttpClient httpClient) { + List phoneNumbers = new ArrayList(); + phoneNumbers.add(redactIfPlaybackMode(getTestPhoneNumber())); + OperatorInformationResult result = this.getClientWithConnectionString(httpClient, "searchOperatorInformation") + .searchOperatorInformation(phoneNumbers); + assertEquals(phoneNumbers.get(0), result.getValues().get(0).getPhoneNumber()); + } + private SyncPoller beginSearchAvailablePhoneNumbersHelper( HttpClient httpClient, String testName, boolean withContext) { PhoneNumberCapabilities capabilities = new PhoneNumberCapabilities(); diff --git a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersIntegrationTestBase.java b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersIntegrationTestBase.java index ac819bc9e07fb..43789327783c4 100644 --- a/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersIntegrationTestBase.java +++ b/sdk/communication/azure-communication-phonenumbers/src/test/java/com/azure/communication/phonenumbers/PhoneNumbersIntegrationTestBase.java @@ -53,15 +53,19 @@ protected PhoneNumbersClientBuilder getClientBuilderWithConnectionString(HttpCli private void addTestProxyMatchers() { interceptorManager.addMatchers(Arrays.asList( - new CustomMatcher().setHeadersKeyOnlyMatch(Arrays.asList("x-ms-hmac-string-to-sign-base64")))); + new CustomMatcher() + .setHeadersKeyOnlyMatch(Arrays.asList("x-ms-content-sha256", "x-ms-hmac-string-to-sign-base64")))); } private void addTestProxySanitizer() { // sanitize phone numbers interceptorManager.addSanitizers(Arrays.asList( new TestProxySanitizer("(?<=/phoneNumbers/)([^/?]+)", "REDACTED", TestProxySanitizerType.URL), - new TestProxySanitizer("id", null, "REDACTED", TestProxySanitizerType.BODY_KEY), - new TestProxySanitizer("phoneNumber", null, "REDACTED", TestProxySanitizerType.BODY_KEY))); + new TestProxySanitizer("$..id", null, "REDACTED", TestProxySanitizerType.BODY_KEY), + new TestProxySanitizer("$..phoneNumber", null, "REDACTED", TestProxySanitizerType.BODY_KEY), + new TestProxySanitizer("$..nationalFormat", null, "REDACTED", TestProxySanitizerType.BODY_KEY), + new TestProxySanitizer("$..internationalFormat", null, "REDACTED", TestProxySanitizerType.BODY_KEY), + new TestProxySanitizer("((?:\\\\u002B)[0-9]{11,})|((?:\\\\%2B)[0-9]{11,})|((?:[+]?)[0-9]{11,})", "REDACTED", TestProxySanitizerType.BODY_REGEX))); } protected PhoneNumbersClientBuilder getClientBuilderUsingManagedIdentity(HttpClient httpClient) { diff --git a/sdk/communication/azure-communication-phonenumbers/swagger/README.md b/sdk/communication/azure-communication-phonenumbers/swagger/README.md index 0f02dacaea865..a410cc9298a73 100644 --- a/sdk/communication/azure-communication-phonenumbers/swagger/README.md +++ b/sdk/communication/azure-communication-phonenumbers/swagger/README.md @@ -23,10 +23,10 @@ autorest README.md --java --v4 --use=@autorest/java@4.0.2 ### Code generation settings ``` yaml -tag: package-phonenumber-2022-12-01 -require: https://raw.githubusercontent.com/Azure/azure-rest-api-specs/edf1d7365a436f0b124c0cecbefd63499e049af0/specification/communication/data-plane/PhoneNumbers/readme.md +tag: package-phonenumber-2024-03-01-preview +require: https://raw.githubusercontent.com/Azure/azure-rest-api-specs/b56afb26c5450157006a3a1d9be57bae429051a2/specification/communication/data-plane/PhoneNumbers/readme.md override-client-name: PhoneNumberAdminClient -custom-types: PurchasedPhoneNumber,BillingFrequency,PhoneNumberOperationStatus,PhoneNumberOperationStatusCodes,PhoneNumberOperationType,PhoneNumberAssignmentType,PhoneNumberCapabilities,PhoneNumberCapabilityType,PhoneNumberCost,PhoneNumberSearchResult,PhoneNumberType,PhoneNumberCapability,PhoneNumberAdministrativeDivision,PhoneNumberCountry,PhoneNumberLocality,PhoneNumberOffering,AreaCodeResult,AreaCodes,PhoneNumberAreaCode +custom-types: PurchasedPhoneNumber,BillingFrequency,PhoneNumberOperationStatus,PhoneNumberOperationStatusCodes,PhoneNumberOperationType,PhoneNumberAssignmentType,PhoneNumberCapabilities,PhoneNumberCapabilityType,PhoneNumberCost,PhoneNumberSearchResult,PhoneNumberType,PhoneNumberCapability,PhoneNumberAdministrativeDivision,PhoneNumberCountry,PhoneNumberLocality,PhoneNumberOffering,AreaCodeResult,AreaCodes,PhoneNumberAreaCode,PhoneNumberSearchResultError,OperatorDetails,OperatorInformation,OperatorInformationResult,OperatorInformationOptions,OperatorNumberType custom-types-subpackage: models models-subpackage: implementation.models java: true @@ -80,6 +80,8 @@ directive: $["properties"]["phoneNumberType"].readOnly = true; $["properties"]["assignmentType"].readOnly = true; $["properties"]["capabilities"].readOnly = true; + $["properties"]["error"].readOnly = true; + $["properties"]["errorCode"].readOnly = true; ``` ### Rename PhoneNumberOperation to PhoneNumberRawOperation @@ -149,7 +151,6 @@ directive: $["properties"]["localizedName"].readOnly = true; ``` - ### Add readonly attribute to PhoneNumberLocalities properties ```yaml directive: @@ -190,4 +191,54 @@ directive: transform: > $["properties"]["localizedName"].readOnly = true; $["properties"]["countryCode"].readOnly = true; -``` \ No newline at end of file +``` + +### Add readonly attribute to OperatorDetails properties +```yaml +directive: + - from: swagger-document + where: $.definitions.OperatorDetails + transform: > + $["properties"]["name"].readOnly = true; + $["properties"]["mobileNetworkCode"].readOnly = true; + $["properties"]["mobileCountryCode"].readOnly = true; +``` + +### Add readonly attribute to OperatorInformation properties +```yaml +directive: + - from: swagger-document + where: $.definitions.OperatorInformation + transform: > + $["properties"]["phoneNumber"].readOnly = true; + $["properties"]["numberType"].readOnly = true; + $["properties"]["isoCountryCode"].readOnly = true; + $["properties"]["operatorDetails"].readOnly = true; + $["properties"]["nationalFormat"].readOnly = true; + $["properties"]["internationalFormat"].readOnly = true; +``` + +### Add readonly attribute to OperatorInformationResult properties +```yaml +directive: + - from: swagger-document + where: $.definitions.OperatorInformationResult + transform: > + $["properties"]["values"].readOnly = true; +``` + +``` yaml +directive: + from: swagger-document + where: $.definitions.PhoneNumberSearchResult.properties.error.x-ms-enum + transform: > + $["name"] = "PhoneNumberSearchResultError"; +``` + +``` yaml +directive: + from: swagger-document + where: $.parameters.Endpoint + transform: > + $["format"] = ""; +``` diff --git a/sdk/communication/azure-communication-phonenumbers/tests.yml b/sdk/communication/azure-communication-phonenumbers/tests.yml index 7fe880a8f4681..7e795c2399dca 100644 --- a/sdk/communication/azure-communication-phonenumbers/tests.yml +++ b/sdk/communication/azure-communication-phonenumbers/tests.yml @@ -6,36 +6,36 @@ parameters: type: boolean default: false -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-phonenumbers - SafeName: azurecommunicationphonenumbers - ${{ if eq(parameters.runOnlyPPE, true) }}: - Clouds: 'PPE' - ${{ if eq(parameters.runOnlyPPE, false) }}: - Clouds: 'Public,PPE,Int' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-communication-services-cloud-test-resources-common) - - $(sub-config-communication-services-cloud-test-resources-java) - Int: - SubscriptionConfigurations: - - $(sub-config-communication-int-test-resources-common) - - $(sub-config-communication-int-test-resources-java) +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-phonenumbers + SafeName: azurecommunicationphonenumbers + ${{ if eq(parameters.runOnlyPPE, true) }}: + Clouds: 'PPE' + ${{ if eq(parameters.runOnlyPPE, false) }}: + Clouds: 'Public,PPE,Int' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-communication-services-cloud-test-resources-common) + - $(sub-config-communication-services-cloud-test-resources-java) + Int: + SubscriptionConfigurations: + - $(sub-config-communication-int-test-resources-common) + - $(sub-config-communication-int-test-resources-java) - MatrixReplace: - - SKIP_UPDATE_CAPABILITIES_LIVE_TESTS=false/true - PPE: - SubscriptionConfigurations: - - $(sub-config-communication-ppe-test-resources-common) - - $(sub-config-communication-ppe-test-resources-java) - MatrixReplace: - - SKIP_UPDATE_CAPABILITIES_LIVE_TESTS=false/true - MatrixConfigs: - - Name: PhoneNumbers_java_livetest_matrix - Path: sdk/communication/azure-communication-phonenumbers/phone-numbers-livetest-matrix.json - Selection: sparse - GenerateVMJobs: true + MatrixReplace: + - SKIP_UPDATE_CAPABILITIES_LIVE_TESTS=false/true + PPE: + SubscriptionConfigurations: + - $(sub-config-communication-ppe-test-resources-common) + - $(sub-config-communication-ppe-test-resources-java) + MatrixReplace: + - SKIP_UPDATE_CAPABILITIES_LIVE_TESTS=false/true + MatrixConfigs: + - Name: PhoneNumbers_java_livetest_matrix + Path: sdk/communication/azure-communication-phonenumbers/phone-numbers-livetest-matrix.json + Selection: sparse + GenerateVMJobs: true diff --git a/sdk/communication/azure-communication-rooms/CHANGELOG.md b/sdk/communication/azure-communication-rooms/CHANGELOG.md index 5cc0a3d11f02f..5b739d75b69bc 100644 --- a/sdk/communication/azure-communication-rooms/CHANGELOG.md +++ b/sdk/communication/azure-communication-rooms/CHANGELOG.md @@ -10,6 +10,18 @@ ### Other Changes + +## 1.0.9 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.0.8 (2024-02-20) ### Other Changes diff --git a/sdk/communication/azure-communication-rooms/pom.xml b/sdk/communication/azure-communication-rooms/pom.xml index f3498c0f9c027..b5aa9eb2594bf 100644 --- a/sdk/communication/azure-communication-rooms/pom.xml +++ b/sdk/communication/azure-communication-rooms/pom.xml @@ -64,12 +64,12 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure azure-communication-identity - 1.5.2 + 1.5.3 test @@ -99,7 +99,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -111,7 +111,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/communication/azure-communication-rooms/tests.yml b/sdk/communication/azure-communication-rooms/tests.yml index 35fd2d3b34bb2..8b53a1306bd7e 100644 --- a/sdk/communication/azure-communication-rooms/tests.yml +++ b/sdk/communication/azure-communication-rooms/tests.yml @@ -1,20 +1,20 @@ trigger: none -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-rooms - SafeName: azurecommunicationrooms - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-communication-services-cloud-test-resources-common) - - $(sub-config-communication-services-cloud-test-resources-java) - PPE: - SubscriptionConfigurations: - - $(sub-config-communication-ppe-test-resources-common) - - $(sub-config-communication-ppe-test-resources-java) - Clouds: Public,PPE - TestResourceDirectories: - - communication/test-resources/ +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-rooms + SafeName: azurecommunicationrooms + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-communication-services-cloud-test-resources-common) + - $(sub-config-communication-services-cloud-test-resources-java) + PPE: + SubscriptionConfigurations: + - $(sub-config-communication-ppe-test-resources-common) + - $(sub-config-communication-ppe-test-resources-java) + Clouds: Public,PPE + TestResourceDirectories: + - communication/test-resources/ diff --git a/sdk/communication/azure-communication-sms/CHANGELOG.md b/sdk/communication/azure-communication-sms/CHANGELOG.md index d723a51173512..7ee79d0da4e41 100644 --- a/sdk/communication/azure-communication-sms/CHANGELOG.md +++ b/sdk/communication/azure-communication-sms/CHANGELOG.md @@ -10,6 +10,18 @@ ### Other Changes + +## 1.1.22 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-communication-common` from `1.3.0` to version `1.3.1`. + + ## 1.1.21 (2024-02-20) ### Other Changes diff --git a/sdk/communication/azure-communication-sms/pom.xml b/sdk/communication/azure-communication-sms/pom.xml index ddfa4a2bb7e7c..8b40e8ec69513 100644 --- a/sdk/communication/azure-communication-sms/pom.xml +++ b/sdk/communication/azure-communication-sms/pom.xml @@ -60,7 +60,7 @@ com.azure azure-communication-common - 1.3.0 + 1.3.1 com.azure @@ -95,7 +95,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -107,7 +107,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/communication/azure-communication-sms/tests.yml b/sdk/communication/azure-communication-sms/tests.yml index a9d6fe86125eb..9144b1f87120c 100644 --- a/sdk/communication/azure-communication-sms/tests.yml +++ b/sdk/communication/azure-communication-sms/tests.yml @@ -1,20 +1,20 @@ trigger: none -stages: - - template: /sdk/communication/communication-tests-template.yml - parameters: - PackageName: azure-communication-sms - SafeName: azurecommunicationsms - Clouds: 'Public,Int' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-communication-services-cloud-test-resources-common) - - $(sub-config-communication-services-cloud-test-resources-java) - - $(sub-config-communication-services-sms-cloud-test-resources) - Int: - SubscriptionConfigurations: - - $(sub-config-communication-int-test-resources-common) - - $(sub-config-communication-int-test-resources-java) - - $(sub-config-communication-sms-int-test-resources) +extends: + template: /sdk/communication/communication-tests-template.yml + parameters: + PackageName: azure-communication-sms + SafeName: azurecommunicationsms + Clouds: 'Public,Int' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-communication-services-cloud-test-resources-common) + - $(sub-config-communication-services-cloud-test-resources-java) + - $(sub-config-communication-services-sms-cloud-test-resources) + Int: + SubscriptionConfigurations: + - $(sub-config-communication-int-test-resources-common) + - $(sub-config-communication-int-test-resources-java) + - $(sub-config-communication-sms-int-test-resources) diff --git a/sdk/communication/azure-resourcemanager-communication/CHANGELOG.md b/sdk/communication/azure-resourcemanager-communication/CHANGELOG.md index aedda9d24eb99..9adbd5793c75e 100644 --- a/sdk/communication/azure-resourcemanager-communication/CHANGELOG.md +++ b/sdk/communication/azure-resourcemanager-communication/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.1.0-beta.3 (Unreleased) +## 2.2.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,35 @@ ### Other Changes +## 2.1.0 (2024-03-18) + +- Azure Resource Manager Communication client library for Java. This package contains Microsoft Azure SDK for Communication Management SDK. REST API for Azure Communication Services. Package tag package-2023-04. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +* `models.ManagedServiceIdentityType` was added + +* `models.ManagedServiceIdentity` was added + +* `models.UserAssignedIdentity` was added + +#### `models.CommunicationServiceResource$Update` was modified + +* `withIdentity(models.ManagedServiceIdentity)` was added + +#### `models.CommunicationServiceResource$Definition` was modified + +* `withIdentity(models.ManagedServiceIdentity)` was added + +#### `models.CommunicationServiceResourceUpdate` was modified + +* `withIdentity(models.ManagedServiceIdentity)` was added +* `identity()` was added + +#### `models.CommunicationServiceResource` was modified + +* `identity()` was added + ## 2.1.0-beta.2 (2023-11-23) - Azure Resource Manager Communication client library for Java. This package contains Microsoft Azure SDK for Communication Management SDK. REST API for Azure Communication Services. Package tag package-preview-2023-06. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/communication/azure-resourcemanager-communication/README.md b/sdk/communication/azure-resourcemanager-communication/README.md index 62e1a98711c57..cfc9dbda5c21f 100644 --- a/sdk/communication/azure-resourcemanager-communication/README.md +++ b/sdk/communication/azure-resourcemanager-communication/README.md @@ -2,7 +2,7 @@ Azure Resource Manager Communication client library for Java. -This package contains Microsoft Azure SDK for Communication Management SDK. REST API for Azure Communication Services. Package tag package-preview-2023-06. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for Communication Management SDK. REST API for Azure Communication Services. Package tag package-2023-04. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-communication - 2.1.0-beta.2 + 2.1.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/communication/azure-resourcemanager-communication/SAMPLE.md b/sdk/communication/azure-resourcemanager-communication/SAMPLE.md index bfa8e9baae0e4..69b2739b02863 100644 --- a/sdk/communication/azure-resourcemanager-communication/SAMPLE.md +++ b/sdk/communication/azure-resourcemanager-communication/SAMPLE.md @@ -44,20 +44,6 @@ - [Delete](#senderusernames_delete) - [Get](#senderusernames_get) - [ListByDomains](#senderusernames_listbydomains) - -## SuppressionListAddresses - -- [CreateOrUpdate](#suppressionlistaddresses_createorupdate) -- [Delete](#suppressionlistaddresses_delete) -- [Get](#suppressionlistaddresses_get) -- [List](#suppressionlistaddresses_list) - -## SuppressionLists - -- [CreateOrUpdate](#suppressionlists_createorupdate) -- [Delete](#suppressionlists_delete) -- [Get](#suppressionlists_get) -- [ListByDomain](#suppressionlists_listbydomain) ### CommunicationServices_CheckNameAvailability ```java @@ -68,27 +54,37 @@ import com.azure.resourcemanager.communication.models.NameAvailabilityParameters */ public final class CommunicationServicesCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/checkNameAvailabilityAvailable.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/checkNameAvailabilityAvailable.json */ /** * Sample code: Check name availability available. * * @param manager Entry point to CommunicationManager. */ - public static void checkNameAvailabilityAvailable(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().checkNameAvailabilityWithResponse(new NameAvailabilityParameters().withName("MyCommunicationService").withType("Microsoft.Communication/CommunicationServices"), com.azure.core.util.Context.NONE); + public static void + checkNameAvailabilityAvailable(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.communicationServices().checkNameAvailabilityWithResponse(new NameAvailabilityParameters() + .withName("MyCommunicationService").withType("Microsoft.Communication/CommunicationServices"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/checkNameAvailabilityUnavailable.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/checkNameAvailabilityUnavailable.json */ /** * Sample code: Check name availability unavailable. * * @param manager Entry point to CommunicationManager. */ - public static void checkNameAvailabilityUnavailable(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().checkNameAvailabilityWithResponse(new NameAvailabilityParameters().withName("MyCommunicationService").withType("Microsoft.Communication/CommunicationServices"), com.azure.core.util.Context.NONE); + public static void + checkNameAvailabilityUnavailable(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.communicationServices().checkNameAvailabilityWithResponse(new NameAvailabilityParameters() + .withName("MyCommunicationService").withType("Microsoft.Communication/CommunicationServices"), + com.azure.core.util.Context.NONE); } } ``` @@ -98,14 +94,15 @@ public final class CommunicationServicesCheckNameAvailabilitySamples { ```java import com.azure.resourcemanager.communication.models.ManagedServiceIdentity; import com.azure.resourcemanager.communication.models.ManagedServiceIdentityType; -import java.util.stream.Collectors; /** * Samples for CommunicationServices CreateOrUpdate. */ public final class CommunicationServicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/createOrUpdate.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/createOrUpdate.json */ /** * Sample code: Create or update resource. @@ -113,19 +110,26 @@ public final class CommunicationServicesCreateOrUpdateSamples { * @param manager Entry point to CommunicationManager. */ public static void createOrUpdateResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().define("MyCommunicationResource").withRegion("Global").withExistingResourceGroup("MyResourceGroup").withDataLocation("United States").create(); + manager.communicationServices().define("MyCommunicationResource").withRegion("Global") + .withExistingResourceGroup("MyResourceGroup").withDataLocation("United States").create(); } /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/createOrUpdateWithSystemAssignedIdentity.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/createOrUpdateWithSystemAssignedIdentity.json */ /** * Sample code: Create or update resource with managed identity. * * @param manager Entry point to CommunicationManager. */ - public static void createOrUpdateResourceWithManagedIdentity(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().define("MyCommunicationResource").withRegion("Global").withExistingResourceGroup("MyResourceGroup").withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED)).withDataLocation("United States").create(); + public static void createOrUpdateResourceWithManagedIdentity( + com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.communicationServices().define("MyCommunicationResource").withRegion("Global") + .withExistingResourceGroup("MyResourceGroup") + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED)) + .withDataLocation("United States").create(); } } ``` @@ -138,7 +142,9 @@ public final class CommunicationServicesCreateOrUpdateSamples { */ public final class CommunicationServicesDeleteSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/delete.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/delete.json */ /** * Sample code: Delete resource. @@ -146,7 +152,8 @@ public final class CommunicationServicesDeleteSamples { * @param manager Entry point to CommunicationManager. */ public static void deleteResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().delete("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE); + manager.communicationServices().delete("MyResourceGroup", "MyCommunicationResource", + com.azure.core.util.Context.NONE); } } ``` @@ -159,7 +166,9 @@ public final class CommunicationServicesDeleteSamples { */ public final class CommunicationServicesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/get.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/get.json */ /** * Sample code: Get resource. @@ -167,7 +176,8 @@ public final class CommunicationServicesGetByResourceGroupSamples { * @param manager Entry point to CommunicationManager. */ public static void getResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE); + manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", + com.azure.core.util.Context.NONE); } } ``` @@ -182,7 +192,9 @@ import com.azure.resourcemanager.communication.models.LinkNotificationHubParamet */ public final class CommunicationServicesLinkNotificationHubSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/linkNotificationHub.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/linkNotificationHub.json */ /** * Sample code: Link notification hub. @@ -190,7 +202,11 @@ public final class CommunicationServicesLinkNotificationHubSamples { * @param manager Entry point to CommunicationManager. */ public static void linkNotificationHub(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().linkNotificationHubWithResponse("MyResourceGroup", "MyCommunicationResource", new LinkNotificationHubParameters().withResourceId("/subscriptions/11112222-3333-4444-5555-666677778888/resourceGroups/MyOtherResourceGroup/providers/Microsoft.NotificationHubs/namespaces/MyNamespace/notificationHubs/MyHub").withConnectionString("Endpoint=sb://MyNamespace.servicebus.windows.net/;SharedAccessKey=abcd1234"), com.azure.core.util.Context.NONE); + manager.communicationServices().linkNotificationHubWithResponse("MyResourceGroup", "MyCommunicationResource", + new LinkNotificationHubParameters().withResourceId( + "/subscriptions/11112222-3333-4444-5555-666677778888/resourceGroups/MyOtherResourceGroup/providers/Microsoft.NotificationHubs/namespaces/MyNamespace/notificationHubs/MyHub") + .withConnectionString("Endpoint=sb://MyNamespace.servicebus.windows.net/;SharedAccessKey=abcd1234"), + com.azure.core.util.Context.NONE); } } ``` @@ -203,7 +219,9 @@ public final class CommunicationServicesLinkNotificationHubSamples { */ public final class CommunicationServicesListSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/listBySubscription.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/listBySubscription.json */ /** * Sample code: List by subscription. @@ -224,7 +242,9 @@ public final class CommunicationServicesListSamples { */ public final class CommunicationServicesListByResourceGroupSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/listByResourceGroup.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/listByResourceGroup.json */ /** * Sample code: List by resource group. @@ -245,7 +265,9 @@ public final class CommunicationServicesListByResourceGroupSamples { */ public final class CommunicationServicesListKeysSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/listKeys.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/listKeys.json */ /** * Sample code: List keys. @@ -253,7 +275,8 @@ public final class CommunicationServicesListKeysSamples { * @param manager Entry point to CommunicationManager. */ public static void listKeys(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().listKeysWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE); + manager.communicationServices().listKeysWithResponse("MyResourceGroup", "MyCommunicationResource", + com.azure.core.util.Context.NONE); } } ``` @@ -263,14 +286,15 @@ public final class CommunicationServicesListKeysSamples { ```java import com.azure.resourcemanager.communication.models.KeyType; import com.azure.resourcemanager.communication.models.RegenerateKeyParameters; -import java.util.stream.Collectors; /** * Samples for CommunicationServices RegenerateKey. */ public final class CommunicationServicesRegenerateKeySamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/regenerateKey.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/regenerateKey.json */ /** * Sample code: Regenerate key. @@ -278,7 +302,8 @@ public final class CommunicationServicesRegenerateKeySamples { * @param manager Entry point to CommunicationManager. */ public static void regenerateKey(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.communicationServices().regenerateKeyWithResponse("MyResourceGroup", "MyCommunicationResource", new RegenerateKeyParameters().withKeyType(KeyType.PRIMARY), com.azure.core.util.Context.NONE); + manager.communicationServices().regenerateKeyWithResponse("MyResourceGroup", "MyCommunicationResource", + new RegenerateKeyParameters().withKeyType(KeyType.PRIMARY), com.azure.core.util.Context.NONE); } } ``` @@ -292,14 +317,15 @@ import com.azure.resourcemanager.communication.models.ManagedServiceIdentityType import com.azure.resourcemanager.communication.models.UserAssignedIdentity; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for CommunicationServices Update. */ public final class CommunicationServicesUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/update.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/update.json */ /** * Sample code: Update resource. @@ -307,60 +333,91 @@ public final class CommunicationServicesUpdateSamples { * @param manager Entry point to CommunicationManager. */ public static void updateResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("newTag", "newVal")).withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED)).apply(); + CommunicationServiceResource resource + = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", + "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("newTag", "newVal")).apply(); } /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/updateWithUserAssignedIdentity.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/updateWithUserAssignedIdentity.json */ /** * Sample code: Update resource to add a User Assigned managed identity. * * @param manager Entry point to CommunicationManager. */ - public static void updateResourceToAddAUserAssignedManagedIdentity(com.azure.resourcemanager.communication.CommunicationManager manager) { - CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("newTag", "newVal")).withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED).withUserAssignedIdentities(mapOf("/user/assigned/resource/id", new UserAssignedIdentity()))).apply(); + public static void updateResourceToAddAUserAssignedManagedIdentity( + com.azure.resourcemanager.communication.CommunicationManager manager) { + CommunicationServiceResource resource + = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", + "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("newTag", "newVal")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("/user/assigned/resource/id", new UserAssignedIdentity()))) + .apply(); } /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/updateWithSystemAssignedIdentity.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/updateWithSystemAssignedIdentity.json */ /** * Sample code: Update resource to add a System Assigned managed identity. * * @param manager Entry point to CommunicationManager. */ - public static void updateResourceToAddASystemAssignedManagedIdentity(com.azure.resourcemanager.communication.CommunicationManager manager) { - CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("newTag", "newVal")).withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED)).apply(); + public static void updateResourceToAddASystemAssignedManagedIdentity( + com.azure.resourcemanager.communication.CommunicationManager manager) { + CommunicationServiceResource resource + = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", + "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("newTag", "newVal")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED)).apply(); } /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/updateRemoveSystemIdentity.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/updateRemoveSystemIdentity.json */ /** * Sample code: Update resource to remove identity. * * @param manager Entry point to CommunicationManager. */ - public static void updateResourceToRemoveIdentity(com.azure.resourcemanager.communication.CommunicationManager manager) { - CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE)).apply(); + public static void + updateResourceToRemoveIdentity(com.azure.resourcemanager.communication.CommunicationManager manager) { + CommunicationServiceResource resource + = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", + "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("newTag", "newVal")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE)).apply(); } /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/updateWithSystemAndUserIdentity.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/updateWithSystemAndUserIdentity.json */ /** * Sample code: Update resource to add System and User managed identities. * * @param manager Entry point to CommunicationManager. */ - public static void updateResourceToAddSystemAndUserManagedIdentities(com.azure.resourcemanager.communication.CommunicationManager manager) { - CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("newTag", "newVal")).withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED).withUserAssignedIdentities(mapOf("/user/assigned/resource/id", new UserAssignedIdentity()))).apply(); + public static void updateResourceToAddSystemAndUserManagedIdentities( + com.azure.resourcemanager.communication.CommunicationManager manager) { + CommunicationServiceResource resource + = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", + "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("newTag", "newVal")) + .withIdentity( + new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("/user/assigned/resource/id", new UserAssignedIdentity()))) + .apply(); } // Use "Map.of" if available @@ -382,14 +439,15 @@ public final class CommunicationServicesUpdateSamples { ```java import com.azure.resourcemanager.communication.models.VerificationParameter; import com.azure.resourcemanager.communication.models.VerificationType; -import java.util.stream.Collectors; /** * Samples for Domains CancelVerification. */ public final class DomainsCancelVerificationSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/cancelVerification.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ + * cancelVerification.json */ /** * Sample code: Cancel verification. @@ -397,7 +455,8 @@ public final class DomainsCancelVerificationSamples { * @param manager Entry point to CommunicationManager. */ public static void cancelVerification(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.domains().cancelVerification("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", new VerificationParameter().withVerificationType(VerificationType.SPF), com.azure.core.util.Context.NONE); + manager.domains().cancelVerification("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", + new VerificationParameter().withVerificationType(VerificationType.SPF), com.azure.core.util.Context.NONE); } } ``` @@ -406,22 +465,26 @@ public final class DomainsCancelVerificationSamples { ```java import com.azure.resourcemanager.communication.models.DomainManagement; -import java.util.stream.Collectors; /** * Samples for Domains CreateOrUpdate. */ public final class DomainsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/createOrUpdate.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ + * createOrUpdate.json */ /** * Sample code: Create or update Domains resource. * * @param manager Entry point to CommunicationManager. */ - public static void createOrUpdateDomainsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.domains().define("mydomain.com").withRegion("Global").withExistingEmailService("MyResourceGroup", "MyEmailServiceResource").withDomainManagement(DomainManagement.CUSTOMER_MANAGED).create(); + public static void + createOrUpdateDomainsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.domains().define("mydomain.com").withRegion("Global") + .withExistingEmailService("MyResourceGroup", "MyEmailServiceResource") + .withDomainManagement(DomainManagement.CUSTOMER_MANAGED).create(); } } ``` @@ -434,7 +497,9 @@ public final class DomainsCreateOrUpdateSamples { */ public final class DomainsDeleteSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/delete.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/delete. + * json */ /** * Sample code: Delete Domains resource. @@ -442,7 +507,8 @@ public final class DomainsDeleteSamples { * @param manager Entry point to CommunicationManager. */ public static void deleteDomainsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.domains().delete("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", com.azure.core.util.Context.NONE); + manager.domains().delete("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", + com.azure.core.util.Context.NONE); } } ``` @@ -455,7 +521,8 @@ public final class DomainsDeleteSamples { */ public final class DomainsGetSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/get.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/get.json */ /** * Sample code: Get Domains resource. @@ -463,7 +530,8 @@ public final class DomainsGetSamples { * @param manager Entry point to CommunicationManager. */ public static void getDomainsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.domains().getWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", com.azure.core.util.Context.NONE); + manager.domains().getWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", + com.azure.core.util.Context.NONE); } } ``` @@ -473,14 +541,15 @@ public final class DomainsGetSamples { ```java import com.azure.resourcemanager.communication.models.VerificationParameter; import com.azure.resourcemanager.communication.models.VerificationType; -import java.util.stream.Collectors; /** * Samples for Domains InitiateVerification. */ public final class DomainsInitiateVerificationSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/initiateVerification.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ + * initiateVerification.json */ /** * Sample code: Initiate verification. @@ -488,7 +557,8 @@ public final class DomainsInitiateVerificationSamples { * @param manager Entry point to CommunicationManager. */ public static void initiateVerification(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.domains().initiateVerification("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", new VerificationParameter().withVerificationType(VerificationType.SPF), com.azure.core.util.Context.NONE); + manager.domains().initiateVerification("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", + new VerificationParameter().withVerificationType(VerificationType.SPF), com.azure.core.util.Context.NONE); } } ``` @@ -501,15 +571,19 @@ public final class DomainsInitiateVerificationSamples { */ public final class DomainsListByEmailServiceResourceSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/listByEmailService.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ + * listByEmailService.json */ /** * Sample code: List Domains resources by EmailServiceName. * * @param manager Entry point to CommunicationManager. */ - public static void listDomainsResourcesByEmailServiceName(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.domains().listByEmailServiceResource("MyResourceGroup", "MyEmailServiceResource", com.azure.core.util.Context.NONE); + public static void + listDomainsResourcesByEmailServiceName(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.domains().listByEmailServiceResource("MyResourceGroup", "MyEmailServiceResource", + com.azure.core.util.Context.NONE); } } ``` @@ -519,14 +593,15 @@ public final class DomainsListByEmailServiceResourceSamples { ```java import com.azure.resourcemanager.communication.models.DomainResource; import com.azure.resourcemanager.communication.models.UserEngagementTracking; -import java.util.stream.Collectors; /** * Samples for Domains Update. */ public final class DomainsUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/update.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/update. + * json */ /** * Sample code: Update Domains resource. @@ -534,7 +609,8 @@ public final class DomainsUpdateSamples { * @param manager Entry point to CommunicationManager. */ public static void updateDomainsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - DomainResource resource = manager.domains().getWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", com.azure.core.util.Context.NONE).getValue(); + DomainResource resource = manager.domains().getWithResponse("MyResourceGroup", "MyEmailServiceResource", + "mydomain.com", com.azure.core.util.Context.NONE).getValue(); resource.update().withUserEngagementTracking(UserEngagementTracking.ENABLED).apply(); } } @@ -548,15 +624,19 @@ public final class DomainsUpdateSamples { */ public final class EmailServicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/createOrUpdate.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * createOrUpdate.json */ /** * Sample code: Create or update EmailService resource. * * @param manager Entry point to CommunicationManager. */ - public static void createOrUpdateEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.emailServices().define("MyEmailServiceResource").withRegion("Global").withExistingResourceGroup("MyResourceGroup").withDataLocation("United States").create(); + public static void + createOrUpdateEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.emailServices().define("MyEmailServiceResource").withRegion("Global") + .withExistingResourceGroup("MyResourceGroup").withDataLocation("United States").create(); } } ``` @@ -569,14 +649,17 @@ public final class EmailServicesCreateOrUpdateSamples { */ public final class EmailServicesDeleteSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/delete.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * delete.json */ /** * Sample code: Delete EmailService resource. * * @param manager Entry point to CommunicationManager. */ - public static void deleteEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + public static void + deleteEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { manager.emailServices().delete("MyResourceGroup", "MyEmailServiceResource", com.azure.core.util.Context.NONE); } } @@ -590,7 +673,9 @@ public final class EmailServicesDeleteSamples { */ public final class EmailServicesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/get.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/get + * .json */ /** * Sample code: Get EmailService resource. @@ -598,7 +683,8 @@ public final class EmailServicesGetByResourceGroupSamples { * @param manager Entry point to CommunicationManager. */ public static void getEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.emailServices().getByResourceGroupWithResponse("MyResourceGroup", "MyEmailServiceResource", com.azure.core.util.Context.NONE); + manager.emailServices().getByResourceGroupWithResponse("MyResourceGroup", "MyEmailServiceResource", + com.azure.core.util.Context.NONE); } } ``` @@ -611,14 +697,17 @@ public final class EmailServicesGetByResourceGroupSamples { */ public final class EmailServicesListSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/listBySubscription.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * listBySubscription.json */ /** * Sample code: List EmailService resources by subscription. * * @param manager Entry point to CommunicationManager. */ - public static void listEmailServiceResourcesBySubscription(com.azure.resourcemanager.communication.CommunicationManager manager) { + public static void + listEmailServiceResourcesBySubscription(com.azure.resourcemanager.communication.CommunicationManager manager) { manager.emailServices().list(com.azure.core.util.Context.NONE); } } @@ -632,14 +721,17 @@ public final class EmailServicesListSamples { */ public final class EmailServicesListByResourceGroupSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/listByResourceGroup.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * listByResourceGroup.json */ /** * Sample code: List EmailService resources by resource group. * * @param manager Entry point to CommunicationManager. */ - public static void listEmailServiceResourcesByResourceGroup(com.azure.resourcemanager.communication.CommunicationManager manager) { + public static void + listEmailServiceResourcesByResourceGroup(com.azure.resourcemanager.communication.CommunicationManager manager) { manager.emailServices().listByResourceGroup("MyResourceGroup", com.azure.core.util.Context.NONE); } } @@ -653,14 +745,17 @@ public final class EmailServicesListByResourceGroupSamples { */ public final class EmailServicesListVerifiedExchangeOnlineDomainsSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/getVerifiedExchangeOnlineDomains.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * getVerifiedExchangeOnlineDomains.json */ /** * Sample code: Get verified Exchange Online domains. * * @param manager Entry point to CommunicationManager. */ - public static void getVerifiedExchangeOnlineDomains(com.azure.resourcemanager.communication.CommunicationManager manager) { + public static void + getVerifiedExchangeOnlineDomains(com.azure.resourcemanager.communication.CommunicationManager manager) { manager.emailServices().listVerifiedExchangeOnlineDomainsWithResponse(com.azure.core.util.Context.NONE); } } @@ -678,15 +773,19 @@ import java.util.Map; */ public final class EmailServicesUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/emailServices/update.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * update.json */ /** * Sample code: Update EmailService resource. * * @param manager Entry point to CommunicationManager. */ - public static void updateEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - EmailServiceResource resource = manager.emailServices().getByResourceGroupWithResponse("MyResourceGroup", "MyEmailServiceResource", com.azure.core.util.Context.NONE).getValue(); + public static void + updateEmailServiceResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + EmailServiceResource resource = manager.emailServices().getByResourceGroupWithResponse("MyResourceGroup", + "MyEmailServiceResource", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("newTag", "newVal")).apply(); } @@ -712,7 +811,9 @@ public final class EmailServicesUpdateSamples { */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/communicationServices/operationsList.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ + * communicationServices/operationsList.json */ /** * Sample code: Operations_List. @@ -733,15 +834,20 @@ public final class OperationsListSamples { */ public final class SenderUsernamesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/senderUsernames/createOrUpdate.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * createOrUpdate.json */ /** * Sample code: Create or update SenderUsernames resource. * * @param manager Entry point to CommunicationManager. */ - public static void createOrUpdateSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.senderUsernames().define("contosoNewsAlerts").withExistingDomain("contosoResourceGroup", "contosoEmailService", "contoso.com").withUsername("contosoNewsAlerts").withDisplayName("Contoso News Alerts").create(); + public static void + createOrUpdateSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.senderUsernames().define("contosoNewsAlerts") + .withExistingDomain("contosoResourceGroup", "contosoEmailService", "contoso.com") + .withUsername("contosoNewsAlerts").withDisplayName("Contoso News Alerts").create(); } } ``` @@ -754,15 +860,19 @@ public final class SenderUsernamesCreateOrUpdateSamples { */ public final class SenderUsernamesDeleteSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/senderUsernames/delete.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * delete.json */ /** * Sample code: Delete SenderUsernames resource. * * @param manager Entry point to CommunicationManager. */ - public static void deleteSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.senderUsernames().deleteWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", "contosoNewsAlerts", com.azure.core.util.Context.NONE); + public static void + deleteSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.senderUsernames().deleteWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", + "contosoNewsAlerts", com.azure.core.util.Context.NONE); } } ``` @@ -775,15 +885,19 @@ public final class SenderUsernamesDeleteSamples { */ public final class SenderUsernamesGetSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/senderUsernames/get.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * get.json */ /** * Sample code: Get SenderUsernames resource. * * @param manager Entry point to CommunicationManager. */ - public static void getSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.senderUsernames().getWithResponse("contosoResourceGroup", "contosoEmailService", "contoso.com", "contosoNewsAlerts", com.azure.core.util.Context.NONE); + public static void + getSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.senderUsernames().getWithResponse("contosoResourceGroup", "contosoEmailService", "contoso.com", + "contosoNewsAlerts", com.azure.core.util.Context.NONE); } } ``` @@ -796,183 +910,19 @@ public final class SenderUsernamesGetSamples { */ public final class SenderUsernamesListByDomainsSamples { /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/senderUsernames/listByDomain.json - */ - /** - * Sample code: Get all SenderUsernames resources for a Domain. - * - * @param manager Entry point to CommunicationManager. - */ - public static void getAllSenderUsernamesResourcesForADomain(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.senderUsernames().listByDomains("contosoResourceGroup", "contosoEmailService", "contoso.com", com.azure.core.util.Context.NONE); - } -} -``` - -### SuppressionListAddresses_CreateOrUpdate - -```java -/** - * Samples for SuppressionListAddresses CreateOrUpdate. - */ -public final class SuppressionListAddressesCreateOrUpdateSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/createOrUpdateAddress.json - */ - /** - * Sample code: CreateOrUpdate SuppressionListAddress resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void createOrUpdateSuppressionListAddressResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().define("11112222-3333-4444-5555-aaaabbbbcccc").withExistingSuppressionList("contosoResourceGroup", "contosoEmailService", "contoso.com", "aaaa1111-bbbb-2222-3333-aaaa11112222").withEmail("newuser1@fabrikam.com").withFirstName("updatedFirstName").create(); - } -} -``` - -### SuppressionListAddresses_Delete - -```java -/** - * Samples for SuppressionListAddresses Delete. - */ -public final class SuppressionListAddressesDeleteSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/deleteAddress.json - */ - /** - * Sample code: Delete a SuppressionListAddress resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void deleteASuppressionListAddressResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().deleteWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", "aaaa1111-bbbb-2222-3333-aaaa11112222", "11112222-3333-4444-5555-999999999999", com.azure.core.util.Context.NONE); - } -} -``` - -### SuppressionListAddresses_Get - -```java -/** - * Samples for SuppressionListAddresses Get. - */ -public final class SuppressionListAddressesGetSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/getAddress.json - */ - /** - * Sample code: Get a SuppressionListAddress resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void getASuppressionListAddressResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().getWithResponse("contosoResourceGroup", "contosoEmailService", "contoso.com", "aaaa1111-bbbb-2222-3333-aaaa11112222", "11112222-3333-4444-5555-aaaabbbbcccc", com.azure.core.util.Context.NONE); - } -} -``` - -### SuppressionListAddresses_List - -```java -/** - * Samples for SuppressionListAddresses List. - */ -public final class SuppressionListAddressesListSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/getAddresses.json - */ - /** - * Sample code: Get all SuppressionListAddresses resources for a SuppressionList resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void getAllSuppressionListAddressesResourcesForASuppressionListResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().list("contosoResourceGroup", "contosoEmailService", "contoso.com", "aaaa1111-bbbb-2222-3333-aaaa11112222", com.azure.core.util.Context.NONE); - } -} -``` - -### SuppressionLists_CreateOrUpdate - -```java -/** - * Samples for SuppressionLists CreateOrUpdate. - */ -public final class SuppressionListsCreateOrUpdateSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/createOrUpdateSuppressionList.json - */ - /** - * Sample code: CreateOrUpdate SuppressionLists resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void createOrUpdateSuppressionListsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().define("aaaa1111-bbbb-2222-3333-aaaa11112222").withExistingDomain("contosoResourceGroup", "contosoEmailService", "contoso.com").withListName("contosoNewsAlerts").create(); - } -} -``` - -### SuppressionLists_Delete - -```java -/** - * Samples for SuppressionLists Delete. - */ -public final class SuppressionListsDeleteSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/deleteSuppressionList.json + * x-ms-original-file: + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * listByDomain.json */ /** - * Sample code: Delete a SuppressionLists resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void deleteASuppressionListsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().deleteWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", "aaaa1111-bbbb-2222-3333-aaaa11112222", com.azure.core.util.Context.NONE); - } -} -``` - -### SuppressionLists_Get - -```java -/** - * Samples for SuppressionLists Get. - */ -public final class SuppressionListsGetSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/getSuppressionList.json - */ - /** - * Sample code: Get a SuppressionList resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void getASuppressionListResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().getWithResponse("contosoResourceGroup", "contosoEmailService", "contoso.com", "aaaa1111-bbbb-2222-3333-aaaa11112222", com.azure.core.util.Context.NONE); - } -} -``` - -### SuppressionLists_ListByDomain - -```java -/** - * Samples for SuppressionLists ListByDomain. - */ -public final class SuppressionListsListByDomainSamples { - /* - * x-ms-original-file: specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/suppressionLists/getSuppressionLists.json - */ - /** - * Sample code: Get all SuppressionLists resources. + * Sample code: Get SenderUsernames resource. * * @param manager Entry point to CommunicationManager. */ - public static void getAllSuppressionListsResources(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().listByDomain("contosoResourceGroup", "contosoEmailService", "contoso.com", com.azure.core.util.Context.NONE); + public static void + getSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { + manager.senderUsernames().listByDomains("contosoResourceGroup", "contosoEmailService", "contoso.com", + com.azure.core.util.Context.NONE); } } ``` diff --git a/sdk/communication/azure-resourcemanager-communication/pom.xml b/sdk/communication/azure-resourcemanager-communication/pom.xml index 7a842d13090f8..8789087559d98 100644 --- a/sdk/communication/azure-resourcemanager-communication/pom.xml +++ b/sdk/communication/azure-resourcemanager-communication/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-communication - 2.1.0-beta.3 + 2.2.0-beta.1 jar Microsoft Azure SDK for Communication Management - This package contains Microsoft Azure SDK for Communication Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. REST API for Azure Communication Services. Package tag package-preview-2023-06. + This package contains Microsoft Azure SDK for Communication Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. REST API for Azure Communication Services. Package tag package-2023-04. https://github.com/Azure/azure-sdk-for-java @@ -45,7 +45,6 @@ UTF-8 0 0 - true @@ -67,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -88,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/CommunicationManager.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/CommunicationManager.java index f87406b77d587..2d522afcf29eb 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/CommunicationManager.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/CommunicationManager.java @@ -30,15 +30,11 @@ import com.azure.resourcemanager.communication.implementation.EmailServicesImpl; import com.azure.resourcemanager.communication.implementation.OperationsImpl; import com.azure.resourcemanager.communication.implementation.SenderUsernamesImpl; -import com.azure.resourcemanager.communication.implementation.SuppressionListAddressesImpl; -import com.azure.resourcemanager.communication.implementation.SuppressionListsImpl; import com.azure.resourcemanager.communication.models.CommunicationServices; import com.azure.resourcemanager.communication.models.Domains; import com.azure.resourcemanager.communication.models.EmailServices; import com.azure.resourcemanager.communication.models.Operations; import com.azure.resourcemanager.communication.models.SenderUsernames; -import com.azure.resourcemanager.communication.models.SuppressionListAddresses; -import com.azure.resourcemanager.communication.models.SuppressionLists; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -61,10 +57,6 @@ public final class CommunicationManager { private SenderUsernames senderUsernames; - private SuppressionLists suppressionLists; - - private SuppressionListAddresses suppressionListAddresses; - private final CommunicationServiceManagementClient clientObject; private CommunicationManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { @@ -224,7 +216,7 @@ public CommunicationManager authenticate(TokenCredential credential, AzureProfil StringBuilder userAgentBuilder = new StringBuilder(); userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.communication") - .append("/").append("2.1.0-beta.2"); + .append("/").append("2.1.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") @@ -323,31 +315,6 @@ public SenderUsernames senderUsernames() { return senderUsernames; } - /** - * Gets the resource collection API of SuppressionLists. It manages SuppressionListResource. - * - * @return Resource collection API of SuppressionLists. - */ - public SuppressionLists suppressionLists() { - if (this.suppressionLists == null) { - this.suppressionLists = new SuppressionListsImpl(clientObject.getSuppressionLists(), this); - } - return suppressionLists; - } - - /** - * Gets the resource collection API of SuppressionListAddresses. It manages SuppressionListAddressResource. - * - * @return Resource collection API of SuppressionListAddresses. - */ - public SuppressionListAddresses suppressionListAddresses() { - if (this.suppressionListAddresses == null) { - this.suppressionListAddresses - = new SuppressionListAddressesImpl(clientObject.getSuppressionListAddresses(), this); - } - return suppressionListAddresses; - } - /** * Gets wrapped service client CommunicationServiceManagementClient providing direct access to the underlying * auto-generated API implementation, based on Azure REST API. diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/CommunicationServiceManagementClient.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/CommunicationServiceManagementClient.java index 5d6ab146d318c..d24e14239ffe7 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/CommunicationServiceManagementClient.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/CommunicationServiceManagementClient.java @@ -80,18 +80,4 @@ public interface CommunicationServiceManagementClient { * @return the SenderUsernamesClient object. */ SenderUsernamesClient getSenderUsernames(); - - /** - * Gets the SuppressionListsClient object to access its operations. - * - * @return the SuppressionListsClient object. - */ - SuppressionListsClient getSuppressionLists(); - - /** - * Gets the SuppressionListAddressesClient object to access its operations. - * - * @return the SuppressionListAddressesClient object. - */ - SuppressionListAddressesClient getSuppressionListAddresses(); } diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListAddressesClient.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListAddressesClient.java deleted file mode 100644 index 869f0e4239d4e..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListAddressesClient.java +++ /dev/null @@ -1,174 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; - -/** - * An instance of this class provides access to all the operations defined in SuppressionListAddressesClient. - */ -public interface SuppressionListAddressesClient { - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName); - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context); - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId, Context context); - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - SuppressionListAddressResourceInner get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId); - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, String addressId, - SuppressionListAddressResourceInner parameters, Context context); - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - SuppressionListAddressResourceInner createOrUpdate(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId, - SuppressionListAddressResourceInner parameters); - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId, Context context); - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String emailServiceName, String domainName, String suppressionListName, - String addressId); -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListsClient.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListsClient.java deleted file mode 100644 index 799363d149f52..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/SuppressionListsClient.java +++ /dev/null @@ -1,167 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; - -/** - * An instance of this class provides access to all the operations defined in SuppressionListsClient. - */ -public interface SuppressionListsClient { - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName); - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName, Context context); - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context); - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - SuppressionListResourceInner get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName); - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, SuppressionListResourceInner parameters, Context context); - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - SuppressionListResourceInner createOrUpdate(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, SuppressionListResourceInner parameters); - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, Context context); - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String emailServiceName, String domainName, String suppressionListName); -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressProperties.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressProperties.java deleted file mode 100644 index d872601d82ee3..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressProperties.java +++ /dev/null @@ -1,172 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; - -/** - * A class that describes the properties of a SuppressionListAddress resource. - */ -@Fluent -public final class SuppressionListAddressProperties { - /* - * Email address of the recipient. - */ - @JsonProperty(value = "email", required = true) - private String email; - - /* - * The first name of the email recipient. - */ - @JsonProperty(value = "firstName") - private String firstName; - - /* - * The last name of the email recipient. - */ - @JsonProperty(value = "lastName") - private String lastName; - - /* - * An optional property to provide contextual notes or a description for an address. - */ - @JsonProperty(value = "notes") - private String notes; - - /* - * The date the address was last updated in a suppression list. - */ - @JsonProperty(value = "lastModified", access = JsonProperty.Access.WRITE_ONLY) - private OffsetDateTime lastModified; - - /* - * The location where the SuppressionListAddress data is stored at rest. This value is inherited from the parent - * Domains resource. - */ - @JsonProperty(value = "dataLocation", access = JsonProperty.Access.WRITE_ONLY) - private String dataLocation; - - /** - * Creates an instance of SuppressionListAddressProperties class. - */ - public SuppressionListAddressProperties() { - } - - /** - * Get the email property: Email address of the recipient. - * - * @return the email value. - */ - public String email() { - return this.email; - } - - /** - * Set the email property: Email address of the recipient. - * - * @param email the email value to set. - * @return the SuppressionListAddressProperties object itself. - */ - public SuppressionListAddressProperties withEmail(String email) { - this.email = email; - return this; - } - - /** - * Get the firstName property: The first name of the email recipient. - * - * @return the firstName value. - */ - public String firstName() { - return this.firstName; - } - - /** - * Set the firstName property: The first name of the email recipient. - * - * @param firstName the firstName value to set. - * @return the SuppressionListAddressProperties object itself. - */ - public SuppressionListAddressProperties withFirstName(String firstName) { - this.firstName = firstName; - return this; - } - - /** - * Get the lastName property: The last name of the email recipient. - * - * @return the lastName value. - */ - public String lastName() { - return this.lastName; - } - - /** - * Set the lastName property: The last name of the email recipient. - * - * @param lastName the lastName value to set. - * @return the SuppressionListAddressProperties object itself. - */ - public SuppressionListAddressProperties withLastName(String lastName) { - this.lastName = lastName; - return this; - } - - /** - * Get the notes property: An optional property to provide contextual notes or a description for an address. - * - * @return the notes value. - */ - public String notes() { - return this.notes; - } - - /** - * Set the notes property: An optional property to provide contextual notes or a description for an address. - * - * @param notes the notes value to set. - * @return the SuppressionListAddressProperties object itself. - */ - public SuppressionListAddressProperties withNotes(String notes) { - this.notes = notes; - return this; - } - - /** - * Get the lastModified property: The date the address was last updated in a suppression list. - * - * @return the lastModified value. - */ - public OffsetDateTime lastModified() { - return this.lastModified; - } - - /** - * Get the dataLocation property: The location where the SuppressionListAddress data is stored at rest. This value - * is inherited from the parent Domains resource. - * - * @return the dataLocation value. - */ - public String dataLocation() { - return this.dataLocation; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (email() == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - "Missing required property email in model SuppressionListAddressProperties")); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(SuppressionListAddressProperties.class); -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressResourceInner.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressResourceInner.java deleted file mode 100644 index 1ec545359920a..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListAddressResourceInner.java +++ /dev/null @@ -1,175 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; - -/** - * A object that represents a SuppressionList record. - */ -@Fluent -public final class SuppressionListAddressResourceInner extends ProxyResource { - /* - * The properties of a SuppressionListAddress resource. - */ - @JsonProperty(value = "properties") - private SuppressionListAddressProperties innerProperties; - - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. - */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** - * Creates an instance of SuppressionListAddressResourceInner class. - */ - public SuppressionListAddressResourceInner() { - } - - /** - * Get the innerProperties property: The properties of a SuppressionListAddress resource. - * - * @return the innerProperties value. - */ - private SuppressionListAddressProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - - /** - * Get the email property: Email address of the recipient. - * - * @return the email value. - */ - public String email() { - return this.innerProperties() == null ? null : this.innerProperties().email(); - } - - /** - * Set the email property: Email address of the recipient. - * - * @param email the email value to set. - * @return the SuppressionListAddressResourceInner object itself. - */ - public SuppressionListAddressResourceInner withEmail(String email) { - if (this.innerProperties() == null) { - this.innerProperties = new SuppressionListAddressProperties(); - } - this.innerProperties().withEmail(email); - return this; - } - - /** - * Get the firstName property: The first name of the email recipient. - * - * @return the firstName value. - */ - public String firstName() { - return this.innerProperties() == null ? null : this.innerProperties().firstName(); - } - - /** - * Set the firstName property: The first name of the email recipient. - * - * @param firstName the firstName value to set. - * @return the SuppressionListAddressResourceInner object itself. - */ - public SuppressionListAddressResourceInner withFirstName(String firstName) { - if (this.innerProperties() == null) { - this.innerProperties = new SuppressionListAddressProperties(); - } - this.innerProperties().withFirstName(firstName); - return this; - } - - /** - * Get the lastName property: The last name of the email recipient. - * - * @return the lastName value. - */ - public String lastName() { - return this.innerProperties() == null ? null : this.innerProperties().lastName(); - } - - /** - * Set the lastName property: The last name of the email recipient. - * - * @param lastName the lastName value to set. - * @return the SuppressionListAddressResourceInner object itself. - */ - public SuppressionListAddressResourceInner withLastName(String lastName) { - if (this.innerProperties() == null) { - this.innerProperties = new SuppressionListAddressProperties(); - } - this.innerProperties().withLastName(lastName); - return this; - } - - /** - * Get the notes property: An optional property to provide contextual notes or a description for an address. - * - * @return the notes value. - */ - public String notes() { - return this.innerProperties() == null ? null : this.innerProperties().notes(); - } - - /** - * Set the notes property: An optional property to provide contextual notes or a description for an address. - * - * @param notes the notes value to set. - * @return the SuppressionListAddressResourceInner object itself. - */ - public SuppressionListAddressResourceInner withNotes(String notes) { - if (this.innerProperties() == null) { - this.innerProperties = new SuppressionListAddressProperties(); - } - this.innerProperties().withNotes(notes); - return this; - } - - /** - * Get the lastModified property: The date the address was last updated in a suppression list. - * - * @return the lastModified value. - */ - public OffsetDateTime lastModified() { - return this.innerProperties() == null ? null : this.innerProperties().lastModified(); - } - - /** - * Get the dataLocation property: The location where the SuppressionListAddress data is stored at rest. This value - * is inherited from the parent Domains resource. - * - * @return the dataLocation value. - */ - public String dataLocation() { - return this.innerProperties() == null ? null : this.innerProperties().dataLocation(); - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListProperties.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListProperties.java deleted file mode 100644 index 60665aea39266..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListProperties.java +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * A class that describes the properties of a SuppressionList resource. - */ -@Fluent -public final class SuppressionListProperties { - /* - * The the name of the suppression list. This value must match one of the valid sender usernames of the sending - * domain. - */ - @JsonProperty(value = "listName") - private String listName; - - /* - * The date the resource was last updated. - */ - @JsonProperty(value = "lastUpdatedTimeStamp", access = JsonProperty.Access.WRITE_ONLY) - private String lastUpdatedTimestamp; - - /* - * The date the resource was created. - */ - @JsonProperty(value = "createdTimeStamp", access = JsonProperty.Access.WRITE_ONLY) - private String createdTimestamp; - - /* - * The location where the SuppressionListAddress data is stored at rest. This value is inherited from the parent - * Domains resource. - */ - @JsonProperty(value = "dataLocation", access = JsonProperty.Access.WRITE_ONLY) - private String dataLocation; - - /** - * Creates an instance of SuppressionListProperties class. - */ - public SuppressionListProperties() { - } - - /** - * Get the listName property: The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * - * @return the listName value. - */ - public String listName() { - return this.listName; - } - - /** - * Set the listName property: The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * - * @param listName the listName value to set. - * @return the SuppressionListProperties object itself. - */ - public SuppressionListProperties withListName(String listName) { - this.listName = listName; - return this; - } - - /** - * Get the lastUpdatedTimestamp property: The date the resource was last updated. - * - * @return the lastUpdatedTimestamp value. - */ - public String lastUpdatedTimestamp() { - return this.lastUpdatedTimestamp; - } - - /** - * Get the createdTimestamp property: The date the resource was created. - * - * @return the createdTimestamp value. - */ - public String createdTimestamp() { - return this.createdTimestamp; - } - - /** - * Get the dataLocation property: The location where the SuppressionListAddress data is stored at rest. This value - * is inherited from the parent Domains resource. - * - * @return the dataLocation value. - */ - public String dataLocation() { - return this.dataLocation; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListResourceInner.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListResourceInner.java deleted file mode 100644 index 7c6c73180aab3..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/fluent/models/SuppressionListResourceInner.java +++ /dev/null @@ -1,116 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * A class representing a SuppressionList resource. - */ -@Fluent -public final class SuppressionListResourceInner extends ProxyResource { - /* - * The properties of a SuppressionList resource. - */ - @JsonProperty(value = "properties") - private SuppressionListProperties innerProperties; - - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. - */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** - * Creates an instance of SuppressionListResourceInner class. - */ - public SuppressionListResourceInner() { - } - - /** - * Get the innerProperties property: The properties of a SuppressionList resource. - * - * @return the innerProperties value. - */ - private SuppressionListProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - - /** - * Get the listName property: The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * - * @return the listName value. - */ - public String listName() { - return this.innerProperties() == null ? null : this.innerProperties().listName(); - } - - /** - * Set the listName property: The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * - * @param listName the listName value to set. - * @return the SuppressionListResourceInner object itself. - */ - public SuppressionListResourceInner withListName(String listName) { - if (this.innerProperties() == null) { - this.innerProperties = new SuppressionListProperties(); - } - this.innerProperties().withListName(listName); - return this; - } - - /** - * Get the lastUpdatedTimestamp property: The date the resource was last updated. - * - * @return the lastUpdatedTimestamp value. - */ - public String lastUpdatedTimestamp() { - return this.innerProperties() == null ? null : this.innerProperties().lastUpdatedTimestamp(); - } - - /** - * Get the createdTimestamp property: The date the resource was created. - * - * @return the createdTimestamp value. - */ - public String createdTimestamp() { - return this.innerProperties() == null ? null : this.innerProperties().createdTimestamp(); - } - - /** - * Get the dataLocation property: The location where the SuppressionListAddress data is stored at rest. This value - * is inherited from the parent Domains resource. - * - * @return the dataLocation value. - */ - public String dataLocation() { - return this.innerProperties() == null ? null : this.innerProperties().dataLocation(); - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceManagementClientImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceManagementClientImpl.java index 2b69cf4b39121..f297a227c9385 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceManagementClientImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceManagementClientImpl.java @@ -28,8 +28,6 @@ import com.azure.resourcemanager.communication.fluent.EmailServicesClient; import com.azure.resourcemanager.communication.fluent.OperationsClient; import com.azure.resourcemanager.communication.fluent.SenderUsernamesClient; -import com.azure.resourcemanager.communication.fluent.SuppressionListAddressesClient; -import com.azure.resourcemanager.communication.fluent.SuppressionListsClient; import java.io.IOException; import java.lang.reflect.Type; import java.nio.ByteBuffer; @@ -198,34 +196,6 @@ public SenderUsernamesClient getSenderUsernames() { return this.senderUsernames; } - /** - * The SuppressionListsClient object to access its operations. - */ - private final SuppressionListsClient suppressionLists; - - /** - * Gets the SuppressionListsClient object to access its operations. - * - * @return the SuppressionListsClient object. - */ - public SuppressionListsClient getSuppressionLists() { - return this.suppressionLists; - } - - /** - * The SuppressionListAddressesClient object to access its operations. - */ - private final SuppressionListAddressesClient suppressionListAddresses; - - /** - * Gets the SuppressionListAddressesClient object to access its operations. - * - * @return the SuppressionListAddressesClient object. - */ - public SuppressionListAddressesClient getSuppressionListAddresses() { - return this.suppressionListAddresses; - } - /** * Initializes an instance of CommunicationServiceManagementClient client. * @@ -243,14 +213,12 @@ public SuppressionListAddressesClient getSuppressionListAddresses() { this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-06-01-preview"; + this.apiVersion = "2023-04-01"; this.operations = new OperationsClientImpl(this); this.communicationServices = new CommunicationServicesClientImpl(this); this.domains = new DomainsClientImpl(this); this.emailServices = new EmailServicesClientImpl(this); this.senderUsernames = new SenderUsernamesClientImpl(this); - this.suppressionLists = new SuppressionListsClientImpl(this); - this.suppressionListAddresses = new SuppressionListAddressesClientImpl(this); } /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceResourceImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceResourceImpl.java index f6dc1574b70cf..35e0b47bb48a8 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceResourceImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServiceResourceImpl.java @@ -164,8 +164,9 @@ public CommunicationServiceResource apply(Context context) { com.azure.resourcemanager.communication.CommunicationManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.communicationServiceName = Utils.getValueFromIdByName(innerObject.id(), "communicationServices"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.communicationServiceName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "communicationServices"); } public CommunicationServiceResource refresh() { diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServicesImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServicesImpl.java index f3afb6fd3bf71..575c15c42268a 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServicesImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/CommunicationServicesImpl.java @@ -82,24 +82,28 @@ public LinkedNotificationHub linkNotificationHub(String resourceGroupName, Strin public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CommunicationServiceResourceImpl(inner1, this.manager())); } public Response getByResourceGroupWithResponse(String resourceGroupName, @@ -178,12 +182,12 @@ public CommunicationServiceKeys regenerateKey(String resourceGroupName, String c } public CommunicationServiceResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String communicationServiceName = Utils.getValueFromIdByName(id, "communicationServices"); + String communicationServiceName = ResourceManagerUtils.getValueFromIdByName(id, "communicationServices"); if (communicationServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'communicationServices'.", id))); @@ -193,12 +197,12 @@ public CommunicationServiceResource getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String communicationServiceName = Utils.getValueFromIdByName(id, "communicationServices"); + String communicationServiceName = ResourceManagerUtils.getValueFromIdByName(id, "communicationServices"); if (communicationServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'communicationServices'.", id))); @@ -207,12 +211,12 @@ public Response getByIdWithResponse(String id, Con } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String communicationServiceName = Utils.getValueFromIdByName(id, "communicationServices"); + String communicationServiceName = ResourceManagerUtils.getValueFromIdByName(id, "communicationServices"); if (communicationServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'communicationServices'.", id))); @@ -221,12 +225,12 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String communicationServiceName = Utils.getValueFromIdByName(id, "communicationServices"); + String communicationServiceName = ResourceManagerUtils.getValueFromIdByName(id, "communicationServices"); if (communicationServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'communicationServices'.", id))); diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainResourceImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainResourceImpl.java index 4cb397ef66dc7..9ccd9a727e910 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainResourceImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainResourceImpl.java @@ -158,9 +158,9 @@ public DomainResource apply(Context context) { com.azure.resourcemanager.communication.CommunicationManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.emailServiceName = Utils.getValueFromIdByName(innerObject.id(), "emailServices"); - this.domainName = Utils.getValueFromIdByName(innerObject.id(), "domains"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.emailServiceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "emailServices"); + this.domainName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "domains"); } public DomainResource refresh() { diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainsImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainsImpl.java index d4211e204ebe8..1b33deb6111ab 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainsImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/DomainsImpl.java @@ -60,14 +60,14 @@ public void delete(String resourceGroupName, String emailServiceName, String dom public PagedIterable listByEmailServiceResource(String resourceGroupName, String emailServiceName) { PagedIterable inner = this.serviceClient().listByEmailServiceResource(resourceGroupName, emailServiceName); - return Utils.mapPage(inner, inner1 -> new DomainResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DomainResourceImpl(inner1, this.manager())); } public PagedIterable listByEmailServiceResource(String resourceGroupName, String emailServiceName, Context context) { PagedIterable inner = this.serviceClient().listByEmailServiceResource(resourceGroupName, emailServiceName, context); - return Utils.mapPage(inner, inner1 -> new DomainResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DomainResourceImpl(inner1, this.manager())); } public void initiateVerification(String resourceGroupName, String emailServiceName, String domainName, @@ -91,17 +91,17 @@ public void cancelVerification(String resourceGroupName, String emailServiceName } public DomainResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); @@ -110,17 +110,17 @@ public DomainResource getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); @@ -129,17 +129,17 @@ public Response getByIdWithResponse(String id, Context context) } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); @@ -148,17 +148,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServiceResourceImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServiceResourceImpl.java index e0f8c95daa74e..b90d6eaab9c82 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServiceResourceImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServiceResourceImpl.java @@ -127,8 +127,8 @@ public EmailServiceResource apply(Context context) { com.azure.resourcemanager.communication.CommunicationManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.emailServiceName = Utils.getValueFromIdByName(innerObject.id(), "emailServices"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.emailServiceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "emailServices"); } public EmailServiceResource refresh() { diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServicesImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServicesImpl.java index c5eb60ab3d84c..1bded74cf4130 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServicesImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/EmailServicesImpl.java @@ -60,23 +60,23 @@ public void delete(String resourceGroupName, String emailServiceName, Context co public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new EmailServiceResourceImpl(inner1, this.manager())); } public Response> listVerifiedExchangeOnlineDomainsWithResponse(Context context) { @@ -93,12 +93,12 @@ public List listVerifiedExchangeOnlineDomains() { } public EmailServiceResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); @@ -107,12 +107,12 @@ public EmailServiceResource getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); @@ -121,12 +121,12 @@ public Response getByIdWithResponse(String id, Context con } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); @@ -135,12 +135,12 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/OperationsImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/OperationsImpl.java index 9ae692e3eb4fc..00ae41663b401 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/OperationsImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/OperationsImpl.java @@ -27,12 +27,12 @@ public OperationsImpl(OperationsClient innerClient, public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/Utils.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/ResourceManagerUtils.java similarity index 99% rename from sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/Utils.java rename to sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/ResourceManagerUtils.java index 948ad475cf5d9..31004fac799fa 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/Utils.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/ResourceManagerUtils.java @@ -19,8 +19,8 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { - private Utils() { +final class ResourceManagerUtils { + private ResourceManagerUtils() { } static String getValueFromIdByName(String id, String name) { diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernameResourceImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernameResourceImpl.java index c03428c1b0ed9..990d14def0cfe 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernameResourceImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernameResourceImpl.java @@ -119,10 +119,10 @@ public SenderUsernameResource apply(Context context) { com.azure.resourcemanager.communication.CommunicationManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.emailServiceName = Utils.getValueFromIdByName(innerObject.id(), "emailServices"); - this.domainName = Utils.getValueFromIdByName(innerObject.id(), "domains"); - this.senderUsername = Utils.getValueFromIdByName(innerObject.id(), "senderUsernames"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.emailServiceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "emailServices"); + this.domainName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "domains"); + this.senderUsername = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "senderUsernames"); } public SenderUsernameResource refresh() { diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernamesImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernamesImpl.java index 8952ec9d44d4d..b83e97238d9ec 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernamesImpl.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SenderUsernamesImpl.java @@ -31,14 +31,14 @@ public PagedIterable listByDomains(String resourceGroupN String domainName) { PagedIterable inner = this.serviceClient().listByDomains(resourceGroupName, emailServiceName, domainName); - return Utils.mapPage(inner, inner1 -> new SenderUsernameResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SenderUsernameResourceImpl(inner1, this.manager())); } public PagedIterable listByDomains(String resourceGroupName, String emailServiceName, String domainName, Context context) { PagedIterable inner = this.serviceClient().listByDomains(resourceGroupName, emailServiceName, domainName, context); - return Utils.mapPage(inner, inner1 -> new SenderUsernameResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SenderUsernameResourceImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String emailServiceName, @@ -75,22 +75,22 @@ public void delete(String resourceGroupName, String emailServiceName, String dom } public SenderUsernameResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); } - String senderUsername = Utils.getValueFromIdByName(id, "senderUsernames"); + String senderUsername = ResourceManagerUtils.getValueFromIdByName(id, "senderUsernames"); if (senderUsername == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'senderUsernames'.", id))); @@ -100,22 +100,22 @@ public SenderUsernameResource getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); } - String senderUsername = Utils.getValueFromIdByName(id, "senderUsernames"); + String senderUsername = ResourceManagerUtils.getValueFromIdByName(id, "senderUsernames"); if (senderUsername == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'senderUsernames'.", id))); @@ -124,22 +124,22 @@ public Response getByIdWithResponse(String id, Context c } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); } - String senderUsername = Utils.getValueFromIdByName(id, "senderUsernames"); + String senderUsername = ResourceManagerUtils.getValueFromIdByName(id, "senderUsernames"); if (senderUsername == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'senderUsernames'.", id))); @@ -148,22 +148,22 @@ public void deleteById(String id) { } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); + String emailServiceName = ResourceManagerUtils.getValueFromIdByName(id, "emailServices"); if (emailServiceName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); } - String domainName = Utils.getValueFromIdByName(id, "domains"); + String domainName = ResourceManagerUtils.getValueFromIdByName(id, "domains"); if (domainName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); } - String senderUsername = Utils.getValueFromIdByName(id, "senderUsernames"); + String senderUsername = ResourceManagerUtils.getValueFromIdByName(id, "senderUsernames"); if (senderUsername == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'senderUsernames'.", id))); diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressResourceImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressResourceImpl.java deleted file mode 100644 index 79ae511bb3ac0..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressResourceImpl.java +++ /dev/null @@ -1,179 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.implementation; - -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListAddressResource; -import java.time.OffsetDateTime; - -public final class SuppressionListAddressResourceImpl implements SuppressionListAddressResource, - SuppressionListAddressResource.Definition, SuppressionListAddressResource.Update { - private SuppressionListAddressResourceInner innerObject; - - private final com.azure.resourcemanager.communication.CommunicationManager serviceManager; - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public SystemData systemData() { - return this.innerModel().systemData(); - } - - public String email() { - return this.innerModel().email(); - } - - public String firstName() { - return this.innerModel().firstName(); - } - - public String lastName() { - return this.innerModel().lastName(); - } - - public String notes() { - return this.innerModel().notes(); - } - - public OffsetDateTime lastModified() { - return this.innerModel().lastModified(); - } - - public String dataLocation() { - return this.innerModel().dataLocation(); - } - - public String resourceGroupName() { - return resourceGroupName; - } - - public SuppressionListAddressResourceInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.communication.CommunicationManager manager() { - return this.serviceManager; - } - - private String resourceGroupName; - - private String emailServiceName; - - private String domainName; - - private String suppressionListName; - - private String addressId; - - public SuppressionListAddressResourceImpl withExistingSuppressionList(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName) { - this.resourceGroupName = resourceGroupName; - this.emailServiceName = emailServiceName; - this.domainName = domainName; - this.suppressionListName = suppressionListName; - return this; - } - - public SuppressionListAddressResource create() { - this.innerObject = serviceManager - .serviceClient().getSuppressionListAddresses().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, addressId, this.innerModel(), Context.NONE) - .getValue(); - return this; - } - - public SuppressionListAddressResource create(Context context) { - this.innerObject - = serviceManager - .serviceClient().getSuppressionListAddresses().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, addressId, this.innerModel(), context) - .getValue(); - return this; - } - - SuppressionListAddressResourceImpl(String name, - com.azure.resourcemanager.communication.CommunicationManager serviceManager) { - this.innerObject = new SuppressionListAddressResourceInner(); - this.serviceManager = serviceManager; - this.addressId = name; - } - - public SuppressionListAddressResourceImpl update() { - return this; - } - - public SuppressionListAddressResource apply() { - this.innerObject = serviceManager - .serviceClient().getSuppressionListAddresses().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, addressId, this.innerModel(), Context.NONE) - .getValue(); - return this; - } - - public SuppressionListAddressResource apply(Context context) { - this.innerObject - = serviceManager - .serviceClient().getSuppressionListAddresses().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, addressId, this.innerModel(), context) - .getValue(); - return this; - } - - SuppressionListAddressResourceImpl(SuppressionListAddressResourceInner innerObject, - com.azure.resourcemanager.communication.CommunicationManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.emailServiceName = Utils.getValueFromIdByName(innerObject.id(), "emailServices"); - this.domainName = Utils.getValueFromIdByName(innerObject.id(), "domains"); - this.suppressionListName = Utils.getValueFromIdByName(innerObject.id(), "suppressionLists"); - this.addressId = Utils.getValueFromIdByName(innerObject.id(), "suppressionListAddresses"); - } - - public SuppressionListAddressResource refresh() { - this.innerObject - = serviceManager.serviceClient().getSuppressionListAddresses().getWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, addressId, Context.NONE).getValue(); - return this; - } - - public SuppressionListAddressResource refresh(Context context) { - this.innerObject = serviceManager.serviceClient().getSuppressionListAddresses() - .getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, context) - .getValue(); - return this; - } - - public SuppressionListAddressResourceImpl withEmail(String email) { - this.innerModel().withEmail(email); - return this; - } - - public SuppressionListAddressResourceImpl withFirstName(String firstName) { - this.innerModel().withFirstName(firstName); - return this; - } - - public SuppressionListAddressResourceImpl withLastName(String lastName) { - this.innerModel().withLastName(lastName); - return this; - } - - public SuppressionListAddressResourceImpl withNotes(String notes) { - this.innerModel().withNotes(notes); - return this; - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesClientImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesClientImpl.java deleted file mode 100644 index 48f9c40e59a57..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesClientImpl.java +++ /dev/null @@ -1,900 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.implementation; - -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.Delete; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.Get; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Put; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.communication.fluent.SuppressionListAddressesClient; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListAddressResourceCollection; -import reactor.core.publisher.Mono; - -/** - * An instance of this class provides access to all the operations defined in SuppressionListAddressesClient. - */ -public final class SuppressionListAddressesClientImpl implements SuppressionListAddressesClient { - /** - * The proxy service used to perform REST calls. - */ - private final SuppressionListAddressesService service; - - /** - * The service client containing this operation class. - */ - private final CommunicationServiceManagementClientImpl client; - - /** - * Initializes an instance of SuppressionListAddressesClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - SuppressionListAddressesClientImpl(CommunicationServiceManagementClientImpl client) { - this.service = RestProxy.create(SuppressionListAddressesService.class, client.getHttpPipeline(), - client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for CommunicationServiceManagementClientSuppressionListAddresses to be - * used by the proxy service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "CommunicationService") - public interface SuppressionListAddressesService { - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}/suppressionListAddresses") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list(@HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, @HeaderParam("Accept") String accept, - Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}/suppressionListAddresses/{addressId}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get(@HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, @PathParam("addressId") String addressId, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}/suppressionListAddresses/{addressId}") - @ExpectedResponses({ 200, 201 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate(@HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, @PathParam("addressId") String addressId, - @BodyParam("application/json") SuppressionListAddressResourceInner parameters, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}/suppressionListAddresses/{addressId}") - @ExpectedResponses({ 200, 204 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, @PathParam("addressId") String addressId, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("{nextLink}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, Context context); - } - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list along with {@link PagedResponse} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list along with {@link PagedResponse} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), - resourceGroupName, emailServiceName, domainName, suppressionListName, accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, emailServiceName, domainName, suppressionListName), - nextLink -> listNextSinglePageAsync(nextLink)); - } - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, context), - nextLink -> listNextSinglePageAsync(nextLink, context)); - } - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName) { - return new PagedIterable<>(listAsync(resourceGroupName, emailServiceName, domainName, suppressionListName)); - } - - /** - * Get - * - * Get all the addresses in a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the addresses in a suppression list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context) { - return new PagedIterable<>( - listAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, context)); - } - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, String addressId) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (addressId == null) { - return Mono.error(new IllegalArgumentException("Parameter addressId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, String addressId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (addressId == null) { - return Mono.error(new IllegalArgumentException("Parameter addressId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), - resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, accept, context); - } - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId) { - return getWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, String addressId, Context context) { - return getWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - context).block(); - } - - /** - * Get - * - * Get a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionListAddress. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public SuppressionListAddressResourceInner get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId) { - return getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - Context.NONE).getValue(); - } - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String emailServiceName, String domainName, String suppressionListName, - String addressId, SuppressionListAddressResourceInner parameters) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (addressId == null) { - return Mono.error(new IllegalArgumentException("Parameter addressId is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, parameters, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String emailServiceName, String domainName, String suppressionListName, - String addressId, SuppressionListAddressResourceInner parameters, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (addressId == null) { - return Mono.error(new IllegalArgumentException("Parameter addressId is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, parameters, accept, context); - } - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, String addressId, - SuppressionListAddressResourceInner parameters) { - return createOrUpdateWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, String addressId, - SuppressionListAddressResourceInner parameters, Context context) { - return createOrUpdateWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, parameters, context).block(); - } - - /** - * Create Or Update - * - * Create or update a SuppressionListAddress. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a object that represents a SuppressionList record. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public SuppressionListAddressResourceInner createOrUpdate(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId, - SuppressionListAddressResourceInner parameters) { - return createOrUpdateWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, parameters, Context.NONE).getValue(); - } - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (addressId == null) { - return Mono.error(new IllegalArgumentException("Parameter addressId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - addressId, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (addressId == null) { - return Mono.error(new IllegalArgumentException("Parameter addressId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), - resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, accept, context); - } - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId) { - return deleteWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId) - .flatMap(ignored -> Mono.empty()); - } - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId, Context context) { - return deleteWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - context).block(); - } - - /** - * Delete - * - * Operation to delete a single address from a suppression list. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param addressId The id of the address in a suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String emailServiceName, String domainName, String suppressionListName, - String addressId) { - deleteWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - Context.NONE); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of addresses in a suppression list along with {@link PagedResponse} on successful completion - * of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return collection of addresses in a suppression list along with {@link PagedResponse} on successful completion - * of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink, - Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.listNext(nextLink, this.client.getEndpoint(), accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesImpl.java deleted file mode 100644 index 6611b075166ce..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListAddressesImpl.java +++ /dev/null @@ -1,209 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.communication.fluent.SuppressionListAddressesClient; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListAddressResource; -import com.azure.resourcemanager.communication.models.SuppressionListAddresses; - -public final class SuppressionListAddressesImpl implements SuppressionListAddresses { - private static final ClientLogger LOGGER = new ClientLogger(SuppressionListAddressesImpl.class); - - private final SuppressionListAddressesClient innerClient; - - private final com.azure.resourcemanager.communication.CommunicationManager serviceManager; - - public SuppressionListAddressesImpl(SuppressionListAddressesClient innerClient, - com.azure.resourcemanager.communication.CommunicationManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName) { - PagedIterable inner - = this.serviceClient().list(resourceGroupName, emailServiceName, domainName, suppressionListName); - return Utils.mapPage(inner, inner1 -> new SuppressionListAddressResourceImpl(inner1, this.manager())); - } - - public PagedIterable list(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context) { - PagedIterable inner - = this.serviceClient().list(resourceGroupName, emailServiceName, domainName, suppressionListName, context); - return Utils.mapPage(inner, inner1 -> new SuppressionListAddressResourceImpl(inner1, this.manager())); - } - - public Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, String addressId, Context context) { - Response inner = this.serviceClient().getWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, addressId, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new SuppressionListAddressResourceImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public SuppressionListAddressResource get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId) { - SuppressionListAddressResourceInner inner - = this.serviceClient().get(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId); - if (inner != null) { - return new SuppressionListAddressResourceImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, String addressId, Context context) { - return this.serviceClient().deleteWithResponse(resourceGroupName, emailServiceName, domainName, - suppressionListName, addressId, context); - } - - public void delete(String resourceGroupName, String emailServiceName, String domainName, String suppressionListName, - String addressId) { - this.serviceClient().delete(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId); - } - - public SuppressionListAddressResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - String addressId = Utils.getValueFromIdByName(id, "suppressionListAddresses"); - if (addressId == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException(String - .format("The resource ID '%s' is not valid. Missing path segment 'suppressionListAddresses'.", id))); - } - return this.getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - Context.NONE).getValue(); - } - - public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - String addressId = Utils.getValueFromIdByName(id, "suppressionListAddresses"); - if (addressId == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException(String - .format("The resource ID '%s' is not valid. Missing path segment 'suppressionListAddresses'.", id))); - } - return this.getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - context); - } - - public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - String addressId = Utils.getValueFromIdByName(id, "suppressionListAddresses"); - if (addressId == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException(String - .format("The resource ID '%s' is not valid. Missing path segment 'suppressionListAddresses'.", id))); - } - this.deleteWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - Context.NONE); - } - - public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - String addressId = Utils.getValueFromIdByName(id, "suppressionListAddresses"); - if (addressId == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException(String - .format("The resource ID '%s' is not valid. Missing path segment 'suppressionListAddresses'.", id))); - } - return this.deleteWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, addressId, - context); - } - - private SuppressionListAddressesClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.communication.CommunicationManager manager() { - return this.serviceManager; - } - - public SuppressionListAddressResourceImpl define(String name) { - return new SuppressionListAddressResourceImpl(name, this.manager()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListResourceImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListResourceImpl.java deleted file mode 100644 index 5a49c380727be..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListResourceImpl.java +++ /dev/null @@ -1,144 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.implementation; - -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListResource; - -public final class SuppressionListResourceImpl - implements SuppressionListResource, SuppressionListResource.Definition, SuppressionListResource.Update { - private SuppressionListResourceInner innerObject; - - private final com.azure.resourcemanager.communication.CommunicationManager serviceManager; - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public SystemData systemData() { - return this.innerModel().systemData(); - } - - public String listName() { - return this.innerModel().listName(); - } - - public String lastUpdatedTimestamp() { - return this.innerModel().lastUpdatedTimestamp(); - } - - public String createdTimestamp() { - return this.innerModel().createdTimestamp(); - } - - public String dataLocation() { - return this.innerModel().dataLocation(); - } - - public String resourceGroupName() { - return resourceGroupName; - } - - public SuppressionListResourceInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.communication.CommunicationManager manager() { - return this.serviceManager; - } - - private String resourceGroupName; - - private String emailServiceName; - - private String domainName; - - private String suppressionListName; - - public SuppressionListResourceImpl withExistingDomain(String resourceGroupName, String emailServiceName, - String domainName) { - this.resourceGroupName = resourceGroupName; - this.emailServiceName = emailServiceName; - this.domainName = domainName; - return this; - } - - public SuppressionListResource create() { - this.innerObject - = serviceManager.serviceClient().getSuppressionLists().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, this.innerModel(), Context.NONE).getValue(); - return this; - } - - public SuppressionListResource create(Context context) { - this.innerObject - = serviceManager.serviceClient().getSuppressionLists().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, this.innerModel(), context).getValue(); - return this; - } - - SuppressionListResourceImpl(String name, - com.azure.resourcemanager.communication.CommunicationManager serviceManager) { - this.innerObject = new SuppressionListResourceInner(); - this.serviceManager = serviceManager; - this.suppressionListName = name; - } - - public SuppressionListResourceImpl update() { - return this; - } - - public SuppressionListResource apply() { - this.innerObject - = serviceManager.serviceClient().getSuppressionLists().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, this.innerModel(), Context.NONE).getValue(); - return this; - } - - public SuppressionListResource apply(Context context) { - this.innerObject - = serviceManager.serviceClient().getSuppressionLists().createOrUpdateWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, this.innerModel(), context).getValue(); - return this; - } - - SuppressionListResourceImpl(SuppressionListResourceInner innerObject, - com.azure.resourcemanager.communication.CommunicationManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.emailServiceName = Utils.getValueFromIdByName(innerObject.id(), "emailServices"); - this.domainName = Utils.getValueFromIdByName(innerObject.id(), "domains"); - this.suppressionListName = Utils.getValueFromIdByName(innerObject.id(), "suppressionLists"); - } - - public SuppressionListResource refresh() { - this.innerObject = serviceManager.serviceClient().getSuppressionLists() - .getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, Context.NONE) - .getValue(); - return this; - } - - public SuppressionListResource refresh(Context context) { - this.innerObject = serviceManager.serviceClient().getSuppressionLists() - .getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, context).getValue(); - return this; - } - - public SuppressionListResourceImpl withListName(String listName) { - this.innerModel().withListName(listName); - return this; - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsClientImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsClientImpl.java deleted file mode 100644 index 65bf17e244c73..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsClientImpl.java +++ /dev/null @@ -1,856 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.implementation; - -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.Delete; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.Get; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Put; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.communication.fluent.SuppressionListsClient; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListResourceCollection; -import reactor.core.publisher.Mono; - -/** - * An instance of this class provides access to all the operations defined in SuppressionListsClient. - */ -public final class SuppressionListsClientImpl implements SuppressionListsClient { - /** - * The proxy service used to perform REST calls. - */ - private final SuppressionListsService service; - - /** - * The service client containing this operation class. - */ - private final CommunicationServiceManagementClientImpl client; - - /** - * Initializes an instance of SuppressionListsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - SuppressionListsClientImpl(CommunicationServiceManagementClientImpl client) { - this.service - = RestProxy.create(SuppressionListsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for CommunicationServiceManagementClientSuppressionLists to be used by - * the proxy service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "CommunicationService") - public interface SuppressionListsService { - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByDomain(@HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get(@HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, @HeaderParam("Accept") String accept, - Context context); - - @Headers({ "Content-Type: application/json" }) - @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}") - @ExpectedResponses({ 200, 201 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate(@HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, - @BodyParam("application/json") SuppressionListResourceInner parameters, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Communication/emailServices/{emailServiceName}/domains/{domainName}/suppressionLists/{suppressionListName}") - @ExpectedResponses({ 200, 204 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("emailServiceName") String emailServiceName, @PathParam("domainName") String domainName, - @PathParam("suppressionListName") String suppressionListName, @HeaderParam("Accept") String accept, - Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("{nextLink}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByDomainNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, Context context); - } - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection along with {@link PagedResponse} on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDomainSinglePageAsync(String resourceGroupName, - String emailServiceName, String domainName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listByDomain(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection along with {@link PagedResponse} on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDomainSinglePageAsync(String resourceGroupName, - String emailServiceName, String domainName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listByDomain(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), - resourceGroupName, emailServiceName, domainName, accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDomainAsync(String resourceGroupName, String emailServiceName, - String domainName) { - return new PagedFlux<>(() -> listByDomainSinglePageAsync(resourceGroupName, emailServiceName, domainName), - nextLink -> listByDomainNextSinglePageAsync(nextLink)); - } - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByDomainAsync(String resourceGroupName, String emailServiceName, - String domainName, Context context) { - return new PagedFlux<>( - () -> listByDomainSinglePageAsync(resourceGroupName, emailServiceName, domainName, context), - nextLink -> listByDomainNextSinglePageAsync(nextLink, context)); - } - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName) { - return new PagedIterable<>(listByDomainAsync(resourceGroupName, emailServiceName, domainName)); - } - - /** - * List - * - * List all suppression lists for a domains resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName, Context context) { - return new PagedIterable<>(listByDomainAsync(resourceGroupName, emailServiceName, domainName, context)); - } - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), - resourceGroupName, emailServiceName, domainName, suppressionListName, accept, context); - } - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName) { - return getWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context) { - return getWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, context) - .block(); - } - - /** - * Get - * - * Get a SuppressionList resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a SuppressionList resource. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public SuppressionListResourceInner get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName) { - return getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, Context.NONE) - .getValue(); - } - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, - SuppressionListResourceInner parameters) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - parameters, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, SuppressionListResourceInner parameters, - Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - parameters, accept, context); - } - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, SuppressionListResourceInner parameters) { - return createOrUpdateWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, - parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse(String resourceGroupName, - String emailServiceName, String domainName, String suppressionListName, SuppressionListResourceInner parameters, - Context context) { - return createOrUpdateWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, - parameters, context).block(); - } - - /** - * Create Or Update - * - * Add a new SuppressionList resource under the parent Domains resource or update an existing SuppressionList - * resource. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param parameters Parameters for the create or update operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a SuppressionList resource. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public SuppressionListResourceInner createOrUpdate(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, SuppressionListResourceInner parameters) { - return createOrUpdateWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, - parameters, Context.NONE).getValue(); - } - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), - this.client.getSubscriptionId(), resourceGroupName, emailServiceName, domainName, suppressionListName, - accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (emailServiceName == null) { - return Mono - .error(new IllegalArgumentException("Parameter emailServiceName is required and cannot be null.")); - } - if (domainName == null) { - return Mono.error(new IllegalArgumentException("Parameter domainName is required and cannot be null.")); - } - if (suppressionListName == null) { - return Mono - .error(new IllegalArgumentException("Parameter suppressionListName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), - resourceGroupName, emailServiceName, domainName, suppressionListName, accept, context); - } - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName) { - return deleteWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName) - .flatMap(ignored -> Mono.empty()); - } - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, Context context) { - return deleteWithResponseAsync(resourceGroupName, emailServiceName, domainName, suppressionListName, context) - .block(); - } - - /** - * Delete - * - * Delete a SuppressionList. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName) { - deleteWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, Context.NONE); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection along with {@link PagedResponse} on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDomainNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listByDomainNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a class representing a Domains SuppressionListResource collection along with {@link PagedResponse} on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByDomainNextSinglePageAsync(String nextLink, - Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.listByDomainNext(nextLink, this.client.getEndpoint(), accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsImpl.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsImpl.java deleted file mode 100644 index da7eb0ac9b929..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/implementation/SuppressionListsImpl.java +++ /dev/null @@ -1,186 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.communication.fluent.SuppressionListsClient; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListResource; -import com.azure.resourcemanager.communication.models.SuppressionLists; - -public final class SuppressionListsImpl implements SuppressionLists { - private static final ClientLogger LOGGER = new ClientLogger(SuppressionListsImpl.class); - - private final SuppressionListsClient innerClient; - - private final com.azure.resourcemanager.communication.CommunicationManager serviceManager; - - public SuppressionListsImpl(SuppressionListsClient innerClient, - com.azure.resourcemanager.communication.CommunicationManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName) { - PagedIterable inner - = this.serviceClient().listByDomain(resourceGroupName, emailServiceName, domainName); - return Utils.mapPage(inner, inner1 -> new SuppressionListResourceImpl(inner1, this.manager())); - } - - public PagedIterable listByDomain(String resourceGroupName, String emailServiceName, - String domainName, Context context) { - PagedIterable inner - = this.serviceClient().listByDomain(resourceGroupName, emailServiceName, domainName, context); - return Utils.mapPage(inner, inner1 -> new SuppressionListResourceImpl(inner1, this.manager())); - } - - public Response getWithResponse(String resourceGroupName, String emailServiceName, - String domainName, String suppressionListName, Context context) { - Response inner = this.serviceClient().getWithResponse(resourceGroupName, - emailServiceName, domainName, suppressionListName, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new SuppressionListResourceImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public SuppressionListResource get(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName) { - SuppressionListResourceInner inner - = this.serviceClient().get(resourceGroupName, emailServiceName, domainName, suppressionListName); - if (inner != null) { - return new SuppressionListResourceImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response deleteWithResponse(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName, Context context) { - return this.serviceClient().deleteWithResponse(resourceGroupName, emailServiceName, domainName, - suppressionListName, context); - } - - public void delete(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName) { - this.serviceClient().delete(resourceGroupName, emailServiceName, domainName, suppressionListName); - } - - public SuppressionListResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - return this.getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, Context.NONE) - .getValue(); - } - - public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - return this.getWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, context); - } - - public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - this.deleteWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, Context.NONE); - } - - public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String emailServiceName = Utils.getValueFromIdByName(id, "emailServices"); - if (emailServiceName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'emailServices'.", id))); - } - String domainName = Utils.getValueFromIdByName(id, "domains"); - if (domainName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'domains'.", id))); - } - String suppressionListName = Utils.getValueFromIdByName(id, "suppressionLists"); - if (suppressionListName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'suppressionLists'.", id))); - } - return this.deleteWithResponse(resourceGroupName, emailServiceName, domainName, suppressionListName, context); - } - - private SuppressionListsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.communication.CommunicationManager manager() { - return this.serviceManager; - } - - public SuppressionListResourceImpl define(String name) { - return new SuppressionListResourceImpl(name, this.manager()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResource.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResource.java deleted file mode 100644 index bcd43c596fdc6..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResource.java +++ /dev/null @@ -1,311 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.models; - -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import java.time.OffsetDateTime; - -/** - * An immutable client-side representation of SuppressionListAddressResource. - */ -public interface SuppressionListAddressResource { - /** - * Gets the id property: Fully qualified resource Id for the resource. - * - * @return the id value. - */ - String id(); - - /** - * Gets the name property: The name of the resource. - * - * @return the name value. - */ - String name(); - - /** - * Gets the type property: The type of the resource. - * - * @return the type value. - */ - String type(); - - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - - /** - * Gets the email property: Email address of the recipient. - * - * @return the email value. - */ - String email(); - - /** - * Gets the firstName property: The first name of the email recipient. - * - * @return the firstName value. - */ - String firstName(); - - /** - * Gets the lastName property: The last name of the email recipient. - * - * @return the lastName value. - */ - String lastName(); - - /** - * Gets the notes property: An optional property to provide contextual notes or a description for an address. - * - * @return the notes value. - */ - String notes(); - - /** - * Gets the lastModified property: The date the address was last updated in a suppression list. - * - * @return the lastModified value. - */ - OffsetDateTime lastModified(); - - /** - * Gets the dataLocation property: The location where the SuppressionListAddress data is stored at rest. This value - * is inherited from the parent Domains resource. - * - * @return the dataLocation value. - */ - String dataLocation(); - - /** - * Gets the name of the resource group. - * - * @return the name of the resource group. - */ - String resourceGroupName(); - - /** - * Gets the inner com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner object. - * - * @return the inner object. - */ - SuppressionListAddressResourceInner innerModel(); - - /** - * The entirety of the SuppressionListAddressResource definition. - */ - interface Definition - extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { - } - - /** - * The SuppressionListAddressResource definition stages. - */ - interface DefinitionStages { - /** - * The first stage of the SuppressionListAddressResource definition. - */ - interface Blank extends WithParentResource { - } - - /** - * The stage of the SuppressionListAddressResource definition allowing to specify parent resource. - */ - interface WithParentResource { - /** - * Specifies resourceGroupName, emailServiceName, domainName, suppressionListName. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @param suppressionListName The name of the suppression list. - * @return the next definition stage. - */ - WithCreate withExistingSuppressionList(String resourceGroupName, String emailServiceName, String domainName, - String suppressionListName); - } - - /** - * The stage of the SuppressionListAddressResource definition which contains all the minimum required properties - * for the resource to be created, but also allows for any other optional properties to be specified. - */ - interface WithCreate extends DefinitionStages.WithEmail, DefinitionStages.WithFirstName, - DefinitionStages.WithLastName, DefinitionStages.WithNotes { - /** - * Executes the create request. - * - * @return the created resource. - */ - SuppressionListAddressResource create(); - - /** - * Executes the create request. - * - * @param context The context to associate with this operation. - * @return the created resource. - */ - SuppressionListAddressResource create(Context context); - } - - /** - * The stage of the SuppressionListAddressResource definition allowing to specify email. - */ - interface WithEmail { - /** - * Specifies the email property: Email address of the recipient.. - * - * @param email Email address of the recipient. - * @return the next definition stage. - */ - WithCreate withEmail(String email); - } - - /** - * The stage of the SuppressionListAddressResource definition allowing to specify firstName. - */ - interface WithFirstName { - /** - * Specifies the firstName property: The first name of the email recipient.. - * - * @param firstName The first name of the email recipient. - * @return the next definition stage. - */ - WithCreate withFirstName(String firstName); - } - - /** - * The stage of the SuppressionListAddressResource definition allowing to specify lastName. - */ - interface WithLastName { - /** - * Specifies the lastName property: The last name of the email recipient.. - * - * @param lastName The last name of the email recipient. - * @return the next definition stage. - */ - WithCreate withLastName(String lastName); - } - - /** - * The stage of the SuppressionListAddressResource definition allowing to specify notes. - */ - interface WithNotes { - /** - * Specifies the notes property: An optional property to provide contextual notes or a description for an - * address.. - * - * @param notes An optional property to provide contextual notes or a description for an address. - * @return the next definition stage. - */ - WithCreate withNotes(String notes); - } - } - - /** - * Begins update for the SuppressionListAddressResource resource. - * - * @return the stage of resource update. - */ - SuppressionListAddressResource.Update update(); - - /** - * The template for SuppressionListAddressResource update. - */ - interface Update - extends UpdateStages.WithEmail, UpdateStages.WithFirstName, UpdateStages.WithLastName, UpdateStages.WithNotes { - /** - * Executes the update request. - * - * @return the updated resource. - */ - SuppressionListAddressResource apply(); - - /** - * Executes the update request. - * - * @param context The context to associate with this operation. - * @return the updated resource. - */ - SuppressionListAddressResource apply(Context context); - } - - /** - * The SuppressionListAddressResource update stages. - */ - interface UpdateStages { - /** - * The stage of the SuppressionListAddressResource update allowing to specify email. - */ - interface WithEmail { - /** - * Specifies the email property: Email address of the recipient.. - * - * @param email Email address of the recipient. - * @return the next definition stage. - */ - Update withEmail(String email); - } - - /** - * The stage of the SuppressionListAddressResource update allowing to specify firstName. - */ - interface WithFirstName { - /** - * Specifies the firstName property: The first name of the email recipient.. - * - * @param firstName The first name of the email recipient. - * @return the next definition stage. - */ - Update withFirstName(String firstName); - } - - /** - * The stage of the SuppressionListAddressResource update allowing to specify lastName. - */ - interface WithLastName { - /** - * Specifies the lastName property: The last name of the email recipient.. - * - * @param lastName The last name of the email recipient. - * @return the next definition stage. - */ - Update withLastName(String lastName); - } - - /** - * The stage of the SuppressionListAddressResource update allowing to specify notes. - */ - interface WithNotes { - /** - * Specifies the notes property: An optional property to provide contextual notes or a description for an - * address.. - * - * @param notes An optional property to provide contextual notes or a description for an address. - * @return the next definition stage. - */ - Update withNotes(String notes); - } - } - - /** - * Refreshes the resource to sync with Azure. - * - * @return the refreshed resource. - */ - SuppressionListAddressResource refresh(); - - /** - * Refreshes the resource to sync with Azure. - * - * @param context The context to associate with this operation. - * @return the refreshed resource. - */ - SuppressionListAddressResource refresh(Context context); -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResourceCollection.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResourceCollection.java deleted file mode 100644 index 11d67128edf82..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListAddressResourceCollection.java +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.models; - -import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** - * Collection of addresses in a suppression list. Response will include a nextLink if response contains more pages. - */ -@Fluent -public final class SuppressionListAddressResourceCollection { - /* - * List of suppressed email addresses. - */ - @JsonProperty(value = "value") - private List value; - - /* - * The URL the client should use to fetch the next page (per server side paging). - */ - @JsonProperty(value = "nextLink") - private String nextLink; - - /** - * Creates an instance of SuppressionListAddressResourceCollection class. - */ - public SuppressionListAddressResourceCollection() { - } - - /** - * Get the value property: List of suppressed email addresses. - * - * @return the value value. - */ - public List value() { - return this.value; - } - - /** - * Set the value property: List of suppressed email addresses. - * - * @param value the value value to set. - * @return the SuppressionListAddressResourceCollection object itself. - */ - public SuppressionListAddressResourceCollection withValue(List value) { - this.value = value; - return this; - } - - /** - * Get the nextLink property: The URL the client should use to fetch the next page (per server side paging). - * - * @return the nextLink value. - */ - public String nextLink() { - return this.nextLink; - } - - /** - * Set the nextLink property: The URL the client should use to fetch the next page (per server side paging). - * - * @param nextLink the nextLink value to set. - * @return the SuppressionListAddressResourceCollection object itself. - */ - public SuppressionListAddressResourceCollection withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (value() != null) { - value().forEach(e -> e.validate()); - } - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResource.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResource.java deleted file mode 100644 index 9bc0a9e516374..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResource.java +++ /dev/null @@ -1,217 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.models; - -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; - -/** - * An immutable client-side representation of SuppressionListResource. - */ -public interface SuppressionListResource { - /** - * Gets the id property: Fully qualified resource Id for the resource. - * - * @return the id value. - */ - String id(); - - /** - * Gets the name property: The name of the resource. - * - * @return the name value. - */ - String name(); - - /** - * Gets the type property: The type of the resource. - * - * @return the type value. - */ - String type(); - - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - - /** - * Gets the listName property: The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * - * @return the listName value. - */ - String listName(); - - /** - * Gets the lastUpdatedTimestamp property: The date the resource was last updated. - * - * @return the lastUpdatedTimestamp value. - */ - String lastUpdatedTimestamp(); - - /** - * Gets the createdTimestamp property: The date the resource was created. - * - * @return the createdTimestamp value. - */ - String createdTimestamp(); - - /** - * Gets the dataLocation property: The location where the SuppressionListAddress data is stored at rest. This value - * is inherited from the parent Domains resource. - * - * @return the dataLocation value. - */ - String dataLocation(); - - /** - * Gets the name of the resource group. - * - * @return the name of the resource group. - */ - String resourceGroupName(); - - /** - * Gets the inner com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner object. - * - * @return the inner object. - */ - SuppressionListResourceInner innerModel(); - - /** - * The entirety of the SuppressionListResource definition. - */ - interface Definition - extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { - } - - /** - * The SuppressionListResource definition stages. - */ - interface DefinitionStages { - /** - * The first stage of the SuppressionListResource definition. - */ - interface Blank extends WithParentResource { - } - - /** - * The stage of the SuppressionListResource definition allowing to specify parent resource. - */ - interface WithParentResource { - /** - * Specifies resourceGroupName, emailServiceName, domainName. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param emailServiceName The name of the EmailService resource. - * @param domainName The name of the Domains resource. - * @return the next definition stage. - */ - WithCreate withExistingDomain(String resourceGroupName, String emailServiceName, String domainName); - } - - /** - * The stage of the SuppressionListResource definition which contains all the minimum required properties for - * the resource to be created, but also allows for any other optional properties to be specified. - */ - interface WithCreate extends DefinitionStages.WithListName { - /** - * Executes the create request. - * - * @return the created resource. - */ - SuppressionListResource create(); - - /** - * Executes the create request. - * - * @param context The context to associate with this operation. - * @return the created resource. - */ - SuppressionListResource create(Context context); - } - - /** - * The stage of the SuppressionListResource definition allowing to specify listName. - */ - interface WithListName { - /** - * Specifies the listName property: The the name of the suppression list. This value must match one of the - * valid sender usernames of the sending domain.. - * - * @param listName The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * @return the next definition stage. - */ - WithCreate withListName(String listName); - } - } - - /** - * Begins update for the SuppressionListResource resource. - * - * @return the stage of resource update. - */ - SuppressionListResource.Update update(); - - /** - * The template for SuppressionListResource update. - */ - interface Update extends UpdateStages.WithListName { - /** - * Executes the update request. - * - * @return the updated resource. - */ - SuppressionListResource apply(); - - /** - * Executes the update request. - * - * @param context The context to associate with this operation. - * @return the updated resource. - */ - SuppressionListResource apply(Context context); - } - - /** - * The SuppressionListResource update stages. - */ - interface UpdateStages { - /** - * The stage of the SuppressionListResource update allowing to specify listName. - */ - interface WithListName { - /** - * Specifies the listName property: The the name of the suppression list. This value must match one of the - * valid sender usernames of the sending domain.. - * - * @param listName The the name of the suppression list. This value must match one of the valid sender - * usernames of the sending domain. - * @return the next definition stage. - */ - Update withListName(String listName); - } - } - - /** - * Refreshes the resource to sync with Azure. - * - * @return the refreshed resource. - */ - SuppressionListResource refresh(); - - /** - * Refreshes the resource to sync with Azure. - * - * @param context The context to associate with this operation. - * @return the refreshed resource. - */ - SuppressionListResource refresh(Context context); -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResourceCollection.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResourceCollection.java deleted file mode 100644 index 4d53a64e2ee4c..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/com/azure/resourcemanager/communication/models/SuppressionListResourceCollection.java +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.models; - -import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** - * A class representing a Domains SuppressionListResource collection. - */ -@Fluent -public final class SuppressionListResourceCollection { - /* - * List of SuppressionListResource - */ - @JsonProperty(value = "value") - private List value; - - /* - * The URL the client should use to fetch the next page (per server side paging). - */ - @JsonProperty(value = "nextLink") - private String nextLink; - - /** - * Creates an instance of SuppressionListResourceCollection class. - */ - public SuppressionListResourceCollection() { - } - - /** - * Get the value property: List of SuppressionListResource. - * - * @return the value value. - */ - public List value() { - return this.value; - } - - /** - * Set the value property: List of SuppressionListResource. - * - * @param value the value value to set. - * @return the SuppressionListResourceCollection object itself. - */ - public SuppressionListResourceCollection withValue(List value) { - this.value = value; - return this; - } - - /** - * Get the nextLink property: The URL the client should use to fetch the next page (per server side paging). - * - * @return the nextLink value. - */ - public String nextLink() { - return this.nextLink; - } - - /** - * Set the nextLink property: The URL the client should use to fetch the next page (per server side paging). - * - * @param nextLink the nextLink value to set. - * @return the SuppressionListResourceCollection object itself. - */ - public SuppressionListResourceCollection withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (value() != null) { - value().forEach(e -> e.validate()); - } - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/java/module-info.java b/sdk/communication/azure-resourcemanager-communication/src/main/java/module-info.java index 742f4c0af53a3..1a22b6345135e 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/java/module-info.java +++ b/sdk/communication/azure-resourcemanager-communication/src/main/java/module-info.java @@ -4,12 +4,10 @@ module com.azure.resourcemanager.communication { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.communication; exports com.azure.resourcemanager.communication.fluent; exports com.azure.resourcemanager.communication.fluent.models; exports com.azure.resourcemanager.communication.models; - opens com.azure.resourcemanager.communication.fluent.models to com.azure.core, com.fasterxml.jackson.databind; opens com.azure.resourcemanager.communication.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/proxy-config.json b/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/proxy-config.json index ad2ec93356c5e..066a7deb03188 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/proxy-config.json +++ b/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/proxy-config.json @@ -1 +1 @@ -[ [ "com.azure.resourcemanager.communication.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.communication.implementation.CommunicationServicesClientImpl$CommunicationServicesService" ], [ "com.azure.resourcemanager.communication.implementation.DomainsClientImpl$DomainsService" ], [ "com.azure.resourcemanager.communication.implementation.EmailServicesClientImpl$EmailServicesService" ], [ "com.azure.resourcemanager.communication.implementation.SenderUsernamesClientImpl$SenderUsernamesService" ], [ "com.azure.resourcemanager.communication.implementation.SuppressionListsClientImpl$SuppressionListsService" ], [ "com.azure.resourcemanager.communication.implementation.SuppressionListAddressesClientImpl$SuppressionListAddressesService" ] ] \ No newline at end of file +[ [ "com.azure.resourcemanager.communication.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.communication.implementation.CommunicationServicesClientImpl$CommunicationServicesService" ], [ "com.azure.resourcemanager.communication.implementation.DomainsClientImpl$DomainsService" ], [ "com.azure.resourcemanager.communication.implementation.EmailServicesClientImpl$EmailServicesService" ], [ "com.azure.resourcemanager.communication.implementation.SenderUsernamesClientImpl$SenderUsernamesService" ] ] \ No newline at end of file diff --git a/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/reflect-config.json b/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/reflect-config.json index ed4dd8c577598..13883cefcfb4e 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/reflect-config.json +++ b/sdk/communication/azure-resourcemanager-communication/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-communication/reflect-config.json @@ -173,36 +173,6 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.communication.models.SuppressionListResourceCollection", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.communication.fluent.models.SuppressionListProperties", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.communication.models.SuppressionListAddressResourceCollection", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressProperties", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.communication.models.Origin", "allDeclaredConstructors" : true, diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilitySamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilitySamples.java index d54724fd2359f..7b5b100a733d2 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilitySamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilitySamples.java @@ -12,7 +12,7 @@ public final class CommunicationServicesCheckNameAvailabilitySamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/checkNameAvailabilityAvailable.json */ /** @@ -29,7 +29,7 @@ public final class CommunicationServicesCheckNameAvailabilitySamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/checkNameAvailabilityUnavailable.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateSamples.java index a02af9e01f85e..c0ef8e0533ea1 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateSamples.java @@ -13,7 +13,7 @@ public final class CommunicationServicesCreateOrUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/createOrUpdate.json */ /** @@ -28,7 +28,7 @@ public static void createOrUpdateResource(com.azure.resourcemanager.communicatio /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/createOrUpdateWithSystemAssignedIdentity.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteSamples.java index db6ec059486be..51f7e71f26f92 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteSamples.java @@ -10,7 +10,7 @@ public final class CommunicationServicesDeleteSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/delete.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupSamples.java index 3884c8f5867be..ddb5c746d0843 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class CommunicationServicesGetByResourceGroupSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/get.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubSamples.java index 1300edb3e443c..1a344e25ec799 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubSamples.java @@ -12,7 +12,7 @@ public final class CommunicationServicesLinkNotificationHubSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/linkNotificationHub.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupSamples.java index e8a216aeaec00..341fcfbc849c6 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class CommunicationServicesListByResourceGroupSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/listByResourceGroup.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListKeysSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListKeysSamples.java index 8b9b24691e83d..5be88879ee4bd 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListKeysSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListKeysSamples.java @@ -10,7 +10,7 @@ public final class CommunicationServicesListKeysSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/listKeys.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListSamples.java index 0a1555164427f..bd0f4147bed32 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListSamples.java @@ -10,7 +10,7 @@ public final class CommunicationServicesListSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/listBySubscription.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesRegenerateKeySamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesRegenerateKeySamples.java index e9f08b5530c62..fe9a09952eaa0 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesRegenerateKeySamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesRegenerateKeySamples.java @@ -13,7 +13,7 @@ public final class CommunicationServicesRegenerateKeySamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/regenerateKey.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesUpdateSamples.java index 2f2cb03e74bb6..d7bedd5b90e89 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/CommunicationServicesUpdateSamples.java @@ -17,7 +17,7 @@ public final class CommunicationServicesUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/update.json */ /** @@ -29,13 +29,12 @@ public static void updateResource(com.azure.resourcemanager.communication.Commun CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("newTag", "newVal")) - .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED)).apply(); + resource.update().withTags(mapOf("newTag", "newVal")).apply(); } /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/updateWithUserAssignedIdentity.json */ /** @@ -56,7 +55,7 @@ public static void updateResourceToAddAUserAssignedManagedIdentity( /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/updateWithSystemAssignedIdentity.json */ /** @@ -75,7 +74,7 @@ public static void updateResourceToAddASystemAssignedManagedIdentity( /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/updateRemoveSystemIdentity.json */ /** @@ -88,12 +87,13 @@ public static void updateResourceToAddASystemAssignedManagedIdentity( CommunicationServiceResource resource = manager.communicationServices().getByResourceGroupWithResponse("MyResourceGroup", "MyCommunicationResource", com.azure.core.util.Context.NONE).getValue(); - resource.update().withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE)).apply(); + resource.update().withTags(mapOf("newTag", "newVal")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE)).apply(); } /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/updateWithSystemAndUserIdentity.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCancelVerificationSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCancelVerificationSamples.java index 9f6133a52b278..5174fb25105bf 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCancelVerificationSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCancelVerificationSamples.java @@ -13,7 +13,7 @@ public final class DomainsCancelVerificationSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ * cancelVerification.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCreateOrUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCreateOrUpdateSamples.java index f19f411171796..4ed0aa6e3bf92 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCreateOrUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsCreateOrUpdateSamples.java @@ -12,7 +12,7 @@ public final class DomainsCreateOrUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ * createOrUpdate.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsDeleteSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsDeleteSamples.java index 21b285508e546..e24f594cc2c51 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsDeleteSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsDeleteSamples.java @@ -10,8 +10,8 @@ public final class DomainsDeleteSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ - * delete.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/delete. + * json */ /** * Sample code: Delete Domains resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsGetSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsGetSamples.java index 1efcf0811bbb4..0cdc1b87862c5 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsGetSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsGetSamples.java @@ -10,8 +10,7 @@ public final class DomainsGetSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ - * get.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/get.json */ /** * Sample code: Get Domains resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsInitiateVerificationSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsInitiateVerificationSamples.java index 88f7b5f42a9c4..88c3b133e4795 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsInitiateVerificationSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsInitiateVerificationSamples.java @@ -13,7 +13,7 @@ public final class DomainsInitiateVerificationSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ * initiateVerification.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsListByEmailServiceResourceSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsListByEmailServiceResourceSamples.java index dd2d048387531..f13500d4c989e 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsListByEmailServiceResourceSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsListByEmailServiceResourceSamples.java @@ -10,7 +10,7 @@ public final class DomainsListByEmailServiceResourceSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/ * listByEmailService.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsUpdateSamples.java index eeeea15474df6..ae5a64899dcdf 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/DomainsUpdateSamples.java @@ -13,8 +13,8 @@ public final class DomainsUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/domains/ - * update.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/domains/update. + * json */ /** * Sample code: Update Domains resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateSamples.java index c4d1f6742264f..61f8ceba24424 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateSamples.java @@ -10,8 +10,8 @@ public final class EmailServicesCreateOrUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/createOrUpdate.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * createOrUpdate.json */ /** * Sample code: Create or update EmailService resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteSamples.java index 602adbc1af537..f2f87f8196bf8 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteSamples.java @@ -10,8 +10,8 @@ public final class EmailServicesDeleteSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/delete.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * delete.json */ /** * Sample code: Delete EmailService resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupSamples.java index e57d7b4382898..7f5f09eaefc0f 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupSamples.java @@ -10,8 +10,8 @@ public final class EmailServicesGetByResourceGroupSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/get.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/get + * .json */ /** * Sample code: Get EmailService resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupSamples.java index c25238a605dd4..71e848533b301 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupSamples.java @@ -10,8 +10,8 @@ public final class EmailServicesListByResourceGroupSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/listByResourceGroup.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * listByResourceGroup.json */ /** * Sample code: List EmailService resources by resource group. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListSamples.java index 97083f6bbff25..6c08aae8b9bf1 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListSamples.java @@ -10,8 +10,8 @@ public final class EmailServicesListSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/listBySubscription.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * listBySubscription.json */ /** * Sample code: List EmailService resources by subscription. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsSamples.java index 9e6daaa3471fc..577b34d64a73f 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsSamples.java @@ -10,8 +10,8 @@ public final class EmailServicesListVerifiedExchangeOnlineDomainsSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/getVerifiedExchangeOnlineDomains.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * getVerifiedExchangeOnlineDomains.json */ /** * Sample code: Get verified Exchange Online domains. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesUpdateSamples.java index 9d29d469a5a1b..6234c0f31ba8a 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/EmailServicesUpdateSamples.java @@ -14,8 +14,8 @@ public final class EmailServicesUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * emailServices/update.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/emailServices/ + * update.json */ /** * Sample code: Update EmailService resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/OperationsListSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/OperationsListSamples.java index 584f921d1d06e..4585b92af1729 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/OperationsListSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/OperationsListSamples.java @@ -10,7 +10,7 @@ public final class OperationsListSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/ * communicationServices/operationsList.json */ /** diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateSamples.java index 42bbf7c995f17..39a1a9e47691d 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateSamples.java @@ -10,8 +10,8 @@ public final class SenderUsernamesCreateOrUpdateSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * senderUsernames/createOrUpdate.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * createOrUpdate.json */ /** * Sample code: Create or update SenderUsernames resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteSamples.java index 6329d8aca9656..dafdb1813a6f0 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteSamples.java @@ -10,8 +10,8 @@ public final class SenderUsernamesDeleteSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * senderUsernames/delete.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * delete.json */ /** * Sample code: Delete SenderUsernames resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetSamples.java index 971245520e655..640700a845d2b 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetSamples.java @@ -10,8 +10,8 @@ public final class SenderUsernamesGetSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * senderUsernames/get.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * get.json */ /** * Sample code: Get SenderUsernames resource. diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsSamples.java index 0f3e21f7f8d94..2b5e206b125f8 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsSamples.java +++ b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsSamples.java @@ -10,16 +10,16 @@ public final class SenderUsernamesListByDomainsSamples { /* * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * senderUsernames/listByDomain.json + * specification/communication/resource-manager/Microsoft.Communication/stable/2023-04-01/examples/senderUsernames/ + * listByDomain.json */ /** - * Sample code: Get all SenderUsernames resources for a Domain. + * Sample code: Get SenderUsernames resource. * * @param manager Entry point to CommunicationManager. */ public static void - getAllSenderUsernamesResourcesForADomain(com.azure.resourcemanager.communication.CommunicationManager manager) { + getSenderUsernamesResource(com.azure.resourcemanager.communication.CommunicationManager manager) { manager.senderUsernames().listByDomains("contosoResourceGroup", "contosoEmailService", "contoso.com", com.azure.core.util.Context.NONE); } diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateSamples.java deleted file mode 100644 index 3c80491ed8756..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateSamples.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionListAddresses CreateOrUpdate. - */ -public final class SuppressionListAddressesCreateOrUpdateSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/createOrUpdateAddress.json - */ - /** - * Sample code: CreateOrUpdate SuppressionListAddress resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void createOrUpdateSuppressionListAddressResource( - com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().define("11112222-3333-4444-5555-aaaabbbbcccc") - .withExistingSuppressionList("contosoResourceGroup", "contosoEmailService", "contoso.com", - "aaaa1111-bbbb-2222-3333-aaaa11112222") - .withEmail("newuser1@fabrikam.com").withFirstName("updatedFirstName").create(); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteSamples.java deleted file mode 100644 index 866a99a1b2f20..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteSamples.java +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionListAddresses Delete. - */ -public final class SuppressionListAddressesDeleteSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/deleteAddress.json - */ - /** - * Sample code: Delete a SuppressionListAddress resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void - deleteASuppressionListAddressResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().deleteWithResponse("MyResourceGroup", "MyEmailServiceResource", - "mydomain.com", "aaaa1111-bbbb-2222-3333-aaaa11112222", "11112222-3333-4444-5555-999999999999", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesGetSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesGetSamples.java deleted file mode 100644 index 373c99924bec9..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesGetSamples.java +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionListAddresses Get. - */ -public final class SuppressionListAddressesGetSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/getAddress.json - */ - /** - * Sample code: Get a SuppressionListAddress resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void - getASuppressionListAddressResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().getWithResponse("contosoResourceGroup", "contosoEmailService", "contoso.com", - "aaaa1111-bbbb-2222-3333-aaaa11112222", "11112222-3333-4444-5555-aaaabbbbcccc", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesListSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesListSamples.java deleted file mode 100644 index 32fb10618f993..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesListSamples.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionListAddresses List. - */ -public final class SuppressionListAddressesListSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/getAddresses.json - */ - /** - * Sample code: Get all SuppressionListAddresses resources for a SuppressionList resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void getAllSuppressionListAddressesResourcesForASuppressionListResource( - com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionListAddresses().list("contosoResourceGroup", "contosoEmailService", "contoso.com", - "aaaa1111-bbbb-2222-3333-aaaa11112222", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsCreateOrUpdateSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsCreateOrUpdateSamples.java deleted file mode 100644 index cec7541cbd0c3..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsCreateOrUpdateSamples.java +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionLists CreateOrUpdate. - */ -public final class SuppressionListsCreateOrUpdateSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/createOrUpdateSuppressionList.json - */ - /** - * Sample code: CreateOrUpdate SuppressionLists resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void - createOrUpdateSuppressionListsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().define("aaaa1111-bbbb-2222-3333-aaaa11112222") - .withExistingDomain("contosoResourceGroup", "contosoEmailService", "contoso.com") - .withListName("contosoNewsAlerts").create(); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteSamples.java deleted file mode 100644 index e07609776f31b..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsDeleteSamples.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionLists Delete. - */ -public final class SuppressionListsDeleteSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/deleteSuppressionList.json - */ - /** - * Sample code: Delete a SuppressionLists resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void - deleteASuppressionListsResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().deleteWithResponse("MyResourceGroup", "MyEmailServiceResource", "mydomain.com", - "aaaa1111-bbbb-2222-3333-aaaa11112222", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetSamples.java deleted file mode 100644 index 28a2edb3079a8..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetSamples.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionLists Get. - */ -public final class SuppressionListsGetSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/getSuppressionList.json - */ - /** - * Sample code: Get a SuppressionList resource. - * - * @param manager Entry point to CommunicationManager. - */ - public static void - getASuppressionListResource(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().getWithResponse("contosoResourceGroup", "contosoEmailService", "contoso.com", - "aaaa1111-bbbb-2222-3333-aaaa11112222", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainSamples.java b/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainSamples.java deleted file mode 100644 index 313c2d8a40aba..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/samples/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainSamples.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -/** - * Samples for SuppressionLists ListByDomain. - */ -public final class SuppressionListsListByDomainSamples { - /* - * x-ms-original-file: - * specification/communication/resource-manager/Microsoft.Communication/preview/2023-06-01-preview/examples/ - * suppressionLists/getSuppressionLists.json - */ - /** - * Sample code: Get all SuppressionLists resources. - * - * @param manager Entry point to CommunicationManager. - */ - public static void - getAllSuppressionListsResources(com.azure.resourcemanager.communication.CommunicationManager manager) { - manager.suppressionLists().listByDomain("contosoResourceGroup", "contosoEmailService", "contoso.com", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceInnerTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceInnerTests.java index 26ea6a96e533d..97a08b2778288 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceInnerTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceInnerTests.java @@ -18,7 +18,7 @@ public final class CommunicationServiceResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CommunicationServiceResourceInner model = BinaryData.fromString( - "{\"properties\":{\"provisioningState\":\"Updating\",\"hostName\":\"ljfmppee\",\"dataLocation\":\"vmgxsab\",\"notificationHubId\":\"qduujitcjczdz\",\"version\":\"ndhkrw\",\"immutableResourceId\":\"appd\",\"linkedDomains\":[\"kvwrwjfeu\",\"nhutjeltmrldhugj\",\"zdatqxhocdg\"]},\"identity\":{\"principalId\":\"3364eec9-ade1-40e9-a6b6-c1ebdbb90729\",\"tenantId\":\"4d6900be-a481-4fe4-8f6c-0f9ac88c28f4\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"icndvkaozwyifty\":{\"principalId\":\"3e6b52c5-3b05-4ff7-8a7a-68fa15023315\",\"clientId\":\"c61c00e7-4756-49b4-b899-415d17db5edf\"},\"urokft\":{\"principalId\":\"e977beae-b8a5-426d-9b27-894420e6ba02\",\"clientId\":\"5e81e17e-fe2c-4cf3-822a-3776dc7075d2\"},\"lniwpwcukjfkgiaw\":{\"principalId\":\"b0400f16-b86f-47bf-be2c-2902a6ecefc9\",\"clientId\":\"5c4ab0f5-66b3-4383-bdd3-221551430340\"}}},\"location\":\"lryplwckbasyy\",\"tags\":{\"phejkotynqgoulz\":\"dhsgcba\",\"gakeqsr\":\"dlikwyqkgfgibma\",\"qqedqytbciqfou\":\"yb\"},\"id\":\"lmmnkzsmodmglo\",\"name\":\"gpbkwtmut\",\"type\":\"uqktap\"}") + "{\"properties\":{\"provisioningState\":\"Updating\",\"hostName\":\"ljfmppee\",\"dataLocation\":\"vmgxsab\",\"notificationHubId\":\"qduujitcjczdz\",\"version\":\"ndhkrw\",\"immutableResourceId\":\"appd\",\"linkedDomains\":[\"kvwrwjfeu\",\"nhutjeltmrldhugj\",\"zdatqxhocdg\"]},\"identity\":{\"principalId\":\"3be39144-f5db-4d95-a423-901fe2c71743\",\"tenantId\":\"5eed8049-2baf-4291-8ae1-a28d5ca55c0f\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"icndvkaozwyifty\":{\"principalId\":\"2dba58c6-e1b7-48a0-8457-6f9eda4da645\",\"clientId\":\"39146c26-b00c-4f50-8261-625c41383ead\"},\"urokft\":{\"principalId\":\"2affca77-44bb-412b-923b-629c333f41b1\",\"clientId\":\"d9e394ab-bcf6-427f-8a26-28182ed95220\"},\"lniwpwcukjfkgiaw\":{\"principalId\":\"8680008c-30a6-4d01-8316-a3a6716c164d\",\"clientId\":\"19d70193-a0f9-41f3-ba3a-40df75ed6670\"}}},\"location\":\"lryplwckbasyy\",\"tags\":{\"phejkotynqgoulz\":\"dhsgcba\",\"gakeqsr\":\"dlikwyqkgfgibma\",\"qqedqytbciqfou\":\"yb\"},\"id\":\"lmmnkzsmodmglo\",\"name\":\"gpbkwtmut\",\"type\":\"uqktap\"}") .toObject(CommunicationServiceResourceInner.class); Assertions.assertEquals("lryplwckbasyy", model.location()); Assertions.assertEquals("dhsgcba", model.tags().get("phejkotynqgoulz")); diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceListTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceListTests.java index b89d186effcfa..510c778a19b97 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceListTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceListTests.java @@ -19,7 +19,7 @@ public final class CommunicationServiceResourceListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CommunicationServiceResourceList model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"mkkvnip\",\"dataLocation\":\"oxzjnchgejspod\",\"notificationHubId\":\"ilzyd\",\"version\":\"o\",\"immutableResourceId\":\"yahux\",\"linkedDomains\":[\"mqnjaqw\",\"xj\"]},\"identity\":{\"principalId\":\"f23243a9-97a8-4db7-9630-79edf0f5ec5b\",\"tenantId\":\"8bda4b90-13fd-4b49-967f-939e714660f3\",\"type\":\"SystemAssigned,UserAssigned\",\"userAssignedIdentities\":{\"tegjvwmf\":{\"principalId\":\"918e4f18-5c32-48e7-b6fd-ffac2b2256f4\",\"clientId\":\"05fae119-b77b-4d17-8814-96e439c2fb82\"},\"scmdvpjhulsuu\":{\"principalId\":\"3a087802-ad98-4d9d-9f8f-6217eba9e6ce\",\"clientId\":\"a378160e-3ffa-4c66-88a3-403fabb2932b\"},\"jozkrwfndiod\":{\"principalId\":\"5e1a04a4-5740-460c-8ed5-423b2500ba73\",\"clientId\":\"b4b0ea7a-f230-4708-8133-960a22271c73\"},\"lwejdpv\":{\"principalId\":\"9ec2ade8-bd9d-4015-b062-94498fb8d4cb\",\"clientId\":\"f49d609a-4d01-4c6e-9317-3466b417bd76\"}}},\"location\":\"yoqpsoaccta\",\"tags\":{\"dfdosygexp\":\"ljlahbcryf\",\"dphlxaolt\":\"ojakhmsbzjhcrze\"},\"id\":\"qtrgqjbpfzfsinzg\",\"name\":\"f\",\"type\":\"jrwzox\"},{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"lluwfzitonpeq\",\"dataLocation\":\"pjkjlxofpdv\",\"notificationHubId\":\"fxxypininmayhuy\",\"version\":\"kpode\",\"immutableResourceId\":\"oginuvamiheognar\",\"linkedDomains\":[\"theotusiv\"]},\"identity\":{\"principalId\":\"ca5f15e1-4a32-4f3d-b695-af9387168a20\",\"tenantId\":\"e9113944-d7cd-4fa8-b7ea-9324f26b9eb5\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"nhungbw\":{\"principalId\":\"ae3f5fa4-ccd9-4efe-b9fb-5a87f0690c7e\",\"clientId\":\"a4717b06-5633-4bac-aa21-86fac88b45fc\"}}},\"location\":\"rnfygxgispem\",\"tags\":{\"fxqeof\":\"fkufublj\",\"jqul\":\"aeqjhqjbasvms\",\"clxxwrljdo\":\"gsntnbybkzgcwr\"},\"id\":\"skcqvkocrcjd\",\"name\":\"wtnhxbnjbiksqr\",\"type\":\"lssai\"}],\"nextLink\":\"p\"}") + "{\"value\":[{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"mkkvnip\",\"dataLocation\":\"oxzjnchgejspod\",\"notificationHubId\":\"ilzyd\",\"version\":\"o\",\"immutableResourceId\":\"yahux\",\"linkedDomains\":[\"mqnjaqw\",\"xj\"]},\"identity\":{\"principalId\":\"ef9eb9da-900a-4144-ad27-a9b0a76e515d\",\"tenantId\":\"ee78ee04-5488-49a6-b105-eaa8a59fd4b0\",\"type\":\"SystemAssigned,UserAssigned\",\"userAssignedIdentities\":{\"tegjvwmf\":{\"principalId\":\"86be38b7-b6b7-4b42-bd48-8be5806672da\",\"clientId\":\"b7b48f5e-f095-4c96-96a6-554206c17dd1\"},\"scmdvpjhulsuu\":{\"principalId\":\"b9807eea-7284-4e49-9b53-ca66d768d5cb\",\"clientId\":\"bdce0daa-ecdc-4e3a-8ffe-97c4b145c93a\"},\"jozkrwfndiod\":{\"principalId\":\"fd1ce9f2-ee54-4212-abbc-317e0597ecaf\",\"clientId\":\"82f841a7-6458-4ad9-9b7f-5ca541098843\"},\"lwejdpv\":{\"principalId\":\"09c23d8b-6df2-4ef9-a643-2e400db91f86\",\"clientId\":\"09585243-6cce-4b77-a7c8-021c95eb8e16\"}}},\"location\":\"yoqpsoaccta\",\"tags\":{\"dfdosygexp\":\"ljlahbcryf\",\"dphlxaolt\":\"ojakhmsbzjhcrze\"},\"id\":\"qtrgqjbpfzfsinzg\",\"name\":\"f\",\"type\":\"jrwzox\"},{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"lluwfzitonpeq\",\"dataLocation\":\"pjkjlxofpdv\",\"notificationHubId\":\"fxxypininmayhuy\",\"version\":\"kpode\",\"immutableResourceId\":\"oginuvamiheognar\",\"linkedDomains\":[\"theotusiv\"]},\"identity\":{\"principalId\":\"b384ea4b-2bc3-4f19-9d4a-c97ac5dfcda2\",\"tenantId\":\"3d8dba7a-3e4b-44cd-9473-650ba5b0d1cb\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"nhungbw\":{\"principalId\":\"ecf55d34-1c17-4213-8a5b-b05dcccdc962\",\"clientId\":\"b6ed71ad-2d97-4eec-b344-85f512a698be\"}}},\"location\":\"rnfygxgispem\",\"tags\":{\"fxqeof\":\"fkufublj\",\"jqul\":\"aeqjhqjbasvms\",\"clxxwrljdo\":\"gsntnbybkzgcwr\"},\"id\":\"skcqvkocrcjd\",\"name\":\"wtnhxbnjbiksqr\",\"type\":\"lssai\"}],\"nextLink\":\"p\"}") .toObject(CommunicationServiceResourceList.class); Assertions.assertEquals("yoqpsoaccta", model.value().get(0).location()); Assertions.assertEquals("ljlahbcryf", model.value().get(0).tags().get("dfdosygexp")); diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceUpdateTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceUpdateTests.java index 1fc4a26486b36..8eb78e0926395 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceUpdateTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServiceResourceUpdateTests.java @@ -18,7 +18,7 @@ public final class CommunicationServiceResourceUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CommunicationServiceResourceUpdate model = BinaryData.fromString( - "{\"properties\":{\"linkedDomains\":[\"jvtbvpyss\",\"dnrujqguhmuouqfp\"]},\"identity\":{\"principalId\":\"51901feb-bc28-41a5-a7c6-f4eebc3d2566\",\"tenantId\":\"e97f6445-65e2-45aa-a746-a642218ffb13\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"tnwu\":{\"principalId\":\"2b601cee-d688-4cc6-8cfc-aaa8f3e38af9\",\"clientId\":\"81a43ba3-be08-4adb-a815-cabb740eed27\"}}},\"tags\":{\"x\":\"a\",\"hr\":\"fizuckyf\"}}") + "{\"properties\":{\"linkedDomains\":[\"jvtbvpyss\",\"dnrujqguhmuouqfp\"]},\"identity\":{\"principalId\":\"6211b490-391f-4b88-a622-b29fa0524dee\",\"tenantId\":\"49124dfc-70a5-407e-941f-67e66e775255\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"tnwu\":{\"principalId\":\"d34810c6-782e-42c1-9285-10f9e3568d49\",\"clientId\":\"5d70e04d-9075-4b18-8d87-b46281661276\"}}},\"tags\":{\"x\":\"a\",\"hr\":\"fizuckyf\"}}") .toObject(CommunicationServiceResourceUpdate.class); Assertions.assertEquals("a", model.tags().get("x")); Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, model.identity().type()); diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilityWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilityWithResponseMockTests.java index d5f349f8d14a2..99d2b86cd9aca 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCheckNameAvailabilityWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"nameAvailable\":true,\"reason\":\"AlreadyExists\",\"message\":\"lxotogtwrupq\"}"; + String responseStr = "{\"nameAvailable\":false,\"reason\":\"Invalid\",\"message\":\"eyp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,12 +51,12 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { CheckNameAvailabilityResponse response = manager.communicationServices() .checkNameAvailabilityWithResponse( - new NameAvailabilityParameters().withName("rdqmhjjdhtldwkyz").withType("utknc"), + new NameAvailabilityParameters().withName("xzko").withType("cukoklyaxuconu"), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals(true, response.nameAvailable()); - Assertions.assertEquals(CheckNameAvailabilityReason.ALREADY_EXISTS, response.reason()); - Assertions.assertEquals("lxotogtwrupq", response.message()); + Assertions.assertEquals(false, response.nameAvailable()); + Assertions.assertEquals(CheckNameAvailabilityReason.INVALID, response.reason()); + Assertions.assertEquals("eyp", response.message()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateMockTests.java index 0710fe2fa3349..bceb5521724ba 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesCreateOrUpdateMockTests.java @@ -37,7 +37,7 @@ public void testCreateOrUpdate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"hostName\":\"vwvxyslqbhsfx\",\"dataLocation\":\"blytk\",\"notificationHubId\":\"mpew\",\"version\":\"fbkrvrnsvs\",\"immutableResourceId\":\"johxcrsb\",\"linkedDomains\":[\"asrru\",\"wbhsqfsub\",\"gjb\"]},\"identity\":{\"principalId\":\"974e5b1a-902e-4a38-b9c0-f30b360899b1\",\"tenantId\":\"dea5e349-e51c-42d7-be6a-f3f0da212032\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"fbjfdtwssotftpvj\":{\"principalId\":\"3d583232-f6aa-4850-84e7-5f64daa54956\",\"clientId\":\"53e4bbc9-815a-4ac8-b954-cac4e7201cdb\"},\"xilzznf\":{\"principalId\":\"eac58f4a-2e55-476f-b158-d385f59cd9c2\",\"clientId\":\"5557c7dc-b550-4e11-87d7-2a0f7c9c3b4d\"}}},\"location\":\"nvwpmqtaruouj\",\"tags\":{\"ewgdrjervn\":\"jhwqytjrybnw\",\"eh\":\"enq\"},\"id\":\"ndoygmifthnzdnd\",\"name\":\"l\",\"type\":\"nayqi\"}"; + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"hostName\":\"sfwxosowzxc\",\"dataLocation\":\"gicjooxdjeb\",\"notificationHubId\":\"ucww\",\"version\":\"ovbvmeueciv\",\"immutableResourceId\":\"zceuojgjrw\",\"linkedDomains\":[\"iotwmcdytdxwit\",\"nrjawgqwg\",\"hniskxfbkpyc\"]},\"identity\":{\"principalId\":\"81106beb-092d-40db-ba2b-4f7e526ae0c0\",\"tenantId\":\"80113ea7-1002-4294-a060-b9b71f05085d\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"dauwhvylwzbtd\":{\"principalId\":\"fc83ff1f-459a-446d-be3a-41088cf5e029\",\"clientId\":\"a6ee0e13-1149-444f-8ca1-eafb2a27af50\"},\"jznb\":{\"principalId\":\"6212c5f4-7e5f-47cb-8702-39f5f15b8259\",\"clientId\":\"42a647a1-eccd-4289-9371-48ded81c8b76\"}}},\"location\":\"ow\",\"tags\":{\"lupj\":\"rzqlveu\",\"riplrbpbewtg\":\"khfxobbcswsrt\"},\"id\":\"fgb\",\"name\":\"c\",\"type\":\"wxzvlvqhjkb\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -54,20 +54,19 @@ public void testCreateOrUpdate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - CommunicationServiceResource response = manager.communicationServices().define("lhbnxkna") - .withRegion("uaopppcqeq").withExistingResourceGroup("ixhnrztf") - .withTags(mapOf("ahzxctobgbk", "z", "mgrcfbu", "moizpos", "mjh", "rmfqjhhkxbpvj", "tswb", "xjyngudivk")) - .withIdentity( - new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) - .withUserAssignedIdentities(mapOf("kteo", new UserAssignedIdentity(), "wptfdy", - new UserAssignedIdentity(), "qbuaceopzfqr", new UserAssignedIdentity()))) - .withDataLocation("gdtpnapnyiro").withLinkedDomains(Arrays.asList("xgk")).create(); + CommunicationServiceResource response + = manager.communicationServices().define("kfrlhrxsbky").withRegion("hcdhmdual") + .withExistingResourceGroup("bh").withTags(mapOf("adm", "qpv", "r", "sr", "fmisg", "vxpvgomz")) + .withIdentity( + new ManagedServiceIdentity().withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("ahvljuaha", new UserAssignedIdentity()))) + .withDataLocation("z").withLinkedDomains(Arrays.asList("eyvjusrtslhspkde", "maofmxagkv")).create(); - Assertions.assertEquals("nvwpmqtaruouj", response.location()); - Assertions.assertEquals("jhwqytjrybnw", response.tags().get("ewgdrjervn")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.identity().type()); - Assertions.assertEquals("blytk", response.dataLocation()); - Assertions.assertEquals("asrru", response.linkedDomains().get(0)); + Assertions.assertEquals("ow", response.location()); + Assertions.assertEquals("rzqlveu", response.tags().get("lupj")); + Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); + Assertions.assertEquals("gicjooxdjeb", response.dataLocation()); + Assertions.assertEquals("iotwmcdytdxwit", response.linkedDomains().get(0)); } // Use "Map.of" if available diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteMockTests.java index 7f7ba762c1f00..9e29fe4973ccd 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.communicationServices().delete("oekqvk", "lns", com.azure.core.util.Context.NONE); + manager.communicationServices().delete("uscrpabgyepsb", "tazqugxywpmueefj", com.azure.core.util.Context.NONE); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupWithResponseMockTests.java index 7ff7a41783f5e..4ca0737f4029b 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesGetByResourceGroupWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetByResourceGroupWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Unknown\",\"hostName\":\"zvlvqhjkbegib\",\"dataLocation\":\"nmxiebwwaloayqc\",\"notificationHubId\":\"rtzju\",\"version\":\"wyzmhtxon\",\"immutableResourceId\":\"ts\",\"linkedDomains\":[\"cbpwxqpsrknft\",\"uvriuhprwm\",\"yvxqtayriwwroy\",\"bexrmcq\"]},\"identity\":{\"principalId\":\"0d7ca64b-8a60-40d3-b8a4-0fb239e8b065\",\"tenantId\":\"704faab2-1e7c-4462-83e3-6c11b6771823\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"nmefqsgzvahapj\":{\"principalId\":\"720a123b-41c3-43db-a217-e374a01ddf7b\",\"clientId\":\"db121949-3483-4c70-87bf-9e3eec546ade\"}}},\"location\":\"hpvgqz\",\"tags\":{\"zovawjvz\":\"vxdjzlmwlxkvugf\",\"nxipeil\":\"nluthnnp\",\"dzumveekg\":\"jzuaejxdultskzbb\"},\"id\":\"wozuhkf\",\"name\":\"bsjyofdx\",\"type\":\"uusdttouwa\"}"; + = "{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"vmkfssxqu\",\"dataLocation\":\"kfplgmgsxnk\",\"notificationHubId\":\"kde\",\"version\":\"pvlopwiyighxpkd\",\"immutableResourceId\":\"baiuebbaumny\",\"linkedDomains\":[\"edeojnabc\"]},\"identity\":{\"principalId\":\"90fdda1a-2423-45b5-b479-d2875d87c96a\",\"tenantId\":\"90b43a81-6830-43c7-95fa-530d23bbd8e2\",\"type\":\"None\",\"userAssignedIdentities\":{\"ebtfhvpesap\":{\"principalId\":\"36b69f17-8df3-4d27-87c9-5b344e36ea49\",\"clientId\":\"a6beacf0-a8b3-42d5-b5e2-90c549cec587\"},\"dqmh\":{\"principalId\":\"639a2178-e33d-4939-8fa0-52a5a9c226aa\",\"clientId\":\"1944f363-64fc-450c-8df0-f000f6bca8df\"},\"htldwk\":{\"principalId\":\"94579b49-d73c-4fef-b4b8-87e7d6301a16\",\"clientId\":\"9af4da14-ac33-4e0a-877a-d9a295dc0d35\"}}},\"location\":\"xuutkncwscwsv\",\"tags\":{\"rupqsxvnmicy\":\"togt\",\"vei\":\"vce\",\"dhbt\":\"ovnotyfjfcnjbkcn\"},\"id\":\"kphywpnvjto\",\"name\":\"nermcl\",\"type\":\"plpho\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,12 +50,12 @@ public void testGetByResourceGroupWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); CommunicationServiceResource response = manager.communicationServices() - .getByResourceGroupWithResponse("riplrbpbewtg", "fgb", com.azure.core.util.Context.NONE).getValue(); + .getByResourceGroupWithResponse("ovplw", "bhvgy", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("hpvgqz", response.location()); - Assertions.assertEquals("vxdjzlmwlxkvugf", response.tags().get("zovawjvz")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.identity().type()); - Assertions.assertEquals("nmxiebwwaloayqc", response.dataLocation()); - Assertions.assertEquals("cbpwxqpsrknft", response.linkedDomains().get(0)); + Assertions.assertEquals("xuutkncwscwsv", response.location()); + Assertions.assertEquals("togt", response.tags().get("rupqsxvnmicy")); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.identity().type()); + Assertions.assertEquals("kfplgmgsxnk", response.dataLocation()); + Assertions.assertEquals("edeojnabc", response.linkedDomains().get(0)); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubWithResponseMockTests.java index adeafecbf7e8a..ab9d87051b2da 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesLinkNotificationHubWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testLinkNotificationHubWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"resourceId\":\"phywpnvj\"}"; + String responseStr = "{\"resourceId\":\"civfsnkymuctq\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testLinkNotificationHubWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); LinkedNotificationHub response = manager.communicationServices() - .linkNotificationHubWithResponse("xvnmicykvceov", "ilovnot", - new LinkNotificationHubParameters().withResourceId("fj").withConnectionString("cnjbkcnxdhbt"), + .linkNotificationHubWithResponse("wrmjmwvvjektc", "senhwlrs", + new LinkNotificationHubParameters().withResourceId("frzpwvlqdqgb").withConnectionString("qylihkaetckt"), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("phywpnvj", response.resourceId()); + Assertions.assertEquals("civfsnkymuctq", response.resourceId()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupMockTests.java index 6f5617efde707..6f7856f58e7ba 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListByResourceGroupMockTests.java @@ -33,7 +33,7 @@ public void testListByResourceGroup() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"hostName\":\"zlfmisgwbnbbeld\",\"dataLocation\":\"wkz\",\"notificationHubId\":\"liourqhak\",\"version\":\"hashsfwxosow\",\"immutableResourceId\":\"cugicjoox\",\"linkedDomains\":[\"bwpucwwfvovbv\",\"euecivyhzceuoj\",\"jrwjueiotwm\"]},\"identity\":{\"principalId\":\"2fd1e2fe-543c-45e1-a389-77c60fad415a\",\"tenantId\":\"dc909fc1-c659-4b48-9dac-d0d8e9bc696f\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{\"xnrj\":{\"principalId\":\"289bdce0-818e-482d-bc4b-86ecee3e8a42\",\"clientId\":\"bc2b9b90-1c59-421f-a915-257be86f90c5\"}}},\"location\":\"gqwgxhniskxfbkp\",\"tags\":{\"l\":\"klwndnhjdauwhv\",\"xujznbmpowu\":\"zbtd\"},\"id\":\"przqlveu\",\"name\":\"lupj\",\"type\":\"khfxobbcswsrt\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"arbu\",\"dataLocation\":\"rcvpnazzmhjrunmp\",\"notificationHubId\":\"tdbhrbnla\",\"version\":\"xmyskp\",\"immutableResourceId\":\"enbtkcxywny\",\"linkedDomains\":[\"synlqidybyxczfc\"]},\"identity\":{\"principalId\":\"bf2be09a-79d1-4bfd-8556-fdcfa2f1c7d1\",\"tenantId\":\"7272b381-7584-43b2-9f5d-9987dbf5ed10\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"p\":{\"principalId\":\"41190edf-d46b-47c5-b956-6de1e0f169b6\",\"clientId\":\"f540ba33-a605-41e3-bb82-9635801f5ec5\"},\"rqlfktsthsucocmn\":{\"principalId\":\"2960e378-b58a-46d4-9647-3731440d096a\",\"clientId\":\"34d003dd-841b-4af2-ae4b-23dd8601e310\"},\"zt\":{\"principalId\":\"73f7f370-6377-4fa5-b684-b37feee8e273\",\"clientId\":\"4c7ff0ab-7003-4ee7-97b3-faef757a6879\"}}},\"location\":\"twwrqp\",\"tags\":{\"xibxujwbhqwalm\":\"ckzywbiexzfeyue\",\"ux\":\"zyoxaepdkzjan\",\"zt\":\"hdwbavxbniwdjs\"},\"id\":\"dbpgnxytxhp\",\"name\":\"xbzpfzab\",\"type\":\"lcuhxwtctyqiklb\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,13 +51,12 @@ public void testListByResourceGroup() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.communicationServices().listByResourceGroup("rcrgvx", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("gqwgxhniskxfbkp", response.iterator().next().location()); - Assertions.assertEquals("klwndnhjdauwhv", response.iterator().next().tags().get("l")); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, - response.iterator().next().identity().type()); - Assertions.assertEquals("wkz", response.iterator().next().dataLocation()); - Assertions.assertEquals("bwpucwwfvovbv", response.iterator().next().linkedDomains().get(0)); + = manager.communicationServices().listByResourceGroup("buynhijggm", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("twwrqp", response.iterator().next().location()); + Assertions.assertEquals("ckzywbiexzfeyue", response.iterator().next().tags().get("xibxujwbhqwalm")); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.iterator().next().identity().type()); + Assertions.assertEquals("rcvpnazzmhjrunmp", response.iterator().next().dataLocation()); + Assertions.assertEquals("synlqidybyxczfc", response.iterator().next().linkedDomains().get(0)); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListMockTests.java index f3703c656b114..9460b6767a49d 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/CommunicationServicesListMockTests.java @@ -33,7 +33,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"rmclfplphoxu\",\"dataLocation\":\"crpab\",\"notificationHubId\":\"epsbjtazqu\",\"version\":\"ywpmueefjzwfqkq\",\"immutableResourceId\":\"ids\",\"linkedDomains\":[\"nobglaocq\",\"tcc\",\"g\"]},\"identity\":{\"principalId\":\"7003f4bf-93d3-453d-9de3-0be1a5e78250\",\"tenantId\":\"0d8d20d9-f552-447e-b412-91d468ba50ac\",\"type\":\"SystemAssigned,UserAssigned\",\"userAssignedIdentities\":{\"yrxvwfudwpznt\":{\"principalId\":\"f164af8c-2ede-4e44-82d8-78154418b5f0\",\"clientId\":\"6bf885ab-5a37-435a-b09a-88c9cc2fd899\"},\"zhlrqjb\":{\"principalId\":\"6660e23a-50ce-4f20-8f7d-4a7df832b73d\",\"clientId\":\"6fccf7f4-49c4-4207-8224-43a5cfb1cbb2\"}}},\"location\":\"kfrlhrxsbky\",\"tags\":{\"uzbpzkafku\":\"ca\",\"rnwb\":\"b\",\"hspkdeemao\":\"ehhseyvjusrts\",\"gkvtmelmqkrhah\":\"mx\"},\"id\":\"ljuahaquhcdh\",\"name\":\"duala\",\"type\":\"xqpvfadmw\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Running\",\"hostName\":\"brjcxe\",\"dataLocation\":\"fuwutttxf\",\"notificationHubId\":\"rbirphxe\",\"version\":\"yva\",\"immutableResourceId\":\"nljky\",\"linkedDomains\":[\"vuujq\"]},\"identity\":{\"principalId\":\"d1370ea4-70f5-4ddf-bb49-cb5a41fbae09\",\"tenantId\":\"f90dff6b-94fb-40e7-83dc-85438df93e2f\",\"type\":\"None\",\"userAssignedIdentities\":{\"yoxgvcltbgsnc\":{\"principalId\":\"191c3c83-857b-4b1a-bcda-1b65b274bc6a\",\"clientId\":\"2942f4e1-ae8f-4303-b88f-2ca0345e5ee2\"},\"jeszzhbijhtxfv\":{\"principalId\":\"9cd64346-ab86-4494-8699-1fa0d1670311\",\"clientId\":\"e5d37ff6-9006-4a9c-85bd-7d287ddbc539\"}}},\"location\":\"bfs\",\"tags\":{\"pvecxgodeb\":\"eh\",\"pukgriwflzlfb\":\"qkkrb\",\"qzahmgkbrp\":\"zpuzycisp\"},\"id\":\"y\",\"name\":\"hibnuqqkpika\",\"type\":\"rgvtqag\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -53,11 +53,10 @@ public void testList() throws Exception { PagedIterable response = manager.communicationServices().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("kfrlhrxsbky", response.iterator().next().location()); - Assertions.assertEquals("ca", response.iterator().next().tags().get("uzbpzkafku")); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED, - response.iterator().next().identity().type()); - Assertions.assertEquals("crpab", response.iterator().next().dataLocation()); - Assertions.assertEquals("nobglaocq", response.iterator().next().linkedDomains().get(0)); + Assertions.assertEquals("bfs", response.iterator().next().location()); + Assertions.assertEquals("eh", response.iterator().next().tags().get("pvecxgodeb")); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.iterator().next().identity().type()); + Assertions.assertEquals("fuwutttxf", response.iterator().next().dataLocation()); + Assertions.assertEquals("vuujq", response.iterator().next().linkedDomains().get(0)); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/DomainsDeleteMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/DomainsDeleteMockTests.java index 2bab7b106a68a..ebfd25caa0cfe 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/DomainsDeleteMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/DomainsDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.domains().delete("od", "xzb", "cblylpstdbhhxsr", com.azure.core.util.Context.NONE); + manager.domains().delete("yriwwroyqb", "xrmcqibycnojvk", "mefqsgzvahapjyzh", com.azure.core.util.Context.NONE); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateMockTests.java index 3016de4e9a866..241724500860e 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesCreateOrUpdateMockTests.java @@ -33,7 +33,7 @@ public void testCreateOrUpdate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"dataLocation\":\"qxwbpokulpiu\"},\"location\":\"aasipqi\",\"tags\":{\"rpqlp\":\"yuq\",\"auvfbtkuwhhmhyk\":\"wcciuqgbdbu\",\"hkoymkcdyhbp\":\"joxafnndlpi\"},\"id\":\"kpw\",\"name\":\"reqnovvqfov\",\"type\":\"jxywsuws\"}"; + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"dataLocation\":\"xogaokonzmnsikv\"},\"location\":\"qzeqqkdltfzxm\",\"tags\":{\"dkwobdagx\":\"hgure\"},\"id\":\"ibqdxbxwakbogqx\",\"name\":\"dlkzgxhuri\",\"type\":\"lbpodxunk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,14 +50,14 @@ public void testCreateOrUpdate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - EmailServiceResource response = manager.emailServices().define("gkopkwhojvpajqgx").withRegion("oz") - .withExistingResourceGroup("mquxvypo").withTags(mapOf("dd", "helxprglya", "rjxgciqib", "kcbcue", - "sdqrhzoymibmrq", "hos", "dtmhrkwofyyvoqa", "ibahwflus")) - .withDataLocation("cmbqfqvmk").create(); + EmailServiceResource response + = manager.emailServices().define("mnvdfzn").withRegion("hh").withExistingResourceGroup("mcwyhzdxssadb") + .withTags(mapOf("wjmy", "zdzucerscdntnevf", "s", "tdss", "emwabnet", "tmweriofzpyq", "d", "hhszh")) + .withDataLocation("dvxzbncblylpst").create(); - Assertions.assertEquals("aasipqi", response.location()); - Assertions.assertEquals("yuq", response.tags().get("rpqlp")); - Assertions.assertEquals("qxwbpokulpiu", response.dataLocation()); + Assertions.assertEquals("qzeqqkdltfzxm", response.location()); + Assertions.assertEquals("hgure", response.tags().get("dkwobdagx")); + Assertions.assertEquals("xogaokonzmnsikv", response.dataLocation()); } // Use "Map.of" if available diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteMockTests.java index f5ab6677190a0..dbea86887aa5c 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.emailServices().delete("ohdneuel", "phsdyhto", com.azure.core.util.Context.NONE); + manager.emailServices().delete("j", "n", com.azure.core.util.Context.NONE); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupWithResponseMockTests.java index 4fdf78c390ce8..d373dd89895f5 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesGetByResourceGroupWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetByResourceGroupWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Creating\",\"dataLocation\":\"zgx\"},\"location\":\"ripl\",\"tags\":{\"wlrbqtkoievseo\":\"dxunkbebxmubyyn\",\"wzizxbmpgcjefuzm\":\"gqrlltmuwla\"},\"id\":\"vpbttd\",\"name\":\"morppxebmnzbtbh\",\"type\":\"pglkf\"}"; + = "{\"properties\":{\"provisioningState\":\"Running\",\"dataLocation\":\"gylgqgitxmedjvcs\"},\"location\":\"n\",\"tags\":{\"zhxgktrmgucn\":\"ncw\",\"llwptfdy\":\"pkteo\",\"rhhuaopppcqeqx\":\"pfqbuaceopzf\",\"izpost\":\"lzdahzxctobgbkdm\"},\"id\":\"grcfb\",\"name\":\"nrmfqjhhk\",\"type\":\"bpvjymjhx\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testGetByResourceGroupWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); EmailServiceResource response = manager.emailServices() - .getByResourceGroupWithResponse("dkwobdagx", "ibqdxbxwakbogqx", com.azure.core.util.Context.NONE) + .getByResourceGroupWithResponse("bnxknalaulppg", "dtpnapnyiropuhp", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("ripl", response.location()); - Assertions.assertEquals("dxunkbebxmubyyn", response.tags().get("wlrbqtkoievseo")); - Assertions.assertEquals("zgx", response.dataLocation()); + Assertions.assertEquals("n", response.location()); + Assertions.assertEquals("ncw", response.tags().get("zhxgktrmgucn")); + Assertions.assertEquals("gylgqgitxmedjvcs", response.dataLocation()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupMockTests.java index 942c67f99bda4..5feebaecd34f3 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListByResourceGroupMockTests.java @@ -32,7 +32,7 @@ public void testListByResourceGroup() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"dataLocation\":\"bobzdopcjwvnhd\"},\"location\":\"wmgxcxrsl\",\"tags\":{\"qsluicp\":\"twuoegrpkhjwni\"},\"id\":\"ggkzzlvmbmpa\",\"name\":\"modfvuefywsbpfvm\",\"type\":\"yhrfouyftaakcpw\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Updating\",\"dataLocation\":\"jzbexilzznfq\"},\"location\":\"vwpm\",\"tags\":{\"jhwqytjrybnw\":\"ruoujmk\",\"enq\":\"ewgdrjervn\",\"ndoygmifthnzdnd\":\"eh\",\"nayqi\":\"l\"},\"id\":\"ynduha\",\"name\":\"hqlkthumaqo\",\"type\":\"bgycduiertgccym\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,10 +50,10 @@ public void testListByResourceGroup() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.emailServices().listByResourceGroup("pkeqdcvdrhvoo", com.azure.core.util.Context.NONE); + = manager.emailServices().listByResourceGroup("t", com.azure.core.util.Context.NONE); - Assertions.assertEquals("wmgxcxrsl", response.iterator().next().location()); - Assertions.assertEquals("twuoegrpkhjwni", response.iterator().next().tags().get("qsluicp")); - Assertions.assertEquals("bobzdopcjwvnhd", response.iterator().next().dataLocation()); + Assertions.assertEquals("vwpm", response.iterator().next().location()); + Assertions.assertEquals("ruoujmk", response.iterator().next().tags().get("jhwqytjrybnw")); + Assertions.assertEquals("jzbexilzznfq", response.iterator().next().dataLocation()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListMockTests.java index 1169ea28f4bf3..03b8ff7ed227d 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"Moving\",\"dataLocation\":\"dowwquuvxzxclvit\"},\"location\":\"qzonosggbhcohf\",\"tags\":{\"dkzzewkfvhqcrail\":\"jnkaljutiiswacff\",\"rwdmhdlxyjrxsa\":\"pnppfuf\"},\"id\":\"afcnih\",\"name\":\"wqapnedgfbcvk\",\"type\":\"vq\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Creating\",\"dataLocation\":\"vkr\"},\"location\":\"wbxqzvszjfau\",\"tags\":{\"tvtc\":\"dxxiv\",\"wvxysl\":\"aqtdoqmcbx\",\"ytkblmpew\":\"bhsfxob\",\"shqjohxcrsbf\":\"wfbkrvrns\"},\"id\":\"vasrruvwb\",\"name\":\"sqfsubcgjbirxb\",\"type\":\"ybsrfbjfdtwss\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,8 +51,8 @@ public void testList() throws Exception { PagedIterable response = manager.emailServices().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("qzonosggbhcohf", response.iterator().next().location()); - Assertions.assertEquals("jnkaljutiiswacff", response.iterator().next().tags().get("dkzzewkfvhqcrail")); - Assertions.assertEquals("dowwquuvxzxclvit", response.iterator().next().dataLocation()); + Assertions.assertEquals("wbxqzvszjfau", response.iterator().next().location()); + Assertions.assertEquals("dxxiv", response.iterator().next().tags().get("tvtc")); + Assertions.assertEquals("vkr", response.iterator().next().dataLocation()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsWithResponseMockTests.java index c04d9dd9284a4..2dd28ba89640d 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/EmailServicesListVerifiedExchangeOnlineDomainsWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testListVerifiedExchangeOnlineDomainsWithResponse() throws Exception HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "[\"zvqtmnubexkp\",\"ksmond\"]"; + String responseStr = "[\"olpsslqlf\",\"mdnbbglzpswiy\"]"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,6 +50,6 @@ public void testListVerifiedExchangeOnlineDomainsWithResponse() throws Exception List response = manager.emailServices() .listVerifiedExchangeOnlineDomainsWithResponse(com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("zvqtmnubexkp", response.get(0)); + Assertions.assertEquals("olpsslqlf", response.get(0)); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/ManagedServiceIdentityTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/ManagedServiceIdentityTests.java index cf0f0a56858d6..fd700b0ae41d9 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/ManagedServiceIdentityTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/ManagedServiceIdentityTests.java @@ -16,7 +16,7 @@ public final class ManagedServiceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ManagedServiceIdentity model = BinaryData.fromString( - "{\"principalId\":\"7da3f15f-3155-44b6-af7b-57d700e8f3dc\",\"tenantId\":\"49937368-fbce-4402-86f3-d3aa0a9448f0\",\"type\":\"None\",\"userAssignedIdentities\":{\"txilnerkujy\":{\"principalId\":\"67f3950e-cd0f-487e-816c-470e8537908e\",\"clientId\":\"59f0b0a9-0888-4c2b-ac73-b383c3bc4c43\"},\"eju\":{\"principalId\":\"28575b7b-d38a-4ae5-b132-6015536496ec\",\"clientId\":\"a64c4366-9d5f-4d2d-a216-cdc7b15630f9\"},\"awrlyx\":{\"principalId\":\"fbdd6559-a515-49de-8f75-d3f5ee30a72a\",\"clientId\":\"71db179b-92ad-4b09-830b-cfe4be1c994e\"},\"cpr\":{\"principalId\":\"10c013a1-f476-4177-9ed5-85e204c97d46\",\"clientId\":\"0f13e40b-e69a-425e-bc08-3d4491477e09\"}}}") + "{\"principalId\":\"bebc74ca-97ca-44ea-b2d5-b55c1210d16f\",\"tenantId\":\"f5291bd7-d061-4a17-9d4a-c6f025d175a6\",\"type\":\"None\",\"userAssignedIdentities\":{\"txilnerkujy\":{\"principalId\":\"4f4d8daf-9f71-44c5-ba07-b2172fb018a3\",\"clientId\":\"2421076f-59d0-4b0c-9893-f5e492a2ea38\"},\"eju\":{\"principalId\":\"5e7b6896-820d-42ee-bb9f-d596b1919f12\",\"clientId\":\"92a91233-d202-4ba9-989d-b036975c0d20\"},\"awrlyx\":{\"principalId\":\"bf808346-170c-4992-86a8-e44f9f227be1\",\"clientId\":\"e6ec8881-63f6-4bbc-99ae-c2e74353c994\"},\"cpr\":{\"principalId\":\"67c80dc7-9173-4dc8-be62-a43794323825\",\"clientId\":\"aff379ad-161b-4630-a4b8-5a5d9036999f\"}}}") .toObject(ManagedServiceIdentity.class); Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/OperationsListMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/OperationsListMockTests.java index db96b8934e2c3..fa93a4597df0f 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/OperationsListMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/OperationsListMockTests.java @@ -31,7 +31,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"ghxpkdw\",\"isDataAction\":true,\"display\":{\"provider\":\"ebb\",\"resource\":\"m\",\"operation\":\"qupedeojnab\",\"description\":\"hsmtxpsiebtfhvp\"},\"origin\":\"user\",\"actionType\":\"Internal\"}]}"; + = "{\"value\":[{\"name\":\"qftiy\",\"isDataAction\":true,\"display\":{\"provider\":\"cqvyxlwhzlsico\",\"resource\":\"qqn\",\"operation\":\"lryav\",\"description\":\"heun\"},\"origin\":\"system\",\"actionType\":\"Internal\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateWithResponseMockTests.java index b3eb3ebcffe0a..2cca857d1184b 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesCreateOrUpdateWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"dataLocation\":\"pxgy\",\"username\":\"agvrvmnpkuk\",\"displayName\":\"i\",\"provisioningState\":\"Updating\"},\"id\":\"xgwim\",\"name\":\"njhf\",\"type\":\"xw\"}"; + = "{\"properties\":{\"dataLocation\":\"vqtmnub\",\"username\":\"xkp\",\"displayName\":\"smond\",\"provisioningState\":\"Creating\"},\"id\":\"xvy\",\"name\":\"omgkopkwho\",\"type\":\"v\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,10 @@ public void testCreateOrUpdateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SenderUsernameResource response - = manager.senderUsernames().define("v").withExistingDomain("lla", "melwuipiccjz", "z") - .withUsername("ayrhyrnx").withDisplayName("ueedndrdvs").create(); + = manager.senderUsernames().define("opcjwvnhd").withExistingDomain("cvkcvqvpkeqdcv", "rhvoods", "tbobz") + .withUsername("twuoegrpkhjwni").withDisplayName("sluicpdggkzz").create(); - Assertions.assertEquals("agvrvmnpkuk", response.username()); - Assertions.assertEquals("i", response.displayName()); + Assertions.assertEquals("xkp", response.username()); + Assertions.assertEquals("smond", response.displayName()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteWithResponseMockTests.java index a222b53cfd714..7541539748e07 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.senderUsernames().deleteWithResponse("syocogjltdtbnnha", "oocrkvcikhnv", "amqgxqquezikyw", "gxk", + manager.senderUsernames().deleteWithResponse("lxyjr", "sag", "fcnihgwq", "pnedgf", com.azure.core.util.Context.NONE); } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetWithResponseMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetWithResponseMockTests.java index 8473d4bb825f2..162afa504a495 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetWithResponseMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"dataLocation\":\"yhqmibzyhwi\",\"username\":\"smypyynpc\",\"displayName\":\"umnzgmwznmabik\",\"provisioningState\":\"Updating\"},\"id\":\"gj\",\"name\":\"xbldtlwwrlkdmtn\",\"type\":\"vokotllxdyh\"}"; + = "{\"properties\":{\"dataLocation\":\"hfwdsjnkaljutiis\",\"username\":\"acffgdkzzewkfvhq\",\"displayName\":\"a\",\"provisioningState\":\"Updating\"},\"id\":\"n\",\"name\":\"pfuflrw\",\"type\":\"mh\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,9 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SenderUsernameResource response = manager.senderUsernames() - .getWithResponse("sjswsrms", "yzrpzbchckqqzq", "ox", "ysuiizynkedya", com.azure.core.util.Context.NONE) - .getValue(); + .getWithResponse("quuvxzxcl", "ithhqzon", "sg", "b", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("smypyynpc", response.username()); - Assertions.assertEquals("umnzgmwznmabik", response.displayName()); + Assertions.assertEquals("acffgdkzzewkfvhq", response.username()); + Assertions.assertEquals("a", response.displayName()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsMockTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsMockTests.java index 3d440f5f4301c..56f11ba4254f4 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsMockTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SenderUsernamesListByDomainsMockTests.java @@ -32,7 +32,7 @@ public void testListByDomains() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"dataLocation\":\"qkacewii\",\"username\":\"fpubjibwwi\",\"displayName\":\"ohqkvpuvksgpls\",\"provisioningState\":\"Failed\"},\"id\":\"n\",\"name\":\"synljphuopxodl\",\"type\":\"iyntorzihle\"}]}"; + = "{\"value\":[{\"properties\":{\"dataLocation\":\"tmuwlauwzi\",\"username\":\"xbmp\",\"displayName\":\"jefuzmuvpbttdumo\",\"provisioningState\":\"Succeeded\"},\"id\":\"ebmnzbtbhjpglk\",\"name\":\"gohdneuelfphsd\",\"type\":\"htozfikdow\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,10 @@ public void testListByDomains() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.senderUsernames().listByDomains("rsndsytgadgvra", - "aeneqnzarrwl", "uu", com.azure.core.util.Context.NONE); + PagedIterable response = manager.senderUsernames().listByDomains("ebxmubyynt", "lrb", + "tkoievseotgq", com.azure.core.util.Context.NONE); - Assertions.assertEquals("fpubjibwwi", response.iterator().next().username()); - Assertions.assertEquals("ohqkvpuvksgpls", response.iterator().next().displayName()); + Assertions.assertEquals("xbmp", response.iterator().next().username()); + Assertions.assertEquals("jefuzmuvpbttdumo", response.iterator().next().displayName()); } } diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressPropertiesTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressPropertiesTests.java deleted file mode 100644 index 4a93a2ff0a3de..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressPropertiesTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressProperties; -import org.junit.jupiter.api.Assertions; - -public final class SuppressionListAddressPropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SuppressionListAddressProperties model = BinaryData.fromString( - "{\"email\":\"bhvgy\",\"firstName\":\"uosvmkfssxqukk\",\"lastName\":\"l\",\"notes\":\"gsxnkjzkdeslpv\",\"lastModified\":\"2021-06-26T13:13:32Z\",\"dataLocation\":\"i\"}") - .toObject(SuppressionListAddressProperties.class); - Assertions.assertEquals("bhvgy", model.email()); - Assertions.assertEquals("uosvmkfssxqukk", model.firstName()); - Assertions.assertEquals("l", model.lastName()); - Assertions.assertEquals("gsxnkjzkdeslpv", model.notes()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SuppressionListAddressProperties model = new SuppressionListAddressProperties().withEmail("bhvgy") - .withFirstName("uosvmkfssxqukk").withLastName("l").withNotes("gsxnkjzkdeslpv"); - model = BinaryData.fromObject(model).toObject(SuppressionListAddressProperties.class); - Assertions.assertEquals("bhvgy", model.email()); - Assertions.assertEquals("uosvmkfssxqukk", model.firstName()); - Assertions.assertEquals("l", model.lastName()); - Assertions.assertEquals("gsxnkjzkdeslpv", model.notes()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceCollectionTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceCollectionTests.java deleted file mode 100644 index 8d313aabb1501..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceCollectionTests.java +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListAddressResourceCollection; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class SuppressionListAddressResourceCollectionTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SuppressionListAddressResourceCollection model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"email\":\"ibnuqqkpik\",\"firstName\":\"rgvtqag\",\"lastName\":\"uynhijg\",\"notes\":\"ebf\",\"lastModified\":\"2021-09-24T19:25:12Z\",\"dataLocation\":\"butr\"},\"id\":\"pnazzm\",\"name\":\"jrunmpxtt\",\"type\":\"bh\"},{\"properties\":{\"email\":\"nlankxmyskpb\",\"firstName\":\"nbtkcxywnytnr\",\"lastName\":\"nlqidybyxczf\",\"notes\":\"haaxdbabphl\",\"lastModified\":\"2021-02-16T08:29:45Z\",\"dataLocation\":\"fkts\"},\"id\":\"sucocmnyyazttbtw\",\"name\":\"rq\",\"type\":\"uedck\"}],\"nextLink\":\"wbiexzfey\"}") - .toObject(SuppressionListAddressResourceCollection.class); - Assertions.assertEquals("ibnuqqkpik", model.value().get(0).email()); - Assertions.assertEquals("rgvtqag", model.value().get(0).firstName()); - Assertions.assertEquals("uynhijg", model.value().get(0).lastName()); - Assertions.assertEquals("ebf", model.value().get(0).notes()); - Assertions.assertEquals("wbiexzfey", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SuppressionListAddressResourceCollection model - = new SuppressionListAddressResourceCollection().withValue(Arrays.asList( - new SuppressionListAddressResourceInner().withEmail("ibnuqqkpik").withFirstName("rgvtqag") - .withLastName("uynhijg").withNotes("ebf"), - new SuppressionListAddressResourceInner().withEmail("nlankxmyskpb").withFirstName("nbtkcxywnytnr") - .withLastName("nlqidybyxczf").withNotes("haaxdbabphl"))) - .withNextLink("wbiexzfey"); - model = BinaryData.fromObject(model).toObject(SuppressionListAddressResourceCollection.class); - Assertions.assertEquals("ibnuqqkpik", model.value().get(0).email()); - Assertions.assertEquals("rgvtqag", model.value().get(0).firstName()); - Assertions.assertEquals("uynhijg", model.value().get(0).lastName()); - Assertions.assertEquals("ebf", model.value().get(0).notes()); - Assertions.assertEquals("wbiexzfey", model.nextLink()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceInnerTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceInnerTests.java deleted file mode 100644 index 62960286b5189..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressResourceInnerTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListAddressResourceInner; -import org.junit.jupiter.api.Assertions; - -public final class SuppressionListAddressResourceInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SuppressionListAddressResourceInner model = BinaryData.fromString( - "{\"properties\":{\"email\":\"axibxujw\",\"firstName\":\"qwalmuzyoxaepd\",\"lastName\":\"jancu\",\"notes\":\"hdwbavxbniwdjs\",\"lastModified\":\"2021-12-07T05:36:29Z\",\"dataLocation\":\"dbpgnxytxhp\"},\"id\":\"bzpfzab\",\"name\":\"lcuhxwtctyqiklb\",\"type\":\"ovplw\"}") - .toObject(SuppressionListAddressResourceInner.class); - Assertions.assertEquals("axibxujw", model.email()); - Assertions.assertEquals("qwalmuzyoxaepd", model.firstName()); - Assertions.assertEquals("jancu", model.lastName()); - Assertions.assertEquals("hdwbavxbniwdjs", model.notes()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SuppressionListAddressResourceInner model = new SuppressionListAddressResourceInner().withEmail("axibxujw") - .withFirstName("qwalmuzyoxaepd").withLastName("jancu").withNotes("hdwbavxbniwdjs"); - model = BinaryData.fromObject(model).toObject(SuppressionListAddressResourceInner.class); - Assertions.assertEquals("axibxujw", model.email()); - Assertions.assertEquals("qwalmuzyoxaepd", model.firstName()); - Assertions.assertEquals("jancu", model.lastName()); - Assertions.assertEquals("hdwbavxbniwdjs", model.notes()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListPropertiesTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListPropertiesTests.java deleted file mode 100644 index 362c8677713d2..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListPropertiesTests.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListProperties; -import org.junit.jupiter.api.Assertions; - -public final class SuppressionListPropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SuppressionListProperties model = BinaryData.fromString( - "{\"listName\":\"odebfqkkrbmpu\",\"lastUpdatedTimeStamp\":\"riwflzlfb\",\"createdTimeStamp\":\"puz\",\"dataLocation\":\"ispnqzahmgkbrp\"}") - .toObject(SuppressionListProperties.class); - Assertions.assertEquals("odebfqkkrbmpu", model.listName()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SuppressionListProperties model = new SuppressionListProperties().withListName("odebfqkkrbmpu"); - model = BinaryData.fromObject(model).toObject(SuppressionListProperties.class); - Assertions.assertEquals("odebfqkkrbmpu", model.listName()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceCollectionTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceCollectionTests.java deleted file mode 100644 index e90c818895397..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceCollectionTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; -import com.azure.resourcemanager.communication.models.SuppressionListResourceCollection; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class SuppressionListResourceCollectionTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SuppressionListResourceCollection model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"listName\":\"iyqzrnk\",\"lastUpdatedTimeStamp\":\"vyxlwhzlsicohoqq\",\"createdTimeStamp\":\"vlryavwhheunmmq\",\"dataLocation\":\"yxzk\"},\"id\":\"ocukoklyax\",\"name\":\"conuqszfkbeype\",\"type\":\"rmjmwvvjektc\"},{\"properties\":{\"listName\":\"nhwlrsffrzpwvl\",\"lastUpdatedTimeStamp\":\"q\",\"createdTimeStamp\":\"iqylihkaetck\",\"dataLocation\":\"fcivfsnkym\"},\"id\":\"tqhjfbebrjcx\",\"name\":\"rfuwutt\",\"type\":\"xfvjrbirp\"}],\"nextLink\":\"epcyvahfnlj\"}") - .toObject(SuppressionListResourceCollection.class); - Assertions.assertEquals("iyqzrnk", model.value().get(0).listName()); - Assertions.assertEquals("epcyvahfnlj", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SuppressionListResourceCollection model = new SuppressionListResourceCollection() - .withValue(Arrays.asList(new SuppressionListResourceInner().withListName("iyqzrnk"), - new SuppressionListResourceInner().withListName("nhwlrsffrzpwvl"))) - .withNextLink("epcyvahfnlj"); - model = BinaryData.fromObject(model).toObject(SuppressionListResourceCollection.class); - Assertions.assertEquals("iyqzrnk", model.value().get(0).listName()); - Assertions.assertEquals("epcyvahfnlj", model.nextLink()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceInnerTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceInnerTests.java deleted file mode 100644 index 42bb1c1317922..0000000000000 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListResourceInnerTests.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.communication.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.communication.fluent.models.SuppressionListResourceInner; -import org.junit.jupiter.api.Assertions; - -public final class SuppressionListResourceInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SuppressionListResourceInner model = BinaryData.fromString( - "{\"properties\":{\"listName\":\"xjvuujqgidokg\",\"lastUpdatedTimeStamp\":\"jyoxgvclt\",\"createdTimeStamp\":\"sncghkjeszz\",\"dataLocation\":\"ijhtxf\"},\"id\":\"xbf\",\"name\":\"mxnehmp\",\"type\":\"ec\"}") - .toObject(SuppressionListResourceInner.class); - Assertions.assertEquals("xjvuujqgidokg", model.listName()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SuppressionListResourceInner model = new SuppressionListResourceInner().withListName("xjvuujqgidokg"); - model = BinaryData.fromObject(model).toObject(SuppressionListResourceInner.class); - Assertions.assertEquals("xjvuujqgidokg", model.listName()); - } -} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/UserAssignedIdentityTests.java b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/UserAssignedIdentityTests.java index 5d8da4c074c8e..6d555ee7cf05e 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/UserAssignedIdentityTests.java +++ b/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/UserAssignedIdentityTests.java @@ -11,7 +11,7 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { UserAssignedIdentity model = BinaryData.fromString( - "{\"principalId\":\"5688cec6-fe41-47f6-b13e-8bf2b671e519\",\"clientId\":\"b428f3e0-2e4a-4810-b7f6-927949014134\"}") + "{\"principalId\":\"31004c3d-9ac5-4e79-bf00-0c50e42a2fdf\",\"clientId\":\"3f12c4e5-9bcd-457f-8bba-911e58c6db1d\"}") .toObject(UserAssignedIdentity.class); } diff --git a/sdk/communication/communication-tests-template.yml b/sdk/communication/communication-tests-template.yml index 5daf721140dc1..d285bb7701806 100644 --- a/sdk/communication/communication-tests-template.yml +++ b/sdk/communication/communication-tests-template.yml @@ -42,22 +42,25 @@ parameters: MatrixReplace: - SKIP_UPDATE_CAPABILITIES_LIVE_TESTS=false/true -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - CloudConfig: ${{ parameters.CloudConfig }} - Clouds: ${{ parameters.Clouds }} - TestMode: ${{ parameters.TestMode }} - Artifacts: - - name: ${{ parameters.PackageName }} - groupId: com.azure - safeName: ${{ parameters.SafeName }} - ServiceDirectory: communication - TestResourceDirectories: ${{ parameters.TestResourceDirectories }} - EnvVars: - SKIP_LIVE_TEST: TRUE - ${{ each var in parameters.EnVars }}: - ${{ var.key }}: ${{ var.value }} - MatrixConfigs: - - ${{ each config in parameters.MatrixConfigs }}: - - ${{ config }} +extends: + template: /eng/pipelines/templates/stages/1es-redirect.yml + parameters: + stages: + - template: /eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml + parameters: + CloudConfig: ${{ parameters.CloudConfig }} + Clouds: ${{ parameters.Clouds }} + TestMode: ${{ parameters.TestMode }} + Artifacts: + - name: ${{ parameters.PackageName }} + groupId: com.azure + safeName: ${{ parameters.SafeName }} + ServiceDirectory: communication + TestResourceDirectories: ${{ parameters.TestResourceDirectories }} + EnvVars: + SKIP_LIVE_TEST: TRUE + ${{ each var in parameters.EnVars }}: + ${{ var.key }}: ${{ var.value }} + MatrixConfigs: + - ${{ each config in parameters.MatrixConfigs }}: + - ${{ config }} diff --git a/sdk/confidentialledger/azure-resourcemanager-confidentialledger/pom.xml b/sdk/confidentialledger/azure-resourcemanager-confidentialledger/pom.xml index 39d47b6102403..2f1695edd6642 100644 --- a/sdk/confidentialledger/azure-resourcemanager-confidentialledger/pom.xml +++ b/sdk/confidentialledger/azure-resourcemanager-confidentialledger/pom.xml @@ -69,7 +69,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -95,13 +95,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -113,7 +113,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/confidentialledger/azure-security-confidentialledger/CHANGELOG.md b/sdk/confidentialledger/azure-security-confidentialledger/CHANGELOG.md index 4b9a84c9adaee..61ec09a9d8f78 100644 --- a/sdk/confidentialledger/azure-security-confidentialledger/CHANGELOG.md +++ b/sdk/confidentialledger/azure-security-confidentialledger/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.0.18 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.0.17 (2024-02-20) ### Other Changes diff --git a/sdk/confidentialledger/azure-security-confidentialledger/pom.xml b/sdk/confidentialledger/azure-security-confidentialledger/pom.xml index a40d331b05cf7..da92d88b9eafc 100644 --- a/sdk/confidentialledger/azure-security-confidentialledger/pom.xml +++ b/sdk/confidentialledger/azure-security-confidentialledger/pom.xml @@ -70,7 +70,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/confidentialledger/azure-security-confidentialledger/tests.yml b/sdk/confidentialledger/azure-security-confidentialledger/tests.yml index 71f819885091a..10ff264ec364d 100644 --- a/sdk/confidentialledger/azure-security-confidentialledger/tests.yml +++ b/sdk/confidentialledger/azure-security-confidentialledger/tests.yml @@ -1,13 +1,13 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: confidentialledger - Artifacts: - - name: azure-security-confidentialledger - groupId: com.azure - safeName: azuresecurityconfidentialledger - EnvVars: - LEDGERURI: $(LEDGER_URI) - IDENTITYSERVICEURI: https://identity.confidential-ledger.core.azure.com/ledgerIdentity/$(LEDGER_NAME) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: confidentialledger + Artifacts: + - name: azure-security-confidentialledger + groupId: com.azure + safeName: azuresecurityconfidentialledger + EnvVars: + LEDGERURI: $(LEDGER_URI) + IDENTITYSERVICEURI: https://identity.confidential-ledger.core.azure.com/ledgerIdentity/$(LEDGER_NAME) diff --git a/sdk/confluent/azure-resourcemanager-confluent/CHANGELOG.md b/sdk/confluent/azure-resourcemanager-confluent/CHANGELOG.md index d5ff86bfffb2f..6c98baaddf577 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/CHANGELOG.md +++ b/sdk/confluent/azure-resourcemanager-confluent/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.1 (Unreleased) +## 1.2.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,95 @@ ### Other Changes +## 1.1.0 (2024-03-21) + +- Azure Resource Manager Confluent client library for Java. This package contains Microsoft Azure SDK for Confluent Management SDK. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +* `models.SCMetadataEntity` was added + +* `models.ApiKeyResourceEntity` was added + +* `models.SchemaRegistryClusterRecord` was added + +* `models.ListSchemaRegistryClustersResponse` was added + +* `models.GetEnvironmentsResponse` was added + +* `models.ApiKeyRecord` was added + +* `models.AccessRoleBindingNameListSuccessResponse` was added + +* `models.SchemaRegistryClusterSpecEntity` was added + +* `models.ListRegionsSuccessResponse` was added + +* `models.SCEnvironmentRecord` was added + +* `models.RegionRecord` was added + +* `models.CreateApiKeyModel` was added + +* `models.SCClusterSpecEntity` was added + +* `models.RegionSpecEntity` was added + +* `models.ListClustersSuccessResponse` was added + +* `models.SchemaRegistryClusterStatusEntity` was added + +* `models.SCClusterRecord` was added + +* `models.SCClusterByokEntity` was added + +* `models.SCClusterNetworkEnvironmentEntity` was added + +* `models.ApiKeyOwnerEntity` was added + +* `models.SchemaRegistryClusterEnvironmentRegionEntity` was added + +* `models.AccessCreateRoleBindingRequestModel` was added + +* `models.ApiKeySpecEntity` was added + +#### `models.Organizations` was modified + +* `listEnvironments(java.lang.String,java.lang.String)` was added +* `deleteClusterApiKeyWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `getClusterApiKey(java.lang.String,java.lang.String,java.lang.String)` was added +* `getSchemaRegistryClusterById(java.lang.String,java.lang.String,java.lang.String,java.lang.String)` was added +* `listSchemaRegistryClusters(java.lang.String,java.lang.String,java.lang.String)` was added +* `getSchemaRegistryClusterByIdWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listClusters(java.lang.String,java.lang.String,java.lang.String)` was added +* `deleteClusterApiKey(java.lang.String,java.lang.String,java.lang.String)` was added +* `createApiKey(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.CreateApiKeyModel)` was added +* `listEnvironments(java.lang.String,java.lang.String,java.lang.Integer,java.lang.String,com.azure.core.util.Context)` was added +* `getClusterByIdWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `createApiKeyWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.CreateApiKeyModel,com.azure.core.util.Context)` was added +* `listRegions(java.lang.String,java.lang.String,models.ListAccessRequestModel)` was added +* `listSchemaRegistryClusters(java.lang.String,java.lang.String,java.lang.String,java.lang.Integer,java.lang.String,com.azure.core.util.Context)` was added +* `getClusterApiKeyWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listRegionsWithResponse(java.lang.String,java.lang.String,models.ListAccessRequestModel,com.azure.core.util.Context)` was added +* `getEnvironmentById(java.lang.String,java.lang.String,java.lang.String)` was added +* `getClusterById(java.lang.String,java.lang.String,java.lang.String,java.lang.String)` was added +* `listClusters(java.lang.String,java.lang.String,java.lang.String,java.lang.Integer,java.lang.String,com.azure.core.util.Context)` was added +* `getEnvironmentByIdWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added + +#### `models.Access` was modified + +* `createRoleBinding(java.lang.String,java.lang.String,models.AccessCreateRoleBindingRequestModel)` was added +* `listRoleBindingNameList(java.lang.String,java.lang.String,models.ListAccessRequestModel)` was added +* `listRoleBindingNameListWithResponse(java.lang.String,java.lang.String,models.ListAccessRequestModel,com.azure.core.util.Context)` was added +* `deleteRoleBinding(java.lang.String,java.lang.String,java.lang.String)` was added +* `createRoleBindingWithResponse(java.lang.String,java.lang.String,models.AccessCreateRoleBindingRequestModel,com.azure.core.util.Context)` was added +* `deleteRoleBindingWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added + +#### `models.OrganizationResource` was modified + +* `listRegionsWithResponse(models.ListAccessRequestModel,com.azure.core.util.Context)` was added +* `listRegions(models.ListAccessRequestModel)` was added + ## 1.0.0 (2023-11-16) - Azure Resource Manager Confluent client library for Java. This package contains Microsoft Azure SDK for Confluent Management SDK. Package tag package-2023-08-22. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/confluent/azure-resourcemanager-confluent/README.md b/sdk/confluent/azure-resourcemanager-confluent/README.md index 6c0e2371f61d0..3f402eccfcf1d 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/README.md +++ b/sdk/confluent/azure-resourcemanager-confluent/README.md @@ -2,7 +2,7 @@ Azure Resource Manager Confluent client library for Java. -This package contains Microsoft Azure SDK for Confluent Management SDK. Package tag package-2023-08-22. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for Confluent Management SDK. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-confluent - 1.0.0 + 1.1.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/confluent/azure-resourcemanager-confluent/SAMPLE.md b/sdk/confluent/azure-resourcemanager-confluent/SAMPLE.md index 9b80dd14cff52..8874bf6c0efbb 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/SAMPLE.md +++ b/sdk/confluent/azure-resourcemanager-confluent/SAMPLE.md @@ -3,10 +3,13 @@ ## Access +- [CreateRoleBinding](#access_createrolebinding) +- [DeleteRoleBinding](#access_deleterolebinding) - [InviteUser](#access_inviteuser) - [ListClusters](#access_listclusters) - [ListEnvironments](#access_listenvironments) - [ListInvitations](#access_listinvitations) +- [ListRoleBindingNameList](#access_listrolebindingnamelist) - [ListRoleBindings](#access_listrolebindings) - [ListServiceAccounts](#access_listserviceaccounts) - [ListUsers](#access_listusers) @@ -19,10 +22,20 @@ ## Organization - [Create](#organization_create) +- [CreateApiKey](#organization_createapikey) - [Delete](#organization_delete) +- [DeleteClusterApiKey](#organization_deleteclusterapikey) - [GetByResourceGroup](#organization_getbyresourcegroup) +- [GetClusterApiKey](#organization_getclusterapikey) +- [GetClusterById](#organization_getclusterbyid) +- [GetEnvironmentById](#organization_getenvironmentbyid) +- [GetSchemaRegistryClusterById](#organization_getschemaregistryclusterbyid) - [List](#organization_list) - [ListByResourceGroup](#organization_listbyresourcegroup) +- [ListClusters](#organization_listclusters) +- [ListEnvironments](#organization_listenvironments) +- [ListRegions](#organization_listregions) +- [ListSchemaRegistryClusters](#organization_listschemaregistryclusters) - [Update](#organization_update) ## OrganizationOperations @@ -33,18 +46,72 @@ - [ValidateOrganization](#validations_validateorganization) - [ValidateOrganizationV2](#validations_validateorganizationv2) +### Access_CreateRoleBinding + +```java +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; + +/** + * Samples for Access CreateRoleBinding. + */ +public final class AccessCreateRoleBindingSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_CreateRoleBinding. + * json + */ + /** + * Sample code: Access_CreateRoleBinding. + * + * @param manager Entry point to ConfluentManager. + */ + public static void accessCreateRoleBinding(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.access().createRoleBindingWithResponse("myResourceGroup", "myOrganization", + new AccessCreateRoleBindingRequestModel().withPrincipal("User:u-111aaa").withRoleName("CloudClusterAdmin") + .withCrnPattern( + "crn://confluent.cloud/organization=1111aaaa-11aa-11aa-11aa-111111aaaaaa/environment=env-aaa1111/cloud-cluster=lkc-1111aaa"), + com.azure.core.util.Context.NONE); + } +} +``` + +### Access_DeleteRoleBinding + +```java +/** + * Samples for Access DeleteRoleBinding. + */ +public final class AccessDeleteRoleBindingSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_DeleteRoleBinding. + * json + */ + /** + * Sample code: Access_DeleteRoleBinding. + * + * @param manager Entry point to ConfluentManager. + */ + public static void accessDeleteRoleBinding(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.access().deleteRoleBindingWithResponse("myResourceGroup", "myOrganization", "dlz-f3a90de", + com.azure.core.util.Context.NONE); + } +} +``` + ### Access_InviteUser ```java -import com.azure.resourcemanager.confluent.models.AccessInvitedUserDetails; import com.azure.resourcemanager.confluent.models.AccessInviteUserAccountModel; +import com.azure.resourcemanager.confluent.models.AccessInvitedUserDetails; /** * Samples for Access InviteUser. */ public final class AccessInviteUserSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_InviteUser.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_InviteUser.json */ /** * Sample code: Access_InviteUser. @@ -52,7 +119,10 @@ public final class AccessInviteUserSamples { * @param manager Entry point to ConfluentManager. */ public static void accessInviteUser(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().inviteUserWithResponse("myResourceGroup", "myOrganization", new AccessInviteUserAccountModel().withInvitedUserDetails(new AccessInvitedUserDetails().withInvitedEmail("user2@onmicrosoft.com").withAuthType("AUTH_TYPE_SSO")), com.azure.core.util.Context.NONE); + manager.access().inviteUserWithResponse("myResourceGroup", "myOrganization", + new AccessInviteUserAccountModel().withInvitedUserDetails( + new AccessInvitedUserDetails().withInvitedEmail("user2@onmicrosoft.com").withAuthType("AUTH_TYPE_SSO")), + com.azure.core.util.Context.NONE); } } ``` @@ -69,7 +139,8 @@ import java.util.Map; */ public final class AccessListClustersSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_ClusterList.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_ClusterList.json */ /** * Sample code: Access_ClusterList. @@ -77,7 +148,10 @@ public final class AccessListClustersSamples { * @param manager Entry point to ConfluentManager. */ public static void accessClusterList(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().listClustersWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel().withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), com.azure.core.util.Context.NONE); + manager.access() + .listClustersWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel() + .withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -106,7 +180,9 @@ import java.util.Map; */ public final class AccessListEnvironmentsSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_EnvironmentList.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_EnvironmentList. + * json */ /** * Sample code: Access_EnvironmentList. @@ -114,7 +190,10 @@ public final class AccessListEnvironmentsSamples { * @param manager Entry point to ConfluentManager. */ public static void accessEnvironmentList(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().listEnvironmentsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel().withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), com.azure.core.util.Context.NONE); + manager.access() + .listEnvironmentsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel() + .withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -143,7 +222,9 @@ import java.util.Map; */ public final class AccessListInvitationsSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_InvitationsList.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_InvitationsList. + * json */ /** * Sample code: Access_InvitationsList. @@ -151,7 +232,52 @@ public final class AccessListInvitationsSamples { * @param manager Entry point to ConfluentManager. */ public static void accessInvitationsList(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().listInvitationsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel().withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder", "status", "INVITE_STATUS_SENT")), com.azure.core.util.Context.NONE); + manager.access().listInvitationsWithResponse("myResourceGroup", "myOrganization", + new ListAccessRequestModel().withSearchFilters( + mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder", "status", "INVITE_STATUS_SENT")), + com.azure.core.util.Context.NONE); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} +``` + +### Access_ListRoleBindingNameList + +```java +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Access ListRoleBindingNameList. + */ +public final class AccessListRoleBindingNameListSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Access_RoleBindingNameList.json + */ + /** + * Sample code: Access_RoleBindingNameList. + * + * @param manager Entry point to ConfluentManager. + */ + public static void accessRoleBindingNameList(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.access().listRoleBindingNameListWithResponse("myResourceGroup", "myOrganization", + new ListAccessRequestModel().withSearchFilters(mapOf("crn_pattern", + "crn://confluent.cloud/organization=1aa7de07-298e-479c-8f2f-16ac91fd8e76", "namespace", + "public,dataplane,networking,identity,datagovernance,connect,streamcatalog,pipelines,ksql")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -180,7 +306,9 @@ import java.util.Map; */ public final class AccessListRoleBindingsSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_RoleBindingList.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_RoleBindingList. + * json */ /** * Sample code: Access_RoleBindingList. @@ -188,7 +316,10 @@ public final class AccessListRoleBindingsSamples { * @param manager Entry point to ConfluentManager. */ public static void accessRoleBindingList(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().listRoleBindingsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel().withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), com.azure.core.util.Context.NONE); + manager.access() + .listRoleBindingsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel() + .withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -217,7 +348,8 @@ import java.util.Map; */ public final class AccessListServiceAccountsSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_ServiceAccountsList.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Access_ServiceAccountsList.json */ /** * Sample code: Access_ServiceAccountsList. @@ -225,7 +357,10 @@ public final class AccessListServiceAccountsSamples { * @param manager Entry point to ConfluentManager. */ public static void accessServiceAccountsList(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().listServiceAccountsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel().withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), com.azure.core.util.Context.NONE); + manager.access() + .listServiceAccountsWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel() + .withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -254,7 +389,8 @@ import java.util.Map; */ public final class AccessListUsersSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_UsersList.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_UsersList.json */ /** * Sample code: Access_UsersList. @@ -262,7 +398,9 @@ public final class AccessListUsersSamples { * @param manager Entry point to ConfluentManager. */ public static void accessUsersList(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.access().listUsersWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel().withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), com.azure.core.util.Context.NONE); + manager.access().listUsersWithResponse("myResourceGroup", "myOrganization", new ListAccessRequestModel() + .withSearchFilters(mapOf("pageSize", "10", "pageToken", "fakeTokenPlaceholder")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -282,14 +420,14 @@ public final class AccessListUsersSamples { ### MarketplaceAgreements_Create ```java -import com.azure.resourcemanager.confluent.fluent.models.ConfluentAgreementResourceInner; /** * Samples for MarketplaceAgreements Create. */ public final class MarketplaceAgreementsCreateSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/MarketplaceAgreements_Create.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * MarketplaceAgreements_Create.json */ /** * Sample code: MarketplaceAgreements_Create. @@ -310,7 +448,8 @@ public final class MarketplaceAgreementsCreateSamples { */ public final class MarketplaceAgreementsListSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/MarketplaceAgreements_List.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * MarketplaceAgreements_List.json */ /** * Sample code: MarketplaceAgreements_List. @@ -338,7 +477,8 @@ import java.util.Map; */ public final class OrganizationCreateSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Create.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Create.json */ /** * Sample code: Organization_Create. @@ -346,7 +486,16 @@ public final class OrganizationCreateSamples { * @param manager Entry point to ConfluentManager. */ public static void organizationCreate(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.organizations().define("myOrganization").withRegion("West US").withExistingResourceGroup("myResourceGroup").withOfferDetail(new OfferDetail().withPublisherId("string").withId("string").withPlanId("string").withPlanName("string").withTermUnit("string").withPrivateOfferId("string").withPrivateOfferIds(Arrays.asList("string"))).withUserDetail(new UserDetail().withFirstName("string").withLastName("string").withEmailAddress("contoso@microsoft.com").withUserPrincipalName("contoso@microsoft.com").withAadEmail("contoso@microsoft.com")).withTags(mapOf("Environment", "Dev")).withLinkOrganization(new LinkOrganization().withToken("fakeTokenPlaceholder")).create(); + manager.organizations().define("myOrganization").withRegion("West US") + .withExistingResourceGroup("myResourceGroup") + .withOfferDetail( + new OfferDetail().withPublisherId("string").withId("string").withPlanId("string").withPlanName("string") + .withTermUnit("string").withPrivateOfferId("string").withPrivateOfferIds(Arrays.asList("string"))) + .withUserDetail(new UserDetail().withFirstName("string").withLastName("string") + .withEmailAddress("contoso@microsoft.com").withUserPrincipalName("contoso@microsoft.com") + .withAadEmail("contoso@microsoft.com")) + .withTags(mapOf("Environment", "Dev")) + .withLinkOrganization(new LinkOrganization().withToken("fakeTokenPlaceholder")).create(); } // Use "Map.of" if available @@ -363,6 +512,33 @@ public final class OrganizationCreateSamples { } ``` +### Organization_CreateApiKey + +```java +import com.azure.resourcemanager.confluent.models.CreateApiKeyModel; + +/** + * Samples for Organization CreateApiKey. + */ +public final class OrganizationCreateApiKeySamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_CreateClusterAPIKey.json + */ + /** + * Sample code: Organization_CreateAPIKey. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationCreateAPIKey(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().createApiKeyWithResponse( + "myResourceGroup", "myOrganization", "env-12132", "clusterId-123", new CreateApiKeyModel() + .withName("CI kafka access key").withDescription("This API key provides kafka access to cluster x"), + com.azure.core.util.Context.NONE); + } +} +``` + ### Organization_Delete ```java @@ -371,7 +547,8 @@ public final class OrganizationCreateSamples { */ public final class OrganizationDeleteSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Delete.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Delete.json */ /** * Sample code: Confluent_Delete. @@ -384,6 +561,29 @@ public final class OrganizationDeleteSamples { } ``` +### Organization_DeleteClusterApiKey + +```java +/** + * Samples for Organization DeleteClusterApiKey. + */ +public final class OrganizationDeleteClusterApiKeySamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_DeleteClusterAPIKey.json + */ + /** + * Sample code: Organization_DeleteClusterAPIKey. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationDeleteClusterAPIKey(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().deleteClusterApiKeyWithResponse("myResourceGroup", "myOrganization", "ZFZ6SZZZWGYBEIFB", + com.azure.core.util.Context.NONE); + } +} +``` + ### Organization_GetByResourceGroup ```java @@ -392,7 +592,8 @@ public final class OrganizationDeleteSamples { */ public final class OrganizationGetByResourceGroupSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Get.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Get.json */ /** * Sample code: Organization_Get. @@ -400,7 +601,101 @@ public final class OrganizationGetByResourceGroupSamples { * @param manager Entry point to ConfluentManager. */ public static void organizationGet(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.organizations().getByResourceGroupWithResponse("myResourceGroup", "myOrganization", com.azure.core.util.Context.NONE); + manager.organizations().getByResourceGroupWithResponse("myResourceGroup", "myOrganization", + com.azure.core.util.Context.NONE); + } +} +``` + +### Organization_GetClusterApiKey + +```java +/** + * Samples for Organization GetClusterApiKey. + */ +public final class OrganizationGetClusterApiKeySamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetClusterAPIKey.json + */ + /** + * Sample code: Organization_GetClusterAPIKey. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationGetClusterAPIKey(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getClusterApiKeyWithResponse("myResourceGroup", "myOrganization", "apiKeyId-123", + com.azure.core.util.Context.NONE); + } +} +``` + +### Organization_GetClusterById + +```java +/** + * Samples for Organization GetClusterById. + */ +public final class OrganizationGetClusterByIdSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetClusterById.json + */ + /** + * Sample code: Organization_GetClusterById. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationGetClusterById(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getClusterByIdWithResponse("myResourceGroup", "myOrganization", "env-12132", + "dlz-f3a90de", com.azure.core.util.Context.NONE); + } +} +``` + +### Organization_GetEnvironmentById + +```java +/** + * Samples for Organization GetEnvironmentById. + */ +public final class OrganizationGetEnvironmentByIdSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetEnvironmentById.json + */ + /** + * Sample code: Organization_GetEnvironmentById. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationGetEnvironmentById(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getEnvironmentByIdWithResponse("myResourceGroup", "myOrganization", "dlz-f3a90de", + com.azure.core.util.Context.NONE); + } +} +``` + +### Organization_GetSchemaRegistryClusterById + +```java +/** + * Samples for Organization GetSchemaRegistryClusterById. + */ +public final class OrganizationGetSchemaRegistryClusterByIdSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetSchemaRegistryClusterById.json + */ + /** + * Sample code: Organization_GetSchemaRegistryClusterById. + * + * @param manager Entry point to ConfluentManager. + */ + public static void + organizationGetSchemaRegistryClusterById(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getSchemaRegistryClusterByIdWithResponse("myResourceGroup", "myOrganization", + "env-stgcczjp2j3", "lsrc-stgczkq22z", com.azure.core.util.Context.NONE); } } ``` @@ -413,7 +708,8 @@ public final class OrganizationGetByResourceGroupSamples { */ public final class OrganizationListSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_ListBySubscription.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_ListBySubscription.json */ /** * Sample code: Organization_ListBySubscription. @@ -434,7 +730,8 @@ public final class OrganizationListSamples { */ public final class OrganizationListByResourceGroupSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_ListByResourceGroup.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_ListByResourceGroup.json */ /** * Sample code: Organization_ListByResourceGroup. @@ -447,6 +744,119 @@ public final class OrganizationListByResourceGroupSamples { } ``` +### Organization_ListClusters + +```java +/** + * Samples for Organization ListClusters. + */ +public final class OrganizationListClustersSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_ClusterList. + * json + */ + /** + * Sample code: Organization_ListClusters. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationListClusters(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listClusters("myResourceGroup", "myOrganization", "env-12132", 10, null, + com.azure.core.util.Context.NONE); + } +} +``` + +### Organization_ListEnvironments + +```java +/** + * Samples for Organization ListEnvironments. + */ +public final class OrganizationListEnvironmentsSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_EnvironmentList.json + */ + /** + * Sample code: Organization_ListEnvironments. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationListEnvironments(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listEnvironments("myResourceGroup", "myOrganization", 10, null, + com.azure.core.util.Context.NONE); + } +} +``` + +### Organization_ListRegions + +```java +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Organization ListRegions. + */ +public final class OrganizationListRegionsSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_ListRegions. + * json + */ + /** + * Sample code: Organization_ListRegions. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationListRegions(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listRegionsWithResponse("myResourceGroup", "myOrganization", + new ListAccessRequestModel() + .withSearchFilters(mapOf("cloud", "azure", "packages", "ADVANCED,ESSENTIALS", "region", "eastus")), + com.azure.core.util.Context.NONE); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} +``` + +### Organization_ListSchemaRegistryClusters + +```java +/** + * Samples for Organization ListSchemaRegistryClusters. + */ +public final class OrganizationListSchemaRegistryClustersSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_ListSchemaRegistryClusters.json + */ + /** + * Sample code: Organization_ListSchemaRegistryClusters. + * + * @param manager Entry point to ConfluentManager. + */ + public static void + organizationListSchemaRegistryClusters(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listSchemaRegistryClusters("myResourceGroup", "myOrganization", "env-stgcczjp2j3", null, + null, com.azure.core.util.Context.NONE); + } +} +``` + ### Organization_Update ```java @@ -459,7 +869,8 @@ import java.util.Map; */ public final class OrganizationUpdateSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Update.json + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Update.json */ /** * Sample code: Confluent_Update. @@ -467,7 +878,9 @@ public final class OrganizationUpdateSamples { * @param manager Entry point to ConfluentManager. */ public static void confluentUpdate(com.azure.resourcemanager.confluent.ConfluentManager manager) { - OrganizationResource resource = manager.organizations().getByResourceGroupWithResponse("myResourceGroup", "myOrganization", com.azure.core.util.Context.NONE).getValue(); + OrganizationResource resource = manager.organizations() + .getByResourceGroupWithResponse("myResourceGroup", "myOrganization", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withTags(mapOf("client", "dev-client", "env", "dev")).apply(); } @@ -493,7 +906,8 @@ public final class OrganizationUpdateSamples { */ public final class OrganizationOperationsListSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/OrganizationOperations_List.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * OrganizationOperations_List.json */ /** * Sample code: OrganizationOperations_List. @@ -521,7 +935,8 @@ import java.util.Map; */ public final class ValidationsValidateOrganizationSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Validations_ValidateOrganizations.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Validations_ValidateOrganizations.json */ /** * Sample code: Validations_ValidateOrganizations. @@ -529,7 +944,15 @@ public final class ValidationsValidateOrganizationSamples { * @param manager Entry point to ConfluentManager. */ public static void validationsValidateOrganizations(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.validations().validateOrganizationWithResponse("myResourceGroup", "myOrganization", new OrganizationResourceInner().withLocation("West US").withTags(mapOf("Environment", "Dev")).withOfferDetail(new OfferDetail().withPublisherId("string").withId("string").withPlanId("string").withPlanName("string").withTermUnit("string").withPrivateOfferId("string").withPrivateOfferIds(Arrays.asList("string"))).withUserDetail(new UserDetail().withFirstName("string").withLastName("string").withEmailAddress("abc@microsoft.com").withUserPrincipalName("abc@microsoft.com").withAadEmail("abc@microsoft.com")), com.azure.core.util.Context.NONE); + manager.validations().validateOrganizationWithResponse("myResourceGroup", "myOrganization", + new OrganizationResourceInner().withLocation("West US").withTags(mapOf("Environment", "Dev")) + .withOfferDetail(new OfferDetail().withPublisherId("string").withId("string").withPlanId("string") + .withPlanName("string").withTermUnit("string").withPrivateOfferId("string") + .withPrivateOfferIds(Arrays.asList("string"))) + .withUserDetail(new UserDetail().withFirstName("string").withLastName("string") + .withEmailAddress("abc@microsoft.com").withUserPrincipalName("abc@microsoft.com") + .withAadEmail("abc@microsoft.com")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -561,7 +984,8 @@ import java.util.Map; */ public final class ValidationsValidateOrganizationV2Samples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Validations_ValidateOrganizationsV2.json + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Validations_ValidateOrganizationsV2.json */ /** * Sample code: Validations_ValidateOrganizations. @@ -569,7 +993,15 @@ public final class ValidationsValidateOrganizationV2Samples { * @param manager Entry point to ConfluentManager. */ public static void validationsValidateOrganizations(com.azure.resourcemanager.confluent.ConfluentManager manager) { - manager.validations().validateOrganizationV2WithResponse("myResourceGroup", "myOrganization", new OrganizationResourceInner().withLocation("West US").withTags(mapOf("Environment", "Dev")).withOfferDetail(new OfferDetail().withPublisherId("string").withId("string").withPlanId("string").withPlanName("string").withTermUnit("string").withPrivateOfferId("string").withPrivateOfferIds(Arrays.asList("string"))).withUserDetail(new UserDetail().withFirstName("string").withLastName("string").withEmailAddress("abc@microsoft.com").withUserPrincipalName("abc@microsoft.com").withAadEmail("abc@microsoft.com")), com.azure.core.util.Context.NONE); + manager.validations().validateOrganizationV2WithResponse("myResourceGroup", "myOrganization", + new OrganizationResourceInner().withLocation("West US").withTags(mapOf("Environment", "Dev")) + .withOfferDetail(new OfferDetail().withPublisherId("string").withId("string").withPlanId("string") + .withPlanName("string").withTermUnit("string").withPrivateOfferId("string") + .withPrivateOfferIds(Arrays.asList("string"))) + .withUserDetail(new UserDetail().withFirstName("string").withLastName("string") + .withEmailAddress("abc@microsoft.com").withUserPrincipalName("abc@microsoft.com") + .withAadEmail("abc@microsoft.com")), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/pom.xml b/sdk/confluent/azure-resourcemanager-confluent/pom.xml index b7a7a2f242f15..e2f9a03b86c7b 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/pom.xml +++ b/sdk/confluent/azure-resourcemanager-confluent/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-confluent - 1.1.0-beta.1 + 1.2.0-beta.1 jar Microsoft Azure SDK for Confluent Management - This package contains Microsoft Azure SDK for Confluent Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Package tag package-2023-08-22. + This package contains Microsoft Azure SDK for Confluent Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Package tag package-2024-02. https://github.com/Azure/azure-sdk-for-java @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +108,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/ConfluentManager.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/ConfluentManager.java index b102da150f6b6..6a010e94095ef 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/ConfluentManager.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/ConfluentManager.java @@ -215,7 +215,7 @@ public ConfluentManager authenticate(TokenCredential credential, AzureProfile pr StringBuilder userAgentBuilder = new StringBuilder(); userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.confluent").append("/") - .append("1.0.0"); + .append("1.1.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/AccessClient.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/AccessClient.java index 07dca16d422b4..4831093cfb5c4 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/AccessClient.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/AccessClient.java @@ -14,9 +14,12 @@ import com.azure.resourcemanager.confluent.fluent.models.AccessListRoleBindingsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.AccessListServiceAccountsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.AccessListUsersSuccessResponseInner; +import com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.InvitationRecordInner; +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; import com.azure.resourcemanager.confluent.models.AccessInviteUserAccountModel; import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.RoleBindingRecord; /** * An instance of this class provides access to all the operations defined in AccessClient. @@ -25,7 +28,7 @@ public interface AccessClient { /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -41,7 +44,7 @@ Response listUsersWithResponse(String resou /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -56,7 +59,7 @@ AccessListUsersSuccessResponseInner listUsers(String resourceGroupName, String o /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -72,7 +75,7 @@ Response listServiceAccountsWithR /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -87,7 +90,7 @@ AccessListServiceAccountsSuccessResponseInner listServiceAccounts(String resourc /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -103,7 +106,7 @@ Response listInvitationsWithResponse( /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -118,7 +121,7 @@ AccessListInvitationsSuccessResponseInner listInvitations(String resourceGroupNa /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @param context The context to associate with this operation. @@ -134,7 +137,7 @@ Response inviteUserWithResponse(String resourceGroupName, /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -156,7 +159,7 @@ InvitationRecordInner inviteUser(String resourceGroupName, String organizationNa * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response along with {@link Response}. + * @return details of the environments returned on successful response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response listEnvironmentsWithResponse(String resourceGroupName, @@ -171,7 +174,7 @@ Response listEnvironmentsWithRespons * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response. + * @return details of the environments returned on successful response. */ @ServiceMethod(returns = ReturnType.SINGLE) AccessListEnvironmentsSuccessResponseInner listEnvironments(String resourceGroupName, String organizationName, @@ -187,7 +190,7 @@ AccessListEnvironmentsSuccessResponseInner listEnvironments(String resourceGroup * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response}. + * @return details of the clusters returned on successful response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response listClustersWithResponse(String resourceGroupName, @@ -202,7 +205,7 @@ Response listClustersWithResponse(String * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response. + * @return details of the clusters returned on successful response. */ @ServiceMethod(returns = ReturnType.SINGLE) AccessListClusterSuccessResponseInner listClusters(String resourceGroupName, String organizationName, @@ -218,7 +221,7 @@ AccessListClusterSuccessResponseInner listClusters(String resourceGroupName, Str * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response}. + * @return details of the role bindings returned on successful response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response listRoleBindingsWithResponse(String resourceGroupName, @@ -233,9 +236,100 @@ Response listRoleBindingsWithRespons * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response. + * @return details of the role bindings returned on successful response. */ @ServiceMethod(returns = ReturnType.SINGLE) AccessListRoleBindingsSuccessResponseInner listRoleBindings(String resourceGroupName, String organizationName, ListAccessRequestModel body); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createRoleBindingWithResponse(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body, Context context); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + RoleBindingRecord createRoleBinding(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response deleteRoleBindingWithResponse(String resourceGroupName, String organizationName, + String roleBindingId, Context context); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void deleteRoleBinding(String resourceGroupName, String organizationName, String roleBindingId); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listRoleBindingNameListWithResponse( + String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AccessRoleBindingNameListSuccessResponseInner listRoleBindingNameList(String resourceGroupName, + String organizationName, ListAccessRequestModel body); } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ConfluentManagementClient.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ConfluentManagementClient.java index 91e383651b98a..9af539dd74ab7 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ConfluentManagementClient.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ConfluentManagementClient.java @@ -12,7 +12,7 @@ */ public interface ConfluentManagementClient { /** - * Gets The ID of the target subscription. The value must be an UUID. + * Gets Microsoft Azure subscription id. * * @return the subscriptionId value. */ diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/OrganizationsClient.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/OrganizationsClient.java index 72560d3ef97d1..3d1c075f494a7 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/OrganizationsClient.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/OrganizationsClient.java @@ -11,7 +11,14 @@ import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.OrganizationResourceInner; +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.azure.resourcemanager.confluent.models.CreateApiKeyModel; +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; import com.azure.resourcemanager.confluent.models.OrganizationResourceUpdate; /** @@ -43,7 +50,7 @@ public interface OrganizationsClient { /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -55,7 +62,7 @@ public interface OrganizationsClient { /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -68,7 +75,7 @@ public interface OrganizationsClient { /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -83,7 +90,7 @@ Response getByResourceGroupWithResponse(String resour /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -96,7 +103,7 @@ Response getByResourceGroupWithResponse(String resour /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -110,7 +117,7 @@ SyncPoller, OrganizationResourceInner> beg /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -126,7 +133,7 @@ SyncPoller, OrganizationResourceInner> beg /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -139,7 +146,7 @@ SyncPoller, OrganizationResourceInner> beg /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -155,7 +162,7 @@ OrganizationResourceInner create(String resourceGroupName, String organizationNa /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Updated Organization resource. * @param context The context to associate with this operation. @@ -171,7 +178,7 @@ Response updateWithResponse(String resourceGroupName, /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -184,7 +191,7 @@ Response updateWithResponse(String resourceGroupName, /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -197,7 +204,7 @@ Response updateWithResponse(String resourceGroupName, /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -211,7 +218,7 @@ Response updateWithResponse(String resourceGroupName, /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -223,7 +230,7 @@ Response updateWithResponse(String resourceGroupName, /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -232,4 +239,326 @@ Response updateWithResponse(String resourceGroupName, */ @ServiceMethod(returns = ReturnType.SINGLE) void delete(String resourceGroupName, String organizationName, Context context); + + /** + * Lists of all the environments in a organization. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listEnvironments(String resourceGroupName, String organizationName); + + /** + * Lists of all the environments in a organization. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listEnvironments(String resourceGroupName, String organizationName, + Integer pageSize, String pageToken, Context context); + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getEnvironmentByIdWithResponse(String resourceGroupName, String organizationName, + String environmentId, Context context); + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + SCEnvironmentRecordInner getEnvironmentById(String resourceGroupName, String organizationName, + String environmentId); + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId); + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId, Integer pageSize, String pageToken, Context context); + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId); + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken, Context context); + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listRegionsWithResponse(String resourceGroupName, String organizationName, + ListAccessRequestModel body, Context context); + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ListRegionsSuccessResponseInner listRegions(String resourceGroupName, String organizationName, + ListAccessRequestModel body); + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createApiKeyWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, CreateApiKeyModel body, Context context); + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ApiKeyRecordInner createApiKey(String resourceGroupName, String organizationName, String environmentId, + String clusterId, CreateApiKeyModel body); + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response deleteClusterApiKeyWithResponse(String resourceGroupName, String organizationName, String apiKeyId, + Context context); + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void deleteClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId); + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getClusterApiKeyWithResponse(String resourceGroupName, String organizationName, + String apiKeyId, Context context); + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ApiKeyRecordInner getClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId); + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getSchemaRegistryClusterByIdWithResponse(String resourceGroupName, + String organizationName, String environmentId, String clusterId, Context context); + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + SchemaRegistryClusterRecordInner getSchemaRegistryClusterById(String resourceGroupName, String organizationName, + String environmentId, String clusterId); + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getClusterByIdWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, Context context); + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + SCClusterRecordInner getClusterById(String resourceGroupName, String organizationName, String environmentId, + String clusterId); } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ValidationsClient.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ValidationsClient.java index c777fc43838d2..b73562bbd3835 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ValidationsClient.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/ValidationsClient.java @@ -18,7 +18,7 @@ public interface ValidationsClient { /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -34,7 +34,7 @@ Response validateOrganizationWithResponse(String reso /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -49,7 +49,7 @@ OrganizationResourceInner validateOrganization(String resourceGroupName, String /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -65,7 +65,7 @@ Response validateOrganizationV2WithResponse(String reso /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListClusterSuccessResponseInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListClusterSuccessResponseInner.java index 02c3cb93804fa..e1710c526fffd 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListClusterSuccessResponseInner.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListClusterSuccessResponseInner.java @@ -11,7 +11,7 @@ import java.util.List; /** - * List cluster success response. + * Details of the clusters returned on successful response. */ @Fluent public final class AccessListClusterSuccessResponseInner { @@ -28,7 +28,7 @@ public final class AccessListClusterSuccessResponseInner { private ConfluentListMetadata metadata; /* - * Data of the environments list + * List of clusters */ @JsonProperty(value = "data") private List data; @@ -80,7 +80,7 @@ public AccessListClusterSuccessResponseInner withMetadata(ConfluentListMetadata } /** - * Get the data property: Data of the environments list. + * Get the data property: List of clusters. * * @return the data value. */ @@ -89,7 +89,7 @@ public List data() { } /** - * Set the data property: Data of the environments list. + * Set the data property: List of clusters. * * @param data the data value to set. * @return the AccessListClusterSuccessResponseInner object itself. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListEnvironmentsSuccessResponseInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListEnvironmentsSuccessResponseInner.java index e363d104bf875..1770543beb35a 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListEnvironmentsSuccessResponseInner.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListEnvironmentsSuccessResponseInner.java @@ -11,7 +11,7 @@ import java.util.List; /** - * List environments success response. + * Details of the environments returned on successful response. */ @Fluent public final class AccessListEnvironmentsSuccessResponseInner { @@ -22,13 +22,13 @@ public final class AccessListEnvironmentsSuccessResponseInner { private String kind; /* - * Metadata of the list + * Metadata of the environment list */ @JsonProperty(value = "metadata") private ConfluentListMetadata metadata; /* - * Data of the environments list + * Environment list data */ @JsonProperty(value = "data") private List data; @@ -60,7 +60,7 @@ public AccessListEnvironmentsSuccessResponseInner withKind(String kind) { } /** - * Get the metadata property: Metadata of the list. + * Get the metadata property: Metadata of the environment list. * * @return the metadata value. */ @@ -69,7 +69,7 @@ public ConfluentListMetadata metadata() { } /** - * Set the metadata property: Metadata of the list. + * Set the metadata property: Metadata of the environment list. * * @param metadata the metadata value to set. * @return the AccessListEnvironmentsSuccessResponseInner object itself. @@ -80,7 +80,7 @@ public AccessListEnvironmentsSuccessResponseInner withMetadata(ConfluentListMeta } /** - * Get the data property: Data of the environments list. + * Get the data property: Environment list data. * * @return the data value. */ @@ -89,7 +89,7 @@ public List data() { } /** - * Set the data property: Data of the environments list. + * Set the data property: Environment list data. * * @param data the data value to set. * @return the AccessListEnvironmentsSuccessResponseInner object itself. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListRoleBindingsSuccessResponseInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListRoleBindingsSuccessResponseInner.java index 2d564001f1e3a..09552c1458778 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListRoleBindingsSuccessResponseInner.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessListRoleBindingsSuccessResponseInner.java @@ -11,7 +11,7 @@ import java.util.List; /** - * List cluster success response. + * Details of the role bindings returned on successful response. */ @Fluent public final class AccessListRoleBindingsSuccessResponseInner { @@ -28,7 +28,7 @@ public final class AccessListRoleBindingsSuccessResponseInner { private ConfluentListMetadata metadata; /* - * Data of the environments list + * List of role binding */ @JsonProperty(value = "data") private List data; @@ -80,7 +80,7 @@ public AccessListRoleBindingsSuccessResponseInner withMetadata(ConfluentListMeta } /** - * Get the data property: Data of the environments list. + * Get the data property: List of role binding. * * @return the data value. */ @@ -89,7 +89,7 @@ public List data() { } /** - * Set the data property: Data of the environments list. + * Set the data property: List of role binding. * * @param data the data value to set. * @return the AccessListRoleBindingsSuccessResponseInner object itself. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessRoleBindingNameListSuccessResponseInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessRoleBindingNameListSuccessResponseInner.java new file mode 100644 index 0000000000000..2df5ff42b04d5 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/AccessRoleBindingNameListSuccessResponseInner.java @@ -0,0 +1,111 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.ConfluentListMetadata; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Details of the role binding names returned on successful response. + */ +@Fluent +public final class AccessRoleBindingNameListSuccessResponseInner { + /* + * Type of response + */ + @JsonProperty(value = "kind") + private String kind; + + /* + * Metadata of the list + */ + @JsonProperty(value = "metadata") + private ConfluentListMetadata metadata; + + /* + * List of role binding names + */ + @JsonProperty(value = "data") + private List data; + + /** + * Creates an instance of AccessRoleBindingNameListSuccessResponseInner class. + */ + public AccessRoleBindingNameListSuccessResponseInner() { + } + + /** + * Get the kind property: Type of response. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Type of response. + * + * @param kind the kind value to set. + * @return the AccessRoleBindingNameListSuccessResponseInner object itself. + */ + public AccessRoleBindingNameListSuccessResponseInner withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Get the metadata property: Metadata of the list. + * + * @return the metadata value. + */ + public ConfluentListMetadata metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Metadata of the list. + * + * @param metadata the metadata value to set. + * @return the AccessRoleBindingNameListSuccessResponseInner object itself. + */ + public AccessRoleBindingNameListSuccessResponseInner withMetadata(ConfluentListMetadata metadata) { + this.metadata = metadata; + return this; + } + + /** + * Get the data property: List of role binding names. + * + * @return the data value. + */ + public List data() { + return this.data; + } + + /** + * Set the data property: List of role binding names. + * + * @param data the data value to set. + * @return the AccessRoleBindingNameListSuccessResponseInner object itself. + */ + public AccessRoleBindingNameListSuccessResponseInner withData(List data) { + this.data = data; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyProperties.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyProperties.java new file mode 100644 index 0000000000000..de26f7cf7a094 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyProperties.java @@ -0,0 +1,88 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.ApiKeySpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * API Key Properties. + */ +@Fluent +public final class ApiKeyProperties { + /* + * Metadata of the record + */ + @JsonProperty(value = "metadata") + private SCMetadataEntity metadata; + + /* + * Specification of the API Key + */ + @JsonProperty(value = "spec") + private ApiKeySpecEntity spec; + + /** + * Creates an instance of ApiKeyProperties class. + */ + public ApiKeyProperties() { + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the ApiKeyProperties object itself. + */ + public ApiKeyProperties withMetadata(SCMetadataEntity metadata) { + this.metadata = metadata; + return this; + } + + /** + * Get the spec property: Specification of the API Key. + * + * @return the spec value. + */ + public ApiKeySpecEntity spec() { + return this.spec; + } + + /** + * Set the spec property: Specification of the API Key. + * + * @param spec the spec value to set. + * @return the ApiKeyProperties object itself. + */ + public ApiKeyProperties withSpec(ApiKeySpecEntity spec) { + this.spec = spec; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + if (spec() != null) { + spec().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyRecordInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyRecordInner.java new file mode 100644 index 0000000000000..60be809eb5259 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ApiKeyRecordInner.java @@ -0,0 +1,146 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.ApiKeySpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Details API key. + */ +@Fluent +public final class ApiKeyRecordInner { + /* + * Type of api key + */ + @JsonProperty(value = "kind") + private String kind; + + /* + * Id of the api key + */ + @JsonProperty(value = "id") + private String id; + + /* + * API Key Properties + */ + @JsonProperty(value = "properties") + private ApiKeyProperties innerProperties; + + /** + * Creates an instance of ApiKeyRecordInner class. + */ + public ApiKeyRecordInner() { + } + + /** + * Get the kind property: Type of api key. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Type of api key. + * + * @param kind the kind value to set. + * @return the ApiKeyRecordInner object itself. + */ + public ApiKeyRecordInner withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Get the id property: Id of the api key. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Id of the api key. + * + * @param id the id value to set. + * @return the ApiKeyRecordInner object itself. + */ + public ApiKeyRecordInner withId(String id) { + this.id = id; + return this; + } + + /** + * Get the innerProperties property: API Key Properties. + * + * @return the innerProperties value. + */ + private ApiKeyProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.innerProperties() == null ? null : this.innerProperties().metadata(); + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the ApiKeyRecordInner object itself. + */ + public ApiKeyRecordInner withMetadata(SCMetadataEntity metadata) { + if (this.innerProperties() == null) { + this.innerProperties = new ApiKeyProperties(); + } + this.innerProperties().withMetadata(metadata); + return this; + } + + /** + * Get the spec property: Specification of the API Key. + * + * @return the spec value. + */ + public ApiKeySpecEntity spec() { + return this.innerProperties() == null ? null : this.innerProperties().spec(); + } + + /** + * Set the spec property: Specification of the API Key. + * + * @param spec the spec value to set. + * @return the ApiKeyRecordInner object itself. + */ + public ApiKeyRecordInner withSpec(ApiKeySpecEntity spec) { + if (this.innerProperties() == null) { + this.innerProperties = new ApiKeyProperties(); + } + this.innerProperties().withSpec(spec); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ClusterProperties.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ClusterProperties.java new file mode 100644 index 0000000000000..0e00dcebe5131 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ClusterProperties.java @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.ClusterStatusEntity; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Cluster Properties. + */ +@Fluent +public final class ClusterProperties { + /* + * Metadata of the record + */ + @JsonProperty(value = "metadata") + private SCMetadataEntity metadata; + + /* + * Specification of the cluster + */ + @JsonProperty(value = "spec") + private SCClusterSpecEntity spec; + + /* + * Specification of the cluster status + */ + @JsonProperty(value = "status") + private ClusterStatusEntity status; + + /** + * Creates an instance of ClusterProperties class. + */ + public ClusterProperties() { + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the ClusterProperties object itself. + */ + public ClusterProperties withMetadata(SCMetadataEntity metadata) { + this.metadata = metadata; + return this; + } + + /** + * Get the spec property: Specification of the cluster. + * + * @return the spec value. + */ + public SCClusterSpecEntity spec() { + return this.spec; + } + + /** + * Set the spec property: Specification of the cluster. + * + * @param spec the spec value to set. + * @return the ClusterProperties object itself. + */ + public ClusterProperties withSpec(SCClusterSpecEntity spec) { + this.spec = spec; + return this; + } + + /** + * Get the status property: Specification of the cluster status. + * + * @return the status value. + */ + public ClusterStatusEntity status() { + return this.status; + } + + /** + * Set the status property: Specification of the cluster status. + * + * @param status the status value to set. + * @return the ClusterProperties object itself. + */ + public ClusterProperties withStatus(ClusterStatusEntity status) { + this.status = status; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + if (spec() != null) { + spec().validate(); + } + if (status() != null) { + status().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/EnvironmentProperties.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/EnvironmentProperties.java new file mode 100644 index 0000000000000..a9eab745ada81 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/EnvironmentProperties.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Environment resource property. + */ +@Fluent +public final class EnvironmentProperties { + /* + * Metadata of the record + */ + @JsonProperty(value = "metadata") + private SCMetadataEntity metadata; + + /** + * Creates an instance of EnvironmentProperties class. + */ + public EnvironmentProperties() { + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the EnvironmentProperties object itself. + */ + public EnvironmentProperties withMetadata(SCMetadataEntity metadata) { + this.metadata = metadata; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ListRegionsSuccessResponseInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ListRegionsSuccessResponseInner.java new file mode 100644 index 0000000000000..07372836b95b9 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/ListRegionsSuccessResponseInner.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.RegionRecord; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Result of POST request to list regions supported by confluent. + */ +@Fluent +public final class ListRegionsSuccessResponseInner { + /* + * List of regions supported by confluent + */ + @JsonProperty(value = "data") + private List data; + + /** + * Creates an instance of ListRegionsSuccessResponseInner class. + */ + public ListRegionsSuccessResponseInner() { + } + + /** + * Get the data property: List of regions supported by confluent. + * + * @return the data value. + */ + public List data() { + return this.data; + } + + /** + * Set the data property: List of regions supported by confluent. + * + * @param data the data value to set. + * @return the ListRegionsSuccessResponseInner object itself. + */ + public ListRegionsSuccessResponseInner withData(List data) { + this.data = data; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (data() != null) { + data().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/RegionProperties.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/RegionProperties.java new file mode 100644 index 0000000000000..29a777d62833e --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/RegionProperties.java @@ -0,0 +1,88 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.RegionSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Region Properties. + */ +@Fluent +public final class RegionProperties { + /* + * Metadata of the record + */ + @JsonProperty(value = "metadata") + private SCMetadataEntity metadata; + + /* + * Specification of the region + */ + @JsonProperty(value = "spec") + private RegionSpecEntity spec; + + /** + * Creates an instance of RegionProperties class. + */ + public RegionProperties() { + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the RegionProperties object itself. + */ + public RegionProperties withMetadata(SCMetadataEntity metadata) { + this.metadata = metadata; + return this; + } + + /** + * Get the spec property: Specification of the region. + * + * @return the spec value. + */ + public RegionSpecEntity spec() { + return this.spec; + } + + /** + * Set the spec property: Specification of the region. + * + * @param spec the spec value to set. + * @return the RegionProperties object itself. + */ + public RegionProperties withSpec(RegionSpecEntity spec) { + this.spec = spec; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + if (spec() != null) { + spec().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCClusterRecordInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCClusterRecordInner.java new file mode 100644 index 0000000000000..64a66966c358d --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCClusterRecordInner.java @@ -0,0 +1,196 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.ClusterStatusEntity; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Details of cluster record. + */ +@Fluent +public final class SCClusterRecordInner { + /* + * Type of cluster + */ + @JsonProperty(value = "kind") + private String kind; + + /* + * Id of the cluster + */ + @JsonProperty(value = "id") + private String id; + + /* + * Cluster Properties + */ + @JsonProperty(value = "properties") + private ClusterProperties innerProperties; + + /* + * Display name of the cluster + */ + @JsonProperty(value = "name") + private String name; + + /** + * Creates an instance of SCClusterRecordInner class. + */ + public SCClusterRecordInner() { + } + + /** + * Get the kind property: Type of cluster. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Type of cluster. + * + * @param kind the kind value to set. + * @return the SCClusterRecordInner object itself. + */ + public SCClusterRecordInner withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Get the id property: Id of the cluster. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Id of the cluster. + * + * @param id the id value to set. + * @return the SCClusterRecordInner object itself. + */ + public SCClusterRecordInner withId(String id) { + this.id = id; + return this; + } + + /** + * Get the innerProperties property: Cluster Properties. + * + * @return the innerProperties value. + */ + private ClusterProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the name property: Display name of the cluster. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Display name of the cluster. + * + * @param name the name value to set. + * @return the SCClusterRecordInner object itself. + */ + public SCClusterRecordInner withName(String name) { + this.name = name; + return this; + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.innerProperties() == null ? null : this.innerProperties().metadata(); + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the SCClusterRecordInner object itself. + */ + public SCClusterRecordInner withMetadata(SCMetadataEntity metadata) { + if (this.innerProperties() == null) { + this.innerProperties = new ClusterProperties(); + } + this.innerProperties().withMetadata(metadata); + return this; + } + + /** + * Get the spec property: Specification of the cluster. + * + * @return the spec value. + */ + public SCClusterSpecEntity spec() { + return this.innerProperties() == null ? null : this.innerProperties().spec(); + } + + /** + * Set the spec property: Specification of the cluster. + * + * @param spec the spec value to set. + * @return the SCClusterRecordInner object itself. + */ + public SCClusterRecordInner withSpec(SCClusterSpecEntity spec) { + if (this.innerProperties() == null) { + this.innerProperties = new ClusterProperties(); + } + this.innerProperties().withSpec(spec); + return this; + } + + /** + * Get the status property: Specification of the cluster status. + * + * @return the status value. + */ + public ClusterStatusEntity status() { + return this.innerProperties() == null ? null : this.innerProperties().status(); + } + + /** + * Set the status property: Specification of the cluster status. + * + * @param status the status value to set. + * @return the SCClusterRecordInner object itself. + */ + public SCClusterRecordInner withStatus(ClusterStatusEntity status) { + if (this.innerProperties() == null) { + this.innerProperties = new ClusterProperties(); + } + this.innerProperties().withStatus(status); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCEnvironmentRecordInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCEnvironmentRecordInner.java new file mode 100644 index 0000000000000..3677b3958a0fd --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SCEnvironmentRecordInner.java @@ -0,0 +1,148 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Details about environment name, metadata and environment id of an environment. + */ +@Fluent +public final class SCEnvironmentRecordInner { + /* + * Type of environment + */ + @JsonProperty(value = "kind") + private String kind; + + /* + * Id of the environment + */ + @JsonProperty(value = "id") + private String id; + + /* + * Environment properties + */ + @JsonProperty(value = "properties") + private EnvironmentProperties innerProperties; + + /* + * Display name of the environment + */ + @JsonProperty(value = "name") + private String name; + + /** + * Creates an instance of SCEnvironmentRecordInner class. + */ + public SCEnvironmentRecordInner() { + } + + /** + * Get the kind property: Type of environment. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Type of environment. + * + * @param kind the kind value to set. + * @return the SCEnvironmentRecordInner object itself. + */ + public SCEnvironmentRecordInner withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Get the id property: Id of the environment. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Id of the environment. + * + * @param id the id value to set. + * @return the SCEnvironmentRecordInner object itself. + */ + public SCEnvironmentRecordInner withId(String id) { + this.id = id; + return this; + } + + /** + * Get the innerProperties property: Environment properties. + * + * @return the innerProperties value. + */ + private EnvironmentProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the name property: Display name of the environment. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Display name of the environment. + * + * @param name the name value to set. + * @return the SCEnvironmentRecordInner object itself. + */ + public SCEnvironmentRecordInner withName(String name) { + this.name = name; + return this; + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.innerProperties() == null ? null : this.innerProperties().metadata(); + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the SCEnvironmentRecordInner object itself. + */ + public SCEnvironmentRecordInner withMetadata(SCMetadataEntity metadata) { + if (this.innerProperties() == null) { + this.innerProperties = new EnvironmentProperties(); + } + this.innerProperties().withMetadata(metadata); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterProperties.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterProperties.java new file mode 100644 index 0000000000000..fec259686dba9 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterProperties.java @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Schema Registry Cluster Properties. + */ +@Fluent +public final class SchemaRegistryClusterProperties { + /* + * Metadata of the record + */ + @JsonProperty(value = "metadata") + private SCMetadataEntity metadata; + + /* + * Specification of the schema registry cluster + */ + @JsonProperty(value = "spec") + private SchemaRegistryClusterSpecEntity spec; + + /* + * Specification of the cluster status + */ + @JsonProperty(value = "status") + private SchemaRegistryClusterStatusEntity status; + + /** + * Creates an instance of SchemaRegistryClusterProperties class. + */ + public SchemaRegistryClusterProperties() { + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.metadata; + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the SchemaRegistryClusterProperties object itself. + */ + public SchemaRegistryClusterProperties withMetadata(SCMetadataEntity metadata) { + this.metadata = metadata; + return this; + } + + /** + * Get the spec property: Specification of the schema registry cluster. + * + * @return the spec value. + */ + public SchemaRegistryClusterSpecEntity spec() { + return this.spec; + } + + /** + * Set the spec property: Specification of the schema registry cluster. + * + * @param spec the spec value to set. + * @return the SchemaRegistryClusterProperties object itself. + */ + public SchemaRegistryClusterProperties withSpec(SchemaRegistryClusterSpecEntity spec) { + this.spec = spec; + return this; + } + + /** + * Get the status property: Specification of the cluster status. + * + * @return the status value. + */ + public SchemaRegistryClusterStatusEntity status() { + return this.status; + } + + /** + * Set the status property: Specification of the cluster status. + * + * @param status the status value to set. + * @return the SchemaRegistryClusterProperties object itself. + */ + public SchemaRegistryClusterProperties withStatus(SchemaRegistryClusterStatusEntity status) { + this.status = status; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (metadata() != null) { + metadata().validate(); + } + if (spec() != null) { + spec().validate(); + } + if (status() != null) { + status().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterRecordInner.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterRecordInner.java new file mode 100644 index 0000000000000..46cac907678fb --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/fluent/models/SchemaRegistryClusterRecordInner.java @@ -0,0 +1,170 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Details of schema registry cluster record. + */ +@Fluent +public final class SchemaRegistryClusterRecordInner { + /* + * Kind of the cluster + */ + @JsonProperty(value = "kind") + private String kind; + + /* + * Id of the cluster + */ + @JsonProperty(value = "id") + private String id; + + /* + * Schema Registry Cluster Properties + */ + @JsonProperty(value = "properties") + private SchemaRegistryClusterProperties innerProperties; + + /** + * Creates an instance of SchemaRegistryClusterRecordInner class. + */ + public SchemaRegistryClusterRecordInner() { + } + + /** + * Get the kind property: Kind of the cluster. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Kind of the cluster. + * + * @param kind the kind value to set. + * @return the SchemaRegistryClusterRecordInner object itself. + */ + public SchemaRegistryClusterRecordInner withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Get the id property: Id of the cluster. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Id of the cluster. + * + * @param id the id value to set. + * @return the SchemaRegistryClusterRecordInner object itself. + */ + public SchemaRegistryClusterRecordInner withId(String id) { + this.id = id; + return this; + } + + /** + * Get the innerProperties property: Schema Registry Cluster Properties. + * + * @return the innerProperties value. + */ + private SchemaRegistryClusterProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.innerProperties() == null ? null : this.innerProperties().metadata(); + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the SchemaRegistryClusterRecordInner object itself. + */ + public SchemaRegistryClusterRecordInner withMetadata(SCMetadataEntity metadata) { + if (this.innerProperties() == null) { + this.innerProperties = new SchemaRegistryClusterProperties(); + } + this.innerProperties().withMetadata(metadata); + return this; + } + + /** + * Get the spec property: Specification of the schema registry cluster. + * + * @return the spec value. + */ + public SchemaRegistryClusterSpecEntity spec() { + return this.innerProperties() == null ? null : this.innerProperties().spec(); + } + + /** + * Set the spec property: Specification of the schema registry cluster. + * + * @param spec the spec value to set. + * @return the SchemaRegistryClusterRecordInner object itself. + */ + public SchemaRegistryClusterRecordInner withSpec(SchemaRegistryClusterSpecEntity spec) { + if (this.innerProperties() == null) { + this.innerProperties = new SchemaRegistryClusterProperties(); + } + this.innerProperties().withSpec(spec); + return this; + } + + /** + * Get the status property: Specification of the cluster status. + * + * @return the status value. + */ + public SchemaRegistryClusterStatusEntity status() { + return this.innerProperties() == null ? null : this.innerProperties().status(); + } + + /** + * Set the status property: Specification of the cluster status. + * + * @param status the status value to set. + * @return the SchemaRegistryClusterRecordInner object itself. + */ + public SchemaRegistryClusterRecordInner withStatus(SchemaRegistryClusterStatusEntity status) { + if (this.innerProperties() == null) { + this.innerProperties = new SchemaRegistryClusterProperties(); + } + this.innerProperties().withStatus(status); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessClientImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessClientImpl.java index 95605b5fe0d27..3b4b31c56b514 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessClientImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessClientImpl.java @@ -5,6 +5,7 @@ package com.azure.resourcemanager.confluent.implementation; import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; import com.azure.core.annotation.ExpectedResponses; import com.azure.core.annotation.HeaderParam; import com.azure.core.annotation.Headers; @@ -29,9 +30,12 @@ import com.azure.resourcemanager.confluent.fluent.models.AccessListRoleBindingsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.AccessListServiceAccountsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.AccessListUsersSuccessResponseInner; +import com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.InvitationRecordInner; +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; import com.azure.resourcemanager.confluent.models.AccessInviteUserAccountModel; import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.RoleBindingRecord; import reactor.core.publisher.Mono; /** @@ -142,12 +146,45 @@ Mono> listRoleBindings(@Hos @PathParam("organizationName") String organizationName, @BodyParam("application/json") ListAccessRequestModel body, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/access/default/createRoleBinding") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> createRoleBinding(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, + @BodyParam("application/json") AccessCreateRoleBindingRequestModel body, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/access/default/deleteRoleBinding/{roleBindingId}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> deleteRoleBinding(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("roleBindingId") String roleBindingId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/access/default/listRoleBindingNameList") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listRoleBindingNameList( + @HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, + @BodyParam("application/json") ListAccessRequestModel body, @HeaderParam("Accept") String accept, + Context context); } /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -189,7 +226,7 @@ private Mono> listUsersWithRespons /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -231,7 +268,7 @@ private Mono> listUsersWithRespons /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -249,7 +286,7 @@ private Mono listUsersAsync(String resource /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -267,7 +304,7 @@ public Response listUsersWithResponse(Strin /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -284,7 +321,7 @@ public AccessListUsersSuccessResponseInner listUsers(String resourceGroupName, S /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -327,7 +364,7 @@ private Mono> listServic /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -370,7 +407,7 @@ private Mono> listServic /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -388,7 +425,7 @@ private Mono listServiceAccountsA /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -406,7 +443,7 @@ public Response listServiceAccoun /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -423,7 +460,7 @@ public AccessListServiceAccountsSuccessResponseInner listServiceAccounts(String /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -465,7 +502,7 @@ private Mono> listInvitation /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -507,7 +544,7 @@ private Mono> listInvitation /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -525,7 +562,7 @@ private Mono listInvitationsAsync(Str /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -543,7 +580,7 @@ public Response listInvitationsWithRe /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -560,7 +597,7 @@ public AccessListInvitationsSuccessResponseInner listInvitations(String resource /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -602,7 +639,7 @@ private Mono> inviteUserWithResponseAsync(String /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @param context The context to associate with this operation. @@ -644,7 +681,7 @@ private Mono> inviteUserWithResponseAsync(String /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -662,7 +699,7 @@ private Mono inviteUserAsync(String resourceGroupName, St /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @param context The context to associate with this operation. @@ -680,7 +717,7 @@ public Response inviteUserWithResponse(String resourceGro /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -703,7 +740,8 @@ public InvitationRecordInner inviteUser(String resourceGroupName, String organiz * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response along with {@link Response} on successful completion of {@link Mono}. + * @return details of the environments returned on successful response along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listEnvironmentsWithResponseAsync( @@ -746,7 +784,8 @@ private Mono> listEnvironme * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response along with {@link Response} on successful completion of {@link Mono}. + * @return details of the environments returned on successful response along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listEnvironmentsWithResponseAsync( @@ -787,7 +826,7 @@ private Mono> listEnvironme * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response on successful completion of {@link Mono}. + * @return details of the environments returned on successful response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono listEnvironmentsAsync(String resourceGroupName, @@ -806,7 +845,7 @@ private Mono listEnvironmentsAsync(S * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response along with {@link Response}. + * @return details of the environments returned on successful response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response listEnvironmentsWithResponse(String resourceGroupName, @@ -823,7 +862,7 @@ public Response listEnvironmentsWith * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response. + * @return details of the environments returned on successful response. */ @ServiceMethod(returns = ReturnType.SINGLE) public AccessListEnvironmentsSuccessResponseInner listEnvironments(String resourceGroupName, @@ -840,7 +879,8 @@ public AccessListEnvironmentsSuccessResponseInner listEnvironments(String resour * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response} on successful completion of {@link Mono}. + * @return details of the clusters returned on successful response along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> @@ -883,7 +923,8 @@ public AccessListEnvironmentsSuccessResponseInner listEnvironments(String resour * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response} on successful completion of {@link Mono}. + * @return details of the clusters returned on successful response along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listClustersWithResponseAsync( @@ -924,7 +965,7 @@ private Mono> listClustersWithRe * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response on successful completion of {@link Mono}. + * @return details of the clusters returned on successful response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono listClustersAsync(String resourceGroupName, @@ -943,7 +984,7 @@ private Mono listClustersAsync(String res * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response}. + * @return details of the clusters returned on successful response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response listClustersWithResponse(String resourceGroupName, @@ -960,7 +1001,7 @@ public Response listClustersWithResponse( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response. + * @return details of the clusters returned on successful response. */ @ServiceMethod(returns = ReturnType.SINGLE) public AccessListClusterSuccessResponseInner listClusters(String resourceGroupName, String organizationName, @@ -977,7 +1018,8 @@ public AccessListClusterSuccessResponseInner listClusters(String resourceGroupNa * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response} on successful completion of {@link Mono}. + * @return details of the role bindings returned on successful response along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listRoleBindingsWithResponseAsync( @@ -1020,7 +1062,8 @@ private Mono> listRoleBindi * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response} on successful completion of {@link Mono}. + * @return details of the role bindings returned on successful response along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listRoleBindingsWithResponseAsync( @@ -1061,7 +1104,7 @@ private Mono> listRoleBindi * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response on successful completion of {@link Mono}. + * @return details of the role bindings returned on successful response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono listRoleBindingsAsync(String resourceGroupName, @@ -1080,7 +1123,7 @@ private Mono listRoleBindingsAsync(S * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response}. + * @return details of the role bindings returned on successful response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response listRoleBindingsWithResponse(String resourceGroupName, @@ -1097,11 +1140,422 @@ public Response listRoleBindingsWith * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response. + * @return details of the role bindings returned on successful response. */ @ServiceMethod(returns = ReturnType.SINGLE) public AccessListRoleBindingsSuccessResponseInner listRoleBindings(String resourceGroupName, String organizationName, ListAccessRequestModel body) { return listRoleBindingsWithResponse(resourceGroupName, organizationName, body, Context.NONE).getValue(); } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createRoleBindingWithResponseAsync(String resourceGroupName, + String organizationName, AccessCreateRoleBindingRequestModel body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createRoleBinding(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, body, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createRoleBindingWithResponseAsync(String resourceGroupName, + String organizationName, AccessCreateRoleBindingRequestModel body, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createRoleBinding(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, body, accept, context); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createRoleBindingAsync(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body) { + return createRoleBindingWithResponseAsync(resourceGroupName, organizationName, body) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createRoleBindingWithResponse(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body, Context context) { + return createRoleBindingWithResponseAsync(resourceGroupName, organizationName, body, context).block(); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public RoleBindingRecord createRoleBinding(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body) { + return createRoleBindingWithResponse(resourceGroupName, organizationName, body, Context.NONE).getValue(); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteRoleBindingWithResponseAsync(String resourceGroupName, String organizationName, + String roleBindingId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (roleBindingId == null) { + return Mono.error(new IllegalArgumentException("Parameter roleBindingId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.deleteRoleBinding(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, roleBindingId, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteRoleBindingWithResponseAsync(String resourceGroupName, String organizationName, + String roleBindingId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (roleBindingId == null) { + return Mono.error(new IllegalArgumentException("Parameter roleBindingId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.deleteRoleBinding(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, roleBindingId, accept, context); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteRoleBindingAsync(String resourceGroupName, String organizationName, String roleBindingId) { + return deleteRoleBindingWithResponseAsync(resourceGroupName, organizationName, roleBindingId) + .flatMap(ignored -> Mono.empty()); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteRoleBindingWithResponse(String resourceGroupName, String organizationName, + String roleBindingId, Context context) { + return deleteRoleBindingWithResponseAsync(resourceGroupName, organizationName, roleBindingId, context).block(); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void deleteRoleBinding(String resourceGroupName, String organizationName, String roleBindingId) { + deleteRoleBindingWithResponse(resourceGroupName, organizationName, roleBindingId, Context.NONE); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listRoleBindingNameListWithResponseAsync( + String resourceGroupName, String organizationName, ListAccessRequestModel body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listRoleBindingNameList(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, body, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listRoleBindingNameListWithResponseAsync( + String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listRoleBindingNameList(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, body, accept, context); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listRoleBindingNameListAsync(String resourceGroupName, + String organizationName, ListAccessRequestModel body) { + return listRoleBindingNameListWithResponseAsync(resourceGroupName, organizationName, body) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listRoleBindingNameListWithResponse( + String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context) { + return listRoleBindingNameListWithResponseAsync(resourceGroupName, organizationName, body, context).block(); + } + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AccessRoleBindingNameListSuccessResponseInner listRoleBindingNameList(String resourceGroupName, + String organizationName, ListAccessRequestModel body) { + return listRoleBindingNameListWithResponse(resourceGroupName, organizationName, body, Context.NONE).getValue(); + } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessImpl.java index c00be662e1a4a..5ac1b1f6f2c3b 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessImpl.java @@ -15,8 +15,10 @@ import com.azure.resourcemanager.confluent.fluent.models.AccessListRoleBindingsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.AccessListServiceAccountsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.AccessListUsersSuccessResponseInner; +import com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.InvitationRecordInner; import com.azure.resourcemanager.confluent.models.Access; +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; import com.azure.resourcemanager.confluent.models.AccessInviteUserAccountModel; import com.azure.resourcemanager.confluent.models.AccessListClusterSuccessResponse; import com.azure.resourcemanager.confluent.models.AccessListEnvironmentsSuccessResponse; @@ -24,8 +26,10 @@ import com.azure.resourcemanager.confluent.models.AccessListRoleBindingsSuccessResponse; import com.azure.resourcemanager.confluent.models.AccessListServiceAccountsSuccessResponse; import com.azure.resourcemanager.confluent.models.AccessListUsersSuccessResponse; +import com.azure.resourcemanager.confluent.models.AccessRoleBindingNameListSuccessResponse; import com.azure.resourcemanager.confluent.models.InvitationRecord; import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.RoleBindingRecord; public final class AccessImpl implements Access { private static final ClientLogger LOGGER = new ClientLogger(AccessImpl.class); @@ -199,6 +203,49 @@ public AccessListRoleBindingsSuccessResponse listRoleBindings(String resourceGro } } + public Response createRoleBindingWithResponse(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body, Context context) { + return this.serviceClient().createRoleBindingWithResponse(resourceGroupName, organizationName, body, context); + } + + public RoleBindingRecord createRoleBinding(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body) { + return this.serviceClient().createRoleBinding(resourceGroupName, organizationName, body); + } + + public Response deleteRoleBindingWithResponse(String resourceGroupName, String organizationName, + String roleBindingId, Context context) { + return this.serviceClient().deleteRoleBindingWithResponse(resourceGroupName, organizationName, roleBindingId, + context); + } + + public void deleteRoleBinding(String resourceGroupName, String organizationName, String roleBindingId) { + this.serviceClient().deleteRoleBinding(resourceGroupName, organizationName, roleBindingId); + } + + public Response listRoleBindingNameListWithResponse( + String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context) { + Response inner = this.serviceClient() + .listRoleBindingNameListWithResponse(resourceGroupName, organizationName, body, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new AccessRoleBindingNameListSuccessResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public AccessRoleBindingNameListSuccessResponse listRoleBindingNameList(String resourceGroupName, + String organizationName, ListAccessRequestModel body) { + AccessRoleBindingNameListSuccessResponseInner inner + = this.serviceClient().listRoleBindingNameList(resourceGroupName, organizationName, body); + if (inner != null) { + return new AccessRoleBindingNameListSuccessResponseImpl(inner, this.manager()); + } else { + return null; + } + } + private AccessClient serviceClient() { return this.innerClient; } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessRoleBindingNameListSuccessResponseImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessRoleBindingNameListSuccessResponseImpl.java new file mode 100644 index 0000000000000..7401b0de29d00 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/AccessRoleBindingNameListSuccessResponseImpl.java @@ -0,0 +1,48 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.implementation; + +import com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner; +import com.azure.resourcemanager.confluent.models.AccessRoleBindingNameListSuccessResponse; +import com.azure.resourcemanager.confluent.models.ConfluentListMetadata; +import java.util.Collections; +import java.util.List; + +public final class AccessRoleBindingNameListSuccessResponseImpl implements AccessRoleBindingNameListSuccessResponse { + private AccessRoleBindingNameListSuccessResponseInner innerObject; + + private final com.azure.resourcemanager.confluent.ConfluentManager serviceManager; + + AccessRoleBindingNameListSuccessResponseImpl(AccessRoleBindingNameListSuccessResponseInner innerObject, + com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String kind() { + return this.innerModel().kind(); + } + + public ConfluentListMetadata metadata() { + return this.innerModel().metadata(); + } + + public List data() { + List inner = this.innerModel().data(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public AccessRoleBindingNameListSuccessResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.confluent.ConfluentManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ApiKeyRecordImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ApiKeyRecordImpl.java new file mode 100644 index 0000000000000..6ef762605edde --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ApiKeyRecordImpl.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.implementation; + +import com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner; +import com.azure.resourcemanager.confluent.models.ApiKeyRecord; +import com.azure.resourcemanager.confluent.models.ApiKeySpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; + +public final class ApiKeyRecordImpl implements ApiKeyRecord { + private ApiKeyRecordInner innerObject; + + private final com.azure.resourcemanager.confluent.ConfluentManager serviceManager; + + ApiKeyRecordImpl(ApiKeyRecordInner innerObject, + com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String kind() { + return this.innerModel().kind(); + } + + public String id() { + return this.innerModel().id(); + } + + public SCMetadataEntity metadata() { + return this.innerModel().metadata(); + } + + public ApiKeySpecEntity spec() { + return this.innerModel().spec(); + } + + public ApiKeyRecordInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.confluent.ConfluentManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientBuilder.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientBuilder.java index 210846f47877a..eb95366576716 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientBuilder.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientBuilder.java @@ -20,12 +20,12 @@ @ServiceClientBuilder(serviceClients = { ConfluentManagementClientImpl.class }) public final class ConfluentManagementClientBuilder { /* - * The ID of the target subscription. The value must be an UUID. + * Microsoft Azure subscription id */ private String subscriptionId; /** - * Sets The ID of the target subscription. The value must be an UUID. + * Sets Microsoft Azure subscription id. * * @param subscriptionId the subscriptionId value. * @return the ConfluentManagementClientBuilder. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientImpl.java index b8c3ff539f42c..0dfbc97f22b5a 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ConfluentManagementClientImpl.java @@ -43,12 +43,12 @@ @ServiceClient(builder = ConfluentManagementClientBuilder.class) public final class ConfluentManagementClientImpl implements ConfluentManagementClient { /** - * The ID of the target subscription. The value must be an UUID. + * Microsoft Azure subscription id. */ private final String subscriptionId; /** - * Gets The ID of the target subscription. The value must be an UUID. + * Gets Microsoft Azure subscription id. * * @return the subscriptionId value. */ @@ -203,7 +203,7 @@ public AccessClient getAccess() { * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. * @param environment The Azure environment. - * @param subscriptionId The ID of the target subscription. The value must be an UUID. + * @param subscriptionId Microsoft Azure subscription id. * @param endpoint server parameter. */ ConfluentManagementClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, @@ -213,7 +213,7 @@ public AccessClient getAccess() { this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-08-22"; + this.apiVersion = "2024-02-13"; this.marketplaceAgreements = new MarketplaceAgreementsClientImpl(this); this.organizationOperations = new OrganizationOperationsClientImpl(this); this.organizations = new OrganizationsClientImpl(this); diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ListRegionsSuccessResponseImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ListRegionsSuccessResponseImpl.java new file mode 100644 index 0000000000000..8c98694adbbc2 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ListRegionsSuccessResponseImpl.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.implementation; + +import com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner; +import com.azure.resourcemanager.confluent.models.ListRegionsSuccessResponse; +import com.azure.resourcemanager.confluent.models.RegionRecord; +import java.util.Collections; +import java.util.List; + +public final class ListRegionsSuccessResponseImpl implements ListRegionsSuccessResponse { + private ListRegionsSuccessResponseInner innerObject; + + private final com.azure.resourcemanager.confluent.ConfluentManager serviceManager; + + ListRegionsSuccessResponseImpl(ListRegionsSuccessResponseInner innerObject, + com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public List data() { + List inner = this.innerModel().data(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public ListRegionsSuccessResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.confluent.ConfluentManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/MarketplaceAgreementsImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/MarketplaceAgreementsImpl.java index 6c779e9687f96..de73f898a3d8c 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/MarketplaceAgreementsImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/MarketplaceAgreementsImpl.java @@ -29,12 +29,14 @@ public MarketplaceAgreementsImpl(MarketplaceAgreementsClient innerClient, public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ConfluentAgreementResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new ConfluentAgreementResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ConfluentAgreementResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new ConfluentAgreementResourceImpl(inner1, this.manager())); } public Response createWithResponse(ConfluentAgreementResourceInner body, diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationOperationsImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationOperationsImpl.java index 0522091727eb5..f5d39009e232e 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationOperationsImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationOperationsImpl.java @@ -27,12 +27,12 @@ public OrganizationOperationsImpl(OrganizationOperationsClient innerClient, public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationResultImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationResultImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationResultImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationResultImpl(inner1, this.manager())); } private OrganizationOperationsClient serviceClient() { diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationResourceImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationResourceImpl.java index 5e999c3e2cade..4a4c2c0e2b2e7 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationResourceImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationResourceImpl.java @@ -4,11 +4,14 @@ package com.azure.resourcemanager.confluent.implementation; +import com.azure.core.http.rest.Response; import com.azure.core.management.Region; import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.confluent.fluent.models.OrganizationResourceInner; import com.azure.resourcemanager.confluent.models.LinkOrganization; +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.ListRegionsSuccessResponse; import com.azure.resourcemanager.confluent.models.OfferDetail; import com.azure.resourcemanager.confluent.models.OrganizationResource; import com.azure.resourcemanager.confluent.models.OrganizationResourceUpdate; @@ -151,8 +154,8 @@ public OrganizationResource apply(Context context) { com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.organizationName = Utils.getValueFromIdByName(innerObject.id(), "organizations"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.organizationName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "organizations"); } public OrganizationResource refresh() { @@ -167,6 +170,15 @@ public OrganizationResource refresh(Context context) { return this; } + public Response listRegionsWithResponse(ListAccessRequestModel body, Context context) { + return serviceManager.organizations().listRegionsWithResponse(resourceGroupName, organizationName, body, + context); + } + + public ListRegionsSuccessResponse listRegions(ListAccessRequestModel body) { + return serviceManager.organizations().listRegions(resourceGroupName, organizationName, body); + } + public OrganizationResourceImpl withRegion(Region location) { this.innerModel().withLocation(location.toString()); return this; diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsClientImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsClientImpl.java index 3d6742c3946dd..5857f58d2f182 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsClientImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsClientImpl.java @@ -14,6 +14,7 @@ import com.azure.core.annotation.HostParam; import com.azure.core.annotation.Patch; import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; import com.azure.core.annotation.Put; import com.azure.core.annotation.QueryParam; import com.azure.core.annotation.ReturnType; @@ -33,7 +34,17 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.confluent.fluent.OrganizationsClient; +import com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.OrganizationResourceInner; +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.azure.resourcemanager.confluent.models.CreateApiKeyModel; +import com.azure.resourcemanager.confluent.models.GetEnvironmentsResponse; +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.ListClustersSuccessResponse; +import com.azure.resourcemanager.confluent.models.ListSchemaRegistryClustersResponse; import com.azure.resourcemanager.confluent.models.OrganizationResourceListResult; import com.azure.resourcemanager.confluent.models.OrganizationResourceUpdate; import java.nio.ByteBuffer; @@ -131,6 +142,112 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("organizationName") String organizationName, @HeaderParam("Accept") String accept, Context context); + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listEnvironments(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @QueryParam("pageSize") Integer pageSize, + @QueryParam("pageToken") String pageToken, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments/{environmentId}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getEnvironmentById(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("environmentId") String environmentId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments/{environmentId}/clusters") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listClusters(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("environmentId") String environmentId, + @QueryParam("pageSize") Integer pageSize, @QueryParam("pageToken") String pageToken, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments/{environmentId}/schemaRegistryClusters") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listSchemaRegistryClusters( + @HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("environmentId") String environmentId, + @QueryParam("pageSize") Integer pageSize, @QueryParam("pageToken") String pageToken, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/listRegions") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listRegions(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, + @BodyParam("application/json") ListAccessRequestModel body, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments/{environmentId}/clusters/{clusterId}/createAPIKey") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> createApiKey(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("environmentId") String environmentId, + @PathParam("clusterId") String clusterId, @BodyParam("application/json") CreateApiKeyModel body, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/apiKeys/{apiKeyId}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> deleteClusterApiKey(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("apiKeyId") String apiKeyId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/apiKeys/{apiKeyId}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getClusterApiKey(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("apiKeyId") String apiKeyId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments/{environmentId}/schemaRegistryClusters/{clusterId}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getSchemaRegistryClusterById( + @HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("environmentId") String environmentId, + @PathParam("clusterId") String clusterId, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Confluent/organizations/{organizationName}/environments/{environmentId}/clusters/{clusterId}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getClusterById(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("organizationName") String organizationName, @PathParam("environmentId") String environmentId, + @PathParam("clusterId") String clusterId, @HeaderParam("Accept") String accept, Context context); + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") @ExpectedResponses({ 200 }) @@ -146,6 +263,30 @@ Mono> listBySubscriptionNext( Mono> listByResourceGroupNext( @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listEnvironmentsNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listClustersNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listSchemaRegistryClustersNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** @@ -261,7 +402,7 @@ public PagedIterable list(Context context) { /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -295,7 +436,7 @@ public PagedIterable list(Context context) { /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -330,7 +471,7 @@ private Mono> listByResourceGroupSingle /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -345,7 +486,7 @@ private PagedFlux listByResourceGroupAsync(String res /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -361,7 +502,7 @@ private PagedFlux listByResourceGroupAsync(String res /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -375,7 +516,7 @@ public PagedIterable listByResourceGroup(String resou /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -390,7 +531,7 @@ public PagedIterable listByResourceGroup(String resou /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -427,7 +568,7 @@ private Mono> getByResourceGroupWithResponse /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -464,7 +605,7 @@ private Mono> getByResourceGroupWithResponse /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -480,7 +621,7 @@ private Mono getByResourceGroupAsync(String resourceG /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -497,7 +638,7 @@ public Response getByResourceGroupWithResponse(String /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -512,7 +653,7 @@ public OrganizationResourceInner getByResourceGroup(String resourceGroupName, St /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -552,7 +693,7 @@ private Mono>> createWithResponseAsync(String resource /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -592,7 +733,7 @@ private Mono>> createWithResponseAsync(String resource /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -612,7 +753,7 @@ private Mono>> createWithResponseAsync(String resource /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -632,7 +773,7 @@ private Mono>> createWithResponseAsync(String resource /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -654,7 +795,7 @@ private PollerFlux, OrganizationResourceIn /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -671,7 +812,7 @@ private PollerFlux, OrganizationResourceIn /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -689,7 +830,7 @@ public SyncPoller, OrganizationResourceInn /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -707,7 +848,7 @@ private Mono createAsync(String resourceGroupName, St /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -724,7 +865,7 @@ private Mono createAsync(String resourceGroupName, St /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -743,7 +884,7 @@ private Mono createAsync(String resourceGroupName, St /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -759,7 +900,7 @@ public OrganizationResourceInner create(String resourceGroupName, String organiz /** * Create Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -777,7 +918,7 @@ public OrganizationResourceInner create(String resourceGroupName, String organiz /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Updated Organization resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -817,7 +958,7 @@ private Mono> updateWithResponseAsync(String /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Updated Organization resource. * @param context The context to associate with this operation. @@ -857,7 +998,7 @@ private Mono> updateWithResponseAsync(String /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -874,7 +1015,7 @@ private Mono updateAsync(String resourceGroupName, St /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Updated Organization resource. * @param context The context to associate with this operation. @@ -892,7 +1033,7 @@ public Response updateWithResponse(String resourceGro /** * Update Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -908,7 +1049,7 @@ public OrganizationResourceInner update(String resourceGroupName, String organiz /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -944,7 +1085,7 @@ private Mono>> deleteWithResponseAsync(String resource /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -980,7 +1121,7 @@ private Mono>> deleteWithResponseAsync(String resource /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -997,7 +1138,7 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1017,7 +1158,7 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1032,7 +1173,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1049,7 +1190,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1065,7 +1206,7 @@ private Mono deleteAsync(String resourceGroupName, String organizationName /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1082,7 +1223,7 @@ private Mono deleteAsync(String resourceGroupName, String organizationName /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1096,7 +1237,7 @@ public void delete(String resourceGroupName, String organizationName) { /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1109,113 +1250,1695 @@ public void delete(String resourceGroupName, String organizationName, Context co } /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items + * Lists of all the environments in a organization. * - * The nextLink parameter. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * @return result of GET request to list Confluent operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } + private Mono> listEnvironmentsSinglePageAsync(String resourceGroupName, + String organizationName, Integer pageSize, String pageToken) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } final String accept = "application/json"; return FluxUtil - .withContext( - context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), + .withContext(context -> service.listEnvironments(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, pageSize, pageToken, accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Get the next page of items. + * Lists of all the environments in a organization. * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * @return result of GET request to list Confluent operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, - Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } + private Mono> listEnvironmentsSinglePageAsync(String resourceGroupName, + String organizationName, Integer pageSize, String pageToken, Context context) { if (this.client.getEndpoint() == null) { return Mono.error( new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } final String accept = "application/json"; context = this.client.mergeContext(context); - return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + return service + .listEnvironments(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, organizationName, pageSize, pageToken, accept, context) .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)); } /** - * Get the next page of items. + * Lists of all the environments in a organization. * - * @param nextLink The URL to get the next list of items + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listEnvironmentsAsync(String resourceGroupName, String organizationName, + Integer pageSize, String pageToken) { + return new PagedFlux<>( + () -> listEnvironmentsSinglePageAsync(resourceGroupName, organizationName, pageSize, pageToken), + nextLink -> listEnvironmentsNextSinglePageAsync(nextLink)); + } + + /** + * Lists of all the environments in a organization. * - * The nextLink parameter. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listEnvironmentsAsync(String resourceGroupName, + String organizationName) { + final Integer pageSize = null; + final String pageToken = null; + return new PagedFlux<>( + () -> listEnvironmentsSinglePageAsync(resourceGroupName, organizationName, pageSize, pageToken), + nextLink -> listEnvironmentsNextSinglePageAsync(nextLink)); } /** - * Get the next page of items. + * Lists of all the environments in a organization. * - * @param nextLink The URL to get the next list of items + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listEnvironmentsAsync(String resourceGroupName, String organizationName, + Integer pageSize, String pageToken, Context context) { + return new PagedFlux<>( + () -> listEnvironmentsSinglePageAsync(resourceGroupName, organizationName, pageSize, pageToken, context), + nextLink -> listEnvironmentsNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists of all the environments in a organization. * - * The nextLink parameter. + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listEnvironments(String resourceGroupName, String organizationName) { + final Integer pageSize = null; + final String pageToken = null; + return new PagedIterable<>(listEnvironmentsAsync(resourceGroupName, organizationName, pageSize, pageToken)); + } + + /** + * Lists of all the environments in a organization. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list operation along with {@link PagedResponse} on successful completion of + * @return result of GET request to list Confluent operations as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listEnvironments(String resourceGroupName, String organizationName, + Integer pageSize, String pageToken, Context context) { + return new PagedIterable<>( + listEnvironmentsAsync(resourceGroupName, organizationName, pageSize, pageToken, context)); + } + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id along with {@link Response} on successful completion of * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, - Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + private Mono> getEnvironmentByIdWithResponseAsync(String resourceGroupName, + String organizationName, String environmentId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getEnvironmentById(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getEnvironmentByIdWithResponseAsync(String resourceGroupName, + String organizationName, String environmentId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getEnvironmentById(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, accept, context); + } + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getEnvironmentByIdAsync(String resourceGroupName, String organizationName, + String environmentId) { + return getEnvironmentByIdWithResponseAsync(resourceGroupName, organizationName, environmentId) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getEnvironmentByIdWithResponse(String resourceGroupName, + String organizationName, String environmentId, Context context) { + return getEnvironmentByIdWithResponseAsync(resourceGroupName, organizationName, environmentId, context).block(); + } + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public SCEnvironmentRecordInner getEnvironmentById(String resourceGroupName, String organizationName, + String environmentId) { + return getEnvironmentByIdWithResponse(resourceGroupName, organizationName, environmentId, Context.NONE) + .getValue(); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listClustersSinglePageAsync(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listClusters(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, pageSize, + pageToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listClustersSinglePageAsync(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listClusters(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, organizationName, environmentId, pageSize, pageToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listClustersAsync(String resourceGroupName, String organizationName, + String environmentId, Integer pageSize, String pageToken) { + return new PagedFlux<>( + () -> listClustersSinglePageAsync(resourceGroupName, organizationName, environmentId, pageSize, pageToken), + nextLink -> listClustersNextSinglePageAsync(nextLink)); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listClustersAsync(String resourceGroupName, String organizationName, + String environmentId) { + final Integer pageSize = null; + final String pageToken = null; + return new PagedFlux<>( + () -> listClustersSinglePageAsync(resourceGroupName, organizationName, environmentId, pageSize, pageToken), + nextLink -> listClustersNextSinglePageAsync(nextLink)); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listClustersAsync(String resourceGroupName, String organizationName, + String environmentId, Integer pageSize, String pageToken, Context context) { + return new PagedFlux<>(() -> listClustersSinglePageAsync(resourceGroupName, organizationName, environmentId, + pageSize, pageToken, context), nextLink -> listClustersNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId) { + final Integer pageSize = null; + final String pageToken = null; + return new PagedIterable<>( + listClustersAsync(resourceGroupName, organizationName, environmentId, pageSize, pageToken)); + } + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId, Integer pageSize, String pageToken, Context context) { + return new PagedIterable<>( + listClustersAsync(resourceGroupName, organizationName, environmentId, pageSize, pageToken, context)); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSchemaRegistryClustersSinglePageAsync( + String resourceGroupName, String organizationName, String environmentId, Integer pageSize, String pageToken) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listSchemaRegistryClusters(this.client.getEndpoint(), + this.client.getApiVersion(), this.client.getSubscriptionId(), resourceGroupName, organizationName, + environmentId, pageSize, pageToken, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSchemaRegistryClustersSinglePageAsync( + String resourceGroupName, String organizationName, String environmentId, Integer pageSize, String pageToken, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listSchemaRegistryClusters(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, pageSize, + pageToken, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listSchemaRegistryClustersAsync(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken) { + return new PagedFlux<>(() -> listSchemaRegistryClustersSinglePageAsync(resourceGroupName, organizationName, + environmentId, pageSize, pageToken), nextLink -> listSchemaRegistryClustersNextSinglePageAsync(nextLink)); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listSchemaRegistryClustersAsync(String resourceGroupName, + String organizationName, String environmentId) { + final Integer pageSize = null; + final String pageToken = null; + return new PagedFlux<>(() -> listSchemaRegistryClustersSinglePageAsync(resourceGroupName, organizationName, + environmentId, pageSize, pageToken), nextLink -> listSchemaRegistryClustersNextSinglePageAsync(nextLink)); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listSchemaRegistryClustersAsync(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken, Context context) { + return new PagedFlux<>( + () -> listSchemaRegistryClustersSinglePageAsync(resourceGroupName, organizationName, environmentId, + pageSize, pageToken, context), + nextLink -> listSchemaRegistryClustersNextSinglePageAsync(nextLink, context)); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId) { + final Integer pageSize = null; + final String pageToken = null; + return new PagedIterable<>( + listSchemaRegistryClustersAsync(resourceGroupName, organizationName, environmentId, pageSize, pageToken)); + } + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken, Context context) { + return new PagedIterable<>(listSchemaRegistryClustersAsync(resourceGroupName, organizationName, environmentId, + pageSize, pageToken, context)); + } + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listRegionsWithResponseAsync(String resourceGroupName, + String organizationName, ListAccessRequestModel body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listRegions(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, body, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listRegionsWithResponseAsync(String resourceGroupName, + String organizationName, ListAccessRequestModel body, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listRegions(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, body, accept, context); + } + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listRegionsAsync(String resourceGroupName, String organizationName, + ListAccessRequestModel body) { + return listRegionsWithResponseAsync(resourceGroupName, organizationName, body) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listRegionsWithResponse(String resourceGroupName, + String organizationName, ListAccessRequestModel body, Context context) { + return listRegionsWithResponseAsync(resourceGroupName, organizationName, body, context).block(); + } + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ListRegionsSuccessResponseInner listRegions(String resourceGroupName, String organizationName, + ListAccessRequestModel body) { + return listRegionsWithResponse(resourceGroupName, organizationName, body, Context.NONE).getValue(); + } + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createApiKeyWithResponseAsync(String resourceGroupName, + String organizationName, String environmentId, String clusterId, CreateApiKeyModel body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + if (clusterId == null) { + return Mono.error(new IllegalArgumentException("Parameter clusterId is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createApiKey(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, clusterId, body, + accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createApiKeyWithResponseAsync(String resourceGroupName, + String organizationName, String environmentId, String clusterId, CreateApiKeyModel body, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + if (clusterId == null) { + return Mono.error(new IllegalArgumentException("Parameter clusterId is required and cannot be null.")); + } + if (body == null) { + return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); + } else { + body.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createApiKey(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, clusterId, body, + accept, context); + } + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createApiKeyAsync(String resourceGroupName, String organizationName, + String environmentId, String clusterId, CreateApiKeyModel body) { + return createApiKeyWithResponseAsync(resourceGroupName, organizationName, environmentId, clusterId, body) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createApiKeyWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, CreateApiKeyModel body, Context context) { + return createApiKeyWithResponseAsync(resourceGroupName, organizationName, environmentId, clusterId, body, + context).block(); + } + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ApiKeyRecordInner createApiKey(String resourceGroupName, String organizationName, String environmentId, + String clusterId, CreateApiKeyModel body) { + return createApiKeyWithResponse(resourceGroupName, organizationName, environmentId, clusterId, body, + Context.NONE).getValue(); + } + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteClusterApiKeyWithResponseAsync(String resourceGroupName, String organizationName, + String apiKeyId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (apiKeyId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiKeyId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.deleteClusterApiKey(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, apiKeyId, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteClusterApiKeyWithResponseAsync(String resourceGroupName, String organizationName, + String apiKeyId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (apiKeyId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiKeyId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.deleteClusterApiKey(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, apiKeyId, accept, context); + } + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteClusterApiKeyAsync(String resourceGroupName, String organizationName, String apiKeyId) { + return deleteClusterApiKeyWithResponseAsync(resourceGroupName, organizationName, apiKeyId) + .flatMap(ignored -> Mono.empty()); + } + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteClusterApiKeyWithResponse(String resourceGroupName, String organizationName, + String apiKeyId, Context context) { + return deleteClusterApiKeyWithResponseAsync(resourceGroupName, organizationName, apiKeyId, context).block(); + } + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void deleteClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId) { + deleteClusterApiKeyWithResponse(resourceGroupName, organizationName, apiKeyId, Context.NONE); + } + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getClusterApiKeyWithResponseAsync(String resourceGroupName, + String organizationName, String apiKeyId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (apiKeyId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiKeyId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getClusterApiKey(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, apiKeyId, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getClusterApiKeyWithResponseAsync(String resourceGroupName, + String organizationName, String apiKeyId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (apiKeyId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiKeyId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getClusterApiKey(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, apiKeyId, accept, context); + } + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getClusterApiKeyAsync(String resourceGroupName, String organizationName, + String apiKeyId) { + return getClusterApiKeyWithResponseAsync(resourceGroupName, organizationName, apiKeyId) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getClusterApiKeyWithResponse(String resourceGroupName, String organizationName, + String apiKeyId, Context context) { + return getClusterApiKeyWithResponseAsync(resourceGroupName, organizationName, apiKeyId, context).block(); + } + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ApiKeyRecordInner getClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId) { + return getClusterApiKeyWithResponse(resourceGroupName, organizationName, apiKeyId, Context.NONE).getValue(); + } + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getSchemaRegistryClusterByIdWithResponseAsync( + String resourceGroupName, String organizationName, String environmentId, String clusterId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + if (clusterId == null) { + return Mono.error(new IllegalArgumentException("Parameter clusterId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getSchemaRegistryClusterById(this.client.getEndpoint(), + this.client.getApiVersion(), this.client.getSubscriptionId(), resourceGroupName, organizationName, + environmentId, clusterId, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getSchemaRegistryClusterByIdWithResponseAsync( + String resourceGroupName, String organizationName, String environmentId, String clusterId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + if (clusterId == null) { + return Mono.error(new IllegalArgumentException("Parameter clusterId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getSchemaRegistryClusterById(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, clusterId, accept, + context); + } + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getSchemaRegistryClusterByIdAsync(String resourceGroupName, + String organizationName, String environmentId, String clusterId) { + return getSchemaRegistryClusterByIdWithResponseAsync(resourceGroupName, organizationName, environmentId, + clusterId).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getSchemaRegistryClusterByIdWithResponse(String resourceGroupName, + String organizationName, String environmentId, String clusterId, Context context) { + return getSchemaRegistryClusterByIdWithResponseAsync(resourceGroupName, organizationName, environmentId, + clusterId, context).block(); + } + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public SchemaRegistryClusterRecordInner getSchemaRegistryClusterById(String resourceGroupName, + String organizationName, String environmentId, String clusterId) { + return getSchemaRegistryClusterByIdWithResponse(resourceGroupName, organizationName, environmentId, clusterId, + Context.NONE).getValue(); + } + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getClusterByIdWithResponseAsync(String resourceGroupName, + String organizationName, String environmentId, String clusterId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + if (clusterId == null) { + return Mono.error(new IllegalArgumentException("Parameter clusterId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getClusterById(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, clusterId, accept, + context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getClusterByIdWithResponseAsync(String resourceGroupName, + String organizationName, String environmentId, String clusterId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (organizationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter organizationName is required and cannot be null.")); + } + if (environmentId == null) { + return Mono.error(new IllegalArgumentException("Parameter environmentId is required and cannot be null.")); + } + if (clusterId == null) { + return Mono.error(new IllegalArgumentException("Parameter clusterId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getClusterById(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, organizationName, environmentId, clusterId, accept, + context); + } + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getClusterByIdAsync(String resourceGroupName, String organizationName, + String environmentId, String clusterId) { + return getClusterByIdWithResponseAsync(resourceGroupName, organizationName, environmentId, clusterId) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getClusterByIdWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, Context context) { + return getClusterByIdWithResponseAsync(resourceGroupName, organizationName, environmentId, clusterId, context) + .block(); + } + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public SCClusterRecordInner getClusterById(String resourceGroupName, String organizationName, String environmentId, + String clusterId) { + return getClusterByIdWithResponse(resourceGroupName, organizationName, environmentId, clusterId, Context.NONE) + .getValue(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a list operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a list operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a list operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of a list operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { return Mono.error( @@ -1227,4 +2950,183 @@ private Mono> listByResourceGroupNextSi .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)); } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listEnvironmentsNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listEnvironmentsNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listEnvironmentsNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listEnvironmentsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listClustersNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listClustersNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listClustersNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listClustersNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list schema registry clusters in the environment of a confluent organization + * along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + listSchemaRegistryClustersNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listSchemaRegistryClustersNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list schema registry clusters in the environment of a confluent organization + * along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + listSchemaRegistryClustersNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listSchemaRegistryClustersNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsImpl.java index 6ad36d3fc77d9..a7b50fb2f851d 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/OrganizationsImpl.java @@ -10,9 +10,21 @@ import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.confluent.fluent.OrganizationsClient; +import com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner; import com.azure.resourcemanager.confluent.fluent.models.OrganizationResourceInner; +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.azure.resourcemanager.confluent.models.ApiKeyRecord; +import com.azure.resourcemanager.confluent.models.CreateApiKeyModel; +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.ListRegionsSuccessResponse; import com.azure.resourcemanager.confluent.models.OrganizationResource; import com.azure.resourcemanager.confluent.models.Organizations; +import com.azure.resourcemanager.confluent.models.SCClusterRecord; +import com.azure.resourcemanager.confluent.models.SCEnvironmentRecord; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterRecord; public final class OrganizationsImpl implements Organizations { private static final ClientLogger LOGGER = new ClientLogger(OrganizationsImpl.class); @@ -29,23 +41,23 @@ public OrganizationsImpl(OrganizationsClient innerClient, public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); } public Response getByResourceGroupWithResponse(String resourceGroupName, @@ -77,13 +89,203 @@ public void delete(String resourceGroupName, String organizationName, Context co this.serviceClient().delete(resourceGroupName, organizationName, context); } + public PagedIterable listEnvironments(String resourceGroupName, String organizationName) { + PagedIterable inner + = this.serviceClient().listEnvironments(resourceGroupName, organizationName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SCEnvironmentRecordImpl(inner1, this.manager())); + } + + public PagedIterable listEnvironments(String resourceGroupName, String organizationName, + Integer pageSize, String pageToken, Context context) { + PagedIterable inner + = this.serviceClient().listEnvironments(resourceGroupName, organizationName, pageSize, pageToken, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SCEnvironmentRecordImpl(inner1, this.manager())); + } + + public Response getEnvironmentByIdWithResponse(String resourceGroupName, + String organizationName, String environmentId, Context context) { + Response inner = this.serviceClient() + .getEnvironmentByIdWithResponse(resourceGroupName, organizationName, environmentId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SCEnvironmentRecordImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public SCEnvironmentRecord getEnvironmentById(String resourceGroupName, String organizationName, + String environmentId) { + SCEnvironmentRecordInner inner + = this.serviceClient().getEnvironmentById(resourceGroupName, organizationName, environmentId); + if (inner != null) { + return new SCEnvironmentRecordImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId) { + PagedIterable inner + = this.serviceClient().listClusters(resourceGroupName, organizationName, environmentId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SCClusterRecordImpl(inner1, this.manager())); + } + + public PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId, Integer pageSize, String pageToken, Context context) { + PagedIterable inner = this.serviceClient().listClusters(resourceGroupName, + organizationName, environmentId, pageSize, pageToken, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SCClusterRecordImpl(inner1, this.manager())); + } + + public PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId) { + PagedIterable inner + = this.serviceClient().listSchemaRegistryClusters(resourceGroupName, organizationName, environmentId); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SchemaRegistryClusterRecordImpl(inner1, this.manager())); + } + + public PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken, Context context) { + PagedIterable inner = this.serviceClient().listSchemaRegistryClusters( + resourceGroupName, organizationName, environmentId, pageSize, pageToken, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SchemaRegistryClusterRecordImpl(inner1, this.manager())); + } + + public Response listRegionsWithResponse(String resourceGroupName, + String organizationName, ListAccessRequestModel body, Context context) { + Response inner + = this.serviceClient().listRegionsWithResponse(resourceGroupName, organizationName, body, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ListRegionsSuccessResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ListRegionsSuccessResponse listRegions(String resourceGroupName, String organizationName, + ListAccessRequestModel body) { + ListRegionsSuccessResponseInner inner + = this.serviceClient().listRegions(resourceGroupName, organizationName, body); + if (inner != null) { + return new ListRegionsSuccessResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response createApiKeyWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, CreateApiKeyModel body, Context context) { + Response inner = this.serviceClient().createApiKeyWithResponse(resourceGroupName, + organizationName, environmentId, clusterId, body, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ApiKeyRecordImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ApiKeyRecord createApiKey(String resourceGroupName, String organizationName, String environmentId, + String clusterId, CreateApiKeyModel body) { + ApiKeyRecordInner inner + = this.serviceClient().createApiKey(resourceGroupName, organizationName, environmentId, clusterId, body); + if (inner != null) { + return new ApiKeyRecordImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response deleteClusterApiKeyWithResponse(String resourceGroupName, String organizationName, + String apiKeyId, Context context) { + return this.serviceClient().deleteClusterApiKeyWithResponse(resourceGroupName, organizationName, apiKeyId, + context); + } + + public void deleteClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId) { + this.serviceClient().deleteClusterApiKey(resourceGroupName, organizationName, apiKeyId); + } + + public Response getClusterApiKeyWithResponse(String resourceGroupName, String organizationName, + String apiKeyId, Context context) { + Response inner + = this.serviceClient().getClusterApiKeyWithResponse(resourceGroupName, organizationName, apiKeyId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ApiKeyRecordImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ApiKeyRecord getClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId) { + ApiKeyRecordInner inner = this.serviceClient().getClusterApiKey(resourceGroupName, organizationName, apiKeyId); + if (inner != null) { + return new ApiKeyRecordImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response getSchemaRegistryClusterByIdWithResponse(String resourceGroupName, + String organizationName, String environmentId, String clusterId, Context context) { + Response inner + = this.serviceClient().getSchemaRegistryClusterByIdWithResponse(resourceGroupName, organizationName, + environmentId, clusterId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SchemaRegistryClusterRecordImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public SchemaRegistryClusterRecord getSchemaRegistryClusterById(String resourceGroupName, String organizationName, + String environmentId, String clusterId) { + SchemaRegistryClusterRecordInner inner = this.serviceClient().getSchemaRegistryClusterById(resourceGroupName, + organizationName, environmentId, clusterId); + if (inner != null) { + return new SchemaRegistryClusterRecordImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response getClusterByIdWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, Context context) { + Response inner = this.serviceClient().getClusterByIdWithResponse(resourceGroupName, + organizationName, environmentId, clusterId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SCClusterRecordImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public SCClusterRecord getClusterById(String resourceGroupName, String organizationName, String environmentId, + String clusterId) { + SCClusterRecordInner inner + = this.serviceClient().getClusterById(resourceGroupName, organizationName, environmentId, clusterId); + if (inner != null) { + return new SCClusterRecordImpl(inner, this.manager()); + } else { + return null; + } + } + public OrganizationResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String organizationName = Utils.getValueFromIdByName(id, "organizations"); + String organizationName = ResourceManagerUtils.getValueFromIdByName(id, "organizations"); if (organizationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'organizations'.", id))); @@ -92,12 +294,12 @@ public OrganizationResource getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String organizationName = Utils.getValueFromIdByName(id, "organizations"); + String organizationName = ResourceManagerUtils.getValueFromIdByName(id, "organizations"); if (organizationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'organizations'.", id))); @@ -106,12 +308,12 @@ public Response getByIdWithResponse(String id, Context con } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String organizationName = Utils.getValueFromIdByName(id, "organizations"); + String organizationName = ResourceManagerUtils.getValueFromIdByName(id, "organizations"); if (organizationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'organizations'.", id))); @@ -120,12 +322,12 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String organizationName = Utils.getValueFromIdByName(id, "organizations"); + String organizationName = ResourceManagerUtils.getValueFromIdByName(id, "organizations"); if (organizationName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'organizations'.", id))); diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/Utils.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ResourceManagerUtils.java similarity index 99% rename from sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/Utils.java rename to sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ResourceManagerUtils.java index 194fbe93c316b..6490c75bb35fe 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/Utils.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ResourceManagerUtils.java @@ -19,8 +19,8 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { - private Utils() { +final class ResourceManagerUtils { + private ResourceManagerUtils() { } static String getValueFromIdByName(String id, String name) { diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCClusterRecordImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCClusterRecordImpl.java new file mode 100644 index 0000000000000..2264401bf9f1e --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCClusterRecordImpl.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.implementation; + +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.azure.resourcemanager.confluent.models.ClusterStatusEntity; +import com.azure.resourcemanager.confluent.models.SCClusterRecord; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; + +public final class SCClusterRecordImpl implements SCClusterRecord { + private SCClusterRecordInner innerObject; + + private final com.azure.resourcemanager.confluent.ConfluentManager serviceManager; + + SCClusterRecordImpl(SCClusterRecordInner innerObject, + com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String kind() { + return this.innerModel().kind(); + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public SCMetadataEntity metadata() { + return this.innerModel().metadata(); + } + + public SCClusterSpecEntity spec() { + return this.innerModel().spec(); + } + + public ClusterStatusEntity status() { + return this.innerModel().status(); + } + + public SCClusterRecordInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.confluent.ConfluentManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCEnvironmentRecordImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCEnvironmentRecordImpl.java new file mode 100644 index 0000000000000..9910b08302467 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SCEnvironmentRecordImpl.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.implementation; + +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.azure.resourcemanager.confluent.models.SCEnvironmentRecord; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; + +public final class SCEnvironmentRecordImpl implements SCEnvironmentRecord { + private SCEnvironmentRecordInner innerObject; + + private final com.azure.resourcemanager.confluent.ConfluentManager serviceManager; + + SCEnvironmentRecordImpl(SCEnvironmentRecordInner innerObject, + com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String kind() { + return this.innerModel().kind(); + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public SCMetadataEntity metadata() { + return this.innerModel().metadata(); + } + + public SCEnvironmentRecordInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.confluent.ConfluentManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SchemaRegistryClusterRecordImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SchemaRegistryClusterRecordImpl.java new file mode 100644 index 0000000000000..c3d2762b4ebfe --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/SchemaRegistryClusterRecordImpl.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.implementation; + +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterRecord; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; + +public final class SchemaRegistryClusterRecordImpl implements SchemaRegistryClusterRecord { + private SchemaRegistryClusterRecordInner innerObject; + + private final com.azure.resourcemanager.confluent.ConfluentManager serviceManager; + + SchemaRegistryClusterRecordImpl(SchemaRegistryClusterRecordInner innerObject, + com.azure.resourcemanager.confluent.ConfluentManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String kind() { + return this.innerModel().kind(); + } + + public String id() { + return this.innerModel().id(); + } + + public SCMetadataEntity metadata() { + return this.innerModel().metadata(); + } + + public SchemaRegistryClusterSpecEntity spec() { + return this.innerModel().spec(); + } + + public SchemaRegistryClusterStatusEntity status() { + return this.innerModel().status(); + } + + public SchemaRegistryClusterRecordInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.confluent.ConfluentManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ValidationsClientImpl.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ValidationsClientImpl.java index 904db4790f00c..dbcd0cd4a1923 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ValidationsClientImpl.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/implementation/ValidationsClientImpl.java @@ -85,7 +85,7 @@ Mono> validateOrganizationV2(@HostParam("$host /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -127,7 +127,7 @@ private Mono> validateOrganizationWithRespon /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -169,7 +169,7 @@ private Mono> validateOrganizationWithRespon /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -187,7 +187,7 @@ private Mono validateOrganizationAsync(String resourc /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -205,7 +205,7 @@ public Response validateOrganizationWithResponse(Stri /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -222,7 +222,7 @@ public OrganizationResourceInner validateOrganization(String resourceGroupName, /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -265,7 +265,7 @@ private Mono> validateOrganizationV2WithRespon /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -307,7 +307,7 @@ private Mono> validateOrganizationV2WithRespon /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -325,7 +325,7 @@ private Mono validateOrganizationV2Async(String resourc /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -343,7 +343,7 @@ public Response validateOrganizationV2WithResponse(Stri /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Access.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Access.java index bafda6c5576ac..bc09e22cf7804 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Access.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Access.java @@ -14,7 +14,7 @@ public interface Access { /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -29,7 +29,7 @@ Response listUsersWithResponse(String resourceGr /** * Organization users details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -43,7 +43,7 @@ AccessListUsersSuccessResponse listUsers(String resourceGroupName, String organi /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -58,7 +58,7 @@ Response listServiceAccountsWithRespon /** * Organization service accounts details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -72,7 +72,7 @@ AccessListServiceAccountsSuccessResponse listServiceAccounts(String resourceGrou /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @param context The context to associate with this operation. @@ -87,7 +87,7 @@ Response listInvitationsWithResponse(Strin /** * Organization accounts invitation details. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body List Access Request Model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -101,7 +101,7 @@ AccessListInvitationsSuccessResponse listInvitations(String resourceGroupName, S /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @param context The context to associate with this operation. @@ -116,7 +116,7 @@ Response inviteUserWithResponse(String resourceGroupName, Stri /** * Invite user to the organization. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Invite user account model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -136,7 +136,7 @@ Response inviteUserWithResponse(String resourceGroupName, Stri * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response along with {@link Response}. + * @return details of the environments returned on successful response along with {@link Response}. */ Response listEnvironmentsWithResponse(String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context); @@ -150,7 +150,7 @@ Response listEnvironmentsWithResponse(Str * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list environments success response. + * @return details of the environments returned on successful response. */ AccessListEnvironmentsSuccessResponse listEnvironments(String resourceGroupName, String organizationName, ListAccessRequestModel body); @@ -165,7 +165,7 @@ AccessListEnvironmentsSuccessResponse listEnvironments(String resourceGroupName, * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response}. + * @return details of the clusters returned on successful response along with {@link Response}. */ Response listClustersWithResponse(String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context); @@ -179,7 +179,7 @@ Response listClustersWithResponse(String resou * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response. + * @return details of the clusters returned on successful response. */ AccessListClusterSuccessResponse listClusters(String resourceGroupName, String organizationName, ListAccessRequestModel body); @@ -194,7 +194,7 @@ AccessListClusterSuccessResponse listClusters(String resourceGroupName, String o * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response along with {@link Response}. + * @return details of the role bindings returned on successful response along with {@link Response}. */ Response listRoleBindingsWithResponse(String resourceGroupName, String organizationName, ListAccessRequestModel body, Context context); @@ -208,8 +208,93 @@ Response listRoleBindingsWithResponse(Str * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list cluster success response. + * @return details of the role bindings returned on successful response. */ AccessListRoleBindingsSuccessResponse listRoleBindings(String resourceGroupName, String organizationName, ListAccessRequestModel body); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding along with {@link Response}. + */ + Response createRoleBindingWithResponse(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body, Context context); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body Create role binding Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details on principal, role name and crn pattern of a role binding. + */ + RoleBindingRecord createRoleBinding(String resourceGroupName, String organizationName, + AccessCreateRoleBindingRequestModel body); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteRoleBindingWithResponse(String resourceGroupName, String organizationName, + String roleBindingId, Context context); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param roleBindingId Confluent Role binding id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteRoleBinding(String resourceGroupName, String organizationName, String roleBindingId); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response along with {@link Response}. + */ + Response listRoleBindingNameListWithResponse(String resourceGroupName, + String organizationName, ListAccessRequestModel body, Context context); + + /** + * Organization role bindings. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details of the role binding names returned on successful response. + */ + AccessRoleBindingNameListSuccessResponse listRoleBindingNameList(String resourceGroupName, String organizationName, + ListAccessRequestModel body); } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessCreateRoleBindingRequestModel.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessCreateRoleBindingRequestModel.java new file mode 100644 index 0000000000000..5ae350a95eefd --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessCreateRoleBindingRequestModel.java @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Create role binding request model. + */ +@Fluent +public final class AccessCreateRoleBindingRequestModel { + /* + * The principal User or Group to bind the role to + */ + @JsonProperty(value = "principal") + private String principal; + + /* + * The name of the role to bind to the principal + */ + @JsonProperty(value = "role_name") + private String roleName; + + /* + * A CRN that specifies the scope and resource patterns necessary for the role to bind + */ + @JsonProperty(value = "crn_pattern") + private String crnPattern; + + /** + * Creates an instance of AccessCreateRoleBindingRequestModel class. + */ + public AccessCreateRoleBindingRequestModel() { + } + + /** + * Get the principal property: The principal User or Group to bind the role to. + * + * @return the principal value. + */ + public String principal() { + return this.principal; + } + + /** + * Set the principal property: The principal User or Group to bind the role to. + * + * @param principal the principal value to set. + * @return the AccessCreateRoleBindingRequestModel object itself. + */ + public AccessCreateRoleBindingRequestModel withPrincipal(String principal) { + this.principal = principal; + return this; + } + + /** + * Get the roleName property: The name of the role to bind to the principal. + * + * @return the roleName value. + */ + public String roleName() { + return this.roleName; + } + + /** + * Set the roleName property: The name of the role to bind to the principal. + * + * @param roleName the roleName value to set. + * @return the AccessCreateRoleBindingRequestModel object itself. + */ + public AccessCreateRoleBindingRequestModel withRoleName(String roleName) { + this.roleName = roleName; + return this; + } + + /** + * Get the crnPattern property: A CRN that specifies the scope and resource patterns necessary for the role to + * bind. + * + * @return the crnPattern value. + */ + public String crnPattern() { + return this.crnPattern; + } + + /** + * Set the crnPattern property: A CRN that specifies the scope and resource patterns necessary for the role to + * bind. + * + * @param crnPattern the crnPattern value to set. + * @return the AccessCreateRoleBindingRequestModel object itself. + */ + public AccessCreateRoleBindingRequestModel withCrnPattern(String crnPattern) { + this.crnPattern = crnPattern; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListClusterSuccessResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListClusterSuccessResponse.java index fda1bc4e9c1f0..0b6415472e5e9 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListClusterSuccessResponse.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListClusterSuccessResponse.java @@ -26,7 +26,7 @@ public interface AccessListClusterSuccessResponse { ConfluentListMetadata metadata(); /** - * Gets the data property: Data of the environments list. + * Gets the data property: List of clusters. * * @return the data value. */ diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListEnvironmentsSuccessResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListEnvironmentsSuccessResponse.java index 02dca2f03a333..6481022ddeb25 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListEnvironmentsSuccessResponse.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListEnvironmentsSuccessResponse.java @@ -19,14 +19,14 @@ public interface AccessListEnvironmentsSuccessResponse { String kind(); /** - * Gets the metadata property: Metadata of the list. + * Gets the metadata property: Metadata of the environment list. * * @return the metadata value. */ ConfluentListMetadata metadata(); /** - * Gets the data property: Data of the environments list. + * Gets the data property: Environment list data. * * @return the data value. */ diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListRoleBindingsSuccessResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListRoleBindingsSuccessResponse.java index a102b86db33ea..7316c91af807b 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListRoleBindingsSuccessResponse.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessListRoleBindingsSuccessResponse.java @@ -26,7 +26,7 @@ public interface AccessListRoleBindingsSuccessResponse { ConfluentListMetadata metadata(); /** - * Gets the data property: Data of the environments list. + * Gets the data property: List of role binding. * * @return the data value. */ diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessRoleBindingNameListSuccessResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessRoleBindingNameListSuccessResponse.java new file mode 100644 index 0000000000000..e1d530f444b02 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/AccessRoleBindingNameListSuccessResponse.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner; +import java.util.List; + +/** + * An immutable client-side representation of AccessRoleBindingNameListSuccessResponse. + */ +public interface AccessRoleBindingNameListSuccessResponse { + /** + * Gets the kind property: Type of response. + * + * @return the kind value. + */ + String kind(); + + /** + * Gets the metadata property: Metadata of the list. + * + * @return the metadata value. + */ + ConfluentListMetadata metadata(); + + /** + * Gets the data property: List of role binding names. + * + * @return the data value. + */ + List data(); + + /** + * Gets the inner com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner + * object. + * + * @return the inner object. + */ + AccessRoleBindingNameListSuccessResponseInner innerModel(); +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyOwnerEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyOwnerEntity.java new file mode 100644 index 0000000000000..5de30d35f8a0d --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyOwnerEntity.java @@ -0,0 +1,132 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * API Key Owner details which can be a user or service account. + */ +@Fluent +public final class ApiKeyOwnerEntity { + /* + * API Key owner id + */ + @JsonProperty(value = "id") + private String id; + + /* + * API URL for accessing or modifying the referred object + */ + @JsonProperty(value = "related") + private String related; + + /* + * CRN reference to the referred resource + */ + @JsonProperty(value = "resourceName") + private String resourceName; + + /* + * Type of the owner service or user account + */ + @JsonProperty(value = "kind") + private String kind; + + /** + * Creates an instance of ApiKeyOwnerEntity class. + */ + public ApiKeyOwnerEntity() { + } + + /** + * Get the id property: API Key owner id. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: API Key owner id. + * + * @param id the id value to set. + * @return the ApiKeyOwnerEntity object itself. + */ + public ApiKeyOwnerEntity withId(String id) { + this.id = id; + return this; + } + + /** + * Get the related property: API URL for accessing or modifying the referred object. + * + * @return the related value. + */ + public String related() { + return this.related; + } + + /** + * Set the related property: API URL for accessing or modifying the referred object. + * + * @param related the related value to set. + * @return the ApiKeyOwnerEntity object itself. + */ + public ApiKeyOwnerEntity withRelated(String related) { + this.related = related; + return this; + } + + /** + * Get the resourceName property: CRN reference to the referred resource. + * + * @return the resourceName value. + */ + public String resourceName() { + return this.resourceName; + } + + /** + * Set the resourceName property: CRN reference to the referred resource. + * + * @param resourceName the resourceName value to set. + * @return the ApiKeyOwnerEntity object itself. + */ + public ApiKeyOwnerEntity withResourceName(String resourceName) { + this.resourceName = resourceName; + return this; + } + + /** + * Get the kind property: Type of the owner service or user account. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Type of the owner service or user account. + * + * @param kind the kind value to set. + * @return the ApiKeyOwnerEntity object itself. + */ + public ApiKeyOwnerEntity withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyRecord.java new file mode 100644 index 0000000000000..737668e631760 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyRecord.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner; + +/** + * An immutable client-side representation of ApiKeyRecord. + */ +public interface ApiKeyRecord { + /** + * Gets the kind property: Type of api key. + * + * @return the kind value. + */ + String kind(); + + /** + * Gets the id property: Id of the api key. + * + * @return the id value. + */ + String id(); + + /** + * Gets the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + SCMetadataEntity metadata(); + + /** + * Gets the spec property: Specification of the API Key. + * + * @return the spec value. + */ + ApiKeySpecEntity spec(); + + /** + * Gets the inner com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner object. + * + * @return the inner object. + */ + ApiKeyRecordInner innerModel(); +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyResourceEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyResourceEntity.java new file mode 100644 index 0000000000000..780e53b3a4cf2 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeyResourceEntity.java @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * API Key Resource details which can be kafka cluster or schema registry cluster. + */ +@Fluent +public final class ApiKeyResourceEntity { + /* + * Id of the resource + */ + @JsonProperty(value = "id") + private String id; + + /* + * The environment of the api key + */ + @JsonProperty(value = "environment") + private String environment; + + /* + * API URL for accessing or modifying the api key resource object + */ + @JsonProperty(value = "related") + private String related; + + /* + * CRN reference to the referred resource + */ + @JsonProperty(value = "resourceName") + private String resourceName; + + /* + * Type of the owner which can be service or user account + */ + @JsonProperty(value = "kind") + private String kind; + + /** + * Creates an instance of ApiKeyResourceEntity class. + */ + public ApiKeyResourceEntity() { + } + + /** + * Get the id property: Id of the resource. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Id of the resource. + * + * @param id the id value to set. + * @return the ApiKeyResourceEntity object itself. + */ + public ApiKeyResourceEntity withId(String id) { + this.id = id; + return this; + } + + /** + * Get the environment property: The environment of the api key. + * + * @return the environment value. + */ + public String environment() { + return this.environment; + } + + /** + * Set the environment property: The environment of the api key. + * + * @param environment the environment value to set. + * @return the ApiKeyResourceEntity object itself. + */ + public ApiKeyResourceEntity withEnvironment(String environment) { + this.environment = environment; + return this; + } + + /** + * Get the related property: API URL for accessing or modifying the api key resource object. + * + * @return the related value. + */ + public String related() { + return this.related; + } + + /** + * Set the related property: API URL for accessing or modifying the api key resource object. + * + * @param related the related value to set. + * @return the ApiKeyResourceEntity object itself. + */ + public ApiKeyResourceEntity withRelated(String related) { + this.related = related; + return this; + } + + /** + * Get the resourceName property: CRN reference to the referred resource. + * + * @return the resourceName value. + */ + public String resourceName() { + return this.resourceName; + } + + /** + * Set the resourceName property: CRN reference to the referred resource. + * + * @param resourceName the resourceName value to set. + * @return the ApiKeyResourceEntity object itself. + */ + public ApiKeyResourceEntity withResourceName(String resourceName) { + this.resourceName = resourceName; + return this; + } + + /** + * Get the kind property: Type of the owner which can be service or user account. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Type of the owner which can be service or user account. + * + * @param kind the kind value to set. + * @return the ApiKeyResourceEntity object itself. + */ + public ApiKeyResourceEntity withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeySpecEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeySpecEntity.java new file mode 100644 index 0000000000000..6134eaf67263b --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ApiKeySpecEntity.java @@ -0,0 +1,164 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Spec of the API Key record. + */ +@Fluent +public final class ApiKeySpecEntity { + /* + * The description of the API Key + */ + @JsonProperty(value = "description") + private String description; + + /* + * The name of the API Key + */ + @JsonProperty(value = "name") + private String name; + + /* + * API Key Secret + */ + @JsonProperty(value = "secret") + private String secret; + + /* + * Specification of the cluster + */ + @JsonProperty(value = "resource") + private ApiKeyResourceEntity resource; + + /* + * Specification of the cluster + */ + @JsonProperty(value = "owner") + private ApiKeyOwnerEntity owner; + + /** + * Creates an instance of ApiKeySpecEntity class. + */ + public ApiKeySpecEntity() { + } + + /** + * Get the description property: The description of the API Key. + * + * @return the description value. + */ + public String description() { + return this.description; + } + + /** + * Set the description property: The description of the API Key. + * + * @param description the description value to set. + * @return the ApiKeySpecEntity object itself. + */ + public ApiKeySpecEntity withDescription(String description) { + this.description = description; + return this; + } + + /** + * Get the name property: The name of the API Key. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: The name of the API Key. + * + * @param name the name value to set. + * @return the ApiKeySpecEntity object itself. + */ + public ApiKeySpecEntity withName(String name) { + this.name = name; + return this; + } + + /** + * Get the secret property: API Key Secret. + * + * @return the secret value. + */ + public String secret() { + return this.secret; + } + + /** + * Set the secret property: API Key Secret. + * + * @param secret the secret value to set. + * @return the ApiKeySpecEntity object itself. + */ + public ApiKeySpecEntity withSecret(String secret) { + this.secret = secret; + return this; + } + + /** + * Get the resource property: Specification of the cluster. + * + * @return the resource value. + */ + public ApiKeyResourceEntity resource() { + return this.resource; + } + + /** + * Set the resource property: Specification of the cluster. + * + * @param resource the resource value to set. + * @return the ApiKeySpecEntity object itself. + */ + public ApiKeySpecEntity withResource(ApiKeyResourceEntity resource) { + this.resource = resource; + return this; + } + + /** + * Get the owner property: Specification of the cluster. + * + * @return the owner value. + */ + public ApiKeyOwnerEntity owner() { + return this.owner; + } + + /** + * Set the owner property: Specification of the cluster. + * + * @param owner the owner value to set. + * @return the ApiKeySpecEntity object itself. + */ + public ApiKeySpecEntity withOwner(ApiKeyOwnerEntity owner) { + this.owner = owner; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (resource() != null) { + resource().validate(); + } + if (owner() != null) { + owner().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ClusterRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ClusterRecord.java index ef16c002c7ef1..5e8128ec0e50f 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ClusterRecord.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ClusterRecord.java @@ -8,18 +8,18 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * Record of the environment. + * Details of cluster record. */ @Fluent public final class ClusterRecord { /* - * Type of environment + * Type of cluster */ @JsonProperty(value = "kind") private String kind; /* - * Id of the environment + * Id of the cluster */ @JsonProperty(value = "id") private String id; @@ -31,7 +31,7 @@ public final class ClusterRecord { private MetadataEntity metadata; /* - * Display name of the user + * Display name of the cluster */ @JsonProperty(value = "display_name") private String displayName; @@ -55,7 +55,7 @@ public ClusterRecord() { } /** - * Get the kind property: Type of environment. + * Get the kind property: Type of cluster. * * @return the kind value. */ @@ -64,7 +64,7 @@ public String kind() { } /** - * Set the kind property: Type of environment. + * Set the kind property: Type of cluster. * * @param kind the kind value to set. * @return the ClusterRecord object itself. @@ -75,7 +75,7 @@ public ClusterRecord withKind(String kind) { } /** - * Get the id property: Id of the environment. + * Get the id property: Id of the cluster. * * @return the id value. */ @@ -84,7 +84,7 @@ public String id() { } /** - * Set the id property: Id of the environment. + * Set the id property: Id of the cluster. * * @param id the id value to set. * @return the ClusterRecord object itself. @@ -115,7 +115,7 @@ public ClusterRecord withMetadata(MetadataEntity metadata) { } /** - * Get the displayName property: Display name of the user. + * Get the displayName property: Display name of the cluster. * * @return the displayName value. */ @@ -124,7 +124,7 @@ public String displayName() { } /** - * Set the displayName property: Display name of the user. + * Set the displayName property: Display name of the cluster. * * @param displayName the displayName value to set. * @return the ClusterRecord object itself. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/CreateApiKeyModel.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/CreateApiKeyModel.java new file mode 100644 index 0000000000000..03f84c55879c1 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/CreateApiKeyModel.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Create API Key model. + */ +@Fluent +public final class CreateApiKeyModel { + /* + * Name of the API Key + */ + @JsonProperty(value = "name") + private String name; + + /* + * Description of the API Key + */ + @JsonProperty(value = "description") + private String description; + + /** + * Creates an instance of CreateApiKeyModel class. + */ + public CreateApiKeyModel() { + } + + /** + * Get the name property: Name of the API Key. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the API Key. + * + * @param name the name value to set. + * @return the CreateApiKeyModel object itself. + */ + public CreateApiKeyModel withName(String name) { + this.name = name; + return this; + } + + /** + * Get the description property: Description of the API Key. + * + * @return the description value. + */ + public String description() { + return this.description; + } + + /** + * Set the description property: Description of the API Key. + * + * @param description the description value to set. + * @return the CreateApiKeyModel object itself. + */ + public CreateApiKeyModel withDescription(String description) { + this.description = description; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/EnvironmentRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/EnvironmentRecord.java index ca296587a9c92..61ea9ff4a004c 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/EnvironmentRecord.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/EnvironmentRecord.java @@ -8,7 +8,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * Record of the environment. + * Details about environment name, metadata and environment id of an environment. */ @Fluent public final class EnvironmentRecord { diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/GetEnvironmentsResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/GetEnvironmentsResponse.java new file mode 100644 index 0000000000000..77dc0f69e041d --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/GetEnvironmentsResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Result of GET request to list Confluent operations. + */ +@Fluent +public final class GetEnvironmentsResponse { + /* + * List of environments in a confluent organization + */ + @JsonProperty(value = "value") + private List value; + + /* + * URL to get the next set of environment records if there are any. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of GetEnvironmentsResponse class. + */ + public GetEnvironmentsResponse() { + } + + /** + * Get the value property: List of environments in a confluent organization. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: List of environments in a confluent organization. + * + * @param value the value value to set. + * @return the GetEnvironmentsResponse object itself. + */ + public GetEnvironmentsResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: URL to get the next set of environment records if there are any. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: URL to get the next set of environment records if there are any. + * + * @param nextLink the nextLink value to set. + * @return the GetEnvironmentsResponse object itself. + */ + public GetEnvironmentsResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListClustersSuccessResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListClustersSuccessResponse.java new file mode 100644 index 0000000000000..115fdb9150bc5 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListClustersSuccessResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Result of GET request to list clusters in the environment of a confluent organization. + */ +@Fluent +public final class ListClustersSuccessResponse { + /* + * List of clusters in an environment of a confluent organization + */ + @JsonProperty(value = "value") + private List value; + + /* + * URL to get the next set of cluster records if there are any. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of ListClustersSuccessResponse class. + */ + public ListClustersSuccessResponse() { + } + + /** + * Get the value property: List of clusters in an environment of a confluent organization. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: List of clusters in an environment of a confluent organization. + * + * @param value the value value to set. + * @return the ListClustersSuccessResponse object itself. + */ + public ListClustersSuccessResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: URL to get the next set of cluster records if there are any. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: URL to get the next set of cluster records if there are any. + * + * @param nextLink the nextLink value to set. + * @return the ListClustersSuccessResponse object itself. + */ + public ListClustersSuccessResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListRegionsSuccessResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListRegionsSuccessResponse.java new file mode 100644 index 0000000000000..ab7d8dedf819c --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListRegionsSuccessResponse.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner; +import java.util.List; + +/** + * An immutable client-side representation of ListRegionsSuccessResponse. + */ +public interface ListRegionsSuccessResponse { + /** + * Gets the data property: List of regions supported by confluent. + * + * @return the data value. + */ + List data(); + + /** + * Gets the inner com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner object. + * + * @return the inner object. + */ + ListRegionsSuccessResponseInner innerModel(); +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListSchemaRegistryClustersResponse.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListSchemaRegistryClustersResponse.java new file mode 100644 index 0000000000000..4169d3530a389 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/ListSchemaRegistryClustersResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Result of GET request to list schema registry clusters in the environment of a confluent organization. + */ +@Fluent +public final class ListSchemaRegistryClustersResponse { + /* + * List of schema registry clusters in an environment of a confluent organization + */ + @JsonProperty(value = "value") + private List value; + + /* + * URL to get the next set of schema registry cluster records if there are any. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of ListSchemaRegistryClustersResponse class. + */ + public ListSchemaRegistryClustersResponse() { + } + + /** + * Get the value property: List of schema registry clusters in an environment of a confluent organization. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: List of schema registry clusters in an environment of a confluent organization. + * + * @param value the value value to set. + * @return the ListSchemaRegistryClustersResponse object itself. + */ + public ListSchemaRegistryClustersResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: URL to get the next set of schema registry cluster records if there are any. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: URL to get the next set of schema registry cluster records if there are any. + * + * @param nextLink the nextLink value to set. + * @return the ListSchemaRegistryClustersResponse object itself. + */ + public ListSchemaRegistryClustersResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/OrganizationResource.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/OrganizationResource.java index 074b3cdb45ebf..f59edff6ff9f1 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/OrganizationResource.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/OrganizationResource.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.confluent.models; +import com.azure.core.http.rest.Response; import com.azure.core.management.Region; import com.azure.core.management.SystemData; import com.azure.core.util.Context; @@ -182,7 +183,7 @@ interface WithResourceGroup { /** * Specifies resourceGroupName. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @return the next definition stage. */ WithOfferDetail withExistingResourceGroup(String resourceGroupName); @@ -321,4 +322,27 @@ interface WithTags { * @return the refreshed resource. */ OrganizationResource refresh(Context context); + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent along with {@link Response}. + */ + Response listRegionsWithResponse(ListAccessRequestModel body, Context context); + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent. + */ + ListRegionsSuccessResponse listRegions(ListAccessRequestModel body); } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Organizations.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Organizations.java index 3946f337803f1..1147fad1172bb 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Organizations.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Organizations.java @@ -35,7 +35,7 @@ public interface Organizations { /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -46,7 +46,7 @@ public interface Organizations { /** * List all Organizations under the specified resource group. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -58,7 +58,7 @@ public interface Organizations { /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -72,7 +72,7 @@ Response getByResourceGroupWithResponse(String resourceGro /** * Get the properties of a specific Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -84,7 +84,7 @@ Response getByResourceGroupWithResponse(String resourceGro /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -95,7 +95,7 @@ Response getByResourceGroupWithResponse(String resourceGro /** * Delete Organization resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -104,6 +104,307 @@ Response getByResourceGroupWithResponse(String resourceGro */ void delete(String resourceGroupName, String organizationName, Context context); + /** + * Lists of all the environments in a organization. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedIterable}. + */ + PagedIterable listEnvironments(String resourceGroupName, String organizationName); + + /** + * Lists of all the environments in a organization. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list Confluent operations as paginated response with {@link PagedIterable}. + */ + PagedIterable listEnvironments(String resourceGroupName, String organizationName, + Integer pageSize, String pageToken, Context context); + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id along with {@link Response}. + */ + Response getEnvironmentByIdWithResponse(String resourceGroupName, String organizationName, + String environmentId, Context context); + + /** + * Get Environment details by environment Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return environment details by environment Id. + */ + SCEnvironmentRecord getEnvironmentById(String resourceGroupName, String organizationName, String environmentId); + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedIterable}. + */ + PagedIterable listClusters(String resourceGroupName, String organizationName, + String environmentId); + + /** + * Lists of all the clusters in a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of GET request to list clusters in the environment of a confluent organization as paginated + * response with {@link PagedIterable}. + */ + PagedIterable listClusters(String resourceGroupName, String organizationName, String environmentId, + Integer pageSize, String pageToken, Context context); + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedIterable}. + */ + PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId); + + /** + * Get schema registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param pageSize Pagination size. + * @param pageToken An opaque pagination token to fetch the next set of records. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry clusters as paginated response with {@link PagedIterable}. + */ + PagedIterable listSchemaRegistryClusters(String resourceGroupName, + String organizationName, String environmentId, Integer pageSize, String pageToken, Context context); + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent along with {@link Response}. + */ + Response listRegionsWithResponse(String resourceGroupName, String organizationName, + ListAccessRequestModel body, Context context); + + /** + * cloud provider regions available for creating Schema Registry clusters. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param body List Access Request Model. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return result of POST request to list regions supported by confluent. + */ + ListRegionsSuccessResponse listRegions(String resourceGroupName, String organizationName, + ListAccessRequestModel body); + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key along with {@link Response}. + */ + Response createApiKeyWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, CreateApiKeyModel body, Context context); + + /** + * Creates API key for a schema registry Cluster ID or Kafka Cluster ID under a environment. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param body Request payload for get creating API Key for schema registry Cluster ID or Kafka Cluster ID under a + * environment. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return details API key. + */ + ApiKeyRecord createApiKey(String resourceGroupName, String organizationName, String environmentId, String clusterId, + CreateApiKeyModel body); + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response deleteClusterApiKeyWithResponse(String resourceGroupName, String organizationName, String apiKeyId, + Context context); + + /** + * Deletes API key of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId); + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster along with {@link Response}. + */ + Response getClusterApiKeyWithResponse(String resourceGroupName, String organizationName, + String apiKeyId, Context context); + + /** + * Get API key details of a kafka or schema registry cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param apiKeyId Confluent API Key id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return aPI key details of a kafka or schema registry cluster. + */ + ApiKeyRecord getClusterApiKey(String resourceGroupName, String organizationName, String apiKeyId); + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id along with {@link Response}. + */ + Response getSchemaRegistryClusterByIdWithResponse(String resourceGroupName, + String organizationName, String environmentId, String clusterId, Context context); + + /** + * Get schema registry cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return schema registry cluster by Id. + */ + SchemaRegistryClusterRecord getSchemaRegistryClusterById(String resourceGroupName, String organizationName, + String environmentId, String clusterId); + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id along with {@link Response}. + */ + Response getClusterByIdWithResponse(String resourceGroupName, String organizationName, + String environmentId, String clusterId, Context context); + + /** + * Get cluster by Id. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param organizationName Organization resource name. + * @param environmentId Confluent environment id. + * @param clusterId Confluent kafka or schema registry cluster id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return cluster by Id. + */ + SCClusterRecord getClusterById(String resourceGroupName, String organizationName, String environmentId, + String clusterId); + /** * Get the properties of a specific Organization resource. * diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionRecord.java new file mode 100644 index 0000000000000..6464be6080629 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionRecord.java @@ -0,0 +1,145 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.confluent.fluent.models.RegionProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Details of region record. + */ +@Fluent +public final class RegionRecord { + /* + * Kind of the cluster + */ + @JsonProperty(value = "kind") + private String kind; + + /* + * Id of the cluster + */ + @JsonProperty(value = "id") + private String id; + + /* + * Region Properties + */ + @JsonProperty(value = "properties") + private RegionProperties innerProperties; + + /** + * Creates an instance of RegionRecord class. + */ + public RegionRecord() { + } + + /** + * Get the kind property: Kind of the cluster. + * + * @return the kind value. + */ + public String kind() { + return this.kind; + } + + /** + * Set the kind property: Kind of the cluster. + * + * @param kind the kind value to set. + * @return the RegionRecord object itself. + */ + public RegionRecord withKind(String kind) { + this.kind = kind; + return this; + } + + /** + * Get the id property: Id of the cluster. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Id of the cluster. + * + * @param id the id value to set. + * @return the RegionRecord object itself. + */ + public RegionRecord withId(String id) { + this.id = id; + return this; + } + + /** + * Get the innerProperties property: Region Properties. + * + * @return the innerProperties value. + */ + private RegionProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + public SCMetadataEntity metadata() { + return this.innerProperties() == null ? null : this.innerProperties().metadata(); + } + + /** + * Set the metadata property: Metadata of the record. + * + * @param metadata the metadata value to set. + * @return the RegionRecord object itself. + */ + public RegionRecord withMetadata(SCMetadataEntity metadata) { + if (this.innerProperties() == null) { + this.innerProperties = new RegionProperties(); + } + this.innerProperties().withMetadata(metadata); + return this; + } + + /** + * Get the spec property: Specification of the region. + * + * @return the spec value. + */ + public RegionSpecEntity spec() { + return this.innerProperties() == null ? null : this.innerProperties().spec(); + } + + /** + * Set the spec property: Specification of the region. + * + * @param spec the spec value to set. + * @return the RegionRecord object itself. + */ + public RegionRecord withSpec(RegionSpecEntity spec) { + if (this.innerProperties() == null) { + this.innerProperties = new RegionProperties(); + } + this.innerProperties().withSpec(spec); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionSpecEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionSpecEntity.java new file mode 100644 index 0000000000000..0572837a43354 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RegionSpecEntity.java @@ -0,0 +1,133 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Region spec details. + */ +@Fluent +public final class RegionSpecEntity { + /* + * Display Name of the region + */ + @JsonProperty(value = "name") + private String name; + + /* + * Cloud provider name + */ + @JsonProperty(value = "cloud") + private String cloud; + + /* + * Region name + */ + @JsonProperty(value = "regionName") + private String regionName; + + /* + * The packages property. + */ + @JsonProperty(value = "packages") + private List packages; + + /** + * Creates an instance of RegionSpecEntity class. + */ + public RegionSpecEntity() { + } + + /** + * Get the name property: Display Name of the region. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Display Name of the region. + * + * @param name the name value to set. + * @return the RegionSpecEntity object itself. + */ + public RegionSpecEntity withName(String name) { + this.name = name; + return this; + } + + /** + * Get the cloud property: Cloud provider name. + * + * @return the cloud value. + */ + public String cloud() { + return this.cloud; + } + + /** + * Set the cloud property: Cloud provider name. + * + * @param cloud the cloud value to set. + * @return the RegionSpecEntity object itself. + */ + public RegionSpecEntity withCloud(String cloud) { + this.cloud = cloud; + return this; + } + + /** + * Get the regionName property: Region name. + * + * @return the regionName value. + */ + public String regionName() { + return this.regionName; + } + + /** + * Set the regionName property: Region name. + * + * @param regionName the regionName value to set. + * @return the RegionSpecEntity object itself. + */ + public RegionSpecEntity withRegionName(String regionName) { + this.regionName = regionName; + return this; + } + + /** + * Get the packages property: The packages property. + * + * @return the packages value. + */ + public List packages() { + return this.packages; + } + + /** + * Set the packages property: The packages property. + * + * @param packages the packages value to set. + * @return the RegionSpecEntity object itself. + */ + public RegionSpecEntity withPackages(List packages) { + this.packages = packages; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RoleBindingRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RoleBindingRecord.java index 6d4aefb099b2d..4fe3ef81162d5 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RoleBindingRecord.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/RoleBindingRecord.java @@ -8,7 +8,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * Record of the environment. + * Details on principal, role name and crn pattern of a role binding. */ @Fluent public final class RoleBindingRecord { @@ -19,7 +19,7 @@ public final class RoleBindingRecord { private String kind; /* - * Id of the role + * Id of the role binding */ @JsonProperty(value = "id") private String id; @@ -75,7 +75,7 @@ public RoleBindingRecord withKind(String kind) { } /** - * Get the id property: Id of the role. + * Get the id property: Id of the role binding. * * @return the id value. */ @@ -84,7 +84,7 @@ public String id() { } /** - * Set the id property: Id of the role. + * Set the id property: Id of the role binding. * * @param id the id value to set. * @return the RoleBindingRecord object itself. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterByokEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterByokEntity.java new file mode 100644 index 0000000000000..32a6536074b79 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterByokEntity.java @@ -0,0 +1,106 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The network associated with this object. + */ +@Fluent +public final class SCClusterByokEntity { + /* + * ID of the referred resource + */ + @JsonProperty(value = "id") + private String id; + + /* + * API URL for accessing or modifying the referred object + */ + @JsonProperty(value = "related") + private String related; + + /* + * CRN reference to the referred resource + */ + @JsonProperty(value = "resourceName") + private String resourceName; + + /** + * Creates an instance of SCClusterByokEntity class. + */ + public SCClusterByokEntity() { + } + + /** + * Get the id property: ID of the referred resource. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: ID of the referred resource. + * + * @param id the id value to set. + * @return the SCClusterByokEntity object itself. + */ + public SCClusterByokEntity withId(String id) { + this.id = id; + return this; + } + + /** + * Get the related property: API URL for accessing or modifying the referred object. + * + * @return the related value. + */ + public String related() { + return this.related; + } + + /** + * Set the related property: API URL for accessing or modifying the referred object. + * + * @param related the related value to set. + * @return the SCClusterByokEntity object itself. + */ + public SCClusterByokEntity withRelated(String related) { + this.related = related; + return this; + } + + /** + * Get the resourceName property: CRN reference to the referred resource. + * + * @return the resourceName value. + */ + public String resourceName() { + return this.resourceName; + } + + /** + * Set the resourceName property: CRN reference to the referred resource. + * + * @param resourceName the resourceName value to set. + * @return the SCClusterByokEntity object itself. + */ + public SCClusterByokEntity withResourceName(String resourceName) { + this.resourceName = resourceName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterNetworkEnvironmentEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterNetworkEnvironmentEntity.java new file mode 100644 index 0000000000000..855f378759641 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterNetworkEnvironmentEntity.java @@ -0,0 +1,132 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The environment or the network to which cluster belongs. + */ +@Fluent +public final class SCClusterNetworkEnvironmentEntity { + /* + * ID of the referred resource + */ + @JsonProperty(value = "id") + private String id; + + /* + * Environment of the referred resource + */ + @JsonProperty(value = "environment") + private String environment; + + /* + * API URL for accessing or modifying the referred object + */ + @JsonProperty(value = "related") + private String related; + + /* + * CRN reference to the referred resource + */ + @JsonProperty(value = "resourceName") + private String resourceName; + + /** + * Creates an instance of SCClusterNetworkEnvironmentEntity class. + */ + public SCClusterNetworkEnvironmentEntity() { + } + + /** + * Get the id property: ID of the referred resource. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: ID of the referred resource. + * + * @param id the id value to set. + * @return the SCClusterNetworkEnvironmentEntity object itself. + */ + public SCClusterNetworkEnvironmentEntity withId(String id) { + this.id = id; + return this; + } + + /** + * Get the environment property: Environment of the referred resource. + * + * @return the environment value. + */ + public String environment() { + return this.environment; + } + + /** + * Set the environment property: Environment of the referred resource. + * + * @param environment the environment value to set. + * @return the SCClusterNetworkEnvironmentEntity object itself. + */ + public SCClusterNetworkEnvironmentEntity withEnvironment(String environment) { + this.environment = environment; + return this; + } + + /** + * Get the related property: API URL for accessing or modifying the referred object. + * + * @return the related value. + */ + public String related() { + return this.related; + } + + /** + * Set the related property: API URL for accessing or modifying the referred object. + * + * @param related the related value to set. + * @return the SCClusterNetworkEnvironmentEntity object itself. + */ + public SCClusterNetworkEnvironmentEntity withRelated(String related) { + this.related = related; + return this; + } + + /** + * Get the resourceName property: CRN reference to the referred resource. + * + * @return the resourceName value. + */ + public String resourceName() { + return this.resourceName; + } + + /** + * Set the resourceName property: CRN reference to the referred resource. + * + * @param resourceName the resourceName value to set. + * @return the SCClusterNetworkEnvironmentEntity object itself. + */ + public SCClusterNetworkEnvironmentEntity withResourceName(String resourceName) { + this.resourceName = resourceName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterRecord.java new file mode 100644 index 0000000000000..1824b1ba6ed7d --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterRecord.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; + +/** + * An immutable client-side representation of SCClusterRecord. + */ +public interface SCClusterRecord { + /** + * Gets the kind property: Type of cluster. + * + * @return the kind value. + */ + String kind(); + + /** + * Gets the id property: Id of the cluster. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: Display name of the cluster. + * + * @return the name value. + */ + String name(); + + /** + * Gets the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + SCMetadataEntity metadata(); + + /** + * Gets the spec property: Specification of the cluster. + * + * @return the spec value. + */ + SCClusterSpecEntity spec(); + + /** + * Gets the status property: Specification of the cluster status. + * + * @return the status value. + */ + ClusterStatusEntity status(); + + /** + * Gets the inner com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner object. + * + * @return the inner object. + */ + SCClusterRecordInner innerModel(); +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterSpecEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterSpecEntity.java new file mode 100644 index 0000000000000..670a15e225450 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCClusterSpecEntity.java @@ -0,0 +1,352 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Spec of the cluster record. + */ +@Fluent +public final class SCClusterSpecEntity { + /* + * The name of the cluster + */ + @JsonProperty(value = "name") + private String name; + + /* + * The availability zone configuration of the cluster + */ + @JsonProperty(value = "availability") + private String availability; + + /* + * The cloud service provider + */ + @JsonProperty(value = "cloud") + private String cloud; + + /* + * type of zone availability + */ + @JsonProperty(value = "zone") + private String zone; + + /* + * The cloud service provider region + */ + @JsonProperty(value = "region") + private String region; + + /* + * The bootstrap endpoint used by Kafka clients to connect to the cluster + */ + @JsonProperty(value = "kafkaBootstrapEndpoint") + private String kafkaBootstrapEndpoint; + + /* + * The cluster HTTP request URL. + */ + @JsonProperty(value = "httpEndpoint") + private String httpEndpoint; + + /* + * The Kafka API cluster endpoint + */ + @JsonProperty(value = "apiEndpoint") + private String apiEndpoint; + + /* + * Specification of the cluster configuration + */ + @JsonProperty(value = "config") + private ClusterConfigEntity config; + + /* + * Specification of the cluster environment + */ + @JsonProperty(value = "environment") + private SCClusterNetworkEnvironmentEntity environment; + + /* + * Specification of the cluster network + */ + @JsonProperty(value = "network") + private SCClusterNetworkEnvironmentEntity network; + + /* + * Specification of the cluster byok + */ + @JsonProperty(value = "byok") + private SCClusterByokEntity byok; + + /** + * Creates an instance of SCClusterSpecEntity class. + */ + public SCClusterSpecEntity() { + } + + /** + * Get the name property: The name of the cluster. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: The name of the cluster. + * + * @param name the name value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withName(String name) { + this.name = name; + return this; + } + + /** + * Get the availability property: The availability zone configuration of the cluster. + * + * @return the availability value. + */ + public String availability() { + return this.availability; + } + + /** + * Set the availability property: The availability zone configuration of the cluster. + * + * @param availability the availability value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withAvailability(String availability) { + this.availability = availability; + return this; + } + + /** + * Get the cloud property: The cloud service provider. + * + * @return the cloud value. + */ + public String cloud() { + return this.cloud; + } + + /** + * Set the cloud property: The cloud service provider. + * + * @param cloud the cloud value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withCloud(String cloud) { + this.cloud = cloud; + return this; + } + + /** + * Get the zone property: type of zone availability. + * + * @return the zone value. + */ + public String zone() { + return this.zone; + } + + /** + * Set the zone property: type of zone availability. + * + * @param zone the zone value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withZone(String zone) { + this.zone = zone; + return this; + } + + /** + * Get the region property: The cloud service provider region. + * + * @return the region value. + */ + public String region() { + return this.region; + } + + /** + * Set the region property: The cloud service provider region. + * + * @param region the region value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withRegion(String region) { + this.region = region; + return this; + } + + /** + * Get the kafkaBootstrapEndpoint property: The bootstrap endpoint used by Kafka clients to connect to the cluster. + * + * @return the kafkaBootstrapEndpoint value. + */ + public String kafkaBootstrapEndpoint() { + return this.kafkaBootstrapEndpoint; + } + + /** + * Set the kafkaBootstrapEndpoint property: The bootstrap endpoint used by Kafka clients to connect to the cluster. + * + * @param kafkaBootstrapEndpoint the kafkaBootstrapEndpoint value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withKafkaBootstrapEndpoint(String kafkaBootstrapEndpoint) { + this.kafkaBootstrapEndpoint = kafkaBootstrapEndpoint; + return this; + } + + /** + * Get the httpEndpoint property: The cluster HTTP request URL. + * + * @return the httpEndpoint value. + */ + public String httpEndpoint() { + return this.httpEndpoint; + } + + /** + * Set the httpEndpoint property: The cluster HTTP request URL. + * + * @param httpEndpoint the httpEndpoint value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withHttpEndpoint(String httpEndpoint) { + this.httpEndpoint = httpEndpoint; + return this; + } + + /** + * Get the apiEndpoint property: The Kafka API cluster endpoint. + * + * @return the apiEndpoint value. + */ + public String apiEndpoint() { + return this.apiEndpoint; + } + + /** + * Set the apiEndpoint property: The Kafka API cluster endpoint. + * + * @param apiEndpoint the apiEndpoint value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withApiEndpoint(String apiEndpoint) { + this.apiEndpoint = apiEndpoint; + return this; + } + + /** + * Get the config property: Specification of the cluster configuration. + * + * @return the config value. + */ + public ClusterConfigEntity config() { + return this.config; + } + + /** + * Set the config property: Specification of the cluster configuration. + * + * @param config the config value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withConfig(ClusterConfigEntity config) { + this.config = config; + return this; + } + + /** + * Get the environment property: Specification of the cluster environment. + * + * @return the environment value. + */ + public SCClusterNetworkEnvironmentEntity environment() { + return this.environment; + } + + /** + * Set the environment property: Specification of the cluster environment. + * + * @param environment the environment value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withEnvironment(SCClusterNetworkEnvironmentEntity environment) { + this.environment = environment; + return this; + } + + /** + * Get the network property: Specification of the cluster network. + * + * @return the network value. + */ + public SCClusterNetworkEnvironmentEntity network() { + return this.network; + } + + /** + * Set the network property: Specification of the cluster network. + * + * @param network the network value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withNetwork(SCClusterNetworkEnvironmentEntity network) { + this.network = network; + return this; + } + + /** + * Get the byok property: Specification of the cluster byok. + * + * @return the byok value. + */ + public SCClusterByokEntity byok() { + return this.byok; + } + + /** + * Set the byok property: Specification of the cluster byok. + * + * @param byok the byok value to set. + * @return the SCClusterSpecEntity object itself. + */ + public SCClusterSpecEntity withByok(SCClusterByokEntity byok) { + this.byok = byok; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (config() != null) { + config().validate(); + } + if (environment() != null) { + environment().validate(); + } + if (network() != null) { + network().validate(); + } + if (byok() != null) { + byok().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCEnvironmentRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCEnvironmentRecord.java new file mode 100644 index 0000000000000..2396cb1ca85fc --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCEnvironmentRecord.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; + +/** + * An immutable client-side representation of SCEnvironmentRecord. + */ +public interface SCEnvironmentRecord { + /** + * Gets the kind property: Type of environment. + * + * @return the kind value. + */ + String kind(); + + /** + * Gets the id property: Id of the environment. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: Display name of the environment. + * + * @return the name value. + */ + String name(); + + /** + * Gets the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + SCMetadataEntity metadata(); + + /** + * Gets the inner com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner object. + * + * @return the inner object. + */ + SCEnvironmentRecordInner innerModel(); +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCMetadataEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCMetadataEntity.java new file mode 100644 index 0000000000000..3fe85ea220a8a --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SCMetadataEntity.java @@ -0,0 +1,158 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Metadata of the data record. + */ +@Fluent +public final class SCMetadataEntity { + /* + * Self lookup url + */ + @JsonProperty(value = "self") + private String self; + + /* + * Resource name of the record + */ + @JsonProperty(value = "resourceName") + private String resourceName; + + /* + * Created Date Time + */ + @JsonProperty(value = "createdTimestamp") + private String createdTimestamp; + + /* + * Updated Date time + */ + @JsonProperty(value = "updatedTimestamp") + private String updatedTimestamp; + + /* + * Deleted Date time + */ + @JsonProperty(value = "deletedTimestamp") + private String deletedTimestamp; + + /** + * Creates an instance of SCMetadataEntity class. + */ + public SCMetadataEntity() { + } + + /** + * Get the self property: Self lookup url. + * + * @return the self value. + */ + public String self() { + return this.self; + } + + /** + * Set the self property: Self lookup url. + * + * @param self the self value to set. + * @return the SCMetadataEntity object itself. + */ + public SCMetadataEntity withSelf(String self) { + this.self = self; + return this; + } + + /** + * Get the resourceName property: Resource name of the record. + * + * @return the resourceName value. + */ + public String resourceName() { + return this.resourceName; + } + + /** + * Set the resourceName property: Resource name of the record. + * + * @param resourceName the resourceName value to set. + * @return the SCMetadataEntity object itself. + */ + public SCMetadataEntity withResourceName(String resourceName) { + this.resourceName = resourceName; + return this; + } + + /** + * Get the createdTimestamp property: Created Date Time. + * + * @return the createdTimestamp value. + */ + public String createdTimestamp() { + return this.createdTimestamp; + } + + /** + * Set the createdTimestamp property: Created Date Time. + * + * @param createdTimestamp the createdTimestamp value to set. + * @return the SCMetadataEntity object itself. + */ + public SCMetadataEntity withCreatedTimestamp(String createdTimestamp) { + this.createdTimestamp = createdTimestamp; + return this; + } + + /** + * Get the updatedTimestamp property: Updated Date time. + * + * @return the updatedTimestamp value. + */ + public String updatedTimestamp() { + return this.updatedTimestamp; + } + + /** + * Set the updatedTimestamp property: Updated Date time. + * + * @param updatedTimestamp the updatedTimestamp value to set. + * @return the SCMetadataEntity object itself. + */ + public SCMetadataEntity withUpdatedTimestamp(String updatedTimestamp) { + this.updatedTimestamp = updatedTimestamp; + return this; + } + + /** + * Get the deletedTimestamp property: Deleted Date time. + * + * @return the deletedTimestamp value. + */ + public String deletedTimestamp() { + return this.deletedTimestamp; + } + + /** + * Set the deletedTimestamp property: Deleted Date time. + * + * @param deletedTimestamp the deletedTimestamp value to set. + * @return the SCMetadataEntity object itself. + */ + public SCMetadataEntity withDeletedTimestamp(String deletedTimestamp) { + this.deletedTimestamp = deletedTimestamp; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterEnvironmentRegionEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterEnvironmentRegionEntity.java new file mode 100644 index 0000000000000..cde7276bbcc30 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterEnvironmentRegionEntity.java @@ -0,0 +1,106 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The environment associated with this object. + */ +@Fluent +public final class SchemaRegistryClusterEnvironmentRegionEntity { + /* + * ID of the referred resource + */ + @JsonProperty(value = "id") + private String id; + + /* + * API URL for accessing or modifying the referred object + */ + @JsonProperty(value = "related") + private String related; + + /* + * CRN reference to the referred resource + */ + @JsonProperty(value = "resourceName") + private String resourceName; + + /** + * Creates an instance of SchemaRegistryClusterEnvironmentRegionEntity class. + */ + public SchemaRegistryClusterEnvironmentRegionEntity() { + } + + /** + * Get the id property: ID of the referred resource. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: ID of the referred resource. + * + * @param id the id value to set. + * @return the SchemaRegistryClusterEnvironmentRegionEntity object itself. + */ + public SchemaRegistryClusterEnvironmentRegionEntity withId(String id) { + this.id = id; + return this; + } + + /** + * Get the related property: API URL for accessing or modifying the referred object. + * + * @return the related value. + */ + public String related() { + return this.related; + } + + /** + * Set the related property: API URL for accessing or modifying the referred object. + * + * @param related the related value to set. + * @return the SchemaRegistryClusterEnvironmentRegionEntity object itself. + */ + public SchemaRegistryClusterEnvironmentRegionEntity withRelated(String related) { + this.related = related; + return this; + } + + /** + * Get the resourceName property: CRN reference to the referred resource. + * + * @return the resourceName value. + */ + public String resourceName() { + return this.resourceName; + } + + /** + * Set the resourceName property: CRN reference to the referred resource. + * + * @param resourceName the resourceName value to set. + * @return the SchemaRegistryClusterEnvironmentRegionEntity object itself. + */ + public SchemaRegistryClusterEnvironmentRegionEntity withResourceName(String resourceName) { + this.resourceName = resourceName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterRecord.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterRecord.java new file mode 100644 index 0000000000000..06c2f45b6da69 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterRecord.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; + +/** + * An immutable client-side representation of SchemaRegistryClusterRecord. + */ +public interface SchemaRegistryClusterRecord { + /** + * Gets the kind property: Kind of the cluster. + * + * @return the kind value. + */ + String kind(); + + /** + * Gets the id property: Id of the cluster. + * + * @return the id value. + */ + String id(); + + /** + * Gets the metadata property: Metadata of the record. + * + * @return the metadata value. + */ + SCMetadataEntity metadata(); + + /** + * Gets the spec property: Specification of the schema registry cluster. + * + * @return the spec value. + */ + SchemaRegistryClusterSpecEntity spec(); + + /** + * Gets the status property: Specification of the cluster status. + * + * @return the status value. + */ + SchemaRegistryClusterStatusEntity status(); + + /** + * Gets the inner com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner object. + * + * @return the inner object. + */ + SchemaRegistryClusterRecordInner innerModel(); +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterSpecEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterSpecEntity.java new file mode 100644 index 0000000000000..d5344cfd11e94 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterSpecEntity.java @@ -0,0 +1,190 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Details of schema registry cluster spec. + */ +@Fluent +public final class SchemaRegistryClusterSpecEntity { + /* + * Name of the schema registry cluster + */ + @JsonProperty(value = "name") + private String name; + + /* + * Http endpoint of the cluster + */ + @JsonProperty(value = "httpEndpoint") + private String httpEndpoint; + + /* + * Type of the cluster package Advanced, essentials + */ + @JsonProperty(value = "package") + private String packageProperty; + + /* + * Region details of the schema registry cluster + */ + @JsonProperty(value = "region") + private SchemaRegistryClusterEnvironmentRegionEntity region; + + /* + * Environment details of the schema registry cluster + */ + @JsonProperty(value = "environment") + private SchemaRegistryClusterEnvironmentRegionEntity environment; + + /* + * The cloud service provider + */ + @JsonProperty(value = "cloud") + private String cloud; + + /** + * Creates an instance of SchemaRegistryClusterSpecEntity class. + */ + public SchemaRegistryClusterSpecEntity() { + } + + /** + * Get the name property: Name of the schema registry cluster. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the schema registry cluster. + * + * @param name the name value to set. + * @return the SchemaRegistryClusterSpecEntity object itself. + */ + public SchemaRegistryClusterSpecEntity withName(String name) { + this.name = name; + return this; + } + + /** + * Get the httpEndpoint property: Http endpoint of the cluster. + * + * @return the httpEndpoint value. + */ + public String httpEndpoint() { + return this.httpEndpoint; + } + + /** + * Set the httpEndpoint property: Http endpoint of the cluster. + * + * @param httpEndpoint the httpEndpoint value to set. + * @return the SchemaRegistryClusterSpecEntity object itself. + */ + public SchemaRegistryClusterSpecEntity withHttpEndpoint(String httpEndpoint) { + this.httpEndpoint = httpEndpoint; + return this; + } + + /** + * Get the packageProperty property: Type of the cluster package Advanced, essentials. + * + * @return the packageProperty value. + */ + public String packageProperty() { + return this.packageProperty; + } + + /** + * Set the packageProperty property: Type of the cluster package Advanced, essentials. + * + * @param packageProperty the packageProperty value to set. + * @return the SchemaRegistryClusterSpecEntity object itself. + */ + public SchemaRegistryClusterSpecEntity withPackageProperty(String packageProperty) { + this.packageProperty = packageProperty; + return this; + } + + /** + * Get the region property: Region details of the schema registry cluster. + * + * @return the region value. + */ + public SchemaRegistryClusterEnvironmentRegionEntity region() { + return this.region; + } + + /** + * Set the region property: Region details of the schema registry cluster. + * + * @param region the region value to set. + * @return the SchemaRegistryClusterSpecEntity object itself. + */ + public SchemaRegistryClusterSpecEntity withRegion(SchemaRegistryClusterEnvironmentRegionEntity region) { + this.region = region; + return this; + } + + /** + * Get the environment property: Environment details of the schema registry cluster. + * + * @return the environment value. + */ + public SchemaRegistryClusterEnvironmentRegionEntity environment() { + return this.environment; + } + + /** + * Set the environment property: Environment details of the schema registry cluster. + * + * @param environment the environment value to set. + * @return the SchemaRegistryClusterSpecEntity object itself. + */ + public SchemaRegistryClusterSpecEntity withEnvironment(SchemaRegistryClusterEnvironmentRegionEntity environment) { + this.environment = environment; + return this; + } + + /** + * Get the cloud property: The cloud service provider. + * + * @return the cloud value. + */ + public String cloud() { + return this.cloud; + } + + /** + * Set the cloud property: The cloud service provider. + * + * @param cloud the cloud value to set. + * @return the SchemaRegistryClusterSpecEntity object itself. + */ + public SchemaRegistryClusterSpecEntity withCloud(String cloud) { + this.cloud = cloud; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (region() != null) { + region().validate(); + } + if (environment() != null) { + environment().validate(); + } + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterStatusEntity.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterStatusEntity.java new file mode 100644 index 0000000000000..bb38ca321ba08 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/SchemaRegistryClusterStatusEntity.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Status of the schema registry cluster record. + */ +@Fluent +public final class SchemaRegistryClusterStatusEntity { + /* + * The lifecycle phase of the cluster + */ + @JsonProperty(value = "phase") + private String phase; + + /** + * Creates an instance of SchemaRegistryClusterStatusEntity class. + */ + public SchemaRegistryClusterStatusEntity() { + } + + /** + * Get the phase property: The lifecycle phase of the cluster. + * + * @return the phase value. + */ + public String phase() { + return this.phase; + } + + /** + * Set the phase property: The lifecycle phase of the cluster. + * + * @param phase the phase value to set. + * @return the SchemaRegistryClusterStatusEntity object itself. + */ + public SchemaRegistryClusterStatusEntity withPhase(String phase) { + this.phase = phase; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Validations.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Validations.java index 32d9bae53f753..dde40740bb3f9 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Validations.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/com/azure/resourcemanager/confluent/models/Validations.java @@ -15,7 +15,7 @@ public interface Validations { /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -30,7 +30,7 @@ Response validateOrganizationWithResponse(String resourceG /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -44,7 +44,7 @@ OrganizationResource validateOrganization(String resourceGroupName, String organ /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @param context The context to associate with this operation. @@ -59,7 +59,7 @@ Response validateOrganizationV2WithResponse(String resourceG /** * Organization Validate proxy resource. * - * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param resourceGroupName Resource group name. * @param organizationName Organization resource name. * @param body Organization resource model. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/module-info.java b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/module-info.java index 074b073bf4834..d2bda5487cba1 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/java/module-info.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/java/module-info.java @@ -4,12 +4,10 @@ module com.azure.resourcemanager.confluent { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.confluent; exports com.azure.resourcemanager.confluent.fluent; exports com.azure.resourcemanager.confluent.fluent.models; exports com.azure.resourcemanager.confluent.models; - opens com.azure.resourcemanager.confluent.fluent.models to com.azure.core, com.fasterxml.jackson.databind; opens com.azure.resourcemanager.confluent.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-confluent/reflect-config.json b/sdk/confluent/azure-resourcemanager-confluent/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-confluent/reflect-config.json index 127ecf52333f7..5147677cd0f51 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-confluent/reflect-config.json +++ b/sdk/confluent/azure-resourcemanager-confluent/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-confluent/reflect-config.json @@ -183,6 +183,146 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.GetEnvironmentsResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.EnvironmentProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SCMetadataEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.ListClustersSuccessResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.ClusterProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SCClusterSpecEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SCClusterNetworkEnvironmentEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SCClusterByokEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.ListSchemaRegistryClustersResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SchemaRegistryClusterEnvironmentRegionEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.RegionRecord", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.RegionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.RegionSpecEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.CreateApiKeyModel", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.ApiKeyRecordInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.fluent.models.ApiKeyProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.ApiKeySpecEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.ApiKeyResourceEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.confluent.models.ApiKeyOwnerEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.confluent.models.ProvisionState", "allDeclaredConstructors" : true, diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingSamples.java new file mode 100644 index 0000000000000..bfa10615dcbfd --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingSamples.java @@ -0,0 +1,30 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; + +/** + * Samples for Access CreateRoleBinding. + */ +public final class AccessCreateRoleBindingSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_CreateRoleBinding. + * json + */ + /** + * Sample code: Access_CreateRoleBinding. + * + * @param manager Entry point to ConfluentManager. + */ + public static void accessCreateRoleBinding(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.access().createRoleBindingWithResponse("myResourceGroup", "myOrganization", + new AccessCreateRoleBindingRequestModel().withPrincipal("User:u-111aaa").withRoleName("CloudClusterAdmin") + .withCrnPattern( + "crn://confluent.cloud/organization=1111aaaa-11aa-11aa-11aa-111111aaaaaa/environment=env-aaa1111/cloud-cluster=lkc-1111aaa"), + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingSamples.java new file mode 100644 index 0000000000000..74795c2584180 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Access DeleteRoleBinding. + */ +public final class AccessDeleteRoleBindingSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_DeleteRoleBinding. + * json + */ + /** + * Sample code: Access_DeleteRoleBinding. + * + * @param manager Entry point to ConfluentManager. + */ + public static void accessDeleteRoleBinding(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.access().deleteRoleBindingWithResponse("myResourceGroup", "myOrganization", "dlz-f3a90de", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserSamples.java index f93a052124c1f..4753851e47553 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserSamples.java @@ -13,7 +13,7 @@ public final class AccessInviteUserSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_InviteUser.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_InviteUser.json */ /** * Sample code: Access_InviteUser. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListClustersSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListClustersSamples.java index 09e61bc535e87..01b3156b1e57e 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListClustersSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListClustersSamples.java @@ -14,7 +14,7 @@ public final class AccessListClustersSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_ClusterList.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_ClusterList.json */ /** * Sample code: Access_ClusterList. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsSamples.java index fa94baf15094b..6fb16bfd2248e 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsSamples.java @@ -14,7 +14,7 @@ public final class AccessListEnvironmentsSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_EnvironmentList. + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_EnvironmentList. * json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsSamples.java index 451c72a2a5d86..1aab342a80cb4 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsSamples.java @@ -14,7 +14,7 @@ public final class AccessListInvitationsSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_InvitationsList. + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_InvitationsList. * json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListSamples.java new file mode 100644 index 0000000000000..71c9f34068111 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListSamples.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Access ListRoleBindingNameList. + */ +public final class AccessListRoleBindingNameListSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Access_RoleBindingNameList.json + */ + /** + * Sample code: Access_RoleBindingNameList. + * + * @param manager Entry point to ConfluentManager. + */ + public static void accessRoleBindingNameList(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.access().listRoleBindingNameListWithResponse("myResourceGroup", "myOrganization", + new ListAccessRequestModel().withSearchFilters(mapOf("crn_pattern", + "crn://confluent.cloud/organization=1aa7de07-298e-479c-8f2f-16ac91fd8e76", "namespace", + "public,dataplane,networking,identity,datagovernance,connect,streamcatalog,pipelines,ksql")), + com.azure.core.util.Context.NONE); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsSamples.java index 08cf3f78797a7..396cbeeb868cc 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsSamples.java @@ -14,7 +14,7 @@ public final class AccessListRoleBindingsSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_RoleBindingList. + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_RoleBindingList. * json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsSamples.java index 84f622c510061..bb49ccc6042f5 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsSamples.java @@ -13,7 +13,7 @@ */ public final class AccessListServiceAccountsSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * Access_ServiceAccountsList.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListUsersSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListUsersSamples.java index b1af1ff14b275..af9dda39484cd 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListUsersSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/AccessListUsersSamples.java @@ -14,7 +14,7 @@ public final class AccessListUsersSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Access_UsersList.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Access_UsersList.json */ /** * Sample code: Access_UsersList. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateSamples.java index 9d19d658de9af..17406021a0cc3 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateSamples.java @@ -9,7 +9,7 @@ */ public final class MarketplaceAgreementsCreateSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * MarketplaceAgreements_Create.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListSamples.java index 5d65608c12d6e..bd445943d0bc6 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListSamples.java @@ -9,7 +9,7 @@ */ public final class MarketplaceAgreementsListSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * MarketplaceAgreements_List.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateApiKeySamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateApiKeySamples.java new file mode 100644 index 0000000000000..4f0cdcfeb38f8 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateApiKeySamples.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.resourcemanager.confluent.models.CreateApiKeyModel; + +/** + * Samples for Organization CreateApiKey. + */ +public final class OrganizationCreateApiKeySamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_CreateClusterAPIKey.json + */ + /** + * Sample code: Organization_CreateAPIKey. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationCreateAPIKey(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().createApiKeyWithResponse( + "myResourceGroup", "myOrganization", "env-12132", "clusterId-123", new CreateApiKeyModel() + .withName("CI kafka access key").withDescription("This API key provides kafka access to cluster x"), + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateSamples.java index db01d006f4bcc..99e4bf3b260f9 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationCreateSamples.java @@ -17,7 +17,7 @@ public final class OrganizationCreateSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Create.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Create.json */ /** * Sample code: Organization_Create. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteClusterApiKeySamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteClusterApiKeySamples.java new file mode 100644 index 0000000000000..8c21778613202 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteClusterApiKeySamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization DeleteClusterApiKey. + */ +public final class OrganizationDeleteClusterApiKeySamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_DeleteClusterAPIKey.json + */ + /** + * Sample code: Organization_DeleteClusterAPIKey. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationDeleteClusterAPIKey(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().deleteClusterApiKeyWithResponse("myResourceGroup", "myOrganization", "ZFZ6SZZZWGYBEIFB", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteSamples.java index 27b9453bbfc41..d320cb003005b 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationDeleteSamples.java @@ -10,7 +10,7 @@ public final class OrganizationDeleteSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Delete.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Delete.json */ /** * Sample code: Confluent_Delete. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetByResourceGroupSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetByResourceGroupSamples.java index 76f111765c7af..fe40b7424636a 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetByResourceGroupSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class OrganizationGetByResourceGroupSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Get.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Get.json */ /** * Sample code: Organization_Get. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterApiKeySamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterApiKeySamples.java new file mode 100644 index 0000000000000..3ff9ffe62d60f --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterApiKeySamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization GetClusterApiKey. + */ +public final class OrganizationGetClusterApiKeySamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetClusterAPIKey.json + */ + /** + * Sample code: Organization_GetClusterAPIKey. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationGetClusterAPIKey(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getClusterApiKeyWithResponse("myResourceGroup", "myOrganization", "apiKeyId-123", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterByIdSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterByIdSamples.java new file mode 100644 index 0000000000000..580a75e7f1b0a --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetClusterByIdSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization GetClusterById. + */ +public final class OrganizationGetClusterByIdSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetClusterById.json + */ + /** + * Sample code: Organization_GetClusterById. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationGetClusterById(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getClusterByIdWithResponse("myResourceGroup", "myOrganization", "env-12132", + "dlz-f3a90de", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetEnvironmentByIdSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetEnvironmentByIdSamples.java new file mode 100644 index 0000000000000..b72cccd3d4096 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetEnvironmentByIdSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization GetEnvironmentById. + */ +public final class OrganizationGetEnvironmentByIdSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetEnvironmentById.json + */ + /** + * Sample code: Organization_GetEnvironmentById. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationGetEnvironmentById(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getEnvironmentByIdWithResponse("myResourceGroup", "myOrganization", "dlz-f3a90de", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetSchemaRegistryClusterByIdSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetSchemaRegistryClusterByIdSamples.java new file mode 100644 index 0000000000000..da584b03de01e --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationGetSchemaRegistryClusterByIdSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization GetSchemaRegistryClusterById. + */ +public final class OrganizationGetSchemaRegistryClusterByIdSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_GetSchemaRegistryClusterById.json + */ + /** + * Sample code: Organization_GetSchemaRegistryClusterById. + * + * @param manager Entry point to ConfluentManager. + */ + public static void + organizationGetSchemaRegistryClusterById(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().getSchemaRegistryClusterByIdWithResponse("myResourceGroup", "myOrganization", + "env-stgcczjp2j3", "lsrc-stgczkq22z", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListByResourceGroupSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListByResourceGroupSamples.java index b9de4b954ba11..1ac26a5e67b33 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListByResourceGroupSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListByResourceGroupSamples.java @@ -9,7 +9,7 @@ */ public final class OrganizationListByResourceGroupSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * Organization_ListByResourceGroup.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListClustersSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListClustersSamples.java new file mode 100644 index 0000000000000..54bdc5d074493 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListClustersSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization ListClusters. + */ +public final class OrganizationListClustersSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_ClusterList. + * json + */ + /** + * Sample code: Organization_ListClusters. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationListClusters(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listClusters("myResourceGroup", "myOrganization", "env-12132", 10, null, + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListEnvironmentsSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListEnvironmentsSamples.java new file mode 100644 index 0000000000000..365a68c15f7a8 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListEnvironmentsSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization ListEnvironments. + */ +public final class OrganizationListEnvironmentsSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_EnvironmentList.json + */ + /** + * Sample code: Organization_ListEnvironments. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationListEnvironments(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listEnvironments("myResourceGroup", "myOrganization", 10, null, + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListRegionsSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListRegionsSamples.java new file mode 100644 index 0000000000000..3e9aa7e612379 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListRegionsSamples.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Organization ListRegions. + */ +public final class OrganizationListRegionsSamples { + /* + * x-ms-original-file: + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_ListRegions. + * json + */ + /** + * Sample code: Organization_ListRegions. + * + * @param manager Entry point to ConfluentManager. + */ + public static void organizationListRegions(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listRegionsWithResponse("myResourceGroup", "myOrganization", + new ListAccessRequestModel() + .withSearchFilters(mapOf("cloud", "azure", "packages", "ADVANCED,ESSENTIALS", "region", "eastus")), + com.azure.core.util.Context.NONE); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSamples.java index b019e4375c65d..a4c58397030be 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSamples.java @@ -9,7 +9,7 @@ */ public final class OrganizationListSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * Organization_ListBySubscription.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSchemaRegistryClustersSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSchemaRegistryClustersSamples.java new file mode 100644 index 0000000000000..b2824c81d1dbd --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationListSchemaRegistryClustersSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +/** + * Samples for Organization ListSchemaRegistryClusters. + */ +public final class OrganizationListSchemaRegistryClustersSamples { + /* + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ + * Organization_ListSchemaRegistryClusters.json + */ + /** + * Sample code: Organization_ListSchemaRegistryClusters. + * + * @param manager Entry point to ConfluentManager. + */ + public static void + organizationListSchemaRegistryClusters(com.azure.resourcemanager.confluent.ConfluentManager manager) { + manager.organizations().listSchemaRegistryClusters("myResourceGroup", "myOrganization", "env-stgcczjp2j3", null, + null, com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListSamples.java index 0d3fbd6b69037..f7e9b3da5c2ca 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListSamples.java @@ -9,7 +9,7 @@ */ public final class OrganizationOperationsListSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * OrganizationOperations_List.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationUpdateSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationUpdateSamples.java index 42039476357a5..704d85e155f3c 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationUpdateSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/OrganizationUpdateSamples.java @@ -14,7 +14,7 @@ public final class OrganizationUpdateSamples { /* * x-ms-original-file: - * specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/Organization_Update.json + * specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/Organization_Update.json */ /** * Sample code: Confluent_Update. diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationSamples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationSamples.java index 8e7dd1ccedea9..b612fab64e0a2 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationSamples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationSamples.java @@ -16,7 +16,7 @@ */ public final class ValidationsValidateOrganizationSamples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * Validations_ValidateOrganizations.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationV2Samples.java b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationV2Samples.java index 0583e45c58b29..d2ccb817e66e5 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationV2Samples.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/samples/java/com/azure/resourcemanager/confluent/generated/ValidationsValidateOrganizationV2Samples.java @@ -16,7 +16,7 @@ */ public final class ValidationsValidateOrganizationV2Samples { /* - * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2023-08-22/examples/ + * x-ms-original-file: specification/confluent/resource-manager/Microsoft.Confluent/stable/2024-02-13/examples/ * Validations_ValidateOrganizationsV2.json */ /** diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingRequestModelTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingRequestModelTests.java new file mode 100644 index 0000000000000..5a11611706fd9 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingRequestModelTests.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; +import org.junit.jupiter.api.Assertions; + +public final class AccessCreateRoleBindingRequestModelTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + AccessCreateRoleBindingRequestModel model = BinaryData + .fromString("{\"principal\":\"i\",\"role_name\":\"byuqerpqlp\",\"crn_pattern\":\"cciuqgbdbutau\"}") + .toObject(AccessCreateRoleBindingRequestModel.class); + Assertions.assertEquals("i", model.principal()); + Assertions.assertEquals("byuqerpqlp", model.roleName()); + Assertions.assertEquals("cciuqgbdbutau", model.crnPattern()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + AccessCreateRoleBindingRequestModel model = new AccessCreateRoleBindingRequestModel().withPrincipal("i") + .withRoleName("byuqerpqlp").withCrnPattern("cciuqgbdbutau"); + model = BinaryData.fromObject(model).toObject(AccessCreateRoleBindingRequestModel.class); + Assertions.assertEquals("i", model.principal()); + Assertions.assertEquals("byuqerpqlp", model.roleName()); + Assertions.assertEquals("cciuqgbdbutau", model.crnPattern()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingWithResponseMockTests.java new file mode 100644 index 0000000000000..a69f8cae14cc8 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessCreateRoleBindingWithResponseMockTests.java @@ -0,0 +1,69 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.AccessCreateRoleBindingRequestModel; +import com.azure.resourcemanager.confluent.models.RoleBindingRecord; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class AccessCreateRoleBindingWithResponseMockTests { + @Test + public void testCreateRoleBindingWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"kind\":\"njlx\",\"id\":\"rhwpus\",\"metadata\":{\"self\":\"aqehg\",\"resource_name\":\"ohzjqatucoigeb\",\"created_at\":\"cnwfepbnwgfmxjg\",\"updated_at\":\"bjb\",\"deleted_at\":\"lfgtdysnaquflqbc\"},\"principal\":\"hamzjrwdkqze\",\"role_name\":\"jleziunjx\",\"crn_pattern\":\"zantkwceg\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + RoleBindingRecord response = manager.access() + .createRoleBindingWithResponse("gepuslvyjtc", "uwkasiz", new AccessCreateRoleBindingRequestModel() + .withPrincipal("sfuughtuqfecjx").withRoleName("gtuhxuicbu").withCrnPattern("mr"), + com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals("njlx", response.kind()); + Assertions.assertEquals("rhwpus", response.id()); + Assertions.assertEquals("aqehg", response.metadata().self()); + Assertions.assertEquals("ohzjqatucoigeb", response.metadata().resourceName()); + Assertions.assertEquals("cnwfepbnwgfmxjg", response.metadata().createdAt()); + Assertions.assertEquals("bjb", response.metadata().updatedAt()); + Assertions.assertEquals("lfgtdysnaquflqbc", response.metadata().deletedAt()); + Assertions.assertEquals("hamzjrwdkqze", response.principal()); + Assertions.assertEquals("jleziunjx", response.roleName()); + Assertions.assertEquals("zantkwceg", response.crnPattern()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingWithResponseMockTests.java new file mode 100644 index 0000000000000..d4597dc29f344 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessDeleteRoleBindingWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class AccessDeleteRoleBindingWithResponseMockTests { + @Test + public void testDeleteRoleBindingWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.access().deleteRoleBindingWithResponse("amlbnseqacjjvpil", "uooqjagmdit", "ueio", + com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserWithResponseMockTests.java index dbe12a3f07fea..ab04be4a7c89f 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessInviteUserWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testInviteUserWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"ijnkrxfrdd\",\"id\":\"ratiz\",\"metadata\":{\"self\":\"nasx\",\"resource_name\":\"tozqyzhftwesgo\",\"created_at\":\"zhonnxkrlgnyhmo\",\"updated_at\":\"xkk\",\"deleted_at\":\"h\"},\"email\":\"gh\",\"auth_type\":\"bdhqxvcxgf\",\"status\":\"dsofbshrns\",\"accepted_at\":\"uswdv\",\"expires_at\":\"ybycnunvj\"}"; + = "{\"kind\":\"jlzqnhc\",\"id\":\"ql\",\"metadata\":{\"self\":\"oibgsxg\",\"resource_name\":\"fyq\",\"created_at\":\"mpqoxw\",\"updated_at\":\"fdbxiqxeiiqbim\",\"deleted_at\":\"mwwinhehfqpofv\"},\"email\":\"cblembnkbwv\",\"auth_type\":\"xk\",\"status\":\"vqihebwtswbzuwf\",\"accepted_at\":\"urageg\",\"expires_at\":\"vcjfelisdjubggb\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,26 +50,25 @@ public void testInviteUserWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - InvitationRecord response - = manager.access() - .inviteUserWithResponse( - "uu", "fdlwg", new AccessInviteUserAccountModel().withOrganizationId("tsbwtovvtgse") - .withEmail("qfi").withUpn("x").withInvitedUserDetails(new AccessInvitedUserDetails() - .withInvitedEmail("pirgnepttwq").withAuthType("niffcdmqnroj")), - com.azure.core.util.Context.NONE) - .getValue(); + InvitationRecord response = manager.access() + .inviteUserWithResponse("phai", "mxyasflvgsgzw", + new AccessInviteUserAccountModel().withOrganizationId("akoi").withEmail("nsmjbl") + .withUpn("jhlnymzotqyryu").withInvitedUserDetails( + new AccessInvitedUserDetails().withInvitedEmail("mqqvxmvwfgtay").withAuthType("nsup")), + com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("ijnkrxfrdd", response.kind()); - Assertions.assertEquals("ratiz", response.id()); - Assertions.assertEquals("nasx", response.metadata().self()); - Assertions.assertEquals("tozqyzhftwesgo", response.metadata().resourceName()); - Assertions.assertEquals("zhonnxkrlgnyhmo", response.metadata().createdAt()); - Assertions.assertEquals("xkk", response.metadata().updatedAt()); - Assertions.assertEquals("h", response.metadata().deletedAt()); - Assertions.assertEquals("gh", response.email()); - Assertions.assertEquals("bdhqxvcxgf", response.authType()); - Assertions.assertEquals("dsofbshrns", response.status()); - Assertions.assertEquals("uswdv", response.acceptedAt()); - Assertions.assertEquals("ybycnunvj", response.expiresAt()); + Assertions.assertEquals("jlzqnhc", response.kind()); + Assertions.assertEquals("ql", response.id()); + Assertions.assertEquals("oibgsxg", response.metadata().self()); + Assertions.assertEquals("fyq", response.metadata().resourceName()); + Assertions.assertEquals("mpqoxw", response.metadata().createdAt()); + Assertions.assertEquals("fdbxiqxeiiqbim", response.metadata().updatedAt()); + Assertions.assertEquals("mwwinhehfqpofv", response.metadata().deletedAt()); + Assertions.assertEquals("cblembnkbwv", response.email()); + Assertions.assertEquals("xk", response.authType()); + Assertions.assertEquals("vqihebwtswbzuwf", response.status()); + Assertions.assertEquals("urageg", response.acceptedAt()); + Assertions.assertEquals("vcjfelisdjubggb", response.expiresAt()); } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListClustersWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListClustersWithResponseMockTests.java index 97195ab409236..b30a09fb810d6 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListClustersWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListClustersWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testListClustersWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"sx\",\"metadata\":{\"first\":\"msvpkjpr\",\"last\":\"wcfzqljyxgt\",\"prev\":\"heyd\",\"next\":\"dshmkxmaehvbbx\",\"total_size\":870827030},\"data\":[{\"kind\":\"fnhtbaxkgxyw\",\"id\":\"kpyklyhp\",\"metadata\":{\"self\":\"dpvruud\",\"resource_name\":\"zibt\",\"created_at\":\"stgktst\",\"updated_at\":\"xeclzedqbcvhzlhp\",\"deleted_at\":\"dqkdlwwqfbu\"},\"display_name\":\"kxtrq\",\"spec\":{\"display_name\":\"mlmbtxhwgfwsrta\",\"availability\":\"oezbrhubsk\",\"cloud\":\"dyg\",\"zone\":\"okkqfqjbvleo\",\"region\":\"ml\",\"kafka_bootstrap_endpoint\":\"qtqzfavyv\",\"http_endpoint\":\"qybaryeua\",\"api_endpoint\":\"kq\",\"config\":{\"kind\":\"gzslesjcbhernnti\"},\"environment\":{\"id\":\"jcvb\",\"environment\":\"wr\",\"related\":\"hwagohbuffkmrqe\",\"resource_name\":\"vhmxtdrjfu\"},\"network\":{\"id\":\"oe\",\"environment\":\"vewzcj\",\"related\":\"m\",\"resource_name\":\"pmguaadraufac\"},\"byok\":{\"id\":\"hzovaj\",\"related\":\"iuxxpshneekulfg\",\"resource_name\":\"qubkw\"}},\"status\":{\"phase\":\"nrdsutujbazpjuoh\",\"cku\":1097085357}},{\"kind\":\"flnorwmduvwp\",\"id\":\"vxwmygd\",\"metadata\":{\"self\":\"pqchiszep\",\"resource_name\":\"bjcrxgibbdaxco\",\"created_at\":\"ozauorsukokwb\",\"updated_at\":\"lhlv\",\"deleted_at\":\"uepzl\"},\"display_name\":\"hw\",\"spec\":{\"display_name\":\"l\",\"availability\":\"eyuqdunvmnnrwr\",\"cloud\":\"ork\",\"zone\":\"lywjhh\",\"region\":\"nhxmsi\",\"kafka_bootstrap_endpoint\":\"omi\",\"http_endpoint\":\"xggdufi\",\"api_endpoint\":\"dieuzaofj\",\"config\":{\"kind\":\"cyyysfgdot\"},\"environment\":{\"id\":\"iipuipwoqonm\",\"environment\":\"jeknizshq\",\"related\":\"impevf\",\"resource_name\":\"b\"},\"network\":{\"id\":\"ilbywdxsm\",\"environment\":\"cwrwfs\",\"related\":\"fnynszqujizdvoqy\",\"resource_name\":\"byowbblgyavutp\"},\"byok\":{\"id\":\"oxoismsksbpim\",\"related\":\"oljxkcgx\",\"resource_name\":\"xsffgcviz\"}},\"status\":{\"phase\":\"wlvwlyoupf\",\"cku\":1825475274}},{\"kind\":\"jub\",\"id\":\"hgkfmin\",\"metadata\":{\"self\":\"wzf\",\"resource_name\":\"sttktlahbqa\",\"created_at\":\"xtgzukxitmmqtgqq\",\"updated_at\":\"hrnxrxc\",\"deleted_at\":\"uisavokq\"},\"display_name\":\"fvazivjlfrqttba\",\"spec\":{\"display_name\":\"atnwxyiopi\",\"availability\":\"qqfkuv\",\"cloud\":\"xkdmligo\",\"zone\":\"brxk\",\"region\":\"loazuruocbgoo\",\"kafka_bootstrap_endpoint\":\"te\",\"http_endpoint\":\"bfhjxakvvjgsl\",\"api_endpoint\":\"dilmyww\",\"config\":{\"kind\":\"kxn\"},\"environment\":{\"id\":\"abgyvudt\",\"environment\":\"ewbcihxuuw\",\"related\":\"j\",\"resource_name\":\"ccybvp\"},\"network\":{\"id\":\"kkudzp\",\"environment\":\"wjplma\",\"related\":\"tcyohpfkyrk\",\"resource_name\":\"dg\"},\"byok\":{\"id\":\"sjkmnwqj\",\"related\":\"baiyhddviaceg\",\"resource_name\":\"m\"}},\"status\":{\"phase\":\"pmvmemfnczdwvv\",\"cku\":128886656}}]}"; + = "{\"kind\":\"iqg\",\"metadata\":{\"first\":\"okzrus\",\"last\":\"vhczznvfby\",\"prev\":\"sxjwwixz\",\"next\":\"mwmxqhndvnoamld\",\"total_size\":1759834581},\"data\":[{\"kind\":\"djh\",\"id\":\"lzok\",\"metadata\":{\"self\":\"xpelnjetagltsx\",\"resource_name\":\"tft\",\"created_at\":\"pnpbswveflocc\",\"updated_at\":\"mozi\",\"deleted_at\":\"ipgawtxx\"},\"display_name\":\"y\",\"spec\":{\"display_name\":\"cjxgrytf\",\"availability\":\"cy\",\"cloud\":\"lrmcaykg\",\"zone\":\"oxuztrksx\",\"region\":\"ndfcpfn\",\"kafka_bootstrap_endpoint\":\"thjtwk\",\"http_endpoint\":\"osrxuzvoa\",\"api_endpoint\":\"tcqiosmg\",\"config\":{\"kind\":\"hgxqdlyrt\"},\"environment\":{\"id\":\"ap\",\"environment\":\"tz\",\"related\":\"tbhjmznnbsoqe\",\"resource_name\":\"larvlagunbtg\"},\"network\":{\"id\":\"wlnbm\",\"environment\":\"reeudzqavb\",\"related\":\"qmjxlyyzglgouwtl\",\"resource_name\":\"jyuojqtobaxkjeyt\"},\"byok\":{\"id\":\"bfjkw\",\"related\":\"snkq\",\"resource_name\":\"syrq\"}},\"status\":{\"phase\":\"qhd\",\"cku\":2136418443}},{\"kind\":\"ulkpakd\",\"id\":\"fmjnnawtqa\",\"metadata\":{\"self\":\"uckpggqoweyir\",\"resource_name\":\"lisn\",\"created_at\":\"fl\",\"updated_at\":\"mpizru\",\"deleted_at\":\"pqxpx\"},\"display_name\":\"fcngjsa\",\"spec\":{\"display_name\":\"ixtmkzjvkviirhgf\",\"availability\":\"wsdpgratzvzb\",\"cloud\":\"byvi\",\"zone\":\"ctbrxkjzwrgxffm\",\"region\":\"kwfbkgo\",\"kafka_bootstrap_endpoint\":\"wopdbydpiz\",\"http_endpoint\":\"clnapxbiygnugjkn\",\"api_endpoint\":\"mfcttux\",\"config\":{\"kind\":\"i\"},\"environment\":{\"id\":\"qoiquvrehmrnjhv\",\"environment\":\"jztczytqj\",\"related\":\"h\",\"resource_name\":\"unfprnjletlxs\"},\"network\":{\"id\":\"ddoui\",\"environment\":\"mowaziynknlqwzdv\",\"related\":\"w\",\"resource_name\":\"qszdtmaajquhuxyl\"},\"byok\":{\"id\":\"m\",\"related\":\"g\",\"resource_name\":\"mzyospspshck\"}},\"status\":{\"phase\":\"jpmspbpssdfppy\",\"cku\":1546979917}},{\"kind\":\"eyujtvczkcnyxrx\",\"id\":\"njdxvglnkvxl\",\"metadata\":{\"self\":\"glqivbgkcv\",\"resource_name\":\"pzvuqdflvo\",\"created_at\":\"yp\",\"updated_at\":\"ubcpzgpxti\",\"deleted_at\":\"j\"},\"display_name\":\"idibgqjxgpn\",\"spec\":{\"display_name\":\"ov\",\"availability\":\"pikqmh\",\"cloud\":\"owjrmzvuporqz\",\"zone\":\"uydzvk\",\"region\":\"xcnqmxqpswokmvkh\",\"kafka_bootstrap_endpoint\":\"gdhbe\",\"http_endpoint\":\"qkzszuwiwtglxxh\",\"api_endpoint\":\"fpgpicrmnzhrgm\",\"config\":{\"kind\":\"sxvpqcbfrmbodths\"},\"environment\":{\"id\":\"vriibakclacjfr\",\"environment\":\"ousxauzlwvsgmw\",\"related\":\"qf\",\"resource_name\":\"zvuxm\"},\"network\":{\"id\":\"svth\",\"environment\":\"pz\",\"related\":\"kovmribiatt\",\"resource_name\":\"lu\"},\"byok\":{\"id\":\"tangcfhnykzcu\",\"related\":\"wvxwlmzqwmvt\",\"resource_name\":\"jmxmcuqud\"}},\"status\":{\"phase\":\"clxyn\",\"cku\":267466114}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,46 +51,48 @@ public void testListClustersWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - AccessListClusterSuccessResponse response = manager.access().listClustersWithResponse("jdftuljltd", "ceamtm", - new ListAccessRequestModel().withSearchFilters(mapOf("ejwcwwqiok", "o")), com.azure.core.util.Context.NONE) + AccessListClusterSuccessResponse response = manager.access() + .listClustersWithResponse("izcil", "ghgshejjtbxqmu", + new ListAccessRequestModel().withSearchFilters(mapOf("rsbycucrwn", "lxqzvn", "qbsms", "mikzeb")), + com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("sx", response.kind()); - Assertions.assertEquals("msvpkjpr", response.metadata().first()); - Assertions.assertEquals("wcfzqljyxgt", response.metadata().last()); - Assertions.assertEquals("heyd", response.metadata().prev()); - Assertions.assertEquals("dshmkxmaehvbbx", response.metadata().next()); - Assertions.assertEquals(870827030, response.metadata().totalSize()); - Assertions.assertEquals("fnhtbaxkgxyw", response.data().get(0).kind()); - Assertions.assertEquals("kpyklyhp", response.data().get(0).id()); - Assertions.assertEquals("dpvruud", response.data().get(0).metadata().self()); - Assertions.assertEquals("zibt", response.data().get(0).metadata().resourceName()); - Assertions.assertEquals("stgktst", response.data().get(0).metadata().createdAt()); - Assertions.assertEquals("xeclzedqbcvhzlhp", response.data().get(0).metadata().updatedAt()); - Assertions.assertEquals("dqkdlwwqfbu", response.data().get(0).metadata().deletedAt()); - Assertions.assertEquals("kxtrq", response.data().get(0).displayName()); - Assertions.assertEquals("mlmbtxhwgfwsrta", response.data().get(0).spec().displayName()); - Assertions.assertEquals("oezbrhubsk", response.data().get(0).spec().availability()); - Assertions.assertEquals("dyg", response.data().get(0).spec().cloud()); - Assertions.assertEquals("okkqfqjbvleo", response.data().get(0).spec().zone()); - Assertions.assertEquals("ml", response.data().get(0).spec().region()); - Assertions.assertEquals("qtqzfavyv", response.data().get(0).spec().kafkaBootstrapEndpoint()); - Assertions.assertEquals("qybaryeua", response.data().get(0).spec().httpEndpoint()); - Assertions.assertEquals("kq", response.data().get(0).spec().apiEndpoint()); - Assertions.assertEquals("gzslesjcbhernnti", response.data().get(0).spec().config().kind()); - Assertions.assertEquals("jcvb", response.data().get(0).spec().environment().id()); - Assertions.assertEquals("wr", response.data().get(0).spec().environment().environment()); - Assertions.assertEquals("hwagohbuffkmrqe", response.data().get(0).spec().environment().related()); - Assertions.assertEquals("vhmxtdrjfu", response.data().get(0).spec().environment().resourceName()); - Assertions.assertEquals("oe", response.data().get(0).spec().network().id()); - Assertions.assertEquals("vewzcj", response.data().get(0).spec().network().environment()); - Assertions.assertEquals("m", response.data().get(0).spec().network().related()); - Assertions.assertEquals("pmguaadraufac", response.data().get(0).spec().network().resourceName()); - Assertions.assertEquals("hzovaj", response.data().get(0).spec().byok().id()); - Assertions.assertEquals("iuxxpshneekulfg", response.data().get(0).spec().byok().related()); - Assertions.assertEquals("qubkw", response.data().get(0).spec().byok().resourceName()); - Assertions.assertEquals("nrdsutujbazpjuoh", response.data().get(0).status().phase()); - Assertions.assertEquals(1097085357, response.data().get(0).status().cku()); + Assertions.assertEquals("iqg", response.kind()); + Assertions.assertEquals("okzrus", response.metadata().first()); + Assertions.assertEquals("vhczznvfby", response.metadata().last()); + Assertions.assertEquals("sxjwwixz", response.metadata().prev()); + Assertions.assertEquals("mwmxqhndvnoamld", response.metadata().next()); + Assertions.assertEquals(1759834581, response.metadata().totalSize()); + Assertions.assertEquals("djh", response.data().get(0).kind()); + Assertions.assertEquals("lzok", response.data().get(0).id()); + Assertions.assertEquals("xpelnjetagltsx", response.data().get(0).metadata().self()); + Assertions.assertEquals("tft", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("pnpbswveflocc", response.data().get(0).metadata().createdAt()); + Assertions.assertEquals("mozi", response.data().get(0).metadata().updatedAt()); + Assertions.assertEquals("ipgawtxx", response.data().get(0).metadata().deletedAt()); + Assertions.assertEquals("y", response.data().get(0).displayName()); + Assertions.assertEquals("cjxgrytf", response.data().get(0).spec().displayName()); + Assertions.assertEquals("cy", response.data().get(0).spec().availability()); + Assertions.assertEquals("lrmcaykg", response.data().get(0).spec().cloud()); + Assertions.assertEquals("oxuztrksx", response.data().get(0).spec().zone()); + Assertions.assertEquals("ndfcpfn", response.data().get(0).spec().region()); + Assertions.assertEquals("thjtwk", response.data().get(0).spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("osrxuzvoa", response.data().get(0).spec().httpEndpoint()); + Assertions.assertEquals("tcqiosmg", response.data().get(0).spec().apiEndpoint()); + Assertions.assertEquals("hgxqdlyrt", response.data().get(0).spec().config().kind()); + Assertions.assertEquals("ap", response.data().get(0).spec().environment().id()); + Assertions.assertEquals("tz", response.data().get(0).spec().environment().environment()); + Assertions.assertEquals("tbhjmznnbsoqe", response.data().get(0).spec().environment().related()); + Assertions.assertEquals("larvlagunbtg", response.data().get(0).spec().environment().resourceName()); + Assertions.assertEquals("wlnbm", response.data().get(0).spec().network().id()); + Assertions.assertEquals("reeudzqavb", response.data().get(0).spec().network().environment()); + Assertions.assertEquals("qmjxlyyzglgouwtl", response.data().get(0).spec().network().related()); + Assertions.assertEquals("jyuojqtobaxkjeyt", response.data().get(0).spec().network().resourceName()); + Assertions.assertEquals("bfjkw", response.data().get(0).spec().byok().id()); + Assertions.assertEquals("snkq", response.data().get(0).spec().byok().related()); + Assertions.assertEquals("syrq", response.data().get(0).spec().byok().resourceName()); + Assertions.assertEquals("qhd", response.data().get(0).status().phase()); + Assertions.assertEquals(2136418443, response.data().get(0).status().cku()); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsWithResponseMockTests.java index 51c9fff5f21b9..94ced199cb026 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListEnvironmentsWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testListEnvironmentsWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"havgrvk\",\"metadata\":{\"first\":\"vjzhpjbib\",\"last\":\"mfxumvfcluyovw\",\"prev\":\"bkfezzxscyhwzdgi\",\"next\":\"jbzbomvzzbtdcq\",\"total_size\":1359383511},\"data\":[{\"kind\":\"jviylwdshfs\",\"id\":\"rbgyefry\",\"metadata\":{\"self\":\"aojfm\",\"resource_name\":\"cotmr\",\"created_at\":\"irctymoxoftpipiw\",\"updated_at\":\"zuhx\",\"deleted_at\":\"pqjlihhyusps\"},\"display_name\":\"sdvlmfwdgzxulucv\"},{\"kind\":\"mrsreuzvxurisjnh\",\"id\":\"txifqj\",\"metadata\":{\"self\":\"mrhublwpc\",\"resource_name\":\"utr\",\"created_at\":\"upauut\",\"updated_at\":\"oqh\",\"deleted_at\":\"ejqgw\"},\"display_name\":\"nfqn\"},{\"kind\":\"ypsxjvfoim\",\"id\":\"slirciz\",\"metadata\":{\"self\":\"ydfce\",\"resource_name\":\"vlhv\",\"created_at\":\"dyftumrtwna\",\"updated_at\":\"slbi\",\"deleted_at\":\"ojgcyzt\"},\"display_name\":\"mznbaeqphch\"},{\"kind\":\"rn\",\"id\":\"x\",\"metadata\":{\"self\":\"wrykqgai\",\"resource_name\":\"viklb\",\"created_at\":\"vkhbejdznx\",\"updated_at\":\"dsrhnjiv\",\"deleted_at\":\"v\"},\"display_name\":\"ovqfzge\"}]}"; + = "{\"kind\":\"tkvnlvxbcuiiznkt\",\"metadata\":{\"first\":\"nsnvpd\",\"last\":\"mik\",\"prev\":\"tbzbkiwbuqnyophz\",\"next\":\"l\",\"total_size\":348954725},\"data\":[{\"kind\":\"bcunezzceze\",\"id\":\"w\",\"metadata\":{\"self\":\"lwxjwetn\",\"resource_name\":\"ihclafzv\",\"created_at\":\"lpt\",\"updated_at\":\"qqwzt\",\"deleted_at\":\"w\"},\"display_name\":\"chcxwaxfewzj\"},{\"kind\":\"exfdeqvhpsylk\",\"id\":\"hkbffmbm\",\"metadata\":{\"self\":\"rgywwp\",\"resource_name\":\"xs\",\"created_at\":\"tf\",\"updated_at\":\"gicgaaoepttaq\",\"deleted_at\":\"dewemxswv\"},\"display_name\":\"unzzjgehk\"},{\"kind\":\"imrt\",\"id\":\"okffqyinljqepqwh\",\"metadata\":{\"self\":\"onsts\",\"resource_name\":\"yxgvelfcld\",\"created_at\":\"cb\",\"updated_at\":\"ds\",\"deleted_at\":\"wcobie\"},\"display_name\":\"tmninw\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,26 +52,25 @@ public void testListEnvironmentsWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); AccessListEnvironmentsSuccessResponse response = manager.access() - .listEnvironmentsWithResponse("rtkfawnopq", "ikyzirtxdy", - new ListAccessRequestModel().withSearchFilters( - mapOf("sewgioilqukr", "ejnt", "ggufhyaomtb", "dxtqmieoxo")), + .listEnvironmentsWithResponse("igkxkbsazga", "gacyrcmjdmspo", + new ListAccessRequestModel().withSearchFilters(mapOf("frzgbzjed", "vuhrylni")), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("havgrvk", response.kind()); - Assertions.assertEquals("vjzhpjbib", response.metadata().first()); - Assertions.assertEquals("mfxumvfcluyovw", response.metadata().last()); - Assertions.assertEquals("bkfezzxscyhwzdgi", response.metadata().prev()); - Assertions.assertEquals("jbzbomvzzbtdcq", response.metadata().next()); - Assertions.assertEquals(1359383511, response.metadata().totalSize()); - Assertions.assertEquals("jviylwdshfs", response.data().get(0).kind()); - Assertions.assertEquals("rbgyefry", response.data().get(0).id()); - Assertions.assertEquals("aojfm", response.data().get(0).metadata().self()); - Assertions.assertEquals("cotmr", response.data().get(0).metadata().resourceName()); - Assertions.assertEquals("irctymoxoftpipiw", response.data().get(0).metadata().createdAt()); - Assertions.assertEquals("zuhx", response.data().get(0).metadata().updatedAt()); - Assertions.assertEquals("pqjlihhyusps", response.data().get(0).metadata().deletedAt()); - Assertions.assertEquals("sdvlmfwdgzxulucv", response.data().get(0).displayName()); + Assertions.assertEquals("tkvnlvxbcuiiznkt", response.kind()); + Assertions.assertEquals("nsnvpd", response.metadata().first()); + Assertions.assertEquals("mik", response.metadata().last()); + Assertions.assertEquals("tbzbkiwbuqnyophz", response.metadata().prev()); + Assertions.assertEquals("l", response.metadata().next()); + Assertions.assertEquals(348954725, response.metadata().totalSize()); + Assertions.assertEquals("bcunezzceze", response.data().get(0).kind()); + Assertions.assertEquals("w", response.data().get(0).id()); + Assertions.assertEquals("lwxjwetn", response.data().get(0).metadata().self()); + Assertions.assertEquals("ihclafzv", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("lpt", response.data().get(0).metadata().createdAt()); + Assertions.assertEquals("qqwzt", response.data().get(0).metadata().updatedAt()); + Assertions.assertEquals("w", response.data().get(0).metadata().deletedAt()); + Assertions.assertEquals("chcxwaxfewzj", response.data().get(0).displayName()); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsWithResponseMockTests.java index 1e23fe56f1c36..a5a77ab414877 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListInvitationsWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testListInvitationsWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"mkqjj\",\"metadata\":{\"first\":\"envrkpyouaibrebq\",\"last\":\"ysjkixqtnqttez\",\"prev\":\"fffiak\",\"next\":\"pqqmted\",\"total_size\":1028485886},\"data\":[{\"kind\":\"hyeozphvwau\",\"id\":\"ncyg\",\"metadata\":{\"self\":\"vipmdscwxqupevzh\",\"resource_name\":\"totxhojujb\",\"created_at\":\"elmcuvhixbjxyfw\",\"updated_at\":\"lrcoolsttpki\",\"deleted_at\":\"kbnujr\"},\"email\":\"vtylbfpncu\",\"auth_type\":\"oiwiithtywub\",\"status\":\"bihwqknfdnt\",\"accepted_at\":\"chrdgoihxumwcto\",\"expires_at\":\"zj\"}]}"; + = "{\"kind\":\"jwogqqnobpudc\",\"metadata\":{\"first\":\"tqwpwya\",\"last\":\"zasqbucljgkyexao\",\"prev\":\"yaipidsda\",\"next\":\"tx\",\"total_size\":634071043},\"data\":[{\"kind\":\"qwazlnqnmcjngzq\",\"id\":\"xtbjwgnyfusfzsv\",\"metadata\":{\"self\":\"kzhajqglcfhm\",\"resource_name\":\"qryxyn\",\"created_at\":\"zrdpsovwxznptgoe\",\"updated_at\":\"bbabp\",\"deleted_at\":\"vf\"},\"email\":\"kvntjlrigjkskyri\",\"auth_type\":\"vzidsxwaab\",\"status\":\"ifrygzn\",\"accepted_at\":\"axri\",\"expires_at\":\"zob\"},{\"kind\":\"pxl\",\"id\":\"lnelxieixynl\",\"metadata\":{\"self\":\"cwcrojphs\",\"resource_name\":\"cawjutifdwfmvi\",\"created_at\":\"rqjb\",\"updated_at\":\"zhraglkafh\",\"deleted_at\":\"qjujeickpzvcp\"},\"email\":\"mxelnwcltyjed\",\"auth_type\":\"xm\",\"status\":\"mkqscaz\",\"accepted_at\":\"wxtzxpuamwab\",\"expires_at\":\"rvxcush\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,30 +52,28 @@ public void testListInvitationsWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); AccessListInvitationsSuccessResponse response = manager.access() - .listInvitationsWithResponse("hjglikk", "wslolbqp", - new ListAccessRequestModel().withSearchFilters( - mapOf("elfk", "lmv", "njwmwkpnbsazejj", "gplcrpwjxeznoigb", "augzxnfaazpxdtn", "qkagfhsxt")), - com.azure.core.util.Context.NONE) + .listInvitationsWithResponse("paq", "ferr", + new ListAccessRequestModel().withSearchFilters(mapOf("kmfx", "x")), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("mkqjj", response.kind()); - Assertions.assertEquals("envrkpyouaibrebq", response.metadata().first()); - Assertions.assertEquals("ysjkixqtnqttez", response.metadata().last()); - Assertions.assertEquals("fffiak", response.metadata().prev()); - Assertions.assertEquals("pqqmted", response.metadata().next()); - Assertions.assertEquals(1028485886, response.metadata().totalSize()); - Assertions.assertEquals("hyeozphvwau", response.data().get(0).kind()); - Assertions.assertEquals("ncyg", response.data().get(0).id()); - Assertions.assertEquals("vipmdscwxqupevzh", response.data().get(0).metadata().self()); - Assertions.assertEquals("totxhojujb", response.data().get(0).metadata().resourceName()); - Assertions.assertEquals("elmcuvhixbjxyfw", response.data().get(0).metadata().createdAt()); - Assertions.assertEquals("lrcoolsttpki", response.data().get(0).metadata().updatedAt()); - Assertions.assertEquals("kbnujr", response.data().get(0).metadata().deletedAt()); - Assertions.assertEquals("vtylbfpncu", response.data().get(0).email()); - Assertions.assertEquals("oiwiithtywub", response.data().get(0).authType()); - Assertions.assertEquals("bihwqknfdnt", response.data().get(0).status()); - Assertions.assertEquals("chrdgoihxumwcto", response.data().get(0).acceptedAt()); - Assertions.assertEquals("zj", response.data().get(0).expiresAt()); + Assertions.assertEquals("jwogqqnobpudc", response.kind()); + Assertions.assertEquals("tqwpwya", response.metadata().first()); + Assertions.assertEquals("zasqbucljgkyexao", response.metadata().last()); + Assertions.assertEquals("yaipidsda", response.metadata().prev()); + Assertions.assertEquals("tx", response.metadata().next()); + Assertions.assertEquals(634071043, response.metadata().totalSize()); + Assertions.assertEquals("qwazlnqnmcjngzq", response.data().get(0).kind()); + Assertions.assertEquals("xtbjwgnyfusfzsv", response.data().get(0).id()); + Assertions.assertEquals("kzhajqglcfhm", response.data().get(0).metadata().self()); + Assertions.assertEquals("qryxyn", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("zrdpsovwxznptgoe", response.data().get(0).metadata().createdAt()); + Assertions.assertEquals("bbabp", response.data().get(0).metadata().updatedAt()); + Assertions.assertEquals("vf", response.data().get(0).metadata().deletedAt()); + Assertions.assertEquals("kvntjlrigjkskyri", response.data().get(0).email()); + Assertions.assertEquals("vzidsxwaab", response.data().get(0).authType()); + Assertions.assertEquals("ifrygzn", response.data().get(0).status()); + Assertions.assertEquals("axri", response.data().get(0).acceptedAt()); + Assertions.assertEquals("zob", response.data().get(0).expiresAt()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListWithResponseMockTests.java similarity index 58% rename from sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateMockTests.java rename to sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListWithResponseMockTests.java index acb1a97a247d6..2476a8d04383e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingNameListWithResponseMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.generated; +package com.azure.resourcemanager.confluent.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -11,8 +11,9 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.netapp.NetAppFilesManager; -import com.azure.resourcemanager.netapp.models.BackupVault; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.AccessRoleBindingNameListSuccessResponse; +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -25,15 +26,15 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class BackupVaultsCreateOrUpdateMockTests { +public final class AccessListRoleBindingNameListWithResponseMockTests { @Test - public void testCreateOrUpdate() throws Exception { + public void testListRoleBindingNameListWithResponse() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Succeeded\"},\"location\":\"drwjjkhvyom\",\"tags\":{\"vxnqmhrpqpd\":\"u\",\"ssffxuifmc\":\"wmkoisq\"},\"id\":\"ypobkdqzr\",\"name\":\"zsylollgt\",\"type\":\"czzydmxzjij\"}"; + = "{\"kind\":\"tuwkffdj\",\"metadata\":{\"first\":\"ysidfvclgl\",\"last\":\"fuijtkbus\",\"prev\":\"gsfikayiansha\",\"next\":\"jtjiqx\",\"total_size\":1119516110},\"data\":[\"ttvwkpqh\"]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -46,17 +47,24 @@ public void testCreateOrUpdate() throws Exception { return Mono.just(httpResponse); })); - NetAppFilesManager manager = NetAppFilesManager.configure().withHttpClient(httpClient).authenticate( + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - BackupVault response = manager.backupVaults().define("ljvrcmyfqipgxhnp").withRegion("ey") - .withExistingNetAppAccount("iwzcxmjpbyeph", "gt").withTags(mapOf("jlrxwtoaukhfk", "lpaugmrm", - "oaedsxjwuivedwcg", "cisiz", "mguaml", "eewxeiqbpsmg", "lzgaufcshhvnew", "dlrgms")) - .create(); + AccessRoleBindingNameListSuccessResponse response = manager.access() + .listRoleBindingNameListWithResponse("kjbsah", "tdtpdelqacslmo", + new ListAccessRequestModel() + .withSearchFilters(mapOf("k", "bnfxofvc", "xejw", "dirazf", "jurbuhhlkyqltqsr", "bmdujtmvcopexc")), + com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("drwjjkhvyom", response.location()); - Assertions.assertEquals("u", response.tags().get("vxnqmhrpqpd")); + Assertions.assertEquals("tuwkffdj", response.kind()); + Assertions.assertEquals("ysidfvclgl", response.metadata().first()); + Assertions.assertEquals("fuijtkbus", response.metadata().last()); + Assertions.assertEquals("gsfikayiansha", response.metadata().prev()); + Assertions.assertEquals("jtjiqx", response.metadata().next()); + Assertions.assertEquals(1119516110, response.metadata().totalSize()); + Assertions.assertEquals("ttvwkpqh", response.data().get(0)); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsWithResponseMockTests.java index 201002eeedf96..9a8e3d78dbd5a 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListRoleBindingsWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testListRoleBindingsWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"qcaaewdaomdjvl\",\"metadata\":{\"first\":\"xkzb\",\"last\":\"sgeivsiy\",\"prev\":\"kdncj\",\"next\":\"onbzoggculapzwy\",\"total_size\":1152319383},\"data\":[{\"kind\":\"qxepnylbfuaj\",\"id\":\"jtlvofqzhvfciby\",\"metadata\":{\"self\":\"wuxrkjpvdwxfzwi\",\"resource_name\":\"wzjbh\",\"created_at\":\"sxjrk\",\"updated_at\":\"btrnegvm\",\"deleted_at\":\"uqeqv\"},\"principal\":\"spastjbkkdmf\",\"role_name\":\"est\",\"crn_pattern\":\"lx\"},{\"kind\":\"ilozapeewchpxlk\",\"id\":\"kuziycsle\",\"metadata\":{\"self\":\"uztcktyhjtqed\",\"resource_name\":\"zulwmmrqzzrrj\",\"created_at\":\"gl\",\"updated_at\":\"zgkrvqe\",\"deleted_at\":\"toepryu\"},\"principal\":\"wytpzdmovz\",\"role_name\":\"va\",\"crn_pattern\":\"zqadf\"}]}"; + = "{\"kind\":\"qylkmqpzoyhlf\",\"metadata\":{\"first\":\"wgcloxoebqinji\",\"last\":\"wjfuj\",\"prev\":\"afcba\",\"next\":\"pzpofoiyjw\",\"total_size\":1217605698},\"data\":[{\"kind\":\"kkholvdndvia\",\"id\":\"gphuartvtiu\",\"metadata\":{\"self\":\"fchnmnah\",\"resource_name\":\"xhk\",\"created_at\":\"qirwrweoox\",\"updated_at\":\"i\",\"deleted_at\":\"xwrsnew\"},\"principal\":\"zqvbubqm\",\"role_name\":\"hsycxhxzgaz\",\"crn_pattern\":\"abo\"},{\"kind\":\"vmfqhppubo\",\"id\":\"epdfgkmtdherng\",\"metadata\":{\"self\":\"juahokqto\",\"resource_name\":\"auxofshfph\",\"created_at\":\"nulaiywzejywhsl\",\"updated_at\":\"ojpllndnpdwrpqaf\",\"deleted_at\":\"ug\"},\"principal\":\"n\",\"role_name\":\"yetefyp\",\"crn_pattern\":\"octfjgtixrjvzuyt\"},{\"kind\":\"mlmuowol\",\"id\":\"uir\",\"metadata\":{\"self\":\"ons\",\"resource_name\":\"nw\",\"created_at\":\"gajinnixjawrtmj\",\"updated_at\":\"myccx\",\"deleted_at\":\"hcoxov\"},\"principal\":\"khenlus\",\"role_name\":\"rd\",\"crn_pattern\":\"xtxrdcqtjvidt\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,30 +51,29 @@ public void testListRoleBindingsWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - AccessListRoleBindingsSuccessResponse response - = manager.access() - .listRoleBindingsWithResponse("xlllchp", "db", - new ListAccessRequestModel().withSearchFilters(mapOf("hfuk", "wrd", "fcvlerch", "vsjcswsmystuluqy", - "xsspuunnoxyhk", "qbmfpjbabwidf", "ddrihpf", "g")), - com.azure.core.util.Context.NONE) - .getValue(); + AccessListRoleBindingsSuccessResponse response = manager.access() + .listRoleBindingsWithResponse("vgfab", "iyji", + new ListAccessRequestModel() + .withSearchFilters(mapOf("neiknpg", "phdu", "btozipqwje", "xgjiuqh", "xxgewpk", "mur")), + com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("qcaaewdaomdjvl", response.kind()); - Assertions.assertEquals("xkzb", response.metadata().first()); - Assertions.assertEquals("sgeivsiy", response.metadata().last()); - Assertions.assertEquals("kdncj", response.metadata().prev()); - Assertions.assertEquals("onbzoggculapzwy", response.metadata().next()); - Assertions.assertEquals(1152319383, response.metadata().totalSize()); - Assertions.assertEquals("qxepnylbfuaj", response.data().get(0).kind()); - Assertions.assertEquals("jtlvofqzhvfciby", response.data().get(0).id()); - Assertions.assertEquals("wuxrkjpvdwxfzwi", response.data().get(0).metadata().self()); - Assertions.assertEquals("wzjbh", response.data().get(0).metadata().resourceName()); - Assertions.assertEquals("sxjrk", response.data().get(0).metadata().createdAt()); - Assertions.assertEquals("btrnegvm", response.data().get(0).metadata().updatedAt()); - Assertions.assertEquals("uqeqv", response.data().get(0).metadata().deletedAt()); - Assertions.assertEquals("spastjbkkdmf", response.data().get(0).principal()); - Assertions.assertEquals("est", response.data().get(0).roleName()); - Assertions.assertEquals("lx", response.data().get(0).crnPattern()); + Assertions.assertEquals("qylkmqpzoyhlf", response.kind()); + Assertions.assertEquals("wgcloxoebqinji", response.metadata().first()); + Assertions.assertEquals("wjfuj", response.metadata().last()); + Assertions.assertEquals("afcba", response.metadata().prev()); + Assertions.assertEquals("pzpofoiyjw", response.metadata().next()); + Assertions.assertEquals(1217605698, response.metadata().totalSize()); + Assertions.assertEquals("kkholvdndvia", response.data().get(0).kind()); + Assertions.assertEquals("gphuartvtiu", response.data().get(0).id()); + Assertions.assertEquals("fchnmnah", response.data().get(0).metadata().self()); + Assertions.assertEquals("xhk", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("qirwrweoox", response.data().get(0).metadata().createdAt()); + Assertions.assertEquals("i", response.data().get(0).metadata().updatedAt()); + Assertions.assertEquals("xwrsnew", response.data().get(0).metadata().deletedAt()); + Assertions.assertEquals("zqvbubqm", response.data().get(0).principal()); + Assertions.assertEquals("hsycxhxzgaz", response.data().get(0).roleName()); + Assertions.assertEquals("abo", response.data().get(0).crnPattern()); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsWithResponseMockTests.java index 4a8da14556a7b..0975fec556d7c 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListServiceAccountsWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testListServiceAccountsWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"phqamvdkfwynwcvt\",\"metadata\":{\"first\":\"ayhmtnvyqiatkz\",\"last\":\"cnpwzcjaesgvvscc\",\"prev\":\"jguq\",\"next\":\"wygzlvdnkfxusem\",\"total_size\":1095077901},\"data\":[{\"kind\":\"hapfcqdpsqx\",\"id\":\"psvuoymgc\",\"metadata\":{\"self\":\"vezrypqlmfeo\",\"resource_name\":\"rqwky\",\"created_at\":\"ob\",\"updated_at\":\"gxedkow\",\"deleted_at\":\"bqpc\"},\"display_name\":\"kbwcc\",\"description\":\"jvcdwxlpqekf\"},{\"kind\":\"khtj\",\"id\":\"i\",\"metadata\":{\"self\":\"fq\",\"resource_name\":\"mtdh\",\"created_at\":\"dvypgikdgsz\",\"updated_at\":\"kbir\",\"deleted_at\":\"uzhlhkjoqrv\"},\"display_name\":\"aatjinrvgoupmfi\",\"description\":\"fggjioolvr\"},{\"kind\":\"kvtkkg\",\"id\":\"qwjygvja\",\"metadata\":{\"self\":\"lmhvkzuhbxv\",\"resource_name\":\"hgsopbyrqufegx\",\"created_at\":\"wz\",\"updated_at\":\"nhlmctlpdng\",\"deleted_at\":\"vgbmhr\"},\"display_name\":\"kw\",\"description\":\"ijejvegrhbpn\"},{\"kind\":\"xexccbdreaxhcexd\",\"id\":\"vqahqkghtpwi\",\"metadata\":{\"self\":\"yjsvfyc\",\"resource_name\":\"bfvoowvrv\",\"created_at\":\"gjqppy\",\"updated_at\":\"tronzmyhgfi\",\"deleted_at\":\"sxkm\"},\"display_name\":\"a\",\"description\":\"rrjreafxtsgu\"}]}"; + = "{\"kind\":\"rlpyznuciqdsmexi\",\"metadata\":{\"first\":\"fuxtyasiibmiybnn\",\"last\":\"tgnljhnmgixhcmav\",\"prev\":\"foudor\",\"next\":\"gyyprotwy\",\"total_size\":677853893},\"data\":[{\"kind\":\"xhugcm\",\"id\":\"avlg\",\"metadata\":{\"self\":\"mftpmdtz\",\"resource_name\":\"ltfvnz\",\"created_at\":\"jtotpvopvpbd\",\"updated_at\":\"qgqqihedsvqwthmk\",\"deleted_at\":\"bcysih\"},\"display_name\":\"qcwdhoh\",\"description\":\"tmcdzsufcohd\"},{\"kind\":\"zlmcmuapcvhdb\",\"id\":\"wqqxeysko\",\"metadata\":{\"self\":\"inkfkbgbz\",\"resource_name\":\"wxeqocljmygvkzqk\",\"created_at\":\"eokbze\",\"updated_at\":\"zrxcczurt\",\"deleted_at\":\"ipqxbkwvzgnzv\"},\"display_name\":\"bzdixzmq\",\"description\":\"odawopqhewjptmcg\"},{\"kind\":\"ostzelndlatu\",\"id\":\"zlbiojlvfhrbbpn\",\"metadata\":{\"self\":\"cwwyyur\",\"resource_name\":\"chpp\",\"created_at\":\"rsnm\",\"updated_at\":\"ayzejnhlbkpbz\",\"deleted_at\":\"piljhahzvech\"},\"display_name\":\"bnwieholew\",\"description\":\"iuubwefqsf\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,27 +52,26 @@ public void testListServiceAccountsWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); AccessListServiceAccountsSuccessResponse response = manager.access() - .listServiceAccountsWithResponse( - "dvriiiojnal", "hfkvtvsexsowuel", new ListAccessRequestModel().withSearchFilters(mapOf("wws", - "hahhxvrhmzkwpj", "qxujxukndxd", "ughftqsx", "syqtfi", "grjguufzd", "phoszqz", "whbotzingamv")), + .listServiceAccountsWithResponse("nw", "acevehjkuyx", + new ListAccessRequestModel().withSearchFilters(mapOf("faey", "gaoql", "hriypoqeyhlqhy", "inmfgvxirp")), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("phqamvdkfwynwcvt", response.kind()); - Assertions.assertEquals("ayhmtnvyqiatkz", response.metadata().first()); - Assertions.assertEquals("cnpwzcjaesgvvscc", response.metadata().last()); - Assertions.assertEquals("jguq", response.metadata().prev()); - Assertions.assertEquals("wygzlvdnkfxusem", response.metadata().next()); - Assertions.assertEquals(1095077901, response.metadata().totalSize()); - Assertions.assertEquals("hapfcqdpsqx", response.data().get(0).kind()); - Assertions.assertEquals("psvuoymgc", response.data().get(0).id()); - Assertions.assertEquals("vezrypqlmfeo", response.data().get(0).metadata().self()); - Assertions.assertEquals("rqwky", response.data().get(0).metadata().resourceName()); - Assertions.assertEquals("ob", response.data().get(0).metadata().createdAt()); - Assertions.assertEquals("gxedkow", response.data().get(0).metadata().updatedAt()); - Assertions.assertEquals("bqpc", response.data().get(0).metadata().deletedAt()); - Assertions.assertEquals("kbwcc", response.data().get(0).displayName()); - Assertions.assertEquals("jvcdwxlpqekf", response.data().get(0).description()); + Assertions.assertEquals("rlpyznuciqdsmexi", response.kind()); + Assertions.assertEquals("fuxtyasiibmiybnn", response.metadata().first()); + Assertions.assertEquals("tgnljhnmgixhcmav", response.metadata().last()); + Assertions.assertEquals("foudor", response.metadata().prev()); + Assertions.assertEquals("gyyprotwy", response.metadata().next()); + Assertions.assertEquals(677853893, response.metadata().totalSize()); + Assertions.assertEquals("xhugcm", response.data().get(0).kind()); + Assertions.assertEquals("avlg", response.data().get(0).id()); + Assertions.assertEquals("mftpmdtz", response.data().get(0).metadata().self()); + Assertions.assertEquals("ltfvnz", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("jtotpvopvpbd", response.data().get(0).metadata().createdAt()); + Assertions.assertEquals("qgqqihedsvqwthmk", response.data().get(0).metadata().updatedAt()); + Assertions.assertEquals("bcysih", response.data().get(0).metadata().deletedAt()); + Assertions.assertEquals("qcwdhoh", response.data().get(0).displayName()); + Assertions.assertEquals("tmcdzsufcohd", response.data().get(0).description()); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListUsersWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListUsersWithResponseMockTests.java index 014f3a9bfa916..c6ad370a3a428 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListUsersWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessListUsersWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testListUsersWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"kind\":\"fpel\",\"metadata\":{\"first\":\"pv\",\"last\":\"r\",\"prev\":\"vu\",\"next\":\"raehtwdwrft\",\"total_size\":767551661},\"data\":[{\"kind\":\"cdl\",\"id\":\"shfwpracstwity\",\"metadata\":{\"self\":\"vxccedcp\",\"resource_name\":\"dyodnwzxltj\",\"created_at\":\"nhltiugcxn\",\"updated_at\":\"vwxqibyqunyo\",\"deleted_at\":\"wlmdjrkv\"},\"email\":\"bvfvpdbod\",\"full_name\":\"izsjqlhkrr\",\"auth_type\":\"deibqip\"},{\"kind\":\"ghvxndzwmkrefa\",\"id\":\"jorwkqnyhgbij\",\"metadata\":{\"self\":\"vfxzsjab\",\"resource_name\":\"systawfsdjp\",\"created_at\":\"vp\",\"updated_at\":\"xbkzbzkdvncj\",\"deleted_at\":\"udurgkakmokz\"},\"email\":\"jk\",\"full_name\":\"fhmouwq\",\"auth_type\":\"zrfze\"},{\"kind\":\"ebizikayuh\",\"id\":\"bjbsybb\",\"metadata\":{\"self\":\"vtldgmfpgvmpip\",\"resource_name\":\"ltha\",\"created_at\":\"x\",\"updated_at\":\"mwutwbdsre\",\"deleted_at\":\"drhneuyow\"},\"email\":\"d\",\"full_name\":\"t\",\"auth_type\":\"ib\"},{\"kind\":\"cgpik\",\"id\":\"imejzanl\",\"metadata\":{\"self\":\"iavrm\",\"resource_name\":\"onok\",\"created_at\":\"rjqc\",\"updated_at\":\"gzpfrla\",\"deleted_at\":\"zrnw\"},\"email\":\"indfpwpjyl\",\"full_name\":\"tlhflsjcdhszf\",\"auth_type\":\"fbgofeljagrqmqh\"}]}"; + = "{\"kind\":\"jtlo\",\"metadata\":{\"first\":\"uojrngiflr\",\"last\":\"asccbiui\",\"prev\":\"dlyjdf\",\"next\":\"mkyoqufdvruzsl\",\"total_size\":15628056},\"data\":[{\"kind\":\"tfnmdx\",\"id\":\"ngfdgugeyzihgrky\",\"metadata\":{\"self\":\"absnmfpp\",\"resource_name\":\"jee\",\"created_at\":\"hyhsgzfczbg\",\"updated_at\":\"fgbegl\",\"deleted_at\":\"leohibetnluankr\"},\"email\":\"xeeebtijvacvbmqz\",\"full_name\":\"q\",\"auth_type\":\"aj\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,29 +51,26 @@ public void testListUsersWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - AccessListUsersSuccessResponse response = manager.access() - .listUsersWithResponse("rgzdwmsweyp", "w", - new ListAccessRequestModel().withSearchFilters( - mapOf("ktt", "gicccnxqhuex", "dtclusiypb", "stvlzywemhzrnc", "ukyhejhzis", "fgytguslfeadcyg")), - com.azure.core.util.Context.NONE) + AccessListUsersSuccessResponse response = manager.access().listUsersWithResponse("qbnj", "rcgegydcwboxjum", + new ListAccessRequestModel().withSearchFilters(mapOf("ouau", "olihrra")), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("fpel", response.kind()); - Assertions.assertEquals("pv", response.metadata().first()); - Assertions.assertEquals("r", response.metadata().last()); - Assertions.assertEquals("vu", response.metadata().prev()); - Assertions.assertEquals("raehtwdwrft", response.metadata().next()); - Assertions.assertEquals(767551661, response.metadata().totalSize()); - Assertions.assertEquals("cdl", response.data().get(0).kind()); - Assertions.assertEquals("shfwpracstwity", response.data().get(0).id()); - Assertions.assertEquals("vxccedcp", response.data().get(0).metadata().self()); - Assertions.assertEquals("dyodnwzxltj", response.data().get(0).metadata().resourceName()); - Assertions.assertEquals("nhltiugcxn", response.data().get(0).metadata().createdAt()); - Assertions.assertEquals("vwxqibyqunyo", response.data().get(0).metadata().updatedAt()); - Assertions.assertEquals("wlmdjrkv", response.data().get(0).metadata().deletedAt()); - Assertions.assertEquals("bvfvpdbod", response.data().get(0).email()); - Assertions.assertEquals("izsjqlhkrr", response.data().get(0).fullName()); - Assertions.assertEquals("deibqip", response.data().get(0).authType()); + Assertions.assertEquals("jtlo", response.kind()); + Assertions.assertEquals("uojrngiflr", response.metadata().first()); + Assertions.assertEquals("asccbiui", response.metadata().last()); + Assertions.assertEquals("dlyjdf", response.metadata().prev()); + Assertions.assertEquals("mkyoqufdvruzsl", response.metadata().next()); + Assertions.assertEquals(15628056, response.metadata().totalSize()); + Assertions.assertEquals("tfnmdx", response.data().get(0).kind()); + Assertions.assertEquals("ngfdgugeyzihgrky", response.data().get(0).id()); + Assertions.assertEquals("absnmfpp", response.data().get(0).metadata().self()); + Assertions.assertEquals("jee", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("hyhsgzfczbg", response.data().get(0).metadata().createdAt()); + Assertions.assertEquals("fgbegl", response.data().get(0).metadata().updatedAt()); + Assertions.assertEquals("leohibetnluankr", response.data().get(0).metadata().deletedAt()); + Assertions.assertEquals("xeeebtijvacvbmqz", response.data().get(0).email()); + Assertions.assertEquals("q", response.data().get(0).fullName()); + Assertions.assertEquals("aj", response.data().get(0).authType()); } // Use "Map.of" if available diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessRoleBindingNameListSuccessResponseInnerTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessRoleBindingNameListSuccessResponseInnerTests.java new file mode 100644 index 0000000000000..179898a9b4ead --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/AccessRoleBindingNameListSuccessResponseInnerTests.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.AccessRoleBindingNameListSuccessResponseInner; +import com.azure.resourcemanager.confluent.models.ConfluentListMetadata; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class AccessRoleBindingNameListSuccessResponseInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + AccessRoleBindingNameListSuccessResponseInner model = BinaryData.fromString( + "{\"kind\":\"btkuwhh\",\"metadata\":{\"first\":\"k\",\"last\":\"oxafn\",\"prev\":\"lpichk\",\"next\":\"mkcdyhbpkkpwdre\",\"total_size\":1405054510},\"data\":[\"qfovljxywsuws\"]}") + .toObject(AccessRoleBindingNameListSuccessResponseInner.class); + Assertions.assertEquals("btkuwhh", model.kind()); + Assertions.assertEquals("k", model.metadata().first()); + Assertions.assertEquals("oxafn", model.metadata().last()); + Assertions.assertEquals("lpichk", model.metadata().prev()); + Assertions.assertEquals("mkcdyhbpkkpwdre", model.metadata().next()); + Assertions.assertEquals(1405054510, model.metadata().totalSize()); + Assertions.assertEquals("qfovljxywsuws", model.data().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + AccessRoleBindingNameListSuccessResponseInner model = new AccessRoleBindingNameListSuccessResponseInner() + .withKind("btkuwhh").withMetadata(new ConfluentListMetadata().withFirst("k").withLast("oxafn") + .withPrev("lpichk").withNext("mkcdyhbpkkpwdre").withTotalSize(1405054510)) + .withData(Arrays.asList("qfovljxywsuws")); + model = BinaryData.fromObject(model).toObject(AccessRoleBindingNameListSuccessResponseInner.class); + Assertions.assertEquals("btkuwhh", model.kind()); + Assertions.assertEquals("k", model.metadata().first()); + Assertions.assertEquals("oxafn", model.metadata().last()); + Assertions.assertEquals("lpichk", model.metadata().prev()); + Assertions.assertEquals("mkcdyhbpkkpwdre", model.metadata().next()); + Assertions.assertEquals(1405054510, model.metadata().totalSize()); + Assertions.assertEquals("qfovljxywsuws", model.data().get(0)); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyOwnerEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyOwnerEntityTests.java new file mode 100644 index 0000000000000..78e3ff74276c2 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyOwnerEntityTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.ApiKeyOwnerEntity; +import org.junit.jupiter.api.Assertions; + +public final class ApiKeyOwnerEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ApiKeyOwnerEntity model = BinaryData + .fromString("{\"id\":\"htba\",\"related\":\"gx\",\"resourceName\":\"rc\",\"kind\":\"yklyhpluodpvruud\"}") + .toObject(ApiKeyOwnerEntity.class); + Assertions.assertEquals("htba", model.id()); + Assertions.assertEquals("gx", model.related()); + Assertions.assertEquals("rc", model.resourceName()); + Assertions.assertEquals("yklyhpluodpvruud", model.kind()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ApiKeyOwnerEntity model = new ApiKeyOwnerEntity().withId("htba").withRelated("gx").withResourceName("rc") + .withKind("yklyhpluodpvruud"); + model = BinaryData.fromObject(model).toObject(ApiKeyOwnerEntity.class); + Assertions.assertEquals("htba", model.id()); + Assertions.assertEquals("gx", model.related()); + Assertions.assertEquals("rc", model.resourceName()); + Assertions.assertEquals("yklyhpluodpvruud", model.kind()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyResourceEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyResourceEntityTests.java new file mode 100644 index 0000000000000..d88ccdd1fee83 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ApiKeyResourceEntityTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.ApiKeyResourceEntity; +import org.junit.jupiter.api.Assertions; + +public final class ApiKeyResourceEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ApiKeyResourceEntity model = BinaryData.fromString( + "{\"id\":\"jmsvpkjp\",\"environment\":\"kwcf\",\"related\":\"ljyxgtczhe\",\"resourceName\":\"bsdshmkxmaehvbbx\",\"kind\":\"iplt\"}") + .toObject(ApiKeyResourceEntity.class); + Assertions.assertEquals("jmsvpkjp", model.id()); + Assertions.assertEquals("kwcf", model.environment()); + Assertions.assertEquals("ljyxgtczhe", model.related()); + Assertions.assertEquals("bsdshmkxmaehvbbx", model.resourceName()); + Assertions.assertEquals("iplt", model.kind()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ApiKeyResourceEntity model = new ApiKeyResourceEntity().withId("jmsvpkjp").withEnvironment("kwcf") + .withRelated("ljyxgtczhe").withResourceName("bsdshmkxmaehvbbx").withKind("iplt"); + model = BinaryData.fromObject(model).toObject(ApiKeyResourceEntity.class); + Assertions.assertEquals("jmsvpkjp", model.id()); + Assertions.assertEquals("kwcf", model.environment()); + Assertions.assertEquals("ljyxgtczhe", model.related()); + Assertions.assertEquals("bsdshmkxmaehvbbx", model.resourceName()); + Assertions.assertEquals("iplt", model.kind()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ClusterPropertiesTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ClusterPropertiesTests.java new file mode 100644 index 0000000000000..e7cca66af596b --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ClusterPropertiesTests.java @@ -0,0 +1,97 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.ClusterProperties; +import com.azure.resourcemanager.confluent.models.ClusterConfigEntity; +import com.azure.resourcemanager.confluent.models.ClusterStatusEntity; +import com.azure.resourcemanager.confluent.models.SCClusterByokEntity; +import com.azure.resourcemanager.confluent.models.SCClusterNetworkEnvironmentEntity; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import org.junit.jupiter.api.Assertions; + +public final class ClusterPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ClusterProperties model = BinaryData.fromString( + "{\"metadata\":{\"self\":\"vxccedcp\",\"resourceName\":\"dyodnwzxltj\",\"createdTimestamp\":\"nhltiugcxn\",\"updatedTimestamp\":\"vwxqibyqunyo\",\"deletedTimestamp\":\"wlmdjrkv\"},\"spec\":{\"name\":\"vfvpdbodaciz\",\"availability\":\"q\",\"cloud\":\"krribdeibqi\",\"zone\":\"kghv\",\"region\":\"dzwmkrefajpj\",\"kafkaBootstrapEndpoint\":\"wkqnyhg\",\"httpEndpoint\":\"j\",\"apiEndpoint\":\"ivfxzsjabibsyst\",\"config\":{\"kind\":\"sdjpvkvp\"},\"environment\":{\"id\":\"bkzbzkd\",\"environment\":\"cjabudurgkakmo\",\"related\":\"hjjklff\",\"resourceName\":\"ouw\"},\"network\":{\"id\":\"zrfze\",\"environment\":\"ebizikayuh\",\"related\":\"bjbsybb\",\"resourceName\":\"r\"},\"byok\":{\"id\":\"dgmfpgvmpipasl\",\"related\":\"aqfxss\",\"resourceName\":\"u\"}},\"status\":{\"phase\":\"dsrezpdrhneuyow\",\"cku\":150720980}}") + .toObject(ClusterProperties.class); + Assertions.assertEquals("vxccedcp", model.metadata().self()); + Assertions.assertEquals("dyodnwzxltj", model.metadata().resourceName()); + Assertions.assertEquals("nhltiugcxn", model.metadata().createdTimestamp()); + Assertions.assertEquals("vwxqibyqunyo", model.metadata().updatedTimestamp()); + Assertions.assertEquals("wlmdjrkv", model.metadata().deletedTimestamp()); + Assertions.assertEquals("vfvpdbodaciz", model.spec().name()); + Assertions.assertEquals("q", model.spec().availability()); + Assertions.assertEquals("krribdeibqi", model.spec().cloud()); + Assertions.assertEquals("kghv", model.spec().zone()); + Assertions.assertEquals("dzwmkrefajpj", model.spec().region()); + Assertions.assertEquals("wkqnyhg", model.spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("j", model.spec().httpEndpoint()); + Assertions.assertEquals("ivfxzsjabibsyst", model.spec().apiEndpoint()); + Assertions.assertEquals("sdjpvkvp", model.spec().config().kind()); + Assertions.assertEquals("bkzbzkd", model.spec().environment().id()); + Assertions.assertEquals("cjabudurgkakmo", model.spec().environment().environment()); + Assertions.assertEquals("hjjklff", model.spec().environment().related()); + Assertions.assertEquals("ouw", model.spec().environment().resourceName()); + Assertions.assertEquals("zrfze", model.spec().network().id()); + Assertions.assertEquals("ebizikayuh", model.spec().network().environment()); + Assertions.assertEquals("bjbsybb", model.spec().network().related()); + Assertions.assertEquals("r", model.spec().network().resourceName()); + Assertions.assertEquals("dgmfpgvmpipasl", model.spec().byok().id()); + Assertions.assertEquals("aqfxss", model.spec().byok().related()); + Assertions.assertEquals("u", model.spec().byok().resourceName()); + Assertions.assertEquals("dsrezpdrhneuyow", model.status().phase()); + Assertions.assertEquals(150720980, model.status().cku()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ClusterProperties model = new ClusterProperties() + .withMetadata(new SCMetadataEntity().withSelf("vxccedcp").withResourceName("dyodnwzxltj") + .withCreatedTimestamp("nhltiugcxn").withUpdatedTimestamp("vwxqibyqunyo") + .withDeletedTimestamp("wlmdjrkv")) + .withSpec(new SCClusterSpecEntity().withName("vfvpdbodaciz").withAvailability("q").withCloud("krribdeibqi") + .withZone("kghv").withRegion("dzwmkrefajpj").withKafkaBootstrapEndpoint("wkqnyhg").withHttpEndpoint("j") + .withApiEndpoint("ivfxzsjabibsyst").withConfig(new ClusterConfigEntity().withKind("sdjpvkvp")) + .withEnvironment(new SCClusterNetworkEnvironmentEntity().withId("bkzbzkd") + .withEnvironment("cjabudurgkakmo").withRelated("hjjklff").withResourceName("ouw")) + .withNetwork(new SCClusterNetworkEnvironmentEntity().withId("zrfze").withEnvironment("ebizikayuh") + .withRelated("bjbsybb").withResourceName("r")) + .withByok( + new SCClusterByokEntity().withId("dgmfpgvmpipasl").withRelated("aqfxss").withResourceName("u"))) + .withStatus(new ClusterStatusEntity().withPhase("dsrezpdrhneuyow").withCku(150720980)); + model = BinaryData.fromObject(model).toObject(ClusterProperties.class); + Assertions.assertEquals("vxccedcp", model.metadata().self()); + Assertions.assertEquals("dyodnwzxltj", model.metadata().resourceName()); + Assertions.assertEquals("nhltiugcxn", model.metadata().createdTimestamp()); + Assertions.assertEquals("vwxqibyqunyo", model.metadata().updatedTimestamp()); + Assertions.assertEquals("wlmdjrkv", model.metadata().deletedTimestamp()); + Assertions.assertEquals("vfvpdbodaciz", model.spec().name()); + Assertions.assertEquals("q", model.spec().availability()); + Assertions.assertEquals("krribdeibqi", model.spec().cloud()); + Assertions.assertEquals("kghv", model.spec().zone()); + Assertions.assertEquals("dzwmkrefajpj", model.spec().region()); + Assertions.assertEquals("wkqnyhg", model.spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("j", model.spec().httpEndpoint()); + Assertions.assertEquals("ivfxzsjabibsyst", model.spec().apiEndpoint()); + Assertions.assertEquals("sdjpvkvp", model.spec().config().kind()); + Assertions.assertEquals("bkzbzkd", model.spec().environment().id()); + Assertions.assertEquals("cjabudurgkakmo", model.spec().environment().environment()); + Assertions.assertEquals("hjjklff", model.spec().environment().related()); + Assertions.assertEquals("ouw", model.spec().environment().resourceName()); + Assertions.assertEquals("zrfze", model.spec().network().id()); + Assertions.assertEquals("ebizikayuh", model.spec().network().environment()); + Assertions.assertEquals("bjbsybb", model.spec().network().related()); + Assertions.assertEquals("r", model.spec().network().resourceName()); + Assertions.assertEquals("dgmfpgvmpipasl", model.spec().byok().id()); + Assertions.assertEquals("aqfxss", model.spec().byok().related()); + Assertions.assertEquals("u", model.spec().byok().resourceName()); + Assertions.assertEquals("dsrezpdrhneuyow", model.status().phase()); + Assertions.assertEquals(150720980, model.status().cku()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/CreateApiKeyModelTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/CreateApiKeyModelTests.java new file mode 100644 index 0000000000000..38e239c1e8e20 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/CreateApiKeyModelTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.CreateApiKeyModel; +import org.junit.jupiter.api.Assertions; + +public final class CreateApiKeyModelTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CreateApiKeyModel model + = BinaryData.fromString("{\"name\":\"wp\",\"description\":\"sutrgjup\"}").toObject(CreateApiKeyModel.class); + Assertions.assertEquals("wp", model.name()); + Assertions.assertEquals("sutrgjup", model.description()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CreateApiKeyModel model = new CreateApiKeyModel().withName("wp").withDescription("sutrgjup"); + model = BinaryData.fromObject(model).toObject(CreateApiKeyModel.class); + Assertions.assertEquals("wp", model.name()); + Assertions.assertEquals("sutrgjup", model.description()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/EnvironmentPropertiesTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/EnvironmentPropertiesTests.java new file mode 100644 index 0000000000000..d251f4b2b8ce3 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/EnvironmentPropertiesTests.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.EnvironmentProperties; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import org.junit.jupiter.api.Assertions; + +public final class EnvironmentPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + EnvironmentProperties model = BinaryData.fromString( + "{\"metadata\":{\"self\":\"jhxbld\",\"resourceName\":\"wwrlkdmtncv\",\"createdTimestamp\":\"otllxdyhgsyo\",\"updatedTimestamp\":\"gjltdtbnnhado\",\"deletedTimestamp\":\"rkvcikhnvpa\"}}") + .toObject(EnvironmentProperties.class); + Assertions.assertEquals("jhxbld", model.metadata().self()); + Assertions.assertEquals("wwrlkdmtncv", model.metadata().resourceName()); + Assertions.assertEquals("otllxdyhgsyo", model.metadata().createdTimestamp()); + Assertions.assertEquals("gjltdtbnnhado", model.metadata().updatedTimestamp()); + Assertions.assertEquals("rkvcikhnvpa", model.metadata().deletedTimestamp()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + EnvironmentProperties model = new EnvironmentProperties().withMetadata(new SCMetadataEntity().withSelf("jhxbld") + .withResourceName("wwrlkdmtncv").withCreatedTimestamp("otllxdyhgsyo").withUpdatedTimestamp("gjltdtbnnhado") + .withDeletedTimestamp("rkvcikhnvpa")); + model = BinaryData.fromObject(model).toObject(EnvironmentProperties.class); + Assertions.assertEquals("jhxbld", model.metadata().self()); + Assertions.assertEquals("wwrlkdmtncv", model.metadata().resourceName()); + Assertions.assertEquals("otllxdyhgsyo", model.metadata().createdTimestamp()); + Assertions.assertEquals("gjltdtbnnhado", model.metadata().updatedTimestamp()); + Assertions.assertEquals("rkvcikhnvpa", model.metadata().deletedTimestamp()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/GetEnvironmentsResponseTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/GetEnvironmentsResponseTests.java new file mode 100644 index 0000000000000..bf5db74e27662 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/GetEnvironmentsResponseTests.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.azure.resourcemanager.confluent.models.GetEnvironmentsResponse; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class GetEnvironmentsResponseTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + GetEnvironmentsResponse model = BinaryData.fromString( + "{\"value\":[{\"kind\":\"dsytgadgvr\",\"id\":\"aeneqnzarrwl\",\"properties\":{\"metadata\":{\"self\":\"jfqka\",\"resourceName\":\"wiipfpub\",\"createdTimestamp\":\"bwwift\",\"updatedTimestamp\":\"qkvpuvksgplsakn\",\"deletedTimestamp\":\"fsynljphuop\"}},\"name\":\"dlqiyntorzih\"}],\"nextLink\":\"osjswsr\"}") + .toObject(GetEnvironmentsResponse.class); + Assertions.assertEquals("dsytgadgvr", model.value().get(0).kind()); + Assertions.assertEquals("aeneqnzarrwl", model.value().get(0).id()); + Assertions.assertEquals("dlqiyntorzih", model.value().get(0).name()); + Assertions.assertEquals("jfqka", model.value().get(0).metadata().self()); + Assertions.assertEquals("wiipfpub", model.value().get(0).metadata().resourceName()); + Assertions.assertEquals("bwwift", model.value().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("qkvpuvksgplsakn", model.value().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("fsynljphuop", model.value().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("osjswsr", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + GetEnvironmentsResponse model + = new GetEnvironmentsResponse().withValue(Arrays.asList( + new SCEnvironmentRecordInner().withKind("dsytgadgvr").withId("aeneqnzarrwl").withName("dlqiyntorzih") + .withMetadata(new SCMetadataEntity().withSelf("jfqka").withResourceName("wiipfpub") + .withCreatedTimestamp("bwwift").withUpdatedTimestamp("qkvpuvksgplsakn") + .withDeletedTimestamp("fsynljphuop")))) + .withNextLink("osjswsr"); + model = BinaryData.fromObject(model).toObject(GetEnvironmentsResponse.class); + Assertions.assertEquals("dsytgadgvr", model.value().get(0).kind()); + Assertions.assertEquals("aeneqnzarrwl", model.value().get(0).id()); + Assertions.assertEquals("dlqiyntorzih", model.value().get(0).name()); + Assertions.assertEquals("jfqka", model.value().get(0).metadata().self()); + Assertions.assertEquals("wiipfpub", model.value().get(0).metadata().resourceName()); + Assertions.assertEquals("bwwift", model.value().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("qkvpuvksgplsakn", model.value().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("fsynljphuop", model.value().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("osjswsr", model.nextLink()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListClustersSuccessResponseTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListClustersSuccessResponseTests.java new file mode 100644 index 0000000000000..2ee82ca335f97 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListClustersSuccessResponseTests.java @@ -0,0 +1,140 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.azure.resourcemanager.confluent.models.ClusterConfigEntity; +import com.azure.resourcemanager.confluent.models.ClusterStatusEntity; +import com.azure.resourcemanager.confluent.models.ListClustersSuccessResponse; +import com.azure.resourcemanager.confluent.models.SCClusterByokEntity; +import com.azure.resourcemanager.confluent.models.SCClusterNetworkEnvironmentEntity; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ListClustersSuccessResponseTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ListClustersSuccessResponse model = BinaryData.fromString( + "{\"value\":[{\"kind\":\"mtdaa\",\"id\":\"dvwvgpio\",\"properties\":{\"metadata\":{\"self\":\"rtfudxepxg\",\"resourceName\":\"agvrvmnpkuk\",\"createdTimestamp\":\"i\",\"updatedTimestamp\":\"blxgwimf\",\"deletedTimestamp\":\"hfjx\"},\"spec\":{\"name\":\"zk\",\"availability\":\"oqreyfkzikfjawn\",\"cloud\":\"ivx\",\"zone\":\"zel\",\"region\":\"irels\",\"kafkaBootstrapEndpoint\":\"aenwabf\",\"httpEndpoint\":\"kl\",\"apiEndpoint\":\"xbjhwuaanozjosph\",\"config\":{\"kind\":\"l\"},\"environment\":{\"id\":\"v\",\"environment\":\"glrvimjwosytxi\",\"related\":\"skfc\",\"resourceName\":\"qumiek\"},\"network\":{\"id\":\"zikhl\",\"environment\":\"jhdgqggebdunyga\",\"related\":\"idb\",\"resourceName\":\"atpxl\"},\"byok\":{\"id\":\"cyjmoadsuvarmy\",\"related\":\"mjsjqb\",\"resourceName\":\"hyxxrwlycoduhpk\"}},\"status\":{\"phase\":\"ymareqnajxqugj\",\"cku\":123081657}},\"name\":\"ubeddg\"},{\"kind\":\"ofwq\",\"id\":\"qal\",\"properties\":{\"metadata\":{\"self\":\"jijpxac\",\"resourceName\":\"udfnbyxba\",\"createdTimestamp\":\"bjyvay\",\"updatedTimestamp\":\"imrzrtuzqog\",\"deletedTimestamp\":\"xnevfdnwn\"},\"spec\":{\"name\":\"wzsyyceuzs\",\"availability\":\"bjudpfrxtrthzv\",\"cloud\":\"tdwkqbrq\",\"zone\":\"paxh\",\"region\":\"iilivpdtiirqtd\",\"kafkaBootstrapEndpoint\":\"axoruzfgsquy\",\"httpEndpoint\":\"rxxle\",\"apiEndpoint\":\"ramxjezwlwnw\",\"config\":{\"kind\":\"lcvydy\"},\"environment\":{\"id\":\"dooaojkniodko\",\"environment\":\"bw\",\"related\":\"jhemms\",\"resourceName\":\"dkcrodt\"},\"network\":{\"id\":\"fw\",\"environment\":\"fltkacjv\",\"related\":\"kdlfoa\",\"resourceName\":\"gkfpaga\"},\"byok\":{\"id\":\"ulpqblylsyxkqjn\",\"related\":\"ervtiagxs\",\"resourceName\":\"zuempsbzkf\"}},\"status\":{\"phase\":\"yvpnqicvinvkjj\",\"cku\":602936654}},\"name\":\"buukzclewyhml\"},{\"kind\":\"aztz\",\"id\":\"fn\",\"properties\":{\"metadata\":{\"self\":\"yfzqwhxxbu\",\"resourceName\":\"a\",\"createdTimestamp\":\"feqztppriol\",\"updatedTimestamp\":\"rjaltolmncw\",\"deletedTimestamp\":\"bqwcsdbnwdcf\"},\"spec\":{\"name\":\"qdpfuvglsbjjca\",\"availability\":\"xbvtvudu\",\"cloud\":\"cormr\",\"zone\":\"qtvcofudflvkgj\",\"region\":\"gdknnqv\",\"kafkaBootstrapEndpoint\":\"znqntoru\",\"httpEndpoint\":\"gsahmkycgrauw\",\"apiEndpoint\":\"etaebu\",\"config\":{\"kind\":\"dmovsm\"},\"environment\":{\"id\":\"wabm\",\"environment\":\"efkifr\",\"related\":\"puqujmqlgkfbtn\",\"resourceName\":\"aongbj\"},\"network\":{\"id\":\"ujitcjedftww\",\"environment\":\"zkoj\",\"related\":\"c\",\"resourceName\":\"foqouicybx\"},\"byok\":{\"id\":\"gszufoxciqopid\",\"related\":\"mciodhkhazxkhn\",\"resourceName\":\"onlwntoeg\"}},\"status\":{\"phase\":\"wbw\",\"cku\":924333329}},\"name\":\"zcmrvexztvb\"}],\"nextLink\":\"gsfraoyzkoow\"}") + .toObject(ListClustersSuccessResponse.class); + Assertions.assertEquals("mtdaa", model.value().get(0).kind()); + Assertions.assertEquals("dvwvgpio", model.value().get(0).id()); + Assertions.assertEquals("ubeddg", model.value().get(0).name()); + Assertions.assertEquals("rtfudxepxg", model.value().get(0).metadata().self()); + Assertions.assertEquals("agvrvmnpkuk", model.value().get(0).metadata().resourceName()); + Assertions.assertEquals("i", model.value().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("blxgwimf", model.value().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("hfjx", model.value().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("zk", model.value().get(0).spec().name()); + Assertions.assertEquals("oqreyfkzikfjawn", model.value().get(0).spec().availability()); + Assertions.assertEquals("ivx", model.value().get(0).spec().cloud()); + Assertions.assertEquals("zel", model.value().get(0).spec().zone()); + Assertions.assertEquals("irels", model.value().get(0).spec().region()); + Assertions.assertEquals("aenwabf", model.value().get(0).spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("kl", model.value().get(0).spec().httpEndpoint()); + Assertions.assertEquals("xbjhwuaanozjosph", model.value().get(0).spec().apiEndpoint()); + Assertions.assertEquals("l", model.value().get(0).spec().config().kind()); + Assertions.assertEquals("v", model.value().get(0).spec().environment().id()); + Assertions.assertEquals("glrvimjwosytxi", model.value().get(0).spec().environment().environment()); + Assertions.assertEquals("skfc", model.value().get(0).spec().environment().related()); + Assertions.assertEquals("qumiek", model.value().get(0).spec().environment().resourceName()); + Assertions.assertEquals("zikhl", model.value().get(0).spec().network().id()); + Assertions.assertEquals("jhdgqggebdunyga", model.value().get(0).spec().network().environment()); + Assertions.assertEquals("idb", model.value().get(0).spec().network().related()); + Assertions.assertEquals("atpxl", model.value().get(0).spec().network().resourceName()); + Assertions.assertEquals("cyjmoadsuvarmy", model.value().get(0).spec().byok().id()); + Assertions.assertEquals("mjsjqb", model.value().get(0).spec().byok().related()); + Assertions.assertEquals("hyxxrwlycoduhpk", model.value().get(0).spec().byok().resourceName()); + Assertions.assertEquals("ymareqnajxqugj", model.value().get(0).status().phase()); + Assertions.assertEquals(123081657, model.value().get(0).status().cku()); + Assertions.assertEquals("gsfraoyzkoow", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ListClustersSuccessResponse model = new ListClustersSuccessResponse() + .withValue(Arrays.asList( + new SCClusterRecordInner().withKind("mtdaa").withId("dvwvgpio").withName("ubeddg") + .withMetadata(new SCMetadataEntity().withSelf("rtfudxepxg").withResourceName("agvrvmnpkuk") + .withCreatedTimestamp("i").withUpdatedTimestamp("blxgwimf").withDeletedTimestamp("hfjx")) + .withSpec(new SCClusterSpecEntity().withName("zk").withAvailability("oqreyfkzikfjawn") + .withCloud("ivx").withZone("zel").withRegion("irels").withKafkaBootstrapEndpoint("aenwabf") + .withHttpEndpoint("kl").withApiEndpoint("xbjhwuaanozjosph") + .withConfig(new ClusterConfigEntity().withKind("l")) + .withEnvironment(new SCClusterNetworkEnvironmentEntity().withId("v") + .withEnvironment("glrvimjwosytxi").withRelated("skfc").withResourceName("qumiek")) + .withNetwork(new SCClusterNetworkEnvironmentEntity().withId("zikhl") + .withEnvironment("jhdgqggebdunyga").withRelated("idb").withResourceName("atpxl")) + .withByok(new SCClusterByokEntity().withId("cyjmoadsuvarmy").withRelated("mjsjqb") + .withResourceName("hyxxrwlycoduhpk"))) + .withStatus(new ClusterStatusEntity().withPhase("ymareqnajxqugj").withCku(123081657)), + new SCClusterRecordInner().withKind("ofwq").withId("qal").withName("buukzclewyhml") + .withMetadata(new SCMetadataEntity().withSelf("jijpxac").withResourceName("udfnbyxba") + .withCreatedTimestamp("bjyvay").withUpdatedTimestamp("imrzrtuzqog") + .withDeletedTimestamp("xnevfdnwn")) + .withSpec(new SCClusterSpecEntity().withName("wzsyyceuzs").withAvailability("bjudpfrxtrthzv") + .withCloud("tdwkqbrq").withZone("paxh").withRegion("iilivpdtiirqtd") + .withKafkaBootstrapEndpoint("axoruzfgsquy").withHttpEndpoint("rxxle") + .withApiEndpoint("ramxjezwlwnw").withConfig(new ClusterConfigEntity().withKind("lcvydy")) + .withEnvironment(new SCClusterNetworkEnvironmentEntity().withId("dooaojkniodko") + .withEnvironment("bw").withRelated("jhemms").withResourceName("dkcrodt")) + .withNetwork(new SCClusterNetworkEnvironmentEntity().withId("fw").withEnvironment("fltkacjv") + .withRelated("kdlfoa").withResourceName("gkfpaga")) + .withByok(new SCClusterByokEntity().withId("ulpqblylsyxkqjn").withRelated("ervtiagxs") + .withResourceName("zuempsbzkf"))) + .withStatus(new ClusterStatusEntity().withPhase("yvpnqicvinvkjj").withCku(602936654)), + new SCClusterRecordInner().withKind("aztz").withId("fn").withName("zcmrvexztvb") + .withMetadata(new SCMetadataEntity().withSelf("yfzqwhxxbu").withResourceName("a") + .withCreatedTimestamp("feqztppriol").withUpdatedTimestamp("rjaltolmncw") + .withDeletedTimestamp("bqwcsdbnwdcf")) + .withSpec(new SCClusterSpecEntity().withName("qdpfuvglsbjjca").withAvailability("xbvtvudu") + .withCloud("cormr").withZone("qtvcofudflvkgj").withRegion("gdknnqv") + .withKafkaBootstrapEndpoint("znqntoru").withHttpEndpoint("gsahmkycgrauw") + .withApiEndpoint("etaebu").withConfig(new ClusterConfigEntity().withKind("dmovsm")) + .withEnvironment(new SCClusterNetworkEnvironmentEntity().withId("wabm") + .withEnvironment("efkifr").withRelated("puqujmqlgkfbtn").withResourceName("aongbj")) + .withNetwork(new SCClusterNetworkEnvironmentEntity().withId("ujitcjedftww") + .withEnvironment("zkoj").withRelated("c").withResourceName("foqouicybx")) + .withByok(new SCClusterByokEntity().withId("gszufoxciqopid").withRelated("mciodhkhazxkhn") + .withResourceName("onlwntoeg"))) + .withStatus(new ClusterStatusEntity().withPhase("wbw").withCku(924333329)))) + .withNextLink("gsfraoyzkoow"); + model = BinaryData.fromObject(model).toObject(ListClustersSuccessResponse.class); + Assertions.assertEquals("mtdaa", model.value().get(0).kind()); + Assertions.assertEquals("dvwvgpio", model.value().get(0).id()); + Assertions.assertEquals("ubeddg", model.value().get(0).name()); + Assertions.assertEquals("rtfudxepxg", model.value().get(0).metadata().self()); + Assertions.assertEquals("agvrvmnpkuk", model.value().get(0).metadata().resourceName()); + Assertions.assertEquals("i", model.value().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("blxgwimf", model.value().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("hfjx", model.value().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("zk", model.value().get(0).spec().name()); + Assertions.assertEquals("oqreyfkzikfjawn", model.value().get(0).spec().availability()); + Assertions.assertEquals("ivx", model.value().get(0).spec().cloud()); + Assertions.assertEquals("zel", model.value().get(0).spec().zone()); + Assertions.assertEquals("irels", model.value().get(0).spec().region()); + Assertions.assertEquals("aenwabf", model.value().get(0).spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("kl", model.value().get(0).spec().httpEndpoint()); + Assertions.assertEquals("xbjhwuaanozjosph", model.value().get(0).spec().apiEndpoint()); + Assertions.assertEquals("l", model.value().get(0).spec().config().kind()); + Assertions.assertEquals("v", model.value().get(0).spec().environment().id()); + Assertions.assertEquals("glrvimjwosytxi", model.value().get(0).spec().environment().environment()); + Assertions.assertEquals("skfc", model.value().get(0).spec().environment().related()); + Assertions.assertEquals("qumiek", model.value().get(0).spec().environment().resourceName()); + Assertions.assertEquals("zikhl", model.value().get(0).spec().network().id()); + Assertions.assertEquals("jhdgqggebdunyga", model.value().get(0).spec().network().environment()); + Assertions.assertEquals("idb", model.value().get(0).spec().network().related()); + Assertions.assertEquals("atpxl", model.value().get(0).spec().network().resourceName()); + Assertions.assertEquals("cyjmoadsuvarmy", model.value().get(0).spec().byok().id()); + Assertions.assertEquals("mjsjqb", model.value().get(0).spec().byok().related()); + Assertions.assertEquals("hyxxrwlycoduhpk", model.value().get(0).spec().byok().resourceName()); + Assertions.assertEquals("ymareqnajxqugj", model.value().get(0).status().phase()); + Assertions.assertEquals(123081657, model.value().get(0).status().cku()); + Assertions.assertEquals("gsfraoyzkoow", model.nextLink()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListRegionsSuccessResponseInnerTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListRegionsSuccessResponseInnerTests.java new file mode 100644 index 0000000000000..3b84bee5c2050 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListRegionsSuccessResponseInnerTests.java @@ -0,0 +1,56 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.ListRegionsSuccessResponseInner; +import com.azure.resourcemanager.confluent.models.RegionRecord; +import com.azure.resourcemanager.confluent.models.RegionSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ListRegionsSuccessResponseInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ListRegionsSuccessResponseInner model = BinaryData.fromString( + "{\"data\":[{\"kind\":\"vvt\",\"id\":\"einqf\",\"properties\":{\"metadata\":{\"self\":\"qknp\",\"resourceName\":\"gnepttwqmsni\",\"createdTimestamp\":\"cdm\",\"updatedTimestamp\":\"r\",\"deletedTimestamp\":\"lpijnkrxfrd\"},\"spec\":{\"name\":\"ratiz\",\"cloud\":\"onasxifto\",\"regionName\":\"yzhftwesgogczh\",\"packages\":[\"xkr\",\"gnyhmossxkkg\",\"h\",\"rghxjb\"]}}}]}") + .toObject(ListRegionsSuccessResponseInner.class); + Assertions.assertEquals("vvt", model.data().get(0).kind()); + Assertions.assertEquals("einqf", model.data().get(0).id()); + Assertions.assertEquals("qknp", model.data().get(0).metadata().self()); + Assertions.assertEquals("gnepttwqmsni", model.data().get(0).metadata().resourceName()); + Assertions.assertEquals("cdm", model.data().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("r", model.data().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("lpijnkrxfrd", model.data().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("ratiz", model.data().get(0).spec().name()); + Assertions.assertEquals("onasxifto", model.data().get(0).spec().cloud()); + Assertions.assertEquals("yzhftwesgogczh", model.data().get(0).spec().regionName()); + Assertions.assertEquals("xkr", model.data().get(0).spec().packages().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ListRegionsSuccessResponseInner model = new ListRegionsSuccessResponseInner() + .withData(Arrays.asList(new RegionRecord().withKind("vvt").withId("einqf") + .withMetadata(new SCMetadataEntity().withSelf("qknp").withResourceName("gnepttwqmsni") + .withCreatedTimestamp("cdm").withUpdatedTimestamp("r").withDeletedTimestamp("lpijnkrxfrd")) + .withSpec( + new RegionSpecEntity().withName("ratiz").withCloud("onasxifto").withRegionName("yzhftwesgogczh") + .withPackages(Arrays.asList("xkr", "gnyhmossxkkg", "h", "rghxjb"))))); + model = BinaryData.fromObject(model).toObject(ListRegionsSuccessResponseInner.class); + Assertions.assertEquals("vvt", model.data().get(0).kind()); + Assertions.assertEquals("einqf", model.data().get(0).id()); + Assertions.assertEquals("qknp", model.data().get(0).metadata().self()); + Assertions.assertEquals("gnepttwqmsni", model.data().get(0).metadata().resourceName()); + Assertions.assertEquals("cdm", model.data().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("r", model.data().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("lpijnkrxfrd", model.data().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("ratiz", model.data().get(0).spec().name()); + Assertions.assertEquals("onasxifto", model.data().get(0).spec().cloud()); + Assertions.assertEquals("yzhftwesgogczh", model.data().get(0).spec().regionName()); + Assertions.assertEquals("xkr", model.data().get(0).spec().packages().get(0)); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListSchemaRegistryClustersResponseTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListSchemaRegistryClustersResponseTests.java new file mode 100644 index 0000000000000..c72c4cbd79797 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/ListSchemaRegistryClustersResponseTests.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.azure.resourcemanager.confluent.models.ListSchemaRegistryClustersResponse; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterEnvironmentRegionEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ListSchemaRegistryClustersResponseTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ListSchemaRegistryClustersResponse model = BinaryData.fromString( + "{\"value\":[{\"kind\":\"fhwygzlvdnkfxus\",\"id\":\"dwzrmuh\",\"properties\":{\"metadata\":{\"self\":\"qdpsqxqvpsvu\",\"resourceName\":\"mgccelvezrypq\",\"createdTimestamp\":\"feo\",\"updatedTimestamp\":\"rqwky\",\"deletedTimestamp\":\"ob\"},\"spec\":{\"name\":\"xedk\",\"httpEndpoint\":\"epbqpcrfkbw\",\"package\":\"snjvcdwxlpqekftn\",\"region\":{\"id\":\"jsyingwfqatm\",\"related\":\"htmdvy\",\"resourceName\":\"ikdgszywkbir\"},\"environment\":{\"id\":\"zh\",\"related\":\"kj\",\"resourceName\":\"rvqqaatj\"},\"cloud\":\"rv\"},\"status\":{\"phase\":\"pmfi\"}}}],\"nextLink\":\"fggjioolvr\"}") + .toObject(ListSchemaRegistryClustersResponse.class); + Assertions.assertEquals("fhwygzlvdnkfxus", model.value().get(0).kind()); + Assertions.assertEquals("dwzrmuh", model.value().get(0).id()); + Assertions.assertEquals("qdpsqxqvpsvu", model.value().get(0).metadata().self()); + Assertions.assertEquals("mgccelvezrypq", model.value().get(0).metadata().resourceName()); + Assertions.assertEquals("feo", model.value().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("rqwky", model.value().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("ob", model.value().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("xedk", model.value().get(0).spec().name()); + Assertions.assertEquals("epbqpcrfkbw", model.value().get(0).spec().httpEndpoint()); + Assertions.assertEquals("snjvcdwxlpqekftn", model.value().get(0).spec().packageProperty()); + Assertions.assertEquals("jsyingwfqatm", model.value().get(0).spec().region().id()); + Assertions.assertEquals("htmdvy", model.value().get(0).spec().region().related()); + Assertions.assertEquals("ikdgszywkbir", model.value().get(0).spec().region().resourceName()); + Assertions.assertEquals("zh", model.value().get(0).spec().environment().id()); + Assertions.assertEquals("kj", model.value().get(0).spec().environment().related()); + Assertions.assertEquals("rvqqaatj", model.value().get(0).spec().environment().resourceName()); + Assertions.assertEquals("rv", model.value().get(0).spec().cloud()); + Assertions.assertEquals("pmfi", model.value().get(0).status().phase()); + Assertions.assertEquals("fggjioolvr", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ListSchemaRegistryClustersResponse model = new ListSchemaRegistryClustersResponse().withValue( + Arrays.asList(new SchemaRegistryClusterRecordInner().withKind("fhwygzlvdnkfxus").withId("dwzrmuh") + .withMetadata(new SCMetadataEntity().withSelf("qdpsqxqvpsvu").withResourceName("mgccelvezrypq") + .withCreatedTimestamp("feo").withUpdatedTimestamp("rqwky").withDeletedTimestamp("ob")) + .withSpec(new SchemaRegistryClusterSpecEntity().withName("xedk").withHttpEndpoint("epbqpcrfkbw") + .withPackageProperty("snjvcdwxlpqekftn") + .withRegion(new SchemaRegistryClusterEnvironmentRegionEntity().withId("jsyingwfqatm") + .withRelated("htmdvy").withResourceName("ikdgszywkbir")) + .withEnvironment(new SchemaRegistryClusterEnvironmentRegionEntity().withId("zh").withRelated("kj") + .withResourceName("rvqqaatj")) + .withCloud("rv")) + .withStatus(new SchemaRegistryClusterStatusEntity().withPhase("pmfi")))) + .withNextLink("fggjioolvr"); + model = BinaryData.fromObject(model).toObject(ListSchemaRegistryClustersResponse.class); + Assertions.assertEquals("fhwygzlvdnkfxus", model.value().get(0).kind()); + Assertions.assertEquals("dwzrmuh", model.value().get(0).id()); + Assertions.assertEquals("qdpsqxqvpsvu", model.value().get(0).metadata().self()); + Assertions.assertEquals("mgccelvezrypq", model.value().get(0).metadata().resourceName()); + Assertions.assertEquals("feo", model.value().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("rqwky", model.value().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("ob", model.value().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("xedk", model.value().get(0).spec().name()); + Assertions.assertEquals("epbqpcrfkbw", model.value().get(0).spec().httpEndpoint()); + Assertions.assertEquals("snjvcdwxlpqekftn", model.value().get(0).spec().packageProperty()); + Assertions.assertEquals("jsyingwfqatm", model.value().get(0).spec().region().id()); + Assertions.assertEquals("htmdvy", model.value().get(0).spec().region().related()); + Assertions.assertEquals("ikdgszywkbir", model.value().get(0).spec().region().resourceName()); + Assertions.assertEquals("zh", model.value().get(0).spec().environment().id()); + Assertions.assertEquals("kj", model.value().get(0).spec().environment().related()); + Assertions.assertEquals("rvqqaatj", model.value().get(0).spec().environment().resourceName()); + Assertions.assertEquals("rv", model.value().get(0).spec().cloud()); + Assertions.assertEquals("pmfi", model.value().get(0).status().phase()); + Assertions.assertEquals("fggjioolvr", model.nextLink()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateWithResponseMockTests.java index 1b23e78da2c1b..7db090c51bddd 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsCreateWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testCreateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"publisher\":\"zrpzb\",\"product\":\"ckqqzqioxiysui\",\"plan\":\"ynkedyatrwyhqmib\",\"licenseTextLink\":\"hwit\",\"privacyPolicyLink\":\"ypyynpcdpumnzg\",\"retrieveDatetime\":\"2021-04-26T17:55:24Z\",\"signature\":\"mabiknsorgjhxb\",\"accepted\":false},\"id\":\"lwwrl\",\"name\":\"dmtnc\",\"type\":\"ok\"}"; + = "{\"properties\":{\"publisher\":\"kmr\",\"product\":\"mvvhmxtdrjfuta\",\"plan\":\"ebjvewzcjzn\",\"licenseTextLink\":\"cpmguaadraufact\",\"privacyPolicyLink\":\"hzovaj\",\"retrieveDatetime\":\"2021-02-15T15:09:11Z\",\"signature\":\"xxpshneeku\",\"accepted\":false},\"id\":\"slqubkwdl\",\"name\":\"nrdsutujbazpjuoh\",\"type\":\"inyflnorwmduvwp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,19 +50,19 @@ public void testCreateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ConfluentAgreementResource response = manager.marketplaceAgreements() - .createWithResponse(new ConfluentAgreementResourceInner().withPublisher("tgadgvraeaen").withProduct("nzar") - .withPlan("lquuijfqkacewii").withLicenseTextLink("pubjibw").withPrivacyPolicyLink("f") - .withRetrieveDatetime(OffsetDateTime.parse("2021-01-20T11:33:50Z")).withSignature("kvpuvksgplsaknyn") + .createWithResponse(new ConfluentAgreementResourceInner().withPublisher("oookkqfq").withProduct("vleo") + .withPlan("ml").withLicenseTextLink("qtqzfavyv").withPrivacyPolicyLink("qybaryeua") + .withRetrieveDatetime(OffsetDateTime.parse("2021-06-23T02:25:47Z")).withSignature("abqgzslesjcbh") .withAccepted(false), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("zrpzb", response.publisher()); - Assertions.assertEquals("ckqqzqioxiysui", response.product()); - Assertions.assertEquals("ynkedyatrwyhqmib", response.plan()); - Assertions.assertEquals("hwit", response.licenseTextLink()); - Assertions.assertEquals("ypyynpcdpumnzg", response.privacyPolicyLink()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-26T17:55:24Z"), response.retrieveDatetime()); - Assertions.assertEquals("mabiknsorgjhxb", response.signature()); + Assertions.assertEquals("kmr", response.publisher()); + Assertions.assertEquals("mvvhmxtdrjfuta", response.product()); + Assertions.assertEquals("ebjvewzcjzn", response.plan()); + Assertions.assertEquals("cpmguaadraufact", response.licenseTextLink()); + Assertions.assertEquals("hzovaj", response.privacyPolicyLink()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-15T15:09:11Z"), response.retrieveDatetime()); + Assertions.assertEquals("xxpshneeku", response.signature()); Assertions.assertEquals(false, response.accepted()); } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListMockTests.java index 52f9c8bdbf4fc..8a699f067dc16 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/MarketplaceAgreementsListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"publisher\":\"obyu\",\"product\":\"rpqlp\",\"plan\":\"cciuqgbdbutau\",\"licenseTextLink\":\"btkuwhh\",\"privacyPolicyLink\":\"ykojoxafnndlpic\",\"retrieveDatetime\":\"2021-09-25T08:27:56Z\",\"signature\":\"mkcdyhbpkkpwdre\",\"accepted\":true},\"id\":\"vvqfovljxyws\",\"name\":\"w\",\"type\":\"yrs\"}]}"; + = "{\"value\":[{\"properties\":{\"publisher\":\"bth\",\"product\":\"tgk\",\"plan\":\"tvdxeclzedqb\",\"licenseTextLink\":\"hzlhplodqkdlww\",\"privacyPolicyLink\":\"bum\",\"retrieveDatetime\":\"2021-01-09T13:22:40Z\",\"signature\":\"rqjfsmlm\",\"accepted\":true},\"id\":\"hwgfwsrt\",\"name\":\"wcoezbrhub\",\"type\":\"kh\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,14 +52,14 @@ public void testList() throws Exception { PagedIterable response = manager.marketplaceAgreements().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("obyu", response.iterator().next().publisher()); - Assertions.assertEquals("rpqlp", response.iterator().next().product()); - Assertions.assertEquals("cciuqgbdbutau", response.iterator().next().plan()); - Assertions.assertEquals("btkuwhh", response.iterator().next().licenseTextLink()); - Assertions.assertEquals("ykojoxafnndlpic", response.iterator().next().privacyPolicyLink()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-25T08:27:56Z"), + Assertions.assertEquals("bth", response.iterator().next().publisher()); + Assertions.assertEquals("tgk", response.iterator().next().product()); + Assertions.assertEquals("tvdxeclzedqb", response.iterator().next().plan()); + Assertions.assertEquals("hzlhplodqkdlww", response.iterator().next().licenseTextLink()); + Assertions.assertEquals("bum", response.iterator().next().privacyPolicyLink()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-09T13:22:40Z"), response.iterator().next().retrieveDatetime()); - Assertions.assertEquals("mkcdyhbpkkpwdre", response.iterator().next().signature()); + Assertions.assertEquals("rqjfsmlm", response.iterator().next().signature()); Assertions.assertEquals(true, response.iterator().next().accepted()); } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListMockTests.java index e663d0b7e9c25..ef344d83d3107 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationOperationsListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"llxdyhgs\",\"display\":{\"provider\":\"ogjltdtbnnhad\",\"resource\":\"crkvcikhnv\",\"operation\":\"mqg\",\"description\":\"queziky\"},\"isDataAction\":false}]}"; + = "{\"value\":[{\"name\":\"vxwmygd\",\"display\":{\"provider\":\"pqchiszep\",\"resource\":\"bjcrxgibbdaxco\",\"operation\":\"ozauorsukokwb\",\"description\":\"lhlv\"},\"isDataAction\":false}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,11 +52,11 @@ public void testList() throws Exception { PagedIterable response = manager.organizationOperations().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("llxdyhgs", response.iterator().next().name()); - Assertions.assertEquals("ogjltdtbnnhad", response.iterator().next().display().provider()); - Assertions.assertEquals("crkvcikhnv", response.iterator().next().display().resource()); - Assertions.assertEquals("mqg", response.iterator().next().display().operation()); - Assertions.assertEquals("queziky", response.iterator().next().display().description()); + Assertions.assertEquals("vxwmygd", response.iterator().next().name()); + Assertions.assertEquals("pqchiszep", response.iterator().next().display().provider()); + Assertions.assertEquals("bjcrxgibbdaxco", response.iterator().next().display().resource()); + Assertions.assertEquals("ozauorsukokwb", response.iterator().next().display().operation()); + Assertions.assertEquals("lhlv", response.iterator().next().display().description()); Assertions.assertEquals(false, response.iterator().next().isDataAction()); } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteClusterApiKeyWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteClusterApiKeyWithResponseMockTests.java new file mode 100644 index 0000000000000..80670f7f3b682 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteClusterApiKeyWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsDeleteClusterApiKeyWithResponseMockTests { + @Test + public void testDeleteClusterApiKeyWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.organizations().deleteClusterApiKeyWithResponse("tilaxh", "fhqlyvi", "ouwivkxoyzunbixx", + com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteMockTests.java index 4f09c3fe0645c..6b39e9a402542 100644 --- a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.organizations().delete("ggkfpagaowpul", "qblylsyxkqj", com.azure.core.util.Context.NONE); + manager.organizations().delete("sspuunnoxyhkx", "qddrihpfhoqcaae", com.azure.core.util.Context.NONE); } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetClusterByIdWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetClusterByIdWithResponseMockTests.java new file mode 100644 index 0000000000000..f1648610186f8 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetClusterByIdWithResponseMockTests.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.SCClusterRecord; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsGetClusterByIdWithResponseMockTests { + @Test + public void testGetClusterByIdWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"kind\":\"bxv\",\"id\":\"htnsi\",\"properties\":{\"metadata\":{\"self\":\"z\",\"resourceName\":\"es\",\"createdTimestamp\":\"dlpagzrcxfail\",\"updatedTimestamp\":\"xwmdboxd\",\"deletedTimestamp\":\"sftufqobrjlna\"},\"spec\":{\"name\":\"ckknhxkizvy\",\"availability\":\"rzvul\",\"cloud\":\"aaeranokqgukk\",\"zone\":\"nvbroylaxx\",\"region\":\"cdisd\",\"kafkaBootstrapEndpoint\":\"fj\",\"httpEndpoint\":\"svgjrwhryvy\",\"apiEndpoint\":\"t\",\"config\":{\"kind\":\"xgccknfnw\"},\"environment\":{\"id\":\"mvpdvjdhttzaef\",\"environment\":\"x\",\"related\":\"ch\",\"resourceName\":\"hk\"},\"network\":{\"id\":\"jdqnsdfzpbgt\",\"environment\":\"ylkdghrje\",\"related\":\"tl\",\"resourceName\":\"ez\"},\"byok\":{\"id\":\"okvbwnhhtqlgehg\",\"related\":\"ipifhpfeoajvg\",\"resourceName\":\"txjcsheafidlt\"}},\"status\":{\"phase\":\"resmkssjhoiftxfk\",\"cku\":182361785}},\"name\":\"prhptillu\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + SCClusterRecord response = manager.organizations().getClusterByIdWithResponse("zeexavoxtfgle", "dmdqb", + "pypqtgsfj", "cbslhhx", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("bxv", response.kind()); + Assertions.assertEquals("htnsi", response.id()); + Assertions.assertEquals("prhptillu", response.name()); + Assertions.assertEquals("z", response.metadata().self()); + Assertions.assertEquals("es", response.metadata().resourceName()); + Assertions.assertEquals("dlpagzrcxfail", response.metadata().createdTimestamp()); + Assertions.assertEquals("xwmdboxd", response.metadata().updatedTimestamp()); + Assertions.assertEquals("sftufqobrjlna", response.metadata().deletedTimestamp()); + Assertions.assertEquals("ckknhxkizvy", response.spec().name()); + Assertions.assertEquals("rzvul", response.spec().availability()); + Assertions.assertEquals("aaeranokqgukk", response.spec().cloud()); + Assertions.assertEquals("nvbroylaxx", response.spec().zone()); + Assertions.assertEquals("cdisd", response.spec().region()); + Assertions.assertEquals("fj", response.spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("svgjrwhryvy", response.spec().httpEndpoint()); + Assertions.assertEquals("t", response.spec().apiEndpoint()); + Assertions.assertEquals("xgccknfnw", response.spec().config().kind()); + Assertions.assertEquals("mvpdvjdhttzaef", response.spec().environment().id()); + Assertions.assertEquals("x", response.spec().environment().environment()); + Assertions.assertEquals("ch", response.spec().environment().related()); + Assertions.assertEquals("hk", response.spec().environment().resourceName()); + Assertions.assertEquals("jdqnsdfzpbgt", response.spec().network().id()); + Assertions.assertEquals("ylkdghrje", response.spec().network().environment()); + Assertions.assertEquals("tl", response.spec().network().related()); + Assertions.assertEquals("ez", response.spec().network().resourceName()); + Assertions.assertEquals("okvbwnhhtqlgehg", response.spec().byok().id()); + Assertions.assertEquals("ipifhpfeoajvg", response.spec().byok().related()); + Assertions.assertEquals("txjcsheafidlt", response.spec().byok().resourceName()); + Assertions.assertEquals("resmkssjhoiftxfk", response.status().phase()); + Assertions.assertEquals(182361785, response.status().cku()); + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetEnvironmentByIdWithResponseMockTests.java similarity index 55% rename from sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateWithResponseMockTests.java rename to sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetEnvironmentByIdWithResponseMockTests.java index febe4e7e5b004..8ad9dde789a6f 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetEnvironmentByIdWithResponseMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.generated; +package com.azure.resourcemanager.confluent.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -11,8 +11,8 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.communication.CommunicationManager; -import com.azure.resourcemanager.communication.models.SuppressionListAddressResource; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.SCEnvironmentRecord; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -23,15 +23,15 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SuppressionListAddressesCreateOrUpdateWithResponseMockTests { +public final class OrganizationsGetEnvironmentByIdWithResponseMockTests { @Test - public void testCreateOrUpdateWithResponse() throws Exception { + public void testGetEnvironmentByIdWithResponse() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"email\":\"skdsnfdsdoakg\",\"firstName\":\"lmkk\",\"lastName\":\"vdlhewpusdsttwv\",\"notes\":\"vbbejdcng\",\"lastModified\":\"2021-02-12T16:56:22Z\",\"dataLocation\":\"akufgmjz\"},\"id\":\"rdgrtw\",\"name\":\"enuuzkopbm\",\"type\":\"nrfdw\"}"; + = "{\"kind\":\"lds\",\"id\":\"stjbkkdmfl\",\"properties\":{\"metadata\":{\"self\":\"mjlxrrilozapeewc\",\"resourceName\":\"xlktwkuzi\",\"createdTimestamp\":\"slevufuztc\",\"updatedTimestamp\":\"yhjtqedcgzu\",\"deletedTimestamp\":\"mmrqz\"}},\"name\":\"rjvpglydzgkrvqee\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -44,17 +44,21 @@ public void testCreateOrUpdateWithResponse() throws Exception { return Mono.just(httpResponse); })); - CommunicationManager manager = CommunicationManager.configure().withHttpClient(httpClient).authenticate( + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - SuppressionListAddressResource response = manager.suppressionListAddresses().define("hkh") - .withExistingSuppressionList("ojvdcpzfoqo", "i", "ybxarzgszu", "oxciqopidoamcio").withEmail("xkhnzbonlwnto") - .withFirstName("okdwb").withLastName("kszzcmrvexztv").withNotes("qgsfraoyzkoow").create(); + SCEnvironmentRecord response = manager.organizations() + .getEnvironmentByIdWithResponse("wzjbh", "zsxjrkambtrneg", "mnvuqe", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("skdsnfdsdoakg", response.email()); - Assertions.assertEquals("lmkk", response.firstName()); - Assertions.assertEquals("vdlhewpusdsttwv", response.lastName()); - Assertions.assertEquals("vbbejdcng", response.notes()); + Assertions.assertEquals("lds", response.kind()); + Assertions.assertEquals("stjbkkdmfl", response.id()); + Assertions.assertEquals("rjvpglydzgkrvqee", response.name()); + Assertions.assertEquals("mjlxrrilozapeewc", response.metadata().self()); + Assertions.assertEquals("xlktwkuzi", response.metadata().resourceName()); + Assertions.assertEquals("slevufuztc", response.metadata().createdTimestamp()); + Assertions.assertEquals("yhjtqedcgzu", response.metadata().updatedTimestamp()); + Assertions.assertEquals("mmrqz", response.metadata().deletedTimestamp()); } } diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetSchemaRegistryClusterByIdWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetSchemaRegistryClusterByIdWithResponseMockTests.java new file mode 100644 index 0000000000000..135dbe7da441c --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsGetSchemaRegistryClusterByIdWithResponseMockTests.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterRecord; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsGetSchemaRegistryClusterByIdWithResponseMockTests { + @Test + public void testGetSchemaRegistryClusterByIdWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"kind\":\"vjskgfmoc\",\"id\":\"hpqgatjeaahhvj\",\"properties\":{\"metadata\":{\"self\":\"kzyb\",\"resourceName\":\"jid\",\"createdTimestamp\":\"syxkyxvxevblb\",\"updatedTimestamp\":\"dnlj\",\"deletedTimestamp\":\"geuaulx\"},\"spec\":{\"name\":\"mjbnk\",\"httpEndpoint\":\"xynenl\",\"package\":\"xeizzg\",\"region\":{\"id\":\"nsrmffeycx\",\"related\":\"tpiymerteea\",\"resourceName\":\"xqiekkkzddrtk\"},\"environment\":{\"id\":\"jbmxvavre\",\"related\":\"eesvecu\",\"resourceName\":\"pxtxsuwp\"},\"cloud\":\"ujwsawddjibabxvi\"},\"status\":{\"phase\":\"v\"}}}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + SchemaRegistryClusterRecord response + = manager.organizations().getSchemaRegistryClusterByIdWithResponse("ajuwas", "vdaeyyguxakjsq", "hzbezkgi", + "sidxasicdd", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("vjskgfmoc", response.kind()); + Assertions.assertEquals("hpqgatjeaahhvj", response.id()); + Assertions.assertEquals("kzyb", response.metadata().self()); + Assertions.assertEquals("jid", response.metadata().resourceName()); + Assertions.assertEquals("syxkyxvxevblb", response.metadata().createdTimestamp()); + Assertions.assertEquals("dnlj", response.metadata().updatedTimestamp()); + Assertions.assertEquals("geuaulx", response.metadata().deletedTimestamp()); + Assertions.assertEquals("mjbnk", response.spec().name()); + Assertions.assertEquals("xynenl", response.spec().httpEndpoint()); + Assertions.assertEquals("xeizzg", response.spec().packageProperty()); + Assertions.assertEquals("nsrmffeycx", response.spec().region().id()); + Assertions.assertEquals("tpiymerteea", response.spec().region().related()); + Assertions.assertEquals("xqiekkkzddrtk", response.spec().region().resourceName()); + Assertions.assertEquals("jbmxvavre", response.spec().environment().id()); + Assertions.assertEquals("eesvecu", response.spec().environment().related()); + Assertions.assertEquals("pxtxsuwp", response.spec().environment().resourceName()); + Assertions.assertEquals("ujwsawddjibabxvi", response.spec().cloud()); + Assertions.assertEquals("v", response.status().phase()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListClustersMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListClustersMockTests.java new file mode 100644 index 0000000000000..50b7ad3944b1c --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListClustersMockTests.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.SCClusterRecord; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsListClustersMockTests { + @Test + public void testListClusters() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"kind\":\"dflgzuri\",\"id\":\"aecxndtic\",\"properties\":{\"metadata\":{\"self\":\"zmlqtmldgxo\",\"resourceName\":\"irclnpk\",\"createdTimestamp\":\"ayzri\",\"updatedTimestamp\":\"hya\",\"deletedTimestamp\":\"vjlboxqvk\"},\"spec\":{\"name\":\"xhom\",\"availability\":\"nhdwdigumbnra\",\"cloud\":\"zzp\",\"zone\":\"a\",\"region\":\"sdzhezww\",\"kafkaBootstrapEndpoint\":\"iqyuvvfo\",\"httpEndpoint\":\"p\",\"apiEndpoint\":\"qyikvy\",\"config\":{\"kind\":\"yavluwmncstt\"},\"environment\":{\"id\":\"y\",\"environment\":\"poekrsgsgb\",\"related\":\"uzqgnjdgkynsc\",\"resourceName\":\"qhzvhxnkomt\"},\"network\":{\"id\":\"otppnv\",\"environment\":\"zxhi\",\"related\":\"rbbcevq\",\"resourceName\":\"tltdhlfkqojpy\"},\"byok\":{\"id\":\"trdcnifmzzs\",\"related\":\"m\",\"resourceName\":\"nysuxmprafwgckh\"}},\"status\":{\"phase\":\"vdff\",\"cku\":848051352}},\"name\":\"qrouda\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.organizations().listClusters("toepryu", "nwy", "pzdm", + 1383258036, "zvfvaawz", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("dflgzuri", response.iterator().next().kind()); + Assertions.assertEquals("aecxndtic", response.iterator().next().id()); + Assertions.assertEquals("qrouda", response.iterator().next().name()); + Assertions.assertEquals("zmlqtmldgxo", response.iterator().next().metadata().self()); + Assertions.assertEquals("irclnpk", response.iterator().next().metadata().resourceName()); + Assertions.assertEquals("ayzri", response.iterator().next().metadata().createdTimestamp()); + Assertions.assertEquals("hya", response.iterator().next().metadata().updatedTimestamp()); + Assertions.assertEquals("vjlboxqvk", response.iterator().next().metadata().deletedTimestamp()); + Assertions.assertEquals("xhom", response.iterator().next().spec().name()); + Assertions.assertEquals("nhdwdigumbnra", response.iterator().next().spec().availability()); + Assertions.assertEquals("zzp", response.iterator().next().spec().cloud()); + Assertions.assertEquals("a", response.iterator().next().spec().zone()); + Assertions.assertEquals("sdzhezww", response.iterator().next().spec().region()); + Assertions.assertEquals("iqyuvvfo", response.iterator().next().spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("p", response.iterator().next().spec().httpEndpoint()); + Assertions.assertEquals("qyikvy", response.iterator().next().spec().apiEndpoint()); + Assertions.assertEquals("yavluwmncstt", response.iterator().next().spec().config().kind()); + Assertions.assertEquals("y", response.iterator().next().spec().environment().id()); + Assertions.assertEquals("poekrsgsgb", response.iterator().next().spec().environment().environment()); + Assertions.assertEquals("uzqgnjdgkynsc", response.iterator().next().spec().environment().related()); + Assertions.assertEquals("qhzvhxnkomt", response.iterator().next().spec().environment().resourceName()); + Assertions.assertEquals("otppnv", response.iterator().next().spec().network().id()); + Assertions.assertEquals("zxhi", response.iterator().next().spec().network().environment()); + Assertions.assertEquals("rbbcevq", response.iterator().next().spec().network().related()); + Assertions.assertEquals("tltdhlfkqojpy", response.iterator().next().spec().network().resourceName()); + Assertions.assertEquals("trdcnifmzzs", response.iterator().next().spec().byok().id()); + Assertions.assertEquals("m", response.iterator().next().spec().byok().related()); + Assertions.assertEquals("nysuxmprafwgckh", response.iterator().next().spec().byok().resourceName()); + Assertions.assertEquals("vdff", response.iterator().next().status().phase()); + Assertions.assertEquals(848051352, response.iterator().next().status().cku()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListEnvironmentsMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListEnvironmentsMockTests.java new file mode 100644 index 0000000000000..4860f0e0abafd --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListEnvironmentsMockTests.java @@ -0,0 +1,64 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.SCEnvironmentRecord; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsListEnvironmentsMockTests { + @Test + public void testListEnvironments() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"kind\":\"kdncj\",\"id\":\"onbzoggculapzwy\",\"properties\":{\"metadata\":{\"self\":\"gtqxep\",\"resourceName\":\"lbfu\",\"createdTimestamp\":\"lyjt\",\"updatedTimestamp\":\"of\",\"deletedTimestamp\":\"hvfcibyfmow\"}},\"name\":\"rkjpvdwxfzwii\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.organizations().listEnvironments("dao", "djvlpj", + 945284283, "kzbrmsgeivsiy", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("kdncj", response.iterator().next().kind()); + Assertions.assertEquals("onbzoggculapzwy", response.iterator().next().id()); + Assertions.assertEquals("rkjpvdwxfzwii", response.iterator().next().name()); + Assertions.assertEquals("gtqxep", response.iterator().next().metadata().self()); + Assertions.assertEquals("lbfu", response.iterator().next().metadata().resourceName()); + Assertions.assertEquals("lyjt", response.iterator().next().metadata().createdTimestamp()); + Assertions.assertEquals("of", response.iterator().next().metadata().updatedTimestamp()); + Assertions.assertEquals("hvfcibyfmow", response.iterator().next().metadata().deletedTimestamp()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListRegionsWithResponseMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListRegionsWithResponseMockTests.java new file mode 100644 index 0000000000000..bc6c140a4d4ff --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListRegionsWithResponseMockTests.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.ListAccessRequestModel; +import com.azure.resourcemanager.confluent.models.ListRegionsSuccessResponse; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsListRegionsWithResponseMockTests { + @Test + public void testListRegionsWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"data\":[{\"kind\":\"vekqvgqo\",\"id\":\"ifzmpjwyivqi\",\"properties\":{\"metadata\":{\"self\":\"vhrfsphuagrt\",\"resourceName\":\"kteusqczk\",\"createdTimestamp\":\"klxubyja\",\"updatedTimestamp\":\"mmfblcqcuubgqib\",\"deletedTimestamp\":\"a\"},\"spec\":{\"name\":\"tttwgdslqxih\",\"cloud\":\"moo\",\"regionName\":\"qseypxiutcxa\",\"packages\":[\"y\",\"petogebjox\",\"lhvnhlab\",\"q\"]}}},{\"kind\":\"kzjcjbtrgae\",\"id\":\"vibr\",\"properties\":{\"metadata\":{\"self\":\"toqbeitpkxztmoob\",\"resourceName\":\"ft\",\"createdTimestamp\":\"gfcwqmpimaqxzhem\",\"updatedTimestamp\":\"h\",\"deletedTimestamp\":\"uj\"},\"spec\":{\"name\":\"wkozz\",\"cloud\":\"ulkb\",\"regionName\":\"pfajnjwltlwtjj\",\"packages\":[\"talhsnvkcdmxzr\",\"oaimlnw\",\"aaomylweazu\",\"cse\"]}}},{\"kind\":\"wwnpj\",\"id\":\"fz\",\"properties\":{\"metadata\":{\"self\":\"hwahfbousn\",\"resourceName\":\"pgfewetwlyx\",\"createdTimestamp\":\"cxy\",\"updatedTimestamp\":\"hdjhlimmbcx\",\"deletedTimestamp\":\"bcporxvxcjzhqizx\"},\"spec\":{\"name\":\"tgqscjavftjuh\",\"cloud\":\"azkmtgguwp\",\"regionName\":\"r\",\"packages\":[\"ivmmghfcfiwrxgk\",\"euvyinzqodfvpgs\"]}}}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + ListRegionsSuccessResponse response = manager.organizations() + .listRegionsWithResponse("cuplcplcwkhih", "hlhzdsqtzbsrgno", + new ListAccessRequestModel().withSearchFilters(mapOf("oteyowc", "hfgmvecactxm")), + com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals("vekqvgqo", response.data().get(0).kind()); + Assertions.assertEquals("ifzmpjwyivqi", response.data().get(0).id()); + Assertions.assertEquals("vhrfsphuagrt", response.data().get(0).metadata().self()); + Assertions.assertEquals("kteusqczk", response.data().get(0).metadata().resourceName()); + Assertions.assertEquals("klxubyja", response.data().get(0).metadata().createdTimestamp()); + Assertions.assertEquals("mmfblcqcuubgqib", response.data().get(0).metadata().updatedTimestamp()); + Assertions.assertEquals("a", response.data().get(0).metadata().deletedTimestamp()); + Assertions.assertEquals("tttwgdslqxih", response.data().get(0).spec().name()); + Assertions.assertEquals("moo", response.data().get(0).spec().cloud()); + Assertions.assertEquals("qseypxiutcxa", response.data().get(0).spec().regionName()); + Assertions.assertEquals("y", response.data().get(0).spec().packages().get(0)); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListSchemaRegistryClustersMockTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListSchemaRegistryClustersMockTests.java new file mode 100644 index 0000000000000..8fd2bba70197b --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/OrganizationsListSchemaRegistryClustersMockTests.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.confluent.ConfluentManager; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterRecord; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OrganizationsListSchemaRegistryClustersMockTests { + @Test + public void testListSchemaRegistryClusters() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"kind\":\"fdn\",\"id\":\"ydvfvfcjnae\",\"properties\":{\"metadata\":{\"self\":\"vhmgorffukis\",\"resourceName\":\"w\",\"createdTimestamp\":\"hwplefaxvx\",\"updatedTimestamp\":\"cbtgnhnz\",\"deletedTimestamp\":\"qxtjjfzqlqhyca\"},\"spec\":{\"name\":\"ggxdb\",\"httpEndpoint\":\"smieknlra\",\"package\":\"aawiuagydwqfb\",\"region\":{\"id\":\"rfgi\",\"related\":\"tcojocqwo\",\"resourceName\":\"nzjvusfzldm\"},\"environment\":{\"id\":\"xylfsb\",\"related\":\"adpysownbt\",\"resourceName\":\"bugrj\"},\"cloud\":\"to\"},\"status\":{\"phase\":\"isofieypefojyqd\"}}}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + ConfluentManager manager = ConfluentManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.organizations().listSchemaRegistryClusters( + "pavehhr", "kbunzoz", "dhcxgkmoy", 1670062911, "dyuib", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("fdn", response.iterator().next().kind()); + Assertions.assertEquals("ydvfvfcjnae", response.iterator().next().id()); + Assertions.assertEquals("vhmgorffukis", response.iterator().next().metadata().self()); + Assertions.assertEquals("w", response.iterator().next().metadata().resourceName()); + Assertions.assertEquals("hwplefaxvx", response.iterator().next().metadata().createdTimestamp()); + Assertions.assertEquals("cbtgnhnz", response.iterator().next().metadata().updatedTimestamp()); + Assertions.assertEquals("qxtjjfzqlqhyca", response.iterator().next().metadata().deletedTimestamp()); + Assertions.assertEquals("ggxdb", response.iterator().next().spec().name()); + Assertions.assertEquals("smieknlra", response.iterator().next().spec().httpEndpoint()); + Assertions.assertEquals("aawiuagydwqfb", response.iterator().next().spec().packageProperty()); + Assertions.assertEquals("rfgi", response.iterator().next().spec().region().id()); + Assertions.assertEquals("tcojocqwo", response.iterator().next().spec().region().related()); + Assertions.assertEquals("nzjvusfzldm", response.iterator().next().spec().region().resourceName()); + Assertions.assertEquals("xylfsb", response.iterator().next().spec().environment().id()); + Assertions.assertEquals("adpysownbt", response.iterator().next().spec().environment().related()); + Assertions.assertEquals("bugrj", response.iterator().next().spec().environment().resourceName()); + Assertions.assertEquals("to", response.iterator().next().spec().cloud()); + Assertions.assertEquals("isofieypefojyqd", response.iterator().next().status().phase()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionPropertiesTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionPropertiesTests.java new file mode 100644 index 0000000000000..6d02f33d7c60b --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionPropertiesTests.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.RegionProperties; +import com.azure.resourcemanager.confluent.models.RegionSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class RegionPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RegionProperties model = BinaryData.fromString( + "{\"metadata\":{\"self\":\"uyovw\",\"resourceName\":\"bkfezzxscyhwzdgi\",\"createdTimestamp\":\"jbzbomvzzbtdcq\",\"updatedTimestamp\":\"niyujv\",\"deletedTimestamp\":\"l\"},\"spec\":{\"name\":\"hfssnrb\",\"cloud\":\"efr\",\"regionName\":\"sgaojfmwncot\",\"packages\":[\"hirctymoxoftpipi\"]}}") + .toObject(RegionProperties.class); + Assertions.assertEquals("uyovw", model.metadata().self()); + Assertions.assertEquals("bkfezzxscyhwzdgi", model.metadata().resourceName()); + Assertions.assertEquals("jbzbomvzzbtdcq", model.metadata().createdTimestamp()); + Assertions.assertEquals("niyujv", model.metadata().updatedTimestamp()); + Assertions.assertEquals("l", model.metadata().deletedTimestamp()); + Assertions.assertEquals("hfssnrb", model.spec().name()); + Assertions.assertEquals("efr", model.spec().cloud()); + Assertions.assertEquals("sgaojfmwncot", model.spec().regionName()); + Assertions.assertEquals("hirctymoxoftpipi", model.spec().packages().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RegionProperties model = new RegionProperties() + .withMetadata(new SCMetadataEntity().withSelf("uyovw").withResourceName("bkfezzxscyhwzdgi") + .withCreatedTimestamp("jbzbomvzzbtdcq").withUpdatedTimestamp("niyujv").withDeletedTimestamp("l")) + .withSpec(new RegionSpecEntity().withName("hfssnrb").withCloud("efr").withRegionName("sgaojfmwncot") + .withPackages(Arrays.asList("hirctymoxoftpipi"))); + model = BinaryData.fromObject(model).toObject(RegionProperties.class); + Assertions.assertEquals("uyovw", model.metadata().self()); + Assertions.assertEquals("bkfezzxscyhwzdgi", model.metadata().resourceName()); + Assertions.assertEquals("jbzbomvzzbtdcq", model.metadata().createdTimestamp()); + Assertions.assertEquals("niyujv", model.metadata().updatedTimestamp()); + Assertions.assertEquals("l", model.metadata().deletedTimestamp()); + Assertions.assertEquals("hfssnrb", model.spec().name()); + Assertions.assertEquals("efr", model.spec().cloud()); + Assertions.assertEquals("sgaojfmwncot", model.spec().regionName()); + Assertions.assertEquals("hirctymoxoftpipi", model.spec().packages().get(0)); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionRecordTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionRecordTests.java new file mode 100644 index 0000000000000..4e9f7c640a560 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionRecordTests.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.RegionRecord; +import com.azure.resourcemanager.confluent.models.RegionSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class RegionRecordTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RegionRecord model = BinaryData.fromString( + "{\"kind\":\"qxvcxgfrpdsofb\",\"id\":\"rnsvbuswd\",\"properties\":{\"metadata\":{\"self\":\"bycnunvjsrtkf\",\"resourceName\":\"nopqgikyzirtx\",\"createdTimestamp\":\"uxzejntpsew\",\"updatedTimestamp\":\"oi\",\"deletedTimestamp\":\"ukry\"},\"spec\":{\"name\":\"qmi\",\"cloud\":\"xorgg\",\"regionName\":\"hyaomtbghhavgr\",\"packages\":[\"fo\",\"jzhpjbibgjmfx\",\"mv\"]}}}") + .toObject(RegionRecord.class); + Assertions.assertEquals("qxvcxgfrpdsofb", model.kind()); + Assertions.assertEquals("rnsvbuswd", model.id()); + Assertions.assertEquals("bycnunvjsrtkf", model.metadata().self()); + Assertions.assertEquals("nopqgikyzirtx", model.metadata().resourceName()); + Assertions.assertEquals("uxzejntpsew", model.metadata().createdTimestamp()); + Assertions.assertEquals("oi", model.metadata().updatedTimestamp()); + Assertions.assertEquals("ukry", model.metadata().deletedTimestamp()); + Assertions.assertEquals("qmi", model.spec().name()); + Assertions.assertEquals("xorgg", model.spec().cloud()); + Assertions.assertEquals("hyaomtbghhavgr", model.spec().regionName()); + Assertions.assertEquals("fo", model.spec().packages().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RegionRecord model = new RegionRecord().withKind("qxvcxgfrpdsofb").withId("rnsvbuswd") + .withMetadata(new SCMetadataEntity().withSelf("bycnunvjsrtkf").withResourceName("nopqgikyzirtx") + .withCreatedTimestamp("uxzejntpsew").withUpdatedTimestamp("oi").withDeletedTimestamp("ukry")) + .withSpec(new RegionSpecEntity().withName("qmi").withCloud("xorgg").withRegionName("hyaomtbghhavgr") + .withPackages(Arrays.asList("fo", "jzhpjbibgjmfx", "mv"))); + model = BinaryData.fromObject(model).toObject(RegionRecord.class); + Assertions.assertEquals("qxvcxgfrpdsofb", model.kind()); + Assertions.assertEquals("rnsvbuswd", model.id()); + Assertions.assertEquals("bycnunvjsrtkf", model.metadata().self()); + Assertions.assertEquals("nopqgikyzirtx", model.metadata().resourceName()); + Assertions.assertEquals("uxzejntpsew", model.metadata().createdTimestamp()); + Assertions.assertEquals("oi", model.metadata().updatedTimestamp()); + Assertions.assertEquals("ukry", model.metadata().deletedTimestamp()); + Assertions.assertEquals("qmi", model.spec().name()); + Assertions.assertEquals("xorgg", model.spec().cloud()); + Assertions.assertEquals("hyaomtbghhavgr", model.spec().regionName()); + Assertions.assertEquals("fo", model.spec().packages().get(0)); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionSpecEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionSpecEntityTests.java new file mode 100644 index 0000000000000..31dbe74df663e --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/RegionSpecEntityTests.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.RegionSpecEntity; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class RegionSpecEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RegionSpecEntity model = BinaryData.fromString( + "{\"name\":\"czuhxacpqjlihh\",\"cloud\":\"spskasdvlmfwdgz\",\"regionName\":\"lucvpam\",\"packages\":[\"euzvx\",\"risjnhnytxifqjz\",\"xmrhu\"]}") + .toObject(RegionSpecEntity.class); + Assertions.assertEquals("czuhxacpqjlihh", model.name()); + Assertions.assertEquals("spskasdvlmfwdgz", model.cloud()); + Assertions.assertEquals("lucvpam", model.regionName()); + Assertions.assertEquals("euzvx", model.packages().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RegionSpecEntity model = new RegionSpecEntity().withName("czuhxacpqjlihh").withCloud("spskasdvlmfwdgz") + .withRegionName("lucvpam").withPackages(Arrays.asList("euzvx", "risjnhnytxifqjz", "xmrhu")); + model = BinaryData.fromObject(model).toObject(RegionSpecEntity.class); + Assertions.assertEquals("czuhxacpqjlihh", model.name()); + Assertions.assertEquals("spskasdvlmfwdgz", model.cloud()); + Assertions.assertEquals("lucvpam", model.regionName()); + Assertions.assertEquals("euzvx", model.packages().get(0)); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterByokEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterByokEntityTests.java new file mode 100644 index 0000000000000..399d3e5c80979 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterByokEntityTests.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.SCClusterByokEntity; +import org.junit.jupiter.api.Assertions; + +public final class SCClusterByokEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SCClusterByokEntity model + = BinaryData.fromString("{\"id\":\"hmtnvy\",\"related\":\"atkzwpcnpw\",\"resourceName\":\"jaesgvvsccya\"}") + .toObject(SCClusterByokEntity.class); + Assertions.assertEquals("hmtnvy", model.id()); + Assertions.assertEquals("atkzwpcnpw", model.related()); + Assertions.assertEquals("jaesgvvsccya", model.resourceName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SCClusterByokEntity model + = new SCClusterByokEntity().withId("hmtnvy").withRelated("atkzwpcnpw").withResourceName("jaesgvvsccya"); + model = BinaryData.fromObject(model).toObject(SCClusterByokEntity.class); + Assertions.assertEquals("hmtnvy", model.id()); + Assertions.assertEquals("atkzwpcnpw", model.related()); + Assertions.assertEquals("jaesgvvsccya", model.resourceName()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterNetworkEnvironmentEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterNetworkEnvironmentEntityTests.java new file mode 100644 index 0000000000000..65b2aaabe7e40 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterNetworkEnvironmentEntityTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.SCClusterNetworkEnvironmentEntity; +import org.junit.jupiter.api.Assertions; + +public final class SCClusterNetworkEnvironmentEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SCClusterNetworkEnvironmentEntity model = BinaryData.fromString( + "{\"id\":\"zdmsyqtfi\",\"environment\":\"hbotzingamvppho\",\"related\":\"qzudphq\",\"resourceName\":\"vdkfwynwcvtbvk\"}") + .toObject(SCClusterNetworkEnvironmentEntity.class); + Assertions.assertEquals("zdmsyqtfi", model.id()); + Assertions.assertEquals("hbotzingamvppho", model.environment()); + Assertions.assertEquals("qzudphq", model.related()); + Assertions.assertEquals("vdkfwynwcvtbvk", model.resourceName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SCClusterNetworkEnvironmentEntity model = new SCClusterNetworkEnvironmentEntity().withId("zdmsyqtfi") + .withEnvironment("hbotzingamvppho").withRelated("qzudphq").withResourceName("vdkfwynwcvtbvk"); + model = BinaryData.fromObject(model).toObject(SCClusterNetworkEnvironmentEntity.class); + Assertions.assertEquals("zdmsyqtfi", model.id()); + Assertions.assertEquals("hbotzingamvppho", model.environment()); + Assertions.assertEquals("qzudphq", model.related()); + Assertions.assertEquals("vdkfwynwcvtbvk", model.resourceName()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterRecordInnerTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterRecordInnerTests.java new file mode 100644 index 0000000000000..98e284703ca39 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterRecordInnerTests.java @@ -0,0 +1,105 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SCClusterRecordInner; +import com.azure.resourcemanager.confluent.models.ClusterConfigEntity; +import com.azure.resourcemanager.confluent.models.ClusterStatusEntity; +import com.azure.resourcemanager.confluent.models.SCClusterByokEntity; +import com.azure.resourcemanager.confluent.models.SCClusterNetworkEnvironmentEntity; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import org.junit.jupiter.api.Assertions; + +public final class SCClusterRecordInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SCClusterRecordInner model = BinaryData.fromString( + "{\"kind\":\"mnguxawqaldsyu\",\"id\":\"imerqfobwyznk\",\"properties\":{\"metadata\":{\"self\":\"t\",\"resourceName\":\"fhpagmhrskdsnf\",\"createdTimestamp\":\"doakgtdlmkkzevdl\",\"updatedTimestamp\":\"wpusdsttwvogv\",\"deletedTimestamp\":\"ejdcngqqmoakuf\"},\"spec\":{\"name\":\"zr\",\"availability\":\"dgrtwaenuuzkopbm\",\"cloud\":\"rfdwoyu\",\"zone\":\"ziuiefozbhdm\",\"region\":\"l\",\"kafkaBootstrapEndpoint\":\"qhoftrmaequiah\",\"httpEndpoint\":\"cslfaoqzpiyylha\",\"apiEndpoint\":\"swhccsphk\",\"config\":{\"kind\":\"witqscywuggwoluh\"},\"environment\":{\"id\":\"wem\",\"environment\":\"i\",\"related\":\"brgz\",\"resourceName\":\"msweypqwdxggicc\"},\"network\":{\"id\":\"qhuexm\",\"environment\":\"tlstvlzywem\",\"related\":\"rncsdtclu\",\"resourceName\":\"ypbsfgytguslfead\"},\"byok\":{\"id\":\"qukyhejhzi\",\"related\":\"gfpelolppvksrpqv\",\"resourceName\":\"zraehtwd\"}},\"status\":{\"phase\":\"tswiby\",\"cku\":403427600}},\"name\":\"bhshfwpracstwity\"}") + .toObject(SCClusterRecordInner.class); + Assertions.assertEquals("mnguxawqaldsyu", model.kind()); + Assertions.assertEquals("imerqfobwyznk", model.id()); + Assertions.assertEquals("bhshfwpracstwity", model.name()); + Assertions.assertEquals("t", model.metadata().self()); + Assertions.assertEquals("fhpagmhrskdsnf", model.metadata().resourceName()); + Assertions.assertEquals("doakgtdlmkkzevdl", model.metadata().createdTimestamp()); + Assertions.assertEquals("wpusdsttwvogv", model.metadata().updatedTimestamp()); + Assertions.assertEquals("ejdcngqqmoakuf", model.metadata().deletedTimestamp()); + Assertions.assertEquals("zr", model.spec().name()); + Assertions.assertEquals("dgrtwaenuuzkopbm", model.spec().availability()); + Assertions.assertEquals("rfdwoyu", model.spec().cloud()); + Assertions.assertEquals("ziuiefozbhdm", model.spec().zone()); + Assertions.assertEquals("l", model.spec().region()); + Assertions.assertEquals("qhoftrmaequiah", model.spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("cslfaoqzpiyylha", model.spec().httpEndpoint()); + Assertions.assertEquals("swhccsphk", model.spec().apiEndpoint()); + Assertions.assertEquals("witqscywuggwoluh", model.spec().config().kind()); + Assertions.assertEquals("wem", model.spec().environment().id()); + Assertions.assertEquals("i", model.spec().environment().environment()); + Assertions.assertEquals("brgz", model.spec().environment().related()); + Assertions.assertEquals("msweypqwdxggicc", model.spec().environment().resourceName()); + Assertions.assertEquals("qhuexm", model.spec().network().id()); + Assertions.assertEquals("tlstvlzywem", model.spec().network().environment()); + Assertions.assertEquals("rncsdtclu", model.spec().network().related()); + Assertions.assertEquals("ypbsfgytguslfead", model.spec().network().resourceName()); + Assertions.assertEquals("qukyhejhzi", model.spec().byok().id()); + Assertions.assertEquals("gfpelolppvksrpqv", model.spec().byok().related()); + Assertions.assertEquals("zraehtwd", model.spec().byok().resourceName()); + Assertions.assertEquals("tswiby", model.status().phase()); + Assertions.assertEquals(403427600, model.status().cku()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SCClusterRecordInner model + = new SCClusterRecordInner().withKind("mnguxawqaldsyu").withId("imerqfobwyznk").withName("bhshfwpracstwity") + .withMetadata(new SCMetadataEntity().withSelf("t").withResourceName("fhpagmhrskdsnf") + .withCreatedTimestamp("doakgtdlmkkzevdl").withUpdatedTimestamp("wpusdsttwvogv") + .withDeletedTimestamp("ejdcngqqmoakuf")) + .withSpec(new SCClusterSpecEntity().withName("zr").withAvailability("dgrtwaenuuzkopbm") + .withCloud("rfdwoyu").withZone("ziuiefozbhdm").withRegion("l") + .withKafkaBootstrapEndpoint("qhoftrmaequiah").withHttpEndpoint("cslfaoqzpiyylha") + .withApiEndpoint("swhccsphk").withConfig(new ClusterConfigEntity().withKind("witqscywuggwoluh")) + .withEnvironment(new SCClusterNetworkEnvironmentEntity().withId("wem").withEnvironment("i") + .withRelated("brgz").withResourceName("msweypqwdxggicc")) + .withNetwork(new SCClusterNetworkEnvironmentEntity().withId("qhuexm").withEnvironment("tlstvlzywem") + .withRelated("rncsdtclu").withResourceName("ypbsfgytguslfead")) + .withByok(new SCClusterByokEntity().withId("qukyhejhzi").withRelated("gfpelolppvksrpqv") + .withResourceName("zraehtwd"))) + .withStatus(new ClusterStatusEntity().withPhase("tswiby").withCku(403427600)); + model = BinaryData.fromObject(model).toObject(SCClusterRecordInner.class); + Assertions.assertEquals("mnguxawqaldsyu", model.kind()); + Assertions.assertEquals("imerqfobwyznk", model.id()); + Assertions.assertEquals("bhshfwpracstwity", model.name()); + Assertions.assertEquals("t", model.metadata().self()); + Assertions.assertEquals("fhpagmhrskdsnf", model.metadata().resourceName()); + Assertions.assertEquals("doakgtdlmkkzevdl", model.metadata().createdTimestamp()); + Assertions.assertEquals("wpusdsttwvogv", model.metadata().updatedTimestamp()); + Assertions.assertEquals("ejdcngqqmoakuf", model.metadata().deletedTimestamp()); + Assertions.assertEquals("zr", model.spec().name()); + Assertions.assertEquals("dgrtwaenuuzkopbm", model.spec().availability()); + Assertions.assertEquals("rfdwoyu", model.spec().cloud()); + Assertions.assertEquals("ziuiefozbhdm", model.spec().zone()); + Assertions.assertEquals("l", model.spec().region()); + Assertions.assertEquals("qhoftrmaequiah", model.spec().kafkaBootstrapEndpoint()); + Assertions.assertEquals("cslfaoqzpiyylha", model.spec().httpEndpoint()); + Assertions.assertEquals("swhccsphk", model.spec().apiEndpoint()); + Assertions.assertEquals("witqscywuggwoluh", model.spec().config().kind()); + Assertions.assertEquals("wem", model.spec().environment().id()); + Assertions.assertEquals("i", model.spec().environment().environment()); + Assertions.assertEquals("brgz", model.spec().environment().related()); + Assertions.assertEquals("msweypqwdxggicc", model.spec().environment().resourceName()); + Assertions.assertEquals("qhuexm", model.spec().network().id()); + Assertions.assertEquals("tlstvlzywem", model.spec().network().environment()); + Assertions.assertEquals("rncsdtclu", model.spec().network().related()); + Assertions.assertEquals("ypbsfgytguslfead", model.spec().network().resourceName()); + Assertions.assertEquals("qukyhejhzi", model.spec().byok().id()); + Assertions.assertEquals("gfpelolppvksrpqv", model.spec().byok().related()); + Assertions.assertEquals("zraehtwd", model.spec().byok().resourceName()); + Assertions.assertEquals("tswiby", model.status().phase()); + Assertions.assertEquals(403427600, model.status().cku()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterSpecEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterSpecEntityTests.java new file mode 100644 index 0000000000000..cf753b1c12409 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCClusterSpecEntityTests.java @@ -0,0 +1,76 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.ClusterConfigEntity; +import com.azure.resourcemanager.confluent.models.SCClusterByokEntity; +import com.azure.resourcemanager.confluent.models.SCClusterNetworkEnvironmentEntity; +import com.azure.resourcemanager.confluent.models.SCClusterSpecEntity; +import org.junit.jupiter.api.Assertions; + +public final class SCClusterSpecEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SCClusterSpecEntity model = BinaryData.fromString( + "{\"name\":\"ytisibir\",\"availability\":\"pikpz\",\"cloud\":\"ejzanlfz\",\"zone\":\"av\",\"region\":\"bzonok\",\"kafkaBootstrapEndpoint\":\"rjqc\",\"httpEndpoint\":\"gzpfrla\",\"apiEndpoint\":\"zrnw\",\"config\":{\"kind\":\"ndfpwpj\"},\"environment\":{\"id\":\"bt\",\"environment\":\"flsjc\",\"related\":\"szfjvfbgofelja\",\"resourceName\":\"qmqhldvriii\"},\"network\":{\"id\":\"al\",\"environment\":\"fk\",\"related\":\"vsexsowuelu\",\"resourceName\":\"hahhxvrhmzkwpj\"},\"byok\":{\"id\":\"spughftqsxhq\",\"related\":\"j\",\"resourceName\":\"kndxdigrjgu\"}}") + .toObject(SCClusterSpecEntity.class); + Assertions.assertEquals("ytisibir", model.name()); + Assertions.assertEquals("pikpz", model.availability()); + Assertions.assertEquals("ejzanlfz", model.cloud()); + Assertions.assertEquals("av", model.zone()); + Assertions.assertEquals("bzonok", model.region()); + Assertions.assertEquals("rjqc", model.kafkaBootstrapEndpoint()); + Assertions.assertEquals("gzpfrla", model.httpEndpoint()); + Assertions.assertEquals("zrnw", model.apiEndpoint()); + Assertions.assertEquals("ndfpwpj", model.config().kind()); + Assertions.assertEquals("bt", model.environment().id()); + Assertions.assertEquals("flsjc", model.environment().environment()); + Assertions.assertEquals("szfjvfbgofelja", model.environment().related()); + Assertions.assertEquals("qmqhldvriii", model.environment().resourceName()); + Assertions.assertEquals("al", model.network().id()); + Assertions.assertEquals("fk", model.network().environment()); + Assertions.assertEquals("vsexsowuelu", model.network().related()); + Assertions.assertEquals("hahhxvrhmzkwpj", model.network().resourceName()); + Assertions.assertEquals("spughftqsxhq", model.byok().id()); + Assertions.assertEquals("j", model.byok().related()); + Assertions.assertEquals("kndxdigrjgu", model.byok().resourceName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SCClusterSpecEntity model = new SCClusterSpecEntity().withName("ytisibir").withAvailability("pikpz") + .withCloud("ejzanlfz").withZone("av").withRegion("bzonok").withKafkaBootstrapEndpoint("rjqc") + .withHttpEndpoint("gzpfrla").withApiEndpoint("zrnw") + .withConfig(new ClusterConfigEntity().withKind("ndfpwpj")) + .withEnvironment(new SCClusterNetworkEnvironmentEntity().withId("bt").withEnvironment("flsjc") + .withRelated("szfjvfbgofelja").withResourceName("qmqhldvriii")) + .withNetwork(new SCClusterNetworkEnvironmentEntity().withId("al").withEnvironment("fk") + .withRelated("vsexsowuelu").withResourceName("hahhxvrhmzkwpj")) + .withByok( + new SCClusterByokEntity().withId("spughftqsxhq").withRelated("j").withResourceName("kndxdigrjgu")); + model = BinaryData.fromObject(model).toObject(SCClusterSpecEntity.class); + Assertions.assertEquals("ytisibir", model.name()); + Assertions.assertEquals("pikpz", model.availability()); + Assertions.assertEquals("ejzanlfz", model.cloud()); + Assertions.assertEquals("av", model.zone()); + Assertions.assertEquals("bzonok", model.region()); + Assertions.assertEquals("rjqc", model.kafkaBootstrapEndpoint()); + Assertions.assertEquals("gzpfrla", model.httpEndpoint()); + Assertions.assertEquals("zrnw", model.apiEndpoint()); + Assertions.assertEquals("ndfpwpj", model.config().kind()); + Assertions.assertEquals("bt", model.environment().id()); + Assertions.assertEquals("flsjc", model.environment().environment()); + Assertions.assertEquals("szfjvfbgofelja", model.environment().related()); + Assertions.assertEquals("qmqhldvriii", model.environment().resourceName()); + Assertions.assertEquals("al", model.network().id()); + Assertions.assertEquals("fk", model.network().environment()); + Assertions.assertEquals("vsexsowuelu", model.network().related()); + Assertions.assertEquals("hahhxvrhmzkwpj", model.network().resourceName()); + Assertions.assertEquals("spughftqsxhq", model.byok().id()); + Assertions.assertEquals("j", model.byok().related()); + Assertions.assertEquals("kndxdigrjgu", model.byok().resourceName()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCEnvironmentRecordInnerTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCEnvironmentRecordInnerTests.java new file mode 100644 index 0000000000000..6e737539ebac1 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCEnvironmentRecordInnerTests.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SCEnvironmentRecordInner; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import org.junit.jupiter.api.Assertions; + +public final class SCEnvironmentRecordInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SCEnvironmentRecordInner model = BinaryData.fromString( + "{\"kind\":\"lyzrpzbchckqqzqi\",\"id\":\"iysui\",\"properties\":{\"metadata\":{\"self\":\"ked\",\"resourceName\":\"trwyhqmib\",\"createdTimestamp\":\"hwit\",\"updatedTimestamp\":\"ypyynpcdpumnzg\",\"deletedTimestamp\":\"z\"}},\"name\":\"abikns\"}") + .toObject(SCEnvironmentRecordInner.class); + Assertions.assertEquals("lyzrpzbchckqqzqi", model.kind()); + Assertions.assertEquals("iysui", model.id()); + Assertions.assertEquals("abikns", model.name()); + Assertions.assertEquals("ked", model.metadata().self()); + Assertions.assertEquals("trwyhqmib", model.metadata().resourceName()); + Assertions.assertEquals("hwit", model.metadata().createdTimestamp()); + Assertions.assertEquals("ypyynpcdpumnzg", model.metadata().updatedTimestamp()); + Assertions.assertEquals("z", model.metadata().deletedTimestamp()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SCEnvironmentRecordInner model = new SCEnvironmentRecordInner().withKind("lyzrpzbchckqqzqi").withId("iysui") + .withName("abikns").withMetadata(new SCMetadataEntity().withSelf("ked").withResourceName("trwyhqmib") + .withCreatedTimestamp("hwit").withUpdatedTimestamp("ypyynpcdpumnzg").withDeletedTimestamp("z")); + model = BinaryData.fromObject(model).toObject(SCEnvironmentRecordInner.class); + Assertions.assertEquals("lyzrpzbchckqqzqi", model.kind()); + Assertions.assertEquals("iysui", model.id()); + Assertions.assertEquals("abikns", model.name()); + Assertions.assertEquals("ked", model.metadata().self()); + Assertions.assertEquals("trwyhqmib", model.metadata().resourceName()); + Assertions.assertEquals("hwit", model.metadata().createdTimestamp()); + Assertions.assertEquals("ypyynpcdpumnzg", model.metadata().updatedTimestamp()); + Assertions.assertEquals("z", model.metadata().deletedTimestamp()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCMetadataEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCMetadataEntityTests.java new file mode 100644 index 0000000000000..85142cc2faddb --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SCMetadataEntityTests.java @@ -0,0 +1,36 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import org.junit.jupiter.api.Assertions; + +public final class SCMetadataEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SCMetadataEntity model = BinaryData.fromString( + "{\"self\":\"gxqquezik\",\"resourceName\":\"ggxkallatmelwuip\",\"createdTimestamp\":\"cjzkzivgvvcna\",\"updatedTimestamp\":\"hyrnxxmu\",\"deletedTimestamp\":\"dndrdvstkwqqtche\"}") + .toObject(SCMetadataEntity.class); + Assertions.assertEquals("gxqquezik", model.self()); + Assertions.assertEquals("ggxkallatmelwuip", model.resourceName()); + Assertions.assertEquals("cjzkzivgvvcna", model.createdTimestamp()); + Assertions.assertEquals("hyrnxxmu", model.updatedTimestamp()); + Assertions.assertEquals("dndrdvstkwqqtche", model.deletedTimestamp()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SCMetadataEntity model = new SCMetadataEntity().withSelf("gxqquezik").withResourceName("ggxkallatmelwuip") + .withCreatedTimestamp("cjzkzivgvvcna").withUpdatedTimestamp("hyrnxxmu") + .withDeletedTimestamp("dndrdvstkwqqtche"); + model = BinaryData.fromObject(model).toObject(SCMetadataEntity.class); + Assertions.assertEquals("gxqquezik", model.self()); + Assertions.assertEquals("ggxkallatmelwuip", model.resourceName()); + Assertions.assertEquals("cjzkzivgvvcna", model.createdTimestamp()); + Assertions.assertEquals("hyrnxxmu", model.updatedTimestamp()); + Assertions.assertEquals("dndrdvstkwqqtche", model.deletedTimestamp()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterEnvironmentRegionEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterEnvironmentRegionEntityTests.java new file mode 100644 index 0000000000000..901321e8b496d --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterEnvironmentRegionEntityTests.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterEnvironmentRegionEntity; +import org.junit.jupiter.api.Assertions; + +public final class SchemaRegistryClusterEnvironmentRegionEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SchemaRegistryClusterEnvironmentRegionEntity model + = BinaryData.fromString("{\"id\":\"xum\",\"related\":\"ton\",\"resourceName\":\"jl\"}") + .toObject(SchemaRegistryClusterEnvironmentRegionEntity.class); + Assertions.assertEquals("xum", model.id()); + Assertions.assertEquals("ton", model.related()); + Assertions.assertEquals("jl", model.resourceName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SchemaRegistryClusterEnvironmentRegionEntity model = new SchemaRegistryClusterEnvironmentRegionEntity() + .withId("xum").withRelated("ton").withResourceName("jl"); + model = BinaryData.fromObject(model).toObject(SchemaRegistryClusterEnvironmentRegionEntity.class); + Assertions.assertEquals("xum", model.id()); + Assertions.assertEquals("ton", model.related()); + Assertions.assertEquals("jl", model.resourceName()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterPropertiesTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterPropertiesTests.java new file mode 100644 index 0000000000000..015f26564f966 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterPropertiesTests.java @@ -0,0 +1,70 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterProperties; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterEnvironmentRegionEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; +import org.junit.jupiter.api.Assertions; + +public final class SchemaRegistryClusterPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SchemaRegistryClusterProperties model = BinaryData.fromString( + "{\"metadata\":{\"self\":\"olbq\",\"resourceName\":\"uzlm\",\"createdTimestamp\":\"elfk\",\"updatedTimestamp\":\"plcrpwjxeznoig\",\"deletedTimestamp\":\"njwmwkpnbsazejj\"},\"spec\":{\"name\":\"agfhsxtta\",\"httpEndpoint\":\"zxnfaaz\",\"package\":\"dtnkdmkq\",\"region\":{\"id\":\"wuenvr\",\"related\":\"yo\",\"resourceName\":\"ibreb\"},\"environment\":{\"id\":\"ysjkixqtnqttez\",\"related\":\"fffiak\",\"resourceName\":\"pqqmted\"},\"cloud\":\"mmji\"},\"status\":{\"phase\":\"ozphvwauyqncygu\"}}") + .toObject(SchemaRegistryClusterProperties.class); + Assertions.assertEquals("olbq", model.metadata().self()); + Assertions.assertEquals("uzlm", model.metadata().resourceName()); + Assertions.assertEquals("elfk", model.metadata().createdTimestamp()); + Assertions.assertEquals("plcrpwjxeznoig", model.metadata().updatedTimestamp()); + Assertions.assertEquals("njwmwkpnbsazejj", model.metadata().deletedTimestamp()); + Assertions.assertEquals("agfhsxtta", model.spec().name()); + Assertions.assertEquals("zxnfaaz", model.spec().httpEndpoint()); + Assertions.assertEquals("dtnkdmkq", model.spec().packageProperty()); + Assertions.assertEquals("wuenvr", model.spec().region().id()); + Assertions.assertEquals("yo", model.spec().region().related()); + Assertions.assertEquals("ibreb", model.spec().region().resourceName()); + Assertions.assertEquals("ysjkixqtnqttez", model.spec().environment().id()); + Assertions.assertEquals("fffiak", model.spec().environment().related()); + Assertions.assertEquals("pqqmted", model.spec().environment().resourceName()); + Assertions.assertEquals("mmji", model.spec().cloud()); + Assertions.assertEquals("ozphvwauyqncygu", model.status().phase()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SchemaRegistryClusterProperties model = new SchemaRegistryClusterProperties() + .withMetadata(new SCMetadataEntity().withSelf("olbq").withResourceName("uzlm").withCreatedTimestamp("elfk") + .withUpdatedTimestamp("plcrpwjxeznoig").withDeletedTimestamp("njwmwkpnbsazejj")) + .withSpec(new SchemaRegistryClusterSpecEntity().withName("agfhsxtta").withHttpEndpoint("zxnfaaz") + .withPackageProperty("dtnkdmkq") + .withRegion(new SchemaRegistryClusterEnvironmentRegionEntity().withId("wuenvr").withRelated("yo") + .withResourceName("ibreb")) + .withEnvironment(new SchemaRegistryClusterEnvironmentRegionEntity().withId("ysjkixqtnqttez") + .withRelated("fffiak").withResourceName("pqqmted")) + .withCloud("mmji")) + .withStatus(new SchemaRegistryClusterStatusEntity().withPhase("ozphvwauyqncygu")); + model = BinaryData.fromObject(model).toObject(SchemaRegistryClusterProperties.class); + Assertions.assertEquals("olbq", model.metadata().self()); + Assertions.assertEquals("uzlm", model.metadata().resourceName()); + Assertions.assertEquals("elfk", model.metadata().createdTimestamp()); + Assertions.assertEquals("plcrpwjxeznoig", model.metadata().updatedTimestamp()); + Assertions.assertEquals("njwmwkpnbsazejj", model.metadata().deletedTimestamp()); + Assertions.assertEquals("agfhsxtta", model.spec().name()); + Assertions.assertEquals("zxnfaaz", model.spec().httpEndpoint()); + Assertions.assertEquals("dtnkdmkq", model.spec().packageProperty()); + Assertions.assertEquals("wuenvr", model.spec().region().id()); + Assertions.assertEquals("yo", model.spec().region().related()); + Assertions.assertEquals("ibreb", model.spec().region().resourceName()); + Assertions.assertEquals("ysjkixqtnqttez", model.spec().environment().id()); + Assertions.assertEquals("fffiak", model.spec().environment().related()); + Assertions.assertEquals("pqqmted", model.spec().environment().resourceName()); + Assertions.assertEquals("mmji", model.spec().cloud()); + Assertions.assertEquals("ozphvwauyqncygu", model.status().phase()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterRecordInnerTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterRecordInnerTests.java new file mode 100644 index 0000000000000..7126afbd36d4c --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterRecordInnerTests.java @@ -0,0 +1,76 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.fluent.models.SchemaRegistryClusterRecordInner; +import com.azure.resourcemanager.confluent.models.SCMetadataEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterEnvironmentRegionEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; +import org.junit.jupiter.api.Assertions; + +public final class SchemaRegistryClusterRecordInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SchemaRegistryClusterRecordInner model = BinaryData.fromString( + "{\"kind\":\"kvtkkg\",\"id\":\"qwjygvja\",\"properties\":{\"metadata\":{\"self\":\"mhvkzuhbxvvyh\",\"resourceName\":\"opbyrqufegxu\",\"createdTimestamp\":\"zfbn\",\"updatedTimestamp\":\"mctlpdngitv\",\"deletedTimestamp\":\"mhrixkwmyijejve\"},\"spec\":{\"name\":\"bpnaixexccbdre\",\"httpEndpoint\":\"hcexdrrvqa\",\"package\":\"kghtpwijnh\",\"region\":{\"id\":\"vfycxzb\",\"related\":\"oowvrv\",\"resourceName\":\"gjqppy\"},\"environment\":{\"id\":\"ronzmyhgfip\",\"related\":\"xkmcwaekrrjre\",\"resourceName\":\"xt\"},\"cloud\":\"umh\"},\"status\":{\"phase\":\"ikkx\"}}}") + .toObject(SchemaRegistryClusterRecordInner.class); + Assertions.assertEquals("kvtkkg", model.kind()); + Assertions.assertEquals("qwjygvja", model.id()); + Assertions.assertEquals("mhvkzuhbxvvyh", model.metadata().self()); + Assertions.assertEquals("opbyrqufegxu", model.metadata().resourceName()); + Assertions.assertEquals("zfbn", model.metadata().createdTimestamp()); + Assertions.assertEquals("mctlpdngitv", model.metadata().updatedTimestamp()); + Assertions.assertEquals("mhrixkwmyijejve", model.metadata().deletedTimestamp()); + Assertions.assertEquals("bpnaixexccbdre", model.spec().name()); + Assertions.assertEquals("hcexdrrvqa", model.spec().httpEndpoint()); + Assertions.assertEquals("kghtpwijnh", model.spec().packageProperty()); + Assertions.assertEquals("vfycxzb", model.spec().region().id()); + Assertions.assertEquals("oowvrv", model.spec().region().related()); + Assertions.assertEquals("gjqppy", model.spec().region().resourceName()); + Assertions.assertEquals("ronzmyhgfip", model.spec().environment().id()); + Assertions.assertEquals("xkmcwaekrrjre", model.spec().environment().related()); + Assertions.assertEquals("xt", model.spec().environment().resourceName()); + Assertions.assertEquals("umh", model.spec().cloud()); + Assertions.assertEquals("ikkx", model.status().phase()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SchemaRegistryClusterRecordInner model + = new SchemaRegistryClusterRecordInner().withKind("kvtkkg").withId("qwjygvja") + .withMetadata(new SCMetadataEntity().withSelf("mhvkzuhbxvvyh").withResourceName("opbyrqufegxu") + .withCreatedTimestamp("zfbn").withUpdatedTimestamp("mctlpdngitv") + .withDeletedTimestamp("mhrixkwmyijejve")) + .withSpec(new SchemaRegistryClusterSpecEntity().withName("bpnaixexccbdre") + .withHttpEndpoint("hcexdrrvqa").withPackageProperty("kghtpwijnh") + .withRegion(new SchemaRegistryClusterEnvironmentRegionEntity().withId("vfycxzb") + .withRelated("oowvrv").withResourceName("gjqppy")) + .withEnvironment(new SchemaRegistryClusterEnvironmentRegionEntity().withId("ronzmyhgfip") + .withRelated("xkmcwaekrrjre").withResourceName("xt")) + .withCloud("umh")) + .withStatus(new SchemaRegistryClusterStatusEntity().withPhase("ikkx")); + model = BinaryData.fromObject(model).toObject(SchemaRegistryClusterRecordInner.class); + Assertions.assertEquals("kvtkkg", model.kind()); + Assertions.assertEquals("qwjygvja", model.id()); + Assertions.assertEquals("mhvkzuhbxvvyh", model.metadata().self()); + Assertions.assertEquals("opbyrqufegxu", model.metadata().resourceName()); + Assertions.assertEquals("zfbn", model.metadata().createdTimestamp()); + Assertions.assertEquals("mctlpdngitv", model.metadata().updatedTimestamp()); + Assertions.assertEquals("mhrixkwmyijejve", model.metadata().deletedTimestamp()); + Assertions.assertEquals("bpnaixexccbdre", model.spec().name()); + Assertions.assertEquals("hcexdrrvqa", model.spec().httpEndpoint()); + Assertions.assertEquals("kghtpwijnh", model.spec().packageProperty()); + Assertions.assertEquals("vfycxzb", model.spec().region().id()); + Assertions.assertEquals("oowvrv", model.spec().region().related()); + Assertions.assertEquals("gjqppy", model.spec().region().resourceName()); + Assertions.assertEquals("ronzmyhgfip", model.spec().environment().id()); + Assertions.assertEquals("xkmcwaekrrjre", model.spec().environment().related()); + Assertions.assertEquals("xt", model.spec().environment().resourceName()); + Assertions.assertEquals("umh", model.spec().cloud()); + Assertions.assertEquals("ikkx", model.status().phase()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterSpecEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterSpecEntityTests.java new file mode 100644 index 0000000000000..7f0a3c0464fb5 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterSpecEntityTests.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterEnvironmentRegionEntity; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterSpecEntity; +import org.junit.jupiter.api.Assertions; + +public final class SchemaRegistryClusterSpecEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SchemaRegistryClusterSpecEntity model = BinaryData.fromString( + "{\"name\":\"vipmdscwxqupevzh\",\"httpEndpoint\":\"totxhojujb\",\"package\":\"elmcuvhixbjxyfw\",\"region\":{\"id\":\"r\",\"related\":\"o\",\"resourceName\":\"ttpkiwkkbnujrywv\"},\"environment\":{\"id\":\"bfpncurdo\",\"related\":\"iithtywu\",\"resourceName\":\"cbihwqk\"},\"cloud\":\"dntwjchrdgo\"}") + .toObject(SchemaRegistryClusterSpecEntity.class); + Assertions.assertEquals("vipmdscwxqupevzh", model.name()); + Assertions.assertEquals("totxhojujb", model.httpEndpoint()); + Assertions.assertEquals("elmcuvhixbjxyfw", model.packageProperty()); + Assertions.assertEquals("r", model.region().id()); + Assertions.assertEquals("o", model.region().related()); + Assertions.assertEquals("ttpkiwkkbnujrywv", model.region().resourceName()); + Assertions.assertEquals("bfpncurdo", model.environment().id()); + Assertions.assertEquals("iithtywu", model.environment().related()); + Assertions.assertEquals("cbihwqk", model.environment().resourceName()); + Assertions.assertEquals("dntwjchrdgo", model.cloud()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SchemaRegistryClusterSpecEntity model = new SchemaRegistryClusterSpecEntity().withName("vipmdscwxqupevzh") + .withHttpEndpoint("totxhojujb").withPackageProperty("elmcuvhixbjxyfw") + .withRegion(new SchemaRegistryClusterEnvironmentRegionEntity().withId("r").withRelated("o") + .withResourceName("ttpkiwkkbnujrywv")) + .withEnvironment(new SchemaRegistryClusterEnvironmentRegionEntity().withId("bfpncurdo") + .withRelated("iithtywu").withResourceName("cbihwqk")) + .withCloud("dntwjchrdgo"); + model = BinaryData.fromObject(model).toObject(SchemaRegistryClusterSpecEntity.class); + Assertions.assertEquals("vipmdscwxqupevzh", model.name()); + Assertions.assertEquals("totxhojujb", model.httpEndpoint()); + Assertions.assertEquals("elmcuvhixbjxyfw", model.packageProperty()); + Assertions.assertEquals("r", model.region().id()); + Assertions.assertEquals("o", model.region().related()); + Assertions.assertEquals("ttpkiwkkbnujrywv", model.region().resourceName()); + Assertions.assertEquals("bfpncurdo", model.environment().id()); + Assertions.assertEquals("iithtywu", model.environment().related()); + Assertions.assertEquals("cbihwqk", model.environment().resourceName()); + Assertions.assertEquals("dntwjchrdgo", model.cloud()); + } +} diff --git a/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterStatusEntityTests.java b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterStatusEntityTests.java new file mode 100644 index 0000000000000..fc0ac24a9c060 --- /dev/null +++ b/sdk/confluent/azure-resourcemanager-confluent/src/test/java/com/azure/resourcemanager/confluent/generated/SchemaRegistryClusterStatusEntityTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.confluent.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.confluent.models.SchemaRegistryClusterStatusEntity; +import org.junit.jupiter.api.Assertions; + +public final class SchemaRegistryClusterStatusEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SchemaRegistryClusterStatusEntity model + = BinaryData.fromString("{\"phase\":\"dfdlwggyts\"}").toObject(SchemaRegistryClusterStatusEntity.class); + Assertions.assertEquals("dfdlwggyts", model.phase()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SchemaRegistryClusterStatusEntity model = new SchemaRegistryClusterStatusEntity().withPhase("dfdlwggyts"); + model = BinaryData.fromObject(model).toObject(SchemaRegistryClusterStatusEntity.class); + Assertions.assertEquals("dfdlwggyts", model.phase()); + } +} diff --git a/sdk/connectedvmware/azure-resourcemanager-connectedvmware/pom.xml b/sdk/connectedvmware/azure-resourcemanager-connectedvmware/pom.xml index 911a7c4b20c7c..d888cdf8ed756 100644 --- a/sdk/connectedvmware/azure-resourcemanager-connectedvmware/pom.xml +++ b/sdk/connectedvmware/azure-resourcemanager-connectedvmware/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/consumption/azure-resourcemanager-consumption/pom.xml b/sdk/consumption/azure-resourcemanager-consumption/pom.xml index 6b96eb3c61e8b..5d58eb2904afd 100644 --- a/sdk/consumption/azure-resourcemanager-consumption/pom.xml +++ b/sdk/consumption/azure-resourcemanager-consumption/pom.xml @@ -54,7 +54,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/containerregistry/azure-containers-containerregistry-perf/pom.xml b/sdk/containerregistry/azure-containers-containerregistry-perf/pom.xml index d9219076336b0..84b14a3299e06 100644 --- a/sdk/containerregistry/azure-containers-containerregistry-perf/pom.xml +++ b/sdk/containerregistry/azure-containers-containerregistry-perf/pom.xml @@ -53,12 +53,12 @@ com.azure.resourcemanager azure-resourcemanager-containerregistry - 2.36.0 + 2.37.0 com.azure azure-identity - 1.11.3 + 1.11.4 diff --git a/sdk/containerregistry/azure-containers-containerregistry/CHANGELOG.md b/sdk/containerregistry/azure-containers-containerregistry/CHANGELOG.md index d79423bca2f7e..10fc4bd2c7021 100644 --- a/sdk/containerregistry/azure-containers-containerregistry/CHANGELOG.md +++ b/sdk/containerregistry/azure-containers-containerregistry/CHANGELOG.md @@ -10,6 +10,27 @@ ### Other Changes + +## 1.2.6 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + +## 1.2.5 (2024-02-22) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.45.1` to version `1.46.0`. +- Upgraded `azure-core-http-netty` from `1.13.11` to version `1.14.0`. + + ## 1.2.4 (2023-12-04) ### Other Changes diff --git a/sdk/containerregistry/azure-containers-containerregistry/pom.xml b/sdk/containerregistry/azure-containers-containerregistry/pom.xml index aca02a20c1a33..3dd0e82ac27c9 100644 --- a/sdk/containerregistry/azure-containers-containerregistry/pom.xml +++ b/sdk/containerregistry/azure-containers-containerregistry/pom.xml @@ -87,19 +87,19 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -123,13 +123,13 @@ com.azure.resourcemanager azure-resourcemanager-containerregistry - 2.36.0 + 2.37.0 test com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/containerregistry/tests.yml b/sdk/containerregistry/tests.yml index e64c3b403760d..cd4fa61389ff0 100644 --- a/sdk/containerregistry/tests.yml +++ b/sdk/containerregistry/tests.yml @@ -1,18 +1,18 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: containerregistry - SupportedClouds: 'Public,UsGov,China' - Artifacts: - - name: azure-containers-containerregistry - groupId: com.azure - safeName: azurecontainerscontainerregistry - TimeoutInMinutes: 90 - EnvVars: - AZURE_LOG_LEVEL: 2 - AZURE_CLIENT_ID: $(CONTAINERREGISTRY_CLIENT_ID) - AZURE_CLIENT_SECRET: $(CONTAINERREGISTRY_CLIENT_SECRET) - AZURE_TENANT_ID: $(CONTAINERREGISTRY_TENANT_ID) - AZURE_SUBSCRIPTION_ID: $(CONTAINERREGISTRY_SUBSCRIPTION_ID) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: containerregistry + SupportedClouds: 'Public,UsGov,China' + Artifacts: + - name: azure-containers-containerregistry + groupId: com.azure + safeName: azurecontainerscontainerregistry + TimeoutInMinutes: 90 + EnvVars: + AZURE_LOG_LEVEL: 2 + AZURE_CLIENT_ID: $(CONTAINERREGISTRY_CLIENT_ID) + AZURE_CLIENT_SECRET: $(CONTAINERREGISTRY_CLIENT_SECRET) + AZURE_TENANT_ID: $(CONTAINERREGISTRY_TENANT_ID) + AZURE_SUBSCRIPTION_ID: $(CONTAINERREGISTRY_SUBSCRIPTION_ID) diff --git a/sdk/containerservicefleet/azure-resourcemanager-containerservicefleet/pom.xml b/sdk/containerservicefleet/azure-resourcemanager-containerservicefleet/pom.xml index 0962fed19e3bc..ea7e8fff76c1d 100644 --- a/sdk/containerservicefleet/azure-resourcemanager-containerservicefleet/pom.xml +++ b/sdk/containerservicefleet/azure-resourcemanager-containerservicefleet/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/containerservicefleet/tests.mgmt.yml b/sdk/containerservicefleet/tests.mgmt.yml index f02e8a5931b8d..3c9a27f6df7c1 100644 --- a/sdk/containerservicefleet/tests.mgmt.yml +++ b/sdk/containerservicefleet/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: containerservicefleet - Artifacts: - - name: azure-resourcemanager-containerservicefleet - groupId: com.azure.resourcemanager - safeName: azureresourcemanagercontainerservicefleet - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: containerservicefleet + Artifacts: + - name: azure-resourcemanager-containerservicefleet + groupId: com.azure.resourcemanager + safeName: azureresourcemanagercontainerservicefleet + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/contentsafety/azure-ai-contentsafety/CHANGELOG.md b/sdk/contentsafety/azure-ai-contentsafety/CHANGELOG.md index 021de57266347..68d2c07939475 100644 --- a/sdk/contentsafety/azure-ai-contentsafety/CHANGELOG.md +++ b/sdk/contentsafety/azure-ai-contentsafety/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.0.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.45.1` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.13.11` to version `1.14.1`. + + ## 1.0.0 (2023-12-13) ### Features Added diff --git a/sdk/contentsafety/azure-ai-contentsafety/pom.xml b/sdk/contentsafety/azure-ai-contentsafety/pom.xml index 43dfad8c82072..b9a29fc3c547f 100644 --- a/sdk/contentsafety/azure-ai-contentsafety/pom.xml +++ b/sdk/contentsafety/azure-ai-contentsafety/pom.xml @@ -78,7 +78,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/core/azure-core-amqp-experimental/pom.xml b/sdk/core/azure-core-amqp-experimental/pom.xml index d07dad5cd5435..026496cd94fc3 100644 --- a/sdk/core/azure-core-amqp-experimental/pom.xml +++ b/sdk/core/azure-core-amqp-experimental/pom.xml @@ -90,7 +90,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-amqp/pom.xml b/sdk/core/azure-core-amqp/pom.xml index e27d8af5a420e..d8ded8732686e 100644 --- a/sdk/core/azure-core-amqp/pom.xml +++ b/sdk/core/azure-core-amqp/pom.xml @@ -93,7 +93,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -126,13 +126,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/core/azure-core-experimental/pom.xml b/sdk/core/azure-core-experimental/pom.xml index 63d4366496fe2..2a6dbce0fd26f 100644 --- a/sdk/core/azure-core-experimental/pom.xml +++ b/sdk/core/azure-core-experimental/pom.xml @@ -91,7 +91,7 @@ com.azure azure-xml - 1.0.0-beta.3 + 1.0.0-beta.4 test @@ -122,7 +122,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-http-jdk-httpclient/pom.xml b/sdk/core/azure-core-http-jdk-httpclient/pom.xml index d4ad81ac01bb1..3773ddb13ba92 100644 --- a/sdk/core/azure-core-http-jdk-httpclient/pom.xml +++ b/sdk/core/azure-core-http-jdk-httpclient/pom.xml @@ -105,7 +105,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-http-netty/pom.xml b/sdk/core/azure-core-http-netty/pom.xml index 62ed732cf0c83..ab9d5a737ae48 100644 --- a/sdk/core/azure-core-http-netty/pom.xml +++ b/sdk/core/azure-core-http-netty/pom.xml @@ -88,69 +88,69 @@ io.netty netty-handler - 4.1.101.Final + 4.1.107.Final io.netty netty-handler-proxy - 4.1.101.Final + 4.1.107.Final io.netty netty-buffer - 4.1.101.Final + 4.1.107.Final io.netty netty-codec - 4.1.101.Final + 4.1.107.Final io.netty netty-codec-http - 4.1.101.Final + 4.1.107.Final io.netty netty-codec-http2 - 4.1.101.Final + 4.1.107.Final io.netty netty-transport-native-unix-common - 4.1.101.Final + 4.1.107.Final io.netty netty-transport-native-epoll - 4.1.101.Final + 4.1.107.Final linux-x86_64 io.netty netty-transport-native-kqueue - 4.1.101.Final + 4.1.107.Final osx-x86_64 io.netty netty-tcnative-boringssl-static - 2.0.62.Final + 2.0.65.Final ${boring-ssl-classifier} io.projectreactor.netty reactor-netty-http - 1.0.40 + 1.0.43 io.netty netty-common - 4.1.101.Final + 4.1.107.Final @@ -176,7 +176,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -210,18 +210,18 @@ - io.netty:netty-tcnative-boringssl-static:[2.0.62.Final] - io.projectreactor.netty:reactor-netty-http:[1.0.40] - io.netty:netty-buffer:[4.1.101.Final] - io.netty:netty-common:[4.1.101.Final] - io.netty:netty-codec:[4.1.101.Final] - io.netty:netty-codec-http:[4.1.101.Final] - io.netty:netty-codec-http2:[4.1.101.Final] - io.netty:netty-handler:[4.1.101.Final] - io.netty:netty-handler-proxy:[4.1.101.Final] - io.netty:netty-transport-native-unix-common:[4.1.101.Final] - io.netty:netty-transport-native-epoll:[4.1.101.Final] - io.netty:netty-transport-native-kqueue:[4.1.101.Final] + io.netty:netty-tcnative-boringssl-static:[2.0.65.Final] + io.projectreactor.netty:reactor-netty-http:[1.0.43] + io.netty:netty-buffer:[4.1.107.Final] + io.netty:netty-common:[4.1.107.Final] + io.netty:netty-codec:[4.1.107.Final] + io.netty:netty-codec-http:[4.1.107.Final] + io.netty:netty-codec-http2:[4.1.107.Final] + io.netty:netty-handler:[4.1.107.Final] + io.netty:netty-handler-proxy:[4.1.107.Final] + io.netty:netty-transport-native-unix-common:[4.1.107.Final] + io.netty:netty-transport-native-epoll:[4.1.107.Final] + io.netty:netty-transport-native-kqueue:[4.1.107.Final] diff --git a/sdk/core/azure-core-http-netty/src/main/java/com/azure/core/http/netty/implementation/HttpProxyHandler.java b/sdk/core/azure-core-http-netty/src/main/java/com/azure/core/http/netty/implementation/HttpProxyHandler.java index 046d8457f36c1..68a27d8aa8e83 100644 --- a/sdk/core/azure-core-http-netty/src/main/java/com/azure/core/http/netty/implementation/HttpProxyHandler.java +++ b/sdk/core/azure-core-http-netty/src/main/java/com/azure/core/http/netty/implementation/HttpProxyHandler.java @@ -139,6 +139,7 @@ protected void removeDecoder(ChannelHandlerContext ctx) { this.codec.removeInboundHandler(); } + @SuppressWarnings("deprecation") @Override protected Object newInitialMessage(ChannelHandlerContext ctx) { // This needs to handle no authorization proxying. diff --git a/sdk/core/azure-core-http-okhttp/pom.xml b/sdk/core/azure-core-http-okhttp/pom.xml index d880d029b2830..47301ce37dfc9 100644 --- a/sdk/core/azure-core-http-okhttp/pom.xml +++ b/sdk/core/azure-core-http-okhttp/pom.xml @@ -84,7 +84,7 @@ com.squareup.okio okio-jvm - 3.6.0 + 3.9.0 @@ -117,7 +117,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -152,7 +152,7 @@ com.squareup.okhttp3:okhttp:[4.12.0] - com.squareup.okio:okio-jvm:[3.6.0] + com.squareup.okio:okio-jvm:[3.9.0] diff --git a/sdk/core/azure-core-http-vertx/pom.xml b/sdk/core/azure-core-http-vertx/pom.xml index 778ca79408acb..4e3fe097a13f8 100644 --- a/sdk/core/azure-core-http-vertx/pom.xml +++ b/sdk/core/azure-core-http-vertx/pom.xml @@ -85,20 +85,20 @@ io.vertx vertx-codegen - 4.5.4 + 4.5.5 provided io.vertx vertx-core - 4.5.4 + 4.5.5 io.vertx vertx-reactive-streams - 4.5.4 + 4.5.5 @@ -125,7 +125,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -159,13 +159,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -180,9 +180,9 @@ - io.vertx:vertx-codegen:[4.5.4] - io.vertx:vertx-core:[4.5.4] - io.vertx:vertx-reactive-streams:[4.5.4] + io.vertx:vertx-codegen:[4.5.5] + io.vertx:vertx-core:[4.5.5] + io.vertx:vertx-reactive-streams:[4.5.5] diff --git a/sdk/core/azure-core-management/pom.xml b/sdk/core/azure-core-management/pom.xml index 5e72ed4f7d291..9d8a4f0fc1367 100644 --- a/sdk/core/azure-core-management/pom.xml +++ b/sdk/core/azure-core-management/pom.xml @@ -100,7 +100,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-metrics-opentelemetry/pom.xml b/sdk/core/azure-core-metrics-opentelemetry/pom.xml index 120347a186a83..5dc06e3d6031d 100644 --- a/sdk/core/azure-core-metrics-opentelemetry/pom.xml +++ b/sdk/core/azure-core-metrics-opentelemetry/pom.xml @@ -48,7 +48,7 @@ io.opentelemetry opentelemetry-api - 1.35.0 + 1.36.0 com.azure @@ -66,14 +66,14 @@ io.opentelemetry opentelemetry-sdk - 1.35.0 + 1.36.0 test io.opentelemetry opentelemetry-sdk-testing - 1.35.0 + 1.36.0 test @@ -122,7 +122,7 @@ io.opentelemetry opentelemetry-exporter-otlp - 1.35.0 + 1.36.0 test @@ -137,11 +137,11 @@ - io.opentelemetry:opentelemetry-api:[1.35.0] - io.opentelemetry:opentelemetry-sdk:[1.35.0] - io.opentelemetry:opentelemetry-sdk-testing:[1.35.0] - io.opentelemetry:opentelemetry-exporter-logging:[1.35.0] - io.opentelemetry:opentelemetry-exporter-otlp:[1.35.0] + io.opentelemetry:opentelemetry-api:[1.36.0] + io.opentelemetry:opentelemetry-sdk:[1.36.0] + io.opentelemetry:opentelemetry-sdk-testing:[1.36.0] + io.opentelemetry:opentelemetry-exporter-logging:[1.36.0] + io.opentelemetry:opentelemetry-exporter-otlp:[1.36.0] diff --git a/sdk/core/azure-core-perf/pom.xml b/sdk/core/azure-core-perf/pom.xml index 3ec33c3297c05..33d2c0c331d41 100644 --- a/sdk/core/azure-core-perf/pom.xml +++ b/sdk/core/azure-core-perf/pom.xml @@ -73,7 +73,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-serializer-avro-apache/pom.xml b/sdk/core/azure-core-serializer-avro-apache/pom.xml index 675a6cce9ec86..011cba1f872fc 100644 --- a/sdk/core/azure-core-serializer-avro-apache/pom.xml +++ b/sdk/core/azure-core-serializer-avro-apache/pom.xml @@ -94,7 +94,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-serializer-avro-jackson/pom.xml b/sdk/core/azure-core-serializer-avro-jackson/pom.xml index 5a279a015e575..d162508a8ea0e 100644 --- a/sdk/core/azure-core-serializer-avro-jackson/pom.xml +++ b/sdk/core/azure-core-serializer-avro-jackson/pom.xml @@ -87,7 +87,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-serializer-json-gson/pom.xml b/sdk/core/azure-core-serializer-json-gson/pom.xml index 7c61a2adeb157..205b8456a5342 100644 --- a/sdk/core/azure-core-serializer-json-gson/pom.xml +++ b/sdk/core/azure-core-serializer-json-gson/pom.xml @@ -88,7 +88,7 @@ com.google.code.gson gson - 2.10 + 2.10.1 @@ -120,7 +120,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -135,7 +135,7 @@ - com.google.code.gson:gson:[2.10] + com.google.code.gson:gson:[2.10.1] diff --git a/sdk/core/azure-core-serializer-json-jackson/pom.xml b/sdk/core/azure-core-serializer-json-jackson/pom.xml index 85ccceb9a7f0a..e8ba72ff83f89 100644 --- a/sdk/core/azure-core-serializer-json-jackson/pom.xml +++ b/sdk/core/azure-core-serializer-json-jackson/pom.xml @@ -144,7 +144,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/core/azure-core-test/assets.json b/sdk/core/azure-core-test/assets.json index 8b1f1986b603d..95c239c924ba0 100644 --- a/sdk/core/azure-core-test/assets.json +++ b/sdk/core/azure-core-test/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/core/azure-core-test", - "Tag": "java/core/azure-core-test_7e6e34fcb0" + "Tag": "java/core/azure-core-test_60793a10db" } diff --git a/sdk/core/azure-core-test/pom.xml b/sdk/core/azure-core-test/pom.xml index f1e5fc9f325cc..c740bab2383d7 100644 --- a/sdk/core/azure-core-test/pom.xml +++ b/sdk/core/azure-core-test/pom.xml @@ -81,13 +81,13 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 io.projectreactor.netty reactor-netty-http - 1.0.40 + 1.0.43 org.apache.commons @@ -152,9 +152,9 @@ - io.projectreactor:reactor-test:[3.4.34] + io.projectreactor:reactor-test:[3.4.36] com.fasterxml.jackson.dataformat:jackson-dataformat-xml:[2.13.5] - io.projectreactor.netty:reactor-netty-http:[1.0.40] + io.projectreactor.netty:reactor-netty-http:[1.0.43] org.junit.jupiter:junit-jupiter-api:[5.9.3] org.junit.jupiter:junit-jupiter-params:[5.9.3] diff --git a/sdk/core/azure-core-test/src/main/java/com/azure/core/test/http/TestProxyTestServer.java b/sdk/core/azure-core-test/src/main/java/com/azure/core/test/http/TestProxyTestServer.java index a560d35310505..c977672c92115 100644 --- a/sdk/core/azure-core-test/src/main/java/com/azure/core/test/http/TestProxyTestServer.java +++ b/sdk/core/azure-core-test/src/main/java/com/azure/core/test/http/TestProxyTestServer.java @@ -19,7 +19,7 @@ public class TestProxyTestServer implements Closeable { private static final String TEST_JSON_RESPONSE_BODY = "{\"modelId\":\"0cd2728b-210e-4c05-b706-f70554276bcc\"," + "\"createdDateTime\":\"2022-08-31T00:00:00Z\",\"apiVersion\":\"2022-08-31\"," - + " \"accountKey\" : \"secret_account_key\"}"; + + " \"accountKey\" : \"secret_account_key\"," + " \"client_secret\" : \"secret_client_secret\"}"; private static final String TEST_XML_RESPONSE_BODY = "{\"Body\":\"" + "sensitiveInformation=\",\"primaryKey\":" + "\"fakePrimaryKey\", \"TableName\":\"listtable09bf2a3d\"}"; diff --git a/sdk/core/azure-core-test/src/main/java/com/azure/core/test/utils/TestProxyUtils.java b/sdk/core/azure-core-test/src/main/java/com/azure/core/test/utils/TestProxyUtils.java index 38ee0236c0eb7..6072f67ae27af 100644 --- a/sdk/core/azure-core-test/src/main/java/com/azure/core/test/utils/TestProxyUtils.java +++ b/sdk/core/azure-core-test/src/main/java/com/azure/core/test/utils/TestProxyUtils.java @@ -48,7 +48,7 @@ public class TestProxyUtils { private static final List JSON_PROPERTIES_TO_REDACT = new ArrayList<>(Arrays.asList("authHeader", "accountKey", "accessToken", "accountName", "applicationId", - "apiKey", "connectionString", "url", "host", "password", "userName")); + "apiKey", "client_secret", "connectionString", "url", "host", "password", "userName")); private static final Map HEADER_KEY_REGEX_TO_REDACT = new HashMap() { { diff --git a/sdk/core/azure-core-test/src/test/java/com/azure/core/test/TestProxyTests.java b/sdk/core/azure-core-test/src/test/java/com/azure/core/test/TestProxyTests.java index 9eeea7e6abc3f..ddfd5294fe16e 100644 --- a/sdk/core/azure-core-test/src/test/java/com/azure/core/test/TestProxyTests.java +++ b/sdk/core/azure-core-test/src/test/java/com/azure/core/test/TestProxyTests.java @@ -234,6 +234,7 @@ public void testRecordWithRedaction() { .startsWith("https://REDACTED/fr/models//905a58f9-131e-42b8-8410-493ab1517d62")); // custom sanitizers assertEquals(REDACTED, record.getResponse().get("modelId")); + assertEquals(REDACTED, record.getResponse().get("client_secret")); } } diff --git a/sdk/core/azure-core-tracing-opentelemetry-samples/pom.xml b/sdk/core/azure-core-tracing-opentelemetry-samples/pom.xml index 01d819bd3dfce..fcdaf5c0fa8fd 100644 --- a/sdk/core/azure-core-tracing-opentelemetry-samples/pom.xml +++ b/sdk/core/azure-core-tracing-opentelemetry-samples/pom.xml @@ -53,31 +53,31 @@ com.azure azure-data-appconfiguration - 1.5.2 + 1.5.3 test io.opentelemetry opentelemetry-exporter-logging - 1.35.0 + 1.36.0 test io.opentelemetry opentelemetry-sdk-extension-autoconfigure - 1.35.0 + 1.36.0 test com.azure azure-security-keyvault-secrets - 4.8.0 + 4.8.1 test com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/core/azure-core-tracing-opentelemetry/pom.xml b/sdk/core/azure-core-tracing-opentelemetry/pom.xml index 4dbac690ee314..4e2d81da3c0bd 100644 --- a/sdk/core/azure-core-tracing-opentelemetry/pom.xml +++ b/sdk/core/azure-core-tracing-opentelemetry/pom.xml @@ -60,7 +60,7 @@ io.opentelemetry opentelemetry-api - 1.35.0 + 1.36.0 com.azure @@ -78,7 +78,7 @@ io.opentelemetry opentelemetry-sdk - 1.35.0 + 1.36.0 test @@ -115,7 +115,7 @@ io.opentelemetry opentelemetry-sdk-testing test - 1.35.0 + 1.36.0 org.openjdk.jmh @@ -141,9 +141,9 @@ - io.opentelemetry:opentelemetry-api:[1.35.0] - io.opentelemetry:opentelemetry-sdk:[1.35.0] - io.opentelemetry:opentelemetry-sdk-testing:[1.35.0] + io.opentelemetry:opentelemetry-api:[1.36.0] + io.opentelemetry:opentelemetry-sdk:[1.36.0] + io.opentelemetry:opentelemetry-sdk-testing:[1.36.0] diff --git a/sdk/core/azure-core-version-tests/pom.xml b/sdk/core/azure-core-version-tests/pom.xml index d89ed2c5f8db1..ed057ae1cb4d4 100644 --- a/sdk/core/azure-core-version-tests/pom.xml +++ b/sdk/core/azure-core-version-tests/pom.xml @@ -97,7 +97,7 @@ io.projectreactor reactor-core - 3.4.34 + 3.4.36 + 3.4.36 test @@ -151,7 +151,7 @@ - io.projectreactor:reactor-core:[3.4.34] + io.projectreactor:reactor-core:[3.4.36] com.fasterxml.jackson.core:jackson-annotations:[2.13.5] com.fasterxml.jackson.core:jackson-core:[2.13.5] com.fasterxml.jackson.core:jackson-databind:[2.13.5] diff --git a/sdk/core/azure-core/CHANGELOG.md b/sdk/core/azure-core/CHANGELOG.md index 241f304482020..7b8546e9b8cae 100644 --- a/sdk/core/azure-core/CHANGELOG.md +++ b/sdk/core/azure-core/CHANGELOG.md @@ -8,6 +8,9 @@ ### Bugs Fixed +- Fixed a bug where `text/event-stream` content type wasn't being handled correctly. + Replaced content type exact match `equals` by `startsWith`. ([#39128](https://github.com/Azure/azure-sdk-for-java/issues/39128)) + ### Other Changes ## 1.47.0 (2024-03-01) diff --git a/sdk/core/azure-core/pom.xml b/sdk/core/azure-core/pom.xml index 6386304cc9b58..55bb27ae6c57e 100644 --- a/sdk/core/azure-core/pom.xml +++ b/sdk/core/azure-core/pom.xml @@ -143,7 +143,7 @@ io.projectreactor reactor-core - 3.4.34 + 3.4.36 @@ -163,7 +163,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -236,7 +236,7 @@ - io.projectreactor:reactor-core:[3.4.34] + io.projectreactor:reactor-core:[3.4.36] com.fasterxml.jackson.core:jackson-annotations:[2.13.5] com.fasterxml.jackson.core:jackson-core:[2.13.5] com.fasterxml.jackson.core:jackson-databind:[2.13.5] diff --git a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/ReflectionSerializable.java b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/ReflectionSerializable.java index fb08375e0630c..850161f2a87e0 100644 --- a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/ReflectionSerializable.java +++ b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/ReflectionSerializable.java @@ -58,17 +58,23 @@ public final class ReflectionSerializable { try { xmlSerializable = Class.forName("com.azure.xml.XmlSerializable"); xmlReader = Class.forName("com.azure.xml.XmlReader"); - - Class xmlProviders = Class.forName("com.azure.xml.XmlProviders"); - - xmlReaderCreator = ReflectionUtils.getMethodInvoker(xmlProviders, - xmlProviders.getDeclaredMethod("createReader", byte[].class)); - - xmlWriterCreator = ReflectionUtils.getMethodInvoker(xmlProviders, - xmlProviders.getDeclaredMethod("createWriter", OutputStream.class)); - Class xmlWriter = Class.forName("com.azure.xml.XmlWriter"); + try { + Class xmlProviders = Class.forName("com.azure.xml.XmlProviders"); + + xmlReaderCreator = ReflectionUtils.getMethodInvoker(xmlProviders, + xmlProviders.getDeclaredMethod("createReader", byte[].class)); + + xmlWriterCreator = ReflectionUtils.getMethodInvoker(xmlProviders, + xmlProviders.getDeclaredMethod("createWriter", OutputStream.class)); + } catch (LinkageError | ReflectiveOperationException ex) { + xmlReaderCreator = ReflectionUtils.getMethodInvoker(xmlReader, + xmlReader.getDeclaredMethod("fromBytes", byte[].class)); + xmlWriterCreator = ReflectionUtils.getMethodInvoker(xmlWriter, + xmlWriter.getDeclaredMethod("toStream", OutputStream.class)); + } + xmlWriterWriteStartDocument = ReflectionUtils.getMethodInvoker(xmlWriter, xmlWriter.getDeclaredMethod("writeStartDocument")); diff --git a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java index 0e9811ef1d41e..37e7693d40b87 100644 --- a/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java +++ b/sdk/core/azure-core/src/main/java/com/azure/core/implementation/http/rest/AsyncRestProxy.java @@ -204,7 +204,7 @@ static Mono handleBodyReturnType(HttpResponse sourceResponse, Function getResponseHeaderAndReplayability() { Arguments.of(new HttpHeaders().set(HttpHeaderName.CONTENT_TYPE, ContentType.APPLICATION_OCTET_STREAM), true), Arguments.of(new HttpHeaders().set(HttpHeaderName.CONTENT_TYPE, "text/event-stream"), false), + Arguments.of(new HttpHeaders().set(HttpHeaderName.CONTENT_TYPE, "text/event-stream; charset=utf-8"), false), Arguments.of(new HttpHeaders().set(HttpHeaderName.CONTENT_TYPE, ContentType.APPLICATION_JSON), true), Arguments.of(new HttpHeaders().set(HttpHeaderName.CONTENT_TYPE, "application/xml"), true)); } diff --git a/sdk/core/ci.yml b/sdk/core/ci.yml index 370782a6fab18..d60f1e11b262b 100644 --- a/sdk/core/ci.yml +++ b/sdk/core/ci.yml @@ -96,10 +96,11 @@ parameters: default: true extends: - template: ../../eng/pipelines/templates/stages/archetype-sdk-client.yml + template: /eng/pipelines/templates/stages/archetype-sdk-client.yml parameters: ServiceDirectory: core JavadocSafeJavaBuildVersion: '1.21' + TimeoutInMinutes: 90 EnableBatchRelease: true ReleaseDependsOnLiveTests: ${{ parameters.release_dependsonlivetests }} Artifacts: @@ -163,6 +164,12 @@ extends: groupId: com.azure safeName: azurecoretracingopentelemetry releaseInBatch: ${{ parameters.release_azurecoretracingopentelemetry }} + AdditionalModules: + - name: azure-core-perf + groupId: com.azure + # required by the above perf library + - name: perf-test-core + groupId: com.azure MatrixReplace: - TestGoals=(surefire:test)/$1 failsafe:integration-test failsafe:verify AdditionalStagesAfterBuild: @@ -172,7 +179,6 @@ extends: jobs: - template: /eng/pipelines/templates/jobs/ci.versions.tests.yml parameters: - ServiceDirectory: core Artifacts: - name: azure-core-version-tests groupId: com.azure @@ -191,59 +197,6 @@ extends: TestGoals: surefire:test LiveTestStages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: core - Artifacts: - - name: azure-core - groupId: com.azure - safeName: azurecore - - name: azure-core-amqp - groupId: com.azure - safeName: azurecoreamqp - - name: azure-core-amqp-experimental - groupId: com.azure - safeName: azurecoreamqpexperimental - - name: azure-core-experimental - groupId: com.azure - safeName: azurecoreexperimental - skipPublishDocMs: true - - name: azure-core-http-jdk-httpclient - groupId: com.azure - safeName: azurecorehttpjdkhttpclient - - name: azure-core-http-netty - groupId: com.azure - safeName: azurecorehttpnetty - - name: azure-core-http-okhttp - groupId: com.azure - safeName: azurecorehttpokhttp - - name: azure-core-management - groupId: com.azure - safeName: azurecoremanagement - - name: azure-core-serializer-avro-apache - groupId: com.azure - safeName: azurecoreserializeravroapache - - name: azure-core-metrics-opentelemetry - groupId: com.azure - safeName: azurecoremetricsopentelemetry - - name: azure-core-serializer-json-gson - groupId: com.azure - safeName: azurecoreserializerjsongson - - name: azure-core-serializer-json-jackson - groupId: com.azure - safeName: azurecoreserializerjsonjackson - - name: azure-core-test - groupId: com.azure - safeName: azurecoretest - skipPublishDocMs: true - - name: azure-core-tracing-opentelemetry - groupId: com.azure - safeName: azurecoretracingopentelemetry - - name: azure-core-tracing-opentelemetry-samples - groupId: com.azure - safeName: azurecoretracingopentelemetrysamples - MatrixReplace: - - TestGoals=(surefire:test)/$1 failsafe:integration-test failsafe:verify - template: /sdk/storage/tests-template.yml parameters: AdditionalMatrixReplace: diff --git a/sdk/cosmos/azure-cosmos-benchmark/ctl-image-build.yml b/sdk/cosmos/azure-cosmos-benchmark/ctl-image-build.yml index 38c6de167b131..696359daf3bac 100644 --- a/sdk/cosmos/azure-cosmos-benchmark/ctl-image-build.yml +++ b/sdk/cosmos/azure-cosmos-benchmark/ctl-image-build.yml @@ -1,80 +1,88 @@ -jobs: - - job: Cosmos_CTL - timeoutInMinutes: 20 - variables: - - template: /eng/pipelines/templates/variables/globals.yml - - name: ContainerRegistryName - value: 'javactl' - - name: ContainerRegistryUserName - value: 'javactl' - - name: ContainerRegistryUrl - value: 'javactl.azurecr.io' +extends: + template: /eng/pipelines/templates/stages/1es-redirect.yml + parameters: + stages: + - stage: Publish_Cosmos_CTL + jobs: + - job: Cosmos_CTL + timeoutInMinutes: 20 + variables: + - template: /eng/pipelines/templates/variables/globals.yml + - name: ContainerRegistryName + value: 'javactl' + - name: ContainerRegistryUserName + value: 'javactl' + - name: ContainerRegistryUrl + value: 'javactl.azurecr.io' + - template: /eng/pipelines/templates/variables/image.yml - pool: - name: 'azsdk-pool-mms-ubuntu-2004-general' - vmImage: 'MMSUbuntu20.04' + pool: + name: $(LINUXPOOL) + image: $(LINUXVMIMAGE) + os: $(LINUXOS) - steps: - - task: Maven@3 - displayName: 'Build Cosmos and Benchmark' - inputs: - mavenPomFile: pom.xml - goals: 'package' - # The -Ppackage-assembly profile packages up both libraries into one package - options: '$(DefaultOptions) -Ppackage-assembly -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Drevapi.skip=true -pl com.azure:azure-cosmos,com.azure:azure-cosmos-benchmark -am' - javaHomeOption: 'JDKVersion' - jdkVersionOption: $(JavaBuildVersion) - jdkArchitectureOption: 'x64' - publishJUnitResults: false + steps: + - task: Maven@3 + displayName: 'Build Cosmos and Benchmark' + inputs: + mavenPomFile: pom.xml + goals: 'package' + # The -Ppackage-assembly profile packages up both libraries into one package + options: '$(DefaultOptions) -Ppackage-assembly -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Drevapi.skip=true -pl com.azure:azure-cosmos,com.azure:azure-cosmos-benchmark -am' + javaHomeOption: 'JDKVersion' + jdkVersionOption: $(JavaBuildVersion) + jdkArchitectureOption: 'x64' + publishJUnitResults: false - - task: CopyFiles@2 - displayName: 'Copy benchmark jar' - inputs: - SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/target' - Contents: '*' - TargetFolder: $(Agent.TempDirectory)/ctl + - task: CopyFiles@2 + displayName: 'Copy benchmark jar' + inputs: + SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/target' + Contents: '*' + TargetFolder: $(Agent.TempDirectory)/ctl - - task: CopyFiles@2 - displayName: 'Copy docker config files' - inputs: - SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/ctl' - TargetFolder: $(Agent.TempDirectory)/ctl + - task: CopyFiles@2 + displayName: 'Copy docker config files' + inputs: + SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/ctl' + TargetFolder: $(Agent.TempDirectory)/ctl - - task: CopyFiles@2 - displayName: 'Copy benchmark jar for linkedin workload' - inputs: - SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/target' - Contents: '*' - TargetFolder: $(Agent.TempDirectory)/ctl/linkedin + - task: CopyFiles@2 + displayName: 'Copy benchmark jar for linkedin workload' + inputs: + SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/target' + Contents: '*' + TargetFolder: $(Agent.TempDirectory)/ctl/linkedin - - task: CopyFiles@2 - displayName: 'Copy docker config files for linkedin workload' - inputs: - SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/ctl/linkedin' - TargetFolder: $(Agent.TempDirectory)/ctl/linkedin + - task: CopyFiles@2 + displayName: 'Copy docker config files for linkedin workload' + inputs: + SourceFolder: '$(build.sourcesdirectory)/sdk/cosmos/azure-cosmos-benchmark/ctl/linkedin' + TargetFolder: $(Agent.TempDirectory)/ctl/linkedin - # Below will build the image and push it to azure container registry - - pwsh: | - cd $(Agent.TempDirectory)/ctl - Write-Host "Executing docker build . -t java-ctl" - docker build . -t java-ctl - Write-Host "Executing az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd)" - az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd) - Write-Host "Executing docker tag java-ctl $(ContainerRegistryUrl)/javactl/benchmark" - docker tag java-ctl $(ContainerRegistryUrl)/javactl/benchmark - Write-Host "Executing docker push $(ContainerRegistryUrl)/javactl/benchmark" - docker push $(ContainerRegistryUrl)/javactl/benchmark - displayName: 'Build and push docker image to registry' + # Below will build the image and push it to azure container registry + - pwsh: | + cd $(Agent.TempDirectory)/ctl + Write-Host "Executing docker build . -t java-ctl" + docker build . -t java-ctl + Write-Host "Executing az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd)" + az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd) + Write-Host "Executing docker tag java-ctl $(ContainerRegistryUrl)/javactl/benchmark" + docker tag java-ctl $(ContainerRegistryUrl)/javactl/benchmark + Write-Host "Executing docker push $(ContainerRegistryUrl)/javactl/benchmark" + docker push $(ContainerRegistryUrl)/javactl/benchmark + displayName: 'Build and push docker image to registry' + + # Below will build the image and push it to azure container registry for linkedin workload + - pwsh: | + cd $(Agent.TempDirectory)/ctl/linkedin + Write-Host "Executing docker build . -t java-ctl-linkedin" + docker build . -t java-ctl-linkedin + Write-Host "Executing az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd)" + az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd) + Write-Host "Executing docker tag java-ctl-linkedin $(ContainerRegistryUrl)/javactl/linkedinbenchmark" + docker tag java-ctl-linkedin $(ContainerRegistryUrl)/javactl/linkedinbenchmark + Write-Host "Executing docker push $(ContainerRegistryUrl)/javactl/linkedinbenchmark" + docker push $(ContainerRegistryUrl)/javactl/linkedinbenchmark + displayName: 'Build and push docker image to registry for linkedin workload' - # Below will build the image and push it to azure container registry for linkedin workload - - pwsh: | - cd $(Agent.TempDirectory)/ctl/linkedin - Write-Host "Executing docker build . -t java-ctl-linkedin" - docker build . -t java-ctl-linkedin - Write-Host "Executing az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd)" - az acr login --name $(ContainerRegistryName) -u $(ContainerRegistryUserName) -p $(java-cosmos-container-registry-pwd) - Write-Host "Executing docker tag java-ctl-linkedin $(ContainerRegistryUrl)/javactl/linkedinbenchmark" - docker tag java-ctl-linkedin $(ContainerRegistryUrl)/javactl/linkedinbenchmark - Write-Host "Executing docker push $(ContainerRegistryUrl)/javactl/linkedinbenchmark" - docker push $(ContainerRegistryUrl)/javactl/linkedinbenchmark - displayName: 'Build and push docker image to registry for linkedin workload' diff --git a/sdk/cosmos/azure-cosmos-benchmark/pom.xml b/sdk/cosmos/azure-cosmos-benchmark/pom.xml index fbdeb72303ae4..b4c9da6dc97cd 100644 --- a/sdk/cosmos/azure-cosmos-benchmark/pom.xml +++ b/sdk/cosmos/azure-cosmos-benchmark/pom.xml @@ -51,19 +51,19 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 com.azure azure-cosmos-encryption - 2.9.0-beta.1 + 2.9.0 com.azure azure-identity - 1.11.3 + 1.11.4 com.azure @@ -182,7 +182,7 @@ Licensed under the MIT License. com.azure azure-security-keyvault-keys - 4.8.0 + 4.8.1 compile diff --git a/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/InvitationsEntityConfiguration.java b/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/InvitationsEntityConfiguration.java index 70a91a09c1a0f..51c495e58efb6 100644 --- a/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/InvitationsEntityConfiguration.java +++ b/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/InvitationsEntityConfiguration.java @@ -7,7 +7,7 @@ import com.azure.cosmos.benchmark.linkedin.data.entity.InvitationDataGenerator; import com.azure.cosmos.benchmark.linkedin.data.entity.InvitationsCollectionAttributes; import com.azure.cosmos.benchmark.linkedin.data.entity.InvitationsKeyGenerator; -import com.azure.cosmos.implementation.guava25.base.Preconditions; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; import java.util.function.Supplier; diff --git a/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/entity/InvitationDataGenerator.java b/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/entity/InvitationDataGenerator.java index d1427d2e51df6..fba8ee313af74 100644 --- a/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/entity/InvitationDataGenerator.java +++ b/sdk/cosmos/azure-cosmos-benchmark/src/main/java/com/azure/cosmos/benchmark/linkedin/data/entity/InvitationDataGenerator.java @@ -7,7 +7,7 @@ import com.azure.cosmos.benchmark.linkedin.data.Key; import com.azure.cosmos.benchmark.linkedin.data.KeyGenerator; import com.azure.cosmos.benchmark.linkedin.impl.Constants; -import com.azure.cosmos.implementation.guava25.base.Preconditions; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; import com.fasterxml.jackson.databind.node.BooleanNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.LongNode; diff --git a/sdk/cosmos/azure-cosmos-dotnet-benchmark/pom.xml b/sdk/cosmos/azure-cosmos-dotnet-benchmark/pom.xml index 5dd9c0bc35c6a..557edc274dfbf 100644 --- a/sdk/cosmos/azure-cosmos-dotnet-benchmark/pom.xml +++ b/sdk/cosmos/azure-cosmos-dotnet-benchmark/pom.xml @@ -50,7 +50,7 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 diff --git a/sdk/cosmos/azure-cosmos-dotnet-benchmark/src/main/java/com/azure/cosmos/dotnet/benchmark/OperationResult.java b/sdk/cosmos/azure-cosmos-dotnet-benchmark/src/main/java/com/azure/cosmos/dotnet/benchmark/OperationResult.java index 83ad6f0f3d180..986bf06f1a9dc 100644 --- a/sdk/cosmos/azure-cosmos-dotnet-benchmark/src/main/java/com/azure/cosmos/dotnet/benchmark/OperationResult.java +++ b/sdk/cosmos/azure-cosmos-dotnet-benchmark/src/main/java/com/azure/cosmos/dotnet/benchmark/OperationResult.java @@ -3,7 +3,7 @@ package com.azure.cosmos.dotnet.benchmark; -import com.azure.cosmos.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; public class OperationResult { private String containerName; diff --git a/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md b/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md index d83d767f801fe..cac0209b00fad 100644 --- a/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-encryption/CHANGELOG.md @@ -1,14 +1,8 @@ ## Release History -### 2.9.0-beta.1 (Unreleased) - -#### Features Added - -#### Breaking Changes - -#### Bugs Fixed - +### 2.9.0 (2024-03-26) #### Other Changes +* Updated `azure-cosmos` to version `4.57.0`. ### 2.8.0 (2024-02-08) #### Other Changes diff --git a/sdk/cosmos/azure-cosmos-encryption/README.md b/sdk/cosmos/azure-cosmos-encryption/README.md index 4a2aa544f0da6..19da9e1195448 100644 --- a/sdk/cosmos/azure-cosmos-encryption/README.md +++ b/sdk/cosmos/azure-cosmos-encryption/README.md @@ -12,7 +12,7 @@ The Azure Cosmos Encryption Plugin is used for encrypting data with a user-provi com.azure azure-cosmos-encryption - 2.8.0 + 2.9.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/cosmos/azure-cosmos-encryption/pom.xml b/sdk/cosmos/azure-cosmos-encryption/pom.xml index 88966b037fdc9..ea1d311c501a5 100644 --- a/sdk/cosmos/azure-cosmos-encryption/pom.xml +++ b/sdk/cosmos/azure-cosmos-encryption/pom.xml @@ -13,7 +13,7 @@ Licensed under the MIT License. com.azure azure-cosmos-encryption - 2.9.0-beta.1 + 2.9.0 Encryption Plugin for Azure Cosmos DB SDK This Package contains Encryption Plugin for Microsoft Azure Cosmos SDK jar @@ -57,13 +57,13 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.57.0 com.azure azure-security-keyvault-keys - 4.8.0 + 4.8.1 test @@ -80,7 +80,7 @@ Licensed under the MIT License. com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -174,7 +174,7 @@ Licensed under the MIT License. io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -196,13 +196,13 @@ Licensed under the MIT License. net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionAsyncContainer.java b/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionAsyncContainer.java index 0de8fc4a4a544..fd740b6b05ddc 100644 --- a/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionAsyncContainer.java +++ b/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionAsyncContainer.java @@ -22,7 +22,7 @@ import com.azure.cosmos.implementation.apachecommons.lang.tuple.Pair; import com.azure.cosmos.implementation.batch.ItemBatchOperation; import com.azure.cosmos.implementation.batch.ItemBulkOperation; -import com.azure.cosmos.implementation.guava25.base.Preconditions; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; import com.azure.cosmos.implementation.patch.PatchOperation; import com.azure.cosmos.implementation.patch.PatchOperationCore; import com.azure.cosmos.implementation.patch.PatchOperationType; @@ -68,7 +68,7 @@ import java.util.function.Function; import static com.azure.cosmos.implementation.Utils.getEffectiveCosmosChangeFeedRequestOptions; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /** * CosmosAsyncContainer with encryption capabilities. diff --git a/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionContainer.java b/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionContainer.java index 593ca0801a47d..0bec715508004 100644 --- a/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionContainer.java +++ b/sdk/cosmos/azure-cosmos-encryption/src/main/java/com/azure/cosmos/encryption/CosmosEncryptionContainer.java @@ -30,7 +30,7 @@ import java.util.List; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /** * CosmosContainer with encryption capabilities. diff --git a/sdk/cosmos/azure-cosmos-encryption/src/samples/java/com/azure/cosmos/encryption/Program.java b/sdk/cosmos/azure-cosmos-encryption/src/samples/java/com/azure/cosmos/encryption/Program.java index 4c8a47ec840a7..c6e7b2106307c 100644 --- a/sdk/cosmos/azure-cosmos-encryption/src/samples/java/com/azure/cosmos/encryption/Program.java +++ b/sdk/cosmos/azure-cosmos-encryption/src/samples/java/com/azure/cosmos/encryption/Program.java @@ -12,7 +12,7 @@ import com.azure.cosmos.encryption.models.CosmosEncryptionType; import com.azure.cosmos.encryption.models.SqlQuerySpecWithEncryption; import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.guava25.base.Preconditions; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; import com.azure.cosmos.models.ClientEncryptionIncludedPath; import com.azure.cosmos.models.ClientEncryptionPolicy; import com.azure.cosmos.models.CosmosClientEncryptionKeyProperties; diff --git a/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/EncryptionSyncApiCrudTest.java b/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/EncryptionSyncApiCrudTest.java index 01b7c552a7048..fdb1db2ca8755 100644 --- a/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/EncryptionSyncApiCrudTest.java +++ b/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/EncryptionSyncApiCrudTest.java @@ -16,7 +16,7 @@ import com.azure.cosmos.models.PartitionKey; import com.azure.cosmos.models.SqlParameter; import com.azure.cosmos.models.SqlQuerySpec; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.models.CosmosBulkItemResponse; import com.azure.cosmos.models.CosmosBulkOperationResponse; import com.azure.cosmos.models.CosmosBulkOperations; diff --git a/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/TestSuiteBase.java b/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/TestSuiteBase.java index 81ce7236c34c2..fdff809900bd5 100644 --- a/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/TestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-encryption/src/test/java/com/azure/cosmos/encryption/TestSuiteBase.java @@ -28,8 +28,8 @@ import com.azure.cosmos.implementation.TestConfigurations; import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.directconnectivity.Protocol; -import com.azure.cosmos.implementation.guava25.base.CaseFormat; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.CaseFormat; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import com.azure.cosmos.models.ClientEncryptionIncludedPath; import com.azure.cosmos.models.ClientEncryptionPolicy; import com.azure.cosmos.models.CompositePath; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md b/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md index 840ffe29ca47f..354d8f951db4b 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-kafka-connect/CHANGELOG.md @@ -3,8 +3,7 @@ ### 1.0.0-beta.1 (Unreleased) #### Features Added -* Added Source connector. See [PR 38748](https://github.com/Azure/azure-sdk-for-java/pull/38748) -* Added Sink connector. See [PR 38973](https://github.com/Azure/azure-sdk-for-java/pull/38973) +* Added Source connector. See [PR 39410](https://github.com/Azure/azure-sdk-for-java/pull/39410) #### Breaking Changes diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml index e89355097659b..6cadacaf53afa 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml +++ b/sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -53,8 +53,10 @@ Licensed under the MIT License. --add-opens com.azure.cosmos/com.azure.cosmos.implementation.routing=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.caches=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.faultinjection=ALL-UNNAMED - --add-opens com.azure.cosmos/com.azure.cosmos.implementation.apachecommons.lang=ALL-UNNAMED - --add-opens com.azure.cosmos/com.azure.cosmos.implementation.guava25.base=ALL-UNNAMED + + --add-exports com.azure.cosmos/com.azure.cosmos.implementation.changefeed.common=com.azure.cosmos.kafka.connect + --add-exports com.azure.cosmos/com.azure.cosmos.implementation.feedranges=com.azure.cosmos.kafka.connect + --add-exports com.azure.cosmos/com.azure.cosmos.implementation.query=com.azure.cosmos.kafka.connect @@ -70,7 +72,7 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java index 4bbb794a5906a..54f074b7477f3 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnector.java @@ -4,18 +4,14 @@ package com.azure.cosmos.kafka.connect; import com.azure.cosmos.CosmosAsyncClient; -import com.azure.cosmos.implementation.Strings; -import com.azure.cosmos.implementation.apachecommons.lang.tuple.Pair; -import com.azure.cosmos.implementation.changefeed.common.ChangeFeedState; -import com.azure.cosmos.implementation.changefeed.common.ChangeFeedStateV1; -import com.azure.cosmos.implementation.feedranges.FeedRangeContinuation; -import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; -import com.azure.cosmos.implementation.feedranges.FeedRangeInternal; -import com.azure.cosmos.implementation.query.CompositeContinuationToken; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.CosmosAsyncContainer; +import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; +import com.azure.cosmos.kafka.connect.implementation.Strings; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.tuple.Pair; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceOffsetStorageReader; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceTask; @@ -23,18 +19,23 @@ import com.azure.cosmos.kafka.connect.implementation.source.FeedRangeContinuationTopicOffset; import com.azure.cosmos.kafka.connect.implementation.source.FeedRangeTaskUnit; import com.azure.cosmos.kafka.connect.implementation.source.FeedRangesMetadataTopicOffset; +import com.azure.cosmos.kafka.connect.implementation.source.KafkaCosmosChangeFeedState; import com.azure.cosmos.kafka.connect.implementation.source.MetadataMonitorThread; import com.azure.cosmos.kafka.connect.implementation.source.MetadataTaskUnit; import com.azure.cosmos.models.CosmosContainerProperties; +import com.azure.cosmos.models.FeedRange; +import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.source.SourceConnector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -146,10 +147,10 @@ private Pair> getAllTaskUnits() { List allContainers = this.monitorThread.getAllContainers().block(); Map containerTopicMap = this.getContainersTopicMap(allContainers); List allFeedRangeTaskUnits = new ArrayList<>(); - Map>> updatedContainerToFeedRangesMap = new ConcurrentHashMap<>(); + Map> updatedContainerToFeedRangesMap = new ConcurrentHashMap<>(); for (CosmosContainerProperties containerProperties : allContainers) { - Map, String> effectiveFeedRangesContinuationMap = + Map effectiveFeedRangesContinuationMap = this.getEffectiveFeedRangesContinuationMap( this.config.getContainersConfig().getDatabaseName(), containerProperties); @@ -160,7 +161,7 @@ private Pair> getAllTaskUnits() { ); // add feedRange task unit - for (Range effectiveFeedRange : effectiveFeedRangesContinuationMap.keySet()) { + for (FeedRange effectiveFeedRange : effectiveFeedRangesContinuationMap.keySet()) { allFeedRangeTaskUnits.add( new FeedRangeTaskUnit( this.config.getContainersConfig().getDatabaseName(), @@ -184,7 +185,7 @@ private Pair> getAllTaskUnits() { return Pair.of(metadataTaskUnit, allFeedRangeTaskUnits); } - private Map, String> getEffectiveFeedRangesContinuationMap( + private Map getEffectiveFeedRangesContinuationMap( String databaseName, CosmosContainerProperties containerProperties) { // Return effective feed ranges to be used for copying data from container @@ -196,119 +197,126 @@ private Map, String> getEffectiveFeedRangesContinuationMap( // If a merge is detected, we will use the matched feedRanges from the offsets, // otherwise use the current feedRange, but constructing the continuationState based on the previous feedRange - List> containerFeedRanges = this.getFeedRanges(containerProperties); - containerFeedRanges.sort(new Comparator>() { - @Override - public int compare(Range o1, Range o2) { - return o1.getMin().compareTo(o2.getMin()); - } - }); + List containerFeedRanges = this.getFeedRanges(containerProperties); FeedRangesMetadataTopicOffset feedRangesMetadataTopicOffset = this.offsetStorageReader.getFeedRangesMetadataOffset(databaseName, containerProperties.getResourceId()); - Map, String> effectiveFeedRangesContinuationMap = new LinkedHashMap<>(); - - for (Range containerFeedRange : containerFeedRanges) { - if (feedRangesMetadataTopicOffset == null) { - // there is no existing offset, return the current container feedRanges with continuationState null (start from refresh) - effectiveFeedRangesContinuationMap.put(containerFeedRange, Strings.Emtpy); - } else { - // there is existing offsets, need to find out effective feedRanges based on the offset - effectiveFeedRangesContinuationMap.putAll( - this.getEffectiveContinuationMapForSingleFeedRange( + Map effectiveFeedRangesContinuationMap = new LinkedHashMap<>(); + CosmosAsyncContainer container = this.cosmosClient.getDatabase(databaseName).getContainer(containerProperties.getId()); + + Flux.fromIterable(containerFeedRanges) + .flatMap(containerFeedRange -> { + if (feedRangesMetadataTopicOffset == null) { + return Mono.just( + Collections.singletonMap(containerFeedRange, Strings.Emtpy)); + } else { + // there is existing offsets, need to find out effective feedRanges based on the offset + return this.getEffectiveContinuationMapForSingleFeedRange( databaseName, containerProperties.getResourceId(), containerFeedRange, - feedRangesMetadataTopicOffset.getFeedRanges()) - ); - } - } + container, + feedRangesMetadataTopicOffset.getFeedRanges()); + } + }) + .doOnNext(map -> { + effectiveFeedRangesContinuationMap.putAll(map); + }) + .blockLast(); return effectiveFeedRangesContinuationMap; } - private Map, String> getEffectiveContinuationMapForSingleFeedRange( + private Mono> getEffectiveContinuationMapForSingleFeedRange( String databaseName, String containerRid, - Range containerFeedRange, - List> rangesFromMetadataTopicOffset) { + FeedRange containerFeedRange, + CosmosAsyncContainer cosmosAsyncContainer, + List rangesFromMetadataTopicOffset) { //first try to find out whether there is exact feedRange matching FeedRangeContinuationTopicOffset feedRangeContinuationTopicOffset = this.offsetStorageReader.getFeedRangeContinuationOffset(databaseName, containerRid, containerFeedRange); - Map, String> effectiveContinuationMap = new LinkedHashMap<>(); + Map effectiveContinuationMap = new LinkedHashMap<>(); if (feedRangeContinuationTopicOffset != null) { // we can find the continuation offset based on exact feedRange matching effectiveContinuationMap.put( containerFeedRange, this.getContinuationStateFromOffset( - containerRid, feedRangeContinuationTopicOffset, containerFeedRange)); - return effectiveContinuationMap; + return Mono.just(effectiveContinuationMap); } // we can not find the continuation offset based on the exact feed range matching // it means the previous Partition key range could have gone due to container split/merge // need to find out overlapped feedRanges from offset - List> overlappedFeedRangesFromOffset = - rangesFromMetadataTopicOffset - .stream() - .filter(rangeFromOffset -> Range.checkOverlapping(rangeFromOffset, containerFeedRange)) - .collect(Collectors.toList()); - - if (overlappedFeedRangesFromOffset.size() == 1) { - // split - use the current containerFeedRange, but construct the continuationState based on the feedRange from offset - effectiveContinuationMap.put( - containerFeedRange, - this.getContinuationStateFromOffset( - containerRid, - this.offsetStorageReader.getFeedRangeContinuationOffset(databaseName, containerRid, overlappedFeedRangesFromOffset.get(0)), - containerFeedRange)); - return effectiveContinuationMap; - } - - if (overlappedFeedRangesFromOffset.size() > 1) { - // merge - use the feed ranges from the offset - for (Range overlappedRangeFromOffset : overlappedFeedRangesFromOffset) { - effectiveContinuationMap.put( - overlappedRangeFromOffset, - this.getContinuationStateFromOffset( - containerRid, - this.offsetStorageReader.getFeedRangeContinuationOffset(databaseName, containerRid, overlappedRangeFromOffset), - overlappedRangeFromOffset)); - } - - return effectiveContinuationMap; - } - - // Can not find overlapped ranges from offset, this should never happen, fail - LOGGER.error("Can not find overlapped ranges for feedRange {}", containerFeedRange); - throw new IllegalStateException("Can not find overlapped ranges for feedRange " + containerFeedRange); + return Flux.fromIterable(rangesFromMetadataTopicOffset) + .flatMap(rangeFromOffset -> { + return ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .checkFeedRangeOverlapping(cosmosAsyncContainer, rangeFromOffset, containerFeedRange) + .flatMap(overlapped -> { + if (overlapped) { + return Mono.just(rangeFromOffset); + } else { + return Mono.empty(); + } + }); + }) + .collectList() + .flatMap(overlappedFeedRangesFromOffset -> { + if (overlappedFeedRangesFromOffset.size() == 1) { + // split - use the current containerFeedRange, but construct the continuationState based on the feedRange from offset + effectiveContinuationMap.put( + containerFeedRange, + this.getContinuationStateFromOffset( + this.offsetStorageReader.getFeedRangeContinuationOffset(databaseName, containerRid, overlappedFeedRangesFromOffset.get(0)), + containerFeedRange)); + return Mono.just(effectiveContinuationMap); + } + + if (overlappedFeedRangesFromOffset.size() > 1) { + // merge - use the feed ranges from the offset + for (FeedRange overlappedRangeFromOffset : overlappedFeedRangesFromOffset) { + effectiveContinuationMap.put( + overlappedRangeFromOffset, + this.getContinuationStateFromOffset( + this.offsetStorageReader.getFeedRangeContinuationOffset(databaseName, containerRid, overlappedRangeFromOffset), + overlappedRangeFromOffset)); + } + + return Mono.just(effectiveContinuationMap); + } + + // Can not find overlapped ranges from offset, this should never happen, fail + LOGGER.error("Can not find overlapped ranges for feedRange {}", containerFeedRange); + return Mono.error(new IllegalStateException("Can not find overlapped ranges for feedRange " + containerFeedRange)); + }); } private String getContinuationStateFromOffset( - String containerRid, FeedRangeContinuationTopicOffset feedRangeContinuationTopicOffset, - Range range) { - - ChangeFeedState stateFromOffset = ChangeFeedStateV1.fromString(feedRangeContinuationTopicOffset.getContinuationState()); - String itemLsn = feedRangeContinuationTopicOffset.getItemLsn(); - return new ChangeFeedStateV1( - containerRid, - new FeedRangeEpkImpl(range), - stateFromOffset.getMode(), - stateFromOffset.getStartFromSettings(), - FeedRangeContinuation.create( - containerRid, - new FeedRangeEpkImpl(range), - Arrays.asList(new CompositeContinuationToken(itemLsn, range)))).toString(); + FeedRange feedRange) { + + KafkaCosmosChangeFeedState changeFeedState = + new KafkaCosmosChangeFeedState( + feedRangeContinuationTopicOffset.getResponseContinuation(), + feedRange, + feedRangeContinuationTopicOffset.getItemLsn()); + + try { + return KafkaCosmosUtils.getSimpleObjectMapper().writeValueAsString(changeFeedState); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } } - private List> getFeedRanges(CosmosContainerProperties containerProperties) { + private List getFeedRanges(CosmosContainerProperties containerProperties) { return this.cosmosClient .getDatabase(this.config.getContainersConfig().getDatabaseName()) .getContainer(containerProperties.getId()) @@ -317,10 +325,7 @@ private List> getFeedRanges(CosmosContainerProperties containerPro KafkaCosmosExceptionsHelper.convertToConnectException( throwable, "GetFeedRanges failed for container " + containerProperties.getId())) - .block() - .stream() - .map(feedRange -> FeedRangeInternal.normalizeRange(((FeedRangeEpkImpl) feedRange).getRange())) - .collect(Collectors.toList()); + .block(); } private Map getContainersTopicMap(List allContainers) { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosAccountConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosAccountConfig.java index 49e2f731a05a3..d8d291dc7c773 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosAccountConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosAccountConfig.java @@ -3,11 +3,11 @@ package com.azure.cosmos.kafka.connect.implementation; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import java.util.List; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public class CosmosAccountConfig { private final String endpoint; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java index 10589369d0e8d..63dab30e29096 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/CosmosClientStore.java @@ -7,7 +7,7 @@ import com.azure.cosmos.CosmosClientBuilder; import com.azure.cosmos.GatewayConnectionConfig; import com.azure.cosmos.ThrottlingRetryOptions; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import java.time.Duration; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java index 5c75056b6f7be..ca2a8096bf350 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConfig.java @@ -3,8 +3,7 @@ package com.azure.cosmos.kafka.connect.implementation; -import com.azure.cosmos.implementation.Strings; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceContainersConfig; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java index 01a89ef20f532..50db99a79c634 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosConstants.java @@ -10,4 +10,20 @@ public class KafkaCosmosConstants { public static final String CURRENT_VERSION = CoreUtils.getProperties(PROPERTIES_FILE_NAME).get("version"); public static final String CURRENT_NAME = CoreUtils.getProperties(PROPERTIES_FILE_NAME).get("name"); public static final String USER_AGENT_SUFFIX = String.format("KafkaConnect/%s/%s", CURRENT_NAME, CURRENT_VERSION); + + public static class StatusCodes { + public static final int NOTFOUND = 404; + public static final int REQUEST_TIMEOUT = 408; + public static final int GONE = 410; + public static final int CONFLICT = 409; + public static final int PRECONDITION_FAILED = 412; + public static final int SERVICE_UNAVAILABLE = 503; + public static final int INTERNAL_SERVER_ERROR = 500; + } + + public static class SubStatusCodes { + public static final int READ_SESSION_NOT_AVAILABLE = 1002; + public static final int PARTITION_KEY_RANGE_GONE = 1002; + public static final int COMPLETING_SPLIT_OR_MERGE = 1007; + } } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java index 0d5a8bb8759e3..a8adfd35a22ab 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosExceptionsHelper.java @@ -4,17 +4,16 @@ package com.azure.cosmos.kafka.connect.implementation; import com.azure.cosmos.CosmosException; -import com.azure.cosmos.implementation.HttpConstants; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; public class KafkaCosmosExceptionsHelper { public static boolean isTransientFailure(int statusCode, int substatusCode) { - return statusCode == HttpConstants.StatusCodes.GONE - || statusCode == HttpConstants.StatusCodes.SERVICE_UNAVAILABLE - || statusCode == HttpConstants.StatusCodes.INTERNAL_SERVER_ERROR - || statusCode == HttpConstants.StatusCodes.REQUEST_TIMEOUT - || (statusCode == HttpConstants.StatusCodes.NOTFOUND && substatusCode == HttpConstants.SubStatusCodes.READ_SESSION_NOT_AVAILABLE); + return statusCode == KafkaCosmosConstants.StatusCodes.GONE + || statusCode == KafkaCosmosConstants.StatusCodes.SERVICE_UNAVAILABLE + || statusCode == KafkaCosmosConstants.StatusCodes.INTERNAL_SERVER_ERROR + || statusCode == KafkaCosmosConstants.StatusCodes.REQUEST_TIMEOUT + || (statusCode == KafkaCosmosConstants.StatusCodes.NOTFOUND && substatusCode == KafkaCosmosConstants.SubStatusCodes.READ_SESSION_NOT_AVAILABLE); } @@ -37,9 +36,9 @@ public static boolean isFeedRangeGoneException(Throwable throwable) { } public static boolean isFeedRangeGoneException(int statusCode, int substatusCode) { - return statusCode == HttpConstants.StatusCodes.GONE - && (substatusCode == HttpConstants.SubStatusCodes.PARTITION_KEY_RANGE_GONE - || substatusCode == HttpConstants.SubStatusCodes.COMPLETING_SPLIT_OR_MERGE); + return statusCode == KafkaCosmosConstants.StatusCodes.GONE + && (substatusCode == KafkaCosmosConstants.SubStatusCodes.PARTITION_KEY_RANGE_GONE + || substatusCode == KafkaCosmosConstants.SubStatusCodes.COMPLETING_SPLIT_OR_MERGE); } public static ConnectException convertToConnectException(Throwable throwable, String message) { @@ -52,7 +51,7 @@ public static ConnectException convertToConnectException(Throwable throwable, St public static boolean isResourceExistsException(Throwable throwable) { if (throwable instanceof CosmosException) { - return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.CONFLICT; + return ((CosmosException) throwable).getStatusCode() == KafkaCosmosConstants.StatusCodes.CONFLICT; } return false; @@ -60,7 +59,7 @@ public static boolean isResourceExistsException(Throwable throwable) { public static boolean isNotFoundException(Throwable throwable) { if (throwable instanceof CosmosException) { - return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.NOTFOUND; + return ((CosmosException) throwable).getStatusCode() == KafkaCosmosConstants.StatusCodes.NOTFOUND; } return false; @@ -68,7 +67,7 @@ public static boolean isNotFoundException(Throwable throwable) { public static boolean isPreconditionFailedException(Throwable throwable) { if (throwable instanceof CosmosException) { - return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.PRECONDITION_FAILED; + return ((CosmosException) throwable).getStatusCode() == KafkaCosmosConstants.StatusCodes.PRECONDITION_FAILED; } return false; @@ -76,7 +75,7 @@ public static boolean isPreconditionFailedException(Throwable throwable) { public static boolean isTimeoutException(Throwable throwable) { if (throwable instanceof CosmosException) { - return ((CosmosException) throwable).getStatusCode() == HttpConstants.StatusCodes.REQUEST_TIMEOUT; + return ((CosmosException) throwable).getStatusCode() == KafkaCosmosConstants.StatusCodes.REQUEST_TIMEOUT; } return false; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosUtils.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosUtils.java new file mode 100644 index 0000000000000..5ddac7a1047a9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/KafkaCosmosUtils.java @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import com.fasterxml.jackson.module.afterburner.AfterburnerModule; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaCosmosUtils { + private final static Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosUtils.class); + private static final int JAVA_VERSION = getJavaVersion(); + private static ObjectMapper simpleObjectMapper = createAndInitializeObjectMapper(false); + + private static int getJavaVersion() { + int version = -1; + try { + String completeJavaVersion = System.getProperty("java.version"); + String[] versionElements = completeJavaVersion.split("\\."); + int versionFirstPart = Integer.parseInt(versionElements[0]); + // Java 8 or lower format is 1.6.0, 1.7.0, 1.7.0, 1.8.0 + // Java 9 or higher format is 9.0, 10.0, 11.0 + if (versionFirstPart == 1) { + version = Integer.parseInt(versionElements[1]); + } else { + version = versionFirstPart; + } + return version; + } catch (Exception ex) { + // Consumed the exception we got during parsing + // For unknown version we wil mark it as -1 + LOGGER.warn("Error while fetching java version", ex); + return version; + } + } + + private static ObjectMapper createAndInitializeObjectMapper(boolean allowDuplicateProperties) { + ObjectMapper objectMapper = new ObjectMapper(); + objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + objectMapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true); + objectMapper.configure(JsonParser.Feature.ALLOW_TRAILING_COMMA, true); + if (!allowDuplicateProperties) { + objectMapper.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, true); + } + objectMapper.configure(DeserializationFeature.ACCEPT_FLOAT_AS_INT, false); + + + // We will not register after burner for java 16+, due to its breaking changes + // https://github.com/Azure/azure-sdk-for-java/issues/23005 + if (JAVA_VERSION != -1 && JAVA_VERSION < 16) { + objectMapper.registerModule(new AfterburnerModule()); + } + + objectMapper.registerModule(new JavaTimeModule()); + + return objectMapper; + } + + public static ObjectMapper getSimpleObjectMapper() { + return KafkaCosmosUtils.simpleObjectMapper; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/Strings.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/Strings.java new file mode 100644 index 0000000000000..4182330f22100 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/Strings.java @@ -0,0 +1,133 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation; + +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; + +/** + * While this class is public, but it is not part of our published public APIs. + * This is meant to be internally used only by our sdk. + */ +public class Strings { + public static final String Emtpy = ""; + + private final static String UTF8_CHARSET = StandardCharsets.UTF_8.name(); + + public static boolean isNullOrWhiteSpace(String str) { + return StringUtils.isEmpty(str) || StringUtils.isWhitespace(str); + } + public static boolean isNullOrEmpty(String str) { + return StringUtils.isEmpty(str); + } + + public static String toString(boolean value) { + return Boolean.toString(value); + } + + public static String toString(int value) { + return Integer.toString(value); + } + + public static boolean areEqual(String str1, String str2) { + return StringUtils.equals(str1, str2); + } + + public static boolean areEqualIgnoreCase(String str1, String str2) { + return StringUtils.equalsIgnoreCase(str1, str2); + } + + public static boolean containsIgnoreCase(String str1, String str2) { + return StringUtils.containsIgnoreCase(str1, str2); + } + + public static String fromCamelCaseToUpperCase(String str) { + if (str == null) { + return null; + } + + StringBuilder result = new StringBuilder(str); + + int i = 1; + while (i < result.length()) { + if (Character.isUpperCase(result.charAt(i))) { + result.insert(i, '_'); + i += 2; + } else { + result.replace(i, i + 1, Character.toString(Character.toUpperCase(result.charAt(i)))); + i ++; + } + } + + return result.toString(); + } + + public static String encodeURIComponent(String text) { + String result; + try { + result = URLEncoder.encode(text, UTF8_CHARSET); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException(e); + } + + // after URLEncoding - the following transformations need to be applied + // to get to encodeUriComponent consistent behavior + // "+" -> "%20" + // "%21" -> "!" + // "%27" -> "'" + // "%28" -> "(" + // "%29" -> ")" + // "%7E" -> "~" + + final int len = result.length(); + final StringBuilder buf = new StringBuilder( + result.length() + 4); // leaving enough buffer for two '+' replacements + // without having to allocate new buffer + for (int i = 0; i < len; i++) { + char currentChar = result.charAt(i); + + if (currentChar == '+') { + buf.append("%20"); + } else if (currentChar == '%' && i < len - 2) { + char nextChar = result.charAt(i + 1); + char secondToNextChar = result.charAt(i + 2); + if (nextChar == '7' && secondToNextChar == 'E') { + i += 2; + buf.append('~'); + } else if (nextChar == '2') { + switch (secondToNextChar) { + case '1': + buf.append('!'); + i += 2; + break; + case '7': + buf.append('\''); + i += 2; + break; + case '8': + buf.append('('); + i += 2; + break; + case '9': + buf.append(')'); + i += 2; + break; + default: + buf.append(currentChar); + } + } else { + buf.append(currentChar); + } + } else { + buf.append(currentChar); + } + } + + return buf.toString(); + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/ArrayUtils.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/ArrayUtils.java new file mode 100644 index 0000000000000..c86d3c28f8d8c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/ArrayUtils.java @@ -0,0 +1,185 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang; + +import java.lang.reflect.Array; + +public class ArrayUtils { + /** + * An empty immutable {@code String} array. + */ + public static final String[] EMPTY_STRING_ARRAY = new String[0]; + + /** + * The index value when an element is not found in a list or array: {@code -1}. + * This value is returned by methods in this class and can also be used in comparisons with values returned by + * various method from {@link java.util.List}. + */ + public static final int INDEX_NOT_FOUND = -1; + + private ArrayUtils() { + super(); + } + + /** + *

Returns the length of the specified array. + * This method can deal with {@code Object} arrays and with primitive arrays. + * + *

If the input array is {@code null}, {@code 0} is returned. + * + * @param array the array to retrieve the length from, may be null + * @return The length of the array, or {@code 0} if the array is {@code null} + * @throws IllegalArgumentException if the object argument is not an array. + */ + public static int getLength(final Object array) { + if (array == null) { + return 0; + } + return Array.getLength(array); + } + + /** + *

Finds the index of the given object in the array. + * + *

This method returns {@link #INDEX_NOT_FOUND} ({@code -1}) for a {@code null} input array. + * + * @param array the array to search through for the object, may be {@code null} + * @param objectToFind the object to find, may be {@code null} + * @return the index of the object within the array, + * {@link #INDEX_NOT_FOUND} ({@code -1}) if not found or {@code null} array input + */ + public static int indexOf(final Object[] array, final Object objectToFind) { + return indexOf(array, objectToFind, 0); + } + + /** + *

Finds the index of the given object in the array starting at the given index. + * + *

This method returns {@link #INDEX_NOT_FOUND} ({@code -1}) for a {@code null} input array. + * + *

A negative startIndex is treated as zero. A startIndex larger than the array + * length will return {@link #INDEX_NOT_FOUND} ({@code -1}). + * + * @param array the array to search through for the object, may be {@code null} + * @param objectToFind the object to find, may be {@code null} + * @param startIndex the index to start searching at + * @return the index of the object within the array starting at the index, + * {@link #INDEX_NOT_FOUND} ({@code -1}) if not found or {@code null} array input + */ + public static int indexOf(final Object[] array, final Object objectToFind, int startIndex) { + if (array == null) { + return INDEX_NOT_FOUND; + } + if (startIndex < 0) { + startIndex = 0; + } + if (objectToFind == null) { + for (int i = startIndex; i < array.length; i++) { + if (array[i] == null) { + return i; + } + } + } else { + for (int i = startIndex; i < array.length; i++) { + if (objectToFind.equals(array[i])) { + return i; + } + } + } + return INDEX_NOT_FOUND; + } + + /** + *

Checks if the object is in the given array. + * + *

The method returns {@code false} if a {@code null} array is passed in. + * + * @param array the array to search through + * @param objectToFind the object to find + * @return {@code true} if the array contains the object + */ + public static boolean contains(final Object[] array, final Object objectToFind) { + return indexOf(array, objectToFind) != INDEX_NOT_FOUND; + } + + // ---------------------------------------------------------------------- + /** + *

Checks if an array of Objects is empty or {@code null}. + * + * @param array the array to test + * @return {@code true} if the array is empty or {@code null} + */ + public static boolean isEmpty(final Object[] array) { + return getLength(array) == 0; + } + + /** + *

Checks if an array of primitive chars is empty or {@code null}. + * + * @param array the array to test + * @return {@code true} if the array is empty or {@code null} + */ + public static boolean isEmpty(final char[] array) { + return getLength(array) == 0; + } + + /** + *

Copies the given array and adds the given element at the end of the new array. + * + * @param the component type of the array + * @param array the array to "add" the element to, may be {@code null} + * @param element the object to add, may be {@code null} + * @return A new array containing the existing elements plus the new element + * @throws IllegalArgumentException if both arguments are null + */ + public static T[] add(final T[] array, final T element) { + Class type; + if (array != null) { + type = array.getClass().getComponentType(); + } else if (element != null) { + type = element.getClass(); + } else { + throw new IllegalArgumentException("Arguments cannot both be null"); + } + @SuppressWarnings("unchecked") // type must be T + final + T[] newArray = (T[]) copyArrayGrow1(array, type); + newArray[newArray.length - 1] = element; + return newArray; + } + + /** + * Returns a copy of the given array of size 1 greater than the argument. + * The last value of the array is left to the default value. + * + * @param array The array to copy, must not be {@code null}. + * @param newArrayComponentType If {@code array} is {@code null}, create a + * size 1 array of this type. + * @return A new copy of the array of size 1 greater than the input. + */ + private static Object copyArrayGrow1(final Object array, final Class newArrayComponentType) { + if (array != null) { + final int arrayLength = Array.getLength(array); + final Object newArray = Array.newInstance(array.getClass().getComponentType(), arrayLength + 1); + System.arraycopy(array, 0, newArray, 0, arrayLength); + return newArray; + } + return Array.newInstance(newArrayComponentType, 1); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/CharSequenceUtils.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/CharSequenceUtils.java new file mode 100644 index 0000000000000..bfb7d12f1c1c4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/CharSequenceUtils.java @@ -0,0 +1,82 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang; + +public class CharSequenceUtils { + + private CharSequenceUtils() { + super(); + } + + /** + * Green implementation of regionMatches. + * + * @param cs the {@code CharSequence} to be processed + * @param ignoreCase whether or not to be case insensitive + * @param thisStart the index to start on the {@code cs} CharSequence + * @param substring the {@code CharSequence} to be looked for + * @param start the index to start on the {@code substring} CharSequence + * @param length character length of the region + * @return whether the region matched + */ + static boolean regionMatches(final CharSequence cs, final boolean ignoreCase, final int thisStart, + final CharSequence substring, final int start, final int length) { + if (cs instanceof String && substring instanceof String) { + return ((String) cs).regionMatches(ignoreCase, thisStart, (String) substring, start, length); + } + int index1 = thisStart; + int index2 = start; + int tmpLen = length; + + // Extract these first so we detect NPEs the same as the java.lang.String version + final int srcLen = cs.length() - thisStart; + final int otherLen = substring.length() - start; + + // Check for invalid parameters + if (thisStart < 0 || start < 0 || length < 0) { + return false; + } + + // Check that the regions are long enough + if (srcLen < length || otherLen < length) { + return false; + } + + while (tmpLen-- > 0) { + final char c1 = cs.charAt(index1++); + final char c2 = substring.charAt(index2++); + + if (c1 == c2) { + continue; + } + + if (!ignoreCase) { + return false; + } + + // The same check as in String.regionMatches(): + if (Character.toUpperCase(c1) != Character.toUpperCase(c2) + && Character.toLowerCase(c1) != Character.toLowerCase(c2)) { + return false; + } + } + return true; + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/StringUtils.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/StringUtils.java new file mode 100644 index 0000000000000..a0d60110432f9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/StringUtils.java @@ -0,0 +1,1428 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +public class StringUtils { + + private static final int STRING_BUILDER_SIZE = 256; + + // Performance testing notes (JDK 1.4, Jul03, scolebourne) + // Whitespace: + // Character.isWhitespace() is faster than WHITESPACE.indexOf() + // where WHITESPACE is a string of all whitespace characters + // + // Character access: + // String.charAt(n) versus toCharArray(), then array[n] + // String.charAt(n) is about 15% worse for a 10K string + // They are about equal for a length 50 string + // String.charAt(n) is about 4 times better for a length 3 string + // String.charAt(n) is best bet overall + // + // Append: + // String.concat about twice as fast as StringBuffer.append + // (not sure who tested this) + + /** + * A String for a space character. + */ + public static final String SPACE = " "; + + /** + * The empty String {@code ""}. + */ + public static final String EMPTY = ""; + + /** + * Represents a failed index search. + */ + public static final int INDEX_NOT_FOUND = -1; + + /** + *

The maximum size to which the padding constant(s) can expand.

+ */ + private static final int PAD_LIMIT = 8192; + + /** + *

{@code StringUtils} instances should NOT be constructed in + * standard programming. Instead, the class should be used as + * {@code StringUtils.trim(" foo ");}.

+ * + *

This constructor is public to permit tools that require a JavaBean + * instance to operate.

+ */ + public StringUtils() { + super(); + } + + // Empty checks + //----------------------------------------------------------------------- + /** + *

Checks if a CharSequence is empty ("") or null.

+ * + * @param cs the CharSequence to check, may be null + * @return {@code true} if the CharSequence is empty or null + */ + public static boolean isEmpty(final CharSequence cs) { + return cs == null || cs.length() == 0; + } + + /** + *

Checks if a CharSequence is not empty ("") and not null.

+ * + * @param cs the CharSequence to check, may be null + * @return {@code true} if the CharSequence is not empty and not null + */ + public static boolean isNotEmpty(final CharSequence cs) { + return !isEmpty(cs); + } + + /** + *

Removes control characters (char <= 32) from both + * ends of this String, handling {@code null} by returning + * {@code null}.

+ * + * @param str the String to be trimmed, may be null + * @return the trimmed string, {@code null} if null String input + */ + public static String trim(final String str) { + return str == null ? null : str.trim(); + } + + /** + *

Removes control characters (char <= 32) from both + * ends of this String returning {@code null} if the String is + * empty ("") after the trim or if it is {@code null}. + * + * @param str the String to be trimmed, may be null + * @return the trimmed String, + * {@code null} if only chars <= 32, empty or null String input + */ + public static String trimToNull(final String str) { + final String ts = trim(str); + return isEmpty(ts) ? null : ts; + } + + /** + *

Strips any of a set of characters from the start and end of a String. + * This is similar to {@link String#trim()} but allows the characters + * to be stripped to be controlled.

+ * + * @param str the String to remove characters from, may be null + * @param stripChars the characters to remove, null treated as whitespace + * @return the stripped String, {@code null} if null String input + */ + public static String strip(String str, final String stripChars) { + if (isEmpty(str)) { + return str; + } + str = stripStart(str, stripChars); + return stripEnd(str, stripChars); + } + + /** + *

Strips any of a set of characters from the start of a String.

+ * + * @param str the String to remove characters from, may be null + * @param stripChars the characters to remove, null treated as whitespace + * @return the stripped String, {@code null} if null String input + */ + public static String stripStart(final String str, final String stripChars) { + int strLen; + if (str == null || (strLen = str.length()) == 0) { + return str; + } + int start = 0; + if (stripChars == null) { + while (start != strLen && Character.isWhitespace(str.charAt(start))) { + start++; + } + } else if (stripChars.isEmpty()) { + return str; + } else { + while (start != strLen && stripChars.indexOf(str.charAt(start)) != INDEX_NOT_FOUND) { + start++; + } + } + return str.substring(start); + } + + /** + *

Strips any of a set of characters from the end of a String.

+ * + * + * @param str the String to remove characters from, may be null + * @param stripChars the set of characters to remove, null treated as whitespace + * @return the stripped String, {@code null} if null String input + */ + public static String stripEnd(final String str, final String stripChars) { + int end; + if (str == null || (end = str.length()) == 0) { + return str; + } + + if (stripChars == null) { + while (end != 0 && Character.isWhitespace(str.charAt(end - 1))) { + end--; + } + } else if (stripChars.isEmpty()) { + return str; + } else { + while (end != 0 && stripChars.indexOf(str.charAt(end - 1)) != INDEX_NOT_FOUND) { + end--; + } + } + return str.substring(0, end); + } + + /** + *

Compares two CharSequences, returning {@code true} if they represent + * equal sequences of characters.

+ * + * + * @see Object#equals(Object) + * @param cs1 the first CharSequence, may be {@code null} + * @param cs2 the second CharSequence, may be {@code null} + * @return {@code true} if the CharSequences are equal (case-sensitive), or both {@code null} + */ + public static boolean equals(final CharSequence cs1, final CharSequence cs2) { + if (cs1 == cs2) { + return true; + } + if (cs1 == null || cs2 == null) { + return false; + } + if (cs1.length() != cs2.length()) { + return false; + } + if (cs1 instanceof String && cs2 instanceof String) { + return cs1.equals(cs2); + } + return CharSequenceUtils.regionMatches(cs1, false, 0, cs2, 0, cs1.length()); + } + + /** + *

Compares two CharSequences, returning {@code true} if they represent + * equal sequences of characters, ignoring case.

+ * + *

{@code null}s are handled without exceptions. Two {@code null} + * references are considered equal. Comparison is case insensitive.

+ * + *
+     * StringUtils.equalsIgnoreCase(null, null)   = true
+     * StringUtils.equalsIgnoreCase(null, "abc")  = false
+     * StringUtils.equalsIgnoreCase("abc", null)  = false
+     * StringUtils.equalsIgnoreCase("abc", "abc") = true
+     * StringUtils.equalsIgnoreCase("abc", "ABC") = true
+     * 
+ * + * @param str1 the first CharSequence, may be null + * @param str2 the second CharSequence, may be null + * @return {@code true} if the CharSequence are equal, case insensitive, or + * both {@code null} + */ + public static boolean equalsIgnoreCase(final CharSequence str1, final CharSequence str2) { + if (str1 == null || str2 == null) { + return str1 == str2; + } else if (str1 == str2) { + return true; + } else if (str1.length() != str2.length()) { + return false; + } else { + return CharSequenceUtils.regionMatches(str1, true, 0, str2, 0, str1.length()); + } + } + + /** + *

Compare two Strings lexicographically, as per {@link String#compareTo(String)}, returning :

+ *
    + *
  • {@code int = 0}, if {@code str1} is equal to {@code str2} (or both {@code null})
  • + *
  • {@code int < 0}, if {@code str1} is less than {@code str2}
  • + *
  • {@code int > 0}, if {@code str1} is greater than {@code str2}
  • + *
+ * + *

This is a {@code null} safe version of :

+ *
str1.compareTo(str2)
+ * + *

{@code null} value is considered less than non-{@code null} value. + * Two {@code null} references are considered equal.

+ * + *
+     * StringUtils.compare(null, null)   = 0
+     * StringUtils.compare(null , "a")   < 0
+     * StringUtils.compare("a", null)    > 0
+     * StringUtils.compare("abc", "abc") = 0
+     * StringUtils.compare("a", "b")     < 0
+     * StringUtils.compare("b", "a")     > 0
+     * StringUtils.compare("a", "B")     > 0
+     * StringUtils.compare("ab", "abc")  < 0
+     * 
+ * + * @see #compare(String, String, boolean) + * @see String#compareTo(String) + * @param str1 the String to compare from + * @param str2 the String to compare to + * @return < 0, 0, > 0, if {@code str1} is respectively less, equal or greater than {@code str2} + */ + public static int compare(final String str1, final String str2) { + return compare(str1, str2, true); + } + + /** + *

Compare two Strings lexicographically, as per {@link String#compareTo(String)}, returning :

+ *
    + *
  • {@code int = 0}, if {@code str1} is equal to {@code str2} (or both {@code null})
  • + *
  • {@code int < 0}, if {@code str1} is less than {@code str2}
  • + *
  • {@code int > 0}, if {@code str1} is greater than {@code str2}
  • + *
+ * + *

This is a {@code null} safe version of :

+ *
str1.compareTo(str2)
+ * + *

{@code null} inputs are handled according to the {@code nullIsLess} parameter. + * Two {@code null} references are considered equal.

+ * + *
+     * StringUtils.compare(null, null, *)     = 0
+     * StringUtils.compare(null , "a", true)  < 0
+     * StringUtils.compare(null , "a", false) > 0
+     * StringUtils.compare("a", null, true)   > 0
+     * StringUtils.compare("a", null, false)  < 0
+     * StringUtils.compare("abc", "abc", *)   = 0
+     * StringUtils.compare("a", "b", *)       < 0
+     * StringUtils.compare("b", "a", *)       > 0
+     * StringUtils.compare("a", "B", *)       > 0
+     * StringUtils.compare("ab", "abc", *)    < 0
+     * 
+ * + * @see String#compareTo(String) + * @param str1 the String to compare from + * @param str2 the String to compare to + * @param nullIsLess whether consider {@code null} value less than non-{@code null} value + * @return < 0, 0, > 0, if {@code str1} is respectively less, equal ou greater than {@code str2} + */ + public static int compare(final String str1, final String str2, final boolean nullIsLess) { + if (str1 == str2) { + return 0; + } + if (str1 == null) { + return nullIsLess ? -1 : 1; + } + if (str2 == null) { + return nullIsLess ? 1 : - 1; + } + return str1.compareTo(str2); + } + + /** + *

Checks if CharSequence contains a search CharSequence irrespective of case, + * handling {@code null}. Case-insensitivity is defined as by + * {@link String#equalsIgnoreCase(String)}. + * + *

A {@code null} CharSequence will return {@code false}.

+ * + *
+     * StringUtils.containsIgnoreCase(null, *) = false
+     * StringUtils.containsIgnoreCase(*, null) = false
+     * StringUtils.containsIgnoreCase("", "") = true
+     * StringUtils.containsIgnoreCase("abc", "") = true
+     * StringUtils.containsIgnoreCase("abc", "a") = true
+     * StringUtils.containsIgnoreCase("abc", "z") = false
+     * StringUtils.containsIgnoreCase("abc", "A") = true
+     * StringUtils.containsIgnoreCase("abc", "Z") = false
+     * 
+ * + * @param str the CharSequence to check, may be null + * @param searchStr the CharSequence to find, may be null + * @return true if the CharSequence contains the search CharSequence irrespective of + * case or false if not or {@code null} string input + */ + public static boolean containsIgnoreCase(final CharSequence str, final CharSequence searchStr) { + if (str == null || searchStr == null) { + return false; + } + final int len = searchStr.length(); + final int max = str.length() - len; + for (int i = 0; i <= max; i++) { + if (CharSequenceUtils.regionMatches(str, true, i, searchStr, 0, len)) { + return true; + } + } + return false; + } + + /** + *

Checks if the CharSequence contains any character in the given + * set of characters.

+ * + *

A {@code null} CharSequence will return {@code false}. + * A {@code null} or zero length search array will return {@code false}.

+ * + *
+     * StringUtils.containsAny(null, *)                = false
+     * StringUtils.containsAny("", *)                  = false
+     * StringUtils.containsAny(*, null)                = false
+     * StringUtils.containsAny(*, [])                  = false
+     * StringUtils.containsAny("zzabyycdxx",['z','a']) = true
+     * StringUtils.containsAny("zzabyycdxx",['b','y']) = true
+     * StringUtils.containsAny("zzabyycdxx",['z','y']) = true
+     * StringUtils.containsAny("aba", ['z'])           = false
+     * 
+ * + * @param cs the CharSequence to check, may be null + * @param searchChars the chars to search for, may be null + * @return the {@code true} if any of the chars are found, + * {@code false} if no match or null input + */ + public static boolean containsAny(final CharSequence cs, final char... searchChars) { + if (isEmpty(cs) || ArrayUtils.isEmpty(searchChars)) { + return false; + } + final int csLength = cs.length(); + final int searchLength = searchChars.length; + final int csLast = csLength - 1; + final int searchLast = searchLength - 1; + for (int i = 0; i < csLength; i++) { + final char ch = cs.charAt(i); + for (int j = 0; j < searchLength; j++) { + if (searchChars[j] == ch) { + if (Character.isHighSurrogate(ch)) { + if (j == searchLast) { + // missing low surrogate, fine, like String.indexOf(String) + return true; + } + if (i < csLast && searchChars[j + 1] == cs.charAt(i + 1)) { + return true; + } + } else { + // ch is in the Basic Multilingual Plane + return true; + } + } + } + } + return false; + } + + /** + *

Checks that the CharSequence does not contain certain characters.

+ * + *

A {@code null} CharSequence will return {@code true}. + * A {@code null} invalid character array will return {@code true}. + * An empty CharSequence (length()=0) always returns true.

+ * + *
+     * StringUtils.containsNone(null, *)       = true
+     * StringUtils.containsNone(*, null)       = true
+     * StringUtils.containsNone("", *)         = true
+     * StringUtils.containsNone("ab", '')      = true
+     * StringUtils.containsNone("abab", 'xyz') = true
+     * StringUtils.containsNone("ab1", 'xyz')  = true
+     * StringUtils.containsNone("abz", 'xyz')  = false
+     * 
+ * + * @param cs the CharSequence to check, may be null + * @param searchChars an array of invalid chars, may be null + * @return true if it contains none of the invalid chars, or is null + */ + public static boolean containsNone(final CharSequence cs, final char... searchChars) { + if (cs == null || searchChars == null) { + return true; + } + final int csLen = cs.length(); + final int csLast = csLen - 1; + final int searchLen = searchChars.length; + final int searchLast = searchLen - 1; + for (int i = 0; i < csLen; i++) { + final char ch = cs.charAt(i); + for (int j = 0; j < searchLen; j++) { + if (searchChars[j] == ch) { + if (Character.isHighSurrogate(ch)) { + if (j == searchLast) { + // missing low surrogate, fine, like String.indexOf(String) + return false; + } + if (i < csLast && searchChars[j + 1] == cs.charAt(i + 1)) { + return false; + } + } else { + // ch is in the Basic Multilingual Plane + return false; + } + } + } + } + return true; + } + + /** + *

Gets a substring from the specified String avoiding exceptions.

+ * + *

A negative start position can be used to start {@code n} + * characters from the end of the String.

+ * + *

A {@code null} String will return {@code null}. + * An empty ("") String will return "".

+ * + *
+     * StringUtils.substring(null, *)   = null
+     * StringUtils.substring("", *)     = ""
+     * StringUtils.substring("abc", 0)  = "abc"
+     * StringUtils.substring("abc", 2)  = "c"
+     * StringUtils.substring("abc", 4)  = ""
+     * StringUtils.substring("abc", -2) = "bc"
+     * StringUtils.substring("abc", -4) = "abc"
+     * 
+ * + * @param str the String to get the substring from, may be null + * @param start the position to start from, negative means + * count back from the end of the String by this many characters + * @return substring from start position, {@code null} if null String input + */ + public static String substring(final String str, int start) { + if (str == null) { + return null; + } + + // handle negatives, which means last n characters + if (start < 0) { + start = str.length() + start; // remember start is negative + } + + if (start < 0) { + start = 0; + } + if (start > str.length()) { + return EMPTY; + } + + return str.substring(start); + } + + private static StringBuilder newStringBuilder(final int noOfItems) { + return new StringBuilder(noOfItems * 16); + } + + /** + *

Splits the provided text into an array, using whitespace as the + * separator. + * Whitespace is defined by {@link Character#isWhitespace(char)}.

+ * + *

The separator is not included in the returned String array. + * Adjacent separators are treated as one separator. + * For more control over the split use the StrTokenizer class.

+ * + *

A {@code null} input String returns {@code null}.

+ * + *
+     * StringUtils.split(null)       = null
+     * StringUtils.split("")         = []
+     * StringUtils.split("abc def")  = ["abc", "def"]
+     * StringUtils.split("abc  def") = ["abc", "def"]
+     * StringUtils.split(" abc ")    = ["abc"]
+     * 
+ * + * @param str the String to parse, may be null + * @return an array of parsed Strings, {@code null} if null String input + */ + public static String[] split(final String str) { + return split(str, null, -1); + } + + /** + *

Splits the provided text into an array, separator specified. + * This is an alternative to using StringTokenizer.

+ * + *

The separator is not included in the returned String array. + * Adjacent separators are treated as one separator. + * For more control over the split use the StrTokenizer class.

+ * + *

A {@code null} input String returns {@code null}.

+ * + *
+     * StringUtils.split(null, *)         = null
+     * StringUtils.split("", *)           = []
+     * StringUtils.split("a.b.c", '.')    = ["a", "b", "c"]
+     * StringUtils.split("a..b.c", '.')   = ["a", "b", "c"]
+     * StringUtils.split("a:b:c", '.')    = ["a:b:c"]
+     * StringUtils.split("a b c", ' ')    = ["a", "b", "c"]
+     * 
+ * + * @param str the String to parse, may be null + * @param separatorChar the character used as the delimiter + * @return an array of parsed Strings, {@code null} if null String input + */ + public static String[] split(final String str, final char separatorChar) { + return splitWorker(str, separatorChar, false); + } + + /** + *

Splits the provided text into an array, separators specified. + * This is an alternative to using StringTokenizer.

+ * + *

The separator is not included in the returned String array. + * Adjacent separators are treated as one separator. + * For more control over the split use the StrTokenizer class.

+ * + *

A {@code null} input String returns {@code null}. + * A {@code null} separatorChars splits on whitespace.

+ * + *
+     * StringUtils.split(null, *)         = null
+     * StringUtils.split("", *)           = []
+     * StringUtils.split("abc def", null) = ["abc", "def"]
+     * StringUtils.split("abc def", " ")  = ["abc", "def"]
+     * StringUtils.split("abc  def", " ") = ["abc", "def"]
+     * StringUtils.split("ab:cd:ef", ":") = ["ab", "cd", "ef"]
+     * 
+ * + * @param str the String to parse, may be null + * @param separatorChars the characters used as the delimiters, + * {@code null} splits on whitespace + * @return an array of parsed Strings, {@code null} if null String input + */ + public static String[] split(final String str, final String separatorChars) { + return splitWorker(str, separatorChars, -1, false); + } + + /** + *

Splits the provided text into an array with a maximum length, + * separators specified.

+ * + *

The separator is not included in the returned String array. + * Adjacent separators are treated as one separator.

+ * + *

A {@code null} input String returns {@code null}. + * A {@code null} separatorChars splits on whitespace.

+ * + *

If more than {@code max} delimited substrings are found, the last + * returned string includes all characters after the first {@code max - 1} + * returned strings (including separator characters).

+ * + *
+     * StringUtils.split(null, *, *)            = null
+     * StringUtils.split("", *, *)              = []
+     * StringUtils.split("ab cd ef", null, 0)   = ["ab", "cd", "ef"]
+     * StringUtils.split("ab   cd ef", null, 0) = ["ab", "cd", "ef"]
+     * StringUtils.split("ab:cd:ef", ":", 0)    = ["ab", "cd", "ef"]
+     * StringUtils.split("ab:cd:ef", ":", 2)    = ["ab", "cd:ef"]
+     * 
+ * + * @param str the String to parse, may be null + * @param separatorChars the characters used as the delimiters, + * {@code null} splits on whitespace + * @param max the maximum number of elements to include in the + * array. A zero or negative value implies no limit + * @return an array of parsed Strings, {@code null} if null String input + */ + public static String[] split(final String str, final String separatorChars, final int max) { + return splitWorker(str, separatorChars, max, false); + } + + /** + * Performs the logic for the {@code split} and + * {@code splitPreserveAllTokens} methods that do not return a + * maximum array length. + * + * @param str the String to parse, may be {@code null} + * @param separatorChar the separate character + * @param preserveAllTokens if {@code true}, adjacent separators are + * treated as empty token separators; if {@code false}, adjacent + * separators are treated as one separator. + * @return an array of parsed Strings, {@code null} if null String input + */ + private static String[] splitWorker(final String str, final char separatorChar, final boolean preserveAllTokens) { + // Performance tuned for 2.0 (JDK1.4) + + if (str == null) { + return null; + } + final int len = str.length(); + if (len == 0) { + return ArrayUtils.EMPTY_STRING_ARRAY; + } + final List list = new ArrayList<>(); + int i = 0, start = 0; + boolean match = false; + boolean lastMatch = false; + while (i < len) { + if (str.charAt(i) == separatorChar) { + if (match || preserveAllTokens) { + list.add(str.substring(start, i)); + match = false; + lastMatch = true; + } + start = ++i; + continue; + } + lastMatch = false; + match = true; + i++; + } + if (match || preserveAllTokens && lastMatch) { + list.add(str.substring(start, i)); + } + return list.toArray(new String[list.size()]); + } + + /** + * Performs the logic for the {@code split} and + * {@code splitPreserveAllTokens} methods that return a maximum array + * length. + * + * @param str the String to parse, may be {@code null} + * @param separatorChars the separate character + * @param max the maximum number of elements to include in the + * array. A zero or negative value implies no limit. + * @param preserveAllTokens if {@code true}, adjacent separators are + * treated as empty token separators; if {@code false}, adjacent + * separators are treated as one separator. + * @return an array of parsed Strings, {@code null} if null String input + */ + private static String[] splitWorker(final String str, final String separatorChars, final int max, final boolean preserveAllTokens) { + // Performance tuned for 2.0 (JDK1.4) + // Direct code is quicker than StringTokenizer. + // Also, StringTokenizer uses isSpace() not isWhitespace() + + if (str == null) { + return null; + } + final int len = str.length(); + if (len == 0) { + return ArrayUtils.EMPTY_STRING_ARRAY; + } + final List list = new ArrayList<>(); + int sizePlus1 = 1; + int i = 0, start = 0; + boolean match = false; + boolean lastMatch = false; + if (separatorChars == null) { + // Null separator means use whitespace + while (i < len) { + if (Character.isWhitespace(str.charAt(i))) { + if (match || preserveAllTokens) { + lastMatch = true; + if (sizePlus1++ == max) { + i = len; + lastMatch = false; + } + list.add(str.substring(start, i)); + match = false; + } + start = ++i; + continue; + } + lastMatch = false; + match = true; + i++; + } + } else if (separatorChars.length() == 1) { + // Optimise 1 character case + final char sep = separatorChars.charAt(0); + while (i < len) { + if (str.charAt(i) == sep) { + if (match || preserveAllTokens) { + lastMatch = true; + if (sizePlus1++ == max) { + i = len; + lastMatch = false; + } + list.add(str.substring(start, i)); + match = false; + } + start = ++i; + continue; + } + lastMatch = false; + match = true; + i++; + } + } else { + // standard case + while (i < len) { + if (separatorChars.indexOf(str.charAt(i)) >= 0) { + if (match || preserveAllTokens) { + lastMatch = true; + if (sizePlus1++ == max) { + i = len; + lastMatch = false; + } + list.add(str.substring(start, i)); + match = false; + } + start = ++i; + continue; + } + lastMatch = false; + match = true; + i++; + } + } + if (match || preserveAllTokens && lastMatch) { + list.add(str.substring(start, i)); + } + return list.toArray(new String[list.size()]); + } + + /** + *

Joins the elements of the provided array into a single String + * containing the provided list of elements.

+ * + *

No delimiter is added before or after the list. + * A {@code null} separator is the same as an empty String (""). + * Null objects or empty strings within the array are represented by + * empty strings.

+ * + *
+     * StringUtils.join(null, *)                = null
+     * StringUtils.join([], *)                  = ""
+     * StringUtils.join([null], *)              = ""
+     * StringUtils.join(["a", "b", "c"], "--")  = "a--b--c"
+     * StringUtils.join(["a", "b", "c"], null)  = "abc"
+     * StringUtils.join(["a", "b", "c"], "")    = "abc"
+     * StringUtils.join([null, "", "a"], ',')   = ",,a"
+     * 
+ * + * @param array the array of values to join together, may be null + * @param separator the separator character to use, null treated as "" + * @return the joined String, {@code null} if null array input + */ + public static String join(final Object[] array, final String separator) { + if (array == null) { + return null; + } + return join(array, separator, 0, array.length); + } + + /** + *

Joins the elements of the provided array into a single String + * containing the provided list of elements.

+ * + *

No delimiter is added before or after the list. + * A {@code null} separator is the same as an empty String (""). + * Null objects or empty strings within the array are represented by + * empty strings.

+ * + *
+     * StringUtils.join(null, *, *, *)                = null
+     * StringUtils.join([], *, *, *)                  = ""
+     * StringUtils.join([null], *, *, *)              = ""
+     * StringUtils.join(["a", "b", "c"], "--", 0, 3)  = "a--b--c"
+     * StringUtils.join(["a", "b", "c"], "--", 1, 3)  = "b--c"
+     * StringUtils.join(["a", "b", "c"], "--", 2, 3)  = "c"
+     * StringUtils.join(["a", "b", "c"], "--", 2, 2)  = ""
+     * StringUtils.join(["a", "b", "c"], null, 0, 3)  = "abc"
+     * StringUtils.join(["a", "b", "c"], "", 0, 3)    = "abc"
+     * StringUtils.join([null, "", "a"], ',', 0, 3)   = ",,a"
+     * 
+ * + * @param array the array of values to join together, may be null + * @param separator the separator character to use, null treated as "" + * @param startIndex the first index to start joining from. + * @param endIndex the index to stop joining from (exclusive). + * @return the joined String, {@code null} if null array input; or the empty string + * if {@code endIndex - startIndex <= 0}. The number of joined entries is given by + * {@code endIndex - startIndex} + * @throws ArrayIndexOutOfBoundsException ife
+ * {@code startIndex < 0} or
+ * {@code startIndex >= array.length()} or
+ * {@code endIndex < 0} or
+ * {@code endIndex > array.length()} + */ + public static String join(final Object[] array, String separator, final int startIndex, final int endIndex) { + if (array == null) { + return null; + } + if (separator == null) { + separator = EMPTY; + } + + // endIndex - startIndex > 0: Len = NofStrings *(len(firstString) + len(separator)) + // (Assuming that all Strings are roughly equally long) + final int noOfItems = endIndex - startIndex; + if (noOfItems <= 0) { + return EMPTY; + } + + final StringBuilder buf = newStringBuilder(noOfItems); + + for (int i = startIndex; i < endIndex; i++) { + if (i > startIndex) { + buf.append(separator); + } + if (array[i] != null) { + buf.append(array[i]); + } + } + return buf.toString(); + } + + /** + *

Joins the elements of the provided {@code Iterator} into + * a single String containing the provided elements.

+ * + *

No delimiter is added before or after the list. + * A {@code null} separator is the same as an empty String ("").

+ * + *

See the examples here: {@link #join(Object[],String)}.

+ * + * @param iterator the {@code Iterator} of values to join together, may be null + * @param separator the separator character to use, null treated as "" + * @return the joined String, {@code null} if null iterator input + */ + public static String join(final Iterator iterator, final String separator) { + + // handle null, zero and one elements before building a buffer + if (iterator == null) { + return null; + } + if (!iterator.hasNext()) { + return EMPTY; + } + final Object first = iterator.next(); + if (!iterator.hasNext()) { + return Objects.toString(first, ""); + } + + // two or more elements + final StringBuilder buf = new StringBuilder(STRING_BUILDER_SIZE); // Java default is 16, probably too small + if (first != null) { + buf.append(first); + } + + while (iterator.hasNext()) { + if (separator != null) { + buf.append(separator); + } + final Object obj = iterator.next(); + if (obj != null) { + buf.append(obj); + } + } + return buf.toString(); + } + + /** + *

Joins the elements of the provided {@code Iterable} into + * a single String containing the provided elements.

+ * + *

No delimiter is added before or after the list. + * A {@code null} separator is the same as an empty String ("").

+ * + *

See the examples here: {@link #join(Object[],String)}.

+ * + * @param iterable the {@code Iterable} providing the values to join together, may be null + * @param separator the separator character to use, null treated as "" + * @return the joined String, {@code null} if null iterator input + */ + public static String join(final Iterable iterable, final String separator) { + if (iterable == null) { + return null; + } + return join(iterable.iterator(), separator); + } + + /** + *

Removes a substring only if it is at the beginning of a source string, + * otherwise returns the source string.

+ * + *

A {@code null} source string will return {@code null}. + * An empty ("") source string will return the empty string. + * A {@code null} search string will return the source string.

+ * + *
+     * StringUtils.removeStart(null, *)      = null
+     * StringUtils.removeStart("", *)        = ""
+     * StringUtils.removeStart(*, null)      = *
+     * StringUtils.removeStart("www.domain.com", "www.")   = "domain.com"
+     * StringUtils.removeStart("domain.com", "www.")       = "domain.com"
+     * StringUtils.removeStart("www.domain.com", "domain") = "www.domain.com"
+     * StringUtils.removeStart("abc", "")    = "abc"
+     * 
+ * + * @param str the source String to search, may be null + * @param remove the String to search for and remove, may be null + * @return the substring with the string removed if found, + * {@code null} if null String input + */ + public static String removeStart(final String str, final String remove) { + if (isEmpty(str) || isEmpty(remove)) { + return str; + } + if (str.startsWith(remove)){ + return str.substring(remove.length()); + } + return str; + } + + /** + *

Removes a substring only if it is at the end of a source string, + * otherwise returns the source string.

+ * + *

A {@code null} source string will return {@code null}. + * An empty ("") source string will return the empty string. + * A {@code null} search string will return the source string.

+ * + *
+     * StringUtils.removeEnd(null, *)      = null
+     * StringUtils.removeEnd("", *)        = ""
+     * StringUtils.removeEnd(*, null)      = *
+     * StringUtils.removeEnd("www.domain.com", ".com.")  = "www.domain.com"
+     * StringUtils.removeEnd("www.domain.com", ".com")   = "www.domain"
+     * StringUtils.removeEnd("www.domain.com", "domain") = "www.domain.com"
+     * StringUtils.removeEnd("abc", "")    = "abc"
+     * 
+ * + * @param str the source String to search, may be null + * @param remove the String to search for and remove, may be null + * @return the substring with the string removed if found, + * {@code null} if null String input + */ + public static String removeEnd(final String str, final String remove) { + if (isEmpty(str) || isEmpty(remove)) { + return str; + } + if (str.endsWith(remove)) { + return str.substring(0, str.length() - remove.length()); + } + return str; + } + + /** + *

Replaces all occurrences of a String within another String.

+ * + *

A {@code null} reference passed to this method is a no-op.

+ * + *
+     * StringUtils.replace(null, *, *)        = null
+     * StringUtils.replace("", *, *)          = ""
+     * StringUtils.replace("any", null, *)    = "any"
+     * StringUtils.replace("any", *, null)    = "any"
+     * StringUtils.replace("any", "", *)      = "any"
+     * StringUtils.replace("aba", "a", null)  = "aba"
+     * StringUtils.replace("aba", "a", "")    = "b"
+     * StringUtils.replace("aba", "a", "z")   = "zbz"
+     * 
+ * + * @see #replace(String text, String searchString, String replacement, int max) + * @param text text to search and replace in, may be null + * @param searchString the String to search for, may be null + * @param replacement the String to replace it with, may be null + * @return the text with any replacements processed, + * {@code null} if null String input + */ + public static String replace(final String text, final String searchString, final String replacement) { + return replace(text, searchString, replacement, -1); + } + + /** + *

Replaces a String with another String inside a larger String, + * for the first {@code max} values of the search String.

+ * + *

A {@code null} reference passed to this method is a no-op.

+ * + *
+     * StringUtils.replace(null, *, *, *)         = null
+     * StringUtils.replace("", *, *, *)           = ""
+     * StringUtils.replace("any", null, *, *)     = "any"
+     * StringUtils.replace("any", *, null, *)     = "any"
+     * StringUtils.replace("any", "", *, *)       = "any"
+     * StringUtils.replace("any", *, *, 0)        = "any"
+     * StringUtils.replace("abaa", "a", null, -1) = "abaa"
+     * StringUtils.replace("abaa", "a", "", -1)   = "b"
+     * StringUtils.replace("abaa", "a", "z", 0)   = "abaa"
+     * StringUtils.replace("abaa", "a", "z", 1)   = "zbaa"
+     * StringUtils.replace("abaa", "a", "z", 2)   = "zbza"
+     * StringUtils.replace("abaa", "a", "z", -1)  = "zbzz"
+     * 
+ * + * @param text text to search and replace in, may be null + * @param searchString the String to search for, may be null + * @param replacement the String to replace it with, may be null + * @param max maximum number of values to replace, or {@code -1} if no maximum + * @return the text with any replacements processed, + * {@code null} if null String input + */ + public static String replace(final String text, final String searchString, final String replacement, final int max) { + return replace(text, searchString, replacement, max, false); + } + + /** + *

Replaces a String with another String inside a larger String, + * for the first {@code max} values of the search String, + * case sensitively/insensisitively based on {@code ignoreCase} value.

+ * + *

A {@code null} reference passed to this method is a no-op.

+ * + *
+     * StringUtils.replace(null, *, *, *, false)         = null
+     * StringUtils.replace("", *, *, *, false)           = ""
+     * StringUtils.replace("any", null, *, *, false)     = "any"
+     * StringUtils.replace("any", *, null, *, false)     = "any"
+     * StringUtils.replace("any", "", *, *, false)       = "any"
+     * StringUtils.replace("any", *, *, 0, false)        = "any"
+     * StringUtils.replace("abaa", "a", null, -1, false) = "abaa"
+     * StringUtils.replace("abaa", "a", "", -1, false)   = "b"
+     * StringUtils.replace("abaa", "a", "z", 0, false)   = "abaa"
+     * StringUtils.replace("abaa", "A", "z", 1, false)   = "abaa"
+     * StringUtils.replace("abaa", "A", "z", 1, true)   = "zbaa"
+     * StringUtils.replace("abAa", "a", "z", 2, true)   = "zbza"
+     * StringUtils.replace("abAa", "a", "z", -1, true)  = "zbzz"
+     * 
+ * + * @param text text to search and replace in, may be null + * @param searchString the String to search for (case insensitive), may be null + * @param replacement the String to replace it with, may be null + * @param max maximum number of values to replace, or {@code -1} if no maximum + * @param ignoreCase if true replace is case insensitive, otherwise case sensitive + * @return the text with any replacements processed, + * {@code null} if null String input + */ + private static String replace(final String text, String searchString, final String replacement, int max, final boolean ignoreCase) { + if (isEmpty(text) || isEmpty(searchString) || replacement == null || max == 0) { + return text; + } + String searchText = text; + if (ignoreCase) { + searchText = text.toLowerCase(); + searchString = searchString.toLowerCase(); + } + int start = 0; + int end = searchText.indexOf(searchString, start); + if (end == INDEX_NOT_FOUND) { + return text; + } + final int replLength = searchString.length(); + int increase = replacement.length() - replLength; + increase = increase < 0 ? 0 : increase; + increase *= max < 0 ? 16 : max > 64 ? 64 : max; + final StringBuilder buf = new StringBuilder(text.length() + increase); + while (end != INDEX_NOT_FOUND) { + buf.append(text, start, end).append(replacement); + start = end + replLength; + if (--max == 0) { + break; + } + end = searchText.indexOf(searchString, start); + } + buf.append(text, start, text.length()); + return buf.toString(); + } + + /** + *

Repeat a String {@code repeat} times to form a + * new String.

+ * + *
+     * StringUtils.repeat(null, 2) = null
+     * StringUtils.repeat("", 0)   = ""
+     * StringUtils.repeat("", 2)   = ""
+     * StringUtils.repeat("a", 3)  = "aaa"
+     * StringUtils.repeat("ab", 2) = "abab"
+     * StringUtils.repeat("a", -2) = ""
+     * 
+ * + * @param str the String to repeat, may be null + * @param repeat number of times to repeat str, negative treated as zero + * @return a new String consisting of the original String repeated, + * {@code null} if null String input + */ + public static String repeat(final String str, final int repeat) { + // Performance tuned for 2.0 (JDK1.4) + + if (str == null) { + return null; + } + if (repeat <= 0) { + return EMPTY; + } + final int inputLength = str.length(); + if (repeat == 1 || inputLength == 0) { + return str; + } + if (inputLength == 1 && repeat <= PAD_LIMIT) { + return repeat(str.charAt(0), repeat); + } + + final int outputLength = inputLength * repeat; + switch (inputLength) { + case 1 : + return repeat(str.charAt(0), repeat); + case 2 : + final char ch0 = str.charAt(0); + final char ch1 = str.charAt(1); + final char[] output2 = new char[outputLength]; + for (int i = repeat * 2 - 2; i >= 0; i--, i--) { + output2[i] = ch0; + output2[i + 1] = ch1; + } + return new String(output2); + default : + final StringBuilder buf = new StringBuilder(outputLength); + for (int i = 0; i < repeat; i++) { + buf.append(str); + } + return buf.toString(); + } + } + + /** + *

Returns padding using the specified delimiter repeated + * to a given length.

+ * + *
+     * StringUtils.repeat('e', 0)  = ""
+     * StringUtils.repeat('e', 3)  = "eee"
+     * StringUtils.repeat('e', -2) = ""
+     * 
+ * + *

Note: this method does not support padding with + * Unicode Supplementary Characters + * as they require a pair of {@code char}s to be represented. + * If you are needing to support full I18N of your applications + * consider using {@link #repeat(String, int)} instead. + *

+ * + * @param ch character to repeat + * @param repeat number of times to repeat char, negative treated as zero + * @return String with repeated character + * @see #repeat(String, int) + */ + public static String repeat(final char ch, final int repeat) { + if (repeat <= 0) { + return EMPTY; + } + final char[] buf = new char[repeat]; + for (int i = repeat - 1; i >= 0; i--) { + buf[i] = ch; + } + return new String(buf); + } + + /** + *

Left pad a String with a specified character.

+ * + *

Pad to a size of {@code size}.

+ * + *
+     * StringUtils.leftPad(null, *, *)     = null
+     * StringUtils.leftPad("", 3, 'z')     = "zzz"
+     * StringUtils.leftPad("bat", 3, 'z')  = "bat"
+     * StringUtils.leftPad("bat", 5, 'z')  = "zzbat"
+     * StringUtils.leftPad("bat", 1, 'z')  = "bat"
+     * StringUtils.leftPad("bat", -1, 'z') = "bat"
+     * 
+ * + * @param str the String to pad out, may be null + * @param size the size to pad to + * @param padChar the character to pad with + * @return left padded String or original String if no padding is necessary, + * {@code null} if null String input + */ + public static String leftPad(final String str, final int size, final char padChar) { + if (str == null) { + return null; + } + final int pads = size - str.length(); + if (pads <= 0) { + return str; // returns original String when possible + } + if (pads > PAD_LIMIT) { + return leftPad(str, size, String.valueOf(padChar)); + } + return repeat(padChar, pads).concat(str); + } + + /** + *

Left pad a String with a specified String.

+ * + *

Pad to a size of {@code size}.

+ * + *
+     * StringUtils.leftPad(null, *, *)      = null
+     * StringUtils.leftPad("", 3, "z")      = "zzz"
+     * StringUtils.leftPad("bat", 3, "yz")  = "bat"
+     * StringUtils.leftPad("bat", 5, "yz")  = "yzbat"
+     * StringUtils.leftPad("bat", 8, "yz")  = "yzyzybat"
+     * StringUtils.leftPad("bat", 1, "yz")  = "bat"
+     * StringUtils.leftPad("bat", -1, "yz") = "bat"
+     * StringUtils.leftPad("bat", 5, null)  = "  bat"
+     * StringUtils.leftPad("bat", 5, "")    = "  bat"
+     * 
+ * + * @param str the String to pad out, may be null + * @param size the size to pad to + * @param padStr the String to pad with, null or empty treated as single space + * @return left padded String or original String if no padding is necessary, + * {@code null} if null String input + */ + public static String leftPad(final String str, final int size, String padStr) { + if (str == null) { + return null; + } + if (isEmpty(padStr)) { + padStr = SPACE; + } + final int padLen = padStr.length(); + final int strLen = str.length(); + final int pads = size - strLen; + if (pads <= 0) { + return str; // returns original String when possible + } + if (padLen == 1 && pads <= PAD_LIMIT) { + return leftPad(str, size, padStr.charAt(0)); + } + + if (pads == padLen) { + return padStr.concat(str); + } else if (pads < padLen) { + return padStr.substring(0, pads).concat(str); + } else { + final char[] padding = new char[pads]; + final char[] padChars = padStr.toCharArray(); + for (int i = 0; i < pads; i++) { + padding[i] = padChars[i % padLen]; + } + return new String(padding).concat(str); + } + } + + /** + *

Converts a String to upper case as per {@link String#toUpperCase()}.

+ * + *

A {@code null} input String returns {@code null}.

+ * + * @param str the String to upper case, may be null + * @return the upper cased String, {@code null} if null String input + */ + public static String upperCase(final String str) { + if (str == null) { + return null; + } + return str.toUpperCase(); + } + + /** + *

Checks if the CharSequence contains only Unicode digits. + * A decimal point is not a Unicode digit and returns false.

+ * + *

{@code null} will return {@code false}. + * An empty CharSequence (length()=0) will return {@code false}.

+ * + *

Note that the method does not allow for a leading sign, either positive or negative. + * Also, if a String passes the numeric test, it may still generate a NumberFormatException + * when parsed by Integer.parseInt or Long.parseLong, e.g. if the value is outside the range + * for int or long respectively.

+ * + * + * @param cs the CharSequence to check, may be null + * @return {@code true} if only contains digits, and is non-null + */ + public static boolean isNumeric(final CharSequence cs) { + if (isEmpty(cs)) { + return false; + } + final int sz = cs.length(); + for (int i = 0; i < sz; i++) { + if (!Character.isDigit(cs.charAt(i))) { + return false; + } + } + return true; + } + + /** + *

Checks if the CharSequence contains only whitespace.

+ * + * @param cs the CharSequence to check, may be null + * @return {@code true} if only contains whitespace, and is non-null + */ + public static boolean isWhitespace(final CharSequence cs) { + if (cs == null) { + return false; + } + final int sz = cs.length(); + for (int i = 0; i < sz; i++) { + if (!Character.isWhitespace(cs.charAt(i))) { + return false; + } + } + return true; + } + + /** + *

Returns either the passed in String, or if the String is + * {@code null}, the value of {@code defaultStr}.

+ * + *
+     * StringUtils.defaultString(null, "NULL")  = "NULL"
+     * StringUtils.defaultString("", "NULL")    = ""
+     * StringUtils.defaultString("bat", "NULL") = "bat"
+     * 
+ * + * @see String#valueOf(Object) + * @param str the String to check, may be null + * @param defaultStr the default String to return + * if the input is {@code null}, may be null + * @return the passed in String, or the default if it was {@code null} + */ + public static String defaultString(final String str, final String defaultStr) { + return str == null ? defaultStr : str; + } + + /** + * Deletes all whitespaces from a String as defined by + * {@link Character#isWhitespace(char)}. + * + *
+     * StringUtils.deleteWhitespace(null)         = null
+     * StringUtils.deleteWhitespace("")           = ""
+     * StringUtils.deleteWhitespace("abc")        = "abc"
+     * StringUtils.deleteWhitespace("   ab  c  ") = "abc"
+     * 
+ * + * @param str the String to delete whitespace from, may be null + * @return the String without whitespaces, {@code null} if null String input + */ + public static String deleteWhitespace(final String str) { + if (isEmpty(str)) { + return str; + } + final int sz = str.length(); + final char[] chs = new char[sz]; + int count = 0; + for (int i = 0; i < sz; i++) { + if (!Character.isWhitespace(str.charAt(i))) { + chs[count++] = str.charAt(i); + } + } + if (count == sz) { + return str; + } + if (count == 0) { + return EMPTY; + } + return new String(chs, 0, count); + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/Validate.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/Validate.java new file mode 100644 index 0000000000000..1deabae346994 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/Validate.java @@ -0,0 +1,60 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang; + +public class Validate { + private Validate() { + super(); + } + + /** + *

Validate that the argument condition is {@code true}; otherwise + * throwing an exception with the specified message. + * + * @param expression the boolean expression to check + * @param message the exception message if invalid, not null + * @param values the optional values for the formatted exception message, null array not recommended + * @throws IllegalArgumentException if expression is {@code false} + */ + public static void isTrue(final boolean expression, final String message, final Object... values) { + if (!expression) { + throw new IllegalArgumentException(String.format(message, values)); + } + } + + /** + * Validate that the specified primitive value falls between the two + * inclusive values specified; otherwise, throws an exception with the + * specified message. + * + * @param start the inclusive start value + * @param end the inclusive end value + * @param value the value to validate + * @param message the exception message if invalid, not null + * + * @throws IllegalArgumentException if the value falls outside the boundaries + * + */ + public static void inclusiveBetween(final long start, final long end, final long value, final String message) { + // TODO when breaking BC, consider returning value + if (value < start || value > end) { + throw new IllegalArgumentException(message); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/Builder.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/Builder.java new file mode 100644 index 0000000000000..d6b4517d3bfd0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/Builder.java @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.builder; + +public interface Builder { + /** + * Returns a reference to the object being constructed or result being + * calculated by the builder. + * + * @return the object constructed or result calculated by the builder. + */ + T build(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/CompareToBuilder.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/CompareToBuilder.java new file mode 100644 index 0000000000000..e49ae9dc3d3de --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/builder/CompareToBuilder.java @@ -0,0 +1,715 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.builder; + +import java.util.Comparator; + +public class CompareToBuilder implements Builder { + /** + * Current state of the comparison as appended fields are checked. + */ + private int comparison; + + /** + *

Constructor for CompareToBuilder.

+ * + *

Starts off assuming that the objects are equal. Multiple calls are + * then made to the various append methods, followed by a call to + * {@link #toComparison} to get the result.

+ */ + public CompareToBuilder() { + super(); + comparison = 0; + } + + //----------------------------------------------------------------------- + /** + *

Appends to the builder the comparison of + * two Objects.

+ * + *
    + *
  1. Check if lhs == rhs
  2. + *
  3. Check if either lhs or rhs is null, + * a null object is less than a non-null object
  4. + *
  5. Check the object contents
  6. + *
+ * + *

lhs must either be an array or implement {@link Comparable}.

+ * + * @param lhs left-hand object + * @param rhs right-hand object + * @return this - used to chain append calls + * @throws ClassCastException if rhs is not assignment-compatible + * with lhs + */ + public CompareToBuilder append(final Object lhs, final Object rhs) { + return append(lhs, rhs, null); + } + + /** + *

Appends to the builder the comparison of + * two Objects.

+ * + *
    + *
  1. Check if lhs == rhs
  2. + *
  3. Check if either lhs or rhs is null, + * a null object is less than a non-null object
  4. + *
  5. Check the object contents
  6. + *
+ * + *

If lhs is an array, array comparison methods will be used. + * Otherwise comparator will be used to compare the objects. + * If comparator is null, lhs must + * implement {@link Comparable} instead.

+ * + * @param lhs left-hand object + * @param rhs right-hand object + * @param comparator Comparator used to compare the objects, + * null means treat lhs as Comparable + * @return this - used to chain append calls + * @throws ClassCastException if rhs is not assignment-compatible + * with lhs + */ + public CompareToBuilder append(final Object lhs, final Object rhs, final Comparator comparator) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.getClass().isArray()) { + // factor out array case in order to keep method small enough to be inlined + appendArray(lhs, rhs, comparator); + } else { + // the simple case, not an array, just test the element + if (comparator == null) { + @SuppressWarnings("unchecked") // assume this can be done; if not throw CCE as per Javadoc + final Comparable comparable = (Comparable) lhs; + comparison = comparable.compareTo(rhs); + } else { + @SuppressWarnings("unchecked") // assume this can be done; if not throw CCE as per Javadoc + final Comparator comparator2 = (Comparator) comparator; + comparison = comparator2.compare(lhs, rhs); + } + } + return this; + } + + private void appendArray(final Object lhs, final Object rhs, final Comparator comparator) { + // switch on type of array, to dispatch to the correct handler + // handles multi dimensional arrays + // throws a ClassCastException if rhs is not the correct array type + if (lhs instanceof long[]) { + append((long[]) lhs, (long[]) rhs); + } else if (lhs instanceof int[]) { + append((int[]) lhs, (int[]) rhs); + } else if (lhs instanceof short[]) { + append((short[]) lhs, (short[]) rhs); + } else if (lhs instanceof char[]) { + append((char[]) lhs, (char[]) rhs); + } else if (lhs instanceof byte[]) { + append((byte[]) lhs, (byte[]) rhs); + } else if (lhs instanceof double[]) { + append((double[]) lhs, (double[]) rhs); + } else if (lhs instanceof float[]) { + append((float[]) lhs, (float[]) rhs); + } else if (lhs instanceof boolean[]) { + append((boolean[]) lhs, (boolean[]) rhs); + } else { + // not an array of primitives + // throws a ClassCastException if rhs is not an array + append((Object[]) lhs, (Object[]) rhs, comparator); + } + } + + //------------------------------------------------------------------------- + /** + * Appends to the builder the comparison of + * two longs. + * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final long lhs, final long rhs) { + if (comparison != 0) { + return this; + } + comparison = Long.compare(lhs, rhs); + return this; + } + + /** + * Appends to the builder the comparison of + * two ints. + * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final int lhs, final int rhs) { + if (comparison != 0) { + return this; + } + comparison = Integer.compare(lhs, rhs); + return this; + } + + /** + * Appends to the builder the comparison of + * two shorts. + * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final short lhs, final short rhs) { + if (comparison != 0) { + return this; + } + comparison = Short.compare(lhs, rhs); + return this; + } + + /** + * Appends to the builder the comparison of + * two chars. + * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final char lhs, final char rhs) { + if (comparison != 0) { + return this; + } + comparison = Character.compare(lhs, rhs); + return this; + } + + /** + * Appends to the builder the comparison of + * two bytes. + * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final byte lhs, final byte rhs) { + if (comparison != 0) { + return this; + } + comparison = Byte.compare(lhs, rhs); + return this; + } + + /** + *

Appends to the builder the comparison of + * two doubles.

+ * + *

This handles NaNs, Infinities, and -0.0.

+ * + *

It is compatible with the hash code generated by + * HashCodeBuilder.

+ * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final double lhs, final double rhs) { + if (comparison != 0) { + return this; + } + comparison = Double.compare(lhs, rhs); + return this; + } + + /** + *

Appends to the builder the comparison of + * two floats.

+ * + *

This handles NaNs, Infinities, and -0.0.

+ * + *

It is compatible with the hash code generated by + * HashCodeBuilder.

+ * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final float lhs, final float rhs) { + if (comparison != 0) { + return this; + } + comparison = Float.compare(lhs, rhs); + return this; + } + + /** + * Appends to the builder the comparison of + * two booleanss. + * + * @param lhs left-hand value + * @param rhs right-hand value + * @return this - used to chain append calls + */ + public CompareToBuilder append(final boolean lhs, final boolean rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs) { + comparison = 1; + } else { + comparison = -1; + } + return this; + } + + //----------------------------------------------------------------------- + /** + *

Appends to the builder the deep comparison of + * two Object arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a short length array is less than a long length array
  6. + *
  7. Check array contents element by element using {@link #append(Object, Object, Comparator)}
  8. + *
+ * + *

This method will also will be called for the top level of multi-dimensional, + * ragged, and multi-typed arrays.

+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + * @throws ClassCastException if rhs is not assignment-compatible + * with lhs + */ + public CompareToBuilder append(final Object[] lhs, final Object[] rhs) { + return append(lhs, rhs, null); + } + + /** + *

Appends to the builder the deep comparison of + * two Object arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a short length array is less than a long length array
  6. + *
  7. Check array contents element by element using {@link #append(Object, Object, Comparator)}
  8. + *
+ * + *

This method will also will be called for the top level of multi-dimensional, + * ragged, and multi-typed arrays.

+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @param comparator Comparator to use to compare the array elements, + * null means to treat lhs elements as Comparable. + * @return this - used to chain append calls + * @throws ClassCastException if rhs is not assignment-compatible + * with lhs + */ + public CompareToBuilder append(final Object[] lhs, final Object[] rhs, final Comparator comparator) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i], comparator); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two long arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(long, long)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final long[] lhs, final long[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two int arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(int, int)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final int[] lhs, final int[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two short arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(short, short)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final short[] lhs, final short[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two char arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(char, char)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final char[] lhs, final char[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two byte arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(byte, byte)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final byte[] lhs, final byte[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two double arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(double, double)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final double[] lhs, final double[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two float arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(float, float)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final float[] lhs, final float[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + *

Appends to the builder the deep comparison of + * two boolean arrays.

+ * + *
    + *
  1. Check if arrays are the same using ==
  2. + *
  3. Check if for null, null is less than non-null
  4. + *
  5. Check array length, a shorter length array is less than a longer length array
  6. + *
  7. Check array contents element by element using {@link #append(boolean, boolean)}
  8. + *
+ * + * @param lhs left-hand array + * @param rhs right-hand array + * @return this - used to chain append calls + */ + public CompareToBuilder append(final boolean[] lhs, final boolean[] rhs) { + if (comparison != 0) { + return this; + } + if (lhs == rhs) { + return this; + } + if (lhs == null) { + comparison = -1; + return this; + } + if (rhs == null) { + comparison = 1; + return this; + } + if (lhs.length != rhs.length) { + comparison = lhs.length < rhs.length ? -1 : 1; + return this; + } + for (int i = 0; i < lhs.length && comparison == 0; i++) { + append(lhs[i], rhs[i]); + } + return this; + } + + /** + * Returns a negative integer, a positive integer, or zero as + * the builder has judged the "left-hand" side + * as less than, greater than, or equal to the "right-hand" + * side. + * + * @return final comparison result + * @see #build() + */ + public int toComparison() { + return comparison; + } + + /** + * Returns a negative Integer, a positive Integer, or zero as + * the builder has judged the "left-hand" side + * as less than, greater than, or equal to the "right-hand" + * side. + * + * @return final comparison result as an Integer + * @see #toComparison() + */ + @Override + public Integer build() { + return Integer.valueOf(toComparison()); + } +} + + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/ImmutablePair.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/ImmutablePair.java new file mode 100644 index 0000000000000..520e756298565 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/ImmutablePair.java @@ -0,0 +1,108 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.tuple; + +public final class ImmutablePair extends Pair { + /** + * An immutable pair of nulls. + */ + // This is not defined with generics to avoid warnings in call sites. + @SuppressWarnings("rawtypes") + private static final ImmutablePair NULL = of(null, null); + /** Serialization version */ + private static final long serialVersionUID = -8635883027105195960L; + + + /** + * Returns an immutable pair of nulls. + * + * @param the left element of this pair. Value is {@code null}. + * @param the right element of this pair. Value is {@code null}. + * @return an immutable pair of nulls. + */ + @SuppressWarnings("unchecked") + public static ImmutablePair nullPair() { + return NULL; + } + + /** Left object */ + public final L left; + /** Right object */ + public final R right; + + /** + *

Obtains an immutable pair of two objects inferring the generic types.

+ * + *

This factory allows the pair to be created using inference to + * obtain the generic types.

+ * + * @param the left element type + * @param the right element type + * @param left the left element, may be null + * @param right the right element, may be null + * @return a pair formed from the two parameters, not null + */ + public static ImmutablePair of(final L left, final R right) { + return new ImmutablePair<>(left, right); + } + + /** + * Create a new pair instance. + * + * @param left the left value, may be null + * @param right the right value, may be null + */ + public ImmutablePair(final L left, final R right) { + super(); + this.left = left; + this.right = right; + } + + //----------------------------------------------------------------------- + /** + * {@inheritDoc} + */ + @Override + public L getLeft() { + return left; + } + + /** + * {@inheritDoc} + */ + @Override + public R getRight() { + return right; + } + + /** + *

Throws {@code UnsupportedOperationException}.

+ * + *

This pair is immutable, so this operation is not supported.

+ * + * @param value the value to set + * @return never + * @throws UnsupportedOperationException as this operation is not supported + */ + @Override + public R setValue(final R value) { + throw new UnsupportedOperationException(); + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/Pair.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/Pair.java new file mode 100644 index 0000000000000..0442f6542a607 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/apachecommons/lang/tuple/Pair.java @@ -0,0 +1,163 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.tuple; + +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.builder.CompareToBuilder; + +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; + +public abstract class Pair implements Map.Entry, Comparable>, Serializable { + /** Serialization version */ + private static final long serialVersionUID = -8251365186387295956L; + + /** + *

Obtains an immutable pair of two objects inferring the generic types.

+ * + *

This factory allows the pair to be created using inference to + * obtain the generic types.

+ * + * @param the left element type + * @param the right element type + * @param left the left element, may be null + * @param right the right element, may be null + * @return a pair formed from the two parameters, not null + */ + public static Pair of(final L left, final R right) { + return new ImmutablePair<>(left, right); + } + + //----------------------------------------------------------------------- + /** + *

Gets the left element from this pair.

+ * + *

When treated as a key-value pair, this is the key.

+ * + * @return the left element, may be null + */ + public abstract L getLeft(); + + /** + *

Gets the right element from this pair.

+ * + *

When treated as a key-value pair, this is the value.

+ * + * @return the right element, may be null + */ + public abstract R getRight(); + + /** + *

Gets the key from this pair.

+ * + *

This method implements the {@code Map.Entry} interface returning the + * left element as the key.

+ * + * @return the left element as the key, may be null + */ + @Override + public final L getKey() { + return getLeft(); + } + + /** + *

Gets the value from this pair.

+ * + *

This method implements the {@code Map.Entry} interface returning the + * right element as the value.

+ * + * @return the right element as the value, may be null + */ + @Override + public R getValue() { + return getRight(); + } + + //----------------------------------------------------------------------- + /** + *

Compares the pair based on the left element followed by the right element. + * The types must be {@code Comparable}.

+ * + * @param other the other pair, not null + * @return negative if this is less, zero if equal, positive if greater + */ + @Override + public int compareTo(final Pair other) { + return new CompareToBuilder().append(getLeft(), other.getLeft()) + .append(getRight(), other.getRight()).toComparison(); + } + + /** + *

Compares this pair to another based on the two elements.

+ * + * @param obj the object to compare to, null returns false + * @return true if the elements of the pair are equal + */ + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof Map.Entry) { + final Map.Entry other = (Map.Entry) obj; + return Objects.equals(getKey(), other.getKey()) + && Objects.equals(getValue(), other.getValue()); + } + return false; + } + + /** + *

Returns a suitable hash code. + * The hash code follows the definition in {@code Map.Entry}.

+ * + * @return the hash code + */ + @Override + public int hashCode() { + // see Map.Entry API specification + return (getKey() == null ? 0 : getKey().hashCode()) ^ + (getValue() == null ? 0 : getValue().hashCode()); + } + + /** + *

Returns a String representation of this pair using the format {@code ($left,$right)}.

+ * + * @return a string describing this object, not null + */ + @Override + public String toString() { + return "(" + getLeft() + ',' + getRight() + ')'; + } + + /** + *

Formats the receiver using the given format.

+ * + *

This uses {@link java.util.Formattable} to perform the formatting. Two variables may + * be used to embed the left and right elements. Use {@code %1$s} for the left + * element (key) and {@code %2$s} for the right element (value). + * The default format used by {@code toString()} is {@code (%1$s,%2$s)}.

+ * + * @param format the format string, optionally containing {@code %1$s} and {@code %2$s}, not null + * @return the formatted string, not null + */ + public String toString(final String format) { + return String.format(format, getLeft(), getRight()); + } +} + diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Absent.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Absent.java new file mode 100644 index 0000000000000..828fe2ed40806 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Absent.java @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.Collections; +import java.util.Set; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** Implementation of an {@link Optional} not containing a reference. */ +final class Absent extends Optional { + static final Absent INSTANCE = new Absent<>(); + + @SuppressWarnings("unchecked") // implementation is "fully variant" + static Optional withType() { + return (Optional) INSTANCE; + } + + private Absent() {} + + @Override + public boolean isPresent() { + return false; + } + + @Override + public T get() { + throw new IllegalStateException("Optional.get() cannot be called on an absent value"); + } + + @Override + public T or(T defaultValue) { + return checkNotNull(defaultValue, "use Optional.orNull() instead of Optional.or(null)"); + } + + @SuppressWarnings("unchecked") // safe covariant cast + @Override + public Optional or(Optional secondChoice) { + return (Optional) checkNotNull(secondChoice); + } + + @Override + public T or(Supplier supplier) { + return checkNotNull( + supplier.get(), "use Optional.orNull() instead of a Supplier that returns null"); + } + + @Override + + public T orNull() { + return null; + } + + @Override + public Set asSet() { + return Collections.emptySet(); + } + + @Override + public Optional transform(Function function) { + checkNotNull(function); + return Optional.absent(); + } + + @Override + public boolean equals(Object object) { + return object == this; + } + + @Override + public int hashCode() { + return 0x79a31aac; + } + + @Override + public String toString() { + return "Optional.absent()"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/AbstractIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/AbstractIterator.java new file mode 100644 index 0000000000000..cdc23d595d7e4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/AbstractIterator.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; + + +/** + * Note this class is a copy of {@link AbstractIterator} (for dependency + * reasons). + */ +abstract class AbstractIterator implements Iterator { + private State state = State.NOT_READY; + + protected AbstractIterator() {} + + private enum State { + READY, + NOT_READY, + DONE, + FAILED, + } + + private T next; + + protected abstract T computeNext(); + + + protected final T endOfData() { + state = State.DONE; + return null; + } + + @Override + public final boolean hasNext() { + checkState(state != State.FAILED); + switch (state) { + case READY: + return true; + case DONE: + return false; + default: + } + return tryToComputeNext(); + } + + private boolean tryToComputeNext() { + state = State.FAILED; // temporary pessimism + next = computeNext(); + if (state != State.DONE) { + state = State.READY; + return true; + } + return false; + } + + @Override + public final T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + state = State.NOT_READY; + T result = next; + next = null; + return result; + } + + @Override + public final void remove() { + throw new UnsupportedOperationException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ascii.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ascii.java new file mode 100644 index 0000000000000..e22678fec920f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ascii.java @@ -0,0 +1,636 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +/** + * Static methods pertaining to ASCII characters (those in the range of values {@code 0x00} through + * {@code 0x7F}), and to strings containing such characters. + * + *

ASCII utilities also exist in other classes of this package: + * + *

    + * + *
  • {@link Charsets#US_ASCII} specifies the {@code Charset} of ASCII characters. + *
  • {@link CharMatcher#ascii} matches ASCII characters and provides text processing methods + * which operate only on the ASCII characters of a string. + *
+ * + * @author Catherine Berry + * @author Gregory Kick + * @since 7.0 + */ +public final class Ascii { + + private Ascii() {} + + /* The ASCII control characters, per RFC 20. */ + /** + * Null ('\0'): The all-zeros character which may serve to accomplish time fill and media fill. + * Normally used as a C string terminator. + * + *

Although RFC 20 names this as "Null", note that it is distinct from the C/C++ "NULL" + * pointer. + * + * @since 8.0 + */ + public static final byte NUL = 0; + + /** + * Start of Heading: A communication control character used at the beginning of a sequence of + * characters which constitute a machine-sensible address or routing information. Such a sequence + * is referred to as the "heading." An STX character has the effect of terminating a heading. + * + * @since 8.0 + */ + public static final byte SOH = 1; + + /** + * Start of Text: A communication control character which precedes a sequence of characters that + * is to be treated as an entity and entirely transmitted through to the ultimate destination. + * Such a sequence is referred to as "text." STX may be used to terminate a sequence of characters + * started by SOH. + * + * @since 8.0 + */ + public static final byte STX = 2; + + /** + * End of Text: A communication control character used to terminate a sequence of characters + * started with STX and transmitted as an entity. + * + * @since 8.0 + */ + public static final byte ETX = 3; + + /** + * End of Transmission: A communication control character used to indicate the conclusion of a + * transmission, which may have contained one or more texts and any associated headings. + * + * @since 8.0 + */ + public static final byte EOT = 4; + + /** + * Enquiry: A communication control character used in data communication systems as a request for + * a response from a remote station. It may be used as a "Who Are You" (WRU) to obtain + * identification, or may be used to obtain station status, or both. + * + * @since 8.0 + */ + public static final byte ENQ = 5; + + /** + * Acknowledge: A communication control character transmitted by a receiver as an affirmative + * response to a sender. + * + * @since 8.0 + */ + public static final byte ACK = 6; + + /** + * Bell ('\a'): A character for use when there is a need to call for human attention. It may + * control alarm or attention devices. + * + * @since 8.0 + */ + public static final byte BEL = 7; + + /** + * Backspace ('\b'): A format effector which controls the movement of the printing position one + * printing space backward on the same printing line. (Applicable also to display devices.) + * + * @since 8.0 + */ + public static final byte BS = 8; + + /** + * Horizontal Tabulation ('\t'): A format effector which controls the movement of the printing + * position to the next in a series of predetermined positions along the printing line. + * (Applicable also to display devices and the skip function on punched cards.) + * + * @since 8.0 + */ + public static final byte HT = 9; + + /** + * Line Feed ('\n'): A format effector which controls the movement of the printing position to the + * next printing line. (Applicable also to display devices.) Where appropriate, this character may + * have the meaning "New Line" (NL), a format effector which controls the movement of the printing + * point to the first printing position on the next printing line. Use of this convention requires + * agreement between sender and recipient of data. + * + * @since 8.0 + */ + public static final byte LF = 10; + + /** + * Alternate name for {@link #LF}. ({@code LF} is preferred.) + * + * @since 8.0 + */ + public static final byte NL = 10; + + /** + * Vertical Tabulation ('\v'): A format effector which controls the movement of the printing + * position to the next in a series of predetermined printing lines. (Applicable also to display + * devices.) + * + * @since 8.0 + */ + public static final byte VT = 11; + + /** + * Form Feed ('\f'): A format effector which controls the movement of the printing position to the + * first pre-determined printing line on the next form or page. (Applicable also to display + * devices.) + * + * @since 8.0 + */ + public static final byte FF = 12; + + /** + * Carriage Return ('\r'): A format effector which controls the movement of the printing position + * to the first printing position on the same printing line. (Applicable also to display devices.) + * + * @since 8.0 + */ + public static final byte CR = 13; + + /** + * Shift Out: A control character indicating that the code combinations which follow shall be + * interpreted as outside of the character set of the standard code table until a Shift In + * character is reached. + * + * @since 8.0 + */ + public static final byte SO = 14; + + /** + * Shift In: A control character indicating that the code combinations which follow shall be + * interpreted according to the standard code table. + * + * @since 8.0 + */ + public static final byte SI = 15; + + /** + * Data Link Escape: A communication control character which will change the meaning of a limited + * number of contiguously following characters. It is used exclusively to provide supplementary + * controls in data communication networks. + * + * @since 8.0 + */ + public static final byte DLE = 16; + + /** + * Device Control 1. Characters for the control of ancillary devices associated with data + * processing or telecommunication systems, more especially switching devices "on" or "off." (If a + * single "stop" control is required to interrupt or turn off ancillary devices, DC4 is the + * preferred assignment.) + * + * @since 8.0 + */ + public static final byte DC1 = 17; // aka XON + + /** + * Transmission On: Although originally defined as DC1, this ASCII control character is now better + * known as the XON code used for software flow control in serial communications. The main use is + * restarting the transmission after the communication has been stopped by the XOFF control code. + * + * @since 8.0 + */ + public static final byte XON = 17; // aka DC1 + + /** + * Device Control 2. Characters for the control of ancillary devices associated with data + * processing or telecommunication systems, more especially switching devices "on" or "off." (If a + * single "stop" control is required to interrupt or turn off ancillary devices, DC4 is the + * preferred assignment.) + * + * @since 8.0 + */ + public static final byte DC2 = 18; + + /** + * Device Control 3. Characters for the control of ancillary devices associated with data + * processing or telecommunication systems, more especially switching devices "on" or "off." (If a + * single "stop" control is required to interrupt or turn off ancillary devices, DC4 is the + * preferred assignment.) + * + * @since 8.0 + */ + public static final byte DC3 = 19; // aka XOFF + + /** + * Transmission off. See {@link #XON} for explanation. + * + * @since 8.0 + */ + public static final byte XOFF = 19; // aka DC3 + + /** + * Device Control 4. Characters for the control of ancillary devices associated with data + * processing or telecommunication systems, more especially switching devices "on" or "off." (If a + * single "stop" control is required to interrupt or turn off ancillary devices, DC4 is the + * preferred assignment.) + * + * @since 8.0 + */ + public static final byte DC4 = 20; + + /** + * Negative Acknowledge: A communication control character transmitted by a receiver as a negative + * response to the sender. + * + * @since 8.0 + */ + public static final byte NAK = 21; + + /** + * Synchronous Idle: A communication control character used by a synchronous transmission system + * in the absence of any other character to provide a signal from which synchronism may be + * achieved or retained. + * + * @since 8.0 + */ + public static final byte SYN = 22; + + /** + * End of Transmission Block: A communication control character used to indicate the end of a + * block of data for communication purposes. ETB is used for blocking data where the block + * structure is not necessarily related to the processing format. + * + * @since 8.0 + */ + public static final byte ETB = 23; + + /** + * Cancel: A control character used to indicate that the data with which it is sent is in error or + * is to be disregarded. + * + * @since 8.0 + */ + public static final byte CAN = 24; + + /** + * End of Medium: A control character associated with the sent data which may be used to identify + * the physical end of the medium, or the end of the used, or wanted, portion of information + * recorded on a medium. (The position of this character does not necessarily correspond to the + * physical end of the medium.) + * + * @since 8.0 + */ + public static final byte EM = 25; + + /** + * Substitute: A character that may be substituted for a character which is determined to be + * invalid or in error. + * + * @since 8.0 + */ + public static final byte SUB = 26; + + /** + * Escape: A control character intended to provide code extension (supplementary characters) in + * general information interchange. The Escape character itself is a prefix affecting the + * interpretation of a limited number of contiguously following characters. + * + * @since 8.0 + */ + public static final byte ESC = 27; + + /** + * File Separator: These four information separators may be used within data in optional fashion, + * except that their hierarchical relationship shall be: FS is the most inclusive, then GS, then + * RS, and US is least inclusive. (The content and length of a File, Group, Record, or Unit are + * not specified.) + * + * @since 8.0 + */ + public static final byte FS = 28; + + /** + * Group Separator: These four information separators may be used within data in optional fashion, + * except that their hierarchical relationship shall be: FS is the most inclusive, then GS, then + * RS, and US is least inclusive. (The content and length of a File, Group, Record, or Unit are + * not specified.) + * + * @since 8.0 + */ + public static final byte GS = 29; + + /** + * Record Separator: These four information separators may be used within data in optional + * fashion, except that their hierarchical relationship shall be: FS is the most inclusive, then + * GS, then RS, and US is least inclusive. (The content and length of a File, Group, Record, or + * Unit are not specified.) + * + * @since 8.0 + */ + public static final byte RS = 30; + + /** + * Unit Separator: These four information separators may be used within data in optional fashion, + * except that their hierarchical relationship shall be: FS is the most inclusive, then GS, then + * RS, and US is least inclusive. (The content and length of a File, Group, Record, or Unit are + * not specified.) + * + * @since 8.0 + */ + public static final byte US = 31; + + /** + * Space: A normally non-printing graphic character used to separate words. It is also a format + * effector which controls the movement of the printing position, one printing position forward. + * (Applicable also to display devices.) + * + * @since 8.0 + */ + public static final byte SP = 32; + + /** + * Alternate name for {@link #SP}. + * + * @since 8.0 + */ + public static final byte SPACE = 32; + + /** + * Delete: This character is used primarily to "erase" or "obliterate" erroneous or unwanted + * characters in perforated tape. + * + * @since 8.0 + */ + public static final byte DEL = 127; + + /** + * The minimum value of an ASCII character. + * + * @since 9.0 (was type {@code int} before 12.0) + */ + public static final char MIN = 0; + + /** + * The maximum value of an ASCII character. + * + * @since 9.0 (was type {@code int} before 12.0) + */ + public static final char MAX = 127; + + /** A bit mask which selects the bit encoding ASCII character case. */ + private static final char CASE_MASK = 0x20; + + /** + * Returns a copy of the input string in which all {@linkplain #isUpperCase(char) uppercase ASCII + * characters} have been converted to lowercase. All other characters are copied without + * modification. + */ + public static String toLowerCase(String string) { + int length = string.length(); + for (int i = 0; i < length; i++) { + if (isUpperCase(string.charAt(i))) { + char[] chars = string.toCharArray(); + for (; i < length; i++) { + char c = chars[i]; + if (isUpperCase(c)) { + chars[i] = (char) (c ^ CASE_MASK); + } + } + return String.valueOf(chars); + } + } + return string; + } + + /** + * Returns a copy of the input character sequence in which all {@linkplain #isUpperCase(char) + * uppercase ASCII characters} have been converted to lowercase. All other characters are copied + * without modification. + * + * @since 14.0 + */ + public static String toLowerCase(CharSequence chars) { + if (chars instanceof String) { + return toLowerCase((String) chars); + } + char[] newChars = new char[chars.length()]; + for (int i = 0; i < newChars.length; i++) { + newChars[i] = toLowerCase(chars.charAt(i)); + } + return String.valueOf(newChars); + } + + /** + * If the argument is an {@linkplain #isUpperCase(char) uppercase ASCII character} returns the + * lowercase equivalent. Otherwise returns the argument. + */ + public static char toLowerCase(char c) { + return isUpperCase(c) ? (char) (c ^ CASE_MASK) : c; + } + + /** + * Returns a copy of the input string in which all {@linkplain #isLowerCase(char) lowercase ASCII + * characters} have been converted to uppercase. All other characters are copied without + * modification. + */ + public static String toUpperCase(String string) { + int length = string.length(); + for (int i = 0; i < length; i++) { + if (isLowerCase(string.charAt(i))) { + char[] chars = string.toCharArray(); + for (; i < length; i++) { + char c = chars[i]; + if (isLowerCase(c)) { + chars[i] = (char) (c ^ CASE_MASK); + } + } + return String.valueOf(chars); + } + } + return string; + } + + /** + * Returns a copy of the input character sequence in which all {@linkplain #isLowerCase(char) + * lowercase ASCII characters} have been converted to uppercase. All other characters are copied + * without modification. + * + * @since 14.0 + */ + public static String toUpperCase(CharSequence chars) { + if (chars instanceof String) { + return toUpperCase((String) chars); + } + char[] newChars = new char[chars.length()]; + for (int i = 0; i < newChars.length; i++) { + newChars[i] = toUpperCase(chars.charAt(i)); + } + return String.valueOf(newChars); + } + + /** + * If the argument is a {@linkplain #isLowerCase(char) lowercase ASCII character} returns the + * uppercase equivalent. Otherwise returns the argument. + */ + public static char toUpperCase(char c) { + return isLowerCase(c) ? (char) (c ^ CASE_MASK) : c; + } + + /** + * Indicates whether {@code c} is one of the twenty-six lowercase ASCII alphabetic characters + * between {@code 'a'} and {@code 'z'} inclusive. All others (including non-ASCII characters) + * return {@code false}. + */ + public static boolean isLowerCase(char c) { + // Note: This was benchmarked against the alternate expression "(char)(c - 'a') < 26" (Nov '13) + // and found to perform at least as well, or better. + return (c >= 'a') && (c <= 'z'); + } + + /** + * Indicates whether {@code c} is one of the twenty-six uppercase ASCII alphabetic characters + * between {@code 'A'} and {@code 'Z'} inclusive. All others (including non-ASCII characters) + * return {@code false}. + */ + public static boolean isUpperCase(char c) { + return (c >= 'A') && (c <= 'Z'); + } + + /** + * Truncates the given character sequence to the given maximum length. If the length of the + * sequence is greater than {@code maxLength}, the returned string will be exactly {@code + * maxLength} chars in length and will end with the given {@code truncationIndicator}. Otherwise, + * the sequence will be returned as a string with no changes to the content. + * + *

Examples: + * + *

{@code
+   * Ascii.truncate("foobar", 7, "..."); // returns "foobar"
+   * Ascii.truncate("foobar", 5, "..."); // returns "fo..."
+   * }
+ * + *

Note: This method may work with certain non-ASCII text but is not safe for use + * with arbitrary Unicode text. It is mostly intended for use with text that is known to be safe + * for use with it (such as all-ASCII text) and for simple debugging text. When using this method, + * consider the following: + * + *

    + *
  • it may split surrogate pairs + *
  • it may split characters and combining characters + *
  • it does not consider word boundaries + *
  • if truncating for display to users, there are other considerations that must be taken + * into account + *
  • the appropriate truncation indicator may be locale-dependent + *
  • it is safe to use non-ASCII characters in the truncation indicator + *
+ * + * + * @throws IllegalArgumentException if {@code maxLength} is less than the length of {@code + * truncationIndicator} + * @since 16.0 + */ + public static String truncate(CharSequence seq, int maxLength, String truncationIndicator) { + checkNotNull(seq); + + // length to truncate the sequence to, not including the truncation indicator + int truncationLength = maxLength - truncationIndicator.length(); + + // in this worst case, this allows a maxLength equal to the length of the truncationIndicator, + // meaning that a string will be truncated to just the truncation indicator itself + checkArgument( + truncationLength >= 0, + "maxLength (%s) must be >= length of the truncation indicator (%s)", + maxLength, + truncationIndicator.length()); + + if (seq.length() <= maxLength) { + String string = seq.toString(); + if (string.length() <= maxLength) { + return string; + } + // if the length of the toString() result was > maxLength for some reason, truncate that + seq = string; + } + + return new StringBuilder(maxLength) + .append(seq, 0, truncationLength) + .append(truncationIndicator) + .toString(); + } + + /** + * Indicates whether the contents of the given character sequences {@code s1} and {@code s2} are + * equal, ignoring the case of any ASCII alphabetic characters between {@code 'a'} and {@code 'z'} + * or {@code 'A'} and {@code 'Z'} inclusive. + * + *

This method is significantly faster than {@link String#equalsIgnoreCase} and should be used + * in preference if at least one of the parameters is known to contain only ASCII characters. + * + *

Note however that this method does not always behave identically to expressions such as: + * + *

    + *
  • {@code string.toUpperCase().equals("UPPER CASE ASCII")} + *
  • {@code string.toLowerCase().equals("lower case ascii")} + *
+ * + *

due to case-folding of some non-ASCII characters (which does not occur in {@link + * String#equalsIgnoreCase}). However in almost all cases that ASCII strings are used, the author + * probably wanted the behavior provided by this method rather than the subtle and sometimes + * surprising behavior of {@code toUpperCase()} and {@code toLowerCase()}. + * + * @since 16.0 + */ + public static boolean equalsIgnoreCase(CharSequence s1, CharSequence s2) { + // Calling length() is the null pointer check (so do it before we can exit early). + int length = s1.length(); + if (s1 == s2) { + return true; + } + if (length != s2.length()) { + return false; + } + for (int i = 0; i < length; i++) { + char c1 = s1.charAt(i); + char c2 = s2.charAt(i); + if (c1 == c2) { + continue; + } + int alphaIndex = getAlphaIndex(c1); + // This was also benchmarked using '&' to avoid branching (but always evaluate the rhs), + // however this showed no obvious improvement. + if (alphaIndex < 26 && alphaIndex == getAlphaIndex(c2)) { + continue; + } + return false; + } + return true; + } + + /** + * Returns the non-negative index value of the alpha character {@code c}, regardless of case. Ie, + * 'a'/'A' returns 0 and 'z'/'Z' returns 25. Non-alpha characters return a value of 26 or greater. + */ + private static int getAlphaIndex(char c) { + // Fold upper-case ASCII to lower-case and make zero-indexed and unsigned (by casting to char). + return (char) ((c | CASE_MASK) - 'a'); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CaseFormat.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CaseFormat.java new file mode 100644 index 0000000000000..9dfc942662ca0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CaseFormat.java @@ -0,0 +1,211 @@ +/* + * Copyright (C) 2006 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * Utility class for converting between various ASCII case formats. Behavior is undefined for + * non-ASCII input. + * + * @author Mike Bostock + * @since 1.0 + */ +public enum CaseFormat { + /** Hyphenated variable naming convention, e.g., "lower-hyphen". */ + LOWER_HYPHEN(CharMatcher.is('-'), "-") { + @Override + String normalizeWord(String word) { + return Ascii.toLowerCase(word); + } + + @Override + String convert(CaseFormat format, String s) { + if (format == LOWER_UNDERSCORE) { + return s.replace('-', '_'); + } + if (format == UPPER_UNDERSCORE) { + return Ascii.toUpperCase(s.replace('-', '_')); + } + return super.convert(format, s); + } + }, + + /** C++ variable naming convention, e.g., "lower_underscore". */ + LOWER_UNDERSCORE(CharMatcher.is('_'), "_") { + @Override + String normalizeWord(String word) { + return Ascii.toLowerCase(word); + } + + @Override + String convert(CaseFormat format, String s) { + if (format == LOWER_HYPHEN) { + return s.replace('_', '-'); + } + if (format == UPPER_UNDERSCORE) { + return Ascii.toUpperCase(s); + } + return super.convert(format, s); + } + }, + + /** Java variable naming convention, e.g., "lowerCamel". */ + LOWER_CAMEL(CharMatcher.inRange('A', 'Z'), "") { + @Override + String normalizeWord(String word) { + return firstCharOnlyToUpper(word); + } + }, + + /** Java and C++ class naming convention, e.g., "UpperCamel". */ + UPPER_CAMEL(CharMatcher.inRange('A', 'Z'), "") { + @Override + String normalizeWord(String word) { + return firstCharOnlyToUpper(word); + } + }, + + /** Java and C++ constant naming convention, e.g., "UPPER_UNDERSCORE". */ + UPPER_UNDERSCORE(CharMatcher.is('_'), "_") { + @Override + String normalizeWord(String word) { + return Ascii.toUpperCase(word); + } + + @Override + String convert(CaseFormat format, String s) { + if (format == LOWER_HYPHEN) { + return Ascii.toLowerCase(s.replace('_', '-')); + } + if (format == LOWER_UNDERSCORE) { + return Ascii.toLowerCase(s); + } + return super.convert(format, s); + } + }; + + private final CharMatcher wordBoundary; + private final String wordSeparator; + + CaseFormat(CharMatcher wordBoundary, String wordSeparator) { + this.wordBoundary = wordBoundary; + this.wordSeparator = wordSeparator; + } + + /** + * Converts the specified {@code String str} from this format to the specified {@code format}. A + * "best effort" approach is taken; if {@code str} does not conform to the assumed format, then + * the behavior of this method is undefined but we make a reasonable effort at converting anyway. + */ + public final String to(CaseFormat format, String str) { + checkNotNull(format); + checkNotNull(str); + return (format == this) ? str : convert(format, str); + } + + /** Enum values can override for performance reasons. */ + String convert(CaseFormat format, String s) { + // deal with camel conversion + StringBuilder out = null; + int i = 0; + int j = -1; + while ((j = wordBoundary.indexIn(s, ++j)) != -1) { + if (i == 0) { + // include some extra space for separators + out = new StringBuilder(s.length() + 4 * wordSeparator.length()); + out.append(format.normalizeFirstWord(s.substring(i, j))); + } else { + out.append(format.normalizeWord(s.substring(i, j))); + } + out.append(format.wordSeparator); + i = j + wordSeparator.length(); + } + return (i == 0) + ? format.normalizeFirstWord(s) + : out.append(format.normalizeWord(s.substring(i))).toString(); + } + + /** + * Returns a {@code Converter} that converts strings from this format to {@code targetFormat}. + * + * @since 16.0 + */ + public Converter converterTo(CaseFormat targetFormat) { + return new StringConverter(this, targetFormat); + } + + private static final class StringConverter extends Converter + implements Serializable { + + private final CaseFormat sourceFormat; + private final CaseFormat targetFormat; + + StringConverter(CaseFormat sourceFormat, CaseFormat targetFormat) { + this.sourceFormat = checkNotNull(sourceFormat); + this.targetFormat = checkNotNull(targetFormat); + } + + @Override + protected String doForward(String s) { + return sourceFormat.to(targetFormat, s); + } + + @Override + protected String doBackward(String s) { + return targetFormat.to(sourceFormat, s); + } + + @Override + public boolean equals(Object object) { + if (object instanceof StringConverter) { + StringConverter that = (StringConverter) object; + return sourceFormat.equals(that.sourceFormat) && targetFormat.equals(that.targetFormat); + } + return false; + } + + @Override + public int hashCode() { + return sourceFormat.hashCode() ^ targetFormat.hashCode(); + } + + @Override + public String toString() { + return sourceFormat + ".converterTo(" + targetFormat + ")"; + } + + private static final long serialVersionUID = 0L; + } + + abstract String normalizeWord(String word); + + private String normalizeFirstWord(String word) { + return (this == LOWER_CAMEL) ? Ascii.toLowerCase(word) : normalizeWord(word); + } + + private static String firstCharOnlyToUpper(String word) { + return (word.isEmpty()) + ? word + : Ascii.toUpperCase(word.charAt(0)) + Ascii.toLowerCase(word.substring(1)); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CharMatcher.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CharMatcher.java new file mode 100644 index 0000000000000..981345acf4f98 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CharMatcher.java @@ -0,0 +1,1981 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.Arrays; +import java.util.BitSet; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; + +/** + * Determines a true or false value for any Java {@code char} value, just as {@link Predicate} does + * for any {@link Object}. Also offers basic text processing methods based on this function. + * Implementations are strongly encouraged to be side-effect-free and immutable. + * + *

Throughout the documentation of this class, the phrase "matching character" is used to mean + * "any {@code char} value {@code c} for which {@code this.matches(c)} returns {@code true}". + * + *

Warning: This class deals only with {@code char} values, that is, BMP characters. It does not understand + * supplementary Unicode code + * points in the range {@code 0x10000} to {@code 0x10FFFF} which includes the majority of + * assigned characters, including important CJK characters and emoji. + * + *

Supplementary characters are encoded + * into a {@code String} using surrogate pairs, and a {@code CharMatcher} treats these just as + * two separate characters. {@link #countIn} counts each supplementary character as 2 {@code char}s. + * + *

For up-to-date Unicode character properties (digit, letter, etc.) and support for + * supplementary code points, use ICU4J UCharacter and UnicodeSet (freeze() after building). For + * basic text processing based on UnicodeSet use the ICU4J UnicodeSetSpanner. + * + *

Example usages: + * + *

+ *   String trimmed = {@link #whitespace() whitespace()}.{@link #trimFrom trimFrom}(userInput);
+ *   if ({@link #ascii() ascii()}.{@link #matchesAllOf matchesAllOf}(s)) { ... }
+ * + *

See the Guava User Guide article on {@code CharMatcher} + * . + * + * @author Kevin Bourrillion + * @since 1.0 + */ +public abstract class CharMatcher implements Predicate { + /* + * N777777777NO + * N7777777777777N + * M777777777777777N + * $N877777777D77777M + * N M77777777ONND777M + * MN777777777NN D777 + * N7ZN777777777NN ~M7778 + * N777777777777MMNN88777N + * N777777777777MNZZZ7777O + * DZN7777O77777777777777 + * N7OONND7777777D77777N + * 8$M++++?N???$77777$ + * M7++++N+M77777777N + * N77O777777777777$ M + * DNNM$$$$777777N D + * N$N:=N$777N7777M NZ + * 77Z::::N777777777 ODZZZ + * 77N::::::N77777777M NNZZZ$ + * $777:::::::77777777MN ZM8ZZZZZ + * 777M::::::Z7777777Z77 N++ZZZZNN + * 7777M:::::M7777777$777M $++IZZZZM + * M777$:::::N777777$M7777M +++++ZZZDN + * NN$::::::7777$$M777777N N+++ZZZZNZ + * N::::::N:7$O:77777777 N++++ZZZZN + * M::::::::::::N77777777+ +?+++++ZZZM + * 8::::::::::::D77777777M O+++++ZZ + * ::::::::::::M777777777N O+?D + * M:::::::::::M77777777778 77= + * D=::::::::::N7777777777N 777 + * INN===::::::=77777777777N I777N + * ?777N========N7777777777787M N7777 + * 77777$D======N77777777777N777N? N777777 + * I77777$$$N7===M$$77777777$77777777$MMZ77777777N + * $$$$$$$$$$$NIZN$$$$$$$$$M$$7777777777777777ON + * M$$$$$$$$M M$$$$$$$$N=N$$$$7777777$$$ND + * O77Z$$$$$$$ M$$$$$$$$MNI==$DNNNNM=~N + * 7 :N MNN$$$$M$ $$$777$8 8D8I + * NMM.:7O 777777778 + * 7777777MN + * M NO .7: + * M : M + * 8 + */ + + // Constant matcher factory methods + + /** + * Matches any character. + * + * @since 19.0 (since 1.0 as constant {@code ANY}) + */ + public static CharMatcher any() { + return Any.INSTANCE; + } + + /** + * Matches no characters. + * + * @since 19.0 (since 1.0 as constant {@code NONE}) + */ + public static CharMatcher none() { + return None.INSTANCE; + } + + /** + * Determines whether a character is whitespace according to the latest Unicode standard, as + * illustrated here. + * This is not the same definition used by other Java APIs. (See a comparison of several definitions of "whitespace".) + * + *

All Unicode White_Space characters are on the BMP and thus supported by this API. + * + *

Note: as the Unicode definition evolves, we will modify this matcher to keep it up to + * date. + * + * @since 19.0 (since 1.0 as constant {@code WHITESPACE}) + */ + public static CharMatcher whitespace() { + return Whitespace.INSTANCE; + } + + /** + * Determines whether a character is a breaking whitespace (that is, a whitespace which can be + * interpreted as a break between words for formatting purposes). See {@link #whitespace()} for a + * discussion of that term. + * + * @since 19.0 (since 2.0 as constant {@code BREAKING_WHITESPACE}) + */ + public static CharMatcher breakingWhitespace() { + return BreakingWhitespace.INSTANCE; + } + + /** + * Determines whether a character is ASCII, meaning that its code point is less than 128. + * + * @since 19.0 (since 1.0 as constant {@code ASCII}) + */ + public static CharMatcher ascii() { + return Ascii.INSTANCE; + } + + /** + * Determines whether a character is a BMP digit according to Unicode. If + * you only care to match ASCII digits, you can use {@code inRange('0', '9')}. + * + * @deprecated Many digits are supplementary characters; see the class documentation. + * @since 19.0 (since 1.0 as constant {@code DIGIT}) + */ + @Deprecated + public static CharMatcher digit() { + return Digit.INSTANCE; + } + + /** + * Determines whether a character is a BMP digit according to {@linkplain Character#isDigit(char) + * Java's definition}. If you only care to match ASCII digits, you can use {@code inRange('0', + * '9')}. + * + * @deprecated Many digits are supplementary characters; see the class documentation. + * @since 19.0 (since 1.0 as constant {@code JAVA_DIGIT}) + */ + @Deprecated + public static CharMatcher javaDigit() { + return JavaDigit.INSTANCE; + } + + /** + * Determines whether a character is a BMP letter according to {@linkplain + * Character#isLetter(char) Java's definition}. If you only care to match letters of the Latin + * alphabet, you can use {@code inRange('a', 'z').or(inRange('A', 'Z'))}. + * + * @deprecated Most letters are supplementary characters; see the class documentation. + * @since 19.0 (since 1.0 as constant {@code JAVA_LETTER}) + */ + @Deprecated + public static CharMatcher javaLetter() { + return JavaLetter.INSTANCE; + } + + /** + * Determines whether a character is a BMP letter or digit according to {@linkplain + * Character#isLetterOrDigit(char) Java's definition}. + * + * @deprecated Most letters and digits are supplementary characters; see the class documentation. + * @since 19.0 (since 1.0 as constant {@code JAVA_LETTER_OR_DIGIT}). + */ + @Deprecated + public static CharMatcher javaLetterOrDigit() { + return JavaLetterOrDigit.INSTANCE; + } + + /** + * Determines whether a BMP character is upper case according to {@linkplain + * Character#isUpperCase(char) Java's definition}. + * + * @deprecated Some uppercase characters are supplementary characters; see the class + * documentation. + * @since 19.0 (since 1.0 as constant {@code JAVA_UPPER_CASE}) + */ + @Deprecated + public static CharMatcher javaUpperCase() { + return JavaUpperCase.INSTANCE; + } + + /** + * Determines whether a BMP character is lower case according to {@linkplain + * Character#isLowerCase(char) Java's definition}. + * + * @deprecated Some lowercase characters are supplementary characters; see the class + * documentation. + * @since 19.0 (since 1.0 as constant {@code JAVA_LOWER_CASE}) + */ + @Deprecated + public static CharMatcher javaLowerCase() { + return JavaLowerCase.INSTANCE; + } + + /** + * Determines whether a character is an ISO control character as specified by {@link + * Character#isISOControl(char)}. + * + *

All ISO control codes are on the BMP and thus supported by this API. + * + * @since 19.0 (since 1.0 as constant {@code JAVA_ISO_CONTROL}) + */ + public static CharMatcher javaIsoControl() { + return JavaIsoControl.INSTANCE; + } + + /** + * Determines whether a character is invisible; that is, if its Unicode category is any of + * SPACE_SEPARATOR, LINE_SEPARATOR, PARAGRAPH_SEPARATOR, CONTROL, FORMAT, SURROGATE, and + * PRIVATE_USE according to ICU4J. + * + *

See also the Unicode Default_Ignorable_Code_Point property (available via ICU). + * + * @deprecated Most invisible characters are supplementary characters; see the class + * documentation. + * @since 19.0 (since 1.0 as constant {@code INVISIBLE}) + */ + @Deprecated + public static CharMatcher invisible() { + return Invisible.INSTANCE; + } + + /** + * Determines whether a character is single-width (not double-width). When in doubt, this matcher + * errs on the side of returning {@code false} (that is, it tends to assume a character is + * double-width). + * + *

Note: as the reference file evolves, we will modify this matcher to keep it up to + * date. + * + *

See also UAX #11 East Asian Width. + * + * @deprecated Many such characters are supplementary characters; see the class documentation. + * @since 19.0 (since 1.0 as constant {@code SINGLE_WIDTH}) + */ + @Deprecated + public static CharMatcher singleWidth() { + return SingleWidth.INSTANCE; + } + + // Legacy constants + + /** + * Determines whether a character is whitespace according to the latest Unicode + * standard, as illustrated + * here. + * This is not the same definition used by other Java APIs. (See a + * comparison of several definitions of + * "whitespace".) + * + *

Note: as the Unicode definition evolves, we will modify this constant + * to keep it up to date. + * + * @deprecated Use {@link #whitespace()} instead. This constant is scheduled to be + * removed in June 2018. + */ + + @Deprecated + public static final CharMatcher WHITESPACE = whitespace(); + + /** + * Determines whether a character is a breaking whitespace (that is, a whitespace + * which can be interpreted as a break between words for formatting purposes). See + * {@link #whitespace} for a discussion of that term. + * + * @since 2.0 + * @deprecated Use {@link #breakingWhitespace()} instead. This constant is scheduled + * to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher BREAKING_WHITESPACE = breakingWhitespace(); + + /** + * Determines whether a character is ASCII, meaning that its code point is less than + * 128. + * + * @deprecated Use {@link #ascii()} instead. This constant is scheduled to be + * removed in June 2018. + */ + + @Deprecated + public static final CharMatcher ASCII = ascii(); + + /** + * Determines whether a character is a digit according to + * + * Unicode. If you only care to match ASCII digits, you can use + * {@code inRange('0', '9')}. + * + * @deprecated Many digits are supplementary characters; see the class + * documentation. If you need to use this, use {@link #digit()} instead. This + * . constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher DIGIT = digit(); + + /** + * Determines whether a character is a digit according to + * {@linkplain Character#isDigit(char) Java's definition}. If you only care to match + * ASCII digits, you can use {@code inRange('0', '9')}. + * + * @deprecated Many digits are supplementary characters; see the class + * documentation. If you need to use this, use {@link #javaDigit()} instead. + * This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher JAVA_DIGIT = javaDigit(); + + /** + * Determines whether a character is a letter according to + * {@linkplain Character#isLetter(char) Java's definition}. If you only care to + * match letters of the Latin alphabet, you can use + * {@code inRange('a', 'z').or(inRange('A', 'Z'))}. + * + * @deprecated Most letters are supplementary characters; see the class + * documentation. If you need to use this, use {@link #javaLetter()} instead. + * This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher JAVA_LETTER = javaLetter(); + + /** + * Determines whether a character is a letter or digit according to + * {@linkplain Character#isLetterOrDigit(char) Java's definition}. + * + * @deprecated Most letters and digits are supplementary characters; see the class + * documentation. If you need to use this, use {@link #javaLetterOrDigit()} + * instead. This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher JAVA_LETTER_OR_DIGIT = javaLetterOrDigit(); + + /** + * Determines whether a character is upper case according to + * {@linkplain Character#isUpperCase(char) Java's definition}. + * + * @deprecated Some uppercase letters are supplementary characters; see the class + * documentation. If you need to use this, use {@link #javaUpperCase()} instead. + * This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher JAVA_UPPER_CASE = javaUpperCase(); + + /** + * Determines whether a character is lower case according to + * {@linkplain Character#isLowerCase(char) Java's definition}. + * + * @deprecated Some lowercase letters are supplementary characters; see the class + * documentation. If you need to use this, use {@link #javaLowerCase()} instead. + * This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher JAVA_LOWER_CASE = javaLowerCase(); + + /** + * Determines whether a character is an ISO control character as specified by + * {@link Character#isISOControl(char)}. + * + * @deprecated Use {@link #javaIsoControl()} instead. This constant is scheduled to + * be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher JAVA_ISO_CONTROL = javaIsoControl(); + + /** + * Determines whether a character is invisible; that is, if its Unicode category is + * any of SPACE_SEPARATOR, LINE_SEPARATOR, PARAGRAPH_SEPARATOR, CONTROL, FORMAT, + * SURROGATE, and PRIVATE_USE according to ICU4J. + * + * @deprecated Most invisible characters are supplementary characters; see the class + * documentation. If you need to use this, use {@link #invisible()} instead. + * This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher INVISIBLE = invisible(); + + /** + * Determines whether a character is single-width (not double-width). When in doubt, + * this matcher errs on the side of returning {@code false} (that is, it tends to + * assume a character is double-width). + * + *

Note: as the reference file evolves, we will modify this constant to + * keep it up to date. + * + * @deprecated Many such characters are supplementary characters; see the class + * documentation. If you need to use this, use {@link #singleWidth()} instead. + * This constant is scheduled to be removed in June 2018. + */ + + @Deprecated + public static final CharMatcher SINGLE_WIDTH = singleWidth(); + + /** + * Matches any character. + * + * @deprecated Use {@link #any()} instead. This constant is scheduled to be + * removed in June 2018. + */ + + @Deprecated + public static final CharMatcher ANY = any(); + + /** + * Matches no characters. + * + * @deprecated Use {@link #none()} instead. This constant is scheduled to be + * removed in June 2018. + */ + + @Deprecated + public static final CharMatcher NONE = none(); + + // Static factories + + /** Returns a {@code char} matcher that matches only one specified BMP character. */ + public static CharMatcher is(final char match) { + return new Is(match); + } + + /** + * Returns a {@code char} matcher that matches any character except the BMP character specified. + * + *

To negate another {@code CharMatcher}, use {@link #negate()}. + */ + public static CharMatcher isNot(final char match) { + return new IsNot(match); + } + + /** + * Returns a {@code char} matcher that matches any BMP character present in the given character + * sequence. Returns a bogus matcher if the sequence contains supplementary characters. + */ + public static CharMatcher anyOf(final CharSequence sequence) { + switch (sequence.length()) { + case 0: + return none(); + case 1: + return is(sequence.charAt(0)); + case 2: + return isEither(sequence.charAt(0), sequence.charAt(1)); + default: + // TODO(lowasser): is it potentially worth just going ahead and building a precomputed + // matcher? + return new AnyOf(sequence); + } + } + + /** + * Returns a {@code char} matcher that matches any BMP character not present in the given + * character sequence. Returns a bogus matcher if the sequence contains supplementary characters. + */ + public static CharMatcher noneOf(CharSequence sequence) { + return anyOf(sequence).negate(); + } + + /** + * Returns a {@code char} matcher that matches any character in a given BMP range (both endpoints + * are inclusive). For example, to match any lowercase letter of the English alphabet, use {@code + * CharMatcher.inRange('a', 'z')}. + * + * @throws IllegalArgumentException if {@code endInclusive < startInclusive} + */ + public static CharMatcher inRange(final char startInclusive, final char endInclusive) { + return new InRange(startInclusive, endInclusive); + } + + /** + * Returns a matcher with identical behavior to the given {@link Character}-based predicate, but + * which operates on primitive {@code char} instances instead. + */ + public static CharMatcher forPredicate(final Predicate predicate) { + return predicate instanceof CharMatcher ? (CharMatcher) predicate : new ForPredicate(predicate); + } + + // Constructors + + /** + * Constructor for use by subclasses. When subclassing, you may want to override {@code + * toString()} to provide a useful description. + */ + protected CharMatcher() {} + + // Abstract methods + + /** Determines a true or false value for the given character. */ + public abstract boolean matches(char c); + + // Non-static factories + + /** Returns a matcher that matches any character not matched by this matcher. */ + // @Override under Java 8 but not under Java 7 + public CharMatcher negate() { + return new Negated(this); + } + + /** + * Returns a matcher that matches any character matched by both this matcher and {@code other}. + */ + public CharMatcher and(CharMatcher other) { + return new And(this, other); + } + + /** + * Returns a matcher that matches any character matched by either this matcher or {@code other}. + */ + public CharMatcher or(CharMatcher other) { + return new Or(this, other); + } + + /** + * Returns a {@code char} matcher functionally equivalent to this one, but which may be faster to + * query than the original; your mileage may vary. Precomputation takes time and is likely to be + * worthwhile only if the precomputed matcher is queried many thousands of times. + * + *

This method has no effect (returns {@code this}) when called in GWT: it's unclear whether a + * precomputed matcher is faster, but it certainly consumes more memory, which doesn't seem like a + * worthwhile tradeoff in a browser. + */ + public CharMatcher precomputed() { + return Platform.precomputeCharMatcher(this); + } + + private static final int DISTINCT_CHARS = Character.MAX_VALUE - Character.MIN_VALUE + 1; + + /** + * This is the actual implementation of {@link #precomputed}, but we bounce calls through a method + * on {@link Platform} so that we can have different behavior in GWT. + * + *

This implementation tries to be smart in a number of ways. It recognizes cases where the + * negation is cheaper to precompute than the matcher itself; it tries to build small hash tables + * for matchers that only match a few characters, and so on. In the worst-case scenario, it + * constructs an eight-kilobyte bit array and queries that. In many situations this produces a + * matcher which is faster to query than the original. + */ + CharMatcher precomputedInternal() { + final BitSet table = new BitSet(); + setBits(table); + int totalCharacters = table.cardinality(); + if (totalCharacters * 2 <= DISTINCT_CHARS) { + return precomputedPositive(totalCharacters, table, toString()); + } else { + // TODO(lowasser): is it worth it to worry about the last character of large matchers? + table.flip(Character.MIN_VALUE, Character.MAX_VALUE + 1); + int negatedCharacters = DISTINCT_CHARS - totalCharacters; + String suffix = ".negate()"; + final String description = toString(); + String negatedDescription = + description.endsWith(suffix) + ? description.substring(0, description.length() - suffix.length()) + : description + suffix; + return new NegatedFastMatcher( + precomputedPositive(negatedCharacters, table, negatedDescription)) { + @Override + public String toString() { + return description; + } + }; + } + } + + /** + * Helper method for {@link #precomputedInternal} that doesn't test if the negation is cheaper. + */ + private static CharMatcher precomputedPositive( + int totalCharacters, BitSet table, String description) { + switch (totalCharacters) { + case 0: + return none(); + case 1: + return is((char) table.nextSetBit(0)); + case 2: + char c1 = (char) table.nextSetBit(0); + char c2 = (char) table.nextSetBit(c1 + 1); + return isEither(c1, c2); + default: + return isSmall(totalCharacters, table.length()) + ? SmallCharMatcher.from(table, description) + : new BitSetMatcher(table, description); + } + } + + private static boolean isSmall(int totalCharacters, int tableLength) { + return totalCharacters <= SmallCharMatcher.MAX_SIZE + && tableLength > (totalCharacters * 4 * Character.SIZE); + // err on the side of BitSetMatcher + } + + /** Sets bits in {@code table} matched by this matcher. */ + void setBits(BitSet table) { + for (int c = Character.MAX_VALUE; c >= Character.MIN_VALUE; c--) { + if (matches((char) c)) { + table.set(c); + } + } + } + + // Text processing routines + + /** + * Returns {@code true} if a character sequence contains at least one matching BMP character. + * Equivalent to {@code !matchesNoneOf(sequence)}. + * + *

The default implementation iterates over the sequence, invoking {@link #matches} for each + * character, until this returns {@code true} or the end is reached. + * + * @param sequence the character sequence to examine, possibly empty + * @return {@code true} if this matcher matches at least one character in the sequence + * @since 8.0 + */ + public boolean matchesAnyOf(CharSequence sequence) { + return !matchesNoneOf(sequence); + } + + /** + * Returns {@code true} if a character sequence contains only matching BMP characters. + * + *

The default implementation iterates over the sequence, invoking {@link #matches} for each + * character, until this returns {@code false} or the end is reached. + * + * @param sequence the character sequence to examine, possibly empty + * @return {@code true} if this matcher matches every character in the sequence, including when + * the sequence is empty + */ + public boolean matchesAllOf(CharSequence sequence) { + for (int i = sequence.length() - 1; i >= 0; i--) { + if (!matches(sequence.charAt(i))) { + return false; + } + } + return true; + } + + /** + * Returns {@code true} if a character sequence contains no matching BMP characters. Equivalent to + * {@code !matchesAnyOf(sequence)}. + * + *

The default implementation iterates over the sequence, invoking {@link #matches} for each + * character, until this returns {@code true} or the end is reached. + * + * @param sequence the character sequence to examine, possibly empty + * @return {@code true} if this matcher matches no characters in the sequence, including when the + * sequence is empty + */ + public boolean matchesNoneOf(CharSequence sequence) { + return indexIn(sequence) == -1; + } + + /** + * Returns the index of the first matching BMP character in a character sequence, or {@code -1} if + * no matching character is present. + * + *

The default implementation iterates over the sequence in forward order calling {@link + * #matches} for each character. + * + * @param sequence the character sequence to examine from the beginning + * @return an index, or {@code -1} if no character matches + */ + public int indexIn(CharSequence sequence) { + return indexIn(sequence, 0); + } + + /** + * Returns the index of the first matching BMP character in a character sequence, starting from a + * given position, or {@code -1} if no character matches after that position. + * + *

The default implementation iterates over the sequence in forward order, beginning at {@code + * start}, calling {@link #matches} for each character. + * + * @param sequence the character sequence to examine + * @param start the first index to examine; must be nonnegative and no greater than {@code + * sequence.length()} + * @return the index of the first matching character, guaranteed to be no less than {@code start}, + * or {@code -1} if no character matches + * @throws IndexOutOfBoundsException if start is negative or greater than {@code + * sequence.length()} + */ + public int indexIn(CharSequence sequence, int start) { + int length = sequence.length(); + checkPositionIndex(start, length); + for (int i = start; i < length; i++) { + if (matches(sequence.charAt(i))) { + return i; + } + } + return -1; + } + + /** + * Returns the index of the last matching BMP character in a character sequence, or {@code -1} if + * no matching character is present. + * + *

The default implementation iterates over the sequence in reverse order calling {@link + * #matches} for each character. + * + * @param sequence the character sequence to examine from the end + * @return an index, or {@code -1} if no character matches + */ + public int lastIndexIn(CharSequence sequence) { + for (int i = sequence.length() - 1; i >= 0; i--) { + if (matches(sequence.charAt(i))) { + return i; + } + } + return -1; + } + + /** + * Returns the number of matching {@code char}s found in a character sequence. + * + *

Counts 2 per supplementary character, such as for {@link #whitespace}().{@link #negate}(). + */ + public int countIn(CharSequence sequence) { + int count = 0; + for (int i = 0; i < sequence.length(); i++) { + if (matches(sequence.charAt(i))) { + count++; + } + } + return count; + } + + /** + * Returns a string containing all non-matching characters of a character sequence, in order. For + * example: + * + *

{@code
+   * CharMatcher.is('a').removeFrom("bazaar")
+   * }
+ * + * ... returns {@code "bzr"}. + */ + public String removeFrom(CharSequence sequence) { + String string = sequence.toString(); + int pos = indexIn(string); + if (pos == -1) { + return string; + } + + char[] chars = string.toCharArray(); + int spread = 1; + + // This unusual loop comes from extensive benchmarking + OUT: + while (true) { + pos++; + while (true) { + if (pos == chars.length) { + break OUT; + } + if (matches(chars[pos])) { + break; + } + chars[pos - spread] = chars[pos]; + pos++; + } + spread++; + } + return new String(chars, 0, pos - spread); + } + + /** + * Returns a string containing all matching BMP characters of a character sequence, in order. For + * example: + * + *
{@code
+   * CharMatcher.is('a').retainFrom("bazaar")
+   * }
+ * + * ... returns {@code "aaa"}. + */ + public String retainFrom(CharSequence sequence) { + return negate().removeFrom(sequence); + } + + /** + * Returns a string copy of the input character sequence, with each matching BMP character + * replaced by a given replacement character. For example: + * + *
{@code
+   * CharMatcher.is('a').replaceFrom("radar", 'o')
+   * }
+ * + * ... returns {@code "rodor"}. + * + *

The default implementation uses {@link #indexIn(CharSequence)} to find the first matching + * character, then iterates the remainder of the sequence calling {@link #matches(char)} for each + * character. + * + * @param sequence the character sequence to replace matching characters in + * @param replacement the character to append to the result string in place of each matching + * character in {@code sequence} + * @return the new string + */ + public String replaceFrom(CharSequence sequence, char replacement) { + String string = sequence.toString(); + int pos = indexIn(string); + if (pos == -1) { + return string; + } + char[] chars = string.toCharArray(); + chars[pos] = replacement; + for (int i = pos + 1; i < chars.length; i++) { + if (matches(chars[i])) { + chars[i] = replacement; + } + } + return new String(chars); + } + + /** + * Returns a string copy of the input character sequence, with each matching BMP character + * replaced by a given replacement sequence. For example: + * + *

{@code
+   * CharMatcher.is('a').replaceFrom("yaha", "oo")
+   * }
+ * + * ... returns {@code "yoohoo"}. + * + *

Note: If the replacement is a fixed string with only one character, you are better + * off calling {@link #replaceFrom(CharSequence, char)} directly. + * + * @param sequence the character sequence to replace matching characters in + * @param replacement the characters to append to the result string in place of each matching + * character in {@code sequence} + * @return the new string + */ + public String replaceFrom(CharSequence sequence, CharSequence replacement) { + int replacementLen = replacement.length(); + if (replacementLen == 0) { + return removeFrom(sequence); + } + if (replacementLen == 1) { + return replaceFrom(sequence, replacement.charAt(0)); + } + + String string = sequence.toString(); + int pos = indexIn(string); + if (pos == -1) { + return string; + } + + int len = string.length(); + StringBuilder buf = new StringBuilder((len * 3 / 2) + 16); + + int oldpos = 0; + do { + buf.append(string, oldpos, pos); + buf.append(replacement); + oldpos = pos + 1; + pos = indexIn(string, oldpos); + } while (pos != -1); + + buf.append(string, oldpos, len); + return buf.toString(); + } + + /** + * Returns a substring of the input character sequence that omits all matching BMP characters from + * the beginning and from the end of the string. For example: + * + *

{@code
+   * CharMatcher.anyOf("ab").trimFrom("abacatbab")
+   * }
+ * + * ... returns {@code "cat"}. + * + *

Note that: + * + *

{@code
+   * CharMatcher.inRange('\0', ' ').trimFrom(str)
+   * }
+ * + * ... is equivalent to {@link String#trim()}. + */ + public String trimFrom(CharSequence sequence) { + int len = sequence.length(); + int first; + int last; + + for (first = 0; first < len; first++) { + if (!matches(sequence.charAt(first))) { + break; + } + } + for (last = len - 1; last > first; last--) { + if (!matches(sequence.charAt(last))) { + break; + } + } + + return sequence.subSequence(first, last + 1).toString(); + } + + /** + * Returns a substring of the input character sequence that omits all matching BMP characters from + * the beginning of the string. For example: + * + *
{@code
+   * CharMatcher.anyOf("ab").trimLeadingFrom("abacatbab")
+   * }
+ * + * ... returns {@code "catbab"}. + */ + public String trimLeadingFrom(CharSequence sequence) { + int len = sequence.length(); + for (int first = 0; first < len; first++) { + if (!matches(sequence.charAt(first))) { + return sequence.subSequence(first, len).toString(); + } + } + return ""; + } + + /** + * Returns a substring of the input character sequence that omits all matching BMP characters from + * the end of the string. For example: + * + *
{@code
+   * CharMatcher.anyOf("ab").trimTrailingFrom("abacatbab")
+   * }
+ * + * ... returns {@code "abacat"}. + */ + public String trimTrailingFrom(CharSequence sequence) { + int len = sequence.length(); + for (int last = len - 1; last >= 0; last--) { + if (!matches(sequence.charAt(last))) { + return sequence.subSequence(0, last + 1).toString(); + } + } + return ""; + } + + /** + * Returns a string copy of the input character sequence, with each group of consecutive matching + * BMP characters replaced by a single replacement character. For example: + * + *
{@code
+   * CharMatcher.anyOf("eko").collapseFrom("bookkeeper", '-')
+   * }
+ * + * ... returns {@code "b-p-r"}. + * + *

The default implementation uses {@link #indexIn(CharSequence)} to find the first matching + * character, then iterates the remainder of the sequence calling {@link #matches(char)} for each + * character. + * + * @param sequence the character sequence to replace matching groups of characters in + * @param replacement the character to append to the result string in place of each group of + * matching characters in {@code sequence} + * @return the new string + */ + public String collapseFrom(CharSequence sequence, char replacement) { + // This implementation avoids unnecessary allocation. + int len = sequence.length(); + for (int i = 0; i < len; i++) { + char c = sequence.charAt(i); + if (matches(c)) { + if (c == replacement && (i == len - 1 || !matches(sequence.charAt(i + 1)))) { + // a no-op replacement + i++; + } else { + StringBuilder builder = new StringBuilder(len).append(sequence, 0, i).append(replacement); + return finishCollapseFrom(sequence, i + 1, len, replacement, builder, true); + } + } + } + // no replacement needed + return sequence.toString(); + } + + /** + * Collapses groups of matching characters exactly as {@link #collapseFrom} does, except that + * groups of matching BMP characters at the start or end of the sequence are removed without + * replacement. + */ + public String trimAndCollapseFrom(CharSequence sequence, char replacement) { + // This implementation avoids unnecessary allocation. + int len = sequence.length(); + int first = 0; + int last = len - 1; + + while (first < len && matches(sequence.charAt(first))) { + first++; + } + + while (last > first && matches(sequence.charAt(last))) { + last--; + } + + return (first == 0 && last == len - 1) + ? collapseFrom(sequence, replacement) + : finishCollapseFrom( + sequence, first, last + 1, replacement, new StringBuilder(last + 1 - first), false); + } + + private String finishCollapseFrom( + CharSequence sequence, + int start, + int end, + char replacement, + StringBuilder builder, + boolean inMatchingGroup) { + for (int i = start; i < end; i++) { + char c = sequence.charAt(i); + if (matches(c)) { + if (!inMatchingGroup) { + builder.append(replacement); + inMatchingGroup = true; + } + } else { + builder.append(c); + inMatchingGroup = false; + } + } + return builder.toString(); + } + + /** + * @deprecated Provided only to satisfy the {@link Predicate} interface; use {@link #matches} + * instead. + */ + @Deprecated + @Override + public boolean apply(Character character) { + return matches(character); + } + + /** + * Returns a string representation of this {@code CharMatcher}, such as {@code + * CharMatcher.or(WHITESPACE, JAVA_DIGIT)}. + */ + @Override + public String toString() { + return super.toString(); + } + + /** + * Returns the Java Unicode escape sequence for the given {@code char}, in the form "\u12AB" where + * "12AB" is the four hexadecimal digits representing the 16-bit code unit. + */ + private static String showCharacter(char c) { + String hex = "0123456789ABCDEF"; + char[] tmp = {'\\', 'u', '\0', '\0', '\0', '\0'}; + for (int i = 0; i < 4; i++) { + tmp[5 - i] = hex.charAt(c & 0xF); + c = (char) (c >> 4); + } + return String.copyValueOf(tmp); + } + + // Fast matchers + + /** A matcher for which precomputation will not yield any significant benefit. */ + abstract static class FastMatcher extends CharMatcher { + + @Override + public final CharMatcher precomputed() { + return this; + } + + @Override + public CharMatcher negate() { + return new NegatedFastMatcher(this); + } + } + + /** {@link FastMatcher} which overrides {@code toString()} with a custom name. */ + abstract static class NamedFastMatcher extends FastMatcher { + + private final String description; + + NamedFastMatcher(String description) { + this.description = checkNotNull(description); + } + + @Override + public final String toString() { + return description; + } + } + + /** Negation of a {@link FastMatcher}. */ + static class NegatedFastMatcher extends Negated { + + NegatedFastMatcher(CharMatcher original) { + super(original); + } + + @Override + public final CharMatcher precomputed() { + return this; + } + } + + /** Fast matcher using a {@link BitSet} table of matching characters. */ + private static final class BitSetMatcher extends NamedFastMatcher { + + private final BitSet table; + + private BitSetMatcher(BitSet table, String description) { + super(description); + if (table.length() + Long.SIZE < table.size()) { + table = (BitSet) table.clone(); + // If only we could actually call BitSet.trimToSize() ourselves... + } + this.table = table; + } + + @Override + public boolean matches(char c) { + return table.get(c); + } + + @Override + void setBits(BitSet bitSet) { + bitSet.or(table); + } + } + + // Static constant implementation classes + + /** Implementation of {@link #any()}. */ + private static final class Any extends NamedFastMatcher { + + static final Any INSTANCE = new Any(); + + private Any() { + super("CharMatcher.any()"); + } + + @Override + public boolean matches(char c) { + return true; + } + + @Override + public int indexIn(CharSequence sequence) { + return (sequence.length() == 0) ? -1 : 0; + } + + @Override + public int indexIn(CharSequence sequence, int start) { + int length = sequence.length(); + checkPositionIndex(start, length); + return (start == length) ? -1 : start; + } + + @Override + public int lastIndexIn(CharSequence sequence) { + return sequence.length() - 1; + } + + @Override + public boolean matchesAllOf(CharSequence sequence) { + checkNotNull(sequence); + return true; + } + + @Override + public boolean matchesNoneOf(CharSequence sequence) { + return sequence.length() == 0; + } + + @Override + public String removeFrom(CharSequence sequence) { + checkNotNull(sequence); + return ""; + } + + @Override + public String replaceFrom(CharSequence sequence, char replacement) { + char[] array = new char[sequence.length()]; + Arrays.fill(array, replacement); + return new String(array); + } + + @Override + public String replaceFrom(CharSequence sequence, CharSequence replacement) { + StringBuilder result = new StringBuilder(sequence.length() * replacement.length()); + for (int i = 0; i < sequence.length(); i++) { + result.append(replacement); + } + return result.toString(); + } + + @Override + public String collapseFrom(CharSequence sequence, char replacement) { + return (sequence.length() == 0) ? "" : String.valueOf(replacement); + } + + @Override + public String trimFrom(CharSequence sequence) { + checkNotNull(sequence); + return ""; + } + + @Override + public int countIn(CharSequence sequence) { + return sequence.length(); + } + + @Override + public CharMatcher and(CharMatcher other) { + return checkNotNull(other); + } + + @Override + public CharMatcher or(CharMatcher other) { + checkNotNull(other); + return this; + } + + @Override + public CharMatcher negate() { + return none(); + } + } + + /** Implementation of {@link #none()}. */ + private static final class None extends NamedFastMatcher { + + static final None INSTANCE = new None(); + + private None() { + super("CharMatcher.none()"); + } + + @Override + public boolean matches(char c) { + return false; + } + + @Override + public int indexIn(CharSequence sequence) { + checkNotNull(sequence); + return -1; + } + + @Override + public int indexIn(CharSequence sequence, int start) { + int length = sequence.length(); + checkPositionIndex(start, length); + return -1; + } + + @Override + public int lastIndexIn(CharSequence sequence) { + checkNotNull(sequence); + return -1; + } + + @Override + public boolean matchesAllOf(CharSequence sequence) { + return sequence.length() == 0; + } + + @Override + public boolean matchesNoneOf(CharSequence sequence) { + checkNotNull(sequence); + return true; + } + + @Override + public String removeFrom(CharSequence sequence) { + return sequence.toString(); + } + + @Override + public String replaceFrom(CharSequence sequence, char replacement) { + return sequence.toString(); + } + + @Override + public String replaceFrom(CharSequence sequence, CharSequence replacement) { + checkNotNull(replacement); + return sequence.toString(); + } + + @Override + public String collapseFrom(CharSequence sequence, char replacement) { + return sequence.toString(); + } + + @Override + public String trimFrom(CharSequence sequence) { + return sequence.toString(); + } + + @Override + public String trimLeadingFrom(CharSequence sequence) { + return sequence.toString(); + } + + @Override + public String trimTrailingFrom(CharSequence sequence) { + return sequence.toString(); + } + + @Override + public int countIn(CharSequence sequence) { + checkNotNull(sequence); + return 0; + } + + @Override + public CharMatcher and(CharMatcher other) { + checkNotNull(other); + return this; + } + + @Override + public CharMatcher or(CharMatcher other) { + return checkNotNull(other); + } + + @Override + public CharMatcher negate() { + return any(); + } + } + + /** Implementation of {@link #whitespace()}. */ + + static final class Whitespace extends NamedFastMatcher { + + static final String TABLE = + "\u2002\u3000\r\u0085\u200A\u2005\u2000\u3000" + + "\u2029\u000B\u3000\u2008\u2003\u205F\u3000\u1680" + + "\u0009\u0020\u2006\u2001\u202F\u00A0\u000C\u2009" + + "\u3000\u2004\u3000\u3000\u2028\n\u2007\u3000"; + static final int MULTIPLIER = 1682554634; + static final int SHIFT = Integer.numberOfLeadingZeros(TABLE.length() - 1); + + static final Whitespace INSTANCE = new Whitespace(); + + Whitespace() { + super("CharMatcher.whitespace()"); + } + + @Override + public boolean matches(char c) { + return TABLE.charAt((MULTIPLIER * c) >>> SHIFT) == c; + } + + @Override + void setBits(BitSet table) { + for (int i = 0; i < TABLE.length(); i++) { + table.set(TABLE.charAt(i)); + } + } + } + + /** Implementation of {@link #breakingWhitespace()}. */ + private static final class BreakingWhitespace extends CharMatcher { + + static final CharMatcher INSTANCE = new BreakingWhitespace(); + + @Override + public boolean matches(char c) { + switch (c) { + case '\t': + case '\n': + case '\013': + case '\f': + case '\r': + case ' ': + case '\u0085': + case '\u1680': + case '\u2028': + case '\u2029': + case '\u205f': + case '\u3000': + return true; + case '\u2007': + return false; + default: + return c >= '\u2000' && c <= '\u200a'; + } + } + + @Override + public String toString() { + return "CharMatcher.breakingWhitespace()"; + } + } + + /** Implementation of {@link #ascii()}. */ + private static final class Ascii extends NamedFastMatcher { + + static final Ascii INSTANCE = new Ascii(); + + Ascii() { + super("CharMatcher.ascii()"); + } + + @Override + public boolean matches(char c) { + return c <= '\u007f'; + } + } + + /** Implementation that matches characters that fall within multiple ranges. */ + private static class RangesMatcher extends CharMatcher { + + private final String description; + private final char[] rangeStarts; + private final char[] rangeEnds; + + RangesMatcher(String description, char[] rangeStarts, char[] rangeEnds) { + this.description = description; + this.rangeStarts = rangeStarts; + this.rangeEnds = rangeEnds; + checkArgument(rangeStarts.length == rangeEnds.length); + for (int i = 0; i < rangeStarts.length; i++) { + checkArgument(rangeStarts[i] <= rangeEnds[i]); + if (i + 1 < rangeStarts.length) { + checkArgument(rangeEnds[i] < rangeStarts[i + 1]); + } + } + } + + @Override + public boolean matches(char c) { + int index = Arrays.binarySearch(rangeStarts, c); + if (index >= 0) { + return true; + } else { + index = ~index - 1; + return index >= 0 && c <= rangeEnds[index]; + } + } + + @Override + public String toString() { + return description; + } + } + + /** Implementation of {@link #digit()}. */ + private static final class Digit extends RangesMatcher { + // Plug the following UnicodeSet pattern into + // https://unicode.org/cldr/utility/list-unicodeset.jsp + // [[:Nd:]&[:nv=0:]&[\u0000-\uFFFF]] + // and get the zeroes from there. + + // Must be in ascending order. + private static final String ZEROES = + "0\u0660\u06f0\u07c0\u0966\u09e6\u0a66\u0ae6\u0b66\u0be6\u0c66\u0ce6\u0d66\u0de6" + + "\u0e50\u0ed0\u0f20\u1040\u1090\u17e0\u1810\u1946\u19d0\u1a80\u1a90\u1b50\u1bb0" + + "\u1c40\u1c50\ua620\ua8d0\ua900\ua9d0\ua9f0\uaa50\uabf0\uff10"; + + private static char[] zeroes() { + return ZEROES.toCharArray(); + } + + private static char[] nines() { + char[] nines = new char[ZEROES.length()]; + for (int i = 0; i < ZEROES.length(); i++) { + nines[i] = (char) (ZEROES.charAt(i) + 9); + } + return nines; + } + + static final Digit INSTANCE = new Digit(); + + private Digit() { + super("CharMatcher.digit()", zeroes(), nines()); + } + } + + /** Implementation of {@link #javaDigit()}. */ + private static final class JavaDigit extends CharMatcher { + + static final JavaDigit INSTANCE = new JavaDigit(); + + @Override + public boolean matches(char c) { + return Character.isDigit(c); + } + + @Override + public String toString() { + return "CharMatcher.javaDigit()"; + } + } + + /** Implementation of {@link #javaLetter()}. */ + private static final class JavaLetter extends CharMatcher { + + static final JavaLetter INSTANCE = new JavaLetter(); + + @Override + public boolean matches(char c) { + return Character.isLetter(c); + } + + @Override + public String toString() { + return "CharMatcher.javaLetter()"; + } + } + + /** Implementation of {@link #javaLetterOrDigit()}. */ + private static final class JavaLetterOrDigit extends CharMatcher { + + static final JavaLetterOrDigit INSTANCE = new JavaLetterOrDigit(); + + @Override + public boolean matches(char c) { + return Character.isLetterOrDigit(c); + } + + @Override + public String toString() { + return "CharMatcher.javaLetterOrDigit()"; + } + } + + /** Implementation of {@link #javaUpperCase()}. */ + private static final class JavaUpperCase extends CharMatcher { + + static final JavaUpperCase INSTANCE = new JavaUpperCase(); + + @Override + public boolean matches(char c) { + return Character.isUpperCase(c); + } + + @Override + public String toString() { + return "CharMatcher.javaUpperCase()"; + } + } + + /** Implementation of {@link #javaLowerCase()}. */ + private static final class JavaLowerCase extends CharMatcher { + + static final JavaLowerCase INSTANCE = new JavaLowerCase(); + + @Override + public boolean matches(char c) { + return Character.isLowerCase(c); + } + + @Override + public String toString() { + return "CharMatcher.javaLowerCase()"; + } + } + + /** Implementation of {@link #javaIsoControl()}. */ + private static final class JavaIsoControl extends NamedFastMatcher { + + static final JavaIsoControl INSTANCE = new JavaIsoControl(); + + private JavaIsoControl() { + super("CharMatcher.javaIsoControl()"); + } + + @Override + public boolean matches(char c) { + return c <= '\u001f' || (c >= '\u007f' && c <= '\u009f'); + } + } + + /** Implementation of {@link #invisible()}. */ + private static final class Invisible extends RangesMatcher { + // Plug the following UnicodeSet pattern into + // https://unicode.org/cldr/utility/list-unicodeset.jsp + // [[[:Zs:][:Zl:][:Zp:][:Cc:][:Cf:][:Cs:][:Co:]]&[\u0000-\uFFFF]] + // with the "Abbreviate" option, and get the ranges from there. + private static final String RANGE_STARTS = + "\u0000\u007f\u00ad\u0600\u061c\u06dd\u070f\u08e2\u1680\u180e\u2000\u2028\u205f\u2066" + + "\u3000\ud800\ufeff\ufff9"; + private static final String RANGE_ENDS = // inclusive ends + "\u0020\u00a0\u00ad\u0605\u061c\u06dd\u070f\u08e2\u1680\u180e\u200f\u202f\u2064\u206f" + + "\u3000\uf8ff\ufeff\ufffb"; + + static final Invisible INSTANCE = new Invisible(); + + private Invisible() { + super("CharMatcher.invisible()", RANGE_STARTS.toCharArray(), RANGE_ENDS.toCharArray()); + } + } + + /** Implementation of {@link #singleWidth()}. */ + private static final class SingleWidth extends RangesMatcher { + + static final SingleWidth INSTANCE = new SingleWidth(); + + private SingleWidth() { + super( + "CharMatcher.singleWidth()", + "\u0000\u05be\u05d0\u05f3\u0600\u0750\u0e00\u1e00\u2100\ufb50\ufe70\uff61".toCharArray(), + "\u04f9\u05be\u05ea\u05f4\u06ff\u077f\u0e7f\u20af\u213a\ufdff\ufeff\uffdc".toCharArray()); + } + } + + // Non-static factory implementation classes + + /** Implementation of {@link #negate()}. */ + private static class Negated extends CharMatcher { + + final CharMatcher original; + + Negated(CharMatcher original) { + this.original = checkNotNull(original); + } + + @Override + public boolean matches(char c) { + return !original.matches(c); + } + + @Override + public boolean matchesAllOf(CharSequence sequence) { + return original.matchesNoneOf(sequence); + } + + @Override + public boolean matchesNoneOf(CharSequence sequence) { + return original.matchesAllOf(sequence); + } + + @Override + public int countIn(CharSequence sequence) { + return sequence.length() - original.countIn(sequence); + } + + @Override + void setBits(BitSet table) { + BitSet tmp = new BitSet(); + original.setBits(tmp); + tmp.flip(Character.MIN_VALUE, Character.MAX_VALUE + 1); + table.or(tmp); + } + + @Override + public CharMatcher negate() { + return original; + } + + @Override + public String toString() { + return original + ".negate()"; + } + } + + /** Implementation of {@link #and(CharMatcher)}. */ + private static final class And extends CharMatcher { + + final CharMatcher first; + final CharMatcher second; + + And(CharMatcher a, CharMatcher b) { + first = checkNotNull(a); + second = checkNotNull(b); + } + + @Override + public boolean matches(char c) { + return first.matches(c) && second.matches(c); + } + + @Override + void setBits(BitSet table) { + BitSet tmp1 = new BitSet(); + first.setBits(tmp1); + BitSet tmp2 = new BitSet(); + second.setBits(tmp2); + tmp1.and(tmp2); + table.or(tmp1); + } + + @Override + public String toString() { + return "CharMatcher.and(" + first + ", " + second + ")"; + } + } + + /** Implementation of {@link #or(CharMatcher)}. */ + private static final class Or extends CharMatcher { + + final CharMatcher first; + final CharMatcher second; + + Or(CharMatcher a, CharMatcher b) { + first = checkNotNull(a); + second = checkNotNull(b); + } + + @Override + void setBits(BitSet table) { + first.setBits(table); + second.setBits(table); + } + + @Override + public boolean matches(char c) { + return first.matches(c) || second.matches(c); + } + + @Override + public String toString() { + return "CharMatcher.or(" + first + ", " + second + ")"; + } + } + + // Static factory implementations + + /** Implementation of {@link #is(char)}. */ + private static final class Is extends FastMatcher { + + private final char match; + + Is(char match) { + this.match = match; + } + + @Override + public boolean matches(char c) { + return c == match; + } + + @Override + public String replaceFrom(CharSequence sequence, char replacement) { + return sequence.toString().replace(match, replacement); + } + + @Override + public CharMatcher and(CharMatcher other) { + return other.matches(match) ? this : none(); + } + + @Override + public CharMatcher or(CharMatcher other) { + return other.matches(match) ? other : super.or(other); + } + + @Override + public CharMatcher negate() { + return isNot(match); + } + + @Override + void setBits(BitSet table) { + table.set(match); + } + + @Override + public String toString() { + return "CharMatcher.is('" + showCharacter(match) + "')"; + } + } + + /** Implementation of {@link #isNot(char)}. */ + private static final class IsNot extends FastMatcher { + + private final char match; + + IsNot(char match) { + this.match = match; + } + + @Override + public boolean matches(char c) { + return c != match; + } + + @Override + public CharMatcher and(CharMatcher other) { + return other.matches(match) ? super.and(other) : other; + } + + @Override + public CharMatcher or(CharMatcher other) { + return other.matches(match) ? any() : this; + } + + @Override + void setBits(BitSet table) { + table.set(0, match); + table.set(match + 1, Character.MAX_VALUE + 1); + } + + @Override + public CharMatcher negate() { + return is(match); + } + + @Override + public String toString() { + return "CharMatcher.isNot('" + showCharacter(match) + "')"; + } + } + + private static IsEither isEither(char c1, char c2) { + return new IsEither(c1, c2); + } + + /** Implementation of {@link #anyOf(CharSequence)} for exactly two characters. */ + private static final class IsEither extends FastMatcher { + + private final char match1; + private final char match2; + + IsEither(char match1, char match2) { + this.match1 = match1; + this.match2 = match2; + } + + @Override + public boolean matches(char c) { + return c == match1 || c == match2; + } + + @Override + void setBits(BitSet table) { + table.set(match1); + table.set(match2); + } + + @Override + public String toString() { + return "CharMatcher.anyOf(\"" + showCharacter(match1) + showCharacter(match2) + "\")"; + } + } + + /** Implementation of {@link #anyOf(CharSequence)} for three or more characters. */ + private static final class AnyOf extends CharMatcher { + + private final char[] chars; + + public AnyOf(CharSequence chars) { + this.chars = chars.toString().toCharArray(); + Arrays.sort(this.chars); + } + + @Override + public boolean matches(char c) { + return Arrays.binarySearch(chars, c) >= 0; + } + + @Override + void setBits(BitSet table) { + for (char c : chars) { + table.set(c); + } + } + + @Override + public String toString() { + StringBuilder description = new StringBuilder("CharMatcher.anyOf(\""); + for (char c : chars) { + description.append(showCharacter(c)); + } + description.append("\")"); + return description.toString(); + } + } + + /** Implementation of {@link #inRange(char, char)}. */ + private static final class InRange extends FastMatcher { + + private final char startInclusive; + private final char endInclusive; + + InRange(char startInclusive, char endInclusive) { + checkArgument(endInclusive >= startInclusive); + this.startInclusive = startInclusive; + this.endInclusive = endInclusive; + } + + @Override + public boolean matches(char c) { + return startInclusive <= c && c <= endInclusive; + } + + @Override + void setBits(BitSet table) { + table.set(startInclusive, endInclusive + 1); + } + + @Override + public String toString() { + return "CharMatcher.inRange('" + + showCharacter(startInclusive) + + "', '" + + showCharacter(endInclusive) + + "')"; + } + } + + /** Implementation of {@link #forPredicate(Predicate)}. */ + private static final class ForPredicate extends CharMatcher { + + private final Predicate predicate; + + ForPredicate(Predicate predicate) { + this.predicate = checkNotNull(predicate); + } + + @Override + public boolean matches(char c) { + return predicate.apply(c); + } + + @SuppressWarnings("deprecation") // intentional; deprecation is for callers primarily + @Override + public boolean apply(Character character) { + return predicate.apply(checkNotNull(character)); + } + + @Override + public String toString() { + return "CharMatcher.forPredicate(" + predicate + ")"; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Charsets.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Charsets.java new file mode 100644 index 0000000000000..f06ac8310517c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Charsets.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.nio.charset.Charset; + +/** + * Contains constant definitions for the six standard {@link Charset} instances, which are + * guaranteed to be supported by all Java platform implementations. + * + *

Assuming you're free to choose, note that {@link #UTF_8} is widely preferred. + * + *

See the Guava User Guide article on {@code Charsets}. + * + * @author Mike Bostock + * @since 1.0 + */ +public final class Charsets { + private Charsets() {} + + /** + * US-ASCII: seven-bit ASCII, the Basic Latin block of the Unicode character set (ISO646-US). + * + *

Note for Java 7 and later: this constant should be treated as deprecated; use {@link + * java.nio.charset.StandardCharsets#US_ASCII} instead. + * + */ + public static final Charset US_ASCII = Charset.forName("US-ASCII"); + + /** + * ISO-8859-1: ISO Latin Alphabet Number 1 (ISO-LATIN-1). + * + *

Note for Java 7 and later: this constant should be treated as deprecated; use {@link + * java.nio.charset.StandardCharsets#ISO_8859_1} instead. + * + */ + public static final Charset ISO_8859_1 = Charset.forName("ISO-8859-1"); + + /** + * UTF-8: eight-bit UCS Transformation Format. + * + *

Note for Java 7 and later: this constant should be treated as deprecated; use {@link + * java.nio.charset.StandardCharsets#UTF_8} instead. + * + */ + public static final Charset UTF_8 = Charset.forName("UTF-8"); + + /** + * UTF-16BE: sixteen-bit UCS Transformation Format, big-endian byte order. + * + *

Note for Java 7 and later: this constant should be treated as deprecated; use {@link + * java.nio.charset.StandardCharsets#UTF_16BE} instead. + * + */ + public static final Charset UTF_16BE = Charset.forName("UTF-16BE"); + + /** + * UTF-16LE: sixteen-bit UCS Transformation Format, little-endian byte order. + * + *

Note for Java 7 and later: this constant should be treated as deprecated; use {@link + * java.nio.charset.StandardCharsets#UTF_16LE} instead. + * + */ + public static final Charset UTF_16LE = Charset.forName("UTF-16LE"); + + /** + * UTF-16: sixteen-bit UCS Transformation Format, byte order identified by an optional byte-order + * mark. + * + *

Note for Java 7 and later: this constant should be treated as deprecated; use {@link + * java.nio.charset.StandardCharsets#UTF_16} instead. + * + */ + public static final Charset UTF_16 = Charset.forName("UTF-16"); + + /* + * Please do not add new Charset references to this class, unless those character encodings are + * part of the set required to be supported by all Java platform implementations! Any Charsets + * initialized here may cause unexpected delays when this class is loaded. See the Charset + * Javadocs for the list of built-in character encodings. + */ +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonMatcher.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonMatcher.java new file mode 100644 index 0000000000000..b91f4a0be2abd --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonMatcher.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * The subset of the {@link java.util.regex.Matcher} API which is used by this package, and also + * shared with the {@code re2j} library. For internal use only. Please refer to the {@code Matcher} + * javadoc for details. + */ +abstract class CommonMatcher { + abstract boolean matches(); + + abstract boolean find(); + + abstract boolean find(int index); + + abstract String replaceAll(String replacement); + + abstract int end(); + + abstract int start(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonPattern.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonPattern.java new file mode 100644 index 0000000000000..ad883139046f9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/CommonPattern.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * The subset of the {@link java.util.regex.Pattern} API which is used by this package, and also + * shared with the {@code re2j} library. For internal use only. Please refer to the {@code Pattern} + * javadoc for details. + */ +abstract class CommonPattern { + abstract CommonMatcher matcher(CharSequence t); + + abstract String pattern(); + + abstract int flags(); + + // Re-declare these as abstract to force subclasses to override. + @Override + public abstract String toString(); + + @Override + public abstract int hashCode(); + + @Override + public abstract boolean equals(Object o); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Converter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Converter.java new file mode 100644 index 0000000000000..14a22b5b9f541 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Converter.java @@ -0,0 +1,518 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.Iterator; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * A function from {@code A} to {@code B} with an associated reverse function from {@code B} + * to {@code A}; used for converting back and forth between different representations of the same + * information. + * + *

Invertibility

+ * + *

The reverse operation may be a strict inverse (meaning that {@code + * converter.reverse().convert(converter.convert(a)).equals(a)} is always true). However, it is very + * common (perhaps more common) for round-trip conversion to be lossy. Consider an + * example round-trip using {@link Doubles#stringConverter}: + * + *

    + *
  1. {@code stringConverter().convert("1.00")} returns the {@code Double} value {@code 1.0} + *
  2. {@code stringConverter().reverse().convert(1.0)} returns the string {@code "1.0"} -- + * not the same string ({@code "1.00"}) we started with + *
+ * + *

Note that it should still be the case that the round-tripped and original objects are + * similar. + * + *

Nullability

+ * + *

A converter always converts {@code null} to {@code null} and non-null references to non-null + * references. It would not make sense to consider {@code null} and a non-null reference to be + * "different representations of the same information", since one is distinguishable from + * missing information and the other is not. The {@link #convert} method handles this null + * behavior for all converters; implementations of {@link #doForward} and {@link #doBackward} are + * guaranteed to never be passed {@code null}, and must never return {@code null}. + * + * + *

Common ways to use

+ * + *

Getting a converter: + * + *

    + *
  • Use a provided converter implementation, such as {@link Enums#stringConverter}, {@link + * Ints#stringConverter Ints.stringConverter} or the {@linkplain + * #reverse reverse} views of these. + *
  • Convert between specific preset values using {@link + * Maps#asConverter Maps.asConverter}. For example, use this to + * create a "fake" converter for a unit test. It is unnecessary (and confusing) to mock + * the {@code Converter} type using a mocking framework. + *
  • Extend this class and implement its {@link #doForward} and {@link #doBackward} methods. + *
  • Java 8 users: you may prefer to pass two lambda expressions or method references to + * the {@link #from from} factory method. + *
+ * + *

Using a converter: + * + *

    + *
  • Convert one instance in the "forward" direction using {@code converter.convert(a)}. + *
  • Convert multiple instances "forward" using {@code converter.convertAll(as)}. + *
  • Convert in the "backward" direction using {@code converter.reverse().convert(b)} or {@code + * converter.reverse().convertAll(bs)}. + *
  • Use {@code converter} or {@code converter.reverse()} anywhere a {@link + * java.util.function.Function} is accepted (for example {@link java.util.stream.Stream#map + * Stream.map}). + *
  • Do not call {@link #doForward} or {@link #doBackward} directly; these exist only to + * be overridden. + *
+ * + *

Example

+ * + *
+ *   return new Converter<Integer, String>() {
+ *     protected String doForward(Integer i) {
+ *       return Integer.toHexString(i);
+ *     }
+ *
+ *     protected Integer doBackward(String s) {
+ *       return parseUnsignedInt(s, 16);
+ *     }
+ *   };
+ * + *

An alternative using Java 8: + * + *

{@code
+ * return Converter.from(
+ *     Integer::toHexString,
+ *     s -> parseUnsignedInt(s, 16));
+ * }
+ * + * @author Mike Ward + * @author Kurt Alfred Kluever + * @author Gregory Kick + * @since 16.0 + */ +public abstract class Converter implements Function { + private final boolean handleNullAutomatically; + + // We lazily cache the reverse view to avoid allocating on every call to reverse(). + private transient Converter reverse; + + /** Constructor for use by subclasses. */ + protected Converter() { + this(true); + } + + /** Constructor used only by {@code LegacyConverter} to suspend automatic null-handling. */ + Converter(boolean handleNullAutomatically) { + this.handleNullAutomatically = handleNullAutomatically; + } + + // SPI methods (what subclasses must implement) + + /** + * Returns a representation of {@code a} as an instance of type {@code B}. If {@code a} cannot be + * converted, an unchecked exception (such as {@link IllegalArgumentException}) should be thrown. + * + * @param a the instance to convert; will never be null + * @return the converted instance; must not be null + */ + protected abstract B doForward(A a); + + /** + * Returns a representation of {@code b} as an instance of type {@code A}. If {@code b} cannot be + * converted, an unchecked exception (such as {@link IllegalArgumentException}) should be thrown. + * + * @param b the instance to convert; will never be null + * @return the converted instance; must not be null + * @throws UnsupportedOperationException if backward conversion is not implemented; this should be + * very rare. Note that if backward conversion is not only unimplemented but + * unimplementable (for example, consider a {@code Converter}), + * then this is not logically a {@code Converter} at all, and should just implement {@link + * Function}. + */ + protected abstract A doBackward(B b); + + // API (consumer-side) methods + + /** + * Returns a representation of {@code a} as an instance of type {@code B}. + * + * @return the converted value; is null if and only if {@code a} is null + */ + + public final B convert(A a) { + return correctedDoForward(a); + } + + + B correctedDoForward(A a) { + if (handleNullAutomatically) { + // TODO(kevinb): we shouldn't be checking for a null result at runtime. Assert? + return a == null ? null : checkNotNull(doForward(a)); + } else { + return doForward(a); + } + } + + + A correctedDoBackward(B b) { + if (handleNullAutomatically) { + // TODO(kevinb): we shouldn't be checking for a null result at runtime. Assert? + return b == null ? null : checkNotNull(doBackward(b)); + } else { + return doBackward(b); + } + } + + /** + * Returns an iterable that applies {@code convert} to each element of {@code fromIterable}. The + * conversion is done lazily. + * + *

The returned iterable's iterator supports {@code remove()} if the input iterator does. After + * a successful {@code remove()} call, {@code fromIterable} no longer contains the corresponding + * element. + */ + public Iterable convertAll(final Iterable fromIterable) { + checkNotNull(fromIterable, "fromIterable"); + return new Iterable() { + @Override + public Iterator iterator() { + return new Iterator() { + private final Iterator fromIterator = fromIterable.iterator(); + + @Override + public boolean hasNext() { + return fromIterator.hasNext(); + } + + @Override + public B next() { + return convert(fromIterator.next()); + } + + @Override + public void remove() { + fromIterator.remove(); + } + }; + } + }; + } + + /** + * Returns the reversed view of this converter, which converts {@code this.convert(a)} back to a + * value roughly equivalent to {@code a}. + * + *

The returned converter is serializable if {@code this} converter is. + * + *

Note: you should not override this method. It is non-final for legacy reasons. + */ + public Converter reverse() { + Converter result = reverse; + return (result == null) ? reverse = new ReverseConverter<>(this) : result; + } + + private static final class ReverseConverter extends Converter + implements Serializable { + final Converter original; + + ReverseConverter(Converter original) { + this.original = original; + } + + /* + * These gymnastics are a little confusing. Basically this class has neither legacy nor + * non-legacy behavior; it just needs to let the behavior of the backing converter shine + * through. So, we override the correctedDo* methods, after which the do* methods should never + * be reached. + */ + + @Override + protected A doForward(B b) { + throw new AssertionError(); + } + + @Override + protected B doBackward(A a) { + throw new AssertionError(); + } + + @Override + + A correctedDoForward(B b) { + return original.correctedDoBackward(b); + } + + @Override + + B correctedDoBackward(A a) { + return original.correctedDoForward(a); + } + + @Override + public Converter reverse() { + return original; + } + + @Override + public boolean equals(Object object) { + if (object instanceof ReverseConverter) { + ReverseConverter that = (ReverseConverter) object; + return this.original.equals(that.original); + } + return false; + } + + @Override + public int hashCode() { + return ~original.hashCode(); + } + + @Override + public String toString() { + return original + ".reverse()"; + } + + private static final long serialVersionUID = 0L; + } + + /** + * Returns a converter whose {@code convert} method applies {@code secondConverter} to the result + * of this converter. Its {@code reverse} method applies the converters in reverse order. + * + *

The returned converter is serializable if {@code this} converter and {@code secondConverter} + * are. + */ + public final Converter andThen(Converter secondConverter) { + return doAndThen(secondConverter); + } + + /** Package-private non-final implementation of andThen() so only we can override it. */ + Converter doAndThen(Converter secondConverter) { + return new ConverterComposition<>(this, checkNotNull(secondConverter)); + } + + private static final class ConverterComposition extends Converter + implements Serializable { + final Converter first; + final Converter second; + + ConverterComposition(Converter first, Converter second) { + this.first = first; + this.second = second; + } + + /* + * These gymnastics are a little confusing. Basically this class has neither legacy nor + * non-legacy behavior; it just needs to let the behaviors of the backing converters shine + * through (which might even differ from each other!). So, we override the correctedDo* methods, + * after which the do* methods should never be reached. + */ + + @Override + protected C doForward(A a) { + throw new AssertionError(); + } + + @Override + protected A doBackward(C c) { + throw new AssertionError(); + } + + @Override + + C correctedDoForward(A a) { + return second.correctedDoForward(first.correctedDoForward(a)); + } + + @Override + + A correctedDoBackward(C c) { + return first.correctedDoBackward(second.correctedDoBackward(c)); + } + + @Override + public boolean equals(Object object) { + if (object instanceof ConverterComposition) { + ConverterComposition that = (ConverterComposition) object; + return this.first.equals(that.first) && this.second.equals(that.second); + } + return false; + } + + @Override + public int hashCode() { + return 31 * first.hashCode() + second.hashCode(); + } + + @Override + public String toString() { + return first + ".andThen(" + second + ")"; + } + + private static final long serialVersionUID = 0L; + } + + /** + * @deprecated Provided to satisfy the {@code Function} interface; use {@link #convert} instead. + */ + @Deprecated + @Override + + public final B apply(A a) { + return convert(a); + } + + /** + * Indicates whether another object is equal to this converter. + * + *

Most implementations will have no reason to override the behavior of {@link Object#equals}. + * However, an implementation may also choose to return {@code true} whenever {@code object} is a + * {@link Converter} that it considers interchangeable with this one. "Interchangeable" + * typically means that {@code Objects.equal(this.convert(a), that.convert(a))} is true for + * all {@code a} of type {@code A} (and similarly for {@code reverse}). Note that a {@code false} + * result from this method does not imply that the converters are known not to be + * interchangeable. + */ + @Override + public boolean equals(Object object) { + return super.equals(object); + } + + // Static converters + + /** + * Returns a converter based on separate forward and backward functions. This is useful if the + * function instances already exist, or so that you can supply lambda expressions. If those + * circumstances don't apply, you probably don't need to use this; subclass {@code Converter} and + * implement its {@link #doForward} and {@link #doBackward} methods directly. + * + *

These functions will never be passed {@code null} and must not under any circumstances + * return {@code null}. If a value cannot be converted, the function should throw an unchecked + * exception (typically, but not necessarily, {@link IllegalArgumentException}). + * + *

The returned converter is serializable if both provided functions are. + * + * @since 17.0 + */ + public static Converter from( + Function forwardFunction, + Function backwardFunction) { + return new FunctionBasedConverter<>(forwardFunction, backwardFunction); + } + + private static final class FunctionBasedConverter extends Converter + implements Serializable { + private final Function forwardFunction; + private final Function backwardFunction; + + private FunctionBasedConverter( + Function forwardFunction, + Function backwardFunction) { + this.forwardFunction = checkNotNull(forwardFunction); + this.backwardFunction = checkNotNull(backwardFunction); + } + + @Override + protected B doForward(A a) { + return forwardFunction.apply(a); + } + + @Override + protected A doBackward(B b) { + return backwardFunction.apply(b); + } + + @Override + public boolean equals(Object object) { + if (object instanceof FunctionBasedConverter) { + FunctionBasedConverter that = (FunctionBasedConverter) object; + return this.forwardFunction.equals(that.forwardFunction) + && this.backwardFunction.equals(that.backwardFunction); + } + return false; + } + + @Override + public int hashCode() { + return forwardFunction.hashCode() * 31 + backwardFunction.hashCode(); + } + + @Override + public String toString() { + return "Converter.from(" + forwardFunction + ", " + backwardFunction + ")"; + } + } + + /** Returns a serializable converter that always converts or reverses an object to itself. */ + @SuppressWarnings("unchecked") // implementation is "fully variant" + public static Converter identity() { + return (IdentityConverter) IdentityConverter.INSTANCE; + } + + /** + * A converter that always converts or reverses an object to itself. Note that T is now a + * "pass-through type". + */ + private static final class IdentityConverter extends Converter implements Serializable { + @SuppressWarnings("rawtypes") + static final IdentityConverter INSTANCE = new IdentityConverter(); + + @Override + protected T doForward(T t) { + return t; + } + + @Override + protected T doBackward(T t) { + return t; + } + + @Override + public IdentityConverter reverse() { + return this; + } + + @Override + Converter doAndThen(Converter otherConverter) { + return checkNotNull(otherConverter, "otherConverter"); + } + + /* + * We *could* override convertAll() to return its input, but it's a rather pointless + * optimization and opened up a weird type-safety problem. + */ + + @Override + public String toString() { + return "Converter.identity()"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 0L; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Defaults.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Defaults.java new file mode 100644 index 0000000000000..f5e13f35d2ade --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Defaults.java @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +/** + * This class provides default values for all Java types, as defined by the JLS. + * + * @author Ben Yu + * @since 1.0 + */ +public final class Defaults { + private Defaults() {} + + private static final Double DOUBLE_DEFAULT = Double.valueOf(0d); + private static final Float FLOAT_DEFAULT = Float.valueOf(0f); + + /** + * Returns the default value of {@code type} as defined by JLS --- {@code 0} for numbers, {@code + * false} for {@code boolean} and {@code '\0'} for {@code char}. For non-primitive types and + * {@code void}, {@code null} is returned. + */ + + @SuppressWarnings("unchecked") + public static T defaultValue(Class type) { + checkNotNull(type); + if (type == boolean.class) { + return (T) Boolean.FALSE; + } else if (type == char.class) { + return (T) Character.valueOf('\0'); + } else if (type == byte.class) { + return (T) Byte.valueOf((byte) 0); + } else if (type == short.class) { + return (T) Short.valueOf((short) 0); + } else if (type == int.class) { + return (T) Integer.valueOf(0); + } else if (type == long.class) { + return (T) Long.valueOf(0L); + } else if (type == float.class) { + return (T) FLOAT_DEFAULT; + } else if (type == double.class) { + return (T) DOUBLE_DEFAULT; + } else { + return null; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Enums.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Enums.java new file mode 100644 index 0000000000000..b04af90394a01 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Enums.java @@ -0,0 +1,148 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.lang.ref.WeakReference; +import java.lang.reflect.Field; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Map; +import java.util.WeakHashMap; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * Utility methods for working with {@link Enum} instances. + * + * @author Steve McKay + * @since 9.0 + */ +public final class Enums { + + private Enums() {} + + /** + * Returns the {@link Field} in which {@code enumValue} is defined. For example, to get the {@code + * Description} annotation on the {@code GOLF} constant of enum {@code Sport}, use {@code + * Enums.getField(Sport.GOLF).getAnnotation(Description.class)}. + * + * @since 12.0 + */ + public static Field getField(Enum enumValue) { + Class clazz = enumValue.getDeclaringClass(); + try { + return clazz.getDeclaredField(enumValue.name()); + } catch (NoSuchFieldException impossible) { + throw new AssertionError(impossible); + } + } + + /** + * Returns an optional enum constant for the given type, using {@link Enum#valueOf}. If the + * constant does not exist, {@link Optional#absent} is returned. A common use case is for parsing + * user input or falling back to a default enum constant. For example, {@code + * Enums.getIfPresent(Country.class, countryInput).or(Country.DEFAULT);} + * + * @since 12.0 + */ + public static > Optional getIfPresent(Class enumClass, String value) { + checkNotNull(enumClass); + checkNotNull(value); + return Platform.getEnumIfPresent(enumClass, value); + } + + private static final Map>, Map>>> + enumConstantCache = new WeakHashMap<>(); + + private static > Map>> populateCache( + Class enumClass) { + Map>> result = new HashMap<>(); + for (T enumInstance : EnumSet.allOf(enumClass)) { + result.put(enumInstance.name(), new WeakReference>(enumInstance)); + } + enumConstantCache.put(enumClass, result); + return result; + } + + static > Map>> getEnumConstants( + Class enumClass) { + synchronized (enumConstantCache) { + Map>> constants = enumConstantCache.get(enumClass); + if (constants == null) { + constants = populateCache(enumClass); + } + return constants; + } + } + + /** + * Returns a converter that converts between strings and {@code enum} values of type {@code + * enumClass} using {@link Enum#valueOf(Class, String)} and {@link Enum#name()}. The converter + * will throw an {@code IllegalArgumentException} if the argument is not the name of any enum + * constant in the specified enum. + * + * @since 16.0 + */ + public static > Converter stringConverter(final Class enumClass) { + return new StringConverter(enumClass); + } + + private static final class StringConverter> extends Converter + implements Serializable { + + private final Class enumClass; + + StringConverter(Class enumClass) { + this.enumClass = checkNotNull(enumClass); + } + + @Override + protected T doForward(String value) { + return Enum.valueOf(enumClass, value); + } + + @Override + protected String doBackward(T enumValue) { + return enumValue.name(); + } + + @Override + public boolean equals(Object object) { + if (object instanceof StringConverter) { + StringConverter that = (StringConverter) object; + return this.enumClass.equals(that.enumClass); + } + return false; + } + + @Override + public int hashCode() { + return enumClass.hashCode(); + } + + @Override + public String toString() { + return "Enums.stringConverter(" + enumClass.getName() + ".class)"; + } + + private static final long serialVersionUID = 0L; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Equivalence.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Equivalence.java new file mode 100644 index 0000000000000..647e85912b46c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Equivalence.java @@ -0,0 +1,377 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.function.BiPredicate; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * A strategy for determining whether two instances are considered equivalent, and for computing + * hash codes in a manner consistent with that equivalence. Two examples of equivalences are the + * {@linkplain #identity() identity equivalence} and the {@linkplain #equals "equals" equivalence}. + * + * @author Bob Lee + * @author Ben Yu + * @author Gregory Kick + * @since 10.0 (mostly + * source-compatible since 4.0) + */ +public abstract class Equivalence implements BiPredicate { + /** Constructor for use by subclasses. */ + protected Equivalence() {} + + /** + * Returns {@code true} if the given objects are considered equivalent. + * + *

This method describes an equivalence relation on object references, meaning that for + * all references {@code x}, {@code y}, and {@code z} (any of which may be null): + * + *

    + *
  • {@code equivalent(x, x)} is true (reflexive property) + *
  • {@code equivalent(x, y)} and {@code equivalent(y, x)} each return the same result + * (symmetric property) + *
  • If {@code equivalent(x, y)} and {@code equivalent(y, z)} are both true, then {@code + * equivalent(x, z)} is also true (transitive property) + *
+ * + *

Note that all calls to {@code equivalent(x, y)} are expected to return the same result as + * long as neither {@code x} nor {@code y} is modified. + */ + public final boolean equivalent(T a, T b) { + if (a == b) { + return true; + } + if (a == null || b == null) { + return false; + } + return doEquivalent(a, b); + } + + /** + * @deprecated Provided only to satisfy the {@link BiPredicate} interface; use {@link #equivalent} + * instead. + * @since 21.0 + */ + @Deprecated + @Override + public final boolean test(T t, T u) { + return equivalent(t, u); + } + + /** + * Implemented by the user to determine whether {@code a} and {@code b} are considered equivalent, + * subject to the requirements specified in {@link #equivalent}. + * + *

This method should not be called except by {@link #equivalent}. When {@link #equivalent} + * calls this method, {@code a} and {@code b} are guaranteed to be distinct, non-null instances. + * + * @since 10.0 (previously, subclasses would override equivalent()) + */ + protected abstract boolean doEquivalent(T a, T b); + + /** + * Returns a hash code for {@code t}. + * + *

The {@code hash} has the following properties: + * + *

    + *
  • It is consistent: for any reference {@code x}, multiple invocations of {@code + * hash(x}} consistently return the same value provided {@code x} remains unchanged + * according to the definition of the equivalence. The hash need not remain consistent from + * one execution of an application to another execution of the same application. + *
  • It is distributable across equivalence: for any references {@code x} and {@code + * y}, if {@code equivalent(x, y)}, then {@code hash(x) == hash(y)}. It is not + * necessary that the hash be distributable across inequivalence. If {@code + * equivalence(x, y)} is false, {@code hash(x) == hash(y)} may still be true. + *
  • {@code hash(null)} is {@code 0}. + *
+ */ + public final int hash(T t) { + if (t == null) { + return 0; + } + return doHash(t); + } + + /** + * Implemented by the user to return a hash code for {@code t}, subject to the requirements + * specified in {@link #hash}. + * + *

This method should not be called except by {@link #hash}. When {@link #hash} calls this + * method, {@code t} is guaranteed to be non-null. + * + * @since 10.0 (previously, subclasses would override hash()) + */ + protected abstract int doHash(T t); + + /** + * Returns a new equivalence relation for {@code F} which evaluates equivalence by first applying + * {@code function} to the argument, then evaluating using {@code this}. That is, for any pair of + * non-null objects {@code x} and {@code y}, {@code equivalence.onResultOf(function).equivalent(a, + * b)} is true if and only if {@code equivalence.equivalent(function.apply(a), function.apply(b))} + * is true. + * + *

For example: + * + *

{@code
+   * Equivalence SAME_AGE = Equivalence.equals().onResultOf(GET_PERSON_AGE);
+   * }
+ * + *

{@code function} will never be invoked with a null value. + * + *

Note that {@code function} must be consistent according to {@code this} equivalence + * relation. That is, invoking {@link Function#apply} multiple times for a given value must return + * equivalent results. For example, {@code + * Equivalence.identity().onResultOf(Functions.toStringFunction())} is broken because it's not + * guaranteed that {@link Object#toString}) always returns the same string instance. + * + * @since 10.0 + */ + public final Equivalence onResultOf(Function function) { + return new FunctionalEquivalence<>(function, this); + } + + /** + * Returns a wrapper of {@code reference} that implements {@link Wrapper#equals(Object) + * Object.equals()} such that {@code wrap(a).equals(wrap(b))} if and only if {@code equivalent(a, + * b)}. + * + * @since 10.0 + */ + public final Wrapper wrap(S reference) { + return new Wrapper(this, reference); + } + + /** + * Wraps an object so that {@link #equals(Object)} and {@link #hashCode()} delegate to an {@link + * Equivalence}. + * + *

For example, given an {@link Equivalence} for {@link String strings} named {@code equiv} + * that tests equivalence using their lengths: + * + *

{@code
+   * equiv.wrap("a").equals(equiv.wrap("b")) // true
+   * equiv.wrap("a").equals(equiv.wrap("hello")) // false
+   * }
+ * + *

Note in particular that an equivalence wrapper is never equal to the object it wraps. + * + *

{@code
+   * equiv.wrap(obj).equals(obj) // always false
+   * }
+ * + * @since 10.0 + */ + public static final class Wrapper implements Serializable { + private final Equivalence equivalence; + private final T reference; + + private Wrapper(Equivalence equivalence, T reference) { + this.equivalence = checkNotNull(equivalence); + this.reference = reference; + } + + /** Returns the (possibly null) reference wrapped by this instance. */ + + public T get() { + return reference; + } + + /** + * Returns {@code true} if {@link Equivalence#equivalent(Object, Object)} applied to the wrapped + * references is {@code true} and both wrappers use the {@link Object#equals(Object) same} + * equivalence. + */ + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof Wrapper) { + Wrapper that = (Wrapper) obj; // note: not necessarily a Wrapper + + if (this.equivalence.equals(that.equivalence)) { + /* + * We'll accept that as sufficient "proof" that either equivalence should be able to + * handle either reference, so it's safe to circumvent compile-time type checking. + */ + @SuppressWarnings("unchecked") + Equivalence equivalence = (Equivalence) this.equivalence; + return equivalence.equivalent(this.reference, that.reference); + } + } + return false; + } + + /** Returns the result of {@link Equivalence#hash(Object)} applied to the wrapped reference. */ + @Override + public int hashCode() { + return equivalence.hash(reference); + } + + /** + * Returns a string representation for this equivalence wrapper. The form of this string + * representation is not specified. + */ + @Override + public String toString() { + return equivalence + ".wrap(" + reference + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns an equivalence over iterables based on the equivalence of their elements. More + * specifically, two iterables are considered equivalent if they both contain the same number of + * elements, and each pair of corresponding elements is equivalent according to {@code this}. Null + * iterables are equivalent to one another. + * + *

Note that this method performs a similar function for equivalences as {@link + * Ordering#lexicographical} does for orderings. + * + * @since 10.0 + */ + public final Equivalence> pairwise() { + // Ideally, the returned equivalence would support Iterable. However, + // the need for this is so rare that it's not worth making callers deal with the ugly wildcard. + return new PairwiseEquivalence(this); + } + + /** + * Returns a predicate that evaluates to true if and only if the input is equivalent to {@code + * target} according to this equivalence relation. + * + * @since 10.0 + */ + public final Predicate equivalentTo(T target) { + return new EquivalentToPredicate(this, target); + } + + private static final class EquivalentToPredicate implements Predicate, Serializable { + + private final Equivalence equivalence; + private final T target; + + EquivalentToPredicate(Equivalence equivalence, T target) { + this.equivalence = checkNotNull(equivalence); + this.target = target; + } + + @Override + public boolean apply(T input) { + return equivalence.equivalent(input, target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj instanceof EquivalentToPredicate) { + EquivalentToPredicate that = (EquivalentToPredicate) obj; + return equivalence.equals(that.equivalence) && Objects.equal(target, that.target); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(equivalence, target); + } + + @Override + public String toString() { + return equivalence + ".equivalentTo(" + target + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns an equivalence that delegates to {@link Object#equals} and {@link Object#hashCode}. + * {@link Equivalence#equivalent} returns {@code true} if both values are null, or if neither + * value is null and {@link Object#equals} returns {@code true}. {@link Equivalence#hash} returns + * {@code 0} if passed a null value. + * + * @since 13.0 + * @since 8.0 (in Equivalences with null-friendly behavior) + * @since 4.0 (in Equivalences) + */ + public static Equivalence equals() { + return Equals.INSTANCE; + } + + /** + * Returns an equivalence that uses {@code ==} to compare values and {@link + * System#identityHashCode(Object)} to compute the hash code. {@link Equivalence#equivalent} + * returns {@code true} if {@code a == b}, including in the case that a and b are both null. + * + * @since 13.0 + * @since 4.0 (in Equivalences) + */ + public static Equivalence identity() { + return Identity.INSTANCE; + } + + static final class Equals extends Equivalence implements Serializable { + + static final Equals INSTANCE = new Equals(); + + @Override + protected boolean doEquivalent(Object a, Object b) { + return a.equals(b); + } + + @Override + protected int doHash(Object o) { + return o.hashCode(); + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 1; + } + + static final class Identity extends Equivalence implements Serializable { + + static final Identity INSTANCE = new Identity(); + + @Override + protected boolean doEquivalent(Object a, Object b) { + return false; + } + + @Override + protected int doHash(Object o) { + return System.identityHashCode(o); + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 1; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/ExtraObjectsMethodsForWeb.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/ExtraObjectsMethodsForWeb.java new file mode 100644 index 0000000000000..24d176beec171 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/ExtraObjectsMethodsForWeb.java @@ -0,0 +1,26 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +/** + * Holder for extra methods of {@code Objects} only in web. Intended to be empty for regular + * version. + */ +abstract class ExtraObjectsMethodsForWeb {} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizablePhantomReference.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizablePhantomReference.java new file mode 100644 index 0000000000000..f0c010098a4e2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizablePhantomReference.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.lang.ref.PhantomReference; +import java.lang.ref.ReferenceQueue; + +/** + * Phantom reference with a {@code finalizeReferent()} method which a background thread invokes + * after the garbage collector reclaims the referent. This is a simpler alternative to using a + * {@link ReferenceQueue}. + * + *

Unlike a normal phantom reference, this reference will be cleared automatically. + * + * @author Bob Lee + * @since 2.0 + */ +public abstract class FinalizablePhantomReference extends PhantomReference + implements FinalizableReference { + /** + * Constructs a new finalizable phantom reference. + * + * @param referent to phantom reference + * @param queue that should finalize the referent + */ + protected FinalizablePhantomReference(T referent, FinalizableReferenceQueue queue) { + super(referent, queue.queue); + queue.cleanUp(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReference.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReference.java new file mode 100644 index 0000000000000..d1d322e55ac5e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReference.java @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * Implemented by references that have code to run after garbage collection of their referents. + * + * @see FinalizableReferenceQueue + * @author Bob Lee + * @since 2.0 + */ +public interface FinalizableReference { + /** + * Invoked on a background thread after the referent has been garbage collected unless security + * restrictions prevented starting a background thread, in which case this method is invoked when + * new references are created. + */ + void finalizeReferent(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReferenceQueue.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReferenceQueue.java new file mode 100644 index 0000000000000..7250213114451 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableReferenceQueue.java @@ -0,0 +1,356 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.lang.ref.PhantomReference; +import java.lang.ref.Reference; +import java.lang.ref.ReferenceQueue; +import java.lang.reflect.Method; +import java.net.URL; +import java.net.URLClassLoader; + + +/** + * A reference queue with an associated background thread that dequeues references and invokes + * {@link FinalizableReference#finalizeReferent()} on them. + * + *

Keep a strong reference to this object until all of the associated referents have been + * finalized. If this object is garbage collected earlier, the backing thread will not invoke {@code + * finalizeReferent()} on the remaining references. + * + *

As an example of how this is used, imagine you have a class {@code MyServer} that creates a a + * {@link java.net.ServerSocket ServerSocket}, and you would like to ensure that the {@code + * ServerSocket} is closed even if the {@code MyServer} object is garbage-collected without calling + * its {@code close} method. You could use a finalizer to accomplish this, but that has a + * number of well-known problems. Here is how you might use this class instead: + * + *

{@code
+ * public class MyServer implements Closeable {
+ *   private static final FinalizableReferenceQueue frq = new FinalizableReferenceQueue();
+ *   // You might also share this between several objects.
+ *
+ *   private static final Set> references = Sets.newConcurrentHashSet();
+ *   // This ensures that the FinalizablePhantomReference itself is not garbage-collected.
+ *
+ *   private final ServerSocket serverSocket;
+ *
+ *   private MyServer(...) {
+ *     ...
+ *     this.serverSocket = new ServerSocket(...);
+ *     ...
+ *   }
+ *
+ *   public static MyServer create(...) {
+ *     MyServer myServer = new MyServer(...);
+ *     final ServerSocket serverSocket = myServer.serverSocket;
+ *     Reference reference = new FinalizablePhantomReference(myServer, frq) {
+ *       public void finalizeReferent() {
+ *         references.remove(this):
+ *         if (!serverSocket.isClosed()) {
+ *           ...log a message about how nobody called close()...
+ *           try {
+ *             serverSocket.close();
+ *           } catch (IOException e) {
+ *             ...
+ *           }
+ *         }
+ *       }
+ *     };
+ *     references.add(reference);
+ *     return myServer;
+ *   }
+ *
+ *   public void close() {
+ *     serverSocket.close();
+ *   }
+ * }
+ * }
+ * + * @author Bob Lee + * @since 2.0 + */ +public class FinalizableReferenceQueue implements Closeable { + /* + * The Finalizer thread keeps a phantom reference to this object. When the client (for example, a + * map built by MapMaker) no longer has a strong reference to this object, the garbage collector + * will reclaim it and enqueue the phantom reference. The enqueued reference will trigger the + * Finalizer to stop. + * + * If this library is loaded in the system class loader, FinalizableReferenceQueue can load + * Finalizer directly with no problems. + * + * If this library is loaded in an application class loader, it's important that Finalizer not + * have a strong reference back to the class loader. Otherwise, you could have a graph like this: + * + * Finalizer Thread runs instance of -> Finalizer.class loaded by -> Application class loader + * which loaded -> ReferenceMap.class which has a static -> FinalizableReferenceQueue instance + * + * Even if no other references to classes from the application class loader remain, the Finalizer + * thread keeps an indirect strong reference to the queue in ReferenceMap, which keeps the + * Finalizer running, and as a result, the application class loader can never be reclaimed. + * + * This means that dynamically loaded web applications and OSGi bundles can't be unloaded. + * + * If the library is loaded in an application class loader, we try to break the cycle by loading + * Finalizer in its own independent class loader: + * + * System class loader -> Application class loader -> ReferenceMap -> FinalizableReferenceQueue -> + * etc. -> Decoupled class loader -> Finalizer + * + * Now, Finalizer no longer keeps an indirect strong reference to the static + * FinalizableReferenceQueue field in ReferenceMap. The application class loader can be reclaimed + * at which point the Finalizer thread will stop and its decoupled class loader can also be + * reclaimed. + * + * If any of this fails along the way, we fall back to loading Finalizer directly in the + * application class loader. + * + * NOTE: The tests for this behavior (FinalizableReferenceQueueClassLoaderUnloadingTest) fail + * strangely when run in JDK 9. We are considering this a known issue. Please see + * https://github.com/google/guava/issues/3086 for more information. + */ + + private static final Logger logger = LoggerFactory.getLogger(FinalizableReferenceQueue.class.getName()); + + private static final String FINALIZER_CLASS_NAME = "com.google.common.base.internal.Finalizer"; + + /** Reference to Finalizer.startFinalizer(). */ + private static final Method startFinalizer; + + static { + Class finalizer = + loadFinalizer(new SystemLoader(), new DecoupledLoader(), new DirectLoader()); + startFinalizer = getStartFinalizer(finalizer); + } + + /** The actual reference queue that our background thread will poll. */ + final ReferenceQueue queue; + + final PhantomReference frqRef; + + /** Whether or not the background thread started successfully. */ + final boolean threadStarted; + + /** Constructs a new queue. */ + public FinalizableReferenceQueue() { + // We could start the finalizer lazily, but I'd rather it blow up early. + queue = new ReferenceQueue<>(); + frqRef = new PhantomReference(this, queue); + boolean threadStarted = false; + try { + startFinalizer.invoke(null, FinalizableReference.class, queue, frqRef); + threadStarted = true; + } catch (IllegalAccessException impossible) { + throw new AssertionError(impossible); // startFinalizer() is public + } catch (Throwable t) { + logger.info( + "Failed to start reference finalizer thread." + + " Reference cleanup will only occur when new references are created.", + t); + } + + this.threadStarted = threadStarted; + } + + @Override + public void close() { + frqRef.enqueue(); + cleanUp(); + } + + /** + * Repeatedly dequeues references from the queue and invokes {@link + * FinalizableReference#finalizeReferent()} on them until the queue is empty. This method is a + * no-op if the background thread was created successfully. + */ + void cleanUp() { + if (threadStarted) { + return; + } + + Reference reference; + while ((reference = queue.poll()) != null) { + /* + * This is for the benefit of phantom references. Weak and soft references will have already + * been cleared by this point. + */ + reference.clear(); + try { + ((FinalizableReference) reference).finalizeReferent(); + } catch (Throwable t) { + logger.error("Error cleaning up after reference.", t); + } + } + } + + /** + * Iterates through the given loaders until it finds one that can load Finalizer. + * + * @return Finalizer.class + */ + private static Class loadFinalizer(FinalizerLoader... loaders) { + for (FinalizerLoader loader : loaders) { + Class finalizer = loader.loadFinalizer(); + if (finalizer != null) { + return finalizer; + } + } + + throw new AssertionError(); + } + + /** Loads Finalizer.class. */ + interface FinalizerLoader { + + /** + * Returns Finalizer.class or null if this loader shouldn't or can't load it. + * + * @throws SecurityException if we don't have the appropriate privileges + */ + + Class loadFinalizer(); + } + + /** + * Tries to load Finalizer from the system class loader. If Finalizer is in the system class path, + * we needn't create a separate loader. + */ + static class SystemLoader implements FinalizerLoader { + // This is used by the ClassLoader-leak test in FinalizableReferenceQueueTest to disable + // finding Finalizer on the system class path even if it is there. + static boolean disabled; + + + @Override + public Class loadFinalizer() { + if (disabled) { + return null; + } + ClassLoader systemLoader; + try { + systemLoader = ClassLoader.getSystemClassLoader(); + } catch (SecurityException e) { + logger.info("Not allowed to access system class loader."); + return null; + } + if (systemLoader != null) { + try { + return systemLoader.loadClass(FINALIZER_CLASS_NAME); + } catch (ClassNotFoundException e) { + // Ignore. Finalizer is simply in a child class loader. + return null; + } + } else { + return null; + } + } + } + + /** + * Try to load Finalizer in its own class loader. If Finalizer's thread had a direct reference to + * our class loader (which could be that of a dynamically loaded web application or OSGi bundle), + * it would prevent our class loader from getting garbage collected. + */ + static class DecoupledLoader implements FinalizerLoader { + private static final String LOADING_ERROR = + "Could not load Finalizer in its own class loader. Loading Finalizer in the current class " + + "loader instead. As a result, you will not be able to garbage collect this class " + + "loader. To support reclaiming this class loader, either resolve the underlying " + + "issue, or move Guava to your system class path."; + + + @Override + public Class loadFinalizer() { + try { + /* + * We use URLClassLoader because it's the only concrete class loader implementation in the + * JDK. If we used our own ClassLoader subclass, Finalizer would indirectly reference this + * class loader: + * + * Finalizer.class -> CustomClassLoader -> CustomClassLoader.class -> This class loader + * + * System class loader will (and must) be the parent. + */ + ClassLoader finalizerLoader = newLoader(getBaseUrl()); + return finalizerLoader.loadClass(FINALIZER_CLASS_NAME); + } catch (Exception e) { + logger.warn(LOADING_ERROR, e); + return null; + } + } + + /** Gets URL for base of path containing Finalizer.class. */ + URL getBaseUrl() throws IOException { + // Find URL pointing to Finalizer.class file. + String finalizerPath = FINALIZER_CLASS_NAME.replace('.', '/') + ".class"; + URL finalizerUrl = getClass().getClassLoader().getResource(finalizerPath); + if (finalizerUrl == null) { + throw new FileNotFoundException(finalizerPath); + } + + // Find URL pointing to base of class path. + String urlString = finalizerUrl.toString(); + if (!urlString.endsWith(finalizerPath)) { + throw new IOException("Unsupported path style: " + urlString); + } + urlString = urlString.substring(0, urlString.length() - finalizerPath.length()); + return new URL(finalizerUrl, urlString); + } + + /** Creates a class loader with the given base URL as its classpath. */ + URLClassLoader newLoader(URL base) { + // We use the bootstrap class loader as the parent because Finalizer by design uses + // only standard Java classes. That also means that FinalizableReferenceQueueTest + // doesn't pick up the wrong version of the Finalizer class. + return new URLClassLoader(new URL[] {base}, null); + } + } + + /** + * Loads Finalizer directly using the current class loader. We won't be able to garbage collect + * this class loader, but at least the world doesn't end. + */ + static class DirectLoader implements FinalizerLoader { + @Override + public Class loadFinalizer() { + try { + return Class.forName(FINALIZER_CLASS_NAME); + } catch (ClassNotFoundException e) { + throw new AssertionError(e); + } + } + } + + /** Looks up Finalizer.startFinalizer(). */ + static Method getStartFinalizer(Class finalizer) { + try { + return finalizer.getMethod( + "startFinalizer", Class.class, ReferenceQueue.class, PhantomReference.class); + } catch (NoSuchMethodException e) { + throw new AssertionError(e); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableSoftReference.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableSoftReference.java new file mode 100644 index 0000000000000..2e45fd55a4b61 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableSoftReference.java @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.lang.ref.ReferenceQueue; +import java.lang.ref.SoftReference; + +/** + * Soft reference with a {@code finalizeReferent()} method which a background thread invokes after + * the garbage collector reclaims the referent. This is a simpler alternative to using a {@link + * ReferenceQueue}. + * + * @author Bob Lee + * @since 2.0 + */ +public abstract class FinalizableSoftReference extends SoftReference + implements FinalizableReference { + /** + * Constructs a new finalizable soft reference. + * + * @param referent to softly reference + * @param queue that should finalize the referent + */ + protected FinalizableSoftReference(T referent, FinalizableReferenceQueue queue) { + super(referent, queue.queue); + queue.cleanUp(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableWeakReference.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableWeakReference.java new file mode 100644 index 0000000000000..55891f815f71a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FinalizableWeakReference.java @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; + +/** + * Weak reference with a {@code finalizeReferent()} method which a background thread invokes after + * the garbage collector reclaims the referent. This is a simpler alternative to using a {@link + * ReferenceQueue}. + * + * @author Bob Lee + * @since 2.0 + */ +public abstract class FinalizableWeakReference extends WeakReference + implements FinalizableReference { + /** + * Constructs a new finalizable weak reference. + * + * @param referent to weakly reference + * @param queue that should finalize the referent + */ + protected FinalizableWeakReference(T referent, FinalizableReferenceQueue queue) { + super(referent, queue.queue); + queue.cleanUp(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Function.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Function.java new file mode 100644 index 0000000000000..83f9a1c7c2f09 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Function.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * Legacy version of {@link java.util.function.Function java.util.function.Function}. + * + *

The {@link Functions} class provides common functions and related utilities. + * + *

As this interface extends {@code java.util.function.Function}, an instance of this type can be + * used as a {@code java.util.function.Function} directly. To use a {@code + * java.util.function.Function} in a context where a {@code com.google.common.base.Function} is + * needed, use {@code function::apply}. + * + *

This interface is now a legacy type. Use {@code java.util.function.Function} (or the + * appropriate primitive specialization such as {@code ToIntFunction}) instead whenever possible. + * Otherwise, at least reduce explicit dependencies on this type by using lambda expressions + * or method references instead of classes, leaving your code easier to migrate in the future. + * + *

See the Guava User Guide article on the use of {@code Function}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +@FunctionalInterface +public interface Function extends java.util.function.Function { + @Override + + T apply(F input); + + /** + * May return {@code true} if {@code object} is a {@code Function} that behaves identically + * to this function. + * + *

Warning: do not depend on the behavior of this method. + * + *

Historically, {@code Function} instances in this library have implemented this method to + * recognize certain cases where distinct {@code Function} instances would in fact behave + * identically. However, as code migrates to {@code java.util.function}, that behavior will + * disappear. It is best not to depend on it. + */ + @Override + boolean equals(Object object); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FunctionalEquivalence.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FunctionalEquivalence.java new file mode 100644 index 0000000000000..4578a380ff0d5 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/FunctionalEquivalence.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +import java.io.Serializable; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * Equivalence applied on functional result. + * + * @author Bob Lee + * @since 10.0 + */ + +final class FunctionalEquivalence extends Equivalence implements Serializable { + + private static final long serialVersionUID = 0; + + private final Function function; + private final Equivalence resultEquivalence; + + FunctionalEquivalence(Function function, Equivalence resultEquivalence) { + this.function = checkNotNull(function); + this.resultEquivalence = checkNotNull(resultEquivalence); + } + + @Override + protected boolean doEquivalent(F a, F b) { + return resultEquivalence.equivalent(function.apply(a), function.apply(b)); + } + + @Override + protected int doHash(F a) { + return resultEquivalence.hash(function.apply(a)); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof FunctionalEquivalence) { + FunctionalEquivalence that = (FunctionalEquivalence) obj; + return function.equals(that.function) && resultEquivalence.equals(that.resultEquivalence); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(function, resultEquivalence); + } + + @Override + public String toString() { + return resultEquivalence + ".onResultOf(" + function + ")"; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Functions.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Functions.java new file mode 100644 index 0000000000000..04f0f941769f7 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Functions.java @@ -0,0 +1,405 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.Map; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * Static utility methods pertaining to {@code com.google.common.base.Function} instances; see that + * class for information about migrating to {@code java.util.function}. + * + *

All methods return serializable functions as long as they're given serializable parameters. + * + *

See the Guava User Guide article on the use of {@code Function}. + * + * @author Mike Bostock + * @author Jared Levy + * @since 2.0 + */ +public final class Functions { + private Functions() {} + + /** + * A function equivalent to the method reference {@code Object::toString}, for users not yet using + * Java 8. The function simply invokes {@code toString} on its argument and returns the result. It + * throws a {@link NullPointerException} on null input. + * + *

Warning: The returned function may not be consistent with equals (as + * documented at {@link Function#apply}). For example, this function yields different results for + * the two equal instances {@code ImmutableSet.of(1, 2)} and {@code ImmutableSet.of(2, 1)}. + * + *

Warning: as with all function types in this package, avoid depending on the specific + * {@code equals}, {@code hashCode} or {@code toString} behavior of the returned function. A + * future migration to {@code java.util.function} will not preserve this behavior. + * + *

For Java 8 users: use the method reference {@code Object::toString} instead. In the + * future, when this class requires Java 8, this method will be deprecated. See {@link Function} + * for more important information about the Java 8 transition. + */ + public static Function toStringFunction() { + return ToStringFunction.INSTANCE; + } + + // enum singleton pattern + private enum ToStringFunction implements Function { + INSTANCE; + + @Override + public String apply(Object o) { + checkNotNull(o); // eager for GWT. + return o.toString(); + } + + @Override + public String toString() { + return "Functions.toStringFunction()"; + } + } + + /** Returns the identity function. */ + // implementation is "fully variant"; E has become a "pass-through" type + @SuppressWarnings("unchecked") + public static Function identity() { + return (Function) IdentityFunction.INSTANCE; + } + + // enum singleton pattern + private enum IdentityFunction implements Function { + INSTANCE; + + @Override + + public Object apply(Object o) { + return o; + } + + @Override + public String toString() { + return "Functions.identity()"; + } + } + + /** + * Returns a function which performs a map lookup. The returned function throws an {@link + * IllegalArgumentException} if given a key that does not exist in the map. See also {@link + * #forMap(Map, Object)}, which returns a default value in this case. + * + *

Note: if {@code map} is a {@link BiMap BiMap} (or can be one), you + * can use {@link Maps#asConverter Maps.asConverter} instead to get a + * function that also supports reverse conversion. + * + *

Java 8 users: if you are okay with {@code null} being returned for an unrecognized + * key (instead of an exception being thrown), you can use the method reference {@code map::get} + * instead. + */ + public static Function forMap(Map map) { + return new FunctionForMapNoDefault<>(map); + } + + /** + * Returns a function which performs a map lookup with a default value. The function created by + * this method returns {@code defaultValue} for all inputs that do not belong to the map's key + * set. See also {@link #forMap(Map)}, which throws an exception in this case. + * + *

Java 8 users: you can just write the lambda expression {@code k -> + * map.getWithDefault(k, defaultValue)} instead. + * + * @param map source map that determines the function behavior + * @param defaultValue the value to return for inputs that aren't map keys + * @return function that returns {@code map.get(a)} when {@code a} is a key, or {@code + * defaultValue} otherwise + */ + public static Function forMap( + Map map, V defaultValue) { + return new ForMapWithDefault<>(map, defaultValue); + } + + private static class FunctionForMapNoDefault implements Function, Serializable { + final Map map; + + FunctionForMapNoDefault(Map map) { + this.map = checkNotNull(map); + } + + @Override + public V apply(K key) { + V result = map.get(key); + checkArgument(result != null || map.containsKey(key), "Key '%s' not present in map", key); + return result; + } + + @Override + public boolean equals(Object o) { + if (o instanceof FunctionForMapNoDefault) { + FunctionForMapNoDefault that = (FunctionForMapNoDefault) o; + return map.equals(that.map); + } + return false; + } + + @Override + public int hashCode() { + return map.hashCode(); + } + + @Override + public String toString() { + return "Functions.forMap(" + map + ")"; + } + + private static final long serialVersionUID = 0; + } + + private static class ForMapWithDefault implements Function, Serializable { + final Map map; + final V defaultValue; + + ForMapWithDefault(Map map, V defaultValue) { + this.map = checkNotNull(map); + this.defaultValue = defaultValue; + } + + @Override + public V apply(K key) { + V result = map.get(key); + return (result != null || map.containsKey(key)) ? result : defaultValue; + } + + @Override + public boolean equals(Object o) { + if (o instanceof ForMapWithDefault) { + ForMapWithDefault that = (ForMapWithDefault) o; + return map.equals(that.map) && Objects.equal(defaultValue, that.defaultValue); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(map, defaultValue); + } + + @Override + public String toString() { + // TODO(cpovirk): maybe remove "defaultValue=" to make this look like the method call does + return "Functions.forMap(" + map + ", defaultValue=" + defaultValue + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns the composition of two functions. For {@code f: A->B} and {@code g: B->C}, composition + * is defined as the function h such that {@code h(a) == g(f(a))} for each {@code a}. + * + *

Java 8 users: use {@code g.compose(f)} or (probably clearer) {@code f.andThen(g)} + * instead. + * + * @param g the second function to apply + * @param f the first function to apply + * @return the composition of {@code f} and {@code g} + * @see function composition + */ + public static Function compose(Function g, Function f) { + return new FunctionComposition<>(g, f); + } + + private static class FunctionComposition implements Function, Serializable { + private final Function g; + private final Function f; + + public FunctionComposition(Function g, Function f) { + this.g = checkNotNull(g); + this.f = checkNotNull(f); + } + + @Override + public C apply(A a) { + return g.apply(f.apply(a)); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof FunctionComposition) { + FunctionComposition that = (FunctionComposition) obj; + return f.equals(that.f) && g.equals(that.g); + } + return false; + } + + @Override + public int hashCode() { + return f.hashCode() ^ g.hashCode(); + } + + @Override + public String toString() { + // TODO(cpovirk): maybe make this look like the method call does ("Functions.compose(...)") + return g + "(" + f + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Creates a function that returns the same boolean output as the given predicate for all inputs. + * + *

The returned function is consistent with equals (as documented at {@link + * Function#apply}) if and only if {@code predicate} is itself consistent with equals. + * + *

Java 8 users: use the method reference {@code predicate::test} instead. + */ + public static Function forPredicate(Predicate predicate) { + return new PredicateFunction(predicate); + } + + /** @see Functions#forPredicate */ + private static class PredicateFunction implements Function, Serializable { + private final Predicate predicate; + + private PredicateFunction(Predicate predicate) { + this.predicate = checkNotNull(predicate); + } + + @Override + public Boolean apply(T t) { + return predicate.apply(t); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof PredicateFunction) { + PredicateFunction that = (PredicateFunction) obj; + return predicate.equals(that.predicate); + } + return false; + } + + @Override + public int hashCode() { + return predicate.hashCode(); + } + + @Override + public String toString() { + return "Functions.forPredicate(" + predicate + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a function that ignores its input and always returns {@code value}. + * + *

Java 8 users: use the lambda expression {@code o -> value} instead. + * + * @param value the constant value for the function to return + * @return a function that always returns {@code value} + */ + public static Function constant(E value) { + return new ConstantFunction(value); + } + + private static class ConstantFunction implements Function, Serializable { + private final E value; + + public ConstantFunction(E value) { + this.value = value; + } + + @Override + public E apply(Object from) { + return value; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ConstantFunction) { + ConstantFunction that = (ConstantFunction) obj; + return Objects.equal(value, that.value); + } + return false; + } + + @Override + public int hashCode() { + return (value == null) ? 0 : value.hashCode(); + } + + @Override + public String toString() { + return "Functions.constant(" + value + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a function that ignores its input and returns the result of {@code supplier.get()}. + * + *

Java 8 users: use the lambda expression {@code o -> supplier.get()} instead. + * + * @since 10.0 + */ + public static Function forSupplier(Supplier supplier) { + return new SupplierFunction(supplier); + } + + /** @see Functions#forSupplier */ + private static class SupplierFunction implements Function, Serializable { + + private final Supplier supplier; + + private SupplierFunction(Supplier supplier) { + this.supplier = checkNotNull(supplier); + } + + @Override + public T apply(Object input) { + return supplier.get(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof SupplierFunction) { + SupplierFunction that = (SupplierFunction) obj; + return this.supplier.equals(that.supplier); + } + return false; + } + + @Override + public int hashCode() { + return supplier.hashCode(); + } + + @Override + public String toString() { + return "Functions.forSupplier(" + supplier + ")"; + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/JdkPattern.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/JdkPattern.java new file mode 100644 index 0000000000000..25edfa95a2d54 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/JdkPattern.java @@ -0,0 +1,105 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** A regex pattern implementation which is backed by the {@link Pattern}. */ +final class JdkPattern extends CommonPattern implements Serializable { + private final Pattern pattern; + + JdkPattern(Pattern pattern) { + this.pattern = Preconditions.checkNotNull(pattern); + } + + @Override + CommonMatcher matcher(CharSequence t) { + return new JdkMatcher(pattern.matcher(t)); + } + + @Override + String pattern() { + return pattern.pattern(); + } + + @Override + int flags() { + return pattern.flags(); + } + + @Override + public String toString() { + return pattern.toString(); + } + + @Override + public int hashCode() { + return pattern.hashCode(); + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof JdkPattern)) { + return false; + } + return pattern.equals(((JdkPattern) o).pattern); + } + + private static final class JdkMatcher extends CommonMatcher { + final Matcher matcher; + + JdkMatcher(Matcher matcher) { + this.matcher = Preconditions.checkNotNull(matcher); + } + + @Override + boolean matches() { + return matcher.matches(); + } + + @Override + boolean find() { + return matcher.find(); + } + + @Override + boolean find(int index) { + return matcher.find(index); + } + + @Override + String replaceAll(String replacement) { + return matcher.replaceAll(replacement); + } + + @Override + int end() { + return matcher.end(); + } + + @Override + int start() { + return matcher.start(); + } + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Joiner.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Joiner.java new file mode 100644 index 0000000000000..1702ec042f748 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Joiner.java @@ -0,0 +1,468 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.IOException; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * An object which joins pieces of text (specified as an array, {@link Iterable}, varargs or even a + * {@link Map}) with a separator. It either appends the results to an {@link Appendable} or returns + * them as a {@link String}. Example: + * + *

{@code
+ * Joiner joiner = Joiner.on("; ").skipNulls();
+ *  . . .
+ * return joiner.join("Harry", null, "Ron", "Hermione");
+ * }
+ * + *

This returns the string {@code "Harry; Ron; Hermione"}. Note that all input elements are + * converted to strings using {@link Object#toString()} before being appended. + * + *

If neither {@link #skipNulls()} nor {@link #useForNull(String)} is specified, the joining + * methods will throw {@link NullPointerException} if any given element is null. + * + *

Warning: joiner instances are always immutable; a configuration method such as {@code + * useForNull} has no effect on the instance it is invoked on! You must store and use the new joiner + * instance returned by the method. This makes joiners thread-safe, and safe to store as {@code + * static final} constants. + * + *

{@code
+ * // Bad! Do not do this!
+ * Joiner joiner = Joiner.on(',');
+ * joiner.skipNulls(); // does nothing!
+ * return joiner.join("wrong", null, "wrong");
+ * }
+ * + *

See the Guava User Guide article on {@code Joiner}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public class Joiner { + /** Returns a joiner which automatically places {@code separator} between consecutive elements. */ + public static Joiner on(String separator) { + return new Joiner(separator); + } + + /** Returns a joiner which automatically places {@code separator} between consecutive elements. */ + public static Joiner on(char separator) { + return new Joiner(String.valueOf(separator)); + } + + private final String separator; + + private Joiner(String separator) { + this.separator = checkNotNull(separator); + } + + private Joiner(Joiner prototype) { + this.separator = prototype.separator; + } + + /** + * Appends the string representation of each of {@code parts}, using the previously configured + * separator between each, to {@code appendable}. + */ + public A appendTo(A appendable, Iterable parts) throws IOException { + return appendTo(appendable, parts.iterator()); + } + + /** + * Appends the string representation of each of {@code parts}, using the previously configured + * separator between each, to {@code appendable}. + * + * @since 11.0 + */ + public A appendTo(A appendable, Iterator parts) throws IOException { + checkNotNull(appendable); + if (parts.hasNext()) { + appendable.append(toString(parts.next())); + while (parts.hasNext()) { + appendable.append(separator); + appendable.append(toString(parts.next())); + } + } + return appendable; + } + + /** + * Appends the string representation of each of {@code parts}, using the previously configured + * separator between each, to {@code appendable}. + */ + public final A appendTo(A appendable, Object[] parts) throws IOException { + return appendTo(appendable, Arrays.asList(parts)); + } + + /** Appends to {@code appendable} the string representation of each of the remaining arguments. */ + public final A appendTo( + A appendable, Object first, Object second, Object... rest) + throws IOException { + return appendTo(appendable, iterable(first, second, rest)); + } + + /** + * Appends the string representation of each of {@code parts}, using the previously configured + * separator between each, to {@code builder}. Identical to {@link #appendTo(Appendable, + * Iterable)}, except that it does not throw {@link IOException}. + */ + public final StringBuilder appendTo(StringBuilder builder, Iterable parts) { + return appendTo(builder, parts.iterator()); + } + + /** + * Appends the string representation of each of {@code parts}, using the previously configured + * separator between each, to {@code builder}. Identical to {@link #appendTo(Appendable, + * Iterable)}, except that it does not throw {@link IOException}. + * + * @since 11.0 + */ + public final StringBuilder appendTo(StringBuilder builder, Iterator parts) { + try { + appendTo((Appendable) builder, parts); + } catch (IOException impossible) { + throw new AssertionError(impossible); + } + return builder; + } + + /** + * Appends the string representation of each of {@code parts}, using the previously configured + * separator between each, to {@code builder}. Identical to {@link #appendTo(Appendable, + * Iterable)}, except that it does not throw {@link IOException}. + */ + public final StringBuilder appendTo(StringBuilder builder, Object[] parts) { + return appendTo(builder, Arrays.asList(parts)); + } + + /** + * Appends to {@code builder} the string representation of each of the remaining arguments. + * Identical to {@link #appendTo(Appendable, Object, Object, Object...)}, except that it does not + * throw {@link IOException}. + */ + public final StringBuilder appendTo( + StringBuilder builder, + Object first, + Object second, + Object... rest) { + return appendTo(builder, iterable(first, second, rest)); + } + + /** + * Returns a string containing the string representation of each of {@code parts}, using the + * previously configured separator between each. + */ + public final String join(Iterable parts) { + return join(parts.iterator()); + } + + /** + * Returns a string containing the string representation of each of {@code parts}, using the + * previously configured separator between each. + * + * @since 11.0 + */ + public final String join(Iterator parts) { + return appendTo(new StringBuilder(), parts).toString(); + } + + /** + * Returns a string containing the string representation of each of {@code parts}, using the + * previously configured separator between each. + */ + public final String join(Object[] parts) { + return join(Arrays.asList(parts)); + } + + /** + * Returns a string containing the string representation of each argument, using the previously + * configured separator between each. + */ + public final String join( + Object first, Object second, Object... rest) { + return join(iterable(first, second, rest)); + } + + /** + * Returns a joiner with the same behavior as this one, except automatically substituting {@code + * nullText} for any provided null elements. + */ + public Joiner useForNull(final String nullText) { + checkNotNull(nullText); + return new Joiner(this) { + @Override + CharSequence toString(Object part) { + return (part == null) ? nullText : Joiner.this.toString(part); + } + + @Override + public Joiner useForNull(String nullText) { + throw new UnsupportedOperationException("already specified useForNull"); + } + + @Override + public Joiner skipNulls() { + throw new UnsupportedOperationException("already specified useForNull"); + } + }; + } + + /** + * Returns a joiner with the same behavior as this joiner, except automatically skipping over any + * provided null elements. + */ + public Joiner skipNulls() { + return new Joiner(this) { + @Override + public A appendTo(A appendable, Iterator parts) throws IOException { + checkNotNull(appendable, "appendable"); + checkNotNull(parts, "parts"); + while (parts.hasNext()) { + Object part = parts.next(); + if (part != null) { + appendable.append(Joiner.this.toString(part)); + break; + } + } + while (parts.hasNext()) { + Object part = parts.next(); + if (part != null) { + appendable.append(separator); + appendable.append(Joiner.this.toString(part)); + } + } + return appendable; + } + + @Override + public Joiner useForNull(String nullText) { + throw new UnsupportedOperationException("already specified skipNulls"); + } + + @Override + public MapJoiner withKeyValueSeparator(String kvs) { + throw new UnsupportedOperationException("can't use .skipNulls() with maps"); + } + }; + } + + /** + * Returns a {@code MapJoiner} using the given key-value separator, and the same configuration as + * this {@code Joiner} otherwise. + * + * @since 20.0 + */ + public MapJoiner withKeyValueSeparator(char keyValueSeparator) { + return withKeyValueSeparator(String.valueOf(keyValueSeparator)); + } + + /** + * Returns a {@code MapJoiner} using the given key-value separator, and the same configuration as + * this {@code Joiner} otherwise. + */ + public MapJoiner withKeyValueSeparator(String keyValueSeparator) { + return new MapJoiner(this, keyValueSeparator); + } + + /** + * An object that joins map entries in the same manner as {@code Joiner} joins iterables and + * arrays. Like {@code Joiner}, it is thread-safe and immutable. + * + *

In addition to operating on {@code Map} instances, {@code MapJoiner} can operate on {@code + * Multimap} entries in two distinct modes: + * + *

+ * + * @since 2.0 + */ + public static final class MapJoiner { + private final Joiner joiner; + private final String keyValueSeparator; + + private MapJoiner(Joiner joiner, String keyValueSeparator) { + this.joiner = joiner; // only "this" is ever passed, so don't checkNotNull + this.keyValueSeparator = checkNotNull(keyValueSeparator); + } + + /** + * Appends the string representation of each entry of {@code map}, using the previously + * configured separator and key-value separator, to {@code appendable}. + */ + public A appendTo(A appendable, Map map) throws IOException { + return appendTo(appendable, map.entrySet()); + } + + /** + * Appends the string representation of each entry of {@code map}, using the previously + * configured separator and key-value separator, to {@code builder}. Identical to {@link + * #appendTo(Appendable, Map)}, except that it does not throw {@link IOException}. + */ + public StringBuilder appendTo(StringBuilder builder, Map map) { + return appendTo(builder, map.entrySet()); + } + + /** + * Appends the string representation of each entry in {@code entries}, using the previously + * configured separator and key-value separator, to {@code appendable}. + * + * @since 10.0 + */ + + public A appendTo(A appendable, Iterable> entries) + throws IOException { + return appendTo(appendable, entries.iterator()); + } + + /** + * Appends the string representation of each entry in {@code entries}, using the previously + * configured separator and key-value separator, to {@code appendable}. + * + * @since 11.0 + */ + + public A appendTo(A appendable, Iterator> parts) + throws IOException { + checkNotNull(appendable); + if (parts.hasNext()) { + Entry entry = parts.next(); + appendable.append(joiner.toString(entry.getKey())); + appendable.append(keyValueSeparator); + appendable.append(joiner.toString(entry.getValue())); + while (parts.hasNext()) { + appendable.append(joiner.separator); + Entry e = parts.next(); + appendable.append(joiner.toString(e.getKey())); + appendable.append(keyValueSeparator); + appendable.append(joiner.toString(e.getValue())); + } + } + return appendable; + } + + /** + * Appends the string representation of each entry in {@code entries}, using the previously + * configured separator and key-value separator, to {@code builder}. Identical to {@link + * #appendTo(Appendable, Iterable)}, except that it does not throw {@link IOException}. + * + * @since 10.0 + */ + + public StringBuilder appendTo(StringBuilder builder, Iterable> entries) { + return appendTo(builder, entries.iterator()); + } + + /** + * Appends the string representation of each entry in {@code entries}, using the previously + * configured separator and key-value separator, to {@code builder}. Identical to {@link + * #appendTo(Appendable, Iterable)}, except that it does not throw {@link IOException}. + * + * @since 11.0 + */ + + public StringBuilder appendTo(StringBuilder builder, Iterator> entries) { + try { + appendTo((Appendable) builder, entries); + } catch (IOException impossible) { + throw new AssertionError(impossible); + } + return builder; + } + + /** + * Returns a string containing the string representation of each entry of {@code map}, using the + * previously configured separator and key-value separator. + */ + public String join(Map map) { + return join(map.entrySet()); + } + + /** + * Returns a string containing the string representation of each entry in {@code entries}, using + * the previously configured separator and key-value separator. + * + * @since 10.0 + */ + + public String join(Iterable> entries) { + return join(entries.iterator()); + } + + /** + * Returns a string containing the string representation of each entry in {@code entries}, using + * the previously configured separator and key-value separator. + * + * @since 11.0 + */ + + public String join(Iterator> entries) { + return appendTo(new StringBuilder(), entries).toString(); + } + + /** + * Returns a map joiner with the same behavior as this one, except automatically substituting + * {@code nullText} for any provided null keys or values. + */ + public MapJoiner useForNull(String nullText) { + return new MapJoiner(joiner.useForNull(nullText), keyValueSeparator); + } + } + + CharSequence toString(Object part) { + checkNotNull(part); // checkNotNull for GWT (do not optimize). + return (part instanceof CharSequence) ? (CharSequence) part : part.toString(); + } + + private static Iterable iterable( + final Object first, final Object second, final Object[] rest) { + checkNotNull(rest); + return new AbstractList() { + @Override + public int size() { + return rest.length + 2; + } + + @Override + public Object get(int index) { + switch (index) { + case 0: + return first; + case 1: + return second; + default: + return rest[index - 2]; + } + } + }; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/MoreObjects.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/MoreObjects.java new file mode 100644 index 0000000000000..809a5ccb05016 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/MoreObjects.java @@ -0,0 +1,373 @@ +/* + * Copyright (C) 2014 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.Arrays; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * Helper functions that operate on any {@code Object}, and are not already provided in {@link + * java.util.Objects}. + * + *

See the Guava User Guide on writing {@code Object} + * methods with {@code MoreObjects}. + * + * @author Laurence Gonsalves + * @since 18.0 (since 2.0 as {@code Objects}) + */ +public final class MoreObjects { + /** + * Returns the first of two given parameters that is not {@code null}, if either is, or otherwise + * throws a {@link NullPointerException}. + * + *

To find the first non-null element in an iterable, use {@code Iterables.find(iterable, + * Predicates.notNull())}. For varargs, use {@code Iterables.find(Arrays.asList(a, b, c, ...), + * Predicates.notNull())}, static importing as necessary. + * + *

Note: if {@code first} is represented as an {@link Optional}, this can be + * accomplished with {@link Optional#or(Object) first.or(second)}. That approach also allows for + * lazy evaluation of the fallback instance, using {@link Optional#or(Supplier) + * first.or(supplier)}. + * + * @return {@code first} if it is non-null; otherwise {@code second} if it is non-null + * @throws NullPointerException if both {@code first} and {@code second} are null + * @since 18.0 (since 3.0 as {@code Objects.firstNonNull()}). + */ + public static T firstNonNull(T first, T second) { + if (first != null) { + return first; + } + if (second != null) { + return second; + } + throw new NullPointerException("Both parameters are null"); + } + + /** + * Creates an instance of {@link ToStringHelper}. + * + *

This is helpful for implementing {@link Object#toString()}. Specification by example: + * + *

{@code
+   * // Returns "ClassName{}"
+   * MoreObjects.toStringHelper(this)
+   *     .toString();
+   *
+   * // Returns "ClassName{x=1}"
+   * MoreObjects.toStringHelper(this)
+   *     .add("x", 1)
+   *     .toString();
+   *
+   * // Returns "MyObject{x=1}"
+   * MoreObjects.toStringHelper("MyObject")
+   *     .add("x", 1)
+   *     .toString();
+   *
+   * // Returns "ClassName{x=1, y=foo}"
+   * MoreObjects.toStringHelper(this)
+   *     .add("x", 1)
+   *     .add("y", "foo")
+   *     .toString();
+   *
+   * // Returns "ClassName{x=1}"
+   * MoreObjects.toStringHelper(this)
+   *     .omitNullValues()
+   *     .add("x", 1)
+   *     .add("y", null)
+   *     .toString();
+   * }
+ * + *

Note that in GWT, class names are often obfuscated. + * + * @param self the object to generate the string for (typically {@code this}), used only for its + * class name + * @since 18.0 (since 2.0 as {@code Objects.toStringHelper()}). + */ + public static ToStringHelper toStringHelper(Object self) { + return new ToStringHelper(self.getClass().getSimpleName()); + } + + /** + * Creates an instance of {@link ToStringHelper} in the same manner as {@link + * #toStringHelper(Object)}, but using the simple name of {@code clazz} instead of using an + * instance's {@link Object#getClass()}. + * + *

Note that in GWT, class names are often obfuscated. + * + * @param clazz the {@link Class} of the instance + * @since 18.0 (since 7.0 as {@code Objects.toStringHelper()}). + */ + public static ToStringHelper toStringHelper(Class clazz) { + return new ToStringHelper(clazz.getSimpleName()); + } + + /** + * Creates an instance of {@link ToStringHelper} in the same manner as {@link + * #toStringHelper(Object)}, but using {@code className} instead of using an instance's {@link + * Object#getClass()}. + * + * @param className the name of the instance type + * @since 18.0 (since 7.0 as {@code Objects.toStringHelper()}). + */ + public static ToStringHelper toStringHelper(String className) { + return new ToStringHelper(className); + } + + /** + * Support class for {@link MoreObjects#toStringHelper}. + * + * @author Jason Lee + * @since 18.0 (since 2.0 as {@code Objects.ToStringHelper}). + */ + public static final class ToStringHelper { + private final String className; + private final ValueHolder holderHead = new ValueHolder(); + private ValueHolder holderTail = holderHead; + private boolean omitNullValues = false; + + /** Use {@link MoreObjects#toStringHelper(Object)} to create an instance. */ + private ToStringHelper(String className) { + this.className = checkNotNull(className); + } + + /** + * Configures the {@link ToStringHelper} so {@link #toString()} will ignore properties with null + * value. The order of calling this method, relative to the {@code add()}/{@code addValue()} + * methods, is not significant. + * + * @since 18.0 (since 12.0 as {@code Objects.ToStringHelper.omitNullValues()}). + */ + public ToStringHelper omitNullValues() { + omitNullValues = true; + return this; + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. If {@code value} + * is {@code null}, the string {@code "null"} is used, unless {@link #omitNullValues()} is + * called, in which case this name/value pair will not be added. + */ + public ToStringHelper add(String name, Object value) { + return addHolder(name, value); + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.add()}). + */ + public ToStringHelper add(String name, boolean value) { + return addHolder(name, String.valueOf(value)); + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.add()}). + */ + public ToStringHelper add(String name, char value) { + return addHolder(name, String.valueOf(value)); + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.add()}). + */ + public ToStringHelper add(String name, double value) { + return addHolder(name, String.valueOf(value)); + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.add()}). + */ + public ToStringHelper add(String name, float value) { + return addHolder(name, String.valueOf(value)); + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.add()}). + */ + public ToStringHelper add(String name, int value) { + return addHolder(name, String.valueOf(value)); + } + + /** + * Adds a name/value pair to the formatted output in {@code name=value} format. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.add()}). + */ + public ToStringHelper add(String name, long value) { + return addHolder(name, String.valueOf(value)); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, Object)} instead and give value a + * readable name. + */ + public ToStringHelper addValue(Object value) { + return addHolder(value); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, boolean)} instead and give value a + * readable name. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.addValue()}). + */ + public ToStringHelper addValue(boolean value) { + return addHolder(String.valueOf(value)); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, char)} instead and give value a + * readable name. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.addValue()}). + */ + public ToStringHelper addValue(char value) { + return addHolder(String.valueOf(value)); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, double)} instead and give value a + * readable name. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.addValue()}). + */ + public ToStringHelper addValue(double value) { + return addHolder(String.valueOf(value)); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, float)} instead and give value a + * readable name. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.addValue()}). + */ + public ToStringHelper addValue(float value) { + return addHolder(String.valueOf(value)); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, int)} instead and give value a + * readable name. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.addValue()}). + */ + public ToStringHelper addValue(int value) { + return addHolder(String.valueOf(value)); + } + + /** + * Adds an unnamed value to the formatted output. + * + *

It is strongly encouraged to use {@link #add(String, long)} instead and give value a + * readable name. + * + * @since 18.0 (since 11.0 as {@code Objects.ToStringHelper.addValue()}). + */ + public ToStringHelper addValue(long value) { + return addHolder(String.valueOf(value)); + } + + /** + * Returns a string in the format specified by {@link MoreObjects#toStringHelper(Object)}. + * + *

After calling this method, you can keep adding more properties to later call toString() + * again and get a more complete representation of the same object; but properties cannot be + * removed, so this only allows limited reuse of the helper instance. The helper allows + * duplication of properties (multiple name/value pairs with the same name can be added). + */ + @Override + public String toString() { + // create a copy to keep it consistent in case value changes + boolean omitNullValuesSnapshot = omitNullValues; + String nextSeparator = ""; + StringBuilder builder = new StringBuilder(32).append(className).append('{'); + for (ValueHolder valueHolder = holderHead.next; + valueHolder != null; + valueHolder = valueHolder.next) { + Object value = valueHolder.value; + if (!omitNullValuesSnapshot || value != null) { + builder.append(nextSeparator); + nextSeparator = ", "; + + if (valueHolder.name != null) { + builder.append(valueHolder.name).append('='); + } + if (value != null && value.getClass().isArray()) { + Object[] objectArray = {value}; + String arrayString = Arrays.deepToString(objectArray); + builder.append(arrayString, 1, arrayString.length() - 1); + } else { + builder.append(value); + } + } + } + return builder.append('}').toString(); + } + + private ValueHolder addHolder() { + ValueHolder valueHolder = new ValueHolder(); + holderTail = holderTail.next = valueHolder; + return valueHolder; + } + + private ToStringHelper addHolder(Object value) { + ValueHolder valueHolder = addHolder(); + valueHolder.value = value; + return this; + } + + private ToStringHelper addHolder(String name, Object value) { + ValueHolder valueHolder = addHolder(); + valueHolder.value = value; + valueHolder.name = checkNotNull(name); + return this; + } + + private static final class ValueHolder { + String name; + Object value; + ValueHolder next; + } + } + + private MoreObjects() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Objects.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Objects.java new file mode 100644 index 0000000000000..79cdb9735fa34 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Objects.java @@ -0,0 +1,81 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.Arrays; + + +/** + * Helper functions that can operate on any {@code Object}. + * + *

See the Guava User Guide on writing {@code Object} + * methods with {@code Objects}. + * + * @author Laurence Gonsalves + * @since 2.0 + */ +public final class Objects extends ExtraObjectsMethodsForWeb { + private Objects() {} + + /** + * Determines whether two possibly-null objects are equal. Returns: + * + *

    + *
  • {@code true} if {@code a} and {@code b} are both null. + *
  • {@code true} if {@code a} and {@code b} are both non-null and they are equal according to + * {@link Object#equals(Object)}. + *
  • {@code false} in all other situations. + *
+ * + *

This assumes that any non-null objects passed to this function conform to the {@code + * equals()} contract. + * + *

Note for Java 7 and later: This method should be treated as deprecated; use {@link + * java.util.Objects#equals} instead. + */ + public static boolean equal(Object a, Object b) { + return a == b || (a != null && a.equals(b)); + } + + /** + * Generates a hash code for multiple values. The hash code is generated by calling {@link + * Arrays#hashCode(Object[])}. Note that array arguments to this method, with the exception of a + * single Object array, do not get any special handling; their hash codes are based on identity + * and not contents. + * + *

This is useful for implementing {@link Object#hashCode()}. For example, in an object that + * has three properties, {@code x}, {@code y}, and {@code z}, one could write: + * + *

{@code
+   * public int hashCode() {
+   *   return Objects.hashCode(getX(), getY(), getZ());
+   * }
+   * }
+ * + *

Warning: When a single object is supplied, the returned hash code does not equal the + * hash code of that object. + * + *

Note for Java 7 and later: This method should be treated as deprecated; use {@link + * java.util.Objects#hash} instead. + */ + public static int hashCode(Object... objects) { + return Arrays.hashCode(objects); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Optional.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Optional.java new file mode 100644 index 0000000000000..7b0415726249f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Optional.java @@ -0,0 +1,355 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.Iterator; +import java.util.Set; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * An immutable object that may contain a non-null reference to another object. Each instance of + * this type either contains a non-null reference, or contains nothing (in which case we say that + * the reference is "absent"); it is never said to "contain {@code null}". + * + *

A non-null {@code Optional} reference can be used as a replacement for a nullable {@code T} + * reference. It allows you to represent "a {@code T} that must be present" and a "a {@code T} that + * might be absent" as two distinct types in your program, which can aid clarity. + * + *

Some uses of this class include + * + *

    + *
  • As a method return type, as an alternative to returning {@code null} to indicate that no + * value was available + *
  • To distinguish between "unknown" (for example, not present in a map) and "known to have no + * value" (present in the map, with value {@code Optional.absent()}) + *
  • To wrap nullable references for storage in a collection that does not support {@code null} + * (though there are several other + * approaches to this that should be considered first) + *
+ * + *

A common alternative to using this class is to find or create a suitable null object for the type in question. + * + *

This class is not intended as a direct analogue of any existing "option" or "maybe" construct + * from other programming environments, though it may bear some similarities. + * + *

Comparison to {@code java.util.Optional} (JDK 8 and higher): A new {@code Optional} + * class was added for Java 8. The two classes are extremely similar, but incompatible (they cannot + * share a common supertype). All known differences are listed either here or with the + * relevant methods below. + * + *

    + *
  • This class is serializable; {@code java.util.Optional} is not. + *
  • {@code java.util.Optional} has the additional methods {@code ifPresent}, {@code filter}, + * {@code flatMap}, and {@code orElseThrow}. + *
  • {@code java.util} offers the primitive-specialized versions {@code OptionalInt}, {@code + * OptionalLong} and {@code OptionalDouble}, the use of which is recommended; Guava does not + * have these. + *
+ * + *

There are no plans to deprecate this class in the foreseeable future. However, we do + * gently recommend that you prefer the new, standard Java class whenever possible. + * + *

See the Guava User Guide article on using {@code + * Optional}. + * + * @param the type of instance that can be contained. {@code Optional} is naturally covariant on + * this type, so it is safe to cast an {@code Optional} to {@code Optional} for any + * supertype {@code S} of {@code T}. + * @author Kurt Alfred Kluever + * @author Kevin Bourrillion + * @since 10.0 + */ +public abstract class Optional implements Serializable { + /** + * Returns an {@code Optional} instance with no contained reference. + * + *

Comparison to {@code java.util.Optional}: this method is equivalent to Java 8's + * {@code Optional.empty}. + */ + public static Optional absent() { + return Absent.withType(); + } + + /** + * Returns an {@code Optional} instance containing the given non-null reference. To have {@code + * null} treated as {@link #absent}, use {@link #fromNullable} instead. + * + *

Comparison to {@code java.util.Optional}: no differences. + * + * @throws NullPointerException if {@code reference} is null + */ + public static Optional of(T reference) { + return new Present(checkNotNull(reference)); + } + + /** + * If {@code nullableReference} is non-null, returns an {@code Optional} instance containing that + * reference; otherwise returns {@link Optional#absent}. + * + *

Comparison to {@code java.util.Optional}: this method is equivalent to Java 8's + * {@code Optional.ofNullable}. + */ + public static Optional fromNullable(T nullableReference) { + return (nullableReference == null) ? Optional.absent() : new Present(nullableReference); + } + + /** + * Returns the equivalent {@code com.google.common.base.Optional} value to the given {@code + * java.util.Optional}, or {@code null} if the argument is null. + * + * @since 21.0 + */ + + public static Optional fromJavaUtil(java.util.Optional javaUtilOptional) { + return (javaUtilOptional == null) ? null : fromNullable(javaUtilOptional.orElse(null)); + } + + /** + * Returns the equivalent {@code java.util.Optional} value to the given {@code + * com.google.common.base.Optional}, or {@code null} if the argument is null. + * + *

If {@code googleOptional} is known to be non-null, use {@code googleOptional.toJavaUtil()} + * instead. + * + *

Unfortunately, the method reference {@code Optional::toJavaUtil} will not work, because it + * could refer to either the static or instance version of this method. Write out the lambda + * expression {@code o -> Optional.toJavaUtil(o)} instead. + * + * @since 21.0 + */ + + public static java.util.Optional toJavaUtil(Optional googleOptional) { + return googleOptional == null ? null : googleOptional.toJavaUtil(); + } + + /** + * Returns the equivalent {@code java.util.Optional} value to this optional. + * + *

Unfortunately, the method reference {@code Optional::toJavaUtil} will not work, because it + * could refer to either the static or instance version of this method. Write out the lambda + * expression {@code o -> o.toJavaUtil()} instead. + * + * @since 21.0 + */ + public java.util.Optional toJavaUtil() { + return java.util.Optional.ofNullable(orNull()); + } + + Optional() {} + + /** + * Returns {@code true} if this holder contains a (non-null) instance. + * + *

Comparison to {@code java.util.Optional}: no differences. + */ + public abstract boolean isPresent(); + + /** + * Returns the contained instance, which must be present. If the instance might be absent, use + * {@link #or(Object)} or {@link #orNull} instead. + * + *

Comparison to {@code java.util.Optional}: when the value is absent, this method + * throws {@link IllegalStateException}, whereas the Java 8 counterpart throws {@link + * java.util.NoSuchElementException NoSuchElementException}. + * + * @throws IllegalStateException if the instance is absent ({@link #isPresent} returns {@code + * false}); depending on this specific exception type (over the more general {@link + * RuntimeException}) is discouraged + */ + public abstract T get(); + + /** + * Returns the contained instance if it is present; {@code defaultValue} otherwise. If no default + * value should be required because the instance is known to be present, use {@link #get()} + * instead. For a default value of {@code null}, use {@link #orNull}. + * + *

Note about generics: The signature {@code public T or(T defaultValue)} is overly + * restrictive. However, the ideal signature, {@code public S or(S)}, is not legal + * Java. As a result, some sensible operations involving subtypes are compile errors: + * + *

{@code
+   * Optional optionalInt = getSomeOptionalInt();
+   * Number value = optionalInt.or(0.5); // error
+   *
+   * FluentIterable numbers = getSomeNumbers();
+   * Optional first = numbers.first();
+   * Number value = first.or(0.5); // error
+   * }
+ * + *

As a workaround, it is always safe to cast an {@code Optional} to {@code + * Optional}. Casting either of the above example {@code Optional} instances to {@code + * Optional} (where {@code Number} is the desired output type) solves the problem: + * + *

{@code
+   * Optional optionalInt = (Optional) getSomeOptionalInt();
+   * Number value = optionalInt.or(0.5); // fine
+   *
+   * FluentIterable numbers = getSomeNumbers();
+   * Optional first = (Optional) numbers.first();
+   * Number value = first.or(0.5); // fine
+   * }
+ * + *

Comparison to {@code java.util.Optional}: this method is similar to Java 8's {@code + * Optional.orElse}, but will not accept {@code null} as a {@code defaultValue} ({@link #orNull} + * must be used instead). As a result, the value returned by this method is guaranteed non-null, + * which is not the case for the {@code java.util} equivalent. + */ + public abstract T or(T defaultValue); + + /** + * Returns this {@code Optional} if it has a value present; {@code secondChoice} otherwise. + * + *

Comparison to {@code java.util.Optional}: this method has no equivalent in Java 8's + * {@code Optional} class; write {@code thisOptional.isPresent() ? thisOptional : secondChoice} + * instead. + */ + public abstract Optional or(Optional secondChoice); + + /** + * Returns the contained instance if it is present; {@code supplier.get()} otherwise. + * + *

Comparison to {@code java.util.Optional}: this method is similar to Java 8's {@code + * Optional.orElseGet}, except when {@code supplier} returns {@code null}. In this case this + * method throws an exception, whereas the Java 8 method returns the {@code null} to the caller. + * + * @throws NullPointerException if this optional's value is absent and the supplier returns {@code + * null} + */ + + public abstract T or(Supplier supplier); + + /** + * Returns the contained instance if it is present; {@code null} otherwise. If the instance is + * known to be present, use {@link #get()} instead. + * + *

Comparison to {@code java.util.Optional}: this method is equivalent to Java 8's + * {@code Optional.orElse(null)}. + */ + + public abstract T orNull(); + + /** + * Returns an immutable singleton {@link Set} whose only element is the contained instance if it + * is present; an empty immutable {@link Set} otherwise. + * + *

Comparison to {@code java.util.Optional}: this method has no equivalent in Java 8's + * {@code Optional} class. However, this common usage: + * + *

{@code
+   * for (Foo foo : possibleFoo.asSet()) {
+   *   doSomethingWith(foo);
+   * }
+   * }
+ * + * ... can be replaced with: + * + *
{@code
+   * possibleFoo.ifPresent(foo -> doSomethingWith(foo));
+   * }
+ * + * @since 11.0 + */ + public abstract Set asSet(); + + /** + * If the instance is present, it is transformed with the given {@link Function}; otherwise, + * {@link Optional#absent} is returned. + * + *

Comparison to {@code java.util.Optional}: this method is similar to Java 8's {@code + * Optional.map}, except when {@code function} returns {@code null}. In this case this method + * throws an exception, whereas the Java 8 method returns {@code Optional.absent()}. + * + * @throws NullPointerException if the function returns {@code null} + * @since 12.0 + */ + public abstract Optional transform(Function function); + + /** + * Returns {@code true} if {@code object} is an {@code Optional} instance, and either the + * contained references are {@linkplain Object#equals equal} to each other or both are absent. + * Note that {@code Optional} instances of differing parameterized types can be equal. + * + *

Comparison to {@code java.util.Optional}: no differences. + */ + @Override + public abstract boolean equals(Object object); + + /** + * Returns a hash code for this instance. + * + *

Comparison to {@code java.util.Optional}: this class leaves the specific choice of + * hash code unspecified, unlike the Java 8 equivalent. + */ + @Override + public abstract int hashCode(); + + /** + * Returns a string representation for this instance. + * + *

Comparison to {@code java.util.Optional}: this class leaves the specific string + * representation unspecified, unlike the Java 8 equivalent. + */ + @Override + public abstract String toString(); + + /** + * Returns the value of each present instance from the supplied {@code optionals}, in order, + * skipping over occurrences of {@link Optional#absent}. Iterators are unmodifiable and are + * evaluated lazily. + * + *

Comparison to {@code java.util.Optional}: this method has no equivalent in Java 8's + * {@code Optional} class; use {@code + * optionals.stream().filter(Optional::isPresent).map(Optional::get)} instead. + * + * @since 11.0 (generics widened in 13.0) + */ + + public static Iterable presentInstances( + final Iterable> optionals) { + checkNotNull(optionals); + return new Iterable() { + @Override + public Iterator iterator() { + return new AbstractIterator() { + private final Iterator> iterator = + checkNotNull(optionals.iterator()); + + @Override + protected T computeNext() { + while (iterator.hasNext()) { + Optional optional = iterator.next(); + if (optional.isPresent()) { + return optional.get(); + } + } + return endOfData(); + } + }; + } + }; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PairwiseEquivalence.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PairwiseEquivalence.java new file mode 100644 index 0000000000000..aa59d1eb4b254 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PairwiseEquivalence.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.Iterator; + + +final class PairwiseEquivalence extends Equivalence> implements Serializable { + + final Equivalence elementEquivalence; + + PairwiseEquivalence(Equivalence elementEquivalence) { + this.elementEquivalence = Preconditions.checkNotNull(elementEquivalence); + } + + @Override + protected boolean doEquivalent(Iterable iterableA, Iterable iterableB) { + Iterator iteratorA = iterableA.iterator(); + Iterator iteratorB = iterableB.iterator(); + + while (iteratorA.hasNext() && iteratorB.hasNext()) { + if (!elementEquivalence.equivalent(iteratorA.next(), iteratorB.next())) { + return false; + } + } + + return !iteratorA.hasNext() && !iteratorB.hasNext(); + } + + @Override + protected int doHash(Iterable iterable) { + int hash = 78721; + for (T element : iterable) { + hash = hash * 24943 + elementEquivalence.hash(element); + } + return hash; + } + + @Override + public boolean equals(Object object) { + if (object instanceof PairwiseEquivalence) { + PairwiseEquivalence that = (PairwiseEquivalence) object; + return this.elementEquivalence.equals(that.elementEquivalence); + } + + return false; + } + + @Override + public int hashCode() { + return elementEquivalence.hashCode() ^ 0x46a3eb07; + } + + @Override + public String toString() { + return elementEquivalence + ".pairwise()"; + } + + private static final long serialVersionUID = 1; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PatternCompiler.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PatternCompiler.java new file mode 100644 index 0000000000000..8ef1bec251b50 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/PatternCompiler.java @@ -0,0 +1,33 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * Pluggable interface for compiling a regex pattern. By default this package uses the {@code + * java.util.regex} library, but an alternate implementation can be supplied using the {@link + * java.util.ServiceLoader} mechanism. + */ +public interface PatternCompiler { + /** + * Compiles the given pattern. + * + * @throws IllegalArgumentException if the pattern is invalid + */ + CommonPattern compile(String pattern); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Platform.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Platform.java new file mode 100644 index 0000000000000..f9c0e52400e16 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Platform.java @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.ref.WeakReference; +import java.util.Iterator; +import java.util.Locale; +import java.util.ServiceConfigurationError; +import java.util.ServiceLoader; +import java.util.regex.Pattern; + + +/** + * Methods factored out so that they can be emulated differently in GWT. + * + * @author Jesse Wilson + */ +final class Platform { + private static final Logger logger = LoggerFactory.getLogger(Platform.class.getName()); + private static final PatternCompiler patternCompiler = loadPatternCompiler(); + + private Platform() {} + + /** Calls {@link System#nanoTime()}. */ + static long systemNanoTime() { + return System.nanoTime(); + } + + static CharMatcher precomputeCharMatcher(CharMatcher matcher) { + return matcher.precomputedInternal(); + } + + static > Optional getEnumIfPresent(Class enumClass, String value) { + WeakReference> ref = Enums.getEnumConstants(enumClass).get(value); + return ref == null ? Optional.absent() : Optional.of(enumClass.cast(ref.get())); + } + + static String formatCompact4Digits(double value) { + return String.format(Locale.ROOT, "%.4g", value); + } + + static boolean stringIsNullOrEmpty(String string) { + return string == null || string.isEmpty(); + } + + static String nullToEmpty(String string) { + return (string == null) ? "" : string; + } + + static String emptyToNull(String string) { + return stringIsNullOrEmpty(string) ? null : string; + } + + static CommonPattern compilePattern(String pattern) { + Preconditions.checkNotNull(pattern); + return patternCompiler.compile(pattern); + } + + static boolean usingJdkPatternCompiler() { + return patternCompiler instanceof JdkPatternCompiler; + } + + private static PatternCompiler loadPatternCompiler() { + ServiceLoader loader = ServiceLoader.load(PatternCompiler.class); + // Returns the first PatternCompiler that loads successfully. + try { + for (Iterator it = loader.iterator(); it.hasNext(); ) { + try { + return it.next(); + } catch (ServiceConfigurationError e) { + logPatternCompilerError(e); + } + } + } catch (ServiceConfigurationError e) { // from hasNext() + logPatternCompilerError(e); + } + // Fall back to the JDK regex library. + return new JdkPatternCompiler(); + } + + private static void logPatternCompilerError(ServiceConfigurationError e) { + logger.warn("Error loading regex compiler, falling back to next option", e); + } + + private static final class JdkPatternCompiler implements PatternCompiler { + @Override + public CommonPattern compile(String pattern) { + return new JdkPattern(Pattern.compile(pattern)); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Preconditions.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Preconditions.java new file mode 100644 index 0000000000000..58465650c0107 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Preconditions.java @@ -0,0 +1,1454 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +/** + * Static convenience methods that help a method or constructor check whether it was invoked + * correctly (that is, whether its preconditions were met). + * + *

If the precondition is not met, the {@code Preconditions} method throws an unchecked exception + * of a specified type, which helps the method in which the exception was thrown communicate that + * its caller has made a mistake. This allows constructs such as + * + *

{@code
+ * public static double sqrt(double value) {
+ *   if (value < 0) {
+ *     throw new IllegalArgumentException("input is negative: " + value);
+ *   }
+ *   // calculate square root
+ * }
+ * }
+ * + *

to be replaced with the more compact + * + *

{@code
+ * public static double sqrt(double value) {
+ *   checkArgument(value >= 0, "input is negative: %s", value);
+ *   // calculate square root
+ * }
+ * }
+ * + *

so that a hypothetical bad caller of this method, such as: + * + *

{@code
+ *   void exampleBadCaller() {
+ *     double d = sqrt(-1.0);
+ * }
+ * }
+ * + *

would be flagged as having called {@code sqrt()} with an illegal argument. + * + *

Performance

+ * + *

Avoid passing message arguments that are expensive to compute; your code will always compute + * them, even though they usually won't be needed. If you have such arguments, use the conventional + * if/throw idiom instead. + * + *

Depending on your message arguments, memory may be allocated for boxing and varargs array + * creation. However, the methods of this class have a large number of overloads that prevent such + * allocations in many common cases. + * + *

The message string is not formatted unless the exception will be thrown, so the cost of the + * string formatting itself should not be a concern. + * + *

As with any performance concerns, you should consider profiling your code (in a production + * environment if possible) before spending a lot of effort on tweaking a particular element. + * + *

Other types of preconditions

+ * + *

Not every type of precondition failure is supported by these methods. Continue to throw + * standard JDK exceptions such as {@link java.util.NoSuchElementException} or {@link + * UnsupportedOperationException} in the situations they are intended for. + * + *

Non-preconditions

+ * + *

It is of course possible to use the methods of this class to check for invalid conditions + * which are not the caller's fault. Doing so is not recommended because it is + * misleading to future readers of the code and of stack traces. See Conditional failures + * explained in the Guava User Guide for more advice. Notably, {@link Verify} offers assertions + * similar to those in this class for non-precondition checks. + * + *

{@code java.util.Objects.requireNonNull()}

+ * + *

Projects which use {@code com.google.common} should generally avoid the use of {@link + * java.util.Objects#requireNonNull(Object)}. Instead, use whichever of {@link + * #checkNotNull(Object)} or {@link Verify#verifyNotNull(Object)} is appropriate to the situation. + * (The same goes for the message-accepting overloads.) + * + *

Only {@code %s} is supported

+ * + *

In {@code Preconditions} error message template strings, only the {@code "%s"} specifier is + * supported, not the full range of {@link java.util.Formatter} specifiers. + * + *

More information

+ * + *

See the Guava User Guide on using {@code + * Preconditions}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public final class Preconditions { + private Preconditions() {} + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + * @param expression a boolean expression + * @throws IllegalArgumentException if {@code expression} is false + */ + public static void checkArgument(boolean expression) { + if (!expression) { + throw new IllegalArgumentException(); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + * @param expression a boolean expression + * @param errorMessage the exception message to use if the check fails; will be converted to a + * string using {@link String#valueOf(Object)} + * @throws IllegalArgumentException if {@code expression} is false + */ + public static void checkArgument(boolean expression, Object errorMessage) { + if (!expression) { + throw new IllegalArgumentException(String.valueOf(errorMessage)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + * @param expression a boolean expression + * @param errorMessageTemplate a template for the exception message should the check fail. The + * message is formed by replacing each {@code %s} placeholder in the template with an + * argument. These are matched by position - the first {@code %s} gets {@code + * errorMessageArgs[0]}, etc. Unmatched arguments will be appended to the formatted message in + * square braces. Unmatched placeholders will be left as-is. + * @param errorMessageArgs the arguments to be substituted into the message template. Arguments + * are converted to strings using {@link String#valueOf(Object)}. + * @throws IllegalArgumentException if {@code expression} is false + */ + public static void checkArgument( + boolean expression, + String errorMessageTemplate, + Object... errorMessageArgs) { + if (!expression) { + throw new IllegalArgumentException(format(errorMessageTemplate, errorMessageArgs)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument(boolean b, String errorMessageTemplate, char p1) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument(boolean b, String errorMessageTemplate, int p1) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument(boolean b, String errorMessageTemplate, long p1) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, Object p1) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, char p1, char p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, char p1, int p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, char p1, long p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, char p1, Object p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, int p1, char p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, int p1, int p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, int p1, long p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, int p1, Object p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, long p1, char p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, long p1, int p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, long p1, long p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, long p1, Object p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, Object p1, char p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, Object p1, int p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, String errorMessageTemplate, Object p1, long p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, + String errorMessageTemplate, + Object p1, + Object p2) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2, p3)); + } + } + + /** + * Ensures the truth of an expression involving one or more parameters to the calling method. + * + *

See {@link #checkArgument(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkArgument( + boolean b, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3, + Object p4) { + if (!b) { + throw new IllegalArgumentException(format(errorMessageTemplate, p1, p2, p3, p4)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + * @param expression a boolean expression + * @throws IllegalStateException if {@code expression} is false + * @see Verify#verify Verify.verify() + */ + public static void checkState(boolean expression) { + if (!expression) { + throw new IllegalStateException(); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + * @param expression a boolean expression + * @param errorMessage the exception message to use if the check fails; will be converted to a + * string using {@link String#valueOf(Object)} + * @throws IllegalStateException if {@code expression} is false + * @see Verify#verify Verify.verify() + */ + public static void checkState(boolean expression, Object errorMessage) { + if (!expression) { + throw new IllegalStateException(String.valueOf(errorMessage)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + * @param expression a boolean expression + * @param errorMessageTemplate a template for the exception message should the check fail. The + * message is formed by replacing each {@code %s} placeholder in the template with an + * argument. These are matched by position - the first {@code %s} gets {@code + * errorMessageArgs[0]}, etc. Unmatched arguments will be appended to the formatted message in + * square braces. Unmatched placeholders will be left as-is. + * @param errorMessageArgs the arguments to be substituted into the message template. Arguments + * are converted to strings using {@link String#valueOf(Object)}. + * @throws IllegalStateException if {@code expression} is false + * @see Verify#verify Verify.verify() + */ + public static void checkState( + boolean expression, + String errorMessageTemplate, + Object... errorMessageArgs) { + if (!expression) { + throw new IllegalStateException(format(errorMessageTemplate, errorMessageArgs)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState(boolean b, String errorMessageTemplate, char p1) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState(boolean b, String errorMessageTemplate, int p1) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState(boolean b, String errorMessageTemplate, long p1) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, Object p1) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, char p1, char p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, char p1, int p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, char p1, long p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, char p1, Object p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, int p1, char p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, int p1, int p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, int p1, long p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, int p1, Object p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, long p1, char p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, long p1, int p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, long p1, long p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, long p1, Object p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, Object p1, char p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, Object p1, int p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, String errorMessageTemplate, Object p1, long p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, + String errorMessageTemplate, + Object p1, + Object p2) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2, p3)); + } + } + + /** + * Ensures the truth of an expression involving the state of the calling instance, but not + * involving any parameters to the calling method. + * + *

See {@link #checkState(boolean, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static void checkState( + boolean b, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3, + Object p4) { + if (!b) { + throw new IllegalStateException(format(errorMessageTemplate, p1, p2, p3, p4)); + } + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + * @param reference an object reference + * @return the non-null reference that was validated + * @throws NullPointerException if {@code reference} is null + * @see Verify#verifyNotNull Verify.verifyNotNull() + */ + public static T checkNotNull(T reference) { + if (reference == null) { + throw new NullPointerException(); + } + return reference; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + * @param reference an object reference + * @param errorMessage the exception message to use if the check fails; will be converted to a + * string using {@link String#valueOf(Object)} + * @return the non-null reference that was validated + * @throws NullPointerException if {@code reference} is null + * @see Verify#verifyNotNull Verify.verifyNotNull() + */ + public static T checkNotNull(T reference, Object errorMessage) { + if (reference == null) { + throw new NullPointerException(String.valueOf(errorMessage)); + } + return reference; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + * @param reference an object reference + * @param errorMessageTemplate a template for the exception message should the check fail. The + * message is formed by replacing each {@code %s} placeholder in the template with an + * argument. These are matched by position - the first {@code %s} gets {@code + * errorMessageArgs[0]}, etc. Unmatched arguments will be appended to the formatted message in + * square braces. Unmatched placeholders will be left as-is. + * @param errorMessageArgs the arguments to be substituted into the message template. Arguments + * are converted to strings using {@link String#valueOf(Object)}. + * @return the non-null reference that was validated + * @throws NullPointerException if {@code reference} is null + * @see Verify#verifyNotNull Verify.verifyNotNull() + */ + public static T checkNotNull( + T reference, + String errorMessageTemplate, + Object... errorMessageArgs) { + if (reference == null) { + throw new NullPointerException(format(errorMessageTemplate, errorMessageArgs)); + } + return reference; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull(T obj, String errorMessageTemplate, char p1) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull(T obj, String errorMessageTemplate, int p1) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull(T obj, String errorMessageTemplate, long p1) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, Object p1) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, char p1, char p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, char p1, int p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, char p1, long p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, char p1, Object p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, int p1, char p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, int p1, int p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, int p1, long p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, int p1, Object p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, long p1, char p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, long p1, int p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, long p1, long p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, long p1, Object p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, Object p1, char p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, Object p1, int p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, String errorMessageTemplate, Object p1, long p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, + String errorMessageTemplate, + Object p1, + Object p2) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2, p3)); + } + return obj; + } + + /** + * Ensures that an object reference passed as a parameter to the calling method is not null. + * + *

See {@link #checkNotNull(Object, String, Object...)} for details. + * + * @since 20.0 (varargs overload since 2.0) + */ + public static T checkNotNull( + T obj, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3, + Object p4) { + if (obj == null) { + throw new NullPointerException(format(errorMessageTemplate, p1, p2, p3, p4)); + } + return obj; + } + + /* + * All recent hotspots (as of 2009) *really* like to have the natural code + * + * if (guardExpression) { + * throw new BadException(messageExpression); + * } + * + * refactored so that messageExpression is moved to a separate String-returning method. + * + * if (guardExpression) { + * throw new BadException(badMsg(...)); + * } + * + * The alternative natural refactorings into void or Exception-returning methods are much slower. + * This is a big deal - we're talking factors of 2-8 in microbenchmarks, not just 10-20%. (This is + * a hotspot optimizer bug, which should be fixed, but that's a separate, big project). + * + * The coding pattern above is heavily used in java.util, e.g. in ArrayList. There is a + * RangeCheckMicroBenchmark in the JDK that was used to test this. + * + * But the methods in this class want to throw different exceptions, depending on the args, so it + * appears that this pattern is not directly applicable. But we can use the ridiculous, devious + * trick of throwing an exception in the middle of the construction of another exception. Hotspot + * is fine with that. + */ + + /** + * Ensures that {@code index} specifies a valid element in an array, list or string of size + * {@code size}. An element index may range from zero, inclusive, to {@code size}, exclusive. + * + * @param index a user-supplied index identifying an element of an array, list or string + * @param size the size of that array, list or string + * @return the value of {@code index} + * @throws IndexOutOfBoundsException if {@code index} is negative or is not less than {@code size} + * @throws IllegalArgumentException if {@code size} is negative + */ + public static int checkElementIndex(int index, int size) { + return checkElementIndex(index, size, "index"); + } + + /** + * Ensures that {@code index} specifies a valid element in an array, list or string of size + * {@code size}. An element index may range from zero, inclusive, to {@code size}, exclusive. + * + * @param index a user-supplied index identifying an element of an array, list or string + * @param size the size of that array, list or string + * @param desc the text to use to describe this index in an error message + * @return the value of {@code index} + * @throws IndexOutOfBoundsException if {@code index} is negative or is not less than {@code size} + * @throws IllegalArgumentException if {@code size} is negative + */ + public static int checkElementIndex(int index, int size, String desc) { + // Carefully optimized for execution by hotspot (explanatory comment above) + if (index < 0 || index >= size) { + throw new IndexOutOfBoundsException(badElementIndex(index, size, desc)); + } + return index; + } + + private static String badElementIndex(int index, int size, String desc) { + if (index < 0) { + return format("%s (%s) must not be negative", desc, index); + } else if (size < 0) { + throw new IllegalArgumentException("negative size: " + size); + } else { // index >= size + return format("%s (%s) must be less than size (%s)", desc, index, size); + } + } + + /** + * Ensures that {@code index} specifies a valid position in an array, list or string of + * size {@code size}. A position index may range from zero to {@code size}, inclusive. + * + * @param index a user-supplied index identifying a position in an array, list or string + * @param size the size of that array, list or string + * @return the value of {@code index} + * @throws IndexOutOfBoundsException if {@code index} is negative or is greater than {@code size} + * @throws IllegalArgumentException if {@code size} is negative + */ + public static int checkPositionIndex(int index, int size) { + return checkPositionIndex(index, size, "index"); + } + + /** + * Ensures that {@code index} specifies a valid position in an array, list or string of + * size {@code size}. A position index may range from zero to {@code size}, inclusive. + * + * @param index a user-supplied index identifying a position in an array, list or string + * @param size the size of that array, list or string + * @param desc the text to use to describe this index in an error message + * @return the value of {@code index} + * @throws IndexOutOfBoundsException if {@code index} is negative or is greater than {@code size} + * @throws IllegalArgumentException if {@code size} is negative + */ + public static int checkPositionIndex(int index, int size, String desc) { + // Carefully optimized for execution by hotspot (explanatory comment above) + if (index < 0 || index > size) { + throw new IndexOutOfBoundsException(badPositionIndex(index, size, desc)); + } + return index; + } + + private static String badPositionIndex(int index, int size, String desc) { + if (index < 0) { + return format("%s (%s) must not be negative", desc, index); + } else if (size < 0) { + throw new IllegalArgumentException("negative size: " + size); + } else { // index > size + return format("%s (%s) must not be greater than size (%s)", desc, index, size); + } + } + + /** + * Ensures that {@code start} and {@code end} specify a valid positions in an array, list + * or string of size {@code size}, and are in order. A position index may range from zero to + * {@code size}, inclusive. + * + * @param start a user-supplied index identifying a starting position in an array, list or string + * @param end a user-supplied index identifying a ending position in an array, list or string + * @param size the size of that array, list or string + * @throws IndexOutOfBoundsException if either index is negative or is greater than {@code size}, + * or if {@code end} is less than {@code start} + * @throws IllegalArgumentException if {@code size} is negative + */ + public static void checkPositionIndexes(int start, int end, int size) { + // Carefully optimized for execution by hotspot (explanatory comment above) + if (start < 0 || end < start || end > size) { + throw new IndexOutOfBoundsException(badPositionIndexes(start, end, size)); + } + } + + private static String badPositionIndexes(int start, int end, int size) { + if (start < 0 || start > size) { + return badPositionIndex(start, size, "start index"); + } + if (end < 0 || end > size) { + return badPositionIndex(end, size, "end index"); + } + // end < start + return format("end index (%s) must not be less than start index (%s)", end, start); + } + + /** + * Substitutes each {@code %s} in {@code template} with an argument. These are matched by + * position: the first {@code %s} gets {@code args[0]}, etc. If there are more arguments than + * placeholders, the unmatched arguments will be appended to the end of the formatted message in + * square braces. + * + * @param template a string containing 0 or more {@code %s} placeholders. null is treated as + * "null". + * @param args the arguments to be substituted into the message template. Arguments are converted + * to strings using {@link String#valueOf(Object)}. Arguments can be null. + */ + // Note that this is somewhat-improperly used from Verify.java as well. + static String format(String template, Object... args) { + template = String.valueOf(template); // null -> "null" + + args = args == null ? new Object[] {"(Object[])null"} : args; + + // start substituting the arguments into the '%s' placeholders + StringBuilder builder = new StringBuilder(template.length() + 16 * args.length); + int templateStart = 0; + int i = 0; + while (i < args.length) { + int placeholderStart = template.indexOf("%s", templateStart); + if (placeholderStart == -1) { + break; + } + builder.append(template, templateStart, placeholderStart); + builder.append(args[i++]); + templateStart = placeholderStart + 2; + } + builder.append(template, templateStart, template.length()); + + // if we run out of placeholders, append the extra args in square braces + if (i < args.length) { + builder.append(" ["); + builder.append(args[i++]); + while (i < args.length) { + builder.append(", "); + builder.append(args[i++]); + } + builder.append(']'); + } + + return builder.toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicate.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicate.java new file mode 100644 index 0000000000000..4eeb1baac53a6 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicate.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +/** + * Legacy version of {@link java.util.function.Predicate java.util.function.Predicate}. Determines a + * true or false value for a given input. + * + *

As this interface extends {@code java.util.function.Predicate}, an instance of this type may + * be used as a {@code Predicate} directly. To use a {@code java.util.function.Predicate} where a + * {@code com.google.common.base.Predicate} is expected, use the method reference {@code + * predicate::test}. + * + *

This interface is now a legacy type. Use {@code java.util.function.Predicate} (or the + * appropriate primitive specialization such as {@code IntPredicate}) instead whenever possible. + * Otherwise, at least reduce explicit dependencies on this type by using lambda expressions + * or method references instead of classes, leaving your code easier to migrate in the future. + * + *

The {@link Predicates} class provides common predicates and related utilities. + * + *

See the Guava User Guide article on the use of {@code Predicate}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +@FunctionalInterface +public interface Predicate extends java.util.function.Predicate { + /** + * Returns the result of applying this predicate to {@code input} (Java 8 users, see notes in the + * class documentation above). This method is generally expected, but not absolutely + * required, to have the following properties: + * + *

    + *
  • Its execution does not cause any observable side effects. + *
  • The computation is consistent with equals; that is, {@link Objects#equal + * Objects.equal}{@code (a, b)} implies that {@code predicate.apply(a) == + * predicate.apply(b))}. + *
+ * + * @throws NullPointerException if {@code input} is null and this predicate does not accept null + * arguments + */ + boolean apply(T input); + + /** + * Indicates whether another object is equal to this predicate. + * + *

Most implementations will have no reason to override the behavior of {@link Object#equals}. + * However, an implementation may also choose to return {@code true} whenever {@code object} is a + * {@link Predicate} that it considers interchangeable with this one. "Interchangeable" + * typically means that {@code this.apply(t) == that.apply(t)} for all {@code t} of type + * {@code T}). Note that a {@code false} result from this method does not imply that the + * predicates are known not to be interchangeable. + */ + @Override + boolean equals(Object object); + + @Override + default boolean test(T input) { + return apply(input); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicates.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicates.java new file mode 100644 index 0000000000000..6f4524f0e2722 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Predicates.java @@ -0,0 +1,695 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.regex.Pattern; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** + * Static utility methods pertaining to {@code Predicate} instances. + * + *

All methods return serializable predicates as long as they're given serializable parameters. + * + *

See the Guava User Guide article on the use of {@code Predicate}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public final class Predicates { + private Predicates() {} + + // TODO(kevinb): considering having these implement a VisitablePredicate + // interface which specifies an accept(PredicateVisitor) method. + + /** Returns a predicate that always evaluates to {@code true}. */ + public static Predicate alwaysTrue() { + return ObjectPredicate.ALWAYS_TRUE.withNarrowedType(); + } + + /** Returns a predicate that always evaluates to {@code false}. */ + public static Predicate alwaysFalse() { + return ObjectPredicate.ALWAYS_FALSE.withNarrowedType(); + } + + /** + * Returns a predicate that evaluates to {@code true} if the object reference being tested is + * null. + */ + public static Predicate isNull() { + return ObjectPredicate.IS_NULL.withNarrowedType(); + } + + /** + * Returns a predicate that evaluates to {@code true} if the object reference being tested is not + * null. + */ + public static Predicate notNull() { + return ObjectPredicate.NOT_NULL.withNarrowedType(); + } + + /** + * Returns a predicate that evaluates to {@code true} if the given predicate evaluates to {@code + * false}. + */ + public static Predicate not(Predicate predicate) { + return new NotPredicate(predicate); + } + + /** + * Returns a predicate that evaluates to {@code true} if each of its components evaluates to + * {@code true}. The components are evaluated in order, and evaluation will be "short-circuited" + * as soon as a false predicate is found. It defensively copies the iterable passed in, so future + * changes to it won't alter the behavior of this predicate. If {@code components} is empty, the + * returned predicate will always evaluate to {@code true}. + */ + public static Predicate and(Iterable> components) { + return new AndPredicate(defensiveCopy(components)); + } + + /** + * Returns a predicate that evaluates to {@code true} if each of its components evaluates to + * {@code true}. The components are evaluated in order, and evaluation will be "short-circuited" + * as soon as a false predicate is found. It defensively copies the array passed in, so future + * changes to it won't alter the behavior of this predicate. If {@code components} is empty, the + * returned predicate will always evaluate to {@code true}. + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static Predicate and(Predicate... components) { + return new AndPredicate(defensiveCopy(components)); + } + + /** + * Returns a predicate that evaluates to {@code true} if both of its components evaluate to {@code + * true}. The components are evaluated in order, and evaluation will be "short-circuited" as soon + * as a false predicate is found. + */ + public static Predicate and(Predicate first, Predicate second) { + return new AndPredicate(Predicates.asList(checkNotNull(first), checkNotNull(second))); + } + + /** + * Returns a predicate that evaluates to {@code true} if any one of its components evaluates to + * {@code true}. The components are evaluated in order, and evaluation will be "short-circuited" + * as soon as a true predicate is found. It defensively copies the iterable passed in, so future + * changes to it won't alter the behavior of this predicate. If {@code components} is empty, the + * returned predicate will always evaluate to {@code false}. + */ + public static Predicate or(Iterable> components) { + return new OrPredicate(defensiveCopy(components)); + } + + /** + * Returns a predicate that evaluates to {@code true} if any one of its components evaluates to + * {@code true}. The components are evaluated in order, and evaluation will be "short-circuited" + * as soon as a true predicate is found. It defensively copies the array passed in, so future + * changes to it won't alter the behavior of this predicate. If {@code components} is empty, the + * returned predicate will always evaluate to {@code false}. + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static Predicate or(Predicate... components) { + return new OrPredicate(defensiveCopy(components)); + } + + /** + * Returns a predicate that evaluates to {@code true} if either of its components evaluates to + * {@code true}. The components are evaluated in order, and evaluation will be "short-circuited" + * as soon as a true predicate is found. + */ + public static Predicate or(Predicate first, Predicate second) { + return new OrPredicate(Predicates.asList(checkNotNull(first), checkNotNull(second))); + } + + /** + * Returns a predicate that evaluates to {@code true} if the object being tested {@code equals()} + * the given target or both are null. + */ + public static Predicate equalTo(T target) { + return (target == null) ? Predicates.isNull() : new IsEqualToPredicate(target); + } + + /** + * Returns a predicate that evaluates to {@code true} if the object being tested is an instance of + * the given class. If the object being tested is {@code null} this predicate evaluates to {@code + * false}. + * + *

If you want to filter an {@code Iterable} to narrow its type, consider using {@link + * Iterables#filter(Iterable, Class)} in preference. + * + *

Warning: contrary to the typical assumptions about predicates (as documented at + * {@link Predicate#apply}), the returned predicate may not be consistent with equals. For + * example, {@code instanceOf(ArrayList.class)} will yield different results for the two equal + * instances {@code Lists.newArrayList(1)} and {@code Arrays.asList(1)}. + */ + public static Predicate instanceOf(Class clazz) { + return new InstanceOfPredicate(clazz); + } + + /** + * Returns a predicate that evaluates to {@code true} if the class being tested is assignable to + * (is a subtype of) {@code clazz}. Example: + * + *
{@code
+   * List> classes = Arrays.asList(
+   *     Object.class, String.class, Number.class, Long.class);
+   * return Iterables.filter(classes, subtypeOf(Number.class));
+   * }
+ * + * The code above returns an iterable containing {@code Number.class} and {@code Long.class}. + * + * @since 20.0 (since 10.0 under the incorrect name {@code assignableFrom}) + */ + + public static Predicate> subtypeOf(Class clazz) { + return new SubtypeOfPredicate(clazz); + } + + /** + * Returns a predicate that evaluates to {@code true} if the object reference being tested is a + * member of the given collection. It does not defensively copy the collection passed in, so + * future changes to it will alter the behavior of the predicate. + * + *

This method can technically accept any {@code Collection}, but using a typed collection + * helps prevent bugs. This approach doesn't block any potential users since it is always possible + * to use {@code Predicates.in()}. + * + * @param target the collection that may contain the function input + */ + public static Predicate in(Collection target) { + return new InPredicate(target); + } + + /** + * Returns the composition of a function and a predicate. For every {@code x}, the generated + * predicate returns {@code predicate(function(x))}. + * + * @return the composition of the provided function and predicate + */ + public static Predicate compose( + Predicate predicate, Function function) { + return new CompositionPredicate<>(predicate, function); + } + + /** + * Returns a predicate that evaluates to {@code true} if the {@code CharSequence} being tested + * contains any match for the given regular expression pattern. The test used is equivalent to + * {@code Pattern.compile(pattern).matcher(arg).find()} + * + * @throws IllegalArgumentException if the pattern is invalid + * @since 3.0 + */ + public static Predicate containsPattern(String pattern) { + return new ContainsPatternFromStringPredicate(pattern); + } + + /** + * Returns a predicate that evaluates to {@code true} if the {@code CharSequence} being tested + * contains any match for the given regular expression pattern. The test used is equivalent to + * {@code pattern.matcher(arg).find()} + * + * @since 3.0 + */ + public static Predicate contains(Pattern pattern) { + return new ContainsPatternPredicate(new JdkPattern(pattern)); + } + + // End public API, begin private implementation classes. + + // Package private for GWT serialization. + enum ObjectPredicate implements Predicate { + /** @see Predicates#alwaysTrue() */ + ALWAYS_TRUE { + @Override + public boolean apply(Object o) { + return true; + } + + @Override + public String toString() { + return "Predicates.alwaysTrue()"; + } + }, + /** @see Predicates#alwaysFalse() */ + ALWAYS_FALSE { + @Override + public boolean apply(Object o) { + return false; + } + + @Override + public String toString() { + return "Predicates.alwaysFalse()"; + } + }, + /** @see Predicates#isNull() */ + IS_NULL { + @Override + public boolean apply(Object o) { + return o == null; + } + + @Override + public String toString() { + return "Predicates.isNull()"; + } + }, + /** @see Predicates#notNull() */ + NOT_NULL { + @Override + public boolean apply(Object o) { + return o != null; + } + + @Override + public String toString() { + return "Predicates.notNull()"; + } + }; + + @SuppressWarnings("unchecked") // safe contravariant cast + Predicate withNarrowedType() { + return (Predicate) this; + } + } + + /** @see Predicates#not(Predicate) */ + private static class NotPredicate implements Predicate, Serializable { + final Predicate predicate; + + NotPredicate(Predicate predicate) { + this.predicate = checkNotNull(predicate); + } + + @Override + public boolean apply(T t) { + return !predicate.apply(t); + } + + @Override + public int hashCode() { + return ~predicate.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof NotPredicate) { + NotPredicate that = (NotPredicate) obj; + return predicate.equals(that.predicate); + } + return false; + } + + @Override + public String toString() { + return "Predicates.not(" + predicate + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#and(Iterable) */ + private static class AndPredicate implements Predicate, Serializable { + private final List> components; + + private AndPredicate(List> components) { + this.components = components; + } + + @Override + public boolean apply(T t) { + // Avoid using the Iterator to avoid generating garbage (issue 820). + for (int i = 0; i < components.size(); i++) { + if (!components.get(i).apply(t)) { + return false; + } + } + return true; + } + + @Override + public int hashCode() { + // add a random number to avoid collisions with OrPredicate + return components.hashCode() + 0x12472c2c; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof AndPredicate) { + AndPredicate that = (AndPredicate) obj; + return components.equals(that.components); + } + return false; + } + + @Override + public String toString() { + return toStringHelper("and", components); + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#or(Iterable) */ + private static class OrPredicate implements Predicate, Serializable { + private final List> components; + + private OrPredicate(List> components) { + this.components = components; + } + + @Override + public boolean apply(T t) { + // Avoid using the Iterator to avoid generating garbage (issue 820). + for (int i = 0; i < components.size(); i++) { + if (components.get(i).apply(t)) { + return true; + } + } + return false; + } + + @Override + public int hashCode() { + // add a random number to avoid collisions with AndPredicate + return components.hashCode() + 0x053c91cf; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof OrPredicate) { + OrPredicate that = (OrPredicate) obj; + return components.equals(that.components); + } + return false; + } + + @Override + public String toString() { + return toStringHelper("or", components); + } + + private static final long serialVersionUID = 0; + } + + private static String toStringHelper(String methodName, Iterable components) { + StringBuilder builder = new StringBuilder("Predicates.").append(methodName).append('('); + boolean first = true; + for (Object o : components) { + if (!first) { + builder.append(','); + } + builder.append(o); + first = false; + } + return builder.append(')').toString(); + } + + /** @see Predicates#equalTo(Object) */ + private static class IsEqualToPredicate implements Predicate, Serializable { + private final T target; + + private IsEqualToPredicate(T target) { + this.target = target; + } + + @Override + public boolean apply(T t) { + return target.equals(t); + } + + @Override + public int hashCode() { + return target.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof IsEqualToPredicate) { + IsEqualToPredicate that = (IsEqualToPredicate) obj; + return target.equals(that.target); + } + return false; + } + + @Override + public String toString() { + return "Predicates.equalTo(" + target + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#instanceOf(Class) */ + private static class InstanceOfPredicate implements Predicate, Serializable { + private final Class clazz; + + private InstanceOfPredicate(Class clazz) { + this.clazz = checkNotNull(clazz); + } + + @Override + public boolean apply(Object o) { + return clazz.isInstance(o); + } + + @Override + public int hashCode() { + return clazz.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof InstanceOfPredicate) { + InstanceOfPredicate that = (InstanceOfPredicate) obj; + return clazz == that.clazz; + } + return false; + } + + @Override + public String toString() { + return "Predicates.instanceOf(" + clazz.getName() + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#subtypeOf(Class) */ + private static class SubtypeOfPredicate implements Predicate>, Serializable { + private final Class clazz; + + private SubtypeOfPredicate(Class clazz) { + this.clazz = checkNotNull(clazz); + } + + @Override + public boolean apply(Class input) { + return clazz.isAssignableFrom(input); + } + + @Override + public int hashCode() { + return clazz.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof SubtypeOfPredicate) { + SubtypeOfPredicate that = (SubtypeOfPredicate) obj; + return clazz == that.clazz; + } + return false; + } + + @Override + public String toString() { + return "Predicates.subtypeOf(" + clazz.getName() + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#in(Collection) */ + private static class InPredicate implements Predicate, Serializable { + private final Collection target; + + private InPredicate(Collection target) { + this.target = checkNotNull(target); + } + + @Override + public boolean apply(T t) { + try { + return target.contains(t); + } catch (NullPointerException | ClassCastException e) { + return false; + } + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof InPredicate) { + InPredicate that = (InPredicate) obj; + return target.equals(that.target); + } + return false; + } + + @Override + public int hashCode() { + return target.hashCode(); + } + + @Override + public String toString() { + return "Predicates.in(" + target + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#compose(Predicate, Function) */ + private static class CompositionPredicate implements Predicate, Serializable { + final Predicate p; + final Function f; + + private CompositionPredicate(Predicate p, Function f) { + this.p = checkNotNull(p); + this.f = checkNotNull(f); + } + + @Override + public boolean apply(A a) { + return p.apply(f.apply(a)); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof CompositionPredicate) { + CompositionPredicate that = (CompositionPredicate) obj; + return f.equals(that.f) && p.equals(that.p); + } + return false; + } + + @Override + public int hashCode() { + return f.hashCode() ^ p.hashCode(); + } + + @Override + public String toString() { + // TODO(cpovirk): maybe make this look like the method call does ("Predicates.compose(...)") + return p + "(" + f + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#contains(Pattern) */ + private static class ContainsPatternPredicate implements Predicate, Serializable { + final CommonPattern pattern; + + ContainsPatternPredicate(CommonPattern pattern) { + this.pattern = checkNotNull(pattern); + } + + @Override + public boolean apply(CharSequence t) { + return pattern.matcher(t).find(); + } + + @Override + public int hashCode() { + // Pattern uses Object.hashCode, so we have to reach + // inside to build a hashCode consistent with equals. + + return Objects.hashCode(pattern.pattern(), pattern.flags()); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ContainsPatternPredicate) { + ContainsPatternPredicate that = (ContainsPatternPredicate) obj; + + // Pattern uses Object (identity) equality, so we have to reach + // inside to compare individual fields. + return Objects.equal(pattern.pattern(), that.pattern.pattern()) + && pattern.flags() == that.pattern.flags(); + } + return false; + } + + @Override + public String toString() { + String patternString = + MoreObjects.toStringHelper(pattern) + .add("pattern", pattern.pattern()) + .add("pattern.flags", pattern.flags()) + .toString(); + return "Predicates.contains(" + patternString + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** @see Predicates#containsPattern(String) */ + private static class ContainsPatternFromStringPredicate extends ContainsPatternPredicate { + + ContainsPatternFromStringPredicate(String string) { + super(Platform.compilePattern(string)); + } + + @Override + public String toString() { + return "Predicates.containsPattern(" + pattern.pattern() + ")"; + } + + private static final long serialVersionUID = 0; + } + + private static List> asList( + Predicate first, Predicate second) { + // TODO(kevinb): understand why we still get a warning despite @SafeVarargs! + return Arrays.>asList(first, second); + } + + @SafeVarargs + @SuppressWarnings("varargs") + private static List defensiveCopy(T... array) { + return defensiveCopy(Arrays.asList(array)); + } + + static List defensiveCopy(Iterable iterable) { + ArrayList list = new ArrayList(); + for (T element : iterable) { + list.add(checkNotNull(element)); + } + return list; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Present.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Present.java new file mode 100644 index 0000000000000..32ec9a903bcef --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Present.java @@ -0,0 +1,101 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.Collections; +import java.util.Set; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +/** Implementation of an {@link Optional} containing a reference. */ +final class Present extends Optional { + private final T reference; + + Present(T reference) { + this.reference = reference; + } + + @Override + public boolean isPresent() { + return true; + } + + @Override + public T get() { + return reference; + } + + @Override + public T or(T defaultValue) { + checkNotNull(defaultValue, "use Optional.orNull() instead of Optional.or(null)"); + return reference; + } + + @Override + public Optional or(Optional secondChoice) { + checkNotNull(secondChoice); + return this; + } + + @Override + public T or(Supplier supplier) { + checkNotNull(supplier); + return reference; + } + + @Override + public T orNull() { + return reference; + } + + @Override + public Set asSet() { + return Collections.singleton(reference); + } + + @Override + public Optional transform(Function function) { + return new Present( + checkNotNull( + function.apply(reference), + "the Function passed to Optional.transform() must not return null.")); + } + + @Override + public boolean equals(Object object) { + if (object instanceof Present) { + Present other = (Present) object; + return reference.equals(other.reference); + } + return false; + } + + @Override + public int hashCode() { + return 0x598df91c + reference.hashCode(); + } + + @Override + public String toString() { + return "Optional.of(" + reference + ")"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/SmallCharMatcher.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/SmallCharMatcher.java new file mode 100644 index 0000000000000..7088575ac28bf --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/SmallCharMatcher.java @@ -0,0 +1,146 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.BitSet; + +/** + * An immutable version of CharMatcher for smallish sets of characters that uses a hash table with + * linear probing to check for matches. + * + * @author Christopher Swenson + */ +final class SmallCharMatcher extends CharMatcher.NamedFastMatcher { + static final int MAX_SIZE = 1023; + private final char[] table; + private final boolean containsZero; + private final long filter; + + private SmallCharMatcher(char[] table, long filter, boolean containsZero, String description) { + super(description); + this.table = table; + this.filter = filter; + this.containsZero = containsZero; + } + + private static final int C1 = 0xcc9e2d51; + private static final int C2 = 0x1b873593; + + /* + * This method was rewritten in Java from an intermediate step of the Murmur hash function in + * http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp, which contained the + * following header: + * + * MurmurHash3 was written by Austin Appleby, and is placed in the public domain. The author + * hereby disclaims copyright to this source code. + */ + static int smear(int hashCode) { + return C2 * Integer.rotateLeft(hashCode * C1, 15); + } + + private boolean checkFilter(int c) { + return 1 == (1 & (filter >> c)); + } + + // This is all essentially copied from ImmutableSet, but we have to duplicate because + // of dependencies. + + // Represents how tightly we can pack things, as a maximum. + private static final double DESIRED_LOAD_FACTOR = 0.5; + + /** + * Returns an array size suitable for the backing array of a hash table that uses open addressing + * with linear probing in its implementation. The returned size is the smallest power of two that + * can hold setSize elements with the desired load factor. + */ + + static int chooseTableSize(int setSize) { + if (setSize == 1) { + return 2; + } + // Correct the size for open addressing to match desired load factor. + // Round up to the next highest power of 2. + int tableSize = Integer.highestOneBit(setSize - 1) << 1; + while (tableSize * DESIRED_LOAD_FACTOR < setSize) { + tableSize <<= 1; + } + return tableSize; + } + + static CharMatcher from(BitSet chars, String description) { + // Compute the filter. + long filter = 0; + int size = chars.cardinality(); + boolean containsZero = chars.get(0); + // Compute the hash table. + char[] table = new char[chooseTableSize(size)]; + int mask = table.length - 1; + for (int c = chars.nextSetBit(0); c != -1; c = chars.nextSetBit(c + 1)) { + // Compute the filter at the same time. + filter |= 1L << c; + int index = smear(c) & mask; + while (true) { + // Check for empty. + if (table[index] == 0) { + table[index] = (char) c; + break; + } + // Linear probing. + index = (index + 1) & mask; + } + } + return new SmallCharMatcher(table, filter, containsZero, description); + } + + @Override + public boolean matches(char c) { + if (c == 0) { + return containsZero; + } + if (!checkFilter(c)) { + return false; + } + int mask = table.length - 1; + int startingIndex = smear(c) & mask; + int index = startingIndex; + do { + if (table[index] == 0) { // Check for empty. + return false; + } else if (table[index] == c) { // Check for match. + return true; + } else { // Linear probing. + index = (index + 1) & mask; + } + // Check to see if we wrapped around the whole table. + } while (index != startingIndex); + return false; + } + + @Override + void setBits(BitSet table) { + if (containsZero) { + table.set(0); + } + for (char c : this.table) { + if (c != 0) { + table.set(c); + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Splitter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Splitter.java new file mode 100644 index 0000000000000..421f75d226f5f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Splitter.java @@ -0,0 +1,598 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +/** + * Extracts non-overlapping substrings from an input string, typically by recognizing appearances of + * a separator sequence. This separator can be specified as a single {@linkplain #on(char) + * character}, fixed {@linkplain #on(String) string}, {@linkplain #onPattern regular expression} or + * {@link #on(CharMatcher) CharMatcher} instance. Or, instead of using a separator at all, a + * splitter can extract adjacent substrings of a given {@linkplain #fixedLength fixed length}. + * + *

For example, this expression: + * + *

{@code
+ * Splitter.on(',').split("foo,bar,qux")
+ * }
+ * + * ... produces an {@code Iterable} containing {@code "foo"}, {@code "bar"} and {@code "qux"}, in + * that order. + * + *

By default, {@code Splitter}'s behavior is simplistic and unassuming. The following + * expression: + * + *

{@code
+ * Splitter.on(',').split(" foo,,,  bar ,")
+ * }
+ * + * ... yields the substrings {@code [" foo", "", "", " bar ", ""]}. If this is not the desired + * behavior, use configuration methods to obtain a new splitter instance with modified + * behavior: + * + *
{@code
+ * private static final Splitter MY_SPLITTER = Splitter.on(',')
+ *     .trimResults()
+ *     .omitEmptyStrings();
+ * }
+ * + *

Now {@code MY_SPLITTER.split("foo,,, bar ,")} returns just {@code ["foo", "bar"]}. Note that + * the order in which these configuration methods are called is never significant. + * + *

Warning: Splitter instances are immutable. Invoking a configuration method has no + * effect on the receiving instance; you must store and use the new splitter instance it returns + * instead. + * + *

{@code
+ * // Do NOT do this
+ * Splitter splitter = Splitter.on('/');
+ * splitter.trimResults(); // does nothing!
+ * return splitter.split("wrong / wrong / wrong");
+ * }
+ * + *

For separator-based splitters that do not use {@code omitEmptyStrings}, an input string + * containing {@code n} occurrences of the separator naturally yields an iterable of size {@code n + + * 1}. So if the separator does not occur anywhere in the input, a single substring is returned + * containing the entire input. Consequently, all splitters split the empty string to {@code [""]} + * (note: even fixed-length splitters). + * + *

Splitter instances are thread-safe immutable, and are therefore safe to store as {@code static + * final} constants. + * + *

The {@link Joiner} class provides the inverse operation to splitting, but note that a + * round-trip between the two should be assumed to be lossy. + * + *

Exception: for consistency with separator-based splitters, {@code split("")} does not + * yield an empty iterable, but an iterable containing {@code ""}. This is the only case in which + * {@code Iterables.size(split(input))} does not equal {@code IntMath.divide(input.length(), + * length, CEILING)}. To avoid this behavior, use {@code omitEmptyStrings}. + * + * @param length the desired length of pieces after splitting, a positive integer + * @return a splitter, with default settings, that can split into fixed sized pieces + * @throws IllegalArgumentException if {@code length} is zero or negative + */ + public static Splitter fixedLength(final int length) { + checkArgument(length > 0, "The length may not be less than 1"); + + return new Splitter( + new Strategy() { + @Override + public SplittingIterator iterator(final Splitter splitter, CharSequence toSplit) { + return new SplittingIterator(splitter, toSplit) { + @Override + public int separatorStart(int start) { + int nextChunkStart = start + length; + return (nextChunkStart < toSplit.length() ? nextChunkStart : -1); + } + + @Override + public int separatorEnd(int separatorPosition) { + return separatorPosition; + } + }; + } + }); + } + + /** + * Returns a splitter that behaves equivalently to {@code this} splitter, but automatically omits + * empty strings from the results. For example, {@code + * Splitter.on(',').omitEmptyStrings().split(",a,,,b,c,,")} returns an iterable containing only + * {@code ["a", "b", "c"]}. + * + *

If either {@code trimResults} option is also specified when creating a splitter, that + * splitter always trims results first before checking for emptiness. So, for example, {@code + * Splitter.on(':').omitEmptyStrings().trimResults().split(": : : ")} returns an empty iterable. + * + *

Note that it is ordinarily not possible for {@link #split(CharSequence)} to return an empty + * iterable, but when using this option, it can (if the input sequence consists of nothing but + * separators). + * + * @return a splitter with the desired configuration + */ + public Splitter omitEmptyStrings() { + return new Splitter(strategy, true, trimmer, limit); + } + + /** + * Returns a splitter that behaves equivalently to {@code this} splitter but stops splitting after + * it reaches the limit. The limit defines the maximum number of items returned by the iterator, + * or the maximum size of the list returned by {@link #splitToList}. + * + *

For example, {@code Splitter.on(',').limit(3).split("a,b,c,d")} returns an iterable + * containing {@code ["a", "b", "c,d"]}. When omitting empty strings, the omitted strings do not + * count. Hence, {@code Splitter.on(',').limit(3).omitEmptyStrings().split("a,,,b,,,c,d")} returns + * an iterable containing {@code ["a", "b", "c,d"}. When trim is requested, all entries are + * trimmed, including the last. Hence {@code Splitter.on(',').limit(3).trimResults().split(" a , b + * , c , d ")} results in {@code ["a", "b", "c , d"]}. + * + * @param limit the maximum number of items returned + * @return a splitter with the desired configuration + * @since 9.0 + */ + public Splitter limit(int limit) { + checkArgument(limit > 0, "must be greater than zero: %s", limit); + return new Splitter(strategy, omitEmptyStrings, trimmer, limit); + } + + /** + * Returns a splitter that behaves equivalently to {@code this} splitter, but automatically + * removes leading and trailing {@linkplain CharMatcher#whitespace whitespace} from each returned + * substring; equivalent to {@code trimResults(CharMatcher.whitespace())}. For example, {@code + * Splitter.on(',').trimResults().split(" a, b ,c ")} returns an iterable containing {@code ["a", + * "b", "c"]}. + * + * @return a splitter with the desired configuration + */ + public Splitter trimResults() { + return trimResults(CharMatcher.whitespace()); + } + + /** + * Returns a splitter that behaves equivalently to {@code this} splitter, but removes all leading + * or trailing characters matching the given {@code CharMatcher} from each returned substring. For + * example, {@code Splitter.on(',').trimResults(CharMatcher.is('_')).split("_a ,_b_ ,c__")} + * returns an iterable containing {@code ["a ", "b_ ", "c"]}. + * + * @param trimmer a {@link CharMatcher} that determines whether a character should be removed from + * the beginning/end of a subsequence + * @return a splitter with the desired configuration + */ + // TODO(kevinb): throw if a trimmer was already specified! + public Splitter trimResults(CharMatcher trimmer) { + checkNotNull(trimmer); + return new Splitter(strategy, omitEmptyStrings, trimmer, limit); + } + + /** + * Splits {@code sequence} into string components and makes them available through an {@link + * Iterator}, which may be lazily evaluated. If you want an eagerly computed {@link List}, use + * {@link #splitToList(CharSequence)}. + * + * @param sequence the sequence of characters to split + * @return an iteration over the segments split from the parameter + */ + public Iterable split(final CharSequence sequence) { + checkNotNull(sequence); + + return new Iterable() { + @Override + public Iterator iterator() { + return splittingIterator(sequence); + } + + @Override + public String toString() { + return Joiner.on(", ") + .appendTo(new StringBuilder().append('['), this) + .append(']') + .toString(); + } + }; + } + + private Iterator splittingIterator(CharSequence sequence) { + return strategy.iterator(this, sequence); + } + + /** + * Splits {@code sequence} into string components and returns them as an immutable list. If you + * want an {@link Iterable} which may be lazily evaluated, use {@link #split(CharSequence)}. + * + * @param sequence the sequence of characters to split + * @return an immutable list of the segments split from the parameter + * @since 15.0 + */ + + public List splitToList(CharSequence sequence) { + checkNotNull(sequence); + + Iterator iterator = splittingIterator(sequence); + List result = new ArrayList<>(); + + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + return Collections.unmodifiableList(result); + } + + /** + * Returns a {@code MapSplitter} which splits entries based on this splitter, and splits entries + * into keys and values using the specified separator. + * + * @since 10.0 + */ + + public MapSplitter withKeyValueSeparator(String separator) { + return withKeyValueSeparator(on(separator)); + } + + /** + * Returns a {@code MapSplitter} which splits entries based on this splitter, and splits entries + * into keys and values using the specified separator. + * + * @since 14.0 + */ + + public MapSplitter withKeyValueSeparator(char separator) { + return withKeyValueSeparator(on(separator)); + } + + /** + * Returns a {@code MapSplitter} which splits entries based on this splitter, and splits entries + * into keys and values using the specified key-value splitter. + * + * @since 10.0 + */ + + public MapSplitter withKeyValueSeparator(Splitter keyValueSplitter) { + return new MapSplitter(this, keyValueSplitter); + } + + /** + * An object that splits strings into maps as {@code Splitter} splits iterables and lists. Like + * {@code Splitter}, it is thread-safe and immutable. The common way to build instances is by + * providing an additional {@linkplain Splitter#withKeyValueSeparator key-value separator} to + * {@link Splitter}. + * + * @since 10.0 + */ + + public static final class MapSplitter { + private static final String INVALID_ENTRY_MESSAGE = "Chunk [%s] is not a valid entry"; + private final Splitter outerSplitter; + private final Splitter entrySplitter; + + private MapSplitter(Splitter outerSplitter, Splitter entrySplitter) { + this.outerSplitter = outerSplitter; // only "this" is passed + this.entrySplitter = checkNotNull(entrySplitter); + } + + /** + * Splits {@code sequence} into substrings, splits each substring into an entry, and returns an + * unmodifiable map with each of the entries. For example, {@code + * Splitter.on(';').trimResults().withKeyValueSeparator("=>").split("a=>b ; c=>b")} will return + * a mapping from {@code "a"} to {@code "b"} and {@code "c"} to {@code "b"}. + * + *

The returned map preserves the order of the entries from {@code sequence}. + * + * @throws IllegalArgumentException if the specified sequence does not split into valid map + * entries, or if there are duplicate keys + */ + public Map split(CharSequence sequence) { + Map map = new LinkedHashMap<>(); + for (String entry : outerSplitter.split(sequence)) { + Iterator entryFields = entrySplitter.splittingIterator(entry); + + checkArgument(entryFields.hasNext(), INVALID_ENTRY_MESSAGE, entry); + String key = entryFields.next(); + checkArgument(!map.containsKey(key), "Duplicate key [%s] found.", key); + + checkArgument(entryFields.hasNext(), INVALID_ENTRY_MESSAGE, entry); + String value = entryFields.next(); + map.put(key, value); + + checkArgument(!entryFields.hasNext(), INVALID_ENTRY_MESSAGE, entry); + } + return Collections.unmodifiableMap(map); + } + } + + private interface Strategy { + Iterator iterator(Splitter splitter, CharSequence toSplit); + } + + private abstract static class SplittingIterator extends AbstractIterator { + final CharSequence toSplit; + final CharMatcher trimmer; + final boolean omitEmptyStrings; + + /** + * Returns the first index in {@code toSplit} at or after {@code start} that contains the + * separator. + */ + abstract int separatorStart(int start); + + /** + * Returns the first index in {@code toSplit} after {@code separatorPosition} that does not + * contain a separator. This method is only invoked after a call to {@code separatorStart}. + */ + abstract int separatorEnd(int separatorPosition); + + int offset = 0; + int limit; + + protected SplittingIterator(Splitter splitter, CharSequence toSplit) { + this.trimmer = splitter.trimmer; + this.omitEmptyStrings = splitter.omitEmptyStrings; + this.limit = splitter.limit; + this.toSplit = toSplit; + } + + @Override + protected String computeNext() { + /* + * The returned string will be from the end of the last match to the beginning of the next + * one. nextStart is the start position of the returned substring, while offset is the place + * to start looking for a separator. + */ + int nextStart = offset; + while (offset != -1) { + int start = nextStart; + int end; + + int separatorPosition = separatorStart(offset); + if (separatorPosition == -1) { + end = toSplit.length(); + offset = -1; + } else { + end = separatorPosition; + offset = separatorEnd(separatorPosition); + } + if (offset == nextStart) { + /* + * This occurs when some pattern has an empty match, even if it doesn't match the empty + * string -- for example, if it requires lookahead or the like. The offset must be + * increased to look for separators beyond this point, without changing the start position + * of the next returned substring -- so nextStart stays the same. + */ + offset++; + if (offset > toSplit.length()) { + offset = -1; + } + continue; + } + + while (start < end && trimmer.matches(toSplit.charAt(start))) { + start++; + } + while (end > start && trimmer.matches(toSplit.charAt(end - 1))) { + end--; + } + + if (omitEmptyStrings && start == end) { + // Don't include the (unused) separator in next split string. + nextStart = offset; + continue; + } + + if (limit == 1) { + // The limit has been reached, return the rest of the string as the + // final item. This is tested after empty string removal so that + // empty strings do not count towards the limit. + end = toSplit.length(); + offset = -1; + // Since we may have changed the end, we need to trim it again. + while (end > start && trimmer.matches(toSplit.charAt(end - 1))) { + end--; + } + } else { + limit--; + } + + return toSplit.subSequence(start, end).toString(); + } + return endOfData(); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/StandardSystemProperty.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/StandardSystemProperty.java new file mode 100644 index 0000000000000..e64154908d36c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/StandardSystemProperty.java @@ -0,0 +1,138 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * Represents a {@linkplain System#getProperties() standard system property}. + * + * @author Kurt Alfred Kluever + * @since 15.0 + */ +public enum StandardSystemProperty { + + /** Java Runtime Environment version. */ + JAVA_VERSION("java.version"), + + /** Java Runtime Environment vendor. */ + JAVA_VENDOR("java.vendor"), + + /** Java vendor URL. */ + JAVA_VENDOR_URL("java.vendor.url"), + + /** Java installation directory. */ + JAVA_HOME("java.home"), + + /** Java Virtual Machine specification version. */ + JAVA_VM_SPECIFICATION_VERSION("java.vm.specification.version"), + + /** Java Virtual Machine specification vendor. */ + JAVA_VM_SPECIFICATION_VENDOR("java.vm.specification.vendor"), + + /** Java Virtual Machine specification name. */ + JAVA_VM_SPECIFICATION_NAME("java.vm.specification.name"), + + /** Java Virtual Machine implementation version. */ + JAVA_VM_VERSION("java.vm.version"), + + /** Java Virtual Machine implementation vendor. */ + JAVA_VM_VENDOR("java.vm.vendor"), + + /** Java Virtual Machine implementation name. */ + JAVA_VM_NAME("java.vm.name"), + + /** Java Runtime Environment specification version. */ + JAVA_SPECIFICATION_VERSION("java.specification.version"), + + /** Java Runtime Environment specification vendor. */ + JAVA_SPECIFICATION_VENDOR("java.specification.vendor"), + + /** Java Runtime Environment specification name. */ + JAVA_SPECIFICATION_NAME("java.specification.name"), + + /** Java class format version number. */ + JAVA_CLASS_VERSION("java.class.version"), + + /** Java class path. */ + JAVA_CLASS_PATH("java.class.path"), + + /** List of paths to search when loading libraries. */ + JAVA_LIBRARY_PATH("java.library.path"), + + /** Default temp file path. */ + JAVA_IO_TMPDIR("java.io.tmpdir"), + + /** Name of JIT compiler to use. */ + JAVA_COMPILER("java.compiler"), + + /** Path of extension directory or directories. */ + JAVA_EXT_DIRS("java.ext.dirs"), + + /** Operating system name. */ + OS_NAME("os.name"), + + /** Operating system architecture. */ + OS_ARCH("os.arch"), + + /** Operating system version. */ + OS_VERSION("os.version"), + + /** File separator ("/" on UNIX). */ + FILE_SEPARATOR("file.separator"), + + /** Path separator (":" on UNIX). */ + PATH_SEPARATOR("path.separator"), + + /** Line separator ("\n" on UNIX). */ + LINE_SEPARATOR("line.separator"), + + /** User's account name. */ + USER_NAME("user.name"), + + /** User's home directory. */ + USER_HOME("user.home"), + + /** User's current working directory. */ + USER_DIR("user.dir"); + + private final String key; + + StandardSystemProperty(String key) { + this.key = key; + } + + /** Returns the key used to lookup this system property. */ + public String key() { + return key; + } + + /** + * Returns the current value for this system property by delegating to {@link + * System#getProperty(String)}. + */ + + public String value() { + return System.getProperty(key); + } + + /** Returns a string representation of this system property. */ + @Override + public String toString() { + return key() + "=" + value(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Stopwatch.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Stopwatch.java new file mode 100644 index 0000000000000..14a36c6d2c8d2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Stopwatch.java @@ -0,0 +1,264 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static java.util.concurrent.TimeUnit.DAYS; +import static java.util.concurrent.TimeUnit.HOURS; +import static java.util.concurrent.TimeUnit.MICROSECONDS; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +/** + * An object that measures elapsed time in nanoseconds. It is useful to measure elapsed time using + * this class instead of direct calls to {@link System#nanoTime} for a few reasons: + * + *

    + *
  • An alternate time source can be substituted, for testing or performance reasons. + *
  • As documented by {@code nanoTime}, the value returned has no absolute meaning, and can only + * be interpreted as relative to another timestamp returned by {@code nanoTime} at a different + * time. {@code Stopwatch} is a more effective abstraction because it exposes only these + * relative values, not the absolute ones. + *
+ * + *

Basic usage: + * + *

{@code
+ * Stopwatch stopwatch = Stopwatch.createStarted();
+ * doSomething();
+ * stopwatch.stop(); // optional
+ *
+ * Duration duration = stopwatch.elapsed();
+ *
+ * log.info("time: " + stopwatch); // formatted string like "12.3 ms"
+ * }
+ * + *

Stopwatch methods are not idempotent; it is an error to start or stop a stopwatch that is + * already in the desired state. + * + *

When testing code that uses this class, use {@link #createUnstarted(Ticker)} or {@link + * #createStarted(Ticker)} to supply a fake or mock ticker. This allows you to simulate any valid + * behavior of the stopwatch. + * + *

Note: This class is not thread-safe. + * + *

Warning for Android users: a stopwatch with default behavior may not continue to keep + * time while the device is asleep. Instead, create one like this: + * + *

{@code
+ * Stopwatch.createStarted(
+ *      new Ticker() {
+ *        public long read() {
+ *          return android.os.SystemClock.elapsedRealtimeNanos();
+ *        }
+ *      });
+ * }
+ * + * @author Kevin Bourrillion + * @since 10.0 + */ +public final class Stopwatch { + private final Ticker ticker; + private boolean isRunning; + private long elapsedNanos; + private long startTick; + + /** + * Creates (but does not start) a new stopwatch using {@link System#nanoTime} as its time source. + * + * @since 15.0 + */ + public static Stopwatch createUnstarted() { + return new Stopwatch(); + } + + /** + * Creates (but does not start) a new stopwatch, using the specified time source. + * + * @since 15.0 + */ + public static Stopwatch createUnstarted(Ticker ticker) { + return new Stopwatch(ticker); + } + + /** + * Creates (and starts) a new stopwatch using {@link System#nanoTime} as its time source. + * + * @since 15.0 + */ + public static Stopwatch createStarted() { + return new Stopwatch().start(); + } + + /** + * Creates (and starts) a new stopwatch, using the specified time source. + * + * @since 15.0 + */ + public static Stopwatch createStarted(Ticker ticker) { + return new Stopwatch(ticker).start(); + } + + Stopwatch() { + this.ticker = Ticker.systemTicker(); + } + + Stopwatch(Ticker ticker) { + this.ticker = checkNotNull(ticker, "ticker"); + } + + /** + * Returns {@code true} if {@link #start()} has been called on this stopwatch, and {@link #stop()} + * has not been called since the last call to {@code start()}. + */ + public boolean isRunning() { + return isRunning; + } + + /** + * Starts the stopwatch. + * + * @return this {@code Stopwatch} instance + * @throws IllegalStateException if the stopwatch is already running. + */ + public Stopwatch start() { + checkState(!isRunning, "This stopwatch is already running."); + isRunning = true; + startTick = ticker.read(); + return this; + } + + /** + * Stops the stopwatch. Future reads will return the fixed duration that had elapsed up to this + * point. + * + * @return this {@code Stopwatch} instance + * @throws IllegalStateException if the stopwatch is already stopped. + */ + public Stopwatch stop() { + long tick = ticker.read(); + checkState(isRunning, "This stopwatch is already stopped."); + isRunning = false; + elapsedNanos += tick - startTick; + return this; + } + + /** + * Sets the elapsed time for this stopwatch to zero, and places it in a stopped state. + * + * @return this {@code Stopwatch} instance + */ + public Stopwatch reset() { + elapsedNanos = 0; + isRunning = false; + return this; + } + + private long elapsedNanos() { + return isRunning ? ticker.read() - startTick + elapsedNanos : elapsedNanos; + } + + /** + * Returns the current elapsed time shown on this stopwatch, expressed in the desired time unit, + * with any fraction rounded down. + * + *

Note: the overhead of measurement can be more than a microsecond, so it is generally + * not useful to specify {@link TimeUnit#NANOSECONDS} precision here. + * + *

It is generally not a good idea to use an ambiguous, unitless {@code long} to represent + * elapsed time. Therefore, we recommend using {@link #elapsed()} instead, which returns a + * strongly-typed {@link Duration} instance. + * + * @since 14.0 (since 10.0 as {@code elapsedTime()}) + */ + public long elapsed(TimeUnit desiredUnit) { + return desiredUnit.convert(elapsedNanos(), NANOSECONDS); + } + + /** + * Returns the current elapsed time shown on this stopwatch as a {@link Duration}. Unlike {@link + * #elapsed(TimeUnit)}, this method does not lose any precision due to rounding. + * + * @since 22.0 + */ + public Duration elapsed() { + return Duration.ofNanos(elapsedNanos()); + } + + /** Returns a string representation of the current elapsed time. */ + @Override + public String toString() { + long nanos = elapsedNanos(); + + TimeUnit unit = chooseUnit(nanos); + double value = (double) nanos / NANOSECONDS.convert(1, unit); + + // Too bad this functionality is not exposed as a regular method call + return Platform.formatCompact4Digits(value) + " " + abbreviate(unit); + } + + private static TimeUnit chooseUnit(long nanos) { + if (DAYS.convert(nanos, NANOSECONDS) > 0) { + return DAYS; + } + if (HOURS.convert(nanos, NANOSECONDS) > 0) { + return HOURS; + } + if (MINUTES.convert(nanos, NANOSECONDS) > 0) { + return MINUTES; + } + if (SECONDS.convert(nanos, NANOSECONDS) > 0) { + return SECONDS; + } + if (MILLISECONDS.convert(nanos, NANOSECONDS) > 0) { + return MILLISECONDS; + } + if (MICROSECONDS.convert(nanos, NANOSECONDS) > 0) { + return MICROSECONDS; + } + return NANOSECONDS; + } + + private static String abbreviate(TimeUnit unit) { + switch (unit) { + case NANOSECONDS: + return "ns"; + case MICROSECONDS: + return "\u03bcs"; // μs + case MILLISECONDS: + return "ms"; + case SECONDS: + return "s"; + case MINUTES: + return "min"; + case HOURS: + return "h"; + case DAYS: + return "d"; + default: + throw new AssertionError(); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Strings.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Strings.java new file mode 100644 index 0000000000000..69a27202ef133 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Strings.java @@ -0,0 +1,223 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +/** + * Static utility methods pertaining to {@code String} or {@code CharSequence} instances. + * + * @author Kevin Bourrillion + * @since 3.0 + */ +public final class Strings { + private Strings() {} + + /** + * Returns the given string if it is non-null; the empty string otherwise. + * + * @param string the string to test and possibly return + * @return {@code string} itself if it is non-null; {@code ""} if it is null + */ + public static String nullToEmpty(String string) { + return Platform.nullToEmpty(string); + } + + /** + * Returns the given string if it is nonempty; {@code null} otherwise. + * + * @param string the string to test and possibly return + * @return {@code string} itself if it is nonempty; {@code null} if it is empty or null + */ + + public static String emptyToNull(String string) { + return Platform.emptyToNull(string); + } + + /** + * Returns {@code true} if the given string is null or is the empty string. + * + *

Consider normalizing your string references with {@link #nullToEmpty}. If you do, you can + * use {@link String#isEmpty()} instead of this method, and you won't need special null-safe forms + * of methods like {@link String#toUpperCase} either. Or, if you'd like to normalize "in the other + * direction," converting empty strings to {@code null}, you can use {@link #emptyToNull}. + * + * @param string a string reference to check + * @return {@code true} if the string is null or is the empty string + */ + public static boolean isNullOrEmpty(String string) { + return Platform.stringIsNullOrEmpty(string); + } + + /** + * Returns a string, of length at least {@code minLength}, consisting of {@code string} prepended + * with as many copies of {@code padChar} as are necessary to reach that length. For example, + * + *

    + *
  • {@code padStart("7", 3, '0')} returns {@code "007"} + *
  • {@code padStart("2010", 3, '0')} returns {@code "2010"} + *
+ * + *

See {@link java.util.Formatter} for a richer set of formatting capabilities. + * + * @param string the string which should appear at the end of the result + * @param minLength the minimum length the resulting string must have. Can be zero or negative, in + * which case the input string is always returned. + * @param padChar the character to insert at the beginning of the result until the minimum length + * is reached + * @return the padded string + */ + public static String padStart(String string, int minLength, char padChar) { + checkNotNull(string); // eager for GWT. + if (string.length() >= minLength) { + return string; + } + StringBuilder sb = new StringBuilder(minLength); + for (int i = string.length(); i < minLength; i++) { + sb.append(padChar); + } + sb.append(string); + return sb.toString(); + } + + /** + * Returns a string, of length at least {@code minLength}, consisting of {@code string} appended + * with as many copies of {@code padChar} as are necessary to reach that length. For example, + * + *

    + *
  • {@code padEnd("4.", 5, '0')} returns {@code "4.000"} + *
  • {@code padEnd("2010", 3, '!')} returns {@code "2010"} + *
+ * + *

See {@link java.util.Formatter} for a richer set of formatting capabilities. + * + * @param string the string which should appear at the beginning of the result + * @param minLength the minimum length the resulting string must have. Can be zero or negative, in + * which case the input string is always returned. + * @param padChar the character to append to the end of the result until the minimum length is + * reached + * @return the padded string + */ + public static String padEnd(String string, int minLength, char padChar) { + checkNotNull(string); // eager for GWT. + if (string.length() >= minLength) { + return string; + } + StringBuilder sb = new StringBuilder(minLength); + sb.append(string); + for (int i = string.length(); i < minLength; i++) { + sb.append(padChar); + } + return sb.toString(); + } + + /** + * Returns a string consisting of a specific number of concatenated copies of an input string. For + * example, {@code repeat("hey", 3)} returns the string {@code "heyheyhey"}. + * + * @param string any non-null string + * @param count the number of times to repeat it; a nonnegative integer + * @return a string containing {@code string} repeated {@code count} times (the empty string if + * {@code count} is zero) + * @throws IllegalArgumentException if {@code count} is negative + */ + public static String repeat(String string, int count) { + checkNotNull(string); // eager for GWT. + + if (count <= 1) { + checkArgument(count >= 0, "invalid count: %s", count); + return (count == 0) ? "" : string; + } + + // IF YOU MODIFY THE CODE HERE, you must update StringsRepeatBenchmark + final int len = string.length(); + final long longSize = (long) len * (long) count; + final int size = (int) longSize; + if (size != longSize) { + throw new ArrayIndexOutOfBoundsException("Required array size too large: " + longSize); + } + + final char[] array = new char[size]; + string.getChars(0, len, array, 0); + int n; + for (n = len; n < size - n; n <<= 1) { + System.arraycopy(array, 0, array, n, n); + } + System.arraycopy(array, 0, array, n, size - n); + return new String(array); + } + + /** + * Returns the longest string {@code prefix} such that {@code a.toString().startsWith(prefix) && + * b.toString().startsWith(prefix)}, taking care not to split surrogate pairs. If {@code a} and + * {@code b} have no common prefix, returns the empty string. + * + * @since 11.0 + */ + public static String commonPrefix(CharSequence a, CharSequence b) { + checkNotNull(a); + checkNotNull(b); + + int maxPrefixLength = Math.min(a.length(), b.length()); + int p = 0; + while (p < maxPrefixLength && a.charAt(p) == b.charAt(p)) { + p++; + } + if (validSurrogatePairAt(a, p - 1) || validSurrogatePairAt(b, p - 1)) { + p--; + } + return a.subSequence(0, p).toString(); + } + + /** + * Returns the longest string {@code suffix} such that {@code a.toString().endsWith(suffix) && + * b.toString().endsWith(suffix)}, taking care not to split surrogate pairs. If {@code a} and + * {@code b} have no common suffix, returns the empty string. + * + * @since 11.0 + */ + public static String commonSuffix(CharSequence a, CharSequence b) { + checkNotNull(a); + checkNotNull(b); + + int maxSuffixLength = Math.min(a.length(), b.length()); + int s = 0; + while (s < maxSuffixLength && a.charAt(a.length() - s - 1) == b.charAt(b.length() - s - 1)) { + s++; + } + if (validSurrogatePairAt(a, a.length() - s - 1) + || validSurrogatePairAt(b, b.length() - s - 1)) { + s--; + } + return a.subSequence(a.length() - s, a.length()).toString(); + } + + /** + * True when a valid surrogate pair starts at the given {@code index} in the given {@code string}. + * Out-of-range indexes return false. + */ + + static boolean validSurrogatePairAt(CharSequence string, int index) { + return index >= 0 + && index <= (string.length() - 2) + && Character.isHighSurrogate(string.charAt(index)) + && Character.isLowSurrogate(string.charAt(index + 1)); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Supplier.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Supplier.java new file mode 100644 index 0000000000000..ad05f57d59245 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Supplier.java @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +/** + * Legacy version of {@link java.util.function.Supplier java.util.function.Supplier}. Semantically, + * this could be a factory, generator, builder, closure, or something else entirely. No guarantees + * are implied by this interface. + * + *

The {@link Suppliers} class provides common suppliers and related utilities. + * + *

As this interface extends {@code java.util.function.Supplier}, an instance of this type can be + * used as a {@code java.util.function.Supplier} directly. To use a {@code + * java.util.function.Supplier} in a context where a {@code com.google.common.base.Supplier} is + * needed, use {@code supplier::get}. + * + *

See the Guava User Guide article on the use of {@code Function}. + * + * @author Harry Heymann + * @since 2.0 + */ +@FunctionalInterface +public interface Supplier extends java.util.function.Supplier { + /** + * Retrieves an instance of the appropriate type. The returned object may or may not be a new + * instance, depending on the implementation. + * + * @return an instance of the appropriate type + */ + @Override + T get(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Suppliers.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Suppliers.java new file mode 100644 index 0000000000000..08af49b466765 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Suppliers.java @@ -0,0 +1,364 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +import java.io.Serializable; +import java.util.concurrent.TimeUnit; + + +/** + * Useful suppliers. + * + *

All methods return serializable suppliers as long as they're given serializable parameters. + * + * @author Laurence Gonsalves + * @author Harry Heymann + * @since 2.0 + */ +public final class Suppliers { + private Suppliers() {} + + /** + * Returns a new supplier which is the composition of the provided function and supplier. In other + * words, the new supplier's value will be computed by retrieving the value from {@code supplier}, + * and then applying {@code function} to that value. Note that the resulting supplier will not + * call {@code supplier} or invoke {@code function} until it is called. + */ + public static Supplier compose(Function function, Supplier supplier) { + Preconditions.checkNotNull(function); + Preconditions.checkNotNull(supplier); + return new SupplierComposition<>(function, supplier); + } + + private static class SupplierComposition implements Supplier, Serializable { + final Function function; + final Supplier supplier; + + SupplierComposition(Function function, Supplier supplier) { + this.function = function; + this.supplier = supplier; + } + + @Override + public T get() { + return function.apply(supplier.get()); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof SupplierComposition) { + SupplierComposition that = (SupplierComposition) obj; + return function.equals(that.function) && supplier.equals(that.supplier); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(function, supplier); + } + + @Override + public String toString() { + return "Suppliers.compose(" + function + ", " + supplier + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a supplier which caches the instance retrieved during the first call to {@code get()} + * and returns that value on subsequent calls to {@code get()}. See: memoization + * + *

The returned supplier is thread-safe. The delegate's {@code get()} method will be invoked at + * most once unless the underlying {@code get()} throws an exception. The supplier's serialized + * form does not contain the cached value, which will be recalculated when {@code get()} is called + * on the reserialized instance. + * + *

When the underlying delegate throws an exception then this memoizing supplier will keep + * delegating calls until it returns valid data. + * + *

If {@code delegate} is an instance created by an earlier call to {@code memoize}, it is + * returned directly. + */ + public static Supplier memoize(Supplier delegate) { + if (delegate instanceof NonSerializableMemoizingSupplier + || delegate instanceof MemoizingSupplier) { + return delegate; + } + return delegate instanceof Serializable + ? new MemoizingSupplier(delegate) + : new NonSerializableMemoizingSupplier(delegate); + } + + + static class MemoizingSupplier implements Supplier, Serializable { + final Supplier delegate; + transient volatile boolean initialized; + // "value" does not need to be volatile; visibility piggy-backs + // on volatile read of "initialized". + transient T value; + + MemoizingSupplier(Supplier delegate) { + this.delegate = Preconditions.checkNotNull(delegate); + } + + @Override + public T get() { + // A 2-field variant of Double Checked Locking. + if (!initialized) { + synchronized (this) { + if (!initialized) { + T t = delegate.get(); + value = t; + initialized = true; + return t; + } + } + } + return value; + } + + @Override + public String toString() { + return "Suppliers.memoize(" + + (initialized ? "" : delegate) + + ")"; + } + + private static final long serialVersionUID = 0; + } + + + static class NonSerializableMemoizingSupplier implements Supplier { + volatile Supplier delegate; + volatile boolean initialized; + // "value" does not need to be volatile; visibility piggy-backs + // on volatile read of "initialized". + T value; + + NonSerializableMemoizingSupplier(Supplier delegate) { + this.delegate = Preconditions.checkNotNull(delegate); + } + + @Override + public T get() { + // A 2-field variant of Double Checked Locking. + if (!initialized) { + synchronized (this) { + if (!initialized) { + T t = delegate.get(); + value = t; + initialized = true; + // Release the delegate to GC. + delegate = null; + return t; + } + } + } + return value; + } + + @Override + public String toString() { + Supplier delegate = this.delegate; + return "Suppliers.memoize(" + + (delegate == null ? "" : delegate) + + ")"; + } + } + + /** + * Returns a supplier that caches the instance supplied by the delegate and removes the cached + * value after the specified time has passed. Subsequent calls to {@code get()} return the cached + * value if the expiration time has not passed. After the expiration time, a new value is + * retrieved, cached, and returned. See: memoization + * + *

The returned supplier is thread-safe. The supplier's serialized form does not contain the + * cached value, which will be recalculated when {@code get()} is called on the reserialized + * instance. The actual memoization does not happen when the underlying delegate throws an + * exception. + * + *

When the underlying delegate throws an exception then this memoizing supplier will keep + * delegating calls until it returns valid data. + * + * @param duration the length of time after a value is created that it should stop being returned + * by subsequent {@code get()} calls + * @param unit the unit that {@code duration} is expressed in + * @throws IllegalArgumentException if {@code duration} is not positive + * @since 2.0 + */ + public static Supplier memoizeWithExpiration( + Supplier delegate, long duration, TimeUnit unit) { + return new ExpiringMemoizingSupplier(delegate, duration, unit); + } + + + static class ExpiringMemoizingSupplier implements Supplier, Serializable { + final Supplier delegate; + final long durationNanos; + transient volatile T value; + // The special value 0 means "not yet initialized". + transient volatile long expirationNanos; + + ExpiringMemoizingSupplier(Supplier delegate, long duration, TimeUnit unit) { + this.delegate = Preconditions.checkNotNull(delegate); + this.durationNanos = unit.toNanos(duration); + Preconditions.checkArgument(duration > 0); + } + + @Override + public T get() { + // Another variant of Double Checked Locking. + // + // We use two volatile reads. We could reduce this to one by + // putting our fields into a holder class, but (at least on x86) + // the extra memory consumption and indirection are more + // expensive than the extra volatile reads. + long nanos = expirationNanos; + long now = Platform.systemNanoTime(); + if (nanos == 0 || now - nanos >= 0) { + synchronized (this) { + if (nanos == expirationNanos) { // recheck for lost race + T t = delegate.get(); + value = t; + nanos = now + durationNanos; + // In the very unlikely event that nanos is 0, set it to 1; + // no one will notice 1 ns of tardiness. + expirationNanos = (nanos == 0) ? 1 : nanos; + return t; + } + } + } + return value; + } + + @Override + public String toString() { + // This is a little strange if the unit the user provided was not NANOS, + // but we don't want to store the unit just for toString + return "Suppliers.memoizeWithExpiration(" + delegate + ", " + durationNanos + ", NANOS)"; + } + + private static final long serialVersionUID = 0; + } + + /** Returns a supplier that always supplies {@code instance}. */ + public static Supplier ofInstance(T instance) { + return new SupplierOfInstance(instance); + } + + private static class SupplierOfInstance implements Supplier, Serializable { + final T instance; + + SupplierOfInstance(T instance) { + this.instance = instance; + } + + @Override + public T get() { + return instance; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof SupplierOfInstance) { + SupplierOfInstance that = (SupplierOfInstance) obj; + return Objects.equal(instance, that.instance); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(instance); + } + + @Override + public String toString() { + return "Suppliers.ofInstance(" + instance + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a supplier whose {@code get()} method synchronizes on {@code delegate} before calling + * it, making it thread-safe. + */ + public static Supplier synchronizedSupplier(Supplier delegate) { + return new ThreadSafeSupplier(Preconditions.checkNotNull(delegate)); + } + + private static class ThreadSafeSupplier implements Supplier, Serializable { + final Supplier delegate; + + ThreadSafeSupplier(Supplier delegate) { + this.delegate = delegate; + } + + @Override + public T get() { + synchronized (delegate) { + return delegate.get(); + } + } + + @Override + public String toString() { + return "Suppliers.synchronizedSupplier(" + delegate + ")"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a function that accepts a supplier and returns the result of invoking {@link + * Supplier#get} on that supplier. + * + *

Java 8 users: use the method reference {@code Supplier::get} instead. + * + * @since 8.0 + */ + public static Function, T> supplierFunction() { + @SuppressWarnings("unchecked") // implementation is "fully variant" + SupplierFunction sf = (SupplierFunction) SupplierFunctionImpl.INSTANCE; + return sf; + } + + private interface SupplierFunction extends Function, T> {} + + private enum SupplierFunctionImpl implements SupplierFunction { + INSTANCE; + + // Note: This makes T a "pass-through type" + @Override + public Object apply(Supplier input) { + return input.get(); + } + + @Override + public String toString() { + return "Suppliers.supplierFunction()"; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Throwables.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Throwables.java new file mode 100644 index 0000000000000..acddd2c99ff85 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Throwables.java @@ -0,0 +1,523 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.AbstractList; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static java.util.Arrays.asList; +import static java.util.Collections.unmodifiableList; + + +/** + * Static utility methods pertaining to instances of {@link Throwable}. + * + *

See the Guava User Guide entry on Throwables. + * + * @author Kevin Bourrillion + * @author Ben Yu + * @since 1.0 + */ +public final class Throwables { + private Throwables() {} + + /** + * Throws {@code throwable} if it is an instance of {@code declaredType}. Example usage: + * + *

+   * for (Foo foo : foos) {
+   *   try {
+   *     foo.bar();
+   *   } catch (BarException | RuntimeException | Error t) {
+   *     failure = t;
+   *   }
+   * }
+   * if (failure != null) {
+   *   throwIfInstanceOf(failure, BarException.class);
+   *   throwIfUnchecked(failure);
+   *   throw new AssertionError(failure);
+   * }
+   * 
+ * + * @since 20.0 + */ + public static void throwIfInstanceOf( + Throwable throwable, Class declaredType) throws X { + checkNotNull(throwable); + if (declaredType.isInstance(throwable)) { + throw declaredType.cast(throwable); + } + } + + /** + * Propagates {@code throwable} exactly as-is, if and only if it is an instance of {@code + * declaredType}. Example usage: + * + *
+   * try {
+   *   someMethodThatCouldThrowAnything();
+   * } catch (IKnowWhatToDoWithThisException e) {
+   *   handle(e);
+   * } catch (Throwable t) {
+   *   Throwables.propagateIfInstanceOf(t, IOException.class);
+   *   Throwables.propagateIfInstanceOf(t, SQLException.class);
+   *   throw Throwables.propagate(t);
+   * }
+   * 
+ * + * @deprecated Use {@link #throwIfInstanceOf}, which has the same behavior but rejects {@code + * null}. + */ + @Deprecated + public static void propagateIfInstanceOf( + Throwable throwable, Class declaredType) throws X { + if (throwable != null) { + throwIfInstanceOf(throwable, declaredType); + } + } + + /** + * Throws {@code throwable} if it is a {@link RuntimeException} or {@link Error}. Example usage: + * + *
+   * for (Foo foo : foos) {
+   *   try {
+   *     foo.bar();
+   *   } catch (RuntimeException | Error t) {
+   *     failure = t;
+   *   }
+   * }
+   * if (failure != null) {
+   *   throwIfUnchecked(failure);
+   *   throw new AssertionError(failure);
+   * }
+   * 
+ * + * @since 20.0 + */ + public static void throwIfUnchecked(Throwable throwable) { + checkNotNull(throwable); + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } + if (throwable instanceof Error) { + throw (Error) throwable; + } + } + + /** + * Propagates {@code throwable} exactly as-is, if and only if it is an instance of {@link + * RuntimeException} or {@link Error}. Example usage: + * + *
+   * try {
+   *   someMethodThatCouldThrowAnything();
+   * } catch (IKnowWhatToDoWithThisException e) {
+   *   handle(e);
+   * } catch (Throwable t) {
+   *   Throwables.propagateIfPossible(t);
+   *   throw new RuntimeException("unexpected", t);
+   * }
+   * 
+ * + * @deprecated Use {@link #throwIfUnchecked}, which has the same behavior but rejects {@code + * null}. + */ + @Deprecated + public static void propagateIfPossible(Throwable throwable) { + if (throwable != null) { + throwIfUnchecked(throwable); + } + } + + /** + * Propagates {@code throwable} exactly as-is, if and only if it is an instance of {@link + * RuntimeException}, {@link Error}, or {@code declaredType}. Example usage: + * + *
+   * try {
+   *   someMethodThatCouldThrowAnything();
+   * } catch (IKnowWhatToDoWithThisException e) {
+   *   handle(e);
+   * } catch (Throwable t) {
+   *   Throwables.propagateIfPossible(t, OtherException.class);
+   *   throw new RuntimeException("unexpected", t);
+   * }
+   * 
+ * + * @param throwable the Throwable to possibly propagate + * @param declaredType the single checked exception type declared by the calling method + */ + public static void propagateIfPossible( + Throwable throwable, Class declaredType) throws X { + propagateIfInstanceOf(throwable, declaredType); + propagateIfPossible(throwable); + } + + /** + * Propagates {@code throwable} exactly as-is, if and only if it is an instance of {@link + * RuntimeException}, {@link Error}, {@code declaredType1}, or {@code declaredType2}. In the + * unlikely case that you have three or more declared checked exception types, you can handle them + * all by invoking these methods repeatedly. See usage example in {@link + * #propagateIfPossible(Throwable, Class)}. + * + * @param throwable the Throwable to possibly propagate + * @param declaredType1 any checked exception type declared by the calling method + * @param declaredType2 any other checked exception type declared by the calling method + */ + public static void propagateIfPossible( + Throwable throwable, Class declaredType1, Class declaredType2) + throws X1, X2 { + checkNotNull(declaredType2); + propagateIfInstanceOf(throwable, declaredType1); + propagateIfPossible(throwable, declaredType2); + } + + /** + * Propagates {@code throwable} as-is if it is an instance of {@link RuntimeException} or {@link + * Error}, or else as a last resort, wraps it in a {@code RuntimeException} and then propagates. + * + *

This method always throws an exception. The {@code RuntimeException} return type allows + * client code to signal to the compiler that statements after the call are unreachable. Example + * usage: + * + *

+   * T doSomething() {
+   *   try {
+   *     return someMethodThatCouldThrowAnything();
+   *   } catch (IKnowWhatToDoWithThisException e) {
+   *     return handle(e);
+   *   } catch (Throwable t) {
+   *     throw Throwables.propagate(t);
+   *   }
+   * }
+   * 
+ * + * @param throwable the Throwable to propagate + * @return nothing will ever be returned; this return type is only for your convenience, as + * illustrated in the example above + * @deprecated Use {@code throw e} or {@code throw new RuntimeException(e)} directly, or use a + * combination of {@link #throwIfUnchecked} and {@code throw new RuntimeException(e)}. For + * background on the deprecation, read Why we deprecated + * {@code Throwables.propagate}. + */ + @Deprecated + public static RuntimeException propagate(Throwable throwable) { + throwIfUnchecked(throwable); + throw new RuntimeException(throwable); + } + + /** + * Returns the innermost cause of {@code throwable}. The first throwable in a chain provides + * context from when the error or exception was initially detected. Example usage: + * + *
+   * assertEquals("Unable to assign a customer id", Throwables.getRootCause(e).getMessage());
+   * 
+ * + * @throws IllegalArgumentException if there is a loop in the causal chain + */ + public static Throwable getRootCause(Throwable throwable) { + // Keep a second pointer that slowly walks the causal chain. If the fast pointer ever catches + // the slower pointer, then there's a loop. + Throwable slowPointer = throwable; + boolean advanceSlowPointer = false; + + Throwable cause; + while ((cause = throwable.getCause()) != null) { + throwable = cause; + + if (throwable == slowPointer) { + throw new IllegalArgumentException("Loop in causal chain detected.", throwable); + } + if (advanceSlowPointer) { + slowPointer = slowPointer.getCause(); + } + advanceSlowPointer = !advanceSlowPointer; // only advance every other iteration + } + return throwable; + } + + /** + * Gets a {@code Throwable} cause chain as a list. The first entry in the list will be {@code + * throwable} followed by its cause hierarchy. Note that this is a snapshot of the cause chain and + * will not reflect any subsequent changes to the cause chain. + * + *

Here's an example of how it can be used to find specific types of exceptions in the cause + * chain: + * + *

+   * Iterables.filter(Throwables.getCausalChain(e), IOException.class));
+   * 
+ * + * @param throwable the non-null {@code Throwable} to extract causes from + * @return an unmodifiable list containing the cause chain starting with {@code throwable} + * @throws IllegalArgumentException if there is a loop in the causal chain + */ + // TODO(kevinb): decide best return type + public static List getCausalChain(Throwable throwable) { + checkNotNull(throwable); + List causes = new ArrayList<>(4); + causes.add(throwable); + + // Keep a second pointer that slowly walks the causal chain. If the fast pointer ever catches + // the slower pointer, then there's a loop. + Throwable slowPointer = throwable; + boolean advanceSlowPointer = false; + + Throwable cause; + while ((cause = throwable.getCause()) != null) { + throwable = cause; + causes.add(throwable); + + if (throwable == slowPointer) { + throw new IllegalArgumentException("Loop in causal chain detected.", throwable); + } + if (advanceSlowPointer) { + slowPointer = slowPointer.getCause(); + } + advanceSlowPointer = !advanceSlowPointer; // only advance every other iteration + } + return Collections.unmodifiableList(causes); + } + + /** + * Returns {@code throwable}'s cause, cast to {@code expectedCauseType}. + * + *

Prefer this method instead of manually casting an exception's cause. For example, {@code + * (IOException) e.getCause()} throws a {@link ClassCastException} that discards the original + * exception {@code e} if the cause is not an {@link IOException}, but {@code + * Throwables.getCauseAs(e, IOException.class)} keeps {@code e} as the {@link + * ClassCastException}'s cause. + * + * @throws ClassCastException if the cause cannot be cast to the expected type. The {@code + * ClassCastException}'s cause is {@code throwable}. + * @since 22.0 + */ + + public static X getCauseAs( + Throwable throwable, Class expectedCauseType) { + try { + return expectedCauseType.cast(throwable.getCause()); + } catch (ClassCastException e) { + e.initCause(throwable); + throw e; + } + } + + /** + * Returns a string containing the result of {@link Throwable#toString() toString()}, followed by + * the full, recursive stack trace of {@code throwable}. Note that you probably should not be + * parsing the resulting string; if you need programmatic access to the stack frames, you can call + * {@link Throwable#getStackTrace()}. + */ + public static String getStackTraceAsString(Throwable throwable) { + StringWriter stringWriter = new StringWriter(); + throwable.printStackTrace(new PrintWriter(stringWriter)); + return stringWriter.toString(); + } + + /** + * Returns the stack trace of {@code throwable}, possibly providing slower iteration over the full + * trace but faster iteration over parts of the trace. Here, "slower" and "faster" are defined in + * comparison to the normal way to access the stack trace, {@link Throwable#getStackTrace() + * throwable.getStackTrace()}. Note, however, that this method's special implementation is not + * available for all platforms and configurations. If that implementation is unavailable, this + * method falls back to {@code getStackTrace}. Callers that require the special implementation can + * check its availability with {@link #lazyStackTraceIsLazy()}. + * + *

The expected (but not guaranteed) performance of the special implementation differs from + * {@code getStackTrace} in one main way: The {@code lazyStackTrace} call itself returns quickly + * by delaying the per-stack-frame work until each element is accessed. Roughly speaking: + * + *

    + *
  • {@code getStackTrace} takes {@code stackSize} time to return but then negligible time to + * retrieve each element of the returned list. + *
  • {@code lazyStackTrace} takes negligible time to return but then {@code 1/stackSize} time + * to retrieve each element of the returned list (probably slightly more than {@code + * 1/stackSize}). + *
+ * + *

Note: The special implementation does not respect calls to {@link Throwable#setStackTrace + * throwable.setStackTrace}. Instead, it always reflects the original stack trace from the + * exception's creation. + * + * @since 19.0 + */ + // TODO(cpovirk): Say something about the possibility that List access could fail at runtime? + + // TODO(cpovirk): Consider making this available under GWT (slow implementation only). + public static List lazyStackTrace(Throwable throwable) { + return lazyStackTraceIsLazy() + ? jlaStackTrace(throwable) + : unmodifiableList(asList(throwable.getStackTrace())); + } + + /** + * Returns whether {@link #lazyStackTrace} will use the special implementation described in its + * documentation. + * + * @since 19.0 + */ + + public static boolean lazyStackTraceIsLazy() { + return getStackTraceElementMethod != null && getStackTraceDepthMethod != null; + } + + private static List jlaStackTrace(final Throwable t) { + checkNotNull(t); + /* + * TODO(cpovirk): Consider optimizing iterator() to catch IOOBE instead of doing bounds checks. + * + * TODO(cpovirk): Consider the UnsignedBytes pattern if it performs faster and doesn't cause + * AOSP grief. + */ + return new AbstractList() { + @Override + public StackTraceElement get(int n) { + return (StackTraceElement) + invokeAccessibleNonThrowingMethod(getStackTraceElementMethod, jla, t, n); + } + + @Override + public int size() { + return (Integer) invokeAccessibleNonThrowingMethod(getStackTraceDepthMethod, jla, t); + } + }; + } + + private static Object invokeAccessibleNonThrowingMethod( + Method method, Object receiver, Object... params) { + try { + return method.invoke(receiver, params); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } catch (InvocationTargetException e) { + throw propagate(e.getCause()); + } + } + + /** JavaLangAccess class name to load using reflection */ + private static final String JAVA_LANG_ACCESS_CLASSNAME = "sun.misc.JavaLangAccess"; + + /** SharedSecrets class name to load using reflection */ + + static final String SHARED_SECRETS_CLASSNAME = "sun.misc.SharedSecrets"; + + /** Access to some fancy internal JVM internals. */ + + private static final Object jla = getJLA(); + + /** + * The "getStackTraceElementMethod" method, only available on some JDKs so we use reflection to + * find it when available. When this is null, use the slow way. + */ + + private static final Method getStackTraceElementMethod = (jla == null) ? null : getGetMethod(); + + /** + * The "getStackTraceDepth" method, only available on some JDKs so we use reflection to find it + * when available. When this is null, use the slow way. + */ + + private static final Method getStackTraceDepthMethod = (jla == null) ? null : getSizeMethod(); + + /** + * Returns the JavaLangAccess class that is present in all Sun JDKs. It is not whitelisted for + * AppEngine, and not present in non-Sun JDKs. + */ + + @SuppressWarnings("removal") + private static Object getJLA() { + try { + /* + * We load sun.misc.* classes using reflection since Android doesn't support these classes and + * would result in compilation failure if we directly refer to these classes. + */ + Class sharedSecrets = Class.forName(SHARED_SECRETS_CLASSNAME, false, null); + Method langAccess = sharedSecrets.getMethod("getJavaLangAccess"); + return langAccess.invoke(null); + } catch (ThreadDeath death) { + throw death; + } catch (Throwable t) { + /* + * This is not one of AppEngine's whitelisted classes, so even in Sun JDKs, this can fail with + * a NoClassDefFoundError. Other apps might deny access to sun.misc packages. + */ + return null; + } + } + + /** + * Returns the Method that can be used to resolve an individual StackTraceElement, or null if that + * method cannot be found (it is only to be found in fairly recent JDKs). + */ + + private static Method getGetMethod() { + return getJlaMethod("getStackTraceElement", Throwable.class, int.class); + } + + /** + * Returns the Method that can be used to return the size of a stack, or null if that method + * cannot be found (it is only to be found in fairly recent JDKs). Tries to test method {@link + * sun.misc.JavaLangAccess#getStackTraceDepth(Throwable)} getStackTraceDepth} prior to return it + * (might fail some JDKs). + * + *

See Throwables#lazyStackTrace throws + * UnsupportedOperationException. + */ + + private static Method getSizeMethod() { + try { + Method getStackTraceDepth = getJlaMethod("getStackTraceDepth", Throwable.class); + if (getStackTraceDepth == null) { + return null; + } + getStackTraceDepth.invoke(getJLA(), new Throwable()); + return getStackTraceDepth; + } catch (UnsupportedOperationException | IllegalAccessException | InvocationTargetException e) { + return null; + } + } + + + @SuppressWarnings("removal") + private static Method getJlaMethod(String name, Class... parameterTypes) throws ThreadDeath { + try { + return Class.forName(JAVA_LANG_ACCESS_CLASSNAME, false, null).getMethod(name, parameterTypes); + } catch (ThreadDeath death) { + throw death; + } catch (Throwable t) { + /* + * Either the JavaLangAccess class itself is not found, or the method is not supported on the + * JVM. + */ + return null; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ticker.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ticker.java new file mode 100644 index 0000000000000..ce36a3787c791 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Ticker.java @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + + +/** + * A time source; returns a time value representing the number of nanoseconds elapsed since some + * fixed but arbitrary point in time. Note that most users should use {@link Stopwatch} instead of + * interacting with this class directly. + * + *

Warning: this interface can only be used to measure elapsed time, not wall time. + * + * @author Kevin Bourrillion + * @since 10.0 (mostly + * source-compatible since 9.0) + */ + +public abstract class Ticker { + /** Constructor for use by subclasses. */ + protected Ticker() {} + + /** Returns the number of nanoseconds elapsed since this ticker's fixed point of reference. */ + public abstract long read(); + + /** + * A ticker that reads the current time using {@link System#nanoTime}. + * + * @since 10.0 + */ + public static Ticker systemTicker() { + return SYSTEM_TICKER; + } + + private static final Ticker SYSTEM_TICKER = + new Ticker() { + @Override + public long read() { + return Platform.systemNanoTime(); + } + }; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Utf8.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Utf8.java new file mode 100644 index 0000000000000..53fb6a9158303 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Utf8.java @@ -0,0 +1,201 @@ +/* + * Copyright (C) 2013 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; +import static java.lang.Character.MAX_SURROGATE; +import static java.lang.Character.MIN_SURROGATE; + +/** + * Low-level, high-performance utility methods related to the {@linkplain Charsets#UTF_8 UTF-8} + * character encoding. UTF-8 is defined in section D92 of The Unicode Standard Core + * Specification, Chapter 3. + * + *

The variant of UTF-8 implemented by this class is the restricted definition of UTF-8 + * introduced in Unicode 3.1. One implication of this is that it rejects "non-shortest form" byte sequences, + * even though the JDK decoder may accept them. + * + * @author Martin Buchholz + * @author Clément Roux + * @since 16.0 + */ + +public final class Utf8 { + /** + * Returns the number of bytes in the UTF-8-encoded form of {@code sequence}. For a string, this + * method is equivalent to {@code string.getBytes(UTF_8).length}, but is more efficient in both + * time and space. + * + * @throws IllegalArgumentException if {@code sequence} contains ill-formed UTF-16 (unpaired + * surrogates) + */ + public static int encodedLength(CharSequence sequence) { + // Warning to maintainers: this implementation is highly optimized. + int utf16Length = sequence.length(); + int utf8Length = utf16Length; + int i = 0; + + // This loop optimizes for pure ASCII. + while (i < utf16Length && sequence.charAt(i) < 0x80) { + i++; + } + + // This loop optimizes for chars less than 0x800. + for (; i < utf16Length; i++) { + char c = sequence.charAt(i); + if (c < 0x800) { + utf8Length += ((0x7f - c) >>> 31); // branch free! + } else { + utf8Length += encodedLengthGeneral(sequence, i); + break; + } + } + + if (utf8Length < utf16Length) { + // Necessary and sufficient condition for overflow because of maximum 3x expansion + throw new IllegalArgumentException( + "UTF-8 length does not fit in int: " + (utf8Length + (1L << 32))); + } + return utf8Length; + } + + private static int encodedLengthGeneral(CharSequence sequence, int start) { + int utf16Length = sequence.length(); + int utf8Length = 0; + for (int i = start; i < utf16Length; i++) { + char c = sequence.charAt(i); + if (c < 0x800) { + utf8Length += (0x7f - c) >>> 31; // branch free! + } else { + utf8Length += 2; + // jdk7+: if (Character.isSurrogate(c)) { + if (MIN_SURROGATE <= c && c <= MAX_SURROGATE) { + // Check that we have a well-formed surrogate pair. + if (Character.codePointAt(sequence, i) == c) { + throw new IllegalArgumentException(unpairedSurrogateMsg(i)); + } + i++; + } + } + } + return utf8Length; + } + + /** + * Returns {@code true} if {@code bytes} is a well-formed UTF-8 byte sequence according to + * Unicode 6.0. Note that this is a stronger criterion than simply whether the bytes can be + * decoded. For example, some versions of the JDK decoder will accept "non-shortest form" byte + * sequences, but encoding never reproduces these. Such byte sequences are not considered + * well-formed. + * + *

This method returns {@code true} if and only if {@code Arrays.equals(bytes, new + * String(bytes, UTF_8).getBytes(UTF_8))} does, but is more efficient in both time and space. + */ + public static boolean isWellFormed(byte[] bytes) { + return isWellFormed(bytes, 0, bytes.length); + } + + /** + * Returns whether the given byte array slice is a well-formed UTF-8 byte sequence, as defined by + * {@link #isWellFormed(byte[])}. Note that this can be false even when {@code + * isWellFormed(bytes)} is true. + * + * @param bytes the input buffer + * @param off the offset in the buffer of the first byte to read + * @param len the number of bytes to read from the buffer + */ + public static boolean isWellFormed(byte[] bytes, int off, int len) { + int end = off + len; + checkPositionIndexes(off, end, bytes.length); + // Look for the first non-ASCII character. + for (int i = off; i < end; i++) { + if (bytes[i] < 0) { + return isWellFormedSlowPath(bytes, i, end); + } + } + return true; + } + + private static boolean isWellFormedSlowPath(byte[] bytes, int off, int end) { + int index = off; + while (true) { + int byte1; + + // Optimize for interior runs of ASCII bytes. + do { + if (index >= end) { + return true; + } + } while ((byte1 = bytes[index++]) >= 0); + + if (byte1 < (byte) 0xE0) { + // Two-byte form. + if (index == end) { + return false; + } + // Simultaneously check for illegal trailing-byte in leading position + // and overlong 2-byte form. + if (byte1 < (byte) 0xC2 || bytes[index++] > (byte) 0xBF) { + return false; + } + } else if (byte1 < (byte) 0xF0) { + // Three-byte form. + if (index + 1 >= end) { + return false; + } + int byte2 = bytes[index++]; + if (byte2 > (byte) 0xBF + // Overlong? 5 most significant bits must not all be zero. + || (byte1 == (byte) 0xE0 && byte2 < (byte) 0xA0) + // Check for illegal surrogate codepoints. + || (byte1 == (byte) 0xED && (byte) 0xA0 <= byte2) + // Third byte trailing-byte test. + || bytes[index++] > (byte) 0xBF) { + return false; + } + } else { + // Four-byte form. + if (index + 2 >= end) { + return false; + } + int byte2 = bytes[index++]; + if (byte2 > (byte) 0xBF + // Check that 1 <= plane <= 16. Tricky optimized form of: + // if (byte1 > (byte) 0xF4 + // || byte1 == (byte) 0xF0 && byte2 < (byte) 0x90 + // || byte1 == (byte) 0xF4 && byte2 > (byte) 0x8F) + || (((byte1 << 28) + (byte2 - (byte) 0x90)) >> 30) != 0 + // Third byte trailing-byte test + || bytes[index++] > (byte) 0xBF + // Fourth byte trailing-byte test + || bytes[index++] > (byte) 0xBF) { + return false; + } + } + } + } + + private static String unpairedSurrogateMsg(int i) { + return "Unpaired surrogate at index " + i; + } + + private Utf8() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Verify.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Verify.java new file mode 100644 index 0000000000000..4cf22f0ccda0b --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/Verify.java @@ -0,0 +1,524 @@ +/* + * Copyright (C) 2013 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.format; + +/** + * Static convenience methods that serve the same purpose as Java language assertions, + * except that they are always enabled. These methods should be used instead of Java assertions + * whenever there is a chance the check may fail "in real life". Example: + * + *

{@code
+ * Bill bill = remoteService.getLastUnpaidBill();
+ *
+ * // In case bug 12345 happens again we'd rather just die
+ * Verify.verify(bill.status() == Status.UNPAID,
+ *     "Unexpected bill status: %s", bill.status());
+ * }
+ * + *

Comparison to alternatives

+ * + *

Note: In some cases the differences explained below can be subtle. When it's unclear + * which approach to use, don't worry too much about it; just pick something that seems + * reasonable and it will be fine. + * + *

    + *
  • If checking whether the caller has violated your method or constructor's contract + * (such as by passing an invalid argument), use the utilities of the {@link Preconditions} + * class instead. + *
  • If checking an impossible condition (which cannot happen unless your own + * class or its trusted dependencies is badly broken), this is what ordinary Java + * assertions are for. Note that assertions are not enabled by default; they are essentially + * considered "compiled comments." + *
  • An explicit {@code if/throw} (as illustrated below) is always acceptable; we still + * recommend using our {@link VerifyException} exception type. Throwing a plain {@link + * RuntimeException} is frowned upon. + *
  • Use of {@link java.util.Objects#requireNonNull(Object)} is generally discouraged, since + * {@link #verifyNotNull(Object)} and {@link Preconditions#checkNotNull(Object)} perform the + * same function with more clarity. + *
+ * + *

Warning about performance

+ * + *

Remember that parameter values for message construction must all be computed eagerly, and + * autoboxing and varargs array creation may happen as well, even when the verification succeeds and + * the message ends up unneeded. Performance-sensitive verification checks should continue to use + * usual form: + * + *

{@code
+ * Bill bill = remoteService.getLastUnpaidBill();
+ * if (bill.status() != Status.UNPAID) {
+ *   throw new VerifyException("Unexpected bill status: " + bill.status());
+ * }
+ * }
+ * + *

Only {@code %s} is supported

+ * + *

As with {@link Preconditions} error message template strings, only the {@code "%s"} specifier + * is supported, not the full range of {@link java.util.Formatter} specifiers. However, note that if + * the number of arguments does not match the number of occurrences of {@code "%s"} in the format + * string, {@code Verify} will still behave as expected, and will still include all argument values + * in the error message; the message will simply not be formatted exactly as intended. + * + *

More information

+ * + * See Conditional + * failures explained in the Guava User Guide for advice on when this class should be used. + * + * @since 17.0 + */ +public final class Verify { + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with no + * message otherwise. + * + * @throws VerifyException if {@code expression} is {@code false} + * @see Preconditions#checkState Preconditions.checkState() + */ + public static void verify(boolean expression) { + if (!expression) { + throw new VerifyException(); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + * @param expression a boolean expression + * @param errorMessageTemplate a template for the exception message should the check fail. The + * message is formed by replacing each {@code %s} placeholder in the template with an + * argument. These are matched by position - the first {@code %s} gets {@code + * errorMessageArgs[0]}, etc. Unmatched arguments will be appended to the formatted message in + * square braces. Unmatched placeholders will be left as-is. + * @param errorMessageArgs the arguments to be substituted into the message template. Arguments + * are converted to strings using {@link String#valueOf(Object)}. + * @throws VerifyException if {@code expression} is {@code false} + * @see Preconditions#checkState Preconditions.checkState() + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object... errorMessageArgs) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, errorMessageArgs)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, char p1) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify(boolean expression, String errorMessageTemplate, int p1) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, long p1) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, Object p1) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, char p1, char p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, int p1, char p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, long p1, char p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object p1, + char p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, char p1, int p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, int p1, int p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, long p1, int p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object p1, + int p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, char p1, long p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, int p1, long p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, String errorMessageTemplate, long p1, long p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object p1, + long p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + char p1, + Object p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + int p1, + Object p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + long p1, + Object p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object p1, + Object p2) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2, p3)); + } + } + + /** + * Ensures that {@code expression} is {@code true}, throwing a {@code VerifyException} with a + * custom message otherwise. + * + *

See {@link #verify(boolean, String, Object...)} for details. + * + * @since 23.1 (varargs overload since 17.0) + */ + public static void verify( + boolean expression, + String errorMessageTemplate, + Object p1, + Object p2, + Object p3, + Object p4) { + if (!expression) { + throw new VerifyException(format(errorMessageTemplate, p1, p2, p3, p4)); + } + } + + /** + * Ensures that {@code reference} is non-null, throwing a {@code VerifyException} with a default + * message otherwise. + * + * @return {@code reference}, guaranteed to be non-null, for convenience + * @throws VerifyException if {@code reference} is {@code null} + * @see Preconditions#checkNotNull Preconditions.checkNotNull() + */ + public static T verifyNotNull(T reference) { + return verifyNotNull(reference, "expected a non-null reference"); + } + + /** + * Ensures that {@code reference} is non-null, throwing a {@code VerifyException} with a custom + * message otherwise. + * + * @param errorMessageTemplate a template for the exception message should the check fail. The + * message is formed by replacing each {@code %s} placeholder in the template with an + * argument. These are matched by position - the first {@code %s} gets {@code + * errorMessageArgs[0]}, etc. Unmatched arguments will be appended to the formatted message in + * square braces. Unmatched placeholders will be left as-is. + * @param errorMessageArgs the arguments to be substituted into the message template. Arguments + * are converted to strings using {@link String#valueOf(Object)}. + * @return {@code reference}, guaranteed to be non-null, for convenience + * @throws VerifyException if {@code reference} is {@code null} + * @see Preconditions#checkNotNull Preconditions.checkNotNull() + */ + public static T verifyNotNull( + T reference, + String errorMessageTemplate, + Object... errorMessageArgs) { + verify(reference != null, errorMessageTemplate, errorMessageArgs); + return reference; + } + + // TODO(kevinb): consider T verifySingleton(Iterable) to take over for + // Iterables.getOnlyElement() + + private Verify() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/VerifyException.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/VerifyException.java new file mode 100644 index 0000000000000..454b7dfa4f618 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/VerifyException.java @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2013 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; + +/** + * Exception thrown upon the failure of a verification check, + * including those performed by the convenience methods of the {@link Verify} class. + * + * @since 17.0 + */ +public class VerifyException extends RuntimeException { + /** Constructs a {@code VerifyException} with no message. */ + public VerifyException() {} + + /** Constructs a {@code VerifyException} with the message {@code message}. */ + public VerifyException(String message) { + super(message); + } + + /** + * Constructs a {@code VerifyException} with the cause {@code cause} and a message that is {@code + * null} if {@code cause} is null, and {@code cause.toString()} otherwise. + * + * @since 19.0 + */ + public VerifyException(Throwable cause) { + super(cause); + } + + /** + * Constructs a {@code VerifyException} with the message {@code message} and the cause {@code + * cause}. + * + * @since 19.0 + */ + public VerifyException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/internal/Finalizer.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/internal/Finalizer.java new file mode 100644 index 0000000000000..2ac13d825daab --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/internal/Finalizer.java @@ -0,0 +1,239 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base.internal; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.ref.PhantomReference; +import java.lang.ref.Reference; +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; + + +/** + * Thread that finalizes referents. All references should implement {@code + * com.google.common.base.FinalizableReference}. + * + *

While this class is public, we consider it to be *internal* and not part of our published API. + * It is public so we can access it reflectively across class loaders in secure environments. + * + *

This class can't depend on other Guava code. If we were to load this class in the same class + * loader as the rest of Guava, this thread would keep an indirect strong reference to the class + * loader and prevent it from being garbage collected. This poses a problem for environments where + * you want to throw away the class loader. For example, dynamically reloading a web application or + * unloading an OSGi bundle. + * + *

{@code com.google.common.base.FinalizableReferenceQueue} loads this class in its own class + * loader. That way, this class doesn't prevent the main class loader from getting garbage + * collected, and this class can detect when the main class loader has been garbage collected and + * stop itself. + */ +public class Finalizer implements Runnable { + + private static final Logger logger = LoggerFactory.getLogger(Finalizer.class.getName()); + + /** Name of FinalizableReference.class. */ + private static final String FINALIZABLE_REFERENCE = "com.google.common.base.FinalizableReference"; + + /** + * Starts the Finalizer thread. FinalizableReferenceQueue calls this method reflectively. + * + * @param finalizableReferenceClass FinalizableReference.class. + * @param queue a reference queue that the thread will poll. + * @param frqReference a phantom reference to the FinalizableReferenceQueue, which will be queued + * either when the FinalizableReferenceQueue is no longer referenced anywhere, or when its + * close() method is called. + */ + public static void startFinalizer( + Class finalizableReferenceClass, + ReferenceQueue queue, + PhantomReference frqReference) { + /* + * We use FinalizableReference.class for two things: + * + * 1) To invoke FinalizableReference.finalizeReferent() + * + * 2) To detect when FinalizableReference's class loader has to be garbage collected, at which + * point, Finalizer can stop running + */ + if (!finalizableReferenceClass.getName().equals(FINALIZABLE_REFERENCE)) { + throw new IllegalArgumentException("Expected " + FINALIZABLE_REFERENCE + "."); + } + + Finalizer finalizer = new Finalizer(finalizableReferenceClass, queue, frqReference); + String threadName = Finalizer.class.getName(); + Thread thread = null; + if (bigThreadConstructor != null) { + try { + boolean inheritThreadLocals = false; + long defaultStackSize = 0; + thread = + bigThreadConstructor.newInstance( + (ThreadGroup) null, finalizer, threadName, defaultStackSize, inheritThreadLocals); + } catch (Throwable t) { + logger.info("Failed to create a thread without inherited thread-local values", t); + } + } + if (thread == null) { + thread = new Thread((ThreadGroup) null, finalizer, threadName); + } + thread.setDaemon(true); + + try { + if (inheritableThreadLocals != null) { + inheritableThreadLocals.set(thread, null); + } + } catch (Throwable t) { + logger.info("Failed to clear thread local values inherited by reference finalizer thread.", t); + } + + thread.start(); + } + + private final WeakReference> finalizableReferenceClassReference; + private final PhantomReference frqReference; + private final ReferenceQueue queue; + + // By preference, we will use the Thread constructor that has an `inheritThreadLocals` parameter. + // But before Java 9, our only way not to inherit ThreadLocals is to zap them after the thread + // is created, by accessing a private field. + + private static final Constructor bigThreadConstructor = getBigThreadConstructor(); + + + private static final Field inheritableThreadLocals = + (bigThreadConstructor == null) ? getInheritableThreadLocalsField() : null; + + /** Constructs a new finalizer thread. */ + private Finalizer( + Class finalizableReferenceClass, + ReferenceQueue queue, + PhantomReference frqReference) { + this.queue = queue; + + this.finalizableReferenceClassReference = + new WeakReference>(finalizableReferenceClass); + + // Keep track of the FRQ that started us so we know when to stop. + this.frqReference = frqReference; + } + + /** Loops continuously, pulling references off the queue and cleaning them up. */ + @SuppressWarnings("InfiniteLoopStatement") + @Override + public void run() { + while (true) { + try { + if (!cleanUp(queue.remove())) { + break; + } + } catch (InterruptedException e) { + // ignore + } + } + } + + /** + * Cleans up a single reference. Catches and logs all throwables. + * + * @return true if the caller should continue, false if the associated FinalizableReferenceQueue + * is no longer referenced. + */ + private boolean cleanUp(Reference reference) { + Method finalizeReferentMethod = getFinalizeReferentMethod(); + if (finalizeReferentMethod == null) { + return false; + } + do { + /* + * This is for the benefit of phantom references. Weak and soft references will have already + * been cleared by this point. + */ + reference.clear(); + + if (reference == frqReference) { + /* + * The client no longer has a reference to the FinalizableReferenceQueue. We can stop. + */ + return false; + } + + try { + finalizeReferentMethod.invoke(reference); + } catch (Throwable t) { + logger.error("Error cleaning up after reference.", t); + } + + /* + * Loop as long as we have references available so as not to waste CPU looking up the Method + * over and over again. + */ + } while ((reference = queue.poll()) != null); + return true; + } + + /** Looks up FinalizableReference.finalizeReferent() method. */ + + private Method getFinalizeReferentMethod() { + Class finalizableReferenceClass = finalizableReferenceClassReference.get(); + if (finalizableReferenceClass == null) { + /* + * FinalizableReference's class loader was reclaimed. While there's a chance that other + * finalizable references could be enqueued subsequently (at which point the class loader + * would be resurrected by virtue of us having a strong reference to it), we should pretty + * much just shut down and make sure we don't keep it alive any longer than necessary. + */ + return null; + } + try { + return finalizableReferenceClass.getMethod("finalizeReferent"); + } catch (NoSuchMethodException e) { + throw new AssertionError(e); + } + } + + + private static Field getInheritableThreadLocalsField() { + try { + Field inheritableThreadLocals = Thread.class.getDeclaredField("inheritableThreadLocals"); + inheritableThreadLocals.setAccessible(true); + return inheritableThreadLocals; + } catch (Throwable t) { + logger.info( + "Couldn't access Thread.inheritableThreadLocals. Reference finalizer threads will " + + "inherit thread local values."); + return null; + } + } + + + private static Constructor getBigThreadConstructor() { + try { + return Thread.class.getConstructor( + ThreadGroup.class, Runnable.class, String.class, long.class, boolean.class); + } catch (Throwable t) { + // Probably pre Java 9. We'll fall back to Thread.inheritableThreadLocals. + return null; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/package-info.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/package-info.java new file mode 100644 index 0000000000000..85c0c03dbbd27 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/base/package-info.java @@ -0,0 +1,22 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Basic utility libraries and interfaces. + */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.base; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractBiMap.java new file mode 100644 index 0000000000000..b2e431d44213d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractBiMap.java @@ -0,0 +1,470 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; + + + +/** + * A general-purpose bimap implementation using any two backing {@code Map} instances. + * + *

Note that this class contains {@code equals()} calls that keep it from supporting {@code + * IdentityHashMap} backing maps. + * + * @author Kevin Bourrillion + * @author Mike Bostock + */ +abstract class AbstractBiMap extends ForwardingMap + implements BiMap, Serializable { + + private transient Map delegate; + transient AbstractBiMap inverse; + + /** Package-private constructor for creating a map-backed bimap. */ + AbstractBiMap(Map forward, Map backward) { + setDelegates(forward, backward); + } + + /** Private constructor for inverse bimap. */ + private AbstractBiMap(Map backward, AbstractBiMap forward) { + delegate = backward; + inverse = forward; + } + + @Override + protected Map delegate() { + return delegate; + } + + /** Returns its input, or throws an exception if this is not a valid key. */ + K checkKey(K key) { + return key; + } + + /** Returns its input, or throws an exception if this is not a valid value. */ + V checkValue(V value) { + return value; + } + + /** + * Specifies the delegate maps going in each direction. Called by the constructor and by + * subclasses during deserialization. + */ + void setDelegates(Map forward, Map backward) { + checkState(delegate == null); + checkState(inverse == null); + checkArgument(forward.isEmpty()); + checkArgument(backward.isEmpty()); + checkArgument(forward != backward); + delegate = forward; + inverse = makeInverse(backward); + } + + AbstractBiMap makeInverse(Map backward) { + return new Inverse<>(backward, this); + } + + void setInverse(AbstractBiMap inverse) { + this.inverse = inverse; + } + + // Query Operations (optimizations) + + @Override + public boolean containsValue(Object value) { + return inverse.containsKey(value); + } + + // Modification Operations + + @Override + public V put(K key, V value) { + return putInBothMaps(key, value, false); + } + + @Override + public V forcePut(K key, V value) { + return putInBothMaps(key, value, true); + } + + private V putInBothMaps(K key, V value, boolean force) { + checkKey(key); + checkValue(value); + boolean containedKey = containsKey(key); + if (containedKey && Objects.equal(value, get(key))) { + return value; + } + if (force) { + inverse().remove(value); + } else { + checkArgument(!containsValue(value), "value already present: %s", value); + } + V oldValue = delegate.put(key, value); + updateInverseMap(key, containedKey, oldValue, value); + return oldValue; + } + + private void updateInverseMap(K key, boolean containedKey, V oldValue, V newValue) { + if (containedKey) { + removeFromInverseMap(oldValue); + } + inverse.delegate.put(newValue, key); + } + + @Override + public V remove(Object key) { + return containsKey(key) ? removeFromBothMaps(key) : null; + } + + private V removeFromBothMaps(Object key) { + V oldValue = delegate.remove(key); + removeFromInverseMap(oldValue); + return oldValue; + } + + private void removeFromInverseMap(V oldValue) { + inverse.delegate.remove(oldValue); + } + + // Bulk Operations + + @Override + public void putAll(Map map) { + for (Entry entry : map.entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + @Override + public void replaceAll(BiFunction function) { + this.delegate.replaceAll(function); + inverse.delegate.clear(); + Entry broken = null; + Iterator> itr = this.delegate.entrySet().iterator(); + while (itr.hasNext()) { + Entry entry = itr.next(); + K k = entry.getKey(); + V v = entry.getValue(); + K conflict = inverse.delegate.putIfAbsent(v, k); + if (conflict != null) { + broken = entry; + // We're definitely going to throw, but we'll try to keep the BiMap in an internally + // consistent state by removing the bad entry. + itr.remove(); + } + } + if (broken != null) { + throw new IllegalArgumentException("value already present: " + broken.getValue()); + } + } + + @Override + public void clear() { + delegate.clear(); + inverse.delegate.clear(); + } + + // Views + + @Override + public BiMap inverse() { + return inverse; + } + + private transient Set keySet; + + @Override + public Set keySet() { + Set result = keySet; + return (result == null) ? keySet = new KeySet() : result; + } + + private class KeySet extends ForwardingSet { + @Override + protected Set delegate() { + return delegate.keySet(); + } + + @Override + public void clear() { + AbstractBiMap.this.clear(); + } + + @Override + public boolean remove(Object key) { + if (!contains(key)) { + return false; + } + removeFromBothMaps(key); + return true; + } + + @Override + public boolean removeAll(Collection keysToRemove) { + return standardRemoveAll(keysToRemove); + } + + @Override + public boolean retainAll(Collection keysToRetain) { + return standardRetainAll(keysToRetain); + } + + @Override + public Iterator iterator() { + return Maps.keyIterator(entrySet().iterator()); + } + } + + private transient Set valueSet; + + @Override + public Set values() { + /* + * We can almost reuse the inverse's keySet, except we have to fix the + * iteration order so that it is consistent with the forward map. + */ + Set result = valueSet; + return (result == null) ? valueSet = new ValueSet() : result; + } + + private class ValueSet extends ForwardingSet { + final Set valuesDelegate = inverse.keySet(); + + @Override + protected Set delegate() { + return valuesDelegate; + } + + @Override + public Iterator iterator() { + return Maps.valueIterator(entrySet().iterator()); + } + + @Override + public Object[] toArray() { + return standardToArray(); + } + + @Override + public T[] toArray(T[] array) { + return standardToArray(array); + } + + @Override + public String toString() { + return standardToString(); + } + } + + private transient Set> entrySet; + + @Override + public Set> entrySet() { + Set> result = entrySet; + return (result == null) ? entrySet = new EntrySet() : result; + } + + class BiMapEntry extends ForwardingMapEntry { + private final Entry delegate; + + BiMapEntry(Entry delegate) { + this.delegate = delegate; + } + + @Override + protected Entry delegate() { + return delegate; + } + + @Override + public V setValue(V value) { + checkValue(value); + // Preconditions keep the map and inverse consistent. + checkState(entrySet().contains(this), "entry no longer in map"); + // similar to putInBothMaps, but set via entry + if (Objects.equal(value, getValue())) { + return value; + } + checkArgument(!containsValue(value), "value already present: %s", value); + V oldValue = delegate.setValue(value); + checkState(Objects.equal(value, get(getKey())), "entry no longer in map"); + updateInverseMap(getKey(), true, oldValue, value); + return oldValue; + } + } + + Iterator> entrySetIterator() { + final Iterator> iterator = delegate.entrySet().iterator(); + return new Iterator>() { + Entry entry; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Entry next() { + entry = iterator.next(); + return new BiMapEntry(entry); + } + + @Override + public void remove() { + checkRemove(entry != null); + V value = entry.getValue(); + iterator.remove(); + removeFromInverseMap(value); + entry = null; + } + }; + } + + private class EntrySet extends ForwardingSet> { + final Set> esDelegate = delegate.entrySet(); + + @Override + protected Set> delegate() { + return esDelegate; + } + + @Override + public void clear() { + AbstractBiMap.this.clear(); + } + + @Override + public boolean remove(Object object) { + if (!esDelegate.contains(object)) { + return false; + } + + // safe because esDelegate.contains(object). + Entry entry = (Entry) object; + inverse.delegate.remove(entry.getValue()); + /* + * Remove the mapping in inverse before removing from esDelegate because + * if entry is part of esDelegate, entry might be invalidated after the + * mapping is removed from esDelegate. + */ + esDelegate.remove(entry); + return true; + } + + @Override + public Iterator> iterator() { + return entrySetIterator(); + } + + // See java.util.Collections.CheckedEntrySet for details on attacks. + + @Override + public Object[] toArray() { + return standardToArray(); + } + + @Override + public T[] toArray(T[] array) { + return standardToArray(array); + } + + @Override + public boolean contains(Object o) { + return Maps.containsEntryImpl(delegate(), o); + } + + @Override + public boolean containsAll(Collection c) { + return standardContainsAll(c); + } + + @Override + public boolean removeAll(Collection c) { + return standardRemoveAll(c); + } + + @Override + public boolean retainAll(Collection c) { + return standardRetainAll(c); + } + } + + /** The inverse of any other {@code AbstractBiMap} subclass. */ + static class Inverse extends AbstractBiMap { + Inverse(Map backward, AbstractBiMap forward) { + super(backward, forward); + } + + /* + * Serialization stores the forward bimap, the inverse of this inverse. + * Deserialization calls inverse() on the forward bimap and returns that + * inverse. + * + * If a bimap and its inverse are serialized together, the deserialized + * instances have inverse() methods that return the other. + */ + + @Override + K checkKey(K key) { + return inverse.checkValue(key); + } + + @Override + V checkValue(V value) { + return inverse.checkKey(value); + } + + /** @serialData the forward bimap */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(inverse()); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + setInverse((AbstractBiMap) stream.readObject()); + } + + Object readResolve() { + return inverse().inverse(); + } + + private static final long serialVersionUID = 0; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIndexedListIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIndexedListIterator.java new file mode 100644 index 0000000000000..db57062147aea --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIndexedListIterator.java @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; + +import java.util.ListIterator; +import java.util.NoSuchElementException; + +/** + * This class provides a skeletal implementation of the {@link ListIterator} interface across a + * fixed number of elements that may be retrieved by position. It does not support {@link #remove}, + * {@link #set}, or {@link #add}. + * + * @author Jared Levy + */ +abstract class AbstractIndexedListIterator extends UnmodifiableListIterator { + private final int size; + private int position; + + /** Returns the element with the specified index. This method is called by {@link #next()}. */ + protected abstract E get(int index); + + /** + * Constructs an iterator across a sequence of the given size whose initial position is 0. That + * is, the first call to {@link #next()} will return the first element (or throw {@link + * NoSuchElementException} if {@code size} is zero). + * + * @throws IllegalArgumentException if {@code size} is negative + */ + protected AbstractIndexedListIterator(int size) { + this(size, 0); + } + + /** + * Constructs an iterator across a sequence of the given size with the given initial position. + * That is, the first call to {@link #nextIndex()} will return {@code position}, and the first + * call to {@link #next()} will return the element at that index, if available. Calls to {@link + * #previous()} can retrieve the preceding {@code position} elements. + * + * @throws IndexOutOfBoundsException if {@code position} is negative or is greater than {@code + * size} + * @throws IllegalArgumentException if {@code size} is negative + */ + protected AbstractIndexedListIterator(int size, int position) { + checkPositionIndex(position, size); + this.size = size; + this.position = position; + } + + @Override + public final boolean hasNext() { + return position < size; + } + + @Override + public final E next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return get(position++); + } + + @Override + public final int nextIndex() { + return position; + } + + @Override + public final boolean hasPrevious() { + return position > 0; + } + + @Override + public final E previous() { + if (!hasPrevious()) { + throw new NoSuchElementException(); + } + return get(--position); + } + + @Override + public final int previousIndex() { + return position - 1; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIterator.java new file mode 100644 index 0000000000000..761648c4b1442 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractIterator.java @@ -0,0 +1,172 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; + +import java.util.NoSuchElementException; + + +/** + * This class provides a skeletal implementation of the {@code Iterator} interface, to make this + * interface easier to implement for certain types of data sources. + * + *

{@code Iterator} requires its implementations to support querying the end-of-data status + * without changing the iterator's state, using the {@link #hasNext} method. But many data sources, + * such as {@link java.io.Reader#read()}, do not expose this information; the only way to discover + * whether there is any data left is by trying to retrieve it. These types of data sources are + * ordinarily difficult to write iterators for. But using this class, one must implement only the + * {@link #computeNext} method, and invoke the {@link #endOfData} method when appropriate. + * + *

Another example is an iterator that skips over null elements in a backing iterator. This could + * be implemented as: + * + *

{@code
+ * public static Iterator skipNulls(final Iterator in) {
+ *   return new AbstractIterator() {
+ *     protected String computeNext() {
+ *       while (in.hasNext()) {
+ *         String s = in.next();
+ *         if (s != null) {
+ *           return s;
+ *         }
+ *       }
+ *       return endOfData();
+ *     }
+ *   };
+ * }
+ * }
+ * + *

This class supports iterators that include null elements. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +// When making changes to this class, please also update the copy at +// com.google.common.base.AbstractIterator +public abstract class AbstractIterator extends UnmodifiableIterator { + private State state = State.NOT_READY; + + /** Constructor for use by subclasses. */ + protected AbstractIterator() {} + + private enum State { + /** We have computed the next element and haven't returned it yet. */ + READY, + + /** We haven't yet computed or have already returned the element. */ + NOT_READY, + + /** We have reached the end of the data and are finished. */ + DONE, + + /** We've suffered an exception and are kaput. */ + FAILED, + } + + private T next; + + /** + * Returns the next element. Note: the implementation must call {@link #endOfData()} when + * there are no elements left in the iteration. Failure to do so could result in an infinite loop. + * + *

The initial invocation of {@link #hasNext()} or {@link #next()} calls this method, as does + * the first invocation of {@code hasNext} or {@code next} following each successful call to + * {@code next}. Once the implementation either invokes {@code endOfData} or throws an exception, + * {@code computeNext} is guaranteed to never be called again. + * + *

If this method throws an exception, it will propagate outward to the {@code hasNext} or + * {@code next} invocation that invoked this method. Any further attempts to use the iterator will + * result in an {@link IllegalStateException}. + * + *

The implementation of this method may not invoke the {@code hasNext}, {@code next}, or + * {@link #peek()} methods on this instance; if it does, an {@code IllegalStateException} will + * result. + * + * @return the next element if there was one. If {@code endOfData} was called during execution, + * the return value will be ignored. + * @throws RuntimeException if any unrecoverable error happens. This exception will propagate + * outward to the {@code hasNext()}, {@code next()}, or {@code peek()} invocation that invoked + * this method. Any further attempts to use the iterator will result in an {@link + * IllegalStateException}. + */ + protected abstract T computeNext(); + + /** + * Implementations of {@link #computeNext} must invoke this method when there are no + * elements left in the iteration. + * + * @return {@code null}; a convenience so your {@code computeNext} implementation can use the + * simple statement {@code return endOfData();} + */ + protected final T endOfData() { + state = State.DONE; + return null; + } + + @Override + public final boolean hasNext() { + checkState(state != State.FAILED); + switch (state) { + case DONE: + return false; + case READY: + return true; + default: + } + return tryToComputeNext(); + } + + private boolean tryToComputeNext() { + state = State.FAILED; // temporary pessimism + next = computeNext(); + if (state != State.DONE) { + state = State.READY; + return true; + } + return false; + } + + @Override + public final T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + state = State.NOT_READY; + T result = next; + next = null; + return result; + } + + /** + * Returns the next element in the iteration without advancing the iteration, according to the + * contract of {@link PeekingIterator#peek()}. + * + *

Implementations of {@code AbstractIterator} that wish to expose this functionality should + * implement {@code PeekingIterator}. + */ + public final T peek() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return next; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractListMultimap.java new file mode 100644 index 0000000000000..d97e0e4097845 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractListMultimap.java @@ -0,0 +1,139 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; + + +/** + * Basic implementation of the {@link ListMultimap} interface. It's a wrapper around {@link + * AbstractMapBasedMultimap} that converts the returned collections into {@code Lists}. The {@link + * #createCollection} method must return a {@code List}. + * + * @author Jared Levy + * @since 2.0 + */ +abstract class AbstractListMultimap extends AbstractMapBasedMultimap + implements ListMultimap { + /** + * Creates a new multimap that uses the provided map. + * + * @param map place to store the mapping from each key to its corresponding values + */ + protected AbstractListMultimap(Map> map) { + super(map); + } + + @Override + abstract List createCollection(); + + @Override + List createUnmodifiableEmptyCollection() { + return Collections.emptyList(); + } + + @Override + Collection unmodifiableCollectionSubclass(Collection collection) { + return Collections.unmodifiableList((List) collection); + } + + @Override + Collection wrapCollection(K key, Collection collection) { + return wrapList(key, (List) collection, null); + } + + // Following Javadoc copied from ListMultimap. + + /** + * {@inheritDoc} + * + *

Because the values for a given key may have duplicates and follow the insertion ordering, + * this method returns a {@link List}, instead of the {@link Collection} specified in the {@link + * Multimap} interface. + */ + @Override + public List get(K key) { + return (List) super.get(key); + } + + /** + * {@inheritDoc} + * + *

Because the values for a given key may have duplicates and follow the insertion ordering, + * this method returns a {@link List}, instead of the {@link Collection} specified in the {@link + * Multimap} interface. + */ + @Override + public List removeAll(Object key) { + return (List) super.removeAll(key); + } + + /** + * {@inheritDoc} + * + *

Because the values for a given key may have duplicates and follow the insertion ordering, + * this method returns a {@link List}, instead of the {@link Collection} specified in the {@link + * Multimap} interface. + */ + @Override + public List replaceValues(K key, Iterable values) { + return (List) super.replaceValues(key, values); + } + + /** + * Stores a key-value pair in the multimap. + * + * @param key key to store in the multimap + * @param value value to store in the multimap + * @return {@code true} always + */ + @Override + public boolean put(K key, V value) { + return super.put(key, value); + } + + /** + * {@inheritDoc} + * + *

Though the method signature doesn't say so explicitly, the returned map has {@link List} + * values. + */ + @Override + public Map> asMap() { + return super.asMap(); + } + + /** + * Compares the specified object to this multimap for equality. + * + *

Two {@code ListMultimap} instances are equal if, for each key, they contain the same values + * in the same order. If the value orderings disagree, the multimaps will not be considered equal. + */ + @Override + public boolean equals(Object object) { + return super.equals(object); + } + + private static final long serialVersionUID = 6588350623831699109L; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultimap.java new file mode 100644 index 0000000000000..f5a277b875a40 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultimap.java @@ -0,0 +1,1630 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.ViewCachingAbstractMap; + +import java.io.Serializable; +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.RandomAccess; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.Spliterator; +import java.util.function.BiConsumer; + + + +/** + * Basic implementation of the {@link Multimap} interface. This class represents a multimap as a map + * that associates each key with a collection of values. All methods of {@link Multimap} are + * supported, including those specified as optional in the interface. + * + *

To implement a multimap, a subclass must define the method {@link #createCollection()}, which + * creates an empty collection of values for a key. + * + *

The multimap constructor takes a map that has a single entry for each distinct key. When you + * insert a key-value pair with a key that isn't already in the multimap, {@code + * AbstractMapBasedMultimap} calls {@link #createCollection()} to create the collection of values + * for that key. The subclass should not call {@link #createCollection()} directly, and a new + * instance should be created every time the method is called. + * + *

For example, the subclass could pass a {@link java.util.TreeMap} during construction, and + * {@link #createCollection()} could return a {@link java.util.TreeSet}, in which case the + * multimap's iterators would propagate through the keys and values in sorted order. + * + *

Keys and values may be null, as long as the underlying collection classes support null + * elements. + * + *

The collections created by {@link #createCollection()} may or may not allow duplicates. If the + * collection, such as a {@link Set}, does not support duplicates, an added key-value pair will + * replace an existing pair with the same key and value, if such a pair is present. With collections + * like {@link List} that allow duplicates, the collection will keep the existing key-value pairs + * while adding a new pair. + * + *

This class is not threadsafe when any concurrent operations update the multimap, even if the + * underlying map and {@link #createCollection()} method return threadsafe classes. Concurrent read + * operations will work correctly. To allow concurrent update operations, wrap your multimap with a + * call to {@link Multimaps#synchronizedMultimap}. + * + *

For serialization to work, the subclass must specify explicit {@code readObject} and {@code + * writeObject} methods. + * + * @author Jared Levy + * @author Louis Wasserman + */ +abstract class AbstractMapBasedMultimap extends AbstractMultimap + implements Serializable { + /* + * Here's an outline of the overall design. + * + * The map variable contains the collection of values associated with each + * key. When a key-value pair is added to a multimap that didn't previously + * contain any values for that key, a new collection generated by + * createCollection is added to the map. That same collection instance + * remains in the map as long as the multimap has any values for the key. If + * all values for the key are removed, the key and collection are removed + * from the map. + * + * The get method returns a WrappedCollection, which decorates the collection + * in the map (if the key is present) or an empty collection (if the key is + * not present). When the collection delegate in the WrappedCollection is + * empty, the multimap may contain subsequently added values for that key. To + * handle that situation, the WrappedCollection checks whether map contains + * an entry for the provided key, and if so replaces the delegate. + */ + + private transient Map> map; + private transient int totalSize; + + /** + * Creates a new multimap that uses the provided map. + * + * @param map place to store the mapping from each key to its corresponding values + * @throws IllegalArgumentException if {@code map} is not empty + */ + protected AbstractMapBasedMultimap(Map> map) { + checkArgument(map.isEmpty()); + this.map = map; + } + + /** Used during deserialization only. */ + final void setMap(Map> map) { + this.map = map; + totalSize = 0; + for (Collection values : map.values()) { + checkArgument(!values.isEmpty()); + totalSize += values.size(); + } + } + + /** + * Creates an unmodifiable, empty collection of values. + * + *

This is used in {@link #removeAll} on an empty key. + */ + Collection createUnmodifiableEmptyCollection() { + return unmodifiableCollectionSubclass(createCollection()); + } + + /** + * Creates the collection of values for a single key. + * + *

Collections with weak, soft, or phantom references are not supported. Each call to {@code + * createCollection} should create a new instance. + * + *

The returned collection class determines whether duplicate key-value pairs are allowed. + * + * @return an empty collection of values + */ + abstract Collection createCollection(); + + /** + * Creates the collection of values for an explicitly provided key. By default, it simply calls + * {@link #createCollection()}, which is the correct behavior for most implementations. The {@link + * LinkedHashMultimap} class overrides it. + * + * @param key key to associate with values in the collection + * @return an empty collection of values + */ + Collection createCollection(K key) { + return createCollection(); + } + + Map> backingMap() { + return map; + } + + // Query Operations + + @Override + public int size() { + return totalSize; + } + + @Override + public boolean containsKey(Object key) { + return map.containsKey(key); + } + + // Modification Operations + + @Override + public boolean put(K key, V value) { + Collection collection = map.get(key); + if (collection == null) { + collection = createCollection(key); + if (collection.add(value)) { + totalSize++; + map.put(key, collection); + return true; + } else { + throw new AssertionError("New Collection violated the Collection spec"); + } + } else if (collection.add(value)) { + totalSize++; + return true; + } else { + return false; + } + } + + private Collection getOrCreateCollection(K key) { + Collection collection = map.get(key); + if (collection == null) { + collection = createCollection(key); + map.put(key, collection); + } + return collection; + } + + // Bulk Operations + + /** + * {@inheritDoc} + * + *

The returned collection is immutable. + */ + @Override + public Collection replaceValues(K key, Iterable values) { + Iterator iterator = values.iterator(); + if (!iterator.hasNext()) { + return removeAll(key); + } + + // TODO(lowasser): investigate atomic failure? + Collection collection = getOrCreateCollection(key); + Collection oldValues = createCollection(); + oldValues.addAll(collection); + + totalSize -= collection.size(); + collection.clear(); + + while (iterator.hasNext()) { + if (collection.add(iterator.next())) { + totalSize++; + } + } + + return unmodifiableCollectionSubclass(oldValues); + } + + /** + * {@inheritDoc} + * + *

The returned collection is immutable. + */ + @Override + public Collection removeAll(Object key) { + Collection collection = map.remove(key); + + if (collection == null) { + return createUnmodifiableEmptyCollection(); + } + + Collection output = createCollection(); + output.addAll(collection); + totalSize -= collection.size(); + collection.clear(); + + return unmodifiableCollectionSubclass(output); + } + + Collection unmodifiableCollectionSubclass(Collection collection) { + return Collections.unmodifiableCollection(collection); + } + + @Override + public void clear() { + // Clear each collection, to make previously returned collections empty. + for (Collection collection : map.values()) { + collection.clear(); + } + map.clear(); + totalSize = 0; + } + + // Views + + /** + * {@inheritDoc} + * + *

The returned collection is not serializable. + */ + @Override + public Collection get(K key) { + Collection collection = map.get(key); + if (collection == null) { + collection = createCollection(key); + } + return wrapCollection(key, collection); + } + + /** + * Generates a decorated collection that remains consistent with the values in the multimap for + * the provided key. Changes to the multimap may alter the returned collection, and vice versa. + */ + Collection wrapCollection(K key, Collection collection) { + return new WrappedCollection(key, collection, null); + } + + final List wrapList( + K key, List list, WrappedCollection ancestor) { + return (list instanceof RandomAccess) + ? new RandomAccessWrappedList(key, list, ancestor) + : new WrappedList(key, list, ancestor); + } + + /** + * Collection decorator that stays in sync with the multimap values for a key. There are two kinds + * of wrapped collections: full and subcollections. Both have a delegate pointing to the + * underlying collection class. + * + *

Full collections, identified by a null ancestor field, contain all multimap values for a + * given key. Its delegate is a value in {@link AbstractMapBasedMultimap#map} whenever the + * delegate is non-empty. The {@code refreshIfEmpty}, {@code removeIfEmpty}, and {@code addToMap} + * methods ensure that the {@code WrappedCollection} and map remain consistent. + * + *

A subcollection, such as a sublist, contains some of the values for a given key. Its + * ancestor field points to the full wrapped collection with all values for the key. The + * subcollection {@code refreshIfEmpty}, {@code removeIfEmpty}, and {@code addToMap} methods call + * the corresponding methods of the full wrapped collection. + */ + class WrappedCollection extends AbstractCollection { + final K key; + Collection delegate; + final WrappedCollection ancestor; + final Collection ancestorDelegate; + + WrappedCollection( + K key, Collection delegate, WrappedCollection ancestor) { + this.key = key; + this.delegate = delegate; + this.ancestor = ancestor; + this.ancestorDelegate = (ancestor == null) ? null : ancestor.getDelegate(); + } + + /** + * If the delegate collection is empty, but the multimap has values for the key, replace the + * delegate with the new collection for the key. + * + *

For a subcollection, refresh its ancestor and validate that the ancestor delegate hasn't + * changed. + */ + void refreshIfEmpty() { + if (ancestor != null) { + ancestor.refreshIfEmpty(); + if (ancestor.getDelegate() != ancestorDelegate) { + throw new ConcurrentModificationException(); + } + } else if (delegate.isEmpty()) { + Collection newDelegate = map.get(key); + if (newDelegate != null) { + delegate = newDelegate; + } + } + } + + /** + * If collection is empty, remove it from {@code AbstractMapBasedMultimap.this.map}. For + * subcollections, check whether the ancestor collection is empty. + */ + void removeIfEmpty() { + if (ancestor != null) { + ancestor.removeIfEmpty(); + } else if (delegate.isEmpty()) { + map.remove(key); + } + } + + K getKey() { + return key; + } + + /** + * Add the delegate to the map. Other {@code WrappedCollection} methods should call this method + * after adding elements to a previously empty collection. + * + *

Subcollection add the ancestor's delegate instead. + */ + void addToMap() { + if (ancestor != null) { + ancestor.addToMap(); + } else { + map.put(key, delegate); + } + } + + @Override + public int size() { + refreshIfEmpty(); + return delegate.size(); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + refreshIfEmpty(); + return delegate.equals(object); + } + + @Override + public int hashCode() { + refreshIfEmpty(); + return delegate.hashCode(); + } + + @Override + public String toString() { + refreshIfEmpty(); + return delegate.toString(); + } + + Collection getDelegate() { + return delegate; + } + + @Override + public Iterator iterator() { + refreshIfEmpty(); + return new WrappedIterator(); + } + + @Override + public Spliterator spliterator() { + refreshIfEmpty(); + return delegate.spliterator(); + } + + /** Collection iterator for {@code WrappedCollection}. */ + class WrappedIterator implements Iterator { + final Iterator delegateIterator; + final Collection originalDelegate = delegate; + + WrappedIterator() { + delegateIterator = iteratorOrListIterator(delegate); + } + + WrappedIterator(Iterator delegateIterator) { + this.delegateIterator = delegateIterator; + } + + /** + * If the delegate changed since the iterator was created, the iterator is no longer valid. + */ + void validateIterator() { + refreshIfEmpty(); + if (delegate != originalDelegate) { + throw new ConcurrentModificationException(); + } + } + + @Override + public boolean hasNext() { + validateIterator(); + return delegateIterator.hasNext(); + } + + @Override + public V next() { + validateIterator(); + return delegateIterator.next(); + } + + @Override + public void remove() { + delegateIterator.remove(); + totalSize--; + removeIfEmpty(); + } + + Iterator getDelegateIterator() { + validateIterator(); + return delegateIterator; + } + } + + @Override + public boolean add(V value) { + refreshIfEmpty(); + boolean wasEmpty = delegate.isEmpty(); + boolean changed = delegate.add(value); + if (changed) { + totalSize++; + if (wasEmpty) { + addToMap(); + } + } + return changed; + } + + WrappedCollection getAncestor() { + return ancestor; + } + + // The following methods are provided for better performance. + + @Override + public boolean addAll(Collection collection) { + if (collection.isEmpty()) { + return false; + } + int oldSize = size(); // calls refreshIfEmpty + boolean changed = delegate.addAll(collection); + if (changed) { + int newSize = delegate.size(); + totalSize += (newSize - oldSize); + if (oldSize == 0) { + addToMap(); + } + } + return changed; + } + + @Override + public boolean contains(Object o) { + refreshIfEmpty(); + return delegate.contains(o); + } + + @Override + public boolean containsAll(Collection c) { + refreshIfEmpty(); + return delegate.containsAll(c); + } + + @Override + public void clear() { + int oldSize = size(); // calls refreshIfEmpty + if (oldSize == 0) { + return; + } + delegate.clear(); + totalSize -= oldSize; + removeIfEmpty(); // maybe shouldn't be removed if this is a sublist + } + + @Override + public boolean remove(Object o) { + refreshIfEmpty(); + boolean changed = delegate.remove(o); + if (changed) { + totalSize--; + removeIfEmpty(); + } + return changed; + } + + @Override + public boolean removeAll(Collection c) { + if (c.isEmpty()) { + return false; + } + int oldSize = size(); // calls refreshIfEmpty + boolean changed = delegate.removeAll(c); + if (changed) { + int newSize = delegate.size(); + totalSize += (newSize - oldSize); + removeIfEmpty(); + } + return changed; + } + + @Override + public boolean retainAll(Collection c) { + checkNotNull(c); + int oldSize = size(); // calls refreshIfEmpty + boolean changed = delegate.retainAll(c); + if (changed) { + int newSize = delegate.size(); + totalSize += (newSize - oldSize); + removeIfEmpty(); + } + return changed; + } + } + + private static Iterator iteratorOrListIterator(Collection collection) { + return (collection instanceof List) + ? ((List) collection).listIterator() + : collection.iterator(); + } + + /** Set decorator that stays in sync with the multimap values for a key. */ + class WrappedSet extends WrappedCollection implements Set { + WrappedSet(K key, Set delegate) { + super(key, delegate, null); + } + + @Override + public boolean removeAll(Collection c) { + if (c.isEmpty()) { + return false; + } + int oldSize = size(); // calls refreshIfEmpty + + // Guava issue 1013: AbstractSet and most JDK set implementations are + // susceptible to quadratic removeAll performance on lists; + // use a slightly smarter implementation here + boolean changed = Sets.removeAllImpl((Set) delegate, c); + if (changed) { + int newSize = delegate.size(); + totalSize += (newSize - oldSize); + removeIfEmpty(); + } + return changed; + } + } + + /** SortedSet decorator that stays in sync with the multimap values for a key. */ + class WrappedSortedSet extends WrappedCollection implements SortedSet { + WrappedSortedSet( + K key, SortedSet delegate, WrappedCollection ancestor) { + super(key, delegate, ancestor); + } + + SortedSet getSortedSetDelegate() { + return (SortedSet) getDelegate(); + } + + @Override + public Comparator comparator() { + return getSortedSetDelegate().comparator(); + } + + @Override + public V first() { + refreshIfEmpty(); + return getSortedSetDelegate().first(); + } + + @Override + public V last() { + refreshIfEmpty(); + return getSortedSetDelegate().last(); + } + + @Override + public SortedSet headSet(V toElement) { + refreshIfEmpty(); + return new WrappedSortedSet( + getKey(), + getSortedSetDelegate().headSet(toElement), + (getAncestor() == null) ? this : getAncestor()); + } + + @Override + public SortedSet subSet(V fromElement, V toElement) { + refreshIfEmpty(); + return new WrappedSortedSet( + getKey(), + getSortedSetDelegate().subSet(fromElement, toElement), + (getAncestor() == null) ? this : getAncestor()); + } + + @Override + public SortedSet tailSet(V fromElement) { + refreshIfEmpty(); + return new WrappedSortedSet( + getKey(), + getSortedSetDelegate().tailSet(fromElement), + (getAncestor() == null) ? this : getAncestor()); + } + } + + class WrappedNavigableSet extends WrappedSortedSet implements NavigableSet { + WrappedNavigableSet( + K key, NavigableSet delegate, WrappedCollection ancestor) { + super(key, delegate, ancestor); + } + + @Override + NavigableSet getSortedSetDelegate() { + return (NavigableSet) super.getSortedSetDelegate(); + } + + @Override + public V lower(V v) { + return getSortedSetDelegate().lower(v); + } + + @Override + public V floor(V v) { + return getSortedSetDelegate().floor(v); + } + + @Override + public V ceiling(V v) { + return getSortedSetDelegate().ceiling(v); + } + + @Override + public V higher(V v) { + return getSortedSetDelegate().higher(v); + } + + @Override + public V pollFirst() { + return Iterators.pollNext(iterator()); + } + + @Override + public V pollLast() { + return Iterators.pollNext(descendingIterator()); + } + + private NavigableSet wrap(NavigableSet wrapped) { + return new WrappedNavigableSet(key, wrapped, (getAncestor() == null) ? this : getAncestor()); + } + + @Override + public NavigableSet descendingSet() { + return wrap(getSortedSetDelegate().descendingSet()); + } + + @Override + public Iterator descendingIterator() { + return new WrappedIterator(getSortedSetDelegate().descendingIterator()); + } + + @Override + public NavigableSet subSet( + V fromElement, boolean fromInclusive, V toElement, boolean toInclusive) { + return wrap( + getSortedSetDelegate().subSet(fromElement, fromInclusive, toElement, toInclusive)); + } + + @Override + public NavigableSet headSet(V toElement, boolean inclusive) { + return wrap(getSortedSetDelegate().headSet(toElement, inclusive)); + } + + @Override + public NavigableSet tailSet(V fromElement, boolean inclusive) { + return wrap(getSortedSetDelegate().tailSet(fromElement, inclusive)); + } + } + + /** List decorator that stays in sync with the multimap values for a key. */ + class WrappedList extends WrappedCollection implements List { + WrappedList(K key, List delegate, WrappedCollection ancestor) { + super(key, delegate, ancestor); + } + + List getListDelegate() { + return (List) getDelegate(); + } + + @Override + public boolean addAll(int index, Collection c) { + if (c.isEmpty()) { + return false; + } + int oldSize = size(); // calls refreshIfEmpty + boolean changed = getListDelegate().addAll(index, c); + if (changed) { + int newSize = getDelegate().size(); + totalSize += (newSize - oldSize); + if (oldSize == 0) { + addToMap(); + } + } + return changed; + } + + @Override + public V get(int index) { + refreshIfEmpty(); + return getListDelegate().get(index); + } + + @Override + public V set(int index, V element) { + refreshIfEmpty(); + return getListDelegate().set(index, element); + } + + @Override + public void add(int index, V element) { + refreshIfEmpty(); + boolean wasEmpty = getDelegate().isEmpty(); + getListDelegate().add(index, element); + totalSize++; + if (wasEmpty) { + addToMap(); + } + } + + @Override + public V remove(int index) { + refreshIfEmpty(); + V value = getListDelegate().remove(index); + totalSize--; + removeIfEmpty(); + return value; + } + + @Override + public int indexOf(Object o) { + refreshIfEmpty(); + return getListDelegate().indexOf(o); + } + + @Override + public int lastIndexOf(Object o) { + refreshIfEmpty(); + return getListDelegate().lastIndexOf(o); + } + + @Override + public ListIterator listIterator() { + refreshIfEmpty(); + return new WrappedListIterator(); + } + + @Override + public ListIterator listIterator(int index) { + refreshIfEmpty(); + return new WrappedListIterator(index); + } + + @Override + public List subList(int fromIndex, int toIndex) { + refreshIfEmpty(); + return wrapList( + getKey(), + getListDelegate().subList(fromIndex, toIndex), + (getAncestor() == null) ? this : getAncestor()); + } + + /** ListIterator decorator. */ + private class WrappedListIterator extends WrappedIterator implements ListIterator { + WrappedListIterator() {} + + public WrappedListIterator(int index) { + super(getListDelegate().listIterator(index)); + } + + private ListIterator getDelegateListIterator() { + return (ListIterator) getDelegateIterator(); + } + + @Override + public boolean hasPrevious() { + return getDelegateListIterator().hasPrevious(); + } + + @Override + public V previous() { + return getDelegateListIterator().previous(); + } + + @Override + public int nextIndex() { + return getDelegateListIterator().nextIndex(); + } + + @Override + public int previousIndex() { + return getDelegateListIterator().previousIndex(); + } + + @Override + public void set(V value) { + getDelegateListIterator().set(value); + } + + @Override + public void add(V value) { + boolean wasEmpty = isEmpty(); + getDelegateListIterator().add(value); + totalSize++; + if (wasEmpty) { + addToMap(); + } + } + } + } + + /** + * List decorator that stays in sync with the multimap values for a key and supports rapid random + * access. + */ + private class RandomAccessWrappedList extends WrappedList implements RandomAccess { + RandomAccessWrappedList( + K key, List delegate, WrappedCollection ancestor) { + super(key, delegate, ancestor); + } + } + + @Override + Set createKeySet() { + return new KeySet(map); + } + + final Set createMaybeNavigableKeySet() { + if (map instanceof NavigableMap) { + return new NavigableKeySet((NavigableMap>) map); + } else if (map instanceof SortedMap) { + return new SortedKeySet((SortedMap>) map); + } else { + return new KeySet(map); + } + } + + private class KeySet extends Maps.KeySet> { + KeySet(final Map> subMap) { + super(subMap); + } + + @Override + public Iterator iterator() { + final Iterator>> entryIterator = map().entrySet().iterator(); + return new Iterator() { + Entry> entry; + + @Override + public boolean hasNext() { + return entryIterator.hasNext(); + } + + @Override + public K next() { + entry = entryIterator.next(); + return entry.getKey(); + } + + @Override + public void remove() { + checkRemove(entry != null); + Collection collection = entry.getValue(); + entryIterator.remove(); + totalSize -= collection.size(); + collection.clear(); + entry = null; + } + }; + } + + // The following methods are included for better performance. + + @Override + public Spliterator spliterator() { + return map().keySet().spliterator(); + } + + @Override + public boolean remove(Object key) { + int count = 0; + Collection collection = map().remove(key); + if (collection != null) { + count = collection.size(); + collection.clear(); + totalSize -= count; + } + return count > 0; + } + + @Override + public void clear() { + Iterators.clear(iterator()); + } + + @Override + public boolean containsAll(Collection c) { + return map().keySet().containsAll(c); + } + + @Override + public boolean equals(Object object) { + return this == object || this.map().keySet().equals(object); + } + + @Override + public int hashCode() { + return map().keySet().hashCode(); + } + } + + private class SortedKeySet extends KeySet implements SortedSet { + + SortedKeySet(SortedMap> subMap) { + super(subMap); + } + + SortedMap> sortedMap() { + return (SortedMap>) super.map(); + } + + @Override + public Comparator comparator() { + return sortedMap().comparator(); + } + + @Override + public K first() { + return sortedMap().firstKey(); + } + + @Override + public SortedSet headSet(K toElement) { + return new SortedKeySet(sortedMap().headMap(toElement)); + } + + @Override + public K last() { + return sortedMap().lastKey(); + } + + @Override + public SortedSet subSet(K fromElement, K toElement) { + return new SortedKeySet(sortedMap().subMap(fromElement, toElement)); + } + + @Override + public SortedSet tailSet(K fromElement) { + return new SortedKeySet(sortedMap().tailMap(fromElement)); + } + } + + class NavigableKeySet extends SortedKeySet implements NavigableSet { + NavigableKeySet(NavigableMap> subMap) { + super(subMap); + } + + @Override + NavigableMap> sortedMap() { + return (NavigableMap>) super.sortedMap(); + } + + @Override + public K lower(K k) { + return sortedMap().lowerKey(k); + } + + @Override + public K floor(K k) { + return sortedMap().floorKey(k); + } + + @Override + public K ceiling(K k) { + return sortedMap().ceilingKey(k); + } + + @Override + public K higher(K k) { + return sortedMap().higherKey(k); + } + + @Override + public K pollFirst() { + return Iterators.pollNext(iterator()); + } + + @Override + public K pollLast() { + return Iterators.pollNext(descendingIterator()); + } + + @Override + public NavigableSet descendingSet() { + return new NavigableKeySet(sortedMap().descendingMap()); + } + + @Override + public Iterator descendingIterator() { + return descendingSet().iterator(); + } + + @Override + public NavigableSet headSet(K toElement) { + return headSet(toElement, false); + } + + @Override + public NavigableSet headSet(K toElement, boolean inclusive) { + return new NavigableKeySet(sortedMap().headMap(toElement, inclusive)); + } + + @Override + public NavigableSet subSet(K fromElement, K toElement) { + return subSet(fromElement, true, toElement, false); + } + + @Override + public NavigableSet subSet( + K fromElement, boolean fromInclusive, K toElement, boolean toInclusive) { + return new NavigableKeySet( + sortedMap().subMap(fromElement, fromInclusive, toElement, toInclusive)); + } + + @Override + public NavigableSet tailSet(K fromElement) { + return tailSet(fromElement, true); + } + + @Override + public NavigableSet tailSet(K fromElement, boolean inclusive) { + return new NavigableKeySet(sortedMap().tailMap(fromElement, inclusive)); + } + } + + /** Removes all values for the provided key. */ + private void removeValuesForKey(Object key) { + Collection collection = Maps.safeRemove(map, key); + + if (collection != null) { + int count = collection.size(); + collection.clear(); + totalSize -= count; + } + } + + private abstract class Itr implements Iterator { + final Iterator>> keyIterator; + K key; + Collection collection; + Iterator valueIterator; + + Itr() { + keyIterator = map.entrySet().iterator(); + key = null; + collection = null; + valueIterator = Iterators.emptyModifiableIterator(); + } + + abstract T output(K key, V value); + + @Override + public boolean hasNext() { + return keyIterator.hasNext() || valueIterator.hasNext(); + } + + @Override + public T next() { + if (!valueIterator.hasNext()) { + Entry> mapEntry = keyIterator.next(); + key = mapEntry.getKey(); + collection = mapEntry.getValue(); + valueIterator = collection.iterator(); + } + return output(key, valueIterator.next()); + } + + @Override + public void remove() { + valueIterator.remove(); + if (collection.isEmpty()) { + keyIterator.remove(); + } + totalSize--; + } + } + + /** + * {@inheritDoc} + * + *

The iterator generated by the returned collection traverses the values for one key, followed + * by the values of a second key, and so on. + */ + @Override + public Collection values() { + return super.values(); + } + + @Override + Collection createValues() { + return new Values(); + } + + @Override + Iterator valueIterator() { + return new Itr() { + @Override + V output(K key, V value) { + return value; + } + }; + } + + @Override + Spliterator valueSpliterator() { + return CollectSpliterators.flatMap( + map.values().spliterator(), Collection::spliterator, Spliterator.SIZED, size()); + } + + /* + * TODO(kevinb): should we copy this javadoc to each concrete class, so that + * classes like LinkedHashMultimap that need to say something different are + * still able to {@inheritDoc} all the way from Multimap? + */ + + @Override + Multiset createKeys() { + return new Multimaps.Keys(this); + } + + /** + * {@inheritDoc} + * + *

The iterator generated by the returned collection traverses the values for one key, followed + * by the values of a second key, and so on. + * + *

Each entry is an immutable snapshot of a key-value mapping in the multimap, taken at the + * time the entry is returned by a method call to the collection or its iterator. + */ + @Override + public Collection> entries() { + return super.entries(); + } + + @Override + Collection> createEntries() { + if (this instanceof SetMultimap) { + return new EntrySet(); + } else { + return new Entries(); + } + } + + /** + * Returns an iterator across all key-value map entries, used by {@code entries().iterator()} and + * {@code values().iterator()}. The default behavior, which traverses the values for one key, the + * values for a second key, and so on, suffices for most {@code AbstractMapBasedMultimap} + * implementations. + * + * @return an iterator across map entries + */ + @Override + Iterator> entryIterator() { + return new Itr>() { + @Override + Entry output(K key, V value) { + return Maps.immutableEntry(key, value); + } + }; + } + + @Override + Spliterator> entrySpliterator() { + return CollectSpliterators.flatMap( + map.entrySet().spliterator(), + keyToValueCollectionEntry -> { + K key = keyToValueCollectionEntry.getKey(); + Collection valueCollection = keyToValueCollectionEntry.getValue(); + return CollectSpliterators.map( + valueCollection.spliterator(), (V value) -> Maps.immutableEntry(key, value)); + }, + Spliterator.SIZED, + size()); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + map.forEach( + (key, valueCollection) -> valueCollection.forEach(value -> action.accept(key, value))); + } + + @Override + Map> createAsMap() { + return new AsMap(map); + } + + final Map> createMaybeNavigableAsMap() { + if (map instanceof NavigableMap) { + return new NavigableAsMap((NavigableMap>) map); + } else if (map instanceof SortedMap) { + return new SortedAsMap((SortedMap>) map); + } else { + return new AsMap(map); + } + } + + private class AsMap extends ViewCachingAbstractMap> { + /** + * Usually the same as map, but smaller for the headMap(), tailMap(), or subMap() of a + * SortedAsMap. + */ + final transient Map> submap; + + AsMap(Map> submap) { + this.submap = submap; + } + + @Override + protected Set>> createEntrySet() { + return new AsMapEntries(); + } + + // The following methods are included for performance. + + @Override + public boolean containsKey(Object key) { + return Maps.safeContainsKey(submap, key); + } + + @Override + public Collection get(Object key) { + Collection collection = Maps.safeGet(submap, key); + if (collection == null) { + return null; + } + @SuppressWarnings({"unchecked", "rawtypes"}) + K k = (K) key; + return wrapCollection(k, collection); + } + + @Override + public Set keySet() { + return AbstractMapBasedMultimap.this.keySet(); + } + + @Override + public int size() { + return submap.size(); + } + + @Override + public Collection remove(Object key) { + Collection collection = submap.remove(key); + if (collection == null) { + return null; + } + + Collection output = createCollection(); + output.addAll(collection); + totalSize -= collection.size(); + collection.clear(); + return output; + } + + @Override + public boolean equals(Object object) { + return this == object || submap.equals(object); + } + + @Override + public int hashCode() { + return submap.hashCode(); + } + + @Override + public String toString() { + return submap.toString(); + } + + @Override + public void clear() { + if (submap == map) { + AbstractMapBasedMultimap.this.clear(); + } else { + Iterators.clear(new AsMapIterator()); + } + } + + Entry> wrapEntry(Entry> entry) { + K key = entry.getKey(); + return Maps.immutableEntry(key, wrapCollection(key, entry.getValue())); + } + + class AsMapEntries extends Maps.EntrySet> { + @Override + Map> map() { + return AsMap.this; + } + + @Override + public Iterator>> iterator() { + return new AsMapIterator(); + } + + @Override + public Spliterator>> spliterator() { + return CollectSpliterators.map(submap.entrySet().spliterator(), AsMap.this::wrapEntry); + } + + // The following methods are included for performance. + + @Override + public boolean contains(Object o) { + return Collections2.safeContains(submap.entrySet(), o); + } + + @Override + public boolean remove(Object o) { + if (!contains(o)) { + return false; + } + Entry entry = (Entry) o; + removeValuesForKey(entry.getKey()); + return true; + } + } + + /** Iterator across all keys and value collections. */ + class AsMapIterator implements Iterator>> { + final Iterator>> delegateIterator = submap.entrySet().iterator(); + Collection collection; + + @Override + public boolean hasNext() { + return delegateIterator.hasNext(); + } + + @Override + public Entry> next() { + Entry> entry = delegateIterator.next(); + collection = entry.getValue(); + return wrapEntry(entry); + } + + @Override + public void remove() { + checkRemove(collection != null); + delegateIterator.remove(); + totalSize -= collection.size(); + collection.clear(); + collection = null; + } + } + } + + private class SortedAsMap extends AsMap implements SortedMap> { + SortedAsMap(SortedMap> submap) { + super(submap); + } + + SortedMap> sortedMap() { + return (SortedMap>) submap; + } + + @Override + public Comparator comparator() { + return sortedMap().comparator(); + } + + @Override + public K firstKey() { + return sortedMap().firstKey(); + } + + @Override + public K lastKey() { + return sortedMap().lastKey(); + } + + @Override + public SortedMap> headMap(K toKey) { + return new SortedAsMap(sortedMap().headMap(toKey)); + } + + @Override + public SortedMap> subMap(K fromKey, K toKey) { + return new SortedAsMap(sortedMap().subMap(fromKey, toKey)); + } + + @Override + public SortedMap> tailMap(K fromKey) { + return new SortedAsMap(sortedMap().tailMap(fromKey)); + } + + SortedSet sortedKeySet; + + // returns a SortedSet, even though returning a Set would be sufficient to + // satisfy the SortedMap.keySet() interface + @Override + public SortedSet keySet() { + SortedSet result = sortedKeySet; + return (result == null) ? sortedKeySet = createKeySet() : result; + } + + @Override + SortedSet createKeySet() { + return new SortedKeySet(sortedMap()); + } + } + + class NavigableAsMap extends SortedAsMap implements NavigableMap> { + + NavigableAsMap(NavigableMap> submap) { + super(submap); + } + + @Override + NavigableMap> sortedMap() { + return (NavigableMap>) super.sortedMap(); + } + + @Override + public Entry> lowerEntry(K key) { + Entry> entry = sortedMap().lowerEntry(key); + return (entry == null) ? null : wrapEntry(entry); + } + + @Override + public K lowerKey(K key) { + return sortedMap().lowerKey(key); + } + + @Override + public Entry> floorEntry(K key) { + Entry> entry = sortedMap().floorEntry(key); + return (entry == null) ? null : wrapEntry(entry); + } + + @Override + public K floorKey(K key) { + return sortedMap().floorKey(key); + } + + @Override + public Entry> ceilingEntry(K key) { + Entry> entry = sortedMap().ceilingEntry(key); + return (entry == null) ? null : wrapEntry(entry); + } + + @Override + public K ceilingKey(K key) { + return sortedMap().ceilingKey(key); + } + + @Override + public Entry> higherEntry(K key) { + Entry> entry = sortedMap().higherEntry(key); + return (entry == null) ? null : wrapEntry(entry); + } + + @Override + public K higherKey(K key) { + return sortedMap().higherKey(key); + } + + @Override + public Entry> firstEntry() { + Entry> entry = sortedMap().firstEntry(); + return (entry == null) ? null : wrapEntry(entry); + } + + @Override + public Entry> lastEntry() { + Entry> entry = sortedMap().lastEntry(); + return (entry == null) ? null : wrapEntry(entry); + } + + @Override + public Entry> pollFirstEntry() { + return pollAsMapEntry(entrySet().iterator()); + } + + @Override + public Entry> pollLastEntry() { + return pollAsMapEntry(descendingMap().entrySet().iterator()); + } + + Entry> pollAsMapEntry(Iterator>> entryIterator) { + if (!entryIterator.hasNext()) { + return null; + } + Entry> entry = entryIterator.next(); + Collection output = createCollection(); + output.addAll(entry.getValue()); + entryIterator.remove(); + return Maps.immutableEntry(entry.getKey(), unmodifiableCollectionSubclass(output)); + } + + @Override + public NavigableMap> descendingMap() { + return new NavigableAsMap(sortedMap().descendingMap()); + } + + @Override + public NavigableSet keySet() { + return (NavigableSet) super.keySet(); + } + + @Override + NavigableSet createKeySet() { + return new NavigableKeySet(sortedMap()); + } + + @Override + public NavigableSet navigableKeySet() { + return keySet(); + } + + @Override + public NavigableSet descendingKeySet() { + return descendingMap().navigableKeySet(); + } + + @Override + public NavigableMap> subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public NavigableMap> subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return new NavigableAsMap(sortedMap().subMap(fromKey, fromInclusive, toKey, toInclusive)); + } + + @Override + public NavigableMap> headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public NavigableMap> headMap(K toKey, boolean inclusive) { + return new NavigableAsMap(sortedMap().headMap(toKey, inclusive)); + } + + @Override + public NavigableMap> tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + @Override + public NavigableMap> tailMap(K fromKey, boolean inclusive) { + return new NavigableAsMap(sortedMap().tailMap(fromKey, inclusive)); + } + } + + private static final long serialVersionUID = 2447537837011683357L; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultiset.java new file mode 100644 index 0000000000000..e68299747f333 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapBasedMultiset.java @@ -0,0 +1,333 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.InvalidObjectException; +import java.io.ObjectStreamException; +import java.io.Serializable; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.function.ObjIntConsumer; + + + +/** + * Basic implementation of {@code Multiset} backed by an instance of {@code Map}. + * + *

For serialization to work, the subclass must specify explicit {@code readObject} and {@code + * writeObject} methods. + * + * @author Kevin Bourrillion + */ +abstract class AbstractMapBasedMultiset extends AbstractMultiset implements Serializable { + // TODO(lowasser): consider overhauling this back to Map + private transient Map backingMap; + + /* + * Cache the size for efficiency. Using a long lets us avoid the need for + * overflow checking and ensures that size() will function correctly even if + * the multiset had once been larger than Integer.MAX_VALUE. + */ + private transient long size; + + /** Standard constructor. */ + protected AbstractMapBasedMultiset(Map backingMap) { + checkArgument(backingMap.isEmpty()); + this.backingMap = backingMap; + } + + /** Used during deserialization only. The backing map must be empty. */ + void setBackingMap(Map backingMap) { + this.backingMap = backingMap; + } + + // Required Implementations + + /** + * {@inheritDoc} + * + *

Invoking {@link Multiset.Entry#getCount} on an entry in the returned set always returns the + * current count of that element in the multiset, as opposed to the count at the time the entry + * was retrieved. + */ + @Override + public Set> entrySet() { + return super.entrySet(); + } + + @Override + Iterator elementIterator() { + final Iterator> backingEntries = backingMap.entrySet().iterator(); + return new Iterator() { + Map.Entry toRemove; + + @Override + public boolean hasNext() { + return backingEntries.hasNext(); + } + + @Override + public E next() { + final Map.Entry mapEntry = backingEntries.next(); + toRemove = mapEntry; + return mapEntry.getKey(); + } + + @Override + public void remove() { + checkRemove(toRemove != null); + size -= toRemove.getValue().getAndSet(0); + backingEntries.remove(); + toRemove = null; + } + }; + } + + @Override + Iterator> entryIterator() { + final Iterator> backingEntries = backingMap.entrySet().iterator(); + return new Iterator>() { + Map.Entry toRemove; + + @Override + public boolean hasNext() { + return backingEntries.hasNext(); + } + + @Override + public Multiset.Entry next() { + final Map.Entry mapEntry = backingEntries.next(); + toRemove = mapEntry; + return new Multisets.AbstractEntry() { + @Override + public E getElement() { + return mapEntry.getKey(); + } + + @Override + public int getCount() { + Count count = mapEntry.getValue(); + if (count == null || count.get() == 0) { + Count frequency = backingMap.get(getElement()); + if (frequency != null) { + return frequency.get(); + } + } + return (count == null) ? 0 : count.get(); + } + }; + } + + @Override + public void remove() { + checkRemove(toRemove != null); + size -= toRemove.getValue().getAndSet(0); + backingEntries.remove(); + toRemove = null; + } + }; + } + + @Override + public void forEachEntry(ObjIntConsumer action) { + checkNotNull(action); + backingMap.forEach((element, count) -> action.accept(element, count.get())); + } + + @Override + public void clear() { + for (Count frequency : backingMap.values()) { + frequency.set(0); + } + backingMap.clear(); + size = 0L; + } + + @Override + int distinctElements() { + return backingMap.size(); + } + + // Optimizations - Query Operations + + @Override + public int size() { + return Ints.saturatedCast(size); + } + + @Override + public Iterator iterator() { + return new MapBasedMultisetIterator(); + } + + /* + * Not subclassing AbstractMultiset$MultisetIterator because next() needs to + * retrieve the Map.Entry entry, which can then be used for + * a more efficient remove() call. + */ + private class MapBasedMultisetIterator implements Iterator { + final Iterator> entryIterator; + Map.Entry currentEntry; + int occurrencesLeft; + boolean canRemove; + + MapBasedMultisetIterator() { + this.entryIterator = backingMap.entrySet().iterator(); + } + + @Override + public boolean hasNext() { + return occurrencesLeft > 0 || entryIterator.hasNext(); + } + + @Override + public E next() { + if (occurrencesLeft == 0) { + currentEntry = entryIterator.next(); + occurrencesLeft = currentEntry.getValue().get(); + } + occurrencesLeft--; + canRemove = true; + return currentEntry.getKey(); + } + + @Override + public void remove() { + checkRemove(canRemove); + int frequency = currentEntry.getValue().get(); + if (frequency <= 0) { + throw new ConcurrentModificationException(); + } + if (currentEntry.getValue().addAndGet(-1) == 0) { + entryIterator.remove(); + } + size--; + canRemove = false; + } + } + + @Override + public int count(Object element) { + Count frequency = Maps.safeGet(backingMap, element); + return (frequency == null) ? 0 : frequency.get(); + } + + // Optional Operations - Modification Operations + + /** + * {@inheritDoc} + * + * @throws IllegalArgumentException if the call would result in more than {@link + * Integer#MAX_VALUE} occurrences of {@code element} in this multiset. + */ + @Override + public int add(E element, int occurrences) { + if (occurrences == 0) { + return count(element); + } + checkArgument(occurrences > 0, "occurrences cannot be negative: %s", occurrences); + Count frequency = backingMap.get(element); + int oldCount; + if (frequency == null) { + oldCount = 0; + backingMap.put(element, new Count(occurrences)); + } else { + oldCount = frequency.get(); + long newCount = (long) oldCount + (long) occurrences; + checkArgument(newCount <= Integer.MAX_VALUE, "too many occurrences: %s", newCount); + frequency.add(occurrences); + } + size += occurrences; + return oldCount; + } + + @Override + public int remove(Object element, int occurrences) { + if (occurrences == 0) { + return count(element); + } + checkArgument(occurrences > 0, "occurrences cannot be negative: %s", occurrences); + Count frequency = backingMap.get(element); + if (frequency == null) { + return 0; + } + + int oldCount = frequency.get(); + + int numberRemoved; + if (oldCount > occurrences) { + numberRemoved = occurrences; + } else { + numberRemoved = oldCount; + backingMap.remove(element); + } + + frequency.add(-numberRemoved); + size -= numberRemoved; + return oldCount; + } + + // Roughly a 33% performance improvement over AbstractMultiset.setCount(). + @Override + public int setCount(E element, int count) { + checkNonnegative(count, "count"); + + Count existingCounter; + int oldCount; + if (count == 0) { + existingCounter = backingMap.remove(element); + oldCount = getAndSet(existingCounter, count); + } else { + existingCounter = backingMap.get(element); + oldCount = getAndSet(existingCounter, count); + + if (existingCounter == null) { + backingMap.put(element, new Count(count)); + } + } + + size += (count - oldCount); + return oldCount; + } + + private static int getAndSet(Count i, int count) { + if (i == null) { + return 0; + } + + return i.getAndSet(count); + } + + // Don't allow default serialization. + private void readObjectNoData() throws ObjectStreamException { + throw new InvalidObjectException("Stream data required"); + } + + private static final long serialVersionUID = -2250766705698539974L; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapEntry.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapEntry.java new file mode 100644 index 0000000000000..0121bebe4eae1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMapEntry.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import java.util.Map.Entry; + + +/** + * Implementation of the {@code equals}, {@code hashCode}, and {@code toString} methods of {@code + * Entry}. + * + * @author Jared Levy + */ +abstract class AbstractMapEntry implements Entry { + + @Override + public abstract K getKey(); + + @Override + public abstract V getValue(); + + @Override + public V setValue(V value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean equals(Object object) { + if (object instanceof Entry) { + Entry that = (Entry) object; + return Objects.equal(this.getKey(), that.getKey()) + && Objects.equal(this.getValue(), that.getValue()); + } + return false; + } + + @Override + public int hashCode() { + K k = getKey(); + V v = getValue(); + return ((k == null) ? 0 : k.hashCode()) ^ ((v == null) ? 0 : v.hashCode()); + } + + /** Returns a string representation of the form {@code {key}={value}}. */ + @Override + public String toString() { + return getKey() + "=" + getValue(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultimap.java new file mode 100644 index 0000000000000..cce28431158e8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultimap.java @@ -0,0 +1,258 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; + + + +/** + * A skeleton {@code Multimap} implementation, not necessarily in terms of a {@code Map}. + * + * @author Louis Wasserman + */ +abstract class AbstractMultimap implements Multimap { + @Override + public boolean isEmpty() { + return size() == 0; + } + + @Override + public boolean containsValue(Object value) { + for (Collection collection : asMap().values()) { + if (collection.contains(value)) { + return true; + } + } + + return false; + } + + @Override + public boolean containsEntry(Object key, Object value) { + Collection collection = asMap().get(key); + return collection != null && collection.contains(value); + } + + @Override + public boolean remove(Object key, Object value) { + Collection collection = asMap().get(key); + return collection != null && collection.remove(value); + } + + @Override + public boolean put(K key, V value) { + return get(key).add(value); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public boolean putAll(K key, Iterable values) { + checkNotNull(values); + // make sure we only call values.iterator() once + // and we only call get(key) if values is nonempty + if (values instanceof Collection) { + Collection valueCollection = (Collection) values; + return !valueCollection.isEmpty() && get(key).addAll(valueCollection); + } else { + Iterator valueItr = values.iterator(); + return valueItr.hasNext() && Iterators.addAll(get(key), valueItr); + } + } + + @Override + public boolean putAll(Multimap multimap) { + boolean changed = false; + for (Entry entry : multimap.entries()) { + changed |= put(entry.getKey(), entry.getValue()); + } + return changed; + } + + @Override + public Collection replaceValues(K key, Iterable values) { + checkNotNull(values); + Collection result = removeAll(key); + putAll(key, values); + return result; + } + + private transient Collection> entries; + + @Override + public Collection> entries() { + Collection> result = entries; + return (result == null) ? entries = createEntries() : result; + } + + abstract Collection> createEntries(); + + class Entries extends Multimaps.Entries { + @Override + Multimap multimap() { + return AbstractMultimap.this; + } + + @Override + public Iterator> iterator() { + return entryIterator(); + } + + @Override + public Spliterator> spliterator() { + return entrySpliterator(); + } + } + + class EntrySet extends Entries implements Set> { + @Override + public int hashCode() { + return Sets.hashCodeImpl(this); + } + + @Override + public boolean equals(Object obj) { + return Sets.equalsImpl(this, obj); + } + } + + abstract Iterator> entryIterator(); + + Spliterator> entrySpliterator() { + return Spliterators.spliterator( + entryIterator(), size(), (this instanceof SetMultimap) ? Spliterator.DISTINCT : 0); + } + + private transient Set keySet; + + @Override + public Set keySet() { + Set result = keySet; + return (result == null) ? keySet = createKeySet() : result; + } + + abstract Set createKeySet(); + + private transient Multiset keys; + + @Override + public Multiset keys() { + Multiset result = keys; + return (result == null) ? keys = createKeys() : result; + } + + abstract Multiset createKeys(); + + private transient Collection values; + + @Override + public Collection values() { + Collection result = values; + return (result == null) ? values = createValues() : result; + } + + abstract Collection createValues(); + + class Values extends AbstractCollection { + @Override + public Iterator iterator() { + return valueIterator(); + } + + @Override + public Spliterator spliterator() { + return valueSpliterator(); + } + + @Override + public int size() { + return AbstractMultimap.this.size(); + } + + @Override + public boolean contains(Object o) { + return AbstractMultimap.this.containsValue(o); + } + + @Override + public void clear() { + AbstractMultimap.this.clear(); + } + } + + Iterator valueIterator() { + return Maps.valueIterator(entries().iterator()); + } + + Spliterator valueSpliterator() { + return Spliterators.spliterator(valueIterator(), size(), 0); + } + + private transient Map> asMap; + + @Override + public Map> asMap() { + Map> result = asMap; + return (result == null) ? asMap = createAsMap() : result; + } + + abstract Map> createAsMap(); + + // Comparison and hashing + + @Override + public boolean equals(Object object) { + return Multimaps.equalsImpl(this, object); + } + + /** + * Returns the hash code for this multimap. + * + *

The hash code of a multimap is defined as the hash code of the map view, as returned by + * {@link Multimap#asMap}. + * + * @see Map#hashCode + */ + @Override + public int hashCode() { + return asMap().hashCode(); + } + + /** + * Returns a string representation of the multimap, generated by calling {@code toString} on the + * map returned by {@link Multimap#asMap}. + * + * @return a string representation of the multimap + */ + @Override + public String toString() { + return asMap().toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultiset.java new file mode 100644 index 0000000000000..d987d5633e3a2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractMultiset.java @@ -0,0 +1,220 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multisets.setCountImpl; + +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Iterator; +import java.util.Set; + + + +/** + * This class provides a skeletal implementation of the {@link Multiset} interface. A new multiset + * implementation can be created easily by extending this class and implementing the {@link + * Multiset#entrySet()} method, plus optionally overriding {@link #add(Object, int)} and {@link + * #remove(Object, int)} to enable modifications to the multiset. + * + *

The {@link #count} and {@link #size} implementations all iterate across the set returned by + * {@link Multiset#entrySet()}, as do many methods acting on the set returned by {@link + * #elementSet()}. Override those methods for better performance. + * + * @author Kevin Bourrillion + * @author Louis Wasserman + */ +abstract class AbstractMultiset extends AbstractCollection implements Multiset { + // Query Operations + + @Override + public boolean isEmpty() { + return entrySet().isEmpty(); + } + + @Override + public boolean contains(Object element) { + return count(element) > 0; + } + + // Modification Operations + @Override + public final boolean add(E element) { + add(element, 1); + return true; + } + + @Override + public int add(E element, int occurrences) { + throw new UnsupportedOperationException(); + } + + @Override + public final boolean remove(Object element) { + return remove(element, 1) > 0; + } + + @Override + public int remove(Object element, int occurrences) { + throw new UnsupportedOperationException(); + } + + @Override + public int setCount(E element, int count) { + return setCountImpl(this, element, count); + } + + @Override + public boolean setCount(E element, int oldCount, int newCount) { + return setCountImpl(this, element, oldCount, newCount); + } + + // Bulk Operations + + /** + * {@inheritDoc} + * + *

This implementation is highly efficient when {@code elementsToAdd} is itself a {@link + * Multiset}. + */ + @Override + public final boolean addAll(Collection elementsToAdd) { + return Multisets.addAllImpl(this, elementsToAdd); + } + + @Override + public final boolean removeAll(Collection elementsToRemove) { + return Multisets.removeAllImpl(this, elementsToRemove); + } + + @Override + public final boolean retainAll(Collection elementsToRetain) { + return Multisets.retainAllImpl(this, elementsToRetain); + } + + @Override + public abstract void clear(); + + // Views + + private transient Set elementSet; + + @Override + public Set elementSet() { + Set result = elementSet; + if (result == null) { + elementSet = result = createElementSet(); + } + return result; + } + + /** + * Creates a new instance of this multiset's element set, which will be returned by {@link + * #elementSet()}. + */ + Set createElementSet() { + return new ElementSet(); + } + + class ElementSet extends Multisets.ElementSet { + @Override + Multiset multiset() { + return AbstractMultiset.this; + } + + @Override + public Iterator iterator() { + return elementIterator(); + } + } + + abstract Iterator elementIterator(); + + private transient Set> entrySet; + + @Override + public Set> entrySet() { + Set> result = entrySet; + if (result == null) { + entrySet = result = createEntrySet(); + } + return result; + } + + class EntrySet extends Multisets.EntrySet { + @Override + Multiset multiset() { + return AbstractMultiset.this; + } + + @Override + public Iterator> iterator() { + return entryIterator(); + } + + @Override + public int size() { + return distinctElements(); + } + } + + Set> createEntrySet() { + return new EntrySet(); + } + + abstract Iterator> entryIterator(); + + abstract int distinctElements(); + + // Object methods + + /** + * {@inheritDoc} + * + *

This implementation returns {@code true} if {@code object} is a multiset of the same size + * and if, for each element, the two multisets have the same count. + */ + @Override + public final boolean equals(Object object) { + return Multisets.equalsImpl(this, object); + } + + /** + * {@inheritDoc} + * + *

This implementation returns the hash code of {@link Multiset#entrySet()}. + */ + @Override + public final int hashCode() { + return entrySet().hashCode(); + } + + /** + * {@inheritDoc} + * + *

This implementation returns the result of invoking {@code toString} on {@link + * Multiset#entrySet()}. + */ + @Override + public final String toString() { + return entrySet().toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractNavigableMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractNavigableMap.java new file mode 100644 index 0000000000000..9ae34af45d2c0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractNavigableMap.java @@ -0,0 +1,180 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.IteratorBasedAbstractMap; +import java.util.Iterator; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.SortedMap; + + +/** + * Skeletal implementation of {@link NavigableMap}. + * + * @author Louis Wasserman + */ +abstract class AbstractNavigableMap extends IteratorBasedAbstractMap + implements NavigableMap { + + @Override + + public abstract V get(Object key); + + @Override + + public Entry firstEntry() { + return Iterators.getNext(entryIterator(), null); + } + + @Override + + public Entry lastEntry() { + return Iterators.getNext(descendingEntryIterator(), null); + } + + @Override + + public Entry pollFirstEntry() { + return Iterators.pollNext(entryIterator()); + } + + @Override + + public Entry pollLastEntry() { + return Iterators.pollNext(descendingEntryIterator()); + } + + @Override + public K firstKey() { + Entry entry = firstEntry(); + if (entry == null) { + throw new NoSuchElementException(); + } else { + return entry.getKey(); + } + } + + @Override + public K lastKey() { + Entry entry = lastEntry(); + if (entry == null) { + throw new NoSuchElementException(); + } else { + return entry.getKey(); + } + } + + @Override + + public Entry lowerEntry(K key) { + return headMap(key, false).lastEntry(); + } + + @Override + + public Entry floorEntry(K key) { + return headMap(key, true).lastEntry(); + } + + @Override + + public Entry ceilingEntry(K key) { + return tailMap(key, true).firstEntry(); + } + + @Override + + public Entry higherEntry(K key) { + return tailMap(key, false).firstEntry(); + } + + @Override + public K lowerKey(K key) { + return Maps.keyOrNull(lowerEntry(key)); + } + + @Override + public K floorKey(K key) { + return Maps.keyOrNull(floorEntry(key)); + } + + @Override + public K ceilingKey(K key) { + return Maps.keyOrNull(ceilingEntry(key)); + } + + @Override + public K higherKey(K key) { + return Maps.keyOrNull(higherEntry(key)); + } + + abstract Iterator> descendingEntryIterator(); + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public SortedMap headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public SortedMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + @Override + public NavigableSet navigableKeySet() { + return new Maps.NavigableKeySet<>(this); + } + + @Override + public Set keySet() { + return navigableKeySet(); + } + + @Override + public NavigableSet descendingKeySet() { + return descendingMap().navigableKeySet(); + } + + @Override + public NavigableMap descendingMap() { + return new DescendingMap(); + } + + private final class DescendingMap extends Maps.DescendingMap { + @Override + NavigableMap forward() { + return AbstractNavigableMap.this; + } + + @Override + Iterator> entryIterator() { + return descendingEntryIterator(); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractRangeSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractRangeSet.java new file mode 100644 index 0000000000000..b5d8318db621f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractRangeSet.java @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * A skeletal implementation of {@code RangeSet}. + * + * @author Louis Wasserman + */ +@SuppressWarnings("rawtypes") +abstract class AbstractRangeSet implements RangeSet { + AbstractRangeSet() {} + + @Override + public boolean contains(C value) { + return rangeContaining(value) != null; + } + + @Override + public abstract Range rangeContaining(C value); + + @Override + public boolean isEmpty() { + return asRanges().isEmpty(); + } + + @Override + public void add(Range range) { + throw new UnsupportedOperationException(); + } + + @Override + public void remove(Range range) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + remove(Range.all()); + } + + @Override + public boolean enclosesAll(RangeSet other) { + return enclosesAll(other.asRanges()); + } + + @Override + public void addAll(RangeSet other) { + addAll(other.asRanges()); + } + + @Override + public void removeAll(RangeSet other) { + removeAll(other.asRanges()); + } + + @Override + public boolean intersects(Range otherRange) { + return !subRangeSet(otherRange).isEmpty(); + } + + @Override + public abstract boolean encloses(Range otherRange); + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } else if (obj instanceof RangeSet) { + RangeSet other = (RangeSet) obj; + return this.asRanges().equals(other.asRanges()); + } + return false; + } + + @Override + public final int hashCode() { + return asRanges().hashCode(); + } + + @Override + public final String toString() { + return asRanges().toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSequentialIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSequentialIterator.java new file mode 100644 index 0000000000000..b0a18302533d7 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSequentialIterator.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.NoSuchElementException; + + +/** + * This class provides a skeletal implementation of the {@code Iterator} interface for sequences + * whose next element can always be derived from the previous element. Null elements are not + * supported, nor is the {@link #remove()} method. + * + *

Example: + * + *

{@code
+ * Iterator powersOfTwo =
+ *     new AbstractSequentialIterator(1) {
+ *       protected Integer computeNext(Integer previous) {
+ *         return (previous == 1 << 30) ? null : previous * 2;
+ *       }
+ *     };
+ * }
+ * + * @author Chris Povirk + * @since 12.0 (in Guava as {@code AbstractLinkedIterator} since 8.0) + */ +public abstract class AbstractSequentialIterator extends UnmodifiableIterator { + private T nextOrNull; + + /** + * Creates a new iterator with the given first element, or, if {@code firstOrNull} is null, + * creates a new empty iterator. + */ + protected AbstractSequentialIterator(T firstOrNull) { + this.nextOrNull = firstOrNull; + } + + /** + * Returns the element that follows {@code previous}, or returns {@code null} if no elements + * remain. This method is invoked during each call to {@link #next()} in order to compute the + * result of a future call to {@code next()}. + */ + protected abstract T computeNext(T previous); + + @Override + public final boolean hasNext() { + return nextOrNull != null; + } + + @Override + public final T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + try { + return nextOrNull; + } finally { + nextOrNull = computeNext(nextOrNull); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSetMultimap.java new file mode 100644 index 0000000000000..3a349eb386df1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSetMultimap.java @@ -0,0 +1,150 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + + +/** + * Basic implementation of the {@link SetMultimap} interface. It's a wrapper around {@link + * AbstractMapBasedMultimap} that converts the returned collections into {@code Sets}. The {@link + * #createCollection} method must return a {@code Set}. + * + * @author Jared Levy + */ +abstract class AbstractSetMultimap extends AbstractMapBasedMultimap + implements SetMultimap { + /** + * Creates a new multimap that uses the provided map. + * + * @param map place to store the mapping from each key to its corresponding values + */ + protected AbstractSetMultimap(Map> map) { + super(map); + } + + @Override + abstract Set createCollection(); + + @Override + Set createUnmodifiableEmptyCollection() { + return Collections.emptySet(); + } + + @Override + Collection unmodifiableCollectionSubclass(Collection collection) { + return Collections.unmodifiableSet((Set) collection); + } + + @Override + Collection wrapCollection(K key, Collection collection) { + return new WrappedSet(key, (Set) collection); + } + + // Following Javadoc copied from SetMultimap. + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} interface. + */ + @Override + public Set get(K key) { + return (Set) super.get(key); + } + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} interface. + */ + @Override + public Set> entries() { + return (Set>) super.entries(); + } + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} interface. + */ + @Override + public Set removeAll(Object key) { + return (Set) super.removeAll(key); + } + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} interface. + * + *

Any duplicates in {@code values} will be stored in the multimap once. + */ + @Override + public Set replaceValues(K key, Iterable values) { + return (Set) super.replaceValues(key, values); + } + + /** + * {@inheritDoc} + * + *

Though the method signature doesn't say so explicitly, the returned map has {@link Set} + * values. + */ + @Override + public Map> asMap() { + return super.asMap(); + } + + /** + * Stores a key-value pair in the multimap. + * + * @param key key to store in the multimap + * @param value value to store in the multimap + * @return {@code true} if the method increased the size of the multimap, or {@code false} if the + * multimap already contained the key-value pair + */ + @Override + public boolean put(K key, V value) { + return super.put(key, value); + } + + /** + * Compares the specified object to this multimap for equality. + * + *

Two {@code SetMultimap} instances are equal if, for each key, they contain the same values. + * Equality does not depend on the ordering of keys or values. + */ + @Override + public boolean equals(Object object) { + return super.equals(object); + } + + private static final long serialVersionUID = 7431625294878419160L; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedKeySortedSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedKeySortedSetMultimap.java new file mode 100644 index 0000000000000..8c493faeccbaa --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedKeySortedSetMultimap.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; + +/** + * Basic implementation of a {@link SortedSetMultimap} with a sorted key set. + * + *

This superclass allows {@code TreeMultimap} to override methods to return navigable set and + * map types in non-GWT only, while GWT code will inherit the SortedMap/SortedSet overrides. + * + * @author Louis Wasserman + */ +abstract class AbstractSortedKeySortedSetMultimap extends AbstractSortedSetMultimap { + + AbstractSortedKeySortedSetMultimap(SortedMap> map) { + super(map); + } + + @Override + public SortedMap> asMap() { + return (SortedMap>) super.asMap(); + } + + @Override + SortedMap> backingMap() { + return (SortedMap>) super.backingMap(); + } + + @Override + public SortedSet keySet() { + return (SortedSet) super.keySet(); + } + + @Override + Set createKeySet() { + return createMaybeNavigableKeySet(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedMultiset.java new file mode 100644 index 0000000000000..a5d0abbb981f0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedMultiset.java @@ -0,0 +1,148 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; + + + +/** + * This class provides a skeletal implementation of the {@link SortedMultiset} interface. + * + *

The {@link #count} and {@link #size} implementations all iterate across the set returned by + * {@link Multiset#entrySet()}, as do many methods acting on the set returned by {@link + * #elementSet()}. Override those methods for better performance. + * + * @author Louis Wasserman + */ +abstract class AbstractSortedMultiset extends AbstractMultiset implements SortedMultiset { + @GwtTransient + final Comparator comparator; + + // needed for serialization + @SuppressWarnings({"unchecked", "rawtypes"}) + AbstractSortedMultiset() { + this((Comparator) Ordering.natural()); + } + + AbstractSortedMultiset(Comparator comparator) { + this.comparator = checkNotNull(comparator); + } + + @Override + public NavigableSet elementSet() { + return (NavigableSet) super.elementSet(); + } + + @Override + NavigableSet createElementSet() { + return new SortedMultisets.NavigableElementSet(this); + } + + @Override + public Comparator comparator() { + return comparator; + } + + @Override + public Entry firstEntry() { + Iterator> entryIterator = entryIterator(); + return entryIterator.hasNext() ? entryIterator.next() : null; + } + + @Override + public Entry lastEntry() { + Iterator> entryIterator = descendingEntryIterator(); + return entryIterator.hasNext() ? entryIterator.next() : null; + } + + @Override + public Entry pollFirstEntry() { + Iterator> entryIterator = entryIterator(); + if (entryIterator.hasNext()) { + Entry result = entryIterator.next(); + result = Multisets.immutableEntry(result.getElement(), result.getCount()); + entryIterator.remove(); + return result; + } + return null; + } + + @Override + public Entry pollLastEntry() { + Iterator> entryIterator = descendingEntryIterator(); + if (entryIterator.hasNext()) { + Entry result = entryIterator.next(); + result = Multisets.immutableEntry(result.getElement(), result.getCount()); + entryIterator.remove(); + return result; + } + return null; + } + + @Override + public SortedMultiset subMultiset( + E fromElement, + BoundType fromBoundType, + E toElement, + BoundType toBoundType) { + // These are checked elsewhere, but NullPointerTester wants them checked eagerly. + checkNotNull(fromBoundType); + checkNotNull(toBoundType); + return tailMultiset(fromElement, fromBoundType).headMultiset(toElement, toBoundType); + } + + abstract Iterator> descendingEntryIterator(); + + Iterator descendingIterator() { + return Multisets.iteratorImpl(descendingMultiset()); + } + + private transient SortedMultiset descendingMultiset; + + @Override + public SortedMultiset descendingMultiset() { + SortedMultiset result = descendingMultiset; + return (result == null) ? descendingMultiset = createDescendingMultiset() : result; + } + + SortedMultiset createDescendingMultiset() { + class DescendingMultisetImpl extends DescendingMultiset { + @Override + SortedMultiset forwardMultiset() { + return AbstractSortedMultiset.this; + } + + @Override + Iterator> entryIterator() { + return descendingEntryIterator(); + } + + @Override + public Iterator iterator() { + return descendingIterator(); + } + } + return new DescendingMultisetImpl(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedSetMultimap.java new file mode 100644 index 0000000000000..ed88a6db63464 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AbstractSortedSetMultimap.java @@ -0,0 +1,147 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.NavigableSet; +import java.util.SortedSet; + + +/** + * Basic implementation of the {@link SortedSetMultimap} interface. It's a wrapper around {@link + * AbstractMapBasedMultimap} that converts the returned collections into sorted sets. The {@link + * #createCollection} method must return a {@code SortedSet}. + * + * @author Jared Levy + */ +abstract class AbstractSortedSetMultimap extends AbstractSetMultimap + implements SortedSetMultimap { + /** + * Creates a new multimap that uses the provided map. + * + * @param map place to store the mapping from each key to its corresponding values + */ + protected AbstractSortedSetMultimap(Map> map) { + super(map); + } + + @Override + abstract SortedSet createCollection(); + + @Override + SortedSet createUnmodifiableEmptyCollection() { + return unmodifiableCollectionSubclass(createCollection()); + } + + @Override + SortedSet unmodifiableCollectionSubclass(Collection collection) { + if (collection instanceof NavigableSet) { + return Sets.unmodifiableNavigableSet((NavigableSet) collection); + } else { + return Collections.unmodifiableSortedSet((SortedSet) collection); + } + } + + @Override + Collection wrapCollection(K key, Collection collection) { + if (collection instanceof NavigableSet) { + return new WrappedNavigableSet(key, (NavigableSet) collection, null); + } else { + return new WrappedSortedSet(key, (SortedSet) collection, null); + } + } + + // Following Javadoc copied from Multimap and SortedSetMultimap. + + /** + * Returns a collection view of all values associated with a key. If no mappings in the multimap + * have the provided key, an empty collection is returned. + * + *

Changes to the returned collection will update the underlying multimap, and vice versa. + * + *

Because a {@code SortedSetMultimap} has unique sorted values for a given key, this method + * returns a {@link SortedSet}, instead of the {@link Collection} specified in the {@link + * Multimap} interface. + */ + @Override + public SortedSet get(K key) { + return (SortedSet) super.get(key); + } + + /** + * Removes all values associated with a given key. The returned collection is immutable. + * + *

Because a {@code SortedSetMultimap} has unique sorted values for a given key, this method + * returns a {@link SortedSet}, instead of the {@link Collection} specified in the {@link + * Multimap} interface. + */ + @Override + public SortedSet removeAll(Object key) { + return (SortedSet) super.removeAll(key); + } + + /** + * Stores a collection of values with the same key, replacing any existing values for that key. + * The returned collection is immutable. + * + *

Because a {@code SortedSetMultimap} has unique sorted values for a given key, this method + * returns a {@link SortedSet}, instead of the {@link Collection} specified in the {@link + * Multimap} interface. + * + *

Any duplicates in {@code values} will be stored in the multimap once. + */ + @Override + public SortedSet replaceValues(K key, Iterable values) { + return (SortedSet) super.replaceValues(key, values); + } + + /** + * Returns a map view that associates each key with the corresponding values in the multimap. + * Changes to the returned map, such as element removal, will update the underlying multimap. The + * map does not support {@code setValue} on its entries, {@code put}, or {@code putAll}. + * + *

When passed a key that is present in the map, {@code asMap().get(Object)} has the same + * behavior as {@link #get}, returning a live collection. When passed a key that is not present, + * however, {@code asMap().get(Object)} returns {@code null} instead of an empty collection. + * + *

Though the method signature doesn't say so explicitly, the returned map has {@link + * SortedSet} values. + */ + @Override + public Map> asMap() { + return super.asMap(); + } + + /** + * {@inheritDoc} + * + *

Consequently, the values do not follow their natural ordering or the ordering of the value + * comparator. + */ + @Override + public Collection values() { + return super.values(); + } + + private static final long serialVersionUID = 430848587173315748L; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AllEqualOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AllEqualOrdering.java new file mode 100644 index 0000000000000..36eb7fb936a40 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/AllEqualOrdering.java @@ -0,0 +1,66 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.List; + + +/** + * An ordering that treats all references as equals, even nulls. + * + * @author Emily Soldal + */ +final class AllEqualOrdering extends Ordering implements Serializable { + static final AllEqualOrdering INSTANCE = new AllEqualOrdering(); + + @Override + public int compare(Object left, Object right) { + return 0; + } + + @Override + public List sortedCopy(Iterable iterable) { + return Lists.newArrayList(iterable); + } + + @Override + public ImmutableList immutableSortedCopy(Iterable iterable) { + return ImmutableList.copyOf(iterable); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public Ordering reverse() { + return (Ordering) this; + } + + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "Ordering.allEqual()"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimap.java new file mode 100644 index 0000000000000..811b3f9111af1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimap.java @@ -0,0 +1,169 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Implementation of {@code Multimap} that uses an {@code ArrayList} to store the values for a given + * key. A {@link HashMap} associates each key with an {@link ArrayList} of values. + * + *

When iterating through the collections supplied by this class, the ordering of values for a + * given key agrees with the order in which the values were added. + * + *

This multimap allows duplicate key-value pairs. After adding a new key-value pair equal to an + * existing key-value pair, the {@code ArrayListMultimap} will contain entries for both the new + * value and the old value. + * + *

Keys and values may be null. All optional multimap methods are supported, and all returned + * views are modifiable. + * + *

The lists returned by {@link #get}, {@link #removeAll}, and {@link #replaceValues} all + * implement {@link java.util.RandomAccess}. + * + *

This class is not threadsafe when any concurrent operations update the multimap. Concurrent + * read operations will work correctly. To allow concurrent update operations, wrap your multimap + * with a call to {@link Multimaps#synchronizedListMultimap}. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @since 2.0 + */ +public final class ArrayListMultimap + extends ArrayListMultimapGwtSerializationDependencies { + // Default from ArrayList + private static final int DEFAULT_VALUES_PER_KEY = 3; + + transient int expectedValuesPerKey; + + /** + * Creates a new, empty {@code ArrayListMultimap} with the default initial capacities. + * + *

This method will soon be deprecated in favor of {@code + * MultimapBuilder.hashKeys().arrayListValues().build()}. + */ + public static ArrayListMultimap create() { + return new ArrayListMultimap<>(); + } + + /** + * Constructs an empty {@code ArrayListMultimap} with enough capacity to hold the specified + * numbers of keys and values without resizing. + * + *

This method will soon be deprecated in favor of {@code + * MultimapBuilder.hashKeys(expectedKeys).arrayListValues(expectedValuesPerKey).build()}. + * + * @param expectedKeys the expected number of distinct keys + * @param expectedValuesPerKey the expected average number of values per key + * @throws IllegalArgumentException if {@code expectedKeys} or {@code expectedValuesPerKey} is + * negative + */ + public static ArrayListMultimap create(int expectedKeys, int expectedValuesPerKey) { + return new ArrayListMultimap<>(expectedKeys, expectedValuesPerKey); + } + + /** + * Constructs an {@code ArrayListMultimap} with the same mappings as the specified multimap. + * + *

This method will soon be deprecated in favor of {@code + * MultimapBuilder.hashKeys().arrayListValues().build(multimap)}. + * + * @param multimap the multimap whose contents are copied to this multimap + */ + public static ArrayListMultimap create(Multimap multimap) { + return new ArrayListMultimap<>(multimap); + } + + private ArrayListMultimap() { + this(12, DEFAULT_VALUES_PER_KEY); + } + + private ArrayListMultimap(int expectedKeys, int expectedValuesPerKey) { + super(Platform.>newHashMapWithExpectedSize(expectedKeys)); + checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + this.expectedValuesPerKey = expectedValuesPerKey; + } + + private ArrayListMultimap(Multimap multimap) { + this( + multimap.keySet().size(), + (multimap instanceof ArrayListMultimap) + ? ((ArrayListMultimap) multimap).expectedValuesPerKey + : DEFAULT_VALUES_PER_KEY); + putAll(multimap); + } + + /** + * Creates a new, empty {@code ArrayList} to hold the collection of values for an arbitrary key. + */ + @Override + List createCollection() { + return new ArrayList(expectedValuesPerKey); + } + + /** + * Reduces the memory used by this {@code ArrayListMultimap}, if feasible. + * + * @deprecated For a {@link ListMultimap} that automatically trims to size, use {@link + * ImmutableListMultimap}. If you need a mutable collection, remove the {@code trimToSize} + * call, or switch to a {@code HashMap>}. + */ + @Deprecated + public void trimToSize() { + for (Collection collection : backingMap().values()) { + ArrayList arrayList = (ArrayList) collection; + arrayList.trimToSize(); + } + } + + /** + * @serialData expectedValuesPerKey, number of distinct keys, and then for each distinct key: the + * key, number of values for that key, and the key's values + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + Serialization.writeMultimap(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + expectedValuesPerKey = DEFAULT_VALUES_PER_KEY; + int distinctKeys = Serialization.readCount(stream); + Map> map = Maps.newHashMap(); + setMap(map); + Serialization.populateMultimap(this, stream, distinctKeys); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimapGwtSerializationDependencies.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimapGwtSerializationDependencies.java new file mode 100644 index 0000000000000..541567a06235d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ArrayListMultimapGwtSerializationDependencies.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Map; + +/** + * A dummy superclass to support GWT serialization of the element types of an {@link + * ArrayListMultimap}. The GWT supersource for this class contains a field for each type. + * + *

For details about this hack, see {@link GwtSerializationDependencies}, which takes the same + * approach but with a subclass rather than a superclass. + * + *

TODO(cpovirk): Consider applying this subclass approach to our other types. + */ +abstract class ArrayListMultimapGwtSerializationDependencies + extends AbstractListMultimap { + ArrayListMultimapGwtSerializationDependencies(Map> map) { + super(map); + } + // TODO(cpovirk): Maybe I should have just one shared superclass for AbstractMultimap itself? +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BiMap.java new file mode 100644 index 0000000000000..df3adb5958427 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BiMap.java @@ -0,0 +1,107 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Map; +import java.util.Set; + + +/** + * A bimap (or "bidirectional map") is a map that preserves the uniqueness of its values as well as + * that of its keys. This constraint enables bimaps to support an "inverse view", which is another + * bimap containing the same entries as this bimap but with reversed keys and values. + * + *

See the Guava User Guide article on {@code BiMap}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public interface BiMap extends Map { + // Modification Operations + + /** + * {@inheritDoc} + * + * @throws IllegalArgumentException if the given value is already bound to a different key in this + * bimap. The bimap will remain unmodified in this event. To avoid this exception, call {@link + * #forcePut} instead. + */ + @Override + + V put(K key, V value); + + /** + * An alternate form of {@code put} that silently removes any existing entry with the value {@code + * value} before proceeding with the {@link #put} operation. If the bimap previously contained the + * provided key-value mapping, this method has no effect. + * + *

Note that a successful call to this method could cause the size of the bimap to increase by + * one, stay the same, or even decrease by one. + * + *

Warning: If an existing entry with this value is removed, the key for that entry is + * discarded and not returned. + * + * @param key the key with which the specified value is to be associated + * @param value the value to be associated with the specified key + * @return the value which was previously associated with the key, which may be {@code null}, or + * {@code null} if there was no previous entry + */ + + V forcePut(K key, V value); + + // Bulk Operations + + /** + * {@inheritDoc} + * + *

Warning: the results of calling this method may vary depending on the iteration order + * of {@code map}. + * + * @throws IllegalArgumentException if an attempt to {@code put} any entry fails. Note that some + * map entries may have been added to the bimap before the exception was thrown. + */ + @Override + void putAll(Map map); + + // Views + + /** + * {@inheritDoc} + * + *

Because a bimap has unique values, this method returns a {@link Set}, instead of the {@link + * java.util.Collection} specified in the {@link Map} interface. + */ + @Override + Set values(); + + /** + * Returns the inverse view of this bimap, which maps each of this bimap's values to its + * associated key. The two bimaps are backed by the same data; any changes to one will appear in + * the other. + * + *

Note:There is no guaranteed correspondence between the iteration order of a bimap and + * that of its inverse. + * + * @return the inverse view of this bimap + */ + BiMap inverse(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BoundType.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BoundType.java new file mode 100644 index 0000000000000..a121b6c99b3a1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/BoundType.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +/** + * Indicates whether an endpoint of some range is contained in the range itself ("closed") or not + * ("open"). If a range is unbounded on a side, it is neither open nor closed on that side; the + * bound simply does not exist. + * + * @since 10.0 + */ +public enum BoundType { + /** The endpoint value is not considered part of the set ("exclusive"). */ + OPEN(false), + CLOSED(true); + + final boolean inclusive; + + BoundType(boolean inclusive) { + this.inclusive = inclusive; + } + + /** Returns the bound type corresponding to a boolean value for inclusivity. */ + static BoundType forBoolean(boolean inclusive) { + return inclusive ? CLOSED : OPEN; + } + + BoundType flip() { + return forBoolean(!inclusive); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ByFunctionOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ByFunctionOrdering.java new file mode 100644 index 0000000000000..8d30e88f02d33 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ByFunctionOrdering.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.io.Serializable; + + +/** + * An ordering that orders elements by applying an order to the result of a function on those + * elements. + */ +final class ByFunctionOrdering extends Ordering implements Serializable { + final Function function; + final Ordering ordering; + + ByFunctionOrdering(Function function, Ordering ordering) { + this.function = checkNotNull(function); + this.ordering = checkNotNull(ordering); + } + + @Override + public int compare(F left, F right) { + return ordering.compare(function.apply(left), function.apply(right)); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof ByFunctionOrdering) { + ByFunctionOrdering that = (ByFunctionOrdering) object; + return this.function.equals(that.function) && this.ordering.equals(that.ordering); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(function, ordering); + } + + @Override + public String toString() { + return ordering + ".onResultOf(" + function + ")"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CartesianList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CartesianList.java new file mode 100644 index 0000000000000..ffc1b28a233d8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CartesianList.java @@ -0,0 +1,118 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; + +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import java.util.AbstractList; +import java.util.List; +import java.util.ListIterator; +import java.util.RandomAccess; + + +/** + * Implementation of {@link Lists#cartesianProduct(List)}. + * + * @author Louis Wasserman + */ +final class CartesianList extends AbstractList> implements RandomAccess { + + private final transient ImmutableList> axes; + private final transient int[] axesSizeProduct; + + static List> create(List> lists) { + ImmutableList.Builder> axesBuilder = new ImmutableList.Builder<>(lists.size()); + for (List list : lists) { + List copy = ImmutableList.copyOf(list); + if (copy.isEmpty()) { + return ImmutableList.of(); + } + axesBuilder.add(copy); + } + return new CartesianList(axesBuilder.build()); + } + + CartesianList(ImmutableList> axes) { + this.axes = axes; + int[] axesSizeProduct = new int[axes.size() + 1]; + axesSizeProduct[axes.size()] = 1; + try { + for (int i = axes.size() - 1; i >= 0; i--) { + axesSizeProduct[i] = IntMath.checkedMultiply(axesSizeProduct[i + 1], axes.get(i).size()); + } + } catch (ArithmeticException e) { + throw new IllegalArgumentException( + "Cartesian product too large; must have size at most Integer.MAX_VALUE"); + } + this.axesSizeProduct = axesSizeProduct; + } + + private int getAxisIndexForProductIndex(int index, int axis) { + return (index / axesSizeProduct[axis + 1]) % axes.get(axis).size(); + } + + @Override + public ImmutableList get(final int index) { + checkElementIndex(index, size()); + return new ImmutableList() { + + @Override + public int size() { + return axes.size(); + } + + @Override + public E get(int axis) { + checkElementIndex(axis, size()); + int axisIndex = getAxisIndexForProductIndex(index, axis); + return axes.get(axis).get(axisIndex); + } + + @Override + boolean isPartialView() { + return true; + } + }; + } + + @Override + public int size() { + return axesSizeProduct[0]; + } + + @Override + public boolean contains(Object o) { + if (!(o instanceof List)) { + return false; + } + List list = (List) o; + if (list.size() != axes.size()) { + return false; + } + ListIterator itr = list.listIterator(); + while (itr.hasNext()) { + int index = itr.nextIndex(); + if (!axes.get(index).contains(itr.next())) { + return false; + } + } + return true; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ClassToInstanceMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ClassToInstanceMap.java new file mode 100644 index 0000000000000..aeada7ed9da73 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ClassToInstanceMap.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Map; + + +/** + * A map, each entry of which maps a Java raw type to an + * instance of that type. In addition to implementing {@code Map}, the additional type-safe + * operations {@link #putInstance} and {@link #getInstance} are available. + * + *

Like any other {@code Map}, this map may contain entries for primitive types, + * and a primitive type and its corresponding wrapper type may map to different values. + * + *

See the Guava User Guide article on {@code + * ClassToInstanceMap}. + * + *

To map a generic type to an instance of that type, use {@link + * com.azure.cosmos.reflect.TypeToInstanceMap} instead. + * + * @param the common supertype that all entries must share; often this is simply {@link Object} + * @author Kevin Bourrillion + * @since 2.0 + */ +public interface ClassToInstanceMap extends Map, B> { + /** + * Returns the value the specified class is mapped to, or {@code null} if no entry for this class + * is present. This will only return a value that was bound to this specific class, not a value + * that may have been bound to a subtype. + */ + T getInstance(Class type); + + /** + * Maps the specified class to the specified value. Does not associate this value with any + * of the class's supertypes. + * + * @return the value previously associated with this class (possibly {@code null}), or {@code + * null} if there was no previous entry. + */ + T putInstance(Class type, T value); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectCollectors.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectCollectors.java new file mode 100644 index 0000000000000..f13eef4f79236 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectCollectors.java @@ -0,0 +1,136 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Comparator; +import java.util.function.Function; +import java.util.stream.Collector; + +/** Collectors utilities for {@code common.collect} internals. */ +final class CollectCollectors { + static Collector> toImmutableBiMap( + Function keyFunction, + Function valueFunction) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + return Collector.of( + ImmutableBiMap.Builder::new, + (builder, input) -> builder.put(keyFunction.apply(input), valueFunction.apply(input)), + ImmutableBiMap.Builder::combine, + ImmutableBiMap.Builder::build, + new Collector.Characteristics[0]); + } + + private static final Collector> TO_IMMUTABLE_LIST = + Collector.of( + ImmutableList::builder, + ImmutableList.Builder::add, + ImmutableList.Builder::combine, + ImmutableList.Builder::build); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Collector> toImmutableList() { + return (Collector) TO_IMMUTABLE_LIST; + } + + static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + return Collector.of( + ImmutableMap.Builder::new, + (builder, input) -> builder.put(keyFunction.apply(input), valueFunction.apply(input)), + ImmutableMap.Builder::combine, + ImmutableMap.Builder::build); + } + + private static final Collector> TO_IMMUTABLE_SET = + Collector.of( + ImmutableSet::builder, + ImmutableSet.Builder::add, + ImmutableSet.Builder::combine, + ImmutableSet.Builder::build); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Collector> toImmutableSet() { + return (Collector) TO_IMMUTABLE_SET; + } + + static Collector> toImmutableSortedMap( + Comparator comparator, + Function keyFunction, + Function valueFunction) { + checkNotNull(comparator); + checkNotNull(keyFunction); + checkNotNull(valueFunction); + /* + * We will always fail if there are duplicate keys, and the keys are always sorted by + * the Comparator, so the entries can come in in arbitrary order -- so we report UNORDERED. + */ + return Collector.of( + () -> new ImmutableSortedMap.Builder(comparator), + (builder, input) -> builder.put(keyFunction.apply(input), valueFunction.apply(input)), + ImmutableSortedMap.Builder::combine, + ImmutableSortedMap.Builder::build, + Collector.Characteristics.UNORDERED); + } + + static Collector> toImmutableSortedSet( + Comparator comparator) { + checkNotNull(comparator); + return Collector.of( + () -> new ImmutableSortedSet.Builder(comparator), + ImmutableSortedSet.Builder::add, + ImmutableSortedSet.Builder::combine, + ImmutableSortedSet.Builder::build); + } + + @SuppressWarnings("rawtypes") + private static final Collector, ?, ImmutableRangeSet> + TO_IMMUTABLE_RANGE_SET = + Collector.of( + ImmutableRangeSet::builder, + ImmutableRangeSet.Builder::add, + ImmutableRangeSet.Builder::combine, + ImmutableRangeSet.Builder::build); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static > + Collector, ?, ImmutableRangeSet> toImmutableRangeSet() { + return (Collector) TO_IMMUTABLE_RANGE_SET; + } + + static , V> + Collector> toImmutableRangeMap( + Function> keyFunction, + Function valueFunction) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + return Collector.of( + ImmutableRangeMap::builder, + (builder, input) -> builder.put(keyFunction.apply(input), valueFunction.apply(input)), + ImmutableRangeMap.Builder::combine, + ImmutableRangeMap.Builder::build); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectPreconditions.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectPreconditions.java new file mode 100644 index 0000000000000..cbd2325f26f91 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectPreconditions.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; + + +/** Precondition checks useful in collection implementations. */ +final class CollectPreconditions { + + static void checkEntryNotNull(Object key, Object value) { + if (key == null) { + throw new NullPointerException("null key in entry: null=" + value); + } else if (value == null) { + throw new NullPointerException("null value in entry: " + key + "=null"); + } + } + + static int checkNonnegative(int value, String name) { + if (value < 0) { + throw new IllegalArgumentException(name + " cannot be negative but was: " + value); + } + return value; + } + + static long checkNonnegative(long value, String name) { + if (value < 0) { + throw new IllegalArgumentException(name + " cannot be negative but was: " + value); + } + return value; + } + + static void checkPositive(int value, String name) { + if (value <= 0) { + throw new IllegalArgumentException(name + " must be positive but was: " + value); + } + } + + /** + * Precondition tester for {@code Iterator.remove()} that throws an exception with a consistent + * error message. + */ + static void checkRemove(boolean canRemove) { + checkState(canRemove, "no calls to next() since the last call to remove()"); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectSpliterators.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectSpliterators.java new file mode 100644 index 0000000000000..4b50f2b1d0d43 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CollectSpliterators.java @@ -0,0 +1,293 @@ +/* + * Copyright (C) 2015 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Comparator; +import java.util.Spliterator; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.IntConsumer; +import java.util.function.IntFunction; +import java.util.function.Predicate; +import java.util.stream.IntStream; + + +/** Spliterator utilities for {@code common.collect} internals. */ +final class CollectSpliterators { + private CollectSpliterators() {} + + static Spliterator indexed(int size, int extraCharacteristics, IntFunction function) { + return indexed(size, extraCharacteristics, function, null); + } + + static Spliterator indexed( + int size, + int extraCharacteristics, + IntFunction function, + Comparator comparator) { + if (comparator != null) { + checkArgument((extraCharacteristics & (Spliterator.SORTED)) != 0); + } + class WithCharacteristics implements Spliterator { + private final OfInt delegate; + + WithCharacteristics(OfInt delegate) { + this.delegate = delegate; + } + + @Override + public boolean tryAdvance(Consumer action) { + return delegate.tryAdvance((IntConsumer) i -> action.accept(function.apply(i))); + } + + @Override + public void forEachRemaining(Consumer action) { + delegate.forEachRemaining((IntConsumer) i -> action.accept(function.apply(i))); + } + + @Override + + public Spliterator trySplit() { + OfInt split = delegate.trySplit(); + return (split == null) ? null : new WithCharacteristics(split); + } + + @Override + public long estimateSize() { + return delegate.estimateSize(); + } + + @Override + public int characteristics() { + return Spliterator.ORDERED + | Spliterator.SIZED + | Spliterator.SUBSIZED + | extraCharacteristics; + } + + @Override + public Comparator getComparator() { + if (hasCharacteristics(Spliterator.SORTED)) { + return comparator; + } else { + throw new IllegalStateException(); + } + } + } + return new WithCharacteristics(IntStream.range(0, size).spliterator()); + } + + /** + * Returns a {@code Spliterator} over the elements of {@code fromSpliterator} mapped by {@code + * function}. + */ + static Spliterator map( + Spliterator fromSpliterator, Function function) { + checkNotNull(fromSpliterator); + checkNotNull(function); + return new Spliterator() { + + @Override + public boolean tryAdvance(Consumer action) { + return fromSpliterator.tryAdvance( + fromElement -> action.accept(function.apply(fromElement))); + } + + @Override + public void forEachRemaining(Consumer action) { + fromSpliterator.forEachRemaining(fromElement -> action.accept(function.apply(fromElement))); + } + + @Override + public Spliterator trySplit() { + Spliterator fromSplit = fromSpliterator.trySplit(); + return (fromSplit != null) ? map(fromSplit, function) : null; + } + + @Override + public long estimateSize() { + return fromSpliterator.estimateSize(); + } + + @Override + public int characteristics() { + return fromSpliterator.characteristics() + & ~(Spliterator.DISTINCT | Spliterator.NONNULL | Spliterator.SORTED); + } + }; + } + + /** Returns a {@code Spliterator} filtered by the specified predicate. */ + static Spliterator filter(Spliterator fromSpliterator, Predicate predicate) { + checkNotNull(fromSpliterator); + checkNotNull(predicate); + class Splitr implements Spliterator, Consumer { + T holder = null; + + @Override + public void accept(T t) { + this.holder = t; + } + + @Override + public boolean tryAdvance(Consumer action) { + while (fromSpliterator.tryAdvance(this)) { + try { + if (predicate.test(holder)) { + action.accept(holder); + return true; + } + } finally { + holder = null; + } + } + return false; + } + + @Override + public Spliterator trySplit() { + Spliterator fromSplit = fromSpliterator.trySplit(); + return (fromSplit == null) ? null : filter(fromSplit, predicate); + } + + @Override + public long estimateSize() { + return fromSpliterator.estimateSize() / 2; + } + + @Override + public Comparator getComparator() { + return fromSpliterator.getComparator(); + } + + @Override + public int characteristics() { + return fromSpliterator.characteristics() + & (Spliterator.DISTINCT + | Spliterator.NONNULL + | Spliterator.ORDERED + | Spliterator.SORTED); + } + } + return new Splitr(); + } + + /** + * Returns a {@code Spliterator} that iterates over the elements of the spliterators generated by + * applying {@code function} to the elements of {@code fromSpliterator}. + */ + static Spliterator flatMap( + Spliterator fromSpliterator, + Function> function, + int topCharacteristics, + long topSize) { + checkArgument( + (topCharacteristics & Spliterator.SUBSIZED) == 0, + "flatMap does not support SUBSIZED characteristic"); + checkArgument( + (topCharacteristics & Spliterator.SORTED) == 0, + "flatMap does not support SORTED characteristic"); + checkNotNull(fromSpliterator); + checkNotNull(function); + class FlatMapSpliterator implements Spliterator { + Spliterator prefix; + final Spliterator from; + int characteristics; + long estimatedSize; + + FlatMapSpliterator( + Spliterator prefix, Spliterator from, int characteristics, long estimatedSize) { + this.prefix = prefix; + this.from = from; + this.characteristics = characteristics; + this.estimatedSize = estimatedSize; + } + + @Override + public boolean tryAdvance(Consumer action) { + while (true) { + if (prefix != null && prefix.tryAdvance(action)) { + if (estimatedSize != Long.MAX_VALUE) { + estimatedSize--; + } + return true; + } else { + prefix = null; + } + if (!from.tryAdvance(fromElement -> prefix = function.apply(fromElement))) { + return false; + } + } + } + + @Override + public void forEachRemaining(Consumer action) { + if (prefix != null) { + prefix.forEachRemaining(action); + prefix = null; + } + from.forEachRemaining(fromElement -> function.apply(fromElement).forEachRemaining(action)); + estimatedSize = 0; + } + + @Override + public Spliterator trySplit() { + Spliterator fromSplit = from.trySplit(); + if (fromSplit != null) { + int splitCharacteristics = characteristics & ~Spliterator.SIZED; + long estSplitSize = estimateSize(); + if (estSplitSize < Long.MAX_VALUE) { + estSplitSize /= 2; + this.estimatedSize -= estSplitSize; + this.characteristics = splitCharacteristics; + } + Spliterator result = + new FlatMapSpliterator(this.prefix, fromSplit, splitCharacteristics, estSplitSize); + this.prefix = null; + return result; + } else if (prefix != null) { + Spliterator result = prefix; + this.prefix = null; + return result; + } else { + return null; + } + } + + @Override + public long estimateSize() { + if (prefix != null) { + estimatedSize = Math.max(estimatedSize, prefix.estimateSize()); + } + return Math.max(estimatedSize, 0); + } + + @Override + public int characteristics() { + return characteristics; + } + } + return new FlatMapSpliterator(null, fromSpliterator, topCharacteristics, topSize); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Collections2.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Collections2.java new file mode 100644 index 0000000000000..422ac796fafeb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Collections2.java @@ -0,0 +1,697 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.util.AbstractCollection; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Spliterator; +import java.util.function.Consumer; + + +/** + * Provides static methods for working with {@code Collection} instances. + * + *

Java 8 users: several common uses for this class are now more comprehensively addressed + * by the new {@link java.util.stream.Stream} library. Read the method documentation below for + * comparisons. These methods are not being deprecated, but we gently encourage you to migrate to + * streams. + * + * @author Chris Povirk + * @author Mike Bostock + * @author Jared Levy + * @since 2.0 + */ +public final class Collections2 { + private Collections2() {} + + /** + * Returns the elements of {@code unfiltered} that satisfy a predicate. The returned collection is + * a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting collection's iterator does not support {@code remove()}, but all other + * collection methods are supported. When given an element that doesn't satisfy the predicate, the + * collection's {@code add()} and {@code addAll()} methods throw an {@link + * IllegalArgumentException}. When methods such as {@code removeAll()} and {@code clear()} are + * called on the filtered collection, only elements that satisfy the filter will be removed from + * the underlying collection. + * + *

The returned collection isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered collection's methods, such as {@code size()}, iterate across every + * element in the underlying collection and determine which elements satisfy the filter. When a + * live view is not needed, it may be faster to copy {@code Iterables.filter(unfiltered, + * predicate)} and use the copy. + * + *

Warning: {@code predicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. (See {@link + * Iterables#filter(Iterable, Class)} for related functionality.) + * + *

{@code Stream} equivalent: {@link java.util.stream.Stream#filter Stream.filter}. + */ + // TODO(kevinb): how can we omit that Iterables link when building gwt + // javadoc? + public static Collection filter(Collection unfiltered, Predicate predicate) { + if (unfiltered instanceof FilteredCollection) { + // Support clear(), removeAll(), and retainAll() when filtering a filtered + // collection. + return ((FilteredCollection) unfiltered).createCombined(predicate); + } + + return new FilteredCollection(checkNotNull(unfiltered), checkNotNull(predicate)); + } + + /** + * Delegates to {@link Collection#contains}. Returns {@code false} if the {@code contains} method + * throws a {@code ClassCastException} or {@code NullPointerException}. + */ + static boolean safeContains(Collection collection, Object object) { + checkNotNull(collection); + try { + return collection.contains(object); + } catch (ClassCastException | NullPointerException e) { + return false; + } + } + + /** + * Delegates to {@link Collection#remove}. Returns {@code false} if the {@code remove} method + * throws a {@code ClassCastException} or {@code NullPointerException}. + */ + static boolean safeRemove(Collection collection, Object object) { + checkNotNull(collection); + try { + return collection.remove(object); + } catch (ClassCastException | NullPointerException e) { + return false; + } + } + + static class FilteredCollection extends AbstractCollection { + final Collection unfiltered; + final Predicate predicate; + + FilteredCollection(Collection unfiltered, Predicate predicate) { + this.unfiltered = unfiltered; + this.predicate = predicate; + } + + FilteredCollection createCombined(Predicate newPredicate) { + return new FilteredCollection(unfiltered, Predicates.and(predicate, newPredicate)); + // . above needed to compile in JDK 5 + } + + @Override + public boolean add(E element) { + checkArgument(predicate.apply(element)); + return unfiltered.add(element); + } + + @Override + public boolean addAll(Collection collection) { + for (E element : collection) { + checkArgument(predicate.apply(element)); + } + return unfiltered.addAll(collection); + } + + @Override + public void clear() { + Iterables.removeIf(unfiltered, predicate); + } + + @Override + public boolean contains(Object element) { + if (safeContains(unfiltered, element)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // element is in unfiltered, so it must be an E + E e = (E) element; + return predicate.apply(e); + } + return false; + } + + @Override + public boolean containsAll(Collection collection) { + return containsAllImpl(this, collection); + } + + @Override + public boolean isEmpty() { + return !Iterables.any(unfiltered, predicate); + } + + @Override + public Iterator iterator() { + return Iterators.filter(unfiltered.iterator(), predicate); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.filter(unfiltered.spliterator(), predicate); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + unfiltered.forEach( + (E e) -> { + if (predicate.test(e)) { + action.accept(e); + } + }); + } + + @Override + public boolean remove(Object element) { + return contains(element) && unfiltered.remove(element); + } + + @Override + public boolean removeAll(final Collection collection) { + return removeIf(collection::contains); + } + + @Override + public boolean retainAll(final Collection collection) { + return removeIf(element -> !collection.contains(element)); + } + + @Override + public boolean removeIf(java.util.function.Predicate filter) { + checkNotNull(filter); + return unfiltered.removeIf(element -> predicate.apply(element) && filter.test(element)); + } + + @Override + public int size() { + int size = 0; + for (E e : unfiltered) { + if (predicate.apply(e)) { + size++; + } + } + return size; + } + + @Override + public Object[] toArray() { + // creating an ArrayList so filtering happens once + return Lists.newArrayList(iterator()).toArray(); + } + + @Override + public T[] toArray(T[] array) { + return Lists.newArrayList(iterator()).toArray(array); + } + } + + /** + * Returns a collection that applies {@code function} to each element of {@code fromCollection}. + * The returned collection is a live view of {@code fromCollection}; changes to one affect the + * other. + * + *

The returned collection's {@code add()} and {@code addAll()} methods throw an {@link + * UnsupportedOperationException}. All other collection methods are supported, as long as {@code + * fromCollection} supports them. + * + *

The returned collection isn't threadsafe or serializable, even if {@code fromCollection} is. + * + *

When a live view is not needed, it may be faster to copy the transformed collection + * and use the copy. + * + *

If the input {@code Collection} is known to be a {@code List}, consider {@link + * Lists#transform}. If only an {@code Iterable} is available, use {@link Iterables#transform}. + * + *

{@code Stream} equivalent: {@link java.util.stream.Stream#map Stream.map}. + */ + public static Collection transform( + Collection fromCollection, Function function) { + return new TransformedCollection<>(fromCollection, function); + } + + static class TransformedCollection extends AbstractCollection { + final Collection fromCollection; + final Function function; + + TransformedCollection(Collection fromCollection, Function function) { + this.fromCollection = checkNotNull(fromCollection); + this.function = checkNotNull(function); + } + + @Override + public void clear() { + fromCollection.clear(); + } + + @Override + public boolean isEmpty() { + return fromCollection.isEmpty(); + } + + @Override + public Iterator iterator() { + return Iterators.transform(fromCollection.iterator(), function); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.map(fromCollection.spliterator(), function); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + fromCollection.forEach((F f) -> action.accept(function.apply(f))); + } + + @Override + public boolean removeIf(java.util.function.Predicate filter) { + checkNotNull(filter); + return fromCollection.removeIf(element -> filter.test(function.apply(element))); + } + + @Override + public int size() { + return fromCollection.size(); + } + } + + /** + * Returns {@code true} if the collection {@code self} contains all of the elements in the + * collection {@code c}. + * + *

This method iterates over the specified collection {@code c}, checking each element returned + * by the iterator in turn to see if it is contained in the specified collection {@code self}. If + * all elements are so contained, {@code true} is returned, otherwise {@code false}. + * + * @param self a collection which might contain all elements in {@code c} + * @param c a collection whose elements might be contained by {@code self} + */ + static boolean containsAllImpl(Collection self, Collection c) { + for (Object o : c) { + if (!self.contains(o)) { + return false; + } + } + return true; + } + + /** An implementation of {@link Collection#toString()}. */ + static String toStringImpl(final Collection collection) { + StringBuilder sb = newStringBuilderForCollection(collection.size()).append('['); + boolean first = true; + for (Object o : collection) { + if (!first) { + sb.append(", "); + } + first = false; + if (o == collection) { + sb.append("(this Collection)"); + } else { + sb.append(o); + } + } + return sb.append(']').toString(); + } + + /** Returns best-effort-sized StringBuilder based on the given collection size. */ + static StringBuilder newStringBuilderForCollection(int size) { + checkNonnegative(size, "size"); + return new StringBuilder((int) Math.min(size * 8L, Ints.MAX_POWER_OF_TWO)); + } + + /** Used to avoid http://bugs.sun.com/view_bug.do?bug_id=6558557 */ + static Collection cast(Iterable iterable) { + return (Collection) iterable; + } + + /** + * Returns a {@link Collection} of all the permutations of the specified {@link Iterable}. + * + *

Notes: This is an implementation of the algorithm for Lexicographical Permutations + * Generation, described in Knuth's "The Art of Computer Programming", Volume 4, Chapter 7, + * Section 7.2.1.2. The iteration order follows the lexicographical order. This means that the + * first permutation will be in ascending order, and the last will be in descending order. + * + *

Duplicate elements are considered equal. For example, the list [1, 1] will have only one + * permutation, instead of two. This is why the elements have to implement {@link Comparable}. + * + *

An empty iterable has only one permutation, which is an empty list. + * + *

This method is equivalent to {@code Collections2.orderedPermutations(list, + * Ordering.natural())}. + * + * @param elements the original iterable whose elements have to be permuted. + * @return an immutable {@link Collection} containing all the different permutations of the + * original iterable. + * @throws NullPointerException if the specified iterable is null or has any null elements. + * @since 12.0 + */ + + public static > Collection> orderedPermutations( + Iterable elements) { + return orderedPermutations(elements, Ordering.natural()); + } + + /** + * Returns a {@link Collection} of all the permutations of the specified {@link Iterable} using + * the specified {@link Comparator} for establishing the lexicographical ordering. + * + *

Examples: + * + *

{@code
+   * for (List perm : orderedPermutations(asList("b", "c", "a"))) {
+   *   println(perm);
+   * }
+   * // -> ["a", "b", "c"]
+   * // -> ["a", "c", "b"]
+   * // -> ["b", "a", "c"]
+   * // -> ["b", "c", "a"]
+   * // -> ["c", "a", "b"]
+   * // -> ["c", "b", "a"]
+   *
+   * for (List perm : orderedPermutations(asList(1, 2, 2, 1))) {
+   *   println(perm);
+   * }
+   * // -> [1, 1, 2, 2]
+   * // -> [1, 2, 1, 2]
+   * // -> [1, 2, 2, 1]
+   * // -> [2, 1, 1, 2]
+   * // -> [2, 1, 2, 1]
+   * // -> [2, 2, 1, 1]
+   * }
+ * + *

Notes: This is an implementation of the algorithm for Lexicographical Permutations + * Generation, described in Knuth's "The Art of Computer Programming", Volume 4, Chapter 7, + * Section 7.2.1.2. The iteration order follows the lexicographical order. This means that the + * first permutation will be in ascending order, and the last will be in descending order. + * + *

Elements that compare equal are considered equal and no new permutations are created by + * swapping them. + * + *

An empty iterable has only one permutation, which is an empty list. + * + * @param elements the original iterable whose elements have to be permuted. + * @param comparator a comparator for the iterable's elements. + * @return an immutable {@link Collection} containing all the different permutations of the + * original iterable. + * @throws NullPointerException If the specified iterable is null, has any null elements, or if + * the specified comparator is null. + * @since 12.0 + */ + + public static Collection> orderedPermutations( + Iterable elements, Comparator comparator) { + return new OrderedPermutationCollection(elements, comparator); + } + + private static final class OrderedPermutationCollection extends AbstractCollection> { + final ImmutableList inputList; + final Comparator comparator; + final int size; + + OrderedPermutationCollection(Iterable input, Comparator comparator) { + this.inputList = ImmutableList.sortedCopyOf(comparator, input); + this.comparator = comparator; + this.size = calculateSize(inputList, comparator); + } + + /** + * The number of permutations with repeated elements is calculated as follows: + * + *

    + *
  • For an empty list, it is 1 (base case). + *
  • When r numbers are added to a list of n-r elements, the number of permutations is + * increased by a factor of (n choose r). + *
+ */ + private static int calculateSize( + List sortedInputList, Comparator comparator) { + int permutations = 1; + int n = 1; + int r = 1; + while (n < sortedInputList.size()) { + int comparison = comparator.compare(sortedInputList.get(n - 1), sortedInputList.get(n)); + if (comparison < 0) { + // We move to the next non-repeated element. + permutations = IntMath.saturatedMultiply(permutations, IntMath.binomial(n, r)); + r = 0; + if (permutations == Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } + } + n++; + r++; + } + return IntMath.saturatedMultiply(permutations, IntMath.binomial(n, r)); + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Iterator> iterator() { + return new OrderedPermutationIterator(inputList, comparator); + } + + @Override + public boolean contains(Object obj) { + if (obj instanceof List) { + List list = (List) obj; + return isPermutation(inputList, list); + } + return false; + } + + @Override + public String toString() { + return "orderedPermutationCollection(" + inputList + ")"; + } + } + + private static final class OrderedPermutationIterator extends AbstractIterator> { + + List nextPermutation; + final Comparator comparator; + + OrderedPermutationIterator(List list, Comparator comparator) { + this.nextPermutation = Lists.newArrayList(list); + this.comparator = comparator; + } + + @Override + protected List computeNext() { + if (nextPermutation == null) { + return endOfData(); + } + ImmutableList next = ImmutableList.copyOf(nextPermutation); + calculateNextPermutation(); + return next; + } + + void calculateNextPermutation() { + int j = findNextJ(); + if (j == -1) { + nextPermutation = null; + return; + } + + int l = findNextL(j); + Collections.swap(nextPermutation, j, l); + int n = nextPermutation.size(); + Collections.reverse(nextPermutation.subList(j + 1, n)); + } + + int findNextJ() { + for (int k = nextPermutation.size() - 2; k >= 0; k--) { + if (comparator.compare(nextPermutation.get(k), nextPermutation.get(k + 1)) < 0) { + return k; + } + } + return -1; + } + + int findNextL(int j) { + E ak = nextPermutation.get(j); + for (int l = nextPermutation.size() - 1; l > j; l--) { + if (comparator.compare(ak, nextPermutation.get(l)) < 0) { + return l; + } + } + throw new AssertionError("this statement should be unreachable"); + } + } + + /** + * Returns a {@link Collection} of all the permutations of the specified {@link Collection}. + * + *

Notes: This is an implementation of the Plain Changes algorithm for permutations + * generation, described in Knuth's "The Art of Computer Programming", Volume 4, Chapter 7, + * Section 7.2.1.2. + * + *

If the input list contains equal elements, some of the generated permutations will be equal. + * + *

An empty collection has only one permutation, which is an empty list. + * + * @param elements the original collection whose elements have to be permuted. + * @return an immutable {@link Collection} containing all the different permutations of the + * original collection. + * @throws NullPointerException if the specified collection is null or has any null elements. + * @since 12.0 + */ + + public static Collection> permutations(Collection elements) { + return new PermutationCollection(ImmutableList.copyOf(elements)); + } + + private static final class PermutationCollection extends AbstractCollection> { + final ImmutableList inputList; + + PermutationCollection(ImmutableList input) { + this.inputList = input; + } + + @Override + public int size() { + return IntMath.factorial(inputList.size()); + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Iterator> iterator() { + return new PermutationIterator(inputList); + } + + @Override + public boolean contains(Object obj) { + if (obj instanceof List) { + List list = (List) obj; + return isPermutation(inputList, list); + } + return false; + } + + @Override + public String toString() { + return "permutations(" + inputList + ")"; + } + } + + private static class PermutationIterator extends AbstractIterator> { + final List list; + final int[] c; + final int[] o; + int j; + + PermutationIterator(List list) { + this.list = new ArrayList(list); + int n = list.size(); + c = new int[n]; + o = new int[n]; + Arrays.fill(c, 0); + Arrays.fill(o, 1); + j = Integer.MAX_VALUE; + } + + @Override + protected List computeNext() { + if (j <= 0) { + return endOfData(); + } + ImmutableList next = ImmutableList.copyOf(list); + calculateNextPermutation(); + return next; + } + + void calculateNextPermutation() { + j = list.size() - 1; + int s = 0; + + // Handle the special case of an empty list. Skip the calculation of the + // next permutation. + if (j == -1) { + return; + } + + while (true) { + int q = c[j] + o[j]; + if (q < 0) { + switchDirection(); + continue; + } + if (q == j + 1) { + if (j == 0) { + break; + } + s++; + switchDirection(); + continue; + } + + Collections.swap(list, j - c[j] + s, j - q + s); + c[j] = q; + break; + } + } + + void switchDirection() { + o[j] = -o[j]; + j--; + } + } + + /** Returns {@code true} if the second list is a permutation of the first. */ + private static boolean isPermutation(List first, List second) { + if (first.size() != second.size()) { + return false; + } + Multiset firstMultiset = HashMultiset.create(first); + Multiset secondMultiset = HashMultiset.create(second); + return firstMultiset.equals(secondMultiset); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashMap.java new file mode 100644 index 0000000000000..ae908f77b25b4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashMap.java @@ -0,0 +1,849 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.Hashing.smearedHash; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collection; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; + + + +/** + * CompactHashMap is an implementation of a Map. All optional operations (put and remove) are + * supported. Null keys and values are supported. + * + *

{@code containsKey(k)}, {@code put(k, v)} and {@code remove(k)} are all (expected and + * amortized) constant time operations. Expected in the hashtable sense (depends on the hash + * function doing a good job of distributing the elements to the buckets to a distribution not far + * from uniform), and amortized since some operations can trigger a hash table resize. + * + *

Unlike {@code java.util.HashMap}, iteration is only proportional to the actual {@code size()}, + * which is optimal, and not the size of the internal hashtable, which could be much larger + * than {@code size()}. Furthermore, this structure places significantly reduced load on the garbage + * collector by only using a constant number of internal objects. + * + *

If there are no removals, then iteration order for the {@link #entrySet}, {@link #keySet}, and + * {@link #values} views is the same as insertion order. Any removal invalidates any ordering + * guarantees. + * + *

This class should not be assumed to be universally superior to {@code java.util.HashMap}. + * Generally speaking, this class reduces object allocation and memory consumption at the price of + * moderately increased constant factors of CPU. Only use this class when there is a specific + * reason to prioritize memory over CPU. + * + * @author Louis Wasserman + */ +class CompactHashMap extends AbstractMap implements Serializable { + /* + * TODO: Make this a drop-in replacement for j.u. versions, actually drop them in, and test the + * world. Figure out what sort of space-time tradeoff we're actually going to get here with the + * *Map variants. Followon optimizations, such as using 16-bit indices for small collections, will + * take more work to implement. This class is particularly hard to benchmark, because the benefit + * is not only in less allocation, but also having the GC do less work to scan the heap because of + * fewer references, which is particularly hard to quantify. + */ + + /** Creates an empty {@code CompactHashMap} instance. */ + public static CompactHashMap create() { + return new CompactHashMap<>(); + } + + /** + * Creates a {@code CompactHashMap} instance, with a high enough "initial capacity" that it + * should hold {@code expectedSize} elements without growth. + * + * @param expectedSize the number of elements you expect to add to the returned set + * @return a new, empty {@code CompactHashMap} with enough capacity to hold {@code expectedSize} + * elements without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + */ + public static CompactHashMap createWithExpectedSize(int expectedSize) { + return new CompactHashMap<>(expectedSize); + } + + private static final int MAXIMUM_CAPACITY = 1 << 30; + + // TODO(user): decide, and inline, load factor. 0.75? + static final float DEFAULT_LOAD_FACTOR = 1.0f; + + /** Bitmask that selects the low 32 bits. */ + private static final long NEXT_MASK = (1L << 32) - 1; + + /** Bitmask that selects the high 32 bits. */ + private static final long HASH_MASK = ~NEXT_MASK; + + // TODO(user): decide default size + static final int DEFAULT_SIZE = 3; + + // used to indicate blank table entries + static final int UNSET = -1; + + /** + * The hashtable. Its values are indexes to the keys, values, and entries arrays. + * + *

Currently, the UNSET value means "null pointer", and any non negative value x is the actual + * index. + * + *

Its size must be a power of two. + */ + private transient int[] table; + + /** + * Contains the logical entries, in the range of [0, size()). The high 32 bits of each long is the + * smeared hash of the element, whereas the low 32 bits is the "next" pointer (pointing to the + * next entry in the bucket chain). The pointers in [size(), entries.length) are all "null" + * (UNSET). + */ + transient long[] entries; + + /** + * The keys of the entries in the map, in the range of [0, size()). The keys in [size(), + * keys.length) are all {@code null}. + */ + transient Object[] keys; + + /** + * The values of the entries in the map, in the range of [0, size()). The values in [size(), + * values.length) are all {@code null}. + */ + transient Object[] values; + + /** The load factor. */ + transient float loadFactor; + + /** + * Keeps track of modifications of this set, to make it possible to throw + * ConcurrentModificationException in the iterator. Note that we choose not to make this volatile, + * so we do less of a "best effort" to track such errors, for better performance. + */ + transient int modCount; + + /** When we have this many elements, resize the hashtable. */ + private transient int threshold; + + /** The number of elements contained in the set. */ + private transient int size; + + /** Constructs a new empty instance of {@code CompactHashMap}. */ + CompactHashMap() { + init(DEFAULT_SIZE, DEFAULT_LOAD_FACTOR); + } + + /** + * Constructs a new instance of {@code CompactHashMap} with the specified capacity. + * + * @param capacity the initial capacity of this {@code CompactHashMap}. + */ + CompactHashMap(int capacity) { + this(capacity, DEFAULT_LOAD_FACTOR); + } + + CompactHashMap(int expectedSize, float loadFactor) { + init(expectedSize, loadFactor); + } + + /** Pseudoconstructor for serialization support. */ + void init(int expectedSize, float loadFactor) { + Preconditions.checkArgument(expectedSize >= 0, "Initial capacity must be non-negative"); + Preconditions.checkArgument(loadFactor > 0, "Illegal load factor"); + int buckets = Hashing.closedTableSize(expectedSize, loadFactor); + this.table = newTable(buckets); + this.loadFactor = loadFactor; + + this.keys = new Object[expectedSize]; + this.values = new Object[expectedSize]; + + this.entries = newEntries(expectedSize); + this.threshold = Math.max(1, (int) (buckets * loadFactor)); + } + + private static int[] newTable(int size) { + int[] array = new int[size]; + Arrays.fill(array, UNSET); + return array; + } + + private static long[] newEntries(int size) { + long[] array = new long[size]; + Arrays.fill(array, UNSET); + return array; + } + + private int hashTableMask() { + return table.length - 1; + } + + private static int getHash(long entry) { + return (int) (entry >>> 32); + } + + /** Returns the index, or UNSET if the pointer is "null" */ + private static int getNext(long entry) { + return (int) entry; + } + + /** Returns a new entry value by changing the "next" index of an existing entry */ + private static long swapNext(long entry, int newNext) { + return (HASH_MASK & entry) | (NEXT_MASK & newNext); + } + + /** + * Mark an access of the specified entry. Used only in {@code CompactLinkedHashMap} for LRU + * ordering. + */ + void accessEntry(int index) { + // no-op by default + } + + @Override + + public V put(K key, V value) { + long[] entries = this.entries; + Object[] keys = this.keys; + Object[] values = this.values; + + int hash = smearedHash(key); + int tableIndex = hash & hashTableMask(); + int newEntryIndex = this.size; // current size, and pointer to the entry to be appended + int next = table[tableIndex]; + if (next == UNSET) { + table[tableIndex] = newEntryIndex; + } else { + int last; + long entry; + do { + last = next; + entry = entries[next]; + if (getHash(entry) == hash && Objects.equal(key, keys[next])) { + @SuppressWarnings({"unchecked", "rawtypes"}) // known to be a V + + V oldValue = (V) values[next]; + + values[next] = value; + accessEntry(next); + return oldValue; + } + next = getNext(entry); + } while (next != UNSET); + entries[last] = swapNext(entry, newEntryIndex); + } + if (newEntryIndex == Integer.MAX_VALUE) { + throw new IllegalStateException("Cannot contain more than Integer.MAX_VALUE elements!"); + } + int newSize = newEntryIndex + 1; + resizeMeMaybe(newSize); + insertEntry(newEntryIndex, key, value, hash); + this.size = newSize; + if (newEntryIndex >= threshold) { + resizeTable(2 * table.length); + } + modCount++; + return null; + } + + /** + * Creates a fresh entry with the specified object at the specified position in the entry arrays. + */ + void insertEntry(int entryIndex, K key, V value, int hash) { + this.entries[entryIndex] = ((long) hash << 32) | (NEXT_MASK & UNSET); + this.keys[entryIndex] = key; + this.values[entryIndex] = value; + } + + /** Returns currentSize + 1, after resizing the entries storage if necessary. */ + private void resizeMeMaybe(int newSize) { + int entriesSize = entries.length; + if (newSize > entriesSize) { + int newCapacity = entriesSize + Math.max(1, entriesSize >>> 1); + if (newCapacity < 0) { + newCapacity = Integer.MAX_VALUE; + } + if (newCapacity != entriesSize) { + resizeEntries(newCapacity); + } + } + } + + /** + * Resizes the internal entries array to the specified capacity, which may be greater or less than + * the current capacity. + */ + void resizeEntries(int newCapacity) { + this.keys = Arrays.copyOf(keys, newCapacity); + this.values = Arrays.copyOf(values, newCapacity); + long[] entries = this.entries; + int oldCapacity = entries.length; + entries = Arrays.copyOf(entries, newCapacity); + if (newCapacity > oldCapacity) { + Arrays.fill(entries, oldCapacity, newCapacity, UNSET); + } + this.entries = entries; + } + + private void resizeTable(int newCapacity) { // newCapacity always a power of two + int[] oldTable = table; + int oldCapacity = oldTable.length; + if (oldCapacity >= MAXIMUM_CAPACITY) { + threshold = Integer.MAX_VALUE; + return; + } + int newThreshold = 1 + (int) (newCapacity * loadFactor); + int[] newTable = newTable(newCapacity); + long[] entries = this.entries; + + int mask = newTable.length - 1; + for (int i = 0; i < size; i++) { + long oldEntry = entries[i]; + int hash = getHash(oldEntry); + int tableIndex = hash & mask; + int next = newTable[tableIndex]; + newTable[tableIndex] = i; + entries[i] = ((long) hash << 32) | (NEXT_MASK & next); + } + + this.threshold = newThreshold; + this.table = newTable; + } + + private int indexOf(Object key) { + int hash = smearedHash(key); + int next = table[hash & hashTableMask()]; + while (next != UNSET) { + long entry = entries[next]; + if (getHash(entry) == hash && Objects.equal(key, keys[next])) { + return next; + } + next = getNext(entry); + } + return -1; + } + + @Override + public boolean containsKey(Object key) { + return indexOf(key) != -1; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // values only contains Vs + @Override + public V get(Object key) { + int index = indexOf(key); + accessEntry(index); + return (index == -1) ? null : (V) values[index]; + } + + @Override + + public V remove(Object key) { + return remove(key, smearedHash(key)); + } + + + private V remove(Object key, int hash) { + int tableIndex = hash & hashTableMask(); + int next = table[tableIndex]; + if (next == UNSET) { // empty bucket + return null; + } + int last = UNSET; + do { + if (getHash(entries[next]) == hash) { + if (Objects.equal(key, keys[next])) { + @SuppressWarnings({"unchecked", "rawtypes"}) // values only contains Vs + + V oldValue = (V) values[next]; + + if (last == UNSET) { + // we need to update the root link from table[] + table[tableIndex] = getNext(entries[next]); + } else { + // we need to update the link from the chain + entries[last] = swapNext(entries[last], getNext(entries[next])); + } + + moveLastEntry(next); + size--; + modCount++; + return oldValue; + } + } + last = next; + next = getNext(entries[next]); + } while (next != UNSET); + return null; + } + + private V removeEntry(int entryIndex) { + return remove(keys[entryIndex], getHash(entries[entryIndex])); + } + + /** + * Moves the last entry in the entry array into {@code dstIndex}, and nulls out its old position. + */ + void moveLastEntry(int dstIndex) { + int srcIndex = size() - 1; + if (dstIndex < srcIndex) { + // move last entry to deleted spot + keys[dstIndex] = keys[srcIndex]; + values[dstIndex] = values[srcIndex]; + keys[srcIndex] = null; + values[srcIndex] = null; + + // move the last entry to the removed spot, just like we moved the element + long lastEntry = entries[srcIndex]; + entries[dstIndex] = lastEntry; + entries[srcIndex] = UNSET; + + // also need to update whoever's "next" pointer was pointing to the last entry place + // reusing "tableIndex" and "next"; these variables were no longer needed + int tableIndex = getHash(lastEntry) & hashTableMask(); + int lastNext = table[tableIndex]; + if (lastNext == srcIndex) { + // we need to update the root pointer + table[tableIndex] = dstIndex; + } else { + // we need to update a pointer in an entry + int previous; + long entry; + do { + previous = lastNext; + lastNext = getNext(entry = entries[lastNext]); + } while (lastNext != srcIndex); + // here, entries[previous] points to the old entry location; update it + entries[previous] = swapNext(entry, dstIndex); + } + } else { + keys[dstIndex] = null; + values[dstIndex] = null; + entries[dstIndex] = UNSET; + } + } + + int firstEntryIndex() { + return isEmpty() ? -1 : 0; + } + + int getSuccessor(int entryIndex) { + return (entryIndex + 1 < size) ? entryIndex + 1 : -1; + } + + /** + * Updates the index an iterator is pointing to after a call to remove: returns the index of the + * entry that should be looked at after a removal on indexRemoved, with indexBeforeRemove as the + * index that *was* the next entry that would be looked at. + */ + int adjustAfterRemove(int indexBeforeRemove, @SuppressWarnings("unused") int indexRemoved) { + return indexBeforeRemove - 1; + } + + private abstract class Itr implements Iterator { + int expectedModCount = modCount; + int currentIndex = firstEntryIndex(); + int indexToRemove = -1; + + @Override + public boolean hasNext() { + return currentIndex >= 0; + } + + abstract T getOutput(int entry); + + @Override + public T next() { + checkForConcurrentModification(); + if (!hasNext()) { + throw new NoSuchElementException(); + } + indexToRemove = currentIndex; + T result = getOutput(currentIndex); + currentIndex = getSuccessor(currentIndex); + return result; + } + + @Override + public void remove() { + checkForConcurrentModification(); + checkRemove(indexToRemove >= 0); + expectedModCount++; + removeEntry(indexToRemove); + currentIndex = adjustAfterRemove(currentIndex, indexToRemove); + indexToRemove = -1; + } + + private void checkForConcurrentModification() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + } + } + + @Override + @SuppressWarnings("unchecked") + public void replaceAll(BiFunction function) { + checkNotNull(function); + for (int i = 0; i < size; i++) { + //noinspection unchecked + values[i] = function.apply((K) keys[i], (V) values[i]); + } + } + + private transient Set keySetView; + + @Override + public Set keySet() { + return (keySetView == null) ? keySetView = createKeySet() : keySetView; + } + + Set createKeySet() { + return new KeySetView(); + } + + class KeySetView extends Maps.KeySet { + KeySetView() { + super(CompactHashMap.this); + } + + @Override + public Object[] toArray() { + return ObjectArrays.copyAsObjectArray(keys, 0, size); + } + + @Override + public T[] toArray(T[] a) { + return ObjectArrays.toArrayImpl(keys, 0, size, a); + } + + @Override + public boolean remove(Object o) { + int index = indexOf(o); + if (index == -1) { + return false; + } else { + removeEntry(index); + return true; + } + } + + @Override + public Iterator iterator() { + return keySetIterator(); + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(keys, 0, size, Spliterator.DISTINCT | Spliterator.ORDERED); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public void forEach(Consumer action) { + checkNotNull(action); + for (int i = 0; i < size; i++) { + action.accept((K) keys[i]); + } + } + } + + Iterator keySetIterator() { + return new Itr() { + @SuppressWarnings({"unchecked", "rawtypes"}) // keys only contains Ks + @Override + K getOutput(int entry) { + return (K) keys[entry]; + } + }; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + for (int i = 0; i < size; i++) { + action.accept((K) keys[i], (V) values[i]); + } + } + + private transient Set> entrySetView; + + @Override + public Set> entrySet() { + return (entrySetView == null) ? entrySetView = createEntrySet() : entrySetView; + } + + Set> createEntrySet() { + return new EntrySetView(); + } + + class EntrySetView extends Maps.EntrySet { + @Override + Map map() { + return CompactHashMap.this; + } + + @Override + public Iterator> iterator() { + return entrySetIterator(); + } + + @Override + public Spliterator> spliterator() { + return CollectSpliterators.indexed( + size, Spliterator.DISTINCT | Spliterator.ORDERED, MapEntry::new); + } + + @Override + public boolean contains(Object o) { + if (o instanceof Entry) { + Entry entry = (Entry) o; + int index = indexOf(entry.getKey()); + return index != -1 && Objects.equal(values[index], entry.getValue()); + } + return false; + } + + @Override + public boolean remove(Object o) { + if (o instanceof Entry) { + Entry entry = (Entry) o; + int index = indexOf(entry.getKey()); + if (index != -1 && Objects.equal(values[index], entry.getValue())) { + removeEntry(index); + return true; + } + } + return false; + } + } + + Iterator> entrySetIterator() { + return new Itr>() { + @Override + Entry getOutput(int entry) { + return new MapEntry(entry); + } + }; + } + + final class MapEntry extends AbstractMapEntry { + private final K key; + + private int lastKnownIndex; + + @SuppressWarnings({"unchecked", "rawtypes"}) // keys only contains Ks + MapEntry(int index) { + this.key = (K) keys[index]; + this.lastKnownIndex = index; + } + + @Override + public K getKey() { + return key; + } + + private void updateLastKnownIndex() { + if (lastKnownIndex == -1 + || lastKnownIndex >= size() + || !Objects.equal(key, keys[lastKnownIndex])) { + lastKnownIndex = indexOf(key); + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // values only contains Vs + @Override + public V getValue() { + updateLastKnownIndex(); + return (lastKnownIndex == -1) ? null : (V) values[lastKnownIndex]; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // values only contains Vs + @Override + public V setValue(V value) { + updateLastKnownIndex(); + if (lastKnownIndex == -1) { + put(key, value); + return null; + } else { + V old = (V) values[lastKnownIndex]; + values[lastKnownIndex] = value; + return old; + } + } + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return size == 0; + } + + @Override + public boolean containsValue(Object value) { + for (int i = 0; i < size; i++) { + if (Objects.equal(value, values[i])) { + return true; + } + } + return false; + } + + private transient Collection valuesView; + + @Override + public Collection values() { + return (valuesView == null) ? valuesView = createValues() : valuesView; + } + + Collection createValues() { + return new ValuesView(); + } + + class ValuesView extends Maps.Values { + ValuesView() { + super(CompactHashMap.this); + } + + @Override + public Iterator iterator() { + return valuesIterator(); + } + + @Override + @SuppressWarnings("unchecked") + public void forEach(Consumer action) { + checkNotNull(action); + for (int i = 0; i < size; i++) { + //noinspection unchecked + action.accept((V) values[i]); + } + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(values, 0, size, Spliterator.ORDERED); + } + + @Override + public Object[] toArray() { + return ObjectArrays.copyAsObjectArray(values, 0, size); + } + + @Override + public T[] toArray(T[] a) { + return ObjectArrays.toArrayImpl(values, 0, size, a); + } + } + + Iterator valuesIterator() { + return new Itr() { + @SuppressWarnings({"unchecked", "rawtypes"}) // values only contains Vs + @Override + V getOutput(int entry) { + return (V) values[entry]; + } + }; + } + + /** + * Ensures that this {@code CompactHashMap} has the smallest representation in memory, given its + * current size. + */ + public void trimToSize() { + int size = this.size; + if (size < entries.length) { + resizeEntries(size); + } + // size / loadFactor gives the table size of the appropriate load factor, + // but that may not be a power of two. We floor it to a power of two by + // keeping its highest bit. But the smaller table may have a load factor + // larger than what we want; then we want to go to the next power of 2 if we can + int minimumTableSize = Math.max(1, Integer.highestOneBit((int) (size / loadFactor))); + if (minimumTableSize < MAXIMUM_CAPACITY) { + double load = (double) size / minimumTableSize; + if (load > loadFactor) { + minimumTableSize <<= 1; // increase to next power if possible + } + } + + if (minimumTableSize < table.length) { + resizeTable(minimumTableSize); + } + } + + @Override + public void clear() { + modCount++; + Arrays.fill(keys, 0, size, null); + Arrays.fill(values, 0, size, null); + Arrays.fill(table, UNSET); + Arrays.fill(entries, UNSET); + this.size = 0; + } + + /** + * The serial form currently mimics Android's java.util.HashMap version, e.g. see + * http://omapzoom.org/?p=platform/libcore.git;a=blob;f=luni/src/main/java/java/util/HashMap.java + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeInt(size); + for (int i = 0; i < size; i++) { + stream.writeObject(keys[i]); + stream.writeObject(values[i]); + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + init(DEFAULT_SIZE, DEFAULT_LOAD_FACTOR); + int elementCount = stream.readInt(); + for (int i = elementCount; --i >= 0; ) { + K key = (K) stream.readObject(); + V value = (V) stream.readObject(); + put(key, value); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashSet.java new file mode 100644 index 0000000000000..7c66b793c534d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactHashSet.java @@ -0,0 +1,567 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.Hashing.smearedHash; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.AbstractSet; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.Consumer; + + + +/** + * CompactHashSet is an implementation of a Set. All optional operations (adding and removing) are + * supported. The elements can be any objects. + * + *

{@code contains(x)}, {@code add(x)} and {@code remove(x)}, are all (expected and amortized) + * constant time operations. Expected in the hashtable sense (depends on the hash function doing a + * good job of distributing the elements to the buckets to a distribution not far from uniform), and + * amortized since some operations can trigger a hash table resize. + * + *

Unlike {@code java.util.HashSet}, iteration is only proportional to the actual {@code size()}, + * which is optimal, and not the size of the internal hashtable, which could be much larger + * than {@code size()}. Furthermore, this structure only depends on a fixed number of arrays; {@code + * add(x)} operations do not create objects for the garbage collector to deal with, and for + * every element added, the garbage collector will have to traverse {@code 1.5} references on + * average, in the marking phase, not {@code 5.0} as in {@code java.util.HashSet}. + * + *

If there are no removals, then {@link #iterator iteration} order is the same as insertion + * order. Any removal invalidates any ordering guarantees. + * + *

This class should not be assumed to be universally superior to {@code java.util.HashSet}. + * Generally speaking, this class reduces object allocation and memory consumption at the price of + * moderately increased constant factors of CPU. Only use this class when there is a specific + * reason to prioritize memory over CPU. + * + * @author Dimitris Andreou + */ +class CompactHashSet extends AbstractSet implements Serializable { + // TODO(user): cache all field accesses in local vars + + /** Creates an empty {@code CompactHashSet} instance. */ + public static CompactHashSet create() { + return new CompactHashSet(); + } + + /** + * Creates a mutable {@code CompactHashSet} instance containing the elements of the given + * collection in unspecified order. + * + * @param collection the elements that the set should contain + * @return a new {@code CompactHashSet} containing those elements (minus duplicates) + */ + public static CompactHashSet create(Collection collection) { + CompactHashSet set = createWithExpectedSize(collection.size()); + set.addAll(collection); + return set; + } + + /** + * Creates a mutable {@code CompactHashSet} instance containing the given elements in + * unspecified order. + * + * @param elements the elements that the set should contain + * @return a new {@code CompactHashSet} containing those elements (minus duplicates) + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static CompactHashSet create(E... elements) { + CompactHashSet set = createWithExpectedSize(elements.length); + Collections.addAll(set, elements); + return set; + } + + /** + * Creates a {@code CompactHashSet} instance, with a high enough "initial capacity" that it + * should hold {@code expectedSize} elements without growth. + * + * @param expectedSize the number of elements you expect to add to the returned set + * @return a new, empty {@code CompactHashSet} with enough capacity to hold {@code expectedSize} + * elements without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + */ + public static CompactHashSet createWithExpectedSize(int expectedSize) { + return new CompactHashSet(expectedSize); + } + + private static final int MAXIMUM_CAPACITY = 1 << 30; + + // TODO(user): decide, and inline, load factor. 0.75? + private static final float DEFAULT_LOAD_FACTOR = 1.0f; + + /** Bitmask that selects the low 32 bits. */ + private static final long NEXT_MASK = (1L << 32) - 1; + + /** Bitmask that selects the high 32 bits. */ + private static final long HASH_MASK = ~NEXT_MASK; + + // TODO(user): decide default size + private static final int DEFAULT_SIZE = 3; + + static final int UNSET = -1; + + /** + * The hashtable. Its values are indexes to both the elements and entries arrays. + * + *

Currently, the UNSET value means "null pointer", and any non negative value x is the actual + * index. + * + *

Its size must be a power of two. + */ + private transient int[] table; + + /** + * Contains the logical entries, in the range of [0, size()). The high 32 bits of each long is the + * smeared hash of the element, whereas the low 32 bits is the "next" pointer (pointing to the + * next entry in the bucket chain). The pointers in [size(), entries.length) are all "null" + * (UNSET). + */ + private transient long[] entries; + + /** The elements contained in the set, in the range of [0, size()). */ + transient Object[] elements; + + /** The load factor. */ + transient float loadFactor; + + /** + * Keeps track of modifications of this set, to make it possible to throw + * ConcurrentModificationException in the iterator. Note that we choose not to make this volatile, + * so we do less of a "best effort" to track such errors, for better performance. + */ + transient int modCount; + + /** When we have this many elements, resize the hashtable. */ + private transient int threshold; + + /** The number of elements contained in the set. */ + private transient int size; + + /** Constructs a new empty instance of {@code CompactHashSet}. */ + CompactHashSet() { + init(DEFAULT_SIZE, DEFAULT_LOAD_FACTOR); + } + + /** + * Constructs a new instance of {@code CompactHashSet} with the specified capacity. + * + * @param expectedSize the initial capacity of this {@code CompactHashSet}. + */ + CompactHashSet(int expectedSize) { + init(expectedSize, DEFAULT_LOAD_FACTOR); + } + + /** Pseudoconstructor for serialization support. */ + void init(int expectedSize, float loadFactor) { + Preconditions.checkArgument(expectedSize >= 0, "Initial capacity must be non-negative"); + Preconditions.checkArgument(loadFactor > 0, "Illegal load factor"); + int buckets = Hashing.closedTableSize(expectedSize, loadFactor); + this.table = newTable(buckets); + this.loadFactor = loadFactor; + this.elements = new Object[expectedSize]; + this.entries = newEntries(expectedSize); + this.threshold = Math.max(1, (int) (buckets * loadFactor)); + } + + private static int[] newTable(int size) { + int[] array = new int[size]; + Arrays.fill(array, UNSET); + return array; + } + + private static long[] newEntries(int size) { + long[] array = new long[size]; + Arrays.fill(array, UNSET); + return array; + } + + private static int getHash(long entry) { + return (int) (entry >>> 32); + } + + /** Returns the index, or UNSET if the pointer is "null" */ + private static int getNext(long entry) { + return (int) entry; + } + + /** Returns a new entry value by changing the "next" index of an existing entry */ + private static long swapNext(long entry, int newNext) { + return (HASH_MASK & entry) | (NEXT_MASK & newNext); + } + + private int hashTableMask() { + return table.length - 1; + } + + @Override + public boolean add(E object) { + long[] entries = this.entries; + Object[] elements = this.elements; + int hash = smearedHash(object); + int tableIndex = hash & hashTableMask(); + int newEntryIndex = this.size; // current size, and pointer to the entry to be appended + int next = table[tableIndex]; + if (next == UNSET) { // uninitialized bucket + table[tableIndex] = newEntryIndex; + } else { + int last; + long entry; + do { + last = next; + entry = entries[next]; + if (getHash(entry) == hash && Objects.equal(object, elements[next])) { + return false; + } + next = getNext(entry); + } while (next != UNSET); + entries[last] = swapNext(entry, newEntryIndex); + } + if (newEntryIndex == Integer.MAX_VALUE) { + throw new IllegalStateException("Cannot contain more than Integer.MAX_VALUE elements!"); + } + int newSize = newEntryIndex + 1; + resizeMeMaybe(newSize); + insertEntry(newEntryIndex, object, hash); + this.size = newSize; + if (newEntryIndex >= threshold) { + resizeTable(2 * table.length); + } + modCount++; + return true; + } + + /** + * Creates a fresh entry with the specified object at the specified position in the entry arrays. + */ + void insertEntry(int entryIndex, E object, int hash) { + this.entries[entryIndex] = ((long) hash << 32) | (NEXT_MASK & UNSET); + this.elements[entryIndex] = object; + } + + /** Returns currentSize + 1, after resizing the entries storage if necessary. */ + private void resizeMeMaybe(int newSize) { + int entriesSize = entries.length; + if (newSize > entriesSize) { + int newCapacity = entriesSize + Math.max(1, entriesSize >>> 1); + if (newCapacity < 0) { + newCapacity = Integer.MAX_VALUE; + } + if (newCapacity != entriesSize) { + resizeEntries(newCapacity); + } + } + } + + /** + * Resizes the internal entries array to the specified capacity, which may be greater or less than + * the current capacity. + */ + void resizeEntries(int newCapacity) { + this.elements = Arrays.copyOf(elements, newCapacity); + long[] entries = this.entries; + int oldSize = entries.length; + entries = Arrays.copyOf(entries, newCapacity); + if (newCapacity > oldSize) { + Arrays.fill(entries, oldSize, newCapacity, UNSET); + } + this.entries = entries; + } + + private void resizeTable(int newCapacity) { // newCapacity always a power of two + int[] oldTable = table; + int oldCapacity = oldTable.length; + if (oldCapacity >= MAXIMUM_CAPACITY) { + threshold = Integer.MAX_VALUE; + return; + } + int newThreshold = 1 + (int) (newCapacity * loadFactor); + int[] newTable = newTable(newCapacity); + long[] entries = this.entries; + + int mask = newTable.length - 1; + for (int i = 0; i < size; i++) { + long oldEntry = entries[i]; + int hash = getHash(oldEntry); + int tableIndex = hash & mask; + int next = newTable[tableIndex]; + newTable[tableIndex] = i; + entries[i] = ((long) hash << 32) | (NEXT_MASK & next); + } + + this.threshold = newThreshold; + this.table = newTable; + } + + @Override + public boolean contains(Object object) { + int hash = smearedHash(object); + int next = table[hash & hashTableMask()]; + while (next != UNSET) { + long entry = entries[next]; + if (getHash(entry) == hash && Objects.equal(object, elements[next])) { + return true; + } + next = getNext(entry); + } + return false; + } + + @Override + public boolean remove(Object object) { + return remove(object, smearedHash(object)); + } + + private boolean remove(Object object, int hash) { + int tableIndex = hash & hashTableMask(); + int next = table[tableIndex]; + if (next == UNSET) { + return false; + } + int last = UNSET; + do { + if (getHash(entries[next]) == hash && Objects.equal(object, elements[next])) { + if (last == UNSET) { + // we need to update the root link from table[] + table[tableIndex] = getNext(entries[next]); + } else { + // we need to update the link from the chain + entries[last] = swapNext(entries[last], getNext(entries[next])); + } + + moveEntry(next); + size--; + modCount++; + return true; + } + last = next; + next = getNext(entries[next]); + } while (next != UNSET); + return false; + } + + /** + * Moves the last entry in the entry array into {@code dstIndex}, and nulls out its old position. + */ + void moveEntry(int dstIndex) { + int srcIndex = size() - 1; + if (dstIndex < srcIndex) { + // move last entry to deleted spot + elements[dstIndex] = elements[srcIndex]; + elements[srcIndex] = null; + + // move the last entry to the removed spot, just like we moved the element + long lastEntry = entries[srcIndex]; + entries[dstIndex] = lastEntry; + entries[srcIndex] = UNSET; + + // also need to update whoever's "next" pointer was pointing to the last entry place + // reusing "tableIndex" and "next"; these variables were no longer needed + int tableIndex = getHash(lastEntry) & hashTableMask(); + int lastNext = table[tableIndex]; + if (lastNext == srcIndex) { + // we need to update the root pointer + table[tableIndex] = dstIndex; + } else { + // we need to update a pointer in an entry + int previous; + long entry; + do { + previous = lastNext; + lastNext = getNext(entry = entries[lastNext]); + } while (lastNext != srcIndex); + // here, entries[previous] points to the old entry location; update it + entries[previous] = swapNext(entry, dstIndex); + } + } else { + elements[dstIndex] = null; + entries[dstIndex] = UNSET; + } + } + + int firstEntryIndex() { + return isEmpty() ? -1 : 0; + } + + int getSuccessor(int entryIndex) { + return (entryIndex + 1 < size) ? entryIndex + 1 : -1; + } + + /** + * Updates the index an iterator is pointing to after a call to remove: returns the index of the + * entry that should be looked at after a removal on indexRemoved, with indexBeforeRemove as the + * index that *was* the next entry that would be looked at. + */ + int adjustAfterRemove(int indexBeforeRemove, @SuppressWarnings("unused") int indexRemoved) { + return indexBeforeRemove - 1; + } + + @Override + public Iterator iterator() { + return new Iterator() { + int expectedModCount = modCount; + int index = firstEntryIndex(); + int indexToRemove = -1; + + @Override + public boolean hasNext() { + return index >= 0; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public E next() { + checkForConcurrentModification(); + if (!hasNext()) { + throw new NoSuchElementException(); + } + indexToRemove = index; + E result = (E) elements[index]; + index = getSuccessor(index); + return result; + } + + @Override + public void remove() { + checkForConcurrentModification(); + checkRemove(indexToRemove >= 0); + expectedModCount++; + CompactHashSet.this.remove(elements[indexToRemove], getHash(entries[indexToRemove])); + index = adjustAfterRemove(index, indexToRemove); + indexToRemove = -1; + } + + private void checkForConcurrentModification() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + } + }; + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(elements, 0, size, Spliterator.DISTINCT | Spliterator.ORDERED); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public void forEach(Consumer action) { + checkNotNull(action); + for (int i = 0; i < size; i++) { + action.accept((E) elements[i]); + } + } + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return size == 0; + } + + @Override + public Object[] toArray() { + return Arrays.copyOf(elements, size); + } + + @Override + public T[] toArray(T[] a) { + return ObjectArrays.toArrayImpl(elements, 0, size, a); + } + + /** + * Ensures that this {@code CompactHashSet} has the smallest representation in memory, given its + * current size. + */ + public void trimToSize() { + int size = this.size; + if (size < entries.length) { + resizeEntries(size); + } + // size / loadFactor gives the table size of the appropriate load factor, + // but that may not be a power of two. We floor it to a power of two by + // keeping its highest bit. But the smaller table may have a load factor + // larger than what we want; then we want to go to the next power of 2 if we can + int minimumTableSize = Math.max(1, Integer.highestOneBit((int) (size / loadFactor))); + if (minimumTableSize < MAXIMUM_CAPACITY) { + double load = (double) size / minimumTableSize; + if (load > loadFactor) { + minimumTableSize <<= 1; // increase to next power if possible + } + } + + if (minimumTableSize < table.length) { + resizeTable(minimumTableSize); + } + } + + @Override + public void clear() { + modCount++; + Arrays.fill(elements, 0, size, null); + Arrays.fill(table, UNSET); + Arrays.fill(entries, UNSET); + this.size = 0; + } + + /** + * The serial form currently mimics Android's java.util.HashSet version, e.g. see + * http://omapzoom.org/?p=platform/libcore.git;a=blob;f=luni/src/main/java/java/util/HashSet.java + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeInt(size); + for (E e : this) { + stream.writeObject(e); + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + init(DEFAULT_SIZE, DEFAULT_LOAD_FACTOR); + int elementCount = stream.readInt(); + for (int i = elementCount; --i >= 0; ) { + E element = (E) stream.readObject(); + add(element); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashMap.java new file mode 100644 index 0000000000000..50f9583bdc8ab --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashMap.java @@ -0,0 +1,288 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + + +/** + * CompactLinkedHashMap is an implementation of a Map with insertion or LRU iteration order, + * maintained with a doubly linked list through the entries. All optional operations (put and + * remove) are supported. Null keys and values are supported. + * + *

{@code containsKey(k)}, {@code put(k, v)} and {@code remove(k)} are all (expected and + * amortized) constant time operations. Expected in the hashtable sense (depends on the hash + * function doing a good job of distributing the elements to the buckets to a distribution not far + * from uniform), and amortized since some operations can trigger a hash table resize. + * + *

As compared with {@link java.util.LinkedHashMap}, this structure places significantly reduced + * load on the garbage collector by only using a constant number of internal objects. + * + *

This class should not be assumed to be universally superior to {@code + * java.util.LinkedHashMap}. Generally speaking, this class reduces object allocation and memory + * consumption at the price of moderately increased constant factors of CPU. Only use this class + * when there is a specific reason to prioritize memory over CPU. + * + * @author Louis Wasserman + */ +class CompactLinkedHashMap extends CompactHashMap { + // TODO(lowasser): implement removeEldestEntry so this can be used as a drop-in replacement + + /** + * Creates an empty {@code CompactLinkedHashMap} instance. + */ + public static CompactLinkedHashMap create() { + return new CompactLinkedHashMap<>(); + } + + /** + * Creates a {@code CompactLinkedHashMap} instance, with a high enough "initial capacity" + * that it should hold {@code expectedSize} elements without growth. + * + * @param expectedSize the number of elements you expect to add to the returned set + * @return a new, empty {@code CompactLinkedHashMap} with enough capacity to hold {@code + * expectedSize} elements without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + */ + public static CompactLinkedHashMap createWithExpectedSize(int expectedSize) { + return new CompactLinkedHashMap<>(expectedSize); + } + + private static final int ENDPOINT = -2; + + /** + * Contains the link pointers corresponding with the entries, in the range of [0, size()). The + * high 32 bits of each long is the "prev" pointer, whereas the low 32 bits is the "succ" pointer + * (pointing to the next entry in the linked list). The pointers in [size(), entries.length) are + * all "null" (UNSET). + * + * A node with "prev" pointer equal to {@code ENDPOINT} is the first node in the linked list, + * and a node with "next" pointer equal to {@code ENDPOINT} is the last node. + */ + transient long[] links; + + /** + * Pointer to the first node in the linked list, or {@code ENDPOINT} if there are no entries. + */ + private transient int firstEntry; + + /** + * Pointer to the last node in the linked list, or {@code ENDPOINT} if there are no entries. + */ + private transient int lastEntry; + + private final boolean accessOrder; + + CompactLinkedHashMap() { + this(DEFAULT_SIZE); + } + + CompactLinkedHashMap(int expectedSize) { + this(expectedSize, DEFAULT_LOAD_FACTOR, false); + } + + CompactLinkedHashMap(int expectedSize, float loadFactor, boolean accessOrder) { + super(expectedSize, loadFactor); + this.accessOrder = accessOrder; + } + + @Override + void init(int expectedSize, float loadFactor) { + super.init(expectedSize, loadFactor); + firstEntry = ENDPOINT; + lastEntry = ENDPOINT; + links = new long[expectedSize]; + Arrays.fill(links, UNSET); + } + + private int getPredecessor(int entry) { + return (int) (links[entry] >>> 32); + } + + @Override + int getSuccessor(int entry) { + return (int) links[entry]; + } + + private void setSuccessor(int entry, int succ) { + long succMask = (~0L) >>> 32; + links[entry] = (links[entry] & ~succMask) | (succ & succMask); + } + + private void setPredecessor(int entry, int pred) { + long predMask = (~0L) << 32; + links[entry] = (links[entry] & ~predMask) | ((long) pred << 32); + } + + private void setSucceeds(int pred, int succ) { + if (pred == ENDPOINT) { + firstEntry = succ; + } else { + setSuccessor(pred, succ); + } + if (succ == ENDPOINT) { + lastEntry = pred; + } else { + setPredecessor(succ, pred); + } + } + + @Override + void insertEntry(int entryIndex, K key, V value, int hash) { + super.insertEntry(entryIndex, key, value, hash); + setSucceeds(lastEntry, entryIndex); + setSucceeds(entryIndex, ENDPOINT); + } + + @Override + void accessEntry(int index) { + if (accessOrder) { + // delete from previous position... + setSucceeds(getPredecessor(index), getSuccessor(index)); + // ...and insert at the end. + setSucceeds(lastEntry, index); + setSucceeds(index, ENDPOINT); + modCount++; + } + } + + @Override + void moveLastEntry(int dstIndex) { + int srcIndex = size() - 1; + setSucceeds(getPredecessor(dstIndex), getSuccessor(dstIndex)); + if (dstIndex < srcIndex) { + setSucceeds(getPredecessor(srcIndex), dstIndex); + setSucceeds(dstIndex, getSuccessor(srcIndex)); + } + super.moveLastEntry(dstIndex); + } + + @Override + void resizeEntries(int newCapacity) { + super.resizeEntries(newCapacity); + links = Arrays.copyOf(links, newCapacity); + } + + @Override + int firstEntryIndex() { + return firstEntry; + } + + @Override + int adjustAfterRemove(int indexBeforeRemove, int indexRemoved) { + return (indexBeforeRemove >= size()) ? indexRemoved : indexBeforeRemove; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + for (int i = firstEntry; i != ENDPOINT; i = getSuccessor(i)) { + action.accept((K) keys[i], (V) values[i]); + } + } + + @Override + Set> createEntrySet() { + class EntrySetImpl extends EntrySetView { + @Override + public Spliterator> spliterator() { + return Spliterators.spliterator(this, Spliterator.ORDERED | Spliterator.DISTINCT); + } + } + return new EntrySetImpl(); + } + + @Override + Set createKeySet() { + class KeySetImpl extends KeySetView { + @Override + public Object[] toArray() { + return ObjectArrays.toArrayImpl(this); + } + + @Override + public T[] toArray(T[] a) { + return ObjectArrays.toArrayImpl(this, a); + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(this, Spliterator.ORDERED | Spliterator.DISTINCT); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public void forEach(Consumer action) { + checkNotNull(action); + for (int i = firstEntry; i != ENDPOINT; i = getSuccessor(i)) { + action.accept((K) keys[i]); + } + } + } + return new KeySetImpl(); + } + + @Override + Collection createValues() { + class ValuesImpl extends ValuesView { + @Override + public Object[] toArray() { + return ObjectArrays.toArrayImpl(this); + } + + @Override + public T[] toArray(T[] a) { + return ObjectArrays.toArrayImpl(this, a); + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(this, Spliterator.ORDERED); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public void forEach(Consumer action) { + checkNotNull(action); + for (int i = firstEntry; i != ENDPOINT; i = getSuccessor(i)) { + action.accept((V) values[i]); + } + } + } + return new ValuesImpl(); + } + + @Override + public void clear() { + super.clear(); + this.firstEntry = ENDPOINT; + this.lastEntry = ENDPOINT; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashSet.java new file mode 100644 index 0000000000000..93774223d928d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompactLinkedHashSet.java @@ -0,0 +1,239 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.Consumer; + + +/** + * CompactLinkedHashSet is an implementation of a Set, which a predictable iteration order that + * matches the insertion order. All optional operations (adding and removing) are supported. All + * elements, including {@code null}, are permitted. + * + *

{@code contains(x)}, {@code add(x)} and {@code remove(x)}, are all (expected and amortized) + * constant time operations. Expected in the hashtable sense (depends on the hash function doing a + * good job of distributing the elements to the buckets to a distribution not far from uniform), and + * amortized since some operations can trigger a hash table resize. + * + *

This implementation consumes significantly less memory than {@code java.util.LinkedHashSet} or + * even {@code java.util.HashSet}, and places considerably less load on the garbage collector. Like + * {@code java.util.LinkedHashSet}, it offers insertion-order iteration, with identical behavior. + * + *

This class should not be assumed to be universally superior to {@code + * java.util.LinkedHashSet}. Generally speaking, this class reduces object allocation and memory + * consumption at the price of moderately increased constant factors of CPU. Only use this class + * when there is a specific reason to prioritize memory over CPU. + * + * @author Louis Wasserman + */ +class CompactLinkedHashSet extends CompactHashSet { + + /** + * Creates an empty {@code CompactLinkedHashSet} instance. + */ + public static CompactLinkedHashSet create() { + return new CompactLinkedHashSet(); + } + + /** + * Creates a mutable {@code CompactLinkedHashSet} instance containing the elements + * of the given collection in the order returned by the collection's iterator. + * + * @param collection the elements that the set should contain + * @return a new {@code CompactLinkedHashSet} containing those elements (minus duplicates) + */ + public static CompactLinkedHashSet create(Collection collection) { + CompactLinkedHashSet set = createWithExpectedSize(collection.size()); + set.addAll(collection); + return set; + } + + /** + * Creates a {@code CompactLinkedHashSet} instance containing the given elements in + * unspecified order. + * + * @param elements the elements that the set should contain + * @return a new {@code CompactLinkedHashSet} containing those elements (minus duplicates) + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static CompactLinkedHashSet create(E... elements) { + CompactLinkedHashSet set = createWithExpectedSize(elements.length); + Collections.addAll(set, elements); + return set; + } + + /** + * Creates a {@code CompactLinkedHashSet} instance, with a high enough "initial capacity" + * that it should hold {@code expectedSize} elements without rebuilding internal + * data structures. + * + * @param expectedSize the number of elements you expect to add to the returned set + * @return a new, empty {@code CompactLinkedHashSet} with enough capacity to hold {@code + * expectedSize} elements without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + */ + public static CompactLinkedHashSet createWithExpectedSize(int expectedSize) { + return new CompactLinkedHashSet(expectedSize); + } + + private static final int ENDPOINT = -2; + + // TODO(user): predecessors and successors should be collocated (reducing cache misses). + // Might also explore collocating all of [hash, next, predecessor, succesor] fields of an + // entry in a *single* long[], though that reduces the maximum size of the set by a factor of 2 + + /** + * Pointer to the predecessor of an entry in insertion order. ENDPOINT indicates a node is the + * first node in insertion order; all values at indices ≥ {@link #size()} are UNSET. + */ + private transient int[] predecessor; + + /** + * Pointer to the successor of an entry in insertion order. ENDPOINT indicates a node is the last + * node in insertion order; all values at indices ≥ {@link #size()} are UNSET. + */ + private transient int[] successor; + + private transient int firstEntry; + private transient int lastEntry; + + CompactLinkedHashSet() { + super(); + } + + CompactLinkedHashSet(int expectedSize) { + super(expectedSize); + } + + @Override + void init(int expectedSize, float loadFactor) { + super.init(expectedSize, loadFactor); + this.predecessor = new int[expectedSize]; + this.successor = new int[expectedSize]; + + Arrays.fill(predecessor, UNSET); + Arrays.fill(successor, UNSET); + firstEntry = ENDPOINT; + lastEntry = ENDPOINT; + } + + private void succeeds(int pred, int succ) { + if (pred == ENDPOINT) { + firstEntry = succ; + } else { + successor[pred] = succ; + } + + if (succ == ENDPOINT) { + lastEntry = pred; + } else { + predecessor[succ] = pred; + } + } + + @Override + void insertEntry(int entryIndex, E object, int hash) { + super.insertEntry(entryIndex, object, hash); + succeeds(lastEntry, entryIndex); + succeeds(entryIndex, ENDPOINT); + } + + @Override + void moveEntry(int dstIndex) { + int srcIndex = size() - 1; + super.moveEntry(dstIndex); + + succeeds(predecessor[dstIndex], successor[dstIndex]); + if (srcIndex != dstIndex) { + succeeds(predecessor[srcIndex], dstIndex); + succeeds(dstIndex, successor[srcIndex]); + } + predecessor[srcIndex] = UNSET; + successor[srcIndex] = UNSET; + } + + @Override + public void clear() { + super.clear(); + firstEntry = ENDPOINT; + lastEntry = ENDPOINT; + Arrays.fill(predecessor, UNSET); + Arrays.fill(successor, UNSET); + } + + @Override + void resizeEntries(int newCapacity) { + super.resizeEntries(newCapacity); + int oldCapacity = predecessor.length; + predecessor = Arrays.copyOf(predecessor, newCapacity); + successor = Arrays.copyOf(successor, newCapacity); + + if (oldCapacity < newCapacity) { + Arrays.fill(predecessor, oldCapacity, newCapacity, UNSET); + Arrays.fill(successor, oldCapacity, newCapacity, UNSET); + } + } + + @Override + public Object[] toArray() { + return ObjectArrays.toArrayImpl(this); + } + + @Override + public T[] toArray(T[] a) { + return ObjectArrays.toArrayImpl(this, a); + } + + @Override + int firstEntryIndex() { + return firstEntry; + } + + @Override + int adjustAfterRemove(int indexBeforeRemove, int indexRemoved) { + return (indexBeforeRemove == size()) ? indexRemoved : indexBeforeRemove; + } + + @Override + int getSuccessor(int entryIndex) { + return successor[entryIndex]; + } + + @Override public Spliterator spliterator() { + return Spliterators.spliterator(this, Spliterator.ORDERED | Spliterator.DISTINCT); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override public void forEach(Consumer action) { + checkNotNull(action); + for (int i = firstEntry; i != ENDPOINT; i = successor[i]) { + action.accept((E) elements[i]); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparatorOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparatorOrdering.java new file mode 100644 index 0000000000000..eba46d6189cb9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparatorOrdering.java @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Comparator; + + +/** An ordering for a pre-existing comparator. */ +final class ComparatorOrdering extends Ordering implements Serializable { + final Comparator comparator; + + ComparatorOrdering(Comparator comparator) { + this.comparator = checkNotNull(comparator); + } + + @Override + public int compare(T a, T b) { + return comparator.compare(a, b); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof ComparatorOrdering) { + ComparatorOrdering that = (ComparatorOrdering) object; + return this.comparator.equals(that.comparator); + } + return false; + } + + @Override + public int hashCode() { + return comparator.hashCode(); + } + + @Override + public String toString() { + return comparator.toString(); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Comparators.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Comparators.java new file mode 100644 index 0000000000000..b40f7d927446b --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Comparators.java @@ -0,0 +1,193 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collector; + +/** + * Provides static methods for working with {@link Comparator} instances. For many other helpful + * comparator utilities, see either {@code Comparator} itself (for Java 8 or later), or {@code + * com.google.common.collect.Ordering} (otherwise). + * + *

Relationship to {@code Ordering}

+ * + *

In light of the significant enhancements to {@code Comparator} in Java 8, the overwhelming + * majority of usages of {@code Ordering} can be written using only built-in JDK APIs. This class is + * intended to "fill the gap" and provide those features of {@code Ordering} not already provided by + * the JDK. + * + * @since 21.0 + * @author Louis Wasserman + */ + +public final class Comparators { + private Comparators() {} + + /** + * Returns a new comparator which sorts iterables by comparing corresponding elements pairwise + * until a nonzero result is found; imposes "dictionary order." If the end of one iterable is + * reached, but not the other, the shorter iterable is considered to be less than the longer one. + * For example, a lexicographical natural ordering over integers considers {@code [] < [1] < [1, + * 1] < [1, 2] < [2]}. + * + *

Note that {@code Collections.reverseOrder(lexicographical(comparator))} is not equivalent to + * {@code lexicographical(Collections.reverseOrder(comparator))} (consider how each would order + * {@code [1]} and {@code [1, 1]}). + */ + // Note: 90% of the time we don't add type parameters or wildcards that serve only to "tweak" the + // desired return type. However, *nested* generics introduce a special class of problems that we + // think tip it over into being worthwhile. + public static Comparator> lexicographical(Comparator comparator) { + return new LexicographicalOrdering(checkNotNull(comparator)); + } + + /** + * Returns {@code true} if each element in {@code iterable} after the first is greater than or + * equal to the element that preceded it, according to the specified comparator. Note that this is + * always true when the iterable has fewer than two elements. + */ + public static boolean isInOrder(Iterable iterable, Comparator comparator) { + checkNotNull(comparator); + Iterator it = iterable.iterator(); + if (it.hasNext()) { + T prev = it.next(); + while (it.hasNext()) { + T next = it.next(); + if (comparator.compare(prev, next) > 0) { + return false; + } + prev = next; + } + } + return true; + } + + /** + * Returns {@code true} if each element in {@code iterable} after the first is strictly + * greater than the element that preceded it, according to the specified comparator. Note that + * this is always true when the iterable has fewer than two elements. + */ + public static boolean isInStrictOrder( + Iterable iterable, Comparator comparator) { + checkNotNull(comparator); + Iterator it = iterable.iterator(); + if (it.hasNext()) { + T prev = it.next(); + while (it.hasNext()) { + T next = it.next(); + if (comparator.compare(prev, next) >= 0) { + return false; + } + prev = next; + } + } + return true; + } + + /** + * Returns a {@code Collector} that returns the {@code k} smallest (relative to the specified + * {@code Comparator}) input elements, in ascending order, as an unmodifiable {@code List}. Ties + * are broken arbitrarily. + * + *

For example: + * + *

{@code
+   * Stream.of("foo", "quux", "banana", "elephant")
+   *     .collect(least(2, comparingInt(String::length)))
+   * // returns {"foo", "quux"}
+   * }
+ * + *

This {@code Collector} uses O(k) memory and takes expected time O(n) (worst-case O(n log + * k)), as opposed to e.g. {@code Stream.sorted(comparator).limit(k)}, which currently takes O(n + * log n) time and O(n) space. + * + * @throws IllegalArgumentException if {@code k < 0} + * @since 22.0 + */ + public static Collector> least(int k, Comparator comparator) { + checkNonnegative(k, "k"); + checkNotNull(comparator); + return Collector.of( + () -> TopKSelector.least(k, comparator), + TopKSelector::offer, + TopKSelector::combine, + TopKSelector::topK, + Collector.Characteristics.UNORDERED); + } + + /** + * Returns a {@code Collector} that returns the {@code k} greatest (relative to the specified + * {@code Comparator}) input elements, in descending order, as an unmodifiable {@code List}. Ties + * are broken arbitrarily. + * + *

For example: + * + *

{@code
+   * Stream.of("foo", "quux", "banana", "elephant")
+   *     .collect(greatest(2, comparingInt(String::length)))
+   * // returns {"elephant", "banana"}
+   * }
+ * + *

This {@code Collector} uses O(k) memory and takes expected time O(n) (worst-case O(n log + * k)), as opposed to e.g. {@code Stream.sorted(comparator.reversed()).limit(k)}, which currently + * takes O(n log n) time and O(n) space. + * + * @throws IllegalArgumentException if {@code k < 0} + * @since 22.0 + */ + public static Collector> greatest(int k, Comparator comparator) { + return least(k, comparator.reversed()); + } + + /** + * Returns a comparator of {@link Optional} values which treats {@link Optional#empty} as less + * than all other values, and orders the rest using {@code valueComparator} on the contained + * value. + * + * @since 22.0 + */ + + public static Comparator> emptiesFirst(Comparator valueComparator) { + checkNotNull(valueComparator); + return Comparator.comparing(o -> o.orElse(null), Comparator.nullsFirst(valueComparator)); + } + + /** + * Returns a comparator of {@link Optional} values which treats {@link Optional#empty} as greater + * than all other values, and orders the rest using {@code valueComparator} on the contained + * value. + * + * @since 22.0 + */ + + public static Comparator> emptiesLast(Comparator valueComparator) { + checkNotNull(valueComparator); + return Comparator.comparing(o -> o.orElse(null), Comparator.nullsLast(valueComparator)); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparisonChain.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparisonChain.java new file mode 100644 index 0000000000000..83cba31ca3eaf --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComparisonChain.java @@ -0,0 +1,253 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Booleans; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Longs; +import java.util.Comparator; + + +/** + * A utility for performing a chained comparison statement. For example: + * + *

{@code
+ * public int compareTo(Foo that) {
+ *   return ComparisonChain.start()
+ *       .compare(this.aString, that.aString)
+ *       .compare(this.anInt, that.anInt)
+ *       .compare(this.anEnum, that.anEnum, Ordering.natural().nullsLast())
+ *       .result();
+ * }
+ * }
+ * + *

The value of this expression will have the same sign as the first nonzero comparison + * result in the chain, or will be zero if every comparison result was zero. + * + *

Note: {@code ComparisonChain} instances are immutable. For this utility to work + * correctly, calls must be chained as illustrated above. + * + *

Performance note: Even though the {@code ComparisonChain} caller always invokes its {@code + * compare} methods unconditionally, the {@code ComparisonChain} implementation stops calling its + * inputs' {@link Comparable#compareTo compareTo} and {@link Comparator#compare compare} methods as + * soon as one of them returns a nonzero result. This optimization is typically important only in + * the presence of expensive {@code compareTo} and {@code compare} implementations. + * + *

See the Guava User Guide article on {@code + * ComparisonChain}. + * + * @author Mark Davis + * @author Kevin Bourrillion + * @since 2.0 + */ +public abstract class ComparisonChain { + private ComparisonChain() {} + + /** Begins a new chained comparison statement. See example in the class documentation. */ + public static ComparisonChain start() { + return ACTIVE; + } + + private static final ComparisonChain ACTIVE = + new ComparisonChain() { + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public ComparisonChain compare(Comparable left, Comparable right) { + return classify(left.compareTo(right)); + } + + @Override + public ComparisonChain compare( + T left, T right, Comparator comparator) { + return classify(comparator.compare(left, right)); + } + + @Override + public ComparisonChain compare(int left, int right) { + return classify(Ints.compare(left, right)); + } + + @Override + public ComparisonChain compare(long left, long right) { + return classify(Longs.compare(left, right)); + } + + @Override + public ComparisonChain compare(float left, float right) { + return classify(Float.compare(left, right)); + } + + @Override + public ComparisonChain compare(double left, double right) { + return classify(Double.compare(left, right)); + } + + @Override + public ComparisonChain compareTrueFirst(boolean left, boolean right) { + return classify(Booleans.compare(right, left)); // reversed + } + + @Override + public ComparisonChain compareFalseFirst(boolean left, boolean right) { + return classify(Booleans.compare(left, right)); + } + + ComparisonChain classify(int result) { + return (result < 0) ? LESS : (result > 0) ? GREATER : ACTIVE; + } + + @Override + public int result() { + return 0; + } + }; + + private static final ComparisonChain LESS = new InactiveComparisonChain(-1); + + private static final ComparisonChain GREATER = new InactiveComparisonChain(1); + + private static final class InactiveComparisonChain extends ComparisonChain { + final int result; + + InactiveComparisonChain(int result) { + this.result = result; + } + + @SuppressWarnings("rawtypes") + @Override + public ComparisonChain compare(Comparable left, Comparable right) { + return this; + } + + @Override + public ComparisonChain compare( + T left, T right, Comparator comparator) { + return this; + } + + @Override + public ComparisonChain compare(int left, int right) { + return this; + } + + @Override + public ComparisonChain compare(long left, long right) { + return this; + } + + @Override + public ComparisonChain compare(float left, float right) { + return this; + } + + @Override + public ComparisonChain compare(double left, double right) { + return this; + } + + @Override + public ComparisonChain compareTrueFirst(boolean left, boolean right) { + return this; + } + + @Override + public ComparisonChain compareFalseFirst(boolean left, boolean right) { + return this; + } + + @Override + public int result() { + return result; + } + } + + /** + * Compares two comparable objects as specified by {@link Comparable#compareTo}, if the + * result of this comparison chain has not already been determined. + */ + public abstract ComparisonChain compare(Comparable left, Comparable right); + + /** + * Compares two objects using a comparator, if the result of this comparison chain has not + * already been determined. + */ + public abstract ComparisonChain compare( + T left, T right, Comparator comparator); + + /** + * Compares two {@code int} values as specified by {@link Ints#compare}, if the result of + * this comparison chain has not already been determined. + */ + public abstract ComparisonChain compare(int left, int right); + + /** + * Compares two {@code long} values as specified by {@link Longs#compare}, if the result of + * this comparison chain has not already been determined. + */ + public abstract ComparisonChain compare(long left, long right); + + /** + * Compares two {@code float} values as specified by {@link Float#compare}, if the result + * of this comparison chain has not already been determined. + */ + public abstract ComparisonChain compare(float left, float right); + + /** + * Compares two {@code double} values as specified by {@link Double#compare}, if the result + * of this comparison chain has not already been determined. + */ + public abstract ComparisonChain compare(double left, double right); + + /** + * Discouraged synonym for {@link #compareFalseFirst}. + * + * @deprecated Use {@link #compareFalseFirst}; or, if the parameters passed are being either + * negated or reversed, undo the negation or reversal and use {@link #compareTrueFirst}. + * @since 19.0 + */ + @Deprecated + public final ComparisonChain compare(Boolean left, Boolean right) { + return compareFalseFirst(left, right); + } + + /** + * Compares two {@code boolean} values, considering {@code true} to be less than {@code false}, + * if the result of this comparison chain has not already been determined. + * + * @since 12.0 + */ + public abstract ComparisonChain compareTrueFirst(boolean left, boolean right); + + /** + * Compares two {@code boolean} values, considering {@code false} to be less than {@code true}, + * if the result of this comparison chain has not already been determined. + * + * @since 12.0 (present as {@code compare} since 2.0) + */ + public abstract ComparisonChain compareFalseFirst(boolean left, boolean right); + + /** + * Ends this comparison chain and returns its result: a value having the same sign as the first + * nonzero comparison result in the chain, or zero if every result was zero. + */ + public abstract int result(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompoundOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompoundOrdering.java new file mode 100644 index 0000000000000..108739e189d25 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/CompoundOrdering.java @@ -0,0 +1,75 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Comparator; + +/** An ordering that tries several comparators in order. */ +final class CompoundOrdering extends Ordering implements Serializable { + final Comparator[] comparators; + + @SuppressWarnings({"unchecked", "rawtypes"}) + CompoundOrdering(Comparator primary, Comparator secondary) { + this.comparators = (Comparator[]) new Comparator[] {primary, secondary}; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + CompoundOrdering(Iterable> comparators) { + this.comparators = Iterables.toArray(comparators, new Comparator[0]); + } + + @Override + public int compare(T left, T right) { + for (int i = 0; i < comparators.length; i++) { + int result = comparators[i].compare(left, right); + if (result != 0) { + return result; + } + } + return 0; + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof CompoundOrdering) { + CompoundOrdering that = (CompoundOrdering) object; + return Arrays.equals(this.comparators, that.comparators); + } + return false; + } + + @Override + public int hashCode() { + return Arrays.hashCode(comparators); + } + + @Override + public String toString() { + return "Ordering.compound(" + Arrays.toString(comparators) + ")"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComputationException.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComputationException.java new file mode 100644 index 0000000000000..fe4d0e2a73aaa --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ComputationException.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + + +/** + * Wraps an exception that occurred during a computation. + * + * @author Bob Lee + * @since 2.0 + */ +public class ComputationException extends RuntimeException { + /** Creates a new instance with the given cause. */ + public ComputationException(Throwable cause) { + super(cause); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConcurrentHashMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConcurrentHashMultiset.java new file mode 100644 index 0000000000000..6b28b9d218362 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConcurrentHashMultiset.java @@ -0,0 +1,596 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Serialization.FieldSetter; +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + + +/** + * A multiset that supports concurrent modifications and that provides atomic versions of most + * {@code Multiset} operations (exceptions where noted). Null elements are not supported. + * + *

See the Guava User Guide article on {@code + * Multiset}. + * + * @author Cliff L. Biffle + * @author mike nonemacher + * @since 2.0 + */ +public final class ConcurrentHashMultiset extends AbstractMultiset implements Serializable { + + /* + * The ConcurrentHashMultiset's atomic operations are implemented primarily in terms of + * AtomicInteger's atomic operations, with some help from ConcurrentMap's atomic operations on + * creation and removal (including automatic removal of zeroes). If the modification of an + * AtomicInteger results in zero, we compareAndSet the value to zero; if that succeeds, we remove + * the entry from the Map. If another operation sees a zero in the map, it knows that the entry is + * about to be removed, so this operation may remove it (often by replacing it with a new + * AtomicInteger). + */ + + /** The number of occurrences of each element. */ + private final transient ConcurrentMap countMap; + + // This constant allows the deserialization code to set a final field. This holder class + // makes sure it is not initialized unless an instance is deserialized. + private static class FieldSettersHolder { + @SuppressWarnings("rawtypes") + static final FieldSetter COUNT_MAP_FIELD_SETTER = + Serialization.getFieldSetter(ConcurrentHashMultiset.class, "countMap"); + } + + /** + * Creates a new, empty {@code ConcurrentHashMultiset} using the default initial capacity, load + * factor, and concurrency settings. + */ + public static ConcurrentHashMultiset create() { + // TODO(schmoe): provide a way to use this class with other (possibly arbitrary) + // ConcurrentMap implementors. One possibility is to extract most of this class into + // an AbstractConcurrentMapMultiset. + return new ConcurrentHashMultiset(new ConcurrentHashMap()); + } + + /** + * Creates a new {@code ConcurrentHashMultiset} containing the specified elements, using the + * default initial capacity, load factor, and concurrency settings. + * + *

This implementation is highly efficient when {@code elements} is itself a {@link Multiset}. + * + * @param elements the elements that the multiset should contain + */ + public static ConcurrentHashMultiset create(Iterable elements) { + ConcurrentHashMultiset multiset = ConcurrentHashMultiset.create(); + Iterables.addAll(multiset, elements); + return multiset; + } + + /** + * Creates a new, empty {@code ConcurrentHashMultiset} using {@code countMap} as the internal + * backing map. + * + *

This instance will assume ownership of {@code countMap}, and other code should not maintain + * references to the map or modify it in any way. + * + *

The returned multiset is serializable if the input map is. + * + * @param countMap backing map for storing the elements in the multiset and their counts. It must + * be empty. + * @throws IllegalArgumentException if {@code countMap} is not empty + * @since 20.0 + */ + + public static ConcurrentHashMultiset create(ConcurrentMap countMap) { + return new ConcurrentHashMultiset(countMap); + } + + + ConcurrentHashMultiset(ConcurrentMap countMap) { + checkArgument(countMap.isEmpty(), "the backing map (%s) must be empty", countMap); + this.countMap = countMap; + } + + // Query Operations + + /** + * Returns the number of occurrences of {@code element} in this multiset. + * + * @param element the element to look for + * @return the nonnegative number of occurrences of the element + */ + @Override + public int count(Object element) { + AtomicInteger existingCounter = Maps.safeGet(countMap, element); + return (existingCounter == null) ? 0 : existingCounter.get(); + } + + /** + * {@inheritDoc} + * + *

If the data in the multiset is modified by any other threads during this method, it is + * undefined which (if any) of these modifications will be reflected in the result. + */ + @Override + public int size() { + long sum = 0L; + for (AtomicInteger value : countMap.values()) { + sum += value.get(); + } + return Ints.saturatedCast(sum); + } + + /* + * Note: the superclass toArray() methods assume that size() gives a correct + * answer, which ours does not. + */ + + @Override + public Object[] toArray() { + return snapshot().toArray(); + } + + @Override + public T[] toArray(T[] array) { + return snapshot().toArray(array); + } + + /* + * We'd love to use 'new ArrayList(this)' or 'list.addAll(this)', but + * either of these would recurse back to us again! + */ + private List snapshot() { + List list = Lists.newArrayListWithExpectedSize(size()); + for (Entry entry : entrySet()) { + E element = entry.getElement(); + for (int i = entry.getCount(); i > 0; i--) { + list.add(element); + } + } + return list; + } + + // Modification Operations + + /** + * Adds a number of occurrences of the specified element to this multiset. + * + * @param element the element to add + * @param occurrences the number of occurrences to add + * @return the previous count of the element before the operation; possibly zero + * @throws IllegalArgumentException if {@code occurrences} is negative, or if the resulting amount + * would exceed {@link Integer#MAX_VALUE} + */ + @Override + public int add(E element, int occurrences) { + checkNotNull(element); + if (occurrences == 0) { + return count(element); + } + CollectPreconditions.checkPositive(occurrences, "occurences"); + + while (true) { + AtomicInteger existingCounter = Maps.safeGet(countMap, element); + if (existingCounter == null) { + existingCounter = countMap.putIfAbsent(element, new AtomicInteger(occurrences)); + if (existingCounter == null) { + return 0; + } + // existingCounter != null: fall through to operate against the existing AtomicInteger + } + + while (true) { + int oldValue = existingCounter.get(); + if (oldValue != 0) { + try { + int newValue = IntMath.checkedAdd(oldValue, occurrences); + if (existingCounter.compareAndSet(oldValue, newValue)) { + // newValue can't == 0, so no need to check & remove + return oldValue; + } + } catch (ArithmeticException overflow) { + throw new IllegalArgumentException( + "Overflow adding " + occurrences + " occurrences to a count of " + oldValue); + } + } else { + // In the case of a concurrent remove, we might observe a zero value, which means another + // thread is about to remove (element, existingCounter) from the map. Rather than wait, + // we can just do that work here. + AtomicInteger newCounter = new AtomicInteger(occurrences); + if ((countMap.putIfAbsent(element, newCounter) == null) + || countMap.replace(element, existingCounter, newCounter)) { + return 0; + } + break; + } + } + + // If we're still here, there was a race, so just try again. + } + } + + /** + * Removes a number of occurrences of the specified element from this multiset. If the multiset + * contains fewer than this number of occurrences to begin with, all occurrences will be removed. + * + * @param element the element whose occurrences should be removed + * @param occurrences the number of occurrences of the element to remove + * @return the count of the element before the operation; possibly zero + * @throws IllegalArgumentException if {@code occurrences} is negative + */ + /* + * TODO(cpovirk): remove and removeExactly currently accept null inputs only + * if occurrences == 0. This satisfies both NullPointerTester and + * CollectionRemoveTester.testRemove_nullAllowed, but it's not clear that it's + * a good policy, especially because, in order for the test to pass, the + * parameter must be misleadingly annotated as . I suspect that + * we'll want to remove , add an eager checkNotNull, and loosen up + * testRemove_nullAllowed. + */ + @Override + public int remove(Object element, int occurrences) { + if (occurrences == 0) { + return count(element); + } + CollectPreconditions.checkPositive(occurrences, "occurences"); + + AtomicInteger existingCounter = Maps.safeGet(countMap, element); + if (existingCounter == null) { + return 0; + } + while (true) { + int oldValue = existingCounter.get(); + if (oldValue != 0) { + int newValue = Math.max(0, oldValue - occurrences); + if (existingCounter.compareAndSet(oldValue, newValue)) { + if (newValue == 0) { + // Just CASed to 0; remove the entry to clean up the map. If the removal fails, + // another thread has already replaced it with a new counter, which is fine. + countMap.remove(element, existingCounter); + } + return oldValue; + } + } else { + return 0; + } + } + } + + /** + * Removes exactly the specified number of occurrences of {@code element}, or makes no change if + * this is not possible. + * + *

This method, in contrast to {@link #remove(Object, int)}, has no effect when the element + * count is smaller than {@code occurrences}. + * + * @param element the element to remove + * @param occurrences the number of occurrences of {@code element} to remove + * @return {@code true} if the removal was possible (including if {@code occurrences} is zero) + * @throws IllegalArgumentException if {@code occurrences} is negative + */ + public boolean removeExactly(Object element, int occurrences) { + if (occurrences == 0) { + return true; + } + CollectPreconditions.checkPositive(occurrences, "occurences"); + + AtomicInteger existingCounter = Maps.safeGet(countMap, element); + if (existingCounter == null) { + return false; + } + while (true) { + int oldValue = existingCounter.get(); + if (oldValue < occurrences) { + return false; + } + int newValue = oldValue - occurrences; + if (existingCounter.compareAndSet(oldValue, newValue)) { + if (newValue == 0) { + // Just CASed to 0; remove the entry to clean up the map. If the removal fails, + // another thread has already replaced it with a new counter, which is fine. + countMap.remove(element, existingCounter); + } + return true; + } + } + } + + /** + * Adds or removes occurrences of {@code element} such that the {@link #count} of the element + * becomes {@code count}. + * + * @return the count of {@code element} in the multiset before this call + * @throws IllegalArgumentException if {@code count} is negative + */ + @Override + public int setCount(E element, int count) { + checkNotNull(element); + checkNonnegative(count, "count"); + while (true) { + AtomicInteger existingCounter = Maps.safeGet(countMap, element); + if (existingCounter == null) { + if (count == 0) { + return 0; + } else { + existingCounter = countMap.putIfAbsent(element, new AtomicInteger(count)); + if (existingCounter == null) { + return 0; + } + // existingCounter != null: fall through + } + } + + while (true) { + int oldValue = existingCounter.get(); + if (oldValue == 0) { + if (count == 0) { + return 0; + } else { + AtomicInteger newCounter = new AtomicInteger(count); + if ((countMap.putIfAbsent(element, newCounter) == null) + || countMap.replace(element, existingCounter, newCounter)) { + return 0; + } + } + break; + } else { + if (existingCounter.compareAndSet(oldValue, count)) { + if (count == 0) { + // Just CASed to 0; remove the entry to clean up the map. If the removal fails, + // another thread has already replaced it with a new counter, which is fine. + countMap.remove(element, existingCounter); + } + return oldValue; + } + } + } + } + } + + /** + * Sets the number of occurrences of {@code element} to {@code newCount}, but only if the count is + * currently {@code expectedOldCount}. If {@code element} does not appear in the multiset exactly + * {@code expectedOldCount} times, no changes will be made. + * + * @return {@code true} if the change was successful. This usually indicates that the multiset has + * been modified, but not always: in the case that {@code expectedOldCount == newCount}, the + * method will return {@code true} if the condition was met. + * @throws IllegalArgumentException if {@code expectedOldCount} or {@code newCount} is negative + */ + @Override + public boolean setCount(E element, int expectedOldCount, int newCount) { + checkNotNull(element); + checkNonnegative(expectedOldCount, "oldCount"); + checkNonnegative(newCount, "newCount"); + + AtomicInteger existingCounter = Maps.safeGet(countMap, element); + if (existingCounter == null) { + if (expectedOldCount != 0) { + return false; + } else if (newCount == 0) { + return true; + } else { + // if our write lost the race, it must have lost to a nonzero value, so we can stop + return countMap.putIfAbsent(element, new AtomicInteger(newCount)) == null; + } + } + int oldValue = existingCounter.get(); + if (oldValue == expectedOldCount) { + if (oldValue == 0) { + if (newCount == 0) { + // Just observed a 0; try to remove the entry to clean up the map + countMap.remove(element, existingCounter); + return true; + } else { + AtomicInteger newCounter = new AtomicInteger(newCount); + return (countMap.putIfAbsent(element, newCounter) == null) + || countMap.replace(element, existingCounter, newCounter); + } + } else { + if (existingCounter.compareAndSet(oldValue, newCount)) { + if (newCount == 0) { + // Just CASed to 0; remove the entry to clean up the map. If the removal fails, + // another thread has already replaced it with a new counter, which is fine. + countMap.remove(element, existingCounter); + } + return true; + } + } + } + return false; + } + + // Views + + @Override + Set createElementSet() { + final Set delegate = countMap.keySet(); + return new ForwardingSet() { + @Override + protected Set delegate() { + return delegate; + } + + @Override + public boolean contains(Object object) { + return object != null && Collections2.safeContains(delegate, object); + } + + @Override + public boolean containsAll(Collection collection) { + return standardContainsAll(collection); + } + + @Override + public boolean remove(Object object) { + return object != null && Collections2.safeRemove(delegate, object); + } + + @Override + public boolean removeAll(Collection c) { + return standardRemoveAll(c); + } + }; + } + + @Override + Iterator elementIterator() { + throw new AssertionError("should never be called"); + } + + /** @deprecated Internal method, use {@link #entrySet()}. */ + @Deprecated + @Override + public Set> createEntrySet() { + return new EntrySet(); + } + + @Override + int distinctElements() { + return countMap.size(); + } + + @Override + public boolean isEmpty() { + return countMap.isEmpty(); + } + + @Override + Iterator> entryIterator() { + // AbstractIterator makes this fairly clean, but it doesn't support remove(). To support + // remove(), we create an AbstractIterator, and then use ForwardingIterator to delegate to it. + final Iterator> readOnlyIterator = + new AbstractIterator>() { + private final Iterator> mapEntries = + countMap.entrySet().iterator(); + + @Override + protected Entry computeNext() { + while (true) { + if (!mapEntries.hasNext()) { + return endOfData(); + } + Map.Entry mapEntry = mapEntries.next(); + int count = mapEntry.getValue().get(); + if (count != 0) { + return Multisets.immutableEntry(mapEntry.getKey(), count); + } + } + } + }; + + return new ForwardingIterator>() { + private Entry last; + + @Override + protected Iterator> delegate() { + return readOnlyIterator; + } + + @Override + public Entry next() { + last = super.next(); + return last; + } + + @Override + public void remove() { + checkRemove(last != null); + ConcurrentHashMultiset.this.setCount(last.getElement(), 0); + last = null; + } + }; + } + + @Override + public Iterator iterator() { + return Multisets.iteratorImpl(this); + } + + @Override + public void clear() { + countMap.clear(); + } + + private class EntrySet extends AbstractMultiset.EntrySet { + @Override + ConcurrentHashMultiset multiset() { + return ConcurrentHashMultiset.this; + } + + /* + * Note: the superclass toArray() methods assume that size() gives a correct + * answer, which ours does not. + */ + + @Override + public Object[] toArray() { + return snapshot().toArray(); + } + + @Override + public T[] toArray(T[] array) { + return snapshot().toArray(array); + } + + private List> snapshot() { + List> list = Lists.newArrayListWithExpectedSize(size()); + // Not Iterables.addAll(list, this), because that'll forward right back here. + Iterators.addAll(list, iterator()); + return list; + } + } + + /** @serialData the ConcurrentMap of elements and their counts. */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(countMap); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + ConcurrentMap deserializedCountMap = + (ConcurrentMap) stream.readObject(); + FieldSettersHolder.COUNT_MAP_FIELD_SETTER.set(this, deserializedCountMap); + } + + private static final long serialVersionUID = 1; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConsumingQueueIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConsumingQueueIterator.java new file mode 100644 index 0000000000000..a5c8c3832f369 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ConsumingQueueIterator.java @@ -0,0 +1,49 @@ +/* + * Copyright (C) 2015 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.Queue; + +/** + * An Iterator implementation which draws elements from a queue, removing them from the queue as it + * iterates. + */ +class ConsumingQueueIterator extends AbstractIterator { + private final Queue queue; + + @SafeVarargs + @SuppressWarnings("varargs") + ConsumingQueueIterator(T... elements) { + this.queue = new ArrayDeque(elements.length); + Collections.addAll(queue, elements); + } + + ConsumingQueueIterator(Queue queue) { + this.queue = checkNotNull(queue); + } + + @Override + public T computeNext() { + return queue.isEmpty() ? endOfData() : queue.remove(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ContiguousSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ContiguousSet.java new file mode 100644 index 0000000000000..7e816483dd9a4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ContiguousSet.java @@ -0,0 +1,255 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import java.util.Collections; +import java.util.NoSuchElementException; +import java.util.Set; + +/** + * A sorted set of contiguous values in a given {@link DiscreteDomain}. Example: + * + *

{@code
+ * ContiguousSet.create(Range.closed(5, 42), DiscreteDomain.integers())
+ * }
+ * + *

Note that because bounded ranges over {@code int} and {@code long} values are so common, this + * particular example can be written as just: + * + *

{@code
+ * ContiguousSet.closed(5, 42)
+ * }
+ * + *

Warning: Be extremely careful what you do with conceptually large instances (such as + * {@code ContiguousSet.create(Range.greaterThan(0), DiscreteDomain.integers()}). Certain operations + * on such a set can be performed efficiently, but others (such as {@link Set#hashCode} or {@link + * Collections#frequency}) can cause major performance problems. + * + * @author Gregory Kick + * @since 10.0 + */ +@SuppressWarnings("rawtypes") // allow ungenerified Comparable types +public abstract class ContiguousSet extends ImmutableSortedSet { + /** + * Returns a {@code ContiguousSet} containing the same values in the given domain {@linkplain + * Range#contains contained} by the range. + * + * @throws IllegalArgumentException if neither range nor the domain has a lower bound, or if + * neither has an upper bound + * @since 13.0 + */ + public static ContiguousSet create( + Range range, DiscreteDomain domain) { + checkNotNull(range); + checkNotNull(domain); + Range effectiveRange = range; + try { + if (!range.hasLowerBound()) { + effectiveRange = effectiveRange.intersection(Range.atLeast(domain.minValue())); + } + if (!range.hasUpperBound()) { + effectiveRange = effectiveRange.intersection(Range.atMost(domain.maxValue())); + } + } catch (NoSuchElementException e) { + throw new IllegalArgumentException(e); + } + + // Per class spec, we are allowed to throw CCE if necessary + boolean empty = + effectiveRange.isEmpty() + || Range.compareOrThrow( + range.lowerBound.leastValueAbove(domain), + range.upperBound.greatestValueBelow(domain)) + > 0; + + return empty + ? new EmptyContiguousSet(domain) + : new RegularContiguousSet(effectiveRange, domain); + } + + /** + * Returns a nonempty contiguous set containing all {@code int} values from {@code lower} + * (inclusive) to {@code upper} (inclusive). (These are the same values contained in {@code + * Range.closed(lower, upper)}.) + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 23.0 + */ + + public static ContiguousSet closed(int lower, int upper) { + return create(Range.closed(lower, upper), DiscreteDomain.integers()); + } + + /** + * Returns a nonempty contiguous set containing all {@code long} values from {@code lower} + * (inclusive) to {@code upper} (inclusive). (These are the same values contained in {@code + * Range.closed(lower, upper)}.) + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 23.0 + */ + + public static ContiguousSet closed(long lower, long upper) { + return create(Range.closed(lower, upper), DiscreteDomain.longs()); + } + + /** + * Returns a contiguous set containing all {@code int} values from {@code lower} (inclusive) to + * {@code upper} (exclusive). If the endpoints are equal, an empty set is returned. (These are the + * same values contained in {@code Range.closedOpen(lower, upper)}.) + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 23.0 + */ + + public static ContiguousSet closedOpen(int lower, int upper) { + return create(Range.closedOpen(lower, upper), DiscreteDomain.integers()); + } + + /** + * Returns a contiguous set containing all {@code long} values from {@code lower} (inclusive) to + * {@code upper} (exclusive). If the endpoints are equal, an empty set is returned. (These are the + * same values contained in {@code Range.closedOpen(lower, upper)}.) + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 23.0 + */ + + public static ContiguousSet closedOpen(long lower, long upper) { + return create(Range.closedOpen(lower, upper), DiscreteDomain.longs()); + } + + final DiscreteDomain domain; + + ContiguousSet(DiscreteDomain domain) { + super(Ordering.natural()); + this.domain = domain; + } + + @Override + public ContiguousSet headSet(C toElement) { + return headSetImpl(checkNotNull(toElement), false); + } + + /** @since 12.0 */ + @Override + public ContiguousSet headSet(C toElement, boolean inclusive) { + return headSetImpl(checkNotNull(toElement), inclusive); + } + + @Override + public ContiguousSet subSet(C fromElement, C toElement) { + checkNotNull(fromElement); + checkNotNull(toElement); + checkArgument(comparator().compare(fromElement, toElement) <= 0); + return subSetImpl(fromElement, true, toElement, false); + } + + /** @since 12.0 */ + @Override + public ContiguousSet subSet( + C fromElement, boolean fromInclusive, C toElement, boolean toInclusive) { + checkNotNull(fromElement); + checkNotNull(toElement); + checkArgument(comparator().compare(fromElement, toElement) <= 0); + return subSetImpl(fromElement, fromInclusive, toElement, toInclusive); + } + + @Override + public ContiguousSet tailSet(C fromElement) { + return tailSetImpl(checkNotNull(fromElement), true); + } + + /** @since 12.0 */ + @Override + public ContiguousSet tailSet(C fromElement, boolean inclusive) { + return tailSetImpl(checkNotNull(fromElement), inclusive); + } + + /* + * These methods perform most headSet, subSet, and tailSet logic, besides parameter validation. + */ + // TODO(kevinb): we can probably make these real @Overrides now + /* @Override */ + abstract ContiguousSet headSetImpl(C toElement, boolean inclusive); + + /* @Override */ + abstract ContiguousSet subSetImpl( + C fromElement, boolean fromInclusive, C toElement, boolean toInclusive); + + /* @Override */ + abstract ContiguousSet tailSetImpl(C fromElement, boolean inclusive); + + /** + * Returns the set of values that are contained in both this set and the other. + * + *

This method should always be used instead of {@link Sets#intersection} for {@link + * ContiguousSet} instances. + */ + public abstract ContiguousSet intersection(ContiguousSet other); + + /** + * Returns a range, closed on both ends, whose endpoints are the minimum and maximum values + * contained in this set. This is equivalent to {@code range(CLOSED, CLOSED)}. + * + * @throws NoSuchElementException if this set is empty + */ + public abstract Range range(); + + /** + * Returns the minimal range with the given boundary types for which all values in this set are + * {@linkplain Range#contains(Comparable) contained} within the range. + * + *

Note that this method will return ranges with unbounded endpoints if {@link BoundType#OPEN} + * is requested for a domain minimum or maximum. For example, if {@code set} was created from the + * range {@code [1..Integer.MAX_VALUE]} then {@code set.range(CLOSED, OPEN)} must return {@code + * [1..∞)}. + * + * @throws NoSuchElementException if this set is empty + */ + public abstract Range range(BoundType lowerBoundType, BoundType upperBoundType); + + @Override + ImmutableSortedSet createDescendingSet() { + return new DescendingImmutableSortedSet(this); + } + + /** Returns a short-hand representation of the contents such as {@code "[1..100]"}. */ + @Override + public String toString() { + return range().toString(); + } + + /** + * Not supported. {@code ContiguousSet} instances are constructed with {@link #create}. This + * method exists only to hide {@link ImmutableSet#builder} from consumers of {@code + * ContiguousSet}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link #create}. + */ + @Deprecated + public static Builder builder() { + throw new UnsupportedOperationException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Count.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Count.java new file mode 100644 index 0000000000000..d6e8e5fcc9a5e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Count.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + + +/** + * A mutable value of type {@code int}, for multisets to use in tracking counts of values. + * + * @author Louis Wasserman + */ +final class Count implements Serializable { + private int value; + + Count(int value) { + this.value = value; + } + + public int get() { + return value; + } + + public void add(int delta) { + value += delta; + } + + public int addAndGet(int delta) { + return value += delta; + } + + public void set(int newValue) { + value = newValue; + } + + public int getAndSet(int newValue) { + int result = value; + value = newValue; + return result; + } + + @Override + public int hashCode() { + return value; + } + + @Override + public boolean equals(Object obj) { + return obj instanceof Count && ((Count) obj).value == value; + } + + @Override + public String toString() { + return Integer.toString(value); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Cut.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Cut.java new file mode 100644 index 0000000000000..d170ca05739f6 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Cut.java @@ -0,0 +1,477 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Booleans; +import java.io.Serializable; +import java.util.NoSuchElementException; + + +/** + * Implementation detail for the internal structure of {@link Range} instances. Represents a unique + * way of "cutting" a "number line" (actually of instances of type {@code C}, not necessarily + * "numbers") into two sections; this can be done below a certain value, above a certain value, + * below all values or above all values. With this object defined in this way, an interval can + * always be represented by a pair of {@code Cut} instances. + * + * @author Kevin Bourrillion + */ +@SuppressWarnings("rawtypes") +abstract class Cut implements Comparable>, Serializable { + final C endpoint; + + Cut(C endpoint) { + this.endpoint = endpoint; + } + + abstract boolean isLessThan(C value); + + abstract BoundType typeAsLowerBound(); + + abstract BoundType typeAsUpperBound(); + + abstract Cut withLowerBoundType(BoundType boundType, DiscreteDomain domain); + + abstract Cut withUpperBoundType(BoundType boundType, DiscreteDomain domain); + + abstract void describeAsLowerBound(StringBuilder sb); + + abstract void describeAsUpperBound(StringBuilder sb); + + abstract C leastValueAbove(DiscreteDomain domain); + + abstract C greatestValueBelow(DiscreteDomain domain); + + /* + * The canonical form is a BelowValue cut whenever possible, otherwise ABOVE_ALL, or + * (only in the case of types that are unbounded below) BELOW_ALL. + */ + Cut canonical(DiscreteDomain domain) { + return this; + } + + // note: overridden by {BELOW,ABOVE}_ALL + @Override + public int compareTo(Cut that) { + if (that == belowAll()) { + return 1; + } + if (that == aboveAll()) { + return -1; + } + int result = Range.compareOrThrow(endpoint, that.endpoint); + if (result != 0) { + return result; + } + // same value. below comes before above + return Booleans.compare(this instanceof AboveValue, that instanceof AboveValue); + } + + C endpoint() { + return endpoint; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // catching CCE + @Override + public boolean equals(Object obj) { + if (obj instanceof Cut) { + // It might not really be a Cut, but we'll catch a CCE if it's not + Cut that = (Cut) obj; + try { + int compareResult = compareTo(that); + return compareResult == 0; + } catch (ClassCastException ignored) { + } + } + return false; + } + + // Prevent "missing hashCode" warning by explicitly forcing subclasses implement it + @Override + public abstract int hashCode(); + + /* + * The implementation neither produces nor consumes any non-null instance of type C, so + * casting the type parameter is safe. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + static Cut belowAll() { + return (Cut) BelowAll.INSTANCE; + } + + private static final long serialVersionUID = 0; + + private static final class BelowAll extends Cut> { + private static final BelowAll INSTANCE = new BelowAll(); + + private BelowAll() { + super(null); + } + + @Override + Comparable endpoint() { + throw new IllegalStateException("range unbounded on this side"); + } + + @Override + boolean isLessThan(Comparable value) { + return true; + } + + @Override + BoundType typeAsLowerBound() { + throw new IllegalStateException(); + } + + @Override + BoundType typeAsUpperBound() { + throw new AssertionError("this statement should be unreachable"); + } + + @Override + Cut> withLowerBoundType( + BoundType boundType, DiscreteDomain> domain) { + throw new IllegalStateException(); + } + + @Override + Cut> withUpperBoundType( + BoundType boundType, DiscreteDomain> domain) { + throw new AssertionError("this statement should be unreachable"); + } + + @Override + void describeAsLowerBound(StringBuilder sb) { + sb.append("(-\u221e"); + } + + @Override + void describeAsUpperBound(StringBuilder sb) { + throw new AssertionError(); + } + + @Override + Comparable leastValueAbove(DiscreteDomain> domain) { + return domain.minValue(); + } + + @Override + Comparable greatestValueBelow(DiscreteDomain> domain) { + throw new AssertionError(); + } + + @Override + Cut> canonical(DiscreteDomain> domain) { + try { + return Cut.>belowValue(domain.minValue()); + } catch (NoSuchElementException e) { + return this; + } + } + + @Override + public int compareTo(Cut> o) { + return (o == this) ? 0 : -1; + } + + @Override + public int hashCode() { + return System.identityHashCode(this); + } + + @Override + public String toString() { + return "-\u221e"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 0; + } + + /* + * The implementation neither produces nor consumes any non-null instance of + * type C, so casting the type parameter is safe. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + static Cut aboveAll() { + return (Cut) AboveAll.INSTANCE; + } + + private static final class AboveAll extends Cut> { + private static final AboveAll INSTANCE = new AboveAll(); + + private AboveAll() { + super(null); + } + + @Override + Comparable endpoint() { + throw new IllegalStateException("range unbounded on this side"); + } + + @Override + boolean isLessThan(Comparable value) { + return false; + } + + @Override + BoundType typeAsLowerBound() { + throw new AssertionError("this statement should be unreachable"); + } + + @Override + BoundType typeAsUpperBound() { + throw new IllegalStateException(); + } + + @Override + Cut> withLowerBoundType( + BoundType boundType, DiscreteDomain> domain) { + throw new AssertionError("this statement should be unreachable"); + } + + @Override + Cut> withUpperBoundType( + BoundType boundType, DiscreteDomain> domain) { + throw new IllegalStateException(); + } + + @Override + void describeAsLowerBound(StringBuilder sb) { + throw new AssertionError(); + } + + @Override + void describeAsUpperBound(StringBuilder sb) { + sb.append("+\u221e)"); + } + + @Override + Comparable leastValueAbove(DiscreteDomain> domain) { + throw new AssertionError(); + } + + @Override + Comparable greatestValueBelow(DiscreteDomain> domain) { + return domain.maxValue(); + } + + @Override + public int compareTo(Cut> o) { + return (o == this) ? 0 : 1; + } + + @Override + public int hashCode() { + return System.identityHashCode(this); + } + + @Override + public String toString() { + return "+\u221e"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 0; + } + + @SuppressWarnings("rawtypes") + static Cut belowValue(C endpoint) { + return new BelowValue(endpoint); + } + + @SuppressWarnings("rawtypes") + private static final class BelowValue extends Cut { + BelowValue(C endpoint) { + super(checkNotNull(endpoint)); + } + + @Override + boolean isLessThan(C value) { + return Range.compareOrThrow(endpoint, value) <= 0; + } + + @Override + BoundType typeAsLowerBound() { + return BoundType.CLOSED; + } + + @Override + BoundType typeAsUpperBound() { + return BoundType.OPEN; + } + + @Override + Cut withLowerBoundType(BoundType boundType, DiscreteDomain domain) { + switch (boundType) { + case CLOSED: + return this; + case OPEN: + C previous = domain.previous(endpoint); + return (previous == null) ? Cut.belowAll() : new AboveValue(previous); + default: + throw new AssertionError(); + } + } + + @Override + Cut withUpperBoundType(BoundType boundType, DiscreteDomain domain) { + switch (boundType) { + case CLOSED: + C previous = domain.previous(endpoint); + return (previous == null) ? Cut.aboveAll() : new AboveValue(previous); + case OPEN: + return this; + default: + throw new AssertionError(); + } + } + + @Override + void describeAsLowerBound(StringBuilder sb) { + sb.append('[').append(endpoint); + } + + @Override + void describeAsUpperBound(StringBuilder sb) { + sb.append(endpoint).append(')'); + } + + @Override + C leastValueAbove(DiscreteDomain domain) { + return endpoint; + } + + @Override + C greatestValueBelow(DiscreteDomain domain) { + return domain.previous(endpoint); + } + + @Override + public int hashCode() { + return endpoint.hashCode(); + } + + @Override + public String toString() { + return "\\" + endpoint + "/"; + } + + private static final long serialVersionUID = 0; + } + + @SuppressWarnings("rawtypes") + static Cut aboveValue(C endpoint) { + return new AboveValue(endpoint); + } + + @SuppressWarnings("rawtypes") + private static final class AboveValue extends Cut { + AboveValue(C endpoint) { + super(checkNotNull(endpoint)); + } + + @Override + boolean isLessThan(C value) { + return Range.compareOrThrow(endpoint, value) < 0; + } + + @Override + BoundType typeAsLowerBound() { + return BoundType.OPEN; + } + + @Override + BoundType typeAsUpperBound() { + return BoundType.CLOSED; + } + + @Override + Cut withLowerBoundType(BoundType boundType, DiscreteDomain domain) { + switch (boundType) { + case OPEN: + return this; + case CLOSED: + C next = domain.next(endpoint); + return (next == null) ? Cut.belowAll() : belowValue(next); + default: + throw new AssertionError(); + } + } + + @Override + Cut withUpperBoundType(BoundType boundType, DiscreteDomain domain) { + switch (boundType) { + case OPEN: + C next = domain.next(endpoint); + return (next == null) ? Cut.aboveAll() : belowValue(next); + case CLOSED: + return this; + default: + throw new AssertionError(); + } + } + + @Override + void describeAsLowerBound(StringBuilder sb) { + sb.append('(').append(endpoint); + } + + @Override + void describeAsUpperBound(StringBuilder sb) { + sb.append(endpoint).append(']'); + } + + @Override + C leastValueAbove(DiscreteDomain domain) { + return domain.next(endpoint); + } + + @Override + C greatestValueBelow(DiscreteDomain domain) { + return endpoint; + } + + @Override + Cut canonical(DiscreteDomain domain) { + C next = leastValueAbove(domain); + return (next != null) ? belowValue(next) : Cut.aboveAll(); + } + + @Override + public int hashCode() { + return ~endpoint.hashCode(); + } + + @Override + public String toString() { + return "/" + endpoint + "\\"; + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedMultiset.java new file mode 100644 index 0000000000000..53776a6fe4a8e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedMultiset.java @@ -0,0 +1,84 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * A descending wrapper around an {@code ImmutableSortedMultiset} + * + * @author Louis Wasserman + */ +@SuppressWarnings("serial") // uses writeReplace, not default serialization +final class DescendingImmutableSortedMultiset extends ImmutableSortedMultiset { + private final transient ImmutableSortedMultiset forward; + + DescendingImmutableSortedMultiset(ImmutableSortedMultiset forward) { + this.forward = forward; + } + + @Override + public int count(Object element) { + return forward.count(element); + } + + @Override + public Entry firstEntry() { + return forward.lastEntry(); + } + + @Override + public Entry lastEntry() { + return forward.firstEntry(); + } + + @Override + public int size() { + return forward.size(); + } + + @Override + public ImmutableSortedSet elementSet() { + return forward.elementSet().descendingSet(); + } + + @Override + Entry getEntry(int index) { + return forward.entrySet().asList().reverse().get(index); + } + + @Override + public ImmutableSortedMultiset descendingMultiset() { + return forward; + } + + @Override + public ImmutableSortedMultiset headMultiset(E upperBound, BoundType boundType) { + return forward.tailMultiset(upperBound, boundType).descendingMultiset(); + } + + @Override + public ImmutableSortedMultiset tailMultiset(E lowerBound, BoundType boundType) { + return forward.headMultiset(lowerBound, boundType).descendingMultiset(); + } + + @Override + boolean isPartialView() { + return forward.isPartialView(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedSet.java new file mode 100644 index 0000000000000..f95f49420e8a4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingImmutableSortedSet.java @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Skeletal implementation of {@link ImmutableSortedSet#descendingSet()}. + * + * @author Louis Wasserman + */ +final class DescendingImmutableSortedSet extends ImmutableSortedSet { + private final ImmutableSortedSet forward; + + DescendingImmutableSortedSet(ImmutableSortedSet forward) { + super(Ordering.from(forward.comparator()).reverse()); + this.forward = forward; + } + + @Override + public boolean contains(Object object) { + return forward.contains(object); + } + + @Override + public int size() { + return forward.size(); + } + + @Override + public UnmodifiableIterator iterator() { + return forward.descendingIterator(); + } + + @Override + ImmutableSortedSet headSetImpl(E toElement, boolean inclusive) { + return forward.tailSet(toElement, inclusive).descendingSet(); + } + + @Override + ImmutableSortedSet subSetImpl( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return forward.subSet(toElement, toInclusive, fromElement, fromInclusive).descendingSet(); + } + + @Override + ImmutableSortedSet tailSetImpl(E fromElement, boolean inclusive) { + return forward.headSet(fromElement, inclusive).descendingSet(); + } + + @Override + public ImmutableSortedSet descendingSet() { + return forward; + } + + @Override + public UnmodifiableIterator descendingIterator() { + return forward.iterator(); + } + + @Override + ImmutableSortedSet createDescendingSet() { + throw new AssertionError("should never be called"); + } + + @Override + public E lower(E element) { + return forward.higher(element); + } + + @Override + public E floor(E element) { + return forward.ceiling(element); + } + + @Override + public E ceiling(E element) { + return forward.floor(element); + } + + @Override + public E higher(E element) { + return forward.lower(element); + } + + @Override + int indexOf(Object target) { + int index = forward.indexOf(target); + if (index == -1) { + return index; + } else { + return size() - 1 - index; + } + } + + @Override + boolean isPartialView() { + return forward.isPartialView(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingMultiset.java new file mode 100644 index 0000000000000..7ba6a91ad1913 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DescendingMultiset.java @@ -0,0 +1,157 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.Set; + + +/** + * A skeleton implementation of a descending multiset. Only needs {@code forwardMultiset()} and + * {@code entryIterator()}. + * + * @author Louis Wasserman + */ +abstract class DescendingMultiset extends ForwardingMultiset implements SortedMultiset { + abstract SortedMultiset forwardMultiset(); + + private transient Comparator comparator; + + @Override + public Comparator comparator() { + Comparator result = comparator; + if (result == null) { + return comparator = Ordering.from(forwardMultiset().comparator()).reverse(); + } + return result; + } + + private transient NavigableSet elementSet; + + @Override + public NavigableSet elementSet() { + NavigableSet result = elementSet; + if (result == null) { + return elementSet = new SortedMultisets.NavigableElementSet(this); + } + return result; + } + + @Override + public Entry pollFirstEntry() { + return forwardMultiset().pollLastEntry(); + } + + @Override + public Entry pollLastEntry() { + return forwardMultiset().pollFirstEntry(); + } + + @Override + public SortedMultiset headMultiset(E toElement, BoundType boundType) { + return forwardMultiset().tailMultiset(toElement, boundType).descendingMultiset(); + } + + @Override + public SortedMultiset subMultiset( + E fromElement, BoundType fromBoundType, E toElement, BoundType toBoundType) { + return forwardMultiset() + .subMultiset(toElement, toBoundType, fromElement, fromBoundType) + .descendingMultiset(); + } + + @Override + public SortedMultiset tailMultiset(E fromElement, BoundType boundType) { + return forwardMultiset().headMultiset(fromElement, boundType).descendingMultiset(); + } + + @Override + protected Multiset delegate() { + return forwardMultiset(); + } + + @Override + public SortedMultiset descendingMultiset() { + return forwardMultiset(); + } + + @Override + public Entry firstEntry() { + return forwardMultiset().lastEntry(); + } + + @Override + public Entry lastEntry() { + return forwardMultiset().firstEntry(); + } + + abstract Iterator> entryIterator(); + + private transient Set> entrySet; + + @Override + public Set> entrySet() { + Set> result = entrySet; + return (result == null) ? entrySet = createEntrySet() : result; + } + + Set> createEntrySet() { + class EntrySetImpl extends Multisets.EntrySet { + @Override + Multiset multiset() { + return DescendingMultiset.this; + } + + @Override + public Iterator> iterator() { + return entryIterator(); + } + + @Override + public int size() { + return forwardMultiset().entrySet().size(); + } + } + return new EntrySetImpl(); + } + + @Override + public Iterator iterator() { + return Multisets.iteratorImpl(this); + } + + @Override + public Object[] toArray() { + return standardToArray(); + } + + @Override + public T[] toArray(T[] array) { + return standardToArray(array); + } + + @Override + public String toString() { + return entrySet().toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DiscreteDomain.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DiscreteDomain.java new file mode 100644 index 0000000000000..0f3a4017efa08 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/DiscreteDomain.java @@ -0,0 +1,322 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.Serializable; +import java.math.BigInteger; +import java.util.NoSuchElementException; + +/** + * A descriptor for a discrete {@code Comparable} domain such as all {@link Integer} + * instances. A discrete domain is one that supports the three basic operations: {@link #next}, + * {@link #previous} and {@link #distance}, according to their specifications. The methods {@link + * #minValue} and {@link #maxValue} should also be overridden for bounded types. + * + *

A discrete domain always represents the entire set of values of its type; it cannot + * represent partial domains such as "prime integers" or "strings of length 5." + * + *

See the Guava User Guide section on {@code + * DiscreteDomain}. + * + * @author Kevin Bourrillion + * @since 10.0 + */ +@SuppressWarnings("rawtypes") +public abstract class DiscreteDomain { + + /** + * Returns the discrete domain for values of type {@code Integer}. + * + * @since 14.0 (since 10.0 as {@code DiscreteDomains.integers()}) + */ + public static DiscreteDomain integers() { + return IntegerDomain.INSTANCE; + } + + private static final class IntegerDomain extends DiscreteDomain implements Serializable { + private static final IntegerDomain INSTANCE = new IntegerDomain(); + + IntegerDomain() { + super(true); + } + + @Override + public Integer next(Integer value) { + int i = value; + return (i == Integer.MAX_VALUE) ? null : i + 1; + } + + @Override + public Integer previous(Integer value) { + int i = value; + return (i == Integer.MIN_VALUE) ? null : i - 1; + } + + @Override + Integer offset(Integer origin, long distance) { + checkNonnegative(distance, "distance"); + return Ints.checkedCast(origin.longValue() + distance); + } + + @Override + public long distance(Integer start, Integer end) { + return (long) end - start; + } + + @Override + public Integer minValue() { + return Integer.MIN_VALUE; + } + + @Override + public Integer maxValue() { + return Integer.MAX_VALUE; + } + + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "DiscreteDomain.integers()"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns the discrete domain for values of type {@code Long}. + * + * @since 14.0 (since 10.0 as {@code DiscreteDomains.longs()}) + */ + public static DiscreteDomain longs() { + return LongDomain.INSTANCE; + } + + private static final class LongDomain extends DiscreteDomain implements Serializable { + private static final LongDomain INSTANCE = new LongDomain(); + + LongDomain() { + super(true); + } + + @Override + public Long next(Long value) { + long l = value; + return (l == Long.MAX_VALUE) ? null : l + 1; + } + + @Override + public Long previous(Long value) { + long l = value; + return (l == Long.MIN_VALUE) ? null : l - 1; + } + + @Override + Long offset(Long origin, long distance) { + checkNonnegative(distance, "distance"); + long result = origin + distance; + if (result < 0) { + checkArgument(origin < 0, "overflow"); + } + return result; + } + + @Override + public long distance(Long start, Long end) { + long result = end - start; + if (end > start && result < 0) { // overflow + return Long.MAX_VALUE; + } + if (end < start && result > 0) { // underflow + return Long.MIN_VALUE; + } + return result; + } + + @Override + public Long minValue() { + return Long.MIN_VALUE; + } + + @Override + public Long maxValue() { + return Long.MAX_VALUE; + } + + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "DiscreteDomain.longs()"; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns the discrete domain for values of type {@code BigInteger}. + * + * @since 15.0 + */ + public static DiscreteDomain bigIntegers() { + return BigIntegerDomain.INSTANCE; + } + + private static final class BigIntegerDomain extends DiscreteDomain + implements Serializable { + private static final BigIntegerDomain INSTANCE = new BigIntegerDomain(); + + BigIntegerDomain() { + super(true); + } + + private static final BigInteger MIN_LONG = BigInteger.valueOf(Long.MIN_VALUE); + private static final BigInteger MAX_LONG = BigInteger.valueOf(Long.MAX_VALUE); + + @Override + public BigInteger next(BigInteger value) { + return value.add(BigInteger.ONE); + } + + @Override + public BigInteger previous(BigInteger value) { + return value.subtract(BigInteger.ONE); + } + + @Override + BigInteger offset(BigInteger origin, long distance) { + checkNonnegative(distance, "distance"); + return origin.add(BigInteger.valueOf(distance)); + } + + @Override + public long distance(BigInteger start, BigInteger end) { + return end.subtract(start).max(MIN_LONG).min(MAX_LONG).longValue(); + } + + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "DiscreteDomain.bigIntegers()"; + } + + private static final long serialVersionUID = 0; + } + + final boolean supportsFastOffset; + + /** Constructor for use by subclasses. */ + protected DiscreteDomain() { + this(false); + } + + /** Private constructor for built-in DiscreteDomains supporting fast offset. */ + private DiscreteDomain(boolean supportsFastOffset) { + this.supportsFastOffset = supportsFastOffset; + } + + /** + * Returns, conceptually, "origin + distance", or equivalently, the result of calling {@link + * #next} on {@code origin} {@code distance} times. + */ + C offset(C origin, long distance) { + checkNonnegative(distance, "distance"); + for (long i = 0; i < distance; i++) { + origin = next(origin); + } + return origin; + } + + /** + * Returns the unique least value of type {@code C} that is greater than {@code value}, or {@code + * null} if none exists. Inverse operation to {@link #previous}. + * + * @param value any value of type {@code C} + * @return the least value greater than {@code value}, or {@code null} if {@code value} is {@code + * maxValue()} + */ + public abstract C next(C value); + + /** + * Returns the unique greatest value of type {@code C} that is less than {@code value}, or {@code + * null} if none exists. Inverse operation to {@link #next}. + * + * @param value any value of type {@code C} + * @return the greatest value less than {@code value}, or {@code null} if {@code value} is {@code + * minValue()} + */ + public abstract C previous(C value); + + /** + * Returns a signed value indicating how many nested invocations of {@link #next} (if positive) or + * {@link #previous} (if negative) are needed to reach {@code end} starting from {@code start}. + * For example, if {@code end = next(next(next(start)))}, then {@code distance(start, end) == 3} + * and {@code distance(end, start) == -3}. As well, {@code distance(a, a)} is always zero. + * + *

Note that this function is necessarily well-defined for any discrete type. + * + * @return the distance as described above, or {@link Long#MIN_VALUE} or {@link Long#MAX_VALUE} if + * the distance is too small or too large, respectively. + */ + public abstract long distance(C start, C end); + + /** + * Returns the minimum value of type {@code C}, if it has one. The minimum value is the unique + * value for which {@link Comparable#compareTo(Object)} never returns a positive value for any + * input of type {@code C}. + * + *

The default implementation throws {@code NoSuchElementException}. + * + * @return the minimum value of type {@code C}; never null + * @throws NoSuchElementException if the type has no (practical) minimum value; for example, + * {@link BigInteger} + */ + public C minValue() { + throw new NoSuchElementException(); + } + + /** + * Returns the maximum value of type {@code C}, if it has one. The maximum value is the unique + * value for which {@link Comparable#compareTo(Object)} never returns a negative value for any + * input of type {@code C}. + * + *

The default implementation throws {@code NoSuchElementException}. + * + * @return the maximum value of type {@code C}; never null + * @throws NoSuchElementException if the type has no (practical) maximum value; for example, + * {@link BigInteger} + */ + public C maxValue() { + throw new NoSuchElementException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyContiguousSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyContiguousSet.java new file mode 100644 index 0000000000000..fe1f21278d924 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyContiguousSet.java @@ -0,0 +1,163 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.NoSuchElementException; +import java.util.Set; + + +/** + * An empty contiguous set. + * + * @author Gregory Kick + */ +@SuppressWarnings({"unchecked", "rawtypes"}) // allow ungenerified Comparable types +final class EmptyContiguousSet extends ContiguousSet { + EmptyContiguousSet(DiscreteDomain domain) { + super(domain); + } + + @Override + public C first() { + throw new NoSuchElementException(); + } + + @Override + public C last() { + throw new NoSuchElementException(); + } + + @Override + public int size() { + return 0; + } + + @Override + public ContiguousSet intersection(ContiguousSet other) { + return this; + } + + @Override + public Range range() { + throw new NoSuchElementException(); + } + + @Override + public Range range(BoundType lowerBoundType, BoundType upperBoundType) { + throw new NoSuchElementException(); + } + + @Override + ContiguousSet headSetImpl(C toElement, boolean inclusive) { + return this; + } + + @Override + ContiguousSet subSetImpl( + C fromElement, boolean fromInclusive, C toElement, boolean toInclusive) { + return this; + } + + @Override + ContiguousSet tailSetImpl(C fromElement, boolean fromInclusive) { + return this; + } + + @Override + public boolean contains(Object object) { + return false; + } + + @Override + int indexOf(Object target) { + return -1; + } + + @Override + public UnmodifiableIterator iterator() { + return Iterators.emptyIterator(); + } + + @Override + public UnmodifiableIterator descendingIterator() { + return Iterators.emptyIterator(); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public boolean isEmpty() { + return true; + } + + @Override + public ImmutableList asList() { + return ImmutableList.of(); + } + + @Override + public String toString() { + return "[]"; + } + + @Override + public boolean equals(Object object) { + if (object instanceof Set) { + Set that = (Set) object; + return that.isEmpty(); + } + return false; + } + + @Override + boolean isHashCodeFast() { + return true; + } + + @Override + public int hashCode() { + return 0; + } + + private static final class SerializedForm implements Serializable { + private final DiscreteDomain domain; + + private SerializedForm(DiscreteDomain domain) { + this.domain = domain; + } + + private Object readResolve() { + return new EmptyContiguousSet(domain); + } + + private static final long serialVersionUID = 0; + } + + @Override + Object writeReplace() { + return new SerializedForm(domain); + } + + ImmutableSortedSet createDescendingSet() { + return ImmutableSortedSet.emptySet(Ordering.natural().reverse()); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableListMultimap.java new file mode 100644 index 0000000000000..eaf992cb7dcc0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableListMultimap.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Implementation of {@link ImmutableListMultimap} with no entries. + * + * @author Jared Levy + */ +class EmptyImmutableListMultimap extends ImmutableListMultimap { + static final EmptyImmutableListMultimap INSTANCE = new EmptyImmutableListMultimap(); + + private EmptyImmutableListMultimap() { + super(ImmutableMap.>of(), 0); + } + + private Object readResolve() { + return INSTANCE; // preserve singleton property + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableSetMultimap.java new file mode 100644 index 0000000000000..844a0a0804d7c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EmptyImmutableSetMultimap.java @@ -0,0 +1,41 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Implementation of {@link ImmutableListMultimap} with no entries. + * + * @author Mike Ward + */ +public class EmptyImmutableSetMultimap extends ImmutableSetMultimap { + static final EmptyImmutableSetMultimap INSTANCE = new EmptyImmutableSetMultimap(); + + private EmptyImmutableSetMultimap() { + super(ImmutableMap.>of(), 0, null); + } + + private Object readResolve() { + return INSTANCE; // preserve singleton property + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumBiMap.java new file mode 100644 index 0000000000000..1e5f2694f119f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumBiMap.java @@ -0,0 +1,144 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.EnumMap; +import java.util.Map; + +/** + * A {@code BiMap} backed by two {@code EnumMap} instances. Null keys and values are not permitted. + * An {@code EnumBiMap} and its inverse are both serializable. + * + *

See the Guava User Guide article on {@code BiMap}. + * + * @author Mike Bostock + * @since 2.0 + */ +public final class EnumBiMap, V extends Enum> extends AbstractBiMap { + private transient Class keyType; + private transient Class valueType; + + /** + * Returns a new, empty {@code EnumBiMap} using the specified key and value types. + * + * @param keyType the key type + * @param valueType the value type + */ + public static , V extends Enum> EnumBiMap create( + Class keyType, Class valueType) { + return new EnumBiMap<>(keyType, valueType); + } + + /** + * Returns a new bimap with the same mappings as the specified map. If the specified map is an + * {@code EnumBiMap}, the new bimap has the same types as the provided map. Otherwise, the + * specified map must contain at least one mapping, in order to determine the key and value types. + * + * @param map the map whose mappings are to be placed in this map + * @throws IllegalArgumentException if map is not an {@code EnumBiMap} instance and contains no + * mappings + */ + public static , V extends Enum> EnumBiMap create(Map map) { + EnumBiMap bimap = create(inferKeyType(map), inferValueType(map)); + bimap.putAll(map); + return bimap; + } + + private EnumBiMap(Class keyType, Class valueType) { + super( + WellBehavedMap.wrap(new EnumMap(keyType)), + WellBehavedMap.wrap(new EnumMap(valueType))); + this.keyType = keyType; + this.valueType = valueType; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static > Class inferKeyType(Map map) { + if (map instanceof EnumBiMap) { + return ((EnumBiMap) map).keyType(); + } + if (map instanceof EnumHashBiMap) { + return ((EnumHashBiMap) map).keyType(); + } + checkArgument(!map.isEmpty()); + return map.keySet().iterator().next().getDeclaringClass(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private static > Class inferValueType(Map map) { + if (map instanceof EnumBiMap) { + return ((EnumBiMap) map).valueType; + } + checkArgument(!map.isEmpty()); + return map.values().iterator().next().getDeclaringClass(); + } + + /** Returns the associated key type. */ + public Class keyType() { + return keyType; + } + + /** Returns the associated value type. */ + public Class valueType() { + return valueType; + } + + @Override + K checkKey(K key) { + return checkNotNull(key); + } + + @Override + V checkValue(V value) { + return checkNotNull(value); + } + + /** + * @serialData the key class, value class, number of entries, first key, first value, second key, + * second value, and so on. + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(keyType); + stream.writeObject(valueType); + Serialization.writeMap(this, stream); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading fields populated by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + keyType = (Class) stream.readObject(); + valueType = (Class) stream.readObject(); + setDelegates( + WellBehavedMap.wrap(new EnumMap(keyType)), + WellBehavedMap.wrap(new EnumMap(valueType))); + Serialization.populateMap(this, stream); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumHashBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumHashBiMap.java new file mode 100644 index 0000000000000..0f01fff03d836 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumHashBiMap.java @@ -0,0 +1,122 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.Map; + + +/** + * A {@code BiMap} backed by an {@code EnumMap} instance for keys-to-values, and a {@code HashMap} + * instance for values-to-keys. Null keys are not permitted, but null values are. An {@code + * EnumHashBiMap} and its inverse are both serializable. + * + *

See the Guava User Guide article on {@code BiMap}. + * + * @author Mike Bostock + * @since 2.0 + */ +public final class EnumHashBiMap, V> extends AbstractBiMap { + private transient Class keyType; + + /** + * Returns a new, empty {@code EnumHashBiMap} using the specified key type. + * + * @param keyType the key type + */ + public static , V> EnumHashBiMap create(Class keyType) { + return new EnumHashBiMap<>(keyType); + } + + /** + * Constructs a new bimap with the same mappings as the specified map. If the specified map is an + * {@code EnumHashBiMap} or an {@link EnumBiMap}, the new bimap has the same key type as the input + * bimap. Otherwise, the specified map must contain at least one mapping, in order to determine + * the key type. + * + * @param map the map whose mappings are to be placed in this map + * @throws IllegalArgumentException if map is not an {@code EnumBiMap} or an {@code EnumHashBiMap} + * instance and contains no mappings + */ + public static , V> EnumHashBiMap create(Map map) { + EnumHashBiMap bimap = create(EnumBiMap.inferKeyType(map)); + bimap.putAll(map); + return bimap; + } + + private EnumHashBiMap(Class keyType) { + super( + WellBehavedMap.wrap(new EnumMap(keyType)), + Maps.newHashMapWithExpectedSize(keyType.getEnumConstants().length)); + this.keyType = keyType; + } + + // Overriding these 3 methods to show that values may be null (but not keys) + + @Override + K checkKey(K key) { + return checkNotNull(key); + } + + @Override + public V put(K key, V value) { + return super.put(key, value); + } + + @Override + public V forcePut(K key, V value) { + return super.forcePut(key, value); + } + + /** Returns the associated key type. */ + public Class keyType() { + return keyType; + } + + /** + * @serialData the key class, number of entries, first key, first value, second key, second value, + * and so on. + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(keyType); + Serialization.writeMap(this, stream); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading field populated by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + keyType = (Class) stream.readObject(); + setDelegates( + WellBehavedMap.wrap(new EnumMap(keyType)), + new HashMap(keyType.getEnumConstants().length * 3 / 2)); + Serialization.populateMap(this, stream); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumMultiset.java new file mode 100644 index 0000000000000..60e6ef6b48650 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EnumMultiset.java @@ -0,0 +1,312 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.ObjIntConsumer; + + +/** + * Multiset implementation specialized for enum elements, supporting all single-element operations + * in O(1). + * + *

See the Guava User Guide article on {@code + * Multiset}. + * + * @author Jared Levy + * @since 2.0 + */ +public final class EnumMultiset> extends AbstractMultiset + implements Serializable { + /** Creates an empty {@code EnumMultiset}. */ + public static > EnumMultiset create(Class type) { + return new EnumMultiset(type); + } + + /** + * Creates a new {@code EnumMultiset} containing the specified elements. + * + *

This implementation is highly efficient when {@code elements} is itself a {@link Multiset}. + * + * @param elements the elements that the multiset should contain + * @throws IllegalArgumentException if {@code elements} is empty + */ + public static > EnumMultiset create(Iterable elements) { + Iterator iterator = elements.iterator(); + checkArgument(iterator.hasNext(), "EnumMultiset constructor passed empty Iterable"); + EnumMultiset multiset = new EnumMultiset<>(iterator.next().getDeclaringClass()); + Iterables.addAll(multiset, elements); + return multiset; + } + + /** + * Returns a new {@code EnumMultiset} instance containing the given elements. Unlike {@link + * EnumMultiset#create(Iterable)}, this method does not produce an exception on an empty iterable. + * + * @since 14.0 + */ + public static > EnumMultiset create(Iterable elements, Class type) { + EnumMultiset result = create(type); + Iterables.addAll(result, elements); + return result; + } + + private transient Class type; + private transient E[] enumConstants; + private transient int[] counts; + private transient int distinctElements; + private transient long size; + + /** Creates an empty {@code EnumMultiset}. */ + private EnumMultiset(Class type) { + this.type = type; + checkArgument(type.isEnum()); + this.enumConstants = type.getEnumConstants(); + this.counts = new int[enumConstants.length]; + } + + private boolean isActuallyE(Object o) { + if (o instanceof Enum) { + Enum e = (Enum) o; + int index = e.ordinal(); + return index < enumConstants.length && enumConstants[index] == e; + } + return false; + } + + /** + * Returns {@code element} cast to {@code E}, if it actually is a nonnull E. Otherwise, throws + * either a NullPointerException or a ClassCastException as appropriate. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + void checkIsE(Object element) { + checkNotNull(element); + if (!isActuallyE(element)) { + throw new ClassCastException("Expected an " + type + " but got " + element); + } + } + + @Override + int distinctElements() { + return distinctElements; + } + + @Override + public int size() { + return Ints.saturatedCast(size); + } + + @Override + public int count(Object element) { + if (element == null || !isActuallyE(element)) { + return 0; + } + Enum e = (Enum) element; + return counts[e.ordinal()]; + } + + // Modification Operations + @Override + public int add(E element, int occurrences) { + checkIsE(element); + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(element); + } + int index = element.ordinal(); + int oldCount = counts[index]; + long newCount = (long) oldCount + occurrences; + checkArgument(newCount <= Integer.MAX_VALUE, "too many occurrences: %s", newCount); + counts[index] = (int) newCount; + if (oldCount == 0) { + distinctElements++; + } + size += occurrences; + return oldCount; + } + + // Modification Operations + @Override + public int remove(Object element, int occurrences) { + if (element == null || !isActuallyE(element)) { + return 0; + } + Enum e = (Enum) element; + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(element); + } + int index = e.ordinal(); + int oldCount = counts[index]; + if (oldCount == 0) { + return 0; + } else if (oldCount <= occurrences) { + counts[index] = 0; + distinctElements--; + size -= oldCount; + } else { + counts[index] = oldCount - occurrences; + size -= occurrences; + } + return oldCount; + } + + // Modification Operations + @Override + public int setCount(E element, int count) { + checkIsE(element); + checkNonnegative(count, "count"); + int index = element.ordinal(); + int oldCount = counts[index]; + counts[index] = count; + size += count - oldCount; + if (oldCount == 0 && count > 0) { + distinctElements++; + } else if (oldCount > 0 && count == 0) { + distinctElements--; + } + return oldCount; + } + + @Override + public void clear() { + Arrays.fill(counts, 0); + size = 0; + distinctElements = 0; + } + + abstract class Itr implements Iterator { + int index = 0; + int toRemove = -1; + + abstract T output(int index); + + @Override + public boolean hasNext() { + for (; index < enumConstants.length; index++) { + if (counts[index] > 0) { + return true; + } + } + return false; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + T result = output(index); + toRemove = index; + index++; + return result; + } + + @Override + public void remove() { + checkRemove(toRemove >= 0); + if (counts[toRemove] > 0) { + distinctElements--; + size -= counts[toRemove]; + counts[toRemove] = 0; + } + toRemove = -1; + } + } + + @Override + Iterator elementIterator() { + return new Itr() { + @Override + E output(int index) { + return enumConstants[index]; + } + }; + } + + @Override + Iterator> entryIterator() { + return new Itr>() { + @Override + Entry output(final int index) { + return new Multisets.AbstractEntry() { + @Override + public E getElement() { + return enumConstants[index]; + } + + @Override + public int getCount() { + return counts[index]; + } + }; + } + }; + } + + @Override + public void forEachEntry(ObjIntConsumer action) { + checkNotNull(action); + for (int i = 0; i < enumConstants.length; i++) { + if (counts[i] > 0) { + action.accept(enumConstants[i], counts[i]); + } + } + } + + @Override + public Iterator iterator() { + return Multisets.iteratorImpl(this); + } + + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(type); + Serialization.writeMultiset(this, stream); + } + + /** + * @serialData the {@code Class} for the enum type, the number of distinct elements, the first + * element, its count, the second element, its count, and so on + */ + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + Class localType = (Class) stream.readObject(); + type = localType; + enumConstants = type.getEnumConstants(); + counts = new int[enumConstants.length]; + Serialization.populateMultiset(this, stream); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EvictingQueue.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EvictingQueue.java new file mode 100644 index 0000000000000..21354e411ec32 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/EvictingQueue.java @@ -0,0 +1,139 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + + +import java.io.Serializable; +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Queue; + +/** + * A non-blocking queue which automatically evicts elements from the head of the queue when + * attempting to add new elements onto the queue and it is full. This queue orders elements FIFO + * (first-in-first-out). This data structure is logically equivalent to a circular buffer (i.e., + * cyclic buffer or ring buffer). + * + *

An evicting queue must be configured with a maximum size. Each time an element is added to a + * full queue, the queue automatically removes its head element. This is different from conventional + * bounded queues, which either block or reject new elements when full. + * + *

This class is not thread-safe, and does not accept null elements. + * + * @author Kurt Alfred Kluever + * @since 15.0 + */ + +public final class EvictingQueue extends ForwardingQueue implements Serializable { + + private final Queue delegate; + + final int maxSize; + + private EvictingQueue(int maxSize) { + checkArgument(maxSize >= 0, "maxSize (%s) must >= 0", maxSize); + this.delegate = new ArrayDeque(maxSize); + this.maxSize = maxSize; + } + + /** + * Creates and returns a new evicting queue that will hold up to {@code maxSize} elements. + * + *

When {@code maxSize} is zero, elements will be evicted immediately after being added to the + * queue. + */ + public static EvictingQueue create(int maxSize) { + return new EvictingQueue(maxSize); + } + + /** + * Returns the number of additional elements that this queue can accept without evicting; zero if + * the queue is currently full. + * + * @since 16.0 + */ + public int remainingCapacity() { + return maxSize - size(); + } + + @Override + protected Queue delegate() { + return delegate; + } + + /** + * Adds the given element to this queue. If the queue is currently full, the element at the head + * of the queue is evicted to make room. + * + * @return {@code true} always + */ + @Override + public boolean offer(E e) { + return add(e); + } + + /** + * Adds the given element to this queue. If the queue is currently full, the element at the head + * of the queue is evicted to make room. + * + * @return {@code true} always + */ + @Override + public boolean add(E e) { + checkNotNull(e); // check before removing + if (maxSize == 0) { + return true; + } + if (size() == maxSize) { + delegate.remove(); + } + delegate.add(e); + return true; + } + + @Override + public boolean addAll(Collection collection) { + int size = collection.size(); + if (size >= maxSize) { + clear(); + return Iterables.addAll(this, Iterables.skip(collection, size - maxSize)); + } + return standardAddAll(collection); + } + + @Override + public boolean contains(Object object) { + return delegate().contains(checkNotNull(object)); + } + + @Override + public boolean remove(Object object) { + return delegate().remove(checkNotNull(object)); + } + + // TODO(kak): Do we want to checkNotNull each element in containsAll, removeAll, and retainAll? + + private static final long serialVersionUID = 0L; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ExplicitOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ExplicitOrdering.java new file mode 100644 index 0000000000000..4b396d22c2519 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ExplicitOrdering.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.List; + + +/** An ordering that compares objects according to a given order. */ +final class ExplicitOrdering extends Ordering implements Serializable { + final ImmutableMap rankMap; + + ExplicitOrdering(List valuesInOrder) { + this(Maps.indexMap(valuesInOrder)); + } + + ExplicitOrdering(ImmutableMap rankMap) { + this.rankMap = rankMap; + } + + @Override + public int compare(T left, T right) { + return rank(left) - rank(right); // safe because both are nonnegative + } + + private int rank(T value) { + Integer rank = rankMap.get(value); + if (rank == null) { + throw new Ordering.IncomparableValueException(value); + } + return rank; + } + + @Override + public boolean equals(Object object) { + if (object instanceof ExplicitOrdering) { + ExplicitOrdering that = (ExplicitOrdering) object; + return this.rankMap.equals(that.rankMap); + } + return false; + } + + @Override + public int hashCode() { + return rankMap.hashCode(); + } + + @Override + public String toString() { + return "Ordering.explicit(" + rankMap.keySet() + ")"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntryMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntryMultimap.java new file mode 100644 index 0000000000000..183dca39a2391 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntryMultimap.java @@ -0,0 +1,413 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.in; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.not; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.ViewCachingAbstractMap; + +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + + +/** + * Implementation of {@link Multimaps#filterEntries(Multimap, Predicate)}. + * + * @author Jared Levy + * @author Louis Wasserman + */ +class FilteredEntryMultimap extends AbstractMultimap implements FilteredMultimap { + final Multimap unfiltered; + final Predicate> predicate; + + FilteredEntryMultimap(Multimap unfiltered, Predicate> predicate) { + this.unfiltered = checkNotNull(unfiltered); + this.predicate = checkNotNull(predicate); + } + + @Override + public Multimap unfiltered() { + return unfiltered; + } + + @Override + public Predicate> entryPredicate() { + return predicate; + } + + @Override + public int size() { + return entries().size(); + } + + private boolean satisfies(K key, V value) { + return predicate.apply(Maps.immutableEntry(key, value)); + } + + final class ValuePredicate implements Predicate { + private final K key; + + ValuePredicate(K key) { + this.key = key; + } + + @Override + public boolean apply(V value) { + return satisfies(key, value); + } + } + + static Collection filterCollection( + Collection collection, Predicate predicate) { + if (collection instanceof Set) { + return Sets.filter((Set) collection, predicate); + } else { + return Collections2.filter(collection, predicate); + } + } + + @Override + public boolean containsKey(Object key) { + return asMap().get(key) != null; + } + + @Override + public Collection removeAll(Object key) { + return MoreObjects.firstNonNull(asMap().remove(key), unmodifiableEmptyCollection()); + } + + Collection unmodifiableEmptyCollection() { + // These return false, rather than throwing a UOE, on remove calls. + return (unfiltered instanceof SetMultimap) + ? Collections.emptySet() + : Collections.emptyList(); + } + + @Override + public void clear() { + entries().clear(); + } + + @Override + public Collection get(final K key) { + return filterCollection(unfiltered.get(key), new ValuePredicate(key)); + } + + @Override + Collection> createEntries() { + return filterCollection(unfiltered.entries(), predicate); + } + + @Override + Collection createValues() { + return new FilteredMultimapValues<>(this); + } + + @Override + Iterator> entryIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Map> createAsMap() { + return new AsMap(); + } + + @Override + Set createKeySet() { + return asMap().keySet(); + } + + boolean removeEntriesIf(Predicate>> predicate) { + Iterator>> entryIterator = unfiltered.asMap().entrySet().iterator(); + boolean changed = false; + while (entryIterator.hasNext()) { + Entry> entry = entryIterator.next(); + K key = entry.getKey(); + Collection collection = filterCollection(entry.getValue(), new ValuePredicate(key)); + if (!collection.isEmpty() && predicate.apply(Maps.immutableEntry(key, collection))) { + if (collection.size() == entry.getValue().size()) { + entryIterator.remove(); + } else { + collection.clear(); + } + changed = true; + } + } + return changed; + } + + class AsMap extends ViewCachingAbstractMap> { + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + public void clear() { + FilteredEntryMultimap.this.clear(); + } + + @Override + public Collection get(Object key) { + Collection result = unfiltered.asMap().get(key); + if (result == null) { + return null; + } + @SuppressWarnings({"unchecked", "rawtypes"}) // key is equal to a K, if not a K itself + K k = (K) key; + result = filterCollection(result, new ValuePredicate(k)); + return result.isEmpty() ? null : result; + } + + @Override + public Collection remove(Object key) { + Collection collection = unfiltered.asMap().get(key); + if (collection == null) { + return null; + } + @SuppressWarnings({"unchecked", "rawtypes"}) // it's definitely equal to a K + K k = (K) key; + List result = Lists.newArrayList(); + Iterator itr = collection.iterator(); + while (itr.hasNext()) { + V v = itr.next(); + if (satisfies(k, v)) { + itr.remove(); + result.add(v); + } + } + if (result.isEmpty()) { + return null; + } else if (unfiltered instanceof SetMultimap) { + return Collections.unmodifiableSet(Sets.newLinkedHashSet(result)); + } else { + return Collections.unmodifiableList(result); + } + } + + @Override + Set createKeySet() { + class KeySetImpl extends Maps.KeySet> { + KeySetImpl() { + super(AsMap.this); + } + + @Override + public boolean removeAll(Collection c) { + return removeEntriesIf(Maps.keyPredicateOnEntries(in(c))); + } + + @Override + public boolean retainAll(Collection c) { + return removeEntriesIf(Maps.keyPredicateOnEntries(not(in(c)))); + } + + @Override + public boolean remove(Object o) { + return AsMap.this.remove(o) != null; + } + } + return new KeySetImpl(); + } + + @Override + Set>> createEntrySet() { + class EntrySetImpl extends Maps.EntrySet> { + @Override + Map> map() { + return AsMap.this; + } + + @Override + public Iterator>> iterator() { + return new AbstractIterator>>() { + final Iterator>> backingIterator = + unfiltered.asMap().entrySet().iterator(); + + @Override + protected Entry> computeNext() { + while (backingIterator.hasNext()) { + Entry> entry = backingIterator.next(); + K key = entry.getKey(); + Collection collection = + filterCollection(entry.getValue(), new ValuePredicate(key)); + if (!collection.isEmpty()) { + return Maps.immutableEntry(key, collection); + } + } + return endOfData(); + } + }; + } + + @Override + public boolean removeAll(Collection c) { + return removeEntriesIf(in(c)); + } + + @Override + public boolean retainAll(Collection c) { + return removeEntriesIf(not(in(c))); + } + + @Override + public int size() { + return Iterators.size(iterator()); + } + } + return new EntrySetImpl(); + } + + @Override + Collection> createValues() { + class ValuesImpl extends Maps.Values> { + ValuesImpl() { + super(AsMap.this); + } + + @Override + public boolean remove(Object o) { + if (o instanceof Collection) { + Collection c = (Collection) o; + Iterator>> entryIterator = + unfiltered.asMap().entrySet().iterator(); + while (entryIterator.hasNext()) { + Entry> entry = entryIterator.next(); + K key = entry.getKey(); + Collection collection = + filterCollection(entry.getValue(), new ValuePredicate(key)); + if (!collection.isEmpty() && c.equals(collection)) { + if (collection.size() == entry.getValue().size()) { + entryIterator.remove(); + } else { + collection.clear(); + } + return true; + } + } + } + return false; + } + + @Override + public boolean removeAll(Collection c) { + return removeEntriesIf(Maps.>valuePredicateOnEntries(in(c))); + } + + @Override + public boolean retainAll(Collection c) { + return removeEntriesIf(Maps.>valuePredicateOnEntries(not(in(c)))); + } + } + return new ValuesImpl(); + } + } + + @Override + Multiset createKeys() { + return new Keys(); + } + + class Keys extends Multimaps.Keys { + Keys() { + super(FilteredEntryMultimap.this); + } + + @Override + public int remove(Object key, int occurrences) { + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(key); + } + Collection collection = unfiltered.asMap().get(key); + if (collection == null) { + return 0; + } + @SuppressWarnings({"unchecked", "rawtypes"}) // key is equal to a K, if not a K itself + K k = (K) key; + int oldCount = 0; + Iterator itr = collection.iterator(); + while (itr.hasNext()) { + V v = itr.next(); + if (satisfies(k, v)) { + oldCount++; + if (oldCount <= occurrences) { + itr.remove(); + } + } + } + return oldCount; + } + + @Override + public Set> entrySet() { + return new Multisets.EntrySet() { + + @Override + Multiset multiset() { + return Keys.this; + } + + @Override + public Iterator> iterator() { + return Keys.this.entryIterator(); + } + + @Override + public int size() { + return FilteredEntryMultimap.this.keySet().size(); + } + + private boolean removeEntriesIf(final Predicate> predicate) { + return FilteredEntryMultimap.this.removeEntriesIf( + new Predicate>>() { + @Override + public boolean apply(Map.Entry> entry) { + return predicate.apply( + Multisets.immutableEntry(entry.getKey(), entry.getValue().size())); + } + }); + } + + @Override + public boolean removeAll(Collection c) { + return removeEntriesIf(in(c)); + } + + @Override + public boolean retainAll(Collection c) { + return removeEntriesIf(not(in(c))); + } + }; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntrySetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntrySetMultimap.java new file mode 100644 index 0000000000000..bbb48cc450120 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredEntrySetMultimap.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.util.Map.Entry; +import java.util.Set; + +/** + * Implementation of {@link Multimaps#filterEntries(SetMultimap, Predicate)}. + * + * @author Louis Wasserman + */ +final class FilteredEntrySetMultimap extends FilteredEntryMultimap + implements FilteredSetMultimap { + + FilteredEntrySetMultimap(SetMultimap unfiltered, Predicate> predicate) { + super(unfiltered, predicate); + } + + @Override + public SetMultimap unfiltered() { + return (SetMultimap) unfiltered; + } + + @Override + public Set get(K key) { + return (Set) super.get(key); + } + + @Override + public Set removeAll(Object key) { + return (Set) super.removeAll(key); + } + + @Override + public Set replaceValues(K key, Iterable values) { + return (Set) super.replaceValues(key, values); + } + + @Override + Set> createEntries() { + return Sets.filter(unfiltered().entries(), entryPredicate()); + } + + @Override + public Set> entries() { + return (Set>) super.entries(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyListMultimap.java new file mode 100644 index 0000000000000..250d651abf57d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyListMultimap.java @@ -0,0 +1,58 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.util.List; + + +/** + * Implementation of {@link Multimaps#filterKeys(ListMultimap, Predicate)}. + * + * @author Louis Wasserman + */ +final class FilteredKeyListMultimap extends FilteredKeyMultimap + implements ListMultimap { + FilteredKeyListMultimap(ListMultimap unfiltered, Predicate keyPredicate) { + super(unfiltered, keyPredicate); + } + + @Override + public ListMultimap unfiltered() { + return (ListMultimap) super.unfiltered(); + } + + @Override + public List get(K key) { + return (List) super.get(key); + } + + @Override + public List removeAll(Object key) { + return (List) super.removeAll(key); + } + + @Override + public List replaceValues(K key, Iterable values) { + return (List) super.replaceValues(key, values); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyMultimap.java new file mode 100644 index 0000000000000..655f9139442f2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeyMultimap.java @@ -0,0 +1,219 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + + +/** + * Implementation of {@link Multimaps#filterKeys(Multimap, Predicate)}. + * + * @author Louis Wasserman + */ +class FilteredKeyMultimap extends AbstractMultimap implements FilteredMultimap { + final Multimap unfiltered; + final Predicate keyPredicate; + + FilteredKeyMultimap(Multimap unfiltered, Predicate keyPredicate) { + this.unfiltered = checkNotNull(unfiltered); + this.keyPredicate = checkNotNull(keyPredicate); + } + + @Override + public Multimap unfiltered() { + return unfiltered; + } + + @Override + public Predicate> entryPredicate() { + return Maps.keyPredicateOnEntries(keyPredicate); + } + + @Override + public int size() { + int size = 0; + for (Collection collection : asMap().values()) { + size += collection.size(); + } + return size; + } + + @Override + public boolean containsKey(Object key) { + if (unfiltered.containsKey(key)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // k is equal to a K, if not one itself + K k = (K) key; + return keyPredicate.apply(k); + } + return false; + } + + @Override + public Collection removeAll(Object key) { + return containsKey(key) ? unfiltered.removeAll(key) : unmodifiableEmptyCollection(); + } + + Collection unmodifiableEmptyCollection() { + if (unfiltered instanceof SetMultimap) { + return ImmutableSet.of(); + } else { + return ImmutableList.of(); + } + } + + @Override + public void clear() { + keySet().clear(); + } + + @Override + Set createKeySet() { + return Sets.filter(unfiltered.keySet(), keyPredicate); + } + + @Override + public Collection get(K key) { + if (keyPredicate.apply(key)) { + return unfiltered.get(key); + } else if (unfiltered instanceof SetMultimap) { + return new AddRejectingSet<>(key); + } else { + return new AddRejectingList<>(key); + } + } + + static class AddRejectingSet extends ForwardingSet { + final K key; + + AddRejectingSet(K key) { + this.key = key; + } + + @Override + public boolean add(V element) { + throw new IllegalArgumentException("Key does not satisfy predicate: " + key); + } + + @Override + public boolean addAll(Collection collection) { + checkNotNull(collection); + throw new IllegalArgumentException("Key does not satisfy predicate: " + key); + } + + @Override + protected Set delegate() { + return Collections.emptySet(); + } + } + + static class AddRejectingList extends ForwardingList { + final K key; + + AddRejectingList(K key) { + this.key = key; + } + + @Override + public boolean add(V v) { + add(0, v); + return true; + } + + @Override + public void add(int index, V element) { + checkPositionIndex(index, 0); + throw new IllegalArgumentException("Key does not satisfy predicate: " + key); + } + + @Override + public boolean addAll(Collection collection) { + addAll(0, collection); + return true; + } + + @Override + public boolean addAll(int index, Collection elements) { + checkNotNull(elements); + checkPositionIndex(index, 0); + throw new IllegalArgumentException("Key does not satisfy predicate: " + key); + } + + @Override + protected List delegate() { + return Collections.emptyList(); + } + } + + @Override + Iterator> entryIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Collection> createEntries() { + return new Entries(); + } + + class Entries extends ForwardingCollection> { + @Override + protected Collection> delegate() { + return Collections2.filter(unfiltered.entries(), entryPredicate()); + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public boolean remove(Object o) { + if (o instanceof Entry) { + Entry entry = (Entry) o; + if (unfiltered.containsKey(entry.getKey()) + // if this holds, then we know entry.getKey() is a K + && keyPredicate.apply((K) entry.getKey())) { + return unfiltered.remove(entry.getKey(), entry.getValue()); + } + } + return false; + } + } + + @Override + Collection createValues() { + return new FilteredMultimapValues<>(this); + } + + @Override + Map> createAsMap() { + return Maps.filterKeys(unfiltered.asMap(), keyPredicate); + } + + @Override + Multiset createKeys() { + return Multisets.filter(unfiltered.keys(), keyPredicate); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeySetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeySetMultimap.java new file mode 100644 index 0000000000000..c9423938f3311 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredKeySetMultimap.java @@ -0,0 +1,82 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.util.Map.Entry; +import java.util.Set; + + +/** + * Implementation of {@link Multimaps#filterKeys(SetMultimap, Predicate)}. + * + * @author Louis Wasserman + */ +final class FilteredKeySetMultimap extends FilteredKeyMultimap + implements FilteredSetMultimap { + + FilteredKeySetMultimap(SetMultimap unfiltered, Predicate keyPredicate) { + super(unfiltered, keyPredicate); + } + + @Override + public SetMultimap unfiltered() { + return (SetMultimap) unfiltered; + } + + @Override + public Set get(K key) { + return (Set) super.get(key); + } + + @Override + public Set removeAll(Object key) { + return (Set) super.removeAll(key); + } + + @Override + public Set replaceValues(K key, Iterable values) { + return (Set) super.replaceValues(key, values); + } + + @Override + public Set> entries() { + return (Set>) super.entries(); + } + + @Override + Set> createEntries() { + return new EntrySet(); + } + + class EntrySet extends Entries implements Set> { + @Override + public int hashCode() { + return Sets.hashCodeImpl(this); + } + + @Override + public boolean equals(Object o) { + return Sets.equalsImpl(this, o); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimap.java new file mode 100644 index 0000000000000..6fb7eb19ad2c4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimap.java @@ -0,0 +1,36 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.util.Map.Entry; + +/** + * An interface for all filtered multimap types. + * + * @author Louis Wasserman + */ +interface FilteredMultimap extends Multimap { + Multimap unfiltered(); + + Predicate> entryPredicate(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimapValues.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimapValues.java new file mode 100644 index 0000000000000..974903404865a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredMultimapValues.java @@ -0,0 +1,97 @@ +/* + * Copyright (C) 2013 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; + +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map.Entry; + + +/** + * Implementation for {@link FilteredMultimap#values()}. + * + * @author Louis Wasserman + */ +final class FilteredMultimapValues extends AbstractCollection { + private final FilteredMultimap multimap; + + FilteredMultimapValues(FilteredMultimap multimap) { + this.multimap = checkNotNull(multimap); + } + + @Override + public Iterator iterator() { + return Maps.valueIterator(multimap.entries().iterator()); + } + + @Override + public boolean contains(Object o) { + return multimap.containsValue(o); + } + + @Override + public int size() { + return multimap.size(); + } + + @Override + public boolean remove(Object o) { + Predicate> entryPredicate = multimap.entryPredicate(); + for (Iterator> unfilteredItr = multimap.unfiltered().entries().iterator(); + unfilteredItr.hasNext(); ) { + Entry entry = unfilteredItr.next(); + if (entryPredicate.apply(entry) && Objects.equal(entry.getValue(), o)) { + unfilteredItr.remove(); + return true; + } + } + return false; + } + + @Override + public boolean removeAll(Collection c) { + return Iterables.removeIf( + multimap.unfiltered().entries(), + // explicit > is required to build with JDK6 + Predicates.>and( + multimap.entryPredicate(), Maps.valuePredicateOnEntries(Predicates.in(c)))); + } + + @Override + public boolean retainAll(Collection c) { + return Iterables.removeIf( + multimap.unfiltered().entries(), + // explicit > is required to build with JDK6 + Predicates.>and( + multimap.entryPredicate(), + Maps.valuePredicateOnEntries(Predicates.not(Predicates.in(c))))); + } + + @Override + public void clear() { + multimap.clear(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredSetMultimap.java new file mode 100644 index 0000000000000..7e7bfdda3c613 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FilteredSetMultimap.java @@ -0,0 +1,32 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * A supertype for filtered {@link SetMultimap} implementations. + * + * @author Louis Wasserman + */ +interface FilteredSetMultimap extends FilteredMultimap, SetMultimap { + @Override + SetMultimap unfiltered(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FluentIterable.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FluentIterable.java new file mode 100644 index 0000000000000..3e66a5cdd9163 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/FluentIterable.java @@ -0,0 +1,860 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Joiner; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Optional; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.SortedSet; +import java.util.stream.Stream; + + +/** + * A discouraged (but not deprecated) precursor to Java's superior {@link Stream} library. + * + *

The following types of methods are provided: + * + *

    + *
  • chaining methods which return a new {@code FluentIterable} based in some way on the + * contents of the current one (for example {@link #transform}) + *
  • element extraction methods which facilitate the retrieval of certain elements (for example + * {@link #last}) + *
  • query methods which answer questions about the {@code FluentIterable}'s contents (for + * example {@link #anyMatch}) + *
  • conversion methods which copy the {@code FluentIterable}'s contents into a new collection + * or array (for example {@link #toList}) + *
+ * + *

Several lesser-used features are currently available only as static methods on the {@link + * Iterables} class. + * + *

+ * + *

Comparison to streams

+ * + *

{@link Stream} is similar to this class, but generally more powerful, and certainly more + * standard. Key differences include: + * + *

    + *
  • A stream is single-use; it becomes invalid as soon as any "terminal operation" such + * as {@code findFirst()} or {@code iterator()} is invoked. (Even though {@code Stream} + * contains all the right method signatures to implement {@link Iterable}, it does not + * actually do so, to avoid implying repeat-iterability.) {@code FluentIterable}, on the other + * hand, is multiple-use, and does implement {@link Iterable}. + *
  • Streams offer many features not found here, including {@code min/max}, {@code distinct}, + * {@code reduce}, {@code sorted}, the very powerful {@code collect}, and built-in support for + * parallelizing stream operations. + *
  • {@code FluentIterable} contains several features not available on {@code Stream}, which are + * noted in the method descriptions below. + *
  • Streams include primitive-specialized variants such as {@code IntStream}, the use of which + * is strongly recommended. + *
  • Streams are standard Java, not requiring a third-party dependency. + *
+ * + *

Example

+ * + *

Here is an example that accepts a list from a database call, filters it based on a predicate, + * transforms it by invoking {@code toString()} on each element, and returns the first 10 elements + * as a {@code List}: + * + *

{@code
+ * ImmutableList results =
+ *     FluentIterable.from(database.getClientList())
+ *         .filter(Client::isActiveInLastMonth)
+ *         .transform(Object::toString)
+ *         .limit(10)
+ *         .toList();
+ * }
+ * + * The approximate stream equivalent is: + * + *
{@code
+ * List results =
+ *     database.getClientList()
+ *         .stream()
+ *         .filter(Client::isActiveInLastMonth)
+ *         .map(Object::toString)
+ *         .limit(10)
+ *         .collect(Collectors.toList());
+ * }
+ * + * @author Marcin Mikosik + * @since 12.0 + */ +public abstract class FluentIterable implements Iterable { + // We store 'iterable' and use it instead of 'this' to allow Iterables to perform instanceof + // checks on the _original_ iterable when FluentIterable.from is used. + // To avoid a self retain cycle under j2objc, we store Optional.absent() instead of + // Optional.of(this). To access the iterator delegate, call #getDelegate(), which converts to + // absent() back to 'this'. + private final Optional> iterableDelegate; + + /** Constructor for use by subclasses. */ + protected FluentIterable() { + this.iterableDelegate = Optional.absent(); + } + + FluentIterable(Iterable iterable) { + checkNotNull(iterable); + this.iterableDelegate = Optional.fromNullable(this != iterable ? iterable : null); + } + + private Iterable getDelegate() { + return iterableDelegate.or(this); + } + + /** + * Returns a fluent iterable that wraps {@code iterable}, or {@code iterable} itself if it is + * already a {@code FluentIterable}. + * + *

{@code Stream} equivalent: {@link Collection#stream} if {@code iterable} is a {@link + * Collection}; {@link Streams#stream(Iterable)} otherwise. + */ + public static FluentIterable from(final Iterable iterable) { + return (iterable instanceof FluentIterable) + ? (FluentIterable) iterable + : new FluentIterable(iterable) { + @Override + public Iterator iterator() { + return iterable.iterator(); + } + }; + } + + /** + * Returns a fluent iterable containing {@code elements} in the specified order. + * + *

The returned iterable is an unmodifiable view of the input array. + * + *

{@code Stream} equivalent: {@link Stream#of(Object[]) + * Stream.of(T...)}. + * + * @since 20.0 (since 18.0 as an overload of {@code of}) + */ + + public static FluentIterable from(E[] elements) { + return from(Arrays.asList(elements)); + } + + /** + * Construct a fluent iterable from another fluent iterable. This is obviously never necessary, + * but is intended to help call out cases where one migration from {@code Iterable} to {@code + * FluentIterable} has obviated the need to explicitly convert to a {@code FluentIterable}. + * + * @deprecated instances of {@code FluentIterable} don't need to be converted to {@code + * FluentIterable} + */ + @Deprecated + public static FluentIterable from(FluentIterable iterable) { + return checkNotNull(iterable); + } + + /** + * Returns a fluent iterable that combines two iterables. The returned iterable has an iterator + * that traverses the elements in {@code a}, followed by the elements in {@code b}. The source + * iterators are not polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

{@code Stream} equivalent: {@link Stream#concat}. + * + * @since 20.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static FluentIterable concat(Iterable a, Iterable b) { + return concatNoDefensiveCopy(a, b); + } + + /** + * Returns a fluent iterable that combines three iterables. The returned iterable has an iterator + * that traverses the elements in {@code a}, followed by the elements in {@code b}, followed by + * the elements in {@code c}. The source iterators are not polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

{@code Stream} equivalent: use nested calls to {@link Stream#concat}, or see the + * advice in {@link #concat(Iterable...)}. + * + * @since 20.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static FluentIterable concat( + Iterable a, Iterable b, Iterable c) { + return concatNoDefensiveCopy(a, b, c); + } + + /** + * Returns a fluent iterable that combines four iterables. The returned iterable has an iterator + * that traverses the elements in {@code a}, followed by the elements in {@code b}, followed by + * the elements in {@code c}, followed by the elements in {@code d}. The source iterators are not + * polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

{@code Stream} equivalent: use nested calls to {@link Stream#concat}, or see the + * advice in {@link #concat(Iterable...)}. + * + * @since 20.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static FluentIterable concat( + Iterable a, + Iterable b, + Iterable c, + Iterable d) { + return concatNoDefensiveCopy(a, b, c, d); + } + + /** + * Returns a fluent iterable that combines several iterables. The returned iterable has an + * iterator that traverses the elements of each iterable in {@code inputs}. The input iterators + * are not polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

{@code Stream} equivalent: to concatenate an arbitrary number of streams, use {@code + * Stream.of(stream1, stream2, ...).flatMap(s -> s)}. If the sources are iterables, use {@code + * Stream.of(iter1, iter2, ...).flatMap(Streams::stream)}. + * + * @throws NullPointerException if any of the provided iterables is {@code null} + * @since 20.0 + */ + + @SafeVarargs + @SuppressWarnings("varargs") + public static FluentIterable concat(Iterable... inputs) { + return concatNoDefensiveCopy(Arrays.copyOf(inputs, inputs.length)); + } + + /** + * Returns a fluent iterable that combines several iterables. The returned iterable has an + * iterator that traverses the elements of each iterable in {@code inputs}. The input iterators + * are not polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. The methods of the returned iterable may throw {@code + * NullPointerException} if any of the input iterators is {@code null}. + * + *

{@code Stream} equivalent: {@code streamOfStreams.flatMap(s -> s)} or {@code + * streamOfIterables.flatMap(Streams::stream)}. (See {@link Streams#stream}.) + * + * @since 20.0 + */ + + public static FluentIterable concat( + final Iterable> inputs) { + checkNotNull(inputs); + return new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.concat(Iterators.transform(inputs.iterator(), Iterables.toIterator())); + } + }; + } + + /** Concatenates a varargs array of iterables without making a defensive copy of the array. */ + @SafeVarargs + @SuppressWarnings("varargs") + private static FluentIterable concatNoDefensiveCopy( + final Iterable... inputs) { + for (Iterable input : inputs) { + checkNotNull(input); + } + return new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.concat( + /* lazily generate the iterators on each input only as needed */ + new AbstractIndexedListIterator>(inputs.length) { + @Override + public Iterator get(int i) { + return inputs[i].iterator(); + } + }); + } + }; + } + + /** + * Returns a fluent iterable containing no elements. + * + *

{@code Stream} equivalent: {@link Stream#empty}. + * + * @since 20.0 + */ + + public static FluentIterable of() { + return FluentIterable.from(ImmutableList.of()); + } + + /** + * Returns a fluent iterable containing the specified elements in order. + * + *

{@code Stream} equivalent: {@link Stream#of(Object[]) + * Stream.of(T...)}. + * + * @since 20.0 + */ + + @SafeVarargs + @SuppressWarnings("varargs") + public static FluentIterable of(E element, E... elements) { + return from(Lists.asList(element, elements)); + } + + /** + * Returns a string representation of this fluent iterable, with the format {@code [e1, e2, ..., + * en]}. + * + *

{@code Stream} equivalent: {@code stream.collect(Collectors.joining(", ", "[", "]"))} + * or (less efficiently) {@code stream.collect(Collectors.toList()).toString()}. + */ + @Override + public String toString() { + return Iterables.toString(getDelegate()); + } + + /** + * Returns the number of elements in this fluent iterable. + * + *

{@code Stream} equivalent: {@link Stream#count}. + */ + public final int size() { + return Iterables.size(getDelegate()); + } + + /** + * Returns {@code true} if this fluent iterable contains any object for which {@code + * equals(target)} is true. + * + *

{@code Stream} equivalent: {@code stream.anyMatch(Predicate.isEqual(target))}. + */ + public final boolean contains(Object target) { + return Iterables.contains(getDelegate(), target); + } + + /** + * Returns a fluent iterable whose {@code Iterator} cycles indefinitely over the elements of this + * fluent iterable. + * + *

That iterator supports {@code remove()} if {@code iterable.iterator()} does. After {@code + * remove()} is called, subsequent cycles omit the removed element, which is no longer in this + * fluent iterable. The iterator's {@code hasNext()} method returns {@code true} until this fluent + * iterable is empty. + * + *

Warning: Typical uses of the resulting iterator may produce an infinite loop. You + * should use an explicit {@code break} or be certain that you will eventually remove all the + * elements. + * + *

{@code Stream} equivalent: if the source iterable has only a single element {@code + * e}, use {@code Stream.generate(() -> e)}. Otherwise, collect your stream into a collection and + * use {@code Stream.generate(() -> collection).flatMap(Collection::stream)}. + */ + public final FluentIterable cycle() { + return from(Iterables.cycle(getDelegate())); + } + + /** + * Returns a fluent iterable whose iterators traverse first the elements of this fluent iterable, + * followed by those of {@code other}. The iterators are not polled until necessary. + * + *

The returned iterable's {@code Iterator} supports {@code remove()} when the corresponding + * {@code Iterator} supports it. + * + *

{@code Stream} equivalent: {@link Stream#concat}. + * + * @since 18.0 + */ + + public final FluentIterable append(Iterable other) { + return FluentIterable.concat(getDelegate(), other); + } + + /** + * Returns a fluent iterable whose iterators traverse first the elements of this fluent iterable, + * followed by {@code elements}. + * + *

{@code Stream} equivalent: {@code Stream.concat(thisStream, Stream.of(elements))}. + * + * @since 18.0 + */ + + @SafeVarargs + @SuppressWarnings("varargs") + public final FluentIterable append(E... elements) { + return FluentIterable.concat(getDelegate(), Arrays.asList(elements)); + } + + /** + * Returns the elements from this fluent iterable that satisfy a predicate. The resulting fluent + * iterable's iterator does not support {@code remove()}. + * + *

{@code Stream} equivalent: {@link Stream#filter} (same). + */ + public final FluentIterable filter(Predicate predicate) { + return from(Iterables.filter(getDelegate(), predicate)); + } + + /** + * Returns the elements from this fluent iterable that are instances of class {@code type}. + * + *

{@code Stream} equivalent: {@code stream.filter(type::isInstance).map(type::cast)}. + * This does perform a little more work than necessary, so another option is to insert an + * unchecked cast at some later point: + * + *

+   * {@code @SuppressWarnings({"unchecked", "rawtypes"}) // safe because of ::isInstance check
+   * ImmutableList result =
+   *     (ImmutableList) stream.filter(NewType.class::isInstance).collect(toImmutableList());}
+   * 
+ */ + public final FluentIterable filter(Class type) { + return from(Iterables.filter(getDelegate(), type)); + } + + /** + * Returns {@code true} if any element in this fluent iterable satisfies the predicate. + * + *

{@code Stream} equivalent: {@link Stream#anyMatch} (same). + */ + public final boolean anyMatch(Predicate predicate) { + return Iterables.any(getDelegate(), predicate); + } + + /** + * Returns {@code true} if every element in this fluent iterable satisfies the predicate. If this + * fluent iterable is empty, {@code true} is returned. + * + *

{@code Stream} equivalent: {@link Stream#allMatch} (same). + */ + public final boolean allMatch(Predicate predicate) { + return Iterables.all(getDelegate(), predicate); + } + + /** + * Returns an {@link Optional} containing the first element in this fluent iterable that satisfies + * the given predicate, if such an element exists. + * + *

Warning: avoid using a {@code predicate} that matches {@code null}. If {@code null} + * is matched in this fluent iterable, a {@link NullPointerException} will be thrown. + * + *

{@code Stream} equivalent: {@code stream.filter(predicate).findFirst()}. + */ + public final Optional firstMatch(Predicate predicate) { + return Iterables.tryFind(getDelegate(), predicate); + } + + /** + * Returns a fluent iterable that applies {@code function} to each element of this fluent + * iterable. + * + *

The returned fluent iterable's iterator supports {@code remove()} if this iterable's + * iterator does. After a successful {@code remove()} call, this fluent iterable no longer + * contains the corresponding element. + * + *

{@code Stream} equivalent: {@link Stream#map}. + */ + public final FluentIterable transform(Function function) { + return from(Iterables.transform(getDelegate(), function)); + } + + /** + * Applies {@code function} to each element of this fluent iterable and returns a fluent iterable + * with the concatenated combination of results. {@code function} returns an Iterable of results. + * + *

The returned fluent iterable's iterator supports {@code remove()} if this function-returned + * iterables' iterator does. After a successful {@code remove()} call, the returned fluent + * iterable no longer contains the corresponding element. + * + *

{@code Stream} equivalent: {@link Stream#flatMap} (using a function that produces + * streams, not iterables). + * + * @since 13.0 (required {@code Function>} until 14.0) + */ + public FluentIterable transformAndConcat( + Function> function) { + return FluentIterable.concat(transform(function)); + } + + /** + * Returns an {@link Optional} containing the first element in this fluent iterable. If the + * iterable is empty, {@code Optional.absent()} is returned. + * + *

{@code Stream} equivalent: if the goal is to obtain any element, {@link + * Stream#findAny}; if it must specifically be the first element, {@code Stream#findFirst}. + * + * @throws NullPointerException if the first element is null; if this is a possibility, use {@code + * iterator().next()} or {@link Iterables#getFirst} instead. + */ + public final Optional first() { + Iterator iterator = getDelegate().iterator(); + return iterator.hasNext() ? Optional.of(iterator.next()) : Optional.absent(); + } + + /** + * Returns an {@link Optional} containing the last element in this fluent iterable. If the + * iterable is empty, {@code Optional.absent()} is returned. If the underlying {@code iterable} is + * a {@link List} with {@link java.util.RandomAccess} support, then this operation is guaranteed + * to be {@code O(1)}. + * + *

{@code Stream} equivalent: {@code stream.reduce((a, b) -> b)}. + * + * @throws NullPointerException if the last element is null; if this is a possibility, use {@link + * Iterables#getLast} instead. + */ + public final Optional last() { + // Iterables#getLast was inlined here so we don't have to throw/catch a NSEE + + // TODO(kevinb): Support a concurrently modified collection? + Iterable iterable = getDelegate(); + if (iterable instanceof List) { + List list = (List) iterable; + if (list.isEmpty()) { + return Optional.absent(); + } + return Optional.of(list.get(list.size() - 1)); + } + Iterator iterator = iterable.iterator(); + if (!iterator.hasNext()) { + return Optional.absent(); + } + + /* + * TODO(kevinb): consider whether this "optimization" is worthwhile. Users with SortedSets tend + * to know they are SortedSets and probably would not call this method. + */ + if (iterable instanceof SortedSet) { + SortedSet sortedSet = (SortedSet) iterable; + return Optional.of(sortedSet.last()); + } + + while (true) { + E current = iterator.next(); + if (!iterator.hasNext()) { + return Optional.of(current); + } + } + } + + /** + * Returns a view of this fluent iterable that skips its first {@code numberToSkip} elements. If + * this fluent iterable contains fewer than {@code numberToSkip} elements, the returned fluent + * iterable skips all of its elements. + * + *

Modifications to this fluent iterable before a call to {@code iterator()} are reflected in + * the returned fluent iterable. That is, the its iterator skips the first {@code numberToSkip} + * elements that exist when the iterator is created, not when {@code skip()} is called. + * + *

The returned fluent iterable's iterator supports {@code remove()} if the {@code Iterator} of + * this fluent iterable supports it. Note that it is not possible to delete the last + * skipped element by immediately calling {@code remove()} on the returned fluent iterable's + * iterator, as the {@code Iterator} contract states that a call to {@code * remove()} before a + * call to {@code next()} will throw an {@link IllegalStateException}. + * + *

{@code Stream} equivalent: {@link Stream#skip} (same). + */ + public final FluentIterable skip(int numberToSkip) { + return from(Iterables.skip(getDelegate(), numberToSkip)); + } + + /** + * Creates a fluent iterable with the first {@code size} elements of this fluent iterable. If this + * fluent iterable does not contain that many elements, the returned fluent iterable will have the + * same behavior as this fluent iterable. The returned fluent iterable's iterator supports {@code + * remove()} if this fluent iterable's iterator does. + * + *

{@code Stream} equivalent: {@link Stream#limit} (same). + * + * @param maxSize the maximum number of elements in the returned fluent iterable + * @throws IllegalArgumentException if {@code size} is negative + */ + public final FluentIterable limit(int maxSize) { + return from(Iterables.limit(getDelegate(), maxSize)); + } + + /** + * Determines whether this fluent iterable is empty. + * + *

{@code Stream} equivalent: {@code !stream.findAny().isPresent()}. + */ + public final boolean isEmpty() { + return !getDelegate().iterator().hasNext(); + } + + /** + * Returns an {@code ImmutableList} containing all of the elements from this fluent iterable in + * proper sequence. + * + *

{@code Stream} equivalent: pass {@link ImmutableList#toImmutableList} to {@code + * stream.collect()}. + * + * @throws NullPointerException if any element is {@code null} + * @since 14.0 (since 12.0 as {@code toImmutableList()}). + */ + public final ImmutableList toList() { + return ImmutableList.copyOf(getDelegate()); + } + + /** + * Returns an {@code ImmutableList} containing all of the elements from this {@code + * FluentIterable} in the order specified by {@code comparator}. To produce an {@code + * ImmutableList} sorted by its natural ordering, use {@code toSortedList(Ordering.natural())}. + * + *

{@code Stream} equivalent: pass {@link ImmutableList#toImmutableList} to {@code + * stream.sorted(comparator).collect()}. + * + * @param comparator the function by which to sort list elements + * @throws NullPointerException if any element of this iterable is {@code null} + * @since 14.0 (since 13.0 as {@code toSortedImmutableList()}). + */ + public final ImmutableList toSortedList(Comparator comparator) { + return Ordering.from(comparator).immutableSortedCopy(getDelegate()); + } + + /** + * Returns an {@code ImmutableSet} containing all of the elements from this fluent iterable with + * duplicates removed. + * + *

{@code Stream} equivalent: pass {@link ImmutableSet#toImmutableSet} to {@code + * stream.collect()}. + * + * @throws NullPointerException if any element is {@code null} + * @since 14.0 (since 12.0 as {@code toImmutableSet()}). + */ + public final ImmutableSet toSet() { + return ImmutableSet.copyOf(getDelegate()); + } + + /** + * Returns an {@code ImmutableSortedSet} containing all of the elements from this {@code + * FluentIterable} in the order specified by {@code comparator}, with duplicates (determined by + * {@code comparator.compare(x, y) == 0}) removed. To produce an {@code ImmutableSortedSet} sorted + * by its natural ordering, use {@code toSortedSet(Ordering.natural())}. + * + *

{@code Stream} equivalent: pass {@link ImmutableSortedSet#toImmutableSortedSet} to + * {@code stream.collect()}. + * + * @param comparator the function by which to sort set elements + * @throws NullPointerException if any element of this iterable is {@code null} + * @since 14.0 (since 12.0 as {@code toImmutableSortedSet()}). + */ + public final ImmutableSortedSet toSortedSet(Comparator comparator) { + return ImmutableSortedSet.copyOf(comparator, getDelegate()); + } + + /** + * Returns an {@code ImmutableMultiset} containing all of the elements from this fluent iterable. + * + *

{@code Stream} equivalent: pass {@link ImmutableMultiset#toImmutableMultiset} to + * {@code stream.collect()}. + * + * @throws NullPointerException if any element is null + * @since 19.0 + */ + public final ImmutableMultiset toMultiset() { + return ImmutableMultiset.copyOf(getDelegate()); + } + + /** + * Returns an immutable map whose keys are the distinct elements of this {@code FluentIterable} + * and whose value for each key was computed by {@code valueFunction}. The map's iteration order + * is the order of the first appearance of each key in this iterable. + * + *

When there are multiple instances of a key in this iterable, it is unspecified whether + * {@code valueFunction} will be applied to more than one instance of that key and, if it is, + * which result will be mapped to that key in the returned map. + * + *

{@code Stream} equivalent: {@code stream.collect(ImmutableMap.toImmutableMap(k -> k, + * valueFunction))}. + * + * @throws NullPointerException if any element of this iterable is {@code null}, or if {@code + * valueFunction} produces {@code null} for any key + * @since 14.0 + */ + public final ImmutableMap toMap(Function valueFunction) { + return Maps.toMap(getDelegate(), valueFunction); + } + + /** + * Creates an index {@code ImmutableListMultimap} that contains the results of applying a + * specified function to each item in this {@code FluentIterable} of values. Each element of this + * iterable will be stored as a value in the resulting multimap, yielding a multimap with the same + * size as this iterable. The key used to store that value in the multimap will be the result of + * calling the function on that value. The resulting multimap is created as an immutable snapshot. + * In the returned multimap, keys appear in the order they are first encountered, and the values + * corresponding to each key appear in the same order as they are encountered. + * + *

{@code Stream} equivalent: {@code stream.collect(Collectors.groupingBy(keyFunction))} + * behaves similarly, but returns a mutable {@code Map>} instead, and may not preserve + * the order of entries). + * + * @param keyFunction the function used to produce the key for each value + * @throws NullPointerException if any element of this iterable is {@code null}, or if {@code + * keyFunction} produces {@code null} for any key + * @since 14.0 + */ + public final ImmutableListMultimap index(Function keyFunction) { + return Multimaps.index(getDelegate(), keyFunction); + } + + /** + * Returns a map with the contents of this {@code FluentIterable} as its {@code values}, indexed + * by keys derived from those values. In other words, each input value produces an entry in the + * map whose key is the result of applying {@code keyFunction} to that value. These entries appear + * in the same order as they appeared in this fluent iterable. Example usage: + * + *

{@code
+   * Color red = new Color("red", 255, 0, 0);
+   * ...
+   * FluentIterable allColors = FluentIterable.from(ImmutableSet.of(red, green, blue));
+   *
+   * Map colorForName = allColors.uniqueIndex(toStringFunction());
+   * assertThat(colorForName).containsEntry("red", red);
+   * }
+ * + *

If your index may associate multiple values with each key, use {@link #index(Function) + * index}. + * + *

{@code Stream} equivalent: {@code + * stream.collect(ImmutableMap.toImmutableMap(keyFunction, v -> v))}. + * + * @param keyFunction the function used to produce the key for each value + * @return a map mapping the result of evaluating the function {@code keyFunction} on each value + * in this fluent iterable to that value + * @throws IllegalArgumentException if {@code keyFunction} produces the same key for more than one + * value in this fluent iterable + * @throws NullPointerException if any element of this iterable is {@code null}, or if {@code + * keyFunction} produces {@code null} for any key + * @since 14.0 + */ + public final ImmutableMap uniqueIndex(Function keyFunction) { + return Maps.uniqueIndex(getDelegate(), keyFunction); + } + + /** + * Returns an array containing all of the elements from this fluent iterable in iteration order. + * + *

{@code Stream} equivalent: if an object array is acceptable, use {@code + * stream.toArray()}; if {@code type} is a class literal such as {@code MyType.class}, use {@code + * stream.toArray(MyType[]::new)}. Otherwise use {@code stream.toArray( len -> (E[]) + * Array.newInstance(type, len))}. + * + * @param type the type of the elements + * @return a newly-allocated array into which all the elements of this fluent iterable have been + * copied + */ + public final E[] toArray(Class type) { + return Iterables.toArray(getDelegate(), type); + } + + /** + * Copies all the elements from this fluent iterable to {@code collection}. This is equivalent to + * calling {@code Iterables.addAll(collection, this)}. + * + *

{@code Stream} equivalent: {@code stream.forEachOrdered(collection::add)} or {@code + * stream.forEach(collection::add)}. + * + * @param collection the collection to copy elements to + * @return {@code collection}, for convenience + * @since 14.0 + */ + public final > C copyInto(C collection) { + checkNotNull(collection); + Iterable iterable = getDelegate(); + if (iterable instanceof Collection) { + collection.addAll(Collections2.cast(iterable)); + } else { + for (E item : iterable) { + collection.add(item); + } + } + return collection; + } + + /** + * Returns a {@link String} containing all of the elements of this fluent iterable joined with + * {@code joiner}. + * + *

{@code Stream} equivalent: {@code joiner.join(stream.iterator())}, or, if you are not + * using any optional {@code Joiner} features, {@code + * stream.collect(Collectors.joining(delimiter)}. + * + * @since 18.0 + */ + + public final String join(Joiner joiner) { + return joiner.join(this); + } + + /** + * Returns the element at the specified position in this fluent iterable. + * + *

{@code Stream} equivalent: {@code stream.skip(position).findFirst().get()} (but note + * that this throws different exception types, and throws an exception if {@code null} would be + * returned). + * + * @param position position of the element to return + * @return the element at the specified position in this fluent iterable + * @throws IndexOutOfBoundsException if {@code position} is negative or greater than or equal to + * the size of this fluent iterable + */ + // TODO(kevinb): add ? + public final E get(int position) { + return Iterables.get(getDelegate(), position); + } + + /** + * Returns a stream of this fluent iterable's contents (similar to calling {@link + * Collection#stream} on a collection). + * + *

Note: the earlier in the chain you can switch to {@code Stream} usage (ideally not + * going through {@code FluentIterable} at all), the more performant and idiomatic your code will + * be. This method is a transitional aid, to be used only when really necessary. + * + * @since 21.0 + */ + public final Stream stream() { + return Streams.stream(getDelegate()); + } + + /** Function that transforms {@code Iterable} into a fluent iterable. */ + private static class FromIterableFunction implements Function, FluentIterable> { + @Override + public FluentIterable apply(Iterable fromObject) { + return FluentIterable.from(fromObject); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingCollection.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingCollection.java new file mode 100644 index 0000000000000..f2a1edd540eaf --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingCollection.java @@ -0,0 +1,252 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.util.Collection; +import java.util.Iterator; + + +/** + * A collection which forwards all its method calls to another collection. Subclasses should + * override one or more methods to modify the behavior of the backing collection as desired per the + * decorator pattern. + * + *

Warning: The methods of {@code ForwardingCollection} forward indiscriminately to + * the methods of the delegate. For example, overriding {@link #add} alone will not change + * the behavior of {@link #addAll}, which can lead to unexpected behavior. In this case, you should + * override {@code addAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardAddAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingCollection}. + * + *

The {@code standard} methods are not guaranteed to be thread-safe, even when all of the + * methods that they depend on are thread-safe. + * + * @author Kevin Bourrillion + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingCollection extends ForwardingObject implements Collection { + // TODO(lowasser): identify places where thread safety is actually lost + + /** Constructor for use by subclasses. */ + protected ForwardingCollection() {} + + @Override + protected abstract Collection delegate(); + + @Override + public Iterator iterator() { + return delegate().iterator(); + } + + @Override + public int size() { + return delegate().size(); + } + + @Override + public boolean removeAll(Collection collection) { + return delegate().removeAll(collection); + } + + @Override + public boolean isEmpty() { + return delegate().isEmpty(); + } + + @Override + public boolean contains(Object object) { + return delegate().contains(object); + } + + @Override + public boolean add(E element) { + return delegate().add(element); + } + + @Override + public boolean remove(Object object) { + return delegate().remove(object); + } + + @Override + public boolean containsAll(Collection collection) { + return delegate().containsAll(collection); + } + + @Override + public boolean addAll(Collection collection) { + return delegate().addAll(collection); + } + + @Override + public boolean retainAll(Collection collection) { + return delegate().retainAll(collection); + } + + @Override + public void clear() { + delegate().clear(); + } + + @Override + public Object[] toArray() { + return delegate().toArray(); + } + + @Override + public T[] toArray(T[] array) { + return delegate().toArray(array); + } + + /** + * A sensible definition of {@link #contains} in terms of {@link #iterator}. If you override + * {@link #iterator}, you may wish to override {@link #contains} to forward to this + * implementation. + * + * @since 7.0 + */ + protected boolean standardContains(Object object) { + return Iterators.contains(iterator(), object); + } + + /** + * A sensible definition of {@link #containsAll} in terms of {@link #contains} . If you override + * {@link #contains}, you may wish to override {@link #containsAll} to forward to this + * implementation. + * + * @since 7.0 + */ + protected boolean standardContainsAll(Collection collection) { + return Collections2.containsAllImpl(this, collection); + } + + /** + * A sensible definition of {@link #addAll} in terms of {@link #add}. If you override {@link + * #add}, you may wish to override {@link #addAll} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardAddAll(Collection collection) { + return Iterators.addAll(this, collection.iterator()); + } + + /** + * A sensible definition of {@link #remove} in terms of {@link #iterator}, using the iterator's + * {@code remove} method. If you override {@link #iterator}, you may wish to override {@link + * #remove} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardRemove(Object object) { + Iterator iterator = iterator(); + while (iterator.hasNext()) { + if (Objects.equal(iterator.next(), object)) { + iterator.remove(); + return true; + } + } + return false; + } + + /** + * A sensible definition of {@link #removeAll} in terms of {@link #iterator}, using the iterator's + * {@code remove} method. If you override {@link #iterator}, you may wish to override {@link + * #removeAll} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardRemoveAll(Collection collection) { + return Iterators.removeAll(iterator(), collection); + } + + /** + * A sensible definition of {@link #retainAll} in terms of {@link #iterator}, using the iterator's + * {@code remove} method. If you override {@link #iterator}, you may wish to override {@link + * #retainAll} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardRetainAll(Collection collection) { + return Iterators.retainAll(iterator(), collection); + } + + /** + * A sensible definition of {@link #clear} in terms of {@link #iterator}, using the iterator's + * {@code remove} method. If you override {@link #iterator}, you may wish to override {@link + * #clear} to forward to this implementation. + * + * @since 7.0 + */ + protected void standardClear() { + Iterators.clear(iterator()); + } + + /** + * A sensible definition of {@link #isEmpty} as {@code !iterator().hasNext}. If you override + * {@link #isEmpty}, you may wish to override {@link #isEmpty} to forward to this implementation. + * Alternately, it may be more efficient to implement {@code isEmpty} as {@code size() == 0}. + * + * @since 7.0 + */ + protected boolean standardIsEmpty() { + return !iterator().hasNext(); + } + + /** + * A sensible definition of {@link #toString} in terms of {@link #iterator}. If you override + * {@link #iterator}, you may wish to override {@link #toString} to forward to this + * implementation. + * + * @since 7.0 + */ + protected String standardToString() { + return Collections2.toStringImpl(this); + } + + /** + * A sensible definition of {@link #toArray()} in terms of {@link #toArray(Object[])}. If you + * override {@link #toArray(Object[])}, you may wish to override {@link #toArray} to forward to + * this implementation. + * + * @since 7.0 + */ + protected Object[] standardToArray() { + Object[] newArray = new Object[size()]; + return toArray(newArray); + } + + /** + * A sensible definition of {@link #toArray(Object[])} in terms of {@link #size} and {@link + * #iterator}. If you override either of these methods, you may wish to override {@link #toArray} + * to forward to this implementation. + * + * @since 7.0 + */ + protected T[] standardToArray(T[] array) { + return ObjectArrays.toArrayImpl(this, array); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingConcurrentMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingConcurrentMap.java new file mode 100644 index 0000000000000..7697dbdcd3350 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingConcurrentMap.java @@ -0,0 +1,68 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.concurrent.ConcurrentMap; + +/** + * A concurrent map which forwards all its method calls to another concurrent map. Subclasses should + * override one or more methods to modify the behavior of the backing map as desired per the decorator pattern. + * + *

{@code default} method warning: This class forwards calls to only some {@code + * default} methods. Specifically, it forwards calls only for methods that existed before + * {@code default} methods were introduced. For newer methods, like {@code forEach}, it inherits + * their default implementations. When those implementations invoke methods, they invoke methods on + * the {@code ForwardingConcurrentMap}. + * + * @author Charles Fry + * @since 2.0 + */ +public abstract class ForwardingConcurrentMap extends ForwardingMap + implements ConcurrentMap { + + /** Constructor for use by subclasses. */ + protected ForwardingConcurrentMap() {} + + @Override + protected abstract ConcurrentMap delegate(); + + @Override + public V putIfAbsent(K key, V value) { + return delegate().putIfAbsent(key, value); + } + + @Override + public boolean remove(Object key, Object value) { + return delegate().remove(key, value); + } + + @Override + public V replace(K key, V value) { + return delegate().replace(key, value); + } + + @Override + public boolean replace(K key, V oldValue, V newValue) { + return delegate().replace(key, oldValue, newValue); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableCollection.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableCollection.java new file mode 100644 index 0000000000000..4e357ab71959d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableCollection.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Dummy class that makes the GWT serialization policy happy. It isn't used on the server-side. + * + * @author Hayward Chan + */ +class ForwardingImmutableCollection { + private ForwardingImmutableCollection() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableList.java new file mode 100644 index 0000000000000..ba51958f78d16 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableList.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Unused stub class, unreferenced under Java and manually emulated under GWT. + * + * @author Chris Povirk + */ +abstract class ForwardingImmutableList { + private ForwardingImmutableList() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableMap.java new file mode 100644 index 0000000000000..90b87b88479b0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableMap.java @@ -0,0 +1,31 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Unused stub class, unreferenced under Java and manually emulated under GWT. + * + * @author Chris Povirk + */ +abstract class ForwardingImmutableMap { + private ForwardingImmutableMap() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableSet.java new file mode 100644 index 0000000000000..617d90c8257c2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingImmutableSet.java @@ -0,0 +1,30 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +/** + * Unused stub class, unreferenced under Java and manually emulated under GWT. + * + * @author Chris Povirk + */ +abstract class ForwardingImmutableSet { + private ForwardingImmutableSet() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingIterator.java new file mode 100644 index 0000000000000..55b4ad81e9cbc --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingIterator.java @@ -0,0 +1,62 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Iterator; + +/** + * An iterator which forwards all its method calls to another iterator. Subclasses should override + * one or more methods to modify the behavior of the backing iterator as desired per the decorator pattern. + * + *

{@code default} method warning: This class forwards calls to only some {@code + * default} methods. Specifically, it forwards calls only for methods that existed before {@code default} + * methods were introduced. For newer methods, like {@code forEachRemaining}, it inherits their + * default implementations. When those implementations invoke methods, they invoke methods on the + * {@code ForwardingIterator}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public abstract class ForwardingIterator extends ForwardingObject implements Iterator { + + /** Constructor for use by subclasses. */ + protected ForwardingIterator() {} + + @Override + protected abstract Iterator delegate(); + + @Override + public boolean hasNext() { + return delegate().hasNext(); + } + + @Override + public T next() { + return delegate().next(); + } + + @Override + public void remove() { + delegate().remove(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingList.java new file mode 100644 index 0000000000000..a49be639044ea --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingList.java @@ -0,0 +1,238 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; + + +/** + * A list which forwards all its method calls to another list. Subclasses should override one or + * more methods to modify the behavior of the backing list as desired per the decorator pattern. + * + *

This class does not implement {@link java.util.RandomAccess}. If the delegate supports random + * access, the {@code ForwardingList} subclass should implement the {@code RandomAccess} interface. + * + *

Warning: The methods of {@code ForwardingList} forward indiscriminately to the + * methods of the delegate. For example, overriding {@link #add} alone will not change the + * behavior of {@link #addAll}, which can lead to unexpected behavior. In this case, you should + * override {@code addAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardAddAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingList}. + * + *

The {@code standard} methods and any collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingList extends ForwardingCollection implements List { + // TODO(lowasser): identify places where thread safety is actually lost + + /** Constructor for use by subclasses. */ + protected ForwardingList() {} + + @Override + protected abstract List delegate(); + + @Override + public void add(int index, E element) { + delegate().add(index, element); + } + + @Override + public boolean addAll(int index, Collection elements) { + return delegate().addAll(index, elements); + } + + @Override + public E get(int index) { + return delegate().get(index); + } + + @Override + public int indexOf(Object element) { + return delegate().indexOf(element); + } + + @Override + public int lastIndexOf(Object element) { + return delegate().lastIndexOf(element); + } + + @Override + public ListIterator listIterator() { + return delegate().listIterator(); + } + + @Override + public ListIterator listIterator(int index) { + return delegate().listIterator(index); + } + + @Override + public E remove(int index) { + return delegate().remove(index); + } + + @Override + public E set(int index, E element) { + return delegate().set(index, element); + } + + @Override + public List subList(int fromIndex, int toIndex) { + return delegate().subList(fromIndex, toIndex); + } + + @Override + public boolean equals(Object object) { + return object == this || delegate().equals(object); + } + + @Override + public int hashCode() { + return delegate().hashCode(); + } + + /** + * A sensible default implementation of {@link #add(Object)}, in terms of {@link #add(int, + * Object)}. If you override {@link #add(int, Object)}, you may wish to override {@link + * #add(Object)} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardAdd(E element) { + add(size(), element); + return true; + } + + /** + * A sensible default implementation of {@link #addAll(int, Collection)}, in terms of the {@code + * add} method of {@link #listIterator(int)}. If you override {@link #listIterator(int)}, you may + * wish to override {@link #addAll(int, Collection)} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardAddAll(int index, Iterable elements) { + return Lists.addAllImpl(this, index, elements); + } + + /** + * A sensible default implementation of {@link #indexOf}, in terms of {@link #listIterator()}. If + * you override {@link #listIterator()}, you may wish to override {@link #indexOf} to forward to + * this implementation. + * + * @since 7.0 + */ + protected int standardIndexOf(Object element) { + return Lists.indexOfImpl(this, element); + } + + /** + * A sensible default implementation of {@link #lastIndexOf}, in terms of {@link + * #listIterator(int)}. If you override {@link #listIterator(int)}, you may wish to override + * {@link #lastIndexOf} to forward to this implementation. + * + * @since 7.0 + */ + protected int standardLastIndexOf(Object element) { + return Lists.lastIndexOfImpl(this, element); + } + + /** + * A sensible default implementation of {@link #iterator}, in terms of {@link #listIterator()}. If + * you override {@link #listIterator()}, you may wish to override {@link #iterator} to forward to + * this implementation. + * + * @since 7.0 + */ + protected Iterator standardIterator() { + return listIterator(); + } + + /** + * A sensible default implementation of {@link #listIterator()}, in terms of {@link + * #listIterator(int)}. If you override {@link #listIterator(int)}, you may wish to override + * {@link #listIterator()} to forward to this implementation. + * + * @since 7.0 + */ + protected ListIterator standardListIterator() { + return listIterator(0); + } + + /** + * A sensible default implementation of {@link #listIterator(int)}, in terms of {@link #size}, + * {@link #get(int)}, {@link #set(int, Object)}, {@link #add(int, Object)}, and {@link + * #remove(int)}. If you override any of these methods, you may wish to override {@link + * #listIterator(int)} to forward to this implementation. + * + * @since 7.0 + */ + + protected ListIterator standardListIterator(int start) { + return Lists.listIteratorImpl(this, start); + } + + /** + * A sensible default implementation of {@link #subList(int, int)}. If you override any other + * methods, you may wish to override {@link #subList(int, int)} to forward to this implementation. + * + * @since 7.0 + */ + + protected List standardSubList(int fromIndex, int toIndex) { + return Lists.subListImpl(this, fromIndex, toIndex); + } + + /** + * A sensible definition of {@link #equals(Object)} in terms of {@link #size} and {@link + * #iterator}. If you override either of those methods, you may wish to override {@link + * #equals(Object)} to forward to this implementation. + * + * @since 7.0 + */ + + protected boolean standardEquals(Object object) { + return Lists.equalsImpl(this, object); + } + + /** + * A sensible definition of {@link #hashCode} in terms of {@link #iterator}. If you override + * {@link #iterator}, you may wish to override {@link #hashCode} to forward to this + * implementation. + * + * @since 7.0 + */ + + protected int standardHashCode() { + return Lists.hashCodeImpl(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListIterator.java new file mode 100644 index 0000000000000..ddf2b883be257 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListIterator.java @@ -0,0 +1,78 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.ListIterator; + +/** + * A list iterator which forwards all its method calls to another list iterator. Subclasses should + * override one or more methods to modify the behavior of the backing iterator as desired per the decorator pattern. + * + *

{@code default} method warning: This class forwards calls to only some {@code + * default} methods. Specifically, it forwards calls only for methods that existed before {@code + * default} methods were introduced. For newer methods, like {@code forEachRemaining}, it + * inherits their default implementations. When those implementations invoke methods, they invoke + * methods on the {@code ForwardingListIterator}. + * + * @author Mike Bostock + * @since 2.0 + */ +public abstract class ForwardingListIterator extends ForwardingIterator + implements ListIterator { + + /** Constructor for use by subclasses. */ + protected ForwardingListIterator() {} + + @Override + protected abstract ListIterator delegate(); + + @Override + public void add(E element) { + delegate().add(element); + } + + @Override + public boolean hasPrevious() { + return delegate().hasPrevious(); + } + + @Override + public int nextIndex() { + return delegate().nextIndex(); + } + + @Override + public E previous() { + return delegate().previous(); + } + + @Override + public int previousIndex() { + return delegate().previousIndex(); + } + + @Override + public void set(E element) { + delegate().set(element); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListMultimap.java new file mode 100644 index 0000000000000..ab46e21e5ec27 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingListMultimap.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.List; + + +/** + * A list multimap which forwards all its method calls to another list multimap. Subclasses should + * override one or more methods to modify the behavior of the backing multimap as desired per the decorator pattern. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingListMultimap}. + * + * @author Kurt Alfred Kluever + * @since 3.0 + */ +public abstract class ForwardingListMultimap extends ForwardingMultimap + implements ListMultimap { + + /** Constructor for use by subclasses. */ + protected ForwardingListMultimap() {} + + @Override + protected abstract ListMultimap delegate(); + + @Override + public List get(K key) { + return delegate().get(key); + } + + @Override + public List removeAll(Object key) { + return delegate().removeAll(key); + } + + @Override + public List replaceValues(K key, Iterable values) { + return delegate().replaceValues(key, values); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMap.java new file mode 100644 index 0000000000000..81c91b577d4bb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMap.java @@ -0,0 +1,305 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + + +/** + * A map which forwards all its method calls to another map. Subclasses should override one or more + * methods to modify the behavior of the backing map as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingMap} forward indiscriminately to the + * methods of the delegate. For example, overriding {@link #put} alone will not change the + * behavior of {@link #putAll}, which can lead to unexpected behavior. In this case, you should + * override {@code putAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardPutAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingMap}. + * + *

Each of the {@code standard} methods, where appropriate, use {@link Objects#equal} to test + * equality for both keys and values. This may not be the desired behavior for map implementations + * that use non-standard notions of key equality, such as a {@code SortedMap} whose comparator is + * not consistent with {@code equals}. + * + *

The {@code standard} methods and the collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Kevin Bourrillion + * @author Jared Levy + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingMap extends ForwardingObject implements Map { + // TODO(lowasser): identify places where thread safety is actually lost + + /** Constructor for use by subclasses. */ + protected ForwardingMap() {} + + @Override + protected abstract Map delegate(); + + @Override + public int size() { + return delegate().size(); + } + + @Override + public boolean isEmpty() { + return delegate().isEmpty(); + } + + @Override + public V remove(Object object) { + return delegate().remove(object); + } + + @Override + public void clear() { + delegate().clear(); + } + + @Override + public boolean containsKey(Object key) { + return delegate().containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return delegate().containsValue(value); + } + + @Override + public V get(Object key) { + return delegate().get(key); + } + + @Override + public V put(K key, V value) { + return delegate().put(key, value); + } + + @Override + public void putAll(Map map) { + delegate().putAll(map); + } + + @Override + public Set keySet() { + return delegate().keySet(); + } + + @Override + public Collection values() { + return delegate().values(); + } + + @Override + public Set> entrySet() { + return delegate().entrySet(); + } + + @Override + public boolean equals(Object object) { + return object == this || delegate().equals(object); + } + + @Override + public int hashCode() { + return delegate().hashCode(); + } + + /** + * A sensible definition of {@link #putAll(Map)} in terms of {@link #put(Object, Object)}. If you + * override {@link #put(Object, Object)}, you may wish to override {@link #putAll(Map)} to forward + * to this implementation. + * + * @since 7.0 + */ + protected void standardPutAll(Map map) { + Maps.putAllImpl(this, map); + } + + /** + * A sensible, albeit inefficient, definition of {@link #remove} in terms of the {@code iterator} + * method of {@link #entrySet}. If you override {@link #entrySet}, you may wish to override {@link + * #remove} to forward to this implementation. + * + *

Alternately, you may wish to override {@link #remove} with {@code keySet().remove}, assuming + * that approach would not lead to an infinite loop. + * + * @since 7.0 + */ + + protected V standardRemove(Object key) { + Iterator> entryIterator = entrySet().iterator(); + while (entryIterator.hasNext()) { + Entry entry = entryIterator.next(); + if (Objects.equal(entry.getKey(), key)) { + V value = entry.getValue(); + entryIterator.remove(); + return value; + } + } + return null; + } + + /** + * A sensible definition of {@link #clear} in terms of the {@code iterator} method of {@link + * #entrySet}. In many cases, you may wish to override {@link #clear} to forward to this + * implementation. + * + * @since 7.0 + */ + protected void standardClear() { + Iterators.clear(entrySet().iterator()); + } + + /** + * A sensible implementation of {@link Map#keySet} in terms of the following methods: {@link + * ForwardingMap#clear}, {@link ForwardingMap#containsKey}, {@link ForwardingMap#isEmpty}, {@link + * ForwardingMap#remove}, {@link ForwardingMap#size}, and the {@link Set#iterator} method of + * {@link ForwardingMap#entrySet}. In many cases, you may wish to override {@link + * ForwardingMap#keySet} to forward to this implementation or a subclass thereof. + * + * @since 10.0 + */ + + protected class StandardKeySet extends Maps.KeySet { + /** Constructor for use by subclasses. */ + public StandardKeySet() { + super(ForwardingMap.this); + } + } + + /** + * A sensible, albeit inefficient, definition of {@link #containsKey} in terms of the {@code + * iterator} method of {@link #entrySet}. If you override {@link #entrySet}, you may wish to + * override {@link #containsKey} to forward to this implementation. + * + * @since 7.0 + */ + + protected boolean standardContainsKey(Object key) { + return Maps.containsKeyImpl(this, key); + } + + /** + * A sensible implementation of {@link Map#values} in terms of the following methods: {@link + * ForwardingMap#clear}, {@link ForwardingMap#containsValue}, {@link ForwardingMap#isEmpty}, + * {@link ForwardingMap#size}, and the {@link Set#iterator} method of {@link + * ForwardingMap#entrySet}. In many cases, you may wish to override {@link ForwardingMap#values} + * to forward to this implementation or a subclass thereof. + * + * @since 10.0 + */ + + protected class StandardValues extends Maps.Values { + /** Constructor for use by subclasses. */ + public StandardValues() { + super(ForwardingMap.this); + } + } + + /** + * A sensible definition of {@link #containsValue} in terms of the {@code iterator} method of + * {@link #entrySet}. If you override {@link #entrySet}, you may wish to override {@link + * #containsValue} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardContainsValue(Object value) { + return Maps.containsValueImpl(this, value); + } + + /** + * A sensible implementation of {@link Map#entrySet} in terms of the following methods: {@link + * ForwardingMap#clear}, {@link ForwardingMap#containsKey}, {@link ForwardingMap#get}, {@link + * ForwardingMap#isEmpty}, {@link ForwardingMap#remove}, and {@link ForwardingMap#size}. In many + * cases, you may wish to override {@link #entrySet} to forward to this implementation or a + * subclass thereof. + * + * @since 10.0 + */ + + protected abstract class StandardEntrySet extends Maps.EntrySet { + /** Constructor for use by subclasses. */ + public StandardEntrySet() {} + + @Override + Map map() { + return ForwardingMap.this; + } + } + + /** + * A sensible definition of {@link #isEmpty} in terms of the {@code iterator} method of {@link + * #entrySet}. If you override {@link #entrySet}, you may wish to override {@link #isEmpty} to + * forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardIsEmpty() { + return !entrySet().iterator().hasNext(); + } + + /** + * A sensible definition of {@link #equals} in terms of the {@code equals} method of {@link + * #entrySet}. If you override {@link #entrySet}, you may wish to override {@link #equals} to + * forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardEquals(Object object) { + return Maps.equalsImpl(this, object); + } + + /** + * A sensible definition of {@link #hashCode} in terms of the {@code iterator} method of {@link + * #entrySet}. If you override {@link #entrySet}, you may wish to override {@link #hashCode} to + * forward to this implementation. + * + * @since 7.0 + */ + protected int standardHashCode() { + return Sets.hashCodeImpl(entrySet()); + } + + /** + * A sensible definition of {@link #toString} in terms of the {@code iterator} method of {@link + * #entrySet}. If you override {@link #entrySet}, you may wish to override {@link #toString} to + * forward to this implementation. + * + * @since 7.0 + */ + protected String standardToString() { + return Maps.toStringImpl(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMapEntry.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMapEntry.java new file mode 100644 index 0000000000000..73b0a334eea3d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMapEntry.java @@ -0,0 +1,126 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.util.Map.Entry; + + +/** + * A map entry which forwards all its method calls to another map entry. Subclasses should override + * one or more methods to modify the behavior of the backing map entry as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingMapEntry} forward indiscriminately to + * the methods of the delegate. For example, overriding {@link #getValue} alone will not + * change the behavior of {@link #equals}, which can lead to unexpected behavior. In this case, you + * should override {@code equals} as well, either providing your own implementation, or delegating + * to the provided {@code standardEquals} method. + * + *

Each of the {@code standard} methods, where appropriate, use {@link Objects#equal} to test + * equality for both keys and values. This may not be the desired behavior for map implementations + * that use non-standard notions of key equality, such as the entry of a {@code SortedMap} whose + * comparator is not consistent with {@code equals}. + * + *

The {@code standard} methods are not guaranteed to be thread-safe, even when all of the + * methods that they depend on are thread-safe. + * + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingMapEntry extends ForwardingObject implements Entry { + // TODO(lowasser): identify places where thread safety is actually lost + + /** Constructor for use by subclasses. */ + protected ForwardingMapEntry() {} + + @Override + protected abstract Entry delegate(); + + @Override + public K getKey() { + return delegate().getKey(); + } + + @Override + public V getValue() { + return delegate().getValue(); + } + + @Override + public V setValue(V value) { + return delegate().setValue(value); + } + + @Override + public boolean equals(Object object) { + return delegate().equals(object); + } + + @Override + public int hashCode() { + return delegate().hashCode(); + } + + /** + * A sensible definition of {@link #equals(Object)} in terms of {@link #getKey()} and {@link + * #getValue()}. If you override either of these methods, you may wish to override {@link + * #equals(Object)} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardEquals(Object object) { + if (object instanceof Entry) { + Entry that = (Entry) object; + return Objects.equal(this.getKey(), that.getKey()) + && Objects.equal(this.getValue(), that.getValue()); + } + return false; + } + + /** + * A sensible definition of {@link #hashCode()} in terms of {@link #getKey()} and {@link + * #getValue()}. If you override either of these methods, you may wish to override {@link + * #hashCode()} to forward to this implementation. + * + * @since 7.0 + */ + protected int standardHashCode() { + K k = getKey(); + V v = getValue(); + return ((k == null) ? 0 : k.hashCode()) ^ ((v == null) ? 0 : v.hashCode()); + } + + /** + * A sensible definition of {@link #toString} in terms of {@link #getKey} and {@link #getValue}. + * If you override either of these methods, you may wish to override {@link #equals} to forward to + * this implementation. + * + * @since 7.0 + */ + + protected String standardToString() { + return getKey() + "=" + getValue(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultimap.java new file mode 100644 index 0000000000000..1cd05ae8e0997 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultimap.java @@ -0,0 +1,148 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + + +/** + * A multimap which forwards all its method calls to another multimap. Subclasses should override + * one or more methods to modify the behavior of the backing multimap as desired per the decorator pattern. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingMultimap}. + * + * @author Robert Konigsberg + * @since 2.0 + */ +public abstract class ForwardingMultimap extends ForwardingObject implements Multimap { + + /** Constructor for use by subclasses. */ + protected ForwardingMultimap() {} + + @Override + protected abstract Multimap delegate(); + + @Override + public Map> asMap() { + return delegate().asMap(); + } + + @Override + public void clear() { + delegate().clear(); + } + + @Override + public boolean containsEntry(Object key, Object value) { + return delegate().containsEntry(key, value); + } + + @Override + public boolean containsKey(Object key) { + return delegate().containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return delegate().containsValue(value); + } + + @Override + public Collection> entries() { + return delegate().entries(); + } + + @Override + public Collection get(K key) { + return delegate().get(key); + } + + @Override + public boolean isEmpty() { + return delegate().isEmpty(); + } + + @Override + public Multiset keys() { + return delegate().keys(); + } + + @Override + public Set keySet() { + return delegate().keySet(); + } + + @Override + public boolean put(K key, V value) { + return delegate().put(key, value); + } + + @Override + public boolean putAll(K key, Iterable values) { + return delegate().putAll(key, values); + } + + @Override + public boolean putAll(Multimap multimap) { + return delegate().putAll(multimap); + } + + @Override + public boolean remove(Object key, Object value) { + return delegate().remove(key, value); + } + + @Override + public Collection removeAll(Object key) { + return delegate().removeAll(key); + } + + @Override + public Collection replaceValues(K key, Iterable values) { + return delegate().replaceValues(key, values); + } + + @Override + public int size() { + return delegate().size(); + } + + @Override + public Collection values() { + return delegate().values(); + } + + @Override + public boolean equals(Object object) { + return object == this || delegate().equals(object); + } + + @Override + public int hashCode() { + return delegate().hashCode(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultiset.java new file mode 100644 index 0000000000000..f823f2b2e6bad --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingMultiset.java @@ -0,0 +1,312 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.util.Collection; +import java.util.Iterator; +import java.util.Set; + + +/** + * A multiset which forwards all its method calls to another multiset. Subclasses should override + * one or more methods to modify the behavior of the backing multiset as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingMultiset} forward indiscriminately to + * the methods of the delegate. For example, overriding {@link #add(Object, int)} alone will + * not change the behavior of {@link #add(Object)}, which can lead to unexpected behavior. In + * this case, you should override {@code add(Object)} as well, either providing your own + * implementation, or delegating to the provided {@code standardAdd} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingMultiset}. + * + *

The {@code standard} methods and any collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Kevin Bourrillion + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingMultiset extends ForwardingCollection implements Multiset { + + /** Constructor for use by subclasses. */ + protected ForwardingMultiset() {} + + @Override + protected abstract Multiset delegate(); + + @Override + public int count(Object element) { + return delegate().count(element); + } + + @Override + public int add(E element, int occurrences) { + return delegate().add(element, occurrences); + } + + @Override + public int remove(Object element, int occurrences) { + return delegate().remove(element, occurrences); + } + + @Override + public Set elementSet() { + return delegate().elementSet(); + } + + @Override + public Set> entrySet() { + return delegate().entrySet(); + } + + @Override + public boolean equals(Object object) { + return object == this || delegate().equals(object); + } + + @Override + public int hashCode() { + return delegate().hashCode(); + } + + @Override + public int setCount(E element, int count) { + return delegate().setCount(element, count); + } + + @Override + public boolean setCount(E element, int oldCount, int newCount) { + return delegate().setCount(element, oldCount, newCount); + } + + /** + * A sensible definition of {@link #contains} in terms of {@link #count}. If you override {@link + * #count}, you may wish to override {@link #contains} to forward to this implementation. + * + * @since 7.0 + */ + @Override + protected boolean standardContains(Object object) { + return count(object) > 0; + } + + /** + * A sensible definition of {@link #clear} in terms of the {@code iterator} method of {@link + * #entrySet}. If you override {@link #entrySet}, you may wish to override {@link #clear} to + * forward to this implementation. + * + * @since 7.0 + */ + @Override + protected void standardClear() { + Iterators.clear(entrySet().iterator()); + } + + /** + * A sensible, albeit inefficient, definition of {@link #count} in terms of {@link #entrySet}. If + * you override {@link #entrySet}, you may wish to override {@link #count} to forward to this + * implementation. + * + * @since 7.0 + */ + + protected int standardCount(Object object) { + for (Entry entry : this.entrySet()) { + if (Objects.equal(entry.getElement(), object)) { + return entry.getCount(); + } + } + return 0; + } + + /** + * A sensible definition of {@link #add(Object)} in terms of {@link #add(Object, int)}. If you + * override {@link #add(Object, int)}, you may wish to override {@link #add(Object)} to forward to + * this implementation. + * + * @since 7.0 + */ + protected boolean standardAdd(E element) { + add(element, 1); + return true; + } + + /** + * A sensible definition of {@link #addAll(Collection)} in terms of {@link #add(Object)} and + * {@link #add(Object, int)}. If you override either of these methods, you may wish to override + * {@link #addAll(Collection)} to forward to this implementation. + * + * @since 7.0 + */ + + @Override + protected boolean standardAddAll(Collection elementsToAdd) { + return Multisets.addAllImpl(this, elementsToAdd); + } + + /** + * A sensible definition of {@link #remove(Object)} in terms of {@link #remove(Object, int)}. If + * you override {@link #remove(Object, int)}, you may wish to override {@link #remove(Object)} to + * forward to this implementation. + * + * @since 7.0 + */ + @Override + protected boolean standardRemove(Object element) { + return remove(element, 1) > 0; + } + + /** + * A sensible definition of {@link #removeAll} in terms of the {@code removeAll} method of {@link + * #elementSet}. If you override {@link #elementSet}, you may wish to override {@link #removeAll} + * to forward to this implementation. + * + * @since 7.0 + */ + @Override + protected boolean standardRemoveAll(Collection elementsToRemove) { + return Multisets.removeAllImpl(this, elementsToRemove); + } + + /** + * A sensible definition of {@link #retainAll} in terms of the {@code retainAll} method of {@link + * #elementSet}. If you override {@link #elementSet}, you may wish to override {@link #retainAll} + * to forward to this implementation. + * + * @since 7.0 + */ + @Override + protected boolean standardRetainAll(Collection elementsToRetain) { + return Multisets.retainAllImpl(this, elementsToRetain); + } + + /** + * A sensible definition of {@link #setCount(Object, int)} in terms of {@link #count(Object)}, + * {@link #add(Object, int)}, and {@link #remove(Object, int)}. {@link #entrySet()}. If you + * override any of these methods, you may wish to override {@link #setCount(Object, int)} to + * forward to this implementation. + * + * @since 7.0 + */ + protected int standardSetCount(E element, int count) { + return Multisets.setCountImpl(this, element, count); + } + + /** + * A sensible definition of {@link #setCount(Object, int, int)} in terms of {@link #count(Object)} + * and {@link #setCount(Object, int)}. If you override either of these methods, you may wish to + * override {@link #setCount(Object, int, int)} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardSetCount(E element, int oldCount, int newCount) { + return Multisets.setCountImpl(this, element, oldCount, newCount); + } + + /** + * A sensible implementation of {@link Multiset#elementSet} in terms of the following methods: + * {@link ForwardingMultiset#clear}, {@link ForwardingMultiset#contains}, {@link + * ForwardingMultiset#containsAll}, {@link ForwardingMultiset#count}, {@link + * ForwardingMultiset#isEmpty}, the {@link Set#size} and {@link Set#iterator} methods of {@link + * ForwardingMultiset#entrySet}, and {@link ForwardingMultiset#remove(Object, int)}. In many + * situations, you may wish to override {@link ForwardingMultiset#elementSet} to forward to this + * implementation or a subclass thereof. + * + * @since 10.0 + */ + + protected class StandardElementSet extends Multisets.ElementSet { + /** Constructor for use by subclasses. */ + public StandardElementSet() {} + + @Override + Multiset multiset() { + return ForwardingMultiset.this; + } + + @Override + public Iterator iterator() { + return Multisets.elementIterator(multiset().entrySet().iterator()); + } + } + + /** + * A sensible definition of {@link #iterator} in terms of {@link #entrySet} and {@link + * #remove(Object)}. If you override either of these methods, you may wish to override {@link + * #iterator} to forward to this implementation. + * + * @since 7.0 + */ + protected Iterator standardIterator() { + return Multisets.iteratorImpl(this); + } + + /** + * A sensible, albeit inefficient, definition of {@link #size} in terms of {@link #entrySet}. If + * you override {@link #entrySet}, you may wish to override {@link #size} to forward to this + * implementation. + * + * @since 7.0 + */ + protected int standardSize() { + return Multisets.linearTimeSizeImpl(this); + } + + /** + * A sensible, albeit inefficient, definition of {@link #equals} in terms of {@code + * entrySet().size()} and {@link #count}. If you override either of these methods, you may wish to + * override {@link #equals} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardEquals(Object object) { + return Multisets.equalsImpl(this, object); + } + + /** + * A sensible definition of {@link #hashCode} as {@code entrySet().hashCode()} . If you override + * {@link #entrySet}, you may wish to override {@link #hashCode} to forward to this + * implementation. + * + * @since 7.0 + */ + protected int standardHashCode() { + return entrySet().hashCode(); + } + + /** + * A sensible definition of {@link #toString} as {@code entrySet().toString()} . If you override + * {@link #entrySet}, you may wish to override {@link #toString} to forward to this + * implementation. + * + * @since 7.0 + */ + @Override + protected String standardToString() { + return entrySet().toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableMap.java new file mode 100644 index 0000000000000..87abca1023b0c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableMap.java @@ -0,0 +1,409 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.keyOrNull; + + +import java.util.Iterator; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.NoSuchElementException; +import java.util.SortedMap; +import java.util.function.BiFunction; + +/** + * A navigable map which forwards all its method calls to another navigable map. Subclasses should + * override one or more methods to modify the behavior of the backing map as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingNavigableMap} forward indiscriminately + * to the methods of the delegate. For example, overriding {@link #put} alone will not change + * the behavior of {@link #putAll}, which can lead to unexpected behavior. In this case, you should + * override {@code putAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardPutAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingNavigableMap}. + * + *

Each of the {@code standard} methods uses the map's comparator (or the natural ordering of the + * elements, if there is no comparator) to test element equality. As a result, if the comparator is + * not consistent with equals, some of the standard implementations may violate the {@code Map} + * contract. + * + *

The {@code standard} methods and the collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Louis Wasserman + * @since 12.0 + */ +public abstract class ForwardingNavigableMap extends ForwardingSortedMap + implements NavigableMap { + + /** Constructor for use by subclasses. */ + protected ForwardingNavigableMap() {} + + @Override + protected abstract NavigableMap delegate(); + + @Override + public Entry lowerEntry(K key) { + return delegate().lowerEntry(key); + } + + /** + * A sensible definition of {@link #lowerEntry} in terms of the {@code lastEntry()} of {@link + * #headMap(Object, boolean)}. If you override {@code headMap}, you may wish to override {@code + * lowerEntry} to forward to this implementation. + */ + protected Entry standardLowerEntry(K key) { + return headMap(key, false).lastEntry(); + } + + @Override + public K lowerKey(K key) { + return delegate().lowerKey(key); + } + + /** + * A sensible definition of {@link #lowerKey} in terms of {@code lowerEntry}. If you override + * {@link #lowerEntry}, you may wish to override {@code lowerKey} to forward to this + * implementation. + */ + protected K standardLowerKey(K key) { + return keyOrNull(lowerEntry(key)); + } + + @Override + public Entry floorEntry(K key) { + return delegate().floorEntry(key); + } + + /** + * A sensible definition of {@link #floorEntry} in terms of the {@code lastEntry()} of {@link + * #headMap(Object, boolean)}. If you override {@code headMap}, you may wish to override {@code + * floorEntry} to forward to this implementation. + */ + protected Entry standardFloorEntry(K key) { + return headMap(key, true).lastEntry(); + } + + @Override + public K floorKey(K key) { + return delegate().floorKey(key); + } + + /** + * A sensible definition of {@link #floorKey} in terms of {@code floorEntry}. If you override + * {@code floorEntry}, you may wish to override {@code floorKey} to forward to this + * implementation. + */ + protected K standardFloorKey(K key) { + return keyOrNull(floorEntry(key)); + } + + @Override + public Entry ceilingEntry(K key) { + return delegate().ceilingEntry(key); + } + + /** + * A sensible definition of {@link #ceilingEntry} in terms of the {@code firstEntry()} of {@link + * #tailMap(Object, boolean)}. If you override {@code tailMap}, you may wish to override {@code + * ceilingEntry} to forward to this implementation. + */ + protected Entry standardCeilingEntry(K key) { + return tailMap(key, true).firstEntry(); + } + + @Override + public K ceilingKey(K key) { + return delegate().ceilingKey(key); + } + + /** + * A sensible definition of {@link #ceilingKey} in terms of {@code ceilingEntry}. If you override + * {@code ceilingEntry}, you may wish to override {@code ceilingKey} to forward to this + * implementation. + */ + protected K standardCeilingKey(K key) { + return keyOrNull(ceilingEntry(key)); + } + + @Override + public Entry higherEntry(K key) { + return delegate().higherEntry(key); + } + + /** + * A sensible definition of {@link #higherEntry} in terms of the {@code firstEntry()} of {@link + * #tailMap(Object, boolean)}. If you override {@code tailMap}, you may wish to override {@code + * higherEntry} to forward to this implementation. + */ + protected Entry standardHigherEntry(K key) { + return tailMap(key, false).firstEntry(); + } + + @Override + public K higherKey(K key) { + return delegate().higherKey(key); + } + + /** + * A sensible definition of {@link #higherKey} in terms of {@code higherEntry}. If you override + * {@code higherEntry}, you may wish to override {@code higherKey} to forward to this + * implementation. + */ + protected K standardHigherKey(K key) { + return keyOrNull(higherEntry(key)); + } + + @Override + public Entry firstEntry() { + return delegate().firstEntry(); + } + + /** + * A sensible definition of {@link #firstEntry} in terms of the {@code iterator()} of {@link + * #entrySet}. If you override {@code entrySet}, you may wish to override {@code firstEntry} to + * forward to this implementation. + */ + protected Entry standardFirstEntry() { + return Iterables.getFirst(entrySet(), null); + } + + /** + * A sensible definition of {@link #firstKey} in terms of {@code firstEntry}. If you override + * {@code firstEntry}, you may wish to override {@code firstKey} to forward to this + * implementation. + */ + protected K standardFirstKey() { + Entry entry = firstEntry(); + if (entry == null) { + throw new NoSuchElementException(); + } else { + return entry.getKey(); + } + } + + @Override + public Entry lastEntry() { + return delegate().lastEntry(); + } + + /** + * A sensible definition of {@link #lastEntry} in terms of the {@code iterator()} of the {@link + * #entrySet} of {@link #descendingMap}. If you override {@code descendingMap}, you may wish to + * override {@code lastEntry} to forward to this implementation. + */ + protected Entry standardLastEntry() { + return Iterables.getFirst(descendingMap().entrySet(), null); + } + + /** + * A sensible definition of {@link #lastKey} in terms of {@code lastEntry}. If you override {@code + * lastEntry}, you may wish to override {@code lastKey} to forward to this implementation. + */ + protected K standardLastKey() { + Entry entry = lastEntry(); + if (entry == null) { + throw new NoSuchElementException(); + } else { + return entry.getKey(); + } + } + + @Override + public Entry pollFirstEntry() { + return delegate().pollFirstEntry(); + } + + /** + * A sensible definition of {@link #pollFirstEntry} in terms of the {@code iterator} of {@code + * entrySet}. If you override {@code entrySet}, you may wish to override {@code pollFirstEntry} to + * forward to this implementation. + */ + protected Entry standardPollFirstEntry() { + return Iterators.pollNext(entrySet().iterator()); + } + + @Override + public Entry pollLastEntry() { + return delegate().pollLastEntry(); + } + + /** + * A sensible definition of {@link #pollFirstEntry} in terms of the {@code iterator} of the {@code + * entrySet} of {@code descendingMap}. If you override {@code descendingMap}, you may wish to + * override {@code pollFirstEntry} to forward to this implementation. + */ + protected Entry standardPollLastEntry() { + return Iterators.pollNext(descendingMap().entrySet().iterator()); + } + + @Override + public NavigableMap descendingMap() { + return delegate().descendingMap(); + } + + /** + * A sensible implementation of {@link NavigableMap#descendingMap} in terms of the methods of this + * {@code NavigableMap}. In many cases, you may wish to override {@link + * ForwardingNavigableMap#descendingMap} to forward to this implementation or a subclass thereof. + * + *

In particular, this map iterates over entries with repeated calls to {@link + * NavigableMap#lowerEntry}. If a more efficient means of iteration is available, you may wish to + * override the {@code entryIterator()} method of this class. + * + * @since 12.0 + */ + + protected class StandardDescendingMap extends Maps.DescendingMap { + /** Constructor for use by subclasses. */ + public StandardDescendingMap() {} + + @Override + NavigableMap forward() { + return ForwardingNavigableMap.this; + } + + @Override + public void replaceAll(BiFunction function) { + forward().replaceAll(function); + } + + @Override + protected Iterator> entryIterator() { + return new Iterator>() { + private Entry toRemove = null; + private Entry nextOrNull = forward().lastEntry(); + + @Override + public boolean hasNext() { + return nextOrNull != null; + } + + @Override + public Entry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + try { + return nextOrNull; + } finally { + toRemove = nextOrNull; + nextOrNull = forward().lowerEntry(nextOrNull.getKey()); + } + } + + @Override + public void remove() { + checkRemove(toRemove != null); + forward().remove(toRemove.getKey()); + toRemove = null; + } + }; + } + } + + @Override + public NavigableSet navigableKeySet() { + return delegate().navigableKeySet(); + } + + /** + * A sensible implementation of {@link NavigableMap#navigableKeySet} in terms of the methods of + * this {@code NavigableMap}. In many cases, you may wish to override {@link + * ForwardingNavigableMap#navigableKeySet} to forward to this implementation or a subclass + * thereof. + * + * @since 12.0 + */ + + protected class StandardNavigableKeySet extends Maps.NavigableKeySet { + /** Constructor for use by subclasses. */ + public StandardNavigableKeySet() { + super(ForwardingNavigableMap.this); + } + } + + @Override + public NavigableSet descendingKeySet() { + return delegate().descendingKeySet(); + } + + /** + * A sensible definition of {@link #descendingKeySet} as the {@code navigableKeySet} of {@link + * #descendingMap}. (The {@link StandardDescendingMap} implementation implements {@code + * navigableKeySet} on its own, so as not to cause an infinite loop.) If you override {@code + * descendingMap}, you may wish to override {@code descendingKeySet} to forward to this + * implementation. + */ + + protected NavigableSet standardDescendingKeySet() { + return descendingMap().navigableKeySet(); + } + + /** + * A sensible definition of {@link #subMap(Object, Object)} in terms of {@link #subMap(Object, + * boolean, Object, boolean)}. If you override {@code subMap(K, boolean, K, boolean)}, you may + * wish to override {@code subMap} to forward to this implementation. + */ + @Override + protected SortedMap standardSubMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public NavigableMap subMap(K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return delegate().subMap(fromKey, fromInclusive, toKey, toInclusive); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + return delegate().headMap(toKey, inclusive); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + return delegate().tailMap(fromKey, inclusive); + } + + /** + * A sensible definition of {@link #headMap(Object)} in terms of {@link #headMap(Object, + * boolean)}. If you override {@code headMap(K, boolean)}, you may wish to override {@code + * headMap} to forward to this implementation. + */ + protected SortedMap standardHeadMap(K toKey) { + return headMap(toKey, false); + } + + /** + * A sensible definition of {@link #tailMap(Object)} in terms of {@link #tailMap(Object, + * boolean)}. If you override {@code tailMap(K, boolean)}, you may wish to override {@code + * tailMap} to forward to this implementation. + */ + protected SortedMap standardTailMap(K fromKey) { + return tailMap(fromKey, true); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableSet.java new file mode 100644 index 0000000000000..4f5c1d4a4ef25 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingNavigableSet.java @@ -0,0 +1,237 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.SortedSet; + +/** + * A navigable set which forwards all its method calls to another navigable set. Subclasses should + * override one or more methods to modify the behavior of the backing set as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingNavigableSet} forward indiscriminately + * to the methods of the delegate. For example, overriding {@link #add} alone will not change + * the behavior of {@link #addAll}, which can lead to unexpected behavior. In this case, you should + * override {@code addAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardAddAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingNavigableSet}. + * + *

Each of the {@code standard} methods uses the set's comparator (or the natural ordering of the + * elements, if there is no comparator) to test element equality. As a result, if the comparator is + * not consistent with equals, some of the standard implementations may violate the {@code Set} + * contract. + * + *

The {@code standard} methods and the collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Louis Wasserman + * @since 12.0 + */ +public abstract class ForwardingNavigableSet extends ForwardingSortedSet + implements NavigableSet { + + /** Constructor for use by subclasses. */ + protected ForwardingNavigableSet() {} + + @Override + protected abstract NavigableSet delegate(); + + @Override + public E lower(E e) { + return delegate().lower(e); + } + + /** + * A sensible definition of {@link #lower} in terms of the {@code descendingIterator} method of + * {@link #headSet(Object, boolean)}. If you override {@link #headSet(Object, boolean)}, you may + * wish to override {@link #lower} to forward to this implementation. + */ + protected E standardLower(E e) { + return Iterators.getNext(headSet(e, false).descendingIterator(), null); + } + + @Override + public E floor(E e) { + return delegate().floor(e); + } + + /** + * A sensible definition of {@link #floor} in terms of the {@code descendingIterator} method of + * {@link #headSet(Object, boolean)}. If you override {@link #headSet(Object, boolean)}, you may + * wish to override {@link #floor} to forward to this implementation. + */ + protected E standardFloor(E e) { + return Iterators.getNext(headSet(e, true).descendingIterator(), null); + } + + @Override + public E ceiling(E e) { + return delegate().ceiling(e); + } + + /** + * A sensible definition of {@link #ceiling} in terms of the {@code iterator} method of {@link + * #tailSet(Object, boolean)}. If you override {@link #tailSet(Object, boolean)}, you may wish to + * override {@link #ceiling} to forward to this implementation. + */ + protected E standardCeiling(E e) { + return Iterators.getNext(tailSet(e, true).iterator(), null); + } + + @Override + public E higher(E e) { + return delegate().higher(e); + } + + /** + * A sensible definition of {@link #higher} in terms of the {@code iterator} method of {@link + * #tailSet(Object, boolean)}. If you override {@link #tailSet(Object, boolean)}, you may wish to + * override {@link #higher} to forward to this implementation. + */ + protected E standardHigher(E e) { + return Iterators.getNext(tailSet(e, false).iterator(), null); + } + + @Override + public E pollFirst() { + return delegate().pollFirst(); + } + + /** + * A sensible definition of {@link #pollFirst} in terms of the {@code iterator} method. If you + * override {@link #iterator} you may wish to override {@link #pollFirst} to forward to this + * implementation. + */ + protected E standardPollFirst() { + return Iterators.pollNext(iterator()); + } + + @Override + public E pollLast() { + return delegate().pollLast(); + } + + /** + * A sensible definition of {@link #pollLast} in terms of the {@code descendingIterator} method. + * If you override {@link #descendingIterator} you may wish to override {@link #pollLast} to + * forward to this implementation. + */ + protected E standardPollLast() { + return Iterators.pollNext(descendingIterator()); + } + + protected E standardFirst() { + return iterator().next(); + } + + protected E standardLast() { + return descendingIterator().next(); + } + + @Override + public NavigableSet descendingSet() { + return delegate().descendingSet(); + } + + /** + * A sensible implementation of {@link NavigableSet#descendingSet} in terms of the other methods + * of {@link NavigableSet}, notably including {@link NavigableSet#descendingIterator}. + * + *

In many cases, you may wish to override {@link ForwardingNavigableSet#descendingSet} to + * forward to this implementation or a subclass thereof. + * + * @since 12.0 + */ + + protected class StandardDescendingSet extends Sets.DescendingSet { + /** Constructor for use by subclasses. */ + public StandardDescendingSet() { + super(ForwardingNavigableSet.this); + } + } + + @Override + public Iterator descendingIterator() { + return delegate().descendingIterator(); + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return delegate().subSet(fromElement, fromInclusive, toElement, toInclusive); + } + + /** + * A sensible definition of {@link #subSet(Object, boolean, Object, boolean)} in terms of the + * {@code headSet} and {@code tailSet} methods. In many cases, you may wish to override {@link + * #subSet(Object, boolean, Object, boolean)} to forward to this implementation. + */ + + protected NavigableSet standardSubSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return tailSet(fromElement, fromInclusive).headSet(toElement, toInclusive); + } + + /** + * A sensible definition of {@link #subSet(Object, Object)} in terms of the {@link #subSet(Object, + * boolean, Object, boolean)} method. If you override {@link #subSet(Object, boolean, Object, + * boolean)}, you may wish to override {@link #subSet(Object, Object)} to forward to this + * implementation. + */ + @Override + protected SortedSet standardSubSet(E fromElement, E toElement) { + return subSet(fromElement, true, toElement, false); + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + return delegate().headSet(toElement, inclusive); + } + + /** + * A sensible definition of {@link #headSet(Object)} in terms of the {@link #headSet(Object, + * boolean)} method. If you override {@link #headSet(Object, boolean)}, you may wish to override + * {@link #headSet(Object)} to forward to this implementation. + */ + protected SortedSet standardHeadSet(E toElement) { + return headSet(toElement, false); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + return delegate().tailSet(fromElement, inclusive); + } + + /** + * A sensible definition of {@link #tailSet(Object)} in terms of the {@link #tailSet(Object, + * boolean)} method. If you override {@link #tailSet(Object, boolean)}, you may wish to override + * {@link #tailSet(Object)} to forward to this implementation. + */ + protected SortedSet standardTailSet(E fromElement) { + return tailSet(fromElement, true); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingObject.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingObject.java new file mode 100644 index 0000000000000..c1ed192f4b766 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingObject.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + +/** + * An abstract base class for implementing the decorator pattern. The {@link + * #delegate()} method must be overridden to return the instance being decorated. + * + *

This class does not forward the {@code hashCode} and {@code equals} methods through to + * the backing object, but relies on {@code Object}'s implementation. This is necessary to preserve + * the symmetry of {@code equals}. Custom definitions of equality are usually based on an interface, + * such as {@code Set} or {@code List}, so that the implementation of {@code equals} can cast the + * object being tested for equality to the custom interface. {@code ForwardingObject} implements no + * such custom interfaces directly; they are implemented only in subclasses. Therefore, forwarding + * {@code equals} would break symmetry, as the forwarding object might consider itself equal to the + * object being tested, but the reverse could not be true. This behavior is consistent with the + * JDK's collection wrappers, such as {@link java.util.Collections#unmodifiableCollection}. Use an + * interface-specific subclass of {@code ForwardingObject}, such as {@link ForwardingList}, to + * preserve equality behavior, or override {@code equals} directly. + * + *

The {@code toString} method is forwarded to the delegate. Although this class does not + * implement {@link Serializable}, a serializable subclass may be created since this class has a + * parameter-less constructor. + * + * @author Mike Bostock + * @since 2.0 + */ +public abstract class ForwardingObject { + + /** Constructor for use by subclasses. */ + protected ForwardingObject() {} + + /** + * Returns the backing delegate instance that methods are forwarded to. Abstract subclasses + * generally override this method with an abstract method that has a more specific return type, + * such as {@link ForwardingSet#delegate}. Concrete subclasses override this method to supply the + * instance being decorated. + */ + protected abstract Object delegate(); + + /** Returns the string representation generated by the delegate's {@code toString} method. */ + @Override + public String toString() { + return delegate().toString(); + } + + /* No equals or hashCode. See class comments for details. */ +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingQueue.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingQueue.java new file mode 100644 index 0000000000000..757c64250c5c5 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingQueue.java @@ -0,0 +1,122 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.NoSuchElementException; +import java.util.Queue; + +/** + * A queue which forwards all its method calls to another queue. Subclasses should override one or + * more methods to modify the behavior of the backing queue as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingQueue} forward indiscriminately to the + * methods of the delegate. For example, overriding {@link #add} alone will not change the + * behavior of {@link #offer} which can lead to unexpected behavior. In this case, you should + * override {@code offer} as well, either providing your own implementation, or delegating to the + * provided {@code standardOffer} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingQueue}. + * + *

The {@code standard} methods are not guaranteed to be thread-safe, even when all of the + * methods that they depend on are thread-safe. + * + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingQueue extends ForwardingCollection implements Queue { + + /** Constructor for use by subclasses. */ + protected ForwardingQueue() {} + + @Override + protected abstract Queue delegate(); + + @Override + public boolean offer(E o) { + return delegate().offer(o); + } + + @Override + public E poll() { + return delegate().poll(); + } + + @Override + public E remove() { + return delegate().remove(); + } + + @Override + public E peek() { + return delegate().peek(); + } + + @Override + public E element() { + return delegate().element(); + } + + /** + * A sensible definition of {@link #offer} in terms of {@link #add}. If you override {@link #add}, + * you may wish to override {@link #offer} to forward to this implementation. + * + * @since 7.0 + */ + protected boolean standardOffer(E e) { + try { + return add(e); + } catch (IllegalStateException caught) { + return false; + } + } + + /** + * A sensible definition of {@link #peek} in terms of {@link #element}. If you override {@link + * #element}, you may wish to override {@link #peek} to forward to this implementation. + * + * @since 7.0 + */ + protected E standardPeek() { + try { + return element(); + } catch (NoSuchElementException caught) { + return null; + } + } + + /** + * A sensible definition of {@link #poll} in terms of {@link #remove}. If you override {@link + * #remove}, you may wish to override {@link #poll} to forward to this implementation. + * + * @since 7.0 + */ + protected E standardPoll() { + try { + return remove(); + } catch (NoSuchElementException caught) { + return null; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSet.java new file mode 100644 index 0000000000000..674dc1cd9c290 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSet.java @@ -0,0 +1,102 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Collection; +import java.util.Set; + + +/** + * A set which forwards all its method calls to another set. Subclasses should override one or more + * methods to modify the behavior of the backing set as desired per the decorator pattern. + * + *

Warning: The methods of {@code ForwardingSet} forward indiscriminately to the + * methods of the delegate. For example, overriding {@link #add} alone will not change the + * behavior of {@link #addAll}, which can lead to unexpected behavior. In this case, you should + * override {@code addAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardAddAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingSet}. + * + *

The {@code standard} methods are not guaranteed to be thread-safe, even when all of the + * methods that they depend on are thread-safe. + * + * @author Kevin Bourrillion + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingSet extends ForwardingCollection implements Set { + // TODO(lowasser): identify places where thread safety is actually lost + + /** Constructor for use by subclasses. */ + protected ForwardingSet() {} + + @Override + protected abstract Set delegate(); + + @Override + public boolean equals(Object object) { + return object == this || delegate().equals(object); + } + + @Override + public int hashCode() { + return delegate().hashCode(); + } + + /** + * A sensible definition of {@link #removeAll} in terms of {@link #iterator} and {@link #remove}. + * If you override {@code iterator} or {@code remove}, you may wish to override {@link #removeAll} + * to forward to this implementation. + * + * @since 7.0 (this version overrides the {@code ForwardingCollection} version as of 12.0) + */ + @Override + protected boolean standardRemoveAll(Collection collection) { + return Sets.removeAllImpl(this, checkNotNull(collection)); // for GWT + } + + /** + * A sensible definition of {@link #equals} in terms of {@link #size} and {@link #containsAll}. If + * you override either of those methods, you may wish to override {@link #equals} to forward to + * this implementation. + * + * @since 7.0 + */ + protected boolean standardEquals(Object object) { + return Sets.equalsImpl(this, object); + } + + /** + * A sensible definition of {@link #hashCode} in terms of {@link #iterator}. If you override + * {@link #iterator}, you may wish to override {@link #equals} to forward to this implementation. + * + * @since 7.0 + */ + protected int standardHashCode() { + return Sets.hashCodeImpl(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSetMultimap.java new file mode 100644 index 0000000000000..13bfdbca238ad --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSetMultimap.java @@ -0,0 +1,64 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Map.Entry; +import java.util.Set; + + +/** + * A set multimap which forwards all its method calls to another set multimap. Subclasses should + * override one or more methods to modify the behavior of the backing multimap as desired per the decorator pattern. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingSetMultimap}. + * + * @author Kurt Alfred Kluever + * @since 3.0 + */ +public abstract class ForwardingSetMultimap extends ForwardingMultimap + implements SetMultimap { + + @Override + protected abstract SetMultimap delegate(); + + @Override + public Set> entries() { + return delegate().entries(); + } + + @Override + public Set get(K key) { + return delegate().get(key); + } + + @Override + public Set removeAll(Object key) { + return delegate().removeAll(key); + } + + @Override + public Set replaceValues(K key, Iterable values) { + return delegate().replaceValues(key, values); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMap.java new file mode 100644 index 0000000000000..86ef94511aeec --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMap.java @@ -0,0 +1,155 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; + + +import java.util.Comparator; +import java.util.NoSuchElementException; +import java.util.SortedMap; + + +/** + * A sorted map which forwards all its method calls to another sorted map. Subclasses should + * override one or more methods to modify the behavior of the backing sorted map as desired per the + * decorator pattern. + * + *

Warning: The methods of {@code ForwardingSortedMap} forward indiscriminately to + * the methods of the delegate. For example, overriding {@link #put} alone will not change + * the behavior of {@link #putAll}, which can lead to unexpected behavior. In this case, you should + * override {@code putAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardPutAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingSortedMap}. + * + *

Each of the {@code standard} methods, where appropriate, use the comparator of the map to test + * equality for both keys and values, unlike {@code ForwardingMap}. + * + *

The {@code standard} methods and the collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingSortedMap extends ForwardingMap + implements SortedMap { + // TODO(lowasser): identify places where thread safety is actually lost + + /** Constructor for use by subclasses. */ + protected ForwardingSortedMap() {} + + @Override + protected abstract SortedMap delegate(); + + @Override + public Comparator comparator() { + return delegate().comparator(); + } + + @Override + public K firstKey() { + return delegate().firstKey(); + } + + @Override + public SortedMap headMap(K toKey) { + return delegate().headMap(toKey); + } + + @Override + public K lastKey() { + return delegate().lastKey(); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return delegate().subMap(fromKey, toKey); + } + + @Override + public SortedMap tailMap(K fromKey) { + return delegate().tailMap(fromKey); + } + + /** + * A sensible implementation of {@link SortedMap#keySet} in terms of the methods of {@code + * ForwardingSortedMap}. In many cases, you may wish to override {@link + * ForwardingSortedMap#keySet} to forward to this implementation or a subclass thereof. + * + * @since 15.0 + */ + + protected class StandardKeySet extends Maps.SortedKeySet { + /** Constructor for use by subclasses. */ + public StandardKeySet() { + super(ForwardingSortedMap.this); + } + } + + // unsafe, but worst case is a CCE is thrown, which callers will be expecting + @SuppressWarnings({"unchecked", "rawtypes"}) + private int unsafeCompare(Object k1, Object k2) { + Comparator comparator = comparator(); + if (comparator == null) { + return ((Comparable) k1).compareTo(k2); + } else { + return ((Comparator) comparator).compare(k1, k2); + } + } + + /** + * A sensible definition of {@link #containsKey} in terms of the {@code firstKey()} method of + * {@link #tailMap}. If you override {@link #tailMap}, you may wish to override {@link + * #containsKey} to forward to this implementation. + * + * @since 7.0 + */ + @Override + + protected boolean standardContainsKey(Object key) { + try { + // any CCE will be caught + @SuppressWarnings({"unchecked", "rawtypes"}) + SortedMap self = (SortedMap) this; + Object ceilingKey = self.tailMap(key).firstKey(); + return unsafeCompare(ceilingKey, key) == 0; + } catch (ClassCastException | NoSuchElementException | NullPointerException e) { + return false; + } + } + + /** + * A sensible default implementation of {@link #subMap(Object, Object)} in terms of {@link + * #headMap(Object)} and {@link #tailMap(Object)}. In some situations, you may wish to override + * {@link #subMap(Object, Object)} to forward to this implementation. + * + * @since 7.0 + */ + + protected SortedMap standardSubMap(K fromKey, K toKey) { + checkArgument(unsafeCompare(fromKey, toKey) <= 0, "fromKey must be <= toKey"); + return tailMap(fromKey).headMap(toKey); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMultiset.java new file mode 100644 index 0000000000000..d0b8a0d94ba11 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedMultiset.java @@ -0,0 +1,228 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; + +/** + * A sorted multiset which forwards all its method calls to another sorted multiset. Subclasses + * should override one or more methods to modify the behavior of the backing multiset as desired per + * the decorator pattern. + * + *

Warning: The methods of {@code ForwardingSortedMultiset} forward + * indiscriminately to the methods of the delegate. For example, overriding {@link + * #add(Object, int)} alone will not change the behavior of {@link #add(Object)}, which can + * lead to unexpected behavior. In this case, you should override {@code add(Object)} as well, + * either providing your own implementation, or delegating to the provided {@code standardAdd} + * method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingSortedMultiset}. + * + *

The {@code standard} methods and any collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Louis Wasserman + * @since 15.0 + */ + +public abstract class ForwardingSortedMultiset extends ForwardingMultiset + implements SortedMultiset { + /** Constructor for use by subclasses. */ + protected ForwardingSortedMultiset() {} + + @Override + protected abstract SortedMultiset delegate(); + + @Override + public NavigableSet elementSet() { + return delegate().elementSet(); + } + + /** + * A sensible implementation of {@link SortedMultiset#elementSet} in terms of the following + * methods: {@link SortedMultiset#clear}, {@link SortedMultiset#comparator}, {@link + * SortedMultiset#contains}, {@link SortedMultiset#containsAll}, {@link SortedMultiset#count}, + * {@link SortedMultiset#firstEntry} {@link SortedMultiset#headMultiset}, {@link + * SortedMultiset#isEmpty}, {@link SortedMultiset#lastEntry}, {@link SortedMultiset#subMultiset}, + * {@link SortedMultiset#tailMultiset}, the {@code size()} and {@code iterator()} methods of + * {@link SortedMultiset#entrySet}, and {@link SortedMultiset#remove(Object, int)}. In many + * situations, you may wish to override {@link SortedMultiset#elementSet} to forward to this + * implementation or a subclass thereof. + * + * @since 15.0 + */ + protected class StandardElementSet extends SortedMultisets.NavigableElementSet { + /** Constructor for use by subclasses. */ + public StandardElementSet() { + super(ForwardingSortedMultiset.this); + } + } + + @Override + public Comparator comparator() { + return delegate().comparator(); + } + + @Override + public SortedMultiset descendingMultiset() { + return delegate().descendingMultiset(); + } + + /** + * A skeleton implementation of a descending multiset view. Normally, {@link + * #descendingMultiset()} will not reflect any changes you make to the behavior of methods such as + * {@link #add(Object)} or {@link #pollFirstEntry}. This skeleton implementation correctly + * delegates each of its operations to the appropriate methods of this {@code + * ForwardingSortedMultiset}. + * + *

In many cases, you may wish to override {@link #descendingMultiset()} to return an instance + * of a subclass of {@code StandardDescendingMultiset}. + * + * @since 15.0 + */ + protected abstract class StandardDescendingMultiset extends DescendingMultiset { + /** Constructor for use by subclasses. */ + public StandardDescendingMultiset() {} + + @Override + SortedMultiset forwardMultiset() { + return ForwardingSortedMultiset.this; + } + } + + @Override + public Entry firstEntry() { + return delegate().firstEntry(); + } + + /** + * A sensible definition of {@link #firstEntry()} in terms of {@code entrySet().iterator()}. + * + *

If you override {@link #entrySet()}, you may wish to override {@link #firstEntry()} to + * forward to this implementation. + */ + protected Entry standardFirstEntry() { + Iterator> entryIterator = entrySet().iterator(); + if (!entryIterator.hasNext()) { + return null; + } + Entry entry = entryIterator.next(); + return Multisets.immutableEntry(entry.getElement(), entry.getCount()); + } + + @Override + public Entry lastEntry() { + return delegate().lastEntry(); + } + + /** + * A sensible definition of {@link #lastEntry()} in terms of {@code + * descendingMultiset().entrySet().iterator()}. + * + *

If you override {@link #descendingMultiset} or {@link #entrySet()}, you may wish to override + * {@link #firstEntry()} to forward to this implementation. + */ + protected Entry standardLastEntry() { + Iterator> entryIterator = descendingMultiset().entrySet().iterator(); + if (!entryIterator.hasNext()) { + return null; + } + Entry entry = entryIterator.next(); + return Multisets.immutableEntry(entry.getElement(), entry.getCount()); + } + + @Override + public Entry pollFirstEntry() { + return delegate().pollFirstEntry(); + } + + /** + * A sensible definition of {@link #pollFirstEntry()} in terms of {@code entrySet().iterator()}. + * + *

If you override {@link #entrySet()}, you may wish to override {@link #pollFirstEntry()} to + * forward to this implementation. + */ + protected Entry standardPollFirstEntry() { + Iterator> entryIterator = entrySet().iterator(); + if (!entryIterator.hasNext()) { + return null; + } + Entry entry = entryIterator.next(); + entry = Multisets.immutableEntry(entry.getElement(), entry.getCount()); + entryIterator.remove(); + return entry; + } + + @Override + public Entry pollLastEntry() { + return delegate().pollLastEntry(); + } + + /** + * A sensible definition of {@link #pollLastEntry()} in terms of {@code + * descendingMultiset().entrySet().iterator()}. + * + *

If you override {@link #descendingMultiset()} or {@link #entrySet()}, you may wish to + * override {@link #pollLastEntry()} to forward to this implementation. + */ + protected Entry standardPollLastEntry() { + Iterator> entryIterator = descendingMultiset().entrySet().iterator(); + if (!entryIterator.hasNext()) { + return null; + } + Entry entry = entryIterator.next(); + entry = Multisets.immutableEntry(entry.getElement(), entry.getCount()); + entryIterator.remove(); + return entry; + } + + @Override + public SortedMultiset headMultiset(E upperBound, BoundType boundType) { + return delegate().headMultiset(upperBound, boundType); + } + + @Override + public SortedMultiset subMultiset( + E lowerBound, BoundType lowerBoundType, E upperBound, BoundType upperBoundType) { + return delegate().subMultiset(lowerBound, lowerBoundType, upperBound, upperBoundType); + } + + /** + * A sensible definition of {@link #subMultiset(Object, BoundType, Object, BoundType)} in terms of + * {@link #headMultiset(Object, BoundType) headMultiset} and {@link #tailMultiset(Object, + * BoundType) tailMultiset}. + * + *

If you override either of these methods, you may wish to override {@link + * #subMultiset(Object, BoundType, Object, BoundType)} to forward to this implementation. + */ + protected SortedMultiset standardSubMultiset( + E lowerBound, BoundType lowerBoundType, E upperBound, BoundType upperBoundType) { + return tailMultiset(lowerBound, lowerBoundType).headMultiset(upperBound, upperBoundType); + } + + @Override + public SortedMultiset tailMultiset(E lowerBound, BoundType boundType) { + return delegate().tailMultiset(lowerBound, boundType); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSet.java new file mode 100644 index 0000000000000..4759173702b1a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSet.java @@ -0,0 +1,164 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.SortedSet; + + +/** + * A sorted set which forwards all its method calls to another sorted set. Subclasses should + * override one or more methods to modify the behavior of the backing sorted set as desired per the + * decorator pattern. + * + *

Warning: The methods of {@code ForwardingSortedSet} forward indiscriminately to + * the methods of the delegate. For example, overriding {@link #add} alone will not change + * the behavior of {@link #addAll}, which can lead to unexpected behavior. In this case, you should + * override {@code addAll} as well, either providing your own implementation, or delegating to the + * provided {@code standardAddAll} method. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingSortedSet}. + * + *

Each of the {@code standard} methods, where appropriate, uses the set's comparator (or the + * natural ordering of the elements, if there is no comparator) to test element equality. As a + * result, if the comparator is not consistent with equals, some of the standard implementations may + * violate the {@code Set} contract. + * + *

The {@code standard} methods and the collection views they return are not guaranteed to be + * thread-safe, even when all of the methods that they depend on are thread-safe. + * + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public abstract class ForwardingSortedSet extends ForwardingSet implements SortedSet { + + /** Constructor for use by subclasses. */ + protected ForwardingSortedSet() {} + + @Override + protected abstract SortedSet delegate(); + + @Override + public Comparator comparator() { + return delegate().comparator(); + } + + @Override + public E first() { + return delegate().first(); + } + + @Override + public SortedSet headSet(E toElement) { + return delegate().headSet(toElement); + } + + @Override + public E last() { + return delegate().last(); + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return delegate().subSet(fromElement, toElement); + } + + @Override + public SortedSet tailSet(E fromElement) { + return delegate().tailSet(fromElement); + } + + // unsafe, but worst case is a CCE is thrown, which callers will be expecting + @SuppressWarnings({"unchecked", "rawtypes"}) + private int unsafeCompare(Object o1, Object o2) { + Comparator comparator = comparator(); + return (comparator == null) + ? ((Comparable) o1).compareTo(o2) + : ((Comparator) comparator).compare(o1, o2); + } + + /** + * A sensible definition of {@link #contains} in terms of the {@code first()} method of {@link + * #tailSet}. If you override {@link #tailSet}, you may wish to override {@link #contains} to + * forward to this implementation. + * + * @since 7.0 + */ + @Override + + protected boolean standardContains(Object object) { + try { + // any ClassCastExceptions are caught + @SuppressWarnings({"unchecked", "rawtypes"}) + SortedSet self = (SortedSet) this; + Object ceiling = self.tailSet(object).first(); + return unsafeCompare(ceiling, object) == 0; + } catch (ClassCastException | NoSuchElementException | NullPointerException e) { + return false; + } + } + + /** + * A sensible definition of {@link #remove} in terms of the {@code iterator()} method of {@link + * #tailSet}. If you override {@link #tailSet}, you may wish to override {@link #remove} to + * forward to this implementation. + * + * @since 7.0 + */ + @Override + + protected boolean standardRemove(Object object) { + try { + // any ClassCastExceptions are caught + @SuppressWarnings({"unchecked", "rawtypes"}) + SortedSet self = (SortedSet) this; + Iterator iterator = self.tailSet(object).iterator(); + if (iterator.hasNext()) { + Object ceiling = iterator.next(); + if (unsafeCompare(ceiling, object) == 0) { + iterator.remove(); + return true; + } + } + } catch (ClassCastException | NullPointerException e) { + return false; + } + return false; + } + + /** + * A sensible default implementation of {@link #subSet(Object, Object)} in terms of {@link + * #headSet(Object)} and {@link #tailSet(Object)}. In some situations, you may wish to override + * {@link #subSet(Object, Object)} to forward to this implementation. + * + * @since 7.0 + */ + + protected SortedSet standardSubSet(E fromElement, E toElement) { + return tailSet(fromElement).headSet(toElement); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSetMultimap.java new file mode 100644 index 0000000000000..18036da4cb339 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ForwardingSortedSetMultimap.java @@ -0,0 +1,67 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Comparator; +import java.util.SortedSet; + + +/** + * A sorted set multimap which forwards all its method calls to another sorted set multimap. + * Subclasses should override one or more methods to modify the behavior of the backing multimap as + * desired per the decorator pattern. + * + *

{@code default} method warning: This class does not forward calls to {@code + * default} methods. Instead, it inherits their default implementations. When those implementations + * invoke methods, they invoke methods on the {@code ForwardingSortedSetMultimap}. + * + * @author Kurt Alfred Kluever + * @since 3.0 + */ +public abstract class ForwardingSortedSetMultimap extends ForwardingSetMultimap + implements SortedSetMultimap { + + /** Constructor for use by subclasses. */ + protected ForwardingSortedSetMultimap() {} + + @Override + protected abstract SortedSetMultimap delegate(); + + @Override + public SortedSet get(K key) { + return delegate().get(key); + } + + @Override + public SortedSet removeAll(Object key) { + return delegate().removeAll(key); + } + + @Override + public SortedSet replaceValues(K key, Iterable values) { + return delegate().replaceValues(key, values); + } + + @Override + public Comparator valueComparator() { + return delegate().valueComparator(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GeneralRange.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GeneralRange.java new file mode 100644 index 0000000000000..432aa9015d7dd --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GeneralRange.java @@ -0,0 +1,299 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.BoundType.CLOSED; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.BoundType.OPEN; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.io.Serializable; +import java.util.Comparator; + + + +/** + * A generalized interval on any ordering, for internal use. Supports {@code null}. Unlike {@link + * Range}, this allows the use of an arbitrary comparator. This is designed for use in the + * implementation of subcollections of sorted collection types. + * + *

Whenever possible, use {@code Range} instead, which is better supported. + * + * @author Louis Wasserman + */ +final class GeneralRange implements Serializable { + /** Converts a Range to a GeneralRange. */ + @SuppressWarnings("rawtypes") + static GeneralRange from(Range range) { + T lowerEndpoint = range.hasLowerBound() ? range.lowerEndpoint() : null; + BoundType lowerBoundType = range.hasLowerBound() ? range.lowerBoundType() : OPEN; + + T upperEndpoint = range.hasUpperBound() ? range.upperEndpoint() : null; + BoundType upperBoundType = range.hasUpperBound() ? range.upperBoundType() : OPEN; + return new GeneralRange( + Ordering.natural(), + range.hasLowerBound(), + lowerEndpoint, + lowerBoundType, + range.hasUpperBound(), + upperEndpoint, + upperBoundType); + } + + /** Returns the whole range relative to the specified comparator. */ + static GeneralRange all(Comparator comparator) { + return new GeneralRange(comparator, false, null, OPEN, false, null, OPEN); + } + + /** + * Returns everything above the endpoint relative to the specified comparator, with the specified + * endpoint behavior. + */ + static GeneralRange downTo( + Comparator comparator, T endpoint, BoundType boundType) { + return new GeneralRange(comparator, true, endpoint, boundType, false, null, OPEN); + } + + /** + * Returns everything below the endpoint relative to the specified comparator, with the specified + * endpoint behavior. + */ + static GeneralRange upTo( + Comparator comparator, T endpoint, BoundType boundType) { + return new GeneralRange(comparator, false, null, OPEN, true, endpoint, boundType); + } + + /** + * Returns everything between the endpoints relative to the specified comparator, with the + * specified endpoint behavior. + */ + static GeneralRange range( + Comparator comparator, + T lower, + BoundType lowerType, + T upper, + BoundType upperType) { + return new GeneralRange(comparator, true, lower, lowerType, true, upper, upperType); + } + + private final Comparator comparator; + private final boolean hasLowerBound; + private final T lowerEndpoint; + private final BoundType lowerBoundType; + private final boolean hasUpperBound; + private final T upperEndpoint; + private final BoundType upperBoundType; + + private GeneralRange( + Comparator comparator, + boolean hasLowerBound, + T lowerEndpoint, + BoundType lowerBoundType, + boolean hasUpperBound, + T upperEndpoint, + BoundType upperBoundType) { + this.comparator = checkNotNull(comparator); + this.hasLowerBound = hasLowerBound; + this.hasUpperBound = hasUpperBound; + this.lowerEndpoint = lowerEndpoint; + this.lowerBoundType = checkNotNull(lowerBoundType); + this.upperEndpoint = upperEndpoint; + this.upperBoundType = checkNotNull(upperBoundType); + + if (hasLowerBound) { + comparator.compare(lowerEndpoint, lowerEndpoint); + } + if (hasUpperBound) { + comparator.compare(upperEndpoint, upperEndpoint); + } + if (hasLowerBound && hasUpperBound) { + int cmp = comparator.compare(lowerEndpoint, upperEndpoint); + // be consistent with Range + checkArgument( + cmp <= 0, "lowerEndpoint (%s) > upperEndpoint (%s)", lowerEndpoint, upperEndpoint); + if (cmp == 0) { + checkArgument(lowerBoundType != OPEN | upperBoundType != OPEN); + } + } + } + + Comparator comparator() { + return comparator; + } + + boolean hasLowerBound() { + return hasLowerBound; + } + + boolean hasUpperBound() { + return hasUpperBound; + } + + boolean isEmpty() { + return (hasUpperBound() && tooLow(getUpperEndpoint())) + || (hasLowerBound() && tooHigh(getLowerEndpoint())); + } + + boolean tooLow(T t) { + if (!hasLowerBound()) { + return false; + } + T lbound = getLowerEndpoint(); + int cmp = comparator.compare(t, lbound); + return cmp < 0 | (cmp == 0 & getLowerBoundType() == OPEN); + } + + boolean tooHigh(T t) { + if (!hasUpperBound()) { + return false; + } + T ubound = getUpperEndpoint(); + int cmp = comparator.compare(t, ubound); + return cmp > 0 | (cmp == 0 & getUpperBoundType() == OPEN); + } + + boolean contains(T t) { + return !tooLow(t) && !tooHigh(t); + } + + /** + * Returns the intersection of the two ranges, or an empty range if their intersection is empty. + */ + GeneralRange intersect(GeneralRange other) { + checkNotNull(other); + checkArgument(comparator.equals(other.comparator)); + + boolean hasLowBound = this.hasLowerBound; + T lowEnd = getLowerEndpoint(); + BoundType lowType = getLowerBoundType(); + if (!hasLowerBound()) { + hasLowBound = other.hasLowerBound; + lowEnd = other.getLowerEndpoint(); + lowType = other.getLowerBoundType(); + } else if (other.hasLowerBound()) { + int cmp = comparator.compare(getLowerEndpoint(), other.getLowerEndpoint()); + if (cmp < 0 || (cmp == 0 && other.getLowerBoundType() == OPEN)) { + lowEnd = other.getLowerEndpoint(); + lowType = other.getLowerBoundType(); + } + } + + boolean hasUpBound = this.hasUpperBound; + T upEnd = getUpperEndpoint(); + BoundType upType = getUpperBoundType(); + if (!hasUpperBound()) { + hasUpBound = other.hasUpperBound; + upEnd = other.getUpperEndpoint(); + upType = other.getUpperBoundType(); + } else if (other.hasUpperBound()) { + int cmp = comparator.compare(getUpperEndpoint(), other.getUpperEndpoint()); + if (cmp > 0 || (cmp == 0 && other.getUpperBoundType() == OPEN)) { + upEnd = other.getUpperEndpoint(); + upType = other.getUpperBoundType(); + } + } + + if (hasLowBound && hasUpBound) { + int cmp = comparator.compare(lowEnd, upEnd); + if (cmp > 0 || (cmp == 0 && lowType == OPEN && upType == OPEN)) { + // force allowed empty range + lowEnd = upEnd; + lowType = OPEN; + upType = CLOSED; + } + } + + return new GeneralRange(comparator, hasLowBound, lowEnd, lowType, hasUpBound, upEnd, upType); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof GeneralRange) { + GeneralRange r = (GeneralRange) obj; + return comparator.equals(r.comparator) + && hasLowerBound == r.hasLowerBound + && hasUpperBound == r.hasUpperBound + && getLowerBoundType().equals(r.getLowerBoundType()) + && getUpperBoundType().equals(r.getUpperBoundType()) + && Objects.equal(getLowerEndpoint(), r.getLowerEndpoint()) + && Objects.equal(getUpperEndpoint(), r.getUpperEndpoint()); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode( + comparator, + getLowerEndpoint(), + getLowerBoundType(), + getUpperEndpoint(), + getUpperBoundType()); + } + + private transient GeneralRange reverse; + + /** Returns the same range relative to the reversed comparator. */ + GeneralRange reverse() { + GeneralRange result = reverse; + if (result == null) { + result = + new GeneralRange( + Ordering.from(comparator).reverse(), + hasUpperBound, + getUpperEndpoint(), + getUpperBoundType(), + hasLowerBound, + getLowerEndpoint(), + getLowerBoundType()); + result.reverse = this; + return this.reverse = result; + } + return result; + } + + @Override + public String toString() { + return comparator + + ":" + + (lowerBoundType == CLOSED ? '[' : '(') + + (hasLowerBound ? lowerEndpoint : "-\u221e") + + ',' + + (hasUpperBound ? upperEndpoint : "\u221e") + + (upperBoundType == CLOSED ? ']' : ')'); + } + + T getLowerEndpoint() { + return lowerEndpoint; + } + + BoundType getLowerBoundType() { + return lowerBoundType; + } + + T getUpperEndpoint() { + return upperEndpoint; + } + + BoundType getUpperBoundType() { + return upperBoundType; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GwtTransient.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GwtTransient.java new file mode 100644 index 0000000000000..8073046abe622 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/GwtTransient.java @@ -0,0 +1,38 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +/** + * Private replacement for {@link com.azure.gwt.user.client.rpc.GwtTransient} to work around + * build-system quirks. This annotation should be used only in {@code + * com.google.common.collect}. + */ +@Documented +@Retention(RUNTIME) +@Target(FIELD) +@interface GwtTransient {} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashBiMap.java new file mode 100644 index 0000000000000..ebcc2a4ffdb5a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashBiMap.java @@ -0,0 +1,740 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.Hashing.smearedHash; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.IteratorBasedAbstractMap; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; + + + +/** + * A {@link BiMap} backed by two hash tables. This implementation allows null keys and values. A + * {@code HashBiMap} and its inverse are both serializable. + * + *

This implementation guarantees insertion-based iteration order of its keys. + * + *

See the Guava User Guide article on {@code BiMap} . + * + * @author Louis Wasserman + * @author Mike Bostock + * @since 2.0 + */ +public final class HashBiMap extends IteratorBasedAbstractMap + implements BiMap, Serializable { + + /** Returns a new, empty {@code HashBiMap} with the default initial capacity (16). */ + public static HashBiMap create() { + return create(16); + } + + /** + * Constructs a new, empty bimap with the specified expected size. + * + * @param expectedSize the expected number of entries + * @throws IllegalArgumentException if the specified expected size is negative + */ + public static HashBiMap create(int expectedSize) { + return new HashBiMap<>(expectedSize); + } + + /** + * Constructs a new bimap containing initial values from {@code map}. The bimap is created with an + * initial capacity sufficient to hold the mappings in the specified map. + */ + public static HashBiMap create(Map map) { + HashBiMap bimap = create(map.size()); + bimap.putAll(map); + return bimap; + } + + private static final class BiEntry extends ImmutableEntry { + final int keyHash; + final int valueHash; + + BiEntry nextInKToVBucket; + BiEntry nextInVToKBucket; + + BiEntry nextInKeyInsertionOrder; + BiEntry prevInKeyInsertionOrder; + + BiEntry(K key, int keyHash, V value, int valueHash) { + super(key, value); + this.keyHash = keyHash; + this.valueHash = valueHash; + } + } + + private static final double LOAD_FACTOR = 1.0; + + private transient BiEntry[] hashTableKToV; + private transient BiEntry[] hashTableVToK; + private transient BiEntry firstInKeyInsertionOrder; + private transient BiEntry lastInKeyInsertionOrder; + private transient int size; + private transient int mask; + private transient int modCount; + + private HashBiMap(int expectedSize) { + init(expectedSize); + } + + private void init(int expectedSize) { + checkNonnegative(expectedSize, "expectedSize"); + int tableSize = Hashing.closedTableSize(expectedSize, LOAD_FACTOR); + this.hashTableKToV = createTable(tableSize); + this.hashTableVToK = createTable(tableSize); + this.firstInKeyInsertionOrder = null; + this.lastInKeyInsertionOrder = null; + this.size = 0; + this.mask = tableSize - 1; + this.modCount = 0; + } + + /** + * Finds and removes {@code entry} from the bucket linked lists in both the key-to-value direction + * and the value-to-key direction. + */ + private void delete(BiEntry entry) { + int keyBucket = entry.keyHash & mask; + BiEntry prevBucketEntry = null; + for (BiEntry bucketEntry = hashTableKToV[keyBucket]; + true; + bucketEntry = bucketEntry.nextInKToVBucket) { + if (bucketEntry == entry) { + if (prevBucketEntry == null) { + hashTableKToV[keyBucket] = entry.nextInKToVBucket; + } else { + prevBucketEntry.nextInKToVBucket = entry.nextInKToVBucket; + } + break; + } + prevBucketEntry = bucketEntry; + } + + int valueBucket = entry.valueHash & mask; + prevBucketEntry = null; + for (BiEntry bucketEntry = hashTableVToK[valueBucket]; + true; + bucketEntry = bucketEntry.nextInVToKBucket) { + if (bucketEntry == entry) { + if (prevBucketEntry == null) { + hashTableVToK[valueBucket] = entry.nextInVToKBucket; + } else { + prevBucketEntry.nextInVToKBucket = entry.nextInVToKBucket; + } + break; + } + prevBucketEntry = bucketEntry; + } + + if (entry.prevInKeyInsertionOrder == null) { + firstInKeyInsertionOrder = entry.nextInKeyInsertionOrder; + } else { + entry.prevInKeyInsertionOrder.nextInKeyInsertionOrder = entry.nextInKeyInsertionOrder; + } + + if (entry.nextInKeyInsertionOrder == null) { + lastInKeyInsertionOrder = entry.prevInKeyInsertionOrder; + } else { + entry.nextInKeyInsertionOrder.prevInKeyInsertionOrder = entry.prevInKeyInsertionOrder; + } + + size--; + modCount++; + } + + private void insert(BiEntry entry, BiEntry oldEntryForKey) { + int keyBucket = entry.keyHash & mask; + entry.nextInKToVBucket = hashTableKToV[keyBucket]; + hashTableKToV[keyBucket] = entry; + + int valueBucket = entry.valueHash & mask; + entry.nextInVToKBucket = hashTableVToK[valueBucket]; + hashTableVToK[valueBucket] = entry; + + if (oldEntryForKey == null) { + entry.prevInKeyInsertionOrder = lastInKeyInsertionOrder; + entry.nextInKeyInsertionOrder = null; + if (lastInKeyInsertionOrder == null) { + firstInKeyInsertionOrder = entry; + } else { + lastInKeyInsertionOrder.nextInKeyInsertionOrder = entry; + } + lastInKeyInsertionOrder = entry; + } else { + entry.prevInKeyInsertionOrder = oldEntryForKey.prevInKeyInsertionOrder; + if (entry.prevInKeyInsertionOrder == null) { + firstInKeyInsertionOrder = entry; + } else { + entry.prevInKeyInsertionOrder.nextInKeyInsertionOrder = entry; + } + entry.nextInKeyInsertionOrder = oldEntryForKey.nextInKeyInsertionOrder; + if (entry.nextInKeyInsertionOrder == null) { + lastInKeyInsertionOrder = entry; + } else { + entry.nextInKeyInsertionOrder.prevInKeyInsertionOrder = entry; + } + } + + size++; + modCount++; + } + + private BiEntry seekByKey(Object key, int keyHash) { + for (BiEntry entry = hashTableKToV[keyHash & mask]; + entry != null; + entry = entry.nextInKToVBucket) { + if (keyHash == entry.keyHash && Objects.equal(key, entry.key)) { + return entry; + } + } + return null; + } + + private BiEntry seekByValue(Object value, int valueHash) { + for (BiEntry entry = hashTableVToK[valueHash & mask]; + entry != null; + entry = entry.nextInVToKBucket) { + if (valueHash == entry.valueHash && Objects.equal(value, entry.value)) { + return entry; + } + } + return null; + } + + @Override + public boolean containsKey(Object key) { + return seekByKey(key, smearedHash(key)) != null; + } + + @Override + public boolean containsValue(Object value) { + return seekByValue(value, smearedHash(value)) != null; + } + + + @Override + public V get(Object key) { + return Maps.valueOrNull(seekByKey(key, smearedHash(key))); + } + + @Override + public V put(K key, V value) { + return put(key, value, false); + } + + private V put(K key, V value, boolean force) { + int keyHash = smearedHash(key); + int valueHash = smearedHash(value); + + BiEntry oldEntryForKey = seekByKey(key, keyHash); + if (oldEntryForKey != null + && valueHash == oldEntryForKey.valueHash + && Objects.equal(value, oldEntryForKey.value)) { + return value; + } + + BiEntry oldEntryForValue = seekByValue(value, valueHash); + if (oldEntryForValue != null) { + if (force) { + delete(oldEntryForValue); + } else { + throw new IllegalArgumentException("value already present: " + value); + } + } + + BiEntry newEntry = new BiEntry<>(key, keyHash, value, valueHash); + if (oldEntryForKey != null) { + delete(oldEntryForKey); + insert(newEntry, oldEntryForKey); + oldEntryForKey.prevInKeyInsertionOrder = null; + oldEntryForKey.nextInKeyInsertionOrder = null; + return oldEntryForKey.value; + } else { + insert(newEntry, null); + rehashIfNecessary(); + return null; + } + } + + @Override + public V forcePut(K key, V value) { + return put(key, value, true); + } + + + private K putInverse(V value, K key, boolean force) { + int valueHash = smearedHash(value); + int keyHash = smearedHash(key); + + BiEntry oldEntryForValue = seekByValue(value, valueHash); + BiEntry oldEntryForKey = seekByKey(key, keyHash); + if (oldEntryForValue != null + && keyHash == oldEntryForValue.keyHash + && Objects.equal(key, oldEntryForValue.key)) { + return key; + } else if (oldEntryForKey != null && !force) { + throw new IllegalArgumentException("key already present: " + key); + } + + /* + * The ordering here is important: if we deleted the key entry and then the value entry, + * the key entry's prev or next pointer might point to the dead value entry, and when we + * put the new entry in the key entry's position in iteration order, it might invalidate + * the linked list. + */ + + if (oldEntryForValue != null) { + delete(oldEntryForValue); + } + + if (oldEntryForKey != null) { + delete(oldEntryForKey); + } + + BiEntry newEntry = new BiEntry<>(key, keyHash, value, valueHash); + insert(newEntry, oldEntryForKey); + + if (oldEntryForKey != null) { + oldEntryForKey.prevInKeyInsertionOrder = null; + oldEntryForKey.nextInKeyInsertionOrder = null; + } + if (oldEntryForValue != null) { + oldEntryForValue.prevInKeyInsertionOrder = null; + oldEntryForValue.nextInKeyInsertionOrder = null; + } + rehashIfNecessary(); + return Maps.keyOrNull(oldEntryForValue); + } + + private void rehashIfNecessary() { + BiEntry[] oldKToV = hashTableKToV; + if (Hashing.needsResizing(size, oldKToV.length, LOAD_FACTOR)) { + int newTableSize = oldKToV.length * 2; + + this.hashTableKToV = createTable(newTableSize); + this.hashTableVToK = createTable(newTableSize); + this.mask = newTableSize - 1; + this.size = 0; + + for (BiEntry entry = firstInKeyInsertionOrder; + entry != null; + entry = entry.nextInKeyInsertionOrder) { + insert(entry, entry); + } + this.modCount++; + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private BiEntry[] createTable(int length) { + return new BiEntry[length]; + } + + @Override + public V remove(Object key) { + BiEntry entry = seekByKey(key, smearedHash(key)); + if (entry == null) { + return null; + } else { + delete(entry); + entry.prevInKeyInsertionOrder = null; + entry.nextInKeyInsertionOrder = null; + return entry.value; + } + } + + @Override + public void clear() { + size = 0; + Arrays.fill(hashTableKToV, null); + Arrays.fill(hashTableVToK, null); + firstInKeyInsertionOrder = null; + lastInKeyInsertionOrder = null; + modCount++; + } + + @Override + public int size() { + return size; + } + + abstract class Itr implements Iterator { + BiEntry next = firstInKeyInsertionOrder; + BiEntry toRemove = null; + int expectedModCount = modCount; + int remaining = size(); + + @Override + public boolean hasNext() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + return next != null && remaining > 0; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + BiEntry entry = next; + next = entry.nextInKeyInsertionOrder; + toRemove = entry; + remaining--; + return output(entry); + } + + @Override + public void remove() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + checkRemove(toRemove != null); + delete(toRemove); + expectedModCount = modCount; + toRemove = null; + } + + abstract T output(BiEntry entry); + } + + @Override + public Set keySet() { + return new KeySet(); + } + + private final class KeySet extends Maps.KeySet { + KeySet() { + super(HashBiMap.this); + } + + @Override + public Iterator iterator() { + return new Itr() { + @Override + K output(BiEntry entry) { + return entry.key; + } + }; + } + + @Override + public boolean remove(Object o) { + BiEntry entry = seekByKey(o, smearedHash(o)); + if (entry == null) { + return false; + } else { + delete(entry); + entry.prevInKeyInsertionOrder = null; + entry.nextInKeyInsertionOrder = null; + return true; + } + } + } + + @Override + public Set values() { + return inverse().keySet(); + } + + @Override + Iterator> entryIterator() { + return new Itr>() { + @Override + Entry output(BiEntry entry) { + return new MapEntry(entry); + } + + class MapEntry extends AbstractMapEntry { + BiEntry delegate; + + MapEntry(BiEntry entry) { + this.delegate = entry; + } + + @Override + public K getKey() { + return delegate.key; + } + + @Override + public V getValue() { + return delegate.value; + } + + @Override + public V setValue(V value) { + V oldValue = delegate.value; + int valueHash = smearedHash(value); + if (valueHash == delegate.valueHash && Objects.equal(value, oldValue)) { + return value; + } + checkArgument(seekByValue(value, valueHash) == null, "value already present: %s", value); + delete(delegate); + BiEntry newEntry = new BiEntry<>(delegate.key, delegate.keyHash, value, valueHash); + insert(newEntry, delegate); + delegate.prevInKeyInsertionOrder = null; + delegate.nextInKeyInsertionOrder = null; + expectedModCount = modCount; + if (toRemove == delegate) { + toRemove = newEntry; + } + delegate = newEntry; + return oldValue; + } + } + }; + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + for (BiEntry entry = firstInKeyInsertionOrder; + entry != null; + entry = entry.nextInKeyInsertionOrder) { + action.accept(entry.key, entry.value); + } + } + + @Override + public void replaceAll(BiFunction function) { + checkNotNull(function); + BiEntry oldFirst = firstInKeyInsertionOrder; + clear(); + for (BiEntry entry = oldFirst; entry != null; entry = entry.nextInKeyInsertionOrder) { + put(entry.key, function.apply(entry.key, entry.value)); + } + } + + private transient BiMap inverse; + + @Override + public BiMap inverse() { + BiMap result = inverse; + return (result == null) ? inverse = new Inverse() : result; + } + + private final class Inverse extends IteratorBasedAbstractMap + implements BiMap, Serializable { + BiMap forward() { + return HashBiMap.this; + } + + @Override + public int size() { + return size; + } + + @Override + public void clear() { + forward().clear(); + } + + @Override + public boolean containsKey(Object value) { + return forward().containsValue(value); + } + + @Override + public K get(Object value) { + return Maps.keyOrNull(seekByValue(value, smearedHash(value))); + } + + @Override + public K put(V value, K key) { + return putInverse(value, key, false); + } + + @Override + public K forcePut(V value, K key) { + return putInverse(value, key, true); + } + + @Override + public K remove(Object value) { + BiEntry entry = seekByValue(value, smearedHash(value)); + if (entry == null) { + return null; + } else { + delete(entry); + entry.prevInKeyInsertionOrder = null; + entry.nextInKeyInsertionOrder = null; + return entry.key; + } + } + + @Override + public BiMap inverse() { + return forward(); + } + + @Override + public Set keySet() { + return new InverseKeySet(); + } + + private final class InverseKeySet extends Maps.KeySet { + InverseKeySet() { + super(Inverse.this); + } + + @Override + public boolean remove(Object o) { + BiEntry entry = seekByValue(o, smearedHash(o)); + if (entry == null) { + return false; + } else { + delete(entry); + return true; + } + } + + @Override + public Iterator iterator() { + return new Itr() { + @Override + V output(BiEntry entry) { + return entry.value; + } + }; + } + } + + @Override + public Set values() { + return forward().keySet(); + } + + @Override + Iterator> entryIterator() { + return new Itr>() { + @Override + Entry output(BiEntry entry) { + return new InverseEntry(entry); + } + + class InverseEntry extends AbstractMapEntry { + BiEntry delegate; + + InverseEntry(BiEntry entry) { + this.delegate = entry; + } + + @Override + public V getKey() { + return delegate.value; + } + + @Override + public K getValue() { + return delegate.key; + } + + @Override + public K setValue(K key) { + K oldKey = delegate.key; + int keyHash = smearedHash(key); + if (keyHash == delegate.keyHash && Objects.equal(key, oldKey)) { + return key; + } + checkArgument(seekByKey(key, keyHash) == null, "value already present: %s", key); + delete(delegate); + BiEntry newEntry = + new BiEntry<>(key, keyHash, delegate.value, delegate.valueHash); + delegate = newEntry; + insert(newEntry, null); + expectedModCount = modCount; + return oldKey; + } + } + }; + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + HashBiMap.this.forEach((k, v) -> action.accept(v, k)); + } + + @Override + public void replaceAll(BiFunction function) { + checkNotNull(function); + BiEntry oldFirst = firstInKeyInsertionOrder; + clear(); + for (BiEntry entry = oldFirst; entry != null; entry = entry.nextInKeyInsertionOrder) { + put(entry.value, function.apply(entry.value, entry.key)); + } + } + + Object writeReplace() { + return new InverseSerializedForm<>(HashBiMap.this); + } + } + + private static final class InverseSerializedForm implements Serializable { + private final HashBiMap bimap; + + InverseSerializedForm(HashBiMap bimap) { + this.bimap = bimap; + } + + Object readResolve() { + return bimap.inverse(); + } + } + + /** + * @serialData the number of entries, first key, first value, second key, second value, and so on. + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + Serialization.writeMap(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + int size = Serialization.readCount(stream); + init(16); // resist hostile attempts to allocate gratuitous heap + Serialization.populateMap(this, stream, size); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimap.java new file mode 100644 index 0000000000000..96fa6091af8d4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimap.java @@ -0,0 +1,140 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Collection; +import java.util.Map; +import java.util.Set; + +/** + * Implementation of {@link Multimap} using hash tables. + * + *

The multimap does not store duplicate key-value pairs. Adding a new key-value pair equal to an + * existing key-value pair has no effect. + * + *

Keys and values may be null. All optional multimap methods are supported, and all returned + * views are modifiable. + * + *

This class is not threadsafe when any concurrent operations update the multimap. Concurrent + * read operations will work correctly. To allow concurrent update operations, wrap your multimap + * with a call to {@link Multimaps#synchronizedSetMultimap}. + * + * @author Jared Levy + * @since 2.0 + */ +public final class HashMultimap extends HashMultimapGwtSerializationDependencies { + private static final int DEFAULT_VALUES_PER_KEY = 2; + + transient int expectedValuesPerKey = DEFAULT_VALUES_PER_KEY; + + /** + * Creates a new, empty {@code HashMultimap} with the default initial capacities. + * + *

This method will soon be deprecated in favor of {@code + * MultimapBuilder.hashKeys().hashSetValues().build()}. + */ + public static HashMultimap create() { + return new HashMultimap<>(); + } + + /** + * Constructs an empty {@code HashMultimap} with enough capacity to hold the specified numbers of + * keys and values without rehashing. + * + *

This method will soon be deprecated in favor of {@code + * MultimapBuilder.hashKeys(expectedKeys).hashSetValues(expectedValuesPerKey).build()}. + * + * @param expectedKeys the expected number of distinct keys + * @param expectedValuesPerKey the expected average number of values per key + * @throws IllegalArgumentException if {@code expectedKeys} or {@code expectedValuesPerKey} is + * negative + */ + public static HashMultimap create(int expectedKeys, int expectedValuesPerKey) { + return new HashMultimap<>(expectedKeys, expectedValuesPerKey); + } + + /** + * Constructs a {@code HashMultimap} with the same mappings as the specified multimap. If a + * key-value mapping appears multiple times in the input multimap, it only appears once in the + * constructed multimap. + * + *

This method will soon be deprecated in favor of {@code + * MultimapBuilder.hashKeys().hashSetValues().build(multimap)}. + * + * @param multimap the multimap whose contents are copied to this multimap + */ + public static HashMultimap create(Multimap multimap) { + return new HashMultimap<>(multimap); + } + + private HashMultimap() { + this(12, DEFAULT_VALUES_PER_KEY); + } + + private HashMultimap(int expectedKeys, int expectedValuesPerKey) { + super(Platform.>newHashMapWithExpectedSize(expectedKeys)); + Preconditions.checkArgument(expectedValuesPerKey >= 0); + this.expectedValuesPerKey = expectedValuesPerKey; + } + + private HashMultimap(Multimap multimap) { + super(Platform.>newHashMapWithExpectedSize(multimap.keySet().size())); + putAll(multimap); + } + + /** + * {@inheritDoc} + * + *

Creates an empty {@code HashSet} for a collection of values for one key. + * + * @return a new {@code HashSet} containing a collection of values for one key + */ + @Override + Set createCollection() { + return Platform.newHashSetWithExpectedSize(expectedValuesPerKey); + } + + /** + * @serialData expectedValuesPerKey, number of distinct keys, and then for each distinct key: the + * key, number of values for that key, and the key's values + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + Serialization.writeMultimap(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + expectedValuesPerKey = DEFAULT_VALUES_PER_KEY; + int distinctKeys = Serialization.readCount(stream); + Map> map = Platform.newHashMapWithExpectedSize(12); + setMap(map); + Serialization.populateMultimap(this, stream, distinctKeys); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimapGwtSerializationDependencies.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimapGwtSerializationDependencies.java new file mode 100644 index 0000000000000..4e0287d8dfa12 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultimapGwtSerializationDependencies.java @@ -0,0 +1,39 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Map; + +/** + * A dummy superclass to support GWT serialization of the element types of a {@link HashMultimap}. + * The GWT supersource for this class contains a field for each type. + * + *

For details about this hack, see {@link GwtSerializationDependencies}, which takes the same + * approach but with a subclass rather than a superclass. + * + *

TODO(cpovirk): Consider applying this subclass approach to our other types. + */ +abstract class HashMultimapGwtSerializationDependencies extends AbstractSetMultimap { + HashMultimapGwtSerializationDependencies(Map> map) { + super(map); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultiset.java new file mode 100644 index 0000000000000..ab1faa35be8fd --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/HashMultiset.java @@ -0,0 +1,91 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.HashMap; + +/** + * Multiset implementation backed by a {@link HashMap}. + * + * @author Kevin Bourrillion + * @author Jared Levy + * @since 2.0 + */ +public final class HashMultiset extends AbstractMapBasedMultiset { + + /** Creates a new, empty {@code HashMultiset} using the default initial capacity. */ + public static HashMultiset create() { + return new HashMultiset(); + } + + /** + * Creates a new, empty {@code HashMultiset} with the specified expected number of distinct + * elements. + * + * @param distinctElements the expected number of distinct elements + * @throws IllegalArgumentException if {@code distinctElements} is negative + */ + public static HashMultiset create(int distinctElements) { + return new HashMultiset(distinctElements); + } + + /** + * Creates a new {@code HashMultiset} containing the specified elements. + * + *

This implementation is highly efficient when {@code elements} is itself a {@link Multiset}. + * + * @param elements the elements that the multiset should contain + */ + public static HashMultiset create(Iterable elements) { + HashMultiset multiset = create(Multisets.inferDistinctElements(elements)); + Iterables.addAll(multiset, elements); + return multiset; + } + + private HashMultiset() { + super(new HashMap()); + } + + private HashMultiset(int distinctElements) { + super(Maps.newHashMapWithExpectedSize(distinctElements)); + } + + /** + * @serialData the number of distinct elements, the first element, its count, the second element, + * its count, and so on + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + Serialization.writeMultiset(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + int distinctElements = Serialization.readCount(stream); + setBackingMap(Maps.newHashMap()); + Serialization.populateMultiset(this, stream, distinctElements); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Hashing.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Hashing.java new file mode 100644 index 0000000000000..9bf7fade07c99 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Hashing.java @@ -0,0 +1,77 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; + + +/** + * Static methods for implementing hash-based collections. + * + * @author Kevin Bourrillion + * @author Jesse Wilson + * @author Austin Appleby + */ +final class Hashing { + private Hashing() {} + + /* + * These should be ints, but we need to use longs to force GWT to do the multiplications with + * enough precision. + */ + private static final long C1 = 0xcc9e2d51; + private static final long C2 = 0x1b873593; + + /* + * This method was rewritten in Java from an intermediate step of the Murmur hash function in + * http://code.google.com/p/smhasher/source/browse/trunk/MurmurHash3.cpp, which contained the + * following header: + * + * MurmurHash3 was written by Austin Appleby, and is placed in the public domain. The author + * hereby disclaims copyright to this source code. + */ + static int smear(int hashCode) { + return (int) (C2 * Integer.rotateLeft((int) (hashCode * C1), 15)); + } + + static int smearedHash(Object o) { + return smear((o == null) ? 0 : o.hashCode()); + } + + private static final int MAX_TABLE_SIZE = Ints.MAX_POWER_OF_TWO; + + static int closedTableSize(int expectedEntries, double loadFactor) { + // Get the recommended table size. + // Round down to the nearest power of 2. + expectedEntries = Math.max(expectedEntries, 2); + int tableSize = Integer.highestOneBit(expectedEntries); + // Check to make sure that we will not exceed the maximum load factor. + if (expectedEntries > (int) (loadFactor * tableSize)) { + tableSize <<= 1; + return (tableSize > 0) ? tableSize : MAX_TABLE_SIZE; + } + return tableSize; + } + + static boolean needsResizing(int size, int tableSize, double loadFactor) { + return size > loadFactor * tableSize && tableSize < MAX_TABLE_SIZE; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableAsList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableAsList.java new file mode 100644 index 0000000000000..bb250ccb6b6ca --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableAsList.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.InvalidObjectException; +import java.io.ObjectInputStream; +import java.io.Serializable; + +/** + * List returned by {@link ImmutableCollection#asList} that delegates {@code contains} checks to the + * backing collection. + * + * @author Jared Levy + * @author Louis Wasserman + */ +@SuppressWarnings("serial") +abstract class ImmutableAsList extends ImmutableList { + abstract ImmutableCollection delegateCollection(); + + @Override + public boolean contains(Object target) { + // The collection's contains() is at least as fast as ImmutableList's + // and is often faster. + return delegateCollection().contains(target); + } + + @Override + public int size() { + return delegateCollection().size(); + } + + @Override + public boolean isEmpty() { + return delegateCollection().isEmpty(); + } + + @Override + boolean isPartialView() { + return delegateCollection().isPartialView(); + } + + /** Serialized form that leads to the same performance as the original list. */ + static class SerializedForm implements Serializable { + final ImmutableCollection collection; + + SerializedForm(ImmutableCollection collection) { + this.collection = collection; + } + + Object readResolve() { + return collection.asList(); + } + + private static final long serialVersionUID = 0; + } + + private void readObject(ObjectInputStream stream) throws InvalidObjectException { + throw new InvalidObjectException("Use SerializedForm"); + } + + @Override + Object writeReplace() { + return new SerializedForm(delegateCollection()); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMap.java new file mode 100644 index 0000000000000..53c69a2994dd0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMap.java @@ -0,0 +1,433 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + + +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collector; +import java.util.stream.Collectors; + +/** + * A {@link BiMap} whose contents will never change, with many other important properties detailed + * at {@link ImmutableCollection}. + * + * @author Jared Levy + * @since 2.0 + */ +public abstract class ImmutableBiMap extends ImmutableBiMapFauxverideShim + implements BiMap { + + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableBiMap} whose keys + * and values are the result of applying the provided mapping functions to the input elements. + * Entries appear in the result {@code ImmutableBiMap} in encounter order. + * + *

If the mapped keys or values contain duplicates (according to {@link Object#equals(Object)}, + * an {@code IllegalArgumentException} is thrown when the collection operation is performed. (This + * differs from the {@code Collector} returned by {@link Collectors#toMap(Function, Function)}, + * which throws an {@code IllegalStateException}.) + * + * @since 21.0 + */ + + public static Collector> toImmutableBiMap( + Function keyFunction, + Function valueFunction) { + return CollectCollectors.toImmutableBiMap(keyFunction, valueFunction); + } + + /** Returns the empty bimap. */ + // Casting to any type is safe because the set will never hold any elements. + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableBiMap of() { + return (ImmutableBiMap) RegularImmutableBiMap.EMPTY; + } + + /** Returns an immutable bimap containing a single entry. */ + public static ImmutableBiMap of(K k1, V v1) { + return new SingletonImmutableBiMap<>(k1, v1); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys or values are added + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableBiMap of(K k1, V v1, K k2, V v2) { + return RegularImmutableBiMap.fromEntries(entryOf(k1, v1), entryOf(k2, v2)); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys or values are added + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableBiMap of(K k1, V v1, K k2, V v2, K k3, V v3) { + return RegularImmutableBiMap.fromEntries(entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3)); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys or values are added + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableBiMap of(K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + return RegularImmutableBiMap.fromEntries( + entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3), entryOf(k4, v4)); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys or values are added + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableBiMap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + return RegularImmutableBiMap.fromEntries( + entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3), entryOf(k4, v4), entryOf(k5, v5)); + } + + // looking for of() with > 5 entries? Use the builder instead. + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder<>(); + } + + /** + * Returns a new builder, expecting the specified number of entries to be added. + * + *

If {@code expectedSize} is exactly the number of entries added to the builder before {@link + * Builder#build} is called, the builder is likely to perform better than an unsized {@link + * #builder()} would have. + * + *

It is not specified if any performance benefits apply if {@code expectedSize} is close to, + * but not exactly, the number of entries added to the builder. + * + * @since 23.1 + */ + + public static Builder builderWithExpectedSize(int expectedSize) { + checkNonnegative(expectedSize, "expectedSize"); + return new Builder<>(expectedSize); + } + + /** + * A builder for creating immutable bimap instances, especially {@code public static final} bimaps + * ("constant bimaps"). Example: + * + *

{@code
+   * static final ImmutableBiMap WORD_TO_INT =
+   *     new ImmutableBiMap.Builder()
+   *         .put("one", 1)
+   *         .put("two", 2)
+   *         .put("three", 3)
+   *         .build();
+   * }
+ * + *

For small immutable bimaps, the {@code ImmutableBiMap.of()} methods are even more + * convenient. + * + *

By default, a {@code Builder} will generate bimaps that iterate over entries in the order + * they were inserted into the builder. For example, in the above example, {@code + * WORD_TO_INT.entrySet()} is guaranteed to iterate over the entries in the order {@code "one"=1, + * "two"=2, "three"=3}, and {@code keySet()} and {@code values()} respect the same order. If you + * want a different order, consider using {@link #orderEntriesByValue(Comparator)}, which changes + * this builder to sort entries by value. + * + *

Builder instances can be reused - it is safe to call {@link #build} multiple times to build + * multiple bimaps in series. Each bimap is a superset of the bimaps created before it. + * + * @since 2.0 + */ + public static final class Builder extends ImmutableMap.Builder { + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableBiMap#builder}. + */ + public Builder() {} + + Builder(int size) { + super(size); + } + + /** + * Associates {@code key} with {@code value} in the built bimap. Duplicate keys or values are + * not allowed, and will cause {@link #build} to fail. + */ + @Override + public Builder put(K key, V value) { + super.put(key, value); + return this; + } + + /** + * Adds the given {@code entry} to the bimap. Duplicate keys or values are not allowed, and will + * cause {@link #build} to fail. + * + * @since 19.0 + */ + @Override + public Builder put(Entry entry) { + super.put(entry); + return this; + } + + /** + * Associates all of the given map's keys and values in the built bimap. Duplicate keys or + * values are not allowed, and will cause {@link #build} to fail. + * + * @throws NullPointerException if any key or value in {@code map} is null + */ + @Override + public Builder putAll(Map map) { + super.putAll(map); + return this; + } + + /** + * Adds all of the given entries to the built bimap. Duplicate keys or values are not allowed, + * and will cause {@link #build} to fail. + * + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + @Override + public Builder putAll(Iterable> entries) { + super.putAll(entries); + return this; + } + + /** + * Configures this {@code Builder} to order entries by value according to the specified + * comparator. + * + *

The sort order is stable, that is, if two entries have values that compare as equivalent, + * the entry that was inserted first will be first in the built map's iteration order. + * + * @throws IllegalStateException if this method was already called + * @since 19.0 + */ + + @Override + public Builder orderEntriesByValue(Comparator valueComparator) { + super.orderEntriesByValue(valueComparator); + return this; + } + + @Override + Builder combine(ImmutableMap.Builder builder) { + super.combine(builder); + return this; + } + + /** + * Returns a newly-created immutable bimap. The iteration order of the returned bimap is the + * order in which entries were inserted into the builder, unless {@link #orderEntriesByValue} + * was called, in which case entries are sorted by value. + * + * @throws IllegalArgumentException if duplicate keys or values were added + */ + @Override + public ImmutableBiMap build() { + switch (size) { + case 0: + return of(); + case 1: + return of(entries[0].getKey(), entries[0].getValue()); + default: + /* + * If entries is full, or if hash flooding is detected, then this implementation may end + * up using the entries array directly and writing over the entry objects with + * non-terminal entries, but this is safe; if this Builder is used further, it will grow + * the entries array (so it can't affect the original array), and future build() calls + * will always copy any entry objects that cannot be safely reused. + */ + if (valueComparator != null) { + if (entriesUsed) { + entries = Arrays.copyOf(entries, size); + } + Arrays.sort( + entries, + 0, + size, + Ordering.from(valueComparator).onResultOf(Maps.valueFunction())); + } + entriesUsed = true; + return RegularImmutableBiMap.fromEntryArray(size, entries); + } + } + + @Override + + ImmutableBiMap buildJdkBacked() { + checkState( + valueComparator == null, + "buildJdkBacked is for tests only, doesn't support orderEntriesByValue"); + switch (size) { + case 0: + return of(); + case 1: + return of(entries[0].getKey(), entries[0].getValue()); + default: + entriesUsed = true; + return RegularImmutableBiMap.fromEntryArray(size, entries); + } + } + } + + /** + * Returns an immutable bimap containing the same entries as {@code map}. If {@code map} somehow + * contains entries with duplicate keys (for example, if it is a {@code SortedMap} whose + * comparator is not consistent with equals), the results of this method are undefined. + * + *

The returned {@code BiMap} iterates over entries in the same order as the {@code entrySet} + * of the original map. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws IllegalArgumentException if two keys have the same value or two values have the same + * key + * @throws NullPointerException if any key or value in {@code map} is null + */ + public static ImmutableBiMap copyOf(Map map) { + if (map instanceof ImmutableBiMap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe since map is not writable + ImmutableBiMap bimap = (ImmutableBiMap) map; + // TODO(lowasser): if we need to make a copy of a BiMap because the + // forward map is a view, don't make a copy of the non-view delegate map + if (!bimap.isPartialView()) { + return bimap; + } + } + return copyOf(map.entrySet()); + } + + /** + * Returns an immutable bimap containing the given entries. The returned bimap iterates over + * entries in the same order as the original iterable. + * + * @throws IllegalArgumentException if two keys have the same value or two values have the same + * key + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + public static ImmutableBiMap copyOf( + Iterable> entries) { + @SuppressWarnings({"unchecked", "rawtypes"}) // we'll only be using getKey and getValue, which are covariant + Entry[] entryArray = (Entry[]) Iterables.toArray(entries, EMPTY_ENTRY_ARRAY); + switch (entryArray.length) { + case 0: + return of(); + case 1: + Entry entry = entryArray[0]; + return of(entry.getKey(), entry.getValue()); + default: + /* + * The current implementation will end up using entryArray directly, though it will write + * over the (arbitrary, potentially mutable) Entry objects actually stored in entryArray. + */ + return RegularImmutableBiMap.fromEntries(entryArray); + } + } + + ImmutableBiMap() {} + + /** + * {@inheritDoc} + * + *

The inverse of an {@code ImmutableBiMap} is another {@code ImmutableBiMap}. + */ + @Override + public abstract ImmutableBiMap inverse(); + + /** + * Returns an immutable set of the values in this map, in the same order they appear in {@link + * #entrySet}. + */ + @Override + public ImmutableSet values() { + return inverse().keySet(); + } + + @Override + final ImmutableSet createValues() { + throw new AssertionError("should never be called"); + } + + /** + * Guaranteed to throw an exception and leave the bimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public V forcePut(K key, V value) { + throw new UnsupportedOperationException(); + } + + /** + * Serialized type for all ImmutableBiMap instances. It captures the logical contents and they are + * reconstructed using public factory methods. This ensures that the implementation types remain + * as implementation details. + * + *

Since the bimap is immutable, ImmutableBiMap doesn't require special logic for keeping the + * bimap and its inverse in sync during serialization, the way AbstractBiMap does. + */ + private static class SerializedForm extends ImmutableMap.SerializedForm { + SerializedForm(ImmutableBiMap bimap) { + super(bimap); + } + + @Override + Object readResolve() { + Builder builder = new Builder<>(); + return createMap(builder); + } + + private static final long serialVersionUID = 0; + } + + @Override + Object writeReplace() { + return new SerializedForm(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMapFauxverideShim.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMapFauxverideShim.java new file mode 100644 index 0000000000000..cfa7a26d88204 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableBiMapFauxverideShim.java @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2015 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.function.BinaryOperator; +import java.util.function.Function; +import java.util.stream.Collector; + +/** + * "Overrides" the {@link ImmutableMap} static methods that lack {@link ImmutableBiMap} equivalents + * with deprecated, exception-throwing versions. See {@link ImmutableSortedSetFauxverideShim} for + * details. + * + * @author Louis Wasserman + */ +abstract class ImmutableBiMapFauxverideShim extends ImmutableMap { + /** + * Not supported. Use {@link ImmutableBiMap#toImmutableBiMap} instead. This method exists only to + * hide {@link ImmutableMap#toImmutableMap(Function, Function)} from consumers of {@code + * ImmutableBiMap}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableBiMap#toImmutableBiMap}. + */ + @Deprecated + public static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. This method does not make sense for {@code BiMap}. This method exists only to + * hide {@link ImmutableMap#toImmutableMap(Function, Function, BinaryOperator)} from consumers of + * {@code ImmutableBiMap}. + * + * @throws UnsupportedOperationException always + * @deprecated + */ + @Deprecated + public static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction, + BinaryOperator mergeFunction) { + throw new UnsupportedOperationException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableClassToInstanceMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableClassToInstanceMap.java new file mode 100644 index 0000000000000..6ad2540335df9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableClassToInstanceMap.java @@ -0,0 +1,191 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Primitives; +import java.io.Serializable; +import java.util.Map; + + +/** + * A {@link ClassToInstanceMap} whose contents will never change, with many other important + * properties detailed at {@link ImmutableCollection}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public final class ImmutableClassToInstanceMap extends ForwardingMap, B> + implements ClassToInstanceMap, Serializable { + + private static final ImmutableClassToInstanceMap EMPTY = + new ImmutableClassToInstanceMap<>(ImmutableMap., Object>of()); + + /** + * Returns an empty {@code ImmutableClassToInstanceMap}. + * + * @since 19.0 + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableClassToInstanceMap of() { + return (ImmutableClassToInstanceMap) EMPTY; + } + + /** + * Returns an {@code ImmutableClassToInstanceMap} containing a single entry. + * + * @since 19.0 + */ + public static ImmutableClassToInstanceMap of(Class type, T value) { + ImmutableMap, B> map = ImmutableMap., B>of(type, value); + return new ImmutableClassToInstanceMap(map); + } + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * A builder for creating immutable class-to-instance maps. Example: + * + *
{@code
+   * static final ImmutableClassToInstanceMap HANDLERS =
+   *     new ImmutableClassToInstanceMap.Builder()
+   *         .put(FooHandler.class, new FooHandler())
+   *         .put(BarHandler.class, new SubBarHandler())
+   *         .put(Handler.class, new QuuxHandler())
+   *         .build();
+   * }
+ * + *

After invoking {@link #build()} it is still possible to add more entries and build again. + * Thus each map generated by this builder will be a superset of any map generated before it. + * + * @since 2.0 + */ + public static final class Builder { + private final ImmutableMap.Builder, B> mapBuilder = ImmutableMap.builder(); + + /** + * Associates {@code key} with {@code value} in the built map. Duplicate keys are not allowed, + * and will cause {@link #build} to fail. + */ + public Builder put(Class key, T value) { + mapBuilder.put(key, value); + return this; + } + + /** + * Associates all of {@code map's} keys and values in the built map. Duplicate keys are not + * allowed, and will cause {@link #build} to fail. + * + * @throws NullPointerException if any key or value in {@code map} is null + * @throws ClassCastException if any value is not an instance of the type specified by its key + */ + public Builder putAll(Map, ? extends T> map) { + for (Entry, ? extends T> entry : map.entrySet()) { + Class type = entry.getKey(); + T value = entry.getValue(); + mapBuilder.put(type, cast(type, value)); + } + return this; + } + + private static T cast(Class type, B value) { + return Primitives.wrap(type).cast(value); + } + + /** + * Returns a new immutable class-to-instance map containing the entries provided to this + * builder. + * + * @throws IllegalArgumentException if duplicate keys were added + */ + public ImmutableClassToInstanceMap build() { + ImmutableMap, B> map = mapBuilder.build(); + if (map.isEmpty()) { + return of(); + } else { + return new ImmutableClassToInstanceMap(map); + } + } + } + + /** + * Returns an immutable map containing the same entries as {@code map}. If {@code map} somehow + * contains entries with duplicate keys (for example, if it is a {@code SortedMap} whose + * comparator is not consistent with equals), the results of this method are undefined. + * + *

Note: Despite what the method name suggests, if {@code map} is an {@code + * ImmutableClassToInstanceMap}, no copy will actually be performed. + * + * @throws NullPointerException if any key or value in {@code map} is null + * @throws ClassCastException if any value is not an instance of the type specified by its key + */ + public static ImmutableClassToInstanceMap copyOf( + Map, ? extends S> map) { + if (map instanceof ImmutableClassToInstanceMap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // covariant casts safe (unmodifiable) + // Eclipse won't compile if we cast to the parameterized type. + ImmutableClassToInstanceMap cast = (ImmutableClassToInstanceMap) map; + return cast; + } + return new Builder().putAll(map).build(); + } + + private final ImmutableMap, B> delegate; + + private ImmutableClassToInstanceMap(ImmutableMap, B> delegate) { + this.delegate = delegate; + } + + @Override + protected Map, B> delegate() { + return delegate; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) // value could not get in if not a T + + public T getInstance(Class type) { + return (T) delegate.get(checkNotNull(type)); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public T putInstance(Class type, T value) { + throw new UnsupportedOperationException(); + } + + Object readResolve() { + return isEmpty() ? of() : this; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableCollection.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableCollection.java new file mode 100644 index 0000000000000..3768252241700 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableCollection.java @@ -0,0 +1,441 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.ObjectArrays.checkElementsNotNull; + +import java.io.Serializable; +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.Predicate; + + +/** + * A {@link Collection} whose contents will never change, and which offers a few additional + * guarantees detailed below. + * + *

Warning: avoid direct usage of {@link ImmutableCollection} as a type (just as + * with {@link Collection} itself). Prefer subtypes such as {@link ImmutableSet} or {@link + * ImmutableList}, which have well-defined {@link #equals} semantics, thus avoiding a common source + * of bugs and confusion. + * + *

About all {@code Immutable-} collections

+ * + *

The remainder of this documentation applies to every public {@code Immutable-} type in this + * package, whether it is a subtype of {@code ImmutableCollection} or not. + * + *

Guarantees

+ * + *

Each makes the following guarantees: + * + *

    + *
  • Shallow immutability. Elements can never be added, removed or replaced in this + * collection. This is a stronger guarantee than that of {@link + * Collections#unmodifiableCollection}, whose contents change whenever the wrapped collection + * is modified. + *
  • Null-hostility. This collection will never contain a null element. + *
  • Deterministic iteration. The iteration order is always well-defined, depending on + * how the collection was created. Typically this is insertion order unless an explicit + * ordering is otherwise specified (e.g. {@link ImmutableSortedSet#naturalOrder}). See the + * appropriate factory method for details. View collections such as {@link + * ImmutableMultiset#elementSet} iterate in the same order as the parent, except as noted. + *
  • Thread safety. It is safe to access this collection concurrently from multiple + * threads. + *
  • Integrity. This type cannot be subclassed outside this package (which would allow + * these guarantees to be violated). + *
+ * + *

"Interfaces", not implementations

+ * + *

These are classes instead of interfaces to prevent external subtyping, but should be thought + * of as interfaces in every important sense. Each public class such as {@link ImmutableSet} is a + * type offering meaningful behavioral guarantees. This is substantially different from the + * case of (say) {@link HashSet}, which is an implementation, with semantics that were + * largely defined by its supertype. + * + *

For field types and method return types, you should generally use the immutable type (such as + * {@link ImmutableList}) instead of the general collection interface type (such as {@link List}). + * This communicates to your callers all of the semantic guarantees listed above, which is almost + * always very useful information. + * + *

On the other hand, a parameter type of {@link ImmutableList} is generally a nuisance to + * callers. Instead, accept {@link Iterable} and have your method or constructor body pass it to the + * appropriate {@code copyOf} method itself. + * + *

Expressing the immutability guarantee directly in the type that user code references is a + * powerful advantage. Although Java 9 offers certain immutable collection factory methods, like {@code Set.of}, + * we recommend continuing to use these immutable collection classes for this reason. + * + *

Creation

+ * + *

Except for logically "abstract" types like {@code ImmutableCollection} itself, each {@code + * Immutable} type provides the static operations you need to obtain instances of that type. These + * usually include: + * + *

    + *
  • Static methods named {@code of}, accepting an explicit list of elements or entries. + *
  • Static methods named {@code copyOf} (or {@code copyOfSorted}), accepting an existing + * collection whose contents should be copied. + *
  • A static nested {@code Builder} class which can be used to populate a new immutable + * instance. + *
+ * + *

Warnings

+ * + *
    + *
  • Warning: as with any collection, it is almost always a bad idea to modify an element + * (in a way that affects its {@link Object#equals} behavior) while it is contained in a + * collection. Undefined behavior and bugs will result. It's generally best to avoid using + * mutable objects as elements at all, as many users may expect your "immutable" object to be + * deeply immutable. + *
+ * + *

Performance notes

+ * + *
    + *
  • Implementations can be generally assumed to prioritize memory efficiency, then speed of + * access, and lastly speed of creation. + *
  • The {@code copyOf} methods will sometimes recognize that the actual copy operation is + * unnecessary; for example, {@code copyOf(copyOf(anArrayList))} should copy the data only + * once. This reduces the expense of habitually making defensive copies at API boundaries. + * However, the precise conditions for skipping the copy operation are undefined. + *
  • Warning: a view collection such as {@link ImmutableMap#keySet} or {@link + * ImmutableList#subList} may retain a reference to the entire data set, preventing it from + * being garbage collected. If some of the data is no longer reachable through other means, + * this constitutes a memory leak. Pass the view collection to the appropriate {@code copyOf} + * method to obtain a correctly-sized copy. + *
  • The performance of using the associated {@code Builder} class can be assumed to be no + * worse, and possibly better, than creating a mutable collection and copying it. + *
  • Implementations generally do not cache hash codes. If your element or key type has a slow + * {@code hashCode} implementation, it should cache it itself. + *
+ * + *

Example usage

+ * + *
{@code
+ * class Foo {
+ *   private static final ImmutableSet RESERVED_CODES =
+ *       ImmutableSet.of("AZ", "CQ", "ZX");
+ *
+ *   private final ImmutableSet codes;
+ *
+ *   public Foo(Iterable codes) {
+ *     this.codes = ImmutableSet.copyOf(codes);
+ *     checkArgument(Collections.disjoint(this.codes, RESERVED_CODES));
+ *   }
+ * }
+ * }
+ * + *

See also

+ * + *

See the Guava User Guide article on immutable collections. + * + * @since 2.0 + */ +@SuppressWarnings("serial") // we're overriding default serialization +// TODO(kevinb): I think we should push everything down to "BaseImmutableCollection" or something, +// just to do everything we can to emphasize the "practically an interface" nature of this class. +public abstract class ImmutableCollection extends AbstractCollection implements Serializable { + /* + * We expect SIZED (and SUBSIZED, if applicable) to be added by the spliterator factory methods. + * These are properties of the collection as a whole; SIZED and SUBSIZED are more properties of + * the spliterator implementation. + */ + static final int SPLITERATOR_CHARACTERISTICS = + Spliterator.IMMUTABLE | Spliterator.NONNULL | Spliterator.ORDERED; + + ImmutableCollection() {} + + /** Returns an unmodifiable iterator across the elements in this collection. */ + @Override + public abstract UnmodifiableIterator iterator(); + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(this, SPLITERATOR_CHARACTERISTICS); + } + + private static final Object[] EMPTY_ARRAY = {}; + + @Override + public final Object[] toArray() { + int size = size(); + if (size == 0) { + return EMPTY_ARRAY; + } + Object[] result = new Object[size]; + copyIntoArray(result, 0); + return result; + } + + @Override + public final T[] toArray(T[] other) { + checkNotNull(other); + int size = size(); + if (other.length < size) { + other = ObjectArrays.newArray(other, size); + } else if (other.length > size) { + other[size] = null; + } + copyIntoArray(other, 0); + return other; + } + + @Override + public abstract boolean contains(Object object); + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean add(E e) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean remove(Object object) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean addAll(Collection newElements) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean removeAll(Collection oldElements) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean removeIf(Predicate filter) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean retainAll(Collection elementsToKeep) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void clear() { + throw new UnsupportedOperationException(); + } + + /** + * Returns an {@code ImmutableList} containing the same elements, in the same order, as this + * collection. + * + *

Performance note: in most cases this method can return quickly without actually + * copying anything. The exact circumstances under which the copy is performed are undefined and + * subject to change. + * + * @since 2.0 + */ + public ImmutableList asList() { + switch (size()) { + case 0: + return ImmutableList.of(); + case 1: + return ImmutableList.of(iterator().next()); + default: + return new RegularImmutableAsList(this, toArray()); + } + } + + /** + * Returns {@code true} if this immutable collection's implementation contains references to + * user-created objects that aren't accessible via this collection's methods. This is generally + * used to determine whether {@code copyOf} implementations should make an explicit copy to avoid + * memory leaks. + */ + abstract boolean isPartialView(); + + /** + * Copies the contents of this immutable collection into the specified array at the specified + * offset. Returns {@code offset + size()}. + */ + int copyIntoArray(Object[] dst, int offset) { + for (E e : this) { + dst[offset++] = e; + } + return offset; + } + + Object writeReplace() { + // We serialize by default to ImmutableList, the simplest thing that works. + return new ImmutableList.SerializedForm(toArray()); + } + + /** + * Abstract base class for builders of {@link ImmutableCollection} types. + * + * @since 10.0 + */ + public abstract static class Builder { + static final int DEFAULT_INITIAL_CAPACITY = 4; + + static int expandedCapacity(int oldCapacity, int minCapacity) { + if (minCapacity < 0) { + throw new AssertionError("cannot store more than MAX_VALUE elements"); + } + // careful of overflow! + int newCapacity = oldCapacity + (oldCapacity >> 1) + 1; + if (newCapacity < minCapacity) { + newCapacity = Integer.highestOneBit(minCapacity - 1) << 1; + } + if (newCapacity < 0) { + newCapacity = Integer.MAX_VALUE; + // guaranteed to be >= newCapacity + } + return newCapacity; + } + + Builder() {} + + /** + * Adds {@code element} to the {@code ImmutableCollection} being built. + * + *

Note that each builder class covariantly returns its own type from this method. + * + * @param element the element to add + * @return this {@code Builder} instance + * @throws NullPointerException if {@code element} is null + */ + public abstract Builder add(E element); + + /** + * Adds each element of {@code elements} to the {@code ImmutableCollection} being built. + * + *

Note that each builder class overrides this method in order to covariantly return its own + * type. + * + * @param elements the elements to add + * @return this {@code Builder} instance + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @SuppressWarnings("unchecked") + public Builder add(E... elements) { + for (E element : elements) { + add(element); + } + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableCollection} being built. + * + *

Note that each builder class overrides this method in order to covariantly return its own + * type. + * + * @param elements the elements to add + * @return this {@code Builder} instance + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + public Builder addAll(Iterable elements) { + for (E element : elements) { + add(element); + } + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableCollection} being built. + * + *

Note that each builder class overrides this method in order to covariantly return its own + * type. + * + * @param elements the elements to add + * @return this {@code Builder} instance + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + public Builder addAll(Iterator elements) { + while (elements.hasNext()) { + add(elements.next()); + } + return this; + } + + /** + * Returns a newly-created {@code ImmutableCollection} of the appropriate type, containing the + * elements provided to this builder. + * + *

Note that each builder class covariantly returns the appropriate type of {@code + * ImmutableCollection} from this method. + */ + public abstract ImmutableCollection build(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEntry.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEntry.java new file mode 100644 index 0000000000000..61d74ba872070 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEntry.java @@ -0,0 +1,54 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + + +/** @see Maps#immutableEntry(Object, Object) */ +class ImmutableEntry extends AbstractMapEntry implements Serializable { + final K key; + final V value; + + ImmutableEntry(K key, V value) { + this.key = key; + this.value = value; + } + + + @Override + public final K getKey() { + return key; + } + + + @Override + public final V getValue() { + return value; + } + + @Override + public final V setValue(V value) { + throw new UnsupportedOperationException(); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumMap.java new file mode 100644 index 0000000000000..db2a92e447efb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumMap.java @@ -0,0 +1,137 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap.IteratorBasedImmutableMap; + +import java.io.Serializable; +import java.util.EnumMap; +import java.util.Spliterator; +import java.util.function.BiConsumer; + + +/** + * Implementation of {@link ImmutableMap} backed by a non-empty {@link EnumMap}. + * + * @author Louis Wasserman + */ +@SuppressWarnings("serial") // we're overriding default serialization +final class ImmutableEnumMap, V> extends IteratorBasedImmutableMap { + static , V> ImmutableMap asImmutable(EnumMap map) { + switch (map.size()) { + case 0: + return ImmutableMap.of(); + case 1: + Entry entry = Iterables.getOnlyElement(map.entrySet()); + return ImmutableMap.of(entry.getKey(), entry.getValue()); + default: + return new ImmutableEnumMap<>(map); + } + } + + private final transient EnumMap delegate; + + private ImmutableEnumMap(EnumMap delegate) { + this.delegate = delegate; + checkArgument(!delegate.isEmpty()); + } + + @Override + UnmodifiableIterator keyIterator() { + return Iterators.unmodifiableIterator(delegate.keySet().iterator()); + } + + @Override + Spliterator keySpliterator() { + return delegate.keySet().spliterator(); + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean containsKey(Object key) { + return delegate.containsKey(key); + } + + @Override + public V get(Object key) { + return delegate.get(key); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof ImmutableEnumMap) { + object = ((ImmutableEnumMap) object).delegate; + } + return delegate.equals(object); + } + + @Override + UnmodifiableIterator> entryIterator() { + return Maps.unmodifiableEntryIterator(delegate.entrySet().iterator()); + } + + @Override + Spliterator> entrySpliterator() { + return CollectSpliterators.map(delegate.entrySet().spliterator(), Maps::unmodifiableEntry); + } + + @Override + public void forEach(BiConsumer action) { + delegate.forEach(action); + } + + @Override + boolean isPartialView() { + return false; + } + + // All callers of the constructor are restricted to >. + @Override + Object writeReplace() { + return new EnumSerializedForm<>(delegate); + } + + /* + * This class is used to serialize ImmutableEnumMap instances. + */ + private static class EnumSerializedForm, V> implements Serializable { + final EnumMap delegate; + + EnumSerializedForm(EnumMap delegate) { + this.delegate = delegate; + } + + Object readResolve() { + return new ImmutableEnumMap<>(delegate); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumSet.java new file mode 100644 index 0000000000000..e122912d1ad96 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableEnumSet.java @@ -0,0 +1,157 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.Collection; +import java.util.EnumSet; +import java.util.Spliterator; +import java.util.function.Consumer; + +/** + * Implementation of {@link ImmutableSet} backed by a non-empty {@link EnumSet}. + * + * @author Jared Levy + */ +@SuppressWarnings("serial") // we're overriding default serialization +final class ImmutableEnumSet> extends ImmutableSet { + @SuppressWarnings({"rawtypes", "unchecked"}) // necessary to compile against Java 8 + static ImmutableSet asImmutable(EnumSet set) { + switch (set.size()) { + case 0: + return ImmutableSet.of(); + case 1: + return ImmutableSet.of(Iterables.getOnlyElement(set)); + default: + return new ImmutableEnumSet(set); + } + } + + /* + * Notes on EnumSet and >: + * + * This class isn't an arbitrary ForwardingImmutableSet because we need to + * know that calling {@code clone()} during deserialization will return an + * object that no one else has a reference to, allowing us to guarantee + * immutability. Hence, we support only {@link EnumSet}. + */ + private final transient EnumSet delegate; + + private ImmutableEnumSet(EnumSet delegate) { + this.delegate = delegate; + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public UnmodifiableIterator iterator() { + return Iterators.unmodifiableIterator(delegate.iterator()); + } + + @Override + public Spliterator spliterator() { + return delegate.spliterator(); + } + + @Override + public void forEach(Consumer action) { + delegate.forEach(action); + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean contains(Object object) { + return delegate.contains(object); + } + + @Override + public boolean containsAll(Collection collection) { + if (collection instanceof ImmutableEnumSet) { + collection = ((ImmutableEnumSet) collection).delegate; + } + return delegate.containsAll(collection); + } + + @Override + public boolean isEmpty() { + return delegate.isEmpty(); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof ImmutableEnumSet) { + object = ((ImmutableEnumSet) object).delegate; + } + return delegate.equals(object); + } + + @Override + boolean isHashCodeFast() { + return true; + } + + private transient int hashCode; + + @Override + public int hashCode() { + int result = hashCode; + return (result == 0) ? hashCode = delegate.hashCode() : result; + } + + @Override + public String toString() { + return delegate.toString(); + } + + // All callers of the constructor are restricted to >. + @Override + Object writeReplace() { + return new EnumSerializedForm(delegate); + } + + /* + * This class is used to serialize ImmutableEnumSet instances. + */ + private static class EnumSerializedForm> implements Serializable { + final EnumSet delegate; + + EnumSerializedForm(EnumSet delegate) { + this.delegate = delegate; + } + + Object readResolve() { + // EJ2 #76: Write readObject() methods defensively. + return new ImmutableEnumSet(delegate.clone()); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableList.java new file mode 100644 index 0000000000000..5d8dbe849e0ba --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableList.java @@ -0,0 +1,866 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.ObjectArrays.checkElementsNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.RegularImmutableList.EMPTY; + + + +import java.io.InvalidObjectException; +import java.io.ObjectInputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.RandomAccess; +import java.util.Spliterator; +import java.util.function.Consumer; +import java.util.function.UnaryOperator; +import java.util.stream.Collector; + + +/** + * A {@link List} whose contents will never change, with many other important properties detailed at + * {@link ImmutableCollection}. + * + *

See the Guava User Guide article on immutable collections. + * + * @see ImmutableMap + * @see ImmutableSet + * @author Kevin Bourrillion + * @since 2.0 + */ +@SuppressWarnings("serial") // we're overriding default serialization +public abstract class ImmutableList extends ImmutableCollection + implements List, RandomAccess { + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableList}, in encounter order. + * + * @since 21.0 + */ + + public static Collector> toImmutableList() { + return CollectCollectors.toImmutableList(); + } + + /** + * Returns the empty immutable list. This list behaves and performs comparably to {@link + * Collections#emptyList}, and is preferable mainly for consistency and maintainability of your + * code. + */ + // Casting to any type is safe because the list will never hold any elements. + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableList of() { + return (ImmutableList) EMPTY; + } + + /** + * Returns an immutable list containing a single element. This list behaves and performs + * comparably to {@link Collections#singleton}, but will not accept a null element. It is + * preferable mainly for consistency and maintainability of your code. + * + * @throws NullPointerException if {@code element} is null + */ + public static ImmutableList of(E element) { + return new SingletonImmutableList(element); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2) { + return construct(e1, e2); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3) { + return construct(e1, e2, e3); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3, E e4) { + return construct(e1, e2, e3, e4); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3, E e4, E e5) { + return construct(e1, e2, e3, e4, e5); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3, E e4, E e5, E e6) { + return construct(e1, e2, e3, e4, e5, e6); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3, E e4, E e5, E e6, E e7) { + return construct(e1, e2, e3, e4, e5, e6, e7); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3, E e4, E e5, E e6, E e7, E e8) { + return construct(e1, e2, e3, e4, e5, e6, e7, e8); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of(E e1, E e2, E e3, E e4, E e5, E e6, E e7, E e8, E e9) { + return construct(e1, e2, e3, e4, e5, e6, e7, e8, e9); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of( + E e1, E e2, E e3, E e4, E e5, E e6, E e7, E e8, E e9, E e10) { + return construct(e1, e2, e3, e4, e5, e6, e7, e8, e9, e10); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any element is null + */ + public static ImmutableList of( + E e1, E e2, E e3, E e4, E e5, E e6, E e7, E e8, E e9, E e10, E e11) { + return construct(e1, e2, e3, e4, e5, e6, e7, e8, e9, e10, e11); + } + + // These go up to eleven. After that, you just get the varargs form, and + // whatever warnings might come along with it. :( + + /** + * Returns an immutable list containing the given elements, in order. + * + *

The array {@code others} must not be longer than {@code Integer.MAX_VALUE - 12}. + * + * @throws NullPointerException if any element is null + * @since 3.0 (source-compatible since 2.0) + */ + @SafeVarargs // For Eclipse. For internal javac we have disabled this pointless type of warning. + @SuppressWarnings("varargs") + public static ImmutableList of( + E e1, E e2, E e3, E e4, E e5, E e6, E e7, E e8, E e9, E e10, E e11, E e12, E... others) { + checkArgument( + others.length <= Integer.MAX_VALUE - 12, + "the total number of elements must fit in an int"); + Object[] array = new Object[12 + others.length]; + array[0] = e1; + array[1] = e2; + array[2] = e3; + array[3] = e4; + array[4] = e5; + array[5] = e6; + array[6] = e7; + array[7] = e8; + array[8] = e9; + array[9] = e10; + array[10] = e11; + array[11] = e12; + System.arraycopy(others, 0, array, 12, others.length); + return construct(array); + } + + /** + * Returns an immutable list containing the given elements, in order. If {@code elements} is a + * {@link Collection}, this method behaves exactly as {@link #copyOf(Collection)}; otherwise, it + * behaves exactly as {@code copyOf(elements.iterator()}. + * + * @throws NullPointerException if any of {@code elements} is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableList copyOf(Iterable elements) { + checkNotNull(elements); // TODO(kevinb): is this here only for GWT? + return (elements instanceof Collection) + ? copyOf((Collection) elements) + : copyOf(elements.iterator()); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

Note that if {@code list} is a {@code List}, then {@code ImmutableList.copyOf(list)} + * returns an {@code ImmutableList} containing each of the strings in {@code list}, while + * ImmutableList.of(list)} returns an {@code ImmutableList>} containing one element + * (the given list itself). + * + *

This method is safe to use even when {@code elements} is a synchronized or concurrent + * collection that is currently being modified by another thread. + * + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableList copyOf(Collection elements) { + if (elements instanceof ImmutableCollection) { + @SuppressWarnings({"unchecked", "rawtypes"}) // all supported methods are covariant + ImmutableList list = ((ImmutableCollection) elements).asList(); + return list.isPartialView() ? ImmutableList.asImmutableList(list.toArray()) : list; + } + return construct(elements.toArray()); + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableList copyOf(Iterator elements) { + // We special-case for 0 or 1 elements, but going further is madness. + if (!elements.hasNext()) { + return of(); + } + E first = elements.next(); + if (!elements.hasNext()) { + return of(first); + } else { + return new Builder().add(first).addAll(elements).build(); + } + } + + /** + * Returns an immutable list containing the given elements, in order. + * + * @throws NullPointerException if any of {@code elements} is null + * @since 3.0 + */ + public static ImmutableList copyOf(E[] elements) { + switch (elements.length) { + case 0: + return of(); + case 1: + return of(elements[0]); + default: + return construct(elements.clone()); + } + } + + /** + * Returns an immutable list containing the given elements, sorted according to their natural + * order. The sorting algorithm used is stable, so elements that compare as equal will stay in the + * order in which they appear in the input. + * + *

If your data has no duplicates, or you wish to deduplicate elements, use {@code + * ImmutableSortedSet.copyOf(elements)}; if you want a {@code List} you can use its {@code + * asList()} view. + * + *

Java 8 users: If you want to convert a {@link java.util.stream.Stream} to a sorted + * {@code ImmutableList}, use {@code stream.sorted().collect(toImmutableList())}. + * + * @throws NullPointerException if any element in the input is null + * @since 21.0 + */ + public static > ImmutableList sortedCopyOf( + Iterable elements) { + Comparable[] array = Iterables.toArray(elements, new Comparable[0]); + checkElementsNotNull((Object[]) array); + Arrays.sort(array); + return asImmutableList(array); + } + + /** + * Returns an immutable list containing the given elements, in sorted order relative to the + * specified comparator. The sorting algorithm used is stable, so elements that compare as equal + * will stay in the order in which they appear in the input. + * + *

If your data has no duplicates, or you wish to deduplicate elements, use {@code + * ImmutableSortedSet.copyOf(comparator, elements)}; if you want a {@code List} you can use its + * {@code asList()} view. + * + *

Java 8 users: If you want to convert a {@link java.util.stream.Stream} to a sorted + * {@code ImmutableList}, use {@code stream.sorted(comparator).collect(toImmutableList())}. + * + * @throws NullPointerException if any element in the input is null + * @since 21.0 + */ + public static ImmutableList sortedCopyOf( + Comparator comparator, Iterable elements) { + checkNotNull(comparator); + @SuppressWarnings({"unchecked", "rawtypes"}) // all supported methods are covariant + E[] array = (E[]) Iterables.toArray(elements); + checkElementsNotNull(array); + Arrays.sort(array, comparator); + return asImmutableList(array); + } + + /** Views the array as an immutable list. Checks for nulls; does not copy. */ + private static ImmutableList construct(Object... elements) { + return asImmutableList(checkElementsNotNull(elements)); + } + + /** + * Views the array as an immutable list. Does not check for nulls; does not copy. + * + *

The array must be internally created. + */ + static ImmutableList asImmutableList(Object[] elements) { + return asImmutableList(elements, elements.length); + } + + /** + * Views the array as an immutable list. Copies if the specified range does not cover the complete + * array. Does not check for nulls. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + static ImmutableList asImmutableList(Object[] elements, int length) { + switch (length) { + case 0: + return of(); + case 1: + return of((E) elements[0]); + default: + if (length < elements.length) { + elements = Arrays.copyOf(elements, length); + } + return new RegularImmutableList(elements); + } + } + + ImmutableList() {} + + // This declaration is needed to make List.iterator() and + // ImmutableCollection.iterator() consistent. + @Override + public UnmodifiableIterator iterator() { + return listIterator(); + } + + @Override + public UnmodifiableListIterator listIterator() { + return listIterator(0); + } + + @Override + public UnmodifiableListIterator listIterator(int index) { + return new AbstractIndexedListIterator(size(), index) { + @Override + protected E get(int index) { + return ImmutableList.this.get(index); + } + }; + } + + @Override + public void forEach(Consumer consumer) { + checkNotNull(consumer); + int n = size(); + for (int i = 0; i < n; i++) { + consumer.accept(get(i)); + } + } + + @Override + public int indexOf(Object object) { + return (object == null) ? -1 : Lists.indexOfImpl(this, object); + } + + @Override + public int lastIndexOf(Object object) { + return (object == null) ? -1 : Lists.lastIndexOfImpl(this, object); + } + + @Override + public boolean contains(Object object) { + return indexOf(object) >= 0; + } + + // constrain the return type to ImmutableList + + /** + * Returns an immutable list of the elements between the specified {@code fromIndex}, inclusive, + * and {@code toIndex}, exclusive. (If {@code fromIndex} and {@code toIndex} are equal, the empty + * immutable list is returned.) + */ + @Override + public ImmutableList subList(int fromIndex, int toIndex) { + checkPositionIndexes(fromIndex, toIndex, size()); + int length = toIndex - fromIndex; + if (length == size()) { + return this; + } else if (length == 0) { + return of(); + } else if (length == 1) { + return of(get(fromIndex)); + } else { + return subListUnchecked(fromIndex, toIndex); + } + } + + /** + * Called by the default implementation of {@link #subList} when {@code toIndex - fromIndex > 1}, + * after index validation has already been performed. + */ + ImmutableList subListUnchecked(int fromIndex, int toIndex) { + return new SubList(fromIndex, toIndex - fromIndex); + } + + class SubList extends ImmutableList { + final transient int offset; + final transient int length; + + SubList(int offset, int length) { + this.offset = offset; + this.length = length; + } + + @Override + public int size() { + return length; + } + + @Override + public E get(int index) { + checkElementIndex(index, length); + return ImmutableList.this.get(index + offset); + } + + @Override + public ImmutableList subList(int fromIndex, int toIndex) { + checkPositionIndexes(fromIndex, toIndex, length); + return ImmutableList.this.subList(fromIndex + offset, toIndex + offset); + } + + @Override + boolean isPartialView() { + return true; + } + } + + /** + * Guaranteed to throw an exception and leave the list unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean addAll(int index, Collection newElements) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the list unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final E set(int index, E element) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the list unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void add(int index, E element) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the list unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final E remove(int index) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the list unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void replaceAll(UnaryOperator operator) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the list unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void sort(Comparator c) { + throw new UnsupportedOperationException(); + } + + /** + * Returns this list instance. + * + * @since 2.0 + */ + @Override + public final ImmutableList asList() { + return this; + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.indexed(size(), SPLITERATOR_CHARACTERISTICS, this::get); + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + // this loop is faster for RandomAccess instances, which ImmutableLists are + int size = size(); + for (int i = 0; i < size; i++) { + dst[offset + i] = get(i); + } + return offset + size; + } + + /** + * Returns a view of this immutable list in reverse order. For example, {@code ImmutableList.of(1, + * 2, 3).reverse()} is equivalent to {@code ImmutableList.of(3, 2, 1)}. + * + * @return a view of this immutable list in reverse order + * @since 7.0 + */ + public ImmutableList reverse() { + return (size() <= 1) ? this : new ReverseImmutableList(this); + } + + private static class ReverseImmutableList extends ImmutableList { + private final transient ImmutableList forwardList; + + ReverseImmutableList(ImmutableList backingList) { + this.forwardList = backingList; + } + + private int reverseIndex(int index) { + return (size() - 1) - index; + } + + private int reversePosition(int index) { + return size() - index; + } + + @Override + public ImmutableList reverse() { + return forwardList; + } + + @Override + public boolean contains(Object object) { + return forwardList.contains(object); + } + + @Override + public int indexOf(Object object) { + int index = forwardList.lastIndexOf(object); + return (index >= 0) ? reverseIndex(index) : -1; + } + + @Override + public int lastIndexOf(Object object) { + int index = forwardList.indexOf(object); + return (index >= 0) ? reverseIndex(index) : -1; + } + + @Override + public ImmutableList subList(int fromIndex, int toIndex) { + checkPositionIndexes(fromIndex, toIndex, size()); + return forwardList.subList(reversePosition(toIndex), reversePosition(fromIndex)).reverse(); + } + + @Override + public E get(int index) { + checkElementIndex(index, size()); + return forwardList.get(reverseIndex(index)); + } + + @Override + public int size() { + return forwardList.size(); + } + + @Override + boolean isPartialView() { + return forwardList.isPartialView(); + } + } + + @Override + public boolean equals(Object obj) { + return Lists.equalsImpl(this, obj); + } + + @Override + public int hashCode() { + int hashCode = 1; + int n = size(); + for (int i = 0; i < n; i++) { + hashCode = 31 * hashCode + get(i).hashCode(); + + hashCode = ~~hashCode; + // needed to deal with GWT integer overflow + } + return hashCode; + } + + /* + * Serializes ImmutableLists as their logical contents. This ensures that + * implementation types do not leak into the serialized representation. + */ + static class SerializedForm implements Serializable { + final Object[] elements; + + SerializedForm(Object[] elements) { + this.elements = elements; + } + + Object readResolve() { + return copyOf(elements); + } + + private static final long serialVersionUID = 0; + } + + private void readObject(ObjectInputStream stream) throws InvalidObjectException { + throw new InvalidObjectException("Use SerializedForm"); + } + + @Override + Object writeReplace() { + return new SerializedForm(toArray()); + } + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Returns a new builder, expecting the specified number of elements to be added. + * + *

If {@code expectedSize} is exactly the number of elements added to the builder before {@link + * Builder#build} is called, the builder is likely to perform better than an unsized {@link + * #builder()} would have. + * + *

It is not specified if any performance benefits apply if {@code expectedSize} is close to, + * but not exactly, the number of elements added to the builder. + * + * @since 23.1 + */ + + public static Builder builderWithExpectedSize(int expectedSize) { + checkNonnegative(expectedSize, "expectedSize"); + return new Builder(expectedSize); + } + + /** + * A builder for creating immutable list instances, especially {@code public static final} lists + * ("constant lists"). Example: + * + *

{@code
+   * public static final ImmutableList GOOGLE_COLORS
+   *     = new ImmutableList.Builder()
+   *         .addAll(WEBSAFE_COLORS)
+   *         .add(new Color(0, 191, 255))
+   *         .build();
+   * }
+ * + *

Elements appear in the resulting list in the same order they were added to the builder. + * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple lists in series. Each new list contains all the elements of the ones created before + * it. + * + * @since 2.0 + */ + public static final class Builder extends ImmutableCollection.Builder { + Object[] contents; + private int size; + private boolean forceCopy; + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableList#builder}. + */ + public Builder() { + this(DEFAULT_INITIAL_CAPACITY); + } + + Builder(int capacity) { + this.contents = new Object[capacity]; + this.size = 0; + } + + private void getReadyToExpandTo(int minCapacity) { + if (contents.length < minCapacity) { + this.contents = Arrays.copyOf(contents, expandedCapacity(contents.length, minCapacity)); + forceCopy = false; + } else if (forceCopy) { + contents = Arrays.copyOf(contents, contents.length); + forceCopy = false; + } + } + + /** + * Adds {@code element} to the {@code ImmutableList}. + * + * @param element the element to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + */ + @Override + public Builder add(E element) { + checkNotNull(element); + getReadyToExpandTo(size + 1); + contents[size++] = element; + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableList}. + * + * @param elements the {@code Iterable} to add to the {@code ImmutableList} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @SuppressWarnings("unchecked") + @Override + public Builder add(E... elements) { + checkElementsNotNull(elements); + add(elements, elements.length); + return this; + } + + private void add(Object[] elements, int n) { + getReadyToExpandTo(size + n); + System.arraycopy(elements, 0, contents, size, n); + size += n; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableList}. + * + * @param elements the {@code Iterable} to add to the {@code ImmutableList} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @Override + public Builder addAll(Iterable elements) { + checkNotNull(elements); + if (elements instanceof Collection) { + Collection collection = (Collection) elements; + getReadyToExpandTo(size + collection.size()); + if (collection instanceof ImmutableCollection) { + ImmutableCollection immutableCollection = (ImmutableCollection) collection; + size = immutableCollection.copyIntoArray(contents, size); + return this; + } + } + super.addAll(elements); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableList}. + * + * @param elements the {@code Iterable} to add to the {@code ImmutableList} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @Override + public Builder addAll(Iterator elements) { + super.addAll(elements); + return this; + } + + Builder combine(Builder builder) { + checkNotNull(builder); + add(builder.contents, builder.size); + return this; + } + + /** + * Returns a newly-created {@code ImmutableList} based on the contents of the {@code Builder}. + */ + @Override + public ImmutableList build() { + forceCopy = true; + return asImmutableList(contents, size); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableListMultimap.java new file mode 100644 index 0000000000000..d5e7bcd756293 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableListMultimap.java @@ -0,0 +1,497 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +import java.io.IOException; +import java.io.InvalidObjectException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Collection; +import java.util.Comparator; +import java.util.Map.Entry; +import java.util.function.Function; +import java.util.stream.Collector; +import java.util.stream.Collectors; +import java.util.stream.Stream; + + +/** + * A {@link ListMultimap} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Jared Levy + * @since 2.0 + */ +public class ImmutableListMultimap extends ImmutableMultimap + implements ListMultimap { + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableListMultimap} + * whose keys and values are the result of applying the provided mapping functions to the input + * elements. + * + *

For streams with {@linkplain java.util.stream#Ordering defined encounter order}, that order + * is preserved, but entries are grouped by key. + * + *

Example: + * + *

{@code
+   * static final Multimap FIRST_LETTER_MULTIMAP =
+   *     Stream.of("banana", "apple", "carrot", "asparagus", "cherry")
+   *         .collect(toImmutableListMultimap(str -> str.charAt(0), str -> str.substring(1)));
+   *
+   * // is equivalent to
+   *
+   * static final Multimap FIRST_LETTER_MULTIMAP =
+   *     new ImmutableListMultimap.Builder()
+   *         .put('b', "anana")
+   *         .putAll('a', "pple", "sparagus")
+   *         .putAll('c', "arrot", "herry")
+   *         .build();
+   * }
+ * + * @since 21.0 + */ + + public static Collector> toImmutableListMultimap( + Function keyFunction, + Function valueFunction) { + checkNotNull(keyFunction, "keyFunction"); + checkNotNull(valueFunction, "valueFunction"); + return Collector.of( + ImmutableListMultimap::builder, + (builder, t) -> builder.put(keyFunction.apply(t), valueFunction.apply(t)), + Builder::combine, + Builder::build); + } + + /** + * Returns a {@code Collector} accumulating entries into an {@code ImmutableListMultimap}. Each + * input element is mapped to a key and a stream of values, each of which are put into the + * resulting {@code Multimap}, in the encounter order of the stream and the encounter order of the + * streams of values. + * + *

Example: + * + *

{@code
+   * static final ImmutableListMultimap FIRST_LETTER_MULTIMAP =
+   *     Stream.of("banana", "apple", "carrot", "asparagus", "cherry")
+   *         .collect(
+   *             flatteningToImmutableListMultimap(
+   *                  str -> str.charAt(0),
+   *                  str -> str.substring(1).chars().mapToObj(c -> (char) c));
+   *
+   * // is equivalent to
+   *
+   * static final ImmutableListMultimap FIRST_LETTER_MULTIMAP =
+   *     ImmutableListMultimap.builder()
+   *         .putAll('b', Arrays.asList('a', 'n', 'a', 'n', 'a'))
+   *         .putAll('a', Arrays.asList('p', 'p', 'l', 'e'))
+   *         .putAll('c', Arrays.asList('a', 'r', 'r', 'o', 't'))
+   *         .putAll('a', Arrays.asList('s', 'p', 'a', 'r', 'a', 'g', 'u', 's'))
+   *         .putAll('c', Arrays.asList('h', 'e', 'r', 'r', 'y'))
+   *         .build();
+   * }
+   * }
+ * + * @since 21.0 + */ + + public static + Collector> flatteningToImmutableListMultimap( + Function keyFunction, + Function> valuesFunction) { + checkNotNull(keyFunction); + checkNotNull(valuesFunction); + return Collectors.collectingAndThen( + Multimaps.flatteningToMultimap( + input -> checkNotNull(keyFunction.apply(input)), + input -> valuesFunction.apply(input).peek(Preconditions::checkNotNull), + MultimapBuilder.linkedHashKeys().arrayListValues()::build), + ImmutableListMultimap::copyOf); + } + + /** Returns the empty multimap. */ + // Casting is safe because the multimap will never hold any elements. + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableListMultimap of() { + return (ImmutableListMultimap) EmptyImmutableListMultimap.INSTANCE; + } + + /** Returns an immutable multimap containing a single entry. */ + public static ImmutableListMultimap of(K k1, V v1) { + Builder builder = ImmutableListMultimap.builder(); + builder.put(k1, v1); + return builder.build(); + } + + /** Returns an immutable multimap containing the given entries, in order. */ + public static ImmutableListMultimap of(K k1, V v1, K k2, V v2) { + Builder builder = ImmutableListMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + return builder.build(); + } + + /** Returns an immutable multimap containing the given entries, in order. */ + public static ImmutableListMultimap of(K k1, V v1, K k2, V v2, K k3, V v3) { + Builder builder = ImmutableListMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + builder.put(k3, v3); + return builder.build(); + } + + /** Returns an immutable multimap containing the given entries, in order. */ + public static ImmutableListMultimap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + Builder builder = ImmutableListMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + builder.put(k3, v3); + builder.put(k4, v4); + return builder.build(); + } + + /** Returns an immutable multimap containing the given entries, in order. */ + public static ImmutableListMultimap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + Builder builder = ImmutableListMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + builder.put(k3, v3); + builder.put(k4, v4); + builder.put(k5, v5); + return builder.build(); + } + + // looking for of() with > 5 entries? Use the builder instead. + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder<>(); + } + + /** + * A builder for creating immutable {@code ListMultimap} instances, especially {@code public + * static final} multimaps ("constant multimaps"). Example: + * + *
{@code
+   * static final Multimap STRING_TO_INTEGER_MULTIMAP =
+   *     new ImmutableListMultimap.Builder()
+   *         .put("one", 1)
+   *         .putAll("several", 1, 2, 3)
+   *         .putAll("many", 1, 2, 3, 4, 5)
+   *         .build();
+   * }
+ * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple multimaps in series. Each multimap contains the key-value mappings in the previously + * created multimaps. + * + * @since 2.0 + */ + public static final class Builder extends ImmutableMultimap.Builder { + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableListMultimap#builder}. + */ + public Builder() {} + + @Override + public Builder put(K key, V value) { + super.put(key, value); + return this; + } + + /** + * {@inheritDoc} + * + * @since 11.0 + */ + @Override + public Builder put(Entry entry) { + super.put(entry); + return this; + } + + /** + * {@inheritDoc} + * + * @since 19.0 + */ + + @Override + public Builder putAll(Iterable> entries) { + super.putAll(entries); + return this; + } + + @Override + public Builder putAll(K key, Iterable values) { + super.putAll(key, values); + return this; + } + + @SafeVarargs + @SuppressWarnings("varargs") + @Override + public final Builder putAll(K key, V... values) { + super.putAll(key, values); + return this; + } + + @Override + public Builder putAll(Multimap multimap) { + super.putAll(multimap); + return this; + } + + @Override + Builder combine(ImmutableMultimap.Builder other) { + super.combine(other); + return this; + } + + /** + * {@inheritDoc} + * + * @since 8.0 + */ + @Override + public Builder orderKeysBy(Comparator keyComparator) { + super.orderKeysBy(keyComparator); + return this; + } + + /** + * {@inheritDoc} + * + * @since 8.0 + */ + @Override + public Builder orderValuesBy(Comparator valueComparator) { + super.orderValuesBy(valueComparator); + return this; + } + + /** Returns a newly-created immutable list multimap. */ + @Override + public ImmutableListMultimap build() { + return (ImmutableListMultimap) super.build(); + } + } + + /** + * Returns an immutable multimap containing the same mappings as {@code multimap}. The generated + * multimap's key and value orderings correspond to the iteration ordering of the {@code + * multimap.asMap()} view. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if any key or value in {@code multimap} is null + */ + public static ImmutableListMultimap copyOf( + Multimap multimap) { + if (multimap.isEmpty()) { + return of(); + } + + // TODO(lowasser): copy ImmutableSetMultimap by using asList() on the sets + if (multimap instanceof ImmutableListMultimap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe since multimap is not writable + ImmutableListMultimap kvMultimap = (ImmutableListMultimap) multimap; + if (!kvMultimap.isPartialView()) { + return kvMultimap; + } + } + + return fromMapEntries(multimap.asMap().entrySet(), null); + } + + /** + * Returns an immutable multimap containing the specified entries. The returned multimap iterates + * over keys in the order they were first encountered in the input, and the values for each key + * are iterated in the order they were encountered. + * + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + public static ImmutableListMultimap copyOf( + Iterable> entries) { + return new Builder().putAll(entries).build(); + } + + /** Creates an ImmutableListMultimap from an asMap.entrySet. */ + static ImmutableListMultimap fromMapEntries( + Collection>> mapEntries, + Comparator valueComparator) { + if (mapEntries.isEmpty()) { + return of(); + } + ImmutableMap.Builder> builder = + new ImmutableMap.Builder<>(mapEntries.size()); + int size = 0; + + for (Entry> entry : mapEntries) { + K key = entry.getKey(); + Collection values = entry.getValue(); + ImmutableList list = + (valueComparator == null) + ? ImmutableList.copyOf(values) + : ImmutableList.sortedCopyOf(valueComparator, values); + if (!list.isEmpty()) { + builder.put(key, list); + size += list.size(); + } + } + + return new ImmutableListMultimap<>(builder.build(), size); + } + + ImmutableListMultimap(ImmutableMap> map, int size) { + super(map, size); + } + + // views + + /** + * Returns an immutable list of the values for the given key. If no mappings in the multimap have + * the provided key, an empty immutable list is returned. The values are in the same order as the + * parameters used to build this multimap. + */ + @Override + public ImmutableList get(K key) { + // This cast is safe as its type is known in constructor. + ImmutableList list = (ImmutableList) map.get(key); + return (list == null) ? ImmutableList.of() : list; + } + + private transient ImmutableListMultimap inverse; + + /** + * {@inheritDoc} + * + *

Because an inverse of a list multimap can contain multiple pairs with the same key and + * value, this method returns an {@code ImmutableListMultimap} rather than the {@code + * ImmutableMultimap} specified in the {@code ImmutableMultimap} class. + * + * @since 11.0 + */ + @Override + public ImmutableListMultimap inverse() { + ImmutableListMultimap result = inverse; + return (result == null) ? (inverse = invert()) : result; + } + + private ImmutableListMultimap invert() { + Builder builder = builder(); + for (Entry entry : entries()) { + builder.put(entry.getValue(), entry.getKey()); + } + ImmutableListMultimap invertedMultimap = builder.build(); + invertedMultimap.inverse = this; + return invertedMultimap; + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public ImmutableList removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public ImmutableList replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + /** + * @serialData number of distinct keys, and then for each distinct key: the key, the number of + * values for that key, and the key's values + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + Serialization.writeMultimap(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + int keyCount = stream.readInt(); + if (keyCount < 0) { + throw new InvalidObjectException("Invalid key count " + keyCount); + } + ImmutableMap.Builder> builder = ImmutableMap.builder(); + int tmpSize = 0; + + for (int i = 0; i < keyCount; i++) { + Object key = stream.readObject(); + int valueCount = stream.readInt(); + if (valueCount <= 0) { + throw new InvalidObjectException("Invalid value count " + valueCount); + } + + ImmutableList.Builder valuesBuilder = ImmutableList.builder(); + for (int j = 0; j < valueCount; j++) { + valuesBuilder.add(stream.readObject()); + } + builder.put(key, valuesBuilder.build()); + tmpSize += valueCount; + } + + ImmutableMap> tmpMap; + try { + tmpMap = builder.build(); + } catch (IllegalArgumentException e) { + throw (InvalidObjectException) new InvalidObjectException(e.getMessage()).initCause(e); + } + + FieldSettersHolder.MAP_FIELD_SETTER.set(this, tmpMap); + FieldSettersHolder.SIZE_FIELD_SETTER.set(this, tmpSize); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMap.java new file mode 100644 index 0000000000000..04aa061f0bd02 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMap.java @@ -0,0 +1,920 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + + +import java.io.Serializable; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.SortedMap; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.BiFunction; +import java.util.function.BinaryOperator; +import java.util.function.Function; +import java.util.stream.Collector; +import java.util.stream.Collectors; + + + +/** + * A {@link Map} whose contents will never change, with many other important properties detailed at + * {@link ImmutableCollection}. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + * @since 2.0 + */ +@SuppressWarnings("serial") // we're overriding default serialization +public abstract class ImmutableMap implements Map, Serializable { + + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableMap} whose keys + * and values are the result of applying the provided mapping functions to the input elements. + * Entries appear in the result {@code ImmutableMap} in encounter order. + * + *

If the mapped keys contain duplicates (according to {@link Object#equals(Object)}, an {@code + * IllegalArgumentException} is thrown when the collection operation is performed. (This differs + * from the {@code Collector} returned by {@link Collectors#toMap(Function, Function)}, which + * throws an {@code IllegalStateException}.) + * + * @since 21.0 + */ + + public static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction) { + return CollectCollectors.toImmutableMap(keyFunction, valueFunction); + } + + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableMap} whose keys + * and values are the result of applying the provided mapping functions to the input elements. + * + *

If the mapped keys contain duplicates (according to {@link Object#equals(Object)}), the + * values are merged using the specified merging function. Entries will appear in the encounter + * order of the first occurrence of the key. + * + * @since 21.0 + */ + + public static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction, + BinaryOperator mergeFunction) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + checkNotNull(mergeFunction); + return Collectors.collectingAndThen( + Collectors.toMap(keyFunction, valueFunction, mergeFunction, LinkedHashMap::new), + ImmutableMap::copyOf); + } + + /** + * Returns the empty map. This map behaves and performs comparably to {@link + * Collections#emptyMap}, and is preferable mainly for consistency and maintainability of your + * code. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableMap of() { + return (ImmutableMap) RegularImmutableMap.EMPTY; + } + + /** + * Returns an immutable map containing a single entry. This map behaves and performs comparably to + * {@link Collections#singletonMap} but will not accept a null key or value. It is preferable + * mainly for consistency and maintainability of your code. + */ + public static ImmutableMap of(K k1, V v1) { + return ImmutableBiMap.of(k1, v1); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys are provided + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableMap of(K k1, V v1, K k2, V v2) { + return RegularImmutableMap.fromEntries(entryOf(k1, v1), entryOf(k2, v2)); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys are provided + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableMap of(K k1, V v1, K k2, V v2, K k3, V v3) { + return RegularImmutableMap.fromEntries(entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3)); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys are provided + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableMap of(K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + return RegularImmutableMap.fromEntries( + entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3), entryOf(k4, v4)); + } + + /** + * Returns an immutable map containing the given entries, in order. + * + * @throws IllegalArgumentException if duplicate keys are provided + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableMap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + return RegularImmutableMap.fromEntries( + entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3), entryOf(k4, v4), entryOf(k5, v5)); + } + + // looking for of() with > 5 entries? Use the builder instead. + + /** + * Verifies that {@code key} and {@code value} are non-null, and returns a new immutable entry + * with those values. + * + *

A call to {@link Entry#setValue} on the returned entry will always throw {@link + * UnsupportedOperationException}. + */ + static Entry entryOf(K key, V value) { + checkEntryNotNull(key, value); + return new AbstractMap.SimpleImmutableEntry<>(key, value); + } + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder<>(); + } + + /** + * Returns a new builder, expecting the specified number of entries to be added. + * + *

If {@code expectedSize} is exactly the number of entries added to the builder before {@link + * Builder#build} is called, the builder is likely to perform better than an unsized {@link + * #builder()} would have. + * + *

It is not specified if any performance benefits apply if {@code expectedSize} is close to, + * but not exactly, the number of entries added to the builder. + * + * @since 23.1 + */ + + public static Builder builderWithExpectedSize(int expectedSize) { + checkNonnegative(expectedSize, "expectedSize"); + return new Builder<>(expectedSize); + } + + static void checkNoConflict( + boolean safe, String conflictDescription, Entry entry1, Entry entry2) { + if (!safe) { + throw conflictException(conflictDescription, entry1, entry2); + } + } + + static IllegalArgumentException conflictException( + String conflictDescription, Object entry1, Object entry2) { + return new IllegalArgumentException( + "Multiple entries with same " + conflictDescription + ": " + entry1 + " and " + entry2); + } + + /** + * A builder for creating immutable map instances, especially {@code public static final} maps + * ("constant maps"). Example: + * + *

{@code
+   * static final ImmutableMap WORD_TO_INT =
+   *     new ImmutableMap.Builder()
+   *         .put("one", 1)
+   *         .put("two", 2)
+   *         .put("three", 3)
+   *         .build();
+   * }
+ * + *

For small immutable maps, the {@code ImmutableMap.of()} methods are even more + * convenient. + * + *

By default, a {@code Builder} will generate maps that iterate over entries in the order they + * were inserted into the builder, equivalently to {@code LinkedHashMap}. For example, in the + * above example, {@code WORD_TO_INT.entrySet()} is guaranteed to iterate over the entries in the + * order {@code "one"=1, "two"=2, "three"=3}, and {@code keySet()} and {@code values()} respect + * the same order. If you want a different order, consider using {@link ImmutableSortedMap} to + * sort by keys, or call {@link #orderEntriesByValue(Comparator)}, which changes this builder to + * sort entries by value. + * + *

Builder instances can be reused - it is safe to call {@link #build} multiple times to build + * multiple maps in series. Each map is a superset of the maps created before it. + * + * @since 2.0 + */ + public static class Builder { + Comparator valueComparator; + Entry[] entries; + int size; + boolean entriesUsed; + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableMap#builder}. + */ + public Builder() { + this(ImmutableCollection.Builder.DEFAULT_INITIAL_CAPACITY); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + Builder(int initialCapacity) { + this.entries = new Entry[initialCapacity]; + this.size = 0; + this.entriesUsed = false; + } + + private void ensureCapacity(int minCapacity) { + if (minCapacity > entries.length) { + entries = + Arrays.copyOf( + entries, ImmutableCollection.Builder.expandedCapacity(entries.length, minCapacity)); + entriesUsed = false; + } + } + + /** + * Associates {@code key} with {@code value} in the built map. Duplicate keys are not allowed, + * and will cause {@link #build} to fail. + */ + public Builder put(K key, V value) { + ensureCapacity(size + 1); + Entry entry = entryOf(key, value); + // don't inline this: we want to fail atomically if key or value is null + entries[size++] = entry; + return this; + } + + /** + * Adds the given {@code entry} to the map, making it immutable if necessary. Duplicate keys are + * not allowed, and will cause {@link #build} to fail. + * + * @since 11.0 + */ + public Builder put(Entry entry) { + return put(entry.getKey(), entry.getValue()); + } + + /** + * Associates all of the given map's keys and values in the built map. Duplicate keys are not + * allowed, and will cause {@link #build} to fail. + * + * @throws NullPointerException if any key or value in {@code map} is null + */ + public Builder putAll(Map map) { + return putAll(map.entrySet()); + } + + /** + * Adds all of the given entries to the built map. Duplicate keys are not allowed, and will + * cause {@link #build} to fail. + * + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + public Builder putAll(Iterable> entries) { + if (entries instanceof Collection) { + ensureCapacity(size + ((Collection) entries).size()); + } + for (Entry entry : entries) { + put(entry); + } + return this; + } + + /** + * Configures this {@code Builder} to order entries by value according to the specified + * comparator. + * + *

The sort order is stable, that is, if two entries have values that compare as equivalent, + * the entry that was inserted first will be first in the built map's iteration order. + * + * @throws IllegalStateException if this method was already called + * @since 19.0 + */ + + public Builder orderEntriesByValue(Comparator valueComparator) { + checkState(this.valueComparator == null, "valueComparator was already set"); + this.valueComparator = checkNotNull(valueComparator, "valueComparator"); + return this; + } + + Builder combine(Builder other) { + checkNotNull(other); + ensureCapacity(this.size + other.size); + System.arraycopy(other.entries, 0, this.entries, this.size, other.size); + this.size += other.size; + return this; + } + + /* + * TODO(kevinb): Should build() and the ImmutableBiMap & ImmutableSortedMap + * versions throw an IllegalStateException instead? + */ + + /** + * Returns a newly-created immutable map. The iteration order of the returned map is the order + * in which entries were inserted into the builder, unless {@link #orderEntriesByValue} was + * called, in which case entries are sorted by value. + * + * @throws IllegalArgumentException if duplicate keys were added + */ + public ImmutableMap build() { + /* + * If entries is full, or if hash flooding is detected, then this implementation may end up + * using the entries array directly and writing over the entry objects with non-terminal + * entries, but this is safe; if this Builder is used further, it will grow the entries array + * (so it can't affect the original array), and future build() calls will always copy any + * entry objects that cannot be safely reused. + */ + if (valueComparator != null) { + if (entriesUsed) { + entries = Arrays.copyOf(entries, size); + } + Arrays.sort( + entries, 0, size, Ordering.from(valueComparator).onResultOf(Maps.valueFunction())); + } + switch (size) { + case 0: + return of(); + case 1: + return of(entries[0].getKey(), entries[0].getValue()); + default: + entriesUsed = true; + return RegularImmutableMap.fromEntryArray(size, entries); + } + } + + // only for testing JDK backed implementation + ImmutableMap buildJdkBacked() { + checkState( + valueComparator == null, "buildJdkBacked is only for testing; can't use valueComparator"); + switch (size) { + case 0: + return of(); + case 1: + return of(entries[0].getKey(), entries[0].getValue()); + default: + entriesUsed = true; + return JdkBackedImmutableMap.create(size, entries); + } + } + } + + /** + * Returns an immutable map containing the same entries as {@code map}. The returned map iterates + * over entries in the same order as the {@code entrySet} of the original map. If {@code map} + * somehow contains entries with duplicate keys (for example, if it is a {@code SortedMap} whose + * comparator is not consistent with equals), the results of this method are undefined. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if any key or value in {@code map} is null + */ + public static ImmutableMap copyOf(Map map) { + if ((map instanceof ImmutableMap) && !(map instanceof SortedMap)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe since map is not writable + ImmutableMap kvMap = (ImmutableMap) map; + if (!kvMap.isPartialView()) { + return kvMap; + } + } else if (map instanceof EnumMap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe since map is not writable + ImmutableMap kvMap = (ImmutableMap) copyOfEnumMap((EnumMap) map); + return kvMap; + } + return copyOf(map.entrySet()); + } + + /** + * Returns an immutable map containing the specified entries. The returned map iterates over + * entries in the same order as the original iterable. + * + * @throws NullPointerException if any key, value, or entry is null + * @throws IllegalArgumentException if two entries have the same key + * @since 19.0 + */ + + public static ImmutableMap copyOf( + Iterable> entries) { + @SuppressWarnings({"unchecked", "rawtypes"}) // we'll only be using getKey and getValue, which are covariant + Entry[] entryArray = (Entry[]) Iterables.toArray(entries, EMPTY_ENTRY_ARRAY); + switch (entryArray.length) { + case 0: + return of(); + case 1: + Entry onlyEntry = entryArray[0]; + return of(onlyEntry.getKey(), onlyEntry.getValue()); + default: + /* + * The current implementation will end up using entryArray directly, though it will write + * over the (arbitrary, potentially mutable) Entry objects actually stored in entryArray. + */ + return RegularImmutableMap.fromEntries(entryArray); + } + } + + private static , V> ImmutableMap copyOfEnumMap( + EnumMap original) { + EnumMap copy = new EnumMap<>(original); + for (Entry entry : copy.entrySet()) { + checkEntryNotNull(entry.getKey(), entry.getValue()); + } + return ImmutableEnumMap.asImmutable(copy); + } + + static final Entry[] EMPTY_ENTRY_ARRAY = new Entry[0]; + + abstract static class IteratorBasedImmutableMap extends ImmutableMap { + abstract UnmodifiableIterator> entryIterator(); + + Spliterator> entrySpliterator() { + return Spliterators.spliterator( + entryIterator(), + size(), + Spliterator.DISTINCT | Spliterator.NONNULL | Spliterator.IMMUTABLE | Spliterator.ORDERED); + } + + @Override + ImmutableSet createKeySet() { + return new ImmutableMapKeySet<>(this); + } + + @Override + ImmutableSet> createEntrySet() { + class EntrySetImpl extends ImmutableMapEntrySet { + @Override + ImmutableMap map() { + return IteratorBasedImmutableMap.this; + } + + @Override + public UnmodifiableIterator> iterator() { + return entryIterator(); + } + } + return new EntrySetImpl(); + } + + @Override + ImmutableCollection createValues() { + return new ImmutableMapValues<>(this); + } + } + + ImmutableMap() {} + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V put(K k, V v) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V putIfAbsent(K key, V value) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean replace(K key, V oldValue, V newValue) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V replace(K key, V value) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V computeIfAbsent(K key, Function mappingFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V computeIfPresent( + K key, BiFunction remappingFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V compute(K key, BiFunction remappingFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V merge( + K key, V value, BiFunction remappingFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void putAll(Map map) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void replaceAll(BiFunction function) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final V remove(Object o) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean remove(Object key, Object value) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void clear() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isEmpty() { + return size() == 0; + } + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + public boolean containsValue(Object value) { + return values().contains(value); + } + + // Overriding to mark it Nullable + @Override + public abstract V get(Object key); + + /** + * @since 21.0 (but only since 23.5 in the Android flavor). + * Note, however, that Java 8 users can call this method with any version and flavor of Guava. + */ + @Override + public final V getOrDefault(Object key, V defaultValue) { + V result = get(key); + return (result != null) ? result : defaultValue; + } + + private transient ImmutableSet> entrySet; + + /** + * Returns an immutable set of the mappings in this map. The iteration order is specified by the + * method used to create this map. Typically, this is insertion order. + */ + @Override + public ImmutableSet> entrySet() { + ImmutableSet> result = entrySet; + return (result == null) ? entrySet = createEntrySet() : result; + } + + abstract ImmutableSet> createEntrySet(); + + private transient ImmutableSet keySet; + + /** + * Returns an immutable set of the keys in this map, in the same order that they appear in {@link + * #entrySet}. + */ + @Override + public ImmutableSet keySet() { + ImmutableSet result = keySet; + return (result == null) ? keySet = createKeySet() : result; + } + + /* + * This could have a good default implementation of return new ImmutableKeySet(this), + * but ProGuard can't figure out how to eliminate that default when RegularImmutableMap + * overrides it. + */ + abstract ImmutableSet createKeySet(); + + UnmodifiableIterator keyIterator() { + final UnmodifiableIterator> entryIterator = entrySet().iterator(); + return new UnmodifiableIterator() { + @Override + public boolean hasNext() { + return entryIterator.hasNext(); + } + + @Override + public K next() { + return entryIterator.next().getKey(); + } + }; + } + + Spliterator keySpliterator() { + return CollectSpliterators.map(entrySet().spliterator(), Entry::getKey); + } + + private transient ImmutableCollection values; + + /** + * Returns an immutable collection of the values in this map, in the same order that they appear + * in {@link #entrySet}. + */ + @Override + public ImmutableCollection values() { + ImmutableCollection result = values; + return (result == null) ? values = createValues() : result; + } + + /* + * This could have a good default implementation of {@code return new + * ImmutableMapValues(this)}, but ProGuard can't figure out how to eliminate that default + * when RegularImmutableMap overrides it. + */ + abstract ImmutableCollection createValues(); + + // cached so that this.multimapView().inverse() only computes inverse once + private transient ImmutableSetMultimap multimapView; + + /** + * Returns a multimap view of the map. + * + * @since 14.0 + */ + public ImmutableSetMultimap asMultimap() { + if (isEmpty()) { + return ImmutableSetMultimap.of(); + } + ImmutableSetMultimap result = multimapView; + return (result == null) + ? (multimapView = + new ImmutableSetMultimap<>(new MapViewOfValuesAsSingletonSets(), size(), null)) + : result; + } + + private final class MapViewOfValuesAsSingletonSets + extends IteratorBasedImmutableMap> { + + @Override + public int size() { + return ImmutableMap.this.size(); + } + + @Override + ImmutableSet createKeySet() { + return ImmutableMap.this.keySet(); + } + + @Override + public boolean containsKey(Object key) { + return ImmutableMap.this.containsKey(key); + } + + @Override + public ImmutableSet get(Object key) { + V outerValue = ImmutableMap.this.get(key); + return (outerValue == null) ? null : ImmutableSet.of(outerValue); + } + + @Override + boolean isPartialView() { + return ImmutableMap.this.isPartialView(); + } + + @Override + public int hashCode() { + // ImmutableSet.of(value).hashCode() == value.hashCode(), so the hashes are the same + return ImmutableMap.this.hashCode(); + } + + @Override + boolean isHashCodeFast() { + return ImmutableMap.this.isHashCodeFast(); + } + + @Override + UnmodifiableIterator>> entryIterator() { + final Iterator> backingIterator = ImmutableMap.this.entrySet().iterator(); + return new UnmodifiableIterator>>() { + @Override + public boolean hasNext() { + return backingIterator.hasNext(); + } + + @Override + public Entry> next() { + final Entry backingEntry = backingIterator.next(); + return new AbstractMapEntry>() { + @Override + public K getKey() { + return backingEntry.getKey(); + } + + @Override + public ImmutableSet getValue() { + return ImmutableSet.of(backingEntry.getValue()); + } + }; + } + }; + } + } + + @Override + public boolean equals(Object object) { + return Maps.equalsImpl(this, object); + } + + abstract boolean isPartialView(); + + @Override + public int hashCode() { + return Sets.hashCodeImpl(entrySet()); + } + + boolean isHashCodeFast() { + return false; + } + + @Override + public String toString() { + return Maps.toStringImpl(this); + } + + /** + * Serialized type for all ImmutableMap instances. It captures the logical contents and they are + * reconstructed using public factory methods. This ensures that the implementation types remain + * as implementation details. + */ + static class SerializedForm implements Serializable { + private final Object[] keys; + private final Object[] values; + + SerializedForm(ImmutableMap map) { + keys = new Object[map.size()]; + values = new Object[map.size()]; + int i = 0; + for (Entry entry : map.entrySet()) { + keys[i] = entry.getKey(); + values[i] = entry.getValue(); + i++; + } + } + + Object readResolve() { + Builder builder = new Builder<>(keys.length); + return createMap(builder); + } + + Object createMap(Builder builder) { + for (int i = 0; i < keys.length; i++) { + builder.put(keys[i], values[i]); + } + return builder.build(); + } + + private static final long serialVersionUID = 0; + } + + Object writeReplace() { + return new SerializedForm(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntry.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntry.java new file mode 100644 index 0000000000000..3ba28da4c6290 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntry.java @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2013 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; + + + +/** + * Implementation of {@code Entry} for {@link ImmutableMap} that adds extra methods to traverse hash + * buckets for the key and the value. This allows reuse in {@link RegularImmutableMap} and {@link + * RegularImmutableBiMap}, which don't have to recopy the entries created by their {@code Builder} + * implementations. + * + *

This base implementation has no key or value pointers, so instances of ImmutableMapEntry (but + * not its subclasses) can be reused when copied from one ImmutableMap to another. + * + * @author Louis Wasserman + */ +class ImmutableMapEntry extends ImmutableEntry { + /** + * Creates an {@code ImmutableMapEntry} array to hold parameterized entries. The result must never + * be upcast back to ImmutableMapEntry[] (or Object[], etc.), or allowed to escape the class. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // Safe as long as the javadocs are followed + static ImmutableMapEntry[] createEntryArray(int size) { + return new ImmutableMapEntry[size]; + } + + ImmutableMapEntry(K key, V value) { + super(key, value); + checkEntryNotNull(key, value); + } + + ImmutableMapEntry(ImmutableMapEntry contents) { + super(contents.getKey(), contents.getValue()); + // null check would be redundant + } + + + ImmutableMapEntry getNextInKeyBucket() { + return null; + } + + + ImmutableMapEntry getNextInValueBucket() { + return null; + } + + /** + * Returns true if this entry has no bucket links and can safely be reused as a terminal entry in + * a bucket in another map. + */ + boolean isReusable() { + return true; + } + + static class NonTerminalImmutableMapEntry extends ImmutableMapEntry { + private final transient ImmutableMapEntry nextInKeyBucket; + + NonTerminalImmutableMapEntry(K key, V value, ImmutableMapEntry nextInKeyBucket) { + super(key, value); + this.nextInKeyBucket = nextInKeyBucket; + } + + @Override + + final ImmutableMapEntry getNextInKeyBucket() { + return nextInKeyBucket; + } + + @Override + final boolean isReusable() { + return false; + } + } + + static final class NonTerminalImmutableBiMapEntry + extends NonTerminalImmutableMapEntry { + private final transient ImmutableMapEntry nextInValueBucket; + + NonTerminalImmutableBiMapEntry( + K key, + V value, + ImmutableMapEntry nextInKeyBucket, + ImmutableMapEntry nextInValueBucket) { + super(key, value, nextInKeyBucket); + this.nextInValueBucket = nextInValueBucket; + } + + @Override + + ImmutableMapEntry getNextInValueBucket() { + return nextInValueBucket; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntrySet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntrySet.java new file mode 100644 index 0000000000000..3c30b16ee6b8d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapEntrySet.java @@ -0,0 +1,132 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.Map.Entry; +import java.util.Spliterator; +import java.util.function.Consumer; + + +/** + * {@code entrySet()} implementation for {@link ImmutableMap}. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + */ +abstract class ImmutableMapEntrySet extends ImmutableSet> { + static final class RegularEntrySet extends ImmutableMapEntrySet { + private final transient ImmutableMap map; + private final transient ImmutableList> entries; + + RegularEntrySet(ImmutableMap map, Entry[] entries) { + this(map, ImmutableList.>asImmutableList(entries)); + } + + RegularEntrySet(ImmutableMap map, ImmutableList> entries) { + this.map = map; + this.entries = entries; + } + + @Override + ImmutableMap map() { + return map; + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + return entries.copyIntoArray(dst, offset); + } + + @Override + public UnmodifiableIterator> iterator() { + return entries.iterator(); + } + + @Override + public Spliterator> spliterator() { + return entries.spliterator(); + } + + @Override + public void forEach(Consumer> action) { + entries.forEach(action); + } + + @Override + ImmutableList> createAsList() { + return new RegularImmutableAsList<>(this, entries); + } + } + + ImmutableMapEntrySet() {} + + abstract ImmutableMap map(); + + @Override + public int size() { + return map().size(); + } + + @Override + public boolean contains(Object object) { + if (object instanceof Entry) { + Entry entry = (Entry) object; + V value = map().get(entry.getKey()); + return value != null && value.equals(entry.getValue()); + } + return false; + } + + @Override + boolean isPartialView() { + return map().isPartialView(); + } + + @Override + boolean isHashCodeFast() { + return map().isHashCodeFast(); + } + + @Override + public int hashCode() { + return map().hashCode(); + } + + @Override + Object writeReplace() { + return new EntrySetSerializedForm<>(map()); + } + + private static class EntrySetSerializedForm implements Serializable { + final ImmutableMap map; + + EntrySetSerializedForm(ImmutableMap map) { + this.map = map; + } + + Object readResolve() { + return map.entrySet(); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapKeySet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapKeySet.java new file mode 100644 index 0000000000000..db6e2d144cdb5 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapKeySet.java @@ -0,0 +1,97 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Spliterator; +import java.util.function.Consumer; + + +/** + * {@code keySet()} implementation for {@link ImmutableMap}. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + */ +final class ImmutableMapKeySet extends IndexedImmutableSet { + private final ImmutableMap map; + + ImmutableMapKeySet(ImmutableMap map) { + this.map = map; + } + + @Override + public int size() { + return map.size(); + } + + @Override + public UnmodifiableIterator iterator() { + return map.keyIterator(); + } + + @Override + public Spliterator spliterator() { + return map.keySpliterator(); + } + + @Override + public boolean contains(Object object) { + return map.containsKey(object); + } + + @Override + K get(int index) { + return map.entrySet().asList().get(index).getKey(); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + map.forEach((k, v) -> action.accept(k)); + } + + @Override + boolean isPartialView() { + return true; + } + + @Override + Object writeReplace() { + return new KeySetSerializedForm(map); + } + + private static class KeySetSerializedForm implements Serializable { + final ImmutableMap map; + + KeySetSerializedForm(ImmutableMap map) { + this.map = map; + } + + Object readResolve() { + return map.keySet(); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapValues.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapValues.java new file mode 100644 index 0000000000000..b4812397e960c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMapValues.java @@ -0,0 +1,121 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Map.Entry; +import java.util.Spliterator; +import java.util.function.Consumer; + + +/** + * {@code values()} implementation for {@link ImmutableMap}. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + */ +final class ImmutableMapValues extends ImmutableCollection { + private final ImmutableMap map; + + ImmutableMapValues(ImmutableMap map) { + this.map = map; + } + + @Override + public int size() { + return map.size(); + } + + @Override + public UnmodifiableIterator iterator() { + return new UnmodifiableIterator() { + final UnmodifiableIterator> entryItr = map.entrySet().iterator(); + + @Override + public boolean hasNext() { + return entryItr.hasNext(); + } + + @Override + public V next() { + return entryItr.next().getValue(); + } + }; + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.map(map.entrySet().spliterator(), Entry::getValue); + } + + @Override + public boolean contains(Object object) { + return object != null && Iterators.contains(iterator(), object); + } + + @Override + boolean isPartialView() { + return true; + } + + @Override + public ImmutableList asList() { + final ImmutableList> entryList = map.entrySet().asList(); + return new ImmutableAsList() { + @Override + public V get(int index) { + return entryList.get(index).getValue(); + } + + @Override + ImmutableCollection delegateCollection() { + return ImmutableMapValues.this; + } + }; + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + map.forEach((k, v) -> action.accept(v)); + } + + @Override + Object writeReplace() { + return new SerializedForm(map); + } + + private static class SerializedForm implements Serializable { + final ImmutableMap map; + + SerializedForm(ImmutableMap map) { + this.map = map; + } + + Object readResolve() { + return map.values(); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultimap.java new file mode 100644 index 0000000000000..f6f5e869804ec --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultimap.java @@ -0,0 +1,733 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; + + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.Spliterator; +import java.util.function.BiConsumer; + + + +/** + * A {@link Multimap} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

Warning: avoid direct usage of {@link ImmutableMultimap} as a type (as with + * {@link Multimap} itself). Prefer subtypes such as {@link ImmutableSetMultimap} or {@link + * ImmutableListMultimap}, which have well-defined {@link #equals} semantics, thus avoiding a common + * source of bugs and confusion. + * + *

Note: every {@link ImmutableMultimap} offers an {@link #inverse} view, so there is no + * need for a distinct {@code ImmutableBiMultimap} type. + * + *

+ * + *

Key-grouped iteration. All view collections follow the same iteration order. In all + * current implementations, the iteration order always keeps multiple entries with the same key + * together. Any creation method that would customarily respect insertion order (such as {@link + * #copyOf(Multimap)}) instead preserves key-grouped order by inserting entries for an existing key + * immediately after the last entry having that key. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Jared Levy + * @since 2.0 + */ +public abstract class ImmutableMultimap extends AbstractMultimap + implements Serializable { + + /** Returns an empty multimap. */ + public static ImmutableMultimap of() { + return ImmutableListMultimap.of(); + } + + /** Returns an immutable multimap containing a single entry. */ + public static ImmutableMultimap of(K k1, V v1) { + return ImmutableListMultimap.of(k1, v1); + } + + /** Returns an immutable multimap containing the given entries, in order. */ + public static ImmutableMultimap of(K k1, V v1, K k2, V v2) { + return ImmutableListMultimap.of(k1, v1, k2, v2); + } + + /** + * Returns an immutable multimap containing the given entries, in the "key-grouped" insertion + * order described in the class documentation. + */ + public static ImmutableMultimap of(K k1, V v1, K k2, V v2, K k3, V v3) { + return ImmutableListMultimap.of(k1, v1, k2, v2, k3, v3); + } + + /** + * Returns an immutable multimap containing the given entries, in the "key-grouped" insertion + * order described in the class documentation. + */ + public static ImmutableMultimap of(K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + return ImmutableListMultimap.of(k1, v1, k2, v2, k3, v3, k4, v4); + } + + /** + * Returns an immutable multimap containing the given entries, in the "key-grouped" insertion + * order described in the class documentation. + */ + public static ImmutableMultimap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + return ImmutableListMultimap.of(k1, v1, k2, v2, k3, v3, k4, v4, k5, v5); + } + + // looking for of() with > 5 entries? Use the builder instead. + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder<>(); + } + + /** + * A builder for creating immutable multimap instances, especially {@code public static final} + * multimaps ("constant multimaps"). Example: + * + *

{@code
+   * static final Multimap STRING_TO_INTEGER_MULTIMAP =
+   *     new ImmutableMultimap.Builder()
+   *         .put("one", 1)
+   *         .putAll("several", 1, 2, 3)
+   *         .putAll("many", 1, 2, 3, 4, 5)
+   *         .build();
+   * }
+ * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple multimaps in series. Each multimap contains the key-value mappings in the previously + * created multimaps. + * + * @since 2.0 + */ + public static class Builder { + Map> builderMap; + Comparator keyComparator; + Comparator valueComparator; + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableMultimap#builder}. + */ + public Builder() { + this.builderMap = Platform.preservesInsertionOrderOnPutsMap(); + } + + Collection newMutableValueCollection() { + return new ArrayList<>(); + } + + /** Adds a key-value mapping to the built multimap. */ + public Builder put(K key, V value) { + checkEntryNotNull(key, value); + Collection valueCollection = builderMap.get(key); + if (valueCollection == null) { + builderMap.put(key, valueCollection = newMutableValueCollection()); + } + valueCollection.add(value); + return this; + } + + /** + * Adds an entry to the built multimap. + * + * @since 11.0 + */ + public Builder put(Entry entry) { + return put(entry.getKey(), entry.getValue()); + } + + /** + * Adds entries to the built multimap. + * + * @since 19.0 + */ + + public Builder putAll(Iterable> entries) { + for (Entry entry : entries) { + put(entry); + } + return this; + } + + /** + * Stores a collection of values with the same key in the built multimap. + * + * @throws NullPointerException if {@code key}, {@code values}, or any element in {@code values} + * is null. The builder is left in an invalid state. + */ + public Builder putAll(K key, Iterable values) { + if (key == null) { + throw new NullPointerException("null key in entry: null=" + Iterables.toString(values)); + } + Collection valueCollection = builderMap.get(key); + if (valueCollection != null) { + for (V value : values) { + checkEntryNotNull(key, value); + valueCollection.add(value); + } + return this; + } + Iterator valuesItr = values.iterator(); + if (!valuesItr.hasNext()) { + return this; + } + valueCollection = newMutableValueCollection(); + while (valuesItr.hasNext()) { + V value = valuesItr.next(); + checkEntryNotNull(key, value); + valueCollection.add(value); + } + builderMap.put(key, valueCollection); + return this; + } + + /** + * Stores an array of values with the same key in the built multimap. + * + * @throws NullPointerException if the key or any value is null. The builder is left in an + * invalid state. + */ + @SuppressWarnings("unchecked") + public Builder putAll(K key, V... values) { + return putAll(key, Arrays.asList(values)); + } + + /** + * Stores another multimap's entries in the built multimap. The generated multimap's key and + * value orderings correspond to the iteration ordering of the {@code multimap.asMap()} view, + * with new keys and values following any existing keys and values. + * + * @throws NullPointerException if any key or value in {@code multimap} is null. The builder is + * left in an invalid state. + */ + public Builder putAll(Multimap multimap) { + for (Entry> entry : + multimap.asMap().entrySet()) { + putAll(entry.getKey(), entry.getValue()); + } + return this; + } + + /** + * Specifies the ordering of the generated multimap's keys. + * + * @since 8.0 + */ + public Builder orderKeysBy(Comparator keyComparator) { + this.keyComparator = checkNotNull(keyComparator); + return this; + } + + /** + * Specifies the ordering of the generated multimap's values for each key. + * + * @since 8.0 + */ + public Builder orderValuesBy(Comparator valueComparator) { + this.valueComparator = checkNotNull(valueComparator); + return this; + } + + Builder combine(Builder other) { + for (Entry> entry : other.builderMap.entrySet()) { + putAll(entry.getKey(), entry.getValue()); + } + return this; + } + + /** Returns a newly-created immutable multimap. */ + public ImmutableMultimap build() { + Collection>> mapEntries = builderMap.entrySet(); + if (keyComparator != null) { + mapEntries = Ordering.from(keyComparator).onKeys().immutableSortedCopy(mapEntries); + } + return ImmutableListMultimap.fromMapEntries(mapEntries, valueComparator); + } + } + + /** + * Returns an immutable multimap containing the same mappings as {@code multimap}, in the + * "key-grouped" iteration order described in the class documentation. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if any key or value in {@code multimap} is null + */ + public static ImmutableMultimap copyOf(Multimap multimap) { + if (multimap instanceof ImmutableMultimap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe since multimap is not writable + ImmutableMultimap kvMultimap = (ImmutableMultimap) multimap; + if (!kvMultimap.isPartialView()) { + return kvMultimap; + } + } + return ImmutableListMultimap.copyOf(multimap); + } + + /** + * Returns an immutable multimap containing the specified entries. The returned multimap iterates + * over keys in the order they were first encountered in the input, and the values for each key + * are iterated in the order they were encountered. + * + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + public static ImmutableMultimap copyOf( + Iterable> entries) { + return ImmutableListMultimap.copyOf(entries); + } + + final transient ImmutableMap> map; + final transient int size; + + // These constants allow the deserialization code to set final fields. This + // holder class makes sure they are not initialized unless an instance is + // deserialized. + static class FieldSettersHolder { + @SuppressWarnings("rawtypes") + static final Serialization.FieldSetter MAP_FIELD_SETTER = + Serialization.getFieldSetter(ImmutableMultimap.class, "map"); + @SuppressWarnings("rawtypes") + static final Serialization.FieldSetter SIZE_FIELD_SETTER = + Serialization.getFieldSetter(ImmutableMultimap.class, "size"); + } + + ImmutableMultimap(ImmutableMap> map, int size) { + this.map = map; + this.size = size; + } + + // mutators (not supported) + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public ImmutableCollection removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public ImmutableCollection replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + /** + * Returns an immutable collection of the values for the given key. If no mappings in the multimap + * have the provided key, an empty immutable collection is returned. The values are in the same + * order as the parameters used to build this multimap. + */ + @Override + public abstract ImmutableCollection get(K key); + + /** + * Returns an immutable multimap which is the inverse of this one. For every key-value mapping in + * the original, the result will have a mapping with key and value reversed. + * + * @since 11.0 + */ + public abstract ImmutableMultimap inverse(); + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public boolean put(K key, V value) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public boolean putAll(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public boolean putAll(Multimap multimap) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public boolean remove(Object key, Object value) { + throw new UnsupportedOperationException(); + } + + /** + * Returns {@code true} if this immutable multimap's implementation contains references to + * user-created objects that aren't accessible via this multimap's methods. This is generally used + * to determine whether {@code copyOf} implementations should make an explicit copy to avoid + * memory leaks. + */ + boolean isPartialView() { + return map.isPartialView(); + } + + // accessors + + @Override + public boolean containsKey(Object key) { + return map.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return value != null && super.containsValue(value); + } + + @Override + public int size() { + return size; + } + + // views + + /** + * Returns an immutable set of the distinct keys in this multimap, in the same order as they + * appear in this multimap. + */ + @Override + public ImmutableSet keySet() { + return map.keySet(); + } + + @Override + Set createKeySet() { + throw new AssertionError("unreachable"); + } + + /** + * Returns an immutable map that associates each key with its corresponding values in the + * multimap. Keys and values appear in the same order as in this multimap. + */ + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) // a widening cast + public ImmutableMap> asMap() { + return (ImmutableMap) map; + } + + @Override + Map> createAsMap() { + throw new AssertionError("should never be called"); + } + + /** Returns an immutable collection of all key-value pairs in the multimap. */ + @Override + public ImmutableCollection> entries() { + return (ImmutableCollection>) super.entries(); + } + + @Override + ImmutableCollection> createEntries() { + return new EntryCollection<>(this); + } + + private static class EntryCollection extends ImmutableCollection> { + final ImmutableMultimap multimap; + + EntryCollection(ImmutableMultimap multimap) { + this.multimap = multimap; + } + + @Override + public UnmodifiableIterator> iterator() { + return multimap.entryIterator(); + } + + @Override + boolean isPartialView() { + return multimap.isPartialView(); + } + + @Override + public int size() { + return multimap.size(); + } + + @Override + public boolean contains(Object object) { + if (object instanceof Entry) { + Entry entry = (Entry) object; + return multimap.containsEntry(entry.getKey(), entry.getValue()); + } + return false; + } + + private static final long serialVersionUID = 0; + } + + @Override + UnmodifiableIterator> entryIterator() { + return new UnmodifiableIterator>() { + final Iterator>> asMapItr = + map.entrySet().iterator(); + K currentKey = null; + Iterator valueItr = Iterators.emptyIterator(); + + @Override + public boolean hasNext() { + return valueItr.hasNext() || asMapItr.hasNext(); + } + + @Override + public Entry next() { + if (!valueItr.hasNext()) { + Entry> entry = asMapItr.next(); + currentKey = entry.getKey(); + valueItr = entry.getValue().iterator(); + } + return Maps.immutableEntry(currentKey, valueItr.next()); + } + }; + } + + @Override + Spliterator> entrySpliterator() { + return CollectSpliterators.flatMap( + asMap().entrySet().spliterator(), + keyToValueCollectionEntry -> { + K key = keyToValueCollectionEntry.getKey(); + Collection valueCollection = keyToValueCollectionEntry.getValue(); + return CollectSpliterators.map( + valueCollection.spliterator(), (V value) -> Maps.immutableEntry(key, value)); + }, + Spliterator.SIZED | (this instanceof SetMultimap ? Spliterator.DISTINCT : 0), + size()); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + asMap() + .forEach( + (key, valueCollection) -> valueCollection.forEach(value -> action.accept(key, value))); + } + + /** + * Returns an immutable multiset containing all the keys in this multimap, in the same order and + * with the same frequencies as they appear in this multimap; to get only a single occurrence of + * each key, use {@link #keySet}. + */ + @Override + public ImmutableMultiset keys() { + return (ImmutableMultiset) super.keys(); + } + + @Override + ImmutableMultiset createKeys() { + return new Keys(); + } + + @SuppressWarnings("serial") // Uses writeReplace, not default serialization + class Keys extends ImmutableMultiset { + @Override + public boolean contains(Object object) { + return containsKey(object); + } + + @Override + public int count(Object element) { + Collection values = map.get(element); + return (values == null) ? 0 : values.size(); + } + + @Override + public ImmutableSet elementSet() { + return keySet(); + } + + @Override + public int size() { + return ImmutableMultimap.this.size(); + } + + @Override + Entry getEntry(int index) { + Map.Entry> entry = map.entrySet().asList().get(index); + return Multisets.immutableEntry(entry.getKey(), entry.getValue().size()); + } + + @Override + boolean isPartialView() { + return true; + } + + @Override + Object writeReplace() { + return new KeysSerializedForm(ImmutableMultimap.this); + } + } + + private static final class KeysSerializedForm implements Serializable { + final ImmutableMultimap multimap; + + KeysSerializedForm(ImmutableMultimap multimap) { + this.multimap = multimap; + } + + Object readResolve() { + return multimap.keys(); + } + } + + /** + * Returns an immutable collection of the values in this multimap. Its iterator traverses the + * values for the first key, the values for the second key, and so on. + */ + @Override + public ImmutableCollection values() { + return (ImmutableCollection) super.values(); + } + + @Override + ImmutableCollection createValues() { + return new Values<>(this); + } + + @Override + UnmodifiableIterator valueIterator() { + return new UnmodifiableIterator() { + Iterator> valueCollectionItr = map.values().iterator(); + Iterator valueItr = Iterators.emptyIterator(); + + @Override + public boolean hasNext() { + return valueItr.hasNext() || valueCollectionItr.hasNext(); + } + + @Override + public V next() { + if (!valueItr.hasNext()) { + valueItr = valueCollectionItr.next().iterator(); + } + return valueItr.next(); + } + }; + } + + private static final class Values extends ImmutableCollection { + private final transient ImmutableMultimap multimap; + + Values(ImmutableMultimap multimap) { + this.multimap = multimap; + } + + @Override + public boolean contains(Object object) { + return multimap.containsValue(object); + } + + @Override + public UnmodifiableIterator iterator() { + return multimap.valueIterator(); + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + for (ImmutableCollection valueCollection : multimap.map.values()) { + offset = valueCollection.copyIntoArray(dst, offset); + } + return offset; + } + + @Override + public int size() { + return multimap.size(); + } + + @Override + boolean isPartialView() { + return true; + } + + private static final long serialVersionUID = 0; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultiset.java new file mode 100644 index 0000000000000..092bd51ec0c34 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultiset.java @@ -0,0 +1,630 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.function.ToIntFunction; +import java.util.stream.Collector; + + + +/** + * A {@link Multiset} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

Grouped iteration. In all current implementations, duplicate elements always appear + * consecutively when iterating. Elements iterate in order by the first appearance of that + * element when the multiset was created. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Jared Levy + * @author Louis Wasserman + * @since 2.0 + */ +@SuppressWarnings("serial") // we're overriding default serialization +public abstract class ImmutableMultiset extends ImmutableMultisetGwtSerializationDependencies + implements Multiset { + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableMultiset}. Elements iterate in order by the first appearance of that element in + * encounter order. + * + * @since 21.0 + */ + + public static Collector> toImmutableMultiset() { + return toImmutableMultiset(Function.identity(), e -> 1); + } + + /** + * Returns a {@code Collector} that accumulates elements into an {@code ImmutableMultiset} whose + * elements are the result of applying {@code elementFunction} to the inputs, with counts equal to + * the result of applying {@code countFunction} to the inputs. + * + *

If the mapped elements contain duplicates (according to {@link Object#equals}), the first + * occurrence in encounter order appears in the resulting multiset, with count equal to the sum of + * the outputs of {@code countFunction.applyAsInt(t)} for each {@code t} mapped to that element. + * + * @since 22.0 + */ + public static Collector> toImmutableMultiset( + Function elementFunction, ToIntFunction countFunction) { + checkNotNull(elementFunction); + checkNotNull(countFunction); + return Collector.of( + LinkedHashMultiset::create, + (multiset, t) -> + multiset.add(checkNotNull(elementFunction.apply(t)), countFunction.applyAsInt(t)), + (multiset1, multiset2) -> { + multiset1.addAll(multiset2); + return multiset1; + }, + (Multiset multiset) -> copyFromEntries(multiset.entrySet())); + } + + /** Returns the empty immutable multiset. */ + @SuppressWarnings({"unchecked", "rawtypes"}) // all supported methods are covariant + public static ImmutableMultiset of() { + return (ImmutableMultiset) RegularImmutableMultiset.EMPTY; + } + + /** + * Returns an immutable multiset containing a single element. + * + * @throws NullPointerException if {@code element} is null + * @since 6.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // generic array created but never written + public static ImmutableMultiset of(E element) { + return copyFromElements(element); + } + + /** + * Returns an immutable multiset containing the given elements, in order. + * + * @throws NullPointerException if any element is null + * @since 6.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // + public static ImmutableMultiset of(E e1, E e2) { + return copyFromElements(e1, e2); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any element is null + * @since 6.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // + public static ImmutableMultiset of(E e1, E e2, E e3) { + return copyFromElements(e1, e2, e3); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any element is null + * @since 6.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // + public static ImmutableMultiset of(E e1, E e2, E e3, E e4) { + return copyFromElements(e1, e2, e3, e4); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any element is null + * @since 6.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // + public static ImmutableMultiset of(E e1, E e2, E e3, E e4, E e5) { + return copyFromElements(e1, e2, e3, e4, e5); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any element is null + * @since 6.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // + public static ImmutableMultiset of(E e1, E e2, E e3, E e4, E e5, E e6, E... others) { + return new Builder().add(e1).add(e2).add(e3).add(e4).add(e5).add(e6).add(others).build(); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any of {@code elements} is null + * @since 6.0 + */ + public static ImmutableMultiset copyOf(E[] elements) { + return copyFromElements(elements); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableMultiset copyOf(Iterable elements) { + if (elements instanceof ImmutableMultiset) { + @SuppressWarnings({"unchecked", "rawtypes"}) // all supported methods are covariant + ImmutableMultiset result = (ImmutableMultiset) elements; + if (!result.isPartialView()) { + return result; + } + } + + Multiset multiset = + (elements instanceof Multiset) + ? Multisets.cast(elements) + : LinkedHashMultiset.create(elements); + + return copyFromEntries(multiset.entrySet()); + } + + /** + * Returns an immutable multiset containing the given elements, in the "grouped iteration order" + * described in the class documentation. + * + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableMultiset copyOf(Iterator elements) { + Multiset multiset = LinkedHashMultiset.create(); + Iterators.addAll(multiset, elements); + return copyFromEntries(multiset.entrySet()); + } + + @SafeVarargs + @SuppressWarnings("varargs") + private static ImmutableMultiset copyFromElements(E... elements) { + Multiset multiset = LinkedHashMultiset.create(); + Collections.addAll(multiset, elements); + return copyFromEntries(multiset.entrySet()); + } + + static ImmutableMultiset copyFromEntries( + Collection> entries) { + if (entries.isEmpty()) { + return of(); + } else { + return RegularImmutableMultiset.create(entries); + } + } + + ImmutableMultiset() {} + + @Override + public UnmodifiableIterator iterator() { + final Iterator> entryIterator = entrySet().iterator(); + return new UnmodifiableIterator() { + int remaining; + E element; + + @Override + public boolean hasNext() { + return (remaining > 0) || entryIterator.hasNext(); + } + + @Override + public E next() { + if (remaining <= 0) { + Entry entry = entryIterator.next(); + element = entry.getElement(); + remaining = entry.getCount(); + } + remaining--; + return element; + } + }; + } + + private transient ImmutableList asList; + + @Override + public ImmutableList asList() { + ImmutableList result = asList; + return (result == null) ? asList = super.asList() : result; + } + + @Override + public boolean contains(Object object) { + return count(object) > 0; + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final int add(E element, int occurrences) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final int remove(Object element, int occurrences) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final int setCount(E element, int count) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean setCount(E element, int oldCount, int newCount) { + throw new UnsupportedOperationException(); + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + for (Multiset.Entry entry : entrySet()) { + Arrays.fill(dst, offset, offset + entry.getCount(), entry.getElement()); + offset += entry.getCount(); + } + return offset; + } + + @Override + public boolean equals(Object object) { + return Multisets.equalsImpl(this, object); + } + + @Override + public int hashCode() { + return Sets.hashCodeImpl(entrySet()); + } + + @Override + public String toString() { + return entrySet().toString(); + } + + /** @since 21.0 (present with return type {@code Set} since 2.0) */ + @Override + public abstract ImmutableSet elementSet(); + + private transient ImmutableSet> entrySet; + + @Override + public ImmutableSet> entrySet() { + ImmutableSet> es = entrySet; + return (es == null) ? (entrySet = createEntrySet()) : es; + } + + private ImmutableSet> createEntrySet() { + return isEmpty() ? ImmutableSet.>of() : new EntrySet(); + } + + abstract Entry getEntry(int index); + + private final class EntrySet extends IndexedImmutableSet> { + @Override + boolean isPartialView() { + return ImmutableMultiset.this.isPartialView(); + } + + @Override + Entry get(int index) { + return getEntry(index); + } + + @Override + public int size() { + return elementSet().size(); + } + + @Override + public boolean contains(Object o) { + if (o instanceof Entry) { + Entry entry = (Entry) o; + if (entry.getCount() <= 0) { + return false; + } + int count = count(entry.getElement()); + return count == entry.getCount(); + } + return false; + } + + @Override + public int hashCode() { + return ImmutableMultiset.this.hashCode(); + } + + @Override + Object writeReplace() { + return new EntrySetSerializedForm(ImmutableMultiset.this); + } + + private static final long serialVersionUID = 0; + } + + static class EntrySetSerializedForm implements Serializable { + final ImmutableMultiset multiset; + + EntrySetSerializedForm(ImmutableMultiset multiset) { + this.multiset = multiset; + } + + Object readResolve() { + return multiset.entrySet(); + } + } + + @Override + Object writeReplace() { + return new SerializedForm(this); + } + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * A builder for creating immutable multiset instances, especially {@code public static final} + * multisets ("constant multisets"). Example: + * + *

{@code
+   * public static final ImmutableMultiset BEANS =
+   *     new ImmutableMultiset.Builder()
+   *         .addCopies(Bean.COCOA, 4)
+   *         .addCopies(Bean.GARDEN, 6)
+   *         .addCopies(Bean.RED, 8)
+   *         .addCopies(Bean.BLACK_EYED, 10)
+   *         .build();
+   * }
+ * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple multisets in series. + * + * @since 2.0 + */ + @SuppressWarnings("unchecked") + public static class Builder extends ImmutableCollection.Builder { + final Multiset contents; + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableMultiset#builder}. + */ + public Builder() { + this(LinkedHashMultiset.create()); + } + + Builder(Multiset contents) { + this.contents = contents; + } + + /** + * Adds {@code element} to the {@code ImmutableMultiset}. + * + * @param element the element to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + */ + @Override + public Builder add(E element) { + contents.add(checkNotNull(element)); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableMultiset}. + * + * @param elements the elements to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @SuppressWarnings("unchecked") + @Override + public Builder add(E... elements) { + super.add(elements); + return this; + } + + /** + * Adds a number of occurrences of an element to this {@code ImmutableMultiset}. + * + * @param element the element to add + * @param occurrences the number of occurrences of the element to add. May be zero, in which + * case no change will be made. + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + * @throws IllegalArgumentException if {@code occurrences} is negative, or if this operation + * would result in more than {@link Integer#MAX_VALUE} occurrences of the element + */ + public Builder addCopies(E element, int occurrences) { + contents.add(checkNotNull(element), occurrences); + return this; + } + + /** + * Adds or removes the necessary occurrences of an element such that the element attains the + * desired count. + * + * @param element the element to add or remove occurrences of + * @param count the desired count of the element in this multiset + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + * @throws IllegalArgumentException if {@code count} is negative + */ + public Builder setCount(E element, int count) { + contents.setCount(checkNotNull(element), count); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableMultiset}. + * + * @param elements the {@code Iterable} to add to the {@code ImmutableMultiset} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @Override + public Builder addAll(Iterable elements) { + if (elements instanceof Multiset) { + Multiset multiset = Multisets.cast(elements); + multiset.forEachEntry((e, n) -> contents.add(checkNotNull(e), n)); + } else { + super.addAll(elements); + } + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableMultiset}. + * + * @param elements the elements to add to the {@code ImmutableMultiset} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @Override + public Builder addAll(Iterator elements) { + super.addAll(elements); + return this; + } + + /** + * Returns a newly-created {@code ImmutableMultiset} based on the contents of the {@code + * Builder}. + */ + @Override + public ImmutableMultiset build() { + return copyOf(contents); + } + + + ImmutableMultiset buildJdkBacked() { + if (contents.isEmpty()) { + return of(); + } + return JdkBackedImmutableMultiset.create(contents.entrySet()); + } + } + + static final class ElementSet extends ImmutableSet.Indexed { + private final List> entries; + private final Multiset delegate; + + ElementSet(List> entries, Multiset delegate) { + this.entries = entries; + this.delegate = delegate; + } + + @Override + E get(int index) { + return entries.get(index).getElement(); + } + + @Override + public boolean contains(Object object) { + return delegate.contains(object); + } + + @Override + boolean isPartialView() { + return true; + } + + @Override + public int size() { + return entries.size(); + } + } + + static final class SerializedForm implements Serializable { + final Object[] elements; + final int[] counts; + + SerializedForm(Multiset multiset) { + int distinct = multiset.entrySet().size(); + elements = new Object[distinct]; + counts = new int[distinct]; + int i = 0; + for (Entry entry : multiset.entrySet()) { + elements[i] = entry.getElement(); + counts[i] = entry.getCount(); + i++; + } + } + + Object readResolve() { + LinkedHashMultiset multiset = LinkedHashMultiset.create(elements.length); + for (int i = 0; i < elements.length; i++) { + multiset.add(elements[i], counts[i]); + } + return ImmutableMultiset.copyOf(multiset); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultisetGwtSerializationDependencies.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultisetGwtSerializationDependencies.java new file mode 100644 index 0000000000000..1c21b9a6d8435 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableMultisetGwtSerializationDependencies.java @@ -0,0 +1,42 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * A dummy superclass to support GWT serialization of the element type of an {@link + * ImmutableMultiset}. The GWT supersource for this class contains a field of type {@code E}. + * + *

For details about this hack, see {@link GwtSerializationDependencies}, which takes the same + * approach but with a subclass rather than a superclass. + * + *

TODO(cpovirk): Consider applying this subclass approach to our other types. + * + *

For {@code ImmutableMultiset} in particular, I ran into a problem with the {@code + * GwtSerializationDependencies} approach: When autogenerating a serializer for the new class, GWT + * tries to refer to our dummy serializer for the superclass, + * ImmutableMultiset_CustomFieldSerializer. But that type has no methods (since it's never actually + * used). We could probably fix the problem by adding dummy methods to that class, but that is + * starting to sound harder than taking the superclass approach, which I've been coming to like, + * anyway, since it doesn't require us to declare dummy methods (though occasionally constructors) + * and make types non-final. + */ +abstract class ImmutableMultisetGwtSerializationDependencies extends ImmutableCollection {} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeMap.java new file mode 100644 index 0000000000000..b08429902ea78 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeMap.java @@ -0,0 +1,411 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyAbsentBehavior; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyPresentBehavior; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.function.Function; +import java.util.stream.Collector; + + +/** + * A {@link RangeMap} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + * @author Louis Wasserman + * @since 14.0 + */ + +public class ImmutableRangeMap, V> implements RangeMap, Serializable { + + private static final ImmutableRangeMap, Object> EMPTY = + new ImmutableRangeMap<>(ImmutableList.>>of(), ImmutableList.of()); + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableRangeMap}. As in {@link Builder}, overlapping ranges are not permitted. + * + * @since 23.1 + */ + + public static , V> + Collector> toImmutableRangeMap( + Function> keyFunction, + Function valueFunction) { + return CollectCollectors.toImmutableRangeMap(keyFunction, valueFunction); + } + + /** Returns an empty immutable range map. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static , V> ImmutableRangeMap of() { + return (ImmutableRangeMap) EMPTY; + } + + /** Returns an immutable range map mapping a single range to a single value. */ + public static , V> ImmutableRangeMap of(Range range, V value) { + return new ImmutableRangeMap<>(ImmutableList.of(range), ImmutableList.of(value)); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static , V> ImmutableRangeMap copyOf( + RangeMap rangeMap) { + if (rangeMap instanceof ImmutableRangeMap) { + return (ImmutableRangeMap) rangeMap; + } + Map, ? extends V> map = rangeMap.asMapOfRanges(); + ImmutableList.Builder> rangesBuilder = new ImmutableList.Builder<>(map.size()); + ImmutableList.Builder valuesBuilder = new ImmutableList.Builder(map.size()); + for (Entry, ? extends V> entry : map.entrySet()) { + rangesBuilder.add(entry.getKey()); + valuesBuilder.add(entry.getValue()); + } + return new ImmutableRangeMap<>(rangesBuilder.build(), valuesBuilder.build()); + } + + /** Returns a new builder for an immutable range map. */ + public static , V> Builder builder() { + return new Builder<>(); + } + + /** + * A builder for immutable range maps. Overlapping ranges are prohibited. + * + * @since 14.0 + */ + public static final class Builder, V> { + private final List, V>> entries; + + public Builder() { + this.entries = Lists.newArrayList(); + } + + /** + * Associates the specified range with the specified value. + * + * @throws IllegalArgumentException if {@code range} is empty + */ + public Builder put(Range range, V value) { + checkNotNull(range); + checkNotNull(value); + checkArgument(!range.isEmpty(), "Range must not be empty, but was %s", range); + entries.add(Maps.immutableEntry(range, value)); + return this; + } + + /** Copies all associations from the specified range map into this builder. */ + public Builder putAll(RangeMap rangeMap) { + for (Entry, ? extends V> entry : rangeMap.asMapOfRanges().entrySet()) { + put(entry.getKey(), entry.getValue()); + } + return this; + } + + Builder combine(Builder builder) { + entries.addAll(builder.entries); + return this; + } + + /** + * Returns an {@code ImmutableRangeMap} containing the associations previously added to this + * builder. + * + * @throws IllegalArgumentException if any two ranges inserted into this builder overlap + */ + public ImmutableRangeMap build() { + Collections.sort(entries, Range.rangeLexOrdering().onKeys()); + ImmutableList.Builder> rangesBuilder = new ImmutableList.Builder<>(entries.size()); + ImmutableList.Builder valuesBuilder = new ImmutableList.Builder(entries.size()); + for (int i = 0; i < entries.size(); i++) { + Range range = entries.get(i).getKey(); + if (i > 0) { + Range prevRange = entries.get(i - 1).getKey(); + if (range.isConnected(prevRange) && !range.intersection(prevRange).isEmpty()) { + throw new IllegalArgumentException( + "Overlapping ranges: range " + prevRange + " overlaps with entry " + range); + } + } + rangesBuilder.add(range); + valuesBuilder.add(entries.get(i).getValue()); + } + return new ImmutableRangeMap<>(rangesBuilder.build(), valuesBuilder.build()); + } + } + + private final transient ImmutableList> ranges; + private final transient ImmutableList values; + + ImmutableRangeMap(ImmutableList> ranges, ImmutableList values) { + this.ranges = ranges; + this.values = values; + } + + @Override + + public V get(K key) { + int index = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + Cut.belowValue(key), + KeyPresentBehavior.ANY_PRESENT, + KeyAbsentBehavior.NEXT_LOWER); + if (index == -1) { + return null; + } else { + Range range = ranges.get(index); + return range.contains(key) ? values.get(index) : null; + } + } + + @Override + + public Entry, V> getEntry(K key) { + int index = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + Cut.belowValue(key), + KeyPresentBehavior.ANY_PRESENT, + KeyAbsentBehavior.NEXT_LOWER); + if (index == -1) { + return null; + } else { + Range range = ranges.get(index); + return range.contains(key) ? Maps.immutableEntry(range, values.get(index)) : null; + } + } + + @Override + public Range span() { + if (ranges.isEmpty()) { + throw new NoSuchElementException(); + } + Range firstRange = ranges.get(0); + Range lastRange = ranges.get(ranges.size() - 1); + return Range.create(firstRange.lowerBound, lastRange.upperBound); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeMap} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void put(Range range, V value) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeMap} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void putCoalescing(Range range, V value) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeMap} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void putAll(RangeMap rangeMap) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeMap} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeMap} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void remove(Range range) { + throw new UnsupportedOperationException(); + } + + @Override + public ImmutableMap, V> asMapOfRanges() { + if (ranges.isEmpty()) { + return ImmutableMap.of(); + } + RegularImmutableSortedSet> rangeSet = + new RegularImmutableSortedSet<>(ranges, Range.rangeLexOrdering()); + return new ImmutableSortedMap<>(rangeSet, values); + } + + @Override + public ImmutableMap, V> asDescendingMapOfRanges() { + if (ranges.isEmpty()) { + return ImmutableMap.of(); + } + RegularImmutableSortedSet> rangeSet = + new RegularImmutableSortedSet<>(ranges.reverse(), Range.rangeLexOrdering().reverse()); + return new ImmutableSortedMap<>(rangeSet, values.reverse()); + } + + @Override + public ImmutableRangeMap subRangeMap(final Range range) { + if (checkNotNull(range).isEmpty()) { + return ImmutableRangeMap.of(); + } else if (ranges.isEmpty() || range.encloses(span())) { + return this; + } + int lowerIndex = + SortedLists.binarySearch( + ranges, + Range.upperBoundFn(), + range.lowerBound, + KeyPresentBehavior.FIRST_AFTER, + KeyAbsentBehavior.NEXT_HIGHER); + int upperIndex = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + range.upperBound, + KeyPresentBehavior.ANY_PRESENT, + KeyAbsentBehavior.NEXT_HIGHER); + if (lowerIndex >= upperIndex) { + return ImmutableRangeMap.of(); + } + final int off = lowerIndex; + final int len = upperIndex - lowerIndex; + ImmutableList> subRanges = + new ImmutableList>() { + @Override + public int size() { + return len; + } + + @Override + public Range get(int index) { + checkElementIndex(index, len); + if (index == 0 || index == len - 1) { + return ranges.get(index + off).intersection(range); + } else { + return ranges.get(index + off); + } + } + + @Override + boolean isPartialView() { + return true; + } + }; + final ImmutableRangeMap outer = this; + return new ImmutableRangeMap(subRanges, values.subList(lowerIndex, upperIndex)) { + @Override + public ImmutableRangeMap subRangeMap(Range subRange) { + if (range.isConnected(subRange)) { + return outer.subRangeMap(subRange.intersection(range)); + } else { + return ImmutableRangeMap.of(); + } + } + }; + } + + @Override + public int hashCode() { + return asMapOfRanges().hashCode(); + } + + @Override + public boolean equals(Object o) { + if (o instanceof RangeMap) { + RangeMap rangeMap = (RangeMap) o; + return asMapOfRanges().equals(rangeMap.asMapOfRanges()); + } + return false; + } + + @Override + public String toString() { + return asMapOfRanges().toString(); + } + + /** + * This class is used to serialize ImmutableRangeMap instances. Serializes the {@link + * #asMapOfRanges()} form. + */ + private static class SerializedForm, V> implements Serializable { + + private final ImmutableMap, V> mapOfRanges; + + SerializedForm(ImmutableMap, V> mapOfRanges) { + this.mapOfRanges = mapOfRanges; + } + + Object readResolve() { + if (mapOfRanges.isEmpty()) { + return of(); + } else { + return createRangeMap(); + } + } + + Object createRangeMap() { + Builder builder = new Builder<>(); + for (Entry, V> entry : mapOfRanges.entrySet()) { + builder.put(entry.getKey(), entry.getValue()); + } + return builder.build(); + } + + private static final long serialVersionUID = 0; + } + + Object writeReplace() { + return new SerializedForm<>(asMapOfRanges()); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeSet.java new file mode 100644 index 0000000000000..4f910e7e1a3db --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableRangeSet.java @@ -0,0 +1,831 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyAbsentBehavior.NEXT_HIGHER; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyAbsentBehavior.NEXT_LOWER; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyPresentBehavior.ANY_PRESENT; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyAbsentBehavior; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedLists.KeyPresentBehavior; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.Serializable; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.stream.Collector; + + + +/** + * A {@link RangeSet} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + * @author Louis Wasserman + * @since 14.0 + */ + +@SuppressWarnings("rawtypes") +public final class ImmutableRangeSet extends AbstractRangeSet + implements Serializable { + + private static final ImmutableRangeSet> EMPTY = + new ImmutableRangeSet<>(ImmutableList.>>of()); + + private static final ImmutableRangeSet> ALL = + new ImmutableRangeSet<>(ImmutableList.of(Range.>all())); + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableRangeSet}. As in {@link Builder}, overlapping ranges are not permitted and adjacent + * ranges will be merged. + * + * @since 23.1 + */ + + public static > + Collector, ?, ImmutableRangeSet> toImmutableRangeSet() { + return CollectCollectors.toImmutableRangeSet(); + } + + /** Returns an empty immutable range set. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableRangeSet of() { + return (ImmutableRangeSet) EMPTY; + } + + /** + * Returns an immutable range set containing the specified single range. If {@link Range#isEmpty() + * range.isEmpty()}, this is equivalent to {@link ImmutableRangeSet#of()}. + */ + @SuppressWarnings("rawtypes") + public static ImmutableRangeSet of(Range range) { + checkNotNull(range); + if (range.isEmpty()) { + return of(); + } else if (range.equals(Range.all())) { + return all(); + } else { + return new ImmutableRangeSet(ImmutableList.of(range)); + } + } + + /** Returns an immutable range set containing the single range {@link Range#all()}. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + static ImmutableRangeSet all() { + return (ImmutableRangeSet) ALL; + } + + /** Returns an immutable copy of the specified {@code RangeSet}. */ + @SuppressWarnings("rawtypes") + public static ImmutableRangeSet copyOf(RangeSet rangeSet) { + checkNotNull(rangeSet); + if (rangeSet.isEmpty()) { + return of(); + } else if (rangeSet.encloses(Range.all())) { + return all(); + } + + if (rangeSet instanceof ImmutableRangeSet) { + ImmutableRangeSet immutableRangeSet = (ImmutableRangeSet) rangeSet; + if (!immutableRangeSet.isPartialView()) { + return immutableRangeSet; + } + } + return new ImmutableRangeSet(ImmutableList.copyOf(rangeSet.asRanges())); + } + + /** + * Returns an {@code ImmutableRangeSet} containing each of the specified disjoint ranges. + * Overlapping ranges and empty ranges are forbidden, though adjacent ranges are permitted and + * will be merged. + * + * @throws IllegalArgumentException if any ranges overlap or are empty + * @since 21.0 + */ + public static > ImmutableRangeSet copyOf(Iterable> ranges) { + return new Builder().addAll(ranges).build(); + } + + /** + * Returns an {@code ImmutableRangeSet} representing the union of the specified ranges. + * + *

This is the smallest {@code RangeSet} which encloses each of the specified ranges. Duplicate + * or connected ranges are permitted, and will be coalesced in the result. + * + * @since 21.0 + */ + public static > ImmutableRangeSet unionOf(Iterable> ranges) { + return copyOf(TreeRangeSet.create(ranges)); + } + + ImmutableRangeSet(ImmutableList> ranges) { + this.ranges = ranges; + } + + private ImmutableRangeSet(ImmutableList> ranges, ImmutableRangeSet complement) { + this.ranges = ranges; + this.complement = complement; + } + + private final transient ImmutableList> ranges; + + @Override + public boolean intersects(Range otherRange) { + int ceilingIndex = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + otherRange.lowerBound, + Ordering.natural(), + ANY_PRESENT, + NEXT_HIGHER); + if (ceilingIndex < ranges.size() + && ranges.get(ceilingIndex).isConnected(otherRange) + && !ranges.get(ceilingIndex).intersection(otherRange).isEmpty()) { + return true; + } + return ceilingIndex > 0 + && ranges.get(ceilingIndex - 1).isConnected(otherRange) + && !ranges.get(ceilingIndex - 1).intersection(otherRange).isEmpty(); + } + + @Override + public boolean encloses(Range otherRange) { + int index = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + otherRange.lowerBound, + Ordering.natural(), + ANY_PRESENT, + NEXT_LOWER); + return index != -1 && ranges.get(index).encloses(otherRange); + } + + @Override + public Range rangeContaining(C value) { + int index = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + Cut.belowValue(value), + Ordering.natural(), + ANY_PRESENT, + NEXT_LOWER); + if (index != -1) { + Range range = ranges.get(index); + return range.contains(value) ? range : null; + } + return null; + } + + @Override + public Range span() { + if (ranges.isEmpty()) { + throw new NoSuchElementException(); + } + return Range.create(ranges.get(0).lowerBound, ranges.get(ranges.size() - 1).upperBound); + } + + @Override + public boolean isEmpty() { + return ranges.isEmpty(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeSet} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void add(Range range) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeSet} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void addAll(RangeSet other) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeSet} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void addAll(Iterable> other) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeSet} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void remove(Range range) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeSet} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void removeAll(RangeSet other) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the {@code RangeSet} unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public void removeAll(Iterable> other) { + throw new UnsupportedOperationException(); + } + + @Override + public ImmutableSet> asRanges() { + if (ranges.isEmpty()) { + return ImmutableSet.of(); + } + return new RegularImmutableSortedSet<>(ranges, Range.rangeLexOrdering()); + } + + @Override + public ImmutableSet> asDescendingSetOfRanges() { + if (ranges.isEmpty()) { + return ImmutableSet.of(); + } + return new RegularImmutableSortedSet<>(ranges.reverse(), Range.rangeLexOrdering().reverse()); + } + + private transient ImmutableRangeSet complement; + + private final class ComplementRanges extends ImmutableList> { + // True if the "positive" range set is empty or bounded below. + private final boolean positiveBoundedBelow; + + // True if the "positive" range set is empty or bounded above. + private final boolean positiveBoundedAbove; + + private final int size; + + ComplementRanges() { + this.positiveBoundedBelow = ranges.get(0).hasLowerBound(); + this.positiveBoundedAbove = Iterables.getLast(ranges).hasUpperBound(); + + int size = ranges.size() - 1; + if (positiveBoundedBelow) { + size++; + } + if (positiveBoundedAbove) { + size++; + } + this.size = size; + } + + @Override + public int size() { + return size; + } + + @Override + public Range get(int index) { + checkElementIndex(index, size); + + Cut lowerBound; + if (positiveBoundedBelow) { + lowerBound = (index == 0) ? Cut.belowAll() : ranges.get(index - 1).upperBound; + } else { + lowerBound = ranges.get(index).upperBound; + } + + Cut upperBound; + if (positiveBoundedAbove && index == size - 1) { + upperBound = Cut.aboveAll(); + } else { + upperBound = ranges.get(index + (positiveBoundedBelow ? 0 : 1)).lowerBound; + } + + return Range.create(lowerBound, upperBound); + } + + @Override + boolean isPartialView() { + return true; + } + } + + @Override + public ImmutableRangeSet complement() { + ImmutableRangeSet result = complement; + if (result != null) { + return result; + } else if (ranges.isEmpty()) { + return complement = all(); + } else if (ranges.size() == 1 && ranges.get(0).equals(Range.all())) { + return complement = of(); + } else { + ImmutableList> complementRanges = new ComplementRanges(); + result = complement = new ImmutableRangeSet(complementRanges, this); + } + return result; + } + + /** + * Returns a new range set consisting of the union of this range set and {@code other}. + * + *

This is essentially the same as {@code TreeRangeSet.create(this).addAll(other)} except it + * returns an {@code ImmutableRangeSet}. + * + * @since 21.0 + */ + public ImmutableRangeSet union(RangeSet other) { + return unionOf(Iterables.concat(asRanges(), other.asRanges())); + } + + /** + * Returns a new range set consisting of the intersection of this range set and {@code other}. + * + *

This is essentially the same as {@code + * TreeRangeSet.create(this).removeAll(other.complement())} except it returns an {@code + * ImmutableRangeSet}. + * + * @since 21.0 + */ + public ImmutableRangeSet intersection(RangeSet other) { + RangeSet copy = TreeRangeSet.create(this); + copy.removeAll(other.complement()); + return copyOf(copy); + } + + /** + * Returns a new range set consisting of the difference of this range set and {@code other}. + * + *

This is essentially the same as {@code TreeRangeSet.create(this).removeAll(other)} except it + * returns an {@code ImmutableRangeSet}. + * + * @since 21.0 + */ + public ImmutableRangeSet difference(RangeSet other) { + RangeSet copy = TreeRangeSet.create(this); + copy.removeAll(other); + return copyOf(copy); + } + + /** + * Returns a list containing the nonempty intersections of {@code range} with the ranges in this + * range set. + */ + private ImmutableList> intersectRanges(final Range range) { + if (ranges.isEmpty() || range.isEmpty()) { + return ImmutableList.of(); + } else if (range.encloses(span())) { + return ranges; + } + + final int fromIndex; + if (range.hasLowerBound()) { + fromIndex = + SortedLists.binarySearch( + ranges, + Range.upperBoundFn(), + range.lowerBound, + KeyPresentBehavior.FIRST_AFTER, + KeyAbsentBehavior.NEXT_HIGHER); + } else { + fromIndex = 0; + } + + int toIndex; + if (range.hasUpperBound()) { + toIndex = + SortedLists.binarySearch( + ranges, + Range.lowerBoundFn(), + range.upperBound, + KeyPresentBehavior.FIRST_PRESENT, + KeyAbsentBehavior.NEXT_HIGHER); + } else { + toIndex = ranges.size(); + } + final int length = toIndex - fromIndex; + if (length == 0) { + return ImmutableList.of(); + } else { + return new ImmutableList>() { + @Override + public int size() { + return length; + } + + @Override + public Range get(int index) { + checkElementIndex(index, length); + if (index == 0 || index == length - 1) { + return ranges.get(index + fromIndex).intersection(range); + } else { + return ranges.get(index + fromIndex); + } + } + + @Override + boolean isPartialView() { + return true; + } + }; + } + } + + /** Returns a view of the intersection of this range set with the given range. */ + @Override + public ImmutableRangeSet subRangeSet(Range range) { + if (!isEmpty()) { + Range span = span(); + if (range.encloses(span)) { + return this; + } else if (range.isConnected(span)) { + return new ImmutableRangeSet(intersectRanges(range)); + } + } + return of(); + } + + /** + * Returns an {@link ImmutableSortedSet} containing the same values in the given domain + * {@linkplain RangeSet#contains contained} by this range set. + * + *

Note: {@code a.asSet(d).equals(b.asSet(d))} does not imply {@code a.equals(b)}! For + * example, {@code a} and {@code b} could be {@code [2..4]} and {@code (1..5)}, or the empty + * ranges {@code [3..3)} and {@code [4..4)}. + * + *

Warning: Be extremely careful what you do with the {@code asSet} view of a large + * range set (such as {@code ImmutableRangeSet.of(Range.greaterThan(0))}). Certain operations on + * such a set can be performed efficiently, but others (such as {@link Set#hashCode} or {@link + * Collections#frequency}) can cause major performance problems. + * + *

The returned set's {@link Object#toString} method returns a short-hand form of the set's + * contents, such as {@code "[1..100]}"}. + * + * @throws IllegalArgumentException if neither this range nor the domain has a lower bound, or if + * neither has an upper bound + */ + public ImmutableSortedSet asSet(DiscreteDomain domain) { + checkNotNull(domain); + if (isEmpty()) { + return ImmutableSortedSet.of(); + } + Range span = span().canonical(domain); + if (!span.hasLowerBound()) { + // according to the spec of canonical, neither this ImmutableRangeSet nor + // the range have a lower bound + throw new IllegalArgumentException( + "Neither the DiscreteDomain nor this range set are bounded below"); + } else if (!span.hasUpperBound()) { + try { + domain.maxValue(); + } catch (NoSuchElementException e) { + throw new IllegalArgumentException( + "Neither the DiscreteDomain nor this range set are bounded above"); + } + } + + return new AsSet(domain); + } + + private final class AsSet extends ImmutableSortedSet { + private final DiscreteDomain domain; + + AsSet(DiscreteDomain domain) { + super(Ordering.natural()); + this.domain = domain; + } + + private transient Integer size; + + @Override + public int size() { + // racy single-check idiom + Integer result = size; + if (result == null) { + long total = 0; + for (Range range : ranges) { + total += ContiguousSet.create(range, domain).size(); + if (total >= Integer.MAX_VALUE) { + break; + } + } + result = size = Ints.saturatedCast(total); + } + return result.intValue(); + } + + @Override + public UnmodifiableIterator iterator() { + return new AbstractIterator() { + final Iterator> rangeItr = ranges.iterator(); + Iterator elemItr = Iterators.emptyIterator(); + + @Override + protected C computeNext() { + while (!elemItr.hasNext()) { + if (rangeItr.hasNext()) { + elemItr = ContiguousSet.create(rangeItr.next(), domain).iterator(); + } else { + return endOfData(); + } + } + return elemItr.next(); + } + }; + } + + @Override + public UnmodifiableIterator descendingIterator() { + return new AbstractIterator() { + final Iterator> rangeItr = ranges.reverse().iterator(); + Iterator elemItr = Iterators.emptyIterator(); + + @Override + protected C computeNext() { + while (!elemItr.hasNext()) { + if (rangeItr.hasNext()) { + elemItr = ContiguousSet.create(rangeItr.next(), domain).descendingIterator(); + } else { + return endOfData(); + } + } + return elemItr.next(); + } + }; + } + + ImmutableSortedSet subSet(Range range) { + return subRangeSet(range).asSet(domain); + } + + @Override + ImmutableSortedSet headSetImpl(C toElement, boolean inclusive) { + return subSet(Range.upTo(toElement, BoundType.forBoolean(inclusive))); + } + + @Override + ImmutableSortedSet subSetImpl( + C fromElement, boolean fromInclusive, C toElement, boolean toInclusive) { + if (!fromInclusive && !toInclusive && Range.compareOrThrow(fromElement, toElement) == 0) { + return ImmutableSortedSet.of(); + } + return subSet( + Range.range( + fromElement, BoundType.forBoolean(fromInclusive), + toElement, BoundType.forBoolean(toInclusive))); + } + + @Override + ImmutableSortedSet tailSetImpl(C fromElement, boolean inclusive) { + return subSet(Range.downTo(fromElement, BoundType.forBoolean(inclusive))); + } + + @Override + public boolean contains(Object o) { + if (o == null) { + return false; + } + try { + @SuppressWarnings({"unchecked", "rawtypes"}) // we catch CCE's + C c = (C) o; + return ImmutableRangeSet.this.contains(c); + } catch (ClassCastException e) { + return false; + } + } + + @Override + int indexOf(Object target) { + if (contains(target)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // if it's contained, it's definitely a C + C c = (C) target; + long total = 0; + for (Range range : ranges) { + if (range.contains(c)) { + return Ints.saturatedCast(total + ContiguousSet.create(range, domain).indexOf(c)); + } else { + total += ContiguousSet.create(range, domain).size(); + } + } + throw new AssertionError("impossible"); + } + return -1; + } + + @Override + ImmutableSortedSet createDescendingSet() { + return new DescendingImmutableSortedSet(this); + } + + @Override + boolean isPartialView() { + return ranges.isPartialView(); + } + + @Override + public String toString() { + return ranges.toString(); + } + + @Override + Object writeReplace() { + return new AsSetSerializedForm(ranges, domain); + } + } + + @SuppressWarnings("rawtypes") + private static class AsSetSerializedForm implements Serializable { + private final ImmutableList> ranges; + private final DiscreteDomain domain; + + AsSetSerializedForm(ImmutableList> ranges, DiscreteDomain domain) { + this.ranges = ranges; + this.domain = domain; + } + + Object readResolve() { + return new ImmutableRangeSet(ranges).asSet(domain); + } + } + + /** + * Returns {@code true} if this immutable range set's implementation contains references to + * user-created objects that aren't accessible via this range set's methods. This is generally + * used to determine whether {@code copyOf} implementations should make an explicit copy to avoid + * memory leaks. + */ + boolean isPartialView() { + return ranges.isPartialView(); + } + + /** Returns a new builder for an immutable range set. */ + public static > Builder builder() { + return new Builder(); + } + + /** + * A builder for immutable range sets. + * + * @since 14.0 + */ + public static class Builder> { + private final List> ranges; + + public Builder() { + this.ranges = Lists.newArrayList(); + } + + // TODO(lowasser): consider adding union, in addition to add, that does allow overlap + + /** + * Add the specified range to this builder. Adjacent ranges are permitted and will be merged, + * but overlapping ranges will cause an exception when {@link #build()} is called. + * + * @throws IllegalArgumentException if {@code range} is empty + */ + public Builder add(Range range) { + checkArgument(!range.isEmpty(), "range must not be empty, but was %s", range); + ranges.add(range); + return this; + } + + /** + * Add all ranges from the specified range set to this builder. Adjacent ranges are permitted + * and will be merged, but overlapping ranges will cause an exception when {@link #build()} is + * called. + */ + public Builder addAll(RangeSet ranges) { + return addAll(ranges.asRanges()); + } + + /** + * Add all of the specified ranges to this builder. Adjacent ranges are permitted and will be + * merged, but overlapping ranges will cause an exception when {@link #build()} is called. + * + * @throws IllegalArgumentException if any inserted ranges are empty + * @since 21.0 + */ + public Builder addAll(Iterable> ranges) { + for (Range range : ranges) { + add(range); + } + return this; + } + + Builder combine(Builder builder) { + addAll(builder.ranges); + return this; + } + + /** + * Returns an {@code ImmutableRangeSet} containing the ranges added to this builder. + * + * @throws IllegalArgumentException if any input ranges have nonempty overlap + */ + public ImmutableRangeSet build() { + ImmutableList.Builder> mergedRangesBuilder = + new ImmutableList.Builder<>(ranges.size()); + Collections.sort(ranges, Range.rangeLexOrdering()); + PeekingIterator> peekingItr = Iterators.peekingIterator(ranges.iterator()); + while (peekingItr.hasNext()) { + Range range = peekingItr.next(); + while (peekingItr.hasNext()) { + Range nextRange = peekingItr.peek(); + if (range.isConnected(nextRange)) { + checkArgument( + range.intersection(nextRange).isEmpty(), + "Overlapping ranges not permitted but found %s overlapping %s", + range, + nextRange); + range = range.span(peekingItr.next()); + } else { + break; + } + } + mergedRangesBuilder.add(range); + } + ImmutableList> mergedRanges = mergedRangesBuilder.build(); + if (mergedRanges.isEmpty()) { + return of(); + } else if (mergedRanges.size() == 1 + && Iterables.getOnlyElement(mergedRanges).equals(Range.all())) { + return all(); + } else { + return new ImmutableRangeSet(mergedRanges); + } + } + } + + @SuppressWarnings("rawtypes") + private static final class SerializedForm implements Serializable { + private final ImmutableList> ranges; + + SerializedForm(ImmutableList> ranges) { + this.ranges = ranges; + } + + Object readResolve() { + if (ranges.isEmpty()) { + return of(); + } else if (ranges.equals(ImmutableList.of(Range.all()))) { + return all(); + } else { + return new ImmutableRangeSet(ranges); + } + } + } + + Object writeReplace() { + return new SerializedForm(ranges); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSet.java new file mode 100644 index 0000000000000..d0ae4855d4ce3 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSet.java @@ -0,0 +1,833 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.Serializable; +import java.math.RoundingMode; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.Iterator; +import java.util.Set; +import java.util.SortedSet; +import java.util.Spliterator; +import java.util.function.Consumer; +import java.util.stream.Collector; + + +/** + * A {@link Set} whose contents will never change, with many other important properties detailed at + * {@link ImmutableCollection}. + * + * @since 2.0 + */ +@SuppressWarnings("serial") // we're overriding default serialization +public abstract class ImmutableSet extends ImmutableCollection implements Set { + static final int SPLITERATOR_CHARACTERISTICS = + ImmutableCollection.SPLITERATOR_CHARACTERISTICS | Spliterator.DISTINCT; + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableSet}. Elements appear in the resulting set in the encounter order of the stream; if + * the stream contains duplicates (according to {@link Object#equals(Object)}), only the first + * duplicate in encounter order will appear in the result. + * + * @since 21.0 + */ + + public static Collector> toImmutableSet() { + return CollectCollectors.toImmutableSet(); + } + + /** + * Returns the empty immutable set. Preferred over {@link Collections#emptySet} for code + * consistency, and because the return type conveys the immutability guarantee. + */ + @SuppressWarnings({"unchecked"}) // fully variant implementation (never actually produces any Es) + public static ImmutableSet of() { + return (ImmutableSet) RegularImmutableSet.EMPTY; + } + + /** + * Returns an immutable set containing {@code element}. Preferred over {@link + * Collections#singleton} for code consistency, {@code null} rejection, and because the return + * type conveys the immutability guarantee. + */ + public static ImmutableSet of(E element) { + return new SingletonImmutableSet(element); + } + + /** + * Returns an immutable set containing the given elements, minus duplicates, in the order each was + * first specified. That is, if multiple elements are {@linkplain Object#equals equal}, all except + * the first are ignored. + */ + public static ImmutableSet of(E e1, E e2) { + return construct(2, e1, e2); + } + + /** + * Returns an immutable set containing the given elements, minus duplicates, in the order each was + * first specified. That is, if multiple elements are {@linkplain Object#equals equal}, all except + * the first are ignored. + */ + public static ImmutableSet of(E e1, E e2, E e3) { + return construct(3, e1, e2, e3); + } + + /** + * Returns an immutable set containing the given elements, minus duplicates, in the order each was + * first specified. That is, if multiple elements are {@linkplain Object#equals equal}, all except + * the first are ignored. + */ + public static ImmutableSet of(E e1, E e2, E e3, E e4) { + return construct(4, e1, e2, e3, e4); + } + + /** + * Returns an immutable set containing the given elements, minus duplicates, in the order each was + * first specified. That is, if multiple elements are {@linkplain Object#equals equal}, all except + * the first are ignored. + */ + public static ImmutableSet of(E e1, E e2, E e3, E e4, E e5) { + return construct(5, e1, e2, e3, e4, e5); + } + + /** + * Returns an immutable set containing the given elements, minus duplicates, in the order each was + * first specified. That is, if multiple elements are {@linkplain Object#equals equal}, all except + * the first are ignored. + * + *

The array {@code others} must not be longer than {@code Integer.MAX_VALUE - 6}. + * + * @since 3.0 (source-compatible since 2.0) + */ + @SafeVarargs // For Eclipse. For internal javac we have disabled this pointless type of warning. + @SuppressWarnings("varargs") + public static ImmutableSet of(E e1, E e2, E e3, E e4, E e5, E e6, E... others) { + checkArgument( + others.length <= Integer.MAX_VALUE - 6, + "the total number of elements must fit in an int"); + final int paramCount = 6; + Object[] elements = new Object[paramCount + others.length]; + elements[0] = e1; + elements[1] = e2; + elements[2] = e3; + elements[3] = e4; + elements[4] = e5; + elements[5] = e6; + System.arraycopy(others, 0, elements, paramCount, others.length); + return construct(elements.length, elements); + } + + /** + * Constructs an {@code ImmutableSet} from the first {@code n} elements of the specified array. If + * {@code k} is the size of the returned {@code ImmutableSet}, then the unique elements of {@code + * elements} will be in the first {@code k} positions, and {@code elements[i] == null} for {@code + * k <= i < n}. + * + *

This may modify {@code elements}. Additionally, if {@code n == elements.length} and {@code + * elements} contains no duplicates, {@code elements} may be used without copying in the returned + * {@code ImmutableSet}, in which case it may no longer be modified. + * + *

{@code elements} may contain only values of type {@code E}. + * + * @throws NullPointerException if any of the first {@code n} elements of {@code elements} is null + */ + private static ImmutableSet construct(int n, Object... elements) { + switch (n) { + case 0: + return of(); + case 1: + @SuppressWarnings({"unchecked", "rawtypes"}) // safe; elements contains only E's + E elem = (E) elements[0]; + return of(elem); + default: + SetBuilderImpl builder = + new RegularSetBuilderImpl(ImmutableCollection.Builder.DEFAULT_INITIAL_CAPACITY); + for (int i = 0; i < n; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) + E e = (E) checkNotNull(elements[i]); + builder = builder.add(e); + } + return builder.review().build(); + } + } + + /** + * Returns an immutable set containing each of {@code elements}, minus duplicates, in the order + * each appears first in the source collection. + * + *

Performance note: This method will sometimes recognize that the actual copy operation + * is unnecessary; for example, {@code copyOf(copyOf(anArrayList))} will copy the data only once. + * This reduces the expense of habitually making defensive copies at API boundaries. However, the + * precise conditions for skipping the copy operation are undefined. + * + * @throws NullPointerException if any of {@code elements} is null + * @since 7.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableSet copyOf(Collection elements) { + /* + * TODO(lowasser): consider checking for ImmutableAsList here + * TODO(lowasser): consider checking for Multiset here + */ + // Don't refer to ImmutableSortedSet by name so it won't pull in all that code + if (elements instanceof ImmutableSet && !(elements instanceof SortedSet)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // all supported methods are covariant + ImmutableSet set = (ImmutableSet) elements; + if (!set.isPartialView()) { + return set; + } + } else if (elements instanceof EnumSet) { + return copyOfEnumSet((EnumSet) elements); + } + Object[] array = elements.toArray(); + return construct(array.length, array); + } + + /** + * Returns an immutable set containing each of {@code elements}, minus duplicates, in the order + * each appears first in the source iterable. This method iterates over {@code elements} only + * once. + * + *

Performance note: This method will sometimes recognize that the actual copy operation + * is unnecessary; for example, {@code copyOf(copyOf(anArrayList))} should copy the data only + * once. This reduces the expense of habitually making defensive copies at API boundaries. + * However, the precise conditions for skipping the copy operation are undefined. + * + * @throws NullPointerException if any of {@code elements} is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableSet copyOf(Iterable elements) { + return (elements instanceof Collection) + ? copyOf((Collection) elements) + : copyOf(elements.iterator()); + } + + /** + * Returns an immutable set containing each of {@code elements}, minus duplicates, in the order + * each appears first in the source iterator. + * + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableSet copyOf(Iterator elements) { + // We special-case for 0 or 1 elements, but anything further is madness. + if (!elements.hasNext()) { + return of(); + } + E first = elements.next(); + if (!elements.hasNext()) { + return of(first); + } else { + return new Builder().add(first).addAll(elements).build(); + } + } + + /** + * Returns an immutable set containing each of {@code elements}, minus duplicates, in the order + * each appears first in the source array. + * + * @throws NullPointerException if any of {@code elements} is null + * @since 3.0 + */ + public static ImmutableSet copyOf(E[] elements) { + switch (elements.length) { + case 0: + return of(); + case 1: + return of(elements[0]); + default: + return construct(elements.length, elements.clone()); + } + } + + @SuppressWarnings({"rawtypes", "unchecked"}) // necessary to compile against Java 8 + private static ImmutableSet copyOfEnumSet(EnumSet enumSet) { + return ImmutableEnumSet.asImmutable(EnumSet.copyOf(enumSet)); + } + + ImmutableSet() {} + + /** Returns {@code true} if the {@code hashCode()} method runs quickly. */ + boolean isHashCodeFast() { + return false; + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } else if (object instanceof ImmutableSet + && isHashCodeFast() + && ((ImmutableSet) object).isHashCodeFast() + && hashCode() != object.hashCode()) { + return false; + } + return Sets.equalsImpl(this, object); + } + + @Override + public int hashCode() { + return Sets.hashCodeImpl(this); + } + + // This declaration is needed to make Set.iterator() and + // ImmutableCollection.iterator() consistent. + @Override + public abstract UnmodifiableIterator iterator(); + + private transient ImmutableList asList; + + @Override + public ImmutableList asList() { + ImmutableList result = asList; + return (result == null) ? asList = createAsList() : result; + } + + ImmutableList createAsList() { + return new RegularImmutableAsList(this, toArray()); + } + + abstract static class Indexed extends ImmutableSet { + abstract E get(int index); + + @Override + public UnmodifiableIterator iterator() { + return asList().iterator(); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.indexed(size(), SPLITERATOR_CHARACTERISTICS, this::get); + } + + @Override + public void forEach(Consumer consumer) { + checkNotNull(consumer); + int n = size(); + for (int i = 0; i < n; i++) { + consumer.accept(get(i)); + } + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + return asList().copyIntoArray(dst, offset); + } + + @Override + ImmutableList createAsList() { + return new ImmutableAsList() { + @Override + public E get(int index) { + return Indexed.this.get(index); + } + + @Override + Indexed delegateCollection() { + return Indexed.this; + } + }; + } + } + + /* + * This class is used to serialize all ImmutableSet instances, except for + * ImmutableEnumSet/ImmutableSortedSet, regardless of implementation type. It + * captures their "logical contents" and they are reconstructed using public + * static factories. This is necessary to ensure that the existence of a + * particular implementation type is an implementation detail. + */ + private static class SerializedForm implements Serializable { + final Object[] elements; + + SerializedForm(Object[] elements) { + this.elements = elements; + } + + Object readResolve() { + return copyOf(elements); + } + + private static final long serialVersionUID = 0; + } + + @Override + Object writeReplace() { + return new SerializedForm(toArray()); + } + + /** + * Returns a new builder. The generated builder is equivalent to the builder created by the {@link + * Builder} constructor. + */ + public static Builder builder() { + return new Builder(); + } + + /** + * Returns a new builder, expecting the specified number of distinct elements to be added. + * + *

If {@code expectedSize} is exactly the number of distinct elements added to the builder + * before {@link Builder#build} is called, the builder is likely to perform better than an unsized + * {@link #builder()} would have. + * + *

It is not specified if any performance benefits apply if {@code expectedSize} is close to, + * but not exactly, the number of distinct elements added to the builder. + * + * @since 23.1 + */ + + public static Builder builderWithExpectedSize(int expectedSize) { + checkNonnegative(expectedSize, "expectedSize"); + return new Builder(expectedSize); + } + + /** Builds a new open-addressed hash table from the first n objects in elements. */ + static Object[] rebuildHashTable(int newTableSize, Object[] elements, int n) { + Object[] hashTable = new Object[newTableSize]; + int mask = hashTable.length - 1; + for (int i = 0; i < n; i++) { + Object e = elements[i]; + int j0 = Hashing.smear(e.hashCode()); + for (int j = j0; ; j++) { + int index = j & mask; + if (hashTable[index] == null) { + hashTable[index] = e; + break; + } + } + } + return hashTable; + } + + /** + * A builder for creating {@code ImmutableSet} instances. Example: + * + *

{@code
+   * static final ImmutableSet GOOGLE_COLORS =
+   *     ImmutableSet.builder()
+   *         .addAll(WEBSAFE_COLORS)
+   *         .add(new Color(0, 191, 255))
+   *         .build();
+   * }
+ * + *

Elements appear in the resulting set in the same order they were first added to the builder. + * + *

Building does not change the state of the builder, so it is still possible to add more + * elements and to build again. + * + * @since 2.0 + */ + @SuppressWarnings("unchecked") + public static class Builder extends ImmutableCollection.Builder { + private SetBuilderImpl impl; + boolean forceCopy; + + public Builder() { + this(DEFAULT_INITIAL_CAPACITY); + } + + Builder(int capacity) { + impl = new RegularSetBuilderImpl(capacity); + } + + Builder(@SuppressWarnings("unused") boolean subclass) { + this.impl = null; // unused + } + + + void forceJdk() { + this.impl = new JdkBackedSetBuilderImpl(impl); + } + + final void copyIfNecessary() { + if (forceCopy) { + copy(); + forceCopy = false; + } + } + + void copy() { + impl = impl.copy(); + } + + @Override + public Builder add(E element) { + checkNotNull(element); + copyIfNecessary(); + impl = impl.add(element); + return this; + } + + @SuppressWarnings("unchecked") + @Override + public Builder add(E... elements) { + super.add(elements); + return this; + } + + @Override + /** + * Adds each element of {@code elements} to the {@code ImmutableSet}, ignoring duplicate + * elements (only the first duplicate element is added). + * + * @param elements the elements to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + public Builder addAll(Iterable elements) { + super.addAll(elements); + return this; + } + + @Override + public Builder addAll(Iterator elements) { + super.addAll(elements); + return this; + } + + Builder combine(Builder other) { + copyIfNecessary(); + this.impl = this.impl.combine(other.impl); + return this; + } + + @Override + public ImmutableSet build() { + forceCopy = true; + impl = impl.review(); + return impl.build(); + } + } + + /** Swappable internal implementation of an ImmutableSet.Builder. */ + private abstract static class SetBuilderImpl { + E[] dedupedElements; + int distinct; + + @SuppressWarnings({"unchecked", "rawtypes"}) + SetBuilderImpl(int expectedCapacity) { + this.dedupedElements = (E[]) new Object[expectedCapacity]; + this.distinct = 0; + } + + /** Initializes this SetBuilderImpl with a copy of the deduped elements array from toCopy. */ + SetBuilderImpl(SetBuilderImpl toCopy) { + this.dedupedElements = Arrays.copyOf(toCopy.dedupedElements, toCopy.dedupedElements.length); + this.distinct = toCopy.distinct; + } + + /** + * Resizes internal data structures if necessary to store the specified number of distinct + * elements. + */ + private void ensureCapacity(int minCapacity) { + if (minCapacity > dedupedElements.length) { + int newCapacity = + ImmutableCollection.Builder.expandedCapacity(dedupedElements.length, minCapacity); + dedupedElements = Arrays.copyOf(dedupedElements, newCapacity); + } + } + + /** Adds e to the insertion-order array of deduplicated elements. Calls ensureCapacity. */ + final void addDedupedElement(E e) { + ensureCapacity(distinct + 1); + dedupedElements[distinct++] = e; + } + + /** + * Adds e to this SetBuilderImpl, returning the updated result. Only use the returned + * SetBuilderImpl, since we may switch implementations if e.g. hash flooding is detected. + */ + abstract SetBuilderImpl add(E e); + + /** Adds all the elements from the specified SetBuilderImpl to this SetBuilderImpl. */ + final SetBuilderImpl combine(SetBuilderImpl other) { + SetBuilderImpl result = this; + for (int i = 0; i < other.distinct; i++) { + result = result.add(other.dedupedElements[i]); + } + return result; + } + + /** + * Creates a new copy of this SetBuilderImpl. Modifications to that SetBuilderImpl will not + * affect this SetBuilderImpl or sets constructed from this SetBuilderImpl via build(). + */ + abstract SetBuilderImpl copy(); + + /** + * Call this before build(). Does a final check on the internal data structures, e.g. shrinking + * unnecessarily large structures or detecting previously unnoticed hash flooding. + */ + SetBuilderImpl review() { + return this; + } + + abstract ImmutableSet build(); + } + + // We use power-of-2 tables, and this is the highest int that's a power of 2 + static final int MAX_TABLE_SIZE = Ints.MAX_POWER_OF_TWO; + + // Represents how tightly we can pack things, as a maximum. + private static final double DESIRED_LOAD_FACTOR = 0.7; + + // If the set has this many elements, it will "max out" the table size + private static final int CUTOFF = (int) (MAX_TABLE_SIZE * DESIRED_LOAD_FACTOR); + + /** + * Returns an array size suitable for the backing array of a hash table that uses open addressing + * with linear probing in its implementation. The returned size is the smallest power of two that + * can hold setSize elements with the desired load factor. Always returns at least setSize + 2. + */ + + static int chooseTableSize(int setSize) { + setSize = Math.max(setSize, 2); + // Correct the size for open addressing to match desired load factor. + if (setSize < CUTOFF) { + // Round up to the next highest power of 2. + int tableSize = Integer.highestOneBit(setSize - 1) << 1; + while (tableSize * DESIRED_LOAD_FACTOR < setSize) { + tableSize <<= 1; + } + return tableSize; + } + + // The table can't be completely full or we'll get infinite reprobes + checkArgument(setSize < MAX_TABLE_SIZE, "collection too large"); + return MAX_TABLE_SIZE; + } + + /** + * We attempt to detect deliberate hash flooding attempts, and if one is detected, fall back to a + * wrapper around j.u.HashSet, which has built in flooding protection. HASH_FLOODING_FPP is the + * maximum allowed probability of falsely detecting a hash flooding attack if the input is + * randomly generated. + * + *

MAX_RUN_MULTIPLIER was determined experimentally to match this FPP. + */ + static final double HASH_FLOODING_FPP = 0.001; + + // NB: yes, this is surprisingly high, but that's what the experiments said was necessary + static final int MAX_RUN_MULTIPLIER = 12; + + /** + * Checks the whole hash table for poor hash distribution. Takes O(n). + * + *

The online hash flooding detecting in RegularSetBuilderImpl.add can detect e.g. many exactly + * matching hash codes, which would cause construction to take O(n^2), but can't detect e.g. hash + * codes adversarially designed to go into ascending table locations, which keeps construction + * O(n) (as desired) but then can have O(n) queries later. + * + *

If this returns false, then no query can take more than O(log n). + * + *

Note that for a RegularImmutableSet with elements with truly random hash codes, contains + * operations take expected O(1) time but with high probability take O(log n) for at least some + * element. (https://en.wikipedia.org/wiki/Linear_probing#Analysis) + */ + static boolean hashFloodingDetected(Object[] hashTable) { + int maxRunBeforeFallback = maxRunBeforeFallback(hashTable.length); + + // Test for a run wrapping around the end of the table, then check for runs in the middle. + int endOfStartRun; + for (endOfStartRun = 0; endOfStartRun < hashTable.length; ) { + if (hashTable[endOfStartRun] == null) { + break; + } + endOfStartRun++; + if (endOfStartRun > maxRunBeforeFallback) { + return true; + } + } + int startOfEndRun; + for (startOfEndRun = hashTable.length - 1; startOfEndRun > endOfStartRun; startOfEndRun--) { + if (hashTable[startOfEndRun] == null) { + break; + } + if (endOfStartRun + (hashTable.length - 1 - startOfEndRun) > maxRunBeforeFallback) { + return true; + } + } + for (int i = endOfStartRun + 1; i < startOfEndRun; i++) { + for (int runLength = 0; i < startOfEndRun && hashTable[i] != null; i++) { + runLength++; + if (runLength > maxRunBeforeFallback) { + return true; + } + } + } + return false; + } + + /** + * If more than this many consecutive positions are filled in a table of the specified size, + * report probable hash flooding. + */ + static int maxRunBeforeFallback(int tableSize) { + return MAX_RUN_MULTIPLIER * IntMath.log2(tableSize, RoundingMode.UNNECESSARY); + } + + /** + * Default implementation of the guts of ImmutableSet.Builder, creating an open-addressed hash + * table and deduplicating elements as they come, so it only allocates O(max(distinct, + * expectedCapacity)) rather than O(calls to add). + * + *

This implementation attempts to detect hash flooding, and if it's identified, falls back to + * JdkBackedSetBuilderImpl. + */ + private static final class RegularSetBuilderImpl extends SetBuilderImpl { + private Object[] hashTable; + private int maxRunBeforeFallback; + private int expandTableThreshold; + private int hashCode; + + RegularSetBuilderImpl(int expectedCapacity) { + super(expectedCapacity); + int tableSize = chooseTableSize(expectedCapacity); + this.hashTable = new Object[tableSize]; + this.maxRunBeforeFallback = maxRunBeforeFallback(tableSize); + this.expandTableThreshold = (int) (DESIRED_LOAD_FACTOR * tableSize); + } + + RegularSetBuilderImpl(RegularSetBuilderImpl toCopy) { + super(toCopy); + this.hashTable = Arrays.copyOf(toCopy.hashTable, toCopy.hashTable.length); + this.maxRunBeforeFallback = toCopy.maxRunBeforeFallback; + this.expandTableThreshold = toCopy.expandTableThreshold; + this.hashCode = toCopy.hashCode; + } + + void ensureTableCapacity(int minCapacity) { + if (minCapacity > expandTableThreshold && hashTable.length < MAX_TABLE_SIZE) { + int newTableSize = hashTable.length * 2; + hashTable = rebuildHashTable(newTableSize, dedupedElements, distinct); + maxRunBeforeFallback = maxRunBeforeFallback(newTableSize); + expandTableThreshold = (int) (DESIRED_LOAD_FACTOR * newTableSize); + } + } + + @Override + SetBuilderImpl add(E e) { + checkNotNull(e); + int eHash = e.hashCode(); + int i0 = Hashing.smear(eHash); + int mask = hashTable.length - 1; + for (int i = i0; i - i0 < maxRunBeforeFallback; i++) { + int index = i & mask; + Object tableEntry = hashTable[index]; + if (tableEntry == null) { + addDedupedElement(e); + hashTable[index] = e; + hashCode += eHash; + ensureTableCapacity(distinct); // rebuilds table if necessary + return this; + } else if (tableEntry.equals(e)) { // not a new element, ignore + return this; + } + } + // we fell out of the loop due to a long run; fall back to JDK impl + return new JdkBackedSetBuilderImpl(this).add(e); + } + + @Override + SetBuilderImpl copy() { + return new RegularSetBuilderImpl(this); + } + + @Override + SetBuilderImpl review() { + int targetTableSize = chooseTableSize(distinct); + if (targetTableSize * 2 < hashTable.length) { + hashTable = rebuildHashTable(targetTableSize, dedupedElements, distinct); + } + return hashFloodingDetected(hashTable) ? new JdkBackedSetBuilderImpl(this) : this; + } + + @Override + ImmutableSet build() { + switch (distinct) { + case 0: + return of(); + case 1: + return of(dedupedElements[0]); + default: + Object[] elements = + (distinct == dedupedElements.length) + ? dedupedElements + : Arrays.copyOf(dedupedElements, distinct); + return new RegularImmutableSet(elements, hashCode, hashTable, hashTable.length - 1); + } + } + } + + /** + * SetBuilderImpl version that uses a JDK HashSet, which has built in hash flooding protection. + */ + private static final class JdkBackedSetBuilderImpl extends SetBuilderImpl { + private final Set delegate; + + JdkBackedSetBuilderImpl(SetBuilderImpl toCopy) { + super(toCopy); // initializes dedupedElements and distinct + delegate = Sets.newHashSetWithExpectedSize(distinct); + for (int i = 0; i < distinct; i++) { + delegate.add(dedupedElements[i]); + } + } + + @Override + SetBuilderImpl add(E e) { + checkNotNull(e); + if (delegate.add(e)) { + addDedupedElement(e); + } + return this; + } + + @Override + SetBuilderImpl copy() { + return new JdkBackedSetBuilderImpl<>(this); + } + + @Override + ImmutableSet build() { + switch (distinct) { + case 0: + return of(); + case 1: + return of(dedupedElements[0]); + default: + return new JdkBackedImmutableSet( + delegate, ImmutableList.asImmutableList(dedupedElements, distinct)); + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSetMultimap.java new file mode 100644 index 0000000000000..e13336288459d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSetMultimap.java @@ -0,0 +1,637 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +import java.io.IOException; +import java.io.InvalidObjectException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.Comparator; +import java.util.Map.Entry; +import java.util.function.Function; +import java.util.stream.Collector; +import java.util.stream.Collectors; +import java.util.stream.Stream; + + + +/** + * A {@link SetMultimap} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Mike Ward + * @since 2.0 + */ +public class ImmutableSetMultimap extends ImmutableMultimap + implements SetMultimap { + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableSetMultimap} + * whose keys and values are the result of applying the provided mapping functions to the input + * elements. + * + *

For streams with {@linkplain java.util.stream#Ordering defined encounter order}, that order + * is preserved, but entries are grouped by key. + * + *

Example: + * + *

{@code
+   * static final Multimap FIRST_LETTER_MULTIMAP =
+   *     Stream.of("banana", "apple", "carrot", "asparagus", "cherry")
+   *         .collect(toImmutableSetMultimap(str -> str.charAt(0), str -> str.substring(1)));
+   *
+   * // is equivalent to
+   *
+   * static final Multimap FIRST_LETTER_MULTIMAP =
+   *     new ImmutableSetMultimap.Builder()
+   *         .put('b', "anana")
+   *         .putAll('a', "pple", "sparagus")
+   *         .putAll('c', "arrot", "herry")
+   *         .build();
+   * }
+ * + * @since 21.0 + */ + + public static Collector> toImmutableSetMultimap( + Function keyFunction, + Function valueFunction) { + checkNotNull(keyFunction, "keyFunction"); + checkNotNull(valueFunction, "valueFunction"); + return Collector.of( + ImmutableSetMultimap::builder, + (builder, t) -> builder.put(keyFunction.apply(t), valueFunction.apply(t)), + Builder::combine, + Builder::build); + } + + /** + * Returns a {@code Collector} accumulating entries into an {@code ImmutableSetMultimap}. Each + * input element is mapped to a key and a stream of values, each of which are put into the + * resulting {@code Multimap}, in the encounter order of the stream and the encounter order of the + * streams of values. + * + *

Example: + * + *

{@code
+   * static final ImmutableSetMultimap FIRST_LETTER_MULTIMAP =
+   *     Stream.of("banana", "apple", "carrot", "asparagus", "cherry")
+   *         .collect(
+   *             flatteningToImmutableSetMultimap(
+   *                  str -> str.charAt(0),
+   *                  str -> str.substring(1).chars().mapToObj(c -> (char) c));
+   *
+   * // is equivalent to
+   *
+   * static final ImmutableSetMultimap FIRST_LETTER_MULTIMAP =
+   *     ImmutableSetMultimap.builder()
+   *         .putAll('b', Arrays.asList('a', 'n', 'a', 'n', 'a'))
+   *         .putAll('a', Arrays.asList('p', 'p', 'l', 'e'))
+   *         .putAll('c', Arrays.asList('a', 'r', 'r', 'o', 't'))
+   *         .putAll('a', Arrays.asList('s', 'p', 'a', 'r', 'a', 'g', 'u', 's'))
+   *         .putAll('c', Arrays.asList('h', 'e', 'r', 'r', 'y'))
+   *         .build();
+   *
+   * // after deduplication, the resulting multimap is equivalent to
+   *
+   * static final ImmutableSetMultimap FIRST_LETTER_MULTIMAP =
+   *     ImmutableSetMultimap.builder()
+   *         .putAll('b', Arrays.asList('a', 'n'))
+   *         .putAll('a', Arrays.asList('p', 'l', 'e', 's', 'a', 'r', 'g', 'u'))
+   *         .putAll('c', Arrays.asList('a', 'r', 'o', 't', 'h', 'e', 'y'))
+   *         .build();
+   * }
+   * }
+ * + * @since 21.0 + */ + + public static + Collector> flatteningToImmutableSetMultimap( + Function keyFunction, + Function> valuesFunction) { + checkNotNull(keyFunction); + checkNotNull(valuesFunction); + return Collectors.collectingAndThen( + Multimaps.flatteningToMultimap( + input -> checkNotNull(keyFunction.apply(input)), + input -> valuesFunction.apply(input).peek(Preconditions::checkNotNull), + MultimapBuilder.linkedHashKeys().linkedHashSetValues()::build), + ImmutableSetMultimap::copyOf); + } + + /** Returns the empty multimap. */ + // Casting is safe because the multimap will never hold any elements. + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableSetMultimap of() { + return (ImmutableSetMultimap) EmptyImmutableSetMultimap.INSTANCE; + } + + /** Returns an immutable multimap containing a single entry. */ + public static ImmutableSetMultimap of(K k1, V v1) { + Builder builder = ImmutableSetMultimap.builder(); + builder.put(k1, v1); + return builder.build(); + } + + /** + * Returns an immutable multimap containing the given entries, in order. Repeated occurrences of + * an entry (according to {@link Object#equals}) after the first are ignored. + */ + public static ImmutableSetMultimap of(K k1, V v1, K k2, V v2) { + Builder builder = ImmutableSetMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + return builder.build(); + } + + /** + * Returns an immutable multimap containing the given entries, in order. Repeated occurrences of + * an entry (according to {@link Object#equals}) after the first are ignored. + */ + public static ImmutableSetMultimap of(K k1, V v1, K k2, V v2, K k3, V v3) { + Builder builder = ImmutableSetMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + builder.put(k3, v3); + return builder.build(); + } + + /** + * Returns an immutable multimap containing the given entries, in order. Repeated occurrences of + * an entry (according to {@link Object#equals}) after the first are ignored. + */ + public static ImmutableSetMultimap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + Builder builder = ImmutableSetMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + builder.put(k3, v3); + builder.put(k4, v4); + return builder.build(); + } + + /** + * Returns an immutable multimap containing the given entries, in order. Repeated occurrences of + * an entry (according to {@link Object#equals}) after the first are ignored. + */ + public static ImmutableSetMultimap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + Builder builder = ImmutableSetMultimap.builder(); + builder.put(k1, v1); + builder.put(k2, v2); + builder.put(k3, v3); + builder.put(k4, v4); + builder.put(k5, v5); + return builder.build(); + } + + // looking for of() with > 5 entries? Use the builder instead. + + /** Returns a new {@link Builder}. */ + public static Builder builder() { + return new Builder<>(); + } + + /** + * A builder for creating immutable {@code SetMultimap} instances, especially {@code public static + * final} multimaps ("constant multimaps"). Example: + * + *
{@code
+   * static final Multimap STRING_TO_INTEGER_MULTIMAP =
+   *     new ImmutableSetMultimap.Builder()
+   *         .put("one", 1)
+   *         .putAll("several", 1, 2, 3)
+   *         .putAll("many", 1, 2, 3, 4, 5)
+   *         .build();
+   * }
+ * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple multimaps in series. Each multimap contains the key-value mappings in the previously + * created multimaps. + * + * @since 2.0 + */ + public static final class Builder extends ImmutableMultimap.Builder { + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableSetMultimap#builder}. + */ + public Builder() { + super(); + } + + @Override + Collection newMutableValueCollection() { + return Platform.preservesInsertionOrderOnAddsSet(); + } + + /** Adds a key-value mapping to the built multimap if it is not already present. */ + @Override + public Builder put(K key, V value) { + super.put(key, value); + return this; + } + + /** + * Adds an entry to the built multimap if it is not already present. + * + * @since 11.0 + */ + @Override + public Builder put(Entry entry) { + super.put(entry); + return this; + } + + /** + * {@inheritDoc} + * + * @since 19.0 + */ + + @Override + public Builder putAll(Iterable> entries) { + super.putAll(entries); + return this; + } + + @Override + public Builder putAll(K key, Iterable values) { + super.putAll(key, values); + return this; + } + + @Override + @SafeVarargs + @SuppressWarnings("varargs") + public final Builder putAll(K key, V... values) { + return putAll(key, Arrays.asList(values)); + } + + @Override + public Builder putAll(Multimap multimap) { + for (Entry> entry : + multimap.asMap().entrySet()) { + putAll(entry.getKey(), entry.getValue()); + } + return this; + } + + @Override + Builder combine(ImmutableMultimap.Builder other) { + super.combine(other); + return this; + } + + /** + * {@inheritDoc} + * + * @since 8.0 + */ + @Override + public Builder orderKeysBy(Comparator keyComparator) { + super.orderKeysBy(keyComparator); + return this; + } + + /** + * Specifies the ordering of the generated multimap's values for each key. + * + *

If this method is called, the sets returned by the {@code get()} method of the generated + * multimap and its {@link Multimap#asMap()} view are {@link ImmutableSortedSet} instances. + * However, serialization does not preserve that property, though it does maintain the key and + * value ordering. + * + * @since 8.0 + */ + // TODO: Make serialization behavior consistent. + @Override + public Builder orderValuesBy(Comparator valueComparator) { + super.orderValuesBy(valueComparator); + return this; + } + + /** Returns a newly-created immutable set multimap. */ + @Override + public ImmutableSetMultimap build() { + Collection>> mapEntries = builderMap.entrySet(); + if (keyComparator != null) { + mapEntries = Ordering.from(keyComparator).onKeys().immutableSortedCopy(mapEntries); + } + return fromMapEntries(mapEntries, valueComparator); + } + } + + /** + * Returns an immutable set multimap containing the same mappings as {@code multimap}. The + * generated multimap's key and value orderings correspond to the iteration ordering of the {@code + * multimap.asMap()} view. Repeated occurrences of an entry in the multimap after the first are + * ignored. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if any key or value in {@code multimap} is null + */ + public static ImmutableSetMultimap copyOf( + Multimap multimap) { + return copyOf(multimap, null); + } + + private static ImmutableSetMultimap copyOf( + Multimap multimap, Comparator valueComparator) { + checkNotNull(multimap); // eager for GWT + if (multimap.isEmpty() && valueComparator == null) { + return of(); + } + + if (multimap instanceof ImmutableSetMultimap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe since multimap is not writable + ImmutableSetMultimap kvMultimap = (ImmutableSetMultimap) multimap; + if (!kvMultimap.isPartialView()) { + return kvMultimap; + } + } + + return fromMapEntries(multimap.asMap().entrySet(), valueComparator); + } + + /** + * Returns an immutable multimap containing the specified entries. The returned multimap iterates + * over keys in the order they were first encountered in the input, and the values for each key + * are iterated in the order they were encountered. If two values for the same key are {@linkplain + * Object#equals equal}, the first value encountered is used. + * + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + public static ImmutableSetMultimap copyOf( + Iterable> entries) { + return new Builder().putAll(entries).build(); + } + + /** Creates an ImmutableSetMultimap from an asMap.entrySet. */ + static ImmutableSetMultimap fromMapEntries( + Collection>> mapEntries, + Comparator valueComparator) { + if (mapEntries.isEmpty()) { + return of(); + } + ImmutableMap.Builder> builder = + new ImmutableMap.Builder<>(mapEntries.size()); + int size = 0; + + for (Entry> entry : mapEntries) { + K key = entry.getKey(); + Collection values = entry.getValue(); + ImmutableSet set = valueSet(valueComparator, values); + if (!set.isEmpty()) { + builder.put(key, set); + size += set.size(); + } + } + + return new ImmutableSetMultimap<>(builder.build(), size, valueComparator); + } + + /** + * Returned by get() when a missing key is provided. Also holds the comparator, if any, used for + * values. + */ + private final transient ImmutableSet emptySet; + + ImmutableSetMultimap( + ImmutableMap> map, + int size, + Comparator valueComparator) { + super(map, size); + this.emptySet = emptySet(valueComparator); + } + + // views + + /** + * Returns an immutable set of the values for the given key. If no mappings in the multimap have + * the provided key, an empty immutable set is returned. The values are in the same order as the + * parameters used to build this multimap. + */ + @Override + public ImmutableSet get(K key) { + // This cast is safe as its type is known in constructor. + ImmutableSet set = (ImmutableSet) map.get(key); + return MoreObjects.firstNonNull(set, emptySet); + } + + private transient ImmutableSetMultimap inverse; + + /** + * {@inheritDoc} + * + *

Because an inverse of a set multimap cannot contain multiple pairs with the same key and + * value, this method returns an {@code ImmutableSetMultimap} rather than the {@code + * ImmutableMultimap} specified in the {@code ImmutableMultimap} class. + * + * @since 11.0 + */ + public ImmutableSetMultimap inverse() { + ImmutableSetMultimap result = inverse; + return (result == null) ? (inverse = invert()) : result; + } + + private ImmutableSetMultimap invert() { + Builder builder = builder(); + for (Entry entry : entries()) { + builder.put(entry.getValue(), entry.getKey()); + } + ImmutableSetMultimap invertedMultimap = builder.build(); + invertedMultimap.inverse = this; + return invertedMultimap; + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public ImmutableSet removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the multimap unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public ImmutableSet replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + private transient ImmutableSet> entries; + + /** + * Returns an immutable collection of all key-value pairs in the multimap. Its iterator traverses + * the values for the first key, the values for the second key, and so on. + */ + @Override + public ImmutableSet> entries() { + ImmutableSet> result = entries; + return result == null ? (entries = new EntrySet<>(this)) : result; + } + + private static final class EntrySet extends ImmutableSet> { + private final transient ImmutableSetMultimap multimap; + + EntrySet(ImmutableSetMultimap multimap) { + this.multimap = multimap; + } + + @Override + public boolean contains(Object object) { + if (object instanceof Entry) { + Entry entry = (Entry) object; + return multimap.containsEntry(entry.getKey(), entry.getValue()); + } + return false; + } + + @Override + public int size() { + return multimap.size(); + } + + @Override + public UnmodifiableIterator> iterator() { + return multimap.entryIterator(); + } + + @Override + boolean isPartialView() { + return false; + } + } + + private static ImmutableSet valueSet( + Comparator valueComparator, Collection values) { + return (valueComparator == null) + ? ImmutableSet.copyOf(values) + : ImmutableSortedSet.copyOf(valueComparator, values); + } + + private static ImmutableSet emptySet(Comparator valueComparator) { + return (valueComparator == null) + ? ImmutableSet.of() + : ImmutableSortedSet.emptySet(valueComparator); + } + + private static ImmutableSet.Builder valuesBuilder( + Comparator valueComparator) { + return (valueComparator == null) + ? new ImmutableSet.Builder() + : new ImmutableSortedSet.Builder(valueComparator); + } + + /** + * @serialData number of distinct keys, and then for each distinct key: the key, the number of + * values for that key, and the key's values + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(valueComparator()); + Serialization.writeMultimap(this, stream); + } + + + Comparator valueComparator() { + return emptySet instanceof ImmutableSortedSet + ? ((ImmutableSortedSet) emptySet).comparator() + : null; + } + + private static final class SetFieldSettersHolder { + @SuppressWarnings("rawtypes") + static final Serialization.FieldSetter EMPTY_SET_FIELD_SETTER = + Serialization.getFieldSetter(ImmutableSetMultimap.class, "emptySet"); + } + + // Serialization type safety is at the caller's mercy. + @SuppressWarnings({"unchecked", "rawtypes"}) + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + Comparator valueComparator = (Comparator) stream.readObject(); + int keyCount = stream.readInt(); + if (keyCount < 0) { + throw new InvalidObjectException("Invalid key count " + keyCount); + } + ImmutableMap.Builder> builder = ImmutableMap.builder(); + int tmpSize = 0; + + for (int i = 0; i < keyCount; i++) { + Object key = stream.readObject(); + int valueCount = stream.readInt(); + if (valueCount <= 0) { + throw new InvalidObjectException("Invalid value count " + valueCount); + } + + ImmutableSet.Builder valuesBuilder = valuesBuilder(valueComparator); + for (int j = 0; j < valueCount; j++) { + valuesBuilder.add(stream.readObject()); + } + ImmutableSet valueSet = valuesBuilder.build(); + if (valueSet.size() != valueCount) { + throw new InvalidObjectException("Duplicate key-value pairs exist for key " + key); + } + builder.put(key, valueSet); + tmpSize += valueCount; + } + + ImmutableMap> tmpMap; + try { + tmpMap = builder.build(); + } catch (IllegalArgumentException e) { + throw (InvalidObjectException) new InvalidObjectException(e.getMessage()).initCause(e); + } + + FieldSettersHolder.MAP_FIELD_SETTER.set(this, tmpMap); + FieldSettersHolder.SIZE_FIELD_SETTER.set(this, tmpSize); + SetFieldSettersHolder.EMPTY_SET_FIELD_SETTER.set(this, emptySet(valueComparator)); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedAsList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedAsList.java new file mode 100644 index 0000000000000..e99e674f7b82f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedAsList.java @@ -0,0 +1,93 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Comparator; +import java.util.Spliterator; + + +/** + * List returned by {@code ImmutableSortedSet.asList()} when the set isn't empty. + * + * @author Jared Levy + * @author Louis Wasserman + */ +@SuppressWarnings("serial") +final class ImmutableSortedAsList extends RegularImmutableAsList + implements SortedIterable { + ImmutableSortedAsList(ImmutableSortedSet backingSet, ImmutableList backingList) { + super(backingSet, backingList); + } + + @Override + ImmutableSortedSet delegateCollection() { + return (ImmutableSortedSet) super.delegateCollection(); + } + + @Override + public Comparator comparator() { + return delegateCollection().comparator(); + } + + // Override indexOf() and lastIndexOf() to be O(log N) instead of O(N). + + // TODO(cpovirk): consider manual binary search under GWT to preserve O(log N) lookup + @Override + public int indexOf(Object target) { + int index = delegateCollection().indexOf(target); + + // TODO(kevinb): reconsider if it's really worth making feeble attempts at + // sanity for inconsistent comparators. + + // The equals() check is needed when the comparator isn't compatible with + // equals(). + return (index >= 0 && get(index).equals(target)) ? index : -1; + } + + @Override + public int lastIndexOf(Object target) { + return indexOf(target); + } + + @Override + public boolean contains(Object target) { + // Necessary for ISS's with comparators inconsistent with equals. + return indexOf(target) >= 0; + } + + /* + * TODO(cpovirk): if we start to override indexOf/lastIndexOf under GWT, we'll want some way to + * override subList to return an ImmutableSortedAsList for better performance. Right now, I'm not + * sure there's any performance hit from our failure to override subListUnchecked under GWT + */ + @Override + ImmutableList subListUnchecked(int fromIndex, int toIndex) { + ImmutableList parentSubList = super.subListUnchecked(fromIndex, toIndex); + return new RegularImmutableSortedSet(parentSubList, comparator()).asList(); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.indexed( + size(), + ImmutableList.SPLITERATOR_CHARACTERISTICS | Spliterator.SORTED | Spliterator.DISTINCT, + delegateList()::get, + comparator()); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMap.java new file mode 100644 index 0000000000000..d7e57fe9b64d7 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMap.java @@ -0,0 +1,944 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.keyOrNull; + + +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.Spliterator; +import java.util.TreeMap; +import java.util.function.BiConsumer; +import java.util.function.BinaryOperator; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collector; +import java.util.stream.Collectors; + + +/** + * A {@link NavigableMap} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

Warning: as with any sorted collection, you are strongly advised not to use a {@link + * Comparator} or {@link Comparable} type whose comparison behavior is inconsistent with + * equals. That is, {@code a.compareTo(b)} or {@code comparator.compare(a, b)} should equal zero + * if and only if {@code a.equals(b)}. If this advice is not followed, the resulting map will + * not correctly obey its specification. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Jared Levy + * @author Louis Wasserman + * @since 2.0 (implements {@code NavigableMap} since 12.0) + */ +public final class ImmutableSortedMap extends ImmutableSortedMapFauxverideShim + implements NavigableMap { + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableSortedMap} whose + * keys and values are the result of applying the provided mapping functions to the input + * elements. The generated map is sorted by the specified comparator. + * + *

If the mapped keys contain duplicates (according to the specified comparator), an {@code + * IllegalArgumentException} is thrown when the collection operation is performed. (This differs + * from the {@code Collector} returned by {@link Collectors#toMap(Function, Function)}, which + * throws an {@code IllegalStateException}.) + * + * @since 21.0 + */ + + public static Collector> toImmutableSortedMap( + Comparator comparator, + Function keyFunction, + Function valueFunction) { + return CollectCollectors.toImmutableSortedMap(comparator, keyFunction, valueFunction); + } + + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableSortedMap} whose + * keys and values are the result of applying the provided mapping functions to the input + * elements. + * + *

If the mapped keys contain duplicates (according to the comparator), the the values are + * merged using the specified merging function. Entries will appear in the encounter order of the + * first occurrence of the key. + * + * @since 21.0 + */ + + public static Collector> toImmutableSortedMap( + Comparator comparator, + Function keyFunction, + Function valueFunction, + BinaryOperator mergeFunction) { + checkNotNull(comparator); + checkNotNull(keyFunction); + checkNotNull(valueFunction); + checkNotNull(mergeFunction); + return Collectors.collectingAndThen( + Collectors.toMap( + keyFunction, valueFunction, mergeFunction, () -> new TreeMap(comparator)), + ImmutableSortedMap::copyOfSorted); + } + + /* + * TODO(kevinb): Confirm that ImmutableSortedMap is faster to construct and + * uses less memory than TreeMap; then say so in the class Javadoc. + */ + @SuppressWarnings("rawtypes") + private static final Comparator NATURAL_ORDER = Ordering.natural(); + + @SuppressWarnings("rawtypes") + private static final ImmutableSortedMap NATURAL_EMPTY_MAP = + new ImmutableSortedMap<>( + ImmutableSortedSet.emptySet(Ordering.natural()), ImmutableList.of()); + + static ImmutableSortedMap emptyMap(Comparator comparator) { + if (Ordering.natural().equals(comparator)) { + return of(); + } else { + return new ImmutableSortedMap<>( + ImmutableSortedSet.emptySet(comparator), ImmutableList.of()); + } + } + + /** Returns the empty sorted map. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + // unsafe, comparator() returns a comparator on the specified type + // TODO(kevinb): evaluate whether or not of().comparator() should return null + public static ImmutableSortedMap of() { + return (ImmutableSortedMap) NATURAL_EMPTY_MAP; + } + + /** Returns an immutable map containing a single entry. */ + public static , V> ImmutableSortedMap of(K k1, V v1) { + return of(Ordering.natural(), k1, v1); + } + + /** Returns an immutable map containing a single entry. */ + private static ImmutableSortedMap of(Comparator comparator, K k1, V v1) { + return new ImmutableSortedMap<>( + new RegularImmutableSortedSet(ImmutableList.of(k1), checkNotNull(comparator)), + ImmutableList.of(v1)); + } + + /** + * Returns an immutable sorted map containing the given entries, sorted by the natural ordering of + * their keys. + * + * @throws IllegalArgumentException if the two keys are equal according to their natural ordering + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static , V> ImmutableSortedMap of( + K k1, V v1, K k2, V v2) { + return ofEntries(entryOf(k1, v1), entryOf(k2, v2)); + } + + /** + * Returns an immutable sorted map containing the given entries, sorted by the natural ordering of + * their keys. + * + * @throws IllegalArgumentException if any two keys are equal according to their natural ordering + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static , V> ImmutableSortedMap of( + K k1, V v1, K k2, V v2, K k3, V v3) { + return ofEntries(entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3)); + } + + /** + * Returns an immutable sorted map containing the given entries, sorted by the natural ordering of + * their keys. + * + * @throws IllegalArgumentException if any two keys are equal according to their natural ordering + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static , V> ImmutableSortedMap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + return ofEntries(entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3), entryOf(k4, v4)); + } + + /** + * Returns an immutable sorted map containing the given entries, sorted by the natural ordering of + * their keys. + * + * @throws IllegalArgumentException if any two keys are equal according to their natural ordering + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static , V> ImmutableSortedMap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + return ofEntries( + entryOf(k1, v1), entryOf(k2, v2), entryOf(k3, v3), entryOf(k4, v4), entryOf(k5, v5)); + } + + @SuppressWarnings("unchecked") + private static , V> ImmutableSortedMap ofEntries( + Entry... entries) { + return fromEntries(Ordering.natural(), false, entries, entries.length); + } + + /** + * Returns an immutable map containing the same entries as {@code map}, sorted by the natural + * ordering of the keys. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

This method is not type-safe, as it may be called on a map with keys that are not mutually + * comparable. + * + * @throws ClassCastException if the keys in {@code map} are not mutually comparable + * @throws NullPointerException if any key or value in {@code map} is null + * @throws IllegalArgumentException if any two keys are equal according to their natural ordering + */ + public static ImmutableSortedMap copyOf(Map map) { + // Hack around K not being a subtype of Comparable. + // Unsafe, see ImmutableSortedSetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) NATURAL_ORDER; + return copyOfInternal(map, naturalOrder); + } + + /** + * Returns an immutable map containing the same entries as {@code map}, with keys sorted by the + * provided comparator. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if any key or value in {@code map} is null + * @throws IllegalArgumentException if any two keys are equal according to the comparator + */ + public static ImmutableSortedMap copyOf( + Map map, Comparator comparator) { + return copyOfInternal(map, checkNotNull(comparator)); + } + + /** + * Returns an immutable map containing the given entries, with keys sorted by the provided + * comparator. + * + *

This method is not type-safe, as it may be called on a map with keys that are not mutually + * comparable. + * + * @throws NullPointerException if any key or value in {@code map} is null + * @throws IllegalArgumentException if any two keys are equal according to the comparator + * @since 19.0 + */ + + public static ImmutableSortedMap copyOf( + Iterable> entries) { + // Hack around K not being a subtype of Comparable. + // Unsafe, see ImmutableSortedSetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) NATURAL_ORDER; + return copyOf(entries, naturalOrder); + } + + /** + * Returns an immutable map containing the given entries, with keys sorted by the provided + * comparator. + * + * @throws NullPointerException if any key or value in {@code map} is null + * @throws IllegalArgumentException if any two keys are equal according to the comparator + * @since 19.0 + */ + + public static ImmutableSortedMap copyOf( + Iterable> entries, + Comparator comparator) { + return fromEntries(checkNotNull(comparator), false, entries); + } + + /** + * Returns an immutable map containing the same entries as the provided sorted map, with the same + * ordering. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if any key or value in {@code map} is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableSortedMap copyOfSorted(SortedMap map) { + Comparator comparator = map.comparator(); + if (comparator == null) { + // If map has a null comparator, the keys should have a natural ordering, + // even though K doesn't explicitly implement Comparable. + comparator = (Comparator) NATURAL_ORDER; + } + if (map instanceof ImmutableSortedMap) { + // TODO(kevinb): Prove that this cast is safe, even though + // Collections.unmodifiableSortedMap requires the same key type. + @SuppressWarnings({"unchecked", "rawtypes"}) + ImmutableSortedMap kvMap = (ImmutableSortedMap) map; + if (!kvMap.isPartialView()) { + return kvMap; + } + } + return fromEntries(comparator, true, map.entrySet()); + } + + private static ImmutableSortedMap copyOfInternal( + Map map, Comparator comparator) { + boolean sameComparator = false; + if (map instanceof SortedMap) { + SortedMap sortedMap = (SortedMap) map; + Comparator comparator2 = sortedMap.comparator(); + sameComparator = + (comparator2 == null) ? comparator == NATURAL_ORDER : comparator.equals(comparator2); + } + + if (sameComparator && (map instanceof ImmutableSortedMap)) { + // TODO(kevinb): Prove that this cast is safe, even though + // Collections.unmodifiableSortedMap requires the same key type. + @SuppressWarnings({"unchecked", "rawtypes"}) + ImmutableSortedMap kvMap = (ImmutableSortedMap) map; + if (!kvMap.isPartialView()) { + return kvMap; + } + } + return fromEntries(comparator, sameComparator, map.entrySet()); + } + + /** + * Accepts a collection of possibly-null entries. If {@code sameComparator}, then it is assumed + * that they do not need to be sorted or checked for dupes. + */ + private static ImmutableSortedMap fromEntries( + Comparator comparator, + boolean sameComparator, + Iterable> entries) { + // "adding" type params to an array of a raw type should be safe as + // long as no one can ever cast that same array instance back to a + // raw type. + @SuppressWarnings({"unchecked", "rawtypes"}) + Entry[] entryArray = (Entry[]) Iterables.toArray(entries, EMPTY_ENTRY_ARRAY); + return fromEntries(comparator, sameComparator, entryArray, entryArray.length); + } + + private static ImmutableSortedMap fromEntries( + final Comparator comparator, + boolean sameComparator, + Entry[] entryArray, + int size) { + switch (size) { + case 0: + return emptyMap(comparator); + case 1: + return ImmutableSortedMap.of( + comparator, entryArray[0].getKey(), entryArray[0].getValue()); + default: + Object[] keys = new Object[size]; + Object[] values = new Object[size]; + if (sameComparator) { + // Need to check for nulls, but don't need to sort or validate. + for (int i = 0; i < size; i++) { + Object key = entryArray[i].getKey(); + Object value = entryArray[i].getValue(); + checkEntryNotNull(key, value); + keys[i] = key; + values[i] = value; + } + } else { + // Need to sort and check for nulls and dupes. + // Inline the Comparator implementation rather than transforming with a Function + // to save code size. + Arrays.sort( + entryArray, + 0, + size, + new Comparator>() { + @Override + public int compare(Entry e1, Entry e2) { + return comparator.compare(e1.getKey(), e2.getKey()); + } + }); + K prevKey = entryArray[0].getKey(); + keys[0] = prevKey; + values[0] = entryArray[0].getValue(); + checkEntryNotNull(keys[0], values[0]); + for (int i = 1; i < size; i++) { + K key = entryArray[i].getKey(); + V value = entryArray[i].getValue(); + checkEntryNotNull(key, value); + keys[i] = key; + values[i] = value; + checkNoConflict( + comparator.compare(prevKey, key) != 0, "key", entryArray[i - 1], entryArray[i]); + prevKey = key; + } + } + return new ImmutableSortedMap<>( + new RegularImmutableSortedSet(new RegularImmutableList(keys), comparator), + new RegularImmutableList(values)); + } + } + + /** + * Returns a builder that creates immutable sorted maps whose keys are ordered by their natural + * ordering. The sorted maps use {@link Ordering#natural()} as the comparator. + */ + public static , V> Builder naturalOrder() { + return new Builder<>(Ordering.natural()); + } + + /** + * Returns a builder that creates immutable sorted maps with an explicit comparator. If the + * comparator has a more general type than the map's keys, such as creating a {@code + * SortedMap} with a {@code Comparator}, use the {@link Builder} + * constructor instead. + * + * @throws NullPointerException if {@code comparator} is null + */ + public static Builder orderedBy(Comparator comparator) { + return new Builder<>(comparator); + } + + /** + * Returns a builder that creates immutable sorted maps whose keys are ordered by the reverse of + * their natural ordering. + */ + public static , V> Builder reverseOrder() { + return new Builder<>(Ordering.natural().reverse()); + } + + /** + * A builder for creating immutable sorted map instances, especially {@code public static final} + * maps ("constant maps"). Example: + * + *

{@code
+   * static final ImmutableSortedMap INT_TO_WORD =
+   *     new ImmutableSortedMap.Builder(Ordering.natural())
+   *         .put(1, "one")
+   *         .put(2, "two")
+   *         .put(3, "three")
+   *         .build();
+   * }
+ * + *

For small immutable sorted maps, the {@code ImmutableSortedMap.of()} methods are even + * more convenient. + * + *

Builder instances can be reused - it is safe to call {@link #build} multiple times to build + * multiple maps in series. Each map is a superset of the maps created before it. + * + * @since 2.0 + */ + public static class Builder extends ImmutableMap.Builder { + private final Comparator comparator; + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableSortedMap#orderedBy}. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public Builder(Comparator comparator) { + this.comparator = checkNotNull(comparator); + } + + /** + * Associates {@code key} with {@code value} in the built map. Duplicate keys, according to the + * comparator (which might be the keys' natural order), are not allowed, and will cause {@link + * #build} to fail. + */ + @Override + public Builder put(K key, V value) { + super.put(key, value); + return this; + } + + /** + * Adds the given {@code entry} to the map, making it immutable if necessary. Duplicate keys, + * according to the comparator (which might be the keys' natural order), are not allowed, and + * will cause {@link #build} to fail. + * + * @since 11.0 + */ + @Override + public Builder put(Entry entry) { + super.put(entry); + return this; + } + + /** + * Associates all of the given map's keys and values in the built map. Duplicate keys, according + * to the comparator (which might be the keys' natural order), are not allowed, and will cause + * {@link #build} to fail. + * + * @throws NullPointerException if any key or value in {@code map} is null + */ + @Override + public Builder putAll(Map map) { + super.putAll(map); + return this; + } + + /** + * Adds all the given entries to the built map. Duplicate keys, according to the comparator + * (which might be the keys' natural order), are not allowed, and will cause {@link #build} to + * fail. + * + * @throws NullPointerException if any key, value, or entry is null + * @since 19.0 + */ + + @Override + public Builder putAll(Iterable> entries) { + super.putAll(entries); + return this; + } + + /** + * Throws an {@code UnsupportedOperationException}. + * + * @since 19.0 + * @deprecated Unsupported by ImmutableSortedMap.Builder. + */ + + @Override + @Deprecated + public Builder orderEntriesByValue(Comparator valueComparator) { + throw new UnsupportedOperationException("Not available on ImmutableSortedMap.Builder"); + } + + @Override + Builder combine(ImmutableMap.Builder other) { + super.combine(other); + return this; + } + + /** + * Returns a newly-created immutable sorted map. + * + * @throws IllegalArgumentException if any two keys are equal according to the comparator (which + * might be the keys' natural order) + */ + @Override + public ImmutableSortedMap build() { + switch (size) { + case 0: + return emptyMap(comparator); + case 1: + return of(comparator, entries[0].getKey(), entries[0].getValue()); + default: + return fromEntries(comparator, false, entries, size); + } + } + } + + private final transient RegularImmutableSortedSet keySet; + private final transient ImmutableList valueList; + private transient ImmutableSortedMap descendingMap; + + ImmutableSortedMap(RegularImmutableSortedSet keySet, ImmutableList valueList) { + this(keySet, valueList, null); + } + + ImmutableSortedMap( + RegularImmutableSortedSet keySet, + ImmutableList valueList, + ImmutableSortedMap descendingMap) { + this.keySet = keySet; + this.valueList = valueList; + this.descendingMap = descendingMap; + } + + @Override + public int size() { + return valueList.size(); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + ImmutableList keyList = keySet.asList(); + for (int i = 0; i < size(); i++) { + action.accept(keyList.get(i), valueList.get(i)); + } + } + + @Override + public V get(Object key) { + int index = keySet.indexOf(key); + return (index == -1) ? null : valueList.get(index); + } + + @Override + boolean isPartialView() { + return keySet.isPartialView() || valueList.isPartialView(); + } + + /** Returns an immutable set of the mappings in this map, sorted by the key ordering. */ + @Override + public ImmutableSet> entrySet() { + return super.entrySet(); + } + + @Override + ImmutableSet> createEntrySet() { + class EntrySet extends ImmutableMapEntrySet { + @Override + public UnmodifiableIterator> iterator() { + return asList().iterator(); + } + + @Override + public Spliterator> spliterator() { + return asList().spliterator(); + } + + @Override + public void forEach(Consumer> action) { + asList().forEach(action); + } + + @Override + ImmutableList> createAsList() { + return new ImmutableAsList>() { + @Override + public Entry get(int index) { + return new AbstractMap.SimpleImmutableEntry<>( + keySet.asList().get(index), valueList.get(index)); + } + + @Override + public Spliterator> spliterator() { + return CollectSpliterators.indexed( + size(), ImmutableSet.SPLITERATOR_CHARACTERISTICS, this::get); + } + + @Override + ImmutableCollection> delegateCollection() { + return EntrySet.this; + } + }; + } + + @Override + ImmutableMap map() { + return ImmutableSortedMap.this; + } + } + return isEmpty() ? ImmutableSet.>of() : new EntrySet(); + } + + /** Returns an immutable sorted set of the keys in this map. */ + @Override + public ImmutableSortedSet keySet() { + return keySet; + } + + @Override + ImmutableSet createKeySet() { + throw new AssertionError("should never be called"); + } + + /** + * Returns an immutable collection of the values in this map, sorted by the ordering of the + * corresponding keys. + */ + @Override + public ImmutableCollection values() { + return valueList; + } + + @Override + ImmutableCollection createValues() { + throw new AssertionError("should never be called"); + } + + /** + * Returns the comparator that orders the keys, which is {@link Ordering#natural()} when the + * natural ordering of the keys is used. Note that its behavior is not consistent with {@link + * TreeMap#comparator()}, which returns {@code null} to indicate natural ordering. + */ + @Override + public Comparator comparator() { + return keySet().comparator(); + } + + @Override + public K firstKey() { + return keySet().first(); + } + + @Override + public K lastKey() { + return keySet().last(); + } + + private ImmutableSortedMap getSubMap(int fromIndex, int toIndex) { + if (fromIndex == 0 && toIndex == size()) { + return this; + } else if (fromIndex == toIndex) { + return emptyMap(comparator()); + } else { + return new ImmutableSortedMap<>( + keySet.getSubSet(fromIndex, toIndex), valueList.subList(fromIndex, toIndex)); + } + } + + /** + * This method returns a {@code ImmutableSortedMap}, consisting of the entries whose keys are less + * than {@code toKey}. + * + *

The {@link SortedMap#headMap} documentation states that a submap of a submap throws an + * {@link IllegalArgumentException} if passed a {@code toKey} greater than an earlier {@code + * toKey}. However, this method doesn't throw an exception in that situation, but instead keeps + * the original {@code toKey}. + */ + @Override + public ImmutableSortedMap headMap(K toKey) { + return headMap(toKey, false); + } + + /** + * This method returns a {@code ImmutableSortedMap}, consisting of the entries whose keys are less + * than (or equal to, if {@code inclusive}) {@code toKey}. + * + *

The {@link SortedMap#headMap} documentation states that a submap of a submap throws an + * {@link IllegalArgumentException} if passed a {@code toKey} greater than an earlier {@code + * toKey}. However, this method doesn't throw an exception in that situation, but instead keeps + * the original {@code toKey}. + * + * @since 12.0 + */ + @Override + public ImmutableSortedMap headMap(K toKey, boolean inclusive) { + return getSubMap(0, keySet.headIndex(checkNotNull(toKey), inclusive)); + } + + /** + * This method returns a {@code ImmutableSortedMap}, consisting of the entries whose keys ranges + * from {@code fromKey}, inclusive, to {@code toKey}, exclusive. + * + *

The {@link SortedMap#subMap} documentation states that a submap of a submap throws an {@link + * IllegalArgumentException} if passed a {@code fromKey} less than an earlier {@code fromKey}. + * However, this method doesn't throw an exception in that situation, but instead keeps the + * original {@code fromKey}. Similarly, this method keeps the original {@code toKey}, instead of + * throwing an exception, if passed a {@code toKey} greater than an earlier {@code toKey}. + */ + @Override + public ImmutableSortedMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + /** + * This method returns a {@code ImmutableSortedMap}, consisting of the entries whose keys ranges + * from {@code fromKey} to {@code toKey}, inclusive or exclusive as indicated by the boolean + * flags. + * + *

The {@link SortedMap#subMap} documentation states that a submap of a submap throws an {@link + * IllegalArgumentException} if passed a {@code fromKey} less than an earlier {@code fromKey}. + * However, this method doesn't throw an exception in that situation, but instead keeps the + * original {@code fromKey}. Similarly, this method keeps the original {@code toKey}, instead of + * throwing an exception, if passed a {@code toKey} greater than an earlier {@code toKey}. + * + * @since 12.0 + */ + @Override + public ImmutableSortedMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + checkNotNull(fromKey); + checkNotNull(toKey); + checkArgument( + comparator().compare(fromKey, toKey) <= 0, + "expected fromKey <= toKey but %s > %s", + fromKey, + toKey); + return headMap(toKey, toInclusive).tailMap(fromKey, fromInclusive); + } + + /** + * This method returns a {@code ImmutableSortedMap}, consisting of the entries whose keys are + * greater than or equals to {@code fromKey}. + * + *

The {@link SortedMap#tailMap} documentation states that a submap of a submap throws an + * {@link IllegalArgumentException} if passed a {@code fromKey} less than an earlier {@code + * fromKey}. However, this method doesn't throw an exception in that situation, but instead keeps + * the original {@code fromKey}. + */ + @Override + public ImmutableSortedMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + /** + * This method returns a {@code ImmutableSortedMap}, consisting of the entries whose keys are + * greater than (or equal to, if {@code inclusive}) {@code fromKey}. + * + *

The {@link SortedMap#tailMap} documentation states that a submap of a submap throws an + * {@link IllegalArgumentException} if passed a {@code fromKey} less than an earlier {@code + * fromKey}. However, this method doesn't throw an exception in that situation, but instead keeps + * the original {@code fromKey}. + * + * @since 12.0 + */ + @Override + public ImmutableSortedMap tailMap(K fromKey, boolean inclusive) { + return getSubMap(keySet.tailIndex(checkNotNull(fromKey), inclusive), size()); + } + + @Override + public Entry lowerEntry(K key) { + return headMap(key, false).lastEntry(); + } + + @Override + public K lowerKey(K key) { + return keyOrNull(lowerEntry(key)); + } + + @Override + public Entry floorEntry(K key) { + return headMap(key, true).lastEntry(); + } + + @Override + public K floorKey(K key) { + return keyOrNull(floorEntry(key)); + } + + @Override + public Entry ceilingEntry(K key) { + return tailMap(key, true).firstEntry(); + } + + @Override + public K ceilingKey(K key) { + return keyOrNull(ceilingEntry(key)); + } + + @Override + public Entry higherEntry(K key) { + return tailMap(key, false).firstEntry(); + } + + @Override + public K higherKey(K key) { + return keyOrNull(higherEntry(key)); + } + + @Override + public Entry firstEntry() { + return isEmpty() ? null : entrySet().asList().get(0); + } + + @Override + public Entry lastEntry() { + return isEmpty() ? null : entrySet().asList().get(size() - 1); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final Entry pollFirstEntry() { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the map unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final Entry pollLastEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public ImmutableSortedMap descendingMap() { + // TODO(kevinb): the descendingMap is never actually cached at all. Either it should be or the + // code below simplified. + ImmutableSortedMap result = descendingMap; + if (result == null) { + if (isEmpty()) { + return result = emptyMap(Ordering.from(comparator()).reverse()); + } else { + return result = + new ImmutableSortedMap<>( + (RegularImmutableSortedSet) keySet.descendingSet(), valueList.reverse(), this); + } + } + return result; + } + + @Override + public ImmutableSortedSet navigableKeySet() { + return keySet; + } + + @Override + public ImmutableSortedSet descendingKeySet() { + return keySet.descendingSet(); + } + + /** + * Serialized type for all ImmutableSortedMap instances. It captures the logical contents and they + * are reconstructed using public factory methods. This ensures that the implementation types + * remain as implementation details. + */ + private static class SerializedForm extends ImmutableMap.SerializedForm { + private final Comparator comparator; + + @SuppressWarnings({"unchecked", "rawtypes"}) + SerializedForm(ImmutableSortedMap sortedMap) { + super(sortedMap); + comparator = (Comparator) sortedMap.comparator(); + } + + @Override + Object readResolve() { + Builder builder = new Builder<>(comparator); + return createMap(builder); + } + + private static final long serialVersionUID = 0; + } + + @Override + Object writeReplace() { + return new SerializedForm(this); + } + + // This class is never actually serialized directly, but we have to make the + // warning go away (and suppressing would suppress for all nested classes too) + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMapFauxverideShim.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMapFauxverideShim.java new file mode 100644 index 0000000000000..96c15b8cc26a7 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMapFauxverideShim.java @@ -0,0 +1,164 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.function.BinaryOperator; +import java.util.function.Function; +import java.util.stream.Collector; + +/** + * "Overrides" the {@link ImmutableMap} static methods that lack {@link ImmutableSortedMap} + * equivalents with deprecated, exception-throwing versions. See {@link + * ImmutableSortedSetFauxverideShim} for details. + * + * @author Chris Povirk + */ +abstract class ImmutableSortedMapFauxverideShim extends ImmutableMap { + /** + * Not supported. Use {@link ImmutableSortedMap#toImmutableSortedMap}, which offers better + * type-safety, instead. This method exists only to hide {@link ImmutableMap#toImmutableMap} from + * consumers of {@code ImmutableSortedMap}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedMap#toImmutableSortedMap}. + */ + @Deprecated + public static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. Use {@link ImmutableSortedMap#toImmutableSortedMap}, which offers better + * type-safety, instead. This method exists only to hide {@link ImmutableMap#toImmutableMap} from + * consumers of {@code ImmutableSortedMap}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedMap#toImmutableSortedMap}. + */ + @Deprecated + public static Collector> toImmutableMap( + Function keyFunction, + Function valueFunction, + BinaryOperator mergeFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. Use {@link ImmutableSortedMap#naturalOrder}, which offers better type-safety, + * instead. This method exists only to hide {@link ImmutableMap#builder} from consumers of {@code + * ImmutableSortedMap}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedMap#naturalOrder}, which offers better type-safety. + */ + @Deprecated + public static ImmutableSortedMap.Builder builder() { + throw new UnsupportedOperationException(); + } + + /** + * Not supported for ImmutableSortedMap. + * + * @throws UnsupportedOperationException always + * @deprecated Not supported for ImmutableSortedMap. + */ + @Deprecated + public static ImmutableSortedMap.Builder builderWithExpectedSize(int expectedSize) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a map that may contain a non-{@code Comparable} + * key. Proper calls will resolve to the version in {@code ImmutableSortedMap}, not this dummy + * version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass a key of type {@code Comparable} to use {@link + * ImmutableSortedMap#of(Comparable, Object)}. + */ + @Deprecated + public static ImmutableSortedMap of(K k1, V v1) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a map that may contain non-{@code Comparable} + * keys. Proper calls will resolve to the version in {@code ImmutableSortedMap}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass keys of type {@code Comparable} to use {@link + * ImmutableSortedMap#of(Comparable, Object, Comparable, Object)}. + */ + @Deprecated + public static ImmutableSortedMap of(K k1, V v1, K k2, V v2) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a map that may contain non-{@code Comparable} + * keys. Proper calls to will resolve to the version in {@code ImmutableSortedMap}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass keys of type {@code Comparable} to use {@link + * ImmutableSortedMap#of(Comparable, Object, Comparable, Object, Comparable, Object)}. + */ + @Deprecated + public static ImmutableSortedMap of(K k1, V v1, K k2, V v2, K k3, V v3) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a map that may contain non-{@code Comparable} + * keys. Proper calls will resolve to the version in {@code ImmutableSortedMap}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass keys of type {@code Comparable} to use {@link + * ImmutableSortedMap#of(Comparable, Object, Comparable, Object, Comparable, Object, + * Comparable, Object)}. + */ + @Deprecated + public static ImmutableSortedMap of(K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a map that may contain non-{@code Comparable} + * keys. Proper calls will resolve to the version in {@code ImmutableSortedMap}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass keys of type {@code Comparable} to use {@link + * ImmutableSortedMap#of(Comparable, Object, Comparable, Object, Comparable, Object, + * Comparable, Object, Comparable, Object)}. + */ + @Deprecated + public static ImmutableSortedMap of( + K k1, V v1, K k2, V v2, K k3, V v3, K k4, V v4, K k5, V v5) { + throw new UnsupportedOperationException(); + } + + // No copyOf() fauxveride; see ImmutableSortedSetFauxverideShim. +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultiset.java new file mode 100644 index 0000000000000..64f8a97ee5ecb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultiset.java @@ -0,0 +1,586 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.function.ToIntFunction; +import java.util.stream.Collector; + +/** + * A {@link SortedMultiset} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

Warning: as with any sorted collection, you are strongly advised not to use a {@link + * Comparator} or {@link Comparable} type whose comparison behavior is inconsistent with + * equals. That is, {@code a.compareTo(b)} or {@code comparator.compare(a, b)} should equal zero + * if and only if {@code a.equals(b)}. If this advice is not followed, the resulting + * collection will not correctly obey its specification. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Louis Wasserman + * @since 12.0 + */ +public abstract class ImmutableSortedMultiset extends ImmutableSortedMultisetFauxverideShim + implements SortedMultiset { + // TODO(lowasser): GWT compatibility + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableMultiset}. Elements are sorted by the specified comparator. + * + *

Warning: {@code comparator} should be consistent with {@code equals} as + * explained in the {@link Comparator} documentation. + * + * @since 21.0 + */ + + public static Collector> toImmutableSortedMultiset( + Comparator comparator) { + return toImmutableSortedMultiset(comparator, Function.identity(), e -> 1); + } + + /** + * Returns a {@code Collector} that accumulates elements into an {@code ImmutableSortedMultiset} + * whose elements are the result of applying {@code elementFunction} to the inputs, with counts + * equal to the result of applying {@code countFunction} to the inputs. + * + *

If the mapped elements contain duplicates (according to {@code comparator}), the first + * occurrence in encounter order appears in the resulting multiset, with count equal to the sum of + * the outputs of {@code countFunction.applyAsInt(t)} for each {@code t} mapped to that element. + * + * @since 22.0 + */ + public static Collector> toImmutableSortedMultiset( + Comparator comparator, + Function elementFunction, + ToIntFunction countFunction) { + checkNotNull(comparator); + checkNotNull(elementFunction); + checkNotNull(countFunction); + return Collector.of( + () -> TreeMultiset.create(comparator), + (multiset, t) -> + multiset.add(checkNotNull(elementFunction.apply(t)), countFunction.applyAsInt(t)), + (multiset1, multiset2) -> { + multiset1.addAll(multiset2); + return multiset1; + }, + (Multiset multiset) -> copyOfSortedEntries(comparator, multiset.entrySet())); + } + + /** Returns the empty immutable sorted multiset. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableSortedMultiset of() { + return (ImmutableSortedMultiset) RegularImmutableSortedMultiset.NATURAL_EMPTY_MULTISET; + } + + /** Returns an immutable sorted multiset containing a single element. */ + public static > ImmutableSortedMultiset of(E element) { + RegularImmutableSortedSet elementSet = + (RegularImmutableSortedSet) ImmutableSortedSet.of(element); + long[] cumulativeCounts = {0, 1}; + return new RegularImmutableSortedMultiset(elementSet, cumulativeCounts, 0, 1); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedMultiset of(E e1, E e2) { + return copyOf(Ordering.natural(), Arrays.asList(e1, e2)); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedMultiset of(E e1, E e2, E e3) { + return copyOf(Ordering.natural(), Arrays.asList(e1, e2, e3)); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedMultiset of( + E e1, E e2, E e3, E e4) { + return copyOf(Ordering.natural(), Arrays.asList(e1, e2, e3, e4)); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedMultiset of( + E e1, E e2, E e3, E e4, E e5) { + return copyOf(Ordering.natural(), Arrays.asList(e1, e2, e3, e4, e5)); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedMultiset of( + E e1, E e2, E e3, E e4, E e5, E e6, E... remaining) { + int size = remaining.length + 6; + List all = Lists.newArrayListWithCapacity(size); + Collections.addAll(all, e1, e2, e3, e4, e5, e6); + Collections.addAll(all, remaining); + return copyOf(Ordering.natural(), all); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + * @throws NullPointerException if any of {@code elements} is null + */ + public static > ImmutableSortedMultiset copyOf(E[] elements) { + return copyOf(Ordering.natural(), Arrays.asList(elements)); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. To create a copy of a {@code SortedMultiset} that preserves the comparator, call + * {@link #copyOfSorted} instead. This method iterates over {@code elements} at most once. + * + *

Note that if {@code s} is a {@code Multiset}, then {@code + * ImmutableSortedMultiset.copyOf(s)} returns an {@code ImmutableSortedMultiset} + * containing each of the strings in {@code s}, while {@code ImmutableSortedMultiset.of(s)} + * returns an {@code ImmutableSortedMultiset>} containing one element (the given + * multiset itself). + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

This method is not type-safe, as it may be called on elements that are not mutually + * comparable. + * + * @throws ClassCastException if the elements are not mutually comparable + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableSortedMultiset copyOf(Iterable elements) { + // Hack around E not being a subtype of Comparable. + // Unsafe, see ImmutableSortedMultisetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) Ordering.natural(); + return copyOf(naturalOrder, elements); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by their natural + * ordering. + * + *

This method is not type-safe, as it may be called on elements that are not mutually + * comparable. + * + * @throws ClassCastException if the elements are not mutually comparable + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableSortedMultiset copyOf(Iterator elements) { + // Hack around E not being a subtype of Comparable. + // Unsafe, see ImmutableSortedMultisetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) Ordering.natural(); + return copyOf(naturalOrder, elements); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by the given {@code + * Comparator}. + * + * @throws NullPointerException if {@code comparator} or any of {@code elements} is null + */ + public static ImmutableSortedMultiset copyOf( + Comparator comparator, Iterator elements) { + checkNotNull(comparator); + return new Builder(comparator).addAll(elements).build(); + } + + /** + * Returns an immutable sorted multiset containing the given elements sorted by the given {@code + * Comparator}. This method iterates over {@code elements} at most once. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if {@code comparator} or any of {@code elements} is null + */ + public static ImmutableSortedMultiset copyOf( + Comparator comparator, Iterable elements) { + if (elements instanceof ImmutableSortedMultiset) { + @SuppressWarnings({"unchecked", "rawtypes"}) // immutable collections are always safe for covariant casts + ImmutableSortedMultiset multiset = (ImmutableSortedMultiset) elements; + if (comparator.equals(multiset.comparator())) { + if (multiset.isPartialView()) { + return copyOfSortedEntries(comparator, multiset.entrySet().asList()); + } else { + return multiset; + } + } + } + elements = Lists.newArrayList(elements); // defensive copy + TreeMultiset sortedCopy = TreeMultiset.create(checkNotNull(comparator)); + Iterables.addAll(sortedCopy, elements); + return copyOfSortedEntries(comparator, sortedCopy.entrySet()); + } + + /** + * Returns an immutable sorted multiset containing the elements of a sorted multiset, sorted by + * the same {@code Comparator}. That behavior differs from {@link #copyOf(Iterable)}, which always + * uses the natural ordering of the elements. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

This method is safe to use even when {@code sortedMultiset} is a synchronized or concurrent + * collection that is currently being modified by another thread. + * + * @throws NullPointerException if {@code sortedMultiset} or any of its elements is null + */ + public static ImmutableSortedMultiset copyOfSorted(SortedMultiset sortedMultiset) { + return copyOfSortedEntries( + sortedMultiset.comparator(), Lists.newArrayList(sortedMultiset.entrySet())); + } + + private static ImmutableSortedMultiset copyOfSortedEntries( + Comparator comparator, Collection> entries) { + if (entries.isEmpty()) { + return emptyMultiset(comparator); + } + ImmutableList.Builder elementsBuilder = new ImmutableList.Builder(entries.size()); + long[] cumulativeCounts = new long[entries.size() + 1]; + int i = 0; + for (Entry entry : entries) { + elementsBuilder.add(entry.getElement()); + cumulativeCounts[i + 1] = cumulativeCounts[i] + entry.getCount(); + i++; + } + return new RegularImmutableSortedMultiset( + new RegularImmutableSortedSet(elementsBuilder.build(), comparator), + cumulativeCounts, + 0, + entries.size()); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static ImmutableSortedMultiset emptyMultiset(Comparator comparator) { + if (Ordering.natural().equals(comparator)) { + return (ImmutableSortedMultiset) RegularImmutableSortedMultiset.NATURAL_EMPTY_MULTISET; + } else { + return new RegularImmutableSortedMultiset(comparator); + } + } + + ImmutableSortedMultiset() {} + + @Override + public final Comparator comparator() { + return elementSet().comparator(); + } + + @Override + public abstract ImmutableSortedSet elementSet(); + + transient ImmutableSortedMultiset descendingMultiset; + + @Override + public ImmutableSortedMultiset descendingMultiset() { + ImmutableSortedMultiset result = descendingMultiset; + if (result == null) { + return descendingMultiset = + this.isEmpty() + ? emptyMultiset(Ordering.from(comparator()).reverse()) + : new DescendingImmutableSortedMultiset(this); + } + return result; + } + + /** + * {@inheritDoc} + * + *

This implementation is guaranteed to throw an {@link UnsupportedOperationException}. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final Entry pollFirstEntry() { + throw new UnsupportedOperationException(); + } + + /** + * {@inheritDoc} + * + *

This implementation is guaranteed to throw an {@link UnsupportedOperationException}. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final Entry pollLastEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public abstract ImmutableSortedMultiset headMultiset(E upperBound, BoundType boundType); + + @Override + public ImmutableSortedMultiset subMultiset( + E lowerBound, BoundType lowerBoundType, E upperBound, BoundType upperBoundType) { + checkArgument( + comparator().compare(lowerBound, upperBound) <= 0, + "Expected lowerBound <= upperBound but %s > %s", + lowerBound, + upperBound); + return tailMultiset(lowerBound, lowerBoundType).headMultiset(upperBound, upperBoundType); + } + + @Override + public abstract ImmutableSortedMultiset tailMultiset(E lowerBound, BoundType boundType); + + /** + * Returns a builder that creates immutable sorted multisets with an explicit comparator. If the + * comparator has a more general type than the set being generated, such as creating a {@code + * SortedMultiset} with a {@code Comparator}, use the {@link Builder} constructor + * instead. + * + * @throws NullPointerException if {@code comparator} is null + */ + public static Builder orderedBy(Comparator comparator) { + return new Builder(comparator); + } + + /** + * Returns a builder that creates immutable sorted multisets whose elements are ordered by the + * reverse of their natural ordering. + * + *

Note: the type parameter {@code E} extends {@code Comparable} rather than {@code + * Comparable} as a workaround for javac bug 6468354. + */ + public static > Builder reverseOrder() { + return new Builder(Ordering.natural().reverse()); + } + + /** + * Returns a builder that creates immutable sorted multisets whose elements are ordered by their + * natural ordering. The sorted multisets use {@link Ordering#natural()} as the comparator. This + * method provides more type-safety than {@link #builder}, as it can be called only for classes + * that implement {@link Comparable}. + * + *

Note: the type parameter {@code E} extends {@code Comparable} rather than {@code + * Comparable} as a workaround for javac bug 6468354. + */ + public static > Builder naturalOrder() { + return new Builder(Ordering.natural()); + } + + /** + * A builder for creating immutable multiset instances, especially {@code public static final} + * multisets ("constant multisets"). Example: + * + *

{@code
+   * public static final ImmutableSortedMultiset BEANS =
+   *     new ImmutableSortedMultiset.Builder(colorComparator())
+   *         .addCopies(Bean.COCOA, 4)
+   *         .addCopies(Bean.GARDEN, 6)
+   *         .addCopies(Bean.RED, 8)
+   *         .addCopies(Bean.BLACK_EYED, 10)
+   *         .build();
+   * }
+ * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple multisets in series. + * + * @since 12.0 + */ + public static class Builder extends ImmutableMultiset.Builder { + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableSortedMultiset#orderedBy(Comparator)}. + */ + public Builder(Comparator comparator) { + super(TreeMultiset.create(checkNotNull(comparator))); + } + + /** + * Adds {@code element} to the {@code ImmutableSortedMultiset}. + * + * @param element the element to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + */ + @Override + public Builder add(E element) { + super.add(element); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableSortedMultiset}. + * + * @param elements the elements to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @SuppressWarnings("unchecked") + @Override + public Builder add(E... elements) { + super.add(elements); + return this; + } + + /** + * Adds a number of occurrences of an element to this {@code ImmutableSortedMultiset}. + * + * @param element the element to add + * @param occurrences the number of occurrences of the element to add. May be zero, in which + * case no change will be made. + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + * @throws IllegalArgumentException if {@code occurrences} is negative, or if this operation + * would result in more than {@link Integer#MAX_VALUE} occurrences of the element + */ + @Override + public Builder addCopies(E element, int occurrences) { + super.addCopies(element, occurrences); + return this; + } + + /** + * Adds or removes the necessary occurrences of an element such that the element attains the + * desired count. + * + * @param element the element to add or remove occurrences of + * @param count the desired count of the element in this multiset + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + * @throws IllegalArgumentException if {@code count} is negative + */ + @Override + public Builder setCount(E element, int count) { + super.setCount(element, count); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableSortedMultiset}. + * + * @param elements the {@code Iterable} to add to the {@code ImmutableSortedMultiset} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @Override + public Builder addAll(Iterable elements) { + super.addAll(elements); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableSortedMultiset}. + * + * @param elements the elements to add to the {@code ImmutableSortedMultiset} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} is null or contains a null element + */ + @Override + public Builder addAll(Iterator elements) { + super.addAll(elements); + return this; + } + + /** + * Returns a newly-created {@code ImmutableSortedMultiset} based on the contents of the {@code + * Builder}. + */ + @Override + public ImmutableSortedMultiset build() { + return copyOfSorted((SortedMultiset) contents); + } + } + + private static final class SerializedForm implements Serializable { + final Comparator comparator; + final E[] elements; + final int[] counts; + + @SuppressWarnings({"unchecked", "rawtypes"}) + SerializedForm(SortedMultiset multiset) { + this.comparator = multiset.comparator(); + int n = multiset.entrySet().size(); + elements = (E[]) new Object[n]; + counts = new int[n]; + int i = 0; + for (Entry entry : multiset.entrySet()) { + elements[i] = entry.getElement(); + counts[i] = entry.getCount(); + i++; + } + } + + Object readResolve() { + int n = elements.length; + Builder builder = new Builder<>(comparator); + for (int i = 0; i < n; i++) { + builder.addCopies(elements[i], counts[i]); + } + return builder.build(); + } + } + + @Override + Object writeReplace() { + return new SerializedForm(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultisetFauxverideShim.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultisetFauxverideShim.java new file mode 100644 index 0000000000000..bd0c65c57e833 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedMultisetFauxverideShim.java @@ -0,0 +1,202 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.function.Function; +import java.util.function.ToIntFunction; +import java.util.stream.Collector; + +/** + * "Overrides" the {@link ImmutableMultiset} static methods that lack {@link + * ImmutableSortedMultiset} equivalents with deprecated, exception-throwing versions. This prevents + * accidents like the following: + * + *

{@code
+ * List objects = ...;
+ * // Sort them:
+ * Set sorted = ImmutableSortedMultiset.copyOf(objects);
+ * // BAD CODE! The returned multiset is actually an unsorted ImmutableMultiset!
+ * }
+ *
+ * 

While we could put the overrides in {@link ImmutableSortedMultiset} itself, it seems clearer + * to separate these "do not call" methods from those intended for normal use. + * + * @author Louis Wasserman + */ +abstract class ImmutableSortedMultisetFauxverideShim extends ImmutableMultiset { + /** + * Not supported. Use {@link ImmutableSortedMultiset#toImmutableSortedMultiset} instead. This + * method exists only to hide {@link ImmutableMultiset#toImmutableMultiset} from consumers of + * {@code ImmutableSortedMultiset}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedMultiset#toImmutableSortedMultiset}. + * @since 21.0 + */ + @Deprecated + public static Collector> toImmutableMultiset() { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. Use {@link ImmutableSortedMultiset#toImmutableSortedMultiset} instead. This + * method exists only to hide {@link ImmutableMultiset#toImmutableMultiset} from consumers of + * {@code ImmutableSortedMultiset}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedMultiset#toImmutableSortedMultiset}. + * @since 22.0 + */ + @Deprecated + public static Collector> toImmutableMultiset( + Function elementFunction, ToIntFunction countFunction) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. Use {@link ImmutableSortedMultiset#naturalOrder}, which offers better + * type-safety, instead. This method exists only to hide {@link ImmutableMultiset#builder} from + * consumers of {@code ImmutableSortedMultiset}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedMultiset#naturalOrder}, which offers better type-safety. + */ + @Deprecated + public static ImmutableSortedMultiset.Builder builder() { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain a non-{@code + * Comparable} element. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass a parameter of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#of(Comparable)}. + */ + @Deprecated + public static ImmutableSortedMultiset of(E element) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain a non-{@code + * Comparable} element. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#of(Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedMultiset of(E e1, E e2) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain a non-{@code + * Comparable} element. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#of(Comparable, Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedMultiset of(E e1, E e2, E e3) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain a non-{@code + * Comparable} element. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#of(Comparable, Comparable, Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedMultiset of(E e1, E e2, E e3, E e4) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain a non-{@code + * Comparable} element. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#of(Comparable, Comparable, Comparable, Comparable, Comparable)} . + * + */ + @Deprecated + public static ImmutableSortedMultiset of(E e1, E e2, E e3, E e4, E e5) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain a non-{@code + * Comparable} element. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#of(Comparable, Comparable, Comparable, Comparable, Comparable, + * Comparable, Comparable...)} . + */ + @SuppressWarnings("unchecked") + @Deprecated + public static ImmutableSortedMultiset of( + E e1, E e2, E e3, E e4, E e5, E e6, E... remaining) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a multiset that may contain non-{@code + * Comparable} elements. Proper calls will resolve to the version in {@code + * ImmutableSortedMultiset}, not this dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass parameters of type {@code Comparable} to use {@link + * ImmutableSortedMultiset#copyOf(Comparable[])}. + */ + @Deprecated + public static ImmutableSortedMultiset copyOf(E[] elements) { + throw new UnsupportedOperationException(); + } + + /* + * We would like to include an unsupported " copyOf(Iterable)" here, providing only the + * properly typed "> copyOf(Iterable)" in ImmutableSortedMultiset (and + * likewise for the Iterator equivalent). However, due to a change in Sun's interpretation of the + * JLS (as described at http://bugs.sun.com/view_bug.do?bug_id=6182950), the OpenJDK 7 compiler + * available as of this writing rejects our attempts. To maintain compatibility with that version + * and with any other compilers that interpret the JLS similarly, there is no definition of + * copyOf() here, and the definition in ImmutableSortedMultiset matches that in + * ImmutableMultiset. + * + * The result is that ImmutableSortedMultiset.copyOf() may be called on non-Comparable elements. + * We have not discovered a better solution. In retrospect, the static factory methods should + * have gone in a separate class so that ImmutableSortedMultiset wouldn't "inherit" + * too-permissive factory methods from ImmutableMultiset. + */ +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSet.java new file mode 100644 index 0000000000000..ae5668aa83eda --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSet.java @@ -0,0 +1,826 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.ObjectArrays.checkElementsNotNull; + + +import java.io.InvalidObjectException; +import java.io.ObjectInputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.SortedSet; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.Consumer; +import java.util.stream.Collector; + + +/** + * A {@link NavigableSet} whose contents will never change, with many other important properties + * detailed at {@link ImmutableCollection}. + * + *

Warning: as with any sorted collection, you are strongly advised not to use a {@link + * Comparator} or {@link Comparable} type whose comparison behavior is inconsistent with + * equals. That is, {@code a.compareTo(b)} or {@code comparator.compare(a, b)} should equal zero + * if and only if {@code a.equals(b)}. If this advice is not followed, the resulting + * collection will not correctly obey its specification. + * + *

See the Guava User Guide article on immutable collections. + * + * @author Jared Levy + * @author Louis Wasserman + * @since 2.0 (implements {@code NavigableSet} since 12.0) + */ +// TODO(benyu): benchmark and optimize all creation paths, which are a mess now +@SuppressWarnings("serial") // we're overriding default serialization +public abstract class ImmutableSortedSet extends ImmutableSortedSetFauxverideShim + implements NavigableSet, SortedIterable { + static final int SPLITERATOR_CHARACTERISTICS = + ImmutableSet.SPLITERATOR_CHARACTERISTICS | Spliterator.SORTED; + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code + * ImmutableSortedSet}, ordered by the specified comparator. + * + *

If the elements contain duplicates (according to the comparator), only the first duplicate + * in encounter order will appear in the result. + * + * @since 21.0 + */ + + public static Collector> toImmutableSortedSet( + Comparator comparator) { + return CollectCollectors.toImmutableSortedSet(comparator); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static RegularImmutableSortedSet emptySet(Comparator comparator) { + if (Ordering.natural().equals(comparator)) { + return (RegularImmutableSortedSet) RegularImmutableSortedSet.NATURAL_EMPTY_SET; + } else { + return new RegularImmutableSortedSet(ImmutableList.of(), comparator); + } + } + + /** Returns the empty immutable sorted set. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableSortedSet of() { + return (ImmutableSortedSet) RegularImmutableSortedSet.NATURAL_EMPTY_SET; + } + + /** Returns an immutable sorted set containing a single element. */ + public static > ImmutableSortedSet of(E element) { + return new RegularImmutableSortedSet(ImmutableList.of(element), Ordering.natural()); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@link Comparable#compareTo}, only the first + * one specified is included. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedSet of(E e1, E e2) { + return construct(Ordering.natural(), 2, e1, e2); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@link Comparable#compareTo}, only the first + * one specified is included. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedSet of(E e1, E e2, E e3) { + return construct(Ordering.natural(), 3, e1, e2, e3); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@link Comparable#compareTo}, only the first + * one specified is included. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedSet of(E e1, E e2, E e3, E e4) { + return construct(Ordering.natural(), 4, e1, e2, e3, e4); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@link Comparable#compareTo}, only the first + * one specified is included. + * + * @throws NullPointerException if any element is null + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedSet of( + E e1, E e2, E e3, E e4, E e5) { + return construct(Ordering.natural(), 5, e1, e2, e3, e4, e5); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@link Comparable#compareTo}, only the first + * one specified is included. + * + * @throws NullPointerException if any element is null + * @since 3.0 (source-compatible since 2.0) + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSortedSet of( + E e1, E e2, E e3, E e4, E e5, E e6, E... remaining) { + Comparable[] contents = new Comparable[6 + remaining.length]; + contents[0] = e1; + contents[1] = e2; + contents[2] = e3; + contents[3] = e4; + contents[4] = e5; + contents[5] = e6; + System.arraycopy(remaining, 0, contents, 6, remaining.length); + return construct(Ordering.natural(), contents.length, (E[]) contents); + } + + // TODO(kevinb): Consider factory methods that reject duplicates + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@link Comparable#compareTo}, only the first + * one specified is included. + * + * @throws NullPointerException if any of {@code elements} is null + * @since 3.0 + */ + public static > ImmutableSortedSet copyOf(E[] elements) { + return construct(Ordering.natural(), elements.length, elements.clone()); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@code compareTo()}, only the first one + * specified is included. To create a copy of a {@code SortedSet} that preserves the comparator, + * call {@link #copyOfSorted} instead. This method iterates over {@code elements} at most once. + * + *

Note that if {@code s} is a {@code Set}, then {@code ImmutableSortedSet.copyOf(s)} + * returns an {@code ImmutableSortedSet} containing each of the strings in {@code s}, + * while {@code ImmutableSortedSet.of(s)} returns an {@code ImmutableSortedSet>} + * containing one element (the given set itself). + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

This method is not type-safe, as it may be called on elements that are not mutually + * comparable. + * + * @throws ClassCastException if the elements are not mutually comparable + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableSortedSet copyOf(Iterable elements) { + // Hack around E not being a subtype of Comparable. + // Unsafe, see ImmutableSortedSetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) Ordering.natural(); + return copyOf(naturalOrder, elements); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@code compareTo()}, only the first one + * specified is included. To create a copy of a {@code SortedSet} that preserves the comparator, + * call {@link #copyOfSorted} instead. This method iterates over {@code elements} at most once. + * + *

Note that if {@code s} is a {@code Set}, then {@code ImmutableSortedSet.copyOf(s)} + * returns an {@code ImmutableSortedSet} containing each of the strings in {@code s}, + * while {@code ImmutableSortedSet.of(s)} returns an {@code ImmutableSortedSet>} + * containing one element (the given set itself). + * + *

Note: Despite what the method name suggests, if {@code elements} is an {@code + * ImmutableSortedSet}, it may be returned instead of a copy. + * + *

This method is not type-safe, as it may be called on elements that are not mutually + * comparable. + * + *

This method is safe to use even when {@code elements} is a synchronized or concurrent + * collection that is currently being modified by another thread. + * + * @throws ClassCastException if the elements are not mutually comparable + * @throws NullPointerException if any of {@code elements} is null + * @since 7.0 (source-compatible since 2.0) + */ + public static ImmutableSortedSet copyOf(Collection elements) { + // Hack around E not being a subtype of Comparable. + // Unsafe, see ImmutableSortedSetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) Ordering.natural(); + return copyOf(naturalOrder, elements); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by their natural ordering. + * When multiple elements are equivalent according to {@code compareTo()}, only the first one + * specified is included. + * + *

This method is not type-safe, as it may be called on elements that are not mutually + * comparable. + * + * @throws ClassCastException if the elements are not mutually comparable + * @throws NullPointerException if any of {@code elements} is null + */ + public static ImmutableSortedSet copyOf(Iterator elements) { + // Hack around E not being a subtype of Comparable. + // Unsafe, see ImmutableSortedSetFauxverideShim. + @SuppressWarnings({"unchecked", "rawtypes"}) + Ordering naturalOrder = (Ordering) Ordering.natural(); + return copyOf(naturalOrder, elements); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by the given {@code + * Comparator}. When multiple elements are equivalent according to {@code compareTo()}, only the + * first one specified is included. + * + * @throws NullPointerException if {@code comparator} or any of {@code elements} is null + */ + public static ImmutableSortedSet copyOf( + Comparator comparator, Iterator elements) { + return new Builder(comparator).addAll(elements).build(); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by the given {@code + * Comparator}. When multiple elements are equivalent according to {@code compare()}, only the + * first one specified is included. This method iterates over {@code elements} at most once. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + * @throws NullPointerException if {@code comparator} or any of {@code elements} is null + */ + public static ImmutableSortedSet copyOf( + Comparator comparator, Iterable elements) { + checkNotNull(comparator); + boolean hasSameComparator = SortedIterables.hasSameComparator(comparator, elements); + + if (hasSameComparator && (elements instanceof ImmutableSortedSet)) { + @SuppressWarnings({"unchecked", "rawtypes"}) + ImmutableSortedSet original = (ImmutableSortedSet) elements; + if (!original.isPartialView()) { + return original; + } + } + @SuppressWarnings({"unchecked", "rawtypes"}) // elements only contains E's; it's safe. + E[] array = (E[]) Iterables.toArray(elements); + return construct(comparator, array.length, array); + } + + /** + * Returns an immutable sorted set containing the given elements sorted by the given {@code + * Comparator}. When multiple elements are equivalent according to {@code compareTo()}, only the + * first one specified is included. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

This method is safe to use even when {@code elements} is a synchronized or concurrent + * collection that is currently being modified by another thread. + * + * @throws NullPointerException if {@code comparator} or any of {@code elements} is null + * @since 7.0 (source-compatible since 2.0) + */ + public static ImmutableSortedSet copyOf( + Comparator comparator, Collection elements) { + return copyOf(comparator, (Iterable) elements); + } + + /** + * Returns an immutable sorted set containing the elements of a sorted set, sorted by the same + * {@code Comparator}. That behavior differs from {@link #copyOf(Iterable)}, which always uses the + * natural ordering of the elements. + * + *

Despite the method name, this method attempts to avoid actually copying the data when it is + * safe to do so. The exact circumstances under which a copy will or will not be performed are + * undocumented and subject to change. + * + *

This method is safe to use even when {@code sortedSet} is a synchronized or concurrent + * collection that is currently being modified by another thread. + * + * @throws NullPointerException if {@code sortedSet} or any of its elements is null + */ + public static ImmutableSortedSet copyOfSorted(SortedSet sortedSet) { + Comparator comparator = SortedIterables.comparator(sortedSet); + ImmutableList list = ImmutableList.copyOf(sortedSet); + if (list.isEmpty()) { + return emptySet(comparator); + } else { + return new RegularImmutableSortedSet(list, comparator); + } + } + + /** + * Constructs an {@code ImmutableSortedSet} from the first {@code n} elements of {@code contents}. + * If {@code k} is the size of the returned {@code ImmutableSortedSet}, then the sorted unique + * elements are in the first {@code k} positions of {@code contents}, and {@code contents[i] == + * null} for {@code k <= i < n}. + * + *

If {@code k == contents.length}, then {@code contents} may no longer be safe for + * modification. + * + * @throws NullPointerException if any of the first {@code n} elements of {@code contents} is null + */ + @SuppressWarnings("unchecked") + static ImmutableSortedSet construct( + Comparator comparator, int n, E... contents) { + if (n == 0) { + return emptySet(comparator); + } + checkElementsNotNull(contents, n); + Arrays.sort(contents, 0, n, comparator); + int uniques = 1; + for (int i = 1; i < n; i++) { + E cur = contents[i]; + E prev = contents[uniques - 1]; + if (comparator.compare(cur, prev) != 0) { + contents[uniques++] = cur; + } + } + Arrays.fill(contents, uniques, n, null); + return new RegularImmutableSortedSet( + ImmutableList.asImmutableList(contents, uniques), comparator); + } + + /** + * Returns a builder that creates immutable sorted sets with an explicit comparator. If the + * comparator has a more general type than the set being generated, such as creating a {@code + * SortedSet} with a {@code Comparator}, use the {@link Builder} constructor + * instead. + * + * @throws NullPointerException if {@code comparator} is null + */ + public static Builder orderedBy(Comparator comparator) { + return new Builder(comparator); + } + + /** + * Returns a builder that creates immutable sorted sets whose elements are ordered by the reverse + * of their natural ordering. + */ + public static > Builder reverseOrder() { + return new Builder(Collections.reverseOrder()); + } + + /** + * Returns a builder that creates immutable sorted sets whose elements are ordered by their + * natural ordering. The sorted sets use {@link Ordering#natural()} as the comparator. This method + * provides more type-safety than {@link #builder}, as it can be called only for classes that + * implement {@link Comparable}. + */ + public static > Builder naturalOrder() { + return new Builder(Ordering.natural()); + } + + /** + * A builder for creating immutable sorted set instances, especially {@code public static final} + * sets ("constant sets"), with a given comparator. Example: + * + *

{@code
+   * public static final ImmutableSortedSet LUCKY_NUMBERS =
+   *     new ImmutableSortedSet.Builder(ODDS_FIRST_COMPARATOR)
+   *         .addAll(SINGLE_DIGIT_PRIMES)
+   *         .add(42)
+   *         .build();
+   * }
+ * + *

Builder instances can be reused; it is safe to call {@link #build} multiple times to build + * multiple sets in series. Each set is a superset of the set created before it. + * + * @since 2.0 + */ + public static final class Builder extends ImmutableSet.Builder { + private final Comparator comparator; + private E[] elements; + private int n; + + /** + * Creates a new builder. The returned builder is equivalent to the builder generated by {@link + * ImmutableSortedSet#orderedBy}. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public Builder(Comparator comparator) { + super(true); // don't construct guts of hash-based set builder + this.comparator = checkNotNull(comparator); + this.elements = (E[]) new Object[ImmutableCollection.Builder.DEFAULT_INITIAL_CAPACITY]; + this.n = 0; + } + + @Override + void copy() { + elements = Arrays.copyOf(elements, elements.length); + } + + private void sortAndDedup() { + if (n == 0) { + return; + } + Arrays.sort(elements, 0, n, comparator); + int unique = 1; + for (int i = 1; i < n; i++) { + int cmp = comparator.compare(elements[unique - 1], elements[i]); + if (cmp < 0) { + elements[unique++] = elements[i]; + } else if (cmp > 0) { + throw new AssertionError( + "Comparator " + comparator + " compare method violates its contract"); + } + } + Arrays.fill(elements, unique, n, null); + n = unique; + } + + /** + * Adds {@code element} to the {@code ImmutableSortedSet}. If the {@code ImmutableSortedSet} + * already contains {@code element}, then {@code add} has no effect. (only the previously added + * element is retained). + * + * @param element the element to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code element} is null + */ + @Override + public Builder add(E element) { + checkNotNull(element); + copyIfNecessary(); + if (n == elements.length) { + sortAndDedup(); + /* + * Sorting operations can only be allowed to occur once every O(n) operations to keep + * amortized O(n log n) performance. Therefore, ensure there are at least O(n) *unused* + * spaces in the builder array. + */ + int newLength = ImmutableCollection.Builder.expandedCapacity(n, n + 1); + if (newLength > elements.length) { + elements = Arrays.copyOf(elements, newLength); + } + } + elements[n++] = element; + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableSortedSet}, ignoring duplicate + * elements (only the first duplicate element is added). + * + * @param elements the elements to add + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} contains a null element + */ + @SuppressWarnings("unchecked") + @Override + public Builder add(E... elements) { + checkElementsNotNull(elements); + for (E e : elements) { + add(e); + } + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableSortedSet}, ignoring duplicate + * elements (only the first duplicate element is added). + * + * @param elements the elements to add to the {@code ImmutableSortedSet} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} contains a null element + */ + @Override + public Builder addAll(Iterable elements) { + super.addAll(elements); + return this; + } + + /** + * Adds each element of {@code elements} to the {@code ImmutableSortedSet}, ignoring duplicate + * elements (only the first duplicate element is added). + * + * @param elements the elements to add to the {@code ImmutableSortedSet} + * @return this {@code Builder} object + * @throws NullPointerException if {@code elements} contains a null element + */ + @Override + public Builder addAll(Iterator elements) { + super.addAll(elements); + return this; + } + + @Override + Builder combine(ImmutableSet.Builder builder) { + copyIfNecessary(); + Builder other = (Builder) builder; + for (int i = 0; i < other.n; i++) { + add(other.elements[i]); + } + return this; + } + + /** + * Returns a newly-created {@code ImmutableSortedSet} based on the contents of the {@code + * Builder} and its comparator. + */ + @Override + public ImmutableSortedSet build() { + sortAndDedup(); + if (n == 0) { + return emptySet(comparator); + } else { + forceCopy = true; + return new RegularImmutableSortedSet( + ImmutableList.asImmutableList(elements, n), comparator); + } + } + } + + int unsafeCompare(Object a, Object b) { + return unsafeCompare(comparator, a, b); + } + + static int unsafeCompare(Comparator comparator, Object a, Object b) { + // Pretend the comparator can compare anything. If it turns out it can't + // compare a and b, we should get a CCE on the subsequent line. Only methods + // that are spec'd to throw CCE should call this. + @SuppressWarnings({"unchecked", "rawtypes"}) + Comparator unsafeComparator = (Comparator) comparator; + return unsafeComparator.compare(a, b); + } + + final transient Comparator comparator; + + ImmutableSortedSet(Comparator comparator) { + this.comparator = comparator; + } + + /** + * Returns the comparator that orders the elements, which is {@link Ordering#natural()} when the + * natural ordering of the elements is used. Note that its behavior is not consistent with {@link + * SortedSet#comparator()}, which returns {@code null} to indicate natural ordering. + */ + @Override + public Comparator comparator() { + return comparator; + } + + @Override // needed to unify the iterator() methods in Collection and SortedIterable + public abstract UnmodifiableIterator iterator(); + + /** + * {@inheritDoc} + * + *

This method returns a serializable {@code ImmutableSortedSet}. + * + *

The {@link SortedSet#headSet} documentation states that a subset of a subset throws an + * {@link IllegalArgumentException} if passed a {@code toElement} greater than an earlier {@code + * toElement}. However, this method doesn't throw an exception in that situation, but instead + * keeps the original {@code toElement}. + */ + @Override + public ImmutableSortedSet headSet(E toElement) { + return headSet(toElement, false); + } + + /** @since 12.0 */ + @Override + public ImmutableSortedSet headSet(E toElement, boolean inclusive) { + return headSetImpl(checkNotNull(toElement), inclusive); + } + + /** + * {@inheritDoc} + * + *

This method returns a serializable {@code ImmutableSortedSet}. + * + *

The {@link SortedSet#subSet} documentation states that a subset of a subset throws an {@link + * IllegalArgumentException} if passed a {@code fromElement} smaller than an earlier {@code + * fromElement}. However, this method doesn't throw an exception in that situation, but instead + * keeps the original {@code fromElement}. Similarly, this method keeps the original {@code + * toElement}, instead of throwing an exception, if passed a {@code toElement} greater than an + * earlier {@code toElement}. + */ + @Override + public ImmutableSortedSet subSet(E fromElement, E toElement) { + return subSet(fromElement, true, toElement, false); + } + + /** @since 12.0 */ + @Override + public ImmutableSortedSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + checkNotNull(fromElement); + checkNotNull(toElement); + checkArgument(comparator.compare(fromElement, toElement) <= 0); + return subSetImpl(fromElement, fromInclusive, toElement, toInclusive); + } + + /** + * {@inheritDoc} + * + *

This method returns a serializable {@code ImmutableSortedSet}. + * + *

The {@link SortedSet#tailSet} documentation states that a subset of a subset throws an + * {@link IllegalArgumentException} if passed a {@code fromElement} smaller than an earlier {@code + * fromElement}. However, this method doesn't throw an exception in that situation, but instead + * keeps the original {@code fromElement}. + */ + @Override + public ImmutableSortedSet tailSet(E fromElement) { + return tailSet(fromElement, true); + } + + /** @since 12.0 */ + @Override + public ImmutableSortedSet tailSet(E fromElement, boolean inclusive) { + return tailSetImpl(checkNotNull(fromElement), inclusive); + } + + /* + * These methods perform most headSet, subSet, and tailSet logic, besides + * parameter validation. + */ + abstract ImmutableSortedSet headSetImpl(E toElement, boolean inclusive); + + abstract ImmutableSortedSet subSetImpl( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive); + + abstract ImmutableSortedSet tailSetImpl(E fromElement, boolean inclusive); + + /** @since 12.0 */ + @Override + public E lower(E e) { + return Iterators.getNext(headSet(e, false).descendingIterator(), null); + } + + /** @since 12.0 */ + @Override + public E floor(E e) { + return Iterators.getNext(headSet(e, true).descendingIterator(), null); + } + + /** @since 12.0 */ + @Override + public E ceiling(E e) { + return Iterables.getFirst(tailSet(e, true), null); + } + + /** @since 12.0 */ + @Override + public E higher(E e) { + return Iterables.getFirst(tailSet(e, false), null); + } + + @Override + public E first() { + return iterator().next(); + } + + @Override + public E last() { + return descendingIterator().next(); + } + + /** + * Guaranteed to throw an exception and leave the set unmodified. + * + * @since 12.0 + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final E pollFirst() { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the set unmodified. + * + * @since 12.0 + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final E pollLast() { + throw new UnsupportedOperationException(); + } + + transient ImmutableSortedSet descendingSet; + + /** @since 12.0 */ + @Override + public ImmutableSortedSet descendingSet() { + // racy single-check idiom + ImmutableSortedSet result = descendingSet; + if (result == null) { + result = descendingSet = createDescendingSet(); + result.descendingSet = this; + } + return result; + } + + // Most classes should implement this as new DescendingImmutableSortedSet(this), + // but we push down that implementation because ProGuard can't eliminate it even when it's always + // overridden. + abstract ImmutableSortedSet createDescendingSet(); + + @Override + public Spliterator spliterator() { + return new Spliterators.AbstractSpliterator( + size(), SPLITERATOR_CHARACTERISTICS | Spliterator.SIZED) { + final UnmodifiableIterator iterator = iterator(); + + @Override + public boolean tryAdvance(Consumer action) { + if (iterator.hasNext()) { + action.accept(iterator.next()); + return true; + } else { + return false; + } + } + + @Override + public Comparator getComparator() { + return comparator; + } + }; + } + + /** @since 12.0 */ + @Override + public abstract UnmodifiableIterator descendingIterator(); + + /** Returns the position of an element within the set, or -1 if not present. */ + abstract int indexOf(Object target); + + /* + * This class is used to serialize all ImmutableSortedSet instances, + * regardless of implementation type. It captures their "logical contents" + * only. This is necessary to ensure that the existence of a particular + * implementation type is an implementation detail. + */ + private static class SerializedForm implements Serializable { + final Comparator comparator; + final Object[] elements; + + public SerializedForm(Comparator comparator, Object[] elements) { + this.comparator = comparator; + this.elements = elements; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + Object readResolve() { + return new Builder(comparator).add((E[]) elements).build(); + } + + private static final long serialVersionUID = 0; + } + + private void readObject(ObjectInputStream stream) throws InvalidObjectException { + throw new InvalidObjectException("Use SerializedForm"); + } + + @Override + Object writeReplace() { + return new SerializedForm(comparator, toArray()); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSetFauxverideShim.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSetFauxverideShim.java new file mode 100644 index 0000000000000..4663dc0331eb9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ImmutableSortedSetFauxverideShim.java @@ -0,0 +1,199 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.stream.Collector; + +/** + * "Overrides" the {@link ImmutableSet} static methods that lack {@link ImmutableSortedSet} + * equivalents with deprecated, exception-throwing versions. This prevents accidents like the + * following: + * + *

{@code
+ * List objects = ...;
+ * // Sort them:
+ * Set sorted = ImmutableSortedSet.copyOf(objects);
+ * // BAD CODE! The returned set is actually an unsorted ImmutableSet!
+ * }
+ *
+ * 

While we could put the overrides in {@link ImmutableSortedSet} itself, it seems clearer to + * separate these "do not call" methods from those intended for normal use. + * + * @author Chris Povirk + */ +abstract class ImmutableSortedSetFauxverideShim extends ImmutableSet { + /** + * Not supported. Use {@link ImmutableSortedSet#toImmutableSortedSet} instead. This method exists + * only to hide {@link ImmutableSet#toImmutableSet} from consumers of {@code ImmutableSortedSet}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedSet#toImmutableSortedSet}. + * @since 21.0 + */ + @Deprecated + public static Collector> toImmutableSet() { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. Use {@link ImmutableSortedSet#naturalOrder}, which offers better type-safety, + * instead. This method exists only to hide {@link ImmutableSet#builder} from consumers of {@code + * ImmutableSortedSet}. + * + * @throws UnsupportedOperationException always + * @deprecated Use {@link ImmutableSortedSet#naturalOrder}, which offers better type-safety. + */ + @Deprecated + public static ImmutableSortedSet.Builder builder() { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. This method exists only to hide {@link ImmutableSet#builderWithExpectedSize} + * from consumers of {@code ImmutableSortedSet}. + * + * @throws UnsupportedOperationException always + * @deprecated Not supported by ImmutableSortedSet. + */ + @Deprecated + public static ImmutableSortedSet.Builder builderWithExpectedSize(int expectedSize) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain a non-{@code Comparable} + * element. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass a parameter of type {@code Comparable} to use {@link + * ImmutableSortedSet#of(Comparable)}. + */ + @Deprecated + public static ImmutableSortedSet of(E element) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain a non-{@code Comparable} + * element. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedSet#of(Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedSet of(E e1, E e2) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain a non-{@code Comparable} + * element. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedSet#of(Comparable, Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedSet of(E e1, E e2, E e3) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain a non-{@code Comparable} + * element. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedSet#of(Comparable, Comparable, Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedSet of(E e1, E e2, E e3, E e4) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain a non-{@code Comparable} + * element. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedSet#of( Comparable, Comparable, Comparable, Comparable, Comparable)}. + */ + @Deprecated + public static ImmutableSortedSet of(E e1, E e2, E e3, E e4, E e5) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain a non-{@code Comparable} + * element. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass the parameters of type {@code Comparable} to use {@link + * ImmutableSortedSet#of(Comparable, Comparable, Comparable, Comparable, Comparable, + * Comparable, Comparable...)}. + */ + @SuppressWarnings("unchecked") + @Deprecated + public static ImmutableSortedSet of(E e1, E e2, E e3, E e4, E e5, E e6, E... remaining) { + throw new UnsupportedOperationException(); + } + + /** + * Not supported. You are attempting to create a set that may contain non-{@code Comparable} + * elements. Proper calls will resolve to the version in {@code ImmutableSortedSet}, not this + * dummy version. + * + * @throws UnsupportedOperationException always + * @deprecated Pass parameters of type {@code Comparable} to use {@link + * ImmutableSortedSet#copyOf(Comparable[])}. + */ + @Deprecated + public static ImmutableSortedSet copyOf(E[] elements) { + throw new UnsupportedOperationException(); + } + + /* + * We would like to include an unsupported " copyOf(Iterable)" here, + * providing only the properly typed + * "> copyOf(Iterable)" in ImmutableSortedSet (and + * likewise for the Iterator equivalent). However, due to a change in Sun's + * interpretation of the JLS (as described at + * http://bugs.sun.com/view_bug.do?bug_id=6182950), the OpenJDK 7 compiler + * available as of this writing rejects our attempts. To maintain + * compatibility with that version and with any other compilers that interpret + * the JLS similarly, there is no definition of copyOf() here, and the + * definition in ImmutableSortedSet matches that in ImmutableSet. + * + * The result is that ImmutableSortedSet.copyOf() may be called on + * non-Comparable elements. We have not discovered a better solution. In + * retrospect, the static factory methods should have gone in a separate class + * so that ImmutableSortedSet wouldn't "inherit" too-permissive factory + * methods from ImmutableSet. + */ +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/IndexedImmutableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/IndexedImmutableSet.java new file mode 100644 index 0000000000000..5fae370efd8c0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/IndexedImmutableSet.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2018 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Spliterator; +import java.util.function.Consumer; + +abstract class IndexedImmutableSet extends ImmutableSet { + abstract E get(int index); + + @Override + public UnmodifiableIterator iterator() { + return asList().iterator(); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.indexed(size(), SPLITERATOR_CHARACTERISTICS, this::get); + } + + @Override + public void forEach(Consumer consumer) { + checkNotNull(consumer); + int n = size(); + for (int i = 0; i < n; i++) { + consumer.accept(get(i)); + } + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + return asList().copyIntoArray(dst, offset); + } + + @Override + ImmutableList createAsList() { + return new ImmutableAsList() { + @Override + public E get(int index) { + return IndexedImmutableSet.this.get(index); + } + + @Override + boolean isPartialView() { + return IndexedImmutableSet.this.isPartialView(); + } + + @Override + public int size() { + return IndexedImmutableSet.this.size(); + } + + @Override + ImmutableCollection delegateCollection() { + return IndexedImmutableSet.this; + } + }; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interner.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interner.java new file mode 100644 index 0000000000000..3ae52450605e1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interner.java @@ -0,0 +1,46 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +/** + * Provides equivalent behavior to {@link String#intern} for other immutable types. Common + * implementations are available from the {@link Interners} class. + * + * @author Kevin Bourrillion + * @since 3.0 + */ + +public interface Interner { + /** + * Chooses and returns the representative instance for any of a collection of instances that are + * equal to each other. If two {@linkplain Object#equals equal} inputs are given to this method, + * both calls will return the same instance. That is, {@code intern(a).equals(a)} always holds, + * and {@code intern(a) == intern(b)} if and only if {@code a.equals(b)}. Note that {@code + * intern(a)} is permitted to return one instance now and a different instance later if the + * original interned instance was garbage-collected. + * + *

Warning: do not use with mutable objects. + * + * @throws NullPointerException if {@code sample} is null + */ + E intern(E sample); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interners.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interners.java new file mode 100644 index 0000000000000..c9d352bf341d0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Interners.java @@ -0,0 +1,188 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Equivalence; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapMaker.Dummy; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapMakerInternalMap.InternalEntry; + +/** + * Contains static methods pertaining to instances of {@link Interner}. + * + * @author Kevin Bourrillion + * @since 3.0 + */ + +public final class Interners { + private Interners() {} + + /** + * Builder for {@link Interner} instances. + * + * @since 21.0 + */ + public static class InternerBuilder { + private final MapMaker mapMaker = new MapMaker(); + private boolean strong = true; + + private InternerBuilder() {} + + /** + * Instructs the {@link InternerBuilder} to build a strong interner. + * + * @see Interners#newStrongInterner() + */ + public InternerBuilder strong() { + this.strong = true; + return this; + } + + /** + * Instructs the {@link InternerBuilder} to build a weak interner. + * + * @see Interners#newWeakInterner() + */ + public InternerBuilder weak() { + this.strong = false; + return this; + } + + /** + * Sets the concurrency level that will be used by the to-be-built {@link Interner}. + * + * @see MapMaker#concurrencyLevel(int) + */ + public InternerBuilder concurrencyLevel(int concurrencyLevel) { + this.mapMaker.concurrencyLevel(concurrencyLevel); + return this; + } + + public Interner build() { + if (!strong) { + mapMaker.weakKeys(); + } + return new InternerImpl(mapMaker); + } + } + + /** Returns a fresh {@link InternerBuilder} instance. */ + public static InternerBuilder newBuilder() { + return new InternerBuilder(); + } + + /** + * Returns a new thread-safe interner which retains a strong reference to each instance it has + * interned, thus preventing these instances from being garbage-collected. If this retention is + * acceptable, this implementation may perform better than {@link #newWeakInterner}. + */ + public static Interner newStrongInterner() { + return newBuilder().strong().build(); + } + + /** + * Returns a new thread-safe interner which retains a weak reference to each instance it has + * interned, and so does not prevent these instances from being garbage-collected. This most + * likely does not perform as well as {@link #newStrongInterner}, but is the best alternative when + * the memory usage of that implementation is unacceptable. + */ + public static Interner newWeakInterner() { + return newBuilder().weak().build(); + } + + + static final class InternerImpl implements Interner { + // MapMaker is our friend, we know about this type + final MapMakerInternalMap map; + + private InternerImpl(MapMaker mapMaker) { + this.map = + MapMakerInternalMap.createWithDummyValues(mapMaker.keyEquivalence(Equivalence.equals())); + } + + @Override + public E intern(E sample) { + while (true) { + // trying to read the canonical... + InternalEntry entry = map.getEntry(sample); + if (entry != null) { + E canonical = entry.getKey(); + if (canonical != null) { // only matters if weak/soft keys are used + return canonical; + } + } + + // didn't see it, trying to put it instead... + Dummy sneaky = map.putIfAbsent(sample, Dummy.VALUE); + if (sneaky == null) { + return sample; + } else { + /* Someone beat us to it! Trying again... + * + * Technically this loop not guaranteed to terminate, so theoretically (extremely + * unlikely) this thread might starve, but even then, there is always going to be another + * thread doing progress here. + */ + } + } + } + } + + /** + * Returns a function that delegates to the {@link Interner#intern} method of the given interner. + * + * @since 8.0 + */ + public static Function asFunction(Interner interner) { + return new InternerFunction(checkNotNull(interner)); + } + + private static class InternerFunction implements Function { + + private final Interner interner; + + public InternerFunction(Interner interner) { + this.interner = interner; + } + + @Override + public E apply(E input) { + return interner.intern(input); + } + + @Override + public int hashCode() { + return interner.hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof InternerFunction) { + InternerFunction that = (InternerFunction) other; + return interner.equals(that.interner); + } + + return false; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterables.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterables.java new file mode 100644 index 0000000000000..f29ded8d2c64f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterables.java @@ -0,0 +1,1033 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Optional; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.RandomAccess; +import java.util.Set; +import java.util.Spliterator; +import java.util.function.Consumer; +import java.util.stream.Stream; + + +/** + * An assortment of mainly legacy static utility methods that operate on or return objects of type + * {@code Iterable}. Except as noted, each method has a corresponding {@link Iterator}-based method + * in the {@link Iterators} class. + * + *

Java 8 users: several common uses for this class are now more comprehensively addressed + * by the new {@link Stream} library. Read the method documentation below for + * comparisons. This class is not being deprecated, but we gently encourage you to migrate to + * streams. + * + *

Performance notes: Unless otherwise noted, all of the iterables produced in this class + * are lazy, which means that their iterators only advance the backing iteration when + * absolutely necessary. + * + *

See the Guava User Guide article on {@code + * Iterables}. + * + * @author Kevin Bourrillion + * @author Jared Levy + * @since 2.0 + */ +public final class Iterables { + private Iterables() {} + + /** Returns an unmodifiable view of {@code iterable}. */ + public static Iterable unmodifiableIterable(final Iterable iterable) { + checkNotNull(iterable); + if (iterable instanceof UnmodifiableIterable || iterable instanceof ImmutableCollection) { + @SuppressWarnings({"unchecked", "rawtypes"}) // Since it's unmodifiable, the covariant cast is safe + Iterable result = (Iterable) iterable; + return result; + } + return new UnmodifiableIterable<>(iterable); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static Iterable unmodifiableIterable(ImmutableCollection iterable) { + return checkNotNull(iterable); + } + + private static final class UnmodifiableIterable extends FluentIterable { + private final Iterable iterable; + + private UnmodifiableIterable(Iterable iterable) { + this.iterable = iterable; + } + + @Override + public Iterator iterator() { + return Iterators.unmodifiableIterator(iterable.iterator()); + } + + @Override + public void forEach(Consumer action) { + iterable.forEach(action); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // safe upcast, assuming no one has a crazy Spliterator subclass + @Override + public Spliterator spliterator() { + return (Spliterator) iterable.spliterator(); + } + + @Override + public String toString() { + return iterable.toString(); + } + // no equals and hashCode; it would break the contract! + } + + /** Returns the number of elements in {@code iterable}. */ + public static int size(Iterable iterable) { + return (iterable instanceof Collection) + ? ((Collection) iterable).size() + : Iterators.size(iterable.iterator()); + } + + /** + * Returns {@code true} if {@code iterable} contains any element {@code o} for which {@code + * Objects.equals(o, element)} would return {@code true}. Otherwise returns {@code false}, even in + * cases where {@link Collection#contains} might throw {@link NullPointerException} or {@link + * ClassCastException}. + */ + public static boolean contains(Iterable iterable, Object element) { + if (iterable instanceof Collection) { + Collection collection = (Collection) iterable; + return Collections2.safeContains(collection, element); + } + return Iterators.contains(iterable.iterator(), element); + } + + /** + * Removes, from an iterable, every element that belongs to the provided collection. + * + *

This method calls {@link Collection#removeAll} if {@code iterable} is a collection, and + * {@link Iterators#removeAll} otherwise. + * + * @param removeFrom the iterable to (potentially) remove elements from + * @param elementsToRemove the elements to remove + * @return {@code true} if any element was removed from {@code iterable} + */ + public static boolean removeAll(Iterable removeFrom, Collection elementsToRemove) { + return (removeFrom instanceof Collection) + ? ((Collection) removeFrom).removeAll(checkNotNull(elementsToRemove)) + : Iterators.removeAll(removeFrom.iterator(), elementsToRemove); + } + + /** + * Removes, from an iterable, every element that does not belong to the provided collection. + * + *

This method calls {@link Collection#retainAll} if {@code iterable} is a collection, and + * {@link Iterators#retainAll} otherwise. + * + * @param removeFrom the iterable to (potentially) remove elements from + * @param elementsToRetain the elements to retain + * @return {@code true} if any element was removed from {@code iterable} + */ + public static boolean retainAll(Iterable removeFrom, Collection elementsToRetain) { + return (removeFrom instanceof Collection) + ? ((Collection) removeFrom).retainAll(checkNotNull(elementsToRetain)) + : Iterators.retainAll(removeFrom.iterator(), elementsToRetain); + } + + /** + * Removes, from an iterable, every element that satisfies the provided predicate. + * + *

Removals may or may not happen immediately as each element is tested against the predicate. + * The behavior of this method is not specified if {@code predicate} is dependent on {@code + * removeFrom}. + * + *

Java 8 users: if {@code removeFrom} is a {@link Collection}, use {@code + * removeFrom.removeIf(predicate)} instead. + * + * @param removeFrom the iterable to (potentially) remove elements from + * @param predicate a predicate that determines whether an element should be removed + * @return {@code true} if any elements were removed from the iterable + * @throws UnsupportedOperationException if the iterable does not support {@code remove()}. + * @since 2.0 + */ + public static boolean removeIf(Iterable removeFrom, Predicate predicate) { + if (removeFrom instanceof Collection) { + return ((Collection) removeFrom).removeIf(predicate); + } + return Iterators.removeIf(removeFrom.iterator(), predicate); + } + + /** Removes and returns the first matching element, or returns {@code null} if there is none. */ + + static T removeFirstMatching(Iterable removeFrom, Predicate predicate) { + checkNotNull(predicate); + Iterator iterator = removeFrom.iterator(); + while (iterator.hasNext()) { + T next = iterator.next(); + if (predicate.apply(next)) { + iterator.remove(); + return next; + } + } + return null; + } + + /** + * Determines whether two iterables contain equal elements in the same order. More specifically, + * this method returns {@code true} if {@code iterable1} and {@code iterable2} contain the same + * number of elements and every element of {@code iterable1} is equal to the corresponding element + * of {@code iterable2}. + */ + public static boolean elementsEqual(Iterable iterable1, Iterable iterable2) { + if (iterable1 instanceof Collection && iterable2 instanceof Collection) { + Collection collection1 = (Collection) iterable1; + Collection collection2 = (Collection) iterable2; + if (collection1.size() != collection2.size()) { + return false; + } + } + return Iterators.elementsEqual(iterable1.iterator(), iterable2.iterator()); + } + + /** + * Returns a string representation of {@code iterable}, with the format {@code [e1, e2, ..., en]} + * (that is, identical to {@link java.util.Arrays Arrays}{@code + * .toString(Iterables.toArray(iterable))}). Note that for most implementations of {@link + * Collection}, {@code collection.toString()} also gives the same result, but that behavior is not + * generally guaranteed. + */ + public static String toString(Iterable iterable) { + return Iterators.toString(iterable.iterator()); + } + + /** + * Returns the single element contained in {@code iterable}. + * + *

Java 8 users: the {@code Stream} equivalent to this method is {@code + * stream.collect(MoreCollectors.onlyElement())}. + * + * @throws NoSuchElementException if the iterable is empty + * @throws IllegalArgumentException if the iterable contains multiple elements + */ + public static T getOnlyElement(Iterable iterable) { + return Iterators.getOnlyElement(iterable.iterator()); + } + + /** + * Returns the single element contained in {@code iterable}, or {@code defaultValue} if the + * iterable is empty. + * + *

Java 8 users: the {@code Stream} equivalent to this method is {@code + * stream.collect(MoreCollectors.toOptional()).orElse(defaultValue)}. + * + * @throws IllegalArgumentException if the iterator contains multiple elements + */ + + public static T getOnlyElement(Iterable iterable, T defaultValue) { + return Iterators.getOnlyElement(iterable.iterator(), defaultValue); + } + + /** + * Copies an iterable's elements into an array. + * + * @param iterable the iterable to copy + * @param type the type of the elements + * @return a newly-allocated array into which all the elements of the iterable have been copied + */ + public static T[] toArray(Iterable iterable, Class type) { + return toArray(iterable, ObjectArrays.newArray(type, 0)); + } + + static T[] toArray(Iterable iterable, T[] array) { + Collection collection = castOrCopyToCollection(iterable); + return collection.toArray(array); + } + + /** + * Copies an iterable's elements into an array. + * + * @param iterable the iterable to copy + * @return a newly-allocated array into which all the elements of the iterable have been copied + */ + static Object[] toArray(Iterable iterable) { + return castOrCopyToCollection(iterable).toArray(); + } + + /** + * Converts an iterable into a collection. If the iterable is already a collection, it is + * returned. Otherwise, an {@link java.util.ArrayList} is created with the contents of the + * iterable in the same iteration order. + */ + private static Collection castOrCopyToCollection(Iterable iterable) { + return (iterable instanceof Collection) + ? (Collection) iterable + : Lists.newArrayList(iterable.iterator()); + } + + /** + * Adds all elements in {@code iterable} to {@code collection}. + * + * @return {@code true} if {@code collection} was modified as a result of this operation. + */ + public static boolean addAll(Collection addTo, Iterable elementsToAdd) { + if (elementsToAdd instanceof Collection) { + Collection c = Collections2.cast(elementsToAdd); + return addTo.addAll(c); + } + return Iterators.addAll(addTo, checkNotNull(elementsToAdd).iterator()); + } + + /** + * Returns the number of elements in the specified iterable that equal the specified object. This + * implementation avoids a full iteration when the iterable is a {@link Multiset} or {@link Set}. + * + *

Java 8 users: In most cases, the {@code Stream} equivalent of this method is {@code + * stream.filter(element::equals).count()}. If {@code element} might be null, use {@code + * stream.filter(Predicate.isEqual(element)).count()} instead. + * + * @see java.util.Collections#frequency(Collection, Object) Collections.frequency(Collection, + * Object) + */ + public static int frequency(Iterable iterable, Object element) { + if ((iterable instanceof Multiset)) { + return ((Multiset) iterable).count(element); + } else if ((iterable instanceof Set)) { + return ((Set) iterable).contains(element) ? 1 : 0; + } + return Iterators.frequency(iterable.iterator(), element); + } + + /** + * Returns an iterable whose iterators cycle indefinitely over the elements of {@code iterable}. + * + *

That iterator supports {@code remove()} if {@code iterable.iterator()} does. After {@code + * remove()} is called, subsequent cycles omit the removed element, which is no longer in {@code + * iterable}. The iterator's {@code hasNext()} method returns {@code true} until {@code iterable} + * is empty. + * + *

Warning: Typical uses of the resulting iterator may produce an infinite loop. You + * should use an explicit {@code break} or be certain that you will eventually remove all the + * elements. + * + *

To cycle over the iterable {@code n} times, use the following: {@code + * Iterables.concat(Collections.nCopies(n, iterable))} + * + *

Java 8 users: The {@code Stream} equivalent of this method is {@code + * Stream.generate(() -> iterable).flatMap(Streams::stream)}. + */ + public static Iterable cycle(final Iterable iterable) { + checkNotNull(iterable); + return new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.cycle(iterable); + } + + @Override + public Spliterator spliterator() { + return Stream.generate(() -> iterable).flatMap(Streams::stream).spliterator(); + } + + @Override + public String toString() { + return iterable.toString() + " (cycled)"; + } + }; + } + + /** + * Returns an iterable whose iterators cycle indefinitely over the provided elements. + * + *

After {@code remove} is invoked on a generated iterator, the removed element will no longer + * appear in either that iterator or any other iterator created from the same source iterable. + * That is, this method behaves exactly as {@code Iterables.cycle(Lists.newArrayList(elements))}. + * The iterator's {@code hasNext} method returns {@code true} until all of the original elements + * have been removed. + * + *

Warning: Typical uses of the resulting iterator may produce an infinite loop. You + * should use an explicit {@code break} or be certain that you will eventually remove all the + * elements. + * + *

To cycle over the elements {@code n} times, use the following: {@code + * Iterables.concat(Collections.nCopies(n, Arrays.asList(elements)))} + * + *

Java 8 users: If passing a single element {@code e}, the {@code Stream} equivalent of + * this method is {@code Stream.generate(() -> e)}. Otherwise, put the elements in a collection + * and use {@code Stream.generate(() -> collection).flatMap(Collection::stream)}. + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static Iterable cycle(T... elements) { + return cycle(Lists.newArrayList(elements)); + } + + /** + * Combines two iterables into a single iterable. The returned iterable has an iterator that + * traverses the elements in {@code a}, followed by the elements in {@code b}. The source + * iterators are not polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

Java 8 users: The {@code Stream} equivalent of this method is {@code Stream.concat(a, + * b)}. + */ + public static Iterable concat(Iterable a, Iterable b) { + return FluentIterable.concat(a, b); + } + + /** + * Combines three iterables into a single iterable. The returned iterable has an iterator that + * traverses the elements in {@code a}, followed by the elements in {@code b}, followed by the + * elements in {@code c}. The source iterators are not polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

Java 8 users: The {@code Stream} equivalent of this method is {@code + * Streams.concat(a, b, c)}. + */ + public static Iterable concat( + Iterable a, Iterable b, Iterable c) { + return FluentIterable.concat(a, b, c); + } + + /** + * Combines four iterables into a single iterable. The returned iterable has an iterator that + * traverses the elements in {@code a}, followed by the elements in {@code b}, followed by the + * elements in {@code c}, followed by the elements in {@code d}. The source iterators are not + * polled until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

Java 8 users: The {@code Stream} equivalent of this method is {@code + * Streams.concat(a, b, c, d)}. + */ + public static Iterable concat( + Iterable a, + Iterable b, + Iterable c, + Iterable d) { + return FluentIterable.concat(a, b, c, d); + } + + /** + * Combines multiple iterables into a single iterable. The returned iterable has an iterator that + * traverses the elements of each iterable in {@code inputs}. The input iterators are not polled + * until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. + * + *

Java 8 users: The {@code Stream} equivalent of this method is {@code + * Streams.concat(...)}. + * + * @throws NullPointerException if any of the provided iterables is null + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static Iterable concat(Iterable... inputs) { + return FluentIterable.concat(inputs); + } + + /** + * Combines multiple iterables into a single iterable. The returned iterable has an iterator that + * traverses the elements of each iterable in {@code inputs}. The input iterators are not polled + * until necessary. + * + *

The returned iterable's iterator supports {@code remove()} when the corresponding input + * iterator supports it. The methods of the returned iterable may throw {@code + * NullPointerException} if any of the input iterators is null. + * + *

Java 8 users: The {@code Stream} equivalent of this method is {@code + * streamOfStreams.flatMap(s -> s)}. + */ + public static Iterable concat(Iterable> inputs) { + return FluentIterable.concat(inputs); + } + + /** + * Divides an iterable into unmodifiable sublists of the given size (the final iterable may be + * smaller). For example, partitioning an iterable containing {@code [a, b, c, d, e]} with a + * partition size of 3 yields {@code [[a, b, c], [d, e]]} -- an outer iterable containing two + * inner lists of three and two elements, all in the original order. + * + *

Iterators returned by the returned iterable do not support the {@link Iterator#remove()} + * method. The returned lists implement {@link RandomAccess}, whether or not the input list does. + * + *

Note: if {@code iterable} is a {@link List}, use {@link Lists#partition(List, int)} + * instead. + * + * @param iterable the iterable to return a partitioned view of + * @param size the desired size of each partition (the last may be smaller) + * @return an iterable of unmodifiable lists containing the elements of {@code iterable} divided + * into partitions + * @throws IllegalArgumentException if {@code size} is nonpositive + */ + public static Iterable> partition(final Iterable iterable, final int size) { + checkNotNull(iterable); + checkArgument(size > 0); + return new FluentIterable>() { + @Override + public Iterator> iterator() { + return Iterators.partition(iterable.iterator(), size); + } + }; + } + + /** + * Divides an iterable into unmodifiable sublists of the given size, padding the final iterable + * with null values if necessary. For example, partitioning an iterable containing {@code [a, b, + * c, d, e]} with a partition size of 3 yields {@code [[a, b, c], [d, e, null]]} -- an outer + * iterable containing two inner lists of three elements each, all in the original order. + * + *

Iterators returned by the returned iterable do not support the {@link Iterator#remove()} + * method. + * + * @param iterable the iterable to return a partitioned view of + * @param size the desired size of each partition + * @return an iterable of unmodifiable lists containing the elements of {@code iterable} divided + * into partitions (the final iterable may have trailing null elements) + * @throws IllegalArgumentException if {@code size} is nonpositive + */ + public static Iterable> paddedPartition(final Iterable iterable, final int size) { + checkNotNull(iterable); + checkArgument(size > 0); + return new FluentIterable>() { + @Override + public Iterator> iterator() { + return Iterators.paddedPartition(iterable.iterator(), size); + } + }; + } + + /** + * Returns a view of {@code unfiltered} containing all elements that satisfy the input predicate + * {@code retainIfTrue}. The returned iterable's iterator does not support {@code remove()}. + * + *

{@code Stream} equivalent: {@link Stream#filter}. + */ + public static Iterable filter( + final Iterable unfiltered, final Predicate retainIfTrue) { + checkNotNull(unfiltered); + checkNotNull(retainIfTrue); + return new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.filter(unfiltered.iterator(), retainIfTrue); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + unfiltered.forEach( + (T a) -> { + if (retainIfTrue.test(a)) { + action.accept(a); + } + }); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.filter(unfiltered.spliterator(), retainIfTrue); + } + }; + } + + /** + * Returns a view of {@code unfiltered} containing all elements that are of the type {@code + * desiredType}. The returned iterable's iterator does not support {@code remove()}. + * + *

{@code Stream} equivalent: {@code stream.filter(type::isInstance).map(type::cast)}. + * This does perform a little more work than necessary, so another option is to insert an + * unchecked cast at some later point: + * + *

+   * {@code @SuppressWarnings({"unchecked", "rawtypes"}) // safe because of ::isInstance check
+   * ImmutableList result =
+   *     (ImmutableList) stream.filter(NewType.class::isInstance).collect(toImmutableList());}
+   * 
+ */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Iterable filter(final Iterable unfiltered, final Class desiredType) { + checkNotNull(unfiltered); + checkNotNull(desiredType); + return (Iterable) filter(unfiltered, Predicates.instanceOf(desiredType)); + } + + /** + * Returns {@code true} if any element in {@code iterable} satisfies the predicate. + * + *

{@code Stream} equivalent: {@link Stream#anyMatch}. + */ + public static boolean any(Iterable iterable, Predicate predicate) { + return Iterators.any(iterable.iterator(), predicate); + } + + /** + * Returns {@code true} if every element in {@code iterable} satisfies the predicate. If {@code + * iterable} is empty, {@code true} is returned. + * + *

{@code Stream} equivalent: {@link Stream#allMatch}. + */ + public static boolean all(Iterable iterable, Predicate predicate) { + return Iterators.all(iterable.iterator(), predicate); + } + + /** + * Returns the first element in {@code iterable} that satisfies the given predicate; use this + * method only when such an element is known to exist. If it is possible that no element + * will match, use {@link #tryFind} or {@link #find(Iterable, Predicate, Object)} instead. + * + *

{@code Stream} equivalent: {@code stream.filter(predicate).findFirst().get()} + * + * @throws NoSuchElementException if no element in {@code iterable} matches the given predicate + */ + public static T find(Iterable iterable, Predicate predicate) { + return Iterators.find(iterable.iterator(), predicate); + } + + /** + * Returns the first element in {@code iterable} that satisfies the given predicate, or {@code + * defaultValue} if none found. Note that this can usually be handled more naturally using {@code + * tryFind(iterable, predicate).or(defaultValue)}. + * + *

{@code Stream} equivalent: {@code + * stream.filter(predicate).findFirst().orElse(defaultValue)} + * + * @since 7.0 + */ + + public static T find( + Iterable iterable, + Predicate predicate, + T defaultValue) { + return Iterators.find(iterable.iterator(), predicate, defaultValue); + } + + /** + * Returns an {@link Optional} containing the first element in {@code iterable} that satisfies the + * given predicate, if such an element exists. + * + *

Warning: avoid using a {@code predicate} that matches {@code null}. If {@code null} + * is matched in {@code iterable}, a NullPointerException will be thrown. + * + *

{@code Stream} equivalent: {@code stream.filter(predicate).findFirst()} + * + * @since 11.0 + */ + public static Optional tryFind(Iterable iterable, Predicate predicate) { + return Iterators.tryFind(iterable.iterator(), predicate); + } + + /** + * Returns the index in {@code iterable} of the first element that satisfies the provided {@code + * predicate}, or {@code -1} if the Iterable has no such elements. + * + *

More formally, returns the lowest index {@code i} such that {@code + * predicate.apply(Iterables.get(iterable, i))} returns {@code true}, or {@code -1} if there is no + * such index. + * + * @since 2.0 + */ + public static int indexOf(Iterable iterable, Predicate predicate) { + return Iterators.indexOf(iterable.iterator(), predicate); + } + + /** + * Returns a view containing the result of applying {@code function} to each element of {@code + * fromIterable}. + * + *

The returned iterable's iterator supports {@code remove()} if {@code fromIterable}'s + * iterator does. After a successful {@code remove()} call, {@code fromIterable} no longer + * contains the corresponding element. + * + *

If the input {@code Iterable} is known to be a {@code List} or other {@code Collection}, + * consider {@link Lists#transform} and {@link Collections2#transform}. + * + *

{@code Stream} equivalent: {@link Stream#map} + */ + public static Iterable transform( + final Iterable fromIterable, final Function function) { + checkNotNull(fromIterable); + checkNotNull(function); + return new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.transform(fromIterable.iterator(), function); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + fromIterable.forEach((F f) -> action.accept(function.apply(f))); + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.map(fromIterable.spliterator(), function); + } + }; + } + + /** + * Returns the element at the specified position in an iterable. + * + *

{@code Stream} equivalent: {@code stream.skip(position).findFirst().get()} (throws + * {@code NoSuchElementException} if out of bounds) + * + * @param position position of the element to return + * @return the element at the specified position in {@code iterable} + * @throws IndexOutOfBoundsException if {@code position} is negative or greater than or equal to + * the size of {@code iterable} + */ + public static T get(Iterable iterable, int position) { + checkNotNull(iterable); + return (iterable instanceof List) + ? ((List) iterable).get(position) + : Iterators.get(iterable.iterator(), position); + } + + /** + * Returns the element at the specified position in an iterable or a default value otherwise. + * + *

{@code Stream} equivalent: {@code + * stream.skip(position).findFirst().orElse(defaultValue)} (returns the default value if the index + * is out of bounds) + * + * @param position position of the element to return + * @param defaultValue the default value to return if {@code position} is greater than or equal to + * the size of the iterable + * @return the element at the specified position in {@code iterable} or {@code defaultValue} if + * {@code iterable} contains fewer than {@code position + 1} elements. + * @throws IndexOutOfBoundsException if {@code position} is negative + * @since 4.0 + */ + + public static T get( + Iterable iterable, int position, T defaultValue) { + checkNotNull(iterable); + Iterators.checkNonnegative(position); + if (iterable instanceof List) { + List list = Lists.cast(iterable); + return (position < list.size()) ? list.get(position) : defaultValue; + } else { + Iterator iterator = iterable.iterator(); + Iterators.advance(iterator, position); + return Iterators.getNext(iterator, defaultValue); + } + } + + /** + * Returns the first element in {@code iterable} or {@code defaultValue} if the iterable is empty. + * The {@link Iterators} analog to this method is {@link Iterators#getNext}. + * + *

If no default value is desired (and the caller instead wants a {@link + * NoSuchElementException} to be thrown), it is recommended that {@code + * iterable.iterator().next()} is used instead. + * + *

To get the only element in a single-element {@code Iterable}, consider using {@link + * #getOnlyElement(Iterable)} or {@link #getOnlyElement(Iterable, Object)} instead. + * + *

{@code Stream} equivalent: {@code stream.findFirst().orElse(defaultValue)} + * + * @param defaultValue the default value to return if the iterable is empty + * @return the first element of {@code iterable} or the default value + * @since 7.0 + */ + + public static T getFirst(Iterable iterable, T defaultValue) { + return Iterators.getNext(iterable.iterator(), defaultValue); + } + + /** + * Returns the last element of {@code iterable}. If {@code iterable} is a {@link List} with {@link + * RandomAccess} support, then this operation is guaranteed to be {@code O(1)}. + * + *

{@code Stream} equivalent: {@link Streams#findLast Streams.findLast(stream).get()} + * + * @return the last element of {@code iterable} + * @throws NoSuchElementException if the iterable is empty + */ + public static T getLast(Iterable iterable) { + // TODO(kevinb): Support a concurrently modified collection? + if (iterable instanceof List) { + List list = (List) iterable; + if (list.isEmpty()) { + throw new NoSuchElementException(); + } + return getLastInNonemptyList(list); + } + + return Iterators.getLast(iterable.iterator()); + } + + /** + * Returns the last element of {@code iterable} or {@code defaultValue} if the iterable is empty. + * If {@code iterable} is a {@link List} with {@link RandomAccess} support, then this operation is + * guaranteed to be {@code O(1)}. + * + *

{@code Stream} equivalent: {@code Streams.findLast(stream).orElse(defaultValue)} + * + * @param defaultValue the value to return if {@code iterable} is empty + * @return the last element of {@code iterable} or the default value + * @since 3.0 + */ + + public static T getLast(Iterable iterable, T defaultValue) { + if (iterable instanceof Collection) { + Collection c = Collections2.cast(iterable); + if (c.isEmpty()) { + return defaultValue; + } else if (iterable instanceof List) { + return getLastInNonemptyList(Lists.cast(iterable)); + } + } + + return Iterators.getLast(iterable.iterator(), defaultValue); + } + + private static T getLastInNonemptyList(List list) { + return list.get(list.size() - 1); + } + + /** + * Returns a view of {@code iterable} that skips its first {@code numberToSkip} elements. If + * {@code iterable} contains fewer than {@code numberToSkip} elements, the returned iterable skips + * all of its elements. + * + *

Modifications to the underlying {@link Iterable} before a call to {@code iterator()} are + * reflected in the returned iterator. That is, the iterator skips the first {@code numberToSkip} + * elements that exist when the {@code Iterator} is created, not when {@code skip()} is called. + * + *

The returned iterable's iterator supports {@code remove()} if the iterator of the underlying + * iterable supports it. Note that it is not possible to delete the last skipped element by + * immediately calling {@code remove()} on that iterator, as the {@code Iterator} contract states + * that a call to {@code remove()} before a call to {@code next()} will throw an {@link + * IllegalStateException}. + * + *

{@code Stream} equivalent: {@link Stream#skip} + * + * @since 3.0 + */ + public static Iterable skip(final Iterable iterable, final int numberToSkip) { + checkNotNull(iterable); + checkArgument(numberToSkip >= 0, "number to skip cannot be negative"); + + return new FluentIterable() { + @Override + public Iterator iterator() { + if (iterable instanceof List) { + final List list = (List) iterable; + int toSkip = Math.min(list.size(), numberToSkip); + return list.subList(toSkip, list.size()).iterator(); + } + final Iterator iterator = iterable.iterator(); + + Iterators.advance(iterator, numberToSkip); + + /* + * We can't just return the iterator because an immediate call to its + * remove() method would remove one of the skipped elements instead of + * throwing an IllegalStateException. + */ + return new Iterator() { + boolean atStart = true; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public T next() { + T result = iterator.next(); + atStart = false; // not called if next() fails + return result; + } + + @Override + public void remove() { + checkRemove(!atStart); + iterator.remove(); + } + }; + } + + @Override + public Spliterator spliterator() { + if (iterable instanceof List) { + final List list = (List) iterable; + int toSkip = Math.min(list.size(), numberToSkip); + return list.subList(toSkip, list.size()).spliterator(); + } else { + return Streams.stream(iterable).skip(numberToSkip).spliterator(); + } + } + }; + } + + /** + * Returns a view of {@code iterable} containing its first {@code limitSize} elements. If {@code + * iterable} contains fewer than {@code limitSize} elements, the returned view contains all of its + * elements. The returned iterable's iterator supports {@code remove()} if {@code iterable}'s + * iterator does. + * + *

{@code Stream} equivalent: {@link Stream#limit} + * + * @param iterable the iterable to limit + * @param limitSize the maximum number of elements in the returned iterable + * @throws IllegalArgumentException if {@code limitSize} is negative + * @since 3.0 + */ + public static Iterable limit(final Iterable iterable, final int limitSize) { + checkNotNull(iterable); + checkArgument(limitSize >= 0, "limit is negative"); + return new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.limit(iterable.iterator(), limitSize); + } + + @Override + public Spliterator spliterator() { + return Streams.stream(iterable).limit(limitSize).spliterator(); + } + }; + } + + /** + * Returns a view of the supplied iterable that wraps each generated {@link Iterator} through + * {@link Iterators#consumingIterator(Iterator)}. + * + *

Note: If {@code iterable} is a {@link Queue}, the returned iterable will get entries from + * {@link Queue#remove()} since {@link Queue}'s iteration order is undefined. Calling {@link + * Iterator#hasNext()} on a generated iterator from the returned iterable may cause an item to be + * immediately dequeued for return on a subsequent call to {@link Iterator#next()}. + * + * @param iterable the iterable to wrap + * @return a view of the supplied iterable that wraps each generated iterator through {@link + * Iterators#consumingIterator(Iterator)}; for queues, an iterable that generates iterators + * that return and consume the queue's elements in queue order + * @see Iterators#consumingIterator(Iterator) + * @since 2.0 + */ + public static Iterable consumingIterable(final Iterable iterable) { + checkNotNull(iterable); + + return new FluentIterable() { + @Override + public Iterator iterator() { + return (iterable instanceof Queue) + ? new ConsumingQueueIterator<>((Queue) iterable) + : Iterators.consumingIterator(iterable.iterator()); + } + + @Override + public String toString() { + return "Iterables.consumingIterable(...)"; + } + }; + } + + // Methods only in Iterables, not in Iterators + + /** + * Determines if the given iterable contains no elements. + * + *

There is no precise {@link Iterator} equivalent to this method, since one can only ask an + * iterator whether it has any elements remaining (which one does using {@link + * Iterator#hasNext}). + * + *

{@code Stream} equivalent: {@code !stream.findAny().isPresent()} + * + * @return {@code true} if the iterable contains no elements + */ + public static boolean isEmpty(Iterable iterable) { + if (iterable instanceof Collection) { + return ((Collection) iterable).isEmpty(); + } + return !iterable.iterator().hasNext(); + } + + /** + * Returns an iterable over the merged contents of all given {@code iterables}. Equivalent entries + * will not be de-duplicated. + * + *

Callers must ensure that the source {@code iterables} are in non-descending order as this + * method does not sort its input. + * + *

For any equivalent elements across all {@code iterables}, it is undefined which element is + * returned first. + * + * @since 11.0 + */ + + public static Iterable mergeSorted( + final Iterable> iterables, + final Comparator comparator) { + checkNotNull(iterables, "iterables"); + checkNotNull(comparator, "comparator"); + Iterable iterable = + new FluentIterable() { + @Override + public Iterator iterator() { + return Iterators.mergeSorted( + Iterables.transform(iterables, Iterables.toIterator()), comparator); + } + }; + return new UnmodifiableIterable<>(iterable); + } + + // TODO(user): Is this the best place for this? Move to fluent functions? + // Useful as a public method? + static Function, Iterator> toIterator() { + return new Function, Iterator>() { + @Override + public Iterator apply(Iterable iterable) { + return iterable.iterator(); + } + }; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterators.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterators.java new file mode 100644 index 0000000000000..8f2e5f00f1937 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Iterators.java @@ -0,0 +1,1388 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.instanceOf; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Optional; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.util.ArrayDeque; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Deque; +import java.util.Enumeration; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; +import java.util.Queue; + + +/** + * This class contains static utility methods that operate on or return objects of type {@link + * Iterator}. Except as noted, each method has a corresponding {@link Iterable}-based method in the + * {@link Iterables} class. + * + *

Performance notes: Unless otherwise noted, all of the iterators produced in this class + * are lazy, which means that they only advance the backing iteration when absolutely + * necessary. + * + *

See the Guava User Guide section on {@code + * Iterators}. + * + * @author Kevin Bourrillion + * @author Jared Levy + * @since 2.0 + */ +public final class Iterators { + private Iterators() {} + + /** + * Returns the empty iterator. + * + *

The {@link Iterable} equivalent of this method is {@link ImmutableSet#of()}. + */ + static UnmodifiableIterator emptyIterator() { + return emptyListIterator(); + } + + /** + * Returns the empty iterator. + * + *

The {@link Iterable} equivalent of this method is {@link ImmutableSet#of()}. + */ + // Casting to any type is safe since there are no actual elements. + @SuppressWarnings({"unchecked", "rawtypes"}) + static UnmodifiableListIterator emptyListIterator() { + return (UnmodifiableListIterator) ArrayItr.EMPTY; + } + + /** + * This is an enum singleton rather than an anonymous class so ProGuard can figure out it's only + * referenced by emptyModifiableIterator(). + */ + private enum EmptyModifiableIterator implements Iterator { + INSTANCE; + + @Override + public boolean hasNext() { + return false; + } + + @Override + public Object next() { + throw new NoSuchElementException(); + } + + @Override + public void remove() { + checkRemove(false); + } + } + + /** + * Returns the empty {@code Iterator} that throws {@link IllegalStateException} instead of {@link + * UnsupportedOperationException} on a call to {@link Iterator#remove()}. + */ + // Casting to any type is safe since there are no actual elements. + @SuppressWarnings({"unchecked", "rawtypes"}) + static Iterator emptyModifiableIterator() { + return (Iterator) EmptyModifiableIterator.INSTANCE; + } + + /** Returns an unmodifiable view of {@code iterator}. */ + public static UnmodifiableIterator unmodifiableIterator( + final Iterator iterator) { + checkNotNull(iterator); + if (iterator instanceof UnmodifiableIterator) { + @SuppressWarnings({"unchecked", "rawtypes"}) // Since it's unmodifiable, the covariant cast is safe + UnmodifiableIterator result = (UnmodifiableIterator) iterator; + return result; + } + return new UnmodifiableIterator() { + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public T next() { + return iterator.next(); + } + }; + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static UnmodifiableIterator unmodifiableIterator(UnmodifiableIterator iterator) { + return checkNotNull(iterator); + } + + /** + * Returns the number of elements remaining in {@code iterator}. The iterator will be left + * exhausted: its {@code hasNext()} method will return {@code false}. + */ + public static int size(Iterator iterator) { + long count = 0L; + while (iterator.hasNext()) { + iterator.next(); + count++; + } + return Ints.saturatedCast(count); + } + + /** Returns {@code true} if {@code iterator} contains {@code element}. */ + public static boolean contains(Iterator iterator, Object element) { + if (element == null) { + while (iterator.hasNext()) { + if (iterator.next() == null) { + return true; + } + } + } else { + while (iterator.hasNext()) { + if (element.equals(iterator.next())) { + return true; + } + } + } + return false; + } + + /** + * Traverses an iterator and removes every element that belongs to the provided collection. The + * iterator will be left exhausted: its {@code hasNext()} method will return {@code false}. + * + * @param removeFrom the iterator to (potentially) remove elements from + * @param elementsToRemove the elements to remove + * @return {@code true} if any element was removed from {@code iterator} + */ + public static boolean removeAll(Iterator removeFrom, Collection elementsToRemove) { + checkNotNull(elementsToRemove); + boolean result = false; + while (removeFrom.hasNext()) { + if (elementsToRemove.contains(removeFrom.next())) { + removeFrom.remove(); + result = true; + } + } + return result; + } + + /** + * Removes every element that satisfies the provided predicate from the iterator. The iterator + * will be left exhausted: its {@code hasNext()} method will return {@code false}. + * + * @param removeFrom the iterator to (potentially) remove elements from + * @param predicate a predicate that determines whether an element should be removed + * @return {@code true} if any elements were removed from the iterator + * @since 2.0 + */ + public static boolean removeIf(Iterator removeFrom, Predicate predicate) { + checkNotNull(predicate); + boolean modified = false; + while (removeFrom.hasNext()) { + if (predicate.apply(removeFrom.next())) { + removeFrom.remove(); + modified = true; + } + } + return modified; + } + + /** + * Traverses an iterator and removes every element that does not belong to the provided + * collection. The iterator will be left exhausted: its {@code hasNext()} method will return + * {@code false}. + * + * @param removeFrom the iterator to (potentially) remove elements from + * @param elementsToRetain the elements to retain + * @return {@code true} if any element was removed from {@code iterator} + */ + public static boolean retainAll(Iterator removeFrom, Collection elementsToRetain) { + checkNotNull(elementsToRetain); + boolean result = false; + while (removeFrom.hasNext()) { + if (!elementsToRetain.contains(removeFrom.next())) { + removeFrom.remove(); + result = true; + } + } + return result; + } + + /** + * Determines whether two iterators contain equal elements in the same order. More specifically, + * this method returns {@code true} if {@code iterator1} and {@code iterator2} contain the same + * number of elements and every element of {@code iterator1} is equal to the corresponding element + * of {@code iterator2}. + * + *

Note that this will modify the supplied iterators, since they will have been advanced some + * number of elements forward. + */ + public static boolean elementsEqual(Iterator iterator1, Iterator iterator2) { + while (iterator1.hasNext()) { + if (!iterator2.hasNext()) { + return false; + } + Object o1 = iterator1.next(); + Object o2 = iterator2.next(); + if (!Objects.equal(o1, o2)) { + return false; + } + } + return !iterator2.hasNext(); + } + + /** + * Returns a string representation of {@code iterator}, with the format {@code [e1, e2, ..., en]}. + * The iterator will be left exhausted: its {@code hasNext()} method will return {@code false}. + */ + public static String toString(Iterator iterator) { + StringBuilder sb = new StringBuilder().append('['); + boolean first = true; + while (iterator.hasNext()) { + if (!first) { + sb.append(", "); + } + first = false; + sb.append(iterator.next()); + } + return sb.append(']').toString(); + } + + /** + * Returns the single element contained in {@code iterator}. + * + * @throws NoSuchElementException if the iterator is empty + * @throws IllegalArgumentException if the iterator contains multiple elements. The state of the + * iterator is unspecified. + */ + public static T getOnlyElement(Iterator iterator) { + T first = iterator.next(); + if (!iterator.hasNext()) { + return first; + } + + StringBuilder sb = new StringBuilder().append("expected one element but was: <").append(first); + for (int i = 0; i < 4 && iterator.hasNext(); i++) { + sb.append(", ").append(iterator.next()); + } + if (iterator.hasNext()) { + sb.append(", ..."); + } + sb.append('>'); + + throw new IllegalArgumentException(sb.toString()); + } + + /** + * Returns the single element contained in {@code iterator}, or {@code defaultValue} if the + * iterator is empty. + * + * @throws IllegalArgumentException if the iterator contains multiple elements. The state of the + * iterator is unspecified. + */ + + public static T getOnlyElement(Iterator iterator, T defaultValue) { + return iterator.hasNext() ? getOnlyElement(iterator) : defaultValue; + } + + /** + * Copies an iterator's elements into an array. The iterator will be left exhausted: its {@code + * hasNext()} method will return {@code false}. + * + * @param iterator the iterator to copy + * @param type the type of the elements + * @return a newly-allocated array into which all the elements of the iterator have been copied + */ + public static T[] toArray(Iterator iterator, Class type) { + List list = Lists.newArrayList(iterator); + return Iterables.toArray(list, type); + } + + /** + * Adds all elements in {@code iterator} to {@code collection}. The iterator will be left + * exhausted: its {@code hasNext()} method will return {@code false}. + * + * @return {@code true} if {@code collection} was modified as a result of this operation + */ + public static boolean addAll(Collection addTo, Iterator iterator) { + checkNotNull(addTo); + checkNotNull(iterator); + boolean wasModified = false; + while (iterator.hasNext()) { + wasModified |= addTo.add(iterator.next()); + } + return wasModified; + } + + /** + * Returns the number of elements in the specified iterator that equal the specified object. The + * iterator will be left exhausted: its {@code hasNext()} method will return {@code false}. + * + * @see Collections#frequency + */ + public static int frequency(Iterator iterator, Object element) { + int count = 0; + while (contains(iterator, element)) { + // Since it lives in the same class, we know contains gets to the element and then stops, + // though that isn't currently publicly documented. + count++; + } + return count; + } + + /** + * Returns an iterator that cycles indefinitely over the elements of {@code iterable}. + * + *

The returned iterator supports {@code remove()} if the provided iterator does. After {@code + * remove()} is called, subsequent cycles omit the removed element, which is no longer in {@code + * iterable}. The iterator's {@code hasNext()} method returns {@code true} until {@code iterable} + * is empty. + * + *

Warning: Typical uses of the resulting iterator may produce an infinite loop. You + * should use an explicit {@code break} or be certain that you will eventually remove all the + * elements. + */ + public static Iterator cycle(final Iterable iterable) { + checkNotNull(iterable); + return new Iterator() { + Iterator iterator = emptyModifiableIterator(); + + @Override + public boolean hasNext() { + /* + * Don't store a new Iterator until we know the user can't remove() the last returned + * element anymore. Otherwise, when we remove from the old iterator, we may be invalidating + * the new one. The result is a ConcurrentModificationException or other bad behavior. + * + * (If we decide that we really, really hate allocating two Iterators per cycle instead of + * one, we can optimistically store the new Iterator and then be willing to throw it out if + * the user calls remove().) + */ + return iterator.hasNext() || iterable.iterator().hasNext(); + } + + @Override + public T next() { + if (!iterator.hasNext()) { + iterator = iterable.iterator(); + if (!iterator.hasNext()) { + throw new NoSuchElementException(); + } + } + return iterator.next(); + } + + @Override + public void remove() { + iterator.remove(); + } + }; + } + + /** + * Returns an iterator that cycles indefinitely over the provided elements. + * + *

The returned iterator supports {@code remove()}. After {@code remove()} is called, + * subsequent cycles omit the removed element, but {@code elements} does not change. The + * iterator's {@code hasNext()} method returns {@code true} until all of the original elements + * have been removed. + * + *

Warning: Typical uses of the resulting iterator may produce an infinite loop. You + * should use an explicit {@code break} or be certain that you will eventually remove all the + * elements. + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static Iterator cycle(T... elements) { + return cycle(Lists.newArrayList(elements)); + } + + /** + * Returns an Iterator that walks the specified array, nulling out elements behind it. This can + * avoid memory leaks when an element is no longer necessary. + * + *

This is mainly just to avoid the intermediate ArrayDeque in ConsumingQueueIterator. + */ + @SuppressWarnings("unchecked") + private static Iterator consumingForArray(final T... elements) { + return new UnmodifiableIterator() { + int index = 0; + + @Override + public boolean hasNext() { + return index < elements.length; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + T result = elements[index]; + elements[index] = null; + index++; + return result; + } + }; + } + + /** + * Combines two iterators into a single iterator. The returned iterator iterates across the + * elements in {@code a}, followed by the elements in {@code b}. The source iterators are not + * polled until necessary. + * + *

The returned iterator supports {@code remove()} when the corresponding input iterator + * supports it. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Iterator concat(Iterator a, Iterator b) { + checkNotNull(a); + checkNotNull(b); + return concat(consumingForArray(a, b)); + } + + /** + * Combines three iterators into a single iterator. The returned iterator iterates across the + * elements in {@code a}, followed by the elements in {@code b}, followed by the elements in + * {@code c}. The source iterators are not polled until necessary. + * + *

The returned iterator supports {@code remove()} when the corresponding input iterator + * supports it. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Iterator concat( + Iterator a, Iterator b, Iterator c) { + checkNotNull(a); + checkNotNull(b); + checkNotNull(c); + return concat(consumingForArray(a, b, c)); + } + + /** + * Combines four iterators into a single iterator. The returned iterator iterates across the + * elements in {@code a}, followed by the elements in {@code b}, followed by the elements in + * {@code c}, followed by the elements in {@code d}. The source iterators are not polled until + * necessary. + * + *

The returned iterator supports {@code remove()} when the corresponding input iterator + * supports it. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Iterator concat( + Iterator a, + Iterator b, + Iterator c, + Iterator d) { + checkNotNull(a); + checkNotNull(b); + checkNotNull(c); + checkNotNull(d); + return concat(consumingForArray(a, b, c, d)); + } + + /** + * Combines multiple iterators into a single iterator. The returned iterator iterates across the + * elements of each iterator in {@code inputs}. The input iterators are not polled until + * necessary. + * + *

The returned iterator supports {@code remove()} when the corresponding input iterator + * supports it. + * + * @throws NullPointerException if any of the provided iterators is null + */ + @SuppressWarnings("unchecked") + public static Iterator concat(Iterator... inputs) { + return concatNoDefensiveCopy(Arrays.copyOf(inputs, inputs.length)); + } + + /** + * Combines multiple iterators into a single iterator. The returned iterator iterates across the + * elements of each iterator in {@code inputs}. The input iterators are not polled until + * necessary. + * + *

The returned iterator supports {@code remove()} when the corresponding input iterator + * supports it. The methods of the returned iterator may throw {@code NullPointerException} if any + * of the input iterators is null. + */ + public static Iterator concat(Iterator> inputs) { + return new ConcatenatedIterator(inputs); + } + + /** Concats a varargs array of iterators without making a defensive copy of the array. */ + @SuppressWarnings("unchecked") + static Iterator concatNoDefensiveCopy(Iterator... inputs) { + for (Iterator input : checkNotNull(inputs)) { + checkNotNull(input); + } + return concat(consumingForArray(inputs)); + } + + /** + * Divides an iterator into unmodifiable sublists of the given size (the final list may be + * smaller). For example, partitioning an iterator containing {@code [a, b, c, d, e]} with a + * partition size of 3 yields {@code [[a, b, c], [d, e]]} -- an outer iterator containing two + * inner lists of three and two elements, all in the original order. + * + *

The returned lists implement {@link java.util.RandomAccess}. + * + * @param iterator the iterator to return a partitioned view of + * @param size the desired size of each partition (the last may be smaller) + * @return an iterator of immutable lists containing the elements of {@code iterator} divided into + * partitions + * @throws IllegalArgumentException if {@code size} is nonpositive + */ + public static UnmodifiableIterator> partition(Iterator iterator, int size) { + return partitionImpl(iterator, size, false); + } + + /** + * Divides an iterator into unmodifiable sublists of the given size, padding the final iterator + * with null values if necessary. For example, partitioning an iterator containing {@code [a, b, + * c, d, e]} with a partition size of 3 yields {@code [[a, b, c], [d, e, null]]} -- an outer + * iterator containing two inner lists of three elements each, all in the original order. + * + *

The returned lists implement {@link java.util.RandomAccess}. + * + * @param iterator the iterator to return a partitioned view of + * @param size the desired size of each partition + * @return an iterator of immutable lists containing the elements of {@code iterator} divided into + * partitions (the final iterable may have trailing null elements) + * @throws IllegalArgumentException if {@code size} is nonpositive + */ + public static UnmodifiableIterator> paddedPartition(Iterator iterator, int size) { + return partitionImpl(iterator, size, true); + } + + private static UnmodifiableIterator> partitionImpl( + final Iterator iterator, final int size, final boolean pad) { + checkNotNull(iterator); + checkArgument(size > 0); + return new UnmodifiableIterator>() { + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public List next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + Object[] array = new Object[size]; + int count = 0; + for (; count < size && iterator.hasNext(); count++) { + array[count] = iterator.next(); + } + for (int i = count; i < size; i++) { + array[i] = null; // for GWT + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // we only put Ts in it + List list = Collections.unmodifiableList((List) Arrays.asList(array)); + return (pad || count == size) ? list : list.subList(0, count); + } + }; + } + + /** + * Returns a view of {@code unfiltered} containing all elements that satisfy the input predicate + * {@code retainIfTrue}. + */ + public static UnmodifiableIterator filter( + final Iterator unfiltered, final Predicate retainIfTrue) { + checkNotNull(unfiltered); + checkNotNull(retainIfTrue); + return new AbstractIterator() { + @Override + protected T computeNext() { + while (unfiltered.hasNext()) { + T element = unfiltered.next(); + if (retainIfTrue.apply(element)) { + return element; + } + } + return endOfData(); + } + }; + } + + /** + * Returns a view of {@code unfiltered} containing all elements that are of the type {@code + * desiredType}. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // can cast to because non-Ts are removed + public static UnmodifiableIterator filter(Iterator unfiltered, Class desiredType) { + return (UnmodifiableIterator) filter(unfiltered, instanceOf(desiredType)); + } + + /** + * Returns {@code true} if one or more elements returned by {@code iterator} satisfy the given + * predicate. + */ + public static boolean any(Iterator iterator, Predicate predicate) { + return indexOf(iterator, predicate) != -1; + } + + /** + * Returns {@code true} if every element returned by {@code iterator} satisfies the given + * predicate. If {@code iterator} is empty, {@code true} is returned. + */ + public static boolean all(Iterator iterator, Predicate predicate) { + checkNotNull(predicate); + while (iterator.hasNext()) { + T element = iterator.next(); + if (!predicate.apply(element)) { + return false; + } + } + return true; + } + + /** + * Returns the first element in {@code iterator} that satisfies the given predicate; use this + * method only when such an element is known to exist. If no such element is found, the iterator + * will be left exhausted: its {@code hasNext()} method will return {@code false}. If it is + * possible that no element will match, use {@link #tryFind} or {@link #find(Iterator, + * Predicate, Object)} instead. + * + * @throws NoSuchElementException if no element in {@code iterator} matches the given predicate + */ + public static T find(Iterator iterator, Predicate predicate) { + checkNotNull(iterator); + checkNotNull(predicate); + while (iterator.hasNext()) { + T t = iterator.next(); + if (predicate.apply(t)) { + return t; + } + } + throw new NoSuchElementException(); + } + + /** + * Returns the first element in {@code iterator} that satisfies the given predicate. If no such + * element is found, {@code defaultValue} will be returned from this method and the iterator will + * be left exhausted: its {@code hasNext()} method will return {@code false}. Note that this can + * usually be handled more naturally using {@code tryFind(iterator, predicate).or(defaultValue)}. + * + * @since 7.0 + */ + + public static T find( + Iterator iterator, + Predicate predicate, + T defaultValue) { + checkNotNull(iterator); + checkNotNull(predicate); + while (iterator.hasNext()) { + T t = iterator.next(); + if (predicate.apply(t)) { + return t; + } + } + return defaultValue; + } + + /** + * Returns an {@link Optional} containing the first element in {@code iterator} that satisfies the + * given predicate, if such an element exists. If no such element is found, an empty {@link + * Optional} will be returned from this method and the iterator will be left exhausted: its {@code + * hasNext()} method will return {@code false}. + * + *

Warning: avoid using a {@code predicate} that matches {@code null}. If {@code null} + * is matched in {@code iterator}, a NullPointerException will be thrown. + * + * @since 11.0 + */ + public static Optional tryFind(Iterator iterator, Predicate predicate) { + checkNotNull(iterator); + checkNotNull(predicate); + while (iterator.hasNext()) { + T t = iterator.next(); + if (predicate.apply(t)) { + return Optional.of(t); + } + } + return Optional.absent(); + } + + /** + * Returns the index in {@code iterator} of the first element that satisfies the provided {@code + * predicate}, or {@code -1} if the Iterator has no such elements. + * + *

More formally, returns the lowest index {@code i} such that {@code + * predicate.apply(Iterators.get(iterator, i))} returns {@code true}, or {@code -1} if there is no + * such index. + * + *

If -1 is returned, the iterator will be left exhausted: its {@code hasNext()} method will + * return {@code false}. Otherwise, the iterator will be set to the element which satisfies the + * {@code predicate}. + * + * @since 2.0 + */ + public static int indexOf(Iterator iterator, Predicate predicate) { + checkNotNull(predicate, "predicate"); + for (int i = 0; iterator.hasNext(); i++) { + T current = iterator.next(); + if (predicate.apply(current)) { + return i; + } + } + return -1; + } + + /** + * Returns a view containing the result of applying {@code function} to each element of {@code + * fromIterator}. + * + *

The returned iterator supports {@code remove()} if {@code fromIterator} does. After a + * successful {@code remove()} call, {@code fromIterator} no longer contains the corresponding + * element. + */ + public static Iterator transform( + final Iterator fromIterator, final Function function) { + checkNotNull(function); + return new TransformedIterator(fromIterator) { + @Override + T transform(F from) { + return function.apply(from); + } + }; + } + + /** + * Advances {@code iterator} {@code position + 1} times, returning the element at the {@code + * position}th position. + * + * @param position position of the element to return + * @return the element at the specified position in {@code iterator} + * @throws IndexOutOfBoundsException if {@code position} is negative or greater than or equal to + * the number of elements remaining in {@code iterator} + */ + public static T get(Iterator iterator, int position) { + checkNonnegative(position); + int skipped = advance(iterator, position); + if (!iterator.hasNext()) { + throw new IndexOutOfBoundsException( + "position (" + + position + + ") must be less than the number of elements that remained (" + + skipped + + ")"); + } + return iterator.next(); + } + + /** + * Advances {@code iterator} {@code position + 1} times, returning the element at the {@code + * position}th position or {@code defaultValue} otherwise. + * + * @param position position of the element to return + * @param defaultValue the default value to return if the iterator is empty or if {@code position} + * is greater than the number of elements remaining in {@code iterator} + * @return the element at the specified position in {@code iterator} or {@code defaultValue} if + * {@code iterator} produces fewer than {@code position + 1} elements. + * @throws IndexOutOfBoundsException if {@code position} is negative + * @since 4.0 + */ + + public static T get( + Iterator iterator, int position, T defaultValue) { + checkNonnegative(position); + advance(iterator, position); + return getNext(iterator, defaultValue); + } + + static void checkNonnegative(int position) { + if (position < 0) { + throw new IndexOutOfBoundsException("position (" + position + ") must not be negative"); + } + } + + /** + * Returns the next element in {@code iterator} or {@code defaultValue} if the iterator is empty. + * The {@link Iterables} analog to this method is {@link Iterables#getFirst}. + * + * @param defaultValue the default value to return if the iterator is empty + * @return the next element of {@code iterator} or the default value + * @since 7.0 + */ + + public static T getNext(Iterator iterator, T defaultValue) { + return iterator.hasNext() ? iterator.next() : defaultValue; + } + + /** + * Advances {@code iterator} to the end, returning the last element. + * + * @return the last element of {@code iterator} + * @throws NoSuchElementException if the iterator is empty + */ + public static T getLast(Iterator iterator) { + while (true) { + T current = iterator.next(); + if (!iterator.hasNext()) { + return current; + } + } + } + + /** + * Advances {@code iterator} to the end, returning the last element or {@code defaultValue} if the + * iterator is empty. + * + * @param defaultValue the default value to return if the iterator is empty + * @return the last element of {@code iterator} + * @since 3.0 + */ + + public static T getLast(Iterator iterator, T defaultValue) { + return iterator.hasNext() ? getLast(iterator) : defaultValue; + } + + /** + * Calls {@code next()} on {@code iterator}, either {@code numberToAdvance} times or until {@code + * hasNext()} returns {@code false}, whichever comes first. + * + * @return the number of elements the iterator was advanced + * @since 13.0 (since 3.0 as {@code Iterators.skip}) + */ + public static int advance(Iterator iterator, int numberToAdvance) { + checkNotNull(iterator); + checkArgument(numberToAdvance >= 0, "numberToAdvance must be nonnegative"); + + int i; + for (i = 0; i < numberToAdvance && iterator.hasNext(); i++) { + iterator.next(); + } + return i; + } + + /** + * Returns a view containing the first {@code limitSize} elements of {@code iterator}. If {@code + * iterator} contains fewer than {@code limitSize} elements, the returned view contains all of its + * elements. The returned iterator supports {@code remove()} if {@code iterator} does. + * + * @param iterator the iterator to limit + * @param limitSize the maximum number of elements in the returned iterator + * @throws IllegalArgumentException if {@code limitSize} is negative + * @since 3.0 + */ + public static Iterator limit(final Iterator iterator, final int limitSize) { + checkNotNull(iterator); + checkArgument(limitSize >= 0, "limit is negative"); + return new Iterator() { + private int count; + + @Override + public boolean hasNext() { + return count < limitSize && iterator.hasNext(); + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + count++; + return iterator.next(); + } + + @Override + public void remove() { + iterator.remove(); + } + }; + } + + /** + * Returns a view of the supplied {@code iterator} that removes each element from the supplied + * {@code iterator} as it is returned. + * + *

The provided iterator must support {@link Iterator#remove()} or else the returned iterator + * will fail on the first call to {@code next}. + * + * @param iterator the iterator to remove and return elements from + * @return an iterator that removes and returns elements from the supplied iterator + * @since 2.0 + */ + public static Iterator consumingIterator(final Iterator iterator) { + checkNotNull(iterator); + return new UnmodifiableIterator() { + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public T next() { + T next = iterator.next(); + iterator.remove(); + return next; + } + + @Override + public String toString() { + return "Iterators.consumingIterator(...)"; + } + }; + } + + /** + * Deletes and returns the next value from the iterator, or returns {@code null} if there is no + * such value. + */ + + static T pollNext(Iterator iterator) { + if (iterator.hasNext()) { + T result = iterator.next(); + iterator.remove(); + return result; + } else { + return null; + } + } + + // Methods only in Iterators, not in Iterables + + /** Clears the iterator using its remove method. */ + static void clear(Iterator iterator) { + checkNotNull(iterator); + while (iterator.hasNext()) { + iterator.next(); + iterator.remove(); + } + } + + /** + * Returns an iterator containing the elements of {@code array} in order. The returned iterator is + * a view of the array; subsequent changes to the array will be reflected in the iterator. + * + *

Note: It is often preferable to represent your data using a collection type, for + * example using {@link Arrays#asList(Object[])}, making this method unnecessary. + * + *

The {@code Iterable} equivalent of this method is either {@link Arrays#asList(Object[])}, + * {@link ImmutableList#copyOf(Object[])}}, or {@link ImmutableList#of}. + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static UnmodifiableIterator forArray(final T... array) { + return forArray(array, 0, array.length, 0); + } + + /** + * Returns a list iterator containing the elements in the specified range of {@code array} in + * order, starting at the specified index. + * + *

The {@code Iterable} equivalent of this method is {@code + * Arrays.asList(array).subList(offset, offset + length).listIterator(index)}. + */ + static UnmodifiableListIterator forArray( + final T[] array, final int offset, int length, int index) { + checkArgument(length >= 0); + int end = offset + length; + + // Technically we should give a slightly more descriptive error on overflow + Preconditions.checkPositionIndexes(offset, end, array.length); + Preconditions.checkPositionIndex(index, length); + if (length == 0) { + return emptyListIterator(); + } + return new ArrayItr(array, offset, length, index); + } + + private static final class ArrayItr extends AbstractIndexedListIterator { + static final UnmodifiableListIterator EMPTY = new ArrayItr<>(new Object[0], 0, 0, 0); + + private final T[] array; + private final int offset; + + ArrayItr(T[] array, int offset, int length, int index) { + super(length, index); + this.array = array; + this.offset = offset; + } + + @Override + protected T get(int index) { + return array[offset + index]; + } + } + + /** + * Returns an iterator containing only {@code value}. + * + *

The {@link Iterable} equivalent of this method is {@link Collections#singleton}. + */ + public static UnmodifiableIterator singletonIterator(final T value) { + return new UnmodifiableIterator() { + boolean done; + + @Override + public boolean hasNext() { + return !done; + } + + @Override + public T next() { + if (done) { + throw new NoSuchElementException(); + } + done = true; + return value; + } + }; + } + + /** + * Adapts an {@code Enumeration} to the {@code Iterator} interface. + * + *

This method has no equivalent in {@link Iterables} because viewing an {@code Enumeration} as + * an {@code Iterable} is impossible. However, the contents can be copied into a collection + * using {@link Collections#list}. + */ + public static UnmodifiableIterator forEnumeration(final Enumeration enumeration) { + checkNotNull(enumeration); + return new UnmodifiableIterator() { + @Override + public boolean hasNext() { + return enumeration.hasMoreElements(); + } + + @Override + public T next() { + return enumeration.nextElement(); + } + }; + } + + /** + * Adapts an {@code Iterator} to the {@code Enumeration} interface. + * + *

The {@code Iterable} equivalent of this method is either {@link Collections#enumeration} (if + * you have a {@link Collection}), or {@code Iterators.asEnumeration(collection.iterator())}. + */ + public static Enumeration asEnumeration(final Iterator iterator) { + checkNotNull(iterator); + return new Enumeration() { + @Override + public boolean hasMoreElements() { + return iterator.hasNext(); + } + + @Override + public T nextElement() { + return iterator.next(); + } + }; + } + + /** Implementation of PeekingIterator that avoids peeking unless necessary. */ + private static class PeekingImpl implements PeekingIterator { + + private final Iterator iterator; + private boolean hasPeeked; + private E peekedElement; + + public PeekingImpl(Iterator iterator) { + this.iterator = checkNotNull(iterator); + } + + @Override + public boolean hasNext() { + return hasPeeked || iterator.hasNext(); + } + + @Override + public E next() { + if (!hasPeeked) { + return iterator.next(); + } + E result = peekedElement; + hasPeeked = false; + peekedElement = null; + return result; + } + + @Override + public void remove() { + checkState(!hasPeeked, "Can't remove after you've peeked at next"); + iterator.remove(); + } + + @Override + public E peek() { + if (!hasPeeked) { + peekedElement = iterator.next(); + hasPeeked = true; + } + return peekedElement; + } + } + + /** + * Returns a {@code PeekingIterator} backed by the given iterator. + * + *

Calls to the {@code peek} method with no intervening calls to {@code next} do not affect the + * iteration, and hence return the same object each time. A subsequent call to {@code next} is + * guaranteed to return the same object again. For example: + * + *

{@code
+   * PeekingIterator peekingIterator =
+   *     Iterators.peekingIterator(Iterators.forArray("a", "b"));
+   * String a1 = peekingIterator.peek(); // returns "a"
+   * String a2 = peekingIterator.peek(); // also returns "a"
+   * String a3 = peekingIterator.next(); // also returns "a"
+   * }
+ * + *

Any structural changes to the underlying iteration (aside from those performed by the + * iterator's own {@link PeekingIterator#remove()} method) will leave the iterator in an undefined + * state. + * + *

The returned iterator does not support removal after peeking, as explained by {@link + * PeekingIterator#remove()}. + * + *

Note: If the given iterator is already a {@code PeekingIterator}, it might be + * returned to the caller, although this is neither guaranteed to occur nor required to be + * consistent. For example, this method might choose to pass through recognized + * implementations of {@code PeekingIterator} when the behavior of the implementation is known to + * meet the contract guaranteed by this method. + * + *

There is no {@link Iterable} equivalent to this method, so use this method to wrap each + * individual iterator as it is generated. + * + * @param iterator the backing iterator. The {@link PeekingIterator} assumes ownership of this + * iterator, so users should cease making direct calls to it after calling this method. + * @return a peeking iterator backed by that iterator. Apart from the additional {@link + * PeekingIterator#peek()} method, this iterator behaves exactly the same as {@code iterator}. + */ + public static PeekingIterator peekingIterator(Iterator iterator) { + if (iterator instanceof PeekingImpl) { + // Safe to cast to because PeekingImpl only uses T + // covariantly (and cannot be subclassed to add non-covariant uses). + @SuppressWarnings({"unchecked", "rawtypes"}) + PeekingImpl peeking = (PeekingImpl) iterator; + return peeking; + } + return new PeekingImpl(iterator); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static PeekingIterator peekingIterator(PeekingIterator iterator) { + return checkNotNull(iterator); + } + + /** + * Returns an iterator over the merged contents of all given {@code iterators}, traversing every + * element of the input iterators. Equivalent entries will not be de-duplicated. + * + *

Callers must ensure that the source {@code iterators} are in non-descending order as this + * method does not sort its input. + * + *

For any equivalent elements across all {@code iterators}, it is undefined which element is + * returned first. + * + * @since 11.0 + */ + + public static UnmodifiableIterator mergeSorted( + Iterable> iterators, Comparator comparator) { + checkNotNull(iterators, "iterators"); + checkNotNull(comparator, "comparator"); + + return new MergingIterator(iterators, comparator); + } + + /** + * An iterator that performs a lazy N-way merge, calculating the next value each time the iterator + * is polled. This amortizes the sorting cost over the iteration and requires less memory than + * sorting all elements at once. + * + *

Retrieving a single element takes approximately O(log(M)) time, where M is the number of + * iterators. (Retrieving all elements takes approximately O(N*log(M)) time, where N is the total + * number of elements.) + */ + private static class MergingIterator extends UnmodifiableIterator { + final Queue> queue; + + public MergingIterator( + Iterable> iterators, + final Comparator itemComparator) { + // A comparator that's used by the heap, allowing the heap + // to be sorted based on the top of each iterator. + Comparator> heapComparator = + new Comparator>() { + @Override + public int compare(PeekingIterator o1, PeekingIterator o2) { + return itemComparator.compare(o1.peek(), o2.peek()); + } + }; + + queue = new PriorityQueue<>(2, heapComparator); + + for (Iterator iterator : iterators) { + if (iterator.hasNext()) { + queue.add(Iterators.peekingIterator(iterator)); + } + } + } + + @Override + public boolean hasNext() { + return !queue.isEmpty(); + } + + @Override + public T next() { + PeekingIterator nextIter = queue.remove(); + T next = nextIter.next(); + if (nextIter.hasNext()) { + queue.add(nextIter); + } + return next; + } + } + + private static class ConcatenatedIterator implements Iterator { + /* The last iterator to return an element. Calls to remove() go to this iterator. */ + private Iterator toRemove; + + /* The iterator currently returning elements. */ + private Iterator iterator; + + /* + * We track the "meta iterators," the iterators-of-iterators, below. Usually, topMetaIterator + * is the only one in use, but if we encounter nested concatenations, we start a deque of + * meta-iterators rather than letting the nesting get arbitrarily deep. This keeps each + * operation O(1). + */ + + private Iterator> topMetaIterator; + + // Only becomes nonnull if we encounter nested concatenations. + private Deque>> metaIterators; + + ConcatenatedIterator(Iterator> metaIterator) { + iterator = emptyIterator(); + topMetaIterator = checkNotNull(metaIterator); + } + + // Returns a nonempty meta-iterator or, if all meta-iterators are empty, null. + + private Iterator> getTopMetaIterator() { + while (topMetaIterator == null || !topMetaIterator.hasNext()) { + if (metaIterators != null && !metaIterators.isEmpty()) { + topMetaIterator = metaIterators.removeFirst(); + } else { + return null; + } + } + return topMetaIterator; + } + + @Override + public boolean hasNext() { + while (!checkNotNull(iterator).hasNext()) { + // this weird checkNotNull positioning appears required by our tests, which expect + // both hasNext and next to throw NPE if an input iterator is null. + + topMetaIterator = getTopMetaIterator(); + if (topMetaIterator == null) { + return false; + } + + iterator = topMetaIterator.next(); + + if (iterator instanceof ConcatenatedIterator) { + // Instead of taking linear time in the number of nested concatenations, unpack + // them into the queue + @SuppressWarnings({"unchecked", "rawtypes"}) + ConcatenatedIterator topConcat = (ConcatenatedIterator) iterator; + iterator = topConcat.iterator; + + // topConcat.topMetaIterator, then topConcat.metaIterators, then this.topMetaIterator, + // then this.metaIterators + + if (this.metaIterators == null) { + this.metaIterators = new ArrayDeque<>(); + } + this.metaIterators.addFirst(this.topMetaIterator); + if (topConcat.metaIterators != null) { + while (!topConcat.metaIterators.isEmpty()) { + this.metaIterators.addFirst(topConcat.metaIterators.removeLast()); + } + } + this.topMetaIterator = topConcat.topMetaIterator; + } + } + return true; + } + + @Override + public T next() { + if (hasNext()) { + toRemove = iterator; + return iterator.next(); + } else { + throw new NoSuchElementException(); + } + } + + @Override + public void remove() { + CollectPreconditions.checkRemove(toRemove != null); + toRemove.remove(); + toRemove = null; + } + } + + /** Used to avoid http://bugs.sun.com/view_bug.do?bug_id=6558557 */ + static ListIterator cast(Iterator iterator) { + return (ListIterator) iterator; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableBiMap.java new file mode 100644 index 0000000000000..9e8718f23c578 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableBiMap.java @@ -0,0 +1,119 @@ +/* + * Copyright (C) 2018 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Map; + + +/** + * Implementation of ImmutableBiMap backed by a pair of JDK HashMaps, which have smartness + * protecting against hash flooding. + */ +final class JdkBackedImmutableBiMap extends ImmutableBiMap { + + static ImmutableBiMap create(int n, Entry[] entryArray) { + Map forwardDelegate = Maps.newHashMapWithExpectedSize(n); + Map backwardDelegate = Maps.newHashMapWithExpectedSize(n); + for (int i = 0; i < n; i++) { + Entry e = RegularImmutableMap.makeImmutable(entryArray[i]); + entryArray[i] = e; + V oldValue = forwardDelegate.putIfAbsent(e.getKey(), e.getValue()); + if (oldValue != null) { + throw conflictException("key", e.getKey() + "=" + oldValue, entryArray[i]); + } + K oldKey = backwardDelegate.putIfAbsent(e.getValue(), e.getKey()); + if (oldKey != null) { + throw conflictException("value", oldKey + "=" + e.getValue(), entryArray[i]); + } + } + ImmutableList> entryList = ImmutableList.asImmutableList(entryArray, n); + return new JdkBackedImmutableBiMap<>(entryList, forwardDelegate, backwardDelegate); + } + + private final transient ImmutableList> entries; + private final Map forwardDelegate; + private final Map backwardDelegate; + + private JdkBackedImmutableBiMap( + ImmutableList> entries, Map forwardDelegate, Map backwardDelegate) { + this.entries = entries; + this.forwardDelegate = forwardDelegate; + this.backwardDelegate = backwardDelegate; + } + + @Override + public int size() { + return entries.size(); + } + + private transient JdkBackedImmutableBiMap inverse; + + @Override + public ImmutableBiMap inverse() { + JdkBackedImmutableBiMap result = inverse; + if (result == null) { + inverse = + result = + new JdkBackedImmutableBiMap( + new InverseEntries(), backwardDelegate, forwardDelegate); + result.inverse = this; + } + return result; + } + + private final class InverseEntries extends ImmutableList> { + @Override + public Entry get(int index) { + Entry entry = entries.get(index); + return Maps.immutableEntry(entry.getValue(), entry.getKey()); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public int size() { + return entries.size(); + } + } + + @Override + public V get(Object key) { + return forwardDelegate.get(key); + } + + @Override + ImmutableSet> createEntrySet() { + return new ImmutableMapEntrySet.RegularEntrySet(this, entries); + } + + @Override + ImmutableSet createKeySet() { + return new ImmutableMapKeySet(this); + } + + @Override + boolean isPartialView() { + return false; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMap.java new file mode 100644 index 0000000000000..4fbb2ae6abddb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMap.java @@ -0,0 +1,95 @@ +/* + * Copyright (C) 2018 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.RegularImmutableMap.makeImmutable; + +import java.util.Map; +import java.util.function.BiConsumer; + + +/** + * Implementation of ImmutableMap backed by a JDK HashMap, which has smartness protecting against + * hash flooding. + */ +final class JdkBackedImmutableMap extends ImmutableMap { + /** + * Creates an {@code ImmutableMap} backed by a JDK HashMap. Used when probable hash flooding is + * detected. This implementation may replace the entries in entryArray with its own entry objects + * (though they will have the same key/value contents), and will take ownership of entryArray. + */ + static ImmutableMap create(int n, Entry[] entryArray) { + Map delegateMap = Maps.newHashMapWithExpectedSize(n); + for (int i = 0; i < n; i++) { + entryArray[i] = makeImmutable(entryArray[i]); + V oldValue = delegateMap.putIfAbsent(entryArray[i].getKey(), entryArray[i].getValue()); + if (oldValue != null) { + throw conflictException("key", entryArray[i], entryArray[i].getKey() + "=" + oldValue); + } + } + return new JdkBackedImmutableMap<>(delegateMap, ImmutableList.asImmutableList(entryArray, n)); + } + + private final transient Map delegateMap; + private final transient ImmutableList> entries; + + JdkBackedImmutableMap(Map delegateMap, ImmutableList> entries) { + this.delegateMap = delegateMap; + this.entries = entries; + } + + @Override + public int size() { + return entries.size(); + } + + @Override + public V get(Object key) { + return delegateMap.get(key); + } + + @Override + ImmutableSet> createEntrySet() { + return new ImmutableMapEntrySet.RegularEntrySet(this, entries); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + entries.forEach(e -> action.accept(e.getKey(), e.getValue())); + } + + @Override + ImmutableSet createKeySet() { + return new ImmutableMapKeySet(this); + } + + @Override + ImmutableCollection createValues() { + return new ImmutableMapValues(this); + } + + @Override + boolean isPartialView() { + return false; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMultiset.java new file mode 100644 index 0000000000000..4ef3bca5fe5ac --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableMultiset.java @@ -0,0 +1,92 @@ +/* + * Copyright (C) 2018 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.util.Collection; +import java.util.Map; + + +/** + * An implementation of ImmutableMultiset backed by a JDK Map and a list of entries. Used to protect + * against hash flooding attacks. + * + * @author Louis Wasserman + */ +final class JdkBackedImmutableMultiset extends ImmutableMultiset { + private final Map delegateMap; + private final ImmutableList> entries; + private final long size; + + static ImmutableMultiset create(Collection> entries) { + @SuppressWarnings({"unchecked", "rawtypes"}) + Entry[] entriesArray = entries.toArray(new Entry[0]); + Map delegateMap = Maps.newHashMapWithExpectedSize(entriesArray.length); + long size = 0; + for (int i = 0; i < entriesArray.length; i++) { + Entry entry = entriesArray[i]; + int count = entry.getCount(); + size += count; + E element = checkNotNull(entry.getElement()); + delegateMap.put(element, count); + if (!(entry instanceof Multisets.ImmutableEntry)) { + entriesArray[i] = Multisets.immutableEntry(element, count); + } + } + return new JdkBackedImmutableMultiset<>( + delegateMap, ImmutableList.asImmutableList(entriesArray), size); + } + + private JdkBackedImmutableMultiset( + Map delegateMap, ImmutableList> entries, long size) { + this.delegateMap = delegateMap; + this.entries = entries; + this.size = size; + } + + @Override + public int count(Object element) { + return delegateMap.getOrDefault(element, 0); + } + + private transient ImmutableSet elementSet; + + @Override + public ImmutableSet elementSet() { + ImmutableSet result = elementSet; + return (result == null) ? elementSet = new ElementSet(entries, this) : result; + } + + @Override + Entry getEntry(int index) { + return entries.get(index); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public int size() { + return Ints.saturatedCast(size); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableSet.java new file mode 100644 index 0000000000000..8e6e16629bdbf --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/JdkBackedImmutableSet.java @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2018 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Set; + + +/** + * ImmutableSet implementation backed by a JDK HashSet, used to defend against apparent hash + * flooding. This implementation is never used on the GWT client side, but it must be present there + * for serialization to work. + * + * @author Louis Wasserman + */ +final class JdkBackedImmutableSet extends IndexedImmutableSet { + private final Set delegate; + private final ImmutableList delegateList; + + JdkBackedImmutableSet(Set delegate, ImmutableList delegateList) { + this.delegate = delegate; + this.delegateList = delegateList; + } + + @Override + E get(int index) { + return delegateList.get(index); + } + + @Override + public boolean contains(Object object) { + return delegate.contains(object); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public int size() { + return delegateList.size(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LexicographicalOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LexicographicalOrdering.java new file mode 100644 index 0000000000000..7eb0d81c0265d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LexicographicalOrdering.java @@ -0,0 +1,78 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.Iterator; + + +/** An ordering which sorts iterables by comparing corresponding elements pairwise. */ +final class LexicographicalOrdering extends Ordering> implements Serializable { + final Comparator elementOrder; + + LexicographicalOrdering(Comparator elementOrder) { + this.elementOrder = elementOrder; + } + + @Override + public int compare(Iterable leftIterable, Iterable rightIterable) { + Iterator left = leftIterable.iterator(); + Iterator right = rightIterable.iterator(); + while (left.hasNext()) { + if (!right.hasNext()) { + return LEFT_IS_GREATER; // because it's longer + } + int result = elementOrder.compare(left.next(), right.next()); + if (result != 0) { + return result; + } + } + if (right.hasNext()) { + return RIGHT_IS_GREATER; // because it's longer + } + return 0; + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof LexicographicalOrdering) { + LexicographicalOrdering that = (LexicographicalOrdering) object; + return this.elementOrder.equals(that.elementOrder); + } + return false; + } + + @Override + public int hashCode() { + return elementOrder.hashCode() ^ 2075626741; // meaningless + } + + @Override + public String toString() { + return elementOrder + ".lexicographical()"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimap.java new file mode 100644 index 0000000000000..c4848a4d3d5fb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimap.java @@ -0,0 +1,608 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.Consumer; + + +/** + * Implementation of {@code Multimap} that does not allow duplicate key-value entries and that + * returns collections whose iterators follow the ordering in which the data was added to the + * multimap. + * + *

The collections returned by {@code keySet}, {@code keys}, and {@code asMap} iterate through + * the keys in the order they were first added to the multimap. Similarly, {@code get}, {@code + * removeAll}, and {@code replaceValues} return collections that iterate through the values in the + * order they were added. The collections generated by {@code entries} and {@code values} iterate + * across the key-value mappings in the order they were added to the multimap. + * + *

The iteration ordering of the collections generated by {@code keySet}, {@code keys}, and + * {@code asMap} has a few subtleties. As long as the set of keys remains unchanged, adding or + * removing mappings does not affect the key iteration order. However, if you remove all values + * associated with a key and then add the key back to the multimap, that key will come last in the + * key iteration order. + * + *

The multimap does not store duplicate key-value pairs. Adding a new key-value pair equal to an + * existing key-value pair has no effect. + * + *

Keys and values may be null. All optional multimap methods are supported, and all returned + * views are modifiable. + * + *

This class is not threadsafe when any concurrent operations update the multimap. Concurrent + * read operations will work correctly. To allow concurrent update operations, wrap your multimap + * with a call to {@link Multimaps#synchronizedSetMultimap}. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @author Louis Wasserman + * @since 2.0 + */ +public final class LinkedHashMultimap + extends LinkedHashMultimapGwtSerializationDependencies { + + /** Creates a new, empty {@code LinkedHashMultimap} with the default initial capacities. */ + public static LinkedHashMultimap create() { + return new LinkedHashMultimap<>(DEFAULT_KEY_CAPACITY, DEFAULT_VALUE_SET_CAPACITY); + } + + /** + * Constructs an empty {@code LinkedHashMultimap} with enough capacity to hold the specified + * numbers of keys and values without rehashing. + * + * @param expectedKeys the expected number of distinct keys + * @param expectedValuesPerKey the expected average number of values per key + * @throws IllegalArgumentException if {@code expectedKeys} or {@code expectedValuesPerKey} is + * negative + */ + public static LinkedHashMultimap create(int expectedKeys, int expectedValuesPerKey) { + return new LinkedHashMultimap<>( + Maps.capacity(expectedKeys), Maps.capacity(expectedValuesPerKey)); + } + + /** + * Constructs a {@code LinkedHashMultimap} with the same mappings as the specified multimap. If a + * key-value mapping appears multiple times in the input multimap, it only appears once in the + * constructed multimap. The new multimap has the same {@link Multimap#entries()} iteration order + * as the input multimap, except for excluding duplicate mappings. + * + * @param multimap the multimap whose contents are copied to this multimap + */ + public static LinkedHashMultimap create( + Multimap multimap) { + LinkedHashMultimap result = create(multimap.keySet().size(), DEFAULT_VALUE_SET_CAPACITY); + result.putAll(multimap); + return result; + } + + private interface ValueSetLink { + ValueSetLink getPredecessorInValueSet(); + + ValueSetLink getSuccessorInValueSet(); + + void setPredecessorInValueSet(ValueSetLink entry); + + void setSuccessorInValueSet(ValueSetLink entry); + } + + private static void succeedsInValueSet(ValueSetLink pred, ValueSetLink succ) { + pred.setSuccessorInValueSet(succ); + succ.setPredecessorInValueSet(pred); + } + + private static void succeedsInMultimap(ValueEntry pred, ValueEntry succ) { + pred.setSuccessorInMultimap(succ); + succ.setPredecessorInMultimap(pred); + } + + private static void deleteFromValueSet(ValueSetLink entry) { + succeedsInValueSet(entry.getPredecessorInValueSet(), entry.getSuccessorInValueSet()); + } + + private static void deleteFromMultimap(ValueEntry entry) { + succeedsInMultimap(entry.getPredecessorInMultimap(), entry.getSuccessorInMultimap()); + } + + /** + * LinkedHashMultimap entries are in no less than three coexisting linked lists: a bucket in the + * hash table for a {@code Set} associated with a key, the linked list of insertion-ordered + * entries in that {@code Set}, and the linked list of entries in the LinkedHashMultimap as a + * whole. + */ + + static final class ValueEntry extends ImmutableEntry implements ValueSetLink { + final int smearedValueHash; + + ValueEntry nextInValueBucket; + + ValueSetLink predecessorInValueSet; + ValueSetLink successorInValueSet; + + ValueEntry predecessorInMultimap; + ValueEntry successorInMultimap; + + ValueEntry( + K key, + V value, + int smearedValueHash, + ValueEntry nextInValueBucket) { + super(key, value); + this.smearedValueHash = smearedValueHash; + this.nextInValueBucket = nextInValueBucket; + } + + boolean matchesValue(Object v, int smearedVHash) { + return smearedValueHash == smearedVHash && Objects.equal(getValue(), v); + } + + @Override + public ValueSetLink getPredecessorInValueSet() { + return predecessorInValueSet; + } + + @Override + public ValueSetLink getSuccessorInValueSet() { + return successorInValueSet; + } + + @Override + public void setPredecessorInValueSet(ValueSetLink entry) { + predecessorInValueSet = entry; + } + + @Override + public void setSuccessorInValueSet(ValueSetLink entry) { + successorInValueSet = entry; + } + + public ValueEntry getPredecessorInMultimap() { + return predecessorInMultimap; + } + + public ValueEntry getSuccessorInMultimap() { + return successorInMultimap; + } + + public void setSuccessorInMultimap(ValueEntry multimapSuccessor) { + this.successorInMultimap = multimapSuccessor; + } + + public void setPredecessorInMultimap(ValueEntry multimapPredecessor) { + this.predecessorInMultimap = multimapPredecessor; + } + } + + private static final int DEFAULT_KEY_CAPACITY = 16; + private static final int DEFAULT_VALUE_SET_CAPACITY = 2; + static final double VALUE_SET_LOAD_FACTOR = 1.0; + + transient int valueSetCapacity = DEFAULT_VALUE_SET_CAPACITY; + private transient ValueEntry multimapHeaderEntry; + + private LinkedHashMultimap(int keyCapacity, int valueSetCapacity) { + super(Platform.>newLinkedHashMapWithExpectedSize(keyCapacity)); + checkNonnegative(valueSetCapacity, "expectedValuesPerKey"); + + this.valueSetCapacity = valueSetCapacity; + this.multimapHeaderEntry = new ValueEntry<>(null, null, 0, null); + succeedsInMultimap(multimapHeaderEntry, multimapHeaderEntry); + } + + /** + * {@inheritDoc} + * + *

Creates an empty {@code LinkedHashSet} for a collection of values for one key. + * + * @return a new {@code LinkedHashSet} containing a collection of values for one key + */ + @Override + Set createCollection() { + return Platform.newLinkedHashSetWithExpectedSize(valueSetCapacity); + } + + /** + * {@inheritDoc} + * + *

Creates a decorated insertion-ordered set that also keeps track of the order in which + * key-value pairs are added to the multimap. + * + * @param key key to associate with values in the collection + * @return a new decorated set containing a collection of values for one key + */ + @Override + Collection createCollection(K key) { + return new ValueSet(key, valueSetCapacity); + } + + /** + * {@inheritDoc} + * + *

If {@code values} is not empty and the multimap already contains a mapping for {@code key}, + * the {@code keySet()} ordering is unchanged. However, the provided values always come last in + * the {@link #entries()} and {@link #values()} iteration orderings. + */ + @Override + public Set replaceValues(K key, Iterable values) { + return super.replaceValues(key, values); + } + + /** + * Returns a set of all key-value pairs. Changes to the returned set will update the underlying + * multimap, and vice versa. The entries set does not support the {@code add} or {@code addAll} + * operations. + * + *

The iterator generated by the returned set traverses the entries in the order they were + * added to the multimap. + * + *

Each entry is an immutable snapshot of a key-value mapping in the multimap, taken at the + * time the entry is returned by a method call to the collection or its iterator. + */ + @Override + public Set> entries() { + return super.entries(); + } + + /** + * Returns a view collection of all distinct keys contained in this multimap. Note that the + * key set contains a key if and only if this multimap maps that key to at least one value. + * + *

The iterator generated by the returned set traverses the keys in the order they were first + * added to the multimap. + * + *

Changes to the returned set will update the underlying multimap, and vice versa. However, + * adding to the returned set is not possible. + */ + @Override + public Set keySet() { + return super.keySet(); + } + + /** + * Returns a collection of all values in the multimap. Changes to the returned collection will + * update the underlying multimap, and vice versa. + * + *

The iterator generated by the returned collection traverses the values in the order they + * were added to the multimap. + */ + @Override + public Collection values() { + return super.values(); + } + + + final class ValueSet extends Sets.ImprovedAbstractSet implements ValueSetLink { + /* + * We currently use a fixed load factor of 1.0, a bit higher than normal to reduce memory + * consumption. + */ + + private final K key; + ValueEntry[] hashTable; + private int size = 0; + private int modCount = 0; + + // We use the set object itself as the end of the linked list, avoiding an unnecessary + // entry object per key. + private ValueSetLink firstEntry; + private ValueSetLink lastEntry; + + ValueSet(K key, int expectedValues) { + this.key = key; + this.firstEntry = this; + this.lastEntry = this; + // Round expected values up to a power of 2 to get the table size. + int tableSize = Hashing.closedTableSize(expectedValues, VALUE_SET_LOAD_FACTOR); + + @SuppressWarnings({"unchecked", "rawtypes"}) + ValueEntry[] hashTable = new ValueEntry[tableSize]; + this.hashTable = hashTable; + } + + private int mask() { + return hashTable.length - 1; + } + + @Override + public ValueSetLink getPredecessorInValueSet() { + return lastEntry; + } + + @Override + public ValueSetLink getSuccessorInValueSet() { + return firstEntry; + } + + @Override + public void setPredecessorInValueSet(ValueSetLink entry) { + lastEntry = entry; + } + + @Override + public void setSuccessorInValueSet(ValueSetLink entry) { + firstEntry = entry; + } + + @Override + public Iterator iterator() { + return new Iterator() { + ValueSetLink nextEntry = firstEntry; + ValueEntry toRemove; + int expectedModCount = modCount; + + private void checkForComodification() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + } + + @Override + public boolean hasNext() { + checkForComodification(); + return nextEntry != ValueSet.this; + } + + @Override + public V next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + ValueEntry entry = (ValueEntry) nextEntry; + V result = entry.getValue(); + toRemove = entry; + nextEntry = entry.getSuccessorInValueSet(); + return result; + } + + @Override + public void remove() { + checkForComodification(); + checkRemove(toRemove != null); + ValueSet.this.remove(toRemove.getValue()); + expectedModCount = modCount; + toRemove = null; + } + }; + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + for (ValueSetLink entry = firstEntry; + entry != ValueSet.this; + entry = entry.getSuccessorInValueSet()) { + action.accept(((ValueEntry) entry).getValue()); + } + } + + @Override + public int size() { + return size; + } + + @Override + public boolean contains(Object o) { + int smearedHash = Hashing.smearedHash(o); + for (ValueEntry entry = hashTable[smearedHash & mask()]; + entry != null; + entry = entry.nextInValueBucket) { + if (entry.matchesValue(o, smearedHash)) { + return true; + } + } + return false; + } + + @Override + public boolean add(V value) { + int smearedHash = Hashing.smearedHash(value); + int bucket = smearedHash & mask(); + ValueEntry rowHead = hashTable[bucket]; + for (ValueEntry entry = rowHead; entry != null; entry = entry.nextInValueBucket) { + if (entry.matchesValue(value, smearedHash)) { + return false; + } + } + + ValueEntry newEntry = new ValueEntry<>(key, value, smearedHash, rowHead); + succeedsInValueSet(lastEntry, newEntry); + succeedsInValueSet(newEntry, this); + succeedsInMultimap(multimapHeaderEntry.getPredecessorInMultimap(), newEntry); + succeedsInMultimap(newEntry, multimapHeaderEntry); + hashTable[bucket] = newEntry; + size++; + modCount++; + rehashIfNecessary(); + return true; + } + + private void rehashIfNecessary() { + if (Hashing.needsResizing(size, hashTable.length, VALUE_SET_LOAD_FACTOR)) { + @SuppressWarnings({"unchecked", "rawtypes"}) + ValueEntry[] hashTable = new ValueEntry[this.hashTable.length * 2]; + this.hashTable = hashTable; + int mask = hashTable.length - 1; + for (ValueSetLink entry = firstEntry; + entry != this; + entry = entry.getSuccessorInValueSet()) { + ValueEntry valueEntry = (ValueEntry) entry; + int bucket = valueEntry.smearedValueHash & mask; + valueEntry.nextInValueBucket = hashTable[bucket]; + hashTable[bucket] = valueEntry; + } + } + } + + @Override + public boolean remove(Object o) { + int smearedHash = Hashing.smearedHash(o); + int bucket = smearedHash & mask(); + ValueEntry prev = null; + for (ValueEntry entry = hashTable[bucket]; + entry != null; + prev = entry, entry = entry.nextInValueBucket) { + if (entry.matchesValue(o, smearedHash)) { + if (prev == null) { + // first entry in the bucket + hashTable[bucket] = entry.nextInValueBucket; + } else { + prev.nextInValueBucket = entry.nextInValueBucket; + } + deleteFromValueSet(entry); + deleteFromMultimap(entry); + size--; + modCount++; + return true; + } + } + return false; + } + + @Override + public void clear() { + Arrays.fill(hashTable, null); + size = 0; + for (ValueSetLink entry = firstEntry; + entry != this; + entry = entry.getSuccessorInValueSet()) { + ValueEntry valueEntry = (ValueEntry) entry; + deleteFromMultimap(valueEntry); + } + succeedsInValueSet(this, this); + modCount++; + } + } + + @Override + Iterator> entryIterator() { + return new Iterator>() { + ValueEntry nextEntry = multimapHeaderEntry.successorInMultimap; + ValueEntry toRemove; + + @Override + public boolean hasNext() { + return nextEntry != multimapHeaderEntry; + } + + @Override + public Entry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + ValueEntry result = nextEntry; + toRemove = result; + nextEntry = nextEntry.successorInMultimap; + return result; + } + + @Override + public void remove() { + checkRemove(toRemove != null); + LinkedHashMultimap.this.remove(toRemove.getKey(), toRemove.getValue()); + toRemove = null; + } + }; + } + + @Override + Spliterator> entrySpliterator() { + return Spliterators.spliterator(entries(), Spliterator.DISTINCT | Spliterator.ORDERED); + } + + @Override + Iterator valueIterator() { + return Maps.valueIterator(entryIterator()); + } + + @Override + Spliterator valueSpliterator() { + return CollectSpliterators.map(entrySpliterator(), Entry::getValue); + } + + @Override + public void clear() { + super.clear(); + succeedsInMultimap(multimapHeaderEntry, multimapHeaderEntry); + } + + /** + * @serialData the expected values per key, the number of distinct keys, the number of entries, + * and the entries in order + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeInt(keySet().size()); + for (K key : keySet()) { + stream.writeObject(key); + } + stream.writeInt(size()); + for (Entry entry : entries()) { + stream.writeObject(entry.getKey()); + stream.writeObject(entry.getValue()); + } + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + multimapHeaderEntry = new ValueEntry<>(null, null, 0, null); + succeedsInMultimap(multimapHeaderEntry, multimapHeaderEntry); + valueSetCapacity = DEFAULT_VALUE_SET_CAPACITY; + int distinctKeys = stream.readInt(); + Map> map = Platform.newLinkedHashMapWithExpectedSize(12); + for (int i = 0; i < distinctKeys; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) + K key = (K) stream.readObject(); + map.put(key, createCollection(key)); + } + int entries = stream.readInt(); + for (int i = 0; i < entries; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) + K key = (K) stream.readObject(); + @SuppressWarnings({"unchecked", "rawtypes"}) + V value = (V) stream.readObject(); + map.get(key).add(value); + } + setMap(map); + } + + private static final long serialVersionUID = 1; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimapGwtSerializationDependencies.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimapGwtSerializationDependencies.java new file mode 100644 index 0000000000000..0590fd48381ad --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultimapGwtSerializationDependencies.java @@ -0,0 +1,40 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Map; + +/** + * A dummy superclass to support GWT serialization of the element types of a {@link + * LinkedHashMultimap}. The GWT supersource for this class contains a field for each type. + * + *

For details about this hack, see {@link GwtSerializationDependencies}, which takes the same + * approach but with a subclass rather than a superclass. + * + *

TODO(cpovirk): Consider applying this subclass approach to our other types. + */ +abstract class LinkedHashMultimapGwtSerializationDependencies + extends AbstractSetMultimap { + LinkedHashMultimapGwtSerializationDependencies(Map> map) { + super(map); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultiset.java new file mode 100644 index 0000000000000..6924fed91c437 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedHashMultiset.java @@ -0,0 +1,100 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.LinkedHashMap; + +/** + * A {@code Multiset} implementation with predictable iteration order. Its iterator orders elements + * according to when the first occurrence of the element was added. When the multiset contains + * multiple instances of an element, those instances are consecutive in the iteration order. If all + * occurrences of an element are removed, after which that element is added to the multiset, the + * element will appear at the end of the iteration. + * + *

See the Guava User Guide article on {@code + * Multiset}. + * + * @author Kevin Bourrillion + * @author Jared Levy + * @since 2.0 + */ +@SuppressWarnings("serial") // we're overriding default serialization +public final class LinkedHashMultiset extends AbstractMapBasedMultiset { + + /** Creates a new, empty {@code LinkedHashMultiset} using the default initial capacity. */ + public static LinkedHashMultiset create() { + return new LinkedHashMultiset(); + } + + /** + * Creates a new, empty {@code LinkedHashMultiset} with the specified expected number of distinct + * elements. + * + * @param distinctElements the expected number of distinct elements + * @throws IllegalArgumentException if {@code distinctElements} is negative + */ + public static LinkedHashMultiset create(int distinctElements) { + return new LinkedHashMultiset(distinctElements); + } + + /** + * Creates a new {@code LinkedHashMultiset} containing the specified elements. + * + *

This implementation is highly efficient when {@code elements} is itself a {@link Multiset}. + * + * @param elements the elements that the multiset should contain + */ + public static LinkedHashMultiset create(Iterable elements) { + LinkedHashMultiset multiset = create(Multisets.inferDistinctElements(elements)); + Iterables.addAll(multiset, elements); + return multiset; + } + + private LinkedHashMultiset() { + super(new LinkedHashMap()); + } + + private LinkedHashMultiset(int distinctElements) { + super(Maps.newLinkedHashMapWithExpectedSize(distinctElements)); + } + + /** + * @serialData the number of distinct elements, the first element, its count, the second element, + * its count, and so on + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + Serialization.writeMultiset(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + int distinctElements = Serialization.readCount(stream); + setBackingMap(new LinkedHashMap()); + Serialization.populateMultiset(this, stream, distinctElements); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedListMultimap.java new file mode 100644 index 0000000000000..f4f1d8aab2056 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/LinkedListMultimap.java @@ -0,0 +1,840 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; +import static java.util.Collections.unmodifiableList; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.AbstractSequentialList; +import java.util.Collection; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.function.Consumer; + + +/** + * An implementation of {@code ListMultimap} that supports deterministic iteration order for both + * keys and values. The iteration order is preserved across non-distinct key values. For example, + * for the following multimap definition: + * + *

{@code
+ * Multimap multimap = LinkedListMultimap.create();
+ * multimap.put(key1, foo);
+ * multimap.put(key2, bar);
+ * multimap.put(key1, baz);
+ * }
+ * + * ... the iteration order for {@link #keys()} is {@code [key1, key2, key1]}, and similarly for + * {@link #entries()}. Unlike {@link LinkedHashMultimap}, the iteration order is kept consistent + * between keys, entries and values. For example, calling: + * + *
{@code
+ * map.remove(key1, foo);
+ * }
+ * + *

changes the entries iteration order to {@code [key2=bar, key1=baz]} and the key iteration + * order to {@code [key2, key1]}. The {@link #entries()} iterator returns mutable map entries, and + * {@link #replaceValues} attempts to preserve iteration order as much as possible. + * + *

The collections returned by {@link #keySet()} and {@link #asMap} iterate through the keys in + * the order they were first added to the multimap. Similarly, {@link #get}, {@link #removeAll}, and + * {@link #replaceValues} return collections that iterate through the values in the order they were + * added. The collections generated by {@link #entries()}, {@link #keys()}, and {@link #values} + * iterate across the key-value mappings in the order they were added to the multimap. + * + *

The {@link #values()} and {@link #entries()} methods both return a {@code List}, instead of + * the {@code Collection} specified by the {@link ListMultimap} interface. + * + *

The methods {@link #get}, {@link #keySet()}, {@link #keys()}, {@link #values}, {@link + * #entries()}, and {@link #asMap} return collections that are views of the multimap. If the + * multimap is modified while an iteration over any of those collections is in progress, except + * through the iterator's methods, the results of the iteration are undefined. + * + *

Keys and values may be null. All optional multimap methods are supported, and all returned + * views are modifiable. + * + *

This class is not threadsafe when any concurrent operations update the multimap. Concurrent + * read operations will work correctly. To allow concurrent update operations, wrap your multimap + * with a call to {@link Multimaps#synchronizedListMultimap}. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Mike Bostock + * @since 2.0 + */ +public class LinkedListMultimap extends AbstractMultimap + implements ListMultimap, Serializable { + /* + * Order is maintained using a linked list containing all key-value pairs. In + * addition, a series of disjoint linked lists of "siblings", each containing + * the values for a specific key, is used to implement {@link + * ValueForKeyIterator} in constant time. + */ + + private static final class Node extends AbstractMapEntry { + final K key; + V value; + Node next; // the next node (with any key) + Node previous; // the previous node (with any key) + Node nextSibling; // the next node with the same key + Node previousSibling; // the previous node with the same key + + Node(K key, V value) { + this.key = key; + this.value = value; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public V setValue(V newValue) { + V result = value; + this.value = newValue; + return result; + } + } + + private static class KeyList { + Node head; + Node tail; + int count; + + KeyList(Node firstNode) { + this.head = firstNode; + this.tail = firstNode; + firstNode.previousSibling = null; + firstNode.nextSibling = null; + this.count = 1; + } + } + + private transient Node head; // the head for all keys + private transient Node tail; // the tail for all keys + private transient Map> keyToKeyList; + private transient int size; + + /* + * Tracks modifications to keyToKeyList so that addition or removal of keys invalidates + * preexisting iterators. This does *not* track simple additions and removals of values + * that are not the first to be added or last to be removed for their key. + */ + private transient int modCount; + + /** Creates a new, empty {@code LinkedListMultimap} with the default initial capacity. */ + public static LinkedListMultimap create() { + return new LinkedListMultimap<>(); + } + + /** + * Constructs an empty {@code LinkedListMultimap} with enough capacity to hold the specified + * number of keys without rehashing. + * + * @param expectedKeys the expected number of distinct keys + * @throws IllegalArgumentException if {@code expectedKeys} is negative + */ + public static LinkedListMultimap create(int expectedKeys) { + return new LinkedListMultimap<>(expectedKeys); + } + + /** + * Constructs a {@code LinkedListMultimap} with the same mappings as the specified {@code + * Multimap}. The new multimap has the same {@link Multimap#entries()} iteration order as the + * input multimap. + * + * @param multimap the multimap whose contents are copied to this multimap + */ + public static LinkedListMultimap create( + Multimap multimap) { + return new LinkedListMultimap<>(multimap); + } + + LinkedListMultimap() { + this(12); + } + + private LinkedListMultimap(int expectedKeys) { + keyToKeyList = Platform.newHashMapWithExpectedSize(expectedKeys); + } + + private LinkedListMultimap(Multimap multimap) { + this(multimap.keySet().size()); + putAll(multimap); + } + + /** + * Adds a new node for the specified key-value pair before the specified {@code nextSibling} + * element, or at the end of the list if {@code nextSibling} is null. Note: if {@code nextSibling} + * is specified, it MUST be for an node for the same {@code key}! + */ + private Node addNode( + K key, V value, Node nextSibling) { + Node node = new Node<>(key, value); + if (head == null) { // empty list + head = tail = node; + keyToKeyList.put(key, new KeyList(node)); + modCount++; + } else if (nextSibling == null) { // non-empty list, add to tail + tail.next = node; + node.previous = tail; + tail = node; + KeyList keyList = keyToKeyList.get(key); + if (keyList == null) { + keyToKeyList.put(key, keyList = new KeyList<>(node)); + modCount++; + } else { + keyList.count++; + Node keyTail = keyList.tail; + keyTail.nextSibling = node; + node.previousSibling = keyTail; + keyList.tail = node; + } + } else { // non-empty list, insert before nextSibling + KeyList keyList = keyToKeyList.get(key); + keyList.count++; + node.previous = nextSibling.previous; + node.previousSibling = nextSibling.previousSibling; + node.next = nextSibling; + node.nextSibling = nextSibling; + if (nextSibling.previousSibling == null) { // nextSibling was key head + keyToKeyList.get(key).head = node; + } else { + nextSibling.previousSibling.nextSibling = node; + } + if (nextSibling.previous == null) { // nextSibling was head + head = node; + } else { + nextSibling.previous.next = node; + } + nextSibling.previous = node; + nextSibling.previousSibling = node; + } + size++; + return node; + } + + /** + * Removes the specified node from the linked list. This method is only intended to be used from + * the {@code Iterator} classes. See also {@link LinkedListMultimap#removeAllNodes(Object)}. + */ + private void removeNode(Node node) { + if (node.previous != null) { + node.previous.next = node.next; + } else { // node was head + head = node.next; + } + if (node.next != null) { + node.next.previous = node.previous; + } else { // node was tail + tail = node.previous; + } + if (node.previousSibling == null && node.nextSibling == null) { + KeyList keyList = keyToKeyList.remove(node.key); + keyList.count = 0; + modCount++; + } else { + KeyList keyList = keyToKeyList.get(node.key); + keyList.count--; + + if (node.previousSibling == null) { + keyList.head = node.nextSibling; + } else { + node.previousSibling.nextSibling = node.nextSibling; + } + + if (node.nextSibling == null) { + keyList.tail = node.previousSibling; + } else { + node.nextSibling.previousSibling = node.previousSibling; + } + } + size--; + } + + /** Removes all nodes for the specified key. */ + private void removeAllNodes(Object key) { + Iterators.clear(new ValueForKeyIterator(key)); + } + + /** Helper method for verifying that an iterator element is present. */ + private static void checkElement(Object node) { + if (node == null) { + throw new NoSuchElementException(); + } + } + + /** An {@code Iterator} over all nodes. */ + private class NodeIterator implements ListIterator> { + int nextIndex; + Node next; + Node current; + Node previous; + int expectedModCount = modCount; + + NodeIterator(int index) { + int size = size(); + checkPositionIndex(index, size); + if (index >= (size / 2)) { + previous = tail; + nextIndex = size; + while (index++ < size) { + previous(); + } + } else { + next = head; + while (index-- > 0) { + next(); + } + } + current = null; + } + + private void checkForConcurrentModification() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + } + + @Override + public boolean hasNext() { + checkForConcurrentModification(); + return next != null; + } + + @Override + public Node next() { + checkForConcurrentModification(); + checkElement(next); + previous = current = next; + next = next.next; + nextIndex++; + return current; + } + + @Override + public void remove() { + checkForConcurrentModification(); + checkRemove(current != null); + if (current != next) { // after call to next() + previous = current.previous; + nextIndex--; + } else { // after call to previous() + next = current.next; + } + removeNode(current); + current = null; + expectedModCount = modCount; + } + + @Override + public boolean hasPrevious() { + checkForConcurrentModification(); + return previous != null; + } + + @Override + public Node previous() { + checkForConcurrentModification(); + checkElement(previous); + next = current = previous; + previous = previous.previous; + nextIndex--; + return current; + } + + @Override + public int nextIndex() { + return nextIndex; + } + + @Override + public int previousIndex() { + return nextIndex - 1; + } + + @Override + public void set(Entry e) { + throw new UnsupportedOperationException(); + } + + @Override + public void add(Entry e) { + throw new UnsupportedOperationException(); + } + + void setValue(V value) { + checkState(current != null); + current.value = value; + } + } + + /** An {@code Iterator} over distinct keys in key head order. */ + private class DistinctKeyIterator implements Iterator { + final Set seenKeys = Sets.newHashSetWithExpectedSize(keySet().size()); + Node next = head; + Node current; + int expectedModCount = modCount; + + private void checkForConcurrentModification() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + } + + @Override + public boolean hasNext() { + checkForConcurrentModification(); + return next != null; + } + + @Override + public K next() { + checkForConcurrentModification(); + checkElement(next); + current = next; + seenKeys.add(current.key); + do { // skip ahead to next unseen key + next = next.next; + } while ((next != null) && !seenKeys.add(next.key)); + return current.key; + } + + @Override + public void remove() { + checkForConcurrentModification(); + checkRemove(current != null); + removeAllNodes(current.key); + current = null; + expectedModCount = modCount; + } + } + + /** A {@code ListIterator} over values for a specified key. */ + private class ValueForKeyIterator implements ListIterator { + final Object key; + int nextIndex; + Node next; + Node current; + Node previous; + + /** Constructs a new iterator over all values for the specified key. */ + ValueForKeyIterator(Object key) { + this.key = key; + KeyList keyList = keyToKeyList.get(key); + next = (keyList == null) ? null : keyList.head; + } + + /** + * Constructs a new iterator over all values for the specified key starting at the specified + * index. This constructor is optimized so that it starts at either the head or the tail, + * depending on which is closer to the specified index. This allows adds to the tail to be done + * in constant time. + * + * @throws IndexOutOfBoundsException if index is invalid + */ + public ValueForKeyIterator(Object key, int index) { + KeyList keyList = keyToKeyList.get(key); + int size = (keyList == null) ? 0 : keyList.count; + checkPositionIndex(index, size); + if (index >= (size / 2)) { + previous = (keyList == null) ? null : keyList.tail; + nextIndex = size; + while (index++ < size) { + previous(); + } + } else { + next = (keyList == null) ? null : keyList.head; + while (index-- > 0) { + next(); + } + } + this.key = key; + current = null; + } + + @Override + public boolean hasNext() { + return next != null; + } + + @Override + public V next() { + checkElement(next); + previous = current = next; + next = next.nextSibling; + nextIndex++; + return current.value; + } + + @Override + public boolean hasPrevious() { + return previous != null; + } + + @Override + public V previous() { + checkElement(previous); + next = current = previous; + previous = previous.previousSibling; + nextIndex--; + return current.value; + } + + @Override + public int nextIndex() { + return nextIndex; + } + + @Override + public int previousIndex() { + return nextIndex - 1; + } + + @Override + public void remove() { + checkRemove(current != null); + if (current != next) { // after call to next() + previous = current.previousSibling; + nextIndex--; + } else { // after call to previous() + next = current.nextSibling; + } + removeNode(current); + current = null; + } + + @Override + public void set(V value) { + checkState(current != null); + current.value = value; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public void add(V value) { + previous = addNode((K) key, value, next); + nextIndex++; + current = null; + } + } + + // Query Operations + + @Override + public int size() { + return size; + } + + @Override + public boolean isEmpty() { + return head == null; + } + + @Override + public boolean containsKey(Object key) { + return keyToKeyList.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return values().contains(value); + } + + // Modification Operations + + /** + * Stores a key-value pair in the multimap. + * + * @param key key to store in the multimap + * @param value value to store in the multimap + * @return {@code true} always + */ + @Override + public boolean put(K key, V value) { + addNode(key, value, null); + return true; + } + + // Bulk Operations + + /** + * {@inheritDoc} + * + *

If any entries for the specified {@code key} already exist in the multimap, their values are + * changed in-place without affecting the iteration order. + * + *

The returned list is immutable and implements {@link java.util.RandomAccess}. + */ + @Override + public List replaceValues(K key, Iterable values) { + List oldValues = getCopy(key); + ListIterator keyValues = new ValueForKeyIterator(key); + Iterator newValues = values.iterator(); + + // Replace existing values, if any. + while (keyValues.hasNext() && newValues.hasNext()) { + keyValues.next(); + keyValues.set(newValues.next()); + } + + // Remove remaining old values, if any. + while (keyValues.hasNext()) { + keyValues.next(); + keyValues.remove(); + } + + // Add remaining new values, if any. + while (newValues.hasNext()) { + keyValues.add(newValues.next()); + } + + return oldValues; + } + + private List getCopy(Object key) { + return unmodifiableList(Lists.newArrayList(new ValueForKeyIterator(key))); + } + + /** + * {@inheritDoc} + * + *

The returned list is immutable and implements {@link java.util.RandomAccess}. + */ + @Override + public List removeAll(Object key) { + List oldValues = getCopy(key); + removeAllNodes(key); + return oldValues; + } + + @Override + public void clear() { + head = null; + tail = null; + keyToKeyList.clear(); + size = 0; + modCount++; + } + + // Views + + /** + * {@inheritDoc} + * + *

If the multimap is modified while an iteration over the list is in progress (except through + * the iterator's own {@code add}, {@code set} or {@code remove} operations) the results of the + * iteration are undefined. + * + *

The returned list is not serializable and does not have random access. + */ + @Override + public List get(final K key) { + return new AbstractSequentialList() { + @Override + public int size() { + KeyList keyList = keyToKeyList.get(key); + return (keyList == null) ? 0 : keyList.count; + } + + @Override + public ListIterator listIterator(int index) { + return new ValueForKeyIterator(key, index); + } + }; + } + + @Override + Set createKeySet() { + class KeySetImpl extends Sets.ImprovedAbstractSet { + @Override + public int size() { + return keyToKeyList.size(); + } + + @Override + public Iterator iterator() { + return new DistinctKeyIterator(); + } + + @Override + public boolean contains(Object key) { // for performance + return containsKey(key); + } + + @Override + public boolean remove(Object o) { // for performance + return !LinkedListMultimap.this.removeAll(o).isEmpty(); + } + } + return new KeySetImpl(); + } + + @Override + Multiset createKeys() { + return new Multimaps.Keys(this); + } + + /** + * {@inheritDoc} + * + *

The iterator generated by the returned collection traverses the values in the order they + * were added to the multimap. Because the values may have duplicates and follow the insertion + * ordering, this method returns a {@link List}, instead of the {@link Collection} specified in + * the {@link ListMultimap} interface. + */ + @Override + public List values() { + return (List) super.values(); + } + + @Override + List createValues() { + class ValuesImpl extends AbstractSequentialList { + @Override + public int size() { + return size; + } + + @Override + public ListIterator listIterator(int index) { + final NodeIterator nodeItr = new NodeIterator(index); + return new TransformedListIterator, V>(nodeItr) { + @Override + V transform(Entry entry) { + return entry.getValue(); + } + + @Override + public void set(V value) { + nodeItr.setValue(value); + } + }; + } + } + return new ValuesImpl(); + } + + /** + * {@inheritDoc} + * + *

The iterator generated by the returned collection traverses the entries in the order they + * were added to the multimap. Because the entries may have duplicates and follow the insertion + * ordering, this method returns a {@link List}, instead of the {@link Collection} specified in + * the {@link ListMultimap} interface. + * + *

An entry's {@link Entry#getKey} method always returns the same key, regardless of what + * happens subsequently. As long as the corresponding key-value mapping is not removed from the + * multimap, {@link Entry#getValue} returns the value from the multimap, which may change over + * time, and {@link Entry#setValue} modifies that value. Removing the mapping from the multimap + * does not alter the value returned by {@code getValue()}, though a subsequent {@code setValue()} + * call won't update the multimap but will lead to a revised value being returned by {@code + * getValue()}. + */ + @Override + public List> entries() { + return (List>) super.entries(); + } + + @Override + List> createEntries() { + class EntriesImpl extends AbstractSequentialList> { + @Override + public int size() { + return size; + } + + @Override + public ListIterator> listIterator(int index) { + return new NodeIterator(index); + } + + @Override + public void forEach(Consumer> action) { + checkNotNull(action); + for (Node node = head; node != null; node = node.next) { + action.accept(node); + } + } + } + return new EntriesImpl(); + } + + @Override + Iterator> entryIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Map> createAsMap() { + return new Multimaps.AsMap<>(this); + } + + /** + * @serialData the number of distinct keys, and then for each distinct key: the first key, the + * number of values for that key, and the key's values, followed by successive keys and values + * from the entries() ordering + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeInt(size()); + for (Entry entry : entries()) { + stream.writeObject(entry.getKey()); + stream.writeObject(entry.getValue()); + } + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + keyToKeyList = Maps.newLinkedHashMap(); + int size = stream.readInt(); + for (int i = 0; i < size; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + K key = (K) stream.readObject(); + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + V value = (V) stream.readObject(); + put(key, value); + } + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ListMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ListMultimap.java new file mode 100644 index 0000000000000..7bd27b9be1e2e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ListMultimap.java @@ -0,0 +1,96 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.List; +import java.util.Map; + + +/** + * A {@code Multimap} that can hold duplicate key-value pairs and that maintains the insertion + * ordering of values for a given key. See the {@link Multimap} documentation for information common + * to all multimaps. + * + *

The {@link #get}, {@link #removeAll}, and {@link #replaceValues} methods each return a {@link + * List} of values. Though the method signature doesn't say so explicitly, the map returned by + * {@link #asMap} has {@code List} values. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @since 2.0 + */ +public interface ListMultimap extends Multimap { + /** + * {@inheritDoc} + * + *

Because the values for a given key may have duplicates and follow the insertion ordering, + * this method returns a {@link List}, instead of the {@link Collection} specified in + * the {@link Multimap} interface. + */ + @Override + List get(K key); + + /** + * {@inheritDoc} + * + *

Because the values for a given key may have duplicates and follow the insertion ordering, + * this method returns a {@link List}, instead of the {@link Collection} specified in + * the {@link Multimap} interface. + */ + @Override + List removeAll(Object key); + + /** + * {@inheritDoc} + * + *

Because the values for a given key may have duplicates and follow the insertion ordering, + * this method returns a {@link List}, instead of the {@link Collection} specified in + * the {@link Multimap} interface. + */ + @Override + List replaceValues(K key, Iterable values); + + /** + * {@inheritDoc} + * + *

Note: The returned map's values are guaranteed to be of type {@link List}. To obtain + * this map with the more specific generic type {@code Map>}, call {@link + * Multimaps#asMap(ListMultimap)} instead. + */ + @Override + Map> asMap(); + + /** + * Compares the specified object to this multimap for equality. + * + *

Two {@code ListMultimap} instances are equal if, for each key, they contain the same values + * in the same order. If the value orderings disagree, the multimaps will not be considered equal. + * + *

An empty {@code ListMultimap} is equal to any other empty {@code Multimap}, including an + * empty {@code SetMultimap}. + */ + @Override + boolean equals(Object obj); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Lists.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Lists.java new file mode 100644 index 0000000000000..67c59d4005d99 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Lists.java @@ -0,0 +1,1143 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.Serializable; +import java.math.RoundingMode; +import java.util.AbstractList; +import java.util.AbstractSequentialList; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; +import java.util.NoSuchElementException; +import java.util.RandomAccess; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Predicate; + + +/** + * Static utility methods pertaining to {@link List} instances. Also see this class's counterparts + * {@link Sets}, {@link Maps} and {@link Queues}. + * + *

See the Guava User Guide article on {@code Lists}. + * + * @author Kevin Bourrillion + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public final class Lists { + private Lists() {} + + // ArrayList + + /** + * Creates a mutable, empty {@code ArrayList} instance (for Java 6 and earlier). + * + *

Note: if mutability is not required, use {@link ImmutableList#of()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code ArrayList} {@linkplain ArrayList#ArrayList() constructor} + * directly, taking advantage of the new "diamond" syntax. + */ + public static ArrayList newArrayList() { + return new ArrayList<>(); + } + + /** + * Creates a mutable {@code ArrayList} instance containing the given elements. + * + *

Note: essentially the only reason to use this method is when you will need to add or + * remove elements later. Otherwise, for non-null elements use {@link ImmutableList#of()} (for + * varargs) or {@link ImmutableList#copyOf(Object[])} (for an array) instead. If any elements + * might be null, or you need support for {@link List#set(int, Object)}, use {@link + * Arrays#asList}. + * + *

Note that even when you do need the ability to add or remove, this method provides only a + * tiny bit of syntactic sugar for {@code newArrayList(}{@link Arrays#asList asList}{@code + * (...))}, or for creating an empty list then calling {@link Collections#addAll}. This method is + * not actually very useful and will likely be deprecated in the future. + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static ArrayList newArrayList(E... elements) { + checkNotNull(elements); // for GWT + // Avoid integer overflow when a large array is passed in + int capacity = computeArrayListCapacity(elements.length); + ArrayList list = new ArrayList<>(capacity); + Collections.addAll(list, elements); + return list; + } + + /** + * Creates a mutable {@code ArrayList} instance containing the given elements; a very thin + * shortcut for creating an empty list then calling {@link Iterables#addAll}. + * + *

Note: if mutability is not required and the elements are non-null, use {@link + * ImmutableList#copyOf(Iterable)} instead. (Or, change {@code elements} to be a {@link + * FluentIterable} and call {@code elements.toList()}.) + * + *

Note for Java 7 and later: if {@code elements} is a {@link Collection}, you don't + * need this method. Use the {@code ArrayList} {@linkplain ArrayList#ArrayList(Collection) + * constructor} directly, taking advantage of the new "diamond" + * syntax. + */ + public static ArrayList newArrayList(Iterable elements) { + checkNotNull(elements); // for GWT + // Let ArrayList's sizing logic work, if possible + return (elements instanceof Collection) + ? new ArrayList<>(Collections2.cast(elements)) + : newArrayList(elements.iterator()); + } + + /** + * Creates a mutable {@code ArrayList} instance containing the given elements; a very thin + * shortcut for creating an empty list and then calling {@link Iterators#addAll}. + * + *

Note: if mutability is not required and the elements are non-null, use {@link + * ImmutableList#copyOf(Iterator)} instead. + */ + public static ArrayList newArrayList(Iterator elements) { + ArrayList list = newArrayList(); + Iterators.addAll(list, elements); + return list; + } + + + static int computeArrayListCapacity(int arraySize) { + checkNonnegative(arraySize, "arraySize"); + + // TODO(kevinb): Figure out the right behavior, and document it + return Ints.saturatedCast(5L + arraySize + (arraySize / 10)); + } + + /** + * Creates an {@code ArrayList} instance backed by an array with the specified initial size; + * simply delegates to {@link ArrayList#ArrayList(int)}. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use {@code new }{@link ArrayList#ArrayList(int) ArrayList}{@code <>(int)} + * directly, taking advantage of the new "diamond" syntax. + * (Unlike here, there is no risk of overload ambiguity, since the {@code ArrayList} constructors + * very wisely did not accept varargs.) + * + * @param initialArraySize the exact size of the initial backing array for the returned array list + * ({@code ArrayList} documentation calls this value the "capacity") + * @return a new, empty {@code ArrayList} which is guaranteed not to resize itself unless its size + * reaches {@code initialArraySize + 1} + * @throws IllegalArgumentException if {@code initialArraySize} is negative + */ + public static ArrayList newArrayListWithCapacity(int initialArraySize) { + checkNonnegative(initialArraySize, "initialArraySize"); // for GWT. + return new ArrayList<>(initialArraySize); + } + + /** + * Creates an {@code ArrayList} instance to hold {@code estimatedSize} elements, plus an + * unspecified amount of padding; you almost certainly mean to call {@link + * #newArrayListWithCapacity} (see that method for further advice on usage). + * + *

Note: This method will soon be deprecated. Even in the rare case that you do want + * some amount of padding, it's best if you choose your desired amount explicitly. + * + * @param estimatedSize an estimate of the eventual {@link List#size()} of the new list + * @return a new, empty {@code ArrayList}, sized appropriately to hold the estimated number of + * elements + * @throws IllegalArgumentException if {@code estimatedSize} is negative + */ + public static ArrayList newArrayListWithExpectedSize(int estimatedSize) { + return new ArrayList<>(computeArrayListCapacity(estimatedSize)); + } + + // LinkedList + + /** + * Creates a mutable, empty {@code LinkedList} instance (for Java 6 and earlier). + * + *

Note: if you won't be adding any elements to the list, use {@link ImmutableList#of()} + * instead. + * + *

Performance note: {@link ArrayList} and {@link java.util.ArrayDeque} consistently + * outperform {@code LinkedList} except in certain rare and specific situations. Unless you have + * spent a lot of time benchmarking your specific needs, use one of those instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code LinkedList} {@linkplain LinkedList#LinkedList() + * constructor} directly, taking advantage of the new "diamond" + * syntax. + */ + public static LinkedList newLinkedList() { + return new LinkedList<>(); + } + + /** + * Creates a mutable {@code LinkedList} instance containing the given elements; a very thin + * shortcut for creating an empty list then calling {@link Iterables#addAll}. + * + *

Note: if mutability is not required and the elements are non-null, use {@link + * ImmutableList#copyOf(Iterable)} instead. (Or, change {@code elements} to be a {@link + * FluentIterable} and call {@code elements.toList()}.) + * + *

Performance note: {@link ArrayList} and {@link java.util.ArrayDeque} consistently + * outperform {@code LinkedList} except in certain rare and specific situations. Unless you have + * spent a lot of time benchmarking your specific needs, use one of those instead. + * + *

Note for Java 7 and later: if {@code elements} is a {@link Collection}, you don't + * need this method. Use the {@code LinkedList} {@linkplain LinkedList#LinkedList(Collection) + * constructor} directly, taking advantage of the new "diamond" + * syntax. + */ + public static LinkedList newLinkedList(Iterable elements) { + LinkedList list = newLinkedList(); + Iterables.addAll(list, elements); + return list; + } + + /** + * Creates an empty {@code CopyOnWriteArrayList} instance. + * + *

Note: if you need an immutable empty {@link List}, use {@link Collections#emptyList} + * instead. + * + * @return a new, empty {@code CopyOnWriteArrayList} + * @since 12.0 + */ + public static CopyOnWriteArrayList newCopyOnWriteArrayList() { + return new CopyOnWriteArrayList<>(); + } + + /** + * Creates a {@code CopyOnWriteArrayList} instance containing the given elements. + * + * @param elements the elements that the list should contain, in order + * @return a new {@code CopyOnWriteArrayList} containing those elements + * @since 12.0 + */ + public static CopyOnWriteArrayList newCopyOnWriteArrayList( + Iterable elements) { + // We copy elements to an ArrayList first, rather than incurring the + // quadratic cost of adding them to the COWAL directly. + Collection elementsCollection = + (elements instanceof Collection) ? Collections2.cast(elements) : newArrayList(elements); + return new CopyOnWriteArrayList<>(elementsCollection); + } + + /** + * Returns an unmodifiable list containing the specified first element and backed by the specified + * array of additional elements. Changes to the {@code rest} array will be reflected in the + * returned list. Unlike {@link Arrays#asList}, the returned list is unmodifiable. + * + *

This is useful when a varargs method needs to use a signature such as {@code (Foo firstFoo, + * Foo... moreFoos)}, in order to avoid overload ambiguity or to enforce a minimum argument count. + * + *

The returned list is serializable and implements {@link RandomAccess}. + * + * @param first the first element + * @param rest an array of additional elements, possibly empty + * @return an unmodifiable list containing the specified elements + */ + public static List asList(E first, E[] rest) { + return new OnePlusArrayList<>(first, rest); + } + + /** + * Returns an unmodifiable list containing the specified first and second element, and backed by + * the specified array of additional elements. Changes to the {@code rest} array will be reflected + * in the returned list. Unlike {@link Arrays#asList}, the returned list is unmodifiable. + * + *

This is useful when a varargs method needs to use a signature such as {@code (Foo firstFoo, + * Foo secondFoo, Foo... moreFoos)}, in order to avoid overload ambiguity or to enforce a minimum + * argument count. + * + *

The returned list is serializable and implements {@link RandomAccess}. + * + * @param first the first element + * @param second the second element + * @param rest an array of additional elements, possibly empty + * @return an unmodifiable list containing the specified elements + */ + public static List asList(E first, E second, E[] rest) { + return new TwoPlusArrayList<>(first, second, rest); + } + + /** @see Lists#asList(Object, Object[]) */ + private static class OnePlusArrayList extends AbstractList + implements Serializable, RandomAccess { + final E first; + final E[] rest; + + OnePlusArrayList(E first, E[] rest) { + this.first = first; + this.rest = checkNotNull(rest); + } + + @Override + public int size() { + return IntMath.saturatedAdd(rest.length, 1); + } + + @Override + public E get(int index) { + // check explicitly so the IOOBE will have the right message + checkElementIndex(index, size()); + return (index == 0) ? first : rest[index - 1]; + } + + private static final long serialVersionUID = 0; + } + + /** @see Lists#asList(Object, Object, Object[]) */ + private static class TwoPlusArrayList extends AbstractList + implements Serializable, RandomAccess { + final E first; + final E second; + final E[] rest; + + TwoPlusArrayList(E first, E second, E[] rest) { + this.first = first; + this.second = second; + this.rest = checkNotNull(rest); + } + + @Override + public int size() { + return IntMath.saturatedAdd(rest.length, 2); + } + + @Override + public E get(int index) { + switch (index) { + case 0: + return first; + case 1: + return second; + default: + // check explicitly so the IOOBE will have the right message + checkElementIndex(index, size()); + return rest[index - 2]; + } + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns every possible list that can be formed by choosing one element from each of the given + * lists in order; the "n-ary Cartesian + * product" of the lists. For example: + * + *

{@code
+   * Lists.cartesianProduct(ImmutableList.of(
+   *     ImmutableList.of(1, 2),
+   *     ImmutableList.of("A", "B", "C")))
+   * }
+ * + *

returns a list containing six lists in the following order: + * + *

    + *
  • {@code ImmutableList.of(1, "A")} + *
  • {@code ImmutableList.of(1, "B")} + *
  • {@code ImmutableList.of(1, "C")} + *
  • {@code ImmutableList.of(2, "A")} + *
  • {@code ImmutableList.of(2, "B")} + *
  • {@code ImmutableList.of(2, "C")} + *
+ * + *

The result is guaranteed to be in the "traditional", lexicographical order for Cartesian + * products that you would get from nesting for loops: + * + *

{@code
+   * for (B b0 : lists.get(0)) {
+   *   for (B b1 : lists.get(1)) {
+   *     ...
+   *     ImmutableList tuple = ImmutableList.of(b0, b1, ...);
+   *     // operate on tuple
+   *   }
+   * }
+   * }
+ * + *

Note that if any input list is empty, the Cartesian product will also be empty. If no lists + * at all are provided (an empty list), the resulting Cartesian product has one element, an empty + * list (counter-intuitive, but mathematically consistent). + * + *

Performance notes: while the cartesian product of lists of size {@code m, n, p} is a + * list of size {@code m x n x p}, its actual memory consumption is much smaller. When the + * cartesian product is constructed, the input lists are merely copied. Only as the resulting list + * is iterated are the individual lists created, and these are not retained after iteration. + * + * @param lists the lists to choose elements from, in the order that the elements chosen from + * those lists should appear in the resulting lists + * @param any common base class shared by all axes (often just {@link Object}) + * @return the Cartesian product, as an immutable list containing immutable lists + * @throws IllegalArgumentException if the size of the cartesian product would be greater than + * {@link Integer#MAX_VALUE} + * @throws NullPointerException if {@code lists}, any one of the {@code lists}, or any element of + * a provided list is null + * @since 19.0 + */ + public static List> cartesianProduct(List> lists) { + return CartesianList.create(lists); + } + + /** + * Returns every possible list that can be formed by choosing one element from each of the given + * lists in order; the "n-ary Cartesian + * product" of the lists. For example: + * + *

{@code
+   * Lists.cartesianProduct(ImmutableList.of(
+   *     ImmutableList.of(1, 2),
+   *     ImmutableList.of("A", "B", "C")))
+   * }
+ * + *

returns a list containing six lists in the following order: + * + *

    + *
  • {@code ImmutableList.of(1, "A")} + *
  • {@code ImmutableList.of(1, "B")} + *
  • {@code ImmutableList.of(1, "C")} + *
  • {@code ImmutableList.of(2, "A")} + *
  • {@code ImmutableList.of(2, "B")} + *
  • {@code ImmutableList.of(2, "C")} + *
+ * + *

The result is guaranteed to be in the "traditional", lexicographical order for Cartesian + * products that you would get from nesting for loops: + * + *

{@code
+   * for (B b0 : lists.get(0)) {
+   *   for (B b1 : lists.get(1)) {
+   *     ...
+   *     ImmutableList tuple = ImmutableList.of(b0, b1, ...);
+   *     // operate on tuple
+   *   }
+   * }
+   * }
+ * + *

Note that if any input list is empty, the Cartesian product will also be empty. If no lists + * at all are provided (an empty list), the resulting Cartesian product has one element, an empty + * list (counter-intuitive, but mathematically consistent). + * + *

Performance notes: while the cartesian product of lists of size {@code m, n, p} is a + * list of size {@code m x n x p}, its actual memory consumption is much smaller. When the + * cartesian product is constructed, the input lists are merely copied. Only as the resulting list + * is iterated are the individual lists created, and these are not retained after iteration. + * + * @param lists the lists to choose elements from, in the order that the elements chosen from + * those lists should appear in the resulting lists + * @param any common base class shared by all axes (often just {@link Object}) + * @return the Cartesian product, as an immutable list containing immutable lists + * @throws IllegalArgumentException if the size of the cartesian product would be greater than + * {@link Integer#MAX_VALUE} + * @throws NullPointerException if {@code lists}, any one of the {@code lists}, or any element of + * a provided list is null + * @since 19.0 + */ + @SafeVarargs + @SuppressWarnings("varargs") + public static List> cartesianProduct(List... lists) { + return cartesianProduct(Arrays.asList(lists)); + } + + /** + * Returns a list that applies {@code function} to each element of {@code fromList}. The returned + * list is a transformed view of {@code fromList}; changes to {@code fromList} will be reflected + * in the returned list and vice versa. + * + *

Since functions are not reversible, the transform is one-way and new items cannot be stored + * in the returned list. The {@code add}, {@code addAll} and {@code set} methods are unsupported + * in the returned list. + * + *

The function is applied lazily, invoked when needed. This is necessary for the returned list + * to be a view, but it means that the function will be applied many times for bulk operations + * like {@link List#contains} and {@link List#hashCode}. For this to perform well, {@code + * function} should be fast. To avoid lazy evaluation when the returned list doesn't need to be a + * view, copy the returned list into a new list of your choosing. + * + *

If {@code fromList} implements {@link RandomAccess}, so will the returned list. The returned + * list is threadsafe if the supplied list and function are. + * + *

If only a {@code Collection} or {@code Iterable} input is available, use {@link + * Collections2#transform} or {@link Iterables#transform}. + * + *

Note: serializing the returned list is implemented by serializing {@code fromList}, + * its contents, and {@code function} -- not by serializing the transformed values. This + * can lead to surprising behavior, so serializing the returned list is not recommended. + * Instead, copy the list using {@link ImmutableList#copyOf(Collection)} (for example), then + * serialize the copy. Other methods similar to this do not implement serialization at all for + * this reason. + * + *

Java 8 users: many use cases for this method are better addressed by {@link + * java.util.stream.Stream#map}. This method is not being deprecated, but we gently encourage you + * to migrate to streams. + */ + public static List transform( + List fromList, Function function) { + return (fromList instanceof RandomAccess) + ? new TransformingRandomAccessList<>(fromList, function) + : new TransformingSequentialList<>(fromList, function); + } + + /** + * Implementation of a sequential transforming list. + * + * @see Lists#transform + */ + private static class TransformingSequentialList extends AbstractSequentialList + implements Serializable { + final List fromList; + final Function function; + + TransformingSequentialList(List fromList, Function function) { + this.fromList = checkNotNull(fromList); + this.function = checkNotNull(function); + } + + /** + * The default implementation inherited is based on iteration and removal of each element which + * can be overkill. That's why we forward this call directly to the backing list. + */ + @Override + public void clear() { + fromList.clear(); + } + + @Override + public int size() { + return fromList.size(); + } + + @Override + public ListIterator listIterator(final int index) { + return new TransformedListIterator(fromList.listIterator(index)) { + @Override + T transform(F from) { + return function.apply(from); + } + }; + } + + @Override + public boolean removeIf(Predicate filter) { + checkNotNull(filter); + return fromList.removeIf(element -> filter.test(function.apply(element))); + } + + private static final long serialVersionUID = 0; + } + + /** + * Implementation of a transforming random access list. We try to make as many of these methods + * pass-through to the source list as possible so that the performance characteristics of the + * source list and transformed list are similar. + * + * @see Lists#transform + */ + private static class TransformingRandomAccessList extends AbstractList + implements RandomAccess, Serializable { + final List fromList; + final Function function; + + TransformingRandomAccessList(List fromList, Function function) { + this.fromList = checkNotNull(fromList); + this.function = checkNotNull(function); + } + + @Override + public void clear() { + fromList.clear(); + } + + @Override + public T get(int index) { + return function.apply(fromList.get(index)); + } + + @Override + public Iterator iterator() { + return listIterator(); + } + + @Override + public ListIterator listIterator(int index) { + return new TransformedListIterator(fromList.listIterator(index)) { + @Override + T transform(F from) { + return function.apply(from); + } + }; + } + + @Override + public boolean isEmpty() { + return fromList.isEmpty(); + } + + @Override + public boolean removeIf(Predicate filter) { + checkNotNull(filter); + return fromList.removeIf(element -> filter.test(function.apply(element))); + } + + @Override + public T remove(int index) { + return function.apply(fromList.remove(index)); + } + + @Override + public int size() { + return fromList.size(); + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns consecutive {@linkplain List#subList(int, int) sublists} of a list, each of the same + * size (the final list may be smaller). For example, partitioning a list containing {@code [a, b, + * c, d, e]} with a partition size of 3 yields {@code [[a, b, c], [d, e]]} -- an outer list + * containing two inner lists of three and two elements, all in the original order. + * + *

The outer list is unmodifiable, but reflects the latest state of the source list. The inner + * lists are sublist views of the original list, produced on demand using {@link List#subList(int, + * int)}, and are subject to all the usual caveats about modification as explained in that API. + * + * @param list the list to return consecutive sublists of + * @param size the desired size of each sublist (the last may be smaller) + * @return a list of consecutive sublists + * @throws IllegalArgumentException if {@code partitionSize} is nonpositive + */ + public static List> partition(List list, int size) { + checkNotNull(list); + checkArgument(size > 0); + return (list instanceof RandomAccess) + ? new RandomAccessPartition<>(list, size) + : new Partition<>(list, size); + } + + private static class Partition extends AbstractList> { + final List list; + final int size; + + Partition(List list, int size) { + this.list = list; + this.size = size; + } + + @Override + public List get(int index) { + checkElementIndex(index, size()); + int start = index * size; + int end = Math.min(start + size, list.size()); + return list.subList(start, end); + } + + @Override + public int size() { + return IntMath.divide(list.size(), size, RoundingMode.CEILING); + } + + @Override + public boolean isEmpty() { + return list.isEmpty(); + } + } + + private static class RandomAccessPartition extends Partition implements RandomAccess { + RandomAccessPartition(List list, int size) { + super(list, size); + } + } + + /** + * Returns a view of the specified string as an immutable list of {@code Character} values. + * + * @since 7.0 + */ + public static ImmutableList charactersOf(String string) { + return new StringAsImmutableList(checkNotNull(string)); + } + + /** + * Returns a view of the specified {@code CharSequence} as a {@code List}, viewing + * {@code sequence} as a sequence of Unicode code units. The view does not support any + * modification operations, but reflects any changes to the underlying character sequence. + * + * @param sequence the character sequence to view as a {@code List} of characters + * @return an {@code List} view of the character sequence + * @since 7.0 + */ + + public static List charactersOf(CharSequence sequence) { + return new CharSequenceAsList(checkNotNull(sequence)); + } + + @SuppressWarnings("serial") // serialized using ImmutableList serialization + private static final class StringAsImmutableList extends ImmutableList { + + private final String string; + + StringAsImmutableList(String string) { + this.string = string; + } + + @Override + public int indexOf(Object object) { + return (object instanceof Character) ? string.indexOf((Character) object) : -1; + } + + @Override + public int lastIndexOf(Object object) { + return (object instanceof Character) ? string.lastIndexOf((Character) object) : -1; + } + + @Override + public ImmutableList subList(int fromIndex, int toIndex) { + checkPositionIndexes(fromIndex, toIndex, size()); // for GWT + return charactersOf(string.substring(fromIndex, toIndex)); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public Character get(int index) { + checkElementIndex(index, size()); // for GWT + return string.charAt(index); + } + + @Override + public int size() { + return string.length(); + } + } + + private static final class CharSequenceAsList extends AbstractList { + private final CharSequence sequence; + + CharSequenceAsList(CharSequence sequence) { + this.sequence = sequence; + } + + @Override + public Character get(int index) { + checkElementIndex(index, size()); // for GWT + return sequence.charAt(index); + } + + @Override + public int size() { + return sequence.length(); + } + } + + /** + * Returns a reversed view of the specified list. For example, {@code + * Lists.reverse(Arrays.asList(1, 2, 3))} returns a list containing {@code 3, 2, 1}. The returned + * list is backed by this list, so changes in the returned list are reflected in this list, and + * vice-versa. The returned list supports all of the optional list operations supported by this + * list. + * + *

The returned list is random-access if the specified list is random access. + * + * @since 7.0 + */ + public static List reverse(List list) { + if (list instanceof ImmutableList) { + return ((ImmutableList) list).reverse(); + } else if (list instanceof ReverseList) { + return ((ReverseList) list).getForwardList(); + } else if (list instanceof RandomAccess) { + return new RandomAccessReverseList<>(list); + } else { + return new ReverseList<>(list); + } + } + + private static class ReverseList extends AbstractList { + private final List forwardList; + + ReverseList(List forwardList) { + this.forwardList = checkNotNull(forwardList); + } + + List getForwardList() { + return forwardList; + } + + private int reverseIndex(int index) { + int size = size(); + checkElementIndex(index, size); + return (size - 1) - index; + } + + private int reversePosition(int index) { + int size = size(); + checkPositionIndex(index, size); + return size - index; + } + + @Override + public void add(int index, T element) { + forwardList.add(reversePosition(index), element); + } + + @Override + public void clear() { + forwardList.clear(); + } + + @Override + public T remove(int index) { + return forwardList.remove(reverseIndex(index)); + } + + @Override + protected void removeRange(int fromIndex, int toIndex) { + subList(fromIndex, toIndex).clear(); + } + + @Override + public T set(int index, T element) { + return forwardList.set(reverseIndex(index), element); + } + + @Override + public T get(int index) { + return forwardList.get(reverseIndex(index)); + } + + @Override + public int size() { + return forwardList.size(); + } + + @Override + public List subList(int fromIndex, int toIndex) { + checkPositionIndexes(fromIndex, toIndex, size()); + return reverse(forwardList.subList(reversePosition(toIndex), reversePosition(fromIndex))); + } + + @Override + public Iterator iterator() { + return listIterator(); + } + + @Override + public ListIterator listIterator(int index) { + int start = reversePosition(index); + final ListIterator forwardIterator = forwardList.listIterator(start); + return new ListIterator() { + + boolean canRemoveOrSet; + + @Override + public void add(T e) { + forwardIterator.add(e); + forwardIterator.previous(); + canRemoveOrSet = false; + } + + @Override + public boolean hasNext() { + return forwardIterator.hasPrevious(); + } + + @Override + public boolean hasPrevious() { + return forwardIterator.hasNext(); + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + canRemoveOrSet = true; + return forwardIterator.previous(); + } + + @Override + public int nextIndex() { + return reversePosition(forwardIterator.nextIndex()); + } + + @Override + public T previous() { + if (!hasPrevious()) { + throw new NoSuchElementException(); + } + canRemoveOrSet = true; + return forwardIterator.next(); + } + + @Override + public int previousIndex() { + return nextIndex() - 1; + } + + @Override + public void remove() { + checkRemove(canRemoveOrSet); + forwardIterator.remove(); + canRemoveOrSet = false; + } + + @Override + public void set(T e) { + checkState(canRemoveOrSet); + forwardIterator.set(e); + } + }; + } + } + + private static class RandomAccessReverseList extends ReverseList implements RandomAccess { + RandomAccessReverseList(List forwardList) { + super(forwardList); + } + } + + /** An implementation of {@link List#hashCode()}. */ + static int hashCodeImpl(List list) { + // TODO(lowasser): worth optimizing for RandomAccess? + int hashCode = 1; + for (Object o : list) { + hashCode = 31 * hashCode + (o == null ? 0 : o.hashCode()); + + hashCode = ~~hashCode; + // needed to deal with GWT integer overflow + } + return hashCode; + } + + /** An implementation of {@link List#equals(Object)}. */ + static boolean equalsImpl(List thisList, Object other) { + if (other == checkNotNull(thisList)) { + return true; + } + if (!(other instanceof List)) { + return false; + } + List otherList = (List) other; + int size = thisList.size(); + if (size != otherList.size()) { + return false; + } + if (thisList instanceof RandomAccess && otherList instanceof RandomAccess) { + // avoid allocation and use the faster loop + for (int i = 0; i < size; i++) { + if (!Objects.equal(thisList.get(i), otherList.get(i))) { + return false; + } + } + return true; + } else { + return Iterators.elementsEqual(thisList.iterator(), otherList.iterator()); + } + } + + /** An implementation of {@link List#addAll(int, Collection)}. */ + static boolean addAllImpl(List list, int index, Iterable elements) { + boolean changed = false; + ListIterator listIterator = list.listIterator(index); + for (E e : elements) { + listIterator.add(e); + changed = true; + } + return changed; + } + + /** An implementation of {@link List#indexOf(Object)}. */ + static int indexOfImpl(List list, Object element) { + if (list instanceof RandomAccess) { + return indexOfRandomAccess(list, element); + } else { + ListIterator listIterator = list.listIterator(); + while (listIterator.hasNext()) { + if (Objects.equal(element, listIterator.next())) { + return listIterator.previousIndex(); + } + } + return -1; + } + } + + private static int indexOfRandomAccess(List list, Object element) { + int size = list.size(); + if (element == null) { + for (int i = 0; i < size; i++) { + if (list.get(i) == null) { + return i; + } + } + } else { + for (int i = 0; i < size; i++) { + if (element.equals(list.get(i))) { + return i; + } + } + } + return -1; + } + + /** An implementation of {@link List#lastIndexOf(Object)}. */ + static int lastIndexOfImpl(List list, Object element) { + if (list instanceof RandomAccess) { + return lastIndexOfRandomAccess(list, element); + } else { + ListIterator listIterator = list.listIterator(list.size()); + while (listIterator.hasPrevious()) { + if (Objects.equal(element, listIterator.previous())) { + return listIterator.nextIndex(); + } + } + return -1; + } + } + + private static int lastIndexOfRandomAccess(List list, Object element) { + if (element == null) { + for (int i = list.size() - 1; i >= 0; i--) { + if (list.get(i) == null) { + return i; + } + } + } else { + for (int i = list.size() - 1; i >= 0; i--) { + if (element.equals(list.get(i))) { + return i; + } + } + } + return -1; + } + + /** Returns an implementation of {@link List#listIterator(int)}. */ + static ListIterator listIteratorImpl(List list, int index) { + return new AbstractListWrapper<>(list).listIterator(index); + } + + /** An implementation of {@link List#subList(int, int)}. */ + static List subListImpl(final List list, int fromIndex, int toIndex) { + List wrapper; + if (list instanceof RandomAccess) { + wrapper = + new RandomAccessListWrapper(list) { + @Override + public ListIterator listIterator(int index) { + return backingList.listIterator(index); + } + + private static final long serialVersionUID = 0; + }; + } else { + wrapper = + new AbstractListWrapper(list) { + @Override + public ListIterator listIterator(int index) { + return backingList.listIterator(index); + } + + private static final long serialVersionUID = 0; + }; + } + return wrapper.subList(fromIndex, toIndex); + } + + private static class AbstractListWrapper extends AbstractList { + final List backingList; + + AbstractListWrapper(List backingList) { + this.backingList = checkNotNull(backingList); + } + + @Override + public void add(int index, E element) { + backingList.add(index, element); + } + + @Override + public boolean addAll(int index, Collection c) { + return backingList.addAll(index, c); + } + + @Override + public E get(int index) { + return backingList.get(index); + } + + @Override + public E remove(int index) { + return backingList.remove(index); + } + + @Override + public E set(int index, E element) { + return backingList.set(index, element); + } + + @Override + public boolean contains(Object o) { + return backingList.contains(o); + } + + @Override + public int size() { + return backingList.size(); + } + } + + private static class RandomAccessListWrapper extends AbstractListWrapper + implements RandomAccess { + RandomAccessListWrapper(List backingList) { + super(backingList); + } + } + + /** Used to avoid http://bugs.sun.com/view_bug.do?bug_id=6558557 */ + static List cast(Iterable iterable) { + return (List) iterable; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapDifference.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapDifference.java new file mode 100644 index 0000000000000..9473a0cde5a36 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapDifference.java @@ -0,0 +1,109 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Map; + + +/** + * An object representing the differences between two maps. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public interface MapDifference { + /** + * Returns {@code true} if there are no differences between the two maps; that is, if the maps are + * equal. + */ + boolean areEqual(); + + /** + * Returns an unmodifiable map containing the entries from the left map whose keys are not present + * in the right map. + */ + Map entriesOnlyOnLeft(); + + /** + * Returns an unmodifiable map containing the entries from the right map whose keys are not + * present in the left map. + */ + Map entriesOnlyOnRight(); + + /** + * Returns an unmodifiable map containing the entries that appear in both maps; that is, the + * intersection of the two maps. + */ + Map entriesInCommon(); + + /** + * Returns an unmodifiable map describing keys that appear in both maps, but with different + * values. + */ + Map> entriesDiffering(); + + /** + * Compares the specified object with this instance for equality. Returns {@code true} if the + * given object is also a {@code MapDifference} and the values returned by the {@link + * #entriesOnlyOnLeft()}, {@link #entriesOnlyOnRight()}, {@link #entriesInCommon()} and {@link + * #entriesDiffering()} of the two instances are equal. + */ + @Override + boolean equals(Object object); + + /** + * Returns the hash code for this instance. This is defined as the hash code of + * + *

{@code
+   * Arrays.asList(entriesOnlyOnLeft(), entriesOnlyOnRight(),
+   *     entriesInCommon(), entriesDiffering())
+   * }
+ */ + @Override + int hashCode(); + + /** + * A difference between the mappings from two maps with the same key. The {@link #leftValue} and + * {@link #rightValue} are not equal, and one but not both of them may be null. + * + * @since 2.0 + */ + interface ValueDifference { + /** Returns the value from the left map (possibly null). */ + V leftValue(); + + /** Returns the value from the right map (possibly null). */ + V rightValue(); + + /** + * Two instances are considered equal if their {@link #leftValue()} values are equal and their + * {@link #rightValue()} values are also equal. + */ + @Override + boolean equals(Object other); + + /** + * The hash code equals the value {@code Arrays.asList(leftValue(), rightValue()).hashCode()}. + */ + @Override + int hashCode(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMaker.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMaker.java new file mode 100644 index 0000000000000..b4a8ec2463c12 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMaker.java @@ -0,0 +1,302 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Ascii; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Equivalence; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapMakerInternalMap.Strength; +import java.lang.ref.WeakReference; +import java.util.ConcurrentModificationException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + + +/** + * A builder of {@link ConcurrentMap} instances that can have keys or values automatically wrapped + * in {@linkplain WeakReference weak} references. + * + *

Usage example: + * + *

{@code
+ * ConcurrentMap timers = new MapMaker()
+ *     .concurrencyLevel(4)
+ *     .weakKeys()
+ *     .makeMap();
+ * }
+ * + *

These features are all optional; {@code new MapMaker().makeMap()} returns a valid concurrent + * map that behaves similarly to a {@link ConcurrentHashMap}. + * + *

The returned map is implemented as a hash table with similar performance characteristics to + * {@link ConcurrentHashMap}. It supports all optional operations of the {@code ConcurrentMap} + * interface. It does not permit null keys or values. + * + *

Note: by default, the returned map uses equality comparisons (the {@link Object#equals + * equals} method) to determine equality for keys or values. However, if {@link #weakKeys} was + * specified, the map uses identity ({@code ==}) comparisons instead for keys. Likewise, if {@link + * #weakValues} was specified, the map uses identity comparisons for values. + * + *

The view collections of the returned map have weakly consistent iterators. This means + * that they are safe for concurrent use, but if other threads modify the map after the iterator is + * created, it is undefined which of these changes, if any, are reflected in that iterator. These + * iterators never throw {@link ConcurrentModificationException}. + * + *

If {@link #weakKeys} or {@link #weakValues} are requested, it is possible for a key or value + * present in the map to be reclaimed by the garbage collector. Entries with reclaimed keys or + * values may be removed from the map on each map modification or on occasional map accesses; such + * entries may be counted by {@link Map#size}, but will never be visible to read or write + * operations. A partially-reclaimed entry is never exposed to the user. Any {@link java.util.Entry} + * instance retrieved from the map's {@linkplain Map#entrySet entry set} is a snapshot of that + * entry's state at the time of retrieval; such entries do, however, support {@link + * java.util.Entry#setValue}, which simply calls {@link Map#put} on the entry's key. + * + *

The maps produced by {@code MapMaker} are serializable, and the deserialized maps retain all + * the configuration properties of the original map. During deserialization, if the original map had + * used weak references, the entries are reconstructed as they were, but it's not unlikely they'll + * be quickly garbage-collected before they are ever accessed. + * + *

{@code new MapMaker().weakKeys().makeMap()} is a recommended replacement for {@link + * java.util.WeakHashMap}, but note that it compares keys using object identity whereas {@code + * WeakHashMap} uses {@link Object#equals}. + * + * @author Bob Lee + * @author Charles Fry + * @author Kevin Bourrillion + * @since 2.0 + */ +public final class MapMaker { + private static final int DEFAULT_INITIAL_CAPACITY = 16; + private static final int DEFAULT_CONCURRENCY_LEVEL = 4; + + static final int UNSET_INT = -1; + + // TODO(kevinb): dispense with this after benchmarking + boolean useCustomMap; + + int initialCapacity = UNSET_INT; + int concurrencyLevel = UNSET_INT; + + Strength keyStrength; + Strength valueStrength; + + Equivalence keyEquivalence; + + /** + * Constructs a new {@code MapMaker} instance with default settings, including strong keys, strong + * values, and no automatic eviction of any kind. + */ + public MapMaker() {} + + /** + * Sets a custom {@code Equivalence} strategy for comparing keys. + * + *

By default, the map uses {@link Equivalence#identity} to determine key equality when {@link + * #weakKeys} is specified, and {@link Equivalence#equals()} otherwise. The only place this is + * used is in {@link Interners.WeakInterner}. + */ + MapMaker keyEquivalence(Equivalence equivalence) { + checkState(keyEquivalence == null, "key equivalence was already set to %s", keyEquivalence); + keyEquivalence = checkNotNull(equivalence); + this.useCustomMap = true; + return this; + } + + Equivalence getKeyEquivalence() { + return MoreObjects.firstNonNull(keyEquivalence, getKeyStrength().defaultEquivalence()); + } + + /** + * Sets the minimum total size for the internal hash tables. For example, if the initial capacity + * is {@code 60}, and the concurrency level is {@code 8}, then eight segments are created, each + * having a hash table of size eight. Providing a large enough estimate at construction time + * avoids the need for expensive resizing operations later, but setting this value unnecessarily + * high wastes memory. + * + * @throws IllegalArgumentException if {@code initialCapacity} is negative + * @throws IllegalStateException if an initial capacity was already set + */ + public MapMaker initialCapacity(int initialCapacity) { + checkState( + this.initialCapacity == UNSET_INT, + "initial capacity was already set to %s", + this.initialCapacity); + checkArgument(initialCapacity >= 0); + this.initialCapacity = initialCapacity; + return this; + } + + int getInitialCapacity() { + return (initialCapacity == UNSET_INT) ? DEFAULT_INITIAL_CAPACITY : initialCapacity; + } + + /** + * Guides the allowed concurrency among update operations. Used as a hint for internal sizing. The + * table is internally partitioned to try to permit the indicated number of concurrent updates + * without contention. Because assignment of entries to these partitions is not necessarily + * uniform, the actual concurrency observed may vary. Ideally, you should choose a value to + * accommodate as many threads as will ever concurrently modify the table. Using a significantly + * higher value than you need can waste space and time, and a significantly lower value can lead + * to thread contention. But overestimates and underestimates within an order of magnitude do not + * usually have much noticeable impact. A value of one permits only one thread to modify the map + * at a time, but since read operations can proceed concurrently, this still yields higher + * concurrency than full synchronization. Defaults to 4. + * + *

Note: Prior to Guava release 9.0, the default was 16. It is possible the default will + * change again in the future. If you care about this value, you should always choose it + * explicitly. + * + * @throws IllegalArgumentException if {@code concurrencyLevel} is nonpositive + * @throws IllegalStateException if a concurrency level was already set + */ + public MapMaker concurrencyLevel(int concurrencyLevel) { + checkState( + this.concurrencyLevel == UNSET_INT, + "concurrency level was already set to %s", + this.concurrencyLevel); + checkArgument(concurrencyLevel > 0); + this.concurrencyLevel = concurrencyLevel; + return this; + } + + int getConcurrencyLevel() { + return (concurrencyLevel == UNSET_INT) ? DEFAULT_CONCURRENCY_LEVEL : concurrencyLevel; + } + + /** + * Specifies that each key (not value) stored in the map should be wrapped in a {@link + * WeakReference} (by default, strong references are used). + * + *

Warning: when this method is used, the resulting map will use identity ({@code ==}) + * comparison to determine equality of keys, which is a technical violation of the {@link Map} + * specification, and may not be what you expect. + * + * @throws IllegalStateException if the key strength was already set + * @see WeakReference + */ + public MapMaker weakKeys() { + return setKeyStrength(Strength.WEAK); + } + + MapMaker setKeyStrength(Strength strength) { + checkState(keyStrength == null, "Key strength was already set to %s", keyStrength); + keyStrength = checkNotNull(strength); + if (strength != Strength.STRONG) { + // STRONG could be used during deserialization. + useCustomMap = true; + } + return this; + } + + Strength getKeyStrength() { + return MoreObjects.firstNonNull(keyStrength, Strength.STRONG); + } + + /** + * Specifies that each value (not key) stored in the map should be wrapped in a {@link + * WeakReference} (by default, strong references are used). + * + *

Weak values will be garbage collected once they are weakly reachable. This makes them a poor + * candidate for caching. + * + *

Warning: when this method is used, the resulting map will use identity ({@code ==}) + * comparison to determine equality of values. This technically violates the specifications of the + * methods {@link Map#containsValue containsValue}, {@link ConcurrentMap#remove(Object, Object) + * remove(Object, Object)} and {@link ConcurrentMap#replace(Object, Object, Object) replace(K, V, + * V)}, and may not be what you expect. + * + * @throws IllegalStateException if the value strength was already set + * @see WeakReference + */ + public MapMaker weakValues() { + return setValueStrength(Strength.WEAK); + } + + /** + * A dummy singleton value type used by {@link Interners}. + * + *

{@link MapMakerInternalMap} can optimize for memory usage in this case; see {@link + * MapMakerInternalMap#createWithDummyValues}. + */ + enum Dummy { + VALUE + } + + MapMaker setValueStrength(Strength strength) { + checkState(valueStrength == null, "Value strength was already set to %s", valueStrength); + valueStrength = checkNotNull(strength); + if (strength != Strength.STRONG) { + // STRONG could be used during deserialization. + useCustomMap = true; + } + return this; + } + + Strength getValueStrength() { + return MoreObjects.firstNonNull(valueStrength, Strength.STRONG); + } + + /** + * Builds a thread-safe map. This method does not alter the state of this {@code MapMaker} + * instance, so it can be invoked again to create multiple independent maps. + * + *

The bulk operations {@code putAll}, {@code equals}, and {@code clear} are not guaranteed to + * be performed atomically on the returned map. Additionally, {@code size} and {@code + * containsValue} are implemented as bulk read operations, and thus may fail to observe concurrent + * writes. + * + * @return a serializable concurrent map having the requested features + */ + public ConcurrentMap makeMap() { + if (!useCustomMap) { + return new ConcurrentHashMap<>(getInitialCapacity(), 0.75f, getConcurrencyLevel()); + } + return MapMakerInternalMap.create(this); + } + + /** + * Returns a string representation for this MapMaker instance. The exact form of the returned + * string is not specified. + */ + @Override + public String toString() { + MoreObjects.ToStringHelper s = MoreObjects.toStringHelper(this); + if (initialCapacity != UNSET_INT) { + s.add("initialCapacity", initialCapacity); + } + if (concurrencyLevel != UNSET_INT) { + s.add("concurrencyLevel", concurrencyLevel); + } + if (keyStrength != null) { + s.add("keyStrength", Ascii.toLowerCase(keyStrength.toString())); + } + if (valueStrength != null) { + s.add("valueStrength", Ascii.toLowerCase(valueStrength.toString())); + } + if (keyEquivalence != null) { + s.addValue("keyEquivalence"); + } + return s.toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMakerInternalMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMakerInternalMap.java new file mode 100644 index 0000000000000..a5bacd3c594a4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MapMakerInternalMap.java @@ -0,0 +1,2939 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Equivalence; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapMaker.Dummy; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.lang.ref.Reference; +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; +import java.util.AbstractCollection; +import java.util.AbstractMap; +import java.util.AbstractSet; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReferenceArray; +import java.util.concurrent.locks.ReentrantLock; + + + +/** + * The concurrent hash map implementation built by {@link MapMaker}. + * + *

This implementation is heavily derived from revision 1.96 of ConcurrentHashMap.java. + * + * @param the type of the keys in the map + * @param the type of the values in the map + * @param the type of the {@link InternalEntry} entry implementation used internally + * @param the type of the {@link Segment} entry implementation used internally + * @author Bob Lee + * @author Charles Fry + * @author Doug Lea ({@code ConcurrentHashMap}) + */ +@SuppressWarnings("GuardedBy") // TODO(b/35466881): Fix or suppress. +class MapMakerInternalMap< + K, + V, + E extends MapMakerInternalMap.InternalEntry, + S extends MapMakerInternalMap.Segment> + extends AbstractMap implements ConcurrentMap, Serializable { + + /* + * The basic strategy is to subdivide the table among Segments, each of which itself is a + * concurrently readable hash table. The map supports non-blocking reads and concurrent writes + * across different segments. + * + * The page replacement algorithm's data structures are kept casually consistent with the map. The + * ordering of writes to a segment is sequentially consistent. An update to the map and recording + * of reads may not be immediately reflected on the algorithm's data structures. These structures + * are guarded by a lock and operations are applied in batches to avoid lock contention. The + * penalty of applying the batches is spread across threads so that the amortized cost is slightly + * higher than performing just the operation without enforcing the capacity constraint. + * + * This implementation uses a per-segment queue to record a memento of the additions, removals, + * and accesses that were performed on the map. The queue is drained on writes and when it exceeds + * its capacity threshold. + * + * The Least Recently Used page replacement algorithm was chosen due to its simplicity, high hit + * rate, and ability to be implemented with O(1) time complexity. The initial LRU implementation + * operates per-segment rather than globally for increased implementation simplicity. We expect + * the cache hit rate to be similar to that of a global LRU algorithm. + */ + + // Constants + + /** + * The maximum capacity, used if a higher value is implicitly specified by either of the + * constructors with arguments. MUST be a power of two no greater than {@code 1<<30} to ensure + * that entries are indexable using ints. + */ + static final int MAXIMUM_CAPACITY = Ints.MAX_POWER_OF_TWO; + + /** The maximum number of segments to allow; used to bound constructor arguments. */ + static final int MAX_SEGMENTS = 1 << 16; // slightly conservative + + /** Number of (unsynchronized) retries in the containsValue method. */ + static final int CONTAINS_VALUE_RETRIES = 3; + + /** + * Number of cache access operations that can be buffered per segment before the cache's recency + * ordering information is updated. This is used to avoid lock contention by recording a memento + * of reads and delaying a lock acquisition until the threshold is crossed or a mutation occurs. + * + *

This must be a (2^n)-1 as it is used as a mask. + */ + static final int DRAIN_THRESHOLD = 0x3F; + + /** + * Maximum number of entries to be drained in a single cleanup run. This applies independently to + * the cleanup queue and both reference queues. + */ + // TODO(fry): empirically optimize this + static final int DRAIN_MAX = 16; + + static final long CLEANUP_EXECUTOR_DELAY_SECS = 60; + + // Fields + + /** + * Mask value for indexing into segments. The upper bits of a key's hash code are used to choose + * the segment. + */ + final transient int segmentMask; + + /** + * Shift value for indexing within segments. Helps prevent entries that end up in the same segment + * from also ending up in the same bucket. + */ + final transient int segmentShift; + + /** The segments, each of which is a specialized hash table. */ + final transient Segment[] segments; + + /** The concurrency level. */ + final int concurrencyLevel; + + /** Strategy for comparing keys. */ + final Equivalence keyEquivalence; + + /** Strategy for handling entries and segments in a type-safe and efficient manner. */ + final transient InternalEntryHelper entryHelper; + + /** + * Creates a new, empty map with the specified strategy, initial capacity and concurrency level. + */ + private MapMakerInternalMap(MapMaker builder, InternalEntryHelper entryHelper) { + concurrencyLevel = Math.min(builder.getConcurrencyLevel(), MAX_SEGMENTS); + + keyEquivalence = builder.getKeyEquivalence(); + this.entryHelper = entryHelper; + + int initialCapacity = Math.min(builder.getInitialCapacity(), MAXIMUM_CAPACITY); + + // Find power-of-two sizes best matching arguments. Constraints: + // (segmentCount > concurrencyLevel) + int segmentShift = 0; + int segmentCount = 1; + while (segmentCount < concurrencyLevel) { + ++segmentShift; + segmentCount <<= 1; + } + this.segmentShift = 32 - segmentShift; + segmentMask = segmentCount - 1; + + this.segments = newSegmentArray(segmentCount); + + int segmentCapacity = initialCapacity / segmentCount; + if (segmentCapacity * segmentCount < initialCapacity) { + ++segmentCapacity; + } + + int segmentSize = 1; + while (segmentSize < segmentCapacity) { + segmentSize <<= 1; + } + + for (int i = 0; i < this.segments.length; ++i) { + this.segments[i] = createSegment(segmentSize, MapMaker.UNSET_INT); + } + } + + /** Returns a fresh {@link MapMakerInternalMap} as specified by the given {@code builder}. */ + static MapMakerInternalMap, ?> create( + MapMaker builder) { + if (builder.getKeyStrength() == Strength.STRONG + && builder.getValueStrength() == Strength.STRONG) { + return new MapMakerInternalMap<>(builder, StrongKeyStrongValueEntry.Helper.instance()); + } + if (builder.getKeyStrength() == Strength.STRONG + && builder.getValueStrength() == Strength.WEAK) { + return new MapMakerInternalMap<>(builder, StrongKeyWeakValueEntry.Helper.instance()); + } + if (builder.getKeyStrength() == Strength.WEAK + && builder.getValueStrength() == Strength.STRONG) { + return new MapMakerInternalMap<>(builder, WeakKeyStrongValueEntry.Helper.instance()); + } + if (builder.getKeyStrength() == Strength.WEAK && builder.getValueStrength() == Strength.WEAK) { + return new MapMakerInternalMap<>(builder, WeakKeyWeakValueEntry.Helper.instance()); + } + throw new AssertionError(); + } + + /** + * Returns a fresh {@link MapMakerInternalMap} with {@link Dummy} values but otherwise as + * specified by the given {@code builder}. The returned {@link MapMakerInternalMap} will be + * optimized to saved memory. Since {@link Dummy} is a singleton, we don't need to store + * any values at all. Because of this optimization, {@code build.getValueStrength()} must be + * {@link Strength#STRONG}. + * + *

This method is intended to only be used by the internal implementation of {@link Interners}, + * since a map of dummy values is the exact use case there. + */ + static + MapMakerInternalMap, ?> createWithDummyValues( + MapMaker builder) { + if (builder.getKeyStrength() == Strength.STRONG + && builder.getValueStrength() == Strength.STRONG) { + return new MapMakerInternalMap<>(builder, StrongKeyDummyValueEntry.Helper.instance()); + } + if (builder.getKeyStrength() == Strength.WEAK + && builder.getValueStrength() == Strength.STRONG) { + return new MapMakerInternalMap<>(builder, WeakKeyDummyValueEntry.Helper.instance()); + } + if (builder.getValueStrength() == Strength.WEAK) { + throw new IllegalArgumentException("Map cannot have both weak and dummy values"); + } + throw new AssertionError(); + } + + enum Strength { + STRONG { + @Override + Equivalence defaultEquivalence() { + return Equivalence.equals(); + } + }, + + WEAK { + @Override + Equivalence defaultEquivalence() { + return Equivalence.identity(); + } + }; + + /** + * Returns the default equivalence strategy used to compare and hash keys or values referenced + * at this strength. This strategy will be used unless the user explicitly specifies an + * alternate strategy. + */ + abstract Equivalence defaultEquivalence(); + } + + /** + * A helper object for operating on {@link InternalEntry} instances in a type-safe and efficient + * manner. + * + *

For each of the four combinations of strong/weak key and strong/weak value, there are + * corresponding {@link InternalEntry}, {@link Segment}, and {@link InternalEntryHelper} + * implementations. + * + * @param the type of the key in each entry + * @param the type of the value in each entry + * @param the type of the {@link InternalEntry} entry implementation + * @param the type of the {@link Segment} entry implementation + */ + interface InternalEntryHelper< + K, V, E extends InternalEntry, S extends Segment> { + /** The strength of the key type in each entry. */ + Strength keyStrength(); + + /** The strength of the value type in each entry. */ + Strength valueStrength(); + + /** Returns a freshly created segment, typed at the {@code S} type. */ + S newSegment(MapMakerInternalMap map, int initialCapacity, int maxSegmentSize); + + /** + * Returns a freshly created entry, typed at the {@code E} type, for the given {@code segment}. + */ + E newEntry(S segment, K key, int hash, E next); + + /** + * Returns a freshly created entry, typed at the {@code E} type, for the given {@code segment}, + * that is a copy of the given {@code entry}. + */ + E copy(S segment, E entry, E newNext); + + /** + * Sets the value of the given {@code entry} in the given {@code segment} to be the given {@code + * value} + */ + void setValue(S segment, E entry, V value); + } + + /** + * An entry in a hash table of a {@link Segment}. + * + *

Entries in the map can be in the following states: + * + *

Valid: - Live: valid key/value are set + * + *

Invalid: - Collected: key/value was partially collected, but not yet cleaned up + */ + interface InternalEntry> { + /** Gets the next entry in the chain. */ + E getNext(); + + /** Gets the entry's hash. */ + int getHash(); + + /** Gets the key for this entry. */ + K getKey(); + + /** Gets the value for the entry. */ + V getValue(); + } + + /* + * Note: the following classes have a lot of duplicate code. It sucks, but it saves a lot of + * memory. If only Java had mixins! + */ + + /** Base class for {@link InternalEntry} implementations for strong keys. */ + abstract static class AbstractStrongKeyEntry> + implements InternalEntry { + final K key; + final int hash; + final E next; + + AbstractStrongKeyEntry(K key, int hash, E next) { + this.key = key; + this.hash = hash; + this.next = next; + } + + @Override + public K getKey() { + return this.key; + } + + @Override + public int getHash() { + return hash; + } + + @Override + public E getNext() { + return next; + } + } + + /** Marker interface for {@link InternalEntry} implementations for strong values. */ + interface StrongValueEntry> + extends InternalEntry {} + + /** Marker interface for {@link InternalEntry} implementations for weak values. */ + interface WeakValueEntry> extends InternalEntry { + /** Gets the weak value reference held by entry. */ + WeakValueReference getValueReference(); + + /** + * Clears the weak value reference held by the entry. Should be used when the entry's value is + * overwritten. + */ + void clearValue(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // impl never uses a parameter or returns any non-null value + static > + WeakValueReference unsetWeakValueReference() { + return (WeakValueReference) UNSET_WEAK_VALUE_REFERENCE; + } + + /** Concrete implementation of {@link InternalEntry} for strong keys and strong values. */ + static final class StrongKeyStrongValueEntry + extends AbstractStrongKeyEntry> + implements StrongValueEntry> { + private volatile V value = null; + + StrongKeyStrongValueEntry(K key, int hash, StrongKeyStrongValueEntry next) { + super(key, hash, next); + } + + @Override + + public V getValue() { + return value; + } + + void setValue(V value) { + this.value = value; + } + + StrongKeyStrongValueEntry copy(StrongKeyStrongValueEntry newNext) { + StrongKeyStrongValueEntry newEntry = + new StrongKeyStrongValueEntry<>(this.key, this.hash, newNext); + newEntry.value = this.value; + return newEntry; + } + + /** Concrete implementation of {@link InternalEntryHelper} for strong keys and strong values. */ + static final class Helper + implements InternalEntryHelper< + K, V, StrongKeyStrongValueEntry, StrongKeyStrongValueSegment> { + private static final Helper INSTANCE = new Helper<>(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Helper instance() { + return (Helper) INSTANCE; + } + + @Override + public Strength keyStrength() { + return Strength.STRONG; + } + + @Override + public Strength valueStrength() { + return Strength.STRONG; + } + + @Override + public StrongKeyStrongValueSegment newSegment( + MapMakerInternalMap< + K, V, StrongKeyStrongValueEntry, StrongKeyStrongValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + return new StrongKeyStrongValueSegment<>(map, initialCapacity, maxSegmentSize); + } + + @Override + public StrongKeyStrongValueEntry copy( + StrongKeyStrongValueSegment segment, + StrongKeyStrongValueEntry entry, + StrongKeyStrongValueEntry newNext) { + return entry.copy(newNext); + } + + @Override + public void setValue( + StrongKeyStrongValueSegment segment, + StrongKeyStrongValueEntry entry, + V value) { + entry.setValue(value); + } + + @Override + public StrongKeyStrongValueEntry newEntry( + StrongKeyStrongValueSegment segment, + K key, + int hash, + StrongKeyStrongValueEntry next) { + return new StrongKeyStrongValueEntry<>(key, hash, next); + } + } + } + + /** Concrete implementation of {@link InternalEntry} for strong keys and weak values. */ + static final class StrongKeyWeakValueEntry + extends AbstractStrongKeyEntry> + implements WeakValueEntry> { + private volatile WeakValueReference> valueReference = + unsetWeakValueReference(); + + StrongKeyWeakValueEntry(K key, int hash, StrongKeyWeakValueEntry next) { + super(key, hash, next); + } + + @Override + public V getValue() { + return valueReference.get(); + } + + @Override + public void clearValue() { + valueReference.clear(); + } + + void setValue(V value, ReferenceQueue queueForValues) { + WeakValueReference> previous = this.valueReference; + this.valueReference = new WeakValueReferenceImpl<>(queueForValues, value, this); + previous.clear(); + } + + StrongKeyWeakValueEntry copy( + ReferenceQueue queueForValues, StrongKeyWeakValueEntry newNext) { + StrongKeyWeakValueEntry newEntry = new StrongKeyWeakValueEntry<>(key, hash, newNext); + newEntry.valueReference = valueReference.copyFor(queueForValues, newEntry); + return newEntry; + } + + @Override + public WeakValueReference> getValueReference() { + return valueReference; + } + + /** Concrete implementation of {@link InternalEntryHelper} for strong keys and weak values. */ + static final class Helper + implements InternalEntryHelper< + K, V, StrongKeyWeakValueEntry, StrongKeyWeakValueSegment> { + private static final Helper INSTANCE = new Helper<>(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Helper instance() { + return (Helper) INSTANCE; + } + + @Override + public Strength keyStrength() { + return Strength.STRONG; + } + + @Override + public Strength valueStrength() { + return Strength.WEAK; + } + + @Override + public StrongKeyWeakValueSegment newSegment( + MapMakerInternalMap, StrongKeyWeakValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + return new StrongKeyWeakValueSegment<>(map, initialCapacity, maxSegmentSize); + } + + @Override + public StrongKeyWeakValueEntry copy( + StrongKeyWeakValueSegment segment, + StrongKeyWeakValueEntry entry, + StrongKeyWeakValueEntry newNext) { + if (Segment.isCollected(entry)) { + return null; + } + return entry.copy(segment.queueForValues, newNext); + } + + @Override + public void setValue( + StrongKeyWeakValueSegment segment, StrongKeyWeakValueEntry entry, V value) { + entry.setValue(value, segment.queueForValues); + } + + @Override + public StrongKeyWeakValueEntry newEntry( + StrongKeyWeakValueSegment segment, + K key, + int hash, + StrongKeyWeakValueEntry next) { + return new StrongKeyWeakValueEntry<>(key, hash, next); + } + } + } + + /** Concrete implementation of {@link InternalEntry} for strong keys and {@link Dummy} values. */ + static final class StrongKeyDummyValueEntry + extends AbstractStrongKeyEntry> + implements StrongValueEntry> { + StrongKeyDummyValueEntry(K key, int hash, StrongKeyDummyValueEntry next) { + super(key, hash, next); + } + + @Override + public Dummy getValue() { + return Dummy.VALUE; + } + + void setValue(Dummy value) {} + + StrongKeyDummyValueEntry copy(StrongKeyDummyValueEntry newNext) { + return new StrongKeyDummyValueEntry(this.key, this.hash, newNext); + } + + /** + * Concrete implementation of {@link InternalEntryHelper} for strong keys and {@link Dummy} + * values. + */ + static final class Helper + implements InternalEntryHelper< + K, Dummy, StrongKeyDummyValueEntry, StrongKeyDummyValueSegment> { + private static final Helper INSTANCE = new Helper<>(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Helper instance() { + return (Helper) INSTANCE; + } + + @Override + public Strength keyStrength() { + return Strength.STRONG; + } + + @Override + public Strength valueStrength() { + return Strength.STRONG; + } + + @Override + public StrongKeyDummyValueSegment newSegment( + MapMakerInternalMap, StrongKeyDummyValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + return new StrongKeyDummyValueSegment(map, initialCapacity, maxSegmentSize); + } + + @Override + public StrongKeyDummyValueEntry copy( + StrongKeyDummyValueSegment segment, + StrongKeyDummyValueEntry entry, + StrongKeyDummyValueEntry newNext) { + return entry.copy(newNext); + } + + @Override + public void setValue( + StrongKeyDummyValueSegment segment, StrongKeyDummyValueEntry entry, Dummy value) {} + + @Override + public StrongKeyDummyValueEntry newEntry( + StrongKeyDummyValueSegment segment, + K key, + int hash, + StrongKeyDummyValueEntry next) { + return new StrongKeyDummyValueEntry(key, hash, next); + } + } + } + + /** Base class for {@link InternalEntry} implementations for weak keys. */ + abstract static class AbstractWeakKeyEntry> + extends WeakReference implements InternalEntry { + final int hash; + final E next; + + AbstractWeakKeyEntry(ReferenceQueue queue, K key, int hash, E next) { + super(key, queue); + this.hash = hash; + this.next = next; + } + + @Override + public K getKey() { + return get(); + } + + @Override + public int getHash() { + return hash; + } + + @Override + public E getNext() { + return next; + } + } + + /** Concrete implementation of {@link InternalEntry} for weak keys and {@link Dummy} values. */ + static final class WeakKeyDummyValueEntry + extends AbstractWeakKeyEntry> + implements StrongValueEntry> { + WeakKeyDummyValueEntry( + ReferenceQueue queue, K key, int hash, WeakKeyDummyValueEntry next) { + super(queue, key, hash, next); + } + + @Override + public Dummy getValue() { + return Dummy.VALUE; + } + + void setValue(Dummy value) {} + + WeakKeyDummyValueEntry copy( + ReferenceQueue queueForKeys, WeakKeyDummyValueEntry newNext) { + return new WeakKeyDummyValueEntry(queueForKeys, getKey(), this.hash, newNext); + } + + /** + * Concrete implementation of {@link InternalEntryHelper} for weak keys and {@link Dummy} + * values. + */ + static final class Helper + implements InternalEntryHelper< + K, Dummy, WeakKeyDummyValueEntry, WeakKeyDummyValueSegment> { + private static final Helper INSTANCE = new Helper<>(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Helper instance() { + return (Helper) INSTANCE; + } + + @Override + public Strength keyStrength() { + return Strength.WEAK; + } + + @Override + public Strength valueStrength() { + return Strength.STRONG; + } + + @Override + public WeakKeyDummyValueSegment newSegment( + MapMakerInternalMap, WeakKeyDummyValueSegment> map, + int initialCapacity, + int maxSegmentSize) { + return new WeakKeyDummyValueSegment(map, initialCapacity, maxSegmentSize); + } + + @Override + public WeakKeyDummyValueEntry copy( + WeakKeyDummyValueSegment segment, + WeakKeyDummyValueEntry entry, + WeakKeyDummyValueEntry newNext) { + if (entry.getKey() == null) { + // key collected + return null; + } + return entry.copy(segment.queueForKeys, newNext); + } + + @Override + public void setValue( + WeakKeyDummyValueSegment segment, WeakKeyDummyValueEntry entry, Dummy value) {} + + @Override + public WeakKeyDummyValueEntry newEntry( + WeakKeyDummyValueSegment segment, + K key, + int hash, + WeakKeyDummyValueEntry next) { + return new WeakKeyDummyValueEntry(segment.queueForKeys, key, hash, next); + } + } + } + + /** Concrete implementation of {@link InternalEntry} for weak keys and strong values. */ + static final class WeakKeyStrongValueEntry + extends AbstractWeakKeyEntry> + implements StrongValueEntry> { + private volatile V value = null; + + WeakKeyStrongValueEntry( + ReferenceQueue queue, + K key, + int hash, + WeakKeyStrongValueEntry next) { + super(queue, key, hash, next); + } + + @Override + + public V getValue() { + return value; + } + + void setValue(V value) { + this.value = value; + } + + WeakKeyStrongValueEntry copy( + ReferenceQueue queueForKeys, WeakKeyStrongValueEntry newNext) { + WeakKeyStrongValueEntry newEntry = + new WeakKeyStrongValueEntry<>(queueForKeys, getKey(), this.hash, newNext); + newEntry.setValue(value); + return newEntry; + } + + /** Concrete implementation of {@link InternalEntryHelper} for weak keys and strong values. */ + static final class Helper + implements InternalEntryHelper< + K, V, WeakKeyStrongValueEntry, WeakKeyStrongValueSegment> { + private static final Helper INSTANCE = new Helper<>(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Helper instance() { + return (Helper) INSTANCE; + } + + @Override + public Strength keyStrength() { + return Strength.WEAK; + } + + @Override + public Strength valueStrength() { + return Strength.STRONG; + } + + @Override + public WeakKeyStrongValueSegment newSegment( + MapMakerInternalMap, WeakKeyStrongValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + return new WeakKeyStrongValueSegment<>(map, initialCapacity, maxSegmentSize); + } + + @Override + public WeakKeyStrongValueEntry copy( + WeakKeyStrongValueSegment segment, + WeakKeyStrongValueEntry entry, + WeakKeyStrongValueEntry newNext) { + if (entry.getKey() == null) { + // key collected + return null; + } + return entry.copy(segment.queueForKeys, newNext); + } + + @Override + public void setValue( + WeakKeyStrongValueSegment segment, WeakKeyStrongValueEntry entry, V value) { + entry.setValue(value); + } + + @Override + public WeakKeyStrongValueEntry newEntry( + WeakKeyStrongValueSegment segment, + K key, + int hash, + WeakKeyStrongValueEntry next) { + return new WeakKeyStrongValueEntry<>(segment.queueForKeys, key, hash, next); + } + } + } + + /** Concrete implementation of {@link InternalEntry} for weak keys and weak values. */ + static final class WeakKeyWeakValueEntry + extends AbstractWeakKeyEntry> + implements WeakValueEntry> { + private volatile WeakValueReference> valueReference = + unsetWeakValueReference(); + + WeakKeyWeakValueEntry( + ReferenceQueue queue, K key, int hash, WeakKeyWeakValueEntry next) { + super(queue, key, hash, next); + } + + @Override + public V getValue() { + return valueReference.get(); + } + + WeakKeyWeakValueEntry copy( + ReferenceQueue queueForKeys, + ReferenceQueue queueForValues, + WeakKeyWeakValueEntry newNext) { + WeakKeyWeakValueEntry newEntry = + new WeakKeyWeakValueEntry<>(queueForKeys, getKey(), this.hash, newNext); + newEntry.valueReference = valueReference.copyFor(queueForValues, newEntry); + return newEntry; + } + + @Override + public void clearValue() { + valueReference.clear(); + } + + void setValue(V value, ReferenceQueue queueForValues) { + WeakValueReference> previous = this.valueReference; + this.valueReference = new WeakValueReferenceImpl<>(queueForValues, value, this); + previous.clear(); + } + + @Override + public WeakValueReference> getValueReference() { + return valueReference; + } + + /** Concrete implementation of {@link InternalEntryHelper} for weak keys and weak values. */ + static final class Helper + implements InternalEntryHelper< + K, V, WeakKeyWeakValueEntry, WeakKeyWeakValueSegment> { + private static final Helper INSTANCE = new Helper<>(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Helper instance() { + return (Helper) INSTANCE; + } + + @Override + public Strength keyStrength() { + return Strength.WEAK; + } + + @Override + public Strength valueStrength() { + return Strength.WEAK; + } + + @Override + public WeakKeyWeakValueSegment newSegment( + MapMakerInternalMap, WeakKeyWeakValueSegment> map, + int initialCapacity, + int maxSegmentSize) { + return new WeakKeyWeakValueSegment<>(map, initialCapacity, maxSegmentSize); + } + + @Override + public WeakKeyWeakValueEntry copy( + WeakKeyWeakValueSegment segment, + WeakKeyWeakValueEntry entry, + WeakKeyWeakValueEntry newNext) { + if (entry.getKey() == null) { + // key collected + return null; + } + if (Segment.isCollected(entry)) { + return null; + } + return entry.copy(segment.queueForKeys, segment.queueForValues, newNext); + } + + @Override + public void setValue( + WeakKeyWeakValueSegment segment, WeakKeyWeakValueEntry entry, V value) { + entry.setValue(value, segment.queueForValues); + } + + @Override + public WeakKeyWeakValueEntry newEntry( + WeakKeyWeakValueSegment segment, + K key, + int hash, + WeakKeyWeakValueEntry next) { + return new WeakKeyWeakValueEntry<>(segment.queueForKeys, key, hash, next); + } + } + } + + /** A weakly referenced value that also has a reference to its containing entry. */ + interface WeakValueReference> { + /** + * Returns the current value being referenced, or {@code null} if there is none (e.g. because + * either it got collected, or {@link #clear} was called, or it wasn't set in the first place). + */ + + V get(); + + /** Returns the entry which contains this {@link WeakValueReference}. */ + E getEntry(); + + /** Unsets the referenced value. Subsequent calls to {@link #get} will return {@code null}. */ + void clear(); + + /** + * Returns a freshly created {@link WeakValueReference} for the given {@code entry} (and on the + * given {@code queue} with the same value as this {@link WeakValueReference}. + */ + WeakValueReference copyFor(ReferenceQueue queue, E entry); + } + + /** + * A dummy implementation of {@link InternalEntry}, solely for use in the type signature of {@link + * #UNSET_WEAK_VALUE_REFERENCE} below. + */ + static final class DummyInternalEntry + implements InternalEntry { + private DummyInternalEntry() { + throw new AssertionError(); + } + + @Override + public DummyInternalEntry getNext() { + throw new AssertionError(); + } + + @Override + public int getHash() { + throw new AssertionError(); + } + + @Override + public Object getKey() { + throw new AssertionError(); + } + + @Override + public Object getValue() { + throw new AssertionError(); + } + } + + /** + * A singleton {@link WeakValueReference} used to denote an unset value in a entry with weak + * values. + */ + static final WeakValueReference UNSET_WEAK_VALUE_REFERENCE = + new WeakValueReference() { + @Override + public DummyInternalEntry getEntry() { + return null; + } + + @Override + public void clear() {} + + @Override + public Object get() { + return null; + } + + @Override + public WeakValueReference copyFor( + ReferenceQueue queue, DummyInternalEntry entry) { + return this; + } + }; + + /** Concrete implementation of {@link WeakValueReference}. */ + static final class WeakValueReferenceImpl> + extends WeakReference implements WeakValueReference { + final E entry; + + WeakValueReferenceImpl(ReferenceQueue queue, V referent, E entry) { + super(referent, queue); + this.entry = entry; + } + + @Override + public E getEntry() { + return entry; + } + + @Override + public WeakValueReference copyFor(ReferenceQueue queue, E entry) { + return new WeakValueReferenceImpl<>(queue, get(), entry); + } + } + + /** + * Applies a supplemental hash function to a given hash code, which defends against poor quality + * hash functions. This is critical when the concurrent hash map uses power-of-two length hash + * tables, that otherwise encounter collisions for hash codes that do not differ in lower or upper + * bits. + * + * @param h hash code + */ + static int rehash(int h) { + // Spread bits to regularize both segment and index locations, + // using variant of single-word Wang/Jenkins hash. + // TODO(kevinb): use Hashing/move this to Hashing? + h += (h << 15) ^ 0xffffcd7d; + h ^= (h >>> 10); + h += (h << 3); + h ^= (h >>> 6); + h += (h << 2) + (h << 14); + return h ^ (h >>> 16); + } + + /** + * This method is a convenience for testing. Code should call {@link Segment#copyEntry} directly. + */ + // Guarded By Segment.this + + E copyEntry(E original, E newNext) { + int hash = original.getHash(); + return segmentFor(hash).copyEntry(original, newNext); + } + + int hash(Object key) { + int h = keyEquivalence.hash(key); + return rehash(h); + } + + void reclaimValue(WeakValueReference valueReference) { + E entry = valueReference.getEntry(); + int hash = entry.getHash(); + segmentFor(hash).reclaimValue(entry.getKey(), hash, valueReference); + } + + void reclaimKey(E entry) { + int hash = entry.getHash(); + segmentFor(hash).reclaimKey(entry, hash); + } + + /** + * This method is a convenience for testing. Code should call {@link Segment#getLiveValue} + * instead. + */ + + boolean isLiveForTesting(InternalEntry entry) { + return segmentFor(entry.getHash()).getLiveValueForTesting(entry) != null; + } + + /** + * Returns the segment that should be used for a key with the given hash. + * + * @param hash the hash code for the key + * @return the segment + */ + Segment segmentFor(int hash) { + // TODO(fry): Lazily create segments? + return segments[(hash >>> segmentShift) & segmentMask]; + } + + Segment createSegment(int initialCapacity, int maxSegmentSize) { + return entryHelper.newSegment(this, initialCapacity, maxSegmentSize); + } + + /** + * Gets the value from an entry. Returns {@code null} if the entry is invalid, partially-collected + * or computing. + */ + V getLiveValue(E entry) { + if (entry.getKey() == null) { + return null; + } + V value = entry.getValue(); + if (value == null) { + return null; + } + return value; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + final Segment[] newSegmentArray(int ssize) { + return new Segment[ssize]; + } + + // Inner Classes + + /** + * Segments are specialized versions of hash tables. This subclass inherits from ReentrantLock + * opportunistically, just to simplify some locking and avoid separate construction. + */ + @SuppressWarnings("serial") // This class is never serialized. + abstract static class Segment< + K, V, E extends InternalEntry, S extends Segment> + extends ReentrantLock { + + /* + * Segments maintain a table of entry lists that are ALWAYS kept in a consistent state, so can + * be read without locking. Next fields of nodes are immutable (final). All list additions are + * performed at the front of each bin. This makes it easy to check changes, and also fast to + * traverse. When nodes would otherwise be changed, new nodes are created to replace them. This + * works well for hash tables since the bin lists tend to be short. (The average length is less + * than two.) + * + * Read operations can thus proceed without locking, but rely on selected uses of volatiles to + * ensure that completed write operations performed by other threads are noticed. For most + * purposes, the "count" field, tracking the number of elements, serves as that volatile + * variable ensuring visibility. This is convenient because this field needs to be read in many + * read operations anyway: + * + * - All (unsynchronized) read operations must first read the "count" field, and should not + * look at table entries if it is 0. + * + * - All (synchronized) write operations should write to the "count" field after structurally + * changing any bin. The operations must not take any action that could even momentarily + * cause a concurrent read operation to see inconsistent data. This is made easier by the + * nature of the read operations in Map. For example, no operation can reveal that the table + * has grown but the threshold has not yet been updated, so there are no atomicity requirements + * for this with respect to reads. + * + * As a guide, all critical volatile reads and writes to the count field are marked in code + * comments. + */ + + final MapMakerInternalMap map; + + /** + * The number of live elements in this segment's region. This does not include unset elements + * which are awaiting cleanup. + */ + volatile int count; + + /** + * Number of updates that alter the size of the table. This is used during bulk-read methods to + * make sure they see a consistent snapshot: If modCounts change during a traversal of segments + * computing size or checking containsValue, then we might have an inconsistent view of state so + * (usually) must retry. + */ + int modCount; + + /** + * The table is expanded when its size exceeds this threshold. (The value of this field is + * always {@code (int) (capacity * 0.75)}.) + */ + int threshold; + + /** The per-segment table. */ + volatile AtomicReferenceArray table; + + /** The maximum size of this map. MapMaker.UNSET_INT if there is no maximum. */ + final int maxSegmentSize; + + /** + * A counter of the number of reads since the last write, used to drain queues on a small + * fraction of read operations. + */ + final AtomicInteger readCount = new AtomicInteger(); + + Segment(MapMakerInternalMap map, int initialCapacity, int maxSegmentSize) { + this.map = map; + this.maxSegmentSize = maxSegmentSize; + initTable(newEntryArray(initialCapacity)); + } + + /** + * Returns {@code this} up-casted to the specific {@link Segment} implementation type {@code S}. + * + *

This method exists so that the {@link Segment} code can be generic in terms of {@code S}, + * the type of the concrete implementation. + */ + abstract S self(); + + /** Drains the reference queues used by this segment, if any. */ + // @GuardedBy("this") + void maybeDrainReferenceQueues() {} + + /** Clears the reference queues used by this segment, if any. */ + void maybeClearReferenceQueues() {} + + /** Sets the value of the given {@code entry}. */ + void setValue(E entry, V value) { + this.map.entryHelper.setValue(self(), entry, value); + } + + /** Returns a copy of the given {@code entry}. */ + E copyEntry(E original, E newNext) { + return this.map.entryHelper.copy(self(), original, newNext); + } + + AtomicReferenceArray newEntryArray(int size) { + return new AtomicReferenceArray(size); + } + + void initTable(AtomicReferenceArray newTable) { + this.threshold = newTable.length() * 3 / 4; // 0.75 + if (this.threshold == maxSegmentSize) { + // prevent spurious expansion before eviction + this.threshold++; + } + this.table = newTable; + } + + // Convenience methods for testing + + /** + * Unsafe cast of the given entry to {@code E}, the type of the specific {@link InternalEntry} + * implementation type. + * + *

This method is provided as a convenience for tests. Otherwise they'd need to be + * knowledgable about all the implementation details of our type system trickery. + */ + abstract E castForTesting(InternalEntry entry); + + /** Unsafely extracts the key reference queue used by this segment. */ + ReferenceQueue getKeyReferenceQueueForTesting() { + throw new AssertionError(); + } + + /** Unsafely extracts the value reference queue used by this segment. */ + ReferenceQueue getValueReferenceQueueForTesting() { + throw new AssertionError(); + } + + /** Unsafely extracts the weak value reference inside of the given {@code entry}. */ + WeakValueReference getWeakValueReferenceForTesting(InternalEntry entry) { + throw new AssertionError(); + } + + /** + * Unsafely creates of a fresh {@link WeakValueReference}, referencing the given {@code value}, + * for the given {@code entry} + */ + WeakValueReference newWeakValueReferenceForTesting( + InternalEntry entry, V value) { + throw new AssertionError(); + } + + /** + * Unsafely sets the weak value reference inside the given {@code entry} to be the given {@code + * valueReference} + */ + void setWeakValueReferenceForTesting( + InternalEntry entry, + WeakValueReference> valueReference) { + throw new AssertionError(); + } + + /** + * Unsafely sets the given index of this segment's internal hash table to be the given entry. + */ + void setTableEntryForTesting(int i, InternalEntry entry) { + table.set(i, castForTesting(entry)); + } + + /** Unsafely returns a copy of the given entry. */ + E copyForTesting(InternalEntry entry, InternalEntry newNext) { + return this.map.entryHelper.copy(self(), castForTesting(entry), castForTesting(newNext)); + } + + /** Unsafely sets the value of the given entry. */ + void setValueForTesting(InternalEntry entry, V value) { + this.map.entryHelper.setValue(self(), castForTesting(entry), value); + } + + /** Unsafely returns a fresh entry. */ + E newEntryForTesting(K key, int hash, InternalEntry next) { + return this.map.entryHelper.newEntry(self(), key, hash, castForTesting(next)); + } + + /** Unsafely removes the given entry from this segment's hash table. */ + boolean removeTableEntryForTesting(InternalEntry entry) { + return removeEntryForTesting(castForTesting(entry)); + } + + /** Unsafely removes the given entry from the given chain in this segment's hash table. */ + E removeFromChainForTesting(InternalEntry first, InternalEntry entry) { + return removeFromChain(castForTesting(first), castForTesting(entry)); + } + + /** + * Unsafely returns the value of the given entry if it's still live, or {@code null} otherwise. + */ + + V getLiveValueForTesting(InternalEntry entry) { + return getLiveValue(castForTesting(entry)); + } + + // reference queues, for garbage collection cleanup + + /** Cleanup collected entries when the lock is available. */ + void tryDrainReferenceQueues() { + if (tryLock()) { + try { + maybeDrainReferenceQueues(); + } finally { + unlock(); + } + } + } + + // @GuardedBy("this") + void drainKeyReferenceQueue(ReferenceQueue keyReferenceQueue) { + Reference ref; + int i = 0; + while ((ref = keyReferenceQueue.poll()) != null) { + @SuppressWarnings({"unchecked", "rawtypes"}) + E entry = (E) ref; + map.reclaimKey(entry); + if (++i == DRAIN_MAX) { + break; + } + } + } + + // @GuardedBy("this") + void drainValueReferenceQueue(ReferenceQueue valueReferenceQueue) { + Reference ref; + int i = 0; + while ((ref = valueReferenceQueue.poll()) != null) { + @SuppressWarnings({"unchecked", "rawtypes"}) + WeakValueReference valueReference = (WeakValueReference) ref; + map.reclaimValue(valueReference); + if (++i == DRAIN_MAX) { + break; + } + } + } + + void clearReferenceQueue(ReferenceQueue referenceQueue) { + while (referenceQueue.poll() != null) {} + } + + /** Returns first entry of bin for given hash. */ + E getFirst(int hash) { + // read this volatile field only once + AtomicReferenceArray table = this.table; + return table.get(hash & (table.length() - 1)); + } + + // Specialized implementations of map methods + + E getEntry(Object key, int hash) { + if (count != 0) { // read-volatile + for (E e = getFirst(hash); e != null; e = e.getNext()) { + if (e.getHash() != hash) { + continue; + } + + K entryKey = e.getKey(); + if (entryKey == null) { + tryDrainReferenceQueues(); + continue; + } + + if (map.keyEquivalence.equivalent(key, entryKey)) { + return e; + } + } + } + + return null; + } + + E getLiveEntry(Object key, int hash) { + return getEntry(key, hash); + } + + V get(Object key, int hash) { + try { + E e = getLiveEntry(key, hash); + if (e == null) { + return null; + } + + V value = e.getValue(); + if (value == null) { + tryDrainReferenceQueues(); + } + return value; + } finally { + postReadCleanup(); + } + } + + boolean containsKey(Object key, int hash) { + try { + if (count != 0) { // read-volatile + E e = getLiveEntry(key, hash); + return e != null && e.getValue() != null; + } + + return false; + } finally { + postReadCleanup(); + } + } + + /** + * This method is a convenience for testing. Code should call {@link + * MapMakerInternalMap#containsValue} directly. + */ + + boolean containsValue(Object value) { + try { + if (count != 0) { // read-volatile + AtomicReferenceArray table = this.table; + int length = table.length(); + for (int i = 0; i < length; ++i) { + for (E e = table.get(i); e != null; e = e.getNext()) { + V entryValue = getLiveValue(e); + if (entryValue == null) { + continue; + } + if (map.valueEquivalence().equivalent(value, entryValue)) { + return true; + } + } + } + } + + return false; + } finally { + postReadCleanup(); + } + } + + V put(K key, int hash, V value, boolean onlyIfAbsent) { + lock(); + try { + preWriteCleanup(); + + int newCount = this.count + 1; + if (newCount > this.threshold) { // ensure capacity + expand(); + newCount = this.count + 1; + } + + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + // Look for an existing entry. + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + // We found an existing entry. + + V entryValue = e.getValue(); + + if (entryValue == null) { + ++modCount; + setValue(e, value); + newCount = this.count; // count remains unchanged + this.count = newCount; // write-volatile + return null; + } else if (onlyIfAbsent) { + // Mimic + // "if (!map.containsKey(key)) ... + // else return map.get(key); + return entryValue; + } else { + // clobber existing entry, count remains unchanged + ++modCount; + setValue(e, value); + return entryValue; + } + } + } + + // Create a new entry. + ++modCount; + E newEntry = map.entryHelper.newEntry(self(), key, hash, first); + setValue(newEntry, value); + table.set(index, newEntry); + this.count = newCount; // write-volatile + return null; + } finally { + unlock(); + } + } + + /** Expands the table if possible. */ + // @GuardedBy("this") + void expand() { + AtomicReferenceArray oldTable = table; + int oldCapacity = oldTable.length(); + if (oldCapacity >= MAXIMUM_CAPACITY) { + return; + } + + /* + * Reclassify nodes in each list to new Map. Because we are using power-of-two expansion, the + * elements from each bin must either stay at same index, or move with a power of two offset. + * We eliminate unnecessary node creation by catching cases where old nodes can be reused + * because their next fields won't change. Statistically, at the default threshold, only + * about one-sixth of them need cloning when a table doubles. The nodes they replace will be + * garbage collectable as soon as they are no longer referenced by any reader thread that may + * be in the midst of traversing table right now. + */ + + int newCount = count; + AtomicReferenceArray newTable = newEntryArray(oldCapacity << 1); + threshold = newTable.length() * 3 / 4; + int newMask = newTable.length() - 1; + for (int oldIndex = 0; oldIndex < oldCapacity; ++oldIndex) { + // We need to guarantee that any existing reads of old Map can + // proceed. So we cannot yet null out each bin. + E head = oldTable.get(oldIndex); + + if (head != null) { + E next = head.getNext(); + int headIndex = head.getHash() & newMask; + + // Single node on list + if (next == null) { + newTable.set(headIndex, head); + } else { + // Reuse the consecutive sequence of nodes with the same target + // index from the end of the list. tail points to the first + // entry in the reusable list. + E tail = head; + int tailIndex = headIndex; + for (E e = next; e != null; e = e.getNext()) { + int newIndex = e.getHash() & newMask; + if (newIndex != tailIndex) { + // The index changed. We'll need to copy the previous entry. + tailIndex = newIndex; + tail = e; + } + } + newTable.set(tailIndex, tail); + + // Clone nodes leading up to the tail. + for (E e = head; e != tail; e = e.getNext()) { + int newIndex = e.getHash() & newMask; + E newNext = newTable.get(newIndex); + E newFirst = copyEntry(e, newNext); + if (newFirst != null) { + newTable.set(newIndex, newFirst); + } else { + newCount--; + } + } + } + } + } + table = newTable; + this.count = newCount; + } + + boolean replace(K key, int hash, V oldValue, V newValue) { + lock(); + try { + preWriteCleanup(); + + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + // If the value disappeared, this entry is partially collected, + // and we should pretend like it doesn't exist. + V entryValue = e.getValue(); + if (entryValue == null) { + if (isCollected(e)) { + int newCount = this.count - 1; + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + } + return false; + } + + if (map.valueEquivalence().equivalent(oldValue, entryValue)) { + ++modCount; + setValue(e, newValue); + return true; + } else { + // Mimic + // "if (map.containsKey(key) && map.get(key).equals(oldValue))..." + return false; + } + } + } + + return false; + } finally { + unlock(); + } + } + + V replace(K key, int hash, V newValue) { + lock(); + try { + preWriteCleanup(); + + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + // If the value disappeared, this entry is partially collected, + // and we should pretend like it doesn't exist. + V entryValue = e.getValue(); + if (entryValue == null) { + if (isCollected(e)) { + int newCount = this.count - 1; + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + } + return null; + } + + ++modCount; + setValue(e, newValue); + return entryValue; + } + } + + return null; + } finally { + unlock(); + } + } + + V remove(Object key, int hash) { + lock(); + try { + preWriteCleanup(); + + int newCount = this.count - 1; + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + V entryValue = e.getValue(); + + if (entryValue != null) { + // TODO(kak): Remove this branch + } else if (isCollected(e)) { + // TODO(kak): Remove this branch + } else { + return null; + } + + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + return entryValue; + } + } + + return null; + } finally { + unlock(); + } + } + + boolean remove(Object key, int hash, Object value) { + lock(); + try { + preWriteCleanup(); + + int newCount = this.count - 1; + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + V entryValue = e.getValue(); + + boolean explicitRemoval = false; + if (map.valueEquivalence().equivalent(value, entryValue)) { + explicitRemoval = true; + } else if (isCollected(e)) { + // TODO(kak): Remove this branch + } else { + return false; + } + + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + return explicitRemoval; + } + } + + return false; + } finally { + unlock(); + } + } + + void clear() { + if (count != 0) { + lock(); + try { + AtomicReferenceArray table = this.table; + for (int i = 0; i < table.length(); ++i) { + table.set(i, null); + } + maybeClearReferenceQueues(); + readCount.set(0); + + ++modCount; + count = 0; // write-volatile + } finally { + unlock(); + } + } + } + + /** + * Removes an entry from within a table. All entries following the removed node can stay, but + * all preceding ones need to be cloned. + * + *

This method does not decrement count for the removed entry, but does decrement count for + * all partially collected entries which are skipped. As such callers which are modifying count + * must re-read it after calling removeFromChain. + * + * @param first the first entry of the table + * @param entry the entry being removed from the table + * @return the new first entry for the table + */ + // @GuardedBy("this") + E removeFromChain(E first, E entry) { + int newCount = count; + E newFirst = entry.getNext(); + for (E e = first; e != entry; e = e.getNext()) { + E next = copyEntry(e, newFirst); + if (next != null) { + newFirst = next; + } else { + newCount--; + } + } + this.count = newCount; + return newFirst; + } + + /** Removes an entry whose key has been garbage collected. */ + boolean reclaimKey(E entry, int hash) { + lock(); + try { + int newCount = count - 1; + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + if (e == entry) { + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + return true; + } + } + + return false; + } finally { + unlock(); + } + } + + /** Removes an entry whose value has been garbage collected. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + boolean reclaimValue(K key, int hash, WeakValueReference valueReference) { + lock(); + try { + int newCount = this.count - 1; + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + WeakValueReference v = ((WeakValueEntry) e).getValueReference(); + if (v == valueReference) { + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + return true; + } + return false; + } + } + + return false; + } finally { + unlock(); + } + } + + /** Clears a value that has not yet been set, and thus does not require count to be modified. */ + @SuppressWarnings({"unchecked", "rawtypes"}) + boolean clearValueForTesting( + K key, + int hash, + WeakValueReference> valueReference) { + lock(); + try { + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + K entryKey = e.getKey(); + if (e.getHash() == hash + && entryKey != null + && map.keyEquivalence.equivalent(key, entryKey)) { + WeakValueReference v = ((WeakValueEntry) e).getValueReference(); + if (v == valueReference) { + E newFirst = removeFromChain(first, e); + table.set(index, newFirst); + return true; + } + return false; + } + } + + return false; + } finally { + unlock(); + } + } + + // @GuardedBy("this") + boolean removeEntryForTesting(E entry) { + int hash = entry.getHash(); + int newCount = this.count - 1; + AtomicReferenceArray table = this.table; + int index = hash & (table.length() - 1); + E first = table.get(index); + + for (E e = first; e != null; e = e.getNext()) { + if (e == entry) { + ++modCount; + E newFirst = removeFromChain(first, e); + newCount = this.count - 1; + table.set(index, newFirst); + this.count = newCount; // write-volatile + return true; + } + } + + return false; + } + + /** + * Returns {@code true} if the value has been partially collected, meaning that the value is + * null. + */ + static > boolean isCollected(E entry) { + return entry.getValue() == null; + } + + /** + * Gets the value from an entry. Returns {@code null} if the entry is invalid or + * partially-collected. + */ + + V getLiveValue(E entry) { + if (entry.getKey() == null) { + tryDrainReferenceQueues(); + return null; + } + V value = entry.getValue(); + if (value == null) { + tryDrainReferenceQueues(); + return null; + } + + return value; + } + + /** + * Performs routine cleanup following a read. Normally cleanup happens during writes, or from + * the cleanupExecutor. If cleanup is not observed after a sufficient number of reads, try + * cleaning up from the read thread. + */ + void postReadCleanup() { + if ((readCount.incrementAndGet() & DRAIN_THRESHOLD) == 0) { + runCleanup(); + } + } + + /** + * Performs routine cleanup prior to executing a write. This should be called every time a write + * thread acquires the segment lock, immediately after acquiring the lock. + */ + // @GuardedBy("this") + void preWriteCleanup() { + runLockedCleanup(); + } + + void runCleanup() { + runLockedCleanup(); + } + + void runLockedCleanup() { + if (tryLock()) { + try { + maybeDrainReferenceQueues(); + readCount.set(0); + } finally { + unlock(); + } + } + } + } + + /** Concrete implementation of {@link Segment} for strong keys and strong values. */ + static final class StrongKeyStrongValueSegment + extends Segment, StrongKeyStrongValueSegment> { + StrongKeyStrongValueSegment( + MapMakerInternalMap< + K, V, StrongKeyStrongValueEntry, StrongKeyStrongValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + super(map, initialCapacity, maxSegmentSize); + } + + @Override + StrongKeyStrongValueSegment self() { + return this; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public StrongKeyStrongValueEntry castForTesting(InternalEntry entry) { + return (StrongKeyStrongValueEntry) entry; + } + } + + /** Concrete implementation of {@link Segment} for strong keys and weak values. */ + static final class StrongKeyWeakValueSegment + extends Segment, StrongKeyWeakValueSegment> { + private final ReferenceQueue queueForValues = new ReferenceQueue(); + + StrongKeyWeakValueSegment( + MapMakerInternalMap, StrongKeyWeakValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + super(map, initialCapacity, maxSegmentSize); + } + + @Override + StrongKeyWeakValueSegment self() { + return this; + } + + @Override + ReferenceQueue getValueReferenceQueueForTesting() { + return queueForValues; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public StrongKeyWeakValueEntry castForTesting(InternalEntry entry) { + return (StrongKeyWeakValueEntry) entry; + } + + @Override + public WeakValueReference> getWeakValueReferenceForTesting( + InternalEntry e) { + return castForTesting(e).getValueReference(); + } + + @Override + public WeakValueReference> newWeakValueReferenceForTesting( + InternalEntry e, V value) { + return new WeakValueReferenceImpl<>(queueForValues, value, castForTesting(e)); + } + + @Override + public void setWeakValueReferenceForTesting( + InternalEntry e, + WeakValueReference> valueReference) { + StrongKeyWeakValueEntry entry = castForTesting(e); + @SuppressWarnings({"unchecked", "rawtypes"}) + WeakValueReference> newValueReference = + (WeakValueReference>) valueReference; + WeakValueReference> previous = entry.valueReference; + entry.valueReference = newValueReference; + previous.clear(); + } + + @Override + void maybeDrainReferenceQueues() { + drainValueReferenceQueue(queueForValues); + } + + @Override + void maybeClearReferenceQueues() { + clearReferenceQueue(queueForValues); + } + } + + /** Concrete implementation of {@link Segment} for strong keys and {@link Dummy} values. */ + static final class StrongKeyDummyValueSegment + extends Segment, StrongKeyDummyValueSegment> { + StrongKeyDummyValueSegment( + MapMakerInternalMap, StrongKeyDummyValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + super(map, initialCapacity, maxSegmentSize); + } + + @Override + StrongKeyDummyValueSegment self() { + return this; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public StrongKeyDummyValueEntry castForTesting(InternalEntry entry) { + return (StrongKeyDummyValueEntry) entry; + } + } + + /** Concrete implementation of {@link Segment} for weak keys and strong values. */ + static final class WeakKeyStrongValueSegment + extends Segment, WeakKeyStrongValueSegment> { + private final ReferenceQueue queueForKeys = new ReferenceQueue(); + + WeakKeyStrongValueSegment( + MapMakerInternalMap, WeakKeyStrongValueSegment> + map, + int initialCapacity, + int maxSegmentSize) { + super(map, initialCapacity, maxSegmentSize); + } + + @Override + WeakKeyStrongValueSegment self() { + return this; + } + + @Override + ReferenceQueue getKeyReferenceQueueForTesting() { + return queueForKeys; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public WeakKeyStrongValueEntry castForTesting(InternalEntry entry) { + return (WeakKeyStrongValueEntry) entry; + } + + @Override + void maybeDrainReferenceQueues() { + drainKeyReferenceQueue(queueForKeys); + } + + @Override + void maybeClearReferenceQueues() { + clearReferenceQueue(queueForKeys); + } + } + + /** Concrete implementation of {@link Segment} for weak keys and weak values. */ + static final class WeakKeyWeakValueSegment + extends Segment, WeakKeyWeakValueSegment> { + private final ReferenceQueue queueForKeys = new ReferenceQueue(); + private final ReferenceQueue queueForValues = new ReferenceQueue(); + + WeakKeyWeakValueSegment( + MapMakerInternalMap, WeakKeyWeakValueSegment> map, + int initialCapacity, + int maxSegmentSize) { + super(map, initialCapacity, maxSegmentSize); + } + + @Override + WeakKeyWeakValueSegment self() { + return this; + } + + @Override + ReferenceQueue getKeyReferenceQueueForTesting() { + return queueForKeys; + } + + @Override + ReferenceQueue getValueReferenceQueueForTesting() { + return queueForValues; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public WeakKeyWeakValueEntry castForTesting(InternalEntry entry) { + return (WeakKeyWeakValueEntry) entry; + } + + @Override + public WeakValueReference> getWeakValueReferenceForTesting( + InternalEntry e) { + return (castForTesting(e)).getValueReference(); + } + + @Override + public WeakValueReference> newWeakValueReferenceForTesting( + InternalEntry e, V value) { + return new WeakValueReferenceImpl<>(queueForValues, value, castForTesting(e)); + } + + @Override + public void setWeakValueReferenceForTesting( + InternalEntry e, + WeakValueReference> valueReference) { + WeakKeyWeakValueEntry entry = castForTesting(e); + @SuppressWarnings({"unchecked", "rawtypes"}) + WeakValueReference> newValueReference = + (WeakValueReference>) valueReference; + WeakValueReference> previous = entry.valueReference; + entry.valueReference = newValueReference; + previous.clear(); + } + + @Override + void maybeDrainReferenceQueues() { + drainKeyReferenceQueue(queueForKeys); + drainValueReferenceQueue(queueForValues); + } + + @Override + void maybeClearReferenceQueues() { + clearReferenceQueue(queueForKeys); + } + } + + /** Concrete implementation of {@link Segment} for weak keys and {@link Dummy} values. */ + static final class WeakKeyDummyValueSegment + extends Segment, WeakKeyDummyValueSegment> { + private final ReferenceQueue queueForKeys = new ReferenceQueue(); + + WeakKeyDummyValueSegment( + MapMakerInternalMap, WeakKeyDummyValueSegment> map, + int initialCapacity, + int maxSegmentSize) { + super(map, initialCapacity, maxSegmentSize); + } + + @Override + WeakKeyDummyValueSegment self() { + return this; + } + + @Override + ReferenceQueue getKeyReferenceQueueForTesting() { + return queueForKeys; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public WeakKeyDummyValueEntry castForTesting(InternalEntry entry) { + return (WeakKeyDummyValueEntry) entry; + } + + @Override + void maybeDrainReferenceQueues() { + drainKeyReferenceQueue(queueForKeys); + } + + @Override + void maybeClearReferenceQueues() { + clearReferenceQueue(queueForKeys); + } + } + + static final class CleanupMapTask implements Runnable { + final WeakReference> mapReference; + + public CleanupMapTask(MapMakerInternalMap map) { + this.mapReference = new WeakReference>(map); + } + + @Override + public void run() { + MapMakerInternalMap map = mapReference.get(); + if (map == null) { + throw new CancellationException(); + } + + for (Segment segment : map.segments) { + segment.runCleanup(); + } + } + } + + + Strength keyStrength() { + return entryHelper.keyStrength(); + } + + + Strength valueStrength() { + return entryHelper.valueStrength(); + } + + + Equivalence valueEquivalence() { + return entryHelper.valueStrength().defaultEquivalence(); + } + + // ConcurrentMap methods + + @Override + public boolean isEmpty() { + /* + * Sum per-segment modCounts to avoid mis-reporting when elements are concurrently added and + * removed in one segment while checking another, in which case the table was never actually + * empty at any point. (The sum ensures accuracy up through at least 1<<31 per-segment + * modifications before recheck.) Method containsValue() uses similar constructions for + * stability checks. + */ + long sum = 0L; + Segment[] segments = this.segments; + for (int i = 0; i < segments.length; ++i) { + if (segments[i].count != 0) { + return false; + } + sum += segments[i].modCount; + } + + if (sum != 0L) { // recheck unless no modifications + for (int i = 0; i < segments.length; ++i) { + if (segments[i].count != 0) { + return false; + } + sum -= segments[i].modCount; + } + if (sum != 0L) { + return false; + } + } + return true; + } + + @Override + public int size() { + Segment[] segments = this.segments; + long sum = 0; + for (int i = 0; i < segments.length; ++i) { + sum += segments[i].count; + } + return Ints.saturatedCast(sum); + } + + @Override + public V get(Object key) { + if (key == null) { + return null; + } + int hash = hash(key); + return segmentFor(hash).get(key, hash); + } + + /** + * Returns the internal entry for the specified key. The entry may be computing or partially + * collected. Does not impact recency ordering. + */ + E getEntry(Object key) { + if (key == null) { + return null; + } + int hash = hash(key); + return segmentFor(hash).getEntry(key, hash); + } + + @Override + public boolean containsKey(Object key) { + if (key == null) { + return false; + } + int hash = hash(key); + return segmentFor(hash).containsKey(key, hash); + } + + @Override + public boolean containsValue(Object value) { + if (value == null) { + return false; + } + + // This implementation is patterned after ConcurrentHashMap, but without the locking. The only + // way for it to return a false negative would be for the target value to jump around in the map + // such that none of the subsequent iterations observed it, despite the fact that at every point + // in time it was present somewhere int the map. This becomes increasingly unlikely as + // CONTAINS_VALUE_RETRIES increases, though without locking it is theoretically possible. + final Segment[] segments = this.segments; + long last = -1L; + for (int i = 0; i < CONTAINS_VALUE_RETRIES; i++) { + long sum = 0L; + for (Segment segment : segments) { + // ensure visibility of most recent completed write + int unused = segment.count; // read-volatile + + AtomicReferenceArray table = segment.table; + for (int j = 0; j < table.length(); j++) { + for (E e = table.get(j); e != null; e = e.getNext()) { + V v = segment.getLiveValue(e); + if (v != null && valueEquivalence().equivalent(value, v)) { + return true; + } + } + } + sum += segment.modCount; + } + if (sum == last) { + break; + } + last = sum; + } + return false; + } + + @Override + public V put(K key, V value) { + checkNotNull(key); + checkNotNull(value); + int hash = hash(key); + return segmentFor(hash).put(key, hash, value, false); + } + + @Override + public V putIfAbsent(K key, V value) { + checkNotNull(key); + checkNotNull(value); + int hash = hash(key); + return segmentFor(hash).put(key, hash, value, true); + } + + @Override + public void putAll(Map m) { + for (Entry e : m.entrySet()) { + put(e.getKey(), e.getValue()); + } + } + + @Override + public V remove(Object key) { + if (key == null) { + return null; + } + int hash = hash(key); + return segmentFor(hash).remove(key, hash); + } + + @Override + public boolean remove(Object key, Object value) { + if (key == null || value == null) { + return false; + } + int hash = hash(key); + return segmentFor(hash).remove(key, hash, value); + } + + @Override + public boolean replace(K key, V oldValue, V newValue) { + checkNotNull(key); + checkNotNull(newValue); + if (oldValue == null) { + return false; + } + int hash = hash(key); + return segmentFor(hash).replace(key, hash, oldValue, newValue); + } + + @Override + public V replace(K key, V value) { + checkNotNull(key); + checkNotNull(value); + int hash = hash(key); + return segmentFor(hash).replace(key, hash, value); + } + + @Override + public void clear() { + for (Segment segment : segments) { + segment.clear(); + } + } + + transient Set keySet; + + @Override + public Set keySet() { + Set ks = keySet; + return (ks != null) ? ks : (keySet = new KeySet()); + } + + transient Collection values; + + @Override + public Collection values() { + Collection vs = values; + return (vs != null) ? vs : (values = new Values()); + } + + transient Set> entrySet; + + @Override + public Set> entrySet() { + Set> es = entrySet; + return (es != null) ? es : (entrySet = new EntrySet()); + } + + // Iterator Support + + abstract class HashIterator implements Iterator { + + int nextSegmentIndex; + int nextTableIndex; + Segment currentSegment; + AtomicReferenceArray currentTable; + E nextEntry; + WriteThroughEntry nextExternal; + WriteThroughEntry lastReturned; + + HashIterator() { + nextSegmentIndex = segments.length - 1; + nextTableIndex = -1; + advance(); + } + + @Override + public abstract T next(); + + final void advance() { + nextExternal = null; + + if (nextInChain()) { + return; + } + + if (nextInTable()) { + return; + } + + while (nextSegmentIndex >= 0) { + currentSegment = segments[nextSegmentIndex--]; + if (currentSegment.count != 0) { + currentTable = currentSegment.table; + nextTableIndex = currentTable.length() - 1; + if (nextInTable()) { + return; + } + } + } + } + + /** Finds the next entry in the current chain. Returns {@code true} if an entry was found. */ + boolean nextInChain() { + if (nextEntry != null) { + for (nextEntry = nextEntry.getNext(); nextEntry != null; nextEntry = nextEntry.getNext()) { + if (advanceTo(nextEntry)) { + return true; + } + } + } + return false; + } + + /** Finds the next entry in the current table. Returns {@code true} if an entry was found. */ + boolean nextInTable() { + while (nextTableIndex >= 0) { + if ((nextEntry = currentTable.get(nextTableIndex--)) != null) { + if (advanceTo(nextEntry) || nextInChain()) { + return true; + } + } + } + return false; + } + + /** + * Advances to the given entry. Returns {@code true} if the entry was valid, {@code false} if it + * should be skipped. + */ + boolean advanceTo(E entry) { + try { + K key = entry.getKey(); + V value = getLiveValue(entry); + if (value != null) { + nextExternal = new WriteThroughEntry(key, value); + return true; + } else { + // Skip stale entry. + return false; + } + } finally { + currentSegment.postReadCleanup(); + } + } + + @Override + public boolean hasNext() { + return nextExternal != null; + } + + WriteThroughEntry nextEntry() { + if (nextExternal == null) { + throw new NoSuchElementException(); + } + lastReturned = nextExternal; + advance(); + return lastReturned; + } + + @Override + public void remove() { + checkRemove(lastReturned != null); + MapMakerInternalMap.this.remove(lastReturned.getKey()); + lastReturned = null; + } + } + + final class KeyIterator extends HashIterator { + + @Override + public K next() { + return nextEntry().getKey(); + } + } + + final class ValueIterator extends HashIterator { + + @Override + public V next() { + return nextEntry().getValue(); + } + } + + /** + * Custom Entry class used by EntryIterator.next(), that relays setValue changes to the underlying + * map. + */ + final class WriteThroughEntry extends AbstractMapEntry { + final K key; // non-null + V value; // non-null + + WriteThroughEntry(K key, V value) { + this.key = key; + this.value = value; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public boolean equals(Object object) { + // Cannot use key and value equivalence + if (object instanceof Entry) { + Entry that = (Entry) object; + return key.equals(that.getKey()) && value.equals(that.getValue()); + } + return false; + } + + @Override + public int hashCode() { + // Cannot use key and value equivalence + return key.hashCode() ^ value.hashCode(); + } + + @Override + public V setValue(V newValue) { + V oldValue = put(key, newValue); + value = newValue; // only if put succeeds + return oldValue; + } + } + + final class EntryIterator extends HashIterator> { + + @Override + public Entry next() { + return nextEntry(); + } + } + + final class KeySet extends SafeToArraySet { + + @Override + public Iterator iterator() { + return new KeyIterator(); + } + + @Override + public int size() { + return MapMakerInternalMap.this.size(); + } + + @Override + public boolean isEmpty() { + return MapMakerInternalMap.this.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return MapMakerInternalMap.this.containsKey(o); + } + + @Override + public boolean remove(Object o) { + return MapMakerInternalMap.this.remove(o) != null; + } + + @Override + public void clear() { + MapMakerInternalMap.this.clear(); + } + } + + final class Values extends AbstractCollection { + + @Override + public Iterator iterator() { + return new ValueIterator(); + } + + @Override + public int size() { + return MapMakerInternalMap.this.size(); + } + + @Override + public boolean isEmpty() { + return MapMakerInternalMap.this.isEmpty(); + } + + @Override + public boolean contains(Object o) { + return MapMakerInternalMap.this.containsValue(o); + } + + @Override + public void clear() { + MapMakerInternalMap.this.clear(); + } + + // super.toArray() may misbehave if size() is inaccurate, at least on old versions of Android. + // https://code.google.com/p/android/issues/detail?id=36519 / http://r.android.com/47508 + + @Override + public Object[] toArray() { + return toArrayList(this).toArray(); + } + + @Override + public T[] toArray(T[] a) { + return toArrayList(this).toArray(a); + } + } + + final class EntrySet extends SafeToArraySet> { + + @Override + public Iterator> iterator() { + return new EntryIterator(); + } + + @Override + public boolean contains(Object o) { + if (!(o instanceof Entry)) { + return false; + } + Entry e = (Entry) o; + Object key = e.getKey(); + if (key == null) { + return false; + } + V v = MapMakerInternalMap.this.get(key); + + return v != null && valueEquivalence().equivalent(e.getValue(), v); + } + + @Override + public boolean remove(Object o) { + if (!(o instanceof Entry)) { + return false; + } + Entry e = (Entry) o; + Object key = e.getKey(); + return key != null && MapMakerInternalMap.this.remove(key, e.getValue()); + } + + @Override + public int size() { + return MapMakerInternalMap.this.size(); + } + + @Override + public boolean isEmpty() { + return MapMakerInternalMap.this.isEmpty(); + } + + @Override + public void clear() { + MapMakerInternalMap.this.clear(); + } + } + + private abstract static class SafeToArraySet extends AbstractSet { + // super.toArray() may misbehave if size() is inaccurate, at least on old versions of Android. + // https://code.google.com/p/android/issues/detail?id=36519 / http://r.android.com/47508 + + @Override + public Object[] toArray() { + return toArrayList(this).toArray(); + } + + @Override + public T[] toArray(T[] a) { + return toArrayList(this).toArray(a); + } + } + + private static ArrayList toArrayList(Collection c) { + // Avoid calling ArrayList(Collection), which may call back into toArray. + ArrayList result = new ArrayList<>(c.size()); + Iterators.addAll(result, c.iterator()); + return result; + } + + // Serialization Support + + private static final long serialVersionUID = 5; + + Object writeReplace() { + return new SerializationProxy<>( + entryHelper.keyStrength(), + entryHelper.valueStrength(), + keyEquivalence, + entryHelper.valueStrength().defaultEquivalence(), + concurrencyLevel, + this); + } + + /** + * The actual object that gets serialized. Unfortunately, readResolve() doesn't get called when a + * circular dependency is present, so the proxy must be able to behave as the map itself. + */ + abstract static class AbstractSerializationProxy extends ForwardingConcurrentMap + implements Serializable { + private static final long serialVersionUID = 3; + + final Strength keyStrength; + final Strength valueStrength; + final Equivalence keyEquivalence; + final Equivalence valueEquivalence; + final int concurrencyLevel; + + transient ConcurrentMap delegate; + + AbstractSerializationProxy( + Strength keyStrength, + Strength valueStrength, + Equivalence keyEquivalence, + Equivalence valueEquivalence, + int concurrencyLevel, + ConcurrentMap delegate) { + this.keyStrength = keyStrength; + this.valueStrength = valueStrength; + this.keyEquivalence = keyEquivalence; + this.valueEquivalence = valueEquivalence; + this.concurrencyLevel = concurrencyLevel; + this.delegate = delegate; + } + + @Override + protected ConcurrentMap delegate() { + return delegate; + } + + void writeMapTo(ObjectOutputStream out) throws IOException { + out.writeInt(delegate.size()); + for (Entry entry : delegate.entrySet()) { + out.writeObject(entry.getKey()); + out.writeObject(entry.getValue()); + } + out.writeObject(null); // terminate entries + } + + @SuppressWarnings("deprecation") // serialization of deprecated feature + MapMaker readMapMaker(ObjectInputStream in) throws IOException { + int size = in.readInt(); + return new MapMaker() + .initialCapacity(size) + .setKeyStrength(keyStrength) + .setValueStrength(valueStrength) + .keyEquivalence(keyEquivalence) + .concurrencyLevel(concurrencyLevel); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + void readEntries(ObjectInputStream in) throws IOException, ClassNotFoundException { + while (true) { + K key = (K) in.readObject(); + if (key == null) { + break; // terminator + } + V value = (V) in.readObject(); + delegate.put(key, value); + } + } + } + + /** + * The actual object that gets serialized. Unfortunately, readResolve() doesn't get called when a + * circular dependency is present, so the proxy must be able to behave as the map itself. + */ + private static final class SerializationProxy extends AbstractSerializationProxy { + private static final long serialVersionUID = 3; + + SerializationProxy( + Strength keyStrength, + Strength valueStrength, + Equivalence keyEquivalence, + Equivalence valueEquivalence, + int concurrencyLevel, + ConcurrentMap delegate) { + super( + keyStrength, valueStrength, keyEquivalence, valueEquivalence, concurrencyLevel, delegate); + } + + private void writeObject(ObjectOutputStream out) throws IOException { + out.defaultWriteObject(); + writeMapTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + MapMaker mapMaker = readMapMaker(in); + delegate = mapMaker.makeMap(); + readEntries(in); + } + + private Object readResolve() { + return delegate; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Maps.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Maps.java new file mode 100644 index 0000000000000..8a6b005304615 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Maps.java @@ -0,0 +1,4250 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.compose; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Converter; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Equivalence; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapDifference.ValueDifference; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.Serializable; +import java.util.AbstractCollection; +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumMap; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.IdentityHashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.Properties; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.BinaryOperator; +import java.util.function.Consumer; +import java.util.stream.Collector; + + + +/** + * Static utility methods pertaining to {@link Map} instances (including instances of {@link + * SortedMap}, {@link BiMap}, etc.). Also see this class's counterparts {@link Lists}, {@link Sets}. + * + *

See the Guava User Guide article on {@code Maps}. + * + * @author Kevin Bourrillion + * @author Mike Bostock + * @author Isaac Shum + * @author Louis Wasserman + * @since 2.0 + */ +public final class Maps { + private Maps() {} + + private enum EntryFunction implements Function, Object> { + KEY { + @Override + + public Object apply(Entry entry) { + return entry.getKey(); + } + }, + VALUE { + @Override + + public Object apply(Entry entry) { + return entry.getValue(); + } + }; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Function, K> keyFunction() { + return (Function) EntryFunction.KEY; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static Function, V> valueFunction() { + return (Function) EntryFunction.VALUE; + } + + static Iterator keyIterator(Iterator> entryIterator) { + return new TransformedIterator, K>(entryIterator) { + @Override + K transform(Entry entry) { + return entry.getKey(); + } + }; + } + + static Iterator valueIterator(Iterator> entryIterator) { + return new TransformedIterator, V>(entryIterator) { + @Override + V transform(Entry entry) { + return entry.getValue(); + } + }; + } + + /** + * Returns an immutable map instance containing the given entries. Internally, the returned map + * will be backed by an {@link EnumMap}. + * + *

The iteration order of the returned map follows the enum's iteration order, not the order in + * which the elements appear in the given map. + * + * @param map the map to make an immutable copy of + * @return an immutable map containing those entries + * @since 14.0 + */ + + public static , V> ImmutableMap immutableEnumMap( + Map map) { + if (map instanceof ImmutableEnumMap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // safe covariant cast + ImmutableEnumMap result = (ImmutableEnumMap) map; + return result; + } + Iterator> entryItr = map.entrySet().iterator(); + if (!entryItr.hasNext()) { + return ImmutableMap.of(); + } + Entry entry1 = entryItr.next(); + K key1 = entry1.getKey(); + V value1 = entry1.getValue(); + checkEntryNotNull(key1, value1); + Class clazz = key1.getDeclaringClass(); + EnumMap enumMap = new EnumMap<>(clazz); + enumMap.put(key1, value1); + while (entryItr.hasNext()) { + Entry entry = entryItr.next(); + K key = entry.getKey(); + V value = entry.getValue(); + checkEntryNotNull(key, value); + enumMap.put(key, value); + } + return ImmutableEnumMap.asImmutable(enumMap); + } + + private static class Accumulator, V> { + private final BinaryOperator mergeFunction; + private EnumMap map = null; + + Accumulator(BinaryOperator mergeFunction) { + this.mergeFunction = mergeFunction; + } + + void put(K key, V value) { + if (map == null) { + map = new EnumMap<>(key.getDeclaringClass()); + } + map.merge(key, value, mergeFunction); + } + + Accumulator combine(Accumulator other) { + if (this.map == null) { + return other; + } else if (other.map == null) { + return this; + } else { + other.map.forEach(this::put); + return this; + } + } + + ImmutableMap toImmutableMap() { + return (map == null) ? ImmutableMap.of() : ImmutableEnumMap.asImmutable(map); + } + } + + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableMap} whose keys + * and values are the result of applying the provided mapping functions to the input elements. The + * resulting implementation is specialized for enum key types. The returned map and its views will + * iterate over keys in their enum definition order, not encounter order. + * + *

If the mapped keys contain duplicates, an {@code IllegalArgumentException} is thrown when + * the collection operation is performed. (This differs from the {@code Collector} returned by + * {@link java.util.stream.Collectors#toMap(java.util.function.Function, + * java.util.function.Function) Collectors.toMap(Function, Function)}, which throws an {@code + * IllegalStateException}.) + * + * @since 21.0 + */ + + public static , V> Collector> toImmutableEnumMap( + java.util.function.Function keyFunction, + java.util.function.Function valueFunction) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + return Collector.of( + () -> + new Accumulator( + (v1, v2) -> { + throw new IllegalArgumentException("Multiple values for key: " + v1 + ", " + v2); + }), + (accum, t) -> { + K key = checkNotNull(keyFunction.apply(t), "Null key for input %s", t); + V newValue = checkNotNull(valueFunction.apply(t), "Null value for input %s", t); + accum.put(key, newValue); + }, + Accumulator::combine, + Accumulator::toImmutableMap, + Collector.Characteristics.UNORDERED); + } + + /** + * Returns a {@link Collector} that accumulates elements into an {@code ImmutableMap} whose keys + * and values are the result of applying the provided mapping functions to the input elements. The + * resulting implementation is specialized for enum key types. The returned map and its views will + * iterate over keys in their enum definition order, not encounter order. + * + *

If the mapped keys contain duplicates, the values are merged using the specified merging + * function. + * + * @since 21.0 + */ + + public static , V> Collector> toImmutableEnumMap( + java.util.function.Function keyFunction, + java.util.function.Function valueFunction, + BinaryOperator mergeFunction) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + checkNotNull(mergeFunction); + // not UNORDERED because we don't know if mergeFunction is commutative + return Collector.of( + () -> new Accumulator(mergeFunction), + (accum, t) -> { + K key = checkNotNull(keyFunction.apply(t), "Null key for input %s", t); + V newValue = checkNotNull(valueFunction.apply(t), "Null value for input %s", t); + accum.put(key, newValue); + }, + Accumulator::combine, + Accumulator::toImmutableMap); + } + + /** + * Creates a mutable, empty {@code HashMap} instance. + * + *

Note: if mutability is not required, use {@link ImmutableMap#of()} instead. + * + *

Note: if {@code K} is an {@code enum} type, use {@link #newEnumMap} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code HashMap} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @return a new, empty {@code HashMap} + */ + public static HashMap newHashMap() { + return new HashMap<>(); + } + + /** + * Creates a mutable {@code HashMap} instance with the same mappings as the specified map. + * + *

Note: if mutability is not required, use {@link ImmutableMap#copyOf(Map)} instead. + * + *

Note: if {@code K} is an {@link Enum} type, use {@link #newEnumMap} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code HashMap} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @param map the mappings to be placed in the new map + * @return a new {@code HashMap} initialized with the mappings from {@code map} + */ + public static HashMap newHashMap(Map map) { + return new HashMap<>(map); + } + + /** + * Creates a {@code HashMap} instance, with a high enough "initial capacity" that it should + * hold {@code expectedSize} elements without growth. This behavior cannot be broadly guaranteed, + * but it is observed to be true for OpenJDK 1.7. It also can't be guaranteed that the method + * isn't inadvertently oversizing the returned map. + * + * @param expectedSize the number of entries you expect to add to the returned map + * @return a new, empty {@code HashMap} with enough capacity to hold {@code expectedSize} entries + * without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + */ + public static HashMap newHashMapWithExpectedSize(int expectedSize) { + return new HashMap<>(capacity(expectedSize)); + } + + /** + * Returns a capacity that is sufficient to keep the map from being resized as long as it grows no + * larger than expectedSize and the load factor is ≥ its default (0.75). + */ + static int capacity(int expectedSize) { + if (expectedSize < 3) { + checkNonnegative(expectedSize, "expectedSize"); + return expectedSize + 1; + } + if (expectedSize < Ints.MAX_POWER_OF_TWO) { + // This is the calculation used in JDK8 to resize when a putAll + // happens; it seems to be the most conservative calculation we + // can make. 0.75 is the default load factor. + return (int) ((float) expectedSize / 0.75F + 1.0F); + } + return Integer.MAX_VALUE; // any large value + } + + /** + * Creates a mutable, empty, insertion-ordered {@code LinkedHashMap} instance. + * + *

Note: if mutability is not required, use {@link ImmutableMap#of()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code LinkedHashMap} constructor directly, taking advantage of + * the new "diamond" syntax. + * + * @return a new, empty {@code LinkedHashMap} + */ + public static LinkedHashMap newLinkedHashMap() { + return new LinkedHashMap<>(); + } + + /** + * Creates a mutable, insertion-ordered {@code LinkedHashMap} instance with the same + * mappings as the specified map. + * + *

Note: if mutability is not required, use {@link ImmutableMap#copyOf(Map)} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code LinkedHashMap} constructor directly, taking advantage of + * the new "diamond" syntax. + * + * @param map the mappings to be placed in the new map + * @return a new, {@code LinkedHashMap} initialized with the mappings from {@code map} + */ + public static LinkedHashMap newLinkedHashMap(Map map) { + return new LinkedHashMap<>(map); + } + + /** + * Creates a {@code LinkedHashMap} instance, with a high enough "initial capacity" that it + * should hold {@code expectedSize} elements without growth. This behavior cannot be + * broadly guaranteed, but it is observed to be true for OpenJDK 1.7. It also can't be guaranteed + * that the method isn't inadvertently oversizing the returned map. + * + * @param expectedSize the number of entries you expect to add to the returned map + * @return a new, empty {@code LinkedHashMap} with enough capacity to hold {@code expectedSize} + * entries without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + * @since 19.0 + */ + public static LinkedHashMap newLinkedHashMapWithExpectedSize(int expectedSize) { + return new LinkedHashMap<>(capacity(expectedSize)); + } + + /** + * Creates a new empty {@link ConcurrentHashMap} instance. + * + * @since 3.0 + */ + public static ConcurrentMap newConcurrentMap() { + return new ConcurrentHashMap<>(); + } + + /** + * Creates a mutable, empty {@code TreeMap} instance using the natural ordering of its + * elements. + * + *

Note: if mutability is not required, use {@link ImmutableSortedMap#of()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code TreeMap} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @return a new, empty {@code TreeMap} + */ + @SuppressWarnings("rawtypes") + public static TreeMap newTreeMap() { + return new TreeMap<>(); + } + + /** + * Creates a mutable {@code TreeMap} instance with the same mappings as the specified map + * and using the same ordering as the specified map. + * + *

Note: if mutability is not required, use {@link + * ImmutableSortedMap#copyOfSorted(SortedMap)} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code TreeMap} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @param map the sorted map whose mappings are to be placed in the new map and whose comparator + * is to be used to sort the new map + * @return a new {@code TreeMap} initialized with the mappings from {@code map} and using the + * comparator of {@code map} + */ + public static TreeMap newTreeMap(SortedMap map) { + return new TreeMap<>(map); + } + + /** + * Creates a mutable, empty {@code TreeMap} instance using the given comparator. + * + *

Note: if mutability is not required, use {@code + * ImmutableSortedMap.orderedBy(comparator).build()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code TreeMap} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @param comparator the comparator to sort the keys with + * @return a new, empty {@code TreeMap} + */ + public static TreeMap newTreeMap( + Comparator comparator) { + // Ideally, the extra type parameter "C" shouldn't be necessary. It is a + // work-around of a compiler type inference quirk that prevents the + // following code from being compiled: + // Comparator> comparator = null; + // Map, String> map = newTreeMap(comparator); + return new TreeMap<>(comparator); + } + + /** + * Creates an {@code EnumMap} instance. + * + * @param type the key type for this map + * @return a new, empty {@code EnumMap} + */ + public static , V> EnumMap newEnumMap(Class type) { + return new EnumMap<>(checkNotNull(type)); + } + + /** + * Creates an {@code EnumMap} with the same mappings as the specified map. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code EnumMap} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @param map the map from which to initialize this {@code EnumMap} + * @return a new {@code EnumMap} initialized with the mappings from {@code map} + * @throws IllegalArgumentException if {@code m} is not an {@code EnumMap} instance and contains + * no mappings + */ + public static , V> EnumMap newEnumMap(Map map) { + return new EnumMap<>(map); + } + + /** + * Creates an {@code IdentityHashMap} instance. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code IdentityHashMap} constructor directly, taking advantage of + * the new "diamond" syntax. + * + * @return a new, empty {@code IdentityHashMap} + */ + public static IdentityHashMap newIdentityHashMap() { + return new IdentityHashMap<>(); + } + + /** + * Computes the difference between two maps. This difference is an immutable snapshot of the state + * of the maps at the time this method is called. It will never change, even if the maps change at + * a later time. + * + *

Since this method uses {@code HashMap} instances internally, the keys of the supplied maps + * must be well-behaved with respect to {@link Object#equals} and {@link Object#hashCode}. + * + *

Note:If you only need to know whether two maps have the same mappings, call {@code + * left.equals(right)} instead of this method. + * + * @param left the map to treat as the "left" map for purposes of comparison + * @param right the map to treat as the "right" map for purposes of comparison + * @return the difference between the two maps + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static MapDifference difference( + Map left, Map right) { + if (left instanceof SortedMap) { + SortedMap sortedLeft = (SortedMap) left; + return difference(sortedLeft, right); + } + return difference(left, right, Equivalence.equals()); + } + + /** + * Computes the difference between two maps. This difference is an immutable snapshot of the state + * of the maps at the time this method is called. It will never change, even if the maps change at + * a later time. + * + *

Since this method uses {@code HashMap} instances internally, the keys of the supplied maps + * must be well-behaved with respect to {@link Object#equals} and {@link Object#hashCode}. + * + * @param left the map to treat as the "left" map for purposes of comparison + * @param right the map to treat as the "right" map for purposes of comparison + * @param valueEquivalence the equivalence relationship to use to compare values + * @return the difference between the two maps + * @since 10.0 + */ + public static MapDifference difference( + Map left, + Map right, + Equivalence valueEquivalence) { + Preconditions.checkNotNull(valueEquivalence); + + Map onlyOnLeft = newLinkedHashMap(); + Map onlyOnRight = new LinkedHashMap<>(right); // will whittle it down + Map onBoth = newLinkedHashMap(); + Map> differences = newLinkedHashMap(); + doDifference(left, right, valueEquivalence, onlyOnLeft, onlyOnRight, onBoth, differences); + return new MapDifferenceImpl<>(onlyOnLeft, onlyOnRight, onBoth, differences); + } + + /** + * Computes the difference between two sorted maps, using the comparator of the left map, or + * {@code Ordering.natural()} if the left map uses the natural ordering of its elements. This + * difference is an immutable snapshot of the state of the maps at the time this method is called. + * It will never change, even if the maps change at a later time. + * + *

Since this method uses {@code TreeMap} instances internally, the keys of the right map must + * all compare as distinct according to the comparator of the left map. + * + *

Note:If you only need to know whether two sorted maps have the same mappings, call + * {@code left.equals(right)} instead of this method. + * + * @param left the map to treat as the "left" map for purposes of comparison + * @param right the map to treat as the "right" map for purposes of comparison + * @return the difference between the two maps + * @since 11.0 + */ + public static SortedMapDifference difference( + SortedMap left, Map right) { + checkNotNull(left); + checkNotNull(right); + Comparator comparator = orNaturalOrder(left.comparator()); + SortedMap onlyOnLeft = Maps.newTreeMap(comparator); + SortedMap onlyOnRight = Maps.newTreeMap(comparator); + onlyOnRight.putAll(right); // will whittle it down + SortedMap onBoth = Maps.newTreeMap(comparator); + SortedMap> differences = Maps.newTreeMap(comparator); + doDifference(left, right, Equivalence.equals(), onlyOnLeft, onlyOnRight, onBoth, differences); + return new SortedMapDifferenceImpl<>(onlyOnLeft, onlyOnRight, onBoth, differences); + } + + private static void doDifference( + Map left, + Map right, + Equivalence valueEquivalence, + Map onlyOnLeft, + Map onlyOnRight, + Map onBoth, + Map> differences) { + for (Entry entry : left.entrySet()) { + K leftKey = entry.getKey(); + V leftValue = entry.getValue(); + if (right.containsKey(leftKey)) { + V rightValue = onlyOnRight.remove(leftKey); + if (valueEquivalence.equivalent(leftValue, rightValue)) { + onBoth.put(leftKey, leftValue); + } else { + differences.put(leftKey, ValueDifferenceImpl.create(leftValue, rightValue)); + } + } else { + onlyOnLeft.put(leftKey, leftValue); + } + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private static Map unmodifiableMap(Map map) { + if (map instanceof SortedMap) { + return Collections.unmodifiableSortedMap((SortedMap) map); + } else { + return Collections.unmodifiableMap(map); + } + } + + static class MapDifferenceImpl implements MapDifference { + final Map onlyOnLeft; + final Map onlyOnRight; + final Map onBoth; + final Map> differences; + + MapDifferenceImpl( + Map onlyOnLeft, + Map onlyOnRight, + Map onBoth, + Map> differences) { + this.onlyOnLeft = unmodifiableMap(onlyOnLeft); + this.onlyOnRight = unmodifiableMap(onlyOnRight); + this.onBoth = unmodifiableMap(onBoth); + this.differences = unmodifiableMap(differences); + } + + @Override + public boolean areEqual() { + return onlyOnLeft.isEmpty() && onlyOnRight.isEmpty() && differences.isEmpty(); + } + + @Override + public Map entriesOnlyOnLeft() { + return onlyOnLeft; + } + + @Override + public Map entriesOnlyOnRight() { + return onlyOnRight; + } + + @Override + public Map entriesInCommon() { + return onBoth; + } + + @Override + public Map> entriesDiffering() { + return differences; + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof MapDifference) { + MapDifference other = (MapDifference) object; + return entriesOnlyOnLeft().equals(other.entriesOnlyOnLeft()) + && entriesOnlyOnRight().equals(other.entriesOnlyOnRight()) + && entriesInCommon().equals(other.entriesInCommon()) + && entriesDiffering().equals(other.entriesDiffering()); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode( + entriesOnlyOnLeft(), entriesOnlyOnRight(), entriesInCommon(), entriesDiffering()); + } + + @Override + public String toString() { + if (areEqual()) { + return "equal"; + } + + StringBuilder result = new StringBuilder("not equal"); + if (!onlyOnLeft.isEmpty()) { + result.append(": only on left=").append(onlyOnLeft); + } + if (!onlyOnRight.isEmpty()) { + result.append(": only on right=").append(onlyOnRight); + } + if (!differences.isEmpty()) { + result.append(": value differences=").append(differences); + } + return result.toString(); + } + } + + static class ValueDifferenceImpl implements ValueDifference { + private final V left; + private final V right; + + static ValueDifference create(V left, V right) { + return new ValueDifferenceImpl(left, right); + } + + private ValueDifferenceImpl(V left, V right) { + this.left = left; + this.right = right; + } + + @Override + public V leftValue() { + return left; + } + + @Override + public V rightValue() { + return right; + } + + @Override + public boolean equals(Object object) { + if (object instanceof MapDifference.ValueDifference) { + ValueDifference that = (ValueDifference) object; + return Objects.equal(this.left, that.leftValue()) + && Objects.equal(this.right, that.rightValue()); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(left, right); + } + + @Override + public String toString() { + return "(" + left + ", " + right + ")"; + } + } + + static class SortedMapDifferenceImpl extends MapDifferenceImpl + implements SortedMapDifference { + SortedMapDifferenceImpl( + SortedMap onlyOnLeft, + SortedMap onlyOnRight, + SortedMap onBoth, + SortedMap> differences) { + super(onlyOnLeft, onlyOnRight, onBoth, differences); + } + + @Override + public SortedMap> entriesDiffering() { + return (SortedMap>) super.entriesDiffering(); + } + + @Override + public SortedMap entriesInCommon() { + return (SortedMap) super.entriesInCommon(); + } + + @Override + public SortedMap entriesOnlyOnLeft() { + return (SortedMap) super.entriesOnlyOnLeft(); + } + + @Override + public SortedMap entriesOnlyOnRight() { + return (SortedMap) super.entriesOnlyOnRight(); + } + } + + /** + * Returns the specified comparator if not null; otherwise returns {@code Ordering.natural()}. + * This method is an abomination of generics; the only purpose of this method is to contain the + * ugly type-casting in one place. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + static Comparator orNaturalOrder(Comparator comparator) { + if (comparator != null) { // can't use ? : because of javac bug 5080917 + return comparator; + } + return (Comparator) Ordering.natural(); + } + + /** + * Returns a live {@link Map} view whose keys are the contents of {@code set} and whose values are + * computed on demand using {@code function}. To get an immutable copy instead, use {@link + * #toMap(Iterable, Function)}. + * + *

Specifically, for each {@code k} in the backing set, the returned map has an entry mapping + * {@code k} to {@code function.apply(k)}. The {@code keySet}, {@code values}, and {@code + * entrySet} views of the returned map iterate in the same order as the backing set. + * + *

Modifications to the backing set are read through to the returned map. The returned map + * supports removal operations if the backing set does. Removal operations write through to the + * backing set. The returned map does not support put operations. + * + *

Warning: If the function rejects {@code null}, caution is required to make sure the + * set does not contain {@code null}, because the view cannot stop {@code null} from being added + * to the set. + * + *

Warning: This method assumes that for any instance {@code k} of key type {@code K}, + * {@code k.equals(k2)} implies that {@code k2} is also of type {@code K}. Using a key type for + * which this may not hold, such as {@code ArrayList}, may risk a {@code ClassCastException} when + * calling methods on the resulting map view. + * + * @since 14.0 + */ + public static Map asMap(Set set, Function function) { + return new AsMapView<>(set, function); + } + + /** + * Returns a view of the sorted set as a map, mapping keys from the set according to the specified + * function. + * + *

Specifically, for each {@code k} in the backing set, the returned map has an entry mapping + * {@code k} to {@code function.apply(k)}. The {@code keySet}, {@code values}, and {@code + * entrySet} views of the returned map iterate in the same order as the backing set. + * + *

Modifications to the backing set are read through to the returned map. The returned map + * supports removal operations if the backing set does. Removal operations write through to the + * backing set. The returned map does not support put operations. + * + *

Warning: If the function rejects {@code null}, caution is required to make sure the + * set does not contain {@code null}, because the view cannot stop {@code null} from being added + * to the set. + * + *

Warning: This method assumes that for any instance {@code k} of key type {@code K}, + * {@code k.equals(k2)} implies that {@code k2} is also of type {@code K}. Using a key type for + * which this may not hold, such as {@code ArrayList}, may risk a {@code ClassCastException} when + * calling methods on the resulting map view. + * + * @since 14.0 + */ + public static SortedMap asMap(SortedSet set, Function function) { + return new SortedAsMapView<>(set, function); + } + + /** + * Returns a view of the navigable set as a map, mapping keys from the set according to the + * specified function. + * + *

Specifically, for each {@code k} in the backing set, the returned map has an entry mapping + * {@code k} to {@code function.apply(k)}. The {@code keySet}, {@code values}, and {@code + * entrySet} views of the returned map iterate in the same order as the backing set. + * + *

Modifications to the backing set are read through to the returned map. The returned map + * supports removal operations if the backing set does. Removal operations write through to the + * backing set. The returned map does not support put operations. + * + *

Warning: If the function rejects {@code null}, caution is required to make sure the + * set does not contain {@code null}, because the view cannot stop {@code null} from being added + * to the set. + * + *

Warning: This method assumes that for any instance {@code k} of key type {@code K}, + * {@code k.equals(k2)} implies that {@code k2} is also of type {@code K}. Using a key type for + * which this may not hold, such as {@code ArrayList}, may risk a {@code ClassCastException} when + * calling methods on the resulting map view. + * + * @since 14.0 + */ + public static NavigableMap asMap( + NavigableSet set, Function function) { + return new NavigableAsMapView<>(set, function); + } + + private static class AsMapView extends ViewCachingAbstractMap { + + private final Set set; + final Function function; + + Set backingSet() { + return set; + } + + AsMapView(Set set, Function function) { + this.set = checkNotNull(set); + this.function = checkNotNull(function); + } + + @Override + public Set createKeySet() { + return removeOnlySet(backingSet()); + } + + @Override + Collection createValues() { + return Collections2.transform(set, function); + } + + @Override + public int size() { + return backingSet().size(); + } + + @Override + public boolean containsKey(Object key) { + return backingSet().contains(key); + } + + @Override + public V get(Object key) { + return getOrDefault(key, null); + } + + @Override + public V getOrDefault(Object key, V defaultValue) { + if (Collections2.safeContains(backingSet(), key)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // unsafe, but Javadoc warns about it + K k = (K) key; + return function.apply(k); + } else { + return defaultValue; + } + } + + @Override + public V remove(Object key) { + if (backingSet().remove(key)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // unsafe, but Javadoc warns about it + K k = (K) key; + return function.apply(k); + } else { + return null; + } + } + + @Override + public void clear() { + backingSet().clear(); + } + + @Override + protected Set> createEntrySet() { + class EntrySetImpl extends EntrySet { + @Override + Map map() { + return AsMapView.this; + } + + @Override + public Iterator> iterator() { + return asMapEntryIterator(backingSet(), function); + } + } + return new EntrySetImpl(); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + // avoids allocation of entries + backingSet().forEach(k -> action.accept(k, function.apply(k))); + } + } + + static Iterator> asMapEntryIterator( + Set set, final Function function) { + return new TransformedIterator>(set.iterator()) { + @Override + Entry transform(final K key) { + return immutableEntry(key, function.apply(key)); + } + }; + } + + private static class SortedAsMapView extends AsMapView implements SortedMap { + + SortedAsMapView(SortedSet set, Function function) { + super(set, function); + } + + @Override + SortedSet backingSet() { + return (SortedSet) super.backingSet(); + } + + @Override + public Comparator comparator() { + return backingSet().comparator(); + } + + @Override + public Set keySet() { + return removeOnlySortedSet(backingSet()); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return asMap(backingSet().subSet(fromKey, toKey), function); + } + + @Override + public SortedMap headMap(K toKey) { + return asMap(backingSet().headSet(toKey), function); + } + + @Override + public SortedMap tailMap(K fromKey) { + return asMap(backingSet().tailSet(fromKey), function); + } + + @Override + public K firstKey() { + return backingSet().first(); + } + + @Override + public K lastKey() { + return backingSet().last(); + } + } + + private static final class NavigableAsMapView extends AbstractNavigableMap { + /* + * Using AbstractNavigableMap is simpler than extending SortedAsMapView and rewriting all the + * NavigableMap methods. + */ + + private final NavigableSet set; + private final Function function; + + NavigableAsMapView(NavigableSet ks, Function vFunction) { + this.set = checkNotNull(ks); + this.function = checkNotNull(vFunction); + } + + @Override + public NavigableMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return asMap(set.subSet(fromKey, fromInclusive, toKey, toInclusive), function); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + return asMap(set.headSet(toKey, inclusive), function); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + return asMap(set.tailSet(fromKey, inclusive), function); + } + + @Override + public Comparator comparator() { + return set.comparator(); + } + + @Override + + public V get(Object key) { + return getOrDefault(key, null); + } + + @Override + + public V getOrDefault(Object key, V defaultValue) { + if (Collections2.safeContains(set, key)) { + @SuppressWarnings({"unchecked", "rawtypes"}) // unsafe, but Javadoc warns about it + K k = (K) key; + return function.apply(k); + } else { + return defaultValue; + } + } + + @Override + public void clear() { + set.clear(); + } + + @Override + Iterator> entryIterator() { + return asMapEntryIterator(set, function); + } + + @Override + Spliterator> entrySpliterator() { + return CollectSpliterators.map(set.spliterator(), e -> immutableEntry(e, function.apply(e))); + } + + @Override + public void forEach(BiConsumer action) { + set.forEach(k -> action.accept(k, function.apply(k))); + } + + @Override + Iterator> descendingEntryIterator() { + return descendingMap().entrySet().iterator(); + } + + @Override + public NavigableSet navigableKeySet() { + return removeOnlyNavigableSet(set); + } + + @Override + public int size() { + return set.size(); + } + + @Override + public NavigableMap descendingMap() { + return asMap(set.descendingSet(), function); + } + } + + private static Set removeOnlySet(final Set set) { + return new ForwardingSet() { + @Override + protected Set delegate() { + return set; + } + + @Override + public boolean add(E element) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection es) { + throw new UnsupportedOperationException(); + } + }; + } + + private static SortedSet removeOnlySortedSet(final SortedSet set) { + return new ForwardingSortedSet() { + @Override + protected SortedSet delegate() { + return set; + } + + @Override + public boolean add(E element) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection es) { + throw new UnsupportedOperationException(); + } + + @Override + public SortedSet headSet(E toElement) { + return removeOnlySortedSet(super.headSet(toElement)); + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return removeOnlySortedSet(super.subSet(fromElement, toElement)); + } + + @Override + public SortedSet tailSet(E fromElement) { + return removeOnlySortedSet(super.tailSet(fromElement)); + } + }; + } + + private static NavigableSet removeOnlyNavigableSet(final NavigableSet set) { + return new ForwardingNavigableSet() { + @Override + protected NavigableSet delegate() { + return set; + } + + @Override + public boolean add(E element) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection es) { + throw new UnsupportedOperationException(); + } + + @Override + public SortedSet headSet(E toElement) { + return removeOnlySortedSet(super.headSet(toElement)); + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + return removeOnlyNavigableSet(super.headSet(toElement, inclusive)); + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return removeOnlySortedSet(super.subSet(fromElement, toElement)); + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return removeOnlyNavigableSet( + super.subSet(fromElement, fromInclusive, toElement, toInclusive)); + } + + @Override + public SortedSet tailSet(E fromElement) { + return removeOnlySortedSet(super.tailSet(fromElement)); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + return removeOnlyNavigableSet(super.tailSet(fromElement, inclusive)); + } + + @Override + public NavigableSet descendingSet() { + return removeOnlyNavigableSet(super.descendingSet()); + } + }; + } + + /** + * Returns an immutable map whose keys are the distinct elements of {@code keys} and whose value + * for each key was computed by {@code valueFunction}. The map's iteration order is the order of + * the first appearance of each key in {@code keys}. + * + *

When there are multiple instances of a key in {@code keys}, it is unspecified whether {@code + * valueFunction} will be applied to more than one instance of that key and, if it is, which + * result will be mapped to that key in the returned map. + * + *

If {@code keys} is a {@link Set}, a live view can be obtained instead of a copy using {@link + * Maps#asMap(Set, Function)}. + * + * @throws NullPointerException if any element of {@code keys} is {@code null}, or if {@code + * valueFunction} produces {@code null} for any key + * @since 14.0 + */ + public static ImmutableMap toMap( + Iterable keys, Function valueFunction) { + return toMap(keys.iterator(), valueFunction); + } + + /** + * Returns an immutable map whose keys are the distinct elements of {@code keys} and whose value + * for each key was computed by {@code valueFunction}. The map's iteration order is the order of + * the first appearance of each key in {@code keys}. + * + *

When there are multiple instances of a key in {@code keys}, it is unspecified whether {@code + * valueFunction} will be applied to more than one instance of that key and, if it is, which + * result will be mapped to that key in the returned map. + * + * @throws NullPointerException if any element of {@code keys} is {@code null}, or if {@code + * valueFunction} produces {@code null} for any key + * @since 14.0 + */ + public static ImmutableMap toMap( + Iterator keys, Function valueFunction) { + checkNotNull(valueFunction); + // Using LHM instead of a builder so as not to fail on duplicate keys + Map builder = newLinkedHashMap(); + while (keys.hasNext()) { + K key = keys.next(); + builder.put(key, valueFunction.apply(key)); + } + return ImmutableMap.copyOf(builder); + } + + /** + * Returns a map with the given {@code values}, indexed by keys derived from those values. In + * other words, each input value produces an entry in the map whose key is the result of applying + * {@code keyFunction} to that value. These entries appear in the same order as the input values. + * Example usage: + * + *

{@code
+   * Color red = new Color("red", 255, 0, 0);
+   * ...
+   * ImmutableSet allColors = ImmutableSet.of(red, green, blue);
+   *
+   * Map colorForName =
+   *     uniqueIndex(allColors, toStringFunction());
+   * assertThat(colorForName).containsEntry("red", red);
+   * }
+ * + *

If your index may associate multiple values with each key, use {@link + * Multimaps#index(Iterable, Function) Multimaps.index}. + * + * @param values the values to use when constructing the {@code Map} + * @param keyFunction the function used to produce the key for each value + * @return a map mapping the result of evaluating the function {@code keyFunction} on each value + * in the input collection to that value + * @throws IllegalArgumentException if {@code keyFunction} produces the same key for more than one + * value in the input collection + * @throws NullPointerException if any element of {@code values} is {@code null}, or if {@code + * keyFunction} produces {@code null} for any value + */ + public static ImmutableMap uniqueIndex( + Iterable values, Function keyFunction) { + // TODO(lowasser): consider presizing the builder if values is a Collection + return uniqueIndex(values.iterator(), keyFunction); + } + + /** + * Returns a map with the given {@code values}, indexed by keys derived from those values. In + * other words, each input value produces an entry in the map whose key is the result of applying + * {@code keyFunction} to that value. These entries appear in the same order as the input values. + * Example usage: + * + *

{@code
+   * Color red = new Color("red", 255, 0, 0);
+   * ...
+   * Iterator allColors = ImmutableSet.of(red, green, blue).iterator();
+   *
+   * Map colorForName =
+   *     uniqueIndex(allColors, toStringFunction());
+   * assertThat(colorForName).containsEntry("red", red);
+   * }
+ * + *

If your index may associate multiple values with each key, use {@link + * Multimaps#index(Iterator, Function) Multimaps.index}. + * + * @param values the values to use when constructing the {@code Map} + * @param keyFunction the function used to produce the key for each value + * @return a map mapping the result of evaluating the function {@code keyFunction} on each value + * in the input collection to that value + * @throws IllegalArgumentException if {@code keyFunction} produces the same key for more than one + * value in the input collection + * @throws NullPointerException if any element of {@code values} is {@code null}, or if {@code + * keyFunction} produces {@code null} for any value + * @since 10.0 + */ + public static ImmutableMap uniqueIndex( + Iterator values, Function keyFunction) { + checkNotNull(keyFunction); + ImmutableMap.Builder builder = ImmutableMap.builder(); + while (values.hasNext()) { + V value = values.next(); + builder.put(keyFunction.apply(value), value); + } + try { + return builder.build(); + } catch (IllegalArgumentException duplicateKeys) { + throw new IllegalArgumentException( + duplicateKeys.getMessage() + + ". To index multiple values under a key, use Multimaps.index."); + } + } + + /** + * Creates an {@code ImmutableMap} from a {@code Properties} instance. Properties + * normally derive from {@code Map}, but they typically contain strings, which is + * awkward. This method lets you get a plain-old-{@code Map} out of a {@code Properties}. + * + * @param properties a {@code Properties} object to be converted + * @return an immutable map containing all the entries in {@code properties} + * @throws ClassCastException if any key in {@code Properties} is not a {@code String} + * @throws NullPointerException if any key or value in {@code Properties} is null + */ + public static ImmutableMap fromProperties(Properties properties) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + + for (Enumeration e = properties.propertyNames(); e.hasMoreElements(); ) { + String key = (String) e.nextElement(); + builder.put(key, properties.getProperty(key)); + } + + return builder.build(); + } + + /** + * Returns an immutable map entry with the specified key and value. The {@link Entry#setValue} + * operation throws an {@link UnsupportedOperationException}. + * + *

The returned entry is serializable. + * + * @param key the key to be associated with the returned entry + * @param value the value to be associated with the returned entry + */ + public static Entry immutableEntry(K key, V value) { + return new ImmutableEntry<>(key, value); + } + + /** + * Returns an unmodifiable view of the specified set of entries. The {@link Entry#setValue} + * operation throws an {@link UnsupportedOperationException}, as do any operations that would + * modify the returned set. + * + * @param entrySet the entries for which to return an unmodifiable view + * @return an unmodifiable view of the entries + */ + static Set> unmodifiableEntrySet(Set> entrySet) { + return new UnmodifiableEntrySet<>(Collections.unmodifiableSet(entrySet)); + } + + /** + * Returns an unmodifiable view of the specified map entry. The {@link Entry#setValue} operation + * throws an {@link UnsupportedOperationException}. This also has the side-effect of redefining + * {@code equals} to comply with the Entry contract, to avoid a possible nefarious implementation + * of equals. + * + * @param entry the entry for which to return an unmodifiable view + * @return an unmodifiable view of the entry + */ + static Entry unmodifiableEntry(final Entry entry) { + checkNotNull(entry); + return new AbstractMapEntry() { + @Override + public K getKey() { + return entry.getKey(); + } + + @Override + public V getValue() { + return entry.getValue(); + } + }; + } + + static UnmodifiableIterator> unmodifiableEntryIterator( + final Iterator> entryIterator) { + return new UnmodifiableIterator>() { + @Override + public boolean hasNext() { + return entryIterator.hasNext(); + } + + @Override + public Entry next() { + return unmodifiableEntry(entryIterator.next()); + } + }; + } + + static class UnmodifiableEntries extends ForwardingCollection> { + private final Collection> entries; + + UnmodifiableEntries(Collection> entries) { + this.entries = entries; + } + + @Override + protected Collection> delegate() { + return entries; + } + + @Override + public Iterator> iterator() { + return unmodifiableEntryIterator(entries.iterator()); + } + + // See java.util.Collections.UnmodifiableEntrySet for details on attacks. + + @Override + public Object[] toArray() { + return standardToArray(); + } + + @Override + public T[] toArray(T[] array) { + return standardToArray(array); + } + } + + /** @see Maps#unmodifiableEntrySet(Set) */ + static class UnmodifiableEntrySet extends UnmodifiableEntries + implements Set> { + UnmodifiableEntrySet(Set> entries) { + super(entries); + } + + // See java.util.Collections.UnmodifiableEntrySet for details on attacks. + + @Override + public boolean equals(Object object) { + return Sets.equalsImpl(this, object); + } + + @Override + public int hashCode() { + return Sets.hashCodeImpl(this); + } + } + + /** + * Returns a {@link Converter} that converts values using {@link BiMap#get bimap.get()}, and whose + * inverse view converts values using {@link BiMap#inverse bimap.inverse()}{@code .get()}. + * + *

To use a plain {@link Map} as a {@link Function}, see {@link + * com.azure.cosmos.kafka.connect.implementation.guava25.base.Functions#forMap(Map)} or {@link + * com.azure.cosmos.kafka.connect.implementation.guava25.base.Functions#forMap(Map, Object)}. + * + * @since 16.0 + */ + + public static Converter asConverter(final BiMap bimap) { + return new BiMapConverter<>(bimap); + } + + private static final class BiMapConverter extends Converter implements Serializable { + private final BiMap bimap; + + BiMapConverter(BiMap bimap) { + this.bimap = checkNotNull(bimap); + } + + @Override + protected B doForward(A a) { + return convert(bimap, a); + } + + @Override + protected A doBackward(B b) { + return convert(bimap.inverse(), b); + } + + private static Y convert(BiMap bimap, X input) { + Y output = bimap.get(input); + checkArgument(output != null, "No non-null mapping present for input: %s", input); + return output; + } + + @Override + public boolean equals(Object object) { + if (object instanceof BiMapConverter) { + BiMapConverter that = (BiMapConverter) object; + return this.bimap.equals(that.bimap); + } + return false; + } + + @Override + public int hashCode() { + return bimap.hashCode(); + } + + // There's really no good way to implement toString() without printing the entire BiMap, right? + @Override + public String toString() { + return "Maps.asConverter(" + bimap + ")"; + } + + private static final long serialVersionUID = 0L; + } + + /** + * Returns a synchronized (thread-safe) bimap backed by the specified bimap. In order to guarantee + * serial access, it is critical that all access to the backing bimap is accomplished + * through the returned bimap. + * + *

It is imperative that the user manually synchronize on the returned map when accessing any + * of its collection views: + * + *

{@code
+   * BiMap map = Maps.synchronizedBiMap(
+   *     HashBiMap.create());
+   * ...
+   * Set set = map.keySet();  // Needn't be in synchronized block
+   * ...
+   * synchronized (map) {  // Synchronizing on map, not set!
+   *   Iterator it = set.iterator(); // Must be in synchronized block
+   *   while (it.hasNext()) {
+   *     foo(it.next());
+   *   }
+   * }
+   * }
+ * + *

Failure to follow this advice may result in non-deterministic behavior. + * + *

The returned bimap will be serializable if the specified bimap is serializable. + * + * @param bimap the bimap to be wrapped in a synchronized view + * @return a synchronized view of the specified bimap + */ + public static BiMap synchronizedBiMap(BiMap bimap) { + return Synchronized.biMap(bimap, null); + } + + /** + * Returns an unmodifiable view of the specified bimap. This method allows modules to provide + * users with "read-only" access to internal bimaps. Query operations on the returned bimap "read + * through" to the specified bimap, and attempts to modify the returned map, whether direct or via + * its collection views, result in an {@code UnsupportedOperationException}. + * + *

The returned bimap will be serializable if the specified bimap is serializable. + * + * @param bimap the bimap for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified bimap + */ + public static BiMap unmodifiableBiMap(BiMap bimap) { + return new UnmodifiableBiMap<>(bimap, null); + } + + /** @see Maps#unmodifiableBiMap(BiMap) */ + private static class UnmodifiableBiMap extends ForwardingMap + implements BiMap, Serializable { + final Map unmodifiableMap; + final BiMap delegate; + BiMap inverse; + transient Set values; + + UnmodifiableBiMap(BiMap delegate, BiMap inverse) { + unmodifiableMap = Collections.unmodifiableMap(delegate); + this.delegate = delegate; + this.inverse = inverse; + } + + @Override + protected Map delegate() { + return unmodifiableMap; + } + + @Override + public V forcePut(K key, V value) { + throw new UnsupportedOperationException(); + } + + @Override + public BiMap inverse() { + BiMap result = inverse; + return (result == null) + ? inverse = new UnmodifiableBiMap<>(delegate.inverse(), this) + : result; + } + + @Override + public Set values() { + Set result = values; + return (result == null) ? values = Collections.unmodifiableSet(delegate.values()) : result; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a view of a map where each value is transformed by a function. All other properties of + * the map, such as iteration order, are left intact. For example, the code: + * + *

{@code
+   * Map map = ImmutableMap.of("a", 4, "b", 9);
+   * Function sqrt =
+   *     new Function() {
+   *       public Double apply(Integer in) {
+   *         return Math.sqrt((int) in);
+   *       }
+   *     };
+   * Map transformed = Maps.transformValues(map, sqrt);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {a=2.0, b=3.0}}. + * + *

Changes in the underlying map are reflected in this view. Conversely, this view supports + * removal operations, and these are reflected in the underlying map. + * + *

It's acceptable for the underlying map to contain null keys, and even null values provided + * that the function is capable of accepting null input. The transformed map might contain null + * values, if the function sometimes gives a null result. + * + *

The returned map is not thread-safe or serializable, even if the underlying map is. + * + *

The function is applied lazily, invoked when needed. This is necessary for the returned map + * to be a view, but it means that the function will be applied many times for bulk operations + * like {@link Map#containsValue} and {@code Map.toString()}. For this to perform well, {@code + * function} should be fast. To avoid lazy evaluation when the returned map doesn't need to be a + * view, copy the returned map into a new map of your choosing. + */ + public static Map transformValues( + Map fromMap, Function function) { + return transformEntries(fromMap, asEntryTransformer(function)); + } + + /** + * Returns a view of a sorted map where each value is transformed by a function. All other + * properties of the map, such as iteration order, are left intact. For example, the code: + * + *

{@code
+   * SortedMap map = ImmutableSortedMap.of("a", 4, "b", 9);
+   * Function sqrt =
+   *     new Function() {
+   *       public Double apply(Integer in) {
+   *         return Math.sqrt((int) in);
+   *       }
+   *     };
+   * SortedMap transformed =
+   *      Maps.transformValues(map, sqrt);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {a=2.0, b=3.0}}. + * + *

Changes in the underlying map are reflected in this view. Conversely, this view supports + * removal operations, and these are reflected in the underlying map. + * + *

It's acceptable for the underlying map to contain null keys, and even null values provided + * that the function is capable of accepting null input. The transformed map might contain null + * values, if the function sometimes gives a null result. + * + *

The returned map is not thread-safe or serializable, even if the underlying map is. + * + *

The function is applied lazily, invoked when needed. This is necessary for the returned map + * to be a view, but it means that the function will be applied many times for bulk operations + * like {@link Map#containsValue} and {@code Map.toString()}. For this to perform well, {@code + * function} should be fast. To avoid lazy evaluation when the returned map doesn't need to be a + * view, copy the returned map into a new map of your choosing. + * + * @since 11.0 + */ + public static SortedMap transformValues( + SortedMap fromMap, Function function) { + return transformEntries(fromMap, asEntryTransformer(function)); + } + + /** + * Returns a view of a navigable map where each value is transformed by a function. All other + * properties of the map, such as iteration order, are left intact. For example, the code: + * + *

{@code
+   * NavigableMap map = Maps.newTreeMap();
+   * map.put("a", 4);
+   * map.put("b", 9);
+   * Function sqrt =
+   *     new Function() {
+   *       public Double apply(Integer in) {
+   *         return Math.sqrt((int) in);
+   *       }
+   *     };
+   * NavigableMap transformed =
+   *      Maps.transformNavigableValues(map, sqrt);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {a=2.0, b=3.0}}. + * + *

Changes in the underlying map are reflected in this view. Conversely, this view supports + * removal operations, and these are reflected in the underlying map. + * + *

It's acceptable for the underlying map to contain null keys, and even null values provided + * that the function is capable of accepting null input. The transformed map might contain null + * values, if the function sometimes gives a null result. + * + *

The returned map is not thread-safe or serializable, even if the underlying map is. + * + *

The function is applied lazily, invoked when needed. This is necessary for the returned map + * to be a view, but it means that the function will be applied many times for bulk operations + * like {@link Map#containsValue} and {@code Map.toString()}. For this to perform well, {@code + * function} should be fast. To avoid lazy evaluation when the returned map doesn't need to be a + * view, copy the returned map into a new map of your choosing. + * + * @since 13.0 + */ + public static NavigableMap transformValues( + NavigableMap fromMap, Function function) { + return transformEntries(fromMap, asEntryTransformer(function)); + } + + /** + * Returns a view of a map whose values are derived from the original map's entries. In contrast + * to {@link #transformValues}, this method's entry-transformation logic may depend on the key as + * well as the value. + * + *

All other properties of the transformed map, such as iteration order, are left intact. For + * example, the code: + * + *

{@code
+   * Map options =
+   *     ImmutableMap.of("verbose", true, "sort", false);
+   * EntryTransformer flagPrefixer =
+   *     new EntryTransformer() {
+   *       public String transformEntry(String key, Boolean value) {
+   *         return value ? key : "no" + key;
+   *       }
+   *     };
+   * Map transformed =
+   *     Maps.transformEntries(options, flagPrefixer);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {verbose=verbose, sort=nosort}}. + * + *

Changes in the underlying map are reflected in this view. Conversely, this view supports + * removal operations, and these are reflected in the underlying map. + * + *

It's acceptable for the underlying map to contain null keys and null values provided that + * the transformer is capable of accepting null inputs. The transformed map might contain null + * values if the transformer sometimes gives a null result. + * + *

The returned map is not thread-safe or serializable, even if the underlying map is. + * + *

The transformer is applied lazily, invoked when needed. This is necessary for the returned + * map to be a view, but it means that the transformer will be applied many times for bulk + * operations like {@link Map#containsValue} and {@link Object#toString}. For this to perform + * well, {@code transformer} should be fast. To avoid lazy evaluation when the returned map + * doesn't need to be a view, copy the returned map into a new map of your choosing. + * + *

Warning: This method assumes that for any instance {@code k} of {@code + * EntryTransformer} key type {@code K}, {@code k.equals(k2)} implies that {@code k2} is also of + * type {@code K}. Using an {@code EntryTransformer} key type for which this may not hold, such as + * {@code ArrayList}, may risk a {@code ClassCastException} when calling methods on the + * transformed map. + * + * @since 7.0 + */ + public static Map transformEntries( + Map fromMap, EntryTransformer transformer) { + return new TransformedEntriesMap<>(fromMap, transformer); + } + + /** + * Returns a view of a sorted map whose values are derived from the original sorted map's entries. + * In contrast to {@link #transformValues}, this method's entry-transformation logic may depend on + * the key as well as the value. + * + *

All other properties of the transformed map, such as iteration order, are left intact. For + * example, the code: + * + *

{@code
+   * Map options =
+   *     ImmutableSortedMap.of("verbose", true, "sort", false);
+   * EntryTransformer flagPrefixer =
+   *     new EntryTransformer() {
+   *       public String transformEntry(String key, Boolean value) {
+   *         return value ? key : "yes" + key;
+   *       }
+   *     };
+   * SortedMap transformed =
+   *     Maps.transformEntries(options, flagPrefixer);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {sort=yessort, verbose=verbose}}. + * + *

Changes in the underlying map are reflected in this view. Conversely, this view supports + * removal operations, and these are reflected in the underlying map. + * + *

It's acceptable for the underlying map to contain null keys and null values provided that + * the transformer is capable of accepting null inputs. The transformed map might contain null + * values if the transformer sometimes gives a null result. + * + *

The returned map is not thread-safe or serializable, even if the underlying map is. + * + *

The transformer is applied lazily, invoked when needed. This is necessary for the returned + * map to be a view, but it means that the transformer will be applied many times for bulk + * operations like {@link Map#containsValue} and {@link Object#toString}. For this to perform + * well, {@code transformer} should be fast. To avoid lazy evaluation when the returned map + * doesn't need to be a view, copy the returned map into a new map of your choosing. + * + *

Warning: This method assumes that for any instance {@code k} of {@code + * EntryTransformer} key type {@code K}, {@code k.equals(k2)} implies that {@code k2} is also of + * type {@code K}. Using an {@code EntryTransformer} key type for which this may not hold, such as + * {@code ArrayList}, may risk a {@code ClassCastException} when calling methods on the + * transformed map. + * + * @since 11.0 + */ + public static SortedMap transformEntries( + SortedMap fromMap, EntryTransformer transformer) { + return new TransformedEntriesSortedMap<>(fromMap, transformer); + } + + /** + * Returns a view of a navigable map whose values are derived from the original navigable map's + * entries. In contrast to {@link #transformValues}, this method's entry-transformation logic may + * depend on the key as well as the value. + * + *

All other properties of the transformed map, such as iteration order, are left intact. For + * example, the code: + * + *

{@code
+   * NavigableMap options = Maps.newTreeMap();
+   * options.put("verbose", false);
+   * options.put("sort", true);
+   * EntryTransformer flagPrefixer =
+   *     new EntryTransformer() {
+   *       public String transformEntry(String key, Boolean value) {
+   *         return value ? key : ("yes" + key);
+   *       }
+   *     };
+   * NavigableMap transformed =
+   *     LabsMaps.transformNavigableEntries(options, flagPrefixer);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {sort=yessort, verbose=verbose}}. + * + *

Changes in the underlying map are reflected in this view. Conversely, this view supports + * removal operations, and these are reflected in the underlying map. + * + *

It's acceptable for the underlying map to contain null keys and null values provided that + * the transformer is capable of accepting null inputs. The transformed map might contain null + * values if the transformer sometimes gives a null result. + * + *

The returned map is not thread-safe or serializable, even if the underlying map is. + * + *

The transformer is applied lazily, invoked when needed. This is necessary for the returned + * map to be a view, but it means that the transformer will be applied many times for bulk + * operations like {@link Map#containsValue} and {@link Object#toString}. For this to perform + * well, {@code transformer} should be fast. To avoid lazy evaluation when the returned map + * doesn't need to be a view, copy the returned map into a new map of your choosing. + * + *

Warning: This method assumes that for any instance {@code k} of {@code + * EntryTransformer} key type {@code K}, {@code k.equals(k2)} implies that {@code k2} is also of + * type {@code K}. Using an {@code EntryTransformer} key type for which this may not hold, such as + * {@code ArrayList}, may risk a {@code ClassCastException} when calling methods on the + * transformed map. + * + * @since 13.0 + */ + public static NavigableMap transformEntries( + final NavigableMap fromMap, EntryTransformer transformer) { + return new TransformedEntriesNavigableMap<>(fromMap, transformer); + } + + /** + * A transformation of the value of a key-value pair, using both key and value as inputs. To apply + * the transformation to a map, use {@link Maps#transformEntries(Map, EntryTransformer)}. + * + * @param the key type of the input and output entries + * @param the value type of the input entry + * @param the value type of the output entry + * @since 7.0 + */ + @FunctionalInterface + public interface EntryTransformer { + /** + * Determines an output value based on a key-value pair. This method is generally + * expected, but not absolutely required, to have the following properties: + * + *

    + *
  • Its execution does not cause any observable side effects. + *
  • The computation is consistent with equals; that is, {@link Objects#equal + * Objects.equal}{@code (k1, k2) &&} {@link Objects#equal}{@code (v1, v2)} implies that + * {@code Objects.equal(transformer.transform(k1, v1), transformer.transform(k2, v2))}. + *
+ * + * @throws NullPointerException if the key or value is null and this transformer does not accept + * null arguments + */ + V2 transformEntry(K key, V1 value); + } + + /** Views a function as an entry transformer that ignores the entry key. */ + static EntryTransformer asEntryTransformer( + final Function function) { + checkNotNull(function); + return new EntryTransformer() { + @Override + public V2 transformEntry(K key, V1 value) { + return function.apply(value); + } + }; + } + + static Function asValueToValueFunction( + final EntryTransformer transformer, final K key) { + checkNotNull(transformer); + return new Function() { + @Override + public V2 apply(V1 v1) { + return transformer.transformEntry(key, v1); + } + }; + } + + /** Views an entry transformer as a function from {@code Entry} to values. */ + static Function, V2> asEntryToValueFunction( + final EntryTransformer transformer) { + checkNotNull(transformer); + return new Function, V2>() { + @Override + public V2 apply(Entry entry) { + return transformer.transformEntry(entry.getKey(), entry.getValue()); + } + }; + } + + /** Returns a view of an entry transformed by the specified transformer. */ + static Entry transformEntry( + final EntryTransformer transformer, final Entry entry) { + checkNotNull(transformer); + checkNotNull(entry); + return new AbstractMapEntry() { + @Override + public K getKey() { + return entry.getKey(); + } + + @Override + public V2 getValue() { + return transformer.transformEntry(entry.getKey(), entry.getValue()); + } + }; + } + + /** Views an entry transformer as a function from entries to entries. */ + static Function, Entry> asEntryToEntryFunction( + final EntryTransformer transformer) { + checkNotNull(transformer); + return new Function, Entry>() { + @Override + public Entry apply(final Entry entry) { + return transformEntry(transformer, entry); + } + }; + } + + static class TransformedEntriesMap extends IteratorBasedAbstractMap { + final Map fromMap; + final EntryTransformer transformer; + + TransformedEntriesMap( + Map fromMap, EntryTransformer transformer) { + this.fromMap = checkNotNull(fromMap); + this.transformer = checkNotNull(transformer); + } + + @Override + public int size() { + return fromMap.size(); + } + + @Override + public boolean containsKey(Object key) { + return fromMap.containsKey(key); + } + + @Override + + public V2 get(Object key) { + return getOrDefault(key, null); + } + + // safe as long as the user followed the Warning in the javadoc + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + + public V2 getOrDefault(Object key, V2 defaultValue) { + V1 value = fromMap.get(key); + return (value != null || fromMap.containsKey(key)) + ? transformer.transformEntry((K) key, value) + : defaultValue; + } + + // safe as long as the user followed the Warning in the javadoc + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public V2 remove(Object key) { + return fromMap.containsKey(key) + ? transformer.transformEntry((K) key, fromMap.remove(key)) + : null; + } + + @Override + public void clear() { + fromMap.clear(); + } + + @Override + public Set keySet() { + return fromMap.keySet(); + } + + @Override + Iterator> entryIterator() { + return Iterators.transform( + fromMap.entrySet().iterator(), Maps.asEntryToEntryFunction(transformer)); + } + + @Override + Spliterator> entrySpliterator() { + return CollectSpliterators.map( + fromMap.entrySet().spliterator(), Maps.asEntryToEntryFunction(transformer)); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + // avoids creating new Entry objects + fromMap.forEach((k, v1) -> action.accept(k, transformer.transformEntry(k, v1))); + } + + @Override + public Collection values() { + return new Values<>(this); + } + } + + static class TransformedEntriesSortedMap extends TransformedEntriesMap + implements SortedMap { + + protected SortedMap fromMap() { + return (SortedMap) fromMap; + } + + TransformedEntriesSortedMap( + SortedMap fromMap, EntryTransformer transformer) { + super(fromMap, transformer); + } + + @Override + public Comparator comparator() { + return fromMap().comparator(); + } + + @Override + public K firstKey() { + return fromMap().firstKey(); + } + + @Override + public SortedMap headMap(K toKey) { + return transformEntries(fromMap().headMap(toKey), transformer); + } + + @Override + public K lastKey() { + return fromMap().lastKey(); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return transformEntries(fromMap().subMap(fromKey, toKey), transformer); + } + + @Override + public SortedMap tailMap(K fromKey) { + return transformEntries(fromMap().tailMap(fromKey), transformer); + } + } + + private static class TransformedEntriesNavigableMap + extends TransformedEntriesSortedMap implements NavigableMap { + + TransformedEntriesNavigableMap( + NavigableMap fromMap, EntryTransformer transformer) { + super(fromMap, transformer); + } + + @Override + public Entry ceilingEntry(K key) { + return transformEntry(fromMap().ceilingEntry(key)); + } + + @Override + public K ceilingKey(K key) { + return fromMap().ceilingKey(key); + } + + @Override + public NavigableSet descendingKeySet() { + return fromMap().descendingKeySet(); + } + + @Override + public NavigableMap descendingMap() { + return transformEntries(fromMap().descendingMap(), transformer); + } + + @Override + public Entry firstEntry() { + return transformEntry(fromMap().firstEntry()); + } + + @Override + public Entry floorEntry(K key) { + return transformEntry(fromMap().floorEntry(key)); + } + + @Override + public K floorKey(K key) { + return fromMap().floorKey(key); + } + + @Override + public NavigableMap headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + return transformEntries(fromMap().headMap(toKey, inclusive), transformer); + } + + @Override + public Entry higherEntry(K key) { + return transformEntry(fromMap().higherEntry(key)); + } + + @Override + public K higherKey(K key) { + return fromMap().higherKey(key); + } + + @Override + public Entry lastEntry() { + return transformEntry(fromMap().lastEntry()); + } + + @Override + public Entry lowerEntry(K key) { + return transformEntry(fromMap().lowerEntry(key)); + } + + @Override + public K lowerKey(K key) { + return fromMap().lowerKey(key); + } + + @Override + public NavigableSet navigableKeySet() { + return fromMap().navigableKeySet(); + } + + @Override + public Entry pollFirstEntry() { + return transformEntry(fromMap().pollFirstEntry()); + } + + @Override + public Entry pollLastEntry() { + return transformEntry(fromMap().pollLastEntry()); + } + + @Override + public NavigableMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return transformEntries( + fromMap().subMap(fromKey, fromInclusive, toKey, toInclusive), transformer); + } + + @Override + public NavigableMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public NavigableMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + return transformEntries(fromMap().tailMap(fromKey, inclusive), transformer); + } + + + private Entry transformEntry(Entry entry) { + return (entry == null) ? null : Maps.transformEntry(transformer, entry); + } + + @Override + protected NavigableMap fromMap() { + return (NavigableMap) super.fromMap(); + } + } + + static Predicate> keyPredicateOnEntries(Predicate keyPredicate) { + return compose(keyPredicate, Maps.keyFunction()); + } + + static Predicate> valuePredicateOnEntries(Predicate valuePredicate) { + return compose(valuePredicate, Maps.valueFunction()); + } + + /** + * Returns a map containing the mappings in {@code unfiltered} whose keys satisfy a predicate. The + * returned map is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a key that doesn't satisfy the predicate, the map's {@code put()} and + * {@code putAll()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings whose keys satisfy the filter will be removed from the underlying + * map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code keyPredicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + */ + public static Map filterKeys( + Map unfiltered, final Predicate keyPredicate) { + checkNotNull(keyPredicate); + Predicate> entryPredicate = keyPredicateOnEntries(keyPredicate); + return (unfiltered instanceof AbstractFilteredMap) + ? filterFiltered((AbstractFilteredMap) unfiltered, entryPredicate) + : new FilteredKeyMap(checkNotNull(unfiltered), keyPredicate, entryPredicate); + } + + /** + * Returns a sorted map containing the mappings in {@code unfiltered} whose keys satisfy a + * predicate. The returned map is a live view of {@code unfiltered}; changes to one affect the + * other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a key that doesn't satisfy the predicate, the map's {@code put()} and + * {@code putAll()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings whose keys satisfy the filter will be removed from the underlying + * map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code keyPredicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 11.0 + */ + public static SortedMap filterKeys( + SortedMap unfiltered, final Predicate keyPredicate) { + // TODO(lowasser): Return a subclass of Maps.FilteredKeyMap for slightly better + // performance. + return filterEntries(unfiltered, Maps.keyPredicateOnEntries(keyPredicate)); + } + + /** + * Returns a navigable map containing the mappings in {@code unfiltered} whose keys satisfy a + * predicate. The returned map is a live view of {@code unfiltered}; changes to one affect the + * other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a key that doesn't satisfy the predicate, the map's {@code put()} and + * {@code putAll()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings whose keys satisfy the filter will be removed from the underlying + * map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code keyPredicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 14.0 + */ + public static NavigableMap filterKeys( + NavigableMap unfiltered, final Predicate keyPredicate) { + // TODO(lowasser): Return a subclass of Maps.FilteredKeyMap for slightly better + // performance. + return filterEntries(unfiltered, Maps.keyPredicateOnEntries(keyPredicate)); + } + + /** + * Returns a bimap containing the mappings in {@code unfiltered} whose keys satisfy a predicate. + * The returned bimap is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting bimap's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the bimap + * and its views. When given a key that doesn't satisfy the predicate, the bimap's {@code put()}, + * {@code forcePut()} and {@code putAll()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * bimap or its views, only mappings that satisfy the filter will be removed from the underlying + * bimap. + * + *

The returned bimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered bimap's methods, such as {@code size()}, iterate across every key in + * the underlying bimap and determine which satisfy the filter. When a live view is not + * needed, it may be faster to copy the filtered bimap and use the copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals , as documented + * at {@link Predicate#apply}. + * + * @since 14.0 + */ + public static BiMap filterKeys( + BiMap unfiltered, final Predicate keyPredicate) { + checkNotNull(keyPredicate); + return filterEntries(unfiltered, Maps.keyPredicateOnEntries(keyPredicate)); + } + + /** + * Returns a map containing the mappings in {@code unfiltered} whose values satisfy a predicate. + * The returned map is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a value that doesn't satisfy the predicate, the map's {@code put()}, + * {@code putAll()}, and {@link Entry#setValue} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings whose values satisfy the filter will be removed from the underlying + * map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code valuePredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + */ + public static Map filterValues( + Map unfiltered, final Predicate valuePredicate) { + return filterEntries(unfiltered, Maps.valuePredicateOnEntries(valuePredicate)); + } + + /** + * Returns a sorted map containing the mappings in {@code unfiltered} whose values satisfy a + * predicate. The returned map is a live view of {@code unfiltered}; changes to one affect the + * other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a value that doesn't satisfy the predicate, the map's {@code put()}, + * {@code putAll()}, and {@link Entry#setValue} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings whose values satisfy the filter will be removed from the underlying + * map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code valuePredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 11.0 + */ + public static SortedMap filterValues( + SortedMap unfiltered, final Predicate valuePredicate) { + return filterEntries(unfiltered, Maps.valuePredicateOnEntries(valuePredicate)); + } + + /** + * Returns a navigable map containing the mappings in {@code unfiltered} whose values satisfy a + * predicate. The returned map is a live view of {@code unfiltered}; changes to one affect the + * other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a value that doesn't satisfy the predicate, the map's {@code put()}, + * {@code putAll()}, and {@link Entry#setValue} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings whose values satisfy the filter will be removed from the underlying + * map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code valuePredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 14.0 + */ + public static NavigableMap filterValues( + NavigableMap unfiltered, final Predicate valuePredicate) { + return filterEntries(unfiltered, Maps.valuePredicateOnEntries(valuePredicate)); + } + + /** + * Returns a bimap containing the mappings in {@code unfiltered} whose values satisfy a predicate. + * The returned bimap is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting bimap's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the bimap + * and its views. When given a value that doesn't satisfy the predicate, the bimap's {@code + * put()}, {@code forcePut()} and {@code putAll()} methods throw an {@link + * IllegalArgumentException}. Similarly, the map's entries have a {@link Entry#setValue} method + * that throws an {@link IllegalArgumentException} when the provided value doesn't satisfy the + * predicate. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * bimap or its views, only mappings that satisfy the filter will be removed from the underlying + * bimap. + * + *

The returned bimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered bimap's methods, such as {@code size()}, iterate across every value in + * the underlying bimap and determine which satisfy the filter. When a live view is not + * needed, it may be faster to copy the filtered bimap and use the copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals , as documented + * at {@link Predicate#apply}. + * + * @since 14.0 + */ + public static BiMap filterValues( + BiMap unfiltered, final Predicate valuePredicate) { + return filterEntries(unfiltered, Maps.valuePredicateOnEntries(valuePredicate)); + } + + /** + * Returns a map containing the mappings in {@code unfiltered} that satisfy a predicate. The + * returned map is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a key/value pair that doesn't satisfy the predicate, the map's {@code + * put()} and {@code putAll()} methods throw an {@link IllegalArgumentException}. Similarly, the + * map's entries have a {@link Entry#setValue} method that throws an {@link + * IllegalArgumentException} when the existing key and the provided value don't satisfy the + * predicate. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings that satisfy the filter will be removed from the underlying map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. + */ + public static Map filterEntries( + Map unfiltered, Predicate> entryPredicate) { + checkNotNull(entryPredicate); + return (unfiltered instanceof AbstractFilteredMap) + ? filterFiltered((AbstractFilteredMap) unfiltered, entryPredicate) + : new FilteredEntryMap(checkNotNull(unfiltered), entryPredicate); + } + + /** + * Returns a sorted map containing the mappings in {@code unfiltered} that satisfy a predicate. + * The returned map is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a key/value pair that doesn't satisfy the predicate, the map's {@code + * put()} and {@code putAll()} methods throw an {@link IllegalArgumentException}. Similarly, the + * map's entries have a {@link Entry#setValue} method that throws an {@link + * IllegalArgumentException} when the existing key and the provided value don't satisfy the + * predicate. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings that satisfy the filter will be removed from the underlying map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. + * + * @since 11.0 + */ + public static SortedMap filterEntries( + SortedMap unfiltered, Predicate> entryPredicate) { + checkNotNull(entryPredicate); + return (unfiltered instanceof FilteredEntrySortedMap) + ? filterFiltered((FilteredEntrySortedMap) unfiltered, entryPredicate) + : new FilteredEntrySortedMap(checkNotNull(unfiltered), entryPredicate); + } + + /** + * Returns a sorted map containing the mappings in {@code unfiltered} that satisfy a predicate. + * The returned map is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting map's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the map + * and its views. When given a key/value pair that doesn't satisfy the predicate, the map's {@code + * put()} and {@code putAll()} methods throw an {@link IllegalArgumentException}. Similarly, the + * map's entries have a {@link Entry#setValue} method that throws an {@link + * IllegalArgumentException} when the existing key and the provided value don't satisfy the + * predicate. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered map + * or its views, only mappings that satisfy the filter will be removed from the underlying map. + * + *

The returned map isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered map's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying map and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered map and use the copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. + * + * @since 14.0 + */ + public static NavigableMap filterEntries( + NavigableMap unfiltered, Predicate> entryPredicate) { + checkNotNull(entryPredicate); + return (unfiltered instanceof FilteredEntryNavigableMap) + ? filterFiltered((FilteredEntryNavigableMap) unfiltered, entryPredicate) + : new FilteredEntryNavigableMap(checkNotNull(unfiltered), entryPredicate); + } + + /** + * Returns a bimap containing the mappings in {@code unfiltered} that satisfy a predicate. The + * returned bimap is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting bimap's {@code keySet()}, {@code entrySet()}, and {@code values()} views have + * iterators that don't support {@code remove()}, but all other methods are supported by the bimap + * and its views. When given a key/value pair that doesn't satisfy the predicate, the bimap's + * {@code put()}, {@code forcePut()} and {@code putAll()} methods throw an {@link + * IllegalArgumentException}. Similarly, the map's entries have an {@link Entry#setValue} method + * that throws an {@link IllegalArgumentException} when the existing key and the provided value + * don't satisfy the predicate. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * bimap or its views, only mappings that satisfy the filter will be removed from the underlying + * bimap. + * + *

The returned bimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered bimap's methods, such as {@code size()}, iterate across every key/value + * mapping in the underlying bimap and determine which satisfy the filter. When a live view is + * not needed, it may be faster to copy the filtered bimap and use the copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals , as documented + * at {@link Predicate#apply}. + * + * @since 14.0 + */ + public static BiMap filterEntries( + BiMap unfiltered, Predicate> entryPredicate) { + checkNotNull(unfiltered); + checkNotNull(entryPredicate); + return (unfiltered instanceof FilteredEntryBiMap) + ? filterFiltered((FilteredEntryBiMap) unfiltered, entryPredicate) + : new FilteredEntryBiMap(unfiltered, entryPredicate); + } + + /** + * Support {@code clear()}, {@code removeAll()}, and {@code retainAll()} when filtering a filtered + * map. + */ + private static Map filterFiltered( + AbstractFilteredMap map, Predicate> entryPredicate) { + return new FilteredEntryMap<>( + map.unfiltered, Predicates.>and(map.predicate, entryPredicate)); + } + + /** + * Support {@code clear()}, {@code removeAll()}, and {@code retainAll()} when filtering a filtered + * sorted map. + */ + private static SortedMap filterFiltered( + FilteredEntrySortedMap map, Predicate> entryPredicate) { + Predicate> predicate = Predicates.>and(map.predicate, entryPredicate); + return new FilteredEntrySortedMap<>(map.sortedMap(), predicate); + } + + /** + * Support {@code clear()}, {@code removeAll()}, and {@code retainAll()} when filtering a filtered + * navigable map. + */ + private static NavigableMap filterFiltered( + FilteredEntryNavigableMap map, Predicate> entryPredicate) { + Predicate> predicate = + Predicates.>and(map.entryPredicate, entryPredicate); + return new FilteredEntryNavigableMap<>(map.unfiltered, predicate); + } + + /** + * Support {@code clear()}, {@code removeAll()}, and {@code retainAll()} when filtering a filtered + * map. + */ + private static BiMap filterFiltered( + FilteredEntryBiMap map, Predicate> entryPredicate) { + Predicate> predicate = Predicates.>and(map.predicate, entryPredicate); + return new FilteredEntryBiMap<>(map.unfiltered(), predicate); + } + + private abstract static class AbstractFilteredMap extends ViewCachingAbstractMap { + final Map unfiltered; + final Predicate> predicate; + + AbstractFilteredMap(Map unfiltered, Predicate> predicate) { + this.unfiltered = unfiltered; + this.predicate = predicate; + } + + boolean apply(Object key, V value) { + // This method is called only when the key is in the map, implying that + // key is a K. + @SuppressWarnings({"unchecked", "rawtypes"}) + K k = (K) key; + return predicate.apply(Maps.immutableEntry(k, value)); + } + + @Override + public V put(K key, V value) { + checkArgument(apply(key, value)); + return unfiltered.put(key, value); + } + + @Override + public void putAll(Map map) { + for (Entry entry : map.entrySet()) { + checkArgument(apply(entry.getKey(), entry.getValue())); + } + unfiltered.putAll(map); + } + + @Override + public boolean containsKey(Object key) { + return unfiltered.containsKey(key) && apply(key, unfiltered.get(key)); + } + + @Override + public V get(Object key) { + V value = unfiltered.get(key); + return ((value != null) && apply(key, value)) ? value : null; + } + + @Override + public boolean isEmpty() { + return entrySet().isEmpty(); + } + + @Override + public V remove(Object key) { + return containsKey(key) ? unfiltered.remove(key) : null; + } + + @Override + Collection createValues() { + return new FilteredMapValues<>(this, unfiltered, predicate); + } + } + + private static final class FilteredMapValues extends Values { + final Map unfiltered; + final Predicate> predicate; + + FilteredMapValues( + Map filteredMap, Map unfiltered, Predicate> predicate) { + super(filteredMap); + this.unfiltered = unfiltered; + this.predicate = predicate; + } + + @Override + public boolean remove(Object o) { + Iterator> entryItr = unfiltered.entrySet().iterator(); + while (entryItr.hasNext()) { + Entry entry = entryItr.next(); + if (predicate.apply(entry) && Objects.equal(entry.getValue(), o)) { + entryItr.remove(); + return true; + } + } + return false; + } + + @Override + public boolean removeAll(Collection collection) { + Iterator> entryItr = unfiltered.entrySet().iterator(); + boolean result = false; + while (entryItr.hasNext()) { + Entry entry = entryItr.next(); + if (predicate.apply(entry) && collection.contains(entry.getValue())) { + entryItr.remove(); + result = true; + } + } + return result; + } + + @Override + public boolean retainAll(Collection collection) { + Iterator> entryItr = unfiltered.entrySet().iterator(); + boolean result = false; + while (entryItr.hasNext()) { + Entry entry = entryItr.next(); + if (predicate.apply(entry) && !collection.contains(entry.getValue())) { + entryItr.remove(); + result = true; + } + } + return result; + } + + @Override + public Object[] toArray() { + // creating an ArrayList so filtering happens once + return Lists.newArrayList(iterator()).toArray(); + } + + @Override + public T[] toArray(T[] array) { + return Lists.newArrayList(iterator()).toArray(array); + } + } + + private static class FilteredKeyMap extends AbstractFilteredMap { + final Predicate keyPredicate; + + FilteredKeyMap( + Map unfiltered, + Predicate keyPredicate, + Predicate> entryPredicate) { + super(unfiltered, entryPredicate); + this.keyPredicate = keyPredicate; + } + + @Override + protected Set> createEntrySet() { + return Sets.filter(unfiltered.entrySet(), predicate); + } + + @Override + Set createKeySet() { + return Sets.filter(unfiltered.keySet(), keyPredicate); + } + + // The cast is called only when the key is in the unfiltered map, implying + // that key is a K. + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public boolean containsKey(Object key) { + return unfiltered.containsKey(key) && keyPredicate.apply((K) key); + } + } + + static class FilteredEntryMap extends AbstractFilteredMap { + /** + * Entries in this set satisfy the predicate, but they don't validate the input to {@code + * Entry.setValue()}. + */ + final Set> filteredEntrySet; + + FilteredEntryMap(Map unfiltered, Predicate> entryPredicate) { + super(unfiltered, entryPredicate); + filteredEntrySet = Sets.filter(unfiltered.entrySet(), predicate); + } + + @Override + protected Set> createEntrySet() { + return new EntrySet(); + } + + private class EntrySet extends ForwardingSet> { + @Override + protected Set> delegate() { + return filteredEntrySet; + } + + @Override + public Iterator> iterator() { + return new TransformedIterator, Entry>(filteredEntrySet.iterator()) { + @Override + Entry transform(final Entry entry) { + return new ForwardingMapEntry() { + @Override + protected Entry delegate() { + return entry; + } + + @Override + public V setValue(V newValue) { + checkArgument(apply(getKey(), newValue)); + return super.setValue(newValue); + } + }; + } + }; + } + } + + @Override + Set createKeySet() { + return new KeySet(); + } + + static boolean removeAllKeys( + Map map, Predicate> entryPredicate, Collection keyCollection) { + Iterator> entryItr = map.entrySet().iterator(); + boolean result = false; + while (entryItr.hasNext()) { + Entry entry = entryItr.next(); + if (entryPredicate.apply(entry) && keyCollection.contains(entry.getKey())) { + entryItr.remove(); + result = true; + } + } + return result; + } + + static boolean retainAllKeys( + Map map, Predicate> entryPredicate, Collection keyCollection) { + Iterator> entryItr = map.entrySet().iterator(); + boolean result = false; + while (entryItr.hasNext()) { + Entry entry = entryItr.next(); + if (entryPredicate.apply(entry) && !keyCollection.contains(entry.getKey())) { + entryItr.remove(); + result = true; + } + } + return result; + } + + class KeySet extends Maps.KeySet { + KeySet() { + super(FilteredEntryMap.this); + } + + @Override + public boolean remove(Object o) { + if (containsKey(o)) { + unfiltered.remove(o); + return true; + } + return false; + } + + @Override + public boolean removeAll(Collection collection) { + return removeAllKeys(unfiltered, predicate, collection); + } + + @Override + public boolean retainAll(Collection collection) { + return retainAllKeys(unfiltered, predicate, collection); + } + + @Override + public Object[] toArray() { + // creating an ArrayList so filtering happens once + return Lists.newArrayList(iterator()).toArray(); + } + + @Override + public T[] toArray(T[] array) { + return Lists.newArrayList(iterator()).toArray(array); + } + } + } + + private static class FilteredEntrySortedMap extends FilteredEntryMap + implements SortedMap { + + FilteredEntrySortedMap( + SortedMap unfiltered, Predicate> entryPredicate) { + super(unfiltered, entryPredicate); + } + + SortedMap sortedMap() { + return (SortedMap) unfiltered; + } + + @Override + public SortedSet keySet() { + return (SortedSet) super.keySet(); + } + + @Override + SortedSet createKeySet() { + return new SortedKeySet(); + } + + class SortedKeySet extends KeySet implements SortedSet { + @Override + public Comparator comparator() { + return sortedMap().comparator(); + } + + @Override + public SortedSet subSet(K fromElement, K toElement) { + return (SortedSet) subMap(fromElement, toElement).keySet(); + } + + @Override + public SortedSet headSet(K toElement) { + return (SortedSet) headMap(toElement).keySet(); + } + + @Override + public SortedSet tailSet(K fromElement) { + return (SortedSet) tailMap(fromElement).keySet(); + } + + @Override + public K first() { + return firstKey(); + } + + @Override + public K last() { + return lastKey(); + } + } + + @Override + public Comparator comparator() { + return sortedMap().comparator(); + } + + @Override + public K firstKey() { + // correctly throws NoSuchElementException when filtered map is empty. + return keySet().iterator().next(); + } + + @Override + public K lastKey() { + SortedMap headMap = sortedMap(); + while (true) { + // correctly throws NoSuchElementException when filtered map is empty. + K key = headMap.lastKey(); + if (apply(key, unfiltered.get(key))) { + return key; + } + headMap = sortedMap().headMap(key); + } + } + + @Override + public SortedMap headMap(K toKey) { + return new FilteredEntrySortedMap<>(sortedMap().headMap(toKey), predicate); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return new FilteredEntrySortedMap<>(sortedMap().subMap(fromKey, toKey), predicate); + } + + @Override + public SortedMap tailMap(K fromKey) { + return new FilteredEntrySortedMap<>(sortedMap().tailMap(fromKey), predicate); + } + } + + private static class FilteredEntryNavigableMap extends AbstractNavigableMap { + /* + * It's less code to extend AbstractNavigableMap and forward the filtering logic to + * FilteredEntryMap than to extend FilteredEntrySortedMap and reimplement all the NavigableMap + * methods. + */ + + private final NavigableMap unfiltered; + private final Predicate> entryPredicate; + private final Map filteredDelegate; + + FilteredEntryNavigableMap( + NavigableMap unfiltered, Predicate> entryPredicate) { + this.unfiltered = checkNotNull(unfiltered); + this.entryPredicate = entryPredicate; + this.filteredDelegate = new FilteredEntryMap<>(unfiltered, entryPredicate); + } + + @Override + public Comparator comparator() { + return unfiltered.comparator(); + } + + @Override + public NavigableSet navigableKeySet() { + return new NavigableKeySet(this) { + @Override + public boolean removeAll(Collection collection) { + return FilteredEntryMap.removeAllKeys(unfiltered, entryPredicate, collection); + } + + @Override + public boolean retainAll(Collection collection) { + return FilteredEntryMap.retainAllKeys(unfiltered, entryPredicate, collection); + } + }; + } + + @Override + public Collection values() { + return new FilteredMapValues<>(this, unfiltered, entryPredicate); + } + + @Override + Iterator> entryIterator() { + return Iterators.filter(unfiltered.entrySet().iterator(), entryPredicate); + } + + @Override + Iterator> descendingEntryIterator() { + return Iterators.filter(unfiltered.descendingMap().entrySet().iterator(), entryPredicate); + } + + @Override + public int size() { + return filteredDelegate.size(); + } + + @Override + public boolean isEmpty() { + return !Iterables.any(unfiltered.entrySet(), entryPredicate); + } + + @Override + + public V get(Object key) { + return filteredDelegate.get(key); + } + + @Override + public boolean containsKey(Object key) { + return filteredDelegate.containsKey(key); + } + + @Override + public V put(K key, V value) { + return filteredDelegate.put(key, value); + } + + @Override + public V remove(Object key) { + return filteredDelegate.remove(key); + } + + @Override + public void putAll(Map m) { + filteredDelegate.putAll(m); + } + + @Override + public void clear() { + filteredDelegate.clear(); + } + + @Override + public Set> entrySet() { + return filteredDelegate.entrySet(); + } + + @Override + public Entry pollFirstEntry() { + return Iterables.removeFirstMatching(unfiltered.entrySet(), entryPredicate); + } + + @Override + public Entry pollLastEntry() { + return Iterables.removeFirstMatching(unfiltered.descendingMap().entrySet(), entryPredicate); + } + + @Override + public NavigableMap descendingMap() { + return filterEntries(unfiltered.descendingMap(), entryPredicate); + } + + @Override + public NavigableMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return filterEntries( + unfiltered.subMap(fromKey, fromInclusive, toKey, toInclusive), entryPredicate); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + return filterEntries(unfiltered.headMap(toKey, inclusive), entryPredicate); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + return filterEntries(unfiltered.tailMap(fromKey, inclusive), entryPredicate); + } + } + + static final class FilteredEntryBiMap extends FilteredEntryMap + implements BiMap { + private final BiMap inverse; + + private static Predicate> inversePredicate( + final Predicate> forwardPredicate) { + return new Predicate>() { + @Override + public boolean apply(Entry input) { + return forwardPredicate.apply(Maps.immutableEntry(input.getValue(), input.getKey())); + } + }; + } + + FilteredEntryBiMap(BiMap delegate, Predicate> predicate) { + super(delegate, predicate); + this.inverse = + new FilteredEntryBiMap<>(delegate.inverse(), inversePredicate(predicate), this); + } + + private FilteredEntryBiMap( + BiMap delegate, Predicate> predicate, BiMap inverse) { + super(delegate, predicate); + this.inverse = inverse; + } + + BiMap unfiltered() { + return (BiMap) unfiltered; + } + + @Override + public V forcePut(K key, V value) { + checkArgument(apply(key, value)); + return unfiltered().forcePut(key, value); + } + + @Override + public void replaceAll(BiFunction function) { + unfiltered() + .replaceAll( + (key, value) -> + predicate.apply(Maps.immutableEntry(key, value)) + ? function.apply(key, value) + : value); + } + + @Override + public BiMap inverse() { + return inverse; + } + + @Override + public Set values() { + return inverse.keySet(); + } + } + + /** + * Returns an unmodifiable view of the specified navigable map. Query operations on the returned + * map read through to the specified map, and attempts to modify the returned map, whether direct + * or via its views, result in an {@code UnsupportedOperationException}. + * + *

The returned navigable map will be serializable if the specified navigable map is + * serializable. + * + *

This method's signature will not permit you to convert a {@code NavigableMap} to a {@code NavigableMap}. If it permitted this, the returned map's {@code + * comparator()} method might return a {@code Comparator}, which works only on a + * particular subtype of {@code K}, but promise that it's a {@code Comparator}, which + * must work on any type of {@code K}. + * + * @param map the navigable map for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified navigable map + * @since 12.0 + */ + public static NavigableMap unmodifiableNavigableMap( + NavigableMap map) { + checkNotNull(map); + if (map instanceof UnmodifiableNavigableMap) { + @SuppressWarnings({"unchecked", "rawtypes"}) // covariant + NavigableMap result = (NavigableMap) map; + return result; + } else { + return new UnmodifiableNavigableMap<>(map); + } + } + + + private static Entry unmodifiableOrNull(Entry entry) { + return (entry == null) ? null : Maps.unmodifiableEntry(entry); + } + + static class UnmodifiableNavigableMap extends ForwardingSortedMap + implements NavigableMap, Serializable { + private final NavigableMap delegate; + + UnmodifiableNavigableMap(NavigableMap delegate) { + this.delegate = delegate; + } + + UnmodifiableNavigableMap( + NavigableMap delegate, UnmodifiableNavigableMap descendingMap) { + this.delegate = delegate; + this.descendingMap = descendingMap; + } + + @Override + protected SortedMap delegate() { + return Collections.unmodifiableSortedMap(delegate); + } + + @Override + public Entry lowerEntry(K key) { + return unmodifiableOrNull(delegate.lowerEntry(key)); + } + + @Override + public K lowerKey(K key) { + return delegate.lowerKey(key); + } + + @Override + public Entry floorEntry(K key) { + return unmodifiableOrNull(delegate.floorEntry(key)); + } + + @Override + public K floorKey(K key) { + return delegate.floorKey(key); + } + + @Override + public Entry ceilingEntry(K key) { + return unmodifiableOrNull(delegate.ceilingEntry(key)); + } + + @Override + public K ceilingKey(K key) { + return delegate.ceilingKey(key); + } + + @Override + public Entry higherEntry(K key) { + return unmodifiableOrNull(delegate.higherEntry(key)); + } + + @Override + public K higherKey(K key) { + return delegate.higherKey(key); + } + + @Override + public Entry firstEntry() { + return unmodifiableOrNull(delegate.firstEntry()); + } + + @Override + public Entry lastEntry() { + return unmodifiableOrNull(delegate.lastEntry()); + } + + @Override + public final Entry pollFirstEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public final Entry pollLastEntry() { + throw new UnsupportedOperationException(); + } + + private transient UnmodifiableNavigableMap descendingMap; + + @Override + public NavigableMap descendingMap() { + UnmodifiableNavigableMap result = descendingMap; + return (result == null) + ? descendingMap = new UnmodifiableNavigableMap<>(delegate.descendingMap(), this) + : result; + } + + @Override + public Set keySet() { + return navigableKeySet(); + } + + @Override + public NavigableSet navigableKeySet() { + return Sets.unmodifiableNavigableSet(delegate.navigableKeySet()); + } + + @Override + public NavigableSet descendingKeySet() { + return Sets.unmodifiableNavigableSet(delegate.descendingKeySet()); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public NavigableMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return Maps.unmodifiableNavigableMap( + delegate.subMap(fromKey, fromInclusive, toKey, toInclusive)); + } + + @Override + public SortedMap headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + return Maps.unmodifiableNavigableMap(delegate.headMap(toKey, inclusive)); + } + + @Override + public SortedMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + return Maps.unmodifiableNavigableMap(delegate.tailMap(fromKey, inclusive)); + } + } + + /** + * Returns a synchronized (thread-safe) navigable map backed by the specified navigable map. In + * order to guarantee serial access, it is critical that all access to the backing + * navigable map is accomplished through the returned navigable map (or its views). + * + *

It is imperative that the user manually synchronize on the returned navigable map when + * iterating over any of its collection views, or the collections views of any of its {@code + * descendingMap}, {@code subMap}, {@code headMap} or {@code tailMap} views. + * + *

{@code
+   * NavigableMap map = synchronizedNavigableMap(new TreeMap());
+   *
+   * // Needn't be in synchronized block
+   * NavigableSet set = map.navigableKeySet();
+   *
+   * synchronized (map) { // Synchronizing on map, not set!
+   *   Iterator it = set.iterator(); // Must be in synchronized block
+   *   while (it.hasNext()) {
+   *     foo(it.next());
+   *   }
+   * }
+   * }
+ * + *

or: + * + *

{@code
+   * NavigableMap map = synchronizedNavigableMap(new TreeMap());
+   * NavigableMap map2 = map.subMap(foo, false, bar, true);
+   *
+   * // Needn't be in synchronized block
+   * NavigableSet set2 = map2.descendingKeySet();
+   *
+   * synchronized (map) { // Synchronizing on map, not map2 or set2!
+   *   Iterator it = set2.iterator(); // Must be in synchronized block
+   *   while (it.hasNext()) {
+   *     foo(it.next());
+   *   }
+   * }
+   * }
+ * + *

Failure to follow this advice may result in non-deterministic behavior. + * + *

The returned navigable map will be serializable if the specified navigable map is + * serializable. + * + * @param navigableMap the navigable map to be "wrapped" in a synchronized navigable map. + * @return a synchronized view of the specified navigable map. + * @since 13.0 + */ + public static NavigableMap synchronizedNavigableMap( + NavigableMap navigableMap) { + return Synchronized.navigableMap(navigableMap); + } + + /** + * {@code AbstractMap} extension that makes it easy to cache customized keySet, values, and + * entrySet views. + */ + abstract static class ViewCachingAbstractMap extends AbstractMap { + /** + * Creates the entry set to be returned by {@link #entrySet()}. This method is invoked at most + * once on a given map, at the time when {@code entrySet} is first called. + */ + abstract Set> createEntrySet(); + + private transient Set> entrySet; + + @Override + public Set> entrySet() { + Set> result = entrySet; + return (result == null) ? entrySet = createEntrySet() : result; + } + + private transient Set keySet; + + @Override + public Set keySet() { + Set result = keySet; + return (result == null) ? keySet = createKeySet() : result; + } + + Set createKeySet() { + return new KeySet<>(this); + } + + private transient Collection values; + + @Override + public Collection values() { + Collection result = values; + return (result == null) ? values = createValues() : result; + } + + Collection createValues() { + return new Values<>(this); + } + } + + abstract static class IteratorBasedAbstractMap extends AbstractMap { + @Override + public abstract int size(); + + abstract Iterator> entryIterator(); + + Spliterator> entrySpliterator() { + return Spliterators.spliterator( + entryIterator(), size(), Spliterator.SIZED | Spliterator.DISTINCT); + } + + @Override + public Set> entrySet() { + return new EntrySet() { + @Override + Map map() { + return IteratorBasedAbstractMap.this; + } + + @Override + public Iterator> iterator() { + return entryIterator(); + } + + @Override + public Spliterator> spliterator() { + return entrySpliterator(); + } + + @Override + public void forEach(Consumer> action) { + forEachEntry(action); + } + }; + } + + void forEachEntry(Consumer> action) { + entryIterator().forEachRemaining(action); + } + + @Override + public void clear() { + Iterators.clear(entryIterator()); + } + } + + /** + * Delegates to {@link Map#get}. Returns {@code null} on {@code ClassCastException} and {@code + * NullPointerException}. + */ + static V safeGet(Map map, Object key) { + checkNotNull(map); + try { + return map.get(key); + } catch (ClassCastException | NullPointerException e) { + return null; + } + } + + /** + * Delegates to {@link Map#containsKey}. Returns {@code false} on {@code ClassCastException} and + * {@code NullPointerException}. + */ + static boolean safeContainsKey(Map map, Object key) { + checkNotNull(map); + try { + return map.containsKey(key); + } catch (ClassCastException | NullPointerException e) { + return false; + } + } + + /** + * Delegates to {@link Map#remove}. Returns {@code null} on {@code ClassCastException} and {@code + * NullPointerException}. + */ + static V safeRemove(Map map, Object key) { + checkNotNull(map); + try { + return map.remove(key); + } catch (ClassCastException | NullPointerException e) { + return null; + } + } + + /** An admittedly inefficient implementation of {@link Map#containsKey}. */ + static boolean containsKeyImpl(Map map, Object key) { + return Iterators.contains(keyIterator(map.entrySet().iterator()), key); + } + + /** An implementation of {@link Map#containsValue}. */ + static boolean containsValueImpl(Map map, Object value) { + return Iterators.contains(valueIterator(map.entrySet().iterator()), value); + } + + /** + * Implements {@code Collection.contains} safely for forwarding collections of map entries. If + * {@code o} is an instance of {@code Entry}, it is wrapped using {@link #unmodifiableEntry} to + * protect against a possible nefarious equals method. + * + *

Note that {@code c} is the backing (delegate) collection, rather than the forwarding + * collection. + * + * @param c the delegate (unwrapped) collection of map entries + * @param o the object that might be contained in {@code c} + * @return {@code true} if {@code c} contains {@code o} + */ + static boolean containsEntryImpl(Collection> c, Object o) { + if (!(o instanceof Entry)) { + return false; + } + return c.contains(unmodifiableEntry((Entry) o)); + } + + /** + * Implements {@code Collection.remove} safely for forwarding collections of map entries. If + * {@code o} is an instance of {@code Entry}, it is wrapped using {@link #unmodifiableEntry} to + * protect against a possible nefarious equals method. + * + *

Note that {@code c} is backing (delegate) collection, rather than the forwarding collection. + * + * @param c the delegate (unwrapped) collection of map entries + * @param o the object to remove from {@code c} + * @return {@code true} if {@code c} was changed + */ + static boolean removeEntryImpl(Collection> c, Object o) { + if (!(o instanceof Entry)) { + return false; + } + return c.remove(unmodifiableEntry((Entry) o)); + } + + /** An implementation of {@link Map#equals}. */ + static boolean equalsImpl(Map map, Object object) { + if (map == object) { + return true; + } else if (object instanceof Map) { + Map o = (Map) object; + return map.entrySet().equals(o.entrySet()); + } + return false; + } + + /** An implementation of {@link Map#toString}. */ + static String toStringImpl(Map map) { + StringBuilder sb = Collections2.newStringBuilderForCollection(map.size()).append('{'); + boolean first = true; + for (Entry entry : map.entrySet()) { + if (!first) { + sb.append(", "); + } + first = false; + sb.append(entry.getKey()).append('=').append(entry.getValue()); + } + return sb.append('}').toString(); + } + + /** An implementation of {@link Map#putAll}. */ + static void putAllImpl(Map self, Map map) { + for (Entry entry : map.entrySet()) { + self.put(entry.getKey(), entry.getValue()); + } + } + + static class KeySet extends Sets.ImprovedAbstractSet { + final Map map; + + KeySet(Map map) { + this.map = checkNotNull(map); + } + + Map map() { + return map; + } + + @Override + public Iterator iterator() { + return keyIterator(map().entrySet().iterator()); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + // avoids entry allocation for those maps that allocate entries on iteration + map.forEach((k, v) -> action.accept(k)); + } + + @Override + public int size() { + return map().size(); + } + + @Override + public boolean isEmpty() { + return map().isEmpty(); + } + + @Override + public boolean contains(Object o) { + return map().containsKey(o); + } + + @Override + public boolean remove(Object o) { + if (contains(o)) { + map().remove(o); + return true; + } + return false; + } + + @Override + public void clear() { + map().clear(); + } + } + + + static K keyOrNull(Entry entry) { + return (entry == null) ? null : entry.getKey(); + } + + + static V valueOrNull(Entry entry) { + return (entry == null) ? null : entry.getValue(); + } + + static class SortedKeySet extends KeySet implements SortedSet { + SortedKeySet(SortedMap map) { + super(map); + } + + @Override + SortedMap map() { + return (SortedMap) super.map(); + } + + @Override + public Comparator comparator() { + return map().comparator(); + } + + @Override + public SortedSet subSet(K fromElement, K toElement) { + return new SortedKeySet<>(map().subMap(fromElement, toElement)); + } + + @Override + public SortedSet headSet(K toElement) { + return new SortedKeySet<>(map().headMap(toElement)); + } + + @Override + public SortedSet tailSet(K fromElement) { + return new SortedKeySet<>(map().tailMap(fromElement)); + } + + @Override + public K first() { + return map().firstKey(); + } + + @Override + public K last() { + return map().lastKey(); + } + } + + static class NavigableKeySet extends SortedKeySet implements NavigableSet { + NavigableKeySet(NavigableMap map) { + super(map); + } + + @Override + NavigableMap map() { + return (NavigableMap) map; + } + + @Override + public K lower(K e) { + return map().lowerKey(e); + } + + @Override + public K floor(K e) { + return map().floorKey(e); + } + + @Override + public K ceiling(K e) { + return map().ceilingKey(e); + } + + @Override + public K higher(K e) { + return map().higherKey(e); + } + + @Override + public K pollFirst() { + return keyOrNull(map().pollFirstEntry()); + } + + @Override + public K pollLast() { + return keyOrNull(map().pollLastEntry()); + } + + @Override + public NavigableSet descendingSet() { + return map().descendingKeySet(); + } + + @Override + public Iterator descendingIterator() { + return descendingSet().iterator(); + } + + @Override + public NavigableSet subSet( + K fromElement, boolean fromInclusive, K toElement, boolean toInclusive) { + return map().subMap(fromElement, fromInclusive, toElement, toInclusive).navigableKeySet(); + } + + @Override + public SortedSet subSet(K fromElement, K toElement) { + return subSet(fromElement, true, toElement, false); + } + + @Override + public NavigableSet headSet(K toElement, boolean inclusive) { + return map().headMap(toElement, inclusive).navigableKeySet(); + } + + @Override + public SortedSet headSet(K toElement) { + return headSet(toElement, false); + } + + @Override + public NavigableSet tailSet(K fromElement, boolean inclusive) { + return map().tailMap(fromElement, inclusive).navigableKeySet(); + } + + @Override + public SortedSet tailSet(K fromElement) { + return tailSet(fromElement, true); + } + } + + static class Values extends AbstractCollection { + final Map map; + + Values(Map map) { + this.map = checkNotNull(map); + } + + final Map map() { + return map; + } + + @Override + public Iterator iterator() { + return valueIterator(map().entrySet().iterator()); + } + + @Override + public void forEach(Consumer action) { + checkNotNull(action); + // avoids allocation of entries for those maps that generate fresh entries on iteration + map.forEach((k, v) -> action.accept(v)); + } + + @Override + public boolean remove(Object o) { + try { + return super.remove(o); + } catch (UnsupportedOperationException e) { + for (Entry entry : map().entrySet()) { + if (Objects.equal(o, entry.getValue())) { + map().remove(entry.getKey()); + return true; + } + } + return false; + } + } + + @Override + public boolean removeAll(Collection c) { + try { + return super.removeAll(checkNotNull(c)); + } catch (UnsupportedOperationException e) { + Set toRemove = Sets.newHashSet(); + for (Entry entry : map().entrySet()) { + if (c.contains(entry.getValue())) { + toRemove.add(entry.getKey()); + } + } + return map().keySet().removeAll(toRemove); + } + } + + @Override + public boolean retainAll(Collection c) { + try { + return super.retainAll(checkNotNull(c)); + } catch (UnsupportedOperationException e) { + Set toRetain = Sets.newHashSet(); + for (Entry entry : map().entrySet()) { + if (c.contains(entry.getValue())) { + toRetain.add(entry.getKey()); + } + } + return map().keySet().retainAll(toRetain); + } + } + + @Override + public int size() { + return map().size(); + } + + @Override + public boolean isEmpty() { + return map().isEmpty(); + } + + @Override + public boolean contains(Object o) { + return map().containsValue(o); + } + + @Override + public void clear() { + map().clear(); + } + } + + abstract static class EntrySet extends Sets.ImprovedAbstractSet> { + abstract Map map(); + + @Override + public int size() { + return map().size(); + } + + @Override + public void clear() { + map().clear(); + } + + @Override + public boolean contains(Object o) { + if (o instanceof Entry) { + Entry entry = (Entry) o; + Object key = entry.getKey(); + V value = Maps.safeGet(map(), key); + return Objects.equal(value, entry.getValue()) && (value != null || map().containsKey(key)); + } + return false; + } + + @Override + public boolean isEmpty() { + return map().isEmpty(); + } + + @Override + public boolean remove(Object o) { + if (contains(o)) { + Entry entry = (Entry) o; + return map().keySet().remove(entry.getKey()); + } + return false; + } + + @Override + public boolean removeAll(Collection c) { + try { + return super.removeAll(checkNotNull(c)); + } catch (UnsupportedOperationException e) { + // if the iterators don't support remove + return Sets.removeAllImpl(this, c.iterator()); + } + } + + @Override + public boolean retainAll(Collection c) { + try { + return super.retainAll(checkNotNull(c)); + } catch (UnsupportedOperationException e) { + // if the iterators don't support remove + Set keys = Sets.newHashSetWithExpectedSize(c.size()); + for (Object o : c) { + if (contains(o)) { + Entry entry = (Entry) o; + keys.add(entry.getKey()); + } + } + return map().keySet().retainAll(keys); + } + } + } + + abstract static class DescendingMap extends ForwardingMap + implements NavigableMap { + + abstract NavigableMap forward(); + + @Override + protected final Map delegate() { + return forward(); + } + + private transient Comparator comparator; + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public Comparator comparator() { + Comparator result = comparator; + if (result == null) { + Comparator forwardCmp = forward().comparator(); + if (forwardCmp == null) { + forwardCmp = (Comparator) Ordering.natural(); + } + result = comparator = reverse(forwardCmp); + } + return result; + } + + // If we inline this, we get a javac error. + private static Ordering reverse(Comparator forward) { + return Ordering.from(forward).reverse(); + } + + @Override + public K firstKey() { + return forward().lastKey(); + } + + @Override + public K lastKey() { + return forward().firstKey(); + } + + @Override + public Entry lowerEntry(K key) { + return forward().higherEntry(key); + } + + @Override + public K lowerKey(K key) { + return forward().higherKey(key); + } + + @Override + public Entry floorEntry(K key) { + return forward().ceilingEntry(key); + } + + @Override + public K floorKey(K key) { + return forward().ceilingKey(key); + } + + @Override + public Entry ceilingEntry(K key) { + return forward().floorEntry(key); + } + + @Override + public K ceilingKey(K key) { + return forward().floorKey(key); + } + + @Override + public Entry higherEntry(K key) { + return forward().lowerEntry(key); + } + + @Override + public K higherKey(K key) { + return forward().lowerKey(key); + } + + @Override + public Entry firstEntry() { + return forward().lastEntry(); + } + + @Override + public Entry lastEntry() { + return forward().firstEntry(); + } + + @Override + public Entry pollFirstEntry() { + return forward().pollLastEntry(); + } + + @Override + public Entry pollLastEntry() { + return forward().pollFirstEntry(); + } + + @Override + public NavigableMap descendingMap() { + return forward(); + } + + private transient Set> entrySet; + + @Override + public Set> entrySet() { + Set> result = entrySet; + return (result == null) ? entrySet = createEntrySet() : result; + } + + abstract Iterator> entryIterator(); + + Set> createEntrySet() { + class EntrySetImpl extends EntrySet { + @Override + Map map() { + return DescendingMap.this; + } + + @Override + public Iterator> iterator() { + return entryIterator(); + } + } + return new EntrySetImpl(); + } + + @Override + public Set keySet() { + return navigableKeySet(); + } + + private transient NavigableSet navigableKeySet; + + @Override + public NavigableSet navigableKeySet() { + NavigableSet result = navigableKeySet; + return (result == null) ? navigableKeySet = new NavigableKeySet<>(this) : result; + } + + @Override + public NavigableSet descendingKeySet() { + return forward().navigableKeySet(); + } + + @Override + public NavigableMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + return forward().subMap(toKey, toInclusive, fromKey, fromInclusive).descendingMap(); + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + return forward().tailMap(toKey, inclusive).descendingMap(); + } + + @Override + public SortedMap headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + return forward().headMap(fromKey, inclusive).descendingMap(); + } + + @Override + public SortedMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + @Override + public Collection values() { + return new Values<>(this); + } + + @Override + public String toString() { + return standardToString(); + } + } + + /** Returns a map from the ith element of list to i. */ + static ImmutableMap indexMap(Collection list) { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(list.size()); + int i = 0; + for (E e : list) { + builder.put(e, i++); + } + return builder.build(); + } + + /** + * Returns a view of the portion of {@code map} whose keys are contained by {@code range}. + * + *

This method delegates to the appropriate methods of {@link NavigableMap} (namely {@link + * NavigableMap#subMap(Object, boolean, Object, boolean) subMap()}, {@link + * NavigableMap#tailMap(Object, boolean) tailMap()}, and {@link NavigableMap#headMap(Object, + * boolean) headMap()}) to actually construct the view. Consult these methods for a full + * description of the returned view's behavior. + * + *

Warning: {@code Range}s always represent a range of values using the values' natural + * ordering. {@code NavigableMap} on the other hand can specify a custom ordering via a {@link + * Comparator}, which can violate the natural ordering. Using this method (or in general using + * {@code Range}) with unnaturally-ordered maps can lead to unexpected and undefined behavior. + * + * @since 20.0 + */ + + public static , V> NavigableMap subMap( + NavigableMap map, Range range) { + if (map.comparator() != null + && map.comparator() != Ordering.natural() + && range.hasLowerBound() + && range.hasUpperBound()) { + checkArgument( + map.comparator().compare(range.lowerEndpoint(), range.upperEndpoint()) <= 0, + "map is using a custom comparator which is inconsistent with the natural ordering."); + } + if (range.hasLowerBound() && range.hasUpperBound()) { + return map.subMap( + range.lowerEndpoint(), + range.lowerBoundType() == BoundType.CLOSED, + range.upperEndpoint(), + range.upperBoundType() == BoundType.CLOSED); + } else if (range.hasLowerBound()) { + return map.tailMap(range.lowerEndpoint(), range.lowerBoundType() == BoundType.CLOSED); + } else if (range.hasUpperBound()) { + return map.headMap(range.upperEndpoint(), range.upperBoundType() == BoundType.CLOSED); + } + return checkNotNull(map); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MinMaxPriorityQueue.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MinMaxPriorityQueue.java new file mode 100644 index 0000000000000..26f10aa4997c3 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MinMaxPriorityQueue.java @@ -0,0 +1,944 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import java.util.AbstractQueue; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; +import java.util.Queue; + + + +/** + * A double-ended priority queue, which provides constant-time access to both its least element and + * its greatest element, as determined by the queue's specified comparator. If no comparator is + * given at creation time, the natural order of elements is used. If no maximum size is given at + * creation time, the queue is unbounded. + * + *

Usage example: + * + *

{@code
+ * MinMaxPriorityQueue users = MinMaxPriorityQueue.orderedBy(userComparator)
+ *     .maximumSize(1000)
+ *     .create();
+ * }
+ * + *

As a {@link Queue} it functions exactly as a {@link PriorityQueue}: its head element -- the + * implicit target of the methods {@link #peek()}, {@link #poll()} and {@link #remove()} -- is + * defined as the least element in the queue according to the queue's comparator. But unlike + * a regular priority queue, the methods {@link #peekLast}, {@link #pollLast} and {@link + * #removeLast} are also provided, to act on the greatest element in the queue instead. + * + *

A min-max priority queue can be configured with a maximum size. If so, each time the size of + * the queue exceeds that value, the queue automatically removes its greatest element according to + * its comparator (which might be the element that was just added). This is different from + * conventional bounded queues, which either block or reject new elements when full. + * + *

This implementation is based on the min-max heap developed by Atkinson, et al. + * Unlike many other double-ended priority queues, it stores elements in a single array, as compact + * as the traditional heap data structure used in {@link PriorityQueue}. + * + *

This class is not thread-safe, and does not accept null elements. + * + *

Performance notes: + * + *

    + *
  • If you only access one end of the queue, and do use a maximum size, this class will perform + * significantly worse than a {@code PriorityQueue} with manual eviction above the maximum + * size. In many cases {@link Ordering#leastOf} may work for your use case with significantly + * improved (and asymptotically superior) performance. + *
  • The retrieval operations {@link #peek}, {@link #peekFirst}, {@link #peekLast}, {@link + * #element}, and {@link #size} are constant-time. + *
  • The enqueuing and dequeuing operations ({@link #offer}, {@link #add}, and all the forms of + * {@link #poll} and {@link #remove()}) run in {@code O(log n) time}. + *
  • The {@link #remove(Object)} and {@link #contains} operations require linear ({@code O(n)}) + * time. + *
  • If you only access one end of the queue, and don't use a maximum size, this class is + * functionally equivalent to {@link PriorityQueue}, but significantly slower. + *
+ * + * @author Sverre Sundsdal + * @author Torbjorn Gannholm + * @since 8.0 + */ + +public final class MinMaxPriorityQueue extends AbstractQueue { + + /** + * Creates a new min-max priority queue with default settings: natural order, no maximum size, no + * initial contents, and an initial expected size of 11. + */ + @SuppressWarnings("rawtypes") + public static > MinMaxPriorityQueue create() { + return new Builder(Ordering.natural()).create(); + } + + /** + * Creates a new min-max priority queue using natural order, no maximum size, and initially + * containing the given elements. + */ + public static > MinMaxPriorityQueue create( + Iterable initialContents) { + return new Builder(Ordering.natural()).create(initialContents); + } + + /** + * Creates and returns a new builder, configured to build {@code MinMaxPriorityQueue} instances + * that use {@code comparator} to determine the least and greatest elements. + */ + public static Builder orderedBy(Comparator comparator) { + return new Builder(comparator); + } + + /** + * Creates and returns a new builder, configured to build {@code MinMaxPriorityQueue} instances + * sized appropriately to hold {@code expectedSize} elements. + */ + @SuppressWarnings("rawtypes") + public static Builder expectedSize(int expectedSize) { + return new Builder(Ordering.natural()).expectedSize(expectedSize); + } + + /** + * Creates and returns a new builder, configured to build {@code MinMaxPriorityQueue} instances + * that are limited to {@code maximumSize} elements. Each time a queue grows beyond this bound, it + * immediately removes its greatest element (according to its comparator), which might be the + * element that was just added. + */ + @SuppressWarnings("rawtypes") + public static Builder maximumSize(int maximumSize) { + return new Builder(Ordering.natural()).maximumSize(maximumSize); + } + + /** + * The builder class used in creation of min-max priority queues. Instead of constructing one + * directly, use {@link MinMaxPriorityQueue#orderedBy(Comparator)}, {@link + * MinMaxPriorityQueue#expectedSize(int)} or {@link MinMaxPriorityQueue#maximumSize(int)}. + * + * @param the upper bound on the eventual type that can be produced by this builder (for + * example, a {@code Builder} can produce a {@code Queue} or {@code + * Queue} but not a {@code Queue}). + * @since 8.0 + */ + + public static final class Builder { + /* + * TODO(kevinb): when the dust settles, see if we still need this or can + * just default to DEFAULT_CAPACITY. + */ + private static final int UNSET_EXPECTED_SIZE = -1; + + private final Comparator comparator; + private int expectedSize = UNSET_EXPECTED_SIZE; + private int maximumSize = Integer.MAX_VALUE; + + private Builder(Comparator comparator) { + this.comparator = checkNotNull(comparator); + } + + /** + * Configures this builder to build min-max priority queues with an initial expected size of + * {@code expectedSize}. + */ + public Builder expectedSize(int expectedSize) { + checkArgument(expectedSize >= 0); + this.expectedSize = expectedSize; + return this; + } + + /** + * Configures this builder to build {@code MinMaxPriorityQueue} instances that are limited to + * {@code maximumSize} elements. Each time a queue grows beyond this bound, it immediately + * removes its greatest element (according to its comparator), which might be the element that + * was just added. + */ + public Builder maximumSize(int maximumSize) { + checkArgument(maximumSize > 0); + this.maximumSize = maximumSize; + return this; + } + + /** + * Builds a new min-max priority queue using the previously specified options, and having no + * initial contents. + */ + public MinMaxPriorityQueue create() { + return create(Collections.emptySet()); + } + + /** + * Builds a new min-max priority queue using the previously specified options, and having the + * given initial elements. + */ + public MinMaxPriorityQueue create(Iterable initialContents) { + MinMaxPriorityQueue queue = + new MinMaxPriorityQueue( + this, initialQueueSize(expectedSize, maximumSize, initialContents)); + for (T element : initialContents) { + queue.offer(element); + } + return queue; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // safe "contravariant cast" + private Ordering ordering() { + return Ordering.from((Comparator) comparator); + } + } + + private final Heap minHeap; + private final Heap maxHeap; + final int maximumSize; + private Object[] queue; + private int size; + private int modCount; + + private MinMaxPriorityQueue(Builder builder, int queueSize) { + Ordering ordering = builder.ordering(); + this.minHeap = new Heap(ordering); + this.maxHeap = new Heap(ordering.reverse()); + minHeap.otherHeap = maxHeap; + maxHeap.otherHeap = minHeap; + + this.maximumSize = builder.maximumSize; + // TODO(kevinb): pad? + this.queue = new Object[queueSize]; + } + + @Override + public int size() { + return size; + } + + /** + * Adds the given element to this queue. If this queue has a maximum size, after adding {@code + * element} the queue will automatically evict its greatest element (according to its comparator), + * which may be {@code element} itself. + * + * @return {@code true} always + */ + @Override + public boolean add(E element) { + offer(element); + return true; + } + + @Override + public boolean addAll(Collection newElements) { + boolean modified = false; + for (E element : newElements) { + offer(element); + modified = true; + } + return modified; + } + + /** + * Adds the given element to this queue. If this queue has a maximum size, after adding {@code + * element} the queue will automatically evict its greatest element (according to its comparator), + * which may be {@code element} itself. + */ + @Override + public boolean offer(E element) { + checkNotNull(element); + modCount++; + int insertIndex = size++; + + growIfNeeded(); + + // Adds the element to the end of the heap and bubbles it up to the correct + // position. + heapForIndex(insertIndex).bubbleUp(insertIndex, element); + return size <= maximumSize || pollLast() != element; + } + + @Override + public E poll() { + return isEmpty() ? null : removeAndGet(0); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // we must carefully only allow Es to get in + E elementData(int index) { + return (E) queue[index]; + } + + @Override + public E peek() { + return isEmpty() ? null : elementData(0); + } + + /** Returns the index of the max element. */ + private int getMaxElementIndex() { + switch (size) { + case 1: + return 0; // The lone element in the queue is the maximum. + case 2: + return 1; // The lone element in the maxHeap is the maximum. + default: + // The max element must sit on the first level of the maxHeap. It is + // actually the *lesser* of the two from the maxHeap's perspective. + return (maxHeap.compareElements(1, 2) <= 0) ? 1 : 2; + } + } + + /** + * Removes and returns the least element of this queue, or returns {@code null} if the queue is + * empty. + */ + public E pollFirst() { + return poll(); + } + + /** + * Removes and returns the least element of this queue. + * + * @throws NoSuchElementException if the queue is empty + */ + public E removeFirst() { + return remove(); + } + + /** + * Retrieves, but does not remove, the least element of this queue, or returns {@code null} if the + * queue is empty. + */ + public E peekFirst() { + return peek(); + } + + /** + * Removes and returns the greatest element of this queue, or returns {@code null} if the queue is + * empty. + */ + public E pollLast() { + return isEmpty() ? null : removeAndGet(getMaxElementIndex()); + } + + /** + * Removes and returns the greatest element of this queue. + * + * @throws NoSuchElementException if the queue is empty + */ + public E removeLast() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return removeAndGet(getMaxElementIndex()); + } + + /** + * Retrieves, but does not remove, the greatest element of this queue, or returns {@code null} if + * the queue is empty. + */ + public E peekLast() { + return isEmpty() ? null : elementData(getMaxElementIndex()); + } + + /** + * Removes the element at position {@code index}. + * + *

Normally this method leaves the elements at up to {@code index - 1}, inclusive, untouched. + * Under these circumstances, it returns {@code null}. + * + *

Occasionally, in order to maintain the heap invariant, it must swap a later element of the + * list with one before {@code index}. Under these circumstances it returns a pair of elements as + * a {@link MoveDesc}. The first one is the element that was previously at the end of the heap and + * is now at some position before {@code index}. The second element is the one that was swapped + * down to replace the element at {@code index}. This fact is used by iterator.remove so as to + * visit elements during a traversal once and only once. + */ + + MoveDesc removeAt(int index) { + checkPositionIndex(index, size); + modCount++; + size--; + if (size == index) { + queue[size] = null; + return null; + } + E actualLastElement = elementData(size); + int lastElementAt = heapForIndex(size).swapWithConceptuallyLastElement(actualLastElement); + if (lastElementAt == index) { + // 'actualLastElement' is now at 'lastElementAt', and the element that was at 'lastElementAt' + // is now at the end of queue. If that's the element we wanted to remove in the first place, + // don't try to (incorrectly) trickle it. Instead, just delete it and we're done. + queue[size] = null; + return null; + } + E toTrickle = elementData(size); + queue[size] = null; + MoveDesc changes = fillHole(index, toTrickle); + if (lastElementAt < index) { + // Last element is moved to before index, swapped with trickled element. + if (changes == null) { + // The trickled element is still after index. + return new MoveDesc(actualLastElement, toTrickle); + } else { + // The trickled element is back before index, but the replaced element + // has now been moved after index. + return new MoveDesc(actualLastElement, changes.replaced); + } + } + // Trickled element was after index to begin with, no adjustment needed. + return changes; + } + + private MoveDesc fillHole(int index, E toTrickle) { + Heap heap = heapForIndex(index); + // We consider elementData(index) a "hole", and we want to fill it + // with the last element of the heap, toTrickle. + // Since the last element of the heap is from the bottom level, we + // optimistically fill index position with elements from lower levels, + // moving the hole down. In most cases this reduces the number of + // comparisons with toTrickle, but in some cases we will need to bubble it + // all the way up again. + int vacated = heap.fillHoleAt(index); + // Try to see if toTrickle can be bubbled up min levels. + int bubbledTo = heap.bubbleUpAlternatingLevels(vacated, toTrickle); + if (bubbledTo == vacated) { + // Could not bubble toTrickle up min levels, try moving + // it from min level to max level (or max to min level) and bubble up + // there. + return heap.tryCrossOverAndBubbleUp(index, vacated, toTrickle); + } else { + return (bubbledTo < index) ? new MoveDesc(toTrickle, elementData(index)) : null; + } + } + + // Returned from removeAt() to iterator.remove() + static class MoveDesc { + final E toTrickle; + final E replaced; + + MoveDesc(E toTrickle, E replaced) { + this.toTrickle = toTrickle; + this.replaced = replaced; + } + } + + /** Removes and returns the value at {@code index}. */ + private E removeAndGet(int index) { + E value = elementData(index); + removeAt(index); + return value; + } + + private Heap heapForIndex(int i) { + return isEvenLevel(i) ? minHeap : maxHeap; + } + + private static final int EVEN_POWERS_OF_TWO = 0x55555555; + private static final int ODD_POWERS_OF_TWO = 0xaaaaaaaa; + + + static boolean isEvenLevel(int index) { + int oneBased = ~~(index + 1); // for GWT + checkState(oneBased > 0, "negative index"); + return (oneBased & EVEN_POWERS_OF_TWO) > (oneBased & ODD_POWERS_OF_TWO); + } + + /** + * Returns {@code true} if the MinMax heap structure holds. This is only used in testing. + * + *

TODO(kevinb): move to the test class? + */ + + boolean isIntact() { + for (int i = 1; i < size; i++) { + if (!heapForIndex(i).verifyIndex(i)) { + return false; + } + } + return true; + } + + /** + * Each instance of MinMaxPriortyQueue encapsulates two instances of Heap: a min-heap and a + * max-heap. Conceptually, these might each have their own array for storage, but for efficiency's + * sake they are stored interleaved on alternate heap levels in the same array (MMPQ.queue). + */ + private class Heap { + final Ordering ordering; + Heap otherHeap; + + Heap(Ordering ordering) { + this.ordering = ordering; + } + + int compareElements(int a, int b) { + return ordering.compare(elementData(a), elementData(b)); + } + + /** + * Tries to move {@code toTrickle} from a min to a max level and bubble up there. If it moved + * before {@code removeIndex} this method returns a pair as described in {@link #removeAt}. + */ + MoveDesc tryCrossOverAndBubbleUp(int removeIndex, int vacated, E toTrickle) { + int crossOver = crossOver(vacated, toTrickle); + if (crossOver == vacated) { + return null; + } + // Successfully crossed over from min to max. + // Bubble up max levels. + E parent; + // If toTrickle is moved up to a parent of removeIndex, the parent is + // placed in removeIndex position. We must return that to the iterator so + // that it knows to skip it. + if (crossOver < removeIndex) { + // We crossed over to the parent level in crossOver, so the parent + // has already been moved. + parent = elementData(removeIndex); + } else { + parent = elementData(getParentIndex(removeIndex)); + } + // bubble it up the opposite heap + if (otherHeap.bubbleUpAlternatingLevels(crossOver, toTrickle) < removeIndex) { + return new MoveDesc(toTrickle, parent); + } else { + return null; + } + } + + /** Bubbles a value from {@code index} up the appropriate heap if required. */ + void bubbleUp(int index, E x) { + int crossOver = crossOverUp(index, x); + + Heap heap; + if (crossOver == index) { + heap = this; + } else { + index = crossOver; + heap = otherHeap; + } + heap.bubbleUpAlternatingLevels(index, x); + } + + /** + * Bubbles a value from {@code index} up the levels of this heap, and returns the index the + * element ended up at. + */ + int bubbleUpAlternatingLevels(int index, E x) { + while (index > 2) { + int grandParentIndex = getGrandparentIndex(index); + E e = elementData(grandParentIndex); + if (ordering.compare(e, x) <= 0) { + break; + } + queue[index] = e; + index = grandParentIndex; + } + queue[index] = x; + return index; + } + + /** + * Returns the index of minimum value between {@code index} and {@code index + len}, or {@code + * -1} if {@code index} is greater than {@code size}. + */ + int findMin(int index, int len) { + if (index >= size) { + return -1; + } + checkState(index > 0); + int limit = Math.min(index, size - len) + len; + int minIndex = index; + for (int i = index + 1; i < limit; i++) { + if (compareElements(i, minIndex) < 0) { + minIndex = i; + } + } + return minIndex; + } + + /** Returns the minimum child or {@code -1} if no child exists. */ + int findMinChild(int index) { + return findMin(getLeftChildIndex(index), 2); + } + + /** Returns the minimum grand child or -1 if no grand child exists. */ + int findMinGrandChild(int index) { + int leftChildIndex = getLeftChildIndex(index); + if (leftChildIndex < 0) { + return -1; + } + return findMin(getLeftChildIndex(leftChildIndex), 4); + } + + /** + * Moves an element one level up from a min level to a max level (or vice versa). Returns the + * new position of the element. + */ + int crossOverUp(int index, E x) { + if (index == 0) { + queue[0] = x; + return 0; + } + int parentIndex = getParentIndex(index); + E parentElement = elementData(parentIndex); + if (parentIndex != 0) { + // This is a guard for the case of the childless uncle. + // Since the end of the array is actually the middle of the heap, + // a smaller childless uncle can become a child of x when we + // bubble up alternate levels, violating the invariant. + int grandparentIndex = getParentIndex(parentIndex); + int uncleIndex = getRightChildIndex(grandparentIndex); + if (uncleIndex != parentIndex && getLeftChildIndex(uncleIndex) >= size) { + E uncleElement = elementData(uncleIndex); + if (ordering.compare(uncleElement, parentElement) < 0) { + parentIndex = uncleIndex; + parentElement = uncleElement; + } + } + } + if (ordering.compare(parentElement, x) < 0) { + queue[index] = parentElement; + queue[parentIndex] = x; + return parentIndex; + } + queue[index] = x; + return index; + } + + /** + * Swap {@code actualLastElement} with the conceptually correct last element of the heap. + * Returns the index that {@code actualLastElement} now resides in. + * + *

Since the last element of the array is actually in the middle of the sorted structure, a + * childless uncle node could be smaller, which would corrupt the invariant if this element + * becomes the new parent of the uncle. In that case, we first switch the last element with its + * uncle, before returning. + */ + int swapWithConceptuallyLastElement(E actualLastElement) { + int parentIndex = getParentIndex(size); + if (parentIndex != 0) { + int grandparentIndex = getParentIndex(parentIndex); + int uncleIndex = getRightChildIndex(grandparentIndex); + if (uncleIndex != parentIndex && getLeftChildIndex(uncleIndex) >= size) { + E uncleElement = elementData(uncleIndex); + if (ordering.compare(uncleElement, actualLastElement) < 0) { + queue[uncleIndex] = actualLastElement; + queue[size] = uncleElement; + return uncleIndex; + } + } + } + return size; + } + + /** + * Crosses an element over to the opposite heap by moving it one level down (or up if there are + * no elements below it). + * + *

Returns the new position of the element. + */ + int crossOver(int index, E x) { + int minChildIndex = findMinChild(index); + // TODO(kevinb): split the && into two if's and move crossOverUp so it's + // only called when there's no child. + if ((minChildIndex > 0) && (ordering.compare(elementData(minChildIndex), x) < 0)) { + queue[index] = elementData(minChildIndex); + queue[minChildIndex] = x; + return minChildIndex; + } + return crossOverUp(index, x); + } + + /** + * Fills the hole at {@code index} by moving in the least of its grandchildren to this position, + * then recursively filling the new hole created. + * + * @return the position of the new hole (where the lowest grandchild moved from, that had no + * grandchild to replace it) + */ + int fillHoleAt(int index) { + int minGrandchildIndex; + while ((minGrandchildIndex = findMinGrandChild(index)) > 0) { + queue[index] = elementData(minGrandchildIndex); + index = minGrandchildIndex; + } + return index; + } + + private boolean verifyIndex(int i) { + if ((getLeftChildIndex(i) < size) && (compareElements(i, getLeftChildIndex(i)) > 0)) { + return false; + } + if ((getRightChildIndex(i) < size) && (compareElements(i, getRightChildIndex(i)) > 0)) { + return false; + } + if ((i > 0) && (compareElements(i, getParentIndex(i)) > 0)) { + return false; + } + if ((i > 2) && (compareElements(getGrandparentIndex(i), i) > 0)) { + return false; + } + return true; + } + + // These would be static if inner classes could have static members. + + private int getLeftChildIndex(int i) { + return i * 2 + 1; + } + + private int getRightChildIndex(int i) { + return i * 2 + 2; + } + + private int getParentIndex(int i) { + return (i - 1) / 2; + } + + private int getGrandparentIndex(int i) { + return getParentIndex(getParentIndex(i)); // (i - 3) / 4 + } + } + + /** + * Iterates the elements of the queue in no particular order. + * + *

If the underlying queue is modified during iteration an exception will be thrown. + */ + private class QueueIterator implements Iterator { + private int cursor = -1; + private int nextCursor = -1; + private int expectedModCount = modCount; + // The same element is not allowed in both forgetMeNot and skipMe, but duplicates are allowed in + // either of them, up to the same multiplicity as the queue. + private Queue forgetMeNot; + private List skipMe; + private E lastFromForgetMeNot; + private boolean canRemove; + + @Override + public boolean hasNext() { + checkModCount(); + nextNotInSkipMe(cursor + 1); + return (nextCursor < size()) || ((forgetMeNot != null) && !forgetMeNot.isEmpty()); + } + + @Override + public E next() { + checkModCount(); + nextNotInSkipMe(cursor + 1); + if (nextCursor < size()) { + cursor = nextCursor; + canRemove = true; + return elementData(cursor); + } else if (forgetMeNot != null) { + cursor = size(); + lastFromForgetMeNot = forgetMeNot.poll(); + if (lastFromForgetMeNot != null) { + canRemove = true; + return lastFromForgetMeNot; + } + } + throw new NoSuchElementException("iterator moved past last element in queue."); + } + + @Override + public void remove() { + checkRemove(canRemove); + checkModCount(); + canRemove = false; + expectedModCount++; + if (cursor < size()) { + MoveDesc moved = removeAt(cursor); + if (moved != null) { + if (forgetMeNot == null) { + forgetMeNot = new ArrayDeque(); + skipMe = new ArrayList(3); + } + if (!foundAndRemovedExactReference(skipMe, moved.toTrickle)) { + forgetMeNot.add(moved.toTrickle); + } + if (!foundAndRemovedExactReference(forgetMeNot, moved.replaced)) { + skipMe.add(moved.replaced); + } + } + cursor--; + nextCursor--; + } else { // we must have set lastFromForgetMeNot in next() + checkState(removeExact(lastFromForgetMeNot)); + lastFromForgetMeNot = null; + } + } + + /** Returns true if an exact reference (==) was found and removed from the supplied iterable. */ + private boolean foundAndRemovedExactReference(Iterable elements, E target) { + for (Iterator it = elements.iterator(); it.hasNext(); ) { + E element = it.next(); + if (element == target) { + it.remove(); + return true; + } + } + return false; + } + + /** Removes only this exact instance, not others that are equals() */ + private boolean removeExact(Object target) { + for (int i = 0; i < size; i++) { + if (queue[i] == target) { + removeAt(i); + return true; + } + } + return false; + } + + private void checkModCount() { + if (modCount != expectedModCount) { + throw new ConcurrentModificationException(); + } + } + + /** + * Advances nextCursor to the index of the first element after {@code c} that is not in {@code + * skipMe} and returns {@code size()} if there is no such element. + */ + private void nextNotInSkipMe(int c) { + if (nextCursor < c) { + if (skipMe != null) { + while (c < size() && foundAndRemovedExactReference(skipMe, elementData(c))) { + c++; + } + } + nextCursor = c; + } + } + } + + /** + * Returns an iterator over the elements contained in this collection, in no particular + * order. + * + *

The iterator is fail-fast: If the MinMaxPriorityQueue is modified at any time after + * the iterator is created, in any way except through the iterator's own remove method, the + * iterator will generally throw a {@link ConcurrentModificationException}. Thus, in the face of + * concurrent modification, the iterator fails quickly and cleanly, rather than risking arbitrary, + * non-deterministic behavior at an undetermined time in the future. + * + *

Note that the fail-fast behavior of an iterator cannot be guaranteed as it is, generally + * speaking, impossible to make any hard guarantees in the presence of unsynchronized concurrent + * modification. Fail-fast iterators throw {@code ConcurrentModificationException} on a + * best-effort basis. Therefore, it would be wrong to write a program that depended on this + * exception for its correctness: the fail-fast behavior of iterators should be used only to + * detect bugs. + * + * @return an iterator over the elements contained in this collection + */ + @Override + public Iterator iterator() { + return new QueueIterator(); + } + + @Override + public void clear() { + for (int i = 0; i < size; i++) { + queue[i] = null; + } + size = 0; + } + + @Override + public Object[] toArray() { + Object[] copyTo = new Object[size]; + System.arraycopy(queue, 0, copyTo, 0, size); + return copyTo; + } + + /** + * Returns the comparator used to order the elements in this queue. Obeys the general contract of + * {@link PriorityQueue#comparator}, but returns {@link Ordering#natural} instead of {@code null} + * to indicate natural ordering. + */ + public Comparator comparator() { + return minHeap.ordering; + } + + + int capacity() { + return queue.length; + } + + // Size/capacity-related methods + + private static final int DEFAULT_CAPACITY = 11; + + + static int initialQueueSize( + int configuredExpectedSize, int maximumSize, Iterable initialContents) { + // Start with what they said, if they said it, otherwise DEFAULT_CAPACITY + int result = + (configuredExpectedSize == Builder.UNSET_EXPECTED_SIZE) + ? DEFAULT_CAPACITY + : configuredExpectedSize; + + // Enlarge to contain initial contents + if (initialContents instanceof Collection) { + int initialSize = ((Collection) initialContents).size(); + result = Math.max(result, initialSize); + } + + // Now cap it at maxSize + 1 + return capAtMaximumSize(result, maximumSize); + } + + private void growIfNeeded() { + if (size > queue.length) { + int newCapacity = calculateNewCapacity(); + Object[] newQueue = new Object[newCapacity]; + System.arraycopy(queue, 0, newQueue, 0, queue.length); + queue = newQueue; + } + } + + /** Returns ~2x the old capacity if small; ~1.5x otherwise. */ + private int calculateNewCapacity() { + int oldCapacity = queue.length; + int newCapacity = + (oldCapacity < 64) ? (oldCapacity + 1) * 2 : IntMath.checkedMultiply(oldCapacity / 2, 3); + return capAtMaximumSize(newCapacity, maximumSize); + } + + /** There's no reason for the queueSize to ever be more than maxSize + 1 */ + private static int capAtMaximumSize(int queueSize, int maximumSize) { + return Math.min(queueSize - 1, maximumSize) + 1; // don't overflow + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MoreCollectors.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MoreCollectors.java new file mode 100644 index 0000000000000..3c18f6aafcfb9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MoreCollectors.java @@ -0,0 +1,172 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.stream.Collector; + + +/** + * Collectors not present in {@code java.util.stream.Collectors} that are not otherwise associated + * with a {@code com.google.common} type. + * + * @author Louis Wasserman + * @since 21.0 + */ + +public final class MoreCollectors { + + /* + * TODO(lowasser): figure out if we can convert this to a concurrent AtomicReference-based + * collector without breaking j2cl? + */ + private static final Collector> TO_OPTIONAL = + Collector.of( + ToOptionalState::new, + ToOptionalState::add, + ToOptionalState::combine, + ToOptionalState::getOptional, + Collector.Characteristics.UNORDERED); + + /** + * A collector that converts a stream of zero or one elements to an {@code Optional}. The returned + * collector throws an {@code IllegalArgumentException} if the stream consists of two or more + * elements, and a {@code NullPointerException} if the stream consists of exactly one element, + * which is null. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Collector> toOptional() { + return (Collector) TO_OPTIONAL; + } + + private static final Object NULL_PLACEHOLDER = new Object(); + + private static final Collector ONLY_ELEMENT = + Collector.of( + ToOptionalState::new, + (state, o) -> state.add((o == null) ? NULL_PLACEHOLDER : o), + ToOptionalState::combine, + state -> { + Object result = state.getElement(); + return (result == NULL_PLACEHOLDER) ? null : result; + }, + Collector.Characteristics.UNORDERED); + + /** + * A collector that takes a stream containing exactly one element and returns that element. The + * returned collector throws an {@code IllegalArgumentException} if the stream consists of two or + * more elements, and a {@code NoSuchElementException} if the stream is empty. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Collector onlyElement() { + return (Collector) ONLY_ELEMENT; + } + + /** + * This atrocity is here to let us report several of the elements in the stream if there were more + * than one, not just two. + */ + private static final class ToOptionalState { + static final int MAX_EXTRAS = 4; + + Object element; + List extras; + + ToOptionalState() { + element = null; + extras = null; + } + + IllegalArgumentException multiples(boolean overflow) { + StringBuilder sb = + new StringBuilder().append("expected one element but was: <").append(element); + for (Object o : extras) { + sb.append(", ").append(o); + } + if (overflow) { + sb.append(", ..."); + } + sb.append('>'); + throw new IllegalArgumentException(sb.toString()); + } + + void add(Object o) { + checkNotNull(o); + if (element == null) { + this.element = o; + } else if (extras == null) { + extras = new ArrayList<>(MAX_EXTRAS); + extras.add(o); + } else if (extras.size() < MAX_EXTRAS) { + extras.add(o); + } else { + throw multiples(true); + } + } + + ToOptionalState combine(ToOptionalState other) { + if (element == null) { + return other; + } else if (other.element == null) { + return this; + } else { + if (extras == null) { + extras = new ArrayList<>(); + } + extras.add(other.element); + if (other.extras != null) { + this.extras.addAll(other.extras); + } + if (extras.size() > MAX_EXTRAS) { + extras.subList(MAX_EXTRAS, extras.size()).clear(); + throw multiples(true); + } + return this; + } + } + + Optional getOptional() { + if (extras == null) { + return Optional.ofNullable(element); + } else { + throw multiples(false); + } + } + + Object getElement() { + if (element == null) { + throw new NoSuchElementException(); + } else if (extras == null) { + return element; + } else { + throw multiples(false); + } + } + } + + private MoreCollectors() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimap.java new file mode 100644 index 0000000000000..4e55c9cfe2196 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimap.java @@ -0,0 +1,382 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.function.BiConsumer; + + +/** + * A collection that maps keys to values, similar to {@link Map}, but in which each key may be + * associated with multiple values. You can visualize the contents of a multimap either as a + * map from keys to nonempty collections of values: + * + *
    + *
  • a → 1, 2 + *
  • b → 3 + *
+ * + * ... or as a single "flattened" collection of key-value pairs: + * + *
    + *
  • a → 1 + *
  • a → 2 + *
  • b → 3 + *
+ * + *

Important: although the first interpretation resembles how most multimaps are + * implemented, the design of the {@code Multimap} API is based on the second form. + * So, using the multimap shown above as an example, the {@link #size} is {@code 3}, not {@code 2}, + * and the {@link #values} collection is {@code [1, 2, 3]}, not {@code [[1, 2], [3]]}. For those + * times when the first style is more useful, use the multimap's {@link #asMap} view (or create a + * {@code Map>} in the first place). + * + *

Example

+ * + *

The following code: + * + *

{@code
+ * ListMultimap multimap = ArrayListMultimap.create();
+ * for (President pres : US_PRESIDENTS_IN_ORDER) {
+ *   multimap.put(pres.firstName(), pres.lastName());
+ * }
+ * for (String firstName : multimap.keySet()) {
+ *   List lastNames = multimap.get(firstName);
+ *   out.println(firstName + ": " + lastNames);
+ * }
+ * }
+ * + * ... produces output such as: + * + *
{@code
+ * Zachary: [Taylor]
+ * John: [Adams, Adams, Tyler, Kennedy]  // Remember, Quincy!
+ * George: [Washington, Bush, Bush]
+ * Grover: [Cleveland, Cleveland]        // Two, non-consecutive terms, rep'ing NJ!
+ * ...
+ * }
+ * + *

Views

+ * + *

Much of the power of the multimap API comes from the view collections it provides. + * These always reflect the latest state of the multimap itself. When they support modification, the + * changes are write-through (they automatically update the backing multimap). These view + * collections are: + * + *

    + *
  • {@link #asMap}, mentioned above + *
  • {@link #keys}, {@link #keySet}, {@link #values}, {@link #entries}, which are similar to the + * corresponding view collections of {@link Map} + *
  • and, notably, even the collection returned by {@link #get get(key)} is an active view of + * the values corresponding to {@code key} + *
+ * + *

The collections returned by the {@link #replaceValues replaceValues} and {@link #removeAll + * removeAll} methods, which contain values that have just been removed from the multimap, are + * naturally not views. + * + *

Subinterfaces

+ * + *

Instead of using the {@code Multimap} interface directly, prefer the subinterfaces {@link + * ListMultimap} and {@link SetMultimap}. These take their names from the fact that the collections + * they return from {@code get} behave like (and, of course, implement) {@link List} and {@link + * Set}, respectively. + * + *

For example, the "presidents" code snippet above used a {@code ListMultimap}; if it had used a + * {@code SetMultimap} instead, two presidents would have vanished, and last names might or might + * not appear in chronological order. + * + *

Warning: instances of type {@code Multimap} may not implement {@link Object#equals} in + * the way you expect. Multimaps containing the same key-value pairs, even in the same order, may or + * may not be equal and may or may not have the same {@code hashCode}. The recommended subinterfaces + * provide much stronger guarantees. + * + *

Comparison to a map of collections

+ * + *

Multimaps are commonly used in places where a {@code Map>} would otherwise + * have appeared. The differences include: + * + *

    + *
  • There is no need to populate an empty collection before adding an entry with {@link #put + * put}. + *
  • {@code get} never returns {@code null}, only an empty collection. + *
  • A key is contained in the multimap if and only if it maps to at least one value. Any + * operation that causes a key to have zero associated values has the effect of + * removing that key from the multimap. + *
  • The total entry count is available as {@link #size}. + *
  • Many complex operations become easier; for example, {@code + * Collections.min(multimap.values())} finds the smallest value across all keys. + *
+ * + *

Implementations

+ * + *

As always, prefer the immutable implementations, {@link ImmutableListMultimap} and {@link + * ImmutableSetMultimap}. General-purpose mutable implementations are listed above under "All Known + * Implementing Classes". You can also create a custom multimap, backed by any {@code Map} + * and {@link Collection} types, using the {@link Multimaps#newMultimap Multimaps.newMultimap} + * family of methods. Finally, another popular way to obtain a multimap is using {@link + * Multimaps#index Multimaps.index}. See the {@link Multimaps} class for these and other static + * utilities related to multimaps. + * + *

Other Notes

+ * + *

As with {@code Map}, the behavior of a {@code Multimap} is not specified if key objects + * already present in the multimap change in a manner that affects {@code equals} comparisons. Use + * caution if mutable objects are used as keys in a {@code Multimap}. + * + *

All methods that modify the multimap are optional. The view collections returned by the + * multimap may or may not be modifiable. Any modification method that is not supported will throw + * {@link UnsupportedOperationException}. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @since 2.0 + */ +public interface Multimap { + // Query Operations + + /** + * Returns the number of key-value pairs in this multimap. + * + *

Note: this method does not return the number of distinct keys in the multimap, + * which is given by {@code keySet().size()} or {@code asMap().size()}. See the opening section of + * the {@link Multimap} class documentation for clarification. + */ + int size(); + + /** + * Returns {@code true} if this multimap contains no key-value pairs. Equivalent to {@code size() + * == 0}, but can in some cases be more efficient. + */ + boolean isEmpty(); + + /** + * Returns {@code true} if this multimap contains at least one key-value pair with the key {@code + * key}. + */ + boolean containsKey(/*@CompatibleWith("K")*/ Object key); + + /** + * Returns {@code true} if this multimap contains at least one key-value pair with the value + * {@code value}. + */ + boolean containsValue(/*@CompatibleWith("V")*/ Object value); + + /** + * Returns {@code true} if this multimap contains at least one key-value pair with the key {@code + * key} and the value {@code value}. + */ + boolean containsEntry( + /*@CompatibleWith("K")*/ Object key, + /*@CompatibleWith("V")*/ Object value); + + // Modification Operations + + /** + * Stores a key-value pair in this multimap. + * + *

Some multimap implementations allow duplicate key-value pairs, in which case {@code put} + * always adds a new key-value pair and increases the multimap size by 1. Other implementations + * prohibit duplicates, and storing a key-value pair that's already in the multimap has no effect. + * + * @return {@code true} if the method increased the size of the multimap, or {@code false} if the + * multimap already contained the key-value pair and doesn't allow duplicates + */ + boolean put(K key, V value); + + /** + * Removes a single key-value pair with the key {@code key} and the value {@code value} from this + * multimap, if such exists. If multiple key-value pairs in the multimap fit this description, + * which one is removed is unspecified. + * + * @return {@code true} if the multimap changed + */ + boolean remove( + /*@CompatibleWith("K")*/ Object key, + /*@CompatibleWith("V")*/ Object value); + + // Bulk Operations + + /** + * Stores a key-value pair in this multimap for each of {@code values}, all using the same key, + * {@code key}. Equivalent to (but expected to be more efficient than): + * + *

{@code
+   * for (V value : values) {
+   *   put(key, value);
+   * }
+   * }
+ * + *

In particular, this is a no-op if {@code values} is empty. + * + * @return {@code true} if the multimap changed + */ + boolean putAll(K key, Iterable values); + + /** + * Stores all key-value pairs of {@code multimap} in this multimap, in the order returned by + * {@code multimap.entries()}. + * + * @return {@code true} if the multimap changed + */ + boolean putAll(Multimap multimap); + + /** + * Stores a collection of values with the same key, replacing any existing values for that key. + * + *

If {@code values} is empty, this is equivalent to {@link #removeAll(Object) removeAll(key)}. + * + * @return the collection of replaced values, or an empty collection if no values were previously + * associated with the key. The collection may be modifiable, but updating it will have + * no effect on the multimap. + */ + Collection replaceValues(K key, Iterable values); + + /** + * Removes all values associated with the key {@code key}. + * + *

Once this method returns, {@code key} will not be mapped to any values, so it will not + * appear in {@link #keySet()}, {@link #asMap()}, or any other views. + * + * @return the values that were removed (possibly empty). The returned collection may be + * modifiable, but updating it will have no effect on the multimap. + */ + Collection removeAll(/*@CompatibleWith("K")*/ Object key); + + /** Removes all key-value pairs from the multimap, leaving it {@linkplain #isEmpty empty}. */ + void clear(); + + // Views + + /** + * Returns a view collection of the values associated with {@code key} in this multimap, if any. + * Note that when {@code containsKey(key)} is false, this returns an empty collection, not {@code + * null}. + * + *

Changes to the returned collection will update the underlying multimap, and vice versa. + */ + Collection get(K key); + + /** + * Returns a view collection of all distinct keys contained in this multimap. Note that the + * key set contains a key if and only if this multimap maps that key to at least one value. + * + *

Changes to the returned set will update the underlying multimap, and vice versa. However, + * adding to the returned set is not possible. + */ + Set keySet(); + + /** + * Returns a view collection containing the key from each key-value pair in this multimap, + * without collapsing duplicates. This collection has the same size as this multimap, and + * {@code keys().count(k) == get(k).size()} for all {@code k}. + * + *

Changes to the returned multiset will update the underlying multimap, and vice versa. + * However, adding to the returned collection is not possible. + */ + Multiset keys(); + + /** + * Returns a view collection containing the value from each key-value pair contained in + * this multimap, without collapsing duplicates (so {@code values().size() == size()}). + * + *

Changes to the returned collection will update the underlying multimap, and vice versa. + * However, adding to the returned collection is not possible. + */ + Collection values(); + + /** + * Returns a view collection of all key-value pairs contained in this multimap, as {@link Entry} + * instances. + * + *

Changes to the returned collection or the entries it contains will update the underlying + * multimap, and vice versa. However, adding to the returned collection is not possible. + */ + Collection> entries(); + + /** + * Performs the given action for all key-value pairs contained in this multimap. If an ordering is + * specified by the {@code Multimap} implementation, actions will be performed in the order of + * iteration of {@link #entries()}. Exceptions thrown by the action are relayed to the caller. + * + *

To loop over all keys and their associated value collections, write {@code + * Multimaps.asMap(multimap).forEach((key, valueCollection) -> action())}. + * + * @since 21.0 + */ + default void forEach(BiConsumer action) { + checkNotNull(action); + entries().forEach(entry -> action.accept(entry.getKey(), entry.getValue())); + } + + /** + * Returns a view of this multimap as a {@code Map} from each distinct key to the nonempty + * collection of that key's associated values. Note that {@code this.asMap().get(k)} is equivalent + * to {@code this.get(k)} only when {@code k} is a key contained in the multimap; otherwise it + * returns {@code null} as opposed to an empty collection. + * + *

Changes to the returned map or the collections that serve as its values will update the + * underlying multimap, and vice versa. The map does not support {@code put} or {@code putAll}, + * nor do its entries support {@link Entry#setValue setValue}. + */ + Map> asMap(); + + // Comparison and hashing + + /** + * Compares the specified object with this multimap for equality. Two multimaps are equal when + * their map views, as returned by {@link #asMap}, are also equal. + * + *

In general, two multimaps with identical key-value mappings may or may not be equal, + * depending on the implementation. For example, two {@link SetMultimap} instances with the same + * key-value mappings are equal, but equality of two {@link ListMultimap} instances depends on the + * ordering of the values for each key. + * + *

A non-empty {@link SetMultimap} cannot be equal to a non-empty {@link ListMultimap}, since + * their {@link #asMap} views contain unequal collections as values. However, any two empty + * multimaps are equal, because they both have empty {@link #asMap} views. + */ + @Override + boolean equals(Object obj); + + /** + * Returns the hash code for this multimap. + * + *

The hash code of a multimap is defined as the hash code of the map view, as returned by + * {@link Multimap#asMap}. + * + *

In general, two multimaps with identical key-value mappings may or may not have the same + * hash codes, depending on the implementation. For example, two {@link SetMultimap} instances + * with the same key-value mappings will have the same {@code hashCode}, but the {@code hashCode} + * of {@link ListMultimap} instances depends on the ordering of the values for each key. + */ + @Override + int hashCode(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MultimapBuilder.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MultimapBuilder.java new file mode 100644 index 0000000000000..484760c487c33 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MultimapBuilder.java @@ -0,0 +1,475 @@ +/* + * Copyright (C) 2013 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Supplier; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Comparator; +import java.util.EnumMap; +import java.util.EnumSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; + +/** + * A builder for a multimap implementation that allows customization of the backing map and value + * collection implementations used in a particular multimap. + * + *

This can be used to easily configure multimap data structure implementations not provided + * explicitly in {@code com.google.common.collect}, for example: + * + *

{@code
+ * ListMultimap treeListMultimap =
+ *     MultimapBuilder.treeKeys().arrayListValues().build();
+ * SetMultimap hashEnumMultimap =
+ *     MultimapBuilder.hashKeys().enumSetValues(MyEnum.class).build();
+ * }
+ * + *

{@code MultimapBuilder} instances are immutable. Invoking a configuration method has no effect + * on the receiving instance; you must store and use the new builder instance it returns instead. + * + *

The generated multimaps are serializable if the key and value types are serializable, unless + * stated otherwise in one of the configuration methods. + * + * @author Louis Wasserman + * @param An upper bound on the key type of the generated multimap. + * @param An upper bound on the value type of the generated multimap. + * @since 16.0 + */ + +public abstract class MultimapBuilder { + /* + * Leaving K and V as upper bounds rather than the actual key and value types allows type + * parameters to be left implicit more often. CacheBuilder uses the same technique. + */ + + private MultimapBuilder() {} + + private static final int DEFAULT_EXPECTED_KEYS = 8; + + /** Uses a hash table to map keys to value collections. */ + public static MultimapBuilderWithKeys hashKeys() { + return hashKeys(DEFAULT_EXPECTED_KEYS); + } + + /** + * Uses a hash table to map keys to value collections, initialized to expect the specified + * number of keys. + * + * @throws IllegalArgumentException if {@code expectedKeys < 0} + */ + public static MultimapBuilderWithKeys hashKeys(final int expectedKeys) { + checkNonnegative(expectedKeys, "expectedKeys"); + return new MultimapBuilderWithKeys() { + @Override + Map> createMap() { + return Platform.newHashMapWithExpectedSize(expectedKeys); + } + }; + } + + /** + * Uses a hash table to map keys to value collections. + * + *

The collections returned by {@link Multimap#keySet()}, {@link Multimap#keys()}, and {@link + * Multimap#asMap()} will iterate through the keys in the order that they were first added to the + * multimap, save that if all values associated with a key are removed and then the key is added + * back into the multimap, that key will come last in the key iteration order. + */ + public static MultimapBuilderWithKeys linkedHashKeys() { + return linkedHashKeys(DEFAULT_EXPECTED_KEYS); + } + + /** + * Uses an hash table to map keys to value collections, initialized to expect the + * specified number of keys. + * + *

The collections returned by {@link Multimap#keySet()}, {@link Multimap#keys()}, and {@link + * Multimap#asMap()} will iterate through the keys in the order that they were first added to the + * multimap, save that if all values associated with a key are removed and then the key is added + * back into the multimap, that key will come last in the key iteration order. + */ + public static MultimapBuilderWithKeys linkedHashKeys(final int expectedKeys) { + checkNonnegative(expectedKeys, "expectedKeys"); + return new MultimapBuilderWithKeys() { + @Override + Map> createMap() { + return Platform.newLinkedHashMapWithExpectedSize(expectedKeys); + } + }; + } + + /** + * Uses a naturally-ordered {@link TreeMap} to map keys to value collections. + * + *

The collections returned by {@link Multimap#keySet()}, {@link Multimap#keys()}, and {@link + * Multimap#asMap()} will iterate through the keys in sorted order. + * + *

For all multimaps generated by the resulting builder, the {@link Multimap#keySet()} can be + * safely cast to a {@link SortedSet}, and the {@link Multimap#asMap()} can safely be + * cast to a {@link java.util.SortedMap}. + */ + @SuppressWarnings("rawtypes") + public static MultimapBuilderWithKeys treeKeys() { + return treeKeys(Ordering.natural()); + } + + /** + * Uses a {@link TreeMap} sorted by the specified comparator to map keys to value collections. + * + *

The collections returned by {@link Multimap#keySet()}, {@link Multimap#keys()}, and {@link + * Multimap#asMap()} will iterate through the keys in sorted order. + * + *

For all multimaps generated by the resulting builder, the {@link Multimap#keySet()} can be + * safely cast to a {@link SortedSet}, and the {@link Multimap#asMap()} can safely be + * cast to a {@link java.util.SortedMap}. + * + *

Multimaps generated by the resulting builder will not be serializable if {@code comparator} + * is not serializable. + */ + public static MultimapBuilderWithKeys treeKeys(final Comparator comparator) { + checkNotNull(comparator); + return new MultimapBuilderWithKeys() { + @Override + Map> createMap() { + return new TreeMap<>(comparator); + } + }; + } + + /** + * Uses an {@link EnumMap} to map keys to value collections. + * + * @since 16.0 + */ + public static > MultimapBuilderWithKeys enumKeys( + final Class keyClass) { + checkNotNull(keyClass); + return new MultimapBuilderWithKeys() { + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + Map> createMap() { + // K must actually be K0, since enums are effectively final + // (their subclasses are inaccessible) + return (Map>) new EnumMap>(keyClass); + } + }; + } + + private static final class ArrayListSupplier implements Supplier>, Serializable { + private final int expectedValuesPerKey; + + ArrayListSupplier(int expectedValuesPerKey) { + this.expectedValuesPerKey = checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + } + + @Override + public List get() { + return new ArrayList(expectedValuesPerKey); + } + } + + private enum LinkedListSupplier implements Supplier> { + INSTANCE; + + public static Supplier> instance() { + // Each call generates a fresh LinkedList, which can serve as a List for any V. + @SuppressWarnings({"rawtypes", "unchecked"}) + Supplier> result = (Supplier) INSTANCE; + return result; + } + + @Override + public List get() { + return new LinkedList<>(); + } + } + + private static final class HashSetSupplier implements Supplier>, Serializable { + private final int expectedValuesPerKey; + + HashSetSupplier(int expectedValuesPerKey) { + this.expectedValuesPerKey = checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + } + + @Override + public Set get() { + return Platform.newHashSetWithExpectedSize(expectedValuesPerKey); + } + } + + private static final class LinkedHashSetSupplier implements Supplier>, Serializable { + private final int expectedValuesPerKey; + + LinkedHashSetSupplier(int expectedValuesPerKey) { + this.expectedValuesPerKey = checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + } + + @Override + public Set get() { + return Platform.newLinkedHashSetWithExpectedSize(expectedValuesPerKey); + } + } + + private static final class TreeSetSupplier implements Supplier>, Serializable { + private final Comparator comparator; + + TreeSetSupplier(Comparator comparator) { + this.comparator = checkNotNull(comparator); + } + + @Override + public SortedSet get() { + return new TreeSet(comparator); + } + } + + private static final class EnumSetSupplier> + implements Supplier>, Serializable { + private final Class clazz; + + EnumSetSupplier(Class clazz) { + this.clazz = checkNotNull(clazz); + } + + @Override + public Set get() { + return EnumSet.noneOf(clazz); + } + } + + /** + * An intermediate stage in a {@link MultimapBuilder} in which the key-value collection map + * implementation has been specified, but the value collection implementation has not. + * + * @param The upper bound on the key type of the generated multimap. + * @since 16.0 + */ + public abstract static class MultimapBuilderWithKeys { + + private static final int DEFAULT_EXPECTED_VALUES_PER_KEY = 2; + + MultimapBuilderWithKeys() {} + + abstract Map> createMap(); + + /** Uses an {@link ArrayList} to store value collections. */ + public ListMultimapBuilder arrayListValues() { + return arrayListValues(DEFAULT_EXPECTED_VALUES_PER_KEY); + } + + /** + * Uses an {@link ArrayList} to store value collections, initialized to expect the specified + * number of values per key. + * + * @throws IllegalArgumentException if {@code expectedValuesPerKey < 0} + */ + public ListMultimapBuilder arrayListValues(final int expectedValuesPerKey) { + checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + return new ListMultimapBuilder() { + @Override + public ListMultimap build() { + return Multimaps.newListMultimap( + MultimapBuilderWithKeys.this.createMap(), + new ArrayListSupplier(expectedValuesPerKey)); + } + }; + } + + /** Uses a {@link LinkedList} to store value collections. */ + public ListMultimapBuilder linkedListValues() { + return new ListMultimapBuilder() { + @Override + public ListMultimap build() { + return Multimaps.newListMultimap( + MultimapBuilderWithKeys.this.createMap(), LinkedListSupplier.instance()); + } + }; + } + + /** Uses a hash-based {@code Set} to store value collections. */ + public SetMultimapBuilder hashSetValues() { + return hashSetValues(DEFAULT_EXPECTED_VALUES_PER_KEY); + } + + /** + * Uses a hash-based {@code Set} to store value collections, initialized to expect the specified number + * of values per key. + * + * @throws IllegalArgumentException if {@code expectedValuesPerKey < 0} + */ + public SetMultimapBuilder hashSetValues(final int expectedValuesPerKey) { + checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + return new SetMultimapBuilder() { + @Override + public SetMultimap build() { + return Multimaps.newSetMultimap( + MultimapBuilderWithKeys.this.createMap(), + new HashSetSupplier(expectedValuesPerKey)); + } + }; + } + + /** Uses an insertion-ordered hash-based {@code Set} to store value collections. */ + public SetMultimapBuilder linkedHashSetValues() { + return linkedHashSetValues(DEFAULT_EXPECTED_VALUES_PER_KEY); + } + + /** + * Uses an insertion-ordered hash-based {@code Set} to store value collections, initialized to expect the specified + * number of values per key. + * + * @throws IllegalArgumentException if {@code expectedValuesPerKey < 0} + */ + public SetMultimapBuilder linkedHashSetValues(final int expectedValuesPerKey) { + checkNonnegative(expectedValuesPerKey, "expectedValuesPerKey"); + return new SetMultimapBuilder() { + @Override + public SetMultimap build() { + return Multimaps.newSetMultimap( + MultimapBuilderWithKeys.this.createMap(), + new LinkedHashSetSupplier(expectedValuesPerKey)); + } + }; + } + + /** Uses a naturally-ordered {@link TreeSet} to store value collections. */ + @SuppressWarnings("rawtypes") + public SortedSetMultimapBuilder treeSetValues() { + return treeSetValues(Ordering.natural()); + } + + /** + * Uses a {@link TreeSet} ordered by the specified comparator to store value collections. + * + *

Multimaps generated by the resulting builder will not be serializable if {@code + * comparator} is not serializable. + */ + public SortedSetMultimapBuilder treeSetValues(final Comparator comparator) { + checkNotNull(comparator, "comparator"); + return new SortedSetMultimapBuilder() { + @Override + public SortedSetMultimap build() { + return Multimaps.newSortedSetMultimap( + MultimapBuilderWithKeys.this.createMap(), new TreeSetSupplier(comparator)); + } + }; + } + + /** Uses an {@link EnumSet} to store value collections. */ + public > SetMultimapBuilder enumSetValues( + final Class valueClass) { + checkNotNull(valueClass, "valueClass"); + return new SetMultimapBuilder() { + @Override + public SetMultimap build() { + // V must actually be V0, since enums are effectively final + // (their subclasses are inaccessible) + @SuppressWarnings({"unchecked", "rawtypes"}) + Supplier> factory = (Supplier) new EnumSetSupplier(valueClass); + return Multimaps.newSetMultimap(MultimapBuilderWithKeys.this.createMap(), factory); + } + }; + } + } + + /** Returns a new, empty {@code Multimap} with the specified implementation. */ + public abstract Multimap build(); + + /** + * Returns a {@code Multimap} with the specified implementation, initialized with the entries of + * {@code multimap}. + */ + public Multimap build( + Multimap multimap) { + Multimap result = build(); + result.putAll(multimap); + return result; + } + + /** + * A specialization of {@link MultimapBuilder} that generates {@link ListMultimap} instances. + * + * @since 16.0 + */ + public abstract static class ListMultimapBuilder extends MultimapBuilder { + ListMultimapBuilder() {} + + @Override + public abstract ListMultimap build(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public ListMultimap build( + Multimap multimap) { + return (ListMultimap) super.build(multimap); + } + } + + /** + * A specialization of {@link MultimapBuilder} that generates {@link SetMultimap} instances. + * + * @since 16.0 + */ + public abstract static class SetMultimapBuilder extends MultimapBuilder { + SetMultimapBuilder() {} + + @Override + public abstract SetMultimap build(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public SetMultimap build( + Multimap multimap) { + return (SetMultimap) super.build(multimap); + } + } + + /** + * A specialization of {@link MultimapBuilder} that generates {@link SortedSetMultimap} instances. + * + * @since 16.0 + */ + public abstract static class SortedSetMultimapBuilder extends SetMultimapBuilder { + SortedSetMultimapBuilder() {} + + @Override + public abstract SortedSetMultimap build(); + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public SortedSetMultimap build( + Multimap multimap) { + return (SortedSetMultimap) super.build(multimap); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimaps.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimaps.java new file mode 100644 index 0000000000000..6280a01f50e09 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multimaps.java @@ -0,0 +1,2195 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Supplier; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.EntryTransformer; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.AbstractCollection; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableSet; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.SortedSet; +import java.util.Spliterator; +import java.util.function.Consumer; +import java.util.stream.Collector; +import java.util.stream.Stream; + + + +/** + * Provides static methods acting on or generating a {@code Multimap}. + * + *

See the Guava User Guide article on {@code + * Multimaps}. + * + * @author Jared Levy + * @author Robert Konigsberg + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public final class Multimaps { + private Multimaps() {} + + /** + * Returns a {@code Collector} accumulating entries into a {@code Multimap} generated from the + * specified supplier. The keys and values of the entries are the result of applying the provided + * mapping functions to the input elements, accumulated in the encounter order of the stream. + * + *

Example: + * + *

{@code
+   * static final ListMultimap FIRST_LETTER_MULTIMAP =
+   *     Stream.of("banana", "apple", "carrot", "asparagus", "cherry")
+   *         .collect(
+   *             toMultimap(
+   *                  str -> str.charAt(0),
+   *                  str -> str.substring(1),
+   *                  MultimapBuilder.treeKeys().arrayListValues()::build));
+   *
+   * // is equivalent to
+   *
+   * static final ListMultimap FIRST_LETTER_MULTIMAP;
+   *
+   * static {
+   *     FIRST_LETTER_MULTIMAP = MultimapBuilder.treeKeys().arrayListValues().build();
+   *     FIRST_LETTER_MULTIMAP.put('b', "anana");
+   *     FIRST_LETTER_MULTIMAP.put('a', "pple");
+   *     FIRST_LETTER_MULTIMAP.put('a', "sparagus");
+   *     FIRST_LETTER_MULTIMAP.put('c', "arrot");
+   *     FIRST_LETTER_MULTIMAP.put('c', "herry");
+   * }
+   * }
+ * + * @since 21.0 + */ + + public static > Collector toMultimap( + java.util.function.Function keyFunction, + java.util.function.Function valueFunction, + java.util.function.Supplier multimapSupplier) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + checkNotNull(multimapSupplier); + return Collector.of( + multimapSupplier, + (multimap, input) -> multimap.put(keyFunction.apply(input), valueFunction.apply(input)), + (multimap1, multimap2) -> { + multimap1.putAll(multimap2); + return multimap1; + }); + } + + /** + * Returns a {@code Collector} accumulating entries into a {@code Multimap} generated from the + * specified supplier. Each input element is mapped to a key and a stream of values, each of which + * are put into the resulting {@code Multimap}, in the encounter order of the stream and the + * encounter order of the streams of values. + * + *

Example: + * + *

{@code
+   * static final ListMultimap FIRST_LETTER_MULTIMAP =
+   *     Stream.of("banana", "apple", "carrot", "asparagus", "cherry")
+   *         .collect(
+   *             flatteningToMultimap(
+   *                  str -> str.charAt(0),
+   *                  str -> str.substring(1).chars().mapToObj(c -> (char) c),
+   *                  MultimapBuilder.linkedHashKeys().arrayListValues()::build));
+   *
+   * // is equivalent to
+   *
+   * static final ListMultimap FIRST_LETTER_MULTIMAP;
+   *
+   * static {
+   *     FIRST_LETTER_MULTIMAP = MultimapBuilder.linkedHashKeys().arrayListValues().build();
+   *     FIRST_LETTER_MULTIMAP.putAll('b', Arrays.asList('a', 'n', 'a', 'n', 'a'));
+   *     FIRST_LETTER_MULTIMAP.putAll('a', Arrays.asList('p', 'p', 'l', 'e'));
+   *     FIRST_LETTER_MULTIMAP.putAll('c', Arrays.asList('a', 'r', 'r', 'o', 't'));
+   *     FIRST_LETTER_MULTIMAP.putAll('a', Arrays.asList('s', 'p', 'a', 'r', 'a', 'g', 'u', 's'));
+   *     FIRST_LETTER_MULTIMAP.putAll('c', Arrays.asList('h', 'e', 'r', 'r', 'y'));
+   * }
+   * }
+ * + * @since 21.0 + */ + + public static > Collector flatteningToMultimap( + java.util.function.Function keyFunction, + java.util.function.Function> valueFunction, + java.util.function.Supplier multimapSupplier) { + checkNotNull(keyFunction); + checkNotNull(valueFunction); + checkNotNull(multimapSupplier); + return Collector.of( + multimapSupplier, + (multimap, input) -> { + K key = keyFunction.apply(input); + Collection valuesForKey = multimap.get(key); + valueFunction.apply(input).forEachOrdered(valuesForKey::add); + }, + (multimap1, multimap2) -> { + multimap1.putAll(multimap2); + return multimap1; + }); + } + + /** + * Creates a new {@code Multimap} backed by {@code map}, whose internal value collections are + * generated by {@code factory}. + * + *

Warning: do not use this method when the collections returned by {@code factory} + * implement either {@link List} or {@code Set}! Use the more specific method {@link + * #newListMultimap}, {@link #newSetMultimap} or {@link #newSortedSetMultimap} instead, to avoid + * very surprising behavior from {@link Multimap#equals}. + * + *

The {@code factory}-generated and {@code map} classes determine the multimap iteration + * order. They also specify the behavior of the {@code equals}, {@code hashCode}, and {@code + * toString} methods for the multimap and its returned views. However, the multimap's {@code get} + * method returns instances of a different class than {@code factory.get()} does. + * + *

The multimap is serializable if {@code map}, {@code factory}, the collections generated by + * {@code factory}, and the multimap contents are all serializable. + * + *

The multimap is not threadsafe when any concurrent operations update the multimap, even if + * {@code map} and the instances generated by {@code factory} are. Concurrent read operations will + * work correctly. To allow concurrent update operations, wrap the multimap with a call to {@link + * #synchronizedMultimap}. + * + *

Call this method only when the simpler methods {@link ArrayListMultimap#create()}, {@link + * HashMultimap#create()}, {@link LinkedHashMultimap#create()}, {@link + * LinkedListMultimap#create()}, {@link TreeMultimap#create()}, and {@link + * TreeMultimap#create(Comparator, Comparator)} won't suffice. + * + *

Note: the multimap assumes complete ownership over of {@code map} and the collections + * returned by {@code factory}. Those objects should not be manually updated and they should not + * use soft, weak, or phantom references. + * + * @param map place to store the mapping from each key to its corresponding values + * @param factory supplier of new, empty collections that will each hold all values for a given + * key + * @throws IllegalArgumentException if {@code map} is not empty + */ + public static Multimap newMultimap( + Map> map, final Supplier> factory) { + return new CustomMultimap<>(map, factory); + } + + private static class CustomMultimap extends AbstractMapBasedMultimap { + transient Supplier> factory; + + CustomMultimap(Map> map, Supplier> factory) { + super(map); + this.factory = checkNotNull(factory); + } + + @Override + Set createKeySet() { + return createMaybeNavigableKeySet(); + } + + @Override + Map> createAsMap() { + return createMaybeNavigableAsMap(); + } + + @Override + protected Collection createCollection() { + return factory.get(); + } + + @Override + Collection unmodifiableCollectionSubclass(Collection collection) { + if (collection instanceof NavigableSet) { + return Sets.unmodifiableNavigableSet((NavigableSet) collection); + } else if (collection instanceof SortedSet) { + return Collections.unmodifiableSortedSet((SortedSet) collection); + } else if (collection instanceof Set) { + return Collections.unmodifiableSet((Set) collection); + } else if (collection instanceof List) { + return Collections.unmodifiableList((List) collection); + } else { + return Collections.unmodifiableCollection(collection); + } + } + + @Override + Collection wrapCollection(K key, Collection collection) { + if (collection instanceof List) { + return wrapList(key, (List) collection, null); + } else if (collection instanceof NavigableSet) { + return new WrappedNavigableSet(key, (NavigableSet) collection, null); + } else if (collection instanceof SortedSet) { + return new WrappedSortedSet(key, (SortedSet) collection, null); + } else if (collection instanceof Set) { + return new WrappedSet(key, (Set) collection); + } else { + return new WrappedCollection(key, collection, null); + } + } + + // can't use Serialization writeMultimap and populateMultimap methods since + // there's no way to generate the empty backing map. + + /** @serialData the factory and the backing map */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(factory); + stream.writeObject(backingMap()); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + factory = (Supplier>) stream.readObject(); + Map> map = (Map>) stream.readObject(); + setMap(map); + } + + private static final long serialVersionUID = 0; + } + + /** + * Creates a new {@code ListMultimap} that uses the provided map and factory. It can generate a + * multimap based on arbitrary {@link Map} and {@link List} classes. + * + *

The {@code factory}-generated and {@code map} classes determine the multimap iteration + * order. They also specify the behavior of the {@code equals}, {@code hashCode}, and {@code + * toString} methods for the multimap and its returned views. The multimap's {@code get}, {@code + * removeAll}, and {@code replaceValues} methods return {@code RandomAccess} lists if the factory + * does. However, the multimap's {@code get} method returns instances of a different class than + * does {@code factory.get()}. + * + *

The multimap is serializable if {@code map}, {@code factory}, the lists generated by {@code + * factory}, and the multimap contents are all serializable. + * + *

The multimap is not threadsafe when any concurrent operations update the multimap, even if + * {@code map} and the instances generated by {@code factory} are. Concurrent read operations will + * work correctly. To allow concurrent update operations, wrap the multimap with a call to {@link + * #synchronizedListMultimap}. + * + *

Call this method only when the simpler methods {@link ArrayListMultimap#create()} and {@link + * LinkedListMultimap#create()} won't suffice. + * + *

Note: the multimap assumes complete ownership over of {@code map} and the lists returned by + * {@code factory}. Those objects should not be manually updated, they should be empty when + * provided, and they should not use soft, weak, or phantom references. + * + * @param map place to store the mapping from each key to its corresponding values + * @param factory supplier of new, empty lists that will each hold all values for a given key + * @throws IllegalArgumentException if {@code map} is not empty + */ + public static ListMultimap newListMultimap( + Map> map, final Supplier> factory) { + return new CustomListMultimap<>(map, factory); + } + + private static class CustomListMultimap extends AbstractListMultimap { + transient Supplier> factory; + + CustomListMultimap(Map> map, Supplier> factory) { + super(map); + this.factory = checkNotNull(factory); + } + + @Override + Set createKeySet() { + return createMaybeNavigableKeySet(); + } + + @Override + Map> createAsMap() { + return createMaybeNavigableAsMap(); + } + + @Override + protected List createCollection() { + return factory.get(); + } + + /** @serialData the factory and the backing map */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(factory); + stream.writeObject(backingMap()); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + factory = (Supplier>) stream.readObject(); + Map> map = (Map>) stream.readObject(); + setMap(map); + } + + private static final long serialVersionUID = 0; + } + + /** + * Creates a new {@code SetMultimap} that uses the provided map and factory. It can generate a + * multimap based on arbitrary {@link Map} and {@link Set} classes. + * + *

The {@code factory}-generated and {@code map} classes determine the multimap iteration + * order. They also specify the behavior of the {@code equals}, {@code hashCode}, and {@code + * toString} methods for the multimap and its returned views. However, the multimap's {@code get} + * method returns instances of a different class than {@code factory.get()} does. + * + *

The multimap is serializable if {@code map}, {@code factory}, the sets generated by {@code + * factory}, and the multimap contents are all serializable. + * + *

The multimap is not threadsafe when any concurrent operations update the multimap, even if + * {@code map} and the instances generated by {@code factory} are. Concurrent read operations will + * work correctly. To allow concurrent update operations, wrap the multimap with a call to {@link + * #synchronizedSetMultimap}. + * + *

Call this method only when the simpler methods {@link HashMultimap#create()}, {@link + * LinkedHashMultimap#create()}, {@link TreeMultimap#create()}, and {@link + * TreeMultimap#create(Comparator, Comparator)} won't suffice. + * + *

Note: the multimap assumes complete ownership over of {@code map} and the sets returned by + * {@code factory}. Those objects should not be manually updated and they should not use soft, + * weak, or phantom references. + * + * @param map place to store the mapping from each key to its corresponding values + * @param factory supplier of new, empty sets that will each hold all values for a given key + * @throws IllegalArgumentException if {@code map} is not empty + */ + public static SetMultimap newSetMultimap( + Map> map, final Supplier> factory) { + return new CustomSetMultimap<>(map, factory); + } + + private static class CustomSetMultimap extends AbstractSetMultimap { + transient Supplier> factory; + + CustomSetMultimap(Map> map, Supplier> factory) { + super(map); + this.factory = checkNotNull(factory); + } + + @Override + Set createKeySet() { + return createMaybeNavigableKeySet(); + } + + @Override + Map> createAsMap() { + return createMaybeNavigableAsMap(); + } + + @Override + protected Set createCollection() { + return factory.get(); + } + + @Override + Collection unmodifiableCollectionSubclass(Collection collection) { + if (collection instanceof NavigableSet) { + return Sets.unmodifiableNavigableSet((NavigableSet) collection); + } else if (collection instanceof SortedSet) { + return Collections.unmodifiableSortedSet((SortedSet) collection); + } else { + return Collections.unmodifiableSet((Set) collection); + } + } + + @Override + Collection wrapCollection(K key, Collection collection) { + if (collection instanceof NavigableSet) { + return new WrappedNavigableSet(key, (NavigableSet) collection, null); + } else if (collection instanceof SortedSet) { + return new WrappedSortedSet(key, (SortedSet) collection, null); + } else { + return new WrappedSet(key, (Set) collection); + } + } + + /** @serialData the factory and the backing map */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(factory); + stream.writeObject(backingMap()); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + factory = (Supplier>) stream.readObject(); + Map> map = (Map>) stream.readObject(); + setMap(map); + } + + private static final long serialVersionUID = 0; + } + + /** + * Creates a new {@code SortedSetMultimap} that uses the provided map and factory. It can generate + * a multimap based on arbitrary {@link Map} and {@link SortedSet} classes. + * + *

The {@code factory}-generated and {@code map} classes determine the multimap iteration + * order. They also specify the behavior of the {@code equals}, {@code hashCode}, and {@code + * toString} methods for the multimap and its returned views. However, the multimap's {@code get} + * method returns instances of a different class than {@code factory.get()} does. + * + *

The multimap is serializable if {@code map}, {@code factory}, the sets generated by {@code + * factory}, and the multimap contents are all serializable. + * + *

The multimap is not threadsafe when any concurrent operations update the multimap, even if + * {@code map} and the instances generated by {@code factory} are. Concurrent read operations will + * work correctly. To allow concurrent update operations, wrap the multimap with a call to {@link + * #synchronizedSortedSetMultimap}. + * + *

Call this method only when the simpler methods {@link TreeMultimap#create()} and {@link + * TreeMultimap#create(Comparator, Comparator)} won't suffice. + * + *

Note: the multimap assumes complete ownership over of {@code map} and the sets returned by + * {@code factory}. Those objects should not be manually updated and they should not use soft, + * weak, or phantom references. + * + * @param map place to store the mapping from each key to its corresponding values + * @param factory supplier of new, empty sorted sets that will each hold all values for a given + * key + * @throws IllegalArgumentException if {@code map} is not empty + */ + public static SortedSetMultimap newSortedSetMultimap( + Map> map, final Supplier> factory) { + return new CustomSortedSetMultimap<>(map, factory); + } + + private static class CustomSortedSetMultimap extends AbstractSortedSetMultimap { + transient Supplier> factory; + transient Comparator valueComparator; + + CustomSortedSetMultimap(Map> map, Supplier> factory) { + super(map); + this.factory = checkNotNull(factory); + valueComparator = factory.get().comparator(); + } + + @Override + Set createKeySet() { + return createMaybeNavigableKeySet(); + } + + @Override + Map> createAsMap() { + return createMaybeNavigableAsMap(); + } + + @Override + protected SortedSet createCollection() { + return factory.get(); + } + + @Override + public Comparator valueComparator() { + return valueComparator; + } + + /** @serialData the factory and the backing map */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(factory); + stream.writeObject(backingMap()); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + factory = (Supplier>) stream.readObject(); + valueComparator = factory.get().comparator(); + Map> map = (Map>) stream.readObject(); + setMap(map); + } + + private static final long serialVersionUID = 0; + } + + /** + * Copies each key-value mapping in {@code source} into {@code dest}, with its key and value + * reversed. + * + *

If {@code source} is an {@link ImmutableMultimap}, consider using {@link + * ImmutableMultimap#inverse} instead. + * + * @param source any multimap + * @param dest the multimap to copy into; usually empty + * @return {@code dest} + */ + public static > M invertFrom( + Multimap source, M dest) { + checkNotNull(dest); + for (Entry entry : source.entries()) { + dest.put(entry.getValue(), entry.getKey()); + } + return dest; + } + + /** + * Returns a synchronized (thread-safe) multimap backed by the specified multimap. In order to + * guarantee serial access, it is critical that all access to the backing multimap is + * accomplished through the returned multimap. + * + *

It is imperative that the user manually synchronize on the returned multimap when accessing + * any of its collection views: + * + *

{@code
+   * Multimap multimap = Multimaps.synchronizedMultimap(
+   *     HashMultimap.create());
+   * ...
+   * Collection values = multimap.get(key);  // Needn't be in synchronized block
+   * ...
+   * synchronized (multimap) {  // Synchronizing on multimap, not values!
+   *   Iterator i = values.iterator(); // Must be in synchronized block
+   *   while (i.hasNext()) {
+   *     foo(i.next());
+   *   }
+   * }
+   * }
+ * + *

Failure to follow this advice may result in non-deterministic behavior. + * + *

Note that the generated multimap's {@link Multimap#removeAll} and {@link + * Multimap#replaceValues} methods return collections that aren't synchronized. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param multimap the multimap to be wrapped in a synchronized view + * @return a synchronized view of the specified multimap + */ + public static Multimap synchronizedMultimap(Multimap multimap) { + return Synchronized.multimap(multimap, null); + } + + /** + * Returns an unmodifiable view of the specified multimap. Query operations on the returned + * multimap "read through" to the specified multimap, and attempts to modify the returned + * multimap, either directly or through the multimap's views, result in an {@code + * UnsupportedOperationException}. + * + *

Note that the generated multimap's {@link Multimap#removeAll} and {@link + * Multimap#replaceValues} methods return collections that are modifiable. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param delegate the multimap for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified multimap + */ + public static Multimap unmodifiableMultimap(Multimap delegate) { + if (delegate instanceof UnmodifiableMultimap || delegate instanceof ImmutableMultimap) { + return delegate; + } + return new UnmodifiableMultimap<>(delegate); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static Multimap unmodifiableMultimap(ImmutableMultimap delegate) { + return checkNotNull(delegate); + } + + private static class UnmodifiableMultimap extends ForwardingMultimap + implements Serializable { + final Multimap delegate; + transient Collection> entries; + transient Multiset keys; + transient Set keySet; + transient Collection values; + transient Map> map; + + UnmodifiableMultimap(final Multimap delegate) { + this.delegate = checkNotNull(delegate); + } + + @Override + protected Multimap delegate() { + return delegate; + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + public Map> asMap() { + Map> result = map; + if (result == null) { + result = + map = + Collections.unmodifiableMap( + Maps.transformValues( + delegate.asMap(), + new Function, Collection>() { + @Override + public Collection apply(Collection collection) { + return unmodifiableValueCollection(collection); + } + })); + } + return result; + } + + @Override + public Collection> entries() { + Collection> result = entries; + if (result == null) { + entries = result = unmodifiableEntries(delegate.entries()); + } + return result; + } + + @Override + public Collection get(K key) { + return unmodifiableValueCollection(delegate.get(key)); + } + + @Override + public Multiset keys() { + Multiset result = keys; + if (result == null) { + keys = result = Multisets.unmodifiableMultiset(delegate.keys()); + } + return result; + } + + @Override + public Set keySet() { + Set result = keySet; + if (result == null) { + keySet = result = Collections.unmodifiableSet(delegate.keySet()); + } + return result; + } + + @Override + public boolean put(K key, V value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean putAll(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean putAll(Multimap multimap) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object key, Object value) { + throw new UnsupportedOperationException(); + } + + @Override + public Collection removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public Collection replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public Collection values() { + Collection result = values; + if (result == null) { + values = result = Collections.unmodifiableCollection(delegate.values()); + } + return result; + } + + private static final long serialVersionUID = 0; + } + + private static class UnmodifiableListMultimap extends UnmodifiableMultimap + implements ListMultimap { + UnmodifiableListMultimap(ListMultimap delegate) { + super(delegate); + } + + @Override + public ListMultimap delegate() { + return (ListMultimap) super.delegate(); + } + + @Override + public List get(K key) { + return Collections.unmodifiableList(delegate().get(key)); + } + + @Override + public List removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public List replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + private static final long serialVersionUID = 0; + } + + private static class UnmodifiableSetMultimap extends UnmodifiableMultimap + implements SetMultimap { + UnmodifiableSetMultimap(SetMultimap delegate) { + super(delegate); + } + + @Override + public SetMultimap delegate() { + return (SetMultimap) super.delegate(); + } + + @Override + public Set get(K key) { + /* + * Note that this doesn't return a SortedSet when delegate is a + * SortedSetMultiset, unlike (SortedSet) super.get(). + */ + return Collections.unmodifiableSet(delegate().get(key)); + } + + @Override + public Set> entries() { + return Maps.unmodifiableEntrySet(delegate().entries()); + } + + @Override + public Set removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public Set replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + private static final long serialVersionUID = 0; + } + + private static class UnmodifiableSortedSetMultimap extends UnmodifiableSetMultimap + implements SortedSetMultimap { + UnmodifiableSortedSetMultimap(SortedSetMultimap delegate) { + super(delegate); + } + + @Override + public SortedSetMultimap delegate() { + return (SortedSetMultimap) super.delegate(); + } + + @Override + public SortedSet get(K key) { + return Collections.unmodifiableSortedSet(delegate().get(key)); + } + + @Override + public SortedSet removeAll(Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public SortedSet replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public Comparator valueComparator() { + return delegate().valueComparator(); + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a synchronized (thread-safe) {@code SetMultimap} backed by the specified multimap. + * + *

You must follow the warnings described in {@link #synchronizedMultimap}. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param multimap the multimap to be wrapped + * @return a synchronized view of the specified multimap + */ + public static SetMultimap synchronizedSetMultimap(SetMultimap multimap) { + return Synchronized.setMultimap(multimap, null); + } + + /** + * Returns an unmodifiable view of the specified {@code SetMultimap}. Query operations on the + * returned multimap "read through" to the specified multimap, and attempts to modify the returned + * multimap, either directly or through the multimap's views, result in an {@code + * UnsupportedOperationException}. + * + *

Note that the generated multimap's {@link Multimap#removeAll} and {@link + * Multimap#replaceValues} methods return collections that are modifiable. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param delegate the multimap for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified multimap + */ + public static SetMultimap unmodifiableSetMultimap(SetMultimap delegate) { + if (delegate instanceof UnmodifiableSetMultimap || delegate instanceof ImmutableSetMultimap) { + return delegate; + } + return new UnmodifiableSetMultimap<>(delegate); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static SetMultimap unmodifiableSetMultimap( + ImmutableSetMultimap delegate) { + return checkNotNull(delegate); + } + + /** + * Returns a synchronized (thread-safe) {@code SortedSetMultimap} backed by the specified + * multimap. + * + *

You must follow the warnings described in {@link #synchronizedMultimap}. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param multimap the multimap to be wrapped + * @return a synchronized view of the specified multimap + */ + public static SortedSetMultimap synchronizedSortedSetMultimap( + SortedSetMultimap multimap) { + return Synchronized.sortedSetMultimap(multimap, null); + } + + /** + * Returns an unmodifiable view of the specified {@code SortedSetMultimap}. Query operations on + * the returned multimap "read through" to the specified multimap, and attempts to modify the + * returned multimap, either directly or through the multimap's views, result in an {@code + * UnsupportedOperationException}. + * + *

Note that the generated multimap's {@link Multimap#removeAll} and {@link + * Multimap#replaceValues} methods return collections that are modifiable. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param delegate the multimap for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified multimap + */ + public static SortedSetMultimap unmodifiableSortedSetMultimap( + SortedSetMultimap delegate) { + if (delegate instanceof UnmodifiableSortedSetMultimap) { + return delegate; + } + return new UnmodifiableSortedSetMultimap<>(delegate); + } + + /** + * Returns a synchronized (thread-safe) {@code ListMultimap} backed by the specified multimap. + * + *

You must follow the warnings described in {@link #synchronizedMultimap}. + * + * @param multimap the multimap to be wrapped + * @return a synchronized view of the specified multimap + */ + public static ListMultimap synchronizedListMultimap(ListMultimap multimap) { + return Synchronized.listMultimap(multimap, null); + } + + /** + * Returns an unmodifiable view of the specified {@code ListMultimap}. Query operations on the + * returned multimap "read through" to the specified multimap, and attempts to modify the returned + * multimap, either directly or through the multimap's views, result in an {@code + * UnsupportedOperationException}. + * + *

Note that the generated multimap's {@link Multimap#removeAll} and {@link + * Multimap#replaceValues} methods return collections that are modifiable. + * + *

The returned multimap will be serializable if the specified multimap is serializable. + * + * @param delegate the multimap for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified multimap + */ + public static ListMultimap unmodifiableListMultimap(ListMultimap delegate) { + if (delegate instanceof UnmodifiableListMultimap || delegate instanceof ImmutableListMultimap) { + return delegate; + } + return new UnmodifiableListMultimap<>(delegate); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static ListMultimap unmodifiableListMultimap( + ImmutableListMultimap delegate) { + return checkNotNull(delegate); + } + + /** + * Returns an unmodifiable view of the specified collection, preserving the interface for + * instances of {@code SortedSet}, {@code Set}, {@code List} and {@code Collection}, in that order + * of preference. + * + * @param collection the collection for which to return an unmodifiable view + * @return an unmodifiable view of the collection + */ + private static Collection unmodifiableValueCollection(Collection collection) { + if (collection instanceof SortedSet) { + return Collections.unmodifiableSortedSet((SortedSet) collection); + } else if (collection instanceof Set) { + return Collections.unmodifiableSet((Set) collection); + } else if (collection instanceof List) { + return Collections.unmodifiableList((List) collection); + } + return Collections.unmodifiableCollection(collection); + } + + /** + * Returns an unmodifiable view of the specified collection of entries. The {@link Entry#setValue} + * operation throws an {@link UnsupportedOperationException}. If the specified collection is a + * {@code Set}, the returned collection is also a {@code Set}. + * + * @param entries the entries for which to return an unmodifiable view + * @return an unmodifiable view of the entries + */ + private static Collection> unmodifiableEntries( + Collection> entries) { + if (entries instanceof Set) { + return Maps.unmodifiableEntrySet((Set>) entries); + } + return new Maps.UnmodifiableEntries<>(Collections.unmodifiableCollection(entries)); + } + + /** + * Returns {@link ListMultimap#asMap multimap.asMap()}, with its type corrected from {@code Map>} to {@code Map>}. + * + * @since 15.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + // safe by specification of ListMultimap.asMap() + public static Map> asMap(ListMultimap multimap) { + return (Map>) (Map) multimap.asMap(); + } + + /** + * Returns {@link SetMultimap#asMap multimap.asMap()}, with its type corrected from {@code Map>} to {@code Map>}. + * + * @since 15.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + // safe by specification of SetMultimap.asMap() + public static Map> asMap(SetMultimap multimap) { + return (Map>) (Map) multimap.asMap(); + } + + /** + * Returns {@link SortedSetMultimap#asMap multimap.asMap()}, with its type corrected from {@code + * Map>} to {@code Map>}. + * + * @since 15.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + // safe by specification of SortedSetMultimap.asMap() + public static Map> asMap(SortedSetMultimap multimap) { + return (Map>) (Map) multimap.asMap(); + } + + /** + * Returns {@link Multimap#asMap multimap.asMap()}. This is provided for parity with the other + * more strongly-typed {@code asMap()} implementations. + * + * @since 15.0 + */ + + public static Map> asMap(Multimap multimap) { + return multimap.asMap(); + } + + /** + * Returns a multimap view of the specified map. The multimap is backed by the map, so changes to + * the map are reflected in the multimap, and vice versa. If the map is modified while an + * iteration over one of the multimap's collection views is in progress (except through the + * iterator's own {@code remove} operation, or through the {@code setValue} operation on a map + * entry returned by the iterator), the results of the iteration are undefined. + * + *

The multimap supports mapping removal, which removes the corresponding mapping from the map. + * It does not support any operations which might add mappings, such as {@code put}, {@code + * putAll} or {@code replaceValues}. + * + *

The returned multimap will be serializable if the specified map is serializable. + * + * @param map the backing map for the returned multimap view + */ + public static SetMultimap forMap(Map map) { + return new MapMultimap<>(map); + } + + /** @see Multimaps#forMap */ + private static class MapMultimap extends AbstractMultimap + implements SetMultimap, Serializable { + final Map map; + + MapMultimap(Map map) { + this.map = checkNotNull(map); + } + + @Override + public int size() { + return map.size(); + } + + @Override + public boolean containsKey(Object key) { + return map.containsKey(key); + } + + @Override + public boolean containsValue(Object value) { + return map.containsValue(value); + } + + @Override + public boolean containsEntry(Object key, Object value) { + return map.entrySet().contains(Maps.immutableEntry(key, value)); + } + + @Override + public Set get(final K key) { + return new Sets.ImprovedAbstractSet() { + @Override + public Iterator iterator() { + return new Iterator() { + int i; + + @Override + public boolean hasNext() { + return (i == 0) && map.containsKey(key); + } + + @Override + public V next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + i++; + return map.get(key); + } + + @Override + public void remove() { + checkRemove(i == 1); + i = -1; + map.remove(key); + } + }; + } + + @Override + public int size() { + return map.containsKey(key) ? 1 : 0; + } + }; + } + + @Override + public boolean put(K key, V value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean putAll(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean putAll(Multimap multimap) { + throw new UnsupportedOperationException(); + } + + @Override + public Set replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object key, Object value) { + return map.entrySet().remove(Maps.immutableEntry(key, value)); + } + + @Override + public Set removeAll(Object key) { + Set values = new HashSet(2); + if (!map.containsKey(key)) { + return values; + } + values.add(map.remove(key)); + return values; + } + + @Override + public void clear() { + map.clear(); + } + + @Override + Set createKeySet() { + return map.keySet(); + } + + @Override + Collection createValues() { + return map.values(); + } + + @Override + public Set> entries() { + return map.entrySet(); + } + + @Override + Collection> createEntries() { + throw new AssertionError("unreachable"); + } + + @Override + Multiset createKeys() { + return new Keys(this); + } + + @Override + Iterator> entryIterator() { + return map.entrySet().iterator(); + } + + @Override + Map> createAsMap() { + return new AsMap<>(this); + } + + @Override + public int hashCode() { + return map.hashCode(); + } + + private static final long serialVersionUID = 7845222491160860175L; + } + + /** + * Returns a view of a multimap where each value is transformed by a function. All other + * properties of the multimap, such as iteration order, are left intact. For example, the code: + * + *

{@code
+   * Multimap multimap =
+   *     ImmutableSetMultimap.of("a", 2, "b", -3, "b", -3, "a", 4, "c", 6);
+   * Function square = new Function() {
+   *     public String apply(Integer in) {
+   *       return Integer.toString(in * in);
+   *     }
+   * };
+   * Multimap transformed =
+   *     Multimaps.transformValues(multimap, square);
+   *   System.out.println(transformed);
+   * }
+ * + * ... prints {@code {a=[4, 16], b=[9, 9], c=[36]}}. + * + *

Changes in the underlying multimap are reflected in this view. Conversely, this view + * supports removal operations, and these are reflected in the underlying multimap. + * + *

It's acceptable for the underlying multimap to contain null keys, and even null values + * provided that the function is capable of accepting null input. The transformed multimap might + * contain null values, if the function sometimes gives a null result. + * + *

The returned multimap is not thread-safe or serializable, even if the underlying multimap + * is. The {@code equals} and {@code hashCode} methods of the returned multimap are meaningless, + * since there is not a definition of {@code equals} or {@code hashCode} for general collections, + * and {@code get()} will return a general {@code Collection} as opposed to a {@code List} or a + * {@code Set}. + * + *

The function is applied lazily, invoked when needed. This is necessary for the returned + * multimap to be a view, but it means that the function will be applied many times for bulk + * operations like {@link Multimap#containsValue} and {@code Multimap.toString()}. For this to + * perform well, {@code function} should be fast. To avoid lazy evaluation when the returned + * multimap doesn't need to be a view, copy the returned multimap into a new multimap of your + * choosing. + * + * @since 7.0 + */ + public static Multimap transformValues( + Multimap fromMultimap, final Function function) { + checkNotNull(function); + EntryTransformer transformer = Maps.asEntryTransformer(function); + return transformEntries(fromMultimap, transformer); + } + + /** + * Returns a view of a {@code ListMultimap} where each value is transformed by a function. All + * other properties of the multimap, such as iteration order, are left intact. For example, the + * code: + * + *

{@code
+   * ListMultimap multimap
+   *      = ImmutableListMultimap.of("a", 4, "a", 16, "b", 9);
+   * Function sqrt =
+   *     new Function() {
+   *       public Double apply(Integer in) {
+   *         return Math.sqrt((int) in);
+   *       }
+   *     };
+   * ListMultimap transformed = Multimaps.transformValues(map,
+   *     sqrt);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {a=[2.0, 4.0], b=[3.0]}}. + * + *

Changes in the underlying multimap are reflected in this view. Conversely, this view + * supports removal operations, and these are reflected in the underlying multimap. + * + *

It's acceptable for the underlying multimap to contain null keys, and even null values + * provided that the function is capable of accepting null input. The transformed multimap might + * contain null values, if the function sometimes gives a null result. + * + *

The returned multimap is not thread-safe or serializable, even if the underlying multimap + * is. + * + *

The function is applied lazily, invoked when needed. This is necessary for the returned + * multimap to be a view, but it means that the function will be applied many times for bulk + * operations like {@link Multimap#containsValue} and {@code Multimap.toString()}. For this to + * perform well, {@code function} should be fast. To avoid lazy evaluation when the returned + * multimap doesn't need to be a view, copy the returned multimap into a new multimap of your + * choosing. + * + * @since 7.0 + */ + public static ListMultimap transformValues( + ListMultimap fromMultimap, final Function function) { + checkNotNull(function); + EntryTransformer transformer = Maps.asEntryTransformer(function); + return transformEntries(fromMultimap, transformer); + } + + /** + * Returns a view of a multimap whose values are derived from the original multimap's entries. In + * contrast to {@link #transformValues}, this method's entry-transformation logic may depend on + * the key as well as the value. + * + *

All other properties of the transformed multimap, such as iteration order, are left intact. + * For example, the code: + * + *

{@code
+   * SetMultimap multimap =
+   *     ImmutableSetMultimap.of("a", 1, "a", 4, "b", -6);
+   * EntryTransformer transformer =
+   *     new EntryTransformer() {
+   *       public String transformEntry(String key, Integer value) {
+   *          return (value >= 0) ? key : "no" + key;
+   *       }
+   *     };
+   * Multimap transformed =
+   *     Multimaps.transformEntries(multimap, transformer);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {a=[a, a], b=[nob]}}. + * + *

Changes in the underlying multimap are reflected in this view. Conversely, this view + * supports removal operations, and these are reflected in the underlying multimap. + * + *

It's acceptable for the underlying multimap to contain null keys and null values provided + * that the transformer is capable of accepting null inputs. The transformed multimap might + * contain null values if the transformer sometimes gives a null result. + * + *

The returned multimap is not thread-safe or serializable, even if the underlying multimap + * is. The {@code equals} and {@code hashCode} methods of the returned multimap are meaningless, + * since there is not a definition of {@code equals} or {@code hashCode} for general collections, + * and {@code get()} will return a general {@code Collection} as opposed to a {@code List} or a + * {@code Set}. + * + *

The transformer is applied lazily, invoked when needed. This is necessary for the returned + * multimap to be a view, but it means that the transformer will be applied many times for bulk + * operations like {@link Multimap#containsValue} and {@link Object#toString}. For this to perform + * well, {@code transformer} should be fast. To avoid lazy evaluation when the returned multimap + * doesn't need to be a view, copy the returned multimap into a new multimap of your choosing. + * + *

Warning: This method assumes that for any instance {@code k} of {@code + * EntryTransformer} key type {@code K}, {@code k.equals(k2)} implies that {@code k2} is also of + * type {@code K}. Using an {@code EntryTransformer} key type for which this may not hold, such as + * {@code ArrayList}, may risk a {@code ClassCastException} when calling methods on the + * transformed multimap. + * + * @since 7.0 + */ + public static Multimap transformEntries( + Multimap fromMap, EntryTransformer transformer) { + return new TransformedEntriesMultimap<>(fromMap, transformer); + } + + /** + * Returns a view of a {@code ListMultimap} whose values are derived from the original multimap's + * entries. In contrast to {@link #transformValues(ListMultimap, Function)}, this method's + * entry-transformation logic may depend on the key as well as the value. + * + *

All other properties of the transformed multimap, such as iteration order, are left intact. + * For example, the code: + * + *

{@code
+   * Multimap multimap =
+   *     ImmutableMultimap.of("a", 1, "a", 4, "b", 6);
+   * EntryTransformer transformer =
+   *     new EntryTransformer() {
+   *       public String transformEntry(String key, Integer value) {
+   *         return key + value;
+   *       }
+   *     };
+   * Multimap transformed =
+   *     Multimaps.transformEntries(multimap, transformer);
+   * System.out.println(transformed);
+   * }
+ * + * ... prints {@code {"a"=["a1", "a4"], "b"=["b6"]}}. + * + *

Changes in the underlying multimap are reflected in this view. Conversely, this view + * supports removal operations, and these are reflected in the underlying multimap. + * + *

It's acceptable for the underlying multimap to contain null keys and null values provided + * that the transformer is capable of accepting null inputs. The transformed multimap might + * contain null values if the transformer sometimes gives a null result. + * + *

The returned multimap is not thread-safe or serializable, even if the underlying multimap + * is. + * + *

The transformer is applied lazily, invoked when needed. This is necessary for the returned + * multimap to be a view, but it means that the transformer will be applied many times for bulk + * operations like {@link Multimap#containsValue} and {@link Object#toString}. For this to perform + * well, {@code transformer} should be fast. To avoid lazy evaluation when the returned multimap + * doesn't need to be a view, copy the returned multimap into a new multimap of your choosing. + * + *

Warning: This method assumes that for any instance {@code k} of {@code + * EntryTransformer} key type {@code K}, {@code k.equals(k2)} implies that {@code k2} is also of + * type {@code K}. Using an {@code EntryTransformer} key type for which this may not hold, such as + * {@code ArrayList}, may risk a {@code ClassCastException} when calling methods on the + * transformed multimap. + * + * @since 7.0 + */ + public static ListMultimap transformEntries( + ListMultimap fromMap, EntryTransformer transformer) { + return new TransformedEntriesListMultimap<>(fromMap, transformer); + } + + private static class TransformedEntriesMultimap extends AbstractMultimap { + final Multimap fromMultimap; + final EntryTransformer transformer; + + TransformedEntriesMultimap( + Multimap fromMultimap, + final EntryTransformer transformer) { + this.fromMultimap = checkNotNull(fromMultimap); + this.transformer = checkNotNull(transformer); + } + + Collection transform(K key, Collection values) { + Function function = Maps.asValueToValueFunction(transformer, key); + if (values instanceof List) { + return Lists.transform((List) values, function); + } else { + return Collections2.transform(values, function); + } + } + + @Override + Map> createAsMap() { + return Maps.transformEntries( + fromMultimap.asMap(), + new EntryTransformer, Collection>() { + @Override + public Collection transformEntry(K key, Collection value) { + return transform(key, value); + } + }); + } + + @Override + public void clear() { + fromMultimap.clear(); + } + + @Override + public boolean containsKey(Object key) { + return fromMultimap.containsKey(key); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + Collection> createEntries() { + return new AbstractMultimap.Entries(); + } + + @Override + Iterator> entryIterator() { + return Iterators.transform( + fromMultimap.entries().iterator(), Maps.asEntryToEntryFunction(transformer)); + } + + @Override + public Collection get(final K key) { + return transform(key, fromMultimap.get(key)); + } + + @Override + public boolean isEmpty() { + return fromMultimap.isEmpty(); + } + + @Override + Set createKeySet() { + return fromMultimap.keySet(); + } + + @Override + Multiset createKeys() { + return fromMultimap.keys(); + } + + @Override + public boolean put(K key, V2 value) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean putAll(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean putAll(Multimap multimap) { + throw new UnsupportedOperationException(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public boolean remove(Object key, Object value) { + return get((K) key).remove(value); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public Collection removeAll(Object key) { + return transform((K) key, fromMultimap.removeAll(key)); + } + + @Override + public Collection replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + + @Override + public int size() { + return fromMultimap.size(); + } + + @Override + Collection createValues() { + return Collections2.transform( + fromMultimap.entries(), Maps.asEntryToValueFunction(transformer)); + } + } + + private static final class TransformedEntriesListMultimap + extends TransformedEntriesMultimap implements ListMultimap { + + TransformedEntriesListMultimap( + ListMultimap fromMultimap, EntryTransformer transformer) { + super(fromMultimap, transformer); + } + + @Override + List transform(K key, Collection values) { + return Lists.transform((List) values, Maps.asValueToValueFunction(transformer, key)); + } + + @Override + public List get(K key) { + return transform(key, fromMultimap.get(key)); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public List removeAll(Object key) { + return transform((K) key, fromMultimap.removeAll(key)); + } + + @Override + public List replaceValues(K key, Iterable values) { + throw new UnsupportedOperationException(); + } + } + + /** + * Creates an index {@code ImmutableListMultimap} that contains the results of applying a + * specified function to each item in an {@code Iterable} of values. Each value will be stored as + * a value in the resulting multimap, yielding a multimap with the same size as the input + * iterable. The key used to store that value in the multimap will be the result of calling the + * function on that value. The resulting multimap is created as an immutable snapshot. In the + * returned multimap, keys appear in the order they are first encountered, and the values + * corresponding to each key appear in the same order as they are encountered. + * + *

For example, + * + *

{@code
+   * List badGuys =
+   *     Arrays.asList("Inky", "Blinky", "Pinky", "Pinky", "Clyde");
+   * Function stringLengthFunction = ...;
+   * Multimap index =
+   *     Multimaps.index(badGuys, stringLengthFunction);
+   * System.out.println(index);
+   * }
+ * + *

prints + * + *

{@code
+   * {4=[Inky], 6=[Blinky], 5=[Pinky, Pinky, Clyde]}
+   * }
+ * + *

The returned multimap is serializable if its keys and values are all serializable. + * + * @param values the values to use when constructing the {@code ImmutableListMultimap} + * @param keyFunction the function used to produce the key for each value + * @return {@code ImmutableListMultimap} mapping the result of evaluating the function {@code + * keyFunction} on each value in the input collection to that value + * @throws NullPointerException if any element of {@code values} is {@code null}, or if {@code + * keyFunction} produces {@code null} for any key + */ + public static ImmutableListMultimap index( + Iterable values, Function keyFunction) { + return index(values.iterator(), keyFunction); + } + + /** + * Creates an index {@code ImmutableListMultimap} that contains the results of applying a + * specified function to each item in an {@code Iterator} of values. Each value will be stored as + * a value in the resulting multimap, yielding a multimap with the same size as the input + * iterator. The key used to store that value in the multimap will be the result of calling the + * function on that value. The resulting multimap is created as an immutable snapshot. In the + * returned multimap, keys appear in the order they are first encountered, and the values + * corresponding to each key appear in the same order as they are encountered. + * + *

For example, + * + *

{@code
+   * List badGuys =
+   *     Arrays.asList("Inky", "Blinky", "Pinky", "Pinky", "Clyde");
+   * Function stringLengthFunction = ...;
+   * Multimap index =
+   *     Multimaps.index(badGuys.iterator(), stringLengthFunction);
+   * System.out.println(index);
+   * }
+ * + *

prints + * + *

{@code
+   * {4=[Inky], 6=[Blinky], 5=[Pinky, Pinky, Clyde]}
+   * }
+ * + *

The returned multimap is serializable if its keys and values are all serializable. + * + * @param values the values to use when constructing the {@code ImmutableListMultimap} + * @param keyFunction the function used to produce the key for each value + * @return {@code ImmutableListMultimap} mapping the result of evaluating the function {@code + * keyFunction} on each value in the input collection to that value + * @throws NullPointerException if any element of {@code values} is {@code null}, or if {@code + * keyFunction} produces {@code null} for any key + * @since 10.0 + */ + public static ImmutableListMultimap index( + Iterator values, Function keyFunction) { + checkNotNull(keyFunction); + ImmutableListMultimap.Builder builder = ImmutableListMultimap.builder(); + while (values.hasNext()) { + V value = values.next(); + checkNotNull(value, values); + builder.put(keyFunction.apply(value), value); + } + return builder.build(); + } + + static class Keys extends AbstractMultiset { + final Multimap multimap; + + Keys(Multimap multimap) { + this.multimap = multimap; + } + + @Override + Iterator> entryIterator() { + return new TransformedIterator>, Entry>( + multimap.asMap().entrySet().iterator()) { + @Override + Multiset.Entry transform(final Map.Entry> backingEntry) { + return new Multisets.AbstractEntry() { + @Override + public K getElement() { + return backingEntry.getKey(); + } + + @Override + public int getCount() { + return backingEntry.getValue().size(); + } + }; + } + }; + } + + @Override + public Spliterator spliterator() { + return CollectSpliterators.map(multimap.entries().spliterator(), Map.Entry::getKey); + } + + @Override + public void forEach(Consumer consumer) { + checkNotNull(consumer); + multimap.entries().forEach(entry -> consumer.accept(entry.getKey())); + } + + @Override + int distinctElements() { + return multimap.asMap().size(); + } + + @Override + public int size() { + return multimap.size(); + } + + @Override + public boolean contains(Object element) { + return multimap.containsKey(element); + } + + @Override + public Iterator iterator() { + return Maps.keyIterator(multimap.entries().iterator()); + } + + @Override + public int count(Object element) { + Collection values = Maps.safeGet(multimap.asMap(), element); + return (values == null) ? 0 : values.size(); + } + + @Override + public int remove(Object element, int occurrences) { + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(element); + } + + Collection values = Maps.safeGet(multimap.asMap(), element); + + if (values == null) { + return 0; + } + + int oldCount = values.size(); + if (occurrences >= oldCount) { + values.clear(); + } else { + Iterator iterator = values.iterator(); + for (int i = 0; i < occurrences; i++) { + iterator.next(); + iterator.remove(); + } + } + return oldCount; + } + + @Override + public void clear() { + multimap.clear(); + } + + @Override + public Set elementSet() { + return multimap.keySet(); + } + + @Override + Iterator elementIterator() { + throw new AssertionError("should never be called"); + } + } + + /** A skeleton implementation of {@link Multimap#entries()}. */ + abstract static class Entries extends AbstractCollection> { + abstract Multimap multimap(); + + @Override + public int size() { + return multimap().size(); + } + + @Override + public boolean contains(Object o) { + if (o instanceof Map.Entry) { + Entry entry = (Entry) o; + return multimap().containsEntry(entry.getKey(), entry.getValue()); + } + return false; + } + + @Override + public boolean remove(Object o) { + if (o instanceof Map.Entry) { + Entry entry = (Entry) o; + return multimap().remove(entry.getKey(), entry.getValue()); + } + return false; + } + + @Override + public void clear() { + multimap().clear(); + } + } + + /** A skeleton implementation of {@link Multimap#asMap()}. */ + static final class AsMap extends Maps.ViewCachingAbstractMap> { + private final Multimap multimap; + + AsMap(Multimap multimap) { + this.multimap = checkNotNull(multimap); + } + + @Override + public int size() { + return multimap.keySet().size(); + } + + @Override + protected Set>> createEntrySet() { + return new EntrySet(); + } + + void removeValuesForKey(Object key) { + multimap.keySet().remove(key); + } + + class EntrySet extends Maps.EntrySet> { + @Override + Map> map() { + return AsMap.this; + } + + @Override + public Iterator>> iterator() { + return Maps.asMapEntryIterator( + multimap.keySet(), + new Function>() { + @Override + public Collection apply(K key) { + return multimap.get(key); + } + }); + } + + @Override + public boolean remove(Object o) { + if (!contains(o)) { + return false; + } + Entry entry = (Entry) o; + removeValuesForKey(entry.getKey()); + return true; + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public Collection get(Object key) { + return containsKey(key) ? multimap.get((K) key) : null; + } + + @Override + public Collection remove(Object key) { + return containsKey(key) ? multimap.removeAll(key) : null; + } + + @Override + public Set keySet() { + return multimap.keySet(); + } + + @Override + public boolean isEmpty() { + return multimap.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + return multimap.containsKey(key); + } + + @Override + public void clear() { + multimap.clear(); + } + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} whose keys satisfy a + * predicate. The returned multimap is a live view of {@code unfiltered}; changes to one affect + * the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a key that doesn't + * satisfy the predicate, the multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose keys satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code keyPredicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 11.0 + */ + public static Multimap filterKeys( + Multimap unfiltered, final Predicate keyPredicate) { + if (unfiltered instanceof SetMultimap) { + return filterKeys((SetMultimap) unfiltered, keyPredicate); + } else if (unfiltered instanceof ListMultimap) { + return filterKeys((ListMultimap) unfiltered, keyPredicate); + } else if (unfiltered instanceof FilteredKeyMultimap) { + FilteredKeyMultimap prev = (FilteredKeyMultimap) unfiltered; + return new FilteredKeyMultimap<>( + prev.unfiltered, Predicates.and(prev.keyPredicate, keyPredicate)); + } else if (unfiltered instanceof FilteredMultimap) { + FilteredMultimap prev = (FilteredMultimap) unfiltered; + return filterFiltered(prev, Maps.keyPredicateOnEntries(keyPredicate)); + } else { + return new FilteredKeyMultimap<>(unfiltered, keyPredicate); + } + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} whose keys satisfy a + * predicate. The returned multimap is a live view of {@code unfiltered}; changes to one affect + * the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a key that doesn't + * satisfy the predicate, the multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose keys satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code keyPredicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 14.0 + */ + public static SetMultimap filterKeys( + SetMultimap unfiltered, final Predicate keyPredicate) { + if (unfiltered instanceof FilteredKeySetMultimap) { + FilteredKeySetMultimap prev = (FilteredKeySetMultimap) unfiltered; + return new FilteredKeySetMultimap<>( + prev.unfiltered(), Predicates.and(prev.keyPredicate, keyPredicate)); + } else if (unfiltered instanceof FilteredSetMultimap) { + FilteredSetMultimap prev = (FilteredSetMultimap) unfiltered; + return filterFiltered(prev, Maps.keyPredicateOnEntries(keyPredicate)); + } else { + return new FilteredKeySetMultimap<>(unfiltered, keyPredicate); + } + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} whose keys satisfy a + * predicate. The returned multimap is a live view of {@code unfiltered}; changes to one affect + * the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a key that doesn't + * satisfy the predicate, the multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose keys satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code keyPredicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 14.0 + */ + public static ListMultimap filterKeys( + ListMultimap unfiltered, final Predicate keyPredicate) { + if (unfiltered instanceof FilteredKeyListMultimap) { + FilteredKeyListMultimap prev = (FilteredKeyListMultimap) unfiltered; + return new FilteredKeyListMultimap<>( + prev.unfiltered(), Predicates.and(prev.keyPredicate, keyPredicate)); + } else { + return new FilteredKeyListMultimap<>(unfiltered, keyPredicate); + } + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} whose values satisfy a + * predicate. The returned multimap is a live view of {@code unfiltered}; changes to one affect + * the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a value that doesn't + * satisfy the predicate, the multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose value satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code valuePredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 11.0 + */ + public static Multimap filterValues( + Multimap unfiltered, final Predicate valuePredicate) { + return filterEntries(unfiltered, Maps.valuePredicateOnEntries(valuePredicate)); + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} whose values satisfy a + * predicate. The returned multimap is a live view of {@code unfiltered}; changes to one affect + * the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a value that doesn't + * satisfy the predicate, the multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose value satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code valuePredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. + * + * @since 14.0 + */ + public static SetMultimap filterValues( + SetMultimap unfiltered, final Predicate valuePredicate) { + return filterEntries(unfiltered, Maps.valuePredicateOnEntries(valuePredicate)); + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} that satisfy a predicate. The + * returned multimap is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a key/value pair that + * doesn't satisfy the predicate, multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose keys satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. + * + * @since 11.0 + */ + public static Multimap filterEntries( + Multimap unfiltered, Predicate> entryPredicate) { + checkNotNull(entryPredicate); + if (unfiltered instanceof SetMultimap) { + return filterEntries((SetMultimap) unfiltered, entryPredicate); + } + return (unfiltered instanceof FilteredMultimap) + ? filterFiltered((FilteredMultimap) unfiltered, entryPredicate) + : new FilteredEntryMultimap(checkNotNull(unfiltered), entryPredicate); + } + + /** + * Returns a multimap containing the mappings in {@code unfiltered} that satisfy a predicate. The + * returned multimap is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting multimap's views have iterators that don't support {@code remove()}, but all + * other methods are supported by the multimap and its views. When adding a key/value pair that + * doesn't satisfy the predicate, multimap's {@code put()}, {@code putAll()}, and {@code + * replaceValues()} methods throw an {@link IllegalArgumentException}. + * + *

When methods such as {@code removeAll()} and {@code clear()} are called on the filtered + * multimap or its views, only mappings whose keys satisfy the filter will be removed from the + * underlying multimap. + * + *

The returned multimap isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multimap's methods, such as {@code size()}, iterate across every + * key/value mapping in the underlying multimap and determine which satisfy the filter. When a + * live view is not needed, it may be faster to copy the filtered multimap and use the + * copy. + * + *

Warning: {@code entryPredicate} must be consistent with equals, as documented + * at {@link Predicate#apply}. + * + * @since 14.0 + */ + public static SetMultimap filterEntries( + SetMultimap unfiltered, Predicate> entryPredicate) { + checkNotNull(entryPredicate); + return (unfiltered instanceof FilteredSetMultimap) + ? filterFiltered((FilteredSetMultimap) unfiltered, entryPredicate) + : new FilteredEntrySetMultimap(checkNotNull(unfiltered), entryPredicate); + } + + /** + * Support removal operations when filtering a filtered multimap. Since a filtered multimap has + * iterators that don't support remove, passing one to the FilteredEntryMultimap constructor would + * lead to a multimap whose removal operations would fail. This method combines the predicates to + * avoid that problem. + */ + private static Multimap filterFiltered( + FilteredMultimap multimap, Predicate> entryPredicate) { + Predicate> predicate = + Predicates.>and(multimap.entryPredicate(), entryPredicate); + return new FilteredEntryMultimap<>(multimap.unfiltered(), predicate); + } + + /** + * Support removal operations when filtering a filtered multimap. Since a filtered multimap has + * iterators that don't support remove, passing one to the FilteredEntryMultimap constructor would + * lead to a multimap whose removal operations would fail. This method combines the predicates to + * avoid that problem. + */ + private static SetMultimap filterFiltered( + FilteredSetMultimap multimap, Predicate> entryPredicate) { + Predicate> predicate = + Predicates.>and(multimap.entryPredicate(), entryPredicate); + return new FilteredEntrySetMultimap<>(multimap.unfiltered(), predicate); + } + + static boolean equalsImpl(Multimap multimap, Object object) { + if (object == multimap) { + return true; + } + if (object instanceof Multimap) { + Multimap that = (Multimap) object; + return multimap.asMap().equals(that.asMap()); + } + return false; + } + + // TODO(jlevy): Create methods that filter a SortedSetMultimap. +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multiset.java new file mode 100644 index 0000000000000..556de6da8cbc2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multiset.java @@ -0,0 +1,467 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.Spliterator; +import java.util.function.Consumer; +import java.util.function.ObjIntConsumer; + + +/** + * A collection that supports order-independent equality, like {@link Set}, but may have duplicate + * elements. A multiset is also sometimes called a bag. + * + *

Elements of a multiset that are equal to one another are referred to as occurrences of + * the same single element. The total number of occurrences of an element in a multiset is called + * the count of that element (the terms "frequency" and "multiplicity" are equivalent, but + * not used in this API). Since the count of an element is represented as an {@code int}, a multiset + * may never contain more than {@link Integer#MAX_VALUE} occurrences of any one element. + * + *

{@code Multiset} refines the specifications of several methods from {@code Collection}. It + * also defines an additional query operation, {@link #count}, which returns the count of an + * element. There are five new bulk-modification operations, for example {@link #add(Object, int)}, + * to add or remove multiple occurrences of an element at once, or to set the count of an element to + * a specific value. These modification operations are optional, but implementations which support + * the standard collection operations {@link #add(Object)} or {@link #remove(Object)} are encouraged + * to implement the related methods as well. Finally, two collection views are provided: {@link + * #elementSet} contains the distinct elements of the multiset "with duplicates collapsed", and + * {@link #entrySet} is similar but contains {@link Entry Multiset.Entry} instances, each providing + * both a distinct element and the count of that element. + * + *

In addition to these required methods, implementations of {@code Multiset} are expected to + * provide two {@code static} creation methods: {@code create()}, returning an empty multiset, and + * {@code create(Iterable)}, returning a multiset containing the given initial + * elements. This is simply a refinement of {@code Collection}'s constructor recommendations, + * reflecting the new developments of Java 5. + * + *

As with other collection types, the modification operations are optional, and should throw + * {@link UnsupportedOperationException} when they are not implemented. Most implementations should + * support either all add operations or none of them, all removal operations or none of them, and if + * and only if all of these are supported, the {@code setCount} methods as well. + * + *

A multiset uses {@link Object#equals} to determine whether two instances should be considered + * "the same," unless specified otherwise by the implementation. + * + *

Common implementations include {@link ImmutableMultiset}, {@link HashMultiset}, and {@link + * ConcurrentHashMultiset}. + * + *

If your values may be zero, negative, or outside the range of an int, you may wish to use + * {@link com.azure.cosmos.util.concurrent.AtomicLongMap} instead. Note, however, that unlike + * {@code Multiset}, {@code AtomicLongMap} does not automatically remove zeros. + * + *

See the Guava User Guide article on {@code + * Multiset}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public interface Multiset extends Collection { + // Query Operations + + /** + * Returns the total number of all occurrences of all elements in this multiset. + * + *

Note: this method does not return the number of distinct elements in the + * multiset, which is given by {@code entrySet().size()}. + */ + @Override + int size(); + + /** + * Returns the number of occurrences of an element in this multiset (the count of the + * element). Note that for an {@link Object#equals}-based multiset, this gives the same result as + * {@link Collections#frequency} (which would presumably perform more poorly). + * + *

Note: the utility method {@link Iterables#frequency} generalizes this operation; it + * correctly delegates to this method when dealing with a multiset, but it can also accept any + * other iterable type. + * + * @param element the element to count occurrences of + * @return the number of occurrences of the element in this multiset; possibly zero but never + * negative + */ + int count(/*@CompatibleWith("E")*/ Object element); + + // Bulk Operations + + /** + * Adds a number of occurrences of an element to this multiset. Note that if {@code occurrences == + * 1}, this method has the identical effect to {@link #add(Object)}. This method is functionally + * equivalent (except in the case of overflow) to the call {@code + * addAll(Collections.nCopies(element, occurrences))}, which would presumably perform much more + * poorly. + * + * @param element the element to add occurrences of; may be null only if explicitly allowed by the + * implementation + * @param occurrences the number of occurrences of the element to add. May be zero, in which case + * no change will be made. + * @return the count of the element before the operation; possibly zero + * @throws IllegalArgumentException if {@code occurrences} is negative, or if this operation would + * result in more than {@link Integer#MAX_VALUE} occurrences of the element + * @throws NullPointerException if {@code element} is null and this implementation does not permit + * null elements. Note that if {@code occurrences} is zero, the implementation may opt to + * return normally. + */ + int add(E element, int occurrences); + + /** + * Adds a single occurrence of the specified element to this multiset. + * + *

This method refines {@link Collection#add}, which only ensures the presence of the + * element, to further specify that a successful call must always increment the count of the + * element, and the overall size of the collection, by one. + * + *

To both add the element and obtain the previous count of that element, use {@link + * #add(Object, int) add}{@code (element, 1)} instead. + * + * @param element the element to add one occurrence of; may be null only if explicitly allowed by + * the implementation + * @return {@code true} always, since this call is required to modify the multiset, unlike other + * {@link Collection} types + * @throws NullPointerException if {@code element} is null and this implementation does not permit + * null elements + * @throws IllegalArgumentException if {@link Integer#MAX_VALUE} occurrences of {@code element} + * are already contained in this multiset + */ + @Override + boolean add(E element); + + /** + * Removes a number of occurrences of the specified element from this multiset. If the multiset + * contains fewer than this number of occurrences to begin with, all occurrences will be removed. + * Note that if {@code occurrences == 1}, this is functionally equivalent to the call {@code + * remove(element)}. + * + * @param element the element to conditionally remove occurrences of + * @param occurrences the number of occurrences of the element to remove. May be zero, in which + * case no change will be made. + * @return the count of the element before the operation; possibly zero + * @throws IllegalArgumentException if {@code occurrences} is negative + */ + int remove(/*@CompatibleWith("E")*/ Object element, int occurrences); + + /** + * Removes a single occurrence of the specified element from this multiset, if present. + * + *

This method refines {@link Collection#remove} to further specify that it may not + * throw an exception in response to {@code element} being null or of the wrong type. + * + *

To both remove the element and obtain the previous count of that element, use {@link + * #remove(Object, int) remove}{@code (element, 1)} instead. + * + * @param element the element to remove one occurrence of + * @return {@code true} if an occurrence was found and removed + */ + @Override + boolean remove(Object element); + + /** + * Adds or removes the necessary occurrences of an element such that the element attains the + * desired count. + * + * @param element the element to add or remove occurrences of; may be null only if explicitly + * allowed by the implementation + * @param count the desired count of the element in this multiset + * @return the count of the element before the operation; possibly zero + * @throws IllegalArgumentException if {@code count} is negative + * @throws NullPointerException if {@code element} is null and this implementation does not permit + * null elements. Note that if {@code count} is zero, the implementor may optionally return + * zero instead. + */ + int setCount(E element, int count); + + /** + * Conditionally sets the count of an element to a new value, as described in {@link + * #setCount(Object, int)}, provided that the element has the expected current count. If the + * current count is not {@code oldCount}, no change is made. + * + * @param element the element to conditionally set the count of; may be null only if explicitly + * allowed by the implementation + * @param oldCount the expected present count of the element in this multiset + * @param newCount the desired count of the element in this multiset + * @return {@code true} if the condition for modification was met. This implies that the multiset + * was indeed modified, unless {@code oldCount == newCount}. + * @throws IllegalArgumentException if {@code oldCount} or {@code newCount} is negative + * @throws NullPointerException if {@code element} is null and the implementation does not permit + * null elements. Note that if {@code oldCount} and {@code newCount} are both zero, the + * implementor may optionally return {@code true} instead. + */ + boolean setCount(E element, int oldCount, int newCount); + + // Views + + /** + * Returns the set of distinct elements contained in this multiset. The element set is backed by + * the same data as the multiset, so any change to either is immediately reflected in the other. + * The order of the elements in the element set is unspecified. + * + *

If the element set supports any removal operations, these necessarily cause all + * occurrences of the removed element(s) to be removed from the multiset. Implementations are not + * expected to support the add operations, although this is possible. + * + *

A common use for the element set is to find the number of distinct elements in the multiset: + * {@code elementSet().size()}. + * + * @return a view of the set of distinct elements in this multiset + */ + Set elementSet(); + + /** + * Returns a view of the contents of this multiset, grouped into {@code Multiset.Entry} instances, + * each providing an element of the multiset and the count of that element. This set contains + * exactly one entry for each distinct element in the multiset (thus it always has the same size + * as the {@link #elementSet}). The order of the elements in the element set is unspecified. + * + *

The entry set is backed by the same data as the multiset, so any change to either is + * immediately reflected in the other. However, multiset changes may or may not be reflected in + * any {@code Entry} instances already retrieved from the entry set (this is + * implementation-dependent). Furthermore, implementations are not required to support + * modifications to the entry set at all, and the {@code Entry} instances themselves don't even + * have methods for modification. See the specific implementation class for more details on how + * its entry set handles modifications. + * + * @return a set of entries representing the data of this multiset + */ + Set> entrySet(); + + /** + * An unmodifiable element-count pair for a multiset. The {@link Multiset#entrySet} method returns + * a view of the multiset whose elements are of this class. A multiset implementation may return + * Entry instances that are either live "read-through" views to the Multiset, or immutable + * snapshots. Note that this type is unrelated to the similarly-named type {@code Map.Entry}. + * + * @since 2.0 + */ + interface Entry { + + /** + * Returns the multiset element corresponding to this entry. Multiple calls to this method + * always return the same instance. + * + * @return the element corresponding to this entry + */ + E getElement(); + + /** + * Returns the count of the associated element in the underlying multiset. This count may either + * be an unchanging snapshot of the count at the time the entry was retrieved, or a live view of + * the current count of the element in the multiset, depending on the implementation. Note that + * in the former case, this method can never return zero, while in the latter, it will return + * zero if all occurrences of the element were since removed from the multiset. + * + * @return the count of the element; never negative + */ + int getCount(); + + /** + * {@inheritDoc} + * + *

Returns {@code true} if the given object is also a multiset entry and the two entries + * represent the same element and count. That is, two entries {@code a} and {@code b} are equal + * if: + * + *

{@code
+     * Objects.equal(a.getElement(), b.getElement())
+     *     && a.getCount() == b.getCount()
+     * }
+ */ + @Override + // TODO(kevinb): check this wrt TreeMultiset? + boolean equals(Object o); + + /** + * {@inheritDoc} + * + *

The hash code of a multiset entry for element {@code element} and count {@code count} is + * defined as: + * + *

{@code
+     * ((element == null) ? 0 : element.hashCode()) ^ count
+     * }
+ */ + @Override + int hashCode(); + + /** + * Returns the canonical string representation of this entry, defined as follows. If the count + * for this entry is one, this is simply the string representation of the corresponding element. + * Otherwise, it is the string representation of the element, followed by the three characters + * {@code " x "} (space, letter x, space), followed by the count. + */ + @Override + String toString(); + } + + /** + * Runs the specified action for each distinct element in this multiset, and the number of + * occurrences of that element. For some {@code Multiset} implementations, this may be more + * efficient than iterating over the {@link #entrySet()} either explicitly or with {@code + * entrySet().forEach(action)}. + * + * @since 21.0 + */ + + default void forEachEntry(ObjIntConsumer action) { + checkNotNull(action); + entrySet().forEach(entry -> action.accept(entry.getElement(), entry.getCount())); + } + + // Comparison and hashing + + /** + * Compares the specified object with this multiset for equality. Returns {@code true} if the + * given object is also a multiset and contains equal elements with equal counts, regardless of + * order. + */ + @Override + // TODO(kevinb): caveats about equivalence-relation? + boolean equals(Object object); + + /** + * Returns the hash code for this multiset. This is defined as the sum of + * + *
{@code
+   * ((element == null) ? 0 : element.hashCode()) ^ count(element)
+   * }
+ * + *

over all distinct elements in the multiset. It follows that a multiset and its entry set + * always have the same hash code. + */ + @Override + int hashCode(); + + /** + * {@inheritDoc} + * + *

It is recommended, though not mandatory, that this method return the result of invoking + * {@link #toString} on the {@link #entrySet}, yielding a result such as {@code [a x 3, c, d x 2, + * e]}. + */ + @Override + String toString(); + + // Refined Collection Methods + + /** + * {@inheritDoc} + * + *

Elements that occur multiple times in the multiset will appear multiple times in this + * iterator, though not necessarily sequentially. + */ + @Override + Iterator iterator(); + + /** + * Determines whether this multiset contains the specified element. + * + *

This method refines {@link Collection#contains} to further specify that it may not + * throw an exception in response to {@code element} being null or of the wrong type. + * + * @param element the element to check for + * @return {@code true} if this multiset contains at least one occurrence of the element + */ + @Override + boolean contains(Object element); + + /** + * Returns {@code true} if this multiset contains at least one occurrence of each element in the + * specified collection. + * + *

This method refines {@link Collection#containsAll} to further specify that it may not + * throw an exception in response to any of {@code elements} being null or of the wrong type. + * + *

Note: this method does not take into account the occurrence count of an element in + * the two collections; it may still return {@code true} even if {@code elements} contains several + * occurrences of an element and this multiset contains only one. This is no different than any + * other collection type like {@link List}, but it may be unexpected to the user of a multiset. + * + * @param elements the collection of elements to be checked for containment in this multiset + * @return {@code true} if this multiset contains at least one occurrence of each element + * contained in {@code elements} + * @throws NullPointerException if {@code elements} is null + */ + @Override + boolean containsAll(Collection elements); + + /** + * {@inheritDoc} + * + *

Note: This method ignores how often any element might appear in {@code c}, and only + * cares whether or not an element appears at all. If you wish to remove one occurrence in this + * multiset for every occurrence in {@code c}, see {@link Multisets#removeOccurrences(Multiset, + * Multiset)}. + * + *

This method refines {@link Collection#removeAll} to further specify that it may not + * throw an exception in response to any of {@code elements} being null or of the wrong type. + */ + @Override + boolean removeAll(Collection c); + + /** + * {@inheritDoc} + * + *

Note: This method ignores how often any element might appear in {@code c}, and only + * cares whether or not an element appears at all. If you wish to remove one occurrence in this + * multiset for every occurrence in {@code c}, see {@link Multisets#retainOccurrences(Multiset, + * Multiset)}. + * + *

This method refines {@link Collection#retainAll} to further specify that it may not + * throw an exception in response to any of {@code elements} being null or of the wrong type. + * + * @see Multisets#retainOccurrences(Multiset, Multiset) + */ + @Override + boolean retainAll(Collection c); + + /** + * {@inheritDoc} + * + *

Elements that occur multiple times in the multiset will be passed to the {@code Consumer} + * correspondingly many times, though not necessarily sequentially. + */ + @Override + default void forEach(Consumer action) { + checkNotNull(action); + entrySet() + .forEach( + entry -> { + E elem = entry.getElement(); + int count = entry.getCount(); + for (int i = 0; i < count; i++) { + action.accept(elem); + } + }); + } + + @Override + default Spliterator spliterator() { + return Multisets.spliteratorImpl(this); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multisets.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multisets.java new file mode 100644 index 0000000000000..5bfc29da19147 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Multisets.java @@ -0,0 +1,1177 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multiset.Entry; +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.Spliterator; +import java.util.stream.Collector; + + + +/** + * Provides static utility methods for creating and working with {@link Multiset} instances. + * + *

See the Guava User Guide article on {@code + * Multisets}. + * + * @author Kevin Bourrillion + * @author Mike Bostock + * @author Louis Wasserman + * @since 2.0 + */ +public final class Multisets { + private Multisets() {} + + /** + * Returns a {@code Collector} that accumulates elements into a multiset created via the specified + * {@code Supplier}, whose elements are the result of applying {@code elementFunction} to the + * inputs, with counts equal to the result of applying {@code countFunction} to the inputs. + * Elements are added in encounter order. + * + *

If the mapped elements contain duplicates (according to {@link Object#equals}), the element + * will be added more than once, with the count summed over all appearances of the element. + * + *

Note that {@code stream.collect(toMultiset(function, e -> 1, supplier))} is equivalent to + * {@code stream.map(function).collect(Collectors.toCollection(supplier))}. + * + * @since 22.0 + */ + public static > Collector toMultiset( + java.util.function.Function elementFunction, + java.util.function.ToIntFunction countFunction, + java.util.function.Supplier multisetSupplier) { + checkNotNull(elementFunction); + checkNotNull(countFunction); + checkNotNull(multisetSupplier); + return Collector.of( + multisetSupplier, + (ms, t) -> ms.add(elementFunction.apply(t), countFunction.applyAsInt(t)), + (ms1, ms2) -> { + ms1.addAll(ms2); + return ms1; + }); + } + + /** + * Returns an unmodifiable view of the specified multiset. Query operations on the returned + * multiset "read through" to the specified multiset, and attempts to modify the returned multiset + * result in an {@link UnsupportedOperationException}. + * + *

The returned multiset will be serializable if the specified multiset is serializable. + * + * @param multiset the multiset for which an unmodifiable view is to be generated + * @return an unmodifiable view of the multiset + */ + public static Multiset unmodifiableMultiset(Multiset multiset) { + if (multiset instanceof UnmodifiableMultiset || multiset instanceof ImmutableMultiset) { + @SuppressWarnings({"unchecked", "rawtypes"}) // Since it's unmodifiable, the covariant cast is safe + Multiset result = (Multiset) multiset; + return result; + } + return new UnmodifiableMultiset(checkNotNull(multiset)); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + * @since 10.0 + */ + @Deprecated + public static Multiset unmodifiableMultiset(ImmutableMultiset multiset) { + return checkNotNull(multiset); + } + + static class UnmodifiableMultiset extends ForwardingMultiset implements Serializable { + final Multiset delegate; + + UnmodifiableMultiset(Multiset delegate) { + this.delegate = delegate; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + protected Multiset delegate() { + // This is safe because all non-covariant methods are overridden + return (Multiset) delegate; + } + + transient Set elementSet; + + Set createElementSet() { + return Collections.unmodifiableSet(delegate.elementSet()); + } + + @Override + public Set elementSet() { + Set es = elementSet; + return (es == null) ? elementSet = createElementSet() : es; + } + + transient Set> entrySet; + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public Set> entrySet() { + Set> es = entrySet; + return (es == null) + // Safe because the returned set is made unmodifiable and Entry + // itself is readonly + ? entrySet = (Set) Collections.unmodifiableSet(delegate.entrySet()) + : es; + } + + @Override + public Iterator iterator() { + return Iterators.unmodifiableIterator(delegate.iterator()); + } + + @Override + public boolean add(E element) { + throw new UnsupportedOperationException(); + } + + @Override + public int add(E element, int occurences) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean addAll(Collection elementsToAdd) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean remove(Object element) { + throw new UnsupportedOperationException(); + } + + @Override + public int remove(Object element, int occurrences) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean removeAll(Collection elementsToRemove) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean retainAll(Collection elementsToRetain) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + public int setCount(E element, int count) { + throw new UnsupportedOperationException(); + } + + @Override + public boolean setCount(E element, int oldCount, int newCount) { + throw new UnsupportedOperationException(); + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns an unmodifiable view of the specified sorted multiset. Query operations on the returned + * multiset "read through" to the specified multiset, and attempts to modify the returned multiset + * result in an {@link UnsupportedOperationException}. + * + *

The returned multiset will be serializable if the specified multiset is serializable. + * + * @param sortedMultiset the sorted multiset for which an unmodifiable view is to be generated + * @return an unmodifiable view of the multiset + * @since 11.0 + */ + + public static SortedMultiset unmodifiableSortedMultiset(SortedMultiset sortedMultiset) { + // it's in its own file so it can be emulated for GWT + return new UnmodifiableSortedMultiset(checkNotNull(sortedMultiset)); + } + + /** + * Returns an immutable multiset entry with the specified element and count. The entry will be + * serializable if {@code e} is. + * + * @param e the element to be associated with the returned entry + * @param n the count to be associated with the returned entry + * @throws IllegalArgumentException if {@code n} is negative + */ + public static Entry immutableEntry(E e, int n) { + return new ImmutableEntry(e, n); + } + + static class ImmutableEntry extends AbstractEntry implements Serializable { + private final E element; + private final int count; + + ImmutableEntry(E element, int count) { + this.element = element; + this.count = count; + checkNonnegative(count, "count"); + } + + @Override + + public final E getElement() { + return element; + } + + @Override + public final int getCount() { + return count; + } + + public ImmutableEntry nextInBucket() { + return null; + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a view of the elements of {@code unfiltered} that satisfy a predicate. The returned + * multiset is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting multiset's iterators, and those of its {@code entrySet()} and {@code + * elementSet()}, do not support {@code remove()}. However, all other multiset methods supported + * by {@code unfiltered} are supported by the returned multiset. When given an element that + * doesn't satisfy the predicate, the multiset's {@code add()} and {@code addAll()} methods throw + * an {@link IllegalArgumentException}. When methods such as {@code removeAll()} and {@code + * clear()} are called on the filtered multiset, only elements that satisfy the filter will be + * removed from the underlying multiset. + * + *

The returned multiset isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered multiset's methods, such as {@code size()}, iterate across every + * element in the underlying multiset and determine which elements satisfy the filter. When a live + * view is not needed, it may be faster to copy the returned multiset and use the copy. + * + *

Warning: {@code predicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. (See {@link + * Iterables#filter(Iterable, Class)} for related functionality.) + * + * @since 14.0 + */ + + public static Multiset filter(Multiset unfiltered, Predicate predicate) { + if (unfiltered instanceof FilteredMultiset) { + // Support clear(), removeAll(), and retainAll() when filtering a filtered + // collection. + FilteredMultiset filtered = (FilteredMultiset) unfiltered; + Predicate combinedPredicate = Predicates.and(filtered.predicate, predicate); + return new FilteredMultiset(filtered.unfiltered, combinedPredicate); + } + return new FilteredMultiset(unfiltered, predicate); + } + + private static final class FilteredMultiset extends ViewMultiset { + final Multiset unfiltered; + final Predicate predicate; + + FilteredMultiset(Multiset unfiltered, Predicate predicate) { + this.unfiltered = checkNotNull(unfiltered); + this.predicate = checkNotNull(predicate); + } + + @Override + public UnmodifiableIterator iterator() { + return Iterators.filter(unfiltered.iterator(), predicate); + } + + @Override + Set createElementSet() { + return Sets.filter(unfiltered.elementSet(), predicate); + } + + @Override + Iterator elementIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Set> createEntrySet() { + return Sets.filter( + unfiltered.entrySet(), + new Predicate>() { + @Override + public boolean apply(Entry entry) { + return predicate.apply(entry.getElement()); + } + }); + } + + @Override + Iterator> entryIterator() { + throw new AssertionError("should never be called"); + } + + @Override + public int count(Object element) { + int count = unfiltered.count(element); + if (count > 0) { + @SuppressWarnings({"unchecked", "rawtypes"}) // element is equal to an E + E e = (E) element; + return predicate.apply(e) ? count : 0; + } + return 0; + } + + @Override + public int add(E element, int occurrences) { + checkArgument( + predicate.apply(element), "Element %s does not match predicate %s", element, predicate); + return unfiltered.add(element, occurrences); + } + + @Override + public int remove(Object element, int occurrences) { + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(element); + } else { + return contains(element) ? unfiltered.remove(element, occurrences) : 0; + } + } + } + + /** + * Returns the expected number of distinct elements given the specified elements. The number of + * distinct elements is only computed if {@code elements} is an instance of {@code Multiset}; + * otherwise the default value of 11 is returned. + */ + static int inferDistinctElements(Iterable elements) { + if (elements instanceof Multiset) { + return ((Multiset) elements).elementSet().size(); + } + return 11; // initial capacity will be rounded up to 16 + } + + /** + * Returns an unmodifiable view of the union of two multisets. In the returned multiset, the count + * of each element is the maximum of its counts in the two backing multisets. The iteration + * order of the returned multiset matches that of the element set of {@code multiset1} followed by + * the members of the element set of {@code multiset2} that are not contained in {@code + * multiset1}, with repeated occurrences of the same element appearing consecutively. + * + *

Results are undefined if {@code multiset1} and {@code multiset2} are based on different + * equivalence relations (as {@code HashMultiset} and {@code TreeMultiset} are). + * + * @since 14.0 + */ + + public static Multiset union( + final Multiset multiset1, final Multiset multiset2) { + checkNotNull(multiset1); + checkNotNull(multiset2); + + return new ViewMultiset() { + @Override + public boolean contains(Object element) { + return multiset1.contains(element) || multiset2.contains(element); + } + + @Override + public boolean isEmpty() { + return multiset1.isEmpty() && multiset2.isEmpty(); + } + + @Override + public int count(Object element) { + return Math.max(multiset1.count(element), multiset2.count(element)); + } + + @Override + Set createElementSet() { + return Sets.union(multiset1.elementSet(), multiset2.elementSet()); + } + + @Override + Iterator elementIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Iterator> entryIterator() { + final Iterator> iterator1 = multiset1.entrySet().iterator(); + final Iterator> iterator2 = multiset2.entrySet().iterator(); + // TODO(lowasser): consider making the entries live views + return new AbstractIterator>() { + @Override + protected Entry computeNext() { + if (iterator1.hasNext()) { + Entry entry1 = iterator1.next(); + E element = entry1.getElement(); + int count = Math.max(entry1.getCount(), multiset2.count(element)); + return immutableEntry(element, count); + } + while (iterator2.hasNext()) { + Entry entry2 = iterator2.next(); + E element = entry2.getElement(); + if (!multiset1.contains(element)) { + return immutableEntry(element, entry2.getCount()); + } + } + return endOfData(); + } + }; + } + }; + } + + /** + * Returns an unmodifiable view of the intersection of two multisets. In the returned multiset, + * the count of each element is the minimum of its counts in the two backing multisets, + * with elements that would have a count of 0 not included. The iteration order of the returned + * multiset matches that of the element set of {@code multiset1}, with repeated occurrences of the + * same element appearing consecutively. + * + *

Results are undefined if {@code multiset1} and {@code multiset2} are based on different + * equivalence relations (as {@code HashMultiset} and {@code TreeMultiset} are). + * + * @since 2.0 + */ + public static Multiset intersection( + final Multiset multiset1, final Multiset multiset2) { + checkNotNull(multiset1); + checkNotNull(multiset2); + + return new ViewMultiset() { + @Override + public int count(Object element) { + int count1 = multiset1.count(element); + return (count1 == 0) ? 0 : Math.min(count1, multiset2.count(element)); + } + + @Override + Set createElementSet() { + return Sets.intersection(multiset1.elementSet(), multiset2.elementSet()); + } + + @Override + Iterator elementIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Iterator> entryIterator() { + final Iterator> iterator1 = multiset1.entrySet().iterator(); + // TODO(lowasser): consider making the entries live views + return new AbstractIterator>() { + @Override + protected Entry computeNext() { + while (iterator1.hasNext()) { + Entry entry1 = iterator1.next(); + E element = entry1.getElement(); + int count = Math.min(entry1.getCount(), multiset2.count(element)); + if (count > 0) { + return immutableEntry(element, count); + } + } + return endOfData(); + } + }; + } + }; + } + + /** + * Returns an unmodifiable view of the sum of two multisets. In the returned multiset, the count + * of each element is the sum of its counts in the two backing multisets. The iteration + * order of the returned multiset matches that of the element set of {@code multiset1} followed by + * the members of the element set of {@code multiset2} that are not contained in {@code + * multiset1}, with repeated occurrences of the same element appearing consecutively. + * + *

Results are undefined if {@code multiset1} and {@code multiset2} are based on different + * equivalence relations (as {@code HashMultiset} and {@code TreeMultiset} are). + * + * @since 14.0 + */ + + public static Multiset sum( + final Multiset multiset1, final Multiset multiset2) { + checkNotNull(multiset1); + checkNotNull(multiset2); + + // TODO(lowasser): consider making the entries live views + return new ViewMultiset() { + @Override + public boolean contains(Object element) { + return multiset1.contains(element) || multiset2.contains(element); + } + + @Override + public boolean isEmpty() { + return multiset1.isEmpty() && multiset2.isEmpty(); + } + + @Override + public int size() { + return IntMath.saturatedAdd(multiset1.size(), multiset2.size()); + } + + @Override + public int count(Object element) { + return multiset1.count(element) + multiset2.count(element); + } + + @Override + Set createElementSet() { + return Sets.union(multiset1.elementSet(), multiset2.elementSet()); + } + + @Override + Iterator elementIterator() { + throw new AssertionError("should never be called"); + } + + @Override + Iterator> entryIterator() { + final Iterator> iterator1 = multiset1.entrySet().iterator(); + final Iterator> iterator2 = multiset2.entrySet().iterator(); + return new AbstractIterator>() { + @Override + protected Entry computeNext() { + if (iterator1.hasNext()) { + Entry entry1 = iterator1.next(); + E element = entry1.getElement(); + int count = entry1.getCount() + multiset2.count(element); + return immutableEntry(element, count); + } + while (iterator2.hasNext()) { + Entry entry2 = iterator2.next(); + E element = entry2.getElement(); + if (!multiset1.contains(element)) { + return immutableEntry(element, entry2.getCount()); + } + } + return endOfData(); + } + }; + } + }; + } + + /** + * Returns an unmodifiable view of the difference of two multisets. In the returned multiset, the + * count of each element is the result of the zero-truncated subtraction of its count in + * the second multiset from its count in the first multiset, with elements that would have a count + * of 0 not included. The iteration order of the returned multiset matches that of the element set + * of {@code multiset1}, with repeated occurrences of the same element appearing consecutively. + * + *

Results are undefined if {@code multiset1} and {@code multiset2} are based on different + * equivalence relations (as {@code HashMultiset} and {@code TreeMultiset} are). + * + * @since 14.0 + */ + + public static Multiset difference( + final Multiset multiset1, final Multiset multiset2) { + checkNotNull(multiset1); + checkNotNull(multiset2); + + // TODO(lowasser): consider making the entries live views + return new ViewMultiset() { + @Override + public int count(Object element) { + int count1 = multiset1.count(element); + return (count1 == 0) ? 0 : Math.max(0, count1 - multiset2.count(element)); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + Iterator elementIterator() { + final Iterator> iterator1 = multiset1.entrySet().iterator(); + return new AbstractIterator() { + @Override + protected E computeNext() { + while (iterator1.hasNext()) { + Entry entry1 = iterator1.next(); + E element = entry1.getElement(); + if (entry1.getCount() > multiset2.count(element)) { + return element; + } + } + return endOfData(); + } + }; + } + + @Override + Iterator> entryIterator() { + final Iterator> iterator1 = multiset1.entrySet().iterator(); + return new AbstractIterator>() { + @Override + protected Entry computeNext() { + while (iterator1.hasNext()) { + Entry entry1 = iterator1.next(); + E element = entry1.getElement(); + int count = entry1.getCount() - multiset2.count(element); + if (count > 0) { + return immutableEntry(element, count); + } + } + return endOfData(); + } + }; + } + + @Override + int distinctElements() { + return Iterators.size(entryIterator()); + } + }; + } + + /** + * Returns {@code true} if {@code subMultiset.count(o) <= superMultiset.count(o)} for all {@code + * o}. + * + * @since 10.0 + */ + public static boolean containsOccurrences(Multiset superMultiset, Multiset subMultiset) { + checkNotNull(superMultiset); + checkNotNull(subMultiset); + for (Entry entry : subMultiset.entrySet()) { + int superCount = superMultiset.count(entry.getElement()); + if (superCount < entry.getCount()) { + return false; + } + } + return true; + } + + /** + * Modifies {@code multisetToModify} so that its count for an element {@code e} is at most {@code + * multisetToRetain.count(e)}. + * + *

To be precise, {@code multisetToModify.count(e)} is set to {@code + * Math.min(multisetToModify.count(e), multisetToRetain.count(e))}. This is similar to {@link + * #intersection(Multiset, Multiset) intersection} {@code (multisetToModify, multisetToRetain)}, + * but mutates {@code multisetToModify} instead of returning a view. + * + *

In contrast, {@code multisetToModify.retainAll(multisetToRetain)} keeps all occurrences of + * elements that appear at all in {@code multisetToRetain}, and deletes all occurrences of all + * other elements. + * + * @return {@code true} if {@code multisetToModify} was changed as a result of this operation + * @since 10.0 + */ + public static boolean retainOccurrences( + Multiset multisetToModify, Multiset multisetToRetain) { + return retainOccurrencesImpl(multisetToModify, multisetToRetain); + } + + /** Delegate implementation which cares about the element type. */ + private static boolean retainOccurrencesImpl( + Multiset multisetToModify, Multiset occurrencesToRetain) { + checkNotNull(multisetToModify); + checkNotNull(occurrencesToRetain); + // Avoiding ConcurrentModificationExceptions is tricky. + Iterator> entryIterator = multisetToModify.entrySet().iterator(); + boolean changed = false; + while (entryIterator.hasNext()) { + Entry entry = entryIterator.next(); + int retainCount = occurrencesToRetain.count(entry.getElement()); + if (retainCount == 0) { + entryIterator.remove(); + changed = true; + } else if (retainCount < entry.getCount()) { + multisetToModify.setCount(entry.getElement(), retainCount); + changed = true; + } + } + return changed; + } + + /** + * For each occurrence of an element {@code e} in {@code occurrencesToRemove}, removes one + * occurrence of {@code e} in {@code multisetToModify}. + * + *

Equivalently, this method modifies {@code multisetToModify} so that {@code + * multisetToModify.count(e)} is set to {@code Math.max(0, multisetToModify.count(e) - + * Iterables.frequency(occurrencesToRemove, e))}. + * + *

This is not the same as {@code multisetToModify.} {@link Multiset#removeAll + * removeAll}{@code (occurrencesToRemove)}, which removes all occurrences of elements that appear + * in {@code occurrencesToRemove}. However, this operation is equivalent to, albeit + * sometimes more efficient than, the following: + * + *

{@code
+   * for (E e : occurrencesToRemove) {
+   *   multisetToModify.remove(e);
+   * }
+   * }
+ * + * @return {@code true} if {@code multisetToModify} was changed as a result of this operation + * @since 18.0 (present in 10.0 with a requirement that the second parameter be a {@code + * Multiset}) + */ + public static boolean removeOccurrences( + Multiset multisetToModify, Iterable occurrencesToRemove) { + if (occurrencesToRemove instanceof Multiset) { + return removeOccurrences(multisetToModify, (Multiset) occurrencesToRemove); + } else { + checkNotNull(multisetToModify); + checkNotNull(occurrencesToRemove); + boolean changed = false; + for (Object o : occurrencesToRemove) { + changed |= multisetToModify.remove(o); + } + return changed; + } + } + + /** + * For each occurrence of an element {@code e} in {@code occurrencesToRemove}, removes one + * occurrence of {@code e} in {@code multisetToModify}. + * + *

Equivalently, this method modifies {@code multisetToModify} so that {@code + * multisetToModify.count(e)} is set to {@code Math.max(0, multisetToModify.count(e) - + * occurrencesToRemove.count(e))}. + * + *

This is not the same as {@code multisetToModify.} {@link Multiset#removeAll + * removeAll}{@code (occurrencesToRemove)}, which removes all occurrences of elements that appear + * in {@code occurrencesToRemove}. However, this operation is equivalent to, albeit + * sometimes more efficient than, the following: + * + *

{@code
+   * for (E e : occurrencesToRemove) {
+   *   multisetToModify.remove(e);
+   * }
+   * }
+ * + * @return {@code true} if {@code multisetToModify} was changed as a result of this operation + * @since 10.0 (missing in 18.0 when only the overload taking an {@code Iterable} was present) + */ + public static boolean removeOccurrences( + Multiset multisetToModify, Multiset occurrencesToRemove) { + checkNotNull(multisetToModify); + checkNotNull(occurrencesToRemove); + + boolean changed = false; + Iterator> entryIterator = multisetToModify.entrySet().iterator(); + while (entryIterator.hasNext()) { + Entry entry = entryIterator.next(); + int removeCount = occurrencesToRemove.count(entry.getElement()); + if (removeCount >= entry.getCount()) { + entryIterator.remove(); + changed = true; + } else if (removeCount > 0) { + multisetToModify.remove(entry.getElement(), removeCount); + changed = true; + } + } + return changed; + } + + /** + * Implementation of the {@code equals}, {@code hashCode}, and {@code toString} methods of {@link + * Entry}. + */ + abstract static class AbstractEntry implements Entry { + /** + * Indicates whether an object equals this entry, following the behavior specified in {@link + * Entry#equals}. + */ + @Override + public boolean equals(Object object) { + if (object instanceof Multiset.Entry) { + Entry that = (Entry) object; + return this.getCount() == that.getCount() + && Objects.equal(this.getElement(), that.getElement()); + } + return false; + } + + /** + * Return this entry's hash code, following the behavior specified in {@link + * Entry#hashCode}. + */ + @Override + public int hashCode() { + E e = getElement(); + return ((e == null) ? 0 : e.hashCode()) ^ getCount(); + } + + /** + * Returns a string representation of this multiset entry. The string representation consists of + * the associated element if the associated count is one, and otherwise the associated element + * followed by the characters " x " (space, x and space) followed by the count. Elements and + * counts are converted to strings as by {@code String.valueOf}. + */ + @Override + public String toString() { + String text = String.valueOf(getElement()); + int n = getCount(); + return (n == 1) ? text : (text + " x " + n); + } + } + + /** An implementation of {@link Multiset#equals}. */ + static boolean equalsImpl(Multiset multiset, Object object) { + if (object == multiset) { + return true; + } + if (object instanceof Multiset) { + Multiset that = (Multiset) object; + /* + * We can't simply check whether the entry sets are equal, since that + * approach fails when a TreeMultiset has a comparator that returns 0 + * when passed unequal elements. + */ + + if (multiset.size() != that.size() || multiset.entrySet().size() != that.entrySet().size()) { + return false; + } + for (Entry entry : that.entrySet()) { + if (multiset.count(entry.getElement()) != entry.getCount()) { + return false; + } + } + return true; + } + return false; + } + + /** An implementation of {@link Multiset#addAll}. */ + static boolean addAllImpl(Multiset self, Collection elements) { + checkNotNull(self); + checkNotNull(elements); + if (elements instanceof Multiset) { + return addAllImpl(self, cast(elements)); + } else if (elements.isEmpty()) { + return false; + } else { + return Iterators.addAll(self, elements.iterator()); + } + } + + /** A specialization of {@code addAllImpl} for when {@code elements} is itself a Multiset. */ + private static boolean addAllImpl(Multiset self, Multiset elements) { + if (elements.isEmpty()) { + return false; + } + elements.forEachEntry(self::add); + return true; + } + + /** An implementation of {@link Multiset#removeAll}. */ + static boolean removeAllImpl(Multiset self, Collection elementsToRemove) { + Collection collection = + (elementsToRemove instanceof Multiset) + ? ((Multiset) elementsToRemove).elementSet() + : elementsToRemove; + + return self.elementSet().removeAll(collection); + } + + /** An implementation of {@link Multiset#retainAll}. */ + static boolean retainAllImpl(Multiset self, Collection elementsToRetain) { + checkNotNull(elementsToRetain); + Collection collection = + (elementsToRetain instanceof Multiset) + ? ((Multiset) elementsToRetain).elementSet() + : elementsToRetain; + + return self.elementSet().retainAll(collection); + } + + /** An implementation of {@link Multiset#setCount(Object, int)}. */ + static int setCountImpl(Multiset self, E element, int count) { + checkNonnegative(count, "count"); + + int oldCount = self.count(element); + + int delta = count - oldCount; + if (delta > 0) { + self.add(element, delta); + } else if (delta < 0) { + self.remove(element, -delta); + } + + return oldCount; + } + + /** An implementation of {@link Multiset#setCount(Object, int, int)}. */ + static boolean setCountImpl(Multiset self, E element, int oldCount, int newCount) { + checkNonnegative(oldCount, "oldCount"); + checkNonnegative(newCount, "newCount"); + + if (self.count(element) == oldCount) { + self.setCount(element, newCount); + return true; + } else { + return false; + } + } + + static Iterator elementIterator(Iterator> entryIterator) { + return new TransformedIterator, E>(entryIterator) { + @Override + E transform(Entry entry) { + return entry.getElement(); + } + }; + } + + abstract static class ElementSet extends Sets.ImprovedAbstractSet { + abstract Multiset multiset(); + + @Override + public void clear() { + multiset().clear(); + } + + @Override + public boolean contains(Object o) { + return multiset().contains(o); + } + + @Override + public boolean containsAll(Collection c) { + return multiset().containsAll(c); + } + + @Override + public boolean isEmpty() { + return multiset().isEmpty(); + } + + @Override + public abstract Iterator iterator(); + + @Override + public boolean remove(Object o) { + return multiset().remove(o, Integer.MAX_VALUE) > 0; + } + + @Override + public int size() { + return multiset().entrySet().size(); + } + } + + abstract static class EntrySet extends Sets.ImprovedAbstractSet> { + abstract Multiset multiset(); + + @Override + public boolean contains(Object o) { + if (o instanceof Entry) { + /* + * The GWT compiler wrongly issues a warning here. + */ + @SuppressWarnings("cast") + Entry entry = (Entry) o; + if (entry.getCount() <= 0) { + return false; + } + int count = multiset().count(entry.getElement()); + return count == entry.getCount(); + } + return false; + } + + // GWT compiler warning; see contains(). + @SuppressWarnings("cast") + @Override + public boolean remove(Object object) { + if (object instanceof Multiset.Entry) { + Entry entry = (Entry) object; + Object element = entry.getElement(); + int entryCount = entry.getCount(); + if (entryCount != 0) { + // Safe as long as we never add a new entry, which we won't. + @SuppressWarnings({"unchecked", "rawtypes"}) + Multiset multiset = (Multiset) multiset(); + return multiset.setCount(element, entryCount, 0); + } + } + return false; + } + + @Override + public void clear() { + multiset().clear(); + } + } + + /** An implementation of {@link Multiset#iterator}. */ + static Iterator iteratorImpl(Multiset multiset) { + return new MultisetIteratorImpl(multiset, multiset.entrySet().iterator()); + } + + static final class MultisetIteratorImpl implements Iterator { + private final Multiset multiset; + private final Iterator> entryIterator; + private Entry currentEntry; + + /** Count of subsequent elements equal to current element */ + private int laterCount; + + /** Count of all elements equal to current element */ + private int totalCount; + + private boolean canRemove; + + MultisetIteratorImpl(Multiset multiset, Iterator> entryIterator) { + this.multiset = multiset; + this.entryIterator = entryIterator; + } + + @Override + public boolean hasNext() { + return laterCount > 0 || entryIterator.hasNext(); + } + + @Override + public E next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + if (laterCount == 0) { + currentEntry = entryIterator.next(); + totalCount = laterCount = currentEntry.getCount(); + } + laterCount--; + canRemove = true; + return currentEntry.getElement(); + } + + @Override + public void remove() { + checkRemove(canRemove); + if (totalCount == 1) { + entryIterator.remove(); + } else { + multiset.remove(currentEntry.getElement()); + } + totalCount--; + canRemove = false; + } + } + + static Spliterator spliteratorImpl(Multiset multiset) { + Spliterator> entrySpliterator = multiset.entrySet().spliterator(); + return CollectSpliterators.flatMap( + entrySpliterator, + entry -> Collections.nCopies(entry.getCount(), entry.getElement()).spliterator(), + Spliterator.SIZED + | (entrySpliterator.characteristics() + & (Spliterator.ORDERED | Spliterator.NONNULL | Spliterator.IMMUTABLE)), + multiset.size()); + } + + /** An implementation of {@link Multiset#size}. */ + static int linearTimeSizeImpl(Multiset multiset) { + long size = 0; + for (Entry entry : multiset.entrySet()) { + size += entry.getCount(); + } + return Ints.saturatedCast(size); + } + + /** Used to avoid http://bugs.sun.com/view_bug.do?bug_id=6558557 */ + static Multiset cast(Iterable iterable) { + return (Multiset) iterable; + } + + /** + * Returns a copy of {@code multiset} as an {@link ImmutableMultiset} whose iteration order is + * highest count first, with ties broken by the iteration order of the original multiset. + * + * @since 11.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static ImmutableMultiset copyHighestCountFirst(Multiset multiset) { + Entry[] entries = (Entry[]) multiset.entrySet().toArray(new Entry[0]); + Arrays.sort(entries, DecreasingCount.INSTANCE); + return ImmutableMultiset.copyFromEntries(Arrays.asList(entries)); + } + + private static final class DecreasingCount implements Comparator> { + static final DecreasingCount INSTANCE = new DecreasingCount(); + + @Override + public int compare(Entry entry1, Entry entry2) { + return entry2.getCount() - entry1.getCount(); // subtracting two nonnegative integers + } + } + + /** + * An {@link AbstractMultiset} with additional default implementations, some of them linear-time + * implementations in terms of {@code elementSet} and {@code entrySet}. + */ + private abstract static class ViewMultiset extends AbstractMultiset { + @Override + public int size() { + return linearTimeSizeImpl(this); + } + + @Override + public void clear() { + elementSet().clear(); + } + + @Override + public Iterator iterator() { + return iteratorImpl(this); + } + + @Override + int distinctElements() { + return elementSet().size(); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MutableClassToInstanceMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MutableClassToInstanceMap.java new file mode 100644 index 0000000000000..8f3df5406690a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/MutableClassToInstanceMap.java @@ -0,0 +1,180 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Primitives; +import java.io.Serializable; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; +import java.util.Spliterator; + +/** + * A mutable class-to-instance map backed by an arbitrary user-provided map. See also {@link + * ImmutableClassToInstanceMap}. + * + *

See the Guava User Guide article on {@code + * ClassToInstanceMap}. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +@SuppressWarnings("serial") // using writeReplace instead of standard serialization +public final class MutableClassToInstanceMap extends ForwardingMap, B> + implements ClassToInstanceMap, Serializable { + + /** + * Returns a new {@code MutableClassToInstanceMap} instance backed by a {@link HashMap} using the + * default initial capacity and load factor. + */ + public static MutableClassToInstanceMap create() { + return new MutableClassToInstanceMap(new HashMap, B>()); + } + + /** + * Returns a new {@code MutableClassToInstanceMap} instance backed by a given empty {@code + * backingMap}. The caller surrenders control of the backing map, and thus should not allow any + * direct references to it to remain accessible. + */ + public static MutableClassToInstanceMap create(Map, B> backingMap) { + return new MutableClassToInstanceMap(backingMap); + } + + private final Map, B> delegate; + + private MutableClassToInstanceMap(Map, B> delegate) { + this.delegate = checkNotNull(delegate); + } + + @Override + protected Map, B> delegate() { + return delegate; + } + + /** + * Wraps the {@code setValue} implementation of an {@code Entry} to enforce the class constraint. + */ + private static Entry, B> checkedEntry( + final Entry, B> entry) { + return new ForwardingMapEntry, B>() { + @Override + protected Entry, B> delegate() { + return entry; + } + + @Override + public B setValue(B value) { + return super.setValue(cast(getKey(), value)); + } + }; + } + + @Override + public Set, B>> entrySet() { + return new ForwardingSet, B>>() { + + @Override + protected Set, B>> delegate() { + return MutableClassToInstanceMap.this.delegate().entrySet(); + } + + @Override + public Spliterator, B>> spliterator() { + return CollectSpliterators.map( + delegate().spliterator(), MutableClassToInstanceMap::checkedEntry); + } + + @Override + public Iterator, B>> iterator() { + return new TransformedIterator, B>, Entry, B>>( + delegate().iterator()) { + @Override + Entry, B> transform(Entry, B> from) { + return checkedEntry(from); + } + }; + } + + @Override + public Object[] toArray() { + return standardToArray(); + } + + @Override + public T[] toArray(T[] array) { + return standardToArray(array); + } + }; + } + + @Override + public B put(Class key, B value) { + return super.put(key, cast(key, value)); + } + + @Override + public void putAll(Map, ? extends B> map) { + Map, B> copy = new LinkedHashMap<>(map); + for (Entry, B> entry : copy.entrySet()) { + cast((Class) entry.getKey(), entry.getValue()); + } + super.putAll(copy); + } + + @Override + public T putInstance(Class type, T value) { + return cast(type, put(type, value)); + } + + @Override + public T getInstance(Class type) { + return cast(type, get(type)); + } + + private static T cast(Class type, B value) { + return Primitives.wrap(type).cast(value); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private Object writeReplace() { + return new SerializedForm(delegate()); + } + + /** Serialized form of the map, to avoid serializing the constraint. */ + private static final class SerializedForm implements Serializable { + private final Map, B> backingMap; + + SerializedForm(Map, B> backingMap) { + this.backingMap = backingMap; + } + + Object readResolve() { + return create(backingMap); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NaturalOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NaturalOrdering.java new file mode 100644 index 0000000000000..6d44c244896e9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NaturalOrdering.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.Serializable; + + +/** An ordering that uses the natural order of the values. */ +@SuppressWarnings({"unchecked", "rawtypes"}) // TODO(kevinb): the right way to explain this?? +final class NaturalOrdering extends Ordering implements Serializable { + static final NaturalOrdering INSTANCE = new NaturalOrdering(); + + private transient Ordering nullsFirst; + private transient Ordering nullsLast; + + @Override + public int compare(Comparable left, Comparable right) { + checkNotNull(left); // for GWT + checkNotNull(right); + return left.compareTo(right); + } + + @Override + public Ordering nullsFirst() { + Ordering result = nullsFirst; + if (result == null) { + result = nullsFirst = super.nullsFirst(); + } + return (Ordering) result; + } + + @Override + public Ordering nullsLast() { + Ordering result = nullsLast; + if (result == null) { + result = nullsLast = super.nullsLast(); + } + return (Ordering) result; + } + + @Override + public Ordering reverse() { + return (Ordering) ReverseNaturalOrdering.INSTANCE; + } + + // preserving singleton-ness gives equals()/hashCode() for free + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "Ordering.natural()"; + } + + private NaturalOrdering() {} + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsFirstOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsFirstOrdering.java new file mode 100644 index 0000000000000..53e98c02dc11f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsFirstOrdering.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + + +/** An ordering that treats {@code null} as less than all other values. */ +final class NullsFirstOrdering extends Ordering implements Serializable { + final Ordering ordering; + + NullsFirstOrdering(Ordering ordering) { + this.ordering = ordering; + } + + @Override + public int compare(T left, T right) { + if (left == right) { + return 0; + } + if (left == null) { + return RIGHT_IS_GREATER; + } + if (right == null) { + return LEFT_IS_GREATER; + } + return ordering.compare(left, right); + } + + @Override + public Ordering reverse() { + // ordering.reverse() might be optimized, so let it do its thing + return ordering.reverse().nullsLast(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // still need the right way to explain this + @Override + public Ordering nullsFirst() { + return (Ordering) this; + } + + @Override + public Ordering nullsLast() { + return ordering.nullsLast(); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof NullsFirstOrdering) { + NullsFirstOrdering that = (NullsFirstOrdering) object; + return this.ordering.equals(that.ordering); + } + return false; + } + + @Override + public int hashCode() { + return ordering.hashCode() ^ 957692532; // meaningless + } + + @Override + public String toString() { + return ordering + ".nullsFirst()"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsLastOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsLastOrdering.java new file mode 100644 index 0000000000000..9ea78636165bc --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/NullsLastOrdering.java @@ -0,0 +1,88 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + + +/** An ordering that treats {@code null} as greater than all other values. */ +final class NullsLastOrdering extends Ordering implements Serializable { + final Ordering ordering; + + NullsLastOrdering(Ordering ordering) { + this.ordering = ordering; + } + + @Override + public int compare(T left, T right) { + if (left == right) { + return 0; + } + if (left == null) { + return LEFT_IS_GREATER; + } + if (right == null) { + return RIGHT_IS_GREATER; + } + return ordering.compare(left, right); + } + + @Override + public Ordering reverse() { + // ordering.reverse() might be optimized, so let it do its thing + return ordering.reverse().nullsFirst(); + } + + @Override + public Ordering nullsFirst() { + return ordering.nullsFirst(); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // still need the right way to explain this + @Override + public Ordering nullsLast() { + return (Ordering) this; + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof NullsLastOrdering) { + NullsLastOrdering that = (NullsLastOrdering) object; + return this.ordering.equals(that.ordering); + } + return false; + } + + @Override + public int hashCode() { + return ordering.hashCode() ^ -921210296; // meaningless + } + + @Override + public String toString() { + return ordering + ".nullsLast()"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ObjectArrays.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ObjectArrays.java new file mode 100644 index 0000000000000..3aca878745d1f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ObjectArrays.java @@ -0,0 +1,223 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; + +import java.lang.reflect.Array; +import java.util.Arrays; +import java.util.Collection; + + +/** + * Static utility methods pertaining to object arrays. + * + * @author Kevin Bourrillion + * @since 2.0 + */ +public final class ObjectArrays { + + private ObjectArrays() {} + + /** + * Returns a new array of the given length with the specified component type. + * + * @param type the component type + * @param length the length of the new array + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static T[] newArray(Class type, int length) { + return (T[]) Array.newInstance(type, length); + } + + /** + * Returns a new array of the given length with the same type as a reference array. + * + * @param reference any array of the desired type + * @param length the length of the new array + */ + public static T[] newArray(T[] reference, int length) { + return Platform.newArray(reference, length); + } + + /** + * Returns a new array that contains the concatenated contents of two arrays. + * + * @param first the first array of elements to concatenate + * @param second the second array of elements to concatenate + * @param type the component type of the returned array + */ + public static T[] concat(T[] first, T[] second, Class type) { + T[] result = newArray(type, first.length + second.length); + System.arraycopy(first, 0, result, 0, first.length); + System.arraycopy(second, 0, result, first.length, second.length); + return result; + } + + /** + * Returns a new array that prepends {@code element} to {@code array}. + * + * @param element the element to prepend to the front of {@code array} + * @param array the array of elements to append + * @return an array whose size is one larger than {@code array}, with {@code element} occupying + * the first position, and the elements of {@code array} occupying the remaining elements. + */ + public static T[] concat(T element, T[] array) { + T[] result = newArray(array, array.length + 1); + result[0] = element; + System.arraycopy(array, 0, result, 1, array.length); + return result; + } + + /** + * Returns a new array that appends {@code element} to {@code array}. + * + * @param array the array of elements to prepend + * @param element the element to append to the end + * @return an array whose size is one larger than {@code array}, with the same contents as {@code + * array}, plus {@code element} occupying the last position. + */ + public static T[] concat(T[] array, T element) { + T[] result = Arrays.copyOf(array, array.length + 1); + result[array.length] = element; + return result; + } + + /** + * Returns an array containing all of the elements in the specified collection; the runtime type + * of the returned array is that of the specified array. If the collection fits in the specified + * array, it is returned therein. Otherwise, a new array is allocated with the runtime type of the + * specified array and the size of the specified collection. + * + *

If the collection fits in the specified array with room to spare (i.e., the array has more + * elements than the collection), the element in the array immediately following the end of the + * collection is set to {@code null}. This is useful in determining the length of the collection + * only if the caller knows that the collection does not contain any null elements. + * + *

This method returns the elements in the order they are returned by the collection's + * iterator. + * + *

TODO(kevinb): support concurrently modified collections? + * + * @param c the collection for which to return an array of elements + * @param array the array in which to place the collection elements + * @throws ArrayStoreException if the runtime type of the specified array is not a supertype of + * the runtime type of every element in the specified collection + */ + static T[] toArrayImpl(Collection c, T[] array) { + int size = c.size(); + if (array.length < size) { + array = newArray(array, size); + } + fillArray(c, array); + if (array.length > size) { + array[size] = null; + } + return array; + } + + /** + * Implementation of {@link Collection#toArray(Object[])} for collections backed by an object + * array. the runtime type of the returned array is that of the specified array. If the collection + * fits in the specified array, it is returned therein. Otherwise, a new array is allocated with + * the runtime type of the specified array and the size of the specified collection. + * + *

If the collection fits in the specified array with room to spare (i.e., the array has more + * elements than the collection), the element in the array immediately following the end of the + * collection is set to {@code null}. This is useful in determining the length of the collection + * only if the caller knows that the collection does not contain any null elements. + */ + static T[] toArrayImpl(Object[] src, int offset, int len, T[] dst) { + checkPositionIndexes(offset, offset + len, src.length); + if (dst.length < len) { + dst = newArray(dst, len); + } else if (dst.length > len) { + dst[len] = null; + } + System.arraycopy(src, offset, dst, 0, len); + return dst; + } + + /** + * Returns an array containing all of the elements in the specified collection. This method + * returns the elements in the order they are returned by the collection's iterator. The returned + * array is "safe" in that no references to it are maintained by the collection. The caller is + * thus free to modify the returned array. + * + *

This method assumes that the collection size doesn't change while the method is running. + * + *

TODO(kevinb): support concurrently modified collections? + * + * @param c the collection for which to return an array of elements + */ + static Object[] toArrayImpl(Collection c) { + return fillArray(c, new Object[c.size()]); + } + + /** + * Returns a copy of the specified subrange of the specified array that is literally an Object[], + * and not e.g. a {@code String[]}. + */ + static Object[] copyAsObjectArray(Object[] elements, int offset, int length) { + checkPositionIndexes(offset, offset + length, elements.length); + if (length == 0) { + return new Object[0]; + } + Object[] result = new Object[length]; + System.arraycopy(elements, offset, result, 0, length); + return result; + } + + private static Object[] fillArray(Iterable elements, Object[] array) { + int i = 0; + for (Object element : elements) { + array[i++] = element; + } + return array; + } + + /** Swaps {@code array[i]} with {@code array[j]}. */ + static void swap(Object[] array, int i, int j) { + Object temp = array[i]; + array[i] = array[j]; + array[j] = temp; + } + + static Object[] checkElementsNotNull(Object... array) { + return checkElementsNotNull(array, array.length); + } + + static Object[] checkElementsNotNull(Object[] array, int length) { + for (int i = 0; i < length; i++) { + checkElementNotNull(array[i], i); + } + return array; + } + + // We do this instead of Preconditions.checkNotNull to save boxing and array + // creation cost. + static Object checkElementNotNull(Object element, int index) { + if (element == null) { + throw new NullPointerException("at index " + index); + } + return element; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Ordering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Ordering.java new file mode 100644 index 0000000000000..e31535762b6d5 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Ordering.java @@ -0,0 +1,938 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map.Entry; +import java.util.NoSuchElementException; +import java.util.SortedMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + + +/** + * A comparator, with additional methods to support common operations. This is an "enriched" version + * of {@code Comparator} for pre-Java-8 users, in the same sense that {@link FluentIterable} is an + * enriched {@link Iterable} for pre-Java-8 users. + * + *

Three types of methods

+ * + * Like other fluent types, there are three types of methods present: methods for acquiring, + * chaining, and using. + * + *

Acquiring

+ * + *

The common ways to get an instance of {@code Ordering} are: + * + *

    + *
  • Subclass it and implement {@link #compare} instead of implementing {@link Comparator} + * directly + *
  • Pass a pre-existing {@link Comparator} instance to {@link #from(Comparator)} + *
  • Use the natural ordering, {@link Ordering#natural} + *
+ * + *

Chaining

+ * + *

Then you can use the chaining methods to get an altered version of that {@code + * Ordering}, including: + * + *

    + *
  • {@link #reverse} + *
  • {@link #compound(Comparator)} + *
  • {@link #onResultOf(Function)} + *
  • {@link #nullsFirst} / {@link #nullsLast} + *
+ * + *

Using

+ * + *

Finally, use the resulting {@code Ordering} anywhere a {@link Comparator} is required, or use + * any of its special operations, such as: + * + *

    + *
  • {@link #immutableSortedCopy} + *
  • {@link #isOrdered} / {@link #isStrictlyOrdered} + *
  • {@link #min} / {@link #max} + *
+ * + *

Understanding complex orderings

+ * + *

Complex chained orderings like the following example can be challenging to understand. + * + *

{@code
+ * Ordering ordering =
+ *     Ordering.natural()
+ *         .nullsFirst()
+ *         .onResultOf(getBarFunction)
+ *         .nullsLast();
+ * }
+ * + * Note that each chaining method returns a new ordering instance which is backed by the previous + * instance, but has the chance to act on values before handing off to that backing instance. + * As a result, it usually helps to read chained ordering expressions backwards. For example, + * when {@code compare} is called on the above ordering: + * + *
    + *
  1. First, if only one {@code Foo} is null, that null value is treated as greater + *
  2. Next, non-null {@code Foo} values are passed to {@code getBarFunction} (we will be + * comparing {@code Bar} values from now on) + *
  3. Next, if only one {@code Bar} is null, that null value is treated as lesser + *
  4. Finally, natural ordering is used (i.e. the result of {@code Bar.compareTo(Bar)} is + * returned) + *
+ * + *

Alas, {@link #reverse} is a little different. As you read backwards through a chain and + * encounter a call to {@code reverse}, continue working backwards until a result is determined, and + * then reverse that result. + * + *

Additional notes

+ * + *

Except as noted, the orderings returned by the factory methods of this class are serializable + * if and only if the provided instances that back them are. For example, if {@code ordering} and + * {@code function} can themselves be serialized, then {@code ordering.onResultOf(function)} can as + * well. + * + *

For Java 8 users

+ * + *

If you are using Java 8, this class is now obsolete. Most of its functionality is now provided + * by {@link java.util.stream.Stream Stream} and by {@link Comparator} itself, and the rest can now + * be found as static methods in our new {@link Comparators} class. See each method below for + * further instructions. Whenever possible, you should change any references of type {@code + * Ordering} to be of type {@code Comparator} instead. However, at this time we have no plan to + * deprecate this class. + * + *

Many replacements involve adopting {@code Stream}, and these changes can sometimes make your + * code verbose. Whenever following this advice, you should check whether {@code Stream} could be + * adopted more comprehensively in your code; the end result may be quite a bit simpler. + * + *

See also

+ * + *

See the Guava User Guide article on {@code Ordering}. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + * @since 2.0 + */ +public abstract class Ordering implements Comparator { + // Natural order + + /** + * Returns a serializable ordering that uses the natural order of the values. The ordering throws + * a {@link NullPointerException} when passed a null parameter. + * + *

The type specification is {@code }, instead of the technically correct + * {@code >}, to support legacy types from before Java 5. + * + *

Java 8 users: use {@link Comparator#naturalOrder} instead. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) // TODO(kevinb): right way to explain this?? + public static Ordering natural() { + return (Ordering) NaturalOrdering.INSTANCE; + } + + // Static factories + + /** + * Returns an ordering based on an existing comparator instance. Note that it is + * unnecessary to create a new anonymous inner class implementing {@code Comparator} just + * to pass it in here. Instead, simply subclass {@code Ordering} and implement its {@code compare} + * method directly. + * + *

Java 8 users: this class is now obsolete as explained in the class documentation, so + * there is no need to use this method. + * + * @param comparator the comparator that defines the order + * @return comparator itself if it is already an {@code Ordering}; otherwise an ordering that + * wraps that comparator + */ + public static Ordering from(Comparator comparator) { + return (comparator instanceof Ordering) + ? (Ordering) comparator + : new ComparatorOrdering(comparator); + } + + /** + * Simply returns its argument. + * + * @deprecated no need to use this + */ + @Deprecated + public static Ordering from(Ordering ordering) { + return checkNotNull(ordering); + } + + /** + * Returns an ordering that compares objects according to the order in which they appear in the + * given list. Only objects present in the list (according to {@link Object#equals}) may be + * compared. This comparator imposes a "partial ordering" over the type {@code T}. Subsequent + * changes to the {@code valuesInOrder} list will have no effect on the returned comparator. Null + * values in the list are not supported. + * + *

The returned comparator throws a {@link ClassCastException} when it receives an input + * parameter that isn't among the provided values. + * + *

The generated comparator is serializable if all the provided values are serializable. + * + * @param valuesInOrder the values that the returned comparator will be able to compare, in the + * order the comparator should induce + * @return the comparator described above + * @throws NullPointerException if any of the provided values is null + * @throws IllegalArgumentException if {@code valuesInOrder} contains any duplicate values + * (according to {@link Object#equals}) + */ + // TODO(kevinb): provide replacement + public static Ordering explicit(List valuesInOrder) { + return new ExplicitOrdering(valuesInOrder); + } + + /** + * Returns an ordering that compares objects according to the order in which they are given to + * this method. Only objects present in the argument list (according to {@link Object#equals}) may + * be compared. This comparator imposes a "partial ordering" over the type {@code T}. Null values + * in the argument list are not supported. + * + *

The returned comparator throws a {@link ClassCastException} when it receives an input + * parameter that isn't among the provided values. + * + *

The generated comparator is serializable if all the provided values are serializable. + * + * @param leastValue the value which the returned comparator should consider the "least" of all + * values + * @param remainingValuesInOrder the rest of the values that the returned comparator will be able + * to compare, in the order the comparator should follow + * @return the comparator described above + * @throws NullPointerException if any of the provided values is null + * @throws IllegalArgumentException if any duplicate values (according to {@link + * Object#equals(Object)}) are present among the method arguments + */ + // TODO(kevinb): provide replacement + @SuppressWarnings("unchecked") + public static Ordering explicit(T leastValue, T... remainingValuesInOrder) { + return explicit(Lists.asList(leastValue, remainingValuesInOrder)); + } + + // Ordering singletons + + /** + * Returns an ordering which treats all values as equal, indicating "no ordering." Passing this + * ordering to any stable sort algorithm results in no change to the order of elements. + * Note especially that {@link #sortedCopy} and {@link #immutableSortedCopy} are stable, and in + * the returned instance these are implemented by simply copying the source list. + * + *

Example: + * + *

{@code
+   * Ordering.allEqual().nullsLast().sortedCopy(
+   *     asList(t, null, e, s, null, t, null))
+   * }
+ * + *

Assuming {@code t}, {@code e} and {@code s} are non-null, this returns {@code [t, e, s, t, + * null, null, null]} regardless of the true comparison order of those three values (which might + * not even implement {@link Comparable} at all). + * + *

Warning: by definition, this comparator is not consistent with equals (as + * defined {@linkplain Comparator here}). Avoid its use in APIs, such as {@link + * TreeSet#TreeSet(Comparator)}, where such consistency is expected. + * + *

The returned comparator is serializable. + * + *

Java 8 users: Use the lambda expression {@code (a, b) -> 0} instead (in certain cases + * you may need to cast that to {@code Comparator}). + * + * @since 13.0 + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static Ordering allEqual() { + return AllEqualOrdering.INSTANCE; + } + + /** + * Returns an ordering that compares objects by the natural ordering of their string + * representations as returned by {@code toString()}. It does not support null values. + * + *

The comparator is serializable. + * + *

Java 8 users: Use {@code Comparator.comparing(Object::toString)} instead. + */ + public static Ordering usingToString() { + return UsingToStringOrdering.INSTANCE; + } + + /** + * Returns an arbitrary ordering over all objects, for which {@code compare(a, b) == 0} implies + * {@code a == b} (identity equality). There is no meaning whatsoever to the order imposed, but it + * is constant for the life of the VM. + * + *

Because the ordering is identity-based, it is not "consistent with {@link + * Object#equals(Object)}" as defined by {@link Comparator}. Use caution when building a {@link + * SortedSet} or {@link SortedMap} from it, as the resulting collection will not behave exactly + * according to spec. + * + *

This ordering is not serializable, as its implementation relies on {@link + * System#identityHashCode(Object)}, so its behavior cannot be preserved across serialization. + * + * @since 2.0 + */ + // TODO(kevinb): copy to Comparators, etc. + public static Ordering arbitrary() { + return ArbitraryOrderingHolder.ARBITRARY_ORDERING; + } + + private static class ArbitraryOrderingHolder { + static final Ordering ARBITRARY_ORDERING = new ArbitraryOrdering(); + } + + + static class ArbitraryOrdering extends Ordering { + + private final AtomicInteger counter = new AtomicInteger(0); + private final ConcurrentMap uids = + Platform.tryWeakKeys(new MapMaker()).makeMap(); + + private Integer getUid(Object obj) { + Integer uid = uids.get(obj); + if (uid == null) { + // One or more integer values could be skipped in the event of a race + // to generate a UID for the same object from multiple threads, but + // that shouldn't be a problem. + uid = counter.getAndIncrement(); + Integer alreadySet = uids.putIfAbsent(obj, uid); + if (alreadySet != null) { + uid = alreadySet; + } + } + return uid; + } + + @Override + public int compare(Object left, Object right) { + if (left == right) { + return 0; + } else if (left == null) { + return -1; + } else if (right == null) { + return 1; + } + int leftCode = identityHashCode(left); + int rightCode = identityHashCode(right); + if (leftCode != rightCode) { + return leftCode < rightCode ? -1 : 1; + } + + // identityHashCode collision (rare, but not as rare as you'd think) + int result = getUid(left).compareTo(getUid(right)); + if (result == 0) { + throw new AssertionError(); // extremely, extremely unlikely. + } + return result; + } + + @Override + public String toString() { + return "Ordering.arbitrary()"; + } + + /* + * We need to be able to mock identityHashCode() calls for tests, because it + * can take 1-10 seconds to find colliding objects. Mocking frameworks that + * can do magic to mock static method calls still can't do so for a system + * class, so we need the indirection. In production, Hotspot should still + * recognize that the call is 1-morphic and should still be willing to + * inline it if necessary. + */ + int identityHashCode(Object object) { + return System.identityHashCode(object); + } + } + + // Constructor + + /** + * Constructs a new instance of this class (only invokable by the subclass constructor, typically + * implicit). + */ + protected Ordering() {} + + // Instance-based factories (and any static equivalents) + + /** + * Returns the reverse of this ordering; the {@code Ordering} equivalent to {@link + * Collections#reverseOrder(Comparator)}. + * + *

Java 8 users: Use {@code thisComparator.reversed()} instead. + */ + // type parameter lets us avoid the extra in statements like: + // Ordering o = Ordering.natural().reverse(); + public Ordering reverse() { + return new ReverseOrdering(this); + } + + /** + * Returns an ordering that treats {@code null} as less than all other values and uses {@code + * this} to compare non-null values. + * + *

Java 8 users: Use {@code Comparator.nullsFirst(thisComparator)} instead. + */ + // type parameter lets us avoid the extra in statements like: + // Ordering o = Ordering.natural().nullsFirst(); + public Ordering nullsFirst() { + return new NullsFirstOrdering(this); + } + + /** + * Returns an ordering that treats {@code null} as greater than all other values and uses this + * ordering to compare non-null values. + * + *

Java 8 users: Use {@code Comparator.nullsLast(thisComparator)} instead. + */ + // type parameter lets us avoid the extra in statements like: + // Ordering o = Ordering.natural().nullsLast(); + public Ordering nullsLast() { + return new NullsLastOrdering(this); + } + + /** + * Returns a new ordering on {@code F} which orders elements by first applying a function to them, + * then comparing those results using {@code this}. For example, to compare objects by their + * string forms, in a case-insensitive manner, use: + * + *

{@code
+   * Ordering.from(String.CASE_INSENSITIVE_ORDER)
+   *     .onResultOf(Functions.toStringFunction())
+   * }
+ * + *

Java 8 users: Use {@code Comparator.comparing(function, thisComparator)} instead (you + * can omit the comparator if it is the natural order). + */ + public Ordering onResultOf(Function function) { + return new ByFunctionOrdering<>(function, this); + } + + Ordering> onKeys() { + return onResultOf(Maps.keyFunction()); + } + + /** + * Returns an ordering which first uses the ordering {@code this}, but which in the event of a + * "tie", then delegates to {@code secondaryComparator}. For example, to sort a bug list first by + * status and second by priority, you might use {@code byStatus.compound(byPriority)}. For a + * compound ordering with three or more components, simply chain multiple calls to this method. + * + *

An ordering produced by this method, or a chain of calls to this method, is equivalent to + * one created using {@link Ordering#compound(Iterable)} on the same component comparators. + * + *

Java 8 users: Use {@code thisComparator.thenComparing(secondaryComparator)} instead. + * Depending on what {@code secondaryComparator} is, one of the other overloads of {@code + * thenComparing} may be even more useful. + */ + public Ordering compound(Comparator secondaryComparator) { + return new CompoundOrdering(this, checkNotNull(secondaryComparator)); + } + + /** + * Returns an ordering which tries each given comparator in order until a non-zero result is + * found, returning that result, and returning zero only if all comparators return zero. The + * returned ordering is based on the state of the {@code comparators} iterable at the time it was + * provided to this method. + * + *

The returned ordering is equivalent to that produced using {@code + * Ordering.from(comp1).compound(comp2).compound(comp3) . . .}. + * + *

Warning: Supplying an argument with undefined iteration order, such as a {@link + * HashSet}, will produce non-deterministic results. + * + *

Java 8 users: Use a chain of calls to {@link Comparator#thenComparing(Comparator)}, + * or {@code comparatorCollection.stream().reduce(Comparator::thenComparing).get()} (if the + * collection might be empty, also provide a default comparator as the {@code identity} parameter + * to {@code reduce}). + * + * @param comparators the comparators to try in order + */ + public static Ordering compound(Iterable> comparators) { + return new CompoundOrdering(comparators); + } + + /** + * Returns a new ordering which sorts iterables by comparing corresponding elements pairwise until + * a nonzero result is found; imposes "dictionary order". If the end of one iterable is reached, + * but not the other, the shorter iterable is considered to be less than the longer one. For + * example, a lexicographical natural ordering over integers considers {@code [] < [1] < [1, 1] < + * [1, 2] < [2]}. + * + *

Note that {@code ordering.lexicographical().reverse()} is not equivalent to {@code + * ordering.reverse().lexicographical()} (consider how each would order {@code [1]} and {@code [1, + * 1]}). + * + *

Java 8 users: Use {@link Comparators#lexicographical(Comparator)} instead. + * + * @since 2.0 + */ + // type parameter lets us avoid the extra in statements like: + // Ordering> o = + // Ordering.natural().lexicographical(); + public Ordering> lexicographical() { + /* + * Note that technically the returned ordering should be capable of + * handling not just {@code Iterable} instances, but also any {@code + * Iterable}. However, the need for this comes up so rarely + * that it doesn't justify making everyone else deal with the very ugly + * wildcard. + */ + return new LexicographicalOrdering(this); + } + + // Regular instance methods + + // Override to add + @Override + public abstract int compare(T left, T right); + + /** + * Returns the least of the specified values according to this ordering. If there are multiple + * least values, the first of those is returned. The iterator will be left exhausted: its {@code + * hasNext()} method will return {@code false}. + * + *

Java 8 users: Continue to use this method for now. After the next release of Guava, + * use {@code Streams.stream(iterator).min(thisComparator).get()} instead (but note that it does + * not guarantee which tied minimum element is returned). + * + * @param iterator the iterator whose minimum element is to be determined + * @throws NoSuchElementException if {@code iterator} is empty + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + * @since 11.0 + */ + public E min(Iterator iterator) { + // let this throw NoSuchElementException as necessary + E minSoFar = iterator.next(); + + while (iterator.hasNext()) { + minSoFar = min(minSoFar, iterator.next()); + } + + return minSoFar; + } + + /** + * Returns the least of the specified values according to this ordering. If there are multiple + * least values, the first of those is returned. + * + *

Java 8 users: If {@code iterable} is a {@link Collection}, use {@code + * Collections.min(collection, thisComparator)} instead. Otherwise, continue to use this method + * for now. After the next release of Guava, use {@code + * Streams.stream(iterable).min(thisComparator).get()} instead. Note that these alternatives do + * not guarantee which tied minimum element is returned) + * + * @param iterable the iterable whose minimum element is to be determined + * @throws NoSuchElementException if {@code iterable} is empty + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + */ + public E min(Iterable iterable) { + return min(iterable.iterator()); + } + + /** + * Returns the lesser of the two values according to this ordering. If the values compare as 0, + * the first is returned. + * + *

Implementation note: this method is invoked by the default implementations of the + * other {@code min} overloads, so overriding it will affect their behavior. + * + *

Java 8 users: Use {@code Collections.min(Arrays.asList(a, b), thisComparator)} + * instead (but note that it does not guarantee which tied minimum element is returned). + * + * @param a value to compare, returned if less than or equal to b. + * @param b value to compare. + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + */ + public E min(E a, E b) { + return (compare(a, b) <= 0) ? a : b; + } + + /** + * Returns the least of the specified values according to this ordering. If there are multiple + * least values, the first of those is returned. + * + *

Java 8 users: Use {@code Collections.min(Arrays.asList(a, b, c...), thisComparator)} + * instead (but note that it does not guarantee which tied minimum element is returned). + * + * @param a value to compare, returned if less than or equal to the rest. + * @param b value to compare + * @param c value to compare + * @param rest values to compare + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + */ + @SuppressWarnings("unchecked") + public E min(E a, E b, E c, E... rest) { + E minSoFar = min(min(a, b), c); + + for (E r : rest) { + minSoFar = min(minSoFar, r); + } + + return minSoFar; + } + + /** + * Returns the greatest of the specified values according to this ordering. If there are multiple + * greatest values, the first of those is returned. The iterator will be left exhausted: its + * {@code hasNext()} method will return {@code false}. + * + *

Java 8 users: Continue to use this method for now. After the next release of Guava, + * use {@code Streams.stream(iterator).max(thisComparator).get()} instead (but note that it does + * not guarantee which tied maximum element is returned). + * + * @param iterator the iterator whose maximum element is to be determined + * @throws NoSuchElementException if {@code iterator} is empty + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + * @since 11.0 + */ + public E max(Iterator iterator) { + // let this throw NoSuchElementException as necessary + E maxSoFar = iterator.next(); + + while (iterator.hasNext()) { + maxSoFar = max(maxSoFar, iterator.next()); + } + + return maxSoFar; + } + + /** + * Returns the greatest of the specified values according to this ordering. If there are multiple + * greatest values, the first of those is returned. + * + *

Java 8 users: If {@code iterable} is a {@link Collection}, use {@code + * Collections.max(collection, thisComparator)} instead. Otherwise, continue to use this method + * for now. After the next release of Guava, use {@code + * Streams.stream(iterable).max(thisComparator).get()} instead. Note that these alternatives do + * not guarantee which tied maximum element is returned) + * + * @param iterable the iterable whose maximum element is to be determined + * @throws NoSuchElementException if {@code iterable} is empty + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + */ + public E max(Iterable iterable) { + return max(iterable.iterator()); + } + + /** + * Returns the greater of the two values according to this ordering. If the values compare as 0, + * the first is returned. + * + *

Implementation note: this method is invoked by the default implementations of the + * other {@code max} overloads, so overriding it will affect their behavior. + * + *

Java 8 users: Use {@code Collections.max(Arrays.asList(a, b), thisComparator)} + * instead (but note that it does not guarantee which tied maximum element is returned). + * + * @param a value to compare, returned if greater than or equal to b. + * @param b value to compare. + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + */ + public E max(E a, E b) { + return (compare(a, b) >= 0) ? a : b; + } + + /** + * Returns the greatest of the specified values according to this ordering. If there are multiple + * greatest values, the first of those is returned. + * + *

Java 8 users: Use {@code Collections.max(Arrays.asList(a, b, c...), thisComparator)} + * instead (but note that it does not guarantee which tied maximum element is returned). + * + * @param a value to compare, returned if greater than or equal to the rest. + * @param b value to compare + * @param c value to compare + * @param rest values to compare + * @throws ClassCastException if the parameters are not mutually comparable under this + * ordering. + */ + @SuppressWarnings("unchecked") + public E max(E a, E b, E c, E... rest) { + E maxSoFar = max(max(a, b), c); + + for (E r : rest) { + maxSoFar = max(maxSoFar, r); + } + + return maxSoFar; + } + + /** + * Returns the {@code k} least elements of the given iterable according to this ordering, in order + * from least to greatest. If there are fewer than {@code k} elements present, all will be + * included. + * + *

The implementation does not necessarily use a stable sorting algorithm; when multiple + * elements are equivalent, it is undefined which will come first. + * + *

Java 8 users: Use {@code Streams.stream(iterable).collect(Comparators.least(k, + * thisComparator))} instead. + * + * @return an immutable {@code RandomAccess} list of the {@code k} least elements in ascending + * order + * @throws IllegalArgumentException if {@code k} is negative + * @since 8.0 + */ + public List leastOf(Iterable iterable, int k) { + if (iterable instanceof Collection) { + Collection collection = (Collection) iterable; + if (collection.size() <= 2L * k) { + // In this case, just dumping the collection to an array and sorting is + // faster than using the implementation for Iterator, which is + // specialized for k much smaller than n. + + @SuppressWarnings({"unchecked", "rawtypes"}) // c only contains E's and doesn't escape + E[] array = (E[]) collection.toArray(); + Arrays.sort(array, this); + if (array.length > k) { + array = Arrays.copyOf(array, k); + } + return Collections.unmodifiableList(Arrays.asList(array)); + } + } + return leastOf(iterable.iterator(), k); + } + + /** + * Returns the {@code k} least elements from the given iterator according to this ordering, in + * order from least to greatest. If there are fewer than {@code k} elements present, all will be + * included. + * + *

The implementation does not necessarily use a stable sorting algorithm; when multiple + * elements are equivalent, it is undefined which will come first. + * + *

Java 8 users: Continue to use this method for now. After the next release of Guava, + * use {@code Streams.stream(iterator).collect(Comparators.least(k, thisComparator))} instead. + * + * @return an immutable {@code RandomAccess} list of the {@code k} least elements in ascending + * order + * @throws IllegalArgumentException if {@code k} is negative + * @since 14.0 + */ + public List leastOf(Iterator iterator, int k) { + checkNotNull(iterator); + checkNonnegative(k, "k"); + + if (k == 0 || !iterator.hasNext()) { + return Collections.emptyList(); + } else if (k >= Integer.MAX_VALUE / 2) { + // k is really large; just do a straightforward sorted-copy-and-sublist + ArrayList list = Lists.newArrayList(iterator); + Collections.sort(list, this); + if (list.size() > k) { + list.subList(k, list.size()).clear(); + } + list.trimToSize(); + return Collections.unmodifiableList(list); + } else { + TopKSelector selector = TopKSelector.least(k, this); + selector.offerAll(iterator); + return selector.topK(); + } + } + + /** + * Returns the {@code k} greatest elements of the given iterable according to this ordering, in + * order from greatest to least. If there are fewer than {@code k} elements present, all will be + * included. + * + *

The implementation does not necessarily use a stable sorting algorithm; when multiple + * elements are equivalent, it is undefined which will come first. + * + *

Java 8 users: Use {@code Streams.stream(iterable).collect(Comparators.greatest(k, + * thisComparator))} instead. + * + * @return an immutable {@code RandomAccess} list of the {@code k} greatest elements in + * descending order + * @throws IllegalArgumentException if {@code k} is negative + * @since 8.0 + */ + public List greatestOf(Iterable iterable, int k) { + // TODO(kevinb): see if delegation is hurting performance noticeably + // TODO(kevinb): if we change this implementation, add full unit tests. + return reverse().leastOf(iterable, k); + } + + /** + * Returns the {@code k} greatest elements from the given iterator according to this ordering, in + * order from greatest to least. If there are fewer than {@code k} elements present, all will be + * included. + * + *

The implementation does not necessarily use a stable sorting algorithm; when multiple + * elements are equivalent, it is undefined which will come first. + * + *

Java 8 users: Continue to use this method for now. After the next release of Guava, + * use {@code Streams.stream(iterator).collect(Comparators.greatest(k, thisComparator))} instead. + * + * @return an immutable {@code RandomAccess} list of the {@code k} greatest elements in + * descending order + * @throws IllegalArgumentException if {@code k} is negative + * @since 14.0 + */ + public List greatestOf(Iterator iterator, int k) { + return reverse().leastOf(iterator, k); + } + + /** + * Returns a mutable list containing {@code elements} sorted by this ordering; use this + * only when the resulting list may need further modification, or may contain {@code null}. The + * input is not modified. The returned list is serializable and has random access. + * + *

Unlike {@link Sets#newTreeSet(Iterable)}, this method does not discard elements that are + * duplicates according to the comparator. The sort performed is stable, meaning that such + * elements will appear in the returned list in the same order they appeared in {@code elements}. + * + *

Performance note: According to our + * benchmarking + * on Open JDK 7, {@link #immutableSortedCopy} generally performs better (in both time and space) + * than this method, and this method in turn generally performs better than copying the list and + * calling {@link Collections#sort(List)}. + */ + // TODO(kevinb): rerun benchmarks including new options + public List sortedCopy(Iterable elements) { + @SuppressWarnings({"unchecked", "rawtypes"}) // does not escape, and contains only E's + E[] array = (E[]) Iterables.toArray(elements); + Arrays.sort(array, this); + return Lists.newArrayList(Arrays.asList(array)); + } + + /** + * Returns an immutable list containing {@code elements} sorted by this ordering. The input + * is not modified. + * + *

Unlike {@link Sets#newTreeSet(Iterable)}, this method does not discard elements that are + * duplicates according to the comparator. The sort performed is stable, meaning that such + * elements will appear in the returned list in the same order they appeared in {@code elements}. + * + *

Performance note: According to our + * benchmarking + * on Open JDK 7, this method is the most efficient way to make a sorted copy of a collection. + * + * @throws NullPointerException if any element of {@code elements} is {@code null} + * @since 3.0 + */ + // TODO(kevinb): rerun benchmarks including new options + public ImmutableList immutableSortedCopy(Iterable elements) { + return ImmutableList.sortedCopyOf(this, elements); + } + + /** + * Returns {@code true} if each element in {@code iterable} after the first is greater than or + * equal to the element that preceded it, according to this ordering. Note that this is always + * true when the iterable has fewer than two elements. + * + *

Java 8 users: Use the equivalent {@link Comparators#isInOrder(Iterable, Comparator)} + * instead, since the rest of {@code Ordering} is mostly obsolete (as explained in the class + * documentation). + */ + public boolean isOrdered(Iterable iterable) { + Iterator it = iterable.iterator(); + if (it.hasNext()) { + T prev = it.next(); + while (it.hasNext()) { + T next = it.next(); + if (compare(prev, next) > 0) { + return false; + } + prev = next; + } + } + return true; + } + + /** + * Returns {@code true} if each element in {@code iterable} after the first is strictly + * greater than the element that preceded it, according to this ordering. Note that this is always + * true when the iterable has fewer than two elements. + * + *

Java 8 users: Use the equivalent {@link Comparators#isInStrictOrder(Iterable, + * Comparator)} instead, since the rest of {@code Ordering} is mostly obsolete (as explained in + * the class documentation). + */ + public boolean isStrictlyOrdered(Iterable iterable) { + Iterator it = iterable.iterator(); + if (it.hasNext()) { + T prev = it.next(); + while (it.hasNext()) { + T next = it.next(); + if (compare(prev, next) >= 0) { + return false; + } + prev = next; + } + } + return true; + } + + /** + * {@link Collections#binarySearch(List, Object, Comparator) Searches} {@code sortedList} for + * {@code key} using the binary search algorithm. The list must be sorted using this ordering. + * + * @param sortedList the list to be searched + * @param key the key to be searched for + * @deprecated Use {@link Collections#binarySearch(List, Object, Comparator)} directly. + */ + @Deprecated + public int binarySearch(List sortedList, T key) { + return Collections.binarySearch(sortedList, key, this); + } + + /** + * Exception thrown by a {@link Ordering#explicit(List)} or {@link Ordering#explicit(Object, + * Object[])} comparator when comparing a value outside the set of values it can compare. + * Extending {@link ClassCastException} may seem odd, but it is required. + */ + + static class IncomparableValueException extends ClassCastException { + final Object value; + + IncomparableValueException(Object value) { + super("Cannot compare value: " + value); + this.value = value; + } + + private static final long serialVersionUID = 0; + } + + // Never make these public + static final int LEFT_IS_GREATER = 1; + static final int RIGHT_IS_GREATER = -1; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/PeekingIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/PeekingIterator.java new file mode 100644 index 0000000000000..28feb60217a85 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/PeekingIterator.java @@ -0,0 +1,69 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Iterator; +import java.util.NoSuchElementException; + +/** + * An iterator that supports a one-element lookahead while iterating. + * + *

See the Guava User Guide article on {@code + * PeekingIterator}. + * + * @author Mick Killianey + * @since 2.0 + */ +public interface PeekingIterator extends Iterator { + /** + * Returns the next element in the iteration, without advancing the iteration. + * + *

Calls to {@code peek()} should not change the state of the iteration, except that it + * may prevent removal of the most recent element via {@link #remove()}. + * + * @throws NoSuchElementException if the iteration has no more elements according to {@link + * #hasNext()} + */ + E peek(); + + /** + * {@inheritDoc} + * + *

The objects returned by consecutive calls to {@link #peek()} then {@link #next()} are + * guaranteed to be equal to each other. + */ + @Override + E next(); + + /** + * {@inheritDoc} + * + *

Implementations may or may not support removal when a call to {@link #peek()} has occurred + * since the most recent call to {@link #next()}. + * + * @throws IllegalStateException if there has been a call to {@link #peek()} since the most recent + * call to {@link #next()} and this implementation does not support this sequence of calls + * (optional) + */ + @Override + void remove(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Platform.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Platform.java new file mode 100644 index 0000000000000..5bf6e3e05f8ad --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Platform.java @@ -0,0 +1,109 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.lang.reflect.Array; +import java.util.Map; +import java.util.Set; + +/** + * Methods factored out so that they can be emulated differently in GWT. + * + * @author Hayward Chan + */ +final class Platform { + /** Returns the platform preferred implementation of a map based on a hash table. */ + static Map newHashMapWithExpectedSize(int expectedSize) { + return Maps.newHashMapWithExpectedSize(expectedSize); + } + + /** + * Returns the platform preferred implementation of an insertion ordered map based on a hash + * table. + */ + static Map newLinkedHashMapWithExpectedSize(int expectedSize) { + return Maps.newLinkedHashMapWithExpectedSize(expectedSize); + } + + /** Returns the platform preferred implementation of a set based on a hash table. */ + static Set newHashSetWithExpectedSize(int expectedSize) { + return Sets.newHashSetWithExpectedSize(expectedSize); + } + + /** + * Returns the platform preferred implementation of an insertion ordered set based on a hash + * table. + */ + static Set newLinkedHashSetWithExpectedSize(int expectedSize) { + return Sets.newLinkedHashSetWithExpectedSize(expectedSize); + } + + /** + * Returns the platform preferred map implementation that preserves insertion order when used only + * for insertions. + */ + static Map preservesInsertionOrderOnPutsMap() { + return Maps.newLinkedHashMap(); + } + + /** + * Returns the platform preferred set implementation that preserves insertion order when used only + * for insertions. + */ + static Set preservesInsertionOrderOnAddsSet() { + return Sets.newLinkedHashSet(); + } + + /** + * Returns a new array of the given length with the same type as a reference array. + * + * @param reference any array of the desired type + * @param length the length of the new array + */ + static T[] newArray(T[] reference, int length) { + Class type = reference.getClass().getComponentType(); + + // the cast is safe because + // result.getClass() == reference.getClass().getComponentType() + @SuppressWarnings({"unchecked", "rawtypes"}) + T[] result = (T[]) Array.newInstance(type, length); + return result; + } + + /** + * Configures the given map maker to use weak keys, if possible; does nothing otherwise (i.e., in + * GWT). This is sometimes acceptable, when only server-side code could generate enough volume + * that reclamation becomes important. + */ + static MapMaker tryWeakKeys(MapMaker mapMaker) { + return mapMaker.weakKeys(); + } + + static int reduceIterationsIfGwt(int iterations) { + return iterations; + } + + static int reduceExponentIfGwt(int exponent) { + return exponent; + } + + private Platform() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Range.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Range.java new file mode 100644 index 0000000000000..fe07c11bc5814 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Range.java @@ -0,0 +1,689 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Equivalence; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.SortedSet; + + +/** + * A range (or "interval") defines the boundaries around a contiguous span of values of some + * {@code Comparable} type; for example, "integers from 1 to 100 inclusive." Note that it is not + * possible to iterate over these contained values. To do so, pass this range instance and an + * appropriate {@link DiscreteDomain} to {@link ContiguousSet#create}. + * + *

Types of ranges

+ * + *

Each end of the range may be bounded or unbounded. If bounded, there is an associated + * endpoint value, and the range is considered to be either open (does not include the + * endpoint) or closed (includes the endpoint) on that side. With three possibilities on each + * side, this yields nine basic types of ranges, enumerated below. (Notation: a square bracket + * ({@code [ ]}) indicates that the range is closed on that side; a parenthesis ({@code ( )}) means + * it is either open or unbounded. The construct {@code {x | statement}} is read "the set of all + * x such that statement.") + * + *

+ * + * + * + *
Range Types
Notation Definition Factory method + *
{@code (a..b)} {@code {x | a < x < b}} {@link Range#open open} + *
{@code [a..b]} {@code {x | a <= x <= b}}{@link Range#closed closed} + *
{@code (a..b]} {@code {x | a < x <= b}} {@link Range#openClosed openClosed} + *
{@code [a..b)} {@code {x | a <= x < b}} {@link Range#closedOpen closedOpen} + *
{@code (a..+∞)} {@code {x | x > a}} {@link Range#greaterThan greaterThan} + *
{@code [a..+∞)} {@code {x | x >= a}} {@link Range#atLeast atLeast} + *
{@code (-∞..b)} {@code {x | x < b}} {@link Range#lessThan lessThan} + *
{@code (-∞..b]} {@code {x | x <= b}} {@link Range#atMost atMost} + *
{@code (-∞..+∞)}{@code {x}} {@link Range#all all} + *
+ * + *
+ * + *

When both endpoints exist, the upper endpoint may not be less than the lower. The endpoints + * may be equal only if at least one of the bounds is closed: + * + *

    + *
  • {@code [a..a]} : a singleton range + *
  • {@code [a..a); (a..a]} : {@linkplain #isEmpty empty} ranges; also valid + *
  • {@code (a..a)} : invalid; an exception will be thrown + *
+ * + *

Warnings

+ * + *
    + *
  • Use immutable value types only, if at all possible. If you must use a mutable type, do + * not allow the endpoint instances to mutate after the range is created! + *
  • Your value type's comparison method should be {@linkplain Comparable consistent with + * equals} if at all possible. Otherwise, be aware that concepts used throughout this + * documentation such as "equal", "same", "unique" and so on actually refer to whether {@link + * Comparable#compareTo compareTo} returns zero, not whether {@link Object#equals equals} + * returns {@code true}. + *
  • A class which implements {@code Comparable} is very broken, and will cause + * undefined horrible things to happen in {@code Range}. For now, the Range API does not + * prevent its use, because this would also rule out all ungenerified (pre-JDK1.5) data types. + * This may change in the future. + *
+ * + *

Other notes

+ * + *
    + *
  • Instances of this type are obtained using the static factory methods in this class. + *
  • Ranges are convex: whenever two values are contained, all values in between them + * must also be contained. More formally, for any {@code c1 <= c2 <= c3} of type {@code C}, + * {@code r.contains(c1) && r.contains(c3)} implies {@code r.contains(c2)}). This means that a + * {@code Range} can never be used to represent, say, "all prime numbers from + * 1 to 100." + *
  • When evaluated as a {@link Predicate}, a range yields the same result as invoking {@link + * #contains}. + *
  • Terminology note: a range {@code a} is said to be the maximal range having property + * P if, for all ranges {@code b} also having property P, {@code a.encloses(b)}. + * Likewise, {@code a} is minimal when {@code b.encloses(a)} for all {@code b} having + * property P. See, for example, the definition of {@link #intersection intersection}. + *
+ * + *

Further reading

+ * + *

See the Guava User Guide article on {@code Range}. + * + * @author Kevin Bourrillion + * @author Gregory Kick + * @since 10.0 + */ +@SuppressWarnings("rawtypes") +public final class Range extends RangeGwtSerializationDependencies + implements Predicate, Serializable { + + static class LowerBoundFn implements Function { + static final LowerBoundFn INSTANCE = new LowerBoundFn(); + + @Override + public Cut apply(Range range) { + return range.lowerBound; + } + } + + static class UpperBoundFn implements Function { + static final UpperBoundFn INSTANCE = new UpperBoundFn(); + + @Override + public Cut apply(Range range) { + return range.upperBound; + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static > Function, Cut> lowerBoundFn() { + return (Function) LowerBoundFn.INSTANCE; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static > Function, Cut> upperBoundFn() { + return (Function) UpperBoundFn.INSTANCE; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + static > Ordering> rangeLexOrdering() { + return (Ordering>) (Ordering) RangeLexOrdering.INSTANCE; + } + + static > Range create(Cut lowerBound, Cut upperBound) { + return new Range(lowerBound, upperBound); + } + + /** + * Returns a range that contains all values strictly greater than {@code lower} and strictly less + * than {@code upper}. + * + * @throws IllegalArgumentException if {@code lower} is greater than or equal to {@code + * upper} + * @since 14.0 + */ + public static > Range open(C lower, C upper) { + return create(Cut.aboveValue(lower), Cut.belowValue(upper)); + } + + /** + * Returns a range that contains all values greater than or equal to {@code lower} and less than + * or equal to {@code upper}. + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 14.0 + */ + public static > Range closed(C lower, C upper) { + return create(Cut.belowValue(lower), Cut.aboveValue(upper)); + } + + /** + * Returns a range that contains all values greater than or equal to {@code lower} and strictly + * less than {@code upper}. + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 14.0 + */ + public static > Range closedOpen(C lower, C upper) { + return create(Cut.belowValue(lower), Cut.belowValue(upper)); + } + + /** + * Returns a range that contains all values strictly greater than {@code lower} and less than or + * equal to {@code upper}. + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 14.0 + */ + public static > Range openClosed(C lower, C upper) { + return create(Cut.aboveValue(lower), Cut.aboveValue(upper)); + } + + /** + * Returns a range that contains any value from {@code lower} to {@code upper}, where each + * endpoint may be either inclusive (closed) or exclusive (open). + * + * @throws IllegalArgumentException if {@code lower} is greater than {@code upper} + * @since 14.0 + */ + public static > Range range( + C lower, BoundType lowerType, C upper, BoundType upperType) { + checkNotNull(lowerType); + checkNotNull(upperType); + + Cut lowerBound = + (lowerType == BoundType.OPEN) ? Cut.aboveValue(lower) : Cut.belowValue(lower); + Cut upperBound = + (upperType == BoundType.OPEN) ? Cut.belowValue(upper) : Cut.aboveValue(upper); + return create(lowerBound, upperBound); + } + + /** + * Returns a range that contains all values strictly less than {@code endpoint}. + * + * @since 14.0 + */ + public static > Range lessThan(C endpoint) { + return create(Cut.belowAll(), Cut.belowValue(endpoint)); + } + + /** + * Returns a range that contains all values less than or equal to {@code endpoint}. + * + * @since 14.0 + */ + public static > Range atMost(C endpoint) { + return create(Cut.belowAll(), Cut.aboveValue(endpoint)); + } + + /** + * Returns a range with no lower bound up to the given endpoint, which may be either inclusive + * (closed) or exclusive (open). + * + * @since 14.0 + */ + public static > Range upTo(C endpoint, BoundType boundType) { + switch (boundType) { + case OPEN: + return lessThan(endpoint); + case CLOSED: + return atMost(endpoint); + default: + throw new AssertionError(); + } + } + + /** + * Returns a range that contains all values strictly greater than {@code endpoint}. + * + * @since 14.0 + */ + public static > Range greaterThan(C endpoint) { + return create(Cut.aboveValue(endpoint), Cut.aboveAll()); + } + + /** + * Returns a range that contains all values greater than or equal to {@code endpoint}. + * + * @since 14.0 + */ + public static > Range atLeast(C endpoint) { + return create(Cut.belowValue(endpoint), Cut.aboveAll()); + } + + /** + * Returns a range from the given endpoint, which may be either inclusive (closed) or exclusive + * (open), with no upper bound. + * + * @since 14.0 + */ + public static > Range downTo(C endpoint, BoundType boundType) { + switch (boundType) { + case OPEN: + return greaterThan(endpoint); + case CLOSED: + return atLeast(endpoint); + default: + throw new AssertionError(); + } + } + + private static final Range ALL = new Range<>(Cut.belowAll(), Cut.aboveAll()); + + /** + * Returns a range that contains every value of type {@code C}. + * + * @since 14.0 + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > Range all() { + return (Range) ALL; + } + + /** + * Returns a range that {@linkplain Range#contains(Comparable) contains} only the given value. The + * returned range is {@linkplain BoundType#CLOSED closed} on both ends. + * + * @since 14.0 + */ + public static > Range singleton(C value) { + return closed(value, value); + } + + /** + * Returns the minimal range that {@linkplain Range#contains(Comparable) contains} all of the + * given values. The returned range is {@linkplain BoundType#CLOSED closed} on both ends. + * + * @throws ClassCastException if the parameters are not mutually comparable + * @throws NoSuchElementException if {@code values} is empty + * @throws NullPointerException if any of {@code values} is null + * @since 14.0 + */ + public static > Range encloseAll(Iterable values) { + checkNotNull(values); + if (values instanceof SortedSet) { + SortedSet set = cast(values); + Comparator comparator = set.comparator(); + if (Ordering.natural().equals(comparator) || comparator == null) { + return closed(set.first(), set.last()); + } + } + Iterator valueIterator = values.iterator(); + C min = checkNotNull(valueIterator.next()); + C max = min; + while (valueIterator.hasNext()) { + C value = checkNotNull(valueIterator.next()); + min = Ordering.natural().min(min, value); + max = Ordering.natural().max(max, value); + } + return closed(min, max); + } + + final Cut lowerBound; + final Cut upperBound; + + private Range(Cut lowerBound, Cut upperBound) { + this.lowerBound = checkNotNull(lowerBound); + this.upperBound = checkNotNull(upperBound); + if (lowerBound.compareTo(upperBound) > 0 + || lowerBound == Cut.aboveAll() + || upperBound == Cut.belowAll()) { + throw new IllegalArgumentException("Invalid range: " + toString(lowerBound, upperBound)); + } + } + + /** Returns {@code true} if this range has a lower endpoint. */ + public boolean hasLowerBound() { + return lowerBound != Cut.belowAll(); + } + + /** + * Returns the lower endpoint of this range. + * + * @throws IllegalStateException if this range is unbounded below (that is, {@link + * #hasLowerBound()} returns {@code false}) + */ + public C lowerEndpoint() { + return lowerBound.endpoint(); + } + + /** + * Returns the type of this range's lower bound: {@link BoundType#CLOSED} if the range includes + * its lower endpoint, {@link BoundType#OPEN} if it does not. + * + * @throws IllegalStateException if this range is unbounded below (that is, {@link + * #hasLowerBound()} returns {@code false}) + */ + public BoundType lowerBoundType() { + return lowerBound.typeAsLowerBound(); + } + + /** Returns {@code true} if this range has an upper endpoint. */ + public boolean hasUpperBound() { + return upperBound != Cut.aboveAll(); + } + + /** + * Returns the upper endpoint of this range. + * + * @throws IllegalStateException if this range is unbounded above (that is, {@link + * #hasUpperBound()} returns {@code false}) + */ + public C upperEndpoint() { + return upperBound.endpoint(); + } + + /** + * Returns the type of this range's upper bound: {@link BoundType#CLOSED} if the range includes + * its upper endpoint, {@link BoundType#OPEN} if it does not. + * + * @throws IllegalStateException if this range is unbounded above (that is, {@link + * #hasUpperBound()} returns {@code false}) + */ + public BoundType upperBoundType() { + return upperBound.typeAsUpperBound(); + } + + /** + * Returns {@code true} if this range is of the form {@code [v..v)} or {@code (v..v]}. (This does + * not encompass ranges of the form {@code (v..v)}, because such ranges are invalid and + * can't be constructed at all.) + * + *

Note that certain discrete ranges such as the integer range {@code (3..4)} are not + * considered empty, even though they contain no actual values. In these cases, it may be helpful + * to preprocess ranges with {@link #canonical(DiscreteDomain)}. + */ + public boolean isEmpty() { + return lowerBound.equals(upperBound); + } + + /** + * Returns {@code true} if {@code value} is within the bounds of this range. For example, on the + * range {@code [0..2)}, {@code contains(1)} returns {@code true}, while {@code contains(2)} + * returns {@code false}. + */ + public boolean contains(C value) { + checkNotNull(value); + // let this throw CCE if there is some trickery going on + return lowerBound.isLessThan(value) && !upperBound.isLessThan(value); + } + + /** + * @deprecated Provided only to satisfy the {@link Predicate} interface; use {@link #contains} + * instead. + */ + @Deprecated + @Override + public boolean apply(C input) { + return contains(input); + } + + /** + * Returns {@code true} if every element in {@code values} is {@linkplain #contains contained} in + * this range. + */ + public boolean containsAll(Iterable values) { + if (Iterables.isEmpty(values)) { + return true; + } + + // this optimizes testing equality of two range-backed sets + if (values instanceof SortedSet) { + SortedSet set = cast(values); + Comparator comparator = set.comparator(); + if (Ordering.natural().equals(comparator) || comparator == null) { + return contains(set.first()) && contains(set.last()); + } + } + + for (C value : values) { + if (!contains(value)) { + return false; + } + } + return true; + } + + /** + * Returns {@code true} if the bounds of {@code other} do not extend outside the bounds of this + * range. Examples: + * + *

    + *
  • {@code [3..6]} encloses {@code [4..5]} + *
  • {@code (3..6)} encloses {@code (3..6)} + *
  • {@code [3..6]} encloses {@code [4..4)} (even though the latter is empty) + *
  • {@code (3..6]} does not enclose {@code [3..6]} + *
  • {@code [4..5]} does not enclose {@code (3..6)} (even though it contains every value + * contained by the latter range) + *
  • {@code [3..6]} does not enclose {@code (1..1]} (even though it contains every value + * contained by the latter range) + *
+ * + *

Note that if {@code a.encloses(b)}, then {@code b.contains(v)} implies {@code + * a.contains(v)}, but as the last two examples illustrate, the converse is not always true. + * + *

Being reflexive, antisymmetric and transitive, the {@code encloses} relation defines a + * partial order over ranges. There exists a unique {@linkplain Range#all maximal} range + * according to this relation, and also numerous {@linkplain #isEmpty minimal} ranges. Enclosure + * also implies {@linkplain #isConnected connectedness}. + */ + public boolean encloses(Range other) { + return lowerBound.compareTo(other.lowerBound) <= 0 + && upperBound.compareTo(other.upperBound) >= 0; + } + + /** + * Returns {@code true} if there exists a (possibly empty) range which is {@linkplain #encloses + * enclosed} by both this range and {@code other}. + * + *

For example, + * + *

    + *
  • {@code [2, 4)} and {@code [5, 7)} are not connected + *
  • {@code [2, 4)} and {@code [3, 5)} are connected, because both enclose {@code [3, 4)} + *
  • {@code [2, 4)} and {@code [4, 6)} are connected, because both enclose the empty range + * {@code [4, 4)} + *
+ * + *

Note that this range and {@code other} have a well-defined {@linkplain #span union} and + * {@linkplain #intersection intersection} (as a single, possibly-empty range) if and only if this + * method returns {@code true}. + * + *

The connectedness relation is both reflexive and symmetric, but does not form an {@linkplain + * Equivalence equivalence relation} as it is not transitive. + * + *

Note that certain discrete ranges are not considered connected, even though there are no + * elements "between them." For example, {@code [3, 5]} is not considered connected to {@code [6, + * 10]}. In these cases, it may be desirable for both input ranges to be preprocessed with {@link + * #canonical(DiscreteDomain)} before testing for connectedness. + */ + public boolean isConnected(Range other) { + return lowerBound.compareTo(other.upperBound) <= 0 + && other.lowerBound.compareTo(upperBound) <= 0; + } + + /** + * Returns the maximal range {@linkplain #encloses enclosed} by both this range and {@code + * connectedRange}, if such a range exists. + * + *

For example, the intersection of {@code [1..5]} and {@code (3..7)} is {@code (3..5]}. The + * resulting range may be empty; for example, {@code [1..5)} intersected with {@code [5..7)} + * yields the empty range {@code [5..5)}. + * + *

The intersection exists if and only if the two ranges are {@linkplain #isConnected + * connected}. + * + *

The intersection operation is commutative, associative and idempotent, and its identity + * element is {@link Range#all}). + * + * @throws IllegalArgumentException if {@code isConnected(connectedRange)} is {@code false} + */ + public Range intersection(Range connectedRange) { + int lowerCmp = lowerBound.compareTo(connectedRange.lowerBound); + int upperCmp = upperBound.compareTo(connectedRange.upperBound); + if (lowerCmp >= 0 && upperCmp <= 0) { + return this; + } else if (lowerCmp <= 0 && upperCmp >= 0) { + return connectedRange; + } else { + Cut newLower = (lowerCmp >= 0) ? lowerBound : connectedRange.lowerBound; + Cut newUpper = (upperCmp <= 0) ? upperBound : connectedRange.upperBound; + return create(newLower, newUpper); + } + } + + /** + * Returns the minimal range that {@linkplain #encloses encloses} both this range and {@code + * other}. For example, the span of {@code [1..3]} and {@code (5..7)} is {@code [1..7)}. + * + *

If the input ranges are {@linkplain #isConnected connected}, the returned range can + * also be called their union. If they are not, note that the span might contain values + * that are not contained in either input range. + * + *

Like {@link #intersection(Range) intersection}, this operation is commutative, associative + * and idempotent. Unlike it, it is always well-defined for any two input ranges. + */ + public Range span(Range other) { + int lowerCmp = lowerBound.compareTo(other.lowerBound); + int upperCmp = upperBound.compareTo(other.upperBound); + if (lowerCmp <= 0 && upperCmp >= 0) { + return this; + } else if (lowerCmp >= 0 && upperCmp <= 0) { + return other; + } else { + Cut newLower = (lowerCmp <= 0) ? lowerBound : other.lowerBound; + Cut newUpper = (upperCmp >= 0) ? upperBound : other.upperBound; + return create(newLower, newUpper); + } + } + + /** + * Returns the canonical form of this range in the given domain. The canonical form has the + * following properties: + * + *

    + *
  • equivalence: {@code a.canonical().contains(v) == a.contains(v)} for all {@code v} (in + * other words, {@code ContiguousSet.create(a.canonical(domain), domain).equals( + * ContiguousSet.create(a, domain))} + *
  • uniqueness: unless {@code a.isEmpty()}, {@code ContiguousSet.create(a, + * domain).equals(ContiguousSet.create(b, domain))} implies {@code + * a.canonical(domain).equals(b.canonical(domain))} + *
  • idempotence: {@code a.canonical(domain).canonical(domain).equals(a.canonical(domain))} + *
+ * + *

Furthermore, this method guarantees that the range returned will be one of the following + * canonical forms: + * + *

    + *
  • [start..end) + *
  • [start..+∞) + *
  • (-∞..end) (only if type {@code C} is unbounded below) + *
  • (-∞..+∞) (only if type {@code C} is unbounded below) + *
+ */ + public Range canonical(DiscreteDomain domain) { + checkNotNull(domain); + Cut lower = lowerBound.canonical(domain); + Cut upper = upperBound.canonical(domain); + return (lower == lowerBound && upper == upperBound) ? this : create(lower, upper); + } + + /** + * Returns {@code true} if {@code object} is a range having the same endpoints and bound types as + * this range. Note that discrete ranges such as {@code (1..4)} and {@code [2..3]} are not + * equal to one another, despite the fact that they each contain precisely the same set of values. + * Similarly, empty ranges are not equal unless they have exactly the same representation, so + * {@code [3..3)}, {@code (3..3]}, {@code (4..4]} are all unequal. + */ + @Override + public boolean equals(Object object) { + if (object instanceof Range) { + Range other = (Range) object; + return lowerBound.equals(other.lowerBound) && upperBound.equals(other.upperBound); + } + return false; + } + + /** Returns a hash code for this range. */ + @Override + public int hashCode() { + return lowerBound.hashCode() * 31 + upperBound.hashCode(); + } + + /** + * Returns a string representation of this range, such as {@code "[3..5)"} (other examples are + * listed in the class documentation). + */ + @Override + public String toString() { + return toString(lowerBound, upperBound); + } + + private static String toString(Cut lowerBound, Cut upperBound) { + StringBuilder sb = new StringBuilder(16); + lowerBound.describeAsLowerBound(sb); + sb.append(".."); + upperBound.describeAsUpperBound(sb); + return sb.toString(); + } + + /** Used to avoid http://bugs.sun.com/view_bug.do?bug_id=6558557 */ + private static SortedSet cast(Iterable iterable) { + return (SortedSet) iterable; + } + + Object readResolve() { + if (this.equals(ALL)) { + return all(); + } else { + return this; + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // this method may throw CCE + static int compareOrThrow(Comparable left, Comparable right) { + return left.compareTo(right); + } + + /** Needed to serialize sorted collections of Ranges. */ + private static class RangeLexOrdering extends Ordering> implements Serializable { + static final Ordering> INSTANCE = new RangeLexOrdering(); + + @Override + public int compare(Range left, Range right) { + return ComparisonChain.start() + .compare(left.lowerBound, right.lowerBound) + .compare(left.upperBound, right.upperBound) + .result(); + } + + private static final long serialVersionUID = 0; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeGwtSerializationDependencies.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeGwtSerializationDependencies.java new file mode 100644 index 0000000000000..ccacbb97ec8f0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeGwtSerializationDependencies.java @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2016 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + +/** + * A dummy superclass to support GWT serialization of the element type of a {@link Range}. The GWT + * supersource for this class contains a field of type {@code C}. + * + *

For details about this hack, see {@link GwtSerializationDependencies}, which takes the same + * approach but with a subclass rather than a superclass. + * + *

TODO(cpovirk): Consider applying this subclass approach to our other types. + */ +@SuppressWarnings("rawtypes") +abstract class RangeGwtSerializationDependencies implements Serializable {} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeMap.java new file mode 100644 index 0000000000000..42bad7354a52e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeMap.java @@ -0,0 +1,166 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Collection; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; + + +/** + * A mapping from disjoint nonempty ranges to non-null values. Queries look up the value associated + * with the range (if any) that contains a specified key. + * + *

In contrast to {@link RangeSet}, no "coalescing" is done of {@linkplain + * Range#isConnected(Range) connected} ranges, even if they are mapped to the same value. + * + * @author Louis Wasserman + * @since 14.0 + */ + +@SuppressWarnings("rawtypes") +public interface RangeMap { + /** + * Returns the value associated with the specified key, or {@code null} if there is no such value. + * + *

Specifically, if any range in this range map contains the specified key, the value + * associated with that range is returned. + */ + + V get(K key); + + /** + * Returns the range containing this key and its associated value, if such a range is present in + * the range map, or {@code null} otherwise. + */ + + Entry, V> getEntry(K key); + + /** + * Returns the minimal range {@linkplain Range#encloses(Range) enclosing} the ranges in this + * {@code RangeMap}. + * + * @throws NoSuchElementException if this range map is empty + */ + Range span(); + + /** + * Maps a range to a specified value (optional operation). + * + *

Specifically, after a call to {@code put(range, value)}, if {@link + * Range#contains(Comparable) range.contains(k)}, then {@link #get(Comparable) get(k)} will return + * {@code value}. + * + *

If {@code range} {@linkplain Range#isEmpty() is empty}, then this is a no-op. + */ + void put(Range range, V value); + + /** + * Maps a range to a specified value, coalescing this range with any existing ranges with the same + * value that are {@linkplain Range#isConnected connected} to this range. + * + *

The behavior of {@link #get(Comparable) get(k)} after calling this method is identical to + * the behavior described in {@link #put(Range, Object) put(range, value)}, however the ranges + * returned from {@link #asMapOfRanges} will be different if there were existing entries which + * connect to the given range and value. + * + *

Even if the input range is empty, if it is connected on both sides by ranges mapped to the + * same value those two ranges will be coalesced. + * + *

Note: coalescing requires calling {@code .equals()} on any connected values, which + * may be expensive depending on the value type. Using this method on range maps with large values + * such as {@link Collection} types is discouraged. + * + * @since 22.0 + */ + void putCoalescing(Range range, V value); + + /** Puts all the associations from {@code rangeMap} into this range map (optional operation). */ + void putAll(RangeMap rangeMap); + + /** Removes all associations from this range map (optional operation). */ + void clear(); + + /** + * Removes all associations from this range map in the specified range (optional operation). + * + *

If {@code !range.contains(k)}, {@link #get(Comparable) get(k)} will return the same result + * before and after a call to {@code remove(range)}. If {@code range.contains(k)}, then after a + * call to {@code remove(range)}, {@code get(k)} will return {@code null}. + */ + void remove(Range range); + + /** + * Returns a view of this range map as an unmodifiable {@code Map, V>}. Modifications to + * this range map are guaranteed to read through to the returned {@code Map}. + * + *

The returned {@code Map} iterates over entries in ascending order of the bounds of the + * {@code Range} entries. + * + *

It is guaranteed that no empty ranges will be in the returned {@code Map}. + */ + Map, V> asMapOfRanges(); + + /** + * Returns a view of this range map as an unmodifiable {@code Map, V>}. Modifications to + * this range map are guaranteed to read through to the returned {@code Map}. + * + *

The returned {@code Map} iterates over entries in descending order of the bounds of the + * {@code Range} entries. + * + *

It is guaranteed that no empty ranges will be in the returned {@code Map}. + * + * @since 19.0 + */ + Map, V> asDescendingMapOfRanges(); + + /** + * Returns a view of the part of this range map that intersects with {@code range}. + * + *

For example, if {@code rangeMap} had the entries {@code [1, 5] => "foo", (6, 8) => "bar", + * (10, ∞) => "baz"} then {@code rangeMap.subRangeMap(Range.open(3, 12))} would return a range map + * with the entries {@code (3, 5] => "foo", (6, 8) => "bar", (10, 12) => "baz"}. + * + *

The returned range map supports all optional operations that this range map supports, except + * for {@code asMapOfRanges().iterator().remove()}. + * + *

The returned range map will throw an {@link IllegalArgumentException} on an attempt to + * insert a range not {@linkplain Range#encloses(Range) enclosed} by {@code range}. + */ + RangeMap subRangeMap(Range range); + + /** + * Returns {@code true} if {@code obj} is another {@code RangeMap} that has an equivalent {@link + * #asMapOfRanges()}. + */ + @Override + boolean equals(Object o); + + /** Returns {@code asMapOfRanges().hashCode()}. */ + @Override + int hashCode(); + + /** Returns a readable string representation of this range map. */ + @Override + String toString(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeSet.java new file mode 100644 index 0000000000000..338cbc46c82ce --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RangeSet.java @@ -0,0 +1,279 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.NoSuchElementException; +import java.util.Set; + + +/** + * A set comprising zero or more {@linkplain Range#isEmpty nonempty}, {@linkplain + * Range#isConnected(Range) disconnected} ranges of type {@code C}. + * + *

Implementations that choose to support the {@link #add(Range)} operation are required to + * ignore empty ranges and coalesce connected ranges. For example: + * + *

{@code
+ * RangeSet rangeSet = TreeRangeSet.create();
+ * rangeSet.add(Range.closed(1, 10)); // {[1, 10]}
+ * rangeSet.add(Range.closedOpen(11, 15)); // disconnected range; {[1, 10], [11, 15)}
+ * rangeSet.add(Range.closedOpen(15, 20)); // connected range; {[1, 10], [11, 20)}
+ * rangeSet.add(Range.openClosed(0, 0)); // empty range; {[1, 10], [11, 20)}
+ * rangeSet.remove(Range.open(5, 10)); // splits [1, 10]; {[1, 5], [10, 10], [11, 20)}
+ * }
+ * + *

Note that the behavior of {@link Range#isEmpty()} and {@link Range#isConnected(Range)} may not + * be as expected on discrete ranges. See the Javadoc of those methods for details. + * + *

For a {@link Set} whose contents are specified by a {@link Range}, see {@link ContiguousSet}. + * + *

See the Guava User Guide article on RangeSets. + * + * @author Kevin Bourrillion + * @author Louis Wasserman + * @since 14.0 + */ + +@SuppressWarnings("rawtypes") +public interface RangeSet { + // TODO(lowasser): consider adding default implementations of some of these methods + + // Query methods + + /** Determines whether any of this range set's member ranges contains {@code value}. */ + boolean contains(C value); + + /** + * Returns the unique range from this range set that {@linkplain Range#contains contains} {@code + * value}, or {@code null} if this range set does not contain {@code value}. + */ + Range rangeContaining(C value); + + /** + * Returns {@code true} if there exists a non-empty range enclosed by both a member range in this + * range set and the specified range. This is equivalent to calling {@code + * subRangeSet(otherRange)} and testing whether the resulting range set is non-empty. + * + * @since 20.0 + */ + boolean intersects(Range otherRange); + + /** + * Returns {@code true} if there exists a member range in this range set which {@linkplain + * Range#encloses encloses} the specified range. + */ + boolean encloses(Range otherRange); + + /** + * Returns {@code true} if for each member range in {@code other} there exists a member range in + * this range set which {@linkplain Range#encloses encloses} it. It follows that {@code + * this.contains(value)} whenever {@code other.contains(value)}. Returns {@code true} if {@code + * other} is empty. + * + *

This is equivalent to checking if this range set {@link #encloses} each of the ranges in + * {@code other}. + */ + boolean enclosesAll(RangeSet other); + + /** + * Returns {@code true} if for each range in {@code other} there exists a member range in this + * range set which {@linkplain Range#encloses encloses} it. Returns {@code true} if {@code other} + * is empty. + * + *

This is equivalent to checking if this range set {@link #encloses} each range in {@code + * other}. + * + * @since 21.0 + */ + default boolean enclosesAll(Iterable> other) { + for (Range range : other) { + if (!encloses(range)) { + return false; + } + } + return true; + } + + /** Returns {@code true} if this range set contains no ranges. */ + boolean isEmpty(); + + /** + * Returns the minimal range which {@linkplain Range#encloses(Range) encloses} all ranges in this + * range set. + * + * @throws NoSuchElementException if this range set is {@linkplain #isEmpty() empty} + */ + Range span(); + + // Views + + /** + * Returns a view of the {@linkplain Range#isConnected disconnected} ranges that make up this + * range set. The returned set may be empty. The iterators returned by its {@link + * Iterable#iterator} method return the ranges in increasing order of lower bound (equivalently, + * of upper bound). + */ + Set> asRanges(); + + /** + * Returns a descending view of the {@linkplain Range#isConnected disconnected} ranges that make + * up this range set. The returned set may be empty. The iterators returned by its {@link + * Iterable#iterator} method return the ranges in decreasing order of lower bound (equivalently, + * of upper bound). + * + * @since 19.0 + */ + Set> asDescendingSetOfRanges(); + + /** + * Returns a view of the complement of this {@code RangeSet}. + * + *

The returned view supports the {@link #add} operation if this {@code RangeSet} supports + * {@link #remove}, and vice versa. + */ + RangeSet complement(); + + /** + * Returns a view of the intersection of this {@code RangeSet} with the specified range. + * + *

The returned view supports all optional operations supported by this {@code RangeSet}, with + * the caveat that an {@link IllegalArgumentException} is thrown on an attempt to {@linkplain + * #add(Range) add} any range not {@linkplain Range#encloses(Range) enclosed} by {@code view}. + */ + RangeSet subRangeSet(Range view); + + // Modification + + /** + * Adds the specified range to this {@code RangeSet} (optional operation). That is, for equal + * range sets a and b, the result of {@code a.add(range)} is that {@code a} will be the minimal + * range set for which both {@code a.enclosesAll(b)} and {@code a.encloses(range)}. + * + *

Note that {@code range} will be {@linkplain Range#span(Range) coalesced} with any ranges in + * the range set that are {@linkplain Range#isConnected(Range) connected} with it. Moreover, if + * {@code range} is empty, this is a no-op. + * + * @throws UnsupportedOperationException if this range set does not support the {@code add} + * operation + */ + void add(Range range); + + /** + * Removes the specified range from this {@code RangeSet} (optional operation). After this + * operation, if {@code range.contains(c)}, {@code this.contains(c)} will return {@code false}. + * + *

If {@code range} is empty, this is a no-op. + * + * @throws UnsupportedOperationException if this range set does not support the {@code remove} + * operation + */ + void remove(Range range); + + /** + * Removes all ranges from this {@code RangeSet} (optional operation). After this operation, + * {@code this.contains(c)} will return false for all {@code c}. + * + *

This is equivalent to {@code remove(Range.all())}. + * + * @throws UnsupportedOperationException if this range set does not support the {@code clear} + * operation + */ + void clear(); + + /** + * Adds all of the ranges from the specified range set to this range set (optional operation). + * After this operation, this range set is the minimal range set that {@linkplain + * #enclosesAll(RangeSet) encloses} both the original range set and {@code other}. + * + *

This is equivalent to calling {@link #add} on each of the ranges in {@code other} in turn. + * + * @throws UnsupportedOperationException if this range set does not support the {@code addAll} + * operation + */ + void addAll(RangeSet other); + + /** + * Adds all of the specified ranges to this range set (optional operation). After this operation, + * this range set is the minimal range set that {@linkplain #enclosesAll(RangeSet) encloses} both + * the original range set and each range in {@code other}. + * + *

This is equivalent to calling {@link #add} on each of the ranges in {@code other} in turn. + * + * @throws UnsupportedOperationException if this range set does not support the {@code addAll} + * operation + * @since 21.0 + */ + default void addAll(Iterable> ranges) { + for (Range range : ranges) { + add(range); + } + } + + /** + * Removes all of the ranges from the specified range set from this range set (optional + * operation). After this operation, if {@code other.contains(c)}, {@code this.contains(c)} will + * return {@code false}. + * + *

This is equivalent to calling {@link #remove} on each of the ranges in {@code other} in + * turn. + * + * @throws UnsupportedOperationException if this range set does not support the {@code removeAll} + * operation + */ + void removeAll(RangeSet other); + + /** + * Removes all of the specified ranges from this range set (optional operation). + * + *

This is equivalent to calling {@link #remove} on each of the ranges in {@code other} in + * turn. + * + * @throws UnsupportedOperationException if this range set does not support the {@code removeAll} + * operation + * @since 21.0 + */ + default void removeAll(Iterable> ranges) { + for (Range range : ranges) { + remove(range); + } + } + + // Object methods + + /** + * Returns {@code true} if {@code obj} is another {@code RangeSet} that contains the same ranges + * according to {@link Range#equals(Object)}. + */ + @Override + boolean equals(Object obj); + + /** Returns {@code asRanges().hashCode()}. */ + @Override + int hashCode(); + + /** + * Returns a readable string representation of this range set. For example, if this {@code + * RangeSet} consisted of {@code Range.closed(1, 3)} and {@code Range.greaterThan(4)}, this might + * return {@code " [1..3](4..+∞)}"}. + */ + @Override + String toString(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularContiguousSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularContiguousSet.java new file mode 100644 index 0000000000000..116ee971e4a5d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularContiguousSet.java @@ -0,0 +1,235 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.BoundType.CLOSED; + +import java.io.Serializable; +import java.util.Collection; + + +/** + * An implementation of {@link ContiguousSet} that contains one or more elements. + * + * @author Gregory Kick + */ +@SuppressWarnings({"unchecked", "rawtypes"}) // allow ungenerified Comparable types +final class RegularContiguousSet extends ContiguousSet { + private final Range range; + + RegularContiguousSet(Range range, DiscreteDomain domain) { + super(domain); + this.range = range; + } + + private ContiguousSet intersectionInCurrentDomain(Range other) { + return (range.isConnected(other)) + ? ContiguousSet.create(range.intersection(other), domain) + : new EmptyContiguousSet(domain); + } + + @Override + ContiguousSet headSetImpl(C toElement, boolean inclusive) { + return intersectionInCurrentDomain(Range.upTo(toElement, BoundType.forBoolean(inclusive))); + } + + @Override + ContiguousSet subSetImpl( + C fromElement, boolean fromInclusive, C toElement, boolean toInclusive) { + if (fromElement.compareTo(toElement) == 0 && !fromInclusive && !toInclusive) { + // Range would reject our attempt to create (x, x). + return new EmptyContiguousSet(domain); + } + return intersectionInCurrentDomain( + Range.range( + fromElement, BoundType.forBoolean(fromInclusive), + toElement, BoundType.forBoolean(toInclusive))); + } + + @Override + ContiguousSet tailSetImpl(C fromElement, boolean inclusive) { + return intersectionInCurrentDomain(Range.downTo(fromElement, BoundType.forBoolean(inclusive))); + } + + @Override + int indexOf(Object target) { + return contains(target) ? (int) domain.distance(first(), (C) target) : -1; + } + + @Override + public UnmodifiableIterator iterator() { + return new AbstractSequentialIterator(first()) { + final C last = last(); + + @Override + protected C computeNext(C previous) { + return equalsOrThrow(previous, last) ? null : domain.next(previous); + } + }; + } + + @Override + public UnmodifiableIterator descendingIterator() { + return new AbstractSequentialIterator(last()) { + final C first = first(); + + @Override + protected C computeNext(C previous) { + return equalsOrThrow(previous, first) ? null : domain.previous(previous); + } + }; + } + + private static boolean equalsOrThrow(Comparable left, Comparable right) { + return right != null && Range.compareOrThrow(left, right) == 0; + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public C first() { + return range.lowerBound.leastValueAbove(domain); + } + + @Override + public C last() { + return range.upperBound.greatestValueBelow(domain); + } + + @Override + ImmutableList createAsList() { + if (domain.supportsFastOffset) { + return new ImmutableAsList() { + @Override + ImmutableSortedSet delegateCollection() { + return RegularContiguousSet.this; + } + + @Override + public C get(int i) { + checkElementIndex(i, size()); + return domain.offset(first(), i); + } + }; + } else { + return super.createAsList(); + } + } + + @Override + public int size() { + long distance = domain.distance(first(), last()); + return (distance >= Integer.MAX_VALUE) ? Integer.MAX_VALUE : (int) distance + 1; + } + + @Override + public boolean contains(Object object) { + if (object == null) { + return false; + } + try { + return range.contains((C) object); + } catch (ClassCastException e) { + return false; + } + } + + @Override + public boolean containsAll(Collection targets) { + return Collections2.containsAllImpl(this, targets); + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public ContiguousSet intersection(ContiguousSet other) { + checkNotNull(other); + checkArgument(this.domain.equals(other.domain)); + if (other.isEmpty()) { + return other; + } else { + C lowerEndpoint = Ordering.natural().max(this.first(), other.first()); + C upperEndpoint = Ordering.natural().min(this.last(), other.last()); + return (lowerEndpoint.compareTo(upperEndpoint) <= 0) + ? ContiguousSet.create(Range.closed(lowerEndpoint, upperEndpoint), domain) + : new EmptyContiguousSet(domain); + } + } + + @Override + public Range range() { + return range(CLOSED, CLOSED); + } + + @Override + public Range range(BoundType lowerBoundType, BoundType upperBoundType) { + return Range.create( + range.lowerBound.withLowerBoundType(lowerBoundType, domain), + range.upperBound.withUpperBoundType(upperBoundType, domain)); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } else if (object instanceof RegularContiguousSet) { + RegularContiguousSet that = (RegularContiguousSet) object; + if (this.domain.equals(that.domain)) { + return this.first().equals(that.first()) && this.last().equals(that.last()); + } + } + return super.equals(object); + } + + // copied to make sure not to use the GWT-emulated version + @Override + public int hashCode() { + return Sets.hashCodeImpl(this); + } + + private static final class SerializedForm implements Serializable { + final Range range; + final DiscreteDomain domain; + + private SerializedForm(Range range, DiscreteDomain domain) { + this.range = range; + this.domain = domain; + } + + private Object readResolve() { + return new RegularContiguousSet(range, domain); + } + } + + @Override + Object writeReplace() { + return new SerializedForm(range, domain); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableAsList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableAsList.java new file mode 100644 index 0000000000000..bdb6962ece159 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableAsList.java @@ -0,0 +1,74 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.function.Consumer; + +/** + * An {@link ImmutableAsList} implementation specialized for when the delegate collection is already + * backed by an {@code ImmutableList} or array. + * + * @author Louis Wasserman + */ +@SuppressWarnings("serial") // uses writeReplace, not default serialization +class RegularImmutableAsList extends ImmutableAsList { + private final ImmutableCollection delegate; + private final ImmutableList delegateList; + + RegularImmutableAsList(ImmutableCollection delegate, ImmutableList delegateList) { + this.delegate = delegate; + this.delegateList = delegateList; + } + + RegularImmutableAsList(ImmutableCollection delegate, Object[] array) { + this(delegate, ImmutableList.asImmutableList(array)); + } + + @Override + ImmutableCollection delegateCollection() { + return delegate; + } + + ImmutableList delegateList() { + return delegateList; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // safe covariant cast! + @Override + public UnmodifiableListIterator listIterator(int index) { + return (UnmodifiableListIterator) delegateList.listIterator(index); + } + + @Override + public void forEach(Consumer action) { + delegateList.forEach(action); + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + return delegateList.copyIntoArray(dst, offset); + } + + @Override + public E get(int index) { + return delegateList.get(index); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableBiMap.java new file mode 100644 index 0000000000000..c5c621912758b --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableBiMap.java @@ -0,0 +1,305 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMapEntry.createEntryArray; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.RegularImmutableMap.checkNoConflictInKeyBucket; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMapEntry.NonTerminalImmutableBiMapEntry; + +import java.io.Serializable; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + + +/** + * Bimap with zero or more mappings. + * + * @author Louis Wasserman + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +class RegularImmutableBiMap extends ImmutableBiMap { + @SuppressWarnings({"unchecked", "rawtypes"}) + static final RegularImmutableBiMap EMPTY = + new RegularImmutableBiMap<>( + null, null, (Entry[]) ImmutableMap.EMPTY_ENTRY_ARRAY, 0, 0); + + static final double MAX_LOAD_FACTOR = 1.2; + + private final transient ImmutableMapEntry[] keyTable; + private final transient ImmutableMapEntry[] valueTable; + final transient Entry[] entries; + private final transient int mask; + private final transient int hashCode; + + @SuppressWarnings("unchecked") + static ImmutableBiMap fromEntries(Entry... entries) { + return fromEntryArray(entries.length, entries); + } + + static ImmutableBiMap fromEntryArray(int n, Entry[] entryArray) { + checkPositionIndex(n, entryArray.length); + int tableSize = Hashing.closedTableSize(n, MAX_LOAD_FACTOR); + int mask = tableSize - 1; + ImmutableMapEntry[] keyTable = createEntryArray(tableSize); + ImmutableMapEntry[] valueTable = createEntryArray(tableSize); + Entry[] entries; + if (n == entryArray.length) { + entries = entryArray; + } else { + entries = createEntryArray(n); + } + int hashCode = 0; + + for (int i = 0; i < n; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) + Entry entry = entryArray[i]; + K key = entry.getKey(); + V value = entry.getValue(); + checkEntryNotNull(key, value); + int keyHash = key.hashCode(); + int valueHash = value.hashCode(); + int keyBucket = Hashing.smear(keyHash) & mask; + int valueBucket = Hashing.smear(valueHash) & mask; + + ImmutableMapEntry nextInKeyBucket = keyTable[keyBucket]; + int keyBucketLength = checkNoConflictInKeyBucket(key, entry, nextInKeyBucket); + ImmutableMapEntry nextInValueBucket = valueTable[valueBucket]; + int valueBucketLength = checkNoConflictInValueBucket(value, entry, nextInValueBucket); + if (keyBucketLength > RegularImmutableMap.MAX_HASH_BUCKET_LENGTH + || valueBucketLength > RegularImmutableMap.MAX_HASH_BUCKET_LENGTH) { + return JdkBackedImmutableBiMap.create(n, entryArray); + } + ImmutableMapEntry newEntry = + (nextInValueBucket == null && nextInKeyBucket == null) + ? RegularImmutableMap.makeImmutable(entry, key, value) + : new NonTerminalImmutableBiMapEntry<>( + key, value, nextInKeyBucket, nextInValueBucket); + keyTable[keyBucket] = newEntry; + valueTable[valueBucket] = newEntry; + entries[i] = newEntry; + hashCode += keyHash ^ valueHash; + } + return new RegularImmutableBiMap<>(keyTable, valueTable, entries, mask, hashCode); + } + + private RegularImmutableBiMap( + ImmutableMapEntry[] keyTable, + ImmutableMapEntry[] valueTable, + Entry[] entries, + int mask, + int hashCode) { + this.keyTable = keyTable; + this.valueTable = valueTable; + this.entries = entries; + this.mask = mask; + this.hashCode = hashCode; + } + + // checkNoConflictInKeyBucket is static imported from RegularImmutableMap + + /** + * @return number of entries in this bucket + * @throws IllegalArgumentException if another entry in the bucket has the same key + */ + private static int checkNoConflictInValueBucket( + Object value, Entry entry, ImmutableMapEntry valueBucketHead) { + int bucketSize = 0; + for (; valueBucketHead != null; valueBucketHead = valueBucketHead.getNextInValueBucket()) { + checkNoConflict(!value.equals(valueBucketHead.getValue()), "value", entry, valueBucketHead); + bucketSize++; + } + return bucketSize; + } + + @Override + + public V get(Object key) { + return (keyTable == null) ? null : RegularImmutableMap.get(key, keyTable, mask); + } + + @Override + ImmutableSet> createEntrySet() { + return isEmpty() + ? ImmutableSet.>of() + : new ImmutableMapEntrySet.RegularEntrySet(this, entries); + } + + @Override + ImmutableSet createKeySet() { + return new ImmutableMapKeySet<>(this); + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + for (Entry entry : entries) { + action.accept(entry.getKey(), entry.getValue()); + } + } + + @Override + boolean isHashCodeFast() { + return true; + } + + @Override + public int hashCode() { + return hashCode; + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public int size() { + return entries.length; + } + + private transient ImmutableBiMap inverse; + + @Override + public ImmutableBiMap inverse() { + if (isEmpty()) { + return ImmutableBiMap.of(); + } + ImmutableBiMap result = inverse; + return (result == null) ? inverse = new Inverse() : result; + } + + private final class Inverse extends ImmutableBiMap { + + @Override + public int size() { + return inverse().size(); + } + + @Override + public ImmutableBiMap inverse() { + return RegularImmutableBiMap.this; + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + RegularImmutableBiMap.this.forEach((k, v) -> action.accept(v, k)); + } + + @Override + public K get(Object value) { + if (value == null || valueTable == null) { + return null; + } + int bucket = Hashing.smear(value.hashCode()) & mask; + for (ImmutableMapEntry entry = valueTable[bucket]; + entry != null; + entry = entry.getNextInValueBucket()) { + if (value.equals(entry.getValue())) { + return entry.getKey(); + } + } + return null; + } + + @Override + ImmutableSet createKeySet() { + return new ImmutableMapKeySet<>(this); + } + + @Override + ImmutableSet> createEntrySet() { + return new InverseEntrySet(); + } + + final class InverseEntrySet extends ImmutableMapEntrySet { + @Override + ImmutableMap map() { + return Inverse.this; + } + + @Override + boolean isHashCodeFast() { + return true; + } + + @Override + public int hashCode() { + return hashCode; + } + + @Override + public UnmodifiableIterator> iterator() { + return asList().iterator(); + } + + @Override + public void forEach(Consumer> action) { + asList().forEach(action); + } + + @Override + ImmutableList> createAsList() { + return new ImmutableAsList>() { + @Override + public Entry get(int index) { + Entry entry = entries[index]; + return Maps.immutableEntry(entry.getValue(), entry.getKey()); + } + + @Override + ImmutableCollection> delegateCollection() { + return InverseEntrySet.this; + } + }; + } + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + Object writeReplace() { + return new InverseSerializedForm<>(RegularImmutableBiMap.this); + } + } + + private static class InverseSerializedForm implements Serializable { + private final ImmutableBiMap forward; + + InverseSerializedForm(ImmutableBiMap forward) { + this.forward = forward; + } + + Object readResolve() { + return forward.inverse(); + } + + private static final long serialVersionUID = 1; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableList.java new file mode 100644 index 0000000000000..49ed796d1744f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableList.java @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Spliterator; +import java.util.Spliterators; + +/** + * Implementation of {@link ImmutableList} backed by a simple array. + * + * @author Kevin Bourrillion + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +class RegularImmutableList extends ImmutableList { + static final ImmutableList EMPTY = new RegularImmutableList<>(new Object[0]); + + final transient Object[] array; + + RegularImmutableList(Object[] array) { + this.array = array; + } + + @Override + public int size() { + return array.length; + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + int copyIntoArray(Object[] dst, int dstOff) { + System.arraycopy(array, 0, dst, dstOff, array.length); + return dstOff + array.length; + } + + // The fake cast to E is safe because the creation methods only allow E's + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public E get(int index) { + return (E) array[index]; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public UnmodifiableListIterator listIterator(int index) { + // for performance + // The fake cast to E is safe because the creation methods only allow E's + return (UnmodifiableListIterator) Iterators.forArray(array, 0, array.length, index); + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(array, SPLITERATOR_CHARACTERISTICS); + } + + // TODO(lowasser): benchmark optimizations for equals() and see if they're worthwhile +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMap.java new file mode 100644 index 0000000000000..ff3dc94642cec --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMap.java @@ -0,0 +1,309 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMapEntry.createEntryArray; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMapEntry.NonTerminalImmutableMapEntry; + +import java.io.Serializable; +import java.util.function.BiConsumer; + + +/** + * Implementation of {@link ImmutableMap} with two or more entries. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + * @author Gregory Kick + */ +final class RegularImmutableMap extends ImmutableMap { + @SuppressWarnings({"unchecked", "rawtypes"}) + static final ImmutableMap EMPTY = + new RegularImmutableMap<>((Entry[]) ImmutableMap.EMPTY_ENTRY_ARRAY, null, 0); + + /** + * Closed addressing tends to perform well even with high load factors. Being conservative here + * ensures that the table is still likely to be relatively sparse (hence it misses fast) while + * saving space. + */ + static final double MAX_LOAD_FACTOR = 1.2; + + /** + * Maximum allowed false positive probability of detecting a hash flooding attack given random + * input. + */ + static final double HASH_FLOODING_FPP = 0.001; + + /** + * Maximum allowed length of a hash table bucket before falling back to a j.u.HashMap based + * implementation. Experimentally determined. + */ + static final int MAX_HASH_BUCKET_LENGTH = 8; + + // entries in insertion order + final transient Entry[] entries; + // array of linked lists of entries + private final transient ImmutableMapEntry[] table; + // 'and' with an int to get a table index + private final transient int mask; + + @SuppressWarnings("unchecked") + static ImmutableMap fromEntries(Entry... entries) { + return fromEntryArray(entries.length, entries); + } + + /** + * Creates an ImmutableMap from the first n entries in entryArray. This implementation may replace + * the entries in entryArray with its own entry objects (though they will have the same key/value + * contents), and may take ownership of entryArray. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + static ImmutableMap fromEntryArray(int n, Entry[] entryArray) { + checkPositionIndex(n, entryArray.length); + if (n == 0) { + return (RegularImmutableMap) EMPTY; + } + Entry[] entries; + if (n == entryArray.length) { + entries = entryArray; + } else { + entries = createEntryArray(n); + } + int tableSize = Hashing.closedTableSize(n, MAX_LOAD_FACTOR); + ImmutableMapEntry[] table = createEntryArray(tableSize); + int mask = tableSize - 1; + for (int entryIndex = 0; entryIndex < n; entryIndex++) { + Entry entry = entryArray[entryIndex]; + K key = entry.getKey(); + V value = entry.getValue(); + checkEntryNotNull(key, value); + int tableIndex = Hashing.smear(key.hashCode()) & mask; + ImmutableMapEntry existing = table[tableIndex]; + // prepend, not append, so the entries can be immutable + ImmutableMapEntry newEntry = + (existing == null) + ? makeImmutable(entry, key, value) + : new NonTerminalImmutableMapEntry(key, value, existing); + table[tableIndex] = newEntry; + entries[entryIndex] = newEntry; + int bucketSize = checkNoConflictInKeyBucket(key, newEntry, existing); + if (bucketSize > MAX_HASH_BUCKET_LENGTH) { + // probable hash flooding attack, fall back to j.u.HM based implementation and use its + // implementation of hash flooding protection + return JdkBackedImmutableMap.create(n, entryArray); + } + } + return new RegularImmutableMap<>(entries, table, mask); + } + + /** Makes an entry usable internally by a new ImmutableMap without rereading its contents. */ + static ImmutableMapEntry makeImmutable(Entry entry, K key, V value) { + boolean reusable = + entry instanceof ImmutableMapEntry && ((ImmutableMapEntry) entry).isReusable(); + return reusable ? (ImmutableMapEntry) entry : new ImmutableMapEntry(key, value); + } + + /** Makes an entry usable internally by a new ImmutableMap. */ + static ImmutableMapEntry makeImmutable(Entry entry) { + return makeImmutable(entry, entry.getKey(), entry.getValue()); + } + + private RegularImmutableMap(Entry[] entries, ImmutableMapEntry[] table, int mask) { + this.entries = entries; + this.table = table; + this.mask = mask; + } + + /** + * @return number of entries in this bucket + * @throws IllegalArgumentException if another entry in the bucket has the same key + */ + static int checkNoConflictInKeyBucket( + Object key, Entry entry, ImmutableMapEntry keyBucketHead) { + int bucketSize = 0; + for (; keyBucketHead != null; keyBucketHead = keyBucketHead.getNextInKeyBucket()) { + checkNoConflict(!key.equals(keyBucketHead.getKey()), "key", entry, keyBucketHead); + bucketSize++; + } + return bucketSize; + } + + @Override + public V get(Object key) { + return get(key, table, mask); + } + + + static V get( + Object key, ImmutableMapEntry[] keyTable, int mask) { + if (key == null || keyTable == null) { + return null; + } + int index = Hashing.smear(key.hashCode()) & mask; + for (ImmutableMapEntry entry = keyTable[index]; + entry != null; + entry = entry.getNextInKeyBucket()) { + Object candidateKey = entry.getKey(); + + /* + * Assume that equals uses the == optimization when appropriate, and that + * it would check hash codes as an optimization when appropriate. If we + * did these things, it would just make things worse for the most + * performance-conscious users. + */ + if (key.equals(candidateKey)) { + return entry.getValue(); + } + } + return null; + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action); + for (Entry entry : entries) { + action.accept(entry.getKey(), entry.getValue()); + } + } + + @Override + public int size() { + return entries.length; + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + ImmutableSet> createEntrySet() { + return new ImmutableMapEntrySet.RegularEntrySet<>(this, entries); + } + + @Override + ImmutableSet createKeySet() { + return new KeySet<>(this); + } + + private static final class KeySet extends IndexedImmutableSet { + private final RegularImmutableMap map; + + KeySet(RegularImmutableMap map) { + this.map = map; + } + + @Override + K get(int index) { + return map.entries[index].getKey(); + } + + @Override + public boolean contains(Object object) { + return map.containsKey(object); + } + + @Override + boolean isPartialView() { + return true; + } + + @Override + public int size() { + return map.size(); + } + + @Override + Object writeReplace() { + return new SerializedForm(map); + } + + private static class SerializedForm implements Serializable { + final ImmutableMap map; + + SerializedForm(ImmutableMap map) { + this.map = map; + } + + Object readResolve() { + return map.keySet(); + } + + private static final long serialVersionUID = 0; + } + } + + @Override + ImmutableCollection createValues() { + return new Values<>(this); + } + + private static final class Values extends ImmutableList { + final RegularImmutableMap map; + + Values(RegularImmutableMap map) { + this.map = map; + } + + @Override + public V get(int index) { + return map.entries[index].getValue(); + } + + @Override + public int size() { + return map.size(); + } + + @Override + boolean isPartialView() { + return true; + } + + @Override + Object writeReplace() { + return new SerializedForm(map); + } + + private static class SerializedForm implements Serializable { + final ImmutableMap map; + + SerializedForm(ImmutableMap map) { + this.map = map; + } + + Object readResolve() { + return map.values(); + } + + private static final long serialVersionUID = 0; + } + } + + // This class is never actually serialized directly, but we have to make the + // warning go away (and suppressing would suppress for all nested classes too) + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMultiset.java new file mode 100644 index 0000000000000..d21d07746e950 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableMultiset.java @@ -0,0 +1,196 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.util.Arrays; +import java.util.Collection; + + +/** + * Implementation of {@link ImmutableMultiset} with zero or more elements. + * + * @author Jared Levy + * @author Louis Wasserman + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +class RegularImmutableMultiset extends ImmutableMultiset { + static final ImmutableMultiset EMPTY = create(ImmutableList.>of()); + + @SuppressWarnings({"unchecked", "rawtypes"}) + static ImmutableMultiset create(Collection> entries) { + int distinct = entries.size(); + @SuppressWarnings({"unchecked", "rawtypes"}) + Multisets.ImmutableEntry[] entryArray = new Multisets.ImmutableEntry[distinct]; + if (distinct == 0) { + return new RegularImmutableMultiset<>(entryArray, null, 0, 0, ImmutableSet.of()); + } + int tableSize = Hashing.closedTableSize(distinct, MAX_LOAD_FACTOR); + int mask = tableSize - 1; + @SuppressWarnings({"unchecked", "rawtypes"}) + Multisets.ImmutableEntry[] hashTable = new Multisets.ImmutableEntry[tableSize]; + + int index = 0; + int hashCode = 0; + long size = 0; + for (Entry entry : entries) { + E element = checkNotNull(entry.getElement()); + int count = entry.getCount(); + int hash = element.hashCode(); + int bucket = Hashing.smear(hash) & mask; + Multisets.ImmutableEntry bucketHead = hashTable[bucket]; + Multisets.ImmutableEntry newEntry; + if (bucketHead == null) { + boolean canReuseEntry = + entry instanceof Multisets.ImmutableEntry && !(entry instanceof NonTerminalEntry); + newEntry = + canReuseEntry + ? (Multisets.ImmutableEntry) entry + : new Multisets.ImmutableEntry(element, count); + } else { + newEntry = new NonTerminalEntry(element, count, bucketHead); + } + hashCode += hash ^ count; + entryArray[index++] = newEntry; + hashTable[bucket] = newEntry; + size += count; + } + + return hashFloodingDetected(hashTable) + ? JdkBackedImmutableMultiset.create(ImmutableList.asImmutableList(entryArray)) + : new RegularImmutableMultiset( + entryArray, hashTable, Ints.saturatedCast(size), hashCode, null); + } + + private static boolean hashFloodingDetected(Multisets.ImmutableEntry[] hashTable) { + for (int i = 0; i < hashTable.length; i++) { + int bucketLength = 0; + for (Multisets.ImmutableEntry entry = hashTable[i]; + entry != null; + entry = entry.nextInBucket()) { + bucketLength++; + if (bucketLength > MAX_HASH_BUCKET_LENGTH) { + return true; + } + } + } + return false; + } + + /** + * Closed addressing tends to perform well even with high load factors. Being conservative here + * ensures that the table is still likely to be relatively sparse (hence it misses fast) while + * saving space. + */ + static final double MAX_LOAD_FACTOR = 1.0; + + /** + * Maximum allowed false positive probability of detecting a hash flooding attack given random + * input. + */ + static final double HASH_FLOODING_FPP = 0.001; + + /** + * Maximum allowed length of a hash table bucket before falling back to a j.u.HashMap based + * implementation. Experimentally determined. + */ + static final int MAX_HASH_BUCKET_LENGTH = 9; + + private final transient Multisets.ImmutableEntry[] entries; + private final transient Multisets.ImmutableEntry[] hashTable; + private final transient int size; + private final transient int hashCode; + + private transient ImmutableSet elementSet; + + private RegularImmutableMultiset( + Multisets.ImmutableEntry[] entries, + Multisets.ImmutableEntry[] hashTable, + int size, + int hashCode, + ImmutableSet elementSet) { + this.entries = entries; + this.hashTable = hashTable; + this.size = size; + this.hashCode = hashCode; + this.elementSet = elementSet; + } + + private static final class NonTerminalEntry extends Multisets.ImmutableEntry { + private final Multisets.ImmutableEntry nextInBucket; + + NonTerminalEntry(E element, int count, Multisets.ImmutableEntry nextInBucket) { + super(element, count); + this.nextInBucket = nextInBucket; + } + + @Override + public Multisets.ImmutableEntry nextInBucket() { + return nextInBucket; + } + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public int count(Object element) { + Multisets.ImmutableEntry[] hashTable = this.hashTable; + if (element == null || hashTable == null) { + return 0; + } + int hash = Hashing.smearedHash(element); + int mask = hashTable.length - 1; + for (Multisets.ImmutableEntry entry = hashTable[hash & mask]; + entry != null; + entry = entry.nextInBucket()) { + if (Objects.equal(element, entry.getElement())) { + return entry.getCount(); + } + } + return 0; + } + + @Override + public int size() { + return size; + } + + @Override + public ImmutableSet elementSet() { + ImmutableSet result = elementSet; + return (result == null) ? elementSet = new ElementSet(Arrays.asList(entries), this) : result; + } + + @Override + Entry getEntry(int index) { + return entries[index]; + } + + @Override + public int hashCode() { + return hashCode; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSet.java new file mode 100644 index 0000000000000..5442a636656e3 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSet.java @@ -0,0 +1,110 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + + +import java.util.Spliterator; +import java.util.Spliterators; + + +/** + * Implementation of {@link ImmutableSet} with two or more elements. + * + * @author Kevin Bourrillion + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +final class RegularImmutableSet extends ImmutableSet { + static final RegularImmutableSet EMPTY = + new RegularImmutableSet<>(new Object[0], 0, null, 0); + + private final transient Object[] elements; + // the same elements in hashed positions (plus nulls) + final transient Object[] table; + // 'and' with an int to get a valid table index. + private final transient int mask; + private final transient int hashCode; + + RegularImmutableSet(Object[] elements, int hashCode, Object[] table, int mask) { + this.elements = elements; + this.table = table; + this.mask = mask; + this.hashCode = hashCode; + } + + @Override + public boolean contains(Object target) { + Object[] table = this.table; + if (target == null || table == null) { + return false; + } + for (int i = Hashing.smearedHash(target); ; i++) { + i &= mask; + Object candidate = table[i]; + if (candidate == null) { + return false; + } else if (candidate.equals(target)) { + return true; + } + } + } + + @Override + public int size() { + return elements.length; + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public UnmodifiableIterator iterator() { + return (UnmodifiableIterator) Iterators.forArray(elements); + } + + @Override + public Spliterator spliterator() { + return Spliterators.spliterator(elements, SPLITERATOR_CHARACTERISTICS); + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + System.arraycopy(elements, 0, dst, offset, elements.length); + return offset + elements.length; + } + + @Override + ImmutableList createAsList() { + return (table == null) ? ImmutableList.of() : new RegularImmutableAsList(this, elements); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + public int hashCode() { + return hashCode; + } + + @Override + boolean isHashCodeFast() { + return true; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedMultiset.java new file mode 100644 index 0000000000000..3e67f53e3d009 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedMultiset.java @@ -0,0 +1,136 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.BoundType.CLOSED; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.util.Comparator; +import java.util.function.ObjIntConsumer; + + +/** + * An immutable sorted multiset with one or more distinct elements. + * + * @author Louis Wasserman + */ +@SuppressWarnings("serial") // uses writeReplace, not default serialization +final class RegularImmutableSortedMultiset extends ImmutableSortedMultiset { + private static final long[] ZERO_CUMULATIVE_COUNTS = {0}; + + @SuppressWarnings("rawtypes") + static final ImmutableSortedMultiset NATURAL_EMPTY_MULTISET = + new RegularImmutableSortedMultiset<>(Ordering.natural()); + + final transient RegularImmutableSortedSet elementSet; + private final transient long[] cumulativeCounts; + private final transient int offset; + private final transient int length; + + RegularImmutableSortedMultiset(Comparator comparator) { + this.elementSet = ImmutableSortedSet.emptySet(comparator); + this.cumulativeCounts = ZERO_CUMULATIVE_COUNTS; + this.offset = 0; + this.length = 0; + } + + RegularImmutableSortedMultiset( + RegularImmutableSortedSet elementSet, long[] cumulativeCounts, int offset, int length) { + this.elementSet = elementSet; + this.cumulativeCounts = cumulativeCounts; + this.offset = offset; + this.length = length; + } + + private int getCount(int index) { + return (int) (cumulativeCounts[offset + index + 1] - cumulativeCounts[offset + index]); + } + + @Override + Entry getEntry(int index) { + return Multisets.immutableEntry(elementSet.asList().get(index), getCount(index)); + } + + @Override + public void forEachEntry(ObjIntConsumer action) { + checkNotNull(action); + for (int i = 0; i < length; i++) { + action.accept(elementSet.asList().get(i), getCount(i)); + } + } + + @Override + public Entry firstEntry() { + return isEmpty() ? null : getEntry(0); + } + + @Override + public Entry lastEntry() { + return isEmpty() ? null : getEntry(length - 1); + } + + @Override + public int count(Object element) { + int index = elementSet.indexOf(element); + return (index >= 0) ? getCount(index) : 0; + } + + @Override + public int size() { + long size = cumulativeCounts[offset + length] - cumulativeCounts[offset]; + return Ints.saturatedCast(size); + } + + @Override + public ImmutableSortedSet elementSet() { + return elementSet; + } + + @Override + public ImmutableSortedMultiset headMultiset(E upperBound, BoundType boundType) { + return getSubMultiset(0, elementSet.headIndex(upperBound, checkNotNull(boundType) == CLOSED)); + } + + @Override + public ImmutableSortedMultiset tailMultiset(E lowerBound, BoundType boundType) { + return getSubMultiset( + elementSet.tailIndex(lowerBound, checkNotNull(boundType) == CLOSED), length); + } + + ImmutableSortedMultiset getSubMultiset(int from, int to) { + checkPositionIndexes(from, to, length); + if (from == to) { + return emptyMultiset(comparator()); + } else if (from == 0 && to == length) { + return this; + } else { + RegularImmutableSortedSet subElementSet = elementSet.getSubSet(from, to); + return new RegularImmutableSortedMultiset( + subElementSet, cumulativeCounts, offset + from, to - from); + } + } + + @Override + boolean isPartialView() { + return offset > 0 || length < cumulativeCounts.length - 1; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedSet.java new file mode 100644 index 0000000000000..dab65035e6c7f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/RegularImmutableSortedSet.java @@ -0,0 +1,311 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.Spliterator; +import java.util.function.Consumer; + + +/** + * An immutable sorted set with one or more elements. TODO(jlevy): Consider separate class for a + * single-element sorted set. + * + * @author Jared Levy + * @author Louis Wasserman + */ +@SuppressWarnings("serial") +final class RegularImmutableSortedSet extends ImmutableSortedSet { + @SuppressWarnings("rawtypes") + static final RegularImmutableSortedSet NATURAL_EMPTY_SET = + new RegularImmutableSortedSet<>(ImmutableList.of(), Ordering.natural()); + + private final transient ImmutableList elements; + + RegularImmutableSortedSet(ImmutableList elements, Comparator comparator) { + super(comparator); + this.elements = elements; + } + + @Override + public UnmodifiableIterator iterator() { + return elements.iterator(); + } + + @Override + public UnmodifiableIterator descendingIterator() { + return elements.reverse().iterator(); + } + + @Override + public Spliterator spliterator() { + return asList().spliterator(); + } + + @Override + public void forEach(Consumer action) { + elements.forEach(action); + } + + @Override + public int size() { + return elements.size(); + } + + @Override + public boolean contains(Object o) { + try { + return o != null && unsafeBinarySearch(o) >= 0; + } catch (ClassCastException e) { + return false; + } + } + + @Override + public boolean containsAll(Collection targets) { + // TODO(jlevy): For optimal performance, use a binary search when + // targets.size() < size() / log(size()) + // TODO(kevinb): see if we can share code with OrderedIterator after it + // graduates from labs. + if (targets instanceof Multiset) { + targets = ((Multiset) targets).elementSet(); + } + if (!SortedIterables.hasSameComparator(comparator(), targets) || (targets.size() <= 1)) { + return super.containsAll(targets); + } + + /* + * If targets is a sorted set with the same comparator, containsAll can run + * in O(n) time stepping through the two collections. + */ + Iterator thisIterator = iterator(); + + Iterator thatIterator = targets.iterator(); + // known nonempty since we checked targets.size() > 1 + + if (!thisIterator.hasNext()) { + return false; + } + + Object target = thatIterator.next(); + E current = thisIterator.next(); + try { + while (true) { + int cmp = unsafeCompare(current, target); + + if (cmp < 0) { + if (!thisIterator.hasNext()) { + return false; + } + current = thisIterator.next(); + } else if (cmp == 0) { + if (!thatIterator.hasNext()) { + return true; + } + target = thatIterator.next(); + + } else if (cmp > 0) { + return false; + } + } + } catch (NullPointerException | ClassCastException e) { + return false; + } + } + + private int unsafeBinarySearch(Object key) throws ClassCastException { + return Collections.binarySearch(elements, key, unsafeComparator()); + } + + @Override + boolean isPartialView() { + return elements.isPartialView(); + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + return elements.copyIntoArray(dst, offset); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (!(object instanceof Set)) { + return false; + } + + Set that = (Set) object; + if (size() != that.size()) { + return false; + } else if (isEmpty()) { + return true; + } + + if (SortedIterables.hasSameComparator(comparator, that)) { + Iterator otherIterator = that.iterator(); + try { + Iterator iterator = iterator(); + while (iterator.hasNext()) { + Object element = iterator.next(); + Object otherElement = otherIterator.next(); + if (otherElement == null || unsafeCompare(element, otherElement) != 0) { + return false; + } + } + return true; + } catch (ClassCastException e) { + return false; + } catch (NoSuchElementException e) { + return false; // concurrent change to other set + } + } + return containsAll(that); + } + + @Override + public E first() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return elements.get(0); + } + + @Override + public E last() { + if (isEmpty()) { + throw new NoSuchElementException(); + } + return elements.get(size() - 1); + } + + @Override + public E lower(E element) { + int index = headIndex(element, false) - 1; + return (index == -1) ? null : elements.get(index); + } + + @Override + public E floor(E element) { + int index = headIndex(element, true) - 1; + return (index == -1) ? null : elements.get(index); + } + + @Override + public E ceiling(E element) { + int index = tailIndex(element, true); + return (index == size()) ? null : elements.get(index); + } + + @Override + public E higher(E element) { + int index = tailIndex(element, false); + return (index == size()) ? null : elements.get(index); + } + + @Override + ImmutableSortedSet headSetImpl(E toElement, boolean inclusive) { + return getSubSet(0, headIndex(toElement, inclusive)); + } + + int headIndex(E toElement, boolean inclusive) { + int index = Collections.binarySearch(elements, checkNotNull(toElement), comparator()); + if (index >= 0) { + return inclusive ? index + 1 : index; + } else { + return ~index; + } + } + + @Override + ImmutableSortedSet subSetImpl( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return tailSetImpl(fromElement, fromInclusive).headSetImpl(toElement, toInclusive); + } + + @Override + ImmutableSortedSet tailSetImpl(E fromElement, boolean inclusive) { + return getSubSet(tailIndex(fromElement, inclusive), size()); + } + + int tailIndex(E fromElement, boolean inclusive) { + int index = Collections.binarySearch(elements, checkNotNull(fromElement), comparator()); + if (index >= 0) { + return inclusive ? index : index + 1; + } else { + return ~index; + } + } + + // Pretend the comparator can compare anything. If it turns out it can't + // compare two elements, it'll throw a CCE. Only methods that are specified to + // throw CCE should call this. + @SuppressWarnings({"unchecked", "rawtypes"}) + Comparator unsafeComparator() { + return (Comparator) comparator; + } + + RegularImmutableSortedSet getSubSet(int newFromIndex, int newToIndex) { + if (newFromIndex == 0 && newToIndex == size()) { + return this; + } else if (newFromIndex < newToIndex) { + return new RegularImmutableSortedSet( + elements.subList(newFromIndex, newToIndex), comparator); + } else { + return emptySet(comparator); + } + } + + @Override + int indexOf(Object target) { + if (target == null) { + return -1; + } + int position; + try { + position = Collections.binarySearch(elements, target, unsafeComparator()); + } catch (ClassCastException e) { + return -1; + } + return (position >= 0) ? position : -1; + } + + @Override + ImmutableList createAsList() { + return (size() <= 1) ? elements : new ImmutableSortedAsList(this, elements); + } + + @Override + ImmutableSortedSet createDescendingSet() { + Comparator reversedOrder = Collections.reverseOrder(comparator); + return isEmpty() + ? emptySet(reversedOrder) + : new RegularImmutableSortedSet(elements.reverse(), reversedOrder); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseNaturalOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseNaturalOrdering.java new file mode 100644 index 0000000000000..3e5f5cc7c4412 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseNaturalOrdering.java @@ -0,0 +1,103 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Iterator; + +/** An ordering that uses the reverse of the natural order of the values. */ +@SuppressWarnings({"unchecked", "rawtypes"}) // TODO(kevinb): the right way to explain this?? +final class ReverseNaturalOrdering extends Ordering implements Serializable { + static final ReverseNaturalOrdering INSTANCE = new ReverseNaturalOrdering(); + + @Override + public int compare(Comparable left, Comparable right) { + checkNotNull(left); // right null is caught later + if (left == right) { + return 0; + } + + return right.compareTo(left); + } + + @Override + public Ordering reverse() { + return Ordering.natural(); + } + + // Override the min/max methods to "hoist" delegation outside loops + + @Override + public E min(E a, E b) { + return NaturalOrdering.INSTANCE.max(a, b); + } + + @Override + public E min(E a, E b, E c, E... rest) { + return NaturalOrdering.INSTANCE.max(a, b, c, rest); + } + + @Override + public E min(Iterator iterator) { + return NaturalOrdering.INSTANCE.max(iterator); + } + + @Override + public E min(Iterable iterable) { + return NaturalOrdering.INSTANCE.max(iterable); + } + + @Override + public E max(E a, E b) { + return NaturalOrdering.INSTANCE.min(a, b); + } + + @Override + public E max(E a, E b, E c, E... rest) { + return NaturalOrdering.INSTANCE.min(a, b, c, rest); + } + + @Override + public E max(Iterator iterator) { + return NaturalOrdering.INSTANCE.min(iterator); + } + + @Override + public E max(Iterable iterable) { + return NaturalOrdering.INSTANCE.min(iterable); + } + + // preserving singleton-ness gives equals()/hashCode() for free + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "Ordering.natural().reverse()"; + } + + private ReverseNaturalOrdering() {} + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseOrdering.java new file mode 100644 index 0000000000000..b417fe72c70f4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/ReverseOrdering.java @@ -0,0 +1,115 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.Serializable; +import java.util.Iterator; + + +/** An ordering that uses the reverse of a given order. */ +final class ReverseOrdering extends Ordering implements Serializable { + final Ordering forwardOrder; + + ReverseOrdering(Ordering forwardOrder) { + this.forwardOrder = checkNotNull(forwardOrder); + } + + @Override + public int compare(T a, T b) { + return forwardOrder.compare(b, a); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // how to explain? + @Override + public Ordering reverse() { + return (Ordering) forwardOrder; + } + + // Override the min/max methods to "hoist" delegation outside loops + + @Override + public E min(E a, E b) { + return forwardOrder.max(a, b); + } + + @SuppressWarnings("unchecked") + @Override + public E min(E a, E b, E c, E... rest) { + return forwardOrder.max(a, b, c, rest); + } + + @Override + public E min(Iterator iterator) { + return forwardOrder.max(iterator); + } + + @Override + public E min(Iterable iterable) { + return forwardOrder.max(iterable); + } + + @Override + public E max(E a, E b) { + return forwardOrder.min(a, b); + } + + @SuppressWarnings("unchecked") + @Override + public E max(E a, E b, E c, E... rest) { + return forwardOrder.min(a, b, c, rest); + } + + @Override + public E max(Iterator iterator) { + return forwardOrder.min(iterator); + } + + @Override + public E max(Iterable iterable) { + return forwardOrder.min(iterable); + } + + @Override + public int hashCode() { + return -forwardOrder.hashCode(); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof ReverseOrdering) { + ReverseOrdering that = (ReverseOrdering) object; + return this.forwardOrder.equals(that.forwardOrder); + } + return false; + } + + @Override + public String toString() { + return forwardOrder + ".reverse()"; + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Serialization.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Serialization.java new file mode 100644 index 0000000000000..86b58d0dc9782 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Serialization.java @@ -0,0 +1,221 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.lang.reflect.Field; +import java.util.Collection; +import java.util.Map; + +/** + * Provides static methods for serializing collection classes. + * + *

This class assists the implementation of collection classes. Do not use this class to + * serialize collections that are defined elsewhere. + * + * @author Jared Levy + */ +final class Serialization { + private Serialization() {} + + /** + * Reads a count corresponding to a serialized map, multiset, or multimap. It returns the size of + * a map serialized by {@link #writeMap(Map, ObjectOutputStream)}, the number of distinct elements + * in a multiset serialized by {@link #writeMultiset(Multiset, ObjectOutputStream)}, or the number + * of distinct keys in a multimap serialized by {@link #writeMultimap(Multimap, + * ObjectOutputStream)}. + */ + static int readCount(ObjectInputStream stream) throws IOException { + return stream.readInt(); + } + + /** + * Stores the contents of a map in an output stream, as part of serialization. It does not support + * concurrent maps whose content may change while the method is running. + * + *

The serialized output consists of the number of entries, first key, first value, second key, + * second value, and so on. + */ + static void writeMap(Map map, ObjectOutputStream stream) throws IOException { + stream.writeInt(map.size()); + for (Map.Entry entry : map.entrySet()) { + stream.writeObject(entry.getKey()); + stream.writeObject(entry.getValue()); + } + } + + /** + * Populates a map by reading an input stream, as part of deserialization. See {@link #writeMap} + * for the data format. + */ + static void populateMap(Map map, ObjectInputStream stream) + throws IOException, ClassNotFoundException { + int size = stream.readInt(); + populateMap(map, stream, size); + } + + /** + * Populates a map by reading an input stream, as part of deserialization. See {@link #writeMap} + * for the data format. The size is determined by a prior call to {@link #readCount}. + */ + static void populateMap(Map map, ObjectInputStream stream, int size) + throws IOException, ClassNotFoundException { + for (int i = 0; i < size; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeMap + K key = (K) stream.readObject(); + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeMap + V value = (V) stream.readObject(); + map.put(key, value); + } + } + + /** + * Stores the contents of a multiset in an output stream, as part of serialization. It does not + * support concurrent multisets whose content may change while the method is running. + * + *

The serialized output consists of the number of distinct elements, the first element, its + * count, the second element, its count, and so on. + */ + static void writeMultiset(Multiset multiset, ObjectOutputStream stream) + throws IOException { + int entryCount = multiset.entrySet().size(); + stream.writeInt(entryCount); + for (Multiset.Entry entry : multiset.entrySet()) { + stream.writeObject(entry.getElement()); + stream.writeInt(entry.getCount()); + } + } + + /** + * Populates a multiset by reading an input stream, as part of deserialization. See {@link + * #writeMultiset} for the data format. + */ + static void populateMultiset(Multiset multiset, ObjectInputStream stream) + throws IOException, ClassNotFoundException { + int distinctElements = stream.readInt(); + populateMultiset(multiset, stream, distinctElements); + } + + /** + * Populates a multiset by reading an input stream, as part of deserialization. See {@link + * #writeMultiset} for the data format. The number of distinct elements is determined by a prior + * call to {@link #readCount}. + */ + static void populateMultiset( + Multiset multiset, ObjectInputStream stream, int distinctElements) + throws IOException, ClassNotFoundException { + for (int i = 0; i < distinctElements; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeMultiset + E element = (E) stream.readObject(); + int count = stream.readInt(); + multiset.add(element, count); + } + } + + /** + * Stores the contents of a multimap in an output stream, as part of serialization. It does not + * support concurrent multimaps whose content may change while the method is running. The {@link + * Multimap#asMap} view determines the ordering in which data is written to the stream. + * + *

The serialized output consists of the number of distinct keys, and then for each distinct + * key: the key, the number of values for that key, and the key's values. + */ + static void writeMultimap(Multimap multimap, ObjectOutputStream stream) + throws IOException { + stream.writeInt(multimap.asMap().size()); + for (Map.Entry> entry : multimap.asMap().entrySet()) { + stream.writeObject(entry.getKey()); + stream.writeInt(entry.getValue().size()); + for (V value : entry.getValue()) { + stream.writeObject(value); + } + } + } + + /** + * Populates a multimap by reading an input stream, as part of deserialization. See {@link + * #writeMultimap} for the data format. + */ + static void populateMultimap(Multimap multimap, ObjectInputStream stream) + throws IOException, ClassNotFoundException { + int distinctKeys = stream.readInt(); + populateMultimap(multimap, stream, distinctKeys); + } + + /** + * Populates a multimap by reading an input stream, as part of deserialization. See {@link + * #writeMultimap} for the data format. The number of distinct keys is determined by a prior call + * to {@link #readCount}. + */ + static void populateMultimap( + Multimap multimap, ObjectInputStream stream, int distinctKeys) + throws IOException, ClassNotFoundException { + for (int i = 0; i < distinctKeys; i++) { + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeMultimap + K key = (K) stream.readObject(); + Collection values = multimap.get(key); + int valueCount = stream.readInt(); + for (int j = 0; j < valueCount; j++) { + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeMultimap + V value = (V) stream.readObject(); + values.add(value); + } + } + } + + // Secret sauce for setting final fields; don't make it public. + static FieldSetter getFieldSetter(final Class clazz, String fieldName) { + try { + Field field = clazz.getDeclaredField(fieldName); + return new FieldSetter(field); + } catch (NoSuchFieldException e) { + throw new AssertionError(e); // programmer error + } + } + + // Secret sauce for setting final fields; don't make it public. + static final class FieldSetter { + private final Field field; + + private FieldSetter(Field field) { + this.field = field; + field.setAccessible(true); + } + + void set(T instance, Object value) { + try { + field.set(instance, value); + } catch (IllegalAccessException impossible) { + throw new AssertionError(impossible); + } + } + + void set(T instance, int value) { + try { + field.set(instance, value); + } catch (IllegalAccessException impossible) { + throw new AssertionError(impossible); + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SetMultimap.java new file mode 100644 index 0000000000000..e63803e7b1fe0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SetMultimap.java @@ -0,0 +1,117 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + + +/** + * A {@code Multimap} that cannot hold duplicate key-value pairs. Adding a key-value pair that's + * already in the multimap has no effect. See the {@link Multimap} documentation for information + * common to all multimaps. + * + *

The {@link #get}, {@link #removeAll}, and {@link #replaceValues} methods each return a {@link + * Set} of values, while {@link #entries} returns a {@code Set} of map entries. Though the method + * signature doesn't say so explicitly, the map returned by {@link #asMap} has {@code Set} values. + * + *

If the values corresponding to a single key should be ordered according to a {@link + * java.util.Comparator} (or the natural order), see the {@link SortedSetMultimap} subinterface. + * + *

Since the value collections are sets, the behavior of a {@code SetMultimap} is not specified + * if key or value objects already present in the multimap change in a manner that affects + * {@code equals} comparisons. Use caution if mutable objects are used as keys or values in a {@code + * SetMultimap}. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @since 2.0 + */ +public interface SetMultimap extends Multimap { + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} + * interface. + */ + @Override + Set get(K key); + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} + * interface. + */ + @Override + Set removeAll(Object key); + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} + * interface. + * + *

Any duplicates in {@code values} will be stored in the multimap once. + */ + @Override + Set replaceValues(K key, Iterable values); + + /** + * {@inheritDoc} + * + *

Because a {@code SetMultimap} has unique values for a given key, this method returns a + * {@link Set}, instead of the {@link Collection} specified in the {@link Multimap} + * interface. + */ + @Override + Set> entries(); + + /** + * {@inheritDoc} + * + *

Note: The returned map's values are guaranteed to be of type {@link Set}. To obtain + * this map with the more specific generic type {@code Map>}, call {@link + * Multimaps#asMap(SetMultimap)} instead. + */ + @Override + Map> asMap(); + + /** + * Compares the specified object to this multimap for equality. + * + *

Two {@code SetMultimap} instances are equal if, for each key, they contain the same values. + * Equality does not depend on the ordering of keys or values. + * + *

An empty {@code SetMultimap} is equal to any other empty {@code Multimap}, including an + * empty {@code ListMultimap}. + */ + @Override + boolean equals(Object obj); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Sets.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Sets.java new file mode 100644 index 0000000000000..a14f50745335a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Sets.java @@ -0,0 +1,2121 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Collections2.FilteredCollection; +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import java.io.Serializable; +import java.util.AbstractSet; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.NavigableSet; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.function.Consumer; +import java.util.stream.Collector; +import java.util.stream.Stream; + + + +/** + * Static utility methods pertaining to {@link Set} instances. Also see this class's counterparts + * {@link Lists}, {@link Maps} and {@link Queues}. + * + *

See the Guava User Guide article on {@code Sets}. + * + * @author Kevin Bourrillion + * @author Jared Levy + * @author Chris Povirk + * @since 2.0 + */ +public final class Sets { + private Sets() {} + + /** + * {@link AbstractSet} substitute without the potentially-quadratic {@code removeAll} + * implementation. + */ + abstract static class ImprovedAbstractSet extends AbstractSet { + @Override + public boolean removeAll(Collection c) { + return removeAllImpl(this, c); + } + + @Override + public boolean retainAll(Collection c) { + return super.retainAll(checkNotNull(c)); // GWT compatibility + } + } + + /** + * Returns an immutable set instance containing the given enum elements. Internally, the returned + * set will be backed by an {@link EnumSet}. + * + *

The iteration order of the returned set follows the enum's iteration order, not the order in + * which the elements are provided to the method. + * + * @param anElement one of the elements the set should contain + * @param otherElements the rest of the elements the set should contain + * @return an immutable set containing those elements, minus duplicates + */ + // http://code.google.com/p/google-web-toolkit/issues/detail?id=3028 + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSet immutableEnumSet( + E anElement, E... otherElements) { + return ImmutableEnumSet.asImmutable(EnumSet.of(anElement, otherElements)); + } + + /** + * Returns an immutable set instance containing the given enum elements. Internally, the returned + * set will be backed by an {@link EnumSet}. + * + *

The iteration order of the returned set follows the enum's iteration order, not the order in + * which the elements appear in the given collection. + * + * @param elements the elements, all of the same {@code enum} type, that the set should contain + * @return an immutable set containing those elements, minus duplicates + */ + // http://code.google.com/p/google-web-toolkit/issues/detail?id=3028 + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > ImmutableSet immutableEnumSet(Iterable elements) { + if (elements instanceof ImmutableEnumSet) { + return (ImmutableEnumSet) elements; + } else if (elements instanceof Collection) { + Collection collection = (Collection) elements; + if (collection.isEmpty()) { + return ImmutableSet.of(); + } else { + return ImmutableEnumSet.asImmutable(EnumSet.copyOf(collection)); + } + } else { + Iterator itr = elements.iterator(); + if (itr.hasNext()) { + EnumSet enumSet = EnumSet.of(itr.next()); + Iterators.addAll(enumSet, itr); + return ImmutableEnumSet.asImmutable(enumSet); + } else { + return ImmutableSet.of(); + } + } + } + + private static final class Accumulator> { + @SuppressWarnings({"unchecked", "rawtypes"}) + static final Collector, ?, ImmutableSet>> TO_IMMUTABLE_ENUM_SET = + (Collector) + Collector.>of( + Accumulator::new, + Accumulator::add, + Accumulator::combine, + Accumulator::toImmutableSet, + Collector.Characteristics.UNORDERED); + + private EnumSet set; + + void add(E e) { + if (set == null) { + set = EnumSet.of(e); + } else { + set.add(e); + } + } + + Accumulator combine(Accumulator other) { + if (this.set == null) { + return other; + } else if (other.set == null) { + return this; + } else { + this.set.addAll(other.set); + return this; + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + ImmutableSet toImmutableSet() { + return (set == null) ? ImmutableSet.of() : ImmutableEnumSet.asImmutable(set); + } + } + + /** + * Returns a {@code Collector} that accumulates the input elements into a new {@code ImmutableSet} + * with an implementation specialized for enums. Unlike {@link ImmutableSet#toImmutableSet}, the + * resulting set will iterate over elements in their enum definition order, not encounter order. + * + * @since 21.0 + */ + + @SuppressWarnings({"unchecked", "rawtypes"}) + public static > Collector> toImmutableEnumSet() { + return (Collector) Accumulator.TO_IMMUTABLE_ENUM_SET; + } + + /** + * Returns a new, mutable {@code EnumSet} instance containing the given elements in their + * natural order. This method behaves identically to {@link EnumSet#copyOf(Collection)}, but also + * accepts non-{@code Collection} iterables and empty iterables. + */ + public static > EnumSet newEnumSet( + Iterable iterable, Class elementType) { + EnumSet set = EnumSet.noneOf(elementType); + Iterables.addAll(set, iterable); + return set; + } + + // HashSet + + /** + * Creates a mutable, initially empty {@code HashSet} instance. + * + *

Note: if mutability is not required, use {@link ImmutableSet#of()} instead. If {@code + * E} is an {@link Enum} type, use {@link EnumSet#noneOf} instead. Otherwise, strongly consider + * using a {@code LinkedHashSet} instead, at the cost of increased memory footprint, to get + * deterministic iteration behavior. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code HashSet} constructor directly, taking advantage of the new + * "diamond" syntax. + */ + public static HashSet newHashSet() { + return new HashSet(); + } + + /** + * Creates a mutable {@code HashSet} instance initially containing the given elements. + * + *

Note: if elements are non-null and won't be added or removed after this point, use + * {@link ImmutableSet#of()} or {@link ImmutableSet#copyOf(Object[])} instead. If {@code E} is an + * {@link Enum} type, use {@link EnumSet#of(Enum, Enum[])} instead. Otherwise, strongly consider + * using a {@code LinkedHashSet} instead, at the cost of increased memory footprint, to get + * deterministic iteration behavior. + * + *

This method is just a small convenience, either for {@code newHashSet(}{@link Arrays#asList + * asList}{@code (...))}, or for creating an empty set then calling {@link Collections#addAll}. + * This method is not actually very useful and will likely be deprecated in the future. + */ + @SuppressWarnings("unchecked") + public static HashSet newHashSet(E... elements) { + HashSet set = newHashSetWithExpectedSize(elements.length); + Collections.addAll(set, elements); + return set; + } + + /** + * Creates a mutable {@code HashSet} instance containing the given elements. A very thin + * convenience for creating an empty set then calling {@link Collection#addAll} or {@link + * Iterables#addAll}. + * + *

Note: if mutability is not required and the elements are non-null, use {@link + * ImmutableSet#copyOf(Iterable)} instead. (Or, change {@code elements} to be a {@link + * FluentIterable} and call {@code elements.toSet()}.) + * + *

Note: if {@code E} is an {@link Enum} type, use {@link #newEnumSet(Iterable, Class)} + * instead. + * + *

Note for Java 7 and later: if {@code elements} is a {@link Collection}, you don't + * need this method. Instead, use the {@code HashSet} constructor directly, taking advantage of + * the new "diamond" syntax. + * + *

Overall, this method is not very useful and will likely be deprecated in the future. + */ + public static HashSet newHashSet(Iterable elements) { + return (elements instanceof Collection) + ? new HashSet(Collections2.cast(elements)) + : newHashSet(elements.iterator()); + } + + /** + * Creates a mutable {@code HashSet} instance containing the given elements. A very thin + * convenience for creating an empty set and then calling {@link Iterators#addAll}. + * + *

Note: if mutability is not required and the elements are non-null, use {@link + * ImmutableSet#copyOf(Iterator)} instead. + * + *

Note: if {@code E} is an {@link Enum} type, you should create an {@link EnumSet} + * instead. + * + *

Overall, this method is not very useful and will likely be deprecated in the future. + */ + public static HashSet newHashSet(Iterator elements) { + HashSet set = newHashSet(); + Iterators.addAll(set, elements); + return set; + } + + /** + * Returns a new hash set using the smallest initial table size that can hold {@code expectedSize} + * elements without resizing. Note that this is not what {@link HashSet#HashSet(int)} does, but it + * is what most users want and expect it to do. + * + *

This behavior can't be broadly guaranteed, but has been tested with OpenJDK 1.7 and 1.8. + * + * @param expectedSize the number of elements you expect to add to the returned set + * @return a new, empty hash set with enough capacity to hold {@code expectedSize} elements + * without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + */ + public static HashSet newHashSetWithExpectedSize(int expectedSize) { + return new HashSet(Maps.capacity(expectedSize)); + } + + /** + * Creates a thread-safe set backed by a hash map. The set is backed by a {@link + * ConcurrentHashMap} instance, and thus carries the same concurrency guarantees. + * + *

Unlike {@code HashSet}, this class does NOT allow {@code null} to be used as an element. The + * set is serializable. + * + * @return a new, empty thread-safe {@code Set} + * @since 15.0 + */ + public static Set newConcurrentHashSet() { + return Collections.newSetFromMap(new ConcurrentHashMap()); + } + + /** + * Creates a thread-safe set backed by a hash map and containing the given elements. The set is + * backed by a {@link ConcurrentHashMap} instance, and thus carries the same concurrency + * guarantees. + * + *

Unlike {@code HashSet}, this class does NOT allow {@code null} to be used as an element. The + * set is serializable. + * + * @param elements the elements that the set should contain + * @return a new thread-safe set containing those elements (minus duplicates) + * @throws NullPointerException if {@code elements} or any of its contents is null + * @since 15.0 + */ + public static Set newConcurrentHashSet(Iterable elements) { + Set set = newConcurrentHashSet(); + Iterables.addAll(set, elements); + return set; + } + + // LinkedHashSet + + /** + * Creates a mutable, empty {@code LinkedHashSet} instance. + * + *

Note: if mutability is not required, use {@link ImmutableSet#of()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code LinkedHashSet} constructor directly, taking advantage of + * the new "diamond" syntax. + * + * @return a new, empty {@code LinkedHashSet} + */ + public static LinkedHashSet newLinkedHashSet() { + return new LinkedHashSet(); + } + + /** + * Creates a mutable {@code LinkedHashSet} instance containing the given elements in order. + * + *

Note: if mutability is not required and the elements are non-null, use {@link + * ImmutableSet#copyOf(Iterable)} instead. + * + *

Note for Java 7 and later: if {@code elements} is a {@link Collection}, you don't + * need this method. Instead, use the {@code LinkedHashSet} constructor directly, taking advantage + * of the new "diamond" syntax. + * + *

Overall, this method is not very useful and will likely be deprecated in the future. + * + * @param elements the elements that the set should contain, in order + * @return a new {@code LinkedHashSet} containing those elements (minus duplicates) + */ + public static LinkedHashSet newLinkedHashSet(Iterable elements) { + if (elements instanceof Collection) { + return new LinkedHashSet(Collections2.cast(elements)); + } + LinkedHashSet set = newLinkedHashSet(); + Iterables.addAll(set, elements); + return set; + } + + /** + * Creates a {@code LinkedHashSet} instance, with a high enough "initial capacity" that it + * should hold {@code expectedSize} elements without growth. This behavior cannot be + * broadly guaranteed, but it is observed to be true for OpenJDK 1.7. It also can't be guaranteed + * that the method isn't inadvertently oversizing the returned set. + * + * @param expectedSize the number of elements you expect to add to the returned set + * @return a new, empty {@code LinkedHashSet} with enough capacity to hold {@code expectedSize} + * elements without resizing + * @throws IllegalArgumentException if {@code expectedSize} is negative + * @since 11.0 + */ + public static LinkedHashSet newLinkedHashSetWithExpectedSize(int expectedSize) { + return new LinkedHashSet(Maps.capacity(expectedSize)); + } + + // TreeSet + + /** + * Creates a mutable, empty {@code TreeSet} instance sorted by the natural sort ordering of + * its elements. + * + *

Note: if mutability is not required, use {@link ImmutableSortedSet#of()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code TreeSet} constructor directly, taking advantage of the new + * "diamond" syntax. + * + * @return a new, empty {@code TreeSet} + */ + @SuppressWarnings("rawtypes") + public static TreeSet newTreeSet() { + return new TreeSet(); + } + + /** + * Creates a mutable {@code TreeSet} instance containing the given elements sorted by their + * natural ordering. + * + *

Note: if mutability is not required, use {@link ImmutableSortedSet#copyOf(Iterable)} + * instead. + * + *

Note: If {@code elements} is a {@code SortedSet} with an explicit comparator, this + * method has different behavior than {@link TreeSet#TreeSet(SortedSet)}, which returns a {@code + * TreeSet} with that comparator. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code TreeSet} constructor directly, taking advantage of the new + * "diamond" syntax. + * + *

This method is just a small convenience for creating an empty set and then calling {@link + * Iterables#addAll}. This method is not very useful and will likely be deprecated in the future. + * + * @param elements the elements that the set should contain + * @return a new {@code TreeSet} containing those elements (minus duplicates) + */ + @SuppressWarnings("rawtypes") + public static TreeSet newTreeSet(Iterable elements) { + TreeSet set = newTreeSet(); + Iterables.addAll(set, elements); + return set; + } + + /** + * Creates a mutable, empty {@code TreeSet} instance with the given comparator. + * + *

Note: if mutability is not required, use {@code + * ImmutableSortedSet.orderedBy(comparator).build()} instead. + * + *

Note for Java 7 and later: this method is now unnecessary and should be treated as + * deprecated. Instead, use the {@code TreeSet} constructor directly, taking advantage of the new + * "diamond" syntax. One caveat to this is that the {@code + * TreeSet} constructor uses a null {@code Comparator} to mean "natural ordering," whereas this + * factory rejects null. Clean your code accordingly. + * + * @param comparator the comparator to use to sort the set + * @return a new, empty {@code TreeSet} + * @throws NullPointerException if {@code comparator} is null + */ + public static TreeSet newTreeSet(Comparator comparator) { + return new TreeSet(checkNotNull(comparator)); + } + + /** + * Creates an empty {@code Set} that uses identity to determine equality. It compares object + * references, instead of calling {@code equals}, to determine whether a provided object matches + * an element in the set. For example, {@code contains} returns {@code false} when passed an + * object that equals a set member, but isn't the same instance. This behavior is similar to the + * way {@code IdentityHashMap} handles key lookups. + * + * @since 8.0 + */ + public static Set newIdentityHashSet() { + return Collections.newSetFromMap(Maps.newIdentityHashMap()); + } + + /** + * Creates an empty {@code CopyOnWriteArraySet} instance. + * + *

Note: if you need an immutable empty {@link Set}, use {@link Collections#emptySet} + * instead. + * + * @return a new, empty {@code CopyOnWriteArraySet} + * @since 12.0 + */ + public static CopyOnWriteArraySet newCopyOnWriteArraySet() { + return new CopyOnWriteArraySet(); + } + + /** + * Creates a {@code CopyOnWriteArraySet} instance containing the given elements. + * + * @param elements the elements that the set should contain, in order + * @return a new {@code CopyOnWriteArraySet} containing those elements + * @since 12.0 + */ + public static CopyOnWriteArraySet newCopyOnWriteArraySet(Iterable elements) { + // We copy elements to an ArrayList first, rather than incurring the + // quadratic cost of adding them to the COWAS directly. + Collection elementsCollection = + (elements instanceof Collection) + ? Collections2.cast(elements) + : Lists.newArrayList(elements); + return new CopyOnWriteArraySet(elementsCollection); + } + + /** + * Creates an {@code EnumSet} consisting of all enum values that are not in the specified + * collection. If the collection is an {@link EnumSet}, this method has the same behavior as + * {@link EnumSet#complementOf}. Otherwise, the specified collection must contain at least one + * element, in order to determine the element type. If the collection could be empty, use {@link + * #complementOf(Collection, Class)} instead of this method. + * + * @param collection the collection whose complement should be stored in the enum set + * @return a new, modifiable {@code EnumSet} containing all values of the enum that aren't present + * in the given collection + * @throws IllegalArgumentException if {@code collection} is not an {@code EnumSet} instance and + * contains no elements + */ + public static > EnumSet complementOf(Collection collection) { + if (collection instanceof EnumSet) { + return EnumSet.complementOf((EnumSet) collection); + } + checkArgument( + !collection.isEmpty(), "collection is empty; use the other version of this method"); + Class type = collection.iterator().next().getDeclaringClass(); + return makeComplementByHand(collection, type); + } + + /** + * Creates an {@code EnumSet} consisting of all enum values that are not in the specified + * collection. This is equivalent to {@link EnumSet#complementOf}, but can act on any input + * collection, as long as the elements are of enum type. + * + * @param collection the collection whose complement should be stored in the {@code EnumSet} + * @param type the type of the elements in the set + * @return a new, modifiable {@code EnumSet} initially containing all the values of the enum not + * present in the given collection + */ + public static > EnumSet complementOf( + Collection collection, Class type) { + checkNotNull(collection); + return (collection instanceof EnumSet) + ? EnumSet.complementOf((EnumSet) collection) + : makeComplementByHand(collection, type); + } + + private static > EnumSet makeComplementByHand( + Collection collection, Class type) { + EnumSet result = EnumSet.allOf(type); + result.removeAll(collection); + return result; + } + + /** + * Returns a set backed by the specified map. The resulting set displays the same ordering, + * concurrency, and performance characteristics as the backing map. In essence, this factory + * method provides a {@link Set} implementation corresponding to any {@link Map} implementation. + * There is no need to use this method on a {@link Map} implementation that already has a + * corresponding {@link Set} implementation (such as {@link java.util.HashMap} or {@link + * java.util.TreeMap}). + * + *

Each method invocation on the set returned by this method results in exactly one method + * invocation on the backing map or its {@code keySet} view, with one exception. The {@code + * addAll} method is implemented as a sequence of {@code put} invocations on the backing map. + * + *

The specified map must be empty at the time this method is invoked, and should not be + * accessed directly after this method returns. These conditions are ensured if the map is created + * empty, passed directly to this method, and no reference to the map is retained, as illustrated + * in the following code fragment: + * + *

{@code
+   * Set identityHashSet = Sets.newSetFromMap(
+   *     new IdentityHashMap());
+   * }
+   *
+   * 

The returned set is serializable if the backing map is. + * + * @param map the backing map + * @return the set backed by the map + * @throws IllegalArgumentException if {@code map} is not empty + * @deprecated Use {@link Collections#newSetFromMap} instead. + */ + @Deprecated + public static Set newSetFromMap(Map map) { + return Collections.newSetFromMap(map); + } + + /** + * An unmodifiable view of a set which may be backed by other sets; this view will change as the + * backing sets do. Contains methods to copy the data into a new set which will then remain + * stable. There is usually no reason to retain a reference of type {@code SetView}; typically, + * you either use it as a plain {@link Set}, or immediately invoke {@link #immutableCopy} or + * {@link #copyInto} and forget the {@code SetView} itself. + * + * @since 2.0 + */ + public abstract static class SetView extends AbstractSet { + private SetView() {} // no subclasses but our own + + /** + * Returns an immutable copy of the current contents of this set view. Does not support null + * elements. + * + *

Warning: this may have unexpected results if a backing set of this view uses a + * nonstandard notion of equivalence, for example if it is a {@link TreeSet} using a comparator + * that is inconsistent with {@link Object#equals(Object)}. + */ + public ImmutableSet immutableCopy() { + return ImmutableSet.copyOf(this); + } + + /** + * Copies the current contents of this set view into an existing set. This method has equivalent + * behavior to {@code set.addAll(this)}, assuming that all the sets involved are based on the + * same notion of equivalence. + * + * @return a reference to {@code set}, for convenience + */ + // Note: S should logically extend Set but can't due to either + // some javac bug or some weirdness in the spec, not sure which. + public > S copyInto(S set) { + set.addAll(this); + return set; + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean add(E e) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean remove(Object object) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean addAll(Collection newElements) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean removeAll(Collection oldElements) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean removeIf(java.util.function.Predicate filter) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final boolean retainAll(Collection elementsToKeep) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the collection unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void clear() { + throw new UnsupportedOperationException(); + } + + /** + * Scope the return type to {@link UnmodifiableIterator} to ensure this is an unmodifiable view. + * + * @since 20.0 (present with return type {@link Iterator} since 2.0) + */ + @Override + public abstract UnmodifiableIterator iterator(); + } + + /** + * Returns an unmodifiable view of the union of two sets. The returned set contains all + * elements that are contained in either backing set. Iterating over the returned set iterates + * first over all the elements of {@code set1}, then over each element of {@code set2}, in order, + * that is not contained in {@code set1}. + * + *

Results are undefined if {@code set1} and {@code set2} are sets based on different + * equivalence relations (as {@link HashSet}, {@link TreeSet}, and the {@link Map#keySet} of an + * {@code IdentityHashMap} all are). + */ + public static SetView union(final Set set1, final Set set2) { + checkNotNull(set1, "set1"); + checkNotNull(set2, "set2"); + + return new SetView() { + @Override + public int size() { + int size = set1.size(); + for (E e : set2) { + if (!set1.contains(e)) { + size++; + } + } + return size; + } + + @Override + public boolean isEmpty() { + return set1.isEmpty() && set2.isEmpty(); + } + + @Override + public UnmodifiableIterator iterator() { + return new AbstractIterator() { + final Iterator itr1 = set1.iterator(); + final Iterator itr2 = set2.iterator(); + + @Override + protected E computeNext() { + if (itr1.hasNext()) { + return itr1.next(); + } + while (itr2.hasNext()) { + E e = itr2.next(); + if (!set1.contains(e)) { + return e; + } + } + return endOfData(); + } + }; + } + + @Override + public Stream stream() { + return Stream.concat(set1.stream(), set2.stream().filter(e -> !set1.contains(e))); + } + + @Override + public Stream parallelStream() { + return stream().parallel(); + } + + @Override + public boolean contains(Object object) { + return set1.contains(object) || set2.contains(object); + } + + @Override + public > S copyInto(S set) { + set.addAll(set1); + set.addAll(set2); + return set; + } + + @Override + public ImmutableSet immutableCopy() { + return new ImmutableSet.Builder().addAll(set1).addAll(set2).build(); + } + }; + } + + /** + * Returns an unmodifiable view of the intersection of two sets. The returned set contains + * all elements that are contained by both backing sets. The iteration order of the returned set + * matches that of {@code set1}. + * + *

Results are undefined if {@code set1} and {@code set2} are sets based on different + * equivalence relations (as {@code HashSet}, {@code TreeSet}, and the keySet of an {@code + * IdentityHashMap} all are). + * + *

Note: The returned view performs slightly better when {@code set1} is the smaller of + * the two sets. If you have reason to believe one of your sets will generally be smaller than the + * other, pass it first. Unfortunately, since this method sets the generic type of the returned + * set based on the type of the first set passed, this could in rare cases force you to make a + * cast, for example: + * + *

{@code
+   * Set aFewBadObjects = ...
+   * Set manyBadStrings = ...
+   *
+   * // impossible for a non-String to be in the intersection
+   * SuppressWarnings("unchecked")
+   * Set badStrings = (Set) Sets.intersection(
+   *     aFewBadObjects, manyBadStrings);
+   * }
+   *
+   * 

This is unfortunate, but should come up only very rarely. + */ + public static SetView intersection(final Set set1, final Set set2) { + checkNotNull(set1, "set1"); + checkNotNull(set2, "set2"); + + return new SetView() { + @Override + public UnmodifiableIterator iterator() { + return new AbstractIterator() { + final Iterator itr = set1.iterator(); + + @Override + protected E computeNext() { + while (itr.hasNext()) { + E e = itr.next(); + if (set2.contains(e)) { + return e; + } + } + return endOfData(); + } + }; + } + + @Override + public Stream stream() { + return set1.stream().filter(set2::contains); + } + + @Override + public Stream parallelStream() { + return set1.parallelStream().filter(set2::contains); + } + + @Override + public int size() { + int size = 0; + for (E e : set1) { + if (set2.contains(e)) { + size++; + } + } + return size; + } + + @Override + public boolean isEmpty() { + return Collections.disjoint(set1, set2); + } + + @Override + public boolean contains(Object object) { + return set1.contains(object) && set2.contains(object); + } + + @Override + public boolean containsAll(Collection collection) { + return set1.containsAll(collection) && set2.containsAll(collection); + } + }; + } + + /** + * Returns an unmodifiable view of the difference of two sets. The returned set contains + * all elements that are contained by {@code set1} and not contained by {@code set2}. {@code set2} + * may also contain elements not present in {@code set1}; these are simply ignored. The iteration + * order of the returned set matches that of {@code set1}. + * + *

Results are undefined if {@code set1} and {@code set2} are sets based on different + * equivalence relations (as {@code HashSet}, {@code TreeSet}, and the keySet of an {@code + * IdentityHashMap} all are). + */ + public static SetView difference(final Set set1, final Set set2) { + checkNotNull(set1, "set1"); + checkNotNull(set2, "set2"); + + return new SetView() { + @Override + public UnmodifiableIterator iterator() { + return new AbstractIterator() { + final Iterator itr = set1.iterator(); + + @Override + protected E computeNext() { + while (itr.hasNext()) { + E e = itr.next(); + if (!set2.contains(e)) { + return e; + } + } + return endOfData(); + } + }; + } + + @Override + public Stream stream() { + return set1.stream().filter(e -> !set2.contains(e)); + } + + @Override + public Stream parallelStream() { + return set1.parallelStream().filter(e -> !set2.contains(e)); + } + + @Override + public int size() { + int size = 0; + for (E e : set1) { + if (!set2.contains(e)) { + size++; + } + } + return size; + } + + @Override + public boolean isEmpty() { + return set2.containsAll(set1); + } + + @Override + public boolean contains(Object element) { + return set1.contains(element) && !set2.contains(element); + } + }; + } + + /** + * Returns an unmodifiable view of the symmetric difference of two sets. The returned set + * contains all elements that are contained in either {@code set1} or {@code set2} but not in + * both. The iteration order of the returned set is undefined. + * + *

Results are undefined if {@code set1} and {@code set2} are sets based on different + * equivalence relations (as {@code HashSet}, {@code TreeSet}, and the keySet of an {@code + * IdentityHashMap} all are). + * + * @since 3.0 + */ + public static SetView symmetricDifference( + final Set set1, final Set set2) { + checkNotNull(set1, "set1"); + checkNotNull(set2, "set2"); + + return new SetView() { + @Override + public UnmodifiableIterator iterator() { + final Iterator itr1 = set1.iterator(); + final Iterator itr2 = set2.iterator(); + return new AbstractIterator() { + @Override + public E computeNext() { + while (itr1.hasNext()) { + E elem1 = itr1.next(); + if (!set2.contains(elem1)) { + return elem1; + } + } + while (itr2.hasNext()) { + E elem2 = itr2.next(); + if (!set1.contains(elem2)) { + return elem2; + } + } + return endOfData(); + } + }; + } + + @Override + public int size() { + int size = 0; + for (E e : set1) { + if (!set2.contains(e)) { + size++; + } + } + for (E e : set2) { + if (!set1.contains(e)) { + size++; + } + } + return size; + } + + @Override + public boolean isEmpty() { + return set1.equals(set2); + } + + @Override + public boolean contains(Object element) { + return set1.contains(element) ^ set2.contains(element); + } + }; + } + + /** + * Returns the elements of {@code unfiltered} that satisfy a predicate. The returned set is a live + * view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting set's iterator does not support {@code remove()}, but all other set methods + * are supported. When given an element that doesn't satisfy the predicate, the set's {@code + * add()} and {@code addAll()} methods throw an {@link IllegalArgumentException}. When methods + * such as {@code removeAll()} and {@code clear()} are called on the filtered set, only elements + * that satisfy the filter will be removed from the underlying set. + * + *

The returned set isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered set's methods, such as {@code size()}, iterate across every element in + * the underlying set and determine which elements satisfy the filter. When a live view is + * not needed, it may be faster to copy {@code Iterables.filter(unfiltered, predicate)} and + * use the copy. + * + *

Warning: {@code predicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. (See {@link + * Iterables#filter(Iterable, Class)} for related functionality.) + * + *

Java 8 users: many use cases for this method are better addressed by {@link + * Stream#filter}. This method is not being deprecated, but we gently encourage + * you to migrate to streams. + */ + // TODO(kevinb): how to omit that last sentence when building GWT javadoc? + public static Set filter(Set unfiltered, Predicate predicate) { + if (unfiltered instanceof SortedSet) { + return filter((SortedSet) unfiltered, predicate); + } + if (unfiltered instanceof FilteredSet) { + // Support clear(), removeAll(), and retainAll() when filtering a filtered + // collection. + FilteredSet filtered = (FilteredSet) unfiltered; + Predicate combinedPredicate = Predicates.and(filtered.predicate, predicate); + return new FilteredSet((Set) filtered.unfiltered, combinedPredicate); + } + + return new FilteredSet(checkNotNull(unfiltered), checkNotNull(predicate)); + } + + /** + * Returns the elements of a {@code SortedSet}, {@code unfiltered}, that satisfy a predicate. The + * returned set is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting set's iterator does not support {@code remove()}, but all other set methods + * are supported. When given an element that doesn't satisfy the predicate, the set's {@code + * add()} and {@code addAll()} methods throw an {@link IllegalArgumentException}. When methods + * such as {@code removeAll()} and {@code clear()} are called on the filtered set, only elements + * that satisfy the filter will be removed from the underlying set. + * + *

The returned set isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered set's methods, such as {@code size()}, iterate across every element in + * the underlying set and determine which elements satisfy the filter. When a live view is + * not needed, it may be faster to copy {@code Iterables.filter(unfiltered, predicate)} and + * use the copy. + * + *

Warning: {@code predicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. (See {@link + * Iterables#filter(Iterable, Class)} for related functionality.) + * + * @since 11.0 + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static SortedSet filter(SortedSet unfiltered, Predicate predicate) { + if (unfiltered instanceof FilteredSet) { + // Support clear(), removeAll(), and retainAll() when filtering a filtered + // collection. + FilteredSet filtered = (FilteredSet) unfiltered; + Predicate combinedPredicate = Predicates.and(filtered.predicate, predicate); + return new FilteredSortedSet((SortedSet) filtered.unfiltered, combinedPredicate); + } + + return new FilteredSortedSet(checkNotNull(unfiltered), checkNotNull(predicate)); + } + + /** + * Returns the elements of a {@code NavigableSet}, {@code unfiltered}, that satisfy a predicate. + * The returned set is a live view of {@code unfiltered}; changes to one affect the other. + * + *

The resulting set's iterator does not support {@code remove()}, but all other set methods + * are supported. When given an element that doesn't satisfy the predicate, the set's {@code + * add()} and {@code addAll()} methods throw an {@link IllegalArgumentException}. When methods + * such as {@code removeAll()} and {@code clear()} are called on the filtered set, only elements + * that satisfy the filter will be removed from the underlying set. + * + *

The returned set isn't threadsafe or serializable, even if {@code unfiltered} is. + * + *

Many of the filtered set's methods, such as {@code size()}, iterate across every element in + * the underlying set and determine which elements satisfy the filter. When a live view is + * not needed, it may be faster to copy {@code Iterables.filter(unfiltered, predicate)} and + * use the copy. + * + *

Warning: {@code predicate} must be consistent with equals, as documented at + * {@link Predicate#apply}. Do not provide a predicate such as {@code + * Predicates.instanceOf(ArrayList.class)}, which is inconsistent with equals. (See {@link + * Iterables#filter(Iterable, Class)} for related functionality.) + * + * @since 14.0 + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static NavigableSet filter( + NavigableSet unfiltered, Predicate predicate) { + if (unfiltered instanceof FilteredSet) { + // Support clear(), removeAll(), and retainAll() when filtering a filtered + // collection. + FilteredSet filtered = (FilteredSet) unfiltered; + Predicate combinedPredicate = Predicates.and(filtered.predicate, predicate); + return new FilteredNavigableSet((NavigableSet) filtered.unfiltered, combinedPredicate); + } + + return new FilteredNavigableSet(checkNotNull(unfiltered), checkNotNull(predicate)); + } + + private static class FilteredSet extends FilteredCollection implements Set { + FilteredSet(Set unfiltered, Predicate predicate) { + super(unfiltered, predicate); + } + + @Override + public boolean equals(Object object) { + return equalsImpl(this, object); + } + + @Override + public int hashCode() { + return hashCodeImpl(this); + } + } + + private static class FilteredSortedSet extends FilteredSet implements SortedSet { + + FilteredSortedSet(SortedSet unfiltered, Predicate predicate) { + super(unfiltered, predicate); + } + + @Override + public Comparator comparator() { + return ((SortedSet) unfiltered).comparator(); + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return new FilteredSortedSet( + ((SortedSet) unfiltered).subSet(fromElement, toElement), predicate); + } + + @Override + public SortedSet headSet(E toElement) { + return new FilteredSortedSet(((SortedSet) unfiltered).headSet(toElement), predicate); + } + + @Override + public SortedSet tailSet(E fromElement) { + return new FilteredSortedSet(((SortedSet) unfiltered).tailSet(fromElement), predicate); + } + + @Override + public E first() { + return Iterators.find(unfiltered.iterator(), predicate); + } + + @Override + public E last() { + SortedSet sortedUnfiltered = (SortedSet) unfiltered; + while (true) { + E element = sortedUnfiltered.last(); + if (predicate.apply(element)) { + return element; + } + sortedUnfiltered = sortedUnfiltered.headSet(element); + } + } + } + + private static class FilteredNavigableSet extends FilteredSortedSet + implements NavigableSet { + FilteredNavigableSet(NavigableSet unfiltered, Predicate predicate) { + super(unfiltered, predicate); + } + + NavigableSet unfiltered() { + return (NavigableSet) unfiltered; + } + + @Override + + public E lower(E e) { + return Iterators.find(unfiltered().headSet(e, false).descendingIterator(), predicate, null); + } + + @Override + + public E floor(E e) { + return Iterators.find(unfiltered().headSet(e, true).descendingIterator(), predicate, null); + } + + @Override + public E ceiling(E e) { + return Iterables.find(unfiltered().tailSet(e, true), predicate, null); + } + + @Override + public E higher(E e) { + return Iterables.find(unfiltered().tailSet(e, false), predicate, null); + } + + @Override + public E pollFirst() { + return Iterables.removeFirstMatching(unfiltered(), predicate); + } + + @Override + public E pollLast() { + return Iterables.removeFirstMatching(unfiltered().descendingSet(), predicate); + } + + @Override + public NavigableSet descendingSet() { + return Sets.filter(unfiltered().descendingSet(), predicate); + } + + @Override + public Iterator descendingIterator() { + return Iterators.filter(unfiltered().descendingIterator(), predicate); + } + + @Override + public E last() { + return Iterators.find(unfiltered().descendingIterator(), predicate); + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return filter( + unfiltered().subSet(fromElement, fromInclusive, toElement, toInclusive), predicate); + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + return filter(unfiltered().headSet(toElement, inclusive), predicate); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + return filter(unfiltered().tailSet(fromElement, inclusive), predicate); + } + } + + /** + * Returns every possible list that can be formed by choosing one element from each of the given + * sets in order; the "n-ary Cartesian + * product" of the sets. For example: + * + *

{@code
+   * Sets.cartesianProduct(ImmutableList.of(
+   *     ImmutableSet.of(1, 2),
+   *     ImmutableSet.of("A", "B", "C")))
+   * }
+ * + *

returns a set containing six lists: + * + *

    + *
  • {@code ImmutableList.of(1, "A")} + *
  • {@code ImmutableList.of(1, "B")} + *
  • {@code ImmutableList.of(1, "C")} + *
  • {@code ImmutableList.of(2, "A")} + *
  • {@code ImmutableList.of(2, "B")} + *
  • {@code ImmutableList.of(2, "C")} + *
+ * + *

The result is guaranteed to be in the "traditional", lexicographical order for Cartesian + * products that you would get from nesting for loops: + * + *

{@code
+   * for (B b0 : sets.get(0)) {
+   *   for (B b1 : sets.get(1)) {
+   *     ...
+   *     ImmutableList tuple = ImmutableList.of(b0, b1, ...);
+   *     // operate on tuple
+   *   }
+   * }
+   * }
+ * + *

Note that if any input set is empty, the Cartesian product will also be empty. If no sets at + * all are provided (an empty list), the resulting Cartesian product has one element, an empty + * list (counter-intuitive, but mathematically consistent). + * + *

Performance notes: while the cartesian product of sets of size {@code m, n, p} is a + * set of size {@code m x n x p}, its actual memory consumption is much smaller. When the + * cartesian set is constructed, the input sets are merely copied. Only as the resulting set is + * iterated are the individual lists created, and these are not retained after iteration. + * + * @param sets the sets to choose elements from, in the order that the elements chosen from those + * sets should appear in the resulting lists + * @param any common base class shared by all axes (often just {@link Object}) + * @return the Cartesian product, as an immutable set containing immutable lists + * @throws NullPointerException if {@code sets}, any one of the {@code sets}, or any element of a + * provided set is null + * @since 2.0 + */ + public static Set> cartesianProduct(List> sets) { + return CartesianSet.create(sets); + } + + /** + * Returns every possible list that can be formed by choosing one element from each of the given + * sets in order; the "n-ary Cartesian + * product" of the sets. For example: + * + *

{@code
+   * Sets.cartesianProduct(
+   *     ImmutableSet.of(1, 2),
+   *     ImmutableSet.of("A", "B", "C"))
+   * }
+ * + *

returns a set containing six lists: + * + *

    + *
  • {@code ImmutableList.of(1, "A")} + *
  • {@code ImmutableList.of(1, "B")} + *
  • {@code ImmutableList.of(1, "C")} + *
  • {@code ImmutableList.of(2, "A")} + *
  • {@code ImmutableList.of(2, "B")} + *
  • {@code ImmutableList.of(2, "C")} + *
+ * + *

The result is guaranteed to be in the "traditional", lexicographical order for Cartesian + * products that you would get from nesting for loops: + * + *

{@code
+   * for (B b0 : sets.get(0)) {
+   *   for (B b1 : sets.get(1)) {
+   *     ...
+   *     ImmutableList tuple = ImmutableList.of(b0, b1, ...);
+   *     // operate on tuple
+   *   }
+   * }
+   * }
+ * + *

Note that if any input set is empty, the Cartesian product will also be empty. If no sets at + * all are provided (an empty list), the resulting Cartesian product has one element, an empty + * list (counter-intuitive, but mathematically consistent). + * + *

Performance notes: while the cartesian product of sets of size {@code m, n, p} is a + * set of size {@code m x n x p}, its actual memory consumption is much smaller. When the + * cartesian set is constructed, the input sets are merely copied. Only as the resulting set is + * iterated are the individual lists created, and these are not retained after iteration. + * + * @param sets the sets to choose elements from, in the order that the elements chosen from those + * sets should appear in the resulting lists + * @param any common base class shared by all axes (often just {@link Object}) + * @return the Cartesian product, as an immutable set containing immutable lists + * @throws NullPointerException if {@code sets}, any one of the {@code sets}, or any element of a + * provided set is null + * @since 2.0 + */ + @SuppressWarnings("unchecked") + public static Set> cartesianProduct(Set... sets) { + return cartesianProduct(Arrays.asList(sets)); + } + + private static final class CartesianSet extends ForwardingCollection> + implements Set> { + private final transient ImmutableList> axes; + private final transient CartesianList delegate; + + static Set> create(List> sets) { + ImmutableList.Builder> axesBuilder = new ImmutableList.Builder<>(sets.size()); + for (Set set : sets) { + ImmutableSet copy = ImmutableSet.copyOf(set); + if (copy.isEmpty()) { + return ImmutableSet.of(); + } + axesBuilder.add(copy); + } + final ImmutableList> axes = axesBuilder.build(); + ImmutableList> listAxes = + new ImmutableList>() { + @Override + public int size() { + return axes.size(); + } + + @Override + public List get(int index) { + return axes.get(index).asList(); + } + + @Override + boolean isPartialView() { + return true; + } + }; + return new CartesianSet(axes, new CartesianList(listAxes)); + } + + private CartesianSet(ImmutableList> axes, CartesianList delegate) { + this.axes = axes; + this.delegate = delegate; + } + + @Override + protected Collection> delegate() { + return delegate; + } + + @Override + public boolean equals(Object object) { + // Warning: this is broken if size() == 0, so it is critical that we + // substitute an empty ImmutableSet to the user in place of this + if (object instanceof CartesianSet) { + CartesianSet that = (CartesianSet) object; + return this.axes.equals(that.axes); + } + return super.equals(object); + } + + @Override + public int hashCode() { + // Warning: this is broken if size() == 0, so it is critical that we + // substitute an empty ImmutableSet to the user in place of this + + // It's a weird formula, but tests prove it works. + int adjust = size() - 1; + for (int i = 0; i < axes.size(); i++) { + adjust *= 31; + adjust = ~~adjust; + // in GWT, we have to deal with integer overflow carefully + } + int hash = 1; + for (Set axis : axes) { + hash = 31 * hash + (size() / axis.size() * axis.hashCode()); + + hash = ~~hash; + } + hash += adjust; + return ~~hash; + } + } + + /** + * Returns the set of all possible subsets of {@code set}. For example, {@code + * powerSet(ImmutableSet.of(1, 2))} returns the set {@code {{}, {1}, {2}, {1, 2}}}. + * + *

Elements appear in these subsets in the same iteration order as they appeared in the input + * set. The order in which these subsets appear in the outer set is undefined. Note that the power + * set of the empty set is not the empty set, but a one-element set containing the empty set. + * + *

The returned set and its constituent sets use {@code equals} to decide whether two elements + * are identical, even if the input set uses a different concept of equivalence. + * + *

Performance notes: while the power set of a set with size {@code n} is of size {@code + * 2^n}, its memory usage is only {@code O(n)}. When the power set is constructed, the input set + * is merely copied. Only as the power set is iterated are the individual subsets created, and + * these subsets themselves occupy only a small constant amount of memory. + * + * @param set the set of elements to construct a power set from + * @return the power set, as an immutable set of immutable sets + * @throws IllegalArgumentException if {@code set} has more than 30 unique elements (causing the + * power set size to exceed the {@code int} range) + * @throws NullPointerException if {@code set} is or contains {@code null} + * @see Power set article at Wikipedia + * @since 4.0 + */ + public static Set> powerSet(Set set) { + return new PowerSet(set); + } + + private static final class SubSet extends AbstractSet { + private final ImmutableMap inputSet; + private final int mask; + + SubSet(ImmutableMap inputSet, int mask) { + this.inputSet = inputSet; + this.mask = mask; + } + + @Override + public Iterator iterator() { + return new UnmodifiableIterator() { + final ImmutableList elements = inputSet.keySet().asList(); + int remainingSetBits = mask; + + @Override + public boolean hasNext() { + return remainingSetBits != 0; + } + + @Override + public E next() { + int index = Integer.numberOfTrailingZeros(remainingSetBits); + if (index == 32) { + throw new NoSuchElementException(); + } + remainingSetBits &= ~(1 << index); + return elements.get(index); + } + }; + } + + @Override + public int size() { + return Integer.bitCount(mask); + } + + @Override + public boolean contains(Object o) { + Integer index = inputSet.get(o); + return index != null && (mask & (1 << index)) != 0; + } + } + + private static final class PowerSet extends AbstractSet> { + final ImmutableMap inputSet; + + PowerSet(Set input) { + this.inputSet = Maps.indexMap(input); + checkArgument( + inputSet.size() <= 30, "Too many elements to create power set: %s > 30", inputSet.size()); + } + + @Override + public int size() { + return 1 << inputSet.size(); + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Iterator> iterator() { + return new AbstractIndexedListIterator>(size()) { + @Override + protected Set get(final int setBits) { + return new SubSet(inputSet, setBits); + } + }; + } + + @Override + public boolean contains(Object obj) { + if (obj instanceof Set) { + Set set = (Set) obj; + return inputSet.keySet().containsAll(set); + } + return false; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof PowerSet) { + PowerSet that = (PowerSet) obj; + return inputSet.equals(that.inputSet); + } + return super.equals(obj); + } + + @Override + public int hashCode() { + /* + * The sum of the sums of the hash codes in each subset is just the sum of + * each input element's hash code times the number of sets that element + * appears in. Each element appears in exactly half of the 2^n sets, so: + */ + return inputSet.keySet().hashCode() << (inputSet.size() - 1); + } + + @Override + public String toString() { + return "powerSet(" + inputSet + ")"; + } + } + + /** + * Returns the set of all subsets of {@code set} of size {@code size}. For example, {@code + * combinations(ImmutableSet.of(1, 2, 3), 2)} returns the set {@code {{1, 2}, {1, 3}, {2, 3}}}. + * + *

Elements appear in these subsets in the same iteration order as they appeared in the input + * set. The order in which these subsets appear in the outer set is undefined. + * + *

The returned set and its constituent sets use {@code equals} to decide whether two elements + * are identical, even if the input set uses a different concept of equivalence. + * + *

Performance notes: the memory usage of the returned set is only {@code O(n)}. When + * the result set is constructed, the input set is merely copied. Only as the result set is + * iterated are the individual subsets created. Each of these subsets occupies an additional O(n) + * memory but only for as long as the user retains a reference to it. That is, the set returned by + * {@code combinations} does not retain the individual subsets. + * + * @param set the set of elements to take combinations of + * @param size the number of elements per combination + * @return the set of all combinations of {@code size} elements from {@code set} + * @throws IllegalArgumentException if {@code size} is not between 0 and {@code set.size()} + * inclusive + * @throws NullPointerException if {@code set} is or contains {@code null} + * @since 23.0 + */ + + public static Set> combinations(Set set, final int size) { + final ImmutableMap index = Maps.indexMap(set); + checkNonnegative(size, "size"); + checkArgument(size <= index.size(), "size (%s) must be <= set.size() (%s)", size, index.size()); + if (size == 0) { + return ImmutableSet.>of(ImmutableSet.of()); + } else if (size == index.size()) { + return ImmutableSet.>of(index.keySet()); + } + return new AbstractSet>() { + @Override + public boolean contains(Object o) { + if (o instanceof Set) { + Set s = (Set) o; + return s.size() == size && index.keySet().containsAll(s); + } + return false; + } + + @Override + public Iterator> iterator() { + return new AbstractIterator>() { + final BitSet bits = new BitSet(index.size()); + + @Override + protected Set computeNext() { + if (bits.isEmpty()) { + bits.set(0, size); + } else { + int firstSetBit = bits.nextSetBit(0); + int bitToFlip = bits.nextClearBit(firstSetBit); + + if (bitToFlip == index.size()) { + return endOfData(); + } + + /* + * The current set in sorted order looks like + * {firstSetBit, firstSetBit + 1, ..., bitToFlip - 1, ...} + * where it does *not* contain bitToFlip. + * + * The next combination is + * + * {0, 1, ..., bitToFlip - firstSetBit - 2, bitToFlip, ...} + * + * This is lexicographically next if you look at the combinations in descending order + * e.g. {2, 1, 0}, {3, 1, 0}, {3, 2, 0}, {3, 2, 1}, {4, 1, 0}... + */ + + bits.set(0, bitToFlip - firstSetBit - 1); + bits.clear(bitToFlip - firstSetBit - 1, bitToFlip); + bits.set(bitToFlip); + } + final BitSet copy = (BitSet) bits.clone(); + return new AbstractSet() { + @Override + public boolean contains(Object o) { + Integer i = index.get(o); + return i != null && copy.get(i); + } + + @Override + public Iterator iterator() { + return new AbstractIterator() { + int i = -1; + + @Override + protected E computeNext() { + i = copy.nextSetBit(i + 1); + if (i == -1) { + return endOfData(); + } + return index.keySet().asList().get(i); + } + }; + } + + @Override + public int size() { + return size; + } + }; + } + }; + } + + @Override + public int size() { + return IntMath.binomial(index.size(), size); + } + + @Override + public String toString() { + return "Sets.combinations(" + index.keySet() + ", " + size + ")"; + } + }; + } + + /** An implementation for {@link Set#hashCode()}. */ + static int hashCodeImpl(Set s) { + int hashCode = 0; + for (Object o : s) { + hashCode += o != null ? o.hashCode() : 0; + + hashCode = ~~hashCode; + // Needed to deal with unusual integer overflow in GWT. + } + return hashCode; + } + + /** An implementation for {@link Set#equals(Object)}. */ + static boolean equalsImpl(Set s, Object object) { + if (s == object) { + return true; + } + if (object instanceof Set) { + Set o = (Set) object; + + try { + return s.size() == o.size() && s.containsAll(o); + } catch (NullPointerException | ClassCastException ignored) { + return false; + } + } + return false; + } + + /** + * Returns an unmodifiable view of the specified navigable set. This method allows modules to + * provide users with "read-only" access to internal navigable sets. Query operations on the + * returned set "read through" to the specified set, and attempts to modify the returned set, + * whether direct or via its collection views, result in an {@code UnsupportedOperationException}. + * + *

The returned navigable set will be serializable if the specified navigable set is + * serializable. + * + * @param set the navigable set for which an unmodifiable view is to be returned + * @return an unmodifiable view of the specified navigable set + * @since 12.0 + */ + public static NavigableSet unmodifiableNavigableSet(NavigableSet set) { + if (set instanceof ImmutableCollection || set instanceof UnmodifiableNavigableSet) { + return set; + } + return new UnmodifiableNavigableSet(set); + } + + static final class UnmodifiableNavigableSet extends ForwardingSortedSet + implements NavigableSet, Serializable { + private final NavigableSet delegate; + private final SortedSet unmodifiableDelegate; + + UnmodifiableNavigableSet(NavigableSet delegate) { + this.delegate = checkNotNull(delegate); + this.unmodifiableDelegate = Collections.unmodifiableSortedSet(delegate); + } + + @Override + protected SortedSet delegate() { + return unmodifiableDelegate; + } + + // default methods not forwarded by ForwardingSortedSet + + @Override + public boolean removeIf(java.util.function.Predicate filter) { + throw new UnsupportedOperationException(); + } + + @Override + public Stream stream() { + return delegate.stream(); + } + + @Override + public Stream parallelStream() { + return delegate.parallelStream(); + } + + @Override + public void forEach(Consumer action) { + delegate.forEach(action); + } + + @Override + public E lower(E e) { + return delegate.lower(e); + } + + @Override + public E floor(E e) { + return delegate.floor(e); + } + + @Override + public E ceiling(E e) { + return delegate.ceiling(e); + } + + @Override + public E higher(E e) { + return delegate.higher(e); + } + + @Override + public E pollFirst() { + throw new UnsupportedOperationException(); + } + + @Override + public E pollLast() { + throw new UnsupportedOperationException(); + } + + private transient UnmodifiableNavigableSet descendingSet; + + @Override + public NavigableSet descendingSet() { + UnmodifiableNavigableSet result = descendingSet; + if (result == null) { + result = descendingSet = new UnmodifiableNavigableSet(delegate.descendingSet()); + result.descendingSet = this; + } + return result; + } + + @Override + public Iterator descendingIterator() { + return Iterators.unmodifiableIterator(delegate.descendingIterator()); + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return unmodifiableNavigableSet( + delegate.subSet(fromElement, fromInclusive, toElement, toInclusive)); + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + return unmodifiableNavigableSet(delegate.headSet(toElement, inclusive)); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + return unmodifiableNavigableSet(delegate.tailSet(fromElement, inclusive)); + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns a synchronized (thread-safe) navigable set backed by the specified navigable set. In + * order to guarantee serial access, it is critical that all access to the backing + * navigable set is accomplished through the returned navigable set (or its views). + * + *

It is imperative that the user manually synchronize on the returned sorted set when + * iterating over it or any of its {@code descendingSet}, {@code subSet}, {@code headSet}, or + * {@code tailSet} views. + * + *

{@code
+   * NavigableSet set = synchronizedNavigableSet(new TreeSet());
+   *  ...
+   * synchronized (set) {
+   *   // Must be in the synchronized block
+   *   Iterator it = set.iterator();
+   *   while (it.hasNext()) {
+   *     foo(it.next());
+   *   }
+   * }
+   * }
+ * + *

or: + * + *

{@code
+   * NavigableSet set = synchronizedNavigableSet(new TreeSet());
+   * NavigableSet set2 = set.descendingSet().headSet(foo);
+   *  ...
+   * synchronized (set) { // Note: set, not set2!!!
+   *   // Must be in the synchronized block
+   *   Iterator it = set2.descendingIterator();
+   *   while (it.hasNext())
+   *     foo(it.next());
+   *   }
+   * }
+   * }
+ * + *

Failure to follow this advice may result in non-deterministic behavior. + * + *

The returned navigable set will be serializable if the specified navigable set is + * serializable. + * + * @param navigableSet the navigable set to be "wrapped" in a synchronized navigable set. + * @return a synchronized view of the specified navigable set. + * @since 13.0 + */ + public static NavigableSet synchronizedNavigableSet(NavigableSet navigableSet) { + return Synchronized.navigableSet(navigableSet); + } + + /** Remove each element in an iterable from a set. */ + static boolean removeAllImpl(Set set, Iterator iterator) { + boolean changed = false; + while (iterator.hasNext()) { + changed |= set.remove(iterator.next()); + } + return changed; + } + + static boolean removeAllImpl(Set set, Collection collection) { + checkNotNull(collection); // for GWT + if (collection instanceof Multiset) { + collection = ((Multiset) collection).elementSet(); + } + /* + * AbstractSet.removeAll(List) has quadratic behavior if the list size + * is just more than the set's size. We augment the test by + * assuming that sets have fast contains() performance, and other + * collections don't. See + * http://code.google.com/p/guava-libraries/issues/detail?id=1013 + */ + if (collection instanceof Set && collection.size() > set.size()) { + return Iterators.removeAll(set.iterator(), collection); + } else { + return removeAllImpl(set, collection.iterator()); + } + } + + static class DescendingSet extends ForwardingNavigableSet { + private final NavigableSet forward; + + DescendingSet(NavigableSet forward) { + this.forward = forward; + } + + @Override + protected NavigableSet delegate() { + return forward; + } + + @Override + public E lower(E e) { + return forward.higher(e); + } + + @Override + public E floor(E e) { + return forward.ceiling(e); + } + + @Override + public E ceiling(E e) { + return forward.floor(e); + } + + @Override + public E higher(E e) { + return forward.lower(e); + } + + @Override + public E pollFirst() { + return forward.pollLast(); + } + + @Override + public E pollLast() { + return forward.pollFirst(); + } + + @Override + public NavigableSet descendingSet() { + return forward; + } + + @Override + public Iterator descendingIterator() { + return forward.iterator(); + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return forward.subSet(toElement, toInclusive, fromElement, fromInclusive).descendingSet(); + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return standardSubSet(fromElement, toElement); + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + return forward.tailSet(toElement, inclusive).descendingSet(); + } + + @Override + public SortedSet headSet(E toElement) { + return standardHeadSet(toElement); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + return forward.headSet(fromElement, inclusive).descendingSet(); + } + + @Override + public SortedSet tailSet(E fromElement) { + return standardTailSet(fromElement); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + public Comparator comparator() { + Comparator forwardComparator = forward.comparator(); + if (forwardComparator == null) { + return (Comparator) Ordering.natural().reverse(); + } else { + return reverse(forwardComparator); + } + } + + // If we inline this, we get a javac error. + private static Ordering reverse(Comparator forward) { + return Ordering.from(forward).reverse(); + } + + @Override + public E first() { + return forward.last(); + } + + @Override + public E last() { + return forward.first(); + } + + @Override + public Iterator iterator() { + return forward.descendingIterator(); + } + + @Override + public Object[] toArray() { + return standardToArray(); + } + + @Override + public T[] toArray(T[] array) { + return standardToArray(array); + } + + @Override + public String toString() { + return standardToString(); + } + } + + /** + * Returns a view of the portion of {@code set} whose elements are contained by {@code range}. + * + *

This method delegates to the appropriate methods of {@link NavigableSet} (namely {@link + * NavigableSet#subSet(Object, boolean, Object, boolean) subSet()}, {@link + * NavigableSet#tailSet(Object, boolean) tailSet()}, and {@link NavigableSet#headSet(Object, + * boolean) headSet()}) to actually construct the view. Consult these methods for a full + * description of the returned view's behavior. + * + *

Warning: {@code Range}s always represent a range of values using the values' natural + * ordering. {@code NavigableSet} on the other hand can specify a custom ordering via a {@link + * Comparator}, which can violate the natural ordering. Using this method (or in general using + * {@code Range}) with unnaturally-ordered sets can lead to unexpected and undefined behavior. + * + * @since 20.0 + */ + + public static > NavigableSet subSet( + NavigableSet set, Range range) { + if (set.comparator() != null + && set.comparator() != Ordering.natural() + && range.hasLowerBound() + && range.hasUpperBound()) { + checkArgument( + set.comparator().compare(range.lowerEndpoint(), range.upperEndpoint()) <= 0, + "set is using a custom comparator which is inconsistent with the natural ordering."); + } + if (range.hasLowerBound() && range.hasUpperBound()) { + return set.subSet( + range.lowerEndpoint(), + range.lowerBoundType() == BoundType.CLOSED, + range.upperEndpoint(), + range.upperBoundType() == BoundType.CLOSED); + } else if (range.hasLowerBound()) { + return set.tailSet(range.lowerEndpoint(), range.lowerBoundType() == BoundType.CLOSED); + } else if (range.hasUpperBound()) { + return set.headSet(range.upperEndpoint(), range.upperBoundType() == BoundType.CLOSED); + } + return checkNotNull(set); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableBiMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableBiMap.java new file mode 100644 index 0000000000000..34ab9e6578a85 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableBiMap.java @@ -0,0 +1,105 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkEntryNotNull; + +import java.util.function.BiConsumer; + + +/** + * Implementation of {@link ImmutableMap} with exactly one entry. + * + * @author Jesse Wilson + * @author Kevin Bourrillion + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +final class SingletonImmutableBiMap extends ImmutableBiMap { + + final transient K singleKey; + final transient V singleValue; + + SingletonImmutableBiMap(K singleKey, V singleValue) { + checkEntryNotNull(singleKey, singleValue); + this.singleKey = singleKey; + this.singleValue = singleValue; + } + + private SingletonImmutableBiMap(K singleKey, V singleValue, ImmutableBiMap inverse) { + this.singleKey = singleKey; + this.singleValue = singleValue; + this.inverse = inverse; + } + + @Override + public V get(Object key) { + return singleKey.equals(key) ? singleValue : null; + } + + @Override + public int size() { + return 1; + } + + @Override + public void forEach(BiConsumer action) { + checkNotNull(action).accept(singleKey, singleValue); + } + + @Override + public boolean containsKey(Object key) { + return singleKey.equals(key); + } + + @Override + public boolean containsValue(Object value) { + return singleValue.equals(value); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + ImmutableSet> createEntrySet() { + return ImmutableSet.of(Maps.immutableEntry(singleKey, singleValue)); + } + + @Override + ImmutableSet createKeySet() { + return ImmutableSet.of(singleKey); + } + + transient ImmutableBiMap inverse; + + @Override + public ImmutableBiMap inverse() { + // racy single-check idiom + ImmutableBiMap result = inverse; + if (result == null) { + return inverse = new SingletonImmutableBiMap<>(singleValue, singleKey, this); + } else { + return result; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableList.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableList.java new file mode 100644 index 0000000000000..8e862d859edec --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableList.java @@ -0,0 +1,80 @@ +/* + * Copyright (C) 2009 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +import java.util.Collections; +import java.util.Spliterator; + +/** + * Implementation of {@link ImmutableList} with exactly one element. + * + * @author Hayward Chan + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +final class SingletonImmutableList extends ImmutableList { + + final transient E element; + + SingletonImmutableList(E element) { + this.element = checkNotNull(element); + } + + @Override + public E get(int index) { + Preconditions.checkElementIndex(index, 1); + return element; + } + + @Override + public UnmodifiableIterator iterator() { + return Iterators.singletonIterator(element); + } + + @Override + public Spliterator spliterator() { + return Collections.singleton(element).spliterator(); + } + + @Override + public int size() { + return 1; + } + + @Override + public ImmutableList subList(int fromIndex, int toIndex) { + Preconditions.checkPositionIndexes(fromIndex, toIndex, 1); + return (fromIndex == toIndex) ? ImmutableList.of() : this; + } + + @Override + public String toString() { + return '[' + element.toString() + ']'; + } + + @Override + boolean isPartialView() { + return false; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableSet.java new file mode 100644 index 0000000000000..fa1fcddc0fa6f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SingletonImmutableSet.java @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions; + +/** + * Implementation of {@link ImmutableSet} with exactly one element. + * + * @author Kevin Bourrillion + * @author Nick Kralevich + */ +@SuppressWarnings("serial") // uses writeReplace(), not default serialization +final class SingletonImmutableSet extends ImmutableSet { + + final transient E element; + // This is transient because it will be recalculated on the first + // call to hashCode(). + // + // A race condition is avoided since threads will either see that the value + // is zero and recalculate it themselves, or two threads will see it at + // the same time, and both recalculate it. If the cachedHashCode is 0, + // it will always be recalculated, unfortunately. + private transient int cachedHashCode; + + SingletonImmutableSet(E element) { + this.element = Preconditions.checkNotNull(element); + } + + SingletonImmutableSet(E element, int hashCode) { + // Guaranteed to be non-null by the presence of the pre-computed hash code. + this.element = element; + cachedHashCode = hashCode; + } + + @Override + public int size() { + return 1; + } + + @Override + public boolean contains(Object target) { + return element.equals(target); + } + + @Override + public UnmodifiableIterator iterator() { + return Iterators.singletonIterator(element); + } + + @Override + ImmutableList createAsList() { + return ImmutableList.of(element); + } + + @Override + boolean isPartialView() { + return false; + } + + @Override + int copyIntoArray(Object[] dst, int offset) { + dst[offset] = element; + return offset + 1; + } + + @Override + public final int hashCode() { + // Racy single-check. + int code = cachedHashCode; + if (code == 0) { + cachedHashCode = code = element.hashCode(); + } + return code; + } + + @Override + boolean isHashCodeFast() { + return cachedHashCode != 0; + } + + @Override + public String toString() { + return '[' + element.toString() + ']'; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterable.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterable.java new file mode 100644 index 0000000000000..97b6025836cc3 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterable.java @@ -0,0 +1,43 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Comparator; +import java.util.Iterator; + +/** + * An {@code Iterable} whose elements are sorted relative to a {@code Comparator}, typically + * provided at creation time. + * + * @author Louis Wasserman + */ +interface SortedIterable extends Iterable { + /** + * Returns the {@code Comparator} by which the elements of this iterable are ordered, or {@code + * Ordering.natural()} if the elements are ordered by their natural ordering. + */ + Comparator comparator(); + + /** + * Returns an iterator over elements of type {@code T}. The elements are returned in nondecreasing + * order according to the associated {@link #comparator}. + */ + @Override + Iterator iterator(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterables.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterables.java new file mode 100644 index 0000000000000..3bef738fdbf46 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedIterables.java @@ -0,0 +1,61 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the + * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either + * express or implied. See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Comparator; +import java.util.SortedSet; + +/** + * Utilities for dealing with sorted collections of all types. + * + * @author Louis Wasserman + */ +final class SortedIterables { + private SortedIterables() {} + + /** + * Returns {@code true} if {@code elements} is a sorted collection using an ordering equivalent to + * {@code comparator}. + */ + public static boolean hasSameComparator(Comparator comparator, Iterable elements) { + checkNotNull(comparator); + checkNotNull(elements); + Comparator comparator2; + if (elements instanceof SortedSet) { + comparator2 = comparator((SortedSet) elements); + } else if (elements instanceof SortedIterable) { + comparator2 = ((SortedIterable) elements).comparator(); + } else { + return false; + } + return comparator.equals(comparator2); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + // if sortedSet.comparator() is null, the set must be naturally ordered + public static Comparator comparator(SortedSet sortedSet) { + Comparator result = sortedSet.comparator(); + if (result == null) { + result = (Comparator) Ordering.natural(); + } + return result; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedLists.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedLists.java new file mode 100644 index 0000000000000..d6577bd6f1be7 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedLists.java @@ -0,0 +1,285 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.RandomAccess; + + +/** + * Static methods pertaining to sorted {@link List} instances. + * + *

In this documentation, the terms greatest, greater, least, and + * lesser are considered to refer to the comparator on the elements, and the terms + * first and last are considered to refer to the elements' ordering in a list. + * + * @author Louis Wasserman + */ + final class SortedLists { + private SortedLists() {} + + /** + * A specification for which index to return if the list contains at least one element that + * compares as equal to the key. + */ enum KeyPresentBehavior { + /** + * Return the index of any list element that compares as equal to the key. No guarantees are + * made as to which index is returned, if more than one element compares as equal to the key. + */ + ANY_PRESENT { + @Override + int resultIndex( + Comparator comparator, E key, List list, int foundIndex) { + return foundIndex; + } + }, + /** Return the index of the last list element that compares as equal to the key. */ + LAST_PRESENT { + @Override + int resultIndex( + Comparator comparator, E key, List list, int foundIndex) { + // Of course, we have to use binary search to find the precise + // breakpoint... + int lower = foundIndex; + int upper = list.size() - 1; + // Everything between lower and upper inclusive compares at >= 0. + while (lower < upper) { + int middle = (lower + upper + 1) >>> 1; + int c = comparator.compare(list.get(middle), key); + if (c > 0) { + upper = middle - 1; + } else { // c == 0 + lower = middle; + } + } + return lower; + } + }, + /** Return the index of the first list element that compares as equal to the key. */ + FIRST_PRESENT { + @Override + int resultIndex( + Comparator comparator, E key, List list, int foundIndex) { + // Of course, we have to use binary search to find the precise + // breakpoint... + int lower = 0; + int upper = foundIndex; + // Of course, we have to use binary search to find the precise breakpoint... + // Everything between lower and upper inclusive compares at <= 0. + while (lower < upper) { + int middle = (lower + upper) >>> 1; + int c = comparator.compare(list.get(middle), key); + if (c < 0) { + lower = middle + 1; + } else { // c == 0 + upper = middle; + } + } + return lower; + } + }, + /** + * Return the index of the first list element that compares as greater than the key, or {@code + * list.size()} if there is no such element. + */ + FIRST_AFTER { + @Override + public int resultIndex( + Comparator comparator, E key, List list, int foundIndex) { + return LAST_PRESENT.resultIndex(comparator, key, list, foundIndex) + 1; + } + }, + /** + * Return the index of the last list element that compares as less than the key, or {@code -1} + * if there is no such element. + */ + LAST_BEFORE { + @Override + public int resultIndex( + Comparator comparator, E key, List list, int foundIndex) { + return FIRST_PRESENT.resultIndex(comparator, key, list, foundIndex) - 1; + } + }; + + abstract int resultIndex( + Comparator comparator, E key, List list, int foundIndex); + } + + /** + * A specification for which index to return if the list contains no elements that compare as + * equal to the key. + */ enum KeyAbsentBehavior { + /** + * Return the index of the next lower element in the list, or {@code -1} if there is no such + * element. + */ + NEXT_LOWER { + @Override + int resultIndex(int higherIndex) { + return higherIndex - 1; + } + }, + /** + * Return the index of the next higher element in the list, or {@code list.size()} if there is + * no such element. + */ + NEXT_HIGHER { + @Override + public int resultIndex(int higherIndex) { + return higherIndex; + } + }, + /** + * Return {@code ~insertionIndex}, where {@code insertionIndex} is defined as the point at which + * the key would be inserted into the list: the index of the next higher element in the list, or + * {@code list.size()} if there is no such element. + * + *

Note that the return value will be {@code >= 0} if and only if there is an element of the + * list that compares as equal to the key. + * + *

This is equivalent to the behavior of {@link Collections#binarySearch(List, + * Object)} when the key isn't present, since {@code ~insertionIndex} is equal to {@code -1 - + * insertionIndex}. + */ + INVERTED_INSERTION_INDEX { + @Override + public int resultIndex(int higherIndex) { + return ~higherIndex; + } + }; + + abstract int resultIndex(int higherIndex); + } + + /** + * Searches the specified naturally ordered list for the specified object using the binary search + * algorithm. + * + *

Equivalent to {@link #binarySearch(List, Function, Object, Comparator, KeyPresentBehavior, + * KeyAbsentBehavior)} using {@link Ordering#natural}. + */ + @SuppressWarnings("rawtypes") + public static int binarySearch( + List list, + E e, + KeyPresentBehavior presentBehavior, + KeyAbsentBehavior absentBehavior) { + checkNotNull(e); + return binarySearch(list, e, Ordering.natural(), presentBehavior, absentBehavior); + } + + /** + * Binary searches the list for the specified key, using the specified key function. + * + *

Equivalent to {@link #binarySearch(List, Function, Object, Comparator, KeyPresentBehavior, + * KeyAbsentBehavior)} using {@link Ordering#natural}. + */ + @SuppressWarnings("rawtypes") + public static int binarySearch( + List list, + Function keyFunction, + K key, + KeyPresentBehavior presentBehavior, + KeyAbsentBehavior absentBehavior) { + return binarySearch( + list, keyFunction, key, Ordering.natural(), presentBehavior, absentBehavior); + } + + /** + * Binary searches the list for the specified key, using the specified key function. + * + *

Equivalent to {@link #binarySearch(List, Object, Comparator, KeyPresentBehavior, + * KeyAbsentBehavior)} using {@link Lists#transform(List, Function) Lists.transform(list, + * keyFunction)}. + */ + public static int binarySearch( + List list, + Function keyFunction, + K key, + Comparator keyComparator, + KeyPresentBehavior presentBehavior, + KeyAbsentBehavior absentBehavior) { + return binarySearch( + Lists.transform(list, keyFunction), key, keyComparator, presentBehavior, absentBehavior); + } + + /** + * Searches the specified list for the specified object using the binary search algorithm. The + * list must be sorted into ascending order according to the specified comparator (as by the + * {@link Collections#sort(List, Comparator) Collections.sort(List, Comparator)} method), prior to + * making this call. If it is not sorted, the results are undefined. + * + *

If there are elements in the list which compare as equal to the key, the choice of {@link + * KeyPresentBehavior} decides which index is returned. If no elements compare as equal to the + * key, the choice of {@link KeyAbsentBehavior} decides which index is returned. + * + *

This method runs in log(n) time on random-access lists, which offer near-constant-time + * access to each list element. + * + * @param list the list to be searched. + * @param key the value to be searched for. + * @param comparator the comparator by which the list is ordered. + * @param presentBehavior the specification for what to do if at least one element of the list + * compares as equal to the key. + * @param absentBehavior the specification for what to do if no elements of the list compare as + * equal to the key. + * @return the index determined by the {@code KeyPresentBehavior}, if the key is in the list; + * otherwise the index determined by the {@code KeyAbsentBehavior}. + */ + public static int binarySearch( + List list, + E key, + Comparator comparator, + KeyPresentBehavior presentBehavior, + KeyAbsentBehavior absentBehavior) { + checkNotNull(comparator); + checkNotNull(list); + checkNotNull(presentBehavior); + checkNotNull(absentBehavior); + if (!(list instanceof RandomAccess)) { + list = Lists.newArrayList(list); + } + // TODO(lowasser): benchmark when it's best to do a linear search + + int lower = 0; + int upper = list.size() - 1; + + while (lower <= upper) { + int middle = (lower + upper) >>> 1; + int c = comparator.compare(key, list.get(middle)); + if (c < 0) { + upper = middle - 1; + } else if (c > 0) { + lower = middle + 1; + } else { + return lower + + presentBehavior.resultIndex( + comparator, key, list.subList(lower, upper + 1), middle - lower); + } + } + return absentBehavior.resultIndex(lower); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMapDifference.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMapDifference.java new file mode 100644 index 0000000000000..fd33367d383e0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMapDifference.java @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.SortedMap; + +/** + * An object representing the differences between two sorted maps. + * + * @author Louis Wasserman + * @since 8.0 + */ +public interface SortedMapDifference extends MapDifference { + + @Override + SortedMap entriesOnlyOnLeft(); + + @Override + SortedMap entriesOnlyOnRight(); + + @Override + SortedMap entriesInCommon(); + + @Override + SortedMap> entriesDiffering(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultiset.java new file mode 100644 index 0000000000000..35835aceabde9 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultiset.java @@ -0,0 +1,146 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.Set; + +/** + * A {@link Multiset} which maintains the ordering of its elements, according to either their + * natural order or an explicit {@link Comparator}. This order is reflected when iterating over the + * sorted multiset, either directly, or through its {@code elementSet} or {@code entrySet} views. In + * all cases, this implementation uses {@link Comparable#compareTo} or {@link Comparator#compare} + * instead of {@link Object#equals} to determine equivalence of instances. + * + *

Warning: The comparison must be consistent with equals as explained by the + * {@link Comparable} class specification. Otherwise, the resulting multiset will violate the {@link + * Collection} contract, which it is specified in terms of {@link Object#equals}. + * + *

See the Guava User Guide article on {@code + * Multiset}. + * + * @author Louis Wasserman + * @since 11.0 + */ +public interface SortedMultiset extends SortedMultisetBridge, SortedIterable { + /** + * Returns the comparator that orders this multiset, or {@link Ordering#natural()} if the natural + * ordering of the elements is used. + */ + Comparator comparator(); + + /** + * Returns the entry of the first element in this multiset, or {@code null} if this multiset is + * empty. + */ + Entry firstEntry(); + + /** + * Returns the entry of the last element in this multiset, or {@code null} if this multiset is + * empty. + */ + Entry lastEntry(); + + /** + * Returns and removes the entry associated with the lowest element in this multiset, or returns + * {@code null} if this multiset is empty. + */ + Entry pollFirstEntry(); + + /** + * Returns and removes the entry associated with the greatest element in this multiset, or returns + * {@code null} if this multiset is empty. + */ + Entry pollLastEntry(); + + /** + * Returns a {@link NavigableSet} view of the distinct elements in this multiset. + * + * @since 14.0 (present with return type {@code SortedSet} since 11.0) + */ + @Override + NavigableSet elementSet(); + + /** + * {@inheritDoc} + * + *

The {@code entrySet}'s iterator returns entries in ascending element order according to the + * this multiset's comparator. + */ + @Override + Set> entrySet(); + + /** + * {@inheritDoc} + * + *

The iterator returns the elements in ascending order according to this multiset's + * comparator. + */ + @Override + Iterator iterator(); + + /** + * Returns a descending view of this multiset. Modifications made to either map will be reflected + * in the other. + */ + SortedMultiset descendingMultiset(); + + /** + * Returns a view of this multiset restricted to the elements less than {@code upperBound}, + * optionally including {@code upperBound} itself. The returned multiset is a view of this + * multiset, so changes to one will be reflected in the other. The returned multiset supports all + * operations that this multiset supports. + * + *

The returned multiset will throw an {@link IllegalArgumentException} on attempts to add + * elements outside its range. + */ + SortedMultiset headMultiset(E upperBound, BoundType boundType); + + /** + * Returns a view of this multiset restricted to the range between {@code lowerBound} and {@code + * upperBound}. The returned multiset is a view of this multiset, so changes to one will be + * reflected in the other. The returned multiset supports all operations that this multiset + * supports. + * + *

The returned multiset will throw an {@link IllegalArgumentException} on attempts to add + * elements outside its range. + * + *

This method is equivalent to {@code tailMultiset(lowerBound, + * lowerBoundType).headMultiset(upperBound, upperBoundType)}. + */ + SortedMultiset subMultiset( + E lowerBound, BoundType lowerBoundType, E upperBound, BoundType upperBoundType); + + /** + * Returns a view of this multiset restricted to the elements greater than {@code lowerBound}, + * optionally including {@code lowerBound} itself. The returned multiset is a view of this + * multiset, so changes to one will be reflected in the other. The returned multiset supports all + * operations that this multiset supports. + * + *

The returned multiset will throw an {@link IllegalArgumentException} on attempts to add + * elements outside its range. + */ + SortedMultiset tailMultiset(E lowerBound, BoundType boundType); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisetBridge.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisetBridge.java new file mode 100644 index 0000000000000..82e0c5c14c9ab --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisetBridge.java @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.SortedSet; + +/** + * Superinterface of {@link SortedMultiset} to introduce a bridge method for {@code elementSet()}, + * to ensure binary compatibility with older Guava versions that specified {@code elementSet()} to + * return {@code SortedSet}. + * + * @author Louis Wasserman + */ +interface SortedMultisetBridge extends Multiset { + @Override + SortedSet elementSet(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisets.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisets.java new file mode 100644 index 0000000000000..d38dbe96a5dcc --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedMultisets.java @@ -0,0 +1,171 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.BoundType.CLOSED; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.BoundType.OPEN; + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multiset.Entry; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.NavigableSet; +import java.util.NoSuchElementException; +import java.util.SortedSet; + + +/** + * Provides static utility methods for creating and working with {@link SortedMultiset} instances. + * + * @author Louis Wasserman + */ +final class SortedMultisets { + private SortedMultisets() {} + + /** A skeleton implementation for {@link SortedMultiset#elementSet}. */ + static class ElementSet extends Multisets.ElementSet implements SortedSet { + private final SortedMultiset multiset; + + ElementSet(SortedMultiset multiset) { + this.multiset = multiset; + } + + @Override + final SortedMultiset multiset() { + return multiset; + } + + @Override + public Iterator iterator() { + return Multisets.elementIterator(multiset().entrySet().iterator()); + } + + @Override + public Comparator comparator() { + return multiset().comparator(); + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return multiset().subMultiset(fromElement, CLOSED, toElement, OPEN).elementSet(); + } + + @Override + public SortedSet headSet(E toElement) { + return multiset().headMultiset(toElement, OPEN).elementSet(); + } + + @Override + public SortedSet tailSet(E fromElement) { + return multiset().tailMultiset(fromElement, CLOSED).elementSet(); + } + + @Override + public E first() { + return getElementOrThrow(multiset().firstEntry()); + } + + @Override + public E last() { + return getElementOrThrow(multiset().lastEntry()); + } + } + + /** A skeleton navigable implementation for {@link SortedMultiset#elementSet}. */ + static class NavigableElementSet extends ElementSet implements NavigableSet { + NavigableElementSet(SortedMultiset multiset) { + super(multiset); + } + + @Override + public E lower(E e) { + return getElementOrNull(multiset().headMultiset(e, OPEN).lastEntry()); + } + + @Override + public E floor(E e) { + return getElementOrNull(multiset().headMultiset(e, CLOSED).lastEntry()); + } + + @Override + public E ceiling(E e) { + return getElementOrNull(multiset().tailMultiset(e, CLOSED).firstEntry()); + } + + @Override + public E higher(E e) { + return getElementOrNull(multiset().tailMultiset(e, OPEN).firstEntry()); + } + + @Override + public NavigableSet descendingSet() { + return new NavigableElementSet(multiset().descendingMultiset()); + } + + @Override + public Iterator descendingIterator() { + return descendingSet().iterator(); + } + + @Override + public E pollFirst() { + return getElementOrNull(multiset().pollFirstEntry()); + } + + @Override + public E pollLast() { + return getElementOrNull(multiset().pollLastEntry()); + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + return new NavigableElementSet( + multiset() + .subMultiset( + fromElement, BoundType.forBoolean(fromInclusive), + toElement, BoundType.forBoolean(toInclusive))); + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + return new NavigableElementSet( + multiset().headMultiset(toElement, BoundType.forBoolean(inclusive))); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + return new NavigableElementSet( + multiset().tailMultiset(fromElement, BoundType.forBoolean(inclusive))); + } + } + + private static E getElementOrThrow(Entry entry) { + if (entry == null) { + throw new NoSuchElementException(); + } + return entry.getElement(); + } + + private static E getElementOrNull(Entry entry) { + return (entry == null) ? null : entry.getElement(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedSetMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedSetMultimap.java new file mode 100644 index 0000000000000..b156cfd20af1e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/SortedSetMultimap.java @@ -0,0 +1,107 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; + + +/** + * A {@code SetMultimap} whose set of values for a given key are kept sorted; that is, they comprise + * a {@link SortedSet}. It cannot hold duplicate key-value pairs; adding a key-value pair that's + * already in the multimap has no effect. This interface does not specify the ordering of the + * multimap's keys. See the {@link Multimap} documentation for information common to all multimaps. + * + *

The {@link #get}, {@link #removeAll}, and {@link #replaceValues} methods each return a {@link + * SortedSet} of values, while {@link Multimap#entries()} returns a {@link Set} of map entries. + * Though the method signature doesn't say so explicitly, the map returned by {@link #asMap} has + * {@code SortedSet} values. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @since 2.0 + */ +public interface SortedSetMultimap extends SetMultimap { + // Following Javadoc copied from Multimap. + + /** + * Returns a collection view of all values associated with a key. If no mappings in the multimap + * have the provided key, an empty collection is returned. + * + *

Changes to the returned collection will update the underlying multimap, and vice versa. + * + *

Because a {@code SortedSetMultimap} has unique sorted values for a given key, this method + * returns a {@link SortedSet}, instead of the {@link Collection} specified in the + * {@link Multimap} interface. + */ + @Override + SortedSet get(K key); + + /** + * Removes all values associated with a given key. + * + *

Because a {@code SortedSetMultimap} has unique sorted values for a given key, this method + * returns a {@link SortedSet}, instead of the {@link Collection} specified in the + * {@link Multimap} interface. + */ + @Override + SortedSet removeAll(Object key); + + /** + * Stores a collection of values with the same key, replacing any existing values for that key. + * + *

Because a {@code SortedSetMultimap} has unique sorted values for a given key, this method + * returns a {@link SortedSet}, instead of the {@link Collection} specified in the + * {@link Multimap} interface. + * + *

Any duplicates in {@code values} will be stored in the multimap once. + */ + @Override + SortedSet replaceValues(K key, Iterable values); + + /** + * Returns a map view that associates each key with the corresponding values in the multimap. + * Changes to the returned map, such as element removal, will update the underlying multimap. The + * map does not support {@code setValue()} on its entries, {@code put}, or {@code putAll}. + * + *

When passed a key that is present in the map, {@code asMap().get(Object)} has the same + * behavior as {@link #get}, returning a live collection. When passed a key that is not present, + * however, {@code asMap().get(Object)} returns {@code null} instead of an empty collection. + * + *

Note: The returned map's values are guaranteed to be of type {@link SortedSet}. To + * obtain this map with the more specific generic type {@code Map>}, call {@link + * Multimaps#asMap(SortedSetMultimap)} instead. + */ + @Override + Map> asMap(); + + /** + * Returns the comparator that orders the multimap values, with {@code null} indicating that + * natural ordering is used. + */ + Comparator valueComparator(); +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Streams.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Streams.java new file mode 100644 index 0000000000000..e1c72594e4af4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Streams.java @@ -0,0 +1,871 @@ +/* + * Copyright (C) 2015 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You may + * obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.math.LongMath; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.Deque; +import java.util.Iterator; +import java.util.OptionalDouble; +import java.util.OptionalInt; +import java.util.OptionalLong; +import java.util.PrimitiveIterator; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.Spliterators.AbstractSpliterator; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.DoubleConsumer; +import java.util.function.IntConsumer; +import java.util.function.LongConsumer; +import java.util.stream.DoubleStream; +import java.util.stream.IntStream; +import java.util.stream.LongStream; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; + + +/** + * Static utility methods related to {@code Stream} instances. + * + * @since 21.0 + */ + +public final class Streams { + /** + * Returns a sequential {@link Stream} of the contents of {@code iterable}, delegating to {@link + * Collection#stream} if possible. + */ + public static Stream stream(Iterable iterable) { + return (iterable instanceof Collection) + ? ((Collection) iterable).stream() + : StreamSupport.stream(iterable.spliterator(), false); + } + + /** + * Returns {@link Collection#stream}. + * + * @deprecated There is no reason to use this; just invoke {@code collection.stream()} directly. + */ + @Deprecated + public static Stream stream(Collection collection) { + return collection.stream(); + } + + /** + * Returns a sequential {@link Stream} of the remaining contents of {@code iterator}. Do not use + * {@code iterator} directly after passing it to this method. + */ + public static Stream stream(Iterator iterator) { + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, 0), false); + } + + /** + * If a value is present in {@code optional}, returns a stream containing only that element, + * otherwise returns an empty stream. + */ + public static Stream stream(com.azure.cosmos.kafka.connect.implementation.guava25.base.Optional optional) { + return optional.isPresent() ? Stream.of(optional.get()) : Stream.of(); + } + + /** + * If a value is present in {@code optional}, returns a stream containing only that element, + * otherwise returns an empty stream. + * + *

Java 9 users: use {@code optional.stream()} instead. + */ + public static Stream stream(java.util.Optional optional) { + return optional.isPresent() ? Stream.of(optional.get()) : Stream.of(); + } + + /** + * If a value is present in {@code optional}, returns a stream containing only that element, + * otherwise returns an empty stream. + * + *

Java 9 users: use {@code optional.stream()} instead. + */ + public static IntStream stream(OptionalInt optional) { + return optional.isPresent() ? IntStream.of(optional.getAsInt()) : IntStream.empty(); + } + + /** + * If a value is present in {@code optional}, returns a stream containing only that element, + * otherwise returns an empty stream. + * + *

Java 9 users: use {@code optional.stream()} instead. + */ + public static LongStream stream(OptionalLong optional) { + return optional.isPresent() ? LongStream.of(optional.getAsLong()) : LongStream.empty(); + } + + /** + * If a value is present in {@code optional}, returns a stream containing only that element, + * otherwise returns an empty stream. + * + *

Java 9 users: use {@code optional.stream()} instead. + */ + public static DoubleStream stream(OptionalDouble optional) { + return optional.isPresent() ? DoubleStream.of(optional.getAsDouble()) : DoubleStream.empty(); + } + + /** + * Returns a {@link Stream} containing the elements of the first stream, followed by the elements + * of the second stream, and so on. + * + *

This is equivalent to {@code Stream.of(streams).flatMap(stream -> stream)}, but the returned + * stream may perform better. + * + * @see Stream#concat(Stream, Stream) + */ + @SuppressWarnings({"unchecked", "rawtypes", "varargs"}) + @SafeVarargs + public static Stream concat(Stream... streams) { + // TODO(lowasser): consider an implementation that can support SUBSIZED + boolean isParallel = false; + int characteristics = Spliterator.ORDERED | Spliterator.SIZED | Spliterator.NONNULL; + long estimatedSize = 0L; + ImmutableList.Builder> splitrsBuilder = + new ImmutableList.Builder<>(streams.length); + for (Stream stream : streams) { + isParallel |= stream.isParallel(); + Spliterator splitr = stream.spliterator(); + splitrsBuilder.add(splitr); + characteristics &= splitr.characteristics(); + estimatedSize = LongMath.saturatedAdd(estimatedSize, splitr.estimateSize()); + } + return StreamSupport.stream( + CollectSpliterators.flatMap( + splitrsBuilder.build().spliterator(), + splitr -> (Spliterator) splitr, + characteristics, + estimatedSize), + isParallel) + .onClose( + () -> { + for (Stream stream : streams) { + stream.close(); + } + }); + } + + /** + * Returns an {@link IntStream} containing the elements of the first stream, followed by the + * elements of the second stream, and so on. + * + *

This is equivalent to {@code Stream.of(streams).flatMapToInt(stream -> stream)}, but the + * returned stream may perform better. + * + * @see IntStream#concat(IntStream, IntStream) + */ + public static IntStream concat(IntStream... streams) { + // TODO(lowasser): optimize this later + return Stream.of(streams).flatMapToInt(stream -> stream); + } + + /** + * Returns a {@link LongStream} containing the elements of the first stream, followed by the + * elements of the second stream, and so on. + * + *

This is equivalent to {@code Stream.of(streams).flatMapToLong(stream -> stream)}, but the + * returned stream may perform better. + * + * @see LongStream#concat(LongStream, LongStream) + */ + public static LongStream concat(LongStream... streams) { + // TODO(lowasser): optimize this later + return Stream.of(streams).flatMapToLong(stream -> stream); + } + + /** + * Returns a {@link DoubleStream} containing the elements of the first stream, followed by the + * elements of the second stream, and so on. + * + *

This is equivalent to {@code Stream.of(streams).flatMapToDouble(stream -> stream)}, but the + * returned stream may perform better. + * + * @see DoubleStream#concat(DoubleStream, DoubleStream) + */ + public static DoubleStream concat(DoubleStream... streams) { + // TODO(lowasser): optimize this later + return Stream.of(streams).flatMapToDouble(stream -> stream); + } + + /** + * Returns a stream in which each element is the result of passing the corresponding elementY of + * each of {@code streamA} and {@code streamB} to {@code function}. + * + *

For example: + * + *

{@code
+   * Streams.zip(
+   *   Stream.of("foo1", "foo2", "foo3"),
+   *   Stream.of("bar1", "bar2"),
+   *   (arg1, arg2) -> arg1 + ":" + arg2)
+   * }
+ * + *

will return {@code Stream.of("foo1:bar1", "foo2:bar2")}. + * + *

The resulting stream will only be as long as the shorter of the two input streams; if one + * stream is longer, its extra elements will be ignored. + * + *

Note that if you are calling {@link Stream#forEach} on the resulting stream, you might want + * to consider using {@link #forEachPair} instead of this method. + * + *

Performance note: The resulting stream is not efficiently splittable. + * This may harm parallel performance. + */ + public static Stream zip( + Stream streamA, Stream streamB, BiFunction function) { + checkNotNull(streamA); + checkNotNull(streamB); + checkNotNull(function); + boolean isParallel = streamA.isParallel() || streamB.isParallel(); // same as Stream.concat + Spliterator splitrA = streamA.spliterator(); + Spliterator splitrB = streamB.spliterator(); + int characteristics = + splitrA.characteristics() + & splitrB.characteristics() + & (Spliterator.SIZED | Spliterator.ORDERED); + Iterator itrA = Spliterators.iterator(splitrA); + Iterator itrB = Spliterators.iterator(splitrB); + return StreamSupport.stream( + new AbstractSpliterator( + Math.min(splitrA.estimateSize(), splitrB.estimateSize()), characteristics) { + @Override + public boolean tryAdvance(Consumer action) { + if (itrA.hasNext() && itrB.hasNext()) { + action.accept(function.apply(itrA.next(), itrB.next())); + return true; + } + return false; + } + }, + isParallel) + .onClose(streamA::close) + .onClose(streamB::close); + } + + /** + * Invokes {@code consumer} once for each pair of corresponding elements in {@code streamA} + * and {@code streamB}. If one stream is longer than the other, the extra elements are silently + * ignored. Elements passed to the consumer are guaranteed to come from the same position in their + * respective source streams. For example: + * + *

{@code
+   * Streams.forEachPair(
+   *   Stream.of("foo1", "foo2", "foo3"),
+   *   Stream.of("bar1", "bar2"),
+   *   (arg1, arg2) -> System.out.println(arg1 + ":" + arg2)
+   * }
+ * + *

will print: + * + *

{@code
+   * foo1:bar1
+   * foo2:bar2
+   * }
+ * + *

Warning: If either supplied stream is a parallel stream, the same correspondence + * between elements will be made, but the order in which those pairs of elements are passed to the + * consumer is not defined. + * + *

Note that many usages of this method can be replaced with simpler calls to {@link #zip}. + * This method behaves equivalently to {@linkplain #zip zipping} the stream elements into + * temporary pair objects and then using {@link Stream#forEach} on that stream. + * + * @since 22.0 + */ + public static void forEachPair( + Stream streamA, Stream streamB, BiConsumer consumer) { + checkNotNull(consumer); + + if (streamA.isParallel() || streamB.isParallel()) { + zip(streamA, streamB, TemporaryPair::new).forEach(pair -> consumer.accept(pair.a, pair.b)); + } else { + Iterator iterA = streamA.iterator(); + Iterator iterB = streamB.iterator(); + while (iterA.hasNext() && iterB.hasNext()) { + consumer.accept(iterA.next(), iterB.next()); + } + } + } + + // Use this carefully - it doesn't implement value semantics + private static class TemporaryPair { + final A a; + final B b; + + TemporaryPair(A a, B b) { + this.a = a; + this.b = b; + } + } + + /** + * Returns a stream consisting of the results of applying the given function to the elements of + * {@code stream} and their indices in the stream. For example, + * + *

{@code
+   * mapWithIndex(
+   *     Stream.of("a", "b", "c"),
+   *     (str, index) -> str + ":" + index)
+   * }
+ * + *

would return {@code Stream.of("a:0", "b:1", "c:2")}. + * + *

The resulting stream is efficiently splittable + * if and only if {@code stream} was efficiently splittable and its underlying spliterator + * reported {@link Spliterator#SUBSIZED}. This is generally the case if the underlying stream + * comes from a data structure supporting efficient indexed random access, typically an array or + * list. + * + *

The order of the resulting stream is defined if and only if the order of the original stream + * was defined. + */ + public static Stream mapWithIndex( + Stream stream, FunctionWithIndex function) { + checkNotNull(stream); + checkNotNull(function); + boolean isParallel = stream.isParallel(); + Spliterator fromSpliterator = stream.spliterator(); + + if (!fromSpliterator.hasCharacteristics(Spliterator.SUBSIZED)) { + Iterator fromIterator = Spliterators.iterator(fromSpliterator); + return StreamSupport.stream( + new AbstractSpliterator( + fromSpliterator.estimateSize(), + fromSpliterator.characteristics() & (Spliterator.ORDERED | Spliterator.SIZED)) { + long index = 0; + + @Override + public boolean tryAdvance(Consumer action) { + if (fromIterator.hasNext()) { + action.accept(function.apply(fromIterator.next(), index++)); + return true; + } + return false; + } + }, + isParallel) + .onClose(stream::close); + } + class Splitr extends MapWithIndexSpliterator, R, Splitr> implements Consumer { + T holder; + + Splitr(Spliterator splitr, long index) { + super(splitr, index); + } + + @Override + public void accept(T t) { + this.holder = t; + } + + @Override + public boolean tryAdvance(Consumer action) { + if (fromSpliterator.tryAdvance(this)) { + try { + action.accept(function.apply(holder, index++)); + return true; + } finally { + holder = null; + } + } + return false; + } + + @Override + Splitr createSplit(Spliterator from, long i) { + return new Splitr(from, i); + } + } + return StreamSupport.stream(new Splitr(fromSpliterator, 0), isParallel).onClose(stream::close); + } + + /** + * Returns a stream consisting of the results of applying the given function to the elements of + * {@code stream} and their indexes in the stream. For example, + * + *

{@code
+   * mapWithIndex(
+   *     IntStream.of(0, 1, 2),
+   *     (i, index) -> i + ":" + index)
+   * }
+ * + *

...would return {@code Stream.of("0:0", "1:1", "2:2")}. + * + *

The resulting stream is efficiently splittable + * if and only if {@code stream} was efficiently splittable and its underlying spliterator + * reported {@link Spliterator#SUBSIZED}. This is generally the case if the underlying stream + * comes from a data structure supporting efficient indexed random access, typically an array or + * list. + * + *

The order of the resulting stream is defined if and only if the order of the original stream + * was defined. + */ + public static Stream mapWithIndex(IntStream stream, IntFunctionWithIndex function) { + checkNotNull(stream); + checkNotNull(function); + boolean isParallel = stream.isParallel(); + Spliterator.OfInt fromSpliterator = stream.spliterator(); + + if (!fromSpliterator.hasCharacteristics(Spliterator.SUBSIZED)) { + PrimitiveIterator.OfInt fromIterator = Spliterators.iterator(fromSpliterator); + return StreamSupport.stream( + new AbstractSpliterator( + fromSpliterator.estimateSize(), + fromSpliterator.characteristics() & (Spliterator.ORDERED | Spliterator.SIZED)) { + long index = 0; + + @Override + public boolean tryAdvance(Consumer action) { + if (fromIterator.hasNext()) { + action.accept(function.apply(fromIterator.nextInt(), index++)); + return true; + } + return false; + } + }, + isParallel) + .onClose(stream::close); + } + class Splitr extends MapWithIndexSpliterator + implements IntConsumer, Spliterator { + int holder; + + Splitr(OfInt splitr, long index) { + super(splitr, index); + } + + @Override + public void accept(int t) { + this.holder = t; + } + + @Override + public boolean tryAdvance(Consumer action) { + if (fromSpliterator.tryAdvance(this)) { + action.accept(function.apply(holder, index++)); + return true; + } + return false; + } + + @Override + Splitr createSplit(OfInt from, long i) { + return new Splitr(from, i); + } + } + return StreamSupport.stream(new Splitr(fromSpliterator, 0), isParallel).onClose(stream::close); + } + + /** + * Returns a stream consisting of the results of applying the given function to the elements of + * {@code stream} and their indexes in the stream. For example, + * + *

{@code
+   * mapWithIndex(
+   *     LongStream.of(0, 1, 2),
+   *     (i, index) -> i + ":" + index)
+   * }
+ * + *

...would return {@code Stream.of("0:0", "1:1", "2:2")}. + * + *

The resulting stream is efficiently splittable + * if and only if {@code stream} was efficiently splittable and its underlying spliterator + * reported {@link Spliterator#SUBSIZED}. This is generally the case if the underlying stream + * comes from a data structure supporting efficient indexed random access, typically an array or + * list. + * + *

The order of the resulting stream is defined if and only if the order of the original stream + * was defined. + */ + public static Stream mapWithIndex(LongStream stream, LongFunctionWithIndex function) { + checkNotNull(stream); + checkNotNull(function); + boolean isParallel = stream.isParallel(); + Spliterator.OfLong fromSpliterator = stream.spliterator(); + + if (!fromSpliterator.hasCharacteristics(Spliterator.SUBSIZED)) { + PrimitiveIterator.OfLong fromIterator = Spliterators.iterator(fromSpliterator); + return StreamSupport.stream( + new AbstractSpliterator( + fromSpliterator.estimateSize(), + fromSpliterator.characteristics() & (Spliterator.ORDERED | Spliterator.SIZED)) { + long index = 0; + + @Override + public boolean tryAdvance(Consumer action) { + if (fromIterator.hasNext()) { + action.accept(function.apply(fromIterator.nextLong(), index++)); + return true; + } + return false; + } + }, + isParallel) + .onClose(stream::close); + } + class Splitr extends MapWithIndexSpliterator + implements LongConsumer, Spliterator { + long holder; + + Splitr(OfLong splitr, long index) { + super(splitr, index); + } + + @Override + public void accept(long t) { + this.holder = t; + } + + @Override + public boolean tryAdvance(Consumer action) { + if (fromSpliterator.tryAdvance(this)) { + action.accept(function.apply(holder, index++)); + return true; + } + return false; + } + + @Override + Splitr createSplit(OfLong from, long i) { + return new Splitr(from, i); + } + } + return StreamSupport.stream(new Splitr(fromSpliterator, 0), isParallel).onClose(stream::close); + } + + /** + * Returns a stream consisting of the results of applying the given function to the elements of + * {@code stream} and their indexes in the stream. For example, + * + *

{@code
+   * mapWithIndex(
+   *     DoubleStream.of(0, 1, 2),
+   *     (x, index) -> x + ":" + index)
+   * }
+ * + *

...would return {@code Stream.of("0.0:0", "1.0:1", "2.0:2")}. + * + *

The resulting stream is efficiently splittable + * if and only if {@code stream} was efficiently splittable and its underlying spliterator + * reported {@link Spliterator#SUBSIZED}. This is generally the case if the underlying stream + * comes from a data structure supporting efficient indexed random access, typically an array or + * list. + * + *

The order of the resulting stream is defined if and only if the order of the original stream + * was defined. + */ + public static Stream mapWithIndex( + DoubleStream stream, DoubleFunctionWithIndex function) { + checkNotNull(stream); + checkNotNull(function); + boolean isParallel = stream.isParallel(); + Spliterator.OfDouble fromSpliterator = stream.spliterator(); + + if (!fromSpliterator.hasCharacteristics(Spliterator.SUBSIZED)) { + PrimitiveIterator.OfDouble fromIterator = Spliterators.iterator(fromSpliterator); + return StreamSupport.stream( + new AbstractSpliterator( + fromSpliterator.estimateSize(), + fromSpliterator.characteristics() & (Spliterator.ORDERED | Spliterator.SIZED)) { + long index = 0; + + @Override + public boolean tryAdvance(Consumer action) { + if (fromIterator.hasNext()) { + action.accept(function.apply(fromIterator.nextDouble(), index++)); + return true; + } + return false; + } + }, + isParallel) + .onClose(stream::close); + } + class Splitr extends MapWithIndexSpliterator + implements DoubleConsumer, Spliterator { + double holder; + + Splitr(OfDouble splitr, long index) { + super(splitr, index); + } + + @Override + public void accept(double t) { + this.holder = t; + } + + @Override + public boolean tryAdvance(Consumer action) { + if (fromSpliterator.tryAdvance(this)) { + action.accept(function.apply(holder, index++)); + return true; + } + return false; + } + + @Override + Splitr createSplit(OfDouble from, long i) { + return new Splitr(from, i); + } + } + return StreamSupport.stream(new Splitr(fromSpliterator, 0), isParallel).onClose(stream::close); + } + + /** + * An analogue of {@link java.util.function.Function} also accepting an index. + * + *

This interface is only intended for use by callers of {@link #mapWithIndex(Stream, + * FunctionWithIndex)}. + * + * @since 21.0 + */ + + public interface FunctionWithIndex { + /** Applies this function to the given argument and its index within a stream. */ + R apply(T from, long index); + } + + private abstract static class MapWithIndexSpliterator< + F extends Spliterator, R, S extends MapWithIndexSpliterator> + implements Spliterator { + final F fromSpliterator; + long index; + + MapWithIndexSpliterator(F fromSpliterator, long index) { + this.fromSpliterator = fromSpliterator; + this.index = index; + } + + abstract S createSplit(F from, long i); + + @Override + public S trySplit() { + @SuppressWarnings({"unchecked", "rawtypes"}) + F split = (F) fromSpliterator.trySplit(); + if (split == null) { + return null; + } + S result = createSplit(split, index); + this.index += split.getExactSizeIfKnown(); + return result; + } + + @Override + public long estimateSize() { + return fromSpliterator.estimateSize(); + } + + @Override + public int characteristics() { + return fromSpliterator.characteristics() + & (Spliterator.ORDERED | Spliterator.SIZED | Spliterator.SUBSIZED); + } + } + + /** + * An analogue of {@link java.util.function.IntFunction} also accepting an index. + * + *

This interface is only intended for use by callers of {@link #mapWithIndex(IntStream, + * IntFunctionWithIndex)}. + * + * @since 21.0 + */ + + public interface IntFunctionWithIndex { + /** Applies this function to the given argument and its index within a stream. */ + R apply(int from, long index); + } + + /** + * An analogue of {@link java.util.function.LongFunction} also accepting an index. + * + *

This interface is only intended for use by callers of {@link #mapWithIndex(LongStream, + * LongFunctionWithIndex)}. + * + * @since 21.0 + */ + + public interface LongFunctionWithIndex { + /** Applies this function to the given argument and its index within a stream. */ + R apply(long from, long index); + } + + /** + * An analogue of {@link java.util.function.DoubleFunction} also accepting an index. + * + *

This interface is only intended for use by callers of {@link #mapWithIndex(DoubleStream, + * DoubleFunctionWithIndex)}. + * + * @since 21.0 + */ + + public interface DoubleFunctionWithIndex { + /** Applies this function to the given argument and its index within a stream. */ + R apply(double from, long index); + } + + /** + * Returns the last element of the specified stream, or {@link java.util.Optional#empty} if the + * stream is empty. + * + *

Equivalent to {@code stream.reduce((a, b) -> b)}, but may perform significantly better. This + * method's runtime will be between O(log n) and O(n), performing better on efficiently splittable + * streams. + * + *

If the stream has nondeterministic order, this has equivalent semantics to {@link + * Stream#findAny} (which you might as well use). + * + * @see Stream#findFirst() + * @throws NullPointerException if the last element of the stream is null + */ + public static java.util.Optional findLast(Stream stream) { + class OptionalState { + boolean set = false; + T value = null; + + void set(T value) { + this.set = true; + this.value = value; + } + + T get() { + checkState(set); + return value; + } + } + OptionalState state = new OptionalState(); + + Deque> splits = new ArrayDeque<>(); + splits.addLast(stream.spliterator()); + + while (!splits.isEmpty()) { + Spliterator spliterator = splits.removeLast(); + + if (spliterator.getExactSizeIfKnown() == 0) { + continue; // drop this split + } + + // Many spliterators will have trySplits that are SUBSIZED even if they are not themselves + // SUBSIZED. + if (spliterator.hasCharacteristics(Spliterator.SUBSIZED)) { + // we can drill down to exactly the smallest nonempty spliterator + while (true) { + Spliterator prefix = spliterator.trySplit(); + if (prefix == null || prefix.getExactSizeIfKnown() == 0) { + break; + } else if (spliterator.getExactSizeIfKnown() == 0) { + spliterator = prefix; + break; + } + } + + // spliterator is known to be nonempty now + spliterator.forEachRemaining(state::set); + return java.util.Optional.of(state.get()); + } + + Spliterator prefix = spliterator.trySplit(); + if (prefix == null || prefix.getExactSizeIfKnown() == 0) { + // we can't split this any further + spliterator.forEachRemaining(state::set); + if (state.set) { + return java.util.Optional.of(state.get()); + } + // fall back to the last split + continue; + } + splits.addLast(prefix); + splits.addLast(spliterator); + } + return java.util.Optional.empty(); + } + + /** + * Returns the last element of the specified stream, or {@link OptionalInt#empty} if the stream is + * empty. + * + *

Equivalent to {@code stream.reduce((a, b) -> b)}, but may perform significantly better. This + * method's runtime will be between O(log n) and O(n), performing better on efficiently splittable + * streams. + * + * @see IntStream#findFirst() + * @throws NullPointerException if the last element of the stream is null + */ + public static OptionalInt findLast(IntStream stream) { + // findLast(Stream) does some allocation, so we might as well box some more + java.util.Optional boxedLast = findLast(stream.boxed()); + return boxedLast.isPresent() ? OptionalInt.of(boxedLast.get()) : OptionalInt.empty(); + } + + /** + * Returns the last element of the specified stream, or {@link OptionalLong#empty} if the stream + * is empty. + * + *

Equivalent to {@code stream.reduce((a, b) -> b)}, but may perform significantly better. This + * method's runtime will be between O(log n) and O(n), performing better on efficiently splittable + * streams. + * + * @see LongStream#findFirst() + * @throws NullPointerException if the last element of the stream is null + */ + public static OptionalLong findLast(LongStream stream) { + // findLast(Stream) does some allocation, so we might as well box some more + java.util.Optional boxedLast = findLast(stream.boxed()); + return boxedLast.isPresent() ? OptionalLong.of(boxedLast.get()) : OptionalLong.empty(); + } + + /** + * Returns the last element of the specified stream, or {@link OptionalDouble#empty} if the stream + * is empty. + * + *

Equivalent to {@code stream.reduce((a, b) -> b)}, but may perform significantly better. This + * method's runtime will be between O(log n) and O(n), performing better on efficiently splittable + * streams. + * + * @see DoubleStream#findFirst() + * @throws NullPointerException if the last element of the stream is null + */ + public static OptionalDouble findLast(DoubleStream stream) { + // findLast(Stream) does some allocation, so we might as well box some more + java.util.Optional boxedLast = findLast(stream.boxed()); + return boxedLast.isPresent() ? OptionalDouble.of(boxedLast.get()) : OptionalDouble.empty(); + } + + private Streams() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Synchronized.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Synchronized.java new file mode 100644 index 0000000000000..e547d8f9d659d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Synchronized.java @@ -0,0 +1,2153 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.RandomAccess; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.Spliterator; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.UnaryOperator; +import java.util.stream.Stream; + + + +/** + * Synchronized collection views. The returned synchronized collection views are serializable if the + * backing collection and the mutex are serializable. + * + *

If {@code null} is passed as the {@code mutex} parameter to any of this class's top-level + * methods or inner class constructors, the created object uses itself as the synchronization mutex. + * + *

This class should be used by other collection classes only. + * + * @author Mike Bostock + * @author Jared Levy + */ +final class Synchronized { + private Synchronized() {} + + static class SynchronizedObject implements Serializable { + final Object delegate; + final Object mutex; + + SynchronizedObject(Object delegate, Object mutex) { + this.delegate = checkNotNull(delegate); + this.mutex = (mutex == null) ? this : mutex; + } + + Object delegate() { + return delegate; + } + + // No equals and hashCode; see ForwardingObject for details. + + @Override + public String toString() { + synchronized (mutex) { + return delegate.toString(); + } + } + + // Serialization invokes writeObject only when it's private. + // The SynchronizedObject subclasses don't need a writeObject method since + // they don't contain any non-transient member variables, while the + // following writeObject() handles the SynchronizedObject members. + + private void writeObject(ObjectOutputStream stream) throws IOException { + synchronized (mutex) { + stream.defaultWriteObject(); + } + } + + private static final long serialVersionUID = 0; + } + + private static Collection collection( + Collection collection, Object mutex) { + return new SynchronizedCollection(collection, mutex); + } + + + static class SynchronizedCollection extends SynchronizedObject implements Collection { + private SynchronizedCollection(Collection delegate, Object mutex) { + super(delegate, mutex); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + Collection delegate() { + return (Collection) super.delegate(); + } + + @Override + public boolean add(E e) { + synchronized (mutex) { + return delegate().add(e); + } + } + + @Override + public boolean addAll(Collection c) { + synchronized (mutex) { + return delegate().addAll(c); + } + } + + @Override + public void clear() { + synchronized (mutex) { + delegate().clear(); + } + } + + @Override + public boolean contains(Object o) { + synchronized (mutex) { + return delegate().contains(o); + } + } + + @Override + public boolean containsAll(Collection c) { + synchronized (mutex) { + return delegate().containsAll(c); + } + } + + @Override + public boolean isEmpty() { + synchronized (mutex) { + return delegate().isEmpty(); + } + } + + @Override + public Iterator iterator() { + return delegate().iterator(); // manually synchronized + } + + @Override + public Spliterator spliterator() { + synchronized (mutex) { + return delegate().spliterator(); + } + } + + @Override + public Stream stream() { + synchronized (mutex) { + return delegate().stream(); + } + } + + @Override + public Stream parallelStream() { + synchronized (mutex) { + return delegate().parallelStream(); + } + } + + @Override + public void forEach(Consumer action) { + synchronized (mutex) { + delegate().forEach(action); + } + } + + @Override + public boolean remove(Object o) { + synchronized (mutex) { + return delegate().remove(o); + } + } + + @Override + public boolean removeAll(Collection c) { + synchronized (mutex) { + return delegate().removeAll(c); + } + } + + @Override + public boolean retainAll(Collection c) { + synchronized (mutex) { + return delegate().retainAll(c); + } + } + + @Override + public boolean removeIf(Predicate filter) { + synchronized (mutex) { + return delegate().removeIf(filter); + } + } + + @Override + public int size() { + synchronized (mutex) { + return delegate().size(); + } + } + + @Override + public Object[] toArray() { + synchronized (mutex) { + return delegate().toArray(); + } + } + + @Override + public T[] toArray(T[] a) { + synchronized (mutex) { + return delegate().toArray(a); + } + } + + private static final long serialVersionUID = 0; + } + + + static Set set(Set set, Object mutex) { + return new SynchronizedSet(set, mutex); + } + + static class SynchronizedSet extends SynchronizedCollection implements Set { + + SynchronizedSet(Set delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + Set delegate() { + return (Set) super.delegate(); + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + synchronized (mutex) { + return delegate().equals(o); + } + } + + @Override + public int hashCode() { + synchronized (mutex) { + return delegate().hashCode(); + } + } + + private static final long serialVersionUID = 0; + } + + private static SortedSet sortedSet(SortedSet set, Object mutex) { + return new SynchronizedSortedSet(set, mutex); + } + + static class SynchronizedSortedSet extends SynchronizedSet implements SortedSet { + SynchronizedSortedSet(SortedSet delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + SortedSet delegate() { + return (SortedSet) super.delegate(); + } + + @Override + public Comparator comparator() { + synchronized (mutex) { + return delegate().comparator(); + } + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + synchronized (mutex) { + return sortedSet(delegate().subSet(fromElement, toElement), mutex); + } + } + + @Override + public SortedSet headSet(E toElement) { + synchronized (mutex) { + return sortedSet(delegate().headSet(toElement), mutex); + } + } + + @Override + public SortedSet tailSet(E fromElement) { + synchronized (mutex) { + return sortedSet(delegate().tailSet(fromElement), mutex); + } + } + + @Override + public E first() { + synchronized (mutex) { + return delegate().first(); + } + } + + @Override + public E last() { + synchronized (mutex) { + return delegate().last(); + } + } + + private static final long serialVersionUID = 0; + } + + private static List list(List list, Object mutex) { + return (list instanceof RandomAccess) + ? new SynchronizedRandomAccessList(list, mutex) + : new SynchronizedList(list, mutex); + } + + private static class SynchronizedList extends SynchronizedCollection implements List { + SynchronizedList(List delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + List delegate() { + return (List) super.delegate(); + } + + @Override + public void add(int index, E element) { + synchronized (mutex) { + delegate().add(index, element); + } + } + + @Override + public boolean addAll(int index, Collection c) { + synchronized (mutex) { + return delegate().addAll(index, c); + } + } + + @Override + public E get(int index) { + synchronized (mutex) { + return delegate().get(index); + } + } + + @Override + public int indexOf(Object o) { + synchronized (mutex) { + return delegate().indexOf(o); + } + } + + @Override + public int lastIndexOf(Object o) { + synchronized (mutex) { + return delegate().lastIndexOf(o); + } + } + + @Override + public ListIterator listIterator() { + return delegate().listIterator(); // manually synchronized + } + + @Override + public ListIterator listIterator(int index) { + return delegate().listIterator(index); // manually synchronized + } + + @Override + public E remove(int index) { + synchronized (mutex) { + return delegate().remove(index); + } + } + + @Override + public E set(int index, E element) { + synchronized (mutex) { + return delegate().set(index, element); + } + } + + @Override + public void replaceAll(UnaryOperator operator) { + synchronized (mutex) { + delegate().replaceAll(operator); + } + } + + @Override + public void sort(Comparator c) { + synchronized (mutex) { + delegate().sort(c); + } + } + + @Override + public List subList(int fromIndex, int toIndex) { + synchronized (mutex) { + return list(delegate().subList(fromIndex, toIndex), mutex); + } + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + synchronized (mutex) { + return delegate().equals(o); + } + } + + @Override + public int hashCode() { + synchronized (mutex) { + return delegate().hashCode(); + } + } + + private static final long serialVersionUID = 0; + } + + private static class SynchronizedRandomAccessList extends SynchronizedList + implements RandomAccess { + SynchronizedRandomAccessList(List list, Object mutex) { + super(list, mutex); + } + + private static final long serialVersionUID = 0; + } + + static Multiset multiset(Multiset multiset, Object mutex) { + if (multiset instanceof SynchronizedMultiset || multiset instanceof ImmutableMultiset) { + return multiset; + } + return new SynchronizedMultiset(multiset, mutex); + } + + private static class SynchronizedMultiset extends SynchronizedCollection + implements Multiset { + transient Set elementSet; + transient Set> entrySet; + + SynchronizedMultiset(Multiset delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + Multiset delegate() { + return (Multiset) super.delegate(); + } + + @Override + public int count(Object o) { + synchronized (mutex) { + return delegate().count(o); + } + } + + @Override + public int add(E e, int n) { + synchronized (mutex) { + return delegate().add(e, n); + } + } + + @Override + public int remove(Object o, int n) { + synchronized (mutex) { + return delegate().remove(o, n); + } + } + + @Override + public int setCount(E element, int count) { + synchronized (mutex) { + return delegate().setCount(element, count); + } + } + + @Override + public boolean setCount(E element, int oldCount, int newCount) { + synchronized (mutex) { + return delegate().setCount(element, oldCount, newCount); + } + } + + @Override + public Set elementSet() { + synchronized (mutex) { + if (elementSet == null) { + elementSet = typePreservingSet(delegate().elementSet(), mutex); + } + return elementSet; + } + } + + @Override + public Set> entrySet() { + synchronized (mutex) { + if (entrySet == null) { + entrySet = typePreservingSet(delegate().entrySet(), mutex); + } + return entrySet; + } + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + synchronized (mutex) { + return delegate().equals(o); + } + } + + @Override + public int hashCode() { + synchronized (mutex) { + return delegate().hashCode(); + } + } + + private static final long serialVersionUID = 0; + } + + static Multimap multimap(Multimap multimap, Object mutex) { + if (multimap instanceof SynchronizedMultimap || multimap instanceof ImmutableMultimap) { + return multimap; + } + return new SynchronizedMultimap<>(multimap, mutex); + } + + private static class SynchronizedMultimap extends SynchronizedObject + implements Multimap { + transient Set keySet; + transient Collection valuesCollection; + transient Collection> entries; + transient Map> asMap; + transient Multiset keys; + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + Multimap delegate() { + return (Multimap) super.delegate(); + } + + SynchronizedMultimap(Multimap delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + public int size() { + synchronized (mutex) { + return delegate().size(); + } + } + + @Override + public boolean isEmpty() { + synchronized (mutex) { + return delegate().isEmpty(); + } + } + + @Override + public boolean containsKey(Object key) { + synchronized (mutex) { + return delegate().containsKey(key); + } + } + + @Override + public boolean containsValue(Object value) { + synchronized (mutex) { + return delegate().containsValue(value); + } + } + + @Override + public boolean containsEntry(Object key, Object value) { + synchronized (mutex) { + return delegate().containsEntry(key, value); + } + } + + @Override + public Collection get(K key) { + synchronized (mutex) { + return typePreservingCollection(delegate().get(key), mutex); + } + } + + @Override + public boolean put(K key, V value) { + synchronized (mutex) { + return delegate().put(key, value); + } + } + + @Override + public boolean putAll(K key, Iterable values) { + synchronized (mutex) { + return delegate().putAll(key, values); + } + } + + @Override + public boolean putAll(Multimap multimap) { + synchronized (mutex) { + return delegate().putAll(multimap); + } + } + + @Override + public Collection replaceValues(K key, Iterable values) { + synchronized (mutex) { + return delegate().replaceValues(key, values); // copy not synchronized + } + } + + @Override + public boolean remove(Object key, Object value) { + synchronized (mutex) { + return delegate().remove(key, value); + } + } + + @Override + public Collection removeAll(Object key) { + synchronized (mutex) { + return delegate().removeAll(key); // copy not synchronized + } + } + + @Override + public void clear() { + synchronized (mutex) { + delegate().clear(); + } + } + + @Override + public Set keySet() { + synchronized (mutex) { + if (keySet == null) { + keySet = typePreservingSet(delegate().keySet(), mutex); + } + return keySet; + } + } + + @Override + public Collection values() { + synchronized (mutex) { + if (valuesCollection == null) { + valuesCollection = collection(delegate().values(), mutex); + } + return valuesCollection; + } + } + + @Override + public Collection> entries() { + synchronized (mutex) { + if (entries == null) { + entries = typePreservingCollection(delegate().entries(), mutex); + } + return entries; + } + } + + @Override + public void forEach(BiConsumer action) { + synchronized (mutex) { + delegate().forEach(action); + } + } + + @Override + public Map> asMap() { + synchronized (mutex) { + if (asMap == null) { + asMap = new SynchronizedAsMap<>(delegate().asMap(), mutex); + } + return asMap; + } + } + + @Override + public Multiset keys() { + synchronized (mutex) { + if (keys == null) { + keys = multiset(delegate().keys(), mutex); + } + return keys; + } + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + synchronized (mutex) { + return delegate().equals(o); + } + } + + @Override + public int hashCode() { + synchronized (mutex) { + return delegate().hashCode(); + } + } + + private static final long serialVersionUID = 0; + } + + static ListMultimap listMultimap( + ListMultimap multimap, Object mutex) { + if (multimap instanceof SynchronizedListMultimap || multimap instanceof ImmutableListMultimap) { + return multimap; + } + return new SynchronizedListMultimap<>(multimap, mutex); + } + + private static class SynchronizedListMultimap extends SynchronizedMultimap + implements ListMultimap { + SynchronizedListMultimap(ListMultimap delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + ListMultimap delegate() { + return (ListMultimap) super.delegate(); + } + + @Override + public List get(K key) { + synchronized (mutex) { + return list(delegate().get(key), mutex); + } + } + + @Override + public List removeAll(Object key) { + synchronized (mutex) { + return delegate().removeAll(key); // copy not synchronized + } + } + + @Override + public List replaceValues(K key, Iterable values) { + synchronized (mutex) { + return delegate().replaceValues(key, values); // copy not synchronized + } + } + + private static final long serialVersionUID = 0; + } + + static SetMultimap setMultimap( + SetMultimap multimap, Object mutex) { + if (multimap instanceof SynchronizedSetMultimap || multimap instanceof ImmutableSetMultimap) { + return multimap; + } + return new SynchronizedSetMultimap<>(multimap, mutex); + } + + private static class SynchronizedSetMultimap extends SynchronizedMultimap + implements SetMultimap { + transient Set> entrySet; + + SynchronizedSetMultimap(SetMultimap delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + SetMultimap delegate() { + return (SetMultimap) super.delegate(); + } + + @Override + public Set get(K key) { + synchronized (mutex) { + return set(delegate().get(key), mutex); + } + } + + @Override + public Set removeAll(Object key) { + synchronized (mutex) { + return delegate().removeAll(key); // copy not synchronized + } + } + + @Override + public Set replaceValues(K key, Iterable values) { + synchronized (mutex) { + return delegate().replaceValues(key, values); // copy not synchronized + } + } + + @Override + public Set> entries() { + synchronized (mutex) { + if (entrySet == null) { + entrySet = set(delegate().entries(), mutex); + } + return entrySet; + } + } + + private static final long serialVersionUID = 0; + } + + static SortedSetMultimap sortedSetMultimap( + SortedSetMultimap multimap, Object mutex) { + if (multimap instanceof SynchronizedSortedSetMultimap) { + return multimap; + } + return new SynchronizedSortedSetMultimap<>(multimap, mutex); + } + + private static class SynchronizedSortedSetMultimap extends SynchronizedSetMultimap + implements SortedSetMultimap { + SynchronizedSortedSetMultimap(SortedSetMultimap delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + SortedSetMultimap delegate() { + return (SortedSetMultimap) super.delegate(); + } + + @Override + public SortedSet get(K key) { + synchronized (mutex) { + return sortedSet(delegate().get(key), mutex); + } + } + + @Override + public SortedSet removeAll(Object key) { + synchronized (mutex) { + return delegate().removeAll(key); // copy not synchronized + } + } + + @Override + public SortedSet replaceValues(K key, Iterable values) { + synchronized (mutex) { + return delegate().replaceValues(key, values); // copy not synchronized + } + } + + @Override + public Comparator valueComparator() { + synchronized (mutex) { + return delegate().valueComparator(); + } + } + + private static final long serialVersionUID = 0; + } + + private static Collection typePreservingCollection( + Collection collection, Object mutex) { + if (collection instanceof SortedSet) { + return sortedSet((SortedSet) collection, mutex); + } + if (collection instanceof Set) { + return set((Set) collection, mutex); + } + if (collection instanceof List) { + return list((List) collection, mutex); + } + return collection(collection, mutex); + } + + private static Set typePreservingSet(Set set, Object mutex) { + if (set instanceof SortedSet) { + return sortedSet((SortedSet) set, mutex); + } else { + return set(set, mutex); + } + } + + private static class SynchronizedAsMapEntries + extends SynchronizedSet>> { + SynchronizedAsMapEntries(Set>> delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + public Iterator>> iterator() { + // Must be manually synchronized. + return new TransformedIterator>, Entry>>( + super.iterator()) { + @Override + Entry> transform(final Entry> entry) { + return new ForwardingMapEntry>() { + @Override + protected Entry> delegate() { + return entry; + } + + @Override + public Collection getValue() { + return typePreservingCollection(entry.getValue(), mutex); + } + }; + } + }; + } + + // See Collections.CheckedMap.CheckedEntrySet for details on attacks. + + @Override + public Object[] toArray() { + synchronized (mutex) { + return ObjectArrays.toArrayImpl(delegate()); + } + } + + @Override + public T[] toArray(T[] array) { + synchronized (mutex) { + return ObjectArrays.toArrayImpl(delegate(), array); + } + } + + @Override + public boolean contains(Object o) { + synchronized (mutex) { + return Maps.containsEntryImpl(delegate(), o); + } + } + + @Override + public boolean containsAll(Collection c) { + synchronized (mutex) { + return Collections2.containsAllImpl(delegate(), c); + } + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + synchronized (mutex) { + return Sets.equalsImpl(delegate(), o); + } + } + + @Override + public boolean remove(Object o) { + synchronized (mutex) { + return Maps.removeEntryImpl(delegate(), o); + } + } + + @Override + public boolean removeAll(Collection c) { + synchronized (mutex) { + return Iterators.removeAll(delegate().iterator(), c); + } + } + + @Override + public boolean retainAll(Collection c) { + synchronized (mutex) { + return Iterators.retainAll(delegate().iterator(), c); + } + } + + private static final long serialVersionUID = 0; + } + + + static Map map(Map map, Object mutex) { + return new SynchronizedMap<>(map, mutex); + } + + private static class SynchronizedMap extends SynchronizedObject implements Map { + transient Set keySet; + transient Collection values; + transient Set> entrySet; + + SynchronizedMap(Map delegate, Object mutex) { + super(delegate, mutex); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + @Override + Map delegate() { + return (Map) super.delegate(); + } + + @Override + public void clear() { + synchronized (mutex) { + delegate().clear(); + } + } + + @Override + public boolean containsKey(Object key) { + synchronized (mutex) { + return delegate().containsKey(key); + } + } + + @Override + public boolean containsValue(Object value) { + synchronized (mutex) { + return delegate().containsValue(value); + } + } + + @Override + public Set> entrySet() { + synchronized (mutex) { + if (entrySet == null) { + entrySet = set(delegate().entrySet(), mutex); + } + return entrySet; + } + } + + @Override + public void forEach(BiConsumer action) { + synchronized (mutex) { + delegate().forEach(action); + } + } + + @Override + public V get(Object key) { + synchronized (mutex) { + return delegate().get(key); + } + } + + @Override + public V getOrDefault(Object key, V defaultValue) { + synchronized (mutex) { + return delegate().getOrDefault(key, defaultValue); + } + } + + @Override + public boolean isEmpty() { + synchronized (mutex) { + return delegate().isEmpty(); + } + } + + @Override + public Set keySet() { + synchronized (mutex) { + if (keySet == null) { + keySet = set(delegate().keySet(), mutex); + } + return keySet; + } + } + + @Override + public V put(K key, V value) { + synchronized (mutex) { + return delegate().put(key, value); + } + } + + @Override + public V putIfAbsent(K key, V value) { + synchronized (mutex) { + return delegate().putIfAbsent(key, value); + } + } + + @Override + public boolean replace(K key, V oldValue, V newValue) { + synchronized (mutex) { + return delegate().replace(key, oldValue, newValue); + } + } + + @Override + public V replace(K key, V value) { + synchronized (mutex) { + return delegate().replace(key, value); + } + } + + @Override + public V computeIfAbsent(K key, Function mappingFunction) { + synchronized (mutex) { + return delegate().computeIfAbsent(key, mappingFunction); + } + } + + @Override + public V computeIfPresent( + K key, BiFunction remappingFunction) { + synchronized (mutex) { + return delegate().computeIfPresent(key, remappingFunction); + } + } + + @Override + public V compute(K key, BiFunction remappingFunction) { + synchronized (mutex) { + return delegate().compute(key, remappingFunction); + } + } + + @Override + public V merge( + K key, V value, BiFunction remappingFunction) { + synchronized (mutex) { + return delegate().merge(key, value, remappingFunction); + } + } + + @Override + public void putAll(Map map) { + synchronized (mutex) { + delegate().putAll(map); + } + } + + @Override + public void replaceAll(BiFunction function) { + synchronized (mutex) { + delegate().replaceAll(function); + } + } + + @Override + public V remove(Object key) { + synchronized (mutex) { + return delegate().remove(key); + } + } + + @Override + public boolean remove(Object key, Object value) { + synchronized (mutex) { + return delegate().remove(key, value); + } + } + + @Override + public int size() { + synchronized (mutex) { + return delegate().size(); + } + } + + @Override + public Collection values() { + synchronized (mutex) { + if (values == null) { + values = collection(delegate().values(), mutex); + } + return values; + } + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + synchronized (mutex) { + return delegate().equals(o); + } + } + + @Override + public int hashCode() { + synchronized (mutex) { + return delegate().hashCode(); + } + } + + private static final long serialVersionUID = 0; + } + + static SortedMap sortedMap(SortedMap sortedMap, Object mutex) { + return new SynchronizedSortedMap<>(sortedMap, mutex); + } + + static class SynchronizedSortedMap extends SynchronizedMap + implements SortedMap { + + SynchronizedSortedMap(SortedMap delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + SortedMap delegate() { + return (SortedMap) super.delegate(); + } + + @Override + public Comparator comparator() { + synchronized (mutex) { + return delegate().comparator(); + } + } + + @Override + public K firstKey() { + synchronized (mutex) { + return delegate().firstKey(); + } + } + + @Override + public SortedMap headMap(K toKey) { + synchronized (mutex) { + return sortedMap(delegate().headMap(toKey), mutex); + } + } + + @Override + public K lastKey() { + synchronized (mutex) { + return delegate().lastKey(); + } + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + synchronized (mutex) { + return sortedMap(delegate().subMap(fromKey, toKey), mutex); + } + } + + @Override + public SortedMap tailMap(K fromKey) { + synchronized (mutex) { + return sortedMap(delegate().tailMap(fromKey), mutex); + } + } + + private static final long serialVersionUID = 0; + } + + static BiMap biMap(BiMap bimap, Object mutex) { + if (bimap instanceof SynchronizedBiMap || bimap instanceof ImmutableBiMap) { + return bimap; + } + return new SynchronizedBiMap<>(bimap, mutex, null); + } + + + static class SynchronizedBiMap extends SynchronizedMap + implements BiMap, Serializable { + private transient Set valueSet; + private transient BiMap inverse; + + private SynchronizedBiMap( + BiMap delegate, Object mutex, BiMap inverse) { + super(delegate, mutex); + this.inverse = inverse; + } + + @Override + BiMap delegate() { + return (BiMap) super.delegate(); + } + + @Override + public Set values() { + synchronized (mutex) { + if (valueSet == null) { + valueSet = set(delegate().values(), mutex); + } + return valueSet; + } + } + + @Override + public V forcePut(K key, V value) { + synchronized (mutex) { + return delegate().forcePut(key, value); + } + } + + @Override + public BiMap inverse() { + synchronized (mutex) { + if (inverse == null) { + inverse = new SynchronizedBiMap<>(delegate().inverse(), mutex, this); + } + return inverse; + } + } + + private static final long serialVersionUID = 0; + } + + private static class SynchronizedAsMap extends SynchronizedMap> { + transient Set>> asMapEntrySet; + transient Collection> asMapValues; + + SynchronizedAsMap(Map> delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + public Collection get(Object key) { + synchronized (mutex) { + Collection collection = super.get(key); + return (collection == null) ? null : typePreservingCollection(collection, mutex); + } + } + + @Override + public Set>> entrySet() { + synchronized (mutex) { + if (asMapEntrySet == null) { + asMapEntrySet = new SynchronizedAsMapEntries<>(delegate().entrySet(), mutex); + } + return asMapEntrySet; + } + } + + @Override + public Collection> values() { + synchronized (mutex) { + if (asMapValues == null) { + asMapValues = new SynchronizedAsMapValues(delegate().values(), mutex); + } + return asMapValues; + } + } + + @Override + public boolean containsValue(Object o) { + // values() and its contains() method are both synchronized. + return values().contains(o); + } + + private static final long serialVersionUID = 0; + } + + private static class SynchronizedAsMapValues extends SynchronizedCollection> { + SynchronizedAsMapValues(Collection> delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + public Iterator> iterator() { + // Must be manually synchronized. + return new TransformedIterator, Collection>(super.iterator()) { + @Override + Collection transform(Collection from) { + return typePreservingCollection(from, mutex); + } + }; + } + + private static final long serialVersionUID = 0; + } + + + static class SynchronizedNavigableSet extends SynchronizedSortedSet + implements NavigableSet { + SynchronizedNavigableSet(NavigableSet delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + NavigableSet delegate() { + return (NavigableSet) super.delegate(); + } + + @Override + public E ceiling(E e) { + synchronized (mutex) { + return delegate().ceiling(e); + } + } + + @Override + public Iterator descendingIterator() { + return delegate().descendingIterator(); // manually synchronized + } + + transient NavigableSet descendingSet; + + @Override + public NavigableSet descendingSet() { + synchronized (mutex) { + if (descendingSet == null) { + NavigableSet dS = Synchronized.navigableSet(delegate().descendingSet(), mutex); + descendingSet = dS; + return dS; + } + return descendingSet; + } + } + + @Override + public E floor(E e) { + synchronized (mutex) { + return delegate().floor(e); + } + } + + @Override + public NavigableSet headSet(E toElement, boolean inclusive) { + synchronized (mutex) { + return Synchronized.navigableSet(delegate().headSet(toElement, inclusive), mutex); + } + } + + @Override + public SortedSet headSet(E toElement) { + return headSet(toElement, false); + } + + @Override + public E higher(E e) { + synchronized (mutex) { + return delegate().higher(e); + } + } + + @Override + public E lower(E e) { + synchronized (mutex) { + return delegate().lower(e); + } + } + + @Override + public E pollFirst() { + synchronized (mutex) { + return delegate().pollFirst(); + } + } + + @Override + public E pollLast() { + synchronized (mutex) { + return delegate().pollLast(); + } + } + + @Override + public NavigableSet subSet( + E fromElement, boolean fromInclusive, E toElement, boolean toInclusive) { + synchronized (mutex) { + return Synchronized.navigableSet( + delegate().subSet(fromElement, fromInclusive, toElement, toInclusive), mutex); + } + } + + @Override + public SortedSet subSet(E fromElement, E toElement) { + return subSet(fromElement, true, toElement, false); + } + + @Override + public NavigableSet tailSet(E fromElement, boolean inclusive) { + synchronized (mutex) { + return Synchronized.navigableSet(delegate().tailSet(fromElement, inclusive), mutex); + } + } + + @Override + public SortedSet tailSet(E fromElement) { + return tailSet(fromElement, true); + } + + private static final long serialVersionUID = 0; + } + + static NavigableSet navigableSet( + NavigableSet navigableSet, Object mutex) { + return new SynchronizedNavigableSet(navigableSet, mutex); + } + + static NavigableSet navigableSet(NavigableSet navigableSet) { + return navigableSet(navigableSet, null); + } + + static NavigableMap navigableMap(NavigableMap navigableMap) { + return navigableMap(navigableMap, null); + } + + static NavigableMap navigableMap( + NavigableMap navigableMap, Object mutex) { + return new SynchronizedNavigableMap<>(navigableMap, mutex); + } + + + static class SynchronizedNavigableMap extends SynchronizedSortedMap + implements NavigableMap { + + SynchronizedNavigableMap(NavigableMap delegate, Object mutex) { + super(delegate, mutex); + } + + @Override + NavigableMap delegate() { + return (NavigableMap) super.delegate(); + } + + @Override + public Entry ceilingEntry(K key) { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().ceilingEntry(key), mutex); + } + } + + @Override + public K ceilingKey(K key) { + synchronized (mutex) { + return delegate().ceilingKey(key); + } + } + + transient NavigableSet descendingKeySet; + + @Override + public NavigableSet descendingKeySet() { + synchronized (mutex) { + if (descendingKeySet == null) { + return descendingKeySet = Synchronized.navigableSet(delegate().descendingKeySet(), mutex); + } + return descendingKeySet; + } + } + + transient NavigableMap descendingMap; + + @Override + public NavigableMap descendingMap() { + synchronized (mutex) { + if (descendingMap == null) { + return descendingMap = navigableMap(delegate().descendingMap(), mutex); + } + return descendingMap; + } + } + + @Override + public Entry firstEntry() { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().firstEntry(), mutex); + } + } + + @Override + public Entry floorEntry(K key) { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().floorEntry(key), mutex); + } + } + + @Override + public K floorKey(K key) { + synchronized (mutex) { + return delegate().floorKey(key); + } + } + + @Override + public NavigableMap headMap(K toKey, boolean inclusive) { + synchronized (mutex) { + return navigableMap(delegate().headMap(toKey, inclusive), mutex); + } + } + + @Override + public SortedMap headMap(K toKey) { + return headMap(toKey, false); + } + + @Override + public Entry higherEntry(K key) { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().higherEntry(key), mutex); + } + } + + @Override + public K higherKey(K key) { + synchronized (mutex) { + return delegate().higherKey(key); + } + } + + @Override + public Entry lastEntry() { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().lastEntry(), mutex); + } + } + + @Override + public Entry lowerEntry(K key) { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().lowerEntry(key), mutex); + } + } + + @Override + public K lowerKey(K key) { + synchronized (mutex) { + return delegate().lowerKey(key); + } + } + + @Override + public Set keySet() { + return navigableKeySet(); + } + + transient NavigableSet navigableKeySet; + + @Override + public NavigableSet navigableKeySet() { + synchronized (mutex) { + if (navigableKeySet == null) { + return navigableKeySet = Synchronized.navigableSet(delegate().navigableKeySet(), mutex); + } + return navigableKeySet; + } + } + + @Override + public Entry pollFirstEntry() { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().pollFirstEntry(), mutex); + } + } + + @Override + public Entry pollLastEntry() { + synchronized (mutex) { + return nullableSynchronizedEntry(delegate().pollLastEntry(), mutex); + } + } + + @Override + public NavigableMap subMap( + K fromKey, boolean fromInclusive, K toKey, boolean toInclusive) { + synchronized (mutex) { + return navigableMap(delegate().subMap(fromKey, fromInclusive, toKey, toInclusive), mutex); + } + } + + @Override + public SortedMap subMap(K fromKey, K toKey) { + return subMap(fromKey, true, toKey, false); + } + + @Override + public NavigableMap tailMap(K fromKey, boolean inclusive) { + synchronized (mutex) { + return navigableMap(delegate().tailMap(fromKey, inclusive), mutex); + } + } + + @Override + public SortedMap tailMap(K fromKey) { + return tailMap(fromKey, true); + } + + private static final long serialVersionUID = 0; + } + + private static Entry nullableSynchronizedEntry( + Entry entry, Object mutex) { + if (entry == null) { + return null; + } + return new SynchronizedEntry<>(entry, mutex); + } + + private static class SynchronizedEntry extends SynchronizedObject implements Entry { + + SynchronizedEntry(Entry delegate, Object mutex) { + super(delegate, mutex); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // guaranteed by the constructor + @Override + Entry delegate() { + return (Entry) super.delegate(); + } + + @Override + public boolean equals(Object obj) { + synchronized (mutex) { + return delegate().equals(obj); + } + } + + @Override + public int hashCode() { + synchronized (mutex) { + return delegate().hashCode(); + } + } + + @Override + public K getKey() { + synchronized (mutex) { + return delegate().getKey(); + } + } + + @Override + public V getValue() { + synchronized (mutex) { + return delegate().getValue(); + } + } + + @Override + public V setValue(V value) { + synchronized (mutex) { + return delegate().setValue(value); + } + } + + private static final long serialVersionUID = 0; + } + +// static Queue queue(Queue queue, Object mutex) { +// return (queue instanceof SynchronizedQueue) ? queue : new SynchronizedQueue(queue, mutex); +// } +// +// private static class SynchronizedQueue extends SynchronizedCollection implements Queue { +// +// SynchronizedQueue(Queue delegate, Object mutex) { +// super(delegate, mutex); +// } +// +// @Override +// Queue delegate() { +// return (Queue) super.delegate(); +// } +// +// @Override +// public E element() { +// synchronized (mutex) { +// return delegate().element(); +// } +// } +// +// @Override +// public boolean offer(E e) { +// synchronized (mutex) { +// return delegate().offer(e); +// } +// } +// +// @Override +// public E peek() { +// synchronized (mutex) { +// return delegate().peek(); +// } +// } +// +// @Override +// public E poll() { +// synchronized (mutex) { +// return delegate().poll(); +// } +// } +// +// @Override +// public E remove() { +// synchronized (mutex) { +// return delegate().remove(); +// } +// } +// +// private static final long serialVersionUID = 0; +// } +// +// static Deque deque(Deque deque, Object mutex) { +// return new SynchronizedDeque(deque, mutex); +// } +// +// private static final class SynchronizedDeque extends SynchronizedQueue implements Deque { +// +// SynchronizedDeque(Deque delegate, Object mutex) { +// super(delegate, mutex); +// } +// +// @Override +// Deque delegate() { +// return (Deque) super.delegate(); +// } +// +// @Override +// public void addFirst(E e) { +// synchronized (mutex) { +// delegate().addFirst(e); +// } +// } +// +// @Override +// public void addLast(E e) { +// synchronized (mutex) { +// delegate().addLast(e); +// } +// } +// +// @Override +// public boolean offerFirst(E e) { +// synchronized (mutex) { +// return delegate().offerFirst(e); +// } +// } +// +// @Override +// public boolean offerLast(E e) { +// synchronized (mutex) { +// return delegate().offerLast(e); +// } +// } +// +// @Override +// public E removeFirst() { +// synchronized (mutex) { +// return delegate().removeFirst(); +// } +// } +// +// @Override +// public E removeLast() { +// synchronized (mutex) { +// return delegate().removeLast(); +// } +// } +// +// @Override +// public E pollFirst() { +// synchronized (mutex) { +// return delegate().pollFirst(); +// } +// } +// +// @Override +// public E pollLast() { +// synchronized (mutex) { +// return delegate().pollLast(); +// } +// } +// +// @Override +// public E getFirst() { +// synchronized (mutex) { +// return delegate().getFirst(); +// } +// } +// +// @Override +// public E getLast() { +// synchronized (mutex) { +// return delegate().getLast(); +// } +// } +// +// @Override +// public E peekFirst() { +// synchronized (mutex) { +// return delegate().peekFirst(); +// } +// } +// +// @Override +// public E peekLast() { +// synchronized (mutex) { +// return delegate().peekLast(); +// } +// } +// +// @Override +// public boolean removeFirstOccurrence(Object o) { +// synchronized (mutex) { +// return delegate().removeFirstOccurrence(o); +// } +// } +// +// @Override +// public boolean removeLastOccurrence(Object o) { +// synchronized (mutex) { +// return delegate().removeLastOccurrence(o); +// } +// } +// +// @Override +// public void push(E e) { +// synchronized (mutex) { +// delegate().push(e); +// } +// } +// +// @Override +// public E pop() { +// synchronized (mutex) { +// return delegate().pop(); +// } +// } +// +// @Override +// public Iterator descendingIterator() { +// synchronized (mutex) { +// return delegate().descendingIterator(); +// } +// } +// +// private static final long serialVersionUID = 0; +// } +// +// static Table table(Table table, Object mutex) { +// return new SynchronizedTable<>(table, mutex); +// } +// +// private static final class SynchronizedTable extends SynchronizedObject +// implements Table { +// +// SynchronizedTable(Table delegate, Object mutex) { +// super(delegate, mutex); +// } +// +// @SuppressWarnings({"unchecked", "rawtypes"}) +// @Override +// Table delegate() { +// return (Table) super.delegate(); +// } +// +// @Override +// public boolean contains(Object rowKey, Object columnKey) { +// synchronized (mutex) { +// return delegate().contains(rowKey, columnKey); +// } +// } +// +// @Override +// public boolean containsRow(Object rowKey) { +// synchronized (mutex) { +// return delegate().containsRow(rowKey); +// } +// } +// +// @Override +// public boolean containsColumn(Object columnKey) { +// synchronized (mutex) { +// return delegate().containsColumn(columnKey); +// } +// } +// +// @Override +// public boolean containsValue(Object value) { +// synchronized (mutex) { +// return delegate().containsValue(value); +// } +// } +// +// @Override +// public V get(Object rowKey, Object columnKey) { +// synchronized (mutex) { +// return delegate().get(rowKey, columnKey); +// } +// } +// +// @Override +// public boolean isEmpty() { +// synchronized (mutex) { +// return delegate().isEmpty(); +// } +// } +// +// @Override +// public int size() { +// synchronized (mutex) { +// return delegate().size(); +// } +// } +// +// @Override +// public void clear() { +// synchronized (mutex) { +// delegate().clear(); +// } +// } +// +// @Override +// public V put(R rowKey, C columnKey, V value) { +// synchronized (mutex) { +// return delegate().put(rowKey, columnKey, value); +// } +// } +// +// @Override +// public void putAll(Table table) { +// synchronized (mutex) { +// delegate().putAll(table); +// } +// } +// +// @Override +// public V remove(Object rowKey, Object columnKey) { +// synchronized (mutex) { +// return delegate().remove(rowKey, columnKey); +// } +// } +// +// @Override +// public Map row(R rowKey) { +// synchronized (mutex) { +// return map(delegate().row(rowKey), mutex); +// } +// } +// +// @Override +// public Map column(C columnKey) { +// synchronized (mutex) { +// return map(delegate().column(columnKey), mutex); +// } +// } +// +// @Override +// public Set> cellSet() { +// synchronized (mutex) { +// return set(delegate().cellSet(), mutex); +// } +// } +// +// @Override +// public Set rowKeySet() { +// synchronized (mutex) { +// return set(delegate().rowKeySet(), mutex); +// } +// } +// +// @Override +// public Set columnKeySet() { +// synchronized (mutex) { +// return set(delegate().columnKeySet(), mutex); +// } +// } +// +// @Override +// public Collection values() { +// synchronized (mutex) { +// return collection(delegate().values(), mutex); +// } +// } +// +// @Override +// public Map> rowMap() { +// synchronized (mutex) { +// return map( +// Maps.transformValues( +// delegate().rowMap(), +// new com.azure.cosmos.kafka.connect.implementation.guava25.base.Function, Map>() { +// @Override +// public Map apply(Map t) { +// return map(t, mutex); +// } +// }), +// mutex); +// } +// } +// +// @Override +// public Map> columnMap() { +// synchronized (mutex) { +// return map( +// Maps.transformValues( +// delegate().columnMap(), +// new com.azure.cosmos.kafka.connect.implementation.guava25.base.Function, Map>() { +// @Override +// public Map apply(Map t) { +// return map(t, mutex); +// } +// }), +// mutex); +// } +// } +// +// @Override +// public int hashCode() { +// synchronized (mutex) { +// return delegate().hashCode(); +// } +// } +// +// @Override +// public boolean equals(Object obj) { +// if (this == obj) { +// return true; +// } +// synchronized (mutex) { +// return delegate().equals(obj); +// } +// } +// } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Table.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Table.java new file mode 100644 index 0000000000000..b992ead85bc46 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/Table.java @@ -0,0 +1,280 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Objects; +import java.util.Collection; +import java.util.Map; +import java.util.Set; + + +/** + * A collection that associates an ordered pair of keys, called a row key and a column key, with a + * single value. A table may be sparse, with only a small fraction of row key / column key pairs + * possessing a corresponding value. + * + *

The mappings corresponding to a given row key may be viewed as a {@link Map} whose keys are + * the columns. The reverse is also available, associating a column with a row key / value map. Note + * that, in some implementations, data access by column key may have fewer supported operations or + * worse performance than data access by row key. + * + *

The methods returning collections or maps always return views of the underlying table. + * Updating the table can change the contents of those collections, and updating the collections + * will change the table. + * + *

All methods that modify the table are optional, and the views returned by the table may or may + * not be modifiable. When modification isn't supported, those methods will throw an {@link + * UnsupportedOperationException}. + * + *

See the Guava User Guide article on {@code Table}. + * + * @author Jared Levy + * @param the type of the table row keys + * @param the type of the table column keys + * @param the type of the mapped values + * @since 7.0 + */ +public interface Table { + // TODO(jlevy): Consider adding methods similar to ConcurrentMap methods. + + // Accessors + + /** + * Returns {@code true} if the table contains a mapping with the specified row and column keys. + * + * @param rowKey key of row to search for + * @param columnKey key of column to search for + */ + boolean contains( + /*@CompatibleWith("R")*/ Object rowKey, + /*@CompatibleWith("C")*/ Object columnKey); + + /** + * Returns {@code true} if the table contains a mapping with the specified row key. + * + * @param rowKey key of row to search for + */ + boolean containsRow(/*@CompatibleWith("R")*/ Object rowKey); + + /** + * Returns {@code true} if the table contains a mapping with the specified column. + * + * @param columnKey key of column to search for + */ + boolean containsColumn(/*@CompatibleWith("C")*/ Object columnKey); + + /** + * Returns {@code true} if the table contains a mapping with the specified value. + * + * @param value value to search for + */ + boolean containsValue(/*@CompatibleWith("V")*/ Object value); + + /** + * Returns the value corresponding to the given row and column keys, or {@code null} if no such + * mapping exists. + * + * @param rowKey key of row to search for + * @param columnKey key of column to search for + */ + V get( + /*@CompatibleWith("R")*/ Object rowKey, + /*@CompatibleWith("C")*/ Object columnKey); + + /** Returns {@code true} if the table contains no mappings. */ + boolean isEmpty(); + + /** Returns the number of row key / column key / value mappings in the table. */ + int size(); + + /** + * Compares the specified object with this table for equality. Two tables are equal when their + * cell views, as returned by {@link #cellSet}, are equal. + */ + @Override + boolean equals(Object obj); + + /** + * Returns the hash code for this table. The hash code of a table is defined as the hash code of + * its cell view, as returned by {@link #cellSet}. + */ + @Override + int hashCode(); + + // Mutators + + /** Removes all mappings from the table. */ + void clear(); + + /** + * Associates the specified value with the specified keys. If the table already contained a + * mapping for those keys, the old value is replaced with the specified value. + * + * @param rowKey row key that the value should be associated with + * @param columnKey column key that the value should be associated with + * @param value value to be associated with the specified keys + * @return the value previously associated with the keys, or {@code null} if no mapping existed + * for the keys + */ + + V put(R rowKey, C columnKey, V value); + + /** + * Copies all mappings from the specified table to this table. The effect is equivalent to calling + * {@link #put} with each row key / column key / value mapping in {@code table}. + * + * @param table the table to add to this table + */ + void putAll(Table table); + + /** + * Removes the mapping, if any, associated with the given keys. + * + * @param rowKey row key of mapping to be removed + * @param columnKey column key of mapping to be removed + * @return the value previously associated with the keys, or {@code null} if no such value existed + */ + + V remove( + /*@CompatibleWith("R")*/ Object rowKey, + /*@CompatibleWith("C")*/ Object columnKey); + + // Views + + /** + * Returns a view of all mappings that have the given row key. For each row key / column key / + * value mapping in the table with that row key, the returned map associates the column key with + * the value. If no mappings in the table have the provided row key, an empty map is returned. + * + *

Changes to the returned map will update the underlying table, and vice versa. + * + * @param rowKey key of row to search for in the table + * @return the corresponding map from column keys to values + */ + Map row(R rowKey); + + /** + * Returns a view of all mappings that have the given column key. For each row key / column key / + * value mapping in the table with that column key, the returned map associates the row key with + * the value. If no mappings in the table have the provided column key, an empty map is returned. + * + *

Changes to the returned map will update the underlying table, and vice versa. + * + * @param columnKey key of column to search for in the table + * @return the corresponding map from row keys to values + */ + Map column(C columnKey); + + /** + * Returns a set of all row key / column key / value triplets. Changes to the returned set will + * update the underlying table, and vice versa. The cell set does not support the {@code add} or + * {@code addAll} methods. + * + * @return set of table cells consisting of row key / column key / value triplets + */ + Set> cellSet(); + + /** + * Returns a set of row keys that have one or more values in the table. Changes to the set will + * update the underlying table, and vice versa. + * + * @return set of row keys + */ + Set rowKeySet(); + + /** + * Returns a set of column keys that have one or more values in the table. Changes to the set will + * update the underlying table, and vice versa. + * + * @return set of column keys + */ + Set columnKeySet(); + + /** + * Returns a collection of all values, which may contain duplicates. Changes to the returned + * collection will update the underlying table, and vice versa. + * + * @return collection of values + */ + Collection values(); + + /** + * Returns a view that associates each row key with the corresponding map from column keys to + * values. Changes to the returned map will update this table. The returned map does not support + * {@code put()} or {@code putAll()}, or {@code setValue()} on its entries. + * + *

In contrast, the maps returned by {@code rowMap().get()} have the same behavior as those + * returned by {@link #row}. Those maps may support {@code setValue()}, {@code put()}, and {@code + * putAll()}. + * + * @return a map view from each row key to a secondary map from column keys to values + */ + Map> rowMap(); + + /** + * Returns a view that associates each column key with the corresponding map from row keys to + * values. Changes to the returned map will update this table. The returned map does not support + * {@code put()} or {@code putAll()}, or {@code setValue()} on its entries. + * + *

In contrast, the maps returned by {@code columnMap().get()} have the same behavior as those + * returned by {@link #column}. Those maps may support {@code setValue()}, {@code put()}, and + * {@code putAll()}. + * + * @return a map view from each column key to a secondary map from row keys to values + */ + Map> columnMap(); + + /** + * Row key / column key / value triplet corresponding to a mapping in a table. + * + * @since 7.0 + */ + interface Cell { + /** Returns the row key of this cell. */ + + R getRowKey(); + + /** Returns the column key of this cell. */ + + C getColumnKey(); + + /** Returns the value of this cell. */ + + V getValue(); + + /** + * Compares the specified object with this cell for equality. Two cells are equal when they have + * equal row keys, column keys, and values. + */ + @Override + boolean equals(Object obj); + + /** + * Returns the hash code of this cell. + * + *

The hash code of a table cell is equal to {@link Objects#hashCode}{@code (e.getRowKey(), + * e.getColumnKey(), e.getValue())}. + */ + @Override + int hashCode(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TopKSelector.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TopKSelector.java new file mode 100644 index 0000000000000..f05fbd6022a6c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TopKSelector.java @@ -0,0 +1,275 @@ +/* + * Copyright (C) 2014 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import com.azure.cosmos.kafka.connect.implementation.guava25.math.IntMath; +import java.math.RoundingMode; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Stream; + + +/** + * An accumulator that selects the "top" {@code k} elements added to it, relative to a provided + * comparator. "Top" can mean the greatest or the lowest elements, specified in the factory used to + * create the {@code TopKSelector} instance. + * + *

If your input data is available as a {@link Stream}, prefer passing {@link + * Comparators#least(int)} to {@link Stream#collect(java.util.stream.Collector)}. If it is available + * as an {@link Iterable} or {@link Iterator}, prefer {@link Ordering#leastOf(Iterable, int)}. + * + *

This uses the same efficient implementation as {@link Ordering#leastOf(Iterable, int)}, + * offering expected O(n + k log k) performance (worst case O(n log k)) for n calls to {@link + * #offer} and a call to {@link #topK}, with O(k) memory. In comparison, quickselect has the same + * asymptotics but requires O(n) memory, and a {@code PriorityQueue} implementation takes O(n log + * k). In benchmarks, this implementation performs at least as well as either implementation, and + * degrades more gracefully for worst-case input. + * + *

The implementation does not necessarily use a stable sorting algorithm; when multiple + * equivalent elements are added to it, it is undefined which will come first in the output. + * + * @author Louis Wasserman + */ +final class TopKSelector { + + /** + * Returns a {@code TopKSelector} that collects the lowest {@code k} elements added to it, + * relative to the natural ordering of the elements, and returns them via {@link #topK} in + * ascending order. + * + * @throws IllegalArgumentException if {@code k < 0} + */ + public static > TopKSelector least(int k) { + return least(k, Ordering.natural()); + } + + /** + * Returns a {@code TopKSelector} that collects the lowest {@code k} elements added to it, + * relative to the specified comparator, and returns them via {@link #topK} in ascending order. + * + * @throws IllegalArgumentException if {@code k < 0} + */ + public static TopKSelector least(int k, Comparator comparator) { + return new TopKSelector(comparator, k); + } + + /** + * Returns a {@code TopKSelector} that collects the greatest {@code k} elements added to it, + * relative to the natural ordering of the elements, and returns them via {@link #topK} in + * descending order. + * + * @throws IllegalArgumentException if {@code k < 0} + */ + public static > TopKSelector greatest(int k) { + return greatest(k, Ordering.natural()); + } + + /** + * Returns a {@code TopKSelector} that collects the greatest {@code k} elements added to it, + * relative to the specified comparator, and returns them via {@link #topK} in descending order. + * + * @throws IllegalArgumentException if {@code k < 0} + */ + public static TopKSelector greatest(int k, Comparator comparator) { + return new TopKSelector(Ordering.from(comparator).reverse(), k); + } + + private final int k; + private final Comparator comparator; + + /* + * We are currently considering the elements in buffer in the range [0, bufferSize) as candidates + * for the top k elements. Whenever the buffer is filled, we quickselect the top k elements to the + * range [0, k) and ignore the remaining elements. + */ + private final T[] buffer; + private int bufferSize; + + /** + * The largest of the lowest k elements we've seen so far relative to this comparator. If + * bufferSize ≥ k, then we can ignore any elements greater than this value. + */ + private T threshold; + + @SuppressWarnings({"unchecked", "rawtypes"}) + private TopKSelector(Comparator comparator, int k) { + this.comparator = checkNotNull(comparator, "comparator"); + this.k = k; + checkArgument(k >= 0, "k must be nonnegative, was %s", k); + this.buffer = (T[]) new Object[k * 2]; + this.bufferSize = 0; + this.threshold = null; + } + + /** + * Adds {@code elem} as a candidate for the top {@code k} elements. This operation takes amortized + * O(1) time. + */ + public void offer(T elem) { + if (k == 0) { + return; + } else if (bufferSize == 0) { + buffer[0] = elem; + threshold = elem; + bufferSize = 1; + } else if (bufferSize < k) { + buffer[bufferSize++] = elem; + if (comparator.compare(elem, threshold) > 0) { + threshold = elem; + } + } else if (comparator.compare(elem, threshold) < 0) { + // Otherwise, we can ignore elem; we've seen k better elements. + buffer[bufferSize++] = elem; + if (bufferSize == 2 * k) { + trim(); + } + } + } + + /** + * Quickselects the top k elements from the 2k elements in the buffer. O(k) expected time, O(k log + * k) worst case. + */ + private void trim() { + int left = 0; + int right = 2 * k - 1; + + int minThresholdPosition = 0; + // The leftmost position at which the greatest of the k lower elements + // -- the new value of threshold -- might be found. + + int iterations = 0; + int maxIterations = IntMath.log2(right - left, RoundingMode.CEILING) * 3; + while (left < right) { + int pivotIndex = (left + right + 1) >>> 1; + + int pivotNewIndex = partition(left, right, pivotIndex); + + if (pivotNewIndex > k) { + right = pivotNewIndex - 1; + } else if (pivotNewIndex < k) { + left = Math.max(pivotNewIndex, left + 1); + minThresholdPosition = pivotNewIndex; + } else { + break; + } + iterations++; + if (iterations >= maxIterations) { + // We've already taken O(k log k), let's make sure we don't take longer than O(k log k). + Arrays.sort(buffer, left, right, comparator); + break; + } + } + bufferSize = k; + + threshold = buffer[minThresholdPosition]; + for (int i = minThresholdPosition + 1; i < k; i++) { + if (comparator.compare(buffer[i], threshold) > 0) { + threshold = buffer[i]; + } + } + } + + /** + * Partitions the contents of buffer in the range [left, right] around the pivot element + * previously stored in buffer[pivotValue]. Returns the new index of the pivot element, + * pivotNewIndex, so that everything in [left, pivotNewIndex] is ≤ pivotValue and everything in + * (pivotNewIndex, right] is greater than pivotValue. + */ + private int partition(int left, int right, int pivotIndex) { + T pivotValue = buffer[pivotIndex]; + buffer[pivotIndex] = buffer[right]; + + int pivotNewIndex = left; + for (int i = left; i < right; i++) { + if (comparator.compare(buffer[i], pivotValue) < 0) { + swap(pivotNewIndex, i); + pivotNewIndex++; + } + } + buffer[right] = buffer[pivotNewIndex]; + buffer[pivotNewIndex] = pivotValue; + return pivotNewIndex; + } + + private void swap(int i, int j) { + T tmp = buffer[i]; + buffer[i] = buffer[j]; + buffer[j] = tmp; + } + + TopKSelector combine(TopKSelector other) { + for (int i = 0; i < other.bufferSize; i++) { + this.offer(other.buffer[i]); + } + return this; + } + + /** + * Adds each member of {@code elements} as a candidate for the top {@code k} elements. This + * operation takes amortized linear time in the length of {@code elements}. + * + *

If all input data to this {@code TopKSelector} is in a single {@code Iterable}, prefer + * {@link Ordering#leastOf(Iterable, int)}, which provides a simpler API for that use case. + */ + public void offerAll(Iterable elements) { + offerAll(elements.iterator()); + } + + /** + * Adds each member of {@code elements} as a candidate for the top {@code k} elements. This + * operation takes amortized linear time in the length of {@code elements}. The iterator is + * consumed after this operation completes. + * + *

If all input data to this {@code TopKSelector} is in a single {@code Iterator}, prefer + * {@link Ordering#leastOf(Iterator, int)}, which provides a simpler API for that use case. + */ + public void offerAll(Iterator elements) { + while (elements.hasNext()) { + offer(elements.next()); + } + } + + /** + * Returns the top {@code k} elements offered to this {@code TopKSelector}, or all elements if + * fewer than {@code k} have been offered, in the order specified by the factory used to create + * this {@code TopKSelector}. + * + *

The returned list is an unmodifiable copy and will not be affected by further changes to + * this {@code TopKSelector}. This method returns in O(k log k) time. + */ + public List topK() { + Arrays.sort(buffer, 0, bufferSize, comparator); + if (bufferSize > k) { + Arrays.fill(buffer, k, buffer.length, null); + bufferSize = k; + threshold = buffer[k - 1]; + } + // we have to support null elements, so no ImmutableList for us + return Collections.unmodifiableList(Arrays.asList(Arrays.copyOf(buffer, bufferSize))); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedIterator.java new file mode 100644 index 0000000000000..bc4b6377b2cc2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedIterator.java @@ -0,0 +1,56 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.util.Iterator; + +/** + * An iterator that transforms a backing iterator; for internal use. This avoids the object overhead + * of constructing a {@link com.azure.cosmos.kafka.connect.implementation.guava25.base.Function Function} for internal methods. + * + * @author Louis Wasserman + */ +abstract class TransformedIterator implements Iterator { + final Iterator backingIterator; + + TransformedIterator(Iterator backingIterator) { + this.backingIterator = checkNotNull(backingIterator); + } + + abstract T transform(F from); + + @Override + public final boolean hasNext() { + return backingIterator.hasNext(); + } + + @Override + public final T next() { + return transform(backingIterator.next()); + } + + @Override + public final void remove() { + backingIterator.remove(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedListIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedListIterator.java new file mode 100644 index 0000000000000..5320d0d4c869d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TransformedListIterator.java @@ -0,0 +1,72 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; + +import java.util.ListIterator; + +/** + * An iterator that transforms a backing list iterator; for internal use. This avoids the object + * overhead of constructing a {@link Function} for internal methods. + * + * @author Louis Wasserman + */ +abstract class TransformedListIterator extends TransformedIterator + implements ListIterator { + TransformedListIterator(ListIterator backingIterator) { + super(backingIterator); + } + + private ListIterator backingIterator() { + return Iterators.cast(backingIterator); + } + + @Override + public final boolean hasPrevious() { + return backingIterator().hasPrevious(); + } + + @Override + public final T previous() { + return transform(backingIterator().previous()); + } + + @Override + public final int nextIndex() { + return backingIterator().nextIndex(); + } + + @Override + public final int previousIndex() { + return backingIterator().previousIndex(); + } + + @Override + public void set(T element) { + throw new UnsupportedOperationException(); + } + + @Override + public void add(T element) { + throw new UnsupportedOperationException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultimap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultimap.java new file mode 100644 index 0000000000000..de489e630c73c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultimap.java @@ -0,0 +1,221 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Collection; +import java.util.Comparator; +import java.util.Map; +import java.util.NavigableMap; +import java.util.NavigableSet; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; + + +/** + * Implementation of {@code Multimap} whose keys and values are ordered by their natural ordering or + * by supplied comparators. In all cases, this implementation uses {@link Comparable#compareTo} or + * {@link Comparator#compare} instead of {@link Object#equals} to determine equivalence of + * instances. + * + *

Warning: The comparators or comparables used must be consistent with equals as + * explained by the {@link Comparable} class specification. Otherwise, the resulting multiset will + * violate the general contract of {@link SetMultimap}, which it is specified in terms of {@link + * Object#equals}. + * + *

The collections returned by {@code keySet} and {@code asMap} iterate through the keys + * according to the key comparator ordering or the natural ordering of the keys. Similarly, {@code + * get}, {@code removeAll}, and {@code replaceValues} return collections that iterate through the + * values according to the value comparator ordering or the natural ordering of the values. The + * collections generated by {@code entries}, {@code keys}, and {@code values} iterate across the + * keys according to the above key ordering, and for each key they iterate across the values + * according to the value ordering. + * + *

The multimap does not store duplicate key-value pairs. Adding a new key-value pair equal to an + * existing key-value pair has no effect. + * + *

Null keys and values are permitted (provided, of course, that the respective comparators + * support them). All optional multimap methods are supported, and all returned views are + * modifiable. + * + *

This class is not threadsafe when any concurrent operations update the multimap. Concurrent + * read operations will work correctly. To allow concurrent update operations, wrap your multimap + * with a call to {@link Multimaps#synchronizedSortedSetMultimap}. + * + *

See the Guava User Guide article on {@code + * Multimap}. + * + * @author Jared Levy + * @author Louis Wasserman + * @since 2.0 + */ +public class TreeMultimap extends AbstractSortedKeySortedSetMultimap { + private transient Comparator keyComparator; + private transient Comparator valueComparator; + + /** + * Creates an empty {@code TreeMultimap} ordered by the natural ordering of its keys and values. + */ + @SuppressWarnings("rawtypes") + public static TreeMultimap create() { + return new TreeMultimap<>(Ordering.natural(), Ordering.natural()); + } + + /** + * Creates an empty {@code TreeMultimap} instance using explicit comparators. Neither comparator + * may be null; use {@link Ordering#natural()} to specify natural order. + * + * @param keyComparator the comparator that determines the key ordering + * @param valueComparator the comparator that determines the value ordering + */ + public static TreeMultimap create( + Comparator keyComparator, Comparator valueComparator) { + return new TreeMultimap<>(checkNotNull(keyComparator), checkNotNull(valueComparator)); + } + + /** + * Constructs a {@code TreeMultimap}, ordered by the natural ordering of its keys and values, with + * the same mappings as the specified multimap. + * + * @param multimap the multimap whose contents are copied to this multimap + */ + @SuppressWarnings("rawtypes") + public static TreeMultimap create( + Multimap multimap) { + return new TreeMultimap<>(Ordering.natural(), Ordering.natural(), multimap); + } + + TreeMultimap(Comparator keyComparator, Comparator valueComparator) { + super(new TreeMap>(keyComparator)); + this.keyComparator = keyComparator; + this.valueComparator = valueComparator; + } + + private TreeMultimap( + Comparator keyComparator, + Comparator valueComparator, + Multimap multimap) { + this(keyComparator, valueComparator); + putAll(multimap); + } + + @Override + Map> createAsMap() { + return createMaybeNavigableAsMap(); + } + + /** + * {@inheritDoc} + * + *

Creates an empty {@code TreeSet} for a collection of values for one key. + * + * @return a new {@code TreeSet} containing a collection of values for one key + */ + @Override + SortedSet createCollection() { + return new TreeSet(valueComparator); + } + + @Override + Collection createCollection(K key) { + if (key == null) { + keyComparator().compare(key, key); + } + return super.createCollection(key); + } + + /** + * Returns the comparator that orders the multimap keys. + * + * @deprecated Use {@code ((NavigableSet) multimap.keySet()).comparator()} instead. + */ + @Deprecated + public Comparator keyComparator() { + return keyComparator; + } + + @Override + public Comparator valueComparator() { + return valueComparator; + } + + /** @since 14.0 (present with return type {@code SortedSet} since 2.0) */ + @Override + public NavigableSet get(K key) { + return (NavigableSet) super.get(key); + } + + /** + * {@inheritDoc} + * + *

Because a {@code TreeMultimap} has unique sorted keys, this method returns a {@link + * NavigableSet}, instead of the {@link java.util.Set} specified in the {@link Multimap} + * interface. + * + * @since 14.0 (present with return type {@code SortedSet} since 2.0) + */ + @Override + public NavigableSet keySet() { + return (NavigableSet) super.keySet(); + } + + /** + * {@inheritDoc} + * + *

Because a {@code TreeMultimap} has unique sorted keys, this method returns a {@link + * NavigableMap}, instead of the {@link Map} specified in the {@link Multimap} + * interface. + * + * @since 14.0 (present with return type {@code SortedMap} since 2.0) + */ + @Override + public NavigableMap> asMap() { + return (NavigableMap>) super.asMap(); + } + + /** + * @serialData key comparator, value comparator, number of distinct keys, and then for each + * distinct key: the key, number of values for that key, and key values + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(keyComparator()); + stream.writeObject(valueComparator()); + Serialization.writeMultimap(this, stream); + } + + @SuppressWarnings({"unchecked", "rawtypes"}) // reading data stored by writeObject + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + keyComparator = checkNotNull((Comparator) stream.readObject()); + valueComparator = checkNotNull((Comparator) stream.readObject()); + setMap(new TreeMap>(keyComparator)); + Serialization.populateMultimap(this, stream); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultiset.java new file mode 100644 index 0000000000000..14d34b82954c1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeMultiset.java @@ -0,0 +1,1030 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkNonnegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.collect.CollectPreconditions.checkRemove; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects; +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.Comparator; +import java.util.ConcurrentModificationException; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.function.ObjIntConsumer; + + +/** + * A multiset which maintains the ordering of its elements, according to either their natural order + * or an explicit {@link Comparator}. In all cases, this implementation uses {@link + * Comparable#compareTo} or {@link Comparator#compare} instead of {@link Object#equals} to determine + * equivalence of instances. + * + *

Warning: The comparison must be consistent with equals as explained by the + * {@link Comparable} class specification. Otherwise, the resulting multiset will violate the {@link + * java.util.Collection} contract, which is specified in terms of {@link Object#equals}. + * + *

See the Guava User Guide article on {@code + * Multiset}. + * + * @author Louis Wasserman + * @author Jared Levy + * @since 2.0 + */ +public final class TreeMultiset extends AbstractSortedMultiset implements Serializable { + + /** + * Creates a new, empty multiset, sorted according to the elements' natural order. All elements + * inserted into the multiset must implement the {@code Comparable} interface. Furthermore, all + * such elements must be mutually comparable: {@code e1.compareTo(e2)} must not throw a + * {@code ClassCastException} for any elements {@code e1} and {@code e2} in the multiset. If the + * user attempts to add an element to the multiset that violates this constraint (for example, the + * user attempts to add a string element to a set whose elements are integers), the {@code + * add(Object)} call will throw a {@code ClassCastException}. + * + *

The type specification is {@code }, instead of the more specific + * {@code >}, to support classes defined without generics. + */ + @SuppressWarnings("rawtypes") + public static TreeMultiset create() { + return new TreeMultiset(Ordering.natural()); + } + + /** + * Creates a new, empty multiset, sorted according to the specified comparator. All elements + * inserted into the multiset must be mutually comparable by the specified comparator: + * {@code comparator.compare(e1, e2)} must not throw a {@code ClassCastException} for any elements + * {@code e1} and {@code e2} in the multiset. If the user attempts to add an element to the + * multiset that violates this constraint, the {@code add(Object)} call will throw a {@code + * ClassCastException}. + * + * @param comparator the comparator that will be used to sort this multiset. A null value + * indicates that the elements' natural ordering should be used. + */ + @SuppressWarnings({"unchecked", "rawtypes"}) + public static TreeMultiset create(Comparator comparator) { + return (comparator == null) + ? new TreeMultiset((Comparator) Ordering.natural()) + : new TreeMultiset(comparator); + } + + /** + * Creates an empty multiset containing the given initial elements, sorted according to the + * elements' natural order. + * + *

This implementation is highly efficient when {@code elements} is itself a {@link Multiset}. + * + *

The type specification is {@code }, instead of the more specific + * {@code >}, to support classes defined without generics. + */ + @SuppressWarnings("rawtypes") + public static TreeMultiset create(Iterable elements) { + TreeMultiset multiset = create(); + Iterables.addAll(multiset, elements); + return multiset; + } + + private final transient Reference> rootReference; + private final transient GeneralRange range; + private final transient AvlNode header; + + TreeMultiset(Reference> rootReference, GeneralRange range, AvlNode endLink) { + super(range.comparator()); + this.rootReference = rootReference; + this.range = range; + this.header = endLink; + } + + TreeMultiset(Comparator comparator) { + super(comparator); + this.range = GeneralRange.all(comparator); + this.header = new AvlNode(null, 1); + successor(header, header); + this.rootReference = new Reference<>(); + } + + /** A function which can be summed across a subtree. */ + private enum Aggregate { + SIZE { + @Override + int nodeAggregate(AvlNode node) { + return node.elemCount; + } + + @Override + long treeAggregate(AvlNode root) { + return (root == null) ? 0 : root.totalCount; + } + }, + DISTINCT { + @Override + int nodeAggregate(AvlNode node) { + return 1; + } + + @Override + long treeAggregate(AvlNode root) { + return (root == null) ? 0 : root.distinctElements; + } + }; + + abstract int nodeAggregate(AvlNode node); + + abstract long treeAggregate(AvlNode root); + } + + private long aggregateForEntries(Aggregate aggr) { + AvlNode root = rootReference.get(); + long total = aggr.treeAggregate(root); + if (range.hasLowerBound()) { + total -= aggregateBelowRange(aggr, root); + } + if (range.hasUpperBound()) { + total -= aggregateAboveRange(aggr, root); + } + return total; + } + + private long aggregateBelowRange(Aggregate aggr, AvlNode node) { + if (node == null) { + return 0; + } + int cmp = comparator().compare(range.getLowerEndpoint(), node.elem); + if (cmp < 0) { + return aggregateBelowRange(aggr, node.left); + } else if (cmp == 0) { + switch (range.getLowerBoundType()) { + case OPEN: + return aggr.nodeAggregate(node) + aggr.treeAggregate(node.left); + case CLOSED: + return aggr.treeAggregate(node.left); + default: + throw new AssertionError(); + } + } else { + return aggr.treeAggregate(node.left) + + aggr.nodeAggregate(node) + + aggregateBelowRange(aggr, node.right); + } + } + + private long aggregateAboveRange(Aggregate aggr, AvlNode node) { + if (node == null) { + return 0; + } + int cmp = comparator().compare(range.getUpperEndpoint(), node.elem); + if (cmp > 0) { + return aggregateAboveRange(aggr, node.right); + } else if (cmp == 0) { + switch (range.getUpperBoundType()) { + case OPEN: + return aggr.nodeAggregate(node) + aggr.treeAggregate(node.right); + case CLOSED: + return aggr.treeAggregate(node.right); + default: + throw new AssertionError(); + } + } else { + return aggr.treeAggregate(node.right) + + aggr.nodeAggregate(node) + + aggregateAboveRange(aggr, node.left); + } + } + + @Override + public int size() { + return Ints.saturatedCast(aggregateForEntries(Aggregate.SIZE)); + } + + @Override + int distinctElements() { + return Ints.saturatedCast(aggregateForEntries(Aggregate.DISTINCT)); + } + + static int distinctElements(AvlNode node) { + return (node == null) ? 0 : node.distinctElements; + } + + @Override + public int count(Object element) { + try { + @SuppressWarnings({"unchecked", "rawtypes"}) + E e = (E) element; + AvlNode root = rootReference.get(); + if (!range.contains(e) || root == null) { + return 0; + } + return root.count(comparator(), e); + } catch (ClassCastException | NullPointerException e) { + return 0; + } + } + + @Override + public int add(E element, int occurrences) { + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(element); + } + checkArgument(range.contains(element)); + AvlNode root = rootReference.get(); + if (root == null) { + comparator().compare(element, element); + AvlNode newRoot = new AvlNode(element, occurrences); + successor(header, newRoot, header); + rootReference.checkAndSet(root, newRoot); + return 0; + } + int[] result = new int[1]; // used as a mutable int reference to hold result + AvlNode newRoot = root.add(comparator(), element, occurrences, result); + rootReference.checkAndSet(root, newRoot); + return result[0]; + } + + @Override + public int remove(Object element, int occurrences) { + checkNonnegative(occurrences, "occurrences"); + if (occurrences == 0) { + return count(element); + } + AvlNode root = rootReference.get(); + int[] result = new int[1]; // used as a mutable int reference to hold result + AvlNode newRoot; + try { + @SuppressWarnings({"unchecked", "rawtypes"}) + E e = (E) element; + if (!range.contains(e) || root == null) { + return 0; + } + newRoot = root.remove(comparator(), e, occurrences, result); + } catch (ClassCastException | NullPointerException e) { + return 0; + } + rootReference.checkAndSet(root, newRoot); + return result[0]; + } + + @Override + public int setCount(E element, int count) { + checkNonnegative(count, "count"); + if (!range.contains(element)) { + checkArgument(count == 0); + return 0; + } + + AvlNode root = rootReference.get(); + if (root == null) { + if (count > 0) { + add(element, count); + } + return 0; + } + int[] result = new int[1]; // used as a mutable int reference to hold result + AvlNode newRoot = root.setCount(comparator(), element, count, result); + rootReference.checkAndSet(root, newRoot); + return result[0]; + } + + @Override + public boolean setCount(E element, int oldCount, int newCount) { + checkNonnegative(newCount, "newCount"); + checkNonnegative(oldCount, "oldCount"); + checkArgument(range.contains(element)); + + AvlNode root = rootReference.get(); + if (root == null) { + if (oldCount == 0) { + if (newCount > 0) { + add(element, newCount); + } + return true; + } else { + return false; + } + } + int[] result = new int[1]; // used as a mutable int reference to hold result + AvlNode newRoot = root.setCount(comparator(), element, oldCount, newCount, result); + rootReference.checkAndSet(root, newRoot); + return result[0] == oldCount; + } + + @Override + public void clear() { + if (!range.hasLowerBound() && !range.hasUpperBound()) { + // We can do this in O(n) rather than removing one by one, which could force rebalancing. + for (AvlNode current = header.succ; current != header; ) { + AvlNode next = current.succ; + + current.elemCount = 0; + // Also clear these fields so that one deleted Entry doesn't retain all elements. + current.left = null; + current.right = null; + current.pred = null; + current.succ = null; + + current = next; + } + successor(header, header); + rootReference.clear(); + } else { + // TODO(cpovirk): Perhaps we can optimize in this case, too? + Iterators.clear(entryIterator()); + } + } + + private Entry wrapEntry(final AvlNode baseEntry) { + return new Multisets.AbstractEntry() { + @Override + public E getElement() { + return baseEntry.getElement(); + } + + @Override + public int getCount() { + int result = baseEntry.getCount(); + if (result == 0) { + return count(getElement()); + } else { + return result; + } + } + }; + } + + /** Returns the first node in the tree that is in range. */ + + private AvlNode firstNode() { + AvlNode root = rootReference.get(); + if (root == null) { + return null; + } + AvlNode node; + if (range.hasLowerBound()) { + E endpoint = range.getLowerEndpoint(); + node = rootReference.get().ceiling(comparator(), endpoint); + if (node == null) { + return null; + } + if (range.getLowerBoundType() == BoundType.OPEN + && comparator().compare(endpoint, node.getElement()) == 0) { + node = node.succ; + } + } else { + node = header.succ; + } + return (node == header || !range.contains(node.getElement())) ? null : node; + } + + + private AvlNode lastNode() { + AvlNode root = rootReference.get(); + if (root == null) { + return null; + } + AvlNode node; + if (range.hasUpperBound()) { + E endpoint = range.getUpperEndpoint(); + node = rootReference.get().floor(comparator(), endpoint); + if (node == null) { + return null; + } + if (range.getUpperBoundType() == BoundType.OPEN + && comparator().compare(endpoint, node.getElement()) == 0) { + node = node.pred; + } + } else { + node = header.pred; + } + return (node == header || !range.contains(node.getElement())) ? null : node; + } + + @Override + Iterator elementIterator() { + return Multisets.elementIterator(entryIterator()); + } + + @Override + Iterator> entryIterator() { + return new Iterator>() { + AvlNode current = firstNode(); + Entry prevEntry; + + @Override + public boolean hasNext() { + if (current == null) { + return false; + } else if (range.tooHigh(current.getElement())) { + current = null; + return false; + } else { + return true; + } + } + + @Override + public Entry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + Entry result = wrapEntry(current); + prevEntry = result; + if (current.succ == header) { + current = null; + } else { + current = current.succ; + } + return result; + } + + @Override + public void remove() { + checkRemove(prevEntry != null); + setCount(prevEntry.getElement(), 0); + prevEntry = null; + } + }; + } + + @Override + Iterator> descendingEntryIterator() { + return new Iterator>() { + AvlNode current = lastNode(); + Entry prevEntry = null; + + @Override + public boolean hasNext() { + if (current == null) { + return false; + } else if (range.tooLow(current.getElement())) { + current = null; + return false; + } else { + return true; + } + } + + @Override + public Entry next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + Entry result = wrapEntry(current); + prevEntry = result; + if (current.pred == header) { + current = null; + } else { + current = current.pred; + } + return result; + } + + @Override + public void remove() { + checkRemove(prevEntry != null); + setCount(prevEntry.getElement(), 0); + prevEntry = null; + } + }; + } + + @Override + public void forEachEntry(ObjIntConsumer action) { + checkNotNull(action); + for (AvlNode node = firstNode(); + node != header && node != null && !range.tooHigh(node.getElement()); + node = node.succ) { + action.accept(node.getElement(), node.getCount()); + } + } + + @Override + public Iterator iterator() { + return Multisets.iteratorImpl(this); + } + + @Override + public SortedMultiset headMultiset(E upperBound, BoundType boundType) { + return new TreeMultiset( + rootReference, + range.intersect(GeneralRange.upTo(comparator(), upperBound, boundType)), + header); + } + + @Override + public SortedMultiset tailMultiset(E lowerBound, BoundType boundType) { + return new TreeMultiset( + rootReference, + range.intersect(GeneralRange.downTo(comparator(), lowerBound, boundType)), + header); + } + + private static final class Reference { + private T value; + + + public T get() { + return value; + } + + public void checkAndSet(T expected, T newValue) { + if (value != expected) { + throw new ConcurrentModificationException(); + } + value = newValue; + } + + void clear() { + value = null; + } + } + + private static final class AvlNode { + private final E elem; + + // elemCount is 0 iff this node has been deleted. + private int elemCount; + + private int distinctElements; + private long totalCount; + private int height; + private AvlNode left; + private AvlNode right; + private AvlNode pred; + private AvlNode succ; + + AvlNode(E elem, int elemCount) { + checkArgument(elemCount > 0); + this.elem = elem; + this.elemCount = elemCount; + this.totalCount = elemCount; + this.distinctElements = 1; + this.height = 1; + this.left = null; + this.right = null; + } + + public int count(Comparator comparator, E e) { + int cmp = comparator.compare(e, elem); + if (cmp < 0) { + return (left == null) ? 0 : left.count(comparator, e); + } else if (cmp > 0) { + return (right == null) ? 0 : right.count(comparator, e); + } else { + return elemCount; + } + } + + private AvlNode addRightChild(E e, int count) { + right = new AvlNode(e, count); + successor(this, right, succ); + height = Math.max(2, height); + distinctElements++; + totalCount += count; + return this; + } + + private AvlNode addLeftChild(E e, int count) { + left = new AvlNode(e, count); + successor(pred, left, this); + height = Math.max(2, height); + distinctElements++; + totalCount += count; + return this; + } + + AvlNode add(Comparator comparator, E e, int count, int[] result) { + /* + * It speeds things up considerably to unconditionally add count to totalCount here, + * but that destroys failure atomicity in the case of count overflow. =( + */ + int cmp = comparator.compare(e, elem); + if (cmp < 0) { + AvlNode initLeft = left; + if (initLeft == null) { + result[0] = 0; + return addLeftChild(e, count); + } + int initHeight = initLeft.height; + + left = initLeft.add(comparator, e, count, result); + if (result[0] == 0) { + distinctElements++; + } + this.totalCount += count; + return (left.height == initHeight) ? this : rebalance(); + } else if (cmp > 0) { + AvlNode initRight = right; + if (initRight == null) { + result[0] = 0; + return addRightChild(e, count); + } + int initHeight = initRight.height; + + right = initRight.add(comparator, e, count, result); + if (result[0] == 0) { + distinctElements++; + } + this.totalCount += count; + return (right.height == initHeight) ? this : rebalance(); + } + + // adding count to me! No rebalance possible. + result[0] = elemCount; + long resultCount = (long) elemCount + count; + checkArgument(resultCount <= Integer.MAX_VALUE); + this.elemCount += count; + this.totalCount += count; + return this; + } + + AvlNode remove( + Comparator comparator, E e, int count, int[] result) { + int cmp = comparator.compare(e, elem); + if (cmp < 0) { + AvlNode initLeft = left; + if (initLeft == null) { + result[0] = 0; + return this; + } + + left = initLeft.remove(comparator, e, count, result); + + if (result[0] > 0) { + if (count >= result[0]) { + this.distinctElements--; + this.totalCount -= result[0]; + } else { + this.totalCount -= count; + } + } + return (result[0] == 0) ? this : rebalance(); + } else if (cmp > 0) { + AvlNode initRight = right; + if (initRight == null) { + result[0] = 0; + return this; + } + + right = initRight.remove(comparator, e, count, result); + + if (result[0] > 0) { + if (count >= result[0]) { + this.distinctElements--; + this.totalCount -= result[0]; + } else { + this.totalCount -= count; + } + } + return rebalance(); + } + + // removing count from me! + result[0] = elemCount; + if (count >= elemCount) { + return deleteMe(); + } else { + this.elemCount -= count; + this.totalCount -= count; + return this; + } + } + + AvlNode setCount( + Comparator comparator, E e, int count, int[] result) { + int cmp = comparator.compare(e, elem); + if (cmp < 0) { + AvlNode initLeft = left; + if (initLeft == null) { + result[0] = 0; + return (count > 0) ? addLeftChild(e, count) : this; + } + + left = initLeft.setCount(comparator, e, count, result); + + if (count == 0 && result[0] != 0) { + this.distinctElements--; + } else if (count > 0 && result[0] == 0) { + this.distinctElements++; + } + + this.totalCount += count - result[0]; + return rebalance(); + } else if (cmp > 0) { + AvlNode initRight = right; + if (initRight == null) { + result[0] = 0; + return (count > 0) ? addRightChild(e, count) : this; + } + + right = initRight.setCount(comparator, e, count, result); + + if (count == 0 && result[0] != 0) { + this.distinctElements--; + } else if (count > 0 && result[0] == 0) { + this.distinctElements++; + } + + this.totalCount += count - result[0]; + return rebalance(); + } + + // setting my count + result[0] = elemCount; + if (count == 0) { + return deleteMe(); + } + this.totalCount += count - elemCount; + this.elemCount = count; + return this; + } + + AvlNode setCount( + Comparator comparator, + E e, + int expectedCount, + int newCount, + int[] result) { + int cmp = comparator.compare(e, elem); + if (cmp < 0) { + AvlNode initLeft = left; + if (initLeft == null) { + result[0] = 0; + if (expectedCount == 0 && newCount > 0) { + return addLeftChild(e, newCount); + } + return this; + } + + left = initLeft.setCount(comparator, e, expectedCount, newCount, result); + + if (result[0] == expectedCount) { + if (newCount == 0 && result[0] != 0) { + this.distinctElements--; + } else if (newCount > 0 && result[0] == 0) { + this.distinctElements++; + } + this.totalCount += newCount - result[0]; + } + return rebalance(); + } else if (cmp > 0) { + AvlNode initRight = right; + if (initRight == null) { + result[0] = 0; + if (expectedCount == 0 && newCount > 0) { + return addRightChild(e, newCount); + } + return this; + } + + right = initRight.setCount(comparator, e, expectedCount, newCount, result); + + if (result[0] == expectedCount) { + if (newCount == 0 && result[0] != 0) { + this.distinctElements--; + } else if (newCount > 0 && result[0] == 0) { + this.distinctElements++; + } + this.totalCount += newCount - result[0]; + } + return rebalance(); + } + + // setting my count + result[0] = elemCount; + if (expectedCount == elemCount) { + if (newCount == 0) { + return deleteMe(); + } + this.totalCount += newCount - elemCount; + this.elemCount = newCount; + } + return this; + } + + private AvlNode deleteMe() { + int oldElemCount = this.elemCount; + this.elemCount = 0; + successor(pred, succ); + if (left == null) { + return right; + } else if (right == null) { + return left; + } else if (left.height >= right.height) { + AvlNode newTop = pred; + // newTop is the maximum node in my left subtree + newTop.left = left.removeMax(newTop); + newTop.right = right; + newTop.distinctElements = distinctElements - 1; + newTop.totalCount = totalCount - oldElemCount; + return newTop.rebalance(); + } else { + AvlNode newTop = succ; + newTop.right = right.removeMin(newTop); + newTop.left = left; + newTop.distinctElements = distinctElements - 1; + newTop.totalCount = totalCount - oldElemCount; + return newTop.rebalance(); + } + } + + // Removes the minimum node from this subtree to be reused elsewhere + private AvlNode removeMin(AvlNode node) { + if (left == null) { + return right; + } else { + left = left.removeMin(node); + distinctElements--; + totalCount -= node.elemCount; + return rebalance(); + } + } + + // Removes the maximum node from this subtree to be reused elsewhere + private AvlNode removeMax(AvlNode node) { + if (right == null) { + return left; + } else { + right = right.removeMax(node); + distinctElements--; + totalCount -= node.elemCount; + return rebalance(); + } + } + + private void recomputeMultiset() { + this.distinctElements = + 1 + TreeMultiset.distinctElements(left) + TreeMultiset.distinctElements(right); + this.totalCount = elemCount + totalCount(left) + totalCount(right); + } + + private void recomputeHeight() { + this.height = 1 + Math.max(height(left), height(right)); + } + + private void recompute() { + recomputeMultiset(); + recomputeHeight(); + } + + private AvlNode rebalance() { + switch (balanceFactor()) { + case -2: + if (right.balanceFactor() > 0) { + right = right.rotateRight(); + } + return rotateLeft(); + case 2: + if (left.balanceFactor() < 0) { + left = left.rotateLeft(); + } + return rotateRight(); + default: + recomputeHeight(); + return this; + } + } + + private int balanceFactor() { + return height(left) - height(right); + } + + private AvlNode rotateLeft() { + checkState(right != null); + AvlNode newTop = right; + this.right = newTop.left; + newTop.left = this; + newTop.totalCount = this.totalCount; + newTop.distinctElements = this.distinctElements; + this.recompute(); + newTop.recomputeHeight(); + return newTop; + } + + private AvlNode rotateRight() { + checkState(left != null); + AvlNode newTop = left; + this.left = newTop.right; + newTop.right = this; + newTop.totalCount = this.totalCount; + newTop.distinctElements = this.distinctElements; + this.recompute(); + newTop.recomputeHeight(); + return newTop; + } + + private static long totalCount(AvlNode node) { + return (node == null) ? 0 : node.totalCount; + } + + private static int height(AvlNode node) { + return (node == null) ? 0 : node.height; + } + + + private AvlNode ceiling(Comparator comparator, E e) { + int cmp = comparator.compare(e, elem); + if (cmp < 0) { + return (left == null) ? this : MoreObjects.firstNonNull(left.ceiling(comparator, e), this); + } else if (cmp == 0) { + return this; + } else { + return (right == null) ? null : right.ceiling(comparator, e); + } + } + + + private AvlNode floor(Comparator comparator, E e) { + int cmp = comparator.compare(e, elem); + if (cmp > 0) { + return (right == null) ? this : MoreObjects.firstNonNull(right.floor(comparator, e), this); + } else if (cmp == 0) { + return this; + } else { + return (left == null) ? null : left.floor(comparator, e); + } + } + + E getElement() { + return elem; + } + + int getCount() { + return elemCount; + } + + @Override + public String toString() { + return Multisets.immutableEntry(getElement(), getCount()).toString(); + } + } + + private static void successor(AvlNode a, AvlNode b) { + a.succ = b; + b.pred = a; + } + + private static void successor(AvlNode a, AvlNode b, AvlNode c) { + successor(a, b); + successor(b, c); + } + + /* + * TODO(jlevy): Decide whether entrySet() should return entries with an equals() method that + * calls the comparator to compare the two keys. If that change is made, + * AbstractMultiset.equals() can simply check whether two multisets have equal entry sets. + */ + + /** + * @serialData the comparator, the number of distinct elements, the first element, its count, the + * second element, its count, and so on + */ + private void writeObject(ObjectOutputStream stream) throws IOException { + stream.defaultWriteObject(); + stream.writeObject(elementSet().comparator()); + Serialization.writeMultiset(this, stream); + } + + private void readObject(ObjectInputStream stream) throws IOException, ClassNotFoundException { + stream.defaultReadObject(); + @SuppressWarnings({"unchecked", "rawtypes"}) + // reading data stored by writeObject + Comparator comparator = (Comparator) stream.readObject(); + Serialization.getFieldSetter(AbstractSortedMultiset.class, "comparator").set(this, comparator); + Serialization.getFieldSetter(TreeMultiset.class, "range") + .set(this, GeneralRange.all(comparator)); + Serialization.getFieldSetter(TreeMultiset.class, "rootReference") + .set(this, new Reference>()); + AvlNode header = new AvlNode(null, 1); + Serialization.getFieldSetter(TreeMultiset.class, "header").set(this, header); + successor(header, header); + Serialization.populateMultiset(this, stream); + } + + private static final long serialVersionUID = 1; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeMap.java new file mode 100644 index 0000000000000..c135849f269e2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeMap.java @@ -0,0 +1,718 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.compose; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.in; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates.not; + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicate; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps.IteratorBasedAbstractMap; + +import java.util.AbstractMap; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.NoSuchElementException; +import java.util.Set; + + +/** + * An implementation of {@code RangeMap} based on a {@code TreeMap}, supporting all optional + * operations. + * + *

Like all {@code RangeMap} implementations, this supports neither null keys nor null values. + * + * @author Louis Wasserman + * @since 14.0 + */ + +@SuppressWarnings("rawtypes") +public final class TreeRangeMap implements RangeMap { + + private final NavigableMap, RangeMapEntry> entriesByLowerBound; + + public static TreeRangeMap create() { + return new TreeRangeMap<>(); + } + + private TreeRangeMap() { + this.entriesByLowerBound = Maps.newTreeMap(); + } + + @SuppressWarnings("rawtypes") + private static final class RangeMapEntry + extends AbstractMapEntry, V> { + private final Range range; + private final V value; + + RangeMapEntry(Cut lowerBound, Cut upperBound, V value) { + this(Range.create(lowerBound, upperBound), value); + } + + RangeMapEntry(Range range, V value) { + this.range = range; + this.value = value; + } + + @Override + public Range getKey() { + return range; + } + + @Override + public V getValue() { + return value; + } + + public boolean contains(K value) { + return range.contains(value); + } + + Cut getLowerBound() { + return range.lowerBound; + } + + Cut getUpperBound() { + return range.upperBound; + } + } + + @Override + + public V get(K key) { + Entry, V> entry = getEntry(key); + return (entry == null) ? null : entry.getValue(); + } + + @Override + + public Entry, V> getEntry(K key) { + Entry, RangeMapEntry> mapEntry = + entriesByLowerBound.floorEntry(Cut.belowValue(key)); + if (mapEntry != null && mapEntry.getValue().contains(key)) { + return mapEntry.getValue(); + } else { + return null; + } + } + + @Override + public void put(Range range, V value) { + // don't short-circuit if the range is empty - it may be between two ranges we can coalesce. + if (!range.isEmpty()) { + checkNotNull(value); + remove(range); + entriesByLowerBound.put(range.lowerBound, new RangeMapEntry(range, value)); + } + } + + @Override + public void putCoalescing(Range range, V value) { + if (entriesByLowerBound.isEmpty()) { + put(range, value); + return; + } + + Range coalescedRange = coalescedRange(range, checkNotNull(value)); + put(coalescedRange, value); + } + + /** Computes the coalesced range for the given range+value - does not mutate the map. */ + private Range coalescedRange(Range range, V value) { + Range coalescedRange = range; + Entry, RangeMapEntry> lowerEntry = + entriesByLowerBound.lowerEntry(range.lowerBound); + coalescedRange = coalesce(coalescedRange, value, lowerEntry); + + Entry, RangeMapEntry> higherEntry = + entriesByLowerBound.floorEntry(range.upperBound); + coalescedRange = coalesce(coalescedRange, value, higherEntry); + + return coalescedRange; + } + + /** Returns the range that spans the given range and entry, if the entry can be coalesced. */ + private static Range coalesce( + Range range, V value, Entry, RangeMapEntry> entry) { + if (entry != null + && entry.getValue().getKey().isConnected(range) + && entry.getValue().getValue().equals(value)) { + return range.span(entry.getValue().getKey()); + } + return range; + } + + @Override + public void putAll(RangeMap rangeMap) { + for (Entry, V> entry : rangeMap.asMapOfRanges().entrySet()) { + put(entry.getKey(), entry.getValue()); + } + } + + @Override + public void clear() { + entriesByLowerBound.clear(); + } + + @Override + public Range span() { + Entry, RangeMapEntry> firstEntry = entriesByLowerBound.firstEntry(); + Entry, RangeMapEntry> lastEntry = entriesByLowerBound.lastEntry(); + if (firstEntry == null) { + throw new NoSuchElementException(); + } + return Range.create( + firstEntry.getValue().getKey().lowerBound, lastEntry.getValue().getKey().upperBound); + } + + private void putRangeMapEntry(Cut lowerBound, Cut upperBound, V value) { + entriesByLowerBound.put(lowerBound, new RangeMapEntry(lowerBound, upperBound, value)); + } + + @Override + public void remove(Range rangeToRemove) { + if (rangeToRemove.isEmpty()) { + return; + } + + /* + * The comments for this method will use [ ] to indicate the bounds of rangeToRemove and ( ) to + * indicate the bounds of ranges in the range map. + */ + Entry, RangeMapEntry> mapEntryBelowToTruncate = + entriesByLowerBound.lowerEntry(rangeToRemove.lowerBound); + if (mapEntryBelowToTruncate != null) { + // we know ( [ + RangeMapEntry rangeMapEntry = mapEntryBelowToTruncate.getValue(); + if (rangeMapEntry.getUpperBound().compareTo(rangeToRemove.lowerBound) > 0) { + // we know ( [ ) + if (rangeMapEntry.getUpperBound().compareTo(rangeToRemove.upperBound) > 0) { + // we know ( [ ] ), so insert the range ] ) back into the map -- + // it's being split apart + putRangeMapEntry( + rangeToRemove.upperBound, + rangeMapEntry.getUpperBound(), + mapEntryBelowToTruncate.getValue().getValue()); + } + // overwrite mapEntryToTruncateBelow with a truncated range + putRangeMapEntry( + rangeMapEntry.getLowerBound(), + rangeToRemove.lowerBound, + mapEntryBelowToTruncate.getValue().getValue()); + } + } + + Entry, RangeMapEntry> mapEntryAboveToTruncate = + entriesByLowerBound.lowerEntry(rangeToRemove.upperBound); + if (mapEntryAboveToTruncate != null) { + // we know ( ] + RangeMapEntry rangeMapEntry = mapEntryAboveToTruncate.getValue(); + if (rangeMapEntry.getUpperBound().compareTo(rangeToRemove.upperBound) > 0) { + // we know ( ] ), and since we dealt with truncating below already, + // we know [ ( ] ) + putRangeMapEntry( + rangeToRemove.upperBound, + rangeMapEntry.getUpperBound(), + mapEntryAboveToTruncate.getValue().getValue()); + } + } + entriesByLowerBound.subMap(rangeToRemove.lowerBound, rangeToRemove.upperBound).clear(); + } + + @Override + public Map, V> asMapOfRanges() { + return new AsMapOfRanges(entriesByLowerBound.values()); + } + + @Override + public Map, V> asDescendingMapOfRanges() { + return new AsMapOfRanges(entriesByLowerBound.descendingMap().values()); + } + + private final class AsMapOfRanges extends IteratorBasedAbstractMap, V> { + + final Iterable, V>> entryIterable; + + @SuppressWarnings({"unchecked", "rawtypes"}) // it's safe to upcast iterables + AsMapOfRanges(Iterable> entryIterable) { + this.entryIterable = (Iterable) entryIterable; + } + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + public V get(Object key) { + if (key instanceof Range) { + Range range = (Range) key; + RangeMapEntry rangeMapEntry = entriesByLowerBound.get(range.lowerBound); + if (rangeMapEntry != null && rangeMapEntry.getKey().equals(range)) { + return rangeMapEntry.getValue(); + } + } + return null; + } + + @Override + public int size() { + return entriesByLowerBound.size(); + } + + @Override + Iterator, V>> entryIterator() { + return entryIterable.iterator(); + } + } + + @Override + public RangeMap subRangeMap(Range subRange) { + if (subRange.equals(Range.all())) { + return this; + } else { + return new SubRangeMap(subRange); + } + } + + @SuppressWarnings({"unchecked", "rawtypes"}) + private RangeMap emptySubRangeMap() { + return EMPTY_SUB_RANGE_MAP; + } + + private static final RangeMap EMPTY_SUB_RANGE_MAP = + new RangeMap() { + @Override + + public Object get(Comparable key) { + return null; + } + + @Override + + public Entry getEntry(Comparable key) { + return null; + } + + @Override + public Range span() { + throw new NoSuchElementException(); + } + + @Override + public void put(Range range, Object value) { + checkNotNull(range); + throw new IllegalArgumentException( + "Cannot insert range " + range + " into an empty subRangeMap"); + } + + @Override + public void putCoalescing(Range range, Object value) { + checkNotNull(range); + throw new IllegalArgumentException( + "Cannot insert range " + range + " into an empty subRangeMap"); + } + + @Override + public void putAll(RangeMap rangeMap) { + if (!rangeMap.asMapOfRanges().isEmpty()) { + throw new IllegalArgumentException( + "Cannot putAll(nonEmptyRangeMap) into an empty subRangeMap"); + } + } + + @Override + public void clear() {} + + @Override + public void remove(Range range) { + checkNotNull(range); + } + + @Override + public Map asMapOfRanges() { + return Collections.emptyMap(); + } + + @Override + public Map asDescendingMapOfRanges() { + return Collections.emptyMap(); + } + + @Override + public RangeMap subRangeMap(Range range) { + checkNotNull(range); + return this; + } + }; + + private class SubRangeMap implements RangeMap { + + private final Range subRange; + + SubRangeMap(Range subRange) { + this.subRange = subRange; + } + + @Override + + public V get(K key) { + return subRange.contains(key) ? TreeRangeMap.this.get(key) : null; + } + + @Override + + public Entry, V> getEntry(K key) { + if (subRange.contains(key)) { + Entry, V> entry = TreeRangeMap.this.getEntry(key); + if (entry != null) { + return Maps.immutableEntry(entry.getKey().intersection(subRange), entry.getValue()); + } + } + return null; + } + + @Override + public Range span() { + Cut lowerBound; + Entry, RangeMapEntry> lowerEntry = + entriesByLowerBound.floorEntry(subRange.lowerBound); + if (lowerEntry != null + && lowerEntry.getValue().getUpperBound().compareTo(subRange.lowerBound) > 0) { + lowerBound = subRange.lowerBound; + } else { + lowerBound = entriesByLowerBound.ceilingKey(subRange.lowerBound); + if (lowerBound == null || lowerBound.compareTo(subRange.upperBound) >= 0) { + throw new NoSuchElementException(); + } + } + + Cut upperBound; + Entry, RangeMapEntry> upperEntry = + entriesByLowerBound.lowerEntry(subRange.upperBound); + if (upperEntry == null) { + throw new NoSuchElementException(); + } else if (upperEntry.getValue().getUpperBound().compareTo(subRange.upperBound) >= 0) { + upperBound = subRange.upperBound; + } else { + upperBound = upperEntry.getValue().getUpperBound(); + } + return Range.create(lowerBound, upperBound); + } + + @Override + public void put(Range range, V value) { + checkArgument( + subRange.encloses(range), "Cannot put range %s into a subRangeMap(%s)", range, subRange); + TreeRangeMap.this.put(range, value); + } + + @Override + public void putCoalescing(Range range, V value) { + if (entriesByLowerBound.isEmpty() || range.isEmpty() || !subRange.encloses(range)) { + put(range, value); + return; + } + + Range coalescedRange = coalescedRange(range, checkNotNull(value)); + // only coalesce ranges within the subRange + put(coalescedRange.intersection(subRange), value); + } + + @Override + public void putAll(RangeMap rangeMap) { + if (rangeMap.asMapOfRanges().isEmpty()) { + return; + } + Range span = rangeMap.span(); + checkArgument( + subRange.encloses(span), + "Cannot putAll rangeMap with span %s into a subRangeMap(%s)", + span, + subRange); + TreeRangeMap.this.putAll(rangeMap); + } + + @Override + public void clear() { + TreeRangeMap.this.remove(subRange); + } + + @Override + public void remove(Range range) { + if (range.isConnected(subRange)) { + TreeRangeMap.this.remove(range.intersection(subRange)); + } + } + + @Override + public RangeMap subRangeMap(Range range) { + if (!range.isConnected(subRange)) { + return emptySubRangeMap(); + } else { + return TreeRangeMap.this.subRangeMap(range.intersection(subRange)); + } + } + + @Override + public Map, V> asMapOfRanges() { + return new SubRangeMapAsMap(); + } + + @Override + public Map, V> asDescendingMapOfRanges() { + return new SubRangeMapAsMap() { + + @Override + Iterator, V>> entryIterator() { + if (subRange.isEmpty()) { + return Iterators.emptyIterator(); + } + final Iterator> backingItr = + entriesByLowerBound + .headMap(subRange.upperBound, false) + .descendingMap() + .values() + .iterator(); + return new AbstractIterator, V>>() { + + @Override + protected Entry, V> computeNext() { + if (backingItr.hasNext()) { + RangeMapEntry entry = backingItr.next(); + if (entry.getUpperBound().compareTo(subRange.lowerBound) <= 0) { + return endOfData(); + } + return Maps.immutableEntry(entry.getKey().intersection(subRange), entry.getValue()); + } + return endOfData(); + } + }; + } + }; + } + + @Override + public boolean equals(Object o) { + if (o instanceof RangeMap) { + RangeMap rangeMap = (RangeMap) o; + return asMapOfRanges().equals(rangeMap.asMapOfRanges()); + } + return false; + } + + @Override + public int hashCode() { + return asMapOfRanges().hashCode(); + } + + @Override + public String toString() { + return asMapOfRanges().toString(); + } + + class SubRangeMapAsMap extends AbstractMap, V> { + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + public V get(Object key) { + try { + if (key instanceof Range) { + @SuppressWarnings({"unchecked", "rawtypes"}) // we catch ClassCastExceptions + Range r = (Range) key; + if (!subRange.encloses(r) || r.isEmpty()) { + return null; + } + RangeMapEntry candidate = null; + if (r.lowerBound.compareTo(subRange.lowerBound) == 0) { + // r could be truncated on the left + Entry, RangeMapEntry> entry = + entriesByLowerBound.floorEntry(r.lowerBound); + if (entry != null) { + candidate = entry.getValue(); + } + } else { + candidate = entriesByLowerBound.get(r.lowerBound); + } + + if (candidate != null + && candidate.getKey().isConnected(subRange) + && candidate.getKey().intersection(subRange).equals(r)) { + return candidate.getValue(); + } + } + } catch (ClassCastException e) { + return null; + } + return null; + } + + @Override + public V remove(Object key) { + V value = get(key); + if (value != null) { + @SuppressWarnings({"unchecked", "rawtypes"}) // it's definitely in the map, so safe + Range range = (Range) key; + TreeRangeMap.this.remove(range); + return value; + } + return null; + } + + @Override + public void clear() { + SubRangeMap.this.clear(); + } + + private boolean removeEntryIf(Predicate, V>> predicate) { + List> toRemove = Lists.newArrayList(); + for (Entry, V> entry : entrySet()) { + if (predicate.apply(entry)) { + toRemove.add(entry.getKey()); + } + } + for (Range range : toRemove) { + TreeRangeMap.this.remove(range); + } + return !toRemove.isEmpty(); + } + + @Override + public Set> keySet() { + return new Maps.KeySet, V>(SubRangeMapAsMap.this) { + @Override + public boolean remove(Object o) { + return SubRangeMapAsMap.this.remove(o) != null; + } + + @Override + public boolean retainAll(Collection c) { + return removeEntryIf(compose(not(in(c)), Maps.>keyFunction())); + } + }; + } + + @Override + public Set, V>> entrySet() { + return new Maps.EntrySet, V>() { + @Override + Map, V> map() { + return SubRangeMapAsMap.this; + } + + @Override + public Iterator, V>> iterator() { + return entryIterator(); + } + + @Override + public boolean retainAll(Collection c) { + return removeEntryIf(not(in(c))); + } + + @Override + public int size() { + return Iterators.size(iterator()); + } + + @Override + public boolean isEmpty() { + return !iterator().hasNext(); + } + }; + } + + Iterator, V>> entryIterator() { + if (subRange.isEmpty()) { + return Iterators.emptyIterator(); + } + Cut cutToStart = + MoreObjects.firstNonNull( + entriesByLowerBound.floorKey(subRange.lowerBound), subRange.lowerBound); + final Iterator> backingItr = + entriesByLowerBound.tailMap(cutToStart, true).values().iterator(); + return new AbstractIterator, V>>() { + + @Override + protected Entry, V> computeNext() { + while (backingItr.hasNext()) { + RangeMapEntry entry = backingItr.next(); + if (entry.getLowerBound().compareTo(subRange.upperBound) >= 0) { + return endOfData(); + } else if (entry.getUpperBound().compareTo(subRange.lowerBound) > 0) { + // this might not be true e.g. at the start of the iteration + return Maps.immutableEntry(entry.getKey().intersection(subRange), entry.getValue()); + } + } + return endOfData(); + } + }; + } + + @Override + public Collection values() { + return new Maps.Values, V>(this) { + @Override + public boolean removeAll(Collection c) { + return removeEntryIf(compose(in(c), Maps.valueFunction())); + } + + @Override + public boolean retainAll(Collection c) { + return removeEntryIf(compose(not(in(c)), Maps.valueFunction())); + } + }; + } + } + } + + @Override + public boolean equals(Object o) { + if (o instanceof RangeMap) { + RangeMap rangeMap = (RangeMap) o; + return asMapOfRanges().equals(rangeMap.asMapOfRanges()); + } + return false; + } + + @Override + public int hashCode() { + return asMapOfRanges().hashCode(); + } + + @Override + public String toString() { + return entriesByLowerBound.values().toString(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeSet.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeSet.java new file mode 100644 index 0000000000000..3ef9e5e90212e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/TreeRangeSet.java @@ -0,0 +1,934 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + + + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map.Entry; +import java.util.NavigableMap; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.TreeMap; + + + +/** + * An implementation of {@link RangeSet} backed by a {@link TreeMap}. + * + * @author Louis Wasserman + * @since 14.0 + */ + +public class TreeRangeSet> extends AbstractRangeSet + implements Serializable { + + final NavigableMap, Range> rangesByLowerBound; + + /** Creates an empty {@code TreeRangeSet} instance. */ + public static > TreeRangeSet create() { + return new TreeRangeSet(new TreeMap, Range>()); + } + + /** Returns a {@code TreeRangeSet} initialized with the ranges in the specified range set. */ + public static > TreeRangeSet create(RangeSet rangeSet) { + TreeRangeSet result = create(); + result.addAll(rangeSet); + return result; + } + + /** + * Returns a {@code TreeRangeSet} representing the union of the specified ranges. + * + *

This is the smallest {@code RangeSet} which encloses each of the specified ranges. An + * element will be contained in this {@code RangeSet} if and only if it is contained in at least + * one {@code Range} in {@code ranges}. + * + * @since 21.0 + */ + public static > TreeRangeSet create(Iterable> ranges) { + TreeRangeSet result = create(); + result.addAll(ranges); + return result; + } + + private TreeRangeSet(NavigableMap, Range> rangesByLowerCut) { + this.rangesByLowerBound = rangesByLowerCut; + } + + private transient Set> asRanges; + private transient Set> asDescendingSetOfRanges; + + @Override + public Set> asRanges() { + Set> result = asRanges; + return (result == null) ? asRanges = new AsRanges(rangesByLowerBound.values()) : result; + } + + @Override + public Set> asDescendingSetOfRanges() { + Set> result = asDescendingSetOfRanges; + return (result == null) + ? asDescendingSetOfRanges = new AsRanges(rangesByLowerBound.descendingMap().values()) + : result; + } + + final class AsRanges extends ForwardingCollection> implements Set> { + + final Collection> delegate; + + AsRanges(Collection> delegate) { + this.delegate = delegate; + } + + @Override + protected Collection> delegate() { + return delegate; + } + + @Override + public int hashCode() { + return Sets.hashCodeImpl(this); + } + + @Override + public boolean equals(Object o) { + return Sets.equalsImpl(this, o); + } + } + + @Override + + public Range rangeContaining(C value) { + checkNotNull(value); + Entry, Range> floorEntry = rangesByLowerBound.floorEntry(Cut.belowValue(value)); + if (floorEntry != null && floorEntry.getValue().contains(value)) { + return floorEntry.getValue(); + } else { + // TODO(kevinb): revisit this design choice + return null; + } + } + + @Override + public boolean intersects(Range range) { + checkNotNull(range); + Entry, Range> ceilingEntry = rangesByLowerBound.ceilingEntry(range.lowerBound); + if (ceilingEntry != null + && ceilingEntry.getValue().isConnected(range) + && !ceilingEntry.getValue().intersection(range).isEmpty()) { + return true; + } + Entry, Range> priorEntry = rangesByLowerBound.lowerEntry(range.lowerBound); + return priorEntry != null + && priorEntry.getValue().isConnected(range) + && !priorEntry.getValue().intersection(range).isEmpty(); + } + + @Override + public boolean encloses(Range range) { + checkNotNull(range); + Entry, Range> floorEntry = rangesByLowerBound.floorEntry(range.lowerBound); + return floorEntry != null && floorEntry.getValue().encloses(range); + } + + + private Range rangeEnclosing(Range range) { + checkNotNull(range); + Entry, Range> floorEntry = rangesByLowerBound.floorEntry(range.lowerBound); + return (floorEntry != null && floorEntry.getValue().encloses(range)) + ? floorEntry.getValue() + : null; + } + + @Override + public Range span() { + Entry, Range> firstEntry = rangesByLowerBound.firstEntry(); + Entry, Range> lastEntry = rangesByLowerBound.lastEntry(); + if (firstEntry == null) { + throw new NoSuchElementException(); + } + return Range.create(firstEntry.getValue().lowerBound, lastEntry.getValue().upperBound); + } + + @Override + public void add(Range rangeToAdd) { + checkNotNull(rangeToAdd); + + if (rangeToAdd.isEmpty()) { + return; + } + + // We will use { } to illustrate ranges currently in the range set, and < > + // to illustrate rangeToAdd. + Cut lbToAdd = rangeToAdd.lowerBound; + Cut ubToAdd = rangeToAdd.upperBound; + + Entry, Range> entryBelowLB = rangesByLowerBound.lowerEntry(lbToAdd); + if (entryBelowLB != null) { + // { < + Range rangeBelowLB = entryBelowLB.getValue(); + if (rangeBelowLB.upperBound.compareTo(lbToAdd) >= 0) { + // { < }, and we will need to coalesce + if (rangeBelowLB.upperBound.compareTo(ubToAdd) >= 0) { + // { < > } + ubToAdd = rangeBelowLB.upperBound; + /* + * TODO(cpovirk): can we just "return;" here? Or, can we remove this if() entirely? If + * not, add tests to demonstrate the problem with each approach + */ + } + lbToAdd = rangeBelowLB.lowerBound; + } + } + + Entry, Range> entryBelowUB = rangesByLowerBound.floorEntry(ubToAdd); + if (entryBelowUB != null) { + // { > + Range rangeBelowUB = entryBelowUB.getValue(); + if (rangeBelowUB.upperBound.compareTo(ubToAdd) >= 0) { + // { > }, and we need to coalesce + ubToAdd = rangeBelowUB.upperBound; + } + } + + // Remove ranges which are strictly enclosed. + rangesByLowerBound.subMap(lbToAdd, ubToAdd).clear(); + + replaceRangeWithSameLowerBound(Range.create(lbToAdd, ubToAdd)); + } + + @Override + public void remove(Range rangeToRemove) { + checkNotNull(rangeToRemove); + + if (rangeToRemove.isEmpty()) { + return; + } + + // We will use { } to illustrate ranges currently in the range set, and < > + // to illustrate rangeToRemove. + + Entry, Range> entryBelowLB = rangesByLowerBound.lowerEntry(rangeToRemove.lowerBound); + if (entryBelowLB != null) { + // { < + Range rangeBelowLB = entryBelowLB.getValue(); + if (rangeBelowLB.upperBound.compareTo(rangeToRemove.lowerBound) >= 0) { + // { < }, and we will need to subdivide + if (rangeToRemove.hasUpperBound() + && rangeBelowLB.upperBound.compareTo(rangeToRemove.upperBound) >= 0) { + // { < > } + replaceRangeWithSameLowerBound( + Range.create(rangeToRemove.upperBound, rangeBelowLB.upperBound)); + } + replaceRangeWithSameLowerBound( + Range.create(rangeBelowLB.lowerBound, rangeToRemove.lowerBound)); + } + } + + Entry, Range> entryBelowUB = rangesByLowerBound.floorEntry(rangeToRemove.upperBound); + if (entryBelowUB != null) { + // { > + Range rangeBelowUB = entryBelowUB.getValue(); + if (rangeToRemove.hasUpperBound() + && rangeBelowUB.upperBound.compareTo(rangeToRemove.upperBound) >= 0) { + // { > } + replaceRangeWithSameLowerBound( + Range.create(rangeToRemove.upperBound, rangeBelowUB.upperBound)); + } + } + + rangesByLowerBound.subMap(rangeToRemove.lowerBound, rangeToRemove.upperBound).clear(); + } + + private void replaceRangeWithSameLowerBound(Range range) { + if (range.isEmpty()) { + rangesByLowerBound.remove(range.lowerBound); + } else { + rangesByLowerBound.put(range.lowerBound, range); + } + } + + private transient RangeSet complement; + + @Override + public RangeSet complement() { + RangeSet result = complement; + return (result == null) ? complement = new Complement() : result; + } + + + static final class RangesByUpperBound> + extends AbstractNavigableMap, Range> { + private final NavigableMap, Range> rangesByLowerBound; + + /** + * upperBoundWindow represents the headMap/subMap/tailMap view of the entire "ranges by upper + * bound" map; it's a constraint on the *keys*, and does not affect the values. + */ + private final Range> upperBoundWindow; + + RangesByUpperBound(NavigableMap, Range> rangesByLowerBound) { + this.rangesByLowerBound = rangesByLowerBound; + this.upperBoundWindow = Range.all(); + } + + private RangesByUpperBound( + NavigableMap, Range> rangesByLowerBound, Range> upperBoundWindow) { + this.rangesByLowerBound = rangesByLowerBound; + this.upperBoundWindow = upperBoundWindow; + } + + private NavigableMap, Range> subMap(Range> window) { + if (window.isConnected(upperBoundWindow)) { + return new RangesByUpperBound(rangesByLowerBound, window.intersection(upperBoundWindow)); + } else { + return ImmutableSortedMap.of(); + } + } + + @Override + public NavigableMap, Range> subMap( + Cut fromKey, boolean fromInclusive, Cut toKey, boolean toInclusive) { + return subMap( + Range.range( + fromKey, BoundType.forBoolean(fromInclusive), + toKey, BoundType.forBoolean(toInclusive))); + } + + @Override + public NavigableMap, Range> headMap(Cut toKey, boolean inclusive) { + return subMap(Range.upTo(toKey, BoundType.forBoolean(inclusive))); + } + + @Override + public NavigableMap, Range> tailMap(Cut fromKey, boolean inclusive) { + return subMap(Range.downTo(fromKey, BoundType.forBoolean(inclusive))); + } + + @Override + public Comparator> comparator() { + return Ordering.>natural(); + } + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + public Range get(Object key) { + if (key instanceof Cut) { + try { + @SuppressWarnings({"unchecked", "rawtypes"}) // we catch CCEs + Cut cut = (Cut) key; + if (!upperBoundWindow.contains(cut)) { + return null; + } + Entry, Range> candidate = rangesByLowerBound.lowerEntry(cut); + if (candidate != null && candidate.getValue().upperBound.equals(cut)) { + return candidate.getValue(); + } + } catch (ClassCastException e) { + return null; + } + } + return null; + } + + @Override + Iterator, Range>> entryIterator() { + /* + * We want to start the iteration at the first range where the upper bound is in + * upperBoundWindow. + */ + final Iterator> backingItr; + if (!upperBoundWindow.hasLowerBound()) { + backingItr = rangesByLowerBound.values().iterator(); + } else { + Entry, Range> lowerEntry = + rangesByLowerBound.lowerEntry(upperBoundWindow.lowerEndpoint()); + if (lowerEntry == null) { + backingItr = rangesByLowerBound.values().iterator(); + } else if (upperBoundWindow.lowerBound.isLessThan(lowerEntry.getValue().upperBound)) { + backingItr = rangesByLowerBound.tailMap(lowerEntry.getKey(), true).values().iterator(); + } else { + backingItr = + rangesByLowerBound + .tailMap(upperBoundWindow.lowerEndpoint(), true) + .values() + .iterator(); + } + } + return new AbstractIterator, Range>>() { + @Override + protected Entry, Range> computeNext() { + if (!backingItr.hasNext()) { + return endOfData(); + } + Range range = backingItr.next(); + if (upperBoundWindow.upperBound.isLessThan(range.upperBound)) { + return endOfData(); + } else { + return Maps.immutableEntry(range.upperBound, range); + } + } + }; + } + + @Override + Iterator, Range>> descendingEntryIterator() { + Collection> candidates; + if (upperBoundWindow.hasUpperBound()) { + candidates = + rangesByLowerBound + .headMap(upperBoundWindow.upperEndpoint(), false) + .descendingMap() + .values(); + } else { + candidates = rangesByLowerBound.descendingMap().values(); + } + final PeekingIterator> backingItr = Iterators.peekingIterator(candidates.iterator()); + if (backingItr.hasNext() + && upperBoundWindow.upperBound.isLessThan(backingItr.peek().upperBound)) { + backingItr.next(); + } + return new AbstractIterator, Range>>() { + @Override + protected Entry, Range> computeNext() { + if (!backingItr.hasNext()) { + return endOfData(); + } + Range range = backingItr.next(); + return upperBoundWindow.lowerBound.isLessThan(range.upperBound) + ? Maps.immutableEntry(range.upperBound, range) + : endOfData(); + } + }; + } + + @Override + public int size() { + if (upperBoundWindow.equals(Range.all())) { + return rangesByLowerBound.size(); + } + return Iterators.size(entryIterator()); + } + + @Override + public boolean isEmpty() { + return upperBoundWindow.equals(Range.all()) + ? rangesByLowerBound.isEmpty() + : !entryIterator().hasNext(); + } + } + + private static final class ComplementRangesByLowerBound> + extends AbstractNavigableMap, Range> { + private final NavigableMap, Range> positiveRangesByLowerBound; + private final NavigableMap, Range> positiveRangesByUpperBound; + + /** + * complementLowerBoundWindow represents the headMap/subMap/tailMap view of the entire + * "complement ranges by lower bound" map; it's a constraint on the *keys*, and does not affect + * the values. + */ + private final Range> complementLowerBoundWindow; + + ComplementRangesByLowerBound(NavigableMap, Range> positiveRangesByLowerBound) { + this(positiveRangesByLowerBound, Range.>all()); + } + + private ComplementRangesByLowerBound( + NavigableMap, Range> positiveRangesByLowerBound, Range> window) { + this.positiveRangesByLowerBound = positiveRangesByLowerBound; + this.positiveRangesByUpperBound = new RangesByUpperBound(positiveRangesByLowerBound); + this.complementLowerBoundWindow = window; + } + + private NavigableMap, Range> subMap(Range> subWindow) { + if (!complementLowerBoundWindow.isConnected(subWindow)) { + return ImmutableSortedMap.of(); + } else { + subWindow = subWindow.intersection(complementLowerBoundWindow); + return new ComplementRangesByLowerBound(positiveRangesByLowerBound, subWindow); + } + } + + @Override + public NavigableMap, Range> subMap( + Cut fromKey, boolean fromInclusive, Cut toKey, boolean toInclusive) { + return subMap( + Range.range( + fromKey, BoundType.forBoolean(fromInclusive), + toKey, BoundType.forBoolean(toInclusive))); + } + + @Override + public NavigableMap, Range> headMap(Cut toKey, boolean inclusive) { + return subMap(Range.upTo(toKey, BoundType.forBoolean(inclusive))); + } + + @Override + public NavigableMap, Range> tailMap(Cut fromKey, boolean inclusive) { + return subMap(Range.downTo(fromKey, BoundType.forBoolean(inclusive))); + } + + @Override + public Comparator> comparator() { + return Ordering.>natural(); + } + + @Override + Iterator, Range>> entryIterator() { + /* + * firstComplementRangeLowerBound is the first complement range lower bound inside + * complementLowerBoundWindow. Complement range lower bounds are either positive range upper + * bounds, or Cut.belowAll(). + * + * positiveItr starts at the first positive range with lower bound greater than + * firstComplementRangeLowerBound. (Positive range lower bounds correspond to complement range + * upper bounds.) + */ + Collection> positiveRanges; + if (complementLowerBoundWindow.hasLowerBound()) { + positiveRanges = + positiveRangesByUpperBound + .tailMap( + complementLowerBoundWindow.lowerEndpoint(), + complementLowerBoundWindow.lowerBoundType() == BoundType.CLOSED) + .values(); + } else { + positiveRanges = positiveRangesByUpperBound.values(); + } + final PeekingIterator> positiveItr = + Iterators.peekingIterator(positiveRanges.iterator()); + final Cut firstComplementRangeLowerBound; + if (complementLowerBoundWindow.contains(Cut.belowAll()) + && (!positiveItr.hasNext() || positiveItr.peek().lowerBound != Cut.belowAll())) { + firstComplementRangeLowerBound = Cut.belowAll(); + } else if (positiveItr.hasNext()) { + firstComplementRangeLowerBound = positiveItr.next().upperBound; + } else { + return Iterators.emptyIterator(); + } + return new AbstractIterator, Range>>() { + Cut nextComplementRangeLowerBound = firstComplementRangeLowerBound; + + @Override + protected Entry, Range> computeNext() { + if (complementLowerBoundWindow.upperBound.isLessThan(nextComplementRangeLowerBound) + || nextComplementRangeLowerBound == Cut.aboveAll()) { + return endOfData(); + } + Range negativeRange; + if (positiveItr.hasNext()) { + Range positiveRange = positiveItr.next(); + negativeRange = Range.create(nextComplementRangeLowerBound, positiveRange.lowerBound); + nextComplementRangeLowerBound = positiveRange.upperBound; + } else { + negativeRange = Range.create(nextComplementRangeLowerBound, Cut.aboveAll()); + nextComplementRangeLowerBound = Cut.aboveAll(); + } + return Maps.immutableEntry(negativeRange.lowerBound, negativeRange); + } + }; + } + + @Override + Iterator, Range>> descendingEntryIterator() { + /* + * firstComplementRangeUpperBound is the upper bound of the last complement range with lower + * bound inside complementLowerBoundWindow. + * + * positiveItr starts at the first positive range with upper bound less than + * firstComplementRangeUpperBound. (Positive range upper bounds correspond to complement range + * lower bounds.) + */ + Cut startingPoint = + complementLowerBoundWindow.hasUpperBound() + ? complementLowerBoundWindow.upperEndpoint() + : Cut.aboveAll(); + boolean inclusive = + complementLowerBoundWindow.hasUpperBound() + && complementLowerBoundWindow.upperBoundType() == BoundType.CLOSED; + final PeekingIterator> positiveItr = + Iterators.peekingIterator( + positiveRangesByUpperBound + .headMap(startingPoint, inclusive) + .descendingMap() + .values() + .iterator()); + Cut cut; + if (positiveItr.hasNext()) { + cut = + (positiveItr.peek().upperBound == Cut.aboveAll()) + ? positiveItr.next().lowerBound + : positiveRangesByLowerBound.higherKey(positiveItr.peek().upperBound); + } else if (!complementLowerBoundWindow.contains(Cut.belowAll()) + || positiveRangesByLowerBound.containsKey(Cut.belowAll())) { + return Iterators.emptyIterator(); + } else { + cut = positiveRangesByLowerBound.higherKey(Cut.belowAll()); + } + final Cut firstComplementRangeUpperBound = + MoreObjects.firstNonNull(cut, Cut.aboveAll()); + return new AbstractIterator, Range>>() { + Cut nextComplementRangeUpperBound = firstComplementRangeUpperBound; + + @Override + protected Entry, Range> computeNext() { + if (nextComplementRangeUpperBound == Cut.belowAll()) { + return endOfData(); + } else if (positiveItr.hasNext()) { + Range positiveRange = positiveItr.next(); + Range negativeRange = + Range.create(positiveRange.upperBound, nextComplementRangeUpperBound); + nextComplementRangeUpperBound = positiveRange.lowerBound; + if (complementLowerBoundWindow.lowerBound.isLessThan(negativeRange.lowerBound)) { + return Maps.immutableEntry(negativeRange.lowerBound, negativeRange); + } + } else if (complementLowerBoundWindow.lowerBound.isLessThan(Cut.belowAll())) { + Range negativeRange = Range.create(Cut.belowAll(), nextComplementRangeUpperBound); + nextComplementRangeUpperBound = Cut.belowAll(); + return Maps.immutableEntry(Cut.belowAll(), negativeRange); + } + return endOfData(); + } + }; + } + + @Override + public int size() { + return Iterators.size(entryIterator()); + } + + @Override + + public Range get(Object key) { + if (key instanceof Cut) { + try { + @SuppressWarnings({"unchecked", "rawtypes"}) + Cut cut = (Cut) key; + // tailMap respects the current window + Entry, Range> firstEntry = tailMap(cut, true).firstEntry(); + if (firstEntry != null && firstEntry.getKey().equals(cut)) { + return firstEntry.getValue(); + } + } catch (ClassCastException e) { + return null; + } + } + return null; + } + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + } + + private final class Complement extends TreeRangeSet { + Complement() { + super(new ComplementRangesByLowerBound(TreeRangeSet.this.rangesByLowerBound)); + } + + @Override + public void add(Range rangeToAdd) { + TreeRangeSet.this.remove(rangeToAdd); + } + + @Override + public void remove(Range rangeToRemove) { + TreeRangeSet.this.add(rangeToRemove); + } + + @Override + public boolean contains(C value) { + return !TreeRangeSet.this.contains(value); + } + + @Override + public RangeSet complement() { + return TreeRangeSet.this; + } + } + + private static final class SubRangeSetRangesByLowerBound> + extends AbstractNavigableMap, Range> { + /** + * lowerBoundWindow is the headMap/subMap/tailMap view; it only restricts the keys, and does not + * affect the values. + */ + private final Range> lowerBoundWindow; + + /** + * restriction is the subRangeSet view; ranges are truncated to their intersection with + * restriction. + */ + private final Range restriction; + + private final NavigableMap, Range> rangesByLowerBound; + private final NavigableMap, Range> rangesByUpperBound; + + private SubRangeSetRangesByLowerBound( + Range> lowerBoundWindow, + Range restriction, + NavigableMap, Range> rangesByLowerBound) { + this.lowerBoundWindow = checkNotNull(lowerBoundWindow); + this.restriction = checkNotNull(restriction); + this.rangesByLowerBound = checkNotNull(rangesByLowerBound); + this.rangesByUpperBound = new RangesByUpperBound(rangesByLowerBound); + } + + private NavigableMap, Range> subMap(Range> window) { + if (!window.isConnected(lowerBoundWindow)) { + return ImmutableSortedMap.of(); + } else { + return new SubRangeSetRangesByLowerBound( + lowerBoundWindow.intersection(window), restriction, rangesByLowerBound); + } + } + + @Override + public NavigableMap, Range> subMap( + Cut fromKey, boolean fromInclusive, Cut toKey, boolean toInclusive) { + return subMap( + Range.range( + fromKey, + BoundType.forBoolean(fromInclusive), + toKey, + BoundType.forBoolean(toInclusive))); + } + + @Override + public NavigableMap, Range> headMap(Cut toKey, boolean inclusive) { + return subMap(Range.upTo(toKey, BoundType.forBoolean(inclusive))); + } + + @Override + public NavigableMap, Range> tailMap(Cut fromKey, boolean inclusive) { + return subMap(Range.downTo(fromKey, BoundType.forBoolean(inclusive))); + } + + @Override + public Comparator> comparator() { + return Ordering.>natural(); + } + + @Override + public boolean containsKey(Object key) { + return get(key) != null; + } + + @Override + + public Range get(Object key) { + if (key instanceof Cut) { + try { + @SuppressWarnings({"unchecked", "rawtypes"}) // we catch CCE's + Cut cut = (Cut) key; + if (!lowerBoundWindow.contains(cut) + || cut.compareTo(restriction.lowerBound) < 0 + || cut.compareTo(restriction.upperBound) >= 0) { + return null; + } else if (cut.equals(restriction.lowerBound)) { + // it might be present, truncated on the left + Range candidate = Maps.valueOrNull(rangesByLowerBound.floorEntry(cut)); + if (candidate != null && candidate.upperBound.compareTo(restriction.lowerBound) > 0) { + return candidate.intersection(restriction); + } + } else { + Range result = rangesByLowerBound.get(cut); + if (result != null) { + return result.intersection(restriction); + } + } + } catch (ClassCastException e) { + return null; + } + } + return null; + } + + @Override + Iterator, Range>> entryIterator() { + if (restriction.isEmpty()) { + return Iterators.emptyIterator(); + } + final Iterator> completeRangeItr; + if (lowerBoundWindow.upperBound.isLessThan(restriction.lowerBound)) { + return Iterators.emptyIterator(); + } else if (lowerBoundWindow.lowerBound.isLessThan(restriction.lowerBound)) { + // starts at the first range with upper bound strictly greater than restriction.lowerBound + completeRangeItr = + rangesByUpperBound.tailMap(restriction.lowerBound, false).values().iterator(); + } else { + // starts at the first range with lower bound above lowerBoundWindow.lowerBound + completeRangeItr = + rangesByLowerBound + .tailMap( + lowerBoundWindow.lowerBound.endpoint(), + lowerBoundWindow.lowerBoundType() == BoundType.CLOSED) + .values() + .iterator(); + } + final Cut> upperBoundOnLowerBounds = + Ordering.natural() + .min(lowerBoundWindow.upperBound, Cut.belowValue(restriction.upperBound)); + return new AbstractIterator, Range>>() { + @Override + protected Entry, Range> computeNext() { + if (!completeRangeItr.hasNext()) { + return endOfData(); + } + Range nextRange = completeRangeItr.next(); + if (upperBoundOnLowerBounds.isLessThan(nextRange.lowerBound)) { + return endOfData(); + } else { + nextRange = nextRange.intersection(restriction); + return Maps.immutableEntry(nextRange.lowerBound, nextRange); + } + } + }; + } + + @Override + Iterator, Range>> descendingEntryIterator() { + if (restriction.isEmpty()) { + return Iterators.emptyIterator(); + } + Cut> upperBoundOnLowerBounds = + Ordering.natural() + .min(lowerBoundWindow.upperBound, Cut.belowValue(restriction.upperBound)); + final Iterator> completeRangeItr = + rangesByLowerBound + .headMap( + upperBoundOnLowerBounds.endpoint(), + upperBoundOnLowerBounds.typeAsUpperBound() == BoundType.CLOSED) + .descendingMap() + .values() + .iterator(); + return new AbstractIterator, Range>>() { + @Override + protected Entry, Range> computeNext() { + if (!completeRangeItr.hasNext()) { + return endOfData(); + } + Range nextRange = completeRangeItr.next(); + if (restriction.lowerBound.compareTo(nextRange.upperBound) >= 0) { + return endOfData(); + } + nextRange = nextRange.intersection(restriction); + if (lowerBoundWindow.contains(nextRange.lowerBound)) { + return Maps.immutableEntry(nextRange.lowerBound, nextRange); + } else { + return endOfData(); + } + } + }; + } + + @Override + public int size() { + return Iterators.size(entryIterator()); + } + } + + @Override + public RangeSet subRangeSet(Range view) { + return view.equals(Range.all()) ? this : new SubRangeSet(view); + } + + private final class SubRangeSet extends TreeRangeSet { + private final Range restriction; + + SubRangeSet(Range restriction) { + super( + new SubRangeSetRangesByLowerBound( + Range.>all(), restriction, TreeRangeSet.this.rangesByLowerBound)); + this.restriction = restriction; + } + + @Override + public boolean encloses(Range range) { + if (!restriction.isEmpty() && restriction.encloses(range)) { + Range enclosing = TreeRangeSet.this.rangeEnclosing(range); + return enclosing != null && !enclosing.intersection(restriction).isEmpty(); + } + return false; + } + + @Override + + public Range rangeContaining(C value) { + if (!restriction.contains(value)) { + return null; + } + Range result = TreeRangeSet.this.rangeContaining(value); + return (result == null) ? null : result.intersection(restriction); + } + + @Override + public void add(Range rangeToAdd) { + checkArgument( + restriction.encloses(rangeToAdd), + "Cannot add range %s to subRangeSet(%s)", + rangeToAdd, + restriction); + super.add(rangeToAdd); + } + + @Override + public void remove(Range rangeToRemove) { + if (rangeToRemove.isConnected(restriction)) { + TreeRangeSet.this.remove(rangeToRemove.intersection(restriction)); + } + } + + @Override + public boolean contains(C value) { + return restriction.contains(value) && TreeRangeSet.this.contains(value); + } + + @Override + public void clear() { + TreeRangeSet.this.remove(restriction); + } + + @Override + public RangeSet subRangeSet(Range view) { + if (view.encloses(restriction)) { + return this; + } else if (view.isConnected(restriction)) { + return new SubRangeSet(restriction.intersection(view)); + } else { + return ImmutableRangeSet.of(); + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableIterator.java new file mode 100644 index 0000000000000..c5695c49dd563 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableIterator.java @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Iterator; + +/** + * An iterator that does not support {@link #remove}. + * + *

{@code UnmodifiableIterator} is used primarily in conjunction with implementations of {@link + * ImmutableCollection}, such as {@link ImmutableList}. You can, however, convert an existing + * iterator to an {@code UnmodifiableIterator} using {@link Iterators#unmodifiableIterator}. + * + * @author Jared Levy + * @since 2.0 + */ +public abstract class UnmodifiableIterator implements Iterator { + /** Constructor for use by subclasses. */ + protected UnmodifiableIterator() {} + + /** + * Guaranteed to throw an exception and leave the underlying data unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void remove() { + throw new UnsupportedOperationException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableListIterator.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableListIterator.java new file mode 100644 index 0000000000000..e7235e7c8ca21 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableListIterator.java @@ -0,0 +1,59 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.ListIterator; + +/** + * A list iterator that does not support {@link #remove}, {@link #add}, or {@link #set}. + * + * @since 7.0 + * @author Louis Wasserman + */ +public abstract class UnmodifiableListIterator extends UnmodifiableIterator + implements ListIterator { + /** Constructor for use by subclasses. */ + protected UnmodifiableListIterator() {} + + /** + * Guaranteed to throw an exception and leave the underlying data unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void add(E e) { + throw new UnsupportedOperationException(); + } + + /** + * Guaranteed to throw an exception and leave the underlying data unmodified. + * + * @throws UnsupportedOperationException always + * @deprecated Unsupported operation. + */ + @Deprecated + @Override + public final void set(E e) { + throw new UnsupportedOperationException(); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableSortedMultiset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableSortedMultiset.java new file mode 100644 index 0000000000000..8a6db25c13644 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UnmodifiableSortedMultiset.java @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2012 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multisets.UnmodifiableMultiset; + +import java.util.Comparator; +import java.util.NavigableSet; + + +/** + * Implementation of {@link Multisets#unmodifiableSortedMultiset(SortedMultiset)}, split out into + * its own file so it can be GWT emulated (to deal with the differing elementSet() types in GWT and + * non-GWT). + * + * @author Louis Wasserman + */ +final class UnmodifiableSortedMultiset extends UnmodifiableMultiset + implements SortedMultiset { + UnmodifiableSortedMultiset(SortedMultiset delegate) { + super(delegate); + } + + @Override + protected SortedMultiset delegate() { + return (SortedMultiset) super.delegate(); + } + + @Override + public Comparator comparator() { + return delegate().comparator(); + } + + @Override + NavigableSet createElementSet() { + return Sets.unmodifiableNavigableSet(delegate().elementSet()); + } + + @Override + public NavigableSet elementSet() { + return (NavigableSet) super.elementSet(); + } + + private transient UnmodifiableSortedMultiset descendingMultiset; + + @Override + public SortedMultiset descendingMultiset() { + UnmodifiableSortedMultiset result = descendingMultiset; + if (result == null) { + result = new UnmodifiableSortedMultiset(delegate().descendingMultiset()); + result.descendingMultiset = this; + return descendingMultiset = result; + } + return result; + } + + @Override + public Entry firstEntry() { + return delegate().firstEntry(); + } + + @Override + public Entry lastEntry() { + return delegate().lastEntry(); + } + + @Override + public Entry pollFirstEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public Entry pollLastEntry() { + throw new UnsupportedOperationException(); + } + + @Override + public SortedMultiset headMultiset(E upperBound, BoundType boundType) { + return Multisets.unmodifiableSortedMultiset(delegate().headMultiset(upperBound, boundType)); + } + + @Override + public SortedMultiset subMultiset( + E lowerBound, BoundType lowerBoundType, E upperBound, BoundType upperBoundType) { + return Multisets.unmodifiableSortedMultiset( + delegate().subMultiset(lowerBound, lowerBoundType, upperBound, upperBoundType)); + } + + @Override + public SortedMultiset tailMultiset(E lowerBound, BoundType boundType) { + return Multisets.unmodifiableSortedMultiset(delegate().tailMultiset(lowerBound, boundType)); + } + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UsingToStringOrdering.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UsingToStringOrdering.java new file mode 100644 index 0000000000000..dda523cbc41ad --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/UsingToStringOrdering.java @@ -0,0 +1,47 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.io.Serializable; + +/** An ordering that uses the natural order of the string representation of the values. */ +final class UsingToStringOrdering extends Ordering implements Serializable { + static final UsingToStringOrdering INSTANCE = new UsingToStringOrdering(); + + @Override + public int compare(Object left, Object right) { + return left.toString().compareTo(right.toString()); + } + + // preserve singleton-ness, so equals() and hashCode() work correctly + private Object readResolve() { + return INSTANCE; + } + + @Override + public String toString() { + return "Ordering.usingToString()"; + } + + private UsingToStringOrdering() {} + + private static final long serialVersionUID = 0; +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/WellBehavedMap.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/WellBehavedMap.java new file mode 100644 index 0000000000000..26fe017dd8394 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/WellBehavedMap.java @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; + +import java.util.Iterator; +import java.util.Map; +import java.util.Set; + + +/** + * Workaround for EnumMap + * bug. If you want to pass an {@code EnumMap}, with the intention of using its {@code + * entrySet()} method, you should wrap the {@code EnumMap} in this class instead. + * + *

This class is not thread-safe even if the underlying map is. + * + * @author Dimitris Andreou + */ +final class WellBehavedMap extends ForwardingMap { + private final Map delegate; + private Set> entrySet; + + private WellBehavedMap(Map delegate) { + this.delegate = delegate; + } + + /** + * Wraps the given map into a {@code WellBehavedEntriesMap}, which intercepts its {@code + * entrySet()} method by taking the {@code Set keySet()} and transforming it to {@code + * Set>}. All other invocations are delegated as-is. + */ + static WellBehavedMap wrap(Map delegate) { + return new WellBehavedMap<>(delegate); + } + + @Override + protected Map delegate() { + return delegate; + } + + @Override + public Set> entrySet() { + Set> es = entrySet; + if (es != null) { + return es; + } + return entrySet = new EntrySet(); + } + + private final class EntrySet extends Maps.EntrySet { + @Override + Map map() { + return WellBehavedMap.this; + } + + @Override + public Iterator> iterator() { + return new TransformedIterator>(keySet().iterator()) { + @Override + Entry transform(final K key) { + return new AbstractMapEntry() { + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return get(key); + } + + @Override + public V setValue(V value) { + return put(key, value); + } + }; + } + }; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/package-info.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/package-info.java new file mode 100644 index 0000000000000..21d5bf3a173cd --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/collect/package-info.java @@ -0,0 +1,218 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/** + * This package contains generic collection interfaces and implementations, and other utilities for + * working with collections. It is a part of the open-source Guava library. + * + *

Collection Types

+ * + *
+ *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.BiMap} + *
An extension of {@link java.util.Map} that guarantees the uniqueness of its values as well + * as that of its keys. This is sometimes called an "invertible map," since the restriction on + * values enables it to support an {@linkplain com.azure.cosmos.kafka.connect.implementation.guava25.collect.BiMap#inverse inverse + * view} -- which is another instance of {@code BiMap}. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multiset} + *
An extension of {@link java.util.Collection} that may contain duplicate values like a + * {@link java.util.List}, yet has order-independent equality like a {@link java.util.Set}. + * One typical use for a multiset is to represent a histogram. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap} + *
A new type, which is similar to {@link java.util.Map}, but may contain multiple entries + * with the same key. Some behaviors of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap} are left + * unspecified and are provided only by the subtypes mentioned below. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ListMultimap} + *
An extension of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap} which permits duplicate entries, + * supports random access of values for a particular key, and has partially order-dependent + * equality as defined by {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ListMultimap#equals(Object)}. + * {@code ListMultimap} takes its name from the fact that the {@linkplain + * com.azure.cosmos.kafka.connect.implementation.guava25.collect.ListMultimap#get collection of values} associated with a given + * key fulfills the {@link java.util.List} contract. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.SetMultimap} + *
An extension of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap} which has order-independent + * equality and does not allow duplicate entries; that is, while a key may appear twice in a + * {@code SetMultimap}, each must map to a different value. {@code SetMultimap} takes its name + * from the fact that the {@linkplain com.azure.cosmos.kafka.connect.implementation.guava25.collect.SetMultimap#get collection of + * values} associated with a given key fulfills the {@link java.util.Set} contract. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedSetMultimap} + *
An extension of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.SetMultimap} for which the {@linkplain + * com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedSetMultimap#get collection values} associated with a given + * key is a {@link java.util.SortedSet}. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Table} + *
A new type, which is similar to {@link java.util.Map}, but which indexes its values by an + * ordered pair of keys, a row key and column key. + *
{@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ClassToInstanceMap} + *
An extension of {@link java.util.Map} that associates a raw type with an instance of that + * type. + *
+ * + *

Collection Implementations

+ * + *

of {@link java.util.List}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList} + *
+ * + *

of {@link java.util.Set}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSet} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSortedSet} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ContiguousSet} (see {@code Range}) + *
+ * + *

of {@link java.util.Map}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSortedMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapMaker} + *
+ * + *

of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.BiMap}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableBiMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.HashBiMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.EnumBiMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.EnumHashBiMap} + *
+ * + *

of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multiset}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.HashMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.LinkedHashMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.TreeMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.EnumMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ConcurrentHashMultiset} + *
+ * + *

of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableListMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSetMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ArrayListMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.HashMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.TreeMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.LinkedHashMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.LinkedListMultimap} + *
+ * + *

of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Table}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableTable} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ArrayTable} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.HashBasedTable} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.TreeBasedTable} + *
+ * + *

of {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ClassToInstanceMap}

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableClassToInstanceMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.MutableClassToInstanceMap} + *
+ * + *

Classes of static utility methods

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Collections2} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Iterators} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Iterables} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Queues} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Sets} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multisets} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimaps} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Tables} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ObjectArrays} + *
+ * + *

Comparison

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Ordering} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ComparisonChain} + *
+ * + *

Abstract implementations

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.AbstractIterator} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.AbstractSequentialIterator} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableCollection} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.UnmodifiableIterator} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.UnmodifiableListIterator} + *
+ * + *

Ranges

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Range} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.RangeMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.DiscreteDomain} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ContiguousSet} + *
+ * + *

Other

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Interner}, {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.Interners} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.MapDifference}, {@link + * com.azure.cosmos.kafka.connect.implementation.guava25.collect.SortedMapDifference} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.MinMaxPriorityQueue} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.PeekingIterator} + *
+ * + *

Forwarding collections

+ * + *
    + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingCollection} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingConcurrentMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingIterator} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingList} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingListIterator} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingListMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingMapEntry} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingNavigableMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingNavigableSet} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingObject} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingQueue} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingSet} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingSetMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingSortedMap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingSortedMultiset} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingSortedSet} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingSortedSetMultimap} + *
  • {@link com.azure.cosmos.kafka.connect.implementation.guava25.collect.ForwardingTable} + *
+ */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.collect; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/IntMath.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/IntMath.java new file mode 100644 index 0000000000000..036d788f79c2d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/IntMath.java @@ -0,0 +1,724 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.math; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Ints; + +import java.math.BigInteger; +import java.math.RoundingMode; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkNoOverflow; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkNonNegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkPositive; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkRoundingUnnecessary; +import static java.lang.Math.abs; +import static java.lang.Math.min; +import static java.math.RoundingMode.HALF_EVEN; +import static java.math.RoundingMode.HALF_UP; + +/** + * A class for arithmetic on values of type {@code int}. Where possible, methods are defined and + * named analogously to their {@code BigInteger} counterparts. + * + *

The implementations of many methods in this class are based on material from Henry S. Warren, + * Jr.'s Hacker's Delight, (Addison Wesley, 2002). + * + *

Similar functionality for {@code long} and for {@link BigInteger} can be found in {@link + * LongMath} and {@link BigIntegerMath} respectively. For other common operations on {@code int} + * values, see {@link Ints}. + * + * @author Louis Wasserman + * @since 11.0 + */ +public final class IntMath { + // NOTE: Whenever both tests are cheap and functional, it's faster to use &, | instead of &&, || + + static final int MAX_SIGNED_POWER_OF_TWO = 1 << (Integer.SIZE - 2); + + /** + * Returns the smallest power of two greater than or equal to {@code x}. This is equivalent to + * {@code checkedPow(2, log2(x, CEILING))}. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @throws ArithmeticException of the next-higher power of two is not representable as an {@code + * int}, i.e. when {@code x > 2^30} + * @since 20.0 + */ + + public static int ceilingPowerOfTwo(int x) { + checkPositive("x", x); + if (x > MAX_SIGNED_POWER_OF_TWO) { + throw new ArithmeticException("ceilingPowerOfTwo(" + x + ") not representable as an int"); + } + return 1 << -Integer.numberOfLeadingZeros(x - 1); + } + + /** + * Returns the largest power of two less than or equal to {@code x}. This is equivalent to {@code + * checkedPow(2, log2(x, FLOOR))}. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @since 20.0 + */ + + public static int floorPowerOfTwo(int x) { + checkPositive("x", x); + return Integer.highestOneBit(x); + } + + /** + * Returns {@code true} if {@code x} represents a power of two. + * + *

This differs from {@code Integer.bitCount(x) == 1}, because {@code + * Integer.bitCount(Integer.MIN_VALUE) == 1}, but {@link Integer#MIN_VALUE} is not a power of two. + */ + public static boolean isPowerOfTwo(int x) { + return x > 0 & (x & (x - 1)) == 0; + } + + /** + * Returns 1 if {@code x < y} as unsigned integers, and 0 otherwise. Assumes that x - y fits into + * a signed int. The implementation is branch-free, and benchmarks suggest it is measurably (if + * narrowly) faster than the straightforward ternary expression. + */ + + static int lessThanBranchFree(int x, int y) { + // The double negation is optimized away by normal Java, but is necessary for GWT + // to make sure bit twiddling works as expected. + return ~~(x - y) >>> (Integer.SIZE - 1); + } + + /** + * Returns the base-2 logarithm of {@code x}, rounded according to the specified rounding mode. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @throws ArithmeticException if {@code mode} is {@link RoundingMode#UNNECESSARY} and {@code x} + * is not a power of two + */ + @SuppressWarnings("fallthrough") + // TODO(kevinb): remove after this warning is disabled globally + public static int log2(int x, RoundingMode mode) { + checkPositive("x", x); + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(isPowerOfTwo(x)); + // fall through + case DOWN: + case FLOOR: + return (Integer.SIZE - 1) - Integer.numberOfLeadingZeros(x); + + case UP: + case CEILING: + return Integer.SIZE - Integer.numberOfLeadingZeros(x - 1); + + case HALF_DOWN: + case HALF_UP: + case HALF_EVEN: + // Since sqrt(2) is irrational, log2(x) - logFloor cannot be exactly 0.5 + int leadingZeros = Integer.numberOfLeadingZeros(x); + int cmp = MAX_POWER_OF_SQRT2_UNSIGNED >>> leadingZeros; + // floor(2^(logFloor + 0.5)) + int logFloor = (Integer.SIZE - 1) - leadingZeros; + return logFloor + lessThanBranchFree(cmp, x); + + default: + throw new AssertionError(); + } + } + + /** The biggest half power of two that can fit in an unsigned int. */ + static final int MAX_POWER_OF_SQRT2_UNSIGNED = 0xB504F333; + + /** + * Returns the base-10 logarithm of {@code x}, rounded according to the specified rounding mode. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @throws ArithmeticException if {@code mode} is {@link RoundingMode#UNNECESSARY} and {@code x} + * is not a power of ten + */ + @SuppressWarnings("fallthrough") + public static int log10(int x, RoundingMode mode) { + checkPositive("x", x); + int logFloor = log10Floor(x); + int floorPow = powersOf10[logFloor]; + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(x == floorPow); + // fall through + case FLOOR: + case DOWN: + return logFloor; + case CEILING: + case UP: + return logFloor + lessThanBranchFree(floorPow, x); + case HALF_DOWN: + case HALF_UP: + case HALF_EVEN: + // sqrt(10) is irrational, so log10(x) - logFloor is never exactly 0.5 + return logFloor + lessThanBranchFree(halfPowersOf10[logFloor], x); + default: + throw new AssertionError(); + } + } + + private static int log10Floor(int x) { + /* + * Based on Hacker's Delight Fig. 11-5, the two-table-lookup, branch-free implementation. + * + * The key idea is that based on the number of leading zeros (equivalently, floor(log2(x))), we + * can narrow the possible floor(log10(x)) values to two. For example, if floor(log2(x)) is 6, + * then 64 <= x < 128, so floor(log10(x)) is either 1 or 2. + */ + int y = maxLog10ForLeadingZeros[Integer.numberOfLeadingZeros(x)]; + /* + * y is the higher of the two possible values of floor(log10(x)). If x < 10^y, then we want the + * lower of the two possible values, or y - 1, otherwise, we want y. + */ + return y - lessThanBranchFree(x, powersOf10[y]); + } + + // maxLog10ForLeadingZeros[i] == floor(log10(2^(Long.SIZE - i))) + + static final byte[] maxLog10ForLeadingZeros = { + 9, 9, 9, 8, 8, 8, 7, 7, 7, 6, 6, 6, 6, 5, 5, 5, 4, 4, 4, 3, 3, 3, 3, 2, 2, 2, 1, 1, 1, 0, 0, 0, + 0 + }; + + + static final int[] powersOf10 = { + 1, 10, 100, 1000, 10000, 100000, 1000000, 10000000, 100000000, 1000000000 + }; + + // halfPowersOf10[i] = largest int less than 10^(i + 0.5) + + static final int[] halfPowersOf10 = { + 3, 31, 316, 3162, 31622, 316227, 3162277, 31622776, 316227766, Integer.MAX_VALUE + }; + + /** + * Returns {@code b} to the {@code k}th power. Even if the result overflows, it will be equal to + * {@code BigInteger.valueOf(b).pow(k).intValue()}. This implementation runs in {@code O(log k)} + * time. + * + *

Compare {@link #checkedPow}, which throws an {@link ArithmeticException} upon overflow. + * + * @throws IllegalArgumentException if {@code k < 0} + */ + public static int pow(int b, int k) { + checkNonNegative("exponent", k); + switch (b) { + case 0: + return (k == 0) ? 1 : 0; + case 1: + return 1; + case (-1): + return ((k & 1) == 0) ? 1 : -1; + case 2: + return (k < Integer.SIZE) ? (1 << k) : 0; + case (-2): + if (k < Integer.SIZE) { + return ((k & 1) == 0) ? (1 << k) : -(1 << k); + } else { + return 0; + } + default: + // continue below to handle the general case + } + for (int accum = 1; ; k >>= 1) { + switch (k) { + case 0: + return accum; + case 1: + return b * accum; + default: + accum *= ((k & 1) == 0) ? 1 : b; + b *= b; + } + } + } + + /** + * Returns the square root of {@code x}, rounded with the specified rounding mode. + * + * @throws IllegalArgumentException if {@code x < 0} + * @throws ArithmeticException if {@code mode} is {@link RoundingMode#UNNECESSARY} and {@code + * sqrt(x)} is not an integer + */ + @SuppressWarnings("fallthrough") + public static int sqrt(int x, RoundingMode mode) { + checkNonNegative("x", x); + int sqrtFloor = sqrtFloor(x); + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(sqrtFloor * sqrtFloor == x); // fall through + case FLOOR: + case DOWN: + return sqrtFloor; + case CEILING: + case UP: + return sqrtFloor + lessThanBranchFree(sqrtFloor * sqrtFloor, x); + case HALF_DOWN: + case HALF_UP: + case HALF_EVEN: + int halfSquare = sqrtFloor * sqrtFloor + sqrtFloor; + /* + * We wish to test whether or not x <= (sqrtFloor + 0.5)^2 = halfSquare + 0.25. Since both x + * and halfSquare are integers, this is equivalent to testing whether or not x <= + * halfSquare. (We have to deal with overflow, though.) + * + * If we treat halfSquare as an unsigned int, we know that + * sqrtFloor^2 <= x < (sqrtFloor + 1)^2 + * halfSquare - sqrtFloor <= x < halfSquare + sqrtFloor + 1 + * so |x - halfSquare| <= sqrtFloor. Therefore, it's safe to treat x - halfSquare as a + * signed int, so lessThanBranchFree is safe for use. + */ + return sqrtFloor + lessThanBranchFree(halfSquare, x); + default: + throw new AssertionError(); + } + } + + private static int sqrtFloor(int x) { + // There is no loss of precision in converting an int to a double, according to + // http://java.sun.com/docs/books/jls/third_edition/html/conversions.html#5.1.2 + return (int) Math.sqrt(x); + } + + /** + * Returns the result of dividing {@code p} by {@code q}, rounding using the specified {@code + * RoundingMode}. + * + * @throws ArithmeticException if {@code q == 0}, or if {@code mode == UNNECESSARY} and {@code a} + * is not an integer multiple of {@code b} + */ + @SuppressWarnings("fallthrough") + public static int divide(int p, int q, RoundingMode mode) { + checkNotNull(mode); + if (q == 0) { + throw new ArithmeticException("/ by zero"); // for GWT + } + int div = p / q; + int rem = p - q * div; // equal to p % q + + if (rem == 0) { + return div; + } + + /* + * Normal Java division rounds towards 0, consistently with RoundingMode.DOWN. We just have to + * deal with the cases where rounding towards 0 is wrong, which typically depends on the sign of + * p / q. + * + * signum is 1 if p and q are both nonnegative or both negative, and -1 otherwise. + */ + int signum = 1 | ((p ^ q) >> (Integer.SIZE - 1)); + boolean increment; + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(rem == 0); + // fall through + case DOWN: + increment = false; + break; + case UP: + increment = true; + break; + case CEILING: + increment = signum > 0; + break; + case FLOOR: + increment = signum < 0; + break; + case HALF_EVEN: + case HALF_DOWN: + case HALF_UP: + int absRem = abs(rem); + int cmpRemToHalfDivisor = absRem - (abs(q) - absRem); + // subtracting two nonnegative ints can't overflow + // cmpRemToHalfDivisor has the same sign as compare(abs(rem), abs(q) / 2). + if (cmpRemToHalfDivisor == 0) { // exactly on the half mark + increment = (mode == HALF_UP || (mode == HALF_EVEN & (div & 1) != 0)); + } else { + increment = cmpRemToHalfDivisor > 0; // closer to the UP value + } + break; + default: + throw new AssertionError(); + } + return increment ? div + signum : div; + } + + /** + * Returns {@code x mod m}, a non-negative value less than {@code m}. This differs from {@code x % + * m}, which might be negative. + * + *

For example: + * + *

{@code
+   * mod(7, 4) == 3
+   * mod(-7, 4) == 1
+   * mod(-1, 4) == 3
+   * mod(-8, 4) == 0
+   * mod(8, 4) == 0
+   * }
+ * + * @throws ArithmeticException if {@code m <= 0} + * @see + * Remainder Operator + */ + public static int mod(int x, int m) { + if (m <= 0) { + throw new ArithmeticException("Modulus " + m + " must be > 0"); + } + int result = x % m; + return (result >= 0) ? result : result + m; + } + + /** + * Returns the greatest common divisor of {@code a, b}. Returns {@code 0} if {@code a == 0 && b == + * 0}. + * + * @throws IllegalArgumentException if {@code a < 0} or {@code b < 0} + */ + public static int gcd(int a, int b) { + /* + * The reason we require both arguments to be >= 0 is because otherwise, what do you return on + * gcd(0, Integer.MIN_VALUE)? BigInteger.gcd would return positive 2^31, but positive 2^31 isn't + * an int. + */ + checkNonNegative("a", a); + checkNonNegative("b", b); + if (a == 0) { + // 0 % b == 0, so b divides a, but the converse doesn't hold. + // BigInteger.gcd is consistent with this decision. + return b; + } else if (b == 0) { + return a; // similar logic + } + /* + * Uses the binary GCD algorithm; see http://en.wikipedia.org/wiki/Binary_GCD_algorithm. This is + * >40% faster than the Euclidean algorithm in benchmarks. + */ + int aTwos = Integer.numberOfTrailingZeros(a); + a >>= aTwos; // divide out all 2s + int bTwos = Integer.numberOfTrailingZeros(b); + b >>= bTwos; // divide out all 2s + while (a != b) { // both a, b are odd + // The key to the binary GCD algorithm is as follows: + // Both a and b are odd. Assume a > b; then gcd(a - b, b) = gcd(a, b). + // But in gcd(a - b, b), a - b is even and b is odd, so we can divide out powers of two. + + // We bend over backwards to avoid branching, adapting a technique from + // http://graphics.stanford.edu/~seander/bithacks.html#IntegerMinOrMax + + int delta = a - b; // can't overflow, since a and b are nonnegative + + int minDeltaOrZero = delta & (delta >> (Integer.SIZE - 1)); + // equivalent to Math.min(delta, 0) + + a = delta - minDeltaOrZero - minDeltaOrZero; // sets a to Math.abs(a - b) + // a is now nonnegative and even + + b += minDeltaOrZero; // sets b to min(old a, b) + a >>= Integer.numberOfTrailingZeros(a); // divide out all 2s, since 2 doesn't divide b + } + return a << min(aTwos, bTwos); + } + + /** + * Returns the sum of {@code a} and {@code b}, provided it does not overflow. + * + * @throws ArithmeticException if {@code a + b} overflows in signed {@code int} arithmetic + */ + public static int checkedAdd(int a, int b) { + long result = (long) a + b; + checkNoOverflow(result == (int) result, "checkedAdd", a, b); + return (int) result; + } + + /** + * Returns the difference of {@code a} and {@code b}, provided it does not overflow. + * + * @throws ArithmeticException if {@code a - b} overflows in signed {@code int} arithmetic + */ + public static int checkedSubtract(int a, int b) { + long result = (long) a - b; + checkNoOverflow(result == (int) result, "checkedSubtract", a, b); + return (int) result; + } + + /** + * Returns the product of {@code a} and {@code b}, provided it does not overflow. + * + * @throws ArithmeticException if {@code a * b} overflows in signed {@code int} arithmetic + */ + public static int checkedMultiply(int a, int b) { + long result = (long) a * b; + checkNoOverflow(result == (int) result, "checkedMultiply", a, b); + return (int) result; + } + + /** + * Returns the {@code b} to the {@code k}th power, provided it does not overflow. + * + *

{@link #pow} may be faster, but does not check for overflow. + * + * @throws ArithmeticException if {@code b} to the {@code k}th power overflows in signed {@code + * int} arithmetic + */ + public static int checkedPow(int b, int k) { + checkNonNegative("exponent", k); + switch (b) { + case 0: + return (k == 0) ? 1 : 0; + case 1: + return 1; + case (-1): + return ((k & 1) == 0) ? 1 : -1; + case 2: + checkNoOverflow(k < Integer.SIZE - 1, "checkedPow", b, k); + return 1 << k; + case (-2): + checkNoOverflow(k < Integer.SIZE, "checkedPow", b, k); + return ((k & 1) == 0) ? 1 << k : -1 << k; + default: + // continue below to handle the general case + } + int accum = 1; + while (true) { + switch (k) { + case 0: + return accum; + case 1: + return checkedMultiply(accum, b); + default: + if ((k & 1) != 0) { + accum = checkedMultiply(accum, b); + } + k >>= 1; + if (k > 0) { + checkNoOverflow(-FLOOR_SQRT_MAX_INT <= b & b <= FLOOR_SQRT_MAX_INT, "checkedPow", b, k); + b *= b; + } + } + } + } + + /** + * Returns the sum of {@code a} and {@code b} unless it would overflow or underflow in which case + * {@code Integer.MAX_VALUE} or {@code Integer.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static int saturatedAdd(int a, int b) { + return Ints.saturatedCast((long) a + b); + } + + /** + * Returns the difference of {@code a} and {@code b} unless it would overflow or underflow in + * which case {@code Integer.MAX_VALUE} or {@code Integer.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static int saturatedSubtract(int a, int b) { + return Ints.saturatedCast((long) a - b); + } + + /** + * Returns the product of {@code a} and {@code b} unless it would overflow or underflow in which + * case {@code Integer.MAX_VALUE} or {@code Integer.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static int saturatedMultiply(int a, int b) { + return Ints.saturatedCast((long) a * b); + } + + /** + * Returns the {@code b} to the {@code k}th power, unless it would overflow or underflow in which + * case {@code Integer.MAX_VALUE} or {@code Integer.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static int saturatedPow(int b, int k) { + checkNonNegative("exponent", k); + switch (b) { + case 0: + return (k == 0) ? 1 : 0; + case 1: + return 1; + case (-1): + return ((k & 1) == 0) ? 1 : -1; + case 2: + if (k >= Integer.SIZE - 1) { + return Integer.MAX_VALUE; + } + return 1 << k; + case (-2): + if (k >= Integer.SIZE) { + return Integer.MAX_VALUE + (k & 1); + } + return ((k & 1) == 0) ? 1 << k : -1 << k; + default: + // continue below to handle the general case + } + int accum = 1; + // if b is negative and k is odd then the limit is MIN otherwise the limit is MAX + int limit = Integer.MAX_VALUE + ((b >>> Integer.SIZE - 1) & (k & 1)); + while (true) { + switch (k) { + case 0: + return accum; + case 1: + return saturatedMultiply(accum, b); + default: + if ((k & 1) != 0) { + accum = saturatedMultiply(accum, b); + } + k >>= 1; + if (k > 0) { + if (-FLOOR_SQRT_MAX_INT > b | b > FLOOR_SQRT_MAX_INT) { + return limit; + } + b *= b; + } + } + } + } + + static final int FLOOR_SQRT_MAX_INT = 46340; + + /** + * Returns {@code n!}, that is, the product of the first {@code n} positive integers, {@code 1} if + * {@code n == 0}, or {@link Integer#MAX_VALUE} if the result does not fit in a {@code int}. + * + * @throws IllegalArgumentException if {@code n < 0} + */ + public static int factorial(int n) { + checkNonNegative("n", n); + return (n < factorials.length) ? factorials[n] : Integer.MAX_VALUE; + } + + private static final int[] factorials = { + 1, + 1, + 1 * 2, + 1 * 2 * 3, + 1 * 2 * 3 * 4, + 1 * 2 * 3 * 4 * 5, + 1 * 2 * 3 * 4 * 5 * 6, + 1 * 2 * 3 * 4 * 5 * 6 * 7, + 1 * 2 * 3 * 4 * 5 * 6 * 7 * 8, + 1 * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9, + 1 * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10, + 1 * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11, + 1 * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 + }; + + /** + * Returns {@code n} choose {@code k}, also known as the binomial coefficient of {@code n} and + * {@code k}, or {@link Integer#MAX_VALUE} if the result does not fit in an {@code int}. + * + * @throws IllegalArgumentException if {@code n < 0}, {@code k < 0} or {@code k > n} + */ + public static int binomial(int n, int k) { + checkNonNegative("n", n); + checkNonNegative("k", k); + checkArgument(k <= n, "k (%s) > n (%s)", k, n); + if (k > (n >> 1)) { + k = n - k; + } + if (k >= biggestBinomials.length || n > biggestBinomials[k]) { + return Integer.MAX_VALUE; + } + switch (k) { + case 0: + return 1; + case 1: + return n; + default: + long result = 1; + for (int i = 0; i < k; i++) { + result *= n - i; + result /= i + 1; + } + return (int) result; + } + } + + // binomial(biggestBinomials[k], k) fits in an int, but not binomial(biggestBinomials[k]+1,k). + + static int[] biggestBinomials = { + Integer.MAX_VALUE, + Integer.MAX_VALUE, + 65536, + 2345, + 477, + 193, + 110, + 75, + 58, + 49, + 43, + 39, + 37, + 35, + 34, + 34, + 33 + }; + + /** + * Returns the arithmetic mean of {@code x} and {@code y}, rounded towards negative infinity. This + * method is overflow resilient. + * + * @since 14.0 + */ + public static int mean(int x, int y) { + // Efficient method for computing the arithmetic mean. + // The alternative (x + y) / 2 fails for large values. + // The alternative (x + y) >>> 1 fails for negative values. + return (x & y) + ((x ^ y) >> 1); + } + + /** + * Returns {@code true} if {@code n} is a prime number: an integer greater + * than one that cannot be factored into a product of smaller positive integers. + * Returns {@code false} if {@code n} is zero, one, or a composite number (one which can be + * factored into smaller positive integers). + * + *

To test larger numbers, use {@link LongMath#isPrime} or {@link BigInteger#isProbablePrime}. + * + * @throws IllegalArgumentException if {@code n} is negative + * @since 20.0 + */ + + public static boolean isPrime(int n) { + return LongMath.isPrime(n); + } + + private IntMath() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/LongMath.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/LongMath.java new file mode 100644 index 0000000000000..1965ba25ddd4c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/LongMath.java @@ -0,0 +1,1193 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.math; + +import com.azure.cosmos.kafka.connect.implementation.guava25.primitives.UnsignedLongs; + +import java.math.BigInteger; +import java.math.RoundingMode; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkNoOverflow; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkNonNegative; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkPositive; +import static com.azure.cosmos.kafka.connect.implementation.guava25.math.MathPreconditions.checkRoundingUnnecessary; +import static java.lang.Math.abs; +import static java.lang.Math.min; +import static java.math.RoundingMode.HALF_EVEN; +import static java.math.RoundingMode.HALF_UP; + +/** + * A class for arithmetic on values of type {@code long}. Where possible, methods are defined and + * named analogously to their {@code BigInteger} counterparts. + * + *

The implementations of many methods in this class are based on material from Henry S. Warren, + * Jr.'s Hacker's Delight, (Addison Wesley, 2002). + * + *

Similar functionality for {@code int} and for {@link BigInteger} can be found in {@link + * IntMath} and {@link BigIntegerMath} respectively. For other common operations on {@code long} + * values, see {@link com.azure.cosmos.kafka.connect.implementation.guava25.primitives.Longs}. + * + * @author Louis Wasserman + * @since 11.0 + */ +public final class LongMath { + // NOTE: Whenever both tests are cheap and functional, it's faster to use &, | instead of &&, || + + static final long MAX_SIGNED_POWER_OF_TWO = 1L << (Long.SIZE - 2); + + /** + * Returns the smallest power of two greater than or equal to {@code x}. This is equivalent to + * {@code checkedPow(2, log2(x, CEILING))}. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @throws ArithmeticException of the next-higher power of two is not representable as a {@code + * long}, i.e. when {@code x > 2^62} + * @since 20.0 + */ + + public static long ceilingPowerOfTwo(long x) { + checkPositive("x", x); + if (x > MAX_SIGNED_POWER_OF_TWO) { + throw new ArithmeticException("ceilingPowerOfTwo(" + x + ") is not representable as a long"); + } + return 1L << -Long.numberOfLeadingZeros(x - 1); + } + + /** + * Returns the largest power of two less than or equal to {@code x}. This is equivalent to {@code + * checkedPow(2, log2(x, FLOOR))}. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @since 20.0 + */ + + public static long floorPowerOfTwo(long x) { + checkPositive("x", x); + + // Long.highestOneBit was buggy on GWT. We've fixed it, but I'm not certain when the fix will + // be released. + return 1L << ((Long.SIZE - 1) - Long.numberOfLeadingZeros(x)); + } + + /** + * Returns {@code true} if {@code x} represents a power of two. + * + *

This differs from {@code Long.bitCount(x) == 1}, because {@code + * Long.bitCount(Long.MIN_VALUE) == 1}, but {@link Long#MIN_VALUE} is not a power of two. + */ + public static boolean isPowerOfTwo(long x) { + return x > 0 & (x & (x - 1)) == 0; + } + + /** + * Returns 1 if {@code x < y} as unsigned longs, and 0 otherwise. Assumes that x - y fits into a + * signed long. The implementation is branch-free, and benchmarks suggest it is measurably faster + * than the straightforward ternary expression. + */ + + static int lessThanBranchFree(long x, long y) { + // Returns the sign bit of x - y. + return (int) (~~(x - y) >>> (Long.SIZE - 1)); + } + + /** + * Returns the base-2 logarithm of {@code x}, rounded according to the specified rounding mode. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @throws ArithmeticException if {@code mode} is {@link RoundingMode#UNNECESSARY} and {@code x} + * is not a power of two + */ + @SuppressWarnings("fallthrough") + // TODO(kevinb): remove after this warning is disabled globally + public static int log2(long x, RoundingMode mode) { + checkPositive("x", x); + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(isPowerOfTwo(x)); + // fall through + case DOWN: + case FLOOR: + return (Long.SIZE - 1) - Long.numberOfLeadingZeros(x); + + case UP: + case CEILING: + return Long.SIZE - Long.numberOfLeadingZeros(x - 1); + + case HALF_DOWN: + case HALF_UP: + case HALF_EVEN: + // Since sqrt(2) is irrational, log2(x) - logFloor cannot be exactly 0.5 + int leadingZeros = Long.numberOfLeadingZeros(x); + long cmp = MAX_POWER_OF_SQRT2_UNSIGNED >>> leadingZeros; + // floor(2^(logFloor + 0.5)) + int logFloor = (Long.SIZE - 1) - leadingZeros; + return logFloor + lessThanBranchFree(cmp, x); + + default: + throw new AssertionError("impossible"); + } + } + + /** The biggest half power of two that fits into an unsigned long */ + static final long MAX_POWER_OF_SQRT2_UNSIGNED = 0xB504F333F9DE6484L; + + /** + * Returns the base-10 logarithm of {@code x}, rounded according to the specified rounding mode. + * + * @throws IllegalArgumentException if {@code x <= 0} + * @throws ArithmeticException if {@code mode} is {@link RoundingMode#UNNECESSARY} and {@code x} + * is not a power of ten + */ + @SuppressWarnings("fallthrough") + // TODO(kevinb): remove after this warning is disabled globally + public static int log10(long x, RoundingMode mode) { + checkPositive("x", x); + int logFloor = log10Floor(x); + long floorPow = powersOf10[logFloor]; + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(x == floorPow); + // fall through + case FLOOR: + case DOWN: + return logFloor; + case CEILING: + case UP: + return logFloor + lessThanBranchFree(floorPow, x); + case HALF_DOWN: + case HALF_UP: + case HALF_EVEN: + // sqrt(10) is irrational, so log10(x)-logFloor is never exactly 0.5 + return logFloor + lessThanBranchFree(halfPowersOf10[logFloor], x); + default: + throw new AssertionError(); + } + } + + static int log10Floor(long x) { + /* + * Based on Hacker's Delight Fig. 11-5, the two-table-lookup, branch-free implementation. + * + * The key idea is that based on the number of leading zeros (equivalently, floor(log2(x))), we + * can narrow the possible floor(log10(x)) values to two. For example, if floor(log2(x)) is 6, + * then 64 <= x < 128, so floor(log10(x)) is either 1 or 2. + */ + int y = maxLog10ForLeadingZeros[Long.numberOfLeadingZeros(x)]; + /* + * y is the higher of the two possible values of floor(log10(x)). If x < 10^y, then we want the + * lower of the two possible values, or y - 1, otherwise, we want y. + */ + return y - lessThanBranchFree(x, powersOf10[y]); + } + + // maxLog10ForLeadingZeros[i] == floor(log10(2^(Long.SIZE - i))) + + static final byte[] maxLog10ForLeadingZeros = { + 19, 18, 18, 18, 18, 17, 17, 17, 16, 16, 16, 15, 15, 15, 15, 14, 14, 14, 13, 13, 13, 12, 12, 12, + 12, 11, 11, 11, 10, 10, 10, 9, 9, 9, 9, 8, 8, 8, 7, 7, 7, 6, 6, 6, 6, 5, 5, 5, 4, 4, 4, 3, 3, 3, + 3, 2, 2, 2, 1, 1, 1, 0, 0, 0 + }; + + + static final long[] powersOf10 = { + 1L, + 10L, + 100L, + 1000L, + 10000L, + 100000L, + 1000000L, + 10000000L, + 100000000L, + 1000000000L, + 10000000000L, + 100000000000L, + 1000000000000L, + 10000000000000L, + 100000000000000L, + 1000000000000000L, + 10000000000000000L, + 100000000000000000L, + 1000000000000000000L + }; + + // halfPowersOf10[i] = largest long less than 10^(i + 0.5) + + static final long[] halfPowersOf10 = { + 3L, + 31L, + 316L, + 3162L, + 31622L, + 316227L, + 3162277L, + 31622776L, + 316227766L, + 3162277660L, + 31622776601L, + 316227766016L, + 3162277660168L, + 31622776601683L, + 316227766016837L, + 3162277660168379L, + 31622776601683793L, + 316227766016837933L, + 3162277660168379331L + }; + + /** + * Returns {@code b} to the {@code k}th power. Even if the result overflows, it will be equal to + * {@code BigInteger.valueOf(b).pow(k).longValue()}. This implementation runs in {@code O(log k)} + * time. + * + * @throws IllegalArgumentException if {@code k < 0} + */ + public static long pow(long b, int k) { + checkNonNegative("exponent", k); + if (-2 <= b && b <= 2) { + switch ((int) b) { + case 0: + return (k == 0) ? 1 : 0; + case 1: + return 1; + case (-1): + return ((k & 1) == 0) ? 1 : -1; + case 2: + return (k < Long.SIZE) ? 1L << k : 0; + case (-2): + if (k < Long.SIZE) { + return ((k & 1) == 0) ? 1L << k : -(1L << k); + } else { + return 0; + } + default: + throw new AssertionError(); + } + } + for (long accum = 1; ; k >>= 1) { + switch (k) { + case 0: + return accum; + case 1: + return accum * b; + default: + accum *= ((k & 1) == 0) ? 1 : b; + b *= b; + } + } + } + + /** + * Returns the square root of {@code x}, rounded with the specified rounding mode. + * + * @throws IllegalArgumentException if {@code x < 0} + * @throws ArithmeticException if {@code mode} is {@link RoundingMode#UNNECESSARY} and {@code + * sqrt(x)} is not an integer + */ + @SuppressWarnings("fallthrough") + public static long sqrt(long x, RoundingMode mode) { + checkNonNegative("x", x); + if (fitsInInt(x)) { + return IntMath.sqrt((int) x, mode); + } + /* + * Let k be the true value of floor(sqrt(x)), so that + * + * k * k <= x < (k + 1) * (k + 1) + * (double) (k * k) <= (double) x <= (double) ((k + 1) * (k + 1)) + * since casting to double is nondecreasing. + * Note that the right-hand inequality is no longer strict. + * Math.sqrt(k * k) <= Math.sqrt(x) <= Math.sqrt((k + 1) * (k + 1)) + * since Math.sqrt is monotonic. + * (long) Math.sqrt(k * k) <= (long) Math.sqrt(x) <= (long) Math.sqrt((k + 1) * (k + 1)) + * since casting to long is monotonic + * k <= (long) Math.sqrt(x) <= k + 1 + * since (long) Math.sqrt(k * k) == k, as checked exhaustively in + * {@link LongMathTest#testSqrtOfPerfectSquareAsDoubleIsPerfect} + */ + long guess = (long) Math.sqrt(x); + // Note: guess is always <= FLOOR_SQRT_MAX_LONG. + long guessSquared = guess * guess; + // Note (2013-2-26): benchmarks indicate that, inscrutably enough, using if statements is + // faster here than using lessThanBranchFree. + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(guessSquared == x); + return guess; + case FLOOR: + case DOWN: + if (x < guessSquared) { + return guess - 1; + } + return guess; + case CEILING: + case UP: + if (x > guessSquared) { + return guess + 1; + } + return guess; + case HALF_DOWN: + case HALF_UP: + case HALF_EVEN: + long sqrtFloor = guess - ((x < guessSquared) ? 1 : 0); + long halfSquare = sqrtFloor * sqrtFloor + sqrtFloor; + /* + * We wish to test whether or not x <= (sqrtFloor + 0.5)^2 = halfSquare + 0.25. Since both x + * and halfSquare are integers, this is equivalent to testing whether or not x <= + * halfSquare. (We have to deal with overflow, though.) + * + * If we treat halfSquare as an unsigned long, we know that + * sqrtFloor^2 <= x < (sqrtFloor + 1)^2 + * halfSquare - sqrtFloor <= x < halfSquare + sqrtFloor + 1 + * so |x - halfSquare| <= sqrtFloor. Therefore, it's safe to treat x - halfSquare as a + * signed long, so lessThanBranchFree is safe for use. + */ + return sqrtFloor + lessThanBranchFree(halfSquare, x); + default: + throw new AssertionError(); + } + } + + /** + * Returns the result of dividing {@code p} by {@code q}, rounding using the specified {@code + * RoundingMode}. + * + * @throws ArithmeticException if {@code q == 0}, or if {@code mode == UNNECESSARY} and {@code a} + * is not an integer multiple of {@code b} + */ + @SuppressWarnings("fallthrough") + public static long divide(long p, long q, RoundingMode mode) { + checkNotNull(mode); + long div = p / q; // throws if q == 0 + long rem = p - q * div; // equals p % q + + if (rem == 0) { + return div; + } + + /* + * Normal Java division rounds towards 0, consistently with RoundingMode.DOWN. We just have to + * deal with the cases where rounding towards 0 is wrong, which typically depends on the sign of + * p / q. + * + * signum is 1 if p and q are both nonnegative or both negative, and -1 otherwise. + */ + int signum = 1 | (int) ((p ^ q) >> (Long.SIZE - 1)); + boolean increment; + switch (mode) { + case UNNECESSARY: + checkRoundingUnnecessary(rem == 0); + // fall through + case DOWN: + increment = false; + break; + case UP: + increment = true; + break; + case CEILING: + increment = signum > 0; + break; + case FLOOR: + increment = signum < 0; + break; + case HALF_EVEN: + case HALF_DOWN: + case HALF_UP: + long absRem = abs(rem); + long cmpRemToHalfDivisor = absRem - (abs(q) - absRem); + // subtracting two nonnegative longs can't overflow + // cmpRemToHalfDivisor has the same sign as compare(abs(rem), abs(q) / 2). + if (cmpRemToHalfDivisor == 0) { // exactly on the half mark + increment = (mode == HALF_UP | (mode == HALF_EVEN & (div & 1) != 0)); + } else { + increment = cmpRemToHalfDivisor > 0; // closer to the UP value + } + break; + default: + throw new AssertionError(); + } + return increment ? div + signum : div; + } + + /** + * Returns {@code x mod m}, a non-negative value less than {@code m}. This differs from {@code x % + * m}, which might be negative. + * + *

For example: + * + *

{@code
+   * mod(7, 4) == 3
+   * mod(-7, 4) == 1
+   * mod(-1, 4) == 3
+   * mod(-8, 4) == 0
+   * mod(8, 4) == 0
+   * }
+ * + * @throws ArithmeticException if {@code m <= 0} + * @see + * Remainder Operator + */ + public static int mod(long x, int m) { + // Cast is safe because the result is guaranteed in the range [0, m) + return (int) mod(x, (long) m); + } + + /** + * Returns {@code x mod m}, a non-negative value less than {@code m}. This differs from {@code x % + * m}, which might be negative. + * + *

For example: + * + *

{@code
+   * mod(7, 4) == 3
+   * mod(-7, 4) == 1
+   * mod(-1, 4) == 3
+   * mod(-8, 4) == 0
+   * mod(8, 4) == 0
+   * }
+ * + * @throws ArithmeticException if {@code m <= 0} + * @see + * Remainder Operator + */ + public static long mod(long x, long m) { + if (m <= 0) { + throw new ArithmeticException("Modulus must be positive"); + } + long result = x % m; + return (result >= 0) ? result : result + m; + } + + /** + * Returns the greatest common divisor of {@code a, b}. Returns {@code 0} if {@code a == 0 && b == + * 0}. + * + * @throws IllegalArgumentException if {@code a < 0} or {@code b < 0} + */ + public static long gcd(long a, long b) { + /* + * The reason we require both arguments to be >= 0 is because otherwise, what do you return on + * gcd(0, Long.MIN_VALUE)? BigInteger.gcd would return positive 2^63, but positive 2^63 isn't an + * int. + */ + checkNonNegative("a", a); + checkNonNegative("b", b); + if (a == 0) { + // 0 % b == 0, so b divides a, but the converse doesn't hold. + // BigInteger.gcd is consistent with this decision. + return b; + } else if (b == 0) { + return a; // similar logic + } + /* + * Uses the binary GCD algorithm; see http://en.wikipedia.org/wiki/Binary_GCD_algorithm. This is + * >60% faster than the Euclidean algorithm in benchmarks. + */ + int aTwos = Long.numberOfTrailingZeros(a); + a >>= aTwos; // divide out all 2s + int bTwos = Long.numberOfTrailingZeros(b); + b >>= bTwos; // divide out all 2s + while (a != b) { // both a, b are odd + // The key to the binary GCD algorithm is as follows: + // Both a and b are odd. Assume a > b; then gcd(a - b, b) = gcd(a, b). + // But in gcd(a - b, b), a - b is even and b is odd, so we can divide out powers of two. + + // We bend over backwards to avoid branching, adapting a technique from + // http://graphics.stanford.edu/~seander/bithacks.html#IntegerMinOrMax + + long delta = a - b; // can't overflow, since a and b are nonnegative + + long minDeltaOrZero = delta & (delta >> (Long.SIZE - 1)); + // equivalent to Math.min(delta, 0) + + a = delta - minDeltaOrZero - minDeltaOrZero; // sets a to Math.abs(a - b) + // a is now nonnegative and even + + b += minDeltaOrZero; // sets b to min(old a, b) + a >>= Long.numberOfTrailingZeros(a); // divide out all 2s, since 2 doesn't divide b + } + return a << min(aTwos, bTwos); + } + + /** + * Returns the sum of {@code a} and {@code b}, provided it does not overflow. + * + * @throws ArithmeticException if {@code a + b} overflows in signed {@code long} arithmetic + */ + public static long checkedAdd(long a, long b) { + long result = a + b; + checkNoOverflow((a ^ b) < 0 | (a ^ result) >= 0, "checkedAdd", a, b); + return result; + } + + /** + * Returns the difference of {@code a} and {@code b}, provided it does not overflow. + * + * @throws ArithmeticException if {@code a - b} overflows in signed {@code long} arithmetic + */ + public static long checkedSubtract(long a, long b) { + long result = a - b; + checkNoOverflow((a ^ b) >= 0 | (a ^ result) >= 0, "checkedSubtract", a, b); + return result; + } + + /** + * Returns the product of {@code a} and {@code b}, provided it does not overflow. + * + * @throws ArithmeticException if {@code a * b} overflows in signed {@code long} arithmetic + */ + public static long checkedMultiply(long a, long b) { + // Hacker's Delight, Section 2-12 + int leadingZeros = + Long.numberOfLeadingZeros(a) + + Long.numberOfLeadingZeros(~a) + + Long.numberOfLeadingZeros(b) + + Long.numberOfLeadingZeros(~b); + /* + * If leadingZeros > Long.SIZE + 1 it's definitely fine, if it's < Long.SIZE it's definitely + * bad. We do the leadingZeros check to avoid the division below if at all possible. + * + * Otherwise, if b == Long.MIN_VALUE, then the only allowed values of a are 0 and 1. We take + * care of all a < 0 with their own check, because in particular, the case a == -1 will + * incorrectly pass the division check below. + * + * In all other cases, we check that either a is 0 or the result is consistent with division. + */ + if (leadingZeros > Long.SIZE + 1) { + return a * b; + } + checkNoOverflow(leadingZeros >= Long.SIZE, "checkedMultiply", a, b); + checkNoOverflow(a >= 0 | b != Long.MIN_VALUE, "checkedMultiply", a, b); + long result = a * b; + checkNoOverflow(a == 0 || result / a == b, "checkedMultiply", a, b); + return result; + } + + /** + * Returns the {@code b} to the {@code k}th power, provided it does not overflow. + * + * @throws ArithmeticException if {@code b} to the {@code k}th power overflows in signed {@code + * long} arithmetic + */ + public static long checkedPow(long b, int k) { + checkNonNegative("exponent", k); + if (b >= -2 & b <= 2) { + switch ((int) b) { + case 0: + return (k == 0) ? 1 : 0; + case 1: + return 1; + case (-1): + return ((k & 1) == 0) ? 1 : -1; + case 2: + checkNoOverflow(k < Long.SIZE - 1, "checkedPow", b, k); + return 1L << k; + case (-2): + checkNoOverflow(k < Long.SIZE, "checkedPow", b, k); + return ((k & 1) == 0) ? (1L << k) : (-1L << k); + default: + throw new AssertionError(); + } + } + long accum = 1; + while (true) { + switch (k) { + case 0: + return accum; + case 1: + return checkedMultiply(accum, b); + default: + if ((k & 1) != 0) { + accum = checkedMultiply(accum, b); + } + k >>= 1; + if (k > 0) { + checkNoOverflow( + -FLOOR_SQRT_MAX_LONG <= b && b <= FLOOR_SQRT_MAX_LONG, "checkedPow", b, k); + b *= b; + } + } + } + } + + /** + * Returns the sum of {@code a} and {@code b} unless it would overflow or underflow in which case + * {@code Long.MAX_VALUE} or {@code Long.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static long saturatedAdd(long a, long b) { + long naiveSum = a + b; + if ((a ^ b) < 0 | (a ^ naiveSum) >= 0) { + // If a and b have different signs or a has the same sign as the result then there was no + // overflow, return. + return naiveSum; + } + // we did over/under flow, if the sign is negative we should return MAX otherwise MIN + return Long.MAX_VALUE + ((naiveSum >>> (Long.SIZE - 1)) ^ 1); + } + + /** + * Returns the difference of {@code a} and {@code b} unless it would overflow or underflow in + * which case {@code Long.MAX_VALUE} or {@code Long.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static long saturatedSubtract(long a, long b) { + long naiveDifference = a - b; + if ((a ^ b) >= 0 | (a ^ naiveDifference) >= 0) { + // If a and b have the same signs or a has the same sign as the result then there was no + // overflow, return. + return naiveDifference; + } + // we did over/under flow + return Long.MAX_VALUE + ((naiveDifference >>> (Long.SIZE - 1)) ^ 1); + } + + /** + * Returns the product of {@code a} and {@code b} unless it would overflow or underflow in which + * case {@code Long.MAX_VALUE} or {@code Long.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static long saturatedMultiply(long a, long b) { + // see checkedMultiply for explanation + int leadingZeros = + Long.numberOfLeadingZeros(a) + + Long.numberOfLeadingZeros(~a) + + Long.numberOfLeadingZeros(b) + + Long.numberOfLeadingZeros(~b); + if (leadingZeros > Long.SIZE + 1) { + return a * b; + } + // the return value if we will overflow (which we calculate by overflowing a long :) ) + long limit = Long.MAX_VALUE + ((a ^ b) >>> (Long.SIZE - 1)); + if (leadingZeros < Long.SIZE | (a < 0 & b == Long.MIN_VALUE)) { + // overflow + return limit; + } + long result = a * b; + if (a == 0 || result / a == b) { + return result; + } + return limit; + } + + /** + * Returns the {@code b} to the {@code k}th power, unless it would overflow or underflow in which + * case {@code Long.MAX_VALUE} or {@code Long.MIN_VALUE} is returned, respectively. + * + * @since 20.0 + */ + + public static long saturatedPow(long b, int k) { + checkNonNegative("exponent", k); + if (b >= -2 & b <= 2) { + switch ((int) b) { + case 0: + return (k == 0) ? 1 : 0; + case 1: + return 1; + case (-1): + return ((k & 1) == 0) ? 1 : -1; + case 2: + if (k >= Long.SIZE - 1) { + return Long.MAX_VALUE; + } + return 1L << k; + case (-2): + if (k >= Long.SIZE) { + return Long.MAX_VALUE + (k & 1); + } + return ((k & 1) == 0) ? (1L << k) : (-1L << k); + default: + throw new AssertionError(); + } + } + long accum = 1; + // if b is negative and k is odd then the limit is MIN otherwise the limit is MAX + long limit = Long.MAX_VALUE + ((b >>> Long.SIZE - 1) & (k & 1)); + while (true) { + switch (k) { + case 0: + return accum; + case 1: + return saturatedMultiply(accum, b); + default: + if ((k & 1) != 0) { + accum = saturatedMultiply(accum, b); + } + k >>= 1; + if (k > 0) { + if (-FLOOR_SQRT_MAX_LONG > b | b > FLOOR_SQRT_MAX_LONG) { + return limit; + } + b *= b; + } + } + } + } + + static final long FLOOR_SQRT_MAX_LONG = 3037000499L; + + /** + * Returns {@code n!}, that is, the product of the first {@code n} positive integers, {@code 1} if + * {@code n == 0}, or {@link Long#MAX_VALUE} if the result does not fit in a {@code long}. + * + * @throws IllegalArgumentException if {@code n < 0} + */ + public static long factorial(int n) { + checkNonNegative("n", n); + return (n < factorials.length) ? factorials[n] : Long.MAX_VALUE; + } + + static final long[] factorials = { + 1L, + 1L, + 1L * 2, + 1L * 2 * 3, + 1L * 2 * 3 * 4, + 1L * 2 * 3 * 4 * 5, + 1L * 2 * 3 * 4 * 5 * 6, + 1L * 2 * 3 * 4 * 5 * 6 * 7, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14 * 15, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14 * 15 * 16, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14 * 15 * 16 * 17, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14 * 15 * 16 * 17 * 18, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14 * 15 * 16 * 17 * 18 * 19, + 1L * 2 * 3 * 4 * 5 * 6 * 7 * 8 * 9 * 10 * 11 * 12 * 13 * 14 * 15 * 16 * 17 * 18 * 19 * 20 + }; + + /** + * Returns {@code n} choose {@code k}, also known as the binomial coefficient of {@code n} and + * {@code k}, or {@link Long#MAX_VALUE} if the result does not fit in a {@code long}. + * + * @throws IllegalArgumentException if {@code n < 0}, {@code k < 0}, or {@code k > n} + */ + public static long binomial(int n, int k) { + checkNonNegative("n", n); + checkNonNegative("k", k); + checkArgument(k <= n, "k (%s) > n (%s)", k, n); + if (k > (n >> 1)) { + k = n - k; + } + switch (k) { + case 0: + return 1; + case 1: + return n; + default: + if (n < factorials.length) { + return factorials[n] / (factorials[k] * factorials[n - k]); + } else if (k >= biggestBinomials.length || n > biggestBinomials[k]) { + return Long.MAX_VALUE; + } else if (k < biggestSimpleBinomials.length && n <= biggestSimpleBinomials[k]) { + // guaranteed not to overflow + long result = n--; + for (int i = 2; i <= k; n--, i++) { + result *= n; + result /= i; + } + return result; + } else { + int nBits = LongMath.log2(n, RoundingMode.CEILING); + + long result = 1; + long numerator = n--; + long denominator = 1; + + int numeratorBits = nBits; + // This is an upper bound on log2(numerator, ceiling). + + /* + * We want to do this in long math for speed, but want to avoid overflow. We adapt the + * technique previously used by BigIntegerMath: maintain separate numerator and + * denominator accumulators, multiplying the fraction into result when near overflow. + */ + for (int i = 2; i <= k; i++, n--) { + if (numeratorBits + nBits < Long.SIZE - 1) { + // It's definitely safe to multiply into numerator and denominator. + numerator *= n; + denominator *= i; + numeratorBits += nBits; + } else { + // It might not be safe to multiply into numerator and denominator, + // so multiply (numerator / denominator) into result. + result = multiplyFraction(result, numerator, denominator); + numerator = n; + denominator = i; + numeratorBits = nBits; + } + } + return multiplyFraction(result, numerator, denominator); + } + } + } + + /** Returns (x * numerator / denominator), which is assumed to come out to an integral value. */ + static long multiplyFraction(long x, long numerator, long denominator) { + if (x == 1) { + return numerator / denominator; + } + long commonDivisor = gcd(x, denominator); + x /= commonDivisor; + denominator /= commonDivisor; + // We know gcd(x, denominator) = 1, and x * numerator / denominator is exact, + // so denominator must be a divisor of numerator. + return x * (numerator / denominator); + } + + /* + * binomial(biggestBinomials[k], k) fits in a long, but not binomial(biggestBinomials[k] + 1, k). + */ + static final int[] biggestBinomials = { + Integer.MAX_VALUE, + Integer.MAX_VALUE, + Integer.MAX_VALUE, + 3810779, + 121977, + 16175, + 4337, + 1733, + 887, + 534, + 361, + 265, + 206, + 169, + 143, + 125, + 111, + 101, + 94, + 88, + 83, + 79, + 76, + 74, + 72, + 70, + 69, + 68, + 67, + 67, + 66, + 66, + 66, + 66 + }; + + /* + * binomial(biggestSimpleBinomials[k], k) doesn't need to use the slower GCD-based impl, but + * binomial(biggestSimpleBinomials[k] + 1, k) does. + */ + + static final int[] biggestSimpleBinomials = { + Integer.MAX_VALUE, + Integer.MAX_VALUE, + Integer.MAX_VALUE, + 2642246, + 86251, + 11724, + 3218, + 1313, + 684, + 419, + 287, + 214, + 169, + 139, + 119, + 105, + 95, + 87, + 81, + 76, + 73, + 70, + 68, + 66, + 64, + 63, + 62, + 62, + 61, + 61, + 61 + }; + // These values were generated by using checkedMultiply to see when the simple multiply/divide + // algorithm would lead to an overflow. + + static boolean fitsInInt(long x) { + return (int) x == x; + } + + /** + * Returns the arithmetic mean of {@code x} and {@code y}, rounded toward negative infinity. This + * method is resilient to overflow. + * + * @since 14.0 + */ + public static long mean(long x, long y) { + // Efficient method for computing the arithmetic mean. + // The alternative (x + y) / 2 fails for large values. + // The alternative (x + y) >>> 1 fails for negative values. + return (x & y) + ((x ^ y) >> 1); + } + + /* + * This bitmask is used as an optimization for cheaply testing for divisiblity by 2, 3, or 5. + * Each bit is set to 1 for all remainders that indicate divisibility by 2, 3, or 5, so + * 1, 7, 11, 13, 17, 19, 23, 29 are set to 0. 30 and up don't matter because they won't be hit. + */ + private static final int SIEVE_30 = + ~((1 << 1) | (1 << 7) | (1 << 11) | (1 << 13) | (1 << 17) | (1 << 19) | (1 << 23) + | (1 << 29)); + + /** + * Returns {@code true} if {@code n} is a prime number: an integer greater + * than one that cannot be factored into a product of smaller positive integers. + * Returns {@code false} if {@code n} is zero, one, or a composite number (one which can be + * factored into smaller positive integers). + * + *

To test larger numbers, use {@link BigInteger#isProbablePrime}. + * + * @throws IllegalArgumentException if {@code n} is negative + * @since 20.0 + */ + + public static boolean isPrime(long n) { + if (n < 2) { + checkNonNegative("n", n); + return false; + } + if (n == 2 || n == 3 || n == 5 || n == 7 || n == 11 || n == 13) { + return true; + } + + if ((SIEVE_30 & (1 << (n % 30))) != 0) { + return false; + } + if (n % 7 == 0 || n % 11 == 0 || n % 13 == 0) { + return false; + } + if (n < 17 * 17) { + return true; + } + + for (long[] baseSet : millerRabinBaseSets) { + if (n <= baseSet[0]) { + for (int i = 1; i < baseSet.length; i++) { + if (!MillerRabinTester.test(baseSet[i], n)) { + return false; + } + } + return true; + } + } + throw new AssertionError(); + } + + /* + * If n <= millerRabinBases[i][0], then testing n against bases millerRabinBases[i][1..] suffices + * to prove its primality. Values from miller-rabin.appspot.com. + * + * NOTE: We could get slightly better bases that would be treated as unsigned, but benchmarks + * showed negligible performance improvements. + */ + private static final long[][] millerRabinBaseSets = { + {291830, 126401071349994536L}, + {885594168, 725270293939359937L, 3569819667048198375L}, + {273919523040L, 15, 7363882082L, 992620450144556L}, + {47636622961200L, 2, 2570940, 211991001, 3749873356L}, + { + 7999252175582850L, + 2, + 4130806001517L, + 149795463772692060L, + 186635894390467037L, + 3967304179347715805L + }, + { + 585226005592931976L, + 2, + 123635709730000L, + 9233062284813009L, + 43835965440333360L, + 761179012939631437L, + 1263739024124850375L + }, + {Long.MAX_VALUE, 2, 325, 9375, 28178, 450775, 9780504, 1795265022} + }; + + private enum MillerRabinTester { + /** Works for inputs ≤ FLOOR_SQRT_MAX_LONG. */ + SMALL { + @Override + long mulMod(long a, long b, long m) { + /* + * NOTE(lowasser, 2015-Feb-12): Benchmarks suggest that changing this to + * UnsignedLongs.remainder and increasing the threshold to 2^32 doesn't pay for itself, and + * adding another enum constant hurts performance further -- I suspect because bimorphic + * implementation is a sweet spot for the JVM. + */ + return (a * b) % m; + } + + @Override + long squareMod(long a, long m) { + return (a * a) % m; + } + }, + /** Works for all nonnegative signed longs. */ + LARGE { + /** Returns (a + b) mod m. Precondition: {@code 0 <= a}, {@code b < m < 2^63}. */ + private long plusMod(long a, long b, long m) { + return (a >= m - b) ? (a + b - m) : (a + b); + } + + /** Returns (a * 2^32) mod m. a may be any unsigned long. */ + private long times2ToThe32Mod(long a, long m) { + int remainingPowersOf2 = 32; + do { + int shift = Math.min(remainingPowersOf2, Long.numberOfLeadingZeros(a)); + // shift is either the number of powers of 2 left to multiply a by, or the biggest shift + // possible while keeping a in an unsigned long. + a = UnsignedLongs.remainder(a << shift, m); + remainingPowersOf2 -= shift; + } while (remainingPowersOf2 > 0); + return a; + } + + @Override + long mulMod(long a, long b, long m) { + long aHi = a >>> 32; // < 2^31 + long bHi = b >>> 32; // < 2^31 + long aLo = a & 0xFFFFFFFFL; // < 2^32 + long bLo = b & 0xFFFFFFFFL; // < 2^32 + + /* + * a * b == aHi * bHi * 2^64 + (aHi * bLo + aLo * bHi) * 2^32 + aLo * bLo. + * == (aHi * bHi * 2^32 + aHi * bLo + aLo * bHi) * 2^32 + aLo * bLo + * + * We carry out this computation in modular arithmetic. Since times2ToThe32Mod accepts any + * unsigned long, we don't have to do a mod on every operation, only when intermediate + * results can exceed 2^63. + */ + long result = times2ToThe32Mod(aHi * bHi /* < 2^62 */, m); // < m < 2^63 + result += aHi * bLo; // aHi * bLo < 2^63, result < 2^64 + if (result < 0) { + result = UnsignedLongs.remainder(result, m); + } + // result < 2^63 again + result += aLo * bHi; // aLo * bHi < 2^63, result < 2^64 + result = times2ToThe32Mod(result, m); // result < m < 2^63 + return plusMod(result, UnsignedLongs.remainder(aLo * bLo /* < 2^64 */, m), m); + } + + @Override + long squareMod(long a, long m) { + long aHi = a >>> 32; // < 2^31 + long aLo = a & 0xFFFFFFFFL; // < 2^32 + + /* + * a^2 == aHi^2 * 2^64 + aHi * aLo * 2^33 + aLo^2 + * == (aHi^2 * 2^32 + aHi * aLo * 2) * 2^32 + aLo^2 + * We carry out this computation in modular arithmetic. Since times2ToThe32Mod accepts any + * unsigned long, we don't have to do a mod on every operation, only when intermediate + * results can exceed 2^63. + */ + long result = times2ToThe32Mod(aHi * aHi /* < 2^62 */, m); // < m < 2^63 + long hiLo = aHi * aLo * 2; + if (hiLo < 0) { + hiLo = UnsignedLongs.remainder(hiLo, m); + } + // hiLo < 2^63 + result += hiLo; // result < 2^64 + result = times2ToThe32Mod(result, m); // result < m < 2^63 + return plusMod(result, UnsignedLongs.remainder(aLo * aLo /* < 2^64 */, m), m); + } + }; + + static boolean test(long base, long n) { + // Since base will be considered % n, it's okay if base > FLOOR_SQRT_MAX_LONG, + // so long as n <= FLOOR_SQRT_MAX_LONG. + return ((n <= FLOOR_SQRT_MAX_LONG) ? SMALL : LARGE).testWitness(base, n); + } + + /** Returns a * b mod m. */ + abstract long mulMod(long a, long b, long m); + + /** Returns a^2 mod m. */ + abstract long squareMod(long a, long m); + + /** Returns a^p mod m. */ + private long powMod(long a, long p, long m) { + long res = 1; + for (; p != 0; p >>= 1) { + if ((p & 1) != 0) { + res = mulMod(res, a, m); + } + a = squareMod(a, m); + } + return res; + } + + /** Returns true if n is a strong probable prime relative to the specified base. */ + private boolean testWitness(long base, long n) { + int r = Long.numberOfTrailingZeros(n - 1); + long d = (n - 1) >> r; + base %= n; + if (base == 0) { + return true; + } + // Calculate a := base^d mod n. + long a = powMod(base, d, n); + // n passes this test if + // base^d = 1 (mod n) + // or base^(2^j * d) = -1 (mod n) for some 0 <= j < r. + if (a == 1) { + return true; + } + int j = 0; + while (a != n - 1) { + if (++j == r) { + return false; + } + a = squareMod(a, n); + } + return true; + } + } + + private LongMath() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/MathPreconditions.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/MathPreconditions.java new file mode 100644 index 0000000000000..72663fbb34181 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/MathPreconditions.java @@ -0,0 +1,104 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.math; + +import java.math.BigInteger; + + +/** + * A collection of preconditions for math functions. + * + * @author Louis Wasserman + */ +final class MathPreconditions { + static int checkPositive(String role, int x) { + if (x <= 0) { + throw new IllegalArgumentException(role + " (" + x + ") must be > 0"); + } + return x; + } + + static long checkPositive(String role, long x) { + if (x <= 0) { + throw new IllegalArgumentException(role + " (" + x + ") must be > 0"); + } + return x; + } + + static BigInteger checkPositive(String role, BigInteger x) { + if (x.signum() <= 0) { + throw new IllegalArgumentException(role + " (" + x + ") must be > 0"); + } + return x; + } + + static int checkNonNegative(String role, int x) { + if (x < 0) { + throw new IllegalArgumentException(role + " (" + x + ") must be >= 0"); + } + return x; + } + + static long checkNonNegative(String role, long x) { + if (x < 0) { + throw new IllegalArgumentException(role + " (" + x + ") must be >= 0"); + } + return x; + } + + static BigInteger checkNonNegative(String role, BigInteger x) { + if (x.signum() < 0) { + throw new IllegalArgumentException(role + " (" + x + ") must be >= 0"); + } + return x; + } + + static double checkNonNegative(String role, double x) { + if (!(x >= 0)) { // not x < 0, to work with NaN. + throw new IllegalArgumentException(role + " (" + x + ") must be >= 0"); + } + return x; + } + + static void checkRoundingUnnecessary(boolean condition) { + if (!condition) { + throw new ArithmeticException("mode was UNNECESSARY, but rounding was necessary"); + } + } + + static void checkInRange(boolean condition) { + if (!condition) { + throw new ArithmeticException("not in range"); + } + } + + static void checkNoOverflow(boolean condition, String methodName, int a, int b) { + if (!condition) { + throw new ArithmeticException("overflow: " + methodName + "(" + a + ", " + b + ")"); + } + } + + static void checkNoOverflow(boolean condition, String methodName, long a, long b) { + if (!condition) { + throw new ArithmeticException("overflow: " + methodName + "(" + a + ", " + b + ")"); + } + } + + private MathPreconditions() {} +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/package-info.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/package-info.java new file mode 100644 index 0000000000000..20cf723f3d09f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/math/package-info.java @@ -0,0 +1,28 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Arithmetic functions operating on primitive values and {@link java.math.BigInteger} instances. + * + *

This package is a part of the open-source Guava + * library. + * + *

See the Guava User Guide article on math utilities. + */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.math; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Booleans.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Booleans.java new file mode 100644 index 0000000000000..00c16dae49168 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Booleans.java @@ -0,0 +1,552 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + +import java.io.Serializable; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.RandomAccess; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; + +/** + * Static utility methods pertaining to {@code boolean} primitives, that are not already found in + * either {@link Boolean} or {@link Arrays}. + * + *

See the Guava User Guide article on primitive utilities. + * + * @author Kevin Bourrillion + * @since 1.0 + */ +public final class Booleans { + private Booleans() {} + + /** Comparators for {@code Boolean} values. */ + private enum BooleanComparator implements Comparator { + TRUE_FIRST(1, "Booleans.trueFirst()"), + FALSE_FIRST(-1, "Booleans.falseFirst()"); + + private final int trueValue; + private final String toString; + + BooleanComparator(int trueValue, String toString) { + this.trueValue = trueValue; + this.toString = toString; + } + + @Override + public int compare(Boolean a, Boolean b) { + int aVal = a ? trueValue : 0; + int bVal = b ? trueValue : 0; + return bVal - aVal; + } + + @Override + public String toString() { + return toString; + } + } + + /** + * Returns a {@code Comparator} that sorts {@code true} before {@code false}. + * + *

This is particularly useful in Java 8+ in combination with {@code Comparators.comparing}, + * e.g. {@code Comparators.comparing(Foo::hasBar, trueFirst())}. + * + * @since 21.0 + */ + + public static Comparator trueFirst() { + return BooleanComparator.TRUE_FIRST; + } + + /** + * Returns a {@code Comparator} that sorts {@code false} before {@code true}. + * + *

This is particularly useful in Java 8+ in combination with {@code Comparators.comparing}, + * e.g. {@code Comparators.comparing(Foo::hasBar, falseFirst())}. + * + * @since 21.0 + */ + + public static Comparator falseFirst() { + return BooleanComparator.FALSE_FIRST; + } + + /** + * Returns a hash code for {@code value}; equal to the result of invoking {@code ((Boolean) + * value).hashCode()}. + * + *

Java 8 users: use {@link Boolean#hashCode(boolean)} instead. + * + * @param value a primitive {@code boolean} value + * @return a hash code for the value + */ + public static int hashCode(boolean value) { + return value ? 1231 : 1237; + } + + /** + * Compares the two specified {@code boolean} values in the standard way ({@code false} is + * considered less than {@code true}). The sign of the value returned is the same as that of + * {@code ((Boolean) a).compareTo(b)}. + * + *

Note for Java 7 and later: this method should be treated as deprecated; use the + * equivalent {@link Boolean#compare} method instead. + * + * @param a the first {@code boolean} to compare + * @param b the second {@code boolean} to compare + * @return a positive number if only {@code a} is {@code true}, a negative number if only {@code + * b} is true, or zero if {@code a == b} + */ + public static int compare(boolean a, boolean b) { + return (a == b) ? 0 : (a ? 1 : -1); + } + + /** + * Returns {@code true} if {@code target} is present as an element anywhere in {@code array}. + * + *

Note: consider representing the array as a {@link java.util.BitSet} instead, + * replacing {@code Booleans.contains(array, true)} with {@code !bitSet.isEmpty()} and {@code + * Booleans.contains(array, false)} with {@code bitSet.nextClearBit(0) == sizeOfBitSet}. + * + * @param array an array of {@code boolean} values, possibly empty + * @param target a primitive {@code boolean} value + * @return {@code true} if {@code array[i] == target} for some value of {@code i} + */ + public static boolean contains(boolean[] array, boolean target) { + for (boolean value : array) { + if (value == target) { + return true; + } + } + return false; + } + + /** + * Returns the index of the first appearance of the value {@code target} in {@code array}. + * + *

Note: consider representing the array as a {@link java.util.BitSet} instead, and + * using {@link java.util.BitSet#nextSetBit(int)} or {@link java.util.BitSet#nextClearBit(int)}. + * + * @param array an array of {@code boolean} values, possibly empty + * @param target a primitive {@code boolean} value + * @return the least index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int indexOf(boolean[] array, boolean target) { + return indexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int indexOf(boolean[] array, boolean target, int start, int end) { + for (int i = start; i < end; i++) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the start position of the first occurrence of the specified {@code target} within + * {@code array}, or {@code -1} if there is no such occurrence. + * + *

More formally, returns the lowest index {@code i} such that {@code Arrays.copyOfRange(array, + * i, i + target.length)} contains exactly the same elements as {@code target}. + * + * @param array the array to search for the sequence {@code target} + * @param target the array to search for as a sub-sequence of {@code array} + */ + public static int indexOf(boolean[] array, boolean[] target) { + checkNotNull(array, "array"); + checkNotNull(target, "target"); + if (target.length == 0) { + return 0; + } + + outer: + for (int i = 0; i < array.length - target.length + 1; i++) { + for (int j = 0; j < target.length; j++) { + if (array[i + j] != target[j]) { + continue outer; + } + } + return i; + } + return -1; + } + + /** + * Returns the index of the last appearance of the value {@code target} in {@code array}. + * + * @param array an array of {@code boolean} values, possibly empty + * @param target a primitive {@code boolean} value + * @return the greatest index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int lastIndexOf(boolean[] array, boolean target) { + return lastIndexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int lastIndexOf(boolean[] array, boolean target, int start, int end) { + for (int i = end - 1; i >= start; i--) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the values from each provided array combined into a single array. For example, {@code + * concat(new boolean[] {a, b}, new boolean[] {}, new boolean[] {c}} returns the array {@code {a, + * b, c}}. + * + * @param arrays zero or more {@code boolean} arrays + * @return a single array containing all the values from the source arrays, in order + */ + public static boolean[] concat(boolean[]... arrays) { + int length = 0; + for (boolean[] array : arrays) { + length += array.length; + } + boolean[] result = new boolean[length]; + int pos = 0; + for (boolean[] array : arrays) { + System.arraycopy(array, 0, result, pos, array.length); + pos += array.length; + } + return result; + } + + /** + * Returns an array containing the same values as {@code array}, but guaranteed to be of a + * specified minimum length. If {@code array} already has a length of at least {@code minLength}, + * it is returned directly. Otherwise, a new array of size {@code minLength + padding} is + * returned, containing the values of {@code array}, and zeroes in the remaining places. + * + * @param array the source array + * @param minLength the minimum length the returned array must guarantee + * @param padding an extra amount to "grow" the array by if growth is necessary + * @throws IllegalArgumentException if {@code minLength} or {@code padding} is negative + * @return an array containing the values of {@code array}, with guaranteed minimum length {@code + * minLength} + */ + public static boolean[] ensureCapacity(boolean[] array, int minLength, int padding) { + checkArgument(minLength >= 0, "Invalid minLength: %s", minLength); + checkArgument(padding >= 0, "Invalid padding: %s", padding); + return (array.length < minLength) ? Arrays.copyOf(array, minLength + padding) : array; + } + + /** + * Returns a string containing the supplied {@code boolean} values separated by {@code separator}. + * For example, {@code join("-", false, true, false)} returns the string {@code + * "false-true-false"}. + * + * @param separator the text that should appear between consecutive values in the resulting string + * (but not at the start or end) + * @param array an array of {@code boolean} values, possibly empty + */ + public static String join(String separator, boolean... array) { + checkNotNull(separator); + if (array.length == 0) { + return ""; + } + + // For pre-sizing a builder, just get the right order of magnitude + StringBuilder builder = new StringBuilder(array.length * 7); + builder.append(array[0]); + for (int i = 1; i < array.length; i++) { + builder.append(separator).append(array[i]); + } + return builder.toString(); + } + + /** + * Returns a comparator that compares two {@code boolean} arrays lexicographically. That is, it + * compares, using {@link #compare(boolean, boolean)}), the first pair of values that follow any + * common prefix, or when one array is a prefix of the other, treats the shorter array as the + * lesser. For example, {@code [] < [false] < [false, true] < [true]}. + * + *

The returned comparator is inconsistent with {@link Object#equals(Object)} (since arrays + * support only identity equality), but it is consistent with {@link Arrays#equals(boolean[], + * boolean[])}. + * + * @since 2.0 + */ + public static Comparator lexicographicalComparator() { + return LexicographicalComparator.INSTANCE; + } + + private enum LexicographicalComparator implements Comparator { + INSTANCE; + + @Override + public int compare(boolean[] left, boolean[] right) { + int minLength = Math.min(left.length, right.length); + for (int i = 0; i < minLength; i++) { + int result = Booleans.compare(left[i], right[i]); + if (result != 0) { + return result; + } + } + return left.length - right.length; + } + + @Override + public String toString() { + return "Booleans.lexicographicalComparator()"; + } + } + + /** + * Copies a collection of {@code Boolean} instances into a new array of primitive {@code boolean} + * values. + * + *

Elements are copied from the argument collection as if by {@code collection.toArray()}. + * Calling this method is as thread-safe as calling that method. + * + *

Note: consider representing the collection as a {@link java.util.BitSet} instead. + * + * @param collection a collection of {@code Boolean} objects + * @return an array containing the same values as {@code collection}, in the same order, converted + * to primitives + * @throws NullPointerException if {@code collection} or any of its elements is null + */ + public static boolean[] toArray(Collection collection) { + if (collection instanceof BooleanArrayAsList) { + return ((BooleanArrayAsList) collection).toBooleanArray(); + } + + Object[] boxedArray = collection.toArray(); + int len = boxedArray.length; + boolean[] array = new boolean[len]; + for (int i = 0; i < len; i++) { + // checkNotNull for GWT (do not optimize) + array[i] = (Boolean) checkNotNull(boxedArray[i]); + } + return array; + } + + /** + * Returns a fixed-size list backed by the specified array, similar to {@link + * Arrays#asList(Object[])}. The list supports {@link List#set(int, Object)}, but any attempt to + * set a value to {@code null} will result in a {@link NullPointerException}. + * + *

The returned list maintains the values, but not the identities, of {@code Boolean} objects + * written to or read from it. For example, whether {@code list.get(0) == list.get(0)} is true for + * the returned list is unspecified. + * + * @param backingArray the array to back the list + * @return a list view of the array + */ + public static List asList(boolean... backingArray) { + if (backingArray.length == 0) { + return Collections.emptyList(); + } + return new BooleanArrayAsList(backingArray); + } + + private static class BooleanArrayAsList extends AbstractList + implements RandomAccess, Serializable { + final boolean[] array; + final int start; + final int end; + + BooleanArrayAsList(boolean[] array) { + this(array, 0, array.length); + } + + BooleanArrayAsList(boolean[] array, int start, int end) { + this.array = array; + this.start = start; + this.end = end; + } + + @Override + public int size() { + return end - start; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Boolean get(int index) { + checkElementIndex(index, size()); + return array[start + index]; + } + + @Override + public boolean contains(Object target) { + // Overridden to prevent a ton of boxing + return (target instanceof Boolean) + && Booleans.indexOf(array, (Boolean) target, start, end) != -1; + } + + @Override + public int indexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Boolean) { + int i = Booleans.indexOf(array, (Boolean) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public int lastIndexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Boolean) { + int i = Booleans.lastIndexOf(array, (Boolean) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public Boolean set(int index, Boolean element) { + checkElementIndex(index, size()); + boolean oldValue = array[start + index]; + // checkNotNull for GWT (do not optimize) + array[start + index] = checkNotNull(element); + return oldValue; + } + + @Override + public List subList(int fromIndex, int toIndex) { + int size = size(); + checkPositionIndexes(fromIndex, toIndex, size); + if (fromIndex == toIndex) { + return Collections.emptyList(); + } + return new BooleanArrayAsList(array, start + fromIndex, start + toIndex); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof BooleanArrayAsList) { + BooleanArrayAsList that = (BooleanArrayAsList) object; + int size = size(); + if (that.size() != size) { + return false; + } + for (int i = 0; i < size; i++) { + if (array[start + i] != that.array[that.start + i]) { + return false; + } + } + return true; + } + return super.equals(object); + } + + @Override + public int hashCode() { + int result = 1; + for (int i = start; i < end; i++) { + result = 31 * result + Booleans.hashCode(array[i]); + } + return result; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(size() * 7); + builder.append(array[start] ? "[true" : "[false"); + for (int i = start + 1; i < end; i++) { + builder.append(array[i] ? ", true" : ", false"); + } + return builder.append(']').toString(); + } + + boolean[] toBooleanArray() { + return Arrays.copyOfRange(array, start, end); + } + + private static final long serialVersionUID = 0; + } + + /** + * Returns the number of {@code values} that are {@code true}. + * + * @since 16.0 + */ + + public static int countTrue(boolean... values) { + int count = 0; + for (boolean value : values) { + if (value) { + count++; + } + } + return count; + } + + /** + * Reverses the elements of {@code array}. This is equivalent to {@code + * Collections.reverse(Booleans.asList(array))}, but is likely to be more efficient. + * + * @since 23.1 + */ + public static void reverse(boolean[] array) { + checkNotNull(array); + reverse(array, 0, array.length); + } + + /** + * Reverses the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive. This is equivalent to {@code + * Collections.reverse(Booleans.asList(array).subList(fromIndex, toIndex))}, but is likely to be + * more efficient. + * + * @throws IndexOutOfBoundsException if {@code fromIndex < 0}, {@code toIndex > array.length}, or + * {@code toIndex > fromIndex} + * @since 23.1 + */ + public static void reverse(boolean[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + for (int i = fromIndex, j = toIndex - 1; i < j; i++, j--) { + boolean tmp = array[i]; + array[i] = array[j]; + array[j] = tmp; + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Doubles.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Doubles.java new file mode 100644 index 0000000000000..d9216f0bfd09c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Doubles.java @@ -0,0 +1,697 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Converter; + +import java.io.Serializable; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.RandomAccess; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.regex.Pattern; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; +import static java.lang.Double.NEGATIVE_INFINITY; +import static java.lang.Double.POSITIVE_INFINITY; + + +/** + * Static utility methods pertaining to {@code double} primitives, that are not already found in + * either {@link Double} or {@link Arrays}. + * + *

See the Guava User Guide article on primitive utilities. + * + * @author Kevin Bourrillion + * @since 1.0 + */ +public final class Doubles { + private Doubles() {} + + /** + * The number of bytes required to represent a primitive {@code double} value. + * + *

Java 8 users: use {@link Double#BYTES} instead. + * + * @since 10.0 + */ + public static final int BYTES = Double.SIZE / Byte.SIZE; + + /** + * Returns a hash code for {@code value}; equal to the result of invoking {@code ((Double) + * value).hashCode()}. + * + *

Java 8 users: use {@link Double#hashCode(double)} instead. + * + * @param value a primitive {@code double} value + * @return a hash code for the value + */ + public static int hashCode(double value) { + return ((Double) value).hashCode(); + // TODO(kevinb): do it this way when we can (GWT problem): + // long bits = Double.doubleToLongBits(value); + // return (int) (bits ^ (bits >>> 32)); + } + + /** + * Compares the two specified {@code double} values. The sign of the value returned is the same as + * that of ((Double) a).{@linkplain Double#compareTo compareTo}(b). As with that + * method, {@code NaN} is treated as greater than all other values, and {@code 0.0 > -0.0}. + * + *

Note: this method simply delegates to the JDK method {@link Double#compare}. It is + * provided for consistency with the other primitive types, whose compare methods were not added + * to the JDK until JDK 7. + * + * @param a the first {@code double} to compare + * @param b the second {@code double} to compare + * @return a negative value if {@code a} is less than {@code b}; a positive value if {@code a} is + * greater than {@code b}; or zero if they are equal + */ + public static int compare(double a, double b) { + return Double.compare(a, b); + } + + /** + * Returns {@code true} if {@code value} represents a real number. This is equivalent to, but not + * necessarily implemented as, {@code !(Double.isInfinite(value) || Double.isNaN(value))}. + * + *

Java 8 users: use {@link Double#isFinite(double)} instead. + * + * @since 10.0 + */ + public static boolean isFinite(double value) { + return NEGATIVE_INFINITY < value && value < POSITIVE_INFINITY; + } + + /** + * Returns {@code true} if {@code target} is present as an element anywhere in {@code array}. Note + * that this always returns {@code false} when {@code target} is {@code NaN}. + * + * @param array an array of {@code double} values, possibly empty + * @param target a primitive {@code double} value + * @return {@code true} if {@code array[i] == target} for some value of {@code i} + */ + public static boolean contains(double[] array, double target) { + for (double value : array) { + if (value == target) { + return true; + } + } + return false; + } + + /** + * Returns the index of the first appearance of the value {@code target} in {@code array}. Note + * that this always returns {@code -1} when {@code target} is {@code NaN}. + * + * @param array an array of {@code double} values, possibly empty + * @param target a primitive {@code double} value + * @return the least index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int indexOf(double[] array, double target) { + return indexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int indexOf(double[] array, double target, int start, int end) { + for (int i = start; i < end; i++) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the start position of the first occurrence of the specified {@code target} within + * {@code array}, or {@code -1} if there is no such occurrence. + * + *

More formally, returns the lowest index {@code i} such that {@code Arrays.copyOfRange(array, + * i, i + target.length)} contains exactly the same elements as {@code target}. + * + *

Note that this always returns {@code -1} when {@code target} contains {@code NaN}. + * + * @param array the array to search for the sequence {@code target} + * @param target the array to search for as a sub-sequence of {@code array} + */ + public static int indexOf(double[] array, double[] target) { + checkNotNull(array, "array"); + checkNotNull(target, "target"); + if (target.length == 0) { + return 0; + } + + outer: + for (int i = 0; i < array.length - target.length + 1; i++) { + for (int j = 0; j < target.length; j++) { + if (array[i + j] != target[j]) { + continue outer; + } + } + return i; + } + return -1; + } + + /** + * Returns the index of the last appearance of the value {@code target} in {@code array}. Note + * that this always returns {@code -1} when {@code target} is {@code NaN}. + * + * @param array an array of {@code double} values, possibly empty + * @param target a primitive {@code double} value + * @return the greatest index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int lastIndexOf(double[] array, double target) { + return lastIndexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int lastIndexOf(double[] array, double target, int start, int end) { + for (int i = end - 1; i >= start; i--) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the least value present in {@code array}, using the same rules of comparison as {@link + * Math#min(double, double)}. + * + * @param array a nonempty array of {@code double} values + * @return the value present in {@code array} that is less than or equal to every other value in + * the array + * @throws IllegalArgumentException if {@code array} is empty + */ + public static double min(double... array) { + checkArgument(array.length > 0); + double min = array[0]; + for (int i = 1; i < array.length; i++) { + min = Math.min(min, array[i]); + } + return min; + } + + /** + * Returns the greatest value present in {@code array}, using the same rules of comparison as + * {@link Math#max(double, double)}. + * + * @param array a nonempty array of {@code double} values + * @return the value present in {@code array} that is greater than or equal to every other value + * in the array + * @throws IllegalArgumentException if {@code array} is empty + */ + public static double max(double... array) { + checkArgument(array.length > 0); + double max = array[0]; + for (int i = 1; i < array.length; i++) { + max = Math.max(max, array[i]); + } + return max; + } + + /** + * Returns the value nearest to {@code value} which is within the closed range {@code [min..max]}. + * + *

If {@code value} is within the range {@code [min..max]}, {@code value} is returned + * unchanged. If {@code value} is less than {@code min}, {@code min} is returned, and if {@code + * value} is greater than {@code max}, {@code max} is returned. + * + * @param value the {@code double} value to constrain + * @param min the lower bound (inclusive) of the range to constrain {@code value} to + * @param max the upper bound (inclusive) of the range to constrain {@code value} to + * @throws IllegalArgumentException if {@code min > max} + * @since 21.0 + */ + + public static double constrainToRange(double value, double min, double max) { + checkArgument(min <= max, "min (%s) must be less than or equal to max (%s)", min, max); + return Math.min(Math.max(value, min), max); + } + + /** + * Returns the values from each provided array combined into a single array. For example, {@code + * concat(new double[] {a, b}, new double[] {}, new double[] {c}} returns the array {@code {a, b, + * c}}. + * + * @param arrays zero or more {@code double} arrays + * @return a single array containing all the values from the source arrays, in order + */ + public static double[] concat(double[]... arrays) { + int length = 0; + for (double[] array : arrays) { + length += array.length; + } + double[] result = new double[length]; + int pos = 0; + for (double[] array : arrays) { + System.arraycopy(array, 0, result, pos, array.length); + pos += array.length; + } + return result; + } + + private static final class DoubleConverter extends Converter + implements Serializable { + static final DoubleConverter INSTANCE = new DoubleConverter(); + + @Override + protected Double doForward(String value) { + return Double.valueOf(value); + } + + @Override + protected String doBackward(Double value) { + return value.toString(); + } + + @Override + public String toString() { + return "Doubles.stringConverter()"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 1; + } + + /** + * Returns a serializable converter object that converts between strings and doubles using {@link + * Double#valueOf} and {@link Double#toString()}. + * + * @since 16.0 + */ + + public static Converter stringConverter() { + return DoubleConverter.INSTANCE; + } + + /** + * Returns an array containing the same values as {@code array}, but guaranteed to be of a + * specified minimum length. If {@code array} already has a length of at least {@code minLength}, + * it is returned directly. Otherwise, a new array of size {@code minLength + padding} is + * returned, containing the values of {@code array}, and zeroes in the remaining places. + * + * @param array the source array + * @param minLength the minimum length the returned array must guarantee + * @param padding an extra amount to "grow" the array by if growth is necessary + * @throws IllegalArgumentException if {@code minLength} or {@code padding} is negative + * @return an array containing the values of {@code array}, with guaranteed minimum length {@code + * minLength} + */ + public static double[] ensureCapacity(double[] array, int minLength, int padding) { + checkArgument(minLength >= 0, "Invalid minLength: %s", minLength); + checkArgument(padding >= 0, "Invalid padding: %s", padding); + return (array.length < minLength) ? Arrays.copyOf(array, minLength + padding) : array; + } + + /** + * Returns a string containing the supplied {@code double} values, converted to strings as + * specified by {@link Double#toString(double)}, and separated by {@code separator}. For example, + * {@code join("-", 1.0, 2.0, 3.0)} returns the string {@code "1.0-2.0-3.0"}. + * + *

Note that {@link Double#toString(double)} formats {@code double} differently in GWT + * sometimes. In the previous example, it returns the string {@code "1-2-3"}. + * + * @param separator the text that should appear between consecutive values in the resulting string + * (but not at the start or end) + * @param array an array of {@code double} values, possibly empty + */ + public static String join(String separator, double... array) { + checkNotNull(separator); + if (array.length == 0) { + return ""; + } + + // For pre-sizing a builder, just get the right order of magnitude + StringBuilder builder = new StringBuilder(array.length * 12); + builder.append(array[0]); + for (int i = 1; i < array.length; i++) { + builder.append(separator).append(array[i]); + } + return builder.toString(); + } + + /** + * Returns a comparator that compares two {@code double} arrays lexicographically. That is, it + * compares, using {@link #compare(double, double)}), the first pair of values that follow any + * common prefix, or when one array is a prefix of the other, treats the shorter array as the + * lesser. For example, {@code [] < [1.0] < [1.0, 2.0] < [2.0]}. + * + *

The returned comparator is inconsistent with {@link Object#equals(Object)} (since arrays + * support only identity equality), but it is consistent with {@link Arrays#equals(double[], + * double[])}. + * + * @since 2.0 + */ + public static Comparator lexicographicalComparator() { + return LexicographicalComparator.INSTANCE; + } + + private enum LexicographicalComparator implements Comparator { + INSTANCE; + + @Override + public int compare(double[] left, double[] right) { + int minLength = Math.min(left.length, right.length); + for (int i = 0; i < minLength; i++) { + int result = Double.compare(left[i], right[i]); + if (result != 0) { + return result; + } + } + return left.length - right.length; + } + + @Override + public String toString() { + return "Doubles.lexicographicalComparator()"; + } + } + + /** + * Sorts the elements of {@code array} in descending order. + * + *

Note that this method uses the total order imposed by {@link Double#compare}, which treats + * all NaN values as equal and 0.0 as greater than -0.0. + * + * @since 23.1 + */ + public static void sortDescending(double[] array) { + checkNotNull(array); + sortDescending(array, 0, array.length); + } + + /** + * Sorts the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive in descending order. + * + *

Note that this method uses the total order imposed by {@link Double#compare}, which treats + * all NaN values as equal and 0.0 as greater than -0.0. + * + * @since 23.1 + */ + public static void sortDescending(double[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + Arrays.sort(array, fromIndex, toIndex); + reverse(array, fromIndex, toIndex); + } + + /** + * Reverses the elements of {@code array}. This is equivalent to {@code + * Collections.reverse(Doubles.asList(array))}, but is likely to be more efficient. + * + * @since 23.1 + */ + public static void reverse(double[] array) { + checkNotNull(array); + reverse(array, 0, array.length); + } + + /** + * Reverses the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive. This is equivalent to {@code + * Collections.reverse(Doubles.asList(array).subList(fromIndex, toIndex))}, but is likely to be + * more efficient. + * + * @throws IndexOutOfBoundsException if {@code fromIndex < 0}, {@code toIndex > array.length}, or + * {@code toIndex > fromIndex} + * @since 23.1 + */ + public static void reverse(double[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + for (int i = fromIndex, j = toIndex - 1; i < j; i++, j--) { + double tmp = array[i]; + array[i] = array[j]; + array[j] = tmp; + } + } + + /** + * Returns an array containing each value of {@code collection}, converted to a {@code double} + * value in the manner of {@link Number#doubleValue}. + * + *

Elements are copied from the argument collection as if by {@code collection.toArray()}. + * Calling this method is as thread-safe as calling that method. + * + * @param collection a collection of {@code Number} instances + * @return an array containing the same values as {@code collection}, in the same order, converted + * to primitives + * @throws NullPointerException if {@code collection} or any of its elements is null + * @since 1.0 (parameter was {@code Collection} before 12.0) + */ + public static double[] toArray(Collection collection) { + if (collection instanceof DoubleArrayAsList) { + return ((DoubleArrayAsList) collection).toDoubleArray(); + } + + Object[] boxedArray = collection.toArray(); + int len = boxedArray.length; + double[] array = new double[len]; + for (int i = 0; i < len; i++) { + // checkNotNull for GWT (do not optimize) + array[i] = ((Number) checkNotNull(boxedArray[i])).doubleValue(); + } + return array; + } + + /** + * Returns a fixed-size list backed by the specified array, similar to {@link + * Arrays#asList(Object[])}. The list supports {@link List#set(int, Object)}, but any attempt to + * set a value to {@code null} will result in a {@link NullPointerException}. + * + *

The returned list maintains the values, but not the identities, of {@code Double} objects + * written to or read from it. For example, whether {@code list.get(0) == list.get(0)} is true for + * the returned list is unspecified. + * + *

The returned list may have unexpected behavior if it contains {@code NaN}, or if {@code NaN} + * is used as a parameter to any of its methods. + * + *

Note: when possible, you should represent your data as an {@link + * ImmutableDoubleArray} instead, which has an {@link ImmutableDoubleArray#asList asList} view. + * + * @param backingArray the array to back the list + * @return a list view of the array + */ + public static List asList(double... backingArray) { + if (backingArray.length == 0) { + return Collections.emptyList(); + } + return new DoubleArrayAsList(backingArray); + } + + private static class DoubleArrayAsList extends AbstractList + implements RandomAccess, Serializable { + final double[] array; + final int start; + final int end; + + DoubleArrayAsList(double[] array) { + this(array, 0, array.length); + } + + DoubleArrayAsList(double[] array, int start, int end) { + this.array = array; + this.start = start; + this.end = end; + } + + @Override + public int size() { + return end - start; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Double get(int index) { + checkElementIndex(index, size()); + return array[start + index]; + } + + @Override + public Spliterator.OfDouble spliterator() { + return Spliterators.spliterator(array, start, end, 0); + } + + @Override + public boolean contains(Object target) { + // Overridden to prevent a ton of boxing + return (target instanceof Double) + && Doubles.indexOf(array, (Double) target, start, end) != -1; + } + + @Override + public int indexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Double) { + int i = Doubles.indexOf(array, (Double) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public int lastIndexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Double) { + int i = Doubles.lastIndexOf(array, (Double) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public Double set(int index, Double element) { + checkElementIndex(index, size()); + double oldValue = array[start + index]; + // checkNotNull for GWT (do not optimize) + array[start + index] = checkNotNull(element); + return oldValue; + } + + @Override + public List subList(int fromIndex, int toIndex) { + int size = size(); + checkPositionIndexes(fromIndex, toIndex, size); + if (fromIndex == toIndex) { + return Collections.emptyList(); + } + return new DoubleArrayAsList(array, start + fromIndex, start + toIndex); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof DoubleArrayAsList) { + DoubleArrayAsList that = (DoubleArrayAsList) object; + int size = size(); + if (that.size() != size) { + return false; + } + for (int i = 0; i < size; i++) { + if (array[start + i] != that.array[that.start + i]) { + return false; + } + } + return true; + } + return super.equals(object); + } + + @Override + public int hashCode() { + int result = 1; + for (int i = start; i < end; i++) { + result = 31 * result + Doubles.hashCode(array[i]); + } + return result; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(size() * 12); + builder.append('[').append(array[start]); + for (int i = start + 1; i < end; i++) { + builder.append(", ").append(array[i]); + } + return builder.append(']').toString(); + } + + double[] toDoubleArray() { + return Arrays.copyOfRange(array, start, end); + } + + private static final long serialVersionUID = 0; + } + + /** + * This is adapted from the regex suggested by {@link Double#valueOf(String)} for prevalidating + * inputs. All valid inputs must pass this regex, but it's semantically fine if not all inputs + * that pass this regex are valid -- only a performance hit is incurred, not a semantics bug. + */ + static final Pattern FLOATING_POINT_PATTERN = fpPattern(); + + private static Pattern fpPattern() { + String decimal = "(?:\\d++(?:\\.\\d*+)?|\\.\\d++)"; + String completeDec = decimal + "(?:[eE][+-]?\\d++)?[fFdD]?"; + String hex = "(?:\\p{XDigit}++(?:\\.\\p{XDigit}*+)?|\\.\\p{XDigit}++)"; + String completeHex = "0[xX]" + hex + "[pP][+-]?\\d++[fFdD]?"; + String fpPattern = "[+-]?(?:NaN|Infinity|" + completeDec + "|" + completeHex + ")"; + return Pattern.compile(fpPattern); + } + + /** + * Parses the specified string as a double-precision floating point value. The ASCII character + * {@code '-'} ('\u002D') is recognized as the minus sign. + * + *

Unlike {@link Double#parseDouble(String)}, this method returns {@code null} instead of + * throwing an exception if parsing fails. Valid inputs are exactly those accepted by {@link + * Double#valueOf(String)}, except that leading and trailing whitespace is not permitted. + * + *

This implementation is likely to be faster than {@code Double.parseDouble} if many failures + * are expected. + * + * @param string the string representation of a {@code double} value + * @return the floating point value represented by {@code string}, or {@code null} if {@code + * string} has a length of zero or cannot be parsed as a {@code double} value + * @since 14.0 + */ + + + public static Double tryParse(String string) { + if (FLOATING_POINT_PATTERN.matcher(string).matches()) { + // TODO(lowasser): could be potentially optimized, but only with + // extensive testing + try { + return Double.parseDouble(string); + } catch (NumberFormatException e) { + // Double.parseDouble has changed specs several times, so fall through + // gracefully + } + } + return null; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Ints.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Ints.java new file mode 100644 index 0000000000000..2d76554f3054b --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Ints.java @@ -0,0 +1,743 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Converter; + +import java.io.Serializable; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.RandomAccess; +import java.util.Spliterator; +import java.util.Spliterators; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; + +/** + * Static utility methods pertaining to {@code int} primitives, that are not already found in either + * {@link Integer} or {@link Arrays}. + * + *

See the Guava User Guide article on primitive utilities. + * + * @author Kevin Bourrillion + * @since 1.0 + */ +public final class Ints { + private Ints() {} + + /** + * The number of bytes required to represent a primitive {@code int} value. + * + *

Java 8 users: use {@link Integer#BYTES} instead. + */ + public static final int BYTES = Integer.SIZE / Byte.SIZE; + + /** + * The largest power of two that can be represented as an {@code int}. + * + * @since 10.0 + */ + public static final int MAX_POWER_OF_TWO = 1 << (Integer.SIZE - 2); + + /** + * Returns a hash code for {@code value}; equal to the result of invoking {@code ((Integer) + * value).hashCode()}. + * + *

Java 8 users: use {@link Integer#hashCode(int)} instead. + * + * @param value a primitive {@code int} value + * @return a hash code for the value + */ + public static int hashCode(int value) { + return value; + } + + /** + * Returns the {@code int} value that is equal to {@code value}, if possible. + * + * @param value any value in the range of the {@code int} type + * @return the {@code int} value that equals {@code value} + * @throws IllegalArgumentException if {@code value} is greater than {@link Integer#MAX_VALUE} or + * less than {@link Integer#MIN_VALUE} + */ + public static int checkedCast(long value) { + int result = (int) value; + checkArgument(result == value, "Out of range: %s", value); + return result; + } + + /** + * Returns the {@code int} nearest in value to {@code value}. + * + * @param value any {@code long} value + * @return the same value cast to {@code int} if it is in the range of the {@code int} type, + * {@link Integer#MAX_VALUE} if it is too large, or {@link Integer#MIN_VALUE} if it is too + * small + */ + public static int saturatedCast(long value) { + if (value > Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } + if (value < Integer.MIN_VALUE) { + return Integer.MIN_VALUE; + } + return (int) value; + } + + /** + * Compares the two specified {@code int} values. The sign of the value returned is the same as + * that of {@code ((Integer) a).compareTo(b)}. + * + *

Note for Java 7 and later: this method should be treated as deprecated; use the + * equivalent {@link Integer#compare} method instead. + * + * @param a the first {@code int} to compare + * @param b the second {@code int} to compare + * @return a negative value if {@code a} is less than {@code b}; a positive value if {@code a} is + * greater than {@code b}; or zero if they are equal + */ + public static int compare(int a, int b) { + return (a < b) ? -1 : ((a > b) ? 1 : 0); + } + + /** + * Returns {@code true} if {@code target} is present as an element anywhere in {@code array}. + * + * @param array an array of {@code int} values, possibly empty + * @param target a primitive {@code int} value + * @return {@code true} if {@code array[i] == target} for some value of {@code i} + */ + public static boolean contains(int[] array, int target) { + for (int value : array) { + if (value == target) { + return true; + } + } + return false; + } + + /** + * Returns the index of the first appearance of the value {@code target} in {@code array}. + * + * @param array an array of {@code int} values, possibly empty + * @param target a primitive {@code int} value + * @return the least index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int indexOf(int[] array, int target) { + return indexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int indexOf(int[] array, int target, int start, int end) { + for (int i = start; i < end; i++) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the start position of the first occurrence of the specified {@code target} within + * {@code array}, or {@code -1} if there is no such occurrence. + * + *

More formally, returns the lowest index {@code i} such that {@code Arrays.copyOfRange(array, + * i, i + target.length)} contains exactly the same elements as {@code target}. + * + * @param array the array to search for the sequence {@code target} + * @param target the array to search for as a sub-sequence of {@code array} + */ + public static int indexOf(int[] array, int[] target) { + checkNotNull(array, "array"); + checkNotNull(target, "target"); + if (target.length == 0) { + return 0; + } + + outer: + for (int i = 0; i < array.length - target.length + 1; i++) { + for (int j = 0; j < target.length; j++) { + if (array[i + j] != target[j]) { + continue outer; + } + } + return i; + } + return -1; + } + + /** + * Returns the index of the last appearance of the value {@code target} in {@code array}. + * + * @param array an array of {@code int} values, possibly empty + * @param target a primitive {@code int} value + * @return the greatest index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int lastIndexOf(int[] array, int target) { + return lastIndexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int lastIndexOf(int[] array, int target, int start, int end) { + for (int i = end - 1; i >= start; i--) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the least value present in {@code array}. + * + * @param array a nonempty array of {@code int} values + * @return the value present in {@code array} that is less than or equal to every other value in + * the array + * @throws IllegalArgumentException if {@code array} is empty + */ + public static int min(int... array) { + checkArgument(array.length > 0); + int min = array[0]; + for (int i = 1; i < array.length; i++) { + if (array[i] < min) { + min = array[i]; + } + } + return min; + } + + /** + * Returns the greatest value present in {@code array}. + * + * @param array a nonempty array of {@code int} values + * @return the value present in {@code array} that is greater than or equal to every other value + * in the array + * @throws IllegalArgumentException if {@code array} is empty + */ + public static int max(int... array) { + checkArgument(array.length > 0); + int max = array[0]; + for (int i = 1; i < array.length; i++) { + if (array[i] > max) { + max = array[i]; + } + } + return max; + } + + /** + * Returns the value nearest to {@code value} which is within the closed range {@code [min..max]}. + * + *

If {@code value} is within the range {@code [min..max]}, {@code value} is returned + * unchanged. If {@code value} is less than {@code min}, {@code min} is returned, and if {@code + * value} is greater than {@code max}, {@code max} is returned. + * + * @param value the {@code int} value to constrain + * @param min the lower bound (inclusive) of the range to constrain {@code value} to + * @param max the upper bound (inclusive) of the range to constrain {@code value} to + * @throws IllegalArgumentException if {@code min > max} + * @since 21.0 + */ + + public static int constrainToRange(int value, int min, int max) { + checkArgument(min <= max, "min (%s) must be less than or equal to max (%s)", min, max); + return Math.min(Math.max(value, min), max); + } + + /** + * Returns the values from each provided array combined into a single array. For example, {@code + * concat(new int[] {a, b}, new int[] {}, new int[] {c}} returns the array {@code {a, b, c}}. + * + * @param arrays zero or more {@code int} arrays + * @return a single array containing all the values from the source arrays, in order + */ + public static int[] concat(int[]... arrays) { + int length = 0; + for (int[] array : arrays) { + length += array.length; + } + int[] result = new int[length]; + int pos = 0; + for (int[] array : arrays) { + System.arraycopy(array, 0, result, pos, array.length); + pos += array.length; + } + return result; + } + + /** + * Returns a big-endian representation of {@code value} in a 4-element byte array; equivalent to + * {@code ByteBuffer.allocate(4).putInt(value).array()}. For example, the input value {@code + * 0x12131415} would yield the byte array {@code {0x12, 0x13, 0x14, 0x15}}. + * + *

If you need to convert and concatenate several values (possibly even of different types), + * use a shared {@link java.nio.ByteBuffer} instance + */ + public static byte[] toByteArray(int value) { + return new byte[] { + (byte) (value >> 24), (byte) (value >> 16), (byte) (value >> 8), (byte) value + }; + } + + /** + * Returns the {@code int} value whose big-endian representation is stored in the first 4 bytes of + * {@code bytes}; equivalent to {@code ByteBuffer.wrap(bytes).getInt()}. For example, the input + * byte array {@code {0x12, 0x13, 0x14, 0x15, 0x33}} would yield the {@code int} value {@code + * 0x12131415}. + * + *

Arguably, it's preferable to use {@link java.nio.ByteBuffer}; that library exposes much more + * flexibility at little cost in readability. + * + * @throws IllegalArgumentException if {@code bytes} has fewer than 4 elements + */ + public static int fromByteArray(byte[] bytes) { + checkArgument(bytes.length >= BYTES, "array too small: %s < %s", bytes.length, BYTES); + return fromBytes(bytes[0], bytes[1], bytes[2], bytes[3]); + } + + /** + * Returns the {@code int} value whose byte representation is the given 4 bytes, in big-endian + * order; equivalent to {@code Ints.fromByteArray(new byte[] {b1, b2, b3, b4})}. + * + * @since 7.0 + */ + public static int fromBytes(byte b1, byte b2, byte b3, byte b4) { + return b1 << 24 | (b2 & 0xFF) << 16 | (b3 & 0xFF) << 8 | (b4 & 0xFF); + } + + private static final class IntConverter extends Converter + implements Serializable { + static final IntConverter INSTANCE = new IntConverter(); + + @Override + protected Integer doForward(String value) { + return Integer.decode(value); + } + + @Override + protected String doBackward(Integer value) { + return value.toString(); + } + + @Override + public String toString() { + return "Ints.stringConverter()"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 1; + } + + /** + * Returns a serializable converter object that converts between strings and integers using {@link + * Integer#decode} and {@link Integer#toString()}. The returned converter throws {@link + * NumberFormatException} if the input string is invalid. + * + *

Warning: please see {@link Integer#decode} to understand exactly how strings are + * parsed. For example, the string {@code "0123"} is treated as octal and converted to the + * value {@code 83}. + * + * @since 16.0 + */ + + public static Converter stringConverter() { + return IntConverter.INSTANCE; + } + + /** + * Returns an array containing the same values as {@code array}, but guaranteed to be of a + * specified minimum length. If {@code array} already has a length of at least {@code minLength}, + * it is returned directly. Otherwise, a new array of size {@code minLength + padding} is + * returned, containing the values of {@code array}, and zeroes in the remaining places. + * + * @param array the source array + * @param minLength the minimum length the returned array must guarantee + * @param padding an extra amount to "grow" the array by if growth is necessary + * @throws IllegalArgumentException if {@code minLength} or {@code padding} is negative + * @return an array containing the values of {@code array}, with guaranteed minimum length {@code + * minLength} + */ + public static int[] ensureCapacity(int[] array, int minLength, int padding) { + checkArgument(minLength >= 0, "Invalid minLength: %s", minLength); + checkArgument(padding >= 0, "Invalid padding: %s", padding); + return (array.length < minLength) ? Arrays.copyOf(array, minLength + padding) : array; + } + + /** + * Returns a string containing the supplied {@code int} values separated by {@code separator}. For + * example, {@code join("-", 1, 2, 3)} returns the string {@code "1-2-3"}. + * + * @param separator the text that should appear between consecutive values in the resulting string + * (but not at the start or end) + * @param array an array of {@code int} values, possibly empty + */ + public static String join(String separator, int... array) { + checkNotNull(separator); + if (array.length == 0) { + return ""; + } + + // For pre-sizing a builder, just get the right order of magnitude + StringBuilder builder = new StringBuilder(array.length * 5); + builder.append(array[0]); + for (int i = 1; i < array.length; i++) { + builder.append(separator).append(array[i]); + } + return builder.toString(); + } + + /** + * Returns a comparator that compares two {@code int} arrays lexicographically. That is, it + * compares, using {@link #compare(int, int)}), the first pair of values that follow any common + * prefix, or when one array is a prefix of the other, treats the shorter array as the lesser. For + * example, {@code [] < [1] < [1, 2] < [2]}. + * + *

The returned comparator is inconsistent with {@link Object#equals(Object)} (since arrays + * support only identity equality), but it is consistent with {@link Arrays#equals(int[], int[])}. + * + * @since 2.0 + */ + public static Comparator lexicographicalComparator() { + return LexicographicalComparator.INSTANCE; + } + + private enum LexicographicalComparator implements Comparator { + INSTANCE; + + @Override + public int compare(int[] left, int[] right) { + int minLength = Math.min(left.length, right.length); + for (int i = 0; i < minLength; i++) { + int result = Ints.compare(left[i], right[i]); + if (result != 0) { + return result; + } + } + return left.length - right.length; + } + + @Override + public String toString() { + return "Ints.lexicographicalComparator()"; + } + } + + /** + * Sorts the elements of {@code array} in descending order. + * + * @since 23.1 + */ + public static void sortDescending(int[] array) { + checkNotNull(array); + sortDescending(array, 0, array.length); + } + + /** + * Sorts the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive in descending order. + * + * @since 23.1 + */ + public static void sortDescending(int[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + Arrays.sort(array, fromIndex, toIndex); + reverse(array, fromIndex, toIndex); + } + + /** + * Reverses the elements of {@code array}. This is equivalent to {@code + * Collections.reverse(Ints.asList(array))}, but is likely to be more efficient. + * + * @since 23.1 + */ + public static void reverse(int[] array) { + checkNotNull(array); + reverse(array, 0, array.length); + } + + /** + * Reverses the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive. This is equivalent to {@code + * Collections.reverse(Ints.asList(array).subList(fromIndex, toIndex))}, but is likely to be more + * efficient. + * + * @throws IndexOutOfBoundsException if {@code fromIndex < 0}, {@code toIndex > array.length}, or + * {@code toIndex > fromIndex} + * @since 23.1 + */ + public static void reverse(int[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + for (int i = fromIndex, j = toIndex - 1; i < j; i++, j--) { + int tmp = array[i]; + array[i] = array[j]; + array[j] = tmp; + } + } + + /** + * Returns an array containing each value of {@code collection}, converted to a {@code int} value + * in the manner of {@link Number#intValue}. + * + *

Elements are copied from the argument collection as if by {@code collection.toArray()}. + * Calling this method is as thread-safe as calling that method. + * + * @param collection a collection of {@code Number} instances + * @return an array containing the same values as {@code collection}, in the same order, converted + * to primitives + * @throws NullPointerException if {@code collection} or any of its elements is null + * @since 1.0 (parameter was {@code Collection} before 12.0) + */ + public static int[] toArray(Collection collection) { + if (collection instanceof IntArrayAsList) { + return ((IntArrayAsList) collection).toIntArray(); + } + + Object[] boxedArray = collection.toArray(); + int len = boxedArray.length; + int[] array = new int[len]; + for (int i = 0; i < len; i++) { + // checkNotNull for GWT (do not optimize) + array[i] = ((Number) checkNotNull(boxedArray[i])).intValue(); + } + return array; + } + + /** + * Returns a fixed-size list backed by the specified array, similar to {@link + * Arrays#asList(Object[])}. The list supports {@link List#set(int, Object)}, but any attempt to + * set a value to {@code null} will result in a {@link NullPointerException}. + * + *

The returned list maintains the values, but not the identities, of {@code Integer} objects + * written to or read from it. For example, whether {@code list.get(0) == list.get(0)} is true for + * the returned list is unspecified. + * + *

Note: when possible, you should represent your data as an {@link ImmutableIntArray} + * instead, which has an {@link ImmutableIntArray#asList asList} view. + * + * @param backingArray the array to back the list + * @return a list view of the array + */ + public static List asList(int... backingArray) { + if (backingArray.length == 0) { + return Collections.emptyList(); + } + return new IntArrayAsList(backingArray); + } + + private static class IntArrayAsList extends AbstractList + implements RandomAccess, Serializable { + final int[] array; + final int start; + final int end; + + IntArrayAsList(int[] array) { + this(array, 0, array.length); + } + + IntArrayAsList(int[] array, int start, int end) { + this.array = array; + this.start = start; + this.end = end; + } + + @Override + public int size() { + return end - start; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Integer get(int index) { + checkElementIndex(index, size()); + return array[start + index]; + } + + @Override + public Spliterator.OfInt spliterator() { + return Spliterators.spliterator(array, start, end, 0); + } + + @Override + public boolean contains(Object target) { + // Overridden to prevent a ton of boxing + return (target instanceof Integer) && Ints.indexOf(array, (Integer) target, start, end) != -1; + } + + @Override + public int indexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Integer) { + int i = Ints.indexOf(array, (Integer) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public int lastIndexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Integer) { + int i = Ints.lastIndexOf(array, (Integer) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public Integer set(int index, Integer element) { + checkElementIndex(index, size()); + int oldValue = array[start + index]; + // checkNotNull for GWT (do not optimize) + array[start + index] = checkNotNull(element); + return oldValue; + } + + @Override + public List subList(int fromIndex, int toIndex) { + int size = size(); + checkPositionIndexes(fromIndex, toIndex, size); + if (fromIndex == toIndex) { + return Collections.emptyList(); + } + return new IntArrayAsList(array, start + fromIndex, start + toIndex); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof IntArrayAsList) { + IntArrayAsList that = (IntArrayAsList) object; + int size = size(); + if (that.size() != size) { + return false; + } + for (int i = 0; i < size; i++) { + if (array[start + i] != that.array[that.start + i]) { + return false; + } + } + return true; + } + return super.equals(object); + } + + @Override + public int hashCode() { + int result = 1; + for (int i = start; i < end; i++) { + result = 31 * result + Ints.hashCode(array[i]); + } + return result; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(size() * 5); + builder.append('[').append(array[start]); + for (int i = start + 1; i < end; i++) { + builder.append(", ").append(array[i]); + } + return builder.append(']').toString(); + } + + int[] toIntArray() { + return Arrays.copyOfRange(array, start, end); + } + + private static final long serialVersionUID = 0; + } + + /** + * Parses the specified string as a signed decimal integer value. The ASCII character {@code '-'} + * ('\u002D') is recognized as the minus sign. + * + *

Unlike {@link Integer#parseInt(String)}, this method returns {@code null} instead of + * throwing an exception if parsing fails. Additionally, this method only accepts ASCII digits, + * and returns {@code null} if non-ASCII digits are present in the string. + * + *

Note that strings prefixed with ASCII {@code '+'} are rejected, even under JDK 7, despite + * the change to {@link Integer#parseInt(String)} for that version. + * + * @param string the string representation of an integer value + * @return the integer value represented by {@code string}, or {@code null} if {@code string} has + * a length of zero or cannot be parsed as an integer value + * @since 11.0 + */ + + + public static Integer tryParse(String string) { + return tryParse(string, 10); + } + + /** + * Parses the specified string as a signed integer value using the specified radix. The ASCII + * character {@code '-'} ('\u002D') is recognized as the minus sign. + * + *

Unlike {@link Integer#parseInt(String, int)}, this method returns {@code null} instead of + * throwing an exception if parsing fails. Additionally, this method only accepts ASCII digits, + * and returns {@code null} if non-ASCII digits are present in the string. + * + *

Note that strings prefixed with ASCII {@code '+'} are rejected, even under JDK 7, despite + * the change to {@link Integer#parseInt(String, int)} for that version. + * + * @param string the string representation of an integer value + * @param radix the radix to use when parsing + * @return the integer value represented by {@code string} using {@code radix}, or {@code null} if + * {@code string} has a length of zero or cannot be parsed as an integer value + * @throws IllegalArgumentException if {@code radix < Character.MIN_RADIX} or {@code radix > + * Character.MAX_RADIX} + * @since 19.0 + */ + + + public static Integer tryParse(String string, int radix) { + Long result = Longs.tryParse(string, radix); + if (result == null || result.longValue() != result.intValue()) { + return null; + } else { + return result.intValue(); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Longs.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Longs.java new file mode 100644 index 0000000000000..c9822abed502c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Longs.java @@ -0,0 +1,791 @@ +/* + * Copyright (C) 2008 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Converter; + +import java.io.Serializable; +import java.util.AbstractList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.RandomAccess; +import java.util.Spliterator; +import java.util.Spliterators; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkElementIndex; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; + + +/** + * Static utility methods pertaining to {@code long} primitives, that are not already found in + * either {@link Long} or {@link Arrays}. + * + *

See the Guava User Guide article on primitive utilities. + * + * @author Kevin Bourrillion + * @since 1.0 + */ +public final class Longs { + private Longs() {} + + /** + * The number of bytes required to represent a primitive {@code long} value. + * + *

Java 8 users: use {@link Long#BYTES} instead. + */ + public static final int BYTES = Long.SIZE / Byte.SIZE; + + /** + * The largest power of two that can be represented as a {@code long}. + * + * @since 10.0 + */ + public static final long MAX_POWER_OF_TWO = 1L << (Long.SIZE - 2); + + /** + * Returns a hash code for {@code value}; equal to the result of invoking {@code ((Long) + * value).hashCode()}. + * + *

This method always return the value specified by {@link Long#hashCode()} in java, which + * might be different from {@code ((Long) value).hashCode()} in GWT because {@link + * Long#hashCode()} in GWT does not obey the JRE contract. + * + *

Java 8 users: use {@link Long#hashCode(long)} instead. + * + * @param value a primitive {@code long} value + * @return a hash code for the value + */ + public static int hashCode(long value) { + return (int) (value ^ (value >>> 32)); + } + + /** + * Compares the two specified {@code long} values. The sign of the value returned is the same as + * that of {@code ((Long) a).compareTo(b)}. + * + *

Note for Java 7 and later: this method should be treated as deprecated; use the + * equivalent {@link Long#compare} method instead. + * + * @param a the first {@code long} to compare + * @param b the second {@code long} to compare + * @return a negative value if {@code a} is less than {@code b}; a positive value if {@code a} is + * greater than {@code b}; or zero if they are equal + */ + public static int compare(long a, long b) { + return (a < b) ? -1 : ((a > b) ? 1 : 0); + } + + /** + * Returns {@code true} if {@code target} is present as an element anywhere in {@code array}. + * + * @param array an array of {@code long} values, possibly empty + * @param target a primitive {@code long} value + * @return {@code true} if {@code array[i] == target} for some value of {@code i} + */ + public static boolean contains(long[] array, long target) { + for (long value : array) { + if (value == target) { + return true; + } + } + return false; + } + + /** + * Returns the index of the first appearance of the value {@code target} in {@code array}. + * + * @param array an array of {@code long} values, possibly empty + * @param target a primitive {@code long} value + * @return the least index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int indexOf(long[] array, long target) { + return indexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int indexOf(long[] array, long target, int start, int end) { + for (int i = start; i < end; i++) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the start position of the first occurrence of the specified {@code target} within + * {@code array}, or {@code -1} if there is no such occurrence. + * + *

More formally, returns the lowest index {@code i} such that {@code Arrays.copyOfRange(array, + * i, i + target.length)} contains exactly the same elements as {@code target}. + * + * @param array the array to search for the sequence {@code target} + * @param target the array to search for as a sub-sequence of {@code array} + */ + public static int indexOf(long[] array, long[] target) { + checkNotNull(array, "array"); + checkNotNull(target, "target"); + if (target.length == 0) { + return 0; + } + + outer: + for (int i = 0; i < array.length - target.length + 1; i++) { + for (int j = 0; j < target.length; j++) { + if (array[i + j] != target[j]) { + continue outer; + } + } + return i; + } + return -1; + } + + /** + * Returns the index of the last appearance of the value {@code target} in {@code array}. + * + * @param array an array of {@code long} values, possibly empty + * @param target a primitive {@code long} value + * @return the greatest index {@code i} for which {@code array[i] == target}, or {@code -1} if no + * such index exists. + */ + public static int lastIndexOf(long[] array, long target) { + return lastIndexOf(array, target, 0, array.length); + } + + // TODO(kevinb): consider making this public + private static int lastIndexOf(long[] array, long target, int start, int end) { + for (int i = end - 1; i >= start; i--) { + if (array[i] == target) { + return i; + } + } + return -1; + } + + /** + * Returns the least value present in {@code array}. + * + * @param array a nonempty array of {@code long} values + * @return the value present in {@code array} that is less than or equal to every other value in + * the array + * @throws IllegalArgumentException if {@code array} is empty + */ + public static long min(long... array) { + checkArgument(array.length > 0); + long min = array[0]; + for (int i = 1; i < array.length; i++) { + if (array[i] < min) { + min = array[i]; + } + } + return min; + } + + /** + * Returns the greatest value present in {@code array}. + * + * @param array a nonempty array of {@code long} values + * @return the value present in {@code array} that is greater than or equal to every other value + * in the array + * @throws IllegalArgumentException if {@code array} is empty + */ + public static long max(long... array) { + checkArgument(array.length > 0); + long max = array[0]; + for (int i = 1; i < array.length; i++) { + if (array[i] > max) { + max = array[i]; + } + } + return max; + } + + /** + * Returns the value nearest to {@code value} which is within the closed range {@code [min..max]}. + * + *

If {@code value} is within the range {@code [min..max]}, {@code value} is returned + * unchanged. If {@code value} is less than {@code min}, {@code min} is returned, and if {@code + * value} is greater than {@code max}, {@code max} is returned. + * + * @param value the {@code long} value to constrain + * @param min the lower bound (inclusive) of the range to constrain {@code value} to + * @param max the upper bound (inclusive) of the range to constrain {@code value} to + * @throws IllegalArgumentException if {@code min > max} + * @since 21.0 + */ + + public static long constrainToRange(long value, long min, long max) { + checkArgument(min <= max, "min (%s) must be less than or equal to max (%s)", min, max); + return Math.min(Math.max(value, min), max); + } + + /** + * Returns the values from each provided array combined into a single array. For example, {@code + * concat(new long[] {a, b}, new long[] {}, new long[] {c}} returns the array {@code {a, b, c}}. + * + * @param arrays zero or more {@code long} arrays + * @return a single array containing all the values from the source arrays, in order + */ + public static long[] concat(long[]... arrays) { + int length = 0; + for (long[] array : arrays) { + length += array.length; + } + long[] result = new long[length]; + int pos = 0; + for (long[] array : arrays) { + System.arraycopy(array, 0, result, pos, array.length); + pos += array.length; + } + return result; + } + + /** + * Returns a big-endian representation of {@code value} in an 8-element byte array; equivalent to + * {@code ByteBuffer.allocate(8).putLong(value).array()}. For example, the input value {@code + * 0x1213141516171819L} would yield the byte array {@code {0x12, 0x13, 0x14, 0x15, 0x16, 0x17, + * 0x18, 0x19}}. + * + *

If you need to convert and concatenate several values (possibly even of different types), + * use a shared {@link java.nio.ByteBuffer} instance. + */ + public static byte[] toByteArray(long value) { + // Note that this code needs to stay compatible with GWT, which has known + // bugs when narrowing byte casts of long values occur. + byte[] result = new byte[8]; + for (int i = 7; i >= 0; i--) { + result[i] = (byte) (value & 0xffL); + value >>= 8; + } + return result; + } + + /** + * Returns the {@code long} value whose big-endian representation is stored in the first 8 bytes + * of {@code bytes}; equivalent to {@code ByteBuffer.wrap(bytes).getLong()}. For example, the + * input byte array {@code {0x12, 0x13, 0x14, 0x15, 0x16, 0x17, 0x18, 0x19}} would yield the + * {@code long} value {@code 0x1213141516171819L}. + * + *

Arguably, it's preferable to use {@link java.nio.ByteBuffer}; that library exposes much more + * flexibility at little cost in readability. + * + * @throws IllegalArgumentException if {@code bytes} has fewer than 8 elements + */ + public static long fromByteArray(byte[] bytes) { + checkArgument(bytes.length >= BYTES, "array too small: %s < %s", bytes.length, BYTES); + return fromBytes( + bytes[0], bytes[1], bytes[2], bytes[3], bytes[4], bytes[5], bytes[6], bytes[7]); + } + + /** + * Returns the {@code long} value whose byte representation is the given 8 bytes, in big-endian + * order; equivalent to {@code Longs.fromByteArray(new byte[] {b1, b2, b3, b4, b5, b6, b7, b8})}. + * + * @since 7.0 + */ + public static long fromBytes( + byte b1, byte b2, byte b3, byte b4, byte b5, byte b6, byte b7, byte b8) { + return (b1 & 0xFFL) << 56 + | (b2 & 0xFFL) << 48 + | (b3 & 0xFFL) << 40 + | (b4 & 0xFFL) << 32 + | (b5 & 0xFFL) << 24 + | (b6 & 0xFFL) << 16 + | (b7 & 0xFFL) << 8 + | (b8 & 0xFFL); + } + + /* + * Moving asciiDigits into this static holder class lets ProGuard eliminate and inline the Longs + * class. + */ + static final class AsciiDigits { + private AsciiDigits() {} + + private static final byte[] asciiDigits; + + static { + byte[] result = new byte[128]; + Arrays.fill(result, (byte) -1); + for (int i = 0; i <= 9; i++) { + result['0' + i] = (byte) i; + } + for (int i = 0; i <= 26; i++) { + result['A' + i] = (byte) (10 + i); + result['a' + i] = (byte) (10 + i); + } + asciiDigits = result; + } + + static int digit(char c) { + return (c < 128) ? asciiDigits[c] : -1; + } + } + + /** + * Parses the specified string as a signed decimal long value. The ASCII character {@code '-'} ( + * '\u002D') is recognized as the minus sign. + * + *

Unlike {@link Long#parseLong(String)}, this method returns {@code null} instead of throwing + * an exception if parsing fails. Additionally, this method only accepts ASCII digits, and returns + * {@code null} if non-ASCII digits are present in the string. + * + *

Note that strings prefixed with ASCII {@code '+'} are rejected, even under JDK 7, despite + * the change to {@link Long#parseLong(String)} for that version. + * + * @param string the string representation of a long value + * @return the long value represented by {@code string}, or {@code null} if {@code string} has a + * length of zero or cannot be parsed as a long value + * @since 14.0 + */ + + + public static Long tryParse(String string) { + return tryParse(string, 10); + } + + /** + * Parses the specified string as a signed long value using the specified radix. The ASCII + * character {@code '-'} ('\u002D') is recognized as the minus sign. + * + *

Unlike {@link Long#parseLong(String, int)}, this method returns {@code null} instead of + * throwing an exception if parsing fails. Additionally, this method only accepts ASCII digits, + * and returns {@code null} if non-ASCII digits are present in the string. + * + *

Note that strings prefixed with ASCII {@code '+'} are rejected, even under JDK 7, despite + * the change to {@link Long#parseLong(String, int)} for that version. + * + * @param string the string representation of an long value + * @param radix the radix to use when parsing + * @return the long value represented by {@code string} using {@code radix}, or {@code null} if + * {@code string} has a length of zero or cannot be parsed as a long value + * @throws IllegalArgumentException if {@code radix < Character.MIN_RADIX} or {@code radix > + * Character.MAX_RADIX} + * @since 19.0 + */ + + + public static Long tryParse(String string, int radix) { + if (checkNotNull(string).isEmpty()) { + return null; + } + if (radix < Character.MIN_RADIX || radix > Character.MAX_RADIX) { + throw new IllegalArgumentException( + "radix must be between MIN_RADIX and MAX_RADIX but was " + radix); + } + boolean negative = string.charAt(0) == '-'; + int index = negative ? 1 : 0; + if (index == string.length()) { + return null; + } + int digit = AsciiDigits.digit(string.charAt(index++)); + if (digit < 0 || digit >= radix) { + return null; + } + long accum = -digit; + + long cap = Long.MIN_VALUE / radix; + + while (index < string.length()) { + digit = AsciiDigits.digit(string.charAt(index++)); + if (digit < 0 || digit >= radix || accum < cap) { + return null; + } + accum *= radix; + if (accum < Long.MIN_VALUE + digit) { + return null; + } + accum -= digit; + } + + if (negative) { + return accum; + } else if (accum == Long.MIN_VALUE) { + return null; + } else { + return -accum; + } + } + + private static final class LongConverter extends Converter implements Serializable { + static final LongConverter INSTANCE = new LongConverter(); + + @Override + protected Long doForward(String value) { + return Long.decode(value); + } + + @Override + protected String doBackward(Long value) { + return value.toString(); + } + + @Override + public String toString() { + return "Longs.stringConverter()"; + } + + private Object readResolve() { + return INSTANCE; + } + + private static final long serialVersionUID = 1; + } + + /** + * Returns a serializable converter object that converts between strings and longs using {@link + * Long#decode} and {@link Long#toString()}. The returned converter throws {@link + * NumberFormatException} if the input string is invalid. + * + *

Warning: please see {@link Long#decode} to understand exactly how strings are parsed. + * For example, the string {@code "0123"} is treated as octal and converted to the value + * {@code 83L}. + * + * @since 16.0 + */ + + public static Converter stringConverter() { + return LongConverter.INSTANCE; + } + + /** + * Returns an array containing the same values as {@code array}, but guaranteed to be of a + * specified minimum length. If {@code array} already has a length of at least {@code minLength}, + * it is returned directly. Otherwise, a new array of size {@code minLength + padding} is + * returned, containing the values of {@code array}, and zeroes in the remaining places. + * + * @param array the source array + * @param minLength the minimum length the returned array must guarantee + * @param padding an extra amount to "grow" the array by if growth is necessary + * @throws IllegalArgumentException if {@code minLength} or {@code padding} is negative + * @return an array containing the values of {@code array}, with guaranteed minimum length {@code + * minLength} + */ + public static long[] ensureCapacity(long[] array, int minLength, int padding) { + checkArgument(minLength >= 0, "Invalid minLength: %s", minLength); + checkArgument(padding >= 0, "Invalid padding: %s", padding); + return (array.length < minLength) ? Arrays.copyOf(array, minLength + padding) : array; + } + + /** + * Returns a string containing the supplied {@code long} values separated by {@code separator}. + * For example, {@code join("-", 1L, 2L, 3L)} returns the string {@code "1-2-3"}. + * + * @param separator the text that should appear between consecutive values in the resulting string + * (but not at the start or end) + * @param array an array of {@code long} values, possibly empty + */ + public static String join(String separator, long... array) { + checkNotNull(separator); + if (array.length == 0) { + return ""; + } + + // For pre-sizing a builder, just get the right order of magnitude + StringBuilder builder = new StringBuilder(array.length * 10); + builder.append(array[0]); + for (int i = 1; i < array.length; i++) { + builder.append(separator).append(array[i]); + } + return builder.toString(); + } + + /** + * Returns a comparator that compares two {@code long} arrays lexicographically. That is, it + * compares, using {@link #compare(long, long)}), the first pair of values that follow any common + * prefix, or when one array is a prefix of the other, treats the shorter array as the lesser. For + * example, {@code [] < [1L] < [1L, 2L] < [2L]}. + * + *

The returned comparator is inconsistent with {@link Object#equals(Object)} (since arrays + * support only identity equality), but it is consistent with {@link Arrays#equals(long[], + * long[])}. + * + * @since 2.0 + */ + public static Comparator lexicographicalComparator() { + return LexicographicalComparator.INSTANCE; + } + + private enum LexicographicalComparator implements Comparator { + INSTANCE; + + @Override + public int compare(long[] left, long[] right) { + int minLength = Math.min(left.length, right.length); + for (int i = 0; i < minLength; i++) { + int result = Longs.compare(left[i], right[i]); + if (result != 0) { + return result; + } + } + return left.length - right.length; + } + + @Override + public String toString() { + return "Longs.lexicographicalComparator()"; + } + } + + /** + * Sorts the elements of {@code array} in descending order. + * + * @since 23.1 + */ + public static void sortDescending(long[] array) { + checkNotNull(array); + sortDescending(array, 0, array.length); + } + + /** + * Sorts the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive in descending order. + * + * @since 23.1 + */ + public static void sortDescending(long[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + Arrays.sort(array, fromIndex, toIndex); + reverse(array, fromIndex, toIndex); + } + + /** + * Reverses the elements of {@code array}. This is equivalent to {@code + * Collections.reverse(Longs.asList(array))}, but is likely to be more efficient. + * + * @since 23.1 + */ + public static void reverse(long[] array) { + checkNotNull(array); + reverse(array, 0, array.length); + } + + /** + * Reverses the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive. This is equivalent to {@code + * Collections.reverse(Longs.asList(array).subList(fromIndex, toIndex))}, but is likely to be more + * efficient. + * + * @throws IndexOutOfBoundsException if {@code fromIndex < 0}, {@code toIndex > array.length}, or + * {@code toIndex > fromIndex} + * @since 23.1 + */ + public static void reverse(long[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + for (int i = fromIndex, j = toIndex - 1; i < j; i++, j--) { + long tmp = array[i]; + array[i] = array[j]; + array[j] = tmp; + } + } + + /** + * Returns an array containing each value of {@code collection}, converted to a {@code long} value + * in the manner of {@link Number#longValue}. + * + *

Elements are copied from the argument collection as if by {@code collection.toArray()}. + * Calling this method is as thread-safe as calling that method. + * + * @param collection a collection of {@code Number} instances + * @return an array containing the same values as {@code collection}, in the same order, converted + * to primitives + * @throws NullPointerException if {@code collection} or any of its elements is null + * @since 1.0 (parameter was {@code Collection} before 12.0) + */ + public static long[] toArray(Collection collection) { + if (collection instanceof LongArrayAsList) { + return ((LongArrayAsList) collection).toLongArray(); + } + + Object[] boxedArray = collection.toArray(); + int len = boxedArray.length; + long[] array = new long[len]; + for (int i = 0; i < len; i++) { + // checkNotNull for GWT (do not optimize) + array[i] = ((Number) checkNotNull(boxedArray[i])).longValue(); + } + return array; + } + + /** + * Returns a fixed-size list backed by the specified array, similar to {@link + * Arrays#asList(Object[])}. The list supports {@link List#set(int, Object)}, but any attempt to + * set a value to {@code null} will result in a {@link NullPointerException}. + * + *

The returned list maintains the values, but not the identities, of {@code Long} objects + * written to or read from it. For example, whether {@code list.get(0) == list.get(0)} is true for + * the returned list is unspecified. + * + *

Note: when possible, you should represent your data as an {@link ImmutableLongArray} + * instead, which has an {@link ImmutableLongArray#asList asList} view. + * + * @param backingArray the array to back the list + * @return a list view of the array + */ + public static List asList(long... backingArray) { + if (backingArray.length == 0) { + return Collections.emptyList(); + } + return new LongArrayAsList(backingArray); + } + + private static class LongArrayAsList extends AbstractList + implements RandomAccess, Serializable { + final long[] array; + final int start; + final int end; + + LongArrayAsList(long[] array) { + this(array, 0, array.length); + } + + LongArrayAsList(long[] array, int start, int end) { + this.array = array; + this.start = start; + this.end = end; + } + + @Override + public int size() { + return end - start; + } + + @Override + public boolean isEmpty() { + return false; + } + + @Override + public Long get(int index) { + checkElementIndex(index, size()); + return array[start + index]; + } + + @Override + public Spliterator.OfLong spliterator() { + return Spliterators.spliterator(array, start, end, 0); + } + + @Override + public boolean contains(Object target) { + // Overridden to prevent a ton of boxing + return (target instanceof Long) && Longs.indexOf(array, (Long) target, start, end) != -1; + } + + @Override + public int indexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Long) { + int i = Longs.indexOf(array, (Long) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public int lastIndexOf(Object target) { + // Overridden to prevent a ton of boxing + if (target instanceof Long) { + int i = Longs.lastIndexOf(array, (Long) target, start, end); + if (i >= 0) { + return i - start; + } + } + return -1; + } + + @Override + public Long set(int index, Long element) { + checkElementIndex(index, size()); + long oldValue = array[start + index]; + // checkNotNull for GWT (do not optimize) + array[start + index] = checkNotNull(element); + return oldValue; + } + + @Override + public List subList(int fromIndex, int toIndex) { + int size = size(); + checkPositionIndexes(fromIndex, toIndex, size); + if (fromIndex == toIndex) { + return Collections.emptyList(); + } + return new LongArrayAsList(array, start + fromIndex, start + toIndex); + } + + @Override + public boolean equals(Object object) { + if (object == this) { + return true; + } + if (object instanceof LongArrayAsList) { + LongArrayAsList that = (LongArrayAsList) object; + int size = size(); + if (that.size() != size) { + return false; + } + for (int i = 0; i < size; i++) { + if (array[start + i] != that.array[that.start + i]) { + return false; + } + } + return true; + } + return super.equals(object); + } + + @Override + public int hashCode() { + int result = 1; + for (int i = start; i < end; i++) { + result = 31 * result + Longs.hashCode(array[i]); + } + return result; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(size() * 10); + builder.append('[').append(array[start]); + for (int i = start + 1; i < end; i++) { + builder.append(", ").append(array[i]); + } + return builder.append(']').toString(); + } + + long[] toLongArray() { + return Arrays.copyOfRange(array, start, end); + } + + private static final long serialVersionUID = 0; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/ParseRequest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/ParseRequest.java new file mode 100644 index 0000000000000..9cce21150a8f6 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/ParseRequest.java @@ -0,0 +1,57 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + + +/** A string to be parsed as a number and the radix to interpret it in. */ +final class ParseRequest { + final String rawValue; + final int radix; + + private ParseRequest(String rawValue, int radix) { + this.rawValue = rawValue; + this.radix = radix; + } + + static ParseRequest fromString(String stringValue) { + if (stringValue.length() == 0) { + throw new NumberFormatException("empty string"); + } + + // Handle radix specifier if present + String rawValue; + int radix; + char firstChar = stringValue.charAt(0); + if (stringValue.startsWith("0x") || stringValue.startsWith("0X")) { + rawValue = stringValue.substring(2); + radix = 16; + } else if (firstChar == '#') { + rawValue = stringValue.substring(1); + radix = 16; + } else if (firstChar == '0' && stringValue.length() > 1) { + rawValue = stringValue.substring(1); + radix = 8; + } else { + rawValue = stringValue; + radix = 10; + } + + return new ParseRequest(rawValue, radix); + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Primitives.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Primitives.java new file mode 100644 index 0000000000000..8c9ad7cf73738 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/Primitives.java @@ -0,0 +1,140 @@ +/* + * Copyright (C) 2007 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; + +/** + * Contains static utility methods pertaining to primitive types and their corresponding wrapper + * types. + * + * @author Kevin Bourrillion + * @since 1.0 + */ +public final class Primitives { + private Primitives() {} + + /** A map from primitive types to their corresponding wrapper types. */ + private static final Map, Class> PRIMITIVE_TO_WRAPPER_TYPE; + + /** A map from wrapper types to their corresponding primitive types. */ + private static final Map, Class> WRAPPER_TO_PRIMITIVE_TYPE; + + // Sad that we can't use a BiMap. :( + + static { + Map, Class> primToWrap = new HashMap<>(16); + Map, Class> wrapToPrim = new HashMap<>(16); + + add(primToWrap, wrapToPrim, boolean.class, Boolean.class); + add(primToWrap, wrapToPrim, byte.class, Byte.class); + add(primToWrap, wrapToPrim, char.class, Character.class); + add(primToWrap, wrapToPrim, double.class, Double.class); + add(primToWrap, wrapToPrim, float.class, Float.class); + add(primToWrap, wrapToPrim, int.class, Integer.class); + add(primToWrap, wrapToPrim, long.class, Long.class); + add(primToWrap, wrapToPrim, short.class, Short.class); + add(primToWrap, wrapToPrim, void.class, Void.class); + + PRIMITIVE_TO_WRAPPER_TYPE = Collections.unmodifiableMap(primToWrap); + WRAPPER_TO_PRIMITIVE_TYPE = Collections.unmodifiableMap(wrapToPrim); + } + + private static void add( + Map, Class> forward, + Map, Class> backward, + Class key, + Class value) { + forward.put(key, value); + backward.put(value, key); + } + + /** + * Returns an immutable set of all nine primitive types (including {@code void}). Note that a + * simpler way to test whether a {@code Class} instance is a member of this set is to call {@link + * Class#isPrimitive}. + * + * @since 3.0 + */ + public static Set> allPrimitiveTypes() { + return PRIMITIVE_TO_WRAPPER_TYPE.keySet(); + } + + /** + * Returns an immutable set of all nine primitive-wrapper types (including {@link Void}). + * + * @since 3.0 + */ + public static Set> allWrapperTypes() { + return WRAPPER_TO_PRIMITIVE_TYPE.keySet(); + } + + /** + * Returns {@code true} if {@code type} is one of the nine primitive-wrapper types, such as {@link + * Integer}. + * + * @see Class#isPrimitive + */ + public static boolean isWrapperType(Class type) { + return WRAPPER_TO_PRIMITIVE_TYPE.containsKey(checkNotNull(type)); + } + + /** + * Returns the corresponding wrapper type of {@code type} if it is a primitive type; otherwise + * returns {@code type} itself. Idempotent. + * + *

+   *     wrap(int.class) == Integer.class
+   *     wrap(Integer.class) == Integer.class
+   *     wrap(String.class) == String.class
+   * 
+ */ + public static Class wrap(Class type) { + checkNotNull(type); + + // cast is safe: long.class and Long.class are both of type Class + @SuppressWarnings("unchecked") + Class wrapped = (Class) PRIMITIVE_TO_WRAPPER_TYPE.get(type); + return (wrapped == null) ? type : wrapped; + } + + /** + * Returns the corresponding primitive type of {@code type} if it is a wrapper type; otherwise + * returns {@code type} itself. Idempotent. + * + *
+   *     unwrap(Integer.class) == int.class
+   *     unwrap(int.class) == int.class
+   *     unwrap(String.class) == String.class
+   * 
+ */ + public static Class unwrap(Class type) { + checkNotNull(type); + + // cast is safe: long.class and Long.class are both of type Class + @SuppressWarnings("unchecked") + Class unwrapped = (Class) WRAPPER_TO_PRIMITIVE_TYPE.get(type); + return (unwrapped == null) ? type : unwrapped; + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/UnsignedLongs.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/UnsignedLongs.java new file mode 100644 index 0000000000000..7b565de80335e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/UnsignedLongs.java @@ -0,0 +1,497 @@ +/* + * Copyright (C) 2011 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; + +import java.math.BigInteger; +import java.util.Arrays; +import java.util.Comparator; + +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkPositionIndexes; + +/** + * Static utility methods pertaining to {@code long} primitives that interpret values as + * unsigned (that is, any negative value {@code x} is treated as the positive value {@code + * 2^64 + x}). The methods for which signedness is not an issue are in {@link Longs}, as well as + * signed versions of methods for which signedness is an issue. + * + *

In addition, this class provides several static methods for converting a {@code long} to a + * {@code String} and a {@code String} to a {@code long} that treat the {@code long} as an unsigned + * number. + * + *

Users of these utilities must be extremely careful not to mix up signed and unsigned + * {@code long} values. When possible, it is recommended that the {@link UnsignedLong} wrapper class + * be used, at a small efficiency penalty, to enforce the distinction in the type system. + * + *

See the Guava User Guide article on unsigned + * primitive utilities. + * + * @author Louis Wasserman + * @author Brian Milch + * @author Colin Evans + * @since 10.0 + */ + +public final class UnsignedLongs { + private UnsignedLongs() {} + + public static final long MAX_VALUE = -1L; // Equivalent to 2^64 - 1 + + /** + * A (self-inverse) bijection which converts the ordering on unsigned longs to the ordering on + * longs, that is, {@code a <= b} as unsigned longs if and only if {@code flip(a) <= flip(b)} as + * signed longs. + */ + private static long flip(long a) { + return a ^ Long.MIN_VALUE; + } + + /** + * Compares the two specified {@code long} values, treating them as unsigned values between {@code + * 0} and {@code 2^64 - 1} inclusive. + * + *

Java 8 users: use {@link Long#compareUnsigned(long, long)} instead. + * + * @param a the first unsigned {@code long} to compare + * @param b the second unsigned {@code long} to compare + * @return a negative value if {@code a} is less than {@code b}; a positive value if {@code a} is + * greater than {@code b}; or zero if they are equal + */ + public static int compare(long a, long b) { + return Longs.compare(flip(a), flip(b)); + } + + /** + * Returns the least value present in {@code array}, treating values as unsigned. + * + * @param array a nonempty array of unsigned {@code long} values + * @return the value present in {@code array} that is less than or equal to every other value in + * the array according to {@link #compare} + * @throws IllegalArgumentException if {@code array} is empty + */ + public static long min(long... array) { + checkArgument(array.length > 0); + long min = flip(array[0]); + for (int i = 1; i < array.length; i++) { + long next = flip(array[i]); + if (next < min) { + min = next; + } + } + return flip(min); + } + + /** + * Returns the greatest value present in {@code array}, treating values as unsigned. + * + * @param array a nonempty array of unsigned {@code long} values + * @return the value present in {@code array} that is greater than or equal to every other value + * in the array according to {@link #compare} + * @throws IllegalArgumentException if {@code array} is empty + */ + public static long max(long... array) { + checkArgument(array.length > 0); + long max = flip(array[0]); + for (int i = 1; i < array.length; i++) { + long next = flip(array[i]); + if (next > max) { + max = next; + } + } + return flip(max); + } + + /** + * Returns a string containing the supplied unsigned {@code long} values separated by {@code + * separator}. For example, {@code join("-", 1, 2, 3)} returns the string {@code "1-2-3"}. + * + * @param separator the text that should appear between consecutive values in the resulting string + * (but not at the start or end) + * @param array an array of unsigned {@code long} values, possibly empty + */ + public static String join(String separator, long... array) { + checkNotNull(separator); + if (array.length == 0) { + return ""; + } + + // For pre-sizing a builder, just get the right order of magnitude + StringBuilder builder = new StringBuilder(array.length * 5); + builder.append(toString(array[0])); + for (int i = 1; i < array.length; i++) { + builder.append(separator).append(toString(array[i])); + } + return builder.toString(); + } + + /** + * Returns a comparator that compares two arrays of unsigned {@code long} values lexicographically. That is, it + * compares, using {@link #compare(long, long)}), the first pair of values that follow any common + * prefix, or when one array is a prefix of the other, treats the shorter array as the lesser. For + * example, {@code [] < [1L] < [1L, 2L] < [2L] < [1L << 63]}. + * + *

The returned comparator is inconsistent with {@link Object#equals(Object)} (since arrays + * support only identity equality), but it is consistent with {@link Arrays#equals(long[], + * long[])}. + */ + public static Comparator lexicographicalComparator() { + return LexicographicalComparator.INSTANCE; + } + + enum LexicographicalComparator implements Comparator { + INSTANCE; + + @Override + public int compare(long[] left, long[] right) { + int minLength = Math.min(left.length, right.length); + for (int i = 0; i < minLength; i++) { + if (left[i] != right[i]) { + return UnsignedLongs.compare(left[i], right[i]); + } + } + return left.length - right.length; + } + + @Override + public String toString() { + return "UnsignedLongs.lexicographicalComparator()"; + } + } + + /** + * Sorts the array, treating its elements as unsigned 64-bit integers. + * + * @since 23.1 + */ + public static void sort(long[] array) { + checkNotNull(array); + sort(array, 0, array.length); + } + + /** + * Sorts the array between {@code fromIndex} inclusive and {@code toIndex} exclusive, treating its + * elements as unsigned 64-bit integers. + * + * @since 23.1 + */ + public static void sort(long[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + for (int i = fromIndex; i < toIndex; i++) { + array[i] = flip(array[i]); + } + Arrays.sort(array, fromIndex, toIndex); + for (int i = fromIndex; i < toIndex; i++) { + array[i] = flip(array[i]); + } + } + + /** + * Sorts the elements of {@code array} in descending order, interpreting them as unsigned 64-bit + * integers. + * + * @since 23.1 + */ + public static void sortDescending(long[] array) { + checkNotNull(array); + sortDescending(array, 0, array.length); + } + + /** + * Sorts the elements of {@code array} between {@code fromIndex} inclusive and {@code toIndex} + * exclusive in descending order, interpreting them as unsigned 64-bit integers. + * + * @since 23.1 + */ + public static void sortDescending(long[] array, int fromIndex, int toIndex) { + checkNotNull(array); + checkPositionIndexes(fromIndex, toIndex, array.length); + for (int i = fromIndex; i < toIndex; i++) { + array[i] ^= Long.MAX_VALUE; + } + Arrays.sort(array, fromIndex, toIndex); + for (int i = fromIndex; i < toIndex; i++) { + array[i] ^= Long.MAX_VALUE; + } + } + + /** + * Returns dividend / divisor, where the dividend and divisor are treated as unsigned 64-bit + * quantities. + * + *

Java 8 users: use {@link Long#divideUnsigned(long, long)} instead. + * + * @param dividend the dividend (numerator) + * @param divisor the divisor (denominator) + * @throws ArithmeticException if divisor is 0 + */ + public static long divide(long dividend, long divisor) { + if (divisor < 0) { // i.e., divisor >= 2^63: + if (compare(dividend, divisor) < 0) { + return 0; // dividend < divisor + } else { + return 1; // dividend >= divisor + } + } + + // Optimization - use signed division if dividend < 2^63 + if (dividend >= 0) { + return dividend / divisor; + } + + /* + * Otherwise, approximate the quotient, check, and correct if necessary. Our approximation is + * guaranteed to be either exact or one less than the correct value. This follows from fact that + * floor(floor(x)/i) == floor(x/i) for any real x and integer i != 0. The proof is not quite + * trivial. + */ + long quotient = ((dividend >>> 1) / divisor) << 1; + long rem = dividend - quotient * divisor; + return quotient + (compare(rem, divisor) >= 0 ? 1 : 0); + } + + /** + * Returns dividend % divisor, where the dividend and divisor are treated as unsigned 64-bit + * quantities. + * + *

Java 8 users: use {@link Long#remainderUnsigned(long, long)} instead. + * + * @param dividend the dividend (numerator) + * @param divisor the divisor (denominator) + * @throws ArithmeticException if divisor is 0 + * @since 11.0 + */ + public static long remainder(long dividend, long divisor) { + if (divisor < 0) { // i.e., divisor >= 2^63: + if (compare(dividend, divisor) < 0) { + return dividend; // dividend < divisor + } else { + return dividend - divisor; // dividend >= divisor + } + } + + // Optimization - use signed modulus if dividend < 2^63 + if (dividend >= 0) { + return dividend % divisor; + } + + /* + * Otherwise, approximate the quotient, check, and correct if necessary. Our approximation is + * guaranteed to be either exact or one less than the correct value. This follows from the fact + * that floor(floor(x)/i) == floor(x/i) for any real x and integer i != 0. The proof is not + * quite trivial. + */ + long quotient = ((dividend >>> 1) / divisor) << 1; + long rem = dividend - quotient * divisor; + return rem - (compare(rem, divisor) >= 0 ? divisor : 0); + } + + /** + * Returns the unsigned {@code long} value represented by the given decimal string. + * + *

Java 8 users: use {@link Long#parseUnsignedLong(String)} instead. + * + * @throws NumberFormatException if the string does not contain a valid unsigned {@code long} + * value + * @throws NullPointerException if {@code string} is null (in contrast to {@link + * Long#parseLong(String)}) + */ + public static long parseUnsignedLong(String string) { + return parseUnsignedLong(string, 10); + } + + /** + * Returns the unsigned {@code long} value represented by a string with the given radix. + * + *

Java 8 users: use {@link Long#parseUnsignedLong(String, int)} instead. + * + * @param string the string containing the unsigned {@code long} representation to be parsed. + * @param radix the radix to use while parsing {@code string} + * @throws NumberFormatException if the string does not contain a valid unsigned {@code long} with + * the given radix, or if {@code radix} is not between {@link Character#MIN_RADIX} and {@link + * Character#MAX_RADIX}. + * @throws NullPointerException if {@code string} is null (in contrast to {@link + * Long#parseLong(String)}) + */ + public static long parseUnsignedLong(String string, int radix) { + checkNotNull(string); + if (string.length() == 0) { + throw new NumberFormatException("empty string"); + } + if (radix < Character.MIN_RADIX || radix > Character.MAX_RADIX) { + throw new NumberFormatException("illegal radix: " + radix); + } + + int maxSafePos = ParseOverflowDetection.maxSafeDigits[radix] - 1; + long value = 0; + for (int pos = 0; pos < string.length(); pos++) { + int digit = Character.digit(string.charAt(pos), radix); + if (digit == -1) { + throw new NumberFormatException(string); + } + if (pos > maxSafePos && ParseOverflowDetection.overflowInParse(value, digit, radix)) { + throw new NumberFormatException("Too large for unsigned long: " + string); + } + value = (value * radix) + digit; + } + + return value; + } + + /** + * Returns the unsigned {@code long} value represented by the given string. + * + *

Accepts a decimal, hexadecimal, or octal number given by specifying the following prefix: + * + *

    + *
  • {@code 0x}HexDigits + *
  • {@code 0X}HexDigits + *
  • {@code #}HexDigits + *
  • {@code 0}OctalDigits + *
+ * + * @throws NumberFormatException if the string does not contain a valid unsigned {@code long} + * value + * @since 13.0 + */ + public static long decode(String stringValue) { + ParseRequest request = ParseRequest.fromString(stringValue); + + try { + return parseUnsignedLong(request.rawValue, request.radix); + } catch (NumberFormatException e) { + NumberFormatException decodeException = + new NumberFormatException("Error parsing value: " + stringValue); + decodeException.initCause(e); + throw decodeException; + } + } + + /* + * We move the static constants into this class so ProGuard can inline UnsignedLongs entirely + * unless the user is actually calling a parse method. + */ + private static final class ParseOverflowDetection { + private ParseOverflowDetection() {} + + // calculated as 0xffffffffffffffff / radix + static final long[] maxValueDivs = new long[Character.MAX_RADIX + 1]; + static final int[] maxValueMods = new int[Character.MAX_RADIX + 1]; + static final int[] maxSafeDigits = new int[Character.MAX_RADIX + 1]; + + static { + BigInteger overflow = new BigInteger("10000000000000000", 16); + for (int i = Character.MIN_RADIX; i <= Character.MAX_RADIX; i++) { + maxValueDivs[i] = divide(MAX_VALUE, i); + maxValueMods[i] = (int) remainder(MAX_VALUE, i); + maxSafeDigits[i] = overflow.toString(i).length() - 1; + } + } + + /** + * Returns true if (current * radix) + digit is a number too large to be represented by an + * unsigned long. This is useful for detecting overflow while parsing a string representation of + * a number. Does not verify whether supplied radix is valid, passing an invalid radix will give + * undefined results or an ArrayIndexOutOfBoundsException. + */ + static boolean overflowInParse(long current, int digit, int radix) { + if (current >= 0) { + if (current < maxValueDivs[radix]) { + return false; + } + if (current > maxValueDivs[radix]) { + return true; + } + // current == maxValueDivs[radix] + return (digit > maxValueMods[radix]); + } + + // current < 0: high bit is set + return true; + } + } + + /** + * Returns a string representation of x, where x is treated as unsigned. + * + *

Java 8 users: use {@link Long#toUnsignedString(long)} instead. + */ + public static String toString(long x) { + return toString(x, 10); + } + + /** + * Returns a string representation of {@code x} for the given radix, where {@code x} is treated as + * unsigned. + * + *

Java 8 users: use {@link Long#toUnsignedString(long, int)} instead. + * + * @param x the value to convert to a string. + * @param radix the radix to use while working with {@code x} + * @throws IllegalArgumentException if {@code radix} is not between {@link Character#MIN_RADIX} + * and {@link Character#MAX_RADIX}. + */ + public static String toString(long x, int radix) { + checkArgument( + radix >= Character.MIN_RADIX && radix <= Character.MAX_RADIX, + "radix (%s) must be between Character.MIN_RADIX and Character.MAX_RADIX", + radix); + if (x == 0) { + // Simply return "0" + return "0"; + } else if (x > 0) { + return Long.toString(x, radix); + } else { + char[] buf = new char[64]; + int i = buf.length; + if ((radix & (radix - 1)) == 0) { + // Radix is a power of two so we can avoid division. + int shift = Integer.numberOfTrailingZeros(radix); + int mask = radix - 1; + do { + buf[--i] = Character.forDigit(((int) x) & mask, radix); + x >>>= shift; + } while (x != 0); + } else { + // Separate off the last digit using unsigned division. That will leave + // a number that is nonnegative as a signed integer. + long quotient; + if ((radix & 1) == 0) { + // Fast path for the usual case where the radix is even. + quotient = (x >>> 1) / (radix >>> 1); + } else { + quotient = divide(x, radix); + } + long rem = x - quotient * radix; + buf[--i] = Character.forDigit((int) rem, radix); + x = quotient; + // Simple modulo/division approach + while (x > 0) { + buf[--i] = Character.forDigit((int) (x % radix), radix); + x /= radix; + } + } + // Generate string + return new String(buf, i, buf.length - i); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/package-info.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/package-info.java new file mode 100644 index 0000000000000..205ec399e9497 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/guava25/primitives/package-info.java @@ -0,0 +1,24 @@ +/* + * Copyright (C) 2010 The Guava Authors + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except + * in compliance with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ + +/** + * Static utilities for working with the eight primitive types and {@code void}, and value types for + * treating them as unsigned. + * + */ +/* + * Portions Copyright (c) Microsoft Corporation + */ + +package com.azure.cosmos.kafka.connect.implementation.guava25.primitives; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java index d029105846a56..6a65a0f6a9df0 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkConfig.java @@ -3,8 +3,8 @@ package com.azure.cosmos.kafka.connect.implementation.sink; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConfig; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java index 79d881169b3da..8e463335f1cee 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/CosmosSinkTask.java @@ -5,9 +5,9 @@ import com.azure.cosmos.CosmosAsyncClient; import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; import org.slf4j.Logger; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java index 97515fa530feb..586fc1016b9da 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosBulkWriter.java @@ -8,9 +8,9 @@ import com.azure.cosmos.CosmosException; import com.azure.cosmos.implementation.HttpConstants; import com.azure.cosmos.implementation.ImplementationBridgeHelpers; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosSchedulers; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.models.CosmosBulkExecutionOptions; import com.azure.cosmos.models.CosmosBulkItemRequestOptions; import com.azure.cosmos.models.CosmosBulkItemResponse; @@ -31,7 +31,7 @@ import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class KafkaCosmosBulkWriter extends KafkaCosmosWriterBase { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosBulkWriter.class); @@ -136,7 +136,10 @@ private Mono getBulkOperation( CosmosAsyncContainer container, SinkOperation sinkOperation) { - return this.getPartitionKeyDefinition(container) + return ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .getPartitionKeyDefinition(container) .flatMap(partitionKeyDefinition -> { CosmosItemOperation cosmosItemOperation; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java index a13c5d1a5266e..156ac531958ac 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosPointWriter.java @@ -4,10 +4,11 @@ package com.azure.cosmos.kafka.connect.implementation.sink; import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.guava25.base.Function; +import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosSchedulers; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; import com.azure.cosmos.models.CosmosItemRequestOptions; import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.slf4j.Logger; @@ -16,7 +17,7 @@ import java.util.List; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class KafkaCosmosPointWriter extends KafkaCosmosWriterBase { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosPointWriter.class); @@ -83,14 +84,17 @@ private void replaceIfNotModifiedWithRetry(CosmosAsyncContainer container, SinkO CosmosItemRequestOptions itemRequestOptions = new CosmosItemRequestOptions(); itemRequestOptions.setIfMatchETag(etag); - return this.getPartitionKeyDefinition(container) - .flatMap(partitionKeyDefinition -> { - return container.replaceItem( - operation.getSinkRecord().value(), - getId(operation.getSinkRecord().value()), - getPartitionKeyValue(operation.getSinkRecord().value(), partitionKeyDefinition), - itemRequestOptions).then(); - }); + return ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .getPartitionKeyDefinition(container) + .flatMap(partitionKeyDefinition -> { + return container.replaceItem( + operation.getSinkRecord().value(), + getId(operation.getSinkRecord().value()), + getPartitionKeyValue(operation.getSinkRecord().value(), partitionKeyDefinition), + itemRequestOptions).then(); + }); }, (throwable) -> { return KafkaCosmosExceptionsHelper.isNotFoundException(throwable) @@ -111,7 +115,10 @@ private void deleteWithRetry(CosmosAsyncContainer container, SinkOperation sinkO } } - return this.getPartitionKeyDefinition(container) + return ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .getPartitionKeyDefinition(container) .flatMap(partitionKeyDefinition -> { return container.deleteItem( getId(operation.getSinkRecord().value()), diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java index 108f3fefa7cac..19cd602e585d5 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/KafkaCosmosWriterBase.java @@ -4,27 +4,23 @@ package com.azure.cosmos.kafka.connect.implementation.sink; import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.CosmosBridgeInternal; -import com.azure.cosmos.implementation.DocumentCollection; import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.implementation.Strings; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.routing.PartitionKeyInternal; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.models.PartitionKey; import com.azure.cosmos.models.PartitionKeyDefinition; import org.apache.kafka.connect.sink.ErrantRecordReporter; import org.apache.kafka.connect.sink.SinkRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import reactor.core.publisher.Mono; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public abstract class KafkaCosmosWriterBase implements IWriter { private static final Logger LOGGER = LoggerFactory.getLogger(KafkaCosmosWriterBase.class); @@ -81,12 +77,11 @@ protected PartitionKey getPartitionKeyValue(Object recordValue, PartitionKeyDefi String partitionKeyPath = StringUtils.join(partitionKeyDefinition.getPaths(), ""); Map recordMap = (Map) recordValue; Object partitionKeyValue = recordMap.get(partitionKeyPath.substring(1)); - PartitionKeyInternal partitionKeyInternal = PartitionKeyInternal.fromObjectArray(Collections.singletonList(partitionKeyValue), false); return ImplementationBridgeHelpers .PartitionKeyHelper .getPartitionKeyAccessor() - .toPartitionKey(partitionKeyInternal); + .toPartitionKey(Collections.singletonList(partitionKeyValue), false); } protected boolean shouldRetry(Throwable exception, int attemptedCount, int maxRetryCount) { @@ -97,22 +92,6 @@ protected boolean shouldRetry(Throwable exception, int attemptedCount, int maxRe return KafkaCosmosExceptionsHelper.isTransientFailure(exception); } - protected Mono getPartitionKeyDefinition(CosmosAsyncContainer container) { - return Mono.just(CosmosBridgeInternal.getAsyncDocumentClient(container.getDatabase()).getCollectionCache()) - .flatMap(collectionCache -> { - return collectionCache - .resolveByNameAsync( - null, - ImplementationBridgeHelpers - .CosmosAsyncContainerHelper - .getCosmosAsyncContainerAccessor() - .getLinkWithoutTrailingSlash(container), - null, - new DocumentCollection()) - .map(documentCollection -> documentCollection.getPartitionKey()); - }); - } - protected void sendToDlqIfConfigured(SinkOperation sinkOperationContext) { if (this.errantRecordReporter != null) { errantRecordReporter.report(sinkOperationContext.getSinkRecord(), sinkOperationContext.getException()); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java index 1f40827050dcc..6f1a7e8644beb 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/sink/idstrategy/TemplateStrategy.java @@ -3,7 +3,7 @@ package com.azure.cosmos.kafka.connect.implementation.sink.idstrategy; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.sink.SinkRecord; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicOffset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicOffset.java index 10d7885bbde27..3a3b293ae1fc9 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicOffset.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicOffset.java @@ -3,7 +3,7 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.Utils; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -12,14 +12,14 @@ import java.util.List; import java.util.Map; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /** * Containers metadata topic offset. */ public class ContainersMetadataTopicOffset { public static final String CONTAINERS_RESOURCE_IDS_NAME_KEY = "cosmos.source.metadata.containerRids"; - public static final ObjectMapper OBJECT_MAPPER = Utils.getSimpleObjectMapper(); + public static final ObjectMapper OBJECT_MAPPER = KafkaCosmosUtils.getSimpleObjectMapper(); private final List containerRids; public ContainersMetadataTopicOffset(List containerRids) { diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicPartition.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicPartition.java index b2ae0e6de93eb..a4de7f8ae0f58 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicPartition.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/ContainersMetadataTopicPartition.java @@ -3,12 +3,12 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import java.util.HashMap; import java.util.Map; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public class ContainersMetadataTopicPartition { public static final String DATABASE_NAME_KEY = "cosmos.source.metadata.database.name"; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosMetadataConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosMetadataConfig.java index 91f54088cc31b..5925f384b079c 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosMetadataConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosMetadataConfig.java @@ -3,9 +3,9 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public class CosmosMetadataConfig { private final int metadataPollDelayInMs; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java index 1365d1e53575e..24630326da9e2 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceConfig.java @@ -3,9 +3,9 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.Strings; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConfig; +import com.azure.cosmos.kafka.connect.implementation.Strings; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigException; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java index 7f9a2d4284e85..07b5d04621ae6 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceContainersConfig.java @@ -3,13 +3,13 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import java.util.List; import java.util.Map; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class CosmosSourceContainersConfig { public static final String CONTAINER_TOPIC_MAP_SEPARATOR = "#"; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceOffsetStorageReader.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceOffsetStorageReader.java index b7c4118379e57..073c9edc30d7d 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceOffsetStorageReader.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceOffsetStorageReader.java @@ -3,7 +3,7 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.models.FeedRange; import org.apache.kafka.connect.storage.OffsetStorageReader; import java.util.Map; @@ -38,7 +38,7 @@ public ContainersMetadataTopicOffset getContainersMetadataOffset(String database public FeedRangeContinuationTopicOffset getFeedRangeContinuationOffset( String databaseName, String collectionRid, - Range feedRange) { + FeedRange feedRange) { Map topicOffsetMap = this.offsetStorageReader diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java index ec358965e765f..6472f5b61dd8c 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTask.java @@ -3,23 +3,21 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.BridgeInternal; import com.azure.cosmos.CosmosAsyncClient; import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.CosmosBridgeInternal; -import com.azure.cosmos.implementation.AsyncDocumentClient; -import com.azure.cosmos.implementation.PartitionKeyRange; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.apachecommons.lang.tuple.Pair; -import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; -import com.azure.cosmos.implementation.guava25.base.Stopwatch; +import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosConstants; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.tuple.Pair; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Stopwatch; import com.azure.cosmos.models.CosmosChangeFeedRequestOptions; import com.azure.cosmos.models.FeedRange; import com.azure.cosmos.models.FeedResponse; import com.azure.cosmos.models.ModelBridgeInternal; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -104,7 +102,8 @@ public List poll() { ((FeedRangeTaskUnit) taskUnit).getContainerName(), ((FeedRangeTaskUnit) taskUnit).getContainerRid(), ((FeedRangeTaskUnit) taskUnit).getFeedRange(), - stopwatch.elapsed().toMillis()); + stopwatch.elapsed().toMillis() + ); } return results; } catch (Exception e) { @@ -224,34 +223,23 @@ private List handleSuccessfulResponse( private Mono handleFeedRangeGone(FeedRangeTaskUnit feedRangeTaskUnit) { // need to find out whether it is split or merge - AsyncDocumentClient asyncDocumentClient = CosmosBridgeInternal.getAsyncDocumentClient(this.cosmosClient); CosmosAsyncContainer container = this.cosmosClient .getDatabase(feedRangeTaskUnit.getDatabaseName()) .getContainer(feedRangeTaskUnit.getContainerName()); - return asyncDocumentClient - .getCollectionCache() - .resolveByNameAsync(null, BridgeInternal.extractContainerSelfLink(container), null) - .flatMap(collection -> { - return asyncDocumentClient.getPartitionKeyRangeCache().tryGetOverlappingRangesAsync( - null, - collection.getResourceId(), - feedRangeTaskUnit.getFeedRange(), - true, - null); - }) - .flatMap(pkRangesValueHolder -> { - if (pkRangesValueHolder == null || pkRangesValueHolder.v == null) { - return Mono.error(new IllegalStateException("There are no overlapping ranges for the range")); - } - List partitionKeyRanges = pkRangesValueHolder.v; - if (partitionKeyRanges.size() == 1) { + return ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .getOverlappingFeedRanges(container, feedRangeTaskUnit.getFeedRange()) + .flatMap(overlappedRanges -> { + + if (overlappedRanges.size() == 1) { // merge happens LOGGER.info( "FeedRange {} is merged into {}, but we will continue polling data from feedRange {}", feedRangeTaskUnit.getFeedRange(), - partitionKeyRanges.get(0).toRange(), + overlappedRanges.get(0).toString(), feedRangeTaskUnit.getFeedRange()); // Continue using polling data from the current task unit feedRange @@ -260,16 +248,16 @@ private Mono handleFeedRangeGone(FeedRangeTaskUnit feedRangeTaskUnit) { LOGGER.info( "FeedRange {} is split into {}. Will create new task units. ", feedRangeTaskUnit.getFeedRange(), - partitionKeyRanges.stream().map(PartitionKeyRange::toRange).collect(Collectors.toList()) + overlappedRanges.stream().map(FeedRange::toString).collect(Collectors.toList()) ); - for (PartitionKeyRange pkRange : partitionKeyRanges) { + for (FeedRange pkRange : overlappedRanges) { FeedRangeTaskUnit childTaskUnit = new FeedRangeTaskUnit( feedRangeTaskUnit.getDatabaseName(), feedRangeTaskUnit.getContainerName(), feedRangeTaskUnit.getContainerRid(), - pkRange.toRange(), + pkRange, feedRangeTaskUnit.getContinuationState(), feedRangeTaskUnit.getTopic()); this.taskUnitsQueue.add(childTaskUnit); @@ -299,7 +287,7 @@ private String getMessageKey(JsonNode item) { private CosmosChangeFeedRequestOptions getChangeFeedRequestOptions(FeedRangeTaskUnit feedRangeTaskUnit) { CosmosChangeFeedRequestOptions changeFeedRequestOptions = null; - FeedRange changeFeedRange = new FeedRangeEpkImpl(feedRangeTaskUnit.getFeedRange()); + FeedRange changeFeedRange = feedRangeTaskUnit.getFeedRange(); if (StringUtils.isEmpty(feedRangeTaskUnit.getContinuationState())) { switch (this.taskConfig.getChangeFeedConfig().getChangeFeedStartFromModes()) { case BEGINNING: @@ -325,8 +313,21 @@ private CosmosChangeFeedRequestOptions getChangeFeedRequestOptions(FeedRangeTask changeFeedRequestOptions.allVersionsAndDeletes(); } } else { - changeFeedRequestOptions = - CosmosChangeFeedRequestOptions.createForProcessingFromContinuation(feedRangeTaskUnit.getContinuationState()); + try { + KafkaCosmosChangeFeedState kafkaCosmosChangeFeedState = + KafkaCosmosUtils.getSimpleObjectMapper().readValue(feedRangeTaskUnit.getContinuationState(), KafkaCosmosChangeFeedState.class); + + changeFeedRequestOptions = + ImplementationBridgeHelpers.CosmosChangeFeedRequestOptionsHelper + .getCosmosChangeFeedRequestOptionsAccessor() + .createForProcessingFromContinuation( + kafkaCosmosChangeFeedState.getResponseContinuation(), + kafkaCosmosChangeFeedState.getTargetRange(), + kafkaCosmosChangeFeedState.getItemLsn()); + + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } } return changeFeedRequestOptions; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskConfig.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskConfig.java index 147bb61814e91..7a9311c9d73d0 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskConfig.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskConfig.java @@ -3,8 +3,8 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.Utils; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -17,7 +17,7 @@ import java.util.stream.Collectors; public class CosmosSourceTaskConfig extends CosmosSourceConfig { - private static final ObjectMapper OBJECT_MAPPER = Utils.getSimpleObjectMapper(); + private static final ObjectMapper OBJECT_MAPPER = KafkaCosmosUtils.getSimpleObjectMapper(); private static final String SOURCE_TASK_CONFIG_PREFIX = "kafka.connect.cosmos.source.task."; public static final String SOURCE_METADATA_TASK_UNIT = SOURCE_TASK_CONFIG_PREFIX + "metadataTaskUnit"; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicOffset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicOffset.java index 72d294262aed1..17ed0a2583f04 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicOffset.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicOffset.java @@ -3,32 +3,32 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import java.util.HashMap; import java.util.Map; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public class FeedRangeContinuationTopicOffset { private static final String ITEM_LSN_KEY = "cosmos.source.feedRange.item.lsn"; - private static final String CONTINUATION_KEY = "cosmos.source.feedRange.continuation"; + private static final String CONTINUATION_KEY = "cosmos.source.feedRange.responseContinuation"; - private final String continuationState; + private final String responseContinuation; private final String itemLsn; public FeedRangeContinuationTopicOffset( - String continuationState, + String responseContinuation, String itemLsn) { - checkArgument(StringUtils.isNotEmpty(continuationState), "Argument 'continuationState' should not be null"); + checkArgument(StringUtils.isNotEmpty(responseContinuation), "Argument 'responseContinuation' should not be null"); checkArgument(StringUtils.isNotEmpty(itemLsn), "Argument 'itemLsn' should not be null"); this.itemLsn = itemLsn; - this.continuationState = continuationState; + this.responseContinuation = responseContinuation; } - public String getContinuationState() { - return continuationState; + public String getResponseContinuation() { + return responseContinuation; } public String getItemLsn() { @@ -37,7 +37,7 @@ public String getItemLsn() { public static Map toMap(FeedRangeContinuationTopicOffset offset) { Map map = new HashMap<>(); - map.put(CONTINUATION_KEY, offset.getContinuationState()); + map.put(CONTINUATION_KEY, offset.getResponseContinuation()); map.put(ITEM_LSN_KEY, offset.getItemLsn()); return map; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicPartition.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicPartition.java index c981c6b9fe6df..ebaf9418e3bc2 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicPartition.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeContinuationTopicPartition.java @@ -3,14 +3,14 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.models.FeedRange; import java.util.HashMap; import java.util.Map; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class FeedRangeContinuationTopicPartition { private static final String DATABASE_NAME_KEY = "cosmos.source.database.name"; @@ -19,12 +19,12 @@ public class FeedRangeContinuationTopicPartition { private final String databaseName; private final String containerRid; - private final Range feedRange; + private final FeedRange feedRange; public FeedRangeContinuationTopicPartition( String databaseName, String containerRid, - Range feedRange) { + FeedRange feedRange) { checkArgument(StringUtils.isNotEmpty(databaseName), "Argument 'databaseName' should not be null"); checkArgument(StringUtils.isNotEmpty(containerRid), "Argument 'containerRid' should not be null"); checkNotNull(feedRange, "Argument 'feedRange' can not be null"); @@ -42,7 +42,7 @@ public String getContainerRid() { return containerRid; } - public Range getFeedRange() { + public FeedRange getFeedRange() { return feedRange; } @@ -50,7 +50,7 @@ public static Map toMap(FeedRangeContinuationTopicPartition part Map map = new HashMap<>(); map.put(DATABASE_NAME_KEY, partition.getDatabaseName()); map.put(CONTAINER_RESOURCE_ID_KEY, partition.getContainerRid()); - map.put(CONTAINER_FEED_RANGE_KEY, partition.getFeedRange().toJson()); + map.put(CONTAINER_FEED_RANGE_KEY, partition.getFeedRange().toString()); return map; } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeTaskUnit.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeTaskUnit.java index 8a184afbc2fbc..e1c003213ef96 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeTaskUnit.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangeTaskUnit.java @@ -3,8 +3,8 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.models.FeedRange; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.databind.DeserializationContext; @@ -17,8 +17,8 @@ import java.io.IOException; import java.util.Objects; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; @JsonSerialize(using = FeedRangeTaskUnit.FeedRangeTaskUnitSerializer.class) @JsonDeserialize(using = FeedRangeTaskUnit.FeedRangeTaskUnitDeserializer.class) @@ -26,7 +26,7 @@ public class FeedRangeTaskUnit implements ITaskUnit { private String databaseName; private String containerName; private String containerRid; - private Range feedRange; + private FeedRange feedRange; private String continuationState; private String topic; @@ -36,7 +36,7 @@ public FeedRangeTaskUnit( String databaseName, String containerName, String containerRid, - Range feedRange, + FeedRange feedRange, String continuationState, String topic) { @@ -66,7 +66,7 @@ public String getContainerRid() { return containerRid; } - public Range getFeedRange() { + public FeedRange getFeedRange() { return feedRange; } @@ -154,7 +154,7 @@ public FeedRangeTaskUnit deserialize( String databaseName = rootNode.get("databaseName").asText(); String containerName = rootNode.get("containerName").asText(); String containerRid = rootNode.get("containerRid").asText(); - Range feedRange = new Range(rootNode.get("feedRange").asText()); + FeedRange feedRange = FeedRange.fromString(rootNode.get("feedRange").asText()); String continuationState = null; if (rootNode.has("continuationState")) { continuationState = rootNode.get("continuationState").asText(); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicOffset.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicOffset.java index 5350a47ec7443..fc699e80b20bf 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicOffset.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicOffset.java @@ -3,8 +3,8 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.Utils; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; +import com.azure.cosmos.models.FeedRange; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; @@ -14,21 +14,21 @@ import java.util.Map; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class FeedRangesMetadataTopicOffset { public static final String CONTAINER_FEED_RANGES_KEY = "cosmos.source.metadata.container.feedRanges"; - public static final ObjectMapper OBJECT_MAPPER = Utils.getSimpleObjectMapper(); + public static final ObjectMapper OBJECT_MAPPER = KafkaCosmosUtils.getSimpleObjectMapper(); - private final List> feedRanges; + private final List feedRanges; - public FeedRangesMetadataTopicOffset(List> feedRanges) { + public FeedRangesMetadataTopicOffset(List feedRanges) { checkNotNull(feedRanges, "Argument 'feedRanges' can not be null"); this.feedRanges = feedRanges; } - public List> getFeedRanges() { + public List getFeedRanges() { return feedRanges; } @@ -41,7 +41,7 @@ public static Map toMap(FeedRangesMetadataTopicOffset offset) { CONTAINER_FEED_RANGES_KEY, OBJECT_MAPPER .writeValueAsString( - offset.getFeedRanges().stream().map(range -> range.toJson()).collect(Collectors.toList()))); + offset.getFeedRanges().stream().map(range -> range.toString()).collect(Collectors.toList()))); return map; } catch (JsonProcessingException e) { @@ -56,11 +56,11 @@ public static FeedRangesMetadataTopicOffset fromMap(Map offsetMa String feedRangesValue = offsetMap.get(CONTAINER_FEED_RANGES_KEY).toString(); try { - List> feedRanges = + List feedRanges = OBJECT_MAPPER .readValue(feedRangesValue, new TypeReference>() {}) .stream() - .map(rangeJson -> new Range(rangeJson)) + .map(rangeJson -> FeedRange.fromString(rangeJson)) .collect(Collectors.toList()); return new FeedRangesMetadataTopicOffset(feedRanges); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicPartition.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicPartition.java index 03c0c50e8c7e9..dfa53446c3ca5 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicPartition.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/FeedRangesMetadataTopicPartition.java @@ -3,12 +3,12 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import java.util.HashMap; import java.util.Map; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public class FeedRangesMetadataTopicPartition { public static final String DATABASE_NAME_KEY = "cosmos.source.metadata.database.name"; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/KafkaCosmosChangeFeedState.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/KafkaCosmosChangeFeedState.java new file mode 100644 index 0000000000000..d76378b56aeba --- /dev/null +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/KafkaCosmosChangeFeedState.java @@ -0,0 +1,73 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.kafka.connect.implementation.source; + +import com.azure.cosmos.models.FeedRange; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import java.io.IOException; + +@JsonSerialize(using = KafkaCosmosChangeFeedState.KafkaCosmosChangeFeedStateSerializer.class) +@JsonDeserialize(using = KafkaCosmosChangeFeedState.KafkaCosmosChangeFeedStateDeserializer.class) +public class KafkaCosmosChangeFeedState { + private final String responseContinuation; + private final FeedRange targetRange; + private final String itemLsn; + + public KafkaCosmosChangeFeedState(String responseContinuation, FeedRange targetRange, String itemLsn) { + this.responseContinuation = responseContinuation; + this.targetRange = targetRange; + this.itemLsn = itemLsn; + } + + public String getResponseContinuation() { + return responseContinuation; + } + + public FeedRange getTargetRange() { + return targetRange; + } + + public String getItemLsn() { + return itemLsn; + } + + public static class KafkaCosmosChangeFeedStateSerializer extends com.fasterxml.jackson.databind.JsonSerializer { + @Override + public void serialize(KafkaCosmosChangeFeedState kafkaCosmosChangeFeedState, + JsonGenerator writer, + SerializerProvider serializerProvider) throws IOException { + writer.writeStartObject(); + writer.writeStringField("responseContinuation", kafkaCosmosChangeFeedState.getResponseContinuation()); + writer.writeStringField("targetRange", kafkaCosmosChangeFeedState.getTargetRange().toString()); + writer.writeStringField("itemLsn", kafkaCosmosChangeFeedState.getItemLsn()); + writer.writeEndObject(); + } + } + + static class KafkaCosmosChangeFeedStateDeserializer extends StdDeserializer { + KafkaCosmosChangeFeedStateDeserializer() { + super(KafkaCosmosChangeFeedState.class); + } + + @Override + public KafkaCosmosChangeFeedState deserialize( + JsonParser jsonParser, + DeserializationContext deserializationContext) throws IOException { + + final JsonNode rootNode = jsonParser.getCodec().readTree(jsonParser); + String continuationState = rootNode.get("responseContinuation").asText(); + FeedRange targetRange = FeedRange.fromString(rootNode.get("targetRange").asText()); + String continuationLsn = rootNode.get("itemLsn").asText(); + return new KafkaCosmosChangeFeedState(continuationState, targetRange, continuationLsn); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java index 699b3ad9375d5..c40cdfe0df2a1 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThread.java @@ -3,16 +3,12 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.BridgeInternal; import com.azure.cosmos.CosmosAsyncClient; import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.implementation.AsyncDocumentClient; -import com.azure.cosmos.implementation.PartitionKeyRange; -import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; -import com.azure.cosmos.implementation.feedranges.FeedRangeInternal; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosExceptionsHelper; import com.azure.cosmos.models.CosmosContainerProperties; +import com.azure.cosmos.models.FeedRange; import com.azure.cosmos.models.SqlParameter; import com.azure.cosmos.models.SqlQuerySpec; import org.apache.kafka.connect.source.SourceConnectorContext; @@ -29,7 +25,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class MetadataMonitorThread extends Thread { private static final Logger LOGGER = LoggerFactory.getLogger(MetadataMonitorThread.class); @@ -187,13 +183,7 @@ private Mono shouldRequestTaskReconfigurationOnFeedRanges(CosmosContain return container .getFeedRanges() - .map(feedRanges -> { - return feedRanges - .stream() - .map(feedRange -> FeedRangeInternal.normalizeRange(((FeedRangeEpkImpl) feedRange).getRange())) - .collect(Collectors.toList()); - }) - .flatMap(range -> { + .flatMap(feedRanges -> { FeedRangesMetadataTopicOffset topicOffset = this.offsetStorageReader .getFeedRangesMetadataOffset( @@ -205,11 +195,11 @@ private Mono shouldRequestTaskReconfigurationOnFeedRanges(CosmosContain return Mono.just(true); } - List> differences = + List differences = topicOffset .getFeedRanges() .stream() - .filter(normalizedFeedRange -> !range.contains(normalizedFeedRange)) + .filter(normalizedFeedRange -> !feedRanges.contains(normalizedFeedRange)) .collect(Collectors.toList()); if (differences.size() == 0) { @@ -239,7 +229,7 @@ private boolean feedRangesMetadataOffsetExists(CosmosContainerProperties contain private Mono shouldRequestTaskReconfigurationOnFeedRangeChanges( CosmosContainerProperties containerProperties, - List> changes) { + List changes) { if (changes == null || changes.isEmpty()) { return Mono.just(false); } @@ -259,26 +249,18 @@ private Mono shouldRequestTaskReconfigurationOnFeedRangeChanges( private Mono shouldRequestTaskReconfigurationOnFeedRangeChange( CosmosContainerProperties containerProperties, - Range feedRangeChanged) { - - AsyncDocumentClient asyncDocumentClient = BridgeInternal.getContextClient(this.cosmosClient); - - // find out whether it is a split or merge - // for split, we are going to request a task reconfiguration for load-balancing - // for merge, ignore as we are going to continue consuming base on the current feed range - return asyncDocumentClient - .getPartitionKeyRangeCache() - .tryGetOverlappingRangesAsync( - null, - containerProperties.getResourceId(), - feedRangeChanged, - false, - null - ) - .map(pkRangesValueHolder -> { - List matchedPkRanges = - (pkRangesValueHolder == null || pkRangesValueHolder.v == null) ? new ArrayList<>() : pkRangesValueHolder.v; - + FeedRange feedRangeChanged) { + + CosmosAsyncContainer container = + this.cosmosClient + .getDatabase(this.sourceContainersConfig.getDatabaseName()) + .getContainer(containerProperties.getId()); + + return ImplementationBridgeHelpers + .CosmosAsyncContainerHelper + .getCosmosAsyncContainerAccessor() + .getOverlappingFeedRanges(container, feedRangeChanged) + .map(matchedPkRanges -> { if (matchedPkRanges.size() == 0) { LOGGER.warn( "FeedRang {} on container {} is gone but we failed to find at least one matching pkRange", @@ -292,7 +274,7 @@ private Mono shouldRequestTaskReconfigurationOnFeedRangeChange( LOGGER.info( "FeedRange {} is merged into {} on container {}", feedRangeChanged, - matchedPkRanges.get(0).toRange(), + matchedPkRanges.get(0).toString(), containerProperties.getResourceId()); return false; } @@ -300,7 +282,7 @@ private Mono shouldRequestTaskReconfigurationOnFeedRangeChange( LOGGER.info( "FeedRange {} is split into [{}] on container {}", feedRangeChanged, - matchedPkRanges.stream().map(PartitionKeyRange::toRange).collect(Collectors.toList()), + matchedPkRanges.stream().map(FeedRange::toString).collect(Collectors.toList()), containerProperties.getResourceId() ); return true; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataTaskUnit.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataTaskUnit.java index 18a9e593567a4..aac3dd434cd78 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataTaskUnit.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataTaskUnit.java @@ -3,8 +3,8 @@ package com.azure.cosmos.kafka.connect.implementation.source; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.routing.Range; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.models.FeedRange; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.type.TypeReference; @@ -24,21 +24,21 @@ import java.util.Objects; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; @JsonSerialize(using = MetadataTaskUnit.MetadataTaskUnitSerializer.class) @JsonDeserialize(using = MetadataTaskUnit.MetadataTaskUnitDeserializer.class) public class MetadataTaskUnit implements ITaskUnit { private final String databaseName; private final List containerRids; - private final Map>> containersEffectiveRangesMap; + private final Map> containersEffectiveRangesMap; private final String topic; public MetadataTaskUnit( String databaseName, List containerRids, - Map>> containersEffectiveRangesMap, + Map> containersEffectiveRangesMap, String topic) { checkArgument(StringUtils.isNotEmpty(databaseName), "Argument 'databaseName' should not be null"); @@ -60,7 +60,7 @@ public List getContainerRids() { return containerRids; } - public Map>> getContainersEffectiveRangesMap() { + public Map> getContainersEffectiveRangesMap() { return containersEffectiveRangesMap; } @@ -125,7 +125,7 @@ public void serialize(MetadataTaskUnit metadataTaskUnit, getContainersEffectiveRangesMap(). get(containerRid) .stream() - .map(range -> range.toJson()) + .map(range -> range.toString()) .collect(Collectors.toList()))); writer.writeEndObject(); } @@ -153,15 +153,15 @@ public MetadataTaskUnit deserialize( List containerRids = mapper.readValue(rootNode.get("containerRids").asText(), new TypeReference>() {}); ArrayNode arrayNode = (ArrayNode) rootNode.get("containersEffectiveRangesMap"); - Map>> containersEffectiveRangesMap = new HashMap<>(); + Map> containersEffectiveRangesMap = new HashMap<>(); for (JsonNode jsonNode : arrayNode) { String containerRid = jsonNode.get("containerRid").asText(); - List> effectiveRanges = + List effectiveRanges = mapper .readValue( jsonNode.get("effectiveFeedRanges").asText(), new TypeReference>() {}) - .stream().map(rangeJson -> new Range(rangeJson)) + .stream().map(rangeJson -> FeedRange.fromString(rangeJson)) .collect(Collectors.toList()); containersEffectiveRangesMap.put(containerRid, effectiveRanges); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java index cfea2ab8a3213..79b8c3c044017 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/main/java/module-info.java @@ -7,10 +7,11 @@ requires transitive com.azure.cosmos; requires kafka.clients; requires connect.api; + requires com.fasterxml.jackson.module.afterburner; requires json.path; - // public API surface area exports com.azure.cosmos.kafka.connect; uses com.azure.core.util.tracing.Tracer; + uses com.azure.cosmos.kafka.connect.implementation.guava25.base.PatternCompiler; } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java index cc332096b5e16..404a048ef9f35 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSinkConnectorTest.java @@ -3,8 +3,7 @@ package com.azure.cosmos.kafka.connect; -import com.azure.cosmos.implementation.Strings; -import com.azure.cosmos.implementation.TestConfigurations; +import com.azure.cosmos.kafka.connect.implementation.Strings; import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkTask; import com.azure.cosmos.kafka.connect.implementation.sink.IdStrategies; import com.azure.cosmos.kafka.connect.implementation.sink.ItemWriteStrategy; @@ -67,8 +66,8 @@ public void taskConfigs() { CosmosDBSinkConnector sinkConnector = new CosmosDBSinkConnector(); Map sinkConfigMap = new HashMap<>(); - sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); - sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", KafkaCosmosTestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", KafkaCosmosTestConfigurations.MASTER_KEY); sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", singlePartitionContainerName + "#" + singlePartitionContainerName); sinkConnector.start(sinkConfigMap); @@ -78,8 +77,8 @@ public void taskConfigs() { assertThat(taskConfigs.size()).isEqualTo(maxTask); for (Map taskConfig : taskConfigs) { - assertThat(taskConfig.get("kafka.connect.cosmos.accountEndpoint")).isEqualTo(TestConfigurations.HOST); - assertThat(taskConfig.get("kafka.connect.cosmos.accountKey")).isEqualTo(TestConfigurations.MASTER_KEY); + assertThat(taskConfig.get("kafka.connect.cosmos.accountEndpoint")).isEqualTo(KafkaCosmosTestConfigurations.HOST); + assertThat(taskConfig.get("kafka.connect.cosmos.accountKey")).isEqualTo(KafkaCosmosTestConfigurations.MASTER_KEY); assertThat(taskConfig.get("kafka.connect.cosmos.sink.database.name")).isEqualTo(databaseName); assertThat(taskConfig.get("kafka.connect.cosmos.sink.containers.topicMap")) .isEqualTo(singlePartitionContainerName + "#" + singlePartitionContainerName); @@ -119,8 +118,8 @@ public void misFormattedConfig() { private Map getValidSinkConfig() { Map sinkConfigMap = new HashMap<>(); - sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); - sinkConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); + sinkConfigMap.put("kafka.connect.cosmos.accountEndpoint", KafkaCosmosTestConfigurations.HOST); + sinkConfigMap.put("kafka.connect.cosmos.accountKey", KafkaCosmosTestConfigurations.MASTER_KEY); sinkConfigMap.put("kafka.connect.cosmos.sink.database.name", databaseName); sinkConfigMap.put("kafka.connect.cosmos.sink.containers.topicMap", singlePartitionContainerName + "#" + singlePartitionContainerName); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java index a69da0cc6bf4b..efc23147cec37 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDBSourceConnectorTest.java @@ -7,8 +7,6 @@ import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.implementation.Strings; import com.azure.cosmos.implementation.TestConfigurations; -import com.azure.cosmos.implementation.Utils; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.implementation.changefeed.common.ChangeFeedMode; import com.azure.cosmos.implementation.changefeed.common.ChangeFeedStartFromInternal; import com.azure.cosmos.implementation.changefeed.common.ChangeFeedState; @@ -16,8 +14,9 @@ import com.azure.cosmos.implementation.feedranges.FeedRangeContinuation; import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.query.CompositeContinuationToken; -import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.kafka.connect.implementation.source.CosmosChangeFeedModes; import com.azure.cosmos.kafka.connect.implementation.source.CosmosChangeFeedStartFromModes; import com.azure.cosmos.kafka.connect.implementation.source.CosmosSourceConfig; @@ -28,6 +27,7 @@ import com.azure.cosmos.kafka.connect.implementation.source.FeedRangeTaskUnit; import com.azure.cosmos.kafka.connect.implementation.source.FeedRangesMetadataTopicOffset; import com.azure.cosmos.kafka.connect.implementation.source.FeedRangesMetadataTopicPartition; +import com.azure.cosmos.kafka.connect.implementation.source.KafkaCosmosChangeFeedState; import com.azure.cosmos.kafka.connect.implementation.source.MetadataMonitorThread; import com.azure.cosmos.kafka.connect.implementation.source.MetadataTaskUnit; import com.azure.cosmos.models.CosmosContainerProperties; @@ -180,7 +180,7 @@ public void getTaskConfigsAfterSplit() throws JsonProcessingException { new FeedRangeContinuationTopicPartition( databaseName, multiPartitionContainer.getResourceId(), - FeedRangeEpkImpl.forFullRange().getRange()); + FeedRange.forFullRange()); String initialContinuationState = new ChangeFeedStateV1( multiPartitionContainer.getResourceId(), @@ -203,7 +203,7 @@ public void getTaskConfigsAfterSplit() throws JsonProcessingException { FeedRangesMetadataTopicPartition feedRangesMetadataTopicPartition = new FeedRangesMetadataTopicPartition(databaseName, multiPartitionContainer.getResourceId()); FeedRangesMetadataTopicOffset feedRangesMetadataTopicOffset = - new FeedRangesMetadataTopicOffset(Arrays.asList(FeedRangeEpkImpl.forFullRange().getRange())); + new FeedRangesMetadataTopicOffset(Arrays.asList(FeedRange.forFullRange())); initialOffsetMap.put( FeedRangesMetadataTopicPartition.toMap(feedRangesMetadataTopicPartition), FeedRangesMetadataTopicOffset.toMap(feedRangesMetadataTopicOffset)); @@ -252,7 +252,6 @@ public void getTaskConfigsAfterMerge() throws JsonProcessingException { CosmosDBSourceConnector sourceConnector = new CosmosDBSourceConnector(); try { - Map sourceConfigMap = new HashMap<>(); sourceConfigMap.put("kafka.connect.cosmos.accountEndpoint", TestConfigurations.HOST); sourceConfigMap.put("kafka.connect.cosmos.accountKey", TestConfigurations.MASTER_KEY); @@ -272,7 +271,7 @@ public void getTaskConfigsAfterMerge() throws JsonProcessingException { CosmosContainerProperties singlePartitionContainer = getSinglePartitionContainer(cosmosAsyncClient); // constructing feed range continuation offset - List childRanges = + List childRanges = ImplementationBridgeHelpers .CosmosAsyncContainerHelper .getCosmosAsyncContainerAccessor() @@ -285,37 +284,40 @@ public void getTaskConfigsAfterMerge() throws JsonProcessingException { Map, Map> initialOffsetMap = new HashMap<>(); List singlePartitionFeedRangeTaskUnits = new ArrayList<>(); - for (FeedRangeEpkImpl childRange : childRanges) { + for (FeedRange childRange : childRanges) { FeedRangeContinuationTopicPartition feedRangeContinuationTopicPartition = new FeedRangeContinuationTopicPartition( databaseName, singlePartitionContainer.getResourceId(), - childRange.getRange()); + childRange); - String childRangeContinuationState = new ChangeFeedStateV1( + ChangeFeedStateV1 childRangeContinuationState = new ChangeFeedStateV1( singlePartitionContainer.getResourceId(), - childRange, + (FeedRangeEpkImpl)childRange, ChangeFeedMode.INCREMENTAL, ChangeFeedStartFromInternal.createFromBeginning(), FeedRangeContinuation.create( singlePartitionContainer.getResourceId(), - childRange, - Arrays.asList(new CompositeContinuationToken("1", childRange.getRange())))).toString(); + (FeedRangeEpkImpl)childRange, + Arrays.asList(new CompositeContinuationToken("1", ((FeedRangeEpkImpl)childRange).getRange())))); FeedRangeContinuationTopicOffset feedRangeContinuationTopicOffset = - new FeedRangeContinuationTopicOffset(childRangeContinuationState, "1"); + new FeedRangeContinuationTopicOffset(childRangeContinuationState.toString(), "1"); initialOffsetMap.put( FeedRangeContinuationTopicPartition.toMap(feedRangeContinuationTopicPartition), FeedRangeContinuationTopicOffset.toMap(feedRangeContinuationTopicOffset)); + KafkaCosmosChangeFeedState kafkaCosmosChangeFeedState = + new KafkaCosmosChangeFeedState(childRangeContinuationState.toString(), childRange, "1"); + String taskUnitContinuationState = KafkaCosmosUtils.getSimpleObjectMapper().writeValueAsString(kafkaCosmosChangeFeedState); singlePartitionFeedRangeTaskUnits.add( new FeedRangeTaskUnit( databaseName, singlePartitionContainer.getId(), singlePartitionContainer.getResourceId(), - childRange.getRange(), - childRangeContinuationState, + childRange, + taskUnitContinuationState, singlePartitionContainer.getId())); } @@ -326,7 +328,6 @@ public void getTaskConfigsAfterMerge() throws JsonProcessingException { new FeedRangesMetadataTopicOffset( childRanges .stream() - .map(childRange -> childRange.getRange()) .collect(Collectors.toList())); initialOffsetMap.put( @@ -353,10 +354,10 @@ public void getTaskConfigsAfterMerge() throws JsonProcessingException { validateFeedRangeTasks(expectedTaskUnits, taskConfigs); - Map>> containersEffectiveRangesMap = new HashMap<>(); + Map> containersEffectiveRangesMap = new HashMap<>(); containersEffectiveRangesMap.put( singlePartitionContainer.getResourceId(), - childRanges.stream().map(FeedRangeEpkImpl::getRange).collect(Collectors.toList())); + childRanges.stream().collect(Collectors.toList())); MetadataTaskUnit expectedMetadataTaskUnit = new MetadataTaskUnit( @@ -484,26 +485,23 @@ private List getFeedRangeTaskUnits( String feedRangeContinuationState = null; if (StringUtils.isNotEmpty(continuationState)) { ChangeFeedState changeFeedState = ChangeFeedStateV1.fromString(continuationState); - feedRangeContinuationState = - new ChangeFeedStateV1( - changeFeedState.getContainerRid(), - FeedRangeEpkImpl.forFullRange(), - ChangeFeedMode.INCREMENTAL, - ChangeFeedStartFromInternal.createFromBeginning(), - FeedRangeContinuation.create( - changeFeedState.getContainerRid(), - (FeedRangeEpkImpl)feedRange, - Arrays.asList( - new CompositeContinuationToken( - changeFeedState.getContinuation().getCurrentContinuationToken().getToken(), - ((FeedRangeEpkImpl)feedRange).getRange())))).toString(); + KafkaCosmosChangeFeedState kafkaCosmosChangeFeedState = + new KafkaCosmosChangeFeedState( + continuationState, + feedRange, + changeFeedState.getContinuation().getCurrentContinuationToken().getToken()); + try { + feedRangeContinuationState = KafkaCosmosUtils.getSimpleObjectMapper().writeValueAsString(kafkaCosmosChangeFeedState); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } } return new FeedRangeTaskUnit( databaseName, containerProperties.getId(), containerProperties.getResourceId(), - ((FeedRangeEpkImpl)feedRange).getRange(), + feedRange, feedRangeContinuationState, topicName); }) @@ -515,7 +513,7 @@ private MetadataTaskUnit getMetadataTaskUnit( String databaseName, List containers) { - Map>> containersEffectiveRangesMap = new HashMap<>(); + Map> containersEffectiveRangesMap = new HashMap<>(); for (CosmosContainerProperties containerProperties : containers) { List feedRanges = cosmosAsyncClient @@ -524,12 +522,7 @@ private MetadataTaskUnit getMetadataTaskUnit( .getFeedRanges() .block(); - containersEffectiveRangesMap.put( - containerProperties.getResourceId(), - feedRanges - .stream() - .map(feedRange -> ((FeedRangeEpkImpl)feedRange).getRange()) - .collect(Collectors.toList())); + containersEffectiveRangesMap.put(containerProperties.getResourceId(), feedRanges); } return new MetadataTaskUnit( @@ -542,29 +535,31 @@ private MetadataTaskUnit getMetadataTaskUnit( private void validateFeedRangeTasks( List> feedRangeTaskUnits, - List> taskConfig) throws JsonProcessingException { + List> taskConfigs) throws JsonProcessingException { String taskUnitsKey = "kafka.connect.cosmos.source.task.feedRangeTaskUnits"; - for (int i = 0; i< feedRangeTaskUnits.size(); i++) { - List expectedTaskUnits = feedRangeTaskUnits.get(i); - assertThat(taskConfig.get(i).containsKey(taskUnitsKey)).isTrue(); - List taskUnitsFromTaskConfig = - Utils - .getSimpleObjectMapper() - .readValue(taskConfig.get(i).get(taskUnitsKey), new TypeReference>() {}) - .stream() - .map(taskUnitString -> { - try { - return Utils.getSimpleObjectMapper().readValue(taskUnitString, FeedRangeTaskUnit.class); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } - }) - .collect(Collectors.toList()); - - assertThat(expectedTaskUnits.size()).isEqualTo(taskUnitsFromTaskConfig.size()); - assertThat(expectedTaskUnits.containsAll(taskUnitsFromTaskConfig)).isTrue(); + List allTaskUnitsFromTaskConfigs = new ArrayList<>(); + for (Map taskConfig : taskConfigs) { + List taskUnitsFromTaskConfig = KafkaCosmosUtils + .getSimpleObjectMapper() + .readValue(taskConfig.get(taskUnitsKey), new TypeReference>() {}) + .stream() + .map(taskUnitString -> { + try { + return KafkaCosmosUtils.getSimpleObjectMapper().readValue(taskUnitString, FeedRangeTaskUnit.class); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + allTaskUnitsFromTaskConfigs.addAll(taskUnitsFromTaskConfig); } + + List allExpectedTaskUnits = new ArrayList<>(); + feedRangeTaskUnits.forEach(taskUnits -> allExpectedTaskUnits.addAll(taskUnits)); + + assertThat(allExpectedTaskUnits.size()).isEqualTo(allTaskUnitsFromTaskConfigs.size()); + assertThat(allExpectedTaskUnits.containsAll(allTaskUnitsFromTaskConfigs)).isTrue(); } private void validateMetadataTask( @@ -574,7 +569,7 @@ private void validateMetadataTask( String taskUnitKey = "kafka.connect.cosmos.source.task.metadataTaskUnit"; assertThat(taskConfig.containsKey(taskUnitKey)); MetadataTaskUnit metadataTaskUnitFromTaskConfig = - Utils.getSimpleObjectMapper().readValue(taskConfig.get(taskUnitKey), MetadataTaskUnit.class); + KafkaCosmosUtils.getSimpleObjectMapper().readValue(taskConfig.get(taskUnitKey), MetadataTaskUnit.class); assertThat(expectedMetadataTaskUnit).isEqualTo(metadataTaskUnitFromTaskConfig); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java index 9a0714e9f5d03..a0869e70aa3b1 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/CosmosDbSinkConnectorITest.java @@ -5,8 +5,8 @@ import com.azure.cosmos.CosmosAsyncClient; import com.azure.cosmos.CosmosAsyncContainer; -import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; +import com.azure.cosmos.kafka.connect.implementation.KafkaCosmosUtils; import com.azure.cosmos.kafka.connect.implementation.sink.CosmosSinkConfig; import com.fasterxml.jackson.databind.JsonNode; import org.apache.kafka.clients.producer.KafkaProducer; @@ -65,7 +65,7 @@ public void sinkToSingleContainer() throws InterruptedException { for (int i = 0; i < 10; i++) { TestItem testItem = TestItem.createNewItem(); ProducerRecord record = - new ProducerRecord<>(singlePartitionContainerName, testItem.getId(), Utils.getSimpleObjectMapper().valueToTree(testItem)); + new ProducerRecord<>(singlePartitionContainerName, testItem.getId(), KafkaCosmosUtils.getSimpleObjectMapper().valueToTree(testItem)); kafkaProducer.send(record); recordValueIds.add(testItem.getId()); } diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosIntegrationTestSuiteBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosIntegrationTestSuiteBase.java index 6a113f45b0376..600de3a726037 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosIntegrationTestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosIntegrationTestSuiteBase.java @@ -3,7 +3,7 @@ package com.azure.cosmos.kafka.connect; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java index f26d86ce17286..01bf759fa5d3d 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestConfigurations.java @@ -3,8 +3,8 @@ package com.azure.cosmos.kafka.connect; -import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import com.azure.cosmos.implementation.guava25.base.Strings; +import com.azure.cosmos.kafka.connect.implementation.apachecommons.lang.StringUtils; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java index 8e601948a6c31..ef2401aef3634 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/KafkaCosmosTestSuiteBase.java @@ -12,7 +12,6 @@ import com.azure.cosmos.GatewayConnectionConfig; import com.azure.cosmos.ThrottlingRetryOptions; import com.azure.cosmos.implementation.ImplementationBridgeHelpers; -import com.azure.cosmos.implementation.guava27.Strings; import com.azure.cosmos.models.CosmosContainerProperties; import com.azure.cosmos.models.CosmosContainerRequestOptions; import com.azure.cosmos.models.CosmosItemRequestOptions; @@ -24,6 +23,7 @@ import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.shaded.com.google.common.base.Strings; import org.testng.ITest; import org.testng.annotations.AfterMethod; import org.testng.annotations.AfterSuite; diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskTest.java index e137624fe22d8..cfa257a644657 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/CosmosSourceTaskTest.java @@ -6,8 +6,6 @@ import com.azure.cosmos.CosmosAsyncClient; import com.azure.cosmos.CosmosAsyncContainer; import com.azure.cosmos.implementation.TestConfigurations; -import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; -import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.kafka.connect.KafkaCosmosTestSuiteBase; import com.azure.cosmos.kafka.connect.TestItem; import com.azure.cosmos.kafka.connect.implementation.CosmosClientStore; @@ -62,8 +60,8 @@ public void poll() throws InterruptedException { client.getDatabase(databaseName).getContainer(testContainerName).getFeedRanges().block(); assertThat(feedRanges.size()).isEqualTo(1); - Map>> containersEffectiveRangesMap = new HashMap<>(); - containersEffectiveRangesMap.put(testContainer.getResourceId(), Arrays.asList(FeedRangeEpkImpl.forFullRange().getRange())); + Map> containersEffectiveRangesMap = new HashMap<>(); + containersEffectiveRangesMap.put(testContainer.getResourceId(), Arrays.asList(FeedRange.forFullRange())); MetadataTaskUnit metadataTaskUnit = new MetadataTaskUnit( databaseName, Arrays.asList(testContainer.getResourceId()), @@ -76,7 +74,7 @@ public void poll() throws InterruptedException { databaseName, testContainerName, testContainer.getResourceId(), - FeedRangeEpkImpl.forFullRange().getRange(), + FeedRange.forFullRange(), null, testContainerName); taskConfigMap.putAll(CosmosSourceTaskConfig.getFeedRangeTaskUnitsConfigMap(Arrays.asList(feedRangeTaskUnit))); @@ -158,7 +156,7 @@ public void pollWithSpecificFeedRange() { databaseName, multiPartitionContainer.getId(), multiPartitionContainer.getResourceId(), - ((FeedRangeEpkImpl)feedRanges.get(0)).getRange(), + feedRanges.get(0), null, multiPartitionContainer.getId()); taskConfigMap.putAll(CosmosSourceTaskConfig.getFeedRangeTaskUnitsConfigMap(Arrays.asList(feedRangeTaskUnit))); @@ -211,7 +209,7 @@ private void validateMetadataRecords(List sourceRecords, MetadataT for (int i = 0; i < metadataTaskUnit.getContainerRids().size(); i++) { String containerRid = metadataTaskUnit.getContainerRids().get(i); SourceRecord sourceRecord = sourceRecords.get(i + 1); - List> containerFeedRanges = + List containerFeedRanges = metadataTaskUnit.getContainersEffectiveRangesMap().get(containerRid); assertThat(containerFeedRanges).isNotNull(); diff --git a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java index 76479de109810..4b3a565a7c886 100644 --- a/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java +++ b/sdk/cosmos/azure-cosmos-kafka-connect/src/test/java/com/azure/cosmos/kafka/connect/implementation/source/MetadataMonitorThreadTest.java @@ -6,7 +6,6 @@ import com.azure.cosmos.CosmosAsyncClient; import com.azure.cosmos.implementation.ImplementationBridgeHelpers; import com.azure.cosmos.implementation.TestConfigurations; -import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.kafka.connect.InMemoryStorageReader; import com.azure.cosmos.kafka.connect.KafkaCosmosTestSuiteBase; import com.azure.cosmos.kafka.connect.implementation.CosmosAccountConfig; @@ -152,7 +151,7 @@ public void requestTaskReconfigurationOnSplit() throws InterruptedException { FeedRangesMetadataTopicPartition feedRangesMetadataTopicPartition = new FeedRangesMetadataTopicPartition(databaseName, multiPartitionContainer.getResourceId()); FeedRangesMetadataTopicOffset feedRangesMetadataTopicOffset = - new FeedRangesMetadataTopicOffset(Arrays.asList(FeedRangeEpkImpl.forFullRange().getRange())); + new FeedRangesMetadataTopicOffset(Arrays.asList(FeedRange.forFullRange())); Map, Map> feedRangesOffSetMap = new HashMap<>(); feedRangesOffSetMap.put( @@ -220,7 +219,7 @@ public void requestTaskReconfigurationOnMerge() throws InterruptedException { .block(); assertThat(feedRanges.size()).isEqualTo(1); - List childRanges = + List childRanges = ImplementationBridgeHelpers .CosmosAsyncContainerHelper .getCosmosAsyncContainerAccessor() @@ -236,7 +235,6 @@ public void requestTaskReconfigurationOnMerge() throws InterruptedException { new FeedRangesMetadataTopicOffset( childRanges .stream() - .map(FeedRangeEpkImpl::getRange) .collect(Collectors.toList())); Map, Map> feedRangesOffSetMap = new HashMap<>(); diff --git a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/CHANGELOG.md b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/CHANGELOG.md index 73b2b33b19fe5..a98c4fd3803e3 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/CHANGELOG.md @@ -1,5 +1,20 @@ ## Release History +### 4.29.0-beta.1 (Unreleased) + +#### Features Added + +#### Breaking Changes + +#### Bugs Fixed + +#### Other Changes + +### 4.28.4 (2024-03-18) + +#### Other Changes +* Increased queue length of Scheduler in `BulkWriter` by using different schedulers to handle request and response to avoid `ReactorRejectedExecutionException: Scheduler unavailable` error message. - See [PR 39260](https://github.com/Azure/azure-sdk-for-java/pull/39260) + ### 4.28.3 (2024-03-12) #### Other Changes diff --git a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/README.md b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/README.md index 416a6d3cca486..4b0a6b7687622 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/README.md +++ b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/README.md @@ -29,6 +29,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-1_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |--------------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.1.1 - 3.1.2 | [8, 11] | 2.12 | 8.\*, 9.\* | | 4.28.3 | 3.1.1 - 3.1.2 | [8, 11] | 2.12 | 8.\*, 9.\* | | 4.28.2 | 3.1.1 - 3.1.2 | [8, 11] | 2.12 | 8.\*, 9.\* | | 4.28.1 | 3.1.1 - 3.1.2 | [8, 11] | 2.12 | 8.\*, 9.\* | @@ -91,6 +92,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-2_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | @@ -138,6 +140,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-3_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.3 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.2 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.1 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | @@ -167,6 +170,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-4_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.4.0 | [8, 11] | 2.12 | 13.\* | | 4.28.3 | 3.4.0 | [8, 11] | 2.12 | 13.\* | | 4.28.2 | 3.4.0 | [8, 11] | 2.12 | 13.\* | | 4.28.1 | 3.4.0 | [8, 11] | 2.12 | 13.\* | @@ -188,11 +192,11 @@ https://github.com/Azure/azure-sdk-for-java/issues/new ### Download You can use the maven coordinate of the jar to auto install the Spark Connector to your Databricks Runtime 8 from Maven: -`com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12:4.28.3` +`com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12:4.28.4` You can also integrate against Cosmos DB Spark Connector in your SBT project: ```scala -libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-1_2-12" % "4.28.3" +libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-1_2-12" % "4.28.4" ``` Cosmos DB Spark Connector is available on [Maven Central Repo](https://central.sonatype.com/search?namespace=com.azure.cosmos.spark). diff --git a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/pom.xml index 348f9bbff12e1..999ae9d2e0cd8 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/pom.xml @@ -11,7 +11,7 @@ com.azure.cosmos.spark azure-cosmos-spark_3-1_2-12 - 4.28.3 + 4.29.0-beta.1 jar https://github.com/Azure/azure-sdk-for-java/tree/main/sdk/cosmos/azure-cosmos-spark_3-1_2-12 OLTP Spark 3.1 Connector for Azure Cosmos DB SQL API diff --git a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala index 10537c098aa2d..ae3a95f9e93eb 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-1_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-1_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala @@ -2,8 +2,8 @@ // Licensed under the MIT License. package com.azure.cosmos.spark -import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull -import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects.firstNonNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings.emptyToNull import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait import org.apache.spark.TaskContext diff --git a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/CHANGELOG.md b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/CHANGELOG.md index f377ec1c5a778..8e996b0f91453 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/CHANGELOG.md @@ -1,5 +1,20 @@ ## Release History +### 4.29.0-beta.1 (Unreleased) + +#### Features Added + +#### Breaking Changes + +#### Bugs Fixed + +#### Other Changes + +### 4.28.4 (2024-03-18) + +#### Other Changes +* Increased queue length of Scheduler in `BulkWriter` by using different schedulers to handle request and response to avoid `ReactorRejectedExecutionException: Scheduler unavailable` error message. - See [PR 39260](https://github.com/Azure/azure-sdk-for-java/pull/39260) + ### 4.28.3 (2024-03-12) #### Other Changes diff --git a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/README.md b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/README.md index d973d3a593037..42df32000bc12 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/README.md +++ b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/README.md @@ -28,6 +28,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-2_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | @@ -75,6 +76,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-3_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.3 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.2 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.1 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | @@ -104,6 +106,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-1_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |--------------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 8.\*, 9.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 8.\*, 9.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 8.\*, 9.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 8.\*, 9.\* | @@ -166,6 +169,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-4_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.3 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.2 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.1 | 3.4.0 | [8, 11] | 2.12 | 13.* | @@ -186,11 +190,11 @@ https://github.com/Azure/azure-sdk-for-java/issues/new ### Download You can use the maven coordinate of the jar to auto install the Spark Connector to your Databricks Runtime 10 from Maven: -`com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12:4.28.3` +`com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12:4.28.4` You can also integrate against Cosmos DB Spark Connector in your SBT project: ```scala -libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-2_2-12" % "4.28.3" +libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-2_2-12" % "4.28.4" ``` Cosmos DB Spark Connector is available on [Maven Central Repo](https://central.sonatype.com/search?namespace=com.azure.cosmos.spark). diff --git a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/pom.xml index e810ffb08b59e..2f3371d4d9921 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/pom.xml @@ -11,7 +11,7 @@ com.azure.cosmos.spark azure-cosmos-spark_3-2_2-12 - 4.28.3 + 4.29.0-beta.1 jar https://github.com/Azure/azure-sdk-for-java/tree/main/sdk/cosmos/azure-cosmos-spark_3-2_2-12 OLTP Spark 3.2 Connector for Azure Cosmos DB SQL API diff --git a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala index 96835661e0abb..2479dcfd8e848 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-2_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-2_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala @@ -2,8 +2,8 @@ // Licensed under the MIT License. package com.azure.cosmos.spark -import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull -import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects.firstNonNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings.emptyToNull import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait import org.apache.spark.TaskContext import org.apache.spark.sql.connector.metric.CustomTaskMetric diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/CHANGELOG.md b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/CHANGELOG.md index 556211be419ba..544baa6152db4 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/CHANGELOG.md @@ -1,5 +1,20 @@ ## Release History +### 4.29.0-beta.1 (Unreleased) + +#### Features Added + +#### Breaking Changes + +#### Bugs Fixed + +#### Other Changes + +### 4.28.4 (2024-03-18) + +#### Other Changes +* Increased queue length of Scheduler in `BulkWriter` by using different schedulers to handle request and response to avoid `ReactorRejectedExecutionException: Scheduler unavailable` error message. - See [PR 39260](https://github.com/Azure/azure-sdk-for-java/pull/39260) + ### 4.28.3 (2024-03-12) #### Other Changes diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/README.md b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/README.md index 93169c1208571..65e578c697427 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/README.md +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/README.md @@ -28,6 +28,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-3_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.3 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.2 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.1 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | @@ -57,6 +58,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-2_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | @@ -104,6 +106,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-1_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |--------------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | @@ -166,6 +169,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-4_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.3 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.2 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.1 | 3.4.0 | [8, 11] | 2.12 | 13.* | @@ -186,11 +190,11 @@ https://github.com/Azure/azure-sdk-for-java/issues/new ### Download You can use the maven coordinate of the jar to auto install the Spark Connector to your Databricks Runtime 11 from Maven: -`com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12:4.28.3` +`com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12:4.28.4` You can also integrate against Cosmos DB Spark Connector in your SBT project: ```scala -libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-3_2-12" % "4.28.3" +libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-3_2-12" % "4.28.4" ``` Cosmos DB Spark Connector is available on [Maven Central Repo](https://central.sonatype.com/search?namespace=com.azure.cosmos.spark). diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml index 51e26e974238b..05d3d29bb7e20 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml @@ -11,7 +11,7 @@ com.azure.cosmos.spark azure-cosmos-spark_3-3_2-12 - 4.28.3 + 4.29.0-beta.1 jar https://github.com/Azure/azure-sdk-for-java/tree/main/sdk/cosmos/azure-cosmos-spark_3-3_2-12 OLTP Spark 3.3 Connector for Azure Cosmos DB SQL API diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala index 8ee9ecb84edd6..bf25011867e54 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala @@ -2,8 +2,8 @@ // Licensed under the MIT License. package com.azure.cosmos.spark -import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull -import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects.firstNonNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings.emptyToNull import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait import org.apache.spark.TaskContext import org.apache.spark.sql.execution.metric.SQLMetric diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/CHANGELOG.md b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/CHANGELOG.md index c8e1ce708d3ed..c8f066714c27e 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/CHANGELOG.md @@ -1,5 +1,20 @@ ## Release History +### 4.29.0-beta.1 (Unreleased) + +#### Features Added + +#### Breaking Changes + +#### Bugs Fixed + +#### Other Changes + +### 4.28.4 (2024-03-18) + +#### Other Changes +* Increased queue length of Scheduler in `BulkWriter` by using different schedulers to handle request and response to avoid `ReactorRejectedExecutionException: Scheduler unavailable` error message. - See [PR 39260](https://github.com/Azure/azure-sdk-for-java/pull/39260) + ### 4.28.3 (2024-03-12) #### Other Changes diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/README.md b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/README.md index 9f731eec2fd4f..04b0baa8eb176 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/README.md +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/README.md @@ -28,6 +28,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-4_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.3 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.2 | 3.4.0 | [8, 11] | 2.12 | 13.* | | 4.28.1 | 3.4.0 | [8, 11] | 2.12 | 13.* | @@ -48,6 +49,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-3_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.3 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.2 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | | 4.28.1 | 3.3.0 | [8, 11] | 2.12 | 11.\*, 12.\* | @@ -77,6 +79,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-2_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |-----------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | @@ -124,6 +127,7 @@ https://github.com/Azure/azure-sdk-for-java/issues/new #### azure-cosmos-spark_3-1_2-12 | Connector | Supported Spark Versions | Supported JVM Versions | Supported Scala Versions | Supported Databricks Runtimes | |--------------|--------------------------|------------------------|--------------------------|-------------------------------| +| 4.28.4 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.3 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.2 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | | 4.28.1 | 3.2.0 - 3.2.1 | [8, 11] | 2.12 | 10.\* | @@ -186,11 +190,11 @@ https://github.com/Azure/azure-sdk-for-java/issues/new ### Download You can use the maven coordinate of the jar to auto install the Spark Connector to your Databricks Runtime 11 from Maven: -`com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12:4.28.3` +`com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12:4.28.4` You can also integrate against Cosmos DB Spark Connector in your SBT project: ```scala -libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-4_2-12" % "4.28.3" +libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_3-4_2-12" % "4.28.4" ``` Cosmos DB Spark Connector is available on [Maven Central Repo](https://central.sonatype.com/search?namespace=com.azure.cosmos.spark). diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml index 129e0e095c9bb..261372acd86e0 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml @@ -11,7 +11,7 @@ com.azure.cosmos.spark azure-cosmos-spark_3-4_2-12 - 4.28.3 + 4.29.0-beta.1 jar https://github.com/Azure/azure-sdk-for-java/tree/main/sdk/cosmos/azure-cosmos-spark_3-4_2-12 OLTP Spark 3.4 Connector for Azure Cosmos DB SQL API diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala index 3d4050c3ee595..6455abf4e5810 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala @@ -2,8 +2,8 @@ // Licensed under the MIT License. package com.azure.cosmos.spark -import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull -import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects.firstNonNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings.emptyToNull import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait import org.apache.spark.TaskContext import org.apache.spark.sql.execution.metric.SQLMetric diff --git a/sdk/cosmos/azure-cosmos-spark_3_2-12/docs/quick-start.md b/sdk/cosmos/azure-cosmos-spark_3_2-12/docs/quick-start.md index 62d7a92b37e36..203440c0452e2 100644 --- a/sdk/cosmos/azure-cosmos-spark_3_2-12/docs/quick-start.md +++ b/sdk/cosmos/azure-cosmos-spark_3_2-12/docs/quick-start.md @@ -26,16 +26,16 @@ You can use any other Spark 3.1.1 spark offering as well, also you should be abl SLF4J is only needed if you plan to use logging, please also download an SLF4J binding which will link the SLF4J API with the logging implementation of your choice. See the [SLF4J user manual](https://www.slf4j.org/manual.html) for more information. For Spark 3.1: -- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12:4.28.3](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-1_2-12/4.28.3/jar) +- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-1_2-12:4.28.4](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-1_2-12/4.28.4/jar) For Spark 3.2: -- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12:4.28.3](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-2_2-12/4.28.3/jar) +- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-2_2-12:4.28.4](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-2_2-12/4.28.4/jar) For Spark 3.3: -- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12:4.28.3](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-3_2-12/4.28.3/jar) +- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12:4.28.4](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-3_2-12/4.28.4/jar) For Spark 3.4: -- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12:4.28.3](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-4_2-12/4.28.3/jar) +- Install Cosmos DB Spark Connector, in your spark Cluster [com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12:4.28.4](https://search.maven.org/artifact/com.azure.cosmos.spark/azure-cosmos-spark_3-4_2-12/4.28.4/jar) The getting started guide is based on PySpark however you can use the equivalent scala version as well, and you can run the following code snippet in an Azure Databricks PySpark notebook. diff --git a/sdk/cosmos/azure-cosmos-spark_3_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3_2-12/pom.xml index 38bcfde1cfa95..04f7743e5a7c2 100644 --- a/sdk/cosmos/azure-cosmos-spark_3_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3_2-12/pom.xml @@ -63,7 +63,7 @@ com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 org.scala-lang.modules @@ -84,12 +84,12 @@ com.azure.resourcemanager azure-resourcemanager-cosmos - 2.36.0 + 2.37.0 com.azure azure-identity - 1.11.3 + 1.11.4 com.azure @@ -125,7 +125,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/implementation/SparkBridgeImplementationInternal.scala b/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/implementation/SparkBridgeImplementationInternal.scala index b0016b38409b1..a74ddc23344ac 100644 --- a/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/implementation/SparkBridgeImplementationInternal.scala +++ b/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/implementation/SparkBridgeImplementationInternal.scala @@ -5,8 +5,8 @@ package com.azure.cosmos.implementation import com.azure.cosmos.implementation.ImplementationBridgeHelpers.CosmosClientBuilderHelper import com.azure.cosmos.implementation.changefeed.common.{ChangeFeedMode, ChangeFeedStartFromInternal, ChangeFeedState, ChangeFeedStateV1} -import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull -import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.MoreObjects.firstNonNull +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings.emptyToNull import com.azure.cosmos.implementation.query.CompositeContinuationToken import com.azure.cosmos.implementation.routing.Range import com.azure.cosmos.models.{FeedRange, PartitionKey, PartitionKeyBuilder, PartitionKeyDefinition, SparkModelBridgeInternal} diff --git a/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/BulkWriter.scala b/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/BulkWriter.scala index f097230c3deac..e1fafc364d858 100644 --- a/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/BulkWriter.scala +++ b/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/BulkWriter.scala @@ -8,8 +8,7 @@ import com.azure.cosmos.{BridgeInternal, CosmosAsyncContainer, CosmosDiagnostics import com.azure.cosmos.implementation.apachecommons.lang.StringUtils import com.azure.cosmos.implementation.batch.{BatchRequestResponseConstants, BulkExecutorDiagnosticsTracker, ItemBulkOperation} import com.azure.cosmos.models._ -import com.azure.cosmos.spark.BulkWriter.{BulkOperationFailedException, bulkWriterBoundedElastic, getThreadInfo, readManyBoundedElastic} -import com.azure.cosmos.spark.CosmosConstants.StatusCodes +import com.azure.cosmos.spark.BulkWriter.{BulkOperationFailedException, bulkWriterRequestsBoundedElastic, bulkWriterResponsesBoundedElastic, getThreadInfo, readManyBoundedElastic} import com.azure.cosmos.spark.diagnostics.DefaultDiagnostics import reactor.core.Scannable import reactor.core.publisher.Mono @@ -23,7 +22,7 @@ import scala.collection.mutable import scala.concurrent.duration.Duration // scalastyle:on underscore.import import com.azure.cosmos.implementation.ImplementationBridgeHelpers -import com.azure.cosmos.implementation.guava25.base.Preconditions +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions import com.azure.cosmos.implementation.spark.{OperationContextAndListenerTuple, OperationListener} import com.azure.cosmos.models.PartitionKey import com.azure.cosmos.spark.BulkWriter.{DefaultMaxPendingOperationPerCore, emitFailureHandler} @@ -157,15 +156,12 @@ private class BulkWriter(container: CosmosAsyncContainer, writeConfig.maxMicroBatchSize match { case Some(customMaxMicroBatchSize) => - ImplementationBridgeHelpers.CosmosBulkExecutionOptionsHelper - .getCosmosBulkExecutionOptionsAccessor - .setMaxMicroBatchSize( - cosmosBulkExecutionOptions, - Math.max( - 1, - Math.min(customMaxMicroBatchSize, BatchRequestResponseConstants.MAX_OPERATIONS_IN_DIRECT_MODE_BATCH_REQUEST) - ) - ) + cosmosBulkExecutionOptions.setMaxMicroBatchSize( + Math.max( + 1, + Math.min(customMaxMicroBatchSize, BatchRequestResponseConstants.MAX_OPERATIONS_IN_DIRECT_MODE_BATCH_REQUEST) + ) + ) case None => } @@ -461,7 +457,7 @@ private class BulkWriter(container: CosmosAsyncContainer, }) val requestOperationContext = ReadManyOperation.operationContext if (shouldRetry(e.getStatusCode, e.getSubStatusCode, requestOperationContext)) { - log.logWarning(s"for itemId=[${requestOperationContext.itemId}], partitionKeyValue=[${requestOperationContext.partitionKeyValue}], " + + log.logInfo(s"for itemId=[${requestOperationContext.itemId}], partitionKeyValue=[${requestOperationContext.partitionKeyValue}], " + s"encountered status code '${e.getStatusCode}:${e.getSubStatusCode}' in read many, will retry! " + s"attemptNumber=${requestOperationContext.attemptNumber}, exceptionMessage=${e.getMessage}, " + s"Context: {${operationContext.toString}} $getThreadInfo") @@ -555,9 +551,10 @@ private class BulkWriter(container: CosmosAsyncContainer, val bulkOperationResponseFlux: SFlux[CosmosBulkOperationResponse[Object]] = container .executeBulkOperations[Object]( - bulkInputEmitter.asFlux().publishOn(bulkWriterBoundedElastic), + bulkInputEmitter.asFlux().publishOn(bulkWriterRequestsBoundedElastic), cosmosBulkExecutionOptions) - .publishOn(bulkWriterBoundedElastic) + .onBackpressureBuffer() + .publishOn(bulkWriterResponsesBoundedElastic) .asScala bulkOperationResponseFlux.subscribe( @@ -1252,7 +1249,8 @@ private object BulkWriter { private val maxDelayOn408RequestTimeoutInMs = 10000 private val minDelayOn408RequestTimeoutInMs = 1000 private val maxItemOperationsToShowInErrorMessage = 10 - private val BULK_WRITER_BOUNDED_ELASTIC_THREAD_NAME = "bulk-writer-bounded-elastic" + private val BULK_WRITER_REQUESTS_BOUNDED_ELASTIC_THREAD_NAME = "bulk-writer-requests-bounded-elastic" + private val BULK_WRITER_RESPONSES_BOUNDED_ELASTIC_THREAD_NAME = "bulk-writer-responses-bounded-elastic" private val READ_MANY_BOUNDED_ELASTIC_THREAD_NAME = "read-many-bounded-elastic" private val TTL_FOR_SCHEDULER_WORKER_IN_SECONDS = 60 // same as BoundedElasticScheduler.DEFAULT_TTL_SECONDS @@ -1308,17 +1306,25 @@ private object BulkWriter { private val bulkProcessingThresholds = new CosmosBulkExecutionThresholdsState() + // Custom bounded elastic scheduler to consume input flux + val bulkWriterRequestsBoundedElastic: Scheduler = Schedulers.newBoundedElastic( + Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE, + Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE + DefaultMaxPendingOperationPerCore, + BULK_WRITER_REQUESTS_BOUNDED_ELASTIC_THREAD_NAME, + TTL_FOR_SCHEDULER_WORKER_IN_SECONDS, true) + // Custom bounded elastic scheduler to switch off IO thread to process response. - val bulkWriterBoundedElastic: Scheduler = Schedulers.newBoundedElastic( - 2 * Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE, - Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, - BULK_WRITER_BOUNDED_ELASTIC_THREAD_NAME, + val bulkWriterResponsesBoundedElastic: Scheduler = Schedulers.newBoundedElastic( + Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE, + Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE + DefaultMaxPendingOperationPerCore, + BULK_WRITER_RESPONSES_BOUNDED_ELASTIC_THREAD_NAME, TTL_FOR_SCHEDULER_WORKER_IN_SECONDS, true) + // Custom bounded elastic scheduler to switch off IO thread to process response. val readManyBoundedElastic: Scheduler = Schedulers.newBoundedElastic( 2 * Schedulers.DEFAULT_BOUNDED_ELASTIC_SIZE, - Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE, + Schedulers.DEFAULT_BOUNDED_ELASTIC_QUEUESIZE + DefaultMaxPendingOperationPerCore, READ_MANY_BOUNDED_ELASTIC_THREAD_NAME, TTL_FOR_SCHEDULER_WORKER_IN_SECONDS, true) diff --git a/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/PointWriter.scala b/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/PointWriter.scala index 6acf3678a1c40..a7e5734f84fda 100644 --- a/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/PointWriter.scala +++ b/sdk/cosmos/azure-cosmos-spark_3_2-12/src/main/scala/com/azure/cosmos/spark/PointWriter.scala @@ -5,7 +5,7 @@ package com.azure.cosmos.spark import com.azure.cosmos.implementation.ImplementationBridgeHelpers.CosmosItemRequestOptionsHelper import com.azure.cosmos.implementation.apachecommons.lang.StringUtils -import com.azure.cosmos.implementation.guava25.base.Preconditions.checkState +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState import com.azure.cosmos.implementation.spark.{OperationContextAndListenerTuple, OperationListener} import com.azure.cosmos.models.{CosmosItemRequestOptions, CosmosItemResponse, CosmosPatchItemRequestOptions, PartitionKey, PartitionKeyDefinition} import com.azure.cosmos.spark.BulkWriter.getThreadInfo diff --git a/sdk/cosmos/azure-cosmos-test/pom.xml b/sdk/cosmos/azure-cosmos-test/pom.xml index ac00dcded94f7..4278d9a7ed49d 100644 --- a/sdk/cosmos/azure-cosmos-test/pom.xml +++ b/sdk/cosmos/azure-cosmos-test/pom.xml @@ -56,7 +56,7 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 @@ -130,13 +130,13 @@ Licensed under the MIT License. net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConditionBuilder.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConditionBuilder.java index 6e9741e171837..2cb29355dbabb 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConditionBuilder.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConditionBuilder.java @@ -5,8 +5,8 @@ import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /** * Fault injection condition builder. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConnectionErrorResultBuilder.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConnectionErrorResultBuilder.java index ea202d6238617..0f74e8fb2d2ad 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConnectionErrorResultBuilder.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionConnectionErrorResultBuilder.java @@ -5,8 +5,8 @@ import java.time.Duration; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injection connection error result builder. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionEndpointBuilder.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionEndpointBuilder.java index 49758df311b7a..404c6b595736a 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionEndpointBuilder.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionEndpointBuilder.java @@ -5,8 +5,8 @@ import com.azure.cosmos.models.FeedRange; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injection endpoint builder. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRule.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRule.java index 3dec6336b0466..d8ecc6324ecea 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRule.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRule.java @@ -12,7 +12,7 @@ import java.util.Map; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injection rule. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRuleBuilder.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRuleBuilder.java index 303d5ee933284..f757c29f37826 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRuleBuilder.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionRuleBuilder.java @@ -8,8 +8,8 @@ import java.time.Duration; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * The fault injection rule builder. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionServerErrorResultBuilder.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionServerErrorResultBuilder.java index 6f3a5c7f918dd..8997db6d255de 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionServerErrorResultBuilder.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/faultinjection/FaultInjectionServerErrorResultBuilder.java @@ -5,7 +5,7 @@ import java.time.Duration; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injection server error result builder. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionConnectionErrorRule.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionConnectionErrorRule.java index 20b6ad789fa8f..0e998a12142c2 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionConnectionErrorRule.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionConnectionErrorRule.java @@ -14,8 +14,8 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class FaultInjectionConnectionErrorRule implements IFaultInjectionRuleInternal { private final String id; diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleProcessor.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleProcessor.java index 9846a1f620d84..d17612ea4bb97 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleProcessor.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleProcessor.java @@ -43,8 +43,8 @@ import java.util.List; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /** * Enrich the rule with required information: for example physical addresses diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleStore.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleStore.java index 3d5b522868a2b..79dd744da6573 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleStore.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionRuleStore.java @@ -17,8 +17,8 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; import static com.azure.cosmos.test.faultinjection.FaultInjectionConnectionType.DIRECT; import static com.azure.cosmos.test.faultinjection.FaultInjectionConnectionType.GATEWAY; diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorResultInternal.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorResultInternal.java index 145d7440b9d45..c44640d84652d 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorResultInternal.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorResultInternal.java @@ -25,7 +25,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; public class FaultInjectionServerErrorResultInternal { private final FaultInjectionServerErrorType serverErrorType; diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorRule.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorRule.java index b30cf15aff2b1..a9e459f4c4ba4 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorRule.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectionServerErrorRule.java @@ -18,8 +18,8 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; public class FaultInjectionServerErrorRule implements IFaultInjectionRuleInternal { private final String id; diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectorProvider.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectorProvider.java index c0ba1cc303391..b4be831000153 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectorProvider.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/FaultInjectorProvider.java @@ -17,7 +17,7 @@ import java.util.List; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injector provider. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/RntbdConnectionErrorInjector.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/RntbdConnectionErrorInjector.java index 4a363b33d853a..5f3100879e55f 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/RntbdConnectionErrorInjector.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/RntbdConnectionErrorInjector.java @@ -15,7 +15,7 @@ import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injector which can handle {@link FaultInjectionConnectionErrorRule} with direct connection type. diff --git a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/ServerErrorInjector.java b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/ServerErrorInjector.java index 962115467e061..4d56d51a15f80 100644 --- a/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/ServerErrorInjector.java +++ b/sdk/cosmos/azure-cosmos-test/src/main/java/com/azure/cosmos/test/implementation/faultinjection/ServerErrorInjector.java @@ -10,7 +10,7 @@ import java.time.Duration; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /*** * Fault injector which can handle {@link FaultInjectionServerErrorRule} for both direct and gateway connection type. diff --git a/sdk/cosmos/azure-cosmos-tests/pom.xml b/sdk/cosmos/azure-cosmos-tests/pom.xml index c95636f72ac1d..e6f1af910b016 100644 --- a/sdk/cosmos/azure-cosmos-tests/pom.xml +++ b/sdk/cosmos/azure-cosmos-tests/pom.xml @@ -80,7 +80,7 @@ Licensed under the MIT License. --add-opens com.azure.cosmos/com.azure.cosmos.implementation.directconnectivity.TcpServerMock.rntbd=com.fasterxml.jackson.databind --add-opens com.azure.cosmos/com.azure.cosmos.implementation.feedranges=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.apachecommons.lang=ALL-UNNAMED - --add-opens com.azure.cosmos/com.azure.cosmos.implementation.guava25.base=ALL-UNNAMED + --add-opens com.azure.cosmos/com.azure.cosmos.kafka.connect.implementation.guava25.base=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.http=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.patch=ALL-UNNAMED --add-opens com.azure.cosmos/com.azure.cosmos.implementation.query=ALL-UNNAMED @@ -99,7 +99,7 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 com.azure @@ -150,7 +150,7 @@ Licensed under the MIT License. com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -174,7 +174,7 @@ Licensed under the MIT License. io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -200,13 +200,13 @@ Licensed under the MIT License. net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/ClientMetricsTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/ClientMetricsTest.java index 4216d40a14e23..1c30df6c27289 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/ClientMetricsTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/ClientMetricsTest.java @@ -24,7 +24,7 @@ import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdDurableEndpointMetrics; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdEndpoint; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdServiceEndpoint; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.implementation.routing.LocationCache; import com.azure.cosmos.models.CosmosBatch; import com.azure.cosmos.models.CosmosBatchResponse; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosBulkTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosBulkTest.java index f8d332edc29a9..73085adb028c7 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosBulkTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosBulkTest.java @@ -4,8 +4,8 @@ package com.azure.cosmos; import com.azure.cosmos.implementation.ISessionToken; -import com.azure.cosmos.implementation.guava25.base.Function; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.models.CosmosBulkExecutionOptions; import com.azure.cosmos.models.CosmosBulkItemRequestOptions; import com.azure.cosmos.models.CosmosBulkOperations; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosContainerChangeFeedTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosContainerChangeFeedTest.java index 446195305a73d..ca3be85f45e7f 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosContainerChangeFeedTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosContainerChangeFeedTest.java @@ -11,8 +11,8 @@ import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.feedranges.FeedRangeInternal; -import com.azure.cosmos.implementation.guava25.collect.ArrayListMultimap; -import com.azure.cosmos.implementation.guava25.collect.Multimap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ArrayListMultimap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap; import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.models.ChangeFeedPolicy; import com.azure.cosmos.models.CosmosChangeFeedRequestOptions; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosDiagnosticsE2ETest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosDiagnosticsE2ETest.java index df2024c74a414..314f5a031801f 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosDiagnosticsE2ETest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosDiagnosticsE2ETest.java @@ -39,6 +39,7 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; @@ -132,6 +133,7 @@ public void onlyDefaultLogger() { CosmosContainer container = this.getContainer(builder); executeTestCase(container); + // no assertions here - invocations for diagnostics handler are validated above // log4j event logging isn't validated in general in unit tests because it is too brittle to do so // with custom appender @@ -152,8 +154,33 @@ public void onlyLoggerWithCustomConfig() { .diagnosticsHandler(CosmosDiagnosticsHandler.DEFAULT_LOGGING_HANDLER) ); CosmosContainer container = this.getContainer(builder); - executeTestCase(container); + CosmosItemResponse response = executeTestCase(container); + assertThat(response.getDiagnostics()).isNotNull(); + assertThat(response.getDiagnostics().getDiagnosticsContext()).isNotNull(); + // no assertions here - invocations for diagnostics handler are validated above + // log4j event logging isn't validated in general in unit tests because it is too brittle to do so + // with custom appender + } + @Test(groups = { "fast", "emulator" }, timeOut = TIMEOUT) + public void onlyLoggerAlwaysSampledOut() { + CosmosClientBuilder builder = this + .getClientBuilder() + .clientTelemetryConfig( + new CosmosClientTelemetryConfig() + .diagnosticsThresholds( + new CosmosDiagnosticsThresholds() + .setPointOperationLatencyThreshold(Duration.ofMillis(100)) + .setNonPointOperationLatencyThreshold(Duration.ofMillis(2000)) + .setRequestChargeThreshold(100) + ) + .diagnosticsHandler(CosmosDiagnosticsHandler.DEFAULT_LOGGING_HANDLER) + .sampleDiagnostics(0) + ); + CosmosContainer container = this.getContainer(builder); + CosmosItemResponse response = executeTestCase(container); + assertThat(response.getDiagnostics()).isNotNull(); + assertThat(response.getDiagnostics().getDiagnosticsContext()).isNotNull(); // no assertions here - invocations for diagnostics handler are validated above // log4j event logging isn't validated in general in unit tests because it is too brittle to do so // with custom appender @@ -177,10 +204,46 @@ public void onlyCustomLoggerWithCustomConfig() { .diagnosticsHandler(capturingLogger) ); CosmosContainer container = this.getContainer(builder); - executeTestCase(container); + String id = UUID.randomUUID().toString(); + ObjectNode doc = getDocumentDefinition(id); + + CosmosDiagnostics diagnostics = executeDocumentOperation(container, OperationType.Create, id, doc); + assertThat(diagnostics).isNotNull(); + assertThat(diagnostics.getDiagnosticsContext()).isNotNull(); + + diagnostics = executeDocumentOperation(container, OperationType.Query, id, doc); + assertThat(diagnostics).isNotNull(); + assertThat(diagnostics.getDiagnosticsContext()).isNotNull(); assertThat(capturingLogger.getLoggedMessages()).isNotNull(); - assertThat(capturingLogger.getLoggedMessages()).hasSize(1); + assertThat(capturingLogger.getLoggedMessages()).hasSize(2); + } + + @Test(groups = { "fast", "emulator" }, timeOut = TIMEOUT) + public void onlyCustomLoggerAlwaysSampledOut() { + + CapturingLogger capturingLogger = new CapturingLogger(); + + CosmosClientBuilder builder = this + .getClientBuilder() + .clientTelemetryConfig( + new CosmosClientTelemetryConfig() + .diagnosticsThresholds( + new CosmosDiagnosticsThresholds() + .setPointOperationLatencyThreshold(Duration.ofMillis(100)) + .setNonPointOperationLatencyThreshold(Duration.ofMillis(2000)) + .setRequestChargeThreshold(100) + ) + .diagnosticsHandler(capturingLogger) + .sampleDiagnostics(0) + ); + CosmosContainer container = this.getContainer(builder); + CosmosItemResponse response = executeTestCase(container); + + assertThat(response.getDiagnostics()).isNotNull(); + assertThat(response.getDiagnostics().getDiagnosticsContext()).isNotNull(); + assertThat(capturingLogger.getLoggedMessages()).isNotNull(); + assertThat(capturingLogger.getLoggedMessages()).hasSize(0); } @Test(groups = { "fast", "emulator" }, timeOut = TIMEOUT) @@ -368,14 +431,16 @@ public void run() { assertThat(cosmosDiagnosticsNode.get("jvmFatalErrorMapperExecutionCount").asLong()).isGreaterThan(0); } - private void executeTestCase(CosmosContainer container) { + private CosmosItemResponse executeTestCase(CosmosContainer container) { String id = UUID.randomUUID().toString(); CosmosItemResponse response = container.createItem( getDocumentDefinition(id), new PartitionKey(id), - null); + new CosmosItemRequestOptions()); assertThat(response).isNotNull(); assertThat(response.getStatusCode()).isEqualTo(201); + + return response; } private ObjectNode getDocumentDefinition(String documentId) { @@ -407,35 +472,43 @@ private CosmosContainer getContainer(CosmosClientBuilder builder) { return this.client.getDatabase(asyncContainer.getDatabase().getId()).getContainer(asyncContainer.getId()); } - private void executeDocumentOperation( + private CosmosDiagnostics executeDocumentOperation( CosmosContainer cosmosContainer, OperationType operationType, String createdItemId, ObjectNode createdItem) { + + final AtomicReference diagnostics = new AtomicReference<>(null); + switch (operationType) { case Query: String query = String.format("SELECT * from c"); CosmosQueryRequestOptions queryRequestOptions = new CosmosQueryRequestOptions(); queryRequestOptions.setFeedRange(FeedRange.forLogicalPartition(new PartitionKey(createdItemId))); - Iterable> results = cosmosContainer.queryItems(query, queryRequestOptions, JsonNode.class).iterableByPage(); - results.forEach(t -> {}); + Iterable> queryResults = cosmosContainer.queryItems(query, queryRequestOptions, JsonNode.class).iterableByPage(); + queryResults.forEach(t -> diagnostics.set(t.getCosmosDiagnostics())); break; case ReadFeed: CosmosChangeFeedRequestOptions changeFeedRequestOptions = CosmosChangeFeedRequestOptions .createForProcessingFromBeginning(FeedRange.forFullRange()); - cosmosContainer.queryChangeFeed(changeFeedRequestOptions, JsonNode.class).iterableByPage(); + Iterable> changeFeedResults = + cosmosContainer.queryChangeFeed(changeFeedRequestOptions, JsonNode.class).iterableByPage(); + changeFeedResults.forEach(t -> diagnostics.set(t.getCosmosDiagnostics())); break; case Read: - cosmosContainer + CosmosItemResponse readResponse = cosmosContainer .readItem(createdItemId, new PartitionKey(createdItemId), JsonNode.class); + diagnostics.set(readResponse.getDiagnostics()); break; case Replace: - cosmosContainer + CosmosItemResponse replaceResponse = cosmosContainer .replaceItem(createdItem, createdItemId, new PartitionKey(createdItemId), new CosmosItemRequestOptions()); + diagnostics.set(replaceResponse.getDiagnostics()); break; case Delete: try { - cosmosContainer.deleteItem(getDocumentDefinition(UUID.randomUUID().toString()), new CosmosItemRequestOptions()); + CosmosItemResponse deleteResponse = cosmosContainer.deleteItem(getDocumentDefinition(UUID.randomUUID().toString()), new CosmosItemRequestOptions()); + diagnostics.set(deleteResponse.getDiagnostics()); } catch (CosmosException e) { if (!Exceptions.isNotFound(e)) { throw e; @@ -443,11 +516,14 @@ private void executeDocumentOperation( } break; case Create: - cosmosContainer.createItem(getDocumentDefinition(UUID.randomUUID().toString())); + CosmosItemResponse createResponse = cosmosContainer.createItem(getDocumentDefinition(UUID.randomUUID().toString())); + diagnostics.set(createResponse.getDiagnostics()); break; default: throw new IllegalArgumentException("The operation type is not supported"); } + + return diagnostics.get(); } private static class CapturingDiagnosticsHandler implements CosmosDiagnosticsHandler { diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosExceptionTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosExceptionTest.java index f2480f725e25d..71253d00961ee 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosExceptionTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosExceptionTest.java @@ -26,7 +26,7 @@ import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.http.HttpHeaders; import com.azure.cosmos.implementation.CosmosError; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosTracerTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosTracerTest.java index be5a7a2cc9b1a..fb0c3e8eea47a 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosTracerTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/CosmosTracerTest.java @@ -82,7 +82,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.fail; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/DocumentCollectionTests.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/DocumentCollectionTests.java index 214200a920c42..723d1f63ad5ca 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/DocumentCollectionTests.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/DocumentCollectionTests.java @@ -8,7 +8,7 @@ import com.azure.cosmos.models.PartitionKeyDefinition; import com.azure.cosmos.models.PartitionKeyDefinitionVersion; import com.azure.cosmos.models.SpatialSpec; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.assertj.core.api.Assertions; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/SerializableDocumentCollectionTests.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/SerializableDocumentCollectionTests.java index b962d668d99c8..dc5163a825702 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/SerializableDocumentCollectionTests.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/SerializableDocumentCollectionTests.java @@ -4,7 +4,7 @@ package com.azure.cosmos; import com.azure.cosmos.implementation.DocumentCollection; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import com.azure.cosmos.models.IndexingPolicy; import com.azure.cosmos.models.PartitionKeyDefinition; import com.azure.cosmos.models.PartitionKeyDefinitionVersion; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TestObject.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TestObject.java index 88fbc07bb5d7b..9ab5fd969abb9 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TestObject.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TestObject.java @@ -3,7 +3,7 @@ package com.azure.cosmos; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import java.util.List; import java.util.Objects; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TransactionalBatchTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TransactionalBatchTest.java index 742417ba170af..fd8628f9b67c0 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TransactionalBatchTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/TransactionalBatchTest.java @@ -5,7 +5,7 @@ import com.azure.cosmos.implementation.HttpConstants; import com.azure.cosmos.implementation.ISessionToken; -import com.azure.cosmos.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; import com.azure.cosmos.models.CosmosBatch; import com.azure.cosmos.models.CosmosBatchItemRequestOptions; import com.azure.cosmos.models.CosmosBatchResponse; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientConfigDiagnosticsTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientConfigDiagnosticsTest.java index c1bf24636a196..cd5b1b0ee9ad4 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientConfigDiagnosticsTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientConfigDiagnosticsTest.java @@ -12,7 +12,7 @@ import com.azure.cosmos.SessionRetryOptions; import com.azure.cosmos.SessionRetryOptionsBuilder; import com.azure.cosmos.implementation.directconnectivity.RntbdTransportClient; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import com.azure.cosmos.implementation.http.HttpClientConfig; import com.azure.cosmos.models.CosmosContainerIdentity; import com.fasterxml.jackson.core.JsonFactory; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientRetryPolicyTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientRetryPolicyTest.java index 4a9cba157c58a..e2242f9fef4ec 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientRetryPolicyTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ClientRetryPolicyTest.java @@ -6,6 +6,7 @@ import com.azure.cosmos.BridgeInternal; import com.azure.cosmos.CosmosException; import com.azure.cosmos.ThrottlingRetryOptions; +import com.azure.cosmos.implementation.directconnectivity.ChannelAcquisitionException; import io.netty.handler.timeout.ReadTimeoutException; import io.reactivex.subscribers.TestSubscriber; import org.mockito.Mockito; @@ -40,6 +41,24 @@ public static Object[][] operationProvider() { }; } + @DataProvider(name = "tcpNetworkFailureOnWriteArgProvider") + public static Object[][] tcpNetworkFailureOnWriteArgProvider() { + return new Object[][]{ + // internal exception, canUseMultipleWriteLocations, nonIdempotentWriteRetriesEnabled, shouldRetry + { new SocketException("Dummy socket exception"), false, true, false }, + { new SSLHandshakeException("test"), false, true, false }, + { new ChannelAcquisitionException("test channel acquisition failed"), false, true, false }, + + // when canUseMultipleWriteLocations + { new SocketException("Dummy socket exception"), true, false, false }, + { new SSLHandshakeException("test"), true, false, true }, + { new ChannelAcquisitionException("test channel acquisition failed"), true, false, true }, + { new SocketException("Dummy socket exception"), true, true, true }, + { new SSLHandshakeException("test"), true, true, true }, + { new ChannelAcquisitionException("test channel acquisition failed"), true, true, true } + }; + } + @Test(groups = "unit") public void networkFailureOnRead() throws Exception { ThrottlingRetryOptions throttlingRetryOptions = new ThrottlingRetryOptions(); @@ -194,8 +213,12 @@ public void networkFailureOnWrite() throws Exception { } } - @Test(groups = "unit") - public void tcpNetworkFailureOnWrite() throws Exception { + @Test(groups = "unit", dataProvider = "tcpNetworkFailureOnWriteArgProvider") + public void tcpNetworkFailureOnWrite( + Exception exception, + boolean canUseMultiWriteLocations, + boolean nonIdempotentWriteRetriesEnabled, + boolean shouldRetry) throws Exception { ThrottlingRetryOptions retryOptions = new ThrottlingRetryOptions(); GlobalEndpointManager endpointManager = Mockito.mock(GlobalEndpointManager.class); Mockito.doReturn(new URI("http://localhost")).when(endpointManager).resolveServiceEndpoint(Mockito.any(RxDocumentServiceRequest.class)); @@ -204,59 +227,51 @@ public void tcpNetworkFailureOnWrite() throws Exception { ClientRetryPolicy clientRetryPolicy = new ClientRetryPolicy(mockDiagnosticsClientContext(), endpointManager, true, retryOptions, null); //Non retribale exception for write - Exception exception = new SocketException("Dummy SocketException");; GoneException goneException = new GoneException(exception); CosmosException cosmosException = BridgeInternal.createCosmosException(null, HttpConstants.StatusCodes.SERVICE_UNAVAILABLE, goneException); RxDocumentServiceRequest dsr = RxDocumentServiceRequest.createFromName(mockDiagnosticsClientContext(), OperationType.Create, "/dbs/db/colls/col/docs/docId", ResourceType.Document); dsr.requestContext = new DocumentServiceRequestContext(); + dsr.setNonIdempotentWriteRetriesEnabled(nonIdempotentWriteRetriesEnabled); + Mockito.when(endpointManager.canUseMultipleWriteLocations(dsr)).thenReturn(canUseMultiWriteLocations); clientRetryPolicy.onBeforeSendRequest(dsr); - - for (int i = 0; i < 10; i++) { - Mono shouldRetry = clientRetryPolicy.shouldRetry(cosmosException); - // We don't want to retry writes on network failure with non retriable exception - validateSuccess(shouldRetry, ShouldRetryValidator.builder() - .nullException() - .shouldRetry(false) - .build()); - - Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForRead(Mockito.any()); - Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForWrite(Mockito.any()); - } - - // Retriable exception scenario - exception = new SSLHandshakeException("test"); - goneException = new GoneException(exception); - cosmosException = BridgeInternal.createCosmosException(null, HttpConstants.StatusCodes.SERVICE_UNAVAILABLE, goneException); - - Mockito.doReturn(true).when(endpointManager).canUseMultipleWriteLocations(Mockito.any(RxDocumentServiceRequest.class)); - clientRetryPolicy = new ClientRetryPolicy(mockDiagnosticsClientContext(), endpointManager, true, retryOptions, null); - clientRetryPolicy.onBeforeSendRequest(dsr); - - for (int i = 0; i < 10; i++) { - Mono shouldRetry = clientRetryPolicy.shouldRetry(cosmosException); - // We want to retry writes on network failure with retriable exception - if (i < 2) { - validateSuccess(shouldRetry, ShouldRetryValidator.builder() - .nullException() - .shouldRetry(true) - .backOffTime(Duration.ofMillis(0)) - .build()); - - Assert.assertTrue(clientRetryPolicy.canUsePreferredLocations()); - } else { - validateSuccess(shouldRetry, ShouldRetryValidator.builder() + if (shouldRetry) { + for (int i = 0; i < 10; i++) { + Mono retryResult = clientRetryPolicy.shouldRetry(cosmosException); + if (i < 2) { + validateSuccess(retryResult, ShouldRetryValidator.builder() + .nullException() + .shouldRetry(true) + .backOffTime(Duration.ofMillis(0)) + .build()); + + Assert.assertTrue(clientRetryPolicy.canUsePreferredLocations()); + } else { + validateSuccess(retryResult, ShouldRetryValidator.builder() + .nullException() + .shouldRetry(false) + .build()); + + Assert.assertFalse(clientRetryPolicy.canUsePreferredLocations()); + } + + Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForRead(Mockito.any()); + Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForWrite(Mockito.any()); + } + } else { + for (int i = 0; i < 10; i++) { + Mono retryResult = clientRetryPolicy.shouldRetry(cosmosException); + // We don't want to retry writes on network failure with non retriable exception + validateSuccess(retryResult, ShouldRetryValidator.builder() .nullException() .shouldRetry(false) .build()); - Assert.assertFalse(clientRetryPolicy.canUsePreferredLocations()); + Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForRead(Mockito.any()); + Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForWrite(Mockito.any()); } - - Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForRead(Mockito.any()); - Mockito.verify(endpointManager, Mockito.times(0)).markEndpointUnavailableForWrite(Mockito.any()); } } diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ImplementationBridgeHelpersTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ImplementationBridgeHelpersTest.java new file mode 100644 index 0000000000000..a8b6a301f761d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/ImplementationBridgeHelpersTest.java @@ -0,0 +1,104 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.implementation; + +import com.azure.cosmos.BridgeInternal; +import com.azure.cosmos.models.ModelBridgeInternal; +import com.azure.cosmos.util.UtilBridgeInternal; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testng.annotations.Test; + +import java.lang.reflect.Field; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +public class ImplementationBridgeHelpersTest { + + private static final Logger logger = LoggerFactory.getLogger(ImplementationBridgeHelpers.class); + + @Test(groups = { "unit" }) + public void accessorInitialization() { + + String helperClassSuffix = "Helper"; + + Class[] declaredClasses = ImplementationBridgeHelpers.class.getDeclaredClasses(); + + try { + for (Class declaredClass : declaredClasses) { + + if (declaredClass.getSimpleName().endsWith(helperClassSuffix)) { + + Field[] fields = declaredClass.getDeclaredFields(); + boolean isAccessorReset = false; + boolean isClassLoadedReset = false; + + for (Field field : fields) { + + if (field.getName().contains("accessor")) { + field.setAccessible(true); + AtomicReference value = (AtomicReference) FieldUtils.readStaticField(field); + value.set(null); + isAccessorReset = true; + } + + if (field.getName().contains("ClassLoaded")) { + field.setAccessible(true); + AtomicBoolean value = (AtomicBoolean) FieldUtils.readStaticField(field); + value.set(false); + isClassLoadedReset = true; + } + } + assertThat(isAccessorReset).isTrue(); + assertThat(isClassLoadedReset).isTrue(); + } + } + + BridgeInternal.initializeAllAccessors(); + ModelBridgeInternal.initializeAllAccessors(); + UtilBridgeInternal.initializeAllAccessors(); + + declaredClasses = ImplementationBridgeHelpers.class.getDeclaredClasses(); + + for (Class declaredClass : declaredClasses) { + + if (declaredClass.getSimpleName().endsWith(helperClassSuffix)) { + + logger.info("Helper class name : {}", declaredClass.getSimpleName()); + + Field[] fields = declaredClass.getDeclaredFields(); + boolean isAccessorSet = false; + boolean isClassLoaded = false; + + for (Field field : fields) { + + if (field.getName().contains("accessor")) { + field.setAccessible(true); + AtomicReference value = (AtomicReference) FieldUtils.readStaticField(field); + logger.info("Accessor name : {}", field.getName()); + assertThat(value.get()).isNotNull(); + isAccessorSet = true; + } + + if (field.getName().contains("ClassLoaded")) { + field.setAccessible(true); + AtomicBoolean value = (AtomicBoolean) FieldUtils.readStaticField(field); + logger.info("ClassLoaded name : {}", field.getName()); + assertThat(value.get()).isTrue(); + isClassLoaded = true; + } + } + assertThat(isAccessorSet).isTrue(); + assertThat(isClassLoaded).isTrue(); + } + } + } catch (IllegalAccessException e) { + fail("Failed with IllegalAccessException : ", e.getMessage()); + } + } +} diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/RetryCreateDocumentTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/RetryCreateDocumentTest.java index 91f28e0616ea4..f52b1480d8788 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/RetryCreateDocumentTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/RetryCreateDocumentTest.java @@ -4,7 +4,7 @@ package com.azure.cosmos.implementation; import com.azure.cosmos.BridgeInternal; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import org.mockito.Mockito; import org.mockito.stubbing.Answer; import org.testng.annotations.AfterClass; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SessionContainerTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SessionContainerTest.java index 98e7591ee9a8d..3ce73993a0b33 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SessionContainerTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SessionContainerTest.java @@ -4,8 +4,8 @@ package com.azure.cosmos.implementation; import com.azure.cosmos.GatewayTestUtils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.reflect.FieldUtils; import org.assertj.core.api.Assertions; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SqlParameterTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SqlParameterTest.java index 4d25cf8617386..d9f2692efa13b 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SqlParameterTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/SqlParameterTest.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. package com.azure.cosmos.implementation; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.models.SqlParameter; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/TestSuiteBase.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/TestSuiteBase.java index 43876eda4d790..d9a13a4d9f90c 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/TestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/TestSuiteBase.java @@ -14,8 +14,8 @@ import com.azure.cosmos.implementation.AsyncDocumentClient.Builder; import com.azure.cosmos.implementation.clienttelemetry.ClientTelemetry; import com.azure.cosmos.implementation.directconnectivity.Protocol; -import com.azure.cosmos.implementation.guava25.base.CaseFormat; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.CaseFormat; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import com.azure.cosmos.models.CosmosClientTelemetryConfig; import com.azure.cosmos.models.CompositePath; import com.azure.cosmos.models.CompositePathSortOrder; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/AsyncCacheNonBlockingTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/AsyncCacheNonBlockingTest.java index 04b8a68ab2fb6..f612a165dd582 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/AsyncCacheNonBlockingTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/AsyncCacheNonBlockingTest.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. package com.azure.cosmos.implementation.caches; -import com.azure.cosmos.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; import org.reactivestreams.Subscription; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/SerializableAsyncCacheTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/SerializableAsyncCacheTest.java index 632a9e1dca495..826a791ad6386 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/SerializableAsyncCacheTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/caches/SerializableAsyncCacheTest.java @@ -5,7 +5,7 @@ import com.azure.cosmos.implementation.DocumentCollection; import com.azure.cosmos.implementation.apachecommons.lang.RandomStringUtils; import com.azure.cosmos.implementation.directconnectivity.ReflectionUtils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import com.azure.cosmos.models.IndexingPolicy; import com.azure.cosmos.models.PartitionKeyDefinition; import com.azure.cosmos.models.PartitionKeyDefinitionVersion; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/cpu/CpuLoadHistoryTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/cpu/CpuLoadHistoryTest.java index 94c1a6f217900..4c15857acfdeb 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/cpu/CpuLoadHistoryTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/cpu/CpuLoadHistoryTest.java @@ -3,7 +3,7 @@ package com.azure.cosmos.implementation.cpu; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressResolverTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressResolverTest.java index ad8ae1693e529..c1b3fca5d3d54 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressResolverTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressResolverTest.java @@ -19,8 +19,8 @@ import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.apachecommons.lang.tuple.ImmutablePair; import com.azure.cosmos.implementation.caches.RxCollectionCache; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import com.azure.cosmos.implementation.routing.CollectionRoutingMap; import com.azure.cosmos.implementation.routing.IServerIdentity; import com.azure.cosmos.implementation.routing.InMemoryCollectionRoutingMap; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorTest.java index a53835da4bee8..f5bdf841bc036 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorTest.java @@ -6,7 +6,7 @@ import com.azure.cosmos.implementation.DiagnosticsClientContext; import com.azure.cosmos.implementation.GoneException; import com.azure.cosmos.implementation.RxDocumentServiceRequest; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.mockito.ArgumentMatchers; import org.mockito.Mockito; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorWrapper.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorWrapper.java index 47dcf06451260..e2b0932c584cd 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorWrapper.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/AddressSelectorWrapper.java @@ -5,8 +5,8 @@ import com.azure.cosmos.implementation.PartitionKeyRange; import com.azure.cosmos.implementation.RxDocumentServiceRequest; -import com.azure.cosmos.implementation.guava25.base.Predicates; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.assertj.core.api.Condition; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyReaderTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyReaderTest.java index 4e57f701c81d9..880cc32a4f660 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyReaderTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyReaderTest.java @@ -22,7 +22,7 @@ import com.azure.cosmos.implementation.StoreResponseBuilder; import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.VectorSessionToken; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import io.reactivex.subscribers.TestSubscriber; import org.mockito.Mockito; import org.testng.annotations.DataProvider; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriterTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriterTest.java index 397c2e17db388..965c191414394 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriterTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriterTest.java @@ -20,7 +20,7 @@ import com.azure.cosmos.implementation.SessionTokenHelper; import com.azure.cosmos.implementation.StoreResponseBuilder; import com.azure.cosmos.implementation.Utils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import io.reactivex.subscribers.TestSubscriber; import org.mockito.MockedStatic; import org.mockito.Mockito; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/EndpointMock.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/EndpointMock.java index 8113875998d20..861502dbad3f2 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/EndpointMock.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/EndpointMock.java @@ -6,7 +6,7 @@ import com.azure.cosmos.implementation.OperationType; import com.azure.cosmos.implementation.RxDocumentServiceRequest; import com.azure.cosmos.implementation.StoreResponseBuilder; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import java.util.ArrayList; import java.util.HashMap; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GatewayAddressCacheTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GatewayAddressCacheTest.java index a8f5dff83e248..30412dadd8253 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GatewayAddressCacheTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GatewayAddressCacheTest.java @@ -25,8 +25,8 @@ import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.directconnectivity.rntbd.OpenConnectionTask; import com.azure.cosmos.implementation.directconnectivity.rntbd.ProactiveOpenConnectionsProcessor; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.implementation.http.HttpClient; import com.azure.cosmos.implementation.http.HttpClientConfig; import com.azure.cosmos.implementation.routing.PartitionKeyRangeIdentity; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GoneAndRetryWithRetryPolicyTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GoneAndRetryWithRetryPolicyTest.java index 6773fa8e8396a..f5a49fc4f8aa0 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GoneAndRetryWithRetryPolicyTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/GoneAndRetryWithRetryPolicyTest.java @@ -17,7 +17,7 @@ import com.azure.cosmos.implementation.RetryWithException; import com.azure.cosmos.implementation.RxDocumentServiceRequest; import com.azure.cosmos.implementation.ShouldRetryResult; -import com.azure.cosmos.implementation.guava25.base.Supplier; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Supplier; import com.azure.cosmos.implementation.RetryWithException; import org.mockito.Mockito; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/MultiStoreResultValidator.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/MultiStoreResultValidator.java index 9fdafe23c33b7..4bb7755376785 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/MultiStoreResultValidator.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/MultiStoreResultValidator.java @@ -5,7 +5,7 @@ import com.azure.cosmos.CosmosException; import com.azure.cosmos.implementation.FailureValidator; -import com.azure.cosmos.implementation.guava25.base.Predicates; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Predicates; import org.apache.commons.lang3.mutable.MutableObject; import org.assertj.core.api.Assertions; import org.assertj.core.description.Description; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyInternalTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyInternalTest.java index a2ba04c389a2c..19b85c3813c46 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyInternalTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyInternalTest.java @@ -13,8 +13,8 @@ import com.azure.cosmos.implementation.routing.PartitionKeyInternal; import com.azure.cosmos.implementation.routing.PartitionKeyInternalHelper; import com.azure.cosmos.implementation.routing.PartitionKeyInternalUtils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyTest.java index 1f9ad512e4177..064aa33b01579 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/PartitionKeyTest.java @@ -10,7 +10,7 @@ import com.azure.cosmos.implementation.RMResources; import com.azure.cosmos.implementation.Undefined; import com.azure.cosmos.implementation.routing.PartitionKeyInternalHelper; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/QuorumReaderTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/QuorumReaderTest.java index 8a22146d0bf13..9c08e8d8c0370 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/QuorumReaderTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/QuorumReaderTest.java @@ -15,8 +15,8 @@ import com.azure.cosmos.implementation.ResourceType; import com.azure.cosmos.implementation.RxDocumentServiceRequest; import com.azure.cosmos.implementation.StoreResponseBuilder; -import com.azure.cosmos.implementation.guava25.base.Stopwatch; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Stopwatch; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import io.reactivex.subscribers.TestSubscriber; import org.mockito.Mockito; import org.testng.annotations.DataProvider; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/RntbdTransportClientTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/RntbdTransportClientTest.java index 03d295a8a662f..60c89d6e4a1cc 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/RntbdTransportClientTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/RntbdTransportClientTest.java @@ -59,8 +59,8 @@ import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdServiceEndpoint; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdUUID; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdUtils; -import com.azure.cosmos.implementation.guava25.base.Strings; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import io.micrometer.core.instrument.Tag; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderTest.java index ebccbd90c8b84..a43c01cdb2d45 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderTest.java @@ -26,7 +26,7 @@ import com.azure.cosmos.implementation.StoreResponseBuilder; import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.VectorSessionToken; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import io.reactivex.subscribers.TestSubscriber; import org.assertj.core.api.AssertionsForClassTypes; import org.mockito.MockedStatic; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderUnderTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderUnderTest.java index 1c197f06adcaa..a23d628c29452 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderUnderTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/StoreReaderUnderTest.java @@ -5,7 +5,7 @@ import com.azure.cosmos.implementation.ISessionContainer; import com.azure.cosmos.implementation.RxDocumentServiceRequest; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.apache.commons.lang3.tuple.Pair; import reactor.core.publisher.Mono; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdConstants.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdConstants.java index 092a551e5c416..f40163443da20 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdConstants.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdConstants.java @@ -6,9 +6,9 @@ import com.azure.cosmos.implementation.OperationType; import com.azure.cosmos.implementation.ResourceType; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdConstants; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; -import com.azure.cosmos.implementation.guava25.collect.ImmutableSet; -import com.azure.cosmos.implementation.guava25.collect.Sets; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSet; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Sets; import io.netty.handler.codec.DecoderException; import java.util.EnumSet; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdContext.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdContext.java index 0404eb60269d9..44d39d2946d17 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdContext.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdContext.java @@ -6,9 +6,9 @@ import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdContext; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdObjectMapper; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; -import com.azure.cosmos.implementation.guava25.collect.ImmutableSet; -import com.azure.cosmos.implementation.guava25.collect.Sets; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSet; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Sets; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import io.netty.buffer.ByteBuf; @@ -19,7 +19,7 @@ import java.util.UUID; import java.util.stream.Collector; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; /** * Except the constructor method, other methods are copied frm {@link RntbdContext} diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdObjectMapper.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdObjectMapper.java index 3d4a2ba36f0f0..9a218a654aad4 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdObjectMapper.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdObjectMapper.java @@ -29,7 +29,7 @@ import java.time.Instant; import java.util.concurrent.ConcurrentHashMap; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; import static com.azure.cosmos.implementation.guava27.Strings.lenientFormat; /** diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdRequest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdRequest.java index 343191622a988..a8375d69b19b1 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdRequest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdRequest.java @@ -8,7 +8,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import io.netty.buffer.ByteBuf; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; /** * Methods included in this class are copied from {@link RntbdRequest}. diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdToken.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdToken.java index 6903b50846bf0..78baca834a2aa 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdToken.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdToken.java @@ -14,9 +14,9 @@ import io.netty.buffer.ByteBuf; import io.netty.handler.codec.CorruptedFrameException; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkArgument; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; import static com.azure.cosmos.implementation.guava27.Strings.lenientFormat; /** diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenStream.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenStream.java index d77f4ee80c1d5..dddd3f0694700 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenStream.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenStream.java @@ -3,16 +3,16 @@ package com.azure.cosmos.implementation.directconnectivity.TcpServerMock.rntbd; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; -import com.azure.cosmos.implementation.guava25.collect.ImmutableSet; -import com.azure.cosmos.implementation.guava25.collect.Maps; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableSet; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Maps; import io.netty.buffer.ByteBuf; import io.netty.handler.codec.CorruptedFrameException; import io.netty.util.ReferenceCounted; import java.util.stream.Collector; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; import static com.azure.cosmos.implementation.guava27.Strings.lenientFormat; /** diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenType.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenType.java index 048934ed01d2c..e756c206d2699 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenType.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/TcpServerMock/rntbd/ServerRntbdTokenType.java @@ -4,7 +4,7 @@ package com.azure.cosmos.implementation.directconnectivity.TcpServerMock.rntbd; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdUUID; -import com.azure.cosmos.implementation.guava25.base.Utf8; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Utf8; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufUtil; import io.netty.handler.codec.CorruptedFrameException; @@ -12,7 +12,7 @@ import java.nio.charset.StandardCharsets; import java.util.UUID; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkState; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkState; import static com.azure.cosmos.implementation.guava27.Strings.lenientFormat; /** diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/WebExceptionUtilityTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/WebExceptionUtilityTest.java index 23b8e982517b5..28afd757100f9 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/WebExceptionUtilityTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/directconnectivity/WebExceptionUtilityTest.java @@ -3,6 +3,9 @@ package com.azure.cosmos.implementation.directconnectivity; +import com.azure.cosmos.BridgeInternal; +import com.azure.cosmos.implementation.GoneException; +import com.azure.cosmos.implementation.HttpConstants; import com.fasterxml.jackson.databind.JsonMappingException; import io.netty.channel.ChannelException; import io.netty.channel.ConnectTimeoutException; @@ -21,6 +24,8 @@ import java.net.NoRouteToHostException; import java.net.SocketException; import java.net.SocketTimeoutException; +import java.net.URI; +import java.net.URISyntaxException; import java.net.UnknownHostException; import java.net.UnknownServiceException; import java.nio.channels.ClosedChannelException; @@ -34,7 +39,7 @@ public class WebExceptionUtilityTest { @DataProvider(name = "exceptionToIsRetriable") - public Object[][] exceptionToIsRetriable() { + public Object[][] exceptionToIsRetriable() throws URISyntaxException { return new Object[][]{ // exception, is retriable { @@ -66,6 +71,16 @@ public Object[][] exceptionToIsRetriable() { }, { PrematureCloseException.TEST_EXCEPTION, false + }, + { + BridgeInternal.createServiceUnavailableException( + new GoneException( + "Test acquire channel failed", + new URI("https://localhost"), + new ChannelAcquisitionException("Test acquire channel failed"), + HttpConstants.SubStatusCodes.TRANSPORT_GENERATED_410), + HttpConstants.SubStatusCodes.TRANSPORT_GENERATED_410), + true } }; } diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/query/DocumentProducerTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/query/DocumentProducerTest.java index bee53a8ad1987..28569a57f0a9c 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/query/DocumentProducerTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/query/DocumentProducerTest.java @@ -20,10 +20,10 @@ import com.azure.cosmos.implementation.apachecommons.lang.StringUtils; import com.azure.cosmos.implementation.caches.RxPartitionKeyRangeCache; import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; -import com.azure.cosmos.implementation.guava25.base.Strings; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.Iterables; -import com.azure.cosmos.implementation.guava25.collect.LinkedListMultimap; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Strings; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Iterables; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.LinkedListMultimap; import com.azure.cosmos.implementation.query.orderbyquery.OrderByRowResult; import com.azure.cosmos.implementation.query.orderbyquery.OrderbyRowComparer; import com.azure.cosmos.implementation.routing.PartitionKeyRangeIdentity; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/InMemoryCollectionRoutingMapTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/InMemoryCollectionRoutingMapTest.java index 7d52ca67a806d..e429e6e61a41f 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/InMemoryCollectionRoutingMapTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/InMemoryCollectionRoutingMapTest.java @@ -5,7 +5,7 @@ import com.azure.cosmos.implementation.PartitionKeyRange; import com.azure.cosmos.implementation.apachecommons.lang.tuple.ImmutablePair; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import org.apache.commons.lang3.StringUtils; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/LocationCacheTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/LocationCacheTest.java index a1e9fd3c71234..81b5ea6012d4a 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/LocationCacheTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/LocationCacheTest.java @@ -19,8 +19,8 @@ import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.directconnectivity.ReflectionUtils; import com.azure.cosmos.models.ModelBridgeUtils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.Iterables; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Iterables; import org.testng.annotations.AfterClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/PartitionKeyInternalUtils.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/PartitionKeyInternalUtils.java index 58bc08bd5d109..4748f76484025 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/PartitionKeyInternalUtils.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/implementation/routing/PartitionKeyInternalUtils.java @@ -3,7 +3,7 @@ package com.azure.cosmos.implementation.routing; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; public class PartitionKeyInternalUtils { diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ChangeFeedTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ChangeFeedTest.java index 83c1f975c5ea0..73c9df689c81b 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ChangeFeedTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ChangeFeedTest.java @@ -17,8 +17,8 @@ import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.feedranges.FeedRangePartitionKeyImpl; import com.azure.cosmos.implementation.feedranges.FeedRangePartitionKeyRangeImpl; -import com.azure.cosmos.implementation.guava25.collect.ArrayListMultimap; -import com.azure.cosmos.implementation.guava25.collect.Multimap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ArrayListMultimap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Multimap; import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.models.ChangeFeedPolicy; import com.azure.cosmos.models.CosmosChangeFeedRequestOptions; @@ -49,7 +49,7 @@ import java.util.UUID; import java.util.stream.Collectors; -import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkNotNull; +import static com.azure.cosmos.kafka.connect.implementation.guava25.base.Preconditions.checkNotNull; import static java.lang.annotation.ElementType.METHOD; import static org.assertj.core.api.Assertions.assertThat; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ClientRetryPolicyE2ETests.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ClientRetryPolicyE2ETests.java index 884f2bd3f30c3..709fba09ff704 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ClientRetryPolicyE2ETests.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ClientRetryPolicyE2ETests.java @@ -11,12 +11,14 @@ import com.azure.cosmos.CosmosClientBuilder; import com.azure.cosmos.CosmosDiagnostics; import com.azure.cosmos.CosmosException; +import com.azure.cosmos.DirectConnectionConfig; import com.azure.cosmos.implementation.AsyncDocumentClient; import com.azure.cosmos.implementation.DatabaseAccount; import com.azure.cosmos.implementation.DatabaseAccountLocation; import com.azure.cosmos.implementation.GlobalEndpointManager; import com.azure.cosmos.implementation.HttpConstants; import com.azure.cosmos.implementation.OperationType; +import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.throughputControl.TestItem; import com.azure.cosmos.models.CosmosChangeFeedRequestOptions; import com.azure.cosmos.models.CosmosPatchOperations; @@ -32,12 +34,16 @@ import com.azure.cosmos.test.faultinjection.FaultInjectionRule; import com.azure.cosmos.test.faultinjection.FaultInjectionRuleBuilder; import com.azure.cosmos.test.faultinjection.FaultInjectionServerErrorType; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import org.testng.SkipException; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; import org.testng.annotations.DataProvider; import org.testng.annotations.Factory; import org.testng.annotations.Test; +import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; import java.time.Duration; @@ -48,6 +54,8 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; @@ -58,6 +66,15 @@ public class ClientRetryPolicyE2ETests extends TestSuiteBase { private CosmosAsyncContainer cosmosAsyncContainer; private List preferredRegions; + @DataProvider(name = "channelAcquisitionExceptionArgProvider") + public static Object[][] channelAcquisitionExceptionArgProvider() { + return new Object[][]{ + // OperationType, FaultInjectionOperationType + { OperationType.Read, FaultInjectionOperationType.READ_ITEM }, + { OperationType.Create, FaultInjectionOperationType.CREATE_ITEM }, + }; + } + @Factory(dataProvider = "clientBuildersWithSessionConsistency") public ClientRetryPolicyE2ETests(CosmosClientBuilder clientBuilder) { super(clientBuilder); @@ -242,7 +259,12 @@ public void dataPlaneRequestHttpTimeout( if (shouldRetryCrossRegion) { try { CosmosDiagnostics cosmosDiagnostics = - this.performDocumentOperation(cosmosAsyncContainer, operationType, newItem).block(); + this.performDocumentOperation( + cosmosAsyncContainer, + operationType, + newItem, + (testItem) -> new PartitionKey(testItem.getId()) + ).block(); assertThat(cosmosDiagnostics.getContactedRegionNames().size()).isEqualTo(this.preferredRegions.size()); assertThat(cosmosDiagnostics.getContactedRegionNames().containsAll(this.preferredRegions)).isTrue(); @@ -251,7 +273,12 @@ public void dataPlaneRequestHttpTimeout( } } else { try { - this.performDocumentOperation(cosmosAsyncContainer, operationType, newItem).block(); + this.performDocumentOperation( + cosmosAsyncContainer, + operationType, + newItem, + (testItem) -> new PartitionKey(testItem.getId()) + ).block(); fail("dataPlaneRequestHttpTimeout() should have failed for operationType " + operationType); } catch (CosmosException e) { System.out.println("dataPlaneRequestHttpTimeout() preferredRegions " + this.preferredRegions.toString() + " " + e.getDiagnostics()); @@ -266,10 +293,108 @@ public void dataPlaneRequestHttpTimeout( } } + @Test(groups = { "multi-master" }, dataProvider = "channelAcquisitionExceptionArgProvider", timeOut = 8 * TIMEOUT) + public void channelAcquisitionExceptionOnWrites( + OperationType operationType, + FaultInjectionOperationType faultInjectionOperationType) { + if (BridgeInternal + .getContextClient(this.clientWithPreferredRegions) + .getConnectionPolicy() + .getConnectionMode() == ConnectionMode.GATEWAY) { + throw new SkipException("channelAcquisitionExceptionOnWrites() is only meant for Direct mode"); + } + + FaultInjectionRule channelAcquisitionExceptionRule = new FaultInjectionRuleBuilder("channelAcquisitionException" + UUID.randomUUID()) + .condition( + new FaultInjectionConditionBuilder() + .operationType(faultInjectionOperationType) + .region(this.preferredRegions.get(0)) + .build()) + .result( + FaultInjectionResultBuilders.getResultBuilder(FaultInjectionServerErrorType.CONNECTION_DELAY) + .delay(Duration.ofSeconds(2)) + .build() + ) + .build(); + + DirectConnectionConfig directConnectionConfig = DirectConnectionConfig.getDefaultConfig(); + directConnectionConfig.setConnectTimeout(Duration.ofSeconds(1)); + CosmosAsyncClient testClient = getClientBuilder() + .preferredRegions(preferredRegions) + .consistencyLevel(ConsistencyLevel.SESSION) + .endpointDiscoveryEnabled(true) + .multipleWriteRegionsEnabled(true) + .directMode(directConnectionConfig) + .buildAsyncClient(); + CosmosAsyncContainer testContainer = getSharedSinglePartitionCosmosContainer(testClient); + CosmosFaultInjectionHelper.configureFaultInjectionRules(testContainer, Arrays.asList(channelAcquisitionExceptionRule)).block(); + + try { + TestItem createdItem = TestItem.createNewItem(); + testContainer.createItem(createdItem).block(); + + // using a higher concurrency to force channelAcquisitionException to happen + AtomicBoolean channelAcquisitionExceptionTriggeredRetryExists = new AtomicBoolean(false); + Flux.range(1, 10) + .flatMap(index -> + this.performDocumentOperation( + testContainer, + operationType, + createdItem, + (testItem) -> new PartitionKey(testItem.getMypk()))) + .doOnNext(diagnostics -> { + // since we have only injected connection delay error in one region, so we should only see 2 regions being contacted eventually + assertThat(diagnostics.getContactedRegionNames().size()).isEqualTo(2); + assertThat(diagnostics.getContactedRegionNames().containsAll(this.preferredRegions.subList(0, 2))).isTrue(); + + if (isChannelAcquisitionExceptionTriggeredRegionRetryExists(diagnostics.toString())) { + channelAcquisitionExceptionTriggeredRetryExists.compareAndSet(false, true); + } + }) + .doOnError(throwable -> { + if (throwable instanceof CosmosException) { + fail( + "All the requests should succeeded by retrying on another region. Diagnostics: " + + ((CosmosException)throwable).getDiagnostics()); + } + }) + .blockLast(); + + assertThat(channelAcquisitionExceptionTriggeredRetryExists.get()).isTrue(); + } finally { + channelAcquisitionExceptionRule.disable(); + + if (testClient != null) { + cleanUpContainer(testContainer); + testClient.close(); + } + } + } + + private boolean isChannelAcquisitionExceptionTriggeredRegionRetryExists(String cosmosDiagnostics) { + ObjectNode diagnosticsNode; + try { + diagnosticsNode = (ObjectNode) Utils.getSimpleObjectMapper().readTree(cosmosDiagnostics); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + JsonNode responseStatisticsList = diagnosticsNode.get("responseStatisticsList"); + assertThat(responseStatisticsList.isArray()).isTrue(); + assertThat(responseStatisticsList.size()).isGreaterThan(2); + + JsonNode lastStoreResultFromFailedRegion = responseStatisticsList.get(responseStatisticsList.size()-2).get("storeResult"); + assertThat(lastStoreResultFromFailedRegion).isNotNull(); + JsonNode exceptionMessageNode = lastStoreResultFromFailedRegion.get("exceptionMessage"); + assertThat(exceptionMessageNode).isNotNull(); + + return exceptionMessageNode.asText().contains("ChannelAcquisitionException"); + } + private Mono performDocumentOperation( CosmosAsyncContainer cosmosAsyncContainer, OperationType operationType, - TestItem createdItem) { + TestItem createdItem, + Function extractPartitionKeyFunc) { if (operationType == OperationType.Query) { CosmosQueryRequestOptions queryRequestOptions = new CosmosQueryRequestOptions(); String query = String.format("SELECT * from c where c.id = '%s'", createdItem.getId()); @@ -289,7 +414,7 @@ private Mono performDocumentOperation( return cosmosAsyncContainer .readItem( createdItem.getId(), - new PartitionKey(createdItem.getId()), + extractPartitionKeyFunc.apply(createdItem), TestItem.class ) .map(itemResponse -> itemResponse.getDiagnostics()); @@ -300,7 +425,7 @@ private Mono performDocumentOperation( .replaceItem( createdItem, createdItem.getId(), - new PartitionKey(createdItem.getId())) + extractPartitionKeyFunc.apply(createdItem)) .map(itemResponse -> itemResponse.getDiagnostics()); } @@ -322,7 +447,11 @@ private Mono performDocumentOperation( .create() .add("newPath", "newPath"); return cosmosAsyncContainer - .patchItem(createdItem.getId(), new PartitionKey(createdItem.getId()), patchOperations, TestItem.class) + .patchItem( + createdItem.getId(), + extractPartitionKeyFunc.apply(createdItem), + patchOperations, + TestItem.class) .map(itemResponse -> itemResponse.getDiagnostics()); } } diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/DistinctQueryTests.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/DistinctQueryTests.java index eda2314fd4127..40195fa4f6e76 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/DistinctQueryTests.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/DistinctQueryTests.java @@ -11,7 +11,7 @@ import com.azure.cosmos.implementation.FeedResponseValidator; import com.azure.cosmos.implementation.InternalObjectNode; import com.azure.cosmos.implementation.Utils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableMap; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableMap; import com.azure.cosmos.implementation.query.UnorderedDistinctMap; import com.azure.cosmos.implementation.routing.UInt128; import com.azure.cosmos.models.CosmosQueryRequestOptions; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ParallelDocumentQueryTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ParallelDocumentQueryTest.java index 3a6f824f6654c..999d43d77376f 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ParallelDocumentQueryTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/ParallelDocumentQueryTest.java @@ -20,7 +20,7 @@ import com.azure.cosmos.implementation.TestUtils; import com.azure.cosmos.implementation.Utils.ValueHolder; import com.azure.cosmos.implementation.apachecommons.lang.tuple.Pair; -import com.azure.cosmos.implementation.guava25.base.Function; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.Function; import com.azure.cosmos.implementation.guava27.Strings; import com.azure.cosmos.implementation.query.CompositeContinuationToken; import com.azure.cosmos.implementation.routing.Range; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/SinglePartitionDocumentQueryTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/SinglePartitionDocumentQueryTest.java index 432407f8e43ef..c98f3715d5a4a 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/SinglePartitionDocumentQueryTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/SinglePartitionDocumentQueryTest.java @@ -10,7 +10,7 @@ import com.azure.cosmos.implementation.RxDocumentClientImpl; import com.azure.cosmos.implementation.RxStoreModel; import com.azure.cosmos.implementation.directconnectivity.ReflectionUtils; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.models.ModelBridgeInternal; import com.azure.cosmos.models.PartitionKey; import com.azure.cosmos.util.CosmosPagedFlux; diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/TestSuiteBase.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/TestSuiteBase.java index 44353a55feadc..7ee0469636275 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/TestSuiteBase.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/TestSuiteBase.java @@ -33,8 +33,8 @@ import com.azure.cosmos.implementation.TestConfigurations; import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.directconnectivity.Protocol; -import com.azure.cosmos.implementation.guava25.base.CaseFormat; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.base.CaseFormat; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; import com.azure.cosmos.models.ChangeFeedPolicy; import com.azure.cosmos.models.CompositePath; import com.azure.cosmos.models.CompositePathSortOrder; @@ -229,6 +229,44 @@ public static void afterSuite() { } } + protected static void cleanUpContainer(CosmosAsyncContainer cosmosContainer) { + CosmosContainerProperties cosmosContainerProperties = cosmosContainer.read().block().getProperties(); + String cosmosContainerId = cosmosContainerProperties.getId(); + logger.info("Truncating collection {} ...", cosmosContainerId); + List paths = cosmosContainerProperties.getPartitionKeyDefinition().getPaths(); + CosmosQueryRequestOptions options = new CosmosQueryRequestOptions(); + options.setCosmosEndToEndOperationLatencyPolicyConfig( + new CosmosEndToEndOperationLatencyPolicyConfigBuilder(Duration.ofHours(1)) + .build() + ); + options.setMaxDegreeOfParallelism(-1); + int maxItemCount = 100; + + cosmosContainer.queryItems("SELECT * FROM root", options, InternalObjectNode.class) + .byPage(maxItemCount) + .publishOn(Schedulers.parallel()) + .flatMap(page -> Flux.fromIterable(page.getResults())) + .flatMap(doc -> { + + PartitionKey partitionKey = null; + + Object propertyValue = null; + if (paths != null && !paths.isEmpty()) { + List pkPath = PathParser.getPathParts(paths.get(0)); + propertyValue = ModelBridgeInternal.getObjectByPathFromJsonSerializable(doc, pkPath); + if (propertyValue == null) { + partitionKey = PartitionKey.NONE; + } else { + partitionKey = new PartitionKey(propertyValue); + } + } else { + partitionKey = new PartitionKey(null); + } + + return cosmosContainer.deleteItem(doc.getId(), partitionKey); + }).then().block(); + } + protected static void truncateCollection(CosmosAsyncContainer cosmosContainer) { CosmosContainerProperties cosmosContainerProperties = cosmosContainer.read().block().getProperties(); String cosmosContainerId = cosmosContainerProperties.getId(); diff --git a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/UniqueIndexTest.java b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/UniqueIndexTest.java index 25f801b578448..3e711079e59fc 100644 --- a/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/UniqueIndexTest.java +++ b/sdk/cosmos/azure-cosmos-tests/src/test/java/com/azure/cosmos/rx/UniqueIndexTest.java @@ -15,8 +15,8 @@ import com.azure.cosmos.implementation.HttpConstants; import com.azure.cosmos.implementation.TestConfigurations; import com.azure.cosmos.implementation.Utils; -import com.azure.cosmos.implementation.guava25.collect.ImmutableList; -import com.azure.cosmos.implementation.guava25.collect.Lists; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.ImmutableList; +import com.azure.cosmos.kafka.connect.implementation.guava25.collect.Lists; import com.azure.cosmos.models.CosmosContainerProperties; import com.azure.cosmos.models.CosmosItemRequestOptions; import com.azure.cosmos.models.ExcludedPath; diff --git a/sdk/cosmos/azure-cosmos/CHANGELOG.md b/sdk/cosmos/azure-cosmos/CHANGELOG.md index b8fe36014940e..d29e4028989dc 100644 --- a/sdk/cosmos/azure-cosmos/CHANGELOG.md +++ b/sdk/cosmos/azure-cosmos/CHANGELOG.md @@ -1,24 +1,61 @@ ## Release History -### 4.57.0-beta.1 (Unreleased) +### 4.58.0-beta.1 (Unreleased) #### Features Added #### Breaking Changes #### Bugs Fixed -* Suppress exceptions when calling diagnostics handlers. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) #### Other Changes -* Only call System.exit in `DiagnosticsProvider` for `Error` scenario. Also add `System.err` for `Error` cases. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) + +### 4.57.0 (2024-03-25) + +#### Features Added +* Added public APIs `setMaxMicroBatchSize` and `getMaxMicroBatchSize` in `CosmosBulkExecutionOptions` - See [PR 39335](https://github.com/Azure/azure-sdk-for-java/pull/39335) + +#### Bugs Fixed +* Suppressed exceptions when calling diagnostics handlers. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) +* Fixed an issue where no cross region retry for write operations due to channel acquisition timeout. - See [PR 39255](https://github.com/Azure/azure-sdk-for-java/pull/39255) +* Fixed incorrect container tag value in metrics. - See [PR 39322](https://github.com/Azure/azure-sdk-for-java/pull/39322) +* Fixed issue where CosmosDiagnosticsContext is null when diagnostics are sampled out. - See [PR 39352](https://github.com/Azure/azure-sdk-for-java/pull/39352) +#### Other Changes +* Changed logic to only call `System.exit()` in `DiagnosticsProvider` for `Error` scenario. Also added `System.err` for `Error` cases. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) +* Removed `System.exit()` calls from `ImplementationBridgeHelpers`. - See [PR 39387](https://github.com/Azure/azure-sdk-for-java/pull/39387) + +### 4.53.4-hotfix (2024-03-15) + +#### Other Changes +* Removed `System.exit()` calls from `ImplementationBridgeHelpers`. - See [PR 39215](https://github.com/Azure/azure-sdk-for-java/pull/39215) + +### 4.48.3-hotfix (2024-03-15) + +#### Bugs Fixed +* Fixed an issue where `sampleDiagnostics` is not being honored for `query. See [PR 37015](https://github.com/Azure/azure-sdk-for-java/pull/37015) +* Suppressed exceptions when calling diagnostics handlers. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) + +### Other Changes +* Changed logic to only call `System.exit()` in `DiagnosticsProvider` for `Error` scenario. Also added `System.err` for `Error` cases. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) +* Removed `System.exit()` calls from `ImplementationBridgeHelpers`. - See [PR 39182](https://github.com/Azure/azure-sdk-for-java/pull/39182) + +### 4.45.3-hotfix (2024-03-15) + +#### Bugs Fixed +* Fixed an issue where `sampleDiagnostics` is not being honored for `query. See [PR 37015](https://github.com/Azure/azure-sdk-for-java/pull/37015) +* Suppressed exceptions when calling diagnostics handlers. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) + +### Other Changes +* Changed logic to only call `System.exit()` in `DiagnosticsProvider` for `Error` scenario. Also added `System.err` for `Error` cases. - See [PR 39077](https://github.com/Azure/azure-sdk-for-java/pull/39077) +* Removed `System.exit()` calls from `ImplementationBridgeHelpers`. - See [PR 39184](https://github.com/Azure/azure-sdk-for-java/pull/39184) ### 4.53.3-hotfix (2024-03-07) #### Bugs Fixed -* Suppress exceptions when calling diagnostics handlers. - See [PR 39121](https://github.com/Azure/azure-sdk-for-java/pull/39121) +* Suppressed exceptions when calling diagnostics handlers. - See [PR 39121](https://github.com/Azure/azure-sdk-for-java/pull/39121) #### Other Changes -* Only call System.exit in DiagnosticsProvider for Error scenario. Also add System.err for Error cases. - - See [PR 39121](https://github.com/Azure/azure-sdk-for-java/pull/39121) +* Changed logic to only call `System.exit()` in `DiagnosticsProvider` for `Error` scenario. Also added `System.err` for `Error` cases. - See [PR 39121](https://github.com/Azure/azure-sdk-for-java/pull/39121) ### 4.56.0 (2024-02-20) diff --git a/sdk/cosmos/azure-cosmos/README.md b/sdk/cosmos/azure-cosmos/README.md index 3f787719cd289..676002aa394b7 100644 --- a/sdk/cosmos/azure-cosmos/README.md +++ b/sdk/cosmos/azure-cosmos/README.md @@ -40,15 +40,13 @@ and then include the direct dependency in the dependencies section without the v #### Include direct dependency If you want to take dependency on a particular version of the library that is not present in the BOM, add the direct dependency to your project as follows. -[//]: # ({x-version-update-start;com.azure:azure-cosmos;current}) ```xml com.azure azure-cosmos - 4.56.0 + LATEST ``` -[//]: # ({x-version-update-end}) Refer to maven central for previous [releases][cosmos_maven] diff --git a/sdk/cosmos/azure-cosmos/pom.xml b/sdk/cosmos/azure-cosmos/pom.xml index e55f779879d62..9e4ff9ecdff27 100644 --- a/sdk/cosmos/azure-cosmos/pom.xml +++ b/sdk/cosmos/azure-cosmos/pom.xml @@ -13,7 +13,7 @@ Licensed under the MIT License. com.azure azure-cosmos - 4.57.0-beta.1 + 4.58.0-beta.1 Microsoft Azure SDK for SQL API of Azure Cosmos DB Service This Package contains Microsoft Azure Cosmos SDK (with Reactive Extension Reactor support) for Azure Cosmos DB SQL API jar diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java index bd7a295bc9aeb..e068e9b8066e5 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosAsyncContainer.java @@ -18,6 +18,7 @@ import com.azure.cosmos.implementation.Offer; import com.azure.cosmos.implementation.OperationType; import com.azure.cosmos.implementation.PartitionKeyHelper; +import com.azure.cosmos.implementation.PartitionKeyRange; import com.azure.cosmos.implementation.Paths; import com.azure.cosmos.implementation.QueryFeedOperationState; import com.azure.cosmos.implementation.RequestOptions; @@ -71,6 +72,7 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -79,6 +81,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.stream.Collectors; import static com.azure.core.util.FluxUtil.withContext; import static com.azure.cosmos.implementation.guava25.base.Preconditions.checkArgument; @@ -2549,6 +2552,36 @@ public Mono> getFeedRanges() { return this.getFeedRanges(true); } + Mono> getOverlappingFeedRanges(FeedRange feedRange) { + checkNotNull(feedRange, "Argument 'feedRange' must not be null."); + + final AsyncDocumentClient clientWrapper = this.database.getDocClientWrapper(); + + return this.getNormalizedEffectiveRange(feedRange) + .flatMap(normalizedRange -> { + return clientWrapper + .getCollectionCache() + .resolveByNameAsync(null, this.getLinkWithoutTrailingSlash(), null) + .flatMap(collection -> { + return clientWrapper + .getPartitionKeyRangeCache() + .tryGetOverlappingRangesAsync( + null, + collection.getResourceId(), + normalizedRange, + false, + null + ); + }); + }) + .map(pkRangesValueHolder -> { + List matchedPkRanges = + (pkRangesValueHolder == null || pkRangesValueHolder.v == null) ? new ArrayList<>() : pkRangesValueHolder.v; + + return matchedPkRanges.stream().map(pkRange -> new FeedRangeEpkImpl(pkRange.toRange())).collect(Collectors.toList()); + }); + } + Mono> getFeedRanges(boolean forceRefresh) { return this.getDatabase().getDocClientWrapper().getFeedRanges(getLink(), forceRefresh); } @@ -2597,6 +2630,31 @@ Mono> getNormalizedEffectiveRange(FeedRange feedRange) { getCollectionObservable); } + Mono checkFeedRangeOverlapping(FeedRange feedRange1, FeedRange feedRange2) { + checkNotNull(feedRange1, "Argument 'feedRange1' must not be null."); + checkNotNull(feedRange2, "Argument 'feedRange2' must not be null."); + + return this.getNormalizedEffectiveRange(feedRange1) + .flatMap(normalizedRange1 -> { + return this.getNormalizedEffectiveRange(feedRange2) + .map(normalizedRange2 -> Range.checkOverlapping(normalizedRange1, normalizedRange2)); + }); + } + + Mono getPartitionKeyDefinition() { + final AsyncDocumentClient clientWrapper = this.database.getDocClientWrapper(); + return Mono.just(clientWrapper.getCollectionCache()) + .flatMap(collectionCache -> { + return collectionCache + .resolveByNameAsync( + null, + this.getLinkWithoutTrailingSlash(), + null, + null) + .map(documentCollection -> documentCollection.getPartitionKey()); + }); + } + /** * Enable the throughput control group with local control mode. *
@@ -2777,12 +2835,31 @@ public Mono> getFeedRanges(CosmosAsyncContainer cosmosAsyncConta } @Override - public Mono> trySplitFeedRange( + public Mono> trySplitFeedRange( CosmosAsyncContainer cosmosAsyncContainer, FeedRange feedRange, int targetedCountAfterSplit) { - return cosmosAsyncContainer.trySplitFeedRange(feedRange, targetedCountAfterSplit); + return cosmosAsyncContainer.trySplitFeedRange(feedRange, targetedCountAfterSplit) + .map(feedRangeEpks -> feedRangeEpks.stream().collect(Collectors.toList())); + } + + @Override + public Mono checkFeedRangeOverlapping( + CosmosAsyncContainer cosmosAsyncContainer, + FeedRange feedRange1, + FeedRange feedRange2) { + return cosmosAsyncContainer.checkFeedRangeOverlapping(feedRange1, feedRange2); + } + + @Override + public Mono> getOverlappingFeedRanges(CosmosAsyncContainer container, FeedRange feedRange) { + return container.getOverlappingFeedRanges(feedRange); + } + + @Override + public Mono getPartitionKeyDefinition(CosmosAsyncContainer container) { + return container.getPartitionKeyDefinition(); } @Override diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosDiagnosticsContext.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosDiagnosticsContext.java index 1c25f31321363..53b8193ced586 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosDiagnosticsContext.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/CosmosDiagnosticsContext.java @@ -79,6 +79,8 @@ public final class CosmosDiagnosticsContext { private Double samplingRateSnapshot; + private boolean isSampledOut; + private ArrayList requestInfo = null; private final Integer sequenceNumber; @@ -128,6 +130,7 @@ public final class CosmosDiagnosticsContext { this.userAgent = userAgent; this.connectionMode = connectionMode; this.sequenceNumber = sequenceNumber; + this.isSampledOut = false; } /** @@ -556,15 +559,20 @@ void recordOperation(int statusCode, } } - void setSamplingRateSnapshot(double samplingRate) { + void setSamplingRateSnapshot(double samplingRate, boolean isSampledOut) { synchronized (this.spanName) { this.samplingRateSnapshot = samplingRate; + this.isSampledOut = isSampledOut; for (CosmosDiagnostics d : this.diagnostics) { diagAccessor.setSamplingRateSnapshot(d, samplingRate); } } } + boolean isSampledOut() { + return this.isSampledOut; + } + String getRequestDiagnostics() { ObjectNode ctxNode = mapper.createObjectNode(); @@ -1014,9 +1022,9 @@ public String getSpanName(CosmosDiagnosticsContext ctx) { } @Override - public void setSamplingRateSnapshot(CosmosDiagnosticsContext ctx, double samplingRate) { + public void setSamplingRateSnapshot(CosmosDiagnosticsContext ctx, double samplingRate, boolean isSampledOut) { checkNotNull(ctx, "Argument 'ctx' must not be null."); - ctx.setSamplingRateSnapshot(samplingRate); + ctx.setSamplingRateSnapshot(samplingRate, isSampledOut); } @Override diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClearingSessionContainerClientRetryPolicy.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClearingSessionContainerClientRetryPolicy.java index 2b0f8616e9736..add7272930d63 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClearingSessionContainerClientRetryPolicy.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClearingSessionContainerClientRetryPolicy.java @@ -62,7 +62,7 @@ public Mono shouldRetry(Exception e) { Exceptions.isSubStatusCode(clientException, HttpConstants.SubStatusCodes.READ_SESSION_NOT_AVAILABLE)) { // Clear the session token, because the collection name might be reused. - logger.warn("Clear the token for named base request {}", request.getResourceAddress()); + logger.info("Clear the token for named base request {}", request.getResourceAddress()); this.sessionContainer.clearTokenByCollectionFullName(request.getResourceAddress()); diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClientRetryPolicy.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClientRetryPolicy.java index cd540f2fe5804..6e339db58c886 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClientRetryPolicy.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ClientRetryPolicy.java @@ -99,7 +99,7 @@ public Mono shouldRetry(Exception e) { Exceptions.isStatusCode(clientException, HttpConstants.StatusCodes.FORBIDDEN) && Exceptions.isSubStatusCode(clientException, HttpConstants.SubStatusCodes.FORBIDDEN_WRITEFORBIDDEN)) { - logger.warn("Endpoint not writable. Will refresh cache and retry ", e); + logger.info("Endpoint not writable. Will refresh cache and retry ", e); return this.shouldRetryOnEndpointFailureAsync(false, true, false); } @@ -109,7 +109,7 @@ public Mono shouldRetry(Exception e) { Exceptions.isSubStatusCode(clientException, HttpConstants.SubStatusCodes.DATABASE_ACCOUNT_NOTFOUND) && this.isReadRequest) { - logger.warn("Endpoint not available for reads. Will refresh cache and retry. ", e); + logger.info("Endpoint not available for reads. Will refresh cache and retry. ", e); return this.shouldRetryOnEndpointFailureAsync(true, false, false); } @@ -117,7 +117,7 @@ public Mono shouldRetry(Exception e) { if (WebExceptionUtility.isNetworkFailure(e)) { if (clientException != null && Exceptions.isSubStatusCode(clientException, HttpConstants.SubStatusCodes.GATEWAY_ENDPOINT_UNAVAILABLE)) { if (this.isReadRequest || WebExceptionUtility.isWebExceptionRetriable(e)) { - logger.warn("Gateway endpoint not reachable. Will refresh cache and retry. ", e); + logger.info("Gateway endpoint not reachable. Will refresh cache and retry. ", e); return this.shouldRetryOnEndpointFailureAsync(this.isReadRequest, false, true); } else { return this.shouldNotRetryOnEndpointFailureAsync(this.isReadRequest, false, false); @@ -147,7 +147,7 @@ public Mono shouldRetry(Exception e) { Exceptions.isStatusCode(clientException, HttpConstants.StatusCodes.SERVICE_UNAVAILABLE)) { boolean isWebExceptionRetriable = WebExceptionUtility.isWebExceptionRetriable(e); - logger.warn( + logger.info( "Service unavailable - IsReadRequest {}, IsWebExceptionRetriable {}, NonIdempotentWriteRetriesEnabled {}", this.isReadRequest, isWebExceptionRetriable, @@ -359,7 +359,7 @@ private Mono shouldRetryOnBackendServiceUnavailableAsync( return Mono.just(ShouldRetryResult.noRetry()); } - logger.warn("shouldRetryOnServiceUnavailable() Retrying. Received on endpoint {}, IsReadRequest = {}", this.locationEndpoint, isReadRequest); + logger.info("shouldRetryOnServiceUnavailable() Retrying. Received on endpoint {}, IsReadRequest = {}", this.locationEndpoint, isReadRequest); // Retrying on second PreferredLocations // RetryCount is used as zero-based index diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/CosmosPagedFluxOptions.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/CosmosPagedFluxOptions.java index ed2b848c0994a..1d4c11ed06330 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/CosmosPagedFluxOptions.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/CosmosPagedFluxOptions.java @@ -102,9 +102,9 @@ public double getSamplingRateSnapshot() { return samplingRateSnapshot; } - public void setSamplingRateSnapshot(double samplingRateSnapshot) { + public void setSamplingRateSnapshot(double samplingRateSnapshot, boolean isSampledOut) { if (this.operationState != null) { - this.operationState.setSamplingRateSnapshot(samplingRateSnapshot); + this.operationState.setSamplingRateSnapshot(samplingRateSnapshot, isSampledOut); } } } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/DiagnosticsProvider.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/DiagnosticsProvider.java index fcaf1324d92f3..7a3473f174886 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/DiagnosticsProvider.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/DiagnosticsProvider.java @@ -215,7 +215,8 @@ public static reactor.util.context.Context setContextInReactor(Context traceCont public Context startSpan( String spanName, CosmosDiagnosticsContext cosmosCtx, - Context context) { + Context context, + boolean isSampledOut) { checkNotNull(spanName, "Argument 'spanName' must not be null."); checkNotNull(cosmosCtx, "Argument 'cosmosCtx' must not be null."); @@ -225,7 +226,7 @@ public Context startSpan( .requireNonNull(context, "'context' cannot be null.") .addData(COSMOS_DIAGNOSTICS_CONTEXT_KEY, cosmosCtx); - if (this.cosmosTracer == null) { + if (this.cosmosTracer == null || isSampledOut) { return local; } @@ -245,11 +246,12 @@ public void endSpan( int statusCode, Integer actualItemCount, Double requestCharge, - CosmosDiagnostics diagnostics + CosmosDiagnostics diagnostics, + boolean isSampledOut ) { // called in PagedFlux - needs to be exception less - otherwise will result in hanging Flux. try { - this.endSpanCore(signal, cosmosCtx, statusCode, actualItemCount, requestCharge, diagnostics); + this.endSpanCore(signal, cosmosCtx, statusCode, actualItemCount, requestCharge, diagnostics, isSampledOut); } catch (Throwable error) { this.handleErrors(error, 9901); } @@ -261,7 +263,8 @@ private void endSpanCore( int statusCode, Integer actualItemCount, Double requestCharge, - CosmosDiagnostics diagnostics + CosmosDiagnostics diagnostics, + boolean isSampledOut ) { Objects.requireNonNull(signal, "'signal' cannot be null."); @@ -281,7 +284,8 @@ private void endSpanCore( diagnostics, null, context, - ctxAccessor.isEmptyCompletion(cosmosCtx)); + ctxAccessor.isEmptyCompletion(cosmosCtx), + isSampledOut); break; case ON_NEXT: end( @@ -293,7 +297,8 @@ private void endSpanCore( diagnostics, null, context, - false); + false, + isSampledOut); break; case ON_ERROR: Throwable throwable = null; @@ -328,7 +333,8 @@ private void endSpanCore( effectiveDiagnostics, throwable, context, - false); + false, + isSampledOut); break; default: // ON_SUBSCRIBE isn't the right state to end span @@ -336,7 +342,7 @@ private void endSpanCore( } } - public void endSpan(CosmosDiagnosticsContext cosmosCtx, Context context, Throwable throwable) { + public void endSpan(CosmosDiagnosticsContext cosmosCtx, Context context, Throwable throwable, boolean isSampledOut) { // called in PagedFlux - needs to be exception less - otherwise will result in hanging Flux. try { int statusCode = DiagnosticsProvider.ERROR_CODE; @@ -362,13 +368,14 @@ public void endSpan(CosmosDiagnosticsContext cosmosCtx, Context context, Throwab effectiveDiagnostics, throwable, context, - false); + false, + isSampledOut); } catch (Throwable error) { this.handleErrors(error, 9905); } } - public void endSpan(CosmosDiagnosticsContext cosmosCtx, Context context, boolean isForcedEmptyCompletion) { + public void endSpan(CosmosDiagnosticsContext cosmosCtx, Context context, boolean isForcedEmptyCompletion, boolean isSampledOut) { // called in PagedFlux - needs to be exception less - otherwise will result in hanging Flux. try { end( @@ -380,7 +387,8 @@ public void endSpan(CosmosDiagnosticsContext cosmosCtx, Context context, boolean null, null, context, - isForcedEmptyCompletion); + isForcedEmptyCompletion, + isSampledOut); } catch (Throwable error) { this.handleErrors(error, 9904); } @@ -638,8 +646,9 @@ public Flux runUnderSpanInContext(Flux publisher) { public boolean shouldSampleOutOperation(CosmosPagedFluxOptions options) { final double samplingRateSnapshot = clientTelemetryConfigAccessor.getSamplingRate(this.telemetryConfig); - options.setSamplingRateSnapshot(samplingRateSnapshot); - return shouldSampleOutOperation(samplingRateSnapshot); + boolean result = shouldSampleOutOperation(samplingRateSnapshot); + options.setSamplingRateSnapshot(samplingRateSnapshot, result); + return result; } private boolean shouldSampleOutOperation(double samplingRate) { @@ -664,18 +673,10 @@ private Mono diagnosticsEnabledPublisher( Function requestChargeFunc, BiFunction diagnosticsFunc ) { - - if (!isEnabled()) { - return resultPublisher; - } - - final double samplingRateSnapshot = clientTelemetryConfigAccessor.getSamplingRate(this.telemetryConfig); + final double samplingRateSnapshot = isEnabled() ? clientTelemetryConfigAccessor.getSamplingRate(this.telemetryConfig) : 0; + final boolean isSampledOut = this.shouldSampleOutOperation(samplingRateSnapshot); if (cosmosCtx != null) { - ctxAccessor.setSamplingRateSnapshot(cosmosCtx, samplingRateSnapshot); - } - - if (shouldSampleOutOperation(samplingRateSnapshot)) { - return resultPublisher; + ctxAccessor.setSamplingRateSnapshot(cosmosCtx, samplingRateSnapshot, isSampledOut); } Optional callDepth = context.getData(COSMOS_CALL_DEPTH); @@ -700,7 +701,8 @@ private Mono diagnosticsEnabledPublisher( statusCodeFunc.apply(response), actualItemCountFunc.apply(response), requestChargeFunc.apply(response), - diagnosticsFunc.apply(response, samplingRateSnapshot)); + diagnosticsFunc.apply(response, samplingRateSnapshot), + isSampledOut); break; case ON_ERROR: // not adding diagnostics on trace event for exception as this information is already there as @@ -711,12 +713,13 @@ private Mono diagnosticsEnabledPublisher( ERROR_CODE, null, null, - null); + null, + isSampledOut); break; default: break; }}) - .contextWrite(setContextInReactor(this.startSpan(spanName, cosmosCtx, context))); + .contextWrite(setContextInReactor(this.startSpan(spanName, cosmosCtx, context, isSampledOut))); } private Mono publisherWithDiagnostics(Mono resultPublisher, @@ -782,11 +785,12 @@ private void end( CosmosDiagnostics diagnostics, Throwable throwable, Context context, - boolean isForcedEmptyCompletion) { + boolean isForcedEmptyCompletion, + boolean isSampledOut) { checkNotNull(cosmosCtx, "Argument 'cosmosCtx' must not be null."); - // endOperation can be called form two places in Reactor - making sure we process completion only once + // endOperation can be called from two places in Reactor - making sure we process completion only once if (ctxAccessor.endOperation( cosmosCtx, statusCode, @@ -796,12 +800,14 @@ private void end( diagnostics, throwable)) { - if (!isForcedEmptyCompletion) { - this.handleDiagnostics(context, cosmosCtx); - } + if (!isSampledOut) { + if (!isForcedEmptyCompletion) { + this.handleDiagnostics(context, cosmosCtx); + } - if (this.cosmosTracer != null) { - this.cosmosTracer.endSpan(cosmosCtx, context, isForcedEmptyCompletion); + if (this.cosmosTracer != null) { + this.cosmosTracer.endSpan(cosmosCtx, context, isForcedEmptyCompletion); + } } } } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/FeedOperationState.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/FeedOperationState.java index dedae40c9d767..71fe10771329a 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/FeedOperationState.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/FeedOperationState.java @@ -9,6 +9,7 @@ import com.azure.cosmos.CosmosDiagnosticsContext; import com.azure.cosmos.CosmosDiagnosticsThresholds; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -38,6 +39,7 @@ public abstract class FeedOperationState { private final AtomicReference maxItemCount; private final AtomicInteger sequenceNumberGenerator; private final AtomicReference samplingRate; + private final AtomicBoolean isSampledOut; private final CosmosPagedFluxOptions fluxOptions; public FeedOperationState( @@ -71,6 +73,7 @@ public FeedOperationState( this.sequenceNumberGenerator = new AtomicInteger(0); this.fluxOptions = fluxOptions; this.samplingRate = new AtomicReference<>(null); + this.isSampledOut = new AtomicBoolean(false); CosmosDiagnosticsContext cosmosCtx = ctxAccessor.create( checkNotNull(spanName, "Argument 'spanName' must not be null." ), @@ -100,10 +103,11 @@ public Double getSamplingRateSnapshot() { return this.samplingRate.get(); } - public void setSamplingRateSnapshot(double samplingRateSnapshot) { + public void setSamplingRateSnapshot(double samplingRateSnapshot, boolean isSampledOut) { this.samplingRate.set(samplingRateSnapshot); + this.isSampledOut.set(isSampledOut); CosmosDiagnosticsContext ctxSnapshot = this.ctxHolder.get(); - ctxAccessor.setSamplingRateSnapshot(ctxSnapshot, samplingRateSnapshot); + ctxAccessor.setSamplingRateSnapshot(ctxSnapshot, samplingRateSnapshot, isSampledOut); } // Can return null @@ -167,7 +171,7 @@ public void resetDiagnosticsContext() { ); Double samplingRateSnapshot = this.samplingRate.get(); if (samplingRateSnapshot != null) { - ctxAccessor.setSamplingRateSnapshot(cosmosCtx, samplingRateSnapshot); + ctxAccessor.setSamplingRateSnapshot(cosmosCtx, samplingRateSnapshot, this.isSampledOut.get()); } this.ctxHolder.set(cosmosCtx); diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/GlobalEndpointManager.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/GlobalEndpointManager.java index 8c180e4fe89de..7dc27b329c8cd 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/GlobalEndpointManager.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/GlobalEndpointManager.java @@ -293,7 +293,7 @@ private Mono startRefreshLocationTimerAsync(boolean initialization) { .flatMap( t -> { if (this.isClosed) { - logger.warn("client already closed"); + logger.info("client already closed"); // if client is already closed, nothing to be done, just return. return Mono.empty(); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java index 585b05d3b1ba5..193a5d663091e 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/ImplementationBridgeHelpers.java @@ -40,6 +40,7 @@ import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.patch.PatchOperation; import com.azure.cosmos.implementation.routing.PartitionKeyInternal; +import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.implementation.spark.OperationContextAndListenerTuple; import com.azure.cosmos.models.CosmosBatch; import com.azure.cosmos.models.CosmosBatchOperationResult; @@ -64,6 +65,7 @@ import com.azure.cosmos.models.FeedResponse; import com.azure.cosmos.models.ModelBridgeInternal; import com.azure.cosmos.models.PartitionKey; +import com.azure.cosmos.models.PartitionKeyDefinition; import com.azure.cosmos.models.PriorityLevel; import com.azure.cosmos.models.SqlQuerySpec; import com.azure.cosmos.util.CosmosPagedFlux; @@ -121,7 +123,6 @@ public static CosmosClientBuilderAccessor getCosmosClientBuilderAccessor() { CosmosClientBuilderAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosClientBuilderAccessor is not initialized yet!"); - System.exit(9700); // Using a unique status code here to help debug the issue. } return snapshot; @@ -173,7 +174,6 @@ public static PartitionKeyAccessor getPartitionKeyAccessor() { PartitionKeyAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("PartitionKeyAccessor is not initialized yet!"); - System.exit(9701); // Using a unique status code here to help debug the issue. } return snapshot; @@ -181,6 +181,7 @@ public static PartitionKeyAccessor getPartitionKeyAccessor() { public interface PartitionKeyAccessor { PartitionKey toPartitionKey(PartitionKeyInternal partitionKeyInternal); + PartitionKey toPartitionKey(List values, boolean strict); } } @@ -208,7 +209,6 @@ public static DirectConnectionConfigAccessor getDirectConnectionConfigAccessor() DirectConnectionConfigAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("DirectConnectionConfigAccessor is not initialized yet!"); - System.exit(9702); // Using a unique status code here to help debug the issue. } return snapshot; @@ -254,7 +254,6 @@ public static CosmosQueryRequestOptionsAccessor getCosmosQueryRequestOptionsAcce CosmosQueryRequestOptionsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosQueryRequestOptionsAccessor is not initialized yet!"); - System.exit(9703); // Using a unique status code here to help debug the issue. } return snapshot; @@ -307,7 +306,6 @@ public static CosmosReadManyRequestOptionsAccessor getCosmosReadManyRequestOptio CosmosReadManyRequestOptionsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosReadManyRequestOptionsAccessor is not initialized yet!"); - System.exit(9729); // Using a unique status code here to help debug the issue. } return snapshot; @@ -342,7 +340,6 @@ public static CosmosChangeFeedRequestOptionsAccessor getCosmosChangeFeedRequestO CosmosChangeFeedRequestOptionsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosChangeFeedRequestOptionsAccessor is not initialized yet!"); - System.exit(9704); // Using a unique status code here to help debug the issue. } return snapshot; @@ -357,6 +354,7 @@ public interface CosmosChangeFeedRequestOptionsAccessor { CosmosChangeFeedRequestOptions setItemFactoryMethod(CosmosChangeFeedRequestOptions queryRequestOptions, Function factoryMethod); CosmosDiagnosticsThresholds getDiagnosticsThresholds(CosmosChangeFeedRequestOptions options); List getExcludeRegions(CosmosChangeFeedRequestOptions cosmosChangeFeedRequestOptions); + CosmosChangeFeedRequestOptions createForProcessingFromContinuation(String continuation, FeedRange targetRange, String continuationLsn); } } @@ -384,7 +382,6 @@ public static CosmosItemRequestOptionsAccessor getCosmosItemRequestOptionsAccess CosmosItemRequestOptionsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosItemRequestOptionsAccessor is not initialized yet!"); - System.exit(9705); // Using a unique status code here to help debug the issue. } return snapshot; @@ -435,7 +432,6 @@ public static CosmosBulkExecutionOptionsAccessor getCosmosBulkExecutionOptionsAc CosmosBulkExecutionOptionsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBulkExecutionOptionsAccessor is not initialized yet!"); - System.exit(9706); // Using a unique status code here to help debug the issue. } return snapshot; @@ -480,9 +476,6 @@ CosmosBulkExecutionOptions setHeader(CosmosBulkExecutionOptions cosmosBulkExecut Map getCustomOptions(CosmosBulkExecutionOptions cosmosBulkExecutionOptions); List getExcludeRegions(CosmosBulkExecutionOptions cosmosBulkExecutionOptions); int getMaxMicroBatchSize(CosmosBulkExecutionOptions cosmosBulkExecutionOptions); - - void setMaxMicroBatchSize(CosmosBulkExecutionOptions cosmosBulkExecutionOptions, int maxMicroBatchSize); - void setDiagnosticsTracker(CosmosBulkExecutionOptions cosmosBulkExecutionOptions, BulkExecutorDiagnosticsTracker tracker); BulkExecutorDiagnosticsTracker getDiagnosticsTracker(CosmosBulkExecutionOptions cosmosBulkExecutionOptions); } @@ -514,7 +507,6 @@ public static CosmosItemResponseBuilderAccessor getCosmosItemResponseBuilderAcce CosmosItemResponseBuilderAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosItemResponseBuilderAccessor is not initialized yet!"); - System.exit(9707); // Using a unique status code here to help debug the issue. } return snapshot; @@ -567,7 +559,6 @@ public static CosmosClientAccessor getCosmosClientAccessor() { CosmosClientAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosClientAccessor is not initialized yet!"); - System.exit(9708); // Using a unique status code here to help debug the issue. } return snapshot; @@ -603,7 +594,6 @@ public static CosmosContainerPropertiesAccessor getCosmosContainerPropertiesAcce CosmosContainerPropertiesAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosContainerPropertiesAccessor is not initialized yet!"); - System.exit(9709); // Using a unique status code here to help debug the issue. } return snapshot; @@ -641,7 +631,6 @@ public static CosmosPageFluxAccessor getCosmosPageFluxAccessor() { CosmosPageFluxAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosPageFluxAccessor is not initialized yet!"); - System.exit(9710); // Using a unique status code here to help debug the issue. } return snapshot; @@ -677,7 +666,6 @@ public static CosmosAsyncDatabaseAccessor getCosmosAsyncDatabaseAccessor() { CosmosAsyncDatabaseAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosAsyncDatabaseAccessor is not initialized yet!"); - System.exit(9711); // Using a unique status code here to help debug the issue. } return snapshot; @@ -714,7 +702,6 @@ public static CosmosBulkExecutionThresholdsStateAccessor getBulkExecutionThresho CosmosBulkExecutionThresholdsStateAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBulkExecutionThresholdsStateAccessor is not initialized yet!"); - System.exit(9712); // Using a unique status code here to help debug the issue. } return snapshot; @@ -753,7 +740,6 @@ public static CosmosDiagnosticsAccessor getCosmosDiagnosticsAccessor() { CosmosDiagnosticsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosDiagnosticsAccessor is not initialized yet!"); - System.exit(9713); // Using a unique status code here to help debug the issue. } return snapshot; @@ -812,7 +798,6 @@ public static CosmosDiagnosticsContextAccessor getCosmosDiagnosticsContextAccess CosmosDiagnosticsContextAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosDiagnosticsAccessor is not initialized yet!"); - System.exit(9713); // Using a unique status code here to help debug the issue. } return snapshot; @@ -882,7 +867,7 @@ boolean endOperation( String getSpanName(CosmosDiagnosticsContext ctx); - void setSamplingRateSnapshot(CosmosDiagnosticsContext ctx, double samplingRate); + void setSamplingRateSnapshot(CosmosDiagnosticsContext ctx, double samplingRate, boolean isSampledOut); Integer getSequenceNumber(CosmosDiagnosticsContext ctx); @@ -915,7 +900,6 @@ public static CosmosAsyncContainerAccessor getCosmosAsyncContainerAccessor() { CosmosAsyncContainerAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosAsyncContainerAccessor is not initialized yet!"); - System.exit(9714); // Using a unique status code here to help debug the issue. } return snapshot; @@ -962,12 +946,15 @@ Function>> queryItemsInternalFu Mono> getFeedRanges(CosmosAsyncContainer cosmosAsyncContainer, boolean forceRefresh); - Mono> trySplitFeedRange( + Mono> trySplitFeedRange( CosmosAsyncContainer cosmosAsyncContainer, FeedRange feedRange, int targetedCountAfterSplit); String getLinkWithoutTrailingSlash(CosmosAsyncContainer cosmosAsyncContainer); + Mono checkFeedRangeOverlapping(CosmosAsyncContainer container, FeedRange feedRange1, FeedRange feedRange2); + Mono> getOverlappingFeedRanges(CosmosAsyncContainer container, FeedRange feedRange); + Mono getPartitionKeyDefinition(CosmosAsyncContainer container); } } @@ -996,7 +983,6 @@ public static FeedResponseAccessor getFeedResponseAccessor() { FeedResponseAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("FeedResponseAccessor is not initialized yet!"); - System.exit(9715); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1026,7 +1012,6 @@ public static CosmosBatchRequestOptionsAccessor getCosmosBatchRequestOptionsAcce CosmosBatchRequestOptionsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBatchRequestOptionsAccessor is not initialized yet!"); - System.exit(9716); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1067,7 +1052,6 @@ public static CosmosBatchOperationResultAccessor getCosmosBatchOperationResultAc CosmosBatchOperationResultAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBatchOperationResultAccessor is not initialized yet!"); - System.exit(9717); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1104,7 +1088,6 @@ public static CosmosPatchOperationsAccessor getCosmosPatchOperationsAccessor() { CosmosPatchOperationsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosPatchOperationsAccessor is not initialized yet!"); - System.exit(9718); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1140,7 +1123,6 @@ public static CosmosBatchAccessor getCosmosBatchAccessor() { CosmosBatchAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBatchAccessor is not initialized yet!"); - System.exit(9719); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1176,7 +1158,6 @@ public static CosmosBulkItemResponseAccessor getCosmosBulkItemResponseAccessor() CosmosBulkItemResponseAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBulkItemResponseAccessor is not initialized yet!"); - System.exit(9720); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1215,7 +1196,6 @@ public static CosmosBatchResponseAccessor getCosmosBatchResponseAccessor() { CosmosBatchResponseAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosBatchResponseAccessor is not initialized yet!"); - System.exit(9721); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1251,7 +1231,6 @@ public static CosmosAsyncClientEncryptionKeyAccessor getCosmosAsyncClientEncrypt CosmosAsyncClientEncryptionKeyAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosAsyncClientEncryptionKeyAccessor is not initialized yet!"); - System.exit(9722); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1296,7 +1275,6 @@ public static CosmosAsyncClientAccessor getCosmosAsyncClientAccessor() { CosmosAsyncClientAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosAsyncClientAccessor is not initialized yet!"); - System.exit(9723); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1355,7 +1333,6 @@ public static CosmosDiagnosticsThresholdsAccessor getCosmosAsyncClientAccessor() CosmosDiagnosticsThresholdsAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosDiagnosticsThresholdsAccessor is not initialized yet!"); - System.exit(9727); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1386,7 +1363,6 @@ public static CosmosExceptionAccessor getCosmosExceptionAccessor() { CosmosExceptionAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosExceptionAccessor is not initialized yet!"); - System.exit(9800); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1433,7 +1409,6 @@ public static CosmosClientTelemetryConfigAccessor getCosmosClientTelemetryConfig CosmosClientTelemetryConfigAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("CosmosClientTelemetryConfigAccessor is not initialized yet!"); - System.exit(9724); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1503,7 +1478,6 @@ public static PriorityLevelAccessor getPriorityLevelAccessor() { PriorityLevelAccessor snapshot = accessor.get(); if (snapshot == null) { logger.error("PriorityLevelAccessor is not initialized yet!"); - System.exit(9728); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1528,7 +1502,7 @@ public interface PriorityLevelAccessor { public static final class CosmosContainerIdentityHelper { - private static final AtomicReference cosmosContainerIdentityClassLoaded = new AtomicReference<>(false); + private static final AtomicBoolean cosmosContainerIdentityClassLoaded = new AtomicBoolean(false); private static final AtomicReference accessor = new AtomicReference<>(); private CosmosContainerIdentityHelper() {} @@ -1544,7 +1518,6 @@ public static CosmosContainerIdentityAccessor getCosmosContainerIdentityAccessor if (snapshot == null) { logger.error("CosmosContainerIdentityAccessor is not initialized yet!"); - System.exit(9725); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1571,7 +1544,7 @@ public interface CosmosContainerIdentityAccessor { public static final class CosmosContainerProactiveInitConfigHelper { - private static final AtomicReference cosmosContainerProactiveInitConfigClassLoaded = new AtomicReference<>(false); + private static final AtomicBoolean cosmosContainerProactiveInitConfigClassLoaded = new AtomicBoolean(false); private static final AtomicReference accessor = new AtomicReference<>(); private CosmosContainerProactiveInitConfigHelper() {} @@ -1587,7 +1560,6 @@ public static CosmosContainerProactiveInitConfigAccessor getCosmosContainerProac if (snapshot == null) { logger.error("CosmosContainerProactiveInitConfigAccessor is not initialized yet!"); - System.exit(9726); // Using a unique status code here to help debug the issue. } return snapshot; @@ -1611,7 +1583,7 @@ public interface CosmosContainerProactiveInitConfigAccessor { } public static final class CosmosSessionRetryOptionsHelper { - private static final AtomicReference cosmosSessionRetryOptionsClassLoaded = new AtomicReference<>(false); + private static final AtomicBoolean cosmosSessionRetryOptionsClassLoaded = new AtomicBoolean(false); private static final AtomicReference accessor = new AtomicReference<>(); private CosmosSessionRetryOptionsHelper() {} @@ -1627,7 +1599,6 @@ public static CosmosSessionRetryOptionsAccessor getCosmosSessionRetryOptionsAcce if (snapshot == null) { logger.error("cosmosSessionRetryOptionsAccessor is not initialized yet!"); - System.exit(9727); // Using a unique status code here to help debug the issue. } return snapshot; diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RenameCollectionAwareClientRetryPolicy.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RenameCollectionAwareClientRetryPolicy.java index 70963903745c9..ad1858673a37b 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RenameCollectionAwareClientRetryPolicy.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/RenameCollectionAwareClientRetryPolicy.java @@ -60,7 +60,7 @@ public Mono shouldRetry(Exception e) { Exceptions.isStatusCode(clientException, HttpConstants.StatusCodes.NOTFOUND) && Exceptions.isSubStatusCode(clientException, HttpConstants.SubStatusCodes.READ_SESSION_NOT_AVAILABLE)) { // Clear the session token, because the collection name might be reused. - logger.warn("Clear the token for named base request {}", request.getResourceAddress()); + logger.info("Clear the token for named base request {}", request.getResourceAddress()); this.sessionContainer.clearTokenByCollectionFullName(request.getResourceAddress()); diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/VectorSessionToken.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/VectorSessionToken.java index f8f811c42043c..827923b1c2a10 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/VectorSessionToken.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/VectorSessionToken.java @@ -264,7 +264,7 @@ private static boolean tryParseSessionToken( globalLsn.v = -1L; if (Strings.isNullOrEmpty(sessionToken)) { - logger.warn("SESSION token is empty"); + logger.info("SESSION token is empty"); return false; } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/clienttelemetry/ClientTelemetryMetrics.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/clienttelemetry/ClientTelemetryMetrics.java index ba00d71e46fbc..def78d8fa141d 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/clienttelemetry/ClientTelemetryMetrics.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/clienttelemetry/ClientTelemetryMetrics.java @@ -342,9 +342,9 @@ private static Tags createOperationTags( String containerTagValue = escape(accountTagValue) + "/" - + databaseId != null ? escape(databaseId) : "NONE" + + (databaseId != null ? escape(databaseId) : "NONE") + "/" - + containerId != null ? escape(containerId) : "NONE"; + + (containerId != null ? escape(containerId) : "NONE"); effectiveTags.add(Tag.of(TagName.Container.toString(), containerTagValue)); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ChannelAcquisitionException.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ChannelAcquisitionException.java new file mode 100644 index 0000000000000..c2a930c4fc08f --- /dev/null +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ChannelAcquisitionException.java @@ -0,0 +1,17 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.implementation.directconnectivity; + +import java.net.ConnectException; + +public class ChannelAcquisitionException extends ConnectException { + public ChannelAcquisitionException(String message) { + super(message); + } + + @Override + public synchronized Throwable fillInStackTrace() { + return this; + } +} diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriter.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriter.java index 045632516078a..1f9e1631c0881 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriter.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/ConsistencyWriter.java @@ -235,10 +235,15 @@ Mono writePrivateAsync( } } } catch (Throwable throwable) { - logger.error("Unexpected failure in handling orig [{}]", t.getMessage(), t); - logger.error("Unexpected failure in handling orig [{}] : new [{}]", t.getMessage(), throwable.getMessage(), throwable); if (throwable instanceof Error) { + logger.error("Unexpected failure in handling orig [{}]", t.getMessage(), t); + logger.error("Unexpected failure in handling orig [{}] : new [{}]", t.getMessage(), throwable.getMessage(), throwable); throw (Error) throwable; + } else { + // this happens before any retry policy - like for example GoneAndRetryRetryPolicy + // kicks in - no need to spam warn/error level logs yet + logger.info("Unexpected failure in handling orig [{}]", t.getMessage(), t); + logger.info("Unexpected failure in handling orig [{}] : new [{}]", t.getMessage(), throwable.getMessage(), throwable); } } } @@ -273,7 +278,7 @@ Mono writePrivateAsync( .flatMap(v -> { if (!v) { - logger.warn("ConsistencyWriter: Write barrier has not been met for global strong request. SelectedGlobalCommittedLsn: {}", request.requestContext.globalCommittedSelectedLSN); + logger.info("ConsistencyWriter: Write barrier has not been met for global strong request. SelectedGlobalCommittedLsn: {}", request.requestContext.globalCommittedSelectedLSN); return Mono.error(new GoneException(RMResources.GlobalStrongWriteBarrierNotMet, HttpConstants.SubStatusCodes.GLOBAL_STRONG_WRITE_BARRIER_NOT_MET)); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/QuorumReader.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/QuorumReader.java index 1074b285dc1eb..0a1d5e164a32e 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/QuorumReader.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/QuorumReader.java @@ -188,7 +188,7 @@ public Mono readStrongAsync( } // else barrier was not successful - logger.warn( + logger.info( "QuorumSelected: Could not converge on the LSN {} GlobalCommittedLSN {} after primary read barrier with read quorum {} for strong read, Responses: {}", secondaryQuorumReadResult.selectedLsn, secondaryQuorumReadResult.globalCommittedSelectedLsn, @@ -208,7 +208,7 @@ public Mono readStrongAsync( case QuorumNotSelected: if (hasPerformedReadFromPrimary.v) { - logger.warn( + logger.info( "QuorumNotSelected: Primary read already attempted. Quorum could not be " + "selected after retrying on secondaries. ReadQuorumResult StoreResponses: {}", String.join(";", secondaryQuorumReadResult.storeResponses)); @@ -218,7 +218,7 @@ public Mono readStrongAsync( HttpConstants.SubStatusCodes.READ_QUORUM_NOT_MET)); } - logger.warn( + logger.info( "QuorumNotSelected: Quorum could not be selected with read quorum of {}, " + "ReadQuorumResult StoreResponses: {}", readQuorumValue, @@ -244,7 +244,7 @@ public Mono readStrongAsync( } } else if (response.shouldRetryOnSecondary) { shouldRetryOnSecondary.v = true; - logger.warn("QuorumNotSelected: ReadPrimary did not succeed. Will retry " + logger.info("QuorumNotSelected: ReadPrimary did not succeed. Will retry " + "on secondary. ReadQuorumResult StoreResponses: {}", String.join(";", secondaryQuorumReadResult.storeResponses)); hasPerformedReadFromPrimary.v = true; @@ -255,7 +255,7 @@ public Mono readStrongAsync( // Primary replica even for quorum selection in this case for the retry includePrimary.v = true; } else { - logger.warn("QuorumNotSelected: Could not get successful response from ReadPrimary"); + logger.info("QuorumNotSelected: Could not get successful response from ReadPrimary"); return Flux.error(new GoneException(String.format(RMResources.ReadQuorumNotMet, readQuorumValue), HttpConstants.SubStatusCodes.READ_QUORUM_NOT_MET)); } @@ -278,7 +278,7 @@ public Mono readStrongAsync( // In case there is an empty response from above flatMap, it means we could not complete read quorum // So we will throw an error, which will be eventually retried. .switchIfEmpty(Flux.defer(() -> { - logger.warn("Could not complete read quorum with read quorum value of {}", readQuorumValue); + logger.info("Could not complete read quorum with read quorum value of {}", readQuorumValue); return Flux.error(new GoneException( String.format( @@ -466,7 +466,7 @@ private Mono readPrimaryAsync( } if (storeResult.currentReplicaSetSize > readQuorum) { - logger.warn( + logger.info( "Unexpected response. Replica Set size is {} which is greater than min value {}", storeResult.currentReplicaSetSize, readQuorum); return Mono.just(new ReadPrimaryResult(entity.requestContext.requestChargeTracker, /*isSuccessful */ false, /* shouldRetryOnSecondary: */ true, /* response: */ null)); @@ -477,7 +477,7 @@ private Mono readPrimaryAsync( // In case of async replication (if enabled for bounded staleness), the store LSN can be ahead of the quorum committed LSN if the primary is able write to faster than secondary acks. // We pick higher of the 2 LSN and wait for the other to reach that LSN. if (storeResult.lsn != storeResult.quorumAckedLSN) { - logger.warn("Store LSN {} and quorum acked LSN {} don't match", storeResult.lsn, storeResult.quorumAckedLSN); + logger.info("Store LSN {} and quorum acked LSN {} don't match", storeResult.lsn, storeResult.quorumAckedLSN); long higherLsn = storeResult.lsn > storeResult.quorumAckedLSN ? storeResult.lsn : storeResult.quorumAckedLSN; Mono waitForLsnRequestObs = BarrierRequestHelper.createAsync(this.diagnosticsClientContext, entity, this.authorizationTokenProvider, higherLsn, null); @@ -536,14 +536,14 @@ private Mono waitForPrimaryLsnAsync( } if (storeResult.currentReplicaSetSize > readQuorum) { - logger.warn( + logger.info( "Unexpected response. Replica Set size is {} which is greater than min value {}", storeResult.currentReplicaSetSize, readQuorum); return Flux.just(PrimaryReadOutcome.QuorumInconclusive); } // Java this will move to the repeat logic if (storeResult.lsn < lsnToWaitFor || storeResult.quorumAckedLSN < lsnToWaitFor) { - logger.warn( + logger.info( "Store LSN {} or quorum acked LSN {} are lower than expected LSN {}", storeResult.lsn, storeResult.quorumAckedLSN, lsnToWaitFor); return Flux.just(0L).delayElements( diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdClientChannelPool.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdClientChannelPool.java index 6054ebda22f7b..3c7f8159b99e6 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdClientChannelPool.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdClientChannelPool.java @@ -4,6 +4,7 @@ package com.azure.cosmos.implementation.directconnectivity.rntbd; import com.azure.cosmos.implementation.clienttelemetry.ClientTelemetry; +import com.azure.cosmos.implementation.directconnectivity.ChannelAcquisitionException; import com.azure.cosmos.implementation.directconnectivity.rntbd.RntbdEndpoint.Config; import com.azure.cosmos.implementation.faultinjection.RntbdFaultInjectionConnectionCloseEvent; import com.azure.cosmos.implementation.faultinjection.RntbdFaultInjectionConnectionResetEvent; @@ -36,6 +37,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.net.ConnectException; import java.net.SocketAddress; import java.nio.channels.ClosedChannelException; import java.time.Duration; @@ -120,25 +122,25 @@ public final class RntbdClientChannelPool implements ChannelPool { // TODO: moderakh setup proper retry in higher stack for the exceptions here - private static final TimeoutException ACQUISITION_TIMEOUT = ThrowableUtil.unknownStackTrace( - new TimeoutException("acquisition took longer than the configured maximum time"), + private static final ChannelAcquisitionException ACQUISITION_TIMEOUT = ThrowableUtil.unknownStackTrace( + new ChannelAcquisitionException("acquisition took longer than the configured maximum time"), RntbdClientChannelPool.class, ""); private static final ClosedChannelException CHANNEL_CLOSED_ON_ACQUIRE = ThrowableUtil.unknownStackTrace( new ClosedChannelException(), RntbdClientChannelPool.class, "acquire"); - private static final IllegalStateException POOL_CLOSED_ON_ACQUIRE = ThrowableUtil.unknownStackTrace( + private static final ChannelAcquisitionException POOL_CLOSED_ON_ACQUIRE = ThrowableUtil.unknownStackTrace( new ChannelAcquisitionException("service endpoint was closed while acquiring a channel"), RntbdClientChannelPool.class, "acquire"); - private static final IllegalStateException POOL_CLOSED_ON_RELEASE = ThrowableUtil.unknownStackTrace( + private static final ChannelAcquisitionException POOL_CLOSED_ON_RELEASE = ThrowableUtil.unknownStackTrace( new ChannelAcquisitionException("service endpoint was closed while releasing a channel"), RntbdClientChannelPool.class, "release"); private static final AttributeKey POOL_KEY = AttributeKey.newInstance( RntbdClientChannelPool.class.getName()); - private static final IllegalStateException TOO_MANY_PENDING_ACQUISITIONS = ThrowableUtil.unknownStackTrace( + private static final ChannelAcquisitionException TOO_MANY_PENDING_ACQUISITIONS = ThrowableUtil.unknownStackTrace( new ChannelAcquisitionException("too many outstanding acquire operations"), RntbdClientChannelPool.class, "acquire"); @@ -1046,7 +1048,7 @@ private void doClose() { }).addListener(closed -> { if (!closed.isSuccess()) { - logger.error("[{}] close failed due to ", this, closed.cause()); + logger.info("[{}] close failed due to ", this, closed.cause()); } else { logger.debug("[{}] closed", this); } @@ -1462,7 +1464,7 @@ private void releaseAndOfferChannel(final Channel channel, final Promise p this.poolHandler.channelReleased(channel); promise.setSuccess(null); } else { - final IllegalStateException error = new ChannelAcquisitionException(lenientFormat( + final ChannelAcquisitionException error = new ChannelAcquisitionException(lenientFormat( "cannot offer channel back to pool because the pool is at capacity (%s)\n %s\n %s", this.maxChannels, this, @@ -1642,7 +1644,7 @@ private void doOperationComplete(Channel channel) { this.originalPromise.setSuccess(channel); } else { final Throwable cause = completed.cause(); - logger.warn("Channel({}) health check request failed due to:", channel, cause); + logger.info("Channel({}) health check request failed due to:", channel, cause); this.fail(cause); } }); @@ -1792,20 +1794,6 @@ public void serialize( } } - private static class ChannelAcquisitionException extends IllegalStateException { - - private static final long serialVersionUID = -6011782222645074949L; - - public ChannelAcquisitionException(String message) { - super(message); - } - - @Override - public synchronized Throwable fillInStackTrace() { - return this; - } - } - // endregion } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdServiceEndpoint.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdServiceEndpoint.java index 22c37aabdea83..f0120e54cf301 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdServiceEndpoint.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/implementation/directconnectivity/rntbd/RntbdServiceEndpoint.java @@ -875,12 +875,12 @@ synchronized void logAllPools() { } private void logEndpoint(RntbdEndpoint endpoint) { - if (this.logger.isWarnEnabled() && + if (this.logger.isInfoEnabled() && (endpoint.executorTaskQueueMetrics() > MAX_TASK_LIMIT || endpoint.requestQueueLength() > MAX_TASK_LIMIT || endpoint.gettingEstablishedConnectionsMetrics() > 0 || endpoint.channelsMetrics() > endpoint.maxChannels())) { - logger.warn("RntbdEndpoint Identifier {}, Stat {}", getPoolId(endpoint), getPoolStat(endpoint)); + logger.info("RntbdEndpoint Identifier {}, Stat {}", getPoolId(endpoint), getPoolStat(endpoint)); } else if (this.logger.isDebugEnabled()) { logger.debug("RntbdEndpoint Identifier {}, Stat {}", getPoolId(endpoint), getPoolStat(endpoint)); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosBulkExecutionOptions.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosBulkExecutionOptions.java index 93744f38d241b..e32bf5a7e5e0e 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosBulkExecutionOptions.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosBulkExecutionOptions.java @@ -127,11 +127,24 @@ CosmosBulkExecutionOptions setMaxMicroBatchPayloadSizeInBytes(int maxMicroBatchP return this; } - int getMaxMicroBatchSize() { + /** + * The maximum batch size for bulk operations. Once queued docs exceed this value, the micro + * batch will be flushed to the wire. + * + * @return the max micro batch size. + */ + public int getMaxMicroBatchSize() { return maxMicroBatchSize; } - CosmosBulkExecutionOptions setMaxMicroBatchSize(int maxMicroBatchSize) { + /** + * The maximum batch size for bulk operations. Once queued docs exceed this value, the micro + * batch will be flushed to the wire. + * + * @param maxMicroBatchSize maximum batching size. + * @return the bulk processing options. + */ + public CosmosBulkExecutionOptions setMaxMicroBatchSize(int maxMicroBatchSize) { this.maxMicroBatchSize = maxMicroBatchSize; return this; } @@ -442,11 +455,6 @@ public int getMaxMicroBatchSize(CosmosBulkExecutionOptions cosmosBulkExecutionOp return cosmosBulkExecutionOptions.getMaxMicroBatchSize(); } - @Override - public void setMaxMicroBatchSize(CosmosBulkExecutionOptions cosmosBulkExecutionOptions, int maxMicroBatchSize) { - cosmosBulkExecutionOptions.setMaxMicroBatchSize(maxMicroBatchSize); - } - @Override public void setDiagnosticsTracker(CosmosBulkExecutionOptions cosmosBulkExecutionOptions, BulkExecutorDiagnosticsTracker tracker) { cosmosBulkExecutionOptions.setDiagnosticsTracker(tracker); diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosChangeFeedRequestOptions.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosChangeFeedRequestOptions.java index 1ab6c161cad6b..ffeeab187a4fa 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosChangeFeedRequestOptions.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/CosmosChangeFeedRequestOptions.java @@ -11,14 +11,18 @@ import com.azure.cosmos.implementation.changefeed.common.ChangeFeedMode; import com.azure.cosmos.implementation.changefeed.common.ChangeFeedStartFromInternal; import com.azure.cosmos.implementation.changefeed.common.ChangeFeedState; +import com.azure.cosmos.implementation.changefeed.common.ChangeFeedStateV1; import com.azure.cosmos.implementation.feedranges.FeedRangeContinuation; +import com.azure.cosmos.implementation.feedranges.FeedRangeEpkImpl; import com.azure.cosmos.implementation.feedranges.FeedRangeInternal; import com.azure.cosmos.implementation.query.CompositeContinuationToken; +import com.azure.cosmos.implementation.routing.Range; import com.azure.cosmos.implementation.spark.OperationContextAndListenerTuple; import com.azure.cosmos.util.Beta; import com.fasterxml.jackson.databind.JsonNode; import java.time.Instant; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -262,6 +266,44 @@ public static CosmosChangeFeedRequestOptions createForProcessingFromContinuation return createForProcessingFromContinuation(changeFeedState); } + /*** + * Creates a new {@link CosmosChangeFeedRequestOptions} instance to start processing + * change feed items based on a previous continuation. + * + * ONLY used by Kafka connector. + * + * @param continuation The continuation that was retrieved from a previously retrieved FeedResponse + * @param targetRange the new target range + * @param continuationLsn the new continuation lsn + * @return a new {@link CosmosChangeFeedRequestOptions} instance + */ + static CosmosChangeFeedRequestOptions createForProcessingFromContinuation( + String continuation, FeedRange targetRange, String continuationLsn) { + if (targetRange instanceof FeedRangeEpkImpl) { + Range normalizedRange = + FeedRangeInternal.normalizeRange(((FeedRangeEpkImpl) targetRange).getRange()); + + + final ChangeFeedState changeFeedState = ChangeFeedState.fromString(continuation); + ChangeFeedState targetChangeFeedState = + new ChangeFeedStateV1( + changeFeedState.getContainerRid(), + (FeedRangeEpkImpl) targetRange, + changeFeedState.getMode(), + changeFeedState.getStartFromSettings(), + FeedRangeContinuation.create( + changeFeedState.getContainerRid(), + (FeedRangeEpkImpl) targetRange, + Arrays.asList(new CompositeContinuationToken(continuationLsn, normalizedRange)) + ) + ); + + return createForProcessingFromContinuation(targetChangeFeedState); + } + + throw new IllegalStateException("createForProcessingFromContinuation does not support feedRange type " + targetRange.getClass()); + } + static CosmosChangeFeedRequestOptions createForProcessingFromContinuation( ChangeFeedState changeFeedState) { @@ -609,6 +651,15 @@ public CosmosDiagnosticsThresholds getDiagnosticsThresholds(CosmosChangeFeedRequ public List getExcludeRegions(CosmosChangeFeedRequestOptions cosmosChangeFeedRequestOptions) { return cosmosChangeFeedRequestOptions.excludeRegions; } + + @Override + public CosmosChangeFeedRequestOptions createForProcessingFromContinuation( + String continuation, + FeedRange targetRange, + String continuationLsn) { + + return CosmosChangeFeedRequestOptions.createForProcessingFromContinuation(continuation, targetRange, continuationLsn); + } }); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/PartitionKey.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/PartitionKey.java index dd0e9eb0238fd..3531cd67ffbcf 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/PartitionKey.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/models/PartitionKey.java @@ -7,6 +7,8 @@ import com.azure.cosmos.implementation.Utils; import com.azure.cosmos.implementation.routing.PartitionKeyInternal; +import java.util.List; + /** * Represents a partition key value in the Azure Cosmos DB database service. A * partition key identifies the partition where the item is stored in. @@ -96,6 +98,12 @@ static void initialize() { public PartitionKey toPartitionKey(PartitionKeyInternal partitionKeyInternal) { return new PartitionKey(partitionKeyInternal); } + + @Override + public PartitionKey toPartitionKey(List values, boolean strict) { + PartitionKeyInternal partitionKeyInternal = PartitionKeyInternal.fromObjectArray(values, strict); + return new PartitionKey(partitionKeyInternal); + } } ); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/util/CosmosPagedFlux.java b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/util/CosmosPagedFlux.java index 98dff6a9ffe53..968b3bd07d6a6 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/util/CosmosPagedFlux.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/com/azure/cosmos/util/CosmosPagedFlux.java @@ -150,9 +150,7 @@ private Flux> wrapWithTracingIfEnabled(CosmosPagedFluxOptions p FeedOperationState state = pagedFluxOptions.getFeedOperationState(); DiagnosticsProvider tracerProvider = state != null ? state.getDiagnosticsProvider() : null; Object lockHolder = new Object(); - if (tracerProvider == null || - !tracerProvider.isEnabled() - || tracerProvider.shouldSampleOutOperation(pagedFluxOptions)) { + if (tracerProvider == null) { return publisher .doOnEach(signal -> { @@ -170,6 +168,13 @@ private Flux> wrapWithTracingIfEnabled(CosmosPagedFluxOptions p }); } + if (!tracerProvider.isEnabled()) { + pagedFluxOptions.setSamplingRateSnapshot(0, true); + } + + final boolean isSampledOut = tracerProvider.shouldSampleOutOperation(pagedFluxOptions); + final double samplingRateSnapshot = pagedFluxOptions.getSamplingRateSnapshot(); + Flux> result = tracerProvider .runUnderSpanInContext(publisher) .doOnEach(signal -> { @@ -185,39 +190,51 @@ private Flux> wrapWithTracingIfEnabled(CosmosPagedFluxOptions p state.mergeDiagnosticsContext(); CosmosDiagnosticsContext ctxSnapshot = state.getDiagnosticsContextSnapshot(); + + ctxAccessor + .setSamplingRateSnapshot(ctxSnapshot, samplingRateSnapshot, isSampledOut); + tracerProvider.recordFeedResponseConsumerLatency( signal, ctxSnapshot, Duration.ofNanos(feedResponseConsumerLatencyInNanos.get())); - tracerProvider.endSpan(ctxSnapshot, traceCtx, ctxAccessor.isEmptyCompletion(ctxSnapshot)); + tracerProvider.endSpan(ctxSnapshot, traceCtx, ctxAccessor.isEmptyCompletion(ctxSnapshot), isSampledOut); break; case ON_NEXT: this.recordFeedResponse(pagedFluxOptions, tracerProvider, response, feedResponseConsumerLatencyInNanos); state.mergeDiagnosticsContext(); - tracerProvider.endSpan(state.getDiagnosticsContextSnapshot(), traceCtx, false); + CosmosDiagnosticsContext ctxSnapshotOnNext = state.getDiagnosticsContextSnapshot(); + ctxAccessor + .setSamplingRateSnapshot(ctxSnapshotOnNext, samplingRateSnapshot, isSampledOut); + tracerProvider.endSpan(ctxSnapshotOnNext, traceCtx, false, isSampledOut); state.resetDiagnosticsContext(); DiagnosticsProvider.setContextInReactor(tracerProvider.startSpan( state.getSpanName(), state.getDiagnosticsContextSnapshot(), - traceCtx)); + traceCtx, + isSampledOut)); break; case ON_ERROR: state.mergeDiagnosticsContext(); + CosmosDiagnosticsContext ctxSnapshotOnError = state.getDiagnosticsContextSnapshot(); + ctxAccessor + .setSamplingRateSnapshot(ctxSnapshotOnError, samplingRateSnapshot, isSampledOut); tracerProvider.recordFeedResponseConsumerLatency( signal, - state.getDiagnosticsContextSnapshot(), + ctxSnapshotOnError, Duration.ofNanos(feedResponseConsumerLatencyInNanos.get())); // all info is extracted from CosmosException when applicable tracerProvider.endSpan( state.getDiagnosticsContextSnapshot(), traceCtx, - signal.getThrowable() + signal.getThrowable(), + isSampledOut ); break; @@ -234,8 +251,12 @@ private Flux> wrapWithTracingIfEnabled(CosmosPagedFluxOptions p Context traceCtx = DiagnosticsProvider.getContextFromReactorOrNull(reactorCtx); synchronized (lockHolder) { state.mergeDiagnosticsContext(); + CosmosDiagnosticsContext ctxSnapshot = state.getDiagnosticsContextSnapshot(); + + ctxAccessor + .setSamplingRateSnapshot(ctxSnapshot, samplingRateSnapshot, isSampledOut); - tracerProvider.endSpan(state.getDiagnosticsContextSnapshot(), traceCtx, false); + tracerProvider.endSpan(ctxSnapshot, traceCtx, false, isSampledOut); } }) .doOnComplete(() -> { @@ -244,14 +265,17 @@ private Flux> wrapWithTracingIfEnabled(CosmosPagedFluxOptions p state.mergeDiagnosticsContext(); CosmosDiagnosticsContext ctxSnapshot = state.getDiagnosticsContextSnapshot(); - tracerProvider.endSpan(ctxSnapshot, traceCtx, ctxAccessor.isEmptyCompletion(ctxSnapshot)); + ctxAccessor + .setSamplingRateSnapshot(ctxSnapshot, samplingRateSnapshot, isSampledOut); + tracerProvider.endSpan(ctxSnapshot, traceCtx, ctxAccessor.isEmptyCompletion(ctxSnapshot), isSampledOut); } })) .contextWrite(DiagnosticsProvider.setContextInReactor( tracerProvider.startSpan( state.getSpanName(), state.getDiagnosticsContextSnapshot(), - context) + context, + isSampledOut) )); } diff --git a/sdk/cosmos/azure-cosmos/src/main/java/module-info.java b/sdk/cosmos/azure-cosmos/src/main/java/module-info.java index 639cf5023f5ee..85ddd1b4e7af3 100644 --- a/sdk/cosmos/azure-cosmos/src/main/java/module-info.java +++ b/sdk/cosmos/azure-cosmos/src/main/java/module-info.java @@ -35,20 +35,20 @@ // export packages for multiple different modules exports com.azure.cosmos.implementation to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.caches to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.feedranges to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.apachecommons.lang to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.guava25.base to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.guava25.collect to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.guava27 to com.azure.cosmos.encryption, com.azure.cosmos.test, com.azure.cosmos.kafka.connect; + exports com.azure.cosmos.implementation.caches to com.azure.cosmos.encryption, com.azure.cosmos.test; + exports com.azure.cosmos.implementation.feedranges to com.azure.cosmos.encryption, com.azure.cosmos.test; + exports com.azure.cosmos.implementation.apachecommons.lang to com.azure.cosmos.encryption, com.azure.cosmos.test; + exports com.azure.cosmos.implementation.guava25.base to com.azure.cosmos.encryption, com.azure.cosmos.test; + exports com.azure.cosmos.implementation.guava25.collect to com.azure.cosmos.encryption, com.azure.cosmos.test; + exports com.azure.cosmos.implementation.guava27 to com.azure.cosmos.encryption, com.azure.cosmos.test; exports com.azure.cosmos.implementation.directconnectivity to com.azure.cosmos.encryption, com.azure.cosmos.test; opens com.azure.cosmos.implementation to com.fasterxml.jackson.databind, java.logging, com.fasterxml.jackson.module.afterburner; // exporting implementation packages specifically for cosmos encryption exports com.azure.cosmos.implementation.batch to com.azure.cosmos.encryption; exports com.azure.cosmos.implementation.patch to com.azure.cosmos.encryption; - exports com.azure.cosmos.implementation.query to com.azure.cosmos.encryption, com.azure.cosmos.kafka.connect; - exports com.azure.cosmos.implementation.apachecommons.lang.tuple to com.azure.cosmos.encryption, com.azure.cosmos.kafka.connect; + exports com.azure.cosmos.implementation.query to com.azure.cosmos.encryption; + exports com.azure.cosmos.implementation.apachecommons.lang.tuple to com.azure.cosmos.encryption; // exporting some packages specifically for Jackson opens com.azure.cosmos.implementation.caches to com.fasterxml.jackson.databind; @@ -74,10 +74,9 @@ // exporting packages specifically for cosmos test exports com.azure.cosmos.implementation.faultinjection to com.azure.cosmos.test; exports com.azure.cosmos.implementation.directconnectivity.rntbd to com.azure.cosmos.test; - exports com.azure.cosmos.implementation.routing to com.azure.cosmos.test, com.azure.cosmos.kafka.connect; + exports com.azure.cosmos.implementation.routing to com.azure.cosmos.test; opens com.azure.cosmos to com.azure.cosmos.test, com.azure.spring.data.cosmos, com.fasterxml.jackson.databind, com.fasterxml.jackson.module.afterburner, java.logging; opens com.azure.cosmos.models to com.azure.cosmos.test, com.azure.spring.data.cosmos, com.fasterxml.jackson.databind, com.fasterxml.jackson.module.afterburner, java.logging; - exports com.azure.cosmos.implementation.changefeed.common to com.azure.cosmos.kafka.connect; uses com.azure.cosmos.implementation.guava25.base.PatternCompiler; uses com.azure.core.util.tracing.Tracer; diff --git a/sdk/cosmos/ci.yml b/sdk/cosmos/ci.yml index 1b5ac26bda226..39a2b82ec70ce 100644 --- a/sdk/cosmos/ci.yml +++ b/sdk/cosmos/ci.yml @@ -152,7 +152,7 @@ extends: uberJar: true skipPublishDocGithubIo: true skipPublishDocMs: true - releaseInBatch: ${{ parameters.release_azurecosmosspark34 }} + releaseInBatch: ${{ parameters.release_azurecosmosspark34 }} - name: azure-cosmos-test groupId: com.azure safeName: azurecosmostest diff --git a/sdk/cosmos/live-platform-matrix.json b/sdk/cosmos/live-platform-matrix.json index 2f78ef0a12e7d..a564450f6fece 100644 --- a/sdk/cosmos/live-platform-matrix.json +++ b/sdk/cosmos/live-platform-matrix.json @@ -38,7 +38,7 @@ ], "ProfileFlag": "-Pe2e", "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -48,7 +48,7 @@ "ProfileFlag": [ "-Pcfp-split", "-Psplit", "-Pquery", "-Pfast", "-Pdirect" ], "ArmTemplateParameters": "@{ enableMultipleWriteLocations = $false; defaultConsistencyLevel = 'Session' }", "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -58,7 +58,7 @@ "PROTOCOLS": "[\"Tcp\"]", "ProfileFlag": [ "-Pcfp-split", "-Psplit", "-Pquery", "-Pfast" ], "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -68,7 +68,7 @@ "ProfileFlag": "-Pe2e", "ArmTemplateParameters": "@{ enableMultipleWriteLocations = $false; defaultConsistencyLevel = 'Strong' }", "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -78,7 +78,7 @@ "ProfileFlag": [ "-Pdirect" ], "ArmTemplateParameters": "@{ enableMultipleWriteLocations = $false; defaultConsistencyLevel = 'Strong' }", "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -88,7 +88,7 @@ "ProfileFlag": [ "-Plong" ], "ArmTemplateParameters": "@{ enableMultipleWriteLocations = $false; defaultConsistencyLevel = 'Session'; enablePartitionMerge = $true }", "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -103,7 +103,7 @@ "PROTOCOLS": "[\"Tcp\"]", "ProfileFlag": [ "-Pcfp-split", "-Psplit", "-Pquery", "-Pmulti-master", "-Pflaky-multi-master", "-Pfast", "-Pdirect" ], "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } }, { @@ -118,7 +118,7 @@ "PROTOCOLS": "[\"Tcp\"]", "PREFERRED_LOCATIONS": null, "Agent": { - "ubuntu": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } } } ] diff --git a/sdk/cosmos/tests.yml b/sdk/cosmos/tests.yml index 8093ca170cca0..b5f991eba371a 100644 --- a/sdk/cosmos/tests.yml +++ b/sdk/cosmos/tests.yml @@ -1,92 +1,97 @@ trigger: none -variables: - - template: ../../eng/pipelines/templates/variables/globals.yml - - name: AdditionalArgs - value: '-DCOSMOS.CLIENT_TELEMETRY_ENDPOINT=$(cosmos-client-telemetry-endpoint) -DCOSMOS.CLIENT_TELEMETRY_COSMOS_ACCOUNT=$(cosmos-client-telemetry-cosmos-account)' +extends: + template: /eng/pipelines/templates/stages/1es-redirect.yml + parameters: + stages: + - template: /eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml + parameters: + CloudConfig: + Cosmos_Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-cosmos-azure-cloud-test-resources) + Clouds: 'Cosmos_Public_Integration' + MatrixConfigs: + - Name: Cosmos_live_test + Path: sdk/cosmos/live-platform-matrix.json + Selection: all + GenerateVMJobs: true + MatrixReplace: + - .*Version=1.21/1.17 + ServiceDirectory: cosmos + Artifacts: + - name: azure-cosmos + groupId: com.azure + safeName: azurecosmos + AdditionalModules: + - name: azure-cosmos-tests + groupId: com.azure + - name: azure-cosmos-benchmark + groupId: com.azure + TimeoutInMinutes: 120 + MaxParallel: 20 + PreSteps: + - template: /eng/pipelines/templates/steps/install-reporting-tools.yml + TestGoals: 'verify' + TestOptions: '$(ProfileFlag) $(AdditionalArgs) -DskipCompile=true -DskipTestCompile=true -DcreateSourcesJar=false' + TestResultsFiles: '**/junitreports/TEST-*.xml' + AdditionalVariables: + - name: AdditionalArgs + value: '-DCOSMOS.CLIENT_TELEMETRY_ENDPOINT=$(cosmos-client-telemetry-endpoint) -DCOSMOS.CLIENT_TELEMETRY_COSMOS_ACCOUNT=$(cosmos-client-telemetry-cosmos-account)' -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - CloudConfig: - Cosmos_Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-cosmos-azure-cloud-test-resources) - Clouds: 'Cosmos_Public_Integration' - MatrixConfigs: - - Name: Cosmos_live_test - Path: sdk/cosmos/live-platform-matrix.json - Selection: all - GenerateVMJobs: true - MatrixReplace: - - .*Version=1.21/1.17 - ServiceDirectory: cosmos - Artifacts: - - name: azure-cosmos - groupId: com.azure - safeName: azurecosmos - AdditionalModules: - - name: azure-cosmos-tests - groupId: com.azure - - name: azure-cosmos-benchmark - groupId: com.azure - TimeoutInMinutes: 120 - MaxParallel: 20 - PreSteps: - - template: /eng/pipelines/templates/steps/install-reporting-tools.yml - TestGoals: 'verify' - TestOptions: '$(ProfileFlag) $(AdditionalArgs) -DskipCompile=true -DskipTestCompile=true -DcreateSourcesJar=false' - TestResultsFiles: '**/junitreports/TEST-*.xml' + - template: /eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml + parameters: + TestName: 'Spring_Data_Cosmos_Integration' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-cosmos-azure-cloud-test-resources) + MatrixConfigs: + - Name: Cosmos_live_test_integration + Path: sdk/spring/pipeline/cosmos-integration-matrix.json + Selection: all + GenerateVMJobs: true + ServiceDirectory: spring + TestResourceDirectories: + - spring/spring-cloud-azure-integration-tests/test-resources/cosmos-spring + Artifacts: + - name: azure-spring-data-cosmos + groupId: com.azure + safeName: azurespringdatacosmos + TimeoutInMinutes: 90 + PreSteps: + - template: /eng/pipelines/templates/steps/install-reporting-tools.yml + TestGoals: 'verify' + TestOptions: '$(ProfileFlag) -DskipCompile=true -DskipTestCompile=true -DcreateSourcesJar=false' + AdditionalVariables: + - name: AdditionalArgs + value: '-DCOSMOS.CLIENT_TELEMETRY_ENDPOINT=$(cosmos-client-telemetry-endpoint) -DCOSMOS.CLIENT_TELEMETRY_COSMOS_ACCOUNT=$(cosmos-client-telemetry-cosmos-account)' - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - TestName: 'Spring_Data_Cosmos_Integration' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-cosmos-azure-cloud-test-resources) - MatrixConfigs: - - Name: Cosmos_live_test_integration - Path: sdk/spring/pipeline/cosmos-integration-matrix.json - Selection: all - GenerateVMJobs: true - ServiceDirectory: spring - TestResourceDirectories: - - spring/spring-cloud-azure-integration-tests/test-resources/cosmos-spring - Artifacts: - - name: azure-spring-data-cosmos - groupId: com.azure - safeName: azurespringdatacosmos - TimeoutInMinutes: 90 - PreSteps: - - template: /eng/pipelines/templates/steps/install-reporting-tools.yml - TestGoals: 'verify' - TestOptions: '$(ProfileFlag) -DskipCompile=true -DskipTestCompile=true -DcreateSourcesJar=false' + - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + TestName: 'Kafka_Cosmos_Integration' + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + - $(sub-config-cosmos-azure-cloud-test-resources) + MatrixConfigs: + - Name: Kafka_Cosmos_Integration_Test + Path: sdk/cosmos/kafka-integration-matrix.json + Selection: all + GenerateVMJobs: true + ServiceDirectory: cosmos + TestResourceDirectories: + - cosmos/ + Artifacts: + - name: azure-cosmos-kafka-connect + groupId: com.azure.cosmos.kafka + safeName: azurecosmoskafkaconnect + TimeoutInMinutes: 120 + PreSteps: + - template: /eng/pipelines/templates/steps/install-reporting-tools.yml + TestGoals: 'clean verify' + TestOptions: '$(ProfileFlag)' - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - TestName: 'Kafka_Cosmos_Integration' - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - - $(sub-config-cosmos-azure-cloud-test-resources) - MatrixConfigs: - - Name: Kafka_Cosmos_Integration_Test - Path: sdk/cosmos/kafka-integration-matrix.json - Selection: all - GenerateVMJobs: true - ServiceDirectory: cosmos - TestResourceDirectories: - - cosmos/ - Artifacts: - - name: azure-cosmos-kafka-connect - groupId: com.azure.cosmos.kafka - safeName: azurecosmoskafkaconnect - TimeoutInMinutes: 120 - PreSteps: - - template: /eng/pipelines/templates/steps/install-reporting-tools.yml - TestGoals: 'verify' - TestOptions: '$(ProfileFlag) -DskipCompile=true -DskipTestCompile=true -DcreateSourcesJar=false' diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/CHANGELOG.md b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/CHANGELOG.md index e412244ea3152..732bc4fc87e50 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/CHANGELOG.md +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.1 (Unreleased) +## 1.1.0-beta.2 (Unreleased) ### Features Added @@ -10,6 +10,65 @@ ### Other Changes +## 1.1.0-beta.1 (2024-03-15) + +- Azure Resource Manager CosmosDBForPostgreSql client library for Java. This package contains Microsoft Azure SDK for CosmosDBForPostgreSql Management SDK. Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. Package tag package-preview-2023-03. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Breaking Changes + +#### `models.Clusters` was modified + +* `promoteReadReplica(java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed + +#### `models.Cluster` was modified + +* `promoteReadReplica(com.azure.core.util.Context)` was removed + +### Features Added + +* `models.PrincipalType` was added + +* `models.RoleType` was added + +* `models.AuthConfig` was added + +* `models.PasswordAuth` was added + +* `models.PromoteRequest` was added + +* `models.ActiveDirectoryAuth` was added + +#### `models.Cluster$Definition` was modified + +* `withAuthConfig(models.AuthConfig)` was added +* `withEnableGeoBackup(java.lang.Boolean)` was added +* `withDatabaseName(java.lang.String)` was added + +#### `models.Clusters` was modified + +* `promoteReadReplica(java.lang.String,java.lang.String,models.PromoteRequest,com.azure.core.util.Context)` was added + +#### `models.Cluster` was modified + +* `promoteReadReplica(models.PromoteRequest,com.azure.core.util.Context)` was added +* `enableGeoBackup()` was added +* `databaseName()` was added +* `authConfig()` was added + +#### `models.Role$Definition` was modified + +* `withPrincipalType(models.PrincipalType)` was added +* `withObjectId(java.lang.String)` was added +* `withTenantId(java.lang.String)` was added +* `withRoleType(models.RoleType)` was added + +#### `models.Role` was modified + +* `principalType()` was added +* `roleType()` was added +* `objectId()` was added +* `tenantId()` was added + ## 1.0.0 (2023-09-22) - Azure Resource Manager CosmosDBForPostgreSql client library for Java. This package contains Microsoft Azure SDK for CosmosDBForPostgreSql Management SDK. Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. Package tag package-2022-11-08. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/README.md b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/README.md index 107256a1b76f5..d48f734800a89 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/README.md +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/README.md @@ -2,7 +2,7 @@ Azure Resource Manager CosmosDBForPostgreSql client library for Java. -This package contains Microsoft Azure SDK for CosmosDBForPostgreSql Management SDK. Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. Package tag package-2022-11-08. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for CosmosDBForPostgreSql Management SDK. Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. Package tag package-preview-2023-03. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-cosmosdbforpostgresql - 1.0.0 + 1.1.0-beta.1 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -74,6 +74,33 @@ See [API design][design] for general introduction on design and key concepts on ## Examples +```java +cluster = cosmosDBForPostgreSqlManager + .clusters() + .define(clusterName) + .withRegion(REGION) + .withExistingResourceGroup(resourceGroupName) + .withAdministratorLoginPassword(adminPwd) + .withPostgresqlVersion("15") + .withCitusVersion("12.1") + .withMaintenanceWindow(new MaintenanceWindow() + .withCustomWindow("Disabled") + .withDayOfWeek(0) + .withStartHour(0) + .withStartMinute(0)) + .withEnableShardsOnCoordinator(true) + .withEnableHa(false) + .withCoordinatorServerEdition("GeneralPurpose") + .withNodeServerEdition("MemoryOptimized") + .withCoordinatorStorageQuotaInMb(131072) + .withNodeStorageQuotaInMb(524288) + .withCoordinatorVCores(2) + .withNodeVCores(4) + .withCoordinatorEnablePublicIpAccess(true) + .withNodeEnablePublicIpAccess(true) + .withNodeCount(0) + .create(); +``` [Code snippets and samples](https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/SAMPLE.md) @@ -94,7 +121,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/SAMPLE.md b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/SAMPLE.md index d8bd27119f745..e09890835f7c9 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/SAMPLE.md +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/SAMPLE.md @@ -64,22 +64,24 @@ ```java import com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailabilityRequest; -/** Samples for Clusters CheckNameAvailability. */ +/** + * Samples for Clusters CheckNameAvailability. + */ public final class ClustersCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/CheckNameAvailability.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * CheckNameAvailability.json */ /** * Sample code: Check name availability. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void checkNameAvailability( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .checkNameAvailabilityWithResponse( - new NameAvailabilityRequest().withName("name1"), com.azure.core.util.Context.NONE); + public static void + checkNameAvailability(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.clusters().checkNameAvailabilityWithResponse(new NameAvailabilityRequest().withName("name1"), + com.azure.core.util.Context.NONE); } } ``` @@ -91,173 +93,124 @@ import java.time.OffsetDateTime; import java.util.HashMap; import java.util.Map; -/** Samples for Clusters Create. */ +/** + * Samples for Clusters Create. + */ public final class ClustersCreateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateSingleNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateSingleNode.json */ /** * Sample code: Create a new single node cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewSingleNodeCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-singlenode") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf("owner", "JohnDoe")) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.3") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(true) - .withEnableHa(true) - .withCoordinatorServerEdition("GeneralPurpose") - .withCoordinatorStorageQuotaInMb(131072) - .withCoordinatorVCores(8) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeCount(0) - .create(); + manager.clusters().define("testcluster-singlenode").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf("owner", "JohnDoe")).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.3").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(true) + .withEnableHa(true).withCoordinatorServerEdition("GeneralPurpose").withCoordinatorStorageQuotaInMb(131072) + .withCoordinatorVCores(8).withCoordinatorEnablePublicIpAccess(true).withNodeCount(0).create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateBurstablev1.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateBurstablev1.json */ /** * Sample code: Create a new single node Burstable 1 vCore cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewSingleNodeBurstable1VCoreCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-burstablev1") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf("owner", "JohnDoe")) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.3") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(true) - .withEnableHa(false) - .withCoordinatorServerEdition("BurstableMemoryOptimized") - .withCoordinatorStorageQuotaInMb(131072) - .withCoordinatorVCores(1) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeCount(0) - .create(); + manager.clusters().define("testcluster-burstablev1").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf("owner", "JohnDoe")).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.3").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(true) + .withEnableHa(false).withCoordinatorServerEdition("BurstableMemoryOptimized") + .withCoordinatorStorageQuotaInMb(131072).withCoordinatorVCores(1).withCoordinatorEnablePublicIpAccess(true) + .withNodeCount(0).create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateBurstablev2.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateBurstablev2.json */ /** * Sample code: Create a new single node Burstable 2 vCores cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewSingleNodeBurstable2VCoresCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-burstablev2") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf("owner", "JohnDoe")) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.3") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(true) - .withEnableHa(false) - .withCoordinatorServerEdition("BurstableGeneralPurpose") - .withCoordinatorStorageQuotaInMb(131072) - .withCoordinatorVCores(2) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeCount(0) - .create(); + manager.clusters().define("testcluster-burstablev2").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf("owner", "JohnDoe")).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.3").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(true) + .withEnableHa(false).withCoordinatorServerEdition("BurstableGeneralPurpose") + .withCoordinatorStorageQuotaInMb(131072).withCoordinatorVCores(2).withCoordinatorEnablePublicIpAccess(true) + .withNodeCount(0).create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateMultiNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateMultiNode.json */ /** * Sample code: Create a new multi-node cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewMultiNodeCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-multinode") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf()) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.1") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(false) - .withEnableHa(true) - .withCoordinatorServerEdition("GeneralPurpose") - .withCoordinatorStorageQuotaInMb(524288) - .withCoordinatorVCores(4) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeServerEdition("MemoryOptimized") - .withNodeCount(3) - .withNodeStorageQuotaInMb(524288) - .withNodeVCores(8) - .withNodeEnablePublicIpAccess(false) + manager.clusters().define("testcluster-multinode").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf()).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.1").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(false) + .withEnableHa(true).withCoordinatorServerEdition("GeneralPurpose").withCoordinatorStorageQuotaInMb(524288) + .withCoordinatorVCores(4).withCoordinatorEnablePublicIpAccess(true).withNodeServerEdition("MemoryOptimized") + .withNodeCount(3).withNodeStorageQuotaInMb(524288).withNodeVCores(8).withNodeEnablePublicIpAccess(false) .create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateReadReplica.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateReadReplica.json */ /** * Sample code: Create a new cluster as a read replica. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewClusterAsAReadReplica( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") + manager.clusters().define("testcluster").withRegion("westus").withExistingResourceGroup("TestGroup") .withSourceResourceId( "/subscriptions/ffffffff-ffff-ffff-ffff-ffffffffffff/resourceGroups/TestResourceGroup/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/sourcecluster") - .withSourceLocation("westus") - .create(); + .withSourceLocation("westus").create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreatePITR.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreatePITR.json */ /** * Sample code: Create a new cluster as a point in time restore. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewClusterAsAPointInTimeRestore( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") + manager.clusters().define("testcluster").withRegion("westus").withExistingResourceGroup("TestGroup") .withSourceResourceId( "/subscriptions/ffffffff-ffff-ffff-ffff-ffffffffffff/resourceGroups/TestResourceGroup/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/source-cluster") - .withSourceLocation("westus") - .withPointInTimeUtc(OffsetDateTime.parse("2017-12-14T00:00:37.467Z")) - .create(); + .withSourceLocation("westus").withPointInTimeUtc(OffsetDateTime.parse("2017-12-14T00:00:37.467Z")).create(); } // Use "Map.of" if available @@ -277,18 +230,22 @@ public final class ClustersCreateSamples { ### Clusters_Delete ```java -/** Samples for Clusters Delete. */ +/** + * Samples for Clusters Delete. + */ public final class ClustersDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterDelete.json */ /** * Sample code: Delete the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void deleteTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + deleteTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.clusters().delete("TestGroup", "testcluster", com.azure.core.util.Context.NONE); } } @@ -297,21 +254,24 @@ public final class ClustersDeleteSamples { ### Clusters_GetByResourceGroup ```java -/** Samples for Clusters GetByResourceGroup. */ +/** + * Samples for Clusters GetByResourceGroup. + */ public final class ClustersGetByResourceGroupSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterGet.json */ /** * Sample code: Get the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster1", com.azure.core.util.Context.NONE); + public static void + getTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.clusters().getByResourceGroupWithResponse("TestGroup", "testcluster1", + com.azure.core.util.Context.NONE); } } ``` @@ -319,18 +279,22 @@ public final class ClustersGetByResourceGroupSamples { ### Clusters_List ```java -/** Samples for Clusters List. */ +/** + * Samples for Clusters List. + */ public final class ClustersListSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterList.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterList.json */ /** * Sample code: List all the clusters. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void listAllTheClusters( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + listAllTheClusters(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.clusters().list(com.azure.core.util.Context.NONE); } } @@ -339,14 +303,18 @@ public final class ClustersListSamples { ### Clusters_ListByResourceGroup ```java -/** Samples for Clusters ListByResourceGroup. */ +/** + * Samples for Clusters ListByResourceGroup. + */ public final class ClustersListByResourceGroupSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterListByResourceGroup.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterListByResourceGroup.json */ /** * Sample code: List the clusters by resource group. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listTheClustersByResourceGroup( @@ -359,19 +327,24 @@ public final class ClustersListByResourceGroupSamples { ### Clusters_PromoteReadReplica ```java -/** Samples for Clusters PromoteReadReplica. */ + +/** + * Samples for Clusters PromoteReadReplica. + */ public final class ClustersPromoteReadReplicaSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterPromoteReadReplica.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterPromoteReadReplica.json */ /** * Sample code: Promote read replica cluster to an independent read-write cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void promoteReadReplicaClusterToAnIndependentReadWriteCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager.clusters().promoteReadReplica("TestGroup", "testcluster1", com.azure.core.util.Context.NONE); + manager.clusters().promoteReadReplica("TestGroup", "testcluster1", null, com.azure.core.util.Context.NONE); } } ``` @@ -379,14 +352,18 @@ public final class ClustersPromoteReadReplicaSamples { ### Clusters_Restart ```java -/** Samples for Clusters Restart. */ +/** + * Samples for Clusters Restart. + */ public final class ClustersRestartSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterRestart.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterRestart.json */ /** * Sample code: Restart all servers in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void restartAllServersInTheCluster( @@ -399,14 +376,18 @@ public final class ClustersRestartSamples { ### Clusters_Start ```java -/** Samples for Clusters Start. */ +/** + * Samples for Clusters Start. + */ public final class ClustersStartSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterStart.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterStart.json */ /** * Sample code: Start all servers in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void startAllServersInTheCluster( @@ -419,14 +400,18 @@ public final class ClustersStartSamples { ### Clusters_Stop ```java -/** Samples for Clusters Stop. */ +/** + * Samples for Clusters Stop. + */ public final class ClustersStopSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterStop.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterStop.json */ /** * Sample code: Stop all servers in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void stopAllServersInTheCluster( @@ -442,109 +427,95 @@ public final class ClustersStopSamples { import com.azure.resourcemanager.cosmosdbforpostgresql.models.Cluster; import com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow; -/** Samples for Clusters Update. */ +/** + * Samples for Clusters Update. + */ public final class ClustersUpdateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterScaleStorage.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterScaleStorage.json */ /** * Sample code: Scale up storage. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void scaleUpStorage( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); + public static void + scaleUpStorage(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); resource.update().withNodeStorageQuotaInMb(2097152).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterUpdate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterUpdate.json */ /** * Sample code: Update multiple configuration settings of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateMultipleConfigurationSettingsOfTheCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withAdministratorLoginPassword("newpassword") - .withCoordinatorVCores(16) - .withNodeCount(4) - .withNodeVCores(16) - .apply(); + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); + resource.update().withAdministratorLoginPassword("newpassword").withCoordinatorVCores(16).withNodeCount(4) + .withNodeVCores(16).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterScaleCompute.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterScaleCompute.json */ /** * Sample code: Scale compute up or down. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void scaleComputeUpOrDown( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); + public static void + scaleComputeUpOrDown(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); resource.update().withNodeVCores(16).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterAddNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterAddNode.json */ /** * Sample code: Scale out: Add new worker nodes. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void scaleOutAddNewWorkerNodes( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); resource.update().withNodeCount(2).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterUpdateMaintenanceWindow.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterUpdateMaintenanceWindow.json */ /** * Sample code: Update or define maintenance window. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateOrDefineMaintenanceWindow( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withMaintenanceWindow( - new MaintenanceWindow() - .withCustomWindow("Enabled") - .withStartHour(8) - .withStartMinute(0) - .withDayOfWeek(0)) + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); + resource.update().withMaintenanceWindow( + new MaintenanceWindow().withCustomWindow("Enabled").withStartHour(8).withStartMinute(0).withDayOfWeek(0)) .apply(); } } @@ -553,21 +524,24 @@ public final class ClustersUpdateSamples { ### Configurations_Get ```java -/** Samples for Configurations Get. */ +/** + * Samples for Configurations Get. + */ public final class ConfigurationsGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationGet.json */ /** * Sample code: Get configuration details. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getConfigurationDetails( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .getWithResponse("TestResourceGroup", "testcluster", "client_encoding", com.azure.core.util.Context.NONE); + public static void + getConfigurationDetails(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.configurations().getWithResponse("TestResourceGroup", "testcluster", "client_encoding", + com.azure.core.util.Context.NONE); } } ``` @@ -575,22 +549,24 @@ public final class ConfigurationsGetSamples { ### Configurations_GetCoordinator ```java -/** Samples for Configurations GetCoordinator. */ +/** + * Samples for Configurations GetCoordinator. + */ public final class ConfigurationsGetCoordinatorSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationGetCoordinator.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationGetCoordinator.json */ /** * Sample code: Get configuration details for coordinator. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getConfigurationDetailsForCoordinator( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .getCoordinatorWithResponse( - "TestResourceGroup", "testcluster", "array_nulls", com.azure.core.util.Context.NONE); + manager.configurations().getCoordinatorWithResponse("TestResourceGroup", "testcluster", "array_nulls", + com.azure.core.util.Context.NONE); } } ``` @@ -598,21 +574,24 @@ public final class ConfigurationsGetCoordinatorSamples { ### Configurations_GetNode ```java -/** Samples for Configurations GetNode. */ +/** + * Samples for Configurations GetNode. + */ public final class ConfigurationsGetNodeSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationGetNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationGetNode.json */ /** * Sample code: Get configuration details for node. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getConfigurationDetailsForNode( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .getNodeWithResponse("TestResourceGroup", "testcluster", "array_nulls", com.azure.core.util.Context.NONE); + manager.configurations().getNodeWithResponse("TestResourceGroup", "testcluster", "array_nulls", + com.azure.core.util.Context.NONE); } } ``` @@ -620,14 +599,18 @@ public final class ConfigurationsGetNodeSamples { ### Configurations_ListByCluster ```java -/** Samples for Configurations ListByCluster. */ +/** + * Samples for Configurations ListByCluster. + */ public final class ConfigurationsListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationListByCluster.json */ /** * Sample code: List configurations of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listConfigurationsOfTheCluster( @@ -640,21 +623,24 @@ public final class ConfigurationsListByClusterSamples { ### Configurations_ListByServer ```java -/** Samples for Configurations ListByServer. */ +/** + * Samples for Configurations ListByServer. + */ public final class ConfigurationsListByServerSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationListByServer.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationListByServer.json */ /** * Sample code: List configurations of the server that in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listConfigurationsOfTheServerThatInTheCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .listByServer("TestResourceGroup", "testcluster", "testserver", com.azure.core.util.Context.NONE); + manager.configurations().listByServer("TestResourceGroup", "testcluster", "testserver", + com.azure.core.util.Context.NONE); } } ``` @@ -664,26 +650,24 @@ public final class ConfigurationsListByServerSamples { ```java import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** Samples for Configurations UpdateOnCoordinator. */ +/** + * Samples for Configurations UpdateOnCoordinator. + */ public final class ConfigurationsUpdateOnCoordinatorSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationUpdateCoordinator.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationUpdateCoordinator.json */ /** * Sample code: Update single configuration of coordinator. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateSingleConfigurationOfCoordinator( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .updateOnCoordinator( - "TestResourceGroup", - "testcluster", - "array_nulls", - new ServerConfigurationInner().withValue("on"), - com.azure.core.util.Context.NONE); + manager.configurations().updateOnCoordinator("TestResourceGroup", "testcluster", "array_nulls", + new ServerConfigurationInner().withValue("on"), com.azure.core.util.Context.NONE); } } ``` @@ -693,26 +677,24 @@ public final class ConfigurationsUpdateOnCoordinatorSamples { ```java import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** Samples for Configurations UpdateOnNode. */ +/** + * Samples for Configurations UpdateOnNode. + */ public final class ConfigurationsUpdateOnNodeSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationUpdateNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationUpdateNode.json */ /** * Sample code: Update single configuration of nodes. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateSingleConfigurationOfNodes( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .updateOnNode( - "TestResourceGroup", - "testcluster", - "array_nulls", - new ServerConfigurationInner().withValue("off"), - com.azure.core.util.Context.NONE); + manager.configurations().updateOnNode("TestResourceGroup", "testcluster", "array_nulls", + new ServerConfigurationInner().withValue("off"), com.azure.core.util.Context.NONE); } } ``` @@ -720,25 +702,24 @@ public final class ConfigurationsUpdateOnNodeSamples { ### FirewallRules_CreateOrUpdate ```java -/** Samples for FirewallRules CreateOrUpdate. */ +/** + * Samples for FirewallRules CreateOrUpdate. + */ public final class FirewallRulesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleCreate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleCreate.json */ /** * Sample code: Create a firewall rule of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createAFirewallRuleOfTheCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .firewallRules() - .define("rule1") - .withExistingServerGroupsv2("TestGroup", "pgtestsvc4") - .withStartIpAddress("0.0.0.0") - .withEndIpAddress("255.255.255.255") - .create(); + manager.firewallRules().define("rule1").withExistingServerGroupsv2("TestGroup", "pgtestsvc4") + .withStartIpAddress("0.0.0.0").withEndIpAddress("255.255.255.255").create(); } } ``` @@ -746,14 +727,18 @@ public final class FirewallRulesCreateOrUpdateSamples { ### FirewallRules_Delete ```java -/** Samples for FirewallRules Delete. */ +/** + * Samples for FirewallRules Delete. + */ public final class FirewallRulesDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleDelete.json */ /** * Sample code: Delete the firewall rule of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void deleteTheFirewallRuleOfTheCluster( @@ -766,14 +751,18 @@ public final class FirewallRulesDeleteSamples { ### FirewallRules_Get ```java -/** Samples for FirewallRules Get. */ +/** + * Samples for FirewallRules Get. + */ public final class FirewallRulesGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleGet.json */ /** * Sample code: Get the firewall rule of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getTheFirewallRuleOfTheCluster( @@ -786,14 +775,18 @@ public final class FirewallRulesGetSamples { ### FirewallRules_ListByCluster ```java -/** Samples for FirewallRules ListByCluster. */ +/** + * Samples for FirewallRules ListByCluster. + */ public final class FirewallRulesListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleListByCluster.json */ /** * Sample code: List firewall rules of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listFirewallRulesOfTheCluster( @@ -806,14 +799,18 @@ public final class FirewallRulesListByClusterSamples { ### Operations_List ```java -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/OperationList.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * OperationList.json */ /** * Sample code: List all available operations. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listAllAvailableOperations( @@ -829,25 +826,26 @@ public final class OperationsListSamples { import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpointServiceConnectionStatus; import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateLinkServiceConnectionState; -/** Samples for PrivateEndpointConnections CreateOrUpdate. */ +/** + * Samples for PrivateEndpointConnections CreateOrUpdate. + */ public final class PrivateEndpointConnectionsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionCreateOrUpdate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionCreateOrUpdate.json */ /** * Sample code: Approves or Rejects a Private Endpoint Connection with a given name. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void approvesOrRejectsAPrivateEndpointConnectionWithAGivenName( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .define("private-endpoint-connection-name") + manager.privateEndpointConnections().define("private-endpoint-connection-name") .withExistingServerGroupsv2("TestGroup", "testcluster") .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) .withDescription("Approved by johndoe@contoso.com")) .create(); } @@ -857,21 +855,24 @@ public final class PrivateEndpointConnectionsCreateOrUpdateSamples { ### PrivateEndpointConnections_Delete ```java -/** Samples for PrivateEndpointConnections Delete. */ +/** + * Samples for PrivateEndpointConnections Delete. + */ public final class PrivateEndpointConnectionsDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionsDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionsDelete.json */ /** * Sample code: Deletes a private endpoint connection with a given name. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void deletesAPrivateEndpointConnectionWithAGivenName( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .delete("TestGroup", "testcluster", "private-endpoint-connection-name", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().delete("TestGroup", "testcluster", "private-endpoint-connection-name", + com.azure.core.util.Context.NONE); } } ``` @@ -879,22 +880,24 @@ public final class PrivateEndpointConnectionsDeleteSamples { ### PrivateEndpointConnections_Get ```java -/** Samples for PrivateEndpointConnections Get. */ +/** + * Samples for PrivateEndpointConnections Get. + */ public final class PrivateEndpointConnectionsGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionsGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionsGet.json */ /** * Sample code: Gets private endpoint connection. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsPrivateEndpointConnection( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .getWithResponse( - "TestGroup", "testcluster", "private-endpoint-connection-name", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().getWithResponse("TestGroup", "testcluster", + "private-endpoint-connection-name", com.azure.core.util.Context.NONE); } } ``` @@ -902,21 +905,24 @@ public final class PrivateEndpointConnectionsGetSamples { ### PrivateEndpointConnections_ListByCluster ```java -/** Samples for PrivateEndpointConnections ListByCluster. */ +/** + * Samples for PrivateEndpointConnections ListByCluster. + */ public final class PrivateEndpointConnectionsListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionsListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionsListByCluster.json */ /** * Sample code: Gets list of private endpoint connections on a cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsListOfPrivateEndpointConnectionsOnACluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .listByCluster("TestResourceGroup", "testcluster", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().listByCluster("TestResourceGroup", "testcluster", + com.azure.core.util.Context.NONE); } } ``` @@ -924,21 +930,24 @@ public final class PrivateEndpointConnectionsListByClusterSamples { ### PrivateLinkResources_Get ```java -/** Samples for PrivateLinkResources Get. */ +/** + * Samples for PrivateLinkResources Get. + */ public final class PrivateLinkResourcesGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateLinkResourcesGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateLinkResourcesGet.json */ /** * Sample code: Gets a private link resource for cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsAPrivateLinkResourceForCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateLinkResources() - .getWithResponse("TestGroup", "testcluster", "plr", com.azure.core.util.Context.NONE); + manager.privateLinkResources().getWithResponse("TestGroup", "testcluster", "plr", + com.azure.core.util.Context.NONE); } } ``` @@ -946,21 +955,24 @@ public final class PrivateLinkResourcesGetSamples { ### PrivateLinkResources_ListByCluster ```java -/** Samples for PrivateLinkResources ListByCluster. */ +/** + * Samples for PrivateLinkResources ListByCluster. + */ public final class PrivateLinkResourcesListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateLinkResourceListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateLinkResourceListByCluster.json */ /** * Sample code: Gets the private link resources for cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsThePrivateLinkResourcesForCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateLinkResources() - .listByCluster("TestResourceGroup", "testcluster", com.azure.core.util.Context.NONE); + manager.privateLinkResources().listByCluster("TestResourceGroup", "testcluster", + com.azure.core.util.Context.NONE); } } ``` @@ -968,23 +980,23 @@ public final class PrivateLinkResourcesListByClusterSamples { ### Roles_Create ```java -/** Samples for Roles Create. */ +/** + * Samples for Roles Create. + */ public final class RolesCreateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleCreate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleCreate.json */ /** * Sample code: RoleCreate. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void roleCreate( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .roles() - .define("role1") - .withExistingServerGroupsv2("TestGroup", "pgtestsvc4") - .withPassword("password") + public static void + roleCreate(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.roles().define("role1").withExistingServerGroupsv2("TestGroup", "pgtestsvc4").withPassword("password") .create(); } } @@ -993,18 +1005,22 @@ public final class RolesCreateSamples { ### Roles_Delete ```java -/** Samples for Roles Delete. */ +/** + * Samples for Roles Delete. + */ public final class RolesDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleDelete.json */ /** * Sample code: RoleDelete. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void roleDelete( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + roleDelete(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.roles().delete("TestGroup", "pgtestsvc4", "role1", com.azure.core.util.Context.NONE); } } @@ -1013,18 +1029,22 @@ public final class RolesDeleteSamples { ### Roles_Get ```java -/** Samples for Roles Get. */ +/** + * Samples for Roles Get. + */ public final class RolesGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleGet.json */ /** * Sample code: Get the role of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getTheRoleOfTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + getTheRoleOfTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.roles().getWithResponse("TestGroup", "pgtestsvc4", "role1", com.azure.core.util.Context.NONE); } } @@ -1033,14 +1053,18 @@ public final class RolesGetSamples { ### Roles_ListByCluster ```java -/** Samples for Roles ListByCluster. */ +/** + * Samples for Roles ListByCluster. + */ public final class RolesListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleListByCluster.json */ /** * Sample code: RoleList. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void roleList(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { @@ -1052,21 +1076,24 @@ public final class RolesListByClusterSamples { ### Servers_Get ```java -/** Samples for Servers Get. */ +/** + * Samples for Servers Get. + */ public final class ServersGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ServerGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ServerGet.json */ /** * Sample code: Get the server of cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getTheServerOfCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .servers() - .getWithResponse("TestGroup", "testcluster1", "testcluster1-c", com.azure.core.util.Context.NONE); + public static void + getTheServerOfCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.servers().getWithResponse("TestGroup", "testcluster1", "testcluster1-c", + com.azure.core.util.Context.NONE); } } ``` @@ -1074,18 +1101,22 @@ public final class ServersGetSamples { ### Servers_ListByCluster ```java -/** Samples for Servers ListByCluster. */ +/** + * Samples for Servers ListByCluster. + */ public final class ServersListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ServerListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ServerListByCluster.json */ /** * Sample code: List servers of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void listServersOfTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + listServersOfTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.servers().listByCluster("TestGroup", "testcluster1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/pom.xml b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/pom.xml index 8efa13cc11131..d173b35f16cd2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/pom.xml +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-cosmosdbforpostgresql - 1.1.0-beta.1 + 1.1.0-beta.2 jar Microsoft Azure SDK for CosmosDBForPostgreSql Management - This package contains Microsoft Azure SDK for CosmosDBForPostgreSql Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. Package tag package-2022-11-08. + This package contains Microsoft Azure SDK for CosmosDBForPostgreSql Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. Package tag package-preview-2023-03. https://github.com/Azure/azure-sdk-for-java @@ -45,6 +45,7 @@ UTF-8 0 0 + true @@ -66,13 +67,7 @@ com.azure azure-identity - 1.11.3 - test - - - com.azure.resourcemanager - azure-resourcemanager-resources - 2.36.0 + 1.11.4 test @@ -93,18 +88,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -113,5 +106,11 @@ 1.7.36 test + + com.azure.resourcemanager + azure-resourcemanager-resources + 2.37.0 + test + diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/CosmosDBForPostgreSqlManager.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/CosmosDBForPostgreSqlManager.java index fcd5c8e881012..3a0b8d4fd6c89 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/CosmosDBForPostgreSqlManager.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/CosmosDBForPostgreSqlManager.java @@ -49,8 +49,8 @@ import java.util.stream.Collectors; /** - * Entry point to CosmosDBForPostgreSqlManager. Azure Cosmos DB for PostgreSQL database service resource provider REST - * APIs. + * Entry point to CosmosDBForPostgreSqlManager. + * Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. */ public final class CosmosDBForPostgreSqlManager { private Clusters clusters; @@ -71,22 +71,18 @@ public final class CosmosDBForPostgreSqlManager { private final CosmosDBForPostgreSql clientObject; - private CosmosDBForPostgreSqlManager( - HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { + private CosmosDBForPostgreSqlManager(HttpPipeline httpPipeline, AzureProfile profile, + Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new CosmosDBForPostgreSqlBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new CosmosDBForPostgreSqlBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of CosmosDBForPostgreSql service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the CosmosDBForPostgreSql service API instance. @@ -99,7 +95,7 @@ public static CosmosDBForPostgreSqlManager authenticate(TokenCredential credenti /** * Creates an instance of CosmosDBForPostgreSql service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the CosmosDBForPostgreSql service API instance. @@ -112,14 +108,16 @@ public static CosmosDBForPostgreSqlManager authenticate(HttpPipeline httpPipelin /** * Gets a Configurable instance that can be used to create CosmosDBForPostgreSqlManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new CosmosDBForPostgreSqlManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -191,8 +189,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -209,8 +207,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -230,21 +228,12 @@ public CosmosDBForPostgreSqlManager authenticate(TokenCredential credential, Azu Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.cosmosdbforpostgresql") - .append("/") - .append("1.0.0"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.cosmosdbforpostgresql") + .append("/").append("1.1.0-beta.1"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -263,38 +252,25 @@ public CosmosDBForPostgreSqlManager authenticate(TokenCredential credential, Azu policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new CosmosDBForPostgreSqlManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of Clusters. It manages Cluster. - * + * * @return Resource collection API of Clusters. */ public Clusters clusters() { @@ -306,7 +282,7 @@ public Clusters clusters() { /** * Gets the resource collection API of Servers. - * + * * @return Resource collection API of Servers. */ public Servers servers() { @@ -318,7 +294,7 @@ public Servers servers() { /** * Gets the resource collection API of Configurations. - * + * * @return Resource collection API of Configurations. */ public Configurations configurations() { @@ -330,7 +306,7 @@ public Configurations configurations() { /** * Gets the resource collection API of FirewallRules. It manages FirewallRule. - * + * * @return Resource collection API of FirewallRules. */ public FirewallRules firewallRules() { @@ -342,7 +318,7 @@ public FirewallRules firewallRules() { /** * Gets the resource collection API of Roles. It manages Role. - * + * * @return Resource collection API of Roles. */ public Roles roles() { @@ -354,7 +330,7 @@ public Roles roles() { /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -366,20 +342,20 @@ public Operations operations() { /** * Gets the resource collection API of PrivateEndpointConnections. It manages PrivateEndpointConnection. - * + * * @return Resource collection API of PrivateEndpointConnections. */ public PrivateEndpointConnections privateEndpointConnections() { if (this.privateEndpointConnections == null) { - this.privateEndpointConnections = - new PrivateEndpointConnectionsImpl(clientObject.getPrivateEndpointConnections(), this); + this.privateEndpointConnections + = new PrivateEndpointConnectionsImpl(clientObject.getPrivateEndpointConnections(), this); } return privateEndpointConnections; } /** * Gets the resource collection API of PrivateLinkResources. - * + * * @return Resource collection API of PrivateLinkResources. */ public PrivateLinkResources privateLinkResources() { @@ -392,7 +368,7 @@ public PrivateLinkResources privateLinkResources() { /** * Gets wrapped service client CosmosDBForPostgreSql providing direct access to the underlying auto-generated API * implementation, based on Azure REST API. - * + * * @return Wrapped service client CosmosDBForPostgreSql. */ public CosmosDBForPostgreSql serviceClient() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ClustersClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ClustersClient.java index aa0aa2d81eef8..339dc28865c48 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ClustersClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ClustersClient.java @@ -15,12 +15,15 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.NameAvailabilityInner; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterForUpdate; import com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailabilityRequest; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PromoteRequest; -/** An instance of this class provides access to all the operations defined in ClustersClient. */ +/** + * An instance of this class provides access to all the operations defined in ClustersClient. + */ public interface ClustersClient { /** * Lists all clusters in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of clusters as paginated response with {@link PagedIterable}. @@ -30,7 +33,7 @@ public interface ClustersClient { /** * Lists all clusters in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -42,7 +45,7 @@ public interface ClustersClient { /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -54,7 +57,7 @@ public interface ClustersClient { /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -67,7 +70,7 @@ public interface ClustersClient { /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -77,12 +80,12 @@ public interface ClustersClient { * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters); + SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters); /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -93,12 +96,12 @@ SyncPoller, ClusterInner> beginCreate( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context); + SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters, Context context); /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -112,7 +115,7 @@ SyncPoller, ClusterInner> beginCreate( /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -128,7 +131,7 @@ SyncPoller, ClusterInner> beginCreate( /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -136,30 +139,30 @@ SyncPoller, ClusterInner> beginCreate( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response}. + * as cluster creation date and time along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String clusterName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, + Context context); /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time. + * as cluster creation date and time. */ @ServiceMethod(returns = ReturnType.SINGLE) ClusterInner getByResourceGroup(String resourceGroupName, String clusterName); /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -172,7 +175,7 @@ Response getByResourceGroupWithResponse( /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -186,7 +189,7 @@ Response getByResourceGroupWithResponse( /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -198,7 +201,7 @@ Response getByResourceGroupWithResponse( /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -211,7 +214,7 @@ Response getByResourceGroupWithResponse( /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -221,12 +224,12 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterForUpdate parameters); + SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterForUpdate parameters); /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -237,12 +240,12 @@ SyncPoller, ClusterInner> beginUpdate( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterForUpdate parameters, Context context); + SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterForUpdate parameters, Context context); /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -256,7 +259,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -271,7 +274,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -284,7 +287,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -298,7 +301,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -310,7 +313,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -323,7 +326,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -336,7 +339,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -350,7 +353,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -362,7 +365,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -375,7 +378,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -388,7 +391,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -402,7 +405,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -414,7 +417,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -427,7 +430,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -440,9 +443,10 @@ SyncPoller, ClusterInner> beginUpdate( /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -450,12 +454,12 @@ SyncPoller, ClusterInner> beginUpdate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginPromoteReadReplica( - String resourceGroupName, String clusterName, Context context); + SyncPoller, Void> beginPromoteReadReplica(String resourceGroupName, String clusterName, + PromoteRequest promoteRequest, Context context); /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -467,22 +471,24 @@ SyncPoller, Void> beginPromoteReadReplica( /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void promoteReadReplica(String resourceGroupName, String clusterName, Context context); + void promoteReadReplica(String resourceGroupName, String clusterName, PromoteRequest promoteRequest, + Context context); /** * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -491,14 +497,14 @@ SyncPoller, Void> beginPromoteReadReplica( * @return represents cluster name availability along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response checkNameAvailabilityWithResponse( - NameAvailabilityRequest nameAvailabilityRequest, Context context); + Response checkNameAvailabilityWithResponse(NameAvailabilityRequest nameAvailabilityRequest, + Context context); /** * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ConfigurationsClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ConfigurationsClient.java index 61100fdfc3b7b..9382cf8924d3b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ConfigurationsClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ConfigurationsClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ConfigurationInner; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** An instance of this class provides access to all the operations defined in ConfigurationsClient. */ +/** + * An instance of this class provides access to all the operations defined in ConfigurationsClient. + */ public interface ConfigurationsClient { /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -28,12 +30,12 @@ public interface ConfigurationsClient { * @return a list of server configurations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName); + PagedIterable listByServer(String resourceGroupName, String clusterName, + String serverName); /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -44,12 +46,12 @@ PagedIterable listByServer( * @return a list of server configurations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName, Context context); + PagedIterable listByServer(String resourceGroupName, String clusterName, + String serverName, Context context); /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -62,7 +64,7 @@ PagedIterable listByServer( /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -76,7 +78,7 @@ PagedIterable listByServer( /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -87,12 +89,12 @@ PagedIterable listByServer( * @return information of a configuration for coordinator and nodes along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String configurationName, + Context context); /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -106,7 +108,7 @@ Response getWithResponse( /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -117,12 +119,12 @@ Response getWithResponse( * @return information of a configuration for coordinator along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getCoordinatorWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context); + Response getCoordinatorWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context); /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -136,7 +138,7 @@ Response getCoordinatorWithResponse( /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -152,7 +154,7 @@ SyncPoller, ServerConfigurationInner> begin /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -165,15 +167,12 @@ SyncPoller, ServerConfigurationInner> begin */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, ServerConfigurationInner> beginUpdateOnCoordinator( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, + String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters, Context context); /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -184,12 +183,12 @@ SyncPoller, ServerConfigurationInner> begin * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerConfigurationInner updateOnCoordinator( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters); + ServerConfigurationInner updateOnCoordinator(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters); /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -201,16 +200,12 @@ ServerConfigurationInner updateOnCoordinator( * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerConfigurationInner updateOnCoordinator( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context); + ServerConfigurationInner updateOnCoordinator(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters, Context context); /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -221,12 +216,12 @@ ServerConfigurationInner updateOnCoordinator( * @return information of a configuration for worker nodes along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getNodeWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context); + Response getNodeWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context); /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -240,7 +235,7 @@ Response getNodeWithResponse( /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -256,7 +251,7 @@ SyncPoller, ServerConfigurationInner> begin /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -269,15 +264,12 @@ SyncPoller, ServerConfigurationInner> begin */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, ServerConfigurationInner> beginUpdateOnNode( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, + String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters, Context context); /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -288,12 +280,12 @@ SyncPoller, ServerConfigurationInner> begin * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerConfigurationInner updateOnNode( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters); + ServerConfigurationInner updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters); /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -305,10 +297,6 @@ ServerConfigurationInner updateOnNode( * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerConfigurationInner updateOnNode( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context); + ServerConfigurationInner updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters, Context context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/CosmosDBForPostgreSql.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/CosmosDBForPostgreSql.java index a879f1f5c86fa..389796e1efd1e 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/CosmosDBForPostgreSql.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/CosmosDBForPostgreSql.java @@ -7,95 +7,97 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for CosmosDBForPostgreSql class. */ +/** + * The interface for CosmosDBForPostgreSql class. + */ public interface CosmosDBForPostgreSql { /** * Gets The ID of the target subscription. The value must be an UUID. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the ClustersClient object to access its operations. - * + * * @return the ClustersClient object. */ ClustersClient getClusters(); /** * Gets the ServersClient object to access its operations. - * + * * @return the ServersClient object. */ ServersClient getServers(); /** * Gets the ConfigurationsClient object to access its operations. - * + * * @return the ConfigurationsClient object. */ ConfigurationsClient getConfigurations(); /** * Gets the FirewallRulesClient object to access its operations. - * + * * @return the FirewallRulesClient object. */ FirewallRulesClient getFirewallRules(); /** * Gets the RolesClient object to access its operations. - * + * * @return the RolesClient object. */ RolesClient getRoles(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the PrivateEndpointConnectionsClient object to access its operations. - * + * * @return the PrivateEndpointConnectionsClient object. */ PrivateEndpointConnectionsClient getPrivateEndpointConnections(); /** * Gets the PrivateLinkResourcesClient object to access its operations. - * + * * @return the PrivateLinkResourcesClient object. */ PrivateLinkResourcesClient getPrivateLinkResources(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/FirewallRulesClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/FirewallRulesClient.java index e9de7d679aaf0..791776360288d 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/FirewallRulesClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/FirewallRulesClient.java @@ -13,11 +13,13 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.FirewallRuleInner; -/** An instance of this class provides access to all the operations defined in FirewallRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in FirewallRulesClient. + */ public interface FirewallRulesClient { /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -28,12 +30,12 @@ public interface FirewallRulesClient { * @return the {@link SyncPoller} for polling of represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, FirewallRuleInner> beginCreateOrUpdate( - String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters); + SyncPoller, FirewallRuleInner> beginCreateOrUpdate(String resourceGroupName, + String clusterName, String firewallRuleName, FirewallRuleInner parameters); /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -45,16 +47,12 @@ SyncPoller, FirewallRuleInner> beginCreateOrUpdate * @return the {@link SyncPoller} for polling of represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, FirewallRuleInner> beginCreateOrUpdate( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, - Context context); + SyncPoller, FirewallRuleInner> beginCreateOrUpdate(String resourceGroupName, + String clusterName, String firewallRuleName, FirewallRuleInner parameters, Context context); /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -65,12 +63,12 @@ SyncPoller, FirewallRuleInner> beginCreateOrUpdate * @return represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.SINGLE) - FirewallRuleInner createOrUpdate( - String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters); + FirewallRuleInner createOrUpdate(String resourceGroupName, String clusterName, String firewallRuleName, + FirewallRuleInner parameters); /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -82,16 +80,12 @@ FirewallRuleInner createOrUpdate( * @return represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.SINGLE) - FirewallRuleInner createOrUpdate( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, - Context context); + FirewallRuleInner createOrUpdate(String resourceGroupName, String clusterName, String firewallRuleName, + FirewallRuleInner parameters, Context context); /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -101,12 +95,12 @@ FirewallRuleInner createOrUpdate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String firewallRuleName); + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String firewallRuleName); /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -117,12 +111,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String firewallRuleName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String firewallRuleName, Context context); /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -135,7 +129,7 @@ SyncPoller, Void> beginDelete( /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -149,7 +143,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -160,12 +154,12 @@ SyncPoller, Void> beginDelete( * @return information about a cluster firewall rule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String firewallRuleName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String firewallRuleName, + Context context); /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -179,7 +173,7 @@ Response getWithResponse( /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -192,7 +186,7 @@ Response getWithResponse( /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/OperationsClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/OperationsClient.java index 102f95afee1f1..a56b8ac543a0a 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/OperationsClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/OperationsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Lists all of the available REST API operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of resource provider operations as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface OperationsClient { /** * Lists all of the available REST API operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateEndpointConnectionsClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateEndpointConnectionsClient.java index 25de775f79cb0..0a680045e803b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateEndpointConnectionsClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateEndpointConnectionsClient.java @@ -13,11 +13,13 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateEndpointConnectionInner; -/** An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. + */ public interface PrivateEndpointConnectionsClient { /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -30,7 +32,7 @@ public interface PrivateEndpointConnectionsClient { /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -40,12 +42,12 @@ public interface PrivateEndpointConnectionsClient { * @return list of private endpoint connections on a cluster as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context); + PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context); /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -56,12 +58,12 @@ PagedIterable listByCluster( * @return private endpoint connection along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context); /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -71,12 +73,12 @@ Response getWithResponse( * @return private endpoint connection. */ @ServiceMethod(returns = ReturnType.SINGLE) - PrivateEndpointConnectionInner get( - String resourceGroupName, String clusterName, String privateEndpointConnectionName); + PrivateEndpointConnectionInner get(String resourceGroupName, String clusterName, + String privateEndpointConnectionName); /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -88,14 +90,12 @@ PrivateEndpointConnectionInner get( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PrivateEndpointConnectionInner> beginCreateOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, + String resourceGroupName, String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters); /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -108,15 +108,12 @@ SyncPoller, PrivateEndpointConnection */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PrivateEndpointConnectionInner> beginCreateOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, - Context context); + String resourceGroupName, String clusterName, String privateEndpointConnectionName, + PrivateEndpointConnectionInner parameters, Context context); /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -127,15 +124,12 @@ SyncPoller, PrivateEndpointConnection * @return the private endpoint connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PrivateEndpointConnectionInner createOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters); + PrivateEndpointConnectionInner createOrUpdate(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters); /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -147,16 +141,12 @@ PrivateEndpointConnectionInner createOrUpdate( * @return the private endpoint connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PrivateEndpointConnectionInner createOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, - Context context); + PrivateEndpointConnectionInner createOrUpdate(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters, Context context); /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -166,12 +156,12 @@ PrivateEndpointConnectionInner createOrUpdate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String privateEndpointConnectionName); + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName); /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -182,12 +172,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context); /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -200,7 +190,7 @@ SyncPoller, Void> beginDelete( /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateLinkResourcesClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateLinkResourcesClient.java index 233052a63481f..bf025bf1132a6 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateLinkResourcesClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/PrivateLinkResourcesClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateLinkResourceInner; -/** An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. + */ public interface PrivateLinkResourcesClient { /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -28,7 +30,7 @@ public interface PrivateLinkResourcesClient { /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -38,12 +40,12 @@ public interface PrivateLinkResourcesClient { * @return the private link resources for cluster as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context); + PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context); /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. @@ -54,12 +56,12 @@ PagedIterable listByCluster( * @return a private link resource for cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String privateLinkResourceName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, + String privateLinkResourceName, Context context); /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/RolesClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/RolesClient.java index de639c8f6362a..14fda54e9afd7 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/RolesClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/RolesClient.java @@ -13,11 +13,13 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RoleInner; -/** An instance of this class provides access to all the operations defined in RolesClient. */ +/** + * An instance of this class provides access to all the operations defined in RolesClient. + */ public interface RolesClient { /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -32,7 +34,7 @@ public interface RolesClient { /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -46,7 +48,7 @@ public interface RolesClient { /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -57,12 +59,12 @@ public interface RolesClient { * @return the {@link SyncPoller} for polling of represents a cluster role. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, RoleInner> beginCreate( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters); + SyncPoller, RoleInner> beginCreate(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters); /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -74,12 +76,12 @@ SyncPoller, RoleInner> beginCreate( * @return the {@link SyncPoller} for polling of represents a cluster role. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, RoleInner> beginCreate( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context); + SyncPoller, RoleInner> beginCreate(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters, Context context); /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -94,7 +96,7 @@ SyncPoller, RoleInner> beginCreate( /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -106,12 +108,12 @@ SyncPoller, RoleInner> beginCreate( * @return represents a cluster role. */ @ServiceMethod(returns = ReturnType.SINGLE) - RoleInner create( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context); + RoleInner create(String resourceGroupName, String clusterName, String roleName, RoleInner parameters, + Context context); /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -125,7 +127,7 @@ RoleInner create( /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -136,12 +138,12 @@ RoleInner create( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String roleName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, String roleName, + Context context); /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -154,7 +156,7 @@ SyncPoller, Void> beginDelete( /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -168,7 +170,7 @@ SyncPoller, Void> beginDelete( /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -181,7 +183,7 @@ SyncPoller, Void> beginDelete( /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ServersClient.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ServersClient.java index 7a4a96fdc9a2a..42d27f49b4f72 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ServersClient.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/ServersClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterServerInner; -/** An instance of this class provides access to all the operations defined in ServersClient. */ +/** + * An instance of this class provides access to all the operations defined in ServersClient. + */ public interface ServersClient { /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -28,7 +30,7 @@ public interface ServersClient { /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -42,7 +44,7 @@ public interface ServersClient { /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -53,12 +55,12 @@ public interface ServersClient { * @return information about a server in cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String serverName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String serverName, + Context context); /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterInner.java index 798a7869f1cd1..7fc9612e4c221 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterInner.java @@ -7,6 +7,7 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; import com.azure.core.management.SystemData; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.AuthConfig; import com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerNameItem; import com.azure.resourcemanager.cosmosdbforpostgresql.models.SimplePrivateEndpointConnection; @@ -15,7 +16,9 @@ import java.util.List; import java.util.Map; -/** Represents a cluster. */ +/** + * Represents a cluster. + */ @Fluent public final class ClusterInner extends Resource { /* @@ -30,13 +33,15 @@ public final class ClusterInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ClusterInner class. */ + /** + * Creates an instance of ClusterInner class. + */ public ClusterInner() { } /** * Get the innerProperties property: Properties of the cluster. - * + * * @return the innerProperties value. */ private ClusterProperties innerProperties() { @@ -45,21 +50,25 @@ private ClusterProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterInner withTags(Map tags) { super.withTags(tags); @@ -68,7 +77,7 @@ public ClusterInner withTags(Map tags) { /** * Get the administratorLogin property: The administrator's login name of the servers in the cluster. - * + * * @return the administratorLogin value. */ public String administratorLogin() { @@ -77,7 +86,7 @@ public String administratorLogin() { /** * Get the administratorLoginPassword property: The password of the administrator login. Required for creation. - * + * * @return the administratorLoginPassword value. */ public String administratorLoginPassword() { @@ -86,7 +95,7 @@ public String administratorLoginPassword() { /** * Set the administratorLoginPassword property: The password of the administrator login. Required for creation. - * + * * @param administratorLoginPassword the administratorLoginPassword value to set. * @return the ClusterInner object itself. */ @@ -100,7 +109,7 @@ public ClusterInner withAdministratorLoginPassword(String administratorLoginPass /** * Get the provisioningState property: Provisioning state of the cluster. - * + * * @return the provisioningState value. */ public String provisioningState() { @@ -109,7 +118,7 @@ public String provisioningState() { /** * Get the state property: A state of a cluster/server that is visible to user. - * + * * @return the state value. */ public String state() { @@ -118,7 +127,7 @@ public String state() { /** * Get the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @return the postgresqlVersion value. */ public String postgresqlVersion() { @@ -127,7 +136,7 @@ public String postgresqlVersion() { /** * Set the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @param postgresqlVersion the postgresqlVersion value to set. * @return the ClusterInner object itself. */ @@ -141,7 +150,7 @@ public ClusterInner withPostgresqlVersion(String postgresqlVersion) { /** * Get the citusVersion property: The Citus extension version on all cluster servers. - * + * * @return the citusVersion value. */ public String citusVersion() { @@ -150,7 +159,7 @@ public String citusVersion() { /** * Set the citusVersion property: The Citus extension version on all cluster servers. - * + * * @param citusVersion the citusVersion value to set. * @return the ClusterInner object itself. */ @@ -164,7 +173,7 @@ public ClusterInner withCitusVersion(String citusVersion) { /** * Get the maintenanceWindow property: Maintenance window of a cluster. - * + * * @return the maintenanceWindow value. */ public MaintenanceWindow maintenanceWindow() { @@ -173,7 +182,7 @@ public MaintenanceWindow maintenanceWindow() { /** * Set the maintenanceWindow property: Maintenance window of a cluster. - * + * * @param maintenanceWindow the maintenanceWindow value to set. * @return the ClusterInner object itself. */ @@ -187,7 +196,7 @@ public ClusterInner withMaintenanceWindow(MaintenanceWindow maintenanceWindow) { /** * Get the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @return the preferredPrimaryZone value. */ public String preferredPrimaryZone() { @@ -196,7 +205,7 @@ public String preferredPrimaryZone() { /** * Set the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @param preferredPrimaryZone the preferredPrimaryZone value to set. * @return the ClusterInner object itself. */ @@ -209,9 +218,9 @@ public ClusterInner withPreferredPrimaryZone(String preferredPrimaryZone) { } /** - * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @return the enableShardsOnCoordinator value. */ public Boolean enableShardsOnCoordinator() { @@ -219,9 +228,9 @@ public Boolean enableShardsOnCoordinator() { } /** - * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @param enableShardsOnCoordinator the enableShardsOnCoordinator value to set. * @return the ClusterInner object itself. */ @@ -235,7 +244,7 @@ public ClusterInner withEnableShardsOnCoordinator(Boolean enableShardsOnCoordina /** * Get the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @return the enableHa value. */ public Boolean enableHa() { @@ -244,7 +253,7 @@ public Boolean enableHa() { /** * Set the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @param enableHa the enableHa value to set. * @return the ClusterInner object itself. */ @@ -259,7 +268,7 @@ public ClusterInner withEnableHa(Boolean enableHa) { /** * Get the coordinatorServerEdition property: The edition of a coordinator server (default: GeneralPurpose). * Required for creation. - * + * * @return the coordinatorServerEdition value. */ public String coordinatorServerEdition() { @@ -269,7 +278,7 @@ public String coordinatorServerEdition() { /** * Set the coordinatorServerEdition property: The edition of a coordinator server (default: GeneralPurpose). * Required for creation. - * + * * @param coordinatorServerEdition the coordinatorServerEdition value to set. * @return the ClusterInner object itself. */ @@ -284,7 +293,7 @@ public ClusterInner withCoordinatorServerEdition(String coordinatorServerEdition /** * Get the coordinatorStorageQuotaInMb property: The storage of a server in MB. Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the coordinatorStorageQuotaInMb value. */ public Integer coordinatorStorageQuotaInMb() { @@ -294,7 +303,7 @@ public Integer coordinatorStorageQuotaInMb() { /** * Set the coordinatorStorageQuotaInMb property: The storage of a server in MB. Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param coordinatorStorageQuotaInMb the coordinatorStorageQuotaInMb value to set. * @return the ClusterInner object itself. */ @@ -309,7 +318,7 @@ public ClusterInner withCoordinatorStorageQuotaInMb(Integer coordinatorStorageQu /** * Get the coordinatorVCores property: The vCores count of a server (max: 96). Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the coordinatorVCores value. */ public Integer coordinatorVCores() { @@ -319,7 +328,7 @@ public Integer coordinatorVCores() { /** * Set the coordinatorVCores property: The vCores count of a server (max: 96). Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param coordinatorVCores the coordinatorVCores value to set. * @return the ClusterInner object itself. */ @@ -333,7 +342,7 @@ public ClusterInner withCoordinatorVCores(Integer coordinatorVCores) { /** * Get the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @return the coordinatorEnablePublicIpAccess value. */ public Boolean coordinatorEnablePublicIpAccess() { @@ -342,7 +351,7 @@ public Boolean coordinatorEnablePublicIpAccess() { /** * Set the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @param coordinatorEnablePublicIpAccess the coordinatorEnablePublicIpAccess value to set. * @return the ClusterInner object itself. */ @@ -356,7 +365,7 @@ public ClusterInner withCoordinatorEnablePublicIpAccess(Boolean coordinatorEnabl /** * Get the nodeServerEdition property: The edition of a node server (default: MemoryOptimized). - * + * * @return the nodeServerEdition value. */ public String nodeServerEdition() { @@ -365,7 +374,7 @@ public String nodeServerEdition() { /** * Set the nodeServerEdition property: The edition of a node server (default: MemoryOptimized). - * + * * @param nodeServerEdition the nodeServerEdition value to set. * @return the ClusterInner object itself. */ @@ -381,7 +390,7 @@ public ClusterInner withNodeServerEdition(String nodeServerEdition) { * Get the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. Required for creation. - * + * * @return the nodeCount value. */ public Integer nodeCount() { @@ -392,7 +401,7 @@ public Integer nodeCount() { * Set the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. Required for creation. - * + * * @param nodeCount the nodeCount value to set. * @return the ClusterInner object itself. */ @@ -407,7 +416,7 @@ public ClusterInner withNodeCount(Integer nodeCount) { /** * Get the nodeStorageQuotaInMb property: The storage in MB on each worker node. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the nodeStorageQuotaInMb value. */ public Integer nodeStorageQuotaInMb() { @@ -417,7 +426,7 @@ public Integer nodeStorageQuotaInMb() { /** * Set the nodeStorageQuotaInMb property: The storage in MB on each worker node. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param nodeStorageQuotaInMb the nodeStorageQuotaInMb value to set. * @return the ClusterInner object itself. */ @@ -432,7 +441,7 @@ public ClusterInner withNodeStorageQuotaInMb(Integer nodeStorageQuotaInMb) { /** * Get the nodeVCores property: The compute in vCores on each worker node (max: 104). See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the nodeVCores value. */ public Integer nodeVCores() { @@ -442,7 +451,7 @@ public Integer nodeVCores() { /** * Set the nodeVCores property: The compute in vCores on each worker node (max: 104). See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param nodeVCores the nodeVCores value to set. * @return the ClusterInner object itself. */ @@ -456,7 +465,7 @@ public ClusterInner withNodeVCores(Integer nodeVCores) { /** * Get the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @return the nodeEnablePublicIpAccess value. */ public Boolean nodeEnablePublicIpAccess() { @@ -465,7 +474,7 @@ public Boolean nodeEnablePublicIpAccess() { /** * Set the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @param nodeEnablePublicIpAccess the nodeEnablePublicIpAccess value to set. * @return the ClusterInner object itself. */ @@ -479,7 +488,7 @@ public ClusterInner withNodeEnablePublicIpAccess(Boolean nodeEnablePublicIpAcces /** * Get the serverNames property: The list of server names in the cluster. - * + * * @return the serverNames value. */ public List serverNames() { @@ -488,7 +497,7 @@ public List serverNames() { /** * Get the sourceResourceId property: The resource id of source cluster for read replica clusters. - * + * * @return the sourceResourceId value. */ public String sourceResourceId() { @@ -497,7 +506,7 @@ public String sourceResourceId() { /** * Set the sourceResourceId property: The resource id of source cluster for read replica clusters. - * + * * @param sourceResourceId the sourceResourceId value to set. * @return the ClusterInner object itself. */ @@ -511,7 +520,7 @@ public ClusterInner withSourceResourceId(String sourceResourceId) { /** * Get the sourceLocation property: The Azure region of source cluster for read replica clusters. - * + * * @return the sourceLocation value. */ public String sourceLocation() { @@ -520,7 +529,7 @@ public String sourceLocation() { /** * Set the sourceLocation property: The Azure region of source cluster for read replica clusters. - * + * * @param sourceLocation the sourceLocation value to set. * @return the ClusterInner object itself. */ @@ -534,7 +543,7 @@ public ClusterInner withSourceLocation(String sourceLocation) { /** * Get the pointInTimeUtc property: Date and time in UTC (ISO8601 format) for cluster restore. - * + * * @return the pointInTimeUtc value. */ public OffsetDateTime pointInTimeUtc() { @@ -543,7 +552,7 @@ public OffsetDateTime pointInTimeUtc() { /** * Set the pointInTimeUtc property: Date and time in UTC (ISO8601 format) for cluster restore. - * + * * @param pointInTimeUtc the pointInTimeUtc value to set. * @return the ClusterInner object itself. */ @@ -557,7 +566,7 @@ public ClusterInner withPointInTimeUtc(OffsetDateTime pointInTimeUtc) { /** * Get the readReplicas property: The array of read replica clusters. - * + * * @return the readReplicas value. */ public List readReplicas() { @@ -566,7 +575,7 @@ public List readReplicas() { /** * Get the earliestRestoreTime property: The earliest restore point time (ISO8601 format) for the cluster. - * + * * @return the earliestRestoreTime value. */ public OffsetDateTime earliestRestoreTime() { @@ -575,16 +584,87 @@ public OffsetDateTime earliestRestoreTime() { /** * Get the privateEndpointConnections property: The private endpoint connections for a cluster. - * + * * @return the privateEndpointConnections value. */ public List privateEndpointConnections() { return this.innerProperties() == null ? null : this.innerProperties().privateEndpointConnections(); } + /** + * Get the databaseName property: The database name of the cluster. Only one database per cluster is supported. + * + * @return the databaseName value. + */ + public String databaseName() { + return this.innerProperties() == null ? null : this.innerProperties().databaseName(); + } + + /** + * Set the databaseName property: The database name of the cluster. Only one database per cluster is supported. + * + * @param databaseName the databaseName value to set. + * @return the ClusterInner object itself. + */ + public ClusterInner withDatabaseName(String databaseName) { + if (this.innerProperties() == null) { + this.innerProperties = new ClusterProperties(); + } + this.innerProperties().withDatabaseName(databaseName); + return this; + } + + /** + * Get the enableGeoBackup property: If cluster backup is stored in another Azure region in addition to the copy of + * the backup stored in the cluster's region. Enabled only at the time of cluster creation. + * + * @return the enableGeoBackup value. + */ + public Boolean enableGeoBackup() { + return this.innerProperties() == null ? null : this.innerProperties().enableGeoBackup(); + } + + /** + * Set the enableGeoBackup property: If cluster backup is stored in another Azure region in addition to the copy of + * the backup stored in the cluster's region. Enabled only at the time of cluster creation. + * + * @param enableGeoBackup the enableGeoBackup value to set. + * @return the ClusterInner object itself. + */ + public ClusterInner withEnableGeoBackup(Boolean enableGeoBackup) { + if (this.innerProperties() == null) { + this.innerProperties = new ClusterProperties(); + } + this.innerProperties().withEnableGeoBackup(enableGeoBackup); + return this; + } + + /** + * Get the authConfig property: Authentication configuration of a cluster. + * + * @return the authConfig value. + */ + public AuthConfig authConfig() { + return this.innerProperties() == null ? null : this.innerProperties().authConfig(); + } + + /** + * Set the authConfig property: Authentication configuration of a cluster. + * + * @param authConfig the authConfig value to set. + * @return the ClusterInner object itself. + */ + public ClusterInner withAuthConfig(AuthConfig authConfig) { + if (this.innerProperties() == null) { + this.innerProperties = new ClusterProperties(); + } + this.innerProperties().withAuthConfig(authConfig); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterProperties.java index 22d77a68f0f4a..6955296cfb4b8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterProperties.java @@ -5,6 +5,7 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models; import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.AuthConfig; import com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerNameItem; import com.azure.resourcemanager.cosmosdbforpostgresql.models.SimplePrivateEndpointConnection; @@ -12,7 +13,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Properties of the cluster. */ +/** + * Properties of the cluster. + */ @Fluent public final class ClusterProperties { /* @@ -178,13 +181,34 @@ public final class ClusterProperties { @JsonProperty(value = "privateEndpointConnections", access = JsonProperty.Access.WRITE_ONLY) private List privateEndpointConnections; - /** Creates an instance of ClusterProperties class. */ + /* + * The database name of the cluster. Only one database per cluster is supported. + */ + @JsonProperty(value = "databaseName") + private String databaseName; + + /* + * If cluster backup is stored in another Azure region in addition to the copy of the backup stored in the + * cluster's region. Enabled only at the time of cluster creation. + */ + @JsonProperty(value = "enableGeoBackup") + private Boolean enableGeoBackup; + + /* + * Authentication configuration of a cluster. + */ + @JsonProperty(value = "authConfig") + private AuthConfig authConfig; + + /** + * Creates an instance of ClusterProperties class. + */ public ClusterProperties() { } /** * Get the administratorLogin property: The administrator's login name of the servers in the cluster. - * + * * @return the administratorLogin value. */ public String administratorLogin() { @@ -193,7 +217,7 @@ public String administratorLogin() { /** * Get the administratorLoginPassword property: The password of the administrator login. Required for creation. - * + * * @return the administratorLoginPassword value. */ public String administratorLoginPassword() { @@ -202,7 +226,7 @@ public String administratorLoginPassword() { /** * Set the administratorLoginPassword property: The password of the administrator login. Required for creation. - * + * * @param administratorLoginPassword the administratorLoginPassword value to set. * @return the ClusterProperties object itself. */ @@ -213,7 +237,7 @@ public ClusterProperties withAdministratorLoginPassword(String administratorLogi /** * Get the provisioningState property: Provisioning state of the cluster. - * + * * @return the provisioningState value. */ public String provisioningState() { @@ -222,7 +246,7 @@ public String provisioningState() { /** * Get the state property: A state of a cluster/server that is visible to user. - * + * * @return the state value. */ public String state() { @@ -231,7 +255,7 @@ public String state() { /** * Get the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @return the postgresqlVersion value. */ public String postgresqlVersion() { @@ -240,7 +264,7 @@ public String postgresqlVersion() { /** * Set the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @param postgresqlVersion the postgresqlVersion value to set. * @return the ClusterProperties object itself. */ @@ -251,7 +275,7 @@ public ClusterProperties withPostgresqlVersion(String postgresqlVersion) { /** * Get the citusVersion property: The Citus extension version on all cluster servers. - * + * * @return the citusVersion value. */ public String citusVersion() { @@ -260,7 +284,7 @@ public String citusVersion() { /** * Set the citusVersion property: The Citus extension version on all cluster servers. - * + * * @param citusVersion the citusVersion value to set. * @return the ClusterProperties object itself. */ @@ -271,7 +295,7 @@ public ClusterProperties withCitusVersion(String citusVersion) { /** * Get the maintenanceWindow property: Maintenance window of a cluster. - * + * * @return the maintenanceWindow value. */ public MaintenanceWindow maintenanceWindow() { @@ -280,7 +304,7 @@ public MaintenanceWindow maintenanceWindow() { /** * Set the maintenanceWindow property: Maintenance window of a cluster. - * + * * @param maintenanceWindow the maintenanceWindow value to set. * @return the ClusterProperties object itself. */ @@ -291,7 +315,7 @@ public ClusterProperties withMaintenanceWindow(MaintenanceWindow maintenanceWind /** * Get the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @return the preferredPrimaryZone value. */ public String preferredPrimaryZone() { @@ -300,7 +324,7 @@ public String preferredPrimaryZone() { /** * Set the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @param preferredPrimaryZone the preferredPrimaryZone value to set. * @return the ClusterProperties object itself. */ @@ -310,9 +334,9 @@ public ClusterProperties withPreferredPrimaryZone(String preferredPrimaryZone) { } /** - * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @return the enableShardsOnCoordinator value. */ public Boolean enableShardsOnCoordinator() { @@ -320,9 +344,9 @@ public Boolean enableShardsOnCoordinator() { } /** - * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @param enableShardsOnCoordinator the enableShardsOnCoordinator value to set. * @return the ClusterProperties object itself. */ @@ -333,7 +357,7 @@ public ClusterProperties withEnableShardsOnCoordinator(Boolean enableShardsOnCoo /** * Get the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @return the enableHa value. */ public Boolean enableHa() { @@ -342,7 +366,7 @@ public Boolean enableHa() { /** * Set the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @param enableHa the enableHa value to set. * @return the ClusterProperties object itself. */ @@ -354,7 +378,7 @@ public ClusterProperties withEnableHa(Boolean enableHa) { /** * Get the coordinatorServerEdition property: The edition of a coordinator server (default: GeneralPurpose). * Required for creation. - * + * * @return the coordinatorServerEdition value. */ public String coordinatorServerEdition() { @@ -364,7 +388,7 @@ public String coordinatorServerEdition() { /** * Set the coordinatorServerEdition property: The edition of a coordinator server (default: GeneralPurpose). * Required for creation. - * + * * @param coordinatorServerEdition the coordinatorServerEdition value to set. * @return the ClusterProperties object itself. */ @@ -376,7 +400,7 @@ public ClusterProperties withCoordinatorServerEdition(String coordinatorServerEd /** * Get the coordinatorStorageQuotaInMb property: The storage of a server in MB. Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the coordinatorStorageQuotaInMb value. */ public Integer coordinatorStorageQuotaInMb() { @@ -386,7 +410,7 @@ public Integer coordinatorStorageQuotaInMb() { /** * Set the coordinatorStorageQuotaInMb property: The storage of a server in MB. Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param coordinatorStorageQuotaInMb the coordinatorStorageQuotaInMb value to set. * @return the ClusterProperties object itself. */ @@ -398,7 +422,7 @@ public ClusterProperties withCoordinatorStorageQuotaInMb(Integer coordinatorStor /** * Get the coordinatorVCores property: The vCores count of a server (max: 96). Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the coordinatorVCores value. */ public Integer coordinatorVCores() { @@ -408,7 +432,7 @@ public Integer coordinatorVCores() { /** * Set the coordinatorVCores property: The vCores count of a server (max: 96). Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param coordinatorVCores the coordinatorVCores value to set. * @return the ClusterProperties object itself. */ @@ -419,7 +443,7 @@ public ClusterProperties withCoordinatorVCores(Integer coordinatorVCores) { /** * Get the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @return the coordinatorEnablePublicIpAccess value. */ public Boolean coordinatorEnablePublicIpAccess() { @@ -428,7 +452,7 @@ public Boolean coordinatorEnablePublicIpAccess() { /** * Set the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @param coordinatorEnablePublicIpAccess the coordinatorEnablePublicIpAccess value to set. * @return the ClusterProperties object itself. */ @@ -439,7 +463,7 @@ public ClusterProperties withCoordinatorEnablePublicIpAccess(Boolean coordinator /** * Get the nodeServerEdition property: The edition of a node server (default: MemoryOptimized). - * + * * @return the nodeServerEdition value. */ public String nodeServerEdition() { @@ -448,7 +472,7 @@ public String nodeServerEdition() { /** * Set the nodeServerEdition property: The edition of a node server (default: MemoryOptimized). - * + * * @param nodeServerEdition the nodeServerEdition value to set. * @return the ClusterProperties object itself. */ @@ -461,7 +485,7 @@ public ClusterProperties withNodeServerEdition(String nodeServerEdition) { * Get the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. Required for creation. - * + * * @return the nodeCount value. */ public Integer nodeCount() { @@ -472,7 +496,7 @@ public Integer nodeCount() { * Set the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. Required for creation. - * + * * @param nodeCount the nodeCount value to set. * @return the ClusterProperties object itself. */ @@ -484,7 +508,7 @@ public ClusterProperties withNodeCount(Integer nodeCount) { /** * Get the nodeStorageQuotaInMb property: The storage in MB on each worker node. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the nodeStorageQuotaInMb value. */ public Integer nodeStorageQuotaInMb() { @@ -494,7 +518,7 @@ public Integer nodeStorageQuotaInMb() { /** * Set the nodeStorageQuotaInMb property: The storage in MB on each worker node. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param nodeStorageQuotaInMb the nodeStorageQuotaInMb value to set. * @return the ClusterProperties object itself. */ @@ -506,7 +530,7 @@ public ClusterProperties withNodeStorageQuotaInMb(Integer nodeStorageQuotaInMb) /** * Get the nodeVCores property: The compute in vCores on each worker node (max: 104). See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the nodeVCores value. */ public Integer nodeVCores() { @@ -516,7 +540,7 @@ public Integer nodeVCores() { /** * Set the nodeVCores property: The compute in vCores on each worker node (max: 104). See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @param nodeVCores the nodeVCores value to set. * @return the ClusterProperties object itself. */ @@ -527,7 +551,7 @@ public ClusterProperties withNodeVCores(Integer nodeVCores) { /** * Get the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @return the nodeEnablePublicIpAccess value. */ public Boolean nodeEnablePublicIpAccess() { @@ -536,7 +560,7 @@ public Boolean nodeEnablePublicIpAccess() { /** * Set the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @param nodeEnablePublicIpAccess the nodeEnablePublicIpAccess value to set. * @return the ClusterProperties object itself. */ @@ -547,7 +571,7 @@ public ClusterProperties withNodeEnablePublicIpAccess(Boolean nodeEnablePublicIp /** * Get the serverNames property: The list of server names in the cluster. - * + * * @return the serverNames value. */ public List serverNames() { @@ -556,7 +580,7 @@ public List serverNames() { /** * Get the sourceResourceId property: The resource id of source cluster for read replica clusters. - * + * * @return the sourceResourceId value. */ public String sourceResourceId() { @@ -565,7 +589,7 @@ public String sourceResourceId() { /** * Set the sourceResourceId property: The resource id of source cluster for read replica clusters. - * + * * @param sourceResourceId the sourceResourceId value to set. * @return the ClusterProperties object itself. */ @@ -576,7 +600,7 @@ public ClusterProperties withSourceResourceId(String sourceResourceId) { /** * Get the sourceLocation property: The Azure region of source cluster for read replica clusters. - * + * * @return the sourceLocation value. */ public String sourceLocation() { @@ -585,7 +609,7 @@ public String sourceLocation() { /** * Set the sourceLocation property: The Azure region of source cluster for read replica clusters. - * + * * @param sourceLocation the sourceLocation value to set. * @return the ClusterProperties object itself. */ @@ -596,7 +620,7 @@ public ClusterProperties withSourceLocation(String sourceLocation) { /** * Get the pointInTimeUtc property: Date and time in UTC (ISO8601 format) for cluster restore. - * + * * @return the pointInTimeUtc value. */ public OffsetDateTime pointInTimeUtc() { @@ -605,7 +629,7 @@ public OffsetDateTime pointInTimeUtc() { /** * Set the pointInTimeUtc property: Date and time in UTC (ISO8601 format) for cluster restore. - * + * * @param pointInTimeUtc the pointInTimeUtc value to set. * @return the ClusterProperties object itself. */ @@ -616,7 +640,7 @@ public ClusterProperties withPointInTimeUtc(OffsetDateTime pointInTimeUtc) { /** * Get the readReplicas property: The array of read replica clusters. - * + * * @return the readReplicas value. */ public List readReplicas() { @@ -625,7 +649,7 @@ public List readReplicas() { /** * Get the earliestRestoreTime property: The earliest restore point time (ISO8601 format) for the cluster. - * + * * @return the earliestRestoreTime value. */ public OffsetDateTime earliestRestoreTime() { @@ -634,16 +658,78 @@ public OffsetDateTime earliestRestoreTime() { /** * Get the privateEndpointConnections property: The private endpoint connections for a cluster. - * + * * @return the privateEndpointConnections value. */ public List privateEndpointConnections() { return this.privateEndpointConnections; } + /** + * Get the databaseName property: The database name of the cluster. Only one database per cluster is supported. + * + * @return the databaseName value. + */ + public String databaseName() { + return this.databaseName; + } + + /** + * Set the databaseName property: The database name of the cluster. Only one database per cluster is supported. + * + * @param databaseName the databaseName value to set. + * @return the ClusterProperties object itself. + */ + public ClusterProperties withDatabaseName(String databaseName) { + this.databaseName = databaseName; + return this; + } + + /** + * Get the enableGeoBackup property: If cluster backup is stored in another Azure region in addition to the copy of + * the backup stored in the cluster's region. Enabled only at the time of cluster creation. + * + * @return the enableGeoBackup value. + */ + public Boolean enableGeoBackup() { + return this.enableGeoBackup; + } + + /** + * Set the enableGeoBackup property: If cluster backup is stored in another Azure region in addition to the copy of + * the backup stored in the cluster's region. Enabled only at the time of cluster creation. + * + * @param enableGeoBackup the enableGeoBackup value to set. + * @return the ClusterProperties object itself. + */ + public ClusterProperties withEnableGeoBackup(Boolean enableGeoBackup) { + this.enableGeoBackup = enableGeoBackup; + return this; + } + + /** + * Get the authConfig property: Authentication configuration of a cluster. + * + * @return the authConfig value. + */ + public AuthConfig authConfig() { + return this.authConfig; + } + + /** + * Set the authConfig property: Authentication configuration of a cluster. + * + * @param authConfig the authConfig value to set. + * @return the ClusterProperties object itself. + */ + public ClusterProperties withAuthConfig(AuthConfig authConfig) { + this.authConfig = authConfig; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -656,5 +742,8 @@ public void validate() { if (privateEndpointConnections() != null) { privateEndpointConnections().forEach(e -> e.validate()); } + if (authConfig() != null) { + authConfig().validate(); + } } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterPropertiesForUpdate.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterPropertiesForUpdate.java index 82ee433f33683..88d4d1a9d36a5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterPropertiesForUpdate.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterPropertiesForUpdate.java @@ -8,12 +8,14 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties used to update a cluster. */ +/** + * The properties used to update a cluster. + */ @Fluent public final class ClusterPropertiesForUpdate { /* * The password of the administrator login. Each cluster is created with pre-defined administrative role called - * ‘citus’. + * ‘citus’.  */ @JsonProperty(value = "administratorLoginPassword") private String administratorLoginPassword; @@ -111,14 +113,16 @@ public final class ClusterPropertiesForUpdate { @JsonProperty(value = "maintenanceWindow") private MaintenanceWindow maintenanceWindow; - /** Creates an instance of ClusterPropertiesForUpdate class. */ + /** + * Creates an instance of ClusterPropertiesForUpdate class. + */ public ClusterPropertiesForUpdate() { } /** * Get the administratorLoginPassword property: The password of the administrator login. Each cluster is created * with pre-defined administrative role called ‘citus’. . - * + * * @return the administratorLoginPassword value. */ public String administratorLoginPassword() { @@ -128,7 +132,7 @@ public String administratorLoginPassword() { /** * Set the administratorLoginPassword property: The password of the administrator login. Each cluster is created * with pre-defined administrative role called ‘citus’. . - * + * * @param administratorLoginPassword the administratorLoginPassword value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -139,7 +143,7 @@ public ClusterPropertiesForUpdate withAdministratorLoginPassword(String administ /** * Get the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @return the postgresqlVersion value. */ public String postgresqlVersion() { @@ -148,7 +152,7 @@ public String postgresqlVersion() { /** * Set the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @param postgresqlVersion the postgresqlVersion value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -159,7 +163,7 @@ public ClusterPropertiesForUpdate withPostgresqlVersion(String postgresqlVersion /** * Get the citusVersion property: The Citus extension version on all cluster servers. - * + * * @return the citusVersion value. */ public String citusVersion() { @@ -168,7 +172,7 @@ public String citusVersion() { /** * Set the citusVersion property: The Citus extension version on all cluster servers. - * + * * @param citusVersion the citusVersion value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -178,9 +182,9 @@ public ClusterPropertiesForUpdate withCitusVersion(String citusVersion) { } /** - * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @return the enableShardsOnCoordinator value. */ public Boolean enableShardsOnCoordinator() { @@ -188,9 +192,9 @@ public Boolean enableShardsOnCoordinator() { } /** - * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @param enableShardsOnCoordinator the enableShardsOnCoordinator value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -201,7 +205,7 @@ public ClusterPropertiesForUpdate withEnableShardsOnCoordinator(Boolean enableSh /** * Get the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @return the enableHa value. */ public Boolean enableHa() { @@ -210,7 +214,7 @@ public Boolean enableHa() { /** * Set the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @param enableHa the enableHa value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -221,7 +225,7 @@ public ClusterPropertiesForUpdate withEnableHa(Boolean enableHa) { /** * Get the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @return the preferredPrimaryZone value. */ public String preferredPrimaryZone() { @@ -230,7 +234,7 @@ public String preferredPrimaryZone() { /** * Set the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @param preferredPrimaryZone the preferredPrimaryZone value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -241,7 +245,7 @@ public ClusterPropertiesForUpdate withPreferredPrimaryZone(String preferredPrima /** * Get the coordinatorServerEdition property: The edition of the coordinator (default: GeneralPurpose). - * + * * @return the coordinatorServerEdition value. */ public String coordinatorServerEdition() { @@ -250,7 +254,7 @@ public String coordinatorServerEdition() { /** * Set the coordinatorServerEdition property: The edition of the coordinator (default: GeneralPurpose). - * + * * @param coordinatorServerEdition the coordinatorServerEdition value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -261,7 +265,7 @@ public ClusterPropertiesForUpdate withCoordinatorServerEdition(String coordinato /** * Get the coordinatorStorageQuotaInMb property: The storage of the coordinator in MB. - * + * * @return the coordinatorStorageQuotaInMb value. */ public Integer coordinatorStorageQuotaInMb() { @@ -270,7 +274,7 @@ public Integer coordinatorStorageQuotaInMb() { /** * Set the coordinatorStorageQuotaInMb property: The storage of the coordinator in MB. - * + * * @param coordinatorStorageQuotaInMb the coordinatorStorageQuotaInMb value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -281,7 +285,7 @@ public ClusterPropertiesForUpdate withCoordinatorStorageQuotaInMb(Integer coordi /** * Get the coordinatorVCores property: The vCores count of the coordinator (max: 96). - * + * * @return the coordinatorVCores value. */ public Integer coordinatorVCores() { @@ -290,7 +294,7 @@ public Integer coordinatorVCores() { /** * Set the coordinatorVCores property: The vCores count of the coordinator (max: 96). - * + * * @param coordinatorVCores the coordinatorVCores value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -301,7 +305,7 @@ public ClusterPropertiesForUpdate withCoordinatorVCores(Integer coordinatorVCore /** * Get the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @return the coordinatorEnablePublicIpAccess value. */ public Boolean coordinatorEnablePublicIpAccess() { @@ -310,7 +314,7 @@ public Boolean coordinatorEnablePublicIpAccess() { /** * Set the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @param coordinatorEnablePublicIpAccess the coordinatorEnablePublicIpAccess value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -321,7 +325,7 @@ public ClusterPropertiesForUpdate withCoordinatorEnablePublicIpAccess(Boolean co /** * Get the nodeServerEdition property: The edition of a node (default: MemoryOptimized). - * + * * @return the nodeServerEdition value. */ public String nodeServerEdition() { @@ -330,7 +334,7 @@ public String nodeServerEdition() { /** * Set the nodeServerEdition property: The edition of a node (default: MemoryOptimized). - * + * * @param nodeServerEdition the nodeServerEdition value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -343,7 +347,7 @@ public ClusterPropertiesForUpdate withNodeServerEdition(String nodeServerEdition * Get the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. - * + * * @return the nodeCount value. */ public Integer nodeCount() { @@ -354,7 +358,7 @@ public Integer nodeCount() { * Set the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. - * + * * @param nodeCount the nodeCount value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -365,7 +369,7 @@ public ClusterPropertiesForUpdate withNodeCount(Integer nodeCount) { /** * Get the nodeStorageQuotaInMb property: The storage in MB on each worker node. - * + * * @return the nodeStorageQuotaInMb value. */ public Integer nodeStorageQuotaInMb() { @@ -374,7 +378,7 @@ public Integer nodeStorageQuotaInMb() { /** * Set the nodeStorageQuotaInMb property: The storage in MB on each worker node. - * + * * @param nodeStorageQuotaInMb the nodeStorageQuotaInMb value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -385,7 +389,7 @@ public ClusterPropertiesForUpdate withNodeStorageQuotaInMb(Integer nodeStorageQu /** * Get the nodeVCores property: The compute in vCores on each worker node (max: 104). - * + * * @return the nodeVCores value. */ public Integer nodeVCores() { @@ -394,7 +398,7 @@ public Integer nodeVCores() { /** * Set the nodeVCores property: The compute in vCores on each worker node (max: 104). - * + * * @param nodeVCores the nodeVCores value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -405,7 +409,7 @@ public ClusterPropertiesForUpdate withNodeVCores(Integer nodeVCores) { /** * Get the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @return the nodeEnablePublicIpAccess value. */ public Boolean nodeEnablePublicIpAccess() { @@ -414,7 +418,7 @@ public Boolean nodeEnablePublicIpAccess() { /** * Get the maintenanceWindow property: Maintenance window of a cluster. - * + * * @return the maintenanceWindow value. */ public MaintenanceWindow maintenanceWindow() { @@ -423,7 +427,7 @@ public MaintenanceWindow maintenanceWindow() { /** * Set the maintenanceWindow property: Maintenance window of a cluster. - * + * * @param maintenanceWindow the maintenanceWindow value to set. * @return the ClusterPropertiesForUpdate object itself. */ @@ -434,7 +438,7 @@ public ClusterPropertiesForUpdate withMaintenanceWindow(MaintenanceWindow mainte /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerInner.java index 0bd9e3e16550b..56cb9ef686160 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerInner.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerRole; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents a server in a cluster. */ +/** + * Represents a server in a cluster. + */ @Fluent public final class ClusterServerInner extends ProxyResource { /* @@ -25,13 +27,15 @@ public final class ClusterServerInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ClusterServerInner class. */ + /** + * Creates an instance of ClusterServerInner class. + */ public ClusterServerInner() { } /** * Get the innerProperties property: The properties of a server in a cluster. - * + * * @return the innerProperties value. */ private ClusterServerProperties innerProperties() { @@ -40,7 +44,7 @@ private ClusterServerProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -49,7 +53,7 @@ public SystemData systemData() { /** * Get the fullyQualifiedDomainName property: The fully qualified domain name of a server. - * + * * @return the fullyQualifiedDomainName value. */ public String fullyQualifiedDomainName() { @@ -58,7 +62,7 @@ public String fullyQualifiedDomainName() { /** * Get the role property: The role of server in the cluster. - * + * * @return the role value. */ public ServerRole role() { @@ -67,7 +71,7 @@ public ServerRole role() { /** * Set the role property: The role of server in the cluster. - * + * * @param role the role value to set. * @return the ClusterServerInner object itself. */ @@ -81,7 +85,7 @@ public ClusterServerInner withRole(ServerRole role) { /** * Get the state property: A state of a cluster/server that is visible to user. - * + * * @return the state value. */ public String state() { @@ -90,7 +94,7 @@ public String state() { /** * Get the haState property: A state of HA feature for the cluster. - * + * * @return the haState value. */ public String haState() { @@ -99,7 +103,7 @@ public String haState() { /** * Get the availabilityZone property: Availability Zone information of the server. - * + * * @return the availabilityZone value. */ public String availabilityZone() { @@ -108,7 +112,7 @@ public String availabilityZone() { /** * Set the availabilityZone property: Availability Zone information of the server. - * + * * @param availabilityZone the availabilityZone value to set. * @return the ClusterServerInner object itself. */ @@ -122,7 +126,7 @@ public ClusterServerInner withAvailabilityZone(String availabilityZone) { /** * Get the postgresqlVersion property: The major PostgreSQL version of server. - * + * * @return the postgresqlVersion value. */ public String postgresqlVersion() { @@ -131,7 +135,7 @@ public String postgresqlVersion() { /** * Set the postgresqlVersion property: The major PostgreSQL version of server. - * + * * @param postgresqlVersion the postgresqlVersion value to set. * @return the ClusterServerInner object itself. */ @@ -145,7 +149,7 @@ public ClusterServerInner withPostgresqlVersion(String postgresqlVersion) { /** * Get the citusVersion property: The Citus extension version of server. - * + * * @return the citusVersion value. */ public String citusVersion() { @@ -154,7 +158,7 @@ public String citusVersion() { /** * Set the citusVersion property: The Citus extension version of server. - * + * * @param citusVersion the citusVersion value to set. * @return the ClusterServerInner object itself. */ @@ -168,7 +172,7 @@ public ClusterServerInner withCitusVersion(String citusVersion) { /** * Get the serverEdition property: The edition of a server. - * + * * @return the serverEdition value. */ public String serverEdition() { @@ -177,7 +181,7 @@ public String serverEdition() { /** * Set the serverEdition property: The edition of a server. - * + * * @param serverEdition the serverEdition value to set. * @return the ClusterServerInner object itself. */ @@ -191,7 +195,7 @@ public ClusterServerInner withServerEdition(String serverEdition) { /** * Get the storageQuotaInMb property: The storage of a server in MB. - * + * * @return the storageQuotaInMb value. */ public Integer storageQuotaInMb() { @@ -200,7 +204,7 @@ public Integer storageQuotaInMb() { /** * Set the storageQuotaInMb property: The storage of a server in MB. - * + * * @param storageQuotaInMb the storageQuotaInMb value to set. * @return the ClusterServerInner object itself. */ @@ -214,7 +218,7 @@ public ClusterServerInner withStorageQuotaInMb(Integer storageQuotaInMb) { /** * Get the vCores property: The vCores count of a server. - * + * * @return the vCores value. */ public Integer vCores() { @@ -223,7 +227,7 @@ public Integer vCores() { /** * Set the vCores property: The vCores count of a server. - * + * * @param vCores the vCores value to set. * @return the ClusterServerInner object itself. */ @@ -237,7 +241,7 @@ public ClusterServerInner withVCores(Integer vCores) { /** * Get the enableHa property: If high availability (HA) is enabled or not for the server. - * + * * @return the enableHa value. */ public Boolean enableHa() { @@ -246,7 +250,7 @@ public Boolean enableHa() { /** * Set the enableHa property: If high availability (HA) is enabled or not for the server. - * + * * @param enableHa the enableHa value to set. * @return the ClusterServerInner object itself. */ @@ -260,7 +264,7 @@ public ClusterServerInner withEnableHa(Boolean enableHa) { /** * Get the enablePublicIpAccess property: If public access is enabled on server. - * + * * @return the enablePublicIpAccess value. */ public Boolean enablePublicIpAccess() { @@ -270,7 +274,7 @@ public Boolean enablePublicIpAccess() { /** * Get the isReadOnly property: If server database is set to read-only by system maintenance depending on high disk * space usage. - * + * * @return the isReadOnly value. */ public Boolean isReadOnly() { @@ -279,7 +283,7 @@ public Boolean isReadOnly() { /** * Get the administratorLogin property: The administrator's login name of the servers in the cluster. - * + * * @return the administratorLogin value. */ public String administratorLogin() { @@ -288,7 +292,7 @@ public String administratorLogin() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerProperties.java index c52ed425d6f0c..588cdef051f77 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ClusterServerProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerRole; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of a server in cluster. */ +/** + * The properties of a server in cluster. + */ @Fluent public final class ClusterServerProperties extends ServerProperties { /* @@ -54,13 +56,15 @@ public final class ClusterServerProperties extends ServerProperties { @JsonProperty(value = "citusVersion") private String citusVersion; - /** Creates an instance of ClusterServerProperties class. */ + /** + * Creates an instance of ClusterServerProperties class. + */ public ClusterServerProperties() { } /** * Get the fullyQualifiedDomainName property: The fully qualified domain name of a server. - * + * * @return the fullyQualifiedDomainName value. */ public String fullyQualifiedDomainName() { @@ -69,7 +73,7 @@ public String fullyQualifiedDomainName() { /** * Get the role property: The role of server in the cluster. - * + * * @return the role value. */ public ServerRole role() { @@ -78,7 +82,7 @@ public ServerRole role() { /** * Set the role property: The role of server in the cluster. - * + * * @param role the role value to set. * @return the ClusterServerProperties object itself. */ @@ -89,7 +93,7 @@ public ClusterServerProperties withRole(ServerRole role) { /** * Get the state property: A state of a cluster/server that is visible to user. - * + * * @return the state value. */ public String state() { @@ -98,7 +102,7 @@ public String state() { /** * Get the haState property: A state of HA feature for the cluster. - * + * * @return the haState value. */ public String haState() { @@ -107,7 +111,7 @@ public String haState() { /** * Get the availabilityZone property: Availability Zone information of the server. - * + * * @return the availabilityZone value. */ public String availabilityZone() { @@ -116,7 +120,7 @@ public String availabilityZone() { /** * Set the availabilityZone property: Availability Zone information of the server. - * + * * @param availabilityZone the availabilityZone value to set. * @return the ClusterServerProperties object itself. */ @@ -127,7 +131,7 @@ public ClusterServerProperties withAvailabilityZone(String availabilityZone) { /** * Get the postgresqlVersion property: The major PostgreSQL version of server. - * + * * @return the postgresqlVersion value. */ public String postgresqlVersion() { @@ -136,7 +140,7 @@ public String postgresqlVersion() { /** * Set the postgresqlVersion property: The major PostgreSQL version of server. - * + * * @param postgresqlVersion the postgresqlVersion value to set. * @return the ClusterServerProperties object itself. */ @@ -147,7 +151,7 @@ public ClusterServerProperties withPostgresqlVersion(String postgresqlVersion) { /** * Get the citusVersion property: The Citus extension version of server. - * + * * @return the citusVersion value. */ public String citusVersion() { @@ -156,7 +160,7 @@ public String citusVersion() { /** * Set the citusVersion property: The Citus extension version of server. - * + * * @param citusVersion the citusVersion value to set. * @return the ClusterServerProperties object itself. */ @@ -165,28 +169,36 @@ public ClusterServerProperties withCitusVersion(String citusVersion) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterServerProperties withServerEdition(String serverEdition) { super.withServerEdition(serverEdition); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterServerProperties withStorageQuotaInMb(Integer storageQuotaInMb) { super.withStorageQuotaInMb(storageQuotaInMb); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterServerProperties withVCores(Integer vCores) { super.withVCores(vCores); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterServerProperties withEnableHa(Boolean enableHa) { super.withEnableHa(enableHa); @@ -195,7 +207,7 @@ public ClusterServerProperties withEnableHa(Boolean enableHa) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationInner.java index e97037232bb2c..86d787e7ee846 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationInner.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Represents configuration details for coordinator and node. */ +/** + * Represents configuration details for coordinator and node. + */ @Fluent public final class ConfigurationInner extends ProxyResource { /* @@ -28,13 +30,15 @@ public final class ConfigurationInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ConfigurationInner class. */ + /** + * Creates an instance of ConfigurationInner class. + */ public ConfigurationInner() { } /** * Get the innerProperties property: The properties of configuration. - * + * * @return the innerProperties value. */ private ConfigurationProperties innerProperties() { @@ -43,7 +47,7 @@ private ConfigurationProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -52,7 +56,7 @@ public SystemData systemData() { /** * Get the description property: Description of the configuration. - * + * * @return the description value. */ public String description() { @@ -61,7 +65,7 @@ public String description() { /** * Get the dataType property: Data type of the configuration. - * + * * @return the dataType value. */ public ConfigurationDataType dataType() { @@ -70,7 +74,7 @@ public ConfigurationDataType dataType() { /** * Get the allowedValues property: Allowed values of the configuration. - * + * * @return the allowedValues value. */ public String allowedValues() { @@ -79,7 +83,7 @@ public String allowedValues() { /** * Get the requiresRestart property: If configuration change requires restart. - * + * * @return the requiresRestart value. */ public Boolean requiresRestart() { @@ -88,7 +92,7 @@ public Boolean requiresRestart() { /** * Set the requiresRestart property: If configuration change requires restart. - * + * * @param requiresRestart the requiresRestart value to set. * @return the ConfigurationInner object itself. */ @@ -102,7 +106,7 @@ public ConfigurationInner withRequiresRestart(Boolean requiresRestart) { /** * Get the serverRoleGroupConfigurations property: The list of server role group configuration values. - * + * * @return the serverRoleGroupConfigurations value. */ public List serverRoleGroupConfigurations() { @@ -111,12 +115,12 @@ public List serverRoleGroupConfigurations() { /** * Set the serverRoleGroupConfigurations property: The list of server role group configuration values. - * + * * @param serverRoleGroupConfigurations the serverRoleGroupConfigurations value to set. * @return the ConfigurationInner object itself. */ - public ConfigurationInner withServerRoleGroupConfigurations( - List serverRoleGroupConfigurations) { + public ConfigurationInner + withServerRoleGroupConfigurations(List serverRoleGroupConfigurations) { if (this.innerProperties() == null) { this.innerProperties = new ConfigurationProperties(); } @@ -126,7 +130,7 @@ public ConfigurationInner withServerRoleGroupConfigurations( /** * Get the provisioningState property: Provisioning state of the configuration. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -135,7 +139,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationProperties.java index 719a8f0e66a1c..df6046ec86387 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ConfigurationProperties.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The properties of configuration. */ +/** + * The properties of configuration. + */ @Fluent public final class ConfigurationProperties { /* @@ -51,13 +53,15 @@ public final class ConfigurationProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of ConfigurationProperties class. */ + /** + * Creates an instance of ConfigurationProperties class. + */ public ConfigurationProperties() { } /** * Get the description property: Description of the configuration. - * + * * @return the description value. */ public String description() { @@ -66,7 +70,7 @@ public String description() { /** * Get the dataType property: Data type of the configuration. - * + * * @return the dataType value. */ public ConfigurationDataType dataType() { @@ -75,7 +79,7 @@ public ConfigurationDataType dataType() { /** * Get the allowedValues property: Allowed values of the configuration. - * + * * @return the allowedValues value. */ public String allowedValues() { @@ -84,7 +88,7 @@ public String allowedValues() { /** * Get the requiresRestart property: If configuration change requires restart. - * + * * @return the requiresRestart value. */ public Boolean requiresRestart() { @@ -93,7 +97,7 @@ public Boolean requiresRestart() { /** * Set the requiresRestart property: If configuration change requires restart. - * + * * @param requiresRestart the requiresRestart value to set. * @return the ConfigurationProperties object itself. */ @@ -104,7 +108,7 @@ public ConfigurationProperties withRequiresRestart(Boolean requiresRestart) { /** * Get the serverRoleGroupConfigurations property: The list of server role group configuration values. - * + * * @return the serverRoleGroupConfigurations value. */ public List serverRoleGroupConfigurations() { @@ -113,19 +117,19 @@ public List serverRoleGroupConfigurations() { /** * Set the serverRoleGroupConfigurations property: The list of server role group configuration values. - * + * * @param serverRoleGroupConfigurations the serverRoleGroupConfigurations value to set. * @return the ConfigurationProperties object itself. */ - public ConfigurationProperties withServerRoleGroupConfigurations( - List serverRoleGroupConfigurations) { + public ConfigurationProperties + withServerRoleGroupConfigurations(List serverRoleGroupConfigurations) { this.serverRoleGroupConfigurations = serverRoleGroupConfigurations; return this; } /** * Get the provisioningState property: Provisioning state of the configuration. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -134,15 +138,13 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (serverRoleGroupConfigurations() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property serverRoleGroupConfigurations in model ConfigurationProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property serverRoleGroupConfigurations in model ConfigurationProperties")); } else { serverRoleGroupConfigurations().forEach(e -> e.validate()); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleInner.java index 2064938646327..3c59b51046a53 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleInner.java @@ -11,7 +11,9 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents a cluster firewall rule. */ +/** + * Represents a cluster firewall rule. + */ @Fluent public final class FirewallRuleInner extends ProxyResource { /* @@ -26,13 +28,15 @@ public final class FirewallRuleInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of FirewallRuleInner class. */ + /** + * Creates an instance of FirewallRuleInner class. + */ public FirewallRuleInner() { } /** * Get the innerProperties property: The properties of a firewall rule. - * + * * @return the innerProperties value. */ private FirewallRuleProperties innerProperties() { @@ -41,7 +45,7 @@ private FirewallRuleProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -50,7 +54,7 @@ public SystemData systemData() { /** * Get the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @return the startIpAddress value. */ public String startIpAddress() { @@ -59,7 +63,7 @@ public String startIpAddress() { /** * Set the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @param startIpAddress the startIpAddress value to set. * @return the FirewallRuleInner object itself. */ @@ -73,7 +77,7 @@ public FirewallRuleInner withStartIpAddress(String startIpAddress) { /** * Get the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @return the endIpAddress value. */ public String endIpAddress() { @@ -82,7 +86,7 @@ public String endIpAddress() { /** * Set the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @param endIpAddress the endIpAddress value to set. * @return the FirewallRuleInner object itself. */ @@ -96,7 +100,7 @@ public FirewallRuleInner withEndIpAddress(String endIpAddress) { /** * Get the provisioningState property: Provisioning state of the firewall rule. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -105,15 +109,13 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model FirewallRuleInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model FirewallRuleInner")); } else { innerProperties().validate(); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleProperties.java index 1179a5aba67b6..776e3d23c9bfd 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/FirewallRuleProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of a cluster firewall rule. */ +/** + * The properties of a cluster firewall rule. + */ @Fluent public final class FirewallRuleProperties { /* @@ -30,13 +32,15 @@ public final class FirewallRuleProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of FirewallRuleProperties class. */ + /** + * Creates an instance of FirewallRuleProperties class. + */ public FirewallRuleProperties() { } /** * Get the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @return the startIpAddress value. */ public String startIpAddress() { @@ -45,7 +49,7 @@ public String startIpAddress() { /** * Set the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @param startIpAddress the startIpAddress value to set. * @return the FirewallRuleProperties object itself. */ @@ -56,7 +60,7 @@ public FirewallRuleProperties withStartIpAddress(String startIpAddress) { /** * Get the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @return the endIpAddress value. */ public String endIpAddress() { @@ -65,7 +69,7 @@ public String endIpAddress() { /** * Set the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @param endIpAddress the endIpAddress value to set. * @return the FirewallRuleProperties object itself. */ @@ -76,7 +80,7 @@ public FirewallRuleProperties withEndIpAddress(String endIpAddress) { /** * Get the provisioningState property: Provisioning state of the firewall rule. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -85,21 +89,17 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (startIpAddress() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property startIpAddress in model FirewallRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property startIpAddress in model FirewallRuleProperties")); } if (endIpAddress() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property endIpAddress in model FirewallRuleProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property endIpAddress in model FirewallRuleProperties")); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/NameAvailabilityInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/NameAvailabilityInner.java index b2767f5b6b86c..013f99e68d696 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/NameAvailabilityInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/NameAvailabilityInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents cluster name availability. */ +/** + * Represents cluster name availability. + */ @Fluent public final class NameAvailabilityInner { /* @@ -34,13 +36,15 @@ public final class NameAvailabilityInner { @JsonProperty(value = "type") private String type; - /** Creates an instance of NameAvailabilityInner class. */ + /** + * Creates an instance of NameAvailabilityInner class. + */ public NameAvailabilityInner() { } /** * Get the message property: Error message. - * + * * @return the message value. */ public String message() { @@ -49,7 +53,7 @@ public String message() { /** * Set the message property: Error message. - * + * * @param message the message value to set. * @return the NameAvailabilityInner object itself. */ @@ -60,7 +64,7 @@ public NameAvailabilityInner withMessage(String message) { /** * Get the nameAvailable property: Indicates whether the cluster name is available. - * + * * @return the nameAvailable value. */ public Boolean nameAvailable() { @@ -69,7 +73,7 @@ public Boolean nameAvailable() { /** * Set the nameAvailable property: Indicates whether the cluster name is available. - * + * * @param nameAvailable the nameAvailable value to set. * @return the NameAvailabilityInner object itself. */ @@ -80,7 +84,7 @@ public NameAvailabilityInner withNameAvailable(Boolean nameAvailable) { /** * Get the name property: Name of the cluster. - * + * * @return the name value. */ public String name() { @@ -89,7 +93,7 @@ public String name() { /** * Set the name property: Name of the cluster. - * + * * @param name the name value to set. * @return the NameAvailabilityInner object itself. */ @@ -100,7 +104,7 @@ public NameAvailabilityInner withName(String name) { /** * Get the type property: Type of the cluster. - * + * * @return the type value. */ public String type() { @@ -109,7 +113,7 @@ public String type() { /** * Set the type property: Type of the cluster. - * + * * @param type the type value to set. * @return the NameAvailabilityInner object itself. */ @@ -120,7 +124,7 @@ public NameAvailabilityInner withType(String type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/OperationInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/OperationInner.java index 48ccbb7183a0f..30ddcfccba5a9 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/OperationInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/OperationInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** REST API operation definition. */ +/** + * REST API operation definition. + */ @Fluent public final class OperationInner { /* @@ -45,13 +47,15 @@ public final class OperationInner { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map properties; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: The name of the operation being performed on this particular object. - * + * * @return the name value. */ public String name() { @@ -60,7 +64,7 @@ public String name() { /** * Get the display property: The localized display information for this particular operation or action. - * + * * @return the display value. */ public OperationDisplay display() { @@ -69,7 +73,7 @@ public OperationDisplay display() { /** * Get the isDataAction property: Indicates whether the operation is a data action. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -78,7 +82,7 @@ public Boolean isDataAction() { /** * Set the isDataAction property: Indicates whether the operation is a data action. - * + * * @param isDataAction the isDataAction value to set. * @return the OperationInner object itself. */ @@ -89,7 +93,7 @@ public OperationInner withIsDataAction(Boolean isDataAction) { /** * Get the origin property: The intended executor of the operation. - * + * * @return the origin value. */ public OperationOrigin origin() { @@ -98,7 +102,7 @@ public OperationOrigin origin() { /** * Get the properties property: Additional descriptions for the operation. - * + * * @return the properties value. */ public Map properties() { @@ -107,7 +111,7 @@ public Map properties() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionInner.java index 42ba1013b5c98..18e464e6a1621 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionInner.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The private endpoint connection resource. */ +/** + * The private endpoint connection resource. + */ @Fluent public final class PrivateEndpointConnectionInner extends ProxyResource { /* @@ -28,13 +30,15 @@ public final class PrivateEndpointConnectionInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of PrivateEndpointConnectionInner class. */ + /** + * Creates an instance of PrivateEndpointConnectionInner class. + */ public PrivateEndpointConnectionInner() { } /** * Get the innerProperties property: Resource properties. - * + * * @return the innerProperties value. */ private PrivateEndpointConnectionProperties innerProperties() { @@ -43,7 +47,7 @@ private PrivateEndpointConnectionProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -52,7 +56,7 @@ public SystemData systemData() { /** * Get the groupIds property: The group ids for the private endpoint resource. - * + * * @return the groupIds value. */ public List groupIds() { @@ -61,7 +65,7 @@ public List groupIds() { /** * Get the privateEndpoint property: The private endpoint resource. - * + * * @return the privateEndpoint value. */ public PrivateEndpoint privateEndpoint() { @@ -70,7 +74,7 @@ public PrivateEndpoint privateEndpoint() { /** * Set the privateEndpoint property: The private endpoint resource. - * + * * @param privateEndpoint the privateEndpoint value to set. * @return the PrivateEndpointConnectionInner object itself. */ @@ -83,9 +87,9 @@ public PrivateEndpointConnectionInner withPrivateEndpoint(PrivateEndpoint privat } /** - * Get the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Get the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @return the privateLinkServiceConnectionState value. */ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { @@ -93,14 +97,14 @@ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { } /** - * Set the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Set the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. * @return the PrivateEndpointConnectionInner object itself. */ - public PrivateEndpointConnectionInner withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionInner + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { if (this.innerProperties() == null) { this.innerProperties = new PrivateEndpointConnectionProperties(); } @@ -110,7 +114,7 @@ public PrivateEndpointConnectionInner withPrivateLinkServiceConnectionState( /** * Get the provisioningState property: The provisioning state of the private endpoint connection resource. - * + * * @return the provisioningState value. */ public PrivateEndpointConnectionProvisioningState provisioningState() { @@ -119,7 +123,7 @@ public PrivateEndpointConnectionProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionProperties.java index dafec693728c8..5e06153324560 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionProperties.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Properties of the private endpoint connection. */ +/** + * Properties of the private endpoint connection. + */ @Fluent public final class PrivateEndpointConnectionProperties { /* @@ -39,13 +41,15 @@ public final class PrivateEndpointConnectionProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private PrivateEndpointConnectionProvisioningState provisioningState; - /** Creates an instance of PrivateEndpointConnectionProperties class. */ + /** + * Creates an instance of PrivateEndpointConnectionProperties class. + */ public PrivateEndpointConnectionProperties() { } /** * Get the groupIds property: The group ids for the private endpoint resource. - * + * * @return the groupIds value. */ public List groupIds() { @@ -54,7 +58,7 @@ public List groupIds() { /** * Get the privateEndpoint property: The private endpoint resource. - * + * * @return the privateEndpoint value. */ public PrivateEndpoint privateEndpoint() { @@ -63,7 +67,7 @@ public PrivateEndpoint privateEndpoint() { /** * Set the privateEndpoint property: The private endpoint resource. - * + * * @param privateEndpoint the privateEndpoint value to set. * @return the PrivateEndpointConnectionProperties object itself. */ @@ -73,9 +77,9 @@ public PrivateEndpointConnectionProperties withPrivateEndpoint(PrivateEndpoint p } /** - * Get the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Get the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @return the privateLinkServiceConnectionState value. */ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { @@ -83,21 +87,21 @@ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { } /** - * Set the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Set the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. * @return the PrivateEndpointConnectionProperties object itself. */ - public PrivateEndpointConnectionProperties withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionProperties + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { this.privateLinkServiceConnectionState = privateLinkServiceConnectionState; return this; } /** * Get the provisioningState property: The provisioning state of the private endpoint connection resource. - * + * * @return the provisioningState value. */ public PrivateEndpointConnectionProvisioningState provisioningState() { @@ -106,7 +110,7 @@ public PrivateEndpointConnectionProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -114,11 +118,8 @@ public void validate() { privateEndpoint().validate(); } if (privateLinkServiceConnectionState() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property privateLinkServiceConnectionState in model" - + " PrivateEndpointConnectionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property privateLinkServiceConnectionState in model PrivateEndpointConnectionProperties")); } else { privateLinkServiceConnectionState().validate(); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionSimpleProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionSimpleProperties.java index ca268048eb108..7f79e8d3a2c80 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionSimpleProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateEndpointConnectionSimpleProperties.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The properties in private endpoint connection. */ +/** + * The properties in private endpoint connection. + */ @Fluent public final class PrivateEndpointConnectionSimpleProperties { /* @@ -31,13 +33,15 @@ public final class PrivateEndpointConnectionSimpleProperties { @JsonProperty(value = "privateLinkServiceConnectionState") private PrivateLinkServiceConnectionState privateLinkServiceConnectionState; - /** Creates an instance of PrivateEndpointConnectionSimpleProperties class. */ + /** + * Creates an instance of PrivateEndpointConnectionSimpleProperties class. + */ public PrivateEndpointConnectionSimpleProperties() { } /** * Get the privateEndpoint property: Private endpoint which the connection belongs to. - * + * * @return the privateEndpoint value. */ public PrivateEndpointProperty privateEndpoint() { @@ -46,7 +50,7 @@ public PrivateEndpointProperty privateEndpoint() { /** * Set the privateEndpoint property: Private endpoint which the connection belongs to. - * + * * @param privateEndpoint the privateEndpoint value to set. * @return the PrivateEndpointConnectionSimpleProperties object itself. */ @@ -57,7 +61,7 @@ public PrivateEndpointConnectionSimpleProperties withPrivateEndpoint(PrivateEndp /** * Get the groupIds property: Group ids of the private endpoint connection. - * + * * @return the groupIds value. */ public List groupIds() { @@ -66,7 +70,7 @@ public List groupIds() { /** * Set the groupIds property: Group ids of the private endpoint connection. - * + * * @param groupIds the groupIds value to set. * @return the PrivateEndpointConnectionSimpleProperties object itself. */ @@ -76,9 +80,9 @@ public PrivateEndpointConnectionSimpleProperties withGroupIds(List group } /** - * Get the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Get the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @return the privateLinkServiceConnectionState value. */ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { @@ -86,21 +90,21 @@ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { } /** - * Set the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Set the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. * @return the PrivateEndpointConnectionSimpleProperties object itself. */ - public PrivateEndpointConnectionSimpleProperties withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionSimpleProperties + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { this.privateLinkServiceConnectionState = privateLinkServiceConnectionState; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceInner.java index 846734fa5f198..60df34d951532 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A private link resource. */ +/** + * A private link resource. + */ @Fluent public final class PrivateLinkResourceInner extends ProxyResource { /* @@ -25,13 +27,15 @@ public final class PrivateLinkResourceInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of PrivateLinkResourceInner class. */ + /** + * Creates an instance of PrivateLinkResourceInner class. + */ public PrivateLinkResourceInner() { } /** * Get the innerProperties property: Resource properties. - * + * * @return the innerProperties value. */ private PrivateLinkResourceProperties innerProperties() { @@ -40,7 +44,7 @@ private PrivateLinkResourceProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -49,7 +53,7 @@ public SystemData systemData() { /** * Get the groupId property: The private link resource group id. - * + * * @return the groupId value. */ public String groupId() { @@ -58,7 +62,7 @@ public String groupId() { /** * Get the requiredMembers property: The private link resource required member names. - * + * * @return the requiredMembers value. */ public List requiredMembers() { @@ -67,7 +71,7 @@ public List requiredMembers() { /** * Get the requiredZoneNames property: The private link resource private link DNS zone name. - * + * * @return the requiredZoneNames value. */ public List requiredZoneNames() { @@ -76,7 +80,7 @@ public List requiredZoneNames() { /** * Set the requiredZoneNames property: The private link resource private link DNS zone name. - * + * * @param requiredZoneNames the requiredZoneNames value to set. * @return the PrivateLinkResourceInner object itself. */ @@ -90,7 +94,7 @@ public PrivateLinkResourceInner withRequiredZoneNames(List requiredZoneN /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceProperties.java index 150593b98fd15..b87e688503526 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/PrivateLinkResourceProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Properties of a private link resource. */ +/** + * Properties of a private link resource. + */ @Fluent public final class PrivateLinkResourceProperties { /* @@ -29,13 +31,15 @@ public final class PrivateLinkResourceProperties { @JsonProperty(value = "requiredZoneNames") private List requiredZoneNames; - /** Creates an instance of PrivateLinkResourceProperties class. */ + /** + * Creates an instance of PrivateLinkResourceProperties class. + */ public PrivateLinkResourceProperties() { } /** * Get the groupId property: The private link resource group id. - * + * * @return the groupId value. */ public String groupId() { @@ -44,7 +48,7 @@ public String groupId() { /** * Get the requiredMembers property: The private link resource required member names. - * + * * @return the requiredMembers value. */ public List requiredMembers() { @@ -53,7 +57,7 @@ public List requiredMembers() { /** * Get the requiredZoneNames property: The private link resource private link DNS zone name. - * + * * @return the requiredZoneNames value. */ public List requiredZoneNames() { @@ -62,7 +66,7 @@ public List requiredZoneNames() { /** * Set the requiredZoneNames property: The private link resource private link DNS zone name. - * + * * @param requiredZoneNames the requiredZoneNames value to set. * @return the PrivateLinkResourceProperties object itself. */ @@ -73,7 +77,7 @@ public PrivateLinkResourceProperties withRequiredZoneNames(List required /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleInner.java index 6eb3cb6f86875..52d20f0e3df53 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleInner.java @@ -8,10 +8,14 @@ import com.azure.core.management.ProxyResource; import com.azure.core.management.SystemData; import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrincipalType; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.RoleType; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents a cluster role. */ +/** + * Represents a cluster role. + */ @Fluent public final class RoleInner extends ProxyResource { /* @@ -26,13 +30,15 @@ public final class RoleInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of RoleInner class. */ + /** + * Creates an instance of RoleInner class. + */ public RoleInner() { } /** * Get the innerProperties property: The properties of a role. - * + * * @return the innerProperties value. */ private RoleProperties innerProperties() { @@ -41,7 +47,7 @@ private RoleProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -49,8 +55,32 @@ public SystemData systemData() { } /** - * Get the password property: The password of the cluster role. - * + * Get the roleType property: The roleType property. + * + * @return the roleType value. + */ + public RoleType roleType() { + return this.innerProperties() == null ? null : this.innerProperties().roleType(); + } + + /** + * Set the roleType property: The roleType property. + * + * @param roleType the roleType value to set. + * @return the RoleInner object itself. + */ + public RoleInner withRoleType(RoleType roleType) { + if (this.innerProperties() == null) { + this.innerProperties = new RoleProperties(); + } + this.innerProperties().withRoleType(roleType); + return this; + } + + /** + * Get the password property: The password of the cluster role. If an identity is used, password will not be + * required. + * * @return the password value. */ public String password() { @@ -58,8 +88,9 @@ public String password() { } /** - * Set the password property: The password of the cluster role. - * + * Set the password property: The password of the cluster role. If an identity is used, password will not be + * required. + * * @param password the password value to set. * @return the RoleInner object itself. */ @@ -73,23 +104,91 @@ public RoleInner withPassword(String password) { /** * Get the provisioningState property: Provisioning state of the role. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); } + /** + * Get the objectId property: The objectId property. + * + * @return the objectId value. + */ + public String objectId() { + return this.innerProperties() == null ? null : this.innerProperties().objectId(); + } + + /** + * Set the objectId property: The objectId property. + * + * @param objectId the objectId value to set. + * @return the RoleInner object itself. + */ + public RoleInner withObjectId(String objectId) { + if (this.innerProperties() == null) { + this.innerProperties = new RoleProperties(); + } + this.innerProperties().withObjectId(objectId); + return this; + } + + /** + * Get the principalType property: The principalType property. + * + * @return the principalType value. + */ + public PrincipalType principalType() { + return this.innerProperties() == null ? null : this.innerProperties().principalType(); + } + + /** + * Set the principalType property: The principalType property. + * + * @param principalType the principalType value to set. + * @return the RoleInner object itself. + */ + public RoleInner withPrincipalType(PrincipalType principalType) { + if (this.innerProperties() == null) { + this.innerProperties = new RoleProperties(); + } + this.innerProperties().withPrincipalType(principalType); + return this; + } + + /** + * Get the tenantId property: The tenantId property. + * + * @return the tenantId value. + */ + public String tenantId() { + return this.innerProperties() == null ? null : this.innerProperties().tenantId(); + } + + /** + * Set the tenantId property: The tenantId property. + * + * @param tenantId the tenantId value to set. + * @return the RoleInner object itself. + */ + public RoleInner withTenantId(String tenantId) { + if (this.innerProperties() == null) { + this.innerProperties = new RoleProperties(); + } + this.innerProperties().withTenantId(tenantId); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model RoleInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model RoleInner")); } else { innerProperties().validate(); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleProperties.java index e2fb46c24d881..d22be56b854f3 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RoleProperties.java @@ -5,32 +5,70 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrincipalType; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.RoleType; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of a cluster role. */ +/** + * The properties of a cluster role. + */ @Fluent public final class RoleProperties { /* - * The password of the cluster role. + * The roleType property. + */ + @JsonProperty(value = "roleType") + private RoleType roleType; + + /* + * The password of the cluster role. If an identity is used, password will not be required. */ @JsonProperty(value = "password") private String password; + /* + * The externalIdentity property. + */ + @JsonProperty(value = "externalIdentity") + private RolePropertiesExternalIdentity innerExternalIdentity; + /* * Provisioning state of the role */ @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of RoleProperties class. */ + /** + * Creates an instance of RoleProperties class. + */ public RoleProperties() { } /** - * Get the password property: The password of the cluster role. - * + * Get the roleType property: The roleType property. + * + * @return the roleType value. + */ + public RoleType roleType() { + return this.roleType; + } + + /** + * Set the roleType property: The roleType property. + * + * @param roleType the roleType value to set. + * @return the RoleProperties object itself. + */ + public RoleProperties withRoleType(RoleType roleType) { + this.roleType = roleType; + return this; + } + + /** + * Get the password property: The password of the cluster role. If an identity is used, password will not be + * required. + * * @return the password value. */ public String password() { @@ -38,8 +76,9 @@ public String password() { } /** - * Set the password property: The password of the cluster role. - * + * Set the password property: The password of the cluster role. If an identity is used, password will not be + * required. + * * @param password the password value to set. * @return the RoleProperties object itself. */ @@ -48,27 +87,101 @@ public RoleProperties withPassword(String password) { return this; } + /** + * Get the innerExternalIdentity property: The externalIdentity property. + * + * @return the innerExternalIdentity value. + */ + private RolePropertiesExternalIdentity innerExternalIdentity() { + return this.innerExternalIdentity; + } + /** * Get the provisioningState property: Provisioning state of the role. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { return this.provisioningState; } + /** + * Get the objectId property: The objectId property. + * + * @return the objectId value. + */ + public String objectId() { + return this.innerExternalIdentity() == null ? null : this.innerExternalIdentity().objectId(); + } + + /** + * Set the objectId property: The objectId property. + * + * @param objectId the objectId value to set. + * @return the RoleProperties object itself. + */ + public RoleProperties withObjectId(String objectId) { + if (this.innerExternalIdentity() == null) { + this.innerExternalIdentity = new RolePropertiesExternalIdentity(); + } + this.innerExternalIdentity().withObjectId(objectId); + return this; + } + + /** + * Get the principalType property: The principalType property. + * + * @return the principalType value. + */ + public PrincipalType principalType() { + return this.innerExternalIdentity() == null ? null : this.innerExternalIdentity().principalType(); + } + + /** + * Set the principalType property: The principalType property. + * + * @param principalType the principalType value to set. + * @return the RoleProperties object itself. + */ + public RoleProperties withPrincipalType(PrincipalType principalType) { + if (this.innerExternalIdentity() == null) { + this.innerExternalIdentity = new RolePropertiesExternalIdentity(); + } + this.innerExternalIdentity().withPrincipalType(principalType); + return this; + } + + /** + * Get the tenantId property: The tenantId property. + * + * @return the tenantId value. + */ + public String tenantId() { + return this.innerExternalIdentity() == null ? null : this.innerExternalIdentity().tenantId(); + } + + /** + * Set the tenantId property: The tenantId property. + * + * @param tenantId the tenantId value to set. + * @return the RoleProperties object itself. + */ + public RoleProperties withTenantId(String tenantId) { + if (this.innerExternalIdentity() == null) { + this.innerExternalIdentity = new RolePropertiesExternalIdentity(); + } + this.innerExternalIdentity().withTenantId(tenantId); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (password() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property password in model RoleProperties")); + if (innerExternalIdentity() != null) { + innerExternalIdentity().validate(); } } - - private static final ClientLogger LOGGER = new ClientLogger(RoleProperties.class); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RolePropertiesExternalIdentity.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RolePropertiesExternalIdentity.java new file mode 100644 index 0000000000000..1f6119b5232db --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/RolePropertiesExternalIdentity.java @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrincipalType; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The RolePropertiesExternalIdentity model. + */ +@Fluent +public final class RolePropertiesExternalIdentity { + /* + * The objectId property. + */ + @JsonProperty(value = "objectId", required = true) + private String objectId; + + /* + * The principalType property. + */ + @JsonProperty(value = "principalType", required = true) + private PrincipalType principalType; + + /* + * The tenantId property. + */ + @JsonProperty(value = "tenantId") + private String tenantId; + + /** + * Creates an instance of RolePropertiesExternalIdentity class. + */ + public RolePropertiesExternalIdentity() { + } + + /** + * Get the objectId property: The objectId property. + * + * @return the objectId value. + */ + public String objectId() { + return this.objectId; + } + + /** + * Set the objectId property: The objectId property. + * + * @param objectId the objectId value to set. + * @return the RolePropertiesExternalIdentity object itself. + */ + public RolePropertiesExternalIdentity withObjectId(String objectId) { + this.objectId = objectId; + return this; + } + + /** + * Get the principalType property: The principalType property. + * + * @return the principalType value. + */ + public PrincipalType principalType() { + return this.principalType; + } + + /** + * Set the principalType property: The principalType property. + * + * @param principalType the principalType value to set. + * @return the RolePropertiesExternalIdentity object itself. + */ + public RolePropertiesExternalIdentity withPrincipalType(PrincipalType principalType) { + this.principalType = principalType; + return this; + } + + /** + * Get the tenantId property: The tenantId property. + * + * @return the tenantId value. + */ + public String tenantId() { + return this.tenantId; + } + + /** + * Set the tenantId property: The tenantId property. + * + * @param tenantId the tenantId value to set. + * @return the RolePropertiesExternalIdentity object itself. + */ + public RolePropertiesExternalIdentity withTenantId(String tenantId) { + this.tenantId = tenantId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (objectId() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property objectId in model RolePropertiesExternalIdentity")); + } + if (principalType() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property principalType in model RolePropertiesExternalIdentity")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(RolePropertiesExternalIdentity.class); +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationInner.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationInner.java index 7ac043e1389ce..648ce836ea9ec 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationInner.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationInner.java @@ -11,7 +11,9 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents a configuration. */ +/** + * Represents a configuration. + */ @Fluent public final class ServerConfigurationInner extends ProxyResource { /* @@ -26,13 +28,15 @@ public final class ServerConfigurationInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ServerConfigurationInner class. */ + /** + * Creates an instance of ServerConfigurationInner class. + */ public ServerConfigurationInner() { } /** * Get the innerProperties property: The properties of a configuration. - * + * * @return the innerProperties value. */ private ServerConfigurationProperties innerProperties() { @@ -41,7 +45,7 @@ private ServerConfigurationProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -50,7 +54,7 @@ public SystemData systemData() { /** * Get the value property: Value of the configuration. - * + * * @return the value value. */ public String value() { @@ -59,7 +63,7 @@ public String value() { /** * Set the value property: Value of the configuration. - * + * * @param value the value value to set. * @return the ServerConfigurationInner object itself. */ @@ -73,7 +77,7 @@ public ServerConfigurationInner withValue(String value) { /** * Get the source property: Source of the configuration. - * + * * @return the source value. */ public String source() { @@ -82,7 +86,7 @@ public String source() { /** * Get the description property: Description of the configuration. - * + * * @return the description value. */ public String description() { @@ -91,7 +95,7 @@ public String description() { /** * Get the defaultValue property: Default value of the configuration. - * + * * @return the defaultValue value. */ public String defaultValue() { @@ -100,7 +104,7 @@ public String defaultValue() { /** * Get the dataType property: Data type of the configuration. - * + * * @return the dataType value. */ public ConfigurationDataType dataType() { @@ -109,7 +113,7 @@ public ConfigurationDataType dataType() { /** * Get the allowedValues property: Allowed values of the configuration. - * + * * @return the allowedValues value. */ public String allowedValues() { @@ -118,7 +122,7 @@ public String allowedValues() { /** * Get the requiresRestart property: If configuration change requires restart. - * + * * @return the requiresRestart value. */ public Boolean requiresRestart() { @@ -127,7 +131,7 @@ public Boolean requiresRestart() { /** * Get the provisioningState property: Provisioning state of the configuration. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -136,7 +140,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationProperties.java index 57038b9bba765..2a6549277ec89 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/ServerConfigurationProperties.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of a configuration. */ +/** + * The properties of a configuration. + */ @Fluent public final class ServerConfigurationProperties { /* @@ -61,13 +63,15 @@ public final class ServerConfigurationProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of ServerConfigurationProperties class. */ + /** + * Creates an instance of ServerConfigurationProperties class. + */ public ServerConfigurationProperties() { } /** * Get the value property: Value of the configuration. - * + * * @return the value value. */ public String value() { @@ -76,7 +80,7 @@ public String value() { /** * Set the value property: Value of the configuration. - * + * * @param value the value value to set. * @return the ServerConfigurationProperties object itself. */ @@ -87,7 +91,7 @@ public ServerConfigurationProperties withValue(String value) { /** * Get the source property: Source of the configuration. - * + * * @return the source value. */ public String source() { @@ -96,7 +100,7 @@ public String source() { /** * Get the description property: Description of the configuration. - * + * * @return the description value. */ public String description() { @@ -105,7 +109,7 @@ public String description() { /** * Get the defaultValue property: Default value of the configuration. - * + * * @return the defaultValue value. */ public String defaultValue() { @@ -114,7 +118,7 @@ public String defaultValue() { /** * Get the dataType property: Data type of the configuration. - * + * * @return the dataType value. */ public ConfigurationDataType dataType() { @@ -123,7 +127,7 @@ public ConfigurationDataType dataType() { /** * Get the allowedValues property: Allowed values of the configuration. - * + * * @return the allowedValues value. */ public String allowedValues() { @@ -132,7 +136,7 @@ public String allowedValues() { /** * Get the requiresRestart property: If configuration change requires restart. - * + * * @return the requiresRestart value. */ public Boolean requiresRestart() { @@ -141,7 +145,7 @@ public Boolean requiresRestart() { /** * Get the provisioningState property: Provisioning state of the configuration. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -150,15 +154,13 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model ServerConfigurationProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ServerConfigurationProperties")); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/package-info.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/package-info.java index c8e5f81ab2d3b..e3459212be877 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/package-info.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the inner data models for CosmosDBForPostgreSql. Azure Cosmos DB for PostgreSQL database service - * resource provider REST APIs. + * Package containing the inner data models for CosmosDBForPostgreSql. + * Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. */ package com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/package-info.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/package-info.java index 7b95ca7d7afdb..635ba5c275587 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/package-info.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/fluent/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the service clients for CosmosDBForPostgreSql. Azure Cosmos DB for PostgreSQL database service - * resource provider REST APIs. + * Package containing the service clients for CosmosDBForPostgreSql. + * Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. */ package com.azure.resourcemanager.cosmosdbforpostgresql.fluent; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterImpl.java index 647d161b56c58..afae3dba643ba 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterImpl.java @@ -8,9 +8,11 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterInner; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.AuthConfig; import com.azure.resourcemanager.cosmosdbforpostgresql.models.Cluster; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterForUpdate; import com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PromoteRequest; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerNameItem; import com.azure.resourcemanager.cosmosdbforpostgresql.models.SimplePrivateEndpointConnection; import java.time.OffsetDateTime; @@ -171,6 +173,18 @@ public List privateEndpointConnections() { } } + public String databaseName() { + return this.innerModel().databaseName(); + } + + public Boolean enableGeoBackup() { + return this.innerModel().enableGeoBackup(); + } + + public AuthConfig authConfig() { + return this.innerModel().authConfig(); + } + public Region region() { return Region.fromName(this.regionName()); } @@ -203,25 +217,19 @@ public ClusterImpl withExistingResourceGroup(String resourceGroupName) { } public Cluster create() { - this.innerObject = - serviceManager - .serviceClient() - .getClusters() - .create(resourceGroupName, clusterName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getClusters().create(resourceGroupName, clusterName, + this.innerModel(), Context.NONE); return this; } public Cluster create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getClusters() - .create(resourceGroupName, clusterName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getClusters().create(resourceGroupName, clusterName, + this.innerModel(), context); return this; } - ClusterImpl( - String name, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { + ClusterImpl(String name, + com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = new ClusterInner(); this.serviceManager = serviceManager; this.clusterName = name; @@ -233,49 +241,34 @@ public ClusterImpl update() { } public Cluster apply() { - this.innerObject = - serviceManager - .serviceClient() - .getClusters() - .update(resourceGroupName, clusterName, updateParameters, Context.NONE); + this.innerObject = serviceManager.serviceClient().getClusters().update(resourceGroupName, clusterName, + updateParameters, Context.NONE); return this; } public Cluster apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getClusters() - .update(resourceGroupName, clusterName, updateParameters, context); + this.innerObject = serviceManager.serviceClient().getClusters().update(resourceGroupName, clusterName, + updateParameters, context); return this; } - ClusterImpl( - ClusterInner innerObject, + ClusterImpl(ClusterInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.clusterName = Utils.getValueFromIdByName(innerObject.id(), "serverGroupsv2"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.clusterName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "serverGroupsv2"); } public Cluster refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getClusters() - .getByResourceGroupWithResponse(resourceGroupName, clusterName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getClusters() + .getByResourceGroupWithResponse(resourceGroupName, clusterName, Context.NONE).getValue(); return this; } public Cluster refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getClusters() - .getByResourceGroupWithResponse(resourceGroupName, clusterName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getClusters() + .getByResourceGroupWithResponse(resourceGroupName, clusterName, context).getValue(); return this; } @@ -307,8 +300,8 @@ public void promoteReadReplica() { serviceManager.clusters().promoteReadReplica(resourceGroupName, clusterName); } - public void promoteReadReplica(Context context) { - serviceManager.clusters().promoteReadReplica(resourceGroupName, clusterName, context); + public void promoteReadReplica(PromoteRequest promoteRequest, Context context) { + serviceManager.clusters().promoteReadReplica(resourceGroupName, clusterName, promoteRequest, context); } public ClusterImpl withRegion(Region location) { @@ -501,6 +494,21 @@ public ClusterImpl withPointInTimeUtc(OffsetDateTime pointInTimeUtc) { return this; } + public ClusterImpl withDatabaseName(String databaseName) { + this.innerModel().withDatabaseName(databaseName); + return this; + } + + public ClusterImpl withEnableGeoBackup(Boolean enableGeoBackup) { + this.innerModel().withEnableGeoBackup(enableGeoBackup); + return this; + } + + public ClusterImpl withAuthConfig(AuthConfig authConfig) { + this.innerModel().withAuthConfig(authConfig); + return this; + } + private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterServerImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterServerImpl.java index e35c4acace806..413ccee27f1ed 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterServerImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClusterServerImpl.java @@ -14,8 +14,7 @@ public final class ClusterServerImpl implements ClusterServer { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - ClusterServerImpl( - ClusterServerInner innerObject, + ClusterServerImpl(ClusterServerInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersClientImpl.java index 815d4af7b178a..c445bcf5c570b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersClientImpl.java @@ -39,21 +39,28 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterForUpdate; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterListResult; import com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailabilityRequest; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PromoteRequest; import java.nio.ByteBuffer; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ClustersClient. */ +/** + * An instance of this class provides access to all the operations defined in ClustersClient. + */ public final class ClustersClientImpl implements ClustersClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ClustersService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of ClustersClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ClustersClientImpl(CosmosDBForPostgreSqlImpl client) { @@ -68,180 +75,126 @@ public final class ClustersClientImpl implements ClustersClient { @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface ClustersService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @BodyParam("application/json") ClusterInner parameters, - @HeaderParam("Accept") String accept, + Mono>> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @BodyParam("application/json") ClusterInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") - @ExpectedResponses({200, 202, 204}) + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") - @ExpectedResponses({200, 202}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @BodyParam("application/json") ClusterForUpdate parameters, - @HeaderParam("Accept") String accept, + Mono>> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @BodyParam("application/json") ClusterForUpdate parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/restart") - @ExpectedResponses({202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/restart") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> restart( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/start") - @ExpectedResponses({200, 202}) + Mono>> restart(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/start") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> start( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/stop") - @ExpectedResponses({200, 202}) + Mono>> start(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/stop") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> stop( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/promote") - @ExpectedResponses({202}) + Mono>> stop(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/promote") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> promoteReadReplica( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, + Mono>> promoteReadReplica(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @BodyParam("application/json") PromoteRequest promoteRequest, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/providers/Microsoft.DBforPostgreSQL/checkNameAvailability") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> checkNameAvailability( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> checkNameAvailability(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @BodyParam("application/json") NameAvailabilityRequest nameAvailabilityRequest, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Lists all clusters in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of clusters along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -249,43 +202,25 @@ Mono> listByResourceGroupNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all clusters in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -295,40 +230,25 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all clusters in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of clusters as paginated response with {@link PagedFlux}. @@ -340,7 +260,7 @@ private PagedFlux listAsync() { /** * Lists all clusters in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -349,13 +269,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists all clusters in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of clusters as paginated response with {@link PagedIterable}. @@ -367,7 +287,7 @@ public PagedIterable list() { /** * Lists all clusters in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -381,7 +301,7 @@ public PagedIterable list(Context context) { /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -391,16 +311,12 @@ public PagedIterable list(Context context) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -408,31 +324,16 @@ private Mono> listByResourceGroupSinglePageAsync(Str } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -441,19 +342,15 @@ private Mono> listByResourceGroupSinglePageAsync(Str * @return a list of clusters along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -462,27 +359,15 @@ private Mono> listByResourceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -491,14 +376,13 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -508,14 +392,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupNam */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -529,7 +412,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName) /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -544,7 +427,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName, /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -554,19 +437,15 @@ public PagedIterable listByResourceGroup(String resourceGroupName, * @return represents a cluster along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, ClusterInner parameters) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + ClusterInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -582,24 +461,14 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - parameters, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -610,19 +479,15 @@ private Mono>> createWithResponseAsync( * @return represents a cluster along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + ClusterInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -638,21 +503,13 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - parameters, - accept, - context); + return service.create(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, parameters, accept, context); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -662,18 +519,16 @@ private Mono>> createWithResponseAsync( * @return the {@link PollerFlux} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginCreateAsync( - String resourceGroupName, String clusterName, ClusterInner parameters) { + private PollerFlux, ClusterInner> beginCreateAsync(String resourceGroupName, + String clusterName, ClusterInner parameters) { Mono>> mono = createWithResponseAsync(resourceGroupName, clusterName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, this.client.getContext()); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -684,20 +539,18 @@ private PollerFlux, ClusterInner> beginCreateAsync( * @return the {@link PollerFlux} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginCreateAsync( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { + private PollerFlux, ClusterInner> beginCreateAsync(String resourceGroupName, + String clusterName, ClusterInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createWithResponseAsync(resourceGroupName, clusterName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, context); + Mono>> mono + = createWithResponseAsync(resourceGroupName, clusterName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, context); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -707,14 +560,14 @@ private PollerFlux, ClusterInner> beginCreateAsync( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters) { + public SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters) { return this.beginCreateAsync(resourceGroupName, clusterName, parameters).getSyncPoller(); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -725,14 +578,14 @@ public SyncPoller, ClusterInner> beginCreate( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { + public SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters, Context context) { return this.beginCreateAsync(resourceGroupName, clusterName, parameters, context).getSyncPoller(); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -743,14 +596,13 @@ public SyncPoller, ClusterInner> beginCreate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono createAsync(String resourceGroupName, String clusterName, ClusterInner parameters) { - return beginCreateAsync(resourceGroupName, clusterName, parameters) - .last() + return beginCreateAsync(resourceGroupName, clusterName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -761,16 +613,15 @@ private Mono createAsync(String resourceGroupName, String clusterN * @return represents a cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { - return beginCreateAsync(resourceGroupName, clusterName, parameters, context) - .last() + private Mono createAsync(String resourceGroupName, String clusterName, ClusterInner parameters, + Context context) { + return beginCreateAsync(resourceGroupName, clusterName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -786,7 +637,7 @@ public ClusterInner create(String resourceGroupName, String clusterName, Cluster /** * Creates a new cluster with servers. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The required parameters for creating or updating a cluster. @@ -804,29 +655,25 @@ public ClusterInner create(String resourceGroupName, String clusterName, Cluster /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response} on successful completion of {@link Mono}. + * as cluster creation date and time along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String clusterName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -837,24 +684,15 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -862,22 +700,18 @@ private Mono> getByResourceGroupWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response} on successful completion of {@link Mono}. + * as cluster creation date and time along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -888,28 +722,21 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context); } /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time on successful completion of {@link Mono}. + * as cluster creation date and time on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getByResourceGroupAsync(String resourceGroupName, String clusterName) { @@ -920,7 +747,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, Str /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -928,25 +755,25 @@ private Mono getByResourceGroupAsync(String resourceGroupName, Str * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response}. + * as cluster creation date and time along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String clusterName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, clusterName, context).block(); } /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time. + * as cluster creation date and time. */ @ServiceMethod(returns = ReturnType.SINGLE) public ClusterInner getByResourceGroup(String resourceGroupName, String clusterName) { @@ -955,7 +782,7 @@ public ClusterInner getByResourceGroup(String resourceGroupName, String clusterN /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -966,16 +793,12 @@ public ClusterInner getByResourceGroup(String resourceGroupName, String clusterN @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -986,23 +809,14 @@ private Mono>> deleteWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1012,19 +826,15 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1035,20 +845,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context); } /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1059,15 +862,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, clusterName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1077,18 +878,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, clusterName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1103,7 +903,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1113,14 +913,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + Context context) { return this.beginDeleteAsync(resourceGroupName, clusterName, context).getSyncPoller(); } /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1135,7 +935,7 @@ private Mono deleteAsync(String resourceGroupName, String clusterName) { /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1146,14 +946,13 @@ private Mono deleteAsync(String resourceGroupName, String clusterName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, Context context) { - return beginDeleteAsync(resourceGroupName, clusterName, context) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1167,7 +966,7 @@ public void delete(String resourceGroupName, String clusterName) { /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1182,7 +981,7 @@ public void delete(String resourceGroupName, String clusterName, Context context /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1192,19 +991,15 @@ public void delete(String resourceGroupName, String clusterName, Context context * @return represents a cluster along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String clusterName, ClusterForUpdate parameters) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String clusterName, + ClusterForUpdate parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1220,24 +1015,14 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - parameters, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1248,19 +1033,15 @@ private Mono>> updateWithResponseAsync( * @return represents a cluster along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String clusterName, ClusterForUpdate parameters, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String clusterName, + ClusterForUpdate parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1276,21 +1057,13 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - parameters, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, parameters, accept, context); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1300,18 +1073,16 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginUpdateAsync( - String resourceGroupName, String clusterName, ClusterForUpdate parameters) { + private PollerFlux, ClusterInner> beginUpdateAsync(String resourceGroupName, + String clusterName, ClusterForUpdate parameters) { Mono>> mono = updateWithResponseAsync(resourceGroupName, clusterName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, this.client.getContext()); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1322,20 +1093,18 @@ private PollerFlux, ClusterInner> beginUpdateAsync( * @return the {@link PollerFlux} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginUpdateAsync( - String resourceGroupName, String clusterName, ClusterForUpdate parameters, Context context) { + private PollerFlux, ClusterInner> beginUpdateAsync(String resourceGroupName, + String clusterName, ClusterForUpdate parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, clusterName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, clusterName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, context); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1345,14 +1114,14 @@ private PollerFlux, ClusterInner> beginUpdateAsync( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterForUpdate parameters) { + public SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterForUpdate parameters) { return this.beginUpdateAsync(resourceGroupName, clusterName, parameters).getSyncPoller(); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1363,14 +1132,14 @@ public SyncPoller, ClusterInner> beginUpdate( * @return the {@link SyncPoller} for polling of represents a cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterForUpdate parameters, Context context) { + public SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterForUpdate parameters, Context context) { return this.beginUpdateAsync(resourceGroupName, clusterName, parameters, context).getSyncPoller(); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1381,14 +1150,13 @@ public SyncPoller, ClusterInner> beginUpdate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono updateAsync(String resourceGroupName, String clusterName, ClusterForUpdate parameters) { - return beginUpdateAsync(resourceGroupName, clusterName, parameters) - .last() + return beginUpdateAsync(resourceGroupName, clusterName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1399,16 +1167,15 @@ private Mono updateAsync(String resourceGroupName, String clusterN * @return represents a cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String clusterName, ClusterForUpdate parameters, Context context) { - return beginUpdateAsync(resourceGroupName, clusterName, parameters, context) - .last() + private Mono updateAsync(String resourceGroupName, String clusterName, ClusterForUpdate parameters, + Context context) { + return beginUpdateAsync(resourceGroupName, clusterName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1424,7 +1191,7 @@ public ClusterInner update(String resourceGroupName, String clusterName, Cluster /** * Updates an existing cluster. The request body can contain one or several properties from the cluster definition. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param parameters The parameters for updating a cluster. @@ -1435,14 +1202,14 @@ public ClusterInner update(String resourceGroupName, String clusterName, Cluster * @return represents a cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ClusterInner update( - String resourceGroupName, String clusterName, ClusterForUpdate parameters, Context context) { + public ClusterInner update(String resourceGroupName, String clusterName, ClusterForUpdate parameters, + Context context) { return updateAsync(resourceGroupName, clusterName, parameters, context).block(); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1453,16 +1220,12 @@ public ClusterInner update( @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> restartWithResponseAsync(String resourceGroupName, String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1473,23 +1236,14 @@ private Mono>> restartWithResponseAsync(String resourc } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .restart( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) + .withContext(context -> service.restart(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1499,19 +1253,15 @@ private Mono>> restartWithResponseAsync(String resourc * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> restartWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono>> restartWithResponseAsync(String resourceGroupName, String clusterName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1522,20 +1272,13 @@ private Mono>> restartWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .restart( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context); + return service.restart(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1546,15 +1289,13 @@ private Mono>> restartWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginRestartAsync(String resourceGroupName, String clusterName) { Mono>> mono = restartWithResponseAsync(resourceGroupName, clusterName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1564,18 +1305,17 @@ private PollerFlux, Void> beginRestartAsync(String resourceGrou * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginRestartAsync( - String resourceGroupName, String clusterName, Context context) { + private PollerFlux, Void> beginRestartAsync(String resourceGroupName, String clusterName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = restartWithResponseAsync(resourceGroupName, clusterName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1590,7 +1330,7 @@ public SyncPoller, Void> beginRestart(String resourceGroupName, /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1600,14 +1340,14 @@ public SyncPoller, Void> beginRestart(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginRestart( - String resourceGroupName, String clusterName, Context context) { + public SyncPoller, Void> beginRestart(String resourceGroupName, String clusterName, + Context context) { return this.beginRestartAsync(resourceGroupName, clusterName, context).getSyncPoller(); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1622,7 +1362,7 @@ private Mono restartAsync(String resourceGroupName, String clusterName) { /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1633,14 +1373,13 @@ private Mono restartAsync(String resourceGroupName, String clusterName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono restartAsync(String resourceGroupName, String clusterName, Context context) { - return beginRestartAsync(resourceGroupName, clusterName, context) - .last() + return beginRestartAsync(resourceGroupName, clusterName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1654,7 +1393,7 @@ public void restart(String resourceGroupName, String clusterName) { /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1669,7 +1408,7 @@ public void restart(String resourceGroupName, String clusterName, Context contex /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1680,16 +1419,12 @@ public void restart(String resourceGroupName, String clusterName, Context contex @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> startWithResponseAsync(String resourceGroupName, String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1700,23 +1435,14 @@ private Mono>> startWithResponseAsync(String resourceG } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .start( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) + .withContext(context -> service.start(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1726,19 +1452,15 @@ private Mono>> startWithResponseAsync(String resourceG * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> startWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono>> startWithResponseAsync(String resourceGroupName, String clusterName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1749,20 +1471,13 @@ private Mono>> startWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .start( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context); + return service.start(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context); } /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1773,15 +1488,13 @@ private Mono>> startWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginStartAsync(String resourceGroupName, String clusterName) { Mono>> mono = startWithResponseAsync(resourceGroupName, clusterName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1791,18 +1504,17 @@ private PollerFlux, Void> beginStartAsync(String resourceGroupN * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginStartAsync( - String resourceGroupName, String clusterName, Context context) { + private PollerFlux, Void> beginStartAsync(String resourceGroupName, String clusterName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = startWithResponseAsync(resourceGroupName, clusterName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1817,7 +1529,7 @@ public SyncPoller, Void> beginStart(String resourceGroupName, S /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1827,14 +1539,14 @@ public SyncPoller, Void> beginStart(String resourceGroupName, S * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginStart( - String resourceGroupName, String clusterName, Context context) { + public SyncPoller, Void> beginStart(String resourceGroupName, String clusterName, + Context context) { return this.beginStartAsync(resourceGroupName, clusterName, context).getSyncPoller(); } /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1849,7 +1561,7 @@ private Mono startAsync(String resourceGroupName, String clusterName) { /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1860,14 +1572,13 @@ private Mono startAsync(String resourceGroupName, String clusterName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono startAsync(String resourceGroupName, String clusterName, Context context) { - return beginStartAsync(resourceGroupName, clusterName, context) - .last() + return beginStartAsync(resourceGroupName, clusterName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1881,7 +1592,7 @@ public void start(String resourceGroupName, String clusterName) { /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1896,7 +1607,7 @@ public void start(String resourceGroupName, String clusterName, Context context) /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1907,16 +1618,12 @@ public void start(String resourceGroupName, String clusterName, Context context) @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> stopWithResponseAsync(String resourceGroupName, String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1927,23 +1634,14 @@ private Mono>> stopWithResponseAsync(String resourceGr } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .stop( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) + .withContext(context -> service.stop(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1953,19 +1651,15 @@ private Mono>> stopWithResponseAsync(String resourceGr * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono>> stopWithResponseAsync(String resourceGroupName, String clusterName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1976,20 +1670,13 @@ private Mono>> stopWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .stop( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context); + return service.stop(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context); } /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2000,15 +1687,13 @@ private Mono>> stopWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginStopAsync(String resourceGroupName, String clusterName) { Mono>> mono = stopWithResponseAsync(resourceGroupName, clusterName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -2018,18 +1703,17 @@ private PollerFlux, Void> beginStopAsync(String resourceGroupNa * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginStopAsync( - String resourceGroupName, String clusterName, Context context) { + private PollerFlux, Void> beginStopAsync(String resourceGroupName, String clusterName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = stopWithResponseAsync(resourceGroupName, clusterName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2044,7 +1728,7 @@ public SyncPoller, Void> beginStop(String resourceGroupName, St /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -2060,7 +1744,7 @@ public SyncPoller, Void> beginStop(String resourceGroupName, St /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2075,7 +1759,7 @@ private Mono stopAsync(String resourceGroupName, String clusterName) { /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -2086,14 +1770,13 @@ private Mono stopAsync(String resourceGroupName, String clusterName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono stopAsync(String resourceGroupName, String clusterName, Context context) { - return beginStopAsync(resourceGroupName, clusterName, context) - .last() + return beginStopAsync(resourceGroupName, clusterName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2107,7 +1790,7 @@ public void stop(String resourceGroupName, String clusterName) { /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -2122,28 +1805,25 @@ public void stop(String resourceGroupName, String clusterName, Context context) /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> promoteReadReplicaWithResponseAsync( - String resourceGroupName, String clusterName) { + private Mono>> promoteReadReplicaWithResponseAsync(String resourceGroupName, + String clusterName, PromoteRequest promoteRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2152,27 +1832,22 @@ private Mono>> promoteReadReplicaWithResponseAsync( if (clusterName == null) { return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } + if (promoteRequest != null) { + promoteRequest.validate(); + } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .promoteReadReplica( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) + .withContext(context -> service.promoteReadReplica(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, promoteRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2180,19 +1855,15 @@ private Mono>> promoteReadReplicaWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> promoteReadReplicaWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono>> promoteReadReplicaWithResponseAsync(String resourceGroupName, + String clusterName, PromoteRequest promoteRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2201,44 +1872,61 @@ private Mono>> promoteReadReplicaWithResponseAsync( if (clusterName == null) { return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } + if (promoteRequest != null) { + promoteRequest.validate(); + } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .promoteReadReplica( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context); + return service.promoteReadReplica(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, promoteRequest, accept, context); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginPromoteReadReplicaAsync( - String resourceGroupName, String clusterName) { - Mono>> mono = promoteReadReplicaWithResponseAsync(resourceGroupName, clusterName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginPromoteReadReplicaAsync(String resourceGroupName, + String clusterName, PromoteRequest promoteRequest) { + Mono>> mono + = promoteReadReplicaWithResponseAsync(resourceGroupName, clusterName, promoteRequest); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginPromoteReadReplicaAsync(String resourceGroupName, + String clusterName) { + final PromoteRequest promoteRequest = null; + Mono>> mono + = promoteReadReplicaWithResponseAsync(resourceGroupName, clusterName, promoteRequest); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Promotes read replica cluster to an independent read-write cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2246,19 +1934,18 @@ private PollerFlux, Void> beginPromoteReadReplicaAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginPromoteReadReplicaAsync( - String resourceGroupName, String clusterName, Context context) { + private PollerFlux, Void> beginPromoteReadReplicaAsync(String resourceGroupName, + String clusterName, PromoteRequest promoteRequest, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - promoteReadReplicaWithResponseAsync(resourceGroupName, clusterName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = promoteReadReplicaWithResponseAsync(resourceGroupName, clusterName, promoteRequest, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2268,14 +1955,16 @@ private PollerFlux, Void> beginPromoteReadReplicaAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, Void> beginPromoteReadReplica(String resourceGroupName, String clusterName) { - return this.beginPromoteReadReplicaAsync(resourceGroupName, clusterName).getSyncPoller(); + final PromoteRequest promoteRequest = null; + return this.beginPromoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest).getSyncPoller(); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2283,14 +1972,33 @@ public SyncPoller, Void> beginPromoteReadReplica(String resourc * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginPromoteReadReplica( - String resourceGroupName, String clusterName, Context context) { - return this.beginPromoteReadReplicaAsync(resourceGroupName, clusterName, context).getSyncPoller(); + public SyncPoller, Void> beginPromoteReadReplica(String resourceGroupName, String clusterName, + PromoteRequest promoteRequest, Context context) { + return this.beginPromoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest, context) + .getSyncPoller(); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono promoteReadReplicaAsync(String resourceGroupName, String clusterName, + PromoteRequest promoteRequest) { + return beginPromoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Promotes read replica cluster to an independent read-write cluster. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2300,16 +2008,17 @@ public SyncPoller, Void> beginPromoteReadReplica( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono promoteReadReplicaAsync(String resourceGroupName, String clusterName) { - return beginPromoteReadReplicaAsync(resourceGroupName, clusterName) - .last() + final PromoteRequest promoteRequest = null; + return beginPromoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2317,15 +2026,15 @@ private Mono promoteReadReplicaAsync(String resourceGroupName, String clus * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono promoteReadReplicaAsync(String resourceGroupName, String clusterName, Context context) { - return beginPromoteReadReplicaAsync(resourceGroupName, clusterName, context) - .last() + private Mono promoteReadReplicaAsync(String resourceGroupName, String clusterName, + PromoteRequest promoteRequest, Context context) { + return beginPromoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2334,70 +2043,60 @@ private Mono promoteReadReplicaAsync(String resourceGroupName, String clus */ @ServiceMethod(returns = ReturnType.SINGLE) public void promoteReadReplica(String resourceGroupName, String clusterName) { - promoteReadReplicaAsync(resourceGroupName, clusterName).block(); + final PromoteRequest promoteRequest = null; + promoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest).block(); } /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void promoteReadReplica(String resourceGroupName, String clusterName, Context context) { - promoteReadReplicaAsync(resourceGroupName, clusterName, context).block(); + public void promoteReadReplica(String resourceGroupName, String clusterName, PromoteRequest promoteRequest, + Context context) { + promoteReadReplicaAsync(resourceGroupName, clusterName, promoteRequest, context).block(); } /** * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents cluster name availability along with {@link Response} on successful completion of {@link - * Mono}. + * @return represents cluster name availability along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> checkNameAvailabilityWithResponseAsync( - NameAvailabilityRequest nameAvailabilityRequest) { + private Mono> + checkNameAvailabilityWithResponseAsync(NameAvailabilityRequest nameAvailabilityRequest) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (nameAvailabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException("Parameter nameAvailabilityRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter nameAvailabilityRequest is required and cannot be null.")); } else { nameAvailabilityRequest.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - nameAvailabilityRequest, - accept, - context)) + .withContext(context -> service.checkNameAvailability(this.client.getEndpoint(), + this.client.getApiVersion(), this.client.getSubscriptionId(), nameAvailabilityRequest, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -2405,54 +2104,43 @@ private Mono> checkNameAvailabilityWithResponseA * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return represents cluster name availability along with {@link Response} on successful completion of {@link - * Mono}. + * @return represents cluster name availability along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> checkNameAvailabilityWithResponseAsync( - NameAvailabilityRequest nameAvailabilityRequest, Context context) { + private Mono> + checkNameAvailabilityWithResponseAsync(NameAvailabilityRequest nameAvailabilityRequest, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (nameAvailabilityRequest == null) { - return Mono - .error( - new IllegalArgumentException("Parameter nameAvailabilityRequest is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter nameAvailabilityRequest is required and cannot be null.")); } else { nameAvailabilityRequest.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - nameAvailabilityRequest, - accept, - context); + return service.checkNameAvailability(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), nameAvailabilityRequest, accept, context); } /** * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2469,7 +2157,7 @@ private Mono checkNameAvailabilityAsync(NameAvailabilityR * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2478,8 +2166,8 @@ private Mono checkNameAvailabilityAsync(NameAvailabilityR * @return represents cluster name availability along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response checkNameAvailabilityWithResponse( - NameAvailabilityRequest nameAvailabilityRequest, Context context) { + public Response + checkNameAvailabilityWithResponse(NameAvailabilityRequest nameAvailabilityRequest, Context context) { return checkNameAvailabilityWithResponseAsync(nameAvailabilityRequest, context).block(); } @@ -2487,7 +2175,7 @@ public Response checkNameAvailabilityWithResponse( * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2501,9 +2189,10 @@ public NameAvailabilityInner checkNameAvailability(NameAvailabilityRequest nameA /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -2515,31 +2204,22 @@ private Mono> listNextSinglePageAsync(String nextLin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2552,31 +2232,22 @@ private Mono> listNextSinglePageAsync(String nextLin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -2588,32 +2259,24 @@ private Mono> listByResourceGroupNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2626,23 +2289,13 @@ private Mono> listByResourceGroupNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersImpl.java index 33c0bf1246d5e..29a866f954bd0 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ClustersImpl.java @@ -16,6 +16,7 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.Clusters; import com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailability; import com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailabilityRequest; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PromoteRequest; public final class ClustersImpl implements Clusters { private static final ClientLogger LOGGER = new ClientLogger(ClustersImpl.class); @@ -24,8 +25,7 @@ public final class ClustersImpl implements Clusters { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public ClustersImpl( - ClustersClient innerClient, + public ClustersImpl(ClustersClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -33,33 +33,30 @@ public ClustersImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String clusterName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, clusterName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, clusterName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ClusterImpl(inner.getValue(), this.manager())); } else { return null; @@ -111,19 +108,17 @@ public void promoteReadReplica(String resourceGroupName, String clusterName) { this.serviceClient().promoteReadReplica(resourceGroupName, clusterName); } - public void promoteReadReplica(String resourceGroupName, String clusterName, Context context) { - this.serviceClient().promoteReadReplica(resourceGroupName, clusterName, context); + public void promoteReadReplica(String resourceGroupName, String clusterName, PromoteRequest promoteRequest, + Context context) { + this.serviceClient().promoteReadReplica(resourceGroupName, clusterName, promoteRequest, context); } - public Response checkNameAvailabilityWithResponse( - NameAvailabilityRequest nameAvailabilityRequest, Context context) { - Response inner = - this.serviceClient().checkNameAvailabilityWithResponse(nameAvailabilityRequest, context); + public Response checkNameAvailabilityWithResponse(NameAvailabilityRequest nameAvailabilityRequest, + Context context) { + Response inner + = this.serviceClient().checkNameAvailabilityWithResponse(nameAvailabilityRequest, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new NameAvailabilityImpl(inner.getValue(), this.manager())); } else { return null; @@ -140,81 +135,57 @@ public NameAvailability checkNameAvailability(NameAvailabilityRequest nameAvaila } public Cluster getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, clusterName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, clusterName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } this.delete(resourceGroupName, clusterName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } this.delete(resourceGroupName, clusterName, context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationImpl.java index 87086695181fe..fe1f4aaab7218 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationImpl.java @@ -18,8 +18,7 @@ public final class ConfigurationImpl implements Configuration { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - ConfigurationImpl( - ConfigurationInner innerObject, + ConfigurationImpl(ConfigurationInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsClientImpl.java index e4d1d8119ec69..aacf263eafd80 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsClientImpl.java @@ -39,22 +39,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ConfigurationsClient. */ +/** + * An instance of this class provides access to all the operations defined in ConfigurationsClient. + */ public final class ConfigurationsClientImpl implements ConfigurationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ConfigurationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of ConfigurationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ConfigurationsClientImpl(CosmosDBForPostgreSqlImpl client) { - this.service = - RestProxy.create(ConfigurationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ConfigurationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -65,159 +71,115 @@ public final class ConfigurationsClientImpl implements ConfigurationsClient { @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface ConfigurationsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/servers/{serverName}/configurations") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/servers/{serverName}/configurations") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByServer( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("serverName") String serverName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/configurations") - @ExpectedResponses({200}) + Mono> listByServer(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("serverName") String serverName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/configurations") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/configurations/{configurationName}") - @ExpectedResponses({200}) + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/configurations/{configurationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("configurationName") String configurationName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("configurationName") String configurationName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/coordinatorConfigurations/{configurationName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/coordinatorConfigurations/{configurationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getCoordinator( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("configurationName") String configurationName, - @HeaderParam("Accept") String accept, + Mono> getCoordinator(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("configurationName") String configurationName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/coordinatorConfigurations/{configurationName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/coordinatorConfigurations/{configurationName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> updateOnCoordinator( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono>> updateOnCoordinator(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("configurationName") String configurationName, - @BodyParam("application/json") ServerConfigurationInner parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ServerConfigurationInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/nodeConfigurations/{configurationName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/nodeConfigurations/{configurationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getNode( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("configurationName") String configurationName, - @HeaderParam("Accept") String accept, + Mono> getNode(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("configurationName") String configurationName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/nodeConfigurations/{configurationName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/nodeConfigurations/{configurationName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> updateOnNode( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono>> updateOnNode(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("configurationName") String configurationName, - @BodyParam("application/json") ServerConfigurationInner parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ServerConfigurationInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByServerNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByClusterNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of server configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByServerSinglePageAsync( - String resourceGroupName, String clusterName, String serverName) { + private Mono> listByServerSinglePageAsync(String resourceGroupName, + String clusterName, String serverName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -231,33 +193,16 @@ private Mono> listByServerSinglePageAsyn } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByServer( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - serverName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByServer(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, serverName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -265,23 +210,19 @@ private Mono> listByServerSinglePageAsyn * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of server configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByServerSinglePageAsync( - String resourceGroupName, String clusterName, String serverName, Context context) { + private Mono> listByServerSinglePageAsync(String resourceGroupName, + String clusterName, String serverName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -296,29 +237,15 @@ private Mono> listByServerSinglePageAsyn final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByServer( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - serverName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByServer(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, serverName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -328,16 +255,15 @@ private Mono> listByServerSinglePageAsyn * @return a list of server configurations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByServerAsync( - String resourceGroupName, String clusterName, String serverName) { - return new PagedFlux<>( - () -> listByServerSinglePageAsync(resourceGroupName, clusterName, serverName), + private PagedFlux listByServerAsync(String resourceGroupName, String clusterName, + String serverName) { + return new PagedFlux<>(() -> listByServerSinglePageAsync(resourceGroupName, clusterName, serverName), nextLink -> listByServerNextSinglePageAsync(nextLink)); } /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -348,16 +274,15 @@ private PagedFlux listByServerAsync( * @return a list of server configurations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByServerAsync( - String resourceGroupName, String clusterName, String serverName, Context context) { - return new PagedFlux<>( - () -> listByServerSinglePageAsync(resourceGroupName, clusterName, serverName, context), + private PagedFlux listByServerAsync(String resourceGroupName, String clusterName, + String serverName, Context context) { + return new PagedFlux<>(() -> listByServerSinglePageAsync(resourceGroupName, clusterName, serverName, context), nextLink -> listByServerNextSinglePageAsync(nextLink, context)); } /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -367,14 +292,14 @@ private PagedFlux listByServerAsync( * @return a list of server configurations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName) { + public PagedIterable listByServer(String resourceGroupName, String clusterName, + String serverName) { return new PagedIterable<>(listByServerAsync(resourceGroupName, clusterName, serverName)); } /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -385,36 +310,32 @@ public PagedIterable listByServer( * @return a list of server configurations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName, Context context) { + public PagedIterable listByServer(String resourceGroupName, String clusterName, + String serverName, Context context) { return new PagedIterable<>(listByServerAsync(resourceGroupName, clusterName, serverName, context)); } /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -425,55 +346,35 @@ private Mono> listByClusterSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -485,28 +386,15 @@ private Mono> listByClusterSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -516,14 +404,13 @@ private Mono> listByClusterSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName) { - return new PagedFlux<>( - () -> listByClusterSinglePageAsync(resourceGroupName, clusterName), + return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName), nextLink -> listByClusterNextSinglePageAsync(nextLink)); } /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -533,16 +420,15 @@ private PagedFlux listByClusterAsync(String resourceGroupNam * @return a list of cluster configurations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByClusterAsync( - String resourceGroupName, String clusterName, Context context) { - return new PagedFlux<>( - () -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context), + private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, + Context context) { + return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context), nextLink -> listByClusterNextSinglePageAsync(nextLink, context)); } /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -557,7 +443,7 @@ public PagedIterable listByCluster(String resourceGroupName, /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -567,14 +453,14 @@ public PagedIterable listByCluster(String resourceGroupName, * @return a list of cluster configurations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listByClusterAsync(resourceGroupName, clusterName, context)); } /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -582,22 +468,18 @@ public PagedIterable listByCluster( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for coordinator and nodes along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String configurationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -612,24 +494,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -638,22 +510,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for coordinator and nodes along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String configurationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -668,21 +536,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, configurationName, accept, context); } /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -699,7 +559,7 @@ private Mono getAsync(String resourceGroupName, String clust /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -710,14 +570,14 @@ private Mono getAsync(String resourceGroupName, String clust * @return information of a configuration for coordinator and nodes along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context) { return getWithResponseAsync(resourceGroupName, clusterName, configurationName, context).block(); } /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -733,7 +593,7 @@ public ConfigurationInner get(String resourceGroupName, String clusterName, Stri /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -741,22 +601,18 @@ public ConfigurationInner get(String resourceGroupName, String clusterName, Stri * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for coordinator along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getCoordinatorWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName) { + private Mono> getCoordinatorWithResponseAsync(String resourceGroupName, + String clusterName, String configurationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -771,24 +627,14 @@ private Mono> getCoordinatorWithResponseAsync } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getCoordinator( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - accept, - context)) + .withContext(context -> service.getCoordinator(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -797,22 +643,18 @@ private Mono> getCoordinatorWithResponseAsync * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for coordinator along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getCoordinatorWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName, Context context) { + private Mono> getCoordinatorWithResponseAsync(String resourceGroupName, + String clusterName, String configurationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -827,21 +669,13 @@ private Mono> getCoordinatorWithResponseAsync } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getCoordinator( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - accept, - context); + return service.getCoordinator(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, accept, context); } /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -851,15 +685,15 @@ private Mono> getCoordinatorWithResponseAsync * @return information of a configuration for coordinator on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getCoordinatorAsync( - String resourceGroupName, String clusterName, String configurationName) { + private Mono getCoordinatorAsync(String resourceGroupName, String clusterName, + String configurationName) { return getCoordinatorWithResponseAsync(resourceGroupName, clusterName, configurationName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -870,14 +704,14 @@ private Mono getCoordinatorAsync( * @return information of a configuration for coordinator along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getCoordinatorWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context) { + public Response getCoordinatorWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context) { return getCoordinatorWithResponseAsync(resourceGroupName, clusterName, configurationName, context).block(); } /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -887,14 +721,14 @@ public Response getCoordinatorWithResponse( * @return information of a configuration for coordinator. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerConfigurationInner getCoordinator( - String resourceGroupName, String clusterName, String configurationName) { + public ServerConfigurationInner getCoordinator(String resourceGroupName, String clusterName, + String configurationName) { return getCoordinatorWithResponse(resourceGroupName, clusterName, configurationName, Context.NONE).getValue(); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -905,19 +739,15 @@ public ServerConfigurationInner getCoordinator( * @return represents a configuration along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateOnCoordinatorWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { + private Mono>> updateOnCoordinatorWithResponseAsync(String resourceGroupName, + String clusterName, String configurationName, ServerConfigurationInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -937,25 +767,15 @@ private Mono>> updateOnCoordinatorWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateOnCoordinator( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - parameters, - accept, - context)) + .withContext(context -> service.updateOnCoordinator(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -967,23 +787,15 @@ private Mono>> updateOnCoordinatorWithResponseAsync( * @return represents a configuration along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateOnCoordinatorWithResponseAsync( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { + private Mono>> updateOnCoordinatorWithResponseAsync(String resourceGroupName, + String clusterName, String configurationName, ServerConfigurationInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1003,22 +815,14 @@ private Mono>> updateOnCoordinatorWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateOnCoordinator( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - parameters, - accept, - context); + return service.updateOnCoordinator(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, parameters, accept, + context); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1031,21 +835,16 @@ private Mono>> updateOnCoordinatorWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ServerConfigurationInner> beginUpdateOnCoordinatorAsync( String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - Mono>> mono = - updateOnCoordinatorWithResponseAsync(resourceGroupName, clusterName, configurationName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ServerConfigurationInner.class, - ServerConfigurationInner.class, - this.client.getContext()); + Mono>> mono + = updateOnCoordinatorWithResponseAsync(resourceGroupName, clusterName, configurationName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ServerConfigurationInner.class, ServerConfigurationInner.class, + this.client.getContext()); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1058,28 +857,18 @@ private PollerFlux, ServerConfigurationInne */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ServerConfigurationInner> beginUpdateOnCoordinatorAsync( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, + String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateOnCoordinatorWithResponseAsync( - resourceGroupName, clusterName, configurationName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ServerConfigurationInner.class, - ServerConfigurationInner.class, - context); + Mono>> mono = updateOnCoordinatorWithResponseAsync(resourceGroupName, clusterName, + configurationName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ServerConfigurationInner.class, ServerConfigurationInner.class, context); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1092,14 +881,13 @@ private PollerFlux, ServerConfigurationInne @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ServerConfigurationInner> beginUpdateOnCoordinator( String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - return this - .beginUpdateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters) + return this.beginUpdateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters) .getSyncPoller(); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1112,10 +900,7 @@ public SyncPoller, ServerConfigurationInner */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ServerConfigurationInner> beginUpdateOnCoordinator( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, + String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters, Context context) { return this .beginUpdateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters, context) @@ -1124,7 +909,7 @@ public SyncPoller, ServerConfigurationInner /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1135,16 +920,15 @@ public SyncPoller, ServerConfigurationInner * @return represents a configuration on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateOnCoordinatorAsync( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - return beginUpdateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters) - .last() + private Mono updateOnCoordinatorAsync(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters) { + return beginUpdateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1156,20 +940,15 @@ private Mono updateOnCoordinatorAsync( * @return represents a configuration on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateOnCoordinatorAsync( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { + private Mono updateOnCoordinatorAsync(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters, Context context) { return beginUpdateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1180,14 +959,14 @@ private Mono updateOnCoordinatorAsync( * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerConfigurationInner updateOnCoordinator( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { + public ServerConfigurationInner updateOnCoordinator(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters) { return updateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters).block(); } /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1199,18 +978,14 @@ public ServerConfigurationInner updateOnCoordinator( * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerConfigurationInner updateOnCoordinator( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { + public ServerConfigurationInner updateOnCoordinator(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters, Context context) { return updateOnCoordinatorAsync(resourceGroupName, clusterName, configurationName, parameters, context).block(); } /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1218,22 +993,18 @@ public ServerConfigurationInner updateOnCoordinator( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for worker nodes along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getNodeWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName) { + private Mono> getNodeWithResponseAsync(String resourceGroupName, + String clusterName, String configurationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1248,24 +1019,14 @@ private Mono> getNodeWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getNode( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - accept, - context)) + .withContext(context -> service.getNode(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1274,22 +1035,18 @@ private Mono> getNodeWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for worker nodes along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getNodeWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName, Context context) { + private Mono> getNodeWithResponseAsync(String resourceGroupName, + String clusterName, String configurationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1304,21 +1061,13 @@ private Mono> getNodeWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getNode( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - accept, - context); + return service.getNode(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, configurationName, accept, context); } /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1328,15 +1077,15 @@ private Mono> getNodeWithResponseAsync( * @return information of a configuration for worker nodes on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getNodeAsync( - String resourceGroupName, String clusterName, String configurationName) { + private Mono getNodeAsync(String resourceGroupName, String clusterName, + String configurationName) { return getNodeWithResponseAsync(resourceGroupName, clusterName, configurationName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1347,14 +1096,14 @@ private Mono getNodeAsync( * @return information of a configuration for worker nodes along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getNodeWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context) { + public Response getNodeWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context) { return getNodeWithResponseAsync(resourceGroupName, clusterName, configurationName, context).block(); } /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1370,7 +1119,7 @@ public ServerConfigurationInner getNode(String resourceGroupName, String cluster /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1381,19 +1130,15 @@ public ServerConfigurationInner getNode(String resourceGroupName, String cluster * @return represents a configuration along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateOnNodeWithResponseAsync( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { + private Mono>> updateOnNodeWithResponseAsync(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1413,25 +1158,15 @@ private Mono>> updateOnNodeWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateOnNode( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - parameters, - accept, - context)) + .withContext(context -> service.updateOnNode(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1443,23 +1178,15 @@ private Mono>> updateOnNodeWithResponseAsync( * @return represents a configuration along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateOnNodeWithResponseAsync( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { + private Mono>> updateOnNodeWithResponseAsync(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1479,22 +1206,14 @@ private Mono>> updateOnNodeWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateOnNode( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - configurationName, - parameters, - accept, - context); + return service.updateOnNode(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, configurationName, parameters, accept, + context); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1507,21 +1226,16 @@ private Mono>> updateOnNodeWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ServerConfigurationInner> beginUpdateOnNodeAsync( String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - Mono>> mono = - updateOnNodeWithResponseAsync(resourceGroupName, clusterName, configurationName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ServerConfigurationInner.class, - ServerConfigurationInner.class, - this.client.getContext()); + Mono>> mono + = updateOnNodeWithResponseAsync(resourceGroupName, clusterName, configurationName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ServerConfigurationInner.class, ServerConfigurationInner.class, + this.client.getContext()); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1534,27 +1248,18 @@ private PollerFlux, ServerConfigurationInne */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, ServerConfigurationInner> beginUpdateOnNodeAsync( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, + String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateOnNodeWithResponseAsync(resourceGroupName, clusterName, configurationName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - ServerConfigurationInner.class, - ServerConfigurationInner.class, - context); + Mono>> mono + = updateOnNodeWithResponseAsync(resourceGroupName, clusterName, configurationName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), ServerConfigurationInner.class, ServerConfigurationInner.class, context); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1567,14 +1272,13 @@ private PollerFlux, ServerConfigurationInne @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ServerConfigurationInner> beginUpdateOnNode( String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - return this - .beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters) + return this.beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters) .getSyncPoller(); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1587,19 +1291,15 @@ public SyncPoller, ServerConfigurationInner */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, ServerConfigurationInner> beginUpdateOnNode( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, + String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters, Context context) { - return this - .beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters, context) + return this.beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters, context) .getSyncPoller(); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1610,16 +1310,15 @@ public SyncPoller, ServerConfigurationInner * @return represents a configuration on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateOnNodeAsync( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - return beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters) - .last() + private Mono updateOnNodeAsync(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters) { + return beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1631,20 +1330,15 @@ private Mono updateOnNodeAsync( * @return represents a configuration on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateOnNodeAsync( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { - return beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters, context) - .last() + private Mono updateOnNodeAsync(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters, Context context) { + return beginUpdateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1655,14 +1349,14 @@ private Mono updateOnNodeAsync( * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerConfigurationInner updateOnNode( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { + public ServerConfigurationInner updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters) { return updateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters).block(); } /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -1674,25 +1368,22 @@ public ServerConfigurationInner updateOnNode( * @return represents a configuration. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerConfigurationInner updateOnNode( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { + public ServerConfigurationInner updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters, Context context) { return updateOnNodeAsync(resourceGroupName, clusterName, configurationName, parameters, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of server configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByServerNextSinglePageAsync(String nextLink) { @@ -1700,75 +1391,58 @@ private Mono> listByServerNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByServerNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of server configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByServerNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByServerNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByServerNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByServerNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByClusterNextSinglePageAsync(String nextLink) { @@ -1776,37 +1450,29 @@ private Mono> listByClusterNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByClusterNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return a list of cluster configurations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByClusterNextSinglePageAsync(String nextLink, Context context) { @@ -1814,23 +1480,13 @@ private Mono> listByClusterNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByClusterNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByClusterNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsImpl.java index 748f606d5b3c5..29b2385c0ffa1 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ConfigurationsImpl.java @@ -23,47 +23,43 @@ public final class ConfigurationsImpl implements Configurations { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public ConfigurationsImpl( - ConfigurationsClient innerClient, + public ConfigurationsImpl(ConfigurationsClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName) { - PagedIterable inner = - this.serviceClient().listByServer(resourceGroupName, clusterName, serverName); - return Utils.mapPage(inner, inner1 -> new ServerConfigurationImpl(inner1, this.manager())); + public PagedIterable listByServer(String resourceGroupName, String clusterName, + String serverName) { + PagedIterable inner + = this.serviceClient().listByServer(resourceGroupName, clusterName, serverName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ServerConfigurationImpl(inner1, this.manager())); } - public PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName, Context context) { - PagedIterable inner = - this.serviceClient().listByServer(resourceGroupName, clusterName, serverName, context); - return Utils.mapPage(inner, inner1 -> new ServerConfigurationImpl(inner1, this.manager())); + public PagedIterable listByServer(String resourceGroupName, String clusterName, + String serverName, Context context) { + PagedIterable inner + = this.serviceClient().listByServer(resourceGroupName, clusterName, serverName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ServerConfigurationImpl(inner1, this.manager())); } public PagedIterable listByCluster(String resourceGroupName, String clusterName) { PagedIterable inner = this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new ConfigurationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConfigurationImpl(inner1, this.manager())); } public PagedIterable listByCluster(String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new ConfigurationImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConfigurationImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, clusterName, configurationName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, clusterName, configurationName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ConfigurationImpl(inner.getValue(), this.manager())); } else { return null; @@ -79,15 +75,12 @@ public Configuration get(String resourceGroupName, String clusterName, String co } } - public Response getCoordinatorWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context) { - Response inner = - this.serviceClient().getCoordinatorWithResponse(resourceGroupName, clusterName, configurationName, context); + public Response getCoordinatorWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context) { + Response inner = this.serviceClient().getCoordinatorWithResponse(resourceGroupName, + clusterName, configurationName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ServerConfigurationImpl(inner.getValue(), this.manager())); } else { return null; @@ -95,8 +88,8 @@ public Response getCoordinatorWithResponse( } public ServerConfiguration getCoordinator(String resourceGroupName, String clusterName, String configurationName) { - ServerConfigurationInner inner = - this.serviceClient().getCoordinator(resourceGroupName, clusterName, configurationName); + ServerConfigurationInner inner + = this.serviceClient().getCoordinator(resourceGroupName, clusterName, configurationName); if (inner != null) { return new ServerConfigurationImpl(inner, this.manager()); } else { @@ -104,10 +97,10 @@ public ServerConfiguration getCoordinator(String resourceGroupName, String clust } } - public ServerConfiguration updateOnCoordinator( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - ServerConfigurationInner inner = - this.serviceClient().updateOnCoordinator(resourceGroupName, clusterName, configurationName, parameters); + public ServerConfiguration updateOnCoordinator(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters) { + ServerConfigurationInner inner + = this.serviceClient().updateOnCoordinator(resourceGroupName, clusterName, configurationName, parameters); if (inner != null) { return new ServerConfigurationImpl(inner, this.manager()); } else { @@ -115,16 +108,10 @@ public ServerConfiguration updateOnCoordinator( } } - public ServerConfiguration updateOnCoordinator( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { - ServerConfigurationInner inner = - this - .serviceClient() - .updateOnCoordinator(resourceGroupName, clusterName, configurationName, parameters, context); + public ServerConfiguration updateOnCoordinator(String resourceGroupName, String clusterName, + String configurationName, ServerConfigurationInner parameters, Context context) { + ServerConfigurationInner inner = this.serviceClient().updateOnCoordinator(resourceGroupName, clusterName, + configurationName, parameters, context); if (inner != null) { return new ServerConfigurationImpl(inner, this.manager()); } else { @@ -132,15 +119,12 @@ public ServerConfiguration updateOnCoordinator( } } - public Response getNodeWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context) { - Response inner = - this.serviceClient().getNodeWithResponse(resourceGroupName, clusterName, configurationName, context); + public Response getNodeWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context) { + Response inner + = this.serviceClient().getNodeWithResponse(resourceGroupName, clusterName, configurationName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ServerConfigurationImpl(inner.getValue(), this.manager())); } else { return null; @@ -148,8 +132,8 @@ public Response getNodeWithResponse( } public ServerConfiguration getNode(String resourceGroupName, String clusterName, String configurationName) { - ServerConfigurationInner inner = - this.serviceClient().getNode(resourceGroupName, clusterName, configurationName); + ServerConfigurationInner inner + = this.serviceClient().getNode(resourceGroupName, clusterName, configurationName); if (inner != null) { return new ServerConfigurationImpl(inner, this.manager()); } else { @@ -157,10 +141,10 @@ public ServerConfiguration getNode(String resourceGroupName, String clusterName, } } - public ServerConfiguration updateOnNode( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters) { - ServerConfigurationInner inner = - this.serviceClient().updateOnNode(resourceGroupName, clusterName, configurationName, parameters); + public ServerConfiguration updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters) { + ServerConfigurationInner inner + = this.serviceClient().updateOnNode(resourceGroupName, clusterName, configurationName, parameters); if (inner != null) { return new ServerConfigurationImpl(inner, this.manager()); } else { @@ -168,14 +152,10 @@ public ServerConfiguration updateOnNode( } } - public ServerConfiguration updateOnNode( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context) { - ServerConfigurationInner inner = - this.serviceClient().updateOnNode(resourceGroupName, clusterName, configurationName, parameters, context); + public ServerConfiguration updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters, Context context) { + ServerConfigurationInner inner + = this.serviceClient().updateOnNode(resourceGroupName, clusterName, configurationName, parameters, context); if (inner != null) { return new ServerConfigurationImpl(inner, this.manager()); } else { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlBuilder.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlBuilder.java index c9ccadc2ab01f..9dd354f01476d 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlBuilder.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlBuilder.java @@ -14,8 +14,10 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the CosmosDBForPostgreSqlImpl type. */ -@ServiceClientBuilder(serviceClients = {CosmosDBForPostgreSqlImpl.class}) +/** + * A builder for creating a new instance of the CosmosDBForPostgreSqlImpl type. + */ +@ServiceClientBuilder(serviceClients = { CosmosDBForPostgreSqlImpl.class }) public final class CosmosDBForPostgreSqlBuilder { /* * The ID of the target subscription. The value must be an UUID. @@ -24,7 +26,7 @@ public final class CosmosDBForPostgreSqlBuilder { /** * Sets The ID of the target subscription. The value must be an UUID. - * + * * @param subscriptionId the subscriptionId value. * @return the CosmosDBForPostgreSqlBuilder. */ @@ -40,7 +42,7 @@ public CosmosDBForPostgreSqlBuilder subscriptionId(String subscriptionId) { /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the CosmosDBForPostgreSqlBuilder. */ @@ -56,7 +58,7 @@ public CosmosDBForPostgreSqlBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the CosmosDBForPostgreSqlBuilder. */ @@ -72,7 +74,7 @@ public CosmosDBForPostgreSqlBuilder environment(AzureEnvironment environment) { /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the CosmosDBForPostgreSqlBuilder. */ @@ -88,7 +90,7 @@ public CosmosDBForPostgreSqlBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the CosmosDBForPostgreSqlBuilder. */ @@ -104,7 +106,7 @@ public CosmosDBForPostgreSqlBuilder defaultPollInterval(Duration defaultPollInte /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the CosmosDBForPostgreSqlBuilder. */ @@ -115,30 +117,20 @@ public CosmosDBForPostgreSqlBuilder serializerAdapter(SerializerAdapter serializ /** * Builds an instance of CosmosDBForPostgreSqlImpl with the provided parameters. - * + * * @return an instance of CosmosDBForPostgreSqlImpl. */ public CosmosDBForPostgreSqlImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - CosmosDBForPostgreSqlImpl client = - new CosmosDBForPostgreSqlImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - this.subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + CosmosDBForPostgreSqlImpl client = new CosmosDBForPostgreSqlImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlImpl.java index de035b0fc56a0..f26d7019bb9f5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/CosmosDBForPostgreSqlImpl.java @@ -40,171 +40,201 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the CosmosDBForPostgreSqlImpl type. */ +/** + * Initializes a new instance of the CosmosDBForPostgreSqlImpl type. + */ @ServiceClient(builder = CosmosDBForPostgreSqlBuilder.class) public final class CosmosDBForPostgreSqlImpl implements CosmosDBForPostgreSql { - /** The ID of the target subscription. The value must be an UUID. */ + /** + * The ID of the target subscription. The value must be an UUID. + */ private final String subscriptionId; /** * Gets The ID of the target subscription. The value must be an UUID. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The ClustersClient object to access its operations. */ + /** + * The ClustersClient object to access its operations. + */ private final ClustersClient clusters; /** * Gets the ClustersClient object to access its operations. - * + * * @return the ClustersClient object. */ public ClustersClient getClusters() { return this.clusters; } - /** The ServersClient object to access its operations. */ + /** + * The ServersClient object to access its operations. + */ private final ServersClient servers; /** * Gets the ServersClient object to access its operations. - * + * * @return the ServersClient object. */ public ServersClient getServers() { return this.servers; } - /** The ConfigurationsClient object to access its operations. */ + /** + * The ConfigurationsClient object to access its operations. + */ private final ConfigurationsClient configurations; /** * Gets the ConfigurationsClient object to access its operations. - * + * * @return the ConfigurationsClient object. */ public ConfigurationsClient getConfigurations() { return this.configurations; } - /** The FirewallRulesClient object to access its operations. */ + /** + * The FirewallRulesClient object to access its operations. + */ private final FirewallRulesClient firewallRules; /** * Gets the FirewallRulesClient object to access its operations. - * + * * @return the FirewallRulesClient object. */ public FirewallRulesClient getFirewallRules() { return this.firewallRules; } - /** The RolesClient object to access its operations. */ + /** + * The RolesClient object to access its operations. + */ private final RolesClient roles; /** * Gets the RolesClient object to access its operations. - * + * * @return the RolesClient object. */ public RolesClient getRoles() { return this.roles; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The PrivateEndpointConnectionsClient object to access its operations. */ + /** + * The PrivateEndpointConnectionsClient object to access its operations. + */ private final PrivateEndpointConnectionsClient privateEndpointConnections; /** * Gets the PrivateEndpointConnectionsClient object to access its operations. - * + * * @return the PrivateEndpointConnectionsClient object. */ public PrivateEndpointConnectionsClient getPrivateEndpointConnections() { return this.privateEndpointConnections; } - /** The PrivateLinkResourcesClient object to access its operations. */ + /** + * The PrivateLinkResourcesClient object to access its operations. + */ private final PrivateLinkResourcesClient privateLinkResources; /** * Gets the PrivateLinkResourcesClient object to access its operations. - * + * * @return the PrivateLinkResourcesClient object. */ public PrivateLinkResourcesClient getPrivateLinkResources() { @@ -213,7 +243,7 @@ public PrivateLinkResourcesClient getPrivateLinkResources() { /** * Initializes an instance of CosmosDBForPostgreSql client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -221,19 +251,14 @@ public PrivateLinkResourcesClient getPrivateLinkResources() { * @param subscriptionId The ID of the target subscription. The value must be an UUID. * @param endpoint server parameter. */ - CosmosDBForPostgreSqlImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + CosmosDBForPostgreSqlImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2022-11-08"; + this.apiVersion = "2023-03-02-preview"; this.clusters = new ClustersClientImpl(this); this.servers = new ServersClientImpl(this); this.configurations = new ConfigurationsClientImpl(this); @@ -246,7 +271,7 @@ public PrivateLinkResourcesClient getPrivateLinkResources() { /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -255,7 +280,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -265,7 +290,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -275,26 +300,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -307,19 +321,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRuleImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRuleImpl.java index 452700b6f702e..e7e4aa319be87 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRuleImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRuleImpl.java @@ -68,25 +68,19 @@ public FirewallRuleImpl withExistingServerGroupsv2(String resourceGroupName, Str } public FirewallRule create() { - this.innerObject = - serviceManager - .serviceClient() - .getFirewallRules() - .createOrUpdate(resourceGroupName, clusterName, firewallRuleName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getFirewallRules().createOrUpdate(resourceGroupName, + clusterName, firewallRuleName, this.innerModel(), Context.NONE); return this; } public FirewallRule create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getFirewallRules() - .createOrUpdate(resourceGroupName, clusterName, firewallRuleName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getFirewallRules().createOrUpdate(resourceGroupName, + clusterName, firewallRuleName, this.innerModel(), context); return this; } - FirewallRuleImpl( - String name, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { + FirewallRuleImpl(String name, + com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = new FirewallRuleInner(); this.serviceManager = serviceManager; this.firewallRuleName = name; @@ -97,50 +91,35 @@ public FirewallRuleImpl update() { } public FirewallRule apply() { - this.innerObject = - serviceManager - .serviceClient() - .getFirewallRules() - .createOrUpdate(resourceGroupName, clusterName, firewallRuleName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getFirewallRules().createOrUpdate(resourceGroupName, + clusterName, firewallRuleName, this.innerModel(), Context.NONE); return this; } public FirewallRule apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getFirewallRules() - .createOrUpdate(resourceGroupName, clusterName, firewallRuleName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getFirewallRules().createOrUpdate(resourceGroupName, + clusterName, firewallRuleName, this.innerModel(), context); return this; } - FirewallRuleImpl( - FirewallRuleInner innerObject, + FirewallRuleImpl(FirewallRuleInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.clusterName = Utils.getValueFromIdByName(innerObject.id(), "serverGroupsv2"); - this.firewallRuleName = Utils.getValueFromIdByName(innerObject.id(), "firewallRules"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.clusterName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "serverGroupsv2"); + this.firewallRuleName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "firewallRules"); } public FirewallRule refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getFirewallRules() - .getWithResponse(resourceGroupName, clusterName, firewallRuleName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirewallRules() + .getWithResponse(resourceGroupName, clusterName, firewallRuleName, Context.NONE).getValue(); return this; } public FirewallRule refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getFirewallRules() - .getWithResponse(resourceGroupName, clusterName, firewallRuleName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirewallRules() + .getWithResponse(resourceGroupName, clusterName, firewallRuleName, context).getValue(); return this; } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesClientImpl.java index ab92fa3d6bc75..27c382fc6fd69 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesClientImpl.java @@ -38,96 +38,82 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in FirewallRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in FirewallRulesClient. + */ public final class FirewallRulesClientImpl implements FirewallRulesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final FirewallRulesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of FirewallRulesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ FirewallRulesClientImpl(CosmosDBForPostgreSqlImpl client) { - this.service = - RestProxy.create(FirewallRulesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(FirewallRulesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for CosmosDBForPostgreSqlFirewallRules to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for CosmosDBForPostgreSqlFirewallRules to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface FirewallRulesService { - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules/{firewallRuleName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules/{firewallRuleName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("firewallRuleName") String firewallRuleName, - @BodyParam("application/json") FirewallRuleInner parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") FirewallRuleInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules/{firewallRuleName}") - @ExpectedResponses({202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules/{firewallRuleName}") + @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("firewallRuleName") String firewallRuleName, - @HeaderParam("Accept") String accept, + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("firewallRuleName") String firewallRuleName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules/{firewallRuleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules/{firewallRuleName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("firewallRuleName") String firewallRuleName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("firewallRuleName") String firewallRuleName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/firewallRules") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -138,19 +124,15 @@ Mono> listByCluster( * @return represents a cluster firewall rule along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String clusterName, String firewallRuleName, FirewallRuleInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -170,25 +152,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - firewallRuleName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, firewallRuleName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -200,23 +172,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return represents a cluster firewall rule along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String clusterName, String firewallRuleName, FirewallRuleInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -236,22 +200,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - firewallRuleName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, firewallRuleName, parameters, accept, + context); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -264,21 +220,15 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, FirewallRuleInner> beginCreateOrUpdateAsync( String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - FirewallRuleInner.class, - FirewallRuleInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + FirewallRuleInner.class, FirewallRuleInner.class, this.client.getContext()); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -291,23 +241,18 @@ private PollerFlux, FirewallRuleInner> beginCreate */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, FirewallRuleInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, + String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), FirewallRuleInner.class, FirewallRuleInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + FirewallRuleInner.class, FirewallRuleInner.class, context); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -318,16 +263,15 @@ private PollerFlux, FirewallRuleInner> beginCreate * @return the {@link SyncPoller} for polling of represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, FirewallRuleInner> beginCreateOrUpdate( - String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters) + public SyncPoller, FirewallRuleInner> beginCreateOrUpdate(String resourceGroupName, + String clusterName, String firewallRuleName, FirewallRuleInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters) .getSyncPoller(); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -339,20 +283,15 @@ public SyncPoller, FirewallRuleInner> beginCreateO * @return the {@link SyncPoller} for polling of represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, FirewallRuleInner> beginCreateOrUpdate( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context) + public SyncPoller, FirewallRuleInner> beginCreateOrUpdate(String resourceGroupName, + String clusterName, String firewallRuleName, FirewallRuleInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context) .getSyncPoller(); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -363,16 +302,15 @@ public SyncPoller, FirewallRuleInner> beginCreateO * @return represents a cluster firewall rule on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String clusterName, + String firewallRuleName, FirewallRuleInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -384,20 +322,15 @@ private Mono createOrUpdateAsync( * @return represents a cluster firewall rule on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, - Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String clusterName, + String firewallRuleName, FirewallRuleInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -408,14 +341,14 @@ private Mono createOrUpdateAsync( * @return represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.SINGLE) - public FirewallRuleInner createOrUpdate( - String resourceGroupName, String clusterName, String firewallRuleName, FirewallRuleInner parameters) { + public FirewallRuleInner createOrUpdate(String resourceGroupName, String clusterName, String firewallRuleName, + FirewallRuleInner parameters) { return createOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters).block(); } /** * Creates a new cluster firewall rule or updates an existing cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -427,18 +360,14 @@ public FirewallRuleInner createOrUpdate( * @return represents a cluster firewall rule. */ @ServiceMethod(returns = ReturnType.SINGLE) - public FirewallRuleInner createOrUpdate( - String resourceGroupName, - String clusterName, - String firewallRuleName, - FirewallRuleInner parameters, - Context context) { + public FirewallRuleInner createOrUpdate(String resourceGroupName, String clusterName, String firewallRuleName, + FirewallRuleInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, clusterName, firewallRuleName, parameters, context).block(); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -448,19 +377,15 @@ public FirewallRuleInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String firewallRuleName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String firewallRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -475,24 +400,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - firewallRuleName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, firewallRuleName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -503,19 +418,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String firewallRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -530,21 +441,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - firewallRuleName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, firewallRuleName, accept, context); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -554,19 +457,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String firewallRuleName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, clusterName, firewallRuleName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String firewallRuleName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, firewallRuleName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -577,19 +478,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String firewallRuleName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -599,14 +499,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String firewallRuleName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String firewallRuleName) { return this.beginDeleteAsync(resourceGroupName, clusterName, firewallRuleName).getSyncPoller(); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -617,14 +517,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String firewallRuleName, Context context) { return this.beginDeleteAsync(resourceGroupName, clusterName, firewallRuleName, context).getSyncPoller(); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -635,14 +535,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String firewallRuleName) { - return beginDeleteAsync(resourceGroupName, clusterName, firewallRuleName) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, firewallRuleName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -653,16 +552,15 @@ private Mono deleteAsync(String resourceGroupName, String clusterName, Str * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { - return beginDeleteAsync(resourceGroupName, clusterName, firewallRuleName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String clusterName, String firewallRuleName, + Context context) { + return beginDeleteAsync(resourceGroupName, clusterName, firewallRuleName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -677,7 +575,7 @@ public void delete(String resourceGroupName, String clusterName, String firewall /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -693,30 +591,26 @@ public void delete(String resourceGroupName, String clusterName, String firewall /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about a cluster firewall rule along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about a cluster firewall rule along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String firewallRuleName) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String firewallRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -731,24 +625,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - firewallRuleName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, firewallRuleName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -756,23 +640,19 @@ private Mono> getWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about a cluster firewall rule along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about a cluster firewall rule along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String firewallRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -787,21 +667,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - firewallRuleName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, firewallRuleName, accept, context); } /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -818,7 +690,7 @@ private Mono getAsync(String resourceGroupName, String cluste /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -829,14 +701,14 @@ private Mono getAsync(String resourceGroupName, String cluste * @return information about a cluster firewall rule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, + String firewallRuleName, Context context) { return getWithResponseAsync(resourceGroupName, clusterName, firewallRuleName, context).block(); } /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -852,7 +724,7 @@ public FirewallRuleInner get(String resourceGroupName, String clusterName, Strin /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -861,19 +733,15 @@ public FirewallRuleInner get(String resourceGroupName, String clusterName, Strin * @return a list of firewall rules along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -884,27 +752,16 @@ private Mono> listByClusterSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -914,19 +771,15 @@ private Mono> listByClusterSinglePageAsync( * @return a list of firewall rules along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -938,23 +791,15 @@ private Mono> listByClusterSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -969,7 +814,7 @@ private PagedFlux listByClusterAsync(String resourceGroupName /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -979,14 +824,14 @@ private PagedFlux listByClusterAsync(String resourceGroupName * @return a list of firewall rules as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByClusterAsync( - String resourceGroupName, String clusterName, Context context) { + private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, + Context context) { return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context)); } /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1001,7 +846,7 @@ public PagedIterable listByCluster(String resourceGroupName, /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -1011,8 +856,8 @@ public PagedIterable listByCluster(String resourceGroupName, * @return a list of firewall rules as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listByClusterAsync(resourceGroupName, clusterName, context)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesImpl.java index b04927941661d..2594afae4fed4 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/FirewallRulesImpl.java @@ -21,8 +21,7 @@ public final class FirewallRulesImpl implements FirewallRules { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public FirewallRulesImpl( - FirewallRulesClient innerClient, + public FirewallRulesImpl(FirewallRulesClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -36,15 +35,12 @@ public void delete(String resourceGroupName, String clusterName, String firewall this.serviceClient().delete(resourceGroupName, clusterName, firewallRuleName, context); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String firewallRuleName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, clusterName, firewallRuleName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, String firewallRuleName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, clusterName, firewallRuleName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new FirewallRuleImpl(inner.getValue(), this.manager())); } else { return null; @@ -62,119 +58,87 @@ public FirewallRule get(String resourceGroupName, String clusterName, String fir public PagedIterable listByCluster(String resourceGroupName, String clusterName) { PagedIterable inner = this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new FirewallRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new FirewallRuleImpl(inner1, this.manager())); } public PagedIterable listByCluster(String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new FirewallRuleImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new FirewallRuleImpl(inner1, this.manager())); } public FirewallRule getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String firewallRuleName = Utils.getValueFromIdByName(id, "firewallRules"); + String firewallRuleName = ResourceManagerUtils.getValueFromIdByName(id, "firewallRules"); if (firewallRuleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, firewallRuleName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String firewallRuleName = Utils.getValueFromIdByName(id, "firewallRules"); + String firewallRuleName = ResourceManagerUtils.getValueFromIdByName(id, "firewallRules"); if (firewallRuleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, firewallRuleName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String firewallRuleName = Utils.getValueFromIdByName(id, "firewallRules"); + String firewallRuleName = ResourceManagerUtils.getValueFromIdByName(id, "firewallRules"); if (firewallRuleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); } this.delete(resourceGroupName, clusterName, firewallRuleName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String firewallRuleName = Utils.getValueFromIdByName(id, "firewallRules"); + String firewallRuleName = ResourceManagerUtils.getValueFromIdByName(id, "firewallRules"); if (firewallRuleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firewallRules'.", id))); } this.delete(resourceGroupName, clusterName, firewallRuleName, context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/NameAvailabilityImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/NameAvailabilityImpl.java index e9f94f103b082..f9e9d6908c0b9 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/NameAvailabilityImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/NameAvailabilityImpl.java @@ -12,8 +12,7 @@ public final class NameAvailabilityImpl implements NameAvailability { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - NameAvailabilityImpl( - NameAvailabilityInner innerObject, + NameAvailabilityImpl(NameAvailabilityInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationImpl.java index ff8cb1f99cbb5..fa818cca87026 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationImpl.java @@ -16,8 +16,7 @@ public final class OperationImpl implements Operation { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - OperationImpl( - OperationInner innerObject, + OperationImpl(OperationInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsClientImpl.java index 202d943a9c513..e816650e89973 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsClientImpl.java @@ -29,22 +29,28 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(CosmosDBForPostgreSqlImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -55,75 +61,63 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.DBforPostgreSQL/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Lists all of the available REST API operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of resource provider operations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all of the available REST API operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of resource provider operations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Lists all of the available REST API operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of resource provider operations as paginated response with {@link PagedFlux}. @@ -135,7 +129,7 @@ private PagedFlux listAsync() { /** * Lists all of the available REST API operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -149,7 +143,7 @@ private PagedFlux listAsync(Context context) { /** * Lists all of the available REST API operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of resource provider operations as paginated response with {@link PagedIterable}. @@ -161,7 +155,7 @@ public PagedIterable list() { /** * Lists all of the available REST API operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsImpl.java index 12183dbf0f210..d085d7a9771f4 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/OperationsImpl.java @@ -19,8 +19,7 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, + public OperationsImpl(OperationsClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public OperationsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionImpl.java index 6f8fd99aa5598..2b548ca857d30 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionImpl.java @@ -82,27 +82,19 @@ public PrivateEndpointConnectionImpl withExistingServerGroupsv2(String resourceG } public PrivateEndpointConnection create() { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .createOrUpdate( - resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections().createOrUpdate( + resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); return this; } public PrivateEndpointConnection create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .createOrUpdate( - resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections() + .createOrUpdate(resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), context); return this; } - PrivateEndpointConnectionImpl( - String name, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { + PrivateEndpointConnectionImpl(String name, + com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = new PrivateEndpointConnectionInner(); this.serviceManager = serviceManager; this.privateEndpointConnectionName = name; @@ -113,52 +105,36 @@ public PrivateEndpointConnectionImpl update() { } public PrivateEndpointConnection apply() { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .createOrUpdate( - resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections().createOrUpdate( + resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); return this; } public PrivateEndpointConnection apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .createOrUpdate( - resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections() + .createOrUpdate(resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), context); return this; } - PrivateEndpointConnectionImpl( - PrivateEndpointConnectionInner innerObject, + PrivateEndpointConnectionImpl(PrivateEndpointConnectionInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.clusterName = Utils.getValueFromIdByName(innerObject.id(), "serverGroupsv2"); - this.privateEndpointConnectionName = Utils.getValueFromIdByName(innerObject.id(), "privateEndpointConnections"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.clusterName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "serverGroupsv2"); + this.privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "privateEndpointConnections"); } public PrivateEndpointConnection refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections() + .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE).getValue(); return this; } public PrivateEndpointConnection refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections() + .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context).getValue(); return this; } @@ -167,8 +143,8 @@ public PrivateEndpointConnectionImpl withPrivateEndpoint(PrivateEndpoint private return this; } - public PrivateEndpointConnectionImpl withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionImpl + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { this.innerModel().withPrivateLinkServiceConnectionState(privateLinkServiceConnectionState); return this; } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsClientImpl.java index 868c7106a8db2..44ea5dad7f2ba 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsClientImpl.java @@ -38,24 +38,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. + */ public final class PrivateEndpointConnectionsClientImpl implements PrivateEndpointConnectionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PrivateEndpointConnectionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of PrivateEndpointConnectionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PrivateEndpointConnectionsClientImpl(CosmosDBForPostgreSqlImpl client) { - this.service = - RestProxy - .create( - PrivateEndpointConnectionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(PrivateEndpointConnectionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -66,92 +70,68 @@ public final class PrivateEndpointConnectionsClientImpl implements PrivateEndpoi @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface PrivateEndpointConnectionsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") - @ExpectedResponses({200}) + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, @BodyParam("application/json") PrivateEndpointConnectionInner parameters, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") - @ExpectedResponses({202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connections on a cluster along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -162,27 +142,16 @@ private Mono> listByClusterSingleP } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -190,22 +159,18 @@ private Mono> listByClusterSingleP * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connections on a cluster along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -217,23 +182,15 @@ private Mono> listByClusterSingleP final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -248,7 +205,7 @@ private PagedFlux listByClusterAsync(String reso /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -258,14 +215,14 @@ private PagedFlux listByClusterAsync(String reso * @return list of private endpoint connections on a cluster as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByClusterAsync( - String resourceGroupName, String clusterName, Context context) { + private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, + Context context) { return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context)); } /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -280,7 +237,7 @@ public PagedIterable listByCluster(String resour /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -290,14 +247,14 @@ public PagedIterable listByCluster(String resour * @return list of private endpoint connections on a cluster as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listByClusterAsync(resourceGroupName, clusterName, context)); } /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -307,19 +264,15 @@ public PagedIterable listByCluster( * @return private endpoint connection along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String clusterName, String privateEndpointConnectionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -329,31 +282,20 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, privateEndpointConnectionName, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -364,19 +306,15 @@ private Mono> getWithResponseAsync( * @return private endpoint connection along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String clusterName, String privateEndpointConnectionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -386,28 +324,18 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, privateEndpointConnectionName, accept, context); } /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -417,15 +345,15 @@ private Mono> getWithResponseAsync( * @return private endpoint connection on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + private Mono getAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { return getWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -436,14 +364,14 @@ private Mono getAsync( * @return private endpoint connection along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { return getWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).block(); } /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -453,14 +381,14 @@ public Response getWithResponse( * @return private endpoint connection. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PrivateEndpointConnectionInner get( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + public PrivateEndpointConnectionInner get(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { return getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE).getValue(); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -468,26 +396,19 @@ public PrivateEndpointConnectionInner get( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the private endpoint connection resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return the private endpoint connection resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -497,10 +418,8 @@ private Mono>> createOrUpdateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -509,25 +428,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, privateEndpointConnectionName, + parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -536,27 +445,20 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the private endpoint connection resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return the private endpoint connection resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -566,10 +468,8 @@ private Mono>> createOrUpdateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -578,22 +478,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, privateEndpointConnectionName, parameters, + accept, context); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -605,26 +497,18 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PrivateEndpointConnectionInner> - beginCreateOrUpdateAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, + beginCreateOrUpdateAsync(String resourceGroupName, String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PrivateEndpointConnectionInner.class, - PrivateEndpointConnectionInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, clusterName, + privateEndpointConnectionName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PrivateEndpointConnectionInner.class, PrivateEndpointConnectionInner.class, + this.client.getContext()); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -637,29 +521,19 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PrivateEndpointConnectionInner> - beginCreateOrUpdateAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, - Context context) { + beginCreateOrUpdateAsync(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + PrivateEndpointConnectionInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, clusterName, privateEndpointConnectionName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PrivateEndpointConnectionInner.class, - PrivateEndpointConnectionInner.class, - context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, clusterName, + privateEndpointConnectionName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PrivateEndpointConnectionInner.class, PrivateEndpointConnectionInner.class, + context); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -671,18 +545,15 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PrivateEndpointConnectionInner> beginCreateOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, + String resourceGroupName, String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters) + return this.beginCreateOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters) .getSyncPoller(); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -695,20 +566,15 @@ public SyncPoller, PrivateEndpointCon */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PrivateEndpointConnectionInner> beginCreateOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, clusterName, privateEndpointConnectionName, parameters, context) - .getSyncPoller(); + String resourceGroupName, String clusterName, String privateEndpointConnectionName, + PrivateEndpointConnectionInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters, + context).getSyncPoller(); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -719,19 +585,15 @@ public SyncPoller, PrivateEndpointCon * @return the private endpoint connection resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters) { + private Mono createOrUpdateAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters) { return beginCreateOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -743,21 +605,15 @@ private Mono createOrUpdateAsync( * @return the private endpoint connection resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, clusterName, privateEndpointConnectionName, parameters, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters, + context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -768,17 +624,14 @@ private Mono createOrUpdateAsync( * @return the private endpoint connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PrivateEndpointConnectionInner createOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters) { + public PrivateEndpointConnectionInner createOrUpdate(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters) { return createOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters).block(); } /** * Approves or Rejects a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -790,19 +643,15 @@ public PrivateEndpointConnectionInner createOrUpdate( * @return the private endpoint connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PrivateEndpointConnectionInner createOrUpdate( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner parameters, - Context context) { + public PrivateEndpointConnectionInner createOrUpdate(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, clusterName, privateEndpointConnectionName, parameters, context) .block(); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -812,19 +661,15 @@ public PrivateEndpointConnectionInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -834,31 +679,20 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, privateEndpointConnectionName, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -869,19 +703,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -891,28 +721,18 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, privateEndpointConnectionName, accept, context); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -922,19 +742,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -945,19 +763,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -967,14 +784,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { return this.beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName).getSyncPoller(); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -985,16 +802,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { - return this - .beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { + return this.beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context) .getSyncPoller(); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -1005,14 +821,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String privateEndpointConnectionName) { - return beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -1023,16 +838,15 @@ private Mono deleteAsync(String resourceGroupName, String clusterName, Str * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { - return beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + Context context) { + return beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -1047,7 +861,7 @@ public void delete(String resourceGroupName, String clusterName, String privateE /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -1057,8 +871,8 @@ public void delete(String resourceGroupName, String clusterName, String privateE * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + public void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + Context context) { deleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).block(); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsImpl.java index c6545912749a9..569e0327643ea 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateEndpointConnectionsImpl.java @@ -21,47 +21,41 @@ public final class PrivateEndpointConnectionsImpl implements PrivateEndpointConn private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public PrivateEndpointConnectionsImpl( - PrivateEndpointConnectionsClient innerClient, + public PrivateEndpointConnectionsImpl(PrivateEndpointConnectionsClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByCluster(String resourceGroupName, String clusterName) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); } - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + clusterName, privateEndpointConnectionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PrivateEndpointConnectionImpl(inner.getValue(), this.manager())); } else { return null; } } - public PrivateEndpointConnection get( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { - PrivateEndpointConnectionInner inner = - this.serviceClient().get(resourceGroupName, clusterName, privateEndpointConnectionName); + public PrivateEndpointConnection get(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { + PrivateEndpointConnectionInner inner + = this.serviceClient().get(resourceGroupName, clusterName, privateEndpointConnectionName); if (inner != null) { return new PrivateEndpointConnectionImpl(inner, this.manager()); } else { @@ -73,129 +67,88 @@ public void delete(String resourceGroupName, String clusterName, String privateE this.serviceClient().delete(resourceGroupName, clusterName, privateEndpointConnectionName); } - public void delete( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + public void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + Context context) { this.serviceClient().delete(resourceGroupName, clusterName, privateEndpointConnectionName, context); } public PrivateEndpointConnection getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } - return this - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE) + return this.getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } this.delete(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } this.delete(resourceGroupName, clusterName, privateEndpointConnectionName, context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourceImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourceImpl.java index fe7c5ffe73e1a..38a86dc312973 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourceImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourceImpl.java @@ -15,8 +15,7 @@ public final class PrivateLinkResourceImpl implements PrivateLinkResource { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - PrivateLinkResourceImpl( - PrivateLinkResourceInner innerObject, + PrivateLinkResourceImpl(PrivateLinkResourceInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesClientImpl.java index bd2297c901e06..bee3a71c88903 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesClientImpl.java @@ -30,23 +30,28 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateLinkResourceListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. + */ public final class PrivateLinkResourcesClientImpl implements PrivateLinkResourcesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PrivateLinkResourcesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of PrivateLinkResourcesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PrivateLinkResourcesClientImpl(CosmosDBForPostgreSqlImpl client) { - this.service = - RestProxy - .create(PrivateLinkResourcesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(PrivateLinkResourcesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -57,61 +62,47 @@ public final class PrivateLinkResourcesClientImpl implements PrivateLinkResource @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface PrivateLinkResourcesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateLinkResources") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateLinkResources") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateLinkResources/{privateLinkResourceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/privateLinkResources/{privateLinkResourceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("privateLinkResourceName") String privateLinkResourceName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("privateLinkResourceName") String privateLinkResourceName, @HeaderParam("Accept") String accept, Context context); } /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources for cluster along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -122,27 +113,16 @@ private Mono> listByClusterSinglePageAsy } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -150,22 +130,18 @@ private Mono> listByClusterSinglePageAsy * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources for cluster along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -177,23 +153,15 @@ private Mono> listByClusterSinglePageAsy final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -208,7 +176,7 @@ private PagedFlux listByClusterAsync(String resourceGr /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -218,14 +186,14 @@ private PagedFlux listByClusterAsync(String resourceGr * @return the private link resources for cluster as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByClusterAsync( - String resourceGroupName, String clusterName, Context context) { + private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, + Context context) { return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context)); } /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -240,7 +208,7 @@ public PagedIterable listByCluster(String resourceGrou /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -250,14 +218,14 @@ public PagedIterable listByCluster(String resourceGrou * @return the private link resources for cluster as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listByClusterAsync(resourceGroupName, clusterName, context)); } /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. @@ -267,19 +235,15 @@ public PagedIterable listByCluster( * @return a private link resource for cluster along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String privateLinkResourceName) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String privateLinkResourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -289,30 +253,18 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateLinkResourceName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter privateLinkResourceName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter privateLinkResourceName is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateLinkResourceName, - accept, - context)) + return FluxUtil.withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, privateLinkResourceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. @@ -323,19 +275,15 @@ private Mono> getWithResponseAsync( * @return a private link resource for cluster along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String privateLinkResourceName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String privateLinkResourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -345,27 +293,18 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateLinkResourceName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter privateLinkResourceName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter privateLinkResourceName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - privateLinkResourceName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, privateLinkResourceName, accept, context); } /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. @@ -375,15 +314,15 @@ private Mono> getWithResponseAsync( * @return a private link resource for cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String clusterName, String privateLinkResourceName) { + private Mono getAsync(String resourceGroupName, String clusterName, + String privateLinkResourceName) { return getWithResponseAsync(resourceGroupName, clusterName, privateLinkResourceName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. @@ -394,14 +333,14 @@ private Mono getAsync( * @return a private link resource for cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String privateLinkResourceName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, + String privateLinkResourceName, Context context) { return getWithResponseAsync(resourceGroupName, clusterName, privateLinkResourceName, context).block(); } /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesImpl.java index 5e35f3b1806ea..1b13333427697 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/PrivateLinkResourcesImpl.java @@ -21,35 +21,31 @@ public final class PrivateLinkResourcesImpl implements PrivateLinkResources { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public PrivateLinkResourcesImpl( - PrivateLinkResourcesClient innerClient, + public PrivateLinkResourcesImpl(PrivateLinkResourcesClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByCluster(String resourceGroupName, String clusterName) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); } - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String privateLinkResourceName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, clusterName, privateLinkResourceName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, + String privateLinkResourceName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, clusterName, privateLinkResourceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PrivateLinkResourceImpl(inner.getValue(), this.manager())); } else { return null; @@ -57,8 +53,8 @@ public Response getWithResponse( } public PrivateLinkResource get(String resourceGroupName, String clusterName, String privateLinkResourceName) { - PrivateLinkResourceInner inner = - this.serviceClient().get(resourceGroupName, clusterName, privateLinkResourceName); + PrivateLinkResourceInner inner + = this.serviceClient().get(resourceGroupName, clusterName, privateLinkResourceName); if (inner != null) { return new PrivateLinkResourceImpl(inner, this.manager()); } else { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ResourceManagerUtils.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..5670f936554e1 --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RoleImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RoleImpl.java index 70d5796cd7f2c..8558329f045d3 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RoleImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RoleImpl.java @@ -7,16 +7,17 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RoleInner; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrincipalType; import com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState; import com.azure.resourcemanager.cosmosdbforpostgresql.models.Role; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.RoleType; public final class RoleImpl implements Role, Role.Definition { private RoleInner innerObject; private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - RoleImpl( - RoleInner innerObject, + RoleImpl(RoleInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -38,6 +39,10 @@ public SystemData systemData() { return this.innerModel().systemData(); } + public RoleType roleType() { + return this.innerModel().roleType(); + } + public String password() { return this.innerModel().password(); } @@ -46,6 +51,18 @@ public ProvisioningState provisioningState() { return this.innerModel().provisioningState(); } + public String objectId() { + return this.innerModel().objectId(); + } + + public PrincipalType principalType() { + return this.innerModel().principalType(); + } + + public String tenantId() { + return this.innerModel().tenantId(); + } + public RoleInner innerModel() { return this.innerObject; } @@ -67,20 +84,14 @@ public RoleImpl withExistingServerGroupsv2(String resourceGroupName, String clus } public Role create() { - this.innerObject = - serviceManager - .serviceClient() - .getRoles() - .create(resourceGroupName, clusterName, roleName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getRoles().create(resourceGroupName, clusterName, roleName, + this.innerModel(), Context.NONE); return this; } public Role create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getRoles() - .create(resourceGroupName, clusterName, roleName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getRoles().create(resourceGroupName, clusterName, roleName, + this.innerModel(), context); return this; } @@ -91,22 +102,19 @@ public Role create(Context context) { } public Role refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getRoles() - .getWithResponse(resourceGroupName, clusterName, roleName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getRoles() + .getWithResponse(resourceGroupName, clusterName, roleName, Context.NONE).getValue(); return this; } public Role refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getRoles() - .getWithResponse(resourceGroupName, clusterName, roleName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getRoles() + .getWithResponse(resourceGroupName, clusterName, roleName, context).getValue(); + return this; + } + + public RoleImpl withRoleType(RoleType roleType) { + this.innerModel().withRoleType(roleType); return this; } @@ -114,4 +122,19 @@ public RoleImpl withPassword(String password) { this.innerModel().withPassword(password); return this; } + + public RoleImpl withObjectId(String objectId) { + this.innerModel().withObjectId(objectId); + return this; + } + + public RoleImpl withPrincipalType(PrincipalType principalType) { + this.innerModel().withPrincipalType(principalType); + return this; + } + + public RoleImpl withTenantId(String tenantId) { + this.innerModel().withTenantId(tenantId); + return this; + } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesClientImpl.java index b2d186df4afec..35b785f8264a3 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesClientImpl.java @@ -38,17 +38,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in RolesClient. */ +/** + * An instance of this class provides access to all the operations defined in RolesClient. + */ public final class RolesClientImpl implements RolesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final RolesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of RolesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ RolesClientImpl(CosmosDBForPostgreSqlImpl client) { @@ -57,76 +63,53 @@ public final class RolesClientImpl implements RolesClient { } /** - * The interface defining all the services for CosmosDBForPostgreSqlRoles to be used by the proxy service to perform - * REST calls. + * The interface defining all the services for CosmosDBForPostgreSqlRoles to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface RolesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles/{roleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles/{roleName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("roleName") String roleName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles/{roleName}") - @ExpectedResponses({200, 201}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("roleName") String roleName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles/{roleName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("roleName") String roleName, - @BodyParam("application/json") RoleInner parameters, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles/{roleName}") - @ExpectedResponses({202, 204}) + Mono>> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("roleName") String roleName, @BodyParam("application/json") RoleInner parameters, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles/{roleName}") + @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("roleName") String roleName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("roleName") String roleName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/roles") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); } /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -136,19 +119,15 @@ Mono> listByCluster( * @return information about a cluster role along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String roleName) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String roleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -162,24 +141,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - roleName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, roleName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -190,19 +159,15 @@ private Mono> getWithResponseAsync( * @return information about a cluster role along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String roleName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String roleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -216,21 +181,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - roleName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, roleName, accept, context); } /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -247,7 +204,7 @@ private Mono getAsync(String resourceGroupName, String clusterName, S /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -258,14 +215,14 @@ private Mono getAsync(String resourceGroupName, String clusterName, S * @return information about a cluster role along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String roleName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, String roleName, + Context context) { return getWithResponseAsync(resourceGroupName, clusterName, roleName, context).block(); } /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -281,7 +238,7 @@ public RoleInner get(String resourceGroupName, String clusterName, String roleNa /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -292,19 +249,15 @@ public RoleInner get(String resourceGroupName, String clusterName, String roleNa * @return represents a cluster role along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -323,25 +276,14 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - roleName, - parameters, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, roleName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -353,19 +295,15 @@ private Mono>> createWithResponseAsync( * @return represents a cluster role along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -384,22 +322,13 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - roleName, - parameters, - accept, - context); + return service.create(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, roleName, parameters, accept, context); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -410,19 +339,17 @@ private Mono>> createWithResponseAsync( * @return the {@link PollerFlux} for polling of represents a cluster role. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, RoleInner> beginCreateAsync( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters) { - Mono>> mono = - createWithResponseAsync(resourceGroupName, clusterName, roleName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), RoleInner.class, RoleInner.class, this.client.getContext()); + private PollerFlux, RoleInner> beginCreateAsync(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters) { + Mono>> mono + = createWithResponseAsync(resourceGroupName, clusterName, roleName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), RoleInner.class, + RoleInner.class, this.client.getContext()); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -434,20 +361,18 @@ private PollerFlux, RoleInner> beginCreateAsync( * @return the {@link PollerFlux} for polling of represents a cluster role. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, RoleInner> beginCreateAsync( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context) { + private PollerFlux, RoleInner> beginCreateAsync(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createWithResponseAsync(resourceGroupName, clusterName, roleName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), RoleInner.class, RoleInner.class, context); + Mono>> mono + = createWithResponseAsync(resourceGroupName, clusterName, roleName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), RoleInner.class, + RoleInner.class, context); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -458,14 +383,14 @@ private PollerFlux, RoleInner> beginCreateAsync( * @return the {@link SyncPoller} for polling of represents a cluster role. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, RoleInner> beginCreate( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters) { + public SyncPoller, RoleInner> beginCreate(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters) { return this.beginCreateAsync(resourceGroupName, clusterName, roleName, parameters).getSyncPoller(); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -477,14 +402,14 @@ public SyncPoller, RoleInner> beginCreate( * @return the {@link SyncPoller} for polling of represents a cluster role. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, RoleInner> beginCreate( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context) { + public SyncPoller, RoleInner> beginCreate(String resourceGroupName, String clusterName, + String roleName, RoleInner parameters, Context context) { return this.beginCreateAsync(resourceGroupName, clusterName, roleName, parameters, context).getSyncPoller(); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -495,16 +420,15 @@ public SyncPoller, RoleInner> beginCreate( * @return represents a cluster role on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters) { - return beginCreateAsync(resourceGroupName, clusterName, roleName, parameters) - .last() + private Mono createAsync(String resourceGroupName, String clusterName, String roleName, + RoleInner parameters) { + return beginCreateAsync(resourceGroupName, clusterName, roleName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -516,16 +440,15 @@ private Mono createAsync( * @return represents a cluster role on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context) { - return beginCreateAsync(resourceGroupName, clusterName, roleName, parameters, context) - .last() + private Mono createAsync(String resourceGroupName, String clusterName, String roleName, + RoleInner parameters, Context context) { + return beginCreateAsync(resourceGroupName, clusterName, roleName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -542,7 +465,7 @@ public RoleInner create(String resourceGroupName, String clusterName, String rol /** * Creates a new role or updates an existing role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -554,14 +477,14 @@ public RoleInner create(String resourceGroupName, String clusterName, String rol * @return represents a cluster role. */ @ServiceMethod(returns = ReturnType.SINGLE) - public RoleInner create( - String resourceGroupName, String clusterName, String roleName, RoleInner parameters, Context context) { + public RoleInner create(String resourceGroupName, String clusterName, String roleName, RoleInner parameters, + Context context) { return createAsync(resourceGroupName, clusterName, roleName, parameters, context).block(); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -571,19 +494,15 @@ public RoleInner create( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String roleName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String roleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -597,24 +516,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - roleName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, roleName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -625,19 +534,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String roleName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String roleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -651,21 +556,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - roleName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, roleName, accept, context); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -675,18 +572,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String roleName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String roleName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, clusterName, roleName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -697,19 +592,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String roleName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String roleName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, clusterName, roleName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, roleName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -719,14 +613,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String roleName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String roleName) { return this.beginDeleteAsync(resourceGroupName, clusterName, roleName).getSyncPoller(); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -737,14 +631,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String roleName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, String roleName, + Context context) { return this.beginDeleteAsync(resourceGroupName, clusterName, roleName, context).getSyncPoller(); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -755,14 +649,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String roleName) { - return beginDeleteAsync(resourceGroupName, clusterName, roleName) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, roleName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -774,14 +667,13 @@ private Mono deleteAsync(String resourceGroupName, String clusterName, Str */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String roleName, Context context) { - return beginDeleteAsync(resourceGroupName, clusterName, roleName, context) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, roleName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -796,7 +688,7 @@ public void delete(String resourceGroupName, String clusterName, String roleName /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -812,7 +704,7 @@ public void delete(String resourceGroupName, String clusterName, String roleName /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -823,16 +715,12 @@ public void delete(String resourceGroupName, String clusterName, String roleName @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByClusterSinglePageAsync(String resourceGroupName, String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -843,27 +731,16 @@ private Mono> listByClusterSinglePageAsync(String resou } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -873,19 +750,15 @@ private Mono> listByClusterSinglePageAsync(String resou * @return a list of roles along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, String clusterName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -897,23 +770,15 @@ private Mono> listByClusterSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -928,7 +793,7 @@ private PagedFlux listByClusterAsync(String resourceGroupName, String /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -944,7 +809,7 @@ private PagedFlux listByClusterAsync(String resourceGroupName, String /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -959,7 +824,7 @@ public PagedIterable listByCluster(String resourceGroupName, String c /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesImpl.java index 37a5ec8028da4..b251495ea3c3a 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/RolesImpl.java @@ -21,22 +21,18 @@ public final class RolesImpl implements Roles { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public RolesImpl( - RolesClient innerClient, + public RolesImpl(RolesClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String resourceGroupName, String clusterName, String roleName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, clusterName, roleName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, String roleName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, clusterName, roleName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RoleImpl(inner.getValue(), this.manager())); } else { return null; @@ -62,118 +58,86 @@ public void delete(String resourceGroupName, String clusterName, String roleName public PagedIterable listByCluster(String resourceGroupName, String clusterName) { PagedIterable inner = this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new RoleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new RoleImpl(inner1, this.manager())); } public PagedIterable listByCluster(String resourceGroupName, String clusterName, Context context) { PagedIterable inner = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new RoleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new RoleImpl(inner1, this.manager())); } public Role getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String roleName = Utils.getValueFromIdByName(id, "roles"); + String roleName = ResourceManagerUtils.getValueFromIdByName(id, "roles"); if (roleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, roleName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String roleName = Utils.getValueFromIdByName(id, "roles"); + String roleName = ResourceManagerUtils.getValueFromIdByName(id, "roles"); if (roleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, roleName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String roleName = Utils.getValueFromIdByName(id, "roles"); + String roleName = ResourceManagerUtils.getValueFromIdByName(id, "roles"); if (roleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); } this.delete(resourceGroupName, clusterName, roleName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "serverGroupsv2"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "serverGroupsv2"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'serverGroupsv2'.", id))); } - String roleName = Utils.getValueFromIdByName(id, "roles"); + String roleName = ResourceManagerUtils.getValueFromIdByName(id, "roles"); if (roleName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'roles'.", id))); } this.delete(resourceGroupName, clusterName, roleName, context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServerConfigurationImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServerConfigurationImpl.java index 4c559b001c006..8e2d93ebdf627 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServerConfigurationImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServerConfigurationImpl.java @@ -15,8 +15,7 @@ public final class ServerConfigurationImpl implements ServerConfiguration { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - ServerConfigurationImpl( - ServerConfigurationInner innerObject, + ServerConfigurationImpl(ServerConfigurationInner innerObject, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersClientImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersClientImpl.java index 84091644adcaf..854e42811f417 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersClientImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersClientImpl.java @@ -30,17 +30,23 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterServerListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ServersClient. */ +/** + * An instance of this class provides access to all the operations defined in ServersClient. + */ public final class ServersClientImpl implements ServersClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ServersService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final CosmosDBForPostgreSqlImpl client; /** * Initializes an instance of ServersClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ServersClientImpl(CosmosDBForPostgreSqlImpl client) { @@ -55,39 +61,28 @@ public final class ServersClientImpl implements ServersClient { @Host("{$host}") @ServiceInterface(name = "CosmosDBForPostgreSq") public interface ServersService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/servers") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/servers") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/servers/{serverName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/{clusterName}/servers/{serverName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("serverName") String serverName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("serverName") String serverName, @HeaderParam("Accept") String accept, Context context); } /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -96,19 +91,15 @@ Mono> get( * @return a list of servers in a cluster along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -119,27 +110,16 @@ private Mono> listByClusterSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -149,19 +129,15 @@ private Mono> listByClusterSinglePageAsync( * @return a list of servers in a cluster along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -173,23 +149,15 @@ private Mono> listByClusterSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -204,7 +172,7 @@ private PagedFlux listByClusterAsync(String resourceGroupNam /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -214,14 +182,14 @@ private PagedFlux listByClusterAsync(String resourceGroupNam * @return a list of servers in a cluster as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByClusterAsync( - String resourceGroupName, String clusterName, Context context) { + private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, + Context context) { return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context)); } /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -236,7 +204,7 @@ public PagedIterable listByCluster(String resourceGroupName, /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -246,37 +214,33 @@ public PagedIterable listByCluster(String resourceGroupName, * @return a list of servers in a cluster as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listByClusterAsync(resourceGroupName, clusterName, context)); } /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about a server in cluster along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about a server in cluster along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String serverName) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String serverName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -290,24 +254,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - serverName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, serverName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -315,23 +269,19 @@ private Mono> getWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about a server in cluster along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about a server in cluster along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String serverName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String serverName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -345,21 +295,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - serverName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, serverName, accept, context); } /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -376,7 +318,7 @@ private Mono getAsync(String resourceGroupName, String clust /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -387,14 +329,14 @@ private Mono getAsync(String resourceGroupName, String clust * @return information about a server in cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String serverName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, String serverName, + Context context) { return getWithResponseAsync(resourceGroupName, clusterName, serverName, context).block(); } /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersImpl.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersImpl.java index 2da19edec5dac..8cc1b0f03ec42 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersImpl.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/ServersImpl.java @@ -21,8 +21,7 @@ public final class ServersImpl implements Servers { private final com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager; - public ServersImpl( - ServersClient innerClient, + public ServersImpl(ServersClient innerClient, com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,24 +29,21 @@ public ServersImpl( public PagedIterable listByCluster(String resourceGroupName, String clusterName) { PagedIterable inner = this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new ClusterServerImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterServerImpl(inner1, this.manager())); } public PagedIterable listByCluster(String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new ClusterServerImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterServerImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String serverName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, clusterName, serverName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, String serverName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, clusterName, serverName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ClusterServerImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/Utils.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/Utils.java deleted file mode 100644 index d44e406b390c3..0000000000000 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.cosmosdbforpostgresql.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/package-info.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/package-info.java index 991e493d68905..4a93f27d94801 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/package-info.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/implementation/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the implementations for CosmosDBForPostgreSql. Azure Cosmos DB for PostgreSQL database service - * resource provider REST APIs. + * Package containing the implementations for CosmosDBForPostgreSql. + * Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. */ package com.azure.resourcemanager.cosmosdbforpostgresql.implementation; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ActiveDirectoryAuth.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ActiveDirectoryAuth.java new file mode 100644 index 0000000000000..27892b66f633a --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ActiveDirectoryAuth.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for ActiveDirectoryAuth. + */ +public final class ActiveDirectoryAuth extends ExpandableStringEnum { + /** + * Static value enabled for ActiveDirectoryAuth. + */ + public static final ActiveDirectoryAuth ENABLED = fromString("enabled"); + + /** + * Static value disabled for ActiveDirectoryAuth. + */ + public static final ActiveDirectoryAuth DISABLED = fromString("disabled"); + + /** + * Creates a new instance of ActiveDirectoryAuth value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ActiveDirectoryAuth() { + } + + /** + * Creates or finds a ActiveDirectoryAuth from its string representation. + * + * @param name a name to look for. + * @return the corresponding ActiveDirectoryAuth. + */ + @JsonCreator + public static ActiveDirectoryAuth fromString(String name) { + return fromString(name, ActiveDirectoryAuth.class); + } + + /** + * Gets known ActiveDirectoryAuth values. + * + * @return known ActiveDirectoryAuth values. + */ + public static Collection values() { + return values(ActiveDirectoryAuth.class); + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/AuthConfig.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/AuthConfig.java new file mode 100644 index 0000000000000..4296a13c29173 --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/AuthConfig.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Authentication configuration of a cluster. + */ +@Fluent +public final class AuthConfig { + /* + * The activeDirectoryAuth property. + */ + @JsonProperty(value = "activeDirectoryAuth") + private ActiveDirectoryAuth activeDirectoryAuth; + + /* + * The passwordAuth property. + */ + @JsonProperty(value = "passwordAuth") + private PasswordAuth passwordAuth; + + /** + * Creates an instance of AuthConfig class. + */ + public AuthConfig() { + } + + /** + * Get the activeDirectoryAuth property: The activeDirectoryAuth property. + * + * @return the activeDirectoryAuth value. + */ + public ActiveDirectoryAuth activeDirectoryAuth() { + return this.activeDirectoryAuth; + } + + /** + * Set the activeDirectoryAuth property: The activeDirectoryAuth property. + * + * @param activeDirectoryAuth the activeDirectoryAuth value to set. + * @return the AuthConfig object itself. + */ + public AuthConfig withActiveDirectoryAuth(ActiveDirectoryAuth activeDirectoryAuth) { + this.activeDirectoryAuth = activeDirectoryAuth; + return this; + } + + /** + * Get the passwordAuth property: The passwordAuth property. + * + * @return the passwordAuth value. + */ + public PasswordAuth passwordAuth() { + return this.passwordAuth; + } + + /** + * Set the passwordAuth property: The passwordAuth property. + * + * @param passwordAuth the passwordAuth value to set. + * @return the AuthConfig object itself. + */ + public AuthConfig withPasswordAuth(PasswordAuth passwordAuth) { + this.passwordAuth = passwordAuth; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Cluster.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Cluster.java index b2e31ff60e315..e70be71bc994f 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Cluster.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Cluster.java @@ -12,102 +12,104 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Cluster. */ +/** + * An immutable client-side representation of Cluster. + */ public interface Cluster { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the administratorLogin property: The administrator's login name of the servers in the cluster. - * + * * @return the administratorLogin value. */ String administratorLogin(); /** * Gets the administratorLoginPassword property: The password of the administrator login. Required for creation. - * + * * @return the administratorLoginPassword value. */ String administratorLoginPassword(); /** * Gets the provisioningState property: Provisioning state of the cluster. - * + * * @return the provisioningState value. */ String provisioningState(); /** * Gets the state property: A state of a cluster/server that is visible to user. - * + * * @return the state value. */ String state(); /** * Gets the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @return the postgresqlVersion value. */ String postgresqlVersion(); /** * Gets the citusVersion property: The Citus extension version on all cluster servers. - * + * * @return the citusVersion value. */ String citusVersion(); /** * Gets the maintenanceWindow property: Maintenance window of a cluster. - * + * * @return the maintenanceWindow value. */ MaintenanceWindow maintenanceWindow(); /** * Gets the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @return the preferredPrimaryZone value. */ String preferredPrimaryZone(); @@ -115,14 +117,14 @@ public interface Cluster { /** * Gets the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * * @return the enableShardsOnCoordinator value. */ Boolean enableShardsOnCoordinator(); /** * Gets the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @return the enableHa value. */ Boolean enableHa(); @@ -130,7 +132,7 @@ public interface Cluster { /** * Gets the coordinatorServerEdition property: The edition of a coordinator server (default: GeneralPurpose). * Required for creation. - * + * * @return the coordinatorServerEdition value. */ String coordinatorServerEdition(); @@ -138,7 +140,7 @@ public interface Cluster { /** * Gets the coordinatorStorageQuotaInMb property: The storage of a server in MB. Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the coordinatorStorageQuotaInMb value. */ Integer coordinatorStorageQuotaInMb(); @@ -146,21 +148,21 @@ public interface Cluster { /** * Gets the coordinatorVCores property: The vCores count of a server (max: 96). Required for creation. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the coordinatorVCores value. */ Integer coordinatorVCores(); /** * Gets the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @return the coordinatorEnablePublicIpAccess value. */ Boolean coordinatorEnablePublicIpAccess(); /** * Gets the nodeServerEdition property: The edition of a node server (default: MemoryOptimized). - * + * * @return the nodeServerEdition value. */ String nodeServerEdition(); @@ -169,7 +171,7 @@ public interface Cluster { * Gets the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. Required for creation. - * + * * @return the nodeCount value. */ Integer nodeCount(); @@ -177,7 +179,7 @@ public interface Cluster { /** * Gets the nodeStorageQuotaInMb property: The storage in MB on each worker node. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the nodeStorageQuotaInMb value. */ Integer nodeStorageQuotaInMb(); @@ -185,114 +187,141 @@ public interface Cluster { /** * Gets the nodeVCores property: The compute in vCores on each worker node (max: 104). See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. - * + * * @return the nodeVCores value. */ Integer nodeVCores(); /** * Gets the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @return the nodeEnablePublicIpAccess value. */ Boolean nodeEnablePublicIpAccess(); /** * Gets the serverNames property: The list of server names in the cluster. - * + * * @return the serverNames value. */ List serverNames(); /** * Gets the sourceResourceId property: The resource id of source cluster for read replica clusters. - * + * * @return the sourceResourceId value. */ String sourceResourceId(); /** * Gets the sourceLocation property: The Azure region of source cluster for read replica clusters. - * + * * @return the sourceLocation value. */ String sourceLocation(); /** * Gets the pointInTimeUtc property: Date and time in UTC (ISO8601 format) for cluster restore. - * + * * @return the pointInTimeUtc value. */ OffsetDateTime pointInTimeUtc(); /** * Gets the readReplicas property: The array of read replica clusters. - * + * * @return the readReplicas value. */ List readReplicas(); /** * Gets the earliestRestoreTime property: The earliest restore point time (ISO8601 format) for the cluster. - * + * * @return the earliestRestoreTime value. */ OffsetDateTime earliestRestoreTime(); /** * Gets the privateEndpointConnections property: The private endpoint connections for a cluster. - * + * * @return the privateEndpointConnections value. */ List privateEndpointConnections(); + /** + * Gets the databaseName property: The database name of the cluster. Only one database per cluster is supported. + * + * @return the databaseName value. + */ + String databaseName(); + + /** + * Gets the enableGeoBackup property: If cluster backup is stored in another Azure region in addition to the copy of + * the backup stored in the cluster's region. Enabled only at the time of cluster creation. + * + * @return the enableGeoBackup value. + */ + Boolean enableGeoBackup(); + + /** + * Gets the authConfig property: Authentication configuration of a cluster. + * + * @return the authConfig value. + */ + AuthConfig authConfig(); + /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterInner object. - * + * * @return the inner object. */ ClusterInner innerModel(); - /** The entirety of the Cluster definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the Cluster definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The Cluster definition stages. */ + /** + * The Cluster definition stages. + */ interface DefinitionStages { - /** The first stage of the Cluster definition. */ + /** + * The first stage of the Cluster definition. + */ interface Blank extends WithLocation { } - /** The stage of the Cluster definition allowing to specify location. */ + /** + * The stage of the Cluster definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -300,18 +329,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Cluster definition allowing to specify parent resource. */ + /** + * The stage of the Cluster definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -322,505 +353,603 @@ interface WithResourceGroup { * The stage of the Cluster definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithAdministratorLoginPassword, - DefinitionStages.WithPostgresqlVersion, - DefinitionStages.WithCitusVersion, - DefinitionStages.WithMaintenanceWindow, - DefinitionStages.WithPreferredPrimaryZone, - DefinitionStages.WithEnableShardsOnCoordinator, - DefinitionStages.WithEnableHa, - DefinitionStages.WithCoordinatorServerEdition, - DefinitionStages.WithCoordinatorStorageQuotaInMb, - DefinitionStages.WithCoordinatorVCores, - DefinitionStages.WithCoordinatorEnablePublicIpAccess, - DefinitionStages.WithNodeServerEdition, - DefinitionStages.WithNodeCount, - DefinitionStages.WithNodeStorageQuotaInMb, - DefinitionStages.WithNodeVCores, - DefinitionStages.WithNodeEnablePublicIpAccess, - DefinitionStages.WithSourceResourceId, - DefinitionStages.WithSourceLocation, - DefinitionStages.WithPointInTimeUtc { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithAdministratorLoginPassword, + DefinitionStages.WithPostgresqlVersion, DefinitionStages.WithCitusVersion, + DefinitionStages.WithMaintenanceWindow, DefinitionStages.WithPreferredPrimaryZone, + DefinitionStages.WithEnableShardsOnCoordinator, DefinitionStages.WithEnableHa, + DefinitionStages.WithCoordinatorServerEdition, DefinitionStages.WithCoordinatorStorageQuotaInMb, + DefinitionStages.WithCoordinatorVCores, DefinitionStages.WithCoordinatorEnablePublicIpAccess, + DefinitionStages.WithNodeServerEdition, DefinitionStages.WithNodeCount, + DefinitionStages.WithNodeStorageQuotaInMb, DefinitionStages.WithNodeVCores, + DefinitionStages.WithNodeEnablePublicIpAccess, DefinitionStages.WithSourceResourceId, + DefinitionStages.WithSourceLocation, DefinitionStages.WithPointInTimeUtc, DefinitionStages.WithDatabaseName, + DefinitionStages.WithEnableGeoBackup, DefinitionStages.WithAuthConfig { /** * Executes the create request. - * + * * @return the created resource. */ Cluster create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Cluster create(Context context); } - /** The stage of the Cluster definition allowing to specify tags. */ + /** + * The stage of the Cluster definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Cluster definition allowing to specify administratorLoginPassword. */ + /** + * The stage of the Cluster definition allowing to specify administratorLoginPassword. + */ interface WithAdministratorLoginPassword { /** * Specifies the administratorLoginPassword property: The password of the administrator login. Required for * creation.. - * + * * @param administratorLoginPassword The password of the administrator login. Required for creation. * @return the next definition stage. */ WithCreate withAdministratorLoginPassword(String administratorLoginPassword); } - /** The stage of the Cluster definition allowing to specify postgresqlVersion. */ + /** + * The stage of the Cluster definition allowing to specify postgresqlVersion. + */ interface WithPostgresqlVersion { /** * Specifies the postgresqlVersion property: The major PostgreSQL version on all cluster servers.. - * + * * @param postgresqlVersion The major PostgreSQL version on all cluster servers. * @return the next definition stage. */ WithCreate withPostgresqlVersion(String postgresqlVersion); } - /** The stage of the Cluster definition allowing to specify citusVersion. */ + /** + * The stage of the Cluster definition allowing to specify citusVersion. + */ interface WithCitusVersion { /** * Specifies the citusVersion property: The Citus extension version on all cluster servers.. - * + * * @param citusVersion The Citus extension version on all cluster servers. * @return the next definition stage. */ WithCreate withCitusVersion(String citusVersion); } - /** The stage of the Cluster definition allowing to specify maintenanceWindow. */ + /** + * The stage of the Cluster definition allowing to specify maintenanceWindow. + */ interface WithMaintenanceWindow { /** * Specifies the maintenanceWindow property: Maintenance window of a cluster.. - * + * * @param maintenanceWindow Maintenance window of a cluster. * @return the next definition stage. */ WithCreate withMaintenanceWindow(MaintenanceWindow maintenanceWindow); } - /** The stage of the Cluster definition allowing to specify preferredPrimaryZone. */ + /** + * The stage of the Cluster definition allowing to specify preferredPrimaryZone. + */ interface WithPreferredPrimaryZone { /** * Specifies the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster * servers.. - * + * * @param preferredPrimaryZone Preferred primary availability zone (AZ) for all cluster servers. * @return the next definition stage. */ WithCreate withPreferredPrimaryZone(String preferredPrimaryZone); } - /** The stage of the Cluster definition allowing to specify enableShardsOnCoordinator. */ + /** + * The stage of the Cluster definition allowing to specify enableShardsOnCoordinator. + */ interface WithEnableShardsOnCoordinator { /** * Specifies the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. * Should be set to 'true' on single node clusters. Requires shard rebalancing after value is changed.. - * + * * @param enableShardsOnCoordinator If distributed tables are placed on coordinator or not. Should be set to - * 'true' on single node clusters. Requires shard rebalancing after value is changed. + * 'true' on single node clusters. Requires shard rebalancing after value is changed. * @return the next definition stage. */ WithCreate withEnableShardsOnCoordinator(Boolean enableShardsOnCoordinator); } - /** The stage of the Cluster definition allowing to specify enableHa. */ + /** + * The stage of the Cluster definition allowing to specify enableHa. + */ interface WithEnableHa { /** * Specifies the enableHa property: If high availability (HA) is enabled or not for the cluster.. - * + * * @param enableHa If high availability (HA) is enabled or not for the cluster. * @return the next definition stage. */ WithCreate withEnableHa(Boolean enableHa); } - /** The stage of the Cluster definition allowing to specify coordinatorServerEdition. */ + /** + * The stage of the Cluster definition allowing to specify coordinatorServerEdition. + */ interface WithCoordinatorServerEdition { /** * Specifies the coordinatorServerEdition property: The edition of a coordinator server (default: * GeneralPurpose). Required for creation.. - * + * * @param coordinatorServerEdition The edition of a coordinator server (default: GeneralPurpose). Required - * for creation. + * for creation. * @return the next definition stage. */ WithCreate withCoordinatorServerEdition(String coordinatorServerEdition); } - /** The stage of the Cluster definition allowing to specify coordinatorStorageQuotaInMb. */ + /** + * The stage of the Cluster definition allowing to specify coordinatorStorageQuotaInMb. + */ interface WithCoordinatorStorageQuotaInMb { /** * Specifies the coordinatorStorageQuotaInMb property: The storage of a server in MB. Required for creation. * See https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information.. - * + * * @param coordinatorStorageQuotaInMb The storage of a server in MB. Required for creation. See - * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. + * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. * @return the next definition stage. */ WithCreate withCoordinatorStorageQuotaInMb(Integer coordinatorStorageQuotaInMb); } - /** The stage of the Cluster definition allowing to specify coordinatorVCores. */ + /** + * The stage of the Cluster definition allowing to specify coordinatorVCores. + */ interface WithCoordinatorVCores { /** * Specifies the coordinatorVCores property: The vCores count of a server (max: 96). Required for creation. * See https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information.. - * + * * @param coordinatorVCores The vCores count of a server (max: 96). Required for creation. See - * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. + * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. * @return the next definition stage. */ WithCreate withCoordinatorVCores(Integer coordinatorVCores); } - /** The stage of the Cluster definition allowing to specify coordinatorEnablePublicIpAccess. */ + /** + * The stage of the Cluster definition allowing to specify coordinatorEnablePublicIpAccess. + */ interface WithCoordinatorEnablePublicIpAccess { /** * Specifies the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator.. - * + * * @param coordinatorEnablePublicIpAccess If public access is enabled on coordinator. * @return the next definition stage. */ WithCreate withCoordinatorEnablePublicIpAccess(Boolean coordinatorEnablePublicIpAccess); } - /** The stage of the Cluster definition allowing to specify nodeServerEdition. */ + /** + * The stage of the Cluster definition allowing to specify nodeServerEdition. + */ interface WithNodeServerEdition { /** * Specifies the nodeServerEdition property: The edition of a node server (default: MemoryOptimized).. - * + * * @param nodeServerEdition The edition of a node server (default: MemoryOptimized). * @return the next definition stage. */ WithCreate withNodeServerEdition(String nodeServerEdition); } - /** The stage of the Cluster definition allowing to specify nodeCount. */ + /** + * The stage of the Cluster definition allowing to specify nodeCount. + */ interface WithNodeCount { /** * Specifies the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a * single node configuration with the ability to create distributed tables on that node. 2 or more worker * nodes represent multi-node configuration. Node count value cannot be 1. Required for creation.. - * + * * @param nodeCount Worker node count of the cluster. When node count is 0, it represents a single node - * configuration with the ability to create distributed tables on that node. 2 or more worker nodes - * represent multi-node configuration. Node count value cannot be 1. Required for creation. + * configuration with the ability to create distributed tables on that node. 2 or more worker nodes + * represent multi-node configuration. Node count value cannot be 1. Required for creation. * @return the next definition stage. */ WithCreate withNodeCount(Integer nodeCount); } - /** The stage of the Cluster definition allowing to specify nodeStorageQuotaInMb. */ + /** + * The stage of the Cluster definition allowing to specify nodeStorageQuotaInMb. + */ interface WithNodeStorageQuotaInMb { /** * Specifies the nodeStorageQuotaInMb property: The storage in MB on each worker node. See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information.. - * + * * @param nodeStorageQuotaInMb The storage in MB on each worker node. See - * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. + * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. * @return the next definition stage. */ WithCreate withNodeStorageQuotaInMb(Integer nodeStorageQuotaInMb); } - /** The stage of the Cluster definition allowing to specify nodeVCores. */ + /** + * The stage of the Cluster definition allowing to specify nodeVCores. + */ interface WithNodeVCores { /** * Specifies the nodeVCores property: The compute in vCores on each worker node (max: 104). See * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information.. - * + * * @param nodeVCores The compute in vCores on each worker node (max: 104). See - * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. + * https://learn.microsoft.com/azure/cosmos-db/postgresql/resources-compute for more information. * @return the next definition stage. */ WithCreate withNodeVCores(Integer nodeVCores); } - /** The stage of the Cluster definition allowing to specify nodeEnablePublicIpAccess. */ + /** + * The stage of the Cluster definition allowing to specify nodeEnablePublicIpAccess. + */ interface WithNodeEnablePublicIpAccess { /** * Specifies the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes.. - * + * * @param nodeEnablePublicIpAccess If public access is enabled on worker nodes. * @return the next definition stage. */ WithCreate withNodeEnablePublicIpAccess(Boolean nodeEnablePublicIpAccess); } - /** The stage of the Cluster definition allowing to specify sourceResourceId. */ + /** + * The stage of the Cluster definition allowing to specify sourceResourceId. + */ interface WithSourceResourceId { /** * Specifies the sourceResourceId property: The resource id of source cluster for read replica clusters.. - * + * * @param sourceResourceId The resource id of source cluster for read replica clusters. * @return the next definition stage. */ WithCreate withSourceResourceId(String sourceResourceId); } - /** The stage of the Cluster definition allowing to specify sourceLocation. */ + /** + * The stage of the Cluster definition allowing to specify sourceLocation. + */ interface WithSourceLocation { /** * Specifies the sourceLocation property: The Azure region of source cluster for read replica clusters.. - * + * * @param sourceLocation The Azure region of source cluster for read replica clusters. * @return the next definition stage. */ WithCreate withSourceLocation(String sourceLocation); } - /** The stage of the Cluster definition allowing to specify pointInTimeUtc. */ + /** + * The stage of the Cluster definition allowing to specify pointInTimeUtc. + */ interface WithPointInTimeUtc { /** * Specifies the pointInTimeUtc property: Date and time in UTC (ISO8601 format) for cluster restore.. - * + * * @param pointInTimeUtc Date and time in UTC (ISO8601 format) for cluster restore. * @return the next definition stage. */ WithCreate withPointInTimeUtc(OffsetDateTime pointInTimeUtc); } + + /** + * The stage of the Cluster definition allowing to specify databaseName. + */ + interface WithDatabaseName { + /** + * Specifies the databaseName property: The database name of the cluster. Only one database per cluster is + * supported.. + * + * @param databaseName The database name of the cluster. Only one database per cluster is supported. + * @return the next definition stage. + */ + WithCreate withDatabaseName(String databaseName); + } + + /** + * The stage of the Cluster definition allowing to specify enableGeoBackup. + */ + interface WithEnableGeoBackup { + /** + * Specifies the enableGeoBackup property: If cluster backup is stored in another Azure region in addition + * to the copy of the backup stored in the cluster's region. Enabled only at the time of cluster creation.. + * + * @param enableGeoBackup If cluster backup is stored in another Azure region in addition to the copy of the + * backup stored in the cluster's region. Enabled only at the time of cluster creation. + * @return the next definition stage. + */ + WithCreate withEnableGeoBackup(Boolean enableGeoBackup); + } + + /** + * The stage of the Cluster definition allowing to specify authConfig. + */ + interface WithAuthConfig { + /** + * Specifies the authConfig property: Authentication configuration of a cluster.. + * + * @param authConfig Authentication configuration of a cluster. + * @return the next definition stage. + */ + WithCreate withAuthConfig(AuthConfig authConfig); + } } /** * Begins update for the Cluster resource. - * + * * @return the stage of resource update. */ Cluster.Update update(); - /** The template for Cluster update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithAdministratorLoginPassword, - UpdateStages.WithPostgresqlVersion, - UpdateStages.WithCitusVersion, - UpdateStages.WithEnableShardsOnCoordinator, - UpdateStages.WithEnableHa, - UpdateStages.WithPreferredPrimaryZone, - UpdateStages.WithCoordinatorServerEdition, - UpdateStages.WithCoordinatorStorageQuotaInMb, - UpdateStages.WithCoordinatorVCores, - UpdateStages.WithCoordinatorEnablePublicIpAccess, - UpdateStages.WithNodeServerEdition, - UpdateStages.WithNodeCount, - UpdateStages.WithNodeStorageQuotaInMb, - UpdateStages.WithNodeVCores, - UpdateStages.WithMaintenanceWindow { + /** + * The template for Cluster update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithAdministratorLoginPassword, + UpdateStages.WithPostgresqlVersion, UpdateStages.WithCitusVersion, UpdateStages.WithEnableShardsOnCoordinator, + UpdateStages.WithEnableHa, UpdateStages.WithPreferredPrimaryZone, UpdateStages.WithCoordinatorServerEdition, + UpdateStages.WithCoordinatorStorageQuotaInMb, UpdateStages.WithCoordinatorVCores, + UpdateStages.WithCoordinatorEnablePublicIpAccess, UpdateStages.WithNodeServerEdition, + UpdateStages.WithNodeCount, UpdateStages.WithNodeStorageQuotaInMb, UpdateStages.WithNodeVCores, + UpdateStages.WithMaintenanceWindow { /** * Executes the update request. - * + * * @return the updated resource. */ Cluster apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Cluster apply(Context context); } - /** The Cluster update stages. */ + /** + * The Cluster update stages. + */ interface UpdateStages { - /** The stage of the Cluster update allowing to specify tags. */ + /** + * The stage of the Cluster update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Application-specific metadata in the form of key-value pairs.. - * + * * @param tags Application-specific metadata in the form of key-value pairs. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the Cluster update allowing to specify administratorLoginPassword. */ + /** + * The stage of the Cluster update allowing to specify administratorLoginPassword. + */ interface WithAdministratorLoginPassword { /** * Specifies the administratorLoginPassword property: The password of the administrator login. Each cluster * is created with pre-defined administrative role called ‘citus’. . - * + * * @param administratorLoginPassword The password of the administrator login. Each cluster is created with - * pre-defined administrative role called ‘citus’. . + * pre-defined administrative role called ‘citus’. . * @return the next definition stage. */ Update withAdministratorLoginPassword(String administratorLoginPassword); } - /** The stage of the Cluster update allowing to specify postgresqlVersion. */ + /** + * The stage of the Cluster update allowing to specify postgresqlVersion. + */ interface WithPostgresqlVersion { /** * Specifies the postgresqlVersion property: The major PostgreSQL version on all cluster servers.. - * + * * @param postgresqlVersion The major PostgreSQL version on all cluster servers. * @return the next definition stage. */ Update withPostgresqlVersion(String postgresqlVersion); } - /** The stage of the Cluster update allowing to specify citusVersion. */ + /** + * The stage of the Cluster update allowing to specify citusVersion. + */ interface WithCitusVersion { /** * Specifies the citusVersion property: The Citus extension version on all cluster servers.. - * + * * @param citusVersion The Citus extension version on all cluster servers. * @return the next definition stage. */ Update withCitusVersion(String citusVersion); } - /** The stage of the Cluster update allowing to specify enableShardsOnCoordinator. */ + /** + * The stage of the Cluster update allowing to specify enableShardsOnCoordinator. + */ interface WithEnableShardsOnCoordinator { /** * Specifies the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. * Should be set to 'true' on single node clusters. Requires shard rebalancing after value is changed.. - * + * * @param enableShardsOnCoordinator If distributed tables are placed on coordinator or not. Should be set to - * 'true' on single node clusters. Requires shard rebalancing after value is changed. + * 'true' on single node clusters. Requires shard rebalancing after value is changed. * @return the next definition stage. */ Update withEnableShardsOnCoordinator(Boolean enableShardsOnCoordinator); } - /** The stage of the Cluster update allowing to specify enableHa. */ + /** + * The stage of the Cluster update allowing to specify enableHa. + */ interface WithEnableHa { /** * Specifies the enableHa property: If high availability (HA) is enabled or not for the cluster.. - * + * * @param enableHa If high availability (HA) is enabled or not for the cluster. * @return the next definition stage. */ Update withEnableHa(Boolean enableHa); } - /** The stage of the Cluster update allowing to specify preferredPrimaryZone. */ + /** + * The stage of the Cluster update allowing to specify preferredPrimaryZone. + */ interface WithPreferredPrimaryZone { /** * Specifies the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster * servers.. - * + * * @param preferredPrimaryZone Preferred primary availability zone (AZ) for all cluster servers. * @return the next definition stage. */ Update withPreferredPrimaryZone(String preferredPrimaryZone); } - /** The stage of the Cluster update allowing to specify coordinatorServerEdition. */ + /** + * The stage of the Cluster update allowing to specify coordinatorServerEdition. + */ interface WithCoordinatorServerEdition { /** * Specifies the coordinatorServerEdition property: The edition of the coordinator (default: * GeneralPurpose).. - * + * * @param coordinatorServerEdition The edition of the coordinator (default: GeneralPurpose). * @return the next definition stage. */ Update withCoordinatorServerEdition(String coordinatorServerEdition); } - /** The stage of the Cluster update allowing to specify coordinatorStorageQuotaInMb. */ + /** + * The stage of the Cluster update allowing to specify coordinatorStorageQuotaInMb. + */ interface WithCoordinatorStorageQuotaInMb { /** * Specifies the coordinatorStorageQuotaInMb property: The storage of the coordinator in MB.. - * + * * @param coordinatorStorageQuotaInMb The storage of the coordinator in MB. * @return the next definition stage. */ Update withCoordinatorStorageQuotaInMb(Integer coordinatorStorageQuotaInMb); } - /** The stage of the Cluster update allowing to specify coordinatorVCores. */ + /** + * The stage of the Cluster update allowing to specify coordinatorVCores. + */ interface WithCoordinatorVCores { /** * Specifies the coordinatorVCores property: The vCores count of the coordinator (max: 96).. - * + * * @param coordinatorVCores The vCores count of the coordinator (max: 96). * @return the next definition stage. */ Update withCoordinatorVCores(Integer coordinatorVCores); } - /** The stage of the Cluster update allowing to specify coordinatorEnablePublicIpAccess. */ + /** + * The stage of the Cluster update allowing to specify coordinatorEnablePublicIpAccess. + */ interface WithCoordinatorEnablePublicIpAccess { /** * Specifies the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator.. - * + * * @param coordinatorEnablePublicIpAccess If public access is enabled on coordinator. * @return the next definition stage. */ Update withCoordinatorEnablePublicIpAccess(Boolean coordinatorEnablePublicIpAccess); } - /** The stage of the Cluster update allowing to specify nodeServerEdition. */ + /** + * The stage of the Cluster update allowing to specify nodeServerEdition. + */ interface WithNodeServerEdition { /** * Specifies the nodeServerEdition property: The edition of a node (default: MemoryOptimized).. - * + * * @param nodeServerEdition The edition of a node (default: MemoryOptimized). * @return the next definition stage. */ Update withNodeServerEdition(String nodeServerEdition); } - /** The stage of the Cluster update allowing to specify nodeCount. */ + /** + * The stage of the Cluster update allowing to specify nodeCount. + */ interface WithNodeCount { /** * Specifies the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a * single node configuration with the ability to create distributed tables on that node. 2 or more worker * nodes represent multi-node configuration. Node count value cannot be 1.. - * + * * @param nodeCount Worker node count of the cluster. When node count is 0, it represents a single node - * configuration with the ability to create distributed tables on that node. 2 or more worker nodes - * represent multi-node configuration. Node count value cannot be 1. + * configuration with the ability to create distributed tables on that node. 2 or more worker nodes + * represent multi-node configuration. Node count value cannot be 1. * @return the next definition stage. */ Update withNodeCount(Integer nodeCount); } - /** The stage of the Cluster update allowing to specify nodeStorageQuotaInMb. */ + /** + * The stage of the Cluster update allowing to specify nodeStorageQuotaInMb. + */ interface WithNodeStorageQuotaInMb { /** * Specifies the nodeStorageQuotaInMb property: The storage in MB on each worker node.. - * + * * @param nodeStorageQuotaInMb The storage in MB on each worker node. * @return the next definition stage. */ Update withNodeStorageQuotaInMb(Integer nodeStorageQuotaInMb); } - /** The stage of the Cluster update allowing to specify nodeVCores. */ + /** + * The stage of the Cluster update allowing to specify nodeVCores. + */ interface WithNodeVCores { /** * Specifies the nodeVCores property: The compute in vCores on each worker node (max: 104).. - * + * * @param nodeVCores The compute in vCores on each worker node (max: 104). * @return the next definition stage. */ Update withNodeVCores(Integer nodeVCores); } - /** The stage of the Cluster update allowing to specify maintenanceWindow. */ + /** + * The stage of the Cluster update allowing to specify maintenanceWindow. + */ interface WithMaintenanceWindow { /** * Specifies the maintenanceWindow property: Maintenance window of a cluster.. - * + * * @param maintenanceWindow Maintenance window of a cluster. * @return the next definition stage. */ @@ -830,14 +959,14 @@ interface WithMaintenanceWindow { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Cluster refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -845,7 +974,7 @@ interface WithMaintenanceWindow { /** * Restarts all nodes in the cluster. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @@ -853,7 +982,7 @@ interface WithMaintenanceWindow { /** * Restarts all nodes in the cluster. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -863,7 +992,7 @@ interface WithMaintenanceWindow { /** * Starts stopped compute on all cluster nodes. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @@ -871,7 +1000,7 @@ interface WithMaintenanceWindow { /** * Starts stopped compute on all cluster nodes. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -881,7 +1010,7 @@ interface WithMaintenanceWindow { /** * Stops compute on all cluster nodes. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @@ -889,7 +1018,7 @@ interface WithMaintenanceWindow { /** * Stops compute on all cluster nodes. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -899,7 +1028,7 @@ interface WithMaintenanceWindow { /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @@ -907,11 +1036,12 @@ interface WithMaintenanceWindow { /** * Promotes read replica cluster to an independent read-write cluster. - * + * + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void promoteReadReplica(Context context); + void promoteReadReplica(PromoteRequest promoteRequest, Context context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterConfigurationListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterConfigurationListResult.java index a5b2b46fed93f..d33bb3fdf0cff 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterConfigurationListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterConfigurationListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of cluster configurations. */ +/** + * A list of cluster configurations. + */ @Fluent public final class ClusterConfigurationListResult { /* @@ -24,13 +26,15 @@ public final class ClusterConfigurationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ClusterConfigurationListResult class. */ + /** + * Creates an instance of ClusterConfigurationListResult class. + */ public ClusterConfigurationListResult() { } /** * Get the value property: The list of cluster configurations. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: The list of cluster configurations. - * + * * @param value the value value to set. * @return the ClusterConfigurationListResult object itself. */ @@ -50,7 +54,7 @@ public ClusterConfigurationListResult withValue(List value) /** * Get the nextLink property: Link to retrieve next page of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterForUpdate.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterForUpdate.java index c2735b25a05e6..80cc4349a7eac 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterForUpdate.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterForUpdate.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Represents a cluster for update. */ +/** + * Represents a cluster for update. + */ @Fluent public final class ClusterForUpdate { /* @@ -26,13 +28,15 @@ public final class ClusterForUpdate { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of ClusterForUpdate class. */ + /** + * Creates an instance of ClusterForUpdate class. + */ public ClusterForUpdate() { } /** * Get the innerProperties property: Properties of the cluster. - * + * * @return the innerProperties value. */ private ClusterPropertiesForUpdate innerProperties() { @@ -41,7 +45,7 @@ private ClusterPropertiesForUpdate innerProperties() { /** * Get the tags property: Application-specific metadata in the form of key-value pairs. - * + * * @return the tags value. */ public Map tags() { @@ -50,7 +54,7 @@ public Map tags() { /** * Set the tags property: Application-specific metadata in the form of key-value pairs. - * + * * @param tags the tags value to set. * @return the ClusterForUpdate object itself. */ @@ -62,7 +66,7 @@ public ClusterForUpdate withTags(Map tags) { /** * Get the administratorLoginPassword property: The password of the administrator login. Each cluster is created * with pre-defined administrative role called ‘citus’. . - * + * * @return the administratorLoginPassword value. */ public String administratorLoginPassword() { @@ -72,7 +76,7 @@ public String administratorLoginPassword() { /** * Set the administratorLoginPassword property: The password of the administrator login. Each cluster is created * with pre-defined administrative role called ‘citus’. . - * + * * @param administratorLoginPassword the administratorLoginPassword value to set. * @return the ClusterForUpdate object itself. */ @@ -86,7 +90,7 @@ public ClusterForUpdate withAdministratorLoginPassword(String administratorLogin /** * Get the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @return the postgresqlVersion value. */ public String postgresqlVersion() { @@ -95,7 +99,7 @@ public String postgresqlVersion() { /** * Set the postgresqlVersion property: The major PostgreSQL version on all cluster servers. - * + * * @param postgresqlVersion the postgresqlVersion value to set. * @return the ClusterForUpdate object itself. */ @@ -109,7 +113,7 @@ public ClusterForUpdate withPostgresqlVersion(String postgresqlVersion) { /** * Get the citusVersion property: The Citus extension version on all cluster servers. - * + * * @return the citusVersion value. */ public String citusVersion() { @@ -118,7 +122,7 @@ public String citusVersion() { /** * Set the citusVersion property: The Citus extension version on all cluster servers. - * + * * @param citusVersion the citusVersion value to set. * @return the ClusterForUpdate object itself. */ @@ -131,9 +135,9 @@ public ClusterForUpdate withCitusVersion(String citusVersion) { } /** - * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Get the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @return the enableShardsOnCoordinator value. */ public Boolean enableShardsOnCoordinator() { @@ -141,9 +145,9 @@ public Boolean enableShardsOnCoordinator() { } /** - * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be set - * to 'true' on single node clusters. Requires shard rebalancing after value is changed. - * + * Set the enableShardsOnCoordinator property: If distributed tables are placed on coordinator or not. Should be + * set to 'true' on single node clusters. Requires shard rebalancing after value is changed. + * * @param enableShardsOnCoordinator the enableShardsOnCoordinator value to set. * @return the ClusterForUpdate object itself. */ @@ -157,7 +161,7 @@ public ClusterForUpdate withEnableShardsOnCoordinator(Boolean enableShardsOnCoor /** * Get the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @return the enableHa value. */ public Boolean enableHa() { @@ -166,7 +170,7 @@ public Boolean enableHa() { /** * Set the enableHa property: If high availability (HA) is enabled or not for the cluster. - * + * * @param enableHa the enableHa value to set. * @return the ClusterForUpdate object itself. */ @@ -180,7 +184,7 @@ public ClusterForUpdate withEnableHa(Boolean enableHa) { /** * Get the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @return the preferredPrimaryZone value. */ public String preferredPrimaryZone() { @@ -189,7 +193,7 @@ public String preferredPrimaryZone() { /** * Set the preferredPrimaryZone property: Preferred primary availability zone (AZ) for all cluster servers. - * + * * @param preferredPrimaryZone the preferredPrimaryZone value to set. * @return the ClusterForUpdate object itself. */ @@ -203,7 +207,7 @@ public ClusterForUpdate withPreferredPrimaryZone(String preferredPrimaryZone) { /** * Get the coordinatorServerEdition property: The edition of the coordinator (default: GeneralPurpose). - * + * * @return the coordinatorServerEdition value. */ public String coordinatorServerEdition() { @@ -212,7 +216,7 @@ public String coordinatorServerEdition() { /** * Set the coordinatorServerEdition property: The edition of the coordinator (default: GeneralPurpose). - * + * * @param coordinatorServerEdition the coordinatorServerEdition value to set. * @return the ClusterForUpdate object itself. */ @@ -226,7 +230,7 @@ public ClusterForUpdate withCoordinatorServerEdition(String coordinatorServerEdi /** * Get the coordinatorStorageQuotaInMb property: The storage of the coordinator in MB. - * + * * @return the coordinatorStorageQuotaInMb value. */ public Integer coordinatorStorageQuotaInMb() { @@ -235,7 +239,7 @@ public Integer coordinatorStorageQuotaInMb() { /** * Set the coordinatorStorageQuotaInMb property: The storage of the coordinator in MB. - * + * * @param coordinatorStorageQuotaInMb the coordinatorStorageQuotaInMb value to set. * @return the ClusterForUpdate object itself. */ @@ -249,7 +253,7 @@ public ClusterForUpdate withCoordinatorStorageQuotaInMb(Integer coordinatorStora /** * Get the coordinatorVCores property: The vCores count of the coordinator (max: 96). - * + * * @return the coordinatorVCores value. */ public Integer coordinatorVCores() { @@ -258,7 +262,7 @@ public Integer coordinatorVCores() { /** * Set the coordinatorVCores property: The vCores count of the coordinator (max: 96). - * + * * @param coordinatorVCores the coordinatorVCores value to set. * @return the ClusterForUpdate object itself. */ @@ -272,7 +276,7 @@ public ClusterForUpdate withCoordinatorVCores(Integer coordinatorVCores) { /** * Get the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @return the coordinatorEnablePublicIpAccess value. */ public Boolean coordinatorEnablePublicIpAccess() { @@ -281,7 +285,7 @@ public Boolean coordinatorEnablePublicIpAccess() { /** * Set the coordinatorEnablePublicIpAccess property: If public access is enabled on coordinator. - * + * * @param coordinatorEnablePublicIpAccess the coordinatorEnablePublicIpAccess value to set. * @return the ClusterForUpdate object itself. */ @@ -295,7 +299,7 @@ public ClusterForUpdate withCoordinatorEnablePublicIpAccess(Boolean coordinatorE /** * Get the nodeServerEdition property: The edition of a node (default: MemoryOptimized). - * + * * @return the nodeServerEdition value. */ public String nodeServerEdition() { @@ -304,7 +308,7 @@ public String nodeServerEdition() { /** * Set the nodeServerEdition property: The edition of a node (default: MemoryOptimized). - * + * * @param nodeServerEdition the nodeServerEdition value to set. * @return the ClusterForUpdate object itself. */ @@ -320,7 +324,7 @@ public ClusterForUpdate withNodeServerEdition(String nodeServerEdition) { * Get the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. - * + * * @return the nodeCount value. */ public Integer nodeCount() { @@ -331,7 +335,7 @@ public Integer nodeCount() { * Set the nodeCount property: Worker node count of the cluster. When node count is 0, it represents a single node * configuration with the ability to create distributed tables on that node. 2 or more worker nodes represent * multi-node configuration. Node count value cannot be 1. - * + * * @param nodeCount the nodeCount value to set. * @return the ClusterForUpdate object itself. */ @@ -345,7 +349,7 @@ public ClusterForUpdate withNodeCount(Integer nodeCount) { /** * Get the nodeStorageQuotaInMb property: The storage in MB on each worker node. - * + * * @return the nodeStorageQuotaInMb value. */ public Integer nodeStorageQuotaInMb() { @@ -354,7 +358,7 @@ public Integer nodeStorageQuotaInMb() { /** * Set the nodeStorageQuotaInMb property: The storage in MB on each worker node. - * + * * @param nodeStorageQuotaInMb the nodeStorageQuotaInMb value to set. * @return the ClusterForUpdate object itself. */ @@ -368,7 +372,7 @@ public ClusterForUpdate withNodeStorageQuotaInMb(Integer nodeStorageQuotaInMb) { /** * Get the nodeVCores property: The compute in vCores on each worker node (max: 104). - * + * * @return the nodeVCores value. */ public Integer nodeVCores() { @@ -377,7 +381,7 @@ public Integer nodeVCores() { /** * Set the nodeVCores property: The compute in vCores on each worker node (max: 104). - * + * * @param nodeVCores the nodeVCores value to set. * @return the ClusterForUpdate object itself. */ @@ -391,7 +395,7 @@ public ClusterForUpdate withNodeVCores(Integer nodeVCores) { /** * Get the nodeEnablePublicIpAccess property: If public access is enabled on worker nodes. - * + * * @return the nodeEnablePublicIpAccess value. */ public Boolean nodeEnablePublicIpAccess() { @@ -400,7 +404,7 @@ public Boolean nodeEnablePublicIpAccess() { /** * Get the maintenanceWindow property: Maintenance window of a cluster. - * + * * @return the maintenanceWindow value. */ public MaintenanceWindow maintenanceWindow() { @@ -409,7 +413,7 @@ public MaintenanceWindow maintenanceWindow() { /** * Set the maintenanceWindow property: Maintenance window of a cluster. - * + * * @param maintenanceWindow the maintenanceWindow value to set. * @return the ClusterForUpdate object itself. */ @@ -423,7 +427,7 @@ public ClusterForUpdate withMaintenanceWindow(MaintenanceWindow maintenanceWindo /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterListResult.java index 44806cc1dfcd9..8ae8f653c3031 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of clusters. */ +/** + * A list of clusters. + */ @Fluent public final class ClusterListResult { /* @@ -24,13 +26,15 @@ public final class ClusterListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ClusterListResult class. */ + /** + * Creates an instance of ClusterListResult class. + */ public ClusterListResult() { } /** * Get the value property: The list of clusters. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: The list of clusters. - * + * * @param value the value value to set. * @return the ClusterListResult object itself. */ @@ -50,7 +54,7 @@ public ClusterListResult withValue(List value) { /** * Get the nextLink property: The link used to get the next page of cluster list. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServer.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServer.java index 075e004700cd3..ac0964d5384a4 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServer.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServer.java @@ -7,116 +7,118 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterServerInner; -/** An immutable client-side representation of ClusterServer. */ +/** + * An immutable client-side representation of ClusterServer. + */ public interface ClusterServer { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the fullyQualifiedDomainName property: The fully qualified domain name of a server. - * + * * @return the fullyQualifiedDomainName value. */ String fullyQualifiedDomainName(); /** * Gets the role property: The role of server in the cluster. - * + * * @return the role value. */ ServerRole role(); /** * Gets the state property: A state of a cluster/server that is visible to user. - * + * * @return the state value. */ String state(); /** * Gets the haState property: A state of HA feature for the cluster. - * + * * @return the haState value. */ String haState(); /** * Gets the availabilityZone property: Availability Zone information of the server. - * + * * @return the availabilityZone value. */ String availabilityZone(); /** * Gets the postgresqlVersion property: The major PostgreSQL version of server. - * + * * @return the postgresqlVersion value. */ String postgresqlVersion(); /** * Gets the citusVersion property: The Citus extension version of server. - * + * * @return the citusVersion value. */ String citusVersion(); /** * Gets the serverEdition property: The edition of a server. - * + * * @return the serverEdition value. */ String serverEdition(); /** * Gets the storageQuotaInMb property: The storage of a server in MB. - * + * * @return the storageQuotaInMb value. */ Integer storageQuotaInMb(); /** * Gets the vCores property: The vCores count of a server. - * + * * @return the vCores value. */ Integer vCores(); /** * Gets the enableHa property: If high availability (HA) is enabled or not for the server. - * + * * @return the enableHa value. */ Boolean enableHa(); /** * Gets the enablePublicIpAccess property: If public access is enabled on server. - * + * * @return the enablePublicIpAccess value. */ Boolean enablePublicIpAccess(); @@ -124,21 +126,21 @@ public interface ClusterServer { /** * Gets the isReadOnly property: If server database is set to read-only by system maintenance depending on high disk * space usage. - * + * * @return the isReadOnly value. */ Boolean isReadOnly(); /** * Gets the administratorLogin property: The administrator's login name of the servers in the cluster. - * + * * @return the administratorLogin value. */ String administratorLogin(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterServerInner object. - * + * * @return the inner object. */ ClusterServerInner innerModel(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServerListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServerListResult.java index da1026cc60b51..bc13329a1b18c 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServerListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ClusterServerListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of servers in a cluster. */ +/** + * A list of servers in a cluster. + */ @Fluent public final class ClusterServerListResult { /* @@ -18,13 +20,15 @@ public final class ClusterServerListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of ClusterServerListResult class. */ + /** + * Creates an instance of ClusterServerListResult class. + */ public ClusterServerListResult() { } /** * Get the value property: The list of servers in a cluster. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: The list of servers in a cluster. - * + * * @param value the value value to set. * @return the ClusterServerListResult object itself. */ @@ -44,7 +48,7 @@ public ClusterServerListResult withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Clusters.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Clusters.java index 37a82e5cf613b..24408eb38a341 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Clusters.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Clusters.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Clusters. */ +/** + * Resource collection API of Clusters. + */ public interface Clusters { /** * Lists all clusters in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of clusters as paginated response with {@link PagedIterable}. @@ -21,7 +23,7 @@ public interface Clusters { /** * Lists all clusters in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,7 +34,7 @@ public interface Clusters { /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -43,7 +45,7 @@ public interface Clusters { /** * Lists all clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -56,7 +58,7 @@ public interface Clusters { /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -64,27 +66,27 @@ public interface Clusters { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response}. + * as cluster creation date and time along with {@link Response}. */ Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, Context context); /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time. + * as cluster creation date and time. */ Cluster getByResourceGroup(String resourceGroupName, String clusterName); /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -95,7 +97,7 @@ public interface Clusters { /** * Deletes a cluster together with servers in it. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -107,7 +109,7 @@ public interface Clusters { /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -118,7 +120,7 @@ public interface Clusters { /** * Restarts all nodes in the cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -130,7 +132,7 @@ public interface Clusters { /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -141,7 +143,7 @@ public interface Clusters { /** * Starts stopped compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -153,7 +155,7 @@ public interface Clusters { /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -164,7 +166,7 @@ public interface Clusters { /** * Stops compute on all cluster nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -176,7 +178,7 @@ public interface Clusters { /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -187,21 +189,23 @@ public interface Clusters { /** * Promotes read replica cluster to an independent read-write cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. + * @param promoteRequest The parameter for enabling geo-redundant backup while promoting read replica. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void promoteReadReplica(String resourceGroupName, String clusterName, Context context); + void promoteReadReplica(String resourceGroupName, String clusterName, PromoteRequest promoteRequest, + Context context); /** * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -209,14 +213,14 @@ public interface Clusters { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents cluster name availability along with {@link Response}. */ - Response checkNameAvailabilityWithResponse( - NameAvailabilityRequest nameAvailabilityRequest, Context context); + Response checkNameAvailabilityWithResponse(NameAvailabilityRequest nameAvailabilityRequest, + Context context); /** * Checks availability of a cluster name. Cluster names should be globally unique; at least 3 characters and at most * 40 characters long; they must only contain lowercase letters, numbers, and hyphens; and must not start or end * with a hyphen. - * + * * @param nameAvailabilityRequest The required parameters for checking if cluster name is available. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -228,33 +232,33 @@ Response checkNameAvailabilityWithResponse( /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response}. + * as cluster creation date and time along with {@link Response}. */ Cluster getById(String id); /** * Gets information about a cluster such as compute and storage configuration and cluster lifecycle metadata such as * cluster creation date and time. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster such as compute and storage configuration and cluster lifecycle metadata such - * as cluster creation date and time along with {@link Response}. + * as cluster creation date and time along with {@link Response}. */ Response getByIdWithResponse(String id, Context context); /** * Deletes a cluster together with servers in it. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -264,7 +268,7 @@ Response checkNameAvailabilityWithResponse( /** * Deletes a cluster together with servers in it. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -275,7 +279,7 @@ Response checkNameAvailabilityWithResponse( /** * Begins definition for a new Cluster resource. - * + * * @param name resource name. * @return the first stage of the new Cluster definition. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configuration.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configuration.java index 0a23e288c556d..cb275e9861c83 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configuration.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configuration.java @@ -8,81 +8,83 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ConfigurationInner; import java.util.List; -/** An immutable client-side representation of Configuration. */ +/** + * An immutable client-side representation of Configuration. + */ public interface Configuration { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the description property: Description of the configuration. - * + * * @return the description value. */ String description(); /** * Gets the dataType property: Data type of the configuration. - * + * * @return the dataType value. */ ConfigurationDataType dataType(); /** * Gets the allowedValues property: Allowed values of the configuration. - * + * * @return the allowedValues value. */ String allowedValues(); /** * Gets the requiresRestart property: If configuration change requires restart. - * + * * @return the requiresRestart value. */ Boolean requiresRestart(); /** * Gets the serverRoleGroupConfigurations property: The list of server role group configuration values. - * + * * @return the serverRoleGroupConfigurations value. */ List serverRoleGroupConfigurations(); /** * Gets the provisioningState property: Provisioning state of the configuration. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ConfigurationInner object. - * + * * @return the inner object. */ ConfigurationInner innerModel(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ConfigurationDataType.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ConfigurationDataType.java index d7e0e3a6ddc8b..1b7892fd664fa 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ConfigurationDataType.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ConfigurationDataType.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Data type of the configuration. */ +/** + * Data type of the configuration. + */ public final class ConfigurationDataType extends ExpandableStringEnum { - /** Static value Boolean for ConfigurationDataType. */ + /** + * Static value Boolean for ConfigurationDataType. + */ public static final ConfigurationDataType BOOLEAN = fromString("Boolean"); - /** Static value Numeric for ConfigurationDataType. */ + /** + * Static value Numeric for ConfigurationDataType. + */ public static final ConfigurationDataType NUMERIC = fromString("Numeric"); - /** Static value Integer for ConfigurationDataType. */ + /** + * Static value Integer for ConfigurationDataType. + */ public static final ConfigurationDataType INTEGER = fromString("Integer"); - /** Static value Enumeration for ConfigurationDataType. */ + /** + * Static value Enumeration for ConfigurationDataType. + */ public static final ConfigurationDataType ENUMERATION = fromString("Enumeration"); /** * Creates a new instance of ConfigurationDataType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public ConfigurationDataType() { /** * Creates or finds a ConfigurationDataType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ConfigurationDataType. */ @@ -44,7 +54,7 @@ public static ConfigurationDataType fromString(String name) { /** * Gets known ConfigurationDataType values. - * + * * @return known ConfigurationDataType values. */ public static Collection values() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configurations.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configurations.java index 00f79ceb0daa9..9be26396d9b87 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configurations.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Configurations.java @@ -9,11 +9,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** Resource collection API of Configurations. */ +/** + * Resource collection API of Configurations. + */ public interface Configurations { /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -26,7 +28,7 @@ public interface Configurations { /** * List all the configurations of a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -36,12 +38,12 @@ public interface Configurations { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of server configurations as paginated response with {@link PagedIterable}. */ - PagedIterable listByServer( - String resourceGroupName, String clusterName, String serverName, Context context); + PagedIterable listByServer(String resourceGroupName, String clusterName, String serverName, + Context context); /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -53,7 +55,7 @@ PagedIterable listByServer( /** * List all the configurations of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -66,7 +68,7 @@ PagedIterable listByServer( /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -76,12 +78,12 @@ PagedIterable listByServer( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for coordinator and nodes along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String configurationName, + Context context); /** * Gets information of a configuration for coordinator and nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -94,7 +96,7 @@ Response getWithResponse( /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -104,12 +106,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for coordinator along with {@link Response}. */ - Response getCoordinatorWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context); + Response getCoordinatorWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context); /** * Gets information of a configuration for coordinator. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -122,7 +124,7 @@ Response getCoordinatorWithResponse( /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -132,12 +134,12 @@ Response getCoordinatorWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents a configuration. */ - ServerConfiguration updateOnCoordinator( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters); + ServerConfiguration updateOnCoordinator(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters); /** * Updates configuration of coordinator in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -148,16 +150,12 @@ ServerConfiguration updateOnCoordinator( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents a configuration. */ - ServerConfiguration updateOnCoordinator( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context); + ServerConfiguration updateOnCoordinator(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters, Context context); /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -167,12 +165,12 @@ ServerConfiguration updateOnCoordinator( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information of a configuration for worker nodes along with {@link Response}. */ - Response getNodeWithResponse( - String resourceGroupName, String clusterName, String configurationName, Context context); + Response getNodeWithResponse(String resourceGroupName, String clusterName, + String configurationName, Context context); /** * Gets information of a configuration for worker nodes. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -185,7 +183,7 @@ Response getNodeWithResponse( /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -195,12 +193,12 @@ Response getNodeWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents a configuration. */ - ServerConfiguration updateOnNode( - String resourceGroupName, String clusterName, String configurationName, ServerConfigurationInner parameters); + ServerConfiguration updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters); /** * Updates configuration of worker nodes in a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param configurationName The name of the cluster configuration. @@ -211,10 +209,6 @@ ServerConfiguration updateOnNode( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents a configuration. */ - ServerConfiguration updateOnNode( - String resourceGroupName, - String clusterName, - String configurationName, - ServerConfigurationInner parameters, - Context context); + ServerConfiguration updateOnNode(String resourceGroupName, String clusterName, String configurationName, + ServerConfigurationInner parameters, Context context); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRule.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRule.java index 067e56856672b..cd06165f30b41 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRule.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRule.java @@ -8,91 +8,97 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.FirewallRuleInner; -/** An immutable client-side representation of FirewallRule. */ +/** + * An immutable client-side representation of FirewallRule. + */ public interface FirewallRule { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @return the startIpAddress value. */ String startIpAddress(); /** * Gets the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 format. - * + * * @return the endIpAddress value. */ String endIpAddress(); /** * Gets the provisioningState property: Provisioning state of the firewall rule. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.FirewallRuleInner object. - * + * * @return the inner object. */ FirewallRuleInner innerModel(); - /** The entirety of the FirewallRule definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithParentResource, - DefinitionStages.WithStartIpAddress, - DefinitionStages.WithEndIpAddress, - DefinitionStages.WithCreate { + /** + * The entirety of the FirewallRule definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, + DefinitionStages.WithStartIpAddress, DefinitionStages.WithEndIpAddress, DefinitionStages.WithCreate { } - /** The FirewallRule definition stages. */ + /** + * The FirewallRule definition stages. + */ interface DefinitionStages { - /** The first stage of the FirewallRule definition. */ + /** + * The first stage of the FirewallRule definition. + */ interface Blank extends WithParentResource { } - /** The stage of the FirewallRule definition allowing to specify parent resource. */ + /** + * The stage of the FirewallRule definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, clusterName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @return the next definition stage. @@ -100,24 +106,28 @@ interface WithParentResource { WithStartIpAddress withExistingServerGroupsv2(String resourceGroupName, String clusterName); } - /** The stage of the FirewallRule definition allowing to specify startIpAddress. */ + /** + * The stage of the FirewallRule definition allowing to specify startIpAddress. + */ interface WithStartIpAddress { /** * Specifies the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 * format.. - * + * * @param startIpAddress The start IP address of the cluster firewall rule. Must be IPv4 format. * @return the next definition stage. */ WithEndIpAddress withStartIpAddress(String startIpAddress); } - /** The stage of the FirewallRule definition allowing to specify endIpAddress. */ + /** + * The stage of the FirewallRule definition allowing to specify endIpAddress. + */ interface WithEndIpAddress { /** * Specifies the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 * format.. - * + * * @param endIpAddress The end IP address of the cluster firewall rule. Must be IPv4 format. * @return the next definition stage. */ @@ -131,14 +141,14 @@ interface WithEndIpAddress { interface WithCreate { /** * Executes the create request. - * + * * @return the created resource. */ FirewallRule create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ @@ -148,49 +158,57 @@ interface WithCreate { /** * Begins update for the FirewallRule resource. - * + * * @return the stage of resource update. */ FirewallRule.Update update(); - /** The template for FirewallRule update. */ + /** + * The template for FirewallRule update. + */ interface Update extends UpdateStages.WithStartIpAddress, UpdateStages.WithEndIpAddress { /** * Executes the update request. - * + * * @return the updated resource. */ FirewallRule apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ FirewallRule apply(Context context); } - /** The FirewallRule update stages. */ + /** + * The FirewallRule update stages. + */ interface UpdateStages { - /** The stage of the FirewallRule update allowing to specify startIpAddress. */ + /** + * The stage of the FirewallRule update allowing to specify startIpAddress. + */ interface WithStartIpAddress { /** * Specifies the startIpAddress property: The start IP address of the cluster firewall rule. Must be IPv4 * format.. - * + * * @param startIpAddress The start IP address of the cluster firewall rule. Must be IPv4 format. * @return the next definition stage. */ Update withStartIpAddress(String startIpAddress); } - /** The stage of the FirewallRule update allowing to specify endIpAddress. */ + /** + * The stage of the FirewallRule update allowing to specify endIpAddress. + */ interface WithEndIpAddress { /** * Specifies the endIpAddress property: The end IP address of the cluster firewall rule. Must be IPv4 * format.. - * + * * @param endIpAddress The end IP address of the cluster firewall rule. Must be IPv4 format. * @return the next definition stage. */ @@ -200,14 +218,14 @@ interface WithEndIpAddress { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ FirewallRule refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRuleListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRuleListResult.java index 99b8c283fc4e4..90db75b3e62d5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRuleListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRuleListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of firewall rules. */ +/** + * A list of firewall rules. + */ @Fluent public final class FirewallRuleListResult { /* @@ -18,13 +20,15 @@ public final class FirewallRuleListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of FirewallRuleListResult class. */ + /** + * Creates an instance of FirewallRuleListResult class. + */ public FirewallRuleListResult() { } /** * Get the value property: The list of firewall rules in a cluster. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: The list of firewall rules in a cluster. - * + * * @param value the value value to set. * @return the FirewallRuleListResult object itself. */ @@ -44,7 +48,7 @@ public FirewallRuleListResult withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRules.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRules.java index 4b689655587a2..1f44d82ea9c71 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRules.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/FirewallRules.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of FirewallRules. */ +/** + * Resource collection API of FirewallRules. + */ public interface FirewallRules { /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -24,7 +26,7 @@ public interface FirewallRules { /** * Deletes a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -37,7 +39,7 @@ public interface FirewallRules { /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -47,12 +49,12 @@ public interface FirewallRules { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a cluster firewall rule along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String firewallRuleName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String firewallRuleName, + Context context); /** * Gets information about a cluster firewall rule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param firewallRuleName The name of the cluster firewall rule. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Lists all the firewall rules on cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -90,7 +92,7 @@ Response getWithResponse( /** * Gets information about a cluster firewall rule. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -101,7 +103,7 @@ Response getWithResponse( /** * Gets information about a cluster firewall rule. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,7 +115,7 @@ Response getWithResponse( /** * Deletes a cluster firewall rule. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -123,7 +125,7 @@ Response getWithResponse( /** * Deletes a cluster firewall rule. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -134,7 +136,7 @@ Response getWithResponse( /** * Begins definition for a new FirewallRule resource. - * + * * @param name resource name. * @return the first stage of the new FirewallRule definition. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/MaintenanceWindow.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/MaintenanceWindow.java index 6d900314f95d1..911b6ca54def8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/MaintenanceWindow.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/MaintenanceWindow.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Schedule settings for regular cluster updates. */ +/** + * Schedule settings for regular cluster updates. + */ @Fluent public final class MaintenanceWindow { /* @@ -34,13 +36,15 @@ public final class MaintenanceWindow { @JsonProperty(value = "dayOfWeek") private Integer dayOfWeek; - /** Creates an instance of MaintenanceWindow class. */ + /** + * Creates an instance of MaintenanceWindow class. + */ public MaintenanceWindow() { } /** * Get the customWindow property: Indicates whether custom maintenance window is enabled or not. - * + * * @return the customWindow value. */ public String customWindow() { @@ -49,7 +53,7 @@ public String customWindow() { /** * Set the customWindow property: Indicates whether custom maintenance window is enabled or not. - * + * * @param customWindow the customWindow value to set. * @return the MaintenanceWindow object itself. */ @@ -60,7 +64,7 @@ public MaintenanceWindow withCustomWindow(String customWindow) { /** * Get the startHour property: Start hour within preferred day of the week for maintenance window. - * + * * @return the startHour value. */ public Integer startHour() { @@ -69,7 +73,7 @@ public Integer startHour() { /** * Set the startHour property: Start hour within preferred day of the week for maintenance window. - * + * * @param startHour the startHour value to set. * @return the MaintenanceWindow object itself. */ @@ -80,7 +84,7 @@ public MaintenanceWindow withStartHour(Integer startHour) { /** * Get the startMinute property: Start minute within the start hour for maintenance window. - * + * * @return the startMinute value. */ public Integer startMinute() { @@ -89,7 +93,7 @@ public Integer startMinute() { /** * Set the startMinute property: Start minute within the start hour for maintenance window. - * + * * @param startMinute the startMinute value to set. * @return the MaintenanceWindow object itself. */ @@ -100,7 +104,7 @@ public MaintenanceWindow withStartMinute(Integer startMinute) { /** * Get the dayOfWeek property: Preferred day of the week for maintenance window. - * + * * @return the dayOfWeek value. */ public Integer dayOfWeek() { @@ -109,7 +113,7 @@ public Integer dayOfWeek() { /** * Set the dayOfWeek property: Preferred day of the week for maintenance window. - * + * * @param dayOfWeek the dayOfWeek value to set. * @return the MaintenanceWindow object itself. */ @@ -120,7 +124,7 @@ public MaintenanceWindow withDayOfWeek(Integer dayOfWeek) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailability.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailability.java index 85e8ff44120f7..d862334a4c987 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailability.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailability.java @@ -6,39 +6,41 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.NameAvailabilityInner; -/** An immutable client-side representation of NameAvailability. */ +/** + * An immutable client-side representation of NameAvailability. + */ public interface NameAvailability { /** * Gets the message property: Error message. - * + * * @return the message value. */ String message(); /** * Gets the nameAvailable property: Indicates whether the cluster name is available. - * + * * @return the nameAvailable value. */ Boolean nameAvailable(); /** * Gets the name property: Name of the cluster. - * + * * @return the name value. */ String name(); /** * Gets the type property: Type of the cluster. - * + * * @return the type value. */ String type(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.NameAvailabilityInner object. - * + * * @return the inner object. */ NameAvailabilityInner innerModel(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailabilityRequest.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailabilityRequest.java index 22707627b26df..33208b8b64aa0 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailabilityRequest.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/NameAvailabilityRequest.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Request from client to check cluster name availability. */ +/** + * Request from client to check cluster name availability. + */ @Fluent public final class NameAvailabilityRequest { /* @@ -23,14 +25,15 @@ public final class NameAvailabilityRequest { @JsonProperty(value = "type", required = true) private String type = "Microsoft.DBforPostgreSQL/serverGroupsv2"; - /** Creates an instance of NameAvailabilityRequest class. */ + /** + * Creates an instance of NameAvailabilityRequest class. + */ public NameAvailabilityRequest() { - type = "Microsoft.DBforPostgreSQL/serverGroupsv2"; } /** * Get the name property: Cluster name to verify. - * + * * @return the name value. */ public String name() { @@ -39,7 +42,7 @@ public String name() { /** * Set the name property: Cluster name to verify. - * + * * @param name the name value to set. * @return the NameAvailabilityRequest object itself. */ @@ -50,7 +53,7 @@ public NameAvailabilityRequest withName(String name) { /** * Get the type property: Resource type used for verification. - * + * * @return the type value. */ public String type() { @@ -59,7 +62,7 @@ public String type() { /** * Set the type property: Resource type used for verification. - * + * * @param type the type value to set. * @return the NameAvailabilityRequest object itself. */ @@ -70,14 +73,13 @@ public NameAvailabilityRequest withType(String type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property name in model NameAvailabilityRequest")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model NameAvailabilityRequest")); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operation.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operation.java index 67d4ea4ac64fa..d351c2ba9c8e5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operation.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operation.java @@ -7,46 +7,48 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.OperationInner; import java.util.Map; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: The name of the operation being performed on this particular object. - * + * * @return the name value. */ String name(); /** * Gets the display property: The localized display information for this particular operation or action. - * + * * @return the display value. */ OperationDisplay display(); /** * Gets the isDataAction property: Indicates whether the operation is a data action. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the origin property: The intended executor of the operation. - * + * * @return the origin value. */ OperationOrigin origin(); /** * Gets the properties property: Additional descriptions for the operation. - * + * * @return the properties value. */ Map properties(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationDisplay.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationDisplay.java index 306850501bb78..f328ca5d1b73b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationDisplay.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Display metadata associated with the operation. */ +/** + * Display metadata associated with the operation. + */ @Immutable public final class OperationDisplay { /* @@ -34,13 +36,15 @@ public final class OperationDisplay { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: Operation resource provider name. - * + * * @return the provider value. */ public String provider() { @@ -49,7 +53,7 @@ public String provider() { /** * Get the resource property: Resource on which the operation is performed. - * + * * @return the resource value. */ public String resource() { @@ -58,7 +62,7 @@ public String resource() { /** * Get the operation property: Localized friendly name for the operation. - * + * * @return the operation value. */ public String operation() { @@ -67,7 +71,7 @@ public String operation() { /** * Get the description property: Operation description. - * + * * @return the description value. */ public String description() { @@ -76,7 +80,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationListResult.java index 1cbb0a54cdbec..50408822fbf83 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of resource provider operations. */ +/** + * A list of resource provider operations. + */ @Fluent public final class OperationListResult { /* @@ -25,13 +27,15 @@ public final class OperationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationListResult class. */ + /** + * Creates an instance of OperationListResult class. + */ public OperationListResult() { } /** * Get the value property: Collection of available operation details. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: Collection of available operation details. - * + * * @param value the value value to set. * @return the OperationListResult object itself. */ @@ -50,9 +54,9 @@ public OperationListResult withValue(List value) { } /** - * Get the nextLink property: URL client should use to fetch the next page (per server side paging). It's null for - * now, added for future use. - * + * Get the nextLink property: URL client should use to fetch the next page (per server side paging). + * It's null for now, added for future use. + * * @return the nextLink value. */ public String nextLink() { @@ -61,7 +65,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationOrigin.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationOrigin.java index 9e6f0dd815c4e..b47412a939757 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationOrigin.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/OperationOrigin.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The intended executor of the operation. */ +/** + * The intended executor of the operation. + */ public final class OperationOrigin extends ExpandableStringEnum { - /** Static value NotSpecified for OperationOrigin. */ + /** + * Static value NotSpecified for OperationOrigin. + */ public static final OperationOrigin NOT_SPECIFIED = fromString("NotSpecified"); - /** Static value user for OperationOrigin. */ + /** + * Static value user for OperationOrigin. + */ public static final OperationOrigin USER = fromString("user"); - /** Static value system for OperationOrigin. */ + /** + * Static value system for OperationOrigin. + */ public static final OperationOrigin SYSTEM = fromString("system"); /** * Creates a new instance of OperationOrigin value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public OperationOrigin() { /** * Creates or finds a OperationOrigin from its string representation. - * + * * @param name a name to look for. * @return the corresponding OperationOrigin. */ @@ -41,7 +49,7 @@ public static OperationOrigin fromString(String name) { /** * Gets known OperationOrigin values. - * + * * @return known OperationOrigin values. */ public static Collection values() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operations.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operations.java index 4aeb8142743eb..b16d5b1a8c82a 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operations.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Operations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Lists all of the available REST API operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of resource provider operations as paginated response with {@link PagedIterable}. @@ -20,7 +22,7 @@ public interface Operations { /** * Lists all of the available REST API operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PasswordAuth.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PasswordAuth.java new file mode 100644 index 0000000000000..0d5541043c4ae --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PasswordAuth.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for PasswordAuth. + */ +public final class PasswordAuth extends ExpandableStringEnum { + /** + * Static value enabled for PasswordAuth. + */ + public static final PasswordAuth ENABLED = fromString("enabled"); + + /** + * Static value disabled for PasswordAuth. + */ + public static final PasswordAuth DISABLED = fromString("disabled"); + + /** + * Creates a new instance of PasswordAuth value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PasswordAuth() { + } + + /** + * Creates or finds a PasswordAuth from its string representation. + * + * @param name a name to look for. + * @return the corresponding PasswordAuth. + */ + @JsonCreator + public static PasswordAuth fromString(String name) { + return fromString(name, PasswordAuth.class); + } + + /** + * Gets known PasswordAuth values. + * + * @return known PasswordAuth values. + */ + public static Collection values() { + return values(PasswordAuth.class); + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrincipalType.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrincipalType.java new file mode 100644 index 0000000000000..f6afaaf1d081c --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrincipalType.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for PrincipalType. + */ +public final class PrincipalType extends ExpandableStringEnum { + /** + * Static value user for PrincipalType. + */ + public static final PrincipalType USER = fromString("user"); + + /** + * Static value servicePrincipal for PrincipalType. + */ + public static final PrincipalType SERVICE_PRINCIPAL = fromString("servicePrincipal"); + + /** + * Static value group for PrincipalType. + */ + public static final PrincipalType GROUP = fromString("group"); + + /** + * Creates a new instance of PrincipalType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PrincipalType() { + } + + /** + * Creates or finds a PrincipalType from its string representation. + * + * @param name a name to look for. + * @return the corresponding PrincipalType. + */ + @JsonCreator + public static PrincipalType fromString(String name) { + return fromString(name, PrincipalType.class); + } + + /** + * Gets known PrincipalType values. + * + * @return known PrincipalType values. + */ + public static Collection values() { + return values(PrincipalType.class); + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpoint.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpoint.java index b30b74adc9fd8..d37c299cdaceb 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpoint.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpoint.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The private endpoint resource. */ +/** + * The private endpoint resource. + */ @Immutable public final class PrivateEndpoint { /* @@ -16,13 +18,15 @@ public final class PrivateEndpoint { @JsonProperty(value = "id", access = JsonProperty.Access.WRITE_ONLY) private String id; - /** Creates an instance of PrivateEndpoint class. */ + /** + * Creates an instance of PrivateEndpoint class. + */ public PrivateEndpoint() { } /** * Get the id property: The ARM identifier for private endpoint. - * + * * @return the id value. */ public String id() { @@ -31,7 +35,7 @@ public String id() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnection.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnection.java index 5d0353c5bedf0..c9fe933c64460 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnection.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnection.java @@ -9,46 +9,48 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateEndpointConnectionInner; import java.util.List; -/** An immutable client-side representation of PrivateEndpointConnection. */ +/** + * An immutable client-side representation of PrivateEndpointConnection. + */ public interface PrivateEndpointConnection { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the groupIds property: The group ids for the private endpoint resource. - * + * * @return the groupIds value. */ List groupIds(); /** * Gets the privateEndpoint property: The private endpoint resource. - * + * * @return the privateEndpoint value. */ PrivateEndpoint privateEndpoint(); @@ -56,21 +58,21 @@ public interface PrivateEndpointConnection { /** * Gets the privateLinkServiceConnectionState property: A collection of information about the state of the * connection between service consumer and provider. - * + * * @return the privateLinkServiceConnectionState value. */ PrivateLinkServiceConnectionState privateLinkServiceConnectionState(); /** * Gets the provisioningState property: The provisioning state of the private endpoint connection resource. - * + * * @return the provisioningState value. */ PrivateEndpointConnectionProvisioningState provisioningState(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); @@ -78,27 +80,35 @@ public interface PrivateEndpointConnection { /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateEndpointConnectionInner * object. - * + * * @return the inner object. */ PrivateEndpointConnectionInner innerModel(); - /** The entirety of the PrivateEndpointConnection definition. */ + /** + * The entirety of the PrivateEndpointConnection definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The PrivateEndpointConnection definition stages. */ + /** + * The PrivateEndpointConnection definition stages. + */ interface DefinitionStages { - /** The first stage of the PrivateEndpointConnection definition. */ + /** + * The first stage of the PrivateEndpointConnection definition. + */ interface Blank extends WithParentResource { } - /** The stage of the PrivateEndpointConnection definition allowing to specify parent resource. */ + /** + * The stage of the PrivateEndpointConnection definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, clusterName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @return the next definition stage. @@ -114,25 +124,27 @@ interface WithCreate extends DefinitionStages.WithPrivateEndpoint, DefinitionStages.WithPrivateLinkServiceConnectionState { /** * Executes the create request. - * + * * @return the created resource. */ PrivateEndpointConnection create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ PrivateEndpointConnection create(Context context); } - /** The stage of the PrivateEndpointConnection definition allowing to specify privateEndpoint. */ + /** + * The stage of the PrivateEndpointConnection definition allowing to specify privateEndpoint. + */ interface WithPrivateEndpoint { /** * Specifies the privateEndpoint property: The private endpoint resource.. - * + * * @param privateEndpoint The private endpoint resource. * @return the next definition stage. */ @@ -146,9 +158,9 @@ interface WithPrivateLinkServiceConnectionState { /** * Specifies the privateLinkServiceConnectionState property: A collection of information about the state of * the connection between service consumer and provider.. - * + * * @param privateLinkServiceConnectionState A collection of information about the state of the connection - * between service consumer and provider. + * between service consumer and provider. * @return the next definition stage. */ WithCreate withPrivateLinkServiceConnectionState( @@ -158,50 +170,58 @@ WithCreate withPrivateLinkServiceConnectionState( /** * Begins update for the PrivateEndpointConnection resource. - * + * * @return the stage of resource update. */ PrivateEndpointConnection.Update update(); - /** The template for PrivateEndpointConnection update. */ + /** + * The template for PrivateEndpointConnection update. + */ interface Update extends UpdateStages.WithPrivateEndpoint, UpdateStages.WithPrivateLinkServiceConnectionState { /** * Executes the update request. - * + * * @return the updated resource. */ PrivateEndpointConnection apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ PrivateEndpointConnection apply(Context context); } - /** The PrivateEndpointConnection update stages. */ + /** + * The PrivateEndpointConnection update stages. + */ interface UpdateStages { - /** The stage of the PrivateEndpointConnection update allowing to specify privateEndpoint. */ + /** + * The stage of the PrivateEndpointConnection update allowing to specify privateEndpoint. + */ interface WithPrivateEndpoint { /** * Specifies the privateEndpoint property: The private endpoint resource.. - * + * * @param privateEndpoint The private endpoint resource. * @return the next definition stage. */ Update withPrivateEndpoint(PrivateEndpoint privateEndpoint); } - /** The stage of the PrivateEndpointConnection update allowing to specify privateLinkServiceConnectionState. */ + /** + * The stage of the PrivateEndpointConnection update allowing to specify privateLinkServiceConnectionState. + */ interface WithPrivateLinkServiceConnectionState { /** * Specifies the privateLinkServiceConnectionState property: A collection of information about the state of * the connection between service consumer and provider.. - * + * * @param privateLinkServiceConnectionState A collection of information about the state of the connection - * between service consumer and provider. + * between service consumer and provider. * @return the next definition stage. */ Update withPrivateLinkServiceConnectionState( @@ -211,14 +231,14 @@ Update withPrivateLinkServiceConnectionState( /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ PrivateEndpointConnection refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnectionListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnectionListResult.java index 2019a80aab2a2..66a99c69e3c98 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnectionListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnectionListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of private endpoint connections associated with the specified resource. */ +/** + * List of private endpoint connections associated with the specified resource. + */ @Fluent public final class PrivateEndpointConnectionListResult { /* @@ -18,13 +20,15 @@ public final class PrivateEndpointConnectionListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of PrivateEndpointConnectionListResult class. */ + /** + * Creates an instance of PrivateEndpointConnectionListResult class. + */ public PrivateEndpointConnectionListResult() { } /** * Get the value property: Array of private endpoint connections. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Array of private endpoint connections. - * + * * @param value the value value to set. * @return the PrivateEndpointConnectionListResult object itself. */ @@ -44,7 +48,7 @@ public PrivateEndpointConnectionListResult withValue(List { - /** Static value Succeeded for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Succeeded for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Creating for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Creating for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState CREATING = fromString("Creating"); - /** Static value Deleting for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Deleting for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState DELETING = fromString("Deleting"); - /** Static value Failed for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Failed for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState FAILED = fromString("Failed"); /** * Creates a new instance of PrivateEndpointConnectionProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -34,7 +44,7 @@ public PrivateEndpointConnectionProvisioningState() { /** * Creates or finds a PrivateEndpointConnectionProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding PrivateEndpointConnectionProvisioningState. */ @@ -45,7 +55,7 @@ public static PrivateEndpointConnectionProvisioningState fromString(String name) /** * Gets known PrivateEndpointConnectionProvisioningState values. - * + * * @return known PrivateEndpointConnectionProvisioningState values. */ public static Collection values() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnections.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnections.java index 2a5fd762a993e..ab1349e89d07c 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnections.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointConnections.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of PrivateEndpointConnections. */ +/** + * Resource collection API of PrivateEndpointConnections. + */ public interface PrivateEndpointConnections { /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface PrivateEndpointConnections { /** * Gets list of private endpoint connections on a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -33,12 +35,12 @@ public interface PrivateEndpointConnections { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connections on a cluster as paginated response with {@link PagedIterable}. */ - PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context); + PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context); /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -48,12 +50,12 @@ PagedIterable listByCluster( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return private endpoint connection along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context); /** * Gets private endpoint connection. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -66,7 +68,7 @@ Response getWithResponse( /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -78,7 +80,7 @@ Response getWithResponse( /** * Deletes a private endpoint connection with a given name. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the cluster. @@ -91,7 +93,7 @@ Response getWithResponse( /** * Gets private endpoint connection. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -102,7 +104,7 @@ Response getWithResponse( /** * Gets private endpoint connection. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -114,7 +116,7 @@ Response getWithResponse( /** * Deletes a private endpoint connection with a given name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -124,7 +126,7 @@ Response getWithResponse( /** * Deletes a private endpoint connection with a given name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -135,7 +137,7 @@ Response getWithResponse( /** * Begins definition for a new PrivateEndpointConnection resource. - * + * * @param name resource name. * @return the first stage of the new PrivateEndpointConnection definition. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointProperty.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointProperty.java index 2e32607cdb04f..60f7cba5c402a 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointProperty.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointProperty.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Property to represent resource id of the private endpoint. */ +/** + * Property to represent resource id of the private endpoint. + */ @Fluent public final class PrivateEndpointProperty { /* @@ -16,13 +18,15 @@ public final class PrivateEndpointProperty { @JsonProperty(value = "id") private String id; - /** Creates an instance of PrivateEndpointProperty class. */ + /** + * Creates an instance of PrivateEndpointProperty class. + */ public PrivateEndpointProperty() { } /** * Get the id property: Resource id of the private endpoint. - * + * * @return the id value. */ public String id() { @@ -31,7 +35,7 @@ public String id() { /** * Set the id property: Resource id of the private endpoint. - * + * * @param id the id value to set. * @return the PrivateEndpointProperty object itself. */ @@ -42,7 +46,7 @@ public PrivateEndpointProperty withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointServiceConnectionStatus.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointServiceConnectionStatus.java index 9ffbe7724570c..192680e519423 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointServiceConnectionStatus.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateEndpointServiceConnectionStatus.java @@ -8,21 +8,29 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The private endpoint connection status. */ +/** + * The private endpoint connection status. + */ public final class PrivateEndpointServiceConnectionStatus extends ExpandableStringEnum { - /** Static value Pending for PrivateEndpointServiceConnectionStatus. */ + /** + * Static value Pending for PrivateEndpointServiceConnectionStatus. + */ public static final PrivateEndpointServiceConnectionStatus PENDING = fromString("Pending"); - /** Static value Approved for PrivateEndpointServiceConnectionStatus. */ + /** + * Static value Approved for PrivateEndpointServiceConnectionStatus. + */ public static final PrivateEndpointServiceConnectionStatus APPROVED = fromString("Approved"); - /** Static value Rejected for PrivateEndpointServiceConnectionStatus. */ + /** + * Static value Rejected for PrivateEndpointServiceConnectionStatus. + */ public static final PrivateEndpointServiceConnectionStatus REJECTED = fromString("Rejected"); /** * Creates a new instance of PrivateEndpointServiceConnectionStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -31,7 +39,7 @@ public PrivateEndpointServiceConnectionStatus() { /** * Creates or finds a PrivateEndpointServiceConnectionStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding PrivateEndpointServiceConnectionStatus. */ @@ -42,7 +50,7 @@ public static PrivateEndpointServiceConnectionStatus fromString(String name) { /** * Gets known PrivateEndpointServiceConnectionStatus values. - * + * * @return known PrivateEndpointServiceConnectionStatus values. */ public static Collection values() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResource.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResource.java index 3f3bf70ae08a8..ee83152d8aec3 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResource.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResource.java @@ -8,60 +8,62 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateLinkResourceInner; import java.util.List; -/** An immutable client-side representation of PrivateLinkResource. */ +/** + * An immutable client-side representation of PrivateLinkResource. + */ public interface PrivateLinkResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the groupId property: The private link resource group id. - * + * * @return the groupId value. */ String groupId(); /** * Gets the requiredMembers property: The private link resource required member names. - * + * * @return the requiredMembers value. */ List requiredMembers(); /** * Gets the requiredZoneNames property: The private link resource private link DNS zone name. - * + * * @return the requiredZoneNames value. */ List requiredZoneNames(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateLinkResourceInner object. - * + * * @return the inner object. */ PrivateLinkResourceInner innerModel(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResourceListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResourceListResult.java index 345d6f4119ef3..9e8af53f75fa8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResourceListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResourceListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of private link resources. */ +/** + * A list of private link resources. + */ @Fluent public final class PrivateLinkResourceListResult { /* @@ -18,13 +20,15 @@ public final class PrivateLinkResourceListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of PrivateLinkResourceListResult class. */ + /** + * Creates an instance of PrivateLinkResourceListResult class. + */ public PrivateLinkResourceListResult() { } /** * Get the value property: Array of private link resources. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Array of private link resources. - * + * * @param value the value value to set. * @return the PrivateLinkResourceListResult object itself. */ @@ -44,7 +48,7 @@ public PrivateLinkResourceListResult withValue(List va /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResources.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResources.java index acf0cc3f1e660..d5d75b6d72b7a 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResources.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkResources.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of PrivateLinkResources. */ +/** + * Resource collection API of PrivateLinkResources. + */ public interface PrivateLinkResources { /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface PrivateLinkResources { /** * Gets the private link resources for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -37,7 +39,7 @@ public interface PrivateLinkResources { /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. @@ -47,12 +49,12 @@ public interface PrivateLinkResources { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a private link resource for cluster along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String privateLinkResourceName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, + String privateLinkResourceName, Context context); /** * Gets a private link resource for cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param privateLinkResourceName The name of the private link resource. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkServiceConnectionState.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkServiceConnectionState.java index 9f618eec6eb35..9bacb78d0a199 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkServiceConnectionState.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PrivateLinkServiceConnectionState.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** A collection of information about the state of the connection between service consumer and provider. */ +/** + * A collection of information about the state of the connection between service consumer and provider. + */ @Fluent public final class PrivateLinkServiceConnectionState { /* @@ -28,14 +30,16 @@ public final class PrivateLinkServiceConnectionState { @JsonProperty(value = "actionsRequired") private String actionsRequired; - /** Creates an instance of PrivateLinkServiceConnectionState class. */ + /** + * Creates an instance of PrivateLinkServiceConnectionState class. + */ public PrivateLinkServiceConnectionState() { } /** * Get the status property: Indicates whether the connection has been Approved/Rejected/Removed by the owner of the * service. - * + * * @return the status value. */ public PrivateEndpointServiceConnectionStatus status() { @@ -45,7 +49,7 @@ public PrivateEndpointServiceConnectionStatus status() { /** * Set the status property: Indicates whether the connection has been Approved/Rejected/Removed by the owner of the * service. - * + * * @param status the status value to set. * @return the PrivateLinkServiceConnectionState object itself. */ @@ -56,7 +60,7 @@ public PrivateLinkServiceConnectionState withStatus(PrivateEndpointServiceConnec /** * Get the description property: The reason for approval/rejection of the connection. - * + * * @return the description value. */ public String description() { @@ -65,7 +69,7 @@ public String description() { /** * Set the description property: The reason for approval/rejection of the connection. - * + * * @param description the description value to set. * @return the PrivateLinkServiceConnectionState object itself. */ @@ -77,7 +81,7 @@ public PrivateLinkServiceConnectionState withDescription(String description) { /** * Get the actionsRequired property: A message indicating if changes on the service provider require any updates on * the consumer. - * + * * @return the actionsRequired value. */ public String actionsRequired() { @@ -87,7 +91,7 @@ public String actionsRequired() { /** * Set the actionsRequired property: A message indicating if changes on the service provider require any updates on * the consumer. - * + * * @param actionsRequired the actionsRequired value to set. * @return the PrivateLinkServiceConnectionState object itself. */ @@ -98,7 +102,7 @@ public PrivateLinkServiceConnectionState withActionsRequired(String actionsRequi /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PromoteRequest.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PromoteRequest.java new file mode 100644 index 0000000000000..34229bb7c306c --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/PromoteRequest.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Request from client to promote geo-redundant replica. + */ +@Fluent +public final class PromoteRequest { + /* + * Cluster name to verify. + */ + @JsonProperty(value = "enableGeoBackup") + private Boolean enableGeoBackup; + + /** + * Creates an instance of PromoteRequest class. + */ + public PromoteRequest() { + } + + /** + * Get the enableGeoBackup property: Cluster name to verify. + * + * @return the enableGeoBackup value. + */ + public Boolean enableGeoBackup() { + return this.enableGeoBackup; + } + + /** + * Set the enableGeoBackup property: Cluster name to verify. + * + * @param enableGeoBackup the enableGeoBackup value to set. + * @return the PromoteRequest object itself. + */ + public PromoteRequest withEnableGeoBackup(Boolean enableGeoBackup) { + this.enableGeoBackup = enableGeoBackup; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ProvisioningState.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ProvisioningState.java index 015c13bb1f436..ab307dc7128a7 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ProvisioningState.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ProvisioningState.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The current provisioning state. */ +/** + * The current provisioning state. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value InProgress for ProvisioningState. */ + /** + * Static value InProgress for ProvisioningState. + */ public static final ProvisioningState IN_PROGRESS = fromString("InProgress"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -44,7 +54,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Role.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Role.java index 4e5ad1747f0a7..0b4ff876b9dbc 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Role.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Role.java @@ -8,126 +8,218 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RoleInner; -/** An immutable client-side representation of Role. */ +/** + * An immutable client-side representation of Role. + */ public interface Role { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** - * Gets the password property: The password of the cluster role. - * + * Gets the roleType property: The roleType property. + * + * @return the roleType value. + */ + RoleType roleType(); + + /** + * Gets the password property: The password of the cluster role. If an identity is used, password will not be + * required. + * * @return the password value. */ String password(); /** * Gets the provisioningState property: Provisioning state of the role. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); + /** + * Gets the objectId property: The objectId property. + * + * @return the objectId value. + */ + String objectId(); + + /** + * Gets the principalType property: The principalType property. + * + * @return the principalType value. + */ + PrincipalType principalType(); + + /** + * Gets the tenantId property: The tenantId property. + * + * @return the tenantId value. + */ + String tenantId(); + /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RoleInner object. - * + * * @return the inner object. */ RoleInner innerModel(); - /** The entirety of the Role definition. */ + /** + * The entirety of the Role definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithParentResource, - DefinitionStages.WithPassword, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Role definition stages. */ + /** + * The Role definition stages. + */ interface DefinitionStages { - /** The first stage of the Role definition. */ + /** + * The first stage of the Role definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Role definition allowing to specify parent resource. */ + /** + * The stage of the Role definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, clusterName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @return the next definition stage. */ - WithPassword withExistingServerGroupsv2(String resourceGroupName, String clusterName); - } - - /** The stage of the Role definition allowing to specify password. */ - interface WithPassword { - /** - * Specifies the password property: The password of the cluster role.. - * - * @param password The password of the cluster role. - * @return the next definition stage. - */ - WithCreate withPassword(String password); + WithCreate withExistingServerGroupsv2(String resourceGroupName, String clusterName); } /** * The stage of the Role definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate { + interface WithCreate extends DefinitionStages.WithRoleType, DefinitionStages.WithPassword, + DefinitionStages.WithObjectId, DefinitionStages.WithPrincipalType, DefinitionStages.WithTenantId { /** * Executes the create request. - * + * * @return the created resource. */ Role create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Role create(Context context); } + + /** + * The stage of the Role definition allowing to specify roleType. + */ + interface WithRoleType { + /** + * Specifies the roleType property: The roleType property.. + * + * @param roleType The roleType property. + * @return the next definition stage. + */ + WithCreate withRoleType(RoleType roleType); + } + + /** + * The stage of the Role definition allowing to specify password. + */ + interface WithPassword { + /** + * Specifies the password property: The password of the cluster role. If an identity is used, password will + * not be required.. + * + * @param password The password of the cluster role. If an identity is used, password will not be required. + * @return the next definition stage. + */ + WithCreate withPassword(String password); + } + + /** + * The stage of the Role definition allowing to specify objectId. + */ + interface WithObjectId { + /** + * Specifies the objectId property: The objectId property.. + * + * @param objectId The objectId property. + * @return the next definition stage. + */ + WithCreate withObjectId(String objectId); + } + + /** + * The stage of the Role definition allowing to specify principalType. + */ + interface WithPrincipalType { + /** + * Specifies the principalType property: The principalType property.. + * + * @param principalType The principalType property. + * @return the next definition stage. + */ + WithCreate withPrincipalType(PrincipalType principalType); + } + + /** + * The stage of the Role definition allowing to specify tenantId. + */ + interface WithTenantId { + /** + * Specifies the tenantId property: The tenantId property.. + * + * @param tenantId The tenantId property. + * @return the next definition stage. + */ + WithCreate withTenantId(String tenantId); + } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Role refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleListResult.java index efb7c6f842b5a..74cf6495e5b23 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of roles. */ +/** + * A list of roles. + */ @Fluent public final class RoleListResult { /* @@ -18,13 +20,15 @@ public final class RoleListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of RoleListResult class. */ + /** + * Creates an instance of RoleListResult class. + */ public RoleListResult() { } /** * Get the value property: The list of roles in a cluster. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: The list of roles in a cluster. - * + * * @param value the value value to set. * @return the RoleListResult object itself. */ @@ -44,7 +48,7 @@ public RoleListResult withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleType.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleType.java new file mode 100644 index 0000000000000..aa192a2eb546f --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/RoleType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for RoleType. + */ +public final class RoleType extends ExpandableStringEnum { + /** + * Static value user for RoleType. + */ + public static final RoleType USER = fromString("user"); + + /** + * Static value admin for RoleType. + */ + public static final RoleType ADMIN = fromString("admin"); + + /** + * Creates a new instance of RoleType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public RoleType() { + } + + /** + * Creates or finds a RoleType from its string representation. + * + * @param name a name to look for. + * @return the corresponding RoleType. + */ + @JsonCreator + public static RoleType fromString(String name) { + return fromString(name, RoleType.class); + } + + /** + * Gets known RoleType values. + * + * @return known RoleType values. + */ + public static Collection values() { + return values(RoleType.class); + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Roles.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Roles.java index 0dba9feb7d153..1a01eec396cfc 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Roles.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Roles.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Roles. */ +/** + * Resource collection API of Roles. + */ public interface Roles { /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -26,7 +28,7 @@ public interface Roles { /** * Gets information about a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -39,7 +41,7 @@ public interface Roles { /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -51,7 +53,7 @@ public interface Roles { /** * Deletes a cluster role. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param roleName The name of the cluster role. @@ -64,7 +66,7 @@ public interface Roles { /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -76,7 +78,7 @@ public interface Roles { /** * List all the roles in a given cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -89,7 +91,7 @@ public interface Roles { /** * Gets information about a cluster role. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -100,7 +102,7 @@ public interface Roles { /** * Gets information about a cluster role. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -112,7 +114,7 @@ public interface Roles { /** * Deletes a cluster role. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -122,7 +124,7 @@ public interface Roles { /** * Deletes a cluster role. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -133,7 +135,7 @@ public interface Roles { /** * Begins definition for a new Role resource. - * + * * @param name resource name. * @return the first stage of the new Role definition. */ diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfiguration.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfiguration.java index aa4e7412d4986..082967230aa02 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfiguration.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfiguration.java @@ -7,95 +7,97 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** An immutable client-side representation of ServerConfiguration. */ +/** + * An immutable client-side representation of ServerConfiguration. + */ public interface ServerConfiguration { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the value property: Value of the configuration. - * + * * @return the value value. */ String value(); /** * Gets the source property: Source of the configuration. - * + * * @return the source value. */ String source(); /** * Gets the description property: Description of the configuration. - * + * * @return the description value. */ String description(); /** * Gets the defaultValue property: Default value of the configuration. - * + * * @return the defaultValue value. */ String defaultValue(); /** * Gets the dataType property: Data type of the configuration. - * + * * @return the dataType value. */ ConfigurationDataType dataType(); /** * Gets the allowedValues property: Allowed values of the configuration. - * + * * @return the allowedValues value. */ String allowedValues(); /** * Gets the requiresRestart property: If configuration change requires restart. - * + * * @return the requiresRestart value. */ Boolean requiresRestart(); /** * Gets the provisioningState property: Provisioning state of the configuration. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the inner com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner object. - * + * * @return the inner object. */ ServerConfigurationInner innerModel(); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfigurationListResult.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfigurationListResult.java index f61d904c72980..ae8465a417d7c 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfigurationListResult.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerConfigurationListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of server configurations. */ +/** + * A list of server configurations. + */ @Fluent public final class ServerConfigurationListResult { /* @@ -24,13 +26,15 @@ public final class ServerConfigurationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ServerConfigurationListResult class. */ + /** + * Creates an instance of ServerConfigurationListResult class. + */ public ServerConfigurationListResult() { } /** * Get the value property: The list of server configurations. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: The list of server configurations. - * + * * @param value the value value to set. * @return the ServerConfigurationListResult object itself. */ @@ -50,7 +54,7 @@ public ServerConfigurationListResult withValue(List va /** * Get the nextLink property: Link to retrieve next page of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerNameItem.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerNameItem.java index 10590b063b1f3..a915dd7ac73d5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerNameItem.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerNameItem.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The name object for a server. */ +/** + * The name object for a server. + */ @Fluent public final class ServerNameItem { /* @@ -22,13 +24,15 @@ public final class ServerNameItem { @JsonProperty(value = "fullyQualifiedDomainName", access = JsonProperty.Access.WRITE_ONLY) private String fullyQualifiedDomainName; - /** Creates an instance of ServerNameItem class. */ + /** + * Creates an instance of ServerNameItem class. + */ public ServerNameItem() { } /** * Get the name property: The name of a server. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Set the name property: The name of a server. - * + * * @param name the name value to set. * @return the ServerNameItem object itself. */ @@ -48,7 +52,7 @@ public ServerNameItem withName(String name) { /** * Get the fullyQualifiedDomainName property: The fully qualified domain name of a server. - * + * * @return the fullyQualifiedDomainName value. */ public String fullyQualifiedDomainName() { @@ -57,7 +61,7 @@ public String fullyQualifiedDomainName() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerProperties.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerProperties.java index 4d4f8127bec32..4779ba98fb462 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerProperties.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The properties of a server. */ +/** + * The properties of a server. + */ @Fluent public class ServerProperties { /* @@ -52,13 +54,15 @@ public class ServerProperties { @JsonProperty(value = "administratorLogin", access = JsonProperty.Access.WRITE_ONLY) private String administratorLogin; - /** Creates an instance of ServerProperties class. */ + /** + * Creates an instance of ServerProperties class. + */ public ServerProperties() { } /** * Get the serverEdition property: The edition of a server. - * + * * @return the serverEdition value. */ public String serverEdition() { @@ -67,7 +71,7 @@ public String serverEdition() { /** * Set the serverEdition property: The edition of a server. - * + * * @param serverEdition the serverEdition value to set. * @return the ServerProperties object itself. */ @@ -78,7 +82,7 @@ public ServerProperties withServerEdition(String serverEdition) { /** * Get the storageQuotaInMb property: The storage of a server in MB. - * + * * @return the storageQuotaInMb value. */ public Integer storageQuotaInMb() { @@ -87,7 +91,7 @@ public Integer storageQuotaInMb() { /** * Set the storageQuotaInMb property: The storage of a server in MB. - * + * * @param storageQuotaInMb the storageQuotaInMb value to set. * @return the ServerProperties object itself. */ @@ -98,7 +102,7 @@ public ServerProperties withStorageQuotaInMb(Integer storageQuotaInMb) { /** * Get the vCores property: The vCores count of a server. - * + * * @return the vCores value. */ public Integer vCores() { @@ -107,7 +111,7 @@ public Integer vCores() { /** * Set the vCores property: The vCores count of a server. - * + * * @param vCores the vCores value to set. * @return the ServerProperties object itself. */ @@ -118,7 +122,7 @@ public ServerProperties withVCores(Integer vCores) { /** * Get the enableHa property: If high availability (HA) is enabled or not for the server. - * + * * @return the enableHa value. */ public Boolean enableHa() { @@ -127,7 +131,7 @@ public Boolean enableHa() { /** * Set the enableHa property: If high availability (HA) is enabled or not for the server. - * + * * @param enableHa the enableHa value to set. * @return the ServerProperties object itself. */ @@ -138,7 +142,7 @@ public ServerProperties withEnableHa(Boolean enableHa) { /** * Get the enablePublicIpAccess property: If public access is enabled on server. - * + * * @return the enablePublicIpAccess value. */ public Boolean enablePublicIpAccess() { @@ -148,7 +152,7 @@ public Boolean enablePublicIpAccess() { /** * Get the isReadOnly property: If server database is set to read-only by system maintenance depending on high disk * space usage. - * + * * @return the isReadOnly value. */ public Boolean isReadOnly() { @@ -157,7 +161,7 @@ public Boolean isReadOnly() { /** * Get the administratorLogin property: The administrator's login name of the servers in the cluster. - * + * * @return the administratorLogin value. */ public String administratorLogin() { @@ -166,7 +170,7 @@ public String administratorLogin() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRole.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRole.java index 0f381b510c484..e36d803e6ba28 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRole.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRole.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The role of a server. */ +/** + * The role of a server. + */ public final class ServerRole extends ExpandableStringEnum { - /** Static value Coordinator for ServerRole. */ + /** + * Static value Coordinator for ServerRole. + */ public static final ServerRole COORDINATOR = fromString("Coordinator"); - /** Static value Worker for ServerRole. */ + /** + * Static value Worker for ServerRole. + */ public static final ServerRole WORKER = fromString("Worker"); /** * Creates a new instance of ServerRole value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ServerRole() { /** * Creates or finds a ServerRole from its string representation. - * + * * @param name a name to look for. * @return the corresponding ServerRole. */ @@ -38,7 +44,7 @@ public static ServerRole fromString(String name) { /** * Gets known ServerRole values. - * + * * @return known ServerRole values. */ public static Collection values() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRoleGroupConfiguration.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRoleGroupConfiguration.java index dc3a44a74b15b..91d07dd0ebed8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRoleGroupConfiguration.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/ServerRoleGroupConfiguration.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents server role group configuration value. */ +/** + * Represents server role group configuration value. + */ @Fluent public final class ServerRoleGroupConfiguration { /* @@ -35,13 +37,15 @@ public final class ServerRoleGroupConfiguration { @JsonProperty(value = "source", access = JsonProperty.Access.WRITE_ONLY) private String source; - /** Creates an instance of ServerRoleGroupConfiguration class. */ + /** + * Creates an instance of ServerRoleGroupConfiguration class. + */ public ServerRoleGroupConfiguration() { } /** * Get the role property: The role of servers in the server role group. - * + * * @return the role value. */ public ServerRole role() { @@ -50,7 +54,7 @@ public ServerRole role() { /** * Set the role property: The role of servers in the server role group. - * + * * @param role the role value to set. * @return the ServerRoleGroupConfiguration object itself. */ @@ -61,7 +65,7 @@ public ServerRoleGroupConfiguration withRole(ServerRole role) { /** * Get the value property: Value of the configuration. - * + * * @return the value value. */ public String value() { @@ -70,7 +74,7 @@ public String value() { /** * Set the value property: Value of the configuration. - * + * * @param value the value value to set. * @return the ServerRoleGroupConfiguration object itself. */ @@ -81,7 +85,7 @@ public ServerRoleGroupConfiguration withValue(String value) { /** * Get the defaultValue property: Default value of the configuration. - * + * * @return the defaultValue value. */ public String defaultValue() { @@ -90,7 +94,7 @@ public String defaultValue() { /** * Get the source property: Source of the configuration. - * + * * @return the source value. */ public String source() { @@ -99,21 +103,17 @@ public String source() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (role() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property role in model ServerRoleGroupConfiguration")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property role in model ServerRoleGroupConfiguration")); } if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model ServerRoleGroupConfiguration")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ServerRoleGroupConfiguration")); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Servers.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Servers.java index 12af652857c7a..711e67d56c03a 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Servers.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/Servers.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Servers. */ +/** + * Resource collection API of Servers. + */ public interface Servers { /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface Servers { /** * Lists servers of a cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param context The context to associate with this operation. @@ -37,7 +39,7 @@ public interface Servers { /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. @@ -47,12 +49,12 @@ public interface Servers { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a server in cluster along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String serverName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String serverName, + Context context); /** * Gets information about a server in cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the cluster. * @param serverName The name of the server. diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/SimplePrivateEndpointConnection.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/SimplePrivateEndpointConnection.java index 953864c5a9b1c..c2380997f8311 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/SimplePrivateEndpointConnection.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/SimplePrivateEndpointConnection.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A private endpoint connection. */ +/** + * A private endpoint connection. + */ @Fluent public final class SimplePrivateEndpointConnection extends ProxyResource { /* @@ -26,13 +28,15 @@ public final class SimplePrivateEndpointConnection extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of SimplePrivateEndpointConnection class. */ + /** + * Creates an instance of SimplePrivateEndpointConnection class. + */ public SimplePrivateEndpointConnection() { } /** * Get the innerProperties property: Properties of the private endpoint connection. - * + * * @return the innerProperties value. */ private PrivateEndpointConnectionSimpleProperties innerProperties() { @@ -41,7 +45,7 @@ private PrivateEndpointConnectionSimpleProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -50,7 +54,7 @@ public SystemData systemData() { /** * Get the privateEndpoint property: Private endpoint which the connection belongs to. - * + * * @return the privateEndpoint value. */ public PrivateEndpointProperty privateEndpoint() { @@ -59,7 +63,7 @@ public PrivateEndpointProperty privateEndpoint() { /** * Set the privateEndpoint property: Private endpoint which the connection belongs to. - * + * * @param privateEndpoint the privateEndpoint value to set. * @return the SimplePrivateEndpointConnection object itself. */ @@ -73,7 +77,7 @@ public SimplePrivateEndpointConnection withPrivateEndpoint(PrivateEndpointProper /** * Get the groupIds property: Group ids of the private endpoint connection. - * + * * @return the groupIds value. */ public List groupIds() { @@ -82,7 +86,7 @@ public List groupIds() { /** * Set the groupIds property: Group ids of the private endpoint connection. - * + * * @param groupIds the groupIds value to set. * @return the SimplePrivateEndpointConnection object itself. */ @@ -95,9 +99,9 @@ public SimplePrivateEndpointConnection withGroupIds(List groupIds) { } /** - * Get the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Get the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @return the privateLinkServiceConnectionState value. */ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { @@ -105,14 +109,14 @@ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { } /** - * Set the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Set the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. * @return the SimplePrivateEndpointConnection object itself. */ - public SimplePrivateEndpointConnection withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public SimplePrivateEndpointConnection + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { if (this.innerProperties() == null) { this.innerProperties = new PrivateEndpointConnectionSimpleProperties(); } @@ -122,7 +126,7 @@ public SimplePrivateEndpointConnection withPrivateLinkServiceConnectionState( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/package-info.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/package-info.java index b223517568c5d..d878501990350 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/package-info.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the data models for CosmosDBForPostgreSql. Azure Cosmos DB for PostgreSQL database service - * resource provider REST APIs. + * Package containing the data models for CosmosDBForPostgreSql. + * Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. */ package com.azure.resourcemanager.cosmosdbforpostgresql.models; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/package-info.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/package-info.java index 08b96dc0f25eb..c37290e21a82c 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/package-info.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/com/azure/resourcemanager/cosmosdbforpostgresql/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the classes for CosmosDBForPostgreSql. Azure Cosmos DB for PostgreSQL database service resource - * provider REST APIs. + * Package containing the classes for CosmosDBForPostgreSql. + * Azure Cosmos DB for PostgreSQL database service resource provider REST APIs. */ package com.azure.resourcemanager.cosmosdbforpostgresql; diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/module-info.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/module-info.java index 55d14d9dbc1a0..374b549ed9c49 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/module-info.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.cosmosdbforpostgresql { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.cosmosdbforpostgresql; exports com.azure.resourcemanager.cosmosdbforpostgresql.fluent; exports com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models; exports com.azure.resourcemanager.cosmosdbforpostgresql.models; - - opens com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.cosmosdbforpostgresql.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.cosmosdbforpostgresql.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/proxy-config.json b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/proxy-config.json new file mode 100644 index 0000000000000..9d763140b2936 --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.ClustersClientImpl$ClustersService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.ServersClientImpl$ServersService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.ConfigurationsClientImpl$ConfigurationsService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.FirewallRulesClientImpl$FirewallRulesService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.RolesClientImpl$RolesService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.PrivateEndpointConnectionsClientImpl$PrivateEndpointConnectionsService" ], [ "com.azure.resourcemanager.cosmosdbforpostgresql.implementation.PrivateLinkResourcesClientImpl$PrivateLinkResourcesService" ] ] \ No newline at end of file diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/reflect-config.json b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/reflect-config.json new file mode 100644 index 0000000000000..342b813f0f234 --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/reflect-config.json @@ -0,0 +1,266 @@ +[ { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerNameItem", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.SimplePrivateEndpointConnection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateEndpointConnectionSimpleProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpointProperty", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateLinkServiceConnectionState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.AuthConfig", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterForUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterPropertiesForUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterServerListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterServerInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ClusterServerProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerConfigurationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ClusterConfigurationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ConfigurationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ConfigurationProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerRoleGroupConfiguration", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.FirewallRuleInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.FirewallRuleProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.FirewallRuleListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RoleInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RoleProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RolePropertiesExternalIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.RoleListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PromoteRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailabilityRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.NameAvailabilityInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.OperationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpointConnectionListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateEndpointConnectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateEndpointConnectionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpoint", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateLinkResourceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateLinkResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.PrivateLinkResourceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpointServiceConnectionStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ActiveDirectoryAuth", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PasswordAuth", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ServerRole", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ConfigurationDataType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.ProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.RoleType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrincipalType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.OperationOrigin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpointConnectionProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilitySamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilitySamples.java index 531a2fd064269..529e927eed9c2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilitySamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilitySamples.java @@ -6,21 +6,23 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.NameAvailabilityRequest; -/** Samples for Clusters CheckNameAvailability. */ +/** + * Samples for Clusters CheckNameAvailability. + */ public final class ClustersCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/CheckNameAvailability.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * CheckNameAvailability.json */ /** * Sample code: Check name availability. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void checkNameAvailability( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .checkNameAvailabilityWithResponse( - new NameAvailabilityRequest().withName("name1"), com.azure.core.util.Context.NONE); + public static void + checkNameAvailability(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.clusters().checkNameAvailabilityWithResponse(new NameAvailabilityRequest().withName("name1"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCreateSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCreateSamples.java index fb6d0c1ac0a4f..7d4acdbf12d73 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCreateSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCreateSamples.java @@ -8,173 +8,124 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Clusters Create. */ +/** + * Samples for Clusters Create. + */ public final class ClustersCreateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateSingleNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateSingleNode.json */ /** * Sample code: Create a new single node cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewSingleNodeCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-singlenode") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf("owner", "JohnDoe")) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.3") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(true) - .withEnableHa(true) - .withCoordinatorServerEdition("GeneralPurpose") - .withCoordinatorStorageQuotaInMb(131072) - .withCoordinatorVCores(8) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeCount(0) - .create(); + manager.clusters().define("testcluster-singlenode").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf("owner", "JohnDoe")).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.3").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(true) + .withEnableHa(true).withCoordinatorServerEdition("GeneralPurpose").withCoordinatorStorageQuotaInMb(131072) + .withCoordinatorVCores(8).withCoordinatorEnablePublicIpAccess(true).withNodeCount(0).create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateBurstablev1.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateBurstablev1.json */ /** * Sample code: Create a new single node Burstable 1 vCore cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewSingleNodeBurstable1VCoreCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-burstablev1") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf("owner", "JohnDoe")) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.3") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(true) - .withEnableHa(false) - .withCoordinatorServerEdition("BurstableMemoryOptimized") - .withCoordinatorStorageQuotaInMb(131072) - .withCoordinatorVCores(1) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeCount(0) - .create(); + manager.clusters().define("testcluster-burstablev1").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf("owner", "JohnDoe")).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.3").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(true) + .withEnableHa(false).withCoordinatorServerEdition("BurstableMemoryOptimized") + .withCoordinatorStorageQuotaInMb(131072).withCoordinatorVCores(1).withCoordinatorEnablePublicIpAccess(true) + .withNodeCount(0).create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateBurstablev2.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateBurstablev2.json */ /** * Sample code: Create a new single node Burstable 2 vCores cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewSingleNodeBurstable2VCoresCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-burstablev2") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf("owner", "JohnDoe")) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.3") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(true) - .withEnableHa(false) - .withCoordinatorServerEdition("BurstableGeneralPurpose") - .withCoordinatorStorageQuotaInMb(131072) - .withCoordinatorVCores(2) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeCount(0) - .create(); + manager.clusters().define("testcluster-burstablev2").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf("owner", "JohnDoe")).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.3").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(true) + .withEnableHa(false).withCoordinatorServerEdition("BurstableGeneralPurpose") + .withCoordinatorStorageQuotaInMb(131072).withCoordinatorVCores(2).withCoordinatorEnablePublicIpAccess(true) + .withNodeCount(0).create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateMultiNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateMultiNode.json */ /** * Sample code: Create a new multi-node cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewMultiNodeCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster-multinode") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") - .withTags(mapOf()) - .withAdministratorLoginPassword("password") - .withPostgresqlVersion("15") - .withCitusVersion("11.1") - .withPreferredPrimaryZone("1") - .withEnableShardsOnCoordinator(false) - .withEnableHa(true) - .withCoordinatorServerEdition("GeneralPurpose") - .withCoordinatorStorageQuotaInMb(524288) - .withCoordinatorVCores(4) - .withCoordinatorEnablePublicIpAccess(true) - .withNodeServerEdition("MemoryOptimized") - .withNodeCount(3) - .withNodeStorageQuotaInMb(524288) - .withNodeVCores(8) - .withNodeEnablePublicIpAccess(false) + manager.clusters().define("testcluster-multinode").withRegion("westus").withExistingResourceGroup("TestGroup") + .withTags(mapOf()).withAdministratorLoginPassword("password").withPostgresqlVersion("15") + .withCitusVersion("11.1").withPreferredPrimaryZone("1").withEnableShardsOnCoordinator(false) + .withEnableHa(true).withCoordinatorServerEdition("GeneralPurpose").withCoordinatorStorageQuotaInMb(524288) + .withCoordinatorVCores(4).withCoordinatorEnablePublicIpAccess(true).withNodeServerEdition("MemoryOptimized") + .withNodeCount(3).withNodeStorageQuotaInMb(524288).withNodeVCores(8).withNodeEnablePublicIpAccess(false) .create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreateReadReplica.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreateReadReplica.json */ /** * Sample code: Create a new cluster as a read replica. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewClusterAsAReadReplica( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") + manager.clusters().define("testcluster").withRegion("westus").withExistingResourceGroup("TestGroup") .withSourceResourceId( "/subscriptions/ffffffff-ffff-ffff-ffff-ffffffffffff/resourceGroups/TestResourceGroup/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/sourcecluster") - .withSourceLocation("westus") - .create(); + .withSourceLocation("westus").create(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterCreatePITR.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterCreatePITR.json */ /** * Sample code: Create a new cluster as a point in time restore. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createANewClusterAsAPointInTimeRestore( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .define("testcluster") - .withRegion("westus") - .withExistingResourceGroup("TestGroup") + manager.clusters().define("testcluster").withRegion("westus").withExistingResourceGroup("TestGroup") .withSourceResourceId( "/subscriptions/ffffffff-ffff-ffff-ffff-ffffffffffff/resourceGroups/TestResourceGroup/providers/Microsoft.DBforPostgreSQL/serverGroupsv2/source-cluster") - .withSourceLocation("westus") - .withPointInTimeUtc(OffsetDateTime.parse("2017-12-14T00:00:37.467Z")) - .create(); + .withSourceLocation("westus").withPointInTimeUtc(OffsetDateTime.parse("2017-12-14T00:00:37.467Z")).create(); } // Use "Map.of" if available diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteSamples.java index d67909a108b86..95dee4c41b723 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters Delete. */ +/** + * Samples for Clusters Delete. + */ public final class ClustersDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterDelete.json */ /** * Sample code: Delete the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void deleteTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + deleteTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.clusters().delete("TestGroup", "testcluster", com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersGetByResourceGroupSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersGetByResourceGroupSamples.java index 307658010167d..d26615412f56d 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersGetByResourceGroupSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersGetByResourceGroupSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters GetByResourceGroup. */ +/** + * Samples for Clusters GetByResourceGroup. + */ public final class ClustersGetByResourceGroupSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterGet.json */ /** * Sample code: Get the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster1", com.azure.core.util.Context.NONE); + public static void + getTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.clusters().getByResourceGroupWithResponse("TestGroup", "testcluster1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListByResourceGroupSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListByResourceGroupSamples.java index c191499d15f12..f5f8b0016b720 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListByResourceGroupSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListByResourceGroupSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters ListByResourceGroup. */ +/** + * Samples for Clusters ListByResourceGroup. + */ public final class ClustersListByResourceGroupSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterListByResourceGroup.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterListByResourceGroup.json */ /** * Sample code: List the clusters by resource group. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listTheClustersByResourceGroup( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListSamples.java index e54fa455df2cc..1b4ec4737471e 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersListSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters List. */ +/** + * Samples for Clusters List. + */ public final class ClustersListSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterList.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterList.json */ /** * Sample code: List all the clusters. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void listAllTheClusters( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + listAllTheClusters(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.clusters().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersPromoteReadReplicaSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersPromoteReadReplicaSamples.java index b97398b6b4abc..42b10bbcd90eb 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersPromoteReadReplicaSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersPromoteReadReplicaSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters PromoteReadReplica. */ +/** + * Samples for Clusters PromoteReadReplica. + */ public final class ClustersPromoteReadReplicaSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterPromoteReadReplica.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterPromoteReadReplica.json */ /** * Sample code: Promote read replica cluster to an independent read-write cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void promoteReadReplicaClusterToAnIndependentReadWriteCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager.clusters().promoteReadReplica("TestGroup", "testcluster1", com.azure.core.util.Context.NONE); + manager.clusters().promoteReadReplica("TestGroup", "testcluster1", null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersRestartSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersRestartSamples.java index 8dac9401a54aa..707ad8a4fa8a7 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersRestartSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersRestartSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters Restart. */ +/** + * Samples for Clusters Restart. + */ public final class ClustersRestartSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterRestart.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterRestart.json */ /** * Sample code: Restart all servers in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void restartAllServersInTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartSamples.java index 7fba5d03c0e71..1f8dd52a4f7f7 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters Start. */ +/** + * Samples for Clusters Start. + */ public final class ClustersStartSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterStart.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterStart.json */ /** * Sample code: Start all servers in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void startAllServersInTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopSamples.java index 816718089abcf..8aa4f0fba1e04 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Clusters Stop. */ +/** + * Samples for Clusters Stop. + */ public final class ClustersStopSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterStop.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterStop.json */ /** * Sample code: Stop all servers in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void stopAllServersInTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersUpdateSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersUpdateSamples.java index c55381997b578..bc445689f8eec 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersUpdateSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersUpdateSamples.java @@ -7,109 +7,95 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.Cluster; import com.azure.resourcemanager.cosmosdbforpostgresql.models.MaintenanceWindow; -/** Samples for Clusters Update. */ +/** + * Samples for Clusters Update. + */ public final class ClustersUpdateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterScaleStorage.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterScaleStorage.json */ /** * Sample code: Scale up storage. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void scaleUpStorage( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); + public static void + scaleUpStorage(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); resource.update().withNodeStorageQuotaInMb(2097152).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterUpdate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterUpdate.json */ /** * Sample code: Update multiple configuration settings of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateMultipleConfigurationSettingsOfTheCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withAdministratorLoginPassword("newpassword") - .withCoordinatorVCores(16) - .withNodeCount(4) - .withNodeVCores(16) - .apply(); + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); + resource.update().withAdministratorLoginPassword("newpassword").withCoordinatorVCores(16).withNodeCount(4) + .withNodeVCores(16).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterScaleCompute.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterScaleCompute.json */ /** * Sample code: Scale compute up or down. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void scaleComputeUpOrDown( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); + public static void + scaleComputeUpOrDown(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); resource.update().withNodeVCores(16).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterAddNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterAddNode.json */ /** * Sample code: Scale out: Add new worker nodes. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void scaleOutAddNewWorkerNodes( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); resource.update().withNodeCount(2).apply(); } /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ClusterUpdateMaintenanceWindow.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ClusterUpdateMaintenanceWindow.json */ /** * Sample code: Update or define maintenance window. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateOrDefineMaintenanceWindow( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - Cluster resource = - manager - .clusters() - .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withMaintenanceWindow( - new MaintenanceWindow() - .withCustomWindow("Enabled") - .withStartHour(8) - .withStartMinute(0) - .withDayOfWeek(0)) + Cluster resource = manager.clusters() + .getByResourceGroupWithResponse("TestGroup", "testcluster", com.azure.core.util.Context.NONE).getValue(); + resource.update().withMaintenanceWindow( + new MaintenanceWindow().withCustomWindow("Enabled").withStartHour(8).withStartMinute(0).withDayOfWeek(0)) .apply(); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorSamples.java index ba1ab4fe63d9b..772d0dec8b171 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorSamples.java @@ -4,21 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Configurations GetCoordinator. */ +/** + * Samples for Configurations GetCoordinator. + */ public final class ConfigurationsGetCoordinatorSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationGetCoordinator.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationGetCoordinator.json */ /** * Sample code: Get configuration details for coordinator. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getConfigurationDetailsForCoordinator( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .getCoordinatorWithResponse( - "TestResourceGroup", "testcluster", "array_nulls", com.azure.core.util.Context.NONE); + manager.configurations().getCoordinatorWithResponse("TestResourceGroup", "testcluster", "array_nulls", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeSamples.java index 86336d96c196e..442ac79d35c37 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Configurations GetNode. */ +/** + * Samples for Configurations GetNode. + */ public final class ConfigurationsGetNodeSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationGetNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationGetNode.json */ /** * Sample code: Get configuration details for node. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getConfigurationDetailsForNode( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .getNodeWithResponse("TestResourceGroup", "testcluster", "array_nulls", com.azure.core.util.Context.NONE); + manager.configurations().getNodeWithResponse("TestResourceGroup", "testcluster", "array_nulls", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetSamples.java index dbf8bea6c7fc5..9b6656cf893d8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Configurations Get. */ +/** + * Samples for Configurations Get. + */ public final class ConfigurationsGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationGet.json */ /** * Sample code: Get configuration details. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getConfigurationDetails( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .getWithResponse("TestResourceGroup", "testcluster", "client_encoding", com.azure.core.util.Context.NONE); + public static void + getConfigurationDetails(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.configurations().getWithResponse("TestResourceGroup", "testcluster", "client_encoding", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterSamples.java index 8d1fbbc7dac37..a9d6388e421b2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Configurations ListByCluster. */ +/** + * Samples for Configurations ListByCluster. + */ public final class ConfigurationsListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationListByCluster.json */ /** * Sample code: List configurations of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listConfigurationsOfTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerSamples.java index f71c75bab2fe5..1b3fa8ac2c1c0 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Configurations ListByServer. */ +/** + * Samples for Configurations ListByServer. + */ public final class ConfigurationsListByServerSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationListByServer.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationListByServer.json */ /** * Sample code: List configurations of the server that in the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listConfigurationsOfTheServerThatInTheCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .listByServer("TestResourceGroup", "testcluster", "testserver", com.azure.core.util.Context.NONE); + manager.configurations().listByServer("TestResourceGroup", "testcluster", "testserver", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorSamples.java index e03eda246539e..31a419f5c4b8b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorSamples.java @@ -6,25 +6,23 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** Samples for Configurations UpdateOnCoordinator. */ +/** + * Samples for Configurations UpdateOnCoordinator. + */ public final class ConfigurationsUpdateOnCoordinatorSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationUpdateCoordinator.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationUpdateCoordinator.json */ /** * Sample code: Update single configuration of coordinator. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateSingleConfigurationOfCoordinator( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .updateOnCoordinator( - "TestResourceGroup", - "testcluster", - "array_nulls", - new ServerConfigurationInner().withValue("on"), - com.azure.core.util.Context.NONE); + manager.configurations().updateOnCoordinator("TestResourceGroup", "testcluster", "array_nulls", + new ServerConfigurationInner().withValue("on"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeSamples.java index 52e231989de86..b29c5bee82b6b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeSamples.java @@ -6,25 +6,23 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.ServerConfigurationInner; -/** Samples for Configurations UpdateOnNode. */ +/** + * Samples for Configurations UpdateOnNode. + */ public final class ConfigurationsUpdateOnNodeSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ConfigurationUpdateNode.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ConfigurationUpdateNode.json */ /** * Sample code: Update single configuration of nodes. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void updateSingleConfigurationOfNodes( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .configurations() - .updateOnNode( - "TestResourceGroup", - "testcluster", - "array_nulls", - new ServerConfigurationInner().withValue("off"), - com.azure.core.util.Context.NONE); + manager.configurations().updateOnNode("TestResourceGroup", "testcluster", "array_nulls", + new ServerConfigurationInner().withValue("off"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateSamples.java index 814b438541cdd..a36439fa8c937 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateSamples.java @@ -4,24 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for FirewallRules CreateOrUpdate. */ +/** + * Samples for FirewallRules CreateOrUpdate. + */ public final class FirewallRulesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleCreate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleCreate.json */ /** * Sample code: Create a firewall rule of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void createAFirewallRuleOfTheCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .firewallRules() - .define("rule1") - .withExistingServerGroupsv2("TestGroup", "pgtestsvc4") - .withStartIpAddress("0.0.0.0") - .withEndIpAddress("255.255.255.255") - .create(); + manager.firewallRules().define("rule1").withExistingServerGroupsv2("TestGroup", "pgtestsvc4") + .withStartIpAddress("0.0.0.0").withEndIpAddress("255.255.255.255").create(); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesDeleteSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesDeleteSamples.java index de18999f9cf1e..641df7f629a71 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesDeleteSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for FirewallRules Delete. */ +/** + * Samples for FirewallRules Delete. + */ public final class FirewallRulesDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleDelete.json */ /** * Sample code: Delete the firewall rule of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void deleteTheFirewallRuleOfTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetSamples.java index bbe1bf5ae6580..39136341af7d7 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for FirewallRules Get. */ +/** + * Samples for FirewallRules Get. + */ public final class FirewallRulesGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleGet.json */ /** * Sample code: Get the firewall rule of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getTheFirewallRuleOfTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterSamples.java index 752f2e19d89ce..2940fb272ffb6 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for FirewallRules ListByCluster. */ +/** + * Samples for FirewallRules ListByCluster. + */ public final class FirewallRulesListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/FirewallRuleListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * FirewallRuleListByCluster.json */ /** * Sample code: List firewall rules of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listFirewallRulesOfTheCluster( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListSamples.java index 290714e84d0a5..087e2244470db 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/OperationList.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * OperationList.json */ /** * Sample code: List all available operations. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void listAllAvailableOperations( diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateSamples.java index f58ab7aefd6ab..abf21499f25a8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateSamples.java @@ -7,25 +7,26 @@ import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateEndpointServiceConnectionStatus; import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrivateLinkServiceConnectionState; -/** Samples for PrivateEndpointConnections CreateOrUpdate. */ +/** + * Samples for PrivateEndpointConnections CreateOrUpdate. + */ public final class PrivateEndpointConnectionsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionCreateOrUpdate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionCreateOrUpdate.json */ /** * Sample code: Approves or Rejects a Private Endpoint Connection with a given name. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void approvesOrRejectsAPrivateEndpointConnectionWithAGivenName( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .define("private-endpoint-connection-name") + manager.privateEndpointConnections().define("private-endpoint-connection-name") .withExistingServerGroupsv2("TestGroup", "testcluster") .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) .withDescription("Approved by johndoe@contoso.com")) .create(); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsDeleteSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsDeleteSamples.java index 1c7c0590f151a..f65f5da0627f2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsDeleteSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsDeleteSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for PrivateEndpointConnections Delete. */ +/** + * Samples for PrivateEndpointConnections Delete. + */ public final class PrivateEndpointConnectionsDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionsDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionsDelete.json */ /** * Sample code: Deletes a private endpoint connection with a given name. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void deletesAPrivateEndpointConnectionWithAGivenName( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .delete("TestGroup", "testcluster", "private-endpoint-connection-name", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().delete("TestGroup", "testcluster", "private-endpoint-connection-name", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetSamples.java index 3fa6d20b78d6c..1f79c3d96e7fe 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetSamples.java @@ -4,21 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for PrivateEndpointConnections Get. */ +/** + * Samples for PrivateEndpointConnections Get. + */ public final class PrivateEndpointConnectionsGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionsGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionsGet.json */ /** * Sample code: Gets private endpoint connection. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsPrivateEndpointConnection( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .getWithResponse( - "TestGroup", "testcluster", "private-endpoint-connection-name", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().getWithResponse("TestGroup", "testcluster", + "private-endpoint-connection-name", com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterSamples.java index 2cc0b53c8b69e..bb2fcb8b06979 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for PrivateEndpointConnections ListByCluster. */ +/** + * Samples for PrivateEndpointConnections ListByCluster. + */ public final class PrivateEndpointConnectionsListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateEndpointConnectionsListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateEndpointConnectionsListByCluster.json */ /** * Sample code: Gets list of private endpoint connections on a cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsListOfPrivateEndpointConnectionsOnACluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateEndpointConnections() - .listByCluster("TestResourceGroup", "testcluster", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().listByCluster("TestResourceGroup", "testcluster", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetSamples.java index fbccae135930e..9a20d5dfdddc5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for PrivateLinkResources Get. */ +/** + * Samples for PrivateLinkResources Get. + */ public final class PrivateLinkResourcesGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateLinkResourcesGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateLinkResourcesGet.json */ /** * Sample code: Gets a private link resource for cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsAPrivateLinkResourceForCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateLinkResources() - .getWithResponse("TestGroup", "testcluster", "plr", com.azure.core.util.Context.NONE); + manager.privateLinkResources().getWithResponse("TestGroup", "testcluster", "plr", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterSamples.java index 5567257b0d7f8..8a9e5fc865103 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for PrivateLinkResources ListByCluster. */ +/** + * Samples for PrivateLinkResources ListByCluster. + */ public final class PrivateLinkResourcesListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/PrivateLinkResourceListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * PrivateLinkResourceListByCluster.json */ /** * Sample code: Gets the private link resources for cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void getsThePrivateLinkResourcesForCluster( com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .privateLinkResources() - .listByCluster("TestResourceGroup", "testcluster", com.azure.core.util.Context.NONE); + manager.privateLinkResources().listByCluster("TestResourceGroup", "testcluster", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesCreateSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesCreateSamples.java index 5ca6f5b4b7649..b205968bc9531 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesCreateSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesCreateSamples.java @@ -4,23 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Roles Create. */ +/** + * Samples for Roles Create. + */ public final class RolesCreateSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleCreate.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleCreate.json */ /** * Sample code: RoleCreate. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void roleCreate( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .roles() - .define("role1") - .withExistingServerGroupsv2("TestGroup", "pgtestsvc4") - .withPassword("password") + public static void + roleCreate(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.roles().define("role1").withExistingServerGroupsv2("TestGroup", "pgtestsvc4").withPassword("password") .create(); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesDeleteSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesDeleteSamples.java index 1f9004e2df566..6e17fd06abba5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesDeleteSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesDeleteSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Roles Delete. */ +/** + * Samples for Roles Delete. + */ public final class RolesDeleteSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleDelete.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleDelete.json */ /** * Sample code: RoleDelete. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void roleDelete( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + roleDelete(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.roles().delete("TestGroup", "pgtestsvc4", "role1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesGetSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesGetSamples.java index 1202321071102..ef1a036c1a020 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesGetSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesGetSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Roles Get. */ +/** + * Samples for Roles Get. + */ public final class RolesGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleGet.json */ /** * Sample code: Get the role of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getTheRoleOfTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + getTheRoleOfTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.roles().getWithResponse("TestGroup", "pgtestsvc4", "role1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesListByClusterSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesListByClusterSamples.java index 3366ca569b9ec..ce93876719b13 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesListByClusterSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolesListByClusterSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Roles ListByCluster. */ +/** + * Samples for Roles ListByCluster. + */ public final class RolesListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/RoleListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * RoleListByCluster.json */ /** * Sample code: RoleList. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ public static void roleList(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetSamples.java index 788ed7f5f9717..48190e6d6acf5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Servers Get. */ +/** + * Samples for Servers Get. + */ public final class ServersGetSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ServerGet.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ServerGet.json */ /** * Sample code: Get the server of cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void getTheServerOfCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { - manager - .servers() - .getWithResponse("TestGroup", "testcluster1", "testcluster1-c", com.azure.core.util.Context.NONE); + public static void + getTheServerOfCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + manager.servers().getWithResponse("TestGroup", "testcluster1", "testcluster1-c", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterSamples.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterSamples.java index da1bb8715d882..c24d39c013d77 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterSamples.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/samples/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.cosmosdbforpostgresql.generated; -/** Samples for Servers ListByCluster. */ +/** + * Samples for Servers ListByCluster. + */ public final class ServersListByClusterSamples { /* - * x-ms-original-file: specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/stable/2022-11-08/examples/ServerListByCluster.json + * x-ms-original-file: + * specification/postgresqlhsc/resource-manager/Microsoft.DBforPostgreSQL/preview/2023-03-02-preview/examples/ + * ServerListByCluster.json */ /** * Sample code: List servers of the cluster. - * + * * @param manager Entry point to CosmosDBForPostgreSqlManager. */ - public static void listServersOfTheCluster( - com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { + public static void + listServersOfTheCluster(com.azure.resourcemanager.cosmosdbforpostgresql.CosmosDBForPostgreSqlManager manager) { manager.servers().listByCluster("TestGroup", "testcluster1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterConfigurationListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterConfigurationListResultTests.java index a455268a181b8..1d8e7a5a2d1ca 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterConfigurationListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterConfigurationListResultTests.java @@ -15,36 +15,44 @@ public final class ClusterConfigurationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ClusterConfigurationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"description\":\"tnwu\",\"dataType\":\"Numeric\",\"allowedValues\":\"zxufiz\",\"requiresRestart\":false,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"i\",\"defaultValue\":\"fidfvzw\",\"source\":\"uht\"}],\"provisioningState\":\"Canceled\"},\"id\":\"sdkf\",\"name\":\"hwxmnteiwa\",\"type\":\"pvkmijcmmxdcuf\"}],\"nextLink\":\"srp\"}") - .toObject(ClusterConfigurationListResult.class); - Assertions.assertEquals(false, model.value().get(0).requiresRestart()); - Assertions - .assertEquals(ServerRole.COORDINATOR, model.value().get(0).serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("i", model.value().get(0).serverRoleGroupConfigurations().get(0).value()); + ClusterConfigurationListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"description\":\"maajrmvdjwzrlo\",\"dataType\":\"Enumeration\",\"allowedValues\":\"whijcoejctbza\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"cbkbfkg\",\"defaultValue\":\"dkexxppofm\",\"source\":\"x\"},{\"role\":\"Worker\",\"value\":\"jpgd\",\"defaultValue\":\"ocjjxhvpmouexh\",\"source\":\"xibqeojnx\"}],\"provisioningState\":\"Failed\"},\"id\":\"ddntwndei\",\"name\":\"btwnpzaoqvuhrhcf\",\"type\":\"cyddglmjthjqk\"},{\"properties\":{\"description\":\"eicxmqciwqvhkhi\",\"dataType\":\"Integer\",\"allowedValues\":\"dtopbob\",\"requiresRestart\":false,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"e\",\"defaultValue\":\"a\",\"source\":\"uhrzayvvt\"}],\"provisioningState\":\"Canceled\"},\"id\":\"f\",\"name\":\"iotkftutqxl\",\"type\":\"gxlefgugnxkrxd\"},{\"properties\":{\"description\":\"dt\",\"dataType\":\"Numeric\",\"allowedValues\":\"vqdra\",\"requiresRestart\":false,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"igeho\",\"defaultValue\":\"bowsk\",\"source\":\"yktz\"}],\"provisioningState\":\"Succeeded\"},\"id\":\"y\",\"name\":\"gqywgndrv\",\"type\":\"nhzgpphrcgyn\"},{\"properties\":{\"description\":\"pec\",\"dataType\":\"Boolean\",\"allowedValues\":\"coofsxlzev\",\"requiresRestart\":false,\"serverRoleGroupConfigurations\":[{\"role\":\"Worker\",\"value\":\"qabcypm\",\"defaultValue\":\"kwlzuvccfwnfn\",\"source\":\"cfionl\"},{\"role\":\"Worker\",\"value\":\"x\",\"defaultValue\":\"qgtz\",\"source\":\"pnqbqqwxrjfe\"},{\"role\":\"Coordinator\",\"value\":\"lnwsubisn\",\"defaultValue\":\"mpmngnzscxaqwoo\",\"source\":\"cbonqvpk\"},{\"role\":\"Worker\",\"value\":\"rxnjeaseipheofl\",\"defaultValue\":\"eyy\",\"source\":\"nj\"}],\"provisioningState\":\"InProgress\"},\"id\":\"tgrhpdjpjumas\",\"name\":\"azjpqyegualhbxxh\",\"type\":\"jj\"}],\"nextLink\":\"v\"}") + .toObject(ClusterConfigurationListResult.class); + Assertions.assertEquals(true, model.value().get(0).requiresRestart()); + Assertions.assertEquals(ServerRole.COORDINATOR, + model.value().get(0).serverRoleGroupConfigurations().get(0).role()); + Assertions.assertEquals("cbkbfkg", model.value().get(0).serverRoleGroupConfigurations().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ClusterConfigurationListResult model = - new ClusterConfigurationListResult() + ClusterConfigurationListResult model + = new ClusterConfigurationListResult() .withValue( - Arrays - .asList( - new ConfigurationInner() - .withRequiresRestart(false) - .withServerRoleGroupConfigurations( - Arrays - .asList( - new ServerRoleGroupConfiguration() - .withRole(ServerRole.COORDINATOR) - .withValue("i"))))); + Arrays.asList( + new ConfigurationInner().withRequiresRestart(true) + .withServerRoleGroupConfigurations(Arrays.asList( + new ServerRoleGroupConfiguration().withRole(ServerRole.COORDINATOR) + .withValue("cbkbfkg"), + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("jpgd"))), + new ConfigurationInner().withRequiresRestart(false) + .withServerRoleGroupConfigurations(Arrays.asList( + new ServerRoleGroupConfiguration().withRole(ServerRole.COORDINATOR).withValue("e"))), + new ConfigurationInner().withRequiresRestart(false) + .withServerRoleGroupConfigurations(Arrays.asList(new ServerRoleGroupConfiguration() + .withRole(ServerRole.COORDINATOR).withValue("igeho"))), + new ConfigurationInner().withRequiresRestart(false) + .withServerRoleGroupConfigurations(Arrays.asList( + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("qabcypm"), + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("x"), + new ServerRoleGroupConfiguration().withRole(ServerRole.COORDINATOR) + .withValue("lnwsubisn"), + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER) + .withValue("rxnjeaseipheofl"))))); model = BinaryData.fromObject(model).toObject(ClusterConfigurationListResult.class); - Assertions.assertEquals(false, model.value().get(0).requiresRestart()); - Assertions - .assertEquals(ServerRole.COORDINATOR, model.value().get(0).serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("i", model.value().get(0).serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals(true, model.value().get(0).requiresRestart()); + Assertions.assertEquals(ServerRole.COORDINATOR, + model.value().get(0).serverRoleGroupConfigurations().get(0).role()); + Assertions.assertEquals("cbkbfkg", model.value().get(0).serverRoleGroupConfigurations().get(0).value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerInnerTests.java index b0d4e7d2bea3e..ad1f1b07c0f46 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerInnerTests.java @@ -12,41 +12,33 @@ public final class ClusterServerInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ClusterServerInner model = - BinaryData - .fromString( - "{\"properties\":{\"fullyQualifiedDomainName\":\"atscmd\",\"role\":\"Worker\",\"state\":\"u\",\"haState\":\"uuvmkjozkrwfnd\",\"availabilityZone\":\"djpslw\",\"postgresqlVersion\":\"dpvwryoqpsoaccta\",\"citusVersion\":\"kljla\",\"serverEdition\":\"cr\",\"storageQuotaInMb\":1060950423,\"vCores\":1389039399,\"enableHa\":false,\"enablePublicIpAccess\":true,\"isReadOnly\":false,\"administratorLogin\":\"paojakhmsbzjh\"},\"id\":\"zevdphlx\",\"name\":\"olthqtrgqjbp\",\"type\":\"zfsinzgvf\"}") - .toObject(ClusterServerInner.class); - Assertions.assertEquals(ServerRole.WORKER, model.role()); - Assertions.assertEquals("djpslw", model.availabilityZone()); - Assertions.assertEquals("dpvwryoqpsoaccta", model.postgresqlVersion()); - Assertions.assertEquals("kljla", model.citusVersion()); - Assertions.assertEquals("cr", model.serverEdition()); - Assertions.assertEquals(1060950423, model.storageQuotaInMb()); - Assertions.assertEquals(1389039399, model.vCores()); - Assertions.assertEquals(false, model.enableHa()); + ClusterServerInner model = BinaryData.fromString( + "{\"properties\":{\"fullyQualifiedDomainName\":\"ksqrglssai\",\"role\":\"Coordinator\",\"state\":\"wnzlljfmppeeb\",\"haState\":\"gxsabkyq\",\"availabilityZone\":\"ujitcjcz\",\"postgresqlVersion\":\"evndh\",\"citusVersion\":\"wpdappdsbdkv\",\"serverEdition\":\"wjfeusnhutjel\",\"storageQuotaInMb\":391471833,\"vCores\":1969626150,\"enableHa\":true,\"enablePublicIpAccess\":false,\"isReadOnly\":true,\"administratorLogin\":\"atqxho\"},\"id\":\"geablgphuticndvk\",\"name\":\"ozwyiftyhxhuro\",\"type\":\"ftyxolniw\"}") + .toObject(ClusterServerInner.class); + Assertions.assertEquals(ServerRole.COORDINATOR, model.role()); + Assertions.assertEquals("ujitcjcz", model.availabilityZone()); + Assertions.assertEquals("evndh", model.postgresqlVersion()); + Assertions.assertEquals("wpdappdsbdkv", model.citusVersion()); + Assertions.assertEquals("wjfeusnhutjel", model.serverEdition()); + Assertions.assertEquals(391471833, model.storageQuotaInMb()); + Assertions.assertEquals(1969626150, model.vCores()); + Assertions.assertEquals(true, model.enableHa()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ClusterServerInner model = - new ClusterServerInner() - .withRole(ServerRole.WORKER) - .withAvailabilityZone("djpslw") - .withPostgresqlVersion("dpvwryoqpsoaccta") - .withCitusVersion("kljla") - .withServerEdition("cr") - .withStorageQuotaInMb(1060950423) - .withVCores(1389039399) - .withEnableHa(false); + ClusterServerInner model + = new ClusterServerInner().withRole(ServerRole.COORDINATOR).withAvailabilityZone("ujitcjcz") + .withPostgresqlVersion("evndh").withCitusVersion("wpdappdsbdkv").withServerEdition("wjfeusnhutjel") + .withStorageQuotaInMb(391471833).withVCores(1969626150).withEnableHa(true); model = BinaryData.fromObject(model).toObject(ClusterServerInner.class); - Assertions.assertEquals(ServerRole.WORKER, model.role()); - Assertions.assertEquals("djpslw", model.availabilityZone()); - Assertions.assertEquals("dpvwryoqpsoaccta", model.postgresqlVersion()); - Assertions.assertEquals("kljla", model.citusVersion()); - Assertions.assertEquals("cr", model.serverEdition()); - Assertions.assertEquals(1060950423, model.storageQuotaInMb()); - Assertions.assertEquals(1389039399, model.vCores()); - Assertions.assertEquals(false, model.enableHa()); + Assertions.assertEquals(ServerRole.COORDINATOR, model.role()); + Assertions.assertEquals("ujitcjcz", model.availabilityZone()); + Assertions.assertEquals("evndh", model.postgresqlVersion()); + Assertions.assertEquals("wpdappdsbdkv", model.citusVersion()); + Assertions.assertEquals("wjfeusnhutjel", model.serverEdition()); + Assertions.assertEquals(391471833, model.storageQuotaInMb()); + Assertions.assertEquals(1969626150, model.vCores()); + Assertions.assertEquals(true, model.enableHa()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerListResultTests.java index 0d44345015e16..7776fd1fe7a5c 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerListResultTests.java @@ -14,54 +14,42 @@ public final class ClusterServerListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ClusterServerListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"fullyQualifiedDomainName\":\"mt\",\"role\":\"Worker\",\"state\":\"ryrtihfxtijbpzv\",\"haState\":\"wzsymglzufcy\",\"availabilityZone\":\"ohdbihanufh\",\"postgresqlVersion\":\"bj\",\"citusVersion\":\"a\",\"serverEdition\":\"th\",\"storageQuotaInMb\":731883185,\"vCores\":676674728,\"enableHa\":true,\"enablePublicIpAccess\":false,\"isReadOnly\":true,\"administratorLogin\":\"czbysc\"},\"id\":\"q\",\"name\":\"uhivyqniw\",\"type\":\"ybrk\"},{\"properties\":{\"fullyQualifiedDomainName\":\"umjgrtfwvuk\",\"role\":\"Worker\",\"state\":\"dcc\",\"haState\":\"h\",\"availabilityZone\":\"cnyejhkryhtnapcz\",\"postgresqlVersion\":\"okjye\",\"citusVersion\":\"kvnipjoxz\",\"serverEdition\":\"chgejspodm\",\"storageQuotaInMb\":1543997460,\"vCores\":1345546955,\"enableHa\":false,\"enablePublicIpAccess\":false,\"isReadOnly\":true,\"administratorLogin\":\"ahuxinpm\"},\"id\":\"jaqwixjsp\",\"name\":\"ozvcput\",\"type\":\"gjvw\"}]}") - .toObject(ClusterServerListResult.class); - Assertions.assertEquals(ServerRole.WORKER, model.value().get(0).role()); - Assertions.assertEquals("ohdbihanufh", model.value().get(0).availabilityZone()); - Assertions.assertEquals("bj", model.value().get(0).postgresqlVersion()); - Assertions.assertEquals("a", model.value().get(0).citusVersion()); - Assertions.assertEquals("th", model.value().get(0).serverEdition()); - Assertions.assertEquals(731883185, model.value().get(0).storageQuotaInMb()); - Assertions.assertEquals(676674728, model.value().get(0).vCores()); - Assertions.assertEquals(true, model.value().get(0).enableHa()); + ClusterServerListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"fullyQualifiedDomainName\":\"pzvgnwzsymglzufc\",\"role\":\"Coordinator\",\"state\":\"hdbihan\",\"haState\":\"hfcbjysa\",\"availabilityZone\":\"th\",\"postgresqlVersion\":\"hab\",\"citusVersion\":\"pikxwczbyscnpqxu\",\"serverEdition\":\"vyq\",\"storageQuotaInMb\":649847293,\"vCores\":992034682,\"enableHa\":false,\"enablePublicIpAccess\":false,\"isReadOnly\":true,\"administratorLogin\":\"mjgr\"},\"id\":\"wvukx\",\"name\":\"audccsnhs\",\"type\":\"cnyejhkryhtnapcz\"},{\"properties\":{\"fullyQualifiedDomainName\":\"kjyemkk\",\"role\":\"Worker\",\"state\":\"joxzjnchgejspodm\",\"haState\":\"lzydehojwyahux\",\"availabilityZone\":\"pmqnja\",\"postgresqlVersion\":\"ixjsprozvcputeg\",\"citusVersion\":\"wmfdatscmdvpjhul\",\"serverEdition\":\"uvm\",\"storageQuotaInMb\":2017964411,\"vCores\":37430274,\"enableHa\":true,\"enablePublicIpAccess\":true,\"isReadOnly\":false,\"administratorLogin\":\"djpslw\"},\"id\":\"dpvwryoqpsoaccta\",\"name\":\"akl\",\"type\":\"lahbcryff\"},{\"properties\":{\"fullyQualifiedDomainName\":\"osygex\",\"role\":\"Worker\",\"state\":\"akhmsbzjhcrz\",\"haState\":\"dphlxaolt\",\"availabilityZone\":\"trg\",\"postgresqlVersion\":\"bpf\",\"citusVersion\":\"s\",\"serverEdition\":\"zgvfcjrwz\",\"storageQuotaInMb\":1151751299,\"vCores\":1189950130,\"enableHa\":true,\"enablePublicIpAccess\":true,\"isReadOnly\":true,\"administratorLogin\":\"zitonpeqfpjkjl\"},\"id\":\"fpdvhpfxxypi\",\"name\":\"i\",\"type\":\"mayhuybbkpodepoo\"},{\"properties\":{\"fullyQualifiedDomainName\":\"uvamiheognarxzxt\",\"role\":\"Coordinator\",\"state\":\"usivye\",\"haState\":\"ciqihnhung\",\"availabilityZone\":\"jzrnf\",\"postgresqlVersion\":\"xgispemvtzfkufu\",\"citusVersion\":\"jofxqe\",\"serverEdition\":\"jaeq\",\"storageQuotaInMb\":1946289710,\"vCores\":158340850,\"enableHa\":false,\"enablePublicIpAccess\":true,\"isReadOnly\":true,\"administratorLogin\":\"qulngsntnbybkzgc\"},\"id\":\"wclxxwrl\",\"name\":\"douskcqvkocrcjdk\",\"type\":\"tnhxbn\"}]}") + .toObject(ClusterServerListResult.class); + Assertions.assertEquals(ServerRole.COORDINATOR, model.value().get(0).role()); + Assertions.assertEquals("th", model.value().get(0).availabilityZone()); + Assertions.assertEquals("hab", model.value().get(0).postgresqlVersion()); + Assertions.assertEquals("pikxwczbyscnpqxu", model.value().get(0).citusVersion()); + Assertions.assertEquals("vyq", model.value().get(0).serverEdition()); + Assertions.assertEquals(649847293, model.value().get(0).storageQuotaInMb()); + Assertions.assertEquals(992034682, model.value().get(0).vCores()); + Assertions.assertEquals(false, model.value().get(0).enableHa()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ClusterServerListResult model = - new ClusterServerListResult() - .withValue( - Arrays - .asList( - new ClusterServerInner() - .withRole(ServerRole.WORKER) - .withAvailabilityZone("ohdbihanufh") - .withPostgresqlVersion("bj") - .withCitusVersion("a") - .withServerEdition("th") - .withStorageQuotaInMb(731883185) - .withVCores(676674728) - .withEnableHa(true), - new ClusterServerInner() - .withRole(ServerRole.WORKER) - .withAvailabilityZone("cnyejhkryhtnapcz") - .withPostgresqlVersion("okjye") - .withCitusVersion("kvnipjoxz") - .withServerEdition("chgejspodm") - .withStorageQuotaInMb(1543997460) - .withVCores(1345546955) - .withEnableHa(false))); + ClusterServerListResult model = new ClusterServerListResult().withValue(Arrays.asList( + new ClusterServerInner().withRole(ServerRole.COORDINATOR).withAvailabilityZone("th") + .withPostgresqlVersion("hab").withCitusVersion("pikxwczbyscnpqxu").withServerEdition("vyq") + .withStorageQuotaInMb(649847293).withVCores(992034682).withEnableHa(false), + new ClusterServerInner().withRole(ServerRole.WORKER).withAvailabilityZone("pmqnja") + .withPostgresqlVersion("ixjsprozvcputeg").withCitusVersion("wmfdatscmdvpjhul").withServerEdition("uvm") + .withStorageQuotaInMb(2017964411).withVCores(37430274).withEnableHa(true), + new ClusterServerInner().withRole(ServerRole.WORKER).withAvailabilityZone("trg") + .withPostgresqlVersion("bpf").withCitusVersion("s").withServerEdition("zgvfcjrwz") + .withStorageQuotaInMb(1151751299).withVCores(1189950130).withEnableHa(true), + new ClusterServerInner().withRole(ServerRole.COORDINATOR).withAvailabilityZone("jzrnf") + .withPostgresqlVersion("xgispemvtzfkufu").withCitusVersion("jofxqe").withServerEdition("jaeq") + .withStorageQuotaInMb(1946289710).withVCores(158340850).withEnableHa(false))); model = BinaryData.fromObject(model).toObject(ClusterServerListResult.class); - Assertions.assertEquals(ServerRole.WORKER, model.value().get(0).role()); - Assertions.assertEquals("ohdbihanufh", model.value().get(0).availabilityZone()); - Assertions.assertEquals("bj", model.value().get(0).postgresqlVersion()); - Assertions.assertEquals("a", model.value().get(0).citusVersion()); - Assertions.assertEquals("th", model.value().get(0).serverEdition()); - Assertions.assertEquals(731883185, model.value().get(0).storageQuotaInMb()); - Assertions.assertEquals(676674728, model.value().get(0).vCores()); - Assertions.assertEquals(true, model.value().get(0).enableHa()); + Assertions.assertEquals(ServerRole.COORDINATOR, model.value().get(0).role()); + Assertions.assertEquals("th", model.value().get(0).availabilityZone()); + Assertions.assertEquals("hab", model.value().get(0).postgresqlVersion()); + Assertions.assertEquals("pikxwczbyscnpqxu", model.value().get(0).citusVersion()); + Assertions.assertEquals("vyq", model.value().get(0).serverEdition()); + Assertions.assertEquals(649847293, model.value().get(0).storageQuotaInMb()); + Assertions.assertEquals(992034682, model.value().get(0).vCores()); + Assertions.assertEquals(false, model.value().get(0).enableHa()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerPropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerPropertiesTests.java index b8065e9f2e338..f1f0250ddf8a5 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerPropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClusterServerPropertiesTests.java @@ -12,41 +12,33 @@ public final class ClusterServerPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ClusterServerProperties model = - BinaryData - .fromString( - "{\"fullyQualifiedDomainName\":\"rwzoxxjtfelluwf\",\"role\":\"Coordinator\",\"state\":\"np\",\"haState\":\"fpjkjlxofp\",\"availabilityZone\":\"hpf\",\"postgresqlVersion\":\"ypininm\",\"citusVersion\":\"huyb\",\"serverEdition\":\"podepoo\",\"storageQuotaInMb\":895630833,\"vCores\":1290742637,\"enableHa\":false,\"enablePublicIpAccess\":true,\"isReadOnly\":true,\"administratorLogin\":\"narxzxtheotus\"}") - .toObject(ClusterServerProperties.class); - Assertions.assertEquals("podepoo", model.serverEdition()); - Assertions.assertEquals(895630833, model.storageQuotaInMb()); - Assertions.assertEquals(1290742637, model.vCores()); + ClusterServerProperties model = BinaryData.fromString( + "{\"fullyQualifiedDomainName\":\"cukjf\",\"role\":\"Coordinator\",\"state\":\"w\",\"haState\":\"lryplwckbasyy\",\"availabilityZone\":\"ddhsgcbacphe\",\"postgresqlVersion\":\"ot\",\"citusVersion\":\"qgoulznd\",\"serverEdition\":\"kwy\",\"storageQuotaInMb\":1581994668,\"vCores\":1308251370,\"enableHa\":false,\"enablePublicIpAccess\":true,\"isReadOnly\":false,\"administratorLogin\":\"keqsrxybzqqedq\"}") + .toObject(ClusterServerProperties.class); + Assertions.assertEquals("kwy", model.serverEdition()); + Assertions.assertEquals(1581994668, model.storageQuotaInMb()); + Assertions.assertEquals(1308251370, model.vCores()); Assertions.assertEquals(false, model.enableHa()); Assertions.assertEquals(ServerRole.COORDINATOR, model.role()); - Assertions.assertEquals("hpf", model.availabilityZone()); - Assertions.assertEquals("ypininm", model.postgresqlVersion()); - Assertions.assertEquals("huyb", model.citusVersion()); + Assertions.assertEquals("ddhsgcbacphe", model.availabilityZone()); + Assertions.assertEquals("ot", model.postgresqlVersion()); + Assertions.assertEquals("qgoulznd", model.citusVersion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ClusterServerProperties model = - new ClusterServerProperties() - .withServerEdition("podepoo") - .withStorageQuotaInMb(895630833) - .withVCores(1290742637) - .withEnableHa(false) - .withRole(ServerRole.COORDINATOR) - .withAvailabilityZone("hpf") - .withPostgresqlVersion("ypininm") - .withCitusVersion("huyb"); + ClusterServerProperties model + = new ClusterServerProperties().withServerEdition("kwy").withStorageQuotaInMb(1581994668) + .withVCores(1308251370).withEnableHa(false).withRole(ServerRole.COORDINATOR) + .withAvailabilityZone("ddhsgcbacphe").withPostgresqlVersion("ot").withCitusVersion("qgoulznd"); model = BinaryData.fromObject(model).toObject(ClusterServerProperties.class); - Assertions.assertEquals("podepoo", model.serverEdition()); - Assertions.assertEquals(895630833, model.storageQuotaInMb()); - Assertions.assertEquals(1290742637, model.vCores()); + Assertions.assertEquals("kwy", model.serverEdition()); + Assertions.assertEquals(1581994668, model.storageQuotaInMb()); + Assertions.assertEquals(1308251370, model.vCores()); Assertions.assertEquals(false, model.enableHa()); Assertions.assertEquals(ServerRole.COORDINATOR, model.role()); - Assertions.assertEquals("hpf", model.availabilityZone()); - Assertions.assertEquals("ypininm", model.postgresqlVersion()); - Assertions.assertEquals("huyb", model.citusVersion()); + Assertions.assertEquals("ddhsgcbacphe", model.availabilityZone()); + Assertions.assertEquals("ot", model.postgresqlVersion()); + Assertions.assertEquals("qgoulznd", model.citusVersion()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilityWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilityWithResponseMockTests.java index b4fe74131b6bc..39b8f62da074d 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersCheckNameAvailabilityWithResponseMockTests.java @@ -31,45 +31,30 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"message\":\"notyfjfcnjbkcn\",\"nameAvailable\":true,\"name\":\"ttkphywpnvjtoqne\",\"type\":\"clfp\"}"; + String responseStr + = "{\"message\":\"cugicjoox\",\"nameAvailable\":true,\"name\":\"wpucwwfvovbv\",\"type\":\"uecivyhz\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - NameAvailability response = - manager - .clusters() - .checkNameAvailabilityWithResponse( - new NameAvailabilityRequest().withName("eil"), com.azure.core.util.Context.NONE) - .getValue(); + NameAvailability response = manager.clusters().checkNameAvailabilityWithResponse( + new NameAvailabilityRequest().withName("hashsfwxosow"), com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("notyfjfcnjbkcn", response.message()); + Assertions.assertEquals("cugicjoox", response.message()); Assertions.assertEquals(true, response.nameAvailable()); - Assertions.assertEquals("ttkphywpnvjtoqne", response.name()); - Assertions.assertEquals("clfp", response.type()); + Assertions.assertEquals("wpucwwfvovbv", response.name()); + Assertions.assertEquals("uecivyhz", response.type()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteMockTests.java index 128997b17809f..5d2c3f8960cb6 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.clusters().delete("fhvpesaps", "rdqmhjjdhtldwkyz", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.clusters().delete("vfadmws", "crgvxpvgom", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartMockTests.java index 1c17aab328263..8b3c03a9e5ab2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStartMockTests.java @@ -32,30 +32,20 @@ public void testStart() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.clusters().start("uutkncw", "cwsvlxotog", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.clusters().start("lf", "isgwbnbbeldawkz", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopMockTests.java index 9b2b9a1f4be9a..33da145ded6fe 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ClustersStopMockTests.java @@ -32,30 +32,20 @@ public void testStop() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.clusters().stop("wrupqsxvnmicykvc", "o", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.clusters().stop("ali", "urqhaka", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationInnerTests.java index 904ff4e664526..6d88c7cd8a251 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationInnerTests.java @@ -14,29 +14,22 @@ public final class ConfigurationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ConfigurationInner model = - BinaryData - .fromString( - "{\"properties\":{\"description\":\"idnsezcxtb\",\"dataType\":\"Numeric\",\"allowedValues\":\"yc\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Worker\",\"value\":\"mdwzjeiachboo\",\"defaultValue\":\"lnrosfqp\",\"source\":\"ehzzvypyqrim\"},{\"role\":\"Worker\",\"value\":\"npvswjdkirso\",\"defaultValue\":\"qxhcrmn\",\"source\":\"jtckwhdso\"}],\"provisioningState\":\"Succeeded\"},\"id\":\"i\",\"name\":\"jxsqwpgrjbz\",\"type\":\"orcjxvsnby\"}") - .toObject(ConfigurationInner.class); + ConfigurationInner model = BinaryData.fromString( + "{\"properties\":{\"description\":\"gwdslfhotwm\",\"dataType\":\"Integer\",\"allowedValues\":\"wlbjnpgacftade\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Worker\",\"value\":\"yfsoppu\",\"defaultValue\":\"esnzwde\",\"source\":\"avo\"},{\"role\":\"Worker\",\"value\":\"zdmohctbqvu\",\"defaultValue\":\"xdn\",\"source\":\"vo\"}],\"provisioningState\":\"InProgress\"},\"id\":\"jugwdkcglhsl\",\"name\":\"zj\",\"type\":\"yggdtjixh\"}") + .toObject(ConfigurationInner.class); Assertions.assertEquals(true, model.requiresRestart()); Assertions.assertEquals(ServerRole.WORKER, model.serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("mdwzjeiachboo", model.serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals("yfsoppu", model.serverRoleGroupConfigurations().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ConfigurationInner model = - new ConfigurationInner() - .withRequiresRestart(true) - .withServerRoleGroupConfigurations( - Arrays - .asList( - new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("mdwzjeiachboo"), - new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("npvswjdkirso"))); + ConfigurationInner model = new ConfigurationInner().withRequiresRestart(true).withServerRoleGroupConfigurations( + Arrays.asList(new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("yfsoppu"), + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("zdmohctbqvu"))); model = BinaryData.fromObject(model).toObject(ConfigurationInner.class); Assertions.assertEquals(true, model.requiresRestart()); Assertions.assertEquals(ServerRole.WORKER, model.serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("mdwzjeiachboo", model.serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals("yfsoppu", model.serverRoleGroupConfigurations().get(0).value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationPropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationPropertiesTests.java index 124f0efbb68c1..c13a14f6512e6 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationPropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationPropertiesTests.java @@ -14,32 +14,24 @@ public final class ConfigurationPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ConfigurationProperties model = - BinaryData - .fromString( - "{\"description\":\"abnmocpcyshu\",\"dataType\":\"Enumeration\",\"allowedValues\":\"bl\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Worker\",\"value\":\"toqcjmklja\",\"defaultValue\":\"qidtqajzyu\",\"source\":\"kudjkrlkhb\"},{\"role\":\"Worker\",\"value\":\"fepgzgq\",\"defaultValue\":\"zloc\",\"source\":\"c\"},{\"role\":\"Coordinator\",\"value\":\"ierhhbcsglummaj\",\"defaultValue\":\"aodxo\",\"source\":\"bdxkqpxokaj\"}],\"provisioningState\":\"Canceled\"}") - .toObject(ConfigurationProperties.class); + ConfigurationProperties model = BinaryData.fromString( + "{\"description\":\"uofqwe\",\"dataType\":\"Enumeration\",\"allowedValues\":\"enevfyexfwhybci\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"c\",\"defaultValue\":\"tynnaamdectehfi\",\"source\":\"cj\"},{\"role\":\"Worker\",\"value\":\"pvhez\",\"defaultValue\":\"gqhcjrefovg\",\"source\":\"qsl\"},{\"role\":\"Worker\",\"value\":\"yvxyqjp\",\"defaultValue\":\"attpngjcrcczsq\",\"source\":\"hvmdajvnysounq\"}],\"provisioningState\":\"Succeeded\"}") + .toObject(ConfigurationProperties.class); Assertions.assertEquals(true, model.requiresRestart()); - Assertions.assertEquals(ServerRole.WORKER, model.serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("toqcjmklja", model.serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals(ServerRole.COORDINATOR, model.serverRoleGroupConfigurations().get(0).role()); + Assertions.assertEquals("c", model.serverRoleGroupConfigurations().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ConfigurationProperties model = - new ConfigurationProperties() - .withRequiresRestart(true) - .withServerRoleGroupConfigurations( - Arrays - .asList( - new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("toqcjmklja"), - new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("fepgzgq"), - new ServerRoleGroupConfiguration() - .withRole(ServerRole.COORDINATOR) - .withValue("ierhhbcsglummaj"))); + ConfigurationProperties model + = new ConfigurationProperties().withRequiresRestart(true).withServerRoleGroupConfigurations( + Arrays.asList(new ServerRoleGroupConfiguration().withRole(ServerRole.COORDINATOR).withValue("c"), + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("pvhez"), + new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("yvxyqjp"))); model = BinaryData.fromObject(model).toObject(ConfigurationProperties.class); Assertions.assertEquals(true, model.requiresRestart()); - Assertions.assertEquals(ServerRole.WORKER, model.serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("toqcjmklja", model.serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals(ServerRole.COORDINATOR, model.serverRoleGroupConfigurations().get(0).role()); + Assertions.assertEquals("c", model.serverRoleGroupConfigurations().get(0).value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorWithResponseMockTests.java index 1b4e21838624c..bd0f2a6545c58 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetCoordinatorWithResponseMockTests.java @@ -30,42 +30,28 @@ public void testGetCoordinatorWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"value\":\"vpgylgqgitxmed\",\"source\":\"c\",\"description\":\"ynqwwncwzzhxgk\",\"defaultValue\":\"mgucna\",\"dataType\":\"Boolean\",\"allowedValues\":\"oellwp\",\"requiresRestart\":false,\"provisioningState\":\"InProgress\"},\"id\":\"fqbuaceopzf\",\"name\":\"rhhuaopppcqeqx\",\"type\":\"lzdahzxctobgbkdm\"}"; + String responseStr + = "{\"properties\":{\"value\":\"bsrfbj\",\"source\":\"twss\",\"description\":\"ftpvjzbexil\",\"defaultValue\":\"nfqqnvwp\",\"dataType\":\"Integer\",\"allowedValues\":\"ruoujmk\",\"requiresRestart\":false,\"provisioningState\":\"Failed\"},\"id\":\"tjrybnwjewgdr\",\"name\":\"ervnaenqpehi\",\"type\":\"doy\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - ServerConfiguration response = - manager - .configurations() - .getCoordinatorWithResponse( - "tfolhbnx", "nalaulppg", "dtpnapnyiropuhp", com.azure.core.util.Context.NONE) - .getValue(); + ServerConfiguration response = manager.configurations() + .getCoordinatorWithResponse("shqjohxcrsbf", "vasrruvwb", "sqfsubcgjbirxb", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("vpgylgqgitxmed", response.value()); + Assertions.assertEquals("bsrfbj", response.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeWithResponseMockTests.java index dbf8a169b5d36..a55aa3ca12b4f 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetNodeWithResponseMockTests.java @@ -30,42 +30,27 @@ public void testGetNodeWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"value\":\"oygmift\",\"source\":\"zdnds\",\"description\":\"nayqi\",\"defaultValue\":\"nduhavhqlkthum\",\"dataType\":\"Enumeration\",\"allowedValues\":\"bgycduiertgccym\",\"requiresRestart\":true,\"provisioningState\":\"InProgress\"},\"id\":\"slqlfmmdn\",\"name\":\"bglzpswi\",\"type\":\"d\"}"; + String responseStr + = "{\"properties\":{\"value\":\"onz\",\"source\":\"sikvmkqzeqqkdlt\",\"description\":\"xmhhvhgureo\",\"defaultValue\":\"wobdagxtibqdx\",\"dataType\":\"Integer\",\"allowedValues\":\"kbogqxndlkzgx\",\"requiresRestart\":false,\"provisioningState\":\"InProgress\"},\"id\":\"bpodxunkbebxm\",\"name\":\"byyntwlrbqt\",\"type\":\"oievseotgqrlltm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - ServerConfiguration response = - manager - .configurations() - .getNodeWithResponse( - "taruoujmkcj", "wqytjrybnwjewgdr", "ervnaenqpehi", com.azure.core.util.Context.NONE) - .getValue(); + ServerConfiguration response = manager.configurations() + .getNodeWithResponse("nkww", "pp", "flcxoga", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("oygmift", response.value()); + Assertions.assertEquals("onz", response.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetWithResponseMockTests.java index 63d1ad6ecc93a..a53465051eb9e 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsGetWithResponseMockTests.java @@ -31,43 +31,29 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"description\":\"i\",\"dataType\":\"Enumeration\",\"allowedValues\":\"pjzu\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"ultskzbbtdz\",\"defaultValue\":\"veekgpwozuhkfp\",\"source\":\"jyofdxluusdtto\"},{\"role\":\"Coordinator\",\"value\":\"aboekqv\",\"defaultValue\":\"lns\",\"source\":\"bxwyjsflhhcaa\"}],\"provisioningState\":\"Failed\"},\"id\":\"xisxyawjoyaqcsl\",\"name\":\"jpkiidzyexznelix\",\"type\":\"nr\"}"; + String responseStr + = "{\"properties\":{\"description\":\"mfqjhhkxbp\",\"dataType\":\"Enumeration\",\"allowedValues\":\"jhxxjyn\",\"requiresRestart\":false,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"krtswbxqz\",\"defaultValue\":\"zjf\",\"source\":\"vjfdx\"},{\"role\":\"Coordinator\",\"value\":\"vetvt\",\"defaultValue\":\"aqtdoqmcbx\",\"source\":\"vxysl\"}],\"provisioningState\":\"InProgress\"},\"id\":\"fxoblytkb\",\"name\":\"mpew\",\"type\":\"wfbkrvrns\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Configuration response = - manager - .configurations() - .getWithResponse("qzcjrvxdj", "lmwlxkvugfhzo", "awjvzunluthnnp", com.azure.core.util.Context.NONE) - .getValue(); + Configuration response = manager.configurations() + .getWithResponse("lzdahzxctobgbkdm", "izpost", "grcfb", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(true, response.requiresRestart()); + Assertions.assertEquals(false, response.requiresRestart()); Assertions.assertEquals(ServerRole.COORDINATOR, response.serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("ultskzbbtdz", response.serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals("krtswbxqz", response.serverRoleGroupConfigurations().get(0).value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterMockTests.java index 9be3954dcac87..fe5e078eafd14 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByClusterMockTests.java @@ -32,42 +32,31 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"description\":\"btn\",\"dataType\":\"Enumeration\",\"allowedValues\":\"bwwaloa\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"rtzju\",\"defaultValue\":\"wyzmhtxon\",\"source\":\"ts\"},{\"role\":\"Worker\",\"value\":\"jcbpwxqpsrknft\",\"defaultValue\":\"vriuhprwmdyvx\",\"source\":\"ayriwwroyqbexrm\"}],\"provisioningState\":\"Canceled\"},\"id\":\"ycnojvknmefqsg\",\"name\":\"vah\",\"type\":\"pjyzhpv\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"description\":\"git\",\"dataType\":\"Integer\",\"allowedValues\":\"jvc\",\"requiresRestart\":true,\"serverRoleGroupConfigurations\":[{\"role\":\"Coordinator\",\"value\":\"wwncwzzhxgk\",\"defaultValue\":\"mgucna\",\"source\":\"t\"}],\"provisioningState\":\"InProgress\"},\"id\":\"lwptfdy\",\"name\":\"pfqbuaceopzf\",\"type\":\"rhhuaopppcqeqx\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.configurations().listByCluster("c", "wxzvlvqhjkb", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.configurations().listByCluster("dtpnapnyiropuhp", "gvpgy", com.azure.core.util.Context.NONE); Assertions.assertEquals(true, response.iterator().next().requiresRestart()); - Assertions - .assertEquals( - ServerRole.COORDINATOR, response.iterator().next().serverRoleGroupConfigurations().get(0).role()); - Assertions.assertEquals("rtzju", response.iterator().next().serverRoleGroupConfigurations().get(0).value()); + Assertions.assertEquals(ServerRole.COORDINATOR, + response.iterator().next().serverRoleGroupConfigurations().get(0).role()); + Assertions.assertEquals("wwncwzzhxgk", + response.iterator().next().serverRoleGroupConfigurations().get(0).value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerMockTests.java index a110a2f75e79b..176298658a47d 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsListByServerMockTests.java @@ -31,38 +31,27 @@ public void testListByServer() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"value\":\"rjaw\",\"source\":\"wgxhn\",\"description\":\"kxfbkpycgklwndn\",\"defaultValue\":\"dauwhvylwzbtd\",\"dataType\":\"Numeric\",\"allowedValues\":\"znbmpowuwprzq\",\"requiresRestart\":true,\"provisioningState\":\"Canceled\"},\"id\":\"upjm\",\"name\":\"hfxobbcswsrtj\",\"type\":\"iplrbpbewtghfgb\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"value\":\"wyjsflhhcaalnjix\",\"source\":\"xyawj\",\"description\":\"aq\",\"defaultValue\":\"lyjpk\",\"dataType\":\"Integer\",\"allowedValues\":\"yexz\",\"requiresRestart\":true,\"provisioningState\":\"InProgress\"},\"id\":\"nr\",\"name\":\"tfolhbnx\",\"type\":\"nalaulppg\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.configurations().listByServer("ueiotwmcdyt", "x", "it", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.configurations().listByServer("us", "ttouwaboekqvkel", "smv", com.azure.core.util.Context.NONE); - Assertions.assertEquals("rjaw", response.iterator().next().value()); + Assertions.assertEquals("wyjsflhhcaalnjix", response.iterator().next().value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorMockTests.java index 1797b3a37dc37..b862486736cd3 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnCoordinatorMockTests.java @@ -31,45 +31,27 @@ public void testUpdateOnCoordinator() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"value\":\"wfbkrvrns\",\"source\":\"hqjohxcrsbfova\",\"description\":\"ruvw\",\"defaultValue\":\"sqfsubcgjbirxb\",\"dataType\":\"Numeric\",\"allowedValues\":\"rfbjf\",\"requiresRestart\":false,\"provisioningState\":\"Succeeded\"},\"id\":\"t\",\"name\":\"tpvjzbexilzznfqq\",\"type\":\"vwpm\"}"; + String responseStr + = "{\"properties\":{\"value\":\"rsc\",\"source\":\"t\",\"description\":\"vfiwjmygtdss\",\"defaultValue\":\"wtmwerio\",\"dataType\":\"Boolean\",\"allowedValues\":\"qsemwabne\",\"requiresRestart\":false,\"provisioningState\":\"Succeeded\"},\"id\":\"h\",\"name\":\"d\",\"type\":\"lvwiwubmwmbesl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - ServerConfiguration response = - manager - .configurations() - .updateOnCoordinator( - "izpost", - "grcfb", - "nrmfqjhhk", - new ServerConfigurationInner().withValue("pvjymjhxxjyng"), - com.azure.core.util.Context.NONE); + ServerConfiguration response = manager.configurations().updateOnCoordinator("mifthnzdnd", "l", "nayqi", + new ServerConfigurationInner().withValue("nduhavhqlkthum"), com.azure.core.util.Context.NONE); - Assertions.assertEquals("wfbkrvrns", response.value()); + Assertions.assertEquals("rsc", response.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeMockTests.java index ce781856b3909..ea77b1dc0fb92 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ConfigurationsUpdateOnNodeMockTests.java @@ -31,45 +31,27 @@ public void testUpdateOnNode() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"value\":\"mwmbes\",\"source\":\"nkww\",\"description\":\"pjflcxogao\",\"defaultValue\":\"nzmnsikvm\",\"dataType\":\"Boolean\",\"allowedValues\":\"qqkdltfzxmhhvhgu\",\"requiresRestart\":false,\"provisioningState\":\"Succeeded\"},\"id\":\"obdagxtibqdxb\",\"name\":\"wakbogqxndl\",\"type\":\"zgx\"}"; + String responseStr + = "{\"properties\":{\"value\":\"c\",\"source\":\"fwdsj\",\"description\":\"aljutiiswac\",\"defaultValue\":\"gdkz\",\"dataType\":\"Enumeration\",\"allowedValues\":\"fvhqc\",\"requiresRestart\":true,\"provisioningState\":\"Succeeded\"},\"id\":\"n\",\"name\":\"pfuflrw\",\"type\":\"mh\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - ServerConfiguration response = - manager - .configurations() - .updateOnNode( - "cwyhzdxssa", - "bzmnvdfznud", - "od", - new ServerConfigurationInner().withValue("zbn"), - com.azure.core.util.Context.NONE); + ServerConfiguration response = manager.configurations().updateOnNode("wlauwzizxbmpg", "jefuzmuvpbttdumo", "p", + new ServerConfigurationInner().withValue("ebmnzbtbhjpglk"), com.azure.core.util.Context.NONE); - Assertions.assertEquals("mwmbes", response.value()); + Assertions.assertEquals("c", response.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleInnerTests.java index ab645ea60d9df..0c904d2b2ba31 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleInnerTests.java @@ -11,21 +11,19 @@ public final class FirewallRuleInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - FirewallRuleInner model = - BinaryData - .fromString( - "{\"properties\":{\"startIpAddress\":\"clwhijcoejctbz\",\"endIpAddress\":\"qsqsy\",\"provisioningState\":\"Canceled\"},\"id\":\"fkgukdkexxppof\",\"name\":\"xaxcfjpgddtocjjx\",\"type\":\"vpmouexhdzxib\"}") - .toObject(FirewallRuleInner.class); - Assertions.assertEquals("clwhijcoejctbz", model.startIpAddress()); - Assertions.assertEquals("qsqsy", model.endIpAddress()); + FirewallRuleInner model = BinaryData.fromString( + "{\"properties\":{\"startIpAddress\":\"yzydagfuaxbezyi\",\"endIpAddress\":\"okktwhrdxw\",\"provisioningState\":\"Canceled\"},\"id\":\"sm\",\"name\":\"surex\",\"type\":\"moryocfsfksym\"}") + .toObject(FirewallRuleInner.class); + Assertions.assertEquals("yzydagfuaxbezyi", model.startIpAddress()); + Assertions.assertEquals("okktwhrdxw", model.endIpAddress()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FirewallRuleInner model = - new FirewallRuleInner().withStartIpAddress("clwhijcoejctbz").withEndIpAddress("qsqsy"); + FirewallRuleInner model + = new FirewallRuleInner().withStartIpAddress("yzydagfuaxbezyi").withEndIpAddress("okktwhrdxw"); model = BinaryData.fromObject(model).toObject(FirewallRuleInner.class); - Assertions.assertEquals("clwhijcoejctbz", model.startIpAddress()); - Assertions.assertEquals("qsqsy", model.endIpAddress()); + Assertions.assertEquals("yzydagfuaxbezyi", model.startIpAddress()); + Assertions.assertEquals("okktwhrdxw", model.endIpAddress()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleListResultTests.java index 9eff2bb72b08f..1a3b74f3b1627 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRuleListResultTests.java @@ -13,28 +13,19 @@ public final class FirewallRuleListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - FirewallRuleListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"startIpAddress\":\"uhrhcffcyddgl\",\"endIpAddress\":\"jthjqkwpyei\",\"provisioningState\":\"Succeeded\"},\"id\":\"ciwqvhk\",\"name\":\"ixuigdtopbobj\",\"type\":\"ghmewuam\"},{\"properties\":{\"startIpAddress\":\"uhrzayvvt\",\"endIpAddress\":\"gvdfgiotkftutq\",\"provisioningState\":\"Failed\"},\"id\":\"xlefgugnxkrx\",\"name\":\"qmi\",\"type\":\"tthzrvqd\"},{\"properties\":{\"startIpAddress\":\"abhjybi\",\"endIpAddress\":\"ehoqfbowskan\",\"provisioningState\":\"Failed\"},\"id\":\"lcuiywgqywgndr\",\"name\":\"ynhz\",\"type\":\"pphrcgynco\"},{\"properties\":{\"startIpAddress\":\"pec\",\"endIpAddress\":\"vmmcoofs\",\"provisioningState\":\"Canceled\"},\"id\":\"v\",\"name\":\"bmqj\",\"type\":\"abcypmivk\"}]}") - .toObject(FirewallRuleListResult.class); - Assertions.assertEquals("uhrhcffcyddgl", model.value().get(0).startIpAddress()); - Assertions.assertEquals("jthjqkwpyei", model.value().get(0).endIpAddress()); + FirewallRuleListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"startIpAddress\":\"poczvyifqrvkdvjs\",\"endIpAddress\":\"lrmv\",\"provisioningState\":\"Succeeded\"},\"id\":\"atkpnp\",\"name\":\"lexxbczwtru\",\"type\":\"iqzbq\"}]}") + .toObject(FirewallRuleListResult.class); + Assertions.assertEquals("poczvyifqrvkdvjs", model.value().get(0).startIpAddress()); + Assertions.assertEquals("lrmv", model.value().get(0).endIpAddress()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FirewallRuleListResult model = - new FirewallRuleListResult() - .withValue( - Arrays - .asList( - new FirewallRuleInner().withStartIpAddress("uhrhcffcyddgl").withEndIpAddress("jthjqkwpyei"), - new FirewallRuleInner().withStartIpAddress("uhrzayvvt").withEndIpAddress("gvdfgiotkftutq"), - new FirewallRuleInner().withStartIpAddress("abhjybi").withEndIpAddress("ehoqfbowskan"), - new FirewallRuleInner().withStartIpAddress("pec").withEndIpAddress("vmmcoofs"))); + FirewallRuleListResult model = new FirewallRuleListResult().withValue( + Arrays.asList(new FirewallRuleInner().withStartIpAddress("poczvyifqrvkdvjs").withEndIpAddress("lrmv"))); model = BinaryData.fromObject(model).toObject(FirewallRuleListResult.class); - Assertions.assertEquals("uhrhcffcyddgl", model.value().get(0).startIpAddress()); - Assertions.assertEquals("jthjqkwpyei", model.value().get(0).endIpAddress()); + Assertions.assertEquals("poczvyifqrvkdvjs", model.value().get(0).startIpAddress()); + Assertions.assertEquals("lrmv", model.value().get(0).endIpAddress()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulePropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulePropertiesTests.java index 3e3ccec370f04..5bf9b9ec005f7 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulePropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulePropertiesTests.java @@ -11,21 +11,20 @@ public final class FirewallRulePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - FirewallRuleProperties model = - BinaryData - .fromString( - "{\"startIpAddress\":\"eojnxqbzvddn\",\"endIpAddress\":\"wndeicbtwnp\",\"provisioningState\":\"Canceled\"}") - .toObject(FirewallRuleProperties.class); - Assertions.assertEquals("eojnxqbzvddn", model.startIpAddress()); - Assertions.assertEquals("wndeicbtwnp", model.endIpAddress()); + FirewallRuleProperties model = BinaryData + .fromString( + "{\"startIpAddress\":\"dystkiiuxhqyud\",\"endIpAddress\":\"o\",\"provisioningState\":\"Succeeded\"}") + .toObject(FirewallRuleProperties.class); + Assertions.assertEquals("dystkiiuxhqyud", model.startIpAddress()); + Assertions.assertEquals("o", model.endIpAddress()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FirewallRuleProperties model = - new FirewallRuleProperties().withStartIpAddress("eojnxqbzvddn").withEndIpAddress("wndeicbtwnp"); + FirewallRuleProperties model + = new FirewallRuleProperties().withStartIpAddress("dystkiiuxhqyud").withEndIpAddress("o"); model = BinaryData.fromObject(model).toObject(FirewallRuleProperties.class); - Assertions.assertEquals("eojnxqbzvddn", model.startIpAddress()); - Assertions.assertEquals("wndeicbtwnp", model.endIpAddress()); + Assertions.assertEquals("dystkiiuxhqyud", model.startIpAddress()); + Assertions.assertEquals("o", model.endIpAddress()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateMockTests.java index 39b5304f913fa..43e36bfb4d71c 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesCreateOrUpdateMockTests.java @@ -30,45 +30,29 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"startIpAddress\":\"hgw\",\"endIpAddress\":\"apnedgfbcvkc\",\"provisioningState\":\"Succeeded\"},\"id\":\"keqdcvdrhvoods\",\"name\":\"tbobz\",\"type\":\"opcjwvnhd\"}"; + String responseStr + = "{\"properties\":{\"startIpAddress\":\"i\",\"endIpAddress\":\"m\",\"provisioningState\":\"Succeeded\"},\"id\":\"bahwfl\",\"name\":\"szdtmhrkwof\",\"type\":\"yvoqa\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - FirewallRule response = - manager - .firewallRules() - .define("c") - .withExistingServerGroupsv2("sg", "b") - .withStartIpAddress("hfwdsjnkaljutiis") - .withEndIpAddress("acffgdkzzewkfvhq") - .create(); + FirewallRule response + = manager.firewallRules().define("smocmbq").withExistingServerGroupsv2("mquxvypo", "gkopkwhojvpajqgx") + .withStartIpAddress("qvmkcxo").withEndIpAddress("apvhelxprgly").create(); - Assertions.assertEquals("hgw", response.startIpAddress()); - Assertions.assertEquals("apnedgfbcvkc", response.endIpAddress()); + Assertions.assertEquals("i", response.startIpAddress()); + Assertions.assertEquals("m", response.endIpAddress()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetWithResponseMockTests.java index 72b6e4eac9325..788765aeb248b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesGetWithResponseMockTests.java @@ -30,42 +30,28 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"startIpAddress\":\"tkoievseotgq\",\"endIpAddress\":\"l\",\"provisioningState\":\"Failed\"},\"id\":\"wlauwzizxbmpg\",\"name\":\"jefuzmuvpbttdumo\",\"type\":\"p\"}"; + String responseStr + = "{\"properties\":{\"startIpAddress\":\"pnedgf\",\"endIpAddress\":\"cvkcvqvpkeqdcv\",\"provisioningState\":\"Failed\"},\"id\":\"ood\",\"name\":\"otbobzdopcj\",\"type\":\"vnhdldwmgxcxr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - FirewallRule response = - manager - .firewallRules() - .getWithResponse("uriplbpodxunkb", "bxmubyynt", "lrb", com.azure.core.util.Context.NONE) - .getValue(); + FirewallRule response = manager.firewallRules() + .getWithResponse("lxyjr", "sag", "fcnihgwq", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("tkoievseotgq", response.startIpAddress()); - Assertions.assertEquals("l", response.endIpAddress()); + Assertions.assertEquals("pnedgf", response.startIpAddress()); + Assertions.assertEquals("cvkcvqvpkeqdcv", response.endIpAddress()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterMockTests.java index 7651e8bd7cf37..3ff93f6199579 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/FirewallRulesListByClusterMockTests.java @@ -31,39 +31,28 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"startIpAddress\":\"bhjpglkfgohdne\",\"endIpAddress\":\"el\",\"provisioningState\":\"Succeeded\"},\"id\":\"dyhtozfikdowwquu\",\"name\":\"xzxcl\",\"type\":\"ithhqzon\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"startIpAddress\":\"pdggkzzlvm\",\"endIpAddress\":\"mpaxmodfvuefywsb\",\"provisioningState\":\"Succeeded\"},\"id\":\"wyhrfouyftaakc\",\"name\":\"wiyzvqtmnubexkp\",\"type\":\"ksmond\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.firewallRules().listByCluster("xe", "mnzb", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.firewallRules().listByCluster("lpmutwuoegrpkhj", "niyqslui", com.azure.core.util.Context.NONE); - Assertions.assertEquals("bhjpglkfgohdne", response.iterator().next().startIpAddress()); - Assertions.assertEquals("el", response.iterator().next().endIpAddress()); + Assertions.assertEquals("pdggkzzlvm", response.iterator().next().startIpAddress()); + Assertions.assertEquals("mpaxmodfvuefywsb", response.iterator().next().endIpAddress()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/MaintenanceWindowTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/MaintenanceWindowTests.java index 1c0a24d2d7cf1..0067da24b54f2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/MaintenanceWindowTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/MaintenanceWindowTests.java @@ -11,11 +11,9 @@ public final class MaintenanceWindowTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MaintenanceWindow model = - BinaryData - .fromString( - "{\"customWindow\":\"dxbmtqioq\",\"startHour\":1116722151,\"startMinute\":1801213143,\"dayOfWeek\":538577990}") - .toObject(MaintenanceWindow.class); + MaintenanceWindow model = BinaryData.fromString( + "{\"customWindow\":\"dxbmtqioq\",\"startHour\":1116722151,\"startMinute\":1801213143,\"dayOfWeek\":538577990}") + .toObject(MaintenanceWindow.class); Assertions.assertEquals("dxbmtqioq", model.customWindow()); Assertions.assertEquals(1116722151, model.startHour()); Assertions.assertEquals(1801213143, model.startMinute()); @@ -24,12 +22,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MaintenanceWindow model = - new MaintenanceWindow() - .withCustomWindow("dxbmtqioq") - .withStartHour(1116722151) - .withStartMinute(1801213143) - .withDayOfWeek(538577990); + MaintenanceWindow model = new MaintenanceWindow().withCustomWindow("dxbmtqioq").withStartHour(1116722151) + .withStartMinute(1801213143).withDayOfWeek(538577990); model = BinaryData.fromObject(model).toObject(MaintenanceWindow.class); Assertions.assertEquals("dxbmtqioq", model.customWindow()); Assertions.assertEquals(1116722151, model.startHour()); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityInnerTests.java index 1b44d37d046a2..af552d691aef8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityInnerTests.java @@ -11,29 +11,23 @@ public final class NameAvailabilityInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NameAvailabilityInner model = - BinaryData - .fromString( - "{\"message\":\"rjfeallnwsubisnj\",\"nameAvailable\":false,\"name\":\"ngnzscxaqwoochc\",\"type\":\"nqvpkvlrxnje\"}") + NameAvailabilityInner model + = BinaryData.fromString("{\"message\":\"yjr\",\"nameAvailable\":false,\"name\":\"aos\",\"type\":\"xc\"}") .toObject(NameAvailabilityInner.class); - Assertions.assertEquals("rjfeallnwsubisnj", model.message()); + Assertions.assertEquals("yjr", model.message()); Assertions.assertEquals(false, model.nameAvailable()); - Assertions.assertEquals("ngnzscxaqwoochc", model.name()); - Assertions.assertEquals("nqvpkvlrxnje", model.type()); + Assertions.assertEquals("aos", model.name()); + Assertions.assertEquals("xc", model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NameAvailabilityInner model = - new NameAvailabilityInner() - .withMessage("rjfeallnwsubisnj") - .withNameAvailable(false) - .withName("ngnzscxaqwoochc") - .withType("nqvpkvlrxnje"); + NameAvailabilityInner model + = new NameAvailabilityInner().withMessage("yjr").withNameAvailable(false).withName("aos").withType("xc"); model = BinaryData.fromObject(model).toObject(NameAvailabilityInner.class); - Assertions.assertEquals("rjfeallnwsubisnj", model.message()); + Assertions.assertEquals("yjr", model.message()); Assertions.assertEquals(false, model.nameAvailable()); - Assertions.assertEquals("ngnzscxaqwoochc", model.name()); - Assertions.assertEquals("nqvpkvlrxnje", model.type()); + Assertions.assertEquals("aos", model.name()); + Assertions.assertEquals("xc", model.type()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityRequestTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityRequestTests.java index d68f6ff7cf9c8..79448dfd790ca 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityRequestTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/NameAvailabilityRequestTests.java @@ -11,15 +11,15 @@ public final class NameAvailabilityRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NameAvailabilityRequest model = - BinaryData.fromString("{\"name\":\"tqgtzxdpnqbqq\"}").toObject(NameAvailabilityRequest.class); - Assertions.assertEquals("tqgtzxdpnqbqq", model.name()); + NameAvailabilityRequest model + = BinaryData.fromString("{\"name\":\"uuimjmvxieduug\"}").toObject(NameAvailabilityRequest.class); + Assertions.assertEquals("uuimjmvxieduug", model.name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NameAvailabilityRequest model = new NameAvailabilityRequest().withName("tqgtzxdpnqbqq"); + NameAvailabilityRequest model = new NameAvailabilityRequest().withName("uuimjmvxieduug"); model = BinaryData.fromObject(model).toObject(NameAvailabilityRequest.class); - Assertions.assertEquals("tqgtzxdpnqbqq", model.name()); + Assertions.assertEquals("uuimjmvxieduug", model.name()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationDisplayTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationDisplayTests.java index 7e93e332d64df..ab7177b6f0f20 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationDisplayTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationDisplayTests.java @@ -10,11 +10,9 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"auu\",\"resource\":\"jmvxie\",\"operation\":\"ugidyjrr\",\"description\":\"y\"}") - .toObject(OperationDisplay.class); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"deyeamdphagalpbu\",\"resource\":\"gipwhonowkg\",\"operation\":\"wankixzbi\",\"description\":\"eputtmrywnuzoqf\"}") + .toObject(OperationDisplay.class); } @org.junit.jupiter.api.Test diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationInnerTests.java index 1ee5d23bae38a..13dca564dcc64 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationInnerTests.java @@ -11,11 +11,9 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"llr\",\"display\":{\"provider\":\"d\",\"resource\":\"atkpnp\",\"operation\":\"exxbczwtr\",\"description\":\"iqzbq\"},\"isDataAction\":true,\"origin\":\"user\",\"properties\":{\"lhzdobp\":\"dataokacspk\",\"kcciwwzjuqkhr\":\"datajmflbvvnch\",\"oskg\":\"dataajiwkuo\"}}") - .toObject(OperationInner.class); + OperationInner model = BinaryData.fromString( + "{\"name\":\"ldngkpoci\",\"display\":{\"provider\":\"yxoegukgjnp\",\"resource\":\"cgygev\",\"operation\":\"ntypmrbpizcdrqj\",\"description\":\"pyd\"},\"isDataAction\":true,\"origin\":\"user\",\"properties\":{\"w\":\"dataoejzi\",\"tgzfbishcbkh\":\"datafsj\"}}") + .toObject(OperationInner.class); Assertions.assertEquals(true, model.isDataAction()); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationListResultTests.java index 1c41a471b9199..f55b76595a523 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationListResultTests.java @@ -13,25 +13,16 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"pheoflokeyy\",\"display\":{\"provider\":\"jbdlwtgrhpdjpju\",\"resource\":\"sxazjpq\",\"operation\":\"gual\",\"description\":\"xxhejjzzvd\"},\"isDataAction\":true,\"origin\":\"system\",\"properties\":{\"cynpwlbjnp\":\"datafhotw\"}},{\"name\":\"cftadeh\",\"display\":{\"provider\":\"tyfsoppusuesn\",\"resource\":\"dejbavo\",\"operation\":\"zdmohctbqvu\",\"description\":\"xdn\"},\"isDataAction\":false,\"origin\":\"system\",\"properties\":{\"dyggdtjixhbku\":\"datajjugwdkcglhslaz\",\"fyexfwhy\":\"datafqweykhmene\",\"amdecte\":\"datacibvyvdcsitynn\"}},{\"name\":\"iqscjeypv\",\"display\":{\"provider\":\"rkgqhcjrefo\",\"resource\":\"mkqsleyyv\",\"operation\":\"qjpkcattpngjcrc\",\"description\":\"sqpjhvmdajvn\"},\"isDataAction\":true,\"origin\":\"user\",\"properties\":{\"yhltrpmopjmcm\":\"datacanoaeupf\"}},{\"name\":\"u\",\"display\":{\"provider\":\"hfuiuaodsfc\",\"resource\":\"vxodpu\",\"operation\":\"myzydagfuaxbez\",\"description\":\"uokktwhrdxwz\"},\"isDataAction\":false,\"origin\":\"system\",\"properties\":{\"ksymd\":\"dataureximoryocfs\",\"kiiuxhqyudxor\":\"datays\"}}],\"nextLink\":\"nbpoczvyifqrvkdv\"}") - .toObject(OperationListResult.class); + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"clhocohsl\",\"display\":{\"provider\":\"leggzfbu\",\"resource\":\"mvfaxkffeiith\",\"operation\":\"m\",\"description\":\"yvshxmz\"},\"isDataAction\":true,\"origin\":\"user\",\"properties\":{\"pydptko\":\"dataigrxwburvjxxjn\"}}],\"nextLink\":\"kouknvudwtiu\"}") + .toObject(OperationListResult.class); Assertions.assertEquals(true, model.value().get(0).isDataAction()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationListResult model = - new OperationListResult() - .withValue( - Arrays - .asList( - new OperationInner().withIsDataAction(true), - new OperationInner().withIsDataAction(false), - new OperationInner().withIsDataAction(true), - new OperationInner().withIsDataAction(false))); + OperationListResult model + = new OperationListResult().withValue(Arrays.asList(new OperationInner().withIsDataAction(true))); model = BinaryData.fromObject(model).toObject(OperationListResult.class); Assertions.assertEquals(true, model.value().get(0).isDataAction()); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListMockTests.java index 0b8d07ca3d70c..5f16d73d208f6 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/OperationsListMockTests.java @@ -31,34 +31,23 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"hojvpajqgxysmocm\",\"display\":{\"provider\":\"qvmkcxo\",\"resource\":\"pvhelxprg\",\"operation\":\"atddc\",\"description\":\"bcuejrjxgci\"},\"isDataAction\":false,\"origin\":\"system\",\"properties\":{\"bahwfl\":\"dataxsdqrhzoymibmrqy\",\"yvoqa\":\"dataszdtmhrkwof\",\"wo\":\"datapiexpbtgiw\",\"kcnqxwbpo\":\"datanwashrtd\"}}]}"; + String responseStr + = "{\"value\":[{\"name\":\"ovvqfovljxywsu\",\"display\":{\"provider\":\"rsndsytgadgvra\",\"resource\":\"en\",\"operation\":\"nzar\",\"description\":\"lquuijfqkacewii\"},\"isDataAction\":false,\"origin\":\"NotSpecified\",\"properties\":{\"hqkvpuvksgplsak\":\"databwwift\",\"n\":\"dataynfs\",\"ntorzihleosjswsr\":\"datajphuopxodlqi\"}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionInnerTests.java index 6d9bedfef9fce..12f58fe09cf89 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionInnerTests.java @@ -14,33 +14,25 @@ public final class PrivateEndpointConnectionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionInner model = - BinaryData - .fromString( - "{\"properties\":{\"groupIds\":[\"hjfbebrjcxe\",\"fuwutttxf\"],\"privateEndpoint\":{\"id\":\"birphxepcyva\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"jky\",\"actionsRequired\":\"j\"},\"provisioningState\":\"Creating\"},\"id\":\"qgidokgjljyo\",\"name\":\"gvcl\",\"type\":\"bgsncghkjeszzhb\"}") - .toObject(PrivateEndpointConnectionInner.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.PENDING, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("jky", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("j", model.privateLinkServiceConnectionState().actionsRequired()); + PrivateEndpointConnectionInner model = BinaryData.fromString( + "{\"properties\":{\"groupIds\":[\"sthsu\",\"ocmnyyazttbtwwrq\",\"uedck\"],\"privateEndpoint\":{\"id\":\"biexzfeyu\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"bxu\",\"actionsRequired\":\"bhqwalmuzyoxa\"},\"provisioningState\":\"Deleting\"},\"id\":\"zjancuxr\",\"name\":\"d\",\"type\":\"bavxbniwdjswzt\"}") + .toObject(PrivateEndpointConnectionInner.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("bxu", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("bhqwalmuzyoxa", model.privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionInner model = - new PrivateEndpointConnectionInner() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.PENDING) - .withDescription("jky") - .withActionsRequired("j")); + PrivateEndpointConnectionInner model = new PrivateEndpointConnectionInner() + .withPrivateEndpoint(new PrivateEndpoint()).withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("bxu").withActionsRequired("bhqwalmuzyoxa")); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionInner.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.PENDING, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("jky", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("j", model.privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("bxu", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("bhqwalmuzyoxa", model.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionListResultTests.java index db7df280cc861..31a1f5840f594 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionListResultTests.java @@ -16,62 +16,44 @@ public final class PrivateEndpointConnectionListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"groupIds\":[\"csonpclhoco\"],\"privateEndpoint\":{\"id\":\"kevle\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"buhfmvfaxkffeiit\",\"actionsRequired\":\"vmezy\"},\"provisioningState\":\"Failed\"},\"id\":\"mzsb\",\"name\":\"zoggigrxwburvjxx\",\"type\":\"nspydptkoenkoukn\"},{\"properties\":{\"groupIds\":[\"tiukbldngkpoci\",\"azyxoegukg\",\"npiucgygevqznty\"],\"privateEndpoint\":{\"id\":\"bpizcdrqjsdpydn\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"de\",\"actionsRequired\":\"jzicwifsjt\"},\"provisioningState\":\"Deleting\"},\"id\":\"bishcbkhajdeyea\",\"name\":\"dphagalpbuxwgip\",\"type\":\"honowkgshwank\"},{\"properties\":{\"groupIds\":[\"injep\"],\"privateEndpoint\":{\"id\":\"mryw\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"qftiy\",\"actionsRequired\":\"rnkcqvyxlw\"},\"provisioningState\":\"Deleting\"},\"id\":\"icohoqqnwvl\",\"name\":\"yav\",\"type\":\"hheunmmqhgyx\"},{\"properties\":{\"groupIds\":[\"ocukoklyax\"],\"privateEndpoint\":{\"id\":\"nuqszfkbey\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"mjmwvvjektcx\",\"actionsRequired\":\"nhwlrsffrzpwvl\"},\"provisioningState\":\"Succeeded\"},\"id\":\"biqylihkaet\",\"name\":\"kt\",\"type\":\"fcivfsnkym\"}]}") - .toObject(PrivateEndpointConnectionListResult.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, - model.value().get(0).privateLinkServiceConnectionState().status()); - Assertions - .assertEquals("buhfmvfaxkffeiit", model.value().get(0).privateLinkServiceConnectionState().description()); - Assertions.assertEquals("vmezy", model.value().get(0).privateLinkServiceConnectionState().actionsRequired()); + PrivateEndpointConnectionListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"groupIds\":[\"n\",\"cqvyxlwhzlsico\",\"oqqnwvlryav\"],\"privateEndpoint\":{\"id\":\"eun\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"gyxzk\",\"actionsRequired\":\"ocukoklyax\"},\"provisioningState\":\"Succeeded\"},\"id\":\"uqszfk\",\"name\":\"eyp\",\"type\":\"wrmjmwvvjektc\"},{\"properties\":{\"groupIds\":[\"hwlrsf\",\"rzpwvlqdqgbiq\",\"lihkaetcktvfc\",\"vf\"],\"privateEndpoint\":{\"id\":\"ymuctqhjfbebrj\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"fuwutttxf\",\"actionsRequired\":\"rbirphxe\"},\"provisioningState\":\"Succeeded\"},\"id\":\"ahfn\",\"name\":\"jky\",\"type\":\"xjvuujqgidokg\"},{\"properties\":{\"groupIds\":[\"oxgvclt\",\"gsncghkjeszz\",\"bijhtxfvgxbf\",\"mxnehmp\"],\"privateEndpoint\":{\"id\":\"xgodebfqkkrbmp\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"iw\",\"actionsRequired\":\"zlfbxzpuzycispnq\"},\"provisioningState\":\"Deleting\"},\"id\":\"gkbrpyyd\",\"name\":\"ibnuqqkpik\",\"type\":\"drgvtqagn\"},{\"properties\":{\"groupIds\":[\"hijggme\",\"fsiarbutr\",\"vpnazzm\"],\"privateEndpoint\":{\"id\":\"unmpxttd\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"nlankxmyskpb\",\"actionsRequired\":\"nbtkcxywnytnr\"},\"provisioningState\":\"Deleting\"},\"id\":\"qidybyx\",\"name\":\"zfcl\",\"type\":\"aaxdbabphlwrq\"}]}") + .toObject(PrivateEndpointConnectionListResult.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.value().get(0).privateLinkServiceConnectionState().status()); + Assertions.assertEquals("gyxzk", model.value().get(0).privateLinkServiceConnectionState().description()); + Assertions.assertEquals("ocukoklyax", + model.value().get(0).privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionListResult model = - new PrivateEndpointConnectionListResult() + PrivateEndpointConnectionListResult model + = new PrivateEndpointConnectionListResult() .withValue( Arrays .asList( - new PrivateEndpointConnectionInner() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) - .withDescription("buhfmvfaxkffeiit") - .withActionsRequired("vmezy")), - new PrivateEndpointConnectionInner() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) - .withDescription("de") - .withActionsRequired("jzicwifsjt")), - new PrivateEndpointConnectionInner() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) - .withDescription("qftiy") - .withActionsRequired("rnkcqvyxlw")), - new PrivateEndpointConnectionInner() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.PENDING) - .withDescription("mjmwvvjektcx") - .withActionsRequired("nhwlrsffrzpwvl")))); + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("gyxzk").withActionsRequired("ocukoklyax")), + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("fuwutttxf").withActionsRequired("rbirphxe")), + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED).withDescription("iw") + .withActionsRequired("zlfbxzpuzycispnq")), + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.PENDING) + .withDescription("nlankxmyskpb").withActionsRequired("nbtkcxywnytnr")))); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionListResult.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, - model.value().get(0).privateLinkServiceConnectionState().status()); - Assertions - .assertEquals("buhfmvfaxkffeiit", model.value().get(0).privateLinkServiceConnectionState().description()); - Assertions.assertEquals("vmezy", model.value().get(0).privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.value().get(0).privateLinkServiceConnectionState().status()); + Assertions.assertEquals("gyxzk", model.value().get(0).privateLinkServiceConnectionState().description()); + Assertions.assertEquals("ocukoklyax", + model.value().get(0).privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionPropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionPropertiesTests.java index c8c10a73f59f1..cffb96c2a8c9e 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionPropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionPropertiesTests.java @@ -14,33 +14,25 @@ public final class PrivateEndpointConnectionPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionProperties model = - BinaryData - .fromString( - "{\"groupIds\":[\"txfvgx\",\"fsm\",\"nehmpvecx\",\"odebfqkkrbmpu\"],\"privateEndpoint\":{\"id\":\"iw\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"fbxzpuzycisp\",\"actionsRequired\":\"zahmgkbrpyydhibn\"},\"provisioningState\":\"Failed\"}") - .toObject(PrivateEndpointConnectionProperties.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("fbxzpuzycisp", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("zahmgkbrpyydhibn", model.privateLinkServiceConnectionState().actionsRequired()); + PrivateEndpointConnectionProperties model = BinaryData.fromString( + "{\"groupIds\":[\"pgn\"],\"privateEndpoint\":{\"id\":\"x\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"bzpfzab\",\"actionsRequired\":\"cuh\"},\"provisioningState\":\"Succeeded\"}") + .toObject(PrivateEndpointConnectionProperties.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("bzpfzab", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("cuh", model.privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionProperties model = - new PrivateEndpointConnectionProperties() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) - .withDescription("fbxzpuzycisp") - .withActionsRequired("zahmgkbrpyydhibn")); + PrivateEndpointConnectionProperties model = new PrivateEndpointConnectionProperties() + .withPrivateEndpoint(new PrivateEndpoint()).withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("bzpfzab").withActionsRequired("cuh")); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionProperties.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("fbxzpuzycisp", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("zahmgkbrpyydhibn", model.privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("bzpfzab", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("cuh", model.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionSimplePropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionSimplePropertiesTests.java index e873cfdea1d97..713dbe43db4eb 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionSimplePropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionSimplePropertiesTests.java @@ -15,37 +15,29 @@ public final class PrivateEndpointConnectionSimplePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionSimpleProperties model = - BinaryData - .fromString( - "{\"privateEndpoint\":{\"id\":\"jpsq\"},\"groupIds\":[\"poyfdkfogkn\",\"gjofjd\"],\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"deupewnwrei\",\"actionsRequired\":\"zyf\"}}") - .toObject(PrivateEndpointConnectionSimpleProperties.class); + PrivateEndpointConnectionSimpleProperties model = BinaryData.fromString( + "{\"privateEndpoint\":{\"id\":\"jpsq\"},\"groupIds\":[\"poyfdkfogkn\",\"gjofjd\"],\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"deupewnwrei\",\"actionsRequired\":\"zyf\"}}") + .toObject(PrivateEndpointConnectionSimpleProperties.class); Assertions.assertEquals("jpsq", model.privateEndpoint().id()); Assertions.assertEquals("poyfdkfogkn", model.groupIds().get(0)); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.REJECTED, + model.privateLinkServiceConnectionState().status()); Assertions.assertEquals("deupewnwrei", model.privateLinkServiceConnectionState().description()); Assertions.assertEquals("zyf", model.privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionSimpleProperties model = - new PrivateEndpointConnectionSimpleProperties() - .withPrivateEndpoint(new PrivateEndpointProperty().withId("jpsq")) - .withGroupIds(Arrays.asList("poyfdkfogkn", "gjofjd")) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) - .withDescription("deupewnwrei") - .withActionsRequired("zyf")); + PrivateEndpointConnectionSimpleProperties model = new PrivateEndpointConnectionSimpleProperties() + .withPrivateEndpoint(new PrivateEndpointProperty().withId("jpsq")) + .withGroupIds(Arrays.asList("poyfdkfogkn", "gjofjd")).withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) + .withDescription("deupewnwrei").withActionsRequired("zyf")); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionSimpleProperties.class); Assertions.assertEquals("jpsq", model.privateEndpoint().id()); Assertions.assertEquals("poyfdkfogkn", model.groupIds().get(0)); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.REJECTED, + model.privateLinkServiceConnectionState().status()); Assertions.assertEquals("deupewnwrei", model.privateLinkServiceConnectionState().description()); Assertions.assertEquals("zyf", model.privateLinkServiceConnectionState().actionsRequired()); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateMockTests.java index 6ff43b30bf2ba..f619485a8cc00 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsCreateOrUpdateMockTests.java @@ -33,52 +33,34 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"groupIds\":[\"dtlwwrlkd\",\"tncvokot\"],\"privateEndpoint\":{\"id\":\"d\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"y\",\"actionsRequired\":\"ogjltdtbnnhad\"},\"provisioningState\":\"Succeeded\"},\"id\":\"kvci\",\"name\":\"hnvpamqgxq\",\"type\":\"u\"}"; + String responseStr + = "{\"properties\":{\"groupIds\":[\"imjwosyt\",\"itc\"],\"privateEndpoint\":{\"id\":\"cktqumiekkezzi\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"fjhdg\",\"actionsRequired\":\"gebdunygaeq\"},\"provisioningState\":\"Succeeded\"},\"id\":\"fatpxllrxcyjmoa\",\"name\":\"su\",\"type\":\"arm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PrivateEndpointConnection response = - manager - .privateEndpointConnections() - .define("iyntorzihle") - .withExistingServerGroupsv2("n", "synljphuopxodl") - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) - .withDescription("iizynkedyatrwyh") - .withActionsRequired("ibzyhwitsmyp")) - .create(); + PrivateEndpointConnection response = manager.privateEndpointConnections().define("rey") + .withExistingServerGroupsv2("imfnjhfjx", "mszkkfo").withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) + .withDescription("xwczelpcire").withActionsRequired("feaenwab")) + .create(); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, response.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("y", response.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("ogjltdtbnnhad", response.privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.REJECTED, + response.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("fjhdg", response.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("gebdunygaeq", response.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java index 40be293f8e0cb..600aaef5a71c2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java @@ -31,45 +31,30 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"groupIds\":[\"e\"],\"privateEndpoint\":{\"id\":\"arrwlquu\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"kacewiipfp\",\"actionsRequired\":\"ji\"},\"provisioningState\":\"Creating\"},\"id\":\"f\",\"name\":\"ohqkvpuvksgpls\",\"type\":\"kn\"}"; + String responseStr + = "{\"properties\":{\"groupIds\":[\"rhyrnxxmueed\"],\"privateEndpoint\":{\"id\":\"dvstkw\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"healmfmtda\",\"actionsRequired\":\"gdv\"},\"provisioningState\":\"Failed\"},\"id\":\"iohgwxrtfud\",\"name\":\"epxgyqagvr\",\"type\":\"mnpkukghimdblxg\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PrivateEndpointConnection response = - manager - .privateEndpointConnections() - .getWithResponse("reqnovvqfov", "jxywsuws", "rsndsytgadgvra", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, response.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("kacewiipfp", response.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("ji", response.privateLinkServiceConnectionState().actionsRequired()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PrivateEndpointConnection response = manager.privateEndpointConnections() + .getWithResponse("queziky", "ggxkallatmelwuip", "ccjzkzivgvv", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, + response.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("healmfmtda", response.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("gdv", response.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterMockTests.java index 31aaba8ff11ba..399870ee30551 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointConnectionsListByClusterMockTests.java @@ -32,46 +32,32 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"groupIds\":[\"uqerpqlpqwc\"],\"privateEndpoint\":{\"id\":\"qgbdbuta\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"tkuwhhmhykojo\",\"actionsRequired\":\"fnndl\"},\"provisioningState\":\"Deleting\"},\"id\":\"koymkcd\",\"name\":\"h\",\"type\":\"pkkpw\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"groupIds\":[\"i\",\"ynkedyatrwyhqmib\",\"yhwitsmypyynpcdp\",\"mnzgmwznmabi\"],\"privateEndpoint\":{\"id\":\"orgjhxbldt\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"lkdmtncvokotllxd\",\"actionsRequired\":\"gsyocogj\"},\"provisioningState\":\"Creating\"},\"id\":\"bnnhadoocrkvcik\",\"name\":\"nvpamq\",\"type\":\"x\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.privateEndpointConnections().listByCluster("ulpiuj", "aasipqi", com.azure.core.util.Context.NONE); - - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.APPROVED, - response.iterator().next().privateLinkServiceConnectionState().status()); - Assertions - .assertEquals( - "tkuwhhmhykojo", response.iterator().next().privateLinkServiceConnectionState().description()); - Assertions - .assertEquals("fnndl", response.iterator().next().privateLinkServiceConnectionState().actionsRequired()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.privateEndpointConnections() + .listByCluster("slyzrpzbchckqq", "qioxi", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, + response.iterator().next().privateLinkServiceConnectionState().status()); + Assertions.assertEquals("lkdmtncvokotllxd", + response.iterator().next().privateLinkServiceConnectionState().description()); + Assertions.assertEquals("gsyocogj", + response.iterator().next().privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointPropertyTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointPropertyTests.java index a7d0971439f82..21ffe199bb03b 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointPropertyTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointPropertyTests.java @@ -11,8 +11,8 @@ public final class PrivateEndpointPropertyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointProperty model = - BinaryData.fromString("{\"id\":\"sarhmofc\"}").toObject(PrivateEndpointProperty.class); + PrivateEndpointProperty model + = BinaryData.fromString("{\"id\":\"sarhmofc\"}").toObject(PrivateEndpointProperty.class); Assertions.assertEquals("sarhmofc", model.id()); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointTests.java index e5dcb3e362257..9c2db1c70f191 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateEndpointTests.java @@ -10,7 +10,7 @@ public final class PrivateEndpointTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpoint model = BinaryData.fromString("{\"id\":\"pikad\"}").toObject(PrivateEndpoint.class); + PrivateEndpoint model = BinaryData.fromString("{\"id\":\"tyq\"}").toObject(PrivateEndpoint.class); } @org.junit.jupiter.api.Test diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceInnerTests.java index 97e2a9c5e0ae5..a22bc26e5c99e 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceInnerTests.java @@ -12,18 +12,17 @@ public final class PrivateLinkResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkResourceInner model = - BinaryData - .fromString( - "{\"properties\":{\"groupId\":\"xdbabphlwr\",\"requiredMembers\":[\"ktsthsucocmny\"],\"requiredZoneNames\":[\"t\"]},\"id\":\"twwrqp\",\"name\":\"edckzywbiexzfey\",\"type\":\"eaxib\"}") - .toObject(PrivateLinkResourceInner.class); - Assertions.assertEquals("t", model.requiredZoneNames().get(0)); + PrivateLinkResourceInner model = BinaryData.fromString( + "{\"properties\":{\"groupId\":\"lfplp\",\"requiredMembers\":[\"uscrpabgyepsb\"],\"requiredZoneNames\":[\"zq\",\"gxywpmue\",\"fjz\",\"fqkquj\"]},\"id\":\"suyonobglaocq\",\"name\":\"tcc\",\"type\":\"g\"}") + .toObject(PrivateLinkResourceInner.class); + Assertions.assertEquals("zq", model.requiredZoneNames().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkResourceInner model = new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("t")); + PrivateLinkResourceInner model + = new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("zq", "gxywpmue", "fjz", "fqkquj")); model = BinaryData.fromObject(model).toObject(PrivateLinkResourceInner.class); - Assertions.assertEquals("t", model.requiredZoneNames().get(0)); + Assertions.assertEquals("zq", model.requiredZoneNames().get(0)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceListResultTests.java index 787502ef4222f..5c94c92e92e83 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourceListResultTests.java @@ -13,24 +13,19 @@ public final class PrivateLinkResourceListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkResourceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"groupId\":\"agnb\",\"requiredMembers\":[\"hijggme\",\"fsiarbutr\",\"vpnazzm\"],\"requiredZoneNames\":[\"unmpxttd\",\"hrbnlankxmyskpbh\"]},\"id\":\"btkcxywnytnrsyn\",\"name\":\"qidybyx\",\"type\":\"zfcl\"}]}") - .toObject(PrivateLinkResourceListResult.class); - Assertions.assertEquals("unmpxttd", model.value().get(0).requiredZoneNames().get(0)); + PrivateLinkResourceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"groupId\":\"ovplw\",\"requiredMembers\":[\"vgyuguos\",\"mkfssxqukkfplgm\",\"sxnkjzkdeslpvlo\"],\"requiredZoneNames\":[\"yighxpk\"]},\"id\":\"zb\",\"name\":\"iuebbaumny\",\"type\":\"upedeojnabckhs\"},{\"properties\":{\"groupId\":\"psiebtfhvpes\",\"requiredMembers\":[\"krdqmh\",\"jdhtldwkyzxu\",\"tkncwsc\"],\"requiredZoneNames\":[\"lxotogtwrupq\",\"xvnmicykvceov\",\"ilovnot\",\"fj\"]},\"id\":\"njbkcnxdhbttkph\",\"name\":\"wpn\",\"type\":\"jtoqne\"}]}") + .toObject(PrivateLinkResourceListResult.class); + Assertions.assertEquals("yighxpk", model.value().get(0).requiredZoneNames().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkResourceListResult model = - new PrivateLinkResourceListResult() - .withValue( - Arrays - .asList( - new PrivateLinkResourceInner() - .withRequiredZoneNames(Arrays.asList("unmpxttd", "hrbnlankxmyskpbh")))); + PrivateLinkResourceListResult model = new PrivateLinkResourceListResult() + .withValue(Arrays.asList(new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("yighxpk")), + new PrivateLinkResourceInner() + .withRequiredZoneNames(Arrays.asList("lxotogtwrupq", "xvnmicykvceov", "ilovnot", "fj")))); model = BinaryData.fromObject(model).toObject(PrivateLinkResourceListResult.class); - Assertions.assertEquals("unmpxttd", model.value().get(0).requiredZoneNames().get(0)); + Assertions.assertEquals("yighxpk", model.value().get(0).requiredZoneNames().get(0)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcePropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcePropertiesTests.java index 48ebc5326d636..7a37224ea2bc0 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcePropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcePropertiesTests.java @@ -12,19 +12,17 @@ public final class PrivateLinkResourcePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkResourceProperties model = - BinaryData - .fromString( - "{\"groupId\":\"jwbhqwalmuz\",\"requiredMembers\":[\"aepdkzjanc\",\"xrhdwbavxbniwdjs\",\"zt\",\"dbpgnxytxhp\"],\"requiredZoneNames\":[\"zpfzabglc\",\"hxw\"]}") - .toObject(PrivateLinkResourceProperties.class); - Assertions.assertEquals("zpfzabglc", model.requiredZoneNames().get(0)); + PrivateLinkResourceProperties model = BinaryData.fromString( + "{\"groupId\":\"dxyt\",\"requiredMembers\":[\"yrxvwfudwpznt\",\"hdzhlrqj\",\"hckfrlhrx\",\"bkyvp\"],\"requiredZoneNames\":[\"n\",\"z\",\"p\",\"kafkuwbcrnwbm\"]}") + .toObject(PrivateLinkResourceProperties.class); + Assertions.assertEquals("n", model.requiredZoneNames().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkResourceProperties model = - new PrivateLinkResourceProperties().withRequiredZoneNames(Arrays.asList("zpfzabglc", "hxw")); + PrivateLinkResourceProperties model + = new PrivateLinkResourceProperties().withRequiredZoneNames(Arrays.asList("n", "z", "p", "kafkuwbcrnwbm")); model = BinaryData.fromObject(model).toObject(PrivateLinkResourceProperties.class); - Assertions.assertEquals("zpfzabglc", model.requiredZoneNames().get(0)); + Assertions.assertEquals("n", model.requiredZoneNames().get(0)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetWithResponseMockTests.java index e6c3a780c3351..51dabaf611e45 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesGetWithResponseMockTests.java @@ -30,41 +30,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"groupId\":\"ikf\",\"requiredMembers\":[\"n\",\"a\"],\"requiredZoneNames\":[\"wczelpci\",\"elsfeaen\",\"abfatkl\"]},\"id\":\"xbjhwuaanozjosph\",\"name\":\"oulpjrv\",\"type\":\"ag\"}"; + String responseStr + = "{\"properties\":{\"groupId\":\"nwnwme\",\"requiredMembers\":[\"yyceuzsoi\",\"judpfrxt\",\"thzvaytdwkqbrqu\",\"paxh\"],\"requiredZoneNames\":[\"i\",\"ivpdtiir\",\"tdqoaxoruzfgsq\",\"yfxrx\"]},\"id\":\"eptra\",\"name\":\"xje\",\"type\":\"wlwnwxuqlcv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PrivateLinkResource response = - manager - .privateLinkResources() - .getWithResponse("imfnjhfjx", "mszkkfo", "rey", com.azure.core.util.Context.NONE) - .getValue(); + PrivateLinkResource response = manager.privateLinkResources() + .getWithResponse("ayffim", "zrtuzq", "gsexne", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("wczelpci", response.requiredZoneNames().get(0)); + Assertions.assertEquals("i", response.requiredZoneNames().get(0)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterMockTests.java index 61980b963223d..fe0eef0d89eb2 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkResourcesListByClusterMockTests.java @@ -31,38 +31,27 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"groupId\":\"piccjzkzivgv\",\"requiredMembers\":[\"ayrhyrnx\",\"mueedndrdvstk\",\"qqtch\",\"alm\"],\"requiredZoneNames\":[\"d\",\"aygdvwvgpioh\",\"wxrt\"]},\"id\":\"dxepxgyq\",\"name\":\"gvr\",\"type\":\"mnpkukghimdblxg\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"groupId\":\"oduhp\",\"requiredMembers\":[\"gymare\",\"n\",\"jxqugjhky\",\"ubeddg\"],\"requiredZoneNames\":[\"fwqmzqalkrmn\"]},\"id\":\"jpxac\",\"name\":\"qudf\",\"type\":\"byxbaaabjy\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.privateLinkResources().listByCluster("zikywgg", "kallatmel", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.privateLinkResources().listByCluster("wdmjsjqbjhhyx", "rw", com.azure.core.util.Context.NONE); - Assertions.assertEquals("d", response.iterator().next().requiredZoneNames().get(0)); + Assertions.assertEquals("fwqmzqalkrmn", response.iterator().next().requiredZoneNames().get(0)); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkServiceConnectionStateTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkServiceConnectionStateTests.java index ce419c04369ba..9f822e1b45247 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkServiceConnectionStateTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PrivateLinkServiceConnectionStateTests.java @@ -12,11 +12,9 @@ public final class PrivateLinkServiceConnectionStateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkServiceConnectionState model = - BinaryData - .fromString( - "{\"status\":\"Approved\",\"description\":\"yurkdtmlxhekuksj\",\"actionsRequired\":\"ukcdmparcryuanzw\"}") - .toObject(PrivateLinkServiceConnectionState.class); + PrivateLinkServiceConnectionState model = BinaryData.fromString( + "{\"status\":\"Approved\",\"description\":\"yurkdtmlxhekuksj\",\"actionsRequired\":\"ukcdmparcryuanzw\"}") + .toObject(PrivateLinkServiceConnectionState.class); Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, model.status()); Assertions.assertEquals("yurkdtmlxhekuksj", model.description()); Assertions.assertEquals("ukcdmparcryuanzw", model.actionsRequired()); @@ -24,11 +22,9 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkServiceConnectionState model = - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) - .withDescription("yurkdtmlxhekuksj") - .withActionsRequired("ukcdmparcryuanzw"); + PrivateLinkServiceConnectionState model + = new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("yurkdtmlxhekuksj").withActionsRequired("ukcdmparcryuanzw"); model = BinaryData.fromObject(model).toObject(PrivateLinkServiceConnectionState.class); Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, model.status()); Assertions.assertEquals("yurkdtmlxhekuksj", model.description()); diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PromoteRequestTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PromoteRequestTests.java new file mode 100644 index 0000000000000..9fb0bc9762547 --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/PromoteRequestTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PromoteRequest; +import org.junit.jupiter.api.Assertions; + +public final class PromoteRequestTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PromoteRequest model = BinaryData.fromString("{\"enableGeoBackup\":false}").toObject(PromoteRequest.class); + Assertions.assertEquals(false, model.enableGeoBackup()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PromoteRequest model = new PromoteRequest().withEnableGeoBackup(false); + model = BinaryData.fromObject(model).toObject(PromoteRequest.class); + Assertions.assertEquals(false, model.enableGeoBackup()); + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolePropertiesExternalIdentityTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolePropertiesExternalIdentityTests.java new file mode 100644 index 0000000000000..b5e217cce81c4 --- /dev/null +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/RolePropertiesExternalIdentityTests.java @@ -0,0 +1,32 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.cosmosdbforpostgresql.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.cosmosdbforpostgresql.fluent.models.RolePropertiesExternalIdentity; +import com.azure.resourcemanager.cosmosdbforpostgresql.models.PrincipalType; +import org.junit.jupiter.api.Assertions; + +public final class RolePropertiesExternalIdentityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RolePropertiesExternalIdentity model = BinaryData + .fromString("{\"objectId\":\"vvnchrkcc\",\"principalType\":\"user\",\"tenantId\":\"zjuqkhrsaj\"}") + .toObject(RolePropertiesExternalIdentity.class); + Assertions.assertEquals("vvnchrkcc", model.objectId()); + Assertions.assertEquals(PrincipalType.USER, model.principalType()); + Assertions.assertEquals("zjuqkhrsaj", model.tenantId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RolePropertiesExternalIdentity model = new RolePropertiesExternalIdentity().withObjectId("vvnchrkcc") + .withPrincipalType(PrincipalType.USER).withTenantId("zjuqkhrsaj"); + model = BinaryData.fromObject(model).toObject(RolePropertiesExternalIdentity.class); + Assertions.assertEquals("vvnchrkcc", model.objectId()); + Assertions.assertEquals(PrincipalType.USER, model.principalType()); + Assertions.assertEquals("zjuqkhrsaj", model.tenantId()); + } +} diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationInnerTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationInnerTests.java index 4e03d99e19215..bc2fea8e7219e 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationInnerTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationInnerTests.java @@ -11,18 +11,16 @@ public final class ServerConfigurationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServerConfigurationInner model = - BinaryData - .fromString( - "{\"properties\":{\"value\":\"umkdosvqwhbmd\",\"source\":\"bjf\",\"description\":\"gmbmbexppbh\",\"defaultValue\":\"qrolfpf\",\"dataType\":\"Enumeration\",\"allowedValues\":\"gbquxigj\",\"requiresRestart\":true,\"provisioningState\":\"InProgress\"},\"id\":\"o\",\"name\":\"fhrtxilnerkujysv\",\"type\":\"eju\"}") - .toObject(ServerConfigurationInner.class); - Assertions.assertEquals("umkdosvqwhbmd", model.value()); + ServerConfigurationInner model = BinaryData.fromString( + "{\"properties\":{\"value\":\"zafb\",\"source\":\"j\",\"description\":\"btoqcjmkljavbqid\",\"defaultValue\":\"ajzyul\",\"dataType\":\"Boolean\",\"allowedValues\":\"jkrlkhbzhfepg\",\"requiresRestart\":false,\"provisioningState\":\"InProgress\"},\"id\":\"locx\",\"name\":\"c\",\"type\":\"aierhhb\"}") + .toObject(ServerConfigurationInner.class); + Assertions.assertEquals("zafb", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServerConfigurationInner model = new ServerConfigurationInner().withValue("umkdosvqwhbmd"); + ServerConfigurationInner model = new ServerConfigurationInner().withValue("zafb"); model = BinaryData.fromObject(model).toObject(ServerConfigurationInner.class); - Assertions.assertEquals("umkdosvqwhbmd", model.value()); + Assertions.assertEquals("zafb", model.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationListResultTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationListResultTests.java index 6142a9f7cd228..97e8bdd47b616 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationListResultTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationListResultTests.java @@ -13,26 +13,19 @@ public final class ServerConfigurationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServerConfigurationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"value\":\"ispe\",\"source\":\"tzfkufubl\",\"description\":\"fxqeof\",\"defaultValue\":\"e\",\"dataType\":\"Integer\",\"allowedValues\":\"jbasvmsmjqulngs\",\"requiresRestart\":true,\"provisioningState\":\"Canceled\"},\"id\":\"kzgcwrwcl\",\"name\":\"xwrljdouskcqvkoc\",\"type\":\"cjdkwtnhxbnjbi\"},{\"properties\":{\"value\":\"qrglssainqpjwn\",\"source\":\"ljfmppee\",\"description\":\"mgxsab\",\"defaultValue\":\"qduujitcjczdz\",\"dataType\":\"Numeric\",\"allowedValues\":\"hkr\",\"requiresRestart\":false,\"provisioningState\":\"InProgress\"},\"id\":\"dsbdkvwrwjf\",\"name\":\"usnhutje\",\"type\":\"tmrldhugjzzdatq\"},{\"properties\":{\"value\":\"oc\",\"source\":\"eablg\",\"description\":\"uticndvkaozwyif\",\"defaultValue\":\"hxh\",\"dataType\":\"Enumeration\",\"allowedValues\":\"ftyxolniw\",\"requiresRestart\":false,\"provisioningState\":\"Canceled\"},\"id\":\"fkgiawxk\",\"name\":\"ryplwckbasyypn\",\"type\":\"dhsgcba\"},{\"properties\":{\"value\":\"hejkotynqgou\",\"source\":\"ndlik\",\"description\":\"qkgfgibma\",\"defaultValue\":\"akeqs\",\"dataType\":\"Boolean\",\"allowedValues\":\"zqqedq\",\"requiresRestart\":true,\"provisioningState\":\"InProgress\"},\"id\":\"fouflmmnkzsmo\",\"name\":\"mglougpbkw\",\"type\":\"mutduqktaps\"}],\"nextLink\":\"gcue\"}") - .toObject(ServerConfigurationListResult.class); - Assertions.assertEquals("ispe", model.value().get(0).value()); + ServerConfigurationListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"value\":\"duqkt\",\"source\":\"spwgcuertumkdosv\",\"description\":\"hbmdgbbjfdd\",\"defaultValue\":\"bmbexppbhtqqro\",\"dataType\":\"Boolean\",\"allowedValues\":\"psalgbqux\",\"requiresRestart\":false,\"provisioningState\":\"InProgress\"},\"id\":\"zjaoyfhrtxil\",\"name\":\"erkujys\",\"type\":\"l\"},{\"properties\":{\"value\":\"uvfqawrlyxwj\",\"source\":\"prbnwbxgjvtbv\",\"description\":\"sszdnru\",\"defaultValue\":\"guhmuouqfpr\",\"dataType\":\"Integer\",\"allowedValues\":\"nguitnwuizgazxu\",\"requiresRestart\":false,\"provisioningState\":\"Canceled\"},\"id\":\"y\",\"name\":\"i\",\"type\":\"rfidfvzwdz\"},{\"properties\":{\"value\":\"tymw\",\"source\":\"dkfthwxmnt\",\"description\":\"waopvkmijcmmxd\",\"defaultValue\":\"fufsrpymzi\",\"dataType\":\"Numeric\",\"allowedValues\":\"zcxtbzsgfyccsn\",\"requiresRestart\":true,\"provisioningState\":\"Canceled\"},\"id\":\"jeiachboosfl\",\"name\":\"ro\",\"type\":\"fqpte\"},{\"properties\":{\"value\":\"zzvypyqrimzinp\",\"source\":\"wjdk\",\"description\":\"soodqxhcrmnoh\",\"defaultValue\":\"ckwhds\",\"dataType\":\"Enumeration\",\"allowedValues\":\"yip\",\"requiresRestart\":false,\"provisioningState\":\"InProgress\"},\"id\":\"gr\",\"name\":\"bznorcjxvsnby\",\"type\":\"qabnmoc\"}],\"nextLink\":\"ysh\"}") + .toObject(ServerConfigurationListResult.class); + Assertions.assertEquals("duqkt", model.value().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServerConfigurationListResult model = - new ServerConfigurationListResult() - .withValue( - Arrays - .asList( - new ServerConfigurationInner().withValue("ispe"), - new ServerConfigurationInner().withValue("qrglssainqpjwn"), - new ServerConfigurationInner().withValue("oc"), - new ServerConfigurationInner().withValue("hejkotynqgou"))); + ServerConfigurationListResult model = new ServerConfigurationListResult().withValue(Arrays.asList( + new ServerConfigurationInner().withValue("duqkt"), new ServerConfigurationInner().withValue("uvfqawrlyxwj"), + new ServerConfigurationInner().withValue("tymw"), + new ServerConfigurationInner().withValue("zzvypyqrimzinp"))); model = BinaryData.fromObject(model).toObject(ServerConfigurationListResult.class); - Assertions.assertEquals("ispe", model.value().get(0).value()); + Assertions.assertEquals("duqkt", model.value().get(0).value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationPropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationPropertiesTests.java index 2f22cf45cd9cf..389b070b5d550 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationPropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerConfigurationPropertiesTests.java @@ -11,18 +11,16 @@ public final class ServerConfigurationPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServerConfigurationProperties model = - BinaryData - .fromString( - "{\"value\":\"fqawrlyxw\",\"source\":\"cpr\",\"description\":\"wbxgjvt\",\"defaultValue\":\"p\",\"dataType\":\"Numeric\",\"allowedValues\":\"dnrujqguhmuouqfp\",\"requiresRestart\":false,\"provisioningState\":\"Canceled\"}") - .toObject(ServerConfigurationProperties.class); - Assertions.assertEquals("fqawrlyxw", model.value()); + ServerConfigurationProperties model = BinaryData.fromString( + "{\"value\":\"sglumma\",\"source\":\"j\",\"description\":\"dxob\",\"defaultValue\":\"dxkqpx\",\"dataType\":\"Enumeration\",\"allowedValues\":\"ionpimexg\",\"requiresRestart\":false,\"provisioningState\":\"Succeeded\"}") + .toObject(ServerConfigurationProperties.class); + Assertions.assertEquals("sglumma", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServerConfigurationProperties model = new ServerConfigurationProperties().withValue("fqawrlyxw"); + ServerConfigurationProperties model = new ServerConfigurationProperties().withValue("sglumma"); model = BinaryData.fromObject(model).toObject(ServerConfigurationProperties.class); - Assertions.assertEquals("fqawrlyxw", model.value()); + Assertions.assertEquals("sglumma", model.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerNameItemTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerNameItemTests.java index fcc94da87c39b..9d9356a7355f8 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerNameItemTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerNameItemTests.java @@ -11,9 +11,8 @@ public final class ServerNameItemTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServerNameItem model = - BinaryData - .fromString("{\"name\":\"fpownoizhwlr\",\"fullyQualifiedDomainName\":\"bqsoqijg\"}") + ServerNameItem model + = BinaryData.fromString("{\"name\":\"fpownoizhwlr\",\"fullyQualifiedDomainName\":\"bqsoqijg\"}") .toObject(ServerNameItem.class); Assertions.assertEquals("fpownoizhwlr", model.name()); } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerPropertiesTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerPropertiesTests.java index f3627203cde0c..9eb49a00a7924 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerPropertiesTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerPropertiesTests.java @@ -11,29 +11,23 @@ public final class ServerPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServerProperties model = - BinaryData - .fromString( - "{\"serverEdition\":\"yevc\",\"storageQuotaInMb\":308422624,\"vCores\":284710625,\"enableHa\":true,\"enablePublicIpAccess\":false,\"isReadOnly\":false,\"administratorLogin\":\"jzrnf\"}") - .toObject(ServerProperties.class); - Assertions.assertEquals("yevc", model.serverEdition()); - Assertions.assertEquals(308422624, model.storageQuotaInMb()); - Assertions.assertEquals(284710625, model.vCores()); - Assertions.assertEquals(true, model.enableHa()); + ServerProperties model = BinaryData.fromString( + "{\"serverEdition\":\"bciqfouflm\",\"storageQuotaInMb\":1828393347,\"vCores\":492225371,\"enableHa\":false,\"enablePublicIpAccess\":true,\"isReadOnly\":false,\"administratorLogin\":\"ugpbkw\"}") + .toObject(ServerProperties.class); + Assertions.assertEquals("bciqfouflm", model.serverEdition()); + Assertions.assertEquals(1828393347, model.storageQuotaInMb()); + Assertions.assertEquals(492225371, model.vCores()); + Assertions.assertEquals(false, model.enableHa()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServerProperties model = - new ServerProperties() - .withServerEdition("yevc") - .withStorageQuotaInMb(308422624) - .withVCores(284710625) - .withEnableHa(true); + ServerProperties model = new ServerProperties().withServerEdition("bciqfouflm").withStorageQuotaInMb(1828393347) + .withVCores(492225371).withEnableHa(false); model = BinaryData.fromObject(model).toObject(ServerProperties.class); - Assertions.assertEquals("yevc", model.serverEdition()); - Assertions.assertEquals(308422624, model.storageQuotaInMb()); - Assertions.assertEquals(284710625, model.vCores()); - Assertions.assertEquals(true, model.enableHa()); + Assertions.assertEquals("bciqfouflm", model.serverEdition()); + Assertions.assertEquals(1828393347, model.storageQuotaInMb()); + Assertions.assertEquals(492225371, model.vCores()); + Assertions.assertEquals(false, model.enableHa()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerRoleGroupConfigurationTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerRoleGroupConfigurationTests.java index 3aad6504c2154..b1ade11817c11 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerRoleGroupConfigurationTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServerRoleGroupConfigurationTests.java @@ -12,21 +12,19 @@ public final class ServerRoleGroupConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServerRoleGroupConfiguration model = - BinaryData - .fromString( - "{\"role\":\"Worker\",\"value\":\"imexgstxgcpodgma\",\"defaultValue\":\"r\",\"source\":\"djwzrlov\"}") - .toObject(ServerRoleGroupConfiguration.class); - Assertions.assertEquals(ServerRole.WORKER, model.role()); - Assertions.assertEquals("imexgstxgcpodgma", model.value()); + ServerRoleGroupConfiguration model = BinaryData.fromString( + "{\"role\":\"Coordinator\",\"value\":\"oaeupfhyhltrpmo\",\"defaultValue\":\"mcmatuokthfuiu\",\"source\":\"dsfcpkvxodpuoz\"}") + .toObject(ServerRoleGroupConfiguration.class); + Assertions.assertEquals(ServerRole.COORDINATOR, model.role()); + Assertions.assertEquals("oaeupfhyhltrpmo", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServerRoleGroupConfiguration model = - new ServerRoleGroupConfiguration().withRole(ServerRole.WORKER).withValue("imexgstxgcpodgma"); + ServerRoleGroupConfiguration model + = new ServerRoleGroupConfiguration().withRole(ServerRole.COORDINATOR).withValue("oaeupfhyhltrpmo"); model = BinaryData.fromObject(model).toObject(ServerRoleGroupConfiguration.class); - Assertions.assertEquals(ServerRole.WORKER, model.role()); - Assertions.assertEquals("imexgstxgcpodgma", model.value()); + Assertions.assertEquals(ServerRole.COORDINATOR, model.role()); + Assertions.assertEquals("oaeupfhyhltrpmo", model.value()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetWithResponseMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetWithResponseMockTests.java index 1cd68c1566036..a8c63dde96140 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetWithResponseMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersGetWithResponseMockTests.java @@ -31,48 +31,35 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"fullyQualifiedDomainName\":\"laexqp\",\"role\":\"Coordinator\",\"state\":\"mwsrcrgvxpvgo\",\"haState\":\"lf\",\"availabilityZone\":\"sgwbnbbeld\",\"postgresqlVersion\":\"k\",\"citusVersion\":\"ali\",\"serverEdition\":\"rqhakauha\",\"storageQuotaInMb\":798376166,\"vCores\":706986266,\"enableHa\":false,\"enablePublicIpAccess\":false,\"isReadOnly\":false,\"administratorLogin\":\"cugicjoox\"},\"id\":\"ebwpucwwfvo\",\"name\":\"bvmeuecivy\",\"type\":\"zceuojgjrw\"}"; + String responseStr + = "{\"properties\":{\"fullyQualifiedDomainName\":\"xrmcqibycnojvk\",\"role\":\"Coordinator\",\"state\":\"qsgzvahapj\",\"haState\":\"hpvgqz\",\"availabilityZone\":\"rvxdjzlmw\",\"postgresqlVersion\":\"kvugfhzovawjvzun\",\"citusVersion\":\"thnnpr\",\"serverEdition\":\"i\",\"storageQuotaInMb\":1381337865,\"vCores\":1073043267,\"enableHa\":false,\"enablePublicIpAccess\":false,\"isReadOnly\":true,\"administratorLogin\":\"dultskz\"},\"id\":\"tdzumveekgpw\",\"name\":\"zuhkfpbsjyof\",\"type\":\"xl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - ClusterServer response = - manager - .servers() - .getWithResponse("ofmxagkvtmelmqkr", "ahvljuaha", "uhcdhm", com.azure.core.util.Context.NONE) - .getValue(); + ClusterServer response = manager.servers() + .getWithResponse("mtsavjcbpwxqp", "rknftguvriuhprwm", "yvxqtayriwwroy", com.azure.core.util.Context.NONE) + .getValue(); Assertions.assertEquals(ServerRole.COORDINATOR, response.role()); - Assertions.assertEquals("sgwbnbbeld", response.availabilityZone()); - Assertions.assertEquals("k", response.postgresqlVersion()); - Assertions.assertEquals("ali", response.citusVersion()); - Assertions.assertEquals("rqhakauha", response.serverEdition()); - Assertions.assertEquals(798376166, response.storageQuotaInMb()); - Assertions.assertEquals(706986266, response.vCores()); + Assertions.assertEquals("rvxdjzlmw", response.availabilityZone()); + Assertions.assertEquals("kvugfhzovawjvzun", response.postgresqlVersion()); + Assertions.assertEquals("thnnpr", response.citusVersion()); + Assertions.assertEquals("i", response.serverEdition()); + Assertions.assertEquals(1381337865, response.storageQuotaInMb()); + Assertions.assertEquals(1073043267, response.vCores()); Assertions.assertEquals(false, response.enableHa()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterMockTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterMockTests.java index 5a50d4c1101d5..6fa8929c30be1 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterMockTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/ServersListByClusterMockTests.java @@ -32,45 +32,34 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"fullyQualifiedDomainName\":\"kqujidsuyono\",\"role\":\"Worker\",\"state\":\"ocqxtccmg\",\"haState\":\"dxyt\",\"availabilityZone\":\"oyrxvwfudwpzntxh\",\"postgresqlVersion\":\"hl\",\"citusVersion\":\"jbhckfrlhr\",\"serverEdition\":\"bkyvp\",\"storageQuotaInMb\":1219489836,\"vCores\":427278595,\"enableHa\":true,\"enablePublicIpAccess\":false,\"isReadOnly\":false,\"administratorLogin\":\"kuwbcrnwb\"},\"id\":\"hhseyv\",\"name\":\"us\",\"type\":\"tslhspkdeem\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"fullyQualifiedDomainName\":\"btdhxujznbm\",\"role\":\"Worker\",\"state\":\"wpr\",\"haState\":\"lve\",\"availabilityZone\":\"lupj\",\"postgresqlVersion\":\"hfxobbcswsrtj\",\"citusVersion\":\"plrbpbewtghf\",\"serverEdition\":\"lcgwxzvlvqh\",\"storageQuotaInMb\":1823532721,\"vCores\":1432172672,\"enableHa\":false,\"enablePublicIpAccess\":true,\"isReadOnly\":true,\"administratorLogin\":\"ebwwaloayqc\"},\"id\":\"rtzju\",\"name\":\"gwyzm\",\"type\":\"txon\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - CosmosDBForPostgreSqlManager manager = - CosmosDBForPostgreSqlManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + CosmosDBForPostgreSqlManager manager = CosmosDBForPostgreSqlManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.servers().listByCluster("gxywpmue", "fjz", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.servers().listByCluster("kxfbkpycgklwndn", "jdauwhvy", com.azure.core.util.Context.NONE); Assertions.assertEquals(ServerRole.WORKER, response.iterator().next().role()); - Assertions.assertEquals("oyrxvwfudwpzntxh", response.iterator().next().availabilityZone()); - Assertions.assertEquals("hl", response.iterator().next().postgresqlVersion()); - Assertions.assertEquals("jbhckfrlhr", response.iterator().next().citusVersion()); - Assertions.assertEquals("bkyvp", response.iterator().next().serverEdition()); - Assertions.assertEquals(1219489836, response.iterator().next().storageQuotaInMb()); - Assertions.assertEquals(427278595, response.iterator().next().vCores()); - Assertions.assertEquals(true, response.iterator().next().enableHa()); + Assertions.assertEquals("lupj", response.iterator().next().availabilityZone()); + Assertions.assertEquals("hfxobbcswsrtj", response.iterator().next().postgresqlVersion()); + Assertions.assertEquals("plrbpbewtghf", response.iterator().next().citusVersion()); + Assertions.assertEquals("lcgwxzvlvqh", response.iterator().next().serverEdition()); + Assertions.assertEquals(1823532721, response.iterator().next().storageQuotaInMb()); + Assertions.assertEquals(1432172672, response.iterator().next().vCores()); + Assertions.assertEquals(false, response.iterator().next().enableHa()); } } diff --git a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/SimplePrivateEndpointConnectionTests.java b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/SimplePrivateEndpointConnectionTests.java index 49a5955c5d3c3..6271464e19bca 100644 --- a/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/SimplePrivateEndpointConnectionTests.java +++ b/sdk/cosmosdbforpostgresql/azure-resourcemanager-cosmosdbforpostgresql/src/test/java/com/azure/resourcemanager/cosmosdbforpostgresql/generated/SimplePrivateEndpointConnectionTests.java @@ -15,37 +15,29 @@ public final class SimplePrivateEndpointConnectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimplePrivateEndpointConnection model = - BinaryData - .fromString( - "{\"properties\":{\"privateEndpoint\":{\"id\":\"pazlobcufpdz\"},\"groupIds\":[\"t\",\"qqjnqgl\"],\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"foooj\",\"actionsRequired\":\"ifsqesaagdfmg\"}},\"id\":\"lhjxr\",\"name\":\"f\",\"type\":\"wmrvktsizntocipa\"}") - .toObject(SimplePrivateEndpointConnection.class); + SimplePrivateEndpointConnection model = BinaryData.fromString( + "{\"properties\":{\"privateEndpoint\":{\"id\":\"pazlobcufpdz\"},\"groupIds\":[\"t\",\"qqjnqgl\"],\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"foooj\",\"actionsRequired\":\"ifsqesaagdfmg\"}},\"id\":\"lhjxr\",\"name\":\"f\",\"type\":\"wmrvktsizntocipa\"}") + .toObject(SimplePrivateEndpointConnection.class); Assertions.assertEquals("pazlobcufpdz", model.privateEndpoint().id()); Assertions.assertEquals("t", model.groupIds().get(0)); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.PENDING, model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, + model.privateLinkServiceConnectionState().status()); Assertions.assertEquals("foooj", model.privateLinkServiceConnectionState().description()); Assertions.assertEquals("ifsqesaagdfmg", model.privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimplePrivateEndpointConnection model = - new SimplePrivateEndpointConnection() - .withPrivateEndpoint(new PrivateEndpointProperty().withId("pazlobcufpdz")) - .withGroupIds(Arrays.asList("t", "qqjnqgl")) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.PENDING) - .withDescription("foooj") - .withActionsRequired("ifsqesaagdfmg")); + SimplePrivateEndpointConnection model = new SimplePrivateEndpointConnection() + .withPrivateEndpoint(new PrivateEndpointProperty().withId("pazlobcufpdz")) + .withGroupIds(Arrays.asList("t", "qqjnqgl")).withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.PENDING) + .withDescription("foooj").withActionsRequired("ifsqesaagdfmg")); model = BinaryData.fromObject(model).toObject(SimplePrivateEndpointConnection.class); Assertions.assertEquals("pazlobcufpdz", model.privateEndpoint().id()); Assertions.assertEquals("t", model.groupIds().get(0)); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.PENDING, model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, + model.privateLinkServiceConnectionState().status()); Assertions.assertEquals("foooj", model.privateLinkServiceConnectionState().description()); Assertions.assertEquals("ifsqesaagdfmg", model.privateLinkServiceConnectionState().actionsRequired()); } diff --git a/sdk/cosmosdbforpostgresql/tests.mgmt.yml b/sdk/cosmosdbforpostgresql/tests.mgmt.yml index 99883b7454090..f396c10941268 100644 --- a/sdk/cosmosdbforpostgresql/tests.mgmt.yml +++ b/sdk/cosmosdbforpostgresql/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: cosmosdbforpostgresql - Artifacts: - - name: azure-resourcemanager-cosmosdbforpostgresql - groupId: com.azure.resourcemanager - safeName: azureresourcemanagercosmosdbforpostgresql - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: cosmosdbforpostgresql + Artifacts: + - name: azure-resourcemanager-cosmosdbforpostgresql + groupId: com.azure.resourcemanager + safeName: azureresourcemanagercosmosdbforpostgresql + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/costmanagement/azure-resourcemanager-costmanagement/pom.xml b/sdk/costmanagement/azure-resourcemanager-costmanagement/pom.xml index a970c4f5e68ed..41f0e3cc8953c 100644 --- a/sdk/costmanagement/azure-resourcemanager-costmanagement/pom.xml +++ b/sdk/costmanagement/azure-resourcemanager-costmanagement/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/dashboard/azure-resourcemanager-dashboard/pom.xml b/sdk/dashboard/azure-resourcemanager-dashboard/pom.xml index 7ca4f27327c01..ae8d83e17ad97 100644 --- a/sdk/dashboard/azure-resourcemanager-dashboard/pom.xml +++ b/sdk/dashboard/azure-resourcemanager-dashboard/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/dashboard/tests.mgmt.yml b/sdk/dashboard/tests.mgmt.yml index 27b68d054a31a..f72a7aaa15b91 100644 --- a/sdk/dashboard/tests.mgmt.yml +++ b/sdk/dashboard/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: dashboard - Artifacts: - - name: azure-resourcemanager-dashboard - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdashboard - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: dashboard + Artifacts: + - name: azure-resourcemanager-dashboard + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdashboard + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/databox/azure-resourcemanager-databox/pom.xml b/sdk/databox/azure-resourcemanager-databox/pom.xml index 65d11e3879199..e5c3d5d46b090 100644 --- a/sdk/databox/azure-resourcemanager-databox/pom.xml +++ b/sdk/databox/azure-resourcemanager-databox/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/databoxedge/azure-resourcemanager-databoxedge/pom.xml b/sdk/databoxedge/azure-resourcemanager-databoxedge/pom.xml index e9b768f18d7ca..56fee29364980 100644 --- a/sdk/databoxedge/azure-resourcemanager-databoxedge/pom.xml +++ b/sdk/databoxedge/azure-resourcemanager-databoxedge/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/databricks/azure-resourcemanager-databricks/pom.xml b/sdk/databricks/azure-resourcemanager-databricks/pom.xml index 4eedf3729891f..cdc8c9b6faae5 100644 --- a/sdk/databricks/azure-resourcemanager-databricks/pom.xml +++ b/sdk/databricks/azure-resourcemanager-databricks/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/databricks/tests.mgmt.yml b/sdk/databricks/tests.mgmt.yml index 320750245f010..845b968f647b5 100644 --- a/sdk/databricks/tests.mgmt.yml +++ b/sdk/databricks/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: databricks - Artifacts: - - name: azure-resourcemanager-databricks - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdatabricks - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: databricks + Artifacts: + - name: azure-resourcemanager-databricks + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdatabricks + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/datadog/azure-resourcemanager-datadog/pom.xml b/sdk/datadog/azure-resourcemanager-datadog/pom.xml index b99cd3b77ee84..99406efc5da9e 100644 --- a/sdk/datadog/azure-resourcemanager-datadog/pom.xml +++ b/sdk/datadog/azure-resourcemanager-datadog/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/CHANGELOG.md b/sdk/datafactory/azure-resourcemanager-datafactory/CHANGELOG.md index 82770238a2339..c8a77309e31fc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/CHANGELOG.md +++ b/sdk/datafactory/azure-resourcemanager-datafactory/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.0.0-beta.27 (Unreleased) +## 1.0.0-beta.28 (Unreleased) ### Features Added @@ -10,6 +10,38 @@ ### Other Changes +## 1.0.0-beta.27 (2024-03-14) + +- Azure Resource Manager DataFactory client library for Java. This package contains Microsoft Azure SDK for DataFactory Management SDK. The Azure Data Factory V2 management API provides a RESTful set of web services that interact with Azure Data Factory V2 services. Package tag package-2018-06. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +* `models.PostgreSqlV2TableDataset` was added + +* `models.ExpressionV2` was added + +* `models.GoogleBigQueryV2Source` was added + +* `models.PostgreSqlV2Source` was added + +* `models.ExpressionV2Type` was added + +* `models.ServiceNowV2LinkedService` was added + +* `models.ServiceNowV2ObjectDataset` was added + +* `models.ServiceNowV2Source` was added + +* `models.GoogleBigQueryV2AuthenticationType` was added + +* `models.GoogleBigQueryV2LinkedService` was added + +* `models.PostgreSqlV2LinkedService` was added + +* `models.ServiceNowV2AuthenticationType` was added + +* `models.GoogleBigQueryV2ObjectDataset` was added + ## 1.0.0-beta.26 (2024-01-29) - Azure Resource Manager DataFactory client library for Java. This package contains Microsoft Azure SDK for DataFactory Management SDK. The Azure Data Factory V2 management API provides a RESTful set of web services that interact with Azure Data Factory V2 services. Package tag package-2018-06. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/README.md b/sdk/datafactory/azure-resourcemanager-datafactory/README.md index d275ad1656b8c..ffb342b468abd 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/README.md +++ b/sdk/datafactory/azure-resourcemanager-datafactory/README.md @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-datafactory - 1.0.0-beta.26 + 1.0.0-beta.27 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/SAMPLE.md b/sdk/datafactory/azure-resourcemanager-datafactory/SAMPLE.md index b5e2a23ceb870..5f741e5933eac 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/SAMPLE.md +++ b/sdk/datafactory/azure-resourcemanager-datafactory/SAMPLE.md @@ -176,22 +176,27 @@ ```java import com.azure.resourcemanager.datafactory.models.RunFilterParameters; import java.time.OffsetDateTime; -import java.time.format.DateTimeFormatter; /** * Samples for ActivityRuns QueryByPipelineRun. */ public final class ActivityRunsQueryByPipelineRunSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ActivityRuns_QueryByPipelineRun.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ActivityRuns_QueryByPipelineRun.json */ /** * Sample code: ActivityRuns_QueryByPipelineRun. * * @param manager Entry point to DataFactoryManager. */ - public static void activityRunsQueryByPipelineRun(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.activityRuns().queryByPipelineRunWithResponse("exampleResourceGroup", "exampleFactoryName", "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", new RunFilterParameters().withLastUpdatedAfter(OffsetDateTime.parse("2018-06-16T00:36:44.3345758Z")).withLastUpdatedBefore(OffsetDateTime.parse("2018-06-16T00:49:48.3686473Z")), com.azure.core.util.Context.NONE); + public static void + activityRunsQueryByPipelineRun(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.activityRuns().queryByPipelineRunWithResponse("exampleResourceGroup", "exampleFactoryName", + "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", + new RunFilterParameters().withLastUpdatedAfter(OffsetDateTime.parse("2018-06-16T00:36:44.3345758Z")) + .withLastUpdatedBefore(OffsetDateTime.parse("2018-06-16T00:49:48.3686473Z")), + com.azure.core.util.Context.NONE); } } ``` @@ -210,7 +215,8 @@ import java.util.List; */ public final class ChangeDataCaptureCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Create.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_Create.json */ /** * Sample code: ChangeDataCapture_Create. @@ -218,11 +224,18 @@ public final class ChangeDataCaptureCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().define("exampleChangeDataCapture").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withSourceConnectionsInfo((List) null).withTargetConnectionsInfo((List) null).withPolicy((MapperPolicy) null).withDescription("Sample demo change data capture to transfer data from delimited (csv) to Azure SQL Database with automapped and non-automapped mappings.").withAllowVNetOverride(false).create(); + manager.changeDataCaptures().define("exampleChangeDataCapture") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withSourceConnectionsInfo((List) null) + .withTargetConnectionsInfo((List) null).withPolicy((MapperPolicy) null) + .withDescription( + "Sample demo change data capture to transfer data from delimited (csv) to Azure SQL Database with automapped and non-automapped mappings.") + .withAllowVNetOverride(false).create(); } /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Update.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_Update.json */ /** * Sample code: ChangeDataCapture_Update. @@ -230,8 +243,11 @@ public final class ChangeDataCaptureCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - ChangeDataCaptureResource resource = manager.changeDataCaptures().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleChangeDataCapture", null, com.azure.core.util.Context.NONE).getValue(); - resource.update().withDescription("Sample demo change data capture to transfer data from delimited (csv) to Azure SQL Database. Updating table mappings.").withAllowVNetOverride(false).withStatus("Stopped").apply(); + ChangeDataCaptureResource resource = manager.changeDataCaptures().getWithResponse("exampleResourceGroup", + "exampleFactoryName", "exampleChangeDataCapture", null, com.azure.core.util.Context.NONE).getValue(); + resource.update().withDescription( + "Sample demo change data capture to transfer data from delimited (csv) to Azure SQL Database. Updating table mappings.") + .withAllowVNetOverride(false).withStatus("Stopped").apply(); } } ``` @@ -244,7 +260,8 @@ public final class ChangeDataCaptureCreateOrUpdateSamples { */ public final class ChangeDataCaptureDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Delete.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_Delete.json */ /** * Sample code: ChangeDataCapture_Delete. @@ -252,7 +269,8 @@ public final class ChangeDataCaptureDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleChangeDataCapture", com.azure.core.util.Context.NONE); + manager.changeDataCaptures().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleChangeDataCapture", com.azure.core.util.Context.NONE); } } ``` @@ -265,7 +283,9 @@ public final class ChangeDataCaptureDeleteSamples { */ public final class ChangeDataCaptureGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Get + * .json */ /** * Sample code: ChangeDataCapture_Get. @@ -273,7 +293,8 @@ public final class ChangeDataCaptureGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleChangeDataCapture", null, com.azure.core.util.Context.NONE); + manager.changeDataCaptures().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleChangeDataCapture", null, com.azure.core.util.Context.NONE); } } ``` @@ -286,15 +307,18 @@ public final class ChangeDataCaptureGetSamples { */ public final class ChangeDataCaptureListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_ListByFactory.json */ /** * Sample code: ChangeDataCapture_ListByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void changeDataCaptureListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + public static void + changeDataCaptureListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.changeDataCaptures().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -307,7 +331,8 @@ public final class ChangeDataCaptureListByFactorySamples { */ public final class ChangeDataCaptureStartSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Start.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_Start.json */ /** * Sample code: ChangeDataCapture_Start. @@ -315,7 +340,8 @@ public final class ChangeDataCaptureStartSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureStart(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().startWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleChangeDataCapture", com.azure.core.util.Context.NONE); + manager.changeDataCaptures().startWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleChangeDataCapture", com.azure.core.util.Context.NONE); } } ``` @@ -328,7 +354,8 @@ public final class ChangeDataCaptureStartSamples { */ public final class ChangeDataCaptureStatusSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Status.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_Status.json */ /** * Sample code: ChangeDataCapture_Start. @@ -336,7 +363,8 @@ public final class ChangeDataCaptureStatusSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureStart(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().statusWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleChangeDataCapture", com.azure.core.util.Context.NONE); + manager.changeDataCaptures().statusWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleChangeDataCapture", com.azure.core.util.Context.NONE); } } ``` @@ -349,7 +377,8 @@ public final class ChangeDataCaptureStatusSamples { */ public final class ChangeDataCaptureStopSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ChangeDataCapture_Stop.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ChangeDataCapture_Stop.json */ /** * Sample code: ChangeDataCapture_Stop. @@ -357,7 +386,8 @@ public final class ChangeDataCaptureStopSamples { * @param manager Entry point to DataFactoryManager. */ public static void changeDataCaptureStop(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.changeDataCaptures().stopWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleChangeDataCapture", com.azure.core.util.Context.NONE); + manager.changeDataCaptures().stopWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleChangeDataCapture", com.azure.core.util.Context.NONE); } } ``` @@ -372,7 +402,9 @@ import com.azure.resourcemanager.datafactory.models.ManagedIdentityCredential; */ public final class CredentialOperationsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_Create.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_Create. + * json */ /** * Sample code: Credentials_Create. @@ -380,7 +412,11 @@ public final class CredentialOperationsCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void credentialsCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.credentialOperations().define("exampleCredential").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new ManagedIdentityCredential().withResourceId("/subscriptions/12345678-1234-1234-1234-12345678abc/resourcegroups/exampleResourceGroup/providers/Microsoft.ManagedIdentity/userAssignedIdentities/exampleUami")).create(); + manager.credentialOperations().define("exampleCredential") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new ManagedIdentityCredential().withResourceId( + "/subscriptions/12345678-1234-1234-1234-12345678abc/resourcegroups/exampleResourceGroup/providers/Microsoft.ManagedIdentity/userAssignedIdentities/exampleUami")) + .create(); } } ``` @@ -393,7 +429,9 @@ public final class CredentialOperationsCreateOrUpdateSamples { */ public final class CredentialOperationsDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_Delete. + * json */ /** * Sample code: Credentials_Delete. @@ -401,7 +439,8 @@ public final class CredentialOperationsDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void credentialsDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.credentialOperations().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleCredential", com.azure.core.util.Context.NONE); + manager.credentialOperations().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleCredential", com.azure.core.util.Context.NONE); } } ``` @@ -414,7 +453,8 @@ public final class CredentialOperationsDeleteSamples { */ public final class CredentialOperationsGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_Get.json */ /** * Sample code: Credentials_Get. @@ -422,7 +462,8 @@ public final class CredentialOperationsGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void credentialsGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.credentialOperations().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleCredential", null, com.azure.core.util.Context.NONE); + manager.credentialOperations().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleCredential", null, com.azure.core.util.Context.NONE); } } ``` @@ -435,7 +476,8 @@ public final class CredentialOperationsGetSamples { */ public final class CredentialOperationsListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Credentials_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Credentials_ListByFactory.json */ /** * Sample code: Credentials_ListByFactory. @@ -443,7 +485,8 @@ public final class CredentialOperationsListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void credentialsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.credentialOperations().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.credentialOperations().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -476,15 +519,49 @@ import java.util.Map; */ public final class DataFlowDebugSessionAddDataFlowSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlowDebugSession_AddDataFlow.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DataFlowDebugSession_AddDataFlow.json */ /** * Sample code: DataFlowDebugSession_AddDataFlow. * * @param manager Entry point to DataFactoryManager. */ - public static void dataFlowDebugSessionAddDataFlow(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - manager.dataFlowDebugSessions().addDataFlowWithResponse("exampleResourceGroup", "exampleFactoryName", new DataFlowDebugPackage().withSessionId("f06ed247-9d07-49b2-b05e-2cb4a2fc871e").withDataFlow(new DataFlowDebugResource().withName("dataflow1").withProperties(new MappingDataFlow().withSources(Arrays.asList(new DataFlowSource().withName("source1").withDataset(new DatasetReference().withReferenceName("DelimitedText2")))).withSinks(Arrays.asList()).withTransformations(Arrays.asList()).withScript("\n\nsource(output(\n\t\tColumn_1 as string\n\t),\n\tallowSchemaDrift: true,\n\tvalidateSchema: false) ~> source1"))).withDatasets(Arrays.asList(new DatasetDebugResource().withName("dataset1").withProperties(new DelimitedTextDataset().withSchema(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("[{\"type\":\"String\"}]", Object.class, SerializerEncoding.JSON)).withLinkedServiceName(new LinkedServiceReference().withReferenceName("linkedService5")).withAnnotations(Arrays.asList()).withLocation(new AzureBlobStorageLocation().withFileName("Ansiencoding.csv").withContainer("dataflow-sample-data")).withColumnDelimiter(",").withQuoteChar("\"").withEscapeChar("\\").withFirstRowAsHeader(true)))).withLinkedServices(Arrays.asList(new LinkedServiceDebugResource().withName("linkedService1").withProperties(new AzureBlobStorageLinkedService().withAnnotations(Arrays.asList()).withConnectionString("DefaultEndpointsProtocol=https;AccountName=;EndpointSuffix=core.windows.net;").withEncryptedCredential("fakeTokenPlaceholder")))).withDebugSettings(new DataFlowDebugPackageDebugSettings().withSourceSettings(Arrays.asList(new DataFlowSourceSetting().withSourceName("source1").withRowLimit(1000).withAdditionalProperties(mapOf()), new DataFlowSourceSetting().withSourceName("source2").withRowLimit(222).withAdditionalProperties(mapOf()))).withParameters(mapOf("sourcePath", "Toy")).withDatasetParameters(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"Movies\":{\"path\":\"abc\"},\"Output\":{\"time\":\"def\"}}", Object.class, SerializerEncoding.JSON))).withAdditionalProperties(mapOf()), com.azure.core.util.Context.NONE); + public static void dataFlowDebugSessionAddDataFlow(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + manager.dataFlowDebugSessions().addDataFlowWithResponse("exampleResourceGroup", "exampleFactoryName", + new DataFlowDebugPackage().withSessionId("f06ed247-9d07-49b2-b05e-2cb4a2fc871e") + .withDataFlow(new DataFlowDebugResource().withName("dataflow1").withProperties(new MappingDataFlow() + .withSources(Arrays.asList(new DataFlowSource().withName("source1") + .withDataset(new DatasetReference().withReferenceName("DelimitedText2")))) + .withSinks(Arrays.asList()).withTransformations(Arrays.asList()).withScript( + "\n\nsource(output(\n\t\tColumn_1 as string\n\t),\n\tallowSchemaDrift: true,\n\tvalidateSchema: false) ~> source1"))) + .withDatasets(Arrays.asList(new DatasetDebugResource().withName("dataset1") + .withProperties(new DelimitedTextDataset() + .withSchema(SerializerFactory.createDefaultManagementSerializerAdapter() + .deserialize("[{\"type\":\"String\"}]", Object.class, SerializerEncoding.JSON)) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("linkedService5")) + .withAnnotations(Arrays.asList()) + .withLocation(new AzureBlobStorageLocation().withFileName("Ansiencoding.csv") + .withContainer("dataflow-sample-data")) + .withColumnDelimiter(",").withQuoteChar("\"").withEscapeChar("\\").withFirstRowAsHeader(true)))) + .withLinkedServices(Arrays.asList(new LinkedServiceDebugResource().withName("linkedService1") + .withProperties(new AzureBlobStorageLinkedService().withAnnotations(Arrays.asList()) + .withConnectionString( + "DefaultEndpointsProtocol=https;AccountName=;EndpointSuffix=core.windows.net;") + .withEncryptedCredential("fakeTokenPlaceholder")))) + .withDebugSettings(new DataFlowDebugPackageDebugSettings() + .withSourceSettings(Arrays.asList( + new DataFlowSourceSetting().withSourceName("source1").withRowLimit(1000) + .withAdditionalProperties(mapOf()), + new DataFlowSourceSetting() + .withSourceName("source2").withRowLimit(222).withAdditionalProperties(mapOf()))) + .withParameters(mapOf("sourcePath", "Toy")) + .withDatasetParameters(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"Movies\":{\"path\":\"abc\"},\"Output\":{\"time\":\"def\"}}", Object.class, + SerializerEncoding.JSON))) + .withAdditionalProperties(mapOf()), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -512,14 +589,14 @@ import com.azure.resourcemanager.datafactory.models.IntegrationRuntimeDebugResou import com.azure.resourcemanager.datafactory.models.ManagedIntegrationRuntime; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for DataFlowDebugSession Create. */ public final class DataFlowDebugSessionCreateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlowDebugSession_Create.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DataFlowDebugSession_Create.json */ /** * Sample code: DataFlowDebugSession_Create. @@ -527,7 +604,17 @@ public final class DataFlowDebugSessionCreateSamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowDebugSessionCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlowDebugSessions().create("exampleResourceGroup", "exampleFactoryName", new CreateDataFlowDebugSessionRequest().withTimeToLive(60).withIntegrationRuntime(new IntegrationRuntimeDebugResource().withName("ir1").withProperties(new ManagedIntegrationRuntime().withComputeProperties(new IntegrationRuntimeComputeProperties().withLocation("AutoResolve").withDataFlowProperties(new IntegrationRuntimeDataFlowProperties().withComputeType(DataFlowComputeType.GENERAL).withCoreCount(48).withTimeToLive(10).withAdditionalProperties(mapOf())).withAdditionalProperties(mapOf())))), com.azure.core.util.Context.NONE); + manager.dataFlowDebugSessions() + .create("exampleResourceGroup", "exampleFactoryName", + new CreateDataFlowDebugSessionRequest().withTimeToLive(60) + .withIntegrationRuntime(new IntegrationRuntimeDebugResource().withName("ir1") + .withProperties(new ManagedIntegrationRuntime() + .withComputeProperties(new IntegrationRuntimeComputeProperties().withLocation("AutoResolve") + .withDataFlowProperties(new IntegrationRuntimeDataFlowProperties() + .withComputeType(DataFlowComputeType.GENERAL).withCoreCount(48).withTimeToLive(10) + .withAdditionalProperties(mapOf())) + .withAdditionalProperties(mapOf())))), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -554,7 +641,8 @@ import com.azure.resourcemanager.datafactory.models.DeleteDataFlowDebugSessionRe */ public final class DataFlowDebugSessionDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlowDebugSession_Delete.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DataFlowDebugSession_Delete.json */ /** * Sample code: DataFlowDebugSession_Delete. @@ -562,7 +650,9 @@ public final class DataFlowDebugSessionDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowDebugSessionDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlowDebugSessions().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", new DeleteDataFlowDebugSessionRequest().withSessionId("91fb57e0-8292-47be-89ff-c8f2d2bb2a7e"), com.azure.core.util.Context.NONE); + manager.dataFlowDebugSessions().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + new DeleteDataFlowDebugSessionRequest().withSessionId("91fb57e0-8292-47be-89ff-c8f2d2bb2a7e"), + com.azure.core.util.Context.NONE); } } ``` @@ -573,22 +663,27 @@ public final class DataFlowDebugSessionDeleteSamples { import com.azure.resourcemanager.datafactory.models.DataFlowDebugCommandPayload; import com.azure.resourcemanager.datafactory.models.DataFlowDebugCommandRequest; import com.azure.resourcemanager.datafactory.models.DataFlowDebugCommandType; -import java.util.stream.Collectors; /** * Samples for DataFlowDebugSession ExecuteCommand. */ public final class DataFlowDebugSessionExecuteCommandSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlowDebugSession_ExecuteCommand.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DataFlowDebugSession_ExecuteCommand.json */ /** * Sample code: DataFlowDebugSession_ExecuteCommand. * * @param manager Entry point to DataFactoryManager. */ - public static void dataFlowDebugSessionExecuteCommand(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlowDebugSessions().executeCommand("exampleResourceGroup", "exampleFactoryName", new DataFlowDebugCommandRequest().withSessionId("f06ed247-9d07-49b2-b05e-2cb4a2fc871e").withCommand(DataFlowDebugCommandType.EXECUTE_PREVIEW_QUERY).withCommandPayload(new DataFlowDebugCommandPayload().withStreamName("source1").withRowLimits(100)), com.azure.core.util.Context.NONE); + public static void + dataFlowDebugSessionExecuteCommand(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.dataFlowDebugSessions().executeCommand("exampleResourceGroup", "exampleFactoryName", + new DataFlowDebugCommandRequest().withSessionId("f06ed247-9d07-49b2-b05e-2cb4a2fc871e") + .withCommand(DataFlowDebugCommandType.EXECUTE_PREVIEW_QUERY) + .withCommandPayload(new DataFlowDebugCommandPayload().withStreamName("source1").withRowLimits(100)), + com.azure.core.util.Context.NONE); } } ``` @@ -601,15 +696,18 @@ public final class DataFlowDebugSessionExecuteCommandSamples { */ public final class DataFlowDebugSessionQueryByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlowDebugSession_QueryByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DataFlowDebugSession_QueryByFactory.json */ /** * Sample code: DataFlowDebugSession_QueryByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void dataFlowDebugSessionQueryByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlowDebugSessions().queryByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + public static void + dataFlowDebugSessionQueryByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.dataFlowDebugSessions().queryByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -629,7 +727,8 @@ import java.util.Arrays; */ public final class DataFlowsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Create.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Create.json */ /** * Sample code: DataFlows_Create. @@ -637,11 +736,36 @@ public final class DataFlowsCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowsCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlows().define("exampleDataFlow").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new MappingDataFlow().withDescription("Sample demo data flow to convert currencies showing usage of union, derive and conditional split transformation.").withSources(Arrays.asList(new DataFlowSource().withName("USDCurrency").withDataset(new DatasetReference().withReferenceName("CurrencyDatasetUSD")), new DataFlowSource().withName("CADSource").withDataset(new DatasetReference().withReferenceName("CurrencyDatasetCAD")))).withSinks(Arrays.asList(new DataFlowSink().withName("USDSink").withDataset(new DatasetReference().withReferenceName("USDOutput")), new DataFlowSink().withName("CADSink").withDataset(new DatasetReference().withReferenceName("CADOutput")))).withScriptLines(Arrays.asList("source(output(", "PreviousConversionRate as double,", "Country as string,", "DateTime1 as string,", "CurrentConversionRate as double", "),", "allowSchemaDrift: false,", "validateSchema: false) ~> USDCurrency", "source(output(", "PreviousConversionRate as double,", "Country as string,", "DateTime1 as string,", "CurrentConversionRate as double", "),", "allowSchemaDrift: true,", "validateSchema: false) ~> CADSource", "USDCurrency, CADSource union(byName: true)~> Union", "Union derive(NewCurrencyRate = round(CurrentConversionRate*1.25)) ~> NewCurrencyColumn", "NewCurrencyColumn split(Country == 'USD',", "Country == 'CAD',disjoint: false) ~> ConditionalSplit1@(USD, CAD)", "ConditionalSplit1@USD sink(saveMode:'overwrite' ) ~> USDSink", "ConditionalSplit1@CAD sink(saveMode:'overwrite' ) ~> CADSink"))).create(); - } - - /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Update.json + manager.dataFlows().define("exampleDataFlow").withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new MappingDataFlow().withDescription( + "Sample demo data flow to convert currencies showing usage of union, derive and conditional split transformation.") + .withSources(Arrays.asList( + new DataFlowSource().withName("USDCurrency") + .withDataset(new DatasetReference().withReferenceName("CurrencyDatasetUSD")), + new DataFlowSource().withName("CADSource") + .withDataset(new DatasetReference().withReferenceName("CurrencyDatasetCAD")))) + .withSinks(Arrays.asList( + new DataFlowSink().withName("USDSink") + .withDataset(new DatasetReference().withReferenceName("USDOutput")), + new DataFlowSink().withName("CADSink") + .withDataset(new DatasetReference().withReferenceName("CADOutput")))) + .withScriptLines(Arrays.asList("source(output(", "PreviousConversionRate as double,", + "Country as string,", "DateTime1 as string,", "CurrentConversionRate as double", "),", + "allowSchemaDrift: false,", "validateSchema: false) ~> USDCurrency", "source(output(", + "PreviousConversionRate as double,", "Country as string,", "DateTime1 as string,", + "CurrentConversionRate as double", "),", "allowSchemaDrift: true,", + "validateSchema: false) ~> CADSource", "USDCurrency, CADSource union(byName: true)~> Union", + "Union derive(NewCurrencyRate = round(CurrentConversionRate*1.25)) ~> NewCurrencyColumn", + "NewCurrencyColumn split(Country == 'USD',", + "Country == 'CAD',disjoint: false) ~> ConditionalSplit1@(USD, CAD)", + "ConditionalSplit1@USD sink(saveMode:'overwrite' ) ~> USDSink", + "ConditionalSplit1@CAD sink(saveMode:'overwrite' ) ~> CADSink"))) + .create(); + } + + /* + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Update.json */ /** * Sample code: DataFlows_Update. @@ -649,8 +773,32 @@ public final class DataFlowsCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowsUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - DataFlowResource resource = manager.dataFlows().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataFlow", null, com.azure.core.util.Context.NONE).getValue(); - resource.update().withProperties(new MappingDataFlow().withDescription("Sample demo data flow to convert currencies showing usage of union, derive and conditional split transformation.").withSources(Arrays.asList(new DataFlowSource().withName("USDCurrency").withDataset(new DatasetReference().withReferenceName("CurrencyDatasetUSD")), new DataFlowSource().withName("CADSource").withDataset(new DatasetReference().withReferenceName("CurrencyDatasetCAD")))).withSinks(Arrays.asList(new DataFlowSink().withName("USDSink").withDataset(new DatasetReference().withReferenceName("USDOutput")), new DataFlowSink().withName("CADSink").withDataset(new DatasetReference().withReferenceName("CADOutput")))).withScriptLines(Arrays.asList("source(output(", "PreviousConversionRate as double,", "Country as string,", "DateTime1 as string,", "CurrentConversionRate as double", "),", "allowSchemaDrift: false,", "validateSchema: false) ~> USDCurrency", "source(output(", "PreviousConversionRate as double,", "Country as string,", "DateTime1 as string,", "CurrentConversionRate as double", "),", "allowSchemaDrift: true,", "validateSchema: false) ~> CADSource", "USDCurrency, CADSource union(byName: true)~> Union", "Union derive(NewCurrencyRate = round(CurrentConversionRate*1.25)) ~> NewCurrencyColumn", "NewCurrencyColumn split(Country == 'USD',", "Country == 'CAD',disjoint: false) ~> ConditionalSplit1@(USD, CAD)", "ConditionalSplit1@USD sink(saveMode:'overwrite' ) ~> USDSink", "ConditionalSplit1@CAD sink(saveMode:'overwrite' ) ~> CADSink"))).apply(); + DataFlowResource resource = manager.dataFlows().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleDataFlow", null, com.azure.core.util.Context.NONE).getValue(); + resource.update().withProperties(new MappingDataFlow().withDescription( + "Sample demo data flow to convert currencies showing usage of union, derive and conditional split transformation.") + .withSources(Arrays.asList( + new DataFlowSource().withName("USDCurrency") + .withDataset(new DatasetReference().withReferenceName("CurrencyDatasetUSD")), + new DataFlowSource().withName("CADSource") + .withDataset(new DatasetReference().withReferenceName("CurrencyDatasetCAD")))) + .withSinks(Arrays.asList( + new DataFlowSink().withName("USDSink") + .withDataset(new DatasetReference().withReferenceName("USDOutput")), + new DataFlowSink().withName("CADSink") + .withDataset(new DatasetReference().withReferenceName("CADOutput")))) + .withScriptLines(Arrays.asList("source(output(", "PreviousConversionRate as double,", "Country as string,", + "DateTime1 as string,", "CurrentConversionRate as double", "),", "allowSchemaDrift: false,", + "validateSchema: false) ~> USDCurrency", "source(output(", "PreviousConversionRate as double,", + "Country as string,", "DateTime1 as string,", "CurrentConversionRate as double", "),", + "allowSchemaDrift: true,", "validateSchema: false) ~> CADSource", + "USDCurrency, CADSource union(byName: true)~> Union", + "Union derive(NewCurrencyRate = round(CurrentConversionRate*1.25)) ~> NewCurrencyColumn", + "NewCurrencyColumn split(Country == 'USD',", + "Country == 'CAD',disjoint: false) ~> ConditionalSplit1@(USD, CAD)", + "ConditionalSplit1@USD sink(saveMode:'overwrite' ) ~> USDSink", + "ConditionalSplit1@CAD sink(saveMode:'overwrite' ) ~> CADSink"))) + .apply(); } } ``` @@ -663,7 +811,8 @@ public final class DataFlowsCreateOrUpdateSamples { */ public final class DataFlowsDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Delete.json */ /** * Sample code: DataFlows_Delete. @@ -671,7 +820,8 @@ public final class DataFlowsDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowsDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlows().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataFlow", com.azure.core.util.Context.NONE); + manager.dataFlows().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataFlow", + com.azure.core.util.Context.NONE); } } ``` @@ -684,7 +834,8 @@ public final class DataFlowsDeleteSamples { */ public final class DataFlowsGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_Get.json */ /** * Sample code: DataFlows_Get. @@ -692,7 +843,8 @@ public final class DataFlowsGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowsGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlows().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataFlow", null, com.azure.core.util.Context.NONE); + manager.dataFlows().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataFlow", null, + com.azure.core.util.Context.NONE); } } ``` @@ -705,7 +857,8 @@ public final class DataFlowsGetSamples { */ public final class DataFlowsListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DataFlows_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DataFlows_ListByFactory.json */ /** * Sample code: DataFlows_ListByFactory. @@ -713,7 +866,8 @@ public final class DataFlowsListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void dataFlowsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.dataFlows().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.dataFlows().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -732,35 +886,60 @@ import com.azure.resourcemanager.datafactory.models.TextFormat; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for Datasets CreateOrUpdate. */ public final class DatasetsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Create.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Create.json */ /** * Sample code: Datasets_Create. * * @param manager Entry point to DataFactoryManager. */ - public static void datasetsCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - manager.datasets().define("exampleDataset").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new AzureBlobDataset().withLinkedServiceName(new LinkedServiceReference().withReferenceName("exampleLinkedService")).withParameters(mapOf("MyFileName", new ParameterSpecification().withType(ParameterType.STRING), "MyFolderPath", new ParameterSpecification().withType(ParameterType.STRING))).withFolderPath(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@dataset().MyFolderPath\"}", Object.class, SerializerEncoding.JSON)).withFileName(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@dataset().MyFileName\"}", Object.class, SerializerEncoding.JSON)).withFormat(new TextFormat())).create(); + public static void datasetsCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + manager.datasets().define("exampleDataset").withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new AzureBlobDataset() + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("exampleLinkedService")) + .withParameters(mapOf("MyFileName", new ParameterSpecification().withType(ParameterType.STRING), + "MyFolderPath", new ParameterSpecification().withType(ParameterType.STRING))) + .withFolderPath(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@dataset().MyFolderPath\"}", Object.class, + SerializerEncoding.JSON)) + .withFileName(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@dataset().MyFileName\"}", Object.class, + SerializerEncoding.JSON)) + .withFormat(new TextFormat())) + .create(); } /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Update.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Update.json */ /** * Sample code: Datasets_Update. * * @param manager Entry point to DataFactoryManager. */ - public static void datasetsUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - DatasetResource resource = manager.datasets().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataset", null, com.azure.core.util.Context.NONE).getValue(); - resource.update().withProperties(new AzureBlobDataset().withDescription("Example description").withLinkedServiceName(new LinkedServiceReference().withReferenceName("exampleLinkedService")).withParameters(mapOf("MyFileName", new ParameterSpecification().withType(ParameterType.STRING), "MyFolderPath", new ParameterSpecification().withType(ParameterType.STRING))).withFolderPath(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@dataset().MyFolderPath\"}", Object.class, SerializerEncoding.JSON)).withFileName(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@dataset().MyFileName\"}", Object.class, SerializerEncoding.JSON)).withFormat(new TextFormat())).apply(); + public static void datasetsUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + DatasetResource resource = manager.datasets().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleDataset", null, com.azure.core.util.Context.NONE).getValue(); + resource.update().withProperties(new AzureBlobDataset().withDescription("Example description") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("exampleLinkedService")) + .withParameters(mapOf("MyFileName", new ParameterSpecification().withType(ParameterType.STRING), + "MyFolderPath", new ParameterSpecification().withType(ParameterType.STRING))) + .withFolderPath(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@dataset().MyFolderPath\"}", Object.class, + SerializerEncoding.JSON)) + .withFileName(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@dataset().MyFileName\"}", Object.class, SerializerEncoding.JSON)) + .withFormat(new TextFormat())).apply(); } // Use "Map.of" if available @@ -785,7 +964,8 @@ public final class DatasetsCreateOrUpdateSamples { */ public final class DatasetsDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Delete.json */ /** * Sample code: Datasets_Delete. @@ -793,7 +973,8 @@ public final class DatasetsDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void datasetsDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.datasets().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataset", com.azure.core.util.Context.NONE); + manager.datasets().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataset", + com.azure.core.util.Context.NONE); } } ``` @@ -806,7 +987,8 @@ public final class DatasetsDeleteSamples { */ public final class DatasetsGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_Get.json */ /** * Sample code: Datasets_Get. @@ -814,7 +996,8 @@ public final class DatasetsGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void datasetsGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.datasets().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataset", null, com.azure.core.util.Context.NONE); + manager.datasets().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleDataset", null, + com.azure.core.util.Context.NONE); } } ``` @@ -827,7 +1010,8 @@ public final class DatasetsGetSamples { */ public final class DatasetsListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Datasets_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Datasets_ListByFactory.json */ /** * Sample code: Datasets_ListByFactory. @@ -835,7 +1019,8 @@ public final class DatasetsListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void datasetsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.datasets().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.datasets().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -850,15 +1035,19 @@ import com.azure.resourcemanager.datafactory.models.ExposureControlRequest; */ public final class ExposureControlGetFeatureValueSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ExposureControl_GetFeatureValue.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ExposureControl_GetFeatureValue.json */ /** * Sample code: ExposureControl_GetFeatureValue. * * @param manager Entry point to DataFactoryManager. */ - public static void exposureControlGetFeatureValue(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.exposureControls().getFeatureValueWithResponse("WestEurope", new ExposureControlRequest().withFeatureName("ADFIntegrationRuntimeSharingRbac").withFeatureType("Feature"), com.azure.core.util.Context.NONE); + public static void + exposureControlGetFeatureValue(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.exposureControls().getFeatureValueWithResponse("WestEurope", + new ExposureControlRequest().withFeatureName("ADFIntegrationRuntimeSharingRbac").withFeatureType("Feature"), + com.azure.core.util.Context.NONE); } } ``` @@ -873,15 +1062,19 @@ import com.azure.resourcemanager.datafactory.models.ExposureControlRequest; */ public final class ExposureControlGetFeatureValueByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ExposureControl_GetFeatureValueByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ExposureControl_GetFeatureValueByFactory.json */ /** * Sample code: ExposureControl_GetFeatureValueByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void exposureControlGetFeatureValueByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.exposureControls().getFeatureValueByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", new ExposureControlRequest().withFeatureName("ADFIntegrationRuntimeSharingRbac").withFeatureType("Feature"), com.azure.core.util.Context.NONE); + public static void + exposureControlGetFeatureValueByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.exposureControls().getFeatureValueByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", + new ExposureControlRequest().withFeatureName("ADFIntegrationRuntimeSharingRbac").withFeatureType("Feature"), + com.azure.core.util.Context.NONE); } } ``` @@ -898,15 +1091,22 @@ import java.util.Arrays; */ public final class ExposureControlQueryFeatureValuesByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ExposureControl_QueryFeatureValuesByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ExposureControl_QueryFeatureValuesByFactory.json */ /** * Sample code: ExposureControl_QueryFeatureValuesByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void exposureControlQueryFeatureValuesByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.exposureControls().queryFeatureValuesByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", new ExposureControlBatchRequest().withExposureControlRequests(Arrays.asList(new ExposureControlRequest().withFeatureName("ADFIntegrationRuntimeSharingRbac").withFeatureType("Feature"), new ExposureControlRequest().withFeatureName("ADFSampleFeature").withFeatureType("Feature"))), com.azure.core.util.Context.NONE); + public static void + exposureControlQueryFeatureValuesByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.exposureControls().queryFeatureValuesByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", + new ExposureControlBatchRequest().withExposureControlRequests(Arrays.asList( + new ExposureControlRequest().withFeatureName("ADFIntegrationRuntimeSharingRbac") + .withFeatureType("Feature"), + new ExposureControlRequest().withFeatureName("ADFSampleFeature").withFeatureType("Feature"))), + com.azure.core.util.Context.NONE); } } ``` @@ -922,7 +1122,8 @@ import com.azure.resourcemanager.datafactory.models.FactoryVstsConfiguration; */ public final class FactoriesConfigureFactoryRepoSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_ConfigureFactoryRepo.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Factories_ConfigureFactoryRepo.json */ /** * Sample code: Factories_ConfigureFactoryRepo. @@ -930,7 +1131,12 @@ public final class FactoriesConfigureFactoryRepoSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesConfigureFactoryRepo(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.factories().configureFactoryRepoWithResponse("East US", new FactoryRepoUpdate().withFactoryResourceId("/subscriptions/12345678-1234-1234-1234-12345678abc/resourceGroups/exampleResourceGroup/providers/Microsoft.DataFactory/factories/exampleFactoryName").withRepoConfiguration(new FactoryVstsConfiguration().withAccountName("ADF").withRepositoryName("repo").withCollaborationBranch("master").withRootFolder("/").withLastCommitId("").withProjectName("project").withTenantId("")), com.azure.core.util.Context.NONE); + manager.factories().configureFactoryRepoWithResponse("East US", new FactoryRepoUpdate().withFactoryResourceId( + "/subscriptions/12345678-1234-1234-1234-12345678abc/resourceGroups/exampleResourceGroup/providers/Microsoft.DataFactory/factories/exampleFactoryName") + .withRepoConfiguration(new FactoryVstsConfiguration().withAccountName("ADF").withRepositoryName("repo") + .withCollaborationBranch("master").withRootFolder("/").withLastCommitId("").withProjectName("project") + .withTenantId("")), + com.azure.core.util.Context.NONE); } } ``` @@ -943,7 +1149,8 @@ public final class FactoriesConfigureFactoryRepoSamples { */ public final class FactoriesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_CreateOrUpdate.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Factories_CreateOrUpdate.json */ /** * Sample code: Factories_CreateOrUpdate. @@ -951,7 +1158,8 @@ public final class FactoriesCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesCreateOrUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.factories().define("exampleFactoryName").withRegion("East US").withExistingResourceGroup("exampleResourceGroup").create(); + manager.factories().define("exampleFactoryName").withRegion("East US") + .withExistingResourceGroup("exampleResourceGroup").create(); } } ``` @@ -964,7 +1172,8 @@ public final class FactoriesCreateOrUpdateSamples { */ public final class FactoriesDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_Delete.json */ /** * Sample code: Factories_Delete. @@ -972,7 +1181,8 @@ public final class FactoriesDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.factories().deleteByResourceGroupWithResponse("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.factories().deleteByResourceGroupWithResponse("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -985,7 +1195,8 @@ public final class FactoriesDeleteSamples { */ public final class FactoriesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_Get.json */ /** * Sample code: Factories_Get. @@ -993,7 +1204,8 @@ public final class FactoriesGetByResourceGroupSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.factories().getByResourceGroupWithResponse("exampleResourceGroup", "exampleFactoryName", null, com.azure.core.util.Context.NONE); + manager.factories().getByResourceGroupWithResponse("exampleResourceGroup", "exampleFactoryName", null, + com.azure.core.util.Context.NONE); } } ``` @@ -1008,7 +1220,8 @@ import com.azure.resourcemanager.datafactory.models.UserAccessPolicy; */ public final class FactoriesGetDataPlaneAccessSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_GetDataPlaneAccess.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Factories_GetDataPlaneAccess.json */ /** * Sample code: Factories_GetDataPlaneAccess. @@ -1016,7 +1229,10 @@ public final class FactoriesGetDataPlaneAccessSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesGetDataPlaneAccess(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.factories().getDataPlaneAccessWithResponse("exampleResourceGroup", "exampleFactoryName", new UserAccessPolicy().withPermissions("r").withAccessResourcePath("").withProfileName("DefaultProfile").withStartTime("2018-11-10T02:46:20.2659347Z").withExpireTime("2018-11-10T09:46:20.2659347Z"), com.azure.core.util.Context.NONE); + manager.factories().getDataPlaneAccessWithResponse("exampleResourceGroup", "exampleFactoryName", + new UserAccessPolicy().withPermissions("r").withAccessResourcePath("").withProfileName("DefaultProfile") + .withStartTime("2018-11-10T02:46:20.2659347Z").withExpireTime("2018-11-10T09:46:20.2659347Z"), + com.azure.core.util.Context.NONE); } } ``` @@ -1031,7 +1247,8 @@ import com.azure.resourcemanager.datafactory.models.GitHubAccessTokenRequest; */ public final class FactoriesGetGitHubAccessTokenSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_GetGitHubAccessToken.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Factories_GetGitHubAccessToken.json */ /** * Sample code: Factories_GetGitHubAccessToken. @@ -1039,7 +1256,10 @@ public final class FactoriesGetGitHubAccessTokenSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesGetGitHubAccessToken(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.factories().getGitHubAccessTokenWithResponse("exampleResourceGroup", "exampleFactoryName", new GitHubAccessTokenRequest().withGitHubAccessCode("fakeTokenPlaceholder").withGitHubClientId("some").withGitHubAccessTokenBaseUrl("fakeTokenPlaceholder"), com.azure.core.util.Context.NONE); + manager.factories().getGitHubAccessTokenWithResponse("exampleResourceGroup", "exampleFactoryName", + new GitHubAccessTokenRequest().withGitHubAccessCode("fakeTokenPlaceholder").withGitHubClientId("some") + .withGitHubAccessTokenBaseUrl("fakeTokenPlaceholder"), + com.azure.core.util.Context.NONE); } } ``` @@ -1052,7 +1272,8 @@ public final class FactoriesGetGitHubAccessTokenSamples { */ public final class FactoriesListSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_List.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_List.json */ /** * Sample code: Factories_List. @@ -1073,7 +1294,8 @@ public final class FactoriesListSamples { */ public final class FactoriesListByResourceGroupSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_ListByResourceGroup.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Factories_ListByResourceGroup.json */ /** * Sample code: Factories_ListByResourceGroup. @@ -1098,7 +1320,8 @@ import java.util.Map; */ public final class FactoriesUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_Update.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Factories_Update.json */ /** * Sample code: Factories_Update. @@ -1106,7 +1329,8 @@ public final class FactoriesUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void factoriesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - Factory resource = manager.factories().getByResourceGroupWithResponse("exampleResourceGroup", "exampleFactoryName", null, com.azure.core.util.Context.NONE).getValue(); + Factory resource = manager.factories().getByResourceGroupWithResponse("exampleResourceGroup", + "exampleFactoryName", null, com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("exampleTag", "exampleValue")).apply(); } @@ -1136,7 +1360,8 @@ import java.util.Map; */ public final class GlobalParametersCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GlobalParameters_Create.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * GlobalParameters_Create.json */ /** * Sample code: GlobalParameters_Create. @@ -1144,11 +1369,13 @@ public final class GlobalParametersCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void globalParametersCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.globalParameters().define("default").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties((Map) null).create(); + manager.globalParameters().define("default").withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties((Map) null).create(); } /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GlobalParameters_Update.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * GlobalParameters_Update.json */ /** * Sample code: GlobalParameters_Update. @@ -1156,7 +1383,9 @@ public final class GlobalParametersCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void globalParametersUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - GlobalParameterResource resource = manager.globalParameters().getWithResponse("exampleResourceGroup", "exampleFactoryName", "default", com.azure.core.util.Context.NONE).getValue(); + GlobalParameterResource resource = manager.globalParameters() + .getWithResponse("exampleResourceGroup", "exampleFactoryName", "default", com.azure.core.util.Context.NONE) + .getValue(); resource.update().apply(); } } @@ -1170,7 +1399,8 @@ public final class GlobalParametersCreateOrUpdateSamples { */ public final class GlobalParametersDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GlobalParameters_Delete.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * GlobalParameters_Delete.json */ /** * Sample code: GlobalParameters_Delete. @@ -1178,7 +1408,8 @@ public final class GlobalParametersDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void globalParametersDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.globalParameters().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "default", com.azure.core.util.Context.NONE); + manager.globalParameters().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -1191,7 +1422,9 @@ public final class GlobalParametersDeleteSamples { */ public final class GlobalParametersGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GlobalParameters_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GlobalParameters_Get. + * json */ /** * Sample code: GlobalParameters_Get. @@ -1199,7 +1432,8 @@ public final class GlobalParametersGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void globalParametersGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.globalParameters().getWithResponse("exampleResourceGroup", "exampleFactoryName", "default", com.azure.core.util.Context.NONE); + manager.globalParameters().getWithResponse("exampleResourceGroup", "exampleFactoryName", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -1212,7 +1446,8 @@ public final class GlobalParametersGetSamples { */ public final class GlobalParametersListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GlobalParameters_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * GlobalParameters_ListByFactory.json */ /** * Sample code: GlobalParameters_ListByFactory. @@ -1220,7 +1455,8 @@ public final class GlobalParametersListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void globalParametersListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.globalParameters().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.globalParameters().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -1233,15 +1469,18 @@ public final class GlobalParametersListByFactorySamples { */ public final class IntegrationRuntimeNodesDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimeNodes_Delete.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimeNodes_Delete.json */ /** * Sample code: IntegrationRuntimesNodes_Delete. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesNodesDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimeNodes().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", "Node_1", com.azure.core.util.Context.NONE); + public static void + integrationRuntimesNodesDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimeNodes().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", "Node_1", com.azure.core.util.Context.NONE); } } ``` @@ -1254,7 +1493,8 @@ public final class IntegrationRuntimeNodesDeleteSamples { */ public final class IntegrationRuntimeNodesGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimeNodes_Get.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimeNodes_Get.json */ /** * Sample code: IntegrationRuntimeNodes_Get. @@ -1262,7 +1502,8 @@ public final class IntegrationRuntimeNodesGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimeNodesGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimeNodes().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", "Node_1", com.azure.core.util.Context.NONE); + manager.integrationRuntimeNodes().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", "Node_1", com.azure.core.util.Context.NONE); } } ``` @@ -1275,15 +1516,18 @@ public final class IntegrationRuntimeNodesGetSamples { */ public final class IntegrationRuntimeNodesGetIpAddressSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimeNodes_GetIpAddress.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimeNodes_GetIpAddress.json */ /** * Sample code: IntegrationRuntimeNodes_GetIpAddress. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimeNodesGetIpAddress(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimeNodes().getIpAddressWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", "Node_1", com.azure.core.util.Context.NONE); + public static void + integrationRuntimeNodesGetIpAddress(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimeNodes().getIpAddressWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", "Node_1", com.azure.core.util.Context.NONE); } } ``` @@ -1298,7 +1542,8 @@ import com.azure.resourcemanager.datafactory.models.UpdateIntegrationRuntimeNode */ public final class IntegrationRuntimeNodesUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimeNodes_Update.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimeNodes_Update.json */ /** * Sample code: IntegrationRuntimeNodes_Update. @@ -1306,7 +1551,9 @@ public final class IntegrationRuntimeNodesUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimeNodesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimeNodes().updateWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", "Node_1", new UpdateIntegrationRuntimeNodeRequest().withConcurrentJobsLimit(2), com.azure.core.util.Context.NONE); + manager.integrationRuntimeNodes().updateWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", "Node_1", new UpdateIntegrationRuntimeNodeRequest().withConcurrentJobsLimit(2), + com.azure.core.util.Context.NONE); } } ``` @@ -1321,15 +1568,19 @@ import com.azure.resourcemanager.datafactory.models.GetSsisObjectMetadataRequest */ public final class IntegrationRuntimeObjectMetadataGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimeObjectMetadata_Get.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimeObjectMetadata_Get.json */ /** * Sample code: IntegrationRuntimeObjectMetadata_Get. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimeObjectMetadataGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimeObjectMetadatas().getWithResponse("exampleResourceGroup", "exampleFactoryName", "testactivityv2", new GetSsisObjectMetadataRequest().withMetadataPath("ssisFolders"), com.azure.core.util.Context.NONE); + public static void + integrationRuntimeObjectMetadataGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimeObjectMetadatas().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "testactivityv2", new GetSsisObjectMetadataRequest().withMetadataPath("ssisFolders"), + com.azure.core.util.Context.NONE); } } ``` @@ -1342,15 +1593,18 @@ public final class IntegrationRuntimeObjectMetadataGetSamples { */ public final class IntegrationRuntimeObjectMetadataRefreshSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimeObjectMetadata_Refresh.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimeObjectMetadata_Refresh.json */ /** * Sample code: IntegrationRuntimeObjectMetadata_Refresh. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimeObjectMetadataRefresh(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimeObjectMetadatas().refresh("exampleResourceGroup", "exampleFactoryName", "testactivityv2", com.azure.core.util.Context.NONE); + public static void + integrationRuntimeObjectMetadataRefresh(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimeObjectMetadatas().refresh("exampleResourceGroup", "exampleFactoryName", + "testactivityv2", com.azure.core.util.Context.NONE); } } ``` @@ -1365,15 +1619,22 @@ import com.azure.resourcemanager.datafactory.models.CreateLinkedIntegrationRunti */ public final class IntegrationRuntimesCreateLinkedIntegrationRuntimeSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_CreateLinkedIntegrationRuntime.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_CreateLinkedIntegrationRuntime.json */ /** * Sample code: IntegrationRuntimes_CreateLinkedIntegrationRuntime. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesCreateLinkedIntegrationRuntime(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().createLinkedIntegrationRuntimeWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", new CreateLinkedIntegrationRuntimeRequest().withName("bfa92911-9fb6-4fbe-8f23-beae87bc1c83").withSubscriptionId("061774c7-4b5a-4159-a55b-365581830283").withDataFactoryName("e9955d6d-56ea-4be3-841c-52a12c1a9981").withDataFactoryLocation("West US"), com.azure.core.util.Context.NONE); + public static void integrationRuntimesCreateLinkedIntegrationRuntime( + com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().createLinkedIntegrationRuntimeWithResponse("exampleResourceGroup", + "exampleFactoryName", "exampleIntegrationRuntime", + new CreateLinkedIntegrationRuntimeRequest().withName("bfa92911-9fb6-4fbe-8f23-beae87bc1c83") + .withSubscriptionId("061774c7-4b5a-4159-a55b-365581830283") + .withDataFactoryName("e9955d6d-56ea-4be3-841c-52a12c1a9981").withDataFactoryLocation("West US"), + com.azure.core.util.Context.NONE); } } ``` @@ -1388,7 +1649,8 @@ import com.azure.resourcemanager.datafactory.models.SelfHostedIntegrationRuntime */ public final class IntegrationRuntimesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Create.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Create.json */ /** * Sample code: IntegrationRuntimes_Create. @@ -1396,7 +1658,10 @@ public final class IntegrationRuntimesCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().define("exampleIntegrationRuntime").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new SelfHostedIntegrationRuntime().withDescription("A selfhosted integration runtime")).create(); + manager.integrationRuntimes().define("exampleIntegrationRuntime") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new SelfHostedIntegrationRuntime().withDescription("A selfhosted integration runtime")) + .create(); } } ``` @@ -1409,7 +1674,8 @@ public final class IntegrationRuntimesCreateOrUpdateSamples { */ public final class IntegrationRuntimesDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Delete.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Delete.json */ /** * Sample code: IntegrationRuntimes_Delete. @@ -1417,7 +1683,8 @@ public final class IntegrationRuntimesDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + manager.integrationRuntimes().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1430,7 +1697,8 @@ public final class IntegrationRuntimesDeleteSamples { */ public final class IntegrationRuntimesGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Get.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Get.json */ /** * Sample code: IntegrationRuntimes_Get. @@ -1438,7 +1706,8 @@ public final class IntegrationRuntimesGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", null, com.azure.core.util.Context.NONE); + manager.integrationRuntimes().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", null, com.azure.core.util.Context.NONE); } } ``` @@ -1451,15 +1720,18 @@ public final class IntegrationRuntimesGetSamples { */ public final class IntegrationRuntimesGetConnectionInfoSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_GetConnectionInfo.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_GetConnectionInfo.json */ /** * Sample code: IntegrationRuntimes_GetConnectionInfo. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesGetConnectionInfo(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().getConnectionInfoWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + public static void + integrationRuntimesGetConnectionInfo(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().getConnectionInfoWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1472,15 +1744,18 @@ public final class IntegrationRuntimesGetConnectionInfoSamples { */ public final class IntegrationRuntimesGetMonitoringDataSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_GetMonitoringData.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_GetMonitoringData.json */ /** * Sample code: IntegrationRuntimes_GetMonitoringData. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesGetMonitoringData(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().getMonitoringDataWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + public static void + integrationRuntimesGetMonitoringData(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().getMonitoringDataWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1493,7 +1768,8 @@ public final class IntegrationRuntimesGetMonitoringDataSamples { */ public final class IntegrationRuntimesGetStatusSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_GetStatus.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_GetStatus.json */ /** * Sample code: IntegrationRuntimes_GetStatus. @@ -1501,7 +1777,8 @@ public final class IntegrationRuntimesGetStatusSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesGetStatus(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().getStatusWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + manager.integrationRuntimes().getStatusWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1514,15 +1791,18 @@ public final class IntegrationRuntimesGetStatusSamples { */ public final class IntegrationRuntimesListAuthKeysSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_ListAuthKeys.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_ListAuthKeys.json */ /** * Sample code: IntegrationRuntimes_ListAuthKeys. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesListAuthKeys(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().listAuthKeysWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + public static void + integrationRuntimesListAuthKeys(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().listAuthKeysWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1535,15 +1815,18 @@ public final class IntegrationRuntimesListAuthKeysSamples { */ public final class IntegrationRuntimesListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_ListByFactory.json */ /** * Sample code: IntegrationRuntimes_ListByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + public static void + integrationRuntimesListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -1556,15 +1839,18 @@ public final class IntegrationRuntimesListByFactorySamples { */ public final class IntegrationRuntimesListOutboundNetworkDependenciesEndpointsSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_ListOutboundNetworkDependenciesEndpoints.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_ListOutboundNetworkDependenciesEndpoints.json */ /** * Sample code: IntegrationRuntimes_OutboundNetworkDependenciesEndpoints. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesOutboundNetworkDependenciesEndpoints(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().listOutboundNetworkDependenciesEndpointsWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + public static void integrationRuntimesOutboundNetworkDependenciesEndpoints( + com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().listOutboundNetworkDependenciesEndpointsWithResponse("exampleResourceGroup", + "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1574,22 +1860,26 @@ public final class IntegrationRuntimesListOutboundNetworkDependenciesEndpointsSa ```java import com.azure.resourcemanager.datafactory.models.IntegrationRuntimeAuthKeyName; import com.azure.resourcemanager.datafactory.models.IntegrationRuntimeRegenerateKeyParameters; -import java.util.stream.Collectors; /** * Samples for IntegrationRuntimes RegenerateAuthKey. */ public final class IntegrationRuntimesRegenerateAuthKeySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_RegenerateAuthKey.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_RegenerateAuthKey.json */ /** * Sample code: IntegrationRuntimes_RegenerateAuthKey. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesRegenerateAuthKey(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().regenerateAuthKeyWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", new IntegrationRuntimeRegenerateKeyParameters().withKeyName(IntegrationRuntimeAuthKeyName.AUTH_KEY2), com.azure.core.util.Context.NONE); + public static void + integrationRuntimesRegenerateAuthKey(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().regenerateAuthKeyWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", + new IntegrationRuntimeRegenerateKeyParameters().withKeyName(IntegrationRuntimeAuthKeyName.AUTH_KEY2), + com.azure.core.util.Context.NONE); } } ``` @@ -1604,7 +1894,8 @@ import com.azure.resourcemanager.datafactory.models.LinkedIntegrationRuntimeRequ */ public final class IntegrationRuntimesRemoveLinksSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_RemoveLinks.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_RemoveLinks.json */ /** * Sample code: IntegrationRuntimes_Upgrade. @@ -1612,7 +1903,10 @@ public final class IntegrationRuntimesRemoveLinksSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesUpgrade(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().removeLinksWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", new LinkedIntegrationRuntimeRequest().withLinkedFactoryName("exampleFactoryName-linked"), com.azure.core.util.Context.NONE); + manager.integrationRuntimes().removeLinksWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", + new LinkedIntegrationRuntimeRequest().withLinkedFactoryName("exampleFactoryName-linked"), + com.azure.core.util.Context.NONE); } } ``` @@ -1625,7 +1919,8 @@ public final class IntegrationRuntimesRemoveLinksSamples { */ public final class IntegrationRuntimesStartSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Start.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Start.json */ /** * Sample code: IntegrationRuntimes_Start. @@ -1633,7 +1928,8 @@ public final class IntegrationRuntimesStartSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesStart(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().start("exampleResourceGroup", "exampleFactoryName", "exampleManagedIntegrationRuntime", com.azure.core.util.Context.NONE); + manager.integrationRuntimes().start("exampleResourceGroup", "exampleFactoryName", + "exampleManagedIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1646,7 +1942,8 @@ public final class IntegrationRuntimesStartSamples { */ public final class IntegrationRuntimesStopSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Stop.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Stop.json */ /** * Sample code: IntegrationRuntimes_Stop. @@ -1654,7 +1951,8 @@ public final class IntegrationRuntimesStopSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesStop(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().stop("exampleResourceGroup", "exampleFactoryName", "exampleManagedIntegrationRuntime", com.azure.core.util.Context.NONE); + manager.integrationRuntimes().stop("exampleResourceGroup", "exampleFactoryName", + "exampleManagedIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1667,15 +1965,18 @@ public final class IntegrationRuntimesStopSamples { */ public final class IntegrationRuntimesSyncCredentialsSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_SyncCredentials.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_SyncCredentials.json */ /** * Sample code: IntegrationRuntimes_SyncCredentials. * * @param manager Entry point to DataFactoryManager. */ - public static void integrationRuntimesSyncCredentials(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().syncCredentialsWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + public static void + integrationRuntimesSyncCredentials(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.integrationRuntimes().syncCredentialsWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1685,14 +1986,14 @@ public final class IntegrationRuntimesSyncCredentialsSamples { ```java import com.azure.resourcemanager.datafactory.models.IntegrationRuntimeAutoUpdate; import com.azure.resourcemanager.datafactory.models.IntegrationRuntimeResource; -import java.util.stream.Collectors; /** * Samples for IntegrationRuntimes Update. */ public final class IntegrationRuntimesUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Update.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Update.json */ /** * Sample code: IntegrationRuntimes_Update. @@ -1700,7 +2001,8 @@ public final class IntegrationRuntimesUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - IntegrationRuntimeResource resource = manager.integrationRuntimes().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", null, com.azure.core.util.Context.NONE).getValue(); + IntegrationRuntimeResource resource = manager.integrationRuntimes().getWithResponse("exampleResourceGroup", + "exampleFactoryName", "exampleIntegrationRuntime", null, com.azure.core.util.Context.NONE).getValue(); resource.update().withAutoUpdate(IntegrationRuntimeAutoUpdate.OFF).withUpdateDelayOffset("\"PT3H\"").apply(); } } @@ -1714,7 +2016,8 @@ public final class IntegrationRuntimesUpdateSamples { */ public final class IntegrationRuntimesUpgradeSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/IntegrationRuntimes_Upgrade.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * IntegrationRuntimes_Upgrade.json */ /** * Sample code: IntegrationRuntimes_Upgrade. @@ -1722,7 +2025,8 @@ public final class IntegrationRuntimesUpgradeSamples { * @param manager Entry point to DataFactoryManager. */ public static void integrationRuntimesUpgrade(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.integrationRuntimes().upgradeWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); + manager.integrationRuntimes().upgradeWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleIntegrationRuntime", com.azure.core.util.Context.NONE); } } ``` @@ -1741,28 +2045,45 @@ import java.io.IOException; */ public final class LinkedServicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Create.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Create + * .json */ /** * Sample code: LinkedServices_Create. * * @param manager Entry point to DataFactoryManager. */ - public static void linkedServicesCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - manager.linkedServices().define("exampleLinkedService").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new AzureStorageLinkedService().withConnectionString(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"SecureString\",\"value\":\"DefaultEndpointsProtocol=https;AccountName=examplestorageaccount;AccountKey=\"}", Object.class, SerializerEncoding.JSON))).create(); + public static void linkedServicesCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + manager.linkedServices().define("exampleLinkedService") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new AzureStorageLinkedService() + .withConnectionString(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"SecureString\",\"value\":\"DefaultEndpointsProtocol=https;AccountName=examplestorageaccount;AccountKey=\"}", + Object.class, SerializerEncoding.JSON))) + .create(); } /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Update.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Update + * .json */ /** * Sample code: LinkedServices_Update. * * @param manager Entry point to DataFactoryManager. */ - public static void linkedServicesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - LinkedServiceResource resource = manager.linkedServices().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleLinkedService", null, com.azure.core.util.Context.NONE).getValue(); - resource.update().withProperties(new AzureStorageLinkedService().withDescription("Example description").withConnectionString(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"SecureString\",\"value\":\"DefaultEndpointsProtocol=https;AccountName=examplestorageaccount;AccountKey=\"}", Object.class, SerializerEncoding.JSON))).apply(); + public static void linkedServicesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + LinkedServiceResource resource = manager.linkedServices().getWithResponse("exampleResourceGroup", + "exampleFactoryName", "exampleLinkedService", null, com.azure.core.util.Context.NONE).getValue(); + resource.update().withProperties(new AzureStorageLinkedService().withDescription("Example description") + .withConnectionString(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"SecureString\",\"value\":\"DefaultEndpointsProtocol=https;AccountName=examplestorageaccount;AccountKey=\"}", + Object.class, SerializerEncoding.JSON))) + .apply(); } } ``` @@ -1775,7 +2096,9 @@ public final class LinkedServicesCreateOrUpdateSamples { */ public final class LinkedServicesDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Delete + * .json */ /** * Sample code: LinkedServices_Delete. @@ -1783,7 +2106,8 @@ public final class LinkedServicesDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void linkedServicesDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.linkedServices().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleLinkedService", com.azure.core.util.Context.NONE); + manager.linkedServices().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleLinkedService", com.azure.core.util.Context.NONE); } } ``` @@ -1796,7 +2120,9 @@ public final class LinkedServicesDeleteSamples { */ public final class LinkedServicesGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_Get. + * json */ /** * Sample code: LinkedServices_Get. @@ -1804,7 +2130,8 @@ public final class LinkedServicesGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void linkedServicesGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.linkedServices().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleLinkedService", null, com.azure.core.util.Context.NONE); + manager.linkedServices().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleLinkedService", + null, com.azure.core.util.Context.NONE); } } ``` @@ -1817,7 +2144,8 @@ public final class LinkedServicesGetSamples { */ public final class LinkedServicesListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/LinkedServices_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * LinkedServices_ListByFactory.json */ /** * Sample code: LinkedServices_ListByFactory. @@ -1825,7 +2153,8 @@ public final class LinkedServicesListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void linkedServicesListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.linkedServices().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.linkedServices().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -1843,7 +2172,8 @@ import java.util.Map; */ public final class ManagedPrivateEndpointsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedPrivateEndpoints_Create.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedPrivateEndpoints_Create.json */ /** * Sample code: ManagedVirtualNetworks_Create. @@ -1851,7 +2181,14 @@ public final class ManagedPrivateEndpointsCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void managedVirtualNetworksCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedPrivateEndpoints().define("exampleManagedPrivateEndpointName").withExistingManagedVirtualNetwork("exampleResourceGroup", "exampleFactoryName", "exampleManagedVirtualNetworkName").withProperties(new ManagedPrivateEndpoint().withFqdns(Arrays.asList()).withGroupId("blob").withPrivateLinkResourceId("/subscriptions/12345678-1234-1234-1234-12345678abc/resourceGroups/exampleResourceGroup/providers/Microsoft.Storage/storageAccounts/exampleBlobStorage").withAdditionalProperties(mapOf())).create(); + manager.managedPrivateEndpoints().define("exampleManagedPrivateEndpointName") + .withExistingManagedVirtualNetwork("exampleResourceGroup", "exampleFactoryName", + "exampleManagedVirtualNetworkName") + .withProperties(new ManagedPrivateEndpoint().withFqdns(Arrays.asList()).withGroupId("blob") + .withPrivateLinkResourceId( + "/subscriptions/12345678-1234-1234-1234-12345678abc/resourceGroups/exampleResourceGroup/providers/Microsoft.Storage/storageAccounts/exampleBlobStorage") + .withAdditionalProperties(mapOf())) + .create(); } // Use "Map.of" if available @@ -1876,7 +2213,8 @@ public final class ManagedPrivateEndpointsCreateOrUpdateSamples { */ public final class ManagedPrivateEndpointsDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedPrivateEndpoints_Delete.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedPrivateEndpoints_Delete.json */ /** * Sample code: ManagedVirtualNetworks_Delete. @@ -1884,7 +2222,8 @@ public final class ManagedPrivateEndpointsDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void managedVirtualNetworksDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedPrivateEndpoints().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleManagedVirtualNetworkName", "exampleManagedPrivateEndpointName", com.azure.core.util.Context.NONE); + manager.managedPrivateEndpoints().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleManagedVirtualNetworkName", "exampleManagedPrivateEndpointName", com.azure.core.util.Context.NONE); } } ``` @@ -1897,7 +2236,8 @@ public final class ManagedPrivateEndpointsDeleteSamples { */ public final class ManagedPrivateEndpointsGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedPrivateEndpoints_Get.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedPrivateEndpoints_Get.json */ /** * Sample code: ManagedPrivateEndpoints_Get. @@ -1905,7 +2245,9 @@ public final class ManagedPrivateEndpointsGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void managedPrivateEndpointsGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedPrivateEndpoints().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleManagedVirtualNetworkName", "exampleManagedPrivateEndpointName", null, com.azure.core.util.Context.NONE); + manager.managedPrivateEndpoints().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleManagedVirtualNetworkName", "exampleManagedPrivateEndpointName", null, + com.azure.core.util.Context.NONE); } } ``` @@ -1918,15 +2260,18 @@ public final class ManagedPrivateEndpointsGetSamples { */ public final class ManagedPrivateEndpointsListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedPrivateEndpoints_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedPrivateEndpoints_ListByFactory.json */ /** * Sample code: ManagedPrivateEndpoints_ListByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void managedPrivateEndpointsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedPrivateEndpoints().listByFactory("exampleResourceGroup", "exampleFactoryName", "exampleManagedVirtualNetworkName", com.azure.core.util.Context.NONE); + public static void + managedPrivateEndpointsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.managedPrivateEndpoints().listByFactory("exampleResourceGroup", "exampleFactoryName", + "exampleManagedVirtualNetworkName", com.azure.core.util.Context.NONE); } } ``` @@ -1943,7 +2288,8 @@ import java.util.Map; */ public final class ManagedVirtualNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedVirtualNetworks_Create.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedVirtualNetworks_Create.json */ /** * Sample code: ManagedVirtualNetworks_Create. @@ -1951,7 +2297,9 @@ public final class ManagedVirtualNetworksCreateOrUpdateSamples { * @param manager Entry point to DataFactoryManager. */ public static void managedVirtualNetworksCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedVirtualNetworks().define("exampleManagedVirtualNetworkName").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new ManagedVirtualNetwork().withAdditionalProperties(mapOf())).create(); + manager.managedVirtualNetworks().define("exampleManagedVirtualNetworkName") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new ManagedVirtualNetwork().withAdditionalProperties(mapOf())).create(); } // Use "Map.of" if available @@ -1976,7 +2324,8 @@ public final class ManagedVirtualNetworksCreateOrUpdateSamples { */ public final class ManagedVirtualNetworksGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedVirtualNetworks_Get.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedVirtualNetworks_Get.json */ /** * Sample code: ManagedVirtualNetworks_Get. @@ -1984,7 +2333,8 @@ public final class ManagedVirtualNetworksGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void managedVirtualNetworksGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedVirtualNetworks().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleManagedVirtualNetworkName", null, com.azure.core.util.Context.NONE); + manager.managedVirtualNetworks().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleManagedVirtualNetworkName", null, com.azure.core.util.Context.NONE); } } ``` @@ -1997,15 +2347,18 @@ public final class ManagedVirtualNetworksGetSamples { */ public final class ManagedVirtualNetworksListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ManagedVirtualNetworks_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ManagedVirtualNetworks_ListByFactory.json */ /** * Sample code: ManagedVirtualNetworks_ListByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void managedVirtualNetworksListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.managedVirtualNetworks().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + public static void + managedVirtualNetworksListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.managedVirtualNetworks().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -2018,7 +2371,8 @@ public final class ManagedVirtualNetworksListByFactorySamples { */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Operations_List.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Operations_List.json */ /** * Sample code: Operations_List. @@ -2039,7 +2393,9 @@ public final class OperationsListSamples { */ public final class PipelineRunsCancelSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/PipelineRuns_Cancel.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/PipelineRuns_Cancel. + * json */ /** * Sample code: PipelineRuns_Cancel. @@ -2047,7 +2403,8 @@ public final class PipelineRunsCancelSamples { * @param manager Entry point to DataFactoryManager. */ public static void pipelineRunsCancel(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.pipelineRuns().cancelWithResponse("exampleResourceGroup", "exampleFactoryName", "16ac5348-ff82-4f95-a80d-638c1d47b721", null, com.azure.core.util.Context.NONE); + manager.pipelineRuns().cancelWithResponse("exampleResourceGroup", "exampleFactoryName", + "16ac5348-ff82-4f95-a80d-638c1d47b721", null, com.azure.core.util.Context.NONE); } } ``` @@ -2060,7 +2417,8 @@ public final class PipelineRunsCancelSamples { */ public final class PipelineRunsGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/PipelineRuns_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/PipelineRuns_Get.json */ /** * Sample code: PipelineRuns_Get. @@ -2068,7 +2426,8 @@ public final class PipelineRunsGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void pipelineRunsGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.pipelineRuns().getWithResponse("exampleResourceGroup", "exampleFactoryName", "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", com.azure.core.util.Context.NONE); + manager.pipelineRuns().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", com.azure.core.util.Context.NONE); } } ``` @@ -2081,16 +2440,15 @@ import com.azure.resourcemanager.datafactory.models.RunQueryFilter; import com.azure.resourcemanager.datafactory.models.RunQueryFilterOperand; import com.azure.resourcemanager.datafactory.models.RunQueryFilterOperator; import java.time.OffsetDateTime; -import java.time.format.DateTimeFormatter; import java.util.Arrays; -import java.util.stream.Collectors; /** * Samples for PipelineRuns QueryByFactory. */ public final class PipelineRunsQueryByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/PipelineRuns_QueryByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * PipelineRuns_QueryByFactory.json */ /** * Sample code: PipelineRuns_QueryByFactory. @@ -2098,7 +2456,12 @@ public final class PipelineRunsQueryByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void pipelineRunsQueryByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.pipelineRuns().queryByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", new RunFilterParameters().withLastUpdatedAfter(OffsetDateTime.parse("2018-06-16T00:36:44.3345758Z")).withLastUpdatedBefore(OffsetDateTime.parse("2018-06-16T00:49:48.3686473Z")).withFilters(Arrays.asList(new RunQueryFilter().withOperand(RunQueryFilterOperand.PIPELINE_NAME).withOperator(RunQueryFilterOperator.EQUALS).withValues(Arrays.asList("examplePipeline")))), com.azure.core.util.Context.NONE); + manager.pipelineRuns().queryByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", + new RunFilterParameters().withLastUpdatedAfter(OffsetDateTime.parse("2018-06-16T00:36:44.3345758Z")) + .withLastUpdatedBefore(OffsetDateTime.parse("2018-06-16T00:49:48.3686473Z")) + .withFilters(Arrays.asList(new RunQueryFilter().withOperand(RunQueryFilterOperand.PIPELINE_NAME) + .withOperator(RunQueryFilterOperator.EQUALS).withValues(Arrays.asList("examplePipeline")))), + com.azure.core.util.Context.NONE); } } ``` @@ -2125,35 +2488,77 @@ import java.io.IOException; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for Pipelines CreateOrUpdate. */ public final class PipelinesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Create.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Create.json */ /** * Sample code: Pipelines_Create. * * @param manager Entry point to DataFactoryManager. */ - public static void pipelinesCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - manager.pipelines().define("examplePipeline").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withActivities(Arrays.asList(new ForEachActivity().withName("ExampleForeachActivity").withIsSequential(true).withItems(new Expression().withValue("@pipeline().parameters.OutputBlobNameList")).withActivities(Arrays.asList(new CopyActivity().withName("ExampleCopyActivity").withInputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset").withParameters(mapOf("MyFileName", "examplecontainer.csv", "MyFolderPath", "examplecontainer")))).withOutputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset").withParameters(mapOf("MyFileName", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@item()\"}", Object.class, SerializerEncoding.JSON), "MyFolderPath", "examplecontainer")))).withSource(new BlobSource()).withSink(new BlobSink()).withDataIntegrationUnits(32))))).withParameters(mapOf("JobId", new ParameterSpecification().withType(ParameterType.STRING), "OutputBlobNameList", new ParameterSpecification().withType(ParameterType.ARRAY))).withVariables(mapOf("TestVariableArray", new VariableSpecification().withType(VariableType.ARRAY))).withRunDimensions(mapOf("JobId", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@pipeline().parameters.JobId\"}", Object.class, SerializerEncoding.JSON))).withPolicy(new PipelinePolicy().withElapsedTimeMetric(new PipelineElapsedTimeMetricPolicy().withDuration("0.00:10:00"))).create(); - } - - /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Update.json + public static void pipelinesCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + manager.pipelines().define("examplePipeline").withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withActivities(Arrays.asList(new ForEachActivity().withName("ExampleForeachActivity") + .withIsSequential(true) + .withItems(new Expression().withValue("@pipeline().parameters.OutputBlobNameList")) + .withActivities(Arrays.asList(new CopyActivity().withName("ExampleCopyActivity") + .withInputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset").withParameters( + mapOf("MyFileName", "examplecontainer.csv", "MyFolderPath", "examplecontainer")))) + .withOutputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset") + .withParameters(mapOf("MyFileName", + SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@item()\"}", Object.class, + SerializerEncoding.JSON), + "MyFolderPath", "examplecontainer")))) + .withSource(new BlobSource()).withSink(new BlobSink()).withDataIntegrationUnits(32))))) + .withParameters(mapOf("JobId", new ParameterSpecification().withType(ParameterType.STRING), + "OutputBlobNameList", new ParameterSpecification().withType(ParameterType.ARRAY))) + .withVariables(mapOf("TestVariableArray", new VariableSpecification().withType(VariableType.ARRAY))) + .withRunDimensions(mapOf("JobId", + SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@pipeline().parameters.JobId\"}", Object.class, + SerializerEncoding.JSON))) + .withPolicy(new PipelinePolicy() + .withElapsedTimeMetric(new PipelineElapsedTimeMetricPolicy().withDuration("0.00:10:00"))) + .create(); + } + + /* + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Update.json */ /** * Sample code: Pipelines_Update. * * @param manager Entry point to DataFactoryManager. */ - public static void pipelinesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - PipelineResource resource = manager.pipelines().getWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", null, com.azure.core.util.Context.NONE).getValue(); - resource.update().withDescription("Example description").withActivities(Arrays.asList(new ForEachActivity().withName("ExampleForeachActivity").withIsSequential(true).withItems(new Expression().withValue("@pipeline().parameters.OutputBlobNameList")).withActivities(Arrays.asList(new CopyActivity().withName("ExampleCopyActivity").withInputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset").withParameters(mapOf("MyFileName", "examplecontainer.csv", "MyFolderPath", "examplecontainer")))).withOutputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset").withParameters(mapOf("MyFileName", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{\"type\":\"Expression\",\"value\":\"@item()\"}", Object.class, SerializerEncoding.JSON), "MyFolderPath", "examplecontainer")))).withSource(new BlobSource()).withSink(new BlobSink()).withDataIntegrationUnits(32))))).withParameters(mapOf("OutputBlobNameList", new ParameterSpecification().withType(ParameterType.ARRAY))).withPolicy(new PipelinePolicy().withElapsedTimeMetric(new PipelineElapsedTimeMetricPolicy().withDuration("0.00:10:00"))).apply(); + public static void pipelinesUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + PipelineResource resource = manager.pipelines().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "examplePipeline", null, com.azure.core.util.Context.NONE).getValue(); + resource.update().withDescription("Example description").withActivities(Arrays.asList(new ForEachActivity() + .withName("ExampleForeachActivity").withIsSequential(true) + .withItems(new Expression().withValue("@pipeline().parameters.OutputBlobNameList")) + .withActivities(Arrays.asList(new CopyActivity().withName("ExampleCopyActivity") + .withInputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset") + .withParameters(mapOf("MyFileName", "examplecontainer.csv", "MyFolderPath", "examplecontainer")))) + .withOutputs(Arrays.asList(new DatasetReference().withReferenceName("exampleDataset") + .withParameters(mapOf("MyFileName", + SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"type\":\"Expression\",\"value\":\"@item()\"}", Object.class, SerializerEncoding.JSON), + "MyFolderPath", "examplecontainer")))) + .withSource(new BlobSource()).withSink(new BlobSink()).withDataIntegrationUnits(32))))) + .withParameters(mapOf("OutputBlobNameList", new ParameterSpecification().withType(ParameterType.ARRAY))) + .withPolicy(new PipelinePolicy() + .withElapsedTimeMetric(new PipelineElapsedTimeMetricPolicy().withDuration("0.00:10:00"))) + .apply(); } // Use "Map.of" if available @@ -2184,15 +2589,22 @@ import java.util.Map; */ public final class PipelinesCreateRunSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_CreateRun.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_CreateRun. + * json */ /** * Sample code: Pipelines_CreateRun. * * @param manager Entry point to DataFactoryManager. */ - public static void pipelinesCreateRun(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - manager.pipelines().createRunWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", null, null, null, null, mapOf("OutputBlobNameList", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("[\"exampleoutput.csv\"]", Object.class, SerializerEncoding.JSON)), com.azure.core.util.Context.NONE); + public static void pipelinesCreateRun(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + manager.pipelines().createRunWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", null, + null, null, null, + mapOf("OutputBlobNameList", SerializerFactory.createDefaultManagementSerializerAdapter() + .deserialize("[\"exampleoutput.csv\"]", Object.class, SerializerEncoding.JSON)), + com.azure.core.util.Context.NONE); } // Use "Map.of" if available @@ -2217,7 +2629,8 @@ public final class PipelinesCreateRunSamples { */ public final class PipelinesDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Delete.json */ /** * Sample code: Pipelines_Delete. @@ -2225,7 +2638,8 @@ public final class PipelinesDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void pipelinesDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.pipelines().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", com.azure.core.util.Context.NONE); + manager.pipelines().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", + com.azure.core.util.Context.NONE); } } ``` @@ -2238,7 +2652,8 @@ public final class PipelinesDeleteSamples { */ public final class PipelinesGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_Get.json */ /** * Sample code: Pipelines_Get. @@ -2246,7 +2661,8 @@ public final class PipelinesGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void pipelinesGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.pipelines().getWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", null, com.azure.core.util.Context.NONE); + manager.pipelines().getWithResponse("exampleResourceGroup", "exampleFactoryName", "examplePipeline", null, + com.azure.core.util.Context.NONE); } } ``` @@ -2259,7 +2675,8 @@ public final class PipelinesGetSamples { */ public final class PipelinesListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Pipelines_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Pipelines_ListByFactory.json */ /** * Sample code: Pipelines_ListByFactory. @@ -2267,7 +2684,8 @@ public final class PipelinesListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void pipelinesListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.pipelines().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.pipelines().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -2280,15 +2698,18 @@ public final class PipelinesListByFactorySamples { */ public final class PrivateEndPointConnectionsListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/PrivateEndPointConnections_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * PrivateEndPointConnections_ListByFactory.json */ /** * Sample code: privateEndPointConnections_ListByFactory. * * @param manager Entry point to DataFactoryManager. */ - public static void privateEndPointConnectionsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.privateEndPointConnections().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + public static void + privateEndPointConnectionsListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.privateEndPointConnections().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -2305,15 +2726,24 @@ import com.azure.resourcemanager.datafactory.models.PrivateLinkConnectionState; */ public final class PrivateEndpointConnectionOperationCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ApproveRejectPrivateEndpointConnection.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * ApproveRejectPrivateEndpointConnection.json */ /** * Sample code: Approves or rejects a private endpoint connection for a factory. * * @param manager Entry point to DataFactoryManager. */ - public static void approvesOrRejectsAPrivateEndpointConnectionForAFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.privateEndpointConnectionOperations().define("connection").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new PrivateLinkConnectionApprovalRequest().withPrivateLinkServiceConnectionState(new PrivateLinkConnectionState().withStatus("Approved").withDescription("Approved by admin.").withActionsRequired("")).withPrivateEndpoint(new PrivateEndpoint().withId("/subscriptions/12345678-1234-1234-1234-12345678abc/resourceGroups/exampleResourceGroup/providers/Microsoft.DataFactory/factories/exampleFactoryName/privateEndpoints/myPrivateEndpoint"))).create(); + public static void approvesOrRejectsAPrivateEndpointConnectionForAFactory( + com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.privateEndpointConnectionOperations().define("connection") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new PrivateLinkConnectionApprovalRequest() + .withPrivateLinkServiceConnectionState(new PrivateLinkConnectionState().withStatus("Approved") + .withDescription("Approved by admin.").withActionsRequired("")) + .withPrivateEndpoint(new PrivateEndpoint().withId( + "/subscriptions/12345678-1234-1234-1234-12345678abc/resourceGroups/exampleResourceGroup/providers/Microsoft.DataFactory/factories/exampleFactoryName/privateEndpoints/myPrivateEndpoint"))) + .create(); } } ``` @@ -2326,15 +2756,18 @@ public final class PrivateEndpointConnectionOperationCreateOrUpdateSamples { */ public final class PrivateEndpointConnectionOperationDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/DeletePrivateEndpointConnection.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * DeletePrivateEndpointConnection.json */ /** * Sample code: Delete a private endpoint connection for a datafactory. * * @param manager Entry point to DataFactoryManager. */ - public static void deleteAPrivateEndpointConnectionForADatafactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.privateEndpointConnectionOperations().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "connection", com.azure.core.util.Context.NONE); + public static void deleteAPrivateEndpointConnectionForADatafactory( + com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.privateEndpointConnectionOperations().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", + "connection", com.azure.core.util.Context.NONE); } } ``` @@ -2347,15 +2780,18 @@ public final class PrivateEndpointConnectionOperationDeleteSamples { */ public final class PrivateEndpointConnectionOperationGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GetPrivateEndpointConnection.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * GetPrivateEndpointConnection.json */ /** * Sample code: Get a private endpoint connection for a datafactory. * * @param manager Entry point to DataFactoryManager. */ - public static void getAPrivateEndpointConnectionForADatafactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.privateEndpointConnectionOperations().getWithResponse("exampleResourceGroup", "exampleFactoryName", "connection", null, com.azure.core.util.Context.NONE); + public static void + getAPrivateEndpointConnectionForADatafactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.privateEndpointConnectionOperations().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "connection", null, com.azure.core.util.Context.NONE); } } ``` @@ -2368,15 +2804,18 @@ public final class PrivateEndpointConnectionOperationGetSamples { */ public final class PrivateLinkResourcesGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/GetPrivateLinkResources.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * GetPrivateLinkResources.json */ /** * Sample code: Get private link resources of a site. * * @param manager Entry point to DataFactoryManager. */ - public static void getPrivateLinkResourcesOfASite(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.privateLinkResources().getWithResponse("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + public static void + getPrivateLinkResourcesOfASite(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.privateLinkResources().getWithResponse("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -2389,7 +2828,9 @@ public final class PrivateLinkResourcesGetSamples { */ public final class TriggerRunsCancelSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/TriggerRuns_Cancel.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/TriggerRuns_Cancel. + * json */ /** * Sample code: Triggers_Cancel. @@ -2397,7 +2838,8 @@ public final class TriggerRunsCancelSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersCancel(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggerRuns().cancelWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", com.azure.core.util.Context.NONE); + manager.triggerRuns().cancelWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", com.azure.core.util.Context.NONE); } } ``` @@ -2410,16 +2852,15 @@ import com.azure.resourcemanager.datafactory.models.RunQueryFilter; import com.azure.resourcemanager.datafactory.models.RunQueryFilterOperand; import com.azure.resourcemanager.datafactory.models.RunQueryFilterOperator; import java.time.OffsetDateTime; -import java.time.format.DateTimeFormatter; import java.util.Arrays; -import java.util.stream.Collectors; /** * Samples for TriggerRuns QueryByFactory. */ public final class TriggerRunsQueryByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/TriggerRuns_QueryByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * TriggerRuns_QueryByFactory.json */ /** * Sample code: TriggerRuns_QueryByFactory. @@ -2427,7 +2868,12 @@ public final class TriggerRunsQueryByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void triggerRunsQueryByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggerRuns().queryByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", new RunFilterParameters().withLastUpdatedAfter(OffsetDateTime.parse("2018-06-16T00:36:44.3345758Z")).withLastUpdatedBefore(OffsetDateTime.parse("2018-06-16T00:49:48.3686473Z")).withFilters(Arrays.asList(new RunQueryFilter().withOperand(RunQueryFilterOperand.TRIGGER_NAME).withOperator(RunQueryFilterOperator.EQUALS).withValues(Arrays.asList("exampleTrigger")))), com.azure.core.util.Context.NONE); + manager.triggerRuns().queryByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", + new RunFilterParameters().withLastUpdatedAfter(OffsetDateTime.parse("2018-06-16T00:36:44.3345758Z")) + .withLastUpdatedBefore(OffsetDateTime.parse("2018-06-16T00:49:48.3686473Z")) + .withFilters(Arrays.asList(new RunQueryFilter().withOperand(RunQueryFilterOperand.TRIGGER_NAME) + .withOperator(RunQueryFilterOperator.EQUALS).withValues(Arrays.asList("exampleTrigger")))), + com.azure.core.util.Context.NONE); } } ``` @@ -2440,7 +2886,9 @@ public final class TriggerRunsQueryByFactorySamples { */ public final class TriggerRunsRerunSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/TriggerRuns_Rerun.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/TriggerRuns_Rerun. + * json */ /** * Sample code: Triggers_Rerun. @@ -2448,7 +2896,8 @@ public final class TriggerRunsRerunSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersRerun(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggerRuns().rerunWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", com.azure.core.util.Context.NONE); + manager.triggerRuns().rerunWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + "2f7fdb90-5df1-4b8e-ac2f-064cfa58202b", com.azure.core.util.Context.NONE); } } ``` @@ -2466,39 +2915,67 @@ import com.azure.resourcemanager.datafactory.models.TriggerPipelineReference; import com.azure.resourcemanager.datafactory.models.TriggerResource; import java.io.IOException; import java.time.OffsetDateTime; -import java.time.format.DateTimeFormatter; import java.util.Arrays; import java.util.HashMap; import java.util.Map; -import java.util.stream.Collectors; /** * Samples for Triggers CreateOrUpdate. */ public final class TriggersCreateOrUpdateSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Create.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Create.json */ /** * Sample code: Triggers_Create. * * @param manager Entry point to DataFactoryManager. */ - public static void triggersCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - manager.triggers().define("exampleTrigger").withExistingFactory("exampleResourceGroup", "exampleFactoryName").withProperties(new ScheduleTrigger().withPipelines(Arrays.asList(new TriggerPipelineReference().withPipelineReference(new PipelineReference().withReferenceName("examplePipeline")).withParameters(mapOf("OutputBlobNameList", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("[\"exampleoutput.csv\"]", Object.class, SerializerEncoding.JSON))))).withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.MINUTE).withInterval(4).withStartTime(OffsetDateTime.parse("2018-06-16T00:39:13.8441801Z")).withEndTime(OffsetDateTime.parse("2018-06-16T00:55:13.8441801Z")).withTimeZone("UTC").withAdditionalProperties(mapOf()))).create(); + public static void triggersCreate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + manager + .triggers().define( + "exampleTrigger") + .withExistingFactory("exampleResourceGroup", "exampleFactoryName") + .withProperties(new ScheduleTrigger() + .withPipelines(Arrays.asList(new TriggerPipelineReference() + .withPipelineReference(new PipelineReference().withReferenceName("examplePipeline")) + .withParameters(mapOf("OutputBlobNameList", + SerializerFactory.createDefaultManagementSerializerAdapter() + .deserialize("[\"exampleoutput.csv\"]", Object.class, SerializerEncoding.JSON))))) + .withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.MINUTE) + .withInterval(4).withStartTime(OffsetDateTime.parse("2018-06-16T00:39:13.8441801Z")) + .withEndTime(OffsetDateTime.parse("2018-06-16T00:55:13.8441801Z")).withTimeZone("UTC") + .withAdditionalProperties(mapOf()))) + .create(); } /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Update.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Update.json */ /** * Sample code: Triggers_Update. * * @param manager Entry point to DataFactoryManager. */ - public static void triggersUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) throws IOException { - TriggerResource resource = manager.triggers().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", null, com.azure.core.util.Context.NONE).getValue(); - resource.update().withProperties(new ScheduleTrigger().withDescription("Example description").withPipelines(Arrays.asList(new TriggerPipelineReference().withPipelineReference(new PipelineReference().withReferenceName("examplePipeline")).withParameters(mapOf("OutputBlobNameList", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("[\"exampleoutput.csv\"]", Object.class, SerializerEncoding.JSON))))).withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.MINUTE).withInterval(4).withStartTime(OffsetDateTime.parse("2018-06-16T00:39:14.905167Z")).withEndTime(OffsetDateTime.parse("2018-06-16T00:55:14.905167Z")).withTimeZone("UTC").withAdditionalProperties(mapOf()))).apply(); + public static void triggersUpdate(com.azure.resourcemanager.datafactory.DataFactoryManager manager) + throws IOException { + TriggerResource resource = manager.triggers().getWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleTrigger", null, com.azure.core.util.Context.NONE).getValue(); + resource.update() + .withProperties(new ScheduleTrigger().withDescription("Example description") + .withPipelines(Arrays.asList(new TriggerPipelineReference() + .withPipelineReference(new PipelineReference().withReferenceName("examplePipeline")) + .withParameters(mapOf("OutputBlobNameList", + SerializerFactory.createDefaultManagementSerializerAdapter() + .deserialize("[\"exampleoutput.csv\"]", Object.class, SerializerEncoding.JSON))))) + .withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.MINUTE) + .withInterval(4).withStartTime(OffsetDateTime.parse("2018-06-16T00:39:14.905167Z")) + .withEndTime(OffsetDateTime.parse("2018-06-16T00:55:14.905167Z")).withTimeZone("UTC") + .withAdditionalProperties(mapOf()))) + .apply(); } // Use "Map.of" if available @@ -2523,7 +3000,8 @@ public final class TriggersCreateOrUpdateSamples { */ public final class TriggersDeleteSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Delete.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Delete.json */ /** * Sample code: Triggers_Delete. @@ -2531,7 +3009,8 @@ public final class TriggersDeleteSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersDelete(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", com.azure.core.util.Context.NONE); + manager.triggers().deleteWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + com.azure.core.util.Context.NONE); } } ``` @@ -2544,7 +3023,8 @@ public final class TriggersDeleteSamples { */ public final class TriggersGetSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Get.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Get.json */ /** * Sample code: Triggers_Get. @@ -2552,7 +3032,8 @@ public final class TriggersGetSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersGet(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", null, com.azure.core.util.Context.NONE); + manager.triggers().getWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", null, + com.azure.core.util.Context.NONE); } } ``` @@ -2565,15 +3046,18 @@ public final class TriggersGetSamples { */ public final class TriggersGetEventSubscriptionStatusSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_GetEventSubscriptionStatus.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Triggers_GetEventSubscriptionStatus.json */ /** * Sample code: Triggers_GetEventSubscriptionStatus. * * @param manager Entry point to DataFactoryManager. */ - public static void triggersGetEventSubscriptionStatus(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().getEventSubscriptionStatusWithResponse("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", com.azure.core.util.Context.NONE); + public static void + triggersGetEventSubscriptionStatus(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { + manager.triggers().getEventSubscriptionStatusWithResponse("exampleResourceGroup", "exampleFactoryName", + "exampleTrigger", com.azure.core.util.Context.NONE); } } ``` @@ -2586,7 +3070,8 @@ public final class TriggersGetEventSubscriptionStatusSamples { */ public final class TriggersListByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_ListByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Triggers_ListByFactory.json */ /** * Sample code: Triggers_ListByFactory. @@ -2594,7 +3079,8 @@ public final class TriggersListByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersListByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().listByFactory("exampleResourceGroup", "exampleFactoryName", com.azure.core.util.Context.NONE); + manager.triggers().listByFactory("exampleResourceGroup", "exampleFactoryName", + com.azure.core.util.Context.NONE); } } ``` @@ -2609,7 +3095,8 @@ import com.azure.resourcemanager.datafactory.models.TriggerFilterParameters; */ public final class TriggersQueryByFactorySamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_QueryByFactory.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Triggers_QueryByFactory.json */ /** * Sample code: Triggers_QueryByFactory. @@ -2617,7 +3104,8 @@ public final class TriggersQueryByFactorySamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersQueryByFactory(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().queryByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", new TriggerFilterParameters().withParentTriggerName("exampleTrigger"), com.azure.core.util.Context.NONE); + manager.triggers().queryByFactoryWithResponse("exampleResourceGroup", "exampleFactoryName", + new TriggerFilterParameters().withParentTriggerName("exampleTrigger"), com.azure.core.util.Context.NONE); } } ``` @@ -2630,7 +3118,8 @@ public final class TriggersQueryByFactorySamples { */ public final class TriggersStartSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Start.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Start.json */ /** * Sample code: Triggers_Start. @@ -2638,7 +3127,8 @@ public final class TriggersStartSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersStart(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().start("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", com.azure.core.util.Context.NONE); + manager.triggers().start("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + com.azure.core.util.Context.NONE); } } ``` @@ -2651,7 +3141,8 @@ public final class TriggersStartSamples { */ public final class TriggersStopSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Stop.json + * x-ms-original-file: + * specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_Stop.json */ /** * Sample code: Triggers_Stop. @@ -2659,7 +3150,8 @@ public final class TriggersStopSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersStop(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().stop("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", com.azure.core.util.Context.NONE); + manager.triggers().stop("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + com.azure.core.util.Context.NONE); } } ``` @@ -2672,7 +3164,8 @@ public final class TriggersStopSamples { */ public final class TriggersSubscribeToEventsSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_SubscribeToEvents.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Triggers_SubscribeToEvents.json */ /** * Sample code: Triggers_SubscribeToEvents. @@ -2680,7 +3173,8 @@ public final class TriggersSubscribeToEventsSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersSubscribeToEvents(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().subscribeToEvents("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", com.azure.core.util.Context.NONE); + manager.triggers().subscribeToEvents("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + com.azure.core.util.Context.NONE); } } ``` @@ -2693,7 +3187,8 @@ public final class TriggersSubscribeToEventsSamples { */ public final class TriggersUnsubscribeFromEventsSamples { /* - * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/Triggers_UnsubscribeFromEvents.json + * x-ms-original-file: specification/datafactory/resource-manager/Microsoft.DataFactory/stable/2018-06-01/examples/ + * Triggers_UnsubscribeFromEvents.json */ /** * Sample code: Triggers_UnsubscribeFromEvents. @@ -2701,7 +3196,8 @@ public final class TriggersUnsubscribeFromEventsSamples { * @param manager Entry point to DataFactoryManager. */ public static void triggersUnsubscribeFromEvents(com.azure.resourcemanager.datafactory.DataFactoryManager manager) { - manager.triggers().unsubscribeFromEvents("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", com.azure.core.util.Context.NONE); + manager.triggers().unsubscribeFromEvents("exampleResourceGroup", "exampleFactoryName", "exampleTrigger", + com.azure.core.util.Context.NONE); } } ``` diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/pom.xml b/sdk/datafactory/azure-resourcemanager-datafactory/pom.xml index bc57b5694cfc7..aebefc3d05e75 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/pom.xml +++ b/sdk/datafactory/azure-resourcemanager-datafactory/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-datafactory - 1.0.0-beta.27 + 1.0.0-beta.28 jar Microsoft Azure SDK for DataFactory Management @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -109,13 +109,13 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 test com.azure azure-storage-blob - 12.25.2 + 12.25.3 test diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/DataFactoryManager.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/DataFactoryManager.java index 2d3d427183a92..a69029c9a678a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/DataFactoryManager.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/DataFactoryManager.java @@ -289,7 +289,7 @@ public DataFactoryManager authenticate(TokenCredential credential, AzureProfile StringBuilder userAgentBuilder = new StringBuilder(); userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.datafactory") - .append("/").append("1.0.0-beta.26"); + .append("/").append("1.0.0-beta.27"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2DatasetTypeProperties.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2DatasetTypeProperties.java new file mode 100644 index 0000000000000..09a9872a94b96 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2DatasetTypeProperties.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Google BigQuery Dataset Properties. + */ +@Fluent +public final class GoogleBigQueryV2DatasetTypeProperties { + /* + * The table name of the Google BigQuery. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "table") + private Object table; + + /* + * The database name of the Google BigQuery. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "dataset") + private Object dataset; + + /** + * Creates an instance of GoogleBigQueryV2DatasetTypeProperties class. + */ + public GoogleBigQueryV2DatasetTypeProperties() { + } + + /** + * Get the table property: The table name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @return the table value. + */ + public Object table() { + return this.table; + } + + /** + * Set the table property: The table name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @param table the table value to set. + * @return the GoogleBigQueryV2DatasetTypeProperties object itself. + */ + public GoogleBigQueryV2DatasetTypeProperties withTable(Object table) { + this.table = table; + return this; + } + + /** + * Get the dataset property: The database name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @return the dataset value. + */ + public Object dataset() { + return this.dataset; + } + + /** + * Set the dataset property: The database name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @param dataset the dataset value to set. + * @return the GoogleBigQueryV2DatasetTypeProperties object itself. + */ + public GoogleBigQueryV2DatasetTypeProperties withDataset(Object dataset) { + this.dataset = dataset; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2LinkedServiceTypeProperties.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2LinkedServiceTypeProperties.java new file mode 100644 index 0000000000000..a06e475520281 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/GoogleBigQueryV2LinkedServiceTypeProperties.java @@ -0,0 +1,246 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2AuthenticationType; +import com.azure.resourcemanager.datafactory.models.SecretBase; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Google BigQuery service linked service properties. + */ +@Fluent +public final class GoogleBigQueryV2LinkedServiceTypeProperties { + /* + * The default BigQuery project id to query against. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "projectId", required = true) + private Object projectId; + + /* + * The OAuth 2.0 authentication mechanism used for authentication. + */ + @JsonProperty(value = "authenticationType", required = true) + private GoogleBigQueryV2AuthenticationType authenticationType; + + /* + * The client id of the google application used to acquire the refresh token. Type: string (or Expression with + * resultType string). + */ + @JsonProperty(value = "clientId") + private Object clientId; + + /* + * The client secret of the google application used to acquire the refresh token. + */ + @JsonProperty(value = "clientSecret") + private SecretBase clientSecret; + + /* + * The refresh token obtained from Google for authorizing access to BigQuery for UserAuthentication. + */ + @JsonProperty(value = "refreshToken") + private SecretBase refreshToken; + + /* + * The content of the .json key file that is used to authenticate the service account. Type: string (or Expression + * with resultType string). + */ + @JsonProperty(value = "keyFileContent") + private SecretBase keyFileContent; + + /* + * The encrypted credential used for authentication. Credentials are encrypted using the integration runtime + * credential manager. Type: string. + */ + @JsonProperty(value = "encryptedCredential") + private String encryptedCredential; + + /** + * Creates an instance of GoogleBigQueryV2LinkedServiceTypeProperties class. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties() { + } + + /** + * Get the projectId property: The default BigQuery project id to query against. Type: string (or Expression with + * resultType string). + * + * @return the projectId value. + */ + public Object projectId() { + return this.projectId; + } + + /** + * Set the projectId property: The default BigQuery project id to query against. Type: string (or Expression with + * resultType string). + * + * @param projectId the projectId value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties withProjectId(Object projectId) { + this.projectId = projectId; + return this; + } + + /** + * Get the authenticationType property: The OAuth 2.0 authentication mechanism used for authentication. + * + * @return the authenticationType value. + */ + public GoogleBigQueryV2AuthenticationType authenticationType() { + return this.authenticationType; + } + + /** + * Set the authenticationType property: The OAuth 2.0 authentication mechanism used for authentication. + * + * @param authenticationType the authenticationType value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties + withAuthenticationType(GoogleBigQueryV2AuthenticationType authenticationType) { + this.authenticationType = authenticationType; + return this; + } + + /** + * Get the clientId property: The client id of the google application used to acquire the refresh token. Type: + * string (or Expression with resultType string). + * + * @return the clientId value. + */ + public Object clientId() { + return this.clientId; + } + + /** + * Set the clientId property: The client id of the google application used to acquire the refresh token. Type: + * string (or Expression with resultType string). + * + * @param clientId the clientId value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties withClientId(Object clientId) { + this.clientId = clientId; + return this; + } + + /** + * Get the clientSecret property: The client secret of the google application used to acquire the refresh token. + * + * @return the clientSecret value. + */ + public SecretBase clientSecret() { + return this.clientSecret; + } + + /** + * Set the clientSecret property: The client secret of the google application used to acquire the refresh token. + * + * @param clientSecret the clientSecret value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties withClientSecret(SecretBase clientSecret) { + this.clientSecret = clientSecret; + return this; + } + + /** + * Get the refreshToken property: The refresh token obtained from Google for authorizing access to BigQuery for + * UserAuthentication. + * + * @return the refreshToken value. + */ + public SecretBase refreshToken() { + return this.refreshToken; + } + + /** + * Set the refreshToken property: The refresh token obtained from Google for authorizing access to BigQuery for + * UserAuthentication. + * + * @param refreshToken the refreshToken value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties withRefreshToken(SecretBase refreshToken) { + this.refreshToken = refreshToken; + return this; + } + + /** + * Get the keyFileContent property: The content of the .json key file that is used to authenticate the service + * account. Type: string (or Expression with resultType string). + * + * @return the keyFileContent value. + */ + public SecretBase keyFileContent() { + return this.keyFileContent; + } + + /** + * Set the keyFileContent property: The content of the .json key file that is used to authenticate the service + * account. Type: string (or Expression with resultType string). + * + * @param keyFileContent the keyFileContent value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties withKeyFileContent(SecretBase keyFileContent) { + this.keyFileContent = keyFileContent; + return this; + } + + /** + * Get the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @return the encryptedCredential value. + */ + public String encryptedCredential() { + return this.encryptedCredential; + } + + /** + * Set the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @param encryptedCredential the encryptedCredential value to set. + * @return the GoogleBigQueryV2LinkedServiceTypeProperties object itself. + */ + public GoogleBigQueryV2LinkedServiceTypeProperties withEncryptedCredential(String encryptedCredential) { + this.encryptedCredential = encryptedCredential; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (projectId() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property projectId in model GoogleBigQueryV2LinkedServiceTypeProperties")); + } + if (authenticationType() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property authenticationType in model GoogleBigQueryV2LinkedServiceTypeProperties")); + } + if (clientSecret() != null) { + clientSecret().validate(); + } + if (refreshToken() != null) { + refreshToken().validate(); + } + if (keyFileContent() != null) { + keyFileContent().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(GoogleBigQueryV2LinkedServiceTypeProperties.class); +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2LinkedServiceTypeProperties.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2LinkedServiceTypeProperties.java new file mode 100644 index 0000000000000..e9bb6c95133db --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2LinkedServiceTypeProperties.java @@ -0,0 +1,568 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.datafactory.models.AzureKeyVaultSecretReference; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * PostgreSqlV2 linked service properties. + */ +@Fluent +public final class PostgreSqlV2LinkedServiceTypeProperties { + /* + * Server name for connection. Type: string. + */ + @JsonProperty(value = "server", required = true) + private Object server; + + /* + * The port for the connection. Type: integer. + */ + @JsonProperty(value = "port") + private Object port; + + /* + * Username for authentication. Type: string. + */ + @JsonProperty(value = "username", required = true) + private Object username; + + /* + * Database name for connection. Type: string. + */ + @JsonProperty(value = "database", required = true) + private Object database; + + /* + * SSL mode for connection. Type: integer. 0: disable, 1:allow, 2: prefer, 3: require, 4: verify-ca, 5: + * verify-full. Type: integer. + */ + @JsonProperty(value = "sslMode", required = true) + private Object sslMode; + + /* + * Sets the schema search path. Type: string. + */ + @JsonProperty(value = "schema") + private Object schema; + + /* + * Whether connection pooling should be used. Type: boolean. + */ + @JsonProperty(value = "pooling") + private Object pooling; + + /* + * The time to wait (in seconds) while trying to establish a connection before terminating the attempt and + * generating an error. Type: integer. + */ + @JsonProperty(value = "connectionTimeout") + private Object connectionTimeout; + + /* + * The time to wait (in seconds) while trying to execute a command before terminating the attempt and generating an + * error. Set to zero for infinity. Type: integer. + */ + @JsonProperty(value = "commandTimeout") + private Object commandTimeout; + + /* + * Whether to trust the server certificate without validating it. Type: boolean. + */ + @JsonProperty(value = "trustServerCertificate") + private Object trustServerCertificate; + + /* + * Location of a client certificate to be sent to the server. Type: string. + */ + @JsonProperty(value = "sslCertificate") + private Object sslCertificate; + + /* + * Location of a client key for a client certificate to be sent to the server. Type: string. + */ + @JsonProperty(value = "sslKey") + private Object sslKey; + + /* + * Password for a key for a client certificate. Type: string. + */ + @JsonProperty(value = "sslPassword") + private Object sslPassword; + + /* + * Determines the size of the internal buffer uses when reading. Increasing may improve performance if transferring + * large values from the database. Type: integer. + */ + @JsonProperty(value = "readBufferSize") + private Object readBufferSize; + + /* + * When enabled, parameter values are logged when commands are executed. Type: boolean. + */ + @JsonProperty(value = "logParameters") + private Object logParameters; + + /* + * Gets or sets the session timezone. Type: string. + */ + @JsonProperty(value = "timezone") + private Object timezone; + + /* + * Gets or sets the .NET encoding that will be used to encode/decode PostgreSQL string data. Type: string + */ + @JsonProperty(value = "encoding") + private Object encoding; + + /* + * The Azure key vault secret reference of password in connection string. Type: string. + */ + @JsonProperty(value = "password") + private AzureKeyVaultSecretReference password; + + /* + * The encrypted credential used for authentication. Credentials are encrypted using the integration runtime + * credential manager. Type: string. + */ + @JsonProperty(value = "encryptedCredential") + private String encryptedCredential; + + /** + * Creates an instance of PostgreSqlV2LinkedServiceTypeProperties class. + */ + public PostgreSqlV2LinkedServiceTypeProperties() { + } + + /** + * Get the server property: Server name for connection. Type: string. + * + * @return the server value. + */ + public Object server() { + return this.server; + } + + /** + * Set the server property: Server name for connection. Type: string. + * + * @param server the server value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withServer(Object server) { + this.server = server; + return this; + } + + /** + * Get the port property: The port for the connection. Type: integer. + * + * @return the port value. + */ + public Object port() { + return this.port; + } + + /** + * Set the port property: The port for the connection. Type: integer. + * + * @param port the port value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withPort(Object port) { + this.port = port; + return this; + } + + /** + * Get the username property: Username for authentication. Type: string. + * + * @return the username value. + */ + public Object username() { + return this.username; + } + + /** + * Set the username property: Username for authentication. Type: string. + * + * @param username the username value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withUsername(Object username) { + this.username = username; + return this; + } + + /** + * Get the database property: Database name for connection. Type: string. + * + * @return the database value. + */ + public Object database() { + return this.database; + } + + /** + * Set the database property: Database name for connection. Type: string. + * + * @param database the database value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withDatabase(Object database) { + this.database = database; + return this; + } + + /** + * Get the sslMode property: SSL mode for connection. Type: integer. 0: disable, 1:allow, 2: prefer, 3: require, 4: + * verify-ca, 5: verify-full. Type: integer. + * + * @return the sslMode value. + */ + public Object sslMode() { + return this.sslMode; + } + + /** + * Set the sslMode property: SSL mode for connection. Type: integer. 0: disable, 1:allow, 2: prefer, 3: require, 4: + * verify-ca, 5: verify-full. Type: integer. + * + * @param sslMode the sslMode value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withSslMode(Object sslMode) { + this.sslMode = sslMode; + return this; + } + + /** + * Get the schema property: Sets the schema search path. Type: string. + * + * @return the schema value. + */ + public Object schema() { + return this.schema; + } + + /** + * Set the schema property: Sets the schema search path. Type: string. + * + * @param schema the schema value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withSchema(Object schema) { + this.schema = schema; + return this; + } + + /** + * Get the pooling property: Whether connection pooling should be used. Type: boolean. + * + * @return the pooling value. + */ + public Object pooling() { + return this.pooling; + } + + /** + * Set the pooling property: Whether connection pooling should be used. Type: boolean. + * + * @param pooling the pooling value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withPooling(Object pooling) { + this.pooling = pooling; + return this; + } + + /** + * Get the connectionTimeout property: The time to wait (in seconds) while trying to establish a connection before + * terminating the attempt and generating an error. Type: integer. + * + * @return the connectionTimeout value. + */ + public Object connectionTimeout() { + return this.connectionTimeout; + } + + /** + * Set the connectionTimeout property: The time to wait (in seconds) while trying to establish a connection before + * terminating the attempt and generating an error. Type: integer. + * + * @param connectionTimeout the connectionTimeout value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withConnectionTimeout(Object connectionTimeout) { + this.connectionTimeout = connectionTimeout; + return this; + } + + /** + * Get the commandTimeout property: The time to wait (in seconds) while trying to execute a command before + * terminating the attempt and generating an error. Set to zero for infinity. Type: integer. + * + * @return the commandTimeout value. + */ + public Object commandTimeout() { + return this.commandTimeout; + } + + /** + * Set the commandTimeout property: The time to wait (in seconds) while trying to execute a command before + * terminating the attempt and generating an error. Set to zero for infinity. Type: integer. + * + * @param commandTimeout the commandTimeout value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withCommandTimeout(Object commandTimeout) { + this.commandTimeout = commandTimeout; + return this; + } + + /** + * Get the trustServerCertificate property: Whether to trust the server certificate without validating it. Type: + * boolean. + * + * @return the trustServerCertificate value. + */ + public Object trustServerCertificate() { + return this.trustServerCertificate; + } + + /** + * Set the trustServerCertificate property: Whether to trust the server certificate without validating it. Type: + * boolean. + * + * @param trustServerCertificate the trustServerCertificate value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withTrustServerCertificate(Object trustServerCertificate) { + this.trustServerCertificate = trustServerCertificate; + return this; + } + + /** + * Get the sslCertificate property: Location of a client certificate to be sent to the server. Type: string. + * + * @return the sslCertificate value. + */ + public Object sslCertificate() { + return this.sslCertificate; + } + + /** + * Set the sslCertificate property: Location of a client certificate to be sent to the server. Type: string. + * + * @param sslCertificate the sslCertificate value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withSslCertificate(Object sslCertificate) { + this.sslCertificate = sslCertificate; + return this; + } + + /** + * Get the sslKey property: Location of a client key for a client certificate to be sent to the server. Type: + * string. + * + * @return the sslKey value. + */ + public Object sslKey() { + return this.sslKey; + } + + /** + * Set the sslKey property: Location of a client key for a client certificate to be sent to the server. Type: + * string. + * + * @param sslKey the sslKey value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withSslKey(Object sslKey) { + this.sslKey = sslKey; + return this; + } + + /** + * Get the sslPassword property: Password for a key for a client certificate. Type: string. + * + * @return the sslPassword value. + */ + public Object sslPassword() { + return this.sslPassword; + } + + /** + * Set the sslPassword property: Password for a key for a client certificate. Type: string. + * + * @param sslPassword the sslPassword value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withSslPassword(Object sslPassword) { + this.sslPassword = sslPassword; + return this; + } + + /** + * Get the readBufferSize property: Determines the size of the internal buffer uses when reading. Increasing may + * improve performance if transferring large values from the database. Type: integer. + * + * @return the readBufferSize value. + */ + public Object readBufferSize() { + return this.readBufferSize; + } + + /** + * Set the readBufferSize property: Determines the size of the internal buffer uses when reading. Increasing may + * improve performance if transferring large values from the database. Type: integer. + * + * @param readBufferSize the readBufferSize value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withReadBufferSize(Object readBufferSize) { + this.readBufferSize = readBufferSize; + return this; + } + + /** + * Get the logParameters property: When enabled, parameter values are logged when commands are executed. Type: + * boolean. + * + * @return the logParameters value. + */ + public Object logParameters() { + return this.logParameters; + } + + /** + * Set the logParameters property: When enabled, parameter values are logged when commands are executed. Type: + * boolean. + * + * @param logParameters the logParameters value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withLogParameters(Object logParameters) { + this.logParameters = logParameters; + return this; + } + + /** + * Get the timezone property: Gets or sets the session timezone. Type: string. + * + * @return the timezone value. + */ + public Object timezone() { + return this.timezone; + } + + /** + * Set the timezone property: Gets or sets the session timezone. Type: string. + * + * @param timezone the timezone value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withTimezone(Object timezone) { + this.timezone = timezone; + return this; + } + + /** + * Get the encoding property: Gets or sets the .NET encoding that will be used to encode/decode PostgreSQL string + * data. Type: string. + * + * @return the encoding value. + */ + public Object encoding() { + return this.encoding; + } + + /** + * Set the encoding property: Gets or sets the .NET encoding that will be used to encode/decode PostgreSQL string + * data. Type: string. + * + * @param encoding the encoding value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withEncoding(Object encoding) { + this.encoding = encoding; + return this; + } + + /** + * Get the password property: The Azure key vault secret reference of password in connection string. Type: string. + * + * @return the password value. + */ + public AzureKeyVaultSecretReference password() { + return this.password; + } + + /** + * Set the password property: The Azure key vault secret reference of password in connection string. Type: string. + * + * @param password the password value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withPassword(AzureKeyVaultSecretReference password) { + this.password = password; + return this; + } + + /** + * Get the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @return the encryptedCredential value. + */ + public String encryptedCredential() { + return this.encryptedCredential; + } + + /** + * Set the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @param encryptedCredential the encryptedCredential value to set. + * @return the PostgreSqlV2LinkedServiceTypeProperties object itself. + */ + public PostgreSqlV2LinkedServiceTypeProperties withEncryptedCredential(String encryptedCredential) { + this.encryptedCredential = encryptedCredential; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (server() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property server in model PostgreSqlV2LinkedServiceTypeProperties")); + } + if (username() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property username in model PostgreSqlV2LinkedServiceTypeProperties")); + } + if (database() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property database in model PostgreSqlV2LinkedServiceTypeProperties")); + } + if (sslMode() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sslMode in model PostgreSqlV2LinkedServiceTypeProperties")); + } + if (password() != null) { + password().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(PostgreSqlV2LinkedServiceTypeProperties.class); +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2TableDatasetTypeProperties.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2TableDatasetTypeProperties.java new file mode 100644 index 0000000000000..7763656aaf50c --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/PostgreSqlV2TableDatasetTypeProperties.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * PostgreSQLV2 table dataset properties. + */ +@Fluent +public final class PostgreSqlV2TableDatasetTypeProperties { + /* + * The PostgreSQL table name. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "table") + private Object table; + + /* + * The PostgreSQL schema name. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "schema") + private Object schema; + + /** + * Creates an instance of PostgreSqlV2TableDatasetTypeProperties class. + */ + public PostgreSqlV2TableDatasetTypeProperties() { + } + + /** + * Get the table property: The PostgreSQL table name. Type: string (or Expression with resultType string). + * + * @return the table value. + */ + public Object table() { + return this.table; + } + + /** + * Set the table property: The PostgreSQL table name. Type: string (or Expression with resultType string). + * + * @param table the table value to set. + * @return the PostgreSqlV2TableDatasetTypeProperties object itself. + */ + public PostgreSqlV2TableDatasetTypeProperties withTable(Object table) { + this.table = table; + return this; + } + + /** + * Get the schema property: The PostgreSQL schema name. Type: string (or Expression with resultType string). + * + * @return the schema value. + */ + public Object schema() { + return this.schema; + } + + /** + * Set the schema property: The PostgreSQL schema name. Type: string (or Expression with resultType string). + * + * @param schema the schema value to set. + * @return the PostgreSqlV2TableDatasetTypeProperties object itself. + */ + public PostgreSqlV2TableDatasetTypeProperties withSchema(Object schema) { + this.schema = schema; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/ServiceNowV2LinkedServiceTypeProperties.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/ServiceNowV2LinkedServiceTypeProperties.java new file mode 100644 index 0000000000000..e9a1009ba0977 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/fluent/models/ServiceNowV2LinkedServiceTypeProperties.java @@ -0,0 +1,261 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.datafactory.models.SecretBase; +import com.azure.resourcemanager.datafactory.models.ServiceNowV2AuthenticationType; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * ServiceNowV2 server linked service properties. + */ +@Fluent +public final class ServiceNowV2LinkedServiceTypeProperties { + /* + * The endpoint of the ServiceNowV2 server. (i.e. .service-now.com) + */ + @JsonProperty(value = "endpoint", required = true) + private Object endpoint; + + /* + * The authentication type to use. + */ + @JsonProperty(value = "authenticationType", required = true) + private ServiceNowV2AuthenticationType authenticationType; + + /* + * The user name used to connect to the ServiceNowV2 server for Basic and OAuth2 authentication. + */ + @JsonProperty(value = "username") + private Object username; + + /* + * The password corresponding to the user name for Basic and OAuth2 authentication. + */ + @JsonProperty(value = "password") + private SecretBase password; + + /* + * The client id for OAuth2 authentication. + */ + @JsonProperty(value = "clientId") + private Object clientId; + + /* + * The client secret for OAuth2 authentication. + */ + @JsonProperty(value = "clientSecret") + private SecretBase clientSecret; + + /* + * GrantType for OAuth2 authentication. Default value is password. + */ + @JsonProperty(value = "grantType") + private Object grantType; + + /* + * The encrypted credential used for authentication. Credentials are encrypted using the integration runtime + * credential manager. Type: string. + */ + @JsonProperty(value = "encryptedCredential") + private String encryptedCredential; + + /** + * Creates an instance of ServiceNowV2LinkedServiceTypeProperties class. + */ + public ServiceNowV2LinkedServiceTypeProperties() { + } + + /** + * Get the endpoint property: The endpoint of the ServiceNowV2 server. (i.e. <instance>.service-now.com). + * + * @return the endpoint value. + */ + public Object endpoint() { + return this.endpoint; + } + + /** + * Set the endpoint property: The endpoint of the ServiceNowV2 server. (i.e. <instance>.service-now.com). + * + * @param endpoint the endpoint value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withEndpoint(Object endpoint) { + this.endpoint = endpoint; + return this; + } + + /** + * Get the authenticationType property: The authentication type to use. + * + * @return the authenticationType value. + */ + public ServiceNowV2AuthenticationType authenticationType() { + return this.authenticationType; + } + + /** + * Set the authenticationType property: The authentication type to use. + * + * @param authenticationType the authenticationType value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties + withAuthenticationType(ServiceNowV2AuthenticationType authenticationType) { + this.authenticationType = authenticationType; + return this; + } + + /** + * Get the username property: The user name used to connect to the ServiceNowV2 server for Basic and OAuth2 + * authentication. + * + * @return the username value. + */ + public Object username() { + return this.username; + } + + /** + * Set the username property: The user name used to connect to the ServiceNowV2 server for Basic and OAuth2 + * authentication. + * + * @param username the username value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withUsername(Object username) { + this.username = username; + return this; + } + + /** + * Get the password property: The password corresponding to the user name for Basic and OAuth2 authentication. + * + * @return the password value. + */ + public SecretBase password() { + return this.password; + } + + /** + * Set the password property: The password corresponding to the user name for Basic and OAuth2 authentication. + * + * @param password the password value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withPassword(SecretBase password) { + this.password = password; + return this; + } + + /** + * Get the clientId property: The client id for OAuth2 authentication. + * + * @return the clientId value. + */ + public Object clientId() { + return this.clientId; + } + + /** + * Set the clientId property: The client id for OAuth2 authentication. + * + * @param clientId the clientId value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withClientId(Object clientId) { + this.clientId = clientId; + return this; + } + + /** + * Get the clientSecret property: The client secret for OAuth2 authentication. + * + * @return the clientSecret value. + */ + public SecretBase clientSecret() { + return this.clientSecret; + } + + /** + * Set the clientSecret property: The client secret for OAuth2 authentication. + * + * @param clientSecret the clientSecret value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withClientSecret(SecretBase clientSecret) { + this.clientSecret = clientSecret; + return this; + } + + /** + * Get the grantType property: GrantType for OAuth2 authentication. Default value is password. + * + * @return the grantType value. + */ + public Object grantType() { + return this.grantType; + } + + /** + * Set the grantType property: GrantType for OAuth2 authentication. Default value is password. + * + * @param grantType the grantType value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withGrantType(Object grantType) { + this.grantType = grantType; + return this; + } + + /** + * Get the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @return the encryptedCredential value. + */ + public String encryptedCredential() { + return this.encryptedCredential; + } + + /** + * Set the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @param encryptedCredential the encryptedCredential value to set. + * @return the ServiceNowV2LinkedServiceTypeProperties object itself. + */ + public ServiceNowV2LinkedServiceTypeProperties withEncryptedCredential(String encryptedCredential) { + this.encryptedCredential = encryptedCredential; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (endpoint() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property endpoint in model ServiceNowV2LinkedServiceTypeProperties")); + } + if (authenticationType() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property authenticationType in model ServiceNowV2LinkedServiceTypeProperties")); + } + if (password() != null) { + password().validate(); + } + if (clientSecret() != null) { + clientSecret().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(ServiceNowV2LinkedServiceTypeProperties.class); +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/ManagedPrivateEndpointsClientImpl.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/ManagedPrivateEndpointsClientImpl.java index 9031c6d0ca8b1..19f8cfaa94d1c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/ManagedPrivateEndpointsClientImpl.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/ManagedPrivateEndpointsClientImpl.java @@ -418,7 +418,7 @@ private Mono createOrUpdateAsync(String res final String ifMatch = null; return createOrUpdateWithResponseAsync(resourceGroupName, factoryName, managedVirtualNetworkName, managedPrivateEndpointName, managedPrivateEndpoint, ifMatch) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/PipelinesClientImpl.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/PipelinesClientImpl.java index 6ade5b7638c55..97528de2260af 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/PipelinesClientImpl.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/implementation/PipelinesClientImpl.java @@ -817,7 +817,7 @@ private Mono createRunAsync(String resourceGroupName, St final Map parameters = null; return createRunWithResponseAsync(resourceGroupName, factoryName, pipelineName, referencePipelineRunId, isRecovery, startActivityName, startFromFailure, parameters) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/Dataset.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/Dataset.java index 0e677dd71a48c..7b9a281eac408 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/Dataset.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/Dataset.java @@ -70,6 +70,7 @@ @JsonSubTypes.Type(name = "OdbcTable", value = OdbcTableDataset.class), @JsonSubTypes.Type(name = "MySqlTable", value = MySqlTableDataset.class), @JsonSubTypes.Type(name = "PostgreSqlTable", value = PostgreSqlTableDataset.class), + @JsonSubTypes.Type(name = "PostgreSqlV2Table", value = PostgreSqlV2TableDataset.class), @JsonSubTypes.Type(name = "MicrosoftAccessTable", value = MicrosoftAccessTableDataset.class), @JsonSubTypes.Type(name = "SalesforceObject", value = SalesforceObjectDataset.class), @JsonSubTypes.Type(name = "SalesforceServiceCloudObject", value = SalesforceServiceCloudObjectDataset.class), @@ -94,6 +95,7 @@ @JsonSubTypes.Type(name = "DrillTable", value = DrillTableDataset.class), @JsonSubTypes.Type(name = "EloquaObject", value = EloquaObjectDataset.class), @JsonSubTypes.Type(name = "GoogleBigQueryObject", value = GoogleBigQueryObjectDataset.class), + @JsonSubTypes.Type(name = "GoogleBigQueryV2Object", value = GoogleBigQueryV2ObjectDataset.class), @JsonSubTypes.Type(name = "GreenplumTable", value = GreenplumTableDataset.class), @JsonSubTypes.Type(name = "HBaseObject", value = HBaseObjectDataset.class), @JsonSubTypes.Type(name = "HiveObject", value = HiveObjectDataset.class), @@ -129,7 +131,8 @@ @JsonSubTypes.Type(name = "LakeHouseTable", value = LakeHouseTableDataset.class), @JsonSubTypes.Type(name = "SalesforceV2Object", value = SalesforceV2ObjectDataset.class), @JsonSubTypes.Type(name = "SalesforceServiceCloudV2Object", value = SalesforceServiceCloudV2ObjectDataset.class), - @JsonSubTypes.Type(name = "WarehouseTable", value = WarehouseTableDataset.class) }) + @JsonSubTypes.Type(name = "WarehouseTable", value = WarehouseTableDataset.class), + @JsonSubTypes.Type(name = "ServiceNowV2Object", value = ServiceNowV2ObjectDataset.class) }) @Fluent public class Dataset { /* diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2.java new file mode 100644 index 0000000000000..10f9336322baf --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2.java @@ -0,0 +1,136 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Nested representation of a complex expression. + */ +@Fluent +public final class ExpressionV2 { + /* + * Type of expressions supported by the system. Type: string. + */ + @JsonProperty(value = "type") + private ExpressionV2Type type; + + /* + * Value for Constant/Field Type: string. + */ + @JsonProperty(value = "value") + private String value; + + /* + * Expression operator value Type: string. + */ + @JsonProperty(value = "operator") + private String operator; + + /* + * List of nested expressions. + */ + @JsonProperty(value = "operands") + private List operands; + + /** + * Creates an instance of ExpressionV2 class. + */ + public ExpressionV2() { + } + + /** + * Get the type property: Type of expressions supported by the system. Type: string. + * + * @return the type value. + */ + public ExpressionV2Type type() { + return this.type; + } + + /** + * Set the type property: Type of expressions supported by the system. Type: string. + * + * @param type the type value to set. + * @return the ExpressionV2 object itself. + */ + public ExpressionV2 withType(ExpressionV2Type type) { + this.type = type; + return this; + } + + /** + * Get the value property: Value for Constant/Field Type: string. + * + * @return the value value. + */ + public String value() { + return this.value; + } + + /** + * Set the value property: Value for Constant/Field Type: string. + * + * @param value the value value to set. + * @return the ExpressionV2 object itself. + */ + public ExpressionV2 withValue(String value) { + this.value = value; + return this; + } + + /** + * Get the operator property: Expression operator value Type: string. + * + * @return the operator value. + */ + public String operator() { + return this.operator; + } + + /** + * Set the operator property: Expression operator value Type: string. + * + * @param operator the operator value to set. + * @return the ExpressionV2 object itself. + */ + public ExpressionV2 withOperator(String operator) { + this.operator = operator; + return this; + } + + /** + * Get the operands property: List of nested expressions. + * + * @return the operands value. + */ + public List operands() { + return this.operands; + } + + /** + * Set the operands property: List of nested expressions. + * + * @param operands the operands value to set. + * @return the ExpressionV2 object itself. + */ + public ExpressionV2 withOperands(List operands) { + this.operands = operands; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (operands() != null) { + operands().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2Type.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2Type.java new file mode 100644 index 0000000000000..9c1776be11c85 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ExpressionV2Type.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Type of expressions supported by the system. Type: string. + */ +public final class ExpressionV2Type extends ExpandableStringEnum { + /** + * Static value Constant for ExpressionV2Type. + */ + public static final ExpressionV2Type CONSTANT = fromString("Constant"); + + /** + * Static value Field for ExpressionV2Type. + */ + public static final ExpressionV2Type FIELD = fromString("Field"); + + /** + * Static value Unary for ExpressionV2Type. + */ + public static final ExpressionV2Type UNARY = fromString("Unary"); + + /** + * Static value Binary for ExpressionV2Type. + */ + public static final ExpressionV2Type BINARY = fromString("Binary"); + + /** + * Creates a new instance of ExpressionV2Type value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ExpressionV2Type() { + } + + /** + * Creates or finds a ExpressionV2Type from its string representation. + * + * @param name a name to look for. + * @return the corresponding ExpressionV2Type. + */ + @JsonCreator + public static ExpressionV2Type fromString(String name) { + return fromString(name, ExpressionV2Type.class); + } + + /** + * Gets known ExpressionV2Type values. + * + * @return known ExpressionV2Type values. + */ + public static Collection values() { + return values(ExpressionV2Type.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2AuthenticationType.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2AuthenticationType.java new file mode 100644 index 0000000000000..94e26d6b5f4c4 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2AuthenticationType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The OAuth 2.0 authentication mechanism used for authentication. + */ +public final class GoogleBigQueryV2AuthenticationType extends ExpandableStringEnum { + /** + * Static value ServiceAuthentication for GoogleBigQueryV2AuthenticationType. + */ + public static final GoogleBigQueryV2AuthenticationType SERVICE_AUTHENTICATION = fromString("ServiceAuthentication"); + + /** + * Static value UserAuthentication for GoogleBigQueryV2AuthenticationType. + */ + public static final GoogleBigQueryV2AuthenticationType USER_AUTHENTICATION = fromString("UserAuthentication"); + + /** + * Creates a new instance of GoogleBigQueryV2AuthenticationType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public GoogleBigQueryV2AuthenticationType() { + } + + /** + * Creates or finds a GoogleBigQueryV2AuthenticationType from its string representation. + * + * @param name a name to look for. + * @return the corresponding GoogleBigQueryV2AuthenticationType. + */ + @JsonCreator + public static GoogleBigQueryV2AuthenticationType fromString(String name) { + return fromString(name, GoogleBigQueryV2AuthenticationType.class); + } + + /** + * Gets known GoogleBigQueryV2AuthenticationType values. + * + * @return known GoogleBigQueryV2AuthenticationType values. + */ + public static Collection values() { + return values(GoogleBigQueryV2AuthenticationType.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2LinkedService.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2LinkedService.java new file mode 100644 index 0000000000000..57ca16792de8d --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2LinkedService.java @@ -0,0 +1,269 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.datafactory.fluent.models.GoogleBigQueryV2LinkedServiceTypeProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.util.List; +import java.util.Map; + +/** + * Google BigQuery service linked service. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("GoogleBigQueryV2") +@Fluent +public final class GoogleBigQueryV2LinkedService extends LinkedService { + /* + * Google BigQuery service linked service properties. + */ + @JsonProperty(value = "typeProperties", required = true) + private GoogleBigQueryV2LinkedServiceTypeProperties innerTypeProperties + = new GoogleBigQueryV2LinkedServiceTypeProperties(); + + /** + * Creates an instance of GoogleBigQueryV2LinkedService class. + */ + public GoogleBigQueryV2LinkedService() { + } + + /** + * Get the innerTypeProperties property: Google BigQuery service linked service properties. + * + * @return the innerTypeProperties value. + */ + private GoogleBigQueryV2LinkedServiceTypeProperties innerTypeProperties() { + return this.innerTypeProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2LinkedService withConnectVia(IntegrationRuntimeReference connectVia) { + super.withConnectVia(connectVia); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2LinkedService withDescription(String description) { + super.withDescription(description); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2LinkedService withParameters(Map parameters) { + super.withParameters(parameters); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2LinkedService withAnnotations(List annotations) { + super.withAnnotations(annotations); + return this; + } + + /** + * Get the projectId property: The default BigQuery project id to query against. Type: string (or Expression with + * resultType string). + * + * @return the projectId value. + */ + public Object projectId() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().projectId(); + } + + /** + * Set the projectId property: The default BigQuery project id to query against. Type: string (or Expression with + * resultType string). + * + * @param projectId the projectId value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withProjectId(Object projectId) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withProjectId(projectId); + return this; + } + + /** + * Get the authenticationType property: The OAuth 2.0 authentication mechanism used for authentication. + * + * @return the authenticationType value. + */ + public GoogleBigQueryV2AuthenticationType authenticationType() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().authenticationType(); + } + + /** + * Set the authenticationType property: The OAuth 2.0 authentication mechanism used for authentication. + * + * @param authenticationType the authenticationType value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withAuthenticationType(GoogleBigQueryV2AuthenticationType authenticationType) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withAuthenticationType(authenticationType); + return this; + } + + /** + * Get the clientId property: The client id of the google application used to acquire the refresh token. Type: + * string (or Expression with resultType string). + * + * @return the clientId value. + */ + public Object clientId() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().clientId(); + } + + /** + * Set the clientId property: The client id of the google application used to acquire the refresh token. Type: + * string (or Expression with resultType string). + * + * @param clientId the clientId value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withClientId(Object clientId) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withClientId(clientId); + return this; + } + + /** + * Get the clientSecret property: The client secret of the google application used to acquire the refresh token. + * + * @return the clientSecret value. + */ + public SecretBase clientSecret() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().clientSecret(); + } + + /** + * Set the clientSecret property: The client secret of the google application used to acquire the refresh token. + * + * @param clientSecret the clientSecret value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withClientSecret(SecretBase clientSecret) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withClientSecret(clientSecret); + return this; + } + + /** + * Get the refreshToken property: The refresh token obtained from Google for authorizing access to BigQuery for + * UserAuthentication. + * + * @return the refreshToken value. + */ + public SecretBase refreshToken() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().refreshToken(); + } + + /** + * Set the refreshToken property: The refresh token obtained from Google for authorizing access to BigQuery for + * UserAuthentication. + * + * @param refreshToken the refreshToken value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withRefreshToken(SecretBase refreshToken) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withRefreshToken(refreshToken); + return this; + } + + /** + * Get the keyFileContent property: The content of the .json key file that is used to authenticate the service + * account. Type: string (or Expression with resultType string). + * + * @return the keyFileContent value. + */ + public SecretBase keyFileContent() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().keyFileContent(); + } + + /** + * Set the keyFileContent property: The content of the .json key file that is used to authenticate the service + * account. Type: string (or Expression with resultType string). + * + * @param keyFileContent the keyFileContent value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withKeyFileContent(SecretBase keyFileContent) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withKeyFileContent(keyFileContent); + return this; + } + + /** + * Get the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @return the encryptedCredential value. + */ + public String encryptedCredential() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().encryptedCredential(); + } + + /** + * Set the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @param encryptedCredential the encryptedCredential value to set. + * @return the GoogleBigQueryV2LinkedService object itself. + */ + public GoogleBigQueryV2LinkedService withEncryptedCredential(String encryptedCredential) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withEncryptedCredential(encryptedCredential); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerTypeProperties() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerTypeProperties in model GoogleBigQueryV2LinkedService")); + } else { + innerTypeProperties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(GoogleBigQueryV2LinkedService.class); +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2ObjectDataset.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2ObjectDataset.java new file mode 100644 index 0000000000000..e5cff01f36924 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2ObjectDataset.java @@ -0,0 +1,168 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.datafactory.fluent.models.GoogleBigQueryV2DatasetTypeProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.util.List; +import java.util.Map; + +/** + * Google BigQuery service dataset. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("GoogleBigQueryV2Object") +@Fluent +public final class GoogleBigQueryV2ObjectDataset extends Dataset { + /* + * Properties specific to this dataset type. + */ + @JsonProperty(value = "typeProperties") + private GoogleBigQueryV2DatasetTypeProperties innerTypeProperties; + + /** + * Creates an instance of GoogleBigQueryV2ObjectDataset class. + */ + public GoogleBigQueryV2ObjectDataset() { + } + + /** + * Get the innerTypeProperties property: Properties specific to this dataset type. + * + * @return the innerTypeProperties value. + */ + private GoogleBigQueryV2DatasetTypeProperties innerTypeProperties() { + return this.innerTypeProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withDescription(String description) { + super.withDescription(description); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withStructure(Object structure) { + super.withStructure(structure); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withSchema(Object schema) { + super.withSchema(schema); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withLinkedServiceName(LinkedServiceReference linkedServiceName) { + super.withLinkedServiceName(linkedServiceName); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withParameters(Map parameters) { + super.withParameters(parameters); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withAnnotations(List annotations) { + super.withAnnotations(annotations); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2ObjectDataset withFolder(DatasetFolder folder) { + super.withFolder(folder); + return this; + } + + /** + * Get the table property: The table name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @return the table value. + */ + public Object table() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().table(); + } + + /** + * Set the table property: The table name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @param table the table value to set. + * @return the GoogleBigQueryV2ObjectDataset object itself. + */ + public GoogleBigQueryV2ObjectDataset withTable(Object table) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2DatasetTypeProperties(); + } + this.innerTypeProperties().withTable(table); + return this; + } + + /** + * Get the dataset property: The database name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @return the dataset value. + */ + public Object dataset() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().dataset(); + } + + /** + * Set the dataset property: The database name of the Google BigQuery. Type: string (or Expression with resultType + * string). + * + * @param dataset the dataset value to set. + * @return the GoogleBigQueryV2ObjectDataset object itself. + */ + public GoogleBigQueryV2ObjectDataset withDataset(Object dataset) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GoogleBigQueryV2DatasetTypeProperties(); + } + this.innerTypeProperties().withDataset(dataset); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerTypeProperties() != null) { + innerTypeProperties().validate(); + } + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2Source.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2Source.java new file mode 100644 index 0000000000000..404a8aacbe6e6 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/GoogleBigQueryV2Source.java @@ -0,0 +1,116 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * A copy activity Google BigQuery service source. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("GoogleBigQueryV2Source") +@Fluent +public final class GoogleBigQueryV2Source extends TabularSource { + /* + * A query to retrieve data from source. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "query") + private Object query; + + /** + * Creates an instance of GoogleBigQueryV2Source class. + */ + public GoogleBigQueryV2Source() { + } + + /** + * Get the query property: A query to retrieve data from source. Type: string (or Expression with resultType + * string). + * + * @return the query value. + */ + public Object query() { + return this.query; + } + + /** + * Set the query property: A query to retrieve data from source. Type: string (or Expression with resultType + * string). + * + * @param query the query value to set. + * @return the GoogleBigQueryV2Source object itself. + */ + public GoogleBigQueryV2Source withQuery(Object query) { + this.query = query; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2Source withQueryTimeout(Object queryTimeout) { + super.withQueryTimeout(queryTimeout); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2Source withAdditionalColumns(Object additionalColumns) { + super.withAdditionalColumns(additionalColumns); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2Source withSourceRetryCount(Object sourceRetryCount) { + super.withSourceRetryCount(sourceRetryCount); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2Source withSourceRetryWait(Object sourceRetryWait) { + super.withSourceRetryWait(sourceRetryWait); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2Source withMaxConcurrentConnections(Object maxConcurrentConnections) { + super.withMaxConcurrentConnections(maxConcurrentConnections); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public GoogleBigQueryV2Source withDisableMetricsCollection(Object disableMetricsCollection) { + super.withDisableMetricsCollection(disableMetricsCollection); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/LinkedService.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/LinkedService.java index 6f736758e196e..58e62f220ab6c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/LinkedService.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/LinkedService.java @@ -53,6 +53,7 @@ @JsonSubTypes.Type(name = "AzureMySql", value = AzureMySqlLinkedService.class), @JsonSubTypes.Type(name = "MySql", value = MySqlLinkedService.class), @JsonSubTypes.Type(name = "PostgreSql", value = PostgreSqlLinkedService.class), + @JsonSubTypes.Type(name = "PostgreSqlV2", value = PostgreSqlV2LinkedService.class), @JsonSubTypes.Type(name = "Sybase", value = SybaseLinkedService.class), @JsonSubTypes.Type(name = "Db2", value = Db2LinkedService.class), @JsonSubTypes.Type(name = "Teradata", value = TeradataLinkedService.class), @@ -104,6 +105,7 @@ @JsonSubTypes.Type(name = "Drill", value = DrillLinkedService.class), @JsonSubTypes.Type(name = "Eloqua", value = EloquaLinkedService.class), @JsonSubTypes.Type(name = "GoogleBigQuery", value = GoogleBigQueryLinkedService.class), + @JsonSubTypes.Type(name = "GoogleBigQueryV2", value = GoogleBigQueryV2LinkedService.class), @JsonSubTypes.Type(name = "Greenplum", value = GreenplumLinkedService.class), @JsonSubTypes.Type(name = "HBase", value = HBaseLinkedService.class), @JsonSubTypes.Type(name = "Hive", value = HiveLinkedService.class), @@ -145,7 +147,8 @@ @JsonSubTypes.Type(name = "LakeHouse", value = LakeHouseLinkedService.class), @JsonSubTypes.Type(name = "SalesforceV2", value = SalesforceV2LinkedService.class), @JsonSubTypes.Type(name = "SalesforceServiceCloudV2", value = SalesforceServiceCloudV2LinkedService.class), - @JsonSubTypes.Type(name = "Warehouse", value = WarehouseLinkedService.class) }) + @JsonSubTypes.Type(name = "Warehouse", value = WarehouseLinkedService.class), + @JsonSubTypes.Type(name = "ServiceNowV2", value = ServiceNowV2LinkedService.class) }) @Fluent public class LinkedService { /* diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2LinkedService.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2LinkedService.java new file mode 100644 index 0000000000000..f3ae175caa6f7 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2LinkedService.java @@ -0,0 +1,552 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.datafactory.fluent.models.PostgreSqlV2LinkedServiceTypeProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.util.List; +import java.util.Map; + +/** + * Linked service for PostgreSQLV2 data source. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("PostgreSqlV2") +@Fluent +public final class PostgreSqlV2LinkedService extends LinkedService { + /* + * PostgreSQLV2 linked service properties. + */ + @JsonProperty(value = "typeProperties", required = true) + private PostgreSqlV2LinkedServiceTypeProperties innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + + /** + * Creates an instance of PostgreSqlV2LinkedService class. + */ + public PostgreSqlV2LinkedService() { + } + + /** + * Get the innerTypeProperties property: PostgreSQLV2 linked service properties. + * + * @return the innerTypeProperties value. + */ + private PostgreSqlV2LinkedServiceTypeProperties innerTypeProperties() { + return this.innerTypeProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2LinkedService withConnectVia(IntegrationRuntimeReference connectVia) { + super.withConnectVia(connectVia); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2LinkedService withDescription(String description) { + super.withDescription(description); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2LinkedService withParameters(Map parameters) { + super.withParameters(parameters); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2LinkedService withAnnotations(List annotations) { + super.withAnnotations(annotations); + return this; + } + + /** + * Get the server property: Server name for connection. Type: string. + * + * @return the server value. + */ + public Object server() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().server(); + } + + /** + * Set the server property: Server name for connection. Type: string. + * + * @param server the server value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withServer(Object server) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withServer(server); + return this; + } + + /** + * Get the port property: The port for the connection. Type: integer. + * + * @return the port value. + */ + public Object port() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().port(); + } + + /** + * Set the port property: The port for the connection. Type: integer. + * + * @param port the port value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withPort(Object port) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withPort(port); + return this; + } + + /** + * Get the username property: Username for authentication. Type: string. + * + * @return the username value. + */ + public Object username() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().username(); + } + + /** + * Set the username property: Username for authentication. Type: string. + * + * @param username the username value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withUsername(Object username) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withUsername(username); + return this; + } + + /** + * Get the database property: Database name for connection. Type: string. + * + * @return the database value. + */ + public Object database() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().database(); + } + + /** + * Set the database property: Database name for connection. Type: string. + * + * @param database the database value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withDatabase(Object database) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withDatabase(database); + return this; + } + + /** + * Get the sslMode property: SSL mode for connection. Type: integer. 0: disable, 1:allow, 2: prefer, 3: require, 4: + * verify-ca, 5: verify-full. Type: integer. + * + * @return the sslMode value. + */ + public Object sslMode() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().sslMode(); + } + + /** + * Set the sslMode property: SSL mode for connection. Type: integer. 0: disable, 1:allow, 2: prefer, 3: require, 4: + * verify-ca, 5: verify-full. Type: integer. + * + * @param sslMode the sslMode value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withSslMode(Object sslMode) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withSslMode(sslMode); + return this; + } + + /** + * Get the schema property: Sets the schema search path. Type: string. + * + * @return the schema value. + */ + public Object schema() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().schema(); + } + + /** + * Set the schema property: Sets the schema search path. Type: string. + * + * @param schema the schema value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withSchema(Object schema) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withSchema(schema); + return this; + } + + /** + * Get the pooling property: Whether connection pooling should be used. Type: boolean. + * + * @return the pooling value. + */ + public Object pooling() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().pooling(); + } + + /** + * Set the pooling property: Whether connection pooling should be used. Type: boolean. + * + * @param pooling the pooling value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withPooling(Object pooling) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withPooling(pooling); + return this; + } + + /** + * Get the connectionTimeout property: The time to wait (in seconds) while trying to establish a connection before + * terminating the attempt and generating an error. Type: integer. + * + * @return the connectionTimeout value. + */ + public Object connectionTimeout() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().connectionTimeout(); + } + + /** + * Set the connectionTimeout property: The time to wait (in seconds) while trying to establish a connection before + * terminating the attempt and generating an error. Type: integer. + * + * @param connectionTimeout the connectionTimeout value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withConnectionTimeout(Object connectionTimeout) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withConnectionTimeout(connectionTimeout); + return this; + } + + /** + * Get the commandTimeout property: The time to wait (in seconds) while trying to execute a command before + * terminating the attempt and generating an error. Set to zero for infinity. Type: integer. + * + * @return the commandTimeout value. + */ + public Object commandTimeout() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().commandTimeout(); + } + + /** + * Set the commandTimeout property: The time to wait (in seconds) while trying to execute a command before + * terminating the attempt and generating an error. Set to zero for infinity. Type: integer. + * + * @param commandTimeout the commandTimeout value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withCommandTimeout(Object commandTimeout) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withCommandTimeout(commandTimeout); + return this; + } + + /** + * Get the trustServerCertificate property: Whether to trust the server certificate without validating it. Type: + * boolean. + * + * @return the trustServerCertificate value. + */ + public Object trustServerCertificate() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().trustServerCertificate(); + } + + /** + * Set the trustServerCertificate property: Whether to trust the server certificate without validating it. Type: + * boolean. + * + * @param trustServerCertificate the trustServerCertificate value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withTrustServerCertificate(Object trustServerCertificate) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withTrustServerCertificate(trustServerCertificate); + return this; + } + + /** + * Get the sslCertificate property: Location of a client certificate to be sent to the server. Type: string. + * + * @return the sslCertificate value. + */ + public Object sslCertificate() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().sslCertificate(); + } + + /** + * Set the sslCertificate property: Location of a client certificate to be sent to the server. Type: string. + * + * @param sslCertificate the sslCertificate value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withSslCertificate(Object sslCertificate) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withSslCertificate(sslCertificate); + return this; + } + + /** + * Get the sslKey property: Location of a client key for a client certificate to be sent to the server. Type: + * string. + * + * @return the sslKey value. + */ + public Object sslKey() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().sslKey(); + } + + /** + * Set the sslKey property: Location of a client key for a client certificate to be sent to the server. Type: + * string. + * + * @param sslKey the sslKey value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withSslKey(Object sslKey) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withSslKey(sslKey); + return this; + } + + /** + * Get the sslPassword property: Password for a key for a client certificate. Type: string. + * + * @return the sslPassword value. + */ + public Object sslPassword() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().sslPassword(); + } + + /** + * Set the sslPassword property: Password for a key for a client certificate. Type: string. + * + * @param sslPassword the sslPassword value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withSslPassword(Object sslPassword) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withSslPassword(sslPassword); + return this; + } + + /** + * Get the readBufferSize property: Determines the size of the internal buffer uses when reading. Increasing may + * improve performance if transferring large values from the database. Type: integer. + * + * @return the readBufferSize value. + */ + public Object readBufferSize() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().readBufferSize(); + } + + /** + * Set the readBufferSize property: Determines the size of the internal buffer uses when reading. Increasing may + * improve performance if transferring large values from the database. Type: integer. + * + * @param readBufferSize the readBufferSize value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withReadBufferSize(Object readBufferSize) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withReadBufferSize(readBufferSize); + return this; + } + + /** + * Get the logParameters property: When enabled, parameter values are logged when commands are executed. Type: + * boolean. + * + * @return the logParameters value. + */ + public Object logParameters() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().logParameters(); + } + + /** + * Set the logParameters property: When enabled, parameter values are logged when commands are executed. Type: + * boolean. + * + * @param logParameters the logParameters value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withLogParameters(Object logParameters) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withLogParameters(logParameters); + return this; + } + + /** + * Get the timezone property: Gets or sets the session timezone. Type: string. + * + * @return the timezone value. + */ + public Object timezone() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().timezone(); + } + + /** + * Set the timezone property: Gets or sets the session timezone. Type: string. + * + * @param timezone the timezone value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withTimezone(Object timezone) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withTimezone(timezone); + return this; + } + + /** + * Get the encoding property: Gets or sets the .NET encoding that will be used to encode/decode PostgreSQL string + * data. Type: string. + * + * @return the encoding value. + */ + public Object encoding() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().encoding(); + } + + /** + * Set the encoding property: Gets or sets the .NET encoding that will be used to encode/decode PostgreSQL string + * data. Type: string. + * + * @param encoding the encoding value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withEncoding(Object encoding) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withEncoding(encoding); + return this; + } + + /** + * Get the password property: The Azure key vault secret reference of password in connection string. Type: string. + * + * @return the password value. + */ + public AzureKeyVaultSecretReference password() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().password(); + } + + /** + * Set the password property: The Azure key vault secret reference of password in connection string. Type: string. + * + * @param password the password value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withPassword(AzureKeyVaultSecretReference password) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withPassword(password); + return this; + } + + /** + * Get the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @return the encryptedCredential value. + */ + public String encryptedCredential() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().encryptedCredential(); + } + + /** + * Set the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @param encryptedCredential the encryptedCredential value to set. + * @return the PostgreSqlV2LinkedService object itself. + */ + public PostgreSqlV2LinkedService withEncryptedCredential(String encryptedCredential) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withEncryptedCredential(encryptedCredential); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerTypeProperties() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerTypeProperties in model PostgreSqlV2LinkedService")); + } else { + innerTypeProperties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(PostgreSqlV2LinkedService.class); +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2Source.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2Source.java new file mode 100644 index 0000000000000..275110a29d8e2 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2Source.java @@ -0,0 +1,114 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * A copy activity source for PostgreSQL databases. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("PostgreSqlV2Source") +@Fluent +public final class PostgreSqlV2Source extends TabularSource { + /* + * Database query. Type: string (or Expression with resultType string). + */ + @JsonProperty(value = "query") + private Object query; + + /** + * Creates an instance of PostgreSqlV2Source class. + */ + public PostgreSqlV2Source() { + } + + /** + * Get the query property: Database query. Type: string (or Expression with resultType string). + * + * @return the query value. + */ + public Object query() { + return this.query; + } + + /** + * Set the query property: Database query. Type: string (or Expression with resultType string). + * + * @param query the query value to set. + * @return the PostgreSqlV2Source object itself. + */ + public PostgreSqlV2Source withQuery(Object query) { + this.query = query; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2Source withQueryTimeout(Object queryTimeout) { + super.withQueryTimeout(queryTimeout); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2Source withAdditionalColumns(Object additionalColumns) { + super.withAdditionalColumns(additionalColumns); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2Source withSourceRetryCount(Object sourceRetryCount) { + super.withSourceRetryCount(sourceRetryCount); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2Source withSourceRetryWait(Object sourceRetryWait) { + super.withSourceRetryWait(sourceRetryWait); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2Source withMaxConcurrentConnections(Object maxConcurrentConnections) { + super.withMaxConcurrentConnections(maxConcurrentConnections); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2Source withDisableMetricsCollection(Object disableMetricsCollection) { + super.withDisableMetricsCollection(disableMetricsCollection); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2TableDataset.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2TableDataset.java new file mode 100644 index 0000000000000..63457c3b29c51 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/PostgreSqlV2TableDataset.java @@ -0,0 +1,164 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.datafactory.fluent.models.PostgreSqlV2TableDatasetTypeProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.util.List; +import java.util.Map; + +/** + * The PostgreSQLV2 table dataset. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("PostgreSqlV2Table") +@Fluent +public final class PostgreSqlV2TableDataset extends Dataset { + /* + * PostgreSQLV2 table dataset properties. + */ + @JsonProperty(value = "typeProperties") + private PostgreSqlV2TableDatasetTypeProperties innerTypeProperties; + + /** + * Creates an instance of PostgreSqlV2TableDataset class. + */ + public PostgreSqlV2TableDataset() { + } + + /** + * Get the innerTypeProperties property: PostgreSQLV2 table dataset properties. + * + * @return the innerTypeProperties value. + */ + private PostgreSqlV2TableDatasetTypeProperties innerTypeProperties() { + return this.innerTypeProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withDescription(String description) { + super.withDescription(description); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withStructure(Object structure) { + super.withStructure(structure); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withSchema(Object schema) { + super.withSchema(schema); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withLinkedServiceName(LinkedServiceReference linkedServiceName) { + super.withLinkedServiceName(linkedServiceName); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withParameters(Map parameters) { + super.withParameters(parameters); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withAnnotations(List annotations) { + super.withAnnotations(annotations); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PostgreSqlV2TableDataset withFolder(DatasetFolder folder) { + super.withFolder(folder); + return this; + } + + /** + * Get the table property: The PostgreSQL table name. Type: string (or Expression with resultType string). + * + * @return the table value. + */ + public Object table() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().table(); + } + + /** + * Set the table property: The PostgreSQL table name. Type: string (or Expression with resultType string). + * + * @param table the table value to set. + * @return the PostgreSqlV2TableDataset object itself. + */ + public PostgreSqlV2TableDataset withTable(Object table) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2TableDatasetTypeProperties(); + } + this.innerTypeProperties().withTable(table); + return this; + } + + /** + * Get the schema property: The PostgreSQL schema name. Type: string (or Expression with resultType string). + * + * @return the schema value. + */ + public Object schemaTypePropertiesSchema() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().schema(); + } + + /** + * Set the schema property: The PostgreSQL schema name. Type: string (or Expression with resultType string). + * + * @param schema the schema value to set. + * @return the PostgreSqlV2TableDataset object itself. + */ + public PostgreSqlV2TableDataset withSchemaTypePropertiesSchema(Object schema) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new PostgreSqlV2TableDatasetTypeProperties(); + } + this.innerTypeProperties().withSchema(schema); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerTypeProperties() != null) { + innerTypeProperties().validate(); + } + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2AuthenticationType.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2AuthenticationType.java new file mode 100644 index 0000000000000..93e0836a99608 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2AuthenticationType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The authentication type to use. + */ +public final class ServiceNowV2AuthenticationType extends ExpandableStringEnum { + /** + * Static value Basic for ServiceNowV2AuthenticationType. + */ + public static final ServiceNowV2AuthenticationType BASIC = fromString("Basic"); + + /** + * Static value OAuth2 for ServiceNowV2AuthenticationType. + */ + public static final ServiceNowV2AuthenticationType OAUTH2 = fromString("OAuth2"); + + /** + * Creates a new instance of ServiceNowV2AuthenticationType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ServiceNowV2AuthenticationType() { + } + + /** + * Creates or finds a ServiceNowV2AuthenticationType from its string representation. + * + * @param name a name to look for. + * @return the corresponding ServiceNowV2AuthenticationType. + */ + @JsonCreator + public static ServiceNowV2AuthenticationType fromString(String name) { + return fromString(name, ServiceNowV2AuthenticationType.class); + } + + /** + * Gets known ServiceNowV2AuthenticationType values. + * + * @return known ServiceNowV2AuthenticationType values. + */ + public static Collection values() { + return values(ServiceNowV2AuthenticationType.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2LinkedService.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2LinkedService.java new file mode 100644 index 0000000000000..6c9e08fbdb3ec --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2LinkedService.java @@ -0,0 +1,285 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.datafactory.fluent.models.ServiceNowV2LinkedServiceTypeProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.util.List; +import java.util.Map; + +/** + * ServiceNowV2 server linked service. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("ServiceNowV2") +@Fluent +public final class ServiceNowV2LinkedService extends LinkedService { + /* + * ServiceNowV2 server linked service properties. + */ + @JsonProperty(value = "typeProperties", required = true) + private ServiceNowV2LinkedServiceTypeProperties innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + + /** + * Creates an instance of ServiceNowV2LinkedService class. + */ + public ServiceNowV2LinkedService() { + } + + /** + * Get the innerTypeProperties property: ServiceNowV2 server linked service properties. + * + * @return the innerTypeProperties value. + */ + private ServiceNowV2LinkedServiceTypeProperties innerTypeProperties() { + return this.innerTypeProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2LinkedService withConnectVia(IntegrationRuntimeReference connectVia) { + super.withConnectVia(connectVia); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2LinkedService withDescription(String description) { + super.withDescription(description); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2LinkedService withParameters(Map parameters) { + super.withParameters(parameters); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2LinkedService withAnnotations(List annotations) { + super.withAnnotations(annotations); + return this; + } + + /** + * Get the endpoint property: The endpoint of the ServiceNowV2 server. (i.e. <instance>.service-now.com). + * + * @return the endpoint value. + */ + public Object endpoint() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().endpoint(); + } + + /** + * Set the endpoint property: The endpoint of the ServiceNowV2 server. (i.e. <instance>.service-now.com). + * + * @param endpoint the endpoint value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withEndpoint(Object endpoint) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withEndpoint(endpoint); + return this; + } + + /** + * Get the authenticationType property: The authentication type to use. + * + * @return the authenticationType value. + */ + public ServiceNowV2AuthenticationType authenticationType() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().authenticationType(); + } + + /** + * Set the authenticationType property: The authentication type to use. + * + * @param authenticationType the authenticationType value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withAuthenticationType(ServiceNowV2AuthenticationType authenticationType) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withAuthenticationType(authenticationType); + return this; + } + + /** + * Get the username property: The user name used to connect to the ServiceNowV2 server for Basic and OAuth2 + * authentication. + * + * @return the username value. + */ + public Object username() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().username(); + } + + /** + * Set the username property: The user name used to connect to the ServiceNowV2 server for Basic and OAuth2 + * authentication. + * + * @param username the username value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withUsername(Object username) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withUsername(username); + return this; + } + + /** + * Get the password property: The password corresponding to the user name for Basic and OAuth2 authentication. + * + * @return the password value. + */ + public SecretBase password() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().password(); + } + + /** + * Set the password property: The password corresponding to the user name for Basic and OAuth2 authentication. + * + * @param password the password value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withPassword(SecretBase password) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withPassword(password); + return this; + } + + /** + * Get the clientId property: The client id for OAuth2 authentication. + * + * @return the clientId value. + */ + public Object clientId() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().clientId(); + } + + /** + * Set the clientId property: The client id for OAuth2 authentication. + * + * @param clientId the clientId value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withClientId(Object clientId) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withClientId(clientId); + return this; + } + + /** + * Get the clientSecret property: The client secret for OAuth2 authentication. + * + * @return the clientSecret value. + */ + public SecretBase clientSecret() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().clientSecret(); + } + + /** + * Set the clientSecret property: The client secret for OAuth2 authentication. + * + * @param clientSecret the clientSecret value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withClientSecret(SecretBase clientSecret) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withClientSecret(clientSecret); + return this; + } + + /** + * Get the grantType property: GrantType for OAuth2 authentication. Default value is password. + * + * @return the grantType value. + */ + public Object grantType() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().grantType(); + } + + /** + * Set the grantType property: GrantType for OAuth2 authentication. Default value is password. + * + * @param grantType the grantType value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withGrantType(Object grantType) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withGrantType(grantType); + return this; + } + + /** + * Get the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @return the encryptedCredential value. + */ + public String encryptedCredential() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().encryptedCredential(); + } + + /** + * Set the encryptedCredential property: The encrypted credential used for authentication. Credentials are + * encrypted using the integration runtime credential manager. Type: string. + * + * @param encryptedCredential the encryptedCredential value to set. + * @return the ServiceNowV2LinkedService object itself. + */ + public ServiceNowV2LinkedService withEncryptedCredential(String encryptedCredential) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new ServiceNowV2LinkedServiceTypeProperties(); + } + this.innerTypeProperties().withEncryptedCredential(encryptedCredential); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerTypeProperties() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerTypeProperties in model ServiceNowV2LinkedService")); + } else { + innerTypeProperties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(ServiceNowV2LinkedService.class); +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2ObjectDataset.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2ObjectDataset.java new file mode 100644 index 0000000000000..b13ebd44cb027 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2ObjectDataset.java @@ -0,0 +1,141 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.datafactory.fluent.models.GenericDatasetTypeProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.util.List; +import java.util.Map; + +/** + * ServiceNowV2 server dataset. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("ServiceNowV2Object") +@Fluent +public final class ServiceNowV2ObjectDataset extends Dataset { + /* + * Properties specific to this dataset type. + */ + @JsonProperty(value = "typeProperties") + private GenericDatasetTypeProperties innerTypeProperties; + + /** + * Creates an instance of ServiceNowV2ObjectDataset class. + */ + public ServiceNowV2ObjectDataset() { + } + + /** + * Get the innerTypeProperties property: Properties specific to this dataset type. + * + * @return the innerTypeProperties value. + */ + private GenericDatasetTypeProperties innerTypeProperties() { + return this.innerTypeProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withDescription(String description) { + super.withDescription(description); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withStructure(Object structure) { + super.withStructure(structure); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withSchema(Object schema) { + super.withSchema(schema); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withLinkedServiceName(LinkedServiceReference linkedServiceName) { + super.withLinkedServiceName(linkedServiceName); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withParameters(Map parameters) { + super.withParameters(parameters); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withAnnotations(List annotations) { + super.withAnnotations(annotations); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2ObjectDataset withFolder(DatasetFolder folder) { + super.withFolder(folder); + return this; + } + + /** + * Get the tableName property: The table name. Type: string (or Expression with resultType string). + * + * @return the tableName value. + */ + public Object tableName() { + return this.innerTypeProperties() == null ? null : this.innerTypeProperties().tableName(); + } + + /** + * Set the tableName property: The table name. Type: string (or Expression with resultType string). + * + * @param tableName the tableName value to set. + * @return the ServiceNowV2ObjectDataset object itself. + */ + public ServiceNowV2ObjectDataset withTableName(Object tableName) { + if (this.innerTypeProperties() == null) { + this.innerTypeProperties = new GenericDatasetTypeProperties(); + } + this.innerTypeProperties().withTableName(tableName); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerTypeProperties() != null) { + innerTypeProperties().validate(); + } + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2Source.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2Source.java new file mode 100644 index 0000000000000..48b2b95b68322 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/ServiceNowV2Source.java @@ -0,0 +1,117 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * A copy activity ServiceNowV2 server source. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") +@JsonTypeName("ServiceNowV2Source") +@Fluent +public final class ServiceNowV2Source extends TabularSource { + /* + * Expression to filter data from source. + */ + @JsonProperty(value = "expression") + private ExpressionV2 expression; + + /** + * Creates an instance of ServiceNowV2Source class. + */ + public ServiceNowV2Source() { + } + + /** + * Get the expression property: Expression to filter data from source. + * + * @return the expression value. + */ + public ExpressionV2 expression() { + return this.expression; + } + + /** + * Set the expression property: Expression to filter data from source. + * + * @param expression the expression value to set. + * @return the ServiceNowV2Source object itself. + */ + public ServiceNowV2Source withExpression(ExpressionV2 expression) { + this.expression = expression; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2Source withQueryTimeout(Object queryTimeout) { + super.withQueryTimeout(queryTimeout); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2Source withAdditionalColumns(Object additionalColumns) { + super.withAdditionalColumns(additionalColumns); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2Source withSourceRetryCount(Object sourceRetryCount) { + super.withSourceRetryCount(sourceRetryCount); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2Source withSourceRetryWait(Object sourceRetryWait) { + super.withSourceRetryWait(sourceRetryWait); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2Source withMaxConcurrentConnections(Object maxConcurrentConnections) { + super.withMaxConcurrentConnections(maxConcurrentConnections); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public ServiceNowV2Source withDisableMetricsCollection(Object disableMetricsCollection) { + super.withDisableMetricsCollection(disableMetricsCollection); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (expression() != null) { + expression().validate(); + } + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/TabularSource.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/TabularSource.java index 51c17b7ef0190..4dd18c2e1090a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/TabularSource.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/com/azure/resourcemanager/datafactory/models/TabularSource.java @@ -26,6 +26,7 @@ @JsonSubTypes.Type(name = "OdbcSource", value = OdbcSource.class), @JsonSubTypes.Type(name = "MySqlSource", value = MySqlSource.class), @JsonSubTypes.Type(name = "PostgreSqlSource", value = PostgreSqlSource.class), + @JsonSubTypes.Type(name = "PostgreSqlV2Source", value = PostgreSqlV2Source.class), @JsonSubTypes.Type(name = "SybaseSource", value = SybaseSource.class), @JsonSubTypes.Type(name = "SapBwSource", value = SapBwSource.class), @JsonSubTypes.Type(name = "SalesforceSource", value = SalesforceSource.class), @@ -51,6 +52,7 @@ @JsonSubTypes.Type(name = "DrillSource", value = DrillSource.class), @JsonSubTypes.Type(name = "EloquaSource", value = EloquaSource.class), @JsonSubTypes.Type(name = "GoogleBigQuerySource", value = GoogleBigQuerySource.class), + @JsonSubTypes.Type(name = "GoogleBigQueryV2Source", value = GoogleBigQueryV2Source.class), @JsonSubTypes.Type(name = "GreenplumSource", value = GreenplumSource.class), @JsonSubTypes.Type(name = "HBaseSource", value = HBaseSource.class), @JsonSubTypes.Type(name = "HiveSource", value = HiveSource.class), @@ -80,7 +82,8 @@ @JsonSubTypes.Type(name = "GoogleAdWordsSource", value = GoogleAdWordsSource.class), @JsonSubTypes.Type(name = "AmazonRedshiftSource", value = AmazonRedshiftSource.class), @JsonSubTypes.Type(name = "WarehouseSource", value = WarehouseSource.class), - @JsonSubTypes.Type(name = "SalesforceV2Source", value = SalesforceV2Source.class) }) + @JsonSubTypes.Type(name = "SalesforceV2Source", value = SalesforceV2Source.class), + @JsonSubTypes.Type(name = "ServiceNowV2Source", value = ServiceNowV2Source.class) }) @Fluent public class TabularSource extends CopySource { /* diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/module-info.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/module-info.java index 075a711ac7ca4..cbdb53ef51f9a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/module-info.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/java/module-info.java @@ -4,12 +4,10 @@ module com.azure.resourcemanager.datafactory { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.datafactory; exports com.azure.resourcemanager.datafactory.fluent; exports com.azure.resourcemanager.datafactory.fluent.models; exports com.azure.resourcemanager.datafactory.models; - opens com.azure.resourcemanager.datafactory.fluent.models to com.azure.core, com.fasterxml.jackson.databind; opens com.azure.resourcemanager.datafactory.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-datafactory/reflect-config.json b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-datafactory/reflect-config.json index 8cb0e1ac1a10b..51959e24378f7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-datafactory/reflect-config.json +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-datafactory/reflect-config.json @@ -1418,6 +1418,16 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.PostgreSqlV2TableDataset", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.fluent.models.PostgreSqlV2TableDatasetTypeProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.MicrosoftAccessTableDataset", "allDeclaredConstructors" : true, @@ -1638,6 +1648,16 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2ObjectDataset", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.fluent.models.GoogleBigQueryV2DatasetTypeProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.GreenplumTableDataset", "allDeclaredConstructors" : true, @@ -1903,6 +1923,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.ServiceNowV2ObjectDataset", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.ManagedIntegrationRuntime", "allDeclaredConstructors" : true, @@ -2373,6 +2398,16 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.PostgreSqlV2LinkedService", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.fluent.models.PostgreSqlV2LinkedServiceTypeProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.SybaseLinkedService", "allDeclaredConstructors" : true, @@ -2893,6 +2928,16 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2LinkedService", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.fluent.models.GoogleBigQueryV2LinkedServiceTypeProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.GreenplumLinkedService", "allDeclaredConstructors" : true, @@ -3318,6 +3363,16 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.ServiceNowV2LinkedService", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.fluent.models.ServiceNowV2LinkedServiceTypeProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.ControlActivity", "allDeclaredConstructors" : true, @@ -3723,6 +3778,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.PostgreSqlV2Source", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.SybaseSource", "allDeclaredConstructors" : true, @@ -3983,6 +4043,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2Source", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.GreenplumSource", "allDeclaredConstructors" : true, @@ -4938,6 +5003,16 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.ServiceNowV2Source", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.ExpressionV2", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.MultiplePipelineTrigger", "allDeclaredConstructors" : true, @@ -5338,6 +5413,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2AuthenticationType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.HBaseAuthenticationType", "allDeclaredConstructors" : true, @@ -5403,6 +5483,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.ServiceNowV2AuthenticationType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.CassandraSourceReadConsistencyLevels", "allDeclaredConstructors" : true, @@ -5518,6 +5603,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.datafactory.models.ExpressionV2Type", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.datafactory.models.RecurrenceFrequency", "allDeclaredConstructors" : true, diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ActivityPolicyTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ActivityPolicyTests.java index 2f4dcceec324e..f8673593b5520 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ActivityPolicyTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ActivityPolicyTests.java @@ -14,21 +14,21 @@ public final class ActivityPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ActivityPolicy model = BinaryData.fromString( - "{\"timeout\":\"datajsaoggd\",\"retry\":\"datajkoxlccjdooy\",\"retryIntervalInSeconds\":975110556,\"secureInput\":false,\"secureOutput\":true,\"\":{\"bwidsbdftbxruuu\":\"datajhqq\",\"wnpwgchl\":\"dataaarrrgjnqkuca\",\"hflnlutacsslel\":\"databttxfittheaxq\"}}") + "{\"timeout\":\"datab\",\"retry\":\"dataqwwtevfeugc\",\"retryIntervalInSeconds\":67435770,\"secureInput\":true,\"secureOutput\":true,\"\":{\"dleohysdgkb\":\"datavfrymqqfksqfc\",\"jqztrpjme\":\"dataxygwvtkrqiyuqds\"}}") .toObject(ActivityPolicy.class); - Assertions.assertEquals(975110556, model.retryIntervalInSeconds()); - Assertions.assertEquals(false, model.secureInput()); + Assertions.assertEquals(67435770, model.retryIntervalInSeconds()); + Assertions.assertEquals(true, model.secureInput()); Assertions.assertEquals(true, model.secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ActivityPolicy model = new ActivityPolicy().withTimeout("datajsaoggd").withRetry("datajkoxlccjdooy") - .withRetryIntervalInSeconds(975110556).withSecureInput(false).withSecureOutput(true) + ActivityPolicy model = new ActivityPolicy().withTimeout("datab").withRetry("dataqwwtevfeugc") + .withRetryIntervalInSeconds(67435770).withSecureInput(true).withSecureOutput(true) .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(ActivityPolicy.class); - Assertions.assertEquals(975110556, model.retryIntervalInSeconds()); - Assertions.assertEquals(false, model.secureInput()); + Assertions.assertEquals(67435770, model.retryIntervalInSeconds()); + Assertions.assertEquals(true, model.secureInput()); Assertions.assertEquals(true, model.secureOutput()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsObjectDatasetTests.java index d933833745016..3bbb82691f33e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsObjectDatasetTests.java @@ -19,33 +19,34 @@ public final class AmazonMwsObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonMwsObjectDataset model = BinaryData.fromString( - "{\"type\":\"AmazonMWSObject\",\"typeProperties\":{\"tableName\":\"datan\"},\"description\":\"lxcltjhbcycg\",\"structure\":\"datakcsihxvta\",\"schema\":\"datawf\",\"linkedServiceName\":{\"referenceName\":\"pxpry\",\"parameters\":{\"gugwlux\":\"databubwhzq\",\"mkdhwqcqweba\":\"datahtq\",\"phujeucosvk\":\"datamfpk\",\"llgnueezfpffb\":\"dataeergvypaxpjpy\"}},\"parameters\":{\"gzyojfchicpare\":{\"type\":\"Array\",\"defaultValue\":\"datavmcgm\"},\"ojuxil\":{\"type\":\"Bool\",\"defaultValue\":\"dataksgqhb\"},\"fldfljwt\":{\"type\":\"Object\",\"defaultValue\":\"datalkc\"}},\"annotations\":[\"datatsflotumbm\",\"datagftshfgmuxuqiags\",\"dataoikuqirhsk\",\"datapaowkgvnlfueyxfz\"],\"folder\":{\"name\":\"lrjugcfebpiucenb\"},\"\":{\"lsxr\":\"datalldfknbdzw\"}}") + "{\"type\":\"AmazonMWSObject\",\"typeProperties\":{\"tableName\":\"datagirrpwnqtvuxeuj\"},\"description\":\"ssijuaxxfd\",\"structure\":\"datapkcpws\",\"schema\":\"datannmjun\",\"linkedServiceName\":{\"referenceName\":\"tl\",\"parameters\":{\"kcsihxvta\":\"datatjhbcycgq\",\"zqqgug\":\"datawwfopxpryxnsbubw\"}},\"parameters\":{\"mkdhwqcqweba\":{\"type\":\"SecureString\",\"defaultValue\":\"datahtq\"},\"rgvypa\":{\"type\":\"String\",\"defaultValue\":\"datapkephujeucosvkke\"},\"ueez\":{\"type\":\"String\",\"defaultValue\":\"datapyillg\"},\"zyojfch\":{\"type\":\"Float\",\"defaultValue\":\"datafbuqxknvmcgmb\"}},\"annotations\":[\"dataarex\",\"datao\"],\"folder\":{\"name\":\"qhboojuxilozbl\"},\"\":{\"b\":\"datafldfljwt\",\"gftshfgmuxuqiags\":\"datatsflotumbm\",\"paowkgvnlfueyxfz\":\"dataoikuqirhsk\",\"ce\":\"databrlrjugcfebpi\"}}") .toObject(AmazonMwsObjectDataset.class); - Assertions.assertEquals("lxcltjhbcycg", model.description()); - Assertions.assertEquals("pxpry", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("gzyojfchicpare").type()); - Assertions.assertEquals("lrjugcfebpiucenb", model.folder().name()); + Assertions.assertEquals("ssijuaxxfd", model.description()); + Assertions.assertEquals("tl", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("mkdhwqcqweba").type()); + Assertions.assertEquals("qhboojuxilozbl", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonMwsObjectDataset model = new AmazonMwsObjectDataset().withDescription("lxcltjhbcycg") - .withStructure("datakcsihxvta").withSchema("datawf") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pxpry") - .withParameters(mapOf("gugwlux", "databubwhzq", "mkdhwqcqweba", "datahtq", "phujeucosvk", "datamfpk", - "llgnueezfpffb", "dataeergvypaxpjpy"))) - .withParameters(mapOf("gzyojfchicpare", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datavmcgm"), "ojuxil", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataksgqhb"), "fldfljwt", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datalkc"))) - .withAnnotations( - Arrays.asList("datatsflotumbm", "datagftshfgmuxuqiags", "dataoikuqirhsk", "datapaowkgvnlfueyxfz")) - .withFolder(new DatasetFolder().withName("lrjugcfebpiucenb")).withTableName("datan"); + AmazonMwsObjectDataset model = new AmazonMwsObjectDataset().withDescription("ssijuaxxfd") + .withStructure("datapkcpws").withSchema("datannmjun") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("tl") + .withParameters(mapOf("kcsihxvta", "datatjhbcycgq", "zqqgug", "datawwfopxpryxnsbubw"))) + .withParameters(mapOf("mkdhwqcqweba", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datahtq"), + "rgvypa", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datapkephujeucosvkke"), + "ueez", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datapyillg"), + "zyojfch", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datafbuqxknvmcgmb"))) + .withAnnotations(Arrays.asList("dataarex", "datao")) + .withFolder(new DatasetFolder().withName("qhboojuxilozbl")).withTableName("datagirrpwnqtvuxeuj"); model = BinaryData.fromObject(model).toObject(AmazonMwsObjectDataset.class); - Assertions.assertEquals("lxcltjhbcycg", model.description()); - Assertions.assertEquals("pxpry", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("gzyojfchicpare").type()); - Assertions.assertEquals("lrjugcfebpiucenb", model.folder().name()); + Assertions.assertEquals("ssijuaxxfd", model.description()); + Assertions.assertEquals("tl", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("mkdhwqcqweba").type()); + Assertions.assertEquals("qhboojuxilozbl", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsSourceTests.java index 671bb41f434b3..a222680008a76 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonMwsSourceTests.java @@ -11,16 +11,16 @@ public final class AmazonMwsSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonMwsSource model = BinaryData.fromString( - "{\"type\":\"AmazonMWSSource\",\"query\":\"datatmhwgencmoswcxlg\",\"queryTimeout\":\"datauqxews\",\"additionalColumns\":\"datapifzavctywappa\",\"sourceRetryCount\":\"dataprzrsqcu\",\"sourceRetryWait\":\"datanp\",\"maxConcurrentConnections\":\"dataqlanuhmsrnp\",\"disableMetricsCollection\":\"dataaghoeqiwpdxpd\",\"\":{\"yplhsto\":\"dataajq\",\"kgqjd\":\"datayybwlysvp\",\"xppqpsin\":\"dataglecjfgyivsiirx\"}}") + "{\"type\":\"AmazonMWSSource\",\"query\":\"datapsoeocvywtyehln\",\"queryTimeout\":\"dataqeply\",\"additionalColumns\":\"dataad\",\"sourceRetryCount\":\"datagwdxoxjlvvvz\",\"sourceRetryWait\":\"datajvyintgkve\",\"maxConcurrentConnections\":\"dataeldnmb\",\"disableMetricsCollection\":\"databii\",\"\":{\"jaagfeiw\":\"datakxi\",\"zmsivqegmpfzbrh\":\"datauxqw\",\"tkrsjspziiev\":\"dataqj\",\"auyxyoyjas\":\"datattsz\"}}") .toObject(AmazonMwsSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonMwsSource model = new AmazonMwsSource().withSourceRetryCount("dataprzrsqcu").withSourceRetryWait("datanp") - .withMaxConcurrentConnections("dataqlanuhmsrnp").withDisableMetricsCollection("dataaghoeqiwpdxpd") - .withQueryTimeout("datauqxews").withAdditionalColumns("datapifzavctywappa") - .withQuery("datatmhwgencmoswcxlg"); + AmazonMwsSource model + = new AmazonMwsSource().withSourceRetryCount("datagwdxoxjlvvvz").withSourceRetryWait("datajvyintgkve") + .withMaxConcurrentConnections("dataeldnmb").withDisableMetricsCollection("databii") + .withQueryTimeout("dataqeply").withAdditionalColumns("dataad").withQuery("datapsoeocvywtyehln"); model = BinaryData.fromObject(model).toObject(AmazonMwsSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOraclePartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOraclePartitionSettingsTests.java index dc4c82e5fe640..2acdfbf123225 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOraclePartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOraclePartitionSettingsTests.java @@ -11,15 +11,15 @@ public final class AmazonRdsForOraclePartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonRdsForOraclePartitionSettings model = BinaryData.fromString( - "{\"partitionNames\":\"dataske\",\"partitionColumnName\":\"datawqtcownx\",\"partitionUpperBound\":\"datapptv\",\"partitionLowerBound\":\"datadbnu\"}") + "{\"partitionNames\":\"datafdctgsdxjxkd\",\"partitionColumnName\":\"datao\",\"partitionUpperBound\":\"datalprsrkennnyyvv\",\"partitionLowerBound\":\"datasad\"}") .toObject(AmazonRdsForOraclePartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AmazonRdsForOraclePartitionSettings model = new AmazonRdsForOraclePartitionSettings() - .withPartitionNames("dataske").withPartitionColumnName("datawqtcownx").withPartitionUpperBound("datapptv") - .withPartitionLowerBound("datadbnu"); + .withPartitionNames("datafdctgsdxjxkd").withPartitionColumnName("datao") + .withPartitionUpperBound("datalprsrkennnyyvv").withPartitionLowerBound("datasad"); model = BinaryData.fromObject(model).toObject(AmazonRdsForOraclePartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOracleSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOracleSourceTests.java index a395c9426eb87..99679fbc0ffec 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOracleSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForOracleSourceTests.java @@ -12,20 +12,20 @@ public final class AmazonRdsForOracleSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonRdsForOracleSource model = BinaryData.fromString( - "{\"type\":\"AmazonRdsForOracleSource\",\"oracleReaderQuery\":\"datad\",\"queryTimeout\":\"datak\",\"partitionOption\":\"datazzsi\",\"partitionSettings\":{\"partitionNames\":\"dataosacr\",\"partitionColumnName\":\"datascfkefeltx\",\"partitionUpperBound\":\"dataamimgju\",\"partitionLowerBound\":\"datavtgecehennl\"},\"additionalColumns\":\"datahoukfjwkctdnn\",\"sourceRetryCount\":\"datakqe\",\"sourceRetryWait\":\"datasl\",\"maxConcurrentConnections\":\"datajpuywijnlpe\",\"disableMetricsCollection\":\"dataqbnmzkqydt\",\"\":{\"ylisqlekchjd\":\"datawycmwvphrwufnfo\"}}") + "{\"type\":\"AmazonRdsForOracleSource\",\"oracleReaderQuery\":\"datawgikpdpudqiwhvx\",\"queryTimeout\":\"datavpoeuufw\",\"partitionOption\":\"datadeffrbxzjedy\",\"partitionSettings\":{\"partitionNames\":\"datasxspnmfydphl\",\"partitionColumnName\":\"datano\",\"partitionUpperBound\":\"databdvjlqfzlbpeh\",\"partitionLowerBound\":\"datapgllrh\"},\"additionalColumns\":\"dataxstpg\",\"sourceRetryCount\":\"databezmyjqpdchds\",\"sourceRetryWait\":\"datakmgppxzgjysmtskt\",\"maxConcurrentConnections\":\"datasybwd\",\"disableMetricsCollection\":\"datajsokosugrfizfwd\",\"\":{\"nie\":\"datanwhqafzgzmonj\",\"fmognnw\":\"datactwbim\",\"yrfpnbyxy\":\"datardllrqamfj\",\"xhvzgl\":\"dataubvidpskdzs\"}}") .toObject(AmazonRdsForOracleSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonRdsForOracleSource model - = new AmazonRdsForOracleSource().withSourceRetryCount("datakqe").withSourceRetryWait("datasl") - .withMaxConcurrentConnections("datajpuywijnlpe").withDisableMetricsCollection("dataqbnmzkqydt") - .withOracleReaderQuery("datad").withQueryTimeout("datak").withPartitionOption("datazzsi") - .withPartitionSettings(new AmazonRdsForOraclePartitionSettings().withPartitionNames("dataosacr") - .withPartitionColumnName("datascfkefeltx").withPartitionUpperBound("dataamimgju") - .withPartitionLowerBound("datavtgecehennl")) - .withAdditionalColumns("datahoukfjwkctdnn"); + AmazonRdsForOracleSource model = new AmazonRdsForOracleSource().withSourceRetryCount("databezmyjqpdchds") + .withSourceRetryWait("datakmgppxzgjysmtskt").withMaxConcurrentConnections("datasybwd") + .withDisableMetricsCollection("datajsokosugrfizfwd").withOracleReaderQuery("datawgikpdpudqiwhvx") + .withQueryTimeout("datavpoeuufw").withPartitionOption("datadeffrbxzjedy") + .withPartitionSettings(new AmazonRdsForOraclePartitionSettings().withPartitionNames("datasxspnmfydphl") + .withPartitionColumnName("datano").withPartitionUpperBound("databdvjlqfzlbpeh") + .withPartitionLowerBound("datapgllrh")) + .withAdditionalColumns("dataxstpg"); model = BinaryData.fromObject(model).toObject(AmazonRdsForOracleSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerSourceTests.java index cbd0ee95250c8..3873447c7b879 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerSourceTests.java @@ -12,21 +12,20 @@ public final class AmazonRdsForSqlServerSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonRdsForSqlServerSource model = BinaryData.fromString( - "{\"type\":\"AmazonRdsForSqlServerSource\",\"sqlReaderQuery\":\"datayseyvagakfqpaexl\",\"sqlReaderStoredProcedureName\":\"datamekkaeigrh\",\"storedProcedureParameters\":\"datansgvsrtql\",\"isolationLevel\":\"datawjkraleglpynsbl\",\"produceAdditionalTypes\":\"dataiwrubxe\",\"partitionOption\":\"dataalhbrwaltvky\",\"partitionSettings\":{\"partitionColumnName\":\"dataopqtegkrjolbaegh\",\"partitionUpperBound\":\"datasscismrnneklfi\",\"partitionLowerBound\":\"dataysfclxtf\"},\"queryTimeout\":\"datadwqzbiukzmfyfvy\",\"additionalColumns\":\"dataofaiwlnfvexiuuqa\",\"sourceRetryCount\":\"datalseyxpgkmlnj\",\"sourceRetryWait\":\"dataaywgc\",\"maxConcurrentConnections\":\"datanf\",\"disableMetricsCollection\":\"datapyglnfw\",\"\":{\"xblucpmqwkf\":\"datawv\"}}") + "{\"type\":\"AmazonRdsForSqlServerSource\",\"sqlReaderQuery\":\"datajeypdk\",\"sqlReaderStoredProcedureName\":\"datacxzsynbdrqi\",\"storedProcedureParameters\":\"dataihg\",\"isolationLevel\":\"datahyebwg\",\"produceAdditionalTypes\":\"dataovsvjxnsor\",\"partitionOption\":\"datal\",\"partitionSettings\":{\"partitionColumnName\":\"datalyhgiisn\",\"partitionUpperBound\":\"datax\",\"partitionLowerBound\":\"databo\"},\"queryTimeout\":\"datapyilojwcza\",\"additionalColumns\":\"datawtausk\",\"sourceRetryCount\":\"datahhmtypgrkdmezaun\",\"sourceRetryWait\":\"datacqtigav\",\"maxConcurrentConnections\":\"datasnrjhjlploaeppl\",\"disableMetricsCollection\":\"datakcazuj\",\"\":{\"elaaexcnxrtlnzd\":\"datauzbsxhivnc\",\"zzpmm\":\"databqfea\",\"jxphwynimcp\":\"datais\",\"aceoaboozxkdz\":\"datarakucgjre\"}}") .toObject(AmazonRdsForSqlServerSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonRdsForSqlServerSource model = new AmazonRdsForSqlServerSource().withSourceRetryCount("datalseyxpgkmlnj") - .withSourceRetryWait("dataaywgc").withMaxConcurrentConnections("datanf") - .withDisableMetricsCollection("datapyglnfw").withQueryTimeout("datadwqzbiukzmfyfvy") - .withAdditionalColumns("dataofaiwlnfvexiuuqa").withSqlReaderQuery("datayseyvagakfqpaexl") - .withSqlReaderStoredProcedureName("datamekkaeigrh").withStoredProcedureParameters("datansgvsrtql") - .withIsolationLevel("datawjkraleglpynsbl").withProduceAdditionalTypes("dataiwrubxe") - .withPartitionOption("dataalhbrwaltvky") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("dataopqtegkrjolbaegh") - .withPartitionUpperBound("datasscismrnneklfi").withPartitionLowerBound("dataysfclxtf")); + AmazonRdsForSqlServerSource model = new AmazonRdsForSqlServerSource() + .withSourceRetryCount("datahhmtypgrkdmezaun").withSourceRetryWait("datacqtigav") + .withMaxConcurrentConnections("datasnrjhjlploaeppl").withDisableMetricsCollection("datakcazuj") + .withQueryTimeout("datapyilojwcza").withAdditionalColumns("datawtausk").withSqlReaderQuery("datajeypdk") + .withSqlReaderStoredProcedureName("datacxzsynbdrqi").withStoredProcedureParameters("dataihg") + .withIsolationLevel("datahyebwg").withProduceAdditionalTypes("dataovsvjxnsor").withPartitionOption("datal") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datalyhgiisn") + .withPartitionUpperBound("datax").withPartitionLowerBound("databo")); model = BinaryData.fromObject(model).toObject(AmazonRdsForSqlServerSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTests.java index 92f37bcb39be4..c0b5f3fa3e6e4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTests.java @@ -19,32 +19,34 @@ public final class AmazonRdsForSqlServerTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonRdsForSqlServerTableDataset model = BinaryData.fromString( - "{\"type\":\"AmazonRdsForSqlServerTable\",\"typeProperties\":{\"schema\":\"datamhbtqyzycgcmbkyg\",\"table\":\"datajdqosxzmdzlybqfu\"},\"description\":\"ekzfkicx\",\"structure\":\"dataevmnk\",\"schema\":\"datahvsr\",\"linkedServiceName\":{\"referenceName\":\"jokvlwvbjsa\",\"parameters\":{\"a\":\"datavmf\",\"dhgxgiea\":\"datawbpzgfgqp\"}},\"parameters\":{\"uhwyxjsfmaxcebn\":{\"type\":\"Object\",\"defaultValue\":\"dataxavlozukgs\"},\"xpjpvemdf\":{\"type\":\"Bool\",\"defaultValue\":\"dataskemqqerw\"}},\"annotations\":[\"datatu\",\"databrxz\",\"datahyt\"],\"folder\":{\"name\":\"kjgeecwtfma\"},\"\":{\"sl\":\"datamnhtwofxfmhlvyq\",\"iekhjgqq\":\"dataqrmlq\",\"luwozf\":\"dataugwespscvsmsp\"}}") + "{\"type\":\"AmazonRdsForSqlServerTable\",\"typeProperties\":{\"schema\":\"dataumtcqxmyvkxixypa\",\"table\":\"datafjczgohvpsuwi\"},\"description\":\"m\",\"structure\":\"datazbyfkoc\",\"schema\":\"datazdct\",\"linkedServiceName\":{\"referenceName\":\"nlwsc\",\"parameters\":{\"lks\":\"datatwgxrolwv\"}},\"parameters\":{\"hbvjhxvpmq\":{\"type\":\"Object\",\"defaultValue\":\"dataacuctihavi\"},\"phngr\":{\"type\":\"SecureString\",\"defaultValue\":\"dataux\"},\"icgym\":{\"type\":\"Float\",\"defaultValue\":\"dataxoweorocr\"},\"vhtvijvwmrg\":{\"type\":\"String\",\"defaultValue\":\"dataqpfy\"}},\"annotations\":[\"datahrplcxfmbzquuutq\",\"datahbtqyzy\",\"datag\",\"datambky\"],\"folder\":{\"name\":\"jdqosxzmdzlybqfu\"},\"\":{\"fkicxhsevmnkggh\":\"datak\",\"vbjsarxsvmfp\":\"datasryjokvl\"}}") .toObject(AmazonRdsForSqlServerTableDataset.class); - Assertions.assertEquals("ekzfkicx", model.description()); - Assertions.assertEquals("jokvlwvbjsa", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("uhwyxjsfmaxcebn").type()); - Assertions.assertEquals("kjgeecwtfma", model.folder().name()); + Assertions.assertEquals("m", model.description()); + Assertions.assertEquals("nlwsc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("hbvjhxvpmq").type()); + Assertions.assertEquals("jdqosxzmdzlybqfu", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonRdsForSqlServerTableDataset model = new AmazonRdsForSqlServerTableDataset().withDescription("ekzfkicx") - .withStructure("dataevmnk").withSchema("datahvsr") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("jokvlwvbjsa") - .withParameters(mapOf("a", "datavmf", "dhgxgiea", "datawbpzgfgqp"))) - .withParameters(mapOf("uhwyxjsfmaxcebn", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataxavlozukgs"), - "xpjpvemdf", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataskemqqerw"))) - .withAnnotations(Arrays.asList("datatu", "databrxz", "datahyt")) - .withFolder(new DatasetFolder().withName("kjgeecwtfma")) - .withSchemaTypePropertiesSchema("datamhbtqyzycgcmbkyg").withTable("datajdqosxzmdzlybqfu"); + AmazonRdsForSqlServerTableDataset model = new AmazonRdsForSqlServerTableDataset().withDescription("m") + .withStructure("datazbyfkoc").withSchema("datazdct") + .withLinkedServiceName( + new LinkedServiceReference().withReferenceName("nlwsc").withParameters(mapOf("lks", "datatwgxrolwv"))) + .withParameters(mapOf("hbvjhxvpmq", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataacuctihavi"), "phngr", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("dataux"), "icgym", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataxoweorocr"), + "vhtvijvwmrg", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataqpfy"))) + .withAnnotations(Arrays.asList("datahrplcxfmbzquuutq", "datahbtqyzy", "datag", "datambky")) + .withFolder(new DatasetFolder().withName("jdqosxzmdzlybqfu")) + .withSchemaTypePropertiesSchema("dataumtcqxmyvkxixypa").withTable("datafjczgohvpsuwi"); model = BinaryData.fromObject(model).toObject(AmazonRdsForSqlServerTableDataset.class); - Assertions.assertEquals("ekzfkicx", model.description()); - Assertions.assertEquals("jokvlwvbjsa", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("uhwyxjsfmaxcebn").type()); - Assertions.assertEquals("kjgeecwtfma", model.folder().name()); + Assertions.assertEquals("m", model.description()); + Assertions.assertEquals("nlwsc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("hbvjhxvpmq").type()); + Assertions.assertEquals("jdqosxzmdzlybqfu", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTypePropertiesTests.java index 0856bcc1ab870..66a8fe86190da 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRdsForSqlServerTableDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class AmazonRdsForSqlServerTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonRdsForSqlServerTableDatasetTypeProperties model - = BinaryData.fromString("{\"schema\":\"dataasupcvqgxcvwio\",\"table\":\"datacmcgmlmpnvq\"}") + = BinaryData.fromString("{\"schema\":\"datawbpzgfgqp\",\"table\":\"datahgxg\"}") .toObject(AmazonRdsForSqlServerTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonRdsForSqlServerTableDatasetTypeProperties model = new AmazonRdsForSqlServerTableDatasetTypeProperties() - .withSchema("dataasupcvqgxcvwio").withTable("datacmcgmlmpnvq"); + AmazonRdsForSqlServerTableDatasetTypeProperties model + = new AmazonRdsForSqlServerTableDatasetTypeProperties().withSchema("datawbpzgfgqp").withTable("datahgxg"); model = BinaryData.fromObject(model).toObject(AmazonRdsForSqlServerTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRedshiftSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRedshiftSourceTests.java index 1aa9c4f6a3fbc..b5d9d0384828a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRedshiftSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonRedshiftSourceTests.java @@ -16,23 +16,23 @@ public final class AmazonRedshiftSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonRedshiftSource model = BinaryData.fromString( - "{\"type\":\"AmazonRedshiftSource\",\"query\":\"datavfgmezfyelfxlbkb\",\"redshiftUnloadSettings\":{\"s3LinkedServiceName\":{\"referenceName\":\"dkqfjzgyzj\",\"parameters\":{\"fi\":\"datakhgatynkih\"}},\"bucketName\":\"dataybtowj\"},\"queryTimeout\":\"dataopzaen\",\"additionalColumns\":\"datajxztgduw\",\"sourceRetryCount\":\"datav\",\"sourceRetryWait\":\"dataaolfgvtczzvqz\",\"maxConcurrentConnections\":\"datameuip\",\"disableMetricsCollection\":\"datadw\",\"\":{\"abjuaktshw\":\"datarpsjkq\",\"qanbadkzpqc\":\"datapciokdrjdeyf\",\"jexqnrnzlalu\":\"datauplp\"}}") + "{\"type\":\"AmazonRedshiftSource\",\"query\":\"datayipjzgmxqaupy\",\"redshiftUnloadSettings\":{\"s3LinkedServiceName\":{\"referenceName\":\"goyp\",\"parameters\":{\"zjnlfucmhono\":\"dataoyyfysn\",\"elmmx\":\"dataeseuq\"}},\"bucketName\":\"datam\"},\"queryTimeout\":\"dataiyc\",\"additionalColumns\":\"datajswedkfofyfwpu\",\"sourceRetryCount\":\"datapcutzlvxkolvedz\",\"sourceRetryWait\":\"datakrpormvddwgozrd\",\"maxConcurrentConnections\":\"datakmgcxmkrldfoidyh\",\"disableMetricsCollection\":\"datacgv\",\"\":{\"qgysxpaapmt\":\"datasexenywwkdx\",\"ocadtnmqrpjecj\":\"datadqvcifwknlytvxr\"}}") .toObject(AmazonRedshiftSource.class); - Assertions.assertEquals("dkqfjzgyzj", model.redshiftUnloadSettings().s3LinkedServiceName().referenceName()); + Assertions.assertEquals("goyp", model.redshiftUnloadSettings().s3LinkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonRedshiftSource model - = new AmazonRedshiftSource().withSourceRetryCount("datav").withSourceRetryWait("dataaolfgvtczzvqz") - .withMaxConcurrentConnections("datameuip").withDisableMetricsCollection("datadw") - .withQueryTimeout("dataopzaen").withAdditionalColumns("datajxztgduw").withQuery("datavfgmezfyelfxlbkb") - .withRedshiftUnloadSettings(new RedshiftUnloadSettings() - .withS3LinkedServiceName(new LinkedServiceReference().withReferenceName("dkqfjzgyzj") - .withParameters(mapOf("fi", "datakhgatynkih"))) - .withBucketName("dataybtowj")); + AmazonRedshiftSource model = new AmazonRedshiftSource().withSourceRetryCount("datapcutzlvxkolvedz") + .withSourceRetryWait("datakrpormvddwgozrd").withMaxConcurrentConnections("datakmgcxmkrldfoidyh") + .withDisableMetricsCollection("datacgv").withQueryTimeout("dataiyc") + .withAdditionalColumns("datajswedkfofyfwpu").withQuery("datayipjzgmxqaupy") + .withRedshiftUnloadSettings(new RedshiftUnloadSettings() + .withS3LinkedServiceName(new LinkedServiceReference().withReferenceName("goyp") + .withParameters(mapOf("zjnlfucmhono", "dataoyyfysn", "elmmx", "dataeseuq"))) + .withBucketName("datam")); model = BinaryData.fromObject(model).toObject(AmazonRedshiftSource.class); - Assertions.assertEquals("dkqfjzgyzj", model.redshiftUnloadSettings().s3LinkedServiceName().referenceName()); + Assertions.assertEquals("goyp", model.redshiftUnloadSettings().s3LinkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3CompatibleReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3CompatibleReadSettingsTests.java index d36323372636f..732aa2c05131f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3CompatibleReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3CompatibleReadSettingsTests.java @@ -11,19 +11,18 @@ public final class AmazonS3CompatibleReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonS3CompatibleReadSettings model = BinaryData.fromString( - "{\"type\":\"AmazonS3CompatibleReadSettings\",\"recursive\":\"datawwtevfeu\",\"wildcardFolderPath\":\"datawtsztxoswvfrymq\",\"wildcardFileName\":\"dataksq\",\"prefix\":\"dataxdleohysdgkbaxy\",\"fileListPath\":\"datavtkrqiyuqd\",\"enablePartitionDiscovery\":\"datajqztrpjme\",\"partitionRootPath\":\"datafotaaqyx\",\"deleteFilesAfterCompletion\":\"dataoabcoxqaavjkre\",\"modifiedDatetimeStart\":\"dataasv\",\"modifiedDatetimeEnd\":\"datasbvolivianklqclf\",\"maxConcurrentConnections\":\"dataneouxpdnlbp\",\"disableMetricsCollection\":\"datackohn\",\"\":{\"gurwxfspxghw\":\"dataqzek\",\"w\":\"dataiyuofegrzfsfuloo\",\"qsylkkqvmmm\":\"datazotjbvhuidlod\"}}") + "{\"type\":\"AmazonS3CompatibleReadSettings\",\"recursive\":\"datakf\",\"wildcardFolderPath\":\"dataauihnb\",\"wildcardFileName\":\"datahkdwyehqnxuffgj\",\"prefix\":\"dataminhvdkqigppdqsq\",\"fileListPath\":\"datap\",\"enablePartitionDiscovery\":\"dataaxthuhuruo\",\"partitionRootPath\":\"datayotapstkdbn\",\"deleteFilesAfterCompletion\":\"datapcuzexoymfku\",\"modifiedDatetimeStart\":\"dataysgsqzpgrvf\",\"modifiedDatetimeEnd\":\"datayph\",\"maxConcurrentConnections\":\"datarxrpahp\",\"disableMetricsCollection\":\"dataikfenmiflkyf\",\"\":{\"tablilyoomgse\":\"dataolnxhsupilh\",\"kluqf\":\"dataisydhardx\",\"ierxu\":\"datafglftlqa\"}}") .toObject(AmazonS3CompatibleReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AmazonS3CompatibleReadSettings model = new AmazonS3CompatibleReadSettings() - .withMaxConcurrentConnections("dataneouxpdnlbp").withDisableMetricsCollection("datackohn") - .withRecursive("datawwtevfeu").withWildcardFolderPath("datawtsztxoswvfrymq").withWildcardFileName("dataksq") - .withPrefix("dataxdleohysdgkbaxy").withFileListPath("datavtkrqiyuqd") - .withEnablePartitionDiscovery("datajqztrpjme").withPartitionRootPath("datafotaaqyx") - .withDeleteFilesAfterCompletion("dataoabcoxqaavjkre").withModifiedDatetimeStart("dataasv") - .withModifiedDatetimeEnd("datasbvolivianklqclf"); + .withMaxConcurrentConnections("datarxrpahp").withDisableMetricsCollection("dataikfenmiflkyf") + .withRecursive("datakf").withWildcardFolderPath("dataauihnb").withWildcardFileName("datahkdwyehqnxuffgj") + .withPrefix("dataminhvdkqigppdqsq").withFileListPath("datap").withEnablePartitionDiscovery("dataaxthuhuruo") + .withPartitionRootPath("datayotapstkdbn").withDeleteFilesAfterCompletion("datapcuzexoymfku") + .withModifiedDatetimeStart("dataysgsqzpgrvf").withModifiedDatetimeEnd("datayph"); model = BinaryData.fromObject(model).toObject(AmazonS3CompatibleReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3ReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3ReadSettingsTests.java index 604c2bcb268e6..95233f9165568 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3ReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AmazonS3ReadSettingsTests.java @@ -11,18 +11,18 @@ public final class AmazonS3ReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AmazonS3ReadSettings model = BinaryData.fromString( - "{\"type\":\"AmazonS3ReadSettings\",\"recursive\":\"dataiikk\",\"wildcardFolderPath\":\"dataqpliegemtnbkev\",\"wildcardFileName\":\"datak\",\"prefix\":\"dataiksncr\",\"fileListPath\":\"datatlrbzqtu\",\"enablePartitionDiscovery\":\"dataajfay\",\"partitionRootPath\":\"dataohdlpcix\",\"deleteFilesAfterCompletion\":\"dataxnyhivhyujqxyfb\",\"modifiedDatetimeStart\":\"datazyl\",\"modifiedDatetimeEnd\":\"dataabrdnovu\",\"maxConcurrentConnections\":\"datawwjohgcnrkmci\",\"disableMetricsCollection\":\"datawwhyejh\",\"\":{\"cfqa\":\"datafcfyzwkmrjfsq\",\"pasxwiic\":\"dataoveqowqodi\",\"x\":\"datasbjhhadndow\"}}") + "{\"type\":\"AmazonS3ReadSettings\",\"recursive\":\"dataubryhvbvjyf\",\"wildcardFolderPath\":\"dataaupjozgryoc\",\"wildcardFileName\":\"datak\",\"prefix\":\"datailyznbb\",\"fileListPath\":\"datamxznfoa\",\"enablePartitionDiscovery\":\"datajwiswzn\",\"partitionRootPath\":\"databhtleiwfiz\",\"deleteFilesAfterCompletion\":\"datahvq\",\"modifiedDatetimeStart\":\"datahltnds\",\"modifiedDatetimeEnd\":\"datagnyfhqylia\",\"maxConcurrentConnections\":\"databhzteninafyaga\",\"disableMetricsCollection\":\"datavetzkcolbclspq\",\"\":{\"p\":\"dataicurufngpbvdl\",\"zixranbtqejfqgh\":\"datadkiywwenvxu\",\"y\":\"dataadrvxb\"}}") .toObject(AmazonS3ReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AmazonS3ReadSettings model = new AmazonS3ReadSettings().withMaxConcurrentConnections("datawwjohgcnrkmci") - .withDisableMetricsCollection("datawwhyejh").withRecursive("dataiikk") - .withWildcardFolderPath("dataqpliegemtnbkev").withWildcardFileName("datak").withPrefix("dataiksncr") - .withFileListPath("datatlrbzqtu").withEnablePartitionDiscovery("dataajfay") - .withPartitionRootPath("dataohdlpcix").withDeleteFilesAfterCompletion("dataxnyhivhyujqxyfb") - .withModifiedDatetimeStart("datazyl").withModifiedDatetimeEnd("dataabrdnovu"); + AmazonS3ReadSettings model = new AmazonS3ReadSettings().withMaxConcurrentConnections("databhzteninafyaga") + .withDisableMetricsCollection("datavetzkcolbclspq").withRecursive("dataubryhvbvjyf") + .withWildcardFolderPath("dataaupjozgryoc").withWildcardFileName("datak").withPrefix("datailyznbb") + .withFileListPath("datamxznfoa").withEnablePartitionDiscovery("datajwiswzn") + .withPartitionRootPath("databhtleiwfiz").withDeleteFilesAfterCompletion("datahvq") + .withModifiedDatetimeStart("datahltnds").withModifiedDatetimeEnd("datagnyfhqylia"); model = BinaryData.fromObject(model).toObject(AmazonS3ReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTests.java index 18a75f891a7ce..5aea5f50d7e23 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTests.java @@ -20,38 +20,43 @@ public final class AppendVariableActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AppendVariableActivity model = BinaryData.fromString( - "{\"type\":\"AppendVariable\",\"typeProperties\":{\"variableName\":\"ej\",\"value\":\"datahnhwg\"},\"name\":\"unbcvfzc\",\"description\":\"irng\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"rqqfgudob\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Skipped\"],\"\":{\"xvdnc\":\"dataxxcaxgraiki\"}}],\"userProperties\":[{\"name\":\"sfnlgwpuasbfc\",\"value\":\"datalvakhdigxxtf\"}],\"\":{\"ffm\":\"datasdhdiiwvz\",\"artpdyhbpfxm\":\"datatm\",\"vxfglilfjcowr\":\"datahxpmtz\",\"jxsgrtnit\":\"dataqyo\"}}") + "{\"type\":\"AppendVariable\",\"typeProperties\":{\"variableName\":\"iymvgnqqfnvdor\",\"value\":\"datacvgknbmpcnezdpl\"},\"name\":\"bqhbbzfcjmhpobu\",\"description\":\"dyztqpszbtbxymby\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ehxotizv\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Skipped\"],\"\":{\"jiyl\":\"datatfo\",\"ovbzsmgeyok\":\"dataveb\"}},{\"activity\":\"a\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"xymcmogfbweuaz\":\"datawebgaaxffttfql\",\"wnicdgim\":\"datatsgsqoadsbace\"}}],\"userProperties\":[{\"name\":\"mpplbcarc\",\"value\":\"datarftcjxzmxwwmhdl\"},{\"name\":\"fyonnbxvxrcm\",\"value\":\"datadmyjc\"}],\"\":{\"zirkyxhqwoxm\":\"datazodolehchimzrc\"}}") .toObject(AppendVariableActivity.class); - Assertions.assertEquals("unbcvfzc", model.name()); - Assertions.assertEquals("irng", model.description()); + Assertions.assertEquals("bqhbbzfcjmhpobu", model.name()); + Assertions.assertEquals("dyztqpszbtbxymby", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("rqqfgudob", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("sfnlgwpuasbfc", model.userProperties().get(0).name()); - Assertions.assertEquals("ej", model.variableName()); + Assertions.assertEquals("ehxotizv", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("mpplbcarc", model.userProperties().get(0).name()); + Assertions.assertEquals("iymvgnqqfnvdor", model.variableName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AppendVariableActivity model = new AppendVariableActivity().withName("unbcvfzc").withDescription("irng") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("rqqfgudob") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("sfnlgwpuasbfc").withValue("datalvakhdigxxtf"))) - .withVariableName("ej").withValue("datahnhwg"); + AppendVariableActivity model = new AppendVariableActivity().withName("bqhbbzfcjmhpobu") + .withDescription("dyztqpszbtbxymby").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("ehxotizv") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("a") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("mpplbcarc").withValue("datarftcjxzmxwwmhdl"), + new UserProperty().withName("fyonnbxvxrcm").withValue("datadmyjc"))) + .withVariableName("iymvgnqqfnvdor").withValue("datacvgknbmpcnezdpl"); model = BinaryData.fromObject(model).toObject(AppendVariableActivity.class); - Assertions.assertEquals("unbcvfzc", model.name()); - Assertions.assertEquals("irng", model.description()); + Assertions.assertEquals("bqhbbzfcjmhpobu", model.name()); + Assertions.assertEquals("dyztqpszbtbxymby", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("rqqfgudob", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("sfnlgwpuasbfc", model.userProperties().get(0).name()); - Assertions.assertEquals("ej", model.variableName()); + Assertions.assertEquals("ehxotizv", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("mpplbcarc", model.userProperties().get(0).name()); + Assertions.assertEquals("iymvgnqqfnvdor", model.variableName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTypePropertiesTests.java index e39b3b9830359..5ce6cd3622236 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AppendVariableActivityTypePropertiesTests.java @@ -12,16 +12,16 @@ public final class AppendVariableActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AppendVariableActivityTypeProperties model - = BinaryData.fromString("{\"variableName\":\"mpg\",\"value\":\"datayvpxpcjnb\"}") + = BinaryData.fromString("{\"variableName\":\"buanybfmblwhpqn\",\"value\":\"datafpsppkqufdmgm\"}") .toObject(AppendVariableActivityTypeProperties.class); - Assertions.assertEquals("mpg", model.variableName()); + Assertions.assertEquals("buanybfmblwhpqn", model.variableName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AppendVariableActivityTypeProperties model - = new AppendVariableActivityTypeProperties().withVariableName("mpg").withValue("datayvpxpcjnb"); + AppendVariableActivityTypeProperties model = new AppendVariableActivityTypeProperties() + .withVariableName("buanybfmblwhpqn").withValue("datafpsppkqufdmgm"); model = BinaryData.fromObject(model).toObject(AppendVariableActivityTypeProperties.class); - Assertions.assertEquals("mpg", model.variableName()); + Assertions.assertEquals("buanybfmblwhpqn", model.variableName()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSinkTests.java index 8640663f04f92..125d6f93c6023 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSinkTests.java @@ -18,28 +18,29 @@ public final class AvroSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AvroSink model = BinaryData.fromString( - "{\"type\":\"AvroSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataqgcn\",\"disableMetricsCollection\":\"datahgcz\",\"copyBehavior\":\"dataouxigdwpgmhqhv\",\"metadata\":[{\"name\":\"datawxqwcx\",\"value\":\"datalymmhzvnetecfy\"},{\"name\":\"datafkcwfpoaflgkzgzx\",\"value\":\"datavvfkqbgkssygdv\"},{\"name\":\"databbd\",\"value\":\"datal\"}],\"\":{\"nxpsebazbtyrjr\":\"datawpsxygrni\",\"fzrra\":\"dataqgn\",\"jqzgyymyywhfdkj\":\"dataimtu\"}},\"formatSettings\":{\"type\":\"AvroWriteSettings\",\"recordName\":\"ezso\",\"recordNamespace\":\"zdhzk\",\"maxRowsPerFile\":\"datazldplamcc\",\"fileNamePrefix\":\"datajrnveqleozqqwia\",\"\":{\"fhysatoplqcujsrl\":\"datazyvbu\",\"to\":\"datawuqkprfqoco\"}},\"writeBatchSize\":\"datawbsbkobbdss\",\"writeBatchTimeout\":\"datawhf\",\"sinkRetryCount\":\"datawrjbr\",\"sinkRetryWait\":\"datarsetttul\",\"maxConcurrentConnections\":\"dataajb\",\"disableMetricsCollection\":\"dataspfgmosiskihfsl\",\"\":{\"pypxg\":\"datajzgmxqa\",\"yf\":\"dataypokoo\",\"fucmhonoje\":\"datasnozjn\"}}") + "{\"type\":\"AvroSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datavzqhv\",\"disableMetricsCollection\":\"datadsnvtdbeanigozjr\",\"copyBehavior\":\"dataaq\",\"metadata\":[{\"name\":\"datalmzpfylqevwwvzn\",\"value\":\"dataxc\"},{\"name\":\"dataropzgjleec\",\"value\":\"databhqkvbins\"},{\"name\":\"dataw\",\"value\":\"dataegoupdqeflvd\"},{\"name\":\"dataqcqlexobeekzy\",\"value\":\"datapatwbbf\"}],\"\":{\"cahhfuydg\":\"datalhnwoh\",\"lcvibp\":\"datahitavgayusp\",\"efepwkhruzzwgbb\":\"datauzeeb\"}},\"formatSettings\":{\"type\":\"AvroWriteSettings\",\"recordName\":\"vfoyqvnlhsxeas\",\"recordNamespace\":\"qquvvsc\",\"maxRowsPerFile\":\"datakm\",\"fileNamePrefix\":\"datadukp\",\"\":{\"fkxazuboi\":\"dataibwuzvmorsyi\",\"kznnzpvjwegovic\":\"dataorwpbbjzdvaqoil\",\"atyypercazcchv\":\"dataqyrajdvvs\"}},\"writeBatchSize\":\"datacha\",\"writeBatchTimeout\":\"datatvotfhhayfxkf\",\"sinkRetryCount\":\"dataxefzliguw\",\"sinkRetryWait\":\"dataszcmfmynljigjc\",\"maxConcurrentConnections\":\"dataa\",\"disableMetricsCollection\":\"datawtivsk\",\"\":{\"hbxvvu\":\"datapnpunrvjb\",\"kj\":\"dataq\"}}") .toObject(AvroSink.class); - Assertions.assertEquals("ezso", model.formatSettings().recordName()); - Assertions.assertEquals("zdhzk", model.formatSettings().recordNamespace()); + Assertions.assertEquals("vfoyqvnlhsxeas", model.formatSettings().recordName()); + Assertions.assertEquals("qquvvsc", model.formatSettings().recordNamespace()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvroSink model = new AvroSink().withWriteBatchSize("datawbsbkobbdss").withWriteBatchTimeout("datawhf") - .withSinkRetryCount("datawrjbr").withSinkRetryWait("datarsetttul").withMaxConcurrentConnections("dataajb") - .withDisableMetricsCollection("dataspfgmosiskihfsl") - .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataqgcn") - .withDisableMetricsCollection("datahgcz").withCopyBehavior("dataouxigdwpgmhqhv") - .withMetadata(Arrays.asList(new MetadataItem().withName("datawxqwcx").withValue("datalymmhzvnetecfy"), - new MetadataItem().withName("datafkcwfpoaflgkzgzx").withValue("datavvfkqbgkssygdv"), - new MetadataItem().withName("databbd").withValue("datal"))) + AvroSink model = new AvroSink().withWriteBatchSize("datacha").withWriteBatchTimeout("datatvotfhhayfxkf") + .withSinkRetryCount("dataxefzliguw").withSinkRetryWait("dataszcmfmynljigjc") + .withMaxConcurrentConnections("dataa").withDisableMetricsCollection("datawtivsk") + .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("datavzqhv") + .withDisableMetricsCollection("datadsnvtdbeanigozjr").withCopyBehavior("dataaq") + .withMetadata(Arrays.asList(new MetadataItem().withName("datalmzpfylqevwwvzn").withValue("dataxc"), + new MetadataItem().withName("dataropzgjleec").withValue("databhqkvbins"), + new MetadataItem().withName("dataw").withValue("dataegoupdqeflvd"), + new MetadataItem().withName("dataqcqlexobeekzy").withValue("datapatwbbf"))) .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))) - .withFormatSettings(new AvroWriteSettings().withRecordName("ezso").withRecordNamespace("zdhzk") - .withMaxRowsPerFile("datazldplamcc").withFileNamePrefix("datajrnveqleozqqwia")); + .withFormatSettings(new AvroWriteSettings().withRecordName("vfoyqvnlhsxeas").withRecordNamespace("qquvvsc") + .withMaxRowsPerFile("datakm").withFileNamePrefix("datadukp")); model = BinaryData.fromObject(model).toObject(AvroSink.class); - Assertions.assertEquals("ezso", model.formatSettings().recordName()); - Assertions.assertEquals("zdhzk", model.formatSettings().recordNamespace()); + Assertions.assertEquals("vfoyqvnlhsxeas", model.formatSettings().recordName()); + Assertions.assertEquals("qquvvsc", model.formatSettings().recordNamespace()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSourceTests.java index ef174be85c7b0..d11bec34d6d83 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroSourceTests.java @@ -14,18 +14,17 @@ public final class AvroSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AvroSource model = BinaryData.fromString( - "{\"type\":\"AvroSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datarbrntvhppy\",\"disableMetricsCollection\":\"datalzualsvxpola\",\"\":{\"nmuxl\":\"datajmsa\",\"cnqvbnybffc\":\"datahyxryvwnzhsigddg\",\"mzr\":\"datatijcl\"}},\"additionalColumns\":\"datagvgogczgcm\",\"sourceRetryCount\":\"datamkwddgyqe\",\"sourceRetryWait\":\"datacr\",\"maxConcurrentConnections\":\"dataamlrtcbvif\",\"disableMetricsCollection\":\"datanxstowvgw\",\"\":{\"buzvaxl\":\"datasqlyahlaoqkcit\",\"pf\":\"datarznwhictsauvprq\",\"ldonsekazxewnlpc\":\"databxljddkkoyzsyjvk\",\"xmyfrmfclkyncjya\":\"datahczqm\"}}") + "{\"type\":\"AvroSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataggoppmxcm\",\"disableMetricsCollection\":\"datagdd\",\"\":{\"unamzqvawk\":\"datayf\"}},\"additionalColumns\":\"dataxvzcxxfwp\",\"sourceRetryCount\":\"datagqwbrzkmgyl\",\"sourceRetryWait\":\"datacxmh\",\"maxConcurrentConnections\":\"datarecoiqwnqlizlcyh\",\"disableMetricsCollection\":\"datarkf\",\"\":{\"fmbvmajcmpohjdvf\":\"dataiommis\",\"diuyqdjk\":\"datavbwawymahboi\",\"bryhvshkvup\":\"datagcjssqpk\"}}") .toObject(AvroSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvroSource model = new AvroSource().withSourceRetryCount("datamkwddgyqe").withSourceRetryWait("datacr") - .withMaxConcurrentConnections("dataamlrtcbvif").withDisableMetricsCollection("datanxstowvgw") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datarbrntvhppy") - .withDisableMetricsCollection("datalzualsvxpola") - .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) - .withAdditionalColumns("datagvgogczgcm"); + AvroSource model = new AvroSource().withSourceRetryCount("datagqwbrzkmgyl").withSourceRetryWait("datacxmh") + .withMaxConcurrentConnections("datarecoiqwnqlizlcyh").withDisableMetricsCollection("datarkf") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataggoppmxcm") + .withDisableMetricsCollection("datagdd").withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + .withAdditionalColumns("dataxvzcxxfwp"); model = BinaryData.fromObject(model).toObject(AvroSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroWriteSettingsTests.java index a2d9e51ed4b3c..1701d7d603fa2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AvroWriteSettingsTests.java @@ -12,18 +12,18 @@ public final class AvroWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AvroWriteSettings model = BinaryData.fromString( - "{\"type\":\"AvroWriteSettings\",\"recordName\":\"yqqonkre\",\"recordNamespace\":\"ojusmdod\",\"maxRowsPerFile\":\"datak\",\"fileNamePrefix\":\"datantaovlyyk\",\"\":{\"rwvswlhjlbk\":\"datapkdsldy\",\"fwlkyrnmgsbubzfa\":\"dataygszhpnatltjekq\",\"erzsnfgmo\":\"datayuqecwrtre\"}}") + "{\"type\":\"AvroWriteSettings\",\"recordName\":\"gdjvyclasdprknup\",\"recordNamespace\":\"eklgsnpvqczpko\",\"maxRowsPerFile\":\"dataolthowcsueb\",\"fileNamePrefix\":\"datag\",\"\":{\"exar\":\"dataimtxmd\",\"ivftl\":\"dataukoir\"}}") .toObject(AvroWriteSettings.class); - Assertions.assertEquals("yqqonkre", model.recordName()); - Assertions.assertEquals("ojusmdod", model.recordNamespace()); + Assertions.assertEquals("gdjvyclasdprknup", model.recordName()); + Assertions.assertEquals("eklgsnpvqczpko", model.recordNamespace()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AvroWriteSettings model = new AvroWriteSettings().withRecordName("yqqonkre").withRecordNamespace("ojusmdod") - .withMaxRowsPerFile("datak").withFileNamePrefix("datantaovlyyk"); + AvroWriteSettings model = new AvroWriteSettings().withRecordName("gdjvyclasdprknup") + .withRecordNamespace("eklgsnpvqczpko").withMaxRowsPerFile("dataolthowcsueb").withFileNamePrefix("datag"); model = BinaryData.fromObject(model).toObject(AvroWriteSettings.class); - Assertions.assertEquals("yqqonkre", model.recordName()); - Assertions.assertEquals("ojusmdod", model.recordNamespace()); + Assertions.assertEquals("gdjvyclasdprknup", model.recordName()); + Assertions.assertEquals("eklgsnpvqczpko", model.recordNamespace()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTests.java index 6d188a42dde19..5b27d7018a45d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTests.java @@ -11,16 +11,16 @@ public final class AzPowerShellSetupTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzPowerShellSetup model = BinaryData - .fromString("{\"type\":\"AzPowerShellSetup\",\"typeProperties\":{\"version\":\"zbbhxncszdwpave\"}}") - .toObject(AzPowerShellSetup.class); - Assertions.assertEquals("zbbhxncszdwpave", model.version()); + AzPowerShellSetup model + = BinaryData.fromString("{\"type\":\"AzPowerShellSetup\",\"typeProperties\":{\"version\":\"pjfkr\"}}") + .toObject(AzPowerShellSetup.class); + Assertions.assertEquals("pjfkr", model.version()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzPowerShellSetup model = new AzPowerShellSetup().withVersion("zbbhxncszdwpave"); + AzPowerShellSetup model = new AzPowerShellSetup().withVersion("pjfkr"); model = BinaryData.fromObject(model).toObject(AzPowerShellSetup.class); - Assertions.assertEquals("zbbhxncszdwpave", model.version()); + Assertions.assertEquals("pjfkr", model.version()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTypePropertiesTests.java index 800da73f93088..f60b0b43ffc89 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzPowerShellSetupTypePropertiesTests.java @@ -12,14 +12,14 @@ public final class AzPowerShellSetupTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzPowerShellSetupTypeProperties model - = BinaryData.fromString("{\"version\":\"xgmigsoebd\"}").toObject(AzPowerShellSetupTypeProperties.class); - Assertions.assertEquals("xgmigsoebd", model.version()); + = BinaryData.fromString("{\"version\":\"rerdlgbvtpxowg\"}").toObject(AzPowerShellSetupTypeProperties.class); + Assertions.assertEquals("rerdlgbvtpxowg", model.version()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzPowerShellSetupTypeProperties model = new AzPowerShellSetupTypeProperties().withVersion("xgmigsoebd"); + AzPowerShellSetupTypeProperties model = new AzPowerShellSetupTypeProperties().withVersion("rerdlgbvtpxowg"); model = BinaryData.fromObject(model).toObject(AzPowerShellSetupTypeProperties.class); - Assertions.assertEquals("xgmigsoebd", model.version()); + Assertions.assertEquals("rerdlgbvtpxowg", model.version()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSReadSettingsTests.java index 681d460390f1f..53953e7c7a8c6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSReadSettingsTests.java @@ -11,18 +11,18 @@ public final class AzureBlobFSReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureBlobFSReadSettings model = BinaryData.fromString( - "{\"type\":\"AzureBlobFSReadSettings\",\"recursive\":\"databbhukx\",\"wildcardFolderPath\":\"datahyr\",\"wildcardFileName\":\"dataqpgadesnesg\",\"fileListPath\":\"datadvgxte\",\"enablePartitionDiscovery\":\"datasictoq\",\"partitionRootPath\":\"datazmznoe\",\"deleteFilesAfterCompletion\":\"dataduyqb\",\"modifiedDatetimeStart\":\"datayzajdpbnbp\",\"modifiedDatetimeEnd\":\"datanopryutx\",\"maxConcurrentConnections\":\"datanighcx\",\"disableMetricsCollection\":\"datahiihg\",\"\":{\"ipfudzntbzg\":\"dataheitty\",\"kwypbq\":\"datagw\"}}") + "{\"type\":\"AzureBlobFSReadSettings\",\"recursive\":\"datasbk\",\"wildcardFolderPath\":\"dataouytsajjgvu\",\"wildcardFileName\":\"datawichjk\",\"fileListPath\":\"datatlsohrtgpvvcbt\",\"enablePartitionDiscovery\":\"dataiwc\",\"partitionRootPath\":\"datayaoyzjfgv\",\"deleteFilesAfterCompletion\":\"datauimnabgrs\",\"modifiedDatetimeStart\":\"datazmthiecuflazfot\",\"modifiedDatetimeEnd\":\"datakumam\",\"maxConcurrentConnections\":\"datargljekh\",\"disableMetricsCollection\":\"datafgjbeybdukbglniw\",\"\":{\"oexkon\":\"dataysceji\",\"edhaftniffajniwb\":\"dataiacdloehsmvvxkc\"}}") .toObject(AzureBlobFSReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureBlobFSReadSettings model = new AzureBlobFSReadSettings().withMaxConcurrentConnections("datanighcx") - .withDisableMetricsCollection("datahiihg").withRecursive("databbhukx").withWildcardFolderPath("datahyr") - .withWildcardFileName("dataqpgadesnesg").withFileListPath("datadvgxte") - .withEnablePartitionDiscovery("datasictoq").withPartitionRootPath("datazmznoe") - .withDeleteFilesAfterCompletion("dataduyqb").withModifiedDatetimeStart("datayzajdpbnbp") - .withModifiedDatetimeEnd("datanopryutx"); + AzureBlobFSReadSettings model = new AzureBlobFSReadSettings().withMaxConcurrentConnections("datargljekh") + .withDisableMetricsCollection("datafgjbeybdukbglniw").withRecursive("datasbk") + .withWildcardFolderPath("dataouytsajjgvu").withWildcardFileName("datawichjk") + .withFileListPath("datatlsohrtgpvvcbt").withEnablePartitionDiscovery("dataiwc") + .withPartitionRootPath("datayaoyzjfgv").withDeleteFilesAfterCompletion("datauimnabgrs") + .withModifiedDatetimeStart("datazmthiecuflazfot").withModifiedDatetimeEnd("datakumam"); model = BinaryData.fromObject(model).toObject(AzureBlobFSReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSinkTests.java index d6b27f2bd66a7..65cd48947fb78 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSinkTests.java @@ -13,17 +13,19 @@ public final class AzureBlobFSSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureBlobFSSink model = BinaryData.fromString( - "{\"type\":\"AzureBlobFSSink\",\"copyBehavior\":\"dataqnprbvruhdjzivl\",\"metadata\":[{\"name\":\"datamq\",\"value\":\"datambfpt\"}],\"writeBatchSize\":\"dataxmksxxbdtjvvngn\",\"writeBatchTimeout\":\"datakzulmqxficinw\",\"sinkRetryCount\":\"dataveyxtxerxttobo\",\"sinkRetryWait\":\"dataxb\",\"maxConcurrentConnections\":\"datattenruditu\",\"disableMetricsCollection\":\"dataycvt\",\"\":{\"hivxblqvwhjgtbh\":\"datalyimhspj\"}}") + "{\"type\":\"AzureBlobFSSink\",\"copyBehavior\":\"datamiw\",\"metadata\":[{\"name\":\"datakxrqzgshqx\",\"value\":\"dataunuvfslawimhoaqj\"},{\"name\":\"datahlpznmdai\",\"value\":\"datazqz\"},{\"name\":\"datadipnhbsvrlr\",\"value\":\"dataxmnoa\"}],\"writeBatchSize\":\"datayadyfnxtlln\",\"writeBatchTimeout\":\"datamdgsvaekuovwi\",\"sinkRetryCount\":\"dataykprrddbenf\",\"sinkRetryWait\":\"datafszmxpos\",\"maxConcurrentConnections\":\"datasc\",\"disableMetricsCollection\":\"datauldkpdle\",\"\":{\"bxggknm\":\"datajujps\",\"z\":\"datakniqoqyrcpsjea\",\"jwsddyq\":\"datajcsbkmaluchbfrt\",\"txsyufex\":\"dataxpnzpuknfpgg\"}}") .toObject(AzureBlobFSSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureBlobFSSink model = new AzureBlobFSSink().withWriteBatchSize("dataxmksxxbdtjvvngn") - .withWriteBatchTimeout("datakzulmqxficinw").withSinkRetryCount("dataveyxtxerxttobo") - .withSinkRetryWait("dataxb").withMaxConcurrentConnections("datattenruditu") - .withDisableMetricsCollection("dataycvt").withCopyBehavior("dataqnprbvruhdjzivl") - .withMetadata(Arrays.asList(new MetadataItem().withName("datamq").withValue("datambfpt"))); + AzureBlobFSSink model = new AzureBlobFSSink().withWriteBatchSize("datayadyfnxtlln") + .withWriteBatchTimeout("datamdgsvaekuovwi").withSinkRetryCount("dataykprrddbenf") + .withSinkRetryWait("datafszmxpos").withMaxConcurrentConnections("datasc") + .withDisableMetricsCollection("datauldkpdle").withCopyBehavior("datamiw") + .withMetadata(Arrays.asList(new MetadataItem().withName("datakxrqzgshqx").withValue("dataunuvfslawimhoaqj"), + new MetadataItem().withName("datahlpznmdai").withValue("datazqz"), + new MetadataItem().withName("datadipnhbsvrlr").withValue("dataxmnoa"))); model = BinaryData.fromObject(model).toObject(AzureBlobFSSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSourceTests.java index c7786b013011a..7f98d87a0a7f7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSSourceTests.java @@ -11,16 +11,16 @@ public final class AzureBlobFSSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureBlobFSSource model = BinaryData.fromString( - "{\"type\":\"AzureBlobFSSource\",\"treatEmptyAsNull\":\"datay\",\"skipHeaderLineCount\":\"datatrdukdmsktuvjh\",\"recursive\":\"datatvyt\",\"sourceRetryCount\":\"datafbsgrzw\",\"sourceRetryWait\":\"datadudxqebtrpsplwt\",\"maxConcurrentConnections\":\"datacseybvtgcoznnjqx\",\"disableMetricsCollection\":\"datarnkuuotlymyb\",\"\":{\"j\":\"datarkxkmtuynugpt\",\"tqqshb\":\"dataisvfh\"}}") + "{\"type\":\"AzureBlobFSSource\",\"treatEmptyAsNull\":\"datautawgyl\",\"skipHeaderLineCount\":\"datadrfclehlop\",\"recursive\":\"datavpeaeyjl\",\"sourceRetryCount\":\"dataduxhopyavcb\",\"sourceRetryWait\":\"dataembvfa\",\"maxConcurrentConnections\":\"dataxjoa\",\"disableMetricsCollection\":\"dataxmumfbkaxzrycvac\",\"\":{\"aq\":\"datajysyphxeoq\",\"sbbawrb\":\"dataikceiyuv\",\"rtnk\":\"dataooxvprqlxqhqgip\"}}") .toObject(AzureBlobFSSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureBlobFSSource model - = new AzureBlobFSSource().withSourceRetryCount("datafbsgrzw").withSourceRetryWait("datadudxqebtrpsplwt") - .withMaxConcurrentConnections("datacseybvtgcoznnjqx").withDisableMetricsCollection("datarnkuuotlymyb") - .withTreatEmptyAsNull("datay").withSkipHeaderLineCount("datatrdukdmsktuvjh").withRecursive("datatvyt"); + AzureBlobFSSource model = new AzureBlobFSSource().withSourceRetryCount("dataduxhopyavcb") + .withSourceRetryWait("dataembvfa").withMaxConcurrentConnections("dataxjoa") + .withDisableMetricsCollection("dataxmumfbkaxzrycvac").withTreatEmptyAsNull("datautawgyl") + .withSkipHeaderLineCount("datadrfclehlop").withRecursive("datavpeaeyjl"); model = BinaryData.fromObject(model).toObject(AzureBlobFSSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSWriteSettingsTests.java index 6e19f2e3c7416..87ff876fcdc1c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobFSWriteSettingsTests.java @@ -13,16 +13,19 @@ public final class AzureBlobFSWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureBlobFSWriteSettings model = BinaryData.fromString( - "{\"type\":\"AzureBlobFSWriteSettings\",\"blockSizeInMB\":\"dataaxthuhuruo\",\"maxConcurrentConnections\":\"datayotapstkdbn\",\"disableMetricsCollection\":\"datapcuzexoymfku\",\"copyBehavior\":\"dataysgsqzpgrvf\",\"metadata\":[{\"name\":\"datahlfrxrpahpt\",\"value\":\"datakfenmiflky\"}],\"\":{\"tablilyoomgse\":\"datalolnxhsupilh\",\"kluqf\":\"dataisydhardx\",\"ierxu\":\"datafglftlqa\"}}") + "{\"type\":\"AzureBlobFSWriteSettings\",\"blockSizeInMB\":\"datatfozysfjdcokbp\",\"maxConcurrentConnections\":\"dataqelms\",\"disableMetricsCollection\":\"databtneltnbyvbg\",\"copyBehavior\":\"datarumududwecdsybi\",\"metadata\":[{\"name\":\"dataxdkwvceqlyxbyqqo\",\"value\":\"datarekiojusmdodku\"},{\"name\":\"datacn\",\"value\":\"dataovlyykxyfpkdsld\"},{\"name\":\"datarwvswlhjlbk\",\"value\":\"datags\"}],\"\":{\"jekqz\":\"datanatl\",\"zfayyuq\":\"datawlkyrnmgsbu\",\"derzsnfgmohhcgh\":\"datacwrtr\",\"rulboawzplwghfgq\":\"datasmvvfpkymqnvvwfa\"}}") .toObject(AzureBlobFSWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureBlobFSWriteSettings model = new AzureBlobFSWriteSettings().withMaxConcurrentConnections("datayotapstkdbn") - .withDisableMetricsCollection("datapcuzexoymfku").withCopyBehavior("dataysgsqzpgrvf") - .withMetadata(Arrays.asList(new MetadataItem().withName("datahlfrxrpahpt").withValue("datakfenmiflky"))) - .withBlockSizeInMB("dataaxthuhuruo"); + AzureBlobFSWriteSettings model = new AzureBlobFSWriteSettings().withMaxConcurrentConnections("dataqelms") + .withDisableMetricsCollection("databtneltnbyvbg").withCopyBehavior("datarumududwecdsybi") + .withMetadata( + Arrays.asList(new MetadataItem().withName("dataxdkwvceqlyxbyqqo").withValue("datarekiojusmdodku"), + new MetadataItem().withName("datacn").withValue("dataovlyykxyfpkdsld"), + new MetadataItem().withName("datarwvswlhjlbk").withValue("datags"))) + .withBlockSizeInMB("datatfozysfjdcokbp"); model = BinaryData.fromObject(model).toObject(AzureBlobFSWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageReadSettingsTests.java index bf41d13fd68c8..e1f9df609b708 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageReadSettingsTests.java @@ -11,18 +11,18 @@ public final class AzureBlobStorageReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureBlobStorageReadSettings model = BinaryData.fromString( - "{\"type\":\"AzureBlobStorageReadSettings\",\"recursive\":\"datadhlnar\",\"wildcardFolderPath\":\"datauoa\",\"wildcardFileName\":\"datairiccuyqtjvrz\",\"prefix\":\"datagmgfa\",\"fileListPath\":\"datab\",\"enablePartitionDiscovery\":\"dataaenvpzd\",\"partitionRootPath\":\"datapizgaujvc\",\"deleteFilesAfterCompletion\":\"datafybx\",\"modifiedDatetimeStart\":\"datarceomsqarbtrk\",\"modifiedDatetimeEnd\":\"dataoefi\",\"maxConcurrentConnections\":\"datajiudnus\",\"disableMetricsCollection\":\"datamoxohgkd\",\"\":{\"xzqzjv\":\"datahuepuvr\",\"wqyousqmern\":\"datarhyx\",\"memkyouwmj\":\"datajpl\",\"d\":\"datahmkch\"}}") + "{\"type\":\"AzureBlobStorageReadSettings\",\"recursive\":\"dataouxpdnlbp\",\"wildcardFolderPath\":\"datackohn\",\"wildcardFileName\":\"dataaqzekggurwxf\",\"prefix\":\"dataxg\",\"fileListPath\":\"datafiyuofegrzf\",\"enablePartitionDiscovery\":\"dataulookwnzotjbvh\",\"partitionRootPath\":\"datadlo\",\"deleteFilesAfterCompletion\":\"dataqsylkkqvmmm\",\"modifiedDatetimeStart\":\"dataljcauegymc\",\"modifiedDatetimeEnd\":\"datasmnjitxu\",\"maxConcurrentConnections\":\"datalbibwodayipgsh\",\"disableMetricsCollection\":\"dataoecmbyo\",\"\":{\"qpddebokzdshhhd\":\"databvfchfux\",\"zzkzvfywspaja\":\"dataxnzapzibmst\",\"zzpauts\":\"datajhvpktbnmhxt\"}}") .toObject(AzureBlobStorageReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureBlobStorageReadSettings model = new AzureBlobStorageReadSettings() - .withMaxConcurrentConnections("datajiudnus").withDisableMetricsCollection("datamoxohgkd") - .withRecursive("datadhlnar").withWildcardFolderPath("datauoa").withWildcardFileName("datairiccuyqtjvrz") - .withPrefix("datagmgfa").withFileListPath("datab").withEnablePartitionDiscovery("dataaenvpzd") - .withPartitionRootPath("datapizgaujvc").withDeleteFilesAfterCompletion("datafybx") - .withModifiedDatetimeStart("datarceomsqarbtrk").withModifiedDatetimeEnd("dataoefi"); + .withMaxConcurrentConnections("datalbibwodayipgsh").withDisableMetricsCollection("dataoecmbyo") + .withRecursive("dataouxpdnlbp").withWildcardFolderPath("datackohn").withWildcardFileName("dataaqzekggurwxf") + .withPrefix("dataxg").withFileListPath("datafiyuofegrzf").withEnablePartitionDiscovery("dataulookwnzotjbvh") + .withPartitionRootPath("datadlo").withDeleteFilesAfterCompletion("dataqsylkkqvmmm") + .withModifiedDatetimeStart("dataljcauegymc").withModifiedDatetimeEnd("datasmnjitxu"); model = BinaryData.fromObject(model).toObject(AzureBlobStorageReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageWriteSettingsTests.java index d9d35f8e8cd17..a629c06d75d67 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureBlobStorageWriteSettingsTests.java @@ -13,18 +13,18 @@ public final class AzureBlobStorageWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureBlobStorageWriteSettings model = BinaryData.fromString( - "{\"type\":\"AzureBlobStorageWriteSettings\",\"blockSizeInMB\":\"dataejkvcimq\",\"maxConcurrentConnections\":\"datad\",\"disableMetricsCollection\":\"datahhwtgcgefayc\",\"copyBehavior\":\"datagotbjnxozi\",\"metadata\":[{\"name\":\"datapovflxluvmsgdis\",\"value\":\"datanxthu\"},{\"name\":\"datadvokxu\",\"value\":\"datahrdikd\"}],\"\":{\"nzaxzfhhhgyxkgnr\":\"dataahgsibldxyaqdaa\",\"dhkdwyehqn\":\"dataalkfdxauihnb\",\"fgjynmin\":\"datau\",\"p\":\"datavdkqigppdqsqs\"}}") + "{\"type\":\"AzureBlobStorageWriteSettings\",\"blockSizeInMB\":\"datafn\",\"maxConcurrentConnections\":\"datagzmyxsoxqarjt\",\"disableMetricsCollection\":\"datalllmtiyguuhylzbd\",\"copyBehavior\":\"datatdohjxfqyyu\",\"metadata\":[{\"name\":\"dataxznxh\",\"value\":\"datatkkicxj\"},{\"name\":\"dataeilixzjvkq\",\"value\":\"databblh\"}],\"\":{\"xcljnkms\":\"dataxxpn\",\"rivagcsmrtepsyb\":\"datasquxxqcimnchvkj\"}}") .toObject(AzureBlobStorageWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureBlobStorageWriteSettings model - = new AzureBlobStorageWriteSettings().withMaxConcurrentConnections("datad") - .withDisableMetricsCollection("datahhwtgcgefayc").withCopyBehavior("datagotbjnxozi") - .withMetadata(Arrays.asList(new MetadataItem().withName("datapovflxluvmsgdis").withValue("datanxthu"), - new MetadataItem().withName("datadvokxu").withValue("datahrdikd"))) - .withBlockSizeInMB("dataejkvcimq"); + = new AzureBlobStorageWriteSettings().withMaxConcurrentConnections("datagzmyxsoxqarjt") + .withDisableMetricsCollection("datalllmtiyguuhylzbd").withCopyBehavior("datatdohjxfqyyu") + .withMetadata(Arrays.asList(new MetadataItem().withName("dataxznxh").withValue("datatkkicxj"), + new MetadataItem().withName("dataeilixzjvkq").withValue("databblh"))) + .withBlockSizeInMB("datafn"); model = BinaryData.fromObject(model).toObject(AzureBlobStorageWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTests.java index 06a5f55739288..52c62d1dc0aac 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTests.java @@ -22,65 +22,60 @@ public final class AzureDataExplorerCommandActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataExplorerCommandActivity model = BinaryData.fromString( - "{\"type\":\"AzureDataExplorerCommand\",\"typeProperties\":{\"command\":\"datanigjoxhzcmgmcsj\",\"commandTimeout\":\"databu\"},\"linkedServiceName\":{\"referenceName\":\"lotojf\",\"parameters\":{\"wjgjlo\":\"datay\",\"pbih\":\"datahhhkxlqu\"}},\"policy\":{\"timeout\":\"datajym\",\"retry\":\"dataweptejry\",\"retryIntervalInSeconds\":1297104867,\"secureInput\":true,\"secureOutput\":false,\"\":{\"mwjbmrkcqp\":\"datap\"}},\"name\":\"xjjutvovhui\",\"description\":\"lyvqycknqmb\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"eusnncnnq\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Skipped\"],\"\":{\"mplt\":\"datadu\",\"zaneave\":\"dataupfndafrziwjcy\",\"ulhworhzesqdvm\":\"datavd\"}},{\"activity\":\"u\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\"],\"\":{\"ohze\":\"datajv\",\"mxjd\":\"datauulaeshftlsfwpvf\",\"xbrjdpeypuqwd\":\"dataw\",\"vxfsshocxd\":\"datanym\"}},{\"activity\":\"ornzpr\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Succeeded\",\"Completed\"],\"\":{\"pb\":\"datazdfjfnree\",\"zfotfoif\":\"datamyuiq\"}},{\"activity\":\"rikdfacf\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Failed\"],\"\":{\"drjunigxn\":\"databjizfpfxb\",\"nvr\":\"datannghgazdbv\",\"qswrncwhlxvngj\":\"datati\"}}],\"userProperties\":[{\"name\":\"djdpap\",\"value\":\"datadmvrbdtvvtap\"},{\"name\":\"kwkthmexidecdeh\",\"value\":\"datakmfiudnpj\"},{\"name\":\"xfhtsgyyrg\",\"value\":\"dataguv\"},{\"name\":\"gqllgokznffqvtx\",\"value\":\"dataytihhq\"}],\"\":{\"bdfam\":\"datawgrwgdpfzdygtkrs\",\"aiuoncrswvw\":\"dataolvgks\",\"mwzduhixomxv\":\"datauetqcxoamx\",\"ouneozgn\":\"dataruzxsnzxipgfkco\"}}") + "{\"type\":\"AzureDataExplorerCommand\",\"typeProperties\":{\"command\":\"datadsdgvhecqkb\",\"commandTimeout\":\"dataemamyshnksupc\"},\"linkedServiceName\":{\"referenceName\":\"spgbyvumxyqhctrr\",\"parameters\":{\"zuk\":\"datagchtaeacbqkx\",\"nlqfqawyns\":\"datajki\",\"lovavi\":\"datacf\"}},\"policy\":{\"timeout\":\"dataedfyhtfugpp\",\"retry\":\"datadhylx\",\"retryIntervalInSeconds\":1788389204,\"secureInput\":false,\"secureOutput\":false,\"\":{\"bchnh\":\"dataauxzcrfp\",\"ufojuex\":\"dataxmgcq\",\"snaklobc\":\"datakqhgfwyzv\",\"mltdgxiqrgr\":\"datay\"}},\"name\":\"xjfxu\",\"description\":\"yje\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"cnklqipnzgnyb\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Completed\",\"Failed\"],\"\":{\"hzqnhxksc\":\"datanc\"}},{\"activity\":\"ykrzrjjernjkgalo\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Failed\",\"Failed\"],\"\":{\"hctxarl\":\"datatygyia\",\"gp\":\"datasijcmn\"}},{\"activity\":\"gxjmwzkafuvbg\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Completed\"],\"\":{\"xvsmrxypbiwn\":\"datajl\",\"b\":\"dataznaixjsfasxfamn\"}},{\"activity\":\"xbglqybfnxej\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"cnuozjg\":\"datajmpsxot\"}}],\"userProperties\":[{\"name\":\"benwiignrrqv\",\"value\":\"dataxouoqtestr\"}],\"\":{\"yyzaalpwwcwie\":\"dataskmvr\",\"wqmundle\":\"datasswijqsndqjbdtcz\",\"hrygdp\":\"datadlcuedrmqkwkutbt\",\"tcfppjegctsatnry\":\"dataufmvozq\"}}") .toObject(AzureDataExplorerCommandActivity.class); - Assertions.assertEquals("xjjutvovhui", model.name()); - Assertions.assertEquals("lyvqycknqmb", model.description()); + Assertions.assertEquals("xjfxu", model.name()); + Assertions.assertEquals("yje", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("eusnncnnq", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("djdpap", model.userProperties().get(0).name()); - Assertions.assertEquals("lotojf", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1297104867, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("cnklqipnzgnyb", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("benwiignrrqv", model.userProperties().get(0).name()); + Assertions.assertEquals("spgbyvumxyqhctrr", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1788389204, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDataExplorerCommandActivity model = new AzureDataExplorerCommandActivity().withName("xjjutvovhui") - .withDescription("lyvqycknqmb").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + AzureDataExplorerCommandActivity model = new AzureDataExplorerCommandActivity().withName("xjfxu") + .withDescription("yje").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) .withDependsOn(Arrays.asList( - new ActivityDependency() - .withActivity("eusnncnnq") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, - DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("cnklqipnzgnyb") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, + DependencyCondition.COMPLETED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("u") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("ykrzrjjernjkgalo") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ornzpr") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("gxjmwzkafuvbg") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.FAILED, + DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("rikdfacf") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.FAILED)) + new ActivityDependency().withActivity("xbglqybfnxej") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("djdpap").withValue("datadmvrbdtvvtap"), - new UserProperty().withName("kwkthmexidecdeh").withValue("datakmfiudnpj"), - new UserProperty().withName("xfhtsgyyrg").withValue("dataguv"), - new UserProperty().withName("gqllgokznffqvtx").withValue("dataytihhq"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("lotojf") - .withParameters(mapOf("wjgjlo", "datay", "pbih", "datahhhkxlqu"))) - .withPolicy(new ActivityPolicy().withTimeout("datajym").withRetry("dataweptejry") - .withRetryIntervalInSeconds(1297104867).withSecureInput(true).withSecureOutput(false) + .withUserProperties(Arrays.asList(new UserProperty().withName("benwiignrrqv").withValue("dataxouoqtestr"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("spgbyvumxyqhctrr") + .withParameters(mapOf("zuk", "datagchtaeacbqkx", "nlqfqawyns", "datajki", "lovavi", "datacf"))) + .withPolicy(new ActivityPolicy().withTimeout("dataedfyhtfugpp").withRetry("datadhylx") + .withRetryIntervalInSeconds(1788389204).withSecureInput(false).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withCommand("datanigjoxhzcmgmcsj").withCommandTimeout("databu"); + .withCommand("datadsdgvhecqkb").withCommandTimeout("dataemamyshnksupc"); model = BinaryData.fromObject(model).toObject(AzureDataExplorerCommandActivity.class); - Assertions.assertEquals("xjjutvovhui", model.name()); - Assertions.assertEquals("lyvqycknqmb", model.description()); + Assertions.assertEquals("xjfxu", model.name()); + Assertions.assertEquals("yje", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("eusnncnnq", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("djdpap", model.userProperties().get(0).name()); - Assertions.assertEquals("lotojf", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1297104867, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("cnklqipnzgnyb", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("benwiignrrqv", model.userProperties().get(0).name()); + Assertions.assertEquals("spgbyvumxyqhctrr", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1788389204, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTypePropertiesTests.java index c9dbee09f0dce..6c4b42c804d05 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerCommandActivityTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class AzureDataExplorerCommandActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataExplorerCommandActivityTypeProperties model - = BinaryData.fromString("{\"command\":\"datamci\",\"commandTimeout\":\"datalnqexl\"}") + = BinaryData.fromString("{\"command\":\"dataouewrwc\",\"commandTimeout\":\"datavtwvyjpr\"}") .toObject(AzureDataExplorerCommandActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureDataExplorerCommandActivityTypeProperties model = new AzureDataExplorerCommandActivityTypeProperties() - .withCommand("datamci").withCommandTimeout("datalnqexl"); + .withCommand("dataouewrwc").withCommandTimeout("datavtwvyjpr"); model = BinaryData.fromObject(model).toObject(AzureDataExplorerCommandActivityTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSinkTests.java index 604828b3fcf2e..62468deaf1ff2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSinkTests.java @@ -11,17 +11,17 @@ public final class AzureDataExplorerSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataExplorerSink model = BinaryData.fromString( - "{\"type\":\"AzureDataExplorerSink\",\"ingestionMappingName\":\"dataqekqtiuveaz\",\"ingestionMappingAsJson\":\"dataiwbiwygwpwq\",\"flushImmediately\":\"dataiqgslspihu\",\"writeBatchSize\":\"datavviotvoolkm\",\"writeBatchTimeout\":\"datazefbbr\",\"sinkRetryCount\":\"dataa\",\"sinkRetryWait\":\"dataxvufrfwa\",\"maxConcurrentConnections\":\"datas\",\"disableMetricsCollection\":\"datas\",\"\":{\"eyeblkgupgnstq\":\"datavpys\"}}") + "{\"type\":\"AzureDataExplorerSink\",\"ingestionMappingName\":\"datalyjdbsx\",\"ingestionMappingAsJson\":\"dataleqsk\",\"flushImmediately\":\"dataol\",\"writeBatchSize\":\"databellpk\",\"writeBatchTimeout\":\"databrsmy\",\"sinkRetryCount\":\"datandfrfhgowhnvc\",\"sinkRetryWait\":\"datamuvgysto\",\"maxConcurrentConnections\":\"datarktodeertyijlvcm\",\"disableMetricsCollection\":\"datanxxw\",\"\":{\"bkhvlsahjdeb\":\"datadbdkw\",\"cpxlkih\":\"datayqnluszilk\",\"cmcehtrhwr\":\"dataxtbvhms\",\"chyyfgkrp\":\"dataihw\"}}") .toObject(AzureDataExplorerSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDataExplorerSink model - = new AzureDataExplorerSink().withWriteBatchSize("datavviotvoolkm").withWriteBatchTimeout("datazefbbr") - .withSinkRetryCount("dataa").withSinkRetryWait("dataxvufrfwa").withMaxConcurrentConnections("datas") - .withDisableMetricsCollection("datas").withIngestionMappingName("dataqekqtiuveaz") - .withIngestionMappingAsJson("dataiwbiwygwpwq").withFlushImmediately("dataiqgslspihu"); + AzureDataExplorerSink model = new AzureDataExplorerSink().withWriteBatchSize("databellpk") + .withWriteBatchTimeout("databrsmy").withSinkRetryCount("datandfrfhgowhnvc") + .withSinkRetryWait("datamuvgysto").withMaxConcurrentConnections("datarktodeertyijlvcm") + .withDisableMetricsCollection("datanxxw").withIngestionMappingName("datalyjdbsx") + .withIngestionMappingAsJson("dataleqsk").withFlushImmediately("dataol"); model = BinaryData.fromObject(model).toObject(AzureDataExplorerSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSourceTests.java index 7bdec9b493241..25d7fd1c21175 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataExplorerSourceTests.java @@ -11,16 +11,17 @@ public final class AzureDataExplorerSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataExplorerSource model = BinaryData.fromString( - "{\"type\":\"AzureDataExplorerSource\",\"query\":\"datazlcfe\",\"noTruncation\":\"dataryxnklfswzsyigx\",\"queryTimeout\":\"dataxhygc\",\"additionalColumns\":\"databapeuqyz\",\"sourceRetryCount\":\"datasuopcdiaossp\",\"sourceRetryWait\":\"datatgkmrsqaqgllnhgi\",\"maxConcurrentConnections\":\"datawzzk\",\"disableMetricsCollection\":\"dataqrngl\",\"\":{\"ywaojr\":\"datauaefwdkpadkts\"}}") + "{\"type\":\"AzureDataExplorerSource\",\"query\":\"dataqebtrps\",\"noTruncation\":\"datawtlocseybvtgcozn\",\"queryTimeout\":\"dataqxckrnkuuotlymy\",\"additionalColumns\":\"datagmrkxkmtuynugptf\",\"sourceRetryCount\":\"dataisvfh\",\"sourceRetryWait\":\"dataqqshbipzhuhu\",\"maxConcurrentConnections\":\"datazjsezgphip\",\"disableMetricsCollection\":\"dataoqldnhwdfxgec\",\"\":{\"bhsnimompxd\":\"datakdb\",\"fbdmoawhbdxxnm\":\"datamp\",\"iocqoydqyzhfny\":\"dataxz\",\"skt\":\"datagbwdsaqwywayjin\"}}") .toObject(AzureDataExplorerSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDataExplorerSource model = new AzureDataExplorerSource().withSourceRetryCount("datasuopcdiaossp") - .withSourceRetryWait("datatgkmrsqaqgllnhgi").withMaxConcurrentConnections("datawzzk") - .withDisableMetricsCollection("dataqrngl").withQuery("datazlcfe").withNoTruncation("dataryxnklfswzsyigx") - .withQueryTimeout("dataxhygc").withAdditionalColumns("databapeuqyz"); + AzureDataExplorerSource model + = new AzureDataExplorerSource().withSourceRetryCount("dataisvfh").withSourceRetryWait("dataqqshbipzhuhu") + .withMaxConcurrentConnections("datazjsezgphip").withDisableMetricsCollection("dataoqldnhwdfxgec") + .withQuery("dataqebtrps").withNoTruncation("datawtlocseybvtgcozn") + .withQueryTimeout("dataqxckrnkuuotlymy").withAdditionalColumns("datagmrkxkmtuynugptf"); model = BinaryData.fromObject(model).toObject(AzureDataExplorerSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreReadSettingsTests.java index 58390ac6883de..7c53a49580aad 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreReadSettingsTests.java @@ -11,19 +11,19 @@ public final class AzureDataLakeStoreReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataLakeStoreReadSettings model = BinaryData.fromString( - "{\"type\":\"AzureDataLakeStoreReadSettings\",\"recursive\":\"datapohcrkmbajyuegs\",\"wildcardFolderPath\":\"dataqdpzhqeczzutxtid\",\"wildcardFileName\":\"datarexbarg\",\"fileListPath\":\"datae\",\"listAfter\":\"datapy\",\"listBefore\":\"datauomowucj\",\"enablePartitionDiscovery\":\"datanowpvxuuvhwsrme\",\"partitionRootPath\":\"datahw\",\"deleteFilesAfterCompletion\":\"datafkjd\",\"modifiedDatetimeStart\":\"dataqtqjabwtkpjytv\",\"modifiedDatetimeEnd\":\"datair\",\"maxConcurrentConnections\":\"datasuyqvmxnavxzp\",\"disableMetricsCollection\":\"datapt\",\"\":{\"ewhc\":\"dataylq\",\"it\":\"datahexczrmyniwgg\",\"pjttzfswohd\":\"datawolfmfazxwcaic\"}}") + "{\"type\":\"AzureDataLakeStoreReadSettings\",\"recursive\":\"datayjuy\",\"wildcardFolderPath\":\"datalbbugojd\",\"wildcardFileName\":\"dataluyosigkinykjx\",\"fileListPath\":\"dataspeqgedpi\",\"listAfter\":\"dataqpjzt\",\"listBefore\":\"datadiverkwmafyxo\",\"enablePartitionDiscovery\":\"datakvutedetx\",\"partitionRootPath\":\"dataqudjdwcw\",\"deleteFilesAfterCompletion\":\"datacdbkceh\",\"modifiedDatetimeStart\":\"datahnqjbavdblf\",\"modifiedDatetimeEnd\":\"databbvitlnnpafu\",\"maxConcurrentConnections\":\"datar\",\"disableMetricsCollection\":\"databndr\",\"\":{\"vmdccte\":\"datanstl\",\"urz\":\"datavaajyitpyzgwihks\"}}") .toObject(AzureDataLakeStoreReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureDataLakeStoreReadSettings model = new AzureDataLakeStoreReadSettings() - .withMaxConcurrentConnections("datasuyqvmxnavxzp").withDisableMetricsCollection("datapt") - .withRecursive("datapohcrkmbajyuegs").withWildcardFolderPath("dataqdpzhqeczzutxtid") - .withWildcardFileName("datarexbarg").withFileListPath("datae").withListAfter("datapy") - .withListBefore("datauomowucj").withEnablePartitionDiscovery("datanowpvxuuvhwsrme") - .withPartitionRootPath("datahw").withDeleteFilesAfterCompletion("datafkjd") - .withModifiedDatetimeStart("dataqtqjabwtkpjytv").withModifiedDatetimeEnd("datair"); + .withMaxConcurrentConnections("datar").withDisableMetricsCollection("databndr").withRecursive("datayjuy") + .withWildcardFolderPath("datalbbugojd").withWildcardFileName("dataluyosigkinykjx") + .withFileListPath("dataspeqgedpi").withListAfter("dataqpjzt").withListBefore("datadiverkwmafyxo") + .withEnablePartitionDiscovery("datakvutedetx").withPartitionRootPath("dataqudjdwcw") + .withDeleteFilesAfterCompletion("datacdbkceh").withModifiedDatetimeStart("datahnqjbavdblf") + .withModifiedDatetimeEnd("databbvitlnnpafu"); model = BinaryData.fromObject(model).toObject(AzureDataLakeStoreReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSinkTests.java index 3cefce459fa61..57c009410f4c6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSinkTests.java @@ -11,16 +11,17 @@ public final class AzureDataLakeStoreSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataLakeStoreSink model = BinaryData.fromString( - "{\"type\":\"AzureDataLakeStoreSink\",\"copyBehavior\":\"datamyizdglzzaufi\",\"enableAdlsSingleFileParallel\":\"datavyxyrykn\",\"writeBatchSize\":\"datatjgpyvjgsjyjnhwb\",\"writeBatchTimeout\":\"datawrncxw\",\"sinkRetryCount\":\"dataerr\",\"sinkRetryWait\":\"dataamfpin\",\"maxConcurrentConnections\":\"datapb\",\"disableMetricsCollection\":\"datamfbruuhylqgeovn\",\"\":{\"rqu\":\"datafsol\",\"jshicvrmwbgpc\":\"datalnhxr\",\"bxppvpgsrfshkjg\":\"datal\",\"rooogijiqwxwpub\":\"datapboaevtxi\"}}") + "{\"type\":\"AzureDataLakeStoreSink\",\"copyBehavior\":\"datar\",\"enableAdlsSingleFileParallel\":\"datavos\",\"writeBatchSize\":\"dataujhskxxekzys\",\"writeBatchTimeout\":\"datatfimcaxgtwpzqti\",\"sinkRetryCount\":\"dataicsfaqy\",\"sinkRetryWait\":\"datacpdtktfpjkxk\",\"maxConcurrentConnections\":\"datawntnfoqwufor\",\"disableMetricsCollection\":\"databe\",\"\":{\"pslcfwgrzzqf\":\"datapnsyedpyrp\",\"tjzdpllgllvkorg\":\"dataodifghdgsyhncxoq\",\"jaogmttxq\":\"datasoxxoqyik\",\"fnsdccmdplhzjiq\":\"datavmybqjlgr\"}}") .toObject(AzureDataLakeStoreSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDataLakeStoreSink model = new AzureDataLakeStoreSink().withWriteBatchSize("datatjgpyvjgsjyjnhwb") - .withWriteBatchTimeout("datawrncxw").withSinkRetryCount("dataerr").withSinkRetryWait("dataamfpin") - .withMaxConcurrentConnections("datapb").withDisableMetricsCollection("datamfbruuhylqgeovn") - .withCopyBehavior("datamyizdglzzaufi").withEnableAdlsSingleFileParallel("datavyxyrykn"); + AzureDataLakeStoreSink model = new AzureDataLakeStoreSink().withWriteBatchSize("dataujhskxxekzys") + .withWriteBatchTimeout("datatfimcaxgtwpzqti").withSinkRetryCount("dataicsfaqy") + .withSinkRetryWait("datacpdtktfpjkxk").withMaxConcurrentConnections("datawntnfoqwufor") + .withDisableMetricsCollection("databe").withCopyBehavior("datar") + .withEnableAdlsSingleFileParallel("datavos"); model = BinaryData.fromObject(model).toObject(AzureDataLakeStoreSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSourceTests.java index a4cb36f38868d..9e0ee8e8f2686 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreSourceTests.java @@ -11,15 +11,15 @@ public final class AzureDataLakeStoreSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataLakeStoreSource model = BinaryData.fromString( - "{\"type\":\"AzureDataLakeStoreSource\",\"recursive\":\"datawdrvkb\",\"sourceRetryCount\":\"datavnnvk\",\"sourceRetryWait\":\"datazldzzjj\",\"maxConcurrentConnections\":\"datahjqengopdvnzn\",\"disableMetricsCollection\":\"dataiodaj\",\"\":{\"whnzhsmu\":\"datazdyvzmiufbwre\",\"svzdvakqajian\":\"dataedbhnkl\"}}") + "{\"type\":\"AzureDataLakeStoreSource\",\"recursive\":\"dataf\",\"sourceRetryCount\":\"datal\",\"sourceRetryWait\":\"dataxoudjcttavbc\",\"maxConcurrentConnections\":\"datazvqzmlferjw\",\"disableMetricsCollection\":\"datannqudexnicqu\",\"\":{\"hhsvsnedh\":\"datattfqgdoowgqooi\",\"flrrtju\":\"datajiwfvetwfreq\",\"wqal\":\"dataikqzd\",\"cisolkwipvls\":\"datapmiytpji\"}}") .toObject(AzureDataLakeStoreSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDataLakeStoreSource model = new AzureDataLakeStoreSource().withSourceRetryCount("datavnnvk") - .withSourceRetryWait("datazldzzjj").withMaxConcurrentConnections("datahjqengopdvnzn") - .withDisableMetricsCollection("dataiodaj").withRecursive("datawdrvkb"); + AzureDataLakeStoreSource model = new AzureDataLakeStoreSource().withSourceRetryCount("datal") + .withSourceRetryWait("dataxoudjcttavbc").withMaxConcurrentConnections("datazvqzmlferjw") + .withDisableMetricsCollection("datannqudexnicqu").withRecursive("dataf"); model = BinaryData.fromObject(model).toObject(AzureDataLakeStoreSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreWriteSettingsTests.java index dcb86a031284d..42ccebaa5a8ef 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDataLakeStoreWriteSettingsTests.java @@ -13,20 +13,18 @@ public final class AzureDataLakeStoreWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDataLakeStoreWriteSettings model = BinaryData.fromString( - "{\"type\":\"AzureDataLakeStoreWriteSettings\",\"expiryDateTime\":\"datarbjxewcscuveljf\",\"maxConcurrentConnections\":\"datain\",\"disableMetricsCollection\":\"dataziztgddahymv\",\"copyBehavior\":\"datajtdhmig\",\"metadata\":[{\"name\":\"dataexgien\",\"value\":\"datasijqygofshdaehxv\"},{\"name\":\"datafdsafgkysymhuxs\",\"value\":\"datallbpegcetezaa\"},{\"name\":\"dataszrbttz\",\"value\":\"dataeyrw\"},{\"name\":\"datafgoyxxszpaiecurf\",\"value\":\"datafshfmgiixurve\"}],\"\":{\"iuwhcyckekm\":\"datasmulvmywoefkpuuu\",\"oycpotmaosongtbh\":\"datafipygt\"}}") + "{\"type\":\"AzureDataLakeStoreWriteSettings\",\"expiryDateTime\":\"datarmtuprqt\",\"maxConcurrentConnections\":\"dataqkohupyajkde\",\"disableMetricsCollection\":\"dataarjv\",\"copyBehavior\":\"dataozfjbdyyxhjfzjb\",\"metadata\":[{\"name\":\"datalnbklhwri\",\"value\":\"datauljbhgzffe\"},{\"name\":\"datayoiaobbzcdlcc\",\"value\":\"datamvbhb\"}],\"\":{\"x\":\"databxolzin\",\"gzgsgzlbunmjha\":\"datafixrukwxcaagzlqo\",\"ysrreebj\":\"datafiyrywf\",\"dtykhsafrfv\":\"dataslbxfkiiarlldygf\"}}") .toObject(AzureDataLakeStoreWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureDataLakeStoreWriteSettings model - = new AzureDataLakeStoreWriteSettings().withMaxConcurrentConnections("datain") - .withDisableMetricsCollection("dataziztgddahymv").withCopyBehavior("datajtdhmig") - .withMetadata(Arrays.asList(new MetadataItem().withName("dataexgien").withValue("datasijqygofshdaehxv"), - new MetadataItem().withName("datafdsafgkysymhuxs").withValue("datallbpegcetezaa"), - new MetadataItem().withName("dataszrbttz").withValue("dataeyrw"), - new MetadataItem().withName("datafgoyxxszpaiecurf").withValue("datafshfmgiixurve"))) - .withExpiryDateTime("datarbjxewcscuveljf"); + = new AzureDataLakeStoreWriteSettings().withMaxConcurrentConnections("dataqkohupyajkde") + .withDisableMetricsCollection("dataarjv").withCopyBehavior("dataozfjbdyyxhjfzjb") + .withMetadata(Arrays.asList(new MetadataItem().withName("datalnbklhwri").withValue("datauljbhgzffe"), + new MetadataItem().withName("datayoiaobbzcdlcc").withValue("datamvbhb"))) + .withExpiryDateTime("datarmtuprqt"); model = BinaryData.fromObject(model).toObject(AzureDataLakeStoreWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeExportCommandTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeExportCommandTests.java index c632728f683f4..a62c7df53ad6d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeExportCommandTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeExportCommandTests.java @@ -11,14 +11,14 @@ public final class AzureDatabricksDeltaLakeExportCommandTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDatabricksDeltaLakeExportCommand model = BinaryData.fromString( - "{\"type\":\"AzureDatabricksDeltaLakeExportCommand\",\"dateFormat\":\"datapywe\",\"timestampFormat\":\"dataoizabjbh\",\"\":{\"ktigiseeplzrerx\":\"datadijf\",\"ncearkjtmpa\":\"datadsom\"}}") + "{\"type\":\"AzureDatabricksDeltaLakeExportCommand\",\"dateFormat\":\"dataphnag\",\"timestampFormat\":\"datalaxjmnbmfmloq\",\"\":{\"m\":\"datawvtddpicwnbtvlrs\",\"dknxerkaiikbpf\":\"datarm\"}}") .toObject(AzureDatabricksDeltaLakeExportCommand.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureDatabricksDeltaLakeExportCommand model = new AzureDatabricksDeltaLakeExportCommand() - .withDateFormat("datapywe").withTimestampFormat("dataoizabjbh"); + .withDateFormat("dataphnag").withTimestampFormat("datalaxjmnbmfmloq"); model = BinaryData.fromObject(model).toObject(AzureDatabricksDeltaLakeExportCommand.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeImportCommandTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeImportCommandTests.java index 0974281ad892a..b33520337b6f4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeImportCommandTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeImportCommandTests.java @@ -11,14 +11,14 @@ public final class AzureDatabricksDeltaLakeImportCommandTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDatabricksDeltaLakeImportCommand model = BinaryData.fromString( - "{\"type\":\"AzureDatabricksDeltaLakeImportCommand\",\"dateFormat\":\"dataak\",\"timestampFormat\":\"dataotmge\",\"\":{\"sqv\":\"datacqvzmywvsynsccm\",\"olmkklzaba\":\"dataj\",\"oidjhoykg\":\"datavnclnt\"}}") + "{\"type\":\"AzureDatabricksDeltaLakeImportCommand\",\"dateFormat\":\"dataaqdoo\",\"timestampFormat\":\"datanzkmjoybyogw\",\"\":{\"hxawohsj\":\"datasnryk\",\"yzvrixcveserltlh\":\"datawxphnlw\",\"ksfxdmbxfyxweiq\":\"datajgjuopvkrms\",\"iucu\":\"datahfyvkxgoxsv\"}}") .toObject(AzureDatabricksDeltaLakeImportCommand.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDatabricksDeltaLakeImportCommand model - = new AzureDatabricksDeltaLakeImportCommand().withDateFormat("dataak").withTimestampFormat("dataotmge"); + AzureDatabricksDeltaLakeImportCommand model = new AzureDatabricksDeltaLakeImportCommand() + .withDateFormat("dataaqdoo").withTimestampFormat("datanzkmjoybyogw"); model = BinaryData.fromObject(model).toObject(AzureDatabricksDeltaLakeImportCommand.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSinkTests.java index 634d96550a2ff..6b5598ef2cd76 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSinkTests.java @@ -12,18 +12,18 @@ public final class AzureDatabricksDeltaLakeSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDatabricksDeltaLakeSink model = BinaryData.fromString( - "{\"type\":\"AzureDatabricksDeltaLakeSink\",\"preCopyScript\":\"datak\",\"importSettings\":{\"type\":\"AzureDatabricksDeltaLakeImportCommand\",\"dateFormat\":\"dataakgrwtpes\",\"timestampFormat\":\"datanmjmpceu\",\"\":{\"yhhgqys\":\"datadylv\",\"jzhvej\":\"dataju\",\"pa\":\"datajekolnylpyk\",\"heycakkonaydpdw\":\"dataaouyaanfxai\"}},\"writeBatchSize\":\"dataadzyxaanhwuqewcn\",\"writeBatchTimeout\":\"dataksfbkxfkeeqofwb\",\"sinkRetryCount\":\"datapeerwss\",\"sinkRetryWait\":\"datamrpdjrylfpdudx\",\"maxConcurrentConnections\":\"datageurie\",\"disableMetricsCollection\":\"databanfsqfhatqssnge\",\"\":{\"apynpvgyafttbet\":\"dataffgohrhjspsyh\",\"ybjku\":\"dataydtgpvnczfp\",\"zpteron\":\"datakajkyrhucbfkaqlp\"}}") + "{\"type\":\"AzureDatabricksDeltaLakeSink\",\"preCopyScript\":\"dataxpqrkeyhj\",\"importSettings\":{\"type\":\"AzureDatabricksDeltaLakeImportCommand\",\"dateFormat\":\"datazkbrvta\",\"timestampFormat\":\"datahtqvt\",\"\":{\"qjjxhijb\":\"datadijcndwoyqvc\",\"vpd\":\"dataiyuhoxul\",\"mphyacdhjmpnv\":\"datairhg\",\"hljtkuyvytfuq\":\"datakxs\"}},\"writeBatchSize\":\"datatqbxpy\",\"writeBatchTimeout\":\"datawkjeitkfhzv\",\"sinkRetryCount\":\"datandbklscoka\",\"sinkRetryWait\":\"dataqqipvnvdzssss\",\"maxConcurrentConnections\":\"datagh\",\"disableMetricsCollection\":\"datadqkotxodbxzh\",\"\":{\"yy\":\"datawjnnoot\"}}") .toObject(AzureDatabricksDeltaLakeSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDatabricksDeltaLakeSink model = new AzureDatabricksDeltaLakeSink() - .withWriteBatchSize("dataadzyxaanhwuqewcn").withWriteBatchTimeout("dataksfbkxfkeeqofwb") - .withSinkRetryCount("datapeerwss").withSinkRetryWait("datamrpdjrylfpdudx") - .withMaxConcurrentConnections("datageurie").withDisableMetricsCollection("databanfsqfhatqssnge") - .withPreCopyScript("datak").withImportSettings(new AzureDatabricksDeltaLakeImportCommand() - .withDateFormat("dataakgrwtpes").withTimestampFormat("datanmjmpceu")); + AzureDatabricksDeltaLakeSink model = new AzureDatabricksDeltaLakeSink().withWriteBatchSize("datatqbxpy") + .withWriteBatchTimeout("datawkjeitkfhzv").withSinkRetryCount("datandbklscoka") + .withSinkRetryWait("dataqqipvnvdzssss").withMaxConcurrentConnections("datagh") + .withDisableMetricsCollection("datadqkotxodbxzh").withPreCopyScript("dataxpqrkeyhj") + .withImportSettings(new AzureDatabricksDeltaLakeImportCommand().withDateFormat("datazkbrvta") + .withTimestampFormat("datahtqvt")); model = BinaryData.fromObject(model).toObject(AzureDatabricksDeltaLakeSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSourceTests.java index 0cc6a3c950bba..5d27ba2330c34 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureDatabricksDeltaLakeSourceTests.java @@ -12,17 +12,17 @@ public final class AzureDatabricksDeltaLakeSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureDatabricksDeltaLakeSource model = BinaryData.fromString( - "{\"type\":\"AzureDatabricksDeltaLakeSource\",\"query\":\"dataky\",\"exportSettings\":{\"type\":\"AzureDatabricksDeltaLakeExportCommand\",\"dateFormat\":\"dataeeeucvvnbymrgel\",\"timestampFormat\":\"dataduexxf\",\"\":{\"qtnozwphkaqracv\":\"datarzenvkqtvtqpwqz\"}},\"sourceRetryCount\":\"datartl\",\"sourceRetryWait\":\"dataohbzbbjxkamitgvq\",\"maxConcurrentConnections\":\"datadvdbyelrteunkwyp\",\"disableMetricsCollection\":\"dataqfm\",\"\":{\"lylzdbrwjlwfthf\":\"datatdfqlfd\",\"jsfgkwrcbgxypr\":\"dataxwi\"}}") + "{\"type\":\"AzureDatabricksDeltaLakeSource\",\"query\":\"datall\",\"exportSettings\":{\"type\":\"AzureDatabricksDeltaLakeExportCommand\",\"dateFormat\":\"datahthxcrweeqkdmpfm\",\"timestampFormat\":\"datacelsnjf\",\"\":{\"ikmgwxysu\":\"datadcjtveibnt\",\"kstrmsbmdgrzke\":\"datasofdhrif\",\"aaxz\":\"dataplorntnss\",\"pisc\":\"datadlnv\"}},\"sourceRetryCount\":\"datayhtb\",\"sourceRetryWait\":\"dataycacoelvoy\",\"maxConcurrentConnections\":\"datamxqalqqrymjwwox\",\"disableMetricsCollection\":\"dataefellhdsgo\",\"\":{\"malthcbvuvwdp\":\"datab\"}}") .toObject(AzureDatabricksDeltaLakeSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureDatabricksDeltaLakeSource model = new AzureDatabricksDeltaLakeSource().withSourceRetryCount("datartl") - .withSourceRetryWait("dataohbzbbjxkamitgvq").withMaxConcurrentConnections("datadvdbyelrteunkwyp") - .withDisableMetricsCollection("dataqfm").withQuery("dataky") - .withExportSettings(new AzureDatabricksDeltaLakeExportCommand().withDateFormat("dataeeeucvvnbymrgel") - .withTimestampFormat("dataduexxf")); + AzureDatabricksDeltaLakeSource model = new AzureDatabricksDeltaLakeSource().withSourceRetryCount("datayhtb") + .withSourceRetryWait("dataycacoelvoy").withMaxConcurrentConnections("datamxqalqqrymjwwox") + .withDisableMetricsCollection("dataefellhdsgo").withQuery("datall") + .withExportSettings(new AzureDatabricksDeltaLakeExportCommand().withDateFormat("datahthxcrweeqkdmpfm") + .withTimestampFormat("datacelsnjf")); model = BinaryData.fromObject(model).toObject(AzureDatabricksDeltaLakeSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageReadSettingsTests.java index 8d750dfda4a1f..f8cd65c858534 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageReadSettingsTests.java @@ -11,19 +11,18 @@ public final class AzureFileStorageReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureFileStorageReadSettings model = BinaryData.fromString( - "{\"type\":\"AzureFileStorageReadSettings\",\"recursive\":\"dataxjgkrppm\",\"wildcardFolderPath\":\"dataoqtwqciqawhkdkva\",\"wildcardFileName\":\"datapbds\",\"prefix\":\"datatidikxmtmj\",\"fileListPath\":\"datamrjngrjsqtirhab\",\"enablePartitionDiscovery\":\"datapcvsd\",\"partitionRootPath\":\"datambydrgxvnmtjmu\",\"deleteFilesAfterCompletion\":\"datadmudw\",\"modifiedDatetimeStart\":\"dataogmth\",\"modifiedDatetimeEnd\":\"datacyycxl\",\"maxConcurrentConnections\":\"datakfjgj\",\"disableMetricsCollection\":\"datafssgieb\",\"\":{\"lamd\":\"datascmcegyiwzou\"}}") + "{\"type\":\"AzureFileStorageReadSettings\",\"recursive\":\"datajldwxdqt\",\"wildcardFolderPath\":\"datatgn\",\"wildcardFileName\":\"databjvmdkgvu\",\"prefix\":\"datamlsuuhwuox\",\"fileListPath\":\"datai\",\"enablePartitionDiscovery\":\"datazzjo\",\"partitionRootPath\":\"dataygzjrkslqba\",\"deleteFilesAfterCompletion\":\"databjxxcruleim\",\"modifiedDatetimeStart\":\"dataxoign\",\"modifiedDatetimeEnd\":\"datamjmpgzetuvfp\",\"maxConcurrentConnections\":\"datajpmeptnqsnpa\",\"disableMetricsCollection\":\"datasprrvjwbeeolm\",\"\":{\"fk\":\"dataqol\",\"aphlwmivazfnb\":\"datadwzvhtgfdy\"}}") .toObject(AzureFileStorageReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureFileStorageReadSettings model = new AzureFileStorageReadSettings() - .withMaxConcurrentConnections("datakfjgj").withDisableMetricsCollection("datafssgieb") - .withRecursive("dataxjgkrppm").withWildcardFolderPath("dataoqtwqciqawhkdkva") - .withWildcardFileName("datapbds").withPrefix("datatidikxmtmj").withFileListPath("datamrjngrjsqtirhab") - .withEnablePartitionDiscovery("datapcvsd").withPartitionRootPath("datambydrgxvnmtjmu") - .withDeleteFilesAfterCompletion("datadmudw").withModifiedDatetimeStart("dataogmth") - .withModifiedDatetimeEnd("datacyycxl"); + .withMaxConcurrentConnections("datajpmeptnqsnpa").withDisableMetricsCollection("datasprrvjwbeeolm") + .withRecursive("datajldwxdqt").withWildcardFolderPath("datatgn").withWildcardFileName("databjvmdkgvu") + .withPrefix("datamlsuuhwuox").withFileListPath("datai").withEnablePartitionDiscovery("datazzjo") + .withPartitionRootPath("dataygzjrkslqba").withDeleteFilesAfterCompletion("databjxxcruleim") + .withModifiedDatetimeStart("dataxoign").withModifiedDatetimeEnd("datamjmpgzetuvfp"); model = BinaryData.fromObject(model).toObject(AzureFileStorageReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageWriteSettingsTests.java index 67a11f7dacef9..81297a923a343 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFileStorageWriteSettingsTests.java @@ -13,19 +13,18 @@ public final class AzureFileStorageWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureFileStorageWriteSettings model = BinaryData.fromString( - "{\"type\":\"AzureFileStorageWriteSettings\",\"maxConcurrentConnections\":\"datapobjufksddxk\",\"disableMetricsCollection\":\"datawxlylxfpvoylf\",\"copyBehavior\":\"datasrgu\",\"metadata\":[{\"name\":\"datahauivgbim\",\"value\":\"dataedqgyrvulz\"},{\"name\":\"datavckpdp\",\"value\":\"datanrjqskikqd\"},{\"name\":\"dataybqtlvofjjsetiz\",\"value\":\"datanadn\"},{\"name\":\"datasbpxlserqgxnh\",\"value\":\"dataccd\"}],\"\":{\"jbhmpmeglo\":\"datayb\"}}") + "{\"type\":\"AzureFileStorageWriteSettings\",\"maxConcurrentConnections\":\"datavgwvfvsqlyah\",\"disableMetricsCollection\":\"dataoqk\",\"copyBehavior\":\"datatnbuzvaxlt\",\"metadata\":[{\"name\":\"datahic\",\"value\":\"dataauvprqzpfpbxljdd\"},{\"name\":\"dataoyzs\",\"value\":\"datavkcldons\"},{\"name\":\"dataazxewnl\",\"value\":\"datahhczqm\"}],\"\":{\"rmfclkyncjyafzz\":\"datay\",\"avo\":\"databohb\",\"bverbjctszb\":\"dataerduab\"}}") .toObject(AzureFileStorageWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureFileStorageWriteSettings model - = new AzureFileStorageWriteSettings().withMaxConcurrentConnections("datapobjufksddxk") - .withDisableMetricsCollection("datawxlylxfpvoylf").withCopyBehavior("datasrgu") - .withMetadata(Arrays.asList(new MetadataItem().withName("datahauivgbim").withValue("dataedqgyrvulz"), - new MetadataItem().withName("datavckpdp").withValue("datanrjqskikqd"), - new MetadataItem().withName("dataybqtlvofjjsetiz").withValue("datanadn"), - new MetadataItem().withName("datasbpxlserqgxnh").withValue("dataccd"))); + = new AzureFileStorageWriteSettings().withMaxConcurrentConnections("datavgwvfvsqlyah") + .withDisableMetricsCollection("dataoqk").withCopyBehavior("datatnbuzvaxlt") + .withMetadata(Arrays.asList(new MetadataItem().withName("datahic").withValue("dataauvprqzpfpbxljdd"), + new MetadataItem().withName("dataoyzs").withValue("datavkcldons"), + new MetadataItem().withName("dataazxewnl").withValue("datahhczqm"))); model = BinaryData.fromObject(model).toObject(AzureFileStorageWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTests.java index b60a1e55c5cf0..bc36b8203a0a3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTests.java @@ -23,65 +23,61 @@ public final class AzureFunctionActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureFunctionActivity model = BinaryData.fromString( - "{\"type\":\"AzureFunctionActivity\",\"typeProperties\":{\"method\":\"HEAD\",\"functionName\":\"datafexzzijtctfew\",\"headers\":{\"vqpwwv\":\"tzplwyl\"},\"body\":\"dataqsnttwlxvezoald\"},\"linkedServiceName\":{\"referenceName\":\"esxcqtosoanxi\",\"parameters\":{\"s\":\"dataiccgubuiv\",\"wybbda\":\"datafxnenhyhdusaykrj\",\"xmdacetjmap\":\"datadqttzslt\"}},\"policy\":{\"timeout\":\"dataaumqjx\",\"retry\":\"datardcxuiamrxlhfx\",\"retryIntervalInSeconds\":1561615229,\"secureInput\":true,\"secureOutput\":true,\"\":{\"zfyivvtxqp\":\"datat\"}},\"name\":\"mqogtohzfvysv\",\"description\":\"bjdihtxv\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"qdmzjppb\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Skipped\"],\"\":{\"tn\":\"datamz\",\"yyasildbqygnfxg\":\"dataxvt\",\"bkbjs\":\"datazqqwsvjhmqp\",\"pe\":\"datagkdvm\"}},{\"activity\":\"tfbgyznshoe\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Completed\",\"Succeeded\"],\"\":{\"g\":\"datahvuoavpoookhcur\"}},{\"activity\":\"jzznmjwqwyhhgs\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Completed\",\"Failed\"],\"\":{\"rxfbuywzpcxnbbs\":\"datanj\"}}],\"userProperties\":[{\"name\":\"alowmmhhuh\",\"value\":\"datawdc\"},{\"name\":\"vc\",\"value\":\"datafstceehqea\"},{\"name\":\"lfuj\",\"value\":\"dataovtakijwkw\"},{\"name\":\"druumldunalogw\",\"value\":\"dataikfqcbe\"}],\"\":{\"qiulwfzoszgbgtwa\":\"dataszucdvhqec\",\"vdaj\":\"datauiuzsnjjgnmpuqsj\",\"dtbgkxzx\":\"dataczlvcxmtwtbr\"}}") + "{\"type\":\"AzureFunctionActivity\",\"typeProperties\":{\"method\":\"PUT\",\"functionName\":\"dataiarfkfgrdri\",\"headers\":{\"rqpickn\":\"fx\"},\"body\":\"datazdrd\"},\"linkedServiceName\":{\"referenceName\":\"wmueavawywofgc\",\"parameters\":{\"mx\":\"datahjvvrrxclf\",\"igxwxxfkfthw\":\"datafqwyiuhhuftn\",\"lstgsmeijgjbev\":\"dataossokafy\",\"wvdklgwoyw\":\"datasrcsyjx\"}},\"policy\":{\"timeout\":\"datafmenbaj\",\"retry\":\"dataelbcsyaohizf\",\"retryIntervalInSeconds\":1921383033,\"secureInput\":false,\"secureOutput\":false,\"\":{\"y\":\"datatkddohxvcsoq\",\"danufiwtkhcmoc\":\"datacqpmywt\",\"khmbks\":\"datagtmfug\"}},\"name\":\"kkztexds\",\"description\":\"hndcrdvecc\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"ahuu\",\"dependencyConditions\":[\"Failed\",\"Succeeded\"],\"\":{\"pfisyydoy\":\"datavkolfiigoxohjy\",\"hqvmilpgxeaqwogp\":\"dataccwvcfayllx\",\"mcoruti\":\"datatsmyfgted\"}},{\"activity\":\"dkypckhqooqni\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Skipped\"],\"\":{\"ngvpsuk\":\"datatmkmgcipvr\",\"gq\":\"datakmkghugfd\"}}],\"userProperties\":[{\"name\":\"uekoxylcbpb\",\"value\":\"datajtiidozf\"},{\"name\":\"gvq\",\"value\":\"datarrknijdrsvoh\"}],\"\":{\"lzsgpoiccbzqko\":\"dataoiikr\",\"dm\":\"dataja\",\"lz\":\"datazkq\"}}") .toObject(AzureFunctionActivity.class); - Assertions.assertEquals("mqogtohzfvysv", model.name()); - Assertions.assertEquals("bjdihtxv", model.description()); + Assertions.assertEquals("kkztexds", model.name()); + Assertions.assertEquals("hndcrdvecc", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("qdmzjppb", model.dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("ahuu", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("alowmmhhuh", model.userProperties().get(0).name()); - Assertions.assertEquals("esxcqtosoanxi", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1561615229, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals(AzureFunctionActivityMethod.HEAD, model.method()); - Assertions.assertEquals("tzplwyl", model.headers().get("vqpwwv")); + Assertions.assertEquals("uekoxylcbpb", model.userProperties().get(0).name()); + Assertions.assertEquals("wmueavawywofgc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1921383033, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals(AzureFunctionActivityMethod.PUT, model.method()); + Assertions.assertEquals("fx", model.headers().get("rqpickn")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureFunctionActivity model = new AzureFunctionActivity().withName("mqogtohzfvysv").withDescription("bjdihtxv") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("qdmzjppb") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("tfbgyznshoe") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jzznmjwqwyhhgs") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("alowmmhhuh").withValue("datawdc"), - new UserProperty().withName("vc").withValue("datafstceehqea"), - new UserProperty().withName("lfuj").withValue("dataovtakijwkw"), - new UserProperty().withName("druumldunalogw").withValue("dataikfqcbe"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("esxcqtosoanxi").withParameters( - mapOf("s", "dataiccgubuiv", "wybbda", "datafxnenhyhdusaykrj", "xmdacetjmap", "datadqttzslt"))) - .withPolicy(new ActivityPolicy().withTimeout("dataaumqjx").withRetry("datardcxuiamrxlhfx") - .withRetryIntervalInSeconds(1561615229).withSecureInput(true).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withMethod(AzureFunctionActivityMethod.HEAD).withFunctionName("datafexzzijtctfew") - .withHeaders(mapOf("vqpwwv", "tzplwyl")).withBody("dataqsnttwlxvezoald"); + AzureFunctionActivity model + = new AzureFunctionActivity().withName("kkztexds").withDescription("hndcrdvecc") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("ahuu") + .withDependencyConditions( + Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("dkypckhqooqni") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("uekoxylcbpb").withValue("datajtiidozf"), + new UserProperty().withName("gvq").withValue("datarrknijdrsvoh"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("wmueavawywofgc") + .withParameters(mapOf("mx", "datahjvvrrxclf", "igxwxxfkfthw", "datafqwyiuhhuftn", "lstgsmeijgjbev", + "dataossokafy", "wvdklgwoyw", "datasrcsyjx"))) + .withPolicy(new ActivityPolicy().withTimeout("datafmenbaj").withRetry("dataelbcsyaohizf") + .withRetryIntervalInSeconds(1921383033).withSecureInput(false).withSecureOutput(false) + .withAdditionalProperties(mapOf())) + .withMethod(AzureFunctionActivityMethod.PUT).withFunctionName("dataiarfkfgrdri") + .withHeaders(mapOf("rqpickn", "fx")).withBody("datazdrd"); model = BinaryData.fromObject(model).toObject(AzureFunctionActivity.class); - Assertions.assertEquals("mqogtohzfvysv", model.name()); - Assertions.assertEquals("bjdihtxv", model.description()); + Assertions.assertEquals("kkztexds", model.name()); + Assertions.assertEquals("hndcrdvecc", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("qdmzjppb", model.dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("ahuu", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("alowmmhhuh", model.userProperties().get(0).name()); - Assertions.assertEquals("esxcqtosoanxi", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1561615229, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals(AzureFunctionActivityMethod.HEAD, model.method()); - Assertions.assertEquals("tzplwyl", model.headers().get("vqpwwv")); + Assertions.assertEquals("uekoxylcbpb", model.userProperties().get(0).name()); + Assertions.assertEquals("wmueavawywofgc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1921383033, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals(AzureFunctionActivityMethod.PUT, model.method()); + Assertions.assertEquals("fx", model.headers().get("rqpickn")); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTypePropertiesTests.java index 204813a77299a..fbc2ee0172763 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureFunctionActivityTypePropertiesTests.java @@ -15,20 +15,20 @@ public final class AzureFunctionActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureFunctionActivityTypeProperties model = BinaryData.fromString( - "{\"method\":\"TRACE\",\"functionName\":\"datazleeupqls\",\"headers\":{\"dfxnxtiwi\":\"omayraatrjpa\",\"ihsgt\":\"nho\"},\"body\":\"datagmfnpeluvxsicp\"}") + "{\"method\":\"OPTIONS\",\"functionName\":\"dataazqsucttp\",\"headers\":{\"cqnglzfgepblh\":\"gbfiosdizp\"},\"body\":\"datagwvvenmuenoq\"}") .toObject(AzureFunctionActivityTypeProperties.class); - Assertions.assertEquals(AzureFunctionActivityMethod.TRACE, model.method()); - Assertions.assertEquals("omayraatrjpa", model.headers().get("dfxnxtiwi")); + Assertions.assertEquals(AzureFunctionActivityMethod.OPTIONS, model.method()); + Assertions.assertEquals("gbfiosdizp", model.headers().get("cqnglzfgepblh")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureFunctionActivityTypeProperties model = new AzureFunctionActivityTypeProperties() - .withMethod(AzureFunctionActivityMethod.TRACE).withFunctionName("datazleeupqls") - .withHeaders(mapOf("dfxnxtiwi", "omayraatrjpa", "ihsgt", "nho")).withBody("datagmfnpeluvxsicp"); + .withMethod(AzureFunctionActivityMethod.OPTIONS).withFunctionName("dataazqsucttp") + .withHeaders(mapOf("cqnglzfgepblh", "gbfiosdizp")).withBody("datagwvvenmuenoq"); model = BinaryData.fromObject(model).toObject(AzureFunctionActivityTypeProperties.class); - Assertions.assertEquals(AzureFunctionActivityMethod.TRACE, model.method()); - Assertions.assertEquals("omayraatrjpa", model.headers().get("dfxnxtiwi")); + Assertions.assertEquals(AzureFunctionActivityMethod.OPTIONS, model.method()); + Assertions.assertEquals("gbfiosdizp", model.headers().get("cqnglzfgepblh")); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTests.java index 3bcdc339aae60..1615a9d84306a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTests.java @@ -23,84 +23,86 @@ public final class AzureMLBatchExecutionActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLBatchExecutionActivity model = BinaryData.fromString( - "{\"type\":\"AzureMLBatchExecution\",\"typeProperties\":{\"globalParameters\":{\"bmjheyntsdwxpa\":\"dataahdplicivoduda\",\"crkf\":\"dataubgrjkgkoxuedml\",\"bvvjyenwvgvhhouh\":\"datagjywp\"},\"webServiceOutputs\":{\"asrwo\":{\"filePath\":\"datahwlkfljooiiviwlf\",\"linkedServiceName\":{\"referenceName\":\"joycyvxbr\",\"parameters\":{\"wwkofoqrvnhcu\":\"databit\",\"vtempyfjahwypd\":\"dataghvkzm\",\"tgexmxgqgqu\":\"datarqjjlsatoxsgar\",\"tub\":\"datalyrtkvftlbt\"}}},\"yvnbhn\":{\"filePath\":\"dataumnucqewxcwr\",\"linkedServiceName\":{\"referenceName\":\"kwmvcxyuem\",\"parameters\":{\"qisrlhxfmvngdrn\":\"datacytnzyesydw\"}}}},\"webServiceInputs\":{\"ennobjixoqqjbsag\":{\"filePath\":\"datawdborjypr\",\"linkedServiceName\":{\"referenceName\":\"ojwiigtdjqc\",\"parameters\":{\"ozofont\":\"datapkpibeift\",\"qugycorgnxmn\":\"datahlnaymsgbyho\"}}},\"vdtzdrvaxs\":{\"filePath\":\"datalpuqfmrimwlpa\",\"linkedServiceName\":{\"referenceName\":\"hwxu\",\"parameters\":{\"efwnjsorhpgavro\":\"datatnmwlklqhwysie\",\"lsuiyvbildw\":\"datakoezcabf\",\"nvjqeigpylpmtw\":\"datal\"}}}}},\"linkedServiceName\":{\"referenceName\":\"indyurwzrxkha\",\"parameters\":{\"slpbvponxhs\":\"datajambtvc\"}},\"policy\":{\"timeout\":\"datatunnkbekzk\",\"retry\":\"dataonbziklqyzrta\",\"retryIntervalInSeconds\":2109115182,\"secureInput\":true,\"secureOutput\":false,\"\":{\"seqxwcimam\":\"datahasxjmfh\",\"nrj\":\"dataqfrdfoiqfvczuu\",\"roxvsclmt\":\"datarpxlfyytjm\",\"eitphzuaznsbvu\":\"datakmlfcgk\"}},\"name\":\"bestyy\",\"description\":\"jotimpuwgrnyx\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"udranmdcfwaw\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Failed\",\"Succeeded\"],\"\":{\"hic\":\"datacvnafx\"}},{\"activity\":\"cmviclhommhaxt\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Succeeded\",\"Completed\"],\"\":{\"jfzbavqmmk\":\"datan\"}}],\"userProperties\":[{\"name\":\"fjeuwwqstc\",\"value\":\"datapskzplbzyjuqvyw\"},{\"name\":\"lccxdctkh\",\"value\":\"datahbosax\"},{\"name\":\"emspnzqohh\",\"value\":\"dataexgxn\"},{\"name\":\"fodxiy\",\"value\":\"datavfottycfo\"}],\"\":{\"lcb\":\"datadxbsp\",\"zifyhujgrb\":\"dataahxsjxurrhpihtxg\"}}") + "{\"type\":\"AzureMLBatchExecution\",\"typeProperties\":{\"globalParameters\":{\"qqkh\":\"datajthluoyrq\",\"dwsentrcdzyvxwnm\":\"datapws\",\"wpqvdduvxmrbb\":\"dataumd\",\"dtywzrnxiktoki\":\"dataliwfbgkyon\"},\"webServiceOutputs\":{\"lxogimihxyxe\":{\"filePath\":\"datamdadfygj\",\"linkedServiceName\":{\"referenceName\":\"cfprioabqxwid\",\"parameters\":{\"xtsywrmmhaxmo\":\"dataonnolrs\",\"armnseigoalxwuqu\":\"datauotexlpqydgfzet\",\"ghs\":\"datazrskdovgkpqzzrx\"}}}},\"webServiceInputs\":{\"zco\":{\"filePath\":\"datawixdcytd\",\"linkedServiceName\":{\"referenceName\":\"am\",\"parameters\":{\"zlbcamdzoauvwjkg\":\"datab\"}}},\"pgkwtpzbsytwthv\":{\"filePath\":\"datawcnnzacqludq\",\"linkedServiceName\":{\"referenceName\":\"aqxztywzaq\",\"parameters\":{\"lzpowse\":\"datatstmyfebbt\"}}},\"oeky\":{\"filePath\":\"datadtsvgyzmafqsn\",\"linkedServiceName\":{\"referenceName\":\"u\",\"parameters\":{\"qyhr\":\"databyvwejyyngw\"}}},\"veyngzj\":{\"filePath\":\"datanvxco\",\"linkedServiceName\":{\"referenceName\":\"pdgnsmhrpzbyudko\",\"parameters\":{\"d\":\"datajaaocjlwcouwcrex\",\"iukvmzxr\":\"datamkzb\"}}}}},\"linkedServiceName\":{\"referenceName\":\"bk\",\"parameters\":{\"cqhlfqimjlde\":\"datahguvqghueh\"}},\"policy\":{\"timeout\":\"dataqnforujfluomaltv\",\"retry\":\"dataudhtdapkdahy\",\"retryIntervalInSeconds\":1087656358,\"secureInput\":true,\"secureOutput\":false,\"\":{\"xuibyfylh\":\"datagyqrmteicl\",\"uqylmlunquvl\":\"datawqp\"}},\"name\":\"al\",\"description\":\"uztlxfgy\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"dacskulfqcxz\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Skipped\",\"Succeeded\"],\"\":{\"euwfmrckatnjik\":\"datavsgrcrknnru\",\"ieoth\":\"datazhtovs\",\"smavtn\":\"datawokprvpkdkds\"}},{\"activity\":\"gfmtximnpcghcfud\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Succeeded\",\"Completed\"],\"\":{\"erukuoeyyxcdwl\":\"datargerybdiajeeah\",\"xyitezfoekax\":\"datakglahd\"}}],\"userProperties\":[{\"name\":\"tfzaqnoq\",\"value\":\"datafyofohu\"},{\"name\":\"pfxkj\",\"value\":\"datahgwgsbaewkkq\"},{\"name\":\"kuzifsguolfkupmw\",\"value\":\"datazsirhp\"},{\"name\":\"gqdz\",\"value\":\"datadrcj\"}],\"\":{\"liphcpu\":\"databs\",\"e\":\"datadbzxidqqeslnaox\",\"fzyxamyjhp\":\"datautrlzzztg\",\"ily\":\"datazuvsjblqmddtp\"}}") .toObject(AzureMLBatchExecutionActivity.class); - Assertions.assertEquals("bestyy", model.name()); - Assertions.assertEquals("jotimpuwgrnyx", model.description()); + Assertions.assertEquals("al", model.name()); + Assertions.assertEquals("uztlxfgy", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("udranmdcfwaw", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("fjeuwwqstc", model.userProperties().get(0).name()); - Assertions.assertEquals("indyurwzrxkha", model.linkedServiceName().referenceName()); - Assertions.assertEquals(2109115182, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("dacskulfqcxz", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("tfzaqnoq", model.userProperties().get(0).name()); + Assertions.assertEquals("bk", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1087656358, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("joycyvxbr", - model.webServiceOutputs().get("asrwo").linkedServiceName().referenceName()); - Assertions.assertEquals("ojwiigtdjqc", - model.webServiceInputs().get("ennobjixoqqjbsag").linkedServiceName().referenceName()); + Assertions.assertEquals("cfprioabqxwid", + model.webServiceOutputs().get("lxogimihxyxe").linkedServiceName().referenceName()); + Assertions.assertEquals("am", model.webServiceInputs().get("zco").linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMLBatchExecutionActivity model = new AzureMLBatchExecutionActivity().withName("bestyy") - .withDescription("jotimpuwgrnyx").withState(ActivityState.INACTIVE) + AzureMLBatchExecutionActivity model = new AzureMLBatchExecutionActivity().withName("al") + .withDescription("uztlxfgy").withState(ActivityState.INACTIVE) .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("udranmdcfwaw") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, - DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("dacskulfqcxz") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("cmviclhommhaxt") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED, + new ActivityDependency().withActivity("gfmtximnpcghcfud") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("fjeuwwqstc").withValue("datapskzplbzyjuqvyw"), - new UserProperty().withName("lccxdctkh").withValue("datahbosax"), - new UserProperty().withName("emspnzqohh").withValue("dataexgxn"), - new UserProperty().withName("fodxiy").withValue("datavfottycfo"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("indyurwzrxkha") - .withParameters(mapOf("slpbvponxhs", "datajambtvc"))) - .withPolicy(new ActivityPolicy().withTimeout("datatunnkbekzk").withRetry("dataonbziklqyzrta") - .withRetryIntervalInSeconds(2109115182).withSecureInput(true).withSecureOutput(false) + .withUserProperties(Arrays.asList(new UserProperty().withName("tfzaqnoq").withValue("datafyofohu"), + new UserProperty().withName("pfxkj").withValue("datahgwgsbaewkkq"), + new UserProperty().withName("kuzifsguolfkupmw").withValue("datazsirhp"), + new UserProperty().withName("gqdz").withValue("datadrcj"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("bk") + .withParameters(mapOf("cqhlfqimjlde", "datahguvqghueh"))) + .withPolicy(new ActivityPolicy().withTimeout("dataqnforujfluomaltv").withRetry("dataudhtdapkdahy") + .withRetryIntervalInSeconds(1087656358).withSecureInput(true).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withGlobalParameters(mapOf("bmjheyntsdwxpa", "dataahdplicivoduda", "crkf", "dataubgrjkgkoxuedml", - "bvvjyenwvgvhhouh", "datagjywp")) - .withWebServiceOutputs(mapOf("asrwo", - new AzureMLWebServiceFile().withFilePath("datahwlkfljooiiviwlf") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("joycyvxbr") - .withParameters(mapOf("wwkofoqrvnhcu", "databit", "vtempyfjahwypd", "dataghvkzm", "tgexmxgqgqu", - "datarqjjlsatoxsgar", "tub", "datalyrtkvftlbt"))), - "yvnbhn", - new AzureMLWebServiceFile().withFilePath("dataumnucqewxcwr") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("kwmvcxyuem") - .withParameters(mapOf("qisrlhxfmvngdrn", "datacytnzyesydw"))))) - .withWebServiceInputs(mapOf("ennobjixoqqjbsag", - new AzureMLWebServiceFile().withFilePath("datawdborjypr") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ojwiigtdjqc") - .withParameters(mapOf("ozofont", "datapkpibeift", "qugycorgnxmn", "datahlnaymsgbyho"))), - "vdtzdrvaxs", - new AzureMLWebServiceFile().withFilePath("datalpuqfmrimwlpa").withLinkedServiceName( - new LinkedServiceReference().withReferenceName("hwxu").withParameters(mapOf("efwnjsorhpgavro", - "datatnmwlklqhwysie", "lsuiyvbildw", "datakoezcabf", "nvjqeigpylpmtw", "datal"))))); + .withGlobalParameters(mapOf("qqkh", "datajthluoyrq", "dwsentrcdzyvxwnm", "datapws", "wpqvdduvxmrbb", + "dataumd", "dtywzrnxiktoki", "dataliwfbgkyon")) + .withWebServiceOutputs(mapOf("lxogimihxyxe", + new AzureMLWebServiceFile().withFilePath("datamdadfygj") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cfprioabqxwid") + .withParameters(mapOf("xtsywrmmhaxmo", "dataonnolrs", "armnseigoalxwuqu", "datauotexlpqydgfzet", + "ghs", "datazrskdovgkpqzzrx"))))) + .withWebServiceInputs( + mapOf("zco", + new AzureMLWebServiceFile().withFilePath("datawixdcytd") + .withLinkedServiceName(new LinkedServiceReference() + .withReferenceName("am").withParameters(mapOf("zlbcamdzoauvwjkg", "datab"))), + "pgkwtpzbsytwthv", + new AzureMLWebServiceFile().withFilePath("datawcnnzacqludq") + .withLinkedServiceName(new LinkedServiceReference() + .withReferenceName("aqxztywzaq").withParameters(mapOf("lzpowse", "datatstmyfebbt"))), + "oeky", + new AzureMLWebServiceFile().withFilePath("datadtsvgyzmafqsn") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("u") + .withParameters(mapOf("qyhr", "databyvwejyyngw"))), + "veyngzj", + new AzureMLWebServiceFile().withFilePath("datanvxco") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pdgnsmhrpzbyudko") + .withParameters(mapOf("d", "datajaaocjlwcouwcrex", "iukvmzxr", "datamkzb"))))); model = BinaryData.fromObject(model).toObject(AzureMLBatchExecutionActivity.class); - Assertions.assertEquals("bestyy", model.name()); - Assertions.assertEquals("jotimpuwgrnyx", model.description()); + Assertions.assertEquals("al", model.name()); + Assertions.assertEquals("uztlxfgy", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("udranmdcfwaw", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("fjeuwwqstc", model.userProperties().get(0).name()); - Assertions.assertEquals("indyurwzrxkha", model.linkedServiceName().referenceName()); - Assertions.assertEquals(2109115182, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("dacskulfqcxz", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("tfzaqnoq", model.userProperties().get(0).name()); + Assertions.assertEquals("bk", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1087656358, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("joycyvxbr", - model.webServiceOutputs().get("asrwo").linkedServiceName().referenceName()); - Assertions.assertEquals("ojwiigtdjqc", - model.webServiceInputs().get("ennobjixoqqjbsag").linkedServiceName().referenceName()); + Assertions.assertEquals("cfprioabqxwid", + model.webServiceOutputs().get("lxogimihxyxe").linkedServiceName().referenceName()); + Assertions.assertEquals("am", model.webServiceInputs().get("zco").linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTypePropertiesTests.java index fcb6c7b03f7e5..5880d3710fa10 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLBatchExecutionActivityTypePropertiesTests.java @@ -16,28 +16,46 @@ public final class AzureMLBatchExecutionActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLBatchExecutionActivityTypeProperties model = BinaryData.fromString( - "{\"globalParameters\":{\"imo\":\"dataagxj\"},\"webServiceOutputs\":{\"jk\":{\"filePath\":\"datafpscrfbdt\",\"linkedServiceName\":{\"referenceName\":\"cfw\",\"parameters\":{\"lola\":\"datauwjgfi\",\"kjbhsnyycya\":\"dataorzbkfefxvg\",\"el\":\"dataijzrtlfngonhmblk\"}}}},\"webServiceInputs\":{\"hygbe\":{\"filePath\":\"dataneuhpyne\",\"linkedServiceName\":{\"referenceName\":\"cafkqvcfzrg\",\"parameters\":{\"vxcrzpdqwa\":\"dataczwxyghspp\"}}}}}") + "{\"globalParameters\":{\"qtscduuywgcuvcfm\":\"datab\"},\"webServiceOutputs\":{\"bz\":{\"filePath\":\"datapvoazgtlxgtusw\",\"linkedServiceName\":{\"referenceName\":\"gtskolbjylostrc\",\"parameters\":{\"bwaiqs\":\"datace\"}}}},\"webServiceInputs\":{\"crkf\":{\"filePath\":\"datalphaul\",\"linkedServiceName\":{\"referenceName\":\"alspeanheswxll\",\"parameters\":{\"iycjulunbtuf\":\"datablfprskxhghvg\",\"njpivo\":\"datacipi\",\"ahdplicivoduda\":\"datazxkhoabmahj\",\"ubgrjkgkoxuedml\":\"databmjheyntsdwxpa\"}}},\"kofoqrvnhc\":{\"filePath\":\"datagjywp\",\"linkedServiceName\":{\"referenceName\":\"bvvjyenwvgvhhouh\",\"parameters\":{\"joycyvxbr\":\"datahwlkfljooiiviwlf\",\"trw\":\"datahwb\"}}},\"mridcy\":{\"filePath\":\"dataoghvkzmgvtempy\",\"linkedServiceName\":{\"referenceName\":\"jahwypdh\",\"parameters\":{\"xtg\":\"datajlsatoxsga\",\"ftlbtotu\":\"dataxmxgqgquulyrtk\",\"cwrykwmvcxyu\":\"datazasrwoxumnucqew\"}}},\"qugycorgnxmn\":{\"filePath\":\"datanz\",\"linkedServiceName\":{\"referenceName\":\"e\",\"parameters\":{\"lhxfmvngdrn\":\"datawgqis\",\"nbwdborjyprcojwi\":\"datayvnbhn\",\"eiftm\":\"datagtdjqczoqpkpi\",\"hlnaymsgbyho\":\"datazofont\"}}}}}") .toObject(AzureMLBatchExecutionActivityTypeProperties.class); - Assertions.assertEquals("cfw", model.webServiceOutputs().get("jk").linkedServiceName().referenceName()); - Assertions.assertEquals("cafkqvcfzrg", - model.webServiceInputs().get("hygbe").linkedServiceName().referenceName()); + Assertions.assertEquals("gtskolbjylostrc", + model.webServiceOutputs().get("bz").linkedServiceName().referenceName()); + Assertions.assertEquals("alspeanheswxll", + model.webServiceInputs().get("crkf").linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureMLBatchExecutionActivityTypeProperties model - = new AzureMLBatchExecutionActivityTypeProperties().withGlobalParameters(mapOf("imo", "dataagxj")) - .withWebServiceOutputs(mapOf("jk", new AzureMLWebServiceFile().withFilePath("datafpscrfbdt") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cfw").withParameters( - mapOf("lola", "datauwjgfi", "kjbhsnyycya", "dataorzbkfefxvg", "el", "dataijzrtlfngonhmblk"))))) - .withWebServiceInputs(mapOf("hygbe", - new AzureMLWebServiceFile().withFilePath("dataneuhpyne") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cafkqvcfzrg") - .withParameters(mapOf("vxcrzpdqwa", "dataczwxyghspp"))))); + = new AzureMLBatchExecutionActivityTypeProperties().withGlobalParameters(mapOf("qtscduuywgcuvcfm", "datab")) + .withWebServiceOutputs(mapOf("bz", + new AzureMLWebServiceFile().withFilePath("datapvoazgtlxgtusw") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("gtskolbjylostrc") + .withParameters(mapOf("bwaiqs", "datace"))))) + .withWebServiceInputs(mapOf("crkf", + new AzureMLWebServiceFile().withFilePath("datalphaul") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("alspeanheswxll") + .withParameters(mapOf("iycjulunbtuf", "datablfprskxhghvg", "njpivo", "datacipi", + "ahdplicivoduda", "datazxkhoabmahj", "ubgrjkgkoxuedml", "databmjheyntsdwxpa"))), + "kofoqrvnhc", + new AzureMLWebServiceFile().withFilePath("datagjywp") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("bvvjyenwvgvhhouh") + .withParameters(mapOf("joycyvxbr", "datahwlkfljooiiviwlf", "trw", "datahwb"))), + "mridcy", + new AzureMLWebServiceFile().withFilePath("dataoghvkzmgvtempy") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("jahwypdh") + .withParameters(mapOf("xtg", "datajlsatoxsga", "ftlbtotu", "dataxmxgqgquulyrtk", + "cwrykwmvcxyu", "datazasrwoxumnucqew"))), + "qugycorgnxmn", + new AzureMLWebServiceFile().withFilePath("datanz") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("e") + .withParameters(mapOf("lhxfmvngdrn", "datawgqis", "nbwdborjyprcojwi", "datayvnbhn", "eiftm", + "datagtdjqczoqpkpi", "hlnaymsgbyho", "datazofont"))))); model = BinaryData.fromObject(model).toObject(AzureMLBatchExecutionActivityTypeProperties.class); - Assertions.assertEquals("cfw", model.webServiceOutputs().get("jk").linkedServiceName().referenceName()); - Assertions.assertEquals("cafkqvcfzrg", - model.webServiceInputs().get("hygbe").linkedServiceName().referenceName()); + Assertions.assertEquals("gtskolbjylostrc", + model.webServiceOutputs().get("bz").linkedServiceName().referenceName()); + Assertions.assertEquals("alspeanheswxll", + model.webServiceInputs().get("crkf").linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTests.java index 96e594fde2dd2..a35b6b13b6efc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTests.java @@ -22,66 +22,59 @@ public final class AzureMLExecutePipelineActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLExecutePipelineActivity model = BinaryData.fromString( - "{\"type\":\"AzureMLExecutePipeline\",\"typeProperties\":{\"mlPipelineId\":\"databuf\",\"mlPipelineEndpointId\":\"datalkj\",\"version\":\"dataudigwkyy\",\"experimentName\":\"dataedgapraa\",\"mlPipelineParameters\":\"dataxgojtiupjgebns\",\"dataPathAssignments\":\"dataklncqoyghrbabxy\",\"mlParentRunId\":\"datajuxyfpcvblyeo\",\"continueOnStepFailure\":\"datathxkqczmfuhsupi\"},\"linkedServiceName\":{\"referenceName\":\"izkv\",\"parameters\":{\"wembonureklgunp\":\"datahrphvmezdfadrto\",\"gwxctdpjuwu\":\"dataj\",\"txulnntjiucno\":\"dataxxsmookhobzisqp\",\"flt\":\"dataiacvttdyv\"}},\"policy\":{\"timeout\":\"databzfyllkunwinqyw\",\"retry\":\"dataxuxztjece\",\"retryIntervalInSeconds\":1557491500,\"secureInput\":false,\"secureOutput\":true,\"\":{\"ayhp\":\"datalyjlkjhmug\",\"soi\":\"datastlsdgiqgeeqcgu\",\"wkkykaz\":\"dataevrglzx\",\"ktwk\":\"datadaqxnkdqsyhm\"}},\"name\":\"gvhwkw\",\"description\":\"jez\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"bkzzqwikqkxduh\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Failed\",\"Completed\"],\"\":{\"mmgsxolwofofmyl\":\"dataa\",\"lwhtpyk\":\"datadflfx\"}},{\"activity\":\"cccaujga\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Skipped\",\"Succeeded\"],\"\":{\"xbvg\":\"dataxdbgm\",\"p\":\"datamblntdy\"}},{\"activity\":\"sdi\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Failed\"],\"\":{\"suxvliqgawenwtmv\":\"datarwliute\",\"td\":\"datazsbqnvwgizv\",\"kz\":\"datajo\",\"ppjzmpxam\":\"datafnph\"}},{\"activity\":\"qdostvx\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Skipped\",\"Succeeded\"],\"\":{\"sq\":\"datadrbkkoocpdu\",\"pzumczlknfwslv\":\"datazerkpaivkgdrqkv\"}}],\"userProperties\":[{\"name\":\"rvhzfynbxwz\",\"value\":\"dataxmvl\"},{\"name\":\"uvbesra\",\"value\":\"datazxnwxsjygigepfok\"},{\"name\":\"lcnsxhpqey\",\"value\":\"datazydpvv\"}],\"\":{\"qfv\":\"datadjarfdf\",\"gnlbvbdqmjce\":\"datasxl\",\"ohyesmlscvh\":\"datafpubnx\"}}") + "{\"type\":\"AzureMLExecutePipeline\",\"typeProperties\":{\"mlPipelineId\":\"dataggyhpu\",\"mlPipelineEndpointId\":\"dataclehndbutptyabd\",\"version\":\"datakb\",\"experimentName\":\"datapemorfzuhvycd\",\"mlPipelineParameters\":\"dataczmzsfvriskplndd\",\"dataPathAssignments\":\"datacqinvkmkbtp\",\"mlParentRunId\":\"datathzmqabvwbgsanvd\",\"continueOnStepFailure\":\"datambxshrae\"},\"linkedServiceName\":{\"referenceName\":\"lhzme\",\"parameters\":{\"jeqdmolmcyba\":\"datazhrzeibkuuolul\",\"qamvdnexqvt\":\"datakeuraylygclwbu\",\"lkny\":\"datafnhzgtydllauno\",\"hhcqjahhcbzoary\":\"datapglgkeaz\"}},\"policy\":{\"timeout\":\"datamftgmql\",\"retry\":\"dataoyxfrrdbdyhjf\",\"retryIntervalInSeconds\":2106899128,\"secureInput\":false,\"secureOutput\":true,\"\":{\"mjcchqig\":\"datasyydbxlturln\",\"hltgteg\":\"dataamoz\"}},\"name\":\"nguvjryfcxscrs\",\"description\":\"tno\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"m\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"uetoeqfnhmrawm\":\"datanfgfsjptbysvw\",\"ewfjwfkwrthp\":\"datahcdegwtpva\"}},{\"activity\":\"qmtahnimkndujy\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Succeeded\"],\"\":{\"wafslytmttjduco\":\"datam\",\"ovtnfwpmpap\":\"dataxcdh\"}},{\"activity\":\"mpdsvki\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Succeeded\",\"Completed\"],\"\":{\"dg\":\"datakjfvudigwkyykh\"}}],\"userProperties\":[{\"name\":\"aafjxgo\",\"value\":\"datatiupj\"}],\"\":{\"ncqoy\":\"datansuik\",\"bxywojuxyfp\":\"datahrb\"}}") .toObject(AzureMLExecutePipelineActivity.class); - Assertions.assertEquals("gvhwkw", model.name()); - Assertions.assertEquals("jez", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("bkzzqwikqkxduh", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("rvhzfynbxwz", model.userProperties().get(0).name()); - Assertions.assertEquals("izkv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1557491500, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("nguvjryfcxscrs", model.name()); + Assertions.assertEquals("tno", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("m", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("aafjxgo", model.userProperties().get(0).name()); + Assertions.assertEquals("lhzme", model.linkedServiceName().referenceName()); + Assertions.assertEquals(2106899128, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMLExecutePipelineActivity model = new AzureMLExecutePipelineActivity().withName("gvhwkw") - .withDescription("jez").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("bkzzqwikqkxduh") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.FAILED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("cccaujga") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("sdi") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("qdostvx") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("rvhzfynbxwz").withValue("dataxmvl"), - new UserProperty().withName("uvbesra").withValue("datazxnwxsjygigepfok"), - new UserProperty().withName("lcnsxhpqey").withValue("datazydpvv"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("izkv") - .withParameters(mapOf("wembonureklgunp", "datahrphvmezdfadrto", "gwxctdpjuwu", "dataj", "txulnntjiucno", - "dataxxsmookhobzisqp", "flt", "dataiacvttdyv"))) - .withPolicy(new ActivityPolicy().withTimeout("databzfyllkunwinqyw").withRetry("dataxuxztjece") - .withRetryIntervalInSeconds(1557491500).withSecureInput(false).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withMlPipelineId("databuf").withMlPipelineEndpointId("datalkj").withVersion("dataudigwkyy") - .withExperimentName("dataedgapraa").withMlPipelineParameters("dataxgojtiupjgebns") - .withDataPathAssignments("dataklncqoyghrbabxy").withMlParentRunId("datajuxyfpcvblyeo") - .withContinueOnStepFailure("datathxkqczmfuhsupi"); + AzureMLExecutePipelineActivity model + = new AzureMLExecutePipelineActivity().withName("nguvjryfcxscrs").withDescription("tno") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("m").withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED)).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("qmtahnimkndujy") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("mpdsvki") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("aafjxgo").withValue("datatiupj"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("lhzme") + .withParameters(mapOf("jeqdmolmcyba", "datazhrzeibkuuolul", "qamvdnexqvt", "datakeuraylygclwbu", + "lkny", "datafnhzgtydllauno", "hhcqjahhcbzoary", "datapglgkeaz"))) + .withPolicy(new ActivityPolicy().withTimeout("datamftgmql").withRetry("dataoyxfrrdbdyhjf") + .withRetryIntervalInSeconds(2106899128).withSecureInput(false).withSecureOutput(true) + .withAdditionalProperties(mapOf())) + .withMlPipelineId("dataggyhpu").withMlPipelineEndpointId("dataclehndbutptyabd").withVersion("datakb") + .withExperimentName("datapemorfzuhvycd").withMlPipelineParameters("dataczmzsfvriskplndd") + .withDataPathAssignments("datacqinvkmkbtp").withMlParentRunId("datathzmqabvwbgsanvd") + .withContinueOnStepFailure("datambxshrae"); model = BinaryData.fromObject(model).toObject(AzureMLExecutePipelineActivity.class); - Assertions.assertEquals("gvhwkw", model.name()); - Assertions.assertEquals("jez", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("bkzzqwikqkxduh", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("rvhzfynbxwz", model.userProperties().get(0).name()); - Assertions.assertEquals("izkv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1557491500, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("nguvjryfcxscrs", model.name()); + Assertions.assertEquals("tno", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("m", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("aafjxgo", model.userProperties().get(0).name()); + Assertions.assertEquals("lhzme", model.linkedServiceName().referenceName()); + Assertions.assertEquals(2106899128, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTypePropertiesTests.java index 3aea0435ff3a5..42c473864efa6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLExecutePipelineActivityTypePropertiesTests.java @@ -11,17 +11,17 @@ public final class AzureMLExecutePipelineActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLExecutePipelineActivityTypeProperties model = BinaryData.fromString( - "{\"mlPipelineId\":\"datawybbo\",\"mlPipelineEndpointId\":\"datadxhkdy\",\"version\":\"datakufqzuduqfde\",\"experimentName\":\"dataxtplpg\",\"mlPipelineParameters\":\"datazugkfabvek\",\"dataPathAssignments\":\"datalibszcvceglvzh\",\"mlParentRunId\":\"datavv\",\"continueOnStepFailure\":\"dataadpclazaoytku\"}") + "{\"mlPipelineId\":\"datablyeo\",\"mlPipelineEndpointId\":\"datathxkqczmfuhsupi\",\"version\":\"dataizkv\",\"experimentName\":\"datakhrphvmezdfad\",\"mlPipelineParameters\":\"dataok\",\"dataPathAssignments\":\"datambonureklgunpajw\",\"mlParentRunId\":\"dataxctdpj\",\"continueOnStepFailure\":\"dataujxxsmook\"}") .toObject(AzureMLExecutePipelineActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMLExecutePipelineActivityTypeProperties model - = new AzureMLExecutePipelineActivityTypeProperties().withMlPipelineId("datawybbo") - .withMlPipelineEndpointId("datadxhkdy").withVersion("datakufqzuduqfde").withExperimentName("dataxtplpg") - .withMlPipelineParameters("datazugkfabvek").withDataPathAssignments("datalibszcvceglvzh") - .withMlParentRunId("datavv").withContinueOnStepFailure("dataadpclazaoytku"); + AzureMLExecutePipelineActivityTypeProperties model = new AzureMLExecutePipelineActivityTypeProperties() + .withMlPipelineId("datablyeo").withMlPipelineEndpointId("datathxkqczmfuhsupi").withVersion("dataizkv") + .withExperimentName("datakhrphvmezdfad").withMlPipelineParameters("dataok") + .withDataPathAssignments("datambonureklgunpajw").withMlParentRunId("dataxctdpj") + .withContinueOnStepFailure("dataujxxsmook"); model = BinaryData.fromObject(model).toObject(AzureMLExecutePipelineActivityTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTests.java index 6806845133597..b3683764964b2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTests.java @@ -22,64 +22,69 @@ public final class AzureMLUpdateResourceActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLUpdateResourceActivity model = BinaryData.fromString( - "{\"type\":\"AzureMLUpdateResource\",\"typeProperties\":{\"trainedModelName\":\"datagwgqh\",\"trainedModelLinkedServiceName\":{\"referenceName\":\"feasmkdguod\",\"parameters\":{\"rtajlyd\":\"datapwqbotlvcpcxxp\",\"uxbqdwbjh\":\"datafqfvrqruympo\"}},\"trainedModelFilePath\":\"datajzvceyxvfoyuyk\"},\"linkedServiceName\":{\"referenceName\":\"ggyhpu\",\"parameters\":{\"tpt\":\"datalehndb\",\"morfzuhvycdndcz\":\"dataabdkbkblop\",\"kplnd\":\"datazsfvri\",\"btpbwthz\":\"datapqcqinvkm\"}},\"policy\":{\"timeout\":\"databvwb\",\"retry\":\"dataanvdrrmbxshra\",\"retryIntervalInSeconds\":960450511,\"secureInput\":true,\"secureOutput\":false,\"\":{\"zeib\":\"dataqtzh\",\"mo\":\"datauuoluldjeq\",\"aqkeuraylygclwb\":\"datamcy\"}},\"name\":\"oqamvdnexqvthf\",\"description\":\"zgt\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"oelknyopglgkeaz\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Succeeded\",\"Completed\"],\"\":{\"cxmftgmqlco\":\"datahcbzoary\",\"bdyhjfmyc\":\"datayxfrr\"}},{\"activity\":\"ucccb\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Skipped\",\"Succeeded\"],\"\":{\"bmjcch\":\"dataurl\",\"ozlhl\":\"dataigja\"}}],\"userProperties\":[{\"name\":\"egxnguvjryfcx\",\"value\":\"datacr\"},{\"name\":\"wy\",\"value\":\"datanoiriemkxm\"},{\"name\":\"q\",\"value\":\"dataimnfgfsjptb\"},{\"name\":\"svweu\",\"value\":\"datatoe\"}],\"\":{\"wtpvaiewf\":\"datahmrawmchcde\",\"thpg\":\"datawfkw\"}}") + "{\"type\":\"AzureMLUpdateResource\",\"typeProperties\":{\"trainedModelName\":\"datapkoezcab\",\"trainedModelLinkedServiceName\":{\"referenceName\":\"ylsuiyvbildwqlx\",\"parameters\":{\"pylpmtwdvdtzdr\":\"dataqei\",\"urwzrx\":\"dataaxswiind\",\"mbtvcdsl\":\"datahacvsj\"}},\"trainedModelFilePath\":\"databv\"},\"linkedServiceName\":{\"referenceName\":\"nxhszrotunnkb\",\"parameters\":{\"lqyzrtawjkjzvvk\":\"datakaoonbzi\",\"seqxwcimam\":\"datahasxjmfh\",\"nrj\":\"dataqfrdfoiqfvczuu\",\"roxvsclmt\":\"datarpxlfyytjm\"}},\"policy\":{\"timeout\":\"datalfcgkdeitphzuazn\",\"retry\":\"datavubbestyymlj\",\"retryIntervalInSeconds\":235961975,\"secureInput\":false,\"secureOutput\":true,\"\":{\"fudranmd\":\"datanyxrizse\"}},\"name\":\"fwawzjhfauu\",\"description\":\"vnaf\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"mviclhommhaxtegr\",\"dependencyConditions\":[\"Completed\"],\"\":{\"jfzbavqmmk\":\"datan\"}},{\"activity\":\"qdfjeuwwq\",\"dependencyConditions\":[\"Failed\",\"Skipped\"],\"\":{\"qvywolccxdctkhe\":\"datakzplbzyj\",\"gemspn\":\"databosa\",\"hhexgxnmfodxiyz\":\"dataqo\"}},{\"activity\":\"fottycfostzd\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Completed\"],\"\":{\"xurrhpihtxgjzi\":\"databhahxs\"}},{\"activity\":\"yhujgrbjmzagxjoi\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\",\"Skipped\"],\"\":{\"d\":\"datacrf\",\"jzquwjgfihlo\":\"datatcf\",\"efxvggkjbhs\":\"dataauorzbk\",\"yajijzrt\":\"datayy\"}}],\"userProperties\":[{\"name\":\"gonhmblkkel\",\"value\":\"datajk\"},{\"name\":\"emneu\",\"value\":\"datapynenca\"},{\"name\":\"kqvcf\",\"value\":\"datargwxgczwxyghs\"}],\"\":{\"hygbe\":\"datavxcrzpdqwa\",\"lmfh\":\"datafiwbtfki\"}}") .toObject(AzureMLUpdateResourceActivity.class); - Assertions.assertEquals("oqamvdnexqvthf", model.name()); - Assertions.assertEquals("zgt", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals("fwawzjhfauu", model.name()); + Assertions.assertEquals("vnaf", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("oelknyopglgkeaz", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("egxnguvjryfcx", model.userProperties().get(0).name()); - Assertions.assertEquals("ggyhpu", model.linkedServiceName().referenceName()); - Assertions.assertEquals(960450511, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("feasmkdguod", model.trainedModelLinkedServiceName().referenceName()); + Assertions.assertEquals("mviclhommhaxtegr", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("gonhmblkkel", model.userProperties().get(0).name()); + Assertions.assertEquals("nxhszrotunnkb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(235961975, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals("ylsuiyvbildwqlx", model.trainedModelLinkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMLUpdateResourceActivity model = new AzureMLUpdateResourceActivity().withName("oqamvdnexqvthf") - .withDescription("zgt").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency() - .withActivity("oelknyopglgkeaz") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ucccb") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("egxnguvjryfcx").withValue("datacr"), - new UserProperty().withName("wy").withValue("datanoiriemkxm"), - new UserProperty().withName("q").withValue("dataimnfgfsjptb"), - new UserProperty().withName("svweu").withValue("datatoe"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ggyhpu") - .withParameters(mapOf("tpt", "datalehndb", "morfzuhvycdndcz", "dataabdkbkblop", "kplnd", "datazsfvri", - "btpbwthz", "datapqcqinvkm"))) - .withPolicy(new ActivityPolicy() - .withTimeout("databvwb").withRetry("dataanvdrrmbxshra").withRetryIntervalInSeconds(960450511) - .withSecureInput(true).withSecureOutput(false).withAdditionalProperties(mapOf())) - .withTrainedModelName("datagwgqh") - .withTrainedModelLinkedServiceName(new LinkedServiceReference().withReferenceName("feasmkdguod") - .withParameters(mapOf("rtajlyd", "datapwqbotlvcpcxxp", "uxbqdwbjh", "datafqfvrqruympo"))) - .withTrainedModelFilePath("datajzvceyxvfoyuyk"); + AzureMLUpdateResourceActivity model + = new AzureMLUpdateResourceActivity().withName("fwawzjhfauu").withDescription("vnaf") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("mviclhommhaxtegr").withDependencyConditions( + Arrays.asList(DependencyCondition.COMPLETED)).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("qdfjeuwwq") + .withDependencyConditions( + Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("fottycfostzd") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("yhujgrbjmzagxjoi") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("gonhmblkkel").withValue("datajk"), + new UserProperty().withName("emneu").withValue("datapynenca"), + new UserProperty().withName("kqvcf").withValue("datargwxgczwxyghs"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("nxhszrotunnkb") + .withParameters(mapOf("lqyzrtawjkjzvvk", "datakaoonbzi", "seqxwcimam", "datahasxjmfh", "nrj", + "dataqfrdfoiqfvczuu", "roxvsclmt", "datarpxlfyytjm"))) + .withPolicy(new ActivityPolicy().withTimeout("datalfcgkdeitphzuazn").withRetry("datavubbestyymlj") + .withRetryIntervalInSeconds(235961975).withSecureInput(false).withSecureOutput(true) + .withAdditionalProperties(mapOf())) + .withTrainedModelName("datapkoezcab") + .withTrainedModelLinkedServiceName( + new LinkedServiceReference().withReferenceName("ylsuiyvbildwqlx").withParameters( + mapOf("pylpmtwdvdtzdr", "dataqei", "urwzrx", "dataaxswiind", "mbtvcdsl", "datahacvsj"))) + .withTrainedModelFilePath("databv"); model = BinaryData.fromObject(model).toObject(AzureMLUpdateResourceActivity.class); - Assertions.assertEquals("oqamvdnexqvthf", model.name()); - Assertions.assertEquals("zgt", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals("fwawzjhfauu", model.name()); + Assertions.assertEquals("vnaf", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("oelknyopglgkeaz", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("egxnguvjryfcx", model.userProperties().get(0).name()); - Assertions.assertEquals("ggyhpu", model.linkedServiceName().referenceName()); - Assertions.assertEquals(960450511, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("feasmkdguod", model.trainedModelLinkedServiceName().referenceName()); + Assertions.assertEquals("mviclhommhaxtegr", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("gonhmblkkel", model.userProperties().get(0).name()); + Assertions.assertEquals("nxhszrotunnkb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(235961975, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals("ylsuiyvbildwqlx", model.trainedModelLinkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTypePropertiesTests.java index 6823507f90bde..8c0fc9771c2d2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLUpdateResourceActivityTypePropertiesTests.java @@ -15,20 +15,21 @@ public final class AzureMLUpdateResourceActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLUpdateResourceActivityTypeProperties model = BinaryData.fromString( - "{\"trainedModelName\":\"datamtahnimkndujyw\",\"trainedModelLinkedServiceName\":{\"referenceName\":\"xf\",\"parameters\":{\"wafslytmttjduco\":\"datam\",\"ovtnfwpmpap\":\"dataxcdh\"}},\"trainedModelFilePath\":\"datampdsvki\"}") + "{\"trainedModelName\":\"datazsxjrafhdf\",\"trainedModelLinkedServiceName\":{\"referenceName\":\"ukaaw\",\"parameters\":{\"qhefeasmkdguodo\":\"dataijphhuvflgw\",\"vcp\":\"datajpwqbot\",\"qruympov\":\"dataxxpyrtajlydefqfv\",\"fo\":\"dataxbqdwbjhgjzvceyx\"}},\"trainedModelFilePath\":\"datauyk\"}") .toObject(AzureMLUpdateResourceActivityTypeProperties.class); - Assertions.assertEquals("xf", model.trainedModelLinkedServiceName().referenceName()); + Assertions.assertEquals("ukaaw", model.trainedModelLinkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureMLUpdateResourceActivityTypeProperties model - = new AzureMLUpdateResourceActivityTypeProperties().withTrainedModelName("datamtahnimkndujyw") - .withTrainedModelLinkedServiceName(new LinkedServiceReference().withReferenceName("xf") - .withParameters(mapOf("wafslytmttjduco", "datam", "ovtnfwpmpap", "dataxcdh"))) - .withTrainedModelFilePath("datampdsvki"); + = new AzureMLUpdateResourceActivityTypeProperties().withTrainedModelName("datazsxjrafhdf") + .withTrainedModelLinkedServiceName(new LinkedServiceReference().withReferenceName("ukaaw") + .withParameters(mapOf("qhefeasmkdguodo", "dataijphhuvflgw", "vcp", "datajpwqbot", "qruympov", + "dataxxpyrtajlydefqfv", "fo", "dataxbqdwbjhgjzvceyx"))) + .withTrainedModelFilePath("datauyk"); model = BinaryData.fromObject(model).toObject(AzureMLUpdateResourceActivityTypeProperties.class); - Assertions.assertEquals("xf", model.trainedModelLinkedServiceName().referenceName()); + Assertions.assertEquals("ukaaw", model.trainedModelLinkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLWebServiceFileTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLWebServiceFileTests.java index 3904f928a1bfe..80221a82497b2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLWebServiceFileTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMLWebServiceFileTests.java @@ -15,18 +15,18 @@ public final class AzureMLWebServiceFileTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMLWebServiceFile model = BinaryData.fromString( - "{\"filePath\":\"datafiwbtfki\",\"linkedServiceName\":{\"referenceName\":\"lmfh\",\"parameters\":{\"ukaaw\":\"dataxjrafhdf\",\"vf\":\"datanqijphh\"}}}") + "{\"filePath\":\"dataennobjixoqqjbsag\",\"linkedServiceName\":{\"referenceName\":\"lpuqfmrimwlpa\",\"parameters\":{\"s\":\"dataxuiaktnmwlklqhw\",\"r\":\"dataeoefwnjsorhpga\"}}}") .toObject(AzureMLWebServiceFile.class); - Assertions.assertEquals("lmfh", model.linkedServiceName().referenceName()); + Assertions.assertEquals("lpuqfmrimwlpa", model.linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMLWebServiceFile model = new AzureMLWebServiceFile().withFilePath("datafiwbtfki") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("lmfh") - .withParameters(mapOf("ukaaw", "dataxjrafhdf", "vf", "datanqijphh"))); + AzureMLWebServiceFile model = new AzureMLWebServiceFile().withFilePath("dataennobjixoqqjbsag") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("lpuqfmrimwlpa") + .withParameters(mapOf("s", "dataxuiaktnmwlklqhw", "r", "dataeoefwnjsorhpga"))); model = BinaryData.fromObject(model).toObject(AzureMLWebServiceFile.class); - Assertions.assertEquals("lmfh", model.linkedServiceName().referenceName()); + Assertions.assertEquals("lpuqfmrimwlpa", model.linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBSourceTests.java index 0aa3f97906e88..5ef3c116dbb02 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBSourceTests.java @@ -11,16 +11,16 @@ public final class AzureMariaDBSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMariaDBSource model = BinaryData.fromString( - "{\"type\":\"AzureMariaDBSource\",\"query\":\"datahsvsnedhkji\",\"queryTimeout\":\"datavetwf\",\"additionalColumns\":\"dataqvflrrtj\",\"sourceRetryCount\":\"dataikqzd\",\"sourceRetryWait\":\"dataqalxpmiytpjis\",\"maxConcurrentConnections\":\"datasolkwipvlsljut\",\"disableMetricsCollection\":\"datag\",\"\":{\"eaeyjlyxd\":\"dataodrfclehlopipv\",\"yavcbmzem\":\"dataxho\"}}") + "{\"type\":\"AzureMariaDBSource\",\"query\":\"dataht\",\"queryTimeout\":\"datauiptudw\",\"additionalColumns\":\"datasrpsjkqfabju\",\"sourceRetryCount\":\"datats\",\"sourceRetryWait\":\"dataupcio\",\"maxConcurrentConnections\":\"datarjdeyfnqanbadkzp\",\"disableMetricsCollection\":\"datatuplpkjexq\",\"\":{\"goeftrbxomaa\":\"datazlal\",\"gvjmllzykalbaumm\":\"datavarfqverxelquqze\",\"r\":\"datadwqiucpj\",\"ftt\":\"databssjtjwzelx\"}}") .toObject(AzureMariaDBSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureMariaDBSource model - = new AzureMariaDBSource().withSourceRetryCount("dataikqzd").withSourceRetryWait("dataqalxpmiytpjis") - .withMaxConcurrentConnections("datasolkwipvlsljut").withDisableMetricsCollection("datag") - .withQueryTimeout("datavetwf").withAdditionalColumns("dataqvflrrtj").withQuery("datahsvsnedhkji"); + = new AzureMariaDBSource().withSourceRetryCount("datats").withSourceRetryWait("dataupcio") + .withMaxConcurrentConnections("datarjdeyfnqanbadkzp").withDisableMetricsCollection("datatuplpkjexq") + .withQueryTimeout("datauiptudw").withAdditionalColumns("datasrpsjkqfabju").withQuery("dataht"); model = BinaryData.fromObject(model).toObject(AzureMariaDBSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBTableDatasetTests.java index 783807d069210..6753c810e73b9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMariaDBTableDatasetTests.java @@ -19,32 +19,32 @@ public final class AzureMariaDBTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMariaDBTableDataset model = BinaryData.fromString( - "{\"type\":\"AzureMariaDBTable\",\"typeProperties\":{\"tableName\":\"datatn\"},\"description\":\"jewihcigaahm\",\"structure\":\"dataspkdnx\",\"schema\":\"dataz\",\"linkedServiceName\":{\"referenceName\":\"tertnzrrwsc\",\"parameters\":{\"nvtolzj\":\"datahdwi\",\"haknklthqwppv\":\"datafkryxs\"}},\"parameters\":{\"bkabhvxjuaivx\":{\"type\":\"Int\",\"defaultValue\":\"datarvpvdrohul\"},\"kg\":{\"type\":\"Array\",\"defaultValue\":\"datarnygti\"},\"rxzpqditu\":{\"type\":\"Bool\",\"defaultValue\":\"datamkphvdl\"},\"e\":{\"type\":\"String\",\"defaultValue\":\"datatfcieil\"}},\"annotations\":[\"datakehldopjsxvbbwsg\",\"datakkmibnmdp\",\"datad\",\"datapwtgzwmzhcmrloqa\"],\"folder\":{\"name\":\"yzavky\"},\"\":{\"bngzldvvd\":\"dataudnmbj\",\"pmq\":\"dataoptythctoxo\",\"sfzsgzgus\":\"dataerwhemvids\"}}") + "{\"type\":\"AzureMariaDBTable\",\"typeProperties\":{\"tableName\":\"datahvxjuai\"},\"description\":\"znirnygtix\",\"structure\":\"datayob\",\"schema\":\"dataphvd\",\"linkedServiceName\":{\"referenceName\":\"orxzpqdi\",\"parameters\":{\"tkehldopjsxvbb\":\"datacltfcieileem\"}},\"parameters\":{\"bn\":{\"type\":\"Array\",\"defaultValue\":\"datakm\"},\"zwmzhcmrloq\":{\"type\":\"Bool\",\"defaultValue\":\"dataidipwt\"},\"dnmbjqbngzldv\":{\"type\":\"Bool\",\"defaultValue\":\"datatyzavkyjjl\"}},\"annotations\":[\"dataoptythctoxo\"],\"folder\":{\"name\":\"qnerw\"},\"\":{\"pejomeqgxhwisp\":\"datavidsssfzsgzgu\",\"xirppbiichlygkv\":\"dataogdblwjsbaqxaxt\",\"wonkrnizdxywabki\":\"datai\",\"aptgvnaqyjukka\":\"datani\"}}") .toObject(AzureMariaDBTableDataset.class); - Assertions.assertEquals("jewihcigaahm", model.description()); - Assertions.assertEquals("tertnzrrwsc", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("bkabhvxjuaivx").type()); - Assertions.assertEquals("yzavky", model.folder().name()); + Assertions.assertEquals("znirnygtix", model.description()); + Assertions.assertEquals("orxzpqdi", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("bn").type()); + Assertions.assertEquals("qnerw", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMariaDBTableDataset model = new AzureMariaDBTableDataset().withDescription("jewihcigaahm") - .withStructure("dataspkdnx").withSchema("dataz") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("tertnzrrwsc") - .withParameters(mapOf("nvtolzj", "datahdwi", "haknklthqwppv", "datafkryxs"))) - .withParameters(mapOf("bkabhvxjuaivx", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datarvpvdrohul"), "kg", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datarnygti"), "rxzpqditu", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datamkphvdl"), "e", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datatfcieil"))) - .withAnnotations(Arrays.asList("datakehldopjsxvbbwsg", "datakkmibnmdp", "datad", "datapwtgzwmzhcmrloqa")) - .withFolder(new DatasetFolder().withName("yzavky")).withTableName("datatn"); + AzureMariaDBTableDataset model = new AzureMariaDBTableDataset().withDescription("znirnygtix") + .withStructure("datayob").withSchema("dataphvd") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("orxzpqdi") + .withParameters(mapOf("tkehldopjsxvbb", "datacltfcieileem"))) + .withParameters(mapOf("bn", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datakm"), "zwmzhcmrloq", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataidipwt"), + "dnmbjqbngzldv", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datatyzavkyjjl"))) + .withAnnotations(Arrays.asList("dataoptythctoxo")).withFolder(new DatasetFolder().withName("qnerw")) + .withTableName("datahvxjuai"); model = BinaryData.fromObject(model).toObject(AzureMariaDBTableDataset.class); - Assertions.assertEquals("jewihcigaahm", model.description()); - Assertions.assertEquals("tertnzrrwsc", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("bkabhvxjuaivx").type()); - Assertions.assertEquals("yzavky", model.folder().name()); + Assertions.assertEquals("znirnygtix", model.description()); + Assertions.assertEquals("orxzpqdi", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("bn").type()); + Assertions.assertEquals("qnerw", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSinkTests.java index e89ccdfb5e387..0746eb19e1b72 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSinkTests.java @@ -11,16 +11,16 @@ public final class AzureMySqlSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMySqlSink model = BinaryData.fromString( - "{\"type\":\"AzureMySqlSink\",\"preCopyScript\":\"datanwhqafzgzmonj\",\"writeBatchSize\":\"dataienctwbimhfmog\",\"writeBatchTimeout\":\"datawxrdllrqamfjyyrf\",\"sinkRetryCount\":\"databyxygubvidpsk\",\"sinkRetryWait\":\"datassxh\",\"maxConcurrentConnections\":\"datagliufdctgsd\",\"disableMetricsCollection\":\"dataxkddxoatlprs\",\"\":{\"tfplgx\":\"datannnyyvvlgsadpvmn\"}}") + "{\"type\":\"AzureMySqlSink\",\"preCopyScript\":\"datasmgh\",\"writeBatchSize\":\"datatuujcuavctxyrmws\",\"writeBatchTimeout\":\"datazmy\",\"sinkRetryCount\":\"datan\",\"sinkRetryWait\":\"dataajxv\",\"maxConcurrentConnections\":\"dataidlwmewrgu\",\"disableMetricsCollection\":\"dataugpkunvygupgnnvm\",\"\":{\"ekmsn\":\"dataqmxww\",\"jypxcqmdeecdh\":\"datafjbefszfrxfy\",\"mykgrtwh\":\"datajsizyhp\"}}") .toObject(AzureMySqlSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMySqlSink model = new AzureMySqlSink().withWriteBatchSize("dataienctwbimhfmog") - .withWriteBatchTimeout("datawxrdllrqamfjyyrf").withSinkRetryCount("databyxygubvidpsk") - .withSinkRetryWait("datassxh").withMaxConcurrentConnections("datagliufdctgsd") - .withDisableMetricsCollection("dataxkddxoatlprs").withPreCopyScript("datanwhqafzgzmonj"); + AzureMySqlSink model = new AzureMySqlSink().withWriteBatchSize("datatuujcuavctxyrmws") + .withWriteBatchTimeout("datazmy").withSinkRetryCount("datan").withSinkRetryWait("dataajxv") + .withMaxConcurrentConnections("dataidlwmewrgu").withDisableMetricsCollection("dataugpkunvygupgnnvm") + .withPreCopyScript("datasmgh"); model = BinaryData.fromObject(model).toObject(AzureMySqlSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSourceTests.java index c26d27b65fcfb..a783039895e2a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureMySqlSourceTests.java @@ -11,15 +11,15 @@ public final class AzureMySqlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureMySqlSource model = BinaryData.fromString( - "{\"type\":\"AzureMySqlSource\",\"query\":\"datag\",\"queryTimeout\":\"datadmrowhrrguvd\",\"additionalColumns\":\"datagucwawlmsikl\",\"sourceRetryCount\":\"datam\",\"sourceRetryWait\":\"datacphgimi\",\"maxConcurrentConnections\":\"datahrbs\",\"disableMetricsCollection\":\"datavublouel\",\"\":{\"eblpdwckmnpzubzq\":\"datafb\",\"gcpbshq\":\"dataswgfjrg\"}}") + "{\"type\":\"AzureMySqlSource\",\"query\":\"datarzhjqengopdvnz\",\"queryTimeout\":\"dataliodajxvs\",\"additionalColumns\":\"datayvzm\",\"sourceRetryCount\":\"dataf\",\"sourceRetryWait\":\"datareawhnz\",\"maxConcurrentConnections\":\"datamue\",\"disableMetricsCollection\":\"databhnkles\",\"\":{\"cyrdtrd\":\"datavakqajian\",\"h\":\"datakdmsktuv\",\"toyfbsgrzw\":\"dataxtv\",\"ud\":\"dataw\"}}") .toObject(AzureMySqlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureMySqlSource model = new AzureMySqlSource().withSourceRetryCount("datam").withSourceRetryWait("datacphgimi") - .withMaxConcurrentConnections("datahrbs").withDisableMetricsCollection("datavublouel") - .withQueryTimeout("datadmrowhrrguvd").withAdditionalColumns("datagucwawlmsikl").withQuery("datag"); + AzureMySqlSource model = new AzureMySqlSource().withSourceRetryCount("dataf").withSourceRetryWait("datareawhnz") + .withMaxConcurrentConnections("datamue").withDisableMetricsCollection("databhnkles") + .withQueryTimeout("dataliodajxvs").withAdditionalColumns("datayvzm").withQuery("datarzhjqengopdvnz"); model = BinaryData.fromObject(model).toObject(AzureMySqlSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSinkTests.java index 57476f4d0e687..b47407f83306a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSinkTests.java @@ -11,16 +11,16 @@ public final class AzurePostgreSqlSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzurePostgreSqlSink model = BinaryData.fromString( - "{\"type\":\"AzurePostgreSqlSink\",\"preCopyScript\":\"datazlbpehv\",\"writeBatchSize\":\"datagllrhnlxstpgz\",\"writeBatchTimeout\":\"dataezm\",\"sinkRetryCount\":\"dataqpdchdsxvkm\",\"sinkRetryWait\":\"datapxzgj\",\"maxConcurrentConnections\":\"datamtskto\",\"disableMetricsCollection\":\"datay\",\"\":{\"ugrfizfwdm\":\"datadpjsoko\"}}") + "{\"type\":\"AzurePostgreSqlSink\",\"preCopyScript\":\"datamhookefdgfexakct\",\"writeBatchSize\":\"datapszdn\",\"writeBatchTimeout\":\"datao\",\"sinkRetryCount\":\"dataqxmdievkmrso\",\"sinkRetryWait\":\"datayiheheimuqqmd\",\"maxConcurrentConnections\":\"datawxfmrm\",\"disableMetricsCollection\":\"dataf\",\"\":{\"xdldhhkdeviwp\":\"dataypmthfvszlaf\",\"nu\":\"datahfxvl\"}}") .toObject(AzurePostgreSqlSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzurePostgreSqlSink model = new AzurePostgreSqlSink().withWriteBatchSize("datagllrhnlxstpgz") - .withWriteBatchTimeout("dataezm").withSinkRetryCount("dataqpdchdsxvkm").withSinkRetryWait("datapxzgj") - .withMaxConcurrentConnections("datamtskto").withDisableMetricsCollection("datay") - .withPreCopyScript("datazlbpehv"); + AzurePostgreSqlSink model = new AzurePostgreSqlSink().withWriteBatchSize("datapszdn") + .withWriteBatchTimeout("datao").withSinkRetryCount("dataqxmdievkmrso") + .withSinkRetryWait("datayiheheimuqqmd").withMaxConcurrentConnections("datawxfmrm") + .withDisableMetricsCollection("dataf").withPreCopyScript("datamhookefdgfexakct"); model = BinaryData.fromObject(model).toObject(AzurePostgreSqlSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSourceTests.java index 8cfa660a513a8..a102a2399e19f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlSourceTests.java @@ -11,16 +11,16 @@ public final class AzurePostgreSqlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzurePostgreSqlSource model = BinaryData.fromString( - "{\"type\":\"AzurePostgreSqlSource\",\"query\":\"datadaxbesbwcizyjc\",\"queryTimeout\":\"datauasjrs\",\"additionalColumns\":\"datapomihgksqwzuosy\",\"sourceRetryCount\":\"datalr\",\"sourceRetryWait\":\"datazudfar\",\"maxConcurrentConnections\":\"datayrdy\",\"disableMetricsCollection\":\"datawgikpdpudqiwhvx\",\"\":{\"zjedycjisxspn\":\"datapoeuufwsmdeffrb\",\"lnomqbdvjl\":\"datafydphl\"}}") + "{\"type\":\"AzurePostgreSqlSource\",\"query\":\"datahbfttptsdee\",\"queryTimeout\":\"dataovanag\",\"additionalColumns\":\"dataacsfbmb\",\"sourceRetryCount\":\"dataefqku\",\"sourceRetryWait\":\"datayumoamqxwluslxyt\",\"maxConcurrentConnections\":\"databjledjxblobknfpd\",\"disableMetricsCollection\":\"datahzgj\",\"\":{\"ccypxsrhbqlbnufl\":\"datamctbg\",\"xhbpyoqfbj\":\"datazawkkz\",\"jpjnhwwyhx\":\"dataclboi\",\"hhw\":\"dataythxzrvjfsmfk\"}}") .toObject(AzurePostgreSqlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzurePostgreSqlSource model = new AzurePostgreSqlSource().withSourceRetryCount("datalr") - .withSourceRetryWait("datazudfar").withMaxConcurrentConnections("datayrdy") - .withDisableMetricsCollection("datawgikpdpudqiwhvx").withQueryTimeout("datauasjrs") - .withAdditionalColumns("datapomihgksqwzuosy").withQuery("datadaxbesbwcizyjc"); + AzurePostgreSqlSource model + = new AzurePostgreSqlSource().withSourceRetryCount("dataefqku").withSourceRetryWait("datayumoamqxwluslxyt") + .withMaxConcurrentConnections("databjledjxblobknfpd").withDisableMetricsCollection("datahzgj") + .withQueryTimeout("dataovanag").withAdditionalColumns("dataacsfbmb").withQuery("datahbfttptsdee"); model = BinaryData.fromObject(model).toObject(AzurePostgreSqlSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTests.java index 54eb95f05f4d3..25d682c6c0274 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTests.java @@ -19,37 +19,37 @@ public final class AzurePostgreSqlTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzurePostgreSqlTableDataset model = BinaryData.fromString( - "{\"type\":\"AzurePostgreSqlTable\",\"typeProperties\":{\"tableName\":\"dataj\",\"table\":\"datawrduxntpfxxgja\",\"schema\":\"dataxfwf\"},\"description\":\"qv\",\"structure\":\"datafbkqynlzxem\",\"schema\":\"dataupjckiehdm\",\"linkedServiceName\":{\"referenceName\":\"foyrxxxffgmcua\",\"parameters\":{\"csapvbcqpfus\":\"dataeervgc\",\"k\":\"datakijhmine\",\"rkvorlfqmljewyn\":\"dataivp\",\"vlnpbsotmynklnm\":\"datafvvcwvurkmjufa\"}},\"parameters\":{\"keipxutc\":{\"type\":\"Int\",\"defaultValue\":\"datavrkkfcwxizkstxne\"},\"tvsayyaeiiv\":{\"type\":\"Array\",\"defaultValue\":\"dataiuvnfaz\"},\"xqetxtdqius\":{\"type\":\"Float\",\"defaultValue\":\"dataqtjwrvewojoq\"},\"mjsisfqqhc\":{\"type\":\"Array\",\"defaultValue\":\"datazljvgjijzqjhljsa\"}},\"annotations\":[\"dataagsbfeiir\",\"datanjygllfkchhgsj\"],\"folder\":{\"name\":\"c\"},\"\":{\"khdhpmkxdujkxpuq\":\"datawmqcycabaam\",\"ezxiz\":\"datadyoqywsuarpzhry\",\"azccouhwivkd\":\"datasyxbfjilb\",\"pi\":\"datavjsknrbxz\"}}") + "{\"type\":\"AzurePostgreSqlTable\",\"typeProperties\":{\"tableName\":\"datarllld\",\"table\":\"datanbdzwils\",\"schema\":\"datayiqjz\"},\"description\":\"rd\",\"structure\":\"datantpfxxgjahyxfw\",\"schema\":\"datat\",\"linkedServiceName\":{\"referenceName\":\"veyfbkqynlzxeme\",\"parameters\":{\"ehdmvfoyrxxxff\":\"datajck\",\"cgcsapvbcqpf\":\"datamcuanszeerv\",\"vpyr\":\"datasekijhminenkb\",\"fvvcwvurkmjufa\":\"datavorlfqmljewyn\"}},\"parameters\":{\"vrkkfcwxizkstxne\":{\"type\":\"SecureString\",\"defaultValue\":\"databsotmynklnmrznm\"},\"tc\":{\"type\":\"Object\",\"defaultValue\":\"dataipx\"},\"tvsayyaeiiv\":{\"type\":\"Array\",\"defaultValue\":\"dataiuvnfaz\"},\"xqetxtdqius\":{\"type\":\"Float\",\"defaultValue\":\"dataqtjwrvewojoq\"}},\"annotations\":[\"datazljvgjijzqjhljsa\",\"datamjsisfqqhc\",\"dataecagsbfeiirpn\"],\"folder\":{\"name\":\"llfkchhgs\"},\"\":{\"cabaam\":\"datazcajlwmqc\",\"dyoqywsuarpzhry\":\"datakhdhpmkxdujkxpuq\"}}") .toObject(AzurePostgreSqlTableDataset.class); - Assertions.assertEquals("qv", model.description()); - Assertions.assertEquals("foyrxxxffgmcua", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("keipxutc").type()); - Assertions.assertEquals("c", model.folder().name()); + Assertions.assertEquals("rd", model.description()); + Assertions.assertEquals("veyfbkqynlzxeme", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("vrkkfcwxizkstxne").type()); + Assertions.assertEquals("llfkchhgs", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzurePostgreSqlTableDataset model = new AzurePostgreSqlTableDataset().withDescription("qv") - .withStructure("datafbkqynlzxem").withSchema("dataupjckiehdm") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("foyrxxxffgmcua") - .withParameters(mapOf("csapvbcqpfus", "dataeervgc", "k", "datakijhmine", "rkvorlfqmljewyn", "dataivp", - "vlnpbsotmynklnm", "datafvvcwvurkmjufa"))) - .withParameters(mapOf("keipxutc", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datavrkkfcwxizkstxne"), + AzurePostgreSqlTableDataset model = new AzurePostgreSqlTableDataset().withDescription("rd") + .withStructure("datantpfxxgjahyxfw").withSchema("datat") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("veyfbkqynlzxeme") + .withParameters(mapOf("ehdmvfoyrxxxff", "datajck", "cgcsapvbcqpf", "datamcuanszeerv", "vpyr", + "datasekijhminenkb", "fvvcwvurkmjufa", "datavorlfqmljewyn"))) + .withParameters(mapOf("vrkkfcwxizkstxne", + new ParameterSpecification().withType(ParameterType.SECURE_STRING) + .withDefaultValue("databsotmynklnmrznm"), + "tc", new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataipx"), "tvsayyaeiiv", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataiuvnfaz"), "xqetxtdqius", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataqtjwrvewojoq"), - "mjsisfqqhc", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datazljvgjijzqjhljsa"))) - .withAnnotations(Arrays.asList("dataagsbfeiir", "datanjygllfkchhgsj")) - .withFolder(new DatasetFolder().withName("c")).withTableName("dataj").withTable("datawrduxntpfxxgja") - .withSchemaTypePropertiesSchema("dataxfwf"); + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataqtjwrvewojoq"))) + .withAnnotations(Arrays.asList("datazljvgjijzqjhljsa", "datamjsisfqqhc", "dataecagsbfeiirpn")) + .withFolder(new DatasetFolder().withName("llfkchhgs")).withTableName("datarllld").withTable("datanbdzwils") + .withSchemaTypePropertiesSchema("datayiqjz"); model = BinaryData.fromObject(model).toObject(AzurePostgreSqlTableDataset.class); - Assertions.assertEquals("qv", model.description()); - Assertions.assertEquals("foyrxxxffgmcua", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("keipxutc").type()); - Assertions.assertEquals("c", model.folder().name()); + Assertions.assertEquals("rd", model.description()); + Assertions.assertEquals("veyfbkqynlzxeme", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("vrkkfcwxizkstxne").type()); + Assertions.assertEquals("llfkchhgs", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTypePropertiesTests.java index cfad993e56d59..9ae4d5e06a53b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzurePostgreSqlTableDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class AzurePostgreSqlTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzurePostgreSqlTableDatasetTypeProperties model = BinaryData - .fromString("{\"tableName\":\"datavcpi\",\"table\":\"dataqbvxqtolpwbopv\",\"schema\":\"databtzaprjxco\"}") + .fromString("{\"tableName\":\"datazx\",\"table\":\"datads\",\"schema\":\"databfjilbuazccouhw\"}") .toObject(AzurePostgreSqlTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzurePostgreSqlTableDatasetTypeProperties model = new AzurePostgreSqlTableDatasetTypeProperties() - .withTableName("datavcpi").withTable("dataqbvxqtolpwbopv").withSchema("databtzaprjxco"); + .withTableName("datazx").withTable("datads").withSchema("databfjilbuazccouhw"); model = BinaryData.fromObject(model).toObject(AzurePostgreSqlTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureQueueSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureQueueSinkTests.java index d05efc9751482..315be195f4c94 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureQueueSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureQueueSinkTests.java @@ -11,15 +11,16 @@ public final class AzureQueueSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureQueueSink model = BinaryData.fromString( - "{\"type\":\"AzureQueueSink\",\"writeBatchSize\":\"datanazjvyiiezdnez\",\"writeBatchTimeout\":\"dataqzd\",\"sinkRetryCount\":\"datamyutzttroymi\",\"sinkRetryWait\":\"datakuz\",\"maxConcurrentConnections\":\"datacegyztzhcfuwm\",\"disableMetricsCollection\":\"dataz\",\"\":{\"r\":\"datak\",\"okjwsszyetwoukdh\":\"dataogflhhospxbl\",\"rdg\":\"dataf\"}}") + "{\"type\":\"AzureQueueSink\",\"writeBatchSize\":\"dataiottdawgkaohhtt\",\"writeBatchTimeout\":\"datahypidzjjjfcyskpn\",\"sinkRetryCount\":\"dataxoic\",\"sinkRetryWait\":\"datasmfvltbocqhv\",\"maxConcurrentConnections\":\"datam\",\"disableMetricsCollection\":\"datapvgri\",\"\":{\"fmfkuvybem\":\"datagrlgkoqbzrclarr\"}}") .toObject(AzureQueueSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureQueueSink model = new AzureQueueSink().withWriteBatchSize("datanazjvyiiezdnez") - .withWriteBatchTimeout("dataqzd").withSinkRetryCount("datamyutzttroymi").withSinkRetryWait("datakuz") - .withMaxConcurrentConnections("datacegyztzhcfuwm").withDisableMetricsCollection("dataz"); + AzureQueueSink model = new AzureQueueSink().withWriteBatchSize("dataiottdawgkaohhtt") + .withWriteBatchTimeout("datahypidzjjjfcyskpn").withSinkRetryCount("dataxoic") + .withSinkRetryWait("datasmfvltbocqhv").withMaxConcurrentConnections("datam") + .withDisableMetricsCollection("datapvgri"); model = BinaryData.fromObject(model).toObject(AzureQueueSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTests.java index 4dcda7e15a5a9..5b420cf7de533 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTests.java @@ -19,30 +19,34 @@ public final class AzureSearchIndexDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureSearchIndexDataset model = BinaryData.fromString( - "{\"type\":\"AzureSearchIndex\",\"typeProperties\":{\"indexName\":\"dataryqycymzrlcfgdwz\"},\"description\":\"zfdhea\",\"structure\":\"datayciwzkil\",\"schema\":\"dataqadfgesv\",\"linkedServiceName\":{\"referenceName\":\"oha\",\"parameters\":{\"ovqmxqsxofx\":\"dataizmadjrsbgailj\",\"kgltsxooiobhieb\":\"datankiu\",\"tlsrvqzgaqsosrn\":\"datau\",\"npesw\":\"datalvgrghnhuoxrqhjn\"}},\"parameters\":{\"zdvmsnao\":{\"type\":\"Array\",\"defaultValue\":\"datagebzqzmcsviujo\"}},\"annotations\":[\"dataxoxvimdvetqh\",\"databitqsbyu\"],\"folder\":{\"name\":\"omr\"},\"\":{\"xbdpbcehwbd\":\"datamgrmsdbvqxgfygfk\"}}") + "{\"type\":\"AzureSearchIndex\",\"typeProperties\":{\"indexName\":\"datalivgti\"},\"description\":\"kqjqjcaj\",\"structure\":\"datafytkhhkemrv\",\"schema\":\"dataeoj\",\"linkedServiceName\":{\"referenceName\":\"dyulglhelwr\",\"parameters\":{\"gaex\":\"datafqfxspxgogypbz\",\"mb\":\"datanskvctvuz\",\"cyxrn\":\"datattmhlvr\",\"pdwzjggkwdep\":\"dataukfaj\"}},\"parameters\":{\"gtrttcuayiqylnk\":{\"type\":\"Float\",\"defaultValue\":\"datayfiqiidxcorjvudy\"},\"gqexowq\":{\"type\":\"Object\",\"defaultValue\":\"datazifb\"},\"kuobpw\":{\"type\":\"Array\",\"defaultValue\":\"datagqrqkkvfy\"},\"ewhpnyjt\":{\"type\":\"String\",\"defaultValue\":\"datapgobothx\"}},\"annotations\":[\"datazyvextchslro\",\"datadowuwhdlri\"],\"folder\":{\"name\":\"zttcbiich\"},\"\":{\"ycymzrlcfgdwzauz\":\"datadsozodwjcfqoyxry\",\"ilykqadfgesv\":\"datadheadnyciwz\",\"bri\":\"dataoha\",\"ljqovqmxqsxo\":\"datamadjrsbga\"}}") .toObject(AzureSearchIndexDataset.class); - Assertions.assertEquals("zfdhea", model.description()); - Assertions.assertEquals("oha", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("zdvmsnao").type()); - Assertions.assertEquals("omr", model.folder().name()); + Assertions.assertEquals("kqjqjcaj", model.description()); + Assertions.assertEquals("dyulglhelwr", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("gtrttcuayiqylnk").type()); + Assertions.assertEquals("zttcbiich", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureSearchIndexDataset model = new AzureSearchIndexDataset().withDescription("zfdhea") - .withStructure("datayciwzkil").withSchema("dataqadfgesv") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("oha") - .withParameters(mapOf("ovqmxqsxofx", "dataizmadjrsbgailj", "kgltsxooiobhieb", "datankiu", - "tlsrvqzgaqsosrn", "datau", "npesw", "datalvgrghnhuoxrqhjn"))) - .withParameters(mapOf("zdvmsnao", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datagebzqzmcsviujo"))) - .withAnnotations(Arrays.asList("dataxoxvimdvetqh", "databitqsbyu")) - .withFolder(new DatasetFolder().withName("omr")).withIndexName("dataryqycymzrlcfgdwz"); + AzureSearchIndexDataset model = new AzureSearchIndexDataset().withDescription("kqjqjcaj") + .withStructure("datafytkhhkemrv").withSchema("dataeoj") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dyulglhelwr") + .withParameters(mapOf("gaex", "datafqfxspxgogypbz", "mb", "datanskvctvuz", "cyxrn", "datattmhlvr", + "pdwzjggkwdep", "dataukfaj"))) + .withParameters(mapOf("gtrttcuayiqylnk", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datayfiqiidxcorjvudy"), + "gqexowq", new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datazifb"), + "kuobpw", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datagqrqkkvfy"), + "ewhpnyjt", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datapgobothx"))) + .withAnnotations(Arrays.asList("datazyvextchslro", "datadowuwhdlri")) + .withFolder(new DatasetFolder().withName("zttcbiich")).withIndexName("datalivgti"); model = BinaryData.fromObject(model).toObject(AzureSearchIndexDataset.class); - Assertions.assertEquals("zfdhea", model.description()); - Assertions.assertEquals("oha", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("zdvmsnao").type()); - Assertions.assertEquals("omr", model.folder().name()); + Assertions.assertEquals("kqjqjcaj", model.description()); + Assertions.assertEquals("dyulglhelwr", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("gtrttcuayiqylnk").type()); + Assertions.assertEquals("zttcbiich", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTypePropertiesTests.java index e20deaa4bb954..d247922da0d53 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexDatasetTypePropertiesTests.java @@ -10,14 +10,14 @@ public final class AzureSearchIndexDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzureSearchIndexDatasetTypeProperties model = BinaryData.fromString("{\"indexName\":\"datasesboynpytporr\"}") + AzureSearchIndexDatasetTypeProperties model = BinaryData.fromString("{\"indexName\":\"dataxqnkiuokg\"}") .toObject(AzureSearchIndexDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureSearchIndexDatasetTypeProperties model - = new AzureSearchIndexDatasetTypeProperties().withIndexName("datasesboynpytporr"); + = new AzureSearchIndexDatasetTypeProperties().withIndexName("dataxqnkiuokg"); model = BinaryData.fromObject(model).toObject(AzureSearchIndexDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexSinkTests.java index d6b3cfd19921c..61ab3a7dd34e8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSearchIndexSinkTests.java @@ -13,17 +13,17 @@ public final class AzureSearchIndexSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureSearchIndexSink model = BinaryData.fromString( - "{\"type\":\"AzureSearchIndexSink\",\"writeBehavior\":\"Merge\",\"writeBatchSize\":\"datatq\",\"writeBatchTimeout\":\"datapbtqibq\",\"sinkRetryCount\":\"dataugcwzgdfdrdxo\",\"sinkRetryWait\":\"datakgezulnntpbarej\",\"maxConcurrentConnections\":\"datahlxoljbpoeoywykh\",\"disableMetricsCollection\":\"datavwhrivvzr\",\"\":{\"eearbbxaneviqk\":\"datafrxlsypwu\",\"gbpcenirjhd\":\"dataupvidzhjcppq\",\"btlharjbakp\":\"dataxup\"}}") + "{\"type\":\"AzureSearchIndexSink\",\"writeBehavior\":\"Merge\",\"writeBatchSize\":\"datayxauw\",\"writeBatchTimeout\":\"dataqofrkfccqjenz\",\"sinkRetryCount\":\"dataxazywijb\",\"sinkRetryWait\":\"dataae\",\"maxConcurrentConnections\":\"dataozbdwflxdwl\",\"disableMetricsCollection\":\"datataieledmiupddlet\",\"\":{\"ktsgcypjlmsta\":\"datadc\",\"k\":\"datagrebecxuuzeuklu\",\"kttxvmbedvvmr\":\"dataejamychwwrvvtj\"}}") .toObject(AzureSearchIndexSink.class); Assertions.assertEquals(AzureSearchIndexWriteBehaviorType.MERGE, model.writeBehavior()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureSearchIndexSink model = new AzureSearchIndexSink().withWriteBatchSize("datatq") - .withWriteBatchTimeout("datapbtqibq").withSinkRetryCount("dataugcwzgdfdrdxo") - .withSinkRetryWait("datakgezulnntpbarej").withMaxConcurrentConnections("datahlxoljbpoeoywykh") - .withDisableMetricsCollection("datavwhrivvzr").withWriteBehavior(AzureSearchIndexWriteBehaviorType.MERGE); + AzureSearchIndexSink model = new AzureSearchIndexSink().withWriteBatchSize("datayxauw") + .withWriteBatchTimeout("dataqofrkfccqjenz").withSinkRetryCount("dataxazywijb").withSinkRetryWait("dataae") + .withMaxConcurrentConnections("dataozbdwflxdwl").withDisableMetricsCollection("datataieledmiupddlet") + .withWriteBehavior(AzureSearchIndexWriteBehaviorType.MERGE); model = BinaryData.fromObject(model).toObject(AzureSearchIndexSink.class); Assertions.assertEquals(AzureSearchIndexWriteBehaviorType.MERGE, model.writeBehavior()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSqlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSqlSourceTests.java index 48852a5f628f1..9709155118c11 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSqlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSqlSourceTests.java @@ -12,21 +12,20 @@ public final class AzureSqlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureSqlSource model = BinaryData.fromString( - "{\"type\":\"AzureSqlSource\",\"sqlReaderQuery\":\"datakpv\",\"sqlReaderStoredProcedureName\":\"datastzqzhdwrcajfers\",\"storedProcedureParameters\":\"dataxlkcw\",\"isolationLevel\":\"dataejssksgxykdepqcy\",\"produceAdditionalTypes\":\"datahwsxpzkmotgmd\",\"partitionOption\":\"datawwqevbiuntp\",\"partitionSettings\":{\"partitionColumnName\":\"datawjxlycelf\",\"partitionUpperBound\":\"datagywrauuru\",\"partitionLowerBound\":\"datataycnkirmiec\"},\"queryTimeout\":\"dataqcxmpcdbvcxoam\",\"additionalColumns\":\"datafuh\",\"sourceRetryCount\":\"datad\",\"sourceRetryWait\":\"dataawectzjjgvcbtq\",\"maxConcurrentConnections\":\"datapnkyvujhejy\",\"disableMetricsCollection\":\"datavlguysbrn\",\"\":{\"jshtcfnb\":\"dataqhmuqyzxkormr\"}}") + "{\"type\":\"AzureSqlSource\",\"sqlReaderQuery\":\"datak\",\"sqlReaderStoredProcedureName\":\"datayu\",\"storedProcedureParameters\":\"dataemrclsxgpkyetmt\",\"isolationLevel\":\"dataihixisdvy\",\"produceAdditionalTypes\":\"datakeqg\",\"partitionOption\":\"datajsbtosiwcve\",\"partitionSettings\":{\"partitionColumnName\":\"datahbw\",\"partitionUpperBound\":\"dataoc\",\"partitionLowerBound\":\"datazlfhhwdajfth\"},\"queryTimeout\":\"datawuomjd\",\"additionalColumns\":\"dataufqhq\",\"sourceRetryCount\":\"dataio\",\"sourceRetryWait\":\"datahxxbneiobubtpyem\",\"maxConcurrentConnections\":\"datawkryzrdqpkqfuv\",\"disableMetricsCollection\":\"datai\",\"\":{\"jgmfyernckg\":\"databhmy\",\"kafiokeolzizfbun\":\"datawiq\"}}") .toObject(AzureSqlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureSqlSource model - = new AzureSqlSource().withSourceRetryCount("datad").withSourceRetryWait("dataawectzjjgvcbtq") - .withMaxConcurrentConnections("datapnkyvujhejy").withDisableMetricsCollection("datavlguysbrn") - .withQueryTimeout("dataqcxmpcdbvcxoam").withAdditionalColumns("datafuh").withSqlReaderQuery("datakpv") - .withSqlReaderStoredProcedureName("datastzqzhdwrcajfers").withStoredProcedureParameters("dataxlkcw") - .withIsolationLevel("dataejssksgxykdepqcy").withProduceAdditionalTypes("datahwsxpzkmotgmd") - .withPartitionOption("datawwqevbiuntp") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datawjxlycelf") - .withPartitionUpperBound("datagywrauuru").withPartitionLowerBound("datataycnkirmiec")); + AzureSqlSource model = new AzureSqlSource().withSourceRetryCount("dataio") + .withSourceRetryWait("datahxxbneiobubtpyem").withMaxConcurrentConnections("datawkryzrdqpkqfuv") + .withDisableMetricsCollection("datai").withQueryTimeout("datawuomjd").withAdditionalColumns("dataufqhq") + .withSqlReaderQuery("datak").withSqlReaderStoredProcedureName("datayu") + .withStoredProcedureParameters("dataemrclsxgpkyetmt").withIsolationLevel("dataihixisdvy") + .withProduceAdditionalTypes("datakeqg").withPartitionOption("datajsbtosiwcve") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datahbw") + .withPartitionUpperBound("dataoc").withPartitionLowerBound("datazlfhhwdajfth")); model = BinaryData.fromObject(model).toObject(AzureSqlSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTests.java index 451a733fbe16a..f36e10a03f63b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTests.java @@ -18,29 +18,31 @@ public final class AzureSynapseArtifactsLinkedServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureSynapseArtifactsLinkedService model = BinaryData.fromString( - "{\"type\":\"AzureSynapseArtifacts\",\"typeProperties\":{\"endpoint\":\"datarjuingnfunh\",\"authentication\":\"datagxsyiwmopkud\",\"workspaceResourceId\":\"dataztbwzjbqzq\"},\"connectVia\":{\"referenceName\":\"hznh\",\"parameters\":{\"kwp\":\"databzlkdsbekvp\"}},\"description\":\"ieqcnvgzshfaf\",\"parameters\":{\"eplpfrecri\":{\"type\":\"Int\",\"defaultValue\":\"dataeiuexkpgrmwdwlr\"},\"gpdbh\":{\"type\":\"Bool\",\"defaultValue\":\"databafdlsizaob\"}},\"annotations\":[\"datasjsoxuuwuungdvvd\",\"datarcpq\",\"datayinlg\",\"datatvonw\"],\"\":{\"xsupevthuvupds\":\"dataliemitmtkcqixg\",\"ianjmvv\":\"datafqaghwqumecq\"}}") + "{\"type\":\"AzureSynapseArtifacts\",\"typeProperties\":{\"endpoint\":\"datahflnlutacsslel\",\"authentication\":\"dataukqurrtcf\",\"workspaceResourceId\":\"datat\"},\"connectVia\":{\"referenceName\":\"yrsleghozsmjj\",\"parameters\":{\"fsxvqqu\":\"datamozryyyvlxmspjq\",\"putizpvvihgxsd\":\"datazoblxxk\",\"lnarjtuoarjir\":\"datalodfodokhaogewd\"}},\"description\":\"cuyqt\",\"parameters\":{\"nvpzdbzpizgau\":{\"type\":\"Object\",\"defaultValue\":\"datajgmgfayvbsia\"},\"xorrceom\":{\"type\":\"String\",\"defaultValue\":\"datauwfy\"},\"n\":{\"type\":\"Float\",\"defaultValue\":\"datarbtrk\"},\"hg\":{\"type\":\"Object\",\"defaultValue\":\"dataijpjiudnustbmox\"}},\"annotations\":[\"datahbhue\"],\"\":{\"frhyxlwq\":\"datarlxzqzj\"}}") .toObject(AzureSynapseArtifactsLinkedService.class); - Assertions.assertEquals("hznh", model.connectVia().referenceName()); - Assertions.assertEquals("ieqcnvgzshfaf", model.description()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("eplpfrecri").type()); + Assertions.assertEquals("yrsleghozsmjj", model.connectVia().referenceName()); + Assertions.assertEquals("cuyqt", model.description()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("nvpzdbzpizgau").type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureSynapseArtifactsLinkedService model = new AzureSynapseArtifactsLinkedService() - .withConnectVia(new IntegrationRuntimeReference() - .withReferenceName("hznh").withParameters(mapOf("kwp", "databzlkdsbekvp"))) - .withDescription("ieqcnvgzshfaf") - .withParameters(mapOf("eplpfrecri", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataeiuexkpgrmwdwlr"), - "gpdbh", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("databafdlsizaob"))) - .withAnnotations(Arrays.asList("datasjsoxuuwuungdvvd", "datarcpq", "datayinlg", "datatvonw")) - .withEndpoint("datarjuingnfunh").withAuthentication("datagxsyiwmopkud") - .withWorkspaceResourceId("dataztbwzjbqzq"); + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("yrsleghozsmjj") + .withParameters(mapOf("fsxvqqu", "datamozryyyvlxmspjq", "putizpvvihgxsd", "datazoblxxk", + "lnarjtuoarjir", "datalodfodokhaogewd"))) + .withDescription("cuyqt") + .withParameters(mapOf("nvpzdbzpizgau", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datajgmgfayvbsia"), + "xorrceom", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datauwfy"), + "n", new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datarbtrk"), "hg", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataijpjiudnustbmox"))) + .withAnnotations(Arrays.asList("datahbhue")).withEndpoint("datahflnlutacsslel") + .withAuthentication("dataukqurrtcf").withWorkspaceResourceId("datat"); model = BinaryData.fromObject(model).toObject(AzureSynapseArtifactsLinkedService.class); - Assertions.assertEquals("hznh", model.connectVia().referenceName()); - Assertions.assertEquals("ieqcnvgzshfaf", model.description()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("eplpfrecri").type()); + Assertions.assertEquals("yrsleghozsmjj", model.connectVia().referenceName()); + Assertions.assertEquals("cuyqt", model.description()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("nvpzdbzpizgau").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTypePropertiesTests.java index aedc16898a313..92882ca81c56f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureSynapseArtifactsLinkedServiceTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class AzureSynapseArtifactsLinkedServiceTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureSynapseArtifactsLinkedServiceTypeProperties model = BinaryData.fromString( - "{\"endpoint\":\"dataodkq\",\"authentication\":\"datahuxox\",\"workspaceResourceId\":\"datagzvzcfmwfogjrhmt\"}") + "{\"endpoint\":\"dataousqmer\",\"authentication\":\"datajpl\",\"workspaceResourceId\":\"dataemkyouwmjxhm\"}") .toObject(AzureSynapseArtifactsLinkedServiceTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { AzureSynapseArtifactsLinkedServiceTypeProperties model = new AzureSynapseArtifactsLinkedServiceTypeProperties() - .withEndpoint("dataodkq").withAuthentication("datahuxox").withWorkspaceResourceId("datagzvzcfmwfogjrhmt"); + .withEndpoint("dataousqmer").withAuthentication("datajpl").withWorkspaceResourceId("dataemkyouwmjxhm"); model = BinaryData.fromObject(model).toObject(AzureSynapseArtifactsLinkedServiceTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureTableSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureTableSourceTests.java index 449ed010f1cee..b9995d8dad020 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureTableSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/AzureTableSourceTests.java @@ -11,17 +11,17 @@ public final class AzureTableSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AzureTableSource model = BinaryData.fromString( - "{\"type\":\"AzureTableSource\",\"azureTableSourceQuery\":\"dataopocleffgggg\",\"azureTableSourceIgnoreTableNotFound\":\"datasbouhmng\",\"queryTimeout\":\"datan\",\"additionalColumns\":\"dataiusrvswcpsp\",\"sourceRetryCount\":\"dataxigpdiouppg\",\"sourceRetryWait\":\"datasmglutnlynplxx\",\"maxConcurrentConnections\":\"dataogxk\",\"disableMetricsCollection\":\"databcxbgfwwcfwlwnjg\",\"\":{\"fbjesmiyj\":\"dataopcmetdruugimvi\",\"xqsvax\":\"datahjuuepnjuqwnajbb\"}}") + "{\"type\":\"AzureTableSource\",\"azureTableSourceQuery\":\"dataeomufazacwzb\",\"azureTableSourceIgnoreTableNotFound\":\"datapbsspexejhwpnjco\",\"queryTimeout\":\"datajzbovuvmdz\",\"additionalColumns\":\"datatirguajs\",\"sourceRetryCount\":\"dataecbowkhm\",\"sourceRetryWait\":\"datafllpdn\",\"maxConcurrentConnections\":\"dataujxoueqljz\",\"disableMetricsCollection\":\"datancaeykueatztnp\",\"\":{\"gsqwiubgbltjy\":\"datahlniahvl\",\"uwyluktz\":\"datasjs\"}}") .toObject(AzureTableSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureTableSource model - = new AzureTableSource().withSourceRetryCount("dataxigpdiouppg").withSourceRetryWait("datasmglutnlynplxx") - .withMaxConcurrentConnections("dataogxk").withDisableMetricsCollection("databcxbgfwwcfwlwnjg") - .withQueryTimeout("datan").withAdditionalColumns("dataiusrvswcpsp") - .withAzureTableSourceQuery("dataopocleffgggg").withAzureTableSourceIgnoreTableNotFound("datasbouhmng"); + AzureTableSource model = new AzureTableSource().withSourceRetryCount("dataecbowkhm") + .withSourceRetryWait("datafllpdn").withMaxConcurrentConnections("dataujxoueqljz") + .withDisableMetricsCollection("datancaeykueatztnp").withQueryTimeout("datajzbovuvmdz") + .withAdditionalColumns("datatirguajs").withAzureTableSourceQuery("dataeomufazacwzb") + .withAzureTableSourceIgnoreTableNotFound("datapbsspexejhwpnjco"); model = BinaryData.fromObject(model).toObject(AzureTableSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BigDataPoolParametrizationReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BigDataPoolParametrizationReferenceTests.java index 92b0af8dedc44..0378d08709e66 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BigDataPoolParametrizationReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BigDataPoolParametrizationReferenceTests.java @@ -13,7 +13,7 @@ public final class BigDataPoolParametrizationReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BigDataPoolParametrizationReference model - = BinaryData.fromString("{\"type\":\"BigDataPoolReference\",\"referenceName\":\"datavr\"}") + = BinaryData.fromString("{\"type\":\"BigDataPoolReference\",\"referenceName\":\"datacuiipnszrrmq\"}") .toObject(BigDataPoolParametrizationReference.class); Assertions.assertEquals(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE, model.type()); } @@ -21,7 +21,7 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { BigDataPoolParametrizationReference model = new BigDataPoolParametrizationReference() - .withType(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE).withReferenceName("datavr"); + .withType(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE).withReferenceName("datacuiipnszrrmq"); model = BinaryData.fromObject(model).toObject(BigDataPoolParametrizationReference.class); Assertions.assertEquals(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE, model.type()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinaryReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinaryReadSettingsTests.java index 55140c3fe3b62..3628aa6f6f094 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinaryReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinaryReadSettingsTests.java @@ -14,7 +14,7 @@ public final class BinaryReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BinaryReadSettings model = BinaryData.fromString( - "{\"type\":\"BinaryReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"jxfqyyuwci\":\"dataylzbdimtdo\"}},\"\":{\"icxjxueilixzjvk\":\"datanxhbttk\",\"xxpn\":\"datajrbblhcm\",\"squxxqcimnchvkj\":\"dataxcljnkms\",\"gtfozysfjdc\":\"datarivagcsmrtepsyb\"}}") + "{\"type\":\"BinaryReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"wigif\":\"datazpuz\"}},\"\":{\"tlpshxjhanskooal\":\"datays\"}}") .toObject(BinaryReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySinkTests.java index 9433838e46b24..082cd74dd47a6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySinkTests.java @@ -16,23 +16,19 @@ public final class BinarySinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BinarySink model = BinaryData.fromString( - "{\"type\":\"BinarySink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataaaruvbzcqgtz\",\"disableMetricsCollection\":\"datalrmrtdznvjgovy\",\"copyBehavior\":\"datappswlept\",\"metadata\":[{\"name\":\"datakntfwxkeu\",\"value\":\"datagpcrvv\"},{\"name\":\"datadlckpznoveabwpai\",\"value\":\"datakzysdhars\"},{\"name\":\"dataqmrpdx\",\"value\":\"datadvtv\"},{\"name\":\"dataqlt\",\"value\":\"datandf\"}],\"\":{\"tsnqjcmk\":\"datasdiehrajbatgmxko\"}},\"writeBatchSize\":\"databckjrfkwclqmyowd\",\"writeBatchTimeout\":\"datatwaxobdzatqocvrd\",\"sinkRetryCount\":\"datavsclwpsteuvjdnh\",\"sinkRetryWait\":\"datayvymvnlaehit\",\"maxConcurrentConnections\":\"dataibfomohcynorhhbv\",\"disableMetricsCollection\":\"dataxtktkeuapomoof\",\"\":{\"mathiydmkyvsxc\":\"datahptraljcqpu\",\"fmkp\":\"dataivghajpddgfozn\",\"mwptdrrruy\":\"dataoesozcuhunmfz\"}}") + "{\"type\":\"BinarySink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataoogijiqwxw\",\"disableMetricsCollection\":\"databdhqnprb\",\"copyBehavior\":\"datauhdjzivl\",\"metadata\":[{\"name\":\"datamq\",\"value\":\"datambfpt\"}],\"\":{\"zulmqxficinwij\":\"datamksxxbdtjvvngnvk\",\"er\":\"dataeyxt\",\"ytten\":\"datattobosjxb\",\"lyimhspj\":\"datauditumyycvtya\"}},\"writeBatchSize\":\"dataivxblqvwhjgt\",\"writeBatchTimeout\":\"datareoutqohp\",\"sinkRetryCount\":\"dataqi\",\"sinkRetryWait\":\"databougcwzgdfdrdx\",\"maxConcurrentConnections\":\"datatkgezul\",\"disableMetricsCollection\":\"datatp\",\"\":{\"eoy\":\"dataejxjhlxoljbp\",\"havwhrivvzrc\":\"datayk\",\"eearbbxaneviqk\":\"datayfrxlsypwu\"}}") .toObject(BinarySink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BinarySink model - = new BinarySink().withWriteBatchSize("databckjrfkwclqmyowd").withWriteBatchTimeout("datatwaxobdzatqocvrd") - .withSinkRetryCount("datavsclwpsteuvjdnh").withSinkRetryWait("datayvymvnlaehit") - .withMaxConcurrentConnections("dataibfomohcynorhhbv").withDisableMetricsCollection("dataxtktkeuapomoof") - .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataaaruvbzcqgtz") - .withDisableMetricsCollection("datalrmrtdznvjgovy").withCopyBehavior("datappswlept") - .withMetadata(Arrays.asList(new MetadataItem().withName("datakntfwxkeu").withValue("datagpcrvv"), - new MetadataItem().withName("datadlckpznoveabwpai").withValue("datakzysdhars"), - new MetadataItem().withName("dataqmrpdx").withValue("datadvtv"), - new MetadataItem().withName("dataqlt").withValue("datandf"))) - .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))); + BinarySink model = new BinarySink().withWriteBatchSize("dataivxblqvwhjgt") + .withWriteBatchTimeout("datareoutqohp").withSinkRetryCount("dataqi").withSinkRetryWait("databougcwzgdfdrdx") + .withMaxConcurrentConnections("datatkgezul").withDisableMetricsCollection("datatp") + .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataoogijiqwxw") + .withDisableMetricsCollection("databdhqnprb").withCopyBehavior("datauhdjzivl") + .withMetadata(Arrays.asList(new MetadataItem().withName("datamq").withValue("datambfpt"))) + .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))); model = BinaryData.fromObject(model).toObject(BinarySink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySourceTests.java index 1bed0a1785dca..7820561f1cb07 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BinarySourceTests.java @@ -16,17 +16,17 @@ public final class BinarySourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BinarySource model = BinaryData.fromString( - "{\"type\":\"BinarySource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataccfwqisouqyz\",\"disableMetricsCollection\":\"datajehdklvqtm\",\"\":{\"oazsj\":\"dataiaetctjhpg\",\"ccvxqbxgq\":\"datauevfvnnt\",\"kibvw\":\"dataawnriwx\"}},\"formatSettings\":{\"type\":\"BinaryReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"bk\":\"datamxsvz\",\"bxldl\":\"dataal\",\"kalpqlnn\":\"datathmibqgldht\",\"ewlwbxuf\":\"dataevjealx\"}},\"\":{\"lfqvo\":\"datakkvij\"}},\"sourceRetryCount\":\"dataowdqvqfldaqoao\",\"sourceRetryWait\":\"dataqpfwnjdyoxformfe\",\"maxConcurrentConnections\":\"datafq\",\"disableMetricsCollection\":\"dataeszxtesnhxp\",\"\":{\"bvtpqttusuxxbzm\":\"datak\",\"grxzcnwguezx\":\"datavueijrnn\",\"wbwmgqzq\":\"datauim\"}}") + "{\"type\":\"BinarySource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataliteenaheecsf\",\"disableMetricsCollection\":\"datasubzfuhjnmdcyrbz\",\"\":{\"dxtwyxpkwwdkkvd\":\"dataqgvt\",\"iluexvml\":\"datavdvkeyqxjchdnlx\",\"lbqkguchd\":\"dataxqpsqpfxjwt\"}},\"formatSettings\":{\"type\":\"BinaryReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"gddkujvqzcuqculw\":\"datadjikiqtzu\",\"tjcrp\":\"dataxryy\"}},\"\":{\"dldm\":\"dataxlfxsetvdz\",\"iuxotbvflgkkiu\":\"datafqftywbbanzhd\",\"fobpyeo\":\"datahop\"}},\"sourceRetryCount\":\"datataevqafdhpkiiu\",\"sourceRetryWait\":\"datarobckelo\",\"maxConcurrentConnections\":\"datamrvdtqhrtnqssqy\",\"disableMetricsCollection\":\"datapskitokphamefzz\",\"\":{\"fgq\":\"dataoauedmdpndouylf\",\"mzassr\":\"datanaokxouknz\",\"btxtdmutdrrqq\":\"dataqzuk\",\"twqjft\":\"datajhklttliuwd\"}}") .toObject(BinarySource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { BinarySource model - = new BinarySource().withSourceRetryCount("dataowdqvqfldaqoao").withSourceRetryWait("dataqpfwnjdyoxformfe") - .withMaxConcurrentConnections("datafq").withDisableMetricsCollection("dataeszxtesnhxp") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataccfwqisouqyz") - .withDisableMetricsCollection("datajehdklvqtm") + = new BinarySource().withSourceRetryCount("datataevqafdhpkiiu").withSourceRetryWait("datarobckelo") + .withMaxConcurrentConnections("datamrvdtqhrtnqssqy").withDisableMetricsCollection("datapskitokphamefzz") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataliteenaheecsf") + .withDisableMetricsCollection("datasubzfuhjnmdcyrbz") .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) .withFormatSettings(new BinaryReadSettings().withCompressionProperties( new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings")))); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTests.java index a88b95b17724e..2bc6c29b1fd0b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTests.java @@ -18,48 +18,38 @@ public final class BlobEventsTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BlobEventsTrigger model = BinaryData.fromString( - "{\"type\":\"BlobEventsTrigger\",\"typeProperties\":{\"blobPathBeginsWith\":\"kjrtcifxl\",\"blobPathEndsWith\":\"jpu\",\"ignoreEmptyBlobs\":true,\"events\":[\"Microsoft.Storage.BlobDeleted\",\"Microsoft.Storage.BlobCreated\"],\"scope\":\"sjcdmnvtpboh\"},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"qvsdaqfvplfywcbn\",\"name\":\"shmqn\"},\"parameters\":{\"lqs\":\"datavqsqwuwxtqdtvej\",\"fyddrsairxnw\":\"datajxpro\",\"ahe\":\"dataiytkeqjviawspvbc\",\"rmhewdfua\":\"datalmiuprfqyrwtdnr\"}},{\"pipelineReference\":{\"referenceName\":\"kfojn\",\"name\":\"dtuyimdoprkpd\"},\"parameters\":{\"b\":\"datasa\",\"jedyyengnh\":\"datab\"}},{\"pipelineReference\":{\"referenceName\":\"bpqzqaavxxvlss\",\"name\":\"txdr\"},\"parameters\":{\"x\":\"datahqwfrtbvqtog\"}},{\"pipelineReference\":{\"referenceName\":\"vkntygu\",\"name\":\"qobr\"},\"parameters\":{\"gq\":\"datajyami\",\"ibwkiwyt\":\"dataizvvwyhszewhzaiu\"}}],\"description\":\"pbrejvwwb\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datadind\"],\"\":{\"cn\":\"dataiq\",\"gkxrevwvjwtf\":\"datacjli\"}}") + "{\"type\":\"BlobEventsTrigger\",\"typeProperties\":{\"blobPathBeginsWith\":\"vugarykbyuukhssr\",\"blobPathEndsWith\":\"ugorcz\",\"ignoreEmptyBlobs\":false,\"events\":[\"Microsoft.Storage.BlobDeleted\",\"Microsoft.Storage.BlobCreated\"],\"scope\":\"tdwktogmcbl\"},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"nisinpkcww\",\"name\":\"kabj\"},\"parameters\":{\"iaqmuptnhuybt\":\"datahngaczgg\",\"hyozxotwral\":\"datatok\"}}],\"description\":\"jzlnrellwf\",\"runtimeState\":\"Started\",\"annotations\":[\"datalsarfmjschc\",\"dataudromhhs\",\"datamzfvrakpqlltoiu\",\"dataveoibehrh\"],\"\":{\"dkwwuljv\":\"datajxilbsbhaqsucwdc\",\"hnh\":\"dataujgsxrsxbofmva\"}}") .toObject(BlobEventsTrigger.class); - Assertions.assertEquals("pbrejvwwb", model.description()); - Assertions.assertEquals("qvsdaqfvplfywcbn", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("shmqn", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals("kjrtcifxl", model.blobPathBeginsWith()); - Assertions.assertEquals("jpu", model.blobPathEndsWith()); - Assertions.assertEquals(true, model.ignoreEmptyBlobs()); + Assertions.assertEquals("jzlnrellwf", model.description()); + Assertions.assertEquals("nisinpkcww", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("kabj", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("vugarykbyuukhssr", model.blobPathBeginsWith()); + Assertions.assertEquals("ugorcz", model.blobPathEndsWith()); + Assertions.assertEquals(false, model.ignoreEmptyBlobs()); Assertions.assertEquals(BlobEventTypes.MICROSOFT_STORAGE_BLOB_DELETED, model.events().get(0)); - Assertions.assertEquals("sjcdmnvtpboh", model.scope()); + Assertions.assertEquals("tdwktogmcbl", model.scope()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BlobEventsTrigger model = new BlobEventsTrigger().withDescription("pbrejvwwb") - .withAnnotations(Arrays.asList("datadind")) + BlobEventsTrigger model = new BlobEventsTrigger().withDescription("jzlnrellwf") + .withAnnotations(Arrays.asList("datalsarfmjschc", "dataudromhhs", "datamzfvrakpqlltoiu", "dataveoibehrh")) .withPipelines(Arrays.asList(new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("qvsdaqfvplfywcbn").withName("shmqn")) - .withParameters(mapOf("lqs", "datavqsqwuwxtqdtvej", "fyddrsairxnw", "datajxpro", "ahe", - "dataiytkeqjviawspvbc", "rmhewdfua", "datalmiuprfqyrwtdnr")), - new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("kfojn").withName("dtuyimdoprkpd")) - .withParameters(mapOf("b", "datasa", "jedyyengnh", "datab")), - new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("bpqzqaavxxvlss").withName("txdr")) - .withParameters(mapOf("x", "datahqwfrtbvqtog")), - new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("vkntygu").withName("qobr")) - .withParameters(mapOf("gq", "datajyami", "ibwkiwyt", "dataizvvwyhszewhzaiu")))) - .withBlobPathBeginsWith("kjrtcifxl").withBlobPathEndsWith("jpu").withIgnoreEmptyBlobs(true) + .withPipelineReference(new PipelineReference().withReferenceName("nisinpkcww").withName("kabj")) + .withParameters(mapOf("iaqmuptnhuybt", "datahngaczgg", "hyozxotwral", "datatok")))) + .withBlobPathBeginsWith("vugarykbyuukhssr").withBlobPathEndsWith("ugorcz").withIgnoreEmptyBlobs(false) .withEvents(Arrays.asList(BlobEventTypes.MICROSOFT_STORAGE_BLOB_DELETED, BlobEventTypes.MICROSOFT_STORAGE_BLOB_CREATED)) - .withScope("sjcdmnvtpboh"); + .withScope("tdwktogmcbl"); model = BinaryData.fromObject(model).toObject(BlobEventsTrigger.class); - Assertions.assertEquals("pbrejvwwb", model.description()); - Assertions.assertEquals("qvsdaqfvplfywcbn", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("shmqn", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals("kjrtcifxl", model.blobPathBeginsWith()); - Assertions.assertEquals("jpu", model.blobPathEndsWith()); - Assertions.assertEquals(true, model.ignoreEmptyBlobs()); + Assertions.assertEquals("jzlnrellwf", model.description()); + Assertions.assertEquals("nisinpkcww", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("kabj", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("vugarykbyuukhssr", model.blobPathBeginsWith()); + Assertions.assertEquals("ugorcz", model.blobPathEndsWith()); + Assertions.assertEquals(false, model.ignoreEmptyBlobs()); Assertions.assertEquals(BlobEventTypes.MICROSOFT_STORAGE_BLOB_DELETED, model.events().get(0)); - Assertions.assertEquals("sjcdmnvtpboh", model.scope()); + Assertions.assertEquals("tdwktogmcbl", model.scope()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTypePropertiesTests.java index 57ddb0bf9b201..fb6b5d6e31f86 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobEventsTriggerTypePropertiesTests.java @@ -14,26 +14,25 @@ public final class BlobEventsTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BlobEventsTriggerTypeProperties model = BinaryData.fromString( - "{\"blobPathBeginsWith\":\"cyl\",\"blobPathEndsWith\":\"zufznaedzyuxrufw\",\"ignoreEmptyBlobs\":false,\"events\":[\"Microsoft.Storage.BlobDeleted\",\"Microsoft.Storage.BlobCreated\"],\"scope\":\"dphyxlxvo\"}") + "{\"blobPathBeginsWith\":\"swlfuuki\",\"blobPathEndsWith\":\"laytviwvejwt\",\"ignoreEmptyBlobs\":true,\"events\":[\"Microsoft.Storage.BlobCreated\"],\"scope\":\"ps\"}") .toObject(BlobEventsTriggerTypeProperties.class); - Assertions.assertEquals("cyl", model.blobPathBeginsWith()); - Assertions.assertEquals("zufznaedzyuxrufw", model.blobPathEndsWith()); - Assertions.assertEquals(false, model.ignoreEmptyBlobs()); - Assertions.assertEquals(BlobEventTypes.MICROSOFT_STORAGE_BLOB_DELETED, model.events().get(0)); - Assertions.assertEquals("dphyxlxvo", model.scope()); + Assertions.assertEquals("swlfuuki", model.blobPathBeginsWith()); + Assertions.assertEquals("laytviwvejwt", model.blobPathEndsWith()); + Assertions.assertEquals(true, model.ignoreEmptyBlobs()); + Assertions.assertEquals(BlobEventTypes.MICROSOFT_STORAGE_BLOB_CREATED, model.events().get(0)); + Assertions.assertEquals("ps", model.scope()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BlobEventsTriggerTypeProperties model = new BlobEventsTriggerTypeProperties().withBlobPathBeginsWith("cyl") - .withBlobPathEndsWith("zufznaedzyuxrufw").withIgnoreEmptyBlobs(false).withEvents(Arrays - .asList(BlobEventTypes.MICROSOFT_STORAGE_BLOB_DELETED, BlobEventTypes.MICROSOFT_STORAGE_BLOB_CREATED)) - .withScope("dphyxlxvo"); + BlobEventsTriggerTypeProperties model = new BlobEventsTriggerTypeProperties().withBlobPathBeginsWith("swlfuuki") + .withBlobPathEndsWith("laytviwvejwt").withIgnoreEmptyBlobs(true) + .withEvents(Arrays.asList(BlobEventTypes.MICROSOFT_STORAGE_BLOB_CREATED)).withScope("ps"); model = BinaryData.fromObject(model).toObject(BlobEventsTriggerTypeProperties.class); - Assertions.assertEquals("cyl", model.blobPathBeginsWith()); - Assertions.assertEquals("zufznaedzyuxrufw", model.blobPathEndsWith()); - Assertions.assertEquals(false, model.ignoreEmptyBlobs()); - Assertions.assertEquals(BlobEventTypes.MICROSOFT_STORAGE_BLOB_DELETED, model.events().get(0)); - Assertions.assertEquals("dphyxlxvo", model.scope()); + Assertions.assertEquals("swlfuuki", model.blobPathBeginsWith()); + Assertions.assertEquals("laytviwvejwt", model.blobPathEndsWith()); + Assertions.assertEquals(true, model.ignoreEmptyBlobs()); + Assertions.assertEquals(BlobEventTypes.MICROSOFT_STORAGE_BLOB_CREATED, model.events().get(0)); + Assertions.assertEquals("ps", model.scope()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSinkTests.java index 89e281f84549d..3acd6a5a58d36 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSinkTests.java @@ -13,21 +13,20 @@ public final class BlobSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BlobSink model = BinaryData.fromString( - "{\"type\":\"BlobSink\",\"blobWriterOverwriteFiles\":\"dataoiumuxna\",\"blobWriterDateTimeFormat\":\"datavgmckxh\",\"blobWriterAddHeader\":\"datazsmpoiu\",\"copyBehavior\":\"dataatv\",\"metadata\":[{\"name\":\"datajncgjogmv\",\"value\":\"datakekmgpseassdqpw\"},{\"name\":\"dataxwdo\",\"value\":\"datagbvs\"},{\"name\":\"datajfnpw\",\"value\":\"datajci\"},{\"name\":\"datalbpuemqetmo\",\"value\":\"datavhhedc\"}],\"writeBatchSize\":\"datalycrldwccas\",\"writeBatchTimeout\":\"databdvsorvhbygw\",\"sinkRetryCount\":\"dataxqlzzkbx\",\"sinkRetryWait\":\"datacgg\",\"maxConcurrentConnections\":\"dataxbtqizydaiol\",\"disableMetricsCollection\":\"datakghlexvq\",\"\":{\"zxylt\":\"datawmo\",\"e\":\"datapqjfoujeiagny\"}}") + "{\"type\":\"BlobSink\",\"blobWriterOverwriteFiles\":\"datapvidzhjcppqcgbp\",\"blobWriterDateTimeFormat\":\"datani\",\"blobWriterAddHeader\":\"datahd\",\"copyBehavior\":\"datau\",\"metadata\":[{\"name\":\"datalharjbakp\",\"value\":\"datauug\"},{\"name\":\"datagdulvdenh\",\"value\":\"datavaimkoyrp\"},{\"name\":\"datanljduwkbozlmr\",\"value\":\"dataghvlvdjxbjqiab\"}],\"writeBatchSize\":\"dataevvowiypyljzkx\",\"writeBatchTimeout\":\"datafyv\",\"sinkRetryCount\":\"dataftaq\",\"sinkRetryWait\":\"datacrjl\",\"maxConcurrentConnections\":\"dataneykxewemtazmrej\",\"disableMetricsCollection\":\"dataxoqege\",\"\":{\"rt\":\"datalfnuglmyr\",\"zwgrs\":\"datakpo\"}}") .toObject(BlobSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BlobSink model = new BlobSink().withWriteBatchSize("datalycrldwccas").withWriteBatchTimeout("databdvsorvhbygw") - .withSinkRetryCount("dataxqlzzkbx").withSinkRetryWait("datacgg") - .withMaxConcurrentConnections("dataxbtqizydaiol").withDisableMetricsCollection("datakghlexvq") - .withBlobWriterOverwriteFiles("dataoiumuxna").withBlobWriterDateTimeFormat("datavgmckxh") - .withBlobWriterAddHeader("datazsmpoiu").withCopyBehavior("dataatv") - .withMetadata(Arrays.asList(new MetadataItem().withName("datajncgjogmv").withValue("datakekmgpseassdqpw"), - new MetadataItem().withName("dataxwdo").withValue("datagbvs"), - new MetadataItem().withName("datajfnpw").withValue("datajci"), - new MetadataItem().withName("datalbpuemqetmo").withValue("datavhhedc"))); + BlobSink model = new BlobSink().withWriteBatchSize("dataevvowiypyljzkx").withWriteBatchTimeout("datafyv") + .withSinkRetryCount("dataftaq").withSinkRetryWait("datacrjl") + .withMaxConcurrentConnections("dataneykxewemtazmrej").withDisableMetricsCollection("dataxoqege") + .withBlobWriterOverwriteFiles("datapvidzhjcppqcgbp").withBlobWriterDateTimeFormat("datani") + .withBlobWriterAddHeader("datahd").withCopyBehavior("datau") + .withMetadata(Arrays.asList(new MetadataItem().withName("datalharjbakp").withValue("datauug"), + new MetadataItem().withName("datagdulvdenh").withValue("datavaimkoyrp"), + new MetadataItem().withName("datanljduwkbozlmr").withValue("dataghvlvdjxbjqiab"))); model = BinaryData.fromObject(model).toObject(BlobSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSourceTests.java index 44b038e5641c6..1817b5f6b7840 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobSourceTests.java @@ -11,16 +11,15 @@ public final class BlobSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BlobSource model = BinaryData.fromString( - "{\"type\":\"BlobSource\",\"treatEmptyAsNull\":\"datawisdwtugadxupipg\",\"skipHeaderLineCount\":\"datawiwhtjoxxllh\",\"recursive\":\"dataunnwm\",\"sourceRetryCount\":\"dataxyawxkd\",\"sourceRetryWait\":\"datavhyu\",\"maxConcurrentConnections\":\"datahapcqmcvureku\",\"disableMetricsCollection\":\"datajnizwz\",\"\":{\"iifgxnfarmf\":\"datarxf\",\"trulzlrmrtcs\":\"datacqrder\"}}") + "{\"type\":\"BlobSource\",\"treatEmptyAsNull\":\"datax\",\"skipHeaderLineCount\":\"dataa\",\"recursive\":\"dataheguxrziryxrpjr\",\"sourceRetryCount\":\"datamxqvv\",\"sourceRetryWait\":\"dataofts\",\"maxConcurrentConnections\":\"datafwusfbrnjvzl\",\"disableMetricsCollection\":\"datavjemp\",\"\":{\"tbieuqfgkfift\":\"dataslwz\",\"wwutduch\":\"databupuukp\",\"c\":\"datafndijzp\"}}") .toObject(BlobSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BlobSource model = new BlobSource().withSourceRetryCount("dataxyawxkd").withSourceRetryWait("datavhyu") - .withMaxConcurrentConnections("datahapcqmcvureku").withDisableMetricsCollection("datajnizwz") - .withTreatEmptyAsNull("datawisdwtugadxupipg").withSkipHeaderLineCount("datawiwhtjoxxllh") - .withRecursive("dataunnwm"); + BlobSource model = new BlobSource().withSourceRetryCount("datamxqvv").withSourceRetryWait("dataofts") + .withMaxConcurrentConnections("datafwusfbrnjvzl").withDisableMetricsCollection("datavjemp") + .withTreatEmptyAsNull("datax").withSkipHeaderLineCount("dataa").withRecursive("dataheguxrziryxrpjr"); model = BinaryData.fromObject(model).toObject(BlobSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTests.java index 17866c969eca8..f3f0453194f07 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTests.java @@ -18,34 +18,41 @@ public final class BlobTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BlobTrigger model = BinaryData.fromString( - "{\"type\":\"BlobTrigger\",\"typeProperties\":{\"folderPath\":\"jdefsqunernbd\",\"maxConcurrency\":255555015,\"linkedService\":{\"referenceName\":\"cabchdzxjoslshwa\",\"parameters\":{\"kkihxpofv\":\"datazwmnfavllbskls\",\"orppzb\":\"datakcea\",\"vq\":\"datavawbt\"}}},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"naeclrjscdoqocdr\",\"name\":\"uhsjlroaedswhbse\"},\"parameters\":{\"lugzsvzi\":\"databoyrfqyjto\",\"dsvdbdl\":\"datakasbda\"}}],\"description\":\"utwm\",\"runtimeState\":\"Stopped\",\"annotations\":[\"datazlpgxsczrdmbebx\",\"datak\",\"dataokl\"],\"\":{\"hkbf\":\"dataenngutitjwvvvapd\",\"fytqnnbxgo\":\"dataeezzr\",\"zdphidhkigslczk\":\"dataiphlw\",\"ueaxcbfrnttlrum\":\"datalbztsgk\"}}") + "{\"type\":\"BlobTrigger\",\"typeProperties\":{\"folderPath\":\"vbgwxpuwti\",\"maxConcurrency\":1919804309,\"linkedService\":{\"referenceName\":\"ekmzubd\",\"parameters\":{\"w\":\"dataf\",\"ffaqolnfeywbpenq\":\"datalzhujcx\",\"fjdghmnycxmctnjy\":\"datazpzwwsfrpbw\"}}},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"pulwcxmxf\",\"name\":\"kmjeekbmw\"},\"parameters\":{\"wiybleaota\":\"datashtmxyrsn\"}},{\"pipelineReference\":{\"referenceName\":\"cy\",\"name\":\"xosnbwbcnf\"},\"parameters\":{\"ehdccgh\":\"datandbwwnlgaoub\"}}],\"description\":\"qvw\",\"runtimeState\":\"Stopped\",\"annotations\":[\"dataotqsp\",\"datanhlsf\"],\"\":{\"huvuadpdjovwbhei\":\"datawajsbqytfvjvm\",\"jvuwaqiomdlp\":\"datamw\",\"wowmwrn\":\"datakfslm\"}}") .toObject(BlobTrigger.class); - Assertions.assertEquals("utwm", model.description()); - Assertions.assertEquals("naeclrjscdoqocdr", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("uhsjlroaedswhbse", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals("jdefsqunernbd", model.folderPath()); - Assertions.assertEquals(255555015, model.maxConcurrency()); - Assertions.assertEquals("cabchdzxjoslshwa", model.linkedService().referenceName()); + Assertions.assertEquals("qvw", model.description()); + Assertions.assertEquals("pulwcxmxf", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("kmjeekbmw", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("vbgwxpuwti", model.folderPath()); + Assertions.assertEquals(1919804309, model.maxConcurrency()); + Assertions.assertEquals("ekmzubd", model.linkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BlobTrigger model = new BlobTrigger().withDescription("utwm") - .withAnnotations(Arrays.asList("datazlpgxsczrdmbebx", "datak", "dataokl")) - .withPipelines(Arrays.asList(new TriggerPipelineReference() - .withPipelineReference( - new PipelineReference().withReferenceName("naeclrjscdoqocdr").withName("uhsjlroaedswhbse")) - .withParameters(mapOf("lugzsvzi", "databoyrfqyjto", "dsvdbdl", "datakasbda")))) - .withFolderPath("jdefsqunernbd").withMaxConcurrency(255555015) - .withLinkedService(new LinkedServiceReference().withReferenceName("cabchdzxjoslshwa") - .withParameters(mapOf("kkihxpofv", "datazwmnfavllbskls", "orppzb", "datakcea", "vq", "datavawbt"))); + BlobTrigger model + = new BlobTrigger() + .withDescription( + "qvw") + .withAnnotations(Arrays.asList("dataotqsp", "datanhlsf")) + .withPipelines(Arrays.asList( + new TriggerPipelineReference() + .withPipelineReference( + new PipelineReference().withReferenceName("pulwcxmxf").withName("kmjeekbmw")) + .withParameters(mapOf("wiybleaota", "datashtmxyrsn")), + new TriggerPipelineReference() + .withPipelineReference(new PipelineReference().withReferenceName("cy").withName("xosnbwbcnf")) + .withParameters(mapOf("ehdccgh", "datandbwwnlgaoub")))) + .withFolderPath("vbgwxpuwti").withMaxConcurrency(1919804309) + .withLinkedService(new LinkedServiceReference().withReferenceName("ekmzubd").withParameters( + mapOf("w", "dataf", "ffaqolnfeywbpenq", "datalzhujcx", "fjdghmnycxmctnjy", "datazpzwwsfrpbw"))); model = BinaryData.fromObject(model).toObject(BlobTrigger.class); - Assertions.assertEquals("utwm", model.description()); - Assertions.assertEquals("naeclrjscdoqocdr", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("uhsjlroaedswhbse", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals("jdefsqunernbd", model.folderPath()); - Assertions.assertEquals(255555015, model.maxConcurrency()); - Assertions.assertEquals("cabchdzxjoslshwa", model.linkedService().referenceName()); + Assertions.assertEquals("qvw", model.description()); + Assertions.assertEquals("pulwcxmxf", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("kmjeekbmw", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("vbgwxpuwti", model.folderPath()); + Assertions.assertEquals(1919804309, model.maxConcurrency()); + Assertions.assertEquals("ekmzubd", model.linkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTypePropertiesTests.java index d57f82180334e..29f9d53ade079 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/BlobTriggerTypePropertiesTests.java @@ -15,22 +15,22 @@ public final class BlobTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BlobTriggerTypeProperties model = BinaryData.fromString( - "{\"folderPath\":\"ixbyedcavvjpjnn\",\"maxConcurrency\":24880351,\"linkedService\":{\"referenceName\":\"fednmqxbauzv\",\"parameters\":{\"g\":\"dataicp\",\"o\":\"datacvmuqx\"}}}") + "{\"folderPath\":\"uwgrtvyw\",\"maxConcurrency\":633624527,\"linkedService\":{\"referenceName\":\"bgwvhdbievyyp\",\"parameters\":{\"wpptessvmdox\":\"datacaschhfmidkd\"}}}") .toObject(BlobTriggerTypeProperties.class); - Assertions.assertEquals("ixbyedcavvjpjnn", model.folderPath()); - Assertions.assertEquals(24880351, model.maxConcurrency()); - Assertions.assertEquals("fednmqxbauzv", model.linkedService().referenceName()); + Assertions.assertEquals("uwgrtvyw", model.folderPath()); + Assertions.assertEquals(633624527, model.maxConcurrency()); + Assertions.assertEquals("bgwvhdbievyyp", model.linkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BlobTriggerTypeProperties model = new BlobTriggerTypeProperties().withFolderPath("ixbyedcavvjpjnn") - .withMaxConcurrency(24880351).withLinkedService(new LinkedServiceReference() - .withReferenceName("fednmqxbauzv").withParameters(mapOf("g", "dataicp", "o", "datacvmuqx"))); + BlobTriggerTypeProperties model = new BlobTriggerTypeProperties().withFolderPath("uwgrtvyw") + .withMaxConcurrency(633624527).withLinkedService(new LinkedServiceReference() + .withReferenceName("bgwvhdbievyyp").withParameters(mapOf("wpptessvmdox", "datacaschhfmidkd"))); model = BinaryData.fromObject(model).toObject(BlobTriggerTypeProperties.class); - Assertions.assertEquals("ixbyedcavvjpjnn", model.folderPath()); - Assertions.assertEquals(24880351, model.maxConcurrency()); - Assertions.assertEquals("fednmqxbauzv", model.linkedService().referenceName()); + Assertions.assertEquals("uwgrtvyw", model.folderPath()); + Assertions.assertEquals(633624527, model.maxConcurrency()); + Assertions.assertEquals("bgwvhdbievyyp", model.linkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CassandraSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CassandraSourceTests.java index 409b0f39d4471..479a42cd1988c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CassandraSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CassandraSourceTests.java @@ -13,19 +13,19 @@ public final class CassandraSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CassandraSource model = BinaryData.fromString( - "{\"type\":\"CassandraSource\",\"query\":\"databsxhivncuela\",\"consistencyLevel\":\"THREE\",\"queryTimeout\":\"datanxrtlnzdkbqfe\",\"additionalColumns\":\"datazzpmm\",\"sourceRetryCount\":\"datas\",\"sourceRetryWait\":\"dataxphwynimcp\",\"maxConcurrentConnections\":\"dataakucg\",\"disableMetricsCollection\":\"dataeoaceoaboo\",\"\":{\"mrclsxgpk\":\"datadzmtkmnyufd\",\"mtkhihixi\":\"datae\"}}") + "{\"type\":\"CassandraSource\",\"query\":\"databymnfctorqzb\",\"consistencyLevel\":\"SERIAL\",\"queryTimeout\":\"datafqqrarolc\",\"additionalColumns\":\"datahbynlbwcnn\",\"sourceRetryCount\":\"datafgstdifbyfjsl\",\"sourceRetryWait\":\"datagee\",\"maxConcurrentConnections\":\"datasoj\",\"disableMetricsCollection\":\"dataarliig\",\"\":{\"xuuqcmunh\":\"dataqvidsjhx\"}}") .toObject(CassandraSource.class); - Assertions.assertEquals(CassandraSourceReadConsistencyLevels.THREE, model.consistencyLevel()); + Assertions.assertEquals(CassandraSourceReadConsistencyLevels.SERIAL, model.consistencyLevel()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { CassandraSource model - = new CassandraSource().withSourceRetryCount("datas").withSourceRetryWait("dataxphwynimcp") - .withMaxConcurrentConnections("dataakucg").withDisableMetricsCollection("dataeoaceoaboo") - .withQueryTimeout("datanxrtlnzdkbqfe").withAdditionalColumns("datazzpmm").withQuery("databsxhivncuela") - .withConsistencyLevel(CassandraSourceReadConsistencyLevels.THREE); + = new CassandraSource().withSourceRetryCount("datafgstdifbyfjsl").withSourceRetryWait("datagee") + .withMaxConcurrentConnections("datasoj").withDisableMetricsCollection("dataarliig") + .withQueryTimeout("datafqqrarolc").withAdditionalColumns("datahbynlbwcnn").withQuery("databymnfctorqzb") + .withConsistencyLevel(CassandraSourceReadConsistencyLevels.SERIAL); model = BinaryData.fromObject(model).toObject(CassandraSource.class); - Assertions.assertEquals(CassandraSourceReadConsistencyLevels.THREE, model.consistencyLevel()); + Assertions.assertEquals(CassandraSourceReadConsistencyLevels.SERIAL, model.consistencyLevel()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTests.java index b661fa4458f58..fd9d7f78c0b63 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTests.java @@ -17,33 +17,35 @@ public final class ChainingTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ChainingTrigger model = BinaryData.fromString( - "{\"type\":\"ChainingTrigger\",\"pipeline\":{\"pipelineReference\":{\"referenceName\":\"uvuadpdj\",\"name\":\"wbheijmwaj\"},\"parameters\":{\"slmt\":\"dataaqiomdlpbk\",\"rn\":\"dataowm\"}},\"typeProperties\":{\"dependsOn\":[{\"referenceName\":\"wgrtv\",\"name\":\"bobgwvhdbie\"},{\"referenceName\":\"yyp\",\"name\":\"yc\"}],\"runDimension\":\"schhfmi\"},\"description\":\"dywpptessvmdo\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datagarykbyuukhss\"],\"\":{\"csevqtd\":\"dataugorcz\"}}") + "{\"type\":\"ChainingTrigger\",\"pipeline\":{\"pipelineReference\":{\"referenceName\":\"kwdtuwbrw\",\"name\":\"qtyuywzcc\"},\"parameters\":{\"rk\":\"dataliy\"}},\"typeProperties\":{\"dependsOn\":[{\"referenceName\":\"lvitbtloxrbotzvr\",\"name\":\"xpayjselrfqstbfu\"},{\"referenceName\":\"mln\",\"name\":\"vbeyugg\"},{\"referenceName\":\"fshntykenmjznjq\",\"name\":\"yaaevrkxyjsuapp\"}],\"runDimension\":\"mujiguusbwmsy\"},\"description\":\"bjtzd\",\"runtimeState\":\"Disabled\",\"annotations\":[\"dataqiilfovmcjchbof\"],\"\":{\"krerurcjgkau\":\"dataqvjfszvecedoptez\",\"xqsbwepduyqxvj\":\"datazbrdibdbk\",\"yaylt\":\"dataarddbn\"}}") .toObject(ChainingTrigger.class); - Assertions.assertEquals("dywpptessvmdo", model.description()); - Assertions.assertEquals("uvuadpdj", model.pipeline().pipelineReference().referenceName()); - Assertions.assertEquals("wbheijmwaj", model.pipeline().pipelineReference().name()); - Assertions.assertEquals("wgrtv", model.dependsOn().get(0).referenceName()); - Assertions.assertEquals("bobgwvhdbie", model.dependsOn().get(0).name()); - Assertions.assertEquals("schhfmi", model.runDimension()); + Assertions.assertEquals("bjtzd", model.description()); + Assertions.assertEquals("kwdtuwbrw", model.pipeline().pipelineReference().referenceName()); + Assertions.assertEquals("qtyuywzcc", model.pipeline().pipelineReference().name()); + Assertions.assertEquals("lvitbtloxrbotzvr", model.dependsOn().get(0).referenceName()); + Assertions.assertEquals("xpayjselrfqstbfu", model.dependsOn().get(0).name()); + Assertions.assertEquals("mujiguusbwmsy", model.runDimension()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ChainingTrigger model - = new ChainingTrigger().withDescription("dywpptessvmdo").withAnnotations(Arrays.asList("datagarykbyuukhss")) + = new ChainingTrigger().withDescription("bjtzd").withAnnotations(Arrays.asList("dataqiilfovmcjchbof")) .withPipeline(new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("uvuadpdj").withName("wbheijmwaj")) - .withParameters(mapOf("slmt", "dataaqiomdlpbk", "rn", "dataowm"))) - .withDependsOn(Arrays.asList(new PipelineReference().withReferenceName("wgrtv").withName("bobgwvhdbie"), - new PipelineReference().withReferenceName("yyp").withName("yc"))) - .withRunDimension("schhfmi"); + .withPipelineReference(new PipelineReference().withReferenceName("kwdtuwbrw").withName("qtyuywzcc")) + .withParameters(mapOf("rk", "dataliy"))) + .withDependsOn(Arrays.asList( + new PipelineReference().withReferenceName("lvitbtloxrbotzvr").withName("xpayjselrfqstbfu"), + new PipelineReference().withReferenceName("mln").withName("vbeyugg"), + new PipelineReference().withReferenceName("fshntykenmjznjq").withName("yaaevrkxyjsuapp"))) + .withRunDimension("mujiguusbwmsy"); model = BinaryData.fromObject(model).toObject(ChainingTrigger.class); - Assertions.assertEquals("dywpptessvmdo", model.description()); - Assertions.assertEquals("uvuadpdj", model.pipeline().pipelineReference().referenceName()); - Assertions.assertEquals("wbheijmwaj", model.pipeline().pipelineReference().name()); - Assertions.assertEquals("wgrtv", model.dependsOn().get(0).referenceName()); - Assertions.assertEquals("bobgwvhdbie", model.dependsOn().get(0).name()); - Assertions.assertEquals("schhfmi", model.runDimension()); + Assertions.assertEquals("bjtzd", model.description()); + Assertions.assertEquals("kwdtuwbrw", model.pipeline().pipelineReference().referenceName()); + Assertions.assertEquals("qtyuywzcc", model.pipeline().pipelineReference().name()); + Assertions.assertEquals("lvitbtloxrbotzvr", model.dependsOn().get(0).referenceName()); + Assertions.assertEquals("xpayjselrfqstbfu", model.dependsOn().get(0).name()); + Assertions.assertEquals("mujiguusbwmsy", model.runDimension()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTypePropertiesTests.java index 17c0c16c08375..02266f24a8aa7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChainingTriggerTypePropertiesTests.java @@ -13,24 +13,23 @@ public final class ChainingTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ChainingTriggerTypeProperties model = BinaryData.fromString( - "{\"dependsOn\":[{\"referenceName\":\"togm\",\"name\":\"lw\"},{\"referenceName\":\"zvni\",\"name\":\"npkcwwp\"},{\"referenceName\":\"kabj\",\"name\":\"ahngaczg\"}],\"runDimension\":\"f\"}") + ChainingTriggerTypeProperties model = BinaryData + .fromString( + "{\"dependsOn\":[{\"referenceName\":\"twd\",\"name\":\"tpxwgttpxb\"}],\"runDimension\":\"ihzxxnd\"}") .toObject(ChainingTriggerTypeProperties.class); - Assertions.assertEquals("togm", model.dependsOn().get(0).referenceName()); - Assertions.assertEquals("lw", model.dependsOn().get(0).name()); - Assertions.assertEquals("f", model.runDimension()); + Assertions.assertEquals("twd", model.dependsOn().get(0).referenceName()); + Assertions.assertEquals("tpxwgttpxb", model.dependsOn().get(0).name()); + Assertions.assertEquals("ihzxxnd", model.runDimension()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ChainingTriggerTypeProperties model = new ChainingTriggerTypeProperties() - .withDependsOn(Arrays.asList(new PipelineReference().withReferenceName("togm").withName("lw"), - new PipelineReference().withReferenceName("zvni").withName("npkcwwp"), - new PipelineReference().withReferenceName("kabj").withName("ahngaczg"))) - .withRunDimension("f"); + .withDependsOn(Arrays.asList(new PipelineReference().withReferenceName("twd").withName("tpxwgttpxb"))) + .withRunDimension("ihzxxnd"); model = BinaryData.fromObject(model).toObject(ChainingTriggerTypeProperties.class); - Assertions.assertEquals("togm", model.dependsOn().get(0).referenceName()); - Assertions.assertEquals("lw", model.dependsOn().get(0).name()); - Assertions.assertEquals("f", model.runDimension()); + Assertions.assertEquals("twd", model.dependsOn().get(0).referenceName()); + Assertions.assertEquals("tpxwgttpxb", model.dependsOn().get(0).name()); + Assertions.assertEquals("ihzxxnd", model.runDimension()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesCreateOrUpdateWithResponseMockTests.java index e82547c7279cf..cb44ea2e81510 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesCreateOrUpdateWithResponseMockTests.java @@ -44,7 +44,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"folder\":{\"name\":\"jbzekrwpwyiy\"},\"description\":\"rcsekhuzim\",\"sourceConnectionsInfo\":[{\"sourceEntities\":[{},{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"suqpraqjscnilpvx\"},\"linkedServiceType\":\"pewtby\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{},{}]}},{\"sourceEntities\":[{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"hrvembit\"},\"linkedServiceType\":\"axieuntceekh\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{}]}}],\"targetConnectionsInfo\":[{\"targetEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"fbfrzrqsi\"},\"linkedServiceType\":\"b\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{},{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"datactmilpztwzj\",\"databaudtppsu\",\"datakntnvgwgtgxgg\"]},{\"targetEntities\":[{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"l\"},\"linkedServiceType\":\"zqxcygevgjzrsb\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{},{},{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"datafbmwzbfpclenioz\",\"dataruqhrpwz\"]},{\"targetEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"lwalldeyorhg\"},\"linkedServiceType\":\"dvchsvrpno\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{}]},\"dataMapperMappings\":[{},{},{}],\"relationships\":[\"datas\",\"databzrfmfad\",\"datafkjdw\",\"datagyooewyvwwvk\"]},{\"targetEntities\":[{},{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"hkhszesxsyrvj\"},\"linkedServiceType\":\"pknbwh\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"dataf\",\"datashjlpznx\",\"dataymdqljngroawltab\",\"datacmrreajpy\"]}],\"policy\":{\"mode\":\"trnogfyaim\",\"recurrence\":{\"frequency\":\"Hour\",\"interval\":1551436161}},\"allowVNetOverride\":false,\"status\":\"lwubkmdusnwok\"},\"name\":\"x\",\"type\":\"auk\",\"etag\":\"zlfpkinr\",\"\":{\"lzwykkr\":\"datalwrnigztxcjnwz\",\"npgqzmgivk\":\"dataqcqg\",\"ihjlzjt\":\"datahiaaklxpwhv\"},\"id\":\"nrfuabqotyoa\"}"; + = "{\"properties\":{\"folder\":{\"name\":\"dizjd\"},\"description\":\"nehdmywnl\",\"sourceConnectionsInfo\":[{\"sourceEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"ovlufkygkykl\"},\"linkedServiceType\":\"kdblpeutahm\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{},{},{}]}},{\"sourceEntities\":[{},{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"bhiwiktglaux\"},\"linkedServiceType\":\"l\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{}]}},{\"sourceEntities\":[{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"hqgkfcspo\"},\"linkedServiceType\":\"gnbkixit\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{},{}]}},{\"sourceEntities\":[{},{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"woxtevonmq\"},\"linkedServiceType\":\"ib\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{}]}}],\"targetConnectionsInfo\":[{\"targetEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"phgaa\"},\"linkedServiceType\":\"hzg\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{},{},{}]},\"dataMapperMappings\":[{},{},{}],\"relationships\":[\"datajhfifdvxbd\",\"datallvmqyplp\",\"datayzaoqf\",\"dataashugvezulyd\"]},{\"targetEntities\":[{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"gkljukrn\"},\"linkedServiceType\":\"wccq\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{}]},\"dataMapperMappings\":[{},{},{}],\"relationships\":[\"datamez\"]}],\"policy\":{\"mode\":\"if\",\"recurrence\":{\"frequency\":\"Hour\",\"interval\":820437174}},\"allowVNetOverride\":false,\"status\":\"au\"},\"name\":\"rrznuljodvept\",\"type\":\"sncpdtgzkxttiv\",\"etag\":\"tmtwrentmj\",\"\":{\"hlgxiwtehsab\":\"dataaaetiwgbdhv\",\"dpvsauvi\":\"databfbikruvray\"},\"id\":\"svwrcbmxmbutxtt\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -62,82 +62,70 @@ public void testCreateOrUpdateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ChangeDataCaptureResource response - = manager - .changeDataCaptures().define( - "mokaqnu") - .withExistingFactory("slifcwj", "yfdjzefkzcajav") + = manager.changeDataCaptures().define("wlheukb").withExistingFactory("jjxqofpdutnkykbm", "kugzchsxsnipnl") .withSourceConnectionsInfo( Arrays.asList( new MapperSourceConnectionsInfo() - .withSourceEntities(Arrays.asList(new MapperTable(), new MapperTable(), new MapperTable())) + .withSourceEntities(Arrays.asList(new MapperTable(), new MapperTable())) .withConnection(new MapperConnection() - .withLinkedService(new LinkedServiceReference().withReferenceName("jxycdryjgx")) - .withLinkedServiceType("an").withType(ConnectionType.LINKEDSERVICETYPE) - .withIsInlineDataset(true).withCommonDslConnectorProperties(Arrays - .asList(new MapperDslConnectorProperties(), new MapperDslConnectorProperties()))), + .withLinkedService(new LinkedServiceReference().withReferenceName("d")) + .withLinkedServiceType("vesepclpzwogawz").withType(ConnectionType.LINKEDSERVICETYPE) + .withIsInlineDataset(true) + .withCommonDslConnectorProperties(Arrays.asList(new MapperDslConnectorProperties(), + new MapperDslConnectorProperties(), new MapperDslConnectorProperties(), + new MapperDslConnectorProperties()))), new MapperSourceConnectionsInfo() - .withSourceEntities(Arrays.asList(new MapperTable(), new MapperTable(), new MapperTable())) + .withSourceEntities(Arrays.asList(new MapperTable(), new MapperTable(), new MapperTable(), + new MapperTable())) .withConnection(new MapperConnection() - .withLinkedService(new LinkedServiceReference().withReferenceName("vb")) - .withLinkedServiceType("zkhp").withType(ConnectionType.LINKEDSERVICETYPE) - .withIsInlineDataset(true).withCommonDslConnectorProperties(Arrays - .asList(new MapperDslConnectorProperties()))), - new MapperSourceConnectionsInfo().withSourceEntities(Arrays.asList(new MapperTable())) - .withConnection( - new MapperConnection() - .withLinkedService(new LinkedServiceReference().withReferenceName("sbccxb")) - .withLinkedServiceType("kussldjsb").withType(ConnectionType.LINKEDSERVICETYPE) - .withIsInlineDataset(false) - .withCommonDslConnectorProperties(Arrays - .asList(new MapperDslConnectorProperties(), new MapperDslConnectorProperties(), - new MapperDslConnectorProperties()))))) - .withTargetConnectionsInfo( - Arrays.asList( - new MapperTargetConnectionsInfo() - .withTargetEntities(Arrays.asList(new MapperTable(), new MapperTable())) - .withConnection(new MapperConnection() - .withLinkedService(new LinkedServiceReference().withReferenceName("ckt")) - .withLinkedServiceType("eraqlmfyhwdog").withType(ConnectionType.LINKEDSERVICETYPE) - .withIsInlineDataset(false) - .withCommonDslConnectorProperties(Arrays.asList(new MapperDslConnectorProperties()))) - .withDataMapperMappings(Arrays.asList(new DataMapperMapping(), new DataMapperMapping(), - new DataMapperMapping(), new DataMapperMapping())) - .withRelationships(Arrays.asList("dataeyox")), - new MapperTargetConnectionsInfo() - .withTargetEntities(Arrays.asList(new MapperTable(), new MapperTable())) + .withLinkedService(new LinkedServiceReference().withReferenceName("cokrugrpj")) + .withLinkedServiceType("uxv").withType(ConnectionType.LINKEDSERVICETYPE) + .withIsInlineDataset(true) + .withCommonDslConnectorProperties(Arrays.asList(new MapperDslConnectorProperties()))), + new MapperSourceConnectionsInfo() + .withSourceEntities(Arrays.asList(new MapperTable(), new MapperTable(), new MapperTable(), + new MapperTable())) .withConnection(new MapperConnection() - .withLinkedService(new LinkedServiceReference().withReferenceName("sr")) - .withLinkedServiceType("mzswgpxhsbpb").withType(ConnectionType.LINKEDSERVICETYPE) + .withLinkedService(new LinkedServiceReference().withReferenceName("l")) + .withLinkedServiceType("c").withType(ConnectionType.LINKEDSERVICETYPE) .withIsInlineDataset(false) - .withCommonDslConnectorProperties(Arrays.asList(new MapperDslConnectorProperties()))) - .withDataMapperMappings(Arrays.asList(new DataMapperMapping(), new DataMapperMapping(), - new DataMapperMapping(), new DataMapperMapping())) - .withRelationships(Arrays.asList("datagaqyrtbniysp", "dataghnnxkouvsmmi", "datafiigslbri", - "datawknncdfc")))) - .withPolicy(new MapperPolicy().withMode("yulmxonobozg").withRecurrence( - new MapperPolicyRecurrence().withFrequency(FrequencyType.MINUTE).withInterval(337781917))) - .withFolder(new ChangeDataCaptureFolder().withName("gymbefvuutlirzr")).withDescription("lbdezvmfuhearh") - .withAllowVNetOverride(false).withStatus("pzqwcr").withIfMatch("wxhpufvucn").create(); + .withCommonDslConnectorProperties(Arrays.asList(new MapperDslConnectorProperties(), + new MapperDslConnectorProperties(), new MapperDslConnectorProperties()))))) + .withTargetConnectionsInfo(Arrays.asList(new MapperTargetConnectionsInfo() + .withTargetEntities(Arrays.asList(new MapperTable(), new MapperTable())) + .withConnection(new MapperConnection() + .withLinkedService(new LinkedServiceReference().withReferenceName("vscob")) + .withLinkedServiceType("ekrownnxqx").withType(ConnectionType.LINKEDSERVICETYPE) + .withIsInlineDataset(true) + .withCommonDslConnectorProperties(Arrays.asList(new MapperDslConnectorProperties(), + new MapperDslConnectorProperties(), new MapperDslConnectorProperties()))) + .withDataMapperMappings(Arrays.asList(new DataMapperMapping())) + .withRelationships(Arrays.asList("datainqi", "dataaklsvvggxl", "datadkiwkmbvuedio")))) + .withPolicy(new MapperPolicy().withMode("bcwvutsgmkkdydsj").withRecurrence( + new MapperPolicyRecurrence().withFrequency(FrequencyType.MINUTE).withInterval(2132778650))) + .withFolder(new ChangeDataCaptureFolder().withName("dnz")).withDescription("zathxzfqjwvcxcx") + .withAllowVNetOverride(true).withStatus("ew").withIfMatch("txjrhatpebewli").create(); - Assertions.assertEquals("nrfuabqotyoa", response.id()); - Assertions.assertEquals("jbzekrwpwyiy", response.folder().name()); - Assertions.assertEquals("rcsekhuzim", response.description()); - Assertions.assertEquals("suqpraqjscnilpvx", + Assertions.assertEquals("svwrcbmxmbutxtt", response.id()); + Assertions.assertEquals("dizjd", response.folder().name()); + Assertions.assertEquals("nehdmywnl", response.description()); + Assertions.assertEquals("ovlufkygkykl", response.sourceConnectionsInfo().get(0).connection().linkedService().referenceName()); - Assertions.assertEquals("pewtby", response.sourceConnectionsInfo().get(0).connection().linkedServiceType()); + Assertions.assertEquals("kdblpeutahm", + response.sourceConnectionsInfo().get(0).connection().linkedServiceType()); Assertions.assertEquals(ConnectionType.LINKEDSERVICETYPE, response.sourceConnectionsInfo().get(0).connection().type()); - Assertions.assertEquals(false, response.sourceConnectionsInfo().get(0).connection().isInlineDataset()); - Assertions.assertEquals("fbfrzrqsi", + Assertions.assertEquals(true, response.sourceConnectionsInfo().get(0).connection().isInlineDataset()); + Assertions.assertEquals("phgaa", response.targetConnectionsInfo().get(0).connection().linkedService().referenceName()); - Assertions.assertEquals("b", response.targetConnectionsInfo().get(0).connection().linkedServiceType()); + Assertions.assertEquals("hzg", response.targetConnectionsInfo().get(0).connection().linkedServiceType()); Assertions.assertEquals(ConnectionType.LINKEDSERVICETYPE, response.targetConnectionsInfo().get(0).connection().type()); - Assertions.assertEquals(false, response.targetConnectionsInfo().get(0).connection().isInlineDataset()); - Assertions.assertEquals("trnogfyaim", response.policy().mode()); + Assertions.assertEquals(true, response.targetConnectionsInfo().get(0).connection().isInlineDataset()); + Assertions.assertEquals("if", response.policy().mode()); Assertions.assertEquals(FrequencyType.HOUR, response.policy().recurrence().frequency()); - Assertions.assertEquals(1551436161, response.policy().recurrence().interval()); + Assertions.assertEquals(820437174, response.policy().recurrence().interval()); Assertions.assertEquals(false, response.allowVNetOverride()); - Assertions.assertEquals("lwubkmdusnwok", response.status()); + Assertions.assertEquals("au", response.status()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesDeleteWithResponseMockTests.java index 6d2baf2e5a678..f87476f006def 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.changeDataCaptures().deleteWithResponse("fnl", "jivtzshuzyyn", "vrrq", + manager.changeDataCaptures().deleteWithResponse("syizabjix", "ftl", "jesxbovnjswyj", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesGetWithResponseMockTests.java index 949aafb552c19..f78f568a9e27f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesGetWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"folder\":{\"name\":\"bpjwwviesj\"},\"description\":\"izhicxlm\",\"sourceConnectionsInfo\":[{\"sourceEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"qetmp\"},\"linkedServiceType\":\"xrwtygbqow\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{}]}}],\"targetConnectionsInfo\":[{\"targetEntities\":[{},{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"lpntpp\"},\"linkedServiceType\":\"vfnryjqboylmzw\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{},{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"datacgftbflechgiqxk\",\"datajrvrvnqidwsggjkz\",\"datalihdhfcc\",\"databhztqiaydmblpdjt\"]}],\"policy\":{\"mode\":\"ilhcca\",\"recurrence\":{\"frequency\":\"Hour\",\"interval\":1285893270}},\"allowVNetOverride\":false,\"status\":\"cakansjr\"},\"name\":\"ydcslyd\",\"type\":\"wtkce\",\"etag\":\"yngupphvo\",\"\":{\"adbuvvpd\":\"datacj\",\"vz\":\"dataolnndvvgsc\",\"khma\":\"dataudfikduw\",\"colf\":\"datag\"},\"id\":\"tzigq\"}"; + = "{\"properties\":{\"folder\":{\"name\":\"himegpgmug\"},\"description\":\"ds\",\"sourceConnectionsInfo\":[{\"sourceEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"owa\"},\"linkedServiceType\":\"gqmzxyitfbfp\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{},{},{},{}]}}],\"targetConnectionsInfo\":[{\"targetEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"ktauqav\"},\"linkedServiceType\":\"lyntvou\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"dataiquodafliefhfj\"]}],\"policy\":{\"mode\":\"msqmbdkvsmwbml\",\"recurrence\":{\"frequency\":\"Second\",\"interval\":1820521308}},\"allowVNetOverride\":false,\"status\":\"ltfxpzw\"},\"name\":\"tnkkjzjihndqz\",\"type\":\"dhuqlevty\",\"etag\":\"dehdveflkjqxsad\",\"\":{\"lnekyz\":\"datannoulmeqfyuorq\"},\"id\":\"jyzd\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,29 +50,29 @@ public void testGetWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - ChangeDataCaptureResource response = manager.changeDataCaptures().getWithResponse("esaolcebwditccu", - "jlcmzghaolfup", "hrlhtknmpj", "rcrxxkvuzpsoujc", com.azure.core.util.Context.NONE).getValue(); + ChangeDataCaptureResource response = manager.changeDataCaptures().getWithResponse("hlqpnddhzmckd", + "wmsmhnzmvruqxk", "lo", "pguckpwewepqdco", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("tzigq", response.id()); - Assertions.assertEquals("bpjwwviesj", response.folder().name()); - Assertions.assertEquals("izhicxlm", response.description()); - Assertions.assertEquals("qetmp", + Assertions.assertEquals("jyzd", response.id()); + Assertions.assertEquals("himegpgmug", response.folder().name()); + Assertions.assertEquals("ds", response.description()); + Assertions.assertEquals("owa", response.sourceConnectionsInfo().get(0).connection().linkedService().referenceName()); - Assertions.assertEquals("xrwtygbqow", response.sourceConnectionsInfo().get(0).connection().linkedServiceType()); + Assertions.assertEquals("gqmzxyitfbfp", + response.sourceConnectionsInfo().get(0).connection().linkedServiceType()); Assertions.assertEquals(ConnectionType.LINKEDSERVICETYPE, response.sourceConnectionsInfo().get(0).connection().type()); - Assertions.assertEquals(false, response.sourceConnectionsInfo().get(0).connection().isInlineDataset()); - Assertions.assertEquals("lpntpp", + Assertions.assertEquals(true, response.sourceConnectionsInfo().get(0).connection().isInlineDataset()); + Assertions.assertEquals("ktauqav", response.targetConnectionsInfo().get(0).connection().linkedService().referenceName()); - Assertions.assertEquals("vfnryjqboylmzw", - response.targetConnectionsInfo().get(0).connection().linkedServiceType()); + Assertions.assertEquals("lyntvou", response.targetConnectionsInfo().get(0).connection().linkedServiceType()); Assertions.assertEquals(ConnectionType.LINKEDSERVICETYPE, response.targetConnectionsInfo().get(0).connection().type()); - Assertions.assertEquals(true, response.targetConnectionsInfo().get(0).connection().isInlineDataset()); - Assertions.assertEquals("ilhcca", response.policy().mode()); - Assertions.assertEquals(FrequencyType.HOUR, response.policy().recurrence().frequency()); - Assertions.assertEquals(1285893270, response.policy().recurrence().interval()); + Assertions.assertEquals(false, response.targetConnectionsInfo().get(0).connection().isInlineDataset()); + Assertions.assertEquals("msqmbdkvsmwbml", response.policy().mode()); + Assertions.assertEquals(FrequencyType.SECOND, response.policy().recurrence().frequency()); + Assertions.assertEquals(1820521308, response.policy().recurrence().interval()); Assertions.assertEquals(false, response.allowVNetOverride()); - Assertions.assertEquals("cakansjr", response.status()); + Assertions.assertEquals("ltfxpzw", response.status()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesListByFactoryMockTests.java index 3b1beb986d5a3..e4da1b26022ef 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesListByFactoryMockTests.java @@ -34,7 +34,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"folder\":{\"name\":\"qeywmqhnl\"},\"description\":\"fqglvflxltngmulp\",\"sourceConnectionsInfo\":[{\"sourceEntities\":[{},{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"zwuppv\"},\"linkedServiceType\":\"vfzcy\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{}]}},{\"sourceEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"gjp\"},\"linkedServiceType\":\"ibgit\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{}]}}],\"targetConnectionsInfo\":[{\"targetEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"svuaiqqgaysy\"},\"linkedServiceType\":\"jdvtlygwxilbaz\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"datawbfslylqzwql\"]},{\"targetEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"zr\"},\"linkedServiceType\":\"gubpnoiwler\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{},{}]},\"dataMapperMappings\":[{},{}],\"relationships\":[\"datakuaam\",\"datajzrngmkeunpbt\",\"datass\"]}],\"policy\":{\"mode\":\"cg\",\"recurrence\":{\"frequency\":\"Second\",\"interval\":377703757}},\"allowVNetOverride\":true,\"status\":\"lxwxnnj\"},\"name\":\"d\",\"type\":\"enqyclnr\",\"etag\":\"rpungbhoetyvwx\",\"\":{\"w\":\"datahhzjhmxynszadgv\",\"qwsec\":\"dataxmlb\"},\"id\":\"woibqnuhrtiwnbq\"}]}"; + = "{\"value\":[{\"properties\":{\"folder\":{\"name\":\"whohwsm\"},\"description\":\"qvximgjksbpudjh\",\"sourceConnectionsInfo\":[{\"sourceEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"vceudxhnwg\"},\"linkedServiceType\":\"dxwbh\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{},{},{},{}]}}],\"targetConnectionsInfo\":[{\"targetEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"hyjex\"},\"linkedServiceType\":\"wrwu\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"databuiuspbeqs\",\"databscahrfhxrvar\",\"datatbmanqdbuf\"]},{\"targetEntities\":[{}],\"connection\":{\"linkedService\":{\"referenceName\":\"hxngqpbbybhjozn\"},\"linkedServiceType\":\"uvqnopupxbviemy\",\"type\":\"linkedservicetype\",\"isInlineDataset\":true,\"commonDslConnectorProperties\":[{}]},\"dataMapperMappings\":[{},{},{},{}],\"relationships\":[\"datazgsxqv\"]},{\"targetEntities\":[{},{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"fzyrleihkndedhm\"},\"linkedServiceType\":\"pjnvkp\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{}]},\"dataMapperMappings\":[{}],\"relationships\":[\"dataiiecig\"]},{\"targetEntities\":[{},{}],\"connection\":{\"linkedService\":{\"referenceName\":\"hzft\"},\"linkedServiceType\":\"zsacmtcevwzai\",\"type\":\"linkedservicetype\",\"isInlineDataset\":false,\"commonDslConnectorProperties\":[{},{},{},{}]},\"dataMapperMappings\":[{},{},{},{}],\"relationships\":[\"datakwwotrqhqahfz\",\"dataihgm\"]}],\"policy\":{\"mode\":\"dq\",\"recurrence\":{\"frequency\":\"Minute\",\"interval\":1802582831}},\"allowVNetOverride\":false,\"status\":\"jjiecc\"},\"name\":\"iklmcrtcscjufpc\",\"type\":\"ykyxjhqti\",\"etag\":\"osuq\",\"\":{\"ikkuhrrpx\":\"datacxwhcydnaqam\"},\"id\":\"j\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,32 +51,32 @@ public void testListByFactory() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.changeDataCaptures().listByFactory("eylvyhxlptoc", - "gjvflchhsb", com.azure.core.util.Context.NONE); + PagedIterable response = manager.changeDataCaptures().listByFactory("dszpoewfanfejk", + "utnaavtjhikcp", com.azure.core.util.Context.NONE); - Assertions.assertEquals("woibqnuhrtiwnbq", response.iterator().next().id()); - Assertions.assertEquals("qeywmqhnl", response.iterator().next().folder().name()); - Assertions.assertEquals("fqglvflxltngmulp", response.iterator().next().description()); - Assertions.assertEquals("zwuppv", + Assertions.assertEquals("j", response.iterator().next().id()); + Assertions.assertEquals("whohwsm", response.iterator().next().folder().name()); + Assertions.assertEquals("qvximgjksbpudjh", response.iterator().next().description()); + Assertions.assertEquals("vceudxhnwg", response.iterator().next().sourceConnectionsInfo().get(0).connection().linkedService().referenceName()); - Assertions.assertEquals("vfzcy", + Assertions.assertEquals("dxwbh", response.iterator().next().sourceConnectionsInfo().get(0).connection().linkedServiceType()); Assertions.assertEquals(ConnectionType.LINKEDSERVICETYPE, response.iterator().next().sourceConnectionsInfo().get(0).connection().type()); Assertions.assertEquals(true, response.iterator().next().sourceConnectionsInfo().get(0).connection().isInlineDataset()); - Assertions.assertEquals("svuaiqqgaysy", + Assertions.assertEquals("hyjex", response.iterator().next().targetConnectionsInfo().get(0).connection().linkedService().referenceName()); - Assertions.assertEquals("jdvtlygwxilbaz", + Assertions.assertEquals("wrwu", response.iterator().next().targetConnectionsInfo().get(0).connection().linkedServiceType()); Assertions.assertEquals(ConnectionType.LINKEDSERVICETYPE, response.iterator().next().targetConnectionsInfo().get(0).connection().type()); Assertions.assertEquals(false, response.iterator().next().targetConnectionsInfo().get(0).connection().isInlineDataset()); - Assertions.assertEquals("cg", response.iterator().next().policy().mode()); - Assertions.assertEquals(FrequencyType.SECOND, response.iterator().next().policy().recurrence().frequency()); - Assertions.assertEquals(377703757, response.iterator().next().policy().recurrence().interval()); - Assertions.assertEquals(true, response.iterator().next().allowVNetOverride()); - Assertions.assertEquals("lxwxnnj", response.iterator().next().status()); + Assertions.assertEquals("dq", response.iterator().next().policy().mode()); + Assertions.assertEquals(FrequencyType.MINUTE, response.iterator().next().policy().recurrence().frequency()); + Assertions.assertEquals(1802582831, response.iterator().next().policy().recurrence().interval()); + Assertions.assertEquals(false, response.iterator().next().allowVNetOverride()); + Assertions.assertEquals("jjiecc", response.iterator().next().status()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStartWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStartWithResponseMockTests.java index e4b6603e4f711..86b63f4baee77 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStartWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStartWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testStartWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.changeDataCaptures().startWithResponse("ijkvopsamtx", "nelwnod", "nytdboruj", + manager.changeDataCaptures().startWithResponse("jgwrn", "dxab", "xqpsjtipvszn", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStatusWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStatusWithResponseMockTests.java index d3a440c8ac838..2692b7af662d1 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStatusWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStatusWithResponseMockTests.java @@ -29,7 +29,7 @@ public void testStatusWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "\"ky\""; + String responseStr = "\"bthmrfytxvxurm\""; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,8 +47,8 @@ public void testStatusWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); String response = manager.changeDataCaptures() - .statusWithResponse("jottzukqsx", "aojwu", "atqzjz", com.azure.core.util.Context.NONE).getValue(); + .statusWithResponse("hcwkocdq", "zlbqkjqk", "hs", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("ky", response); + Assertions.assertEquals("bthmrfytxvxurm", response); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStopWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStopWithResponseMockTests.java index 912b4ff42deb8..71505c088e8ed 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStopWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ChangeDataCapturesStopWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testStopWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.changeDataCaptures().stopWithResponse("ltuxwfwlfqbgu", "vrwnweiwkbkhdxq", "ccdzb", + manager.changeDataCaptures().stopWithResponse("sbpueymimicjjy", "tupqtlgzuykt", "bbvbsl", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CommonDataServiceForAppsSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CommonDataServiceForAppsSourceTests.java index 1335fd4d4d7e2..36b1a4a4f2c4d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CommonDataServiceForAppsSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CommonDataServiceForAppsSourceTests.java @@ -11,16 +11,15 @@ public final class CommonDataServiceForAppsSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CommonDataServiceForAppsSource model = BinaryData.fromString( - "{\"type\":\"CommonDataServiceForAppsSource\",\"query\":\"dataeqvpedw\",\"additionalColumns\":\"dataqcjreryp\",\"sourceRetryCount\":\"datayqxeyzqnupsi\",\"sourceRetryWait\":\"datalxvaovssibnv\",\"maxConcurrentConnections\":\"datavi\",\"disableMetricsCollection\":\"datadbmzwlejiiyoon\",\"\":{\"dwcfmzmqmgdlgsxk\":\"datalrixzdbntopbab\"}}") + "{\"type\":\"CommonDataServiceForAppsSource\",\"query\":\"datayoyjptkyfrkzg\",\"additionalColumns\":\"datawyqkkd\",\"sourceRetryCount\":\"dataxdrgim\",\"sourceRetryWait\":\"dataoffyboonlovfundk\",\"maxConcurrentConnections\":\"datamyxmsbtq\",\"disableMetricsCollection\":\"datagn\",\"\":{\"dg\":\"databjxgj\",\"hzihlzljqcmmgsm\":\"datawrjuiuzlf\",\"ybdz\":\"dataujunqwkjfmt\"}}") .toObject(CommonDataServiceForAppsSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CommonDataServiceForAppsSource model = new CommonDataServiceForAppsSource() - .withSourceRetryCount("datayqxeyzqnupsi").withSourceRetryWait("datalxvaovssibnv") - .withMaxConcurrentConnections("datavi").withDisableMetricsCollection("datadbmzwlejiiyoon") - .withQuery("dataeqvpedw").withAdditionalColumns("dataqcjreryp"); + CommonDataServiceForAppsSource model = new CommonDataServiceForAppsSource().withSourceRetryCount("dataxdrgim") + .withSourceRetryWait("dataoffyboonlovfundk").withMaxConcurrentConnections("datamyxmsbtq") + .withDisableMetricsCollection("datagn").withQuery("datayoyjptkyfrkzg").withAdditionalColumns("datawyqkkd"); model = BinaryData.fromObject(model).toObject(CommonDataServiceForAppsSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CompressionReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CompressionReadSettingsTests.java index 78d32250d1393..dc1016ed46404 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CompressionReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CompressionReadSettingsTests.java @@ -12,8 +12,8 @@ public final class CompressionReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CompressionReadSettings model = BinaryData - .fromString("{\"type\":\"CompressionReadSettings\",\"\":{\"igmj\":\"dataqedft\",\"v\":\"datakt\"}}") + CompressionReadSettings model = BinaryData.fromString( + "{\"type\":\"CompressionReadSettings\",\"\":{\"rkyaovc\":\"datadvxsgdaajlhgsuq\",\"taingadkrkny\":\"datadsrxhpqlxnbd\",\"fgwbuxqzf\":\"datajngdfzqcjfqm\"}}") .toObject(CompressionReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurObjectDatasetTests.java index 3603c3dd272c9..7852fda1b8c9d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurObjectDatasetTests.java @@ -19,33 +19,36 @@ public final class ConcurObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ConcurObjectDataset model = BinaryData.fromString( - "{\"type\":\"ConcurObject\",\"typeProperties\":{\"tableName\":\"datackfkithue\"},\"description\":\"sg\",\"structure\":\"dataqzgbjw\",\"schema\":\"dataudmpwewpmioleaj\",\"linkedServiceName\":{\"referenceName\":\"b\",\"parameters\":{\"ecmbaaj\":\"datayzwphbjks\"}},\"parameters\":{\"lvzkfekde\":{\"type\":\"SecureString\",\"defaultValue\":\"datak\"},\"b\":{\"type\":\"String\",\"defaultValue\":\"datajqtl\"},\"rqnneqryp\":{\"type\":\"SecureString\",\"defaultValue\":\"datapduibsr\"},\"cxybtdzy\":{\"type\":\"Int\",\"defaultValue\":\"datavshhovtuercpzhb\"}},\"annotations\":[\"dataaoegj\",\"datagpljbnwczsraz\",\"databybicqhxhj\"],\"folder\":{\"name\":\"pasizzfmugykwuy\"},\"\":{\"thdzitjzffph\":\"datatenndz\",\"zmzxvfybxmmrvnu\":\"datarwjqvswtwonad\"}}") + "{\"type\":\"ConcurObject\",\"typeProperties\":{\"tableName\":\"datadfvjsknrbxzepirt\"},\"description\":\"piqdqbvxqto\",\"structure\":\"datawbopvhcbtza\",\"schema\":\"datajxcontickfk\",\"linkedServiceName\":{\"referenceName\":\"thueocsgvuqzgbjw\",\"parameters\":{\"abes\":\"datadmpwewpmiolea\",\"ecmbaaj\":\"datayzwphbjks\",\"zkfekdesb\":\"datafwrdkql\",\"b\":\"datajqtl\"}},\"parameters\":{\"rr\":{\"type\":\"Array\",\"defaultValue\":\"datauibs\"},\"rcpzhbwcxybtdzyc\":{\"type\":\"String\",\"defaultValue\":\"dataeqrypyurvshhovtu\"},\"wczsrazcbybic\":{\"type\":\"Bool\",\"defaultValue\":\"dataoegjzgpljb\"},\"pua\":{\"type\":\"SecureString\",\"defaultValue\":\"datah\"}},\"annotations\":[\"datai\"],\"folder\":{\"name\":\"mu\"},\"\":{\"enndzgthdzit\":\"datawuycuo\",\"vswtwonadezm\":\"datazffpherwj\"}}") .toObject(ConcurObjectDataset.class); - Assertions.assertEquals("sg", model.description()); - Assertions.assertEquals("b", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("lvzkfekde").type()); - Assertions.assertEquals("pasizzfmugykwuy", model.folder().name()); + Assertions.assertEquals("piqdqbvxqto", model.description()); + Assertions.assertEquals("thueocsgvuqzgbjw", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("rr").type()); + Assertions.assertEquals("mu", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ConcurObjectDataset model = new ConcurObjectDataset().withDescription("sg").withStructure("dataqzgbjw") - .withSchema("dataudmpwewpmioleaj") - .withLinkedServiceName( - new LinkedServiceReference().withReferenceName("b").withParameters(mapOf("ecmbaaj", "datayzwphbjks"))) - .withParameters(mapOf("lvzkfekde", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datak"), "b", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datajqtl"), "rqnneqryp", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datapduibsr"), - "cxybtdzy", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datavshhovtuercpzhb"))) - .withAnnotations(Arrays.asList("dataaoegj", "datagpljbnwczsraz", "databybicqhxhj")) - .withFolder(new DatasetFolder().withName("pasizzfmugykwuy")).withTableName("datackfkithue"); + ConcurObjectDataset model = new ConcurObjectDataset().withDescription("piqdqbvxqto") + .withStructure("datawbopvhcbtza").withSchema("datajxcontickfk") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("thueocsgvuqzgbjw") + .withParameters(mapOf("abes", "datadmpwewpmiolea", "ecmbaaj", "datayzwphbjks", "zkfekdesb", + "datafwrdkql", "b", "datajqtl"))) + .withParameters( + mapOf("rr", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datauibs"), + "rcpzhbwcxybtdzyc", + new ParameterSpecification().withType(ParameterType.STRING) + .withDefaultValue("dataeqrypyurvshhovtu"), + "wczsrazcbybic", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataoegjzgpljb"), "pua", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datah"))) + .withAnnotations(Arrays.asList("datai")).withFolder(new DatasetFolder().withName("mu")) + .withTableName("datadfvjsknrbxzepirt"); model = BinaryData.fromObject(model).toObject(ConcurObjectDataset.class); - Assertions.assertEquals("sg", model.description()); - Assertions.assertEquals("b", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("lvzkfekde").type()); - Assertions.assertEquals("pasizzfmugykwuy", model.folder().name()); + Assertions.assertEquals("piqdqbvxqto", model.description()); + Assertions.assertEquals("thueocsgvuqzgbjw", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("rr").type()); + Assertions.assertEquals("mu", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurSourceTests.java index 93d28157c587f..5994be2ab7bae 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ConcurSourceTests.java @@ -11,16 +11,15 @@ public final class ConcurSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ConcurSource model = BinaryData.fromString( - "{\"type\":\"ConcurSource\",\"query\":\"datarctbxpuis\",\"queryTimeout\":\"dataamgnpeosusi\",\"additionalColumns\":\"dataco\",\"sourceRetryCount\":\"datajabd\",\"sourceRetryWait\":\"dataalipbuqkdieuop\",\"maxConcurrentConnections\":\"dataaknhmi\",\"disableMetricsCollection\":\"dataf\",\"\":{\"oy\":\"datafmoonnria\"}}") + "{\"type\":\"ConcurSource\",\"query\":\"datadhismwjkq\",\"queryTimeout\":\"datayuygybshch\",\"additionalColumns\":\"dataeaitzgewwqwibt\",\"sourceRetryCount\":\"datahcgbzrlf\",\"sourceRetryWait\":\"datawusqupk\",\"maxConcurrentConnections\":\"datappmwozwjinxyrtec\",\"disableMetricsCollection\":\"datazslttk\",\"\":{\"mahuw\":\"datakujceeczhsdpfoa\",\"ewrryt\":\"dataodddq\",\"s\":\"datasocqkdclbzqnao\",\"cbhezau\":\"datamp\"}}") .toObject(ConcurSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ConcurSource model - = new ConcurSource().withSourceRetryCount("datajabd").withSourceRetryWait("dataalipbuqkdieuop") - .withMaxConcurrentConnections("dataaknhmi").withDisableMetricsCollection("dataf") - .withQueryTimeout("dataamgnpeosusi").withAdditionalColumns("dataco").withQuery("datarctbxpuis"); + ConcurSource model = new ConcurSource().withSourceRetryCount("datahcgbzrlf").withSourceRetryWait("datawusqupk") + .withMaxConcurrentConnections("datappmwozwjinxyrtec").withDisableMetricsCollection("datazslttk") + .withQueryTimeout("datayuygybshch").withAdditionalColumns("dataeaitzgewwqwibt").withQuery("datadhismwjkq"); model = BinaryData.fromObject(model).toObject(ConcurSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ControlActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ControlActivityTests.java index 62efd346f3762..05b88e4d2f970 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ControlActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ControlActivityTests.java @@ -20,33 +20,50 @@ public final class ControlActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ControlActivity model = BinaryData.fromString( - "{\"type\":\"Container\",\"name\":\"fmpz\",\"description\":\"erjckzxdlupgt\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"foiavpg\",\"dependencyConditions\":[\"Failed\",\"Succeeded\"],\"\":{\"xyzgrrllzxxgo\":\"datauxyitjws\",\"sjfjuphjjuek\":\"datahenqnovtmkobxvex\"}}],\"userProperties\":[{\"name\":\"tgpyoatlpsi\",\"value\":\"dataxqmmxjdk\"}],\"\":{\"k\":\"datailrlfgowvvxjqru\",\"wgmkcaya\":\"databqyfecnsqeewf\",\"zwcfdtxkurppw\":\"dataibhwybwmebm\"}}") + "{\"type\":\"Container\",\"name\":\"xzylqhewhc\",\"description\":\"ex\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"wggmitdwolfmfaz\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"jttzfswohddliikk\":\"datad\",\"gemtnbkevuuky\":\"datasqpli\"}},{\"activity\":\"iksncr\",\"dependencyConditions\":[\"Failed\"],\"\":{\"f\":\"databzqtuhga\",\"ivhyujqxyfb\":\"datayftohdlpcixpxxny\"}},{\"activity\":\"azylaya\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Skipped\"],\"\":{\"nrkmcivhww\":\"dataduwwjohg\",\"wkmrjfs\":\"datayejhwbdfcfy\",\"qowqo\":\"databcfqaqov\",\"sbjhhadndow\":\"dataispasxwiic\"}},{\"activity\":\"x\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"prygjgyov\":\"dataieehpvqfifrr\",\"g\":\"datapgqiisma\",\"kkcxc\":\"datatcoykr\",\"utyszhzlvkmirn\":\"datajkronxmtrhwwdfnc\"}}],\"userProperties\":[{\"name\":\"zarmep\",\"value\":\"datamo\"},{\"name\":\"tfpkskxsyohfrl\",\"value\":\"dataynkgnychuzhng\"},{\"name\":\"tbhjgliioeodgn\",\"value\":\"dataoewfg\"},{\"name\":\"wmm\",\"value\":\"dataixtyavvexj\"}],\"\":{\"vnoqtwqciq\":\"datakonbgegwxjgkrpp\"}}") .toObject(ControlActivity.class); - Assertions.assertEquals("fmpz", model.name()); - Assertions.assertEquals("erjckzxdlupgt", model.description()); + Assertions.assertEquals("xzylqhewhc", model.name()); + Assertions.assertEquals("ex", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("foiavpg", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("tgpyoatlpsi", model.userProperties().get(0).name()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("wggmitdwolfmfaz", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("zarmep", model.userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ControlActivity model = new ControlActivity().withName("fmpz").withDescription("erjckzxdlupgt") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("foiavpg") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("tgpyoatlpsi").withValue("dataxqmmxjdk"))); + ControlActivity model + = new ControlActivity().withName("xzylqhewhc").withDescription("ex").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("wggmitdwolfmfaz") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency() + .withActivity("iksncr").withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("azylaya") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("x") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("zarmep").withValue("datamo"), + new UserProperty().withName("tfpkskxsyohfrl").withValue("dataynkgnychuzhng"), + new UserProperty().withName("tbhjgliioeodgn").withValue("dataoewfg"), + new UserProperty().withName("wmm").withValue("dataixtyavvexj"))); model = BinaryData.fromObject(model).toObject(ControlActivity.class); - Assertions.assertEquals("fmpz", model.name()); - Assertions.assertEquals("erjckzxdlupgt", model.description()); + Assertions.assertEquals("xzylqhewhc", model.name()); + Assertions.assertEquals("ex", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("foiavpg", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("tgpyoatlpsi", model.userProperties().get(0).name()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("wggmitdwolfmfaz", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("zarmep", model.userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityLogSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityLogSettingsTests.java index cd6f98982e73b..8c009230ab0b2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityLogSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityLogSettingsTests.java @@ -11,14 +11,14 @@ public final class CopyActivityLogSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CopyActivityLogSettings model = BinaryData - .fromString("{\"logLevel\":\"dataalvchfumlf\",\"enableReliableLogging\":\"datalzxxkokipklfwnhf\"}") + .fromString("{\"logLevel\":\"datasghdovcpbwfnap\",\"enableReliableLogging\":\"datagvhsixzcdau\"}") .toObject(CopyActivityLogSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CopyActivityLogSettings model = new CopyActivityLogSettings().withLogLevel("dataalvchfumlf") - .withEnableReliableLogging("datalzxxkokipklfwnhf"); + CopyActivityLogSettings model = new CopyActivityLogSettings().withLogLevel("datasghdovcpbwfnap") + .withEnableReliableLogging("datagvhsixzcdau"); model = BinaryData.fromObject(model).toObject(CopyActivityLogSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTests.java index 5d750a9ad486a..836f063d0ade9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTests.java @@ -32,120 +32,117 @@ public final class CopyActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CopyActivity model = BinaryData.fromString( - "{\"type\":\"Copy\",\"typeProperties\":{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datamy\",\"sourceRetryWait\":\"datasgb\",\"maxConcurrentConnections\":\"dataqhei\",\"disableMetricsCollection\":\"datazebmmjaigaxwqip\",\"\":{\"heggtdv\":\"datat\",\"awhoosrso\":\"dataokxx\",\"iq\":\"datahhvmfoejb\",\"xorsdvu\":\"datajpelnudifier\"}},\"sink\":{\"type\":\"CopySink\",\"writeBatchSize\":\"dataqfksjll\",\"writeBatchTimeout\":\"datamd\",\"sinkRetryCount\":\"dataihlvrsqci\",\"sinkRetryWait\":\"datairybwgaccvwky\",\"maxConcurrentConnections\":\"datamazgtbyn\",\"disableMetricsCollection\":\"datahcawexgeqojzvu\",\"\":{\"bobuovsvwnpc\":\"datakojjp\",\"lpz\":\"datadkmtvparyubnyh\"}},\"translator\":\"datavotuc\",\"enableStaging\":\"databp\",\"stagingSettings\":{\"linkedServiceName\":{\"referenceName\":\"umqzftzoe\",\"parameters\":{\"nuqsgertxicemgs\":\"datanvnooklgr\",\"edbsl\":\"datacbbdokph\",\"k\":\"datanunpxswmcc\"}},\"path\":\"dataiaaepxlxbofdc\",\"enableCompression\":\"dataoacfskzw\",\"\":{\"j\":\"datatutqjs\",\"v\":\"dataoixtrnakytzcma\",\"kaarqhpx\":\"datal\"}},\"parallelCopies\":\"datahzsx\",\"dataIntegrationUnits\":\"datagvygmtywivbuit\",\"enableSkipIncompatibleRow\":\"datadjnklvbrsxykwbhz\",\"redirectIncompatibleRowSettings\":{\"linkedServiceName\":\"dataudjitlqxpsn\",\"path\":\"dataxhgdns\",\"\":{\"khdnpnmrxjdfkql\":\"datalndbgaicjkqjzf\"}},\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"ip\",\"parameters\":{\"ywkbiek\":\"dataquas\",\"gnapkpaiedo\":\"datavakqaho\",\"rvlgh\":\"datavoaoavezwclmzm\",\"fzfjtockgqaawy\":\"datarcdiqhvhcbukaw\"}},\"path\":\"datazwoeqljlwfq\",\"logLevel\":\"datayww\",\"enableReliableLogging\":\"dataipkhqhvktcztm\",\"\":{\"gxemvlyaprjz\":\"datahohsp\"}},\"logSettings\":{\"enableCopyActivityLog\":\"dataxqfrntz\",\"copyActivityLogSettings\":{\"logLevel\":\"dataxl\",\"enableReliableLogging\":\"datafauvgt\"},\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"inozsmyvvfa\",\"parameters\":{\"oo\":\"datamrwhknef\",\"pdd\":\"datatmd\",\"laxuybxjwny\":\"datagupiosibg\"}},\"path\":\"datakyrttnriks\"}},\"preserveRules\":[\"dataiksjpkign\",\"datatoqtui\",\"dataipbfs\"],\"preserve\":[\"datalpevz\"],\"validateDataConsistency\":\"dataydnokkkgiec\",\"skipErrorFile\":{\"fileMissing\":\"datatsneftzetjclaqxx\",\"dataInconsistency\":\"dataq\"}},\"inputs\":[{\"referenceName\":\"pdsuyywnaq\",\"parameters\":{\"zwlhufsgc\":\"dataxuhhvgddfzcny\"}},{\"referenceName\":\"wr\",\"parameters\":{\"tbiuikpotjjfe\":\"datadegfhofo\"}}],\"outputs\":[{\"referenceName\":\"kqjpovjvvxpsnr\",\"parameters\":{\"qwqcxyiqppacj\":\"databmhjmqf\"}},{\"referenceName\":\"qrllacylbtkxeijz\",\"parameters\":{\"jezr\":\"dataaaxttexaugojv\",\"awlntenhnqtvx\":\"databo\"}},{\"referenceName\":\"hbehhehotqorrvwl\",\"parameters\":{\"gjir\":\"datajlyugzl\",\"rtkevimrupg\":\"datakkrsyfds\"}}],\"linkedServiceName\":{\"referenceName\":\"jmandrvv\",\"parameters\":{\"fjlpkocrexfmqfu\":\"datalbfvtzdtw\",\"zgzscgs\":\"datalupuvwlfzjrjglac\",\"keytpmlrjnnbmods\":\"datawu\"}},\"policy\":{\"timeout\":\"datatvatuj\",\"retry\":\"dataqvfxvvog\",\"retryIntervalInSeconds\":2139387993,\"secureInput\":true,\"secureOutput\":true,\"\":{\"tsmeac\":\"dataqkkmpfnwdrmzw\",\"mgnvcusvidkzbd\":\"dataypkhnr\"}},\"name\":\"vlsn\",\"description\":\"ymrfomlh\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"smocqwe\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Completed\"],\"\":{\"iwfsqjxxbsafqiw\":\"datatmfcx\",\"g\":\"dataduotsyjzda\"}},{\"activity\":\"fzprhqomuzohnpk\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Skipped\",\"Completed\"],\"\":{\"wzienijunmgdpxei\":\"datanzmffyvowlammvaz\"}},{\"activity\":\"r\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Skipped\"],\"\":{\"cuzvbreh\":\"dataztf\",\"seiidfpwbybmxf\":\"datatqggzahngn\"}},{\"activity\":\"z\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Succeeded\"],\"\":{\"yeml\":\"datasibqilbpyjz\",\"vwr\":\"datasavzcz\"}}],\"userProperties\":[{\"name\":\"alywgqr\",\"value\":\"datavb\"},{\"name\":\"bheyxewcsk\",\"value\":\"datavkwbpr\"}],\"\":{\"ztvnmrgv\":\"datavxiikrja\"}}") + "{\"type\":\"Copy\",\"typeProperties\":{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datapq\",\"sourceRetryWait\":\"datakrmlb\",\"maxConcurrentConnections\":\"dataqogzaw\",\"disableMetricsCollection\":\"dataqdnxum\",\"\":{\"qwsyntcwrluqa\":\"datanvscyu\"}},\"sink\":{\"type\":\"CopySink\",\"writeBatchSize\":\"datalygfvbfejb\",\"writeBatchTimeout\":\"dataklgpifvpsmvksa\",\"sinkRetryCount\":\"datamsnplqfivxfqmdjz\",\"sinkRetryWait\":\"datavmkplrjkmpaxoey\",\"maxConcurrentConnections\":\"dataofaogvmqzagrqcqh\",\"disableMetricsCollection\":\"dataskmkdr\",\"\":{\"ldwcxjvexlutxcmc\":\"datapn\",\"yypvhdulds\":\"datacotqocn\"}},\"translator\":\"databzzbrufk\",\"enableStaging\":\"datalkpvaagrdf\",\"stagingSettings\":{\"linkedServiceName\":{\"referenceName\":\"glqdsphvosucryh\",\"parameters\":{\"dofshgmqxwop\":\"datathzfotfrfhrjka\",\"aittbmobrxhwpg\":\"datanitrmzvnrfkzn\",\"fgqibb\":\"datarnxrjmilogcn\",\"fwyrsfjjsoyu\":\"dataplrtxhzt\"}},\"path\":\"databuyd\",\"enableCompression\":\"datahknttk\",\"\":{\"osbijikjfjibuwh\":\"datalehenjstiwd\",\"xav\":\"datapojujpifxtg\",\"iwx\":\"datajx\"}},\"parallelCopies\":\"dataauh\",\"dataIntegrationUnits\":\"datachphovu\",\"enableSkipIncompatibleRow\":\"datasczwcxlncoh\",\"redirectIncompatibleRowSettings\":{\"linkedServiceName\":\"datafvyriawfwwsgdkbd\",\"path\":\"datas\",\"\":{\"ybuqmk\":\"datawmfcortoxsthjyyi\",\"vyilh\":\"datawdok\",\"srhxoyrgvrtcctm\":\"datasotdiox\",\"kdep\":\"dataglbplqhbrar\"}},\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"ufyqcqfouhyeyxg\",\"parameters\":{\"pwwsiooz\":\"dataochpxps\",\"v\":\"dataugbdkxlwck\",\"sgkeexsozpkvy\":\"datazkzjjtapvqjebtd\"}},\"path\":\"datatytwtfqpmpyww\",\"logLevel\":\"dataukqmjcwdo\",\"enableReliableLogging\":\"datadqun\",\"\":{\"rbn\":\"datacocchdxjrrb\",\"p\":\"dataqpsquou\"}},\"logSettings\":{\"enableCopyActivityLog\":\"datatuhdoimojcm\",\"copyActivityLogSettings\":{\"logLevel\":\"datadxwvorzhzfocgf\",\"enableReliableLogging\":\"datatornvbhuyolwifbd\"},\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"yv\",\"parameters\":{\"yu\":\"datawbbvkthrexzvejq\"}},\"path\":\"datalokeqe\"}},\"preserveRules\":[\"datap\",\"dataiehvgchsg\"],\"preserve\":[\"datawerbpobvjhunicg\",\"dataxceivvmdtk\"],\"validateDataConsistency\":\"dataqhznutrx\",\"skipErrorFile\":{\"fileMissing\":\"datatrnniarjezjhy\",\"dataInconsistency\":\"dataqf\"}},\"inputs\":[{\"referenceName\":\"esqykqfserls\",\"parameters\":{\"tagwmzgvnojgm\":\"datashh\",\"oi\":\"databkali\",\"owxsxbxd\":\"datakehpdssvlubdp\"}},{\"referenceName\":\"mixu\",\"parameters\":{\"qqa\":\"datakcqm\"}},{\"referenceName\":\"ftghmtbufkcnkgh\",\"parameters\":{\"eayodrvwnqb\":\"datairshl\",\"slfxejpocsgig\":\"dataxyofftxzovbhqel\",\"dyjwmglgstr\":\"dataabtx\",\"bcbefohnymfhmlji\":\"datafhehdzov\"}},{\"referenceName\":\"kgfvzvmtjcxi\",\"parameters\":{\"lceetuivmbugizwy\":\"datazxdb\",\"ls\":\"datafhfptbdxtvlpj\"}}],\"outputs\":[{\"referenceName\":\"uzytxeaqig\",\"parameters\":{\"tgcidbrjwi\":\"datatutet\",\"grcmcqppledx\":\"dataouepaqnfy\",\"welutrvd\":\"dataecttub\"}},{\"referenceName\":\"hwpxps\",\"parameters\":{\"fmndrdqqjkeg\":\"dataltslfccyavy\",\"erzkhiovh\":\"datamldkci\"}},{\"referenceName\":\"kwfolpjre\",\"parameters\":{\"iranxqnzssvn\":\"datav\",\"wdoq\":\"datadtqykz\",\"ryoxmfrxfxycji\":\"dataejltrnqxicyo\",\"kipklfwnhfk\":\"datalvchfumlfgmlzxxk\"}}],\"linkedServiceName\":{\"referenceName\":\"lrtffswqdkvljitb\",\"parameters\":{\"laro\":\"datalr\"}},\"policy\":{\"timeout\":\"datamucr\",\"retry\":\"databgsdxtwqqukgonl\",\"retryIntervalInSeconds\":1871793318,\"secureInput\":true,\"secureOutput\":true,\"\":{\"lupccfwqisouqy\":\"datatunrwxsqv\",\"etctjh\":\"dataxzjehdklvqtmzoci\"}},\"name\":\"gmoazsjsuevf\",\"description\":\"ntrccvxqbxgq\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"xekibv\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Failed\",\"Completed\"],\"\":{\"jthmibqgld\":\"datasvzwbktalobxld\",\"yevjealxlewlwb\":\"datattkalpqln\",\"dzowdqvqfl\":\"dataufqfdkkvijilfqv\",\"njdyoxform\":\"dataaqoaopzqpf\"}},{\"activity\":\"ee\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"xokdbvtpqttus\":\"dataeszxtesnhxp\",\"xzcnw\":\"dataxxbzmpvueijrnnwg\"}}],\"userProperties\":[{\"name\":\"zxlu\",\"value\":\"datamkwbwmgqzq\"},{\"name\":\"snycchpcjztziuu\",\"value\":\"datayvpcfvi\"},{\"name\":\"jxciu\",\"value\":\"dataetcxgd\"}],\"\":{\"lwavtzbphxxvf\":\"datale\",\"mo\":\"datajdrqgio\"}}") .toObject(CopyActivity.class); - Assertions.assertEquals("vlsn", model.name()); - Assertions.assertEquals("ymrfomlh", model.description()); + Assertions.assertEquals("gmoazsjsuevf", model.name()); + Assertions.assertEquals("ntrccvxqbxgq", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("smocqwe", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("alywgqr", model.userProperties().get(0).name()); - Assertions.assertEquals("jmandrvv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(2139387993, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("xekibv", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("zxlu", model.userProperties().get(0).name()); + Assertions.assertEquals("lrtffswqdkvljitb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1871793318, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("pdsuyywnaq", model.inputs().get(0).referenceName()); - Assertions.assertEquals("kqjpovjvvxpsnr", model.outputs().get(0).referenceName()); - Assertions.assertEquals("umqzftzoe", model.stagingSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("ip", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("inozsmyvvfa", - model.logSettings().logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("esqykqfserls", model.inputs().get(0).referenceName()); + Assertions.assertEquals("uzytxeaqig", model.outputs().get(0).referenceName()); + Assertions.assertEquals("glqdsphvosucryh", model.stagingSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ufyqcqfouhyeyxg", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("yv", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CopyActivity model = new CopyActivity().withName("vlsn").withDescription("ymrfomlh") + CopyActivity model = new CopyActivity().withName("gmoazsjsuevf").withDescription("ntrccvxqbxgq") .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("smocqwe") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("xekibv") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fzprhqomuzohnpk") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("r") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("z") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("ee") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("alywgqr").withValue("datavb"), - new UserProperty().withName("bheyxewcsk").withValue("datavkwbpr"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("jmandrvv") - .withParameters(mapOf("fjlpkocrexfmqfu", "datalbfvtzdtw", "zgzscgs", "datalupuvwlfzjrjglac", - "keytpmlrjnnbmods", "datawu"))) - .withPolicy(new ActivityPolicy().withTimeout("datatvatuj").withRetry("dataqvfxvvog") - .withRetryIntervalInSeconds(2139387993).withSecureInput(true).withSecureOutput(true) + .withUserProperties(Arrays.asList(new UserProperty().withName("zxlu").withValue("datamkwbwmgqzq"), + new UserProperty().withName("snycchpcjztziuu").withValue("datayvpcfvi"), + new UserProperty().withName("jxciu").withValue("dataetcxgd"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("lrtffswqdkvljitb") + .withParameters(mapOf("laro", "datalr"))) + .withPolicy(new ActivityPolicy().withTimeout("datamucr").withRetry("databgsdxtwqqukgonl") + .withRetryIntervalInSeconds(1871793318).withSecureInput(true).withSecureOutput(true) .withAdditionalProperties(mapOf())) .withInputs(Arrays.asList( - new DatasetReference().withReferenceName("pdsuyywnaq") - .withParameters(mapOf("zwlhufsgc", "dataxuhhvgddfzcny")), - new DatasetReference().withReferenceName("wr").withParameters(mapOf("tbiuikpotjjfe", "datadegfhofo")))) - .withOutputs(Arrays.asList( - new DatasetReference().withReferenceName("kqjpovjvvxpsnr") - .withParameters(mapOf("qwqcxyiqppacj", "databmhjmqf")), - new DatasetReference().withReferenceName("qrllacylbtkxeijz") - .withParameters(mapOf("jezr", "dataaaxttexaugojv", "awlntenhnqtvx", "databo")), - new DatasetReference().withReferenceName("hbehhehotqorrvwl") - .withParameters(mapOf("gjir", "datajlyugzl", "rtkevimrupg", "datakkrsyfds")))) - .withSource(new CopySource().withSourceRetryCount("datamy").withSourceRetryWait("datasgb") - .withMaxConcurrentConnections("dataqhei").withDisableMetricsCollection("datazebmmjaigaxwqip") + new DatasetReference().withReferenceName("esqykqfserls").withParameters( + mapOf("tagwmzgvnojgm", "datashh", "oi", "databkali", "owxsxbxd", "datakehpdssvlubdp")), + new DatasetReference().withReferenceName("mixu").withParameters(mapOf("qqa", "datakcqm")), + new DatasetReference().withReferenceName("ftghmtbufkcnkgh") + .withParameters(mapOf("eayodrvwnqb", "datairshl", "slfxejpocsgig", "dataxyofftxzovbhqel", + "dyjwmglgstr", "dataabtx", "bcbefohnymfhmlji", "datafhehdzov")), + new DatasetReference().withReferenceName("kgfvzvmtjcxi") + .withParameters(mapOf("lceetuivmbugizwy", "datazxdb", "ls", "datafhfptbdxtvlpj")))) + .withOutputs( + Arrays + .asList( + new DatasetReference().withReferenceName("uzytxeaqig") + .withParameters(mapOf("tgcidbrjwi", "datatutet", "grcmcqppledx", "dataouepaqnfy", + "welutrvd", "dataecttub")), + new DatasetReference().withReferenceName("hwpxps") + .withParameters(mapOf("fmndrdqqjkeg", "dataltslfccyavy", "erzkhiovh", "datamldkci")), + new DatasetReference().withReferenceName("kwfolpjre") + .withParameters(mapOf("iranxqnzssvn", "datav", "wdoq", "datadtqykz", "ryoxmfrxfxycji", + "dataejltrnqxicyo", "kipklfwnhfk", "datalvchfumlfgmlzxxk")))) + .withSource(new CopySource().withSourceRetryCount("datapq").withSourceRetryWait("datakrmlb") + .withMaxConcurrentConnections("dataqogzaw").withDisableMetricsCollection("dataqdnxum") .withAdditionalProperties(mapOf("type", "CopySource"))) - .withSink(new CopySink().withWriteBatchSize("dataqfksjll").withWriteBatchTimeout("datamd") - .withSinkRetryCount("dataihlvrsqci").withSinkRetryWait("datairybwgaccvwky") - .withMaxConcurrentConnections("datamazgtbyn").withDisableMetricsCollection("datahcawexgeqojzvu") + .withSink(new CopySink().withWriteBatchSize("datalygfvbfejb").withWriteBatchTimeout("dataklgpifvpsmvksa") + .withSinkRetryCount("datamsnplqfivxfqmdjz").withSinkRetryWait("datavmkplrjkmpaxoey") + .withMaxConcurrentConnections("dataofaogvmqzagrqcqh").withDisableMetricsCollection("dataskmkdr") .withAdditionalProperties(mapOf("type", "CopySink"))) - .withTranslator("datavotuc").withEnableStaging( - "databp") + .withTranslator("databzzbrufk").withEnableStaging("datalkpvaagrdf") .withStagingSettings(new StagingSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("umqzftzoe").withParameters( - mapOf("nuqsgertxicemgs", "datanvnooklgr", "edbsl", "datacbbdokph", "k", "datanunpxswmcc"))) - .withPath("dataiaaepxlxbofdc").withEnableCompression("dataoacfskzw").withAdditionalProperties(mapOf())) - .withParallelCopies("datahzsx").withDataIntegrationUnits("datagvygmtywivbuit") - .withEnableSkipIncompatibleRow("datadjnklvbrsxykwbhz") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("glqdsphvosucryh") + .withParameters(mapOf("dofshgmqxwop", "datathzfotfrfhrjka", "aittbmobrxhwpg", "datanitrmzvnrfkzn", + "fgqibb", "datarnxrjmilogcn", "fwyrsfjjsoyu", "dataplrtxhzt"))) + .withPath("databuyd").withEnableCompression("datahknttk").withAdditionalProperties(mapOf())) + .withParallelCopies("dataauh").withDataIntegrationUnits("datachphovu") + .withEnableSkipIncompatibleRow("datasczwcxlncoh") .withRedirectIncompatibleRowSettings(new RedirectIncompatibleRowSettings() - .withLinkedServiceName("dataudjitlqxpsn").withPath("dataxhgdns").withAdditionalProperties(mapOf())) + .withLinkedServiceName("datafvyriawfwwsgdkbd").withPath("datas").withAdditionalProperties(mapOf())) .withLogStorageSettings( new LogStorageSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ip") - .withParameters(mapOf("ywkbiek", "dataquas", "gnapkpaiedo", "datavakqaho", "rvlgh", - "datavoaoavezwclmzm", "fzfjtockgqaawy", "datarcdiqhvhcbukaw"))) - .withPath("datazwoeqljlwfq").withLogLevel("datayww").withEnableReliableLogging( - "dataipkhqhvktcztm") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ufyqcqfouhyeyxg") + .withParameters(mapOf("pwwsiooz", "dataochpxps", "v", "dataugbdkxlwck", "sgkeexsozpkvy", + "datazkzjjtapvqjebtd"))) + .withPath("datatytwtfqpmpyww").withLogLevel("dataukqmjcwdo").withEnableReliableLogging("datadqun") .withAdditionalProperties(mapOf())) - .withLogSettings(new LogSettings().withEnableCopyActivityLog("dataxqfrntz") - .withCopyActivityLogSettings( - new CopyActivityLogSettings().withLogLevel("dataxl").withEnableReliableLogging("datafauvgt")) - .withLogLocationSettings(new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("inozsmyvvfa") - .withParameters(mapOf("oo", "datamrwhknef", "pdd", "datatmd", "laxuybxjwny", "datagupiosibg"))) - .withPath("datakyrttnriks"))) - .withPreserveRules(Arrays.asList("dataiksjpkign", "datatoqtui", "dataipbfs")) - .withPreserve(Arrays.asList("datalpevz")).withValidateDataConsistency("dataydnokkkgiec").withSkipErrorFile( - new SkipErrorFile().withFileMissing("datatsneftzetjclaqxx").withDataInconsistency("dataq")); + .withLogSettings(new LogSettings().withEnableCopyActivityLog("datatuhdoimojcm") + .withCopyActivityLogSettings(new CopyActivityLogSettings().withLogLevel("datadxwvorzhzfocgf") + .withEnableReliableLogging("datatornvbhuyolwifbd")) + .withLogLocationSettings( + new LogLocationSettings().withLinkedServiceName(new LinkedServiceReference().withReferenceName("yv") + .withParameters(mapOf("yu", "datawbbvkthrexzvejq"))).withPath("datalokeqe"))) + .withPreserveRules(Arrays.asList("datap", "dataiehvgchsg")) + .withPreserve(Arrays.asList("datawerbpobvjhunicg", "dataxceivvmdtk")) + .withValidateDataConsistency("dataqhznutrx").withSkipErrorFile( + new SkipErrorFile().withFileMissing("datatrnniarjezjhy").withDataInconsistency("dataqf")); model = BinaryData.fromObject(model).toObject(CopyActivity.class); - Assertions.assertEquals("vlsn", model.name()); - Assertions.assertEquals("ymrfomlh", model.description()); + Assertions.assertEquals("gmoazsjsuevf", model.name()); + Assertions.assertEquals("ntrccvxqbxgq", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("smocqwe", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("alywgqr", model.userProperties().get(0).name()); - Assertions.assertEquals("jmandrvv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(2139387993, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("xekibv", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("zxlu", model.userProperties().get(0).name()); + Assertions.assertEquals("lrtffswqdkvljitb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1871793318, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("pdsuyywnaq", model.inputs().get(0).referenceName()); - Assertions.assertEquals("kqjpovjvvxpsnr", model.outputs().get(0).referenceName()); - Assertions.assertEquals("umqzftzoe", model.stagingSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("ip", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("inozsmyvvfa", - model.logSettings().logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("esqykqfserls", model.inputs().get(0).referenceName()); + Assertions.assertEquals("uzytxeaqig", model.outputs().get(0).referenceName()); + Assertions.assertEquals("glqdsphvosucryh", model.stagingSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ufyqcqfouhyeyxg", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("yv", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTypePropertiesTests.java index d981cb0863bfa..996f54baba331 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyActivityTypePropertiesTests.java @@ -25,61 +25,55 @@ public final class CopyActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CopyActivityTypeProperties model = BinaryData.fromString( - "{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datapuotmkbofuhmhk\",\"sourceRetryWait\":\"dataouzvegtnphnotr\",\"maxConcurrentConnections\":\"datayje\",\"disableMetricsCollection\":\"dataovjdunbaetsdufe\",\"\":{\"gzrzubdtzsac\":\"datavkuz\",\"kbxkzcfios\":\"datamhzpurnp\",\"d\":\"dataalbx\",\"fpqfpkrmlbkv\":\"datanluvcwuafbhxoa\"}},\"sink\":{\"type\":\"CopySink\",\"writeBatchSize\":\"datagzawfoqdnxu\",\"writeBatchTimeout\":\"datafvnvscyutqwsynt\",\"sinkRetryCount\":\"datarluqaqn\",\"sinkRetryWait\":\"datag\",\"maxConcurrentConnections\":\"datab\",\"disableMetricsCollection\":\"datajbozkl\",\"\":{\"sm\":\"datafv\",\"auwmsnplqfivx\":\"datak\"}},\"translator\":\"datamdjzgovmkplrjk\",\"enableStaging\":\"dataaxoe\",\"stagingSettings\":{\"linkedServiceName\":{\"referenceName\":\"pofaogvmq\",\"parameters\":{\"pkdpnjldwcxjvexl\":\"datarqcqhwfskmkd\",\"yypvhdulds\":\"datatxcmcccotqocn\",\"zzbr\":\"datal\",\"lkpvaagrdf\":\"datafke\"}},\"path\":\"dataglqdsphvosucryh\",\"enableCompression\":\"datahthzfotfrfhrjkah\",\"\":{\"ppnitrmzvnrfkzn\":\"datashgmqxw\",\"rnxrjmilogcn\":\"dataaittbmobrxhwpg\",\"plrtxhzt\":\"datafgqibb\"}},\"parallelCopies\":\"datawyrsfj\",\"dataIntegrationUnits\":\"dataoyusrbuydeyh\",\"enableSkipIncompatibleRow\":\"datattkdrblehenj\",\"redirectIncompatibleRowSettings\":{\"linkedServiceName\":\"dataiwdeosbijikjf\",\"path\":\"databuwhbpojuj\",\"\":{\"viwxola\":\"dataxtgrxavfj\"}},\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"rwchp\",\"parameters\":{\"xlncohywfvyria\":\"datauvrsczw\",\"bdo\":\"datafwwsgd\",\"pfwmfcorto\":\"datas\"}},\"path\":\"datathjyyirybuqmkm\",\"logLevel\":\"dataoknvyilhoso\",\"enableReliableLogging\":\"dataioxgsrhxoyrg\",\"\":{\"glbplqhbrar\":\"datacctm\",\"xufyqc\":\"datakdep\"}},\"logSettings\":{\"enableCopyActivityLog\":\"datauhyeyxgxbgochpxp\",\"copyActivityLogSettings\":{\"logLevel\":\"datawwsioozrugbdkxl\",\"enableReliableLogging\":\"datakuvlzkzjjtapv\"},\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"jebtdpsgkeexs\",\"parameters\":{\"vtytwtfqpmpywwy\":\"datakvy\",\"cdqun\":\"dataukqmjcwdo\",\"chdxjrrby\":\"datauqco\",\"uoup\":\"databnkqps\"}},\"path\":\"datagdtuhdoimojcm\"}},\"preserveRules\":[\"datadxwvorzhzfocgf\"],\"preserve\":[\"dataornvbhuyolw\",\"datafbdwyvvcywb\",\"datavkthrexzve\",\"dataqzy\"],\"validateDataConsistency\":\"datakloke\",\"skipErrorFile\":{\"fileMissing\":\"datawbpi\",\"dataInconsistency\":\"datahvg\"}}") + "{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datajmtdnymbeeysk\",\"sourceRetryWait\":\"datalcaklesjgxdhgezy\",\"maxConcurrentConnections\":\"datahaokhbqmxgglkq\",\"disableMetricsCollection\":\"datapbynetyxuxopoc\",\"\":{\"sbouhmng\":\"datafggggl\",\"vswcpspaoxigpdi\":\"datacnkgius\"}},\"sink\":{\"type\":\"CopySink\",\"writeBatchSize\":\"datappgpqsm\",\"writeBatchTimeout\":\"datautnlynplxx\",\"sinkRetryCount\":\"dataogxk\",\"sinkRetryWait\":\"databcxbgfwwcfwlwnjg\",\"maxConcurrentConnections\":\"datamopcmetd\",\"disableMetricsCollection\":\"dataugimvief\",\"\":{\"n\":\"datasmiyjkhjuuepnjuq\",\"pxqs\":\"datajb\",\"isdwtug\":\"dataaxvq\"}},\"translator\":\"dataxup\",\"enableStaging\":\"datagcbwiw\",\"stagingSettings\":{\"linkedServiceName\":{\"referenceName\":\"joxxllhkzunnw\",\"parameters\":{\"evhyuuih\":\"dataxyawxkd\",\"qmcvu\":\"datap\",\"fiiif\":\"dataekubljnizwztlcr\"}},\"path\":\"datanfarmficqrd\",\"enableCompression\":\"datavtrulzlrm\",\"\":{\"xcgcdfel\":\"datasmpmhlcxb\",\"ygosuhroicjt\":\"dataapbdbicjzntiblx\"}},\"parallelCopies\":\"datadymoanpkcmdixiux\",\"dataIntegrationUnits\":\"datacalgspz\",\"enableSkipIncompatibleRow\":\"datafprzlvhohzkcsjd\",\"redirectIncompatibleRowSettings\":{\"linkedServiceName\":\"dataposmnmkypeqm\",\"path\":\"dataeox\",\"\":{\"hw\":\"dataulavxeaymfl\",\"a\":\"datafunptsryp\",\"tbxcj\":\"datakbwbxvs\"}},\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"ad\",\"parameters\":{\"nthluze\":\"datajquliovrc\",\"reasuwe\":\"datavalezkyfykm\",\"gtyt\":\"dataq\"}},\"path\":\"datacc\",\"logLevel\":\"datafutfqffwvnjgjry\",\"enableReliableLogging\":\"datahizwdswikyewv\",\"\":{\"ip\":\"datawqzwsg\",\"gjrfkqf\":\"dataqemeft\",\"kxk\":\"datadrel\"}},\"logSettings\":{\"enableCopyActivityLog\":\"datauaregjoybnoisbm\",\"copyActivityLogSettings\":{\"logLevel\":\"dataenrcqickhvps\",\"enableReliableLogging\":\"datauiuvingmonq\"},\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"ntyuq\",\"parameters\":{\"kydqy\":\"datayojzvaykfjgakays\"}},\"path\":\"datawmfwr\"}},\"preserveRules\":[\"datazg\",\"datavmuot\",\"datasea\"],\"preserve\":[\"datajampvwxlkhprlt\"],\"validateDataConsistency\":\"datap\",\"skipErrorFile\":{\"fileMissing\":\"datarbyqh\",\"dataInconsistency\":\"dataxbo\"}}") .toObject(CopyActivityTypeProperties.class); - Assertions.assertEquals("pofaogvmq", model.stagingSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("rwchp", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("jebtdpsgkeexs", - model.logSettings().logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("joxxllhkzunnw", model.stagingSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ad", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ntyuq", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { CopyActivityTypeProperties model = new CopyActivityTypeProperties() - .withSource( - new CopySource().withSourceRetryCount("datapuotmkbofuhmhk").withSourceRetryWait("dataouzvegtnphnotr") - .withMaxConcurrentConnections("datayje").withDisableMetricsCollection("dataovjdunbaetsdufe") - .withAdditionalProperties(mapOf("type", "CopySource"))) - .withSink(new CopySink().withWriteBatchSize("datagzawfoqdnxu").withWriteBatchTimeout("datafvnvscyutqwsynt") - .withSinkRetryCount("datarluqaqn").withSinkRetryWait("datag").withMaxConcurrentConnections("datab") - .withDisableMetricsCollection("datajbozkl").withAdditionalProperties(mapOf("type", "CopySink"))) - .withTranslator("datamdjzgovmkplrjk").withEnableStaging("dataaxoe") + .withSource(new CopySource().withSourceRetryCount("datajmtdnymbeeysk") + .withSourceRetryWait("datalcaklesjgxdhgezy").withMaxConcurrentConnections("datahaokhbqmxgglkq") + .withDisableMetricsCollection("datapbynetyxuxopoc") + .withAdditionalProperties(mapOf("type", "CopySource"))) + .withSink(new CopySink().withWriteBatchSize("datappgpqsm").withWriteBatchTimeout("datautnlynplxx") + .withSinkRetryCount("dataogxk").withSinkRetryWait("databcxbgfwwcfwlwnjg") + .withMaxConcurrentConnections("datamopcmetd").withDisableMetricsCollection("dataugimvief") + .withAdditionalProperties(mapOf("type", "CopySink"))) + .withTranslator("dataxup").withEnableStaging("datagcbwiw") .withStagingSettings(new StagingSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pofaogvmq") - .withParameters(mapOf("pkdpnjldwcxjvexl", "datarqcqhwfskmkd", "yypvhdulds", "datatxcmcccotqocn", - "zzbr", "datal", "lkpvaagrdf", "datafke"))) - .withPath("dataglqdsphvosucryh").withEnableCompression("datahthzfotfrfhrjkah") - .withAdditionalProperties(mapOf())) - .withParallelCopies("datawyrsfj").withDataIntegrationUnits("dataoyusrbuydeyh") - .withEnableSkipIncompatibleRow("datattkdrblehenj") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("joxxllhkzunnw") + .withParameters(mapOf("evhyuuih", "dataxyawxkd", "qmcvu", "datap", "fiiif", "dataekubljnizwztlcr"))) + .withPath("datanfarmficqrd").withEnableCompression("datavtrulzlrm").withAdditionalProperties(mapOf())) + .withParallelCopies("datadymoanpkcmdixiux").withDataIntegrationUnits("datacalgspz") + .withEnableSkipIncompatibleRow( + "datafprzlvhohzkcsjd") .withRedirectIncompatibleRowSettings( - new RedirectIncompatibleRowSettings() - .withLinkedServiceName("dataiwdeosbijikjf").withPath("databuwhbpojuj").withAdditionalProperties( + new RedirectIncompatibleRowSettings().withLinkedServiceName("dataposmnmkypeqm").withPath( + "dataeox").withAdditionalProperties( mapOf())) .withLogStorageSettings(new LogStorageSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("rwchp") - .withParameters(mapOf("xlncohywfvyria", "datauvrsczw", "bdo", "datafwwsgd", "pfwmfcorto", "datas"))) - .withPath("datathjyyirybuqmkm").withLogLevel("dataoknvyilhoso") - .withEnableReliableLogging("dataioxgsrhxoyrg").withAdditionalProperties(mapOf())) - .withLogSettings( - new LogSettings().withEnableCopyActivityLog("datauhyeyxgxbgochpxp") - .withCopyActivityLogSettings( - new CopyActivityLogSettings().withLogLevel("datawwsioozrugbdkxl") - .withEnableReliableLogging("datakuvlzkzjjtapv")) - .withLogLocationSettings(new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("jebtdpsgkeexs") - .withParameters(mapOf("vtytwtfqpmpywwy", "datakvy", "cdqun", "dataukqmjcwdo", "chdxjrrby", - "datauqco", "uoup", "databnkqps"))) - .withPath("datagdtuhdoimojcm"))) - .withPreserveRules(Arrays.asList("datadxwvorzhzfocgf")) - .withPreserve(Arrays.asList("dataornvbhuyolw", "datafbdwyvvcywb", "datavkthrexzve", "dataqzy")) - .withValidateDataConsistency("datakloke") - .withSkipErrorFile(new SkipErrorFile().withFileMissing("datawbpi").withDataInconsistency("datahvg")); + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ad") + .withParameters(mapOf("nthluze", "datajquliovrc", "reasuwe", "datavalezkyfykm", "gtyt", "dataq"))) + .withPath("datacc").withLogLevel("datafutfqffwvnjgjry").withEnableReliableLogging( + "datahizwdswikyewv") + .withAdditionalProperties(mapOf())) + .withLogSettings(new LogSettings().withEnableCopyActivityLog("datauaregjoybnoisbm") + .withCopyActivityLogSettings(new CopyActivityLogSettings().withLogLevel("dataenrcqickhvps") + .withEnableReliableLogging("datauiuvingmonq")) + .withLogLocationSettings(new LogLocationSettings().withLinkedServiceName(new LinkedServiceReference() + .withReferenceName("ntyuq").withParameters(mapOf("kydqy", "datayojzvaykfjgakays"))) + .withPath("datawmfwr"))) + .withPreserveRules(Arrays.asList("datazg", "datavmuot", "datasea")) + .withPreserve(Arrays.asList("datajampvwxlkhprlt")).withValidateDataConsistency("datap") + .withSkipErrorFile(new SkipErrorFile().withFileMissing("datarbyqh").withDataInconsistency("dataxbo")); model = BinaryData.fromObject(model).toObject(CopyActivityTypeProperties.class); - Assertions.assertEquals("pofaogvmq", model.stagingSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("rwchp", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("jebtdpsgkeexs", - model.logSettings().logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("joxxllhkzunnw", model.stagingSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ad", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ntyuq", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyComputeScalePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyComputeScalePropertiesTests.java index 2032a8aa059c0..c71d91fa83fce 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyComputeScalePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyComputeScalePropertiesTests.java @@ -14,19 +14,19 @@ public final class CopyComputeScalePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CopyComputeScaleProperties model = BinaryData.fromString( - "{\"dataIntegrationUnit\":202074433,\"timeToLive\":738869043,\"\":{\"abfcve\":\"datalk\",\"uy\":\"databwxpmgyqshsasmr\",\"bxrgrztkyqgu\":\"datat\"}}") + "{\"dataIntegrationUnit\":557038666,\"timeToLive\":1218097718,\"\":{\"hx\":\"datab\",\"soebdltni\":\"datacszdwpaveuxgmi\",\"eilhggajfeudb\":\"dataiimerffhgvcymdd\",\"ih\":\"databmoljirchhwl\"}}") .toObject(CopyComputeScaleProperties.class); - Assertions.assertEquals(202074433, model.dataIntegrationUnit()); - Assertions.assertEquals(738869043, model.timeToLive()); + Assertions.assertEquals(557038666, model.dataIntegrationUnit()); + Assertions.assertEquals(1218097718, model.timeToLive()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CopyComputeScaleProperties model = new CopyComputeScaleProperties().withDataIntegrationUnit(202074433) - .withTimeToLive(738869043).withAdditionalProperties(mapOf()); + CopyComputeScaleProperties model = new CopyComputeScaleProperties().withDataIntegrationUnit(557038666) + .withTimeToLive(1218097718).withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(CopyComputeScaleProperties.class); - Assertions.assertEquals(202074433, model.dataIntegrationUnit()); - Assertions.assertEquals(738869043, model.timeToLive()); + Assertions.assertEquals(557038666, model.dataIntegrationUnit()); + Assertions.assertEquals(1218097718, model.timeToLive()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySinkTests.java index e57cb6abf630a..74136207f142a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySinkTests.java @@ -13,16 +13,15 @@ public final class CopySinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CopySink model = BinaryData.fromString( - "{\"type\":\"CopySink\",\"writeBatchSize\":\"dataemrhbzetss\",\"writeBatchTimeout\":\"datawexbotbrep\",\"sinkRetryCount\":\"datarlieeocyarvsfz\",\"sinkRetryWait\":\"datascootfsgilwis\",\"maxConcurrentConnections\":\"datazpzitustrtrf\",\"disableMetricsCollection\":\"datagknocshm\",\"\":{\"birbrvzhfjqpxyd\":\"dataqtu\",\"nsjssreojsgkoue\":\"dataamctzmwrhccdg\"}}") + "{\"type\":\"CopySink\",\"writeBatchSize\":\"dataofo\",\"writeBatchTimeout\":\"databiuikpotjjfec\",\"sinkRetryCount\":\"datakqjpovjvvxpsnr\",\"sinkRetryWait\":\"datar\",\"maxConcurrentConnections\":\"datah\",\"disableMetricsCollection\":\"dataqfl\",\"\":{\"ylbtkxei\":\"datacxyiqppacjiqrlla\",\"aaxttexaugojv\":\"datazgn\",\"bo\":\"datajezr\"}}") .toObject(CopySink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CopySink model = new CopySink().withWriteBatchSize("dataemrhbzetss").withWriteBatchTimeout("datawexbotbrep") - .withSinkRetryCount("datarlieeocyarvsfz").withSinkRetryWait("datascootfsgilwis") - .withMaxConcurrentConnections("datazpzitustrtrf").withDisableMetricsCollection("datagknocshm") - .withAdditionalProperties(mapOf("type", "CopySink")); + CopySink model = new CopySink().withWriteBatchSize("dataofo").withWriteBatchTimeout("databiuikpotjjfec") + .withSinkRetryCount("datakqjpovjvvxpsnr").withSinkRetryWait("datar").withMaxConcurrentConnections("datah") + .withDisableMetricsCollection("dataqfl").withAdditionalProperties(mapOf("type", "CopySink")); model = BinaryData.fromObject(model).toObject(CopySink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySourceTests.java index 61089570df9aa..59d3029314588 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopySourceTests.java @@ -13,14 +13,14 @@ public final class CopySourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CopySource model = BinaryData.fromString( - "{\"type\":\"CopySource\",\"sourceRetryCount\":\"datazcbohbb\",\"sourceRetryWait\":\"dataoderd\",\"maxConcurrentConnections\":\"datab\",\"disableMetricsCollection\":\"dataverbjctszbvh\",\"\":{\"pukn\":\"dataorvpcoxdl\",\"lnmjkydhb\":\"datamnp\",\"dzri\":\"datafivoz\",\"ozo\":\"datawiucvvrkxpbj\"}}") + "{\"type\":\"CopySource\",\"sourceRetryCount\":\"dataqww\",\"sourceRetryWait\":\"dataqjyiwuveryavb\",\"maxConcurrentConnections\":\"datavcogupshoofasky\",\"disableMetricsCollection\":\"datadjoorbuuhbcck\",\"\":{\"kquhmblnismviaa\":\"datatelmdrmasvghphl\",\"povzespdip\":\"datadexsrglxljuyvk\"}}") .toObject(CopySource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CopySource model = new CopySource().withSourceRetryCount("datazcbohbb").withSourceRetryWait("dataoderd") - .withMaxConcurrentConnections("datab").withDisableMetricsCollection("dataverbjctszbvh") + CopySource model = new CopySource().withSourceRetryCount("dataqww").withSourceRetryWait("dataqjyiwuveryavb") + .withMaxConcurrentConnections("datavcogupshoofasky").withDisableMetricsCollection("datadjoorbuuhbcck") .withAdditionalProperties(mapOf("type", "CopySource")); model = BinaryData.fromObject(model).toObject(CopySource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyTranslatorTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyTranslatorTests.java index 7a00e356df780..9c9a4b5e3efa8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyTranslatorTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CopyTranslatorTests.java @@ -12,8 +12,7 @@ public final class CopyTranslatorTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CopyTranslator model = BinaryData.fromString( - "{\"type\":\"CopyTranslator\",\"\":{\"rwl\":\"datakgtkojrruhzvve\",\"cczkggbmzdnyrmo\":\"datahsnlmdosiyzf\",\"umckcbsakoucss\":\"datamaekc\"}}") + CopyTranslator model = BinaryData.fromString("{\"type\":\"CopyTranslator\",\"\":{\"mszxyfaidz\":\"datalx\"}}") .toObject(CopyTranslator.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTests.java index cfb3ea40db034..fc131b86e0563 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTests.java @@ -18,32 +18,28 @@ public final class CosmosDbMongoDbApiLinkedServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CosmosDbMongoDbApiLinkedService model = BinaryData.fromString( - "{\"type\":\"CosmosDbMongoDbApi\",\"typeProperties\":{\"isServerVersionAbove32\":\"dataalkqlvbkz\",\"connectionString\":\"dataiyrnwwoivin\",\"database\":\"datayo\"},\"connectVia\":{\"referenceName\":\"uwwzcr\",\"parameters\":{\"ajsiueaibqj\":\"datajbelnqalbsoq\",\"xtclveqdqtzsh\":\"dataizbwxuypcuriwl\",\"cmqdlkkwbdek\":\"datalwjxrymiysgh\"}},\"description\":\"dulmkaadnxb\",\"parameters\":{\"xxccf\":{\"type\":\"SecureString\",\"defaultValue\":\"dataialrqhw\"},\"tpzriknoscl\":{\"type\":\"Int\",\"defaultValue\":\"dataozaiyvrex\"},\"wbncl\":{\"type\":\"Int\",\"defaultValue\":\"datawwumbusn\"},\"lrmmu\":{\"type\":\"String\",\"defaultValue\":\"datamsobuahoki\"}},\"annotations\":[\"datawtbscidkwznwt\"],\"\":{\"ogdjwuybcppdvuo\":\"datazdoysmzg\",\"kcwauylkbdskrp\":\"datakvkbpmkupb\",\"zylemp\":\"datafrtlukaf\",\"npw\":\"datahyuxxeike\"}}") + "{\"type\":\"CosmosDbMongoDbApi\",\"typeProperties\":{\"isServerVersionAbove32\":\"dataqhedbqrl\",\"connectionString\":\"datayhzyf\",\"database\":\"datau\"},\"connectVia\":{\"referenceName\":\"krzvd\",\"parameters\":{\"wx\":\"dataevbkkgqfjwgphh\",\"fqzwysmsqqmdajsq\":\"datahw\"}},\"description\":\"xftyifadsliifdr\",\"parameters\":{\"rfvq\":{\"type\":\"Float\",\"defaultValue\":\"dataspbibsnxmfooinp\"},\"cbrewtf\":{\"type\":\"Int\",\"defaultValue\":\"datawgkm\"}},\"annotations\":[\"datatsxnikcgitt\"],\"\":{\"invfcdsijsinybn\":\"datapihtepasjeb\",\"ayxujzoxgakqt\":\"datalx\",\"mah\":\"datarjucazwe\"}}") .toObject(CosmosDbMongoDbApiLinkedService.class); - Assertions.assertEquals("uwwzcr", model.connectVia().referenceName()); - Assertions.assertEquals("dulmkaadnxb", model.description()); - Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("xxccf").type()); + Assertions.assertEquals("krzvd", model.connectVia().referenceName()); + Assertions.assertEquals("xftyifadsliifdr", model.description()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("rfvq").type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { CosmosDbMongoDbApiLinkedService model = new CosmosDbMongoDbApiLinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("uwwzcr") - .withParameters(mapOf("ajsiueaibqj", "datajbelnqalbsoq", "xtclveqdqtzsh", "dataizbwxuypcuriwl", - "cmqdlkkwbdek", "datalwjxrymiysgh"))) - .withDescription("dulmkaadnxb") - .withParameters(mapOf("xxccf", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("dataialrqhw"), - "tpzriknoscl", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataozaiyvrex"), "wbncl", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datawwumbusn"), "lrmmu", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datamsobuahoki"))) - .withAnnotations(Arrays.asList("datawtbscidkwznwt")).withIsServerVersionAbove32("dataalkqlvbkz") - .withConnectionString("dataiyrnwwoivin").withDatabase("datayo"); + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("krzvd") + .withParameters(mapOf("wx", "dataevbkkgqfjwgphh", "fqzwysmsqqmdajsq", "datahw"))) + .withDescription("xftyifadsliifdr") + .withParameters(mapOf("rfvq", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataspbibsnxmfooinp"), + "cbrewtf", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datawgkm"))) + .withAnnotations(Arrays.asList("datatsxnikcgitt")).withIsServerVersionAbove32("dataqhedbqrl") + .withConnectionString("datayhzyf").withDatabase("datau"); model = BinaryData.fromObject(model).toObject(CosmosDbMongoDbApiLinkedService.class); - Assertions.assertEquals("uwwzcr", model.connectVia().referenceName()); - Assertions.assertEquals("dulmkaadnxb", model.description()); - Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("xxccf").type()); + Assertions.assertEquals("krzvd", model.connectVia().referenceName()); + Assertions.assertEquals("xftyifadsliifdr", model.description()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("rfvq").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTypePropertiesTests.java index 4a80ea9dca7bd..ca4d87111afdd 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiLinkedServiceTypePropertiesTests.java @@ -11,15 +11,14 @@ public final class CosmosDbMongoDbApiLinkedServiceTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CosmosDbMongoDbApiLinkedServiceTypeProperties model = BinaryData.fromString( - "{\"isServerVersionAbove32\":\"datahqafuvvysphjkxgf\",\"connectionString\":\"dataesxjeqqfynvck\",\"database\":\"datatanlvoorvzhd\"}") + "{\"isServerVersionAbove32\":\"dataxgce\",\"connectionString\":\"datax\",\"database\":\"dataanudvqannenxg\"}") .toObject(CosmosDbMongoDbApiLinkedServiceTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CosmosDbMongoDbApiLinkedServiceTypeProperties model - = new CosmosDbMongoDbApiLinkedServiceTypeProperties().withIsServerVersionAbove32("datahqafuvvysphjkxgf") - .withConnectionString("dataesxjeqqfynvck").withDatabase("datatanlvoorvzhd"); + CosmosDbMongoDbApiLinkedServiceTypeProperties model = new CosmosDbMongoDbApiLinkedServiceTypeProperties() + .withIsServerVersionAbove32("dataxgce").withConnectionString("datax").withDatabase("dataanudvqannenxg"); model = BinaryData.fromObject(model).toObject(CosmosDbMongoDbApiLinkedServiceTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSinkTests.java index 926732007ad09..7e1f092bc3b45 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSinkTests.java @@ -11,16 +11,16 @@ public final class CosmosDbMongoDbApiSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CosmosDbMongoDbApiSink model = BinaryData.fromString( - "{\"type\":\"CosmosDbMongoDbApiSink\",\"writeBehavior\":\"datamgvm\",\"writeBatchSize\":\"datamjazqlmigkx\",\"writeBatchTimeout\":\"datashadnholkoyxms\",\"sinkRetryCount\":\"datadsqwvzunrqvu\",\"sinkRetryWait\":\"dataxs\",\"maxConcurrentConnections\":\"dataqzdfjwo\",\"disableMetricsCollection\":\"datazifrmoftilhoyemh\",\"\":{\"rcdtkv\":\"datapgdd\",\"ubmhsxtry\":\"datarifcqmfv\"}}") + "{\"type\":\"CosmosDbMongoDbApiSink\",\"writeBehavior\":\"dataxvjjwlwysrs\",\"writeBatchSize\":\"datahciazwebts\",\"writeBatchTimeout\":\"dataqkanuxjud\",\"sinkRetryCount\":\"datazodnxlcdgkc\",\"sinkRetryWait\":\"dataancjlkrskzw\",\"maxConcurrentConnections\":\"databafqzihmvw\",\"disableMetricsCollection\":\"datajwvqiahoqjz\",\"\":{\"hgwzbystwuuwe\":\"datawdlrtcfulmz\",\"qichzcajity\":\"datantjssjbpnatpym\"}}") .toObject(CosmosDbMongoDbApiSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CosmosDbMongoDbApiSink model = new CosmosDbMongoDbApiSink().withWriteBatchSize("datamjazqlmigkx") - .withWriteBatchTimeout("datashadnholkoyxms").withSinkRetryCount("datadsqwvzunrqvu") - .withSinkRetryWait("dataxs").withMaxConcurrentConnections("dataqzdfjwo") - .withDisableMetricsCollection("datazifrmoftilhoyemh").withWriteBehavior("datamgvm"); + CosmosDbMongoDbApiSink model = new CosmosDbMongoDbApiSink().withWriteBatchSize("datahciazwebts") + .withWriteBatchTimeout("dataqkanuxjud").withSinkRetryCount("datazodnxlcdgkc") + .withSinkRetryWait("dataancjlkrskzw").withMaxConcurrentConnections("databafqzihmvw") + .withDisableMetricsCollection("datajwvqiahoqjz").withWriteBehavior("dataxvjjwlwysrs"); model = BinaryData.fromObject(model).toObject(CosmosDbMongoDbApiSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSourceTests.java index 47b5432d7bea2..3e447950a1c2f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbMongoDbApiSourceTests.java @@ -14,20 +14,19 @@ public final class CosmosDbMongoDbApiSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CosmosDbMongoDbApiSource model = BinaryData.fromString( - "{\"type\":\"CosmosDbMongoDbApiSource\",\"filter\":\"datafuhfa\",\"cursorMethods\":{\"project\":\"databvslocdkpvvkqlk\",\"sort\":\"dataxnzjzashhiz\",\"skip\":\"datamibwzuhyda\",\"limit\":\"datakgwtbfxxsfj\",\"\":{\"mkdsvayyhti\":\"datascjig\",\"tlnmmyznwrc\":\"dataxehmnizoybtehk\",\"ohwi\":\"dataqwkqulkzov\"}},\"batchSize\":\"datarqxjxlsso\",\"queryTimeout\":\"datadnypxnsjjj\",\"additionalColumns\":\"datadsvgdbfniqxbc\",\"sourceRetryCount\":\"datahpma\",\"sourceRetryWait\":\"dataczpd\",\"maxConcurrentConnections\":\"dataddtjylimzvjwjhmt\",\"disableMetricsCollection\":\"datacwmabehr\",\"\":{\"vhzrxcae\":\"datakzwtgh\"}}") + "{\"type\":\"CosmosDbMongoDbApiSource\",\"filter\":\"dataeztmdyb\",\"cursorMethods\":{\"project\":\"dataj\",\"sort\":\"dataogtnfla\",\"skip\":\"datapghfvkqijmyqo\",\"limit\":\"datasfaoc\",\"\":{\"dpyohnmru\":\"datarr\",\"eywbhxhawkwcf\":\"datavlwhtfscoups\"}},\"batchSize\":\"dataqexd\",\"queryTimeout\":\"datacvkwwjjotfun\",\"additionalColumns\":\"datauejxvrwalekqedof\",\"sourceRetryCount\":\"databxmlai\",\"sourceRetryWait\":\"datavhlpfjibblm\",\"maxConcurrentConnections\":\"datavzdaycmene\",\"disableMetricsCollection\":\"datayzlslvgqle\",\"\":{\"t\":\"datawbbellcjd\",\"dpmy\":\"datacvddfmflwfxdkp\"}}") .toObject(CosmosDbMongoDbApiSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CosmosDbMongoDbApiSource model = new CosmosDbMongoDbApiSource().withSourceRetryCount("datahpma") - .withSourceRetryWait("dataczpd").withMaxConcurrentConnections("dataddtjylimzvjwjhmt") - .withDisableMetricsCollection("datacwmabehr").withFilter("datafuhfa") - .withCursorMethods( - new MongoDbCursorMethodsProperties().withProject("databvslocdkpvvkqlk").withSort("dataxnzjzashhiz") - .withSkip("datamibwzuhyda").withLimit("datakgwtbfxxsfj").withAdditionalProperties(mapOf())) - .withBatchSize("datarqxjxlsso").withQueryTimeout("datadnypxnsjjj") - .withAdditionalColumns("datadsvgdbfniqxbc"); + CosmosDbMongoDbApiSource model = new CosmosDbMongoDbApiSource().withSourceRetryCount("databxmlai") + .withSourceRetryWait("datavhlpfjibblm").withMaxConcurrentConnections("datavzdaycmene") + .withDisableMetricsCollection("datayzlslvgqle").withFilter("dataeztmdyb") + .withCursorMethods(new MongoDbCursorMethodsProperties().withProject("dataj").withSort("dataogtnfla") + .withSkip("datapghfvkqijmyqo").withLimit("datasfaoc").withAdditionalProperties(mapOf())) + .withBatchSize("dataqexd").withQueryTimeout("datacvkwwjjotfun") + .withAdditionalColumns("datauejxvrwalekqedof"); model = BinaryData.fromObject(model).toObject(CosmosDbMongoDbApiSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSinkTests.java index 9cfe028a405b3..840ae2de970b7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSinkTests.java @@ -11,16 +11,16 @@ public final class CosmosDbSqlApiSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CosmosDbSqlApiSink model = BinaryData.fromString( - "{\"type\":\"CosmosDbSqlApiSink\",\"writeBehavior\":\"datall\",\"writeBatchSize\":\"datavhthxcrwee\",\"writeBatchTimeout\":\"datadmpfmcrcelsnj\",\"sinkRetryCount\":\"datanfdcjtveibntwi\",\"sinkRetryWait\":\"datagwxysut\",\"maxConcurrentConnections\":\"datafdhrifekstrms\",\"disableMetricsCollection\":\"datadgrzkeuplorn\",\"\":{\"zcdlnvupi\":\"datasmaa\",\"elv\":\"datacbzyhtbjyycac\"}}") + "{\"type\":\"CosmosDbSqlApiSink\",\"writeBehavior\":\"datauciwbiwygwpwqu\",\"writeBatchSize\":\"dataqgslspihuxgvvio\",\"writeBatchTimeout\":\"dataoolkmfi\",\"sinkRetryCount\":\"datafbbrndaquxvufr\",\"sinkRetryWait\":\"dataaehssosowav\",\"maxConcurrentConnections\":\"datasieyeblkgupgnst\",\"disableMetricsCollection\":\"datajxfmh\",\"\":{\"qgtjffc\":\"datartvkhufk\",\"tzgk\":\"datare\"}}") .toObject(CosmosDbSqlApiSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CosmosDbSqlApiSink model = new CosmosDbSqlApiSink().withWriteBatchSize("datavhthxcrwee") - .withWriteBatchTimeout("datadmpfmcrcelsnj").withSinkRetryCount("datanfdcjtveibntwi") - .withSinkRetryWait("datagwxysut").withMaxConcurrentConnections("datafdhrifekstrms") - .withDisableMetricsCollection("datadgrzkeuplorn").withWriteBehavior("datall"); + CosmosDbSqlApiSink model = new CosmosDbSqlApiSink().withWriteBatchSize("dataqgslspihuxgvvio") + .withWriteBatchTimeout("dataoolkmfi").withSinkRetryCount("datafbbrndaquxvufr") + .withSinkRetryWait("dataaehssosowav").withMaxConcurrentConnections("datasieyeblkgupgnst") + .withDisableMetricsCollection("datajxfmh").withWriteBehavior("datauciwbiwygwpwqu"); model = BinaryData.fromObject(model).toObject(CosmosDbSqlApiSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSourceTests.java index 24a0992b29d14..fd09a5706641b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CosmosDbSqlApiSourceTests.java @@ -11,17 +11,17 @@ public final class CosmosDbSqlApiSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CosmosDbSqlApiSource model = BinaryData.fromString( - "{\"type\":\"CosmosDbSqlApiSource\",\"query\":\"dataad\",\"pageSize\":\"datamjquliovrcnn\",\"preferredRegions\":\"dataluzeyvalezkyfy\",\"detectDatetime\":\"datanreasuwepqegty\",\"additionalColumns\":\"datayc\",\"sourceRetryCount\":\"dataufutfqffw\",\"sourceRetryWait\":\"datajgjrykshiz\",\"maxConcurrentConnections\":\"datasw\",\"disableMetricsCollection\":\"dataye\",\"\":{\"meftlgjrfkqf\":\"datakzwqzwsguipqq\"}}") + "{\"type\":\"CosmosDbSqlApiSource\",\"query\":\"dataplqnilozf\",\"pageSize\":\"datavsf\",\"preferredRegions\":\"datacarfdmlie\",\"detectDatetime\":\"datawocufcshqfc\",\"additionalColumns\":\"datanxfof\",\"sourceRetryCount\":\"datadroqktegi\",\"sourceRetryWait\":\"datakzctqbvntlzvgjme\",\"maxConcurrentConnections\":\"dataoydyislepd\",\"disableMetricsCollection\":\"dataiklnt\",\"\":{\"ytbjbm\":\"datanump\"}}") .toObject(CosmosDbSqlApiSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CosmosDbSqlApiSource model - = new CosmosDbSqlApiSource().withSourceRetryCount("dataufutfqffw").withSourceRetryWait("datajgjrykshiz") - .withMaxConcurrentConnections("datasw").withDisableMetricsCollection("dataye").withQuery("dataad") - .withPageSize("datamjquliovrcnn").withPreferredRegions("dataluzeyvalezkyfy") - .withDetectDatetime("datanreasuwepqegty").withAdditionalColumns("datayc"); + CosmosDbSqlApiSource model = new CosmosDbSqlApiSource().withSourceRetryCount("datadroqktegi") + .withSourceRetryWait("datakzctqbvntlzvgjme").withMaxConcurrentConnections("dataoydyislepd") + .withDisableMetricsCollection("dataiklnt").withQuery("dataplqnilozf").withPageSize("datavsf") + .withPreferredRegions("datacarfdmlie").withDetectDatetime("datawocufcshqfc") + .withAdditionalColumns("datanxfof"); model = BinaryData.fromObject(model).toObject(CosmosDbSqlApiSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseSourceTests.java index 475f7b2f71d82..2960a8ad173c5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseSourceTests.java @@ -11,16 +11,16 @@ public final class CouchbaseSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CouchbaseSource model = BinaryData.fromString( - "{\"type\":\"CouchbaseSource\",\"query\":\"datakdbmjzobcdvbbuu\",\"queryTimeout\":\"dataelokp\",\"additionalColumns\":\"dataeojxhwgjahxgr\",\"sourceRetryCount\":\"datajgkxvkjd\",\"sourceRetryWait\":\"datalnq\",\"maxConcurrentConnections\":\"datawwtz\",\"disableMetricsCollection\":\"datavwqiwg\",\"\":{\"too\":\"datahubgaaaxigafah\"}}") + "{\"type\":\"CouchbaseSource\",\"query\":\"dataaysxhfupvqjkqlaf\",\"queryTimeout\":\"dataywmcipuye\",\"additionalColumns\":\"datahd\",\"sourceRetryCount\":\"datagaifg\",\"sourceRetryWait\":\"datakgqwmp\",\"maxConcurrentConnections\":\"dataxpcxqcbnkxhcodh\",\"disableMetricsCollection\":\"databxllfwxdou\",\"\":{\"ofhk\":\"datapaqjahjxgedtmz\",\"rfassiii\":\"dataywtacgukierd\",\"ayyxgcgb\":\"datacmrgahs\",\"vqopxun\":\"dataieqonsbukznxd\"}}") .toObject(CouchbaseSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CouchbaseSource model = new CouchbaseSource().withSourceRetryCount("datajgkxvkjd") - .withSourceRetryWait("datalnq").withMaxConcurrentConnections("datawwtz") - .withDisableMetricsCollection("datavwqiwg").withQueryTimeout("dataelokp") - .withAdditionalColumns("dataeojxhwgjahxgr").withQuery("datakdbmjzobcdvbbuu"); + CouchbaseSource model + = new CouchbaseSource().withSourceRetryCount("datagaifg").withSourceRetryWait("datakgqwmp") + .withMaxConcurrentConnections("dataxpcxqcbnkxhcodh").withDisableMetricsCollection("databxllfwxdou") + .withQueryTimeout("dataywmcipuye").withAdditionalColumns("datahd").withQuery("dataaysxhfupvqjkqlaf"); model = BinaryData.fromObject(model).toObject(CouchbaseSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseTableDatasetTests.java index dddc066b708e4..45552c62752f8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CouchbaseTableDatasetTests.java @@ -19,30 +19,31 @@ public final class CouchbaseTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CouchbaseTableDataset model = BinaryData.fromString( - "{\"type\":\"CouchbaseTable\",\"typeProperties\":{\"tableName\":\"datarrsguogk\"},\"description\":\"rotpyabensjflw\",\"structure\":\"datatvvqtmvifgcvsim\",\"schema\":\"databmticxgosnxajp\",\"linkedServiceName\":{\"referenceName\":\"cdfmzxaoxlhmvjc\",\"parameters\":{\"xh\":\"datasbnuc\",\"nkleldk\":\"dataaqoqbvejoysoxovl\",\"qrykkxakruupti\":\"datadlqqhn\"}},\"parameters\":{\"tjekxsnnb\":{\"type\":\"Object\",\"defaultValue\":\"datazgyxccnpxiemacm\"},\"mocnqbbl\":{\"type\":\"Int\",\"defaultValue\":\"datagkt\"}},\"annotations\":[\"dataofzghfuifwxu\",\"dataynohocqxug\"],\"folder\":{\"name\":\"gdcrrfbpl\"},\"\":{\"qe\":\"datahurosdjlzbdmddg\",\"orservpvesors\":\"datay\",\"zydyvtuqvir\":\"dataegclmexafjqzy\",\"igtvjxsocsvjekej\":\"dataunssky\"}}") + "{\"type\":\"CouchbaseTable\",\"typeProperties\":{\"tableName\":\"datafybxmmrvn\"},\"description\":\"qkrrsguogkcb\",\"structure\":\"datatpyabensjflwp\",\"schema\":\"datavvqtmvif\",\"linkedServiceName\":{\"referenceName\":\"cv\",\"parameters\":{\"t\":\"dataalb\"}},\"parameters\":{\"hmvjcnnlsb\":{\"type\":\"Int\",\"defaultValue\":\"datasnxajptcdfmzxaox\"},\"ovlznklel\":{\"type\":\"Float\",\"defaultValue\":\"dataqxhpaqoqbvejoyso\"}},\"annotations\":[\"datadlqqhn\",\"dataqrykkxakruupti\",\"datacg\",\"datapz\"],\"folder\":{\"name\":\"ccnpxiemacmzt\"},\"\":{\"ocnqbblr\":\"dataxsnnbrysgktf\",\"fwxud\":\"databofzghfu\",\"cqxu\":\"datanoh\"}}") .toObject(CouchbaseTableDataset.class); - Assertions.assertEquals("rotpyabensjflw", model.description()); - Assertions.assertEquals("cdfmzxaoxlhmvjc", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("tjekxsnnb").type()); - Assertions.assertEquals("gdcrrfbpl", model.folder().name()); + Assertions.assertEquals("qkrrsguogkcb", model.description()); + Assertions.assertEquals("cv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("hmvjcnnlsb").type()); + Assertions.assertEquals("ccnpxiemacmzt", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CouchbaseTableDataset model = new CouchbaseTableDataset().withDescription("rotpyabensjflw") - .withStructure("datatvvqtmvifgcvsim").withSchema("databmticxgosnxajp") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cdfmzxaoxlhmvjc").withParameters( - mapOf("xh", "datasbnuc", "nkleldk", "dataaqoqbvejoysoxovl", "qrykkxakruupti", "datadlqqhn"))) - .withParameters(mapOf("tjekxsnnb", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datazgyxccnpxiemacm"), - "mocnqbbl", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datagkt"))) - .withAnnotations(Arrays.asList("dataofzghfuifwxu", "dataynohocqxug")) - .withFolder(new DatasetFolder().withName("gdcrrfbpl")).withTableName("datarrsguogk"); + CouchbaseTableDataset model = new CouchbaseTableDataset().withDescription("qkrrsguogkcb") + .withStructure("datatpyabensjflwp").withSchema("datavvqtmvif") + .withLinkedServiceName( + new LinkedServiceReference().withReferenceName("cv").withParameters(mapOf("t", "dataalb"))) + .withParameters(mapOf("hmvjcnnlsb", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datasnxajptcdfmzxaox"), + "ovlznklel", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataqxhpaqoqbvejoyso"))) + .withAnnotations(Arrays.asList("datadlqqhn", "dataqrykkxakruupti", "datacg", "datapz")) + .withFolder(new DatasetFolder().withName("ccnpxiemacmzt")).withTableName("datafybxmmrvn"); model = BinaryData.fromObject(model).toObject(CouchbaseTableDataset.class); - Assertions.assertEquals("rotpyabensjflw", model.description()); - Assertions.assertEquals("cdfmzxaoxlhmvjc", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("tjekxsnnb").type()); - Assertions.assertEquals("gdcrrfbpl", model.folder().name()); + Assertions.assertEquals("qkrrsguogkcb", model.description()); + Assertions.assertEquals("cv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("hmvjcnnlsb").type()); + Assertions.assertEquals("ccnpxiemacmzt", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsCreateOrUpdateWithResponseMockTests.java index 5868b4683addb..7dd2792ecf3c9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsCreateOrUpdateWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"ManagedIdentity\",\"typeProperties\":{\"resourceId\":\"yscnbrwh\"},\"description\":\"tzgmfybr\",\"annotations\":[\"datahdigmgszzhxv\",\"dataunuus\",\"datatzfvzlqspavxfpbi\"],\"\":{\"franngwldymuehvv\":\"databzu\"}},\"name\":\"t\",\"type\":\"clzyunhoog\",\"etag\":\"tjcmlyhpxfhix\",\"id\":\"gvkwenl\"}"; + = "{\"properties\":{\"type\":\"ManagedIdentity\",\"typeProperties\":{\"resourceId\":\"bzekrwpwyiyq\"},\"description\":\"csekhu\",\"annotations\":[\"datae\",\"dataudmpsuqpraqjscni\",\"datapvx\"],\"\":{\"jhrvembitqo\":\"dataewtbyciedxsey\",\"khjxgukzz\":\"dataxieuntce\"}},\"name\":\"wbfbfrz\",\"type\":\"sipqbyvesxuzdae\",\"etag\":\"milpztwzjkbaudtp\",\"id\":\"uqkntnvgwgtgxgg\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,13 +51,15 @@ public void testCreateOrUpdateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ManagedIdentityCredentialResource response - = manager.credentialOperations().define("bsuijxmw").withExistingFactory("kxln", "pbadycnjxyproq") - .withProperties(new ManagedIdentityCredential().withDescription("zouvckreju") - .withAnnotations(Arrays.asList("dataorb", "datafarb", "dataarxyh")).withResourceId("qpjueodhtltoo")) - .withIfMatch("vzepgljtuzqreprn").create(); - - Assertions.assertEquals("gvkwenl", response.id()); - Assertions.assertEquals("tzgmfybr", response.properties().description()); - Assertions.assertEquals("yscnbrwh", response.properties().resourceId()); + = manager.credentialOperations().define("gerqtcxk").withExistingFactory("xono", "o") + .withProperties(new ManagedIdentityCredential().withDescription("crucz") + .withAnnotations( + Arrays.asList("datauxijmawsamdfw", "dataxnjbdglsllm", "dataarmlbqekvj", "dataaqjpsjrpkgvs")) + .withResourceId("z")) + .withIfMatch("arwxhpufvucnq").create(); + + Assertions.assertEquals("uqkntnvgwgtgxgg", response.id()); + Assertions.assertEquals("csekhu", response.properties().description()); + Assertions.assertEquals("bzekrwpwyiyq", response.properties().resourceId()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsDeleteWithResponseMockTests.java index 39d4af693070c..587d2e6a8c063 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.credentialOperations().deleteWithResponse("jjfvanefwsodnlw", "npbgqemjdtc", "wk", + manager.credentialOperations().deleteWithResponse("ouvsmmiqfiigsl", "riawknncdfcyey", "l", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsGetWithResponseMockTests.java index 3e66e5ca61f6a..581878ba4e56e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"ManagedIdentity\",\"typeProperties\":{\"resourceId\":\"sxajrnqouj\"},\"description\":\"pvisqflmalm\",\"annotations\":[\"datardpfanjkenrlcjms\"],\"\":{\"hhiiduogakr\":\"datamebxwd\",\"qdtfjctvoh\":\"datamjodbdcyijnoibcl\",\"vstvpkslq\":\"dataw\"}},\"name\":\"abgo\",\"type\":\"ryli\",\"etag\":\"fnhcklll\",\"id\":\"rkvlqqkpxve\"}"; + = "{\"properties\":{\"type\":\"ManagedIdentity\",\"typeProperties\":{\"resourceId\":\"b\"},\"description\":\"ussldjsb\",\"annotations\":[\"datathfas\",\"dataglcktraeraql\",\"datafyhwdogchdqtlbn\"],\"\":{\"wpeksrhkmzs\":\"datayox\"}},\"name\":\"p\",\"type\":\"sbp\",\"etag\":\"kmgwmmwjugaqyrt\",\"id\":\"iyspbghnnx\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ManagedIdentityCredentialResource response = manager.credentialOperations() - .getWithResponse("zhidpxvkpbaftf", "xhfusjxnadiese", "zfh", "ihrxg", com.azure.core.util.Context.NONE) + .getWithResponse("cdryjgxwdanidr", "qfpxka", "bbpzk", "pmjtfymvlvosbc", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("rkvlqqkpxve", response.id()); - Assertions.assertEquals("pvisqflmalm", response.properties().description()); - Assertions.assertEquals("sxajrnqouj", response.properties().resourceId()); + Assertions.assertEquals("iyspbghnnx", response.id()); + Assertions.assertEquals("ussldjsb", response.properties().description()); + Assertions.assertEquals("b", response.properties().resourceId()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsListByFactoryMockTests.java index ad3bb7fcfc62f..692132c29abce 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CredentialOperationsListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"type\":\"ManagedIdentity\",\"typeProperties\":{\"resourceId\":\"xamsgfvuffd\"},\"description\":\"k\",\"annotations\":[\"datamd\"],\"\":{\"efwxcuzu\":\"datagmjpc\",\"ypr\":\"datalby\",\"nos\":\"datakjiiivbvkvo\"}},\"name\":\"krhrnv\",\"type\":\"judgwdsflitmm\",\"etag\":\"uzofuebabrs\",\"id\":\"vaj\"}]}"; + = "{\"value\":[{\"properties\":{\"type\":\"ManagedIdentity\",\"typeProperties\":{\"resourceId\":\"tuxwfwlfqbg\"},\"description\":\"vrwnweiwkbkhdxq\",\"annotations\":[\"datadzbdjottzuk\",\"datasxgaojwulatq\"],\"\":{\"yfdjzefkzcajav\":\"datavkyrslifcwj\"}},\"name\":\"okaqnuy\",\"type\":\"gymbefvuutlirzr\",\"etag\":\"lbdezvmfuhearh\",\"id\":\"hyugjx\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,10 +50,10 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.credentialOperations().listByFactory("wv", "p", com.azure.core.util.Context.NONE); + = manager.credentialOperations().listByFactory("mn", "tdboru", com.azure.core.util.Context.NONE); - Assertions.assertEquals("vaj", response.iterator().next().id()); - Assertions.assertEquals("k", response.iterator().next().properties().description()); - Assertions.assertEquals("xamsgfvuffd", response.iterator().next().properties().resourceId()); + Assertions.assertEquals("hyugjx", response.iterator().next().id()); + Assertions.assertEquals("vrwnweiwkbkhdxq", response.iterator().next().properties().description()); + Assertions.assertEquals("tuxwfwlfqbg", response.iterator().next().properties().resourceId()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityReferenceObjectTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityReferenceObjectTests.java index d5bfd98628317..3ae750f06a932 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityReferenceObjectTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityReferenceObjectTests.java @@ -17,35 +17,30 @@ public final class CustomActivityReferenceObjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CustomActivityReferenceObject model = BinaryData.fromString( - "{\"linkedServices\":[{\"referenceName\":\"fawfeeatvnmm\",\"parameters\":{\"gwtcll\":\"datawfqrykikh\",\"fohtsmkf\":\"datawaz\"}},{\"referenceName\":\"oxbavfseh\",\"parameters\":{\"qszn\":\"dataqion\",\"o\":\"databi\",\"dteyvpv\":\"dataexjcrw\"}},{\"referenceName\":\"cqjgwtiasfbp\",\"parameters\":{\"seh\":\"dataxxxhbrysns\",\"wqg\":\"dataegvwbykrndxbkv\",\"qyedhyfncwiyfzu\":\"datafhl\",\"ccgtujiwzbzed\":\"databsaaxstnziv\"}}],\"datasets\":[{\"referenceName\":\"ygjhclny\",\"parameters\":{\"mtajjfmkwqpgbti\":\"datacgmcthjgbrxmxqsk\",\"alhawopti\":\"datanidubocmjiibtv\"}},{\"referenceName\":\"fuiav\",\"parameters\":{\"ddmmwylrvztael\":\"datacxsma\",\"rcvo\":\"datauxhkuemcbtumt\",\"wee\":\"datahyqexujll\",\"mgfwxthrcmg\":\"datavqbsythycdckcpfo\"}},{\"referenceName\":\"imgosc\",\"parameters\":{\"na\":\"datagckk\",\"nkeolorosahgcc\":\"dataenyehmwzgf\"}}]}") + "{\"linkedServices\":[{\"referenceName\":\"xbrjdpeypuqwd\",\"parameters\":{\"xdeo\":\"datamfvxfssho\",\"fnreempbumyuiquz\":\"datanzprdgmmgtqgzdf\",\"facflkbgohxbj\":\"dataotfoifjrik\"}},{\"referenceName\":\"zfpfxbqdrjunigxn\",\"parameters\":{\"dbvenvr\":\"dataghga\",\"qswrncwhlxvngj\":\"datati\",\"bd\":\"datapydjdpapndmv\"}}],\"datasets\":[{\"referenceName\":\"tapwkwk\",\"parameters\":{\"ide\":\"datae\"}},{\"referenceName\":\"dehskmfiudnp\",\"parameters\":{\"ql\":\"datafhtsgyyrgdguvk\",\"kznffqv\":\"datag\",\"rwgdpf\":\"dataxnytihhqancw\"}},{\"referenceName\":\"dy\",\"parameters\":{\"laiuoncrsw\":\"datarsnbdfamyolvgk\",\"etqcxoamxum\":\"datawz\",\"uhixomxvbruzx\":\"dataz\",\"douneozgnwmc\":\"datanzxipgfkc\"}}]}") .toObject(CustomActivityReferenceObject.class); - Assertions.assertEquals("fawfeeatvnmm", model.linkedServices().get(0).referenceName()); - Assertions.assertEquals("ygjhclny", model.datasets().get(0).referenceName()); + Assertions.assertEquals("xbrjdpeypuqwd", model.linkedServices().get(0).referenceName()); + Assertions.assertEquals("tapwkwk", model.datasets().get(0).referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CustomActivityReferenceObject model - = new CustomActivityReferenceObject() - .withLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("fawfeeatvnmm") - .withParameters(mapOf("gwtcll", "datawfqrykikh", "fohtsmkf", "datawaz")), - new LinkedServiceReference().withReferenceName("oxbavfseh") - .withParameters(mapOf("qszn", "dataqion", "o", "databi", "dteyvpv", "dataexjcrw")), - new LinkedServiceReference().withReferenceName("cqjgwtiasfbp") - .withParameters(mapOf("seh", "dataxxxhbrysns", "wqg", "dataegvwbykrndxbkv", "qyedhyfncwiyfzu", - "datafhl", "ccgtujiwzbzed", "databsaaxstnziv")))) - .withDatasets(Arrays.asList( - new DatasetReference().withReferenceName("ygjhclny").withParameters( - mapOf("mtajjfmkwqpgbti", "datacgmcthjgbrxmxqsk", "alhawopti", "datanidubocmjiibtv")), - new DatasetReference().withReferenceName("fuiav") - .withParameters(mapOf("ddmmwylrvztael", "datacxsma", "rcvo", "datauxhkuemcbtumt", "wee", - "datahyqexujll", "mgfwxthrcmg", "datavqbsythycdckcpfo")), - new DatasetReference().withReferenceName("imgosc") - .withParameters(mapOf("na", "datagckk", "nkeolorosahgcc", "dataenyehmwzgf")))); + CustomActivityReferenceObject model = new CustomActivityReferenceObject() + .withLinkedServices(Arrays.asList( + new LinkedServiceReference().withReferenceName("xbrjdpeypuqwd") + .withParameters(mapOf("xdeo", "datamfvxfssho", "fnreempbumyuiquz", "datanzprdgmmgtqgzdf", + "facflkbgohxbj", "dataotfoifjrik")), + new LinkedServiceReference().withReferenceName("zfpfxbqdrjunigxn").withParameters( + mapOf("dbvenvr", "dataghga", "qswrncwhlxvngj", "datati", "bd", "datapydjdpapndmv")))) + .withDatasets(Arrays.asList( + new DatasetReference().withReferenceName("tapwkwk").withParameters(mapOf("ide", "datae")), + new DatasetReference().withReferenceName("dehskmfiudnp").withParameters( + mapOf("ql", "datafhtsgyyrgdguvk", "kznffqv", "datag", "rwgdpf", "dataxnytihhqancw")), + new DatasetReference().withReferenceName("dy").withParameters(mapOf("laiuoncrsw", "datarsnbdfamyolvgk", + "etqcxoamxum", "datawz", "uhixomxvbruzx", "dataz", "douneozgnwmc", "datanzxipgfkc")))); model = BinaryData.fromObject(model).toObject(CustomActivityReferenceObject.class); - Assertions.assertEquals("fawfeeatvnmm", model.linkedServices().get(0).referenceName()); - Assertions.assertEquals("ygjhclny", model.datasets().get(0).referenceName()); + Assertions.assertEquals("xbrjdpeypuqwd", model.linkedServices().get(0).referenceName()); + Assertions.assertEquals("tapwkwk", model.datasets().get(0).referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTests.java index 8ca88ab371bb0..ece561b09699f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTests.java @@ -24,90 +24,82 @@ public final class CustomActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CustomActivity model = BinaryData.fromString( - "{\"type\":\"Custom\",\"typeProperties\":{\"command\":\"dataykxlfl\",\"resourceLinkedService\":{\"referenceName\":\"uwunsnyqpmnyvnbm\",\"parameters\":{\"sdnobxcdxpnrae\":\"datamcrevtazclilzve\",\"xr\":\"datadixoflxvsu\"}},\"folderPath\":\"datacozfjsfrbjrbqc\",\"referenceObjects\":{\"linkedServices\":[{\"referenceName\":\"bcy\",\"parameters\":{\"hvbovblxfyle\":\"datasmrfbwsicm\"}},{\"referenceName\":\"gd\",\"parameters\":{\"zszlrv\":\"datarfemnykfzsouou\"}},{\"referenceName\":\"qldgii\",\"parameters\":{\"tyjf\":\"datakxprb\",\"qlruhhkkbfgr\":\"datajhbvllj\",\"pdredcvwsbsdy\":\"datascbmdrbtgm\",\"hgnfaanubjeb\":\"dataqjyrqouyfcfdedeu\"}}],\"datasets\":[{\"referenceName\":\"ksghsowmvtmj\",\"parameters\":{\"b\":\"datan\",\"c\":\"datahhpfj\"}},{\"referenceName\":\"l\",\"parameters\":{\"ovrsrtldijgrbit\":\"dataavdopecj\",\"hcjtwhwgbaj\":\"datadwuoxirziluzokx\"}},{\"referenceName\":\"gctwamjjwvmugis\",\"parameters\":{\"xzgfqtgfbmocv\":\"datazmhhopedmkxtd\",\"sr\":\"datalrvkqxhkh\",\"z\":\"dataqpxaajtiyrqtuz\",\"pazmxssb\":\"dataubxng\"}}]},\"extendedProperties\":{\"cakkewgz\":\"datanatbecuh\",\"sjlqt\":\"datao\",\"hqjdihjo\":\"datajewezcknpmev\",\"l\":\"datadwahehudicx\"},\"retentionTimeInDays\":\"datahfdgvxoiwbmjqv\",\"autoUserSpecification\":\"datazcjvogrripopzy\"},\"linkedServiceName\":{\"referenceName\":\"spwwkdm\",\"parameters\":{\"qmelm\":\"datazdumjqdhrgwyq\",\"ve\":\"datakbepieh\",\"u\":\"dataer\"}},\"policy\":{\"timeout\":\"datanl\",\"retry\":\"datanu\",\"retryIntervalInSeconds\":537670074,\"secureInput\":false,\"secureOutput\":true,\"\":{\"xyphdkxwstabgejo\":\"datatjfkjboyggrmzt\",\"tgoeayhojdgw\":\"dataveg\",\"dpxbwqgk\":\"datazcrssmbdjzc\"}},\"name\":\"xcdtjayevv\",\"description\":\"dnwjp\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"xbmaiivopft\",\"dependencyConditions\":[\"Failed\",\"Succeeded\"],\"\":{\"bjpumpqlugzydyl\":\"datal\",\"dmfo\":\"datarsis\",\"svfnxxkmrfz\":\"datai\"}},{\"activity\":\"tkprbm\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Skipped\",\"Failed\"],\"\":{\"ilvajctpwlf\":\"datawikvakjxduh\"}},{\"activity\":\"fardjqwdrooooobs\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"wajjzxcqnl\":\"dataqcme\",\"lxa\":\"dataxewocwmadyelwo\",\"zqldakbijcxctn\":\"datahanfjrdcaw\"}}],\"userProperties\":[{\"name\":\"yczzwhwsidnqiav\",\"value\":\"datayhnoll\"},{\"name\":\"uhocb\",\"value\":\"dataeoxoe\"},{\"name\":\"prtz\",\"value\":\"datafcurn\"},{\"name\":\"ujcunyua\",\"value\":\"datastok\"}],\"\":{\"yblmcen\":\"datay\",\"amuplxksph\":\"datactc\",\"s\":\"datayubytslfmajswrf\"}}") + "{\"type\":\"Custom\",\"typeProperties\":{\"command\":\"databksafnecwyrt\",\"resourceLinkedService\":{\"referenceName\":\"ujyespcgps\",\"parameters\":{\"rc\":\"dataooxieyywwmiwia\"}},\"folderPath\":\"datayb\",\"referenceObjects\":{\"linkedServices\":[{\"referenceName\":\"ia\",\"parameters\":{\"oyvygdefpy\":\"datazszcrwhr\"}},{\"referenceName\":\"twwaxx\",\"parameters\":{\"lbocecmnqcgbijyp\":\"datadsmravxtglpxmdw\",\"uzchegeogdkcrc\":\"datawbyrkxzebv\"}}],\"datasets\":[{\"referenceName\":\"zeumadlpxirew\",\"parameters\":{\"rmgefxkat\":\"datakicimyykmkelbqm\"}}]},\"extendedProperties\":{\"ptbjooq\":\"datakwldvksigxakg\",\"xkh\":\"databpnkvnuwjrxb\",\"imgzimtzzyjhy\":\"dataeqbxvtgloifmlbh\"},\"retentionTimeInDays\":\"datayxrwfuxx\",\"autoUserSpecification\":\"datavdhmumsmnubc\"},\"linkedServiceName\":{\"referenceName\":\"rps\",\"parameters\":{\"rintaaf\":\"dataxidqnvhrbfepf\",\"fuxhlgoexu\":\"datadysevqppxth\",\"ewhbxvriplgk\":\"datanbfoorgtxd\"}},\"policy\":{\"timeout\":\"datammqvzzoomwfo\",\"retry\":\"datahatpazljajzqgg\",\"retryIntervalInSeconds\":219227608,\"secureInput\":true,\"secureOutput\":false,\"\":{\"y\":\"datavbjk\",\"p\":\"dataodgisfejs\",\"ujyn\":\"datawi\"}},\"name\":\"jwktiyhiyk\",\"description\":\"vaodifupdafuf\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"qhgnrxxhzwtrxpwu\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Skipped\",\"Completed\"],\"\":{\"utdthloafhhiyk\":\"datahgbjukaswgvoa\",\"ptefdvjgbemrjb\":\"datatjsebcuynqdl\",\"pdprjethyhbnoye\":\"datavqu\",\"iqoiblaumog\":\"datauivdrzxobtekl\"}},{\"activity\":\"zxwmwrjmtpgkybdk\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\"],\"\":{\"npxa\":\"dataqbeqz\"}},{\"activity\":\"scnnyg\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\"],\"\":{\"wefzdnyga\":\"datargo\"}},{\"activity\":\"hv\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Failed\",\"Completed\"],\"\":{\"mkkpbybhqwilbsdg\":\"datayqxjpzykkw\",\"kir\":\"datahe\",\"rth\":\"dataojzfsznephbc\",\"ijayvuymib\":\"databgavwbqjeto\"}}],\"userProperties\":[{\"name\":\"j\",\"value\":\"dataxpudeqw\"},{\"name\":\"ivjhm\",\"value\":\"datadvnoxjbhltxtpgq\"}],\"\":{\"afgbcwawblk\":\"datakkta\",\"sgklxgsqhczokun\":\"datacci\",\"cbqvje\":\"dataqqhbjmvbeznlukeq\",\"hmrybbhktnuzor\":\"datanwwqyyfctfsdhmru\"}}") .toObject(CustomActivity.class); - Assertions.assertEquals("xcdtjayevv", model.name()); - Assertions.assertEquals("dnwjp", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("xbmaiivopft", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("yczzwhwsidnqiav", model.userProperties().get(0).name()); - Assertions.assertEquals("spwwkdm", model.linkedServiceName().referenceName()); - Assertions.assertEquals(537670074, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("uwunsnyqpmnyvnbm", model.resourceLinkedService().referenceName()); - Assertions.assertEquals("bcy", model.referenceObjects().linkedServices().get(0).referenceName()); - Assertions.assertEquals("ksghsowmvtmj", model.referenceObjects().datasets().get(0).referenceName()); + Assertions.assertEquals("jwktiyhiyk", model.name()); + Assertions.assertEquals("vaodifupdafuf", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("qhgnrxxhzwtrxpwu", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("j", model.userProperties().get(0).name()); + Assertions.assertEquals("rps", model.linkedServiceName().referenceName()); + Assertions.assertEquals(219227608, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("ujyespcgps", model.resourceLinkedService().referenceName()); + Assertions.assertEquals("ia", model.referenceObjects().linkedServices().get(0).referenceName()); + Assertions.assertEquals("zeumadlpxirew", model.referenceObjects().datasets().get(0).referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CustomActivity model - = new CustomActivity().withName("xcdtjayevv").withDescription("dnwjp").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("xbmaiivopft") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("tkprbm") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fardjqwdrooooobs") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("yczzwhwsidnqiav").withValue("datayhnoll"), - new UserProperty().withName("uhocb").withValue("dataeoxoe"), - new UserProperty().withName("prtz").withValue("datafcurn"), - new UserProperty().withName("ujcunyua").withValue("datastok"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("spwwkdm") - .withParameters(mapOf("qmelm", "datazdumjqdhrgwyq", "ve", "datakbepieh", "u", "dataer"))) - .withPolicy( - new ActivityPolicy().withTimeout("datanl").withRetry("datanu").withRetryIntervalInSeconds(537670074) - .withSecureInput(false).withSecureOutput(true).withAdditionalProperties(mapOf())) - .withCommand("dataykxlfl") - .withResourceLinkedService(new LinkedServiceReference().withReferenceName("uwunsnyqpmnyvnbm") - .withParameters(mapOf("sdnobxcdxpnrae", "datamcrevtazclilzve", "xr", "datadixoflxvsu"))) - .withFolderPath("datacozfjsfrbjrbqc") - .withReferenceObjects(new CustomActivityReferenceObject() - .withLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("bcy") - .withParameters(mapOf("hvbovblxfyle", "datasmrfbwsicm")), - new LinkedServiceReference().withReferenceName("gd") - .withParameters(mapOf("zszlrv", "datarfemnykfzsouou")), - new LinkedServiceReference().withReferenceName("qldgii") - .withParameters(mapOf("tyjf", "datakxprb", "qlruhhkkbfgr", "datajhbvllj", "pdredcvwsbsdy", - "datascbmdrbtgm", "hgnfaanubjeb", "dataqjyrqouyfcfdedeu")))) - .withDatasets(Arrays.asList( - new DatasetReference().withReferenceName("ksghsowmvtmj") - .withParameters(mapOf("b", "datan", "c", "datahhpfj")), - new DatasetReference().withReferenceName("l").withParameters( - mapOf("ovrsrtldijgrbit", "dataavdopecj", "hcjtwhwgbaj", "datadwuoxirziluzokx")), - new DatasetReference().withReferenceName("gctwamjjwvmugis") - .withParameters(mapOf("xzgfqtgfbmocv", "datazmhhopedmkxtd", "sr", "datalrvkqxhkh", "z", - "dataqpxaajtiyrqtuz", "pazmxssb", "dataubxng"))))) - .withExtendedProperties(mapOf("cakkewgz", "datanatbecuh", "sjlqt", "datao", "hqjdihjo", - "datajewezcknpmev", "l", "datadwahehudicx")) - .withRetentionTimeInDays("datahfdgvxoiwbmjqv").withAutoUserSpecification("datazcjvogrripopzy"); + CustomActivity model = new CustomActivity().withName("jwktiyhiyk").withDescription("vaodifupdafuf") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("qhgnrxxhzwtrxpwu") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, + DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("zxwmwrjmtpgkybdk") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, + DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("scnnyg") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("hv") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, + DependencyCondition.FAILED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("j").withValue("dataxpudeqw"), + new UserProperty().withName("ivjhm").withValue("datadvnoxjbhltxtpgq"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("rps") + .withParameters(mapOf("rintaaf", "dataxidqnvhrbfepf", "fuxhlgoexu", "datadysevqppxth", "ewhbxvriplgk", + "datanbfoorgtxd"))) + .withPolicy(new ActivityPolicy() + .withTimeout("datammqvzzoomwfo").withRetry("datahatpazljajzqgg").withRetryIntervalInSeconds(219227608) + .withSecureInput(true).withSecureOutput(false).withAdditionalProperties(mapOf())) + .withCommand("databksafnecwyrt") + .withResourceLinkedService(new LinkedServiceReference().withReferenceName("ujyespcgps") + .withParameters(mapOf("rc", "dataooxieyywwmiwia"))) + .withFolderPath("datayb") + .withReferenceObjects(new CustomActivityReferenceObject() + .withLinkedServices(Arrays.asList( + new LinkedServiceReference().withReferenceName("ia") + .withParameters(mapOf("oyvygdefpy", "datazszcrwhr")), + new LinkedServiceReference().withReferenceName("twwaxx").withParameters( + mapOf("lbocecmnqcgbijyp", "datadsmravxtglpxmdw", "uzchegeogdkcrc", "datawbyrkxzebv")))) + .withDatasets(Arrays.asList(new DatasetReference().withReferenceName("zeumadlpxirew") + .withParameters(mapOf("rmgefxkat", "datakicimyykmkelbqm"))))) + .withExtendedProperties(mapOf("ptbjooq", "datakwldvksigxakg", "xkh", "databpnkvnuwjrxb", "imgzimtzzyjhy", + "dataeqbxvtgloifmlbh")) + .withRetentionTimeInDays("datayxrwfuxx").withAutoUserSpecification("datavdhmumsmnubc"); model = BinaryData.fromObject(model).toObject(CustomActivity.class); - Assertions.assertEquals("xcdtjayevv", model.name()); - Assertions.assertEquals("dnwjp", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("xbmaiivopft", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("yczzwhwsidnqiav", model.userProperties().get(0).name()); - Assertions.assertEquals("spwwkdm", model.linkedServiceName().referenceName()); - Assertions.assertEquals(537670074, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("uwunsnyqpmnyvnbm", model.resourceLinkedService().referenceName()); - Assertions.assertEquals("bcy", model.referenceObjects().linkedServices().get(0).referenceName()); - Assertions.assertEquals("ksghsowmvtmj", model.referenceObjects().datasets().get(0).referenceName()); + Assertions.assertEquals("jwktiyhiyk", model.name()); + Assertions.assertEquals("vaodifupdafuf", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("qhgnrxxhzwtrxpwu", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("j", model.userProperties().get(0).name()); + Assertions.assertEquals("rps", model.linkedServiceName().referenceName()); + Assertions.assertEquals(219227608, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("ujyespcgps", model.resourceLinkedService().referenceName()); + Assertions.assertEquals("ia", model.referenceObjects().linkedServices().get(0).referenceName()); + Assertions.assertEquals("zeumadlpxirew", model.referenceObjects().datasets().get(0).referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTypePropertiesTests.java index 5762e28000b94..045e3fa964d92 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomActivityTypePropertiesTests.java @@ -18,42 +18,41 @@ public final class CustomActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CustomActivityTypeProperties model = BinaryData.fromString( - "{\"command\":\"datagvkqz\",\"resourceLinkedService\":{\"referenceName\":\"oslyu\",\"parameters\":{\"cffbsnlv\":\"dataejualugyu\",\"jdmesoxjkp\":\"datassyzwtzdyzufgnns\",\"pbiou\":\"databgfhjwchv\"}},\"folderPath\":\"dataoxbxtws\",\"referenceObjects\":{\"linkedServices\":[{\"referenceName\":\"ghlokvisqzmhe\",\"parameters\":{\"rtkegrtvwffvbvu\":\"dataioonnfj\",\"eavs\":\"datapy\",\"kani\":\"datazbzykksd\",\"twwbahivfosbr\":\"datadcjxgzpmwxvfrm\"}},{\"referenceName\":\"eywhlqydhh\",\"parameters\":{\"z\":\"datavhucawmhbqjll\"}},{\"referenceName\":\"qvnbxgkudioum\",\"parameters\":{\"qvzwummw\":\"databytzh\"}},{\"referenceName\":\"ax\",\"parameters\":{\"iwkqr\":\"dataihgcdujhz\",\"l\":\"datatrmi\"}}],\"datasets\":[{\"referenceName\":\"dukamtfkufvabci\",\"parameters\":{\"pliwgkozl\":\"datayf\"}},{\"referenceName\":\"sfrajpyuw\",\"parameters\":{\"ungzvytbq\":\"datafglll\"}}]},\"extendedProperties\":{\"aetgmmfdf\":\"datakuyyrcqsyqhugj\",\"epwyyeupkpyzaena\":\"dataq\",\"ygfgchlcbtxc\":\"datafyrlqiykh\"},\"retentionTimeInDays\":\"datakg\",\"autoUserSpecification\":\"datayklyhmymkcc\"}") + "{\"command\":\"dataa\",\"resourceLinkedService\":{\"referenceName\":\"zjqpdfs\",\"parameters\":{\"cfxsgjdiqemcghor\":\"datavitis\",\"vhtmzwgircfnz\":\"datajawfczbbvr\",\"tvlqwpmmmhupvx\":\"dataybrflq\"}},\"folderPath\":\"datapsugebgboqnci\",\"referenceObjects\":{\"linkedServices\":[{\"referenceName\":\"ufo\",\"parameters\":{\"jdaxezfdsoglji\":\"dataqgefxypxmkexjona\"}},{\"referenceName\":\"wduwn\",\"parameters\":{\"fachkzzn\":\"datafwogqwdxtp\"}}],\"datasets\":[{\"referenceName\":\"msfnigjoxhz\",\"parameters\":{\"b\":\"datamcsjyfbutqlotojf\",\"wjgjlo\":\"datay\",\"pbih\":\"datahhhkxlqu\"}},{\"referenceName\":\"injymnqweptejr\",\"parameters\":{\"rkcqpy\":\"datauktcnxtpamwjb\",\"tvovhuifbly\":\"datajj\"}},{\"referenceName\":\"qycknqmbvssjb\",\"parameters\":{\"n\":\"datausnnc\",\"uhsjzduumpl\":\"datai\",\"wjcy\":\"datawupfndafrz\"}},{\"referenceName\":\"zaneave\",\"parameters\":{\"w\":\"datarul\"}}]},\"extendedProperties\":{\"qpawwjvdohzewu\":\"datazesqdvmxuf\"},\"retentionTimeInDays\":\"dataaeshftls\",\"autoUserSpecification\":\"datapvflmxjd\"}") .toObject(CustomActivityTypeProperties.class); - Assertions.assertEquals("oslyu", model.resourceLinkedService().referenceName()); - Assertions.assertEquals("ghlokvisqzmhe", model.referenceObjects().linkedServices().get(0).referenceName()); - Assertions.assertEquals("dukamtfkufvabci", model.referenceObjects().datasets().get(0).referenceName()); + Assertions.assertEquals("zjqpdfs", model.resourceLinkedService().referenceName()); + Assertions.assertEquals("ufo", model.referenceObjects().linkedServices().get(0).referenceName()); + Assertions.assertEquals("msfnigjoxhz", model.referenceObjects().datasets().get(0).referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CustomActivityTypeProperties model = new CustomActivityTypeProperties().withCommand("datagvkqz") - .withResourceLinkedService(new LinkedServiceReference().withReferenceName("oslyu").withParameters( - mapOf("cffbsnlv", "dataejualugyu", "jdmesoxjkp", "datassyzwtzdyzufgnns", "pbiou", "databgfhjwchv"))) - .withFolderPath("dataoxbxtws") - .withReferenceObjects(new CustomActivityReferenceObject() - .withLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("ghlokvisqzmhe") - .withParameters(mapOf("rtkegrtvwffvbvu", "dataioonnfj", "eavs", "datapy", "kani", - "datazbzykksd", "twwbahivfosbr", "datadcjxgzpmwxvfrm")), - new LinkedServiceReference().withReferenceName("eywhlqydhh") - .withParameters(mapOf("z", "datavhucawmhbqjll")), - new LinkedServiceReference().withReferenceName("qvnbxgkudioum") - .withParameters(mapOf("qvzwummw", "databytzh")), - new LinkedServiceReference().withReferenceName("ax") - .withParameters(mapOf("iwkqr", "dataihgcdujhz", "l", "datatrmi")))) - .withDatasets(Arrays.asList( - new DatasetReference().withReferenceName("dukamtfkufvabci") - .withParameters(mapOf("pliwgkozl", "datayf")), - new DatasetReference().withReferenceName("sfrajpyuw") - .withParameters(mapOf("ungzvytbq", "datafglll"))))) - .withExtendedProperties( - mapOf("aetgmmfdf", "datakuyyrcqsyqhugj", "epwyyeupkpyzaena", "dataq", "ygfgchlcbtxc", "datafyrlqiykh")) - .withRetentionTimeInDays("datakg").withAutoUserSpecification("datayklyhmymkcc"); + CustomActivityTypeProperties model + = new CustomActivityTypeProperties().withCommand("dataa") + .withResourceLinkedService(new LinkedServiceReference().withReferenceName("zjqpdfs") + .withParameters(mapOf("cfxsgjdiqemcghor", "datavitis", "vhtmzwgircfnz", "datajawfczbbvr", + "tvlqwpmmmhupvx", "dataybrflq"))) + .withFolderPath("datapsugebgboqnci") + .withReferenceObjects(new CustomActivityReferenceObject() + .withLinkedServices(Arrays.asList( + new LinkedServiceReference().withReferenceName("ufo") + .withParameters(mapOf("jdaxezfdsoglji", "dataqgefxypxmkexjona")), + new LinkedServiceReference().withReferenceName("wduwn") + .withParameters(mapOf("fachkzzn", "datafwogqwdxtp")))) + .withDatasets(Arrays.asList( + new DatasetReference().withReferenceName("msfnigjoxhz").withParameters( + mapOf("b", "datamcsjyfbutqlotojf", "wjgjlo", "datay", "pbih", "datahhhkxlqu")), + new DatasetReference().withReferenceName("injymnqweptejr") + .withParameters(mapOf("rkcqpy", "datauktcnxtpamwjb", "tvovhuifbly", "datajj")), + new DatasetReference().withReferenceName("qycknqmbvssjb") + .withParameters(mapOf("n", "datausnnc", "uhsjzduumpl", "datai", "wjcy", "datawupfndafrz")), + new DatasetReference().withReferenceName("zaneave").withParameters(mapOf("w", "datarul"))))) + .withExtendedProperties(mapOf("qpawwjvdohzewu", "datazesqdvmxuf")) + .withRetentionTimeInDays("dataaeshftls").withAutoUserSpecification("datapvflmxjd"); model = BinaryData.fromObject(model).toObject(CustomActivityTypeProperties.class); - Assertions.assertEquals("oslyu", model.resourceLinkedService().referenceName()); - Assertions.assertEquals("ghlokvisqzmhe", model.referenceObjects().linkedServices().get(0).referenceName()); - Assertions.assertEquals("dukamtfkufvabci", model.referenceObjects().datasets().get(0).referenceName()); + Assertions.assertEquals("zjqpdfs", model.resourceLinkedService().referenceName()); + Assertions.assertEquals("ufo", model.referenceObjects().linkedServices().get(0).referenceName()); + Assertions.assertEquals("msfnigjoxhz", model.referenceObjects().datasets().get(0).referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomDataSourceLinkedServiceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomDataSourceLinkedServiceTests.java index bd7d4dd484f16..6d3f25936d48d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomDataSourceLinkedServiceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomDataSourceLinkedServiceTests.java @@ -18,28 +18,27 @@ public final class CustomDataSourceLinkedServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CustomDataSourceLinkedService model = BinaryData.fromString( - "{\"type\":\"CustomDataSource\",\"typeProperties\":\"datakrjucazwedma\",\"connectVia\":{\"referenceName\":\"lx\",\"parameters\":{\"nenxg\":\"datamxoanudvqa\",\"b\":\"datahmmgblqyfg\"}},\"description\":\"qifsgzfgxwfxji\",\"parameters\":{\"yooghjxhp\":{\"type\":\"Object\",\"defaultValue\":\"datahwnn\"},\"hoyg\":{\"type\":\"Object\",\"defaultValue\":\"dataksqiwlmegjtose\"},\"zzugdorc\":{\"type\":\"Float\",\"defaultValue\":\"dataz\"}},\"annotations\":[\"datavovsirtasepi\"],\"\":{\"amnvrcqjpbainsih\":\"dataexhhjjatlepzbiro\",\"eufj\":\"dataz\",\"zm\":\"datab\"}}") + "{\"type\":\"CustomDataSource\",\"typeProperties\":\"datag\",\"connectVia\":{\"referenceName\":\"vbtkafcnfitpu\",\"parameters\":{\"xb\":\"datakdwyjdvy\",\"nxbdisjeovgc\":\"datawjgyngoudclridql\"}},\"description\":\"chgjonrhdib\",\"parameters\":{\"meouiuvkcnqo\":{\"type\":\"Object\",\"defaultValue\":\"datay\"}},\"annotations\":[\"datab\",\"datawsfllzykzp\",\"datajdslpbyejsgbpjjn\",\"datalbkaknivw\"],\"\":{\"qilsbabqtjch\":\"datanddctkjcqhxdirt\"}}") .toObject(CustomDataSourceLinkedService.class); - Assertions.assertEquals("lx", model.connectVia().referenceName()); - Assertions.assertEquals("qifsgzfgxwfxji", model.description()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("yooghjxhp").type()); + Assertions.assertEquals("vbtkafcnfitpu", model.connectVia().referenceName()); + Assertions.assertEquals("chgjonrhdib", model.description()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("meouiuvkcnqo").type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { CustomDataSourceLinkedService model = new CustomDataSourceLinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("lx") - .withParameters(mapOf("nenxg", "datamxoanudvqa", "b", "datahmmgblqyfg"))) - .withDescription("qifsgzfgxwfxji") - .withParameters(mapOf("yooghjxhp", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datahwnn"), "hoyg", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataksqiwlmegjtose"), - "zzugdorc", new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataz"))) - .withAnnotations(Arrays.asList("datavovsirtasepi")).withTypeProperties("datakrjucazwedma"); + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("vbtkafcnfitpu") + .withParameters(mapOf("xb", "datakdwyjdvy", "nxbdisjeovgc", "datawjgyngoudclridql"))) + .withDescription("chgjonrhdib") + .withParameters(mapOf("meouiuvkcnqo", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datay"))) + .withAnnotations(Arrays.asList("datab", "datawsfllzykzp", "datajdslpbyejsgbpjjn", "datalbkaknivw")) + .withTypeProperties("datag"); model = BinaryData.fromObject(model).toObject(CustomDataSourceLinkedService.class); - Assertions.assertEquals("lx", model.connectVia().referenceName()); - Assertions.assertEquals("qifsgzfgxwfxji", model.description()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("yooghjxhp").type()); + Assertions.assertEquals("vbtkafcnfitpu", model.connectVia().referenceName()); + Assertions.assertEquals("chgjonrhdib", model.description()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("meouiuvkcnqo").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTests.java index 1b0ef6d855f1f..f07c296e2df5f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTests.java @@ -17,43 +17,38 @@ public final class CustomEventsTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CustomEventsTrigger model = BinaryData.fromString( - "{\"type\":\"CustomEventsTrigger\",\"typeProperties\":{\"subjectBeginsWith\":\"uwdesyttkaufabtp\",\"subjectEndsWith\":\"ntnjnkvsnsiphl\",\"events\":[\"datacedz\",\"datadvzbxtzgx\",\"dataxqe\"],\"scope\":\"uzubntuimi\"},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"uby\",\"name\":\"hencgfzhbtzu\"},\"parameters\":{\"txkhyvjomqrioa\":\"datatchxtbcqjvyz\",\"rglmjrufwqpnmcw\":\"datazmrwlsrjjaj\"}},{\"pipelineReference\":{\"referenceName\":\"fpytfdzkbkyt\",\"name\":\"twwk\"},\"parameters\":{\"rzvlupnfrlygyjrl\":\"databdozdkrmplj\",\"gzwhfeqz\":\"datag\",\"djagyksgntgiwaa\":\"datavjlsyzzkgebe\"}},{\"pipelineReference\":{\"referenceName\":\"jbxh\",\"name\":\"hgbloeae\"},\"parameters\":{\"lx\":\"dataum\"}},{\"pipelineReference\":{\"referenceName\":\"gbyxpma\",\"name\":\"rn\"},\"parameters\":{\"faf\":\"datahviqwfctiy\"}}],\"description\":\"rnsktdgbombnc\",\"runtimeState\":\"Stopped\",\"annotations\":[\"datapqwntrqvlcunnbai\"],\"\":{\"kdaphzemnjastkb\":\"datacpruommtucazgrl\",\"qbwim\":\"dataz\",\"r\":\"datajhmgocal\",\"nreukcrcsdaip\":\"datakmwyoukfim\"}}") + "{\"type\":\"CustomEventsTrigger\",\"typeProperties\":{\"subjectBeginsWith\":\"lmytaeallsx\",\"subjectEndsWith\":\"antssbzmo\",\"events\":[\"dataj\"],\"scope\":\"nhmxkgxrf\"},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"wpzuxoynxlkloqp\",\"name\":\"aqcrefk\"},\"parameters\":{\"wecrvkiaognm\":\"datanzowpv\",\"prlqwjwpejtszj\":\"datanrz\",\"vwvycvnow\":\"datavj\"}},{\"pipelineReference\":{\"referenceName\":\"lij\",\"name\":\"lhxwwhusro\"},\"parameters\":{\"wkwmq\":\"datamozafwqmo\",\"aqassukv\":\"dataoqldacxo\"}}],\"description\":\"kzxznc\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datanszmjzsjfc\"],\"\":{\"diirvx\":\"datakp\",\"miychdufla\":\"datahxyslhxokfoma\",\"yqp\":\"datasgutgzcbv\"}}") .toObject(CustomEventsTrigger.class); - Assertions.assertEquals("rnsktdgbombnc", model.description()); - Assertions.assertEquals("uby", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("hencgfzhbtzu", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals("uwdesyttkaufabtp", model.subjectBeginsWith()); - Assertions.assertEquals("ntnjnkvsnsiphl", model.subjectEndsWith()); - Assertions.assertEquals("uzubntuimi", model.scope()); + Assertions.assertEquals("kzxznc", model.description()); + Assertions.assertEquals("wpzuxoynxlkloqp", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("aqcrefk", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("lmytaeallsx", model.subjectBeginsWith()); + Assertions.assertEquals("antssbzmo", model.subjectEndsWith()); + Assertions.assertEquals("nhmxkgxrf", model.scope()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { CustomEventsTrigger model - = new CustomEventsTrigger().withDescription("rnsktdgbombnc") - .withAnnotations(Arrays.asList("datapqwntrqvlcunnbai")) - .withPipelines(Arrays.asList(new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("uby").withName("hencgfzhbtzu")) - .withParameters(mapOf("txkhyvjomqrioa", "datatchxtbcqjvyz", "rglmjrufwqpnmcw", "datazmrwlsrjjaj")), - new TriggerPipelineReference().withPipelineReference( - new PipelineReference().withReferenceName("fpytfdzkbkyt").withName("twwk")) - .withParameters(mapOf("rzvlupnfrlygyjrl", "databdozdkrmplj", "gzwhfeqz", "datag", - "djagyksgntgiwaa", "datavjlsyzzkgebe")), + = new CustomEventsTrigger().withDescription("kzxznc").withAnnotations(Arrays.asList("datanszmjzsjfc")) + .withPipelines(Arrays.asList( new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("jbxh").withName("hgbloeae")) - .withParameters(mapOf("lx", "dataum")), + .withPipelineReference( + new PipelineReference().withReferenceName("wpzuxoynxlkloqp").withName("aqcrefk")) + .withParameters( + mapOf("wecrvkiaognm", "datanzowpv", "prlqwjwpejtszj", "datanrz", "vwvycvnow", "datavj")), new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("gbyxpma").withName("rn")) - .withParameters(mapOf("faf", "datahviqwfctiy")))) - .withSubjectBeginsWith("uwdesyttkaufabtp").withSubjectEndsWith("ntnjnkvsnsiphl") - .withEvents(Arrays.asList("datacedz", "datadvzbxtzgx", "dataxqe")).withScope("uzubntuimi"); + .withPipelineReference(new PipelineReference().withReferenceName("lij").withName("lhxwwhusro")) + .withParameters(mapOf("wkwmq", "datamozafwqmo", "aqassukv", "dataoqldacxo")))) + .withSubjectBeginsWith("lmytaeallsx").withSubjectEndsWith("antssbzmo") + .withEvents(Arrays.asList("dataj")).withScope("nhmxkgxrf"); model = BinaryData.fromObject(model).toObject(CustomEventsTrigger.class); - Assertions.assertEquals("rnsktdgbombnc", model.description()); - Assertions.assertEquals("uby", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("hencgfzhbtzu", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals("uwdesyttkaufabtp", model.subjectBeginsWith()); - Assertions.assertEquals("ntnjnkvsnsiphl", model.subjectEndsWith()); - Assertions.assertEquals("uzubntuimi", model.scope()); + Assertions.assertEquals("kzxznc", model.description()); + Assertions.assertEquals("wpzuxoynxlkloqp", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("aqcrefk", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("lmytaeallsx", model.subjectBeginsWith()); + Assertions.assertEquals("antssbzmo", model.subjectEndsWith()); + Assertions.assertEquals("nhmxkgxrf", model.scope()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTypePropertiesTests.java index fae89dcb514eb..57dad7f09d673 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/CustomEventsTriggerTypePropertiesTests.java @@ -13,21 +13,21 @@ public final class CustomEventsTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CustomEventsTriggerTypeProperties model = BinaryData.fromString( - "{\"subjectBeginsWith\":\"povr\",\"subjectEndsWith\":\"otbybcx\",\"events\":[\"datarlzdnccx\",\"dataq\"],\"scope\":\"pfgjnynuqikoiu\"}") + "{\"subjectBeginsWith\":\"hkm\",\"subjectEndsWith\":\"raoytkkqoaazvmn\",\"events\":[\"datadzfypdsrfpihvij\",\"datajtkpocqboyjjfx\"],\"scope\":\"njduyotqb\"}") .toObject(CustomEventsTriggerTypeProperties.class); - Assertions.assertEquals("povr", model.subjectBeginsWith()); - Assertions.assertEquals("otbybcx", model.subjectEndsWith()); - Assertions.assertEquals("pfgjnynuqikoiu", model.scope()); + Assertions.assertEquals("hkm", model.subjectBeginsWith()); + Assertions.assertEquals("raoytkkqoaazvmn", model.subjectEndsWith()); + Assertions.assertEquals("njduyotqb", model.scope()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CustomEventsTriggerTypeProperties model - = new CustomEventsTriggerTypeProperties().withSubjectBeginsWith("povr").withSubjectEndsWith("otbybcx") - .withEvents(Arrays.asList("datarlzdnccx", "dataq")).withScope("pfgjnynuqikoiu"); + CustomEventsTriggerTypeProperties model = new CustomEventsTriggerTypeProperties().withSubjectBeginsWith("hkm") + .withSubjectEndsWith("raoytkkqoaazvmn") + .withEvents(Arrays.asList("datadzfypdsrfpihvij", "datajtkpocqboyjjfx")).withScope("njduyotqb"); model = BinaryData.fromObject(model).toObject(CustomEventsTriggerTypeProperties.class); - Assertions.assertEquals("povr", model.subjectBeginsWith()); - Assertions.assertEquals("otbybcx", model.subjectEndsWith()); - Assertions.assertEquals("pfgjnynuqikoiu", model.scope()); + Assertions.assertEquals("hkm", model.subjectBeginsWith()); + Assertions.assertEquals("raoytkkqoaazvmn", model.subjectEndsWith()); + Assertions.assertEquals("njduyotqb", model.scope()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandDefaultValueTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandDefaultValueTests.java index ea63d7c1846f7..0811b85b24e69 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandDefaultValueTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandDefaultValueTests.java @@ -11,14 +11,14 @@ public final class DWCopyCommandDefaultValueTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DWCopyCommandDefaultValue model - = BinaryData.fromString("{\"columnName\":\"dataryoleqikcorkem\",\"defaultValue\":\"dataobxfhtbtuvwzjycg\"}") + = BinaryData.fromString("{\"columnName\":\"dataubem\",\"defaultValue\":\"datauygmrenrbn\"}") .toObject(DWCopyCommandDefaultValue.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DWCopyCommandDefaultValue model = new DWCopyCommandDefaultValue().withColumnName("dataryoleqikcorkem") - .withDefaultValue("dataobxfhtbtuvwzjycg"); + DWCopyCommandDefaultValue model + = new DWCopyCommandDefaultValue().withColumnName("dataubem").withDefaultValue("datauygmrenrbn"); model = BinaryData.fromObject(model).toObject(DWCopyCommandDefaultValue.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandSettingsTests.java index c1f6e46fca4ab..dc9b1ef9d73af 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DWCopyCommandSettingsTests.java @@ -16,20 +16,19 @@ public final class DWCopyCommandSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DWCopyCommandSettings model = BinaryData.fromString( - "{\"defaultValues\":[{\"columnName\":\"databwsftytpjmufd\",\"defaultValue\":\"datahqlzantahuykxsjy\"}],\"additionalOptions\":{\"mpfeyjvlzqs\":\"xmoadwiqn\"}}") + "{\"defaultValues\":[{\"columnName\":\"datagows\",\"defaultValue\":\"dataguap\"}],\"additionalOptions\":{\"byfacexp\":\"hh\",\"pkqiqs\":\"pqykicesqpvmoxil\"}}") .toObject(DWCopyCommandSettings.class); - Assertions.assertEquals("xmoadwiqn", model.additionalOptions().get("mpfeyjvlzqs")); + Assertions.assertEquals("hh", model.additionalOptions().get("byfacexp")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DWCopyCommandSettings model - = new DWCopyCommandSettings() - .withDefaultValues(Arrays.asList(new DWCopyCommandDefaultValue().withColumnName("databwsftytpjmufd") - .withDefaultValue("datahqlzantahuykxsjy"))) - .withAdditionalOptions(mapOf("mpfeyjvlzqs", "xmoadwiqn")); + DWCopyCommandSettings model = new DWCopyCommandSettings() + .withDefaultValues( + Arrays.asList(new DWCopyCommandDefaultValue().withColumnName("datagows").withDefaultValue("dataguap"))) + .withAdditionalOptions(mapOf("byfacexp", "hh", "pkqiqs", "pqykicesqpvmoxil")); model = BinaryData.fromObject(model).toObject(DWCopyCommandSettings.class); - Assertions.assertEquals("xmoadwiqn", model.additionalOptions().get("mpfeyjvlzqs")); + Assertions.assertEquals("hh", model.additionalOptions().get("byfacexp")); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsAddDataFlowWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsAddDataFlowWithResponseMockTests.java index 96b4867e1744e..a3e793ac9e249 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsAddDataFlowWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsAddDataFlowWithResponseMockTests.java @@ -49,7 +49,7 @@ public void testAddDataFlowWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"jobVersion\":\"dryeucl\"}"; + String responseStr = "{\"jobVersion\":\"pywpednousxr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -66,85 +66,109 @@ public void testAddDataFlowWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - AddDataFlowToDebugSessionResponse response = manager.dataFlowDebugSessions() - .addDataFlowWithResponse("fhyiaxezpwhczqjo", "vypsgughwo", - new DataFlowDebugPackage().withSessionId("wzpxlx") - .withDataFlow(new DataFlowDebugResource().withName("zu") - .withProperties(new DataFlow().withDescription("hkabeo") - .withAnnotations(Arrays.asList("datao", "datatckmtqn")) - .withFolder(new DataFlowFolder().withName("rqcxhwvzdvujmuk")))) - .withDataFlows(Arrays.asList( - new DataFlowDebugResource().withName("lupsobqpd").withProperties( - new DataFlow().withDescription("qdda").withAnnotations(Arrays.asList("datansaecdcvhxwegd")) - .withFolder(new DataFlowFolder().withName("yphv"))), - new DataFlowDebugResource().withName("fexkbmodbpc") - .withProperties(new DataFlow().withDescription("pwpmyftvejxm") - .withAnnotations(Arrays.asList("datanahhpnbvzdf", "dataxjbzwvnxwdu", "datawdvbnpyed")) - .withFolder(new DataFlowFolder().withName("rtdaqlitoimta"))), - new DataFlowDebugResource().withName("o") - .withProperties(new DataFlow().withDescription("yhhzcjzgij") - .withAnnotations(Arrays.asList("datalloejshfcuzzu", "datacibvrfkxiixnxx", "datavyizya")) - .withFolder(new DataFlowFolder().withName("egijdejscrjc"))))) - .withDatasets( - Arrays - .asList( - new DatasetDebugResource().withName("fosrwzhmlklocyjp") - .withProperties(new Dataset().withDescription("hcbanzkwaat") - .withStructure("datarbaaqt").withSchema("datawzmdencqym") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("qnli") - .withParameters(mapOf("dm", "dataav", "qej", "datactkwxzrkcvbfeuc", - "cjkhexxn", "datazhteti"))) - .withParameters( - mapOf("ekonm", new ParameterSpecification().withType(ParameterType.STRING), - "riqfrrxbssvrh", new ParameterSpecification() - .withType(ParameterType.FLOAT), - "hszm", new ParameterSpecification().withType(ParameterType.OBJECT), - "f", new ParameterSpecification().withType(ParameterType.FLOAT))) - .withAnnotations(Arrays.asList("datasmicjkiz", "dataqdawmrkryixbbhj")) - .withFolder(new DatasetFolder().withName("lzdjzh")) - .withAdditionalProperties(mapOf("type", "Dataset"))))) - .withLinkedServices( - Arrays - .asList( - new LinkedServiceDebugResource() - .withName( - "envhlpuobhaomao") + AddDataFlowToDebugSessionResponse response + = manager.dataFlowDebugSessions() + .addDataFlowWithResponse("zobpxfgp", "dzdzswvfwiunj", + new DataFlowDebugPackage().withSessionId("xgztfzgd") + .withDataFlow(new DataFlowDebugResource().withName("wjcwwbunfymbwin") + .withProperties(new DataFlow().withDescription("vbiryxsa") + .withAnnotations(Arrays.asList("datag", "datansesxwkhkcd", "dataofakmopqfzvvtif", + "dataqsuemewfutovb")) + .withFolder(new DataFlowFolder().withName("fucxtmhmzcnpsd")))) + .withDataFlows(Arrays.asList( + new DataFlowDebugResource().withName("ayjeh") + .withProperties(new DataFlow().withDescription("t") + .withAnnotations(Arrays.asList("dataokttqgokhaj", "dataylkflf", + "dataofjskndwywbptvym", "datampdcddbe")) + .withFolder(new DataFlowFolder().withName("prlxxbmy"))), + new DataFlowDebugResource().withName("wzbkgtgvrrzmkte") + .withProperties(new DataFlow().withDescription("owvqpncifdxtibvq") + .withAnnotations(Arrays.asList("datapfdlcstucsw", "datamnsdw")) + .withFolder(new DataFlowFolder().withName("atfgoerjmhtxip"))), + new DataFlowDebugResource().withName("mlaile") + .withProperties(new DataFlow().withDescription("euxxtslhjcwlfz") + .withAnnotations(Arrays.asList("datawexcktg", "datacccgoik", "datajjskzuhdiyavfey", + "databyduyastybomiyj")) + .withFolder(new DataFlowFolder().withName("seemh"))), + new DataFlowDebugResource().withName("dfsteouzoglvt") + .withProperties(new DataFlow().withDescription("usiv") + .withAnnotations(Arrays.asList("dataunnjwmdtbx", "datatomcba", "dataamtdfpkfw")) + .withFolder(new DataFlowFolder().withName("elxd"))))) + .withDatasets(Arrays.asList( + new DatasetDebugResource().withName("oy") + .withProperties(new Dataset().withDescription("ejvlf").withStructure("datarqkgibpehqb") + .withSchema("datazcmqqehxigsi") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fb") + .withParameters(mapOf("lbzxyejoxd", "dataoaypixryf", "tbpiccriqhiwyk", + "datarjulttqgun", "mgparbirgw", "datazncfh", "irnfnlyvdryx", "datagewd"))) + .withParameters( + mapOf("tazuac", new ParameterSpecification().withType(ParameterType.ARRAY))) + .withAnnotations(Arrays.asList("dataslstekbbq", "datatvpsxycvoex")) + .withFolder(new DatasetFolder().withName("xrvxwlfmbbr")) + .withAdditionalProperties(mapOf("type", "Dataset"))), + new DatasetDebugResource().withName("hrxgvubsxajr").withProperties(new Dataset() + .withDescription("a").withStructure("datannlasf").withSchema("datajyvu") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("yexlpmbtm") + .withParameters(mapOf("amsgfvuffdvukjy", "datavbpnye"))) + .withParameters( + mapOf("nrgmjp", new ParameterSpecification().withType(ParameterType.INT), + "efwxcuzu", new ParameterSpecification().withType(ParameterType.FLOAT), + "bysyprskj", new ParameterSpecification().withType(ParameterType.OBJECT))) + .withAnnotations(Arrays.asList("datavbvkvomnoslbkrh", "datanvozjudg", "datadsflitmm")) + .withFolder(new DatasetFolder().withName("z")) + .withAdditionalProperties(mapOf("type", "Dataset"))))) + .withLinkedServices( + Arrays.asList( + new LinkedServiceDebugResource().withName("cxwkwkxln") .withProperties(new LinkedService() - .withConnectVia( - new IntegrationRuntimeReference().withReferenceName("vxo").withParameters( - mapOf("mfkiop", "datafkvdmjjiqjvufi", "vc", "datakhbfnhspogx"))) - .withDescription("ycrwnay") - .withParameters(mapOf("lfcfeyhnynkmp", - new ParameterSpecification().withType(ParameterType.BOOL), "nvnxsa", - new ParameterSpecification().withType(ParameterType.INT))) - .withAnnotations(Arrays.asList("dataouilbjccjor", "datavr", "datadfgdvifo")) + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("jvzpvisqf") + .withParameters(mapOf("kenrlcjms", "datalmxvmrdpfan", "dahhiid", + "datapgmebx", "jnoibclfqdtfj", "dataogakrpmjodbdcy"))) + .withDescription("v") + .withParameters(mapOf("yvstv", + new ParameterSpecification().withType(ParameterType.ARRAY), "slqyhabgocq", + new ParameterSpecification().withType(ParameterType.ARRAY), "li", + new ParameterSpecification().withType(ParameterType.FLOAT))) + .withAnnotations(Arrays.asList("datanhcklllwgrkvl", + "dataqkpxvemjjfvanefw")) .withAdditionalProperties(mapOf("type", "LinkedService"))), - new LinkedServiceDebugResource().withName("tvp").withProperties(new LinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("mwjsvuziog") - .withParameters(mapOf("xswfytnvcjhjrwn", "dataim"))) - .withDescription("tgc") - .withParameters( - mapOf("drhxfgswyafd", new ParameterSpecification().withType(ParameterType.INT))) - .withAnnotations(Arrays.asList("datayirjbfwrqivi", "datazoqgutrx")) - .withAdditionalProperties(mapOf("type", "LinkedService"))))) - .withStaging(new DataFlowStagingInfo() - .withLinkedService(new LinkedServiceReference().withReferenceName("iga") - .withParameters(mapOf("wmxmdjezhutc", "datajnrlfdqpaf"))) - .withFolderPath("dataqdchmxr")) - .withDebugSettings(new DataFlowDebugPackageDebugSettings() - .withSourceSettings(Arrays.asList( - new DataFlowSourceSetting().withSourceName("h").withRowLimit(1045259897) - .withAdditionalProperties(mapOf()), - new DataFlowSourceSetting().withSourceName("ctqnm").withRowLimit(519045355) - .withAdditionalProperties(mapOf()))) - .withParameters( - mapOf("v", "datav", "yubxexyydibfqrtp", "dataomjm", "mxlnt", "datapplzieovudvpyba")) - .withDatasetParameters("datapbln")) - .withAdditionalProperties(mapOf()), - com.azure.core.util.Context.NONE) - .getValue(); + new LinkedServiceDebugResource().withName("pbitcsbzuyfrann") + .withProperties(new LinkedService() + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("ba") + .withParameters(mapOf("mwbshqpjueo", "datanjxyproqebsuij", "krejuuii", + "datahtltooikzouv", "arxyh", "datarbtfarb", "xbis", "dataukc"))) + .withDescription("itjovjrirg") + .withParameters(mapOf("acb", + new ParameterSpecification().withType(ParameterType.BOOL), "ioxtqxrbrdpz", + new ParameterSpecification().withType(ParameterType.FLOAT), "yczlylud", + new ParameterSpecification().withType(ParameterType.FLOAT), "iaxigeo", + new ParameterSpecification().withType(ParameterType.OBJECT))) + .withAnnotations(Arrays.asList("datahkhnzsrgiwvze", "datag", "datajtuzqreprn")) + .withAdditionalProperties(mapOf("type", "LinkedService"))))) + .withStaging( + new DataFlowStagingInfo() + .withLinkedService(new LinkedServiceReference().withReferenceName("dymuehvvvrtsnc") + .withParameters(mapOf("pxfhixaagvkwe", "dataunhoogagtjcmly", "bgva", "datalcito", + "fnffjxdccwuzqwv", "datazfiwao", "bawzafzdzhh", "datakewlyrweups"))) + .withFolderPath("dataxcelvawwj")) + .withDebugSettings(new DataFlowDebugPackageDebugSettings() + .withSourceSettings(Arrays.asList( + new DataFlowSourceSetting().withSourceName("jv").withRowLimit(1429028510) + .withAdditionalProperties(mapOf()), + new DataFlowSourceSetting().withSourceName("pmlckzdwietfxpdz").withRowLimit(834316963) + .withAdditionalProperties(mapOf()), + new DataFlowSourceSetting().withSourceName("lygjtibhzjhqfuq").withRowLimit(490505781) + .withAdditionalProperties(mapOf()), + new DataFlowSourceSetting().withSourceName("yeczlxunhntsqsp").withRowLimit(1295527653) + .withAdditionalProperties(mapOf()))) + .withParameters(mapOf("veknwldqj", "dataversu", "cwngg", "datagzcwrhhgnmjxxov", "cwiz", + "datavjbgynpapzbbcfu")) + .withDatasetParameters("datam")) + .withAdditionalProperties(mapOf()), + com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("dryeucl", response.jobVersion()); + Assertions.assertEquals("pywpednousxr", response.jobVersion()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsCreateMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsCreateMockTests.java index f907df09a66da..377e227572d03 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsCreateMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsCreateMockTests.java @@ -35,7 +35,7 @@ public void testCreate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"status\":\"ztrmtimtfauylqpz\",\"sessionId\":\"ngfcbldpeforx\"}"; + String responseStr = "{\"status\":\"gaecujlaecwnn\",\"sessionId\":\"f\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,16 +52,17 @@ public void testCreate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - CreateDataFlowDebugSessionResponse response = manager.dataFlowDebugSessions().create("hengkwhkekxohqv", "pwzoq", - new CreateDataFlowDebugSessionRequest().withComputeType("mkjleelakvhgefvp").withCoreCount(384659902) - .withTimeToLive(37607510) - .withIntegrationRuntime(new IntegrationRuntimeDebugResource().withName("exu") - .withProperties(new IntegrationRuntime().withDescription("gqiprbqdtcibb") - .withAdditionalProperties(mapOf("type", "IntegrationRuntime")))), - com.azure.core.util.Context.NONE); + CreateDataFlowDebugSessionResponse response + = manager.dataFlowDebugSessions().create("zudysndiexb", "kwjmqnbaerggqaoh", + new CreateDataFlowDebugSessionRequest().withComputeType("qvb").withCoreCount(1071652867) + .withTimeToLive(834593701) + .withIntegrationRuntime(new IntegrationRuntimeDebugResource().withName("xgidjiijpdbwknbm") + .withProperties(new IntegrationRuntime().withDescription("n") + .withAdditionalProperties(mapOf("type", "IntegrationRuntime")))), + com.azure.core.util.Context.NONE); - Assertions.assertEquals("ztrmtimtfauylqpz", response.status()); - Assertions.assertEquals("ngfcbldpeforx", response.sessionId()); + Assertions.assertEquals("gaecujlaecwnn", response.status()); + Assertions.assertEquals("f", response.sessionId()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsDeleteWithResponseMockTests.java index 11967369d0f4f..860eda143f239 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsDeleteWithResponseMockTests.java @@ -46,8 +46,8 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.dataFlowDebugSessions().deleteWithResponse("qcwpgipttpsedt", "tu", - new DeleteDataFlowDebugSessionRequest().withSessionId("huusr"), com.azure.core.util.Context.NONE); + manager.dataFlowDebugSessions().deleteWithResponse("jlqdpqkcbflzzdd", "rew", + new DeleteDataFlowDebugSessionRequest().withSessionId("rsufvtmseuqguz"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsExecuteCommandMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsExecuteCommandMockTests.java index d14585f32e50b..977e6db353c83 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsExecuteCommandMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsExecuteCommandMockTests.java @@ -34,7 +34,7 @@ public void testExecuteCommand() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"status\":\"rlxnlruxjqqo\",\"data\":\"tnowvpdfy\"}"; + String responseStr = "{\"status\":\"lnjhoemlwea\",\"data\":\"xmshaugenpi\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,14 +51,15 @@ public void testExecuteCommand() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - DataFlowDebugCommandResponse response = manager.dataFlowDebugSessions().executeCommand("utonp", "tazpupkebwses", - new DataFlowDebugCommandRequest().withSessionId("sbrwdfoprdyt") - .withCommand(DataFlowDebugCommandType.EXECUTE_EXPRESSION_QUERY) - .withCommandPayload(new DataFlowDebugCommandPayload().withStreamName("vidzoo").withRowLimits(291421910) - .withColumns(Arrays.asList("tnpkjpcgtgnhzuf", "wf", "j", "vrptqxksidnj")).withExpression("ajgi")), + DataFlowDebugCommandResponse response = manager.dataFlowDebugSessions().executeCommand("rphdakwwiezeut", "r", + new DataFlowDebugCommandRequest().withSessionId("wmo") + .withCommand(DataFlowDebugCommandType.EXECUTE_PREVIEW_QUERY).withCommandPayload( + new DataFlowDebugCommandPayload().withStreamName("wzatvne").withRowLimits(1961533079) + .withColumns(Arrays.asList("qladlpqlwtxshvo", "hhzlmwvc", "hkvafcjektkg", "xrifyr")) + .withExpression("rgiaeqc")), com.azure.core.util.Context.NONE); - Assertions.assertEquals("rlxnlruxjqqo", response.status()); - Assertions.assertEquals("tnowvpdfy", response.data()); + Assertions.assertEquals("lnjhoemlwea", response.status()); + Assertions.assertEquals("xmshaugenpi", response.data()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsQueryByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsQueryByFactoryMockTests.java index 66b52d5356dc7..9d6028a1d6707 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsQueryByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowDebugSessionsQueryByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testQueryByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"dataFlowName\":\"d\",\"computeType\":\"wkjegu\",\"coreCount\":925368644,\"nodeCount\":773936710,\"integrationRuntimeName\":\"inwrhfrbwoylpme\",\"sessionId\":\"bblgwl\",\"startTime\":\"cxc\",\"timeToLiveInMinutes\":292501035,\"lastActivityTime\":\"orhrtihzwd\",\"\":{\"qvkuuyehmn\":\"datawlmhqavmfosfpgqu\"}}]}"; + = "{\"value\":[{\"dataFlowName\":\"dijyfiabhepxqtk\",\"computeType\":\"lizno\",\"coreCount\":349109970,\"nodeCount\":1834406842,\"integrationRuntimeName\":\"qythsltodlvwbgbm\",\"sessionId\":\"trsxhiuhgvgno\",\"startTime\":\"sqf\",\"timeToLiveInMinutes\":1683333500,\"lastActivityTime\":\"ppwvieymkguvr\",\"\":{\"nelqcvmvpp\":\"dataproytd\",\"is\":\"datatsolxnhlrpsign\"}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,17 +49,17 @@ public void testQueryByFactory() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.dataFlowDebugSessions() - .queryByFactory("kpmzkdisrgykrc", "voiv", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.dataFlowDebugSessions().queryByFactory("ijlvkrnso", "oh", com.azure.core.util.Context.NONE); - Assertions.assertEquals("d", response.iterator().next().dataFlowName()); - Assertions.assertEquals("wkjegu", response.iterator().next().computeType()); - Assertions.assertEquals(925368644, response.iterator().next().coreCount()); - Assertions.assertEquals(773936710, response.iterator().next().nodeCount()); - Assertions.assertEquals("inwrhfrbwoylpme", response.iterator().next().integrationRuntimeName()); - Assertions.assertEquals("bblgwl", response.iterator().next().sessionId()); - Assertions.assertEquals("cxc", response.iterator().next().startTime()); - Assertions.assertEquals(292501035, response.iterator().next().timeToLiveInMinutes()); - Assertions.assertEquals("orhrtihzwd", response.iterator().next().lastActivityTime()); + Assertions.assertEquals("dijyfiabhepxqtk", response.iterator().next().dataFlowName()); + Assertions.assertEquals("lizno", response.iterator().next().computeType()); + Assertions.assertEquals(349109970, response.iterator().next().coreCount()); + Assertions.assertEquals(1834406842, response.iterator().next().nodeCount()); + Assertions.assertEquals("qythsltodlvwbgbm", response.iterator().next().integrationRuntimeName()); + Assertions.assertEquals("trsxhiuhgvgno", response.iterator().next().sessionId()); + Assertions.assertEquals("sqf", response.iterator().next().startTime()); + Assertions.assertEquals(1683333500, response.iterator().next().timeToLiveInMinutes()); + Assertions.assertEquals("ppwvieymkguvr", response.iterator().next().lastActivityTime()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsCreateOrUpdateWithResponseMockTests.java index 39c7f8085bad9..547c75b7b5f65 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsCreateOrUpdateWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"DataFlow\",\"description\":\"rbqpzgsrphbfyff\",\"annotations\":[\"datarnvwaxmeyjim\"],\"folder\":{\"name\":\"hcq\"}},\"name\":\"q\",\"type\":\"aj\",\"etag\":\"maimwufuvtyp\",\"id\":\"fllrieknxbcgn\"}"; + = "{\"properties\":{\"type\":\"DataFlow\",\"description\":\"lyzbgrg\",\"annotations\":[\"datavkasxs\",\"dataxwgzoyjp\",\"dataelv\",\"datamyboremswqznqovb\"],\"folder\":{\"name\":\"ayiivoixefn\"}},\"name\":\"fewqnznctnmkits\",\"type\":\"uopevqsab\",\"etag\":\"ounugxnz\",\"id\":\"eggalvo\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,15 +51,14 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - DataFlowResource response - = manager.dataFlows().define("rhiinata").withExistingFactory("jpy", "alt") - .withProperties(new DataFlow().withDescription("i") - .withAnnotations(Arrays.asList("databkciri", "datattzyruvgihwiezcf")) - .withFolder(new DataFlowFolder().withName("dplkuyouqnftd"))) - .withIfMatch("nychw").create(); + DataFlowResource response = manager.dataFlows().define("ykommmygmit").withExistingFactory("bypxxe", "ju") + .withProperties(new DataFlow().withDescription("vxvdugnbdeg") + .withAnnotations(Arrays.asList("datahobtxu", "dataebberydeoeyef")) + .withFolder(new DataFlowFolder().withName("bhqieytu"))) + .withIfMatch("srhqzjzuonttfvj").create(); - Assertions.assertEquals("fllrieknxbcgn", response.id()); - Assertions.assertEquals("rbqpzgsrphbfyff", response.properties().description()); - Assertions.assertEquals("hcq", response.properties().folder().name()); + Assertions.assertEquals("eggalvo", response.id()); + Assertions.assertEquals("lyzbgrg", response.properties().description()); + Assertions.assertEquals("ayiivoixefn", response.properties().folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsDeleteWithResponseMockTests.java index 14f4590f253e1..28f933103e6db 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsDeleteWithResponseMockTests.java @@ -45,8 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.dataFlows().deleteWithResponse("uuldztvumvxk", "xgaidd", "dgkhiqwuwx", - com.azure.core.util.Context.NONE); + manager.dataFlows().deleteWithResponse("r", "gbbiuk", "kmthioae", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsGetWithResponseMockTests.java index cc0de6d18a454..9aefb9f94f9b1 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"DataFlow\",\"description\":\"rstannmjp\",\"annotations\":[\"datatlupzshgsidkzi\",\"dataarpsr\"],\"folder\":{\"name\":\"huokjwvsacwdu\"}},\"name\":\"zuykly\",\"type\":\"skelevard\",\"etag\":\"drtjakezorhjh\",\"id\":\"qh\"}"; + = "{\"properties\":{\"type\":\"DataFlow\",\"description\":\"wvdmwaam\",\"annotations\":[\"dataqsb\"],\"folder\":{\"name\":\"nckeqxtqaomihr\"}},\"name\":\"ksdpc\",\"type\":\"wficzzoxnlvi\",\"etag\":\"dsi\",\"id\":\"zaolzkoyn\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); DataFlowResource response = manager.dataFlows() - .getWithResponse("gjsj", "lhwbypvpd", "vycjuxa", "puphgogmggkkjc", com.azure.core.util.Context.NONE) + .getWithResponse("lruxjqqozx", "nowvpdfypd", "syym", "uueeokvq", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("qh", response.id()); - Assertions.assertEquals("rstannmjp", response.properties().description()); - Assertions.assertEquals("huokjwvsacwdu", response.properties().folder().name()); + Assertions.assertEquals("zaolzkoyn", response.id()); + Assertions.assertEquals("wvdmwaam", response.properties().description()); + Assertions.assertEquals("nckeqxtqaomihr", response.properties().folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsListByFactoryMockTests.java index 02386c5c11066..65e720e02f5d2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataFlowsListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"type\":\"DataFlow\",\"description\":\"yfkrcmxtbwolzu\",\"annotations\":[\"dataqlszlymy\",\"datapwjbuwqm\",\"datacu\"],\"folder\":{\"name\":\"zmhcvrfqqmbuvt\"}},\"name\":\"rmcymwr\",\"type\":\"kaztuldg\",\"etag\":\"edvxhqhptn\",\"id\":\"pafurttshr\"}]}"; + = "{\"value\":[{\"properties\":{\"type\":\"DataFlow\",\"description\":\"lkvmftrqa\",\"annotations\":[\"dataq\",\"datao\",\"datatqvnkofvz\",\"databddvqtruyzbrkzsa\"],\"folder\":{\"name\":\"nsepdwxflmk\"}},\"name\":\"fidr\",\"type\":\"nuththzejkusz\",\"etag\":\"aehtgzr\",\"id\":\"ctrvvjamxgq\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,10 +50,10 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.dataFlows().listByFactory("cydmkyooj", "fsbygmbn", com.azure.core.util.Context.NONE); + = manager.dataFlows().listByFactory("h", "mlcenty", com.azure.core.util.Context.NONE); - Assertions.assertEquals("pafurttshr", response.iterator().next().id()); - Assertions.assertEquals("yfkrcmxtbwolzu", response.iterator().next().properties().description()); - Assertions.assertEquals("zmhcvrfqqmbuvt", response.iterator().next().properties().folder().name()); + Assertions.assertEquals("ctrvvjamxgq", response.iterator().next().id()); + Assertions.assertEquals("lkvmftrqa", response.iterator().next().properties().description()); + Assertions.assertEquals("nsepdwxflmk", response.iterator().next().properties().folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTests.java index 3c0ea840b6486..f2159cc40258d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTests.java @@ -22,59 +22,57 @@ public final class DataLakeAnalyticsUsqlActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DataLakeAnalyticsUsqlActivity model = BinaryData.fromString( - "{\"type\":\"DataLakeAnalyticsU-SQL\",\"typeProperties\":{\"scriptPath\":\"datamvxnumvorosq\",\"scriptLinkedService\":{\"referenceName\":\"sspw\",\"parameters\":{\"spw\":\"datajr\",\"umqmor\":\"dataiitzbyue\"}},\"degreeOfParallelism\":\"datapsflmwduisrvlun\",\"priority\":\"dataenererzthcfnrles\",\"parameters\":{\"czzlkmtrrcbulvau\":\"datacfqzmjmf\",\"rqjsmhkqzvarqi\":\"datakgklqucxewcd\"},\"runtimeVersion\":\"datadllhi\",\"compilationMode\":\"datanvxhxzabxhmdorx\"},\"linkedServiceName\":{\"referenceName\":\"aprksoeq\",\"parameters\":{\"w\":\"datatibadrcyexbjaktg\"}},\"policy\":{\"timeout\":\"datashghorgjidr\",\"retry\":\"dataqcwcdbt\",\"retryIntervalInSeconds\":1188826110,\"secureInput\":true,\"secureOutput\":true,\"\":{\"imhjbxwr\":\"datasuaazkouvvgcw\",\"z\":\"datagaofwo\",\"bnx\":\"dataxp\",\"ys\":\"datam\"}},\"name\":\"shbuxjhqu\",\"description\":\"rhcghnclfahryuzt\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"tcptpqoajgg\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Skipped\",\"Skipped\"],\"\":{\"vnjzaa\":\"datalsxzfycnp\",\"axfjuzgslqpzdxw\":\"dataxwcptoihoyv\"}}],\"userProperties\":[{\"name\":\"lgczvfb\",\"value\":\"datazscepogg\"},{\"name\":\"ppufueiaie\",\"value\":\"dataexyiuhjqd\"},{\"name\":\"lxabtlmszq\",\"value\":\"dataudai\"},{\"name\":\"fqnxjkopivsz\",\"value\":\"datajbpt\"}],\"\":{\"vlo\":\"dataabzjem\",\"nbqsjzncg\":\"datauca\",\"y\":\"datahpqgi\",\"xnxgv\":\"dataojimussvurslw\"}}") + "{\"type\":\"DataLakeAnalyticsU-SQL\",\"typeProperties\":{\"scriptPath\":\"dataobz\",\"scriptLinkedService\":{\"referenceName\":\"sqpstxulnntj\",\"parameters\":{\"niac\":\"datan\",\"llk\":\"datattdyvifltvwebzf\",\"cerqhp\":\"datanwinqywlvxuxztj\"}},\"degreeOfParallelism\":\"datakxjlyjlkjhmug\",\"priority\":\"datayhpdstlsdgi\",\"parameters\":{\"glzxgwk\":\"dataeqcgunsoikev\",\"qxnkdqsyhmrk\":\"dataykazpd\",\"z\":\"datawkfgvhwkwzxj\",\"xduhydxahjudaz\":\"datastirrhbkzzqwikq\"},\"runtimeVersion\":\"datagsxolwofo\",\"compilationMode\":\"datayludflfxwlwh\"},\"linkedServiceName\":{\"referenceName\":\"ykfcccaujgacckjq\",\"parameters\":{\"xbvg\":\"dataxdbgm\",\"p\":\"datamblntdy\"}},\"policy\":{\"timeout\":\"dataigxefscsrw\",\"retry\":\"datauteusuxvliq\",\"retryIntervalInSeconds\":1459080325,\"secureInput\":false,\"secureOutput\":false,\"\":{\"gizvvtdrjockz\":\"datazzsbqnv\",\"ppjzmpxam\":\"datafnph\"}},\"name\":\"qdostvx\",\"description\":\"fnmnfndrbkko\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"qfze\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Failed\",\"Failed\"],\"\":{\"kvn\":\"datadr\",\"arvhzfynbxw\":\"datazumczlknfwslvs\"}}],\"userProperties\":[{\"name\":\"mvlkuvbesrawzxnw\",\"value\":\"datasjygi\"}],\"\":{\"slcnsxhpqey\":\"datafo\",\"chdjarfdfnqfvr\":\"datazydpvv\",\"n\":\"dataxlh\"}}") .toObject(DataLakeAnalyticsUsqlActivity.class); - Assertions.assertEquals("shbuxjhqu", model.name()); - Assertions.assertEquals("rhcghnclfahryuzt", model.description()); + Assertions.assertEquals("qdostvx", model.name()); + Assertions.assertEquals("fnmnfndrbkko", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("tcptpqoajgg", model.dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("qfze", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("lgczvfb", model.userProperties().get(0).name()); - Assertions.assertEquals("aprksoeq", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1188826110, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("sspw", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("mvlkuvbesrawzxnw", model.userProperties().get(0).name()); + Assertions.assertEquals("ykfcccaujgacckjq", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1459080325, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("sqpstxulnntj", model.scriptLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DataLakeAnalyticsUsqlActivity model - = new DataLakeAnalyticsUsqlActivity().withName("shbuxjhqu").withDescription("rhcghnclfahryuzt") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("tcptpqoajgg") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("lgczvfb").withValue("datazscepogg"), - new UserProperty().withName("ppufueiaie").withValue("dataexyiuhjqd"), - new UserProperty().withName("lxabtlmszq").withValue("dataudai"), - new UserProperty().withName("fqnxjkopivsz").withValue("datajbpt"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("aprksoeq") - .withParameters(mapOf("w", "datatibadrcyexbjaktg"))) - .withPolicy(new ActivityPolicy() - .withTimeout("datashghorgjidr").withRetry("dataqcwcdbt").withRetryIntervalInSeconds(1188826110) - .withSecureInput(true).withSecureOutput(true).withAdditionalProperties(mapOf())) - .withScriptPath("datamvxnumvorosq") - .withScriptLinkedService(new LinkedServiceReference().withReferenceName("sspw") - .withParameters(mapOf("spw", "datajr", "umqmor", "dataiitzbyue"))) - .withDegreeOfParallelism("datapsflmwduisrvlun").withPriority("dataenererzthcfnrles") - .withParameters(mapOf("czzlkmtrrcbulvau", "datacfqzmjmf", "rqjsmhkqzvarqi", "datakgklqucxewcd")) - .withRuntimeVersion("datadllhi").withCompilationMode("datanvxhxzabxhmdorx"); + DataLakeAnalyticsUsqlActivity model = new DataLakeAnalyticsUsqlActivity().withName("qdostvx") + .withDescription("fnmnfndrbkko").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("qfze") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("mvlkuvbesrawzxnw").withValue("datasjygi"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ykfcccaujgacckjq") + .withParameters(mapOf("xbvg", "dataxdbgm", "p", "datamblntdy"))) + .withPolicy(new ActivityPolicy() + .withTimeout("dataigxefscsrw").withRetry("datauteusuxvliq").withRetryIntervalInSeconds(1459080325) + .withSecureInput(false).withSecureOutput(false).withAdditionalProperties(mapOf())) + .withScriptPath("dataobz") + .withScriptLinkedService(new LinkedServiceReference().withReferenceName("sqpstxulnntj") + .withParameters(mapOf("niac", "datan", "llk", "datattdyvifltvwebzf", "cerqhp", "datanwinqywlvxuxztj"))) + .withDegreeOfParallelism("datakxjlyjlkjhmug").withPriority("datayhpdstlsdgi") + .withParameters(mapOf("glzxgwk", "dataeqcgunsoikev", "qxnkdqsyhmrk", "dataykazpd", "z", "datawkfgvhwkwzxj", + "xduhydxahjudaz", "datastirrhbkzzqwikq")) + .withRuntimeVersion("datagsxolwofo").withCompilationMode("datayludflfxwlwh"); model = BinaryData.fromObject(model).toObject(DataLakeAnalyticsUsqlActivity.class); - Assertions.assertEquals("shbuxjhqu", model.name()); - Assertions.assertEquals("rhcghnclfahryuzt", model.description()); + Assertions.assertEquals("qdostvx", model.name()); + Assertions.assertEquals("fnmnfndrbkko", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("tcptpqoajgg", model.dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("qfze", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("lgczvfb", model.userProperties().get(0).name()); - Assertions.assertEquals("aprksoeq", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1188826110, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("sspw", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("mvlkuvbesrawzxnw", model.userProperties().get(0).name()); + Assertions.assertEquals("ykfcccaujgacckjq", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1459080325, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("sqpstxulnntj", model.scriptLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTypePropertiesTests.java index 87e933d95c160..18ff03965117e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DataLakeAnalyticsUsqlActivityTypePropertiesTests.java @@ -15,22 +15,23 @@ public final class DataLakeAnalyticsUsqlActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DataLakeAnalyticsUsqlActivityTypeProperties model = BinaryData.fromString( - "{\"scriptPath\":\"datalvkda\",\"scriptLinkedService\":{\"referenceName\":\"lksoqr\",\"parameters\":{\"twsxliwpzuce\":\"dataanw\",\"rvtrwswbm\":\"datazis\"}},\"degreeOfParallelism\":\"datab\",\"priority\":\"databtthzfgpzy\",\"parameters\":{\"jbqfclijec\":\"datausehyvq\"},\"runtimeVersion\":\"datatuoyxdlzxuakba\",\"compilationMode\":\"datakrxnr\"}") + "{\"scriptPath\":\"databvbdqmjcedfpub\",\"scriptLinkedService\":{\"referenceName\":\"xoohyesmlscvhra\",\"parameters\":{\"xhkdyhdk\":\"databorj\",\"qfde\":\"datafqzud\",\"tplpgf\":\"datag\",\"xlibszcvceg\":\"datazugkfabvek\"}},\"degreeOfParallelism\":\"datazh\",\"priority\":\"datavv\",\"parameters\":{\"vxnumvorosqessp\":\"datadpclazaoytkub\",\"rqspwsiitzbyu\":\"datautk\",\"ovpsflmwduis\":\"datalumqmo\",\"e\":\"datavlunyqe\"},\"runtimeVersion\":\"datarzth\",\"compilationMode\":\"datanrlesghhcfqzmjm\"}") .toObject(DataLakeAnalyticsUsqlActivityTypeProperties.class); - Assertions.assertEquals("lksoqr", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("xoohyesmlscvhra", model.scriptLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DataLakeAnalyticsUsqlActivityTypeProperties model - = new DataLakeAnalyticsUsqlActivityTypeProperties().withScriptPath("datalvkda") - .withScriptLinkedService(new LinkedServiceReference().withReferenceName("lksoqr") - .withParameters(mapOf("twsxliwpzuce", "dataanw", "rvtrwswbm", "datazis"))) - .withDegreeOfParallelism("datab").withPriority("databtthzfgpzy") - .withParameters(mapOf("jbqfclijec", "datausehyvq")).withRuntimeVersion("datatuoyxdlzxuakba") - .withCompilationMode("datakrxnr"); + DataLakeAnalyticsUsqlActivityTypeProperties model = new DataLakeAnalyticsUsqlActivityTypeProperties() + .withScriptPath("databvbdqmjcedfpub") + .withScriptLinkedService(new LinkedServiceReference().withReferenceName("xoohyesmlscvhra").withParameters( + mapOf("xhkdyhdk", "databorj", "qfde", "datafqzud", "tplpgf", "datag", "xlibszcvceg", "datazugkfabvek"))) + .withDegreeOfParallelism("datazh").withPriority("datavv") + .withParameters(mapOf("vxnumvorosqessp", "datadpclazaoytkub", "rqspwsiitzbyu", "datautk", "ovpsflmwduis", + "datalumqmo", "e", "datavlunyqe")) + .withRuntimeVersion("datarzth").withCompilationMode("datanrlesghhcfqzmjm"); model = BinaryData.fromObject(model).toObject(DataLakeAnalyticsUsqlActivityTypeProperties.class); - Assertions.assertEquals("lksoqr", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("xoohyesmlscvhra", model.scriptLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTests.java index 8cf1573f8edd4..31847d1986a0a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTests.java @@ -22,54 +22,65 @@ public final class DatabricksNotebookActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DatabricksNotebookActivity model = BinaryData.fromString( - "{\"type\":\"DatabricksNotebook\",\"typeProperties\":{\"notebookPath\":\"datapxdjkwyzqn\",\"baseParameters\":{\"vjkaciqzagfkk\":\"dataym\",\"gqfl\":\"dataywdtgz\"},\"libraries\":[{\"fr\":\"dataak\",\"o\":\"dataoeshqttkqcpc\"}]},\"linkedServiceName\":{\"referenceName\":\"ce\",\"parameters\":{\"orn\":\"datasmnyfahidlscdow\",\"pirgdsqcbxkw\":\"datayjqzjtdkojbxkodc\"}},\"policy\":{\"timeout\":\"datasybw\",\"retry\":\"dataifgjz\",\"retryIntervalInSeconds\":20627567,\"secureInput\":false,\"secureOutput\":true,\"\":{\"rbirv\":\"dataga\"}},\"name\":\"xubbnb\",\"description\":\"yeggaauubkr\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"bxjpytkakhvao\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Completed\",\"Succeeded\"],\"\":{\"jccgg\":\"datapxbqujnorfxi\",\"uraqpcspsbrd\":\"datazvdqpxicpozzhf\",\"ufkzuzzqud\":\"datadomyqbeasbv\",\"wbmfq\":\"datagf\"}}],\"userProperties\":[{\"name\":\"aqltoxhfphaw\",\"value\":\"dataovqtvbusyqyfit\"},{\"name\":\"prbmmfqteox\",\"value\":\"dataikdcjmbwrhpw\"},{\"name\":\"udegykzdspbjks\",\"value\":\"datadsrwhjhivgeran\"}],\"\":{\"jtqqqcxr\":\"dataaf\",\"cgyvzpvz\":\"datawduspxijrremvz\",\"kq\":\"dataduzfybjucfsu\"}}") + "{\"type\":\"DatabricksNotebook\",\"typeProperties\":{\"notebookPath\":\"datafvmfyxle\",\"baseParameters\":{\"mqzndlgqtuqjczco\":\"datakiaybdivxvxwd\",\"fc\":\"datactcwtxa\",\"rzsninkhbm\":\"dataqhmsdod\",\"yt\":\"datalfo\"},\"libraries\":[{\"psepkr\":\"datazxm\",\"mndbbpjdgnjmelxk\":\"datagerqzxkpxrfulqh\",\"ogfaiy\":\"datamfmgboyliopb\"}]},\"linkedServiceName\":{\"referenceName\":\"pfebsu\",\"parameters\":{\"mrickuh\":\"datasrxnneqxsdu\",\"os\":\"databrvhrs\",\"mv\":\"datazvclzutvqkoi\"}},\"policy\":{\"timeout\":\"datakqwjlohkaffynyw\",\"retry\":\"dataqbyty\",\"retryIntervalInSeconds\":1675626334,\"secureInput\":false,\"secureOutput\":false,\"\":{\"vxegiufjnjgupjjp\":\"datapaxka\"}},\"name\":\"ba\",\"description\":\"ftiwbdm\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"xxgocpzqrbtyza\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\"],\"\":{\"wxoog\":\"datauafxpjqz\",\"tvp\":\"dataxgnplz\",\"fvvi\":\"dataigtnjye\",\"lyyvebpykz\":\"dataxoitnqmiwlrijex\"}},{\"activity\":\"rqusb\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Failed\"],\"\":{\"hkyluqxndmtasx\":\"datanxepapmvvkx\",\"lofkvshozjkwjwv\":\"datanbttefbbr\"}},{\"activity\":\"oh\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Skipped\",\"Skipped\"],\"\":{\"i\":\"datastyacbekcxe\",\"kpiedcrtvdcbzpyn\":\"datavbzyhexlh\"}},{\"activity\":\"dtsibtdm\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"vjbpyvoswgkbz\":\"dataawehxshamzfxoc\",\"uiags\":\"datameftgvfu\",\"deykv\":\"datavzghnq\",\"gqdcaimsrryt\":\"datafhuihotz\"}}],\"userProperties\":[{\"name\":\"hbpcrdy\",\"value\":\"dataunrajtbum\"}],\"\":{\"vyutcvumvgttjvc\":\"datarsn\",\"jpgxuxkc\":\"datatltt\",\"biaoaqww\":\"datanxkdqqo\"}}") .toObject(DatabricksNotebookActivity.class); - Assertions.assertEquals("xubbnb", model.name()); - Assertions.assertEquals("yeggaauubkr", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("bxjpytkakhvao", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("aqltoxhfphaw", model.userProperties().get(0).name()); - Assertions.assertEquals("ce", model.linkedServiceName().referenceName()); - Assertions.assertEquals(20627567, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("ba", model.name()); + Assertions.assertEquals("ftiwbdm", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("xxgocpzqrbtyza", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("hbpcrdy", model.userProperties().get(0).name()); + Assertions.assertEquals("pfebsu", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1675626334, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabricksNotebookActivity model = new DatabricksNotebookActivity().withName("xubbnb") - .withDescription("yeggaauubkr").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("bxjpytkakhvao") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("aqltoxhfphaw").withValue("dataovqtvbusyqyfit"), - new UserProperty().withName("prbmmfqteox").withValue("dataikdcjmbwrhpw"), - new UserProperty().withName("udegykzdspbjks").withValue("datadsrwhjhivgeran"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ce") - .withParameters(mapOf("orn", "datasmnyfahidlscdow", "pirgdsqcbxkw", "datayjqzjtdkojbxkodc"))) - .withPolicy( - new ActivityPolicy().withTimeout("datasybw").withRetry("dataifgjz").withRetryIntervalInSeconds(20627567) - .withSecureInput(false).withSecureOutput(true).withAdditionalProperties(mapOf())) - .withNotebookPath("datapxdjkwyzqn") - .withBaseParameters(mapOf("vjkaciqzagfkk", "dataym", "gqfl", "dataywdtgz")) - .withLibraries(Arrays.asList(mapOf("fr", "dataak", "o", "dataoeshqttkqcpc"))); + DatabricksNotebookActivity model = new DatabricksNotebookActivity().withName("ba").withDescription("ftiwbdm") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("xxgocpzqrbtyza") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, + DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("rqusb") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("oh") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, + DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("dtsibtdm") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("hbpcrdy").withValue("dataunrajtbum"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pfebsu") + .withParameters(mapOf("mrickuh", "datasrxnneqxsdu", "os", "databrvhrs", "mv", "datazvclzutvqkoi"))) + .withPolicy(new ActivityPolicy() + .withTimeout("datakqwjlohkaffynyw").withRetry("dataqbyty").withRetryIntervalInSeconds(1675626334) + .withSecureInput(false).withSecureOutput(false).withAdditionalProperties(mapOf())) + .withNotebookPath("datafvmfyxle") + .withBaseParameters(mapOf("mqzndlgqtuqjczco", "datakiaybdivxvxwd", "fc", "datactcwtxa", "rzsninkhbm", + "dataqhmsdod", "yt", "datalfo")) + .withLibraries(Arrays.asList( + mapOf("psepkr", "datazxm", "mndbbpjdgnjmelxk", "datagerqzxkpxrfulqh", "ogfaiy", "datamfmgboyliopb"))); model = BinaryData.fromObject(model).toObject(DatabricksNotebookActivity.class); - Assertions.assertEquals("xubbnb", model.name()); - Assertions.assertEquals("yeggaauubkr", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("bxjpytkakhvao", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("aqltoxhfphaw", model.userProperties().get(0).name()); - Assertions.assertEquals("ce", model.linkedServiceName().referenceName()); - Assertions.assertEquals(20627567, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("ba", model.name()); + Assertions.assertEquals("ftiwbdm", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("xxgocpzqrbtyza", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("hbpcrdy", model.userProperties().get(0).name()); + Assertions.assertEquals("pfebsu", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1675626334, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTypePropertiesTests.java index f27e23417cba3..c884ebac1844c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksNotebookActivityTypePropertiesTests.java @@ -14,16 +14,20 @@ public final class DatabricksNotebookActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DatabricksNotebookActivityTypeProperties model = BinaryData.fromString( - "{\"notebookPath\":\"datagfyjwxwpoywymt\",\"baseParameters\":{\"x\":\"datadgbg\",\"gia\":\"datazrzhkhmw\",\"nlzalsuj\":\"datarftpgqxnyoakd\"},\"libraries\":[{\"bmfejtdboa\":\"datasekbce\"}]}") + "{\"notebookPath\":\"dataixnuffraursqtj\",\"baseParameters\":{\"akkgqfkigukfximw\":\"databhrdpnzqqtipkrea\",\"eriqendtyccn\":\"datanwaymrlvhl\",\"bizjbwufjogswf\":\"datahszgaub\"},\"libraries\":[{\"etaydhfgxyd\":\"databpypwrvnv\",\"hjsraumawfzyvxk\":\"databsx\",\"ofpsr\":\"datat\",\"ijbolksehtyx\":\"dataoujkcpyerfsngt\"},{\"rzdqqo\":\"datasurfnktxht\",\"cttjibognhuqdkq\":\"datadralt\",\"fppwobhkqgb\":\"dataffcvahknv\"}]}") .toObject(DatabricksNotebookActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { DatabricksNotebookActivityTypeProperties model - = new DatabricksNotebookActivityTypeProperties().withNotebookPath("datagfyjwxwpoywymt") - .withBaseParameters(mapOf("x", "datadgbg", "gia", "datazrzhkhmw", "nlzalsuj", "datarftpgqxnyoakd")) - .withLibraries(Arrays.asList(mapOf("bmfejtdboa", "datasekbce"))); + = new DatabricksNotebookActivityTypeProperties().withNotebookPath("dataixnuffraursqtj") + .withBaseParameters(mapOf("akkgqfkigukfximw", "databhrdpnzqqtipkrea", "eriqendtyccn", "datanwaymrlvhl", + "bizjbwufjogswf", "datahszgaub")) + .withLibraries(Arrays.asList( + mapOf("etaydhfgxyd", "databpypwrvnv", "hjsraumawfzyvxk", "databsx", "ofpsr", "datat", + "ijbolksehtyx", "dataoujkcpyerfsngt"), + mapOf("rzdqqo", "datasurfnktxht", "cttjibognhuqdkq", "datadralt", "fppwobhkqgb", "dataffcvahknv"))); model = BinaryData.fromObject(model).toObject(DatabricksNotebookActivityTypeProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTests.java index 72abffab3ec8e..305e56dda92c7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTests.java @@ -22,64 +22,57 @@ public final class DatabricksSparkJarActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DatabricksSparkJarActivity model = BinaryData.fromString( - "{\"type\":\"DatabricksSparkJar\",\"typeProperties\":{\"mainClassName\":\"datanya\",\"parameters\":[\"datapgbhf\",\"dataypykjorlrj\",\"datarzxa\"],\"libraries\":[{\"zahjqslshceyhal\":\"datahka\",\"fcoatsupaqzitho\":\"dataxrmgsn\",\"mtke\":\"datartcdavlri\",\"xirsvjozexxzkci\":\"dataezwkparj\"},{\"dsjblagss\":\"datakeawrumhz\",\"hiwqiqx\":\"datadubqhafxlbomwn\",\"vmzs\":\"datawbormfnntpocf\"},{\"ttswpeaivbzr\":\"dataohytkhquhdyzuehq\",\"qameccuqk\":\"datasoeddwjimrzavcif\"},{\"rbcvhkkdsyx\":\"datatfiomdie\"}]},\"linkedServiceName\":{\"referenceName\":\"ddj\",\"parameters\":{\"wlbccxjbaloihj\":\"datahvpqtxluqp\",\"vanxuiiprf\":\"dataknfvpavraeeiboq\",\"xs\":\"datajmilose\"}},\"policy\":{\"timeout\":\"datajzxeswzgr\",\"retry\":\"datagggj\",\"retryIntervalInSeconds\":1682173257,\"secureInput\":true,\"secureOutput\":false,\"\":{\"uttngatglarczzgu\":\"datagojoemtweh\",\"ceujuclffpvdjfw\":\"datarkrfabffeahypjqa\",\"nhcxh\":\"dataibplbtmwaexybr\",\"heuwcmwixyrv\":\"datashabnpdnbtym\"}},\"name\":\"pu\",\"description\":\"u\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"eihmv\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Skipped\",\"Completed\"],\"\":{\"ynvgfaotokipndek\":\"dataoxqjhmfyvgmdwvvs\",\"x\":\"datazgdrkddzkkikrotn\",\"ejtqoxethrxlpgrv\":\"dataeqdinwq\"}},{\"activity\":\"zjxmnsrejqwy\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"e\":\"dataviearbkdaomxyxn\",\"mrceldfbdxw\":\"datahxtxhxfsk\",\"wbwoflfnii\":\"datawdyqp\",\"cyvtsgopmatu\":\"datalohftmfmxrxrya\"}},{\"activity\":\"tejipqynrlnqo\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\",\"Completed\"],\"\":{\"kvfthbnikoyb\":\"dataxthcdzeuck\",\"zhikub\":\"datasofpwqm\"}}],\"userProperties\":[{\"name\":\"szoxmzvlof\",\"value\":\"datadnvsrvl\"},{\"name\":\"wzuj\",\"value\":\"datafcxp\"},{\"name\":\"x\",\"value\":\"dataqe\"},{\"name\":\"kwcxtxgrhaqbstod\",\"value\":\"datauvsbsdcoqm\"}],\"\":{\"wrjrxccnfykn\":\"dataeqlwkpvsi\",\"mlldeksgejmp\":\"datauocceyeqsehk\"}}") + "{\"type\":\"DatabricksSparkJar\",\"typeProperties\":{\"mainClassName\":\"datajzoi\",\"parameters\":[\"dataizlscnknkukempa\",\"datavajbgpu\"],\"libraries\":[{\"xujl\":\"datakankzyqi\"},{\"oqrutbfkynwwmlzp\":\"datahtrgybfumo\",\"n\":\"datarzazkaljuvm\",\"bfntgsju\":\"datalbzloae\",\"gix\":\"datagueggph\"},{\"qgkvkoynjucmyj\":\"datagvwmvafhriua\"},{\"v\":\"dataafvvnd\"}]},\"linkedServiceName\":{\"referenceName\":\"rqenbgymgjneohx\",\"parameters\":{\"yjs\":\"datazbhgbfsjodsk\",\"rfih\":\"dataxgefk\"}},\"policy\":{\"timeout\":\"datarrchfu\",\"retry\":\"datarqagpjociunndgp\",\"retryIntervalInSeconds\":172680098,\"secureInput\":true,\"secureOutput\":false,\"\":{\"ci\":\"datalqzymivjk\",\"kksywd\":\"datazag\",\"flpuxyakofrsoes\":\"datagzrg\"}},\"name\":\"qttkqcpclootcei\",\"description\":\"smnyfahidlscdow\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"qzjtdkojbx\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Completed\",\"Skipped\"],\"\":{\"sybw\":\"datagdsqcbxkwwn\",\"agrbirvsxubbnbof\":\"datavifgjztzhkhyqjv\",\"aauubkrzc\":\"dataeg\"}},{\"activity\":\"kwwibxjpy\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Succeeded\",\"Completed\"],\"\":{\"pxbqujnorfxi\":\"dataelujwcy\"}}],\"userProperties\":[{\"name\":\"cggwzvdqpxicpo\",\"value\":\"datazhfz\"}],\"\":{\"ps\":\"dataqpc\",\"fkzu\":\"datardkdomyqbeasbvz\",\"bmfqtnqaqlt\":\"datazqudqgfr\"}}") .toObject(DatabricksSparkJarActivity.class); - Assertions.assertEquals("pu", model.name()); - Assertions.assertEquals("u", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals("qttkqcpclootcei", model.name()); + Assertions.assertEquals("smnyfahidlscdow", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("eihmv", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("szoxmzvlof", model.userProperties().get(0).name()); - Assertions.assertEquals("ddj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1682173257, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("qzjtdkojbx", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("cggwzvdqpxicpo", model.userProperties().get(0).name()); + Assertions.assertEquals("rqenbgymgjneohx", model.linkedServiceName().referenceName()); + Assertions.assertEquals(172680098, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabricksSparkJarActivity model = new DatabricksSparkJarActivity().withName("pu").withDescription("u") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + DatabricksSparkJarActivity model = new DatabricksSparkJarActivity().withName("qttkqcpclootcei") + .withDescription("smnyfahidlscdow").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("eihmv") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) + new ActivityDependency() + .withActivity("qzjtdkojbx") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, + DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("zjxmnsrejqwy") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("tejipqynrlnqo") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("kwwibxjpy") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("szoxmzvlof").withValue("datadnvsrvl"), - new UserProperty().withName("wzuj").withValue("datafcxp"), - new UserProperty().withName("x").withValue("dataqe"), - new UserProperty().withName("kwcxtxgrhaqbstod").withValue("datauvsbsdcoqm"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ddj").withParameters( - mapOf("wlbccxjbaloihj", "datahvpqtxluqp", "vanxuiiprf", "dataknfvpavraeeiboq", "xs", "datajmilose"))) - .withPolicy(new ActivityPolicy().withTimeout("datajzxeswzgr").withRetry("datagggj") - .withRetryIntervalInSeconds(1682173257).withSecureInput(true).withSecureOutput(false) + .withUserProperties(Arrays.asList(new UserProperty().withName("cggwzvdqpxicpo").withValue("datazhfz"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("rqenbgymgjneohx") + .withParameters(mapOf("yjs", "datazbhgbfsjodsk", "rfih", "dataxgefk"))) + .withPolicy(new ActivityPolicy().withTimeout("datarrchfu").withRetry("datarqagpjociunndgp") + .withRetryIntervalInSeconds(172680098).withSecureInput(true).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withMainClassName("datanya").withParameters(Arrays.asList("datapgbhf", "dataypykjorlrj", "datarzxa")) + .withMainClassName("datajzoi").withParameters(Arrays.asList("dataizlscnknkukempa", "datavajbgpu")) .withLibraries(Arrays.asList( - mapOf("zahjqslshceyhal", "datahka", "fcoatsupaqzitho", "dataxrmgsn", "mtke", "datartcdavlri", - "xirsvjozexxzkci", "dataezwkparj"), - mapOf("dsjblagss", "datakeawrumhz", "hiwqiqx", "datadubqhafxlbomwn", "vmzs", "datawbormfnntpocf"), - mapOf("ttswpeaivbzr", "dataohytkhquhdyzuehq", "qameccuqk", "datasoeddwjimrzavcif"), - mapOf("rbcvhkkdsyx", "datatfiomdie"))); + mapOf("xujl", "datakankzyqi"), mapOf("oqrutbfkynwwmlzp", "datahtrgybfumo", "n", "datarzazkaljuvm", + "bfntgsju", "datalbzloae", "gix", "datagueggph"), + mapOf("qgkvkoynjucmyj", "datagvwmvafhriua"), mapOf("v", "dataafvvnd"))); model = BinaryData.fromObject(model).toObject(DatabricksSparkJarActivity.class); - Assertions.assertEquals("pu", model.name()); - Assertions.assertEquals("u", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals("qttkqcpclootcei", model.name()); + Assertions.assertEquals("smnyfahidlscdow", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("eihmv", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("szoxmzvlof", model.userProperties().get(0).name()); - Assertions.assertEquals("ddj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1682173257, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("qzjtdkojbx", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("cggwzvdqpxicpo", model.userProperties().get(0).name()); + Assertions.assertEquals("rqenbgymgjneohx", model.linkedServiceName().referenceName()); + Assertions.assertEquals(172680098, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTypePropertiesTests.java index 51cfd70cc64b4..1e6a13b6e8b50 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkJarActivityTypePropertiesTests.java @@ -14,19 +14,20 @@ public final class DatabricksSparkJarActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DatabricksSparkJarActivityTypeProperties model = BinaryData.fromString( - "{\"mainClassName\":\"dataqtjacnbe\",\"parameters\":[\"datahpkaamoovrbv\",\"datauoqbclhn\",\"dataqxuxrggx\"],\"libraries\":[{\"cjawqhpijurqoi\":\"dataqpkckwaafuy\",\"m\":\"dataxibji\"},{\"a\":\"datatkvfurdjjzsijm\"},{\"jmnvukovxfkxnevc\":\"datanbluxomzg\",\"oco\":\"datawuiopgyunf\",\"pqmdii\":\"datacgdki\"},{\"ppcpwcxfn\":\"datapmfakinode\"}]}") + "{\"mainClassName\":\"dataxhfphawjov\",\"parameters\":[\"databusyqyf\",\"datatdprbmmf\",\"datateoxgikdcjmb\",\"datar\"],\"libraries\":[{\"dsrwhjhivgeran\":\"dataudegykzdspbjks\",\"r\":\"dataxnafojtqqqc\"},{\"mvzqcgyvz\":\"dataduspxijrr\",\"ybjucfs\":\"datavzsduz\",\"poywymtwh\":\"datapkqpgfyjwx\",\"x\":\"datadgbg\"},{\"gia\":\"datarzhkhmw\",\"nlzalsuj\":\"datarftpgqxnyoakd\"},{\"ekbcedbmfejtdb\":\"datagz\",\"ykjorlrjgrzxaa\":\"dataacnyacjypgbhfzy\",\"eyhalb\":\"dataibhkaqzahjqslsh\"}]}") .toObject(DatabricksSparkJarActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { DatabricksSparkJarActivityTypeProperties model - = new DatabricksSparkJarActivityTypeProperties().withMainClassName("dataqtjacnbe") - .withParameters(Arrays.asList("datahpkaamoovrbv", "datauoqbclhn", "dataqxuxrggx")) - .withLibraries(Arrays.asList(mapOf("cjawqhpijurqoi", "dataqpkckwaafuy", "m", "dataxibji"), - mapOf("a", "datatkvfurdjjzsijm"), - mapOf("jmnvukovxfkxnevc", "datanbluxomzg", "oco", "datawuiopgyunf", "pqmdii", "datacgdki"), - mapOf("ppcpwcxfn", "datapmfakinode"))); + = new DatabricksSparkJarActivityTypeProperties().withMainClassName("dataxhfphawjov") + .withParameters(Arrays.asList("databusyqyf", "datatdprbmmf", "datateoxgikdcjmb", "datar")) + .withLibraries(Arrays.asList(mapOf("dsrwhjhivgeran", "dataudegykzdspbjks", "r", "dataxnafojtqqqc"), + mapOf("mvzqcgyvz", "dataduspxijrr", "ybjucfs", "datavzsduz", "poywymtwh", "datapkqpgfyjwx", "x", + "datadgbg"), + mapOf("gia", "datarzhkhmw", "nlzalsuj", "datarftpgqxnyoakd"), mapOf("ekbcedbmfejtdb", "datagz", + "ykjorlrjgrzxaa", "dataacnyacjypgbhfzy", "eyhalb", "dataibhkaqzahjqslsh"))); model = BinaryData.fromObject(model).toObject(DatabricksSparkJarActivityTypeProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTests.java index 686104005615e..d0abb90df610a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTests.java @@ -22,66 +22,59 @@ public final class DatabricksSparkPythonActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DatabricksSparkPythonActivity model = BinaryData.fromString( - "{\"type\":\"DatabricksSparkPython\",\"typeProperties\":{\"pythonFile\":\"datays\",\"parameters\":[\"dataaymxldorq\"],\"libraries\":[{\"ssog\":\"datavueyzg\",\"vir\":\"dataeqv\",\"eoma\":\"databguewtcq\",\"ojmxbyviykwrffxo\":\"datanwsgauwe\"}]},\"linkedServiceName\":{\"referenceName\":\"xhcxpzjewo\",\"parameters\":{\"quyatvsn\":\"datacvtdyuozmtsjgnp\",\"gwvblrgrzlrnu\":\"datarxhv\",\"d\":\"datah\"}},\"policy\":{\"timeout\":\"dataidfwzpauw\",\"retry\":\"datahynholojcaybu\",\"retryIntervalInSeconds\":1411197523,\"secureInput\":false,\"secureOutput\":true,\"\":{\"ekxvlejh\":\"dataqlghrcctvlnnkvdr\",\"u\":\"databqzxqid\"}},\"name\":\"wrwjbanteeu\",\"description\":\"icaikfvj\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"lrrqjioltdl\",\"dependencyConditions\":[\"Skipped\",\"Completed\"],\"\":{\"rvghvfodrqmcgeqy\":\"datan\"}},{\"activity\":\"ordnwtucvbviymv\",\"dependencyConditions\":[\"Completed\"],\"\":{\"mpcnezd\":\"datanvdorsgcvgkn\",\"bqhbbzfcjmhpobu\":\"datal\",\"mbyltd\":\"datapdyztqpszbtbx\",\"cotfov\":\"dataraxehxotizvwiha\"}},{\"activity\":\"iyluvebcovbzsmge\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Succeeded\"],\"\":{\"t\":\"datahxswebgaaxff\"}},{\"activity\":\"qlcxymcmogfbweu\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Skipped\",\"Succeeded\"],\"\":{\"cdgim\":\"dataoadsbacemwn\",\"xzmxww\":\"databumpplbcarcyrftc\",\"vxrcmrdmyjcou\":\"datahdlrfyonnb\",\"zirkyxhqwoxm\":\"datazodolehchimzrc\"}}],\"userProperties\":[{\"name\":\"u\",\"value\":\"datanybfmbl\"},{\"name\":\"hpqnzpf\",\"value\":\"datasp\"},{\"name\":\"kqufdmgmfyi\",\"value\":\"datarfkfgr\"},{\"name\":\"rilhyfxmrq\",\"value\":\"dataicknygzdrdicwm\"}],\"\":{\"gcc\":\"datavawywo\",\"fqwyiuhhuftn\":\"datazhjvvrrxclfomx\",\"ossokafy\":\"dataigxwxxfkfthw\"}}") + "{\"type\":\"DatabricksSparkPython\",\"typeProperties\":{\"pythonFile\":\"datarmgsnffcoa\",\"parameters\":[\"datapa\",\"datazith\",\"datajrtcdavlrifmtk\",\"datawezwkparj\"],\"libraries\":[{\"zexxzkcigykea\":\"datasvj\",\"shdubqhafxl\":\"datarumhzgdsjblag\",\"wbormfnntpocf\":\"dataomwnkhiwqiqx\",\"dohytkhq\":\"datavmzs\"},{\"ehqmt\":\"datadyz\"}]},\"linkedServiceName\":{\"referenceName\":\"wpeaivbzrms\",\"parameters\":{\"fqameccuqkoat\":\"datadwjimrzavci\",\"mdiecrbcvhkkdsyx\":\"datai\",\"zwlbccxj\":\"databddjbzohvpqtxluq\"}},\"policy\":{\"timeout\":\"dataoihjnknfvpa\",\"retry\":\"dataaeeiboqc\",\"retryIntervalInSeconds\":1524667796,\"secureInput\":true,\"secureOutput\":false,\"\":{\"osedxsphfj\":\"datafijmi\",\"relggg\":\"dataxeswz\",\"emtweh\":\"datatsjeolxbggoj\",\"rkrfabffeahypjqa\":\"datauttngatglarczzgu\"}},\"name\":\"ceujuclffpvdjfw\",\"description\":\"bplbtmwae\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"cxh\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Completed\"],\"\":{\"btymhh\":\"datad\",\"pu\":\"datauwcmwixyrv\",\"eihmv\":\"databuvviysg\"}},{\"activity\":\"upqfawwoxqjhm\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Succeeded\",\"Skipped\"],\"\":{\"gfaotokipndekpz\":\"datavsnyn\",\"rotnvxyeqdin\":\"datadrkddzkki\"}}],\"userProperties\":[{\"name\":\"ejtqoxethrxlpgrv\",\"value\":\"datazjxmnsrejqwy\"}],\"\":{\"kdaomxyx\":\"datasmhoviear\"}}") .toObject(DatabricksSparkPythonActivity.class); - Assertions.assertEquals("wrwjbanteeu", model.name()); - Assertions.assertEquals("icaikfvj", model.description()); + Assertions.assertEquals("ceujuclffpvdjfw", model.name()); + Assertions.assertEquals("bplbtmwae", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("lrrqjioltdl", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("u", model.userProperties().get(0).name()); - Assertions.assertEquals("xhcxpzjewo", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1411197523, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("cxh", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ejtqoxethrxlpgrv", model.userProperties().get(0).name()); + Assertions.assertEquals("wpeaivbzrms", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1524667796, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabricksSparkPythonActivity model - = new DatabricksSparkPythonActivity().withName("wrwjbanteeu").withDescription("icaikfvj") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("lrrqjioltdl") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ordnwtucvbviymv") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("iyluvebcovbzsmge") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("qlcxymcmogfbweu") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("u").withValue("datanybfmbl"), - new UserProperty().withName("hpqnzpf").withValue("datasp"), - new UserProperty().withName("kqufdmgmfyi").withValue("datarfkfgr"), - new UserProperty().withName("rilhyfxmrq").withValue("dataicknygzdrdicwm"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("xhcxpzjewo").withParameters( - mapOf("quyatvsn", "datacvtdyuozmtsjgnp", "gwvblrgrzlrnu", "datarxhv", "d", "datah"))) - .withPolicy(new ActivityPolicy().withTimeout("dataidfwzpauw").withRetry("datahynholojcaybu") - .withRetryIntervalInSeconds(1411197523).withSecureInput(false).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withPythonFile("datays").withParameters(Arrays.asList("dataaymxldorq")) - .withLibraries(Arrays.asList(mapOf("ssog", "datavueyzg", "vir", "dataeqv", "eoma", "databguewtcq", - "ojmxbyviykwrffxo", "datanwsgauwe"))); + DatabricksSparkPythonActivity model = new DatabricksSparkPythonActivity().withName("ceujuclffpvdjfw") + .withDescription("bplbtmwae").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("cxh") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("upqfawwoxqjhm") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("ejtqoxethrxlpgrv").withValue("datazjxmnsrejqwy"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("wpeaivbzrms") + .withParameters(mapOf("fqameccuqkoat", "datadwjimrzavci", "mdiecrbcvhkkdsyx", "datai", "zwlbccxj", + "databddjbzohvpqtxluq"))) + .withPolicy(new ActivityPolicy().withTimeout("dataoihjnknfvpa").withRetry("dataaeeiboqc") + .withRetryIntervalInSeconds(1524667796).withSecureInput(true).withSecureOutput(false) + .withAdditionalProperties(mapOf())) + .withPythonFile("datarmgsnffcoa") + .withParameters(Arrays.asList("datapa", "datazith", "datajrtcdavlrifmtk", "datawezwkparj")) + .withLibraries(Arrays.asList(mapOf("zexxzkcigykea", "datasvj", "shdubqhafxl", "datarumhzgdsjblag", + "wbormfnntpocf", "dataomwnkhiwqiqx", "dohytkhq", "datavmzs"), mapOf("ehqmt", "datadyz"))); model = BinaryData.fromObject(model).toObject(DatabricksSparkPythonActivity.class); - Assertions.assertEquals("wrwjbanteeu", model.name()); - Assertions.assertEquals("icaikfvj", model.description()); + Assertions.assertEquals("ceujuclffpvdjfw", model.name()); + Assertions.assertEquals("bplbtmwae", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("lrrqjioltdl", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("u", model.userProperties().get(0).name()); - Assertions.assertEquals("xhcxpzjewo", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1411197523, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("cxh", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ejtqoxethrxlpgrv", model.userProperties().get(0).name()); + Assertions.assertEquals("wpeaivbzrms", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1524667796, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTypePropertiesTests.java index 10bb1d8e83ae8..dc8bffd92afef 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatabricksSparkPythonActivityTypePropertiesTests.java @@ -14,19 +14,21 @@ public final class DatabricksSparkPythonActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DatabricksSparkPythonActivityTypeProperties model = BinaryData.fromString( - "{\"pythonFile\":\"datalstgsmeijgjbev\",\"parameters\":[\"datacsyjxdwvdklgw\",\"dataywal\",\"datafmenbaj\",\"dataeelbcsyaohizfysa\"],\"libraries\":[{\"ddohxvcsoqxydcqp\":\"datapft\"},{\"wt\":\"datawttdanuf\",\"fugikhm\":\"datahcmoccgt\"},{\"dcrdveccmqenfgba\":\"datastkkztexdsnmh\",\"denv\":\"datauuyt\"},{\"y\":\"datalfiigoxohjyvpfis\",\"vmilpgx\":\"dataoymccwvcfayllxvh\"}]}") + "{\"pythonFile\":\"databen\",\"parameters\":[\"dataxhxfsknmr\"],\"libraries\":[{\"wbwoflfnii\":\"datafbdxwywdyqp\",\"cyvtsgopmatu\":\"datalohftmfmxrxrya\",\"oelqfsfxthcdzeu\":\"datatejipqynrlnqo\",\"f\":\"datakqkvfthbnikoybrs\"},{\"iszoxmzvlofzdn\":\"dataqmtzhikuby\",\"vxrqegk\":\"datasrvlbwzujafcx\",\"deuvsbsdcoq\":\"datacxtxgrhaqbst\"},{\"vsijwrjrxccnf\":\"dataffeqlwk\",\"q\":\"dataknxuoccey\"},{\"ksgejmpkqtjacnbe\":\"datahkqmlld\",\"ov\":\"dataiqhpkaam\",\"uoqbclhn\":\"databv\",\"kzhqpkckwaafu\":\"dataqxuxrggx\"}]}") .toObject(DatabricksSparkPythonActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabricksSparkPythonActivityTypeProperties model - = new DatabricksSparkPythonActivityTypeProperties().withPythonFile("datalstgsmeijgjbev") - .withParameters(Arrays.asList("datacsyjxdwvdklgw", "dataywal", "datafmenbaj", "dataeelbcsyaohizfysa")) - .withLibraries(Arrays.asList(mapOf("ddohxvcsoqxydcqp", "datapft"), - mapOf("wt", "datawttdanuf", "fugikhm", "datahcmoccgt"), - mapOf("dcrdveccmqenfgba", "datastkkztexdsnmh", "denv", "datauuyt"), - mapOf("y", "datalfiigoxohjyvpfis", "vmilpgx", "dataoymccwvcfayllxvh"))); + DatabricksSparkPythonActivityTypeProperties model = new DatabricksSparkPythonActivityTypeProperties() + .withPythonFile("databen").withParameters(Arrays.asList("dataxhxfsknmr")) + .withLibraries(Arrays.asList( + mapOf("wbwoflfnii", "datafbdxwywdyqp", "cyvtsgopmatu", "datalohftmfmxrxrya", "oelqfsfxthcdzeu", + "datatejipqynrlnqo", "f", "datakqkvfthbnikoybrs"), + mapOf("iszoxmzvlofzdn", "dataqmtzhikuby", "vxrqegk", "datasrvlbwzujafcx", "deuvsbsdcoq", + "datacxtxgrhaqbst"), + mapOf("vsijwrjrxccnf", "dataffeqlwk", "q", "dataknxuoccey"), mapOf("ksgejmpkqtjacnbe", "datahkqmlld", + "ov", "dataiqhpkaam", "uoqbclhn", "databv", "kzhqpkckwaafu", "dataqxuxrggx"))); model = BinaryData.fromObject(model).toObject(DatabricksSparkPythonActivityTypeProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsCreateOrUpdateWithResponseMockTests.java index 27ecc89508a1d..eb20c20e39cac 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsCreateOrUpdateWithResponseMockTests.java @@ -39,7 +39,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"Dataset\",\"description\":\"huair\",\"structure\":\"dataquo\",\"schema\":\"dataamnxzduyd\",\"linkedServiceName\":{\"referenceName\":\"vvwoclmdcoqwdme\",\"parameters\":{\"wuwrtubemptxmue\":\"datajeuguvnwcvlmy\",\"dmfrjqfem\":\"datalseummwpq\"}},\"parameters\":{\"tevqbcdjlnnv\":{\"type\":\"Bool\",\"defaultValue\":\"dataprrvfyjvkmomp\"}},\"annotations\":[\"datajutupgmmtit\"],\"folder\":{\"name\":\"qqak\"},\"\":{\"xdrgxhrtansjbo\":\"datawqavxljaybgxx\",\"ixbfjhvkttusyxzy\":\"datay\",\"oqjttrivif\":\"datafw\",\"payoesx\":\"datajjmtkwgdgfjvit\"}},\"name\":\"vslhncasp\",\"type\":\"glaxvn\",\"etag\":\"hatwxqaggbir\",\"id\":\"haicyuplmdh\"}"; + = "{\"properties\":{\"type\":\"Dataset\",\"description\":\"kvqqtf\",\"structure\":\"dataxoxweuoklwt\",\"schema\":\"datacxndheocjcjocun\",\"linkedServiceName\":{\"referenceName\":\"nwutverplh\",\"parameters\":{\"bhltr\":\"datadvdqcohjwzy\",\"wpudvuphizztklk\":\"databhlhyqgfimllrad\",\"mdefkp\":\"datahdeeht\"}},\"parameters\":{\"opcnnpjulpw\":{\"type\":\"String\",\"defaultValue\":\"datavya\"},\"zjazepbjukikd\":{\"type\":\"Array\",\"defaultValue\":\"datawlwcurkf\"}},\"annotations\":[\"datauxmpnugujiw\",\"dataduns\"],\"folder\":{\"name\":\"obanxshltfghy\"},\"\":{\"vjbfgkqudx\":\"dataruqrobknenpybus\",\"kgawnaeoe\":\"datajrndbinqqr\",\"pvaxyi\":\"dataid\"}},\"name\":\"zpfdoetetis\",\"type\":\"xeclwl\",\"etag\":\"drdnfmxomup\",\"id\":\"pcxiv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -56,29 +56,28 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - DatasetResource response = manager.datasets().define("pzekm").withExistingFactory("mibo", "rgxdcnbzpcxoqum") - .withProperties(new Dataset().withDescription("dvnanx").withStructure("datawzla") - .withSchema("datateqnttmhsrw") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pcxyfjeibcg") - .withParameters(mapOf("ujkhummrxxc", "datapoe", "inmhccwmrckvlbc", "dataptvvwfamhlj", - "fxoktokmsyohxm", "datawdgydbsrj"))) - .withParameters(mapOf("adsdkb", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datatcmwqrb"), "ycnay", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataofrh"), "midtn", - new ParameterSpecification().withType(ParameterType.OBJECT) - .withDefaultValue("datatugrwpwxfkgzgveu"), - "whjthoxl", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datartcacdomz"))) - .withAnnotations(Arrays.asList("dataa", "dataicznotggy")) - .withFolder(new DatasetFolder().withName("sghafzdzdf")) + DatasetResource response = manager.datasets().define("jweelkvikig") + .withExistingFactory("zapuunuyokftd", "wezplnzvrh") + .withProperties(new Dataset().withDescription("rkwgsq").withStructure("datarpcxwthkljktujfc") + .withSchema("dataqfryketwrzx") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("miy") + .withParameters(mapOf("yqnjfjyppix", "datag", "lwyet", "dataubkfjt"))) + .withParameters(mapOf("civanlypspnj", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datasp"), + "aixrbwbkrsmkeiun", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataoangrl"), + "tzzmcrmhhfcai", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datamedzfox"))) + .withAnnotations(Arrays.asList("dataiyuzufd", "datasbvygnfljvraqko", "datacozfauhn", "dataxdy")) + .withFolder(new DatasetFolder().withName("grzcjpkzmhaxtwji")) .withAdditionalProperties(mapOf("type", "Dataset"))) - .withIfMatch("kgrxhpx").create(); + .withIfMatch("vvagvqrwrchwd").create(); - Assertions.assertEquals("haicyuplmdh", response.id()); - Assertions.assertEquals("huair", response.properties().description()); - Assertions.assertEquals("vvwoclmdcoqwdme", response.properties().linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.BOOL, response.properties().parameters().get("tevqbcdjlnnv").type()); - Assertions.assertEquals("qqak", response.properties().folder().name()); + Assertions.assertEquals("pcxiv", response.id()); + Assertions.assertEquals("kvqqtf", response.properties().description()); + Assertions.assertEquals("nwutverplh", response.properties().linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, response.properties().parameters().get("opcnnpjulpw").type()); + Assertions.assertEquals("obanxshltfghy", response.properties().folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsDeleteWithResponseMockTests.java index 3c38c0a0c798b..8ad0e80132ce5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsDeleteWithResponseMockTests.java @@ -45,7 +45,8 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.datasets().deleteWithResponse("nbc", "znlscfbwkh", "e", com.azure.core.util.Context.NONE); + manager.datasets().deleteWithResponse("dqfynrdagmihxjpf", "zpuibczlr", "wfrmqbmcmg", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsGetWithResponseMockTests.java index b165ebc79835f..2d461ea0dbbc8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsGetWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"Dataset\",\"description\":\"izdfulv\",\"structure\":\"dataalvcahy\",\"schema\":\"datahdhtcopzfce\",\"linkedServiceName\":{\"referenceName\":\"btwicghxme\",\"parameters\":{\"motgkyfh\":\"datavmmkwazku\",\"fsvzczisiqns\":\"dataxmwq\",\"fvzxosrstevdts\":\"datavwjfuhqbatdn\"}},\"parameters\":{\"pjslrfpxlutfbhs\":{\"type\":\"Object\",\"defaultValue\":\"datadojimfaa\"}},\"annotations\":[\"datafcbxtabxd\",\"databoyqescvcv\"],\"folder\":{\"name\":\"r\"},\"\":{\"ikqz\":\"datajppmil\",\"lhnkvipjinjik\":\"dataqkxjcqdnzhjlb\"}},\"name\":\"c\",\"type\":\"kcnjzxezo\",\"etag\":\"ltewthslztxix\",\"id\":\"weuxycbvefldfw\"}"; + = "{\"properties\":{\"type\":\"Dataset\",\"description\":\"cwtnzf\",\"structure\":\"dataghnfqwjwwhsf\",\"schema\":\"dataxlbclvp\",\"linkedServiceName\":{\"referenceName\":\"butyrsravsscbls\",\"parameters\":{\"yoilnixwxwaquuvb\":\"datacafgdtuzclfbvv\"}},\"parameters\":{\"zoibip\":{\"type\":\"String\",\"defaultValue\":\"datafeslxwlm\"},\"ufjahu\":{\"type\":\"Bool\",\"defaultValue\":\"datalomidvicd\"},\"klths\":{\"type\":\"Float\",\"defaultValue\":\"databdt\"}},\"annotations\":[\"datanxd\",\"datalovkt\",\"datafdipsshxxo\"],\"folder\":{\"name\":\"dryysv\"},\"\":{\"bfnkj\":\"datazs\",\"pbirltzyuahnlx\":\"datawty\",\"wqmzezf\":\"datadpjssdtysnl\",\"eukqioqhp\":\"datafjjjzcxtzkoloos\"}},\"name\":\"qkkacw\",\"type\":\"qmxkxfmwbrvsl\",\"etag\":\"rlaudemzrp\",\"id\":\"usujbibbgcloknh\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,13 +49,13 @@ public void testGetWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - DatasetResource response = manager.datasets() - .getWithResponse("ql", "zpckxlc", "l", "yrsojqpjbarfn", com.azure.core.util.Context.NONE).getValue(); + DatasetResource response = manager.datasets().getWithResponse("kldxc", "qjnnhotwqkgvr", "limzsutmsmdibzv", + "tempsaykcxu", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("weuxycbvefldfw", response.id()); - Assertions.assertEquals("izdfulv", response.properties().description()); - Assertions.assertEquals("btwicghxme", response.properties().linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, response.properties().parameters().get("pjslrfpxlutfbhs").type()); - Assertions.assertEquals("r", response.properties().folder().name()); + Assertions.assertEquals("usujbibbgcloknh", response.id()); + Assertions.assertEquals("cwtnzf", response.properties().description()); + Assertions.assertEquals("butyrsravsscbls", response.properties().linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, response.properties().parameters().get("zoibip").type()); + Assertions.assertEquals("dryysv", response.properties().folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsListByFactoryMockTests.java index 5eb08c8395ff5..d37a9986d6776 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DatasetsListByFactoryMockTests.java @@ -33,7 +33,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"type\":\"Dataset\",\"description\":\"xvsyl\",\"structure\":\"datameendocqa\",\"schema\":\"datawk\",\"linkedServiceName\":{\"referenceName\":\"ismonwhazalftta\",\"parameters\":{\"srduqhrlltfec\":\"datasve\",\"pjdbz\":\"dataxzhbfibzvxqh\",\"sydjr\":\"datalchv\",\"roohppupucybtra\":\"dataxjawfujvgvrpe\"}},\"parameters\":{\"rho\":{\"type\":\"Array\",\"defaultValue\":\"datam\"}},\"annotations\":[\"dataxeaexwee\",\"datafogv\",\"datamqdnfonncnfjygg\"],\"folder\":{\"name\":\"gv\"},\"\":{\"jw\":\"dataxlrtmjygnixkpadj\",\"xnkmaezxldmzhwuz\":\"datahqeibucmfvuizjr\"}},\"name\":\"dsvgonkomuapys\",\"type\":\"iytgfqvules\",\"etag\":\"dbcypvsvxfyqsf\",\"id\":\"afhbfpzf\"}]}"; + = "{\"value\":[{\"properties\":{\"type\":\"Dataset\",\"description\":\"cxlg\",\"structure\":\"dataxoymjx\",\"schema\":\"datantjhvcorobmqudz\",\"linkedServiceName\":{\"referenceName\":\"pzk\",\"parameters\":{\"q\":\"dataazkcyg\"}},\"parameters\":{\"ss\":{\"type\":\"Object\",\"defaultValue\":\"datapgujibkwwyfsqg\"},\"ohpwnrmhlotknb\":{\"type\":\"Object\",\"defaultValue\":\"dataazvrmulsje\"}},\"annotations\":[\"datazqmudmefsxmdmlow\"],\"folder\":{\"name\":\"xpwfvtwgnm\"},\"\":{\"cj\":\"datarxwkomjsfkdvb\"}},\"name\":\"njnwpivfplb\",\"type\":\"qec\",\"etag\":\"wkzuaxsrmadakj\",\"id\":\"puvyvobkk\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,14 +51,13 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.datasets().listByFactory("yqqoswkshtgfewfl", "byyvau", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("afhbfpzf", response.iterator().next().id()); - Assertions.assertEquals("xvsyl", response.iterator().next().properties().description()); - Assertions.assertEquals("ismonwhazalftta", - response.iterator().next().properties().linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, - response.iterator().next().properties().parameters().get("rho").type()); - Assertions.assertEquals("gv", response.iterator().next().properties().folder().name()); + = manager.datasets().listByFactory("srxjfapiodsnz", "ezwjq", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("puvyvobkk", response.iterator().next().id()); + Assertions.assertEquals("cxlg", response.iterator().next().properties().description()); + Assertions.assertEquals("pzk", response.iterator().next().properties().linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, + response.iterator().next().properties().parameters().get("ss").type()); + Assertions.assertEquals("xpwfvtwgnm", response.iterator().next().properties().folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Db2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Db2SourceTests.java index d33d7436f88ea..969cd580c134c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Db2SourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Db2SourceTests.java @@ -11,15 +11,15 @@ public final class Db2SourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { Db2Source model = BinaryData.fromString( - "{\"type\":\"Db2Source\",\"query\":\"databomf\",\"queryTimeout\":\"datajkerdujfnb\",\"additionalColumns\":\"datamroadutogbkdcts\",\"sourceRetryCount\":\"dataalh\",\"sourceRetryWait\":\"dataneclphmjsqcubyjr\",\"maxConcurrentConnections\":\"datalliteenah\",\"disableMetricsCollection\":\"datacsfttsub\",\"\":{\"z\":\"datahjnmdcyr\"}}") + "{\"type\":\"Db2Source\",\"query\":\"datafmzeu\",\"queryTimeout\":\"dataz\",\"additionalColumns\":\"dataqeqcbygqcwzyto\",\"sourceRetryCount\":\"dataqcthgqyvaoaz\",\"sourceRetryWait\":\"dataykkcqaf\",\"maxConcurrentConnections\":\"datajgixsjhi\",\"disableMetricsCollection\":\"datayekslllzsqolckwh\",\"\":{\"loifxzdohfvxavhf\":\"databnnhwp\",\"qwwzpbamcfr\":\"datal\",\"nrmbcklfpemgfv\":\"dataaytcygoom\"}}") .toObject(Db2Source.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Db2Source model = new Db2Source().withSourceRetryCount("dataalh").withSourceRetryWait("dataneclphmjsqcubyjr") - .withMaxConcurrentConnections("datalliteenah").withDisableMetricsCollection("datacsfttsub") - .withQueryTimeout("datajkerdujfnb").withAdditionalColumns("datamroadutogbkdcts").withQuery("databomf"); + Db2Source model = new Db2Source().withSourceRetryCount("dataqcthgqyvaoaz").withSourceRetryWait("dataykkcqaf") + .withMaxConcurrentConnections("datajgixsjhi").withDisableMetricsCollection("datayekslllzsqolckwh") + .withQueryTimeout("dataz").withAdditionalColumns("dataqeqcbygqcwzyto").withQuery("datafmzeu"); model = BinaryData.fromObject(model).toObject(Db2Source.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTests.java index 602feaf8e6514..5431b1ba17466 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTests.java @@ -25,67 +25,70 @@ public final class DeleteActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DeleteActivity model = BinaryData.fromString( - "{\"type\":\"Delete\",\"typeProperties\":{\"recursive\":\"dataefxkattpkkwldvk\",\"maxConcurrentConnections\":1531552976,\"enableLogging\":\"dataakgo\",\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"bjooqobpnkvnuw\",\"parameters\":{\"mlbhlimgz\":\"databbxkhveqbxvtgloi\",\"xalvdhmumsmnub\":\"datamtzzyjhyieyxrwfu\"}},\"path\":\"dataxrpstypxidqnv\",\"logLevel\":\"databfe\",\"enableReliableLogging\":\"datawr\",\"\":{\"pxthsfuxhlgoexu\":\"dataaaftdysevq\"}},\"dataset\":{\"referenceName\":\"nbfoorgtxd\",\"parameters\":{\"vrip\":\"datahb\",\"mw\":\"datagkwoommqvzzo\"}},\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datafh\",\"disableMetricsCollection\":\"datap\",\"\":{\"arbvblatvbjkq\":\"datajajzqgg\",\"ipe\":\"datatodgisfej\"}}},\"linkedServiceName\":{\"referenceName\":\"fuj\",\"parameters\":{\"tvaodif\":\"datajwktiyhiyk\",\"ops\":\"datapdafuft\"}},\"policy\":{\"timeout\":\"datahgnrxxhzwtrx\",\"retry\":\"datauxyg\",\"retryIntervalInSeconds\":1156991662,\"secureInput\":true,\"secureOutput\":true,\"\":{\"utdthloafhhiyk\":\"dataukaswgvoa\",\"ptefdvjgbemrjb\":\"datatjsebcuynqdl\",\"pdprjethyhbnoye\":\"datavqu\"}},\"name\":\"uivdrzxobtekl\",\"description\":\"q\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"oguzxwmw\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\",\"Failed\"],\"\":{\"yvramrqbeqzh\":\"databdk\",\"nnygajqmpfr\":\"datapxaps\",\"y\":\"dataouwefzd\"}}],\"userProperties\":[{\"name\":\"hv\",\"value\":\"datawhrjakdyqxjpzy\"},{\"name\":\"kwk\",\"value\":\"datakkpbybhqwil\"},{\"name\":\"sdg\",\"value\":\"datahe\"}],\"\":{\"urthmbgavw\":\"datarbojzfsznephb\",\"mi\":\"dataqjetoaijayvu\"}}") + "{\"type\":\"Delete\",\"typeProperties\":{\"recursive\":\"datacrsbrhkdemaxoaj\",\"maxConcurrentConnections\":756643616,\"enableLogging\":\"dataachsojgageyxa\",\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"kzkzp\",\"parameters\":{\"ml\":\"databmgfxwynzbe\"}},\"path\":\"datatgbgcmut\",\"logLevel\":\"datawd\",\"enableReliableLogging\":\"datapuufuh\",\"\":{\"qctqv\":\"datamfdrxyej\"}},\"dataset\":{\"referenceName\":\"ahiiat\",\"parameters\":{\"fchnhjsa\":\"datapoxogvpsm\",\"yikgmlplqgpskyn\":\"datapwx\",\"zk\":\"datak\",\"blgvezhi\":\"datanlvwtsl\"}},\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datayqwlxk\",\"disableMetricsCollection\":\"dataysyutnredzkovt\",\"\":{\"xmyblway\":\"datacaprxh\",\"wyfy\":\"datapaggkrumpu\",\"jdrvjktvpy\":\"datahcboipxhghicwv\"}}},\"linkedServiceName\":{\"referenceName\":\"ooytilsmise\",\"parameters\":{\"qu\":\"datajmm\",\"zh\":\"datacrzgluqacebcn\",\"wkgcpfz\":\"dataaumjuruspflvgl\"}},\"policy\":{\"timeout\":\"databrqgwvx\",\"retry\":\"dataqmvsr\",\"retryIntervalInSeconds\":1080614288,\"secureInput\":false,\"secureOutput\":false,\"\":{\"uoyownygbra\":\"dataob\"}},\"name\":\"cwhebyczweg\",\"description\":\"dp\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"ksqifrgm\",\"dependencyConditions\":[\"Completed\"],\"\":{\"l\":\"datardglecmeg\",\"ryhztwxuizakejo\":\"datadlt\",\"gqezgbqiiweoa\":\"datajnlxjhrzgnfqq\"}},{\"activity\":\"eokrarzkzatznvye\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Skipped\",\"Completed\"],\"\":{\"ukmxgslzbpnlfz\":\"dataz\",\"uotwzes\":\"datawm\"}}],\"userProperties\":[{\"name\":\"cpcpeur\",\"value\":\"dataofzmvt\"}],\"\":{\"h\":\"datacbjrptltytbqhe\",\"slxzwvygquiwcfq\":\"datawjlbygqfmeeuuurx\",\"wwdev\":\"dataob\"}}") .toObject(DeleteActivity.class); - Assertions.assertEquals("uivdrzxobtekl", model.name()); - Assertions.assertEquals("q", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("oguzxwmw", model.dependsOn().get(0).activity()); + Assertions.assertEquals("cwhebyczweg", model.name()); + Assertions.assertEquals("dp", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("ksqifrgm", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("hv", model.userProperties().get(0).name()); - Assertions.assertEquals("fuj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1156991662, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals(1531552976, model.maxConcurrentConnections()); - Assertions.assertEquals("bjooqobpnkvnuw", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("nbfoorgtxd", model.dataset().referenceName()); + Assertions.assertEquals("cpcpeur", model.userProperties().get(0).name()); + Assertions.assertEquals("ooytilsmise", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1080614288, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals(756643616, model.maxConcurrentConnections()); + Assertions.assertEquals("kzkzp", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ahiiat", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeleteActivity model = new DeleteActivity().withName("uivdrzxobtekl").withDescription("q") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("oguzxwmw") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("hv").withValue("datawhrjakdyqxjpzy"), - new UserProperty().withName("kwk").withValue("datakkpbybhqwil"), - new UserProperty().withName("sdg").withValue("datahe"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fuj") - .withParameters(mapOf("tvaodif", "datajwktiyhiyk", "ops", "datapdafuft"))) - .withPolicy(new ActivityPolicy().withTimeout("datahgnrxxhzwtrx").withRetry("datauxyg") - .withRetryIntervalInSeconds(1156991662).withSecureInput(true).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withRecursive("dataefxkattpkkwldvk").withMaxConcurrentConnections( - 1531552976) - .withEnableLogging("dataakgo") - .withLogStorageSettings(new LogStorageSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("bjooqobpnkvnuw").withParameters( - mapOf("mlbhlimgz", "databbxkhveqbxvtgloi", "xalvdhmumsmnub", "datamtzzyjhyieyxrwfu"))) - .withPath("dataxrpstypxidqnv").withLogLevel("databfe").withEnableReliableLogging("datawr") - .withAdditionalProperties(mapOf())) - .withDataset(new DatasetReference().withReferenceName("nbfoorgtxd") - .withParameters(mapOf("vrip", "datahb", "mw", "datagkwoommqvzzo"))) - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datafh") - .withDisableMetricsCollection("datap").withAdditionalProperties(mapOf("type", "StoreReadSettings"))); + DeleteActivity model + = new DeleteActivity().withName("cwhebyczweg").withDescription("dp").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("ksqifrgm") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("eokrarzkzatznvye") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("cpcpeur").withValue("dataofzmvt"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ooytilsmise") + .withParameters(mapOf("qu", "datajmm", "zh", "datacrzgluqacebcn", "wkgcpfz", "dataaumjuruspflvgl"))) + .withPolicy(new ActivityPolicy().withTimeout("databrqgwvx").withRetry("dataqmvsr") + .withRetryIntervalInSeconds(1080614288).withSecureInput(false).withSecureOutput(false) + .withAdditionalProperties(mapOf())) + .withRecursive("datacrsbrhkdemaxoaj").withMaxConcurrentConnections(756643616) + .withEnableLogging("dataachsojgageyxa") + .withLogStorageSettings(new LogStorageSettings() + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("kzkzp") + .withParameters(mapOf("ml", "databmgfxwynzbe"))) + .withPath("datatgbgcmut").withLogLevel("datawd").withEnableReliableLogging("datapuufuh") + .withAdditionalProperties(mapOf())) + .withDataset(new DatasetReference().withReferenceName("ahiiat") + .withParameters(mapOf("fchnhjsa", "datapoxogvpsm", "yikgmlplqgpskyn", "datapwx", "zk", "datak", + "blgvezhi", "datanlvwtsl"))) + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datayqwlxk") + .withDisableMetricsCollection("dataysyutnredzkovt") + .withAdditionalProperties(mapOf("type", "StoreReadSettings"))); model = BinaryData.fromObject(model).toObject(DeleteActivity.class); - Assertions.assertEquals("uivdrzxobtekl", model.name()); - Assertions.assertEquals("q", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("oguzxwmw", model.dependsOn().get(0).activity()); + Assertions.assertEquals("cwhebyczweg", model.name()); + Assertions.assertEquals("dp", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("ksqifrgm", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("hv", model.userProperties().get(0).name()); - Assertions.assertEquals("fuj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1156991662, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals(1531552976, model.maxConcurrentConnections()); - Assertions.assertEquals("bjooqobpnkvnuw", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("nbfoorgtxd", model.dataset().referenceName()); + Assertions.assertEquals("cpcpeur", model.userProperties().get(0).name()); + Assertions.assertEquals("ooytilsmise", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1080614288, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals(756643616, model.maxConcurrentConnections()); + Assertions.assertEquals("kzkzp", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("ahiiat", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTypePropertiesTests.java index 479cd34561a27..d14a1d27423d3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DeleteActivityTypePropertiesTests.java @@ -18,31 +18,30 @@ public final class DeleteActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DeleteActivityTypeProperties model = BinaryData.fromString( - "{\"recursive\":\"datanq\",\"maxConcurrentConnections\":1637432269,\"enableLogging\":\"datau\",\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"qwxivj\",\"parameters\":{\"ink\":\"datadvnoxjbhltxtpgq\"}},\"path\":\"dataayaf\",\"logLevel\":\"datacwawblkkc\",\"enableReliableLogging\":\"dataxsgklxgsqhczoku\",\"\":{\"qzcbqvjejnwwqyy\":\"dataqhbjmvbeznluk\",\"ybbhktnuzorxati\":\"datactfsdhmrughm\",\"lcfxsgjdiqemcgh\":\"datajqpdfsautviti\",\"bvrmvhtmzwgir\":\"datarrjawfcz\"}},\"dataset\":{\"referenceName\":\"fnzpybr\",\"parameters\":{\"hupvxthpsugebgb\":\"datavtvlqwpmm\",\"iiwuufofgfqge\":\"dataqnc\",\"jonasjdaxe\":\"dataxypxmke\",\"ih\":\"datafdsogl\"}},\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datawn\",\"disableMetricsCollection\":\"datai\",\"\":{\"tpmfachkzznnkmms\":\"datagqwd\"}}}") + "{\"recursive\":\"datatejhvggykirqks\",\"maxConcurrentConnections\":1105166320,\"enableLogging\":\"datammmimsyiwc\",\"logStorageSettings\":{\"linkedServiceName\":{\"referenceName\":\"qjb\",\"parameters\":{\"fkwdtsbjmchsefe\":\"datamlmibvczdjkonlv\",\"n\":\"datajyfaqdwfa\"}},\"path\":\"dataetslxerhwlvh\",\"logLevel\":\"dataxxgeladqziljrsyc\",\"enableReliableLogging\":\"datansznjs\",\"\":{\"uyyqsonfxsf\":\"datajupukhxpi\",\"ytedspkduhz\":\"dataedjnxicufxt\",\"ttesqpjcuuy\":\"datavbgcf\",\"bahtlopbns\":\"datatuindpmrijncaqgt\"}},\"dataset\":{\"referenceName\":\"jzrnjcagagmgulln\",\"parameters\":{\"bqzwutakbv\":\"datanoxrmabbetzc\",\"cdue\":\"dataqguaubmcwpllojd\",\"eajrcrbkwc\":\"datatddigmmj\"}},\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataxguihmjcemk\",\"disableMetricsCollection\":\"datacbvh\",\"\":{\"mzntroafz\":\"dataiafzwhr\",\"ogfo\":\"dataq\",\"ojjbky\":\"datarryzbqpksoaxszuh\"}}}") .toObject(DeleteActivityTypeProperties.class); - Assertions.assertEquals(1637432269, model.maxConcurrentConnections()); - Assertions.assertEquals("qwxivj", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("fnzpybr", model.dataset().referenceName()); + Assertions.assertEquals(1105166320, model.maxConcurrentConnections()); + Assertions.assertEquals("qjb", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("jzrnjcagagmgulln", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DeleteActivityTypeProperties model = new DeleteActivityTypeProperties().withRecursive("datanq") - .withMaxConcurrentConnections(1637432269).withEnableLogging("datau") + DeleteActivityTypeProperties model = new DeleteActivityTypeProperties().withRecursive("datatejhvggykirqks") + .withMaxConcurrentConnections(1105166320).withEnableLogging("datammmimsyiwc") .withLogStorageSettings(new LogStorageSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("qwxivj") - .withParameters(mapOf("ink", "datadvnoxjbhltxtpgq"))) - .withPath("dataayaf").withLogLevel("datacwawblkkc").withEnableReliableLogging("dataxsgklxgsqhczoku") - .withAdditionalProperties(mapOf())) - .withDataset(new DatasetReference().withReferenceName("fnzpybr") - .withParameters(mapOf("hupvxthpsugebgb", "datavtvlqwpmm", "iiwuufofgfqge", "dataqnc", "jonasjdaxe", - "dataxypxmke", "ih", "datafdsogl"))) - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datawn") - .withDisableMetricsCollection("datai").withAdditionalProperties(mapOf("type", "StoreReadSettings"))); + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("qjb") + .withParameters(mapOf("fkwdtsbjmchsefe", "datamlmibvczdjkonlv", "n", "datajyfaqdwfa"))) + .withPath("dataetslxerhwlvh").withLogLevel("dataxxgeladqziljrsyc") + .withEnableReliableLogging("datansznjs").withAdditionalProperties(mapOf())) + .withDataset(new DatasetReference().withReferenceName("jzrnjcagagmgulln").withParameters( + mapOf("bqzwutakbv", "datanoxrmabbetzc", "cdue", "dataqguaubmcwpllojd", "eajrcrbkwc", "datatddigmmj"))) + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataxguihmjcemk") + .withDisableMetricsCollection("datacbvh").withAdditionalProperties(mapOf("type", "StoreReadSettings"))); model = BinaryData.fromObject(model).toObject(DeleteActivityTypeProperties.class); - Assertions.assertEquals(1637432269, model.maxConcurrentConnections()); - Assertions.assertEquals("qwxivj", model.logStorageSettings().linkedServiceName().referenceName()); - Assertions.assertEquals("fnzpybr", model.dataset().referenceName()); + Assertions.assertEquals(1105166320, model.maxConcurrentConnections()); + Assertions.assertEquals("qjb", model.logStorageSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("jzrnjcagagmgulln", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextReadSettingsTests.java index 9f9af580dbea1..5fa156e4c2f02 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextReadSettingsTests.java @@ -14,14 +14,14 @@ public final class DelimitedTextReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DelimitedTextReadSettings model = BinaryData.fromString( - "{\"type\":\"DelimitedTextReadSettings\",\"skipLineCount\":\"dataojpnix\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"vmrnwgeajyifei\":\"datak\",\"tlxrdepqtz\":\"datariomjdnkn\",\"o\":\"datahkpko\"}},\"\":{\"uhgmmz\":\"dataobuwhutvcdtgxsy\",\"rlbppjqcwcpd\":\"datap\",\"soasxjjk\":\"dataoskgtall\",\"dx\":\"datampbgrosx\"}}") + "{\"type\":\"DelimitedTextReadSettings\",\"skipLineCount\":\"dataaexrzxvffqc\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"qdibygrqcid\":\"datahrztbyulkoepss\",\"pftrdicstrbq\":\"datawkxikxiqxlxoksy\",\"katccetyyv\":\"dataatkliopgw\"}},\"\":{\"nsdp\":\"dataobb\",\"lzk\":\"datairt\"}}") .toObject(DelimitedTextReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { DelimitedTextReadSettings model - = new DelimitedTextReadSettings().withSkipLineCount("dataojpnix").withCompressionProperties( + = new DelimitedTextReadSettings().withSkipLineCount("dataaexrzxvffqc").withCompressionProperties( new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings"))); model = BinaryData.fromObject(model).toObject(DelimitedTextReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSinkTests.java index e6a5caaa31a76..bae008696c7b4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSinkTests.java @@ -17,23 +17,25 @@ public final class DelimitedTextSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DelimitedTextSink model = BinaryData.fromString( - "{\"type\":\"DelimitedTextSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datar\",\"disableMetricsCollection\":\"datavbjvvcogupsho\",\"copyBehavior\":\"dataaskyyvdjoorbuuhb\",\"metadata\":[{\"name\":\"datacvtelmdrmasvg\",\"value\":\"datahlbkquhmb\"}],\"\":{\"xsrglxljuyvk\":\"datasmviaasd\",\"xq\":\"datapovzespdip\"}},\"formatSettings\":{\"type\":\"DelimitedTextWriteSettings\",\"quoteAllText\":\"datajomddadwosjxyw\",\"fileExtension\":\"datavilkyhtnriyhdb\",\"maxRowsPerFile\":\"datagsjmcybrpwjenb\",\"fileNamePrefix\":\"datakghrrxauuhd\",\"\":{\"wip\":\"dataizyxoyxnhub\",\"ezzyrp\":\"dataryiv\"}},\"writeBatchSize\":\"dataygfpqxsemezk\",\"writeBatchTimeout\":\"datammowftfrqebr\",\"sinkRetryCount\":\"datapotefekfx\",\"sinkRetryWait\":\"dataj\",\"maxConcurrentConnections\":\"datawidnrd\",\"disableMetricsCollection\":\"datakvhuiadyh\",\"\":{\"eadokuqn\":\"datasypgapfdwhw\",\"kflnjaysrjeevm\":\"datao\"}}") + "{\"type\":\"DelimitedTextSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datakqah\",\"disableMetricsCollection\":\"datagnapkpaiedo\",\"copyBehavior\":\"dataoaoavezwc\",\"metadata\":[{\"name\":\"datalrvlg\",\"value\":\"datarcdiqhvhcbukaw\"},{\"name\":\"datazfjtockgqaawyys\",\"value\":\"dataoeql\"},{\"name\":\"dataw\",\"value\":\"datarfyww\"},{\"name\":\"dataipkhqhvktcztm\",\"value\":\"datak\"}],\"\":{\"bxxxqfrn\":\"dataspkgxemvlyaprj\"}},\"formatSettings\":{\"type\":\"DelimitedTextWriteSettings\",\"quoteAllText\":\"datahmx\",\"fileExtension\":\"datapxfauvgtoino\",\"maxRowsPerFile\":\"datamyvvfapfbmrwhkne\",\"fileNamePrefix\":\"dataoo\",\"\":{\"gupiosibg\":\"datadspdd\",\"skyrttnrikss\":\"datalaxuybxjwny\"}},\"writeBatchSize\":\"dataiksjpkign\",\"writeBatchTimeout\":\"datao\",\"sinkRetryCount\":\"datauivipbfsxpslp\",\"sinkRetryWait\":\"datazpqydnokkkg\",\"maxConcurrentConnections\":\"datacjyft\",\"disableMetricsCollection\":\"dataeftzetjclaqx\",\"\":{\"agpdsuyywnaqgo\":\"dataq\",\"zwlhufsgc\":\"dataxuhhvgddfzcny\",\"gtdeg\":\"datawr\"}}") .toObject(DelimitedTextSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DelimitedTextSink model = new DelimitedTextSink().withWriteBatchSize("dataygfpqxsemezk") - .withWriteBatchTimeout("datammowftfrqebr").withSinkRetryCount("datapotefekfx").withSinkRetryWait("dataj") - .withMaxConcurrentConnections("datawidnrd").withDisableMetricsCollection("datakvhuiadyh") - .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("datar") - .withDisableMetricsCollection("datavbjvvcogupsho").withCopyBehavior("dataaskyyvdjoorbuuhb") - .withMetadata( - Arrays.asList(new MetadataItem().withName("datacvtelmdrmasvg").withValue("datahlbkquhmb"))) + DelimitedTextSink model = new DelimitedTextSink().withWriteBatchSize("dataiksjpkign") + .withWriteBatchTimeout("datao").withSinkRetryCount("datauivipbfsxpslp").withSinkRetryWait("datazpqydnokkkg") + .withMaxConcurrentConnections("datacjyft").withDisableMetricsCollection("dataeftzetjclaqx") + .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("datakqah") + .withDisableMetricsCollection("datagnapkpaiedo").withCopyBehavior("dataoaoavezwc") + .withMetadata(Arrays.asList(new MetadataItem().withName("datalrvlg").withValue("datarcdiqhvhcbukaw"), + new MetadataItem().withName("datazfjtockgqaawyys").withValue("dataoeql"), + new MetadataItem().withName("dataw").withValue("datarfyww"), + new MetadataItem().withName("dataipkhqhvktcztm").withValue("datak"))) .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))) - .withFormatSettings(new DelimitedTextWriteSettings().withQuoteAllText("datajomddadwosjxyw") - .withFileExtension("datavilkyhtnriyhdb").withMaxRowsPerFile("datagsjmcybrpwjenb") - .withFileNamePrefix("datakghrrxauuhd")); + .withFormatSettings( + new DelimitedTextWriteSettings().withQuoteAllText("datahmx").withFileExtension("datapxfauvgtoino") + .withMaxRowsPerFile("datamyvvfapfbmrwhkne").withFileNamePrefix("dataoo")); model = BinaryData.fromObject(model).toObject(DelimitedTextSink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSourceTests.java index 92ffeb6db5301..08fca22a55010 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextSourceTests.java @@ -16,22 +16,21 @@ public final class DelimitedTextSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DelimitedTextSource model = BinaryData.fromString( - "{\"type\":\"DelimitedTextSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datavtaulxxznfx\",\"disableMetricsCollection\":\"dataqzikvg\",\"\":{\"rzxvffq\":\"datawodhaslpaae\",\"mhrztbyulk\":\"dataht\"}},\"formatSettings\":{\"type\":\"DelimitedTextReadSettings\",\"skipLineCount\":\"datassoqdibygrqcidiw\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"xlxoksyypft\":\"dataxi\"}},\"\":{\"rbqgatkliopgwpka\":\"datacs\",\"nsdp\":\"dataccetyyvxkwobb\",\"lzk\":\"datairt\"}},\"additionalColumns\":\"dataeegzhhfnaqc\",\"sourceRetryCount\":\"datapcklowuthfwphn\",\"sourceRetryWait\":\"datalbljehw\",\"maxConcurrentConnections\":\"dataxuofneaqahzkv\",\"disableMetricsCollection\":\"datapxhtqwsddxaovubf\",\"\":{\"iibuab\":\"datakeub\",\"bu\":\"datavdwhvnj\",\"eoukvinkgbkzqb\":\"dataudkq\"}}") + "{\"type\":\"DelimitedTextSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datamg\",\"disableMetricsCollection\":\"databqxlsam\",\"\":{\"gvmowyzxqhuhmldh\":\"dataqhwsojnbb\"}},\"formatSettings\":{\"type\":\"DelimitedTextReadSettings\",\"skipLineCount\":\"datackfu\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"dztqhqhwpuaermaw\":\"datao\",\"otwfu\":\"datalnsdazqcem\",\"dfszxbups\":\"datathisxzhik\"}},\"\":{\"tajwkrxmz\":\"datajpgxcgqkhy\"}},\"additionalColumns\":\"datawfn\",\"sourceRetryCount\":\"datalwovkzbomjby\",\"sourceRetryWait\":\"dataprkbzraljwfnc\",\"maxConcurrentConnections\":\"dataaylcpgzmx\",\"disableMetricsCollection\":\"datappqajdm\",\"\":{\"iminetbzfwfuxdt\":\"datantqqguhvnwr\",\"xxquyfft\":\"datajcsqkedlc\"}}") .toObject(DelimitedTextSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DelimitedTextSource model - = new DelimitedTextSource().withSourceRetryCount("datapcklowuthfwphn").withSourceRetryWait("datalbljehw") - .withMaxConcurrentConnections("dataxuofneaqahzkv").withDisableMetricsCollection("datapxhtqwsddxaovubf") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datavtaulxxznfx") - .withDisableMetricsCollection("dataqzikvg") - .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) - .withFormatSettings(new DelimitedTextReadSettings().withSkipLineCount("datassoqdibygrqcidiw") - .withCompressionProperties(new CompressionReadSettings() - .withAdditionalProperties(mapOf("type", "CompressionReadSettings")))) - .withAdditionalColumns("dataeegzhhfnaqc"); + DelimitedTextSource model = new DelimitedTextSource().withSourceRetryCount("datalwovkzbomjby") + .withSourceRetryWait("dataprkbzraljwfnc").withMaxConcurrentConnections("dataaylcpgzmx") + .withDisableMetricsCollection("datappqajdm") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datamg") + .withDisableMetricsCollection("databqxlsam") + .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + .withFormatSettings(new DelimitedTextReadSettings().withSkipLineCount("datackfu").withCompressionProperties( + new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings")))) + .withAdditionalColumns("datawfn"); model = BinaryData.fromObject(model).toObject(DelimitedTextSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextWriteSettingsTests.java index 80d373598c0aa..b9312800c7973 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DelimitedTextWriteSettingsTests.java @@ -11,15 +11,15 @@ public final class DelimitedTextWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DelimitedTextWriteSettings model = BinaryData.fromString( - "{\"type\":\"DelimitedTextWriteSettings\",\"quoteAllText\":\"datareebjmslb\",\"fileExtension\":\"datafkiiarl\",\"maxRowsPerFile\":\"dataygfjdtykhsafr\",\"fileNamePrefix\":\"datawuebrvrh\",\"\":{\"kleipybwhesb\":\"datafffvg\",\"qgir\":\"databvkmtljzilkyvybl\"}}") + "{\"type\":\"DelimitedTextWriteSettings\",\"quoteAllText\":\"datatetfdpu\",\"fileExtension\":\"datavzdtjbesfumed\",\"maxRowsPerFile\":\"dataafkvqhb\",\"fileNamePrefix\":\"datavivqkyaghfvub\",\"\":{\"bjrsdiufqxrl\":\"datasveguxaxijpny\",\"etidne\":\"dataijosuzegmcmlzm\",\"avpqyha\":\"datarvjgwnmxcngow\"}}") .toObject(DelimitedTextWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { DelimitedTextWriteSettings model - = new DelimitedTextWriteSettings().withQuoteAllText("datareebjmslb").withFileExtension("datafkiiarl") - .withMaxRowsPerFile("dataygfjdtykhsafr").withFileNamePrefix("datawuebrvrh"); + = new DelimitedTextWriteSettings().withQuoteAllText("datatetfdpu").withFileExtension("datavzdtjbesfumed") + .withMaxRowsPerFile("dataafkvqhb").withFileNamePrefix("datavivqkyaghfvub"); model = BinaryData.fromObject(model).toObject(DelimitedTextWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DistcpSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DistcpSettingsTests.java index c5c1c42412d71..77cb68ac920a9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DistcpSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DistcpSettingsTests.java @@ -11,14 +11,14 @@ public final class DistcpSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DistcpSettings model = BinaryData.fromString( - "{\"resourceManagerEndpoint\":\"datafk\",\"tempScriptPath\":\"datadwzvhtgfdy\",\"distcpOptions\":\"dataphlwmivazfnbzcjm\"}") + "{\"resourceManagerEndpoint\":\"datawcp\",\"tempScriptPath\":\"dataaosk\",\"distcpOptions\":\"dataalljsoasxjjklm\"}") .toObject(DistcpSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DistcpSettings model = new DistcpSettings().withResourceManagerEndpoint("datafk") - .withTempScriptPath("datadwzvhtgfdy").withDistcpOptions("dataphlwmivazfnbzcjm"); + DistcpSettings model = new DistcpSettings().withResourceManagerEndpoint("datawcp") + .withTempScriptPath("dataaosk").withDistcpOptions("dataalljsoasxjjklm"); model = BinaryData.fromObject(model).toObject(DistcpSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSinkTests.java index 7f15ed1c9751c..64aaa715290b8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSinkTests.java @@ -11,17 +11,17 @@ public final class DocumentDbCollectionSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DocumentDbCollectionSink model = BinaryData.fromString( - "{\"type\":\"DocumentDbCollectionSink\",\"nestingSeparator\":\"dataffjie\",\"writeBehavior\":\"datamhsqyjsw\",\"writeBatchSize\":\"datahafcoayuq\",\"writeBatchTimeout\":\"datatghjmmjmmjnxh\",\"sinkRetryCount\":\"datajtsemnidbaykvlrs\",\"sinkRetryWait\":\"dataniocyo\",\"maxConcurrentConnections\":\"dataimbchi\",\"disableMetricsCollection\":\"datawaffsjqn\",\"\":{\"vaagazryyjjwggp\":\"dataybugojzcargsxmaw\",\"csjw\":\"datadugwddob\",\"hzugwkqnmh\":\"dataxedzmmcgqifh\"}}") + "{\"type\":\"DocumentDbCollectionSink\",\"nestingSeparator\":\"datapxftyhfc\",\"writeBehavior\":\"dataqsd\",\"writeBatchSize\":\"datafqaawr\",\"writeBatchTimeout\":\"datatzslfrztpnry\",\"sinkRetryCount\":\"dataxajr\",\"sinkRetryWait\":\"dataighlhddvno\",\"maxConcurrentConnections\":\"datasowbtnfqlwc\",\"disableMetricsCollection\":\"datajphtehoucmeuuua\",\"\":{\"kqtiuve\":\"datatlcxofqjninrskq\"}}") .toObject(DocumentDbCollectionSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DocumentDbCollectionSink model = new DocumentDbCollectionSink().withWriteBatchSize("datahafcoayuq") - .withWriteBatchTimeout("datatghjmmjmmjnxh").withSinkRetryCount("datajtsemnidbaykvlrs") - .withSinkRetryWait("dataniocyo").withMaxConcurrentConnections("dataimbchi") - .withDisableMetricsCollection("datawaffsjqn").withNestingSeparator("dataffjie") - .withWriteBehavior("datamhsqyjsw"); + DocumentDbCollectionSink model + = new DocumentDbCollectionSink().withWriteBatchSize("datafqaawr").withWriteBatchTimeout("datatzslfrztpnry") + .withSinkRetryCount("dataxajr").withSinkRetryWait("dataighlhddvno") + .withMaxConcurrentConnections("datasowbtnfqlwc").withDisableMetricsCollection("datajphtehoucmeuuua") + .withNestingSeparator("datapxftyhfc").withWriteBehavior("dataqsd"); model = BinaryData.fromObject(model).toObject(DocumentDbCollectionSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSourceTests.java index 918f5730d3069..91e8c847e80b4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DocumentDbCollectionSourceTests.java @@ -11,17 +11,16 @@ public final class DocumentDbCollectionSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DocumentDbCollectionSource model = BinaryData.fromString( - "{\"type\":\"DocumentDbCollectionSource\",\"query\":\"datamhlcxbaxcgcdf\",\"nestingSeparator\":\"datavapbdbicjzntibl\",\"queryTimeout\":\"dataygosuhroicjt\",\"additionalColumns\":\"datadymoanpkcmdixiux\",\"sourceRetryCount\":\"datacalgspz\",\"sourceRetryWait\":\"datafprzlvhohzkcsjd\",\"maxConcurrentConnections\":\"dataposmnmkypeqm\",\"disableMetricsCollection\":\"dataeox\",\"\":{\"hw\":\"dataulavxeaymfl\",\"a\":\"datafunptsryp\",\"tbxcj\":\"datakbwbxvs\"}}") + "{\"type\":\"DocumentDbCollectionSource\",\"query\":\"databutmxtijs\",\"nestingSeparator\":\"datatdp\",\"queryTimeout\":\"datajtwibwcd\",\"additionalColumns\":\"datamnswxq\",\"sourceRetryCount\":\"datahffcanvrdtdl\",\"sourceRetryWait\":\"datamgghutl\",\"maxConcurrentConnections\":\"datazzljyog\",\"disableMetricsCollection\":\"datawnegpbiuwnxhqelj\",\"\":{\"dhg\":\"datalqdikuvjcls\",\"kiw\":\"datakflwnlpbawtpw\"}}") .toObject(DocumentDbCollectionSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DocumentDbCollectionSource model = new DocumentDbCollectionSource().withSourceRetryCount("datacalgspz") - .withSourceRetryWait("datafprzlvhohzkcsjd").withMaxConcurrentConnections("dataposmnmkypeqm") - .withDisableMetricsCollection("dataeox").withQuery("datamhlcxbaxcgcdf") - .withNestingSeparator("datavapbdbicjzntibl").withQueryTimeout("dataygosuhroicjt") - .withAdditionalColumns("datadymoanpkcmdixiux"); + DocumentDbCollectionSource model = new DocumentDbCollectionSource().withSourceRetryCount("datahffcanvrdtdl") + .withSourceRetryWait("datamgghutl").withMaxConcurrentConnections("datazzljyog") + .withDisableMetricsCollection("datawnegpbiuwnxhqelj").withQuery("databutmxtijs") + .withNestingSeparator("datatdp").withQueryTimeout("datajtwibwcd").withAdditionalColumns("datamnswxq"); model = BinaryData.fromObject(model).toObject(DocumentDbCollectionSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillDatasetTypePropertiesTests.java index 4f2787be35b64..181c8d16ce7db 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class DrillDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DrillDatasetTypeProperties model = BinaryData - .fromString("{\"tableName\":\"dataoqejexfd\",\"table\":\"datauhdk\",\"schema\":\"datagywadrklpdyehjr\"}") + .fromString("{\"tableName\":\"datavpnyldjd\",\"table\":\"datavdryknkx\",\"schema\":\"dataxhnrjl\"}") .toObject(DrillDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DrillDatasetTypeProperties model = new DrillDatasetTypeProperties().withTableName("dataoqejexfd") - .withTable("datauhdk").withSchema("datagywadrklpdyehjr"); + DrillDatasetTypeProperties model = new DrillDatasetTypeProperties().withTableName("datavpnyldjd") + .withTable("datavdryknkx").withSchema("dataxhnrjl"); model = BinaryData.fromObject(model).toObject(DrillDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillSourceTests.java index 7d3a1897a3949..5f27e16c61375 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillSourceTests.java @@ -11,16 +11,15 @@ public final class DrillSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DrillSource model = BinaryData.fromString( - "{\"type\":\"DrillSource\",\"query\":\"datazdoblpdtcyv\",\"queryTimeout\":\"datahboplavgfbvro\",\"additionalColumns\":\"datauexqweyslwlppoh\",\"sourceRetryCount\":\"datafgalexy\",\"sourceRetryWait\":\"datagkadtwd\",\"maxConcurrentConnections\":\"databjx\",\"disableMetricsCollection\":\"datavxcjdobsgv\",\"\":{\"ihsclpnbidnlodkq\":\"datakwltnsnhuvmo\",\"oyzgaevrygggcc\":\"dataqnkptixa\",\"sbymnfc\":\"datazmhoplqtzgt\",\"qzbqy\":\"datao\"}}") + "{\"type\":\"DrillSource\",\"query\":\"dataxtkmknacnfzcy\",\"queryTimeout\":\"datahdjpagwszm\",\"additionalColumns\":\"datagzfeyexbg\",\"sourceRetryCount\":\"datayo\",\"sourceRetryWait\":\"datawigvqgc\",\"maxConcurrentConnections\":\"datacqjg\",\"disableMetricsCollection\":\"dataxpbpj\",\"\":{\"ohehhtl\":\"datanvdabaodiytxq\"}}") .toObject(DrillSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DrillSource model = new DrillSource().withSourceRetryCount("datafgalexy").withSourceRetryWait("datagkadtwd") - .withMaxConcurrentConnections("databjx").withDisableMetricsCollection("datavxcjdobsgv") - .withQueryTimeout("datahboplavgfbvro").withAdditionalColumns("datauexqweyslwlppoh") - .withQuery("datazdoblpdtcyv"); + DrillSource model = new DrillSource().withSourceRetryCount("datayo").withSourceRetryWait("datawigvqgc") + .withMaxConcurrentConnections("datacqjg").withDisableMetricsCollection("dataxpbpj") + .withQueryTimeout("datahdjpagwszm").withAdditionalColumns("datagzfeyexbg").withQuery("dataxtkmknacnfzcy"); model = BinaryData.fromObject(model).toObject(DrillSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillTableDatasetTests.java index 633fccab6e9a1..58e40a2c1ac5f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DrillTableDatasetTests.java @@ -19,33 +19,35 @@ public final class DrillTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DrillTableDataset model = BinaryData.fromString( - "{\"type\":\"DrillTable\",\"typeProperties\":{\"tableName\":\"datazjdcwuzscyf\",\"table\":\"dataxecmasjnfgngxaoj\",\"schema\":\"datayvfx\"},\"description\":\"ckmoaljaxvwxt\",\"structure\":\"datazhvojyffwflbk\",\"schema\":\"datadzuiygtcyzcjef\",\"linkedServiceName\":{\"referenceName\":\"ubaldjcgldryvlr\",\"parameters\":{\"jbfomfbozpjyxe\":\"datahzirmxca\",\"jthp\":\"datappqcwdnn\"}},\"parameters\":{\"nsebcxnouspdyzs\":{\"type\":\"Float\",\"defaultValue\":\"dataycympohxubnn\"},\"jvgspj\":{\"type\":\"Object\",\"defaultValue\":\"datamykdy\"},\"ngwqxcrbcrgyoim\":{\"type\":\"Float\",\"defaultValue\":\"datah\"}},\"annotations\":[\"dataz\",\"datacctvkog\"],\"folder\":{\"name\":\"v\"},\"\":{\"jdkjvdr\":\"datannwcnvpnyl\",\"xhnrjl\":\"dataknkxi\"}}") + "{\"type\":\"DrillTable\",\"typeProperties\":{\"tableName\":\"dataugdcr\",\"table\":\"databplvhmhur\",\"schema\":\"datadjlz\"},\"description\":\"m\",\"structure\":\"dataghqeuyaorservpv\",\"schema\":\"dataorsbeg\",\"linkedServiceName\":{\"referenceName\":\"lmexafjqzyhz\",\"parameters\":{\"nsskydigt\":\"datavtuqvirl\",\"ocsvjekejchxzj\":\"datajx\",\"yfyixecmasjnfgng\":\"datacwuzs\"}},\"parameters\":{\"wxtxuzhvojyf\":{\"type\":\"String\",\"defaultValue\":\"dataeyvfxbfckmoaljax\"},\"jef\":{\"type\":\"Float\",\"defaultValue\":\"datalbkjcdzuiygtcyz\"},\"lrlkbh\":{\"type\":\"Int\",\"defaultValue\":\"dataaldjcgldry\"},\"xefppq\":{\"type\":\"String\",\"defaultValue\":\"datamxcaujbfomfbozpj\"}},\"annotations\":[\"datannjjthpsnxebycy\",\"datapohxubn\"],\"folder\":{\"name\":\"sebcxno\"},\"\":{\"gspjlf\":\"datadyzssjlmykdygj\",\"ngwqxcrbcrgyoim\":\"datah\",\"z\":\"datas\",\"htvagwnnw\":\"datacctvkog\"}}") .toObject(DrillTableDataset.class); - Assertions.assertEquals("ckmoaljaxvwxt", model.description()); - Assertions.assertEquals("ubaldjcgldryvlr", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("nsebcxnouspdyzs").type()); - Assertions.assertEquals("v", model.folder().name()); + Assertions.assertEquals("m", model.description()); + Assertions.assertEquals("lmexafjqzyhz", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("wxtxuzhvojyf").type()); + Assertions.assertEquals("sebcxno", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DrillTableDataset model = new DrillTableDataset().withDescription("ckmoaljaxvwxt") - .withStructure("datazhvojyffwflbk").withSchema("datadzuiygtcyzcjef") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ubaldjcgldryvlr") - .withParameters(mapOf("jbfomfbozpjyxe", "datahzirmxca", "jthp", "datappqcwdnn"))) - .withParameters(mapOf("nsebcxnouspdyzs", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataycympohxubnn"), - "jvgspj", new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datamykdy"), - "ngwqxcrbcrgyoim", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datah"))) - .withAnnotations(Arrays.asList("dataz", "datacctvkog")).withFolder(new DatasetFolder().withName("v")) - .withTableName("datazjdcwuzscyf").withTable("dataxecmasjnfgngxaoj") - .withSchemaTypePropertiesSchema("datayvfx"); + DrillTableDataset model = new DrillTableDataset().withDescription("m").withStructure("dataghqeuyaorservpv") + .withSchema("dataorsbeg") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("lmexafjqzyhz").withParameters( + mapOf("nsskydigt", "datavtuqvirl", "ocsvjekejchxzj", "datajx", "yfyixecmasjnfgng", "datacwuzs"))) + .withParameters(mapOf("wxtxuzhvojyf", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataeyvfxbfckmoaljax"), + "jef", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datalbkjcdzuiygtcyz"), + "lrlkbh", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataaldjcgldry"), + "xefppq", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datamxcaujbfomfbozpj"))) + .withAnnotations(Arrays.asList("datannjjthpsnxebycy", "datapohxubn")) + .withFolder(new DatasetFolder().withName("sebcxno")).withTableName("dataugdcr").withTable("databplvhmhur") + .withSchemaTypePropertiesSchema("datadjlz"); model = BinaryData.fromObject(model).toObject(DrillTableDataset.class); - Assertions.assertEquals("ckmoaljaxvwxt", model.description()); - Assertions.assertEquals("ubaldjcgldryvlr", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("nsebcxnouspdyzs").type()); - Assertions.assertEquals("v", model.folder().name()); + Assertions.assertEquals("m", model.description()); + Assertions.assertEquals("lmexafjqzyhz", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("wxtxuzhvojyf").type()); + Assertions.assertEquals("sebcxno", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsAXSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsAXSourceTests.java index d2a8ccb243351..b115ce1bfec1c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsAXSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsAXSourceTests.java @@ -11,16 +11,16 @@ public final class DynamicsAXSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DynamicsAXSource model = BinaryData.fromString( - "{\"type\":\"DynamicsAXSource\",\"query\":\"dataeaqaviq\",\"httpRequestTimeout\":\"dataylwp\",\"queryTimeout\":\"datacyfccnwmdpbso\",\"additionalColumns\":\"datan\",\"sourceRetryCount\":\"dataxpveruuckr\",\"sourceRetryWait\":\"dataraqaptqyrnlyuy\",\"maxConcurrentConnections\":\"datawwnqoubwbssvfzjj\",\"disableMetricsCollection\":\"datacxeosylgjpp\",\"\":{\"mckzeaia\":\"datajbqrfwnpwvpnbgyx\",\"qpzqivfgemvuicxw\":\"datacauvlfsct\",\"jatj\":\"datayrydlvfnucgwfl\",\"dyfjsskvzv\":\"datandbrcdumkqhatcko\"}}") + "{\"type\":\"DynamicsAXSource\",\"query\":\"datanazjvyiiezdnez\",\"httpRequestTimeout\":\"dataqzd\",\"queryTimeout\":\"datamyutzttroymi\",\"additionalColumns\":\"datakuz\",\"sourceRetryCount\":\"datacegyztzhcfuwm\",\"sourceRetryWait\":\"dataz\",\"maxConcurrentConnections\":\"datamklroogflh\",\"disableMetricsCollection\":\"dataspxblyokjwsszye\",\"\":{\"dhnferdggji\":\"datau\",\"gczjxou\":\"datanzvoehgwgqgcnkg\"}}") .toObject(DynamicsAXSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DynamicsAXSource model = new DynamicsAXSource().withSourceRetryCount("dataxpveruuckr") - .withSourceRetryWait("dataraqaptqyrnlyuy").withMaxConcurrentConnections("datawwnqoubwbssvfzjj") - .withDisableMetricsCollection("datacxeosylgjpp").withQueryTimeout("datacyfccnwmdpbso") - .withAdditionalColumns("datan").withQuery("dataeaqaviq").withHttpRequestTimeout("dataylwp"); + DynamicsAXSource model = new DynamicsAXSource().withSourceRetryCount("datacegyztzhcfuwm") + .withSourceRetryWait("dataz").withMaxConcurrentConnections("datamklroogflh") + .withDisableMetricsCollection("dataspxblyokjwsszye").withQueryTimeout("datamyutzttroymi") + .withAdditionalColumns("datakuz").withQuery("datanazjvyiiezdnez").withHttpRequestTimeout("dataqzd"); model = BinaryData.fromObject(model).toObject(DynamicsAXSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsCrmSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsCrmSourceTests.java index 7a736dbf18bd5..d2aa39564d63f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsCrmSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsCrmSourceTests.java @@ -11,16 +11,15 @@ public final class DynamicsCrmSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DynamicsCrmSource model = BinaryData.fromString( - "{\"type\":\"DynamicsCrmSource\",\"query\":\"datawmfwr\",\"additionalColumns\":\"dataizggvmuotc\",\"sourceRetryCount\":\"dataabfyjampvwx\",\"sourceRetryWait\":\"datahpr\",\"maxConcurrentConnections\":\"datax\",\"disableMetricsCollection\":\"datamnqrbyqhywxb\",\"\":{\"uewgy\":\"datajkarggv\",\"qlugnbudjypli\":\"dataolvxw\",\"qdoxooxuaufqoo\":\"datatgtlansykvlxsyc\",\"t\":\"datawx\"}}") + "{\"type\":\"DynamicsCrmSource\",\"query\":\"datasjhh\",\"additionalColumns\":\"datawizqvgadolep\",\"sourceRetryCount\":\"datalzjhaqxfams\",\"sourceRetryWait\":\"datacesvvrabbyfhzy\",\"maxConcurrentConnections\":\"datarxenp\",\"disableMetricsCollection\":\"dataanlb\",\"\":{\"hoxcgfyzluilzgp\":\"datadwrcjtanbwxhlqio\",\"nxjnqm\":\"datahjakz\",\"vpfyupgojrwpoxuy\":\"datajslwmjlpb\"}}") .toObject(DynamicsCrmSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DynamicsCrmSource model - = new DynamicsCrmSource().withSourceRetryCount("dataabfyjampvwx").withSourceRetryWait("datahpr") - .withMaxConcurrentConnections("datax").withDisableMetricsCollection("datamnqrbyqhywxb") - .withQuery("datawmfwr").withAdditionalColumns("dataizggvmuotc"); + DynamicsCrmSource model = new DynamicsCrmSource().withSourceRetryCount("datalzjhaqxfams") + .withSourceRetryWait("datacesvvrabbyfhzy").withMaxConcurrentConnections("datarxenp") + .withDisableMetricsCollection("dataanlb").withQuery("datasjhh").withAdditionalColumns("datawizqvgadolep"); model = BinaryData.fromObject(model).toObject(DynamicsCrmSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsSourceTests.java index d1a7b4a5957ec..65d58f8ed4db9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/DynamicsSourceTests.java @@ -11,15 +11,15 @@ public final class DynamicsSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DynamicsSource model = BinaryData.fromString( - "{\"type\":\"DynamicsSource\",\"query\":\"datarelokxklgluareg\",\"additionalColumns\":\"datay\",\"sourceRetryCount\":\"dataoisbmv\",\"sourceRetryWait\":\"dataenrcqickhvps\",\"maxConcurrentConnections\":\"datauiuvingmonq\",\"disableMetricsCollection\":\"datatyuqdz\",\"\":{\"kydqy\":\"datajzvaykfjgakays\"}}") + "{\"type\":\"DynamicsSource\",\"query\":\"datamtxbi\",\"additionalColumns\":\"datal\",\"sourceRetryCount\":\"dataeqotvocjktihnwyv\",\"sourceRetryWait\":\"datasgfdmtfnbvtxq\",\"maxConcurrentConnections\":\"datab\",\"disableMetricsCollection\":\"dataqbdiahjxcdhp\",\"\":{\"yn\":\"datawsfddyqp\",\"lj\":\"dataowmjsur\"}}") .toObject(DynamicsSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DynamicsSource model = new DynamicsSource().withSourceRetryCount("dataoisbmv") - .withSourceRetryWait("dataenrcqickhvps").withMaxConcurrentConnections("datauiuvingmonq") - .withDisableMetricsCollection("datatyuqdz").withQuery("datarelokxklgluareg").withAdditionalColumns("datay"); + DynamicsSource model = new DynamicsSource().withSourceRetryCount("dataeqotvocjktihnwyv") + .withSourceRetryWait("datasgfdmtfnbvtxq").withMaxConcurrentConnections("datab") + .withDisableMetricsCollection("dataqbdiahjxcdhp").withQuery("datamtxbi").withAdditionalColumns("datal"); model = BinaryData.fromObject(model).toObject(DynamicsSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaObjectDatasetTests.java index 9b06e29a43a64..5a79cfe049861 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaObjectDatasetTests.java @@ -19,34 +19,33 @@ public final class EloquaObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { EloquaObjectDataset model = BinaryData.fromString( - "{\"type\":\"EloquaObject\",\"typeProperties\":{\"tableName\":\"datalvxboc\"},\"description\":\"wmfvuhzmolhveoln\",\"structure\":\"datafm\",\"schema\":\"datadxqupyml\",\"linkedServiceName\":{\"referenceName\":\"klmnjqzmqynhitr\",\"parameters\":{\"nrjocogwf\":\"datagqrbthbfpi\",\"pkhuvnlmdcnut\":\"datakywzrqeiad\",\"ioynctfqhhvv\":\"dataexmizunzbq\"}},\"parameters\":{\"vavlyaqtlocnwme\":{\"type\":\"Array\",\"defaultValue\":\"dataaaaiaib\"},\"v\":{\"type\":\"Int\",\"defaultValue\":\"datazuzqcrlko\"},\"ozf\":{\"type\":\"Bool\",\"defaultValue\":\"datayhenfsfyqncowmh\"},\"agwaakktbjort\":{\"type\":\"Float\",\"defaultValue\":\"datajiaaosla\"}},\"annotations\":[\"dataajqhsnsejplis\",\"dataxyljzbkdw\",\"datafjwxgvtkjctvrpea\",\"datazzkvfc\"],\"folder\":{\"name\":\"vq\"},\"\":{\"tgcptctxpoeg\":\"datahtraitrmsukxtu\"}}") + "{\"type\":\"EloquaObject\",\"typeProperties\":{\"tableName\":\"dataqejex\"},\"description\":\"lhuhdkubgyw\",\"structure\":\"datarklpdyehjrwcflv\",\"schema\":\"dataocaywmfvuhz\",\"linkedServiceName\":{\"referenceName\":\"olhve\",\"parameters\":{\"ymlhklmnjqzm\":\"datagsfmhwdxqu\",\"bthb\":\"dataynhitrnwqgq\"}},\"parameters\":{\"ywzrqeiadd\":{\"type\":\"Bool\",\"defaultValue\":\"datarjocogwfv\"},\"z\":{\"type\":\"Int\",\"defaultValue\":\"datauvnlmdcnutiexm\"},\"tfqhhvvwzprjaaai\":{\"type\":\"String\",\"defaultValue\":\"databqvioyn\"},\"cn\":{\"type\":\"String\",\"defaultValue\":\"datatvavlyaqtl\"}},\"annotations\":[\"datafzvz\",\"datazqcrlkorvbzg\",\"datahenfsfyq\"],\"folder\":{\"name\":\"wmh\"},\"\":{\"agwaakktbjort\":\"datafbywjiaaosla\",\"lislxyljzbkd\":\"datatkajqhsnsej\"}}") .toObject(EloquaObjectDataset.class); - Assertions.assertEquals("wmfvuhzmolhveoln", model.description()); - Assertions.assertEquals("klmnjqzmqynhitr", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("vavlyaqtlocnwme").type()); - Assertions.assertEquals("vq", model.folder().name()); + Assertions.assertEquals("lhuhdkubgyw", model.description()); + Assertions.assertEquals("olhve", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("ywzrqeiadd").type()); + Assertions.assertEquals("wmh", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - EloquaObjectDataset model = new EloquaObjectDataset().withDescription("wmfvuhzmolhveoln") - .withStructure("datafm").withSchema("datadxqupyml") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("klmnjqzmqynhitr") - .withParameters(mapOf("nrjocogwf", "datagqrbthbfpi", "pkhuvnlmdcnut", "datakywzrqeiad", "ioynctfqhhvv", - "dataexmizunzbq"))) - .withParameters(mapOf("vavlyaqtlocnwme", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataaaaiaib"), "v", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datazuzqcrlko"), "ozf", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datayhenfsfyqncowmh"), - "agwaakktbjort", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datajiaaosla"))) - .withAnnotations(Arrays.asList("dataajqhsnsejplis", "dataxyljzbkdw", "datafjwxgvtkjctvrpea", "datazzkvfc")) - .withFolder(new DatasetFolder().withName("vq")).withTableName("datalvxboc"); + EloquaObjectDataset model = new EloquaObjectDataset().withDescription("lhuhdkubgyw") + .withStructure("datarklpdyehjrwcflv").withSchema("dataocaywmfvuhz") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("olhve") + .withParameters(mapOf("ymlhklmnjqzm", "datagsfmhwdxqu", "bthb", "dataynhitrnwqgq"))) + .withParameters(mapOf("ywzrqeiadd", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datarjocogwfv"), "z", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datauvnlmdcnutiexm"), + "tfqhhvvwzprjaaai", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("databqvioyn"), "cn", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datatvavlyaqtl"))) + .withAnnotations(Arrays.asList("datafzvz", "datazqcrlkorvbzg", "datahenfsfyq")) + .withFolder(new DatasetFolder().withName("wmh")).withTableName("dataqejex"); model = BinaryData.fromObject(model).toObject(EloquaObjectDataset.class); - Assertions.assertEquals("wmfvuhzmolhveoln", model.description()); - Assertions.assertEquals("klmnjqzmqynhitr", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("vavlyaqtlocnwme").type()); - Assertions.assertEquals("vq", model.folder().name()); + Assertions.assertEquals("lhuhdkubgyw", model.description()); + Assertions.assertEquals("olhve", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("ywzrqeiadd").type()); + Assertions.assertEquals("wmh", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaSourceTests.java index e4d1970071f30..90f90b9d15f93 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EloquaSourceTests.java @@ -11,15 +11,16 @@ public final class EloquaSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { EloquaSource model = BinaryData.fromString( - "{\"type\":\"EloquaSource\",\"query\":\"datafqqrarolc\",\"queryTimeout\":\"datahbynlbwcnn\",\"additionalColumns\":\"datafgstdifbyfjsl\",\"sourceRetryCount\":\"datagee\",\"sourceRetryWait\":\"datasoj\",\"maxConcurrentConnections\":\"dataarliig\",\"disableMetricsCollection\":\"datav\",\"\":{\"xuuqcmunh\":\"datadsjhx\"}}") + "{\"type\":\"EloquaSource\",\"query\":\"datavijdr\",\"queryTimeout\":\"datayqvhz\",\"additionalColumns\":\"datayvhrenozl\",\"sourceRetryCount\":\"dataqfghlosho\",\"sourceRetryWait\":\"datakpcmtsbandesalv\",\"maxConcurrentConnections\":\"datawrljmlo\",\"disableMetricsCollection\":\"datatzvtfyqe\",\"\":{\"xhcygfg\":\"databsyni\",\"aosttbwap\":\"datamdbazggr\"}}") .toObject(EloquaSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - EloquaSource model = new EloquaSource().withSourceRetryCount("datagee").withSourceRetryWait("datasoj") - .withMaxConcurrentConnections("dataarliig").withDisableMetricsCollection("datav") - .withQueryTimeout("datahbynlbwcnn").withAdditionalColumns("datafgstdifbyfjsl").withQuery("datafqqrarolc"); + EloquaSource model + = new EloquaSource().withSourceRetryCount("dataqfghlosho").withSourceRetryWait("datakpcmtsbandesalv") + .withMaxConcurrentConnections("datawrljmlo").withDisableMetricsCollection("datatzvtfyqe") + .withQueryTimeout("datayqvhz").withAdditionalColumns("datayvhrenozl").withQuery("datavijdr"); model = BinaryData.fromObject(model).toObject(EloquaSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EntityReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EntityReferenceTests.java index 5db863c657eba..f5c771b43ff6a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EntityReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EntityReferenceTests.java @@ -13,19 +13,19 @@ public final class EntityReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { EntityReference model - = BinaryData.fromString("{\"type\":\"IntegrationRuntimeReference\",\"referenceName\":\"somuogeqe\"}") + = BinaryData.fromString("{\"type\":\"IntegrationRuntimeReference\",\"referenceName\":\"bkubzqazdlrkvi\"}") .toObject(EntityReference.class); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE, model.type()); - Assertions.assertEquals("somuogeqe", model.referenceName()); + Assertions.assertEquals("bkubzqazdlrkvi", model.referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { EntityReference model = new EntityReference().withType(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE) - .withReferenceName("somuogeqe"); + .withReferenceName("bkubzqazdlrkvi"); model = BinaryData.fromObject(model).toObject(EntityReference.class); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE, model.type()); - Assertions.assertEquals("somuogeqe", model.referenceName()); + Assertions.assertEquals("bkubzqazdlrkvi", model.referenceName()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTests.java index ce71fb99cd703..d39b0a2cb0313 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTests.java @@ -12,18 +12,18 @@ public final class EnvironmentVariableSetupTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { EnvironmentVariableSetup model = BinaryData.fromString( - "{\"type\":\"EnvironmentVariableSetup\",\"typeProperties\":{\"variableName\":\"ivrsgqbmolxeom\",\"variableValue\":\"bzvtvxx\"}}") + "{\"type\":\"EnvironmentVariableSetup\",\"typeProperties\":{\"variableName\":\"cz\",\"variableValue\":\"tniwfcu\"}}") .toObject(EnvironmentVariableSetup.class); - Assertions.assertEquals("ivrsgqbmolxeom", model.variableName()); - Assertions.assertEquals("bzvtvxx", model.variableValue()); + Assertions.assertEquals("cz", model.variableName()); + Assertions.assertEquals("tniwfcu", model.variableValue()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { EnvironmentVariableSetup model - = new EnvironmentVariableSetup().withVariableName("ivrsgqbmolxeom").withVariableValue("bzvtvxx"); + = new EnvironmentVariableSetup().withVariableName("cz").withVariableValue("tniwfcu"); model = BinaryData.fromObject(model).toObject(EnvironmentVariableSetup.class); - Assertions.assertEquals("ivrsgqbmolxeom", model.variableName()); - Assertions.assertEquals("bzvtvxx", model.variableValue()); + Assertions.assertEquals("cz", model.variableName()); + Assertions.assertEquals("tniwfcu", model.variableValue()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTypePropertiesTests.java index babd96cb108fb..a40f2dbe0d0de 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/EnvironmentVariableSetupTypePropertiesTests.java @@ -12,18 +12,18 @@ public final class EnvironmentVariableSetupTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { EnvironmentVariableSetupTypeProperties model - = BinaryData.fromString("{\"variableName\":\"sfoyacgihnal\",\"variableValue\":\"cuedybkbgdwbmi\"}") + = BinaryData.fromString("{\"variableName\":\"whxm\",\"variableValue\":\"ibxkcegc\"}") .toObject(EnvironmentVariableSetupTypeProperties.class); - Assertions.assertEquals("sfoyacgihnal", model.variableName()); - Assertions.assertEquals("cuedybkbgdwbmi", model.variableValue()); + Assertions.assertEquals("whxm", model.variableName()); + Assertions.assertEquals("ibxkcegc", model.variableValue()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - EnvironmentVariableSetupTypeProperties model = new EnvironmentVariableSetupTypeProperties() - .withVariableName("sfoyacgihnal").withVariableValue("cuedybkbgdwbmi"); + EnvironmentVariableSetupTypeProperties model + = new EnvironmentVariableSetupTypeProperties().withVariableName("whxm").withVariableValue("ibxkcegc"); model = BinaryData.fromObject(model).toObject(EnvironmentVariableSetupTypeProperties.class); - Assertions.assertEquals("sfoyacgihnal", model.variableName()); - Assertions.assertEquals("cuedybkbgdwbmi", model.variableValue()); + Assertions.assertEquals("whxm", model.variableName()); + Assertions.assertEquals("ibxkcegc", model.variableValue()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExcelSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExcelSourceTests.java index 07f6c2f861c15..c7e1896f5fcaa 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExcelSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExcelSourceTests.java @@ -14,18 +14,18 @@ public final class ExcelSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExcelSource model = BinaryData.fromString( - "{\"type\":\"ExcelSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datamerpbctrwrvn\",\"disableMetricsCollection\":\"datamacbrywqqezt\",\"\":{\"hhvsfgywki\":\"datajlgxrsnbtrooa\",\"txngmebvninj\":\"datakh\",\"kqjjouhoxkct\":\"datadk\"}},\"additionalColumns\":\"datacctvcjdrmknkkhv\",\"sourceRetryCount\":\"datajq\",\"sourceRetryWait\":\"datamy\",\"maxConcurrentConnections\":\"datazbxfgosr\",\"disableMetricsCollection\":\"datal\",\"\":{\"sbzmixwaxtnk\":\"datafzsegu\",\"gdaajlhgsuqmrky\":\"datatzdvx\",\"hpqlxnbdjt\":\"dataovcbdsr\"}}") + "{\"type\":\"ExcelSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datanpdjomd\",\"disableMetricsCollection\":\"datadwosjxywwvilkyht\",\"\":{\"jgsjmcybrp\":\"datayhdb\",\"uuhdk\":\"datajenbxtkghrrx\",\"yxnhu\":\"datakizyx\",\"zyrpdxygfpqxseme\":\"datajwipfryivpe\"}},\"additionalColumns\":\"datapdmm\",\"sourceRetryCount\":\"dataft\",\"sourceRetryWait\":\"dataqe\",\"maxConcurrentConnections\":\"datajopotefek\",\"disableMetricsCollection\":\"datamg\",\"\":{\"dkvhui\":\"datawidnrd\",\"hwbead\":\"datadyhodisypgapfd\"}}") .toObject(ExcelSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExcelSource model = new ExcelSource().withSourceRetryCount("datajq").withSourceRetryWait("datamy") - .withMaxConcurrentConnections("datazbxfgosr").withDisableMetricsCollection("datal") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datamerpbctrwrvn") - .withDisableMetricsCollection("datamacbrywqqezt") + ExcelSource model = new ExcelSource().withSourceRetryCount("dataft").withSourceRetryWait("dataqe") + .withMaxConcurrentConnections("datajopotefek").withDisableMetricsCollection("datamg") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datanpdjomd") + .withDisableMetricsCollection("datadwosjxywwvilkyht") .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) - .withAdditionalColumns("datacctvcjdrmknkkhv"); + .withAdditionalColumns("datapdmm"); model = BinaryData.fromObject(model).toObject(ExcelSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTests.java index 4e793eef4c4f8..6408492dfcdf9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTests.java @@ -27,70 +27,85 @@ public final class ExecuteDataFlowActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecuteDataFlowActivity model = BinaryData.fromString( - "{\"type\":\"ExecuteDataFlow\",\"typeProperties\":{\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"mxtmedoykekbdwq\",\"datasetParameters\":\"datau\",\"parameters\":{\"uabeisqkot\":\"datarfjzyini\",\"huyavhesqn\":\"datamhryri\",\"teprs\":\"datas\"},\"\":{\"igadpq\":\"dataakcchcnmzvhdu\",\"uyscwvd\":\"datachgvwggylbmfrxof\",\"o\":\"datazfgdxyrpp\",\"tbpaircnupmz\":\"datadohiotgf\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"ziejo\",\"parameters\":{\"sfhjrsx\":\"dataofmmcej\",\"zxokpqnpdlc\":\"datamlx\"}},\"folderPath\":\"datasekdfhnhbkt\"},\"integrationRuntime\":{\"referenceName\":\"eonl\",\"parameters\":{\"dx\":\"datal\",\"b\":\"datatqknze\"}},\"compute\":{\"computeType\":\"datanqneo\",\"coreCount\":\"datacrmng\"},\"traceLevel\":\"datalinlwcxrxduxctoj\",\"continueOnError\":\"datakmdegmivfuylpctl\",\"runConcurrently\":\"dataob\",\"sourceStagingConcurrency\":\"dataabt\"},\"linkedServiceName\":{\"referenceName\":\"kmk\",\"parameters\":{\"vepmhohqxl\":\"dataqktgkofzzsohcae\"}},\"policy\":{\"timeout\":\"datatliwoodndu\",\"retry\":\"dataykyzirgiyqzuh\",\"retryIntervalInSeconds\":228064254,\"secureInput\":false,\"secureOutput\":true,\"\":{\"adeqslhz\":\"datasdaruwvrvxoozyhu\",\"mqazolroqusrlkp\":\"datay\"}},\"name\":\"ty\",\"description\":\"drnwsfa\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"kdlhuslq\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"tfdcwekbbvtcox\":\"datajdoq\",\"tjcpoynbsttur\":\"datadgjdpyhemazhrwp\",\"lgfecsreojs\":\"dataqvxzlwgaius\"}}],\"userProperties\":[{\"name\":\"xb\",\"value\":\"dataziffpvvgjxbm\"},{\"name\":\"heyamoe\",\"value\":\"datayg\"}],\"\":{\"dizxseyjqklaihq\":\"datait\",\"etp\":\"databrmrhljqqb\",\"tifbvcveomdl\":\"datadjsu\"}}") + "{\"type\":\"ExecuteDataFlow\",\"typeProperties\":{\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ksoxykrma\",\"datasetParameters\":\"datanlbvahjlvbnlxebg\",\"parameters\":{\"yy\":\"datapsged\",\"zffzhbhelg\":\"dataueifmtgntlfdiqzv\",\"y\":\"datalrdgpudbimehdx\",\"ut\":\"datayfhwkbhapfnyo\"},\"\":{\"gzunbcvfz\":\"datahjrmfejeihnh\",\"afrqqfgudobutkq\":\"datapyirngfujv\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"xxcaxgraiki\",\"parameters\":{\"asbfcblvakhdi\":\"datadnchrvsfnlgwp\"}},\"folderPath\":\"dataxtfvoasdhdii\"},\"integrationRuntime\":{\"referenceName\":\"znffmxtmqartpdyh\",\"parameters\":{\"fjcowrzqyoc\":\"dataxmrhxpmtztvxfgli\"}},\"compute\":{\"computeType\":\"datagrtnitaampg\",\"coreCount\":\"datayvpxpcjnb\"},\"traceLevel\":\"datafexzzijtctfew\",\"continueOnError\":\"datawtzp\",\"runConcurrently\":\"datayluvqp\",\"sourceStagingConcurrency\":\"datavoyqs\"},\"linkedServiceName\":{\"referenceName\":\"twlxv\",\"parameters\":{\"sxcqto\":\"dataaldss\"}},\"policy\":{\"timeout\":\"datanxinl\",\"retry\":\"datacc\",\"retryIntervalInSeconds\":999097881,\"secureInput\":false,\"secureOutput\":false,\"\":{\"wybbda\":\"datafxnenhyhdusaykrj\"}},\"name\":\"dqttzslt\",\"description\":\"mdacetjmap\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"jxdirdcxuiamr\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"yivvtx\":\"datacqofpwjjtdz\",\"o\":\"datapem\",\"jdihtxvmnyslpdqd\":\"datatohzfvysvud\"}},{\"activity\":\"zjpp\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"vyyasildbq\":\"datavtymzdtnjxv\",\"mqpjbkb\":\"datagnfxgzzqqwsvj\"}},{\"activity\":\"sugkdvmgpeitfbgy\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Succeeded\"],\"\":{\"ookhcu\":\"datadsgfztmhvuoavp\",\"sutseejtfnjrrxf\":\"datawgbjzznmjwqwyhh\"}},{\"activity\":\"uywzpcx\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"h\":\"datalowm\",\"tceehqeahlfujp\":\"datauhywdckvcof\",\"uumldunalo\":\"datavtakijwkwed\"}}],\"userProperties\":[{\"name\":\"ikfqcbe\",\"value\":\"datansszu\"},{\"name\":\"dvhqecqqiulwfz\",\"value\":\"dataszgbgtwaquiuzsn\"},{\"name\":\"jgnmpu\",\"value\":\"datasjfvdajmczlvcxm\"}],\"\":{\"tbgkx\":\"databrp\",\"lszcwomayr\":\"dataxwjzleeup\",\"dfxnxtiwi\":\"dataatrjpa\",\"ihsgt\":\"datanho\"}}") .toObject(ExecuteDataFlowActivity.class); - Assertions.assertEquals("ty", model.name()); - Assertions.assertEquals("drnwsfa", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("kdlhuslq", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("xb", model.userProperties().get(0).name()); - Assertions.assertEquals("kmk", model.linkedServiceName().referenceName()); - Assertions.assertEquals(228064254, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("dqttzslt", model.name()); + Assertions.assertEquals("mdacetjmap", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("jxdirdcxuiamr", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ikfqcbe", model.userProperties().get(0).name()); + Assertions.assertEquals("twlxv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(999097881, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals(false, model.policy().secureOutput()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("mxtmedoykekbdwq", model.dataFlow().referenceName()); - Assertions.assertEquals("ziejo", model.staging().linkedService().referenceName()); - Assertions.assertEquals("eonl", model.integrationRuntime().referenceName()); + Assertions.assertEquals("ksoxykrma", model.dataFlow().referenceName()); + Assertions.assertEquals("xxcaxgraiki", model.staging().linkedService().referenceName()); + Assertions.assertEquals("znffmxtmqartpdyh", model.integrationRuntime().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExecuteDataFlowActivity model = new ExecuteDataFlowActivity().withName("ty").withDescription("drnwsfa") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("kdlhuslq") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("xb").withValue("dataziffpvvgjxbm"), - new UserProperty().withName("heyamoe").withValue("datayg"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("kmk") - .withParameters(mapOf("vepmhohqxl", "dataqktgkofzzsohcae"))) - .withPolicy(new ActivityPolicy().withTimeout("datatliwoodndu").withRetry("dataykyzirgiyqzuh") - .withRetryIntervalInSeconds(228064254).withSecureInput(false).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withDataFlow(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("mxtmedoykekbdwq").withDatasetParameters("datau") - .withParameters(mapOf("uabeisqkot", "datarfjzyini", "huyavhesqn", "datamhryri", "teprs", "datas")) - .withAdditionalProperties(mapOf())) - .withStaging(new DataFlowStagingInfo() - .withLinkedService(new LinkedServiceReference().withReferenceName("ziejo") - .withParameters(mapOf("sfhjrsx", "dataofmmcej", "zxokpqnpdlc", "datamlx"))) - .withFolderPath("datasekdfhnhbkt")) - .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("eonl") - .withParameters(mapOf("dx", "datal", "b", "datatqknze"))) - .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datanqneo") - .withCoreCount("datacrmng")) - .withTraceLevel("datalinlwcxrxduxctoj").withContinueOnError("datakmdegmivfuylpctl") - .withRunConcurrently("dataob").withSourceStagingConcurrency("dataabt"); + ExecuteDataFlowActivity model + = new ExecuteDataFlowActivity().withName("dqttzslt").withDescription("mdacetjmap") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("jxdirdcxuiamr") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency() + .withActivity("zjpp").withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("sugkdvmgpeitfbgy") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, + DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("uywzpcx") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("ikfqcbe").withValue("datansszu"), + new UserProperty().withName("dvhqecqqiulwfz").withValue("dataszgbgtwaquiuzsn"), + new UserProperty().withName("jgnmpu").withValue("datasjfvdajmczlvcxm"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("twlxv") + .withParameters(mapOf("sxcqto", "dataaldss"))) + .withPolicy(new ActivityPolicy().withTimeout("datanxinl").withRetry("datacc") + .withRetryIntervalInSeconds(999097881).withSecureInput(false).withSecureOutput(false) + .withAdditionalProperties(mapOf())) + .withDataFlow(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("ksoxykrma").withDatasetParameters("datanlbvahjlvbnlxebg") + .withParameters(mapOf("yy", "datapsged", "zffzhbhelg", "dataueifmtgntlfdiqzv", "y", + "datalrdgpudbimehdx", "ut", "datayfhwkbhapfnyo")) + .withAdditionalProperties(mapOf())) + .withStaging(new DataFlowStagingInfo() + .withLinkedService(new LinkedServiceReference().withReferenceName("xxcaxgraiki") + .withParameters(mapOf("asbfcblvakhdi", "datadnchrvsfnlgwp"))) + .withFolderPath("dataxtfvoasdhdii")) + .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("znffmxtmqartpdyh") + .withParameters(mapOf("fjcowrzqyoc", "dataxmrhxpmtztvxfgli"))) + .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datagrtnitaampg") + .withCoreCount("datayvpxpcjnb")) + .withTraceLevel("datafexzzijtctfew").withContinueOnError("datawtzp").withRunConcurrently("datayluvqp") + .withSourceStagingConcurrency("datavoyqs"); model = BinaryData.fromObject(model).toObject(ExecuteDataFlowActivity.class); - Assertions.assertEquals("ty", model.name()); - Assertions.assertEquals("drnwsfa", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("kdlhuslq", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("xb", model.userProperties().get(0).name()); - Assertions.assertEquals("kmk", model.linkedServiceName().referenceName()); - Assertions.assertEquals(228064254, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("dqttzslt", model.name()); + Assertions.assertEquals("mdacetjmap", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("jxdirdcxuiamr", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ikfqcbe", model.userProperties().get(0).name()); + Assertions.assertEquals("twlxv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(999097881, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals(false, model.policy().secureOutput()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("mxtmedoykekbdwq", model.dataFlow().referenceName()); - Assertions.assertEquals("ziejo", model.staging().linkedService().referenceName()); - Assertions.assertEquals("eonl", model.integrationRuntime().referenceName()); + Assertions.assertEquals("ksoxykrma", model.dataFlow().referenceName()); + Assertions.assertEquals("xxcaxgraiki", model.staging().linkedService().referenceName()); + Assertions.assertEquals("znffmxtmqartpdyh", model.integrationRuntime().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesComputeTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesComputeTests.java index ec48fd3e4a270..b88c756187a10 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesComputeTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesComputeTests.java @@ -11,14 +11,14 @@ public final class ExecuteDataFlowActivityTypePropertiesComputeTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecuteDataFlowActivityTypePropertiesCompute model - = BinaryData.fromString("{\"computeType\":\"dataqujtc\",\"coreCount\":\"datajpkaftkaf\"}") + = BinaryData.fromString("{\"computeType\":\"datafxuys\",\"coreCount\":\"datavdqzf\"}") .toObject(ExecuteDataFlowActivityTypePropertiesCompute.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ExecuteDataFlowActivityTypePropertiesCompute model = new ExecuteDataFlowActivityTypePropertiesCompute() - .withComputeType("dataqujtc").withCoreCount("datajpkaftkaf"); + .withComputeType("datafxuys").withCoreCount("datavdqzf"); model = BinaryData.fromObject(model).toObject(ExecuteDataFlowActivityTypePropertiesCompute.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesTests.java index 97373903b5f58..7f0d921a7baed 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteDataFlowActivityTypePropertiesTests.java @@ -20,35 +20,36 @@ public final class ExecuteDataFlowActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecuteDataFlowActivityTypeProperties model = BinaryData.fromString( - "{\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"jguwdfn\",\"datasetParameters\":\"datavuquf\",\"parameters\":{\"vovj\":\"datauibujjvojmyn\"},\"\":{\"awmrhpfg\":\"dataewfqvlh\",\"mefppjxtgffwq\":\"datalrnf\",\"gfgirrzyngdvdr\":\"datad\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"qfrxggvstyxv\",\"parameters\":{\"imfpnp\":\"dataaqf\",\"tdorvxdwgpu\":\"datakdgjnd\",\"dzjmjkg\":\"datajeffpidwqr\"}},\"folderPath\":\"datapplcoqb\"},\"integrationRuntime\":{\"referenceName\":\"e\",\"parameters\":{\"ldlok\":\"datazaja\"}},\"compute\":{\"computeType\":\"datazfltxqpozq\",\"coreCount\":\"dataqfkfevhgjk\"},\"traceLevel\":\"datawjnvhuswmwvqb\",\"continueOnError\":\"datazjmfqusp\",\"runConcurrently\":\"datavthkgjaaqhd\",\"sourceStagingConcurrency\":\"datafjvfrg\"}") + "{\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"gmfnpeluvxsicp\",\"datasetParameters\":\"datakupngo\",\"parameters\":{\"f\":\"dataayrguxfjjg\"},\"\":{\"krppxj\":\"datagrubofhkbjg\",\"ladibsjirhaqedfu\":\"datarujdskkkz\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"akfxz\",\"parameters\":{\"jfrpbdxsjceyyebg\":\"datacwhjv\"}},\"folderPath\":\"datafntrbnvwhq\"},\"integrationRuntime\":{\"referenceName\":\"qdyfugksmxvevu\",\"parameters\":{\"naynlxwukpqcf\":\"datanyuy\",\"hizmzbaqrxzjm\":\"dataxagtiyvdslrrtv\",\"bdwqwh\":\"datatmedoyke\"}},\"compute\":{\"computeType\":\"datayrfjzyiniuua\",\"coreCount\":\"dataisqkotbmhryrif\"},\"traceLevel\":\"datayavhesqnvsqte\",\"continueOnError\":\"dataswnfakcchcnm\",\"runConcurrently\":\"datahduyigadpqpchgv\",\"sourceStagingConcurrency\":\"datagylbmfr\"}") .toObject(ExecuteDataFlowActivityTypeProperties.class); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("jguwdfn", model.dataFlow().referenceName()); - Assertions.assertEquals("qfrxggvstyxv", model.staging().linkedService().referenceName()); - Assertions.assertEquals("e", model.integrationRuntime().referenceName()); + Assertions.assertEquals("gmfnpeluvxsicp", model.dataFlow().referenceName()); + Assertions.assertEquals("akfxz", model.staging().linkedService().referenceName()); + Assertions.assertEquals("qdyfugksmxvevu", model.integrationRuntime().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExecuteDataFlowActivityTypeProperties model = new ExecuteDataFlowActivityTypeProperties() - .withDataFlow(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("jguwdfn").withDatasetParameters("datavuquf") - .withParameters(mapOf("vovj", "datauibujjvojmyn")).withAdditionalProperties(mapOf())) - .withStaging(new DataFlowStagingInfo() - .withLinkedService(new LinkedServiceReference().withReferenceName("qfrxggvstyxv").withParameters( - mapOf("imfpnp", "dataaqf", "tdorvxdwgpu", "datakdgjnd", "dzjmjkg", "datajeffpidwqr"))) - .withFolderPath("datapplcoqb")) - .withIntegrationRuntime( - new IntegrationRuntimeReference().withReferenceName("e").withParameters(mapOf("ldlok", "datazaja"))) - .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datazfltxqpozq") - .withCoreCount("dataqfkfevhgjk")) - .withTraceLevel("datawjnvhuswmwvqb").withContinueOnError("datazjmfqusp") - .withRunConcurrently("datavthkgjaaqhd").withSourceStagingConcurrency("datafjvfrg"); + ExecuteDataFlowActivityTypeProperties model + = new ExecuteDataFlowActivityTypeProperties() + .withDataFlow(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("gmfnpeluvxsicp").withDatasetParameters("datakupngo") + .withParameters(mapOf("f", "dataayrguxfjjg")).withAdditionalProperties(mapOf())) + .withStaging( + new DataFlowStagingInfo().withLinkedService(new LinkedServiceReference().withReferenceName("akfxz") + .withParameters(mapOf("jfrpbdxsjceyyebg", "datacwhjv"))).withFolderPath("datafntrbnvwhq")) + .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("qdyfugksmxvevu") + .withParameters(mapOf("naynlxwukpqcf", "datanyuy", "hizmzbaqrxzjm", "dataxagtiyvdslrrtv", "bdwqwh", + "datatmedoyke"))) + .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datayrfjzyiniuua") + .withCoreCount("dataisqkotbmhryrif")) + .withTraceLevel("datayavhesqnvsqte").withContinueOnError("dataswnfakcchcnm") + .withRunConcurrently("datahduyigadpqpchgv").withSourceStagingConcurrency("datagylbmfr"); model = BinaryData.fromObject(model).toObject(ExecuteDataFlowActivityTypeProperties.class); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("jguwdfn", model.dataFlow().referenceName()); - Assertions.assertEquals("qfrxggvstyxv", model.staging().linkedService().referenceName()); - Assertions.assertEquals("e", model.integrationRuntime().referenceName()); + Assertions.assertEquals("gmfnpeluvxsicp", model.dataFlow().referenceName()); + Assertions.assertEquals("akfxz", model.staging().linkedService().referenceName()); + Assertions.assertEquals("qdyfugksmxvevu", model.integrationRuntime().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityPolicyTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityPolicyTests.java index 488a241d575b3..33c501b1b6cf5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityPolicyTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityPolicyTests.java @@ -13,9 +13,9 @@ public final class ExecutePipelineActivityPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExecutePipelineActivityPolicy model - = BinaryData.fromString("{\"secureInput\":true,\"\":{\"efpyjtwwaxx\":\"datag\"}}") - .toObject(ExecutePipelineActivityPolicy.class); + ExecutePipelineActivityPolicy model = BinaryData.fromString( + "{\"secureInput\":true,\"\":{\"fqbeaizvnnhxgi\":\"dataho\",\"bwpdvedm\":\"datadkrgdascmfnk\",\"onkfb\":\"datackbgxgykxszet\"}}") + .toObject(ExecutePipelineActivityPolicy.class); Assertions.assertEquals(true, model.secureInput()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTests.java index 8d47c0faebd92..a2822973944af 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTests.java @@ -22,62 +22,50 @@ public final class ExecutePipelineActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecutePipelineActivity model = BinaryData.fromString( - "{\"type\":\"ExecutePipeline\",\"policy\":{\"secureInput\":false,\"\":{\"knywxpmefbnccbvc\":\"datavjdxlbsnskcksfx\",\"kmifyx\":\"datao\"}},\"typeProperties\":{\"pipeline\":{\"referenceName\":\"nu\",\"name\":\"isfnbtqdrkwr\"},\"parameters\":{\"fqne\":\"dataoidhbulvkisyh\",\"dddeahfgdjahnsm\":\"datap\",\"qdxjdolobt\":\"datatkh\"},\"waitOnCompletion\":true},\"name\":\"xnlaurviyntc\",\"description\":\"pbzoqtf\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"elblb\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Succeeded\",\"Failed\"],\"\":{\"mmqzbznrjw\":\"dataaxttoenfohipijfy\",\"xdi\":\"dataecktcwgnkxjd\",\"wotpiaklefw\":\"datandz\"}},{\"activity\":\"iabfntrmkeawmfe\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"ffnngiu\":\"dataljbnfw\",\"pzjtiktgmdlw\":\"databpgskgpwspxhhnv\"}},{\"activity\":\"fstizemakgzcmbg\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Skipped\"],\"\":{\"wxabvxw\":\"dataflckumjjpx\",\"gy\":\"dataagoeillsz\",\"bzuilynbdvbuxlji\":\"dataozkm\"}},{\"activity\":\"yrwdmgrfhvew\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Succeeded\"],\"\":{\"pf\":\"datak\",\"hpnwzpkisefygdau\":\"dataswbpjzoyzydlysz\"}}],\"userProperties\":[{\"name\":\"kgmgqynejqk\",\"value\":\"datasxiczvfxoihc\"},{\"name\":\"xexbksaf\",\"value\":\"dataecwyrtluujyespcg\"},{\"name\":\"szwvooxieyyww\",\"value\":\"dataiwiaqrc\"}],\"\":{\"zszcrwhr\":\"databktbviaqv\"}}") + "{\"type\":\"ExecutePipeline\",\"policy\":{\"secureInput\":false,\"\":{\"gawqxrenjzlqbt\":\"dataamdffoibxjgcuppw\"}},\"typeProperties\":{\"pipeline\":{\"referenceName\":\"ft\",\"name\":\"pzhox\"},\"parameters\":{\"v\":\"dataynouhjvtef\",\"paralr\":\"dataedfzxsq\",\"iqgqvprl\":\"datasfnjokrf\",\"xdxnmuosoziqcui\":\"datasglqiuqsqzu\"},\"waitOnCompletion\":true},\"name\":\"yaxpukxt\",\"description\":\"ejxwbr\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"xlkwyqoaejylqge\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Failed\"],\"\":{\"t\":\"dataatyoxvbwsatz\",\"eudpyh\":\"datanrortjtyl\"}},{\"activity\":\"nngijnzlokxihf\",\"dependencyConditions\":[\"Failed\"],\"\":{\"rfvbicd\":\"datajlrfwqnssxid\",\"qnqllmqeauizk\":\"datavypfobzxbfc\",\"e\":\"datajqfachfmvqnkgst\"}}],\"userProperties\":[{\"name\":\"vscfpwpqvg\",\"value\":\"datapwm\"}],\"\":{\"mtvwp\":\"datahburxnagvc\",\"zlfhn\":\"datavptqnqbdxwyo\",\"blvcalb\":\"datajuuwmcugveiiegoo\"}}") .toObject(ExecutePipelineActivity.class); - Assertions.assertEquals("xnlaurviyntc", model.name()); - Assertions.assertEquals("pbzoqtf", model.description()); + Assertions.assertEquals("yaxpukxt", model.name()); + Assertions.assertEquals("ejxwbr", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("elblb", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("kgmgqynejqk", model.userProperties().get(0).name()); + Assertions.assertEquals("xlkwyqoaejylqge", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("vscfpwpqvg", model.userProperties().get(0).name()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals("nu", model.pipeline().referenceName()); - Assertions.assertEquals("isfnbtqdrkwr", model.pipeline().name()); + Assertions.assertEquals("ft", model.pipeline().referenceName()); + Assertions.assertEquals("pzhox", model.pipeline().name()); Assertions.assertEquals(true, model.waitOnCompletion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExecutePipelineActivity model - = new ExecutePipelineActivity().withName("xnlaurviyntc").withDescription("pbzoqtf") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("elblb") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("iabfntrmkeawmfe") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fstizemakgzcmbg") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("yrwdmgrfhvew") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("kgmgqynejqk").withValue("datasxiczvfxoihc"), - new UserProperty().withName("xexbksaf").withValue("dataecwyrtluujyespcg"), - new UserProperty().withName("szwvooxieyyww").withValue("dataiwiaqrc"))) - .withPolicy( - new ExecutePipelineActivityPolicy().withSecureInput(false).withAdditionalProperties(mapOf())) - .withPipeline(new PipelineReference().withReferenceName("nu").withName("isfnbtqdrkwr")) - .withParameters(mapOf("fqne", "dataoidhbulvkisyh", "dddeahfgdjahnsm", "datap", "qdxjdolobt", "datatkh")) - .withWaitOnCompletion(true); + ExecutePipelineActivity model = new ExecutePipelineActivity().withName("yaxpukxt").withDescription("ejxwbr") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("xlkwyqoaejylqge") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, + DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("nngijnzlokxihf") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("vscfpwpqvg").withValue("datapwm"))) + .withPolicy(new ExecutePipelineActivityPolicy().withSecureInput(false).withAdditionalProperties(mapOf())) + .withPipeline(new PipelineReference().withReferenceName("ft").withName("pzhox")) + .withParameters(mapOf("v", "dataynouhjvtef", "paralr", "dataedfzxsq", "iqgqvprl", "datasfnjokrf", + "xdxnmuosoziqcui", "datasglqiuqsqzu")) + .withWaitOnCompletion(true); model = BinaryData.fromObject(model).toObject(ExecutePipelineActivity.class); - Assertions.assertEquals("xnlaurviyntc", model.name()); - Assertions.assertEquals("pbzoqtf", model.description()); + Assertions.assertEquals("yaxpukxt", model.name()); + Assertions.assertEquals("ejxwbr", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("elblb", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("kgmgqynejqk", model.userProperties().get(0).name()); + Assertions.assertEquals("xlkwyqoaejylqge", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("vscfpwpqvg", model.userProperties().get(0).name()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals("nu", model.pipeline().referenceName()); - Assertions.assertEquals("isfnbtqdrkwr", model.pipeline().name()); + Assertions.assertEquals("ft", model.pipeline().referenceName()); + Assertions.assertEquals("pzhox", model.pipeline().name()); Assertions.assertEquals(true, model.waitOnCompletion()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTypePropertiesTests.java index 8e71c450615c5..c97774db16344 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePipelineActivityTypePropertiesTests.java @@ -15,23 +15,23 @@ public final class ExecutePipelineActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecutePipelineActivityTypeProperties model = BinaryData.fromString( - "{\"pipeline\":{\"referenceName\":\"grdsmravxtgl\",\"name\":\"mdwdlbocecmnqcgb\"},\"parameters\":{\"xuzchegeo\":\"datapcwbyrkxzeb\",\"zzeu\":\"datadkcrch\",\"kicimyykmkelbqm\":\"dataadlpxirewhu\"},\"waitOnCompletion\":true}") + "{\"pipeline\":{\"referenceName\":\"wfkczldepz\",\"name\":\"znjwdcisceia\"},\"parameters\":{\"mbvccuikpavi\":\"databudnxawfgm\",\"jsuiou\":\"dataxjiz\",\"enqpthcljvqmbu\":\"datatltcrtmebrssrl\",\"yzvtfexcjqrvp\":\"dataensogdvhqqxggnc\"},\"waitOnCompletion\":true}") .toObject(ExecutePipelineActivityTypeProperties.class); - Assertions.assertEquals("grdsmravxtgl", model.pipeline().referenceName()); - Assertions.assertEquals("mdwdlbocecmnqcgb", model.pipeline().name()); + Assertions.assertEquals("wfkczldepz", model.pipeline().referenceName()); + Assertions.assertEquals("znjwdcisceia", model.pipeline().name()); Assertions.assertEquals(true, model.waitOnCompletion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ExecutePipelineActivityTypeProperties model = new ExecutePipelineActivityTypeProperties() - .withPipeline(new PipelineReference().withReferenceName("grdsmravxtgl").withName("mdwdlbocecmnqcgb")) - .withParameters( - mapOf("xuzchegeo", "datapcwbyrkxzeb", "zzeu", "datadkcrch", "kicimyykmkelbqm", "dataadlpxirewhu")) + .withPipeline(new PipelineReference().withReferenceName("wfkczldepz").withName("znjwdcisceia")) + .withParameters(mapOf("mbvccuikpavi", "databudnxawfgm", "jsuiou", "dataxjiz", "enqpthcljvqmbu", + "datatltcrtmebrssrl", "yzvtfexcjqrvp", "dataensogdvhqqxggnc")) .withWaitOnCompletion(true); model = BinaryData.fromObject(model).toObject(ExecutePipelineActivityTypeProperties.class); - Assertions.assertEquals("grdsmravxtgl", model.pipeline().referenceName()); - Assertions.assertEquals("mdwdlbocecmnqcgb", model.pipeline().name()); + Assertions.assertEquals("wfkczldepz", model.pipeline().referenceName()); + Assertions.assertEquals("znjwdcisceia", model.pipeline().name()); Assertions.assertEquals(true, model.waitOnCompletion()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePowerQueryActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePowerQueryActivityTypePropertiesTests.java index 7e7f36b727f8f..01749465ac609 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePowerQueryActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutePowerQueryActivityTypePropertiesTests.java @@ -24,247 +24,300 @@ public final class ExecutePowerQueryActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecutePowerQueryActivityTypeProperties model = BinaryData.fromString( - "{\"sinks\":{\"xurdfzynfm\":{\"script\":\"rtcjimsge\",\"schemaLinkedService\":{\"referenceName\":\"kwwedbc\",\"parameters\":{\"vqjw\":\"datapjnyrtlinijc\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"dpqln\",\"parameters\":{\"uzspkqxbt\":\"datanbkzqassnwvw\"}},\"name\":\"watwv\",\"description\":\"tudnns\",\"dataset\":{\"referenceName\":\"grqiq\",\"parameters\":{\"mwnrw\":\"databjxo\",\"lfrgv\":\"dataanvjhoshinljquqe\"}},\"linkedService\":{\"referenceName\":\"ulxhpvursmeumy\",\"parameters\":{\"kgsfmgy\":\"dataovypwvhjs\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"c\",\"datasetParameters\":\"datapqzmixw\",\"parameters\":{\"zjbuz\":\"datahjfseh\",\"bc\":\"datainzukzb\",\"gs\":\"datadhleuabsiqnyjjfj\",\"nxexafql\":\"dataup\"},\"\":{\"roltddifywxeua\":\"databzphlkrjsgb\",\"ackfupyivqpczx\":\"databijaqyiyefleju\",\"bperkeyhybc\":\"datazlxowgzt\"}}},\"eocvsrei\":{\"script\":\"qrnu\",\"schemaLinkedService\":{\"referenceName\":\"m\",\"parameters\":{\"zdobh\":\"dataeauifczcrutf\",\"ztfnjp\":\"datajjbx\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"xannjrxvthqjvo\",\"parameters\":{\"ppnzaluafxwa\":\"dataggqwlbxnyp\"}},\"name\":\"frsmxgvfmb\",\"description\":\"rollntvfqrjfzfhs\",\"dataset\":{\"referenceName\":\"sr\",\"parameters\":{\"xzc\":\"datatainplxkzc\",\"cq\":\"dataewpqpwefzlreonsq\",\"di\":\"datan\",\"qvmghz\":\"datagrapmftziracztls\"}},\"linkedService\":{\"referenceName\":\"t\",\"parameters\":{\"wnjo\":\"datatjgeynj\",\"fgjmlp\":\"dataipjqwxsxswineyje\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"rftnfgsmgsf\",\"datasetParameters\":\"datalpbgkefbfn\",\"parameters\":{\"sinsnikm\":\"dataemsfpmoiyk\",\"ebjzo\":\"datanzcenkseqlvqkk\"},\"\":{\"gifmlkvaiolfr\":\"datayshoeqpvkk\"}}}},\"queries\":[{\"queryName\":\"iajksmwrbwejhoc\",\"dataflowSinks\":[{\"script\":\"yq\",\"schemaLinkedService\":{\"referenceName\":\"kpoufupa\",\"parameters\":{\"dzvnyzhbtnagk\":\"databbzje\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"nenozs\",\"parameters\":{\"rtwrnuklshrqrh\":\"datamjschttl\",\"tbdx\":\"datach\",\"drfxqudyadxnrtk\":\"datatvp\"}},\"name\":\"t\",\"description\":\"un\",\"dataset\":{\"referenceName\":\"pv\",\"parameters\":{\"gbgvyz\":\"datajm\",\"dzwubuqxsnc\":\"datao\",\"jzwisxsasgfm\":\"datasrtqortxeuwb\"}},\"linkedService\":{\"referenceName\":\"xzbuhqmmadyi\",\"parameters\":{\"mf\":\"datahrdnlabo\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"wcgkikb\",\"datasetParameters\":\"dataqdopxbnrnn\",\"parameters\":{\"puywx\":\"datasoiiypvd\"},\"\":{\"qsjktqe\":\"datatlqszwcwa\",\"besgdlskwfiwvdq\":\"datamagoqfmks\"}}},{\"script\":\"qrzeotogo\",\"schemaLinkedService\":{\"referenceName\":\"tm\",\"parameters\":{\"mwrbruuwhhb\":\"datairoszxi\",\"ldmxm\":\"dataynfx\",\"vqgaamc\":\"datakh\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"besyh\",\"parameters\":{\"smvlqnksovv\":\"dataosuawd\",\"kpdkwvwxr\":\"datatlpzfjbzosytturz\",\"ygwwxentudpvsnll\":\"datamublfsagp\"}},\"name\":\"jbb\",\"description\":\"mmtlwrwsgyqwf\",\"dataset\":{\"referenceName\":\"pfkyttxgtcovp\",\"parameters\":{\"qdgpnpdmk\":\"datagz\",\"ascxmnbenanhzx\":\"datajg\"}},\"linkedService\":{\"referenceName\":\"xfwcqcxyjurk\",\"parameters\":{\"dngh\":\"datakzizlaha\",\"iuzxphhwnvt\":\"dataezmp\",\"elbabmddgq\":\"datahnwcq\",\"gtdawst\":\"datak\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ngfkrcshbdvqo\",\"datasetParameters\":\"databwenggxesxxpn\",\"parameters\":{\"so\":\"datahwkriv\",\"ypyfhezluhqcte\":\"dataexgykriwpxc\",\"cnqexl\":\"datamuxagcoygznmr\"},\"\":{\"aqjanpidvcogh\":\"datagobaqnhlktdzf\",\"rirxlvus\":\"datatjvsk\",\"zgrbub\":\"datahd\",\"dxxzoywk\":\"dataorkcplpuddn\"}}},{\"script\":\"zo\",\"schemaLinkedService\":{\"referenceName\":\"hqefza\",\"parameters\":{\"pfgrqmquzhr\":\"datacbstxunz\",\"xggspydmuldlz\":\"datanthceoujfazt\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"cuvxygloxf\",\"parameters\":{\"x\":\"datapgneggtyifmfi\",\"unooouq\":\"datadbayxdr\"}},\"name\":\"nvrbhqxewdcdnqh\",\"description\":\"ehpu\",\"dataset\":{\"referenceName\":\"jmluzvtptpj\",\"parameters\":{\"oxbghpgscrp\":\"datazqczigdwgq\",\"njs\":\"databawovawmrmwrzm\",\"nst\":\"datax\",\"imunmgtkyzupjn\":\"datavragr\"}},\"linkedService\":{\"referenceName\":\"jmpxshyxiez\",\"parameters\":{\"lfxf\":\"datadaxgwgbpblsas\",\"taeobr\":\"datazfybpwzgwhntkmu\",\"alpig\":\"databqbftfscbgarfbx\",\"rlmpincatsh\":\"datanshlu\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"xg\",\"datasetParameters\":\"databm\",\"parameters\":{\"vmclnsahpswspyif\":\"datadpjnwgtxp\",\"hutabhmck\":\"datasakaihwdybjgyxb\"},\"\":{\"vpobcxnrwaz\":\"dataadoxlleohvcy\",\"ajhllnk\":\"dataoyh\",\"nbkcweeakgtrwos\":\"dataquwo\"}}},{\"script\":\"okdtdrv\",\"schemaLinkedService\":{\"referenceName\":\"ui\",\"parameters\":{\"vii\":\"datapydywwjsqdchb\",\"chsarf\":\"datauyvllppljcaqeor\",\"obw\":\"dataydetmehdzeyhcft\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"fzlorinwtv\",\"parameters\":{\"eoutztlnhg\":\"datanywfyoimw\",\"locyiuiwkr\":\"datamtddkyyrpbnqi\"}},\"name\":\"pishcrxc\",\"description\":\"nvpiicnwtgyy\",\"dataset\":{\"referenceName\":\"k\",\"parameters\":{\"enzw\":\"datazxhotyhyj\"}},\"linkedService\":{\"referenceName\":\"hvslpythqg\",\"parameters\":{\"czzvfdhsmqygm\":\"datal\",\"bycrjeaxjgokvlix\":\"datafsnl\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"jykafii\",\"datasetParameters\":\"datakvtsmcncj\",\"parameters\":{\"ebioqxattthazqd\":\"datahcelwmlfetlmmd\"},\"\":{\"k\":\"datayz\",\"dolezbya\":\"dataww\",\"zikzt\":\"datazuoheuifshsbtpl\"}}}]},{\"queryName\":\"qkwg\",\"dataflowSinks\":[{\"script\":\"byapckfapzfqzvk\",\"schemaLinkedService\":{\"referenceName\":\"s\",\"parameters\":{\"tmmwbdkpuxda\":\"datarhekxmjkiqb\",\"wjmsogzc\":\"datahpfrexbg\",\"j\":\"dataz\",\"lrzvjvlnafpfou\":\"datawqiawjevdnpkdmq\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"aqba\",\"parameters\":{\"urxwtfpeuftpjldl\":\"datajwcdjxqxfvk\",\"mmbxkbtberyql\":\"dataforvsypjytgz\",\"gm\":\"datazebmikojpggwuwa\"}},\"name\":\"gmg\",\"description\":\"llqfbeuugir\",\"dataset\":{\"referenceName\":\"ezpsydkgtdwlvsf\",\"parameters\":{\"bzhdonyleisaw\":\"dataxdyhoozdcthhecmj\",\"onyrfmozuoop\":\"datadwmuytkkfo\",\"yzyiyvh\":\"datatud\",\"pb\":\"datadkbfqkea\"}},\"linkedService\":{\"referenceName\":\"nqcvdrpwcki\",\"parameters\":{\"tlbaxiaerpoasy\":\"datanu\",\"xoudmaniwkw\":\"datazebbrqnnrdb\",\"edp\":\"datamqymgiydg\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"owcwehjqyullep\",\"datasetParameters\":\"datadfrgobrs\",\"parameters\":{\"ri\":\"dataayy\"},\"\":{\"kxvrig\":\"datameflpimmwirixcz\",\"afpiejbpbdu\":\"datautxzascalwfefyg\",\"o\":\"dataaypsvedxphf\"}}}]},{\"queryName\":\"q\",\"dataflowSinks\":[{\"script\":\"kbe\",\"schemaLinkedService\":{\"referenceName\":\"mmitvviqs\",\"parameters\":{\"sqwjqevwt\":\"dataujhmdpey\",\"ngrr\":\"datarjqnciwybj\",\"ctkgllmpku\":\"datadtin\",\"ynvtimpgusroqk\":\"datablucxyhtkyq\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"kvojrkfcjtjq\",\"parameters\":{\"xbpxzducfz\":\"datawsmgkomqfvhkp\"}},\"name\":\"pgtbytibpgkidi\",\"description\":\"fputcihy\",\"dataset\":{\"referenceName\":\"pksjwaglhwnnfgy\",\"parameters\":{\"kmwvqtmfq\":\"dataou\",\"uugoujsvhezhe\":\"datazvfeljytshjjbo\",\"ayyshf\":\"datayhwo\"}},\"linkedService\":{\"referenceName\":\"nfyzjzey\",\"parameters\":{\"edxfc\":\"datadrnzeidbl\",\"u\":\"datakticwgdivqybvgce\",\"gsuhtlz\":\"datask\",\"elyfuliatbo\":\"datamsqebmfo\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"axeozgjtuhdgms\",\"datasetParameters\":\"datayqehbprrp\",\"parameters\":{\"zlzidu\":\"datantuveqmtlf\",\"ets\":\"datalxw\"},\"\":{\"qudx\":\"datat\",\"iwpaeumely\":\"datarbgqtjjiearyzzxk\",\"ukgmtrnwwwwlv\":\"datahphurzaz\",\"ize\":\"datarditghbaqumqlfno\"}}}]},{\"queryName\":\"yg\",\"dataflowSinks\":[{\"script\":\"yltqryaahlttomlp\",\"schemaLinkedService\":{\"referenceName\":\"jlk\",\"parameters\":{\"ptzefeovsgikvsn\":\"dataeuiw\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"kfsfgabdumhpbci\",\"parameters\":{\"wyltau\":\"dataypbv\",\"rfdyawetkr\":\"datagbelxmulyalupijq\",\"rjdceimludqx\":\"dataqitmcxqahxtnl\"}},\"name\":\"xqqbk\",\"description\":\"nskivoxrjmgm\",\"dataset\":{\"referenceName\":\"i\",\"parameters\":{\"eebcxeecgfs\":\"dataufhx\",\"yltnfwlt\":\"datadjipayybwxqr\",\"amgijevfjnvu\":\"dataukmdeqrp\",\"xtkvpejtdlqorcyp\":\"datakwjmtehpfn\"}},\"linkedService\":{\"referenceName\":\"wfalgzsg\",\"parameters\":{\"ducvhhayqx\":\"dataclzmjhiqgi\",\"ujenobf\":\"datacrsho\",\"vtzrg\":\"dataiscauudxf\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"brfqiekw\",\"datasetParameters\":\"datam\",\"parameters\":{\"oypgqoqvc\":\"datadgv\",\"bkapbgmjodf\":\"datadbypzcqlau\",\"juvjmnsgv\":\"datauhlipxkxhjtgv\",\"lvglwxepiwpi\":\"datayhomd\"},\"\":{\"ofscibrt\":\"dataxmplxz\",\"yzxzkpum\":\"datayjqur\",\"boqeteavphup\":\"datada\"}}}]}],\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"rp\",\"datasetParameters\":\"datavgihknnvjgccq\",\"parameters\":{\"bnrbtattzxvfs\":\"datawofhjonq\",\"bzjvzgyzenveiy\":\"dataufjfuvry\",\"fkbbchdypc\":\"datadngtylvdumpmx\",\"pteclfjauetzppc\":\"datakmrvgdjbl\"},\"\":{\"mmizhdxsybn\":\"datartgnvl\",\"vkcnplvrqkmpqsp\":\"datao\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"nxwtxtet\",\"parameters\":{\"rh\":\"datalzauumzw\",\"ngszvugqwxs\":\"datavkneozp\"}},\"folderPath\":\"datasgfxwyfeqajtzqu\"},\"integrationRuntime\":{\"referenceName\":\"rjwllg\",\"parameters\":{\"zjwugrjio\":\"dataoxkpjzycq\",\"fftv\":\"dataecuxgi\",\"ecs\":\"datalf\",\"sbqdjawul\":\"datafq\"}},\"compute\":{\"computeType\":\"datam\",\"coreCount\":\"datazpldhbapfr\"},\"traceLevel\":\"datawrmdmrhsybvn\",\"continueOnError\":\"dataxmipkatjyxhvjjvs\",\"runConcurrently\":\"datamdlysfroyn\",\"sourceStagingConcurrency\":\"dataoamg\"}") + "{\"sinks\":{\"tah\":{\"script\":\"uq\",\"schemaLinkedService\":{\"referenceName\":\"nvckpd\",\"parameters\":{\"yvgukauhnhdhssul\":\"dataiaengydkgicbki\",\"c\":\"datakgfvgxmnan\",\"coruyistjwlnt\":\"dataetrtvyvxbtpqjgb\",\"tu\":\"datapi\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"ncdebpel\",\"parameters\":{\"scztsatfu\":\"datagulymouwnnhbz\",\"hietzovbu\":\"datafqqjydrhwnnux\",\"xcpnxdzpfzmdsl\":\"datavekbknrr\"}},\"name\":\"gqbyoyhfbbb\",\"description\":\"l\",\"dataset\":{\"referenceName\":\"volqprhnchpet\",\"parameters\":{\"usizsnhekpc\":\"datafmttpzwnrmpuv\",\"iee\":\"dataqesvh\"}},\"linkedService\":{\"referenceName\":\"mubzlmmctdkzp\",\"parameters\":{\"xrsxvz\":\"dataagr\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ib\",\"datasetParameters\":\"dataesbjohrvkpnmaa\",\"parameters\":{\"grxooqqdlmbu\":\"datamai\",\"yfgjxmgwk\":\"datakayfjzycogwj\",\"svwauqxhqcvaib\":\"datarlnosqlf\"},\"\":{\"gtziyl\":\"datakoxljtvefbio\",\"lkzzlokmrudepzl\":\"dataxoaallveezesdn\",\"vgpvhtx\":\"datauzcwlbefjh\"}}},\"hpvursmeum\":{\"script\":\"jgfp\",\"schemaLinkedService\":{\"referenceName\":\"fmgudkfoybih\",\"parameters\":{\"wmddvfubr\":\"datakwtatkkufb\",\"rnggwujyukjfsb\":\"dataomff\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"gb\",\"parameters\":{\"gkbf\":\"dataihxwfmuwhgx\",\"avqqxtyhdik\":\"dataqsislaubij\",\"ryi\":\"dataratzgxta\",\"ovu\":\"datalfb\"}},\"name\":\"vhpichsbzgw\",\"description\":\"arbjjswzkzwtfeg\",\"dataset\":{\"referenceName\":\"hwtag\",\"parameters\":{\"uuljnang\":\"datauqudewjnzlq\",\"lwpa\":\"datacddwmnsapg\",\"tpqvhkjbgcqqeyt\":\"datartgrz\"}},\"linkedService\":{\"referenceName\":\"zoiqai\",\"parameters\":{\"qhayplakqglj\":\"datahel\",\"djbsfpaomlgy\":\"datarqatxzek\",\"kjjrjluqlcj\":\"dataazeftivpypsj\",\"cjimsgeslkwwedb\":\"datamr\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"e\",\"datasetParameters\":\"datanyrtlinijcxvqjw\",\"parameters\":{\"vnbk\":\"datapqlna\",\"kqx\":\"dataqassnwvwluzs\"},\"\":{\"jtudn\":\"datawatwv\",\"xokmwnrwuanvjhos\":\"dataswwgrqiqlopb\",\"oul\":\"datainljquqeplfrgv\"}}},\"grapmftziracztls\":{\"script\":\"ps\",\"schemaLinkedService\":{\"referenceName\":\"ypwvhjskkgsf\",\"parameters\":{\"lp\":\"datapqmc\",\"tfrhjfsehtzjbuzl\":\"datazmix\",\"kzbsbcddhl\":\"datanz\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"abs\",\"parameters\":{\"gs\":\"datayjjfj\",\"nxexafql\":\"dataup\"}},\"name\":\"jmbzph\",\"description\":\"rjsgbcro\",\"dataset\":{\"referenceName\":\"ddifywxe\",\"parameters\":{\"ackfupyivqpczx\":\"databijaqyiyefleju\",\"bperkeyhybc\":\"datazlxowgzt\"}},\"linkedService\":{\"referenceName\":\"urdfzynf\",\"parameters\":{\"gmntzeauifcz\":\"dataqrnu\",\"he\":\"datarutfvzdo\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"xtztfnjpwbx\",\"datasetParameters\":\"datan\",\"parameters\":{\"oydeggqwlbx\":\"datavthqj\",\"wazfrsmxgvfmb\":\"dataypkppnzaluaf\",\"ollntvfq\":\"datae\",\"plxkzcfx\":\"datajfzfhspdsraxztai\"},\"\":{\"cq\":\"dataewpqpwefzlreonsq\",\"di\":\"datan\"}}}},\"queries\":[{\"queryName\":\"ghznltjxstjge\",\"dataflowSinks\":[{\"script\":\"wnjo\",\"schemaLinkedService\":{\"referenceName\":\"pjqwxs\",\"parameters\":{\"lppnmrftnf\":\"dataineyjerfgj\",\"p\":\"datasmgsftk\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"kefbfnszeemsfpm\",\"parameters\":{\"nkseq\":\"dataktsinsnikmwnzc\",\"shoeqpv\":\"datavqkkcebjzocuj\"}},\"name\":\"kpgi\",\"description\":\"lkvaiolfrceoc\",\"dataset\":{\"referenceName\":\"reicpsv\",\"parameters\":{\"yq\":\"dataksmwrbwejhoccrl\",\"adtpbbzjevd\":\"datakkpoufu\",\"ozsflnmjschttl\":\"datavnyzhbtnagkndne\"}},\"linkedService\":{\"referenceName\":\"twrnuklshr\",\"parameters\":{\"vpadrfxqudyad\":\"datatchttbdxj\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"tkdtequnbrpvoswj\",\"datasetParameters\":\"datagbgvyz\",\"parameters\":{\"srtqortxeuwb\":\"datadzwubuqxsnc\",\"zxz\":\"datajzwisxsasgfm\"},\"\":{\"dyic\":\"dataqmm\",\"nlabogmfetq\":\"datathr\",\"ngtssoiiyp\":\"datacgkikbuaqdopxbnr\",\"xygztlqszwcwan\":\"datadbpuy\"}}},{\"script\":\"jktqeomagoqfm\",\"schemaLinkedService\":{\"referenceName\":\"lbesgdlskwfi\",\"parameters\":{\"o\":\"dataqmqqrzeo\",\"roszxiwmw\":\"dataooptmpcp\",\"xmskhdvqga\":\"databruuwhhbkynfxxld\",\"awdls\":\"datamcmfbesyhpzros\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"lqnksovvbtlp\",\"parameters\":{\"pdkwv\":\"databzosytturzh\",\"ygwwxentudpvsnll\":\"dataxrxmublfsagp\",\"pmmtlwrwsgyqwfp\":\"datajbb\"}},\"name\":\"pfkyttxgtcovp\",\"description\":\"pgzgq\",\"dataset\":{\"referenceName\":\"p\",\"parameters\":{\"ena\":\"datamkbjgtascxmn\"}},\"linkedService\":{\"referenceName\":\"zxzw\",\"parameters\":{\"lahaddng\":\"datacqcxyjurkfwokzi\",\"wn\":\"dataoezmpkiuzxph\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"hnwcq\",\"datasetParameters\":\"datalbabmddgqbkng\",\"parameters\":{\"shbdvqoivbwe\":\"datawstytengfkr\",\"rivjso\":\"dataggxesxxpnckbhw\",\"ypyfhezluhqcte\":\"dataexgykriwpxc\"},\"\":{\"zn\":\"dataxagcoy\",\"nh\":\"datarscnqexlvgpgoba\",\"maqj\":\"dataktdz\"}}},{\"script\":\"pidvcoghp\",\"schemaLinkedService\":{\"referenceName\":\"vskfrirxlvusghdv\",\"parameters\":{\"lpu\":\"databubaorkc\",\"oywknlzob\":\"datadnpdxx\",\"gi\":\"datahqefza\",\"nzqpfgrqmq\":\"databstx\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"h\",\"parameters\":{\"tjxggspydmul\":\"datathceoujfa\",\"vxy\":\"datalzrhc\",\"eggtyifmfi\":\"dataloxfnzjpg\"}},\"name\":\"x\",\"description\":\"bayxdrwunoo\",\"dataset\":{\"referenceName\":\"q\",\"parameters\":{\"hpuhljmluzvtpt\":\"datarbhqxewdcdnqhkn\",\"gdwgqkoxbghpg\":\"datajabszqcz\"}},\"linkedService\":{\"referenceName\":\"rpab\",\"parameters\":{\"x\":\"datavawmrmwrzmfnjs\",\"vragr\":\"datanst\",\"mjmpxshyxiezmp\":\"dataimunmgtkyzupjn\",\"lfxf\":\"datadaxgwgbpblsas\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ybpwzg\",\"datasetParameters\":\"datan\",\"parameters\":{\"eobr\":\"datautt\"},\"\":{\"b\":\"databftfs\",\"pigfnshlu\":\"dataarfbxza\",\"bt\":\"datarlmpincatsh\",\"udpjn\":\"datagtibmx\"}}},{\"script\":\"txpbv\",\"schemaLinkedService\":{\"referenceName\":\"lnsahpsw\",\"parameters\":{\"h\":\"dataifgsaka\",\"hmckzbuadoxlle\":\"datadybjgyxbwhuta\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"vcyyvpobcxnrwazi\",\"parameters\":{\"ll\":\"dataaaj\",\"cweeakgtrwosps\":\"datakwquwoxnb\",\"fihpydywwj\":\"datakdtdrvihu\",\"uviifuy\":\"dataqdch\"}},\"name\":\"llppljcaq\",\"description\":\"rvchsarff\",\"dataset\":{\"referenceName\":\"etmehdzeyh\",\"parameters\":{\"zl\":\"datazobwpn\",\"nywfyoimw\":\"datarinwtvsb\",\"mtddkyyrpbnqi\":\"dataeoutztlnhg\",\"pishcrxc\":\"datalocyiuiwkr\"}},\"linkedService\":{\"referenceName\":\"nvpiicnwtgyy\",\"parameters\":{\"wynhv\":\"dataujnzxhotyhyjben\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ythqgzip\",\"datasetParameters\":\"dataczzvfdhsmqygm\",\"parameters\":{\"vlixwebjykafiiz\":\"datanlobycrjeaxjgo\"},\"\":{\"lfetlmmdgeb\":\"datatsmcncjtovhcelw\",\"weryzgkcwwndole\":\"dataoqxattthazq\"}}}]},{\"queryName\":\"yaszuoheuifshs\",\"dataflowSinks\":[{\"script\":\"dzikzt\",\"schemaLinkedService\":{\"referenceName\":\"qkwg\",\"parameters\":{\"ap\":\"dataeb\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"fapzfqzvkorsrp\",\"parameters\":{\"wbdk\":\"dataekxmjkiqbqtm\",\"hpfrexbg\":\"datauxda\"}},\"name\":\"wjmsogzc\",\"description\":\"djtwq\",\"dataset\":{\"referenceName\":\"wjevdnpkd\",\"parameters\":{\"qaqbaevh\":\"datalrzvjvlnafpfou\",\"xwt\":\"datawcdjxqxfvklu\"}},\"linkedService\":{\"referenceName\":\"euftpjldljfo\",\"parameters\":{\"jyt\":\"datay\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"mmbxkbtberyql\",\"datasetParameters\":\"dataebmiko\",\"parameters\":{\"waegmlgmgcnll\":\"datagw\",\"qezpsydkgtdwlv\":\"datafbeuugir\",\"thhecmjgbzhd\":\"datafgqhxdyhoozd\"},\"\":{\"f\":\"dataleisawvdwmuytk\",\"opot\":\"datatonyrfmozu\",\"ai\":\"datadkyzyiyvhgdkbfqk\"}}}]},{\"queryName\":\"kenqcvdrp\",\"dataflowSinks\":[{\"script\":\"vtknu\",\"schemaLinkedService\":{\"referenceName\":\"lbax\",\"parameters\":{\"dbix\":\"datarpoasyzzebbrqnn\",\"giydgee\":\"dataudmaniwkwtmqy\",\"llepppdfrgobr\":\"datapivsowcwehjqy\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"xcayyvriuvmme\",\"parameters\":{\"czxkxvrig\":\"dataimmwiri\",\"afpiejbpbdu\":\"datautxzascalwfefyg\",\"o\":\"dataaypsvedxphf\",\"mmitvviqs\":\"dataqqwxjnkbes\"}},\"name\":\"qoujhmdpe\",\"description\":\"sqwjqevwt\",\"dataset\":{\"referenceName\":\"jqnciwy\",\"parameters\":{\"dtin\":\"datangrr\",\"blucxyhtkyq\":\"datactkgllmpku\",\"wkvojr\":\"dataynvtimpgusroqk\"}},\"linkedService\":{\"referenceName\":\"c\",\"parameters\":{\"k\":\"dataqhfkwsmgkomqfv\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"bpxzd\",\"datasetParameters\":\"datafzd\",\"parameters\":{\"utc\":\"databytibpgkidiujf\"},\"\":{\"ywcrouakmwvqtmfq\":\"dataxapksjwaglhwnnf\",\"uugoujsvhezhe\":\"datazvfeljytshjjbo\",\"ayyshf\":\"datayhwo\"}}},{\"script\":\"nfyzjzey\",\"schemaLinkedService\":{\"referenceName\":\"hdrnzeidblredxf\",\"parameters\":{\"gcebuts\":\"dataticwgdivqyb\",\"om\":\"datadgsuhtl\",\"osnlaxeozg\":\"dataqebmfopelyfuliat\",\"ptoentuve\":\"datatuhdgmshuyqehbpr\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"tlfbzlzi\",\"parameters\":{\"ets\":\"datalxw\",\"tjjiearyzzxk\":\"datahetrqudxzrbg\",\"hphurzaz\":\"dataiwpaeumely\",\"rditghbaqumqlfno\":\"dataukgmtrnwwwwlv\"}},\"name\":\"ize\",\"description\":\"yg\",\"dataset\":{\"referenceName\":\"ldy\",\"parameters\":{\"kjvoeuiwyptzefeo\":\"dataryaahlttomlpisj\",\"fsfgabdumhpbcix\":\"datasgikvsnfn\",\"n\":\"dataaypbvgwylta\"}},\"linkedService\":{\"referenceName\":\"elxmulyal\",\"parameters\":{\"yawetkrmqitmcx\":\"datajqbrf\",\"nlor\":\"dataahx\",\"qxjxqqbkfdnski\":\"datadceimlu\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"rjmgmscicqau\",\"datasetParameters\":\"dataxe\",\"parameters\":{\"ji\":\"datacxeecgfsl\",\"kukmdeqrpu\":\"dataayybwxqryyltnfwl\"},\"\":{\"nvuokwjmtehpfn\":\"dataijevf\",\"kwfalgzsgk\":\"dataxtkvpejtdlqorcyp\",\"ducvhhayqx\":\"datauclzmjhiqgi\",\"ujenobf\":\"datacrsho\"}}},{\"script\":\"scauudxfcvtzr\",\"schemaLinkedService\":{\"referenceName\":\"mxbrfqiek\",\"parameters\":{\"otdgvsoypgqoqv\":\"datam\",\"pbgmjodf\":\"datazdbypzcqlauhbk\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"hlipxkxhj\",\"parameters\":{\"homdplvglwx\":\"datasjuvjmnsgvf\",\"ydxmplxzrofscib\":\"datapiwpi\",\"ri\":\"datatxyjq\"}},\"name\":\"zxzkpumzda\",\"description\":\"oqeteavphup\",\"dataset\":{\"referenceName\":\"rp\",\"parameters\":{\"wofhjonq\":\"datagihknnvjgccqef\",\"ufjfuvry\":\"databnrbtattzxvfs\",\"dngtylvdumpmx\":\"databzjvzgyzenveiy\",\"kmrvgdjbl\":\"datafkbbchdypc\"}},\"linkedService\":{\"referenceName\":\"t\",\"parameters\":{\"tgnv\":\"datafjauetzppccfg\",\"gv\":\"datarmmizhdxsybnw\",\"plvrqkmpqs\":\"datac\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"inxwtxtetwqklz\",\"datasetParameters\":\"dataumzw\",\"parameters\":{\"plng\":\"datarvkneo\",\"yfeqajtzquhqrj\":\"datazvugqwxslisgfx\",\"izjwugr\":\"datallgrckoxkpjzyc\",\"ftvylfkecsdfq\":\"dataiopecuxgim\"},\"\":{\"wuldyjmjv\":\"dataqdj\",\"hs\":\"datapldhbapfrriwrmdm\",\"mipkatjyxhvjjvsv\":\"databvnqa\",\"loamgnpf\":\"datamdlysfroyn\"}}}]}],\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"v\",\"datasetParameters\":\"datasnrknikpgjuk\",\"parameters\":{\"ycl\":\"datayl\"},\"\":{\"shmfbzkfeh\":\"datap\",\"fnsuw\":\"dataslv\",\"xahsqorsa\":\"datauroddohngb\",\"lrnd\":\"datad\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"o\",\"parameters\":{\"hxujgyzfsswe\":\"datadmnvai\",\"rnhpxzjk\":\"datanzfdextd\",\"snhtdskenigo\":\"databvzpcec\"}},\"folderPath\":\"datafud\"},\"integrationRuntime\":{\"referenceName\":\"o\",\"parameters\":{\"cqg\":\"dataueqgrcnf\",\"zlmugxpuget\":\"datavlnv\"}},\"compute\":{\"computeType\":\"datalxdddvf\",\"coreCount\":\"dataazvavspjdxay\"},\"traceLevel\":\"datakdqimumaij\",\"continueOnError\":\"datallkyrs\",\"runConcurrently\":\"datayredzhnylir\",\"sourceStagingConcurrency\":\"dataxykplvjs\"}") .toObject(ExecutePowerQueryActivityTypeProperties.class); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("rp", model.dataFlow().referenceName()); - Assertions.assertEquals("nxwtxtet", model.staging().linkedService().referenceName()); - Assertions.assertEquals("rjwllg", model.integrationRuntime().referenceName()); - Assertions.assertEquals("watwv", model.sinks().get("xurdfzynfm").name()); - Assertions.assertEquals("tudnns", model.sinks().get("xurdfzynfm").description()); - Assertions.assertEquals("grqiq", model.sinks().get("xurdfzynfm").dataset().referenceName()); - Assertions.assertEquals("ulxhpvursmeumy", model.sinks().get("xurdfzynfm").linkedService().referenceName()); - Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, - model.sinks().get("xurdfzynfm").flowlet().type()); - Assertions.assertEquals("c", model.sinks().get("xurdfzynfm").flowlet().referenceName()); - Assertions.assertEquals("kwwedbc", model.sinks().get("xurdfzynfm").schemaLinkedService().referenceName()); - Assertions.assertEquals("dpqln", model.sinks().get("xurdfzynfm").rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("rtcjimsge", model.sinks().get("xurdfzynfm").script()); - Assertions.assertEquals("iajksmwrbwejhoc", model.queries().get(0).queryName()); - Assertions.assertEquals("t", model.queries().get(0).dataflowSinks().get(0).name()); - Assertions.assertEquals("un", model.queries().get(0).dataflowSinks().get(0).description()); - Assertions.assertEquals("pv", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); - Assertions.assertEquals("xzbuhqmmadyi", + Assertions.assertEquals("v", model.dataFlow().referenceName()); + Assertions.assertEquals("o", model.staging().linkedService().referenceName()); + Assertions.assertEquals("o", model.integrationRuntime().referenceName()); + Assertions.assertEquals("gqbyoyhfbbb", model.sinks().get("tah").name()); + Assertions.assertEquals("l", model.sinks().get("tah").description()); + Assertions.assertEquals("volqprhnchpet", model.sinks().get("tah").dataset().referenceName()); + Assertions.assertEquals("mubzlmmctdkzp", model.sinks().get("tah").linkedService().referenceName()); + Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.sinks().get("tah").flowlet().type()); + Assertions.assertEquals("ib", model.sinks().get("tah").flowlet().referenceName()); + Assertions.assertEquals("nvckpd", model.sinks().get("tah").schemaLinkedService().referenceName()); + Assertions.assertEquals("ncdebpel", model.sinks().get("tah").rejectedDataLinkedService().referenceName()); + Assertions.assertEquals("uq", model.sinks().get("tah").script()); + Assertions.assertEquals("ghznltjxstjge", model.queries().get(0).queryName()); + Assertions.assertEquals("kpgi", model.queries().get(0).dataflowSinks().get(0).name()); + Assertions.assertEquals("lkvaiolfrceoc", model.queries().get(0).dataflowSinks().get(0).description()); + Assertions.assertEquals("reicpsv", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); + Assertions.assertEquals("twrnuklshr", model.queries().get(0).dataflowSinks().get(0).linkedService().referenceName()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.queries().get(0).dataflowSinks().get(0).flowlet().type()); - Assertions.assertEquals("wcgkikb", model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); - Assertions.assertEquals("kpoufupa", + Assertions.assertEquals("tkdtequnbrpvoswj", + model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); + Assertions.assertEquals("pjqwxs", model.queries().get(0).dataflowSinks().get(0).schemaLinkedService().referenceName()); - Assertions.assertEquals("nenozs", + Assertions.assertEquals("kefbfnszeemsfpm", model.queries().get(0).dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("yq", model.queries().get(0).dataflowSinks().get(0).script()); + Assertions.assertEquals("wnjo", model.queries().get(0).dataflowSinks().get(0).script()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ExecutePowerQueryActivityTypeProperties model = new ExecutePowerQueryActivityTypeProperties() .withDataFlow(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("rp").withDatasetParameters("datavgihknnvjgccq") - .withParameters(mapOf("bnrbtattzxvfs", "datawofhjonq", "bzjvzgyzenveiy", "dataufjfuvry", "fkbbchdypc", - "datadngtylvdumpmx", "pteclfjauetzppc", "datakmrvgdjbl")) + .withReferenceName("v").withDatasetParameters("datasnrknikpgjuk").withParameters(mapOf("ycl", "datayl")) .withAdditionalProperties(mapOf())) .withStaging(new DataFlowStagingInfo() - .withLinkedService(new LinkedServiceReference().withReferenceName("nxwtxtet") - .withParameters(mapOf("rh", "datalzauumzw", "ngszvugqwxs", "datavkneozp"))) - .withFolderPath("datasgfxwyfeqajtzqu")) - .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("rjwllg").withParameters( - mapOf("zjwugrjio", "dataoxkpjzycq", "fftv", "dataecuxgi", "ecs", "datalf", "sbqdjawul", "datafq"))) - .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datam") - .withCoreCount("datazpldhbapfr")) - .withTraceLevel("datawrmdmrhsybvn").withContinueOnError("dataxmipkatjyxhvjjvs") - .withRunConcurrently("datamdlysfroyn").withSourceStagingConcurrency("dataoamg") - .withSinks( - mapOf("xurdfzynfm", - new PowerQuerySink().withName("watwv").withDescription("tudnns") - .withDataset(new DatasetReference().withReferenceName("grqiq") - .withParameters(mapOf("mwnrw", "databjxo", "lfrgv", "dataanvjhoshinljquqe"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("ulxhpvursmeumy") - .withParameters(mapOf("kgsfmgy", "dataovypwvhjs"))) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("c").withDatasetParameters("datapqzmixw") - .withParameters(mapOf("zjbuz", "datahjfseh", "bc", "datainzukzb", "gs", - "datadhleuabsiqnyjjfj", "nxexafql", "dataup")) - .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("kwwedbc") - .withParameters(mapOf("vqjw", "datapjnyrtlinijc"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("dpqln") - .withParameters(mapOf("uzspkqxbt", "datanbkzqassnwvw"))) - .withScript("rtcjimsge"), - "eocvsrei", - new PowerQuerySink().withName("frsmxgvfmb").withDescription("rollntvfqrjfzfhs") - .withDataset(new DatasetReference().withReferenceName("sr") - .withParameters(mapOf("xzc", "datatainplxkzc", "cq", "dataewpqpwefzlreonsq", "di", "datan", - "qvmghz", "datagrapmftziracztls"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("t") - .withParameters(mapOf("wnjo", "datatjgeynj", "fgjmlp", "dataipjqwxsxswineyje"))) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("rftnfgsmgsf").withDatasetParameters("datalpbgkefbfn") - .withParameters(mapOf("sinsnikm", "dataemsfpmoiyk", "ebjzo", "datanzcenkseqlvqkk")) - .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("m") - .withParameters(mapOf("zdobh", "dataeauifczcrutf", "ztfnjp", "datajjbx"))) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("xannjrxvthqjvo") - .withParameters(mapOf("ppnzaluafxwa", "dataggqwlbxnyp"))) - .withScript("qrnu"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("o").withParameters( + mapOf("hxujgyzfsswe", "datadmnvai", "rnhpxzjk", "datanzfdextd", "snhtdskenigo", "databvzpcec"))) + .withFolderPath("datafud")) + .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("o") + .withParameters(mapOf("cqg", "dataueqgrcnf", "zlmugxpuget", "datavlnv"))) + .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datalxdddvf") + .withCoreCount("dataazvavspjdxay")) + .withTraceLevel("datakdqimumaij").withContinueOnError("datallkyrs").withRunConcurrently("datayredzhnylir") + .withSourceStagingConcurrency("dataxykplvjs") + .withSinks(mapOf("tah", + new PowerQuerySink().withName("gqbyoyhfbbb").withDescription("l") + .withDataset(new DatasetReference().withReferenceName("volqprhnchpet") + .withParameters(mapOf("usizsnhekpc", "datafmttpzwnrmpuv", "iee", "dataqesvh"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("mubzlmmctdkzp") + .withParameters(mapOf("xrsxvz", "dataagr"))) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("ib").withDatasetParameters("dataesbjohrvkpnmaa") + .withParameters(mapOf("grxooqqdlmbu", "datamai", "yfgjxmgwk", "datakayfjzycogwj", + "svwauqxhqcvaib", "datarlnosqlf")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("nvckpd") + .withParameters(mapOf("yvgukauhnhdhssul", "dataiaengydkgicbki", "c", "datakgfvgxmnan", + "coruyistjwlnt", "dataetrtvyvxbtpqjgb", "tu", "datapi"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("ncdebpel") + .withParameters(mapOf("scztsatfu", "datagulymouwnnhbz", "hietzovbu", "datafqqjydrhwnnux", + "xcpnxdzpfzmdsl", "datavekbknrr"))) + .withScript("uq"), + "hpvursmeum", + new PowerQuerySink().withName("vhpichsbzgw").withDescription("arbjjswzkzwtfeg") + .withDataset(new DatasetReference().withReferenceName("hwtag").withParameters( + mapOf("uuljnang", "datauqudewjnzlq", "lwpa", "datacddwmnsapg", "tpqvhkjbgcqqeyt", "datartgrz"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("zoiqai") + .withParameters(mapOf("qhayplakqglj", "datahel", "djbsfpaomlgy", "datarqatxzek", "kjjrjluqlcj", + "dataazeftivpypsj", "cjimsgeslkwwedb", "datamr"))) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("e").withDatasetParameters("datanyrtlinijcxvqjw") + .withParameters(mapOf("vnbk", "datapqlna", "kqx", "dataqassnwvwluzs")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("fmgudkfoybih") + .withParameters(mapOf("wmddvfubr", "datakwtatkkufb", "rnggwujyukjfsb", "dataomff"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("gb") + .withParameters(mapOf("gkbf", "dataihxwfmuwhgx", "avqqxtyhdik", "dataqsislaubij", "ryi", + "dataratzgxta", "ovu", "datalfb"))) + .withScript("jgfp"), + "grapmftziracztls", + new PowerQuerySink().withName("jmbzph").withDescription("rjsgbcro") + .withDataset(new DatasetReference().withReferenceName("ddifywxe") + .withParameters(mapOf("ackfupyivqpczx", "databijaqyiyefleju", "bperkeyhybc", "datazlxowgzt"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("urdfzynf") + .withParameters(mapOf("gmntzeauifcz", "dataqrnu", "he", "datarutfvzdo"))) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("xtztfnjpwbx").withDatasetParameters("datan") + .withParameters(mapOf("oydeggqwlbx", "datavthqj", "wazfrsmxgvfmb", "dataypkppnzaluaf", + "ollntvfq", "datae", "plxkzcfx", "datajfzfhspdsraxztai")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService( + new LinkedServiceReference().withReferenceName("ypwvhjskkgsf").withParameters( + mapOf("lp", "datapqmc", "tfrhjfsehtzjbuzl", "datazmix", "kzbsbcddhl", "datanz"))) + .withRejectedDataLinkedService( + new LinkedServiceReference().withReferenceName( + "abs").withParameters( + mapOf("gs", "datayjjfj", "nxexafql", "dataup"))) + .withScript("ps"))) .withQueries( - Arrays.asList( - new PowerQuerySinkMapping().withQueryName("iajksmwrbwejhoc") - .withDataflowSinks(Arrays.asList( - new PowerQuerySink().withName("t").withDescription("un") - .withDataset(new DatasetReference().withReferenceName("pv") - .withParameters(mapOf("gbgvyz", "datajm", "dzwubuqxsnc", "datao", "jzwisxsasgfm", - "datasrtqortxeuwb"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("xzbuhqmmadyi") - .withParameters(mapOf("mf", "datahrdnlabo"))) + Arrays + .asList( + new PowerQuerySinkMapping().withQueryName("ghznltjxstjge") + .withDataflowSinks(Arrays.asList( + new PowerQuerySink().withName("kpgi").withDescription("lkvaiolfrceoc") + .withDataset(new DatasetReference().withReferenceName("reicpsv") + .withParameters(mapOf("yq", "dataksmwrbwejhoccrl", "adtpbbzjevd", "datakkpoufu", + "ozsflnmjschttl", "datavnyzhbtnagkndne"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("twrnuklshr") + .withParameters(mapOf("vpadrfxqudyad", "datatchttbdxj"))) + .withFlowlet( + new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("tkdtequnbrpvoswj").withDatasetParameters("datagbgvyz") + .withParameters( + mapOf("srtqortxeuwb", "datadzwubuqxsnc", "zxz", "datajzwisxsasgfm")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("pjqwxs") + .withParameters(mapOf("lppnmrftnf", "dataineyjerfgj", "p", "datasmgsftk"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("kefbfnszeemsfpm").withParameters( + mapOf("nkseq", "dataktsinsnikmwnzc", "shoeqpv", "datavqkkcebjzocuj"))) + .withScript("wnjo"), + new PowerQuerySink().withName("pfkyttxgtcovp").withDescription("pgzgq") + .withDataset(new DatasetReference().withReferenceName("p") + .withParameters(mapOf("ena", "datamkbjgtascxmn"))) + .withLinkedService( + new LinkedServiceReference().withReferenceName("zxzw").withParameters( + mapOf("lahaddng", "datacqcxyjurkfwokzi", "wn", "dataoezmpkiuzxph"))) + .withFlowlet( + new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("hnwcq").withDatasetParameters("datalbabmddgqbkng") + .withParameters(mapOf("shbdvqoivbwe", "datawstytengfkr", "rivjso", + "dataggxesxxpnckbhw", "ypyfhezluhqcte", "dataexgykriwpxc")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService( + new LinkedServiceReference().withReferenceName("lbesgdlskwfi") + .withParameters(mapOf("o", "dataqmqqrzeo", "roszxiwmw", "dataooptmpcp", + "xmskhdvqga", "databruuwhhbkynfxxld", "awdls", "datamcmfbesyhpzros"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("lqnksovvbtlp") + .withParameters(mapOf("pdkwv", "databzosytturzh", "ygwwxentudpvsnll", + "dataxrxmublfsagp", "pmmtlwrwsgyqwfp", "datajbb"))) + .withScript("jktqeomagoqfm"), + new PowerQuerySink().withName("x").withDescription("bayxdrwunoo") + .withDataset(new DatasetReference().withReferenceName("q") + .withParameters(mapOf("hpuhljmluzvtpt", "datarbhqxewdcdnqhkn", "gdwgqkoxbghpg", + "datajabszqcz"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("rpab") + .withParameters(mapOf("x", "datavawmrmwrzmfnjs", "vragr", "datanst", + "mjmpxshyxiezmp", "dataimunmgtkyzupjn", "lfxf", "datadaxgwgbpblsas"))) + .withFlowlet( + new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("ybpwzg").withDatasetParameters("datan") + .withParameters(mapOf("eobr", "datautt")).withAdditionalProperties(mapOf())) + .withSchemaLinkedService( + new LinkedServiceReference().withReferenceName("vskfrirxlvusghdv") + .withParameters(mapOf("lpu", "databubaorkc", "oywknlzob", "datadnpdxx", + "gi", "datahqefza", "nzqpfgrqmq", "databstx"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("h") + .withParameters(mapOf("tjxggspydmul", "datathceoujfa", "vxy", "datalzrhc", + "eggtyifmfi", "dataloxfnzjpg"))) + .withScript("pidvcoghp"), + new PowerQuerySink().withName("llppljcaq").withDescription("rvchsarff") + .withDataset(new DatasetReference().withReferenceName("etmehdzeyh") + .withParameters(mapOf("zl", "datazobwpn", "nywfyoimw", "datarinwtvsb", + "mtddkyyrpbnqi", "dataeoutztlnhg", "pishcrxc", "datalocyiuiwkr"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("nvpiicnwtgyy") + .withParameters(mapOf("wynhv", "dataujnzxhotyhyjben"))) + .withFlowlet( + new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("ythqgzip").withDatasetParameters("dataczzvfdhsmqygm") + .withParameters(mapOf("vlixwebjykafiiz", "datanlobycrjeaxjgo")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService( + new LinkedServiceReference().withReferenceName("lnsahpsw").withParameters( + mapOf("h", "dataifgsaka", "hmckzbuadoxlle", "datadybjgyxbwhuta"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("vcyyvpobcxnrwazi") + .withParameters( + mapOf("ll", "dataaaj", "cweeakgtrwosps", "datakwquwoxnb", "fihpydywwj", + "datakdtdrvihu", "uviifuy", "dataqdch"))) + .withScript("txpbv"))), + new PowerQuerySinkMapping().withQueryName("yaszuoheuifshs") + .withDataflowSinks(Arrays.asList(new PowerQuerySink().withName("wjmsogzc") + .withDescription("djtwq") + .withDataset(new DatasetReference().withReferenceName("wjevdnpkd") + .withParameters(mapOf("qaqbaevh", "datalrzvjvlnafpfou", "xwt", "datawcdjxqxfvklu"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("euftpjldljfo") + .withParameters(mapOf("jyt", "datay"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("wcgkikb").withDatasetParameters("dataqdopxbnrnn") - .withParameters(mapOf("puywx", "datasoiiypvd")).withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("kpoufupa") - .withParameters(mapOf("dzvnyzhbtnagk", "databbzje"))) - .withRejectedDataLinkedService(new LinkedServiceReference() - .withReferenceName("nenozs") - .withParameters(mapOf("rtwrnuklshrqrh", "datamjschttl", "tbdx", "datach", - "drfxqudyadxnrtk", "datatvp"))) - .withScript("yq"), - new PowerQuerySink().withName("jbb").withDescription("mmtlwrwsgyqwf") - .withDataset(new DatasetReference().withReferenceName("pfkyttxgtcovp") - .withParameters(mapOf("qdgpnpdmk", "datagz", "ascxmnbenanhzx", "datajg"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("xfwcqcxyjurk") - .withParameters(mapOf("dngh", "datakzizlaha", "iuzxphhwnvt", "dataezmp", - "elbabmddgq", "datahnwcq", "gtdawst", "datak"))) + .withReferenceName("mmbxkbtberyql").withDatasetParameters("dataebmiko") + .withParameters(mapOf("waegmlgmgcnll", "datagw", "qezpsydkgtdwlv", "datafbeuugir", + "thhecmjgbzhd", "datafgqhxdyhoozd")) + .withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("qkwg") + .withParameters(mapOf("ap", "dataeb"))) + .withRejectedDataLinkedService( + new LinkedServiceReference().withReferenceName("fapzfqzvkorsrp") + .withParameters(mapOf("wbdk", "dataekxmjkiqbqtm", "hpfrexbg", "datauxda"))) + .withScript("dzikzt"))), + new PowerQuerySinkMapping().withQueryName("kenqcvdrp").withDataflowSinks(Arrays.asList( + new PowerQuerySink().withName("qoujhmdpe").withDescription("sqwjqevwt") + .withDataset(new DatasetReference().withReferenceName("jqnciwy") + .withParameters(mapOf("dtin", "datangrr", "blucxyhtkyq", "datactkgllmpku", "wkvojr", + "dataynvtimpgusroqk"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("c") + .withParameters(mapOf("k", "dataqhfkwsmgkomqfv"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("ngfkrcshbdvqo").withDatasetParameters("databwenggxesxxpn") - .withParameters(mapOf("so", "datahwkriv", "ypyfhezluhqcte", "dataexgykriwpxc", - "cnqexl", "datamuxagcoygznmr")) + .withReferenceName("bpxzd").withDatasetParameters("datafzd") + .withParameters(mapOf("utc", "databytibpgkidiujf")) .withAdditionalProperties(mapOf())) - .withSchemaLinkedService( - new LinkedServiceReference().withReferenceName("tm").withParameters( - mapOf("mwrbruuwhhb", "datairoszxi", "ldmxm", "dataynfx", "vqgaamc", "datakh"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("besyh") - .withParameters(mapOf("smvlqnksovv", "dataosuawd", "kpdkwvwxr", - "datatlpzfjbzosytturz", "ygwwxentudpvsnll", "datamublfsagp"))) - .withScript("qrzeotogo"), - new PowerQuerySink().withName("nvrbhqxewdcdnqh").withDescription("ehpu") - .withDataset(new DatasetReference().withReferenceName("jmluzvtptpj") - .withParameters(mapOf("oxbghpgscrp", "datazqczigdwgq", "njs", "databawovawmrmwrzm", - "nst", "datax", "imunmgtkyzupjn", "datavragr"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("jmpxshyxiez") - .withParameters(mapOf("lfxf", "datadaxgwgbpblsas", "taeobr", "datazfybpwzgwhntkmu", - "alpig", "databqbftfscbgarfbx", "rlmpincatsh", "datanshlu"))) + .withSchemaLinkedService(new LinkedServiceReference() + .withReferenceName("lbax") + .withParameters(mapOf("dbix", "datarpoasyzzebbrqnn", "giydgee", "dataudmaniwkwtmqy", + "llepppdfrgobr", "datapivsowcwehjqy"))) + .withRejectedDataLinkedService( + new LinkedServiceReference().withReferenceName("xcayyvriuvmme").withParameters( + mapOf( + "czxkxvrig", "dataimmwiri", "afpiejbpbdu", "datautxzascalwfefyg", "o", + "dataaypsvedxphf", "mmitvviqs", "dataqqwxjnkbes"))) + .withScript("vtknu"), + new PowerQuerySink().withName("ize").withDescription("yg") + .withDataset(new DatasetReference().withReferenceName("ldy") + .withParameters(mapOf("kjvoeuiwyptzefeo", "dataryaahlttomlpisj", "fsfgabdumhpbcix", + "datasgikvsnfn", "n", "dataaypbvgwylta"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("elxmulyal") + .withParameters(mapOf("yawetkrmqitmcx", "datajqbrf", "nlor", "dataahx", + "qxjxqqbkfdnski", "datadceimlu"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("xg").withDatasetParameters("databm") - .withParameters( - mapOf("vmclnsahpswspyif", "datadpjnwgtxp", "hutabhmck", "datasakaihwdybjgyxb")) + .withReferenceName("rjmgmscicqau").withDatasetParameters("dataxe") + .withParameters(mapOf("ji", "datacxeecgfsl", "kukmdeqrpu", "dataayybwxqryyltnfwl")) .withAdditionalProperties(mapOf())) .withSchemaLinkedService( - new LinkedServiceReference().withReferenceName("hqefza").withParameters( - mapOf("pfgrqmquzhr", "datacbstxunz", "xggspydmuldlz", "datanthceoujfazt"))) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("cuvxygloxf") - .withParameters(mapOf("x", "datapgneggtyifmfi", "unooouq", "datadbayxdr"))) - .withScript("zo"), - new PowerQuerySink().withName("pishcrxc").withDescription("nvpiicnwtgyy") - .withDataset(new DatasetReference().withReferenceName("k") - .withParameters(mapOf("enzw", "datazxhotyhyj"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("hvslpythqg") - .withParameters(mapOf("czzvfdhsmqygm", "datal", "bycrjeaxjgokvlix", "datafsnl"))) + new LinkedServiceReference().withReferenceName("hdrnzeidblredxf") + .withParameters(mapOf("gcebuts", "dataticwgdivqyb", "om", "datadgsuhtl", + "osnlaxeozg", "dataqebmfopelyfuliat", "ptoentuve", "datatuhdgmshuyqehbpr"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("tlfbzlzi") + .withParameters( + mapOf( + "ets", "datalxw", "tjjiearyzzxk", "datahetrqudxzrbg", "hphurzaz", + "dataiwpaeumely", "rditghbaqumqlfno", "dataukgmtrnwwwwlv"))) + .withScript("nfyzjzey"), + new PowerQuerySink().withName("zxzkpumzda").withDescription("oqeteavphup") + .withDataset(new DatasetReference().withReferenceName("rp").withParameters( + mapOf("wofhjonq", "datagihknnvjgccqef", "ufjfuvry", "databnrbtattzxvfs", + "dngtylvdumpmx", "databzjvzgyzenveiy", "kmrvgdjbl", "datafkbbchdypc"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("t") + .withParameters(mapOf("tgnv", "datafjauetzppccfg", "gv", "datarmmizhdxsybnw", + "plvrqkmpqs", "datac"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("jykafii").withDatasetParameters("datakvtsmcncj") - .withParameters(mapOf("ebioqxattthazqd", "datahcelwmlfetlmmd")) + .withReferenceName("inxwtxtetwqklz").withDatasetParameters("dataumzw") + .withParameters(mapOf("plng", "datarvkneo", "yfeqajtzquhqrj", "datazvugqwxslisgfx", + "izjwugr", "datallgrckoxkpjzyc", "ftvylfkecsdfq", "dataiopecuxgim")) .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("ui") - .withParameters(mapOf("vii", "datapydywwjsqdchb", "chsarf", "datauyvllppljcaqeor", - "obw", "dataydetmehdzeyhcft"))) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("fzlorinwtv").withParameters( - mapOf("eoutztlnhg", "datanywfyoimw", "locyiuiwkr", "datamtddkyyrpbnqi"))) - .withScript("okdtdrv"))), - new PowerQuerySinkMapping().withQueryName("qkwg") - .withDataflowSinks(Arrays.asList(new PowerQuerySink().withName("gmg") - .withDescription("llqfbeuugir") - .withDataset(new DatasetReference().withReferenceName("ezpsydkgtdwlvsf") - .withParameters(mapOf("bzhdonyleisaw", "dataxdyhoozdcthhecmj", "onyrfmozuoop", - "datadwmuytkkfo", "yzyiyvh", "datatud", "pb", "datadkbfqkea"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("nqcvdrpwcki") - .withParameters(mapOf("tlbaxiaerpoasy", "datanu", "xoudmaniwkw", "datazebbrqnnrdb", - "edp", "datamqymgiydg"))) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("owcwehjqyullep").withDatasetParameters("datadfrgobrs") - .withParameters(mapOf("ri", "dataayy")).withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("s") - .withParameters(mapOf("tmmwbdkpuxda", "datarhekxmjkiqb", "wjmsogzc", "datahpfrexbg", - "j", "dataz", "lrzvjvlnafpfou", "datawqiawjevdnpkdmq"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("aqba") - .withParameters(mapOf("urxwtfpeuftpjldl", "datajwcdjxqxfvk", "mmbxkbtberyql", - "dataforvsypjytgz", "gm", "datazebmikojpggwuwa"))) - .withScript("byapckfapzfqzvk"))), - new PowerQuerySinkMapping().withQueryName("q").withDataflowSinks(Arrays.asList(new PowerQuerySink() - .withName("pgtbytibpgkidi").withDescription("fputcihy") - .withDataset(new DatasetReference().withReferenceName("pksjwaglhwnnfgy").withParameters( - mapOf("kmwvqtmfq", "dataou", "uugoujsvhezhe", "datazvfeljytshjjbo", "ayyshf", "datayhwo"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("nfyzjzey") - .withParameters(mapOf("edxfc", "datadrnzeidbl", "u", "datakticwgdivqybvgce", "gsuhtlz", - "datask", "elyfuliatbo", "datamsqebmfo"))) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("axeozgjtuhdgms").withDatasetParameters("datayqehbprrp") - .withParameters(mapOf("zlzidu", "datantuveqmtlf", "ets", "datalxw")) - .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("mmitvviqs") - .withParameters(mapOf("sqwjqevwt", "dataujhmdpey", "ngrr", "datarjqnciwybj", "ctkgllmpku", - "datadtin", "ynvtimpgusroqk", "datablucxyhtkyq"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("kvojrkfcjtjq") - .withParameters(mapOf("xbpxzducfz", "datawsmgkomqfvhkp"))) - .withScript("kbe"))), - new PowerQuerySinkMapping().withQueryName("yg").withDataflowSinks(Arrays.asList(new PowerQuerySink() - .withName("xqqbk").withDescription("nskivoxrjmgm") - .withDataset(new DatasetReference().withReferenceName("i") - .withParameters(mapOf("eebcxeecgfs", "dataufhx", "yltnfwlt", "datadjipayybwxqr", - "amgijevfjnvu", "dataukmdeqrp", "xtkvpejtdlqorcyp", "datakwjmtehpfn"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("wfalgzsg").withParameters( - mapOf("ducvhhayqx", "dataclzmjhiqgi", "ujenobf", "datacrsho", "vtzrg", "dataiscauudxf"))) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("brfqiekw").withDatasetParameters("datam") - .withParameters(mapOf("oypgqoqvc", "datadgv", "bkapbgmjodf", "datadbypzcqlau", "juvjmnsgv", - "datauhlipxkxhjtgv", "lvglwxepiwpi", "datayhomd")) - .withAdditionalProperties(mapOf())) - .withSchemaLinkedService( - new LinkedServiceReference().withReferenceName("jlk") - .withParameters(mapOf("ptzefeovsgikvsn", "dataeuiw"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("kfsfgabdumhpbci") - .withParameters(mapOf("wyltau", "dataypbv", "rfdyawetkr", "datagbelxmulyalupijq", - "rjdceimludqx", "dataqitmcxqahxtnl"))) - .withScript("yltqryaahlttomlp"))))); + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("mxbrfqiek") + .withParameters(mapOf("otdgvsoypgqoqv", "datam", "pbgmjodf", "datazdbypzcqlauhbk"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("hlipxkxhj").withParameters(mapOf("homdplvglwx", + "datasjuvjmnsgvf", "ydxmplxzrofscib", "datapiwpi", "ri", "datatxyjq"))) + .withScript("scauudxfcvtzr"))))); model = BinaryData.fromObject(model).toObject(ExecutePowerQueryActivityTypeProperties.class); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("rp", model.dataFlow().referenceName()); - Assertions.assertEquals("nxwtxtet", model.staging().linkedService().referenceName()); - Assertions.assertEquals("rjwllg", model.integrationRuntime().referenceName()); - Assertions.assertEquals("watwv", model.sinks().get("xurdfzynfm").name()); - Assertions.assertEquals("tudnns", model.sinks().get("xurdfzynfm").description()); - Assertions.assertEquals("grqiq", model.sinks().get("xurdfzynfm").dataset().referenceName()); - Assertions.assertEquals("ulxhpvursmeumy", model.sinks().get("xurdfzynfm").linkedService().referenceName()); - Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, - model.sinks().get("xurdfzynfm").flowlet().type()); - Assertions.assertEquals("c", model.sinks().get("xurdfzynfm").flowlet().referenceName()); - Assertions.assertEquals("kwwedbc", model.sinks().get("xurdfzynfm").schemaLinkedService().referenceName()); - Assertions.assertEquals("dpqln", model.sinks().get("xurdfzynfm").rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("rtcjimsge", model.sinks().get("xurdfzynfm").script()); - Assertions.assertEquals("iajksmwrbwejhoc", model.queries().get(0).queryName()); - Assertions.assertEquals("t", model.queries().get(0).dataflowSinks().get(0).name()); - Assertions.assertEquals("un", model.queries().get(0).dataflowSinks().get(0).description()); - Assertions.assertEquals("pv", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); - Assertions.assertEquals("xzbuhqmmadyi", + Assertions.assertEquals("v", model.dataFlow().referenceName()); + Assertions.assertEquals("o", model.staging().linkedService().referenceName()); + Assertions.assertEquals("o", model.integrationRuntime().referenceName()); + Assertions.assertEquals("gqbyoyhfbbb", model.sinks().get("tah").name()); + Assertions.assertEquals("l", model.sinks().get("tah").description()); + Assertions.assertEquals("volqprhnchpet", model.sinks().get("tah").dataset().referenceName()); + Assertions.assertEquals("mubzlmmctdkzp", model.sinks().get("tah").linkedService().referenceName()); + Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.sinks().get("tah").flowlet().type()); + Assertions.assertEquals("ib", model.sinks().get("tah").flowlet().referenceName()); + Assertions.assertEquals("nvckpd", model.sinks().get("tah").schemaLinkedService().referenceName()); + Assertions.assertEquals("ncdebpel", model.sinks().get("tah").rejectedDataLinkedService().referenceName()); + Assertions.assertEquals("uq", model.sinks().get("tah").script()); + Assertions.assertEquals("ghznltjxstjge", model.queries().get(0).queryName()); + Assertions.assertEquals("kpgi", model.queries().get(0).dataflowSinks().get(0).name()); + Assertions.assertEquals("lkvaiolfrceoc", model.queries().get(0).dataflowSinks().get(0).description()); + Assertions.assertEquals("reicpsv", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); + Assertions.assertEquals("twrnuklshr", model.queries().get(0).dataflowSinks().get(0).linkedService().referenceName()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.queries().get(0).dataflowSinks().get(0).flowlet().type()); - Assertions.assertEquals("wcgkikb", model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); - Assertions.assertEquals("kpoufupa", + Assertions.assertEquals("tkdtequnbrpvoswj", + model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); + Assertions.assertEquals("pjqwxs", model.queries().get(0).dataflowSinks().get(0).schemaLinkedService().referenceName()); - Assertions.assertEquals("nenozs", + Assertions.assertEquals("kefbfnszeemsfpm", model.queries().get(0).dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("yq", model.queries().get(0).dataflowSinks().get(0).script()); + Assertions.assertEquals("wnjo", model.queries().get(0).dataflowSinks().get(0).script()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteWranglingDataflowActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteWranglingDataflowActivityTests.java index 7f2ac121d6189..7eb0265fe0e3f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteWranglingDataflowActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecuteWranglingDataflowActivityTests.java @@ -30,283 +30,284 @@ public final class ExecuteWranglingDataflowActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecuteWranglingDataflowActivity model = BinaryData.fromString( - "{\"type\":\"ExecuteWranglingDataflow\",\"typeProperties\":{\"sinks\":{\"dlrr\":{\"script\":\"cbtlmnrdkiqsqbdv\",\"schemaLinkedService\":{\"referenceName\":\"qsmk\",\"parameters\":{\"mexrofqh\":\"dataljxnkpd\",\"gwov\":\"dataptsdlcsrhttmh\",\"krcwnlyqq\":\"dataduzqu\",\"q\":\"dataknul\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"cusmosjawb\",\"parameters\":{\"r\":\"dataizeuifnd\",\"fs\":\"datanzjyghq\",\"vgec\":\"dataln\",\"cgrkgt\":\"dataems\"}},\"name\":\"vxlaywkbuve\",\"description\":\"tezeyfdgnaoi\",\"dataset\":{\"referenceName\":\"ufdgtw\",\"parameters\":{\"hvg\":\"datas\",\"e\":\"datahfzdgsmee\"}},\"linkedService\":{\"referenceName\":\"rpvggxr\",\"parameters\":{\"egdxjxkxvgode\":\"datakdkjwo\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"faenul\",\"datasetParameters\":\"dataeqefqdvo\",\"parameters\":{\"knn\":\"datar\",\"cvrdwjghttpvom\":\"dataadjllhzl\"},\"\":{\"bifcyptlbad\":\"datasdbvudoieoheebze\"}}},\"nro\":{\"script\":\"vuewr\",\"schemaLinkedService\":{\"referenceName\":\"jmphfhmuaoou\",\"parameters\":{\"srzolhhlggob\":\"datafdg\",\"o\":\"datacfbtgelfkhmgsp\",\"i\":\"datarphzd\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"yhaqkglu\",\"parameters\":{\"tsa\":\"dataq\",\"v\":\"dataare\"}},\"name\":\"mjwmld\",\"description\":\"wglmmc\",\"dataset\":{\"referenceName\":\"yryy\",\"parameters\":{\"ueltcoiszqf\":\"datavenmvhbgpgvlii\",\"lzxjiruq\":\"databiyvowyyvsbjpy\",\"izrorupduwqovlq\":\"datarshtwdgoqxfbsci\"}},\"linkedService\":{\"referenceName\":\"wehagor\",\"parameters\":{\"gpmtzqpivochmexi\":\"dataotqnkovubfug\",\"mih\":\"datahmisvetuqibkj\",\"nuqnt\":\"datarulgypnaqwjsdwn\",\"tryn\":\"databeeziiqixfyg\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"oeuztpssmhdqcrig\",\"datasetParameters\":\"datatodpuqilpdzbyb\",\"parameters\":{\"ucvq\":\"dataxrcfzszpoumk\",\"ktmsphcrn\":\"datauuzftd\"},\"\":{\"sulejukackicrdr\":\"datatzdspykcreuopi\",\"riddcnljllypchqh\":\"datakexcyw\",\"peliktkimmpg\":\"datairsajtd\",\"vwzxqmves\":\"datazxicq\"}}},\"dt\":{\"script\":\"rqdgyttfzozrx\",\"schemaLinkedService\":{\"referenceName\":\"jjimfcgbdupslwl\",\"parameters\":{\"eqeabedfo\":\"dataezxcpxwqgmnq\",\"ahup\":\"datasiplhygp\",\"h\":\"datam\",\"pnrjsw\":\"datahrgeymspvgatzru\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"txrecwdleivmuq\",\"parameters\":{\"lpnyehhqy\":\"datarjv\",\"ka\":\"datajrmxazkqiqzaea\",\"xivhozhr\":\"datapokf\",\"azrbkhy\":\"databvfljxljgtirn\"}},\"name\":\"uf\",\"description\":\"qtvbyfyz\",\"dataset\":{\"referenceName\":\"uupeflk\",\"parameters\":{\"d\":\"dataxnja\",\"ydntupbrvdgtblx\":\"dataeayuowivpne\",\"ztlsnkwullvu\":\"datamdabpifygxuaidr\"}},\"linkedService\":{\"referenceName\":\"ymosj\",\"parameters\":{\"q\":\"datandjjqhins\",\"iwrfocbetlljqkgl\":\"dataudjrotqdiaxf\",\"xvl\":\"datavjaw\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"c\",\"datasetParameters\":\"dataseqmejerjyz\",\"parameters\":{\"ogykrmf\":\"datazbjieeivdrqtlcx\",\"w\":\"datalturxyvgro\",\"rxnlj\":\"datasnpcwymmgb\"},\"\":{\"dbhzcda\":\"datajsnzuebyzn\",\"eeqhjcwrrneor\":\"datanztzhqsbgksfjq\"}}}},\"queries\":[{\"queryName\":\"aumr\",\"dataflowSinks\":[{\"script\":\"ug\",\"schemaLinkedService\":{\"referenceName\":\"mqlb\"},\"rejectedDataLinkedService\":{\"referenceName\":\"pslvvggvnqparq\"},\"name\":\"ygdkovy\",\"description\":\"srboq\",\"dataset\":{\"referenceName\":\"qqqvu\"},\"linkedService\":{\"referenceName\":\"tcdppdmmfdubccc\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"gx\"}},{\"script\":\"tiixb\",\"schemaLinkedService\":{\"referenceName\":\"y\"},\"rejectedDataLinkedService\":{\"referenceName\":\"djcgdharlrfa\"},\"name\":\"dyswlmxe\",\"description\":\"gkf\",\"dataset\":{\"referenceName\":\"gkbujqtklzwmqzk\"},\"linkedService\":{\"referenceName\":\"czpzwfewbj\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"mhpue\"}},{\"script\":\"hhne\",\"schemaLinkedService\":{\"referenceName\":\"xx\"},\"rejectedDataLinkedService\":{\"referenceName\":\"fiali\"},\"name\":\"ymnukvfjbxvhui\",\"description\":\"ody\",\"dataset\":{\"referenceName\":\"otqp\"},\"linkedService\":{\"referenceName\":\"wrahqqumozulefp\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"hx\"}},{\"script\":\"bgw\",\"schemaLinkedService\":{\"referenceName\":\"ysmmz\"},\"rejectedDataLinkedService\":{\"referenceName\":\"kgxzi\"},\"name\":\"hnkmjgn\",\"description\":\"lnnxopixxciy\",\"dataset\":{\"referenceName\":\"mpgopclstysirh\"},\"linkedService\":{\"referenceName\":\"s\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"we\"}}]},{\"queryName\":\"x\",\"dataflowSinks\":[{\"script\":\"yddcqteo\",\"schemaLinkedService\":{\"referenceName\":\"yjim\"},\"rejectedDataLinkedService\":{\"referenceName\":\"bsjtmdwqo\"},\"name\":\"jkqohcfnomwaoe\",\"description\":\"jmmsgukoq\",\"dataset\":{\"referenceName\":\"rkerztenz\"},\"linkedService\":{\"referenceName\":\"ppgchztdzmeiicd\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"orsmd\"}},{\"script\":\"fdqqjdstz\",\"schemaLinkedService\":{\"referenceName\":\"hwbwr\"},\"rejectedDataLinkedService\":{\"referenceName\":\"uvqp\"},\"name\":\"agegzrt\",\"description\":\"sxhlje\",\"dataset\":{\"referenceName\":\"eaf\"},\"linkedService\":{\"referenceName\":\"xbvk\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"nmoekohxkgxyd\"}},{\"script\":\"opzetradbihnpw\",\"schemaLinkedService\":{\"referenceName\":\"gebosbz\"},\"rejectedDataLinkedService\":{\"referenceName\":\"btdh\"},\"name\":\"xuxwvbs\",\"description\":\"zhszeox\",\"dataset\":{\"referenceName\":\"rhakhaec\"},\"linkedService\":{\"referenceName\":\"jfpqemylkygcg\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"v\"}}]},{\"queryName\":\"vsizmtmctehxwnxp\",\"dataflowSinks\":[{\"script\":\"dablqollbvbohp\",\"schemaLinkedService\":{\"referenceName\":\"wsqavpu\"},\"rejectedDataLinkedService\":{\"referenceName\":\"j\"},\"name\":\"qcfzdfmqoeofy\",\"description\":\"ypztnnzy\",\"dataset\":{\"referenceName\":\"qatgazd\"},\"linkedService\":{\"referenceName\":\"ihuaebawapajdkjq\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"zraegyvxlnpvpcrc\"}},{\"script\":\"shjkovrqxoj\",\"schemaLinkedService\":{\"referenceName\":\"fcfivrbgpnkjuaxy\"},\"rejectedDataLinkedService\":{\"referenceName\":\"xetgsdhwmbewsyd\"},\"name\":\"rkchbnatrdr\",\"description\":\"kq\",\"dataset\":{\"referenceName\":\"qhz\"},\"linkedService\":{\"referenceName\":\"atypjk\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"tndl\"}},{\"script\":\"cqbjjp\",\"schemaLinkedService\":{\"referenceName\":\"akpkkvr\"},\"rejectedDataLinkedService\":{\"referenceName\":\"ezhwsvoiodcxjcj\"},\"name\":\"qxybbbytmufzr\",\"description\":\"badnwpruyd\",\"dataset\":{\"referenceName\":\"v\"},\"linkedService\":{\"referenceName\":\"dmeeabcnhiamydwd\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"rqctlixnuzb\"}}]},{\"queryName\":\"fzigfdhpwecrvi\",\"dataflowSinks\":[{\"script\":\"irwfxgupazf\",\"schemaLinkedService\":{\"referenceName\":\"p\"},\"rejectedDataLinkedService\":{\"referenceName\":\"ur\"},\"name\":\"manbsuq\",\"description\":\"mijul\",\"dataset\":{\"referenceName\":\"jorfrgbugprfiymp\"},\"linkedService\":{\"referenceName\":\"bcpieiqolym\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"bcyed\"}},{\"script\":\"hbnkhgpe\",\"schemaLinkedService\":{\"referenceName\":\"ylukpjdmdykjh\"},\"rejectedDataLinkedService\":{\"referenceName\":\"ngdfi\"},\"name\":\"pwpf\",\"description\":\"ljrrgvyu\",\"dataset\":{\"referenceName\":\"nnvckpd\"},\"linkedService\":{\"referenceName\":\"viaengydkgicb\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"yvgukauhnhdhssul\"}}]}],\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"gfvgxmnanrcqet\",\"datasetParameters\":\"datavyvxbtpqj\",\"parameters\":{\"pi\":\"datacoruyistjwlnt\"},\"\":{\"ymouwnnhbz\":\"dataypncdebpelgyagu\",\"fqqjydrhwnnux\":\"datascztsatfu\",\"vekbknrr\":\"datahietzovbu\",\"gqbyoyhfbbb\":\"dataxcpnxdzpfzmdsl\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"y\",\"parameters\":{\"bfmttpz\":\"datalqprhnchpets\",\"pcnqesvhsi\":\"datanrmpuvdusizsnhe\"}},\"folderPath\":\"datavdmubz\"},\"integrationRuntime\":{\"referenceName\":\"m\",\"parameters\":{\"hudagrgxrsx\":\"datakz\",\"rvkpnmaawm\":\"datazwnuibdresbjo\",\"grxooqqdlmbu\":\"datamai\",\"yfgjxmgwk\":\"datakayfjzycogwj\"}},\"compute\":{\"computeType\":\"datanosqlfvsv\",\"coreCount\":\"datauqxhqcvaibzbvko\"},\"traceLevel\":\"datajtvefbiodgtzi\",\"continueOnError\":\"datap\",\"runConcurrently\":\"dataaallvee\",\"sourceStagingConcurrency\":\"datasdnrlkzzlok\"},\"policy\":{\"timeout\":\"datadep\",\"retry\":\"datavuzcw\",\"retryIntervalInSeconds\":1358110966,\"secureInput\":true,\"secureOutput\":false,\"\":{\"txttahsojgf\":\"datapv\",\"fmgudkfoybih\":\"datay\",\"tkkufbi\":\"datapnkwt\"}},\"name\":\"mddv\",\"description\":\"b\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"rnggwujyukjfsb\",\"dependencyConditions\":[\"Succeeded\",\"Completed\"],\"\":{\"ff\":\"datanihxwfmuwhgxvgk\"}},{\"activity\":\"sisla\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"gx\":\"datavqqxtyhdikdrat\",\"vl\":\"dataasry\",\"pichsbzgw\":\"databrovumv\"}},{\"activity\":\"farbj\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"nbhwtagfe\":\"datawtfe\",\"uuljnang\":\"datauqudewjnzlq\",\"lwpa\":\"datacddwmnsapg\"}}],\"userProperties\":[{\"name\":\"grzntpqvhkjbgcqq\",\"value\":\"datay\"}],\"\":{\"nahelfqhaypl\":\"datazoiqai\",\"id\":\"datakqgljcrqatxze\",\"nazefti\":\"databsfpaomlg\",\"uql\":\"datapypsjokjjrj\"}}") + "{\"type\":\"ExecuteWranglingDataflow\",\"typeProperties\":{\"sinks\":{\"tfdcwekbbvtcox\":{\"script\":\"rpps\",\"schemaLinkedService\":{\"referenceName\":\"dohiotgf\",\"parameters\":{\"mzpitziej\":\"datapaircnu\",\"h\":\"dataebzofmmcejvs\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"sxrmlxszxokpqn\",\"parameters\":{\"b\":\"datacyjsekdfhnhbkt\"}},\"name\":\"on\",\"description\":\"umlfdxetqknzev\",\"dataset\":{\"referenceName\":\"pqnqneo\",\"parameters\":{\"qlinlwcxrxd\":\"datarmng\",\"u\":\"dataxctojxtkmdegmiv\"}},\"linkedService\":{\"referenceName\":\"pctlbu\",\"parameters\":{\"kofzzsohc\":\"datapabturkmktcsqkt\",\"vepmhohqxl\":\"datae\",\"tliwoodndu\":\"datay\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"kyzirgiyqz\",\"datasetParameters\":\"datanbaz\",\"parameters\":{\"daruwvrvx\":\"datagb\",\"adeqslhz\":\"dataozyhu\",\"mqazolroqusrlkp\":\"datay\"},\"\":{\"mkbpdpk\":\"dataqydrnwsfa\",\"slqikocgzjmjdoq\":\"datalh\"}}},\"aaq\":{\"script\":\"gjdpy\",\"schemaLinkedService\":{\"referenceName\":\"m\",\"parameters\":{\"poynbsttureqvxzl\":\"datarwpjtj\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"aiusglgfecsr\",\"parameters\":{\"xbmgheyamoety\":\"dataswmkxbbziffpvvg\",\"xseyjqklaihqrbrm\":\"dataevyitidi\",\"pydjsubt\":\"datahljqqbue\",\"df\":\"datafbvcveomdlrsjgu\"}},\"name\":\"dqvuqufaowu\",\"description\":\"ujjvojmynlv\",\"dataset\":{\"referenceName\":\"jslxewfqvlhj\",\"parameters\":{\"fgzlrnfmmefppjxt\":\"datarh\",\"dvdrn\":\"dataffwqbdvgfgirrzyn\"}},\"linkedService\":{\"referenceName\":\"qfrxggvstyxv\",\"parameters\":{\"imfpnp\":\"dataaqf\",\"tdorvxdwgpu\":\"datakdgjnd\",\"dzjmjkg\":\"datajeffpidwqr\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"plcoqbouetfx\",\"datasetParameters\":\"dataj\",\"parameters\":{\"dgq\":\"datadlokhimzfltxqpoz\",\"jwjnvhu\":\"datakfevhgjk\"},\"\":{\"hsvthkg\":\"datawvqbpazjmfqus\"}}},\"i\":{\"script\":\"yifjv\",\"schemaLinkedService\":{\"referenceName\":\"gjnqujtcwp\",\"parameters\":{\"afhvuy\":\"dataaft\",\"bdv\":\"databtlmnrdkiqs\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"qsmk\",\"parameters\":{\"mexrofqh\":\"dataljxnkpd\",\"gwov\":\"dataptsdlcsrhttmh\",\"krcwnlyqq\":\"dataduzqu\",\"q\":\"dataknul\"}},\"name\":\"acusmosjawbnxciz\",\"description\":\"ifndgrjnzjygh\",\"dataset\":{\"referenceName\":\"fs\",\"parameters\":{\"ems\":\"datapvgec\",\"vxlaywkbuve\":\"datacgrkgt\"}},\"linkedService\":{\"referenceName\":\"tezeyfdgnaoi\",\"parameters\":{\"fzdgsmeeqelmrpvg\":\"datafdgtwxiesrhvgp\",\"gqsk\":\"datax\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"wobeg\",\"datasetParameters\":\"datajxkxvg\",\"parameters\":{\"enulrfe\":\"datakfef\"},\"\":{\"llhzlicvrdwjght\":\"dataqdvooqjraknngad\",\"doieo\":\"datapvomxtosdbv\",\"adhdlr\":\"dataeebzewbifcyptl\"}}},\"dspykcreuopigsu\":{\"script\":\"uewrhkjmphfhmua\",\"schemaLinkedService\":{\"referenceName\":\"uulhfdggsr\",\"parameters\":{\"tgelfkhmgs\":\"datahhlggobjcf\",\"aqk\":\"datahocrphzdkikjy\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"upm\",\"parameters\":{\"are\":\"datatsa\",\"mjwmld\":\"datav\",\"ryykon\":\"datarwglmmcat\",\"ltcois\":\"dataenmvhbgpgvliinu\"}},\"name\":\"qfzbiy\",\"description\":\"wyyvsbjpyxlzxjir\",\"dataset\":{\"referenceName\":\"prsh\",\"parameters\":{\"duwqovlqfz\":\"datagoqxfbscitizroru\",\"kovubfugdgpmtzqp\":\"dataehagorbspotq\"}},\"linkedService\":{\"referenceName\":\"ochmeximhmisvetu\",\"parameters\":{\"lgypnaqwjsdwna\":\"datakjamihnr\",\"iiqixfygntrynfoa\":\"datauqntxbee\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ztpss\",\"datasetParameters\":\"datadqcrigygtod\",\"parameters\":{\"pdzbybrvkxrcfzsz\":\"datai\"},\"\":{\"tmsphcrnhvxt\":\"datamkeucvqtuuzftds\"}}}},\"queries\":[{\"queryName\":\"k\",\"dataflowSinks\":[{\"script\":\"crdrdkexcyw\",\"schemaLinkedService\":{\"referenceName\":\"iddcnljlly\"},\"rejectedDataLinkedService\":{\"referenceName\":\"hqhcirsajtdz\"},\"name\":\"eliktk\",\"description\":\"mp\",\"dataset\":{\"referenceName\":\"zxicq\"},\"linkedService\":{\"referenceName\":\"wzxqmve\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"roymrqdgyttfzoz\"}},{\"script\":\"vnj\",\"schemaLinkedService\":{\"referenceName\":\"mfcgb\"},\"rejectedDataLinkedService\":{\"referenceName\":\"pslwlh\"},\"name\":\"uezxcpxwqgmnqueq\",\"description\":\"bedfoosiplhygpsa\",\"dataset\":{\"referenceName\":\"pmmshfhr\"},\"linkedService\":{\"referenceName\":\"y\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"vga\"}},{\"script\":\"rulpnrjswrp\",\"schemaLinkedService\":{\"referenceName\":\"recwdle\"},\"rejectedDataLinkedService\":{\"referenceName\":\"muqmzxrjvh\"},\"name\":\"pnyehhqytjrmxaz\",\"description\":\"iqzaeadkahpokf\",\"dataset\":{\"referenceName\":\"ivhozhrwbvfljxl\"},\"linkedService\":{\"referenceName\":\"tirnpazrbkhyzufk\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"vbyf\"}},{\"script\":\"ihuupeflkwbvx\",\"schemaLinkedService\":{\"referenceName\":\"ahdkeayu\"},\"rejectedDataLinkedService\":{\"referenceName\":\"ivp\"},\"name\":\"emydn\",\"description\":\"pbr\",\"dataset\":{\"referenceName\":\"gtblxamd\"},\"linkedService\":{\"referenceName\":\"pifygxuaidrbz\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"nkwullvuk\"}}]},{\"queryName\":\"mosjzmandjjqh\",\"dataflowSinks\":[{\"script\":\"q\",\"schemaLinkedService\":{\"referenceName\":\"djr\"},\"rejectedDataLinkedService\":{\"referenceName\":\"qdiaxf\"},\"name\":\"iwrfocbetlljqkgl\",\"description\":\"jawaxvlc\",\"dataset\":{\"referenceName\":\"c\"},\"linkedService\":{\"referenceName\":\"seqmejerjyz\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"zbjieeivdrqtlcx\"}}]},{\"queryName\":\"gykrmfxlturxyvg\",\"dataflowSinks\":[{\"script\":\"hsnpc\",\"schemaLinkedService\":{\"referenceName\":\"mm\"},\"rejectedDataLinkedService\":{\"referenceName\":\"erxnljt\"},\"name\":\"ujsnzue\",\"description\":\"znkdbhzcd\",\"dataset\":{\"referenceName\":\"nztzhqsbgksfjq\"},\"linkedService\":{\"referenceName\":\"eqhj\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"rneorbdtli\"}},{\"script\":\"aumr\",\"schemaLinkedService\":{\"referenceName\":\"onugcomqlbjxps\"},\"rejectedDataLinkedService\":{\"referenceName\":\"vggv\"},\"name\":\"qparqzygdko\",\"description\":\"tjsr\",\"dataset\":{\"referenceName\":\"qamqqqvuk\"},\"linkedService\":{\"referenceName\":\"cdppdmmfdu\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"cltygxziti\"}},{\"script\":\"b\",\"schemaLinkedService\":{\"referenceName\":\"y\"},\"rejectedDataLinkedService\":{\"referenceName\":\"djcgdharlrfa\"},\"name\":\"dyswlmxe\",\"description\":\"gkf\",\"dataset\":{\"referenceName\":\"gkbujqtklzwmqzk\"},\"linkedService\":{\"referenceName\":\"czpzwfewbj\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"mhpue\"}},{\"script\":\"hhne\",\"schemaLinkedService\":{\"referenceName\":\"xx\"},\"rejectedDataLinkedService\":{\"referenceName\":\"fiali\"},\"name\":\"ymnukvfjbxvhui\",\"description\":\"ody\",\"dataset\":{\"referenceName\":\"otqp\"},\"linkedService\":{\"referenceName\":\"wrahqqumozulefp\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"hx\"}}]},{\"queryName\":\"bgw\",\"dataflowSinks\":[{\"script\":\"mmztvkg\",\"schemaLinkedService\":{\"referenceName\":\"irhnkmjgnnpl\"},\"rejectedDataLinkedService\":{\"referenceName\":\"xopi\"},\"name\":\"x\",\"description\":\"ypsmpgop\",\"dataset\":{\"referenceName\":\"stysirhnwse\"},\"linkedService\":{\"referenceName\":\"we\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"rnjiyddc\"}},{\"script\":\"eozayji\",\"schemaLinkedService\":{\"referenceName\":\"cb\"},\"rejectedDataLinkedService\":{\"referenceName\":\"tm\"},\"name\":\"wqocjkqohcfnomw\",\"description\":\"ebvjmmsgukoql\",\"dataset\":{\"referenceName\":\"kerztenzkbppgc\"},\"linkedService\":{\"referenceName\":\"tdzmei\"},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"ybcorsmdza\"}}]}],\"dataFlow\":{\"type\":\"DataFlowReference\",\"referenceName\":\"q\",\"datasetParameters\":\"datadstztzhwbw\",\"parameters\":{\"agegzrt\":\"datauvqp\",\"je\":\"datausxh\",\"j\":\"datapeafzvxbvk\"},\"\":{\"trad\":\"dataoekohxkgxydimopz\",\"tdhoxuxwvbsa\":\"dataihnpwsagebosbzyd\"}},\"staging\":{\"linkedService\":{\"referenceName\":\"szeoxzrrha\",\"parameters\":{\"pq\":\"dataechrj\"}},\"folderPath\":\"dataylkygcgqjdvab\"},\"integrationRuntime\":{\"referenceName\":\"izmtmctehx\",\"parameters\":{\"vbohpcw\":\"datapyerhdablqoll\",\"tjjqcfzdfmqoe\":\"datasqavpu\",\"zymoqatga\":\"datafypuypztn\"}},\"compute\":{\"computeType\":\"datahih\",\"coreCount\":\"dataebaw\"},\"traceLevel\":\"dataajdkjqznmzr\",\"continueOnError\":\"datagyvxln\",\"runConcurrently\":\"datapcrcpishjkov\",\"sourceStagingConcurrency\":\"datax\"},\"policy\":{\"timeout\":\"datamfcfivr\",\"retry\":\"datapnkjuaxyyvxetgsd\",\"retryIntervalInSeconds\":328007856,\"secureInput\":false,\"secureOutput\":false,\"\":{\"rkqdbqhz\":\"dataorkchbnatrdr\",\"gut\":\"datavatypjk\",\"vrlbezhwsvoi\":\"datadlehcqbjjphuakpk\"}},\"name\":\"dcxjcjiqxybbbytm\",\"description\":\"zrjcbadnwp\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"spdmeeabc\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"ctlixnuz\":\"dataydwdbcur\",\"fzigfdhpwecrvi\":\"datam\",\"wfxgu\":\"databfui\"}}],\"userProperties\":[{\"name\":\"fimpxyurnmanb\",\"value\":\"datauq\"},{\"name\":\"rm\",\"value\":\"datajuldojor\"},{\"name\":\"rgbugprfiympy\",\"value\":\"databcpieiqolym\"}],\"\":{\"hhbnkhgpesyyluk\":\"databcyed\",\"gdfispwpfjxljr\":\"datajdmdykjhuk\"}}") .toObject(ExecuteWranglingDataflowActivity.class); - Assertions.assertEquals("mddv", model.name()); - Assertions.assertEquals("b", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("rnggwujyukjfsb", model.dependsOn().get(0).activity()); + Assertions.assertEquals("dcxjcjiqxybbbytm", model.name()); + Assertions.assertEquals("zrjcbadnwp", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("spdmeeabc", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("grzntpqvhkjbgcqq", model.userProperties().get(0).name()); - Assertions.assertEquals(1358110966, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals("fimpxyurnmanb", model.userProperties().get(0).name()); + Assertions.assertEquals(328007856, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("vxlaywkbuve", model.sinks().get("dlrr").name()); - Assertions.assertEquals("tezeyfdgnaoi", model.sinks().get("dlrr").description()); - Assertions.assertEquals("ufdgtw", model.sinks().get("dlrr").dataset().referenceName()); - Assertions.assertEquals("rpvggxr", model.sinks().get("dlrr").linkedService().referenceName()); - Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.sinks().get("dlrr").flowlet().type()); - Assertions.assertEquals("faenul", model.sinks().get("dlrr").flowlet().referenceName()); - Assertions.assertEquals("qsmk", model.sinks().get("dlrr").schemaLinkedService().referenceName()); - Assertions.assertEquals("cusmosjawb", model.sinks().get("dlrr").rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("cbtlmnrdkiqsqbdv", model.sinks().get("dlrr").script()); - Assertions.assertEquals("aumr", model.queries().get(0).queryName()); - Assertions.assertEquals("ygdkovy", model.queries().get(0).dataflowSinks().get(0).name()); - Assertions.assertEquals("srboq", model.queries().get(0).dataflowSinks().get(0).description()); - Assertions.assertEquals("qqqvu", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); - Assertions.assertEquals("tcdppdmmfdubccc", + Assertions.assertEquals("on", model.sinks().get("tfdcwekbbvtcox").name()); + Assertions.assertEquals("umlfdxetqknzev", model.sinks().get("tfdcwekbbvtcox").description()); + Assertions.assertEquals("pqnqneo", model.sinks().get("tfdcwekbbvtcox").dataset().referenceName()); + Assertions.assertEquals("pctlbu", model.sinks().get("tfdcwekbbvtcox").linkedService().referenceName()); + Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, + model.sinks().get("tfdcwekbbvtcox").flowlet().type()); + Assertions.assertEquals("kyzirgiyqz", model.sinks().get("tfdcwekbbvtcox").flowlet().referenceName()); + Assertions.assertEquals("dohiotgf", model.sinks().get("tfdcwekbbvtcox").schemaLinkedService().referenceName()); + Assertions.assertEquals("sxrmlxszxokpqn", + model.sinks().get("tfdcwekbbvtcox").rejectedDataLinkedService().referenceName()); + Assertions.assertEquals("rpps", model.sinks().get("tfdcwekbbvtcox").script()); + Assertions.assertEquals("k", model.queries().get(0).queryName()); + Assertions.assertEquals("eliktk", model.queries().get(0).dataflowSinks().get(0).name()); + Assertions.assertEquals("mp", model.queries().get(0).dataflowSinks().get(0).description()); + Assertions.assertEquals("zxicq", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); + Assertions.assertEquals("wzxqmve", model.queries().get(0).dataflowSinks().get(0).linkedService().referenceName()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.queries().get(0).dataflowSinks().get(0).flowlet().type()); - Assertions.assertEquals("gx", model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); - Assertions.assertEquals("mqlb", + Assertions.assertEquals("roymrqdgyttfzoz", + model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); + Assertions.assertEquals("iddcnljlly", model.queries().get(0).dataflowSinks().get(0).schemaLinkedService().referenceName()); - Assertions.assertEquals("pslvvggvnqparq", + Assertions.assertEquals("hqhcirsajtdz", model.queries().get(0).dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("ug", model.queries().get(0).dataflowSinks().get(0).script()); + Assertions.assertEquals("crdrdkexcyw", model.queries().get(0).dataflowSinks().get(0).script()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("gfvgxmnanrcqet", model.dataFlow().referenceName()); - Assertions.assertEquals("y", model.staging().linkedService().referenceName()); - Assertions.assertEquals("m", model.integrationRuntime().referenceName()); + Assertions.assertEquals("q", model.dataFlow().referenceName()); + Assertions.assertEquals("szeoxzrrha", model.staging().linkedService().referenceName()); + Assertions.assertEquals("izmtmctehx", model.integrationRuntime().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ExecuteWranglingDataflowActivity model - = new ExecuteWranglingDataflowActivity().withName("mddv").withDescription("b") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("rnggwujyukjfsb") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("sisla") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("farbj") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("grzntpqvhkjbgcqq").withValue("datay"))) - .withPolicy(new ActivityPolicy().withTimeout("datadep").withRetry("datavuzcw") - .withRetryIntervalInSeconds(1358110966).withSecureInput(true).withSecureOutput(false) - .withAdditionalProperties(mapOf())) - .withSinks(mapOf("dlrr", - new PowerQuerySink().withName("vxlaywkbuve").withDescription("tezeyfdgnaoi") - .withDataset(new DatasetReference().withReferenceName("ufdgtw") - .withParameters(mapOf("hvg", "datas", "e", "datahfzdgsmee"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("rpvggxr") - .withParameters(mapOf("egdxjxkxvgode", "datakdkjwo"))) + = new ExecuteWranglingDataflowActivity().withName("dcxjcjiqxybbbytm").withDescription("zrjcbadnwp") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("spdmeeabc") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("fimpxyurnmanb").withValue("datauq"), + new UserProperty().withName("rm").withValue("datajuldojor"), + new UserProperty().withName("rgbugprfiympy").withValue("databcpieiqolym"))) + .withPolicy(new ActivityPolicy() + .withTimeout("datamfcfivr").withRetry("datapnkjuaxyyvxetgsd").withRetryIntervalInSeconds(328007856) + .withSecureInput(false).withSecureOutput(false).withAdditionalProperties(mapOf())) + .withSinks(mapOf("tfdcwekbbvtcox", + new PowerQuerySink().withName("on").withDescription("umlfdxetqknzev") + .withDataset(new DatasetReference().withReferenceName("pqnqneo") + .withParameters(mapOf("qlinlwcxrxd", "datarmng", "u", "dataxctojxtkmdegmiv"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("pctlbu").withParameters( + mapOf("kofzzsohc", "datapabturkmktcsqkt", "vepmhohqxl", "datae", "tliwoodndu", "datay"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("faenul").withDatasetParameters("dataeqefqdvo") - .withParameters(mapOf("knn", "datar", "cvrdwjghttpvom", "dataadjllhzl")) + .withReferenceName("kyzirgiyqz").withDatasetParameters("datanbaz") + .withParameters( + mapOf("daruwvrvx", "datagb", "adeqslhz", "dataozyhu", "mqazolroqusrlkp", "datay")) .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("qsmk") - .withParameters(mapOf("mexrofqh", "dataljxnkpd", "gwov", "dataptsdlcsrhttmh", "krcwnlyqq", - "dataduzqu", "q", "dataknul"))) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("cusmosjawb").withParameters( - mapOf("r", "dataizeuifnd", "fs", "datanzjyghq", "vgec", "dataln", "cgrkgt", "dataems"))) - .withScript("cbtlmnrdkiqsqbdv"), - "nro", - new PowerQuerySink().withName("mjwmld").withDescription("wglmmc") - .withDataset(new DatasetReference().withReferenceName("yryy") - .withParameters(mapOf("ueltcoiszqf", "datavenmvhbgpgvlii", "lzxjiruq", "databiyvowyyvsbjpy", - "izrorupduwqovlq", "datarshtwdgoqxfbsci"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("wehagor") - .withParameters(mapOf("gpmtzqpivochmexi", "dataotqnkovubfug", "mih", "datahmisvetuqibkj", - "nuqnt", "datarulgypnaqwjsdwn", "tryn", "databeeziiqixfyg"))) + .withSchemaLinkedService(new LinkedServiceReference() + .withReferenceName("dohiotgf") + .withParameters(mapOf("mzpitziej", "datapaircnu", "h", "dataebzofmmcejvs"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("sxrmlxszxokpqn") + .withParameters(mapOf("b", "datacyjsekdfhnhbkt"))) + .withScript("rpps"), + "aaq", + new PowerQuerySink().withName("dqvuqufaowu").withDescription("ujjvojmynlv") + .withDataset(new DatasetReference().withReferenceName("jslxewfqvlhj") + .withParameters(mapOf("fgzlrnfmmefppjxt", "datarh", "dvdrn", "dataffwqbdvgfgirrzyn"))) + .withLinkedService( + new LinkedServiceReference().withReferenceName("qfrxggvstyxv").withParameters( + mapOf("imfpnp", "dataaqf", "tdorvxdwgpu", "datakdgjnd", "dzjmjkg", "datajeffpidwqr"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("oeuztpssmhdqcrig").withDatasetParameters("datatodpuqilpdzbyb") - .withParameters(mapOf("ucvq", "dataxrcfzszpoumk", "ktmsphcrn", "datauuzftd")) + .withReferenceName("plcoqbouetfx").withDatasetParameters("dataj") + .withParameters(mapOf("dgq", "datadlokhimzfltxqpoz", "jwjnvhu", "datakfevhgjk")) .withAdditionalProperties(mapOf())) - .withSchemaLinkedService( - new LinkedServiceReference().withReferenceName("jmphfhmuaoou").withParameters( - mapOf("srzolhhlggob", "datafdg", "o", "datacfbtgelfkhmgsp", "i", "datarphzd"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("yhaqkglu") - .withParameters(mapOf("tsa", "dataq", "v", "dataare"))) - .withScript("vuewr"), - "dt", - new PowerQuerySink().withName("uf").withDescription("qtvbyfyz") - .withDataset(new DatasetReference().withReferenceName("uupeflk") - .withParameters(mapOf("d", "dataxnja", "ydntupbrvdgtblx", "dataeayuowivpne", "ztlsnkwullvu", - "datamdabpifygxuaidr"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("ymosj").withParameters( - mapOf("q", "datandjjqhins", "iwrfocbetlljqkgl", "dataudjrotqdiaxf", "xvl", "datavjaw"))) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("m") + .withParameters(mapOf("poynbsttureqvxzl", "datarwpjtj"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("aiusglgfecsr") + .withParameters(mapOf("xbmgheyamoety", "dataswmkxbbziffpvvg", "xseyjqklaihqrbrm", + "dataevyitidi", "pydjsubt", "datahljqqbue", "df", "datafbvcveomdlrsjgu"))) + .withScript("gjdpy"), + "i", + new PowerQuerySink().withName("acusmosjawbnxciz").withDescription("ifndgrjnzjygh") + .withDataset(new DatasetReference().withReferenceName("fs") + .withParameters(mapOf("ems", "datapvgec", "vxlaywkbuve", "datacgrkgt"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("tezeyfdgnaoi") + .withParameters(mapOf("fzdgsmeeqelmrpvg", "datafdgtwxiesrhvgp", "gqsk", "datax"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("c").withDatasetParameters("dataseqmejerjyz") - .withParameters(mapOf("ogykrmf", "datazbjieeivdrqtlcx", "w", "datalturxyvgro", "rxnlj", - "datasnpcwymmgb")) - .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("jjimfcgbdupslwl") - .withParameters(mapOf("eqeabedfo", "dataezxcpxwqgmnq", "ahup", "datasiplhygp", "h", "datam", - "pnrjsw", "datahrgeymspvgatzru"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("txrecwdleivmuq") - .withParameters(mapOf("lpnyehhqy", "datarjv", "ka", "datajrmxazkqiqzaea", "xivhozhr", - "datapokf", "azrbkhy", "databvfljxljgtirn"))) - .withScript("rqdgyttfzozrx"))) + .withReferenceName("wobeg").withDatasetParameters("datajxkxvg") + .withParameters(mapOf("enulrfe", "datakfef")).withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("gjnqujtcwp") + .withParameters(mapOf("afhvuy", "dataaft", "bdv", "databtlmnrdkiqs"))) + .withRejectedDataLinkedService(new LinkedServiceReference() + .withReferenceName("qsmk") + .withParameters(mapOf("mexrofqh", "dataljxnkpd", "gwov", "dataptsdlcsrhttmh", "krcwnlyqq", + "dataduzqu", "q", "dataknul"))) + .withScript("yifjv"), + "dspykcreuopigsu", + new PowerQuerySink().withName("qfzbiy").withDescription("wyyvsbjpyxlzxjir") + .withDataset(new DatasetReference().withReferenceName("prsh").withParameters( + mapOf("duwqovlqfz", "datagoqxfbscitizroru", "kovubfugdgpmtzqp", "dataehagorbspotq"))) + .withLinkedService( + new LinkedServiceReference().withReferenceName("ochmeximhmisvetu").withParameters( + mapOf("lgypnaqwjsdwna", "datakjamihnr", "iiqixfygntrynfoa", "datauqntxbee"))) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("ztpss").withDatasetParameters("datadqcrigygtod") + .withParameters(mapOf("pdzbybrvkxrcfzsz", "datai")).withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("uulhfdggsr") + .withParameters(mapOf("tgelfkhmgs", "datahhlggobjcf", "aqk", "datahocrphzdkikjy"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("upm") + .withParameters(mapOf("are", "datatsa", "mjwmld", "datav", "ryykon", "datarwglmmcat", + "ltcois", "dataenmvhbgpgvliinu"))) + .withScript("uewrhkjmphfhmua"))) .withQueries(Arrays.asList( - new PowerQuerySinkMapping().withQueryName("aumr") + new PowerQuerySinkMapping().withQueryName("k") .withDataflowSinks(Arrays.asList( - new PowerQuerySink().withName("ygdkovy").withDescription("srboq") - .withDataset(new DatasetReference().withReferenceName("qqqvu")) - .withLinkedService(new LinkedServiceReference().withReferenceName("tcdppdmmfdubccc")) + new PowerQuerySink().withName("eliktk").withDescription("mp") + .withDataset(new DatasetReference().withReferenceName("zxicq")) + .withLinkedService(new LinkedServiceReference().withReferenceName("wzxqmve")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("gx").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("mqlb")) + .withReferenceName("roymrqdgyttfzoz").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("iddcnljlly")) .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("pslvvggvnqparq")) - .withScript("ug"), - new PowerQuerySink().withName("dyswlmxe").withDescription("gkf") - .withDataset(new DatasetReference().withReferenceName("gkbujqtklzwmqzk")) - .withLinkedService(new LinkedServiceReference().withReferenceName("czpzwfewbj")) + new LinkedServiceReference().withReferenceName("hqhcirsajtdz")) + .withScript("crdrdkexcyw"), + new PowerQuerySink().withName("uezxcpxwqgmnqueq").withDescription("bedfoosiplhygpsa") + .withDataset(new DatasetReference().withReferenceName("pmmshfhr")) + .withLinkedService(new LinkedServiceReference().withReferenceName("y")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("mhpue").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("y")) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("djcgdharlrfa")) - .withScript("tiixb"), - new PowerQuerySink().withName("ymnukvfjbxvhui").withDescription("ody") - .withDataset(new DatasetReference().withReferenceName("otqp")) - .withLinkedService(new LinkedServiceReference().withReferenceName("wrahqqumozulefp")) + .withReferenceName("vga").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("mfcgb")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("pslwlh")) + .withScript("vnj"), + new PowerQuerySink().withName("pnyehhqytjrmxaz").withDescription("iqzaeadkahpokf") + .withDataset(new DatasetReference().withReferenceName("ivhozhrwbvfljxl")) + .withLinkedService(new LinkedServiceReference().withReferenceName("tirnpazrbkhyzufk")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("hx").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("xx")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("fiali")) - .withScript("hhne"), - new PowerQuerySink().withName("hnkmjgn").withDescription("lnnxopixxciy") - .withDataset(new DatasetReference().withReferenceName("mpgopclstysirh")) - .withLinkedService(new LinkedServiceReference().withReferenceName("s")) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("we").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("ysmmz")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("kgxzi")) - .withScript("bgw"))), - new PowerQuerySinkMapping().withQueryName("x") - .withDataflowSinks(Arrays.asList( - new PowerQuerySink().withName("jkqohcfnomwaoe").withDescription("jmmsgukoq") - .withDataset(new DatasetReference().withReferenceName("rkerztenz")) - .withLinkedService(new LinkedServiceReference().withReferenceName("ppgchztdzmeiicd")) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("orsmd").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("yjim")) + .withReferenceName("vbyf").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("recwdle")) .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("bsjtmdwqo")) - .withScript("yddcqteo"), - new PowerQuerySink().withName("agegzrt").withDescription("sxhlje") - .withDataset(new DatasetReference().withReferenceName("eaf")) - .withLinkedService(new LinkedServiceReference().withReferenceName("xbvk")) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("nmoekohxkgxyd").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("hwbwr")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("uvqp")) - .withScript("fdqqjdstz"), - new PowerQuerySink().withName("xuxwvbs").withDescription("zhszeox") - .withDataset(new DatasetReference().withReferenceName("rhakhaec")) - .withLinkedService(new LinkedServiceReference().withReferenceName("jfpqemylkygcg")) + new LinkedServiceReference().withReferenceName("muqmzxrjvh")) + .withScript("rulpnrjswrp"), + new PowerQuerySink().withName("emydn").withDescription("pbr") + .withDataset(new DatasetReference().withReferenceName("gtblxamd")) + .withLinkedService(new LinkedServiceReference().withReferenceName("pifygxuaidrbz")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("v").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("gebosbz")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("btdh")) - .withScript("opzetradbihnpw"))), - new PowerQuerySinkMapping().withQueryName("vsizmtmctehxwnxp").withDataflowSinks(Arrays.asList( - new PowerQuerySink().withName("qcfzdfmqoeofy").withDescription("ypztnnzy") - .withDataset(new DatasetReference().withReferenceName("qatgazd")) - .withLinkedService(new LinkedServiceReference().withReferenceName("ihuaebawapajdkjq")) + .withReferenceName("nkwullvuk").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("ahdkeayu")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("ivp")) + .withScript("ihuupeflkwbvx"))), + new PowerQuerySinkMapping().withQueryName("mosjzmandjjqh") + .withDataflowSinks(Arrays.asList(new PowerQuerySink().withName("iwrfocbetlljqkgl") + .withDescription("jawaxvlc").withDataset(new DatasetReference().withReferenceName("c")) + .withLinkedService(new LinkedServiceReference().withReferenceName("seqmejerjyz")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("zraegyvxlnpvpcrc").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("wsqavpu")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("j")) - .withScript("dablqollbvbohp"), - new PowerQuerySink().withName("rkchbnatrdr").withDescription("kq") - .withDataset(new DatasetReference().withReferenceName("qhz")) - .withLinkedService(new LinkedServiceReference().withReferenceName("atypjk")) + .withReferenceName("zbjieeivdrqtlcx").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("djr")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("qdiaxf")) + .withScript("q"))), + new PowerQuerySinkMapping().withQueryName("gykrmfxlturxyvg").withDataflowSinks(Arrays.asList( + new PowerQuerySink().withName("ujsnzue").withDescription("znkdbhzcd") + .withDataset(new DatasetReference().withReferenceName("nztzhqsbgksfjq")) + .withLinkedService(new LinkedServiceReference().withReferenceName("eqhj")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("tndl").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("fcfivrbgpnkjuaxy")) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("xetgsdhwmbewsyd")) - .withScript("shjkovrqxoj"), - new PowerQuerySink().withName("qxybbbytmufzr").withDescription("badnwpruyd") - .withDataset(new DatasetReference().withReferenceName("v")) - .withLinkedService(new LinkedServiceReference().withReferenceName("dmeeabcnhiamydwd")) + .withReferenceName("rneorbdtli").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("mm")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("erxnljt")) + .withScript("hsnpc"), + new PowerQuerySink().withName("qparqzygdko").withDescription("tjsr") + .withDataset(new DatasetReference().withReferenceName("qamqqqvuk")) + .withLinkedService(new LinkedServiceReference().withReferenceName("cdppdmmfdu")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("rqctlixnuzb").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("akpkkvr")) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("ezhwsvoiodcxjcj")) - .withScript("cqbjjp"))), - new PowerQuerySinkMapping().withQueryName("fzigfdhpwecrvi").withDataflowSinks(Arrays.asList( - new PowerQuerySink().withName("manbsuq").withDescription("mijul") - .withDataset(new DatasetReference().withReferenceName("jorfrgbugprfiymp")) - .withLinkedService(new LinkedServiceReference().withReferenceName("bcpieiqolym")) + .withReferenceName("cltygxziti").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("onugcomqlbjxps")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("vggv")) + .withScript("aumr"), + new PowerQuerySink().withName("dyswlmxe").withDescription("gkf") + .withDataset(new DatasetReference().withReferenceName("gkbujqtklzwmqzk")) + .withLinkedService(new LinkedServiceReference().withReferenceName("czpzwfewbj")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("bcyed").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("p")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("ur")) - .withScript("irwfxgupazf"), - new PowerQuerySink().withName("pwpf").withDescription("ljrrgvyu") - .withDataset(new DatasetReference().withReferenceName("nnvckpd")) - .withLinkedService(new LinkedServiceReference().withReferenceName("viaengydkgicb")) + .withReferenceName("mhpue").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("y")) + .withRejectedDataLinkedService( + new LinkedServiceReference().withReferenceName("djcgdharlrfa")) + .withScript("b"), + new PowerQuerySink().withName("ymnukvfjbxvhui").withDescription("ody") + .withDataset(new DatasetReference().withReferenceName("otqp")) + .withLinkedService(new LinkedServiceReference().withReferenceName("wrahqqumozulefp")) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("yvgukauhnhdhssul").withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("ylukpjdmdykjh")) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("ngdfi")) - .withScript("hbnkhgpe"))))) + .withReferenceName("hx").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("xx")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("fiali")) + .withScript("hhne"))), + new PowerQuerySinkMapping().withQueryName("bgw") + .withDataflowSinks(Arrays.asList( + new PowerQuerySink().withName("x").withDescription("ypsmpgop") + .withDataset(new DatasetReference().withReferenceName("stysirhnwse")) + .withLinkedService(new LinkedServiceReference().withReferenceName("we")) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("rnjiyddc").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("irhnkmjgnnpl")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("xopi")) + .withScript("mmztvkg"), + new PowerQuerySink().withName("wqocjkqohcfnomw").withDescription("ebvjmmsgukoql") + .withDataset(new DatasetReference().withReferenceName("kerztenzkbppgc")) + .withLinkedService(new LinkedServiceReference().withReferenceName("tdzmei")) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("ybcorsmdza").withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("cb")) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("tm")) + .withScript("eozayji"))))) .withDataFlow(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("gfvgxmnanrcqet").withDatasetParameters("datavyvxbtpqj") - .withParameters(mapOf("pi", "datacoruyistjwlnt")).withAdditionalProperties(mapOf())) - .withStaging(new DataFlowStagingInfo() - .withLinkedService(new LinkedServiceReference().withReferenceName("y") - .withParameters(mapOf("bfmttpz", "datalqprhnchpets", "pcnqesvhsi", "datanrmpuvdusizsnhe"))) - .withFolderPath("datavdmubz")) - .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("m") - .withParameters(mapOf("hudagrgxrsx", "datakz", "rvkpnmaawm", "datazwnuibdresbjo", "grxooqqdlmbu", - "datamai", "yfgjxmgwk", "datakayfjzycogwj"))) - .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datanosqlfvsv") - .withCoreCount("datauqxhqcvaibzbvko")) - .withTraceLevel("datajtvefbiodgtzi").withContinueOnError("datap").withRunConcurrently("dataaallvee") - .withSourceStagingConcurrency("datasdnrlkzzlok"); + .withReferenceName("q").withDatasetParameters("datadstztzhwbw") + .withParameters(mapOf("agegzrt", "datauvqp", "je", "datausxh", "j", "datapeafzvxbvk")) + .withAdditionalProperties(mapOf())) + .withStaging(new DataFlowStagingInfo().withLinkedService(new LinkedServiceReference() + .withReferenceName("szeoxzrrha").withParameters(mapOf("pq", "dataechrj"))) + .withFolderPath("dataylkygcgqjdvab")) + .withIntegrationRuntime(new IntegrationRuntimeReference().withReferenceName("izmtmctehx") + .withParameters(mapOf("vbohpcw", "datapyerhdablqoll", "tjjqcfzdfmqoe", "datasqavpu", "zymoqatga", + "datafypuypztn"))) + .withCompute(new ExecuteDataFlowActivityTypePropertiesCompute().withComputeType("datahih") + .withCoreCount("dataebaw")) + .withTraceLevel("dataajdkjqznmzr").withContinueOnError("datagyvxln") + .withRunConcurrently("datapcrcpishjkov").withSourceStagingConcurrency("datax"); model = BinaryData.fromObject(model).toObject(ExecuteWranglingDataflowActivity.class); - Assertions.assertEquals("mddv", model.name()); - Assertions.assertEquals("b", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("rnggwujyukjfsb", model.dependsOn().get(0).activity()); + Assertions.assertEquals("dcxjcjiqxybbbytm", model.name()); + Assertions.assertEquals("zrjcbadnwp", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("spdmeeabc", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("grzntpqvhkjbgcqq", model.userProperties().get(0).name()); - Assertions.assertEquals(1358110966, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals("fimpxyurnmanb", model.userProperties().get(0).name()); + Assertions.assertEquals(328007856, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("vxlaywkbuve", model.sinks().get("dlrr").name()); - Assertions.assertEquals("tezeyfdgnaoi", model.sinks().get("dlrr").description()); - Assertions.assertEquals("ufdgtw", model.sinks().get("dlrr").dataset().referenceName()); - Assertions.assertEquals("rpvggxr", model.sinks().get("dlrr").linkedService().referenceName()); - Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.sinks().get("dlrr").flowlet().type()); - Assertions.assertEquals("faenul", model.sinks().get("dlrr").flowlet().referenceName()); - Assertions.assertEquals("qsmk", model.sinks().get("dlrr").schemaLinkedService().referenceName()); - Assertions.assertEquals("cusmosjawb", model.sinks().get("dlrr").rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("cbtlmnrdkiqsqbdv", model.sinks().get("dlrr").script()); - Assertions.assertEquals("aumr", model.queries().get(0).queryName()); - Assertions.assertEquals("ygdkovy", model.queries().get(0).dataflowSinks().get(0).name()); - Assertions.assertEquals("srboq", model.queries().get(0).dataflowSinks().get(0).description()); - Assertions.assertEquals("qqqvu", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); - Assertions.assertEquals("tcdppdmmfdubccc", + Assertions.assertEquals("on", model.sinks().get("tfdcwekbbvtcox").name()); + Assertions.assertEquals("umlfdxetqknzev", model.sinks().get("tfdcwekbbvtcox").description()); + Assertions.assertEquals("pqnqneo", model.sinks().get("tfdcwekbbvtcox").dataset().referenceName()); + Assertions.assertEquals("pctlbu", model.sinks().get("tfdcwekbbvtcox").linkedService().referenceName()); + Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, + model.sinks().get("tfdcwekbbvtcox").flowlet().type()); + Assertions.assertEquals("kyzirgiyqz", model.sinks().get("tfdcwekbbvtcox").flowlet().referenceName()); + Assertions.assertEquals("dohiotgf", model.sinks().get("tfdcwekbbvtcox").schemaLinkedService().referenceName()); + Assertions.assertEquals("sxrmlxszxokpqn", + model.sinks().get("tfdcwekbbvtcox").rejectedDataLinkedService().referenceName()); + Assertions.assertEquals("rpps", model.sinks().get("tfdcwekbbvtcox").script()); + Assertions.assertEquals("k", model.queries().get(0).queryName()); + Assertions.assertEquals("eliktk", model.queries().get(0).dataflowSinks().get(0).name()); + Assertions.assertEquals("mp", model.queries().get(0).dataflowSinks().get(0).description()); + Assertions.assertEquals("zxicq", model.queries().get(0).dataflowSinks().get(0).dataset().referenceName()); + Assertions.assertEquals("wzxqmve", model.queries().get(0).dataflowSinks().get(0).linkedService().referenceName()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.queries().get(0).dataflowSinks().get(0).flowlet().type()); - Assertions.assertEquals("gx", model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); - Assertions.assertEquals("mqlb", + Assertions.assertEquals("roymrqdgyttfzoz", + model.queries().get(0).dataflowSinks().get(0).flowlet().referenceName()); + Assertions.assertEquals("iddcnljlly", model.queries().get(0).dataflowSinks().get(0).schemaLinkedService().referenceName()); - Assertions.assertEquals("pslvvggvnqparq", + Assertions.assertEquals("hqhcirsajtdz", model.queries().get(0).dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("ug", model.queries().get(0).dataflowSinks().get(0).script()); + Assertions.assertEquals("crdrdkexcyw", model.queries().get(0).dataflowSinks().get(0).script()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataFlow().type()); - Assertions.assertEquals("gfvgxmnanrcqet", model.dataFlow().referenceName()); - Assertions.assertEquals("y", model.staging().linkedService().referenceName()); - Assertions.assertEquals("m", model.integrationRuntime().referenceName()); + Assertions.assertEquals("q", model.dataFlow().referenceName()); + Assertions.assertEquals("szeoxzrrha", model.staging().linkedService().referenceName()); + Assertions.assertEquals("izmtmctehx", model.integrationRuntime().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutionActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutionActivityTests.java index 865f16b3d7aed..6bfccc4698094 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutionActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExecutionActivityTests.java @@ -22,56 +22,53 @@ public final class ExecutionActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExecutionActivity model = BinaryData.fromString( - "{\"type\":\"Execution\",\"linkedServiceName\":{\"referenceName\":\"ixhor\",\"parameters\":{\"dqttiqwixsdx\":\"datadxjkdsqeay\",\"tvbqhyszflzjzdci\":\"dataflwfvahbwhrguq\",\"shbkdptcm\":\"dataxlggrtpayfklb\"}},\"policy\":{\"timeout\":\"datanvkhjtrashnf\",\"retry\":\"dataiykloeqxccpqbo\",\"retryIntervalInSeconds\":1543618130,\"secureInput\":true,\"secureOutput\":true,\"\":{\"uu\":\"dataaf\",\"rnlgnwwjikg\":\"dataoqqseoinxkot\",\"kz\":\"datawdcz\"}},\"name\":\"bpj\",\"description\":\"h\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"qznlddngq\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Completed\",\"Succeeded\"],\"\":{\"qh\":\"dataemagsyljwwp\",\"jopbgtzrgyrldoa\":\"datahlajmikqvnrj\",\"k\":\"dataldglz\"}},{\"activity\":\"ftjrederkvbdvl\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"rvffjpwszlfyft\":\"datajjryztghd\",\"daxvirpuwacfqn\":\"dataaetcplxidiuxzzh\"}}],\"userProperties\":[{\"name\":\"b\",\"value\":\"datapgibyzrpqehr\"},{\"name\":\"ldvxcjjhjnpa\",\"value\":\"dataa\"},{\"name\":\"fyxegklywdlxmy\",\"value\":\"dataqmamasc\"},{\"name\":\"qtqzwntnlyrpb\",\"value\":\"datadwiaxsucforg\"}],\"\":{\"nroizz\":\"dataip\",\"zuxpldzkvbe\":\"dataknybfsoayatqk\"}}") + "{\"type\":\"Execution\",\"linkedServiceName\":{\"referenceName\":\"hkdkv\",\"parameters\":{\"otidikxmtmjkfmr\":\"datapbds\",\"vsdyjmbydrgxvnmt\":\"datangrjsqtirhabhhp\",\"qcyycxlllk\":\"datamuxrdmudwruogmth\"}},\"policy\":{\"timeout\":\"datajlwf\",\"retry\":\"datagiebqvusc\",\"retryIntervalInSeconds\":90734181,\"secureInput\":false,\"secureOutput\":false,\"\":{\"wfdtjpsjwlpcxl\":\"datauplamdgffvxnisoo\",\"oouocafaxvhjrpb\":\"datazzcdrgtu\",\"sgn\":\"datarolge\"}},\"name\":\"njtxuuwdmrqah\",\"description\":\"yjahbzbtlmacbwm\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"icel\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Completed\",\"Failed\"],\"\":{\"qgjibrxxiaocr\":\"dataibnd\",\"pugnvhtgwadu\":\"datauhumgw\",\"zxzwinrg\":\"dataokoxqboz\"}},{\"activity\":\"kqobovqlltql\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"jkcol\":\"datanrb\",\"qvv\":\"datazqlyputawdmdikuf\",\"hvfojcvnh\":\"dataujzofyldxk\",\"kysg\":\"dataebuiy\"}}],\"userProperties\":[{\"name\":\"beauvldb\",\"value\":\"datan\"},{\"name\":\"guifqjtoxzxbljpz\",\"value\":\"datauugdarfumitjai\"}],\"\":{\"qrjcozrwrylcttv\":\"dataokfdybvywbgmjrvr\",\"a\":\"datakxgffpvvqwvvnxoq\"}}") .toObject(ExecutionActivity.class); - Assertions.assertEquals("bpj", model.name()); - Assertions.assertEquals("h", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("qznlddngq", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("b", model.userProperties().get(0).name()); - Assertions.assertEquals("ixhor", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1543618130, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals("njtxuuwdmrqah", model.name()); + Assertions.assertEquals("yjahbzbtlmacbwm", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("icel", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("beauvldb", model.userProperties().get(0).name()); + Assertions.assertEquals("hkdkv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(90734181, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExecutionActivity model = new ExecutionActivity().withName("bpj").withDescription("h") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + ExecutionActivity model = new ExecutionActivity().withName("njtxuuwdmrqah").withDescription("yjahbzbtlmacbwm") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("qznlddngq") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("icel") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, + DependencyCondition.COMPLETED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ftjrederkvbdvl") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("kqobovqlltql") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("b").withValue("datapgibyzrpqehr"), - new UserProperty().withName("ldvxcjjhjnpa").withValue("dataa"), - new UserProperty().withName("fyxegklywdlxmy").withValue("dataqmamasc"), - new UserProperty().withName("qtqzwntnlyrpb").withValue("datadwiaxsucforg"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ixhor") - .withParameters(mapOf("dqttiqwixsdx", "datadxjkdsqeay", "tvbqhyszflzjzdci", "dataflwfvahbwhrguq", - "shbkdptcm", "dataxlggrtpayfklb"))) - .withPolicy(new ActivityPolicy().withTimeout("datanvkhjtrashnf").withRetry("dataiykloeqxccpqbo") - .withRetryIntervalInSeconds(1543618130).withSecureInput(true).withSecureOutput(true) + .withUserProperties(Arrays.asList(new UserProperty().withName("beauvldb").withValue("datan"), + new UserProperty().withName("guifqjtoxzxbljpz").withValue("datauugdarfumitjai"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("hkdkv") + .withParameters(mapOf("otidikxmtmjkfmr", "datapbds", "vsdyjmbydrgxvnmt", "datangrjsqtirhabhhp", + "qcyycxlllk", "datamuxrdmudwruogmth"))) + .withPolicy(new ActivityPolicy().withTimeout("datajlwf").withRetry("datagiebqvusc") + .withRetryIntervalInSeconds(90734181).withSecureInput(false).withSecureOutput(false) .withAdditionalProperties(mapOf())); model = BinaryData.fromObject(model).toObject(ExecutionActivity.class); - Assertions.assertEquals("bpj", model.name()); - Assertions.assertEquals("h", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("qznlddngq", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("b", model.userProperties().get(0).name()); - Assertions.assertEquals("ixhor", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1543618130, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals("njtxuuwdmrqah", model.name()); + Assertions.assertEquals("yjahbzbtlmacbwm", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("icel", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("beauvldb", model.userProperties().get(0).name()); + Assertions.assertEquals("hkdkv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(90734181, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExportSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExportSettingsTests.java index b631dc6c7e722..af4700e647e22 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExportSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExportSettingsTests.java @@ -12,9 +12,9 @@ public final class ExportSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExportSettings model = BinaryData.fromString( - "{\"type\":\"ExportSettings\",\"\":{\"xuuqb\":\"datangpvvnbuknvkutls\",\"wgkreozpufkcam\":\"datavpbe\",\"cgdndpbsieymmcb\":\"datacbzgiklqpe\"}}") - .toObject(ExportSettings.class); + ExportSettings model + = BinaryData.fromString("{\"type\":\"ExportSettings\",\"\":{\"dhzwdyva\":\"dataffutezxrp\"}}") + .toObject(ExportSettings.class); } @org.junit.jupiter.api.Test diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueByFactoryWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueByFactoryWithResponseMockTests.java index e3f517ab0b5bf..294cba57b4bd3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueByFactoryWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueByFactoryWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetFeatureValueByFactoryWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"featureName\":\"y\",\"value\":\"lfyddy\"}"; + String responseStr = "{\"featureName\":\"dqoj\",\"value\":\"a\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,8 @@ public void testGetFeatureValueByFactoryWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ExposureControlResponse response = manager.exposureControls() - .getFeatureValueByFactoryWithResponse("mvzfp", "ieofvzeihlubd", - new ExposureControlRequest().withFeatureName("iq").withFeatureType("okyvrzl"), + .getFeatureValueByFactoryWithResponse("f", "jrkueprpnzbfoldb", + new ExposureControlRequest().withFeatureName("iljgyrpvmaywpr").withFeatureType("vqbnzrrkmanr"), com.azure.core.util.Context.NONE) .getValue(); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueWithResponseMockTests.java index c82e194448426..515a34f10ab29 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsGetFeatureValueWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetFeatureValueWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"featureName\":\"gcvqhvbcz\",\"value\":\"spnfxwkjhz\"}"; + String responseStr = "{\"featureName\":\"s\",\"value\":\"treihlszpus\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,8 @@ public void testGetFeatureValueWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ExposureControlResponse response = manager.exposureControls() - .getFeatureValueWithResponse("jcqqv", - new ExposureControlRequest().withFeatureName("nyaeckzcbrxsq").withFeatureType("esddjpeeqy"), + .getFeatureValueWithResponse("ddfvdktbaexbvyu", + new ExposureControlRequest().withFeatureName("bycuuxgdadflil").withFeatureType("ptvmtnougmf"), com.azure.core.util.Context.NONE) .getValue(); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsQueryFeatureValuesByFactoryWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsQueryFeatureValuesByFactoryWithResponseMockTests.java index fc5d3dbdf0a26..eb57e8007e3b6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsQueryFeatureValuesByFactoryWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExposureControlsQueryFeatureValuesByFactoryWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testQueryFeatureValuesByFactoryWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"exposureControlResponses\":[{\"featureName\":\"zcikhcpdohvw\",\"value\":\"tcgybuuubtfxjpg\"},{\"featureName\":\"ynofwyzpn\",\"value\":\"pgwriyxyelzm\"},{\"featureName\":\"harucpkpm\",\"value\":\"nnwob\"}]}"; + = "{\"exposureControlResponses\":[{\"featureName\":\"dgzfqs\",\"value\":\"yuillrrqw\"},{\"featureName\":\"hiqjegea\",\"value\":\"oqg\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,11 +51,11 @@ public void testQueryFeatureValuesByFactoryWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ExposureControlBatchResponse response = manager.exposureControls() - .queryFeatureValuesByFactoryWithResponse("kfdlhytckpfgj", "rd", + .queryFeatureValuesByFactoryWithResponse("vvbsilahskesea", "g", new ExposureControlBatchRequest().withExposureControlRequests(Arrays.asList( - new ExposureControlRequest().withFeatureName("snmzlsgalavx").withFeatureType("vwuhvpipaafvtk"), - new ExposureControlRequest().withFeatureName("ordplhwi").withFeatureType("hox"), - new ExposureControlRequest().withFeatureName("yixbrsjgrjtl").withFeatureType("dcz"))), + new ExposureControlRequest().withFeatureName("essiielbtge").withFeatureType("wcqeihuyrzi"), + new ExposureControlRequest().withFeatureName("yvquufplmpbvzbt").withFeatureType("totpvoe"), + new ExposureControlRequest().withFeatureName("fwrao").withFeatureType("r"))), com.azure.core.util.Context.NONE) .getValue(); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExpressionV2Tests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExpressionV2Tests.java new file mode 100644 index 0000000000000..7a16a04d4157f --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ExpressionV2Tests.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.ExpressionV2; +import com.azure.resourcemanager.datafactory.models.ExpressionV2Type; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ExpressionV2Tests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ExpressionV2 model = BinaryData.fromString( + "{\"type\":\"Unary\",\"value\":\"fgfspzw\",\"operator\":\"cu\",\"operands\":[{\"type\":\"Binary\",\"value\":\"jvlwczwdkk\",\"operator\":\"ooqnvhtrfckrm\",\"operands\":[{\"type\":\"Binary\",\"value\":\"tfmpcbvkoughj\",\"operator\":\"pptsvppfdnihxcij\",\"operands\":[{}]},{\"type\":\"Unary\",\"value\":\"yvscgzqncddaqqkl\",\"operator\":\"ibro\",\"operands\":[{},{}]}]},{\"type\":\"Unary\",\"value\":\"nex\",\"operator\":\"sanglw\",\"operands\":[{\"type\":\"Binary\",\"value\":\"m\",\"operator\":\"rhjj\",\"operands\":[{},{},{},{}]},{\"type\":\"Unary\",\"value\":\"rywucpdzbnty\",\"operator\":\"wnpuyhqayls\",\"operands\":[{},{},{},{}]}]},{\"type\":\"Unary\",\"value\":\"lzrrhabbdqne\",\"operator\":\"fujzwqpkhgrdg\",\"operands\":[{\"type\":\"Field\",\"value\":\"qkv\",\"operator\":\"n\",\"operands\":[{},{},{}]},{\"type\":\"Unary\",\"value\":\"aoetrglpaocq\",\"operator\":\"vleouevuhago\",\"operands\":[{},{},{},{}]},{\"type\":\"Binary\",\"value\":\"tuoq\",\"operator\":\"crdnmhrym\",\"operands\":[{},{}]},{\"type\":\"Unary\",\"value\":\"owubkiocjn\",\"operator\":\"nwktbsckcng\",\"operands\":[{},{},{}]}]},{\"type\":\"Field\",\"value\":\"zoxmajpxbtkzvt\",\"operator\":\"tgrmgcfvfwwev\",\"operands\":[{\"type\":\"Constant\",\"value\":\"x\",\"operator\":\"qaqzttogblriznr\",\"operands\":[{},{},{},{}]},{\"type\":\"Unary\",\"value\":\"p\",\"operator\":\"nqljlw\",\"operands\":[{},{},{}]},{\"type\":\"Unary\",\"value\":\"ln\",\"operator\":\"jqlq\",\"operands\":[{},{}]}]}]}") + .toObject(ExpressionV2.class); + Assertions.assertEquals(ExpressionV2Type.UNARY, model.type()); + Assertions.assertEquals("fgfspzw", model.value()); + Assertions.assertEquals("cu", model.operator()); + Assertions.assertEquals(ExpressionV2Type.BINARY, model.operands().get(0).type()); + Assertions.assertEquals("jvlwczwdkk", model.operands().get(0).value()); + Assertions.assertEquals("ooqnvhtrfckrm", model.operands().get(0).operator()); + Assertions.assertEquals(ExpressionV2Type.BINARY, model.operands().get(0).operands().get(0).type()); + Assertions.assertEquals("tfmpcbvkoughj", model.operands().get(0).operands().get(0).value()); + Assertions.assertEquals("pptsvppfdnihxcij", model.operands().get(0).operands().get(0).operator()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ExpressionV2 model = new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("fgfspzw").withOperator("cu") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.BINARY).withValue("jvlwczwdkk") + .withOperator("ooqnvhtrfckrm") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.BINARY).withValue("tfmpcbvkoughj") + .withOperator("pptsvppfdnihxcij").withOperands(Arrays.asList(new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("yvscgzqncddaqqkl") + .withOperator("ibro").withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2())))), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("nex").withOperator("sanglw") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.BINARY).withValue("m").withOperator("rhjj") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2(), + new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("rywucpdzbnty") + .withOperator("wnpuyhqayls") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2(), + new ExpressionV2())))), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("lzrrhabbdqne") + .withOperator("fujzwqpkhgrdg") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.FIELD).withValue("qkv").withOperator("n") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("aoetrglpaocq") + .withOperator("vleouevuhago") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2(), + new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.BINARY).withValue("tuoq").withOperator("crdnmhrym") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("owubkiocjn") + .withOperator("nwktbsckcng") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2())))), + new ExpressionV2().withType(ExpressionV2Type.FIELD).withValue("zoxmajpxbtkzvt") + .withOperator("tgrmgcfvfwwev") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.CONSTANT).withValue("x") + .withOperator("qaqzttogblriznr") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2(), + new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("p").withOperator("nqljlw") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("ln").withOperator("jqlq") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2())))))); + model = BinaryData.fromObject(model).toObject(ExpressionV2.class); + Assertions.assertEquals(ExpressionV2Type.UNARY, model.type()); + Assertions.assertEquals("fgfspzw", model.value()); + Assertions.assertEquals("cu", model.operator()); + Assertions.assertEquals(ExpressionV2Type.BINARY, model.operands().get(0).type()); + Assertions.assertEquals("jvlwczwdkk", model.operands().get(0).value()); + Assertions.assertEquals("ooqnvhtrfckrm", model.operands().get(0).operator()); + Assertions.assertEquals(ExpressionV2Type.BINARY, model.operands().get(0).operands().get(0).type()); + Assertions.assertEquals("tfmpcbvkoughj", model.operands().get(0).operands().get(0).value()); + Assertions.assertEquals("pptsvppfdnihxcij", model.operands().get(0).operands().get(0).operator()); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoriesDeleteByResourceGroupWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoriesDeleteByResourceGroupWithResponseMockTests.java index d059fd9775a96..2c7a34b1b0fb9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoriesDeleteByResourceGroupWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoriesDeleteByResourceGroupWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.factories().deleteByResourceGroupWithResponse("zdfpeauhld", "bwkxevnroew", + manager.factories().deleteByResourceGroupWithResponse("rok", "lopygrsvyjrqhp", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryIdentityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryIdentityTests.java index f784d08fa0435..b66b3e4c9c202 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryIdentityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryIdentityTests.java @@ -15,7 +15,7 @@ public final class FactoryIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FactoryIdentity model = BinaryData.fromString( - "{\"type\":\"SystemAssigned\",\"principalId\":\"19eee805-5ef6-4e59-ba7b-70ae222977c5\",\"tenantId\":\"1c72a7f5-c097-4d9e-829d-456e654add42\",\"userAssignedIdentities\":{\"qjpkcattpngjcrc\":\"dataleyyvx\"}}") + "{\"type\":\"SystemAssigned\",\"principalId\":\"5aa54869-dc8c-4533-8391-cc95f3e0b9ef\",\"tenantId\":\"bd63df9d-a5bd-4e04-b286-b4ebbba5c965\",\"userAssignedIdentities\":{\"qjpkcattpngjcrc\":\"dataleyyvx\"}}") .toObject(FactoryIdentity.class); Assertions.assertEquals(FactoryIdentityType.SYSTEM_ASSIGNED, model.type()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryUpdateParametersTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryUpdateParametersTests.java index c86665f3d115b..7469bd679dcf0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryUpdateParametersTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FactoryUpdateParametersTests.java @@ -17,7 +17,7 @@ public final class FactoryUpdateParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FactoryUpdateParameters model = BinaryData.fromString( - "{\"tags\":{\"bldngkpoc\":\"kouknvudwtiu\",\"npiucgygevqznty\":\"pazyxoegukg\"},\"identity\":{\"type\":\"SystemAssigned\",\"principalId\":\"7577ba02-7532-4693-956f-6193797d6dcd\",\"tenantId\":\"363b425a-1048-4117-a731-700e76eedd89\",\"userAssignedIdentities\":{\"r\":\"datac\",\"dpydn\":\"dataj\",\"sjttgzfbish\":\"datayhxdeoejzicwi\",\"jdeyeamdpha\":\"databkh\"}},\"properties\":{\"publicNetworkAccess\":\"Disabled\"}}") + "{\"tags\":{\"bldngkpoc\":\"kouknvudwtiu\",\"npiucgygevqznty\":\"pazyxoegukg\"},\"identity\":{\"type\":\"SystemAssigned\",\"principalId\":\"4ef79349-41e7-45a9-9a40-a258f0de4c65\",\"tenantId\":\"bcb38326-fe19-43c5-a108-72b8af5678eb\",\"userAssignedIdentities\":{\"r\":\"datac\",\"dpydn\":\"dataj\",\"sjttgzfbish\":\"datayhxdeoejzicwi\",\"jdeyeamdpha\":\"databkh\"}},\"properties\":{\"publicNetworkAccess\":\"Disabled\"}}") .toObject(FactoryUpdateParameters.class); Assertions.assertEquals("kouknvudwtiu", model.tags().get("bldngkpoc")); Assertions.assertEquals(FactoryIdentityType.SYSTEM_ASSIGNED, model.identity().type()); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerReadSettingsTests.java index 88d13c711146c..95f05b9b076d0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerReadSettingsTests.java @@ -11,18 +11,19 @@ public final class FileServerReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FileServerReadSettings model = BinaryData.fromString( - "{\"type\":\"FileServerReadSettings\",\"recursive\":\"datawsaxpbieehpvq\",\"wildcardFolderPath\":\"datafrrjp\",\"wildcardFileName\":\"datagjgyovcpgqiism\",\"fileListPath\":\"datag\",\"enablePartitionDiscovery\":\"datac\",\"partitionRootPath\":\"datakrbkkcx\",\"deleteFilesAfterCompletion\":\"datajkronxmtrhwwdfnc\",\"modifiedDatetimeStart\":\"datatyszhzlvkm\",\"modifiedDatetimeEnd\":\"datan\",\"fileFilter\":\"databzarmepbmogtfp\",\"maxConcurrentConnections\":\"datakxsyohfrlyy\",\"disableMetricsCollection\":\"datagnychuzhngwtbhj\",\"\":{\"dgnuoewfgt\":\"dataioe\",\"xtyavvexjqdj\":\"datammf\",\"bge\":\"datao\"}}") + "{\"type\":\"FileServerReadSettings\",\"recursive\":\"dataajbcbrtiqpjlakam\",\"wildcardFolderPath\":\"dataqluicrqxqjzmosml\",\"wildcardFileName\":\"datappfgtnsxdj\",\"fileListPath\":\"datatnjpkpmdlttmfhde\",\"enablePartitionDiscovery\":\"dataiaaiqyxlromxpe\",\"partitionRootPath\":\"dataxcnh\",\"deleteFilesAfterCompletion\":\"datacbtyor\",\"modifiedDatetimeStart\":\"datadamyumrobbaxnym\",\"modifiedDatetimeEnd\":\"datag\",\"fileFilter\":\"dataqmkakgw\",\"maxConcurrentConnections\":\"dataznob\",\"disableMetricsCollection\":\"datagyheyayktutflhe\",\"\":{\"zszjqzmqjhghih\":\"dataefsahmdcoeexw\"}}") .toObject(FileServerReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FileServerReadSettings model = new FileServerReadSettings().withMaxConcurrentConnections("datakxsyohfrlyy") - .withDisableMetricsCollection("datagnychuzhngwtbhj").withRecursive("datawsaxpbieehpvq") - .withWildcardFolderPath("datafrrjp").withWildcardFileName("datagjgyovcpgqiism").withFileListPath("datag") - .withEnablePartitionDiscovery("datac").withPartitionRootPath("datakrbkkcx") - .withDeleteFilesAfterCompletion("datajkronxmtrhwwdfnc").withModifiedDatetimeStart("datatyszhzlvkm") - .withModifiedDatetimeEnd("datan").withFileFilter("databzarmepbmogtfp"); + FileServerReadSettings model = new FileServerReadSettings().withMaxConcurrentConnections("dataznob") + .withDisableMetricsCollection("datagyheyayktutflhe").withRecursive("dataajbcbrtiqpjlakam") + .withWildcardFolderPath("dataqluicrqxqjzmosml").withWildcardFileName("datappfgtnsxdj") + .withFileListPath("datatnjpkpmdlttmfhde").withEnablePartitionDiscovery("dataiaaiqyxlromxpe") + .withPartitionRootPath("dataxcnh").withDeleteFilesAfterCompletion("datacbtyor") + .withModifiedDatetimeStart("datadamyumrobbaxnym").withModifiedDatetimeEnd("datag") + .withFileFilter("dataqmkakgw"); model = BinaryData.fromObject(model).toObject(FileServerReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerWriteSettingsTests.java index 6fbe347adf1bb..566f913954399 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileServerWriteSettingsTests.java @@ -13,16 +13,19 @@ public final class FileServerWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FileServerWriteSettings model = BinaryData.fromString( - "{\"type\":\"FileServerWriteSettings\",\"maxConcurrentConnections\":\"datasqvubwwqgiyu\",\"disableMetricsCollection\":\"datarvwjxmwal\",\"copyBehavior\":\"datajtnsnbpiuvqho\",\"metadata\":[{\"name\":\"datartrsnpbsung\",\"value\":\"datakkmkzfbjucg\"},{\"name\":\"datazjyrdiiwhmrhzq\",\"value\":\"datapjydwm\"}],\"\":{\"pqupdcsvzugiur\":\"dataytjpua\",\"jzscrjtnq\":\"datagqlvl\"}}") + "{\"type\":\"FileServerWriteSettings\",\"maxConcurrentConnections\":\"dataebrvrhwqkfffvgbk\",\"disableMetricsCollection\":\"dataipy\",\"copyBehavior\":\"datahesbebvkmt\",\"metadata\":[{\"name\":\"datalkyvybljqgirpitz\",\"value\":\"datamxcukurkg\"},{\"name\":\"dataxqanrk\",\"value\":\"datadjfsvfbjcnad\"},{\"name\":\"databrntvhppykrlz\",\"value\":\"datalsvxpolatorjm\"},{\"name\":\"databnmuxlthyxryv\",\"value\":\"datazhsigddgbcnqv\"}],\"\":{\"lemzrw\":\"databffcvtij\",\"kmkwddgyqeni\":\"datagvgogczgcm\",\"rtcbvifcrnxst\":\"datarznam\"}}") .toObject(FileServerWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FileServerWriteSettings model = new FileServerWriteSettings().withMaxConcurrentConnections("datasqvubwwqgiyu") - .withDisableMetricsCollection("datarvwjxmwal").withCopyBehavior("datajtnsnbpiuvqho") - .withMetadata(Arrays.asList(new MetadataItem().withName("datartrsnpbsung").withValue("datakkmkzfbjucg"), - new MetadataItem().withName("datazjyrdiiwhmrhzq").withValue("datapjydwm"))); + FileServerWriteSettings model = new FileServerWriteSettings() + .withMaxConcurrentConnections("dataebrvrhwqkfffvgbk").withDisableMetricsCollection("dataipy") + .withCopyBehavior("datahesbebvkmt") + .withMetadata(Arrays.asList(new MetadataItem().withName("datalkyvybljqgirpitz").withValue("datamxcukurkg"), + new MetadataItem().withName("dataxqanrk").withValue("datadjfsvfbjcnad"), + new MetadataItem().withName("databrntvhppykrlz").withValue("datalsvxpolatorjm"), + new MetadataItem().withName("databnmuxlthyxryv").withValue("datazhsigddgbcnqv"))); model = BinaryData.fromObject(model).toObject(FileServerWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSinkTests.java index 8a1e65b4d0374..e7743ebb39725 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSinkTests.java @@ -11,16 +11,16 @@ public final class FileSystemSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FileSystemSink model = BinaryData.fromString( - "{\"type\":\"FileSystemSink\",\"copyBehavior\":\"datassayrwy\",\"writeBatchSize\":\"datautezxrpxdhzwdyva\",\"writeBatchTimeout\":\"datav\",\"sinkRetryCount\":\"datawexwhosc\",\"sinkRetryWait\":\"datapmvc\",\"maxConcurrentConnections\":\"datamgqlshglympe\",\"disableMetricsCollection\":\"datazvcfh\",\"\":{\"twz\":\"dataropxeezlqwbgl\",\"orfpizyb\":\"datatkhyrwdsnpuo\",\"pqatkzghwcywrb\":\"dataj\",\"kf\":\"dataxwls\"}}") + "{\"type\":\"FileSystemSink\",\"copyBehavior\":\"datatlmcaehjhwklf\",\"writeBatchSize\":\"dataqqgyp\",\"writeBatchTimeout\":\"datawejbngojnak\",\"sinkRetryCount\":\"dataytk\",\"sinkRetryWait\":\"datafo\",\"maxConcurrentConnections\":\"datafksormf\",\"disableMetricsCollection\":\"datauhwxmnrdfjobhr\",\"\":{\"pbdfrtasau\":\"dataeaupjmjig\",\"yrxyn\":\"dataxtoxlxojijtt\",\"nyciss\":\"datafs\"}}") .toObject(FileSystemSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FileSystemSink model = new FileSystemSink().withWriteBatchSize("datautezxrpxdhzwdyva") - .withWriteBatchTimeout("datav").withSinkRetryCount("datawexwhosc").withSinkRetryWait("datapmvc") - .withMaxConcurrentConnections("datamgqlshglympe").withDisableMetricsCollection("datazvcfh") - .withCopyBehavior("datassayrwy"); + FileSystemSink model + = new FileSystemSink().withWriteBatchSize("dataqqgyp").withWriteBatchTimeout("datawejbngojnak") + .withSinkRetryCount("dataytk").withSinkRetryWait("datafo").withMaxConcurrentConnections("datafksormf") + .withDisableMetricsCollection("datauhwxmnrdfjobhr").withCopyBehavior("datatlmcaehjhwklf"); model = BinaryData.fromObject(model).toObject(FileSystemSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSourceTests.java index 865edcd472d4f..a9069259d1eb6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FileSystemSourceTests.java @@ -11,15 +11,16 @@ public final class FileSystemSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FileSystemSource model = BinaryData.fromString( - "{\"type\":\"FileSystemSource\",\"recursive\":\"datawjcozb\",\"additionalColumns\":\"datat\",\"sourceRetryCount\":\"datacmxqdexnkp\",\"sourceRetryWait\":\"dataxc\",\"maxConcurrentConnections\":\"datamzykp\",\"disableMetricsCollection\":\"datagiumuztb\",\"\":{\"tozrveehmvrv\":\"datafmcnrgwgc\",\"efdhchkkwahaxy\":\"dataurpzry\",\"cktkbgxteehyh\":\"datadlvbomhfqsjz\",\"jytl\":\"datagnl\"}}") + "{\"type\":\"FileSystemSource\",\"recursive\":\"datascjig\",\"additionalColumns\":\"datakdsvayyhtiy\",\"sourceRetryCount\":\"datahmniz\",\"sourceRetryWait\":\"databtehkytl\",\"maxConcurrentConnections\":\"datamyznwrcfqwkqul\",\"disableMetricsCollection\":\"dataovqohwiw\",\"\":{\"sjjjcd\":\"dataxjxlssosndnypx\",\"xb\":\"datasvgdbfni\",\"jgczpdio\":\"datasjhpm\",\"cwmabehr\":\"datadtjylimzvjwjhmtc\"}}") .toObject(FileSystemSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FileSystemSource model = new FileSystemSource().withSourceRetryCount("datacmxqdexnkp") - .withSourceRetryWait("dataxc").withMaxConcurrentConnections("datamzykp") - .withDisableMetricsCollection("datagiumuztb").withRecursive("datawjcozb").withAdditionalColumns("datat"); + FileSystemSource model + = new FileSystemSource().withSourceRetryCount("datahmniz").withSourceRetryWait("databtehkytl") + .withMaxConcurrentConnections("datamyznwrcfqwkqul").withDisableMetricsCollection("dataovqohwiw") + .withRecursive("datascjig").withAdditionalColumns("datakdsvayyhtiy"); model = BinaryData.fromObject(model).toObject(FileSystemSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTests.java index 0e79ecaeff1d5..985d55ec6a182 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTests.java @@ -21,40 +21,48 @@ public final class FilterActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FilterActivity model = BinaryData.fromString( - "{\"type\":\"Filter\",\"typeProperties\":{\"items\":{\"value\":\"rzdqqo\"},\"condition\":{\"value\":\"dralt\"}},\"name\":\"cttjibognhuqdkq\",\"description\":\"fcvahknvnfp\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"gbijzoix\",\"dependencyConditions\":[\"Completed\",\"Succeeded\"],\"\":{\"asvajbg\":\"datacnknkukem\",\"kzyqizxuj\":\"datauwkstka\",\"oqrutbfkynwwmlzp\":\"datakehtrgybfumo\"}}],\"userProperties\":[{\"name\":\"azkalju\",\"value\":\"datamknwlb\"},{\"name\":\"loaepbfntgsj\",\"value\":\"datangueggphwgixypg\"},{\"name\":\"wm\",\"value\":\"dataa\"}],\"\":{\"yjblafvvndkvbcrq\":\"dataiuaaqgkvkoynjuc\",\"eohx\":\"datanbgymgj\",\"dskq\":\"dataiszbhgbfsj\",\"rrchfu\":\"datajsdxgefkvrfihet\"}}") + "{\"type\":\"Filter\",\"typeProperties\":{\"items\":{\"value\":\"nchyoimt\"},\"condition\":{\"value\":\"kjcdjswxek\"}},\"name\":\"hvccxuntghwcb\",\"description\":\"lgbyfcbc\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"zeukumlnfxboqvg\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Skipped\",\"Succeeded\"],\"\":{\"sznxz\":\"datacmuiqir\",\"akrbew\":\"datazbnqmxirspj\",\"g\":\"datazisdnbourw\",\"fecoufnxt\":\"datasdluquyxgmzyqftl\"}},{\"activity\":\"nusqza\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"kwbwxcjf\":\"datay\",\"weguqzlmhpuqlsd\":\"datauzw\"}},{\"activity\":\"tejxlzyyylyx\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Succeeded\",\"Failed\"],\"\":{\"hogjaubpl\":\"dataychob\",\"ndi\":\"datazjglfrwymwujt\",\"tyqbtijyb\":\"datawycwplj\"}}],\"userProperties\":[{\"name\":\"gclppwdfxhz\",\"value\":\"datarsrgbfaq\"}],\"\":{\"burrevuz\":\"datakisipjgvm\",\"ffgconiydgnxs\":\"dataxuubwjopkldubqfb\"}}") .toObject(FilterActivity.class); - Assertions.assertEquals("cttjibognhuqdkq", model.name()); - Assertions.assertEquals("fcvahknvnfp", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("gbijzoix", model.dependsOn().get(0).activity()); + Assertions.assertEquals("hvccxuntghwcb", model.name()); + Assertions.assertEquals("lgbyfcbc", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("zeukumlnfxboqvg", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("azkalju", model.userProperties().get(0).name()); - Assertions.assertEquals("rzdqqo", model.items().value()); - Assertions.assertEquals("dralt", model.condition().value()); + Assertions.assertEquals("gclppwdfxhz", model.userProperties().get(0).name()); + Assertions.assertEquals("nchyoimt", model.items().value()); + Assertions.assertEquals("kjcdjswxek", model.condition().value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FilterActivity model = new FilterActivity().withName("cttjibognhuqdkq").withDescription("fcvahknvnfp") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("gbijzoix") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("azkalju").withValue("datamknwlb"), - new UserProperty().withName("loaepbfntgsj").withValue("datangueggphwgixypg"), - new UserProperty().withName("wm").withValue("dataa"))) - .withItems(new Expression().withValue("rzdqqo")).withCondition(new Expression().withValue("dralt")); + FilterActivity model = new FilterActivity().withName("hvccxuntghwcb").withDescription("lgbyfcbc") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("zeukumlnfxboqvg") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("nusqza") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("tejxlzyyylyx") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("gclppwdfxhz").withValue("datarsrgbfaq"))) + .withItems(new Expression().withValue("nchyoimt")).withCondition(new Expression().withValue("kjcdjswxek")); model = BinaryData.fromObject(model).toObject(FilterActivity.class); - Assertions.assertEquals("cttjibognhuqdkq", model.name()); - Assertions.assertEquals("fcvahknvnfp", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("gbijzoix", model.dependsOn().get(0).activity()); + Assertions.assertEquals("hvccxuntghwcb", model.name()); + Assertions.assertEquals("lgbyfcbc", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("zeukumlnfxboqvg", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("azkalju", model.userProperties().get(0).name()); - Assertions.assertEquals("rzdqqo", model.items().value()); - Assertions.assertEquals("dralt", model.condition().value()); + Assertions.assertEquals("gclppwdfxhz", model.userProperties().get(0).name()); + Assertions.assertEquals("nchyoimt", model.items().value()); + Assertions.assertEquals("kjcdjswxek", model.condition().value()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTypePropertiesTests.java index 12525d3992b19..e4d9179fe217e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FilterActivityTypePropertiesTests.java @@ -13,18 +13,18 @@ public final class FilterActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FilterActivityTypeProperties model - = BinaryData.fromString("{\"items\":{\"value\":\"frqagpjoci\"},\"condition\":{\"value\":\"nnd\"}}") + = BinaryData.fromString("{\"items\":{\"value\":\"yytnmhlank\"},\"condition\":{\"value\":\"s\"}}") .toObject(FilterActivityTypeProperties.class); - Assertions.assertEquals("frqagpjoci", model.items().value()); - Assertions.assertEquals("nnd", model.condition().value()); + Assertions.assertEquals("yytnmhlank", model.items().value()); + Assertions.assertEquals("s", model.condition().value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { FilterActivityTypeProperties model = new FilterActivityTypeProperties() - .withItems(new Expression().withValue("frqagpjoci")).withCondition(new Expression().withValue("nnd")); + .withItems(new Expression().withValue("yytnmhlank")).withCondition(new Expression().withValue("s")); model = BinaryData.fromObject(model).toObject(FilterActivityTypeProperties.class); - Assertions.assertEquals("frqagpjoci", model.items().value()); - Assertions.assertEquals("nnd", model.condition().value()); + Assertions.assertEquals("yytnmhlank", model.items().value()); + Assertions.assertEquals("s", model.condition().value()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTests.java index 38e44397dbd6a..6edc53e985253 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTests.java @@ -22,115 +22,94 @@ public final class ForEachActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ForEachActivity model = BinaryData.fromString( - "{\"type\":\"ForEach\",\"typeProperties\":{\"isSequential\":true,\"batchCount\":100226368,\"items\":{\"value\":\"eobkmx\"},\"activities\":[{\"type\":\"Activity\",\"name\":\"mrbjhyldx\",\"description\":\"ce\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"daq\",\"dependencyConditions\":[\"Skipped\",\"Failed\"],\"\":{\"iugca\":\"datayymb\",\"uvu\":\"datashgryof\",\"uocmxtyjaxk\":\"datakrkibno\",\"vasnmzsvd\":\"datayovwtpm\"}},{\"activity\":\"ryzxhtvythp\",\"dependencyConditions\":[\"Completed\",\"Completed\"],\"\":{\"sixterpbjkhtmm\":\"dataqvyzacjxczj\",\"ncscaynhzmz\":\"datamezlhmtrq\"}},{\"activity\":\"wxwwpi\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"lpphotbs\":\"datauq\",\"qotwcfbqzezchp\":\"datakliuctxfzhv\",\"x\":\"datahuhyxxftrfw\",\"aule\":\"datajcsxqkmo\"}}],\"userProperties\":[{\"name\":\"xznopk\",\"value\":\"dataoffeutvqgnugiiyc\"},{\"name\":\"jf\",\"value\":\"datakntdynbrf\"}],\"\":{\"gilzrruvnlubk\":\"dataabrqdbx\"}},{\"type\":\"Activity\",\"name\":\"gfzii\",\"description\":\"xntuzgceuzhpcmnp\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"vxpfayophpu\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"xufrwiivekrgvzjt\":\"datahbqvbute\"}},{\"activity\":\"jrrkdlw\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\"],\"\":{\"tjnktheh\":\"datao\",\"v\":\"datamijraei\",\"lbnroxgwqgbv\":\"datahhci\",\"bztwkz\":\"datactcbmnecozvx\"}},{\"activity\":\"puwjvju\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Completed\",\"Skipped\"],\"\":{\"jxjnxznlx\":\"datauxtyvpvegxdzopfk\",\"b\":\"datajkteubntqvlktq\",\"jiktwfjyl\":\"dataurblbtvsxnaothlr\",\"nwegyhzucpixfdbi\":\"datammibaowclb\"}}],\"userProperties\":[{\"name\":\"hbcbdpyorhqb\",\"value\":\"datafvhnhyxcws\"},{\"name\":\"hszmuvare\",\"value\":\"datakcxdamnmnm\"}],\"\":{\"pcrs\":\"datapieleruoyf\",\"qonvjur\":\"dataqwqmlis\"}}]},\"name\":\"czdelqazb\",\"description\":\"xg\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"paeceuvqacaedvnl\",\"dependencyConditions\":[\"Failed\"],\"\":{\"qxldkhgngy\":\"datajgdbobr\",\"tfwncxkaz\":\"datafesjksmyeegbe\",\"qvjk\":\"datayd\"}},{\"activity\":\"zorddcwg\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"cyepqtdvgdujkdac\":\"datai\",\"y\":\"datavyeckbudepulbxgd\",\"wmehaic\":\"dataywmezoi\",\"v\":\"datakkcpkvujwf\"}},{\"activity\":\"vvnbbeys\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Failed\"],\"\":{\"euqgpt\":\"datafjxfi\"}},{\"activity\":\"xilwnvfb\",\"dependencyConditions\":[\"Failed\",\"Succeeded\"],\"\":{\"ekjhu\":\"dataaz\",\"idk\":\"datawiitxye\"}}],\"userProperties\":[{\"name\":\"hkcutxmqvbh\",\"value\":\"databwdu\"},{\"name\":\"vkrskqgokhpzvph\",\"value\":\"datak\"},{\"name\":\"fcxvfurkdhopz\",\"value\":\"datamhrfwch\"}],\"\":{\"zkwdexldocq\":\"dataeovji\",\"chltxayqwfu\":\"datalbsvyokiexmfe\",\"e\":\"datavo\"}}") + "{\"type\":\"ForEach\",\"typeProperties\":{\"isSequential\":false,\"batchCount\":2055643232,\"items\":{\"value\":\"bgzyafazwieiz\"},\"activities\":[{\"type\":\"Activity\",\"name\":\"jjdboxuinrsrrij\",\"description\":\"nthtqtbcwtcqj\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"tzbvdz\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Completed\",\"Skipped\"],\"\":{\"gwpyljnqiprjahgq\":\"datajclrtssbkz\",\"ufnumf\":\"databyic\",\"ajpp\":\"dataahnruqhmur\",\"zo\":\"dataflvazp\"}},{\"activity\":\"sqmli\",\"dependencyConditions\":[\"Failed\"],\"\":{\"xnts\":\"datallpobz\",\"wwhml\":\"datayntkfziitbwth\",\"bfg\":\"datas\",\"nzuufpdwk\":\"datajkkra\"}}],\"userProperties\":[{\"name\":\"phwxdwlowymeqiqn\",\"value\":\"dataca\"},{\"name\":\"mxuoxk\",\"value\":\"datapleooom\"}],\"\":{\"gaofobjl\":\"datajfldzvgogqu\",\"exbjbknpzhfh\":\"datanaxfvsyustrb\",\"sl\":\"databhgw\"}},{\"type\":\"Activity\",\"name\":\"qb\",\"description\":\"cjbxochijwpsk\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"ikwahbzdgwki\",\"dependencyConditions\":[\"Completed\"],\"\":{\"xcrxqpen\":\"datarvj\"}},{\"activity\":\"ujxdnia\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Failed\",\"Failed\"],\"\":{\"duwxqytppjdyikdy\":\"datahwgi\",\"utrpdgmukm\":\"dataxhxrkdtuc\",\"xaednczvnwyfzav\":\"datacvftijlshlcrjyne\"}},{\"activity\":\"ajbah\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Succeeded\"],\"\":{\"mqeeodfpl\":\"datajtywluxy\",\"ofxgwyvjef\":\"datafsmpbwwphjwqmc\",\"xjrttzhn\":\"datalxqmtedzxujx\"}},{\"activity\":\"mbjqynwqcovpjv\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Succeeded\"],\"\":{\"k\":\"databdd\"}}],\"userProperties\":[{\"name\":\"rcmayqasdve\",\"value\":\"dataldafx\"},{\"name\":\"pyvlfujsbcfogu\",\"value\":\"databqcqnchdzyju\"},{\"name\":\"dknblbrixvcp\",\"value\":\"datasvprumttrvkhu\"}],\"\":{\"egxgymxplrtue\":\"dataxxwbjbanlmpm\",\"nj\":\"dataqhqu\",\"ebpvhdk\":\"dataybgpjyuvjuowk\",\"ttjmdtfuwx\":\"datadqcgedipnnzmvt\"}}]},\"name\":\"ee\",\"description\":\"mies\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"btumttmixewparb\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Failed\"],\"\":{\"tixggnld\":\"datavqnkwjhj\",\"uklepetsxe\":\"datalgqsoiuncmuv\",\"xhkzcdni\":\"dataneherqbelmsx\"}}],\"userProperties\":[{\"name\":\"d\",\"value\":\"datavsvgydtdto\"}],\"\":{\"pooaskflrqwfmbk\":\"datazotmiizk\",\"bwudiyfixpwrrqiv\":\"datashbrzvnouthbvv\",\"lozg\":\"datazqcmrxh\",\"yttxspaafs\":\"datafhijcetcystrs\"}}") .toObject(ForEachActivity.class); - Assertions.assertEquals("czdelqazb", model.name()); - Assertions.assertEquals("xg", model.description()); + Assertions.assertEquals("ee", model.name()); + Assertions.assertEquals("mies", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("paeceuvqacaedvnl", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("hkcutxmqvbh", model.userProperties().get(0).name()); - Assertions.assertEquals(true, model.isSequential()); - Assertions.assertEquals(100226368, model.batchCount()); - Assertions.assertEquals("eobkmx", model.items().value()); - Assertions.assertEquals("mrbjhyldx", model.activities().get(0).name()); - Assertions.assertEquals("ce", model.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("btumttmixewparb", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("d", model.userProperties().get(0).name()); + Assertions.assertEquals(false, model.isSequential()); + Assertions.assertEquals(2055643232, model.batchCount()); + Assertions.assertEquals("bgzyafazwieiz", model.items().value()); + Assertions.assertEquals("jjdboxuinrsrrij", model.activities().get(0).name()); + Assertions.assertEquals("nthtqtbcwtcqj", model.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("daq", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, + Assertions.assertEquals("tzbvdz", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("xznopk", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("phwxdwlowymeqiqn", model.activities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ForEachActivity model = new ForEachActivity().withName("czdelqazb").withDescription("xg") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("paeceuvqacaedvnl") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("zorddcwg") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("vvnbbeys") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, - DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("xilwnvfb") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("hkcutxmqvbh").withValue("databwdu"), - new UserProperty().withName("vkrskqgokhpzvph").withValue("datak"), - new UserProperty().withName("fcxvfurkdhopz").withValue("datamhrfwch"))) - .withIsSequential( - true) - .withBatchCount( - 100226368) - .withItems( - new Expression().withValue("eobkmx")) - .withActivities( - Arrays.asList( - new Activity() - .withName("mrbjhyldx").withDescription("ce").withState( - ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("daq") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ryzxhtvythp") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("wxwwpi") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("xznopk").withValue("dataoffeutvqgnugiiyc"), new UserProperty() - .withName("jf").withValue("datakntdynbrf"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("gfzii").withDescription("xntuzgceuzhpcmnp").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs( - ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("vxpfayophpu") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jrrkdlw") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("puwjvju") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("hbcbdpyorhqb").withValue("datafvhnhyxcws"), - new UserProperty().withName("hszmuvare").withValue("datakcxdamnmnm"))) - .withAdditionalProperties(mapOf("type", "Activity")))); + ForEachActivity model = new ForEachActivity().withName("ee").withDescription("mies") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("btumttmixewparb") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, + DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("d").withValue("datavsvgydtdto"))) + .withIsSequential(false).withBatchCount(2055643232).withItems(new Expression().withValue("bgzyafazwieiz")) + .withActivities(Arrays.asList( + new Activity().withName("jjdboxuinrsrrij").withDescription("nthtqtbcwtcqj") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("tzbvdz") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.FAILED, DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("sqmli") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("phwxdwlowymeqiqn").withValue("dataca"), + new UserProperty().withName("mxuoxk").withValue("datapleooom"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("qb").withDescription("cjbxochijwpsk").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("ikwahbzdgwki") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("ujxdnia") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.FAILED, DependencyCondition.FAILED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("ajbah") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("mbjqynwqcovpjv") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("rcmayqasdve").withValue("dataldafx"), + new UserProperty().withName("pyvlfujsbcfogu").withValue("databqcqnchdzyju"), + new UserProperty().withName("dknblbrixvcp").withValue("datasvprumttrvkhu"))) + .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(ForEachActivity.class); - Assertions.assertEquals("czdelqazb", model.name()); - Assertions.assertEquals("xg", model.description()); + Assertions.assertEquals("ee", model.name()); + Assertions.assertEquals("mies", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("paeceuvqacaedvnl", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("hkcutxmqvbh", model.userProperties().get(0).name()); - Assertions.assertEquals(true, model.isSequential()); - Assertions.assertEquals(100226368, model.batchCount()); - Assertions.assertEquals("eobkmx", model.items().value()); - Assertions.assertEquals("mrbjhyldx", model.activities().get(0).name()); - Assertions.assertEquals("ce", model.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("btumttmixewparb", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("d", model.userProperties().get(0).name()); + Assertions.assertEquals(false, model.isSequential()); + Assertions.assertEquals(2055643232, model.batchCount()); + Assertions.assertEquals("bgzyafazwieiz", model.items().value()); + Assertions.assertEquals("jjdboxuinrsrrij", model.activities().get(0).name()); + Assertions.assertEquals("nthtqtbcwtcqj", model.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("daq", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, + Assertions.assertEquals("tzbvdz", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("xznopk", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("phwxdwlowymeqiqn", model.activities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTypePropertiesTests.java index 40e3ea018e82f..67af99c5cc0bc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ForEachActivityTypePropertiesTests.java @@ -22,126 +22,115 @@ public final class ForEachActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ForEachActivityTypeProperties model = BinaryData.fromString( - "{\"isSequential\":false,\"batchCount\":257323877,\"items\":{\"value\":\"zif\"},\"activities\":[{\"type\":\"Activity\",\"name\":\"bmeksegdjq\",\"description\":\"ch\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"imenjhtwkn\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Succeeded\"],\"\":{\"lp\":\"datayvno\",\"qkuxajl\":\"datalmjbssxqu\"}},{\"activity\":\"iffzpkrno\",\"dependencyConditions\":[\"Failed\",\"Completed\"],\"\":{\"w\":\"datadir\",\"irxngmmv\":\"datanmaiqdj\"}},{\"activity\":\"rxoidmnsmd\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"fi\":\"datalhkcogxrsb\",\"n\":\"datachfxmedybj\",\"mpzbxqfm\":\"dataoaeudhvszw\",\"djushzfn\":\"dataypwglkvsp\"}}],\"userProperties\":[{\"name\":\"rdsmrvp\",\"value\":\"datawewgda\"},{\"name\":\"hzdhszk\",\"value\":\"datadlilkw\"}],\"\":{\"fxnokpkgrub\":\"datayvdabgctmfntl\",\"hdkx\":\"datazgz\",\"rhgelsvo\":\"datahlinjerkdurch\"}},{\"type\":\"Activity\",\"name\":\"vqjthluo\",\"description\":\"qhq\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"adwsentrcdzyv\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Failed\",\"Succeeded\"],\"\":{\"uvxmrbbgl\":\"datauwpqvd\"}},{\"activity\":\"wfbgkyonadtywzrn\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Completed\"],\"\":{\"cfprioabqxwid\":\"dataptxmdadfygj\"}},{\"activity\":\"xeonnolrsmxt\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Succeeded\"],\"\":{\"uotexlpqydgfzet\":\"dataaxmo\"}},{\"activity\":\"armnseigoalxwuqu\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Skipped\",\"Completed\"],\"\":{\"pqzz\":\"datag\",\"mlxog\":\"dataxhgh\"}}],\"userProperties\":[{\"name\":\"hxyx\",\"value\":\"datalxawixdcy\"},{\"name\":\"dqamiy\",\"value\":\"datab\"},{\"name\":\"zlbcamdzoauvwjkg\",\"value\":\"datazco\"},{\"name\":\"wcnnzacqludq\",\"value\":\"dataaqxztywzaq\"}],\"\":{\"lzpowse\":\"datatstmyfebbt\"}},{\"type\":\"Activity\",\"name\":\"pgkwtpzbsytwthv\",\"description\":\"tsvgyzmafqsn\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"vwej\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"hrxoekyf\":\"datazq\",\"nsmh\":\"datavxcompd\",\"xjaaocjl\":\"datapzbyudkoa\"}},{\"activity\":\"cou\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Failed\",\"Failed\"],\"\":{\"iukvmzxr\":\"datamkzb\",\"jbklt\":\"dataveyngzj\",\"cqhlfqimjlde\":\"datahguvqghueh\",\"ltvvpudhtdapkdah\":\"datauqqnforujfluom\"}},{\"activity\":\"nxtixrkjo\",\"dependencyConditions\":[\"Failed\",\"Failed\"],\"\":{\"xuibyfylh\":\"dataeicl\",\"uqylmlunquvl\":\"datawqp\"}},{\"activity\":\"al\",\"dependencyConditions\":[\"Failed\"],\"\":{\"gynqedn\":\"datalx\",\"qcxzdlfs\":\"dataidacskul\"}}],\"userProperties\":[{\"name\":\"jvsgrcrknn\",\"value\":\"datauceuwfmrckatnjik\"},{\"name\":\"zhtovs\",\"value\":\"dataieoth\"},{\"name\":\"wokprvpkdkds\",\"value\":\"datasmavtn\"},{\"name\":\"gfmtximnpcghcfud\",\"value\":\"dataqefdtpurgerybdi\"}],\"\":{\"xcdwlkkglahdwxyi\":\"dataeahwerukuoey\"}},{\"type\":\"Activity\",\"name\":\"ezfoekaxhjvtfzaq\",\"description\":\"qgfyofoh\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"rhgwgsba\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"zsirhp\":\"datakuzifsguolfkupmw\"}},{\"activity\":\"gqdz\",\"dependencyConditions\":[\"Completed\"],\"\":{\"uddbzxidqqesl\":\"datayywbssliphc\",\"utrlzzztg\":\"dataaoxke\",\"zuvsjblqmddtp\":\"datafzyxamyjhp\",\"joboqts\":\"dataily\"}},{\"activity\":\"d\",\"dependencyConditions\":[\"Failed\"],\"\":{\"fmtm\":\"datacuv\",\"gtskolbjylostrc\":\"datapvoazgtlxgtusw\"}}],\"userProperties\":[{\"name\":\"ce\",\"value\":\"databwaiqs\"}],\"\":{\"anhe\":\"dataigelphauldalsp\",\"hv\":\"datawxllqyvblfprskxh\",\"ipibnj\":\"dataviycjulunbtufi\",\"oabma\":\"dataivoizxk\"}}]}") + "{\"isSequential\":false,\"batchCount\":614609889,\"items\":{\"value\":\"erlrqtqnxhu\"},\"activities\":[{\"type\":\"Activity\",\"name\":\"tqveumwbmqpbfjbs\",\"description\":\"jqkykjzbxmgsxb\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"oqfeobkmxohmr\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Failed\"],\"\":{\"kfxdaqwnkjyfyy\":\"datanwcejcz\",\"gryo\":\"databtiugcaas\",\"onuocmxt\":\"datahuvuokrkib\"}},{\"activity\":\"jaxkby\",\"dependencyConditions\":[\"Completed\"],\"\":{\"ryzxhtvythp\":\"datamyvasnmzsvd\",\"yzacjxczjosixter\":\"datanlmfvq\"}}],\"userProperties\":[{\"name\":\"khtmmkm\",\"value\":\"datazlh\"},{\"name\":\"trqhncscaynh\",\"value\":\"datamziwxwwpi\"},{\"name\":\"wl\",\"value\":\"dataluqqlpphotbsgkl\"},{\"name\":\"uct\",\"value\":\"datafzhvxqotwcfbq\"}],\"\":{\"huhyxxftrfw\":\"datachp\",\"jcsxqkmo\":\"datax\"}},{\"type\":\"Activity\",\"name\":\"aule\",\"description\":\"dxzno\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"utvqgnugiiy\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"crabrqdbxhg\":\"datakntdynbrf\",\"ruvnlubk\":\"datalz\",\"yxntuzgceuzhp\":\"datagfzii\"}},{\"activity\":\"mnpodsqil\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Completed\"],\"\":{\"qvbutesxufrwiiv\":\"dataophpudccaqh\",\"ozccdoat\":\"datakrgvzjtvjrrkdlw\",\"mijraei\":\"datanktheh\",\"hhci\":\"datav\"}},{\"activity\":\"lbnroxgwqgbv\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Skipped\"],\"\":{\"twkzfp\":\"dataecozvxdb\",\"uxtyvpvegxdzopfk\":\"datawjvjuixbtkuv\",\"jkteubntqvlktq\":\"datajxjnxznlx\",\"urblbtvsxnaothlr\":\"datab\"}},{\"activity\":\"jiktwfjyl\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Skipped\"],\"\":{\"ixfdbicpchbc\":\"datawclbznwegyhzuc\",\"hyx\":\"datadpyorhqbpfvh\"}}],\"userProperties\":[{\"name\":\"nhszmuvareakcxda\",\"value\":\"datanmnmqydpieleruoy\"},{\"name\":\"npcrsfqwqm\",\"value\":\"dataisjqo\"},{\"name\":\"vjurjczdelqaz\",\"value\":\"datazixgqrk\"}],\"\":{\"nloqjmojgdb\":\"datawpaeceuvqacaed\",\"jksmyeegbertf\":\"databrrqxldkhgngyofe\",\"rd\":\"datancxkazmydsqvjkfz\",\"bw\":\"datacwgcmmv\"}},{\"type\":\"Activity\",\"name\":\"i\",\"description\":\"y\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"dujkdacuvyeck\",\"dependencyConditions\":[\"Skipped\",\"Completed\"],\"\":{\"dfy\":\"datalbx\",\"wmehaic\":\"dataywmezoi\",\"v\":\"datakkcpkvujwf\",\"fsiiadfjxfiv\":\"datavvnbbeys\"}},{\"activity\":\"uq\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Failed\"],\"\":{\"zl\":\"datawnvfbrgtoqk\",\"wiitxye\":\"datakjhu\",\"ehhkcutxmqvbh\":\"dataidk\"}}],\"userProperties\":[{\"name\":\"duu\",\"value\":\"datakrskqgokhpzvph\"}],\"\":{\"mhrfwch\":\"datafcxvfurkdhopz\"}},{\"type\":\"Activity\",\"name\":\"mgeovjiezk\",\"description\":\"exldocqhl\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"exmfeechl\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Completed\",\"Completed\"],\"\":{\"bmeksegdjq\":\"dataavofeouucgzifo\"}},{\"activity\":\"oc\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Succeeded\",\"Skipped\"],\"\":{\"zcwjaqyvnol\":\"dataimenjhtwkn\",\"m\":\"datapz\",\"quiqkuxajl\":\"databss\"}},{\"activity\":\"iffzpkrno\",\"dependencyConditions\":[\"Failed\",\"Completed\"],\"\":{\"w\":\"datadir\",\"irxngmmv\":\"datanmaiqdj\"}},{\"activity\":\"rxoidmnsmd\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"fi\":\"datalhkcogxrsb\",\"n\":\"datachfxmedybj\",\"mpzbxqfm\":\"dataoaeudhvszw\",\"djushzfn\":\"dataypwglkvsp\"}}],\"userProperties\":[{\"name\":\"rdsmrvp\",\"value\":\"datawewgda\"},{\"name\":\"hzdhszk\",\"value\":\"datadlilkw\"}],\"\":{\"fxnokpkgrub\":\"datayvdabgctmfntl\",\"hdkx\":\"datazgz\",\"rhgelsvo\":\"datahlinjerkdurch\"}}]}") .toObject(ForEachActivityTypeProperties.class); Assertions.assertEquals(false, model.isSequential()); - Assertions.assertEquals(257323877, model.batchCount()); - Assertions.assertEquals("zif", model.items().value()); - Assertions.assertEquals("bmeksegdjq", model.activities().get(0).name()); - Assertions.assertEquals("ch", model.activities().get(0).description()); + Assertions.assertEquals(614609889, model.batchCount()); + Assertions.assertEquals("erlrqtqnxhu", model.items().value()); + Assertions.assertEquals("tqveumwbmqpbfjbs", model.activities().get(0).name()); + Assertions.assertEquals("jqkykjzbxmgsxb", model.activities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("imenjhtwkn", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, + Assertions.assertEquals("oqfeobkmxohmr", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("rdsmrvp", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("khtmmkm", model.activities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ForEachActivityTypeProperties model - = new ForEachActivityTypeProperties().withIsSequential(false).withBatchCount(257323877) - .withItems( - new Expression().withValue("zif")) - .withActivities(Arrays.asList( - new Activity().withName("bmeksegdjq").withDescription("ch").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("imenjhtwkn") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("iffzpkrno") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("rxoidmnsmd") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("rdsmrvp").withValue("datawewgda"), - new UserProperty().withName("hzdhszk").withValue("datadlilkw"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("vqjthluo").withDescription("qhq").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("adwsentrcdzyv") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.FAILED, DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) + ForEachActivityTypeProperties model = new ForEachActivityTypeProperties().withIsSequential(false) + .withBatchCount(614609889).withItems(new Expression().withValue("erlrqtqnxhu")) + .withActivities(Arrays.asList( + new Activity().withName("tqveumwbmqpbfjbs").withDescription("jqkykjzbxmgsxb") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("oqfeobkmxohmr") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.FAILED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("jaxkby") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("khtmmkm").withValue("datazlh"), + new UserProperty().withName("trqhncscaynh").withValue("datamziwxwwpi"), new UserProperty() + .withName("wl").withValue("dataluqqlpphotbsgkl"), + new UserProperty().withName("uct").withValue("datafzhvxqotwcfbq"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("aule").withDescription("dxzno").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("utvqgnugiiy") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("mnpodsqil") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("lbnroxgwqgbv") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("jiktwfjyl") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList( + new UserProperty().withName("nhszmuvareakcxda").withValue("datanmnmqydpieleruoy"), + new UserProperty().withName("npcrsfqwqm").withValue("dataisjqo"), + new UserProperty().withName("vjurjczdelqaz").withValue("datazixgqrk"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("i").withDescription("y").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("dujkdacuvyeck") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("wfbgkyonadtywzrn") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + new ActivityDependency().withActivity("uq") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("duu").withValue("datakrskqgokhpzvph"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity() + .withName("mgeovjiezk").withDescription("exldocqhl").withState( + ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("exmfeechl") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("xeonnolrsmxt") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("armnseigoalxwuqu") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED, - DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("hxyx").withValue("datalxawixdcy"), - new UserProperty().withName("dqamiy").withValue("datab"), - new UserProperty().withName("zlbcamdzoauvwjkg") - .withValue("datazco"), - new UserProperty().withName("wcnnzacqludq").withValue("dataaqxztywzaq"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("pgkwtpzbsytwthv").withDescription("tsvgyzmafqsn") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs( - ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("vwej") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("cou") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.FAILED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("nxtixrkjo") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("al") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("jvsgrcrknn").withValue("datauceuwfmrckatnjik"), - new UserProperty().withName("zhtovs").withValue("dataieoth"), - new UserProperty().withName("wokprvpkdkds") - .withValue("datasmavtn"), - new UserProperty().withName("gfmtximnpcghcfud").withValue("dataqefdtpurgerybdi"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("ezfoekaxhjvtfzaq").withDescription("qgfyofoh") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs( - ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("rhgwgsba") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("gqdz") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("d") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("ce").withValue("databwaiqs"))) - .withAdditionalProperties(mapOf("type", "Activity")))); + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("oc") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("iffzpkrno") + .withDependencyConditions( + Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("rxoidmnsmd") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("rdsmrvp").withValue("datawewgda"), + new UserProperty().withName("hzdhszk").withValue("datadlilkw"))) + .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(ForEachActivityTypeProperties.class); Assertions.assertEquals(false, model.isSequential()); - Assertions.assertEquals(257323877, model.batchCount()); - Assertions.assertEquals("zif", model.items().value()); - Assertions.assertEquals("bmeksegdjq", model.activities().get(0).name()); - Assertions.assertEquals("ch", model.activities().get(0).description()); + Assertions.assertEquals(614609889, model.batchCount()); + Assertions.assertEquals("erlrqtqnxhu", model.items().value()); + Assertions.assertEquals("tqveumwbmqpbfjbs", model.activities().get(0).name()); + Assertions.assertEquals("jqkykjzbxmgsxb", model.activities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("imenjhtwkn", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, + Assertions.assertEquals("oqfeobkmxohmr", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("rdsmrvp", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("khtmmkm", model.activities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatReadSettingsTests.java index 24c70f8e7983a..084ddd494970b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatReadSettingsTests.java @@ -12,8 +12,8 @@ public final class FormatReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - FormatReadSettings model = BinaryData - .fromString("{\"type\":\"FormatReadSettings\",\"\":{\"oebgkx\":\"datady\",\"rvbcgsaaday\":\"datapkl\"}}") + FormatReadSettings model = BinaryData.fromString( + "{\"type\":\"FormatReadSettings\",\"\":{\"hvcrjqzbmyftzbx\":\"datavcjdrmknk\",\"sbzmixwaxtnk\":\"datagosrbullqnfzsegu\"}}") .toObject(FormatReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatWriteSettingsTests.java index af25cfbe32f2e..4bfb078e0a525 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FormatWriteSettingsTests.java @@ -13,7 +13,7 @@ public final class FormatWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FormatWriteSettings model = BinaryData.fromString( - "{\"type\":\"FormatWriteSettings\",\"\":{\"mszobtneltnbyvbg\":\"datapbpqe\",\"azfvx\":\"datadrumududwecdsyb\",\"eqly\":\"datakwv\"}}") + "{\"type\":\"FormatWriteSettings\",\"\":{\"oksstaljiqlxjjl\":\"datagqsqvf\",\"qhscaand\":\"datauymna\",\"lsb\":\"datalvccuvcva\"}}") .toObject(FormatWriteSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FtpReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FtpReadSettingsTests.java index 1ecc647fdba67..b14ae7e2af3d9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FtpReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/FtpReadSettingsTests.java @@ -11,18 +11,18 @@ public final class FtpReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { FtpReadSettings model = BinaryData.fromString( - "{\"type\":\"FtpReadSettings\",\"recursive\":\"datazcluyosigk\",\"wildcardFolderPath\":\"dataykjxqsspeqgedp\",\"wildcardFileName\":\"datajqpjzturdi\",\"enablePartitionDiscovery\":\"datarkwmafyxoqak\",\"partitionRootPath\":\"datat\",\"deleteFilesAfterCompletion\":\"dataetxokqudjdwcwja\",\"fileListPath\":\"datab\",\"useBinaryTransfer\":\"dataehxahnqjbav\",\"disableChunking\":\"datalfefbbv\",\"maxConcurrentConnections\":\"datalnnpafufwrerbnd\",\"disableMetricsCollection\":\"datazfnstlavmdc\",\"\":{\"ajyitpyzgwihk\":\"datamv\",\"vjyfdwaupjozgryo\":\"datawurzaqubryhv\",\"hilyz\":\"datagwk\"}}") + "{\"type\":\"FtpReadSettings\",\"recursive\":\"datadqgy\",\"wildcardFolderPath\":\"dataulzguvckpdp\",\"wildcardFileName\":\"datanrjqskikqd\",\"enablePartitionDiscovery\":\"dataybqtlvofjjsetiz\",\"partitionRootPath\":\"datanadn\",\"deleteFilesAfterCompletion\":\"datasbpxlserqgxnh\",\"fileListPath\":\"dataccd\",\"useBinaryTransfer\":\"dataxybn\",\"disableChunking\":\"datahmpmeglolpot\",\"maxConcurrentConnections\":\"datamb\",\"disableMetricsCollection\":\"dataqjrytymfnojjh\",\"\":{\"y\":\"datathjqgovvivlr\",\"wyt\":\"datarafet\",\"luolgspyqsapnh\":\"datavpiilgy\"}}") .toObject(FtpReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FtpReadSettings model = new FtpReadSettings().withMaxConcurrentConnections("datalnnpafufwrerbnd") - .withDisableMetricsCollection("datazfnstlavmdc").withRecursive("datazcluyosigk") - .withWildcardFolderPath("dataykjxqsspeqgedp").withWildcardFileName("datajqpjzturdi") - .withEnablePartitionDiscovery("datarkwmafyxoqak").withPartitionRootPath("datat") - .withDeleteFilesAfterCompletion("dataetxokqudjdwcwja").withFileListPath("datab") - .withUseBinaryTransfer("dataehxahnqjbav").withDisableChunking("datalfefbbv"); + FtpReadSettings model = new FtpReadSettings().withMaxConcurrentConnections("datamb") + .withDisableMetricsCollection("dataqjrytymfnojjh").withRecursive("datadqgy") + .withWildcardFolderPath("dataulzguvckpdp").withWildcardFileName("datanrjqskikqd") + .withEnablePartitionDiscovery("dataybqtlvofjjsetiz").withPartitionRootPath("datanadn") + .withDeleteFilesAfterCompletion("datasbpxlserqgxnh").withFileListPath("dataccd") + .withUseBinaryTransfer("dataxybn").withDisableChunking("datahmpmeglolpot"); model = BinaryData.fromObject(model).toObject(FtpReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GenericDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GenericDatasetTypePropertiesTests.java index 86da0acc9e0cf..d421b52f45ef0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GenericDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GenericDatasetTypePropertiesTests.java @@ -10,13 +10,13 @@ public final class GenericDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GenericDatasetTypeProperties model - = BinaryData.fromString("{\"tableName\":\"dataypkcpwsrqnn\"}").toObject(GenericDatasetTypeProperties.class); + GenericDatasetTypeProperties model = BinaryData.fromString("{\"tableName\":\"datafzvlqquyhbce\"}") + .toObject(GenericDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GenericDatasetTypeProperties model = new GenericDatasetTypeProperties().withTableName("dataypkcpwsrqnn"); + GenericDatasetTypeProperties model = new GenericDatasetTypeProperties().withTableName("datafzvlqquyhbce"); model = BinaryData.fromObject(model).toObject(GenericDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTests.java index 5c57b3ce5193b..02c4de75ece18 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTests.java @@ -25,58 +25,54 @@ public final class GetMetadataActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GetMetadataActivity model = BinaryData.fromString( - "{\"type\":\"GetMetadata\",\"typeProperties\":{\"dataset\":{\"referenceName\":\"emaxoa\",\"parameters\":{\"yxajkkzkzprjq\":\"datakcachsojgag\",\"xwynzbeemlsrtgb\":\"datamg\"}},\"fieldList\":[\"datautukwdglp\"],\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datauhbdmmfdrxyejjqc\",\"disableMetricsCollection\":\"datavbahiiatpdxpox\",\"\":{\"wxuyikgmlplqgps\":\"datapsmxfchnhjsaq\",\"lvwtslzblgvezhim\":\"dataynkkezkv\",\"dzkovt\":\"dataiyqwlxkyoysyutnr\",\"ayupa\":\"datanmcaprxhixmybl\"}},\"formatSettings\":{\"type\":\"FormatReadSettings\",\"\":{\"vhcboi\":\"dataumpunwyf\"}}},\"linkedServiceName\":{\"referenceName\":\"hghicwvh\",\"parameters\":{\"t\":\"datavj\",\"m\":\"datapyanooytil\",\"wqljmmoquicrz\":\"datas\",\"zh\":\"dataluqacebcn\"}},\"policy\":{\"timeout\":\"datamjuruspfl\",\"retry\":\"datalvwkgcpfz\",\"retryIntervalInSeconds\":741825405,\"secureInput\":true,\"secureOutput\":false,\"\":{\"qqmvsrbmfobtmljo\":\"datax\"}},\"name\":\"guoyo\",\"description\":\"yg\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ebyczwegt\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"rgmidvtu\":\"datazqxxvksqi\"}},{\"activity\":\"dgl\",\"dependencyConditions\":[\"Failed\",\"Completed\"],\"\":{\"dlt\":\"datal\",\"jnlxjhrzgnfqq\":\"dataryhztwxuizakejo\",\"eokrarzkzatznvye\":\"datagqezgbqiiweoa\"}}],\"userProperties\":[{\"name\":\"vzomtzpukmxgs\",\"value\":\"datazbpn\"},{\"name\":\"f\",\"value\":\"dataqwmx\"}],\"\":{\"bofzmvtwyjc\":\"datawzesejdcpcpeu\",\"jlbygq\":\"datajrptltytbqhejhn\",\"slxzwvygquiwcfq\":\"datameeuuurx\",\"wwdev\":\"dataob\"}}") + "{\"type\":\"GetMetadata\",\"typeProperties\":{\"dataset\":{\"referenceName\":\"qwvwfombcgr\",\"parameters\":{\"qtydfyctkr\":\"datanrco\",\"wrhoma\":\"dataagxzmrxxmgzslnnc\"}},\"fieldList\":[\"dataiwupooneoqyetfxy\"],\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataqvkdpnqcup\",\"disableMetricsCollection\":\"datar\",\"\":{\"vsqmzee\":\"datafekcue\"}},\"formatSettings\":{\"type\":\"FormatReadSettings\",\"\":{\"y\":\"datav\"}}},\"linkedServiceName\":{\"referenceName\":\"ifnmccfgkb\",\"parameters\":{\"sdcmgmvatnf\":\"datamnam\",\"vaap\":\"datahyrh\",\"uj\":\"datagyyufhcfeggy\"}},\"policy\":{\"timeout\":\"datavazqsbrqspvl\",\"retry\":\"dataxvuju\",\"retryIntervalInSeconds\":1606585230,\"secureInput\":false,\"secureOutput\":false,\"\":{\"ayoaskullqweo\":\"datainjc\",\"ngymbzawdwtzx\":\"datab\",\"pwvhiaxkm\":\"databqzplzyjktc\",\"fhlwgka\":\"dataitczuscqobujfx\"}},\"name\":\"xp\",\"description\":\"mbdhccmjo\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"swfbqycubm\",\"dependencyConditions\":[\"Completed\",\"Completed\"],\"\":{\"lkxwchslbiptsf\":\"dataewd\",\"unjegomegma\":\"datacwaobuimfda\"}}],\"userProperties\":[{\"name\":\"eablknqnqqcgi\",\"value\":\"dataffdeogm\"}],\"\":{\"po\":\"dataopjlgtcswqxeva\",\"gn\":\"datamxtcnmocskpgn\",\"dezm\":\"dataguqfnhmmvedj\"}}") .toObject(GetMetadataActivity.class); - Assertions.assertEquals("guoyo", model.name()); - Assertions.assertEquals("yg", model.description()); + Assertions.assertEquals("xp", model.name()); + Assertions.assertEquals("mbdhccmjo", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("ebyczwegt", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("vzomtzpukmxgs", model.userProperties().get(0).name()); - Assertions.assertEquals("hghicwvh", model.linkedServiceName().referenceName()); - Assertions.assertEquals(741825405, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("swfbqycubm", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("eablknqnqqcgi", model.userProperties().get(0).name()); + Assertions.assertEquals("ifnmccfgkb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1606585230, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("emaxoa", model.dataset().referenceName()); + Assertions.assertEquals("qwvwfombcgr", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GetMetadataActivity model = new GetMetadataActivity().withName("guoyo").withDescription("yg") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("ebyczwegt") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dgl") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("vzomtzpukmxgs").withValue("datazbpn"), - new UserProperty().withName("f").withValue("dataqwmx"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("hghicwvh").withParameters( - mapOf("t", "datavj", "m", "datapyanooytil", "wqljmmoquicrz", "datas", "zh", "dataluqacebcn"))) - .withPolicy(new ActivityPolicy().withTimeout("datamjuruspfl").withRetry("datalvwkgcpfz") - .withRetryIntervalInSeconds(741825405).withSecureInput(true).withSecureOutput(false) + GetMetadataActivity model = new GetMetadataActivity().withName("xp").withDescription("mbdhccmjo") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("swfbqycubm") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("eablknqnqqcgi").withValue("dataffdeogm"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ifnmccfgkb") + .withParameters(mapOf("sdcmgmvatnf", "datamnam", "vaap", "datahyrh", "uj", "datagyyufhcfeggy"))) + .withPolicy(new ActivityPolicy().withTimeout("datavazqsbrqspvl").withRetry("dataxvuju") + .withRetryIntervalInSeconds(1606585230).withSecureInput(false).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withDataset(new DatasetReference().withReferenceName("emaxoa") - .withParameters(mapOf("yxajkkzkzprjq", "datakcachsojgag", "xwynzbeemlsrtgb", "datamg"))) - .withFieldList(Arrays.asList("datautukwdglp")) - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datauhbdmmfdrxyejjqc") - .withDisableMetricsCollection("datavbahiiatpdxpox") - .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + .withDataset(new DatasetReference().withReferenceName("qwvwfombcgr") + .withParameters(mapOf("qtydfyctkr", "datanrco", "wrhoma", "dataagxzmrxxmgzslnnc"))) + .withFieldList(Arrays.asList("dataiwupooneoqyetfxy")) + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataqvkdpnqcup") + .withDisableMetricsCollection("datar").withAdditionalProperties(mapOf("type", "StoreReadSettings"))) .withFormatSettings(new FormatReadSettings().withAdditionalProperties(mapOf("type", "FormatReadSettings"))); model = BinaryData.fromObject(model).toObject(GetMetadataActivity.class); - Assertions.assertEquals("guoyo", model.name()); - Assertions.assertEquals("yg", model.description()); + Assertions.assertEquals("xp", model.name()); + Assertions.assertEquals("mbdhccmjo", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("ebyczwegt", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("vzomtzpukmxgs", model.userProperties().get(0).name()); - Assertions.assertEquals("hghicwvh", model.linkedServiceName().referenceName()); - Assertions.assertEquals(741825405, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("swfbqycubm", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("eablknqnqqcgi", model.userProperties().get(0).name()); + Assertions.assertEquals("ifnmccfgkb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1606585230, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("emaxoa", model.dataset().referenceName()); + Assertions.assertEquals("qwvwfombcgr", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTypePropertiesTests.java index 30d0aad361c1e..cddb6d5a4b991 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GetMetadataActivityTypePropertiesTests.java @@ -18,22 +18,23 @@ public final class GetMetadataActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GetMetadataActivityTypeProperties model = BinaryData.fromString( - "{\"dataset\":{\"referenceName\":\"mte\",\"parameters\":{\"yya\":\"dataggykirqks\",\"yiw\":\"datammim\"}},\"fieldList\":[\"dataqjb\",\"datarxmlmibvczdjko\",\"datalvlfkwdtsbjmc\"],\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datafezjyfaqdwfalnzd\",\"disableMetricsCollection\":\"dataslxe\",\"\":{\"rsycujnsznjsk\":\"datalvhalxxgeladqzil\",\"sonfxsfje\":\"datajjupukhxpixuyy\"}},\"formatSettings\":{\"type\":\"FormatReadSettings\",\"\":{\"dsp\":\"dataicufxtcyt\",\"sqpjcuuyttuindpm\":\"dataduhztvbgcfltt\",\"nsbjzrnjcagagm\":\"dataijncaqgtsbahtlop\",\"nashnoxr\":\"dataul\"}}}") + "{\"dataset\":{\"referenceName\":\"pbezlucxbuda\",\"parameters\":{\"i\":\"datamvvbwrunrgmyv\",\"xoahfvkyhfdth\":\"datavxlhfmkl\"}},\"fieldList\":[\"datafzxseqscoy\",\"dataxbaw\",\"dataisbhkeskgnj\",\"dataavoqcyl\"],\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datadmrqra\",\"disableMetricsCollection\":\"dataxwzewpngcocboh\",\"\":{\"vsugentrlzbwtivg\":\"dataob\",\"kmwyikoanep\":\"datacrrbswbxizmxvd\"}},\"formatSettings\":{\"type\":\"FormatReadSettings\",\"\":{\"ajonjdhbqw\":\"datavnbzgl\",\"f\":\"dataugsgp\",\"xwmjlmosqh\":\"dataaykzwij\"}}}") .toObject(GetMetadataActivityTypeProperties.class); - Assertions.assertEquals("mte", model.dataset().referenceName()); + Assertions.assertEquals("pbezlucxbuda", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { GetMetadataActivityTypeProperties model = new GetMetadataActivityTypeProperties() - .withDataset(new DatasetReference().withReferenceName("mte") - .withParameters(mapOf("yya", "dataggykirqks", "yiw", "datammim"))) - .withFieldList(Arrays.asList("dataqjb", "datarxmlmibvczdjko", "datalvlfkwdtsbjmc")) - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datafezjyfaqdwfalnzd") - .withDisableMetricsCollection("dataslxe").withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + .withDataset(new DatasetReference().withReferenceName("pbezlucxbuda") + .withParameters(mapOf("i", "datamvvbwrunrgmyv", "xoahfvkyhfdth", "datavxlhfmkl"))) + .withFieldList(Arrays.asList("datafzxseqscoy", "dataxbaw", "dataisbhkeskgnj", "dataavoqcyl")) + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datadmrqra") + .withDisableMetricsCollection("dataxwzewpngcocboh") + .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) .withFormatSettings(new FormatReadSettings().withAdditionalProperties(mapOf("type", "FormatReadSettings"))); model = BinaryData.fromObject(model).toObject(GetMetadataActivityTypeProperties.class); - Assertions.assertEquals("mte", model.dataset().referenceName()); + Assertions.assertEquals("pbezlucxbuda", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersCreateOrUpdateWithResponseMockTests.java index df9402fd89e1d..fda1463aec51c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersCreateOrUpdateWithResponseMockTests.java @@ -35,7 +35,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"ymmsimbesg\":{\"type\":\"String\",\"value\":\"dataoyawgpsx\"}},\"name\":\"ynwjowgdw\",\"type\":\"eyyzmxuelplbbsei\",\"etag\":\"dmfln\",\"id\":\"tltxfobyqq\"}"; + = "{\"properties\":{\"pwvbrtwc\":{\"type\":\"String\",\"value\":\"dataetiydlrjtb\"},\"rkjepdfsg\":{\"type\":\"Int\",\"value\":\"datamjgjlshshva\"},\"bgslllcwf\":{\"type\":\"Int\",\"value\":\"datarfltgbbxghxaqdsi\"}},\"name\":\"bqqu\",\"type\":\"txmg\",\"etag\":\"bcoguerwmljbtytd\",\"id\":\"gpjewqgyex\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,15 +52,17 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - GlobalParameterResource response = manager.globalParameters().define("fmkf").withExistingFactory("del", "ecjmf") - .withProperties(mapOf("amosii", - new GlobalParameterSpecification().withType(GlobalParameterType.OBJECT).withValue("datafthsaf"), "mdt", - new GlobalParameterSpecification().withType(GlobalParameterType.FLOAT).withValue("datac"), "jzphfprf", - new GlobalParameterSpecification().withType(GlobalParameterType.OBJECT).withValue("datadynug"))) - .create(); + GlobalParameterResource response + = manager.globalParameters().define("fzuxzyrzq").withExistingFactory("wvmmy", "rqlpeyi") + .withProperties(mapOf("pq", + new GlobalParameterSpecification().withType(GlobalParameterType.FLOAT) + .withValue("dataabouerncgvjmk"), + "jhspy", + new GlobalParameterSpecification().withType(GlobalParameterType.FLOAT).withValue("dataolmumz"))) + .create(); - Assertions.assertEquals("tltxfobyqq", response.id()); - Assertions.assertEquals(GlobalParameterType.STRING, response.properties().get("ymmsimbesg").type()); + Assertions.assertEquals("gpjewqgyex", response.id()); + Assertions.assertEquals(GlobalParameterType.STRING, response.properties().get("pwvbrtwc").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersDeleteWithResponseMockTests.java index 9d8506fd65d8a..641383d3cfb35 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.globalParameters().deleteWithResponse("qzcp", "sbiruydiws", "vanpzabbfdhis", + manager.globalParameters().deleteWithResponse("hhm", "honnmbaottulka", "wonmizwfuk", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersGetWithResponseMockTests.java index 1d4a67a5dc0d6..befe60cdf857e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersGetWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"oydzmbv\":{\"type\":\"Object\",\"value\":\"datapbbwicteqwjlynlr\"},\"otozxadk\":{\"type\":\"Bool\",\"value\":\"dataadxs\"}},\"name\":\"aptv\",\"type\":\"y\",\"etag\":\"gvzevpypkfvw\",\"id\":\"vxjxvspubfke\"}"; + = "{\"properties\":{\"wiautvehpvlm\":{\"type\":\"Bool\",\"value\":\"datablw\"},\"yfyfubtrtaipji\":{\"type\":\"String\",\"value\":\"datanlmzeqhqf\"}},\"name\":\"dowcgqc\",\"type\":\"vqqyxziutqnv\",\"etag\":\"trnwgchvgpun\",\"id\":\"fogccac\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,9 +50,9 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); GlobalParameterResource response = manager.globalParameters() - .getWithResponse("rd", "onovveouwixtefd", "qprhzsaquha", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("qpmdojbm", "johu", "uvnbiujt", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("vxjxvspubfke", response.id()); - Assertions.assertEquals(GlobalParameterType.OBJECT, response.properties().get("oydzmbv").type()); + Assertions.assertEquals("fogccac", response.id()); + Assertions.assertEquals(GlobalParameterType.BOOL, response.properties().get("wiautvehpvlm").type()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersListByFactoryMockTests.java index 279c39273ab82..7c6311a373508 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GlobalParametersListByFactoryMockTests.java @@ -33,7 +33,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"qeadkvldrcxvoltj\":{\"type\":\"Array\",\"value\":\"datadokv\"},\"eghsq\":{\"type\":\"Bool\",\"value\":\"dataolnqkdtvtpwrmmya\"}},\"name\":\"nyphewcevpmtpqdf\",\"type\":\"sojexd\",\"etag\":\"lvcgarsvqu\",\"id\":\"rkoqdqjhhtxnocix\"}]}"; + = "{\"value\":[{\"properties\":{\"njxnoqxgfvgpimtn\":{\"type\":\"Int\",\"value\":\"dataumz\"}},\"name\":\"fjdgfmesmeaoa\",\"type\":\"qbaeoozjncu\",\"etag\":\"rdyeilxktseaahax\",\"id\":\"fprqwopjnrafli\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,11 +50,11 @@ public void testListByFactory() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.globalParameters().listByFactory("qlwtdijejyxrrmno", - "jwlhulvyz", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.globalParameters().listByFactory("ofr", "wsavdijbium", com.azure.core.util.Context.NONE); - Assertions.assertEquals("rkoqdqjhhtxnocix", response.iterator().next().id()); - Assertions.assertEquals(GlobalParameterType.ARRAY, - response.iterator().next().properties().get("qeadkvldrcxvoltj").type()); + Assertions.assertEquals("fprqwopjnrafli", response.iterator().next().id()); + Assertions.assertEquals(GlobalParameterType.INT, + response.iterator().next().properties().get("njxnoqxgfvgpimtn").type()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleAdWordsSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleAdWordsSourceTests.java index 344ca0d7aec82..93226d86fbd08 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleAdWordsSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleAdWordsSourceTests.java @@ -11,16 +11,16 @@ public final class GoogleAdWordsSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GoogleAdWordsSource model = BinaryData.fromString( - "{\"type\":\"GoogleAdWordsSource\",\"query\":\"datagoodfh\",\"queryTimeout\":\"datauegdynyddp\",\"additionalColumns\":\"dataizlzxht\",\"sourceRetryCount\":\"dataglgmf\",\"sourceRetryWait\":\"dataholaflsjwawlkq\",\"maxConcurrentConnections\":\"datankcwio\",\"disableMetricsCollection\":\"datayk\",\"\":{\"elxrfnp\":\"datamobnehbbchtc\",\"iowandefmebp\":\"dataumgnjmskuveuog\",\"fdsvmpt\":\"datalzpyptgwmrzrh\",\"lgoravovqpnxpufv\":\"datarzilvcncdazw\"}}") + "{\"type\":\"GoogleAdWordsSource\",\"query\":\"dataozqqwiawbwzyvbui\",\"queryTimeout\":\"dataysatoplqc\",\"additionalColumns\":\"datasrlzwuqkprf\",\"sourceRetryCount\":\"datacowtoqfwbsbkob\",\"sourceRetryWait\":\"datassj\",\"maxConcurrentConnections\":\"datahfcxwrjbrxm\",\"disableMetricsCollection\":\"dataetttul\",\"\":{\"mosiskihf\":\"datajbhespf\"}}") .toObject(GoogleAdWordsSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GoogleAdWordsSource model - = new GoogleAdWordsSource().withSourceRetryCount("dataglgmf").withSourceRetryWait("dataholaflsjwawlkq") - .withMaxConcurrentConnections("datankcwio").withDisableMetricsCollection("datayk") - .withQueryTimeout("datauegdynyddp").withAdditionalColumns("dataizlzxht").withQuery("datagoodfh"); + GoogleAdWordsSource model = new GoogleAdWordsSource().withSourceRetryCount("datacowtoqfwbsbkob") + .withSourceRetryWait("datassj").withMaxConcurrentConnections("datahfcxwrjbrxm") + .withDisableMetricsCollection("dataetttul").withQueryTimeout("dataysatoplqc") + .withAdditionalColumns("datasrlzwuqkprf").withQuery("dataozqqwiawbwzyvbui"); model = BinaryData.fromObject(model).toObject(GoogleAdWordsSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryDatasetTypePropertiesTests.java index f666622af18a8..21f8014895ffa 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryDatasetTypePropertiesTests.java @@ -10,16 +10,15 @@ public final class GoogleBigQueryDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GoogleBigQueryDatasetTypeProperties model = BinaryData - .fromString( - "{\"tableName\":\"dataaqermnddlir\",\"table\":\"dataclsaqifepdu\",\"dataset\":\"dataevivkigliokl\"}") - .toObject(GoogleBigQueryDatasetTypeProperties.class); + GoogleBigQueryDatasetTypeProperties model + = BinaryData.fromString("{\"tableName\":\"datax\",\"table\":\"dataxlawmvdyqab\",\"dataset\":\"dataopx\"}") + .toObject(GoogleBigQueryDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GoogleBigQueryDatasetTypeProperties model = new GoogleBigQueryDatasetTypeProperties() - .withTableName("dataaqermnddlir").withTable("dataclsaqifepdu").withDataset("dataevivkigliokl"); + GoogleBigQueryDatasetTypeProperties model = new GoogleBigQueryDatasetTypeProperties().withTableName("datax") + .withTable("dataxlawmvdyqab").withDataset("dataopx"); model = BinaryData.fromObject(model).toObject(GoogleBigQueryDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryObjectDatasetTests.java index 597fdfb96fb4b..9829372969862 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryObjectDatasetTests.java @@ -19,32 +19,32 @@ public final class GoogleBigQueryObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GoogleBigQueryObjectDataset model = BinaryData.fromString( - "{\"type\":\"GoogleBigQueryObject\",\"typeProperties\":{\"tableName\":\"datamemfvrcclclfkfvy\",\"table\":\"datammw\",\"dataset\":\"datapoipjylxtebvse\"},\"description\":\"zvvpaysqwh\",\"structure\":\"datacyandb\",\"schema\":\"databntcvpvd\",\"linkedServiceName\":{\"referenceName\":\"moqqctfvxuosqp\",\"parameters\":{\"wyjzuakkiubeqk\":\"datapjpjmsbzzjsnyf\"}},\"parameters\":{\"hogsezre\":{\"type\":\"Int\",\"defaultValue\":\"dataglhxsoanguhb\"},\"itwkejmg\":{\"type\":\"Float\",\"defaultValue\":\"datagpdtyzpx\"},\"skvsdfvhryp\":{\"type\":\"Array\",\"defaultValue\":\"datadupe\"}},\"annotations\":[\"datammpkapvnpeukg\",\"datamfakeqn\",\"datatromlcsvk\",\"datafpsrowshvfxj\"],\"folder\":{\"name\":\"awmv\"},\"\":{\"znyjyu\":\"dataabjropxfqdml\",\"wgdp\":\"dataql\",\"iri\":\"datah\",\"dpkwdtobpgdcid\":\"dataamqtrhqdoxdegacd\"}}") + "{\"type\":\"GoogleBigQueryObject\",\"typeProperties\":{\"tableName\":\"datajwxgvtkjct\",\"table\":\"datapeawzzkvfccozv\",\"dataset\":\"datasphtraitrmsukxtu\"},\"description\":\"gcptct\",\"structure\":\"dataoegyc\",\"schema\":\"dataem\",\"linkedServiceName\":{\"referenceName\":\"vrcclclfkfv\",\"parameters\":{\"vpoip\":\"datammw\",\"fzvvpaysqwhzdc\":\"dataylxtebvse\",\"dblk\":\"dataa\"}},\"parameters\":{\"fvxuosqpffapjpj\":{\"type\":\"Array\",\"defaultValue\":\"datapvdfmoqqc\"},\"yjzua\":{\"type\":\"Object\",\"defaultValue\":\"datazzjsnyfo\"},\"r\":{\"type\":\"Bool\",\"defaultValue\":\"dataubeqkitt\"}},\"annotations\":[\"dataxsoangu\",\"datab\",\"datahogsezre\"],\"folder\":{\"name\":\"gpdtyzpx\"},\"\":{\"hskvsdfvhrypezam\":\"datawkejmgemudup\",\"keq\":\"datapkapvnpeukgnmf\",\"rowsh\":\"dataitromlcsvktfp\"}}") .toObject(GoogleBigQueryObjectDataset.class); - Assertions.assertEquals("zvvpaysqwh", model.description()); - Assertions.assertEquals("moqqctfvxuosqp", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("hogsezre").type()); - Assertions.assertEquals("awmv", model.folder().name()); + Assertions.assertEquals("gcptct", model.description()); + Assertions.assertEquals("vrcclclfkfv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("fvxuosqpffapjpj").type()); + Assertions.assertEquals("gpdtyzpx", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GoogleBigQueryObjectDataset model = new GoogleBigQueryObjectDataset().withDescription("zvvpaysqwh") - .withStructure("datacyandb").withSchema("databntcvpvd") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("moqqctfvxuosqp") - .withParameters(mapOf("wyjzuakkiubeqk", "datapjpjmsbzzjsnyf"))) - .withParameters(mapOf("hogsezre", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataglhxsoanguhb"), - "itwkejmg", new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datagpdtyzpx"), - "skvsdfvhryp", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datadupe"))) - .withAnnotations(Arrays.asList("datammpkapvnpeukg", "datamfakeqn", "datatromlcsvk", "datafpsrowshvfxj")) - .withFolder(new DatasetFolder().withName("awmv")).withTableName("datamemfvrcclclfkfvy").withTable("datammw") - .withDataset("datapoipjylxtebvse"); + GoogleBigQueryObjectDataset model = new GoogleBigQueryObjectDataset().withDescription("gcptct") + .withStructure("dataoegyc").withSchema("dataem") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("vrcclclfkfv") + .withParameters(mapOf("vpoip", "datammw", "fzvvpaysqwhzdc", "dataylxtebvse", "dblk", "dataa"))) + .withParameters(mapOf("fvxuosqpffapjpj", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datapvdfmoqqc"), "yjzua", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datazzjsnyfo"), "r", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataubeqkitt"))) + .withAnnotations(Arrays.asList("dataxsoangu", "datab", "datahogsezre")) + .withFolder(new DatasetFolder().withName("gpdtyzpx")).withTableName("datajwxgvtkjct") + .withTable("datapeawzzkvfccozv").withDataset("datasphtraitrmsukxtu"); model = BinaryData.fromObject(model).toObject(GoogleBigQueryObjectDataset.class); - Assertions.assertEquals("zvvpaysqwh", model.description()); - Assertions.assertEquals("moqqctfvxuosqp", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("hogsezre").type()); - Assertions.assertEquals("awmv", model.folder().name()); + Assertions.assertEquals("gcptct", model.description()); + Assertions.assertEquals("vrcclclfkfv", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("fvxuosqpffapjpj").type()); + Assertions.assertEquals("gpdtyzpx", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQuerySourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQuerySourceTests.java index 0e7895120a2a7..14b9053013d76 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQuerySourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQuerySourceTests.java @@ -11,16 +11,16 @@ public final class GoogleBigQuerySourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GoogleBigQuerySource model = BinaryData.fromString( - "{\"type\":\"GoogleBigQuerySource\",\"query\":\"datarbgnyvypu\",\"queryTimeout\":\"dataxhowwe\",\"additionalColumns\":\"datayw\",\"sourceRetryCount\":\"datahiuwv\",\"sourceRetryWait\":\"datagejytqnzrcbh\",\"maxConcurrentConnections\":\"datahctjvlwfnzgz\",\"disableMetricsCollection\":\"datafyvytydrdcwbaiaq\",\"\":{\"edusu\":\"datahsmuclxg\",\"vykagsxhzhervv\":\"dataq\"}}") + "{\"type\":\"GoogleBigQuerySource\",\"query\":\"dataanirlydsdmacydqa\",\"queryTimeout\":\"datayvwxubgulyz\",\"additionalColumns\":\"dataasxpprohuabdu\",\"sourceRetryCount\":\"datavsoxnpuapt\",\"sourceRetryWait\":\"datawekiqlscmtcljopi\",\"maxConcurrentConnections\":\"datawxvcfchokkcjjnq\",\"disableMetricsCollection\":\"datajoayaj\",\"\":{\"fbzbxeqzvokfrhfa\":\"datacxjmap\",\"uaxdulv\":\"dataxcgjuc\",\"mksgeqpai\":\"dataefsrxqscdbbwej\",\"eotvnet\":\"datalfscosf\"}}") .toObject(GoogleBigQuerySource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GoogleBigQuerySource model - = new GoogleBigQuerySource().withSourceRetryCount("datahiuwv").withSourceRetryWait("datagejytqnzrcbh") - .withMaxConcurrentConnections("datahctjvlwfnzgz").withDisableMetricsCollection("datafyvytydrdcwbaiaq") - .withQueryTimeout("dataxhowwe").withAdditionalColumns("datayw").withQuery("datarbgnyvypu"); + GoogleBigQuerySource model = new GoogleBigQuerySource().withSourceRetryCount("datavsoxnpuapt") + .withSourceRetryWait("datawekiqlscmtcljopi").withMaxConcurrentConnections("datawxvcfchokkcjjnq") + .withDisableMetricsCollection("datajoayaj").withQueryTimeout("datayvwxubgulyz") + .withAdditionalColumns("dataasxpprohuabdu").withQuery("dataanirlydsdmacydqa"); model = BinaryData.fromObject(model).toObject(GoogleBigQuerySource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2DatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2DatasetTypePropertiesTests.java new file mode 100644 index 0000000000000..f6173a9e25405 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2DatasetTypePropertiesTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.fluent.models.GoogleBigQueryV2DatasetTypeProperties; + +public final class GoogleBigQueryV2DatasetTypePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + GoogleBigQueryV2DatasetTypeProperties model + = BinaryData.fromString("{\"table\":\"dataddpwmgw\",\"dataset\":\"dataukfjvqgl\"}") + .toObject(GoogleBigQueryV2DatasetTypeProperties.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + GoogleBigQueryV2DatasetTypeProperties model + = new GoogleBigQueryV2DatasetTypeProperties().withTable("dataddpwmgw").withDataset("dataukfjvqgl"); + model = BinaryData.fromObject(model).toObject(GoogleBigQueryV2DatasetTypeProperties.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2ObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2ObjectDatasetTests.java new file mode 100644 index 0000000000000..92f641c6a880e --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2ObjectDatasetTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.DatasetFolder; +import com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2ObjectDataset; +import com.azure.resourcemanager.datafactory.models.LinkedServiceReference; +import com.azure.resourcemanager.datafactory.models.ParameterSpecification; +import com.azure.resourcemanager.datafactory.models.ParameterType; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class GoogleBigQueryV2ObjectDatasetTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + GoogleBigQueryV2ObjectDataset model = BinaryData.fromString( + "{\"type\":\"GoogleBigQueryV2Object\",\"typeProperties\":{\"table\":\"datamllznyjyuw\",\"dataset\":\"datazwgdpvhwiril\"},\"description\":\"qtr\",\"structure\":\"datadoxdegacdedpkw\",\"schema\":\"dataobp\",\"linkedServiceName\":{\"referenceName\":\"dcidpdaq\",\"parameters\":{\"lsaqifepdureeviv\":\"datanddlirqq\"}},\"parameters\":{\"tlfytbltytv\":{\"type\":\"SecureString\",\"defaultValue\":\"dataoklsuffpxesw\"},\"clmowurofo\":{\"type\":\"Array\",\"defaultValue\":\"datagcesfdd\"}},\"annotations\":[\"datacj\"],\"folder\":{\"name\":\"weob\"},\"\":{\"hixcc\":\"datadq\",\"og\":\"datakf\",\"bzd\":\"datayoxmyqzyqepg\",\"vo\":\"dataluokc\"}}") + .toObject(GoogleBigQueryV2ObjectDataset.class); + Assertions.assertEquals("qtr", model.description()); + Assertions.assertEquals("dcidpdaq", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("tlfytbltytv").type()); + Assertions.assertEquals("weob", model.folder().name()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + GoogleBigQueryV2ObjectDataset model + = new GoogleBigQueryV2ObjectDataset().withDescription("qtr").withStructure("datadoxdegacdedpkw") + .withSchema("dataobp") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dcidpdaq") + .withParameters(mapOf("lsaqifepdureeviv", "datanddlirqq"))) + .withParameters(mapOf("tlfytbltytv", + new ParameterSpecification().withType(ParameterType.SECURE_STRING) + .withDefaultValue("dataoklsuffpxesw"), + "clmowurofo", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datagcesfdd"))) + .withAnnotations(Arrays.asList("datacj")).withFolder(new DatasetFolder().withName("weob")) + .withTable("datamllznyjyuw").withDataset("datazwgdpvhwiril"); + model = BinaryData.fromObject(model).toObject(GoogleBigQueryV2ObjectDataset.class); + Assertions.assertEquals("qtr", model.description()); + Assertions.assertEquals("dcidpdaq", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("tlfytbltytv").type()); + Assertions.assertEquals("weob", model.folder().name()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2SourceTests.java new file mode 100644 index 0000000000000..b79025c76112b --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleBigQueryV2SourceTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.GoogleBigQueryV2Source; + +public final class GoogleBigQueryV2SourceTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + GoogleBigQueryV2Source model = BinaryData.fromString( + "{\"type\":\"GoogleBigQueryV2Source\",\"query\":\"datahndfpf\",\"queryTimeout\":\"datafdgf\",\"additionalColumns\":\"dataoeh\",\"sourceRetryCount\":\"datapkssjbw\",\"sourceRetryWait\":\"dataxdgcfcfky\",\"maxConcurrentConnections\":\"datajwxhslrbwwk\",\"disableMetricsCollection\":\"datawodhsodofsxjiky\",\"\":{\"cxdmxhuwldfa\":\"datauhuixqwogg\",\"dkbgsg\":\"datakyft\",\"ayqkg\":\"datapyckmncrutoudjm\"}}") + .toObject(GoogleBigQueryV2Source.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + GoogleBigQueryV2Source model + = new GoogleBigQueryV2Source().withSourceRetryCount("datapkssjbw").withSourceRetryWait("dataxdgcfcfky") + .withMaxConcurrentConnections("datajwxhslrbwwk").withDisableMetricsCollection("datawodhsodofsxjiky") + .withQueryTimeout("datafdgf").withAdditionalColumns("dataoeh").withQuery("datahndfpf"); + model = BinaryData.fromObject(model).toObject(GoogleBigQueryV2Source.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleCloudStorageReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleCloudStorageReadSettingsTests.java index cf47b4c6c19f5..87b5e6049616f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleCloudStorageReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GoogleCloudStorageReadSettingsTests.java @@ -11,19 +11,19 @@ public final class GoogleCloudStorageReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GoogleCloudStorageReadSettings model = BinaryData.fromString( - "{\"type\":\"GoogleCloudStorageReadSettings\",\"recursive\":\"dataeyaoyzjfgvxau\",\"wildcardFolderPath\":\"datanabgrsnfzmth\",\"wildcardFileName\":\"datacuf\",\"prefix\":\"datazfot\",\"fileListPath\":\"datakumam\",\"enablePartitionDiscovery\":\"datargljekh\",\"partitionRootPath\":\"datafgjbeybdukbglniw\",\"deleteFilesAfterCompletion\":\"datamysce\",\"modifiedDatetimeStart\":\"datavoexkonciacdl\",\"modifiedDatetimeEnd\":\"datahs\",\"maxConcurrentConnections\":\"datavxkctedhaf\",\"disableMetricsCollection\":\"dataiffaj\",\"\":{\"bbugo\":\"databyzyjuyxy\"}}") + "{\"type\":\"GoogleCloudStorageReadSettings\",\"recursive\":\"datatnsnb\",\"wildcardFolderPath\":\"datauv\",\"wildcardFileName\":\"dataodfmort\",\"prefix\":\"datanpbsungnjkkmkzfb\",\"fileListPath\":\"datacgbgzjyr\",\"enablePartitionDiscovery\":\"dataiwhmrhz\",\"partitionRootPath\":\"datavpjydwmaqeytjp\",\"deleteFilesAfterCompletion\":\"datadp\",\"modifiedDatetimeStart\":\"datapdcsvzugiurhgqlv\",\"modifiedDatetimeEnd\":\"datajzscrjtnq\",\"maxConcurrentConnections\":\"datapobjufksddxk\",\"disableMetricsCollection\":\"datawxlylxfpvoylf\",\"\":{\"ime\":\"datarguecbthauivg\"}}") .toObject(GoogleCloudStorageReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { GoogleCloudStorageReadSettings model - = new GoogleCloudStorageReadSettings().withMaxConcurrentConnections("datavxkctedhaf") - .withDisableMetricsCollection("dataiffaj").withRecursive("dataeyaoyzjfgvxau") - .withWildcardFolderPath("datanabgrsnfzmth").withWildcardFileName("datacuf").withPrefix("datazfot") - .withFileListPath("datakumam").withEnablePartitionDiscovery("datargljekh") - .withPartitionRootPath("datafgjbeybdukbglniw").withDeleteFilesAfterCompletion("datamysce") - .withModifiedDatetimeStart("datavoexkonciacdl").withModifiedDatetimeEnd("datahs"); + = new GoogleCloudStorageReadSettings().withMaxConcurrentConnections("datapobjufksddxk") + .withDisableMetricsCollection("datawxlylxfpvoylf").withRecursive("datatnsnb") + .withWildcardFolderPath("datauv").withWildcardFileName("dataodfmort").withPrefix("datanpbsungnjkkmkzfb") + .withFileListPath("datacgbgzjyr").withEnablePartitionDiscovery("dataiwhmrhz") + .withPartitionRootPath("datavpjydwmaqeytjp").withDeleteFilesAfterCompletion("datadp") + .withModifiedDatetimeStart("datapdcsvzugiurhgqlv").withModifiedDatetimeEnd("datajzscrjtnq"); model = BinaryData.fromObject(model).toObject(GoogleCloudStorageReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumDatasetTypePropertiesTests.java index 7c149c8822adb..f468c2437b649 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumDatasetTypePropertiesTests.java @@ -10,15 +10,15 @@ public final class GreenplumDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GreenplumDatasetTypeProperties model = BinaryData - .fromString("{\"tableName\":\"dataqicmdrgcuzjmvk\",\"table\":\"datar\",\"schema\":\"dataqhgcm\"}") - .toObject(GreenplumDatasetTypeProperties.class); + GreenplumDatasetTypeProperties model + = BinaryData.fromString("{\"tableName\":\"datab\",\"table\":\"datau\",\"schema\":\"datagtxlzncoqxtvytzq\"}") + .toObject(GreenplumDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GreenplumDatasetTypeProperties model = new GreenplumDatasetTypeProperties().withTableName("dataqicmdrgcuzjmvk") - .withTable("datar").withSchema("dataqhgcm"); + GreenplumDatasetTypeProperties model = new GreenplumDatasetTypeProperties().withTableName("datab") + .withTable("datau").withSchema("datagtxlzncoqxtvytzq"); model = BinaryData.fromObject(model).toObject(GreenplumDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumSourceTests.java index f0acde583ddef..89f88de5d4281 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumSourceTests.java @@ -11,16 +11,16 @@ public final class GreenplumSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GreenplumSource model = BinaryData.fromString( - "{\"type\":\"GreenplumSource\",\"query\":\"databrolqxloed\",\"queryTimeout\":\"datarvfrfsyqb\",\"additionalColumns\":\"datawujwowthvuepszzn\",\"sourceRetryCount\":\"datajnsp\",\"sourceRetryWait\":\"dataqo\",\"maxConcurrentConnections\":\"datautqt\",\"disableMetricsCollection\":\"dataivvnmavfzjwdwwnx\",\"\":{\"piwcgcwmsh\":\"datand\",\"pzclkatpuomga\":\"datapqxjxhdwjfx\",\"vcfbflyzcqomlybs\":\"datavcqnjjfm\"}}") + "{\"type\":\"GreenplumSource\",\"query\":\"datafn\",\"queryTimeout\":\"dataeyavldovpwrq\",\"additionalColumns\":\"datazokplzliizb\",\"sourceRetryCount\":\"datajumulhfq\",\"sourceRetryWait\":\"datanchah\",\"maxConcurrentConnections\":\"datanrptrqcap\",\"disableMetricsCollection\":\"datafvowzbk\",\"\":{\"qzzkplqmca\":\"datapzdpujywjmo\",\"jgfpqwwugfwpvj\":\"dataseiauveeng\"}}") .toObject(GreenplumSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GreenplumSource model = new GreenplumSource().withSourceRetryCount("datajnsp").withSourceRetryWait("dataqo") - .withMaxConcurrentConnections("datautqt").withDisableMetricsCollection("dataivvnmavfzjwdwwnx") - .withQueryTimeout("datarvfrfsyqb").withAdditionalColumns("datawujwowthvuepszzn") - .withQuery("databrolqxloed"); + GreenplumSource model + = new GreenplumSource().withSourceRetryCount("datajumulhfq").withSourceRetryWait("datanchah") + .withMaxConcurrentConnections("datanrptrqcap").withDisableMetricsCollection("datafvowzbk") + .withQueryTimeout("dataeyavldovpwrq").withAdditionalColumns("datazokplzliizb").withQuery("datafn"); model = BinaryData.fromObject(model).toObject(GreenplumSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumTableDatasetTests.java index 23dc13383b7b8..92bab8c42fb18 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/GreenplumTableDatasetTests.java @@ -19,31 +19,34 @@ public final class GreenplumTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GreenplumTableDataset model = BinaryData.fromString( - "{\"type\":\"GreenplumTable\",\"typeProperties\":{\"tableName\":\"datafpxeswctlfytb\",\"table\":\"dataytvnpbgcesfd\",\"schema\":\"dataclmowurofo\"},\"description\":\"b\",\"structure\":\"datazzwweoblb\",\"schema\":\"dataq\",\"linkedServiceName\":{\"referenceName\":\"hixcc\",\"parameters\":{\"xmyqzyqepgbb\":\"datasogvy\",\"dpwmgwxwukfjvqg\":\"datadsluokcevoxd\",\"gyphheovejkpalec\":\"dataaxseisvv\",\"pu\":\"datatlthrt\"}},\"parameters\":{\"oll\":{\"type\":\"Object\",\"defaultValue\":\"datagrqefnq\"}},\"annotations\":[\"datarmuzemb\",\"dataqieh\",\"datahjofy\"],\"folder\":{\"name\":\"axoxlorx\"},\"\":{\"glyyhrgmabspmlu\":\"dataqcxuthvp\",\"kedputocrb\":\"datayju\"}}") + "{\"type\":\"GreenplumTable\",\"typeProperties\":{\"tableName\":\"dataeisvvvgyp\",\"table\":\"dataeovej\",\"schema\":\"dataaleczt\"},\"description\":\"h\",\"structure\":\"datazpuvfs\",\"schema\":\"datagrqefnq\",\"linkedServiceName\":{\"referenceName\":\"oll\",\"parameters\":{\"qieh\":\"datarmuzemb\",\"wnaxoxl\":\"datahjofy\",\"slqcxuthv\":\"datarx\"}},\"parameters\":{\"yju\":{\"type\":\"SecureString\",\"defaultValue\":\"datayhrgmabspmlu\"},\"rbfgqi\":{\"type\":\"Object\",\"defaultValue\":\"datadputo\"},\"r\":{\"type\":\"Bool\",\"defaultValue\":\"datargcuzjmvkr\"},\"ybqjvfio\":{\"type\":\"SecureString\",\"defaultValue\":\"datahgcmljzksqi\"}},\"annotations\":[\"dataaqpvhszopeuku\",\"datadswbsskgq\",\"dataemosq\",\"datafsjbpwjwz\"],\"folder\":{\"name\":\"pdzy\"},\"\":{\"udqhad\":\"datahxfpzc\",\"vl\":\"dataj\"}}") .toObject(GreenplumTableDataset.class); - Assertions.assertEquals("b", model.description()); - Assertions.assertEquals("hixcc", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("oll").type()); - Assertions.assertEquals("axoxlorx", model.folder().name()); + Assertions.assertEquals("h", model.description()); + Assertions.assertEquals("oll", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("yju").type()); + Assertions.assertEquals("pdzy", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GreenplumTableDataset model - = new GreenplumTableDataset().withDescription("b").withStructure("datazzwweoblb").withSchema("dataq") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("hixcc") - .withParameters(mapOf("xmyqzyqepgbb", "datasogvy", "dpwmgwxwukfjvqg", "datadsluokcevoxd", - "gyphheovejkpalec", "dataaxseisvv", "pu", "datatlthrt"))) - .withParameters(mapOf("oll", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datagrqefnq"))) - .withAnnotations(Arrays.asList("datarmuzemb", "dataqieh", "datahjofy")) - .withFolder(new DatasetFolder().withName("axoxlorx")).withTableName("datafpxeswctlfytb") - .withTable("dataytvnpbgcesfd").withSchemaTypePropertiesSchema("dataclmowurofo"); + GreenplumTableDataset model = new GreenplumTableDataset().withDescription("h").withStructure("datazpuvfs") + .withSchema("datagrqefnq") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("oll") + .withParameters(mapOf("qieh", "datarmuzemb", "wnaxoxl", "datahjofy", "slqcxuthv", "datarx"))) + .withParameters(mapOf("yju", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datayhrgmabspmlu"), + "rbfgqi", new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datadputo"), + "r", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datargcuzjmvkr"), + "ybqjvfio", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datahgcmljzksqi"))) + .withAnnotations(Arrays.asList("dataaqpvhszopeuku", "datadswbsskgq", "dataemosq", "datafsjbpwjwz")) + .withFolder(new DatasetFolder().withName("pdzy")).withTableName("dataeisvvvgyp").withTable("dataeovej") + .withSchemaTypePropertiesSchema("dataaleczt"); model = BinaryData.fromObject(model).toObject(GreenplumTableDataset.class); - Assertions.assertEquals("b", model.description()); - Assertions.assertEquals("hixcc", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("oll").type()); - Assertions.assertEquals("axoxlorx", model.folder().name()); + Assertions.assertEquals("h", model.description()); + Assertions.assertEquals("oll", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("yju").type()); + Assertions.assertEquals("pdzy", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseObjectDatasetTests.java index 31d091de688e5..87348e8c5306f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseObjectDatasetTests.java @@ -19,38 +19,33 @@ public final class HBaseObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HBaseObjectDataset model = BinaryData.fromString( - "{\"type\":\"HBaseObject\",\"typeProperties\":{\"tableName\":\"dataksqimybq\"},\"description\":\"fiomhcaqpvhs\",\"structure\":\"datapeu\",\"schema\":\"datafdswbss\",\"linkedServiceName\":{\"referenceName\":\"g\",\"parameters\":{\"jbpwjwzqgipdz\":\"datamosqmf\",\"dq\":\"datamzkhxfpzcu\"}},\"parameters\":{\"ncoqxtvytzq\":{\"type\":\"Object\",\"defaultValue\":\"datavvlyibweuaugtxl\"},\"zbdbrlbo\":{\"type\":\"Object\",\"defaultValue\":\"datadjvzmxyrazzstjvc\"},\"upmwxdsokrlnrpey\":{\"type\":\"Float\",\"defaultValue\":\"datayolacbibtkeie\"},\"wvunknsgvx\":{\"type\":\"String\",\"defaultValue\":\"dataiulddgiqlnhcxw\"}},\"annotations\":[\"datameatrtcqyfjvifb\",\"dataojtehqyo\",\"datatrcoufk\",\"datambhukdfpknvk\"],\"folder\":{\"name\":\"zje\"},\"\":{\"hzjlrknckkfxm\":\"datameo\"}}") + "{\"type\":\"HBaseObject\",\"typeProperties\":{\"tableName\":\"datadjvzmxyrazzstjvc\"},\"description\":\"bdbrl\",\"structure\":\"datazlty\",\"schema\":\"dataacbibtk\",\"linkedServiceName\":{\"referenceName\":\"iecup\",\"parameters\":{\"krlnrpeylfiiul\":\"datads\",\"hcxwwwvun\":\"datadgiql\",\"atrtcqyfjvifbmo\":\"datansgvxhxrm\",\"bhukdfpknv\":\"datatehqyoytrcoufkq\"}},\"parameters\":{\"hzjlrknckkfxm\":{\"type\":\"Float\",\"defaultValue\":\"datajezchmeo\"},\"fts\":{\"type\":\"Float\",\"defaultValue\":\"dataqkwqphfv\"},\"zgfctuuzow\":{\"type\":\"Bool\",\"defaultValue\":\"datalpxcachdt\"}},\"annotations\":[\"datavuxnx\",\"datauohshzultdbvm\"],\"folder\":{\"name\":\"ypngocbd\"},\"\":{\"zsuzgrzu\":\"dataivptb\",\"aatvogpyceinha\":\"dataekytkzvtv\",\"khwfjudapbq\":\"datahbdxsbypl\"}}") .toObject(HBaseObjectDataset.class); - Assertions.assertEquals("fiomhcaqpvhs", model.description()); - Assertions.assertEquals("g", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("ncoqxtvytzq").type()); - Assertions.assertEquals("zje", model.folder().name()); + Assertions.assertEquals("bdbrl", model.description()); + Assertions.assertEquals("iecup", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("hzjlrknckkfxm").type()); + Assertions.assertEquals("ypngocbd", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { HBaseObjectDataset model - = new HBaseObjectDataset().withDescription("fiomhcaqpvhs").withStructure("datapeu") - .withSchema("datafdswbss") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("g") - .withParameters(mapOf("jbpwjwzqgipdz", "datamosqmf", "dq", "datamzkhxfpzcu"))) - .withParameters(mapOf("ncoqxtvytzq", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datavvlyibweuaugtxl"), - "zbdbrlbo", - new ParameterSpecification().withType(ParameterType.OBJECT) - .withDefaultValue("datadjvzmxyrazzstjvc"), - "upmwxdsokrlnrpey", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datayolacbibtkeie"), - "wvunknsgvx", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataiulddgiqlnhcxw"))) - .withAnnotations( - Arrays.asList("datameatrtcqyfjvifb", "dataojtehqyo", "datatrcoufk", "datambhukdfpknvk")) - .withFolder(new DatasetFolder().withName("zje")).withTableName("dataksqimybq"); + = new HBaseObjectDataset().withDescription("bdbrl").withStructure("datazlty").withSchema("dataacbibtk") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("iecup") + .withParameters(mapOf("krlnrpeylfiiul", "datads", "hcxwwwvun", "datadgiql", "atrtcqyfjvifbmo", + "datansgvxhxrm", "bhukdfpknv", "datatehqyoytrcoufkq"))) + .withParameters(mapOf("hzjlrknckkfxm", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datajezchmeo"), "fts", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataqkwqphfv"), + "zgfctuuzow", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datalpxcachdt"))) + .withAnnotations(Arrays.asList("datavuxnx", "datauohshzultdbvm")) + .withFolder(new DatasetFolder().withName("ypngocbd")).withTableName("datadjvzmxyrazzstjvc"); model = BinaryData.fromObject(model).toObject(HBaseObjectDataset.class); - Assertions.assertEquals("fiomhcaqpvhs", model.description()); - Assertions.assertEquals("g", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("ncoqxtvytzq").type()); - Assertions.assertEquals("zje", model.folder().name()); + Assertions.assertEquals("bdbrl", model.description()); + Assertions.assertEquals("iecup", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("hzjlrknckkfxm").type()); + Assertions.assertEquals("ypngocbd", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseSourceTests.java index bf167b535379e..a5bf1454c380d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HBaseSourceTests.java @@ -11,16 +11,15 @@ public final class HBaseSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HBaseSource model = BinaryData.fromString( - "{\"type\":\"HBaseSource\",\"query\":\"datao\",\"queryTimeout\":\"datavtvbfpu\",\"additionalColumns\":\"dataobtdhum\",\"sourceRetryCount\":\"dataqwckapoetdfzjwje\",\"sourceRetryWait\":\"datalihanawipdqoz\",\"maxConcurrentConnections\":\"datalqzopvhwmtdbfrj\",\"disableMetricsCollection\":\"datavu\",\"\":{\"rd\":\"datasnfeag\"}}") + "{\"type\":\"HBaseSource\",\"query\":\"datawbqaibkyeysf\",\"queryTimeout\":\"datahdydyybztlylh\",\"additionalColumns\":\"datacjq\",\"sourceRetryCount\":\"datacie\",\"sourceRetryWait\":\"datak\",\"maxConcurrentConnections\":\"dataxf\",\"disableMetricsCollection\":\"datahvecjhbttmhneqd\",\"\":{\"kna\":\"dataeyxxidabqla\",\"ljsfcryqrrsjqt\":\"datacseqo\"}}") .toObject(HBaseSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HBaseSource model - = new HBaseSource().withSourceRetryCount("dataqwckapoetdfzjwje").withSourceRetryWait("datalihanawipdqoz") - .withMaxConcurrentConnections("datalqzopvhwmtdbfrj").withDisableMetricsCollection("datavu") - .withQueryTimeout("datavtvbfpu").withAdditionalColumns("dataobtdhum").withQuery("datao"); + HBaseSource model = new HBaseSource().withSourceRetryCount("datacie").withSourceRetryWait("datak") + .withMaxConcurrentConnections("dataxf").withDisableMetricsCollection("datahvecjhbttmhneqd") + .withQueryTimeout("datahdydyybztlylh").withAdditionalColumns("datacjq").withQuery("datawbqaibkyeysf"); model = BinaryData.fromObject(model).toObject(HBaseSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTests.java index de49c4ae63cc5..2e7b0587f9685 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTests.java @@ -23,74 +23,83 @@ public final class HDInsightHiveActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightHiveActivity model = BinaryData.fromString( - "{\"type\":\"HDInsightHive\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"biwxmcsxidaz\",\"parameters\":{\"yikhdcilinbuok\":\"datah\",\"hqrdvqvalo\":\"datafperheiplzms\"}},{\"referenceName\":\"u\",\"parameters\":{\"zedmfjgklmyomavb\":\"dataigofumbp\"}},{\"referenceName\":\"taoaixipcwyin\",\"parameters\":{\"emqwut\":\"datatqvjn\",\"xugd\":\"datayaeyyiwraowpdm\",\"idhwoyznjddsh\":\"datadmwcxvcron\"}},{\"referenceName\":\"z\",\"parameters\":{\"puotniqzqmpgvyy\":\"datavxv\",\"w\":\"datajwwbrhjhcwcfftsz\"}}],\"arguments\":[\"dataghjgvhgyeoikxjpu\",\"dataggpsaqfn\",\"dataxuwawmbuslegt\",\"dataqzkzworuhhvdeodc\"],\"getDebugInfo\":\"None\",\"scriptPath\":\"dataxbkgpnxus\",\"scriptLinkedService\":{\"referenceName\":\"gpznbklhwutyuvu\",\"parameters\":{\"plnukdawgzhbwh\":\"datarfziubeflvktjbmc\",\"gxck\":\"datavonuhv\",\"jcjvdajxebm\":\"datatjoxocothsg\"}},\"defines\":{\"a\":\"datarctf\",\"nd\":\"datakukra\",\"dhjdwfnbiyxqr\":\"datahwdicntqsrhacjsb\",\"rqllugnxmbwdkz\":\"datauyffkayovljtrml\"},\"variables\":{\"hx\":\"databqukji\",\"tbfmtbprt\":\"datapvpkvceiwcfshhc\",\"qbwgmznvlwcn\":\"datavuxwuepjcugwku\",\"qpubwz\":\"datahqkie\"},\"queryTimeout\":717039514},\"linkedServiceName\":{\"referenceName\":\"xzcj\",\"parameters\":{\"kbeoofox\":\"dataqcwnbx\"}},\"policy\":{\"timeout\":\"dataunewkssxpnhhl\",\"retry\":\"datarjcfyvkk\",\"retryIntervalInSeconds\":651506566,\"secureInput\":false,\"secureOutput\":true,\"\":{\"kgwdng\":\"dataifdfs\",\"xdvrajoghgxgzb\":\"datadnrtydhqkariatxh\"}},\"name\":\"sa\",\"description\":\"amcwet\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"dvuqb\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"qtmpzwwtcwbgmxw\":\"dataffqalm\",\"mizxkmhh\":\"dataynslcty\",\"zl\":\"datawwtarxtdgpm\"}}],\"userProperties\":[{\"name\":\"ozsdnf\",\"value\":\"datapk\"}],\"\":{\"yfubgnm\":\"datauy\",\"da\":\"datan\",\"hulvpnqvcutwngfd\":\"datai\",\"u\":\"dataztmq\"}}") + "{\"type\":\"HDInsightHive\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"vwizjraksahwq\",\"parameters\":{\"misnb\":\"dataltfknro\",\"fvqtvukcfesizkn\":\"datacz\",\"xflzhgr\":\"datac\"}},{\"referenceName\":\"hwysdmovbvn\",\"parameters\":{\"w\":\"dataqofdgzlykczoln\"}},{\"referenceName\":\"gg\",\"parameters\":{\"ukfwmhzarrfttx\":\"datavoxnjbyjgobzj\"}}],\"arguments\":[\"datarjg\",\"datahon\"],\"getDebugInfo\":\"Always\",\"scriptPath\":\"dataab\",\"scriptLinkedService\":{\"referenceName\":\"mkfvsolkjowvz\",\"parameters\":{\"c\":\"datahljtlyguothn\"}},\"defines\":{\"dlbahmivtuphwwy\":\"datauaergjtpriicte\",\"fxfteo\":\"dataxo\",\"qap\":\"datanrziwkcpxgjmyou\"},\"variables\":{\"yrsvtrtxggmpohu\":\"datapcdikk\",\"jzklqkgj\":\"dataas\",\"cwnefdehptlnw\":\"datakntknjhywgzi\"},\"queryTimeout\":196553678},\"linkedServiceName\":{\"referenceName\":\"wbneszxnmgeuoih\",\"parameters\":{\"suwct\":\"datafiwpgxyavcb\",\"yr\":\"databhcjfgxtl\",\"k\":\"datayhpbtwzrziv\",\"oldbuyuxgeph\":\"datadcjym\"}},\"policy\":{\"timeout\":\"dataexfbrsdoxhyiya\",\"retry\":\"datax\",\"retryIntervalInSeconds\":1724799013,\"secureInput\":true,\"secureOutput\":true,\"\":{\"p\":\"dataxynkh\"}},\"name\":\"qlhzdbbitpgr\",\"description\":\"pmsdgmxwfodvzpxm\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"e\",\"dependencyConditions\":[\"Failed\",\"Skipped\"],\"\":{\"dgplagwvgbnx\":\"dataqtbasvj\"}},{\"activity\":\"qudnqcbbbhin\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\"],\"\":{\"yzqdnrgnyb\":\"databfzkvrmdoshi\",\"mkhxunq\":\"dataqwjj\",\"ydabhydcqrs\":\"datavqticgsdcpmclk\"}},{\"activity\":\"qwzndzuxlgmy\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Succeeded\",\"Completed\"],\"\":{\"vb\":\"datazmbap\",\"ymcgtbpbf\":\"dataablmpntjlz\",\"jqwlvsefvkxxd\":\"datagf\",\"zm\":\"datagbnqmhr\"}}],\"userProperties\":[{\"name\":\"grflqbug\",\"value\":\"dataudsmdglqrkst\"}],\"\":{\"ficzw\":\"datadasomxwsflylols\",\"zzeqd\":\"datakglmcg\",\"kar\":\"dataxurbj\",\"jhdcolnxw\":\"datavguzvtwfbqxoqnv\"}}") .toObject(HDInsightHiveActivity.class); - Assertions.assertEquals("sa", model.name()); - Assertions.assertEquals("amcwet", model.description()); + Assertions.assertEquals("qlhzdbbitpgr", model.name()); + Assertions.assertEquals("pmsdgmxwfodvzpxm", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("dvuqb", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("ozsdnf", model.userProperties().get(0).name()); - Assertions.assertEquals("xzcj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(651506566, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("e", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("grflqbug", model.userProperties().get(0).name()); + Assertions.assertEquals("wbneszxnmgeuoih", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1724799013, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("biwxmcsxidaz", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.NONE, model.getDebugInfo()); - Assertions.assertEquals("gpznbklhwutyuvu", model.scriptLinkedService().referenceName()); - Assertions.assertEquals(717039514, model.queryTimeout()); + Assertions.assertEquals("vwizjraksahwq", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); + Assertions.assertEquals("mkfvsolkjowvz", model.scriptLinkedService().referenceName()); + Assertions.assertEquals(196553678, model.queryTimeout()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HDInsightHiveActivity model = new HDInsightHiveActivity().withName("sa").withDescription("amcwet") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("dvuqb") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("ozsdnf").withValue("datapk"))) - .withLinkedServiceName( - new LinkedServiceReference().withReferenceName("xzcj").withParameters(mapOf("kbeoofox", "dataqcwnbx"))) - .withPolicy(new ActivityPolicy().withTimeout("dataunewkssxpnhhl").withRetry("datarjcfyvkk") - .withRetryIntervalInSeconds(651506566).withSecureInput(false).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withStorageLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("biwxmcsxidaz") - .withParameters(mapOf("yikhdcilinbuok", "datah", "hqrdvqvalo", "datafperheiplzms")), - new LinkedServiceReference().withReferenceName("u") - .withParameters(mapOf("zedmfjgklmyomavb", "dataigofumbp")), - new LinkedServiceReference().withReferenceName("taoaixipcwyin").withParameters( - mapOf("emqwut", "datatqvjn", "xugd", "datayaeyyiwraowpdm", "idhwoyznjddsh", "datadmwcxvcron")), - new LinkedServiceReference().withReferenceName("z") - .withParameters(mapOf("puotniqzqmpgvyy", "datavxv", "w", "datajwwbrhjhcwcfftsz")))) - .withArguments( - Arrays.asList("dataghjgvhgyeoikxjpu", "dataggpsaqfn", "dataxuwawmbuslegt", "dataqzkzworuhhvdeodc")) - .withGetDebugInfo(HDInsightActivityDebugInfoOption.NONE).withScriptPath("dataxbkgpnxus") - .withScriptLinkedService(new LinkedServiceReference().withReferenceName("gpznbklhwutyuvu") - .withParameters(mapOf("plnukdawgzhbwh", "datarfziubeflvktjbmc", "gxck", "datavonuhv", "jcjvdajxebm", - "datatjoxocothsg"))) - .withDefines(mapOf("a", "datarctf", "nd", "datakukra", "dhjdwfnbiyxqr", "datahwdicntqsrhacjsb", - "rqllugnxmbwdkz", "datauyffkayovljtrml")) - .withVariables(mapOf("hx", "databqukji", "tbfmtbprt", "datapvpkvceiwcfshhc", "qbwgmznvlwcn", - "datavuxwuepjcugwku", "qpubwz", "datahqkie")) - .withQueryTimeout(717039514); + HDInsightHiveActivity model + = new HDInsightHiveActivity().withName("qlhzdbbitpgr").withDescription("pmsdgmxwfodvzpxm") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("e") + .withDependencyConditions( + Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("qudnqcbbbhin") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("qwzndzuxlgmy") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, + DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("grflqbug").withValue("dataudsmdglqrkst"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("wbneszxnmgeuoih") + .withParameters(mapOf("suwct", "datafiwpgxyavcb", "yr", "databhcjfgxtl", "k", "datayhpbtwzrziv", + "oldbuyuxgeph", "datadcjym"))) + .withPolicy(new ActivityPolicy().withTimeout("dataexfbrsdoxhyiya").withRetry("datax") + .withRetryIntervalInSeconds(1724799013).withSecureInput(true).withSecureOutput(true) + .withAdditionalProperties(mapOf())) + .withStorageLinkedServices(Arrays.asList( + new LinkedServiceReference().withReferenceName("vwizjraksahwq") + .withParameters(mapOf("misnb", "dataltfknro", "fvqtvukcfesizkn", "datacz", "xflzhgr", "datac")), + new LinkedServiceReference().withReferenceName("hwysdmovbvn") + .withParameters(mapOf("w", "dataqofdgzlykczoln")), + new LinkedServiceReference().withReferenceName("gg") + .withParameters(mapOf("ukfwmhzarrfttx", "datavoxnjbyjgobzj")))) + .withArguments(Arrays.asList("datarjg", "datahon")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS).withScriptPath("dataab") + .withScriptLinkedService(new LinkedServiceReference().withReferenceName("mkfvsolkjowvz") + .withParameters(mapOf("c", "datahljtlyguothn"))) + .withDefines( + mapOf("dlbahmivtuphwwy", "datauaergjtpriicte", "fxfteo", "dataxo", "qap", "datanrziwkcpxgjmyou")) + .withVariables( + mapOf("yrsvtrtxggmpohu", "datapcdikk", "jzklqkgj", "dataas", "cwnefdehptlnw", "datakntknjhywgzi")) + .withQueryTimeout(196553678); model = BinaryData.fromObject(model).toObject(HDInsightHiveActivity.class); - Assertions.assertEquals("sa", model.name()); - Assertions.assertEquals("amcwet", model.description()); + Assertions.assertEquals("qlhzdbbitpgr", model.name()); + Assertions.assertEquals("pmsdgmxwfodvzpxm", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("dvuqb", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("ozsdnf", model.userProperties().get(0).name()); - Assertions.assertEquals("xzcj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(651506566, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("e", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("grflqbug", model.userProperties().get(0).name()); + Assertions.assertEquals("wbneszxnmgeuoih", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1724799013, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("biwxmcsxidaz", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.NONE, model.getDebugInfo()); - Assertions.assertEquals("gpznbklhwutyuvu", model.scriptLinkedService().referenceName()); - Assertions.assertEquals(717039514, model.queryTimeout()); + Assertions.assertEquals("vwizjraksahwq", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); + Assertions.assertEquals("mkfvsolkjowvz", model.scriptLinkedService().referenceName()); + Assertions.assertEquals(196553678, model.queryTimeout()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTypePropertiesTests.java index 7b037feef8fe3..1fdc644f44f3e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightHiveActivityTypePropertiesTests.java @@ -17,42 +17,35 @@ public final class HDInsightHiveActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightHiveActivityTypeProperties model = BinaryData.fromString( - "{\"storageLinkedServices\":[{\"referenceName\":\"kpfyojfwvmsfn\",\"parameters\":{\"gnyugsasgh\":\"datathdfqqzhqgmoe\"}},{\"referenceName\":\"i\",\"parameters\":{\"stynjx\":\"datadxhxpqk\",\"ktiojitfa\":\"datandxrofwctjhdbidl\",\"etdrcm\":\"datakpx\",\"xrrf\":\"datasorwtakny\"}},{\"referenceName\":\"oskwujhskxx\",\"parameters\":{\"f\":\"dataysac\",\"csfaqypjcpdt\":\"datamcaxgtwpzqtimq\",\"xkujwn\":\"datatfpj\",\"for\":\"datanfoqw\"}},{\"referenceName\":\"xbeamipnsyedp\",\"parameters\":{\"qfbodifghdgs\":\"dataipslcfwgrz\",\"llvkorg\":\"datahncxoqxtjzdpll\"}}],\"arguments\":[\"dataxxoqy\"],\"getDebugInfo\":\"Always\",\"scriptPath\":\"dataaogmt\",\"scriptLinkedService\":{\"referenceName\":\"qxvmybqjlg\",\"parameters\":{\"h\":\"datansdccmdp\",\"q\":\"datajiqibmiwrhmzkx\",\"w\":\"datagshqxrpunuvfsl\",\"qjalhlpznmda\":\"datamho\"}},\"defines\":{\"zdipnhbs\":\"datazqz\",\"mnoasyyadyf\":\"datarlrcc\"},\"variables\":{\"gsva\":\"datallnzcm\",\"ddbenfjhfszmxpos\":\"datakuovwiwtykpr\",\"ljujpsubxggknmvk\":\"dataqscvyuldkpdlee\",\"oqy\":\"datai\"},\"queryTimeout\":717122256}") + "{\"storageLinkedServices\":[{\"referenceName\":\"iodnntol\",\"parameters\":{\"zvi\":\"dataptngr\",\"cdkomr\":\"dataxacxcac\"}},{\"referenceName\":\"u\",\"parameters\":{\"skyrhsijxm\":\"datajxpdqwywjnxd\"}}],\"arguments\":[\"dataymfxjsuwmbdt\",\"dataetrgzybprrapgh\"],\"getDebugInfo\":\"None\",\"scriptPath\":\"dataqapvnq\",\"scriptLinkedService\":{\"referenceName\":\"qkrsnxuezw\",\"parameters\":{\"rivcxxejpl\":\"dataadtvpgugtmtg\",\"sl\":\"datahkiyiqpiiodvzjk\"}},\"defines\":{\"hjqakacbcbrsnnv\":\"datapopubbwpsncxbkw\"},\"variables\":{\"bis\":\"datagfgtwqmtyfqutmj\",\"dorbufog\":\"dataurilqcsq\",\"auwojgvpqzvtgwlz\":\"datackbizqqajs\",\"br\":\"datacyvrbgi\"},\"queryTimeout\":240651564}") .toObject(HDInsightHiveActivityTypeProperties.class); - Assertions.assertEquals("kpfyojfwvmsfn", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("qxvmybqjlg", model.scriptLinkedService().referenceName()); - Assertions.assertEquals(717122256, model.queryTimeout()); + Assertions.assertEquals("iodnntol", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.NONE, model.getDebugInfo()); + Assertions.assertEquals("qkrsnxuezw", model.scriptLinkedService().referenceName()); + Assertions.assertEquals(240651564, model.queryTimeout()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { HDInsightHiveActivityTypeProperties model = new HDInsightHiveActivityTypeProperties() .withStorageLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("kpfyojfwvmsfn") - .withParameters(mapOf("gnyugsasgh", "datathdfqqzhqgmoe")), - new LinkedServiceReference().withReferenceName("i") - .withParameters(mapOf("stynjx", "datadxhxpqk", "ktiojitfa", "datandxrofwctjhdbidl", "etdrcm", - "datakpx", "xrrf", "datasorwtakny")), - new LinkedServiceReference().withReferenceName("oskwujhskxx") - .withParameters(mapOf("f", "dataysac", "csfaqypjcpdt", "datamcaxgtwpzqtimq", "xkujwn", "datatfpj", - "for", "datanfoqw")), - new LinkedServiceReference().withReferenceName("xbeamipnsyedp") - .withParameters(mapOf("qfbodifghdgs", "dataipslcfwgrz", "llvkorg", "datahncxoqxtjzdpll")))) - .withArguments(Arrays.asList("dataxxoqy")).withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) - .withScriptPath("dataaogmt") - .withScriptLinkedService(new LinkedServiceReference().withReferenceName("qxvmybqjlg") - .withParameters(mapOf("h", "datansdccmdp", "q", "datajiqibmiwrhmzkx", "w", "datagshqxrpunuvfsl", - "qjalhlpznmda", "datamho"))) - .withDefines(mapOf("zdipnhbs", "datazqz", "mnoasyyadyf", "datarlrcc")) - .withVariables(mapOf("gsva", "datallnzcm", "ddbenfjhfszmxpos", "datakuovwiwtykpr", "ljujpsubxggknmvk", - "dataqscvyuldkpdlee", "oqy", "datai")) - .withQueryTimeout(717122256); + new LinkedServiceReference().withReferenceName("iodnntol") + .withParameters(mapOf("zvi", "dataptngr", "cdkomr", "dataxacxcac")), + new LinkedServiceReference().withReferenceName("u") + .withParameters(mapOf("skyrhsijxm", "datajxpdqwywjnxd")))) + .withArguments(Arrays.asList("dataymfxjsuwmbdt", "dataetrgzybprrapgh")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.NONE).withScriptPath("dataqapvnq") + .withScriptLinkedService(new LinkedServiceReference().withReferenceName("qkrsnxuezw") + .withParameters(mapOf("rivcxxejpl", "dataadtvpgugtmtg", "sl", "datahkiyiqpiiodvzjk"))) + .withDefines(mapOf("hjqakacbcbrsnnv", "datapopubbwpsncxbkw")) + .withVariables(mapOf("bis", "datagfgtwqmtyfqutmj", "dorbufog", "dataurilqcsq", "auwojgvpqzvtgwlz", + "datackbizqqajs", "br", "datacyvrbgi")) + .withQueryTimeout(240651564); model = BinaryData.fromObject(model).toObject(HDInsightHiveActivityTypeProperties.class); - Assertions.assertEquals("kpfyojfwvmsfn", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("qxvmybqjlg", model.scriptLinkedService().referenceName()); - Assertions.assertEquals(717122256, model.queryTimeout()); + Assertions.assertEquals("iodnntol", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.NONE, model.getDebugInfo()); + Assertions.assertEquals("qkrsnxuezw", model.scriptLinkedService().referenceName()); + Assertions.assertEquals(240651564, model.queryTimeout()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTests.java index f01a8fa253308..92d3b07948bac 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTests.java @@ -23,64 +23,79 @@ public final class HDInsightMapReduceActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightMapReduceActivity model = BinaryData.fromString( - "{\"type\":\"HDInsightMapReduce\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"usvukgfzbykapme\",\"parameters\":{\"dtg\":\"datapvma\"}}],\"arguments\":[\"dataubnwymyewbfo\",\"datawv\",\"dataetj\",\"datanjbahxyfdd\"],\"getDebugInfo\":\"Failure\",\"className\":\"dataksr\",\"jarFilePath\":\"datajh\",\"jarLinkedService\":{\"referenceName\":\"rmptj\",\"parameters\":{\"pjracyxnzadfl\":\"dataaw\"}},\"jarLibs\":[\"datahgdzrcqdvapo\",\"dataeminerejrd\"],\"defines\":{\"mxhztdcadbmvqgqm\":\"datao\"}},\"linkedServiceName\":{\"referenceName\":\"rpagmhhwcyasz\",\"parameters\":{\"ni\":\"datamtcihupoelj\",\"awbsdeqqbdcbnrg\":\"datayoxajit\",\"mtgtnb\":\"datapnor\",\"rwldeinhnsd\":\"datasopuwesmxodyto\"}},\"policy\":{\"timeout\":\"dataaqtqnqmsiptzg\",\"retry\":\"dataujukenk\",\"retryIntervalInSeconds\":1282940357,\"secureInput\":true,\"secureOutput\":false,\"\":{\"kzxrmmoy\":\"dataobuihprvokodrpy\",\"xoubekafdxgtgcfk\":\"dataufkxy\"}},\"name\":\"aeu\",\"description\":\"htlk\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"qet\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\"],\"\":{\"btskmqkanuxj\":\"datawlwysrswzhciazw\",\"xlcdgkcufan\":\"datadyvzod\",\"sk\":\"datajlk\",\"qjwvqia\":\"datawuubafqzihmvw\"}}],\"userProperties\":[{\"name\":\"jzviv\",\"value\":\"datadlrtcfu\"},{\"name\":\"mzxhgwzby\",\"value\":\"datatwuuwehntjssj\"}],\"\":{\"h\":\"dataatpymvqi\",\"ityjznpryouujqye\":\"dataca\",\"vhc\":\"datazoivincnihm\"}}") + "{\"type\":\"HDInsightMapReduce\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"fjbmowqwodmdrdty\",\"parameters\":{\"ciapvcsw\":\"dataqwa\",\"aqmrimletjv\":\"datajrpcpgcltf\",\"ab\":\"dataptf\",\"zxupwrizkqnbiia\":\"datablhzfglpswg\"}}],\"arguments\":[\"datasjpq\",\"dataas\"],\"getDebugInfo\":\"Failure\",\"className\":\"dataosltch\",\"jarFilePath\":\"datauhvdrfh\",\"jarLinkedService\":{\"referenceName\":\"cxhnojfdiijch\",\"parameters\":{\"emnkykvruomw\":\"dataabtxr\"}},\"jarLibs\":[\"datatzffpcdqh\",\"datapz\",\"dataeqvkuvy\",\"datai\"],\"defines\":{\"ikstapkbd\":\"dataokolc\",\"mugkugwtg\":\"datay\"}},\"linkedServiceName\":{\"referenceName\":\"twayh\",\"parameters\":{\"eyliisatbs\":\"dataqv\",\"qvernqkeiyy\":\"datahtkdcufzxxqdntvf\"}},\"policy\":{\"timeout\":\"datatjoxwugdzwoczf\",\"retry\":\"datafcmpddzzdw\",\"retryIntervalInSeconds\":2049043657,\"secureInput\":true,\"secureOutput\":false,\"\":{\"lflyluwunsny\":\"datamuyyk\"}},\"name\":\"pmn\",\"description\":\"nbmgpomcrev\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"zveisd\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Completed\"],\"\":{\"eodix\":\"datapnr\",\"xr\":\"dataflxvsu\"}},{\"activity\":\"tcozfjsfrbjrbqcb\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Completed\",\"Completed\"],\"\":{\"fbwsicmhhvbovbl\":\"dataasm\",\"gd\":\"datafyle\",\"ezszlr\":\"dataiurfemnykfzsouo\"}},{\"activity\":\"z\",\"dependencyConditions\":[\"Skipped\",\"Failed\"],\"\":{\"bvlljkqlruh\":\"datajnpkxprbutyjfhj\",\"mfp\":\"datakkbfgrmscbmdrbt\"}},{\"activity\":\"redcvwsbsd\",\"dependencyConditions\":[\"Completed\"],\"\":{\"deuphgnfaanub\":\"datarqouyfcfd\",\"dhw\":\"dataeboelksghsowmvtm\",\"c\":\"datafbjhhpfj\",\"pwavdope\":\"datal\"}}],\"userProperties\":[{\"name\":\"ovrsrtldijgrbit\",\"value\":\"datadwuoxirziluzokx\"},{\"name\":\"hcjtwhwgbaj\",\"value\":\"datagctwamjjwvmugis\"},{\"name\":\"nyzmhhopedmk\",\"value\":\"datatdyxzg\"},{\"name\":\"qtgfbmocvb\",\"value\":\"datarvkqxhkhj\"}],\"\":{\"z\":\"dataqpxaajtiyrqtuz\"}}") .toObject(HDInsightMapReduceActivity.class); - Assertions.assertEquals("aeu", model.name()); - Assertions.assertEquals("htlk", model.description()); + Assertions.assertEquals("pmn", model.name()); + Assertions.assertEquals("nbmgpomcrev", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("qet", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("jzviv", model.userProperties().get(0).name()); - Assertions.assertEquals("rpagmhhwcyasz", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1282940357, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("zveisd", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ovrsrtldijgrbit", model.userProperties().get(0).name()); + Assertions.assertEquals("twayh", model.linkedServiceName().referenceName()); + Assertions.assertEquals(2049043657, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("usvukgfzbykapme", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("fjbmowqwodmdrdty", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("rmptj", model.jarLinkedService().referenceName()); + Assertions.assertEquals("cxhnojfdiijch", model.jarLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HDInsightMapReduceActivity model = new HDInsightMapReduceActivity().withName("aeu").withDescription("htlk") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("qet") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("jzviv").withValue("datadlrtcfu"), - new UserProperty().withName("mzxhgwzby").withValue("datatwuuwehntjssj"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("rpagmhhwcyasz") - .withParameters(mapOf("ni", "datamtcihupoelj", "awbsdeqqbdcbnrg", "datayoxajit", "mtgtnb", "datapnor", - "rwldeinhnsd", "datasopuwesmxodyto"))) - .withPolicy(new ActivityPolicy().withTimeout("dataaqtqnqmsiptzg").withRetry("dataujukenk") - .withRetryIntervalInSeconds(1282940357).withSecureInput(true).withSecureOutput(false) + HDInsightMapReduceActivity model = new HDInsightMapReduceActivity().withName("pmn") + .withDescription("nbmgpomcrev").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("zveisd") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, + DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("tcozfjsfrbjrbqcb") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, + DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("z") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("redcvwsbsd") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("ovrsrtldijgrbit").withValue("datadwuoxirziluzokx"), + new UserProperty().withName("hcjtwhwgbaj").withValue("datagctwamjjwvmugis"), + new UserProperty().withName("nyzmhhopedmk").withValue("datatdyxzg"), + new UserProperty().withName("qtgfbmocvb").withValue("datarvkqxhkhj"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("twayh") + .withParameters(mapOf("eyliisatbs", "dataqv", "qvernqkeiyy", "datahtkdcufzxxqdntvf"))) + .withPolicy(new ActivityPolicy().withTimeout("datatjoxwugdzwoczf").withRetry("datafcmpddzzdw") + .withRetryIntervalInSeconds(2049043657).withSecureInput(true).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withStorageLinkedServices(Arrays.asList(new LinkedServiceReference().withReferenceName("usvukgfzbykapme") - .withParameters(mapOf("dtg", "datapvma")))) - .withArguments(Arrays.asList("dataubnwymyewbfo", "datawv", "dataetj", "datanjbahxyfdd")) - .withGetDebugInfo(HDInsightActivityDebugInfoOption.FAILURE).withClassName("dataksr") - .withJarFilePath("datajh") - .withJarLinkedService(new LinkedServiceReference().withReferenceName("rmptj") - .withParameters(mapOf("pjracyxnzadfl", "dataaw"))) - .withJarLibs(Arrays.asList("datahgdzrcqdvapo", "dataeminerejrd")) - .withDefines(mapOf("mxhztdcadbmvqgqm", "datao")); + .withStorageLinkedServices(Arrays.asList(new LinkedServiceReference().withReferenceName("fjbmowqwodmdrdty") + .withParameters(mapOf("ciapvcsw", "dataqwa", "aqmrimletjv", "datajrpcpgcltf", "ab", "dataptf", + "zxupwrizkqnbiia", "datablhzfglpswg")))) + .withArguments(Arrays.asList("datasjpq", "dataas")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.FAILURE).withClassName("dataosltch") + .withJarFilePath("datauhvdrfh") + .withJarLinkedService(new LinkedServiceReference().withReferenceName("cxhnojfdiijch") + .withParameters(mapOf("emnkykvruomw", "dataabtxr"))) + .withJarLibs(Arrays.asList("datatzffpcdqh", "datapz", "dataeqvkuvy", "datai")) + .withDefines(mapOf("ikstapkbd", "dataokolc", "mugkugwtg", "datay")); model = BinaryData.fromObject(model).toObject(HDInsightMapReduceActivity.class); - Assertions.assertEquals("aeu", model.name()); - Assertions.assertEquals("htlk", model.description()); + Assertions.assertEquals("pmn", model.name()); + Assertions.assertEquals("nbmgpomcrev", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("qet", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("jzviv", model.userProperties().get(0).name()); - Assertions.assertEquals("rpagmhhwcyasz", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1282940357, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("zveisd", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ovrsrtldijgrbit", model.userProperties().get(0).name()); + Assertions.assertEquals("twayh", model.linkedServiceName().referenceName()); + Assertions.assertEquals(2049043657, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("usvukgfzbykapme", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("fjbmowqwodmdrdty", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("rmptj", model.jarLinkedService().referenceName()); + Assertions.assertEquals("cxhnojfdiijch", model.jarLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTypePropertiesTests.java index 33b698e434573..c3006170fe7f2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightMapReduceActivityTypePropertiesTests.java @@ -17,30 +17,34 @@ public final class HDInsightMapReduceActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightMapReduceActivityTypeProperties model = BinaryData.fromString( - "{\"storageLinkedServices\":[{\"referenceName\":\"uasutdhmilhzy\",\"parameters\":{\"dyrhmpokfxcbbxmx\":\"datacjunsd\",\"nlwgseou\":\"dataajyrace\",\"eainszruugeaenk\":\"datajetxup\"}}],\"arguments\":[\"datatsmzvaoz\",\"dataajalbsw\",\"databrllvvazu\"],\"getDebugInfo\":\"None\",\"className\":\"datazznw\",\"jarFilePath\":\"dataxzmszxyfaidzv\",\"jarLinkedService\":{\"referenceName\":\"vdivzjyxsjblb\",\"parameters\":{\"tmacnq\":\"databrqb\",\"qfbvbvkwryrzo\":\"datadmyduvawea\"}},\"jarLibs\":[\"datamhd\",\"datagobhltmpaycqqgrs\",\"datattosnz\"],\"defines\":{\"pvpbwtos\":\"dataifacrhpuzcag\",\"emocndb\":\"dataigu\",\"rko\":\"datae\",\"dafilaizcdugn\":\"datawyxodpcgdvytnbkv\"}}") + "{\"storageLinkedServices\":[{\"referenceName\":\"xngspazmxssb\",\"parameters\":{\"cakkewgz\":\"datanatbecuh\",\"sjlqt\":\"datao\",\"hqjdihjo\":\"datajewezcknpmev\",\"l\":\"datadwahehudicx\"}},{\"referenceName\":\"mh\",\"parameters\":{\"jqvmpzcjvogr\":\"datavxoiwb\"}}],\"arguments\":[\"dataopzyd\",\"dataspwwkdm\",\"datan\",\"datazdumjqdhrgwyq\"],\"getDebugInfo\":\"None\",\"className\":\"dataelmqkbepie\",\"jarFilePath\":\"datasveaerg\",\"jarLinkedService\":{\"referenceName\":\"brnlbfnuppwqks\",\"parameters\":{\"xyphdkxwstabgejo\":\"datatjfkjboyggrmzt\",\"tgoeayhojdgw\":\"dataveg\",\"dpxbwqgk\":\"datazcrssmbdjzc\"}},\"jarLibs\":[\"datadtj\"],\"defines\":{\"csflemxbma\":\"datavvuddnwjp\",\"vhlobjpumpq\":\"dataivopftzbk\"}}") .toObject(HDInsightMapReduceActivityTypeProperties.class); - Assertions.assertEquals("uasutdhmilhzy", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("xngspazmxssb", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.NONE, model.getDebugInfo()); - Assertions.assertEquals("vdivzjyxsjblb", model.jarLinkedService().referenceName()); + Assertions.assertEquals("brnlbfnuppwqks", model.jarLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { HDInsightMapReduceActivityTypeProperties model = new HDInsightMapReduceActivityTypeProperties() - .withStorageLinkedServices(Arrays.asList(new LinkedServiceReference().withReferenceName("uasutdhmilhzy") - .withParameters(mapOf("dyrhmpokfxcbbxmx", "datacjunsd", "nlwgseou", "dataajyrace", "eainszruugeaenk", - "datajetxup")))) - .withArguments(Arrays.asList("datatsmzvaoz", "dataajalbsw", "databrllvvazu")) - .withGetDebugInfo(HDInsightActivityDebugInfoOption.NONE).withClassName("datazznw") - .withJarFilePath("dataxzmszxyfaidzv") - .withJarLinkedService(new LinkedServiceReference().withReferenceName("vdivzjyxsjblb") - .withParameters(mapOf("tmacnq", "databrqb", "qfbvbvkwryrzo", "datadmyduvawea"))) - .withJarLibs(Arrays.asList("datamhd", "datagobhltmpaycqqgrs", "datattosnz")).withDefines(mapOf("pvpbwtos", - "dataifacrhpuzcag", "emocndb", "dataigu", "rko", "datae", "dafilaizcdugn", "datawyxodpcgdvytnbkv")); + .withStorageLinkedServices(Arrays.asList( + new LinkedServiceReference().withReferenceName("xngspazmxssb") + .withParameters(mapOf("cakkewgz", "datanatbecuh", "sjlqt", "datao", "hqjdihjo", "datajewezcknpmev", + "l", "datadwahehudicx")), + new LinkedServiceReference().withReferenceName("mh") + .withParameters(mapOf("jqvmpzcjvogr", "datavxoiwb")))) + .withArguments(Arrays.asList("dataopzyd", "dataspwwkdm", "datan", "datazdumjqdhrgwyq")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.NONE).withClassName("dataelmqkbepie") + .withJarFilePath("datasveaerg") + .withJarLinkedService(new LinkedServiceReference().withReferenceName("brnlbfnuppwqks") + .withParameters(mapOf("xyphdkxwstabgejo", "datatjfkjboyggrmzt", "tgoeayhojdgw", "dataveg", "dpxbwqgk", + "datazcrssmbdjzc"))) + .withJarLibs(Arrays.asList("datadtj")) + .withDefines(mapOf("csflemxbma", "datavvuddnwjp", "vhlobjpumpq", "dataivopftzbk")); model = BinaryData.fromObject(model).toObject(HDInsightMapReduceActivityTypeProperties.class); - Assertions.assertEquals("uasutdhmilhzy", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("xngspazmxssb", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.NONE, model.getDebugInfo()); - Assertions.assertEquals("vdivzjyxsjblb", model.jarLinkedService().referenceName()); + Assertions.assertEquals("brnlbfnuppwqks", model.jarLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTests.java index 4a2fd10e027ed..d5b9a5c207490 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTests.java @@ -23,66 +23,61 @@ public final class HDInsightPigActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightPigActivity model = BinaryData.fromString( - "{\"type\":\"HDInsightPig\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"eazzdjcsbkmal\",\"parameters\":{\"tajwsddyqdxpnz\":\"databf\",\"pggitxsyu\":\"dataukn\",\"auwt\":\"dataexivhjy\",\"oxazywijbvq\":\"dataqofrkfccqjenz\"}}],\"arguments\":\"datayjozbdwflxdw\",\"getDebugInfo\":\"Always\",\"scriptPath\":\"dataaieledmiupddlet\",\"scriptLinkedService\":{\"referenceName\":\"udcoktsgcy\",\"parameters\":{\"takgrebecxuu\":\"datam\",\"y\":\"dataeukluukxeja\"}},\"defines\":{\"tjx\":\"datawrv\",\"rt\":\"datattxvmbedvv\",\"abfzaaiihylzwzhl\":\"datam\",\"amtvooaacef\":\"datapmpletheknb\"}},\"linkedServiceName\":{\"referenceName\":\"nsvjctytyt\",\"parameters\":{\"ynvoytdtvkfqz\":\"datatuxvzywimmmmg\"}},\"policy\":{\"timeout\":\"datahpxdg\",\"retry\":\"dataowxcptxvxfwwv\",\"retryIntervalInSeconds\":1409869029,\"secureInput\":true,\"secureOutput\":true,\"\":{\"szjrihcamg\":\"datai\",\"ttitebmnx\":\"dataytv\",\"outqebpuoycaw\":\"datamoadjooernzlzzmy\"}},\"name\":\"txqx\",\"description\":\"fdx\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"skpb\",\"dependencyConditions\":[\"Failed\"],\"\":{\"a\":\"dataukztcuvwwfgjj\"}},{\"activity\":\"oceppwwilyx\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"zwybgaycjphoz\":\"dataifhjymqwjliivyat\",\"uoqnktl\":\"datamcypd\"}}],\"userProperties\":[{\"name\":\"idgw\",\"value\":\"datacosmhgz\"},{\"name\":\"pcgdkpyavfcpohlf\",\"value\":\"datasbaq\"},{\"name\":\"g\",\"value\":\"databjbbyoud\"},{\"name\":\"tdlkucxtyufsouh\",\"value\":\"datamcumuomdlspsb\"}],\"\":{\"lxoxwndfuyj\":\"datanygroqia\",\"ztdwxroambzprhp\":\"dataxgalcyflz\",\"agxhriruvjiz\":\"datawarzjzbbw\"}}") + "{\"type\":\"HDInsightPig\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"ljfkqubnnmzzcbcx\",\"parameters\":{\"dfxxaoyisky\":\"datahlw\"}}],\"arguments\":\"datasxakbqbwjt\",\"getDebugInfo\":\"Always\",\"scriptPath\":\"datamqkgc\",\"scriptLinkedService\":{\"referenceName\":\"dxuwe\",\"parameters\":{\"yfaprhf\":\"dataegszwsn\"}},\"defines\":{\"x\":\"databcribqdsz\",\"yknfrhbknragpnmc\":\"dataaqlyw\"}},\"linkedServiceName\":{\"referenceName\":\"dfxhfgdrzeg\",\"parameters\":{\"arufjfordzwbsk\":\"dataebzoujhijlduuvxk\",\"zlia\":\"dataexgnfjwfo\"}},\"policy\":{\"timeout\":\"dataowlsrxy\",\"retry\":\"datavzqxpmfhe\",\"retryIntervalInSeconds\":1691661362,\"secureInput\":false,\"secureOutput\":false,\"\":{\"sueutby\":\"datazhylzzuxqqrmckqm\",\"nrivsiwwsohsuh\":\"datalzgkzhbnbnjpie\",\"mjtanrirrnqloom\":\"dataikouvpcjyhsz\"}},\"name\":\"ywyqgaskap\",\"description\":\"vknquipipgvfch\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"borwzpfgk\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Succeeded\",\"Skipped\"],\"\":{\"lbfpteiir\":\"datab\",\"bhcfswpda\":\"dataeopx\",\"wvmhbizi\":\"datavcabsmrfxucgvz\"}}],\"userProperties\":[{\"name\":\"seexdboatvsfyxdf\",\"value\":\"dataqrnawnqy\"}],\"\":{\"ngxogqvwchynrdt\":\"dataxacojcaraxorqjbo\",\"oqvcjspjp\":\"dataaptwmawypkpbmi\",\"bmluy\":\"datatsgvvizayg\"}}") .toObject(HDInsightPigActivity.class); - Assertions.assertEquals("txqx", model.name()); - Assertions.assertEquals("fdx", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("skpb", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("idgw", model.userProperties().get(0).name()); - Assertions.assertEquals("nsvjctytyt", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1409869029, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("eazzdjcsbkmal", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("ywyqgaskap", model.name()); + Assertions.assertEquals("vknquipipgvfch", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("borwzpfgk", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("seexdboatvsfyxdf", model.userProperties().get(0).name()); + Assertions.assertEquals("dfxhfgdrzeg", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1691661362, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("ljfkqubnnmzzcbcx", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("udcoktsgcy", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("dxuwe", model.scriptLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HDInsightPigActivity model = new HDInsightPigActivity().withName("txqx").withDescription("fdx") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("skpb") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("oceppwwilyx") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("idgw").withValue("datacosmhgz"), - new UserProperty().withName("pcgdkpyavfcpohlf").withValue("datasbaq"), - new UserProperty().withName("g").withValue("databjbbyoud"), - new UserProperty().withName("tdlkucxtyufsouh").withValue("datamcumuomdlspsb"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("nsvjctytyt") - .withParameters(mapOf("ynvoytdtvkfqz", "datatuxvzywimmmmg"))) - .withPolicy(new ActivityPolicy().withTimeout("datahpxdg").withRetry("dataowxcptxvxfwwv") - .withRetryIntervalInSeconds(1409869029).withSecureInput(true).withSecureOutput(true) + HDInsightPigActivity model = new HDInsightPigActivity().withName("ywyqgaskap").withDescription("vknquipipgvfch") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("borwzpfgk") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, + DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("seexdboatvsfyxdf").withValue("dataqrnawnqy"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dfxhfgdrzeg") + .withParameters(mapOf("arufjfordzwbsk", "dataebzoujhijlduuvxk", "zlia", "dataexgnfjwfo"))) + .withPolicy(new ActivityPolicy().withTimeout("dataowlsrxy").withRetry("datavzqxpmfhe") + .withRetryIntervalInSeconds(1691661362).withSecureInput(false).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withStorageLinkedServices(Arrays.asList(new LinkedServiceReference().withReferenceName("eazzdjcsbkmal") - .withParameters(mapOf("tajwsddyqdxpnz", "databf", "pggitxsyu", "dataukn", "auwt", "dataexivhjy", - "oxazywijbvq", "dataqofrkfccqjenz")))) - .withArguments("datayjozbdwflxdw").withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) - .withScriptPath("dataaieledmiupddlet") - .withScriptLinkedService(new LinkedServiceReference().withReferenceName("udcoktsgcy") - .withParameters(mapOf("takgrebecxuu", "datam", "y", "dataeukluukxeja"))) - .withDefines(mapOf("tjx", "datawrv", "rt", "datattxvmbedvv", "abfzaaiihylzwzhl", "datam", "amtvooaacef", - "datapmpletheknb")); + .withStorageLinkedServices(Arrays.asList(new LinkedServiceReference().withReferenceName("ljfkqubnnmzzcbcx") + .withParameters(mapOf("dfxxaoyisky", "datahlw")))) + .withArguments("datasxakbqbwjt").withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) + .withScriptPath("datamqkgc") + .withScriptLinkedService( + new LinkedServiceReference().withReferenceName("dxuwe").withParameters(mapOf("yfaprhf", "dataegszwsn"))) + .withDefines(mapOf("x", "databcribqdsz", "yknfrhbknragpnmc", "dataaqlyw")); model = BinaryData.fromObject(model).toObject(HDInsightPigActivity.class); - Assertions.assertEquals("txqx", model.name()); - Assertions.assertEquals("fdx", model.description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("skpb", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("idgw", model.userProperties().get(0).name()); - Assertions.assertEquals("nsvjctytyt", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1409869029, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("eazzdjcsbkmal", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("ywyqgaskap", model.name()); + Assertions.assertEquals("vknquipipgvfch", model.description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("borwzpfgk", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("seexdboatvsfyxdf", model.userProperties().get(0).name()); + Assertions.assertEquals("dfxhfgdrzeg", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1691661362, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("ljfkqubnnmzzcbcx", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("udcoktsgcy", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("dxuwe", model.scriptLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTypePropertiesTests.java index 8aa45bf4c0f60..789bdb9fda804 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightPigActivityTypePropertiesTests.java @@ -17,34 +17,36 @@ public final class HDInsightPigActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightPigActivityTypeProperties model = BinaryData.fromString( - "{\"storageLinkedServices\":[{\"referenceName\":\"xxw\",\"parameters\":{\"sxj\":\"datad\",\"rolagbellp\":\"dataleqsk\"}},{\"referenceName\":\"pfbrsmy\",\"parameters\":{\"f\":\"datadf\",\"gystohur\":\"datagowhnvcqhmu\",\"nxxw\":\"datatodeertyijlvcmp\",\"khvlsahjdebwyqn\":\"dataledbdkwz\"}},{\"referenceName\":\"uszilkrcpxlkih\",\"parameters\":{\"wriihwxc\":\"databvhmsvcmcehtr\"}},{\"referenceName\":\"yyfgkrpjo\",\"parameters\":{\"yigfdppgkk\":\"datansqjnuiiytyarp\"}}],\"arguments\":\"dataygjldljgd\",\"getDebugInfo\":\"Always\",\"scriptPath\":\"datatseznowzfxkof\",\"scriptLinkedService\":{\"referenceName\":\"fwculsbnapzfdzmr\",\"parameters\":{\"rj\":\"databclj\",\"l\":\"dataawnz\"}},\"defines\":{\"jbasmrdpb\":\"datafs\"}}") + "{\"storageLinkedServices\":[{\"referenceName\":\"zvqpnjqpwxf\",\"parameters\":{\"qopwmbdleegwlh\":\"datajpayvlnzwi\",\"ueizhydjkmxbghx\":\"datan\",\"uqkb\":\"dataotlfzbjs\",\"qkgausc\":\"datadicoaysar\"}},{\"referenceName\":\"gdyfyjeexmlkxvfe\",\"parameters\":{\"kkva\":\"dataojavqez\",\"lokhmkqyfatddwf\":\"datayyowjpsmnxccasu\",\"lwd\":\"datafjfatoilmwzkxa\",\"hvioccszdaxafu\":\"datatjfnmxzu\"}},{\"referenceName\":\"ccnqfwobnblu\",\"parameters\":{\"rfyetljr\":\"datafimlozlfdxj\"}},{\"referenceName\":\"cgeorm\",\"parameters\":{\"twiocuha\":\"datawcqhaonmfnf\"}}],\"arguments\":\"dataielhtukhei\",\"getDebugInfo\":\"Always\",\"scriptPath\":\"datatjcgdpqkfxd\",\"scriptLinkedService\":{\"referenceName\":\"dvrglql\",\"parameters\":{\"qo\":\"dataese\"}},\"defines\":{\"ckjuwkkvarff\":\"datauqqahmzlet\"}}") .toObject(HDInsightPigActivityTypeProperties.class); - Assertions.assertEquals("xxw", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("zvqpnjqpwxf", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("fwculsbnapzfdzmr", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("dvrglql", model.scriptLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { HDInsightPigActivityTypeProperties model = new HDInsightPigActivityTypeProperties() .withStorageLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("xxw") - .withParameters(mapOf("sxj", "datad", "rolagbellp", "dataleqsk")), - new LinkedServiceReference().withReferenceName("pfbrsmy") - .withParameters(mapOf("f", "datadf", "gystohur", "datagowhnvcqhmu", "nxxw", "datatodeertyijlvcmp", - "khvlsahjdebwyqn", "dataledbdkwz")), - new LinkedServiceReference().withReferenceName("uszilkrcpxlkih") - .withParameters(mapOf("wriihwxc", "databvhmsvcmcehtr")), - new LinkedServiceReference() - .withReferenceName("yyfgkrpjo").withParameters(mapOf("yigfdppgkk", "datansqjnuiiytyarp")))) - .withArguments("dataygjldljgd").withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) - .withScriptPath("datatseznowzfxkof").withScriptLinkedService(new LinkedServiceReference() - .withReferenceName("fwculsbnapzfdzmr").withParameters(mapOf("rj", "databclj", "l", "dataawnz"))) - .withDefines(mapOf("jbasmrdpb", "datafs")); + new LinkedServiceReference().withReferenceName("zvqpnjqpwxf") + .withParameters(mapOf("qopwmbdleegwlh", "datajpayvlnzwi", "ueizhydjkmxbghx", "datan", "uqkb", + "dataotlfzbjs", "qkgausc", "datadicoaysar")), + new LinkedServiceReference().withReferenceName("gdyfyjeexmlkxvfe") + .withParameters(mapOf("kkva", "dataojavqez", "lokhmkqyfatddwf", "datayyowjpsmnxccasu", "lwd", + "datafjfatoilmwzkxa", "hvioccszdaxafu", "datatjfnmxzu")), + new LinkedServiceReference().withReferenceName("ccnqfwobnblu") + .withParameters(mapOf("rfyetljr", "datafimlozlfdxj")), + new LinkedServiceReference().withReferenceName("cgeorm") + .withParameters(mapOf("twiocuha", "datawcqhaonmfnf")))) + .withArguments("dataielhtukhei").withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) + .withScriptPath("datatjcgdpqkfxd") + .withScriptLinkedService( + new LinkedServiceReference().withReferenceName("dvrglql").withParameters(mapOf("qo", "dataese"))) + .withDefines(mapOf("ckjuwkkvarff", "datauqqahmzlet")); model = BinaryData.fromObject(model).toObject(HDInsightPigActivityTypeProperties.class); - Assertions.assertEquals("xxw", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals("zvqpnjqpwxf", model.storageLinkedServices().get(0).referenceName()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("fwculsbnapzfdzmr", model.scriptLinkedService().referenceName()); + Assertions.assertEquals("dvrglql", model.scriptLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTests.java index c760abc092563..d373c05093378 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTests.java @@ -23,75 +23,68 @@ public final class HDInsightSparkActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightSparkActivity model = BinaryData.fromString( - "{\"type\":\"HDInsightSpark\",\"typeProperties\":{\"rootPath\":\"datau\",\"entryFilePath\":\"dataeweeeg\",\"arguments\":[\"datas\",\"dataryfap\"],\"getDebugInfo\":\"Failure\",\"sparkJobLinkedService\":{\"referenceName\":\"dhbcribqdszuxh\",\"parameters\":{\"nmcqudfxh\":\"dataywtyknfrhbknrag\",\"zegm\":\"datagd\",\"jhijlduuv\":\"datauebzo\"}},\"className\":\"l\",\"proxyUser\":\"dataufjfordzwbskfex\",\"sparkConfig\":{\"jzliaqpmowlsrxy\":\"datajwf\",\"pqlonz\":\"dataevzqxpmfheht\",\"qqrmck\":\"datahylzzu\",\"sueutby\":\"datam\"}},\"linkedServiceName\":{\"referenceName\":\"zg\",\"parameters\":{\"ws\":\"databnbnjpiecnrivsi\",\"uvpcjyh\":\"datahsuhkik\"}},\"policy\":{\"timeout\":\"datamjtanrirrnqloom\",\"retry\":\"datawyqgaskapgdvknq\",\"retryIntervalInSeconds\":730899433,\"secureInput\":false,\"secureOutput\":false,\"\":{\"jzborwzpfgks\":\"datahzcpv\",\"fpteiirneopxd\":\"datavbrxlsbgl\"}},\"name\":\"hcfs\",\"description\":\"darvcabsm\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"zgwvmhbiziijd\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"feqrn\":\"dataoatvsfyx\",\"xorqjboyngxo\":\"datawnqyagfxacojcar\",\"wchynrdtn\":\"dataq\"}},{\"activity\":\"ptwmawypkpbmid\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"tsgvvizayg\":\"datapjp\",\"cgzvqpnjqpwx\":\"databmluy\",\"pa\":\"datakui\",\"dleegwlhanyueizh\":\"datavlnzwicqopwm\"}},{\"activity\":\"djkmxbghxiotlfzb\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"aysargqkgaus\":\"datakbsdic\"}},{\"activity\":\"ugdyfyjeex\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Skipped\"],\"\":{\"ekkvayyyowj\":\"datajdgojavqe\",\"s\":\"datasmnxcc\",\"a\":\"datahlokhmkqy\",\"lmwzkxaglwd\":\"dataddwfhfjfato\"}}],\"userProperties\":[{\"name\":\"fnmxzukhv\",\"value\":\"dataoc\"},{\"name\":\"szda\",\"value\":\"dataafuhccnqf\"}],\"\":{\"utmfim\":\"datanbl\",\"cgeorm\":\"dataozlfdxjirfyetljr\",\"wcqhaonmfnf\":\"datai\",\"qielhtuk\":\"datatwiocuha\"}}") + "{\"type\":\"HDInsightSpark\",\"typeProperties\":{\"rootPath\":\"datachzuap\",\"entryFilePath\":\"datahfhuuizyeyfki\",\"arguments\":[\"datayjffpuuyky\"],\"getDebugInfo\":\"Failure\",\"sparkJobLinkedService\":{\"referenceName\":\"eymlctnnsjcuf\",\"parameters\":{\"xg\":\"dataaxuvazzptlda\",\"htykebtvn\":\"dataxhbnqyewinle\",\"riehooxqkc\":\"datadcclpbhntoiviue\",\"kyiqjtx\":\"datayydtnl\"}},\"className\":\"grf\",\"proxyUser\":\"datadkj\",\"sparkConfig\":{\"vii\":\"datahivvoczsryp\",\"usnhnnekhfdlbc\":\"datajjqpsbbxkeygmqnu\",\"cccydldavozmibtk\":\"datacwfcbug\"}},\"linkedServiceName\":{\"referenceName\":\"tpgll\",\"parameters\":{\"zjwizrul\":\"dataankxxwtnrrohlkgz\",\"obenaah\":\"datakwyldttggcpqmk\",\"irhstwpbvwu\":\"datajnmychtvp\",\"nvsu\":\"datahpphjimoecqpqk\"}},\"policy\":{\"timeout\":\"dataxdlrjspxot\",\"retry\":\"datag\",\"retryIntervalInSeconds\":1487197700,\"secureInput\":true,\"secureOutput\":true,\"\":{\"lcdqvunvnggqacf\":\"dataldsftm\",\"dzruuscbs\":\"dataruwqbe\",\"ofoc\":\"datatjdioevifzqqs\",\"uuzpiooacjxsof\":\"datapphwv\"}},\"name\":\"iritpqqpynrl\",\"description\":\"u\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"jnfyub\",\"dependencyConditions\":[\"Completed\",\"Failed\"],\"\":{\"pkleieafpvbsllyo\":\"dataqqnumpnav\"}},{\"activity\":\"iadhnbofeucct\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Failed\",\"Succeeded\"],\"\":{\"rqsobusur\":\"dataylskbvvwdf\",\"knywxpmefbnccbvc\":\"datavjdxlbsnskcksfx\"}}],\"userProperties\":[{\"name\":\"kmifyx\",\"value\":\"datanu\"}],\"\":{\"w\":\"datasfnbtqdr\",\"yhnfqnekpxd\":\"dataidroidhbulvki\",\"jahnsmktk\":\"datadeahfg\"}}") .toObject(HDInsightSparkActivity.class); - Assertions.assertEquals("hcfs", model.name()); - Assertions.assertEquals("darvcabsm", model.description()); + Assertions.assertEquals("iritpqqpynrl", model.name()); + Assertions.assertEquals("u", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("zgwvmhbiziijd", model.dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("jnfyub", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("fnmxzukhv", model.userProperties().get(0).name()); - Assertions.assertEquals("zg", model.linkedServiceName().referenceName()); - Assertions.assertEquals(730899433, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("kmifyx", model.userProperties().get(0).name()); + Assertions.assertEquals("tpgll", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1487197700, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(true, model.policy().secureOutput()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("dhbcribqdszuxh", model.sparkJobLinkedService().referenceName()); - Assertions.assertEquals("l", model.className()); + Assertions.assertEquals("eymlctnnsjcuf", model.sparkJobLinkedService().referenceName()); + Assertions.assertEquals("grf", model.className()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { HDInsightSparkActivity model - = new HDInsightSparkActivity().withName("hcfs").withDescription("darvcabsm").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + = new HDInsightSparkActivity().withName("iritpqqpynrl").withDescription("u").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("zgwvmhbiziijd") + new ActivityDependency().withActivity("jnfyub") .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ptwmawypkpbmid") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("djkmxbghxiotlfzb") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ugdyfyjeex") + new ActivityDependency().withActivity("iadhnbofeucct") .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + DependencyCondition.SKIPPED, DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("fnmxzukhv").withValue("dataoc"), - new UserProperty().withName("szda").withValue("dataafuhccnqf"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("zg") - .withParameters(mapOf("ws", "databnbnjpiecnrivsi", "uvpcjyh", "datahsuhkik"))) - .withPolicy(new ActivityPolicy().withTimeout("datamjtanrirrnqloom").withRetry("datawyqgaskapgdvknq") - .withRetryIntervalInSeconds(730899433).withSecureInput(false).withSecureOutput(false) + .withUserProperties(Arrays.asList(new UserProperty().withName("kmifyx").withValue("datanu"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("tpgll") + .withParameters(mapOf("zjwizrul", "dataankxxwtnrrohlkgz", "obenaah", "datakwyldttggcpqmk", + "irhstwpbvwu", "datajnmychtvp", "nvsu", "datahpphjimoecqpqk"))) + .withPolicy(new ActivityPolicy().withTimeout("dataxdlrjspxot").withRetry("datag") + .withRetryIntervalInSeconds(1487197700).withSecureInput(true).withSecureOutput(true) .withAdditionalProperties(mapOf())) - .withRootPath("datau").withEntryFilePath("dataeweeeg") - .withArguments(Arrays.asList("datas", "dataryfap")) + .withRootPath("datachzuap").withEntryFilePath("datahfhuuizyeyfki") + .withArguments(Arrays.asList("datayjffpuuyky")) .withGetDebugInfo(HDInsightActivityDebugInfoOption.FAILURE) - .withSparkJobLinkedService( - new LinkedServiceReference().withReferenceName("dhbcribqdszuxh").withParameters( - mapOf("nmcqudfxh", "dataywtyknfrhbknrag", "zegm", "datagd", "jhijlduuv", "datauebzo"))) - .withClassName("l").withProxyUser("dataufjfordzwbskfex").withSparkConfig(mapOf("jzliaqpmowlsrxy", - "datajwf", "pqlonz", "dataevzqxpmfheht", "qqrmck", "datahylzzu", "sueutby", "datam")); + .withSparkJobLinkedService(new LinkedServiceReference().withReferenceName("eymlctnnsjcuf") + .withParameters(mapOf("xg", "dataaxuvazzptlda", "htykebtvn", "dataxhbnqyewinle", "riehooxqkc", + "datadcclpbhntoiviue", "kyiqjtx", "datayydtnl"))) + .withClassName("grf").withProxyUser("datadkj").withSparkConfig(mapOf("vii", "datahivvoczsryp", + "usnhnnekhfdlbc", "datajjqpsbbxkeygmqnu", "cccydldavozmibtk", "datacwfcbug")); model = BinaryData.fromObject(model).toObject(HDInsightSparkActivity.class); - Assertions.assertEquals("hcfs", model.name()); - Assertions.assertEquals("darvcabsm", model.description()); + Assertions.assertEquals("iritpqqpynrl", model.name()); + Assertions.assertEquals("u", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("zgwvmhbiziijd", model.dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("jnfyub", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("fnmxzukhv", model.userProperties().get(0).name()); - Assertions.assertEquals("zg", model.linkedServiceName().referenceName()); - Assertions.assertEquals(730899433, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("kmifyx", model.userProperties().get(0).name()); + Assertions.assertEquals("tpgll", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1487197700, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(true, model.policy().secureOutput()); Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("dhbcribqdszuxh", model.sparkJobLinkedService().referenceName()); - Assertions.assertEquals("l", model.className()); + Assertions.assertEquals("eymlctnnsjcuf", model.sparkJobLinkedService().referenceName()); + Assertions.assertEquals("grf", model.className()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTypePropertiesTests.java index 9538596b1dc3c..aa91c6eb672f5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightSparkActivityTypePropertiesTests.java @@ -17,27 +17,27 @@ public final class HDInsightSparkActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightSparkActivityTypeProperties model = BinaryData.fromString( - "{\"rootPath\":\"dataeimwhot\",\"entryFilePath\":\"datacgdpqkfxdqmdvrgl\",\"arguments\":[\"datamke\",\"dataeyqotmj\",\"dataqqahmzleta\"],\"getDebugInfo\":\"Always\",\"sparkJobLinkedService\":{\"referenceName\":\"wkkvarffjuefjbmo\",\"parameters\":{\"qwa\":\"dataodmdrdtywa\",\"jrpcpgcltf\":\"dataciapvcsw\",\"ptf\":\"dataaqmrimletjv\",\"blhzfglpswg\":\"dataab\"}},\"className\":\"xup\",\"proxyUser\":\"dataizkqnbiiand\",\"sparkConfig\":{\"ku\":\"datapqha\"}}") + "{\"rootPath\":\"datalq\",\"entryFilePath\":\"dataxjdolobtzr\",\"arguments\":[\"datalaurviyntcblpbz\",\"dataqtfbjkbfktelbl\",\"dataungrkjbdaxttoenf\"],\"getDebugInfo\":\"Always\",\"sparkJobLinkedService\":{\"referenceName\":\"ijfywmmqzb\",\"parameters\":{\"ecktcwgnkxjd\":\"datajw\"}},\"className\":\"diundzawotpiakle\",\"proxyUser\":\"dataaiabfntrm\",\"sparkConfig\":{\"fn\":\"datawmfeudcgdljbnfwd\"}}") .toObject(HDInsightSparkActivityTypeProperties.class); Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("wkkvarffjuefjbmo", model.sparkJobLinkedService().referenceName()); - Assertions.assertEquals("xup", model.className()); + Assertions.assertEquals("ijfywmmqzb", model.sparkJobLinkedService().referenceName()); + Assertions.assertEquals("diundzawotpiakle", model.className()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HDInsightSparkActivityTypeProperties model = new HDInsightSparkActivityTypeProperties() - .withRootPath("dataeimwhot").withEntryFilePath("datacgdpqkfxdqmdvrgl") - .withArguments(Arrays.asList("datamke", "dataeyqotmj", "dataqqahmzleta")) - .withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) - .withSparkJobLinkedService(new LinkedServiceReference().withReferenceName("wkkvarffjuefjbmo") - .withParameters(mapOf("qwa", "dataodmdrdtywa", "jrpcpgcltf", "dataciapvcsw", "ptf", "dataaqmrimletjv", - "blhzfglpswg", "dataab"))) - .withClassName("xup").withProxyUser("dataizkqnbiiand").withSparkConfig(mapOf("ku", "datapqha")); + HDInsightSparkActivityTypeProperties model + = new HDInsightSparkActivityTypeProperties().withRootPath("datalq").withEntryFilePath("dataxjdolobtzr") + .withArguments(Arrays.asList("datalaurviyntcblpbz", "dataqtfbjkbfktelbl", "dataungrkjbdaxttoenf")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS) + .withSparkJobLinkedService(new LinkedServiceReference().withReferenceName("ijfywmmqzb") + .withParameters(mapOf("ecktcwgnkxjd", "datajw"))) + .withClassName("diundzawotpiakle").withProxyUser("dataaiabfntrm") + .withSparkConfig(mapOf("fn", "datawmfeudcgdljbnfwd")); model = BinaryData.fromObject(model).toObject(HDInsightSparkActivityTypeProperties.class); Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); - Assertions.assertEquals("wkkvarffjuefjbmo", model.sparkJobLinkedService().referenceName()); - Assertions.assertEquals("xup", model.className()); + Assertions.assertEquals("ijfywmmqzb", model.sparkJobLinkedService().referenceName()); + Assertions.assertEquals("diundzawotpiakle", model.className()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTests.java index f1e49acea1ac6..d249b35b304e6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTests.java @@ -23,76 +23,80 @@ public final class HDInsightStreamingActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightStreamingActivity model = BinaryData.fromString( - "{\"type\":\"HDInsightStreaming\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"ljgaykaqwnk\",\"parameters\":{\"uqglgvmuews\":\"dataecjz\",\"e\":\"datahqgjvchl\",\"wizjraksah\":\"datafbtczzjfzjo\"}},{\"referenceName\":\"qpukltfknroxmi\",\"parameters\":{\"vukcf\":\"datazczpfvq\"}}],\"arguments\":[\"datazknhcvxf\",\"datazhgrn\",\"datawysdmovbvnjyqq\"],\"getDebugInfo\":\"Failure\",\"mapper\":\"datagz\",\"reducer\":\"dataykc\",\"input\":\"dataol\",\"output\":\"datadwrggytyvo\",\"filePaths\":[\"datajbyjgobzj\",\"dataukfwmhzarrfttx\",\"dataifrjgvhone\"],\"fileLinkedService\":{\"referenceName\":\"v\",\"parameters\":{\"jowvzyoehlj\":\"datalmkfvsol\"}},\"combiner\":\"datayguothnucqktu\",\"commandEnvironment\":[\"datagjtpriicteyd\"],\"defines\":{\"o\":\"datahmivtuphwwyi\",\"nrziwkcpxgjmyou\":\"datafxfteo\"}},\"linkedServiceName\":{\"referenceName\":\"apua\",\"parameters\":{\"gmpohuvasxjzklq\":\"datadikkmyrsvtrtx\",\"cwnefdehptlnw\":\"datagjukntknjhywgzi\",\"uwbneszxn\":\"datal\",\"iwp\":\"datageuoihtik\"}},\"policy\":{\"timeout\":\"dataav\",\"retry\":\"datadsuwct\",\"retryIntervalInSeconds\":737405762,\"secureInput\":false,\"secureOutput\":true,\"\":{\"zivhk\":\"dataljyreyhpbtwz\"}},\"name\":\"dcjym\",\"description\":\"ldbuyuxgephv\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"sdoxhyi\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\"],\"\":{\"p\":\"datauffkmtiuxynkh\",\"hpmsdgmxwfodvzp\":\"dataqlhzdbbitpgr\",\"evhnqtba\":\"datamojxbvgiee\",\"xmqudnqcbbbhin\":\"datavjodgplagwvgb\"}},{\"activity\":\"syszl\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Completed\",\"Succeeded\"],\"\":{\"yyzqdnrgnybpqw\":\"dataosh\",\"cgsdcpmclkuy\":\"datajbmkhxunqrvqt\",\"cqrssqwz\":\"dataabhy\"}},{\"activity\":\"dzuxlgmypngy\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Failed\",\"Succeeded\"],\"\":{\"mcg\":\"datapjvbzablmpntjlzk\"}}],\"userProperties\":[{\"name\":\"bfbg\",\"value\":\"datawjqwlvsefvkx\"}],\"\":{\"zm\":\"datagbnqmhr\",\"lqrkstjdjdasomxw\":\"dataxwgrflqbugxudsmd\"}}") + "{\"type\":\"HDInsightStreaming\",\"typeProperties\":{\"storageLinkedServices\":[{\"referenceName\":\"zydyl\",\"parameters\":{\"e\":\"dataisddmfom\"}},{\"referenceName\":\"vfnxxkmrf\",\"parameters\":{\"rfhfjwikva\":\"datakprbmca\",\"uhqilvajctpw\":\"datajx\"}},{\"referenceName\":\"frfardjqwdr\",\"parameters\":{\"syd\":\"dataoo\",\"lsxewocwmadyelw\":\"datavdqcmegwajjzxcq\"}},{\"referenceName\":\"ll\",\"parameters\":{\"zqldakbijcxctn\":\"datahanfjrdcaw\",\"avoyhnollnuhoc\":\"datajxyczzwhwsidnq\",\"nhujcunyuasstok\":\"databeoxoewprtzofcu\"}}],\"arguments\":[\"datay\",\"datayblmcen\",\"datactc\"],\"getDebugInfo\":\"Always\",\"mapper\":\"datauplxksphzyu\",\"reducer\":\"datayts\",\"input\":\"datafmajswr\",\"output\":\"datadst\",\"filePaths\":[\"datakqzulosl\"],\"fileLinkedService\":{\"referenceName\":\"fnhej\",\"parameters\":{\"lvi\":\"dataugyuxcffbs\",\"zwtzdyz\":\"datas\",\"sojdmesoxj\":\"datafgn\",\"biouuqox\":\"datapwbgfhjwchvu\"}},\"combiner\":\"datatws\",\"commandEnvironment\":[\"datadghlokvisqzmhei\",\"datasioonnfjgrtkeg\",\"datatvwffvbvuxpyveav\",\"datatzbzykks\"],\"defines\":{\"zdcj\":\"dataan\"}},\"linkedServiceName\":{\"referenceName\":\"zpmwxvfrmvtwwb\",\"parameters\":{\"lqydhhypuvhucawm\":\"datavfosbrqeyw\"}},\"policy\":{\"timeout\":\"datajll\",\"retry\":\"databqvnbxgk\",\"retryIntervalInSeconds\":1536315462,\"secureInput\":false,\"secureOutput\":false,\"\":{\"qvzwummw\":\"databytzh\"}},\"name\":\"ax\",\"description\":\"bihgcduj\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"rbtrmif\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Completed\"],\"\":{\"fkufv\":\"dataam\"}},{\"activity\":\"bcinuby\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Completed\"],\"\":{\"ajpyuwrggfgll\":\"datakozlpsf\",\"gzvytb\":\"datasu\",\"j\":\"dataqmxkuyyrcqsyqhu\"}}],\"userProperties\":[{\"name\":\"tgmmfdfdqoepw\",\"value\":\"datayeupkpyzaenar\"},{\"name\":\"yrlqiykhoygfgch\",\"value\":\"datacbt\"}],\"\":{\"yklyhmymkcc\":\"dataskgu\"}}") .toObject(HDInsightStreamingActivity.class); - Assertions.assertEquals("dcjym", model.name()); - Assertions.assertEquals("ldbuyuxgephv", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("sdoxhyi", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("bfbg", model.userProperties().get(0).name()); - Assertions.assertEquals("apua", model.linkedServiceName().referenceName()); - Assertions.assertEquals(737405762, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("ax", model.name()); + Assertions.assertEquals("bihgcduj", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("rbtrmif", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("tgmmfdfdqoepw", model.userProperties().get(0).name()); + Assertions.assertEquals("zpmwxvfrmvtwwb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1536315462, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("ljgaykaqwnk", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("v", model.fileLinkedService().referenceName()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("zydyl", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); + Assertions.assertEquals("fnhej", model.fileLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HDInsightStreamingActivity model - = new HDInsightStreamingActivity().withName("dcjym").withDescription("ldbuyuxgephv") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("sdoxhyi") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + HDInsightStreamingActivity model = new HDInsightStreamingActivity().withName("ax").withDescription("bihgcduj") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("rbtrmif") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.FAILED, + DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("syszl") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dzuxlgmypngy") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("bfbg").withValue("datawjqwlvsefvkx"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("apua") - .withParameters(mapOf("gmpohuvasxjzklq", "datadikkmyrsvtrtx", "cwnefdehptlnw", - "datagjukntknjhywgzi", "uwbneszxn", "datal", "iwp", "datageuoihtik"))) - .withPolicy(new ActivityPolicy().withTimeout("dataav").withRetry("datadsuwct") - .withRetryIntervalInSeconds(737405762).withSecureInput(false).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withStorageLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("ljgaykaqwnk").withParameters( - mapOf("uqglgvmuews", "dataecjz", "e", "datahqgjvchl", "wizjraksah", "datafbtczzjfzjo")), - new LinkedServiceReference().withReferenceName("qpukltfknroxmi") - .withParameters(mapOf("vukcf", "datazczpfvq")))) - .withArguments(Arrays.asList("datazknhcvxf", "datazhgrn", "datawysdmovbvnjyqq")) - .withGetDebugInfo(HDInsightActivityDebugInfoOption.FAILURE).withMapper("datagz").withReducer("dataykc") - .withInput("dataol").withOutput("datadwrggytyvo") - .withFilePaths(Arrays.asList("datajbyjgobzj", "dataukfwmhzarrfttx", "dataifrjgvhone")) - .withFileLinkedService(new LinkedServiceReference().withReferenceName("v") - .withParameters(mapOf("jowvzyoehlj", "datalmkfvsol"))) - .withCombiner("datayguothnucqktu").withCommandEnvironment(Arrays.asList("datagjtpriicteyd")) - .withDefines(mapOf("o", "datahmivtuphwwyi", "nrziwkcpxgjmyou", "datafxfteo")); + new ActivityDependency().withActivity("bcinuby") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED, + DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("tgmmfdfdqoepw").withValue("datayeupkpyzaenar"), + new UserProperty().withName("yrlqiykhoygfgch").withValue("datacbt"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("zpmwxvfrmvtwwb") + .withParameters(mapOf("lqydhhypuvhucawm", "datavfosbrqeyw"))) + .withPolicy(new ActivityPolicy().withTimeout("datajll").withRetry("databqvnbxgk") + .withRetryIntervalInSeconds(1536315462).withSecureInput(false).withSecureOutput(false) + .withAdditionalProperties(mapOf())) + .withStorageLinkedServices(Arrays.asList( + new LinkedServiceReference().withReferenceName("zydyl").withParameters(mapOf("e", "dataisddmfom")), + new LinkedServiceReference().withReferenceName("vfnxxkmrf") + .withParameters(mapOf("rfhfjwikva", "datakprbmca", "uhqilvajctpw", "datajx")), + new LinkedServiceReference().withReferenceName("frfardjqwdr") + .withParameters(mapOf("syd", "dataoo", "lsxewocwmadyelw", "datavdqcmegwajjzxcq")), + new LinkedServiceReference().withReferenceName("ll") + .withParameters(mapOf("zqldakbijcxctn", "datahanfjrdcaw", "avoyhnollnuhoc", "datajxyczzwhwsidnq", + "nhujcunyuasstok", "databeoxoewprtzofcu")))) + .withArguments(Arrays.asList("datay", "datayblmcen", "datactc")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS).withMapper("datauplxksphzyu") + .withReducer("datayts").withInput("datafmajswr").withOutput("datadst") + .withFilePaths(Arrays.asList("datakqzulosl")) + .withFileLinkedService(new LinkedServiceReference().withReferenceName("fnhej") + .withParameters(mapOf("lvi", "dataugyuxcffbs", "zwtzdyz", "datas", "sojdmesoxj", "datafgn", "biouuqox", + "datapwbgfhjwchvu"))) + .withCombiner("datatws") + .withCommandEnvironment( + Arrays.asList("datadghlokvisqzmhei", "datasioonnfjgrtkeg", "datatvwffvbvuxpyveav", "datatzbzykks")) + .withDefines(mapOf("zdcj", "dataan")); model = BinaryData.fromObject(model).toObject(HDInsightStreamingActivity.class); - Assertions.assertEquals("dcjym", model.name()); - Assertions.assertEquals("ldbuyuxgephv", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("sdoxhyi", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("bfbg", model.userProperties().get(0).name()); - Assertions.assertEquals("apua", model.linkedServiceName().referenceName()); - Assertions.assertEquals(737405762, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals("ax", model.name()); + Assertions.assertEquals("bihgcduj", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("rbtrmif", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("tgmmfdfdqoepw", model.userProperties().get(0).name()); + Assertions.assertEquals("zpmwxvfrmvtwwb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1536315462, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("ljgaykaqwnk", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("v", model.fileLinkedService().referenceName()); + Assertions.assertEquals(false, model.policy().secureOutput()); + Assertions.assertEquals("zydyl", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); + Assertions.assertEquals("fnhej", model.fileLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTypePropertiesTests.java index 16df5ad21b6c7..1a4bf47a1f673 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HDInsightStreamingActivityTypePropertiesTests.java @@ -17,37 +17,36 @@ public final class HDInsightStreamingActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HDInsightStreamingActivityTypeProperties model = BinaryData.fromString( - "{\"storageLinkedServices\":[{\"referenceName\":\"yl\",\"parameters\":{\"eqdj\":\"datajficzwikglmcgyz\"}},{\"referenceName\":\"urbjxka\",\"parameters\":{\"wfbqxo\":\"dataguzv\",\"dcolnxwdp\":\"datanvij\",\"ezptngrpzvimxa\":\"dataiodnntol\",\"komrp\":\"dataxcaczc\"}},{\"referenceName\":\"c\",\"parameters\":{\"nxdyskyrhsijx\":\"dataxpdqwyw\",\"suwmbdtfetrgzy\":\"dataliuymfx\",\"qszqkrsnx\":\"dataprrapghloemqapv\"}}],\"arguments\":[\"datawrbqadtvpgu\"],\"getDebugInfo\":\"Failure\",\"mapper\":\"datatgkrivc\",\"reducer\":\"datax\",\"input\":\"datajplphkiyiqpiiod\",\"output\":\"datazjkz\",\"filePaths\":[\"datavxdpopubbwpsncx\",\"datakwmhjqakac\",\"datacbrsnnvlm\",\"datagfgtwqmtyfqutmj\"],\"fileLinkedService\":{\"referenceName\":\"isjurilqc\",\"parameters\":{\"ckbizqqajs\":\"datadorbufog\",\"cyvrbgi\":\"dataauwojgvpqzvtgwlz\"}},\"combiner\":\"datardekjbljfkqubnn\",\"commandEnvironment\":[\"datacbcxbvnhlw\"],\"defines\":{\"bwjtnfaumqkgccld\":\"dataxxaoyiskyoasxakb\"}}") + "{\"storageLinkedServices\":[{\"referenceName\":\"fawfeeatvnmm\",\"parameters\":{\"gwtcll\":\"datawfqrykikh\",\"fohtsmkf\":\"datawaz\"}},{\"referenceName\":\"oxbavfseh\",\"parameters\":{\"qszn\":\"dataqion\",\"o\":\"databi\",\"dteyvpv\":\"dataexjcrw\"}},{\"referenceName\":\"cqjgwtiasfbp\",\"parameters\":{\"seh\":\"dataxxxhbrysns\",\"wqg\":\"dataegvwbykrndxbkv\",\"qyedhyfncwiyfzu\":\"datafhl\",\"ccgtujiwzbzed\":\"databsaaxstnziv\"}}],\"arguments\":[\"dataygjhclny\",\"dataktc\",\"datamct\"],\"getDebugInfo\":\"Always\",\"mapper\":\"databrxmxqskemtajjf\",\"reducer\":\"datak\",\"input\":\"dataqpgbticn\",\"output\":\"datadubocmjiib\",\"filePaths\":[\"datawalhaw\",\"dataptiqfu\",\"dataavtapcxsmap\"],\"fileLinkedService\":{\"referenceName\":\"mmwylrv\",\"parameters\":{\"mcbtumt\":\"dataelpuxhku\",\"hyqexujll\":\"datarcvo\"}},\"combiner\":\"dataeegvqbsythycdck\",\"commandEnvironment\":[\"dataommgfwx\",\"datahrcmg\"],\"defines\":{\"senyehmwzgfanke\":\"datagosclhjgckkbn\",\"g\":\"datalorosa\"}}") .toObject(HDInsightStreamingActivityTypeProperties.class); - Assertions.assertEquals("yl", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("isjurilqc", model.fileLinkedService().referenceName()); + Assertions.assertEquals("fawfeeatvnmm", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); + Assertions.assertEquals("mmwylrv", model.fileLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { HDInsightStreamingActivityTypeProperties model = new HDInsightStreamingActivityTypeProperties() .withStorageLinkedServices(Arrays.asList( - new LinkedServiceReference().withReferenceName("yl") - .withParameters(mapOf("eqdj", "datajficzwikglmcgyz")), - new LinkedServiceReference().withReferenceName("urbjxka") - .withParameters(mapOf("wfbqxo", "dataguzv", "dcolnxwdp", "datanvij", "ezptngrpzvimxa", - "dataiodnntol", "komrp", "dataxcaczc")), - new LinkedServiceReference().withReferenceName("c") - .withParameters(mapOf("nxdyskyrhsijx", "dataxpdqwyw", "suwmbdtfetrgzy", "dataliuymfx", "qszqkrsnx", - "dataprrapghloemqapv")))) - .withArguments(Arrays.asList("datawrbqadtvpgu")).withGetDebugInfo(HDInsightActivityDebugInfoOption.FAILURE) - .withMapper("datatgkrivc").withReducer("datax").withInput("datajplphkiyiqpiiod").withOutput("datazjkz") - .withFilePaths( - Arrays.asList("datavxdpopubbwpsncx", "datakwmhjqakac", "datacbrsnnvlm", "datagfgtwqmtyfqutmj")) - .withFileLinkedService(new LinkedServiceReference().withReferenceName("isjurilqc") - .withParameters(mapOf("ckbizqqajs", "datadorbufog", "cyvrbgi", "dataauwojgvpqzvtgwlz"))) - .withCombiner("datardekjbljfkqubnn").withCommandEnvironment(Arrays.asList("datacbcxbvnhlw")) - .withDefines(mapOf("bwjtnfaumqkgccld", "dataxxaoyiskyoasxakb")); + new LinkedServiceReference().withReferenceName("fawfeeatvnmm") + .withParameters(mapOf("gwtcll", "datawfqrykikh", "fohtsmkf", "datawaz")), + new LinkedServiceReference().withReferenceName("oxbavfseh") + .withParameters(mapOf("qszn", "dataqion", "o", "databi", "dteyvpv", "dataexjcrw")), + new LinkedServiceReference().withReferenceName("cqjgwtiasfbp") + .withParameters(mapOf("seh", "dataxxxhbrysns", "wqg", "dataegvwbykrndxbkv", "qyedhyfncwiyfzu", + "datafhl", "ccgtujiwzbzed", "databsaaxstnziv")))) + .withArguments(Arrays.asList("dataygjhclny", "dataktc", "datamct")) + .withGetDebugInfo(HDInsightActivityDebugInfoOption.ALWAYS).withMapper("databrxmxqskemtajjf") + .withReducer("datak").withInput("dataqpgbticn").withOutput("datadubocmjiib") + .withFilePaths(Arrays.asList("datawalhaw", "dataptiqfu", "dataavtapcxsmap")) + .withFileLinkedService(new LinkedServiceReference().withReferenceName("mmwylrv") + .withParameters(mapOf("mcbtumt", "dataelpuxhku", "hyqexujll", "datarcvo"))) + .withCombiner("dataeegvqbsythycdck").withCommandEnvironment(Arrays.asList("dataommgfwx", "datahrcmg")) + .withDefines(mapOf("senyehmwzgfanke", "datagosclhjgckkbn", "g", "datalorosa")); model = BinaryData.fromObject(model).toObject(HDInsightStreamingActivityTypeProperties.class); - Assertions.assertEquals("yl", model.storageLinkedServices().get(0).referenceName()); - Assertions.assertEquals(HDInsightActivityDebugInfoOption.FAILURE, model.getDebugInfo()); - Assertions.assertEquals("isjurilqc", model.fileLinkedService().referenceName()); + Assertions.assertEquals("fawfeeatvnmm", model.storageLinkedServices().get(0).referenceName()); + Assertions.assertEquals(HDInsightActivityDebugInfoOption.ALWAYS, model.getDebugInfo()); + Assertions.assertEquals("mmwylrv", model.fileLinkedService().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsReadSettingsTests.java index 4dd677956c83a..a1baafd1764c0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsReadSettingsTests.java @@ -12,20 +12,21 @@ public final class HdfsReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HdfsReadSettings model = BinaryData.fromString( - "{\"type\":\"HdfsReadSettings\",\"recursive\":\"datayktu\",\"wildcardFolderPath\":\"datalhegoxefsa\",\"wildcardFileName\":\"datadcoeexwg\",\"fileListPath\":\"dataz\",\"enablePartitionDiscovery\":\"datazmqjhghihkhjldw\",\"partitionRootPath\":\"dataqt\",\"modifiedDatetimeStart\":\"datatgn\",\"modifiedDatetimeEnd\":\"databjvmdkgvu\",\"distcpSettings\":{\"resourceManagerEndpoint\":\"datamlsuuhwuox\",\"tempScriptPath\":\"datauiaizzjotm\",\"distcpOptions\":\"datazjrkslqbaf\"},\"deleteFilesAfterCompletion\":\"datajxxcruleimswxoig\",\"maxConcurrentConnections\":\"dataumjmpgze\",\"disableMetricsCollection\":\"datavf\",\"\":{\"ks\":\"datajpmeptnqsnpa\",\"qol\":\"datarrvjwbeeolmob\"}}") + "{\"type\":\"HdfsReadSettings\",\"recursive\":\"dataerxfe\",\"wildcardFolderPath\":\"datamzsekvsuzyowr\",\"wildcardFileName\":\"datavofrenuvp\",\"fileListPath\":\"dataltnyyeyj\",\"enablePartitionDiscovery\":\"datafpbxnretpg\",\"partitionRootPath\":\"datatohruqtximrxeyz\",\"modifiedDatetimeStart\":\"datanxb\",\"modifiedDatetimeEnd\":\"datayglfyfcsbkjhoxtb\",\"distcpSettings\":{\"resourceManagerEndpoint\":\"databpef\",\"tempScriptPath\":\"datajpnixd\",\"distcpOptions\":\"datajkfvmrnwgeaj\"},\"deleteFilesAfterCompletion\":\"datafeiiri\",\"maxConcurrentConnections\":\"datajdnknbt\",\"disableMetricsCollection\":\"datardep\",\"\":{\"onovfnob\":\"datazhkp\",\"mmzxpsrlbppjq\":\"datawhutvcdtgxsyfuh\"}}") .toObject(HdfsReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HdfsReadSettings model = new HdfsReadSettings().withMaxConcurrentConnections("dataumjmpgze") - .withDisableMetricsCollection("datavf").withRecursive("datayktu").withWildcardFolderPath("datalhegoxefsa") - .withWildcardFileName("datadcoeexwg").withFileListPath("dataz") - .withEnablePartitionDiscovery("datazmqjhghihkhjldw").withPartitionRootPath("dataqt") - .withModifiedDatetimeStart("datatgn").withModifiedDatetimeEnd("databjvmdkgvu") - .withDistcpSettings(new DistcpSettings().withResourceManagerEndpoint("datamlsuuhwuox") - .withTempScriptPath("datauiaizzjotm").withDistcpOptions("datazjrkslqbaf")) - .withDeleteFilesAfterCompletion("datajxxcruleimswxoig"); + HdfsReadSettings model = new HdfsReadSettings().withMaxConcurrentConnections("datajdnknbt") + .withDisableMetricsCollection("datardep").withRecursive("dataerxfe") + .withWildcardFolderPath("datamzsekvsuzyowr").withWildcardFileName("datavofrenuvp") + .withFileListPath("dataltnyyeyj").withEnablePartitionDiscovery("datafpbxnretpg") + .withPartitionRootPath("datatohruqtximrxeyz").withModifiedDatetimeStart("datanxb") + .withModifiedDatetimeEnd("datayglfyfcsbkjhoxtb") + .withDistcpSettings(new DistcpSettings().withResourceManagerEndpoint("databpef") + .withTempScriptPath("datajpnixd").withDistcpOptions("datajkfvmrnwgeaj")) + .withDeleteFilesAfterCompletion("datafeiiri"); model = BinaryData.fromObject(model).toObject(HdfsReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsSourceTests.java index f0b0db1999766..ee4b65fcdfb2c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HdfsSourceTests.java @@ -12,16 +12,16 @@ public final class HdfsSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HdfsSource model = BinaryData.fromString( - "{\"type\":\"HdfsSource\",\"recursive\":\"dataymijhn\",\"distcpSettings\":{\"resourceManagerEndpoint\":\"datarcoh\",\"tempScriptPath\":\"datauwzunkzbdeyh\",\"distcpOptions\":\"databhobdocfv\"},\"sourceRetryCount\":\"datammdmbylynd\",\"sourceRetryWait\":\"dataujfzxsazuj\",\"maxConcurrentConnections\":\"datawwtlerhpfrarqnj\",\"disableMetricsCollection\":\"datahsxhtvnq\",\"\":{\"wgom\":\"datargm\",\"xfcngef\":\"datascsddlcnwbi\"}}") + "{\"type\":\"HdfsSource\",\"recursive\":\"dataskzw\",\"distcpSettings\":{\"resourceManagerEndpoint\":\"datah\",\"tempScriptPath\":\"datavhzrxcae\",\"distcpOptions\":\"dataxoavlwwpvjrnjwvc\"},\"sourceRetryCount\":\"datarqlceflgsndur\",\"sourceRetryWait\":\"dataozjwm\",\"maxConcurrentConnections\":\"datadehjloz\",\"disableMetricsCollection\":\"datawokuxedpqwzzzi\",\"\":{\"ve\":\"dataxjgxrhajrubcvu\",\"uclkbw\":\"datadfmdjnfeealp\",\"vkhfzldzz\":\"datamwdrvkbcsvn\"}}") .toObject(HdfsSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HdfsSource model = new HdfsSource().withSourceRetryCount("datammdmbylynd").withSourceRetryWait("dataujfzxsazuj") - .withMaxConcurrentConnections("datawwtlerhpfrarqnj").withDisableMetricsCollection("datahsxhtvnq") - .withRecursive("dataymijhn").withDistcpSettings(new DistcpSettings().withResourceManagerEndpoint("datarcoh") - .withTempScriptPath("datauwzunkzbdeyh").withDistcpOptions("databhobdocfv")); + HdfsSource model = new HdfsSource().withSourceRetryCount("datarqlceflgsndur").withSourceRetryWait("dataozjwm") + .withMaxConcurrentConnections("datadehjloz").withDisableMetricsCollection("datawokuxedpqwzzzi") + .withRecursive("dataskzw").withDistcpSettings(new DistcpSettings().withResourceManagerEndpoint("datah") + .withTempScriptPath("datavhzrxcae").withDistcpOptions("dataxoavlwwpvjrnjwvc")); model = BinaryData.fromObject(model).toObject(HdfsSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveDatasetTypePropertiesTests.java index 2863ec1ee331d..49ba7e7615268 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveDatasetTypePropertiesTests.java @@ -10,15 +10,15 @@ public final class HiveDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - HiveDatasetTypeProperties model = BinaryData.fromString( - "{\"tableName\":\"datanocscygimizl\",\"table\":\"datajbwmgksrlmsppp\",\"schema\":\"dataszthjtryjskdiylg\"}") + HiveDatasetTypeProperties model = BinaryData + .fromString("{\"tableName\":\"datanjqhdheosx\",\"table\":\"datafudmpfhwyp\",\"schema\":\"datatjtntc\"}") .toObject(HiveDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HiveDatasetTypeProperties model = new HiveDatasetTypeProperties().withTableName("datanocscygimizl") - .withTable("datajbwmgksrlmsppp").withSchema("dataszthjtryjskdiylg"); + HiveDatasetTypeProperties model = new HiveDatasetTypeProperties().withTableName("datanjqhdheosx") + .withTable("datafudmpfhwyp").withSchema("datatjtntc"); model = BinaryData.fromObject(model).toObject(HiveDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveObjectDatasetTests.java index b67d3977fe3ae..849c01bf1cccc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveObjectDatasetTests.java @@ -19,30 +19,32 @@ public final class HiveObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HiveObjectDataset model = BinaryData.fromString( - "{\"type\":\"HiveObject\",\"typeProperties\":{\"tableName\":\"dataqkwqphfv\",\"table\":\"datatsstwl\",\"schema\":\"datacachdtezgfctu\"},\"description\":\"owqrzvuxn\",\"structure\":\"datauohshzultdbvm\",\"schema\":\"datahypngo\",\"linkedServiceName\":{\"referenceName\":\"bdxvrivptbczsuzg\",\"parameters\":{\"gpycei\":\"datakekytkzvtvmaatv\",\"s\":\"dataharhbdxsbyp\"}},\"parameters\":{\"sezsggdp\":{\"type\":\"Array\",\"defaultValue\":\"dataudapbq\"}},\"annotations\":[\"datacbrtsrdpl\",\"datadyzaciasfzrgu\",\"dataliyvsbf\"],\"folder\":{\"name\":\"vabd\"},\"\":{\"yaosthulzu\":\"datajgxotudamk\",\"xl\":\"dataifgs\"}}") + "{\"type\":\"HiveObject\",\"typeProperties\":{\"tableName\":\"datazsggd\",\"table\":\"datatfcbrtsrdplqdyza\",\"schema\":\"dataasfzrguzliyvsb\"},\"description\":\"inv\",\"structure\":\"datadjuljgxotuda\",\"schema\":\"datai\",\"linkedServiceName\":{\"referenceName\":\"a\",\"parameters\":{\"l\":\"datahulzugifgsp\"}},\"parameters\":{\"bwmgksrlmspp\":{\"type\":\"Float\",\"defaultValue\":\"datascygimizluk\"},\"zuqix\":{\"type\":\"Int\",\"defaultValue\":\"dataszthjtryjskdiylg\"},\"koe\":{\"type\":\"Int\",\"defaultValue\":\"databqowgvmxwbohxd\"}},\"annotations\":[\"datah\",\"datanakaj\",\"datascmne\",\"datavlumqeumz\"],\"folder\":{\"name\":\"mgqaeivjq\"},\"\":{\"dzahktxvcbic\":\"datarbgbzgfh\",\"tpxjvtwkyjdpayx\":\"dataecthotbkjwh\",\"qztjfkgb\":\"datapqiwuzr\",\"en\":\"dataqqjobsyn\"}}") .toObject(HiveObjectDataset.class); - Assertions.assertEquals("owqrzvuxn", model.description()); - Assertions.assertEquals("bdxvrivptbczsuzg", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("sezsggdp").type()); - Assertions.assertEquals("vabd", model.folder().name()); + Assertions.assertEquals("inv", model.description()); + Assertions.assertEquals("a", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("bwmgksrlmspp").type()); + Assertions.assertEquals("mgqaeivjq", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HiveObjectDataset model = new HiveObjectDataset().withDescription("owqrzvuxn") - .withStructure("datauohshzultdbvm").withSchema("datahypngo") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("bdxvrivptbczsuzg") - .withParameters(mapOf("gpycei", "datakekytkzvtvmaatv", "s", "dataharhbdxsbyp"))) - .withParameters(mapOf("sezsggdp", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataudapbq"))) - .withAnnotations(Arrays.asList("datacbrtsrdpl", "datadyzaciasfzrgu", "dataliyvsbf")) - .withFolder(new DatasetFolder().withName("vabd")).withTableName("dataqkwqphfv").withTable("datatsstwl") - .withSchemaTypePropertiesSchema("datacachdtezgfctu"); + HiveObjectDataset model = new HiveObjectDataset().withDescription("inv").withStructure("datadjuljgxotuda") + .withSchema("datai") + .withLinkedServiceName( + new LinkedServiceReference().withReferenceName("a").withParameters(mapOf("l", "datahulzugifgsp"))) + .withParameters(mapOf("bwmgksrlmspp", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datascygimizluk"), "zuqix", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataszthjtryjskdiylg"), + "koe", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("databqowgvmxwbohxd"))) + .withAnnotations(Arrays.asList("datah", "datanakaj", "datascmne", "datavlumqeumz")) + .withFolder(new DatasetFolder().withName("mgqaeivjq")).withTableName("datazsggd") + .withTable("datatfcbrtsrdplqdyza").withSchemaTypePropertiesSchema("dataasfzrguzliyvsb"); model = BinaryData.fromObject(model).toObject(HiveObjectDataset.class); - Assertions.assertEquals("owqrzvuxn", model.description()); - Assertions.assertEquals("bdxvrivptbczsuzg", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("sezsggdp").type()); - Assertions.assertEquals("vabd", model.folder().name()); + Assertions.assertEquals("inv", model.description()); + Assertions.assertEquals("a", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("bwmgksrlmspp").type()); + Assertions.assertEquals("mgqaeivjq", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveSourceTests.java index bc448134a85fb..9d756b64c5969 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HiveSourceTests.java @@ -11,15 +11,16 @@ public final class HiveSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HiveSource model = BinaryData.fromString( - "{\"type\":\"HiveSource\",\"query\":\"dataasc\",\"queryTimeout\":\"datadwijx\",\"additionalColumns\":\"dataltowdwiffagfe\",\"sourceRetryCount\":\"datambpgcbltthsuzx\",\"sourceRetryWait\":\"datavifl\",\"maxConcurrentConnections\":\"datarknsc\",\"disableMetricsCollection\":\"datah\",\"\":{\"ffuzhnusrfffagoo\":\"datajsvpokvhoby\",\"hgonovwu\":\"datafwzysvnvrfjgbxup\",\"gkouf\":\"dataearowrmesziubkyv\"}}") + "{\"type\":\"HiveSource\",\"query\":\"dataeydmeuimlhyze\",\"queryTimeout\":\"dataivkzrvya\",\"additionalColumns\":\"dataqgyui\",\"sourceRetryCount\":\"dataelyjduzapnopoto\",\"sourceRetryWait\":\"datarrqcaglyt\",\"maxConcurrentConnections\":\"datacbdpczmzuwr\",\"disableMetricsCollection\":\"datahfwce\",\"\":{\"cyfccnwmdpbso\":\"dataaqaviqskylwpq\",\"fxpveruuckrzw\":\"datakn\"}}") .toObject(HiveSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HiveSource model = new HiveSource().withSourceRetryCount("datambpgcbltthsuzx").withSourceRetryWait("datavifl") - .withMaxConcurrentConnections("datarknsc").withDisableMetricsCollection("datah") - .withQueryTimeout("datadwijx").withAdditionalColumns("dataltowdwiffagfe").withQuery("dataasc"); + HiveSource model + = new HiveSource().withSourceRetryCount("dataelyjduzapnopoto").withSourceRetryWait("datarrqcaglyt") + .withMaxConcurrentConnections("datacbdpczmzuwr").withDisableMetricsCollection("datahfwce") + .withQueryTimeout("dataivkzrvya").withAdditionalColumns("dataqgyui").withQuery("dataeydmeuimlhyze"); model = BinaryData.fromObject(model).toObject(HiveSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTests.java index 87f919c80f908..614286e8abcc7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTests.java @@ -21,39 +21,37 @@ public final class HttpDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HttpDataset model = BinaryData.fromString( - "{\"type\":\"HttpFile\",\"typeProperties\":{\"relativeUrl\":\"datatfctanetinqxd\",\"requestMethod\":\"datapjnezjighduml\",\"requestBody\":\"datamrzwvwetqffux\",\"additionalHeaders\":\"datahuqhngqq\",\"format\":{\"type\":\"DatasetStorageFormat\",\"serializer\":\"datasot\",\"deserializer\":\"datalmr\",\"\":{\"bboceksram\":\"datajydeatwxpxbxedh\"}},\"compression\":{\"type\":\"datahlugfnlvvk\",\"level\":\"dataurxdqhvhauimn\",\"\":{\"ivlqcwyzhndqkzst\":\"datakqpwqcnbn\",\"u\":\"datapzecdlceirtah\"}}},\"description\":\"imt\",\"structure\":\"dataumviudzpsjqrm\",\"schema\":\"datajmtunlo\",\"linkedServiceName\":{\"referenceName\":\"wuzebfqvm\",\"parameters\":{\"xeudwkhdl\":\"datahzyenfspe\"}},\"parameters\":{\"jcdevzpfreor\":{\"type\":\"Int\",\"defaultValue\":\"datao\"},\"x\":{\"type\":\"String\",\"defaultValue\":\"datayjmgvrlh\"},\"avuafanefic\":{\"type\":\"SecureString\",\"defaultValue\":\"datajnnhbcjywkdywks\"}},\"annotations\":[\"dataplkossjbzvxp\",\"datawdqzuhfgt\"],\"folder\":{\"name\":\"zhfjdccjny\"},\"\":{\"zthcdbszsbz\":\"datatcuhjcgjtjkntomn\"}}") + "{\"type\":\"HttpFile\",\"typeProperties\":{\"relativeUrl\":\"dataxooi\",\"requestMethod\":\"datahiebruptls\",\"requestBody\":\"dataqzgaqsosrnjlvgrg\",\"additionalHeaders\":\"datahuoxrqhjninpesw\",\"format\":{\"type\":\"DatasetStorageFormat\",\"serializer\":\"dataqkgebz\",\"deserializer\":\"datam\",\"\":{\"ojzdvmsnao\":\"dataiu\",\"hdbitq\":\"dataxsxoxvimdvet\",\"omr\":\"databyujs\"}},\"compression\":{\"type\":\"datau\",\"level\":\"datarmsdbvqxgfyg\",\"\":{\"sesboynpytporr\":\"dataxbdpbcehwbd\",\"anetinqxdhnpjne\":\"datakxtfc\",\"mltpmr\":\"datajighd\",\"qhngqqxjbsot\":\"datawvwetqffuxvfh\"}}},\"description\":\"lmr\",\"structure\":\"datau\",\"schema\":\"datadeatwxpx\",\"linkedServiceName\":{\"referenceName\":\"xedhxbboceksra\",\"parameters\":{\"wurxdqhv\":\"datahlugfnlvvk\"}},\"parameters\":{\"ivlqcwyzhndqkzst\":{\"type\":\"Float\",\"defaultValue\":\"datanntfkqpwqcnbn\"},\"eirta\":{\"type\":\"Object\",\"defaultValue\":\"dataecdl\"},\"viudzpsj\":{\"type\":\"Int\",\"defaultValue\":\"datawcimtcau\"},\"unlofwuzebfq\":{\"type\":\"Int\",\"defaultValue\":\"datalujm\"}},\"annotations\":[\"datajbhzyenf\",\"datapetxeudwkh\",\"datalckdoxocjcdevzp\"],\"folder\":{\"name\":\"ortwwyjm\"},\"\":{\"jnnhbcjywkdywks\":\"datalhfxmr\",\"ptplkossjbzv\":\"dataavuafanefic\"}}") .toObject(HttpDataset.class); - Assertions.assertEquals("imt", model.description()); - Assertions.assertEquals("wuzebfqvm", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("jcdevzpfreor").type()); - Assertions.assertEquals("zhfjdccjny", model.folder().name()); + Assertions.assertEquals("lmr", model.description()); + Assertions.assertEquals("xedhxbboceksra", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("ivlqcwyzhndqkzst").type()); + Assertions.assertEquals("ortwwyjm", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpDataset model - = new HttpDataset().withDescription("imt").withStructure("dataumviudzpsjqrm").withSchema("datajmtunlo") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("wuzebfqvm") - .withParameters(mapOf("xeudwkhdl", "datahzyenfspe"))) - .withParameters(mapOf("jcdevzpfreor", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datao"), "x", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datayjmgvrlh"), - "avuafanefic", - new ParameterSpecification().withType(ParameterType.SECURE_STRING) - .withDefaultValue("datajnnhbcjywkdywks"))) - .withAnnotations(Arrays.asList("dataplkossjbzvxp", "datawdqzuhfgt")) - .withFolder(new DatasetFolder().withName("zhfjdccjny")).withRelativeUrl("datatfctanetinqxd") - .withRequestMethod("datapjnezjighduml").withRequestBody("datamrzwvwetqffux") - .withAdditionalHeaders("datahuqhngqq") - .withFormat(new DatasetStorageFormat().withSerializer("datasot").withDeserializer("datalmr") - .withAdditionalProperties(mapOf("type", "DatasetStorageFormat"))) - .withCompression(new DatasetCompression().withType("datahlugfnlvvk").withLevel("dataurxdqhvhauimn") - .withAdditionalProperties(mapOf())); + HttpDataset model = new HttpDataset().withDescription("lmr").withStructure("datau").withSchema("datadeatwxpx") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("xedhxbboceksra") + .withParameters(mapOf("wurxdqhv", "datahlugfnlvvk"))) + .withParameters(mapOf("ivlqcwyzhndqkzst", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datanntfkqpwqcnbn"), + "eirta", new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataecdl"), + "viudzpsj", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datawcimtcau"), + "unlofwuzebfq", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datalujm"))) + .withAnnotations(Arrays.asList("datajbhzyenf", "datapetxeudwkh", "datalckdoxocjcdevzp")) + .withFolder(new DatasetFolder().withName("ortwwyjm")).withRelativeUrl("dataxooi") + .withRequestMethod("datahiebruptls").withRequestBody("dataqzgaqsosrnjlvgrg") + .withAdditionalHeaders("datahuoxrqhjninpesw") + .withFormat(new DatasetStorageFormat().withSerializer("dataqkgebz").withDeserializer("datam") + .withAdditionalProperties(mapOf("type", "DatasetStorageFormat"))) + .withCompression(new DatasetCompression().withType("datau").withLevel("datarmsdbvqxgfyg") + .withAdditionalProperties(mapOf())); model = BinaryData.fromObject(model).toObject(HttpDataset.class); - Assertions.assertEquals("imt", model.description()); - Assertions.assertEquals("wuzebfqvm", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("jcdevzpfreor").type()); - Assertions.assertEquals("zhfjdccjny", model.folder().name()); + Assertions.assertEquals("lmr", model.description()); + Assertions.assertEquals("xedhxbboceksra", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("ivlqcwyzhndqkzst").type()); + Assertions.assertEquals("ortwwyjm", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTypePropertiesTests.java index 6fec81efb0e1e..765545eff8ddf 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpDatasetTypePropertiesTests.java @@ -15,17 +15,17 @@ public final class HttpDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HttpDatasetTypeProperties model = BinaryData.fromString( - "{\"relativeUrl\":\"dataxeyvidcowlrm\",\"requestMethod\":\"datactqxa\",\"requestBody\":\"datajoezvw\",\"additionalHeaders\":\"datayzgavplnd\",\"format\":{\"type\":\"DatasetStorageFormat\",\"serializer\":\"dataiekkiskyyyaekn\",\"deserializer\":\"datafys\",\"\":{\"hoajjylsyqy\":\"datawjlmlcufbbjiutfo\"}},\"compression\":{\"type\":\"dataufzvlqquy\",\"level\":\"dataceevogir\",\"\":{\"dssijuaxxf\":\"datanqtvuxeuj\"}}}") + "{\"relativeUrl\":\"datazw\",\"requestMethod\":\"datazuh\",\"requestBody\":\"datatiaczhfjdccjny\",\"additionalHeaders\":\"databt\",\"format\":{\"type\":\"DatasetStorageFormat\",\"serializer\":\"datajcgjtjkntomnl\",\"deserializer\":\"datahcdb\",\"\":{\"bdctqxavejoezvwk\":\"databzrrxeyvidcowlr\"}},\"compression\":{\"type\":\"datazgavp\",\"level\":\"datadmdfiekkis\",\"\":{\"a\":\"datayaeknfffysh\",\"tfofhoajjylsyqyj\":\"datajlmlcufbbji\"}}}") .toObject(HttpDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpDatasetTypeProperties model = new HttpDatasetTypeProperties().withRelativeUrl("dataxeyvidcowlrm") - .withRequestMethod("datactqxa").withRequestBody("datajoezvw").withAdditionalHeaders("datayzgavplnd") - .withFormat(new DatasetStorageFormat().withSerializer("dataiekkiskyyyaekn").withDeserializer("datafys") + HttpDatasetTypeProperties model = new HttpDatasetTypeProperties().withRelativeUrl("datazw") + .withRequestMethod("datazuh").withRequestBody("datatiaczhfjdccjny").withAdditionalHeaders("databt") + .withFormat(new DatasetStorageFormat().withSerializer("datajcgjtjkntomnl").withDeserializer("datahcdb") .withAdditionalProperties(mapOf("type", "DatasetStorageFormat"))) - .withCompression(new DatasetCompression().withType("dataufzvlqquy").withLevel("dataceevogir") + .withCompression(new DatasetCompression().withType("datazgavp").withLevel("datadmdfiekkis") .withAdditionalProperties(mapOf())); model = BinaryData.fromObject(model).toObject(HttpDatasetTypeProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpReadSettingsTests.java index 5fe669f06353b..6fed3c81808fb 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpReadSettingsTests.java @@ -11,16 +11,16 @@ public final class HttpReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HttpReadSettings model = BinaryData.fromString( - "{\"type\":\"HttpReadSettings\",\"requestMethod\":\"dataaiqyxlr\",\"requestBody\":\"dataxpebsxcnhq\",\"additionalHeaders\":\"databtyorlr\",\"requestTimeout\":\"datamyumro\",\"additionalColumns\":\"dataaxnym\",\"maxConcurrentConnections\":\"datag\",\"disableMetricsCollection\":\"dataqmkakgw\",\"\":{\"sdgyhe\":\"datano\"}}") + "{\"type\":\"HttpReadSettings\",\"requestMethod\":\"datarvigrxmptufde\",\"requestBody\":\"datapqghxdpgihfimlyx\",\"additionalHeaders\":\"dataixjudbiac\",\"requestTimeout\":\"dataoucmfuvuslvbujwp\",\"additionalColumns\":\"dataijpyyvecruhqymwd\",\"maxConcurrentConnections\":\"datahkt\",\"disableMetricsCollection\":\"dataljkh\",\"\":{\"ivapua\":\"datatpgxkkoypxwlvt\",\"pd\":\"dataoswqwbhlrzlgkc\"}}") .toObject(HttpReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpReadSettings model - = new HttpReadSettings().withMaxConcurrentConnections("datag").withDisableMetricsCollection("dataqmkakgw") - .withRequestMethod("dataaiqyxlr").withRequestBody("dataxpebsxcnhq").withAdditionalHeaders("databtyorlr") - .withRequestTimeout("datamyumro").withAdditionalColumns("dataaxnym"); + HttpReadSettings model = new HttpReadSettings().withMaxConcurrentConnections("datahkt") + .withDisableMetricsCollection("dataljkh").withRequestMethod("datarvigrxmptufde") + .withRequestBody("datapqghxdpgihfimlyx").withAdditionalHeaders("dataixjudbiac") + .withRequestTimeout("dataoucmfuvuslvbujwp").withAdditionalColumns("dataijpyyvecruhqymwd"); model = BinaryData.fromObject(model).toObject(HttpReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpSourceTests.java index d287593013619..e7acf3e79d6d7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HttpSourceTests.java @@ -11,15 +11,15 @@ public final class HttpSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HttpSource model = BinaryData.fromString( - "{\"type\":\"HttpSource\",\"httpRequestTimeout\":\"datazhuhujkzjse\",\"sourceRetryCount\":\"dataphipte\",\"sourceRetryWait\":\"datal\",\"maxConcurrentConnections\":\"datahwdfxgeccck\",\"disableMetricsCollection\":\"databzbhs\",\"\":{\"pxdzmpjfbd\":\"datao\",\"iocqoydqyzhfny\":\"dataoawhbdxxnmyxz\",\"skt\":\"datagbwdsaqwywayjin\",\"ygyeyxmuwgn\":\"datarnknnql\"}}") + "{\"type\":\"HttpSource\",\"httpRequestTimeout\":\"datajnhxufocski\",\"sourceRetryCount\":\"dataj\",\"sourceRetryWait\":\"datap\",\"maxConcurrentConnections\":\"datavhfpfsesiywcre\",\"disableMetricsCollection\":\"dataphqqozhesbpq\",\"\":{\"upcdaoatzvaj\":\"datafjktdvdhlkztalu\",\"pqo\":\"datavxhefmotulhilmaz\"}}") .toObject(HttpSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpSource model = new HttpSource().withSourceRetryCount("dataphipte").withSourceRetryWait("datal") - .withMaxConcurrentConnections("datahwdfxgeccck").withDisableMetricsCollection("databzbhs") - .withHttpRequestTimeout("datazhuhujkzjse"); + HttpSource model = new HttpSource().withSourceRetryCount("dataj").withSourceRetryWait("datap") + .withMaxConcurrentConnections("datavhfpfsesiywcre").withDisableMetricsCollection("dataphqqozhesbpq") + .withHttpRequestTimeout("datajnhxufocski"); model = BinaryData.fromObject(model).toObject(HttpSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotObjectDatasetTests.java index 0384bb211b81d..83a87df9b3831 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotObjectDatasetTests.java @@ -19,32 +19,30 @@ public final class HubspotObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HubspotObjectDataset model = BinaryData.fromString( - "{\"type\":\"HubspotObject\",\"typeProperties\":{\"tableName\":\"dataqixpsybq\"},\"description\":\"gvmxwbohxd\",\"structure\":\"dataoexb\",\"schema\":\"datagnaka\",\"linkedServiceName\":{\"referenceName\":\"wscmneev\",\"parameters\":{\"yhmgq\":\"dataqeumz\",\"gbzgfhzdzahktxv\":\"dataeivjqutxr\",\"pxjvtwk\":\"databicfecthotbkjwhz\",\"pqiwuzr\":\"datajdpayx\"}},\"parameters\":{\"qqjobsyn\":{\"type\":\"Float\",\"defaultValue\":\"datafkgb\"},\"q\":{\"type\":\"Object\",\"defaultValue\":\"dataion\"},\"sxjwfudmpfh\":{\"type\":\"Int\",\"defaultValue\":\"datae\"}},\"annotations\":[\"datap\",\"datatjtntc\",\"datagpdbbglaecc\",\"dataokfsp\"],\"folder\":{\"name\":\"ds\"},\"\":{\"prftyptwjwiyyeoh\":\"datar\"}}") + "{\"type\":\"HubspotObject\",\"typeProperties\":{\"tableName\":\"datadbbglaecct\"},\"description\":\"fspvjrds\",\"structure\":\"datavrm\",\"schema\":\"dataftyptwjwiyyeo\",\"linkedServiceName\":{\"referenceName\":\"gmc\",\"parameters\":{\"ioxbgom\":\"datamvphwfnugslvfz\",\"zuox\":\"dataueprpmofxnwc\"}},\"parameters\":{\"nnrnkyj\":{\"type\":\"String\",\"defaultValue\":\"dataxajsehb\"},\"gzehczbni\":{\"type\":\"String\",\"defaultValue\":\"datapcbs\"}},\"annotations\":[\"datahsxvppkjeal\",\"datadbewh\",\"datatvbmyzuqfdqdktr\",\"datat\"],\"folder\":{\"name\":\"zhhqngj\"},\"\":{\"mxlffqgdodnkq\":\"datav\",\"zesimef\":\"dataipgkmjtdazm\",\"fzjlflzagvda\":\"datagd\"}}") .toObject(HubspotObjectDataset.class); - Assertions.assertEquals("gvmxwbohxd", model.description()); - Assertions.assertEquals("wscmneev", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("qqjobsyn").type()); - Assertions.assertEquals("ds", model.folder().name()); + Assertions.assertEquals("fspvjrds", model.description()); + Assertions.assertEquals("gmc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("nnrnkyj").type()); + Assertions.assertEquals("zhhqngj", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HubspotObjectDataset model - = new HubspotObjectDataset().withDescription("gvmxwbohxd").withStructure("dataoexb").withSchema("datagnaka") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("wscmneev") - .withParameters(mapOf("yhmgq", "dataqeumz", "gbzgfhzdzahktxv", "dataeivjqutxr", "pxjvtwk", - "databicfecthotbkjwhz", "pqiwuzr", "datajdpayx"))) - .withParameters(mapOf("qqjobsyn", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datafkgb"), "q", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataion"), - "sxjwfudmpfh", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datae"))) - .withAnnotations(Arrays.asList("datap", "datatjtntc", "datagpdbbglaecc", "dataokfsp")) - .withFolder(new DatasetFolder().withName("ds")).withTableName("dataqixpsybq"); + HubspotObjectDataset model = new HubspotObjectDataset().withDescription("fspvjrds").withStructure("datavrm") + .withSchema("dataftyptwjwiyyeo") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("gmc") + .withParameters(mapOf("ioxbgom", "datamvphwfnugslvfz", "zuox", "dataueprpmofxnwc"))) + .withParameters(mapOf("nnrnkyj", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataxajsehb"), + "gzehczbni", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datapcbs"))) + .withAnnotations(Arrays.asList("datahsxvppkjeal", "datadbewh", "datatvbmyzuqfdqdktr", "datat")) + .withFolder(new DatasetFolder().withName("zhhqngj")).withTableName("datadbbglaecct"); model = BinaryData.fromObject(model).toObject(HubspotObjectDataset.class); - Assertions.assertEquals("gvmxwbohxd", model.description()); - Assertions.assertEquals("wscmneev", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("qqjobsyn").type()); - Assertions.assertEquals("ds", model.folder().name()); + Assertions.assertEquals("fspvjrds", model.description()); + Assertions.assertEquals("gmc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("nnrnkyj").type()); + Assertions.assertEquals("zhhqngj", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotSourceTests.java index f4a9c44ca0bf8..022094851f025 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/HubspotSourceTests.java @@ -11,16 +11,16 @@ public final class HubspotSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HubspotSource model = BinaryData.fromString( - "{\"type\":\"HubspotSource\",\"query\":\"dataaomy\",\"queryTimeout\":\"dataxrw\",\"additionalColumns\":\"databyaavfm\",\"sourceRetryCount\":\"dataamncuhxz\",\"sourceRetryWait\":\"dataakxyhuetztorh\",\"maxConcurrentConnections\":\"datau\",\"disableMetricsCollection\":\"datasszhsewjqgzlo\",\"\":{\"ur\":\"datax\"}}") + "{\"type\":\"HubspotSource\",\"query\":\"dataqaptqyrnlyuyopww\",\"queryTimeout\":\"dataoubwbssvfzjjf\",\"additionalColumns\":\"dataxeosyl\",\"sourceRetryCount\":\"datappqjujbqrfw\",\"sourceRetryWait\":\"datawvpnbgyxo\",\"maxConcurrentConnections\":\"datakzeaiaycauvlfsc\",\"disableMetricsCollection\":\"dataqpzqivfgemvuicxw\",\"\":{\"atjm\":\"dataydlvfnucgwflj\"}}") .toObject(HubspotSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HubspotSource model - = new HubspotSource().withSourceRetryCount("dataamncuhxz").withSourceRetryWait("dataakxyhuetztorh") - .withMaxConcurrentConnections("datau").withDisableMetricsCollection("datasszhsewjqgzlo") - .withQueryTimeout("dataxrw").withAdditionalColumns("databyaavfm").withQuery("dataaomy"); + HubspotSource model = new HubspotSource().withSourceRetryCount("datappqjujbqrfw") + .withSourceRetryWait("datawvpnbgyxo").withMaxConcurrentConnections("datakzeaiaycauvlfsc") + .withDisableMetricsCollection("dataqpzqivfgemvuicxw").withQueryTimeout("dataoubwbssvfzjjf") + .withAdditionalColumns("dataxeosyl").withQuery("dataqaptqyrnlyuyopww"); model = BinaryData.fromObject(model).toObject(HubspotSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTests.java index d929b3ff23f1d..897c3e09fce6e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTests.java @@ -22,226 +22,158 @@ public final class IfConditionActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IfConditionActivity model = BinaryData.fromString( - "{\"type\":\"IfCondition\",\"typeProperties\":{\"expression\":{\"value\":\"abbetzcd\"},\"ifTrueActivities\":[{\"type\":\"Activity\",\"name\":\"wutakbvaqgu\",\"description\":\"bmcwpllo\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"wtddig\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Skipped\",\"Failed\"],\"\":{\"u\":\"datacrbkwcnvgx\",\"iafzwhr\":\"datahmjcemkcwcbvhqj\"}},{\"activity\":\"mzntroafz\",\"dependencyConditions\":[\"Completed\"],\"\":{\"szuhuojjbkyd\":\"datafojrryzbqpksoa\"}},{\"activity\":\"s\",\"dependencyConditions\":[\"Failed\"],\"\":{\"shnksupchzspgby\":\"datacqkbetemam\",\"vuhgchtaea\":\"dataumxyqhctr\",\"zuk\":\"databqkx\"}}],\"userProperties\":[{\"name\":\"ihn\",\"value\":\"dataqfqawynsl\"},{\"name\":\"fx\",\"value\":\"dataovavi\"}],\"\":{\"ud\":\"dataedfyhtfugpp\",\"xzcrf\":\"dataylxqcsumqdria\",\"qlufojuexpkqhg\":\"datakbchnhexmg\"}},{\"type\":\"Activity\",\"name\":\"wyzvnsnak\",\"description\":\"bcuyhmltdgx\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"jfxueqyjee\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Failed\",\"Completed\"],\"\":{\"lala\":\"dataklqipnzgnybu\",\"zq\":\"datazncn\"}},{\"activity\":\"hx\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Completed\",\"Skipped\"],\"\":{\"bh\":\"datarjjernjkgalodf\",\"tygyia\":\"datah\"}}],\"userProperties\":[{\"name\":\"txarl\",\"value\":\"datasijcmn\"},{\"name\":\"gp\",\"value\":\"datagxjmwzkafuvbg\"}],\"\":{\"vsmrxypb\":\"datarsbhjlc\",\"sfas\":\"datawnyznaix\",\"nxejxwcojjmp\":\"datafamnpbyxbglqyb\",\"qcnuoz\":\"dataxo\"}},{\"type\":\"Activity\",\"name\":\"gkcxbenwiignrrqv\",\"description\":\"o\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"raqesk\",\"dependencyConditions\":[\"Completed\",\"Succeeded\"],\"\":{\"wcwiegsswijqs\":\"datazaalp\",\"dleg\":\"datadqjbdtczxwqmu\",\"tbtrhrygd\":\"datalcuedrmqkwk\"}},{\"activity\":\"jufmvozqmt\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"tsatnryw\":\"dataeg\",\"jprr\":\"datauewrwcqrvtwv\",\"cswadvbwewwd\":\"datahxqpmzznmn\"}},{\"activity\":\"eiehwma\",\"dependencyConditions\":[\"Completed\"],\"\":{\"beneqapll\":\"datagkm\"}},{\"activity\":\"dowsj\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Completed\",\"Skipped\"],\"\":{\"catpwqoqnajmwpea\":\"datalivcymnpblhalm\",\"lhlwulu\":\"dataeggiglp\",\"zvixam\":\"datarumoprnb\"}}],\"userProperties\":[{\"name\":\"qrpbwykeeo\",\"value\":\"dataps\"},{\"name\":\"qzpkodbquvf\",\"value\":\"datakaaozpc\"},{\"name\":\"ogwfqtqbnakmgydf\",\"value\":\"datakzgwf\"},{\"name\":\"geqzkpergzscr\",\"value\":\"datakankjkszudx\"}],\"\":{\"j\":\"datavxvoqbruyma\"}}],\"ifFalseActivities\":[{\"type\":\"Activity\",\"name\":\"fxirjf\",\"description\":\"ydjaxzstuhlwz\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ymnyfhkxc\",\"dependencyConditions\":[\"Completed\",\"Failed\"],\"\":{\"tupjqfupoam\":\"datawqpuyrpslueacnf\"}},{\"activity\":\"sdzqxkgrecnq\",\"dependencyConditions\":[\"Completed\"],\"\":{\"lhiqodxsscirgq\":\"dataynrsacdcfwuta\",\"yhfnjif\":\"datanfdehhk\",\"cxwjwsrdzmbz\":\"dataucojkikgbhkvhldn\"}},{\"activity\":\"fzydwexoyfseehvm\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Failed\",\"Completed\"],\"\":{\"badbbw\":\"datafwkhipaodo\",\"jxesmbuhkcshyhga\":\"dataaajgokpnbmhskh\",\"npftwgt\":\"datamtevifeoijep\",\"njpwkwxnmqmytv\":\"datacccyiuehsne\"}}],\"userProperties\":[{\"name\":\"utyfnmwmghaeedq\",\"value\":\"datakhc\"},{\"name\":\"wjn\",\"value\":\"datanlbsvtsjv\"},{\"name\":\"vzafpvwrbqbyxuup\",\"value\":\"datakbbemhwtmeqt\"},{\"name\":\"fsjpvjwbxlgpepx\",\"value\":\"datajjnxdg\"}],\"\":{\"pmm\":\"datakltlpbb\",\"tdzgngnuuz\":\"datahmvadasuevu\"}},{\"type\":\"Activity\",\"name\":\"hgfojdbov\",\"description\":\"nelqlqnw\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"mnfnrpqsjzgncyks\",\"dependencyConditions\":[\"Failed\",\"Skipped\"],\"\":{\"ibiyl\":\"datawak\",\"edxdemcyrblwqhz\":\"dataf\",\"gs\":\"datar\",\"pw\":\"databzpozqluuaugktt\"}},{\"activity\":\"olajevww\",\"dependencyConditions\":[\"Failed\",\"Completed\"],\"\":{\"ftcnzokdademqp\":\"dataqeguenteucaojjbb\",\"ay\":\"dataxekmdkbtmupm\",\"ibu\":\"datajocsq\",\"kgvwkdgsrtm\":\"datalppnevujkzb\"}}],\"userProperties\":[{\"name\":\"jygnhmoeoxso\",\"value\":\"dataljzodcxyg\"},{\"name\":\"mjf\",\"value\":\"datamtxfaucihqs\"},{\"name\":\"gtqaoacnlyzizw\",\"value\":\"dataqvgpidrtbcxi\"}],\"\":{\"jvmnooag\":\"datadblvbwueytxl\",\"enxmpiqlnwfb\":\"dataqnek\"}},{\"type\":\"Activity\",\"name\":\"ohxsmhvj\",\"description\":\"vqhc\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"cvfxsvxkcyhkhw\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"tydfyctkrgagxzmr\":\"datambcgrwlenrcov\",\"rhomavvniw\":\"dataxmgzslnnca\",\"yetfxyx\":\"datapooneo\"}},{\"activity\":\"kqvkdpnqcuprl\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"qmzee\":\"datakcueov\"}},{\"activity\":\"itqvqyinifnmc\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Failed\",\"Succeeded\"],\"\":{\"sdcmgmvatnf\":\"datamnam\",\"vaap\":\"datahyrh\",\"uj\":\"datagyyufhcfeggy\"}}],\"userProperties\":[{\"name\":\"vazqsbrqspvl\",\"value\":\"datauxvujuxvl\"},{\"name\":\"x\",\"value\":\"dataftcinj\"},{\"name\":\"rayoask\",\"value\":\"datallqweoobbngym\"}],\"\":{\"wtzxq\":\"dataw\",\"pwvhiaxkm\":\"dataqzplzyjktc\",\"fhlwgka\":\"dataitczuscqobujfx\",\"kmbdhcc\":\"dataxp\"}},{\"type\":\"Activity\",\"name\":\"jotccbduwswf\",\"description\":\"y\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"agmewdmlkxwch\",\"dependencyConditions\":[\"Failed\",\"Failed\"],\"\":{\"qcwaobuimfdaq\":\"datas\",\"g\":\"datanjegom\",\"iyffde\":\"dataazdgeablknqnqqc\"}},{\"activity\":\"g\",\"dependencyConditions\":[\"Failed\",\"Skipped\"],\"\":{\"po\":\"datajlgtcswqxeva\",\"gn\":\"datamxtcnmocskpgn\"}},{\"activity\":\"guqfnhmmvedj\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"lmvvbw\":\"datatpbezlucxbudaj\"}},{\"activity\":\"unr\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Failed\"],\"\":{\"xoahfvkyhfdth\":\"datavxlhfmkl\",\"oyqxbawtisbhke\":\"datazvfzxseqs\",\"rqr\":\"datakgnjravoqcylgytd\"}}],\"userProperties\":[{\"name\":\"xwzewpngcocboh\",\"value\":\"datasjobzvsugentr\"},{\"name\":\"zbw\",\"value\":\"dataivgdc\"},{\"name\":\"rbswbxizmxvd\",\"value\":\"datakmwyikoanep\"},{\"name\":\"knyvnbzglia\",\"value\":\"dataonjdhbq\"}],\"\":{\"ykz\":\"datagsgpefn\",\"lmosqh\":\"dataijqxwm\"}}]},\"name\":\"wbqcofsqruy\",\"description\":\"qemozjz\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"okaujj\",\"dependencyConditions\":[\"Failed\"],\"\":{\"tkh\":\"dataes\",\"rbshyulk\":\"datacnceowv\",\"cpoyda\":\"dataepnmegc\",\"lpigpzpl\":\"datafxmbxqzczcc\"}},{\"activity\":\"aoiid\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"seeuyx\":\"datarugvanpjvqrw\",\"otpvyt\":\"datarwovgwqzzugsbwq\"}},{\"activity\":\"zsqbckq\",\"dependencyConditions\":[\"Completed\",\"Succeeded\"],\"\":{\"kgyuviqeskindgm\":\"dataznkvyicj\"}}],\"userProperties\":[{\"name\":\"uvyuzzw\",\"value\":\"datahuliflxrnsyvmue\"},{\"name\":\"kwvcogqjimxcth\",\"value\":\"datarx\"},{\"name\":\"kinutdhbmizb\",\"value\":\"datavjez\"}],\"\":{\"stcmavln\":\"datauugvdbpjoycp\",\"nvfgwgoxfd\":\"databm\",\"bifpc\":\"datakezoxhazafmq\",\"gjxklojdydha\":\"dataammpeakdhebzquq\"}}") + "{\"type\":\"IfCondition\",\"typeProperties\":{\"expression\":{\"value\":\"wbqcofsqruy\"},\"ifTrueActivities\":[{\"type\":\"Activity\",\"name\":\"em\",\"description\":\"jzhix\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"uj\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Failed\",\"Completed\"],\"\":{\"bshyul\":\"datawtkhfcnceowvi\",\"zcpoydaifxmbxqzc\":\"datahepnmeg\"}},{\"activity\":\"c\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Succeeded\",\"Failed\"],\"\":{\"vanpjvqrwlseeu\":\"datapldaoiidxknsqdru\"}},{\"activity\":\"xxrwov\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Completed\",\"Completed\"],\"\":{\"vytrzsqbckqgte\":\"databwqrot\"}},{\"activity\":\"inznkvyicj\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Completed\",\"Succeeded\"],\"\":{\"buvyuzzwph\":\"dataeskindgmk\",\"ikwvcogq\":\"dataliflxrnsyvmu\",\"cthrrxrki\":\"dataim\"}}],\"userProperties\":[{\"name\":\"dhbmizbevje\",\"value\":\"dataufxuug\"},{\"name\":\"dbpjoycpys\",\"value\":\"datacmavln\"},{\"name\":\"bm\",\"value\":\"datanvfgwgoxfd\"}],\"\":{\"bifpc\":\"datazoxhazafmq\",\"gjxklojdydha\":\"dataammpeakdhebzquq\",\"sxgjih\":\"datafjwm\"}},{\"type\":\"Activity\",\"name\":\"xoxjghumvpt\",\"description\":\"o\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"lcjuzzzi\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"yhpilqojdmz\":\"dataovbgdbaornx\",\"rymzlqircivxaqz\":\"datajcpzzqji\",\"ye\":\"datavgxqtkcvnyi\"}},{\"activity\":\"wfsivg\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"kklz\":\"datadyztnsutes\",\"vtivefsr\":\"datah\",\"dmcoxobrv\":\"datatcxhpntewvfvs\"}}],\"userProperties\":[{\"name\":\"rvnnfieaqbv\",\"value\":\"datauehggeea\"},{\"name\":\"brslbzcyubqemrxm\",\"value\":\"dataibexaxu\"},{\"name\":\"wwqnwxohbmvg\",\"value\":\"datagdnzvohrnqn\"},{\"name\":\"runkyuzcpi\",\"value\":\"dataas\"}],\"\":{\"ascvcmt\":\"datatiocsfpcyyi\",\"adtyhmoph\":\"dataukboryn\"}}],\"ifFalseActivities\":[{\"type\":\"Activity\",\"name\":\"hvnqwdphnc\",\"description\":\"bqij\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"nhrhxhmtxpx\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Skipped\"],\"\":{\"vijdtmjybb\":\"datantiz\",\"sliou\":\"datadhwadnccunrviqrz\"}},{\"activity\":\"a\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Completed\"],\"\":{\"ir\":\"dataggbuajwrrgqudnm\"}}],\"userProperties\":[{\"name\":\"ztkxfhix\",\"value\":\"datauuzaczmej\"},{\"name\":\"iiegpdhit\",\"value\":\"datatketw\"},{\"name\":\"sko\",\"value\":\"datamqhzys\"},{\"name\":\"chbvejgfx\",\"value\":\"datajqevmzhk\"}],\"\":{\"pnxylhrlbohdxln\":\"datangdgk\"}},{\"type\":\"Activity\",\"name\":\"lvcbcxb\",\"description\":\"snhqqqaedgwghqqi\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"uenb\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"opibaxkywqs\":\"dataoopksmj\",\"pfe\":\"datanrcuvlfzd\"}}],\"userProperties\":[{\"name\":\"cahlsavinoora\",\"value\":\"dataspfinyijmwqgmhf\"},{\"name\":\"lbd\",\"value\":\"datadhedmfidro\"},{\"name\":\"fpucwmdmbys\",\"value\":\"dataqbgndfzheyxccx\"}],\"\":{\"uppkzuxsbbmxfut\":\"dataioawrorexicwb\",\"helyopobg\":\"datay\"}}]},\"name\":\"l\",\"description\":\"ki\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"fmpiffgtqhghy\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Failed\"],\"\":{\"hucxmybuqjpgbiya\":\"datagmlaer\",\"fyinh\":\"datagat\"}},{\"activity\":\"vbmbf\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Failed\",\"Failed\"],\"\":{\"yvopotiefxhaqq\":\"datadkdcvowaslswwg\",\"ypuon\":\"datavhfdezomykjbl\"}},{\"activity\":\"vmymfaiw\",\"dependencyConditions\":[\"Skipped\",\"Completed\"],\"\":{\"va\":\"dataddsxsq\",\"roedwipauclety\":\"datayzd\",\"hboqeue\":\"datazziavguskvvnzn\"}},{\"activity\":\"yfl\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"jnsbdw\":\"datazda\"}}],\"userProperties\":[{\"name\":\"q\",\"value\":\"databafrbhrsp\"},{\"name\":\"kvok\",\"value\":\"datamere\"}],\"\":{\"decxbiknf\":\"dataessuwkcn\",\"fxdntpksb\":\"datapixfdojxby\",\"svahbqoojdnmrxj\":\"dataigegwaidqzfl\",\"nkadanl\":\"dataumrzfdbo\"}}") .toObject(IfConditionActivity.class); - Assertions.assertEquals("wbqcofsqruy", model.name()); - Assertions.assertEquals("qemozjz", model.description()); + Assertions.assertEquals("l", model.name()); + Assertions.assertEquals("ki", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("okaujj", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("uvyuzzw", model.userProperties().get(0).name()); - Assertions.assertEquals("abbetzcd", model.expression().value()); - Assertions.assertEquals("wutakbvaqgu", model.ifTrueActivities().get(0).name()); - Assertions.assertEquals("bmcwpllo", model.ifTrueActivities().get(0).description()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("fmpiffgtqhghy", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("q", model.userProperties().get(0).name()); + Assertions.assertEquals("wbqcofsqruy", model.expression().value()); + Assertions.assertEquals("em", model.ifTrueActivities().get(0).name()); + Assertions.assertEquals("jzhix", model.ifTrueActivities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.ifTrueActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.ifTrueActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("wtddig", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.ifTrueActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("uj", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.ifTrueActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("ihn", model.ifTrueActivities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("fxirjf", model.ifFalseActivities().get(0).name()); - Assertions.assertEquals("ydjaxzstuhlwz", model.ifFalseActivities().get(0).description()); + Assertions.assertEquals("dhbmizbevje", model.ifTrueActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("hvnqwdphnc", model.ifFalseActivities().get(0).name()); + Assertions.assertEquals("bqij", model.ifFalseActivities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.ifFalseActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, - model.ifFalseActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("ymnyfhkxc", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.ifFalseActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("nhrhxhmtxpx", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.ifFalseActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("utyfnmwmghaeedq", model.ifFalseActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("ztkxfhix", model.ifFalseActivities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IfConditionActivity model = new IfConditionActivity().withName("wbqcofsqruy").withDescription("qemozjz") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("okaujj") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("aoiid") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + IfConditionActivity model = new IfConditionActivity().withName("l").withDescription("ki") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("fmpiffgtqhghy") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, + DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("vbmbf") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("vmymfaiw") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("zsqbckq") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("yfl") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("uvyuzzw").withValue("datahuliflxrnsyvmue"), - new UserProperty().withName("kwvcogqjimxcth").withValue("datarx"), - new UserProperty().withName("kinutdhbmizb").withValue("datavjez"))) - .withExpression(new Expression().withValue("abbetzcd")) + .withUserProperties(Arrays.asList(new UserProperty().withName("q").withValue("databafrbhrsp"), + new UserProperty().withName("kvok").withValue("datamere"))) + .withExpression(new Expression().withValue("wbqcofsqruy")) .withIfTrueActivities(Arrays.asList( - new Activity().withName("wutakbvaqgu").withDescription("bmcwpllo").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + new Activity().withName("em").withDescription("jzhix").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("wtddig") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.FAILED, DependencyCondition.SKIPPED, DependencyCondition.FAILED)) + new ActivityDependency() + .withActivity("uj") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, + DependencyCondition.FAILED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("mzntroafz") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("c") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("s") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("ihn").withValue("dataqfqawynsl"), - new UserProperty().withName("fx").withValue("dataovavi"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("wyzvnsnak").withDescription("bcuyhmltdgx").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED).withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("jfxueqyjee") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, - DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("xxrwov") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED, + DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("hx") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.FAILED, DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("inznkvyicj") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED, + DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("txarl").withValue("datasijcmn"), - new UserProperty().withName("gp").withValue("datagxjmwzkafuvbg"))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("dhbmizbevje").withValue("dataufxuug"), + new UserProperty().withName("dbpjoycpys").withValue("datacmavln"), + new UserProperty().withName("bm").withValue("datanvfgwgoxfd"))) .withAdditionalProperties(mapOf("type", "Activity")), new Activity() - .withName("gkcxbenwiignrrqv").withDescription("o").withState( + .withName("xoxjghumvpt").withDescription("o").withState( ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("raqesk") + new ActivityDependency().withActivity("lcjuzzzi") .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jufmvozqmt") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("eiehwma") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dowsj") + new ActivityDependency().withActivity("wfsivg") .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("qrpbwykeeo").withValue("dataps"), - new UserProperty().withName("qzpkodbquvf").withValue("datakaaozpc"), - new UserProperty().withName("ogwfqtqbnakmgydf").withValue("datakzgwf"), - new UserProperty().withName("geqzkpergzscr").withValue("datakankjkszudx"))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("rvnnfieaqbv").withValue("datauehggeea"), + new UserProperty().withName("brslbzcyubqemrxm").withValue("dataibexaxu"), + new UserProperty().withName("wwqnwxohbmvg").withValue("datagdnzvohrnqn"), + new UserProperty().withName("runkyuzcpi").withValue("dataas"))) .withAdditionalProperties(mapOf("type", "Activity")))) - .withIfFalseActivities( - Arrays - .asList( - new Activity().withName("fxirjf").withDescription("ydjaxzstuhlwz") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn( - Arrays - .asList( - new ActivityDependency().withActivity("ymnyfhkxc") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("sdzqxkgrecnq").withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED)).withAdditionalProperties( - mapOf()), - new ActivityDependency().withActivity("fzydwexoyfseehvm") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED, - DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty() - .withName("utyfnmwmghaeedq").withValue("datakhc"), - new UserProperty().withName("wjn").withValue("datanlbsvtsjv"), - new UserProperty().withName("vzafpvwrbqbyxuup").withValue("datakbbemhwtmeqt"), - new UserProperty().withName("fsjpvjwbxlgpepx").withValue("datajjnxdg"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("hgfojdbov").withDescription("nelqlqnw") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs( - ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("mnfnrpqsjzgncyks") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("olajevww") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("jygnhmoeoxso").withValue("dataljzodcxyg"), - new UserProperty().withName("mjf") - .withValue("datamtxfaucihqs"), - new UserProperty().withName("gtqaoacnlyzizw").withValue("dataqvgpidrtbcxi"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("ohxsmhvj").withDescription("vqhc").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn( - Arrays - .asList( - new ActivityDependency().withActivity("cvfxsvxkcyhkhw") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kqvkdpnqcuprl") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("itqvqyinifnmc").withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED, - DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays - .asList(new UserProperty().withName("vazqsbrqspvl").withValue("datauxvujuxvl"), - new UserProperty().withName("x").withValue( - "dataftcinj"), - new UserProperty().withName("rayoask").withValue("datallqweoobbngym"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("jotccbduwswf").withDescription("y").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs( - ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("agmewdmlkxwch") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("g") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("guqfnhmmvedj") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("unr") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.SKIPPED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("xwzewpngcocboh").withValue("datasjobzvsugentr"), - new UserProperty().withName("zbw").withValue("dataivgdc"), - new UserProperty().withName("rbswbxizmxvd").withValue("datakmwyikoanep"), - new UserProperty().withName("knyvnbzglia").withValue("dataonjdhbq"))) - .withAdditionalProperties(mapOf("type", "Activity")))); + .withIfFalseActivities(Arrays.asList( + new Activity().withName("hvnqwdphnc").withDescription("bqij").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("nhrhxhmtxpx") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("a") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("ztkxfhix").withValue("datauuzaczmej"), + new UserProperty().withName("iiegpdhit").withValue("datatketw"), + new UserProperty().withName("sko").withValue("datamqhzys"), + new UserProperty().withName("chbvejgfx").withValue("datajqevmzhk"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("lvcbcxb").withDescription("snhqqqaedgwghqqi").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("uenb") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("cahlsavinoora").withValue("dataspfinyijmwqgmhf"), + new UserProperty().withName("lbd").withValue("datadhedmfidro"), + new UserProperty().withName("fpucwmdmbys").withValue("dataqbgndfzheyxccx"))) + .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(IfConditionActivity.class); - Assertions.assertEquals("wbqcofsqruy", model.name()); - Assertions.assertEquals("qemozjz", model.description()); + Assertions.assertEquals("l", model.name()); + Assertions.assertEquals("ki", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("okaujj", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("uvyuzzw", model.userProperties().get(0).name()); - Assertions.assertEquals("abbetzcd", model.expression().value()); - Assertions.assertEquals("wutakbvaqgu", model.ifTrueActivities().get(0).name()); - Assertions.assertEquals("bmcwpllo", model.ifTrueActivities().get(0).description()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("fmpiffgtqhghy", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("q", model.userProperties().get(0).name()); + Assertions.assertEquals("wbqcofsqruy", model.expression().value()); + Assertions.assertEquals("em", model.ifTrueActivities().get(0).name()); + Assertions.assertEquals("jzhix", model.ifTrueActivities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.ifTrueActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.ifTrueActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("wtddig", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.COMPLETED, + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.ifTrueActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("uj", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.ifTrueActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("ihn", model.ifTrueActivities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("fxirjf", model.ifFalseActivities().get(0).name()); - Assertions.assertEquals("ydjaxzstuhlwz", model.ifFalseActivities().get(0).description()); + Assertions.assertEquals("dhbmizbevje", model.ifTrueActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("hvnqwdphnc", model.ifFalseActivities().get(0).name()); + Assertions.assertEquals("bqij", model.ifFalseActivities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.ifFalseActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, - model.ifFalseActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("ymnyfhkxc", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.ifFalseActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("nhrhxhmtxpx", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.COMPLETED, model.ifFalseActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("utyfnmwmghaeedq", model.ifFalseActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("ztkxfhix", model.ifFalseActivities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTypePropertiesTests.java index 3cccdec2a2497..1efb949cf37f2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IfConditionActivityTypePropertiesTests.java @@ -22,138 +22,135 @@ public final class IfConditionActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IfConditionActivityTypeProperties model = BinaryData.fromString( - "{\"expression\":{\"value\":\"fjwm\"},\"ifTrueActivities\":[{\"type\":\"Activity\",\"name\":\"gjihnxoxjghumv\",\"description\":\"bhogllvfealcju\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"crnovbg\",\"dependencyConditions\":[\"Succeeded\",\"Completed\"],\"\":{\"pzzq\":\"dataxsyhpilqojdmzej\",\"rcivxaq\":\"datainrymzlq\"}}],\"userProperties\":[{\"name\":\"gxqtkcvnyiky\",\"value\":\"dataxwfsivgxelzu\"},{\"name\":\"dyztnsutes\",\"value\":\"datakklz\"},{\"name\":\"h\",\"value\":\"datavtivefsr\"}],\"\":{\"vsidmcoxobrvzder\":\"dataxhpntewv\",\"aqbv\":\"datannfi\"}}],\"ifFalseActivities\":[{\"type\":\"Activity\",\"name\":\"hggee\",\"description\":\"brslbzcyubqemrxm\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"u\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Failed\"],\"\":{\"bmvg\":\"datao\"}},{\"activity\":\"gdnzvohrnqn\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Skipped\"],\"\":{\"cyyicascvcmthu\":\"datazcpifasifdtiocsf\",\"phoem\":\"databorynmadtyhm\"}},{\"activity\":\"vnqwd\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"xhmtxpxdtmrwjk\":\"datatbqijeqfoatqnhr\",\"bkdhwadnccunrviq\":\"datatiznvijdtmjy\",\"sliou\":\"dataz\"}}],\"userProperties\":[{\"name\":\"xqnpnpggbu\",\"value\":\"datajw\"},{\"name\":\"rgq\",\"value\":\"datadnmuirtkqztkx\"},{\"name\":\"hixfuuzaczmejf\",\"value\":\"dataiegpdhityt\"}],\"\":{\"chbvejgfx\":\"datawdskocmqhzys\",\"cyngdgka\":\"datajqevmzhk\",\"hrlb\":\"datanxy\"}},{\"type\":\"Activity\",\"name\":\"hd\",\"description\":\"ndlvcbcxbi\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"aedgwghqqiuuet\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"pksmjpopib\":\"datanbllqvro\",\"rcuvlfzdkpf\":\"dataxkywqsp\"}},{\"activity\":\"upacahlsavin\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Completed\",\"Completed\"],\"\":{\"qgmhfvlbd\":\"datainyijm\"}},{\"activity\":\"dhedmfidro\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Failed\"],\"\":{\"xccxeusioawro\":\"datadmbysvqbgndfzhe\",\"icwbquppkzuxsbbm\":\"datae\",\"y\":\"datafut\",\"l\":\"datahelyopobg\"}},{\"activity\":\"ukiwjezadkfmp\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Completed\"],\"\":{\"erxhucxmybuqj\":\"dataghygzayazrgml\",\"axga\":\"datagbi\",\"pvbmbf\":\"datazfyin\"}}],\"userProperties\":[{\"name\":\"uamdydkdcvowasl\",\"value\":\"datawwgzyvo\"},{\"name\":\"otief\",\"value\":\"datahaqqavhfdezom\"}],\"\":{\"ymfaiwxlrphadds\":\"datablmypuonuv\",\"yzd\":\"datasqcva\",\"zziavguskvvnzn\":\"dataroedwipauclety\"}},{\"type\":\"Activity\",\"name\":\"hboqeue\",\"description\":\"fldjkkva\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"n\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"frb\":\"datafuqqb\"}},{\"activity\":\"rspzkvoknmeredn\",\"dependencyConditions\":[\"Skipped\",\"Failed\"],\"\":{\"decxbiknf\":\"datakcn\",\"fxdntpksb\":\"datapixfdojxby\",\"svahbqoojdnmrxj\":\"dataigegwaidqzfl\"}}],\"userProperties\":[{\"name\":\"rzfdbotnkad\",\"value\":\"datan\"}],\"\":{\"ozvcxxezur\":\"datalrmbgiaoxpf\",\"pbleazvyftklbb\":\"datagucns\",\"rl\":\"dataibgczkk\",\"xlhec\":\"datatdkwibdrivedsh\"}},{\"type\":\"Activity\",\"name\":\"tmwwmybviw\",\"description\":\"jey\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"krhp\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"kcad\":\"datad\",\"rbfa\":\"datayoouvqpc\",\"wzufmyanectf\":\"dataduzzyxlyuw\"}},{\"activity\":\"bfgmgho\",\"dependencyConditions\":[\"Completed\"],\"\":{\"rvqcxr\":\"datant\"}},{\"activity\":\"kcvclq\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Succeeded\",\"Succeeded\"],\"\":{\"hkkktlodsyyzmf\":\"datazs\",\"ejmzbasxapcegtc\":\"datagzljgrtfic\",\"kttjnneynm\":\"dataufet\",\"xgxqdmvfdocjaf\":\"datavqysghk\"}},{\"activity\":\"fiddnktutwcz\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Skipped\",\"Skipped\"],\"\":{\"mobsjudpeed\":\"dataqhyq\",\"igtsrrlelpobm\":\"dataowverhtyc\"}}],\"userProperties\":[{\"name\":\"sisll\",\"value\":\"dataqgluhr\"}],\"\":{\"bzcmwnyudcv\":\"dataojozhdcptxxbuf\",\"rngiffsnt\":\"dataeowepv\",\"zagurgur\":\"datapfqguovqqrcyeumw\"}}]}") + "{\"expression\":{\"value\":\"qlrmbgiaoxpfko\"},\"ifTrueActivities\":[{\"type\":\"Activity\",\"name\":\"xxezur\",\"description\":\"ucnssp\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ftklbbribgc\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Succeeded\"],\"\":{\"xlhec\":\"datatdkwibdrivedsh\"}},{\"activity\":\"tmwwmybviw\",\"dependencyConditions\":[\"Succeeded\",\"Completed\"],\"\":{\"udskcadkyoo\":\"datajglponkrhpyed\",\"f\":\"datavqpcjr\"}},{\"activity\":\"yduzzyx\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Completed\"],\"\":{\"nectfpbfgmghohox\":\"dataufmy\",\"vqcxrrkc\":\"dataonts\",\"zs\":\"dataclqlibpmfn\"}}],\"userProperties\":[{\"name\":\"kktlodsyyzmf\",\"value\":\"datagzljgrtfic\"},{\"name\":\"ejmzbasxapcegtc\",\"value\":\"dataufet\"}],\"\":{\"xgxqdmvfdocjaf\":\"datatjnneynmgvqysghk\",\"wmtfjzuqhyqvm\":\"datafiddnktutwcz\",\"dpeedzowverhtyc\":\"databsj\",\"mdsisll\":\"dataigtsrrlelpobm\"}},{\"type\":\"Activity\",\"name\":\"qgluhr\",\"description\":\"mojozhdcptxx\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"mwnyudcvqeo\",\"dependencyConditions\":[\"Skipped\",\"Failed\"],\"\":{\"ntopfqguovqqrc\":\"datangiff\",\"rpcguwyu\":\"dataeumwvzagurg\",\"qsrqebjgo\":\"datahkefowncudcrwo\"}},{\"activity\":\"mjcahda\",\"dependencyConditions\":[\"Skipped\",\"Skipped\"],\"\":{\"mseao\":\"dataf\",\"bslwxcf\":\"dataq\",\"wnmnxppgfep\":\"datavedxyeba\"}}],\"userProperties\":[{\"name\":\"djva\",\"value\":\"dataae\"},{\"name\":\"oqknz\",\"value\":\"datanvvkfbmrppjf\"},{\"name\":\"eabgpw\",\"value\":\"datas\"}],\"\":{\"n\":\"datai\",\"telimqxwih\":\"datavdjmvzcycg\",\"hz\":\"datapyexjrguziglr\",\"isklotwnppstpq\":\"datamrvgcbf\"}}],\"ifFalseActivities\":[{\"type\":\"Activity\",\"name\":\"eawolhlfffe\",\"description\":\"bmhqy\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"coqtvxhipchdpdev\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"eshxomt\":\"datacik\",\"pypzgdet\":\"datakxpsx\",\"gyhu\":\"datad\",\"zmziiftjig\":\"datasutspocrskkraap\"}},{\"activity\":\"qyzocfyywcfl\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"pwtjoku\":\"datamktbwdfjcepy\",\"bbccbqxwojve\":\"datartqnbdgcnickn\"}},{\"activity\":\"xhf\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\"],\"\":{\"hzwyyyerzbmlhg\":\"datafyjcenkidlpml\",\"wsyx\":\"datatkthevodddne\",\"ohdifbhtxtcqjg\":\"datafdjftcr\"}},{\"activity\":\"d\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"tnej\":\"dataurjxkpha\",\"vuvh\":\"datafljqzbixlzaa\",\"bneepfjibtsp\":\"dataerjrcxyxepl\",\"eigywj\":\"dataiwfqj\"}}],\"userProperties\":[{\"name\":\"gncscwsefdqnsu\",\"value\":\"dataomln\"}],\"\":{\"crllecquo\":\"datajdcvnanej\",\"wvcyprpog\":\"datagyhkvtofxke\",\"ochpzcgs\":\"dataqvuftkiyghcmpyki\"}},{\"type\":\"Activity\",\"name\":\"pklfnst\",\"description\":\"bpwwo\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"rsgfpds\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"kcji\":\"dataepmttrfun\",\"jjfne\":\"dataoczoiduk\",\"mdffyv\":\"datauqalwjcqbnvbz\"}},{\"activity\":\"d\",\"dependencyConditions\":[\"Completed\"],\"\":{\"okirxyffttsdt\":\"dataryvkubfotgivpor\",\"gtrjzimxz\":\"dataql\",\"uladdujzenagmh\":\"datauqcinjejyinlys\"}},{\"activity\":\"mgtbqzftmpgibm\",\"dependencyConditions\":[\"Completed\"],\"\":{\"yjvjyxueuqc\":\"datacprbwsndloldxm\",\"gxak\":\"datagbs\",\"uyokctymsbhdi\":\"datakbryolzbmdntajgg\",\"s\":\"datazao\"}},{\"activity\":\"nxgk\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"euwpivsltlyqc\":\"dataukbpwwfeixm\",\"qcmsrzrcddlzga\":\"datapwndcjr\",\"optrudpm\":\"dataptwqfgqccond\"}}],\"userProperties\":[{\"name\":\"loflcilrafkrvv\",\"value\":\"datawknymqzmui\"},{\"name\":\"uvtgjgpcvdjin\",\"value\":\"dataoslzrbz\"},{\"name\":\"f\",\"value\":\"datavwcjrbjgdvwa\"}],\"\":{\"svximqkuyflzx\":\"datacnevkfkmena\"}}]}") .toObject(IfConditionActivityTypeProperties.class); - Assertions.assertEquals("fjwm", model.expression().value()); - Assertions.assertEquals("gjihnxoxjghumv", model.ifTrueActivities().get(0).name()); - Assertions.assertEquals("bhogllvfealcju", model.ifTrueActivities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.ifTrueActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.ifTrueActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("crnovbg", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, + Assertions.assertEquals("qlrmbgiaoxpfko", model.expression().value()); + Assertions.assertEquals("xxezur", model.ifTrueActivities().get(0).name()); + Assertions.assertEquals("ucnssp", model.ifTrueActivities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.ifTrueActivities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.ifTrueActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("ftklbbribgc", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.ifTrueActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("gxqtkcvnyiky", model.ifTrueActivities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("hggee", model.ifFalseActivities().get(0).name()); - Assertions.assertEquals("brslbzcyubqemrxm", model.ifFalseActivities().get(0).description()); + Assertions.assertEquals("kktlodsyyzmf", model.ifTrueActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("eawolhlfffe", model.ifFalseActivities().get(0).name()); + Assertions.assertEquals("bmhqy", model.ifFalseActivities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.ifFalseActivities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.ifFalseActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("u", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, + Assertions.assertEquals("coqtvxhipchdpdev", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.ifFalseActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("xqnpnpggbu", model.ifFalseActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("gncscwsefdqnsu", model.ifFalseActivities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { IfConditionActivityTypeProperties model - = new IfConditionActivityTypeProperties().withExpression(new Expression().withValue("fjwm")) - .withIfTrueActivities( - Arrays.asList(new Activity().withName("gjihnxoxjghumv").withDescription("bhogllvfealcju") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("crnovbg") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("gxqtkcvnyiky").withValue("dataxwfsivgxelzu"), - new UserProperty().withName("dyztnsutes").withValue("datakklz"), - new UserProperty().withName("h").withValue("datavtivefsr"))) - .withAdditionalProperties(mapOf("type", "Activity")))) - .withIfFalseActivities(Arrays.asList( - new Activity().withName("hggee").withDescription("brslbzcyubqemrxm").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + = new IfConditionActivityTypeProperties() + .withExpression( + new Expression().withValue("qlrmbgiaoxpfko")) + .withIfTrueActivities(Arrays.asList( + new Activity().withName("xxezur").withDescription("ucnssp").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("u") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("gdnzvohrnqn") + new ActivityDependency().withActivity("ftklbbribgc") .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.FAILED, DependencyCondition.SKIPPED)) + DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("vnqwd") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("xqnpnpggbu").withValue("datajw"), - new UserProperty().withName("rgq").withValue("datadnmuirtkqztkx"), - new UserProperty().withName("hixfuuzaczmejf").withValue("dataiegpdhityt"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("hd").withDescription("ndlvcbcxbi").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("aedgwghqqiuuet") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("upacahlsavin") + new ActivityDependency().withActivity("tmwwmybviw") .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, - DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dhedmfidro") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ukiwjezadkfmp") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("yduzzyx") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + DependencyCondition.FAILED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("uamdydkdcvowasl").withValue("datawwgzyvo"), new UserProperty() - .withName("otief").withValue("datahaqqavhfdezom"))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("kktlodsyyzmf").withValue("datagzljgrtfic"), + new UserProperty().withName("ejmzbasxapcegtc").withValue("dataufet"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("hboqeue").withDescription("fldjkkva").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + new Activity() + .withName("qgluhr").withDescription("mojozhdcptxx").withState( + ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("n") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("rspzkvoknmeredn") + new ActivityDependency().withActivity("mwnyudcvqeo") .withDependencyConditions( Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("mjcahda") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("rzfdbotnkad") - .withValue("datan"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("tmwwmybviw").withDescription("jey").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withUserProperties(Arrays.asList(new UserProperty().withName("djva").withValue("dataae"), + new UserProperty().withName("oqknz") + .withValue("datanvvkfbmrppjf"), + new UserProperty().withName("eabgpw").withValue("datas"))) + .withAdditionalProperties(mapOf("type", "Activity")))) + .withIfFalseActivities(Arrays.asList( + new Activity().withName("eawolhlfffe").withDescription("bmhqy").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs( + ActivityOnInactiveMarkAs.FAILED) .withDependsOn( Arrays.asList( - new ActivityDependency().withActivity("krhp") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("coqtvxhipchdpdev") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("bfgmgho") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("qyzocfyywcfl") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kcvclq") - .withDependencyConditions(Arrays.asList( - DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("xhf") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fiddnktutwcz") + new ActivityDependency().withActivity("d") .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("sisll").withValue("dataqgluhr"))) + .withUserProperties(Arrays.asList(new UserProperty().withName("gncscwsefdqnsu") + .withValue("dataomln"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("pklfnst").withDescription("bpwwo").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("rsgfpds") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("d") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("mgtbqzftmpgibm") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("nxgk") + .withDependencyConditions( + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("loflcilrafkrvv").withValue("datawknymqzmui"), + new UserProperty().withName("uvtgjgpcvdjin").withValue("dataoslzrbz"), + new UserProperty().withName("f").withValue("datavwcjrbjgdvwa"))) .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(IfConditionActivityTypeProperties.class); - Assertions.assertEquals("fjwm", model.expression().value()); - Assertions.assertEquals("gjihnxoxjghumv", model.ifTrueActivities().get(0).name()); - Assertions.assertEquals("bhogllvfealcju", model.ifTrueActivities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.ifTrueActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.ifTrueActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("crnovbg", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, + Assertions.assertEquals("qlrmbgiaoxpfko", model.expression().value()); + Assertions.assertEquals("xxezur", model.ifTrueActivities().get(0).name()); + Assertions.assertEquals("ucnssp", model.ifTrueActivities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.ifTrueActivities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.ifTrueActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("ftklbbribgc", model.ifTrueActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.ifTrueActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("gxqtkcvnyiky", model.ifTrueActivities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("hggee", model.ifFalseActivities().get(0).name()); - Assertions.assertEquals("brslbzcyubqemrxm", model.ifFalseActivities().get(0).description()); + Assertions.assertEquals("kktlodsyyzmf", model.ifTrueActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("eawolhlfffe", model.ifFalseActivities().get(0).name()); + Assertions.assertEquals("bmhqy", model.ifFalseActivities().get(0).description()); Assertions.assertEquals(ActivityState.ACTIVE, model.ifFalseActivities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.ifFalseActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("u", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, + Assertions.assertEquals("coqtvxhipchdpdev", model.ifFalseActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.ifFalseActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("xqnpnpggbu", model.ifFalseActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("gncscwsefdqnsu", model.ifFalseActivities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaDatasetTypePropertiesTests.java index bd00b0b78895b..d3b98fae5d401 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaDatasetTypePropertiesTests.java @@ -10,15 +10,15 @@ public final class ImpalaDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ImpalaDatasetTypeProperties model - = BinaryData.fromString("{\"tableName\":\"datauklx\",\"table\":\"datalmzpyq\",\"schema\":\"datahuecxhgs\"}") - .toObject(ImpalaDatasetTypeProperties.class); + ImpalaDatasetTypeProperties model = BinaryData + .fromString("{\"tableName\":\"datamby\",\"table\":\"datavwnbu\",\"schema\":\"dataodtevzshqykebmps\"}") + .toObject(ImpalaDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImpalaDatasetTypeProperties model = new ImpalaDatasetTypeProperties().withTableName("datauklx") - .withTable("datalmzpyq").withSchema("datahuecxhgs"); + ImpalaDatasetTypeProperties model = new ImpalaDatasetTypeProperties().withTableName("datamby") + .withTable("datavwnbu").withSchema("dataodtevzshqykebmps"); model = BinaryData.fromObject(model).toObject(ImpalaDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaObjectDatasetTests.java index f7210c63758b0..ef3b8faec8b9e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaObjectDatasetTests.java @@ -19,32 +19,32 @@ public final class ImpalaObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ImpalaObjectDataset model = BinaryData.fromString( - "{\"type\":\"ImpalaObject\",\"typeProperties\":{\"tableName\":\"datamdjmvph\",\"table\":\"datanugslvfzzioxbg\",\"schema\":\"datavueprpmofxnwcgz\"},\"description\":\"xixtxxxajsehbknn\",\"structure\":\"datakyjfawpcbsog\",\"schema\":\"datahczbnivco\",\"linkedServiceName\":{\"referenceName\":\"sxvppkjealkdb\",\"parameters\":{\"qdkt\":\"dataotvbmyzuqf\",\"jndkvzmx\":\"datajtoqszhhqn\",\"nkqyipgkm\":\"dataffqgdo\",\"ftgdrfzjlflza\":\"datatdazmdzesim\"}},\"parameters\":{\"orzbidaebeznicew\":{\"type\":\"Object\",\"defaultValue\":\"dataabxief\"},\"wwsr\":{\"type\":\"Array\",\"defaultValue\":\"datajwiylciobb\"}},\"annotations\":[\"dataxuecuuue\"],\"folder\":{\"name\":\"nteevfgaxfez\"},\"\":{\"kyrxgmzzeglwd\":\"datasddkodkgxq\",\"kkraj\":\"datafsspfegaoksd\"}}") + "{\"type\":\"ImpalaObject\",\"typeProperties\":{\"tableName\":\"dataxiefcorzbidaeb\",\"table\":\"datanicewd\",\"schema\":\"datajwiylciobb\"},\"description\":\"ws\",\"structure\":\"dataeqx\",\"schema\":\"datacuuuexsmnteevfg\",\"linkedServiceName\":{\"referenceName\":\"xfezraqsddko\",\"parameters\":{\"w\":\"dataxqfkyrxgmzzeg\",\"fegaok\":\"datazfss\"}},\"parameters\":{\"fyuklxkelmz\":{\"type\":\"String\",\"defaultValue\":\"datara\"},\"gsingmhpavsfg\":{\"type\":\"SecureString\",\"defaultValue\":\"databwhuecx\"},\"klj\":{\"type\":\"Float\",\"defaultValue\":\"dataqrwwbdrwro\"}},\"annotations\":[\"dataqhqq\",\"dataarkyulfamea\",\"datasjqenh\"],\"folder\":{\"name\":\"azvgeytlplslfcv\"},\"\":{\"jocrhnxzmfvmw\":\"datasuowtolkyqf\",\"rawwhyxf\":\"datanrtc\"}}") .toObject(ImpalaObjectDataset.class); - Assertions.assertEquals("xixtxxxajsehbknn", model.description()); - Assertions.assertEquals("sxvppkjealkdb", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("orzbidaebeznicew").type()); - Assertions.assertEquals("nteevfgaxfez", model.folder().name()); + Assertions.assertEquals("ws", model.description()); + Assertions.assertEquals("xfezraqsddko", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("fyuklxkelmz").type()); + Assertions.assertEquals("azvgeytlplslfcv", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImpalaObjectDataset model = new ImpalaObjectDataset().withDescription("xixtxxxajsehbknn") - .withStructure("datakyjfawpcbsog").withSchema("datahczbnivco") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("sxvppkjealkdb") - .withParameters(mapOf("qdkt", "dataotvbmyzuqf", "jndkvzmx", "datajtoqszhhqn", "nkqyipgkm", "dataffqgdo", - "ftgdrfzjlflza", "datatdazmdzesim"))) - .withParameters(mapOf("orzbidaebeznicew", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataabxief"), "wwsr", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datajwiylciobb"))) - .withAnnotations(Arrays.asList("dataxuecuuue")).withFolder(new DatasetFolder().withName("nteevfgaxfez")) - .withTableName("datamdjmvph").withTable("datanugslvfzzioxbg") - .withSchemaTypePropertiesSchema("datavueprpmofxnwcgz"); + ImpalaObjectDataset model = new ImpalaObjectDataset().withDescription("ws").withStructure("dataeqx") + .withSchema("datacuuuexsmnteevfg") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("xfezraqsddko") + .withParameters(mapOf("w", "dataxqfkyrxgmzzeg", "fegaok", "datazfss"))) + .withParameters(mapOf("fyuklxkelmz", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datara"), "gsingmhpavsfg", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("databwhuecx"), + "klj", new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataqrwwbdrwro"))) + .withAnnotations(Arrays.asList("dataqhqq", "dataarkyulfamea", "datasjqenh")) + .withFolder(new DatasetFolder().withName("azvgeytlplslfcv")).withTableName("dataxiefcorzbidaeb") + .withTable("datanicewd").withSchemaTypePropertiesSchema("datajwiylciobb"); model = BinaryData.fromObject(model).toObject(ImpalaObjectDataset.class); - Assertions.assertEquals("xixtxxxajsehbknn", model.description()); - Assertions.assertEquals("sxvppkjealkdb", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("orzbidaebeznicew").type()); - Assertions.assertEquals("nteevfgaxfez", model.folder().name()); + Assertions.assertEquals("ws", model.description()); + Assertions.assertEquals("xfezraqsddko", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("fyuklxkelmz").type()); + Assertions.assertEquals("azvgeytlplslfcv", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaSourceTests.java index 782754941d640..78421c27fda7c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImpalaSourceTests.java @@ -11,16 +11,15 @@ public final class ImpalaSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ImpalaSource model = BinaryData.fromString( - "{\"type\":\"ImpalaSource\",\"query\":\"dataljq\",\"queryTimeout\":\"datax\",\"additionalColumns\":\"dataakgd\",\"sourceRetryCount\":\"datanmhvwgchgpbdkqw\",\"sourceRetryWait\":\"dataomapcaxnoqnjfv\",\"maxConcurrentConnections\":\"datavectooxjztt\",\"disableMetricsCollection\":\"datasnmxvsrvkzvxlez\",\"\":{\"n\":\"dataybxehjkqog\",\"osrsfaocr\":\"datalaxspghfvkqijmy\",\"hnmruvvlwh\":\"datazrrgdpy\",\"ywbhxh\":\"datafscoupsf\"}}") + "{\"type\":\"ImpalaSource\",\"query\":\"databrcdumkqhatckom\",\"queryTimeout\":\"datafjs\",\"additionalColumns\":\"datavzvkddaeiepvjr\",\"sourceRetryCount\":\"dataksx\",\"sourceRetryWait\":\"datakb\",\"maxConcurrentConnections\":\"datauawokrhhj\",\"disableMetricsCollection\":\"datahrmuwvs\",\"\":{\"imgg\":\"datauosidtxmbnm\"}}") .toObject(ImpalaSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImpalaSource model - = new ImpalaSource().withSourceRetryCount("datanmhvwgchgpbdkqw").withSourceRetryWait("dataomapcaxnoqnjfv") - .withMaxConcurrentConnections("datavectooxjztt").withDisableMetricsCollection("datasnmxvsrvkzvxlez") - .withQueryTimeout("datax").withAdditionalColumns("dataakgd").withQuery("dataljq"); + ImpalaSource model = new ImpalaSource().withSourceRetryCount("dataksx").withSourceRetryWait("datakb") + .withMaxConcurrentConnections("datauawokrhhj").withDisableMetricsCollection("datahrmuwvs") + .withQueryTimeout("datafjs").withAdditionalColumns("datavzvkddaeiepvjr").withQuery("databrcdumkqhatckom"); model = BinaryData.fromObject(model).toObject(ImpalaSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImportSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImportSettingsTests.java index aa836e099683e..06bb9450de2ec 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImportSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ImportSettingsTests.java @@ -12,9 +12,9 @@ public final class ImportSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ImportSettings model = BinaryData.fromString( - "{\"type\":\"ImportSettings\",\"\":{\"zaf\":\"datarnfikireetvjf\",\"uvndgrolgxajcsi\":\"datagjh\",\"pjmqteirrjj\":\"datafwlyeiaj\"}}") - .toObject(ImportSettings.class); + ImportSettings model + = BinaryData.fromString("{\"type\":\"ImportSettings\",\"\":{\"qmbnfvygttdcfjal\":\"dataojvcr\"}}") + .toObject(ImportSettings.class); } @org.junit.jupiter.api.Test diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSinkTests.java index eede06cc4d607..5ae99cddb3279 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSinkTests.java @@ -11,16 +11,16 @@ public final class InformixSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { InformixSink model = BinaryData.fromString( - "{\"type\":\"InformixSink\",\"preCopyScript\":\"datamre\",\"writeBatchSize\":\"datapx\",\"writeBatchTimeout\":\"datae\",\"sinkRetryCount\":\"datar\",\"sinkRetryWait\":\"datalfnuglmyr\",\"maxConcurrentConnections\":\"datatdkpoxzwgrsnit\",\"disableMetricsCollection\":\"datacaehjhwkl\",\"\":{\"nwe\":\"dataqqgyp\",\"ytk\":\"databngojnaks\"}}") + "{\"type\":\"InformixSink\",\"preCopyScript\":\"dataamgjy\",\"writeBatchSize\":\"datakttit\",\"writeBatchTimeout\":\"datamnx\",\"sinkRetryCount\":\"dataoadjooer\",\"sinkRetryWait\":\"datalzzmy\",\"maxConcurrentConnections\":\"datautqebpuoycawptxq\",\"disableMetricsCollection\":\"dataufdxpwj\",\"\":{\"cuk\":\"datavskpbuoc\",\"cepp\":\"datatcuvwwfgjjcaa\"}}") .toObject(InformixSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - InformixSink model - = new InformixSink().withWriteBatchSize("datapx").withWriteBatchTimeout("datae").withSinkRetryCount("datar") - .withSinkRetryWait("datalfnuglmyr").withMaxConcurrentConnections("datatdkpoxzwgrsnit") - .withDisableMetricsCollection("datacaehjhwkl").withPreCopyScript("datamre"); + InformixSink model = new InformixSink().withWriteBatchSize("datakttit").withWriteBatchTimeout("datamnx") + .withSinkRetryCount("dataoadjooer").withSinkRetryWait("datalzzmy") + .withMaxConcurrentConnections("datautqebpuoycawptxq").withDisableMetricsCollection("dataufdxpwj") + .withPreCopyScript("dataamgjy"); model = BinaryData.fromObject(model).toObject(InformixSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSourceTests.java index 5d924243ce567..e4ed3e83b6cb9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/InformixSourceTests.java @@ -11,16 +11,15 @@ public final class InformixSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { InformixSource model = BinaryData.fromString( - "{\"type\":\"InformixSource\",\"query\":\"dataiiuexyqy\",\"queryTimeout\":\"dataudmlqpward\",\"additionalColumns\":\"datar\",\"sourceRetryCount\":\"datavfiskkqspzwsxn\",\"sourceRetryWait\":\"datackpcssusdr\",\"maxConcurrentConnections\":\"datammrzwm\",\"disableMetricsCollection\":\"datatkcvolaxnuk\",\"\":{\"oxyxiyhmjwsn\":\"dataoumndc\"}}") + "{\"type\":\"InformixSource\",\"query\":\"datazuzvbqbroyrw\",\"queryTimeout\":\"databbfweozkbok\",\"additionalColumns\":\"datasu\",\"sourceRetryCount\":\"datacslzca\",\"sourceRetryWait\":\"datad\",\"maxConcurrentConnections\":\"datafwkpupbsgfnqtxl\",\"disableMetricsCollection\":\"dataoviklxsgstunsatc\",\"\":{\"tgsazwx\":\"datadbehkbuajkodpz\",\"hasjbuhz\":\"datafaas\"}}") .toObject(InformixSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - InformixSource model - = new InformixSource().withSourceRetryCount("datavfiskkqspzwsxn").withSourceRetryWait("datackpcssusdr") - .withMaxConcurrentConnections("datammrzwm").withDisableMetricsCollection("datatkcvolaxnuk") - .withQueryTimeout("dataudmlqpward").withAdditionalColumns("datar").withQuery("dataiiuexyqy"); + InformixSource model = new InformixSource().withSourceRetryCount("datacslzca").withSourceRetryWait("datad") + .withMaxConcurrentConnections("datafwkpupbsgfnqtxl").withDisableMetricsCollection("dataoviklxsgstunsatc") + .withQueryTimeout("databbfweozkbok").withAdditionalColumns("datasu").withQuery("datazuzvbqbroyrw"); model = BinaryData.fromObject(model).toObject(InformixSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeComputePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeComputePropertiesTests.java index f87a118c4dc97..fde879d78f0b2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeComputePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeComputePropertiesTests.java @@ -21,71 +21,75 @@ public final class IntegrationRuntimeComputePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IntegrationRuntimeComputeProperties model = BinaryData.fromString( - "{\"location\":\"szk\",\"nodeSize\":\"bhhpvvaeghtmqonx\",\"numberOfNodes\":388235383,\"maxParallelExecutionsPerNode\":1765220864,\"dataFlowProperties\":{\"computeType\":\"ComputeOptimized\",\"coreCount\":526856500,\"timeToLive\":409228969,\"cleanup\":true,\"customProperties\":[{\"name\":\"brnbox\",\"value\":\"hkmqf\"},{\"name\":\"ksksmqmwo\",\"value\":\"mhmifhfutjyx\"}],\"\":{\"mdqr\":\"datagtguqaemoygcaxj\"}},\"vNetProperties\":{\"vNetId\":\"icxtcllq\",\"subnet\":\"sydksvjfg\",\"publicIPs\":[\"pivjkh\",\"oscoljjhcsgzooef\"],\"subnetId\":\"dttbqkaqdvwoj\",\"\":{\"hrqxrqghotingzi\":\"dataf\",\"elvyhvvpuqy\":\"datakgygawyhpwmdkyfg\",\"cfwoolkugzowg\":\"datapubbkhci\"}},\"copyComputeScaleProperties\":{\"dataIntegrationUnit\":480875321,\"timeToLive\":285819339,\"\":{\"pqscu\":\"datapgblnagjnpahz\",\"gmndwohoeashu\":\"dataileqjzrijebmuio\",\"mzwynsmmphvkyez\":\"datafvbj\"}},\"pipelineExternalComputeScaleProperties\":{\"timeToLive\":1245337089,\"numberOfPipelineNodes\":1528577821,\"numberOfExternalNodes\":1451105056,\"\":{\"lqreof\":\"datawqd\"}},\"\":{\"pyaymh\":\"datarz\",\"qm\":\"datalnmwaxsymnrt\",\"hhpkvyqpvzxxzndw\":\"datamavyotpcvp\",\"r\":\"dataomi\"}}") + "{\"location\":\"wojvxv\",\"nodeSize\":\"hrqxrqghotingzi\",\"numberOfNodes\":896793765,\"maxParallelExecutionsPerNode\":117425644,\"dataFlowProperties\":{\"computeType\":\"MemoryOptimized\",\"coreCount\":199030014,\"timeToLive\":1937656001,\"cleanup\":false,\"customProperties\":[{\"name\":\"yelvy\",\"value\":\"vpuqyrp\"},{\"name\":\"bkhcidcfw\",\"value\":\"lkugzowgmmixfzau\"},{\"name\":\"blnagjnpahzhpqsc\",\"value\":\"ileqjzrijebmuio\"},{\"name\":\"mndwohoeashuxfvb\",\"value\":\"mzwynsmmphvkyez\"}],\"\":{\"jwq\":\"datayuoyj\",\"yaymhx\":\"dataslqreofzrkrzt\",\"vyotpcvpahhpk\":\"datanmwaxsymnrtvqmim\",\"dwtominru\":\"datayqpvzxxz\"}},\"vNetProperties\":{\"vNetId\":\"auygasfmhbxvew\",\"subnet\":\"natxvuzccaliry\",\"publicIPs\":[\"caq\",\"johlcbnrvxyyhh\"],\"subnetId\":\"sztqfrpan\",\"\":{\"imkjzcx\":\"dataiwduukaa\",\"f\":\"datasj\",\"xijovuyxuupzeadd\":\"datavksijrjgyind\",\"zy\":\"datatopdtphvjgv\"}},\"copyComputeScaleProperties\":{\"dataIntegrationUnit\":1732999100,\"timeToLive\":1693962581,\"\":{\"nyzpu\":\"dataxzxbth\",\"koabfcvefbw\":\"datanrmd\"}},\"pipelineExternalComputeScaleProperties\":{\"timeToLive\":1354669108,\"numberOfPipelineNodes\":1621925601,\"numberOfExternalNodes\":66962308,\"\":{\"grz\":\"datasmryuyutkbx\",\"yncyzjnd\":\"datakyqguauuihkybg\"}},\"\":{\"vypmwdz\":\"datamxiurpf\",\"qimyhxnpdggllyd\":\"datajpcroxpp\",\"jnstz\":\"datayufdmzucxvzvwlx\",\"oxklrzats\":\"dataur\"}}") .toObject(IntegrationRuntimeComputeProperties.class); - Assertions.assertEquals("szk", model.location()); - Assertions.assertEquals("bhhpvvaeghtmqonx", model.nodeSize()); - Assertions.assertEquals(388235383, model.numberOfNodes()); - Assertions.assertEquals(1765220864, model.maxParallelExecutionsPerNode()); - Assertions.assertEquals(DataFlowComputeType.COMPUTE_OPTIMIZED, model.dataFlowProperties().computeType()); - Assertions.assertEquals(526856500, model.dataFlowProperties().coreCount()); - Assertions.assertEquals(409228969, model.dataFlowProperties().timeToLive()); - Assertions.assertEquals(true, model.dataFlowProperties().cleanup()); - Assertions.assertEquals("brnbox", model.dataFlowProperties().customProperties().get(0).name()); - Assertions.assertEquals("hkmqf", model.dataFlowProperties().customProperties().get(0).value()); - Assertions.assertEquals("icxtcllq", model.vNetProperties().vNetId()); - Assertions.assertEquals("sydksvjfg", model.vNetProperties().subnet()); - Assertions.assertEquals("pivjkh", model.vNetProperties().publicIPs().get(0)); - Assertions.assertEquals("dttbqkaqdvwoj", model.vNetProperties().subnetId()); - Assertions.assertEquals(480875321, model.copyComputeScaleProperties().dataIntegrationUnit()); - Assertions.assertEquals(285819339, model.copyComputeScaleProperties().timeToLive()); - Assertions.assertEquals(1245337089, model.pipelineExternalComputeScaleProperties().timeToLive()); - Assertions.assertEquals(1528577821, model.pipelineExternalComputeScaleProperties().numberOfPipelineNodes()); - Assertions.assertEquals(1451105056, model.pipelineExternalComputeScaleProperties().numberOfExternalNodes()); + Assertions.assertEquals("wojvxv", model.location()); + Assertions.assertEquals("hrqxrqghotingzi", model.nodeSize()); + Assertions.assertEquals(896793765, model.numberOfNodes()); + Assertions.assertEquals(117425644, model.maxParallelExecutionsPerNode()); + Assertions.assertEquals(DataFlowComputeType.MEMORY_OPTIMIZED, model.dataFlowProperties().computeType()); + Assertions.assertEquals(199030014, model.dataFlowProperties().coreCount()); + Assertions.assertEquals(1937656001, model.dataFlowProperties().timeToLive()); + Assertions.assertEquals(false, model.dataFlowProperties().cleanup()); + Assertions.assertEquals("yelvy", model.dataFlowProperties().customProperties().get(0).name()); + Assertions.assertEquals("vpuqyrp", model.dataFlowProperties().customProperties().get(0).value()); + Assertions.assertEquals("auygasfmhbxvew", model.vNetProperties().vNetId()); + Assertions.assertEquals("natxvuzccaliry", model.vNetProperties().subnet()); + Assertions.assertEquals("caq", model.vNetProperties().publicIPs().get(0)); + Assertions.assertEquals("sztqfrpan", model.vNetProperties().subnetId()); + Assertions.assertEquals(1732999100, model.copyComputeScaleProperties().dataIntegrationUnit()); + Assertions.assertEquals(1693962581, model.copyComputeScaleProperties().timeToLive()); + Assertions.assertEquals(1354669108, model.pipelineExternalComputeScaleProperties().timeToLive()); + Assertions.assertEquals(1621925601, model.pipelineExternalComputeScaleProperties().numberOfPipelineNodes()); + Assertions.assertEquals(66962308, model.pipelineExternalComputeScaleProperties().numberOfExternalNodes()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IntegrationRuntimeComputeProperties model = new IntegrationRuntimeComputeProperties().withLocation("szk") - .withNodeSize("bhhpvvaeghtmqonx").withNumberOfNodes(388235383).withMaxParallelExecutionsPerNode(1765220864) + IntegrationRuntimeComputeProperties model = new IntegrationRuntimeComputeProperties().withLocation("wojvxv") + .withNodeSize("hrqxrqghotingzi").withNumberOfNodes(896793765).withMaxParallelExecutionsPerNode(117425644) .withDataFlowProperties( - new IntegrationRuntimeDataFlowProperties().withComputeType(DataFlowComputeType.COMPUTE_OPTIMIZED) - .withCoreCount(526856500).withTimeToLive(409228969).withCleanup(true) + new IntegrationRuntimeDataFlowProperties().withComputeType(DataFlowComputeType.MEMORY_OPTIMIZED) + .withCoreCount(199030014).withTimeToLive(1937656001).withCleanup(false) .withCustomProperties(Arrays.asList( - new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("brnbox") - .withValue("hkmqf"), - new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("ksksmqmwo") - .withValue("mhmifhfutjyx"))) + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("yelvy") + .withValue("vpuqyrp"), + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("bkhcidcfw") + .withValue("lkugzowgmmixfzau"), + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("blnagjnpahzhpqsc") + .withValue("ileqjzrijebmuio"), + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("mndwohoeashuxfvb") + .withValue("mzwynsmmphvkyez"))) .withAdditionalProperties(mapOf())) - .withVNetProperties(new IntegrationRuntimeVNetProperties().withVNetId("icxtcllq").withSubnet("sydksvjfg") - .withPublicIPs(Arrays.asList("pivjkh", "oscoljjhcsgzooef")).withSubnetId("dttbqkaqdvwoj") - .withAdditionalProperties(mapOf())) - .withCopyComputeScaleProperties(new CopyComputeScaleProperties().withDataIntegrationUnit(480875321) - .withTimeToLive(285819339).withAdditionalProperties(mapOf())) + .withVNetProperties(new IntegrationRuntimeVNetProperties().withVNetId("auygasfmhbxvew") + .withSubnet("natxvuzccaliry").withPublicIPs(Arrays.asList("caq", "johlcbnrvxyyhh")) + .withSubnetId("sztqfrpan").withAdditionalProperties(mapOf())) + .withCopyComputeScaleProperties(new CopyComputeScaleProperties().withDataIntegrationUnit(1732999100) + .withTimeToLive(1693962581).withAdditionalProperties(mapOf())) .withPipelineExternalComputeScaleProperties(new PipelineExternalComputeScaleProperties() - .withTimeToLive(1245337089).withNumberOfPipelineNodes(1528577821).withNumberOfExternalNodes(1451105056) + .withTimeToLive(1354669108).withNumberOfPipelineNodes(1621925601).withNumberOfExternalNodes(66962308) .withAdditionalProperties(mapOf())) .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(IntegrationRuntimeComputeProperties.class); - Assertions.assertEquals("szk", model.location()); - Assertions.assertEquals("bhhpvvaeghtmqonx", model.nodeSize()); - Assertions.assertEquals(388235383, model.numberOfNodes()); - Assertions.assertEquals(1765220864, model.maxParallelExecutionsPerNode()); - Assertions.assertEquals(DataFlowComputeType.COMPUTE_OPTIMIZED, model.dataFlowProperties().computeType()); - Assertions.assertEquals(526856500, model.dataFlowProperties().coreCount()); - Assertions.assertEquals(409228969, model.dataFlowProperties().timeToLive()); - Assertions.assertEquals(true, model.dataFlowProperties().cleanup()); - Assertions.assertEquals("brnbox", model.dataFlowProperties().customProperties().get(0).name()); - Assertions.assertEquals("hkmqf", model.dataFlowProperties().customProperties().get(0).value()); - Assertions.assertEquals("icxtcllq", model.vNetProperties().vNetId()); - Assertions.assertEquals("sydksvjfg", model.vNetProperties().subnet()); - Assertions.assertEquals("pivjkh", model.vNetProperties().publicIPs().get(0)); - Assertions.assertEquals("dttbqkaqdvwoj", model.vNetProperties().subnetId()); - Assertions.assertEquals(480875321, model.copyComputeScaleProperties().dataIntegrationUnit()); - Assertions.assertEquals(285819339, model.copyComputeScaleProperties().timeToLive()); - Assertions.assertEquals(1245337089, model.pipelineExternalComputeScaleProperties().timeToLive()); - Assertions.assertEquals(1528577821, model.pipelineExternalComputeScaleProperties().numberOfPipelineNodes()); - Assertions.assertEquals(1451105056, model.pipelineExternalComputeScaleProperties().numberOfExternalNodes()); + Assertions.assertEquals("wojvxv", model.location()); + Assertions.assertEquals("hrqxrqghotingzi", model.nodeSize()); + Assertions.assertEquals(896793765, model.numberOfNodes()); + Assertions.assertEquals(117425644, model.maxParallelExecutionsPerNode()); + Assertions.assertEquals(DataFlowComputeType.MEMORY_OPTIMIZED, model.dataFlowProperties().computeType()); + Assertions.assertEquals(199030014, model.dataFlowProperties().coreCount()); + Assertions.assertEquals(1937656001, model.dataFlowProperties().timeToLive()); + Assertions.assertEquals(false, model.dataFlowProperties().cleanup()); + Assertions.assertEquals("yelvy", model.dataFlowProperties().customProperties().get(0).name()); + Assertions.assertEquals("vpuqyrp", model.dataFlowProperties().customProperties().get(0).value()); + Assertions.assertEquals("auygasfmhbxvew", model.vNetProperties().vNetId()); + Assertions.assertEquals("natxvuzccaliry", model.vNetProperties().subnet()); + Assertions.assertEquals("caq", model.vNetProperties().publicIPs().get(0)); + Assertions.assertEquals("sztqfrpan", model.vNetProperties().subnetId()); + Assertions.assertEquals(1732999100, model.copyComputeScaleProperties().dataIntegrationUnit()); + Assertions.assertEquals(1693962581, model.copyComputeScaleProperties().timeToLive()); + Assertions.assertEquals(1354669108, model.pipelineExternalComputeScaleProperties().timeToLive()); + Assertions.assertEquals(1621925601, model.pipelineExternalComputeScaleProperties().numberOfPipelineNodes()); + Assertions.assertEquals(66962308, model.pipelineExternalComputeScaleProperties().numberOfExternalNodes()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeCustomerVirtualNetworkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeCustomerVirtualNetworkTests.java index bb1a69b1c6021..2787a0ad4996c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeCustomerVirtualNetworkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeCustomerVirtualNetworkTests.java @@ -11,16 +11,16 @@ public final class IntegrationRuntimeCustomerVirtualNetworkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - IntegrationRuntimeCustomerVirtualNetwork model - = BinaryData.fromString("{\"subnetId\":\"ndyv\"}").toObject(IntegrationRuntimeCustomerVirtualNetwork.class); - Assertions.assertEquals("ndyv", model.subnetId()); + IntegrationRuntimeCustomerVirtualNetwork model = BinaryData.fromString("{\"subnetId\":\"hoodttq\"}") + .toObject(IntegrationRuntimeCustomerVirtualNetwork.class); + Assertions.assertEquals("hoodttq", model.subnetId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { IntegrationRuntimeCustomerVirtualNetwork model - = new IntegrationRuntimeCustomerVirtualNetwork().withSubnetId("ndyv"); + = new IntegrationRuntimeCustomerVirtualNetwork().withSubnetId("hoodttq"); model = BinaryData.fromObject(model).toObject(IntegrationRuntimeCustomerVirtualNetwork.class); - Assertions.assertEquals("ndyv", model.subnetId()); + Assertions.assertEquals("hoodttq", model.subnetId()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesCustomPropertiesItemTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesCustomPropertiesItemTests.java index d211752b67c7b..906b5e408cafc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesCustomPropertiesItemTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesCustomPropertiesItemTests.java @@ -12,18 +12,18 @@ public final class IntegrationRuntimeDataFlowPropertiesCustomPropertiesItemTests @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem model - = BinaryData.fromString("{\"name\":\"kaamim\",\"value\":\"zc\"}") + = BinaryData.fromString("{\"name\":\"vzhn\",\"value\":\"vtoiqofzttqg\"}") .toObject(IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem.class); - Assertions.assertEquals("kaamim", model.name()); - Assertions.assertEquals("zc", model.value()); + Assertions.assertEquals("vzhn", model.name()); + Assertions.assertEquals("vtoiqofzttqg", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem model - = new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("kaamim").withValue("zc"); + = new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("vzhn").withValue("vtoiqofzttqg"); model = BinaryData.fromObject(model).toObject(IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem.class); - Assertions.assertEquals("kaamim", model.name()); - Assertions.assertEquals("zc", model.value()); + Assertions.assertEquals("vzhn", model.name()); + Assertions.assertEquals("vtoiqofzttqg", model.value()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesTests.java index 634eaf8a76b84..e837f9eda7174 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataFlowPropertiesTests.java @@ -17,32 +17,36 @@ public final class IntegrationRuntimeDataFlowPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IntegrationRuntimeDataFlowProperties model = BinaryData.fromString( - "{\"computeType\":\"ComputeOptimized\",\"coreCount\":1479856776,\"timeToLive\":2091496981,\"cleanup\":false,\"customProperties\":[{\"name\":\"hbxve\",\"value\":\"q\"},{\"name\":\"t\",\"value\":\"uzc\"}],\"\":{\"nrvxyyh\":\"datairybytcaqpjohlc\",\"tq\":\"datasis\",\"teqiwd\":\"datarpa\"}}") + "{\"computeType\":\"MemoryOptimized\",\"coreCount\":2135115629,\"timeToLive\":421136528,\"cleanup\":true,\"customProperties\":[{\"name\":\"hqhnfubevwadxcez\",\"value\":\"vltfebqoqiaklqa\"},{\"name\":\"stif\",\"value\":\"twrphmriipzgf\"},{\"name\":\"u\",\"value\":\"cjqnea\"},{\"name\":\"gttbargeey\",\"value\":\"ls\"}],\"\":{\"hsppvjsduouoqte\":\"dataaxoyvgjjpfy\",\"apaseqcppypfre\":\"dataqsomuogeq\"}}") .toObject(IntegrationRuntimeDataFlowProperties.class); - Assertions.assertEquals(DataFlowComputeType.COMPUTE_OPTIMIZED, model.computeType()); - Assertions.assertEquals(1479856776, model.coreCount()); - Assertions.assertEquals(2091496981, model.timeToLive()); - Assertions.assertEquals(false, model.cleanup()); - Assertions.assertEquals("hbxve", model.customProperties().get(0).name()); - Assertions.assertEquals("q", model.customProperties().get(0).value()); + Assertions.assertEquals(DataFlowComputeType.MEMORY_OPTIMIZED, model.computeType()); + Assertions.assertEquals(2135115629, model.coreCount()); + Assertions.assertEquals(421136528, model.timeToLive()); + Assertions.assertEquals(true, model.cleanup()); + Assertions.assertEquals("hqhnfubevwadxcez", model.customProperties().get(0).name()); + Assertions.assertEquals("vltfebqoqiaklqa", model.customProperties().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IntegrationRuntimeDataFlowProperties model - = new IntegrationRuntimeDataFlowProperties().withComputeType(DataFlowComputeType.COMPUTE_OPTIMIZED) - .withCoreCount(1479856776).withTimeToLive(2091496981).withCleanup(false) - .withCustomProperties(Arrays.asList( - new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("hbxve").withValue("q"), - new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("t").withValue("uzc"))) - .withAdditionalProperties(mapOf()); + IntegrationRuntimeDataFlowProperties model = new IntegrationRuntimeDataFlowProperties() + .withComputeType(DataFlowComputeType.MEMORY_OPTIMIZED).withCoreCount(2135115629).withTimeToLive(421136528) + .withCleanup(true) + .withCustomProperties(Arrays.asList( + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("hqhnfubevwadxcez") + .withValue("vltfebqoqiaklqa"), + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("stif") + .withValue("twrphmriipzgf"), + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("u").withValue("cjqnea"), + new IntegrationRuntimeDataFlowPropertiesCustomPropertiesItem().withName("gttbargeey").withValue("ls"))) + .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(IntegrationRuntimeDataFlowProperties.class); - Assertions.assertEquals(DataFlowComputeType.COMPUTE_OPTIMIZED, model.computeType()); - Assertions.assertEquals(1479856776, model.coreCount()); - Assertions.assertEquals(2091496981, model.timeToLive()); - Assertions.assertEquals(false, model.cleanup()); - Assertions.assertEquals("hbxve", model.customProperties().get(0).name()); - Assertions.assertEquals("q", model.customProperties().get(0).value()); + Assertions.assertEquals(DataFlowComputeType.MEMORY_OPTIMIZED, model.computeType()); + Assertions.assertEquals(2135115629, model.coreCount()); + Assertions.assertEquals(421136528, model.timeToLive()); + Assertions.assertEquals(true, model.cleanup()); + Assertions.assertEquals("hqhnfubevwadxcez", model.customProperties().get(0).name()); + Assertions.assertEquals("vltfebqoqiaklqa", model.customProperties().get(0).value()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataProxyPropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataProxyPropertiesTests.java index 217696b72bab8..d773ec5c5f18e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataProxyPropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeDataProxyPropertiesTests.java @@ -14,15 +14,15 @@ public final class IntegrationRuntimeDataProxyPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IntegrationRuntimeDataProxyProperties model = BinaryData.fromString( - "{\"connectVia\":{\"type\":\"LinkedServiceReference\",\"referenceName\":\"bargeeynqlsnrga\"},\"stagingLinkedService\":{\"type\":\"LinkedServiceReference\",\"referenceName\":\"jjpfyxhsppvjs\"},\"path\":\"ouoq\"}") + "{\"connectVia\":{\"type\":\"LinkedServiceReference\",\"referenceName\":\"icbicbogsfo\"},\"stagingLinkedService\":{\"type\":\"LinkedServiceReference\",\"referenceName\":\"iyf\"},\"path\":\"nrukcy\"}") .toObject(IntegrationRuntimeDataProxyProperties.class); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE, model.connectVia().type()); - Assertions.assertEquals("bargeeynqlsnrga", model.connectVia().referenceName()); + Assertions.assertEquals("icbicbogsfo", model.connectVia().referenceName()); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE, model.stagingLinkedService().type()); - Assertions.assertEquals("jjpfyxhsppvjs", model.stagingLinkedService().referenceName()); - Assertions.assertEquals("ouoq", model.path()); + Assertions.assertEquals("iyf", model.stagingLinkedService().referenceName()); + Assertions.assertEquals("nrukcy", model.path()); } @org.junit.jupiter.api.Test @@ -30,18 +30,17 @@ public void testSerialize() throws Exception { IntegrationRuntimeDataProxyProperties model = new IntegrationRuntimeDataProxyProperties() .withConnectVia( new EntityReference().withType(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE) - .withReferenceName("bargeeynqlsnrga")) - .withStagingLinkedService( - new EntityReference().withType(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE) - .withReferenceName("jjpfyxhsppvjs")) - .withPath("ouoq"); + .withReferenceName("icbicbogsfo")) + .withStagingLinkedService(new EntityReference() + .withType(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE).withReferenceName("iyf")) + .withPath("nrukcy"); model = BinaryData.fromObject(model).toObject(IntegrationRuntimeDataProxyProperties.class); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE, model.connectVia().type()); - Assertions.assertEquals("bargeeynqlsnrga", model.connectVia().referenceName()); + Assertions.assertEquals("icbicbogsfo", model.connectVia().referenceName()); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.LINKED_SERVICE_REFERENCE, model.stagingLinkedService().type()); - Assertions.assertEquals("jjpfyxhsppvjs", model.stagingLinkedService().referenceName()); - Assertions.assertEquals("ouoq", model.path()); + Assertions.assertEquals("iyf", model.stagingLinkedService().referenceName()); + Assertions.assertEquals("nrukcy", model.path()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesDeleteWithResponseMockTests.java index 8ab9e63013b27..e22c2680ea614 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.integrationRuntimeNodes().deleteWithResponse("lc", "dnajncefqn", "rgbmuim", "erauoht", + manager.integrationRuntimeNodes().deleteWithResponse("vdqfkjg", "lcfoaabltvltt", "plxbxfrl", "yikcnlbehxoy", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetIpAddressWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetIpAddressWithResponseMockTests.java index f897ae6fe7d7a..ef740fda3bd85 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetIpAddressWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetIpAddressWithResponseMockTests.java @@ -29,7 +29,7 @@ public void testGetIpAddressWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"ipAddress\":\"jujsovd\"}"; + String responseStr = "{\"ipAddress\":\"j\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -46,9 +46,9 @@ public void testGetIpAddressWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - IntegrationRuntimeNodeIpAddress response - = manager.integrationRuntimeNodes().getIpAddressWithResponse("bfekxbcbumjy", "ukezqohthsmdua", "y", - "ryuwuypouvpdc", com.azure.core.util.Context.NONE).getValue(); + IntegrationRuntimeNodeIpAddress response = manager.integrationRuntimeNodes() + .getIpAddressWithResponse("xzhbfibzvxqh", "pjdbz", "lchv", "sydjr", com.azure.core.util.Context.NONE) + .getValue(); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetWithResponseMockTests.java index 45716d1121095..c4f3a06c7e02f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesGetWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"nodeName\":\"jzhhjgvuvjsn\",\"machineName\":\"nuujkjkqyewtlom\",\"hostServiceUri\":\"xaqvrazthdua\",\"status\":\"InitializeFailed\",\"capabilities\":{\"mgheamxidjdpt\":\"hlnhgngqciiopo\"},\"versionStatus\":\"ie\",\"version\":\"auyphugwauipatod\",\"registerTime\":\"2021-11-05T05:14:24Z\",\"lastConnectTime\":\"2021-04-02T01:24:07Z\",\"expiryTime\":\"2021-02-18T13:41:39Z\",\"lastStartTime\":\"2021-12-02T05:39:26Z\",\"lastStopTime\":\"2021-03-24T03:50:49Z\",\"lastUpdateResult\":\"Fail\",\"lastStartUpdateTime\":\"2021-07-06T18:00:42Z\",\"lastEndUpdateTime\":\"2021-08-13T17:30:02Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":1673002875,\"maxConcurrentJobs\":633341735,\"\":{\"blndlahr\":\"dataulefltubwpe\",\"tkehfoephipho\":\"datax\",\"mv\":\"datagmcuqjouk\",\"ogfxbv\":\"dataqismvo\"}}"; + = "{\"nodeName\":\"jxtzy\",\"machineName\":\"qmp\",\"hostServiceUri\":\"xiioasvykt\",\"status\":\"Limited\",\"capabilities\":{\"egqzqdcohsqufsy\":\"wjwl\",\"z\":\"hsnzsjoxuogyakex\",\"nbmngstvnkshaul\":\"lh\",\"oqyin\":\"tvlyl\"},\"versionStatus\":\"gxncoaiyflvvmd\",\"version\":\"ytaocxak\",\"registerTime\":\"2021-02-13T13:27:18Z\",\"lastConnectTime\":\"2021-11-19T07:48:54Z\",\"expiryTime\":\"2021-11-08T15:41:18Z\",\"lastStartTime\":\"2021-10-24T21:52:16Z\",\"lastStopTime\":\"2021-04-19T16:57:35Z\",\"lastUpdateResult\":\"None\",\"lastStartUpdateTime\":\"2021-06-01T19:59:49Z\",\"lastEndUpdateTime\":\"2021-06-04T07:00:41Z\",\"isActiveDispatcher\":false,\"concurrentJobsLimit\":482006335,\"maxConcurrentJobs\":2014408540,\"\":{\"vgusfrkjfrtauf\":\"datalrwwmukx\",\"qmjodvknxj\":\"dataxxvzqin\",\"fqodc\":\"datattkhmhquca\"}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,7 +48,8 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SelfHostedIntegrationRuntimeNode response = manager.integrationRuntimeNodes() - .getWithResponse("iqcjylkdby", "bxjkmavppo", "gimt", "ucls", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("dsygdzzufr", "ewqwdglmfsjpl", "dhzltmywy", "fuovkgqtzg", com.azure.core.util.Context.NONE) + .getValue(); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesUpdateWithResponseMockTests.java index 544f5c4d3376f..3ea6506b02bf4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeNodesUpdateWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"nodeName\":\"sbyfoavozqnn\",\"machineName\":\"xitvmrqbkzchcm\",\"hostServiceUri\":\"skd\",\"status\":\"Initializing\",\"capabilities\":{\"hvqmdoqyo\":\"bwxcabfrvjpfo\",\"suwghtgpgarh\":\"zhundfkpdxfvjdf\"},\"versionStatus\":\"adedivad\",\"version\":\"xvqp\",\"registerTime\":\"2021-03-18T12:51:02Z\",\"lastConnectTime\":\"2021-07-06T05:00:38Z\",\"expiryTime\":\"2020-12-21T21:58:57Z\",\"lastStartTime\":\"2021-01-07T03:04:27Z\",\"lastStopTime\":\"2021-03-16T05:08:47Z\",\"lastUpdateResult\":\"Fail\",\"lastStartUpdateTime\":\"2021-02-26T06:50:06Z\",\"lastEndUpdateTime\":\"2021-07-17T09:02:07Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":690651643,\"maxConcurrentJobs\":1094919284,\"\":{\"teqypt\":\"datanuivpbjclihfzrii\",\"tkqbvtdeouqixgtp\":\"datajqjoamzdsajn\",\"jjuwdvfa\":\"datakbjev\",\"ucobpkphxh\":\"datalbfrch\"}}"; + = "{\"nodeName\":\"xqra\",\"machineName\":\"nkeodgpqdcrnubnt\",\"hostServiceUri\":\"ohtuiwsnccmunhv\",\"status\":\"Initializing\",\"capabilities\":{\"yzfuvbnelmimmc\":\"zvuiprngneymxzd\",\"egfthgjmznp\":\"evbprecge\"},\"versionStatus\":\"vafczgisegdei\",\"version\":\"lcdqxownbjkwgkgo\",\"registerTime\":\"2021-01-21T18:45:42Z\",\"lastConnectTime\":\"2021-10-26T03:12:01Z\",\"expiryTime\":\"2021-06-21T19:10:04Z\",\"lastStartTime\":\"2021-10-30T09:20:31Z\",\"lastStopTime\":\"2021-04-13T23:04:59Z\",\"lastUpdateResult\":\"Fail\",\"lastStartUpdateTime\":\"2021-10-03T06:43:31Z\",\"lastEndUpdateTime\":\"2020-12-30T11:38:38Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":2040293763,\"maxConcurrentJobs\":1930006622,\"\":{\"kshtgfewflxby\":\"dataqqos\",\"ldtmeendocqaptwk\":\"datavaufxxvs\",\"z\":\"dataismonwhazalftta\",\"srduqhrlltfec\":\"datasve\"}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,8 +49,8 @@ public void testUpdateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SelfHostedIntegrationRuntimeNode response = manager.integrationRuntimeNodes() - .updateWithResponse("jtahdtdceuhjxvc", "rx", "eeyptvrbgcprsds", "wozpmhhdnxwkf", - new UpdateIntegrationRuntimeNodeRequest().withConcurrentJobsLimit(1227772272), + .updateWithResponse("xjqysfejddiog", "ckvoxlihfg", "fznzemisqunxwos", "nchrouvtbptdeum", + new UpdateIntegrationRuntimeNodeRequest().withConcurrentJobsLimit(637814538), com.azure.core.util.Context.NONE) .getValue(); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasGetWithResponseMockTests.java index 25294286af6a0..d638707e12f3b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasGetWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"type\":\"SsisObjectMetadata\",\"id\":6879995143339576966,\"name\":\"ruwljfdcy\",\"description\":\"qq\"},{\"type\":\"SsisObjectMetadata\",\"id\":5293849071624099090,\"name\":\"ywbnerygsifsa\",\"description\":\"ccsvajn\"},{\"type\":\"SsisObjectMetadata\",\"id\":8970486371925458316,\"name\":\"yrv\",\"description\":\"ojkysolmzrfhlyn\"},{\"type\":\"SsisObjectMetadata\",\"id\":3036714402389451136,\"name\":\"yysbjt\",\"description\":\"fhnqx\"}],\"nextLink\":\"emvqxxuwsat\"}"; + = "{\"value\":[{\"type\":\"SsisObjectMetadata\",\"id\":8310288998494192853,\"name\":\"zpxlyabjrz\",\"description\":\"sjfwurhkuxp\"}],\"nextLink\":\"wmbgwgmyglnsnkyl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,13 +50,14 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SsisObjectMetadataListResponse response = manager.integrationRuntimeObjectMetadatas() - .getWithResponse("pwabrzrhdezlhs", "c", "dbolczhyqd", - new GetSsisObjectMetadataRequest().withMetadataPath("qoajfoscdemfat"), com.azure.core.util.Context.NONE) + .getWithResponse("igh", "xx", "betmqugovcd", + new GetSsisObjectMetadataRequest().withMetadataPath("lrbsfqrgjejabqv"), + com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals(6879995143339576966L, response.value().get(0).id()); - Assertions.assertEquals("ruwljfdcy", response.value().get(0).name()); - Assertions.assertEquals("qq", response.value().get(0).description()); - Assertions.assertEquals("emvqxxuwsat", response.nextLink()); + Assertions.assertEquals(8310288998494192853L, response.value().get(0).id()); + Assertions.assertEquals("zpxlyabjrz", response.value().get(0).name()); + Assertions.assertEquals("sjfwurhkuxp", response.value().get(0).description()); + Assertions.assertEquals("wmbgwgmyglnsnkyl", response.nextLink()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasRefreshMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasRefreshMockTests.java index ffbbf73e57efa..3d2a96e33a198 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasRefreshMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeObjectMetadatasRefreshMockTests.java @@ -31,7 +31,7 @@ public void testRefresh() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"status\":\"ridzqoiqonnva\",\"name\":\"o\",\"properties\":\"igzlvqmydpoj\",\"error\":\"fixdgkvlzeadq\"}"; + = "{\"status\":\"oshkzibbjbzdnkg\",\"name\":\"bvicwfrybvhg\",\"properties\":\"tjghdfusphokcc\",\"error\":\"nnmpnnq\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,12 +48,12 @@ public void testRefresh() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - SsisObjectMetadataStatusResponse response = manager.integrationRuntimeObjectMetadatas() - .refresh("zifuovgipqgtsgov", "rk", "bhtcrxcnuy", com.azure.core.util.Context.NONE); + SsisObjectMetadataStatusResponse response = manager.integrationRuntimeObjectMetadatas().refresh("yfxsdntukoss", + "flfv", "ygecly", com.azure.core.util.Context.NONE); - Assertions.assertEquals("ridzqoiqonnva", response.status()); - Assertions.assertEquals("o", response.name()); - Assertions.assertEquals("igzlvqmydpoj", response.properties()); - Assertions.assertEquals("fixdgkvlzeadq", response.error()); + Assertions.assertEquals("oshkzibbjbzdnkg", response.status()); + Assertions.assertEquals("bvicwfrybvhg", response.name()); + Assertions.assertEquals("tjghdfusphokcc", response.properties()); + Assertions.assertEquals("nnmpnnq", response.error()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeVNetPropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeVNetPropertiesTests.java index be82d772a80e8..a6cfef509b4f9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeVNetPropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimeVNetPropertiesTests.java @@ -15,24 +15,24 @@ public final class IntegrationRuntimeVNetPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { IntegrationRuntimeVNetProperties model = BinaryData.fromString( - "{\"vNetId\":\"sj\",\"subnet\":\"xvksij\",\"publicIPs\":[\"yindex\",\"jovuyx\",\"up\"],\"subnetId\":\"addat\",\"\":{\"vj\":\"datatphvj\",\"htnyz\":\"dataykngqyiyjxzxb\"}}") + "{\"vNetId\":\"lwyoxzuhellitpqv\",\"subnet\":\"vrsgqbm\",\"publicIPs\":[\"eomebz\"],\"subnetId\":\"vxxfsfo\",\"\":{\"edybkbgdwbmi\":\"datagihnalpc\",\"i\":\"datazikatywedb\"}}") .toObject(IntegrationRuntimeVNetProperties.class); - Assertions.assertEquals("sj", model.vNetId()); - Assertions.assertEquals("xvksij", model.subnet()); - Assertions.assertEquals("yindex", model.publicIPs().get(0)); - Assertions.assertEquals("addat", model.subnetId()); + Assertions.assertEquals("lwyoxzuhellitpqv", model.vNetId()); + Assertions.assertEquals("vrsgqbm", model.subnet()); + Assertions.assertEquals("eomebz", model.publicIPs().get(0)); + Assertions.assertEquals("vxxfsfo", model.subnetId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IntegrationRuntimeVNetProperties model = new IntegrationRuntimeVNetProperties().withVNetId("sj") - .withSubnet("xvksij").withPublicIPs(Arrays.asList("yindex", "jovuyx", "up")).withSubnetId("addat") - .withAdditionalProperties(mapOf()); + IntegrationRuntimeVNetProperties model + = new IntegrationRuntimeVNetProperties().withVNetId("lwyoxzuhellitpqv").withSubnet("vrsgqbm") + .withPublicIPs(Arrays.asList("eomebz")).withSubnetId("vxxfsfo").withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(IntegrationRuntimeVNetProperties.class); - Assertions.assertEquals("sj", model.vNetId()); - Assertions.assertEquals("xvksij", model.subnet()); - Assertions.assertEquals("yindex", model.publicIPs().get(0)); - Assertions.assertEquals("addat", model.subnetId()); + Assertions.assertEquals("lwyoxzuhellitpqv", model.vNetId()); + Assertions.assertEquals("vrsgqbm", model.subnet()); + Assertions.assertEquals("eomebz", model.publicIPs().get(0)); + Assertions.assertEquals("vxxfsfo", model.subnetId()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateLinkedIntegrationRuntimeWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateLinkedIntegrationRuntimeWithResponseMockTests.java index 71beaa4c9eb74..e84d48aab54d7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateLinkedIntegrationRuntimeWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateLinkedIntegrationRuntimeWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testCreateLinkedIntegrationRuntimeWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"name\":\"c\",\"properties\":{\"type\":\"IntegrationRuntimeStatus\",\"dataFactoryName\":\"j\",\"state\":\"Stopped\",\"\":{\"tfsciayclvaivsa\":\"databliegzj\",\"cqj\":\"datarfjhcrqnwoahfaq\",\"tvfs\":\"datawvqif\"}}}"; + = "{\"name\":\"wvcsekwpgdfpoqb\",\"properties\":{\"type\":\"IntegrationRuntimeStatus\",\"dataFactoryName\":\"kqsabyowfr\",\"state\":\"Starting\",\"\":{\"flgtq\":\"datafbdsnc\"}}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,9 +49,9 @@ public void testCreateLinkedIntegrationRuntimeWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); IntegrationRuntimeStatusResponse response = manager.integrationRuntimes() - .createLinkedIntegrationRuntimeWithResponse("pnyu", "fh", "ighnunptjmz", - new CreateLinkedIntegrationRuntimeRequest().withName("rjnddaov").withSubscriptionId("o") - .withDataFactoryName("ztrln").withDataFactoryLocation("vjdv"), + .createLinkedIntegrationRuntimeWithResponse("llcsdgmcjsktej", "mhttiqbnfyixkeav", "ezzpfldd", + new CreateLinkedIntegrationRuntimeRequest().withName("cwhodfw").withSubscriptionId("xrfr") + .withDataFactoryName("byktlo").withDataFactoryLocation("p"), com.azure.core.util.Context.NONE) .getValue(); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateOrUpdateWithResponseMockTests.java index 64d2793b6918b..204d7e908904b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesCreateOrUpdateWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"IntegrationRuntime\",\"description\":\"rdhxamjhpqfj\",\"\":{\"cillfq\":\"datairjkinofw\"}},\"name\":\"yifdrbkprblwsb\",\"type\":\"ekqqtsrupogtrwku\",\"etag\":\"abfjeoqwng\",\"id\":\"y\"}"; + = "{\"properties\":{\"type\":\"IntegrationRuntime\",\"description\":\"fhsgpy\",\"\":{\"lpsjbnnuqszy\":\"datatsdjn\",\"fata\":\"dataoiufrqsmjgddbunx\",\"p\":\"datas\",\"livvnyzc\":\"datagdwhacurmmbuna\"}},\"name\":\"wisuhareqyiadvvg\",\"type\":\"fyel\",\"etag\":\"l\",\"id\":\"yyuxcj\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,14 +51,13 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - IntegrationRuntimeResource response - = manager.integrationRuntimes().define("qivvpvuy").withExistingFactory("rjdpzvh", "ssn") - .withProperties(new IntegrationRuntime().withDescription("n") - .withAdditionalProperties(mapOf("type", "IntegrationRuntime"))) - .withIfMatch("litcydgesflnzibg").create(); + IntegrationRuntimeResource response = manager.integrationRuntimes().define("jxud") + .withExistingFactory("owtazqexwkkjx", "jomnkeaiamh").withProperties(new IntegrationRuntime() + .withDescription("lz").withAdditionalProperties(mapOf("type", "IntegrationRuntime"))) + .withIfMatch("uyxccra").create(); - Assertions.assertEquals("y", response.id()); - Assertions.assertEquals("rdhxamjhpqfj", response.properties().description()); + Assertions.assertEquals("yyuxcj", response.id()); + Assertions.assertEquals("fhsgpy", response.properties().description()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesDeleteWithResponseMockTests.java index eb2a98067d2c6..959ff7cab1633 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.integrationRuntimes().deleteWithResponse("rqcga", "kyofy", "mnfvbfjkvspxxbfq", + manager.integrationRuntimes().deleteWithResponse("xepuvwahfnlk", "yqpkskbid", "zzjpb", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetMonitoringDataWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetMonitoringDataWithResponseMockTests.java index 8c803cfb14539..eb04b391036f6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetMonitoringDataWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetMonitoringDataWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetMonitoringDataWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"name\":\"f\",\"nodes\":[{\"nodeName\":\"ueprpnzbfoldb\",\"availableMemoryInMB\":1536179630,\"cpuUtilization\":2072525779,\"concurrentJobsLimit\":1312014405,\"concurrentJobsRunning\":242782811,\"maxConcurrentJobs\":1274504130,\"sentBytes\":11.889243,\"receivedBytes\":34.154594,\"\":{\"qbnzr\":\"datao\",\"yabvvbsi\":\"datakmanrowdqoj\"}},{\"nodeName\":\"hs\",\"availableMemoryInMB\":546680352,\"cpuUtilization\":284097172,\"concurrentJobsLimit\":81274646,\"concurrentJobsRunning\":1068311231,\"maxConcurrentJobs\":753240513,\"sentBytes\":84.65222,\"receivedBytes\":93.84653,\"\":{\"h\":\"databtgexiwcqe\",\"pbvzbt\":\"datayrzidoyvquufpl\",\"lqrxewdgzfqsr\":\"dataftotpvoehsfwra\"}}]}"; + = "{\"name\":\"gbm\",\"nodes\":[{\"nodeName\":\"erauoht\",\"availableMemoryInMB\":1304587071,\"cpuUtilization\":1389990784,\"concurrentJobsLimit\":1347387035,\"concurrentJobsRunning\":260431426,\"maxConcurrentJobs\":718972733,\"sentBytes\":1.437074,\"receivedBytes\":58.880486,\"\":{\"eeyptvrbgcprsds\":\"datarx\",\"su\":\"datawozpmhhdnxwkf\",\"mxitvmrq\":\"datasbyfoavozqnn\"}},{\"nodeName\":\"zch\",\"availableMemoryInMB\":1378405861,\"cpuUtilization\":735227834,\"concurrentJobsLimit\":1330999405,\"concurrentJobsRunning\":181863906,\"maxConcurrentJobs\":903484269,\"sentBytes\":79.66964,\"receivedBytes\":18.096357,\"\":{\"pfojhvqmdoqyohzh\":\"databfrv\",\"xfvj\":\"datandfkp\",\"arhfeadedivadpcx\":\"datafusuwghtgp\"}},{\"nodeName\":\"pmw\",\"availableMemoryInMB\":1264586477,\"cpuUtilization\":1236271427,\"concurrentJobsLimit\":64649335,\"concurrentJobsRunning\":697187006,\"maxConcurrentJobs\":1636051443,\"sentBytes\":73.32201,\"receivedBytes\":23.75918,\"\":{\"kmnuivpbjcl\":\"datavlc\",\"yp\":\"datahfzriigte\",\"nctkqbvtdeou\":\"datamjqjoamzdsa\",\"juwdvfaulbfrc\":\"dataixgtpykbjevj\"}},{\"nodeName\":\"ucobpkphxh\",\"availableMemoryInMB\":471274210,\"cpuUtilization\":1807912949,\"concurrentJobsLimit\":345365482,\"concurrentJobsRunning\":704883797,\"maxConcurrentJobs\":796383837,\"sentBytes\":91.77441,\"receivedBytes\":82.7453,\"\":{\"wuyp\":\"datazqohthsmduaoypry\",\"jsov\":\"datauvpdclaj\",\"zehxddmaevcjtrw\":\"datafreyrgrgf\"}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,8 +49,8 @@ public void testGetMonitoringDataWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); IntegrationRuntimeMonitoringData response = manager.integrationRuntimes() - .getMonitoringDataWithResponse("fe", "s", "ktreihlszpu", com.azure.core.util.Context.NONE).getValue(); + .getMonitoringDataWithResponse("ogfxbv", "lc", "dnajncefqn", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("f", response.name()); + Assertions.assertEquals("gbm", response.name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetStatusWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetStatusWithResponseMockTests.java index 9c579285dcf10..7e4ecd0fc3a46 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetStatusWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetStatusWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetStatusWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"name\":\"bfcvnhrencxokt\",\"properties\":{\"type\":\"IntegrationRuntimeStatus\",\"dataFactoryName\":\"gnhlpqcctuxxytm\",\"state\":\"Stopped\",\"\":{\"shvyjnrj\":\"dataziafgbfkmqhz\",\"nez\":\"datatnksleurj\",\"wnptmriqeaugid\":\"dataewjxcd\",\"srtzg\":\"datazgst\"}}}"; + = "{\"name\":\"rdhxamjhpqfj\",\"properties\":{\"type\":\"IntegrationRuntimeStatus\",\"dataFactoryName\":\"fi\",\"state\":\"AccessDenied\",\"\":{\"llfq\":\"dataofwzc\"}}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,8 +48,7 @@ public void testGetStatusWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); IntegrationRuntimeStatusResponse response = manager.integrationRuntimes() - .getStatusWithResponse("fkwjiuiryjdwda", "cw", "kxwoqhffnojiq", com.azure.core.util.Context.NONE) - .getValue(); + .getStatusWithResponse("cgesbte", "fenhlitc", "dgesflnzibg", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetWithResponseMockTests.java index 928b04429d996..fc83aab7b5efd 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"IntegrationRuntime\",\"description\":\"myemvyijvvbe\",\"\":{\"zvpgttvykz\":\"datatk\",\"ikhbkcvpubvmsz\":\"datalktenbvpadoseqc\",\"sxncykfq\":\"datazrs\",\"gqctrvfpg\":\"databwes\"}},\"name\":\"l\",\"type\":\"wbavlgovg\",\"etag\":\"ppoddnwhaokkw\",\"id\":\"vimstbyaklfvc\"}"; + = "{\"properties\":{\"type\":\"IntegrationRuntime\",\"description\":\"oahfaqlcqjnwvqif\",\"\":{\"qivvpvuy\":\"datafsvrjdpzvhxssn\",\"aadbwhsvxmvk\":\"datasnmdinnisuua\"}},\"name\":\"fwseoqkal\",\"type\":\"neahowvjup\",\"etag\":\"bupgtrnjz\",\"id\":\"bwabilybmf\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,10 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); IntegrationRuntimeResource response = manager.integrationRuntimes() - .getWithResponse("anhjslkoa", "ocsetkrt", "xjiiqsxecejlyhu", "hqqqqcdzmh", com.azure.core.util.Context.NONE) + .getWithResponse("xmsiblieg", "jktfsci", "yclv", "ivsagrfjhcrq", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("vimstbyaklfvc", response.id()); - Assertions.assertEquals("myemvyijvvbe", response.properties().description()); + Assertions.assertEquals("bwabilybmf", response.id()); + Assertions.assertEquals("oahfaqlcqjnwvqif", response.properties().description()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListByFactoryMockTests.java index e74c1930000bb..153487bc3c4b8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"type\":\"IntegrationRuntime\",\"description\":\"lswipobtbtlmpdr\",\"\":{\"swughx\":\"datalrmmmsaujxaogtwx\",\"ifs\":\"datawfaehryordinfwn\"}},\"name\":\"jufuddtubzekfbuf\",\"type\":\"xmelzlsskphwwn\",\"etag\":\"a\",\"id\":\"ibgyquzofyeqru\"}]}"; + = "{\"value\":[{\"properties\":{\"type\":\"IntegrationRuntime\",\"description\":\"akghvaqbkz\",\"\":{\"spnyutfhqighnun\":\"databxau\",\"ovgio\":\"datatjmzbirjndd\"}},\"name\":\"ztrln\",\"type\":\"vjdv\",\"etag\":\"c\",\"id\":\"j\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,10 @@ public void testListByFactory() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.integrationRuntimes().listByFactory("futohkrqbgx", - "jbapflluyh", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.integrationRuntimes().listByFactory("ljcblppnq", "snvcwji", com.azure.core.util.Context.NONE); - Assertions.assertEquals("ibgyquzofyeqru", response.iterator().next().id()); - Assertions.assertEquals("lswipobtbtlmpdr", response.iterator().next().properties().description()); + Assertions.assertEquals("j", response.iterator().next().id()); + Assertions.assertEquals("akghvaqbkz", response.iterator().next().properties().description()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListOutboundNetworkDependenciesEndpointsWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListOutboundNetworkDependenciesEndpointsWithResponseMockTests.java index fcfbe0f583fbd..34856d2c33a04 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListOutboundNetworkDependenciesEndpointsWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesListOutboundNetworkDependenciesEndpointsWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testListOutboundNetworkDependenciesEndpointsWithResponse() throws Ex ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"category\":\"grsvyjrq\",\"endpoints\":[{\"domainName\":\"vmxibpcnmps\",\"endpointDetails\":[{},{},{}]},{\"domainName\":\"entiprriqwf\",\"endpointDetails\":[{},{}]}]},{\"category\":\"yingr\",\"endpoints\":[{\"domainName\":\"cqndgbxtzyt\",\"endpointDetails\":[{},{},{}]}]},{\"category\":\"hntchigub\",\"endpoints\":[{\"domainName\":\"gyazppefsdoo\",\"endpointDetails\":[{}]}]},{\"category\":\"fiey\",\"endpoints\":[{\"domainName\":\"qtwohfhs\",\"endpointDetails\":[{}]},{\"domainName\":\"umevgoaxtw\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"uirskoaxs\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"qliyxzeniiu\",\"endpointDetails\":[{},{},{}]}]}]}"; + = "{\"value\":[{\"category\":\"eoqwngtiyzzifuov\",\"endpoints\":[{\"domainName\":\"gts\",\"endpointDetails\":[{},{},{}]},{\"domainName\":\"rk\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"crxcnuyfvridzqoi\",\"endpointDetails\":[{},{},{}]}]},{\"category\":\"vayboubi\",\"endpoints\":[{\"domainName\":\"qmydpoj\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"xdg\",\"endpointDetails\":[{},{}]},{\"domainName\":\"eadqopw\",\"endpointDetails\":[{}]},{\"domainName\":\"rhdezlhsdcpdbo\",\"endpointDetails\":[{},{},{}]}]},{\"category\":\"yqdvxqo\",\"endpoints\":[{\"domainName\":\"scde\",\"endpointDetails\":[{},{}]},{\"domainName\":\"ftzxtr\",\"endpointDetails\":[{}]},{\"domainName\":\"ljfdc\",\"endpointDetails\":[{}]}]},{\"category\":\"q\",\"endpoints\":[{\"domainName\":\"dywbnerygsifsahk\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"ajnsuuxbyrvgu\",\"endpointDetails\":[{},{}]},{\"domainName\":\"solmzrfhlynkius\",\"endpointDetails\":[{},{},{},{}]},{\"domainName\":\"bjtsqfhnqxqtemvq\",\"endpointDetails\":[{},{},{},{}]}]}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,10 +49,10 @@ public void testListOutboundNetworkDependenciesEndpointsWithResponse() throws Ex new AzureProfile("", "", AzureEnvironment.AZURE)); IntegrationRuntimeOutboundNetworkDependenciesEndpointsResponse response - = manager.integrationRuntimes().listOutboundNetworkDependenciesEndpointsWithResponse("whjfuoips", - "vcqhzejbro", "h", com.azure.core.util.Context.NONE).getValue(); + = manager.integrationRuntimes().listOutboundNetworkDependenciesEndpointsWithResponse("syifdrbkprblwsb", + "sekqqt", "rupogtrwkuwn", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("grsvyjrq", response.value().get(0).category()); - Assertions.assertEquals("vmxibpcnmps", response.value().get(0).endpoints().get(0).domainName()); + Assertions.assertEquals("eoqwngtiyzzifuov", response.value().get(0).category()); + Assertions.assertEquals("gts", response.value().get(0).endpoints().get(0).domainName()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesRemoveLinksWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesRemoveLinksWithResponseMockTests.java index a72dc64eace15..4ce91815969d4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesRemoveLinksWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesRemoveLinksWithResponseMockTests.java @@ -46,8 +46,8 @@ public void testRemoveLinksWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.integrationRuntimes().removeLinksWithResponse("glljcblppnqosnv", "wjiwgakghv", "qbkzzmw", - new LinkedIntegrationRuntimeRequest().withLinkedFactoryName("xaut"), com.azure.core.util.Context.NONE); + manager.integrationRuntimes().removeLinksWithResponse("lfeolhsyskivlz", "xmqvlgcppn", "iynzdadkurwgty", + new LinkedIntegrationRuntimeRequest().withLinkedFactoryName("njox"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStartMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStartMockTests.java index baf4ff5a2f2ec..2a7aaf3d2e9e0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStartMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStartMockTests.java @@ -30,7 +30,7 @@ public void testStart() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"name\":\"pkjyo\",\"properties\":{\"type\":\"IntegrationRuntimeStatus\",\"dataFactoryName\":\"wcxedkkd\",\"state\":\"Offline\",\"\":{\"piqeisjboghjdih\":\"datasrehgyfif\"}}}"; + = "{\"name\":\"ato\",\"properties\":{\"type\":\"IntegrationRuntimeStatus\",\"dataFactoryName\":\"yrfspmc\",\"state\":\"AccessDenied\",\"\":{\"tynhulefltub\":\"dataisypkif\"}}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,8 +47,8 @@ public void testStart() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - IntegrationRuntimeStatusResponse response - = manager.integrationRuntimes().start("jubjqjxobmv", "jtzatr", "arneug", com.azure.core.util.Context.NONE); + IntegrationRuntimeStatusResponse response = manager.integrationRuntimes().start("heamxidjdptruie", "rauy", + "hugwau", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStopMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStopMockTests.java index 6e4be2858b894..206518cd65c0e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStopMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesStopMockTests.java @@ -45,7 +45,7 @@ public void testStop() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.integrationRuntimes().stop("c", "ddfvdktbaexbvyu", "rbycuuxgda", com.azure.core.util.Context.NONE); + manager.integrationRuntimes().stop("pebblndlahr", "x", "tkehfoephipho", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesSyncCredentialsWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesSyncCredentialsWithResponseMockTests.java index e196ffc1f768a..846fb49cf2311 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesSyncCredentialsWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesSyncCredentialsWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testSyncCredentialsWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.integrationRuntimes().syncCredentialsWithResponse("flil", "mptvm", "noug", + manager.integrationRuntimes().syncCredentialsWithResponse("gmcuqjouk", "mv", "qismvo", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesUpgradeWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesUpgradeWithResponseMockTests.java index 85fb2405d0da4..0710e4d22b8ca 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesUpgradeWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/IntegrationRuntimesUpgradeWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testUpgradeWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.integrationRuntimes().upgradeWithResponse("yuillrrqw", "phiqje", "eafgo", + manager.integrationRuntimes().upgradeWithResponse("cnwqeixyjlfobj", "betsvnloduvcq", "wc", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraObjectDatasetTests.java index f383c91ab3898..e7f4605bccd77 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraObjectDatasetTests.java @@ -19,30 +19,29 @@ public final class JiraObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { JiraObjectDataset model = BinaryData.fromString( - "{\"type\":\"JiraObject\",\"typeProperties\":{\"tableName\":\"datamhp\"},\"description\":\"sfgvrvq\",\"structure\":\"datawbdrwroqkljnzpqh\",\"schema\":\"datasarkyulfa\",\"linkedServiceName\":{\"referenceName\":\"ea\",\"parameters\":{\"geytlplslfc\":\"dataqenhekzaz\",\"ksuowt\":\"datae\",\"rhnxzmfvmw\":\"datalkyqfnjo\",\"rawwhyxf\":\"datanrtc\"}},\"parameters\":{\"uns\":{\"type\":\"String\",\"defaultValue\":\"datadmvwn\"}},\"annotations\":[\"dataevzshqykebmps\",\"dataaezc\",\"datadkckr\"],\"folder\":{\"name\":\"qdmhcejstfs\"},\"\":{\"wxqd\":\"datajakgk\",\"wdjox\":\"dataoqzh\",\"sobvcnsb\":\"datakbd\"}}") + "{\"type\":\"JiraObject\",\"typeProperties\":{\"tableName\":\"datazc\"},\"description\":\"kckrnovqdmh\",\"structure\":\"datajstfs\",\"schema\":\"datacjakgkqwx\",\"linkedServiceName\":{\"referenceName\":\"dsoqzhxwdjoxwkb\",\"parameters\":{\"lfhn\":\"dataobvcnsbioez\",\"t\":\"dataz\"}},\"parameters\":{\"cmwbejywwwvn\":{\"type\":\"Bool\",\"defaultValue\":\"datagtkxncwdytnlr\"}},\"annotations\":[\"datakrmqevrhhafqf\",\"datadfyziruqvgnjxi\",\"datakgyjmzbm\"],\"folder\":{\"name\":\"kyluyug\"},\"\":{\"loxtvq\":\"datadcv\",\"ryhmmglv\":\"datab\",\"nkpsvokkyankxvc\":\"datab\"}}") .toObject(JiraObjectDataset.class); - Assertions.assertEquals("sfgvrvq", model.description()); - Assertions.assertEquals("ea", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.STRING, model.parameters().get("uns").type()); - Assertions.assertEquals("qdmhcejstfs", model.folder().name()); + Assertions.assertEquals("kckrnovqdmh", model.description()); + Assertions.assertEquals("dsoqzhxwdjoxwkb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("cmwbejywwwvn").type()); + Assertions.assertEquals("kyluyug", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JiraObjectDataset model = new JiraObjectDataset().withDescription("sfgvrvq") - .withStructure("datawbdrwroqkljnzpqh").withSchema("datasarkyulfa") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ea") - .withParameters(mapOf("geytlplslfc", "dataqenhekzaz", "ksuowt", "datae", "rhnxzmfvmw", "datalkyqfnjo", - "rawwhyxf", "datanrtc"))) - .withParameters( - mapOf("uns", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datadmvwn"))) - .withAnnotations(Arrays.asList("dataevzshqykebmps", "dataaezc", "datadkckr")) - .withFolder(new DatasetFolder().withName("qdmhcejstfs")).withTableName("datamhp"); + JiraObjectDataset model = new JiraObjectDataset().withDescription("kckrnovqdmh").withStructure("datajstfs") + .withSchema("datacjakgkqwx") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dsoqzhxwdjoxwkb") + .withParameters(mapOf("lfhn", "dataobvcnsbioez", "t", "dataz"))) + .withParameters(mapOf("cmwbejywwwvn", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datagtkxncwdytnlr"))) + .withAnnotations(Arrays.asList("datakrmqevrhhafqf", "datadfyziruqvgnjxi", "datakgyjmzbm")) + .withFolder(new DatasetFolder().withName("kyluyug")).withTableName("datazc"); model = BinaryData.fromObject(model).toObject(JiraObjectDataset.class); - Assertions.assertEquals("sfgvrvq", model.description()); - Assertions.assertEquals("ea", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.STRING, model.parameters().get("uns").type()); - Assertions.assertEquals("qdmhcejstfs", model.folder().name()); + Assertions.assertEquals("kckrnovqdmh", model.description()); + Assertions.assertEquals("dsoqzhxwdjoxwkb", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("cmwbejywwwvn").type()); + Assertions.assertEquals("kyluyug", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraSourceTests.java index 2d93a181b2e62..46e0572b2f0ba 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JiraSourceTests.java @@ -11,16 +11,15 @@ public final class JiraSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { JiraSource model = BinaryData.fromString( - "{\"type\":\"JiraSource\",\"query\":\"datakwcfkcqexd\",\"queryTimeout\":\"datacvkwwjjotfun\",\"additionalColumns\":\"datauejxvrwalekqedof\",\"sourceRetryCount\":\"databxmlai\",\"sourceRetryWait\":\"datavhlpfjibblm\",\"maxConcurrentConnections\":\"datavzdaycmene\",\"disableMetricsCollection\":\"datayzlslvgqle\",\"\":{\"t\":\"datawbbellcjd\",\"dpmy\":\"datacvddfmflwfxdkp\"}}") + "{\"type\":\"JiraSource\",\"query\":\"dataqgpldrn\",\"queryTimeout\":\"datahdb\",\"additionalColumns\":\"databmsbetzufkvx\",\"sourceRetryCount\":\"databddrtngdc\",\"sourceRetryWait\":\"datajzgzaeuu\",\"maxConcurrentConnections\":\"datavheqzl\",\"disableMetricsCollection\":\"datavaskrgoodfhpyue\",\"\":{\"lizlzxh\":\"datanyddp\",\"sjwawl\":\"datacuglgmfznholaf\",\"yk\":\"dataqmznkcwiok\"}}") .toObject(JiraSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JiraSource model = new JiraSource().withSourceRetryCount("databxmlai").withSourceRetryWait("datavhlpfjibblm") - .withMaxConcurrentConnections("datavzdaycmene").withDisableMetricsCollection("datayzlslvgqle") - .withQueryTimeout("datacvkwwjjotfun").withAdditionalColumns("datauejxvrwalekqedof") - .withQuery("datakwcfkcqexd"); + JiraSource model = new JiraSource().withSourceRetryCount("databddrtngdc").withSourceRetryWait("datajzgzaeuu") + .withMaxConcurrentConnections("datavheqzl").withDisableMetricsCollection("datavaskrgoodfhpyue") + .withQueryTimeout("datahdb").withAdditionalColumns("databmsbetzufkvx").withQuery("dataqgpldrn"); model = BinaryData.fromObject(model).toObject(JiraSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonReadSettingsTests.java index 62255d0daa9e9..5f9c827f7b988 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonReadSettingsTests.java @@ -14,7 +14,7 @@ public final class JsonReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { JsonReadSettings model = BinaryData.fromString( - "{\"type\":\"JsonReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"uevnomzlre\":\"datambbhluvdce\"}},\"\":{\"aasqolxaodbhgxb\":\"dataskiegtaenalep\",\"ihggv\":\"datadborqk\"}}") + "{\"type\":\"JsonReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"klowuth\":\"datagzhhfnaqclep\"}},\"\":{\"fn\":\"datahnmllbljehwhxxu\",\"ah\":\"dataa\",\"aovubfl\":\"datakvnapxhtqwsdd\"}}") .toObject(JsonReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSinkTests.java index cc383d8ba4423..9ccb0878cb969 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSinkTests.java @@ -17,22 +17,21 @@ public final class JsonSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { JsonSink model = BinaryData.fromString( - "{\"type\":\"JsonSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datak\",\"disableMetricsCollection\":\"datarsp\",\"copyBehavior\":\"dataghyekggoawlqvuw\",\"metadata\":[{\"name\":\"dataqsgyrznostngx\",\"value\":\"datapkizjnkgdsursu\"},{\"name\":\"datacirkbkqpsv\",\"value\":\"datashxumuuyblol\"}],\"\":{\"bqxlsam\":\"datavmgb\",\"gvmowyzxqhuhmldh\":\"datayjqhwsojnbb\"}},\"formatSettings\":{\"type\":\"JsonWriteSettings\",\"filePattern\":\"datackfu\",\"\":{\"puaermawwlnsdaz\":\"datahotdztqhqh\",\"hisxz\":\"datacemcotwfuo\"}},\"writeBatchSize\":\"datak\",\"writeBatchTimeout\":\"datafszxbupsxqo\",\"sinkRetryCount\":\"datagxcgqkhyvtajwkrx\",\"sinkRetryWait\":\"datalmwfncwlwov\",\"maxConcurrentConnections\":\"databomjby\",\"disableMetricsCollection\":\"dataprkbzraljwfnc\",\"\":{\"ppqajdm\":\"dataylcpgzmxr\",\"wrziminetb\":\"dataunntqqguhv\",\"kedlclxxq\":\"datafwfuxdtpjcs\"}}") + "{\"type\":\"JsonSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datalnt\",\"disableMetricsCollection\":\"datah\",\"copyBehavior\":\"datatvxghbehhe\",\"metadata\":[{\"name\":\"dataorrvwlc\",\"value\":\"datajlyugzl\"},{\"name\":\"datajirj\",\"value\":\"datarsyfdsgrtke\"}],\"\":{\"gevjman\":\"dataru\",\"vtzdtwxfjlpkoc\":\"datarvvjoklb\",\"uvwlfzjrjgla\":\"dataexfmqfuflu\"}},\"formatSettings\":{\"type\":\"JsonWriteSettings\",\"filePattern\":\"datagzscgslwujk\",\"\":{\"vatujphqv\":\"datapmlrjnnbmodsytq\",\"wpiqkkmpfnw\":\"dataxvvogwghxo\",\"hnrgmg\":\"datarmzwmtsmeaciyp\",\"lsnch\":\"datavcusvidkzbdb\"}},\"writeBatchSize\":\"datarfomlh\",\"writeBatchTimeout\":\"dataiktecs\",\"sinkRetryCount\":\"datacqweydaa\",\"sinkRetryWait\":\"datattmfcx\",\"maxConcurrentConnections\":\"datawfsqjxxbs\",\"disableMetricsCollection\":\"dataqiwl\",\"\":{\"zpr\":\"datatsyjzdasgk\",\"lbddlnzmff\":\"dataqomuzohnpkof\",\"junmgd\":\"datavowlammvazvwzien\"}}") .toObject(JsonSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JsonSink model = new JsonSink().withWriteBatchSize("datak").withWriteBatchTimeout("datafszxbupsxqo") - .withSinkRetryCount("datagxcgqkhyvtajwkrx").withSinkRetryWait("datalmwfncwlwov") - .withMaxConcurrentConnections("databomjby").withDisableMetricsCollection("dataprkbzraljwfnc") - .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("datak") - .withDisableMetricsCollection("datarsp").withCopyBehavior("dataghyekggoawlqvuw") - .withMetadata( - Arrays.asList(new MetadataItem().withName("dataqsgyrznostngx").withValue("datapkizjnkgdsursu"), - new MetadataItem().withName("datacirkbkqpsv").withValue("datashxumuuyblol"))) + JsonSink model = new JsonSink().withWriteBatchSize("datarfomlh").withWriteBatchTimeout("dataiktecs") + .withSinkRetryCount("datacqweydaa").withSinkRetryWait("datattmfcx") + .withMaxConcurrentConnections("datawfsqjxxbs").withDisableMetricsCollection("dataqiwl") + .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("datalnt") + .withDisableMetricsCollection("datah").withCopyBehavior("datatvxghbehhe") + .withMetadata(Arrays.asList(new MetadataItem().withName("dataorrvwlc").withValue("datajlyugzl"), + new MetadataItem().withName("datajirj").withValue("datarsyfdsgrtke"))) .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))) - .withFormatSettings(new JsonWriteSettings().withFilePattern("datackfu")); + .withFormatSettings(new JsonWriteSettings().withFilePattern("datagzscgslwujk")); model = BinaryData.fromObject(model).toObject(JsonSink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSourceTests.java index f368b485c9d2e..e76c1316f3071 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonSourceTests.java @@ -16,20 +16,19 @@ public final class JsonSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { JsonSource model = BinaryData.fromString( - "{\"type\":\"JsonSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datahdyasklmyfh\",\"disableMetricsCollection\":\"dataxwedetawljatvfd\",\"\":{\"xcqcwbxxvcvek\":\"datapduttqjtszq\",\"kdwwq\":\"datajdruml\"}},\"formatSettings\":{\"type\":\"JsonReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"wsvtzotmwx\":\"datamwuyar\"}},\"\":{\"wrtmjskb\":\"databdanfexlawkeqjhz\",\"mvounbyvsfqu\":\"dataenjnady\",\"xqbknoxjhedwh\":\"datar\",\"rpajbiig\":\"datamwb\"}},\"additionalColumns\":\"datarezpuzkwi\",\"sourceRetryCount\":\"datafin\",\"sourceRetryWait\":\"datasdtlpshxjhans\",\"maxConcurrentConnections\":\"dataoalcnkgqs\",\"disableMetricsCollection\":\"datafyoksstalj\",\"\":{\"mnaaqhscaanddlv\":\"dataxjjltu\"}}") + "{\"type\":\"JsonSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"databdsgq\",\"disableMetricsCollection\":\"datacid\",\"\":{\"jnhpyylekubiwvjv\":\"datazzhdjbyfdfu\"}},\"formatSettings\":{\"type\":\"JsonReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"xpwjvfisloq\":\"dataqb\",\"pjbdqmnki\":\"datattkbzwgj\",\"xjttnurkmerqza\":\"datajqsshu\"}},\"\":{\"vqvvtjwdlduvimg\":\"dataom\",\"jvbkjtg\":\"dataceormxoxtapaf\"}},\"additionalColumns\":\"datacptavcipydnuj\",\"sourceRetryCount\":\"datalskizprvpuacajx\",\"sourceRetryWait\":\"datagx\",\"maxConcurrentConnections\":\"dataxpzsl\",\"disableMetricsCollection\":\"datarypz\",\"\":{\"zgszjhekbmd\":\"datachqzkfges\",\"ojsrhgpitye\":\"datach\",\"f\":\"datavwysbme\"}}") .toObject(JsonSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JsonSource model = new JsonSource().withSourceRetryCount("datafin").withSourceRetryWait("datasdtlpshxjhans") - .withMaxConcurrentConnections("dataoalcnkgqs").withDisableMetricsCollection("datafyoksstalj") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datahdyasklmyfh") - .withDisableMetricsCollection("dataxwedetawljatvfd") - .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + JsonSource model = new JsonSource().withSourceRetryCount("datalskizprvpuacajx").withSourceRetryWait("datagx") + .withMaxConcurrentConnections("dataxpzsl").withDisableMetricsCollection("datarypz") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("databdsgq") + .withDisableMetricsCollection("datacid").withAdditionalProperties(mapOf("type", "StoreReadSettings"))) .withFormatSettings(new JsonReadSettings().withCompressionProperties( new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings")))) - .withAdditionalColumns("datarezpuzkwi"); + .withAdditionalColumns("datacptavcipydnuj"); model = BinaryData.fromObject(model).toObject(JsonSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonWriteSettingsTests.java index 9aaf52f381017..76cf6ddbf4812 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/JsonWriteSettingsTests.java @@ -10,14 +10,14 @@ public final class JsonWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - JsonWriteSettings model = BinaryData.fromString( - "{\"type\":\"JsonWriteSettings\",\"filePattern\":\"datatzqrm\",\"\":{\"anrk\":\"datakurkggqx\",\"jfsvfbjcn\":\"datac\"}}") + JsonWriteSettings model = BinaryData + .fromString("{\"type\":\"JsonWriteSettings\",\"filePattern\":\"dataorzozf\",\"\":{\"bnefab\":\"dataw\"}}") .toObject(JsonWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - JsonWriteSettings model = new JsonWriteSettings().withFilePattern("datatzqrm"); + JsonWriteSettings model = new JsonWriteSettings().withFilePattern("dataorzozf"); model = BinaryData.fromObject(model).toObject(JsonWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseReadSettingsTests.java index 7b1cbe592755f..c5b73a70c5ae5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseReadSettingsTests.java @@ -11,18 +11,18 @@ public final class LakeHouseReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LakeHouseReadSettings model = BinaryData.fromString( - "{\"type\":\"LakeHouseReadSettings\",\"recursive\":\"dataorjbyyxkcvah\",\"wildcardFolderPath\":\"datays\",\"wildcardFileName\":\"datajivw\",\"fileListPath\":\"datang\",\"enablePartitionDiscovery\":\"datayct\",\"partitionRootPath\":\"datahgjyholsmahbjc\",\"deleteFilesAfterCompletion\":\"dataskqxgbigozrvlkla\",\"modifiedDatetimeStart\":\"datalysse\",\"modifiedDatetimeEnd\":\"datappgsfj\",\"maxConcurrentConnections\":\"datagmogmcjn\",\"disableMetricsCollection\":\"dataukbwypcvqfz\",\"\":{\"erizf\":\"datax\",\"cdpyoqmwpmrlgj\":\"datawlkovopqp\",\"fptvam\":\"dataqs\"}}") + "{\"type\":\"LakeHouseReadSettings\",\"recursive\":\"datagrosxfdxrc\",\"wildcardFolderPath\":\"datambbhluvdce\",\"wildcardFileName\":\"dataevnom\",\"fileListPath\":\"datareiwdskie\",\"enablePartitionDiscovery\":\"dataaenalepta\",\"partitionRootPath\":\"dataqo\",\"deleteFilesAfterCompletion\":\"dataaodbhgxbadbo\",\"modifiedDatetimeStart\":\"datakmihggv\",\"modifiedDatetimeEnd\":\"dataqwyxbatr\",\"maxConcurrentConnections\":\"dataynlslgxif\",\"disableMetricsCollection\":\"datasclqwk\",\"\":{\"gshaqfu\":\"dataepuvambzf\"}}") .toObject(LakeHouseReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LakeHouseReadSettings model = new LakeHouseReadSettings().withMaxConcurrentConnections("datagmogmcjn") - .withDisableMetricsCollection("dataukbwypcvqfz").withRecursive("dataorjbyyxkcvah") - .withWildcardFolderPath("datays").withWildcardFileName("datajivw").withFileListPath("datang") - .withEnablePartitionDiscovery("datayct").withPartitionRootPath("datahgjyholsmahbjc") - .withDeleteFilesAfterCompletion("dataskqxgbigozrvlkla").withModifiedDatetimeStart("datalysse") - .withModifiedDatetimeEnd("datappgsfj"); + LakeHouseReadSettings model = new LakeHouseReadSettings().withMaxConcurrentConnections("dataynlslgxif") + .withDisableMetricsCollection("datasclqwk").withRecursive("datagrosxfdxrc") + .withWildcardFolderPath("datambbhluvdce").withWildcardFileName("dataevnom") + .withFileListPath("datareiwdskie").withEnablePartitionDiscovery("dataaenalepta") + .withPartitionRootPath("dataqo").withDeleteFilesAfterCompletion("dataaodbhgxbadbo") + .withModifiedDatetimeStart("datakmihggv").withModifiedDatetimeEnd("dataqwyxbatr"); model = BinaryData.fromObject(model).toObject(LakeHouseReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSinkTests.java index ad6f3180b20e7..f979fefbed3ae 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSinkTests.java @@ -11,17 +11,17 @@ public final class LakeHouseTableSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LakeHouseTableSink model = BinaryData.fromString( - "{\"type\":\"LakeHouseTableSink\",\"tableActionOption\":\"datavwm\",\"partitionOption\":\"dataovturdhnnm\",\"partitionNameList\":\"dataizwmptsygqzt\",\"writeBatchSize\":\"datavuei\",\"writeBatchTimeout\":\"dataapgqx\",\"sinkRetryCount\":\"databvwxyumqoqw\",\"sinkRetryWait\":\"datab\",\"maxConcurrentConnections\":\"datayeigngrzvegxmx\",\"disableMetricsCollection\":\"datahqxzewlwwdmp\",\"\":{\"tfjvfct\":\"datapccovzkwh\",\"gbnktgotddyd\":\"datafujdapcmga\",\"wsdyvahn\":\"dataatexkwcolnae\",\"cpmvnzhdsa\":\"datad\"}}") + "{\"type\":\"LakeHouseTableSink\",\"tableActionOption\":\"datanpry\",\"partitionOption\":\"dataujqyeyzoivi\",\"partitionNameList\":\"datanihmwvhc\",\"writeBatchSize\":\"datamua\",\"writeBatchTimeout\":\"datatd\",\"sinkRetryCount\":\"datai\",\"sinkRetryWait\":\"datazytdj\",\"maxConcurrentConnections\":\"dataun\",\"disableMetricsCollection\":\"dataadyrhmpokfxcb\",\"\":{\"lnlwgseouh\":\"dataxgajyrac\",\"zru\":\"dataetxupxeain\",\"zfaja\":\"datageaenkfsxtsmzva\",\"ujcqz\":\"databswwbrllvva\"}}") .toObject(LakeHouseTableSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LakeHouseTableSink model = new LakeHouseTableSink().withWriteBatchSize("datavuei") - .withWriteBatchTimeout("dataapgqx").withSinkRetryCount("databvwxyumqoqw").withSinkRetryWait("datab") - .withMaxConcurrentConnections("datayeigngrzvegxmx").withDisableMetricsCollection("datahqxzewlwwdmp") - .withTableActionOption("datavwm").withPartitionOption("dataovturdhnnm") - .withPartitionNameList("dataizwmptsygqzt"); + LakeHouseTableSink model + = new LakeHouseTableSink().withWriteBatchSize("datamua").withWriteBatchTimeout("datatd") + .withSinkRetryCount("datai").withSinkRetryWait("datazytdj").withMaxConcurrentConnections("dataun") + .withDisableMetricsCollection("dataadyrhmpokfxcb").withTableActionOption("datanpry") + .withPartitionOption("dataujqyeyzoivi").withPartitionNameList("datanihmwvhc"); model = BinaryData.fromObject(model).toObject(LakeHouseTableSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSourceTests.java index af55a774d16fb..dd06a58c5a20e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseTableSourceTests.java @@ -11,16 +11,16 @@ public final class LakeHouseTableSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LakeHouseTableSource model = BinaryData.fromString( - "{\"type\":\"LakeHouseTableSource\",\"timestampAsOf\":\"dataddwqiucpjprtbs\",\"versionAsOf\":\"datatjwze\",\"additionalColumns\":\"datar\",\"sourceRetryCount\":\"datat\",\"sourceRetryWait\":\"dataylvrofhhitjhh\",\"maxConcurrentConnections\":\"datavwrc\",\"disableMetricsCollection\":\"datahllmblls\",\"\":{\"ornuoqpo\":\"datadrimoopfrdfjjrh\",\"kwmqevx\":\"datawarsd\",\"olseoixqpn\":\"dataqyavcx\"}}") + "{\"type\":\"LakeHouseTableSource\",\"timestampAsOf\":\"datayeyqsiniejjb\",\"versionAsOf\":\"datav\",\"additionalColumns\":\"datakwrvtlbb\",\"sourceRetryCount\":\"databdtmrijt\",\"sourceRetryWait\":\"dataf\",\"maxConcurrentConnections\":\"databpvizuuluilgmova\",\"disableMetricsCollection\":\"datat\",\"\":{\"cvaa\":\"datajxgqsb\",\"lrmrtdznvjgovy\":\"datauvbzcqgtzx\",\"rkntfwxkeu\":\"datapppswleptta\"}}") .toObject(LakeHouseTableSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { LakeHouseTableSource model - = new LakeHouseTableSource().withSourceRetryCount("datat").withSourceRetryWait("dataylvrofhhitjhh") - .withMaxConcurrentConnections("datavwrc").withDisableMetricsCollection("datahllmblls") - .withTimestampAsOf("dataddwqiucpjprtbs").withVersionAsOf("datatjwze").withAdditionalColumns("datar"); + = new LakeHouseTableSource().withSourceRetryCount("databdtmrijt").withSourceRetryWait("dataf") + .withMaxConcurrentConnections("databpvizuuluilgmova").withDisableMetricsCollection("datat") + .withTimestampAsOf("datayeyqsiniejjb").withVersionAsOf("datav").withAdditionalColumns("datakwrvtlbb"); model = BinaryData.fromObject(model).toObject(LakeHouseTableSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseWriteSettingsTests.java index 2f101e0d510e0..46ca780f36ea7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LakeHouseWriteSettingsTests.java @@ -13,17 +13,17 @@ public final class LakeHouseWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LakeHouseWriteSettings model = BinaryData.fromString( - "{\"type\":\"LakeHouseWriteSettings\",\"maxConcurrentConnections\":\"dataotclmbk\",\"disableMetricsCollection\":\"datajryt\",\"copyBehavior\":\"datafnoj\",\"metadata\":[{\"name\":\"datanthjqgovviv\",\"value\":\"dataxytrafettwytavp\"},{\"name\":\"datalgyql\",\"value\":\"datalgspy\"},{\"name\":\"dataapnhhvp\",\"value\":\"datakourqviy\"}],\"\":{\"jtcpdtd\":\"dataegwezgfqo\",\"kajmnvbigmn\":\"datalyogzbasjckakik\"}}") + "{\"type\":\"LakeHouseWriteSettings\",\"maxConcurrentConnections\":\"dataxnjorvpcoxdlppu\",\"disableMetricsCollection\":\"datan\",\"copyBehavior\":\"datapclnm\",\"metadata\":[{\"name\":\"datahbefivozrdzrikw\",\"value\":\"datacvvrkxpbjgozoel\"},{\"name\":\"dataerpbct\",\"value\":\"datarvns\"},{\"name\":\"dataacbrywqqeztlf\",\"value\":\"datalgxrsn\"}],\"\":{\"a\":\"dataoo\",\"xngmebvni\":\"datahvsfgywkinkhv\",\"xkcttp\":\"datajhdkvkqjjouh\"}}") .toObject(LakeHouseWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LakeHouseWriteSettings model = new LakeHouseWriteSettings().withMaxConcurrentConnections("dataotclmbk") - .withDisableMetricsCollection("datajryt").withCopyBehavior("datafnoj") - .withMetadata(Arrays.asList(new MetadataItem().withName("datanthjqgovviv").withValue("dataxytrafettwytavp"), - new MetadataItem().withName("datalgyql").withValue("datalgspy"), - new MetadataItem().withName("dataapnhhvp").withValue("datakourqviy"))); + LakeHouseWriteSettings model = new LakeHouseWriteSettings().withMaxConcurrentConnections("dataxnjorvpcoxdlppu") + .withDisableMetricsCollection("datan").withCopyBehavior("datapclnm").withMetadata( + Arrays.asList(new MetadataItem().withName("datahbefivozrdzrikw").withValue("datacvvrkxpbjgozoel"), + new MetadataItem().withName("dataerpbct").withValue("datarvns"), + new MetadataItem().withName("dataacbrywqqeztlf").withValue("datalgxrsn"))); model = BinaryData.fromObject(model).toObject(LakeHouseWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedIntegrationRuntimeTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedIntegrationRuntimeTests.java index 6108bfbd458bb..17f0790c55832 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedIntegrationRuntimeTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedIntegrationRuntimeTests.java @@ -11,7 +11,7 @@ public final class LinkedIntegrationRuntimeTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LinkedIntegrationRuntime model = BinaryData.fromString( - "{\"name\":\"uhw\",\"subscriptionId\":\"unw\",\"dataFactoryName\":\"bphylnuzgz\",\"dataFactoryLocation\":\"xdusebkc\",\"createTime\":\"2021-09-21T00:32:46Z\"}") + "{\"name\":\"ajzdj\",\"subscriptionId\":\"brwrfrmhoufokrbg\",\"dataFactoryName\":\"jcksirs\",\"dataFactoryLocation\":\"unnik\",\"createTime\":\"2021-05-07T10:45:42Z\"}") .toObject(LinkedIntegrationRuntime.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesCreateOrUpdateWithResponseMockTests.java index 43f91cef15fad..6cda44a355638 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesCreateOrUpdateWithResponseMockTests.java @@ -38,7 +38,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"LinkedService\",\"connectVia\":{\"referenceName\":\"sf\",\"parameters\":{\"neqmjodvknxjtttk\":\"datafrtaufrxxvzq\",\"s\":\"datamhquc\"}},\"description\":\"odcxvdqfk\",\"parameters\":{\"bxfrliyikcnlbe\":{\"type\":\"Bool\",\"defaultValue\":\"datafoaabltvlttopl\"},\"diogwckvoxlih\":{\"type\":\"Int\",\"defaultValue\":\"datayoxjqysfej\"},\"nchrouvtbptdeum\":{\"type\":\"Int\",\"defaultValue\":\"datafznzemisqunxwos\"},\"b\":{\"type\":\"Array\",\"defaultValue\":\"datazxqr\"}},\"annotations\":[\"dataeodgpqdcrn\"],\"\":{\"wsnccmunh\":\"datatowohtu\",\"rngney\":\"datawcsgczvui\",\"nelmim\":\"dataxzdayzfuv\",\"gjmznptevafczg\":\"dataccevbprecgeregft\"}},\"name\":\"egdeiynlcdqx\",\"type\":\"nbjkwgk\",\"etag\":\"wkazmw\",\"id\":\"sfejkrcgcfjnozi\"}"; + = "{\"properties\":{\"type\":\"LinkedService\",\"connectVia\":{\"referenceName\":\"kwqavxljaybgxxm\",\"parameters\":{\"bfjhvkttusyx\":\"datagxhrtansjboiyqi\",\"tjjmtkwg\":\"datayvfwyoqjttrivi\"}},\"description\":\"fjv\",\"parameters\":{\"s\":{\"type\":\"Object\",\"defaultValue\":\"datayoesxnmvslhnc\"},\"qhatwxq\":{\"type\":\"SecureString\",\"defaultValue\":\"dataglaxvn\"},\"huudtiecnpka\":{\"type\":\"Float\",\"defaultValue\":\"databirzjhaicyuplm\"},\"c\":{\"type\":\"Array\",\"defaultValue\":\"dataqjtoeaugwosrywp\"}},\"annotations\":[\"datania\"],\"\":{\"xrrjudgnphgsdq\":\"datazdecgiomdcolwq\",\"gnl\":\"datawfmvpsvwwtncvnoq\",\"obfnbdpaoijxqgf\":\"dataicovvd\"}},\"name\":\"trvvhxjfkpu\",\"type\":\"sjayrlk\",\"etag\":\"nircmodwslhzla\",\"id\":\"qgavcwxwkjambfs\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -56,26 +56,23 @@ public void testCreateOrUpdateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); LinkedServiceResource response - = manager - .linkedServices().define( - "kuxphbwmbgwgmyg") - .withExistingFactory("zpxlyabjrz", "ssjfwur") + = manager.linkedServices().define("sqtrtc").withExistingFactory("yhodtugrw", "wxfkgzgveudmidt") .withProperties(new LinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("snkylqd").withParameters( - mapOf("frjewqwdglmfsjp", "datadzz", "lt", "datafdh", "qtzghtjx", "dataywykfuovk"))) - .withDescription("ywoq") - .withParameters(mapOf("vykthx", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataiioa"), - "egqzqdcohsqufsy", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datawjwl"))) - .withAnnotations(Arrays.asList("datanzsjoxuogyak", "dataxjzalhun", "datamngstvn")) + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("domzfwh") + .withParameters(mapOf("zt", "dataox", "icznotggy", "dataa", "udmiutzuriqlksba", + "datanssghafzdzdf", "lqheqzd", "datayxtiqzjrxh"))) + .withDescription("a") + .withParameters(mapOf("kgrxhpx", + new ParameterSpecification().withType(ParameterType.STRING) + .withDefaultValue("datagymdywjzqmbeipxd"))) + .withAnnotations(Arrays.asList("datauair", "dataxquoweamnxzdu")) .withAdditionalProperties(mapOf("type", "LinkedService"))) - .withIfMatch("kv").create(); + .withIfMatch("qqak").create(); - Assertions.assertEquals("sfejkrcgcfjnozi", response.id()); - Assertions.assertEquals("sf", response.properties().connectVia().referenceName()); - Assertions.assertEquals("odcxvdqfk", response.properties().description()); - Assertions.assertEquals(ParameterType.BOOL, response.properties().parameters().get("bxfrliyikcnlbe").type()); + Assertions.assertEquals("qgavcwxwkjambfs", response.id()); + Assertions.assertEquals("kwqavxljaybgxxm", response.properties().connectVia().referenceName()); + Assertions.assertEquals("fjv", response.properties().description()); + Assertions.assertEquals(ParameterType.OBJECT, response.properties().parameters().get("s").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesDeleteWithResponseMockTests.java index ea3fa7540c7a7..1454377934bcb 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.linkedServices().deleteWithResponse("mqugovcddxlrb", "fqrgje", "abqvgohiqy", + manager.linkedServices().deleteWithResponse("pf", "ttcmwqrbtad", "dkbndkofrhuycn", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesGetWithResponseMockTests.java index 975bc70a61caf..5690065fde8ef 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesGetWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"LinkedService\",\"connectVia\":{\"referenceName\":\"ovu\",\"parameters\":{\"jxfhsgpymzrts\":\"datacr\"}},\"description\":\"nrlpsjbnnuqsz\",\"parameters\":{\"jgddbunxufataqsf\":{\"type\":\"Object\",\"defaultValue\":\"datafrqs\"},\"cur\":{\"type\":\"Bool\",\"defaultValue\":\"datadwh\"},\"vv\":{\"type\":\"Object\",\"defaultValue\":\"dataunazl\"},\"advvgndfyelpnlpn\":{\"type\":\"Float\",\"defaultValue\":\"datacnqwisuhareqy\"}},\"annotations\":[\"dataxcj\",\"datayfxsdntukoss\",\"dataflfv\"],\"\":{\"i\":\"dataeclyrtoshk\",\"dnkgpvbvicwfrybv\":\"databjb\"}},\"name\":\"ll\",\"type\":\"ghdfusphokcchynn\",\"etag\":\"nnqii\",\"id\":\"pxxwbe\"}"; + = "{\"properties\":{\"type\":\"LinkedService\",\"connectVia\":{\"referenceName\":\"urfjppmiluik\",\"parameters\":{\"lhnkvipjinjik\":\"dataqkxjcqdnzhjlb\"}},\"description\":\"c\",\"parameters\":{\"r\":{\"type\":\"Bool\",\"defaultValue\":\"datajzxezo\"},\"xyc\":{\"type\":\"Array\",\"defaultValue\":\"datawthslztxixngwe\"},\"nlscfbwkhle\":{\"type\":\"Float\",\"defaultValue\":\"datafldfwqnbco\"}},\"annotations\":[\"databoprgxdcnbzpc\"],\"\":{\"rk\":\"dataumepzekmupdvnan\",\"mhsrwqpcxy\":\"datazlaomteqnt\",\"cgelipoequjkhum\":\"datajei\",\"m\":\"datarxxcbptvvwf\"}},\"name\":\"jhinmhccwmr\",\"type\":\"vlbc\",\"etag\":\"dgydbsr\",\"id\":\"fxoktokmsyohxm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,13 +50,12 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); LinkedServiceResource response = manager.linkedServices() - .getWithResponse("tbwe", "tnevbkkdbhg", "rnxqiwgrj", "lieyyfqhndj", com.azure.core.util.Context.NONE) + .getWithResponse("faagpjslrf", "xlutfbhsenn", "cbxta", "xdkboyqescvcvut", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("pxxwbe", response.id()); - Assertions.assertEquals("ovu", response.properties().connectVia().referenceName()); - Assertions.assertEquals("nrlpsjbnnuqsz", response.properties().description()); - Assertions.assertEquals(ParameterType.OBJECT, - response.properties().parameters().get("jgddbunxufataqsf").type()); + Assertions.assertEquals("fxoktokmsyohxm", response.id()); + Assertions.assertEquals("urfjppmiluik", response.properties().connectVia().referenceName()); + Assertions.assertEquals("c", response.properties().description()); + Assertions.assertEquals(ParameterType.BOOL, response.properties().parameters().get("r").type()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesListByFactoryMockTests.java index c575faf3bb890..b1b6b06b84ef7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LinkedServicesListByFactoryMockTests.java @@ -33,7 +33,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"type\":\"LinkedService\",\"connectVia\":{\"referenceName\":\"nwqeixyj\",\"parameters\":{\"oduvcqowcg\":\"databjibetsvn\",\"skivlzvxmqvlgcp\":\"datafeolhs\",\"wgtypnj\":\"datansiynzdadku\",\"kt\":\"dataxollcsdgmcj\"}},\"description\":\"cmhttiqbn\",\"parameters\":{\"lddvvcwho\":{\"type\":\"Int\",\"defaultValue\":\"dataeavbezzp\"},\"yktlofgpnswv\":{\"type\":\"String\",\"defaultValue\":\"datavzxrfrax\"}},\"annotations\":[\"datakwpgdf\",\"dataoqbekkqs\",\"databyowfrw\",\"datarbzfbds\"],\"\":{\"owtazqexwkkjx\":\"dataflgtq\",\"jxud\":\"datajomnkeaiamh\",\"irttlwuqgaaj\":\"dataclzrot\",\"qsbekmeeowdojpja\":\"datafkdf\"}},\"name\":\"ebtvnskyg\",\"type\":\"qiktsyvzhxzj\",\"etag\":\"zijnykf\",\"id\":\"omdqg\"}]}"; + = "{\"value\":[{\"properties\":{\"type\":\"LinkedService\",\"connectVia\":{\"referenceName\":\"elcbmmrhogxex\",\"parameters\":{\"eeifo\":\"datax\",\"cnfjyggio\":\"datavzmqdnfon\",\"arx\":\"datagv\",\"mjygnixkpadjqjwl\":\"datar\"}},\"description\":\"eibucmfv\",\"parameters\":{\"wuzwydsvgonkomua\":{\"type\":\"Object\",\"defaultValue\":\"datasxnkmaezxldmz\"},\"qvul\":{\"type\":\"SecureString\",\"defaultValue\":\"datakwiytg\"},\"vxfyqsfy\":{\"type\":\"Array\",\"defaultValue\":\"datajdbcypv\"}},\"annotations\":[\"datahbfpzfvqlmzpc\",\"datax\",\"datacslmyrsojqpjba\",\"datafnxdi\"],\"\":{\"c\":\"dataulvmval\",\"fcexbtwic\":\"datahysphdhtcop\",\"e\":\"datahx\",\"kuemotgkyfh\":\"datagkvmmkwa\"}},\"name\":\"mwqkfsvzczisiqns\",\"type\":\"wjfuhq\",\"etag\":\"tdnufvzxosrstev\",\"id\":\"ssaubmdoji\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,12 +51,12 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.linkedServices().listByFactory("reyrgrgft", "ehxddmaevcjtrw", com.azure.core.util.Context.NONE); + = manager.linkedServices().listByFactory("wfujvgvrpear", "ohppupucybtr", com.azure.core.util.Context.NONE); - Assertions.assertEquals("omdqg", response.iterator().next().id()); - Assertions.assertEquals("nwqeixyj", response.iterator().next().properties().connectVia().referenceName()); - Assertions.assertEquals("cmhttiqbn", response.iterator().next().properties().description()); - Assertions.assertEquals(ParameterType.INT, - response.iterator().next().properties().parameters().get("lddvvcwho").type()); + Assertions.assertEquals("ssaubmdoji", response.iterator().next().id()); + Assertions.assertEquals("elcbmmrhogxex", response.iterator().next().properties().connectVia().referenceName()); + Assertions.assertEquals("eibucmfv", response.iterator().next().properties().description()); + Assertions.assertEquals(ParameterType.OBJECT, + response.iterator().next().properties().parameters().get("wuzwydsvgonkomua").type()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogLocationSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogLocationSettingsTests.java index 18111208912ed..e2b8d44592023 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogLocationSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogLocationSettingsTests.java @@ -15,19 +15,19 @@ public final class LogLocationSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LogLocationSettings model = BinaryData.fromString( - "{\"linkedServiceName\":{\"referenceName\":\"bqlr\",\"parameters\":{\"kvljitbnhg\":\"dataswq\",\"abgsdxtwqqukgo\":\"datarvlarozswmucr\"}},\"path\":\"datav\"}") + "{\"linkedServiceName\":{\"referenceName\":\"htwhh\",\"parameters\":{\"jkerdujfnb\":\"dataomfo\"}},\"path\":\"datamroadutogbkdcts\"}") .toObject(LogLocationSettings.class); - Assertions.assertEquals("bqlr", model.linkedServiceName().referenceName()); + Assertions.assertEquals("htwhh", model.linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { LogLocationSettings model = new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("bqlr") - .withParameters(mapOf("kvljitbnhg", "dataswq", "abgsdxtwqqukgo", "datarvlarozswmucr"))) - .withPath("datav"); + .withLinkedServiceName( + new LinkedServiceReference().withReferenceName("htwhh").withParameters(mapOf("jkerdujfnb", "dataomfo"))) + .withPath("datamroadutogbkdcts"); model = BinaryData.fromObject(model).toObject(LogLocationSettings.class); - Assertions.assertEquals("bqlr", model.linkedServiceName().referenceName()); + Assertions.assertEquals("htwhh", model.linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogSettingsTests.java index 937a678c2a8d7..357627226a119 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogSettingsTests.java @@ -17,22 +17,22 @@ public final class LogSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LogSettings model = BinaryData.fromString( - "{\"enableCopyActivityLog\":\"datatslfc\",\"copyActivityLogSettings\":{\"logLevel\":\"datavysfmndr\",\"enableReliableLogging\":\"dataqjkegb\"},\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"ldkciherzkh\",\"parameters\":{\"oiranxq\":\"datahjkwfolpjrepah\",\"nldt\":\"datazss\",\"qxi\":\"dataykzmwdoqrejltr\"}},\"path\":\"dataozryoxmfrxfxyc\"}}") + "{\"enableCopyActivityLog\":\"datagzmmrzw\",\"copyActivityLogSettings\":{\"logLevel\":\"datatkcvolaxnuk\",\"enableReliableLogging\":\"datav\"},\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"u\",\"parameters\":{\"zgvaeqiygbo\":\"datacqoxyxiyhmjwsnwk\",\"odidgudarclajben\":\"datavz\",\"kff\":\"datayuufvoj\"}},\"path\":\"datawaewpils\"}}") .toObject(LogSettings.class); - Assertions.assertEquals("ldkciherzkh", model.logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("u", model.logLocationSettings().linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LogSettings model = new LogSettings().withEnableCopyActivityLog("datatslfc") + LogSettings model = new LogSettings().withEnableCopyActivityLog("datagzmmrzw") .withCopyActivityLogSettings( - new CopyActivityLogSettings().withLogLevel("datavysfmndr").withEnableReliableLogging("dataqjkegb")) + new CopyActivityLogSettings().withLogLevel("datatkcvolaxnuk").withEnableReliableLogging("datav")) .withLogLocationSettings(new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ldkciherzkh").withParameters( - mapOf("oiranxq", "datahjkwfolpjrepah", "nldt", "datazss", "qxi", "dataykzmwdoqrejltr"))) - .withPath("dataozryoxmfrxfxyc")); + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("u").withParameters( + mapOf("zgvaeqiygbo", "datacqoxyxiyhmjwsnwk", "odidgudarclajben", "datavz", "kff", "datayuufvoj"))) + .withPath("datawaewpils")); model = BinaryData.fromObject(model).toObject(LogSettings.class); - Assertions.assertEquals("ldkciherzkh", model.logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("u", model.logLocationSettings().linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogStorageSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogStorageSettingsTests.java index 714fa5954c0ac..5a405aac169b3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogStorageSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LogStorageSettingsTests.java @@ -15,20 +15,21 @@ public final class LogStorageSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LogStorageSettings model = BinaryData.fromString( - "{\"linkedServiceName\":{\"referenceName\":\"gfvzvmtjcxig\",\"parameters\":{\"ivmbu\":\"dataxdbglceet\"}},\"path\":\"datazwyw\",\"logLevel\":\"datafptbdx\",\"enableReliableLogging\":\"datalpjulslfiuzytxe\",\"\":{\"cid\":\"datagmqntutetdt\",\"paqnfyhgrcmc\":\"datarjwiwou\",\"ubtwelutrvdz\":\"datappledxyect\",\"pscw\":\"datawp\"}}") + "{\"linkedServiceName\":{\"referenceName\":\"gdlgs\",\"parameters\":{\"stwcyigrhfevxy\":\"databoysquygokh\",\"rtcsucot\":\"dataqukcojyxhhvoo\",\"jnxzvjnmpvsblud\":\"datawyiq\"}},\"path\":\"datahzukrpfbhihd\",\"logLevel\":\"dataiuexyqyfkudmlqpw\",\"enableReliableLogging\":\"datadpwrm\",\"\":{\"k\":\"datais\",\"wsxnyockpcssus\":\"datasp\"}}") .toObject(LogStorageSettings.class); - Assertions.assertEquals("gfvzvmtjcxig", model.linkedServiceName().referenceName()); + Assertions.assertEquals("gdlgs", model.linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { LogStorageSettings model = new LogStorageSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("gfvzvmtjcxig") - .withParameters(mapOf("ivmbu", "dataxdbglceet"))) - .withPath("datazwyw").withLogLevel("datafptbdx").withEnableReliableLogging("datalpjulslfiuzytxe") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("gdlgs") + .withParameters(mapOf("stwcyigrhfevxy", "databoysquygokh", "rtcsucot", "dataqukcojyxhhvoo", + "jnxzvjnmpvsblud", "datawyiq"))) + .withPath("datahzukrpfbhihd").withLogLevel("dataiuexyqyfkudmlqpw").withEnableReliableLogging("datadpwrm") .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(LogStorageSettings.class); - Assertions.assertEquals("gfvzvmtjcxig", model.linkedServiceName().referenceName()); + Assertions.assertEquals("gdlgs", model.linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTests.java index 0f5148ffc9f86..9abf2073843d2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTests.java @@ -24,64 +24,64 @@ public final class LookupActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LookupActivity model = BinaryData.fromString( - "{\"type\":\"Lookup\",\"typeProperties\":{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datawpwdfjsjkond\",\"sourceRetryWait\":\"datancfoqdspslc\",\"maxConcurrentConnections\":\"dataqwrsfdpikxsgga\",\"disableMetricsCollection\":\"datarpp\",\"\":{\"ljtophcwzdwv\":\"dataig\"}},\"dataset\":{\"referenceName\":\"jzokvycinmywjcf\",\"parameters\":{\"xzxbiwnqewqtbzto\":\"dataoztwmvprnqjxs\",\"esgal\":\"dataihpylfdr\",\"kzeolvnosblczct\":\"dataparbjsvqybvge\",\"tcijuntm\":\"dataacbnhkpdcvjhyk\"}},\"firstRowOnly\":\"datahzccqhtlqrf\"},\"linkedServiceName\":{\"referenceName\":\"fxrgxxymtcwac\",\"parameters\":{\"dlqfxidastu\":\"dataadybh\",\"haoviwuttlmfcn\":\"datahnsaespzwgpjri\",\"pihqmmmbokdqkf\":\"datatgai\",\"uwrvg\":\"datapfzxniv\"}},\"policy\":{\"timeout\":\"datarlkgpipwtrt\",\"retry\":\"datafipbddhfk\",\"retryIntervalInSeconds\":1624947960,\"secureInput\":true,\"secureOutput\":true,\"\":{\"hvaovoqonqjl\":\"datawzkefzdu\",\"yqiytrhhmld\":\"datac\"}},\"name\":\"tyz\",\"description\":\"mstflkfgzovbb\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"mcprg\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Succeeded\"],\"\":{\"hwbqi\":\"dataqwvragv\",\"gi\":\"datafrrv\",\"kgekoux\":\"datalmmfb\"}},{\"activity\":\"r\",\"dependencyConditions\":[\"Completed\"],\"\":{\"gcuppwsgawq\":\"datamdffoibx\",\"ox\":\"datarenjzlqbteftctpz\",\"nouh\":\"dataga\",\"ypara\":\"datavtefevhedfzxs\"}}],\"userProperties\":[{\"name\":\"sfnjokrf\",\"value\":\"dataiqgqvprl\"},{\"name\":\"sglqiuqsqzu\",\"value\":\"dataxdxnmuosoziqcui\"},{\"name\":\"kuyaxpuk\",\"value\":\"datatgeejxwbredxmd\"},{\"name\":\"fxlkwy\",\"value\":\"dataoaejylqgenbe\"}],\"\":{\"satzvtinror\":\"dataiatyoxvb\",\"ng\":\"datajtylseudpyhv\",\"lokxihfe\":\"datajn\"}}") + "{\"type\":\"Lookup\",\"typeProperties\":{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datahxqpmzznmn\",\"sourceRetryWait\":\"dataswa\",\"maxConcurrentConnections\":\"databwewwdfeiehwmaxl\",\"disableMetricsCollection\":\"dataag\",\"\":{\"dowsj\":\"databeneqapll\"}},\"dataset\":{\"referenceName\":\"vpvtyullivcymnpb\",\"parameters\":{\"hcatp\":\"datal\",\"qnajmwpeaoeggi\":\"dataq\",\"ugru\":\"datalpglhlwu\"}},\"firstRowOnly\":\"dataprnbozvixamhi\"},\"linkedServiceName\":{\"referenceName\":\"rp\",\"parameters\":{\"s\":\"datakeeoc\",\"kaaozpc\":\"dataqzpkodbquvf\",\"kzgwf\":\"dataogwfqtqbnakmgydf\",\"kankjkszudx\":\"datageqzkpergzscr\"}},\"policy\":{\"timeout\":\"datavxvoqbruyma\",\"retry\":\"datanfofxirj\",\"retryIntervalInSeconds\":1351725857,\"secureInput\":true,\"secureOutput\":true,\"\":{\"vugb\":\"datastuhlwzcn\",\"puyrps\":\"datamnyfhkxcplhqzpw\"}},\"name\":\"ueacnfgt\",\"description\":\"jqfupoamc\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"rec\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Completed\"],\"\":{\"lhiqodxsscirgq\":\"dataynrsacdcfwuta\",\"yhfnjif\":\"datanfdehhk\",\"cxwjwsrdzmbz\":\"dataucojkikgbhkvhldn\"}},{\"activity\":\"fzydwexoyfseehvm\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Failed\",\"Completed\"],\"\":{\"badbbw\":\"datafwkhipaodo\",\"jxesmbuhkcshyhga\":\"dataaajgokpnbmhskh\",\"npftwgt\":\"datamtevifeoijep\",\"njpwkwxnmqmytv\":\"datacccyiuehsne\"}}],\"userProperties\":[{\"name\":\"utyfnmwmghaeedq\",\"value\":\"datakhc\"},{\"name\":\"wjn\",\"value\":\"datanlbsvtsjv\"},{\"name\":\"vzafpvwrbqbyxuup\",\"value\":\"datakbbemhwtmeqt\"},{\"name\":\"fsjpvjwbxlgpepx\",\"value\":\"datajjnxdg\"}],\"\":{\"pmm\":\"datakltlpbb\",\"tdzgngnuuz\":\"datahmvadasuevu\"}}") .toObject(LookupActivity.class); - Assertions.assertEquals("tyz", model.name()); - Assertions.assertEquals("mstflkfgzovbb", model.description()); + Assertions.assertEquals("ueacnfgt", model.name()); + Assertions.assertEquals("jqfupoamc", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("mcprg", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("sfnjokrf", model.userProperties().get(0).name()); - Assertions.assertEquals("fxrgxxymtcwac", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1624947960, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("rec", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("utyfnmwmghaeedq", model.userProperties().get(0).name()); + Assertions.assertEquals("rp", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1351725857, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("jzokvycinmywjcf", model.dataset().referenceName()); + Assertions.assertEquals("vpvtyullivcymnpb", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LookupActivity model = new LookupActivity().withName("tyz").withDescription("mstflkfgzovbb") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + LookupActivity model = new LookupActivity().withName("ueacnfgt").withDescription("jqfupoamc") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("mcprg") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("rec") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("r") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + new ActivityDependency().withActivity("fzydwexoyfseehvm") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("sfnjokrf").withValue("dataiqgqvprl"), - new UserProperty().withName("sglqiuqsqzu").withValue("dataxdxnmuosoziqcui"), - new UserProperty().withName("kuyaxpuk").withValue("datatgeejxwbredxmd"), - new UserProperty().withName("fxlkwy").withValue("dataoaejylqgenbe"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fxrgxxymtcwac") - .withParameters(mapOf("dlqfxidastu", "dataadybh", "haoviwuttlmfcn", "datahnsaespzwgpjri", - "pihqmmmbokdqkf", "datatgai", "uwrvg", "datapfzxniv"))) - .withPolicy(new ActivityPolicy().withTimeout("datarlkgpipwtrt").withRetry("datafipbddhfk") - .withRetryIntervalInSeconds(1624947960).withSecureInput(true).withSecureOutput(true) + .withUserProperties(Arrays.asList(new UserProperty().withName("utyfnmwmghaeedq").withValue("datakhc"), + new UserProperty().withName("wjn").withValue("datanlbsvtsjv"), + new UserProperty().withName("vzafpvwrbqbyxuup").withValue("datakbbemhwtmeqt"), + new UserProperty().withName("fsjpvjwbxlgpepx").withValue("datajjnxdg"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("rp") + .withParameters(mapOf("s", "datakeeoc", "kaaozpc", "dataqzpkodbquvf", "kzgwf", "dataogwfqtqbnakmgydf", + "kankjkszudx", "datageqzkpergzscr"))) + .withPolicy(new ActivityPolicy().withTimeout("datavxvoqbruyma").withRetry("datanfofxirj") + .withRetryIntervalInSeconds(1351725857).withSecureInput(true).withSecureOutput(true) .withAdditionalProperties(mapOf())) - .withSource(new CopySource().withSourceRetryCount("datawpwdfjsjkond").withSourceRetryWait("datancfoqdspslc") - .withMaxConcurrentConnections("dataqwrsfdpikxsgga").withDisableMetricsCollection("datarpp") + .withSource(new CopySource().withSourceRetryCount("datahxqpmzznmn").withSourceRetryWait("dataswa") + .withMaxConcurrentConnections("databwewwdfeiehwmaxl").withDisableMetricsCollection("dataag") .withAdditionalProperties(mapOf("type", "CopySource"))) - .withDataset(new DatasetReference().withReferenceName("jzokvycinmywjcf") - .withParameters(mapOf("xzxbiwnqewqtbzto", "dataoztwmvprnqjxs", "esgal", "dataihpylfdr", - "kzeolvnosblczct", "dataparbjsvqybvge", "tcijuntm", "dataacbnhkpdcvjhyk"))) - .withFirstRowOnly("datahzccqhtlqrf"); + .withDataset(new DatasetReference().withReferenceName("vpvtyullivcymnpb") + .withParameters(mapOf("hcatp", "datal", "qnajmwpeaoeggi", "dataq", "ugru", "datalpglhlwu"))) + .withFirstRowOnly("dataprnbozvixamhi"); model = BinaryData.fromObject(model).toObject(LookupActivity.class); - Assertions.assertEquals("tyz", model.name()); - Assertions.assertEquals("mstflkfgzovbb", model.description()); + Assertions.assertEquals("ueacnfgt", model.name()); + Assertions.assertEquals("jqfupoamc", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("mcprg", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("sfnjokrf", model.userProperties().get(0).name()); - Assertions.assertEquals("fxrgxxymtcwac", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1624947960, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); + Assertions.assertEquals("rec", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("utyfnmwmghaeedq", model.userProperties().get(0).name()); + Assertions.assertEquals("rp", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1351725857, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals("jzokvycinmywjcf", model.dataset().referenceName()); + Assertions.assertEquals("vpvtyullivcymnpb", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTypePropertiesTests.java index 6f96b3975d5b2..f8c98c9b39cd2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/LookupActivityTypePropertiesTests.java @@ -16,22 +16,22 @@ public final class LookupActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LookupActivityTypeProperties model = BinaryData.fromString( - "{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"databljlrfwq\",\"sourceRetryWait\":\"datasxi\",\"maxConcurrentConnections\":\"datarfvbicd\",\"disableMetricsCollection\":\"dataypfobzxbf\",\"\":{\"qfachfmvqn\":\"datanqllmqeauizkm\",\"pwpqvg\":\"datagstqeuujvsc\",\"efhburxnagvcsm\":\"datapwm\",\"d\":\"datavwpxvptqnq\"}},\"dataset\":{\"referenceName\":\"w\",\"parameters\":{\"juuwmcugveiiegoo\":\"datazlfhn\",\"dfqthohfqbeai\":\"datablvcalb\",\"gdascmf\":\"datavnnhxgiydk\",\"xgy\":\"datakabwpdvedmxckb\"}},\"firstRowOnly\":\"datasz\"}") + "{\"source\":{\"type\":\"CopySource\",\"sourceRetryCount\":\"datagfojdb\",\"sourceRetryWait\":\"datasmnelqlqnwv\",\"maxConcurrentConnections\":\"dataxzdimnfnrpq\",\"disableMetricsCollection\":\"datazgncyksblre\",\"\":{\"xdemcyrblw\":\"dataakmibiylkfne\"}},\"dataset\":{\"referenceName\":\"hzyrugstbzpozqlu\",\"parameters\":{\"xol\":\"datagkttlp\",\"lqdotqe\":\"datajevww\"}},\"firstRowOnly\":\"dataenteucaojj\"}") .toObject(LookupActivityTypeProperties.class); - Assertions.assertEquals("w", model.dataset().referenceName()); + Assertions.assertEquals("hzyrugstbzpozqlu", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { LookupActivityTypeProperties model = new LookupActivityTypeProperties() - .withSource(new CopySource().withSourceRetryCount("databljlrfwq").withSourceRetryWait("datasxi") - .withMaxConcurrentConnections("datarfvbicd").withDisableMetricsCollection("dataypfobzxbf") + .withSource(new CopySource().withSourceRetryCount("datagfojdb").withSourceRetryWait("datasmnelqlqnwv") + .withMaxConcurrentConnections("dataxzdimnfnrpq").withDisableMetricsCollection("datazgncyksblre") .withAdditionalProperties(mapOf("type", "CopySource"))) - .withDataset(new DatasetReference().withReferenceName("w").withParameters(mapOf("juuwmcugveiiegoo", - "datazlfhn", "dfqthohfqbeai", "datablvcalb", "gdascmf", "datavnnhxgiydk", "xgy", "datakabwpdvedmxckb"))) - .withFirstRowOnly("datasz"); + .withDataset(new DatasetReference().withReferenceName("hzyrugstbzpozqlu") + .withParameters(mapOf("xol", "datagkttlp", "lqdotqe", "datajevww"))) + .withFirstRowOnly("dataenteucaojj"); model = BinaryData.fromObject(model).toObject(LookupActivityTypeProperties.class); - Assertions.assertEquals("w", model.dataset().referenceName()); + Assertions.assertEquals("hzyrugstbzpozqlu", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoObjectDatasetTests.java index 406cca8713b5d..0cd274a5bb337 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoObjectDatasetTests.java @@ -19,31 +19,32 @@ public final class MagentoObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MagentoObjectDataset model = BinaryData.fromString( - "{\"type\":\"MagentoObject\",\"typeProperties\":{\"tableName\":\"dataznlf\"},\"description\":\"fzx\",\"structure\":\"dataz\",\"schema\":\"dataugtkxncwdytnlr\",\"linkedServiceName\":{\"referenceName\":\"cmwbejywwwvn\",\"parameters\":{\"dfyziruqvgnjxi\":\"datakrmqevrhhafqf\",\"gikyluyu\":\"datakgyjmzbm\",\"c\":\"datambrdcvoloxtv\"}},\"parameters\":{\"vokkyankxvcpt\":{\"type\":\"Object\",\"defaultValue\":\"datammglvnbenkp\"},\"rdxpcpautfzptr\":{\"type\":\"Int\",\"defaultValue\":\"databhnkxasomafegazh\"}},\"annotations\":[\"dataytrtffvpkdx\",\"datayuwenbq\"],\"folder\":{\"name\":\"awvoqatdjkal\"},\"\":{\"smxfzynfemqy\":\"datae\",\"wgssdquupirnb\":\"datakkp\",\"irzyudrq\":\"datalqyvdsqxkjwdzp\"}}") + "{\"type\":\"MagentoObject\",\"typeProperties\":{\"tableName\":\"datagfb\"},\"description\":\"kxasomafe\",\"structure\":\"dataz\",\"schema\":\"datardxpcpautfzptr\",\"linkedServiceName\":{\"referenceName\":\"dzytrtffvp\",\"parameters\":{\"oqatdjka\":\"datacyuwenbqvpraw\"}},\"parameters\":{\"xfzynfemq\":{\"type\":\"SecureString\",\"defaultValue\":\"datacs\"},\"uup\":{\"type\":\"SecureString\",\"defaultValue\":\"datakpgwgssd\"},\"pvirzyud\":{\"type\":\"Int\",\"defaultValue\":\"databnlqyvdsqxkjwd\"}},\"annotations\":[\"dataxrxhxmlfouqp\"],\"folder\":{\"name\":\"andbp\"},\"\":{\"jxcqcaczzvwaeztt\":\"datah\",\"qlyyslg\":\"datajqyfy\",\"bdsvkllrzhshhkb\":\"dataf\",\"rgfwhfzh\":\"datahcazkgdjth\"}}") .toObject(MagentoObjectDataset.class); - Assertions.assertEquals("fzx", model.description()); - Assertions.assertEquals("cmwbejywwwvn", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("vokkyankxvcpt").type()); - Assertions.assertEquals("awvoqatdjkal", model.folder().name()); + Assertions.assertEquals("kxasomafe", model.description()); + Assertions.assertEquals("dzytrtffvp", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("xfzynfemq").type()); + Assertions.assertEquals("andbp", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MagentoObjectDataset model - = new MagentoObjectDataset().withDescription("fzx").withStructure("dataz").withSchema("dataugtkxncwdytnlr") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cmwbejywwwvn").withParameters( - mapOf("dfyziruqvgnjxi", "datakrmqevrhhafqf", "gikyluyu", "datakgyjmzbm", "c", "datambrdcvoloxtv"))) - .withParameters(mapOf("vokkyankxvcpt", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datammglvnbenkp"), - "rdxpcpautfzptr", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("databhnkxasomafegazh"))) - .withAnnotations(Arrays.asList("dataytrtffvpkdx", "datayuwenbq")) - .withFolder(new DatasetFolder().withName("awvoqatdjkal")).withTableName("dataznlf"); + MagentoObjectDataset model = new MagentoObjectDataset().withDescription("kxasomafe").withStructure("dataz") + .withSchema("datardxpcpautfzptr") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dzytrtffvp") + .withParameters(mapOf("oqatdjka", "datacyuwenbqvpraw"))) + .withParameters(mapOf("xfzynfemq", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datacs"), "uup", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datakpgwgssd"), + "pvirzyud", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("databnlqyvdsqxkjwd"))) + .withAnnotations(Arrays.asList("dataxrxhxmlfouqp")).withFolder(new DatasetFolder().withName("andbp")) + .withTableName("datagfb"); model = BinaryData.fromObject(model).toObject(MagentoObjectDataset.class); - Assertions.assertEquals("fzx", model.description()); - Assertions.assertEquals("cmwbejywwwvn", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("vokkyankxvcpt").type()); - Assertions.assertEquals("awvoqatdjkal", model.folder().name()); + Assertions.assertEquals("kxasomafe", model.description()); + Assertions.assertEquals("dzytrtffvp", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("xfzynfemq").type()); + Assertions.assertEquals("andbp", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoSourceTests.java index 8931a4266ecc4..d31b98c31a73a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MagentoSourceTests.java @@ -11,15 +11,16 @@ public final class MagentoSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MagentoSource model = BinaryData.fromString( - "{\"type\":\"MagentoSource\",\"query\":\"datacugchtwxifudlrxb\",\"queryTimeout\":\"dataftpvgmqzitc\",\"additionalColumns\":\"datamlltas\",\"sourceRetryCount\":\"dataqsf\",\"sourceRetryWait\":\"datae\",\"maxConcurrentConnections\":\"dataveg\",\"disableMetricsCollection\":\"databmyvgmbirvv\",\"\":{\"tu\":\"dataqxisav\",\"ridagwuvcdymoq\":\"datawzvlhi\",\"h\":\"datacjkrynziud\"}}") + "{\"type\":\"MagentoSource\",\"query\":\"dataxmobnehbbchtcoel\",\"queryTimeout\":\"datafnpxumgnjmsk\",\"additionalColumns\":\"dataeuogjiowande\",\"sourceRetryCount\":\"dataebpalz\",\"sourceRetryWait\":\"dataptg\",\"maxConcurrentConnections\":\"datarz\",\"disableMetricsCollection\":\"datacfdsvmptnrz\",\"\":{\"ovqpnxpufvggv\":\"datacncdazwtlgora\"}}") .toObject(MagentoSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MagentoSource model = new MagentoSource().withSourceRetryCount("dataqsf").withSourceRetryWait("datae") - .withMaxConcurrentConnections("dataveg").withDisableMetricsCollection("databmyvgmbirvv") - .withQueryTimeout("dataftpvgmqzitc").withAdditionalColumns("datamlltas").withQuery("datacugchtwxifudlrxb"); + MagentoSource model = new MagentoSource().withSourceRetryCount("dataebpalz").withSourceRetryWait("dataptg") + .withMaxConcurrentConnections("datarz").withDisableMetricsCollection("datacfdsvmptnrz") + .withQueryTimeout("datafnpxumgnjmsk").withAdditionalColumns("dataeuogjiowande") + .withQuery("dataxmobnehbbchtcoel"); model = BinaryData.fromObject(model).toObject(MagentoSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsCreateOrUpdateWithResponseMockTests.java index f6e6529b6ca8f..f74039c6e37b7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsCreateOrUpdateWithResponseMockTests.java @@ -36,7 +36,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"connectionState\":{\"actionsRequired\":\"w\",\"description\":\"irnfnlyvdryx\",\"status\":\"qwta\"},\"fqdns\":[\"cawxslstekb\"],\"groupId\":\"q\",\"isReserved\":true,\"privateLinkResourceId\":\"xycvoexbxr\",\"provisioningState\":\"vxwlfmbb\",\"\":{\"zfwyeg\":\"datagmnelo\",\"mdlfkjjucpt\":\"datautfk\",\"msybvjfnuyoy\":\"datakesdfujfpnwfzabl\"}},\"name\":\"a\",\"type\":\"nnlasf\",\"etag\":\"jyvu\",\"id\":\"exlpmbtmc\"}"; + = "{\"properties\":{\"connectionState\":{\"actionsRequired\":\"bhztqiaydmblpdjt\",\"description\":\"ilhcca\",\"status\":\"iifvindcakansjrz\"},\"fqdns\":[\"csly\",\"hpwtkcebi\",\"ngupphvorxocjsa\"],\"groupId\":\"uvvpdjo\",\"isReserved\":true,\"privateLinkResourceId\":\"vvgsczvzdudfikdu\",\"provisioningState\":\"khma\",\"\":{\"ftz\":\"datacolf\"}},\"name\":\"qwfnlpjivtzshuz\",\"type\":\"nivrr\",\"etag\":\"ijkvopsamtx\",\"id\":\"elwno\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -53,18 +53,18 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - ManagedPrivateEndpointResource response = manager.managedPrivateEndpoints().define("e") - .withExistingManagedVirtualNetwork("duyastybomiyj", "jsseemhdf", "lai") + ManagedPrivateEndpointResource response = manager.managedPrivateEndpoints().define("xmlb") + .withExistingManagedVirtualNetwork("erhhzjhmxyns", "adgv", "w") .withProperties(new ManagedPrivateEndpoint().withConnectionState(new ConnectionStateProperties()) - .withFqdns(Arrays.asList("elxd", "zdfst")).withGroupId("uzoglvtzrjlej") - .withPrivateLinkResourceId("zrqkgibpeh") - .withAdditionalProperties(mapOf("isReserved", false, "provisioningState", "ctzcm"))) - .withIfMatch("mgparbirgw").create(); + .withFqdns(Arrays.asList("wditccuzjlcm", "ghaolfupxhrl", "tknmp", "lrcrxxkvuzpsoujc")) + .withGroupId("ubpjwwvies").withPrivateLinkResourceId("zhi") + .withAdditionalProperties(mapOf("isReserved", false, "provisioningState", "lmymncuhqetmpq"))) + .withIfMatch("hdhf").create(); - Assertions.assertEquals("exlpmbtmc", response.id()); - Assertions.assertEquals("cawxslstekb", response.properties().fqdns().get(0)); - Assertions.assertEquals("q", response.properties().groupId()); - Assertions.assertEquals("xycvoexbxr", response.properties().privateLinkResourceId()); + Assertions.assertEquals("elwno", response.id()); + Assertions.assertEquals("csly", response.properties().fqdns().get(0)); + Assertions.assertEquals("uvvpdjo", response.properties().groupId()); + Assertions.assertEquals("vvgsczvzdudfikdu", response.properties().privateLinkResourceId()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsDeleteWithResponseMockTests.java index 7172aede547ca..80ce8728b5171 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.managedPrivateEndpoints().deleteWithResponse("zxxpwex", "ktgpcccgo", "knjjskzuh", "iyavfeyyb", + manager.managedPrivateEndpoints().deleteWithResponse("l", "rctrpun", "bhoety", "wx", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsGetWithResponseMockTests.java index bf34db771a9e3..34634b2d95f6c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"connectionState\":{\"actionsRequired\":\"hmzcnpsdpfwjc\",\"description\":\"bunfymbwinu\",\"status\":\"jtbmjokttqg\"},\"fqdns\":[\"ajuylkfl\"],\"groupId\":\"ofjskndwywbptvym\",\"isReserved\":true,\"privateLinkResourceId\":\"cddbeozh\",\"provisioningState\":\"lxxbmyzfayj\",\"\":{\"xtib\":\"datavowvqpncif\",\"fd\":\"dataqrhz\",\"mat\":\"datacstucswhmnsdwn\"}},\"name\":\"oerjmhtxipwvwz\",\"type\":\"gtgv\",\"etag\":\"zmkteuzeuxxtsl\",\"id\":\"cwl\"}"; + = "{\"properties\":{\"connectionState\":{\"actionsRequired\":\"dvtlygwxilbazru\",\"description\":\"js\",\"status\":\"rowbfslylqzwql\"},\"fqdns\":[\"sjzrifg\"],\"groupId\":\"pnoiwlernc\",\"isReserved\":false,\"privateLinkResourceId\":\"sonkkuaamojzrn\",\"provisioningState\":\"keu\",\"\":{\"cg\":\"datatmssn\"}},\"name\":\"mqgyaknlxwxnn\",\"type\":\"o\",\"etag\":\"ne\",\"id\":\"y\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,12 +48,13 @@ public void testGetWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - ManagedPrivateEndpointResource response = manager.managedPrivateEndpoints().getWithResponse("wugpnses", - "wkhkcdtofakm", "pqfzvvtifcqs", "emewfutovb", "nrfuc", com.azure.core.util.Context.NONE).getValue(); + ManagedPrivateEndpointResource response = manager.managedPrivateEndpoints() + .getWithResponse("gjp", "zibgitkowflc", "xqwy", "vuaiqqgay", "y", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("cwl", response.id()); - Assertions.assertEquals("ajuylkfl", response.properties().fqdns().get(0)); - Assertions.assertEquals("ofjskndwywbptvym", response.properties().groupId()); - Assertions.assertEquals("cddbeozh", response.properties().privateLinkResourceId()); + Assertions.assertEquals("y", response.id()); + Assertions.assertEquals("sjzrifg", response.properties().fqdns().get(0)); + Assertions.assertEquals("pnoiwlernc", response.properties().groupId()); + Assertions.assertEquals("sonkkuaamojzrn", response.properties().privateLinkResourceId()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsListByFactoryMockTests.java index 98804f1c8f93a..846ddfe6b4448 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedPrivateEndpointsListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"connectionState\":{\"actionsRequired\":\"gidjiijpdbwknbm\",\"description\":\"gaecujlaecwnn\",\"status\":\"f\"},\"fqdns\":[\"lvkrnsodohpid\",\"jyfiabhepxqtkynl\",\"znorhwdom\",\"ythsl\"],\"groupId\":\"d\",\"isReserved\":false,\"privateLinkResourceId\":\"gbmpitr\",\"provisioningState\":\"hiu\",\"\":{\"qfdofnppwv\":\"datagnolu\",\"ytdan\":\"dataeymkguvrdqnpr\",\"vmvpp\":\"datalq\",\"is\":\"datatsolxnhlrpsign\"}},\"name\":\"obpxfgp\",\"type\":\"zdzswvfwiunjwqx\",\"etag\":\"tfzgdq\",\"id\":\"vbiryxsa\"}]}"; + = "{\"value\":[{\"properties\":{\"connectionState\":{\"actionsRequired\":\"muvgfkdea\",\"description\":\"xdwwraimjkaz\",\"status\":\"idgoya\"},\"fqdns\":[\"sxkym\"],\"groupId\":\"imbesgi\",\"isReserved\":false,\"privateLinkResourceId\":\"jowgdwcpe\",\"provisioningState\":\"zmxu\",\"\":{\"bseinldmfl\":\"datal\",\"txfob\":\"datagjt\",\"xlptockgjvflc\":\"dataqqleylvy\",\"nlsmfqglvflxltng\":\"datahsbxrsqeywmq\"}},\"name\":\"lpgclo\",\"type\":\"zwuppv\",\"etag\":\"vfzcy\",\"id\":\"bryvlhv\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,11 +50,11 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.managedPrivateEndpoints() - .listByFactory("ounsydjcem", "kyj", "pbhcsviox", com.azure.core.util.Context.NONE); + .listByFactory("ajjyournxq", "uzls", "mbsghzund", com.azure.core.util.Context.NONE); - Assertions.assertEquals("vbiryxsa", response.iterator().next().id()); - Assertions.assertEquals("lvkrnsodohpid", response.iterator().next().properties().fqdns().get(0)); - Assertions.assertEquals("d", response.iterator().next().properties().groupId()); - Assertions.assertEquals("gbmpitr", response.iterator().next().properties().privateLinkResourceId()); + Assertions.assertEquals("bryvlhv", response.iterator().next().id()); + Assertions.assertEquals("sxkym", response.iterator().next().properties().fqdns().get(0)); + Assertions.assertEquals("imbesgi", response.iterator().next().properties().groupId()); + Assertions.assertEquals("jowgdwcpe", response.iterator().next().properties().privateLinkResourceId()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksCreateOrUpdateWithResponseMockTests.java index ab36ecf504ae6..b07fab297cb5b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksCreateOrUpdateWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"vNetId\":\"yiivoixefn\",\"alias\":\"fewqnznctnmkits\",\"\":{\"ounugxnz\":\"dataopevqsabo\",\"ggalvorz\":\"dataq\",\"mqnbaerggq\":\"datadysndiexbskw\"}},\"name\":\"haxqvbzz\",\"type\":\"orengkunydgw\",\"etag\":\"oharvjfzzvj\",\"id\":\"cduwuintiib\"}"; + = "{\"properties\":{\"vNetId\":\"ruydiwsfva\",\"alias\":\"zabbfdhissd\",\"\":{\"fmkf\":\"dataecjmf\"}},\"name\":\"yfthsaf\",\"type\":\"mosiinmc\",\"etag\":\"dtuydynugkjzp\",\"id\":\"prfhpcy\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,12 +52,12 @@ public void testCreateOrUpdateWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ManagedVirtualNetworkResource response - = manager.managedVirtualNetworks().define("txubebb").withExistingFactory("degvxz", "o") + = manager.managedVirtualNetworks().define("vpmtpqdfpgsoje").withExistingFactory("aoeghsqplnyp", "ewc") .withProperties( - new ManagedVirtualNetwork().withAdditionalProperties(mapOf("vNetId", "yde", "alias", "yefgnib"))) - .withIfMatch("ovbrf").create(); + new ManagedVirtualNetwork().withAdditionalProperties(mapOf("vNetId", "jnlvcgar", "alias", "qusg"))) + .withIfMatch("bfkelqzcpts").create(); - Assertions.assertEquals("cduwuintiib", response.id()); + Assertions.assertEquals("prfhpcy", response.id()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksGetWithResponseMockTests.java index 59e0be2ae7d1a..e41a463b70845 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"vNetId\":\"pnuththz\",\"alias\":\"kuszqqae\",\"\":{\"rv\":\"datazrgmc\"}},\"name\":\"amxg\",\"type\":\"bypxxe\",\"etag\":\"uoykomm\",\"id\":\"gmitguvxvdugn\"}"; + = "{\"properties\":{\"vNetId\":\"wdaugdgv\",\"alias\":\"fso\",\"\":{\"ns\":\"datajkiajokjuehcryww\",\"dnwafjibaqlwtd\":\"datarcj\"}},\"name\":\"ejyxrrmnoxjwlhu\",\"type\":\"yzvvidokvzqeadk\",\"etag\":\"drcxvolt\",\"id\":\"zolnqkdtvtpwrmm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,8 +49,8 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); ManagedVirtualNetworkResource response = manager.managedVirtualNetworks() - .getWithResponse("vqtruyzb", "kzsaxm", "nsepdwxflmk", "mfid", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("jbg", "ipc", "qyapn", "nbyhdtjynus", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("gmitguvxvdugn", response.id()); + Assertions.assertEquals("zolnqkdtvtpwrmm", response.id()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksListByFactoryMockTests.java index 53cc263318144..22e68bcc6bfd9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ManagedVirtualNetworksListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"vNetId\":\"dmwaamqfeqsb\",\"alias\":\"gncke\",\"\":{\"tbksdpcpxw\":\"dataqaomih\",\"xzaolzkoyniragb\":\"dataiczzoxnlvixndsi\",\"h\":\"dataiukmkmthioae\",\"d\":\"datamlcenty\"}},\"name\":\"vmf\",\"type\":\"qaewu\",\"etag\":\"o\",\"id\":\"qvnkofvzpbd\"}]}"; + = "{\"value\":[{\"properties\":{\"vNetId\":\"hnoiqtvfr\",\"alias\":\"inavbfkzv\",\"\":{\"rpdveyxcdzuld\":\"datad\"}},\"name\":\"xedm\",\"type\":\"z\",\"etag\":\"hvj\",\"id\":\"jxth\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,9 +49,9 @@ public void testListByFactory() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response - = manager.managedVirtualNetworks().listByFactory("dxsyymhuueeo", "vqwm", com.azure.core.util.Context.NONE); + PagedIterable response = manager.managedVirtualNetworks() + .listByFactory("ptpreputusd", "wnkzwyry", com.azure.core.util.Context.NONE); - Assertions.assertEquals("qvnkofvzpbd", response.iterator().next().id()); + Assertions.assertEquals("jxth", response.iterator().next().id()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBSourceTests.java index c4436be75b4c6..497b40587514d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBSourceTests.java @@ -11,16 +11,15 @@ public final class MariaDBSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MariaDBSource model = BinaryData.fromString( - "{\"type\":\"MariaDBSource\",\"query\":\"datahrygw\",\"queryTimeout\":\"datavuiox\",\"additionalColumns\":\"dataztrfot\",\"sourceRetryCount\":\"datafzcvhfnbccffsb\",\"sourceRetryWait\":\"databt\",\"maxConcurrentConnections\":\"dataqlejxoudjc\",\"disableMetricsCollection\":\"dataa\",\"\":{\"wh\":\"datahpzvqzmlfer\",\"fqttfqgdoowgqooi\":\"datannqudexnicqu\"}}") + "{\"type\":\"MariaDBSource\",\"query\":\"datamezfyelf\",\"queryTimeout\":\"databkbhjdkqfj\",\"additionalColumns\":\"datayzj\",\"sourceRetryCount\":\"dataa\",\"sourceRetryWait\":\"datagatynkihb\",\"maxConcurrentConnections\":\"dataxybtowjz\",\"disableMetricsCollection\":\"datapzaenlzjxztg\",\"\":{\"tczzv\":\"dataunvwvaolfg\"}}") .toObject(MariaDBSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MariaDBSource model - = new MariaDBSource().withSourceRetryCount("datafzcvhfnbccffsb").withSourceRetryWait("databt") - .withMaxConcurrentConnections("dataqlejxoudjc").withDisableMetricsCollection("dataa") - .withQueryTimeout("datavuiox").withAdditionalColumns("dataztrfot").withQuery("datahrygw"); + MariaDBSource model = new MariaDBSource().withSourceRetryCount("dataa").withSourceRetryWait("datagatynkihb") + .withMaxConcurrentConnections("dataxybtowjz").withDisableMetricsCollection("datapzaenlzjxztg") + .withQueryTimeout("databkbhjdkqfj").withAdditionalColumns("datayzj").withQuery("datamezfyelf"); model = BinaryData.fromObject(model).toObject(MariaDBSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBTableDatasetTests.java index 14f67bfbc0f74..a09a5bbbce662 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MariaDBTableDatasetTests.java @@ -19,33 +19,32 @@ public final class MariaDBTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MariaDBTableDataset model = BinaryData.fromString( - "{\"type\":\"MariaDBTable\",\"typeProperties\":{\"tableName\":\"datax\"},\"description\":\"mlfouqpskva\",\"structure\":\"databpmr\",\"schema\":\"datayjxcqcaczzvw\",\"linkedServiceName\":{\"referenceName\":\"ezttqjqyfy\",\"parameters\":{\"lrzhshhkbchcazk\":\"datayyslgyfybdsvk\"}},\"parameters\":{\"fyyqjc\":{\"type\":\"Float\",\"defaultValue\":\"dataprgfwhfzhhrurm\"},\"sddcuqddlda\":{\"type\":\"Bool\",\"defaultValue\":\"datazq\"},\"ojesxjhtyzzwqocy\":{\"type\":\"Int\",\"defaultValue\":\"datafztqewq\"},\"trgu\":{\"type\":\"Bool\",\"defaultValue\":\"dataineuaxpmez\"}},\"annotations\":[\"dataeo\",\"dataxfoa\",\"datazdypz\"],\"folder\":{\"name\":\"mndhgwhlbpju\"},\"\":{\"mitnwlyhbujysvd\":\"dataqxa\",\"dbhatmabtpgn\":\"datayy\"}}") + "{\"type\":\"MariaDBTable\",\"typeProperties\":{\"tableName\":\"datarmrfyyqjcni\"},\"description\":\"qrsdd\",\"structure\":\"dataqddldao\",\"schema\":\"datafztqewq\",\"linkedServiceName\":{\"referenceName\":\"ojesxjhtyzzwqocy\",\"parameters\":{\"trgu\":\"dataineuaxpmez\",\"oyxfoafzdypzlx\":\"datalw\",\"jzqx\":\"datamndhgwhlbpju\",\"lyhbujys\":\"datavmitn\"}},\"parameters\":{\"gnqtjtnnrjewih\":{\"type\":\"Bool\",\"defaultValue\":\"dataddbhatmabt\"},\"hmdfspkdn\":{\"type\":\"String\",\"defaultValue\":\"dataa\"},\"tertnzrrwsc\":{\"type\":\"Int\",\"defaultValue\":\"dataz\"}},\"annotations\":[\"datahdwi\",\"datanvtolzj\"],\"folder\":{\"name\":\"ryxsg\"},\"\":{\"wppvihbmwrv\":\"datanklth\",\"ob\":\"datavdrohu\"}}") .toObject(MariaDBTableDataset.class); - Assertions.assertEquals("mlfouqpskva", model.description()); - Assertions.assertEquals("ezttqjqyfy", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("fyyqjc").type()); - Assertions.assertEquals("mndhgwhlbpju", model.folder().name()); + Assertions.assertEquals("qrsdd", model.description()); + Assertions.assertEquals("ojesxjhtyzzwqocy", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("gnqtjtnnrjewih").type()); + Assertions.assertEquals("ryxsg", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MariaDBTableDataset model = new MariaDBTableDataset().withDescription("mlfouqpskva").withStructure("databpmr") - .withSchema("datayjxcqcaczzvw") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ezttqjqyfy") - .withParameters(mapOf("lrzhshhkbchcazk", "datayyslgyfybdsvk"))) - .withParameters(mapOf("fyyqjc", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataprgfwhfzhhrurm"), - "sddcuqddlda", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datazq"), - "ojesxjhtyzzwqocy", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datafztqewq"), "trgu", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataineuaxpmez"))) - .withAnnotations(Arrays.asList("dataeo", "dataxfoa", "datazdypz")) - .withFolder(new DatasetFolder().withName("mndhgwhlbpju")).withTableName("datax"); + MariaDBTableDataset model + = new MariaDBTableDataset().withDescription("qrsdd").withStructure("dataqddldao").withSchema("datafztqewq") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ojesxjhtyzzwqocy") + .withParameters(mapOf("trgu", "dataineuaxpmez", "oyxfoafzdypzlx", "datalw", "jzqx", + "datamndhgwhlbpju", "lyhbujys", "datavmitn"))) + .withParameters(mapOf("gnqtjtnnrjewih", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataddbhatmabt"), + "hmdfspkdn", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataa"), + "tertnzrrwsc", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataz"))) + .withAnnotations(Arrays.asList("datahdwi", "datanvtolzj")) + .withFolder(new DatasetFolder().withName("ryxsg")).withTableName("datarmrfyyqjcni"); model = BinaryData.fromObject(model).toObject(MariaDBTableDataset.class); - Assertions.assertEquals("mlfouqpskva", model.description()); - Assertions.assertEquals("ezttqjqyfy", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("fyyqjc").type()); - Assertions.assertEquals("mndhgwhlbpju", model.folder().name()); + Assertions.assertEquals("qrsdd", model.description()); + Assertions.assertEquals("ojesxjhtyzzwqocy", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("gnqtjtnnrjewih").type()); + Assertions.assertEquals("ryxsg", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoObjectDatasetTests.java index 69168df4db745..1237ecd4cdb3d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoObjectDatasetTests.java @@ -19,32 +19,29 @@ public final class MarketoObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MarketoObjectDataset model = BinaryData.fromString( - "{\"type\":\"MarketoObject\",\"typeProperties\":{\"tableName\":\"dataomeqg\"},\"description\":\"wisp\",\"structure\":\"datagdblwj\",\"schema\":\"dataaqxaxtuxi\",\"linkedServiceName\":{\"referenceName\":\"ppbiichl\",\"parameters\":{\"zdxywabkitnipapt\":\"datavuixwonkrn\"}},\"parameters\":{\"ewltono\":{\"type\":\"Bool\",\"defaultValue\":\"datayjukkajn\"},\"di\":{\"type\":\"Int\",\"defaultValue\":\"dataemiwfhhawbabhzbf\"},\"zsuspaywvslq\":{\"type\":\"SecureString\",\"defaultValue\":\"dataxydgzfoi\"}},\"annotations\":[\"datanzea\",\"datakxfmu\",\"datadbvytq\"],\"folder\":{\"name\":\"uymkdeuqxlvzpfd\"},\"\":{\"rrmtrxgjmpdvrjz\":\"datagbiwpgopqlktthb\"}}") + "{\"type\":\"MarketoObject\",\"typeProperties\":{\"tableName\":\"dataewltono\"},\"description\":\"femiwfhhawbabhz\",\"structure\":\"datacdikqnxydgzfoiqz\",\"schema\":\"dataspa\",\"linkedServiceName\":{\"referenceName\":\"w\",\"parameters\":{\"eafkxfmuwdbvyt\":\"dataqeron\",\"u\":\"dataavouymkd\"}},\"parameters\":{\"tth\":{\"type\":\"Float\",\"defaultValue\":\"datapfdkaxgbiwpgopql\"}},\"annotations\":[\"datarmt\",\"datax\",\"datajmpdvrjzwaw\",\"dataewajccsdjuz\"],\"folder\":{\"name\":\"jtickzovguzpr\"},\"\":{\"qlrzhtocjzfp\":\"datahboigzxko\",\"jwgiitvjcmimbmsw\":\"dataexuvatzwn\"}}") .toObject(MarketoObjectDataset.class); - Assertions.assertEquals("wisp", model.description()); - Assertions.assertEquals("ppbiichl", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("ewltono").type()); - Assertions.assertEquals("uymkdeuqxlvzpfd", model.folder().name()); + Assertions.assertEquals("femiwfhhawbabhz", model.description()); + Assertions.assertEquals("w", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("tth").type()); + Assertions.assertEquals("jtickzovguzpr", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MarketoObjectDataset model = new MarketoObjectDataset().withDescription("wisp").withStructure("datagdblwj") - .withSchema("dataaqxaxtuxi") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ppbiichl") - .withParameters(mapOf("zdxywabkitnipapt", "datavuixwonkrn"))) - .withParameters(mapOf("ewltono", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datayjukkajn"), "di", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataemiwfhhawbabhzbf"), - "zsuspaywvslq", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("dataxydgzfoi"))) - .withAnnotations(Arrays.asList("datanzea", "datakxfmu", "datadbvytq")) - .withFolder(new DatasetFolder().withName("uymkdeuqxlvzpfd")).withTableName("dataomeqg"); + MarketoObjectDataset model = new MarketoObjectDataset().withDescription("femiwfhhawbabhz") + .withStructure("datacdikqnxydgzfoiqz").withSchema("dataspa") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("w") + .withParameters(mapOf("eafkxfmuwdbvyt", "dataqeron", "u", "dataavouymkd"))) + .withParameters(mapOf("tth", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datapfdkaxgbiwpgopql"))) + .withAnnotations(Arrays.asList("datarmt", "datax", "datajmpdvrjzwaw", "dataewajccsdjuz")) + .withFolder(new DatasetFolder().withName("jtickzovguzpr")).withTableName("dataewltono"); model = BinaryData.fromObject(model).toObject(MarketoObjectDataset.class); - Assertions.assertEquals("wisp", model.description()); - Assertions.assertEquals("ppbiichl", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("ewltono").type()); - Assertions.assertEquals("uymkdeuqxlvzpfd", model.folder().name()); + Assertions.assertEquals("femiwfhhawbabhz", model.description()); + Assertions.assertEquals("w", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("tth").type()); + Assertions.assertEquals("jtickzovguzpr", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoSourceTests.java index ebf8f2c667477..5d4f393344149 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MarketoSourceTests.java @@ -11,16 +11,16 @@ public final class MarketoSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MarketoSource model = BinaryData.fromString( - "{\"type\":\"MarketoSource\",\"query\":\"datafarqxjoazyxmumfb\",\"queryTimeout\":\"dataxzrycvacspzj\",\"additionalColumns\":\"datayphxeoqma\",\"sourceRetryCount\":\"dataikceiyuv\",\"sourceRetryWait\":\"databbawrbqoox\",\"maxConcurrentConnections\":\"datarqlxqhqgip\",\"disableMetricsCollection\":\"datatnkngjnhxufo\",\"\":{\"pfsesi\":\"dataifijdtpedvh\"}}") + "{\"type\":\"MarketoSource\",\"query\":\"dataylvrofhhitjhh\",\"queryTimeout\":\"datavwrc\",\"additionalColumns\":\"datahllmblls\",\"sourceRetryCount\":\"datafdrimoopfr\",\"sourceRetryWait\":\"datajjrhxornuoqpob\",\"maxConcurrentConnections\":\"datarsdx\",\"disableMetricsCollection\":\"datamq\",\"\":{\"lseoixqp\":\"databqyavcxj\",\"fsuwcmzpwkca\":\"datamsfqntakroxku\",\"zq\":\"datafq\"}}") .toObject(MarketoSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MarketoSource model = new MarketoSource().withSourceRetryCount("dataikceiyuv") - .withSourceRetryWait("databbawrbqoox").withMaxConcurrentConnections("datarqlxqhqgip") - .withDisableMetricsCollection("datatnkngjnhxufo").withQueryTimeout("dataxzrycvacspzj") - .withAdditionalColumns("datayphxeoqma").withQuery("datafarqxjoazyxmumfb"); + MarketoSource model + = new MarketoSource().withSourceRetryCount("datafdrimoopfr").withSourceRetryWait("datajjrhxornuoqpob") + .withMaxConcurrentConnections("datarsdx").withDisableMetricsCollection("datamq") + .withQueryTimeout("datavwrc").withAdditionalColumns("datahllmblls").withQuery("dataylvrofhhitjhh"); model = BinaryData.fromObject(model).toObject(MarketoSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MetadataItemTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MetadataItemTests.java index def3478e6c7aa..f026640cb977c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MetadataItemTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MetadataItemTests.java @@ -11,12 +11,12 @@ public final class MetadataItemTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MetadataItem model - = BinaryData.fromString("{\"name\":\"datavvnxoqaaq\",\"value\":\"datal\"}").toObject(MetadataItem.class); + = BinaryData.fromString("{\"name\":\"datagyxkg\",\"value\":\"datay\"}").toObject(MetadataItem.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MetadataItem model = new MetadataItem().withName("datavvnxoqaaq").withValue("datal"); + MetadataItem model = new MetadataItem().withName("datagyxkg").withValue("datay"); model = BinaryData.fromObject(model).toObject(MetadataItem.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSinkTests.java index 5bfdb2292715f..de0cece932b05 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSinkTests.java @@ -11,16 +11,16 @@ public final class MicrosoftAccessSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MicrosoftAccessSink model = BinaryData.fromString( - "{\"type\":\"MicrosoftAccessSink\",\"preCopyScript\":\"datafo\",\"writeBatchSize\":\"datafksormf\",\"writeBatchTimeout\":\"datauhwxmnrdfjobhr\",\"sinkRetryCount\":\"dataneaupjmjigy\",\"sinkRetryWait\":\"datadfrtasa\",\"maxConcurrentConnections\":\"dataxtoxlxojijtt\",\"disableMetricsCollection\":\"datarxynn\",\"\":{\"cpxftyhfcduqsdur\":\"datanyciss\"}}") + "{\"type\":\"MicrosoftAccessSink\",\"preCopyScript\":\"datailyxpqxnlifhjym\",\"writeBatchSize\":\"datajliivyatyzwy\",\"writeBatchTimeout\":\"dataaycjphozymcypdbu\",\"sinkRetryCount\":\"datanktlzngidgwsc\",\"sinkRetryWait\":\"datamhgzapcgdk\",\"maxConcurrentConnections\":\"dataa\",\"disableMetricsCollection\":\"datacpohlfvsb\",\"\":{\"bjbbyoud\":\"datag\",\"mcumuomdlspsb\":\"datatdlkucxtyufsouh\"}}") .toObject(MicrosoftAccessSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MicrosoftAccessSink model = new MicrosoftAccessSink().withWriteBatchSize("datafksormf") - .withWriteBatchTimeout("datauhwxmnrdfjobhr").withSinkRetryCount("dataneaupjmjigy") - .withSinkRetryWait("datadfrtasa").withMaxConcurrentConnections("dataxtoxlxojijtt") - .withDisableMetricsCollection("datarxynn").withPreCopyScript("datafo"); + MicrosoftAccessSink model = new MicrosoftAccessSink().withWriteBatchSize("datajliivyatyzwy") + .withWriteBatchTimeout("dataaycjphozymcypdbu").withSinkRetryCount("datanktlzngidgwsc") + .withSinkRetryWait("datamhgzapcgdk").withMaxConcurrentConnections("dataa") + .withDisableMetricsCollection("datacpohlfvsb").withPreCopyScript("datailyxpqxnlifhjym"); model = BinaryData.fromObject(model).toObject(MicrosoftAccessSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSourceTests.java index 587610206740b..e460cd14fc394 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessSourceTests.java @@ -11,16 +11,16 @@ public final class MicrosoftAccessSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MicrosoftAccessSource model = BinaryData.fromString( - "{\"type\":\"MicrosoftAccessSource\",\"query\":\"dataezgvaeqiygbou\",\"additionalColumns\":\"datajodidgudar\",\"sourceRetryCount\":\"dataajbenf\",\"sourceRetryWait\":\"dataufvojikffczw\",\"maxConcurrentConnections\":\"datawpilsuhsghdovcpb\",\"disableMetricsCollection\":\"datanapg\",\"\":{\"whh\":\"datahsixzcdaukh\"}}") + "{\"type\":\"MicrosoftAccessSource\",\"query\":\"datadljqjstncjwz\",\"additionalColumns\":\"datatezltlundkj\",\"sourceRetryCount\":\"datavhhxivshjuxm\",\"sourceRetryWait\":\"dataythxearlp\",\"maxConcurrentConnections\":\"datajjticly\",\"disableMetricsCollection\":\"dataduxbungmpn\",\"\":{\"goazzy\":\"datagucdfxglrcj\",\"nyhzestt\":\"datajcwuzanpoyrqjoni\",\"gzdbonep\":\"datacloq\"}}") .toObject(MicrosoftAccessSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { MicrosoftAccessSource model - = new MicrosoftAccessSource().withSourceRetryCount("dataajbenf").withSourceRetryWait("dataufvojikffczw") - .withMaxConcurrentConnections("datawpilsuhsghdovcpb").withDisableMetricsCollection("datanapg") - .withQuery("dataezgvaeqiygbou").withAdditionalColumns("datajodidgudar"); + = new MicrosoftAccessSource().withSourceRetryCount("datavhhxivshjuxm").withSourceRetryWait("dataythxearlp") + .withMaxConcurrentConnections("datajjticly").withDisableMetricsCollection("dataduxbungmpn") + .withQuery("datadljqjstncjwz").withAdditionalColumns("datatezltlundkj"); model = BinaryData.fromObject(model).toObject(MicrosoftAccessSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTests.java index 2cda74dfb2c28..bdda954ab2539 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTests.java @@ -19,33 +19,34 @@ public final class MicrosoftAccessTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MicrosoftAccessTableDataset model = BinaryData.fromString( - "{\"type\":\"MicrosoftAccessTable\",\"typeProperties\":{\"tableName\":\"dataxamqecjrzvlcivqx\"},\"description\":\"mklphxwww\",\"structure\":\"datajkbgnfbr\",\"schema\":\"datavfsunhaevla\",\"linkedServiceName\":{\"referenceName\":\"xczywywu\",\"parameters\":{\"rfgimomggewdqbxe\":\"datacorewcnnaaxqjfda\",\"sfx\":\"datafyznvussuqksl\",\"wpmohnrtlikffyd\":\"datayzqbye\",\"fwvzdteqjm\":\"datatkqrfbgyn\"}},\"parameters\":{\"jyoxxjxb\":{\"type\":\"Array\",\"defaultValue\":\"datagkaxnypr\"},\"emqom\":{\"type\":\"Int\",\"defaultValue\":\"datarrlccklyfpjmspa\"},\"hcaptkhjx\":{\"type\":\"Int\",\"defaultValue\":\"datalknuyapvibzicyvi\"}},\"annotations\":[\"databnvfccklzhznfgv\"],\"folder\":{\"name\":\"xmnctigpksywi\"},\"\":{\"efuhb\":\"dataktgkdprtqjytdc\",\"caytnpkvbpbltcws\":\"datawbvjsbgmlamoa\"}}") + "{\"type\":\"MicrosoftAccessTable\",\"typeProperties\":{\"tableName\":\"datanrkbnv\"},\"description\":\"cklzhznfgvlxy\",\"structure\":\"datanctigpksywi\",\"schema\":\"datalktgkdp\",\"linkedServiceName\":{\"referenceName\":\"tqjytdc\",\"parameters\":{\"gmlamoaxc\":\"datauhbdwbvjs\",\"kvbpbl\":\"dataytn\",\"exheeocnqo\":\"datacw\"}},\"parameters\":{\"xyfhxohzbzhhavzf\":{\"type\":\"Object\",\"defaultValue\":\"datavlryszfh\"},\"cofuvtfu\":{\"type\":\"Array\",\"defaultValue\":\"datavkds\"},\"subzsspmj\":{\"type\":\"Bool\",\"defaultValue\":\"datauisaklhjfddxqfu\"},\"wbztrt\":{\"type\":\"Int\",\"defaultValue\":\"datalfauyvxpqwlkqd\"}},\"annotations\":[\"datawvoglff\",\"datadhg\"],\"folder\":{\"name\":\"rmhbtofcv\"},\"\":{\"g\":\"datalhcnsdylmnqunk\"}}") .toObject(MicrosoftAccessTableDataset.class); - Assertions.assertEquals("mklphxwww", model.description()); - Assertions.assertEquals("xczywywu", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("jyoxxjxb").type()); - Assertions.assertEquals("xmnctigpksywi", model.folder().name()); + Assertions.assertEquals("cklzhznfgvlxy", model.description()); + Assertions.assertEquals("tqjytdc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("xyfhxohzbzhhavzf").type()); + Assertions.assertEquals("rmhbtofcv", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MicrosoftAccessTableDataset model = new MicrosoftAccessTableDataset().withDescription("mklphxwww") - .withStructure("datajkbgnfbr").withSchema("datavfsunhaevla") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("xczywywu") - .withParameters(mapOf("rfgimomggewdqbxe", "datacorewcnnaaxqjfda", "sfx", "datafyznvussuqksl", - "wpmohnrtlikffyd", "datayzqbye", "fwvzdteqjm", "datatkqrfbgyn"))) - .withParameters(mapOf("jyoxxjxb", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datagkaxnypr"), "emqom", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datarrlccklyfpjmspa"), - "hcaptkhjx", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datalknuyapvibzicyvi"))) - .withAnnotations(Arrays.asList("databnvfccklzhznfgv")) - .withFolder(new DatasetFolder().withName("xmnctigpksywi")).withTableName("dataxamqecjrzvlcivqx"); + MicrosoftAccessTableDataset model = new MicrosoftAccessTableDataset().withDescription("cklzhznfgvlxy") + .withStructure("datanctigpksywi").withSchema("datalktgkdp") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("tqjytdc") + .withParameters(mapOf("gmlamoaxc", "datauhbdwbvjs", "kvbpbl", "dataytn", "exheeocnqo", "datacw"))) + .withParameters(mapOf("xyfhxohzbzhhavzf", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datavlryszfh"), + "cofuvtfu", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datavkds"), + "subzsspmj", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datauisaklhjfddxqfu"), + "wbztrt", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datalfauyvxpqwlkqd"))) + .withAnnotations(Arrays.asList("datawvoglff", "datadhg")) + .withFolder(new DatasetFolder().withName("rmhbtofcv")).withTableName("datanrkbnv"); model = BinaryData.fromObject(model).toObject(MicrosoftAccessTableDataset.class); - Assertions.assertEquals("mklphxwww", model.description()); - Assertions.assertEquals("xczywywu", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("jyoxxjxb").type()); - Assertions.assertEquals("xmnctigpksywi", model.folder().name()); + Assertions.assertEquals("cklzhznfgvlxy", model.description()); + Assertions.assertEquals("tqjytdc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("xyfhxohzbzhhavzf").type()); + Assertions.assertEquals("rmhbtofcv", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTypePropertiesTests.java index 708f25adcd8fd..a58660ea54f07 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MicrosoftAccessTableDatasetTypePropertiesTests.java @@ -10,14 +10,14 @@ public final class MicrosoftAccessTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MicrosoftAccessTableDatasetTypeProperties model = BinaryData.fromString("{\"tableName\":\"dataheeocnqoubve\"}") + MicrosoftAccessTableDatasetTypeProperties model = BinaryData.fromString("{\"tableName\":\"datatmsgkwedwl\"}") .toObject(MicrosoftAccessTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { MicrosoftAccessTableDatasetTypeProperties model - = new MicrosoftAccessTableDatasetTypeProperties().withTableName("dataheeocnqoubve"); + = new MicrosoftAccessTableDatasetTypeProperties().withTableName("datatmsgkwedwl"); model = BinaryData.fromObject(model).toObject(MicrosoftAccessTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTests.java index ccd11beee9980..30e9d8bb502d2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTests.java @@ -18,37 +18,29 @@ public final class MongoDbAtlasLinkedServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbAtlasLinkedService model = BinaryData.fromString( - "{\"type\":\"MongoDbAtlas\",\"typeProperties\":{\"connectionString\":\"datajawpzrtse\",\"database\":\"dataxpkrwgznacmkw\",\"driverVersion\":\"dataeljniadg\"},\"connectVia\":{\"referenceName\":\"bwnyyihct\",\"parameters\":{\"htvughu\":\"datamzozb\",\"vnfyfyftkvzb\":\"dataiql\"}},\"description\":\"jykiumagfy\",\"parameters\":{\"twf\":{\"type\":\"Bool\",\"defaultValue\":\"datajzpl\"},\"cfqynmldhgzzohta\":{\"type\":\"Int\",\"defaultValue\":\"datacvbzkbebuafbdd\"},\"rksypoosfxzobovs\":{\"type\":\"SecureString\",\"defaultValue\":\"datauhdqhq\"},\"xgllmdeoubvfxi\":{\"type\":\"Float\",\"defaultValue\":\"datatd\"}},\"annotations\":[\"datamncjhmvvzeaic\",\"datavbsjtfqwlbeqcz\",\"datascgropuqgiscgk\",\"dataciyikqlyoorme\"],\"\":{\"hxuznbcpiadj\":\"datalqijykck\"}}") + "{\"type\":\"MongoDbAtlas\",\"typeProperties\":{\"connectionString\":\"datakubwkdi\",\"database\":\"dataslnbyi\",\"driverVersion\":\"dataetwwjvxycygcv\"},\"connectVia\":{\"referenceName\":\"bfeitya\",\"parameters\":{\"pd\":\"datacvttkg\",\"icucqtecie\":\"dataqn\",\"zhjjlypwgwbycfam\":\"datascx\",\"irhhbogxw\":\"dataikisppygothi\"}},\"description\":\"pymyerpus\",\"parameters\":{\"gbzjoyyptnok\":{\"type\":\"String\",\"defaultValue\":\"datatjliwbnwd\"},\"zxkflz\":{\"type\":\"Int\",\"defaultValue\":\"dataxkzeowizvje\"}},\"annotations\":[\"dataxxboauvkkc\",\"datawgnwhldctn\"],\"\":{\"rvhed\":\"datauqmdynmtnlcvl\",\"yag\":\"databzzb\"}}") .toObject(MongoDbAtlasLinkedService.class); - Assertions.assertEquals("bwnyyihct", model.connectVia().referenceName()); - Assertions.assertEquals("jykiumagfy", model.description()); - Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("twf").type()); + Assertions.assertEquals("bfeitya", model.connectVia().referenceName()); + Assertions.assertEquals("pymyerpus", model.description()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("gbzjoyyptnok").type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbAtlasLinkedService model - = new MongoDbAtlasLinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("bwnyyihct") - .withParameters(mapOf("htvughu", "datamzozb", "vnfyfyftkvzb", "dataiql"))) - .withDescription("jykiumagfy") - .withParameters( - mapOf("twf", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datajzpl"), - "cfqynmldhgzzohta", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datacvbzkbebuafbdd"), - "rksypoosfxzobovs", - new ParameterSpecification().withType(ParameterType.SECURE_STRING) - .withDefaultValue("datauhdqhq"), - "xgllmdeoubvfxi", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datatd"))) - .withAnnotations( - Arrays.asList("datamncjhmvvzeaic", "datavbsjtfqwlbeqcz", "datascgropuqgiscgk", "dataciyikqlyoorme")) - .withConnectionString("datajawpzrtse").withDatabase("dataxpkrwgznacmkw") - .withDriverVersion("dataeljniadg"); + MongoDbAtlasLinkedService model = new MongoDbAtlasLinkedService() + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("bfeitya") + .withParameters(mapOf("pd", "datacvttkg", "icucqtecie", "dataqn", "zhjjlypwgwbycfam", "datascx", + "irhhbogxw", "dataikisppygothi"))) + .withDescription("pymyerpus") + .withParameters(mapOf("gbzjoyyptnok", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datatjliwbnwd"), "zxkflz", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataxkzeowizvje"))) + .withAnnotations(Arrays.asList("dataxxboauvkkc", "datawgnwhldctn")).withConnectionString("datakubwkdi") + .withDatabase("dataslnbyi").withDriverVersion("dataetwwjvxycygcv"); model = BinaryData.fromObject(model).toObject(MongoDbAtlasLinkedService.class); - Assertions.assertEquals("bwnyyihct", model.connectVia().referenceName()); - Assertions.assertEquals("jykiumagfy", model.description()); - Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("twf").type()); + Assertions.assertEquals("bfeitya", model.connectVia().referenceName()); + Assertions.assertEquals("pymyerpus", model.description()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("gbzjoyyptnok").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTypePropertiesTests.java index 15e9dad0e7c84..49f4785254adf 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasLinkedServiceTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class MongoDbAtlasLinkedServiceTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbAtlasLinkedServiceTypeProperties model = BinaryData.fromString( - "{\"connectionString\":\"datalvxsulcdwbib\",\"database\":\"dataviacghygm\",\"driverVersion\":\"dataivpzjnrw\"}") + "{\"connectionString\":\"dataxwcndkdlxdlh\",\"database\":\"dataalz\",\"driverVersion\":\"dataawkyxownoypqu\"}") .toObject(MongoDbAtlasLinkedServiceTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { MongoDbAtlasLinkedServiceTypeProperties model = new MongoDbAtlasLinkedServiceTypeProperties() - .withConnectionString("datalvxsulcdwbib").withDatabase("dataviacghygm").withDriverVersion("dataivpzjnrw"); + .withConnectionString("dataxwcndkdlxdlh").withDatabase("dataalz").withDriverVersion("dataawkyxownoypqu"); model = BinaryData.fromObject(model).toObject(MongoDbAtlasLinkedServiceTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSinkTests.java index 92650aa9091ef..84eac473050a9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSinkTests.java @@ -11,16 +11,16 @@ public final class MongoDbAtlasSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbAtlasSink model = BinaryData.fromString( - "{\"type\":\"MongoDbAtlasSink\",\"writeBehavior\":\"dataqejhsirotjklt\",\"writeBatchSize\":\"dataobscptckgqyuvh\",\"writeBatchTimeout\":\"datamjpzgjnqif\",\"sinkRetryCount\":\"datanfdboumpks\",\"sinkRetryWait\":\"datadj\",\"maxConcurrentConnections\":\"datasmdgrhdlwle\",\"disableMetricsCollection\":\"dataqxyjlbkwv\",\"\":{\"akzgryfxwwqbey\":\"datatblsa\"}}") + "{\"type\":\"MongoDbAtlasSink\",\"writeBehavior\":\"datakhgdzrcq\",\"writeBatchSize\":\"dataapohemine\",\"writeBatchTimeout\":\"datajrdxhlovmxhztdca\",\"sinkRetryCount\":\"datamvqgqmi\",\"sinkRetryWait\":\"datapa\",\"maxConcurrentConnections\":\"datah\",\"disableMetricsCollection\":\"datacyasz\",\"\":{\"ni\":\"datamtcihupoelj\",\"awbsdeqqbdcbnrg\":\"datayoxajit\",\"mtgtnb\":\"datapnor\",\"rwldeinhnsd\":\"datasopuwesmxodyto\"}}") .toObject(MongoDbAtlasSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbAtlasSink model = new MongoDbAtlasSink().withWriteBatchSize("dataobscptckgqyuvh") - .withWriteBatchTimeout("datamjpzgjnqif").withSinkRetryCount("datanfdboumpks").withSinkRetryWait("datadj") - .withMaxConcurrentConnections("datasmdgrhdlwle").withDisableMetricsCollection("dataqxyjlbkwv") - .withWriteBehavior("dataqejhsirotjklt"); + MongoDbAtlasSink model + = new MongoDbAtlasSink().withWriteBatchSize("dataapohemine").withWriteBatchTimeout("datajrdxhlovmxhztdca") + .withSinkRetryCount("datamvqgqmi").withSinkRetryWait("datapa").withMaxConcurrentConnections("datah") + .withDisableMetricsCollection("datacyasz").withWriteBehavior("datakhgdzrcq"); model = BinaryData.fromObject(model).toObject(MongoDbAtlasSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSourceTests.java index f27db900c0e45..c0479d76c1bca 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbAtlasSourceTests.java @@ -14,18 +14,19 @@ public final class MongoDbAtlasSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbAtlasSource model = BinaryData.fromString( - "{\"type\":\"MongoDbAtlasSource\",\"filter\":\"datai\",\"cursorMethods\":{\"project\":\"databhmy\",\"sort\":\"datagmf\",\"skip\":\"datar\",\"limit\":\"datakggwiq\",\"\":{\"keolzizfbunzmx\":\"datafi\",\"vjwkpznsfbit\":\"datafttmjomuwl\",\"yo\":\"datafzgpvdlx\"}},\"batchSize\":\"datalcuxzlln\",\"queryTimeout\":\"datagqcfggje\",\"additionalColumns\":\"dataoxmpgfspwhfh\",\"sourceRetryCount\":\"datauu\",\"sourceRetryWait\":\"dataavvzvdfytqz\",\"maxConcurrentConnections\":\"datam\",\"disableMetricsCollection\":\"datawpwrfet\",\"\":{\"nrjmicha\":\"datarmeufhkoe\"}}") + "{\"type\":\"MongoDbAtlasSource\",\"filter\":\"datawthvu\",\"cursorMethods\":{\"project\":\"datazznvdjnspy\",\"sort\":\"dataoygutqtjwiv\",\"skip\":\"datamavfzjwdww\",\"limit\":\"dataxehndcpiwcgcwmsh\",\"\":{\"xopzclka\":\"dataxjxhdwj\",\"mga\":\"datapu\"}},\"batchSize\":\"datac\",\"queryTimeout\":\"datajjfmzv\",\"additionalColumns\":\"databflyzc\",\"sourceRetryCount\":\"datamlybsy\",\"sourceRetryWait\":\"dataon\",\"maxConcurrentConnections\":\"datavbfpu\",\"disableMetricsCollection\":\"dataobtdhum\",\"\":{\"jefclih\":\"datawckapoetdfzj\",\"lqzopvhwmtdbfrj\":\"datanawipdqozv\",\"uv\":\"dataq\",\"feagordbs\":\"dataps\"}}") .toObject(MongoDbAtlasSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbAtlasSource model = new MongoDbAtlasSource().withSourceRetryCount("datauu") - .withSourceRetryWait("dataavvzvdfytqz").withMaxConcurrentConnections("datam") - .withDisableMetricsCollection("datawpwrfet").withFilter("datai") - .withCursorMethods(new MongoDbCursorMethodsProperties().withProject("databhmy").withSort("datagmf") - .withSkip("datar").withLimit("datakggwiq").withAdditionalProperties(mapOf())) - .withBatchSize("datalcuxzlln").withQueryTimeout("datagqcfggje").withAdditionalColumns("dataoxmpgfspwhfh"); + MongoDbAtlasSource model = new MongoDbAtlasSource().withSourceRetryCount("datamlybsy") + .withSourceRetryWait("dataon").withMaxConcurrentConnections("datavbfpu") + .withDisableMetricsCollection("dataobtdhum").withFilter("datawthvu") + .withCursorMethods( + new MongoDbCursorMethodsProperties().withProject("datazznvdjnspy").withSort("dataoygutqtjwiv") + .withSkip("datamavfzjwdww").withLimit("dataxehndcpiwcgcwmsh").withAdditionalProperties(mapOf())) + .withBatchSize("datac").withQueryTimeout("datajjfmzv").withAdditionalColumns("databflyzc"); model = BinaryData.fromObject(model).toObject(MongoDbAtlasSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbCursorMethodsPropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbCursorMethodsPropertiesTests.java index 55a64677411f9..660c8ad6f093d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbCursorMethodsPropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbCursorMethodsPropertiesTests.java @@ -13,15 +13,15 @@ public final class MongoDbCursorMethodsPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbCursorMethodsProperties model = BinaryData.fromString( - "{\"project\":\"dataen\",\"sort\":\"dataqjvdde\",\"skip\":\"datavrjhtpxydiuviup\",\"limit\":\"datatnsyrrybdyqiv\",\"\":{\"zihdkq\":\"datauh\",\"inwgi\":\"datajwthwcpijgasnafd\"}}") + "{\"project\":\"datacntdwijx\",\"sort\":\"dataltowdwiffagfe\",\"skip\":\"datambpgcbltthsuzx\",\"limit\":\"datavifl\",\"\":{\"scobhhblj\":\"datak\",\"us\":\"datavpokvhobygffuzh\"}}") .toObject(MongoDbCursorMethodsProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { MongoDbCursorMethodsProperties model - = new MongoDbCursorMethodsProperties().withProject("dataen").withSort("dataqjvdde") - .withSkip("datavrjhtpxydiuviup").withLimit("datatnsyrrybdyqiv").withAdditionalProperties(mapOf()); + = new MongoDbCursorMethodsProperties().withProject("datacntdwijx").withSort("dataltowdwiffagfe") + .withSkip("datambpgcbltthsuzx").withLimit("datavifl").withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(MongoDbCursorMethodsProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbSourceTests.java index 6e6a901d0f7aa..10e349dcc9b9e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbSourceTests.java @@ -11,15 +11,15 @@ public final class MongoDbSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbSource model = BinaryData.fromString( - "{\"type\":\"MongoDbSource\",\"query\":\"datavyflkeqgxlj\",\"additionalColumns\":\"datatosiwcveq\",\"sourceRetryCount\":\"datahbw\",\"sourceRetryWait\":\"dataoc\",\"maxConcurrentConnections\":\"datazlfhhwdajfth\",\"disableMetricsCollection\":\"datawuomjd\",\"\":{\"qsniobehxxb\":\"datafq\",\"wkryzrdqpkqfuv\":\"dataeiobubtpyemp\"}}") + "{\"type\":\"MongoDbSource\",\"query\":\"datarbgnyvypu\",\"additionalColumns\":\"dataxhowwe\",\"sourceRetryCount\":\"datayw\",\"sourceRetryWait\":\"datahiuwv\",\"maxConcurrentConnections\":\"datagejytqnzrcbh\",\"disableMetricsCollection\":\"datahctjvlwfnzgz\",\"\":{\"fuhsmuclxgcedus\":\"datayvytydrdcwbaiaq\",\"vykagsxhzhervv\":\"datayq\",\"zrvf\":\"dataibrolqxloed\",\"fgwuj\":\"datafsyq\"}}") .toObject(MongoDbSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbSource model = new MongoDbSource().withSourceRetryCount("datahbw").withSourceRetryWait("dataoc") - .withMaxConcurrentConnections("datazlfhhwdajfth").withDisableMetricsCollection("datawuomjd") - .withQuery("datavyflkeqgxlj").withAdditionalColumns("datatosiwcveq"); + MongoDbSource model = new MongoDbSource().withSourceRetryCount("datayw").withSourceRetryWait("datahiuwv") + .withMaxConcurrentConnections("datagejytqnzrcbh").withDisableMetricsCollection("datahctjvlwfnzgz") + .withQuery("datarbgnyvypu").withAdditionalColumns("dataxhowwe"); model = BinaryData.fromObject(model).toObject(MongoDbSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTests.java index a38bb4c01f532..49ab06f618350 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTests.java @@ -18,26 +18,29 @@ public final class MongoDbV2LinkedServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbV2LinkedService model = BinaryData.fromString( - "{\"type\":\"MongoDbV2\",\"typeProperties\":{\"connectionString\":\"datafzzmxqyjy\",\"database\":\"datarjpy\"},\"connectVia\":{\"referenceName\":\"ewvlkzmsyezxifj\",\"parameters\":{\"codzdhe\":\"datauovfotwvkuhtdx\",\"ahtkkccrusrz\":\"datanmcvmxiundif\"}},\"description\":\"oqodya\",\"parameters\":{\"ctilbxn\":{\"type\":\"Int\",\"defaultValue\":\"datalvtiploaws\"}},\"annotations\":[\"datajt\"],\"\":{\"cdrchue\":\"datahpelzavohvdb\",\"lvobctitexrbalv\":\"dataxliqcbonqjoekcjc\",\"pdyrqlazfjl\":\"datafdqilprqhxfqk\",\"q\":\"datackbjvdxhyf\"}}") + "{\"type\":\"MongoDbV2\",\"typeProperties\":{\"connectionString\":\"datasxiaj\",\"database\":\"datacrsdgtj\"},\"connectVia\":{\"referenceName\":\"opnccaxtp\",\"parameters\":{\"feabzjp\":\"datazehijlw\",\"ceqaklsfxlrx\":\"datahjmg\"}},\"description\":\"ymfqmvnhsfjxtusb\",\"parameters\":{\"onalpwa\":{\"type\":\"Float\",\"defaultValue\":\"datakkdptsppgtp\"},\"wnwnjdrnfgtmupbl\":{\"type\":\"Array\",\"defaultValue\":\"datawuyas\"}},\"annotations\":[\"dataqvz\",\"dataikxufuwcajyezlk\",\"dataiveftugiwsvlfp\",\"databpnrgnxwrfu\"],\"\":{\"fzuvuoxzy\":\"datay\"}}") .toObject(MongoDbV2LinkedService.class); - Assertions.assertEquals("ewvlkzmsyezxifj", model.connectVia().referenceName()); - Assertions.assertEquals("oqodya", model.description()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("ctilbxn").type()); + Assertions.assertEquals("opnccaxtp", model.connectVia().referenceName()); + Assertions.assertEquals("ymfqmvnhsfjxtusb", model.description()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("onalpwa").type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { MongoDbV2LinkedService model = new MongoDbV2LinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("ewvlkzmsyezxifj") - .withParameters(mapOf("codzdhe", "datauovfotwvkuhtdx", "ahtkkccrusrz", "datanmcvmxiundif"))) - .withDescription("oqodya") - .withParameters(mapOf("ctilbxn", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datalvtiploaws"))) - .withAnnotations(Arrays.asList("datajt")).withConnectionString("datafzzmxqyjy").withDatabase("datarjpy"); + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("opnccaxtp") + .withParameters(mapOf("feabzjp", "datazehijlw", "ceqaklsfxlrx", "datahjmg"))) + .withDescription("ymfqmvnhsfjxtusb") + .withParameters(mapOf("onalpwa", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datakkdptsppgtp"), + "wnwnjdrnfgtmupbl", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datawuyas"))) + .withAnnotations(Arrays.asList("dataqvz", "dataikxufuwcajyezlk", "dataiveftugiwsvlfp", "databpnrgnxwrfu")) + .withConnectionString("datasxiaj").withDatabase("datacrsdgtj"); model = BinaryData.fromObject(model).toObject(MongoDbV2LinkedService.class); - Assertions.assertEquals("ewvlkzmsyezxifj", model.connectVia().referenceName()); - Assertions.assertEquals("oqodya", model.description()); - Assertions.assertEquals(ParameterType.INT, model.parameters().get("ctilbxn").type()); + Assertions.assertEquals("opnccaxtp", model.connectVia().referenceName()); + Assertions.assertEquals("ymfqmvnhsfjxtusb", model.description()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("onalpwa").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTypePropertiesTests.java index c23ffe864a732..d9e0892f549aa 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2LinkedServiceTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class MongoDbV2LinkedServiceTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbV2LinkedServiceTypeProperties model - = BinaryData.fromString("{\"connectionString\":\"datade\",\"database\":\"datafdz\"}") + = BinaryData.fromString("{\"connectionString\":\"dataqcjclvbqovkz\",\"database\":\"dataeytphnazpgvfcubx\"}") .toObject(MongoDbV2LinkedServiceTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbV2LinkedServiceTypeProperties model - = new MongoDbV2LinkedServiceTypeProperties().withConnectionString("datade").withDatabase("datafdz"); + MongoDbV2LinkedServiceTypeProperties model = new MongoDbV2LinkedServiceTypeProperties() + .withConnectionString("dataqcjclvbqovkz").withDatabase("dataeytphnazpgvfcubx"); model = BinaryData.fromObject(model).toObject(MongoDbV2LinkedServiceTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SinkTests.java index cf35a26c60ed4..4094a15d65678 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SinkTests.java @@ -11,16 +11,16 @@ public final class MongoDbV2SinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbV2Sink model = BinaryData.fromString( - "{\"type\":\"MongoDbV2Sink\",\"writeBehavior\":\"datadnjmjies\",\"writeBatchSize\":\"dataimvziyicxnxci\",\"writeBatchTimeout\":\"datalvzxucnpa\",\"sinkRetryCount\":\"datarairefifbisljhg\",\"sinkRetryWait\":\"datajm\",\"maxConcurrentConnections\":\"datam\",\"disableMetricsCollection\":\"datav\",\"\":{\"jlgiurmlir\":\"datarmmweeuyohjhp\",\"tkdit\":\"dataadqeqfxzcxvpog\",\"xfowfnsyyeytrwyo\":\"datawokefdeeppycwsy\"}}") + "{\"type\":\"MongoDbV2Sink\",\"writeBehavior\":\"datauaqtqnqm\",\"writeBatchSize\":\"dataptzgomuju\",\"writeBatchTimeout\":\"datankuyombkgkyobu\",\"sinkRetryCount\":\"dataprvokodrpyxkzx\",\"sinkRetryWait\":\"datamoycufkxygxoubek\",\"maxConcurrentConnections\":\"datadxgtgcfk\",\"disableMetricsCollection\":\"datae\",\"\":{\"fpqebbqetx\":\"datahtlk\"}}") .toObject(MongoDbV2Sink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbV2Sink model = new MongoDbV2Sink().withWriteBatchSize("dataimvziyicxnxci") - .withWriteBatchTimeout("datalvzxucnpa").withSinkRetryCount("datarairefifbisljhg") - .withSinkRetryWait("datajm").withMaxConcurrentConnections("datam").withDisableMetricsCollection("datav") - .withWriteBehavior("datadnjmjies"); + MongoDbV2Sink model = new MongoDbV2Sink().withWriteBatchSize("dataptzgomuju") + .withWriteBatchTimeout("datankuyombkgkyobu").withSinkRetryCount("dataprvokodrpyxkzx") + .withSinkRetryWait("datamoycufkxygxoubek").withMaxConcurrentConnections("datadxgtgcfk") + .withDisableMetricsCollection("datae").withWriteBehavior("datauaqtqnqm"); model = BinaryData.fromObject(model).toObject(MongoDbV2Sink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SourceTests.java index 3f4c7f31a31a5..74065089bbb48 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MongoDbV2SourceTests.java @@ -14,20 +14,19 @@ public final class MongoDbV2SourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MongoDbV2Source model = BinaryData.fromString( - "{\"type\":\"MongoDbV2Source\",\"filter\":\"datajgsovzbdh\",\"cursorMethods\":{\"project\":\"datapanhygcahijb\",\"sort\":\"datalxsvjzbggs\",\"skip\":\"datanojtyh\",\"limit\":\"datazxzazofronsxj\",\"\":{\"ktnsowszbeflhx\":\"datai\",\"gokyngarwz\":\"datak\"}},\"batchSize\":\"datazjxgassmna\",\"queryTimeout\":\"datapolueylqysgmiix\",\"additionalColumns\":\"dataekcwec\",\"sourceRetryCount\":\"datatkdginm\",\"sourceRetryWait\":\"datagp\",\"maxConcurrentConnections\":\"dataqqcceyowrwvbqv\",\"disableMetricsCollection\":\"dataqgqrsopqgiqf\",\"\":{\"yzcgugslpvy\":\"datalj\"}}") + "{\"type\":\"MongoDbV2Source\",\"filter\":\"dataff\",\"cursorMethods\":{\"project\":\"dataovfwzys\",\"sort\":\"datavrfjgbxupvhgo\",\"skip\":\"datavwuje\",\"limit\":\"dataow\",\"\":{\"aomy\":\"datasziubkyvcgkoufw\",\"vfmsxamncuhxz\":\"datalxrwdjbya\"}},\"batchSize\":\"dataakxyhuetztorh\",\"queryTimeout\":\"datau\",\"additionalColumns\":\"datasszhsewjqgzlo\",\"sourceRetryCount\":\"datahxd\",\"sourceRetryWait\":\"dataegljqpyxi\",\"maxConcurrentConnections\":\"datakgdkanm\",\"disableMetricsCollection\":\"datawgchgpb\",\"\":{\"axno\":\"datawgoomap\",\"jzt\":\"datanjfvjqvectoo\",\"vsrvkzv\":\"dataalsnm\"}}") .toObject(MongoDbV2Source.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MongoDbV2Source model = new MongoDbV2Source().withSourceRetryCount("datatkdginm").withSourceRetryWait("datagp") - .withMaxConcurrentConnections("dataqqcceyowrwvbqv").withDisableMetricsCollection("dataqgqrsopqgiqf") - .withFilter("datajgsovzbdh") + MongoDbV2Source model = new MongoDbV2Source().withSourceRetryCount("datahxd") + .withSourceRetryWait("dataegljqpyxi").withMaxConcurrentConnections("datakgdkanm") + .withDisableMetricsCollection("datawgchgpb").withFilter("dataff") .withCursorMethods( - new MongoDbCursorMethodsProperties().withProject("datapanhygcahijb").withSort("datalxsvjzbggs") - .withSkip("datanojtyh").withLimit("datazxzazofronsxj").withAdditionalProperties(mapOf())) - .withBatchSize("datazjxgassmna").withQueryTimeout("datapolueylqysgmiix") - .withAdditionalColumns("dataekcwec"); + new MongoDbCursorMethodsProperties().withProject("dataovfwzys").withSort("datavrfjgbxupvhgo") + .withSkip("datavwuje").withLimit("dataow").withAdditionalProperties(mapOf())) + .withBatchSize("dataakxyhuetztorh").withQueryTimeout("datau").withAdditionalColumns("datasszhsewjqgzlo"); model = BinaryData.fromObject(model).toObject(MongoDbV2Source.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MultiplePipelineTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MultiplePipelineTriggerTests.java index 9ec561423e1a7..55e4220ec7295 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MultiplePipelineTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MultiplePipelineTriggerTests.java @@ -17,31 +17,31 @@ public final class MultiplePipelineTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MultiplePipelineTrigger model = BinaryData.fromString( - "{\"type\":\"MultiplePipelineTrigger\",\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"dqmbouwuajsitgp\",\"name\":\"blkcvkmedrkol\"},\"parameters\":{\"k\":\"databnrafvk\"}},{\"pipelineReference\":{\"referenceName\":\"bdjxvcxepj\",\"name\":\"cmrhivwcmtretfl\"},\"parameters\":{\"hrzpyxmfip\":\"datavqkbxgzepinyursq\",\"sxarxvft\":\"datagmlfpbwfx\",\"a\":\"datalsu\"}}],\"description\":\"ddgrbclt\",\"runtimeState\":\"Started\",\"annotations\":[\"dataiiujukcdlvptxty\"],\"\":{\"pbodswgngl\":\"datam\",\"scjefapouwsynsb\":\"datallrxpxslccu\",\"pdwyhggvhcoaoeti\":\"datandirdlehjz\",\"eirambfm\":\"datakt\"}}") + "{\"type\":\"MultiplePipelineTrigger\",\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"oxj\",\"name\":\"ha\"},\"parameters\":{\"fcr\":\"dataqktbgu\",\"xergclmqkufq\":\"datatcupo\",\"kqezeee\":\"datamylrtnzyosd\",\"fk\":\"dataligunw\"}},{\"pipelineReference\":{\"referenceName\":\"vwzywxzx\",\"name\":\"htqcwidspegxdeai\"},\"parameters\":{\"yjnmdc\":\"datacmcqslngmsip\",\"cxacgeiyfpfaaah\":\"datatj\",\"fjld\":\"dataphuplfopqgcadnt\"}}],\"description\":\"f\",\"runtimeState\":\"Stopped\",\"annotations\":[\"dataoygcofh\"],\"\":{\"wuwbnngcdtxxyz\":\"datahhirbgmxmvxbaazn\",\"uqtjcyllpas\":\"dataybndiqpadhrij\"}}") .toObject(MultiplePipelineTrigger.class); - Assertions.assertEquals("ddgrbclt", model.description()); - Assertions.assertEquals("dqmbouwuajsitgp", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("blkcvkmedrkol", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("f", model.description()); + Assertions.assertEquals("oxj", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("ha", model.pipelines().get(0).pipelineReference().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MultiplePipelineTrigger model = new MultiplePipelineTrigger().withDescription("ddgrbclt") - .withAnnotations(Arrays.asList("dataiiujukcdlvptxty")) - .withPipelines(Arrays.asList( - new TriggerPipelineReference() - .withPipelineReference( - new PipelineReference().withReferenceName("dqmbouwuajsitgp").withName("blkcvkmedrkol")) - .withParameters(mapOf("k", "databnrafvk")), - new TriggerPipelineReference() - .withPipelineReference( - new PipelineReference().withReferenceName("bdjxvcxepj").withName("cmrhivwcmtretfl")) - .withParameters( - mapOf("hrzpyxmfip", "datavqkbxgzepinyursq", "sxarxvft", "datagmlfpbwfx", "a", "datalsu")))); + MultiplePipelineTrigger model + = new MultiplePipelineTrigger().withDescription("f").withAnnotations(Arrays.asList("dataoygcofh")) + .withPipelines(Arrays.asList( + new TriggerPipelineReference() + .withPipelineReference(new PipelineReference().withReferenceName("oxj").withName("ha")) + .withParameters(mapOf("fcr", "dataqktbgu", "xergclmqkufq", "datatcupo", "kqezeee", + "datamylrtnzyosd", "fk", "dataligunw")), + new TriggerPipelineReference() + .withPipelineReference( + new PipelineReference().withReferenceName("vwzywxzx").withName("htqcwidspegxdeai")) + .withParameters(mapOf("yjnmdc", "datacmcqslngmsip", "cxacgeiyfpfaaah", "datatj", "fjld", + "dataphuplfopqgcadnt")))); model = BinaryData.fromObject(model).toObject(MultiplePipelineTrigger.class); - Assertions.assertEquals("ddgrbclt", model.description()); - Assertions.assertEquals("dqmbouwuajsitgp", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("blkcvkmedrkol", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals("f", model.description()); + Assertions.assertEquals("oxj", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("ha", model.pipelines().get(0).pipelineReference().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MySqlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MySqlSourceTests.java index 141dd1dd18276..18b4d393403ff 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MySqlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/MySqlSourceTests.java @@ -11,16 +11,16 @@ public final class MySqlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MySqlSource model = BinaryData.fromString( - "{\"type\":\"MySqlSource\",\"query\":\"datataevqafdhpkiiu\",\"queryTimeout\":\"datarobckelo\",\"additionalColumns\":\"datamrvdtqhrtnqssqy\",\"sourceRetryCount\":\"datapskitokphamefzz\",\"sourceRetryWait\":\"datajoauedmdpndouylf\",\"maxConcurrentConnections\":\"datagqin\",\"disableMetricsCollection\":\"datakxouknzh\",\"\":{\"mutdrrqqajhkl\":\"datassrsqzuknbtxt\",\"rtwqjf\":\"datatliuw\",\"ciwhumngi\":\"datayaqdswfno\"}}") + "{\"type\":\"MySqlSource\",\"query\":\"databgljcy\",\"queryTimeout\":\"datarzxipxhlxxkviyj\",\"additionalColumns\":\"dataqyejyavxgm\",\"sourceRetryCount\":\"datacnwxkqqxpnj\",\"sourceRetryWait\":\"datazdahvethn\",\"maxConcurrentConnections\":\"dataeggyqlvnhmuut\",\"disableMetricsCollection\":\"datawt\",\"\":{\"fbpfdsatrzqmt\":\"dataccmwsyfsgikgcbjc\"}}") .toObject(MySqlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MySqlSource model = new MySqlSource().withSourceRetryCount("datapskitokphamefzz") - .withSourceRetryWait("datajoauedmdpndouylf").withMaxConcurrentConnections("datagqin") - .withDisableMetricsCollection("datakxouknzh").withQueryTimeout("datarobckelo") - .withAdditionalColumns("datamrvdtqhrtnqssqy").withQuery("datataevqafdhpkiiu"); + MySqlSource model = new MySqlSource().withSourceRetryCount("datacnwxkqqxpnj") + .withSourceRetryWait("datazdahvethn").withMaxConcurrentConnections("dataeggyqlvnhmuut") + .withDisableMetricsCollection("datawt").withQueryTimeout("datarzxipxhlxxkviyj") + .withAdditionalColumns("dataqyejyavxgm").withQuery("databgljcy"); model = BinaryData.fromObject(model).toObject(MySqlSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaPartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaPartitionSettingsTests.java index 48ff5c832a689..aa61d20258429 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaPartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaPartitionSettingsTests.java @@ -11,14 +11,14 @@ public final class NetezzaPartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { NetezzaPartitionSettings model = BinaryData.fromString( - "{\"partitionColumnName\":\"datagfnaoehkgpkss\",\"partitionUpperBound\":\"datawkwxdgcfcfkyyrj\",\"partitionLowerBound\":\"datahslrbwwkbyw\"}") + "{\"partitionColumnName\":\"datavvrhoqyvqd\",\"partitionUpperBound\":\"dataruoduexhskh\",\"partitionLowerBound\":\"datalvo\"}") .toObject(NetezzaPartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NetezzaPartitionSettings model = new NetezzaPartitionSettings().withPartitionColumnName("datagfnaoehkgpkss") - .withPartitionUpperBound("datawkwxdgcfcfkyyrj").withPartitionLowerBound("datahslrbwwkbyw"); + NetezzaPartitionSettings model = new NetezzaPartitionSettings().withPartitionColumnName("datavvrhoqyvqd") + .withPartitionUpperBound("dataruoduexhskh").withPartitionLowerBound("datalvo"); model = BinaryData.fromObject(model).toObject(NetezzaPartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaSourceTests.java index 8d2c89b932765..e42da672f1c36 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NetezzaSourceTests.java @@ -12,18 +12,18 @@ public final class NetezzaSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { NetezzaSource model = BinaryData.fromString( - "{\"type\":\"NetezzaSource\",\"query\":\"dataanirlydsdmacydqa\",\"partitionOption\":\"datayvwxubgulyz\",\"partitionSettings\":{\"partitionColumnName\":\"datasxpprohuabduf\",\"partitionUpperBound\":\"dataso\",\"partitionLowerBound\":\"datapuaptpuwek\"},\"queryTimeout\":\"datal\",\"additionalColumns\":\"datamtcljopivtwxvcfc\",\"sourceRetryCount\":\"datakkcjjnq\",\"sourceRetryWait\":\"datajoayaj\",\"maxConcurrentConnections\":\"datahcxjmapgfbzbxeqz\",\"disableMetricsCollection\":\"datakfrhfafx\",\"\":{\"ulvue\":\"dataucmuax\",\"geqpa\":\"datasrxqscdbbwejrmk\",\"neteehndfpflf\":\"datallfscosfmeot\"}}") + "{\"type\":\"NetezzaSource\",\"query\":\"datacq\",\"partitionOption\":\"datazrflo\",\"partitionSettings\":{\"partitionColumnName\":\"datavem\",\"partitionUpperBound\":\"datayddfqfnftrrhh\",\"partitionLowerBound\":\"dataawizhanvcfx\"},\"queryTimeout\":\"datakdnfgctxultxhqqv\",\"additionalColumns\":\"datadyya\",\"sourceRetryCount\":\"dataxnepub\",\"sourceRetryWait\":\"datainfauytmqvsdyqyj\",\"maxConcurrentConnections\":\"datafotwmxedlcxmyxt\",\"disableMetricsCollection\":\"dataapoj\",\"\":{\"pfgdnq\":\"dataxantlpspi\"}}") .toObject(NetezzaSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NetezzaSource model = new NetezzaSource().withSourceRetryCount("datakkcjjnq").withSourceRetryWait("datajoayaj") - .withMaxConcurrentConnections("datahcxjmapgfbzbxeqz").withDisableMetricsCollection("datakfrhfafx") - .withQueryTimeout("datal").withAdditionalColumns("datamtcljopivtwxvcfc").withQuery("dataanirlydsdmacydqa") - .withPartitionOption("datayvwxubgulyz") - .withPartitionSettings(new NetezzaPartitionSettings().withPartitionColumnName("datasxpprohuabduf") - .withPartitionUpperBound("dataso").withPartitionLowerBound("datapuaptpuwek")); + NetezzaSource model = new NetezzaSource().withSourceRetryCount("dataxnepub") + .withSourceRetryWait("datainfauytmqvsdyqyj").withMaxConcurrentConnections("datafotwmxedlcxmyxt") + .withDisableMetricsCollection("dataapoj").withQueryTimeout("datakdnfgctxultxhqqv") + .withAdditionalColumns("datadyya").withQuery("datacq").withPartitionOption("datazrflo") + .withPartitionSettings(new NetezzaPartitionSettings().withPartitionColumnName("datavem") + .withPartitionUpperBound("datayddfqfnftrrhh").withPartitionLowerBound("dataawizhanvcfx")); model = BinaryData.fromObject(model).toObject(NetezzaSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NotebookParameterTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NotebookParameterTests.java index 1cfb007970427..ea9a655e2d66e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NotebookParameterTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/NotebookParameterTests.java @@ -12,16 +12,16 @@ public final class NotebookParameterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NotebookParameter model = BinaryData.fromString("{\"value\":\"datazlyoiyovc\",\"type\":\"string\"}") + NotebookParameter model = BinaryData.fromString("{\"value\":\"dataxyawtdsnvxhx\",\"type\":\"bool\"}") .toObject(NotebookParameter.class); - Assertions.assertEquals(NotebookParameterType.STRING, model.type()); + Assertions.assertEquals(NotebookParameterType.BOOL, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { NotebookParameter model - = new NotebookParameter().withValue("datazlyoiyovc").withType(NotebookParameterType.STRING); + = new NotebookParameter().withValue("dataxyawtdsnvxhx").withType(NotebookParameterType.BOOL); model = BinaryData.fromObject(model).toObject(NotebookParameter.class); - Assertions.assertEquals(NotebookParameterType.STRING, model.type()); + Assertions.assertEquals(NotebookParameterType.BOOL, model.type()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ODataSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ODataSourceTests.java index eb9851a2b4124..5010746838d78 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ODataSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ODataSourceTests.java @@ -11,16 +11,16 @@ public final class ODataSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ODataSource model = BinaryData.fromString( - "{\"type\":\"ODataSource\",\"query\":\"datandijzpvckh\",\"httpRequestTimeout\":\"datatmx\",\"additionalColumns\":\"datajssytd\",\"sourceRetryCount\":\"dataljtw\",\"sourceRetryWait\":\"datawcdxymnswxqtmhf\",\"maxConcurrentConnections\":\"dataanvrdtdlgxmgg\",\"disableMetricsCollection\":\"datatlhsz\",\"\":{\"pbiuwnxhqeljmf\":\"datayogcpwne\",\"dhg\":\"datalqdikuvjcls\",\"kiw\":\"datakflwnlpbawtpw\"}}") + "{\"type\":\"ODataSource\",\"query\":\"dataqyllcckgfo\",\"httpRequestTimeout\":\"datarbfyjmenq\",\"additionalColumns\":\"datajfxqtvsfsvqy\",\"sourceRetryCount\":\"dataaweixnoblazwhda\",\"sourceRetryWait\":\"dataixfdu\",\"maxConcurrentConnections\":\"datas\",\"disableMetricsCollection\":\"dataitpcsmax\",\"\":{\"a\":\"dataubhmiuxypvua\"}}") .toObject(ODataSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ODataSource model - = new ODataSource().withSourceRetryCount("dataljtw").withSourceRetryWait("datawcdxymnswxqtmhf") - .withMaxConcurrentConnections("dataanvrdtdlgxmgg").withDisableMetricsCollection("datatlhsz") - .withQuery("datandijzpvckh").withHttpRequestTimeout("datatmx").withAdditionalColumns("datajssytd"); + ODataSource model = new ODataSource().withSourceRetryCount("dataaweixnoblazwhda") + .withSourceRetryWait("dataixfdu").withMaxConcurrentConnections("datas") + .withDisableMetricsCollection("dataitpcsmax").withQuery("dataqyllcckgfo") + .withHttpRequestTimeout("datarbfyjmenq").withAdditionalColumns("datajfxqtvsfsvqy"); model = BinaryData.fromObject(model).toObject(ODataSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSinkTests.java index b35501a95d02e..ca8c8940af845 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSinkTests.java @@ -11,16 +11,16 @@ public final class OdbcSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OdbcSink model = BinaryData.fromString( - "{\"type\":\"OdbcSink\",\"preCopyScript\":\"datauug\",\"writeBatchSize\":\"datagdulvdenh\",\"writeBatchTimeout\":\"datavaimkoyrp\",\"sinkRetryCount\":\"datanljduwkbozlmr\",\"sinkRetryWait\":\"dataghvlvdjxbjqiab\",\"maxConcurrentConnections\":\"dataevvowiypyljzkx\",\"disableMetricsCollection\":\"datafyv\",\"\":{\"wemt\":\"datataqydcrjlhmneykx\"}}") + "{\"type\":\"OdbcSink\",\"preCopyScript\":\"datam\",\"writeBatchSize\":\"databfzaaiihyl\",\"writeBatchTimeout\":\"datazhlbpmplethek\",\"sinkRetryCount\":\"datanamtvooaace\",\"sinkRetryWait\":\"dataonsvjc\",\"maxConcurrentConnections\":\"datatytyrv\",\"disableMetricsCollection\":\"dataxvzywimmmmg\",\"\":{\"nvahpxdgy\":\"datavoytdtvkfq\",\"ygc\":\"dataowxcptxvxfwwv\",\"jri\":\"dataaztoias\"}}") .toObject(OdbcSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OdbcSink model = new OdbcSink().withWriteBatchSize("datagdulvdenh").withWriteBatchTimeout("datavaimkoyrp") - .withSinkRetryCount("datanljduwkbozlmr").withSinkRetryWait("dataghvlvdjxbjqiab") - .withMaxConcurrentConnections("dataevvowiypyljzkx").withDisableMetricsCollection("datafyv") - .withPreCopyScript("datauug"); + OdbcSink model = new OdbcSink().withWriteBatchSize("databfzaaiihyl").withWriteBatchTimeout("datazhlbpmplethek") + .withSinkRetryCount("datanamtvooaace").withSinkRetryWait("dataonsvjc") + .withMaxConcurrentConnections("datatytyrv").withDisableMetricsCollection("dataxvzywimmmmg") + .withPreCopyScript("datam"); model = BinaryData.fromObject(model).toObject(OdbcSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSourceTests.java index 2d92c8fe1d67d..e58e2e0e853c0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OdbcSourceTests.java @@ -11,16 +11,15 @@ public final class OdbcSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OdbcSource model = BinaryData.fromString( - "{\"type\":\"OdbcSource\",\"query\":\"datahqgvtzdxtwy\",\"queryTimeout\":\"datakwwdkkvdevdvkey\",\"additionalColumns\":\"datajchd\",\"sourceRetryCount\":\"dataxeiluexvmlg\",\"sourceRetryWait\":\"datapsqpfxjwta\",\"maxConcurrentConnections\":\"dataqkguchdyxrjjdj\",\"disableMetricsCollection\":\"dataiqtz\",\"\":{\"lwnxryyqtjcrpax\":\"dataddkujvqzcuqc\",\"dldm\":\"dataxlfxsetvdz\",\"iuxotbvflgkkiu\":\"datafqftywbbanzhd\",\"fobpyeo\":\"datahop\"}}") + "{\"type\":\"OdbcSource\",\"query\":\"datakpwltozxdzold\",\"queryTimeout\":\"datafnpn\",\"additionalColumns\":\"dataterjjuzarege\",\"sourceRetryCount\":\"dataozpudal\",\"sourceRetryWait\":\"datagdhdtt\",\"maxConcurrentConnections\":\"datakeculxvkuxvccpda\",\"disableMetricsCollection\":\"dataasi\",\"\":{\"ejh\":\"datayvvg\",\"ybneuzueikadhusg\":\"dataoswjwbhtawbc\"}}") .toObject(OdbcSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OdbcSource model - = new OdbcSource().withSourceRetryCount("dataxeiluexvmlg").withSourceRetryWait("datapsqpfxjwta") - .withMaxConcurrentConnections("dataqkguchdyxrjjdj").withDisableMetricsCollection("dataiqtz") - .withQueryTimeout("datakwwdkkvdevdvkey").withAdditionalColumns("datajchd").withQuery("datahqgvtzdxtwy"); + OdbcSource model = new OdbcSource().withSourceRetryCount("dataozpudal").withSourceRetryWait("datagdhdtt") + .withMaxConcurrentConnections("datakeculxvkuxvccpda").withDisableMetricsCollection("dataasi") + .withQueryTimeout("datafnpn").withAdditionalColumns("dataterjjuzarege").withQuery("datakpwltozxdzold"); model = BinaryData.fromObject(model).toObject(OdbcSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Office365SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Office365SourceTests.java index 999d78e813f04..72e4f58493542 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Office365SourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/Office365SourceTests.java @@ -11,17 +11,17 @@ public final class Office365SourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { Office365Source model = BinaryData.fromString( - "{\"type\":\"Office365Source\",\"allowedGroups\":\"dataxoavlwwpvjrnjwvc\",\"userScopeFilterUri\":\"datarqlceflgsndur\",\"dateFilterColumn\":\"dataozjwm\",\"startTime\":\"datadehjloz\",\"endTime\":\"datawokuxedpqwzzzi\",\"outputColumns\":\"databxjgxrhajr\",\"sourceRetryCount\":\"datacv\",\"sourceRetryWait\":\"datave\",\"maxConcurrentConnections\":\"datafmd\",\"disableMetricsCollection\":\"dataf\",\"\":{\"uclkbw\":\"datalp\"}}") + "{\"type\":\"Office365Source\",\"allowedGroups\":\"datacugchtwxifudlrxb\",\"userScopeFilterUri\":\"dataftpvgmqzitc\",\"dateFilterColumn\":\"datamlltas\",\"startTime\":\"dataqsf\",\"endTime\":\"datae\",\"outputColumns\":\"dataveg\",\"sourceRetryCount\":\"databmyvgmbirvv\",\"sourceRetryWait\":\"databqxisavktuxw\",\"maxConcurrentConnections\":\"datal\",\"disableMetricsCollection\":\"databridagwuvcdymoqv\",\"\":{\"agvuioxjwzt\":\"datarynziudmhedhryg\",\"cv\":\"datafotllf\",\"zz\":\"datafnbccffs\"}}") .toObject(Office365Source.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Office365Source model = new Office365Source().withSourceRetryCount("datacv").withSourceRetryWait("datave") - .withMaxConcurrentConnections("datafmd").withDisableMetricsCollection("dataf") - .withAllowedGroups("dataxoavlwwpvjrnjwvc").withUserScopeFilterUri("datarqlceflgsndur") - .withDateFilterColumn("dataozjwm").withStartTime("datadehjloz").withEndTime("datawokuxedpqwzzzi") - .withOutputColumns("databxjgxrhajr"); + Office365Source model = new Office365Source().withSourceRetryCount("databmyvgmbirvv") + .withSourceRetryWait("databqxisavktuxw").withMaxConcurrentConnections("datal") + .withDisableMetricsCollection("databridagwuvcdymoqv").withAllowedGroups("datacugchtwxifudlrxb") + .withUserScopeFilterUri("dataftpvgmqzitc").withDateFilterColumn("datamlltas").withStartTime("dataqsf") + .withEndTime("datae").withOutputColumns("dataveg"); model = BinaryData.fromObject(model).toObject(Office365Source.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OperationsListMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OperationsListMockTests.java index 85b250671f5bb..a75c747fedaba 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OperationsListMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OperationsListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"xilbsbhaqs\",\"origin\":\"wdcgdkwwulj\",\"display\":{\"description\":\"jgsxr\",\"provider\":\"bofmvauhn\",\"resource\":\"nswlfuukildlayt\",\"operation\":\"wvej\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{\"name\":\"dpssklm\",\"displayName\":\"aeallsxfzant\",\"blobDuration\":\"bzmoqxjynh\"},{\"name\":\"kgx\",\"displayName\":\"rmdpwpz\",\"blobDuration\":\"o\"},{\"name\":\"xlkloqpwsaqcr\",\"displayName\":\"kgjdn\",\"blobDuration\":\"wpvrwec\"},{\"name\":\"kiaognmanrzjprlq\",\"displayName\":\"wpejtszjbvjcvw\",\"blobDuration\":\"cvnowzcli\"}],\"metricSpecifications\":[{\"name\":\"hxwwhusrodr\",\"displayName\":\"ozafwqmocwkwmqr\",\"displayDescription\":\"ldacxofaqassu\",\"unit\":\"s\",\"aggregationType\":\"zxznctxoczns\",\"enableRegionalMdmAccount\":\"jzsjfcaqpkpv\",\"sourceMdmAccount\":\"irvxahxys\",\"sourceMdmNamespace\":\"xokfomakmiychduf\",\"availabilities\":[{},{}],\"dimensions\":[{},{},{}]},{\"name\":\"tgzcbvxyqprch\",\"displayName\":\"firaoytkkq\",\"displayDescription\":\"a\",\"unit\":\"mnvndzf\",\"aggregationType\":\"dsrfpihvijs\",\"enableRegionalMdmAccount\":\"kpocqboyjjf\",\"sourceMdmAccount\":\"njduyotqb\",\"sourceMdmNamespace\":\"tx\",\"availabilities\":[{},{}],\"dimensions\":[{},{},{},{}]},{\"name\":\"grixolbzjl\",\"displayName\":\"p\",\"displayDescription\":\"pjstco\",\"unit\":\"i\",\"aggregationType\":\"fybafenwvvxca\",\"enableRegionalMdmAccount\":\"oemcajjazo\",\"sourceMdmAccount\":\"whgnjhxydxic\",\"sourceMdmNamespace\":\"ilgtbslagtmkii\",\"availabilities\":[{},{},{},{}],\"dimensions\":[{},{},{},{}]}]}}}]}"; + = "{\"value\":[{\"name\":\"sdwcx\",\"origin\":\"gidyansnunvgqtvg\",\"display\":{\"description\":\"rbn\",\"provider\":\"dyvrdsvsjgt\",\"resource\":\"rmibm\",\"operation\":\"xspztirvzbmhmk\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{\"name\":\"ngpawy\",\"displayName\":\"bw\",\"blobDuration\":\"fz\"},{\"name\":\"qtpwhicnnanqzrml\",\"displayName\":\"ptmzsdwx\",\"blobDuration\":\"tpuzxinwj\"}],\"metricSpecifications\":[{\"name\":\"rhzd\",\"displayName\":\"eauhldq\",\"displayDescription\":\"kx\",\"unit\":\"nroewwrhvdwrowe\",\"aggregationType\":\"goijhc\",\"enableRegionalMdmAccount\":\"nprviivc\",\"sourceMdmAccount\":\"pc\",\"sourceMdmNamespace\":\"itvym\",\"availabilities\":[{}],\"dimensions\":[{}]}]}}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,33 +51,33 @@ public void testList() throws Exception { PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("xilbsbhaqs", response.iterator().next().name()); - Assertions.assertEquals("wdcgdkwwulj", response.iterator().next().origin()); - Assertions.assertEquals("jgsxr", response.iterator().next().display().description()); - Assertions.assertEquals("bofmvauhn", response.iterator().next().display().provider()); - Assertions.assertEquals("nswlfuukildlayt", response.iterator().next().display().resource()); - Assertions.assertEquals("wvej", response.iterator().next().display().operation()); - Assertions.assertEquals("dpssklm", + Assertions.assertEquals("sdwcx", response.iterator().next().name()); + Assertions.assertEquals("gidyansnunvgqtvg", response.iterator().next().origin()); + Assertions.assertEquals("rbn", response.iterator().next().display().description()); + Assertions.assertEquals("dyvrdsvsjgt", response.iterator().next().display().provider()); + Assertions.assertEquals("rmibm", response.iterator().next().display().resource()); + Assertions.assertEquals("xspztirvzbmhmk", response.iterator().next().display().operation()); + Assertions.assertEquals("ngpawy", response.iterator().next().serviceSpecification().logSpecifications().get(0).name()); - Assertions.assertEquals("aeallsxfzant", + Assertions.assertEquals("bw", response.iterator().next().serviceSpecification().logSpecifications().get(0).displayName()); - Assertions.assertEquals("bzmoqxjynh", + Assertions.assertEquals("fz", response.iterator().next().serviceSpecification().logSpecifications().get(0).blobDuration()); - Assertions.assertEquals("hxwwhusrodr", + Assertions.assertEquals("rhzd", response.iterator().next().serviceSpecification().metricSpecifications().get(0).name()); - Assertions.assertEquals("ozafwqmocwkwmqr", + Assertions.assertEquals("eauhldq", response.iterator().next().serviceSpecification().metricSpecifications().get(0).displayName()); - Assertions.assertEquals("ldacxofaqassu", + Assertions.assertEquals("kx", response.iterator().next().serviceSpecification().metricSpecifications().get(0).displayDescription()); - Assertions.assertEquals("s", + Assertions.assertEquals("nroewwrhvdwrowe", response.iterator().next().serviceSpecification().metricSpecifications().get(0).unit()); - Assertions.assertEquals("zxznctxoczns", + Assertions.assertEquals("goijhc", response.iterator().next().serviceSpecification().metricSpecifications().get(0).aggregationType()); - Assertions.assertEquals("jzsjfcaqpkpv", + Assertions.assertEquals("nprviivc", response.iterator().next().serviceSpecification().metricSpecifications().get(0).enableRegionalMdmAccount()); - Assertions.assertEquals("irvxahxys", + Assertions.assertEquals("pc", response.iterator().next().serviceSpecification().metricSpecifications().get(0).sourceMdmAccount()); - Assertions.assertEquals("xokfomakmiychduf", + Assertions.assertEquals("itvym", response.iterator().next().serviceSpecification().metricSpecifications().get(0).sourceMdmNamespace()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleCloudStorageReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleCloudStorageReadSettingsTests.java index 3b4b2b25021ae..0b48c6ecd9a6b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleCloudStorageReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleCloudStorageReadSettingsTests.java @@ -11,19 +11,19 @@ public final class OracleCloudStorageReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OracleCloudStorageReadSettings model = BinaryData.fromString( - "{\"type\":\"OracleCloudStorageReadSettings\",\"recursive\":\"dataljcauegymc\",\"wildcardFolderPath\":\"datasmnjitxu\",\"wildcardFileName\":\"datalbibwodayipgsh\",\"prefix\":\"dataoecmbyo\",\"fileListPath\":\"datavbvfchfuxuqp\",\"enablePartitionDiscovery\":\"dataebok\",\"partitionRootPath\":\"datashhhdixnzapz\",\"deleteFilesAfterCompletion\":\"datamstvz\",\"modifiedDatetimeStart\":\"datazvfywspajakj\",\"modifiedDatetimeEnd\":\"datapktbnmhxtmzzpa\",\"maxConcurrentConnections\":\"datasrvsbkn\",\"disableMetricsCollection\":\"datauytsaj\",\"\":{\"j\":\"datausnwic\",\"btegiw\":\"datawctlsohrtgpvv\"}}") + "{\"type\":\"OracleCloudStorageReadSettings\",\"recursive\":\"datarbjxewcscuveljf\",\"wildcardFolderPath\":\"datain\",\"wildcardFileName\":\"dataziztgddahymv\",\"prefix\":\"datajtdhmig\",\"fileListPath\":\"dataaoexgienylsijqyg\",\"enablePartitionDiscovery\":\"datashd\",\"partitionRootPath\":\"datahxv\",\"deleteFilesAfterCompletion\":\"datafdsafgkysymhuxs\",\"modifiedDatetimeStart\":\"datallbpegcetezaa\",\"modifiedDatetimeEnd\":\"dataszrbttz\",\"maxConcurrentConnections\":\"dataeyrw\",\"disableMetricsCollection\":\"datafgoyxxszpaiecurf\",\"\":{\"xurveekhsmulv\":\"datashfmgi\",\"iuwhcyckekm\":\"dataywoefkpuuu\",\"oycpotmaosongtbh\":\"datafipygt\",\"nrvwjxmwalh\":\"datahsqvubwwqgiyu\"}}") .toObject(OracleCloudStorageReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { OracleCloudStorageReadSettings model - = new OracleCloudStorageReadSettings().withMaxConcurrentConnections("datasrvsbkn") - .withDisableMetricsCollection("datauytsaj").withRecursive("dataljcauegymc") - .withWildcardFolderPath("datasmnjitxu").withWildcardFileName("datalbibwodayipgsh") - .withPrefix("dataoecmbyo").withFileListPath("datavbvfchfuxuqp").withEnablePartitionDiscovery("dataebok") - .withPartitionRootPath("datashhhdixnzapz").withDeleteFilesAfterCompletion("datamstvz") - .withModifiedDatetimeStart("datazvfywspajakj").withModifiedDatetimeEnd("datapktbnmhxtmzzpa"); + = new OracleCloudStorageReadSettings().withMaxConcurrentConnections("dataeyrw") + .withDisableMetricsCollection("datafgoyxxszpaiecurf").withRecursive("datarbjxewcscuveljf") + .withWildcardFolderPath("datain").withWildcardFileName("dataziztgddahymv").withPrefix("datajtdhmig") + .withFileListPath("dataaoexgienylsijqyg").withEnablePartitionDiscovery("datashd") + .withPartitionRootPath("datahxv").withDeleteFilesAfterCompletion("datafdsafgkysymhuxs") + .withModifiedDatetimeStart("datallbpegcetezaa").withModifiedDatetimeEnd("dataszrbttz"); model = BinaryData.fromObject(model).toObject(OracleCloudStorageReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OraclePartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OraclePartitionSettingsTests.java index 9a364bfbefda3..b53e304cd14c6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OraclePartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OraclePartitionSettingsTests.java @@ -11,15 +11,15 @@ public final class OraclePartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OraclePartitionSettings model = BinaryData.fromString( - "{\"partitionNames\":\"dataas\",\"partitionColumnName\":\"datamwsooq\",\"partitionUpperBound\":\"datavplmyzebvgh\",\"partitionLowerBound\":\"dataydehbvbexrbynnl\"}") + "{\"partitionNames\":\"datapomihgksqwzuosy\",\"partitionColumnName\":\"datalr\",\"partitionUpperBound\":\"datazudfar\",\"partitionLowerBound\":\"datayrdy\"}") .toObject(OraclePartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { OraclePartitionSettings model - = new OraclePartitionSettings().withPartitionNames("dataas").withPartitionColumnName("datamwsooq") - .withPartitionUpperBound("datavplmyzebvgh").withPartitionLowerBound("dataydehbvbexrbynnl"); + = new OraclePartitionSettings().withPartitionNames("datapomihgksqwzuosy").withPartitionColumnName("datalr") + .withPartitionUpperBound("datazudfar").withPartitionLowerBound("datayrdy"); model = BinaryData.fromObject(model).toObject(OraclePartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleServiceCloudSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleServiceCloudSourceTests.java index 4aeb2b8fd4fe2..bfaf1e946eae2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleServiceCloudSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleServiceCloudSourceTests.java @@ -11,16 +11,16 @@ public final class OracleServiceCloudSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OracleServiceCloudSource model = BinaryData.fromString( - "{\"type\":\"OracleServiceCloudSource\",\"query\":\"datadaeiepvjrmk\",\"queryTimeout\":\"dataxymkb\",\"additionalColumns\":\"datauawokrhhj\",\"sourceRetryCount\":\"datahrmuwvs\",\"sourceRetryWait\":\"dataru\",\"maxConcurrentConnections\":\"dataidtxmbnmjimggz\",\"disableMetricsCollection\":\"datagpldrnbjh\",\"\":{\"kvxerbddrtngdct\":\"datafbmsbetzu\",\"gvheqzlqevas\":\"datajzgzaeuu\"}}") + "{\"type\":\"OracleServiceCloudSource\",\"query\":\"datagdwpgmhqhvnexnw\",\"queryTimeout\":\"datawcxaqlym\",\"additionalColumns\":\"datazv\",\"sourceRetryCount\":\"datatecfyusfkcwfpo\",\"sourceRetryWait\":\"datalgkzgzxqwv\",\"maxConcurrentConnections\":\"datakqbgkssygdvll\",\"disableMetricsCollection\":\"datadfulvhpwpsxygrn\",\"\":{\"qgn\":\"dataxpsebazbtyrjr\",\"imtu\":\"datafzrra\",\"kve\":\"datajqzgyymyywhfdkj\",\"mccqljrnveq\":\"datasoztzdhzkbmzldpl\"}}") .toObject(OracleServiceCloudSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OracleServiceCloudSource model - = new OracleServiceCloudSource().withSourceRetryCount("datahrmuwvs").withSourceRetryWait("dataru") - .withMaxConcurrentConnections("dataidtxmbnmjimggz").withDisableMetricsCollection("datagpldrnbjh") - .withQueryTimeout("dataxymkb").withAdditionalColumns("datauawokrhhj").withQuery("datadaeiepvjrmk"); + OracleServiceCloudSource model = new OracleServiceCloudSource().withSourceRetryCount("datatecfyusfkcwfpo") + .withSourceRetryWait("datalgkzgzxqwv").withMaxConcurrentConnections("datakqbgkssygdvll") + .withDisableMetricsCollection("datadfulvhpwpsxygrn").withQueryTimeout("datawcxaqlym") + .withAdditionalColumns("datazv").withQuery("datagdwpgmhqhvnexnw"); model = BinaryData.fromObject(model).toObject(OracleServiceCloudSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSinkTests.java index 16fcb1263b0e4..f11f63fa17466 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSinkTests.java @@ -11,16 +11,16 @@ public final class OracleSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OracleSink model = BinaryData.fromString( - "{\"type\":\"OracleSink\",\"preCopyScript\":\"dataynljigjcnkadmwti\",\"writeBatchSize\":\"dataknewpnpun\",\"writeBatchTimeout\":\"datajbnhbxvvufq\",\"sinkRetryCount\":\"datajuosajqnsrcqdth\",\"sinkRetryWait\":\"dataqamdlcu\",\"maxConcurrentConnections\":\"datamrvryakc\",\"disableMetricsCollection\":\"datasnprda\",\"\":{\"bwobovexsnmwwhbm\":\"datagabbxexacgmtpk\",\"nkmkcimksfejzm\":\"datajlsztpygqwkdlx\",\"nb\":\"datavlbzmngxzp\",\"kjfkaoe\":\"dataovhddvtnbtvl\"}}") + "{\"type\":\"OracleSink\",\"preCopyScript\":\"datatmqquyco\",\"writeBatchSize\":\"datafyojfwvmsfns\",\"writeBatchTimeout\":\"datathdfqqzhqgmoe\",\"sinkRetryCount\":\"datanyugsasg\",\"sinkRetryWait\":\"datai\",\"maxConcurrentConnections\":\"dataxdxhxpqkcstyn\",\"disableMetricsCollection\":\"datafndxrofwctjhd\",\"\":{\"kpx\":\"dataldktiojitfa\",\"sorwtakny\":\"dataetdrcm\"}}") .toObject(OracleSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OracleSink model = new OracleSink().withWriteBatchSize("dataknewpnpun").withWriteBatchTimeout("datajbnhbxvvufq") - .withSinkRetryCount("datajuosajqnsrcqdth").withSinkRetryWait("dataqamdlcu") - .withMaxConcurrentConnections("datamrvryakc").withDisableMetricsCollection("datasnprda") - .withPreCopyScript("dataynljigjcnkadmwti"); + OracleSink model = new OracleSink().withWriteBatchSize("datafyojfwvmsfns") + .withWriteBatchTimeout("datathdfqqzhqgmoe").withSinkRetryCount("datanyugsasg").withSinkRetryWait("datai") + .withMaxConcurrentConnections("dataxdxhxpqkcstyn").withDisableMetricsCollection("datafndxrofwctjhd") + .withPreCopyScript("datatmqquyco"); model = BinaryData.fromObject(model).toObject(OracleSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSourceTests.java index a17057f7adf3c..f9c6a9fc11bfb 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OracleSourceTests.java @@ -12,19 +12,20 @@ public final class OracleSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OracleSource model = BinaryData.fromString( - "{\"type\":\"OracleSource\",\"oracleReaderQuery\":\"datatfkkiupmdaj\",\"queryTimeout\":\"datadvvzb\",\"partitionOption\":\"dataxxokif\",\"partitionSettings\":{\"partitionNames\":\"datawdyzse\",\"partitionColumnName\":\"datamvtqhn\",\"partitionUpperBound\":\"dataiju\",\"partitionLowerBound\":\"datarkqywybxgayomse\"},\"additionalColumns\":\"datacxl\",\"sourceRetryCount\":\"datavqf\",\"sourceRetryWait\":\"datasizxpo\",\"maxConcurrentConnections\":\"datasaploexhimvlocdx\",\"disableMetricsCollection\":\"datakobidhhipntrddy\",\"\":{\"dgmqscijlfulxg\":\"dataanv\"}}") + "{\"type\":\"OracleSource\",\"oracleReaderQuery\":\"datanknnqlty\",\"queryTimeout\":\"dataeyxmuwgnwxt\",\"partitionOption\":\"datawgen\",\"partitionSettings\":{\"partitionNames\":\"dataswcxlgzquqx\",\"partitionColumnName\":\"datasvqpifzavctywa\",\"partitionUpperBound\":\"dataaczprzrs\",\"partitionLowerBound\":\"datau\"},\"additionalColumns\":\"datanp\",\"sourceRetryCount\":\"dataqlanuhmsrnp\",\"sourceRetryWait\":\"dataaghoeqiwpdxpd\",\"maxConcurrentConnections\":\"dataoajqxyplhsto\",\"disableMetricsCollection\":\"datayb\",\"\":{\"gqjdoglec\":\"datasvpi\",\"iniidaxbesbwci\":\"datafgyivsiirxcxppqp\",\"uasjrs\":\"datayjch\"}}") .toObject(OracleSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OracleSource model = new OracleSource().withSourceRetryCount("datavqf").withSourceRetryWait("datasizxpo") - .withMaxConcurrentConnections("datasaploexhimvlocdx").withDisableMetricsCollection("datakobidhhipntrddy") - .withOracleReaderQuery("datatfkkiupmdaj").withQueryTimeout("datadvvzb").withPartitionOption("dataxxokif") - .withPartitionSettings( - new OraclePartitionSettings().withPartitionNames("datawdyzse").withPartitionColumnName("datamvtqhn") - .withPartitionUpperBound("dataiju").withPartitionLowerBound("datarkqywybxgayomse")) - .withAdditionalColumns("datacxl"); + OracleSource model = new OracleSource().withSourceRetryCount("dataqlanuhmsrnp") + .withSourceRetryWait("dataaghoeqiwpdxpd").withMaxConcurrentConnections("dataoajqxyplhsto") + .withDisableMetricsCollection("datayb").withOracleReaderQuery("datanknnqlty") + .withQueryTimeout("dataeyxmuwgnwxt").withPartitionOption("datawgen") + .withPartitionSettings(new OraclePartitionSettings().withPartitionNames("dataswcxlgzquqx") + .withPartitionColumnName("datasvqpifzavctywa").withPartitionUpperBound("dataaczprzrs") + .withPartitionLowerBound("datau")) + .withAdditionalColumns("datanp"); model = BinaryData.fromObject(model).toObject(OracleSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSinkTests.java index e9fc5b40541bc..b5c361f8efa99 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSinkTests.java @@ -17,22 +17,21 @@ public final class OrcSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OrcSink model = BinaryData.fromString( - "{\"type\":\"OrcSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataftqombdsgqxa\",\"disableMetricsCollection\":\"dataduo\",\"copyBehavior\":\"datazhdjbyf\",\"metadata\":[{\"name\":\"datajnhpyylekubiwvjv\",\"value\":\"datanyjqbwxpwjvfi\"}],\"\":{\"ttkbzwgj\":\"dataq\",\"jqsshu\":\"datapjbdqmnki\",\"uwome\":\"dataxjttnurkmerqza\",\"tce\":\"dataqvvtjwdlduvim\"}},\"formatSettings\":{\"type\":\"OrcWriteSettings\",\"maxRowsPerFile\":\"dataxoxtapafbjvbkj\",\"fileNamePrefix\":\"datazkc\",\"\":{\"pydnujgblski\":\"datavc\",\"puacajxdrgxpu\":\"datapr\",\"pzrycchqz\":\"datapzslmfr\"}},\"writeBatchSize\":\"datagesdzgszjhe\",\"writeBatchTimeout\":\"datamdhchi\",\"sinkRetryCount\":\"datasrhgpityeu\",\"sinkRetryWait\":\"dataysbme\",\"maxConcurrentConnections\":\"datag\",\"disableMetricsCollection\":\"datanlihbku\",\"\":{\"tvyzuyqzjfv\":\"dataywy\",\"scyzvv\":\"datanyyjvz\"}}") + "{\"type\":\"OrcSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataivrhjxdn\",\"disableMetricsCollection\":\"dataztf\",\"copyBehavior\":\"datauzvbrehdtqggza\",\"metadata\":[{\"name\":\"datarseiidfpw\",\"value\":\"databmxf\"},{\"name\":\"datazgolfensibqi\",\"value\":\"datapyjzv\"}],\"\":{\"vz\":\"datalfs\"}},\"formatSettings\":{\"type\":\"OrcWriteSettings\",\"maxRowsPerFile\":\"datavwr\",\"fileNamePrefix\":\"datagalywgq\",\"\":{\"csktvkwb\":\"databobheyxe\"}},\"writeBatchSize\":\"datakfvvxiikrja\",\"writeBatchTimeout\":\"datatvnmr\",\"sinkRetryCount\":\"datatypuotmkbofuh\",\"sinkRetryWait\":\"dataksgou\",\"maxConcurrentConnections\":\"dataegtn\",\"disableMetricsCollection\":\"datanotrgyyje\",\"\":{\"ufegbvvkuz\":\"datavjdunbaets\",\"mhzpurnp\":\"datagzrzubdtzsac\",\"albx\":\"datakbxkzcfios\",\"nluvcwuafbhxoa\":\"datad\"}}") .toObject(OrcSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OrcSink model = new OrcSink().withWriteBatchSize("datagesdzgszjhe").withWriteBatchTimeout("datamdhchi") - .withSinkRetryCount("datasrhgpityeu").withSinkRetryWait("dataysbme").withMaxConcurrentConnections("datag") - .withDisableMetricsCollection("datanlihbku") - .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataftqombdsgqxa") - .withDisableMetricsCollection("dataduo").withCopyBehavior("datazhdjbyf") - .withMetadata( - Arrays.asList(new MetadataItem().withName("datajnhpyylekubiwvjv").withValue("datanyjqbwxpwjvfi"))) + OrcSink model = new OrcSink().withWriteBatchSize("datakfvvxiikrja").withWriteBatchTimeout("datatvnmr") + .withSinkRetryCount("datatypuotmkbofuh").withSinkRetryWait("dataksgou") + .withMaxConcurrentConnections("dataegtn").withDisableMetricsCollection("datanotrgyyje") + .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataivrhjxdn") + .withDisableMetricsCollection("dataztf").withCopyBehavior("datauzvbrehdtqggza") + .withMetadata(Arrays.asList(new MetadataItem().withName("datarseiidfpw").withValue("databmxf"), + new MetadataItem().withName("datazgolfensibqi").withValue("datapyjzv"))) .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))) - .withFormatSettings( - new OrcWriteSettings().withMaxRowsPerFile("dataxoxtapafbjvbkj").withFileNamePrefix("datazkc")); + .withFormatSettings(new OrcWriteSettings().withMaxRowsPerFile("datavwr").withFileNamePrefix("datagalywgq")); model = BinaryData.fromObject(model).toObject(OrcSink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSourceTests.java index 4366e159643e7..4b6a24e43a960 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcSourceTests.java @@ -14,18 +14,18 @@ public final class OrcSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OrcSource model = BinaryData.fromString( - "{\"type\":\"OrcSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataz\",\"disableMetricsCollection\":\"datamcmlzmfetidne\",\"\":{\"gowdavpqyhax\":\"datajgwnmxc\",\"gkwpbnefabgt\":\"dataorzozf\",\"ugddycfyfau\":\"dataggoppmxcm\"}},\"additionalColumns\":\"datamzq\",\"sourceRetryCount\":\"datawkesxvzcxxf\",\"sourceRetryWait\":\"dataj\",\"maxConcurrentConnections\":\"dataqwbrzkmgyl\",\"disableMetricsCollection\":\"datacxmh\",\"\":{\"hjhrkfptiiommis\":\"dataecoiqwnqlizlc\",\"vbwawymahboi\":\"datafmbvmajcmpohjdvf\",\"gcjssqpk\":\"datadiuyqdjk\",\"zqwwttqjyiw\":\"databryhvshkvup\"}}") + "{\"type\":\"OrcSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataaaepxlxbofdchbo\",\"disableMetricsCollection\":\"datafsk\",\"\":{\"tutqjs\":\"datas\",\"oixtrnakytzcma\":\"dataj\",\"l\":\"datav\"}},\"additionalColumns\":\"dataaarqhpxwqhzsxgmg\",\"sourceRetryCount\":\"datagmtywivbu\",\"sourceRetryWait\":\"dataeedjnklvbrsxykwb\",\"maxConcurrentConnections\":\"datasd\",\"disableMetricsCollection\":\"datajitlqxpsnnnxhgd\",\"\":{\"dbg\":\"dataxyl\",\"zfzkhdnp\":\"dataicjkq\",\"vquasvywkbiek\":\"datamrxjdfkqlkaipf\"}}") .toObject(OrcSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OrcSource model = new OrcSource().withSourceRetryCount("datawkesxvzcxxf").withSourceRetryWait("dataj") - .withMaxConcurrentConnections("dataqwbrzkmgyl").withDisableMetricsCollection("datacxmh") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataz") - .withDisableMetricsCollection("datamcmlzmfetidne") - .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) - .withAdditionalColumns("datamzq"); + OrcSource model = new OrcSource().withSourceRetryCount("datagmtywivbu") + .withSourceRetryWait("dataeedjnklvbrsxykwb").withMaxConcurrentConnections("datasd") + .withDisableMetricsCollection("datajitlqxpsnnnxhgd") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataaaepxlxbofdchbo") + .withDisableMetricsCollection("datafsk").withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + .withAdditionalColumns("dataaarqhpxwqhzsxgmg"); model = BinaryData.fromObject(model).toObject(OrcSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcWriteSettingsTests.java index 732728ae850bb..992ce1c024b26 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/OrcWriteSettingsTests.java @@ -11,14 +11,14 @@ public final class OrcWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { OrcWriteSettings model = BinaryData.fromString( - "{\"type\":\"OrcWriteSettings\",\"maxRowsPerFile\":\"datacghvsmvvfpkym\",\"fileNamePrefix\":\"datavvwfao\",\"\":{\"armtuprqtcxqkoh\":\"databoawzplwghfgq\",\"kdejparjvsbo\":\"datapya\",\"lnbklhwri\":\"datafjbdyyxhjfzjbwmr\",\"ff\":\"dataruljbhg\"}}") + "{\"type\":\"OrcWriteSettings\",\"maxRowsPerFile\":\"datakinmxanjg\",\"fileNamePrefix\":\"datadhipgfx\",\"\":{\"cshmqxgjzslhopyq\":\"dataaazeqjnouuujlit\"}}") .toObject(OrcWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { OrcWriteSettings model - = new OrcWriteSettings().withMaxRowsPerFile("datacghvsmvvfpkym").withFileNamePrefix("datavvwfao"); + = new OrcWriteSettings().withMaxRowsPerFile("datakinmxanjg").withFileNamePrefix("datadhipgfx"); model = BinaryData.fromObject(model).toObject(OrcWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PackageStoreTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PackageStoreTests.java index ed3675093c48f..8e1dd61289254 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PackageStoreTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PackageStoreTests.java @@ -14,22 +14,23 @@ public final class PackageStoreTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PackageStore model = BinaryData.fromString( - "{\"name\":\"paseqcp\",\"packageStoreLinkedService\":{\"type\":\"IntegrationRuntimeReference\",\"referenceName\":\"reuwv\"}}") + "{\"name\":\"zkifqbxmnnid\",\"packageStoreLinkedService\":{\"type\":\"IntegrationRuntimeReference\",\"referenceName\":\"um\"}}") .toObject(PackageStore.class); - Assertions.assertEquals("paseqcp", model.name()); + Assertions.assertEquals("zkifqbxmnnid", model.name()); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE, model.packageStoreLinkedService().type()); - Assertions.assertEquals("reuwv", model.packageStoreLinkedService().referenceName()); + Assertions.assertEquals("um", model.packageStoreLinkedService().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PackageStore model = new PackageStore().withName("paseqcp").withPackageStoreLinkedService(new EntityReference() - .withType(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE).withReferenceName("reuwv")); + PackageStore model + = new PackageStore().withName("zkifqbxmnnid").withPackageStoreLinkedService(new EntityReference() + .withType(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE).withReferenceName("um")); model = BinaryData.fromObject(model).toObject(PackageStore.class); - Assertions.assertEquals("paseqcp", model.name()); + Assertions.assertEquals("zkifqbxmnnid", model.name()); Assertions.assertEquals(IntegrationRuntimeEntityReferenceType.INTEGRATION_RUNTIME_REFERENCE, model.packageStoreLinkedService().type()); - Assertions.assertEquals("reuwv", model.packageStoreLinkedService().referenceName()); + Assertions.assertEquals("um", model.packageStoreLinkedService().referenceName()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetReadSettingsTests.java index aa1eed63a4dc5..5dbd8133dab64 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetReadSettingsTests.java @@ -14,7 +14,7 @@ public final class ParquetReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ParquetReadSettings model = BinaryData.fromString( - "{\"type\":\"ParquetReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"pd\":\"dataoswqwbhlrzlgkc\",\"yowr\":\"datawerxfezlmzsekvsu\",\"renuvpkpltn\":\"datafvo\"}},\"\":{\"tpgqxtohruqtxi\":\"datayjatfpbxnr\",\"fcsbkjhoxtbsybp\":\"datarxeyzzynxbuvyglf\"}}") + "{\"type\":\"ParquetReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"owuicvj\":\"datamveyrcikedmoufju\",\"rilwkcgusvp\":\"datajszmleuqxh\"}},\"\":{\"vgwqiwodhasl\":\"dataulxxznfxdqqzi\"}}") .toObject(ParquetReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSinkTests.java index 09804400cbc7e..a3cfd641b64ab 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSinkTests.java @@ -17,24 +17,22 @@ public final class ParquetSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ParquetSink model = BinaryData.fromString( - "{\"type\":\"ParquetSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataqxelmmxwmpziycns\",\"disableMetricsCollection\":\"datawedkfofyfwpunw\",\"copyBehavior\":\"datautzlvx\",\"metadata\":[{\"name\":\"dataedzrjkrpormvddw\",\"value\":\"datazrdglk\"},{\"name\":\"datac\",\"value\":\"datakrldfoidyhcwc\"},{\"name\":\"datayuusexenywwkdxq\",\"value\":\"dataysxpaap\"},{\"name\":\"datahdqvcifwk\",\"value\":\"dataytvxrtocadtnmqr\"}],\"\":{\"cyai\":\"datacji\",\"r\":\"dataii\"}},\"formatSettings\":{\"type\":\"ParquetWriteSettings\",\"maxRowsPerFile\":\"datarkvluu\",\"fileNamePrefix\":\"datasu\",\"\":{\"trngj\":\"datarwm\",\"sfbkrtpu\":\"datac\"}},\"writeBatchSize\":\"datayeyqsiniejjb\",\"writeBatchTimeout\":\"datav\",\"sinkRetryCount\":\"datakwrvtlbb\",\"sinkRetryWait\":\"databdtmrijt\",\"maxConcurrentConnections\":\"dataf\",\"disableMetricsCollection\":\"databpvizuuluilgmova\",\"\":{\"jxgqsb\":\"datajs\"}}") + "{\"type\":\"ParquetSink\",\"storeSettings\":{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"dataajqnsrcqd\",\"disableMetricsCollection\":\"datamlqamd\",\"copyBehavior\":\"dataukdmrv\",\"metadata\":[{\"name\":\"datacclsnprdartq\",\"value\":\"databbxexacgmtpkx\"}],\"\":{\"bmujlsztpygq\":\"databovexsnmww\",\"sn\":\"datakdl\",\"mksfejzmyvlbz\":\"datamkc\"}},\"formatSettings\":{\"type\":\"ParquetWriteSettings\",\"maxRowsPerFile\":\"dataxzpdnb\",\"fileNamePrefix\":\"datavhddvtnbtv\",\"\":{\"myizdglzzaufi\":\"datajfkaoew\",\"ykng\":\"datawvyxy\"}},\"writeBatchSize\":\"datajgpyvjg\",\"writeBatchTimeout\":\"datayjnh\",\"sinkRetryCount\":\"databhwrncxwzuer\",\"sinkRetryWait\":\"datapa\",\"maxConcurrentConnections\":\"datapiniopbfkmfbru\",\"disableMetricsCollection\":\"datayl\",\"\":{\"fsol\":\"dataovnlb\",\"lnhxr\":\"datarqu\",\"l\":\"datajshicvrmwbgpc\",\"pboaevtxi\":\"databxppvpgsrfshkjg\"}}") .toObject(ParquetSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ParquetSink model = new ParquetSink().withWriteBatchSize("datayeyqsiniejjb").withWriteBatchTimeout("datav") - .withSinkRetryCount("datakwrvtlbb").withSinkRetryWait("databdtmrijt").withMaxConcurrentConnections("dataf") - .withDisableMetricsCollection("databpvizuuluilgmova") - .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataqxelmmxwmpziycns") - .withDisableMetricsCollection("datawedkfofyfwpunw").withCopyBehavior("datautzlvx") - .withMetadata(Arrays.asList(new MetadataItem().withName("dataedzrjkrpormvddw").withValue("datazrdglk"), - new MetadataItem().withName("datac").withValue("datakrldfoidyhcwc"), - new MetadataItem().withName("datayuusexenywwkdxq").withValue("dataysxpaap"), - new MetadataItem().withName("datahdqvcifwk").withValue("dataytvxrtocadtnmqr"))) + ParquetSink model = new ParquetSink().withWriteBatchSize("datajgpyvjg").withWriteBatchTimeout("datayjnh") + .withSinkRetryCount("databhwrncxwzuer").withSinkRetryWait("datapa") + .withMaxConcurrentConnections("datapiniopbfkmfbru").withDisableMetricsCollection("datayl") + .withStoreSettings(new StoreWriteSettings().withMaxConcurrentConnections("dataajqnsrcqd") + .withDisableMetricsCollection("datamlqamd").withCopyBehavior("dataukdmrv") + .withMetadata( + Arrays.asList(new MetadataItem().withName("datacclsnprdartq").withValue("databbxexacgmtpkx"))) .withAdditionalProperties(mapOf("type", "StoreWriteSettings"))) .withFormatSettings( - new ParquetWriteSettings().withMaxRowsPerFile("datarkvluu").withFileNamePrefix("datasu")); + new ParquetWriteSettings().withMaxRowsPerFile("dataxzpdnb").withFileNamePrefix("datavhddvtnbtv")); model = BinaryData.fromObject(model).toObject(ParquetSink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSourceTests.java index aac7c487ee457..b817a9686662e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetSourceTests.java @@ -16,21 +16,20 @@ public final class ParquetSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ParquetSource model = BinaryData.fromString( - "{\"type\":\"ParquetSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datagadkrknyyjngdf\",\"disableMetricsCollection\":\"datacjfqmyfgwbuxq\",\"\":{\"xzyj\":\"datagbqsv\"}},\"formatSettings\":{\"type\":\"ParquetReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"xrsky\":\"datamqutgxdlznfokc\"}},\"\":{\"teikktret\":\"datapp\",\"nvb\":\"datatsygzjplaxxfnrlt\"}},\"additionalColumns\":\"dataotghxkrrpmgdoli\",\"sourceRetryCount\":\"datazsglavdtttyd\",\"sourceRetryWait\":\"dataomz\",\"maxConcurrentConnections\":\"datakjq\",\"disableMetricsCollection\":\"datahbypwmveyrciked\",\"\":{\"sz\":\"datafjuqowuicvjy\",\"lwkcgu\":\"dataleuqxhmr\"}}") + "{\"type\":\"ParquetSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataqnkoskflnjaysr\",\"disableMetricsCollection\":\"dataevmbhemrhb\",\"\":{\"botbrepefrli\":\"datasspwwe\",\"zgcscootfsgilwis\":\"dataeocyarvs\",\"tust\":\"dataxzpz\"}},\"formatSettings\":{\"type\":\"ParquetReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"zbirb\":\"dataogknocshmpcjqt\"}},\"\":{\"rhccdgunsjssreo\":\"datahfjqpxydpamctzm\",\"ppbghyekggoaw\":\"datasgkouenpgkxyr\"}},\"additionalColumns\":\"datavu\",\"sourceRetryCount\":\"dataqmwqsgy\",\"sourceRetryWait\":\"datanostn\",\"maxConcurrentConnections\":\"datavrpkizj\",\"disableMetricsCollection\":\"datagdsur\",\"\":{\"voxshxumuuyblolr\":\"databcirkbkqp\"}}") .toObject(ParquetSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ParquetSource model - = new ParquetSource().withSourceRetryCount("datazsglavdtttyd").withSourceRetryWait("dataomz") - .withMaxConcurrentConnections("datakjq").withDisableMetricsCollection("datahbypwmveyrciked") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datagadkrknyyjngdf") - .withDisableMetricsCollection("datacjfqmyfgwbuxq") - .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) - .withFormatSettings(new ParquetReadSettings().withCompressionProperties( - new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings")))) - .withAdditionalColumns("dataotghxkrrpmgdoli"); + ParquetSource model = new ParquetSource().withSourceRetryCount("dataqmwqsgy").withSourceRetryWait("datanostn") + .withMaxConcurrentConnections("datavrpkizj").withDisableMetricsCollection("datagdsur") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("dataqnkoskflnjaysr") + .withDisableMetricsCollection("dataevmbhemrhb") + .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + .withFormatSettings(new ParquetReadSettings().withCompressionProperties( + new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings")))) + .withAdditionalColumns("datavu"); model = BinaryData.fromObject(model).toObject(ParquetSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetWriteSettingsTests.java index 6341b19111c1d..a08157217ab5c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ParquetWriteSettingsTests.java @@ -11,14 +11,14 @@ public final class ParquetWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ParquetWriteSettings model = BinaryData.fromString( - "{\"type\":\"ParquetWriteSettings\",\"maxRowsPerFile\":\"dataryoiaob\",\"fileNamePrefix\":\"datacdlccqumvb\",\"\":{\"inxxj\":\"dataiibxol\",\"kwxcaagzlq\":\"dataixr\",\"hacfiyrywfry\":\"datasgzgsgzlbunm\"}}") + "{\"type\":\"ParquetWriteSettings\",\"maxRowsPerFile\":\"datapbxspvkcng\",\"fileNamePrefix\":\"datacnwn\",\"\":{\"bklkhhjx\":\"dataitlam\",\"hqzvwznwcqoapdtj\":\"datawqzsyetbff\"}}") .toObject(ParquetWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ParquetWriteSettings model - = new ParquetWriteSettings().withMaxRowsPerFile("dataryoiaob").withFileNamePrefix("datacdlccqumvb"); + = new ParquetWriteSettings().withMaxRowsPerFile("datapbxspvkcng").withFileNamePrefix("datacnwn"); model = BinaryData.fromObject(model).toObject(ParquetWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalObjectDatasetTests.java index 28fc9008d1a97..e998eef43e2e0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalObjectDatasetTests.java @@ -19,35 +19,34 @@ public final class PaypalObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PaypalObjectDataset model = BinaryData.fromString( - "{\"type\":\"PaypalObject\",\"typeProperties\":{\"tableName\":\"datape\"},\"description\":\"j\",\"structure\":\"datasdjuzmuijtickzo\",\"schema\":\"datau\",\"linkedServiceName\":{\"referenceName\":\"p\",\"parameters\":{\"opq\":\"datahhboigzx\",\"atzw\":\"datarzhtocjzfppexu\",\"it\":\"datakjwg\"}},\"parameters\":{\"bjoypplodaqrbkpo\":{\"type\":\"Array\",\"defaultValue\":\"datambmswskb\"},\"crqaxlmbrtvtgolm\":{\"type\":\"SecureString\",\"defaultValue\":\"datasobggva\"},\"yxhxj\":{\"type\":\"Array\",\"defaultValue\":\"datagtla\"},\"bqnjcsbozvcdq\":{\"type\":\"SecureString\",\"defaultValue\":\"datasxaqqjhdfhfa\"}},\"annotations\":[\"dataydvwr\",\"databivyw\"],\"folder\":{\"name\":\"njuvtz\"},\"\":{\"tjfdzfmnpbdrc\":\"datadlxbaeyocpkv\",\"vdtuoamqobqeh\":\"databjxnnnoztn\",\"f\":\"datapshtisy\",\"zeb\":\"dataoctrzjwnzwc\"}}") + "{\"type\":\"PaypalObject\",\"typeProperties\":{\"tableName\":\"databb\"},\"description\":\"yp\",\"structure\":\"dataodaq\",\"schema\":\"datakp\",\"linkedServiceName\":{\"referenceName\":\"zf\",\"parameters\":{\"valcrqaxlmbrtvtg\":\"databg\"}},\"parameters\":{\"voysxa\":{\"type\":\"Object\",\"defaultValue\":\"datalgtlayyxhx\"},\"njc\":{\"type\":\"SecureString\",\"defaultValue\":\"datahdfhfaob\"},\"ydvwr\":{\"type\":\"SecureString\",\"defaultValue\":\"datazvcdqws\"},\"dlxbaeyocpkv\":{\"type\":\"Int\",\"defaultValue\":\"datavywotjnjuvtzij\"}},\"annotations\":[\"datafdz\",\"datamnpbdrcibj\",\"datannno\",\"datatnhvdtu\"],\"folder\":{\"name\":\"qobqehspshtisy\"},\"\":{\"zeb\":\"dataoctrzjwnzwc\",\"lmlnxrcatkuh\":\"databvwdxgyypmxq\",\"gdkvviilyeshoxf\":\"datak\"}}") .toObject(PaypalObjectDataset.class); - Assertions.assertEquals("j", model.description()); - Assertions.assertEquals("p", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("bjoypplodaqrbkpo").type()); - Assertions.assertEquals("njuvtz", model.folder().name()); + Assertions.assertEquals("yp", model.description()); + Assertions.assertEquals("zf", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("voysxa").type()); + Assertions.assertEquals("qobqehspshtisy", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PaypalObjectDataset model - = new PaypalObjectDataset().withDescription("j").withStructure("datasdjuzmuijtickzo").withSchema("datau") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("p") - .withParameters(mapOf("opq", "datahhboigzx", "atzw", "datarzhtocjzfppexu", "it", "datakjwg"))) - .withParameters(mapOf("bjoypplodaqrbkpo", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datambmswskb"), - "crqaxlmbrtvtgolm", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datasobggva"), - "yxhxj", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datagtla"), - "bqnjcsbozvcdq", - new ParameterSpecification().withType(ParameterType.SECURE_STRING) - .withDefaultValue("datasxaqqjhdfhfa"))) - .withAnnotations(Arrays.asList("dataydvwr", "databivyw")) - .withFolder(new DatasetFolder().withName("njuvtz")).withTableName("datape"); + PaypalObjectDataset model = new PaypalObjectDataset().withDescription("yp").withStructure("dataodaq") + .withSchema("datakp") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("zf") + .withParameters(mapOf("valcrqaxlmbrtvtg", "databg"))) + .withParameters(mapOf("voysxa", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datalgtlayyxhx"), "njc", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datahdfhfaob"), + "ydvwr", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datazvcdqws"), + "dlxbaeyocpkv", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datavywotjnjuvtzij"))) + .withAnnotations(Arrays.asList("datafdz", "datamnpbdrcibj", "datannno", "datatnhvdtu")) + .withFolder(new DatasetFolder().withName("qobqehspshtisy")).withTableName("databb"); model = BinaryData.fromObject(model).toObject(PaypalObjectDataset.class); - Assertions.assertEquals("j", model.description()); - Assertions.assertEquals("p", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("bjoypplodaqrbkpo").type()); - Assertions.assertEquals("njuvtz", model.folder().name()); + Assertions.assertEquals("yp", model.description()); + Assertions.assertEquals("zf", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("voysxa").type()); + Assertions.assertEquals("qobqehspshtisy", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalSourceTests.java index de8cc56e64104..5aa4c901a073c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PaypalSourceTests.java @@ -11,16 +11,16 @@ public final class PaypalSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PaypalSource model = BinaryData.fromString( - "{\"type\":\"PaypalSource\",\"query\":\"datacrejt\",\"queryTimeout\":\"dataqqoz\",\"additionalColumns\":\"datasbpqwnmfjktdvdh\",\"sourceRetryCount\":\"dataztaluuup\",\"sourceRetryWait\":\"dataaoatzvajwvxh\",\"maxConcurrentConnections\":\"datamotulhilmazgp\",\"disableMetricsCollection\":\"datarppsoeo\",\"\":{\"ln\":\"datawtye\",\"dxsgwd\":\"datagqeplyos\"}}") + "{\"type\":\"PaypalSource\",\"query\":\"datajjprd\",\"queryTimeout\":\"datablonlhtgexwjhicu\",\"additionalColumns\":\"dataavimxnhylwogtvl\",\"sourceRetryCount\":\"datagd\",\"sourceRetryWait\":\"datat\",\"maxConcurrentConnections\":\"datadxlfn\",\"disableMetricsCollection\":\"dataclkmggnzlfyxaiaf\",\"\":{\"uoayapzzcxkuusba\":\"dataxekfvycvhw\",\"yak\":\"datacassqeybdnz\",\"zkicxtumqinawct\":\"datarkohfqm\",\"kjnpe\":\"dataarboxaluoadmcv\"}}") .toObject(PaypalSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PaypalSource model - = new PaypalSource().withSourceRetryCount("dataztaluuup").withSourceRetryWait("dataaoatzvajwvxh") - .withMaxConcurrentConnections("datamotulhilmazgp").withDisableMetricsCollection("datarppsoeo") - .withQueryTimeout("dataqqoz").withAdditionalColumns("datasbpqwnmfjktdvdh").withQuery("datacrejt"); + PaypalSource model = new PaypalSource().withSourceRetryCount("datagd").withSourceRetryWait("datat") + .withMaxConcurrentConnections("datadxlfn").withDisableMetricsCollection("dataclkmggnzlfyxaiaf") + .withQueryTimeout("datablonlhtgexwjhicu").withAdditionalColumns("dataavimxnhylwogtvl") + .withQuery("datajjprd"); model = BinaryData.fromObject(model).toObject(PaypalSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixDatasetTypePropertiesTests.java index ea317d2a574b6..19fd2c28f66f8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixDatasetTypePropertiesTests.java @@ -10,15 +10,15 @@ public final class PhoenixDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PhoenixDatasetTypeProperties model = BinaryData - .fromString("{\"tableName\":\"datagvzlzjsbkpcu\",\"table\":\"dataaziydpoknse\",\"schema\":\"datambdqra\"}") - .toObject(PhoenixDatasetTypeProperties.class); + PhoenixDatasetTypeProperties model + = BinaryData.fromString("{\"tableName\":\"datambdqra\",\"table\":\"datad\",\"schema\":\"dataluobbva\"}") + .toObject(PhoenixDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PhoenixDatasetTypeProperties model = new PhoenixDatasetTypeProperties().withTableName("datagvzlzjsbkpcu") - .withTable("dataaziydpoknse").withSchema("datambdqra"); + PhoenixDatasetTypeProperties model = new PhoenixDatasetTypeProperties().withTableName("datambdqra") + .withTable("datad").withSchema("dataluobbva"); model = BinaryData.fromObject(model).toObject(PhoenixDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixObjectDatasetTests.java index 02570827c15bb..be3b7200c4872 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixObjectDatasetTests.java @@ -19,34 +19,35 @@ public final class PhoenixObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PhoenixObjectDataset model = BinaryData.fromString( - "{\"type\":\"PhoenixObject\",\"typeProperties\":{\"tableName\":\"datawdxgyypm\",\"table\":\"datazlmln\",\"schema\":\"datacatkuhs\"},\"description\":\"gdkvviilyeshoxf\",\"structure\":\"datajdmu\",\"schema\":\"datausx\",\"linkedServiceName\":{\"referenceName\":\"ugozwplxzgzumnot\",\"parameters\":{\"giq\":\"datakkbyg\",\"izonzsur\":\"datawyshybbnhtt\",\"asfzhzzcarc\":\"dataco\",\"nhwsgns\":\"datauoxyipdthjf\"}},\"parameters\":{\"fbbach\":{\"type\":\"Array\",\"defaultValue\":\"datalfchnufssjg\"},\"kbuxlepghcnuqhq\":{\"type\":\"Array\",\"defaultValue\":\"datazzunfnbphceei\"},\"fscl\":{\"type\":\"String\",\"defaultValue\":\"datawt\"}},\"annotations\":[\"datagygn\",\"databfytnhdnihuzzjuz\",\"datawgbzdtorbi\",\"datanyfzdpxct\"],\"folder\":{\"name\":\"rxdtej\"},\"\":{\"pjhltylyuud\":\"datazrlwtidcnzalgmp\"}}") + "{\"type\":\"PhoenixObject\",\"typeProperties\":{\"tableName\":\"datadmupbusxyug\",\"table\":\"datawplx\",\"schema\":\"datazu\"},\"description\":\"otiixkkbygbgiq\",\"structure\":\"datayshybb\",\"schema\":\"datattyizonzsurqcoja\",\"linkedServiceName\":{\"referenceName\":\"fzhzzcarciuo\",\"parameters\":{\"hwsgnsputfe\":\"datapdthjfv\",\"hnu\":\"dataf\",\"chxxc\":\"datassjgbfbb\",\"bphceeivkbuxlep\":\"datazunf\"}},\"parameters\":{\"fscl\":{\"type\":\"Object\",\"defaultValue\":\"dataqhqpvtwt\"},\"bfytnhdnihuzzjuz\":{\"type\":\"Bool\",\"defaultValue\":\"datagygn\"},\"tsucrxdtejobjz\":{\"type\":\"Bool\",\"defaultValue\":\"databzdtorbiwnyfzdpx\"}},\"annotations\":[\"datat\",\"datad\",\"datanzalgm\",\"dataupjhltyl\"],\"folder\":{\"name\":\"dvbgvzlzjs\"},\"\":{\"ydpoknse\":\"datacutzaz\"}}") .toObject(PhoenixObjectDataset.class); - Assertions.assertEquals("gdkvviilyeshoxf", model.description()); - Assertions.assertEquals("ugozwplxzgzumnot", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("fbbach").type()); - Assertions.assertEquals("rxdtej", model.folder().name()); + Assertions.assertEquals("otiixkkbygbgiq", model.description()); + Assertions.assertEquals("fzhzzcarciuo", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("fscl").type()); + Assertions.assertEquals("dvbgvzlzjs", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PhoenixObjectDataset model = new PhoenixObjectDataset().withDescription("gdkvviilyeshoxf") - .withStructure("datajdmu").withSchema("datausx") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ugozwplxzgzumnot") - .withParameters(mapOf("giq", "datakkbyg", "izonzsur", "datawyshybbnhtt", "asfzhzzcarc", "dataco", - "nhwsgns", "datauoxyipdthjf"))) - .withParameters(mapOf("fbbach", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datalfchnufssjg"), - "kbuxlepghcnuqhq", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datazzunfnbphceei"), - "fscl", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datawt"))) - .withAnnotations(Arrays.asList("datagygn", "databfytnhdnihuzzjuz", "datawgbzdtorbi", "datanyfzdpxct")) - .withFolder(new DatasetFolder().withName("rxdtej")).withTableName("datawdxgyypm").withTable("datazlmln") - .withSchemaTypePropertiesSchema("datacatkuhs"); + PhoenixObjectDataset model = new PhoenixObjectDataset().withDescription("otiixkkbygbgiq") + .withStructure("datayshybb").withSchema("datattyizonzsurqcoja") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fzhzzcarciuo") + .withParameters(mapOf("hwsgnsputfe", "datapdthjfv", "hnu", "dataf", "chxxc", "datassjgbfbb", + "bphceeivkbuxlep", "datazunf"))) + .withParameters(mapOf("fscl", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataqhqpvtwt"), + "bfytnhdnihuzzjuz", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datagygn"), + "tsucrxdtejobjz", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("databzdtorbiwnyfzdpx"))) + .withAnnotations(Arrays.asList("datat", "datad", "datanzalgm", "dataupjhltyl")) + .withFolder(new DatasetFolder().withName("dvbgvzlzjs")).withTableName("datadmupbusxyug") + .withTable("datawplx").withSchemaTypePropertiesSchema("datazu"); model = BinaryData.fromObject(model).toObject(PhoenixObjectDataset.class); - Assertions.assertEquals("gdkvviilyeshoxf", model.description()); - Assertions.assertEquals("ugozwplxzgzumnot", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("fbbach").type()); - Assertions.assertEquals("rxdtej", model.folder().name()); + Assertions.assertEquals("otiixkkbygbgiq", model.description()); + Assertions.assertEquals("fzhzzcarciuo", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("fscl").type()); + Assertions.assertEquals("dvbgvzlzjs", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixSourceTests.java index eb55622a72e9b..63ba1bbc95a01 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PhoenixSourceTests.java @@ -11,16 +11,16 @@ public final class PhoenixSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PhoenixSource model = BinaryData.fromString( - "{\"type\":\"PhoenixSource\",\"query\":\"dataxjlvvvzpjj\",\"queryTimeout\":\"dataintgkveogeld\",\"additionalColumns\":\"datab\",\"sourceRetryCount\":\"databii\",\"sourceRetryWait\":\"databkxiujaagfeiwuux\",\"maxConcurrentConnections\":\"datamzmsivqeg\",\"disableMetricsCollection\":\"datafzbrha\",\"\":{\"ttsz\":\"datatkrsjspziiev\"}}") + "{\"type\":\"PhoenixSource\",\"query\":\"datawevlohuahl\",\"queryTimeout\":\"datacboxgpmmz\",\"additionalColumns\":\"dataoyllxc\",\"sourceRetryCount\":\"datahzylspz\",\"sourceRetryWait\":\"datarhynlbtr\",\"maxConcurrentConnections\":\"dataecvag\",\"disableMetricsCollection\":\"datarhadg\",\"\":{\"hiafbhzdjv\":\"datarasxeomjqqhbkxi\",\"ggbpdpzgvq\":\"datayrzi\",\"lvxilaytj\":\"dataznxzaliicrutyhm\",\"ghqdlj\":\"datawfqzwn\"}}") .toObject(PhoenixSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { PhoenixSource model - = new PhoenixSource().withSourceRetryCount("databii").withSourceRetryWait("databkxiujaagfeiwuux") - .withMaxConcurrentConnections("datamzmsivqeg").withDisableMetricsCollection("datafzbrha") - .withQueryTimeout("dataintgkveogeld").withAdditionalColumns("datab").withQuery("dataxjlvvvzpjj"); + = new PhoenixSource().withSourceRetryCount("datahzylspz").withSourceRetryWait("datarhynlbtr") + .withMaxConcurrentConnections("dataecvag").withDisableMetricsCollection("datarhadg") + .withQueryTimeout("datacboxgpmmz").withAdditionalColumns("dataoyllxc").withQuery("datawevlohuahl"); model = BinaryData.fromObject(model).toObject(PhoenixSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineExternalComputeScalePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineExternalComputeScalePropertiesTests.java index 135f595783aac..c8a03545d0593 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineExternalComputeScalePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineExternalComputeScalePropertiesTests.java @@ -14,22 +14,22 @@ public final class PipelineExternalComputeScalePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PipelineExternalComputeScaleProperties model = BinaryData.fromString( - "{\"timeToLive\":672832981,\"numberOfPipelineNodes\":929027209,\"numberOfExternalNodes\":964675941,\"\":{\"urp\":\"datakyncyzjndfeemx\",\"ajpcroxpp\":\"dataivypmwd\",\"yufdmzucxvzvwlx\":\"dataqimyhxnpdggllyd\",\"ur\":\"datajnstz\"}}") + "{\"timeToLive\":1340072376,\"numberOfPipelineNodes\":536369061,\"numberOfExternalNodes\":1689303593,\"\":{\"toiwfsz\":\"dataipdjxyotgvraxh\",\"etsluqfgk\":\"datarlkosjwr\",\"imioixviobuwbnge\":\"datad\",\"gqamhbmggnqxnex\":\"datawhdq\"}}") .toObject(PipelineExternalComputeScaleProperties.class); - Assertions.assertEquals(672832981, model.timeToLive()); - Assertions.assertEquals(929027209, model.numberOfPipelineNodes()); - Assertions.assertEquals(964675941, model.numberOfExternalNodes()); + Assertions.assertEquals(1340072376, model.timeToLive()); + Assertions.assertEquals(536369061, model.numberOfPipelineNodes()); + Assertions.assertEquals(1689303593, model.numberOfExternalNodes()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { PipelineExternalComputeScaleProperties model = new PipelineExternalComputeScaleProperties() - .withTimeToLive(672832981).withNumberOfPipelineNodes(929027209).withNumberOfExternalNodes(964675941) + .withTimeToLive(1340072376).withNumberOfPipelineNodes(536369061).withNumberOfExternalNodes(1689303593) .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(PipelineExternalComputeScaleProperties.class); - Assertions.assertEquals(672832981, model.timeToLive()); - Assertions.assertEquals(929027209, model.numberOfPipelineNodes()); - Assertions.assertEquals(964675941, model.numberOfExternalNodes()); + Assertions.assertEquals(1340072376, model.timeToLive()); + Assertions.assertEquals(536369061, model.numberOfPipelineNodes()); + Assertions.assertEquals(1689303593, model.numberOfExternalNodes()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsCancelWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsCancelWithResponseMockTests.java index 51c837469c69f..55035be1ef27f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsCancelWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsCancelWithResponseMockTests.java @@ -45,7 +45,8 @@ public void testCancelWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.pipelineRuns().cancelWithResponse("tkze", "poxdwoxp", "qtzbs", true, com.azure.core.util.Context.NONE); + manager.pipelineRuns().cancelWithResponse("vrfkxiixnxx", "vyizya", "xwegij", true, + com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsGetWithResponseMockTests.java index ac99d05aaf5fb..d3b206f1b7cd7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelineRunsGetWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"runId\":\"ohspvbwjmt\",\"runGroupId\":\"vuecmdmcor\",\"isLatest\":false,\"pipelineName\":\"akigrlmigln\",\"parameters\":{\"uwsfebhvkkpd\":\"qefd\",\"lioagvijr\":\"ek\"},\"runDimensions\":{\"exyrrxbeufzbdd\":\"tejljdrerzj\",\"iz\":\"xf\",\"db\":\"kzmegcjsneybpqot\",\"dvuptret\":\"ljs\"},\"invokedBy\":{\"name\":\"upwtstpbi\",\"id\":\"bsegcogy\",\"invokedByType\":\"ekaajuwkxbg\",\"pipelineName\":\"dfcbjsyo\",\"pipelineRunId\":\"ojvztqragqc\"},\"lastUpdated\":\"2021-10-13T21:56:24Z\",\"runStart\":\"2021-04-05T04:56:19Z\",\"runEnd\":\"2021-05-05T19:26:45Z\",\"durationInMs\":1180738178,\"status\":\"qgyhgzqkkwz\",\"message\":\"bwwops\",\"\":{\"tztwvhgkmxar\":\"datasekrzcnlqstmik\"}}"; + = "{\"runId\":\"qoetckm\",\"runGroupId\":\"nsi\",\"isLatest\":false,\"pipelineName\":\"xhwvzdvujmukadz\",\"parameters\":{\"aecdcvhxwegdsmn\":\"xfqddadezn\"},\"runDimensions\":{\"ps\":\"vxel\",\"myftvejxmy\":\"bqpddypw\"},\"invokedBy\":{\"name\":\"ahhpnbvzdfyxjb\",\"id\":\"vnxwduu\",\"invokedByType\":\"vb\",\"pipelineName\":\"yedrkgrtda\",\"pipelineRunId\":\"itoimtar\"},\"lastUpdated\":\"2021-06-05T12:01:29Z\",\"runStart\":\"2021-11-06T10:40:45Z\",\"runEnd\":\"2021-08-24T21:24:23Z\",\"durationInMs\":1463714124,\"status\":\"pcduyhhz\",\"message\":\"z\",\"\":{\"jci\":\"datawkclloejshfcuzz\"}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,7 +48,7 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PipelineRun response = manager.pipelineRuns() - .getWithResponse("onflknmzaih", "pj", "cseuh", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("hczqjoovypsgugh", "okbwzpxlxbv", "hkabeo", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateOrUpdateWithResponseMockTests.java index fd9fdc59379de..6e894411d2430 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateOrUpdateWithResponseMockTests.java @@ -46,7 +46,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"description\":\"kem\",\"activities\":[{\"type\":\"Activity\",\"name\":\"gezy\",\"description\":\"hiplhwplyv\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"fhxkifjvila\",\"dependencyConditions\":[]},{\"activity\":\"opw\",\"dependencyConditions\":[]},{\"activity\":\"dbewbxaufowh\",\"dependencyConditions\":[]},{\"activity\":\"d\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"aktuge\",\"value\":\"datahqdoctgno\"},{\"name\":\"qw\",\"value\":\"datawtwjzzyi\"},{\"name\":\"bbkly\",\"value\":\"dataelvhxutctakkdjus\"},{\"name\":\"sfjwty\",\"value\":\"datavknbucjy\"}],\"\":{\"owusaddmji\":\"datagdlfgmuojnikwz\",\"yswvpavutiszwy\":\"dataftdfmzlgjcepx\"}},{\"type\":\"Activity\",\"name\":\"lehagbjmwe\",\"description\":\"mpzamq\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"phtdwhm\",\"dependencyConditions\":[]},{\"activity\":\"xhvspum\",\"dependencyConditions\":[]},{\"activity\":\"kmymspatpvebxes\",\"dependencyConditions\":[]},{\"activity\":\"cryn\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"yowwrbxejdwh\",\"value\":\"datashlks\"},{\"name\":\"fpqrusxyugid\",\"value\":\"datagsj\"}],\"\":{\"tkqqdqxslbrttlw\":\"datat\"}},{\"type\":\"Activity\",\"name\":\"sktzrdxxsbbdo\",\"description\":\"jnbcdnjexcyh\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"ubxfb\",\"dependencyConditions\":[]},{\"activity\":\"qgimwivqphddewfa\",\"dependencyConditions\":[]},{\"activity\":\"ajpojz\",\"dependencyConditions\":[]},{\"activity\":\"zzak\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"wnhpcfs\",\"value\":\"datadzivvyky\"},{\"name\":\"avevne\",\"value\":\"datapyzuuqvi\"},{\"name\":\"vryxwzxjxotp\",\"value\":\"dataiwm\"}],\"\":{\"kinsry\":\"dataxwokmdaihgig\"}}],\"parameters\":{\"pvadyxjcckhgstoh\":{\"type\":\"Float\",\"defaultValue\":\"datap\"}},\"variables\":{\"spocutpnyz\":{\"type\":\"Array\",\"defaultValue\":\"datazlmvwufhduniqum\"},\"vtmmvqliqzfjlp\":{\"type\":\"Array\",\"defaultValue\":\"datakd\"}},\"concurrency\":865632766,\"annotations\":[\"datapuavxidytjmk\",\"dataxzgopckm\",\"dataagfbreyvr\"],\"runDimensions\":{\"urlywxjvs\":\"dataikwqtlgfry\",\"qmikljczxotblx\":\"datazchysqypt\"},\"folder\":{\"name\":\"f\"},\"policy\":{\"elapsedTimeMetric\":{\"duration\":\"datasxrxvjvwkzaq\"}}},\"name\":\"qyijyzhmfuksqiqz\",\"type\":\"xunldbkuqcnjiw\",\"etag\":\"nbjk\",\"\":{\"yja\":\"databmfnjuzvwwowy\",\"xjwig\":\"dataehqg\",\"oxpay\":\"datacaim\",\"ger\":\"datazqgsaegaah\"},\"id\":\"hgrgiu\"}"; + = "{\"properties\":{\"description\":\"howvnzwhypjpy\",\"activities\":[{\"type\":\"Activity\",\"name\":\"ptjpsbdche\",\"description\":\"c\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"szdwwkgbzmcprtan\",\"dependencyConditions\":[]},{\"activity\":\"gehbrhwkaatjsseb\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"sjlilpicc\",\"value\":\"dataegtwstqgcjvklnrz\"},{\"name\":\"afxoyddushvyj\",\"value\":\"datahy\"}],\"\":{\"dwbnou\":\"dataxzdw\",\"m\":\"datayznyeghm\"}},{\"type\":\"Activity\",\"name\":\"xdnckgdcszz\",\"description\":\"dfdzleazvldeqmf\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"wseftljzqfpfkd\",\"dependencyConditions\":[]},{\"activity\":\"bezaxi\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"pjxtobeq\",\"value\":\"datazcadoqijfll\"},{\"name\":\"muzeolcgqjtvpalk\",\"value\":\"datawvg\"}],\"\":{\"jpauic\":\"datatdswjtuqw\",\"ugebqhbbqody\":\"datajae\",\"aa\":\"datavpcoi\"}},{\"type\":\"Activity\",\"name\":\"vaecwwdqgooabhfr\",\"description\":\"p\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"fevwcxzxvgf\",\"dependencyConditions\":[]},{\"activity\":\"zc\",\"dependencyConditions\":[]},{\"activity\":\"qwqujpug\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"alhwbypvpds\",\"value\":\"dataycjuxabpuphg\"}],\"\":{\"gzwtl\":\"dataggkkjcizrstannmj\",\"parpsrjsghuokjwv\":\"datapzshgsidkz\",\"klyvzske\":\"dataacwdukhzu\"}},{\"type\":\"Activity\",\"name\":\"eva\",\"description\":\"brdrtjak\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"yzqhbuuldztv\",\"dependencyConditions\":[]},{\"activity\":\"mvxkrxgaiddg\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"hiqwuwxrcydmkyo\",\"value\":\"datajc\"},{\"name\":\"sbygm\",\"value\":\"datanxeyfkrcmxtbwolz\"},{\"name\":\"kjaqlszlymyqpw\",\"value\":\"databuwqmicukwmz\"},{\"name\":\"hcvrfq\",\"value\":\"datambuvtiwrmcym\"}],\"\":{\"dgtkedvxhqhp\":\"datakkaztu\",\"spafurttshruj\":\"datan\",\"mr\":\"datayxal\",\"apzingxbkcirio\":\"dataiina\"}}],\"parameters\":{\"hwiezcfxzjdpl\":{\"type\":\"Float\",\"defaultValue\":\"datauvg\"},\"qnftdl\":{\"type\":\"Float\",\"defaultValue\":\"datao\"}},\"variables\":{\"ffymrzoupipdkgpt\":{\"type\":\"String\",\"defaultValue\":\"datavlgahpztvl\"},\"zrbqpzgsr\":{\"type\":\"Array\",\"defaultValue\":\"datamscrtpznychw\"},\"nvwaxmeyjimf\":{\"type\":\"Array\",\"defaultValue\":\"datafyffytw\"},\"wufuvtypvwfllri\":{\"type\":\"String\",\"defaultValue\":\"datacqvuqwzajdxmai\"}},\"concurrency\":2036809837,\"annotations\":[\"datacgnphengk\",\"datahkekxohqvqp\",\"datazoqtvmkj\"],\"runDimensions\":{\"yvbyagqipr\":\"datalakvhgefvpc\",\"ijkwzjlk\":\"dataqdtcibb\",\"timtf\":\"datamoexughztr\"},\"folder\":{\"name\":\"lq\"},\"policy\":{\"elapsedTimeMetric\":{\"duration\":\"datangfcbldpeforx\"}}},\"name\":\"pmzkdisrgykrcj\",\"type\":\"ivnfdovwkjeguvi\",\"etag\":\"xii\",\"\":{\"oylpmeccbblg\":\"datahfrb\",\"xcrxuyorhrtihzw\":\"datalej\"},\"id\":\"flwlmh\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -63,72 +63,75 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PipelineResource response = manager.pipelines().define("ceghp").withExistingFactory("yysycey", "vmlxhy") - .withDescription("mroyygtetmpw") + PipelineResource response = manager.pipelines().define("ger").withExistingFactory("mkoxpay", "zqgsaegaah") + .withDescription("hgrgiu") .withActivities(Arrays.asList( - new Activity().withName("qd").withDescription("xdfkdwkhmnoecfjw").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + new Activity().withName("ykvo").withDescription("jixbrdamdnebko").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("aihepjscseu") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("ohspvbwjmt").withValue("dataavue"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("yotejljdrerzjwex").withDescription("r").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("dszwrfdjxl").withDependencyConditions(Arrays.asList()) + new ActivityDependency().withActivity("ddcxfuizo").withDependencyConditions(Arrays.asList()) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jsmdrecrrbkm").withDependencyConditions(Arrays.asList()) + new ActivityDependency().withActivity("zme").withDependencyConditions(Arrays.asList()) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("qfi").withDependencyConditions(Arrays.asList()) + new ActivityDependency().withActivity("cjsneybpqot").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("db").withDependencyConditions(Arrays.asList()) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("oapxdmxwet").withValue("dataji"), - new UserProperty().withName("ekqlyctdx").withValue("datargq"), - new UserProperty().withName("fzedf").withValue("datafqwllzbtqzjmi"), - new UserProperty().withName("dcsdfbkiwumrf").withValue("datajrptsjecqwdosbs"))) + .withUserProperties(Arrays.asList(new UserProperty().withName("s").withValue("datadvuptret"), + new UserProperty().withName("auupwt").withValue("datatpbi"), + new UserProperty().withName("ab").withValue("dataegcogyctekaaju"), + new UserProperty().withName("kxbgfed").withValue("datacbjsyorsojv"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("blrqeqcdikcqc").withDescription("dtfthnjxid").withState(ActivityState.ACTIVE) + new Activity().withName("s").withDescription("ikbtz").withState(ActivityState.ACTIVE) .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("xgxtccmq").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kuqrrzuegin").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("lnteoapszxqnjxv") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("xarqtkzeopoxd") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("pn").withValue("datatzbswf"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("predcl").withDescription("lvuzqsv").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("a") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("cqm").withValue("datajihmcgzwoijtlhx"), - new UserProperty().withName("sxxra").withValue("datagaicgqgafkrtsa"), - new UserProperty().withName("agvq").withValue("datatcr"), - new UserProperty().withName("fxqhnw").withValue("dataujrnfdqlzggvo"))) + Arrays.asList(new UserProperty().withName("ymdrbmffcryy").withValue("datakwwhscubgwz"), + new UserProperty().withName("anplzbzcgzhd").withValue("datavk"), + new UserProperty().withName("zvhfog").withValue("dataoocnseoq"))) .withAdditionalProperties(mapOf("type", "Activity")))) - .withParameters(mapOf("cpbtvgiokz", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("databfryortbresx"), - "ubggjdluwbmwu", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datavswdwawomkzussgj"), - "amym", - new ParameterSpecification().withType(ParameterType.SECURE_STRING) - .withDefaultValue("datakzcmfibbozkptw"), - "h", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datawckucsopffhqxhtc"))) - .withVariables(mapOf("krfvl", - new VariableSpecification().withType(VariableType.BOOL).withDefaultValue("datavomcjpjrxvsggaub"), - "gpvirozlfcc", - new VariableSpecification().withType(VariableType.ARRAY).withDefaultValue("datajeoenpihtgigaee"), "p", - new VariableSpecification().withType(VariableType.STRING).withDefaultValue("dataqixxxgltqldlhhqp"))) - .withConcurrency(1217739571) - .withAnnotations(Arrays.asList("datacajhnnbp", "datanogyvpfyjlfnjmwb", "dataoqhy", "datarpwkvz")) - .withRunDimensions(mapOf("o", "datadlhc", "jqz", "datakmpxtfcrugitjnw", "eaqnbkcqoyqmbu", "datagq")) - .withFolder(new PipelineFolder().withName("bzhczyhtjqtzl")) + .withParameters(mapOf("zguu", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("dataedifwdrr"), + "dlxqjshyyrcr", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataoohzifbbsncorini"), + "lt", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataqsfaurmqpkgwf"))) + .withVariables(mapOf("pxsy", + new VariableSpecification().withType(VariableType.ARRAY).withDefaultValue("datahhxlibdnpeamsl"), + "aulsmqohvcvv", new VariableSpecification().withType(VariableType.BOOL).withDefaultValue("datafvwrdy"))) + .withConcurrency(1111207450) + .withAnnotations(Arrays.asList("datagrphoabhkyas", "dataccwievjndvaf", "datacvn", "datayxlcgycvcspcfx")) + .withRunDimensions(mapOf("afmbxtncxbyurut", "dataqioq", "uv", "dataa", "vgjvumdznblkofd", + "datauweqbeygnetuvs", "fiwaklflwqdjzb", "datalrtlhpfu")) + .withFolder(new PipelineFolder().withName("byks")) .withPolicy(new PipelinePolicy() - .withElapsedTimeMetric(new PipelineElapsedTimeMetricPolicy().withDuration("dataanceowvqqzx"))) - .withIfMatch("crcxamgvipzvvrf").create(); + .withElapsedTimeMetric(new PipelineElapsedTimeMetricPolicy().withDuration("datayprrix"))) + .withIfMatch("ccotgqgevie").create(); - Assertions.assertEquals("hgrgiu", response.id()); - Assertions.assertEquals("kem", response.description()); - Assertions.assertEquals("gezy", response.activities().get(0).name()); - Assertions.assertEquals("hiplhwplyv", response.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, response.activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, response.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("fhxkifjvila", response.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("aktuge", response.activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals(ParameterType.FLOAT, response.parameters().get("pvadyxjcckhgstoh").type()); - Assertions.assertEquals(VariableType.ARRAY, response.variables().get("spocutpnyz").type()); - Assertions.assertEquals(865632766, response.concurrency()); - Assertions.assertEquals("f", response.folder().name()); + Assertions.assertEquals("flwlmh", response.id()); + Assertions.assertEquals("howvnzwhypjpy", response.description()); + Assertions.assertEquals("ptjpsbdche", response.activities().get(0).name()); + Assertions.assertEquals("c", response.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, response.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, response.activities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("szdwwkgbzmcprtan", response.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("sjlilpicc", response.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals(ParameterType.FLOAT, response.parameters().get("hwiezcfxzjdpl").type()); + Assertions.assertEquals(VariableType.STRING, response.variables().get("ffymrzoupipdkgpt").type()); + Assertions.assertEquals(2036809837, response.concurrency()); + Assertions.assertEquals("lq", response.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateRunWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateRunWithResponseMockTests.java index 2b3d7f5c07972..0fee93afab148 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateRunWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesCreateRunWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testCreateRunWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"runId\":\"rc\"}"; + String responseStr = "{\"runId\":\"hxjwiggca\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,12 +49,10 @@ public void testCreateRunWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - CreateRunResponse response = manager.pipelines() - .createRunWithResponse("xivktdvwmefjpo", "llyvbvx", "rltrztrloyrjvr", "yr", true, "frsyckqwefmqht", true, - mapOf("hlnbawffrzgeobz", "datavfncp", "grojpnxz", "dataxzraihl"), com.azure.core.util.Context.NONE) - .getValue(); + CreateRunResponse response = manager.pipelines().createRunWithResponse("ksqiqzmgxunld", "k", "qcnjiwzqn", + "jkst", true, "mfnjuzvww", false, mapOf("jagehq", "datay"), com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("rc", response.runId()); + Assertions.assertEquals("hxjwiggca", response.runId()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesDeleteWithResponseMockTests.java index b49f44e670649..e1f570af51061 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.pipelines().deleteWithResponse("tisk", "xeclwl", "odrdnfmxomupdqp", com.azure.core.util.Context.NONE); + manager.pipelines().deleteWithResponse("xvjvwk", "aqqkq", "ijyzhmf", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesGetWithResponseMockTests.java index a6411fcf119bb..db59ad6346e2c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesGetWithResponseMockTests.java @@ -35,7 +35,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"description\":\"kwgsqosrpcxw\",\"activities\":[{\"type\":\"Activity\",\"name\":\"ljktujfcr\",\"description\":\"fryket\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"yifkg\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"njfjyppixfubkfj\",\"value\":\"datavl\"},{\"name\":\"yetyrnhispac\",\"value\":\"datav\"},{\"name\":\"nlypspn\",\"value\":\"datalo\"},{\"name\":\"oangrl\",\"value\":\"dataaixrbwbkrsmkeiun\"}],\"\":{\"tzzmcrmhhfcai\":\"datamedzfox\",\"zufdmsbvyg\":\"dataxuiy\"}},{\"type\":\"Activity\",\"name\":\"fljv\",\"description\":\"qkoecozfauhn\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"grzcjpkzmhaxtwji\",\"dependencyConditions\":[]},{\"activity\":\"lf\",\"dependencyConditions\":[]},{\"activity\":\"kmynm\",\"dependencyConditions\":[]},{\"activity\":\"mglbxoeghordccpk\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"c\",\"value\":\"datalvqlccaiphsart\"},{\"name\":\"iqq\",\"value\":\"datadgyshpvva\"}],\"\":{\"r\":\"datar\",\"oxweuo\":\"datahwdxdkvqqtfjj\"}}],\"parameters\":{\"xndheocj\":{\"type\":\"Object\",\"defaultValue\":\"datae\"}},\"variables\":{\"utve\":{\"type\":\"Array\",\"defaultValue\":\"datanan\"},\"vdqcohjwzynb\":{\"type\":\"Array\",\"defaultValue\":\"datahfwq\"}},\"concurrency\":865354799,\"annotations\":[\"databhlhyqgfimllrad\",\"datawpudvuphizztklk\",\"datahdeeht\"],\"runDimensions\":{\"fopcn\":\"dataefkphskkivy\"},\"folder\":{\"name\":\"ulpwwmxwlwcurk\"},\"policy\":{\"elapsedTimeMetric\":{\"duration\":\"dataazep\"}}},\"name\":\"uk\",\"type\":\"dcvrux\",\"etag\":\"nugujiwidunsvsjo\",\"\":{\"lt\":\"dataxs\",\"qro\":\"dataghykqxr\",\"fgkqudxvjrndbi\":\"dataknenpybuskvj\",\"oeuidhp\":\"dataqqrkkgawna\"},\"id\":\"xyitnzpfdoet\"}"; + = "{\"properties\":{\"description\":\"bu\",\"activities\":[{\"type\":\"Activity\",\"name\":\"btwgdlfgmuojn\",\"description\":\"wzlowusaddmj\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"zlgjcepxay\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"pavutiszwycle\",\"value\":\"dataagbjmwe\"}],\"\":{\"dwhmwxhvsp\":\"datapzamqbcitlyyph\",\"ucrynsqxyowwr\":\"datamokmymspatpvebxe\"}}],\"parameters\":{\"ugidk\":{\"type\":\"String\",\"defaultValue\":\"datawhrshlkswfpqrusx\"}},\"variables\":{\"kqqdqxslbrtt\":{\"type\":\"Array\",\"defaultValue\":\"datadtr\"},\"kjnbc\":{\"type\":\"Bool\",\"defaultValue\":\"datasktzrdxxsbbdo\"},\"cyhsbvair\":{\"type\":\"String\",\"defaultValue\":\"datae\"},\"wfajajpojzazzaks\":{\"type\":\"Array\",\"defaultValue\":\"dataxfbmqgimwivqphdd\"}},\"concurrency\":1823251390,\"annotations\":[\"datapcfsqdzi\",\"datavykysavevnerpyzu\",\"dataqvinvryxwzxjxotp\",\"dataiwm\"],\"runDimensions\":{\"kinsry\":\"dataxwokmdaihgig\"},\"folder\":{\"name\":\"bhpfpvadyxjcckhg\"},\"policy\":{\"elapsedTimeMetric\":{\"duration\":\"datazvrqbzlmv\"}}},\"name\":\"fhduniqumpspo\",\"type\":\"tpnyzytgkd\",\"etag\":\"tmmv\",\"\":{\"xidytj\":\"dataqzfjlprljilpua\",\"yvrlycikwqtlg\":\"datakfxzgopckmmagfbr\",\"w\":\"dataryourl\",\"tsqmikljczxotb\":\"datajvsqzchysqy\"},\"id\":\"lpqfxyywsx\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -53,20 +53,20 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PipelineResource response = manager.pipelines() - .getWithResponse("uun", "yokftdlwezplnzvr", "gjweelkviki", "z", com.azure.core.util.Context.NONE) + .getWithResponse("omqwfwtwjzzyiib", "klya", "lvhxutctakkd", "usasfjwtyvv", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("xyitnzpfdoet", response.id()); - Assertions.assertEquals("kwgsqosrpcxw", response.description()); - Assertions.assertEquals("ljktujfcr", response.activities().get(0).name()); - Assertions.assertEquals("fryket", response.activities().get(0).description()); + Assertions.assertEquals("lpqfxyywsx", response.id()); + Assertions.assertEquals("bu", response.description()); + Assertions.assertEquals("btwgdlfgmuojn", response.activities().get(0).name()); + Assertions.assertEquals("wzlowusaddmj", response.activities().get(0).description()); Assertions.assertEquals(ActivityState.INACTIVE, response.activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, response.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("yifkg", response.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("njfjyppixfubkfj", response.activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals(ParameterType.OBJECT, response.parameters().get("xndheocj").type()); - Assertions.assertEquals(VariableType.ARRAY, response.variables().get("utve").type()); - Assertions.assertEquals(865354799, response.concurrency()); - Assertions.assertEquals("ulpwwmxwlwcurk", response.folder().name()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, response.activities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("zlgjcepxay", response.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("pavutiszwycle", response.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals(ParameterType.STRING, response.parameters().get("ugidk").type()); + Assertions.assertEquals(VariableType.ARRAY, response.variables().get("kqqdqxslbrtt").type()); + Assertions.assertEquals(1823251390, response.concurrency()); + Assertions.assertEquals("bhpfpvadyxjcckhg", response.folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesListByFactoryMockTests.java index 82510bb9a056f..8d5ed35c54aca 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PipelinesListByFactoryMockTests.java @@ -36,7 +36,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"description\":\"qjtoeaugwosrywp\",\"activities\":[{\"type\":\"Activity\",\"name\":\"l\",\"description\":\"iafsizdec\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"lwquxrrju\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"phgsd\",\"value\":\"datatwfmvpsvwwtncvn\"},{\"name\":\"qwgn\",\"value\":\"datalicovvdcobfnb\"}],\"\":{\"xjfkpuszsjay\":\"dataoijxqgfmftrvv\",\"xwnircmodws\":\"datal\",\"wxwkjambfsxsr\":\"datahzlamdqgav\"}},{\"type\":\"Activity\",\"name\":\"jfapiodsn\",\"description\":\"ezwjq\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"uxo\",\"dependencyConditions\":[]},{\"activity\":\"mjxqintjhvcoro\",\"dependencyConditions\":[]},{\"activity\":\"mqudzi\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"k\",\"value\":\"dataea\"},{\"name\":\"kcygyqgvofhpg\",\"value\":\"datajibkwwyfsqga\"},{\"name\":\"svcxazvrmu\",\"value\":\"datasjegohpwnrm\"}],\"\":{\"udmefsxmdml\":\"datatknbruszq\",\"g\":\"datawesixpwfvt\",\"bhcjhin\":\"datameqirxwkomjsfkd\",\"wkzuaxsrmadakj\":\"datanwpivfplbajqecn\"}},{\"type\":\"Activity\",\"name\":\"yp\",\"description\":\"yvobkkekld\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"hotwq\",\"dependencyConditions\":[]},{\"activity\":\"gvrzlimz\",\"dependencyConditions\":[]},{\"activity\":\"utmsmdibzvytem\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"ykcxuvdcw\",\"value\":\"datanzf\"},{\"name\":\"eghn\",\"value\":\"dataqwjwwhs\"},{\"name\":\"jqxlbclvpgbuty\",\"value\":\"datasravsscblsxmsc\"}],\"\":{\"uzclfbvvuyoilni\":\"datad\",\"gxsfeslxwlmx\":\"datawxwaquuvbb\",\"bi\":\"datao\",\"dvicdqufjahuc\":\"dataktblom\"}}],\"parameters\":{\"trfdipss\":{\"type\":\"Object\",\"defaultValue\":\"datacklthsuasnxdhlov\"}},\"variables\":{\"zs\":{\"type\":\"Bool\",\"defaultValue\":\"databydryysvex\"},\"birltzyuahnl\":{\"type\":\"String\",\"defaultValue\":\"datankjjwtyn\"}},\"concurrency\":585093789,\"annotations\":[\"datassdtysnlx\",\"dataqmzezfhfjjj\",\"datacxtzk\"],\"runDimensions\":{\"h\":\"dataosceukqio\"},\"folder\":{\"name\":\"qkkacw\"},\"policy\":{\"elapsedTimeMetric\":{\"duration\":\"dataxk\"}}},\"name\":\"mwbrvslforlaudem\",\"type\":\"pdnusuj\",\"etag\":\"bbg\",\"\":{\"gmihxjpflzpuib\":\"dataknhadqfynrd\"},\"id\":\"lrewfrmqbmcmgeza\"}]}"; + = "{\"value\":[{\"properties\":{\"description\":\"jvrxyr\",\"activities\":[{\"type\":\"Activity\",\"name\":\"rsyckq\",\"description\":\"fmqhtr\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"hhl\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"wffrzgeobzmxzrai\",\"value\":\"datalz\"},{\"name\":\"r\",\"value\":\"datajpnxz\"},{\"name\":\"rc\",\"value\":\"datayysycey\"}],\"\":{\"ghppy\":\"datalxhymc\",\"ygtetmpw\":\"dataro\",\"mnoecfjw\":\"datalbqdxvxdfkdwk\",\"r\":\"datakiupgmdsz\"}},{\"type\":\"Activity\",\"name\":\"djxltjsm\",\"description\":\"ecr\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"is\",\"dependencyConditions\":[]},{\"activity\":\"goapxdmxwetkj\",\"dependencyConditions\":[]},{\"activity\":\"xekql\",\"dependencyConditions\":[]},{\"activity\":\"ctdxargqff\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"fbfqw\",\"value\":\"datalzbtqzjmi\"},{\"name\":\"dcsdfbkiwumrf\",\"value\":\"datajrptsjecqwdosbs\"}],\"\":{\"cpwrgry\":\"dataiacwdxvlku\"}},{\"type\":\"Activity\",\"name\":\"blrqeqcdikcqc\",\"description\":\"dtfthnjxid\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"xgxtccmq\",\"dependencyConditions\":[]},{\"activity\":\"kuqrrzuegin\",\"dependencyConditions\":[]},{\"activity\":\"lnteoapszxqnjxv\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"cqm\",\"value\":\"datajihmcgzwoijtlhx\"},{\"name\":\"sxxra\",\"value\":\"datagaicgqgafkrtsa\"},{\"name\":\"agvq\",\"value\":\"datatcr\"},{\"name\":\"fxqhnw\",\"value\":\"dataujrnfdqlzggvo\"}],\"\":{\"a\":\"dataiumrdbqujyijqc\",\"lbfknxz\":\"datanpsvgupqwqshmn\"}},{\"type\":\"Activity\",\"name\":\"suvjbfryortbre\",\"description\":\"ccpbtvgiokz\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"wawomkzussgj\",\"dependencyConditions\":[]},{\"activity\":\"ubggjdluwbmwu\",\"dependencyConditions\":[]},{\"activity\":\"bekzcmfibboz\",\"dependencyConditions\":[]},{\"activity\":\"ptwvamymswfwc\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"sopffhqxh\",\"value\":\"datacohrhwxvvom\"},{\"name\":\"jpjrxvsggaubrk\",\"value\":\"datafvlqwi\"},{\"name\":\"eoenpihtgigaeeq\",\"value\":\"datapvirozlfcc\"}],\"\":{\"p\":\"dataqixxxgltqldlhhqp\"}}],\"parameters\":{\"nogyvpfyjlfnjmwb\":{\"type\":\"String\",\"defaultValue\":\"datacajhnnbp\"},\"vzqbvdlhc\":{\"type\":\"Int\",\"defaultValue\":\"datahyprpw\"},\"jqz\":{\"type\":\"String\",\"defaultValue\":\"datakmpxtfcrugitjnw\"},\"nbkcqoy\":{\"type\":\"Float\",\"defaultValue\":\"datafea\"}},\"variables\":{\"tzlflqpanceow\":{\"type\":\"Bool\",\"defaultValue\":\"databfbzhczyhtj\"},\"qu\":{\"type\":\"Array\",\"defaultValue\":\"datazxluozmedaq\"}},\"concurrency\":1169190885,\"annotations\":[\"dataux\",\"databsl\",\"datausvbvtcti\",\"dataovfe\"],\"runDimensions\":{\"jjrhvdxfs\":\"datagbacmnj\",\"lsrx\":\"datambbj\"},\"folder\":{\"name\":\"kcvksv\"},\"policy\":{\"elapsedTimeMetric\":{\"duration\":\"datarfnlhlfv\"}}},\"name\":\"ohyecblvpwuqq\",\"type\":\"fuuhm\",\"etag\":\"s\",\"\":{\"zvvrfplkemvvlge\":\"datarcxamgvi\",\"sh\":\"datay\",\"hxkifjvilazopwud\":\"dataplhwplyvqofpemc\",\"xaufowhmd\":\"dataew\"},\"id\":\"gaktugephqdoctg\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -54,21 +54,22 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.pipelines().listByFactory("udtiecnp", "af", com.azure.core.util.Context.NONE); + = manager.pipelines().listByFactory("tdvwmefjpoe", "lyvbvxlrltrztrlo", com.azure.core.util.Context.NONE); - Assertions.assertEquals("lrewfrmqbmcmgeza", response.iterator().next().id()); - Assertions.assertEquals("qjtoeaugwosrywp", response.iterator().next().description()); - Assertions.assertEquals("l", response.iterator().next().activities().get(0).name()); - Assertions.assertEquals("iafsizdec", response.iterator().next().activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, response.iterator().next().activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, + Assertions.assertEquals("gaktugephqdoctg", response.iterator().next().id()); + Assertions.assertEquals("jvrxyr", response.iterator().next().description()); + Assertions.assertEquals("rsyckq", response.iterator().next().activities().get(0).name()); + Assertions.assertEquals("fmqhtr", response.iterator().next().activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, response.iterator().next().activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, response.iterator().next().activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("lwquxrrju", - response.iterator().next().activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("phgsd", response.iterator().next().activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals(ParameterType.OBJECT, response.iterator().next().parameters().get("trfdipss").type()); - Assertions.assertEquals(VariableType.BOOL, response.iterator().next().variables().get("zs").type()); - Assertions.assertEquals(585093789, response.iterator().next().concurrency()); - Assertions.assertEquals("qkkacw", response.iterator().next().folder().name()); + Assertions.assertEquals("hhl", response.iterator().next().activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("wffrzgeobzmxzrai", + response.iterator().next().activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals(ParameterType.STRING, + response.iterator().next().parameters().get("nogyvpfyjlfnjmwb").type()); + Assertions.assertEquals(VariableType.BOOL, response.iterator().next().variables().get("tzlflqpanceow").type()); + Assertions.assertEquals(1169190885, response.iterator().next().concurrency()); + Assertions.assertEquals("kcvksv", response.iterator().next().folder().name()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PolybaseSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PolybaseSettingsTests.java index b73a232b4fa70..3a46a1c64b53a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PolybaseSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PolybaseSettingsTests.java @@ -15,18 +15,18 @@ public final class PolybaseSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PolybaseSettings model = BinaryData.fromString( - "{\"rejectType\":\"percentage\",\"rejectValue\":\"dataekygnepjyuxqdrp\",\"rejectSampleValue\":\"datayxjqranpztla\",\"useTypeDefault\":\"datapfzsf\",\"\":{\"cavk\":\"dataapbrwviovvyhso\"}}") + "{\"rejectType\":\"value\",\"rejectValue\":\"datagrqzwypwhfybflrp\",\"rejectSampleValue\":\"datagqqxe\",\"useTypeDefault\":\"dataghpsqvuised\",\"\":{\"btpvwx\":\"datavfjkxxnqrqdx\",\"zss\":\"datalsvicvpagwohkro\",\"jpiezthflgpsal\":\"datamlozjyovrllvhbgk\",\"wzpfbiqjrz\":\"datanan\"}}") .toObject(PolybaseSettings.class); - Assertions.assertEquals(PolybaseSettingsRejectType.PERCENTAGE, model.rejectType()); + Assertions.assertEquals(PolybaseSettingsRejectType.VALUE, model.rejectType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PolybaseSettings model = new PolybaseSettings().withRejectType(PolybaseSettingsRejectType.PERCENTAGE) - .withRejectValue("dataekygnepjyuxqdrp").withRejectSampleValue("datayxjqranpztla") - .withUseTypeDefault("datapfzsf").withAdditionalProperties(mapOf()); + PolybaseSettings model = new PolybaseSettings().withRejectType(PolybaseSettingsRejectType.VALUE) + .withRejectValue("datagrqzwypwhfybflrp").withRejectSampleValue("datagqqxe") + .withUseTypeDefault("dataghpsqvuised").withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(PolybaseSettings.class); - Assertions.assertEquals(PolybaseSettingsRejectType.PERCENTAGE, model.rejectType()); + Assertions.assertEquals(PolybaseSettingsRejectType.VALUE, model.rejectType()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlSourceTests.java index 4dc03f540e7ce..73bf29740e6a6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlSourceTests.java @@ -11,16 +11,16 @@ public final class PostgreSqlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PostgreSqlSource model = BinaryData.fromString( - "{\"type\":\"PostgreSqlSource\",\"query\":\"datandsjpui\",\"queryTimeout\":\"datavrp\",\"additionalColumns\":\"datagdptfxoffckejxom\",\"sourceRetryCount\":\"datauqwxxynttrn\",\"sourceRetryWait\":\"datavximgns\",\"maxConcurrentConnections\":\"datacxuyzrnngnmf\",\"disableMetricsCollection\":\"datawfoummdomvditp\",\"\":{\"dfhsfnoczefgfq\":\"datalwlirapqhs\",\"jntiqbxzeiudogqf\":\"datae\"}}") + "{\"type\":\"PostgreSqlSource\",\"query\":\"datawtdazebif\",\"queryTimeout\":\"datanxugiorbwyeyrnbu\",\"additionalColumns\":\"dataabtow\",\"sourceRetryCount\":\"datauhlw\",\"sourceRetryWait\":\"datavzuxfsmfpd\",\"maxConcurrentConnections\":\"dataoqpzwfvnoy\",\"disableMetricsCollection\":\"datafjylhvp\",\"\":{\"jyavkyjvctq\":\"datarqwjtswemotjk\"}}") .toObject(PostgreSqlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { PostgreSqlSource model - = new PostgreSqlSource().withSourceRetryCount("datauqwxxynttrn").withSourceRetryWait("datavximgns") - .withMaxConcurrentConnections("datacxuyzrnngnmf").withDisableMetricsCollection("datawfoummdomvditp") - .withQueryTimeout("datavrp").withAdditionalColumns("datagdptfxoffckejxom").withQuery("datandsjpui"); + = new PostgreSqlSource().withSourceRetryCount("datauhlw").withSourceRetryWait("datavzuxfsmfpd") + .withMaxConcurrentConnections("dataoqpzwfvnoy").withDisableMetricsCollection("datafjylhvp") + .withQueryTimeout("datanxugiorbwyeyrnbu").withAdditionalColumns("dataabtow").withQuery("datawtdazebif"); model = BinaryData.fromObject(model).toObject(PostgreSqlSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2SourceTests.java new file mode 100644 index 0000000000000..b8d366a120752 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2SourceTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.PostgreSqlV2Source; + +public final class PostgreSqlV2SourceTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PostgreSqlV2Source model = BinaryData.fromString( + "{\"type\":\"PostgreSqlV2Source\",\"query\":\"datacz\",\"queryTimeout\":\"datapaeyklxsvcbr\",\"additionalColumns\":\"datalt\",\"sourceRetryCount\":\"datamdsngoaofmrph\",\"sourceRetryWait\":\"datas\",\"maxConcurrentConnections\":\"dataunkcgdnhacex\",\"disableMetricsCollection\":\"dataomrrjooepfb\",\"\":{\"gntjmnlpklrjd\":\"dataffxan\",\"kvi\":\"datayp\",\"k\":\"datagnjatjbldgik\",\"akuzezwnqhcpk\":\"datawge\"}}") + .toObject(PostgreSqlV2Source.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PostgreSqlV2Source model + = new PostgreSqlV2Source().withSourceRetryCount("datamdsngoaofmrph").withSourceRetryWait("datas") + .withMaxConcurrentConnections("dataunkcgdnhacex").withDisableMetricsCollection("dataomrrjooepfb") + .withQueryTimeout("datapaeyklxsvcbr").withAdditionalColumns("datalt").withQuery("datacz"); + model = BinaryData.fromObject(model).toObject(PostgreSqlV2Source.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTests.java new file mode 100644 index 0000000000000..788f7cf69066e --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTests.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.DatasetFolder; +import com.azure.resourcemanager.datafactory.models.LinkedServiceReference; +import com.azure.resourcemanager.datafactory.models.ParameterSpecification; +import com.azure.resourcemanager.datafactory.models.ParameterType; +import com.azure.resourcemanager.datafactory.models.PostgreSqlV2TableDataset; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class PostgreSqlV2TableDatasetTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PostgreSqlV2TableDataset model = BinaryData.fromString( + "{\"type\":\"PostgreSqlV2Table\",\"typeProperties\":{\"table\":\"dataxamqecjrzvlcivqx\",\"schema\":\"datamklphxwww\"},\"description\":\"jkbgnfbr\",\"structure\":\"datavfsunhaevla\",\"schema\":\"dataczywywuahwc\",\"linkedServiceName\":{\"referenceName\":\"rewcnnaaxqjfda\",\"parameters\":{\"mggewdqbxexfy\":\"datagim\"}},\"parameters\":{\"yzqbye\":{\"type\":\"Bool\",\"defaultValue\":\"datasuqkslwsfx\"},\"kffydztkqrfbgy\":{\"type\":\"Object\",\"defaultValue\":\"datamohnrtl\"}},\"annotations\":[\"datawvzdte\",\"datajmyqxuhgka\",\"datanyprijyoxxjxbs\"],\"folder\":{\"name\":\"rlcck\"},\"\":{\"mqomxoalknuy\":\"datapjmspau\",\"zi\":\"datapvi\"}}") + .toObject(PostgreSqlV2TableDataset.class); + Assertions.assertEquals("jkbgnfbr", model.description()); + Assertions.assertEquals("rewcnnaaxqjfda", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("yzqbye").type()); + Assertions.assertEquals("rlcck", model.folder().name()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PostgreSqlV2TableDataset model = new PostgreSqlV2TableDataset().withDescription("jkbgnfbr") + .withStructure("datavfsunhaevla").withSchema("dataczywywuahwc") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("rewcnnaaxqjfda") + .withParameters(mapOf("mggewdqbxexfy", "datagim"))) + .withParameters(mapOf("yzqbye", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datasuqkslwsfx"), + "kffydztkqrfbgy", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datamohnrtl"))) + .withAnnotations(Arrays.asList("datawvzdte", "datajmyqxuhgka", "datanyprijyoxxjxbs")) + .withFolder(new DatasetFolder().withName("rlcck")).withTable("dataxamqecjrzvlcivqx") + .withSchemaTypePropertiesSchema("datamklphxwww"); + model = BinaryData.fromObject(model).toObject(PostgreSqlV2TableDataset.class); + Assertions.assertEquals("jkbgnfbr", model.description()); + Assertions.assertEquals("rewcnnaaxqjfda", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("yzqbye").type()); + Assertions.assertEquals("rlcck", model.folder().name()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTypePropertiesTests.java new file mode 100644 index 0000000000000..59a8a0faa268c --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PostgreSqlV2TableDatasetTypePropertiesTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.fluent.models.PostgreSqlV2TableDatasetTypeProperties; + +public final class PostgreSqlV2TableDatasetTypePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PostgreSqlV2TableDatasetTypeProperties model + = BinaryData.fromString("{\"table\":\"datavic\",\"schema\":\"dataaptk\"}") + .toObject(PostgreSqlV2TableDatasetTypeProperties.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PostgreSqlV2TableDatasetTypeProperties model + = new PostgreSqlV2TableDatasetTypeProperties().withTable("datavic").withSchema("dataaptk"); + model = BinaryData.fromObject(model).toObject(PostgreSqlV2TableDatasetTypeProperties.class); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PowerQuerySinkMappingTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PowerQuerySinkMappingTests.java index 6c4990fe0f065..0a66b84716c60 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PowerQuerySinkMappingTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PowerQuerySinkMappingTests.java @@ -20,74 +20,59 @@ public final class PowerQuerySinkMappingTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PowerQuerySinkMapping model = BinaryData.fromString( - "{\"queryName\":\"fiivgdsnrknikp\",\"dataflowSinks\":[{\"script\":\"qnxyloyclrle\",\"schemaLinkedService\":{\"referenceName\":\"shmfbzkfeh\",\"parameters\":{\"oh\":\"datavofnsuwsurod\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"bbxa\",\"parameters\":{\"soiekdmnva\":\"dataorsandslrndi\",\"fdextdarnhpxz\":\"databhxujgyzfsswezn\"}},\"name\":\"kjbvzp\",\"description\":\"cisnhtdskenigoh\",\"dataset\":{\"referenceName\":\"udteowep\",\"parameters\":{\"f\":\"datagrc\",\"vlnv\":\"datacqg\"}},\"linkedService\":{\"referenceName\":\"l\",\"parameters\":{\"d\":\"dataxpugetwgjlx\",\"tzkdqi\":\"datavfnqazvavspjdxa\",\"yredzhnylir\":\"dataumaijcullkyrss\",\"jrrolwrv\":\"datarxykplvjsqazecdo\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"qxstyk\",\"datasetParameters\":\"datafqmgjexi\",\"parameters\":{\"rcnoexwar\":\"datavp\",\"ycevhazwew\":\"dataazfsrvz\",\"dcnxjfg\":\"dataobxlkdycsp\",\"uozwowwmulqgaeqn\":\"dataynuxvyalk\"},\"\":{\"ezcwfscxkrzuze\":\"dataqlb\",\"ebwdvuvq\":\"datadvxmkzgrrgrkovs\",\"kdwagnyahurxtpu\":\"datauplzdoam\"}}},{\"script\":\"radfiwjounvfqy\",\"schemaLinkedService\":{\"referenceName\":\"ivkmdfwfzkocdj\",\"parameters\":{\"vfx\":\"datarbphtllkpkcqzbvy\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"atmvxrjid\",\"parameters\":{\"xhgyllaswwhbmowm\":\"datasypevfrbujlt\"}},\"name\":\"knsknnnpyobyi\",\"description\":\"qsdsuewfgrijdm\",\"dataset\":{\"referenceName\":\"crtmvtfeyopgeiw\",\"parameters\":{\"sxvgwrq\":\"datacizmgg\",\"fm\":\"datawaagzaxqhlerkyim\",\"cyxcluvj\":\"datahwtlli\",\"jpld\":\"datap\"}},\"linkedService\":{\"referenceName\":\"shcjgoobl\",\"parameters\":{\"qwmzzcg\":\"datargcntgq\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"fsmabuur\",\"datasetParameters\":\"dataqwvybxmuehfkbhy\",\"parameters\":{\"chhrnfa\":\"dataeshjj\",\"hmvco\":\"dataqefiwwhbkxzqryov\",\"fmyiwy\":\"dataiagwu\"},\"\":{\"reibcezdtfyarl\":\"dataufmsyfjn\",\"erql\":\"datallg\"}}},{\"script\":\"csffagunybteyht\",\"schemaLinkedService\":{\"referenceName\":\"k\",\"parameters\":{\"cw\":\"datakmfkmc\",\"cwjqtfsx\":\"datagjvyosmxovyfdbah\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"kbezdvnezoua\",\"parameters\":{\"awn\":\"datajwqeypa\",\"hmjlkknwsja\":\"datawhiyus\",\"jnlerm\":\"datamrnrhsv\"}},\"name\":\"ivkzgcqy\",\"description\":\"fekjvcl\",\"dataset\":{\"referenceName\":\"kjzfrbdlsj\",\"parameters\":{\"cjnkawkyh\":\"dataahfvpmwnmu\",\"wstumjtgzutwpsnl\":\"datanwtknpbzkt\"}},\"linkedService\":{\"referenceName\":\"jgrebbon\",\"parameters\":{\"prg\":\"datahfsey\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"esutnsmtgzadpwh\",\"datasetParameters\":\"dataxbvrytt\",\"parameters\":{\"vcxabzwehvsmt\":\"datagmugzssgzkevvqe\",\"ldtkqoajpxtkraf\":\"dataabrhiao\",\"nhhjdfy\":\"dataziquirozqusdz\",\"pwgteroaenvjou\":\"datasiupdmbhau\"},\"\":{\"gfmpdlmry\":\"datajxbraqzrbv\",\"nklbwyqoypodrq\":\"datal\",\"jerokbdkwvjond\":\"datajxmgxsp\"}}},{\"script\":\"iydj\",\"schemaLinkedService\":{\"referenceName\":\"djmylhcj\",\"parameters\":{\"pnlwuhtfa\":\"dataloyvrwz\",\"absqpttulhanjui\":\"dataafklbkigzvugwbc\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"xp\",\"parameters\":{\"ienzskok\":\"datadujuo\"}},\"name\":\"qbjncnbnlppuzbip\",\"description\":\"z\",\"dataset\":{\"referenceName\":\"yrkddpnobc\",\"parameters\":{\"kxcnovkwvzrxaix\":\"datamdswrtifxbhuz\",\"puuvzyfjmor\":\"dataigesbei\"}},\"linkedService\":{\"referenceName\":\"pjaktszr\",\"parameters\":{\"xrwgddgpqfflsw\":\"datarphtjljfmhgd\",\"fr\":\"dataeh\",\"evtykfxos\":\"datazqeinnbu\",\"uhfwklsthjvyk\":\"dataz\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"gwpulrtjweuorojr\",\"datasetParameters\":\"dataxswohshncddz\",\"parameters\":{\"lctgiontvgrloshk\":\"datatntdidhhacpw\",\"v\":\"datathui\"},\"\":{\"osfsfuzqpig\":\"datawswpwbgoetu\"}}}]}") + "{\"queryName\":\"zecdomjrrolwr\",\"dataflowSinks\":[{\"script\":\"qxstyk\",\"schemaLinkedService\":{\"referenceName\":\"fqmgjexi\",\"parameters\":{\"rcnoexwar\":\"datavp\",\"ycevhazwew\":\"dataazfsrvz\",\"dcnxjfg\":\"dataobxlkdycsp\",\"uozwowwmulqgaeqn\":\"dataynuxvyalk\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"tqlbjezcwf\",\"parameters\":{\"epdvxmkzgrrg\":\"datakrzu\",\"sgebw\":\"datako\"}},\"name\":\"vuvq\",\"description\":\"plzdoamqkdwa\",\"dataset\":{\"referenceName\":\"yahurxtpuyuradf\",\"parameters\":{\"k\":\"dataounvfqykni\",\"htllkpkcqzbvyrv\":\"datadfwfzkocdjwjmrb\"}},\"linkedService\":{\"referenceName\":\"cbatmvxrji\",\"parameters\":{\"evfrbujltg\":\"dataosy\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"yl\",\"datasetParameters\":\"dataswwhbmo\",\"parameters\":{\"yqsds\":\"dataknsknnnpyobyi\",\"rijd\":\"dataewf\",\"iwebmcizmggvsxv\":\"datakmcrtmvtfeyopg\",\"erkyimcfmdhwtlli\":\"datawrqywaagzaxqh\"},\"\":{\"vjppujpldthshcj\":\"dataxcl\",\"wmzz\":\"dataoobltoargcntgqy\",\"mabuurtuqwvy\":\"datagbgvf\",\"ymdae\":\"dataxmuehfkb\"}}},{\"script\":\"jjqchhrnfavqefi\",\"schemaLinkedService\":{\"referenceName\":\"hbkxzqryovlhm\",\"parameters\":{\"fmyiwy\":\"databiagwu\",\"msyfjno\":\"datatau\",\"a\":\"dataeibcezdtf\",\"zcsff\":\"datalwllgjerql\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"u\",\"parameters\":{\"spk\":\"datateyh\",\"r\":\"datakvkmfkmchc\",\"fdbahxcwjqtfs\":\"datajvyosmxov\",\"uay\":\"datacakbezdvnez\"}},\"name\":\"ejwqeypaoa\",\"description\":\"kwhiyusjhmjlk\",\"dataset\":{\"referenceName\":\"wsjavmr\",\"parameters\":{\"ivkzgcqy\":\"datasvkjnlerm\",\"frbdlsjftqahfvpm\":\"datavfekjvclbkkj\"}},\"linkedService\":{\"referenceName\":\"mu\",\"parameters\":{\"knpbzktkw\":\"datankawkyhnnw\",\"j\":\"datatumjtgzutwpsnl\",\"fseykprgpqnesu\":\"datagrebbonjhm\",\"pwhldxb\":\"datansmtgza\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"tt\",\"datasetParameters\":\"datasgmugzssgzkevv\",\"parameters\":{\"abrhiao\":\"datavcxabzwehvsmt\",\"ziquirozqusdz\":\"dataldtkqoajpxtkraf\",\"siupdmbhau\":\"datanhhjdfy\",\"jkjxb\":\"datapwgteroaenvjou\"},\"\":{\"vogfmpdlm\":\"datazr\",\"nklbwyqoypodrq\":\"dataynl\",\"jerokbdkwvjond\":\"datajxmgxsp\"}}},{\"script\":\"iydj\",\"schemaLinkedService\":{\"referenceName\":\"djmylhcj\",\"parameters\":{\"pnlwuhtfa\":\"dataloyvrwz\",\"absqpttulhanjui\":\"dataafklbkigzvugwbc\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"xp\",\"parameters\":{\"ienzskok\":\"datadujuo\"}},\"name\":\"qbjncnbnlppuzbip\",\"description\":\"z\",\"dataset\":{\"referenceName\":\"yrkddpnobc\",\"parameters\":{\"kxcnovkwvzrxaix\":\"datamdswrtifxbhuz\",\"puuvzyfjmor\":\"dataigesbei\"}},\"linkedService\":{\"referenceName\":\"pjaktszr\",\"parameters\":{\"xrwgddgpqfflsw\":\"datarphtjljfmhgd\",\"fr\":\"dataeh\",\"evtykfxos\":\"datazqeinnbu\",\"uhfwklsthjvyk\":\"dataz\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"gwpulrtjweuorojr\",\"datasetParameters\":\"dataxswohshncddz\",\"parameters\":{\"lctgiontvgrloshk\":\"datatntdidhhacpw\",\"v\":\"datathui\"},\"\":{\"osfsfuzqpig\":\"datawswpwbgoetu\"}}},{\"script\":\"nmdeimagmw\",\"schemaLinkedService\":{\"referenceName\":\"xeu\",\"parameters\":{\"qqjcyhvyr\":\"datatkllbfnnhert\",\"expcxylquowun\":\"datageuvujywldkjay\",\"wseulfzxgh\":\"dataactjp\"}},\"rejectedDataLinkedService\":{\"referenceName\":\"lrvpaumkz\",\"parameters\":{\"pjn\":\"datangkfipxolpujlm\",\"rmevkckoce\":\"datalsbxvouxcdenthgp\",\"arohw\":\"datahlvfniryh\",\"bamnkgmosayfyvod\":\"dataxjhzwsjqrmxpyz\"}},\"name\":\"tpczzqusf\",\"description\":\"wsrrrgijolyspky\",\"dataset\":{\"referenceName\":\"yaej\",\"parameters\":{\"ecccfycywcuhqfxf\":\"datafkkhgqs\",\"st\":\"datarfzaurermnyphcoo\"}},\"linkedService\":{\"referenceName\":\"bzrnvu\",\"parameters\":{\"cczz\":\"datajyttgkpscm\",\"mgezkb\":\"datala\",\"dngbgpxoe\":\"datairftlomec\"}},\"flowlet\":{\"type\":\"DataFlowReference\",\"referenceName\":\"efavbsbhd\",\"datasetParameters\":\"dataaqafalbkem\",\"parameters\":{\"brrkvxmeihrzi\":\"datavdhvdv\"},\"\":{\"dsu\":\"datanpojmgkeoqrx\",\"kbvaxehiegk\":\"dataonjuwgvse\",\"i\":\"dataukvalcvlbqht\"}}}]}") .toObject(PowerQuerySinkMapping.class); - Assertions.assertEquals("fiivgdsnrknikp", model.queryName()); - Assertions.assertEquals("kjbvzp", model.dataflowSinks().get(0).name()); - Assertions.assertEquals("cisnhtdskenigoh", model.dataflowSinks().get(0).description()); - Assertions.assertEquals("udteowep", model.dataflowSinks().get(0).dataset().referenceName()); - Assertions.assertEquals("l", model.dataflowSinks().get(0).linkedService().referenceName()); + Assertions.assertEquals("zecdomjrrolwr", model.queryName()); + Assertions.assertEquals("vuvq", model.dataflowSinks().get(0).name()); + Assertions.assertEquals("plzdoamqkdwa", model.dataflowSinks().get(0).description()); + Assertions.assertEquals("yahurxtpuyuradf", model.dataflowSinks().get(0).dataset().referenceName()); + Assertions.assertEquals("cbatmvxrji", model.dataflowSinks().get(0).linkedService().referenceName()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataflowSinks().get(0).flowlet().type()); - Assertions.assertEquals("qxstyk", model.dataflowSinks().get(0).flowlet().referenceName()); - Assertions.assertEquals("shmfbzkfeh", model.dataflowSinks().get(0).schemaLinkedService().referenceName()); - Assertions.assertEquals("bbxa", model.dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("qnxyloyclrle", model.dataflowSinks().get(0).script()); + Assertions.assertEquals("yl", model.dataflowSinks().get(0).flowlet().referenceName()); + Assertions.assertEquals("fqmgjexi", model.dataflowSinks().get(0).schemaLinkedService().referenceName()); + Assertions.assertEquals("tqlbjezcwf", model.dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); + Assertions.assertEquals("qxstyk", model.dataflowSinks().get(0).script()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PowerQuerySinkMapping model = new PowerQuerySinkMapping().withQueryName("fiivgdsnrknikp") + PowerQuerySinkMapping model = new PowerQuerySinkMapping().withQueryName("zecdomjrrolwr") .withDataflowSinks(Arrays.asList( - new PowerQuerySink().withName("kjbvzp").withDescription("cisnhtdskenigoh") - .withDataset(new DatasetReference().withReferenceName("udteowep") - .withParameters(mapOf("f", "datagrc", "vlnv", "datacqg"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("l") - .withParameters(mapOf("d", "dataxpugetwgjlx", "tzkdqi", "datavfnqazvavspjdxa", "yredzhnylir", - "dataumaijcullkyrss", "jrrolwrv", "datarxykplvjsqazecdo"))) + new PowerQuerySink().withName("vuvq").withDescription("plzdoamqkdwa") + .withDataset(new DatasetReference().withReferenceName("yahurxtpuyuradf") + .withParameters(mapOf("k", "dataounvfqykni", "htllkpkcqzbvyrv", "datadfwfzkocdjwjmrb"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("cbatmvxrji") + .withParameters(mapOf("evfrbujltg", "dataosy"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("qxstyk").withDatasetParameters("datafqmgjexi") - .withParameters(mapOf("rcnoexwar", "datavp", "ycevhazwew", "dataazfsrvz", "dcnxjfg", - "dataobxlkdycsp", "uozwowwmulqgaeqn", "dataynuxvyalk")) - .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("shmfbzkfeh") - .withParameters(mapOf("oh", "datavofnsuwsurod"))) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("bbxa").withParameters( - mapOf("soiekdmnva", "dataorsandslrndi", "fdextdarnhpxz", "databhxujgyzfsswezn"))) - .withScript("qnxyloyclrle"), - new PowerQuerySink().withName("knsknnnpyobyi").withDescription("qsdsuewfgrijdm") - .withDataset(new DatasetReference().withReferenceName("crtmvtfeyopgeiw") - .withParameters(mapOf("sxvgwrq", "datacizmgg", "fm", "datawaagzaxqhlerkyim", "cyxcluvj", - "datahwtlli", "jpld", "datap"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("shcjgoobl") - .withParameters(mapOf("qwmzzcg", "datargcntgq"))) - .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("fsmabuur").withDatasetParameters("dataqwvybxmuehfkbhy") - .withParameters( - mapOf("chhrnfa", "dataeshjj", "hmvco", "dataqefiwwhbkxzqryov", "fmyiwy", "dataiagwu")) + .withReferenceName("yl").withDatasetParameters("dataswwhbmo") + .withParameters(mapOf("yqsds", "dataknsknnnpyobyi", "rijd", "dataewf", "iwebmcizmggvsxv", + "datakmcrtmvtfeyopg", "erkyimcfmdhwtlli", "datawrqywaagzaxqh")) .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("ivkmdfwfzkocdj") - .withParameters(mapOf("vfx", "datarbphtllkpkcqzbvy"))) - .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("atmvxrjid") - .withParameters(mapOf("xhgyllaswwhbmowm", "datasypevfrbujlt"))) - .withScript("radfiwjounvfqy"), - new PowerQuerySink().withName("ivkzgcqy").withDescription("fekjvcl") - .withDataset(new DatasetReference().withReferenceName("kjzfrbdlsj") - .withParameters(mapOf("cjnkawkyh", "dataahfvpmwnmu", "wstumjtgzutwpsnl", "datanwtknpbzkt"))) - .withLinkedService(new LinkedServiceReference().withReferenceName("jgrebbon") - .withParameters(mapOf("prg", "datahfsey"))) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("fqmgjexi") + .withParameters(mapOf("rcnoexwar", "datavp", "ycevhazwew", "dataazfsrvz", "dcnxjfg", + "dataobxlkdycsp", "uozwowwmulqgaeqn", "dataynuxvyalk"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("tqlbjezcwf") + .withParameters(mapOf("epdvxmkzgrrg", "datakrzu", "sgebw", "datako"))) + .withScript("qxstyk"), + new PowerQuerySink().withName("ejwqeypaoa").withDescription("kwhiyusjhmjlk") + .withDataset(new DatasetReference().withReferenceName("wsjavmr") + .withParameters(mapOf("ivkzgcqy", "datasvkjnlerm", "frbdlsjftqahfvpm", "datavfekjvclbkkj"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("mu") + .withParameters(mapOf("knpbzktkw", "datankawkyhnnw", "j", "datatumjtgzutwpsnl", + "fseykprgpqnesu", "datagrebbonjhm", "pwhldxb", "datansmtgza"))) .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) - .withReferenceName("esutnsmtgzadpwh").withDatasetParameters("dataxbvrytt") - .withParameters(mapOf("vcxabzwehvsmt", "datagmugzssgzkevvqe", "ldtkqoajpxtkraf", "dataabrhiao", - "nhhjdfy", "dataziquirozqusdz", "pwgteroaenvjou", "datasiupdmbhau")) + .withReferenceName("tt").withDatasetParameters("datasgmugzssgzkevv") + .withParameters(mapOf("abrhiao", "datavcxabzwehvsmt", "ziquirozqusdz", "dataldtkqoajpxtkraf", + "siupdmbhau", "datanhhjdfy", "jkjxb", "datapwgteroaenvjou")) .withAdditionalProperties(mapOf())) - .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("k") - .withParameters(mapOf("cw", "datakmfkmc", "cwjqtfsx", "datagjvyosmxovyfdbah"))) - .withRejectedDataLinkedService( - new LinkedServiceReference().withReferenceName("kbezdvnezoua").withParameters( - mapOf("awn", "datajwqeypa", "hmjlkknwsja", "datawhiyus", "jnlerm", "datamrnrhsv"))) - .withScript("csffagunybteyht"), + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("hbkxzqryovlhm") + .withParameters(mapOf("fmyiwy", "databiagwu", "msyfjno", "datatau", "a", "dataeibcezdtf", + "zcsff", "datalwllgjerql"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("u") + .withParameters(mapOf("spk", "datateyh", "r", "datakvkmfkmchc", "fdbahxcwjqtfs", + "datajvyosmxov", "uay", "datacakbezdvnez"))) + .withScript("jjqchhrnfavqefi"), new PowerQuerySink().withName("qbjncnbnlppuzbip").withDescription("z") .withDataset(new DatasetReference().withReferenceName("yrkddpnobc") .withParameters(mapOf("kxcnovkwvzrxaix", "datamdswrtifxbhuz", "puuvzyfjmor", "dataigesbei"))) @@ -102,19 +87,34 @@ public void testSerialize() throws Exception { .withParameters(mapOf("pnlwuhtfa", "dataloyvrwz", "absqpttulhanjui", "dataafklbkigzvugwbc"))) .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("xp") .withParameters(mapOf("ienzskok", "datadujuo"))) - .withScript("iydj"))); + .withScript("iydj"), + new PowerQuerySink().withName("tpczzqusf").withDescription("wsrrrgijolyspky") + .withDataset(new DatasetReference().withReferenceName("yaej") + .withParameters(mapOf("ecccfycywcuhqfxf", "datafkkhgqs", "st", "datarfzaurermnyphcoo"))) + .withLinkedService(new LinkedServiceReference().withReferenceName("bzrnvu").withParameters( + mapOf("cczz", "datajyttgkpscm", "mgezkb", "datala", "dngbgpxoe", "datairftlomec"))) + .withFlowlet(new DataFlowReference().withType(DataFlowReferenceType.DATA_FLOW_REFERENCE) + .withReferenceName("efavbsbhd").withDatasetParameters("dataaqafalbkem") + .withParameters(mapOf("brrkvxmeihrzi", "datavdhvdv")).withAdditionalProperties(mapOf())) + .withSchemaLinkedService(new LinkedServiceReference().withReferenceName("xeu") + .withParameters(mapOf("qqjcyhvyr", "datatkllbfnnhert", "expcxylquowun", "datageuvujywldkjay", + "wseulfzxgh", "dataactjp"))) + .withRejectedDataLinkedService(new LinkedServiceReference().withReferenceName("lrvpaumkz") + .withParameters(mapOf("pjn", "datangkfipxolpujlm", "rmevkckoce", "datalsbxvouxcdenthgp", + "arohw", "datahlvfniryh", "bamnkgmosayfyvod", "dataxjhzwsjqrmxpyz"))) + .withScript("nmdeimagmw"))); model = BinaryData.fromObject(model).toObject(PowerQuerySinkMapping.class); - Assertions.assertEquals("fiivgdsnrknikp", model.queryName()); - Assertions.assertEquals("kjbvzp", model.dataflowSinks().get(0).name()); - Assertions.assertEquals("cisnhtdskenigoh", model.dataflowSinks().get(0).description()); - Assertions.assertEquals("udteowep", model.dataflowSinks().get(0).dataset().referenceName()); - Assertions.assertEquals("l", model.dataflowSinks().get(0).linkedService().referenceName()); + Assertions.assertEquals("zecdomjrrolwr", model.queryName()); + Assertions.assertEquals("vuvq", model.dataflowSinks().get(0).name()); + Assertions.assertEquals("plzdoamqkdwa", model.dataflowSinks().get(0).description()); + Assertions.assertEquals("yahurxtpuyuradf", model.dataflowSinks().get(0).dataset().referenceName()); + Assertions.assertEquals("cbatmvxrji", model.dataflowSinks().get(0).linkedService().referenceName()); Assertions.assertEquals(DataFlowReferenceType.DATA_FLOW_REFERENCE, model.dataflowSinks().get(0).flowlet().type()); - Assertions.assertEquals("qxstyk", model.dataflowSinks().get(0).flowlet().referenceName()); - Assertions.assertEquals("shmfbzkfeh", model.dataflowSinks().get(0).schemaLinkedService().referenceName()); - Assertions.assertEquals("bbxa", model.dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); - Assertions.assertEquals("qnxyloyclrle", model.dataflowSinks().get(0).script()); + Assertions.assertEquals("yl", model.dataflowSinks().get(0).flowlet().referenceName()); + Assertions.assertEquals("fqmgjexi", model.dataflowSinks().get(0).schemaLinkedService().referenceName()); + Assertions.assertEquals("tqlbjezcwf", model.dataflowSinks().get(0).rejectedDataLinkedService().referenceName()); + Assertions.assertEquals("qxstyk", model.dataflowSinks().get(0).script()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoObjectDatasetTests.java index ade9a72f8022e..9a8dc2e3d02e1 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoObjectDatasetTests.java @@ -19,29 +19,29 @@ public final class PrestoObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PrestoObjectDataset model = BinaryData.fromString( - "{\"type\":\"PrestoObject\",\"typeProperties\":{\"tableName\":\"dataarluobbvalq\",\"table\":\"dataknyujxysvclfjy\",\"schema\":\"datavildlfflle\"},\"description\":\"mtxfqpfi\",\"structure\":\"datacgbfou\",\"schema\":\"databpgcryvidbz\",\"linkedServiceName\":{\"referenceName\":\"ylbvj\",\"parameters\":{\"xjftecgprz\":\"datangw\",\"dq\":\"dataqm\"}},\"parameters\":{\"xoyxuuco\":{\"type\":\"String\",\"defaultValue\":\"datayqhaat\"}},\"annotations\":[\"datayruxrzhhlh\",\"datavmgsbpgmncrv\",\"datapi\",\"dataoromppzsauqmeu\"],\"folder\":{\"name\":\"fcmpuaiugoceqtl\"},\"\":{\"ncfunlakgixhqjqh\":\"datajymwiccu\"}}") + "{\"type\":\"PrestoObject\",\"typeProperties\":{\"tableName\":\"dataz\",\"table\":\"datayujxysv\",\"schema\":\"dataf\"},\"description\":\"clvildlf\",\"structure\":\"dataleirmtxfqpfildcg\",\"schema\":\"dataou\",\"linkedServiceName\":{\"referenceName\":\"fbpgcryvidbzdy\",\"parameters\":{\"jftecgprzsqmp\":\"datajatgngwn\",\"akzbyqha\":\"dataq\"}},\"parameters\":{\"lusyruxrz\":{\"type\":\"Array\",\"defaultValue\":\"datayxuuc\"},\"tpiforomppz\":{\"type\":\"Float\",\"defaultValue\":\"datahrvmgsbpgmncr\"}},\"annotations\":[\"dataqmeu\"],\"folder\":{\"name\":\"fcmpuaiugoceqtl\"},\"\":{\"ncfunlakgixhqjqh\":\"datajymwiccu\"}}") .toObject(PrestoObjectDataset.class); - Assertions.assertEquals("mtxfqpfi", model.description()); - Assertions.assertEquals("ylbvj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.STRING, model.parameters().get("xoyxuuco").type()); + Assertions.assertEquals("clvildlf", model.description()); + Assertions.assertEquals("fbpgcryvidbzdy", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("lusyruxrz").type()); Assertions.assertEquals("fcmpuaiugoceqtl", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrestoObjectDataset model = new PrestoObjectDataset().withDescription("mtxfqpfi").withStructure("datacgbfou") - .withSchema("databpgcryvidbz") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ylbvj") - .withParameters(mapOf("xjftecgprz", "datangw", "dq", "dataqm"))) - .withParameters(mapOf("xoyxuuco", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datayqhaat"))) - .withAnnotations(Arrays.asList("datayruxrzhhlh", "datavmgsbpgmncrv", "datapi", "dataoromppzsauqmeu")) - .withFolder(new DatasetFolder().withName("fcmpuaiugoceqtl")).withTableName("dataarluobbvalq") - .withTable("dataknyujxysvclfjy").withSchemaTypePropertiesSchema("datavildlfflle"); + PrestoObjectDataset model = new PrestoObjectDataset().withDescription("clvildlf") + .withStructure("dataleirmtxfqpfildcg").withSchema("dataou") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fbpgcryvidbzdy") + .withParameters(mapOf("jftecgprzsqmp", "datajatgngwn", "akzbyqha", "dataq"))) + .withParameters(mapOf("lusyruxrz", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datayxuuc"), "tpiforomppz", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datahrvmgsbpgmncr"))) + .withAnnotations(Arrays.asList("dataqmeu")).withFolder(new DatasetFolder().withName("fcmpuaiugoceqtl")) + .withTableName("dataz").withTable("datayujxysv").withSchemaTypePropertiesSchema("dataf"); model = BinaryData.fromObject(model).toObject(PrestoObjectDataset.class); - Assertions.assertEquals("mtxfqpfi", model.description()); - Assertions.assertEquals("ylbvj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.STRING, model.parameters().get("xoyxuuco").type()); + Assertions.assertEquals("clvildlf", model.description()); + Assertions.assertEquals("fbpgcryvidbzdy", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("lusyruxrz").type()); Assertions.assertEquals("fcmpuaiugoceqtl", model.folder().name()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoSourceTests.java index 1dd9219a5c4b6..734dcf315eb2f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrestoSourceTests.java @@ -11,15 +11,16 @@ public final class PrestoSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PrestoSource model = BinaryData.fromString( - "{\"type\":\"PrestoSource\",\"query\":\"datauyx\",\"queryTimeout\":\"datayjas\",\"additionalColumns\":\"datahbfttptsdee\",\"sourceRetryCount\":\"dataovanag\",\"sourceRetryWait\":\"dataacsfbmb\",\"maxConcurrentConnections\":\"dataefqku\",\"disableMetricsCollection\":\"datayumoamqxwluslxyt\",\"\":{\"bknfpdi\":\"datajledjxbl\",\"eomctbgoccypx\":\"datahzgj\",\"nuflfz\":\"datarhbql\",\"bpyo\":\"datawkkzex\"}}") + "{\"type\":\"PrestoSource\",\"query\":\"dataqngpvvnbu\",\"queryTimeout\":\"datavkutl\",\"additionalColumns\":\"dataxuuqb\",\"sourceRetryCount\":\"datapbeswgkreozpufk\",\"sourceRetryWait\":\"datamzcbzgi\",\"maxConcurrentConnections\":\"dataqpegcgdndpb\",\"disableMetricsCollection\":\"dataeymmcbiktetzvqt\",\"\":{\"pdnbzqweohmlkzhx\":\"datavcsbyimygswdu\",\"haerhxd\":\"datadmauanxzrqt\",\"bqmoguy\":\"datahkbrkhjjbwelicrx\",\"dxljjzdbzk\":\"datamselwszqveak\"}}") .toObject(PrestoSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrestoSource model = new PrestoSource().withSourceRetryCount("dataovanag").withSourceRetryWait("dataacsfbmb") - .withMaxConcurrentConnections("dataefqku").withDisableMetricsCollection("datayumoamqxwluslxyt") - .withQueryTimeout("datayjas").withAdditionalColumns("datahbfttptsdee").withQuery("datauyx"); + PrestoSource model + = new PrestoSource().withSourceRetryCount("datapbeswgkreozpufk").withSourceRetryWait("datamzcbzgi") + .withMaxConcurrentConnections("dataqpegcgdndpb").withDisableMetricsCollection("dataeymmcbiktetzvqt") + .withQueryTimeout("datavkutl").withAdditionalColumns("dataxuuqb").withQuery("dataqngpvvnbu"); model = BinaryData.fromObject(model).toObject(PrestoSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndPointConnectionsListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndPointConnectionsListByFactoryMockTests.java index a221b196e343a..ff9351e74b139 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndPointConnectionsListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndPointConnectionsListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"waocfnf\",\"privateEndpoint\":{\"id\":\"dccwuz\"},\"privateLinkServiceConnectionState\":{\"status\":\"ckewl\",\"description\":\"weupsubawzafzdzh\",\"actionsRequired\":\"bxcelvaww\"}},\"name\":\"btkyjvzz\",\"type\":\"ylimnm\",\"etag\":\"sjuacdqvr\",\"id\":\"ykqotzpepm\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"bkazv\",\"privateEndpoint\":{\"id\":\"bky\"},\"privateLinkServiceConnectionState\":{\"status\":\"wzbfpcleniozqr\",\"description\":\"hrp\",\"actionsRequired\":\"hpyymlwall\"}},\"name\":\"yorhgo\",\"type\":\"v\",\"etag\":\"svrpnoxbokmqviv\",\"id\":\"fbzrfmfadvfkjdwu\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,15 +49,15 @@ public void testListByFactory() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response - = manager.privateEndPointConnections().listByFactory("itoibgv", "a", com.azure.core.util.Context.NONE); + PagedIterable response = manager.privateEndPointConnections() + .listByFactory("xiwulqp", "qxcygevgj", com.azure.core.util.Context.NONE); - Assertions.assertEquals("ykqotzpepm", response.iterator().next().id()); - Assertions.assertEquals("ckewl", + Assertions.assertEquals("fbzrfmfadvfkjdwu", response.iterator().next().id()); + Assertions.assertEquals("wzbfpcleniozqr", response.iterator().next().properties().privateLinkServiceConnectionState().status()); - Assertions.assertEquals("weupsubawzafzdzh", + Assertions.assertEquals("hrp", response.iterator().next().properties().privateLinkServiceConnectionState().description()); - Assertions.assertEquals("bxcelvaww", + Assertions.assertEquals("hpyymlwall", response.iterator().next().properties().privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsCreateOrUpdateWithResponseMockTests.java index 00e00b3ee245e..783452a8f166b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsCreateOrUpdateWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"atvnejloc\",\"privateEndpoint\":{\"id\":\"adlpqlwtxs\"},\"privateLinkServiceConnectionState\":{\"status\":\"zhhzl\",\"description\":\"vcehkvafcjek\",\"actionsRequired\":\"g\"}},\"name\":\"rifyrap\",\"type\":\"iaeqcg\",\"etag\":\"nj\",\"id\":\"emlw\"}"; + = "{\"properties\":{\"provisioningState\":\"ijadhgka\",\"privateEndpoint\":{\"id\":\"jqutixy\"},\"privateLinkServiceConnectionState\":{\"status\":\"ypmvof\",\"description\":\"nsfb\",\"actionsRequired\":\"dnothibtnuqdded\"}},\"name\":\"hq\",\"type\":\"mnsunxfdpulpn\",\"etag\":\"yxbbdxnnaeyczbky\",\"id\":\"pnbdzjuq\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,18 +51,18 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PrivateEndpointConnectionResource response = manager.privateEndpointConnectionOperations() - .define("vjbgynpapzbbcfu").withExistingFactory("mjxxov", "cwngg") - .withProperties(new PrivateLinkConnectionApprovalRequest() - .withPrivateLinkServiceConnectionState(new PrivateLinkConnectionState().withStatus("zoamttxyddkvidlb") - .withDescription("ycgz").withActionsRequired("cy")) - .withPrivateEndpoint(new PrivateEndpoint().withId("ywpednousxrljl"))) - .withIfMatch("qawmoxvq").create(); + PrivateEndpointConnectionResource response + = manager.privateEndpointConnectionOperations().define("hvuihjlzjt").withExistingFactory("ia", "klxp") + .withProperties(new PrivateLinkConnectionApprovalRequest() + .withPrivateLinkServiceConnectionState(new PrivateLinkConnectionState().withStatus("fuab") + .withDescription("tyoaccrlydml").withActionsRequired("trxqdypmhg")) + .withPrivateEndpoint(new PrivateEndpoint().withId("uqa"))) + .withIfMatch("dy").create(); - Assertions.assertEquals("emlw", response.id()); - Assertions.assertEquals("zhhzl", response.properties().privateLinkServiceConnectionState().status()); - Assertions.assertEquals("vcehkvafcjek", - response.properties().privateLinkServiceConnectionState().description()); - Assertions.assertEquals("g", response.properties().privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals("pnbdzjuq", response.id()); + Assertions.assertEquals("ypmvof", response.properties().privateLinkServiceConnectionState().status()); + Assertions.assertEquals("nsfb", response.properties().privateLinkServiceConnectionState().description()); + Assertions.assertEquals("dnothibtnuqdded", + response.properties().privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsDeleteWithResponseMockTests.java index 09064819456e9..e461a7e0628fa 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsDeleteWithResponseMockTests.java @@ -45,8 +45,8 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.privateEndpointConnectionOperations().deleteWithResponse("hottykfkwzkwuwgz", "ersukveknwldq", - "lgzcwrhhg", com.azure.core.util.Context.NONE); + manager.privateEndpointConnectionOperations().deleteWithResponse("wykk", "qq", "qgknpgqzmgivkf", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsGetWithResponseMockTests.java index 4ec3ac19767ad..b58b30f421b6a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateEndpointConnectionOperationsGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"lehvmraoxnii\",\"privateEndpoint\":{\"id\":\"lygjtibhzjhqfuq\"},\"privateLinkServiceConnectionState\":{\"status\":\"hhigywgqewc\",\"description\":\"jjhn\",\"actionsRequired\":\"rsgrt\"}},\"name\":\"depaun\",\"type\":\"knucsrqfmcrye\",\"etag\":\"lx\",\"id\":\"hntsqsphieqgoioe\"}"; + = "{\"properties\":{\"provisioningState\":\"nupqfsshjlpz\",\"privateEndpoint\":{\"id\":\"ymdqljngroawltab\"},\"privateLinkServiceConnectionState\":{\"status\":\"rrea\",\"description\":\"yeltrnogfy\",\"actionsRequired\":\"m\"}},\"name\":\"ybgrugklwubkmd\",\"type\":\"nwokeuxcm\",\"etag\":\"kirzlfpkinrgqz\",\"id\":\"rnigztxcjnwzvl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,12 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PrivateEndpointConnectionResource response = manager.privateEndpointConnectionOperations() - .getWithResponse("ckzdwietfxpdz", "livk", "xwfk", "n", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("yooewyvwwvki", "uvphkhszesxsyrvj", "wpknbwh", "ev", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("hntsqsphieqgoioe", response.id()); - Assertions.assertEquals("hhigywgqewc", response.properties().privateLinkServiceConnectionState().status()); - Assertions.assertEquals("jjhn", response.properties().privateLinkServiceConnectionState().description()); - Assertions.assertEquals("rsgrt", response.properties().privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals("rnigztxcjnwzvl", response.id()); + Assertions.assertEquals("rrea", response.properties().privateLinkServiceConnectionState().status()); + Assertions.assertEquals("yeltrnogfy", response.properties().privateLinkServiceConnectionState().description()); + Assertions.assertEquals("m", response.properties().privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateLinkResourcesGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateLinkResourcesGetWithResponseMockTests.java index b7b84a93aa634..7adce5ab5c9aa 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateLinkResourcesGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/PrivateLinkResourcesGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"groupId\":\"ipptpreputu\",\"requiredMembers\":[\"wnkzwyry\",\"hhnoiqtvfruy\",\"navbfkzvvrzdbr\"],\"requiredZoneNames\":[\"ey\"]},\"name\":\"dzuldtfx\",\"type\":\"mmxzszhvjfijx\",\"etag\":\"ojbgiip\",\"id\":\"qyapn\"},{\"properties\":{\"groupId\":\"y\",\"requiredMembers\":[\"jy\",\"uswdwdau\"],\"requiredZoneNames\":[\"v\"]},\"name\":\"fso\",\"type\":\"hjkiaj\",\"etag\":\"juehcrywwfns\",\"id\":\"cjadnwafjib\"}]}"; + = "{\"value\":[{\"properties\":{\"groupId\":\"ydzhafp\",\"requiredMembers\":[\"fzntwountffqa\",\"hbumoq\",\"aixsalgzzm\",\"ik\"],\"requiredZoneNames\":[\"gmmegu\",\"efmujgxnsndcmg\",\"ddsgj\"]},\"name\":\"yzcvmpz\",\"type\":\"tuaapyejbs\",\"etag\":\"cqqxymmqxndx\",\"id\":\"zyiycnpowjy\"},{\"properties\":{\"groupId\":\"qohcyhmkp\",\"requiredMembers\":[\"lpxpalljveqx\",\"cbparyoa\"],\"requiredZoneNames\":[\"j\",\"vpvzfvegumsqd\",\"cgfcbat\",\"lrpbgb\"]},\"name\":\"hnmyyagoeby\",\"type\":\"glkvby\",\"etag\":\"eraegfyrgrlnb\",\"id\":\"kojmmcnls\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,8 +49,8 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PrivateLinkResourcesWrapper response = manager.privateLinkResources() - .getWithResponse("agsx", "shau", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("nqyusvgrba", "xsxtku", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("qyapn", response.value().get(0).id()); + Assertions.assertEquals("zyiycnpowjy", response.value().get(0).id()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/QuickBooksSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/QuickBooksSourceTests.java index 8dc8c5501925b..8509a09b65177 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/QuickBooksSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/QuickBooksSourceTests.java @@ -11,16 +11,16 @@ public final class QuickBooksSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { QuickBooksSource model = BinaryData.fromString( - "{\"type\":\"QuickBooksSource\",\"query\":\"databj\",\"queryTimeout\":\"datalboiojpjnhwwyhxz\",\"additionalColumns\":\"datahx\",\"sourceRetryCount\":\"datavjfsmfkdhhwfrm\",\"sourceRetryWait\":\"dataokefdgfexa\",\"maxConcurrentConnections\":\"datat\",\"disableMetricsCollection\":\"datapszdn\",\"\":{\"oyiheheim\":\"datauuqxmdievkmrso\"}}") + "{\"type\":\"QuickBooksSource\",\"query\":\"datagw\",\"queryTimeout\":\"dataujshcsnk\",\"additionalColumns\":\"datagpqxqevt\",\"sourceRetryCount\":\"datavyy\",\"sourceRetryWait\":\"datakjirvjogsalvjl\",\"maxConcurrentConnections\":\"dataimua\",\"disableMetricsCollection\":\"datakympwquu\",\"\":{\"iqeftgunropdpuf\":\"dataofuzthszjyanhs\"}}") .toObject(QuickBooksSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { QuickBooksSource model - = new QuickBooksSource().withSourceRetryCount("datavjfsmfkdhhwfrm").withSourceRetryWait("dataokefdgfexa") - .withMaxConcurrentConnections("datat").withDisableMetricsCollection("datapszdn") - .withQueryTimeout("datalboiojpjnhwwyhxz").withAdditionalColumns("datahx").withQuery("databj"); + = new QuickBooksSource().withSourceRetryCount("datavyy").withSourceRetryWait("datakjirvjogsalvjl") + .withMaxConcurrentConnections("dataimua").withDisableMetricsCollection("datakympwquu") + .withQueryTimeout("dataujshcsnk").withAdditionalColumns("datagpqxqevt").withQuery("datagw"); model = BinaryData.fromObject(model).toObject(QuickBooksSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleOccurrenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleOccurrenceTests.java index bf04e7f76f846..4bf82d7107a94 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleOccurrenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleOccurrenceTests.java @@ -15,19 +15,19 @@ public final class RecurrenceScheduleOccurrenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RecurrenceScheduleOccurrence model = BinaryData.fromString( - "{\"day\":\"Friday\",\"occurrence\":881053682,\"\":{\"ll\":\"datapadhrijxuqtjc\",\"rnb\":\"dataasxlqsfpctq\"}}") + "{\"day\":\"Thursday\",\"occurrence\":61488311,\"\":{\"sneyuirr\":\"datafjf\",\"mwcflphqt\":\"dataxrftfamozyv\",\"gzctfnlaklszbeu\":\"datapcowmukzcrpd\",\"hq\":\"dataqfxixzwi\"}}") .toObject(RecurrenceScheduleOccurrence.class); - Assertions.assertEquals(DayOfWeek.FRIDAY, model.day()); - Assertions.assertEquals(881053682, model.occurrence()); + Assertions.assertEquals(DayOfWeek.THURSDAY, model.day()); + Assertions.assertEquals(61488311, model.occurrence()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RecurrenceScheduleOccurrence model = new RecurrenceScheduleOccurrence().withDay(DayOfWeek.FRIDAY) - .withOccurrence(881053682).withAdditionalProperties(mapOf()); + RecurrenceScheduleOccurrence model = new RecurrenceScheduleOccurrence().withDay(DayOfWeek.THURSDAY) + .withOccurrence(61488311).withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(RecurrenceScheduleOccurrence.class); - Assertions.assertEquals(DayOfWeek.FRIDAY, model.day()); - Assertions.assertEquals(881053682, model.occurrence()); + Assertions.assertEquals(DayOfWeek.THURSDAY, model.day()); + Assertions.assertEquals(61488311, model.occurrence()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleTests.java index 39260b33bcee6..09511d36a73dc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RecurrenceScheduleTests.java @@ -18,33 +18,40 @@ public final class RecurrenceScheduleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RecurrenceSchedule model = BinaryData.fromString( - "{\"minutes\":[838577333,1015295902,1361107137,935311628],\"hours\":[42721077,250190456,1152116541,561427787],\"weekDays\":[\"Wednesday\",\"Sunday\",\"Sunday\"],\"monthDays\":[1433827538,1439724963,1228605740],\"monthlyOccurrences\":[{\"day\":\"Monday\",\"occurrence\":1026545552,\"\":{\"zywxzxrohtqc\":\"dataeeuligunwyfkucv\",\"xdeaisk\":\"dataidspe\",\"yjnmdc\":\"datacmcqslngmsip\",\"cxacgeiyfpfaaah\":\"datatj\"}}],\"\":{\"pqgcadntzfjldnv\":\"datauplf\",\"oygcofh\":\"datapmez\",\"bgmxm\":\"datanehhi\",\"wuwbnngcdtxxyz\":\"dataxbaazn\"}}") + "{\"minutes\":[1651942817,1198550709,1034605947,2073508503],\"hours\":[761365450,2140656266,1965073356,292574962],\"weekDays\":[\"Friday\",\"Monday\"],\"monthDays\":[1518926475,520960902],\"monthlyOccurrences\":[{\"day\":\"Tuesday\",\"occurrence\":503666490,\"\":{\"mmtucazgrlvkdap\":\"datavlcunnbaijobcpru\",\"tkbzz\":\"datazemnja\"}},{\"day\":\"Monday\",\"occurrence\":1914403497,\"\":{\"r\":\"datajhmgocal\",\"nreukcrcsdaip\":\"datakmwyoukfim\"}},{\"day\":\"Wednesday\",\"occurrence\":714649368,\"\":{\"tbybcxgrrlzdn\":\"datav\",\"uqi\":\"datacxtqqpfgjny\",\"j\":\"dataoiuj\",\"lqtdcasjnzeckp\":\"datagsfvyvnpu\"}},{\"day\":\"Tuesday\",\"occurrence\":367076420,\"\":{\"lzicltwan\":\"datac\",\"inlqkzee\":\"databzycxvifkzspwvl\",\"kwxb\":\"datazelmmwmdhmd\",\"pnmnojfmztpw\":\"datavmc\"}}],\"\":{\"rfo\":\"datauntvyeyebw\"}}") .toObject(RecurrenceSchedule.class); - Assertions.assertEquals(838577333, model.minutes().get(0)); - Assertions.assertEquals(42721077, model.hours().get(0)); - Assertions.assertEquals(DaysOfWeek.WEDNESDAY, model.weekDays().get(0)); - Assertions.assertEquals(1433827538, model.monthDays().get(0)); - Assertions.assertEquals(DayOfWeek.MONDAY, model.monthlyOccurrences().get(0).day()); - Assertions.assertEquals(1026545552, model.monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(1651942817, model.minutes().get(0)); + Assertions.assertEquals(761365450, model.hours().get(0)); + Assertions.assertEquals(DaysOfWeek.FRIDAY, model.weekDays().get(0)); + Assertions.assertEquals(1518926475, model.monthDays().get(0)); + Assertions.assertEquals(DayOfWeek.TUESDAY, model.monthlyOccurrences().get(0).day()); + Assertions.assertEquals(503666490, model.monthlyOccurrences().get(0).occurrence()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { RecurrenceSchedule model - = new RecurrenceSchedule().withMinutes(Arrays.asList(838577333, 1015295902, 1361107137, 935311628)) - .withHours(Arrays.asList(42721077, 250190456, 1152116541, 561427787)) - .withWeekDays(Arrays.asList(DaysOfWeek.WEDNESDAY, DaysOfWeek.SUNDAY, DaysOfWeek.SUNDAY)) - .withMonthDays(Arrays.asList(1433827538, 1439724963, 1228605740)) - .withMonthlyOccurrences(Arrays.asList(new RecurrenceScheduleOccurrence().withDay(DayOfWeek.MONDAY) - .withOccurrence(1026545552).withAdditionalProperties(mapOf()))) + = new RecurrenceSchedule().withMinutes(Arrays.asList(1651942817, 1198550709, 1034605947, 2073508503)) + .withHours(Arrays.asList(761365450, 2140656266, 1965073356, 292574962)) + .withWeekDays(Arrays.asList(DaysOfWeek.FRIDAY, DaysOfWeek.MONDAY)) + .withMonthDays(Arrays.asList(1518926475, 520960902)) + .withMonthlyOccurrences(Arrays.asList( + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.TUESDAY).withOccurrence(503666490) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.MONDAY).withOccurrence(1914403497) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.WEDNESDAY).withOccurrence(714649368) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.TUESDAY).withOccurrence(367076420) + .withAdditionalProperties(mapOf()))) .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(RecurrenceSchedule.class); - Assertions.assertEquals(838577333, model.minutes().get(0)); - Assertions.assertEquals(42721077, model.hours().get(0)); - Assertions.assertEquals(DaysOfWeek.WEDNESDAY, model.weekDays().get(0)); - Assertions.assertEquals(1433827538, model.monthDays().get(0)); - Assertions.assertEquals(DayOfWeek.MONDAY, model.monthlyOccurrences().get(0).day()); - Assertions.assertEquals(1026545552, model.monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(1651942817, model.minutes().get(0)); + Assertions.assertEquals(761365450, model.hours().get(0)); + Assertions.assertEquals(DaysOfWeek.FRIDAY, model.weekDays().get(0)); + Assertions.assertEquals(1518926475, model.monthDays().get(0)); + Assertions.assertEquals(DayOfWeek.TUESDAY, model.monthlyOccurrences().get(0).day()); + Assertions.assertEquals(503666490, model.monthlyOccurrences().get(0).occurrence()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedirectIncompatibleRowSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedirectIncompatibleRowSettingsTests.java index 8f938a7c15288..67b6d1f34d3e7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedirectIncompatibleRowSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedirectIncompatibleRowSettingsTests.java @@ -13,14 +13,14 @@ public final class RedirectIncompatibleRowSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RedirectIncompatibleRowSettings model = BinaryData.fromString( - "{\"linkedServiceName\":\"datayod\",\"path\":\"datawnqbpxy\",\"\":{\"elsslfxej\":\"datatxzovbh\",\"jwmglgstrzfhe\":\"dataocsgigsabtxnd\",\"fhmljim\":\"datadzovkbcbefohny\"}}") + "{\"linkedServiceName\":\"databmzwlej\",\"path\":\"datayoonbualri\",\"\":{\"fmzmq\":\"databntopbabndw\"}}") .toObject(RedirectIncompatibleRowSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RedirectIncompatibleRowSettings model = new RedirectIncompatibleRowSettings().withLinkedServiceName("datayod") - .withPath("datawnqbpxy").withAdditionalProperties(mapOf()); + RedirectIncompatibleRowSettings model = new RedirectIncompatibleRowSettings() + .withLinkedServiceName("databmzwlej").withPath("datayoonbualri").withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(RedirectIncompatibleRowSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedshiftUnloadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedshiftUnloadSettingsTests.java index 3108720ab335e..748ae268f528e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedshiftUnloadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RedshiftUnloadSettingsTests.java @@ -15,18 +15,19 @@ public final class RedshiftUnloadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RedshiftUnloadSettings model = BinaryData.fromString( - "{\"s3LinkedServiceName\":{\"referenceName\":\"oeftrbxomaa\",\"parameters\":{\"xelquqzepgvj\":\"datarfqve\"}},\"bucketName\":\"datallzykalbau\"}") + "{\"s3LinkedServiceName\":{\"referenceName\":\"xcyai\",\"parameters\":{\"ssuuzfrwmct\":\"datatrdbtrkvluu\",\"c\":\"datangj\"}},\"bucketName\":\"datasfbkrtpu\"}") .toObject(RedshiftUnloadSettings.class); - Assertions.assertEquals("oeftrbxomaa", model.s3LinkedServiceName().referenceName()); + Assertions.assertEquals("xcyai", model.s3LinkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RedshiftUnloadSettings model = new RedshiftUnloadSettings().withS3LinkedServiceName(new LinkedServiceReference() - .withReferenceName("oeftrbxomaa").withParameters(mapOf("xelquqzepgvj", "datarfqve"))) - .withBucketName("datallzykalbau"); + RedshiftUnloadSettings model = new RedshiftUnloadSettings() + .withS3LinkedServiceName(new LinkedServiceReference().withReferenceName("xcyai") + .withParameters(mapOf("ssuuzfrwmct", "datatrdbtrkvluu", "c", "datangj"))) + .withBucketName("datasfbkrtpu"); model = BinaryData.fromObject(model).toObject(RedshiftUnloadSettings.class); - Assertions.assertEquals("oeftrbxomaa", model.s3LinkedServiceName().referenceName()); + Assertions.assertEquals("xcyai", model.s3LinkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RelationalSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RelationalSourceTests.java index 8964016324a46..ba9b534b7fc73 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RelationalSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RelationalSourceTests.java @@ -11,15 +11,16 @@ public final class RelationalSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RelationalSource model = BinaryData.fromString( - "{\"type\":\"RelationalSource\",\"query\":\"dataoysquygokhpstw\",\"additionalColumns\":\"dataigrhfe\",\"sourceRetryCount\":\"datay\",\"sourceRetryWait\":\"dataukcojyx\",\"maxConcurrentConnections\":\"datavoowrtcsucot\",\"disableMetricsCollection\":\"datayiqz\",\"\":{\"nmpvsblu\":\"datazv\",\"ih\":\"datafbhzukrpfb\"}}") + "{\"type\":\"RelationalSource\",\"query\":\"databckxennzow\",\"additionalColumns\":\"datairhexjpwyhmktpyk\",\"sourceRetryCount\":\"datacpkoamqfdtbao\",\"sourceRetryWait\":\"dataaofkcvhh\",\"maxConcurrentConnections\":\"datavkuuikrsi\",\"disableMetricsCollection\":\"datarwsj\",\"\":{\"jva\":\"dataen\",\"fm\":\"datadqgfvygrfyyknxua\",\"kt\":\"dataynlcimjmurocryfu\"}}") .toObject(RelationalSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RelationalSource model = new RelationalSource().withSourceRetryCount("datay").withSourceRetryWait("dataukcojyx") - .withMaxConcurrentConnections("datavoowrtcsucot").withDisableMetricsCollection("datayiqz") - .withQuery("dataoysquygokhpstw").withAdditionalColumns("dataigrhfe"); + RelationalSource model + = new RelationalSource().withSourceRetryCount("datacpkoamqfdtbao").withSourceRetryWait("dataaofkcvhh") + .withMaxConcurrentConnections("datavkuuikrsi").withDisableMetricsCollection("datarwsj") + .withQuery("databckxennzow").withAdditionalColumns("datairhexjpwyhmktpyk"); model = BinaryData.fromObject(model).toObject(RelationalSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTests.java index a49b012e63107..37f6f73115ec4 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTests.java @@ -14,24 +14,25 @@ public final class RerunTumblingWindowTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RerunTumblingWindowTrigger model = BinaryData.fromString( - "{\"type\":\"RerunTumblingWindowTrigger\",\"typeProperties\":{\"parentTrigger\":\"datayulo\",\"requestedStartTime\":\"2021-04-21T03:19:05Z\",\"requestedEndTime\":\"2021-09-26T14:48:01Z\",\"rerunConcurrency\":1007731752},\"description\":\"cxmxfw\",\"runtimeState\":\"Disabled\",\"annotations\":[\"dataekbmwizish\",\"datamx\"],\"\":{\"kcyf\":\"datanmwiybleaota\",\"sjndbwwnlg\":\"dataxosnbwbcnf\",\"wlixh\":\"dataoubtehdccghdzq\",\"nhlsf\":\"dataotqsp\"}}") + "{\"type\":\"RerunTumblingWindowTrigger\",\"typeProperties\":{\"parentTrigger\":\"datarhgcuejtxxlkokt\",\"requestedStartTime\":\"2021-06-18T21:25:37Z\",\"requestedEndTime\":\"2021-10-10T00:49:20Z\",\"rerunConcurrency\":1762125974},\"description\":\"wtwboxgrvsavoq\",\"runtimeState\":\"Started\",\"annotations\":[\"datakuszllognledhvll\",\"datanyg\",\"datamn\"],\"\":{\"lwdh\":\"dataaqtzop\"}}") .toObject(RerunTumblingWindowTrigger.class); - Assertions.assertEquals("cxmxfw", model.description()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-21T03:19:05Z"), model.requestedStartTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-26T14:48:01Z"), model.requestedEndTime()); - Assertions.assertEquals(1007731752, model.rerunConcurrency()); + Assertions.assertEquals("wtwboxgrvsavoq", model.description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-18T21:25:37Z"), model.requestedStartTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-10T00:49:20Z"), model.requestedEndTime()); + Assertions.assertEquals(1762125974, model.rerunConcurrency()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RerunTumblingWindowTrigger model = new RerunTumblingWindowTrigger().withDescription("cxmxfw") - .withAnnotations(Arrays.asList("dataekbmwizish", "datamx")).withParentTrigger("datayulo") - .withRequestedStartTime(OffsetDateTime.parse("2021-04-21T03:19:05Z")) - .withRequestedEndTime(OffsetDateTime.parse("2021-09-26T14:48:01Z")).withRerunConcurrency(1007731752); + RerunTumblingWindowTrigger model = new RerunTumblingWindowTrigger().withDescription("wtwboxgrvsavoq") + .withAnnotations(Arrays.asList("datakuszllognledhvll", "datanyg", "datamn")) + .withParentTrigger("datarhgcuejtxxlkokt") + .withRequestedStartTime(OffsetDateTime.parse("2021-06-18T21:25:37Z")) + .withRequestedEndTime(OffsetDateTime.parse("2021-10-10T00:49:20Z")).withRerunConcurrency(1762125974); model = BinaryData.fromObject(model).toObject(RerunTumblingWindowTrigger.class); - Assertions.assertEquals("cxmxfw", model.description()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-21T03:19:05Z"), model.requestedStartTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-26T14:48:01Z"), model.requestedEndTime()); - Assertions.assertEquals(1007731752, model.rerunConcurrency()); + Assertions.assertEquals("wtwboxgrvsavoq", model.description()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-18T21:25:37Z"), model.requestedStartTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-10T00:49:20Z"), model.requestedEndTime()); + Assertions.assertEquals(1762125974, model.rerunConcurrency()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTypePropertiesTests.java index 7f78d8648042a..2947f51730990 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RerunTumblingWindowTriggerTypePropertiesTests.java @@ -13,21 +13,21 @@ public final class RerunTumblingWindowTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RerunTumblingWindowTriggerTypeProperties model = BinaryData.fromString( - "{\"parentTrigger\":\"datafjwajsbqytfv\",\"requestedStartTime\":\"2021-04-12T06:16:03Z\",\"requestedEndTime\":\"2021-01-30T03:23:45Z\",\"rerunConcurrency\":1573229736}") + "{\"parentTrigger\":\"databvhflbchzob\",\"requestedStartTime\":\"2021-11-28T06:35:22Z\",\"requestedEndTime\":\"2021-01-04T14:44:46Z\",\"rerunConcurrency\":345398232}") .toObject(RerunTumblingWindowTriggerTypeProperties.class); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-12T06:16:03Z"), model.requestedStartTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-30T03:23:45Z"), model.requestedEndTime()); - Assertions.assertEquals(1573229736, model.rerunConcurrency()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-28T06:35:22Z"), model.requestedStartTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-04T14:44:46Z"), model.requestedEndTime()); + Assertions.assertEquals(345398232, model.rerunConcurrency()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { RerunTumblingWindowTriggerTypeProperties model = new RerunTumblingWindowTriggerTypeProperties() - .withParentTrigger("datafjwajsbqytfv").withRequestedStartTime(OffsetDateTime.parse("2021-04-12T06:16:03Z")) - .withRequestedEndTime(OffsetDateTime.parse("2021-01-30T03:23:45Z")).withRerunConcurrency(1573229736); + .withParentTrigger("databvhflbchzob").withRequestedStartTime(OffsetDateTime.parse("2021-11-28T06:35:22Z")) + .withRequestedEndTime(OffsetDateTime.parse("2021-01-04T14:44:46Z")).withRerunConcurrency(345398232); model = BinaryData.fromObject(model).toObject(RerunTumblingWindowTriggerTypeProperties.class); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-12T06:16:03Z"), model.requestedStartTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-30T03:23:45Z"), model.requestedEndTime()); - Assertions.assertEquals(1573229736, model.rerunConcurrency()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-28T06:35:22Z"), model.requestedStartTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-04T14:44:46Z"), model.requestedEndTime()); + Assertions.assertEquals(345398232, model.rerunConcurrency()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ResponsysSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ResponsysSourceTests.java index 9b564eddd5bcb..97f3e4a51bd07 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ResponsysSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ResponsysSourceTests.java @@ -11,16 +11,16 @@ public final class ResponsysSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ResponsysSource model = BinaryData.fromString( - "{\"type\":\"ResponsysSource\",\"query\":\"datajsf\",\"queryTimeout\":\"datayqrrsjqtdj\",\"additionalColumns\":\"datadm\",\"sourceRetryCount\":\"dataiml\",\"sourceRetryWait\":\"datazezcivkzrvy\",\"maxConcurrentConnections\":\"datalqgyu\",\"disableMetricsCollection\":\"dataoelyjduzapn\",\"\":{\"l\":\"datatodprrqca\",\"mzuwrcehfwce\":\"datatxjcbdpc\"}}") + "{\"type\":\"ResponsysSource\",\"query\":\"datacgcie\",\"queryTimeout\":\"datax\",\"additionalColumns\":\"datazvnghtknr\",\"sourceRetryCount\":\"datahysnmyuvf\",\"sourceRetryWait\":\"datacnrapxw\",\"maxConcurrentConnections\":\"datapxoelfobehr\",\"disableMetricsCollection\":\"dataglojjcziytf\",\"\":{\"nkms\":\"datairmbrdognqa\",\"dzvuhw\":\"dataybh\"}}") .toObject(ResponsysSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ResponsysSource model - = new ResponsysSource().withSourceRetryCount("dataiml").withSourceRetryWait("datazezcivkzrvy") - .withMaxConcurrentConnections("datalqgyu").withDisableMetricsCollection("dataoelyjduzapn") - .withQueryTimeout("datayqrrsjqtdj").withAdditionalColumns("datadm").withQuery("datajsf"); + = new ResponsysSource().withSourceRetryCount("datahysnmyuvf").withSourceRetryWait("datacnrapxw") + .withMaxConcurrentConnections("datapxoelfobehr").withDisableMetricsCollection("dataglojjcziytf") + .withQueryTimeout("datax").withAdditionalColumns("datazvnghtknr").withQuery("datacgcie"); model = BinaryData.fromObject(model).toObject(ResponsysSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTests.java index c1ccc55947888..de73764250ada 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTests.java @@ -19,35 +19,33 @@ public final class RestResourceDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RestResourceDataset model = BinaryData.fromString( - "{\"type\":\"RestResource\",\"typeProperties\":{\"relativeUrl\":\"dataiarz\",\"requestMethod\":\"datadqseypdlmajpuy\",\"requestBody\":\"dataa\",\"additionalHeaders\":{\"uvmsie\":\"datazgccyn\",\"parxtzayq\":\"dataedmmvoneeyr\"},\"paginationRules\":{\"z\":\"dataigeblsp\",\"eozbj\":\"datassiwwv\"}},\"description\":\"qpizdnuehxw\",\"structure\":\"datassjdywbnklg\",\"schema\":\"dataxa\",\"linkedServiceName\":{\"referenceName\":\"tsawv\",\"parameters\":{\"jrmplzmsl\":\"datampt\",\"vrrllfswarmyb\":\"databnknyfuysj\"}},\"parameters\":{\"cbfnxiajuv\":{\"type\":\"Float\",\"defaultValue\":\"datageysyqnipehfw\"},\"zguaxfhvjixgofqd\":{\"type\":\"Bool\",\"defaultValue\":\"datafjisosfzlnraxnf\"},\"uvrqpbxdoicqp\":{\"type\":\"Array\",\"defaultValue\":\"datajmi\"}},\"annotations\":[\"datalydp\"],\"folder\":{\"name\":\"nsbmzjritukoym\"},\"\":{\"ndu\":\"dataexmizzjxwjoqfzw\"}}") + "{\"type\":\"RestResource\",\"typeProperties\":{\"relativeUrl\":\"databbfpxxa\",\"requestMethod\":\"dataoz\",\"requestBody\":\"datagsnuhwy\",\"additionalHeaders\":{\"axcebnbeosk\":\"dataf\",\"v\":\"datamqqerwqxpj\",\"zmhytebjkjgee\":\"datamdfkhttuobr\"},\"paginationRules\":{\"twofxfmhlvy\":\"datafmabvbmn\"}},\"description\":\"sl\",\"structure\":\"datarml\",\"schema\":\"dataiekhjgqq\",\"linkedServiceName\":{\"referenceName\":\"ugwespscvsmsp\",\"parameters\":{\"upcvq\":\"datawozfvza\",\"cgmlmpn\":\"dataxcvwioqhc\"}},\"parameters\":{\"hdqseyp\":{\"type\":\"Array\",\"defaultValue\":\"dataiarz\"}},\"annotations\":[\"dataajpuyx\",\"dataa\",\"datarmzgccynb\",\"datavmsiehedm\"],\"folder\":{\"name\":\"neeyrxparxtz\"},\"\":{\"lzdssi\":\"datawddigebls\",\"p\":\"datawveeozbjkj\",\"dyw\":\"datazdnuehxwltss\"}}") .toObject(RestResourceDataset.class); - Assertions.assertEquals("qpizdnuehxw", model.description()); - Assertions.assertEquals("tsawv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("cbfnxiajuv").type()); - Assertions.assertEquals("nsbmzjritukoym", model.folder().name()); + Assertions.assertEquals("sl", model.description()); + Assertions.assertEquals("ugwespscvsmsp", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("hdqseyp").type()); + Assertions.assertEquals("neeyrxparxtz", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RestResourceDataset model = new RestResourceDataset().withDescription("qpizdnuehxw") - .withStructure("datassjdywbnklg").withSchema("dataxa") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("tsawv") - .withParameters(mapOf("jrmplzmsl", "datampt", "vrrllfswarmyb", "databnknyfuysj"))) - .withParameters(mapOf("cbfnxiajuv", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datageysyqnipehfw"), - "zguaxfhvjixgofqd", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datafjisosfzlnraxnf"), - "uvrqpbxdoicqp", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datajmi"))) - .withAnnotations(Arrays.asList("datalydp")).withFolder(new DatasetFolder().withName("nsbmzjritukoym")) - .withRelativeUrl("dataiarz").withRequestMethod("datadqseypdlmajpuy").withRequestBody("dataa") - .withAdditionalHeaders(mapOf("uvmsie", "datazgccyn", "parxtzayq", "dataedmmvoneeyr")) - .withPaginationRules(mapOf("z", "dataigeblsp", "eozbj", "datassiwwv")); + RestResourceDataset model + = new RestResourceDataset().withDescription("sl").withStructure("datarml").withSchema("dataiekhjgqq") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ugwespscvsmsp") + .withParameters(mapOf("upcvq", "datawozfvza", "cgmlmpn", "dataxcvwioqhc"))) + .withParameters(mapOf("hdqseyp", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataiarz"))) + .withAnnotations(Arrays.asList("dataajpuyx", "dataa", "datarmzgccynb", "datavmsiehedm")) + .withFolder(new DatasetFolder().withName("neeyrxparxtz")).withRelativeUrl("databbfpxxa") + .withRequestMethod("dataoz").withRequestBody("datagsnuhwy") + .withAdditionalHeaders( + mapOf("axcebnbeosk", "dataf", "v", "datamqqerwqxpj", "zmhytebjkjgee", "datamdfkhttuobr")) + .withPaginationRules(mapOf("twofxfmhlvy", "datafmabvbmn")); model = BinaryData.fromObject(model).toObject(RestResourceDataset.class); - Assertions.assertEquals("qpizdnuehxw", model.description()); - Assertions.assertEquals("tsawv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("cbfnxiajuv").type()); - Assertions.assertEquals("nsbmzjritukoym", model.folder().name()); + Assertions.assertEquals("sl", model.description()); + Assertions.assertEquals("ugwespscvsmsp", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("hdqseyp").type()); + Assertions.assertEquals("neeyrxparxtz", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTypePropertiesTests.java index bce7f49730a54..1cfb64f13b45e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestResourceDatasetTypePropertiesTests.java @@ -13,18 +13,17 @@ public final class RestResourceDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RestResourceDatasetTypeProperties model = BinaryData.fromString( - "{\"relativeUrl\":\"datauwdvolxt\",\"requestMethod\":\"dataricdsflzbkiumj\",\"requestBody\":\"dataoxedrmrazhvch\",\"additionalHeaders\":{\"ntnwzruzso\":\"datayiog\",\"lcappnvcebspci\":\"datawxcsmx\",\"mzkwhjjsqwhae\":\"datayomhkdwuwedupb\"},\"paginationRules\":{\"a\":\"datavvkxdbnmc\"}}") + "{\"relativeUrl\":\"dataklgerxactsawv\",\"requestMethod\":\"dataimpthj\",\"requestBody\":\"dataplzmslubnk\",\"additionalHeaders\":{\"lfswarmybwmro\":\"datauysjhvrr\",\"cbfnxiajuv\":\"datageysyqnipehfw\"},\"paginationRules\":{\"zguaxfhvjixgofqd\":\"datafjisosfzlnraxnf\",\"jmi\":\"dataw\",\"ntlydprpensbmzj\":\"datauvrqpbxdoicqp\",\"hbfexmizz\":\"dataitukoy\"}}") .toObject(RestResourceDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { RestResourceDatasetTypeProperties model = new RestResourceDatasetTypeProperties() - .withRelativeUrl("datauwdvolxt").withRequestMethod("dataricdsflzbkiumj") - .withRequestBody("dataoxedrmrazhvch") - .withAdditionalHeaders( - mapOf("ntnwzruzso", "datayiog", "lcappnvcebspci", "datawxcsmx", "mzkwhjjsqwhae", "datayomhkdwuwedupb")) - .withPaginationRules(mapOf("a", "datavvkxdbnmc")); + .withRelativeUrl("dataklgerxactsawv").withRequestMethod("dataimpthj").withRequestBody("dataplzmslubnk") + .withAdditionalHeaders(mapOf("lfswarmybwmro", "datauysjhvrr", "cbfnxiajuv", "datageysyqnipehfw")) + .withPaginationRules(mapOf("zguaxfhvjixgofqd", "datafjisosfzlnraxnf", "jmi", "dataw", "ntlydprpensbmzj", + "datauvrqpbxdoicqp", "hbfexmizz", "dataitukoy")); model = BinaryData.fromObject(model).toObject(RestResourceDatasetTypeProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSinkTests.java index 498a0ad646335..bbe699dfe0fd6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSinkTests.java @@ -11,17 +11,18 @@ public final class RestSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RestSink model = BinaryData.fromString( - "{\"type\":\"RestSink\",\"requestMethod\":\"dataeyrnbubyabtowbu\",\"additionalHeaders\":\"datalwbgvzuxfsmf\",\"httpRequestTimeout\":\"datazuoq\",\"requestInterval\":\"datawfvnoylmfj\",\"httpCompressionType\":\"datahvpljzrq\",\"writeBatchSize\":\"datatswemot\",\"writeBatchTimeout\":\"dataejyavkyjvctq\",\"sinkRetryCount\":\"datacz\",\"sinkRetryWait\":\"datapaeyklxsvcbr\",\"maxConcurrentConnections\":\"datalt\",\"disableMetricsCollection\":\"datamdsngoaofmrph\",\"\":{\"exibo\":\"datafrunkcgdnha\"}}") + "{\"type\":\"RestSink\",\"requestMethod\":\"datawwtlerhpfrarqnj\",\"additionalHeaders\":\"datahsxhtvnq\",\"httpRequestTimeout\":\"datarrgmlw\",\"requestInterval\":\"datam\",\"httpCompressionType\":\"datacsddlcnwbijxf\",\"writeBatchSize\":\"datageffrghwdmr\",\"writeBatchTimeout\":\"datahrr\",\"sinkRetryCount\":\"datavdrggucwa\",\"sinkRetryWait\":\"datam\",\"maxConcurrentConnections\":\"dataklzomdfcp\",\"disableMetricsCollection\":\"dataimijzhrbs\",\"\":{\"qfbgeblp\":\"dataublouelf\",\"swgfjrg\":\"datawckmnpzubzq\"}}") .toObject(RestSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RestSink model = new RestSink().withWriteBatchSize("datatswemot").withWriteBatchTimeout("dataejyavkyjvctq") - .withSinkRetryCount("datacz").withSinkRetryWait("datapaeyklxsvcbr").withMaxConcurrentConnections("datalt") - .withDisableMetricsCollection("datamdsngoaofmrph").withRequestMethod("dataeyrnbubyabtowbu") - .withAdditionalHeaders("datalwbgvzuxfsmf").withHttpRequestTimeout("datazuoq") - .withRequestInterval("datawfvnoylmfj").withHttpCompressionType("datahvpljzrq"); + RestSink model = new RestSink().withWriteBatchSize("datageffrghwdmr").withWriteBatchTimeout("datahrr") + .withSinkRetryCount("datavdrggucwa").withSinkRetryWait("datam") + .withMaxConcurrentConnections("dataklzomdfcp").withDisableMetricsCollection("dataimijzhrbs") + .withRequestMethod("datawwtlerhpfrarqnj").withAdditionalHeaders("datahsxhtvnq") + .withHttpRequestTimeout("datarrgmlw").withRequestInterval("datam") + .withHttpCompressionType("datacsddlcnwbijxf"); model = BinaryData.fromObject(model).toObject(RestSink.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSourceTests.java index 4062a22a1733c..6b7b4373b04fd 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RestSourceTests.java @@ -11,18 +11,18 @@ public final class RestSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RestSource model = BinaryData.fromString( - "{\"type\":\"RestSource\",\"requestMethod\":\"datarjooepfb\",\"requestBody\":\"databffxansgntjmnl\",\"additionalHeaders\":\"datalrjdkyp\",\"paginationRules\":\"datavilgn\",\"httpRequestTimeout\":\"datatjbldgikokjwgej\",\"requestInterval\":\"datauzezwnqhcpkjgsy\",\"additionalColumns\":\"datadt\",\"sourceRetryCount\":\"datamq\",\"sourceRetryWait\":\"datatk\",\"maxConcurrentConnections\":\"datarourtmccdejtoypl\",\"disableMetricsCollection\":\"datavjutckfhmdcvlb\",\"\":{\"mzcxlgmuhxw\":\"datazvujp\",\"hhvvmvsbgyqdhaz\":\"datakbnhmdtjczpfoisp\",\"sovo\":\"datacugg\"}}") + "{\"type\":\"RestSource\",\"requestMethod\":\"datacpbshqzzlcfe\",\"requestBody\":\"dataryxnklfswzsyigx\",\"additionalHeaders\":\"dataxhygc\",\"paginationRules\":\"databapeuqyz\",\"httpRequestTimeout\":\"datasuopcdiaossp\",\"requestInterval\":\"datatgkmrsqaqgllnhgi\",\"additionalColumns\":\"datawzzk\",\"sourceRetryCount\":\"dataqrngl\",\"sourceRetryWait\":\"datatu\",\"maxConcurrentConnections\":\"datafwdkpadktsyy\",\"disableMetricsCollection\":\"dataojrfqtfk\",\"\":{\"xokiffqpwdyzset\":\"datapmdajqpdvvzbej\",\"oij\":\"datamvtqhn\",\"g\":\"datacprkqywyb\",\"lisvqfblsizxpolp\":\"datayomsetzc\"}}") .toObject(RestSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RestSource model = new RestSource().withSourceRetryCount("datamq").withSourceRetryWait("datatk") - .withMaxConcurrentConnections("datarourtmccdejtoypl").withDisableMetricsCollection("datavjutckfhmdcvlb") - .withRequestMethod("datarjooepfb").withRequestBody("databffxansgntjmnl") - .withAdditionalHeaders("datalrjdkyp").withPaginationRules("datavilgn") - .withHttpRequestTimeout("datatjbldgikokjwgej").withRequestInterval("datauzezwnqhcpkjgsy") - .withAdditionalColumns("datadt"); + RestSource model = new RestSource().withSourceRetryCount("dataqrngl").withSourceRetryWait("datatu") + .withMaxConcurrentConnections("datafwdkpadktsyy").withDisableMetricsCollection("dataojrfqtfk") + .withRequestMethod("datacpbshqzzlcfe").withRequestBody("dataryxnklfswzsyigx") + .withAdditionalHeaders("dataxhygc").withPaginationRules("databapeuqyz") + .withHttpRequestTimeout("datasuopcdiaossp").withRequestInterval("datatgkmrsqaqgllnhgi") + .withAdditionalColumns("datawzzk"); model = BinaryData.fromObject(model).toObject(RestSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RetryPolicyTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RetryPolicyTests.java index 31e661164efc0..f4e79866f6570 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RetryPolicyTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/RetryPolicyTests.java @@ -11,15 +11,15 @@ public final class RetryPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - RetryPolicy model = BinaryData.fromString("{\"count\":\"datagwxpu\",\"intervalInSeconds\":1254021355}") + RetryPolicy model = BinaryData.fromString("{\"count\":\"datazws\",\"intervalInSeconds\":1398360325}") .toObject(RetryPolicy.class); - Assertions.assertEquals(1254021355, model.intervalInSeconds()); + Assertions.assertEquals(1398360325, model.intervalInSeconds()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RetryPolicy model = new RetryPolicy().withCount("datagwxpu").withIntervalInSeconds(1254021355); + RetryPolicy model = new RetryPolicy().withCount("datazws").withIntervalInSeconds(1398360325); model = BinaryData.fromObject(model).toObject(RetryPolicy.class); - Assertions.assertEquals(1254021355, model.intervalInSeconds()); + Assertions.assertEquals(1398360325, model.intervalInSeconds()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceMarketingCloudSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceMarketingCloudSourceTests.java index 9a44fe75c66e5..862fb36b66292 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceMarketingCloudSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceMarketingCloudSourceTests.java @@ -11,16 +11,16 @@ public final class SalesforceMarketingCloudSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceMarketingCloudSource model = BinaryData.fromString( - "{\"type\":\"SalesforceMarketingCloudSource\",\"query\":\"datadgnchahldnrpt\",\"queryTimeout\":\"datacapbkfvowzbk\",\"additionalColumns\":\"dataqpzdpu\",\"sourceRetryCount\":\"datawjmohqzzkplqmca\",\"sourceRetryWait\":\"dataeiauveeng\",\"maxConcurrentConnections\":\"datagfpqwwugfw\",\"disableMetricsCollection\":\"datajcewbqaibkyeys\",\"\":{\"xcjqdvcieqzkuixf\":\"datahdydyybztlylh\",\"rzieyxxidab\":\"dataghvecjhbttmhneq\",\"cseqo\":\"datalakkna\"}}") + "{\"type\":\"SalesforceMarketingCloudSource\",\"query\":\"dataiwvwmc\",\"queryTimeout\":\"datayoestplmytnhvyj\",\"additionalColumns\":\"dataerh\",\"sourceRetryCount\":\"datastiawywppq\",\"sourceRetryWait\":\"datajxbdyczplmljcisx\",\"maxConcurrentConnections\":\"datas\",\"disableMetricsCollection\":\"datayt\",\"\":{\"zantahuykxsjymps\":\"dataufdynhq\",\"adwiqnsmpfeyjvl\":\"datam\",\"ryoleqikcorkem\":\"dataqsy\",\"htbtuvwz\":\"databobx\"}}") .toObject(SalesforceMarketingCloudSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SalesforceMarketingCloudSource model = new SalesforceMarketingCloudSource() - .withSourceRetryCount("datawjmohqzzkplqmca").withSourceRetryWait("dataeiauveeng") - .withMaxConcurrentConnections("datagfpqwwugfw").withDisableMetricsCollection("datajcewbqaibkyeys") - .withQueryTimeout("datacapbkfvowzbk").withAdditionalColumns("dataqpzdpu").withQuery("datadgnchahldnrpt"); + .withSourceRetryCount("datastiawywppq").withSourceRetryWait("datajxbdyczplmljcisx") + .withMaxConcurrentConnections("datas").withDisableMetricsCollection("datayt") + .withQueryTimeout("datayoestplmytnhvyj").withAdditionalColumns("dataerh").withQuery("dataiwvwmc"); model = BinaryData.fromObject(model).toObject(SalesforceMarketingCloudSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTests.java index f93a0451f180b..fda6e8d790f0e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTests.java @@ -19,33 +19,30 @@ public final class SalesforceObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceObjectDataset model = BinaryData.fromString( - "{\"type\":\"SalesforceObject\",\"typeProperties\":{\"objectApiName\":\"dataryszfhdxyfh\"},\"description\":\"hzbzhhavz\",\"structure\":\"dataxnvkdslcofuvtfue\",\"schema\":\"datauisaklhjfddxqfu\",\"linkedServiceName\":{\"referenceName\":\"subzsspmj\",\"parameters\":{\"wbztrt\":\"datalfauyvxpqwlkqd\",\"ffjdhgslormhbt\":\"dataldwvog\",\"sdylmnq\":\"datafcvxkylhc\"}},\"parameters\":{\"bgbh\":{\"type\":\"Object\",\"defaultValue\":\"databptmsgkwedwlxtzh\"},\"pkwmamrlfizjud\":{\"type\":\"SecureString\",\"defaultValue\":\"datarpjimvrrqfi\"},\"pngyhylqyafe\":{\"type\":\"String\",\"defaultValue\":\"dataih\"},\"u\":{\"type\":\"Int\",\"defaultValue\":\"dataodx\"}},\"annotations\":[\"dataxnxrqxrtzeargv\"],\"folder\":{\"name\":\"hbjhmvpjxsd\"},\"\":{\"ynepkt\":\"dataignybffqcw\",\"conyse\":\"datamwg\",\"ouoxfalo\":\"datajijfhpxni\"}}") + "{\"type\":\"SalesforceObject\",\"typeProperties\":{\"objectApiName\":\"datahgbgbhudh\"},\"description\":\"jimvrrq\",\"structure\":\"databpk\",\"schema\":\"dataamrlfizjuddnd\",\"linkedServiceName\":{\"referenceName\":\"hupngyhylqyafew\",\"parameters\":{\"xnxrqxrtzeargv\":\"datadxwuuy\"}},\"parameters\":{\"eignybffqc\":{\"type\":\"Int\",\"defaultValue\":\"datajhmvpjxsdh\"},\"tvmwgvconyse\":{\"type\":\"Bool\",\"defaultValue\":\"datanep\"}},\"annotations\":[\"datajfhpxnikouo\",\"datafalo\",\"databskkypor\",\"dataynieunbydlgfaphw\"],\"folder\":{\"name\":\"wtsaynrtvj\"},\"\":{\"hsdbfbm\":\"dataeeoxvqjmrnbl\",\"zmiaoaweacf\":\"dataivixzhpjg\"}}") .toObject(SalesforceObjectDataset.class); - Assertions.assertEquals("hzbzhhavz", model.description()); - Assertions.assertEquals("subzsspmj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("bgbh").type()); - Assertions.assertEquals("hbjhmvpjxsd", model.folder().name()); + Assertions.assertEquals("jimvrrq", model.description()); + Assertions.assertEquals("hupngyhylqyafew", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("eignybffqc").type()); + Assertions.assertEquals("wtsaynrtvj", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceObjectDataset model = new SalesforceObjectDataset().withDescription("hzbzhhavz") - .withStructure("dataxnvkdslcofuvtfue").withSchema("datauisaklhjfddxqfu") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("subzsspmj").withParameters( - mapOf("wbztrt", "datalfauyvxpqwlkqd", "ffjdhgslormhbt", "dataldwvog", "sdylmnq", "datafcvxkylhc"))) - .withParameters(mapOf("bgbh", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("databptmsgkwedwlxtzh"), - "pkwmamrlfizjud", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datarpjimvrrqfi"), - "pngyhylqyafe", new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataih"), - "u", new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataodx"))) - .withAnnotations(Arrays.asList("dataxnxrqxrtzeargv")) - .withFolder(new DatasetFolder().withName("hbjhmvpjxsd")).withObjectApiName("dataryszfhdxyfh"); + SalesforceObjectDataset model = new SalesforceObjectDataset().withDescription("jimvrrq") + .withStructure("databpk").withSchema("dataamrlfizjuddnd") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("hupngyhylqyafew") + .withParameters(mapOf("xnxrqxrtzeargv", "datadxwuuy"))) + .withParameters(mapOf("eignybffqc", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datajhmvpjxsdh"), + "tvmwgvconyse", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datanep"))) + .withAnnotations(Arrays.asList("datajfhpxnikouo", "datafalo", "databskkypor", "dataynieunbydlgfaphw")) + .withFolder(new DatasetFolder().withName("wtsaynrtvj")).withObjectApiName("datahgbgbhudh"); model = BinaryData.fromObject(model).toObject(SalesforceObjectDataset.class); - Assertions.assertEquals("hzbzhhavz", model.description()); - Assertions.assertEquals("subzsspmj", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("bgbh").type()); - Assertions.assertEquals("hbjhmvpjxsd", model.folder().name()); + Assertions.assertEquals("jimvrrq", model.description()); + Assertions.assertEquals("hupngyhylqyafew", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("eignybffqc").type()); + Assertions.assertEquals("wtsaynrtvj", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTypePropertiesTests.java index c8f5ca2194d70..2aaa58786aa30 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceObjectDatasetTypePropertiesTests.java @@ -10,14 +10,15 @@ public final class SalesforceObjectDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SalesforceObjectDatasetTypeProperties model = BinaryData.fromString("{\"objectApiName\":\"dataskk\"}") - .toObject(SalesforceObjectDatasetTypeProperties.class); + SalesforceObjectDatasetTypeProperties model + = BinaryData.fromString("{\"objectApiName\":\"dataubuhruetcnxriqz\"}") + .toObject(SalesforceObjectDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SalesforceObjectDatasetTypeProperties model - = new SalesforceObjectDatasetTypeProperties().withObjectApiName("dataskk"); + = new SalesforceObjectDatasetTypeProperties().withObjectApiName("dataubuhruetcnxriqz"); model = BinaryData.fromObject(model).toObject(SalesforceObjectDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTests.java index ab97e43cf2659..360168fcd5a98 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTests.java @@ -19,30 +19,31 @@ public final class SalesforceServiceCloudObjectDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceServiceCloudObjectDataset model = BinaryData.fromString( - "{\"type\":\"SalesforceServiceCloudObject\",\"typeProperties\":{\"objectApiName\":\"datar\"},\"description\":\"nieu\",\"structure\":\"dataydlgfap\",\"schema\":\"datauubwts\",\"linkedServiceName\":{\"referenceName\":\"yn\",\"parameters\":{\"vqjmrnblihs\":\"datajfqreeo\",\"divixzhpjgqzmiao\":\"databfb\",\"ruetcnx\":\"dataweacfxaubu\",\"nowobwx\":\"dataiqzzdckhsqdrrjsu\"}},\"parameters\":{\"zheahuv\":{\"type\":\"String\",\"defaultValue\":\"datakohlsfjfouqj\"}},\"annotations\":[\"dataqkvadmjhymud\",\"datamaajzd\",\"databhsermclyqwwu\",\"datayqkaaptb\"],\"folder\":{\"name\":\"kb\"},\"\":{\"shvqnpszbeuyb\":\"datatwybloccuhplxzbn\",\"zjfjtvpey\":\"datatc\",\"jgpqfk\":\"datadyuxurxrltqmm\",\"xgwpq\":\"datanaeikczscymqf\"}}") + "{\"type\":\"SalesforceServiceCloudObject\",\"typeProperties\":{\"objectApiName\":\"datak\"},\"description\":\"qdrrj\",\"structure\":\"datar\",\"schema\":\"datawobwxrxm\",\"linkedServiceName\":{\"referenceName\":\"okohlsfj\",\"parameters\":{\"huv\":\"dataqjpzhe\",\"dmjhymudjma\":\"dataqxqkv\"}},\"parameters\":{\"yqkaaptb\":{\"type\":\"Array\",\"defaultValue\":\"databhsermclyqwwu\"},\"bptw\":{\"type\":\"Array\",\"defaultValue\":\"datakb\"},\"u\":{\"type\":\"Bool\",\"defaultValue\":\"dataoc\"}},\"annotations\":[\"dataxzbnss\",\"datavqnpszbeuybut\",\"datadzjfjtvpeyxdyuxu\"],\"folder\":{\"name\":\"ltqmmij\"},\"\":{\"xgwpq\":\"datafkwnaeikczscymqf\",\"mzapdokez\":\"datay\",\"knfzqnzbflbqmhb\":\"datape\",\"ea\":\"datayxxvwedhagqbbse\"}}") .toObject(SalesforceServiceCloudObjectDataset.class); - Assertions.assertEquals("nieu", model.description()); - Assertions.assertEquals("yn", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.STRING, model.parameters().get("zheahuv").type()); - Assertions.assertEquals("kb", model.folder().name()); + Assertions.assertEquals("qdrrj", model.description()); + Assertions.assertEquals("okohlsfj", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("yqkaaptb").type()); + Assertions.assertEquals("ltqmmij", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceServiceCloudObjectDataset model = new SalesforceServiceCloudObjectDataset().withDescription("nieu") - .withStructure("dataydlgfap").withSchema("datauubwts") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("yn") - .withParameters(mapOf("vqjmrnblihs", "datajfqreeo", "divixzhpjgqzmiao", "databfb", "ruetcnx", - "dataweacfxaubu", "nowobwx", "dataiqzzdckhsqdrrjsu"))) - .withParameters(mapOf("zheahuv", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datakohlsfjfouqj"))) - .withAnnotations(Arrays.asList("dataqkvadmjhymud", "datamaajzd", "databhsermclyqwwu", "datayqkaaptb")) - .withFolder(new DatasetFolder().withName("kb")).withObjectApiName("datar"); + SalesforceServiceCloudObjectDataset model = new SalesforceServiceCloudObjectDataset().withDescription("qdrrj") + .withStructure("datar").withSchema("datawobwxrxm") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("okohlsfj") + .withParameters(mapOf("huv", "dataqjpzhe", "dmjhymudjma", "dataqxqkv"))) + .withParameters(mapOf("yqkaaptb", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("databhsermclyqwwu"), + "bptw", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datakb"), "u", + new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataoc"))) + .withAnnotations(Arrays.asList("dataxzbnss", "datavqnpszbeuybut", "datadzjfjtvpeyxdyuxu")) + .withFolder(new DatasetFolder().withName("ltqmmij")).withObjectApiName("datak"); model = BinaryData.fromObject(model).toObject(SalesforceServiceCloudObjectDataset.class); - Assertions.assertEquals("nieu", model.description()); - Assertions.assertEquals("yn", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.STRING, model.parameters().get("zheahuv").type()); - Assertions.assertEquals("kb", model.folder().name()); + Assertions.assertEquals("qdrrj", model.description()); + Assertions.assertEquals("okohlsfj", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("yqkaaptb").type()); + Assertions.assertEquals("ltqmmij", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTypePropertiesTests.java index bcf10f29ba666..06283514f0829 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudObjectDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class SalesforceServiceCloudObjectDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceServiceCloudObjectDatasetTypeProperties model - = BinaryData.fromString("{\"objectApiName\":\"dataumz\"}") + = BinaryData.fromString("{\"objectApiName\":\"dataflmsy\"}") .toObject(SalesforceServiceCloudObjectDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SalesforceServiceCloudObjectDatasetTypeProperties model - = new SalesforceServiceCloudObjectDatasetTypeProperties().withObjectApiName("dataumz"); + = new SalesforceServiceCloudObjectDatasetTypeProperties().withObjectApiName("dataflmsy"); model = BinaryData.fromObject(model).toObject(SalesforceServiceCloudObjectDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSinkTests.java index a0ced11113b1a..daf2199ed0259 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSinkTests.java @@ -13,18 +13,18 @@ public final class SalesforceServiceCloudSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceServiceCloudSink model = BinaryData.fromString( - "{\"type\":\"SalesforceServiceCloudSink\",\"writeBehavior\":\"Insert\",\"externalIdFieldName\":\"datadyzilocn\",\"ignoreNullValues\":\"datatnhi\",\"writeBatchSize\":\"dataegjvhyvnqbhcl\",\"writeBatchTimeout\":\"dataksoqzzyrovvnac\",\"sinkRetryCount\":\"datajmhclhcqcjngwmb\",\"sinkRetryWait\":\"datagsidasovlrj\",\"maxConcurrentConnections\":\"datavyd\",\"disableMetricsCollection\":\"dataebbacscirztsa\",\"\":{\"whsfhecpstfe\":\"databk\",\"qmlgsghcnybhvzl\":\"databsl\",\"sszoqjbnfaxcd\":\"databgwjaepjmkruzo\"}}") + "{\"type\":\"SalesforceServiceCloudSink\",\"writeBehavior\":\"Insert\",\"externalIdFieldName\":\"dataskjbasmrdpbmoq\",\"ignoreNullValues\":\"datavukgfzbykapmeo\",\"writeBatchSize\":\"datapvma\",\"writeBatchTimeout\":\"datatgpnyu\",\"sinkRetryCount\":\"datawymyewb\",\"sinkRetryWait\":\"dataxwv\",\"maxConcurrentConnections\":\"datatjsnjbahxyfd\",\"disableMetricsCollection\":\"databtksrdjhqcrmptj\",\"\":{\"pjracyxnzadfl\":\"dataaw\"}}") .toObject(SalesforceServiceCloudSink.class); Assertions.assertEquals(SalesforceSinkWriteBehavior.INSERT, model.writeBehavior()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceServiceCloudSink model = new SalesforceServiceCloudSink().withWriteBatchSize("dataegjvhyvnqbhcl") - .withWriteBatchTimeout("dataksoqzzyrovvnac").withSinkRetryCount("datajmhclhcqcjngwmb") - .withSinkRetryWait("datagsidasovlrj").withMaxConcurrentConnections("datavyd") - .withDisableMetricsCollection("dataebbacscirztsa").withWriteBehavior(SalesforceSinkWriteBehavior.INSERT) - .withExternalIdFieldName("datadyzilocn").withIgnoreNullValues("datatnhi"); + SalesforceServiceCloudSink model = new SalesforceServiceCloudSink().withWriteBatchSize("datapvma") + .withWriteBatchTimeout("datatgpnyu").withSinkRetryCount("datawymyewb").withSinkRetryWait("dataxwv") + .withMaxConcurrentConnections("datatjsnjbahxyfd").withDisableMetricsCollection("databtksrdjhqcrmptj") + .withWriteBehavior(SalesforceSinkWriteBehavior.INSERT).withExternalIdFieldName("dataskjbasmrdpbmoq") + .withIgnoreNullValues("datavukgfzbykapmeo"); model = BinaryData.fromObject(model).toObject(SalesforceServiceCloudSink.class); Assertions.assertEquals(SalesforceSinkWriteBehavior.INSERT, model.writeBehavior()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSourceTests.java index 59ae310a9e66e..3e285c2ef1f89 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudSourceTests.java @@ -11,16 +11,16 @@ public final class SalesforceServiceCloudSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceServiceCloudSource model = BinaryData.fromString( - "{\"type\":\"SalesforceServiceCloudSource\",\"query\":\"datag\",\"readBehavior\":\"dataotvocjk\",\"additionalColumns\":\"datah\",\"sourceRetryCount\":\"datayvtrsgfdmtfn\",\"sourceRetryWait\":\"datatxqqlbmiq\",\"maxConcurrentConnections\":\"dataiahjxcd\",\"disableMetricsCollection\":\"datadlxwsfddyqpfyn\",\"\":{\"jzsjhhru\":\"datamjsurhl\",\"o\":\"dataizqvga\",\"am\":\"dataepnglzjhaqx\",\"s\":\"dataptc\"}}") + "{\"type\":\"SalesforceServiceCloudSource\",\"query\":\"dataprf\",\"readBehavior\":\"datahfv\",\"additionalColumns\":\"datayqzhoikemhohxa\",\"sourceRetryCount\":\"dataxoowpoogozer\",\"sourceRetryWait\":\"datazvpbnkgkuujeqqjq\",\"maxConcurrentConnections\":\"datajkajlogvfnwq\",\"disableMetricsCollection\":\"datalvazkqkycg\",\"\":{\"c\":\"datawehjybboqyxi\"}}") .toObject(SalesforceServiceCloudSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceServiceCloudSource model = new SalesforceServiceCloudSource().withSourceRetryCount("datayvtrsgfdmtfn") - .withSourceRetryWait("datatxqqlbmiq").withMaxConcurrentConnections("dataiahjxcd") - .withDisableMetricsCollection("datadlxwsfddyqpfyn").withQuery("datag").withReadBehavior("dataotvocjk") - .withAdditionalColumns("datah"); + SalesforceServiceCloudSource model = new SalesforceServiceCloudSource().withSourceRetryCount("dataxoowpoogozer") + .withSourceRetryWait("datazvpbnkgkuujeqqjq").withMaxConcurrentConnections("datajkajlogvfnwq") + .withDisableMetricsCollection("datalvazkqkycg").withQuery("dataprf").withReadBehavior("datahfv") + .withAdditionalColumns("datayqzhoikemhohxa"); model = BinaryData.fromObject(model).toObject(SalesforceServiceCloudSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SinkTests.java index 1f2638b8d31b7..88b8acf390fe3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SinkTests.java @@ -13,18 +13,19 @@ public final class SalesforceServiceCloudV2SinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceServiceCloudV2Sink model = BinaryData.fromString( - "{\"type\":\"SalesforceServiceCloudV2Sink\",\"writeBehavior\":\"Insert\",\"externalIdFieldName\":\"dataqg\",\"ignoreNullValues\":\"datahw\",\"writeBatchSize\":\"datagftshcssli\",\"writeBatchTimeout\":\"datalpphfezzgn\",\"sinkRetryCount\":\"datarokpoppzg\",\"sinkRetryWait\":\"datajr\",\"maxConcurrentConnections\":\"dataowhc\",\"disableMetricsCollection\":\"datasmlwadstlxrgqmu\",\"\":{\"dmhypptfppmu\":\"datao\",\"gwqqtbubkyipz\":\"datawvezt\"}}") + "{\"type\":\"SalesforceServiceCloudV2Sink\",\"writeBehavior\":\"Insert\",\"externalIdFieldName\":\"datayzbnkofcs\",\"ignoreNullValues\":\"datapwahehuc\",\"writeBatchSize\":\"datargpmgtjvuhcwcfz\",\"writeBatchTimeout\":\"dataklvtceaoiuurql\",\"sinkRetryCount\":\"datahebjfhpayww\",\"sinkRetryWait\":\"dataaqsuqp\",\"maxConcurrentConnections\":\"datavxbdlraridiat\",\"disableMetricsCollection\":\"dataxq\",\"\":{\"grn\":\"datayleyopgy\",\"ffosomxmvgjuzg\":\"datafjwoaomogkpc\"}}") .toObject(SalesforceServiceCloudV2Sink.class); Assertions.assertEquals(SalesforceV2SinkWriteBehavior.INSERT, model.writeBehavior()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceServiceCloudV2Sink model = new SalesforceServiceCloudV2Sink().withWriteBatchSize("datagftshcssli") - .withWriteBatchTimeout("datalpphfezzgn").withSinkRetryCount("datarokpoppzg").withSinkRetryWait("datajr") - .withMaxConcurrentConnections("dataowhc").withDisableMetricsCollection("datasmlwadstlxrgqmu") - .withWriteBehavior(SalesforceV2SinkWriteBehavior.INSERT).withExternalIdFieldName("dataqg") - .withIgnoreNullValues("datahw"); + SalesforceServiceCloudV2Sink model + = new SalesforceServiceCloudV2Sink().withWriteBatchSize("datargpmgtjvuhcwcfz") + .withWriteBatchTimeout("dataklvtceaoiuurql").withSinkRetryCount("datahebjfhpayww") + .withSinkRetryWait("dataaqsuqp").withMaxConcurrentConnections("datavxbdlraridiat") + .withDisableMetricsCollection("dataxq").withWriteBehavior(SalesforceV2SinkWriteBehavior.INSERT) + .withExternalIdFieldName("datayzbnkofcs").withIgnoreNullValues("datapwahehuc"); model = BinaryData.fromObject(model).toObject(SalesforceServiceCloudV2Sink.class); Assertions.assertEquals(SalesforceV2SinkWriteBehavior.INSERT, model.writeBehavior()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SourceTests.java index ade91f64291a1..a3447fa1938f6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceServiceCloudV2SourceTests.java @@ -11,16 +11,16 @@ public final class SalesforceServiceCloudV2SourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceServiceCloudV2Source model = BinaryData.fromString( - "{\"type\":\"SalesforceServiceCloudV2Source\",\"SOQLQuery\":\"datacry\",\"includeDeletedObjects\":\"datafglwm\",\"additionalColumns\":\"databxusn\",\"sourceRetryCount\":\"dataqnzxsdbfbkqice\",\"sourceRetryWait\":\"datamztffngrducoz\",\"maxConcurrentConnections\":\"databhtfmgpioxzhp\",\"disableMetricsCollection\":\"datahhuimgdfo\",\"\":{\"l\":\"dataeuotfavmdpe\"}}") + "{\"type\":\"SalesforceServiceCloudV2Source\",\"SOQLQuery\":\"datahrzpyxmfip\",\"includeDeletedObjects\":\"datamlf\",\"additionalColumns\":\"datawfxssxarxvft\",\"sourceRetryCount\":\"datasuqap\",\"sourceRetryWait\":\"datadgrbcltfkyq\",\"maxConcurrentConnections\":\"dataiujukcdlvpt\",\"disableMetricsCollection\":\"dataycupmfp\",\"\":{\"pxslccu\":\"dataswgnglmllr\",\"ndirdlehjz\":\"datascjefapouwsynsb\",\"kt\":\"datapdwyhggvhcoaoeti\",\"ae\":\"dataeirambfm\"}}") .toObject(SalesforceServiceCloudV2Source.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceServiceCloudV2Source model = new SalesforceServiceCloudV2Source() - .withSourceRetryCount("dataqnzxsdbfbkqice").withSourceRetryWait("datamztffngrducoz") - .withMaxConcurrentConnections("databhtfmgpioxzhp").withDisableMetricsCollection("datahhuimgdfo") - .withSoqlQuery("datacry").withIncludeDeletedObjects("datafglwm").withAdditionalColumns("databxusn"); + SalesforceServiceCloudV2Source model = new SalesforceServiceCloudV2Source().withSourceRetryCount("datasuqap") + .withSourceRetryWait("datadgrbcltfkyq").withMaxConcurrentConnections("dataiujukcdlvpt") + .withDisableMetricsCollection("dataycupmfp").withSoqlQuery("datahrzpyxmfip") + .withIncludeDeletedObjects("datamlf").withAdditionalColumns("datawfxssxarxvft"); model = BinaryData.fromObject(model).toObject(SalesforceServiceCloudV2Source.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSinkTests.java index c955c4241dede..53aeaa852a505 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSinkTests.java @@ -13,18 +13,18 @@ public final class SalesforceSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceSink model = BinaryData.fromString( - "{\"type\":\"SalesforceSink\",\"writeBehavior\":\"Upsert\",\"externalIdFieldName\":\"datamhioar\",\"ignoreNullValues\":\"datakhufktqgtj\",\"writeBatchSize\":\"datactreotzgkokfztrv\",\"writeBatchTimeout\":\"databt\",\"sinkRetryCount\":\"dataw\",\"sinkRetryWait\":\"datakegyskmh\",\"maxConcurrentConnections\":\"datasbzn\",\"disableMetricsCollection\":\"datagerwotlp\",\"\":{\"fbdbzbaboeegale\":\"dataoslqmftkqzafjy\"}}") + "{\"type\":\"SalesforceSink\",\"writeBehavior\":\"Upsert\",\"externalIdFieldName\":\"datai\",\"ignoreNullValues\":\"dataqjnuiiytyarpeyig\",\"writeBatchSize\":\"datappgkk\",\"writeBatchTimeout\":\"dataygjldljgd\",\"sinkRetryCount\":\"datagrtse\",\"sinkRetryWait\":\"dataow\",\"maxConcurrentConnections\":\"dataxkofmtfwcu\",\"disableMetricsCollection\":\"databnapzfdzmr\",\"\":{\"rj\":\"databclj\",\"l\":\"dataawnz\"}}") .toObject(SalesforceSink.class); Assertions.assertEquals(SalesforceSinkWriteBehavior.UPSERT, model.writeBehavior()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceSink model - = new SalesforceSink().withWriteBatchSize("datactreotzgkokfztrv").withWriteBatchTimeout("databt") - .withSinkRetryCount("dataw").withSinkRetryWait("datakegyskmh").withMaxConcurrentConnections("datasbzn") - .withDisableMetricsCollection("datagerwotlp").withWriteBehavior(SalesforceSinkWriteBehavior.UPSERT) - .withExternalIdFieldName("datamhioar").withIgnoreNullValues("datakhufktqgtj"); + SalesforceSink model = new SalesforceSink().withWriteBatchSize("datappgkk") + .withWriteBatchTimeout("dataygjldljgd").withSinkRetryCount("datagrtse").withSinkRetryWait("dataow") + .withMaxConcurrentConnections("dataxkofmtfwcu").withDisableMetricsCollection("databnapzfdzmr") + .withWriteBehavior(SalesforceSinkWriteBehavior.UPSERT).withExternalIdFieldName("datai") + .withIgnoreNullValues("dataqjnuiiytyarpeyig"); model = BinaryData.fromObject(model).toObject(SalesforceSink.class); Assertions.assertEquals(SalesforceSinkWriteBehavior.UPSERT, model.writeBehavior()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSourceTests.java index fa2d9ce14744d..55e7630df096e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceSourceTests.java @@ -11,16 +11,16 @@ public final class SalesforceSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceSource model = BinaryData.fromString( - "{\"type\":\"SalesforceSource\",\"query\":\"dataplqnilozf\",\"readBehavior\":\"datavsf\",\"queryTimeout\":\"datacarfdmlie\",\"additionalColumns\":\"datawocufcshqfc\",\"sourceRetryCount\":\"datanxfof\",\"sourceRetryWait\":\"datadroqktegi\",\"maxConcurrentConnections\":\"datakzctqbvntlzvgjme\",\"disableMetricsCollection\":\"dataoydyislepd\",\"\":{\"b\":\"dataklntspfnumpyy\",\"f\":\"databmjbmtxbi\"}}") + "{\"type\":\"SalesforceSource\",\"query\":\"databnekhjzbfb\",\"readBehavior\":\"dataeqkuozarr\",\"queryTimeout\":\"datapyzryjb\",\"additionalColumns\":\"databcvoyqnrjdrc\",\"sourceRetryCount\":\"datarvzewogh\",\"sourceRetryWait\":\"datazxkjqecj\",\"maxConcurrentConnections\":\"dataromeawthycbigpi\",\"disableMetricsCollection\":\"datapxhzjnparsulmuwl\",\"\":{\"xxqgoavzycxpza\":\"datakhe\",\"mftmxwtwzs\":\"datatalo\"}}") .toObject(SalesforceSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceSource model = new SalesforceSource().withSourceRetryCount("datanxfof") - .withSourceRetryWait("datadroqktegi").withMaxConcurrentConnections("datakzctqbvntlzvgjme") - .withDisableMetricsCollection("dataoydyislepd").withQueryTimeout("datacarfdmlie") - .withAdditionalColumns("datawocufcshqfc").withQuery("dataplqnilozf").withReadBehavior("datavsf"); + SalesforceSource model = new SalesforceSource().withSourceRetryCount("datarvzewogh") + .withSourceRetryWait("datazxkjqecj").withMaxConcurrentConnections("dataromeawthycbigpi") + .withDisableMetricsCollection("datapxhzjnparsulmuwl").withQueryTimeout("datapyzryjb") + .withAdditionalColumns("databcvoyqnrjdrc").withQuery("databnekhjzbfb").withReadBehavior("dataeqkuozarr"); model = BinaryData.fromObject(model).toObject(SalesforceSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SinkTests.java index 456f3ccceba3b..71e94685ac5f7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SinkTests.java @@ -13,19 +13,19 @@ public final class SalesforceV2SinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceV2Sink model = BinaryData.fromString( - "{\"type\":\"SalesforceV2Sink\",\"writeBehavior\":\"Insert\",\"externalIdFieldName\":\"dataewognpuupa\",\"ignoreNullValues\":\"dataerqvjwlr\",\"writeBatchSize\":\"datasxuxreg\",\"writeBatchTimeout\":\"dataffkzpfjm\",\"sinkRetryCount\":\"datatjc\",\"sinkRetryWait\":\"datagljrlrkv\",\"maxConcurrentConnections\":\"datan\",\"disableMetricsCollection\":\"dataxmmpuksvoimdg\",\"\":{\"miovbmwitnihw\":\"datapekie\"}}") + "{\"type\":\"SalesforceV2Sink\",\"writeBehavior\":\"Upsert\",\"externalIdFieldName\":\"databqtfcupjmw\",\"ignoreNullValues\":\"datamir\",\"writeBatchSize\":\"dataiqmk\",\"writeBatchTimeout\":\"datafjhtlbrkgh\",\"sinkRetryCount\":\"datafppjunkh\",\"sinkRetryWait\":\"datahkqny\",\"maxConcurrentConnections\":\"datafvzrq\",\"disableMetricsCollection\":\"datahepc\",\"\":{\"qwomkzcmwqfd\":\"dataeqqetasijia\",\"mvqumjmpsxzxbafs\":\"datagpmvl\",\"bzporj\":\"datad\",\"qtcnyhsdgmoxnelh\":\"dataubzkzjazfwywv\"}}") .toObject(SalesforceV2Sink.class); - Assertions.assertEquals(SalesforceV2SinkWriteBehavior.INSERT, model.writeBehavior()); + Assertions.assertEquals(SalesforceV2SinkWriteBehavior.UPSERT, model.writeBehavior()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceV2Sink model = new SalesforceV2Sink().withWriteBatchSize("datasxuxreg") - .withWriteBatchTimeout("dataffkzpfjm").withSinkRetryCount("datatjc").withSinkRetryWait("datagljrlrkv") - .withMaxConcurrentConnections("datan").withDisableMetricsCollection("dataxmmpuksvoimdg") - .withWriteBehavior(SalesforceV2SinkWriteBehavior.INSERT).withExternalIdFieldName("dataewognpuupa") - .withIgnoreNullValues("dataerqvjwlr"); + SalesforceV2Sink model = new SalesforceV2Sink().withWriteBatchSize("dataiqmk") + .withWriteBatchTimeout("datafjhtlbrkgh").withSinkRetryCount("datafppjunkh").withSinkRetryWait("datahkqny") + .withMaxConcurrentConnections("datafvzrq").withDisableMetricsCollection("datahepc") + .withWriteBehavior(SalesforceV2SinkWriteBehavior.UPSERT).withExternalIdFieldName("databqtfcupjmw") + .withIgnoreNullValues("datamir"); model = BinaryData.fromObject(model).toObject(SalesforceV2Sink.class); - Assertions.assertEquals(SalesforceV2SinkWriteBehavior.INSERT, model.writeBehavior()); + Assertions.assertEquals(SalesforceV2SinkWriteBehavior.UPSERT, model.writeBehavior()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SourceTests.java index 6555a6266c02a..0d4baed148be7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SalesforceV2SourceTests.java @@ -11,17 +11,16 @@ public final class SalesforceV2SourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SalesforceV2Source model = BinaryData.fromString( - "{\"type\":\"SalesforceV2Source\",\"SOQLQuery\":\"datasexaejbmtoun\",\"includeDeletedObjects\":\"datad\",\"queryTimeout\":\"dataclssedxiigwxz\",\"additionalColumns\":\"datajpudupishcvs\",\"sourceRetryCount\":\"dataaedsqfdulndy\",\"sourceRetryWait\":\"datahnptfv\",\"maxConcurrentConnections\":\"datajnrom\",\"disableMetricsCollection\":\"dataiaszqhp\",\"\":{\"vifurgnxhoqfvuqi\":\"datackwccpmsyh\",\"ip\":\"datadgkvfghcu\",\"q\":\"dataszrr\",\"xhxkm\":\"datakxyawtdsn\"}}") + "{\"type\":\"SalesforceV2Source\",\"SOQLQuery\":\"datav\",\"includeDeletedObjects\":\"datamlwadst\",\"queryTimeout\":\"datargqmuthxoldmhypp\",\"additionalColumns\":\"datappmulwv\",\"sourceRetryCount\":\"datathgw\",\"sourceRetryWait\":\"datatbubkyipzehitd\",\"maxConcurrentConnections\":\"databouwuajsitgpz\",\"disableMetricsCollection\":\"datalkcvkme\",\"\":{\"ebn\":\"dataolp\",\"jdbdjxvcxepjfxcm\":\"dataafvks\",\"mtret\":\"datahivw\",\"rs\":\"datalirbvqkbxgzepiny\"}}") .toObject(SalesforceV2Source.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SalesforceV2Source model - = new SalesforceV2Source().withSourceRetryCount("dataaedsqfdulndy").withSourceRetryWait("datahnptfv") - .withMaxConcurrentConnections("datajnrom").withDisableMetricsCollection("dataiaszqhp") - .withQueryTimeout("dataclssedxiigwxz").withAdditionalColumns("datajpudupishcvs") - .withSoqlQuery("datasexaejbmtoun").withIncludeDeletedObjects("datad"); + SalesforceV2Source model = new SalesforceV2Source().withSourceRetryCount("datathgw") + .withSourceRetryWait("datatbubkyipzehitd").withMaxConcurrentConnections("databouwuajsitgpz") + .withDisableMetricsCollection("datalkcvkme").withQueryTimeout("datargqmuthxoldmhypp") + .withAdditionalColumns("datappmulwv").withSoqlQuery("datav").withIncludeDeletedObjects("datamlwadst"); model = BinaryData.fromObject(model).toObject(SalesforceV2Source.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwCubeDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwCubeDatasetTests.java index b48cbb2a54561..8f083bfe661b9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwCubeDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwCubeDatasetTests.java @@ -19,34 +19,33 @@ public final class SapBwCubeDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapBwCubeDataset model = BinaryData.fromString( - "{\"type\":\"SapBwCube\",\"description\":\"efqckievyrejyo\",\"structure\":\"datakqf\",\"schema\":\"datasdwmnrtvvbuc\",\"linkedServiceName\":{\"referenceName\":\"nrovome\",\"parameters\":{\"hennmsgpywdib\":\"datasicvwqzocsf\",\"ibrbknuubxc\":\"datagvnrgalvwrhr\",\"qdvnpyeevff\":\"dataojtu\"}},\"parameters\":{\"chrtczwjcu\":{\"type\":\"Array\",\"defaultValue\":\"datatdowlxmwefcbyb\"},\"jqdjlgkuirxxeuwi\":{\"type\":\"Int\",\"defaultValue\":\"datanvy\"},\"viwxohktxagfuj\":{\"type\":\"String\",\"defaultValue\":\"datacvnfgb\"},\"asfeooq\":{\"type\":\"Object\",\"defaultValue\":\"datajnyexbvxgxqq\"}},\"annotations\":[\"datavev\",\"dataarp\",\"dataklqlii\",\"dataeanuwg\"],\"folder\":{\"name\":\"fgijydgs\"},\"\":{\"mwywhrjkejva\":\"datauymtevaeb\",\"gcphivfhrmte\":\"datadogzougxbxxgj\",\"usrjzhdtrsyfezf\":\"datafdvdoeary\"}}") + "{\"type\":\"SapBwCube\",\"description\":\"shennmsgpywdib\",\"structure\":\"datavnrgalv\",\"schema\":\"datahry\",\"linkedServiceName\":{\"referenceName\":\"brbknuubxcwoj\",\"parameters\":{\"fujgtdowlxm\":\"dataqdvnpyeevff\"}},\"parameters\":{\"vyrjqdjlgk\":{\"type\":\"String\",\"defaultValue\":\"dataybpchrtczwjcujyz\"},\"n\":{\"type\":\"SecureString\",\"defaultValue\":\"dataxxeuwiiirc\"},\"ohktxagfujdbqjny\":{\"type\":\"String\",\"defaultValue\":\"datadviw\"}},\"annotations\":[\"datavxgxqqqa\",\"datafeoo\",\"dataftpvevtarp\",\"dataklqlii\"],\"folder\":{\"name\":\"n\"},\"\":{\"gijydg\":\"datajno\"}}") .toObject(SapBwCubeDataset.class); - Assertions.assertEquals("efqckievyrejyo", model.description()); - Assertions.assertEquals("nrovome", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("chrtczwjcu").type()); - Assertions.assertEquals("fgijydgs", model.folder().name()); + Assertions.assertEquals("shennmsgpywdib", model.description()); + Assertions.assertEquals("brbknuubxcwoj", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("vyrjqdjlgk").type()); + Assertions.assertEquals("n", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapBwCubeDataset model = new SapBwCubeDataset().withDescription("efqckievyrejyo").withStructure("datakqf") - .withSchema("datasdwmnrtvvbuc") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("nrovome") - .withParameters(mapOf("hennmsgpywdib", "datasicvwqzocsf", "ibrbknuubxc", "datagvnrgalvwrhr", - "qdvnpyeevff", "dataojtu"))) - .withParameters(mapOf("chrtczwjcu", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datatdowlxmwefcbyb"), - "jqdjlgkuirxxeuwi", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datanvy"), "viwxohktxagfuj", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datacvnfgb"), "asfeooq", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datajnyexbvxgxqq"))) - .withAnnotations(Arrays.asList("datavev", "dataarp", "dataklqlii", "dataeanuwg")) - .withFolder(new DatasetFolder().withName("fgijydgs")); + SapBwCubeDataset model = new SapBwCubeDataset().withDescription("shennmsgpywdib").withStructure("datavnrgalv") + .withSchema("datahry") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("brbknuubxcwoj") + .withParameters(mapOf("fujgtdowlxm", "dataqdvnpyeevff"))) + .withParameters(mapOf("vyrjqdjlgk", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataybpchrtczwjcujyz"), + "n", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("dataxxeuwiiirc"), + "ohktxagfujdbqjny", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datadviw"))) + .withAnnotations(Arrays.asList("datavxgxqqqa", "datafeoo", "dataftpvevtarp", "dataklqlii")) + .withFolder(new DatasetFolder().withName("n")); model = BinaryData.fromObject(model).toObject(SapBwCubeDataset.class); - Assertions.assertEquals("efqckievyrejyo", model.description()); - Assertions.assertEquals("nrovome", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("chrtczwjcu").type()); - Assertions.assertEquals("fgijydgs", model.folder().name()); + Assertions.assertEquals("shennmsgpywdib", model.description()); + Assertions.assertEquals("brbknuubxcwoj", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("vyrjqdjlgk").type()); + Assertions.assertEquals("n", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwSourceTests.java index a3d0d0f20f7c5..f613e470f79a6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapBwSourceTests.java @@ -11,15 +11,15 @@ public final class SapBwSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapBwSource model = BinaryData.fromString( - "{\"type\":\"SapBwSource\",\"query\":\"datarziryx\",\"queryTimeout\":\"datajrutmxqv\",\"additionalColumns\":\"datapoftsapfwusf\",\"sourceRetryCount\":\"datanjvzlynvje\",\"sourceRetryWait\":\"datavu\",\"maxConcurrentConnections\":\"datalwzn\",\"disableMetricsCollection\":\"dataie\",\"\":{\"wwutduch\":\"datagkfiftgbupuukp\"}}") + "{\"type\":\"SapBwSource\",\"query\":\"datanwhcmvdowlqcy\",\"queryTimeout\":\"dataubzixqxx\",\"additionalColumns\":\"dataawbftzn\",\"sourceRetryCount\":\"datarfhj\",\"sourceRetryWait\":\"dataiutbrnr\",\"maxConcurrentConnections\":\"dataljucodrbkdieismd\",\"disableMetricsCollection\":\"datafim\",\"\":{\"foexlcskelwzmji\":\"dataijrlmnkvp\"}}") .toObject(SapBwSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapBwSource model = new SapBwSource().withSourceRetryCount("datanjvzlynvje").withSourceRetryWait("datavu") - .withMaxConcurrentConnections("datalwzn").withDisableMetricsCollection("dataie") - .withQueryTimeout("datajrutmxqv").withAdditionalColumns("datapoftsapfwusf").withQuery("datarziryx"); + SapBwSource model = new SapBwSource().withSourceRetryCount("datarfhj").withSourceRetryWait("dataiutbrnr") + .withMaxConcurrentConnections("dataljucodrbkdieismd").withDisableMetricsCollection("datafim") + .withQueryTimeout("dataubzixqxx").withAdditionalColumns("dataawbftzn").withQuery("datanwhcmvdowlqcy"); model = BinaryData.fromObject(model).toObject(SapBwSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTests.java index c0021500dccbc..e9fcf599672d0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTests.java @@ -19,31 +19,31 @@ public final class SapCloudForCustomerResourceDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapCloudForCustomerResourceDataset model = BinaryData.fromString( - "{\"type\":\"SapCloudForCustomerResource\",\"typeProperties\":{\"path\":\"datamy\"},\"description\":\"dzyyrwnmwtqil\",\"structure\":\"datannkynkstd\",\"schema\":\"datawhjfp\",\"linkedServiceName\":{\"referenceName\":\"fxaqjyihjcwwv\",\"parameters\":{\"spwweifdyfa\":\"datackfavhkh\",\"a\":\"dataexnguwnrdpuz\"}},\"parameters\":{\"bszam\":{\"type\":\"Array\",\"defaultValue\":\"datatgg\"},\"lrnhhjtvhqsz\":{\"type\":\"SecureString\",\"defaultValue\":\"dataejpdcliqwzutiy\"}},\"annotations\":[\"dataovqmqcudptoqwr\",\"datafckjthlokmx\",\"dataawfubkngejjxu\",\"dataowynj\"],\"folder\":{\"name\":\"zmxuktdrsjtmnk\"},\"\":{\"nuhcfhepisq\":\"datauwfzcfdtstiaxty\"}}") + "{\"type\":\"SapCloudForCustomerResource\",\"typeProperties\":{\"path\":\"dataebjuymtevaebzm\"},\"description\":\"whrjkejvaedogzo\",\"structure\":\"dataxbxxgjogcphivfhr\",\"schema\":\"dataenfdvdoea\",\"linkedServiceName\":{\"referenceName\":\"ywusrjzhdtr\",\"parameters\":{\"wnmwtqiljknn\":\"dataezfsmyljdzyy\",\"aqjyih\":\"dataynkstdtfwhjfphf\",\"vhkhpsp\":\"datacwwvaosckf\",\"exnguwnrdpuz\":\"dataweifdyfa\"}},\"parameters\":{\"ybsz\":{\"type\":\"Array\",\"defaultValue\":\"dataujtg\"},\"yelrnh\":{\"type\":\"Array\",\"defaultValue\":\"datajxejpdcliqwzut\"}},\"annotations\":[\"datavhqsz\"],\"folder\":{\"name\":\"ovqmqcudptoqwr\"},\"\":{\"kmxwawfu\":\"datakjthl\"}}") .toObject(SapCloudForCustomerResourceDataset.class); - Assertions.assertEquals("dzyyrwnmwtqil", model.description()); - Assertions.assertEquals("fxaqjyihjcwwv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("bszam").type()); - Assertions.assertEquals("zmxuktdrsjtmnk", model.folder().name()); + Assertions.assertEquals("whrjkejvaedogzo", model.description()); + Assertions.assertEquals("ywusrjzhdtr", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("ybsz").type()); + Assertions.assertEquals("ovqmqcudptoqwr", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SapCloudForCustomerResourceDataset model = new SapCloudForCustomerResourceDataset() - .withDescription("dzyyrwnmwtqil").withStructure("datannkynkstd").withSchema("datawhjfp") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fxaqjyihjcwwv") - .withParameters(mapOf("spwweifdyfa", "datackfavhkh", "a", "dataexnguwnrdpuz"))) - .withParameters(mapOf("bszam", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datatgg"), "lrnhhjtvhqsz", - new ParameterSpecification().withType(ParameterType.SECURE_STRING) - .withDefaultValue("dataejpdcliqwzutiy"))) - .withAnnotations(Arrays.asList("dataovqmqcudptoqwr", "datafckjthlokmx", "dataawfubkngejjxu", "dataowynj")) - .withFolder(new DatasetFolder().withName("zmxuktdrsjtmnk")).withPath("datamy"); + .withDescription("whrjkejvaedogzo").withStructure("dataxbxxgjogcphivfhr").withSchema("dataenfdvdoea") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ywusrjzhdtr") + .withParameters(mapOf("wnmwtqiljknn", "dataezfsmyljdzyy", "aqjyih", "dataynkstdtfwhjfphf", "vhkhpsp", + "datacwwvaosckf", "exnguwnrdpuz", "dataweifdyfa"))) + .withParameters(mapOf("ybsz", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataujtg"), "yelrnh", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datajxejpdcliqwzut"))) + .withAnnotations(Arrays.asList("datavhqsz")).withFolder(new DatasetFolder().withName("ovqmqcudptoqwr")) + .withPath("dataebjuymtevaebzm"); model = BinaryData.fromObject(model).toObject(SapCloudForCustomerResourceDataset.class); - Assertions.assertEquals("dzyyrwnmwtqil", model.description()); - Assertions.assertEquals("fxaqjyihjcwwv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("bszam").type()); - Assertions.assertEquals("zmxuktdrsjtmnk", model.folder().name()); + Assertions.assertEquals("whrjkejvaedogzo", model.description()); + Assertions.assertEquals("ywusrjzhdtr", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("ybsz").type()); + Assertions.assertEquals("ovqmqcudptoqwr", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTypePropertiesTests.java index ec6b1bfc9012a..83fe466a73a47 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerResourceDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class SapCloudForCustomerResourceDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapCloudForCustomerResourceDatasetTypeProperties model - = BinaryData.fromString("{\"path\":\"datacmlroiommemso\"}") + = BinaryData.fromString("{\"path\":\"datakngejjxumowy\"}") .toObject(SapCloudForCustomerResourceDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SapCloudForCustomerResourceDatasetTypeProperties model - = new SapCloudForCustomerResourceDatasetTypeProperties().withPath("datacmlroiommemso"); + = new SapCloudForCustomerResourceDatasetTypeProperties().withPath("datakngejjxumowy"); model = BinaryData.fromObject(model).toObject(SapCloudForCustomerResourceDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSinkTests.java index 6aafaa9ccb7fb..954846d64d26a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSinkTests.java @@ -13,18 +13,18 @@ public final class SapCloudForCustomerSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapCloudForCustomerSink model = BinaryData.fromString( - "{\"type\":\"SapCloudForCustomerSink\",\"writeBehavior\":\"Update\",\"httpRequestTimeout\":\"datagxikzvnghtknrw\",\"writeBatchSize\":\"dataysnmy\",\"writeBatchTimeout\":\"datafmlcnrapxw\",\"sinkRetryCount\":\"datapxoelfobehr\",\"sinkRetryWait\":\"dataglojjcziytf\",\"maxConcurrentConnections\":\"datavirmbr\",\"disableMetricsCollection\":\"datagnqa\",\"\":{\"dzvuhw\":\"datamsfybh\"}}") + "{\"type\":\"SapCloudForCustomerSink\",\"writeBehavior\":\"Update\",\"httpRequestTimeout\":\"datamophtkyzsgayng\",\"writeBatchSize\":\"datawvcnv\",\"writeBatchTimeout\":\"dataqxqhysu\",\"sinkRetryCount\":\"datadnslroqxrvycjdn\",\"sinkRetryWait\":\"datamggy\",\"maxConcurrentConnections\":\"datapmsacbamtoqse\",\"disableMetricsCollection\":\"dataoyxdig\",\"\":{\"osxdsxil\":\"datazmylqhq\"}}") .toObject(SapCloudForCustomerSink.class); Assertions.assertEquals(SapCloudForCustomerSinkWriteBehavior.UPDATE, model.writeBehavior()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapCloudForCustomerSink model = new SapCloudForCustomerSink().withWriteBatchSize("dataysnmy") - .withWriteBatchTimeout("datafmlcnrapxw").withSinkRetryCount("datapxoelfobehr") - .withSinkRetryWait("dataglojjcziytf").withMaxConcurrentConnections("datavirmbr") - .withDisableMetricsCollection("datagnqa").withWriteBehavior(SapCloudForCustomerSinkWriteBehavior.UPDATE) - .withHttpRequestTimeout("datagxikzvnghtknrw"); + SapCloudForCustomerSink model = new SapCloudForCustomerSink().withWriteBatchSize("datawvcnv") + .withWriteBatchTimeout("dataqxqhysu").withSinkRetryCount("datadnslroqxrvycjdn") + .withSinkRetryWait("datamggy").withMaxConcurrentConnections("datapmsacbamtoqse") + .withDisableMetricsCollection("dataoyxdig").withWriteBehavior(SapCloudForCustomerSinkWriteBehavior.UPDATE) + .withHttpRequestTimeout("datamophtkyzsgayng"); model = BinaryData.fromObject(model).toObject(SapCloudForCustomerSink.class); Assertions.assertEquals(SapCloudForCustomerSinkWriteBehavior.UPDATE, model.writeBehavior()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSourceTests.java index 217543943600f..fbb8f35722c9f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapCloudForCustomerSourceTests.java @@ -11,16 +11,16 @@ public final class SapCloudForCustomerSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapCloudForCustomerSource model = BinaryData.fromString( - "{\"type\":\"SapCloudForCustomerSource\",\"query\":\"datarabbyfhz\",\"httpRequestTimeout\":\"datajrxenpkxanlbrcy\",\"queryTimeout\":\"datarc\",\"additionalColumns\":\"dataanbw\",\"sourceRetryCount\":\"datalqioq\",\"sourceRetryWait\":\"dataxcg\",\"maxConcurrentConnections\":\"datazluilzgpghjakzmn\",\"disableMetricsCollection\":\"datanqmajs\",\"\":{\"yupgojrwpoxu\":\"datajlpbxvp\"}}") + "{\"type\":\"SapCloudForCustomerSource\",\"query\":\"datadgem\",\"httpRequestTimeout\":\"datayddzjtxlvgsl\",\"queryTimeout\":\"datalys\",\"additionalColumns\":\"datav\",\"sourceRetryCount\":\"dataak\",\"sourceRetryWait\":\"datapaexllt\",\"maxConcurrentConnections\":\"datakkaei\",\"disableMetricsCollection\":\"datahr\",\"\":{\"lta\":\"datagvsrt\",\"lnwiwrubxey\":\"datajkraleglpyns\",\"w\":\"dataalhbrwaltvky\"}}") .toObject(SapCloudForCustomerSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapCloudForCustomerSource model = new SapCloudForCustomerSource().withSourceRetryCount("datalqioq") - .withSourceRetryWait("dataxcg").withMaxConcurrentConnections("datazluilzgpghjakzmn") - .withDisableMetricsCollection("datanqmajs").withQueryTimeout("datarc").withAdditionalColumns("dataanbw") - .withQuery("datarabbyfhz").withHttpRequestTimeout("datajrxenpkxanlbrcy"); + SapCloudForCustomerSource model = new SapCloudForCustomerSource().withSourceRetryCount("dataak") + .withSourceRetryWait("datapaexllt").withMaxConcurrentConnections("datakkaei") + .withDisableMetricsCollection("datahr").withQueryTimeout("datalys").withAdditionalColumns("datav") + .withQuery("datadgem").withHttpRequestTimeout("datayddzjtxlvgsl"); model = BinaryData.fromObject(model).toObject(SapCloudForCustomerSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTests.java index 413092bee62e1..ced1ab4236f94 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTests.java @@ -19,32 +19,29 @@ public final class SapEccResourceDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapEccResourceDataset model = BinaryData.fromString( - "{\"type\":\"SapEccResource\",\"typeProperties\":{\"path\":\"datagblcyeqdobobaq\"},\"description\":\"bebckce\",\"structure\":\"datasixwnlpjcxbjgf\",\"schema\":\"dataqyyfrridzfpsfyak\",\"linkedServiceName\":{\"referenceName\":\"dfhmlx\",\"parameters\":{\"mkqafzvptriy\":\"dataekn\",\"dvvoydwedggwgcl\":\"datajrgtruwpuqpsrce\",\"drjbjngoars\":\"databwatz\"}},\"parameters\":{\"rqw\":{\"type\":\"Float\",\"defaultValue\":\"dataemzcyniapypimrx\"},\"stuinytkmlfupjzc\":{\"type\":\"Object\",\"defaultValue\":\"datae\"},\"yxjg\":{\"type\":\"Array\",\"defaultValue\":\"datazj\"}},\"annotations\":[\"datauerrdaktnytkbc\",\"datarfcvcp\"],\"folder\":{\"name\":\"j\"},\"\":{\"vlhnhhcikhleb\":\"datapw\",\"giflr\":\"datajgylsac\"}}") + "{\"type\":\"SapEccResource\",\"typeProperties\":{\"path\":\"datajmoozmxuk\"},\"description\":\"rs\",\"structure\":\"datamnkxjou\",\"schema\":\"datazcfdtstiaxtyrnu\",\"linkedServiceName\":{\"referenceName\":\"cfhep\",\"parameters\":{\"q\":\"databcmlroiommems\",\"abebckc\":\"datablcyeqdobobaq\",\"pjcxbjgfm\":\"datasrsixwn\"}},\"parameters\":{\"y\":{\"type\":\"Float\",\"defaultValue\":\"datarridzfps\"}},\"annotations\":[\"datadfhmlx\",\"dataq\",\"dataekn\",\"datamkqafzvptriy\"],\"folder\":{\"name\":\"gt\"},\"\":{\"dwedg\":\"datapuqpsrcekdvvo\",\"clvbwatza\":\"dataw\"}}") .toObject(SapEccResourceDataset.class); - Assertions.assertEquals("bebckce", model.description()); - Assertions.assertEquals("dfhmlx", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("rqw").type()); - Assertions.assertEquals("j", model.folder().name()); + Assertions.assertEquals("rs", model.description()); + Assertions.assertEquals("cfhep", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("y").type()); + Assertions.assertEquals("gt", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapEccResourceDataset model = new SapEccResourceDataset().withDescription("bebckce") - .withStructure("datasixwnlpjcxbjgf").withSchema("dataqyyfrridzfpsfyak") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dfhmlx").withParameters( - mapOf("mkqafzvptriy", "dataekn", "dvvoydwedggwgcl", "datajrgtruwpuqpsrce", "drjbjngoars", "databwatz"))) - .withParameters(mapOf("rqw", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataemzcyniapypimrx"), - "stuinytkmlfupjzc", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datae"), "yxjg", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datazj"))) - .withAnnotations(Arrays.asList("datauerrdaktnytkbc", "datarfcvcp")) - .withFolder(new DatasetFolder().withName("j")).withPath("datagblcyeqdobobaq"); + SapEccResourceDataset model = new SapEccResourceDataset().withDescription("rs").withStructure("datamnkxjou") + .withSchema("datazcfdtstiaxtyrnu") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cfhep").withParameters( + mapOf("q", "databcmlroiommems", "abebckc", "datablcyeqdobobaq", "pjcxbjgfm", "datasrsixwn"))) + .withParameters( + mapOf("y", new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datarridzfps"))) + .withAnnotations(Arrays.asList("datadfhmlx", "dataq", "dataekn", "datamkqafzvptriy")) + .withFolder(new DatasetFolder().withName("gt")).withPath("datajmoozmxuk"); model = BinaryData.fromObject(model).toObject(SapEccResourceDataset.class); - Assertions.assertEquals("bebckce", model.description()); - Assertions.assertEquals("dfhmlx", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("rqw").type()); - Assertions.assertEquals("j", model.folder().name()); + Assertions.assertEquals("rs", model.description()); + Assertions.assertEquals("cfhep", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("y").type()); + Assertions.assertEquals("gt", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTypePropertiesTests.java index 8e17a93f3a979..515da95dd3544 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccResourceDatasetTypePropertiesTests.java @@ -10,13 +10,14 @@ public final class SapEccResourceDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SapEccResourceDatasetTypeProperties model - = BinaryData.fromString("{\"path\":\"dataygotoh\"}").toObject(SapEccResourceDatasetTypeProperties.class); + SapEccResourceDatasetTypeProperties model = BinaryData.fromString("{\"path\":\"datarjbjngoarsr\"}") + .toObject(SapEccResourceDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapEccResourceDatasetTypeProperties model = new SapEccResourceDatasetTypeProperties().withPath("dataygotoh"); + SapEccResourceDatasetTypeProperties model + = new SapEccResourceDatasetTypeProperties().withPath("datarjbjngoarsr"); model = BinaryData.fromObject(model).toObject(SapEccResourceDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccSourceTests.java index ff276eb1e9bfb..94094425d55a6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapEccSourceTests.java @@ -11,16 +11,16 @@ public final class SapEccSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapEccSource model = BinaryData.fromString( - "{\"type\":\"SapEccSource\",\"query\":\"dataqyoyjptkyfrk\",\"httpRequestTimeout\":\"datatxwyqkkdumxd\",\"queryTimeout\":\"dataimsioffyboo\",\"additionalColumns\":\"dataovfundk\",\"sourceRetryCount\":\"datamyxmsbtq\",\"sourceRetryWait\":\"datagn\",\"maxConcurrentConnections\":\"datacbjxgjudgbwr\",\"disableMetricsCollection\":\"dataiuzlfq\",\"\":{\"gsmlujunqwkjf\":\"datahlzljqcm\"}}") + "{\"type\":\"SapEccSource\",\"query\":\"datap\",\"httpRequestTimeout\":\"dataegkrjolbaeghak\",\"queryTimeout\":\"datacismrnneklfibn\",\"additionalColumns\":\"datafcl\",\"sourceRetryCount\":\"datafu\",\"sourceRetryWait\":\"datawqzbiukzmfy\",\"maxConcurrentConnections\":\"datayz\",\"disableMetricsCollection\":\"dataf\",\"\":{\"seyxpgkmlnj\":\"datalnfvexiuuqafo\",\"glnfwjslwvexblu\":\"datauaywgcjqnfafp\"}}") .toObject(SapEccSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapEccSource model = new SapEccSource().withSourceRetryCount("datamyxmsbtq").withSourceRetryWait("datagn") - .withMaxConcurrentConnections("datacbjxgjudgbwr").withDisableMetricsCollection("dataiuzlfq") - .withQueryTimeout("dataimsioffyboo").withAdditionalColumns("dataovfundk").withQuery("dataqyoyjptkyfrk") - .withHttpRequestTimeout("datatxwyqkkdumxd"); + SapEccSource model = new SapEccSource().withSourceRetryCount("datafu").withSourceRetryWait("datawqzbiukzmfy") + .withMaxConcurrentConnections("datayz").withDisableMetricsCollection("dataf") + .withQueryTimeout("datacismrnneklfibn").withAdditionalColumns("datafcl").withQuery("datap") + .withHttpRequestTimeout("dataegkrjolbaeghak"); model = BinaryData.fromObject(model).toObject(SapEccSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaPartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaPartitionSettingsTests.java index aea2c77d60b4d..f72b762a09fd5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaPartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaPartitionSettingsTests.java @@ -10,13 +10,13 @@ public final class SapHanaPartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SapHanaPartitionSettings model = BinaryData.fromString("{\"partitionColumnName\":\"datavodcvfwk\"}") + SapHanaPartitionSettings model = BinaryData.fromString("{\"partitionColumnName\":\"dataujhejytrvlgu\"}") .toObject(SapHanaPartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapHanaPartitionSettings model = new SapHanaPartitionSettings().withPartitionColumnName("datavodcvfwk"); + SapHanaPartitionSettings model = new SapHanaPartitionSettings().withPartitionColumnName("dataujhejytrvlgu"); model = BinaryData.fromObject(model).toObject(SapHanaPartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaSourceTests.java index 04f0f16899128..74a1b09150a63 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaSourceTests.java @@ -12,17 +12,18 @@ public final class SapHanaSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapHanaSource model = BinaryData.fromString( - "{\"type\":\"SapHanaSource\",\"query\":\"datauybdzrxbckx\",\"packetSize\":\"datanzowguirhexj\",\"partitionOption\":\"datayhmktpy\",\"partitionSettings\":{\"partitionColumnName\":\"datacpkoamqfdtbao\"},\"queryTimeout\":\"dataaofkcvhh\",\"additionalColumns\":\"datavkuuikrsi\",\"sourceRetryCount\":\"datarwsj\",\"sourceRetryWait\":\"dataxenxjvapd\",\"maxConcurrentConnections\":\"datafvygrfy\",\"disableMetricsCollection\":\"datanxuacfmbyn\",\"\":{\"tcyzuzvbqbroyrw\":\"datamjmurocryfuv\",\"kffsupwcslz\":\"datatbbfweozkb\"}}") + "{\"type\":\"SapHanaSource\",\"query\":\"datamqwkfgmkpve\",\"packetSize\":\"datatzqzhdwrcajfersx\",\"partitionOption\":\"datalkcwjw\",\"partitionSettings\":{\"partitionColumnName\":\"datasksgxykdepqcy\"},\"queryTimeout\":\"datahwsxpzkmotgmd\",\"additionalColumns\":\"datawwqevbiuntp\",\"sourceRetryCount\":\"datamwjxlyce\",\"sourceRetryWait\":\"dataeqgywrauur\",\"maxConcurrentConnections\":\"datad\",\"disableMetricsCollection\":\"dataycnk\",\"\":{\"amhefuhnbdlza\":\"dataiecfmqcxmpcdbvcx\",\"cmpnk\":\"dataectzjjgvcbt\"}}") .toObject(SapHanaSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapHanaSource model = new SapHanaSource().withSourceRetryCount("datarwsj").withSourceRetryWait("dataxenxjvapd") - .withMaxConcurrentConnections("datafvygrfy").withDisableMetricsCollection("datanxuacfmbyn") - .withQueryTimeout("dataaofkcvhh").withAdditionalColumns("datavkuuikrsi").withQuery("datauybdzrxbckx") - .withPacketSize("datanzowguirhexj").withPartitionOption("datayhmktpy") - .withPartitionSettings(new SapHanaPartitionSettings().withPartitionColumnName("datacpkoamqfdtbao")); + SapHanaSource model + = new SapHanaSource().withSourceRetryCount("datamwjxlyce").withSourceRetryWait("dataeqgywrauur") + .withMaxConcurrentConnections("datad").withDisableMetricsCollection("dataycnk") + .withQueryTimeout("datahwsxpzkmotgmd").withAdditionalColumns("datawwqevbiuntp") + .withQuery("datamqwkfgmkpve").withPacketSize("datatzqzhdwrcajfersx").withPartitionOption("datalkcwjw") + .withPartitionSettings(new SapHanaPartitionSettings().withPartitionColumnName("datasksgxykdepqcy")); model = BinaryData.fromObject(model).toObject(SapHanaSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTests.java index 5ae35e40779dd..6d807fc8fc465 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTests.java @@ -19,33 +19,32 @@ public final class SapHanaTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapHanaTableDataset model = BinaryData.fromString( - "{\"type\":\"SapHanaTable\",\"typeProperties\":{\"schema\":\"dataoidhbxitrapwzhlu\",\"table\":\"datasj\"},\"description\":\"l\",\"structure\":\"dataiemv\",\"schema\":\"datameakosy\",\"linkedServiceName\":{\"referenceName\":\"ycvldeehcbsaip\",\"parameters\":{\"vsluazzxfjv\":\"dataofkegbvbbdledffl\",\"scboxra\":\"dataugpxzeempup\"}},\"parameters\":{\"fdr\":{\"type\":\"Float\",\"defaultValue\":\"datarjgobekxeheowsec\"},\"seesacuicnvq\":{\"type\":\"Int\",\"defaultValue\":\"dataskiwrjsbdb\"},\"vmrfaptndrmmn\":{\"type\":\"Array\",\"defaultValue\":\"datau\"}},\"annotations\":[\"datak\",\"dataxrqkekcdavi\",\"dataebeqrfza\",\"dataqymcwt\"],\"folder\":{\"name\":\"ceplbrzgkuorwpq\"},\"\":{\"ykk\":\"dataweobptscr\",\"sbnlyoifgdfzjqth\":\"dataelayynoyjyfls\",\"kxxlwwo\":\"datakcvoevcwfzo\",\"ubdmg\":\"dataxgbsdzcgcvypj\"}}") + "{\"type\":\"SapHanaTable\",\"typeProperties\":{\"schema\":\"datai\",\"table\":\"datazcyniapy\"},\"description\":\"mrxirqwipzesstu\",\"structure\":\"dataytkmlfupj\",\"schema\":\"datax\",\"linkedServiceName\":{\"referenceName\":\"vzjoyxjgahxue\",\"parameters\":{\"erfcv\":\"dataaktnytkb\",\"hnhhcikh\":\"datapvfqjckmpwyv\",\"lsac\":\"dataebgjg\",\"ygotoh\":\"datagiflr\"}},\"parameters\":{\"tjsjzelsriemvu\":{\"type\":\"String\",\"defaultValue\":\"datadhbxitrapwzhl\"},\"cb\":{\"type\":\"Int\",\"defaultValue\":\"dataakosysycvldee\"}},\"annotations\":[\"datapus\",\"dataofkegbvbbdledffl\",\"datavsluazzxfjv\",\"dataugpxzeempup\"],\"folder\":{\"name\":\"boxraqdczmr\"},\"\":{\"drzzbskiwrjsb\":\"databekxeheowseca\",\"qsuivmrfaptndrmm\":\"databmseesacuicnvq\"}}") .toObject(SapHanaTableDataset.class); - Assertions.assertEquals("l", model.description()); - Assertions.assertEquals("ycvldeehcbsaip", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("fdr").type()); - Assertions.assertEquals("ceplbrzgkuorwpq", model.folder().name()); + Assertions.assertEquals("mrxirqwipzesstu", model.description()); + Assertions.assertEquals("vzjoyxjgahxue", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("tjsjzelsriemvu").type()); + Assertions.assertEquals("boxraqdczmr", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapHanaTableDataset model = new SapHanaTableDataset().withDescription("l").withStructure("dataiemv") - .withSchema("datameakosy") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ycvldeehcbsaip") - .withParameters(mapOf("vsluazzxfjv", "dataofkegbvbbdledffl", "scboxra", "dataugpxzeempup"))) - .withParameters(mapOf("fdr", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datarjgobekxeheowsec"), - "seesacuicnvq", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataskiwrjsbdb"), - "vmrfaptndrmmn", new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datau"))) - .withAnnotations(Arrays.asList("datak", "dataxrqkekcdavi", "dataebeqrfza", "dataqymcwt")) - .withFolder(new DatasetFolder().withName("ceplbrzgkuorwpq")) - .withSchemaTypePropertiesSchema("dataoidhbxitrapwzhlu").withTable("datasj"); + SapHanaTableDataset model = new SapHanaTableDataset().withDescription("mrxirqwipzesstu") + .withStructure("dataytkmlfupj").withSchema("datax") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("vzjoyxjgahxue") + .withParameters(mapOf("erfcv", "dataaktnytkb", "hnhhcikh", "datapvfqjckmpwyv", "lsac", "dataebgjg", + "ygotoh", "datagiflr"))) + .withParameters(mapOf("tjsjzelsriemvu", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datadhbxitrapwzhl"), "cb", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataakosysycvldee"))) + .withAnnotations(Arrays.asList("datapus", "dataofkegbvbbdledffl", "datavsluazzxfjv", "dataugpxzeempup")) + .withFolder(new DatasetFolder().withName("boxraqdczmr")).withSchemaTypePropertiesSchema("datai") + .withTable("datazcyniapy"); model = BinaryData.fromObject(model).toObject(SapHanaTableDataset.class); - Assertions.assertEquals("l", model.description()); - Assertions.assertEquals("ycvldeehcbsaip", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("fdr").type()); - Assertions.assertEquals("ceplbrzgkuorwpq", model.folder().name()); + Assertions.assertEquals("mrxirqwipzesstu", model.description()); + Assertions.assertEquals("vzjoyxjgahxue", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("tjsjzelsriemvu").type()); + Assertions.assertEquals("boxraqdczmr", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTypePropertiesTests.java index 9e0be8b652232..bd949f9968c05 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapHanaTableDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class SapHanaTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapHanaTableDatasetTypeProperties model - = BinaryData.fromString("{\"schema\":\"dataxehujcqgzwv\",\"table\":\"dataiuaoibmjklqrljd\"}") + = BinaryData.fromString("{\"schema\":\"datahnkmxrqkek\",\"table\":\"dataaviiebeqrfz\"}") .toObject(SapHanaTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SapHanaTableDatasetTypeProperties model - = new SapHanaTableDatasetTypeProperties().withSchema("dataxehujcqgzwv").withTable("dataiuaoibmjklqrljd"); + = new SapHanaTableDatasetTypeProperties().withSchema("datahnkmxrqkek").withTable("dataaviiebeqrfz"); model = BinaryData.fromObject(model).toObject(SapHanaTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTests.java index 16684a5cea228..3aa808b6b3599 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTests.java @@ -19,31 +19,33 @@ public final class SapOdpResourceDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapOdpResourceDataset model = BinaryData.fromString( - "{\"type\":\"SapOdpResource\",\"typeProperties\":{\"context\":\"datadwrgavtfyzse\",\"objectName\":\"dataf\"},\"description\":\"ukryxpi\",\"structure\":\"dataapeakfdmcedl\",\"schema\":\"datalxkyoddoq\",\"linkedServiceName\":{\"referenceName\":\"a\",\"parameters\":{\"whqy\":\"datarki\",\"fqeqf\":\"datagqmndkrwwmurhv\",\"lwgebylpzjeldaqw\":\"datarnacki\"}},\"parameters\":{\"potnpkbvzpkod\":{\"type\":\"Array\",\"defaultValue\":\"datanijhwcbrds\"},\"dxuczl\":{\"type\":\"Float\",\"defaultValue\":\"datanqdjgsbtwgn\"},\"iiuv\":{\"type\":\"Int\",\"defaultValue\":\"dataqycznrir\"}},\"annotations\":[\"dataqkqwucqsdgb\"],\"folder\":{\"name\":\"tvmijccpk\"},\"\":{\"ihtnnlbhxjppcbqe\":\"dataamyvwprjm\",\"zayjwdu\":\"datafzfppvo\",\"prklatwiuujxsuj\":\"datajh\",\"pc\":\"datarwgxeegxbnjnczep\"}}") + "{\"type\":\"SapOdpResource\",\"typeProperties\":{\"context\":\"datawuuqbmenxcqsxwc\",\"objectName\":\"dataykc\"},\"description\":\"dek\",\"structure\":\"datanjre\",\"schema\":\"dataptedeuenthshnfi\",\"linkedServiceName\":{\"referenceName\":\"gpgpkkhpjnglaqlm\",\"parameters\":{\"ecpvfpnrzikvo\":\"datamtrdlpxiww\",\"ivxdifbwblijhp\":\"dataloeohy\"}},\"parameters\":{\"xr\":{\"type\":\"SecureString\",\"defaultValue\":\"dataoyxontbwdq\"},\"fewxatktwjrppi\":{\"type\":\"String\",\"defaultValue\":\"dataqrrldxfua\"},\"jklwjp\":{\"type\":\"Array\",\"defaultValue\":\"datarqvelrmdcizhvksb\"}},\"annotations\":[\"datancw\",\"datasmpyeyzolbfnfly\"],\"folder\":{\"name\":\"uduiqoom\"},\"\":{\"liyznghuqzgp\":\"datakqwopws\",\"fy\":\"dataglkfvdwrgav\"}}") .toObject(SapOdpResourceDataset.class); - Assertions.assertEquals("ukryxpi", model.description()); - Assertions.assertEquals("a", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("potnpkbvzpkod").type()); - Assertions.assertEquals("tvmijccpk", model.folder().name()); + Assertions.assertEquals("dek", model.description()); + Assertions.assertEquals("gpgpkkhpjnglaqlm", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("xr").type()); + Assertions.assertEquals("uduiqoom", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapOdpResourceDataset model = new SapOdpResourceDataset().withDescription("ukryxpi") - .withStructure("dataapeakfdmcedl").withSchema("datalxkyoddoq") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("a").withParameters( - mapOf("whqy", "datarki", "fqeqf", "datagqmndkrwwmurhv", "lwgebylpzjeldaqw", "datarnacki"))) - .withParameters(mapOf("potnpkbvzpkod", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datanijhwcbrds"), "dxuczl", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datanqdjgsbtwgn"), "iiuv", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("dataqycznrir"))) - .withAnnotations(Arrays.asList("dataqkqwucqsdgb")).withFolder(new DatasetFolder().withName("tvmijccpk")) - .withContext("datadwrgavtfyzse").withObjectName("dataf"); + SapOdpResourceDataset model = new SapOdpResourceDataset().withDescription("dek").withStructure("datanjre") + .withSchema("dataptedeuenthshnfi") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("gpgpkkhpjnglaqlm") + .withParameters(mapOf("ecpvfpnrzikvo", "datamtrdlpxiww", "ivxdifbwblijhp", "dataloeohy"))) + .withParameters(mapOf("xr", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("dataoyxontbwdq"), + "fewxatktwjrppi", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataqrrldxfua"), "jklwjp", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("datarqvelrmdcizhvksb"))) + .withAnnotations(Arrays.asList("datancw", "datasmpyeyzolbfnfly")) + .withFolder(new DatasetFolder().withName("uduiqoom")).withContext("datawuuqbmenxcqsxwc") + .withObjectName("dataykc"); model = BinaryData.fromObject(model).toObject(SapOdpResourceDataset.class); - Assertions.assertEquals("ukryxpi", model.description()); - Assertions.assertEquals("a", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("potnpkbvzpkod").type()); - Assertions.assertEquals("tvmijccpk", model.folder().name()); + Assertions.assertEquals("dek", model.description()); + Assertions.assertEquals("gpgpkkhpjnglaqlm", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("xr").type()); + Assertions.assertEquals("uduiqoom", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTypePropertiesTests.java index 92d7000473033..1bb78d4859ca9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpResourceDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class SapOdpResourceDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapOdpResourceDatasetTypeProperties model - = BinaryData.fromString("{\"context\":\"datamgbf\",\"objectName\":\"datadquyyaes\"}") + = BinaryData.fromString("{\"context\":\"datasedfmzu\",\"objectName\":\"dataryxpi\"}") .toObject(SapOdpResourceDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SapOdpResourceDatasetTypeProperties model - = new SapOdpResourceDatasetTypeProperties().withContext("datamgbf").withObjectName("datadquyyaes"); + = new SapOdpResourceDatasetTypeProperties().withContext("datasedfmzu").withObjectName("dataryxpi"); model = BinaryData.fromObject(model).toObject(SapOdpResourceDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpSourceTests.java index 25c22a0f6d985..69797636d8ff0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOdpSourceTests.java @@ -11,16 +11,17 @@ public final class SapOdpSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapOdpSource model = BinaryData.fromString( - "{\"type\":\"SapOdpSource\",\"extractionMode\":\"datajjticly\",\"subscriberProcess\":\"dataduxbungmpn\",\"selection\":\"datatgucdfxglrcj\",\"projection\":\"dataoaz\",\"queryTimeout\":\"datazjc\",\"additionalColumns\":\"dataza\",\"sourceRetryCount\":\"dataoyrqjo\",\"sourceRetryWait\":\"datalnyhzes\",\"maxConcurrentConnections\":\"datadcloqpgzdbo\",\"disableMetricsCollection\":\"datapphfmzeufjzqaqe\",\"\":{\"qcthgqyvaoaz\":\"dataygqcwzytom\",\"cqafnv\":\"datakyk\"}}") + "{\"type\":\"SapOdpSource\",\"extractionMode\":\"datax\",\"subscriberProcess\":\"datanrurtnwbjj\",\"selection\":\"dataupckhfbmdemohlsh\",\"projection\":\"dataaoofltb\",\"queryTimeout\":\"datayvmwaejxzkqcm\",\"additionalColumns\":\"datacf\",\"sourceRetryCount\":\"dataxyrtqegabsfjrj\",\"sourceRetryWait\":\"dataq\",\"maxConcurrentConnections\":\"datagorvgdibepg\",\"disableMetricsCollection\":\"databijoehhqwwsgqzi\",\"\":{\"buxqtokckxfkft\":\"dataetwjssyazm\",\"btbogxlyve\":\"datakbyruheawucmqfu\",\"tsfi\":\"datavxjgu\"}}") .toObject(SapOdpSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapOdpSource model = new SapOdpSource().withSourceRetryCount("dataoyrqjo").withSourceRetryWait("datalnyhzes") - .withMaxConcurrentConnections("datadcloqpgzdbo").withDisableMetricsCollection("datapphfmzeufjzqaqe") - .withQueryTimeout("datazjc").withAdditionalColumns("dataza").withExtractionMode("datajjticly") - .withSubscriberProcess("dataduxbungmpn").withSelection("datatgucdfxglrcj").withProjection("dataoaz"); + SapOdpSource model = new SapOdpSource().withSourceRetryCount("dataxyrtqegabsfjrj").withSourceRetryWait("dataq") + .withMaxConcurrentConnections("datagorvgdibepg").withDisableMetricsCollection("databijoehhqwwsgqzi") + .withQueryTimeout("datayvmwaejxzkqcm").withAdditionalColumns("datacf").withExtractionMode("datax") + .withSubscriberProcess("datanrurtnwbjj").withSelection("dataupckhfbmdemohlsh") + .withProjection("dataaoofltb"); model = BinaryData.fromObject(model).toObject(SapOdpSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubSourceTests.java index 2cf185739e090..89762fbbe5c3e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubSourceTests.java @@ -11,17 +11,17 @@ public final class SapOpenHubSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapOpenHubSource model = BinaryData.fromString( - "{\"type\":\"SapOpenHubSource\",\"excludeLastRequest\":\"datap\",\"baseRequestId\":\"datagfnqtxlk\",\"customRfcReadTableFunctionModule\":\"dataviklxsgstunsa\",\"sapDataColumnDelimiter\":\"dataistdbehkb\",\"queryTimeout\":\"datajkodpzqtgsa\",\"additionalColumns\":\"dataxd\",\"sourceRetryCount\":\"dataaskhasjbuhzucdl\",\"sourceRetryWait\":\"datajstncjwze\",\"maxConcurrentConnections\":\"dataezltlundkjphvh\",\"disableMetricsCollection\":\"dataivsh\",\"\":{\"xearlp\":\"datamcjyt\"}}") + "{\"type\":\"SapOpenHubSource\",\"excludeLastRequest\":\"databrngnbqhmuqyzxk\",\"baseRequestId\":\"datam\",\"customRfcReadTableFunctionModule\":\"datajshtcfnb\",\"sapDataColumnDelimiter\":\"datad\",\"queryTimeout\":\"datalyhx\",\"additionalColumns\":\"dataqoe\",\"sourceRetryCount\":\"databoqozxnuxamxikh\",\"sourceRetryWait\":\"dataikglynbqpeojecb\",\"maxConcurrentConnections\":\"datagw\",\"disableMetricsCollection\":\"datahtnywgtsodnxeir\",\"\":{\"g\":\"datajimcfrht\",\"a\":\"datauvoaxqo\",\"cyramvzu\":\"dataptfpbzyqbgg\",\"uwxslzq\":\"dataxtbrqnyurxl\"}}") .toObject(SapOpenHubSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapOpenHubSource model = new SapOpenHubSource().withSourceRetryCount("dataaskhasjbuhzucdl") - .withSourceRetryWait("datajstncjwze").withMaxConcurrentConnections("dataezltlundkjphvh") - .withDisableMetricsCollection("dataivsh").withQueryTimeout("datajkodpzqtgsa") - .withAdditionalColumns("dataxd").withExcludeLastRequest("datap").withBaseRequestId("datagfnqtxlk") - .withCustomRfcReadTableFunctionModule("dataviklxsgstunsa").withSapDataColumnDelimiter("dataistdbehkb"); + SapOpenHubSource model = new SapOpenHubSource().withSourceRetryCount("databoqozxnuxamxikh") + .withSourceRetryWait("dataikglynbqpeojecb").withMaxConcurrentConnections("datagw") + .withDisableMetricsCollection("datahtnywgtsodnxeir").withQueryTimeout("datalyhx") + .withAdditionalColumns("dataqoe").withExcludeLastRequest("databrngnbqhmuqyzxk").withBaseRequestId("datam") + .withCustomRfcReadTableFunctionModule("datajshtcfnb").withSapDataColumnDelimiter("datad"); model = BinaryData.fromObject(model).toObject(SapOpenHubSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTests.java index ae59f14f8be5f..a628310415bfa 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTests.java @@ -19,35 +19,30 @@ public final class SapOpenHubTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapOpenHubTableDataset model = BinaryData.fromString( - "{\"type\":\"SapOpenHubTable\",\"typeProperties\":{\"openHubDestinationName\":\"datauky\",\"excludeLastRequest\":\"dataxrjiqoqovqhg\",\"baseRequestId\":\"datagxuwudgcyqru\"},\"description\":\"mryddnqivahfcq\",\"structure\":\"datajzebp\",\"schema\":\"dataciyoypoedk\",\"linkedServiceName\":{\"referenceName\":\"pwwibpybqeig\",\"parameters\":{\"nhcgn\":\"dataxsxteuikhznff\"}},\"parameters\":{\"rkrgsdc\":{\"type\":\"SecureString\",\"defaultValue\":\"datarfqd\"},\"zfutgpbygbnb\":{\"type\":\"Int\",\"defaultValue\":\"datamgqlwyqznbbyzpo\"},\"ewflwzhxzuxe\":{\"type\":\"Array\",\"defaultValue\":\"dataiqgtzpv\"},\"ajdqxymxx\":{\"type\":\"Float\",\"defaultValue\":\"dataywlrkqsqvvdkfpfj\"}},\"annotations\":[\"datadjidcetfvgwfws\",\"datadigwoup\"],\"folder\":{\"name\":\"ddqsvclrsnxfrp\"},\"\":{\"tfxxepzpxzxlcqz\":\"dataqclmd\",\"jbsmkirpqni\":\"dataxaitiqm\",\"uzltenlb\":\"dataudmhkcomeobwk\",\"uomtxj\":\"dataxlmxozesndo\"}}") + "{\"type\":\"SapOpenHubTable\",\"typeProperties\":{\"openHubDestinationName\":\"datavqymcwtsi\",\"excludeLastRequest\":\"dataeplbrzgkuorwpqbs\",\"baseRequestId\":\"datae\"},\"description\":\"ptscru\",\"structure\":\"dataki\",\"schema\":\"dataayynoyj\",\"linkedServiceName\":{\"referenceName\":\"flsmsbnlyoifg\",\"parameters\":{\"zotkx\":\"datajqthykcvoevcw\",\"cgcvyp\":\"datalwwooxgbsd\",\"jcqgzwvxwi\":\"datahubdmgobxeh\"}},\"parameters\":{\"uky\":{\"type\":\"Int\",\"defaultValue\":\"datamjklqrljd\"}},\"annotations\":[\"datarjiqoqovqhgphgxu\",\"dataud\",\"datacy\"],\"folder\":{\"name\":\"vumryd\"},\"\":{\"iyoypoedkspwwibp\":\"dataivahfcqwnjzebpic\",\"znfffnhcgnaqsrm\":\"databqeigxuyxsxteuik\"}}") .toObject(SapOpenHubTableDataset.class); - Assertions.assertEquals("mryddnqivahfcq", model.description()); - Assertions.assertEquals("pwwibpybqeig", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("rkrgsdc").type()); - Assertions.assertEquals("ddqsvclrsnxfrp", model.folder().name()); + Assertions.assertEquals("ptscru", model.description()); + Assertions.assertEquals("flsmsbnlyoifg", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("uky").type()); + Assertions.assertEquals("vumryd", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapOpenHubTableDataset model = new SapOpenHubTableDataset().withDescription("mryddnqivahfcq") - .withStructure("datajzebp").withSchema("dataciyoypoedk") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pwwibpybqeig") - .withParameters(mapOf("nhcgn", "dataxsxteuikhznff"))) - .withParameters(mapOf("rkrgsdc", - new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datarfqd"), - "zfutgpbygbnb", - new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datamgqlwyqznbbyzpo"), - "ewflwzhxzuxe", - new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataiqgtzpv"), "ajdqxymxx", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataywlrkqsqvvdkfpfj"))) - .withAnnotations(Arrays.asList("datadjidcetfvgwfws", "datadigwoup")) - .withFolder(new DatasetFolder().withName("ddqsvclrsnxfrp")).withOpenHubDestinationName("datauky") - .withExcludeLastRequest("dataxrjiqoqovqhg").withBaseRequestId("datagxuwudgcyqru"); + SapOpenHubTableDataset model + = new SapOpenHubTableDataset().withDescription("ptscru").withStructure("dataki").withSchema("dataayynoyj") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("flsmsbnlyoifg").withParameters( + mapOf("zotkx", "datajqthykcvoevcw", "cgcvyp", "datalwwooxgbsd", "jcqgzwvxwi", "datahubdmgobxeh"))) + .withParameters(mapOf("uky", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datamjklqrljd"))) + .withAnnotations(Arrays.asList("datarjiqoqovqhgphgxu", "dataud", "datacy")) + .withFolder(new DatasetFolder().withName("vumryd")).withOpenHubDestinationName("datavqymcwtsi") + .withExcludeLastRequest("dataeplbrzgkuorwpqbs").withBaseRequestId("datae"); model = BinaryData.fromObject(model).toObject(SapOpenHubTableDataset.class); - Assertions.assertEquals("mryddnqivahfcq", model.description()); - Assertions.assertEquals("pwwibpybqeig", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("rkrgsdc").type()); - Assertions.assertEquals("ddqsvclrsnxfrp", model.folder().name()); + Assertions.assertEquals("ptscru", model.description()); + Assertions.assertEquals("flsmsbnlyoifg", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("uky").type()); + Assertions.assertEquals("vumryd", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTypePropertiesTests.java index 1465ecadf0672..56887321ddd5f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapOpenHubTableDatasetTypePropertiesTests.java @@ -11,15 +11,15 @@ public final class SapOpenHubTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapOpenHubTableDatasetTypeProperties model = BinaryData.fromString( - "{\"openHubDestinationName\":\"datari\",\"excludeLastRequest\":\"datamckik\",\"baseRequestId\":\"datayvurhwishy\"}") + "{\"openHubDestinationName\":\"datafqderkr\",\"excludeLastRequest\":\"datadcob\",\"baseRequestId\":\"datagqlwyqznbby\"}") .toObject(SapOpenHubTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SapOpenHubTableDatasetTypeProperties model - = new SapOpenHubTableDatasetTypeProperties().withOpenHubDestinationName("datari") - .withExcludeLastRequest("datamckik").withBaseRequestId("datayvurhwishy"); + = new SapOpenHubTableDatasetTypeProperties().withOpenHubDestinationName("datafqderkr") + .withExcludeLastRequest("datadcob").withBaseRequestId("datagqlwyqznbby"); model = BinaryData.fromObject(model).toObject(SapOpenHubTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTablePartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTablePartitionSettingsTests.java index 9bea041d42467..d4a0ab3ccb18e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTablePartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTablePartitionSettingsTests.java @@ -11,15 +11,15 @@ public final class SapTablePartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapTablePartitionSettings model = BinaryData.fromString( - "{\"partitionColumnName\":\"datauutkwwtymbc\",\"partitionUpperBound\":\"datawsyfsgikgcbjclf\",\"partitionLowerBound\":\"datafdsatrzqmtuxwtda\",\"maxPartitionsNumber\":\"databifktnxugiorb\"}") + "{\"partitionColumnName\":\"databdeyhweb\",\"partitionUpperBound\":\"datab\",\"partitionLowerBound\":\"datacfvajmmdmbyl\",\"maxPartitionsNumber\":\"datadtqujfzxsazuj\"}") .toObject(SapTablePartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapTablePartitionSettings model = new SapTablePartitionSettings().withPartitionColumnName("datauutkwwtymbc") - .withPartitionUpperBound("datawsyfsgikgcbjclf").withPartitionLowerBound("datafdsatrzqmtuxwtda") - .withMaxPartitionsNumber("databifktnxugiorb"); + SapTablePartitionSettings model + = new SapTablePartitionSettings().withPartitionColumnName("databdeyhweb").withPartitionUpperBound("datab") + .withPartitionLowerBound("datacfvajmmdmbyl").withMaxPartitionsNumber("datadtqujfzxsazuj"); model = BinaryData.fromObject(model).toObject(SapTablePartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTests.java index 0c643e50050de..01c114d6026e1 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTests.java @@ -19,31 +19,30 @@ public final class SapTableResourceDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapTableResourceDataset model = BinaryData.fromString( - "{\"type\":\"SapTableResource\",\"typeProperties\":{\"tableName\":\"dataycdzdob\"},\"description\":\"sdyvfxnzpfd\",\"structure\":\"datapk\",\"schema\":\"datapdpsegiv\",\"linkedServiceName\":{\"referenceName\":\"tabvbbkflewgsl\",\"parameters\":{\"vedwuu\":\"datab\",\"wclykcr\":\"databmenxcqs\",\"bnjrevmpted\":\"datadek\",\"shnfiygpgpkkhp\":\"datauent\"}},\"parameters\":{\"ihnmtrdlpxiwwg\":{\"type\":\"Float\",\"defaultValue\":\"dataql\"},\"fivxdifb\":{\"type\":\"Float\",\"defaultValue\":\"datavfpnrzikvoxloeoh\"}},\"annotations\":[\"dataijhpxukxgoyxontb\",\"datadqrxro\",\"datauqr\"],\"folder\":{\"name\":\"xfuaefewx\"},\"\":{\"mdcizhvk\":\"datatwjrppifeyrqvel\",\"pzwyncwksmpyeyzo\":\"databojklw\",\"uduiqoom\":\"databfnflytf\",\"opwsnliyznghuq\":\"dataswkq\"}}") + "{\"type\":\"SapTableResource\",\"typeProperties\":{\"tableName\":\"dataxwjoqfzwand\"},\"description\":\"duwd\",\"structure\":\"datalxtqm\",\"schema\":\"datac\",\"linkedServiceName\":{\"referenceName\":\"s\",\"parameters\":{\"oxedrmrazhvch\":\"databkiumjf\",\"zsoowxcsmxtlcapp\":\"datavoyiogbntnwzr\"}},\"parameters\":{\"pbkmzkwhjjs\":{\"type\":\"Array\",\"defaultValue\":\"dataspciryomhkdwuwed\"},\"vvkxdbnmc\":{\"type\":\"Float\",\"defaultValue\":\"dataaefe\"}},\"annotations\":[\"dataycdzdob\",\"dataesdyvf\"],\"folder\":{\"name\":\"pfdfu\"},\"\":{\"bbkfl\":\"datawpdpsegivytab\"}}") .toObject(SapTableResourceDataset.class); - Assertions.assertEquals("sdyvfxnzpfd", model.description()); - Assertions.assertEquals("tabvbbkflewgsl", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("ihnmtrdlpxiwwg").type()); - Assertions.assertEquals("xfuaefewx", model.folder().name()); + Assertions.assertEquals("duwd", model.description()); + Assertions.assertEquals("s", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("pbkmzkwhjjs").type()); + Assertions.assertEquals("pfdfu", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapTableResourceDataset model = new SapTableResourceDataset().withDescription("sdyvfxnzpfd") - .withStructure("datapk").withSchema("datapdpsegiv") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("tabvbbkflewgsl") - .withParameters(mapOf("vedwuu", "datab", "wclykcr", "databmenxcqs", "bnjrevmpted", "datadek", - "shnfiygpgpkkhp", "datauent"))) - .withParameters(mapOf("ihnmtrdlpxiwwg", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataql"), "fivxdifb", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datavfpnrzikvoxloeoh"))) - .withAnnotations(Arrays.asList("dataijhpxukxgoyxontb", "datadqrxro", "datauqr")) - .withFolder(new DatasetFolder().withName("xfuaefewx")).withTableName("dataycdzdob"); + SapTableResourceDataset model = new SapTableResourceDataset().withDescription("duwd").withStructure("datalxtqm") + .withSchema("datac") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("s") + .withParameters(mapOf("oxedrmrazhvch", "databkiumjf", "zsoowxcsmxtlcapp", "datavoyiogbntnwzr"))) + .withParameters(mapOf("pbkmzkwhjjs", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("dataspciryomhkdwuwed"), + "vvkxdbnmc", new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("dataaefe"))) + .withAnnotations(Arrays.asList("dataycdzdob", "dataesdyvf")) + .withFolder(new DatasetFolder().withName("pfdfu")).withTableName("dataxwjoqfzwand"); model = BinaryData.fromObject(model).toObject(SapTableResourceDataset.class); - Assertions.assertEquals("sdyvfxnzpfd", model.description()); - Assertions.assertEquals("tabvbbkflewgsl", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("ihnmtrdlpxiwwg").type()); - Assertions.assertEquals("xfuaefewx", model.folder().name()); + Assertions.assertEquals("duwd", model.description()); + Assertions.assertEquals("s", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.ARRAY, model.parameters().get("pbkmzkwhjjs").type()); + Assertions.assertEquals("pfdfu", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTypePropertiesTests.java index 994dd5babc004..a35d3b563a6a2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableResourceDatasetTypePropertiesTests.java @@ -10,14 +10,14 @@ public final class SapTableResourceDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SapTableResourceDatasetTypeProperties model = BinaryData.fromString("{\"tableName\":\"datagpdglkf\"}") + SapTableResourceDatasetTypeProperties model = BinaryData.fromString("{\"tableName\":\"datawgsltutbuve\"}") .toObject(SapTableResourceDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SapTableResourceDatasetTypeProperties model - = new SapTableResourceDatasetTypeProperties().withTableName("datagpdglkf"); + = new SapTableResourceDatasetTypeProperties().withTableName("datawgsltutbuve"); model = BinaryData.fromObject(model).toObject(SapTableResourceDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableSourceTests.java index d0c04b2a8c4b4..2386aa2d3beb3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SapTableSourceTests.java @@ -12,22 +12,21 @@ public final class SapTableSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SapTableSource model = BinaryData.fromString( - "{\"type\":\"SapTableSource\",\"rowCount\":\"dataixsjhinpyekslll\",\"rowSkips\":\"dataqolckwhgnfbnnh\",\"rfcTableFields\":\"datanloifxzdohfvxav\",\"rfcTableOptions\":\"datahlxqwwzp\",\"batchSize\":\"datamcfrf\",\"customRfcReadTableFunctionModule\":\"datatcygoombnrm\",\"sapDataColumnDelimiter\":\"dataklfp\",\"partitionOption\":\"datagfvvnkpwl\",\"partitionSettings\":{\"partitionColumnName\":\"dataxdzold\",\"partitionUpperBound\":\"datafnpn\",\"partitionLowerBound\":\"dataterjjuzarege\",\"maxPartitionsNumber\":\"dataozpudal\"},\"queryTimeout\":\"datagdhdtt\",\"additionalColumns\":\"datakeculxvkuxvccpda\",\"sourceRetryCount\":\"dataasi\",\"sourceRetryWait\":\"datatyvvgxe\",\"maxConcurrentConnections\":\"dataqoswjwbh\",\"disableMetricsCollection\":\"datawbchybne\",\"\":{\"jcywyrzxipxhl\":\"dataeikadhusgxkbg\",\"avxgmogcnwxk\":\"dataxkviyjruqyej\",\"thnlceggyqlvn\":\"dataqxpnjqtzdahv\"}}") + "{\"type\":\"SapTableSource\",\"rowCount\":\"dataiznbif\",\"rowSkips\":\"dataix\",\"rfcTableFields\":\"datakj\",\"rfcTableOptions\":\"dataxl\",\"batchSize\":\"datamvrblj\",\"customRfcReadTableFunctionModule\":\"datawsaskullvtsaujiv\",\"sapDataColumnDelimiter\":\"datazidzq\",\"partitionOption\":\"datafhzxkjygkuidgwdh\",\"partitionSettings\":{\"partitionColumnName\":\"datacozb\",\"partitionUpperBound\":\"datat\",\"partitionLowerBound\":\"datacmxqdexnkp\",\"maxPartitionsNumber\":\"dataxc\"},\"queryTimeout\":\"datamzykp\",\"additionalColumns\":\"datagiumuztb\",\"sourceRetryCount\":\"datatfmcnrgwgcsto\",\"sourceRetryWait\":\"dataveehmvr\",\"maxConcurrentConnections\":\"dataurpzry\",\"disableMetricsCollection\":\"datafdhch\",\"\":{\"fqsjzlcktkbg\":\"dataahaxyrdlvbom\",\"tle\":\"datateehyhxgnlpj\",\"krcohhuwzun\":\"dataymijhn\"}}") .toObject(SapTableSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SapTableSource model = new SapTableSource().withSourceRetryCount("dataasi").withSourceRetryWait("datatyvvgxe") - .withMaxConcurrentConnections("dataqoswjwbh").withDisableMetricsCollection("datawbchybne") - .withQueryTimeout("datagdhdtt").withAdditionalColumns("datakeculxvkuxvccpda") - .withRowCount("dataixsjhinpyekslll").withRowSkips("dataqolckwhgnfbnnh") - .withRfcTableFields("datanloifxzdohfvxav").withRfcTableOptions("datahlxqwwzp").withBatchSize("datamcfrf") - .withCustomRfcReadTableFunctionModule("datatcygoombnrm").withSapDataColumnDelimiter("dataklfp") - .withPartitionOption("datagfvvnkpwl") - .withPartitionSettings(new SapTablePartitionSettings().withPartitionColumnName("dataxdzold") - .withPartitionUpperBound("datafnpn").withPartitionLowerBound("dataterjjuzarege") - .withMaxPartitionsNumber("dataozpudal")); + SapTableSource model = new SapTableSource().withSourceRetryCount("datatfmcnrgwgcsto") + .withSourceRetryWait("dataveehmvr").withMaxConcurrentConnections("dataurpzry") + .withDisableMetricsCollection("datafdhch").withQueryTimeout("datamzykp") + .withAdditionalColumns("datagiumuztb").withRowCount("dataiznbif").withRowSkips("dataix") + .withRfcTableFields("datakj").withRfcTableOptions("dataxl").withBatchSize("datamvrblj") + .withCustomRfcReadTableFunctionModule("datawsaskullvtsaujiv").withSapDataColumnDelimiter("datazidzq") + .withPartitionOption("datafhzxkjygkuidgwdh").withPartitionSettings( + new SapTablePartitionSettings().withPartitionColumnName("datacozb").withPartitionUpperBound("datat") + .withPartitionLowerBound("datacmxqdexnkp").withMaxPartitionsNumber("dataxc")); model = BinaryData.fromObject(model).toObject(SapTableSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerRecurrenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerRecurrenceTests.java index 238f9cf323f3e..3160b1b94132f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerRecurrenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerRecurrenceTests.java @@ -21,53 +21,50 @@ public final class ScheduleTriggerRecurrenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScheduleTriggerRecurrence model = BinaryData.fromString( - "{\"frequency\":\"Day\",\"interval\":1432124324,\"startTime\":\"2021-06-09T21:39:21Z\",\"endTime\":\"2021-03-03T17:05:46Z\",\"timeZone\":\"aylsmehlzplz\",\"schedule\":{\"minutes\":[1544067436,1048730127,1978556507,1911134166],\"hours\":[877839804,817002067,1129234708,834608202],\"weekDays\":[\"Sunday\"],\"monthDays\":[1131953710],\"monthlyOccurrences\":[{\"day\":\"Sunday\",\"occurrence\":1658535673,\"\":{\"etrglp\":\"dataoerqkvufnphbzssa\",\"h\":\"dataocqxdvleouev\",\"q\":\"datagoqxfxjetu\",\"meyn\":\"dataacrdnmhr\"}},{\"day\":\"Sunday\",\"occurrence\":567651088,\"\":{\"rnwkt\":\"datakiocjn\"}},{\"day\":\"Monday\",\"occurrence\":1468937933,\"\":{\"xuzoxma\":\"dataewx\"}},{\"day\":\"Wednesday\",\"occurrence\":1587141246,\"\":{\"grmg\":\"datavtiq\",\"fz\":\"datafvfwwev\",\"vqa\":\"datahex\"}}],\"\":{\"iznriqucolpos\":\"datatogbl\",\"lwkj\":\"dataql\"}},\"\":{\"rj\":\"dataaln\",\"snqktbgu\":\"datalqhtyneyoxjhxha\",\"tcupo\":\"datafcr\"}}") + "{\"frequency\":\"Month\",\"interval\":917730009,\"startTime\":\"2021-03-30T03:31:07Z\",\"endTime\":\"2021-04-02T08:19:10Z\",\"timeZone\":\"zmrwlsrjjaj\",\"schedule\":{\"minutes\":[2020562074],\"hours\":[35520275,1915462888],\"weekDays\":[\"Wednesday\"],\"monthDays\":[1497257373,182132397,79673613,1340181084],\"monthlyOccurrences\":[{\"day\":\"Saturday\",\"occurrence\":1604982715,\"\":{\"zdkrmpljzrzv\":\"databkytrztwwkvwpbd\"}},{\"day\":\"Tuesday\",\"occurrence\":1218396039,\"\":{\"jrlugigzwh\":\"datayg\"}},{\"day\":\"Wednesday\",\"occurrence\":31016802,\"\":{\"yzzk\":\"datal\"}}],\"\":{\"ntgi\":\"dataeydjagyks\",\"b\":\"dataaazfjbxhnah\",\"a\":\"datao\",\"gbyxpma\":\"datawidumilxi\"}},\"\":{\"viqwfctiyaf\":\"datanqcb\"}}") .toObject(ScheduleTriggerRecurrence.class); - Assertions.assertEquals(RecurrenceFrequency.DAY, model.frequency()); - Assertions.assertEquals(1432124324, model.interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-09T21:39:21Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-03-03T17:05:46Z"), model.endTime()); - Assertions.assertEquals("aylsmehlzplz", model.timeZone()); - Assertions.assertEquals(1544067436, model.schedule().minutes().get(0)); - Assertions.assertEquals(877839804, model.schedule().hours().get(0)); - Assertions.assertEquals(DaysOfWeek.SUNDAY, model.schedule().weekDays().get(0)); - Assertions.assertEquals(1131953710, model.schedule().monthDays().get(0)); - Assertions.assertEquals(DayOfWeek.SUNDAY, model.schedule().monthlyOccurrences().get(0).day()); - Assertions.assertEquals(1658535673, model.schedule().monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(RecurrenceFrequency.MONTH, model.frequency()); + Assertions.assertEquals(917730009, model.interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-30T03:31:07Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-02T08:19:10Z"), model.endTime()); + Assertions.assertEquals("zmrwlsrjjaj", model.timeZone()); + Assertions.assertEquals(2020562074, model.schedule().minutes().get(0)); + Assertions.assertEquals(35520275, model.schedule().hours().get(0)); + Assertions.assertEquals(DaysOfWeek.WEDNESDAY, model.schedule().weekDays().get(0)); + Assertions.assertEquals(1497257373, model.schedule().monthDays().get(0)); + Assertions.assertEquals(DayOfWeek.SATURDAY, model.schedule().monthlyOccurrences().get(0).day()); + Assertions.assertEquals(1604982715, model.schedule().monthlyOccurrences().get(0).occurrence()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ScheduleTriggerRecurrence model = new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.DAY) - .withInterval(1432124324).withStartTime(OffsetDateTime.parse("2021-06-09T21:39:21Z")) - .withEndTime(OffsetDateTime.parse("2021-03-03T17:05:46Z")).withTimeZone("aylsmehlzplz") - .withSchedule( - new RecurrenceSchedule().withMinutes(Arrays.asList(1544067436, 1048730127, 1978556507, 1911134166)) - .withHours(Arrays.asList(877839804, 817002067, 1129234708, 834608202)) - .withWeekDays(Arrays.asList(DaysOfWeek.SUNDAY)).withMonthDays(Arrays.asList(1131953710)) - .withMonthlyOccurrences(Arrays.asList( - new RecurrenceScheduleOccurrence().withDay(DayOfWeek.SUNDAY).withOccurrence(1658535673) - .withAdditionalProperties(mapOf()), - new RecurrenceScheduleOccurrence().withDay(DayOfWeek.SUNDAY).withOccurrence(567651088) - .withAdditionalProperties(mapOf()), - new RecurrenceScheduleOccurrence().withDay(DayOfWeek.MONDAY).withOccurrence(1468937933) - .withAdditionalProperties(mapOf()), - new RecurrenceScheduleOccurrence().withDay(DayOfWeek.WEDNESDAY).withOccurrence(1587141246) - .withAdditionalProperties(mapOf()))) - .withAdditionalProperties(mapOf())) + ScheduleTriggerRecurrence model = new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.MONTH) + .withInterval(917730009).withStartTime(OffsetDateTime.parse("2021-03-30T03:31:07Z")) + .withEndTime(OffsetDateTime.parse("2021-04-02T08:19:10Z")).withTimeZone("zmrwlsrjjaj") + .withSchedule(new RecurrenceSchedule().withMinutes(Arrays.asList(2020562074)) + .withHours(Arrays.asList(35520275, 1915462888)).withWeekDays(Arrays.asList(DaysOfWeek.WEDNESDAY)) + .withMonthDays(Arrays.asList(1497257373, 182132397, 79673613, 1340181084)) + .withMonthlyOccurrences(Arrays.asList( + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.SATURDAY).withOccurrence(1604982715) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.TUESDAY).withOccurrence(1218396039) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.WEDNESDAY).withOccurrence(31016802) + .withAdditionalProperties(mapOf()))) + .withAdditionalProperties(mapOf())) .withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(ScheduleTriggerRecurrence.class); - Assertions.assertEquals(RecurrenceFrequency.DAY, model.frequency()); - Assertions.assertEquals(1432124324, model.interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-09T21:39:21Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-03-03T17:05:46Z"), model.endTime()); - Assertions.assertEquals("aylsmehlzplz", model.timeZone()); - Assertions.assertEquals(1544067436, model.schedule().minutes().get(0)); - Assertions.assertEquals(877839804, model.schedule().hours().get(0)); - Assertions.assertEquals(DaysOfWeek.SUNDAY, model.schedule().weekDays().get(0)); - Assertions.assertEquals(1131953710, model.schedule().monthDays().get(0)); - Assertions.assertEquals(DayOfWeek.SUNDAY, model.schedule().monthlyOccurrences().get(0).day()); - Assertions.assertEquals(1658535673, model.schedule().monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(RecurrenceFrequency.MONTH, model.frequency()); + Assertions.assertEquals(917730009, model.interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-30T03:31:07Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-02T08:19:10Z"), model.endTime()); + Assertions.assertEquals("zmrwlsrjjaj", model.timeZone()); + Assertions.assertEquals(2020562074, model.schedule().minutes().get(0)); + Assertions.assertEquals(35520275, model.schedule().hours().get(0)); + Assertions.assertEquals(DaysOfWeek.WEDNESDAY, model.schedule().weekDays().get(0)); + Assertions.assertEquals(1497257373, model.schedule().monthDays().get(0)); + Assertions.assertEquals(DayOfWeek.SATURDAY, model.schedule().monthlyOccurrences().get(0).day()); + Assertions.assertEquals(1604982715, model.schedule().monthlyOccurrences().get(0).occurrence()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTests.java index 62985e556c013..7a7f524496740 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTests.java @@ -24,61 +24,71 @@ public final class ScheduleTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScheduleTrigger model = BinaryData.fromString( - "{\"type\":\"ScheduleTrigger\",\"typeProperties\":{\"recurrence\":{\"frequency\":\"Day\",\"interval\":2044643491,\"startTime\":\"2021-10-29T18:37:49Z\",\"endTime\":\"2021-06-04T10:57:23Z\",\"timeZone\":\"fcupjmwtem\",\"schedule\":{\"minutes\":[967363974],\"hours\":[1814656348,390567237,15394793],\"weekDays\":[\"Sunday\",\"Monday\"],\"monthDays\":[136458587,836657339,1611891888],\"monthlyOccurrences\":[{\"day\":\"Thursday\",\"occurrence\":25086026,\"\":{\"r\":\"datankhxthkqnyvufv\",\"qqetasijiaqqwomk\":\"dataaphepcehe\"}},{\"day\":\"Saturday\",\"occurrence\":1509787884,\"\":{\"sxzxbafscdpbzp\":\"datahgpmvlfmvqumjm\",\"kzjazfw\":\"datarjhub\",\"cnyhsdgm\":\"datawvxq\",\"hxh\":\"dataxne\"}}],\"\":{\"ipwaheh\":\"datazbnkofcs\",\"jvuhcwcfzcsklv\":\"datactprgpmg\",\"bjfhpaywwesa\":\"dataceaoiuurqlcdh\"}},\"\":{\"vxbdlraridiat\":\"dataqps\",\"eyopgyygrnyfj\":\"datahxqsbyy\",\"omxmvgj\":\"dataoaomogkpcwffo\",\"jytoepcdhqjcz\":\"datazgqkxsoavbteaegy\"}}},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"lvujbhwosz\",\"name\":\"fwcihkjjjbi\"},\"parameters\":{\"gzxprtfw\":\"datariizy\",\"lgidqxbrdhuw\":\"dataasodsmjnrkdoomh\",\"iyfgkzwkyqa\":\"dataqnaspjdahienk\"}}],\"description\":\"i\",\"runtimeState\":\"Started\",\"annotations\":[\"datayk\",\"dataidiandktwijoxkk\",\"datanpp\"],\"\":{\"jcimoi\":\"datapjfowjatyhkq\",\"gyhschamwofqntt\":\"dataqzvawfpu\",\"hguubpmvp\":\"datakjcgupxnuv\"}}") + "{\"type\":\"ScheduleTrigger\",\"typeProperties\":{\"recurrence\":{\"frequency\":\"Minute\",\"interval\":1427518805,\"startTime\":\"2021-04-27T23:49:15Z\",\"endTime\":\"2021-06-28T00:05:17Z\",\"timeZone\":\"rnb\",\"schedule\":{\"minutes\":[1119797804],\"hours\":[999351737,732651337],\"weekDays\":[\"Tuesday\",\"Thursday\",\"Thursday\",\"Tuesday\"],\"monthDays\":[466726136,431958697,2021158859,959950633],\"monthlyOccurrences\":[{\"day\":\"Thursday\",\"occurrence\":104057412,\"\":{\"mnfavllbskl\":\"datahwaadcz\",\"kcea\":\"dataakkihxpofv\"}},{\"day\":\"Tuesday\",\"occurrence\":1036046655,\"\":{\"tg\":\"datadvaw\",\"naeclrjscdoqocdr\":\"dataqteg\"}},{\"day\":\"Thursday\",\"occurrence\":1417547783,\"\":{\"uuboyrfqyjtollug\":\"dataroaedswhbse\",\"kasbda\":\"datasvzi\",\"mutwmarfbsz\":\"datadsvdbdl\"}},{\"day\":\"Friday\",\"occurrence\":10999242,\"\":{\"hc\":\"datardmbebxmkwokl\"}}],\"\":{\"bfweezzrzfytq\":\"datagutitjwvvvapdsh\",\"hidh\":\"datanbxgofiphlwyzd\"}},\"\":{\"xcbfrnttlrumvi\":\"dataslczkzlbztsgklue\"}}},\"pipelines\":[{\"pipelineReference\":{\"referenceName\":\"dcavvj\",\"name\":\"nnhafed\"},\"parameters\":{\"vmuqxsoc\":\"dataxbauzvxejicpmgl\",\"djpugais\":\"datakjrtcifxl\",\"dmnvtpbohzcaaq\":\"datavsj\"}},{\"pipelineReference\":{\"referenceName\":\"daqfvplfy\",\"name\":\"b\"},\"parameters\":{\"bbvqsqw\":\"datashmqn\",\"ilqscjxpro\":\"datawxtqdtve\",\"iytkeqjviawspvbc\":\"datafyddrsairxnw\"}}],\"description\":\"heelmiuprfqyrwtd\",\"runtimeState\":\"Stopped\",\"annotations\":[\"datahewd\",\"dataualgkfo\",\"datanqmdtuyimdoprkp\",\"dataghqs\"],\"\":{\"hox\":\"dataebxjedyyeng\"}}") .toObject(ScheduleTrigger.class); - Assertions.assertEquals("i", model.description()); - Assertions.assertEquals("lvujbhwosz", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("fwcihkjjjbi", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals(RecurrenceFrequency.DAY, model.recurrence().frequency()); - Assertions.assertEquals(2044643491, model.recurrence().interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-29T18:37:49Z"), model.recurrence().startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-04T10:57:23Z"), model.recurrence().endTime()); - Assertions.assertEquals("fcupjmwtem", model.recurrence().timeZone()); - Assertions.assertEquals(967363974, model.recurrence().schedule().minutes().get(0)); - Assertions.assertEquals(1814656348, model.recurrence().schedule().hours().get(0)); - Assertions.assertEquals(DaysOfWeek.SUNDAY, model.recurrence().schedule().weekDays().get(0)); - Assertions.assertEquals(136458587, model.recurrence().schedule().monthDays().get(0)); + Assertions.assertEquals("heelmiuprfqyrwtd", model.description()); + Assertions.assertEquals("dcavvj", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("nnhafed", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals(RecurrenceFrequency.MINUTE, model.recurrence().frequency()); + Assertions.assertEquals(1427518805, model.recurrence().interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-27T23:49:15Z"), model.recurrence().startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-28T00:05:17Z"), model.recurrence().endTime()); + Assertions.assertEquals("rnb", model.recurrence().timeZone()); + Assertions.assertEquals(1119797804, model.recurrence().schedule().minutes().get(0)); + Assertions.assertEquals(999351737, model.recurrence().schedule().hours().get(0)); + Assertions.assertEquals(DaysOfWeek.TUESDAY, model.recurrence().schedule().weekDays().get(0)); + Assertions.assertEquals(466726136, model.recurrence().schedule().monthDays().get(0)); Assertions.assertEquals(DayOfWeek.THURSDAY, model.recurrence().schedule().monthlyOccurrences().get(0).day()); - Assertions.assertEquals(25086026, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(104057412, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ScheduleTrigger model = new ScheduleTrigger().withDescription("i") - .withAnnotations(Arrays.asList("datayk", "dataidiandktwijoxkk", "datanpp")) - .withPipelines(Arrays.asList(new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("lvujbhwosz").withName("fwcihkjjjbi")) - .withParameters(mapOf("gzxprtfw", "datariizy", "lgidqxbrdhuw", "dataasodsmjnrkdoomh", "iyfgkzwkyqa", - "dataqnaspjdahienk")))) - .withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.DAY) - .withInterval(2044643491).withStartTime(OffsetDateTime.parse("2021-10-29T18:37:49Z")) - .withEndTime(OffsetDateTime.parse("2021-06-04T10:57:23Z")).withTimeZone("fcupjmwtem") - .withSchedule(new RecurrenceSchedule().withMinutes(Arrays.asList(967363974)) - .withHours(Arrays.asList(1814656348, 390567237, 15394793)) - .withWeekDays(Arrays.asList(DaysOfWeek.SUNDAY, DaysOfWeek.MONDAY)) - .withMonthDays(Arrays.asList(136458587, 836657339, 1611891888)) + ScheduleTrigger model = new ScheduleTrigger().withDescription("heelmiuprfqyrwtd") + .withAnnotations(Arrays.asList("datahewd", "dataualgkfo", "datanqmdtuyimdoprkp", "dataghqs")) + .withPipelines(Arrays.asList( + new TriggerPipelineReference() + .withPipelineReference(new PipelineReference().withReferenceName("dcavvj").withName("nnhafed")) + .withParameters(mapOf("vmuqxsoc", "dataxbauzvxejicpmgl", "djpugais", "datakjrtcifxl", + "dmnvtpbohzcaaq", "datavsj")), + new TriggerPipelineReference() + .withPipelineReference(new PipelineReference().withReferenceName("daqfvplfy").withName("b")) + .withParameters(mapOf("bbvqsqw", "datashmqn", "ilqscjxpro", "datawxtqdtve", "iytkeqjviawspvbc", + "datafyddrsairxnw")))) + .withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.MINUTE) + .withInterval(1427518805).withStartTime(OffsetDateTime.parse("2021-04-27T23:49:15Z")) + .withEndTime(OffsetDateTime.parse("2021-06-28T00:05:17Z")).withTimeZone("rnb") + .withSchedule(new RecurrenceSchedule().withMinutes(Arrays.asList(1119797804)) + .withHours(Arrays.asList(999351737, 732651337)) + .withWeekDays( + Arrays.asList(DaysOfWeek.TUESDAY, DaysOfWeek.THURSDAY, DaysOfWeek.THURSDAY, DaysOfWeek.TUESDAY)) + .withMonthDays(Arrays.asList(466726136, 431958697, 2021158859, 959950633)) .withMonthlyOccurrences(Arrays.asList( - new RecurrenceScheduleOccurrence().withDay(DayOfWeek.THURSDAY).withOccurrence(25086026) + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.THURSDAY).withOccurrence(104057412) .withAdditionalProperties(mapOf()), - new RecurrenceScheduleOccurrence().withDay(DayOfWeek.SATURDAY).withOccurrence(1509787884) + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.TUESDAY).withOccurrence(1036046655) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.THURSDAY).withOccurrence(1417547783) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.FRIDAY).withOccurrence(10999242) .withAdditionalProperties(mapOf()))) .withAdditionalProperties(mapOf())) .withAdditionalProperties(mapOf())); model = BinaryData.fromObject(model).toObject(ScheduleTrigger.class); - Assertions.assertEquals("i", model.description()); - Assertions.assertEquals("lvujbhwosz", model.pipelines().get(0).pipelineReference().referenceName()); - Assertions.assertEquals("fwcihkjjjbi", model.pipelines().get(0).pipelineReference().name()); - Assertions.assertEquals(RecurrenceFrequency.DAY, model.recurrence().frequency()); - Assertions.assertEquals(2044643491, model.recurrence().interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-29T18:37:49Z"), model.recurrence().startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-04T10:57:23Z"), model.recurrence().endTime()); - Assertions.assertEquals("fcupjmwtem", model.recurrence().timeZone()); - Assertions.assertEquals(967363974, model.recurrence().schedule().minutes().get(0)); - Assertions.assertEquals(1814656348, model.recurrence().schedule().hours().get(0)); - Assertions.assertEquals(DaysOfWeek.SUNDAY, model.recurrence().schedule().weekDays().get(0)); - Assertions.assertEquals(136458587, model.recurrence().schedule().monthDays().get(0)); + Assertions.assertEquals("heelmiuprfqyrwtd", model.description()); + Assertions.assertEquals("dcavvj", model.pipelines().get(0).pipelineReference().referenceName()); + Assertions.assertEquals("nnhafed", model.pipelines().get(0).pipelineReference().name()); + Assertions.assertEquals(RecurrenceFrequency.MINUTE, model.recurrence().frequency()); + Assertions.assertEquals(1427518805, model.recurrence().interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-27T23:49:15Z"), model.recurrence().startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-28T00:05:17Z"), model.recurrence().endTime()); + Assertions.assertEquals("rnb", model.recurrence().timeZone()); + Assertions.assertEquals(1119797804, model.recurrence().schedule().minutes().get(0)); + Assertions.assertEquals(999351737, model.recurrence().schedule().hours().get(0)); + Assertions.assertEquals(DaysOfWeek.TUESDAY, model.recurrence().schedule().weekDays().get(0)); + Assertions.assertEquals(466726136, model.recurrence().schedule().monthDays().get(0)); Assertions.assertEquals(DayOfWeek.THURSDAY, model.recurrence().schedule().monthlyOccurrences().get(0).day()); - Assertions.assertEquals(25086026, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(104057412, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTypePropertiesTests.java index cb4e757d8dff1..fcfffe4dd1ff9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScheduleTriggerTypePropertiesTests.java @@ -22,47 +22,54 @@ public final class ScheduleTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScheduleTriggerTypeProperties model = BinaryData.fromString( - "{\"recurrence\":{\"frequency\":\"Year\",\"interval\":1833658573,\"startTime\":\"2021-08-26T04:22:28Z\",\"endTime\":\"2021-10-03T00:24:41Z\",\"timeZone\":\"bdzo\",\"schedule\":{\"minutes\":[1776980821],\"hours\":[1872232648,706756722],\"weekDays\":[\"Sunday\",\"Tuesday\",\"Monday\"],\"monthDays\":[1611379583,1004950603,1477251733],\"monthlyOccurrences\":[{\"day\":\"Wednesday\",\"occurrence\":458247050,\"\":{\"rmwdofgfspzwa\":\"dataqbyubswzaf\",\"wvjcdjvlwczw\":\"datacu\",\"fckrmrbaoidt\":\"datakkscooqnvht\",\"cbvkoughjsxp\":\"datam\"}}],\"\":{\"vyvscgzqncddaq\":\"datavppfdnihxcijftsb\",\"nex\":\"dataklvyibrohfvbg\",\"j\":\"datalsanglwnkkz\"}},\"\":{\"ajyrhrywucpdzbnt\":\"datarhjj\"}}}") + "{\"recurrence\":{\"frequency\":\"NotSpecified\",\"interval\":1095256215,\"startTime\":\"2021-02-27T12:11:30Z\",\"endTime\":\"2021-02-18T05:00:01Z\",\"timeZone\":\"vlsspptxdraji\",\"schedule\":{\"minutes\":[1667593447,1271402426,1565030338,906533139],\"hours\":[1448334718],\"weekDays\":[\"Friday\",\"Friday\"],\"monthDays\":[605430402,547584072],\"monthlyOccurrences\":[{\"day\":\"Thursday\",\"occurrence\":1741967093,\"\":{\"yamijgquizvvwyhs\":\"databrent\",\"iwyt\":\"dataewhzaiuoibw\",\"r\":\"datakp\"}},{\"day\":\"Friday\",\"occurrence\":1418940268,\"\":{\"ttiqac\":\"datanxlhdind\",\"jli\":\"datay\",\"hcylvjzufznaed\":\"datagkxrevwvjwtf\",\"dphyxlxvo\":\"datayuxrufwdbimj\"}},{\"day\":\"Wednesday\",\"occurrence\":1510978008,\"\":{\"pcbn\":\"datayttkaufab\"}}],\"\":{\"hllw\":\"datankvsnsi\",\"xtzgxdxq\":\"dataedzodvz\"}},\"\":{\"tuimi\":\"datazub\",\"encgfz\":\"dataccnubynr\",\"hxtbcqjvyzotxkhy\":\"databtzuddqt\"}}}") .toObject(ScheduleTriggerTypeProperties.class); - Assertions.assertEquals(RecurrenceFrequency.YEAR, model.recurrence().frequency()); - Assertions.assertEquals(1833658573, model.recurrence().interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-26T04:22:28Z"), model.recurrence().startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-03T00:24:41Z"), model.recurrence().endTime()); - Assertions.assertEquals("bdzo", model.recurrence().timeZone()); - Assertions.assertEquals(1776980821, model.recurrence().schedule().minutes().get(0)); - Assertions.assertEquals(1872232648, model.recurrence().schedule().hours().get(0)); - Assertions.assertEquals(DaysOfWeek.SUNDAY, model.recurrence().schedule().weekDays().get(0)); - Assertions.assertEquals(1611379583, model.recurrence().schedule().monthDays().get(0)); - Assertions.assertEquals(DayOfWeek.WEDNESDAY, model.recurrence().schedule().monthlyOccurrences().get(0).day()); - Assertions.assertEquals(458247050, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(RecurrenceFrequency.NOT_SPECIFIED, model.recurrence().frequency()); + Assertions.assertEquals(1095256215, model.recurrence().interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-27T12:11:30Z"), model.recurrence().startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-18T05:00:01Z"), model.recurrence().endTime()); + Assertions.assertEquals("vlsspptxdraji", model.recurrence().timeZone()); + Assertions.assertEquals(1667593447, model.recurrence().schedule().minutes().get(0)); + Assertions.assertEquals(1448334718, model.recurrence().schedule().hours().get(0)); + Assertions.assertEquals(DaysOfWeek.FRIDAY, model.recurrence().schedule().weekDays().get(0)); + Assertions.assertEquals(605430402, model.recurrence().schedule().monthDays().get(0)); + Assertions.assertEquals(DayOfWeek.THURSDAY, model.recurrence().schedule().monthlyOccurrences().get(0).day()); + Assertions.assertEquals(1741967093, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ScheduleTriggerTypeProperties model = new ScheduleTriggerTypeProperties() - .withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.YEAR) - .withInterval(1833658573).withStartTime(OffsetDateTime.parse("2021-08-26T04:22:28Z")) - .withEndTime(OffsetDateTime.parse("2021-10-03T00:24:41Z")).withTimeZone("bdzo") - .withSchedule(new RecurrenceSchedule().withMinutes(Arrays.asList(1776980821)) - .withHours(Arrays.asList(1872232648, 706756722)) - .withWeekDays(Arrays.asList(DaysOfWeek.SUNDAY, DaysOfWeek.TUESDAY, DaysOfWeek.MONDAY)) - .withMonthDays(Arrays.asList(1611379583, 1004950603, 1477251733)) - .withMonthlyOccurrences(Arrays.asList(new RecurrenceScheduleOccurrence() - .withDay(DayOfWeek.WEDNESDAY).withOccurrence(458247050).withAdditionalProperties(mapOf()))) - .withAdditionalProperties(mapOf())) - .withAdditionalProperties(mapOf())); + ScheduleTriggerTypeProperties model + = new ScheduleTriggerTypeProperties() + .withRecurrence(new ScheduleTriggerRecurrence().withFrequency(RecurrenceFrequency.NOT_SPECIFIED) + .withInterval(1095256215).withStartTime(OffsetDateTime.parse("2021-02-27T12:11:30Z")) + .withEndTime(OffsetDateTime.parse("2021-02-18T05:00:01Z")).withTimeZone("vlsspptxdraji") + .withSchedule(new RecurrenceSchedule() + .withMinutes(Arrays.asList(1667593447, 1271402426, 1565030338, 906533139)) + .withHours(Arrays.asList(1448334718)) + .withWeekDays(Arrays.asList(DaysOfWeek.FRIDAY, DaysOfWeek.FRIDAY)) + .withMonthDays(Arrays.asList(605430402, 547584072)) + .withMonthlyOccurrences(Arrays.asList( + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.THURSDAY).withOccurrence(1741967093) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.FRIDAY).withOccurrence(1418940268) + .withAdditionalProperties(mapOf()), + new RecurrenceScheduleOccurrence().withDay(DayOfWeek.WEDNESDAY).withOccurrence(1510978008) + .withAdditionalProperties(mapOf()))) + .withAdditionalProperties(mapOf())) + .withAdditionalProperties(mapOf())); model = BinaryData.fromObject(model).toObject(ScheduleTriggerTypeProperties.class); - Assertions.assertEquals(RecurrenceFrequency.YEAR, model.recurrence().frequency()); - Assertions.assertEquals(1833658573, model.recurrence().interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-26T04:22:28Z"), model.recurrence().startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-03T00:24:41Z"), model.recurrence().endTime()); - Assertions.assertEquals("bdzo", model.recurrence().timeZone()); - Assertions.assertEquals(1776980821, model.recurrence().schedule().minutes().get(0)); - Assertions.assertEquals(1872232648, model.recurrence().schedule().hours().get(0)); - Assertions.assertEquals(DaysOfWeek.SUNDAY, model.recurrence().schedule().weekDays().get(0)); - Assertions.assertEquals(1611379583, model.recurrence().schedule().monthDays().get(0)); - Assertions.assertEquals(DayOfWeek.WEDNESDAY, model.recurrence().schedule().monthlyOccurrences().get(0).day()); - Assertions.assertEquals(458247050, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); + Assertions.assertEquals(RecurrenceFrequency.NOT_SPECIFIED, model.recurrence().frequency()); + Assertions.assertEquals(1095256215, model.recurrence().interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-27T12:11:30Z"), model.recurrence().startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-18T05:00:01Z"), model.recurrence().endTime()); + Assertions.assertEquals("vlsspptxdraji", model.recurrence().timeZone()); + Assertions.assertEquals(1667593447, model.recurrence().schedule().minutes().get(0)); + Assertions.assertEquals(1448334718, model.recurrence().schedule().hours().get(0)); + Assertions.assertEquals(DaysOfWeek.FRIDAY, model.recurrence().schedule().weekDays().get(0)); + Assertions.assertEquals(605430402, model.recurrence().schedule().monthDays().get(0)); + Assertions.assertEquals(DayOfWeek.THURSDAY, model.recurrence().schedule().monthlyOccurrences().get(0).day()); + Assertions.assertEquals(1741967093, model.recurrence().schedule().monthlyOccurrences().get(0).occurrence()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActionTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActionTests.java index 2f24ed867431c..daa28eb2d6548 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActionTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActionTests.java @@ -13,20 +13,20 @@ public final class ScriptActionTests { public void testDeserialize() throws Exception { ScriptAction model = BinaryData .fromString( - "{\"name\":\"lyoix\",\"uri\":\"ei\",\"roles\":\"datanqizvsih\",\"parameters\":\"txjcajhsjuqqtzr\"}") + "{\"name\":\"kgxcewzg\",\"uri\":\"xz\",\"roles\":\"dataktcr\",\"parameters\":\"ttedzyzbvsjuths\"}") .toObject(ScriptAction.class); - Assertions.assertEquals("lyoix", model.name()); - Assertions.assertEquals("ei", model.uri()); - Assertions.assertEquals("txjcajhsjuqqtzr", model.parameters()); + Assertions.assertEquals("kgxcewzg", model.name()); + Assertions.assertEquals("xz", model.uri()); + Assertions.assertEquals("ttedzyzbvsjuths", model.parameters()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ScriptAction model = new ScriptAction().withName("lyoix").withUri("ei").withRoles("datanqizvsih") - .withParameters("txjcajhsjuqqtzr"); + ScriptAction model = new ScriptAction().withName("kgxcewzg").withUri("xz").withRoles("dataktcr") + .withParameters("ttedzyzbvsjuths"); model = BinaryData.fromObject(model).toObject(ScriptAction.class); - Assertions.assertEquals("lyoix", model.name()); - Assertions.assertEquals("ei", model.uri()); - Assertions.assertEquals("txjcajhsjuqqtzr", model.parameters()); + Assertions.assertEquals("kgxcewzg", model.name()); + Assertions.assertEquals("xz", model.uri()); + Assertions.assertEquals("ttedzyzbvsjuths", model.parameters()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityParameterTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityParameterTests.java index 6ce83b88aa32f..469434ba5af84 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityParameterTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityParameterTests.java @@ -14,21 +14,21 @@ public final class ScriptActivityParameterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScriptActivityParameter model = BinaryData.fromString( - "{\"name\":\"datau\",\"type\":\"DateTimeOffset\",\"value\":\"datajoshmmzotcpf\",\"direction\":\"Output\",\"size\":119206096}") + "{\"name\":\"dataj\",\"type\":\"Int64\",\"value\":\"dataiyzjdrkcsheoxss\",\"direction\":\"Output\",\"size\":1317589595}") .toObject(ScriptActivityParameter.class); - Assertions.assertEquals(ScriptActivityParameterType.DATE_TIME_OFFSET, model.type()); + Assertions.assertEquals(ScriptActivityParameterType.INT64, model.type()); Assertions.assertEquals(ScriptActivityParameterDirection.OUTPUT, model.direction()); - Assertions.assertEquals(119206096, model.size()); + Assertions.assertEquals(1317589595, model.size()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ScriptActivityParameter model = new ScriptActivityParameter().withName("datau") - .withType(ScriptActivityParameterType.DATE_TIME_OFFSET).withValue("datajoshmmzotcpf") - .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(119206096); + ScriptActivityParameter model = new ScriptActivityParameter().withName("dataj") + .withType(ScriptActivityParameterType.INT64).withValue("dataiyzjdrkcsheoxss") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(1317589595); model = BinaryData.fromObject(model).toObject(ScriptActivityParameter.class); - Assertions.assertEquals(ScriptActivityParameterType.DATE_TIME_OFFSET, model.type()); + Assertions.assertEquals(ScriptActivityParameterType.INT64, model.type()); Assertions.assertEquals(ScriptActivityParameterDirection.OUTPUT, model.direction()); - Assertions.assertEquals(119206096, model.size()); + Assertions.assertEquals(1317589595, model.size()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityScriptBlockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityScriptBlockTests.java index f09f005c60e09..c1bb7241275b7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityScriptBlockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityScriptBlockTests.java @@ -17,25 +17,35 @@ public final class ScriptActivityScriptBlockTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScriptActivityScriptBlock model = BinaryData.fromString( - "{\"text\":\"datacgmyjmcw\",\"type\":\"Query\",\"parameters\":[{\"name\":\"datarvbzvi\",\"type\":\"Int16\",\"value\":\"dataeblrnu\",\"direction\":\"Output\",\"size\":1720230643}]}") + "{\"text\":\"datacqaejlebcyd\",\"type\":\"Query\",\"parameters\":[{\"name\":\"dataprpwsgazdkcike\",\"type\":\"Double\",\"value\":\"dataialebc\",\"direction\":\"Output\",\"size\":2102195500},{\"name\":\"dataepjsfhxh\",\"type\":\"Guid\",\"value\":\"datakroyjd\",\"direction\":\"Output\",\"size\":2121426093},{\"name\":\"datakgthydyz\",\"type\":\"Single\",\"value\":\"dataguesoivaoryefg\",\"direction\":\"Output\",\"size\":1602176624},{\"name\":\"dataksdatjtg\",\"type\":\"Timespan\",\"value\":\"dataqvindhixddcocs\",\"direction\":\"Output\",\"size\":497633824}]}") .toObject(ScriptActivityScriptBlock.class); Assertions.assertEquals(ScriptType.QUERY, model.type()); - Assertions.assertEquals(ScriptActivityParameterType.INT16, model.parameters().get(0).type()); + Assertions.assertEquals(ScriptActivityParameterType.DOUBLE, model.parameters().get(0).type()); Assertions.assertEquals(ScriptActivityParameterDirection.OUTPUT, model.parameters().get(0).direction()); - Assertions.assertEquals(1720230643, model.parameters().get(0).size()); + Assertions.assertEquals(2102195500, model.parameters().get(0).size()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ScriptActivityScriptBlock model - = new ScriptActivityScriptBlock().withText("datacgmyjmcw").withType(ScriptType.QUERY) - .withParameters(Arrays.asList(new ScriptActivityParameter().withName("datarvbzvi") - .withType(ScriptActivityParameterType.INT16).withValue("dataeblrnu") - .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(1720230643))); + = new ScriptActivityScriptBlock().withText("datacqaejlebcyd").withType(ScriptType.QUERY) + .withParameters(Arrays.asList( + new ScriptActivityParameter().withName("dataprpwsgazdkcike") + .withType(ScriptActivityParameterType.DOUBLE).withValue("dataialebc") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(2102195500), + new ScriptActivityParameter().withName("dataepjsfhxh").withType(ScriptActivityParameterType.GUID) + .withValue("datakroyjd").withDirection(ScriptActivityParameterDirection.OUTPUT) + .withSize(2121426093), + new ScriptActivityParameter().withName("datakgthydyz").withType(ScriptActivityParameterType.SINGLE) + .withValue("dataguesoivaoryefg").withDirection(ScriptActivityParameterDirection.OUTPUT) + .withSize(1602176624), + new ScriptActivityParameter().withName("dataksdatjtg") + .withType(ScriptActivityParameterType.TIMESPAN).withValue("dataqvindhixddcocs") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(497633824))); model = BinaryData.fromObject(model).toObject(ScriptActivityScriptBlock.class); Assertions.assertEquals(ScriptType.QUERY, model.type()); - Assertions.assertEquals(ScriptActivityParameterType.INT16, model.parameters().get(0).type()); + Assertions.assertEquals(ScriptActivityParameterType.DOUBLE, model.parameters().get(0).type()); Assertions.assertEquals(ScriptActivityParameterDirection.OUTPUT, model.parameters().get(0).direction()); - Assertions.assertEquals(1720230643, model.parameters().get(0).size()); + Assertions.assertEquals(2102195500, model.parameters().get(0).size()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTests.java index 88f5425c5b470..7b9bdd3059869 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTests.java @@ -30,95 +30,104 @@ public final class ScriptActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScriptActivity model = BinaryData.fromString( - "{\"type\":\"Script\",\"typeProperties\":{\"scriptBlockExecutionTimeout\":\"datanmdeimagmw\",\"scripts\":[{\"text\":\"datae\",\"type\":\"Query\",\"parameters\":[{\"name\":\"datakllbfnn\",\"type\":\"Int64\",\"value\":\"datagq\",\"direction\":\"Input\",\"size\":1030847893},{\"name\":\"datayrhgeuvujy\",\"type\":\"Int32\",\"value\":\"datajayiexpcxy\",\"direction\":\"Input\",\"size\":279017132},{\"name\":\"datanwactjpgwse\",\"type\":\"Single\",\"value\":\"dataxghaylrvpau\",\"direction\":\"Input\",\"size\":864680631}]}],\"logSettings\":{\"logDestination\":\"ExternalStore\",\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"kfipxolpu\",\"parameters\":{\"vouxcdenthgpw\":\"databpjnalsb\",\"iryhlarohwq\":\"datamevkckocexhlvf\"}},\"path\":\"datahzwsjqrmxpyzn\"}}},\"linkedServiceName\":{\"referenceName\":\"mnkg\",\"parameters\":{\"sfu\":\"dataayfyvodctpczzq\"}},\"policy\":{\"timeout\":\"datarrrgijolyspkyswy\",\"retry\":\"datajffvfkkhgqsjec\",\"retryIntervalInSeconds\":1874344477,\"secureInput\":true,\"secureOutput\":true,\"\":{\"rfzaurermnyphcoo\":\"datahqfxf\",\"obzrnvubszjytt\":\"datast\"}},\"name\":\"kpscmtccz\",\"description\":\"la\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"hi\",\"dependencyConditions\":[\"Skipped\",\"Completed\"],\"\":{\"zdngbgpxoef\":\"datae\",\"albkemodlvdhvdvd\":\"databefavbsbhdtiaqa\",\"nd\":\"datarrkvxmeihrzi\"}},{\"activity\":\"pojmgkeoqrxhdsu\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Failed\"],\"\":{\"eikbvax\":\"datav\"}},{\"activity\":\"hiegkpukv\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Completed\",\"Skipped\"],\"\":{\"jlikiogdtdekm\":\"datati\",\"phjsmkhkuknccdbs\":\"datazsmya\",\"khzwfnsduwt\":\"dataizadmbnqyswpnog\",\"qttzgyond\":\"datarvgzjfptpr\"}},{\"activity\":\"vtfkdz\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Completed\"],\"\":{\"bdhpizkrqkylm\":\"dataqbezvx\",\"zduqthqgngrf\":\"dataydiodcgwbkfcajt\",\"fvjfaqah\":\"dataaexewftqo\"}}],\"userProperties\":[{\"name\":\"kdsbp\",\"value\":\"databdajc\"},{\"name\":\"rlnxjucoj\",\"value\":\"datajryppvdhklcczg\"},{\"name\":\"logzstc\",\"value\":\"datawb\"}],\"\":{\"lxhve\":\"datafrbuvwugwwr\",\"uiodwblau\":\"dataoodxmmtyumejpq\",\"xydvceuywy\":\"datah\"}}") + "{\"type\":\"Script\",\"typeProperties\":{\"scriptBlockExecutionTimeout\":\"dataikiogdtde\",\"scripts\":[{\"text\":\"datazsmya\",\"type\":\"NonQuery\",\"parameters\":[{\"name\":\"datamkhkuknccdbs\",\"type\":\"String\",\"value\":\"datadmbnqyswpnog\",\"direction\":\"Input\",\"size\":2077560759},{\"name\":\"datansduwttrvg\",\"type\":\"Timespan\",\"value\":\"datatprfqttz\",\"direction\":\"Output\",\"size\":1804249174}]},{\"text\":\"datazvtfkdzqtkxiyjq\",\"type\":\"Query\",\"parameters\":[{\"name\":\"dataebdhpizk\",\"type\":\"DateTimeOffset\",\"value\":\"datalmfydiodcgwbk\",\"direction\":\"Output\",\"size\":1540037039},{\"name\":\"datazduqthqgngrf\",\"type\":\"Double\",\"value\":\"dataewftq\",\"direction\":\"Output\",\"size\":563491415},{\"name\":\"dataaqahmeskd\",\"type\":\"Decimal\",\"value\":\"databdajc\",\"direction\":\"Output\",\"size\":130816507}]}],\"logSettings\":{\"logDestination\":\"ActivityOutput\",\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"ojzjryp\",\"parameters\":{\"zst\":\"datahklcczgflo\",\"gwwrclxhveso\":\"datafwbftafrbuvw\",\"xuiod\":\"datadxmmtyumejp\",\"aujhox\":\"datab\"}},\"path\":\"datavceuyw\"}}},\"linkedServiceName\":{\"referenceName\":\"mtzlcvokvo\",\"parameters\":{\"ccnlvyhietezn\":\"datajdy\",\"zulkk\":\"datalsqymvihhgpe\",\"rysanifcfrtijz\":\"datayfiuxdadc\",\"zebqbdewepoan\":\"dataum\"}},\"policy\":{\"timeout\":\"dataspde\",\"retry\":\"datajgjdqlazune\",\"retryIntervalInSeconds\":2015861080,\"secureInput\":false,\"secureOutput\":true,\"\":{\"utjjqzqixsvo\":\"datatiswxcvw\"}},\"name\":\"bjsrisfcc\",\"description\":\"u\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"kyvu\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Succeeded\"],\"\":{\"qk\":\"dataicjmvspydtladf\",\"ue\":\"dataci\"}},{\"activity\":\"yljw\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Completed\"],\"\":{\"bacgrysjgzuzo\":\"datarvbuphbwais\",\"ydd\":\"dataupdcmpfwfdcpedu\",\"nkhgg\":\"datapfdhfp\"}}],\"userProperties\":[{\"name\":\"apsypgmmieheq\",\"value\":\"datatetnywgme\"},{\"name\":\"ihaboj\",\"value\":\"datargetnc\"},{\"name\":\"ljwjrpljkc\",\"value\":\"dataed\"}],\"\":{\"acc\":\"dataefzlwohob\",\"cnoeiqhbr\":\"datalvixf\",\"kpbrr\":\"datacgmyjmcw\",\"teblrnu\":\"databzvink\"}}") .toObject(ScriptActivity.class); - Assertions.assertEquals("kpscmtccz", model.name()); - Assertions.assertEquals("la", model.description()); + Assertions.assertEquals("bjsrisfcc", model.name()); + Assertions.assertEquals("u", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("hi", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("kdsbp", model.userProperties().get(0).name()); - Assertions.assertEquals("mnkg", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1874344477, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("kyvu", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("apsypgmmieheq", model.userProperties().get(0).name()); + Assertions.assertEquals("mtzlcvokvo", model.linkedServiceName().referenceName()); + Assertions.assertEquals(2015861080, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals(ScriptType.QUERY, model.scripts().get(0).type()); - Assertions.assertEquals(ScriptActivityParameterType.INT64, model.scripts().get(0).parameters().get(0).type()); + Assertions.assertEquals(ScriptType.NON_QUERY, model.scripts().get(0).type()); + Assertions.assertEquals(ScriptActivityParameterType.STRING, model.scripts().get(0).parameters().get(0).type()); Assertions.assertEquals(ScriptActivityParameterDirection.INPUT, model.scripts().get(0).parameters().get(0).direction()); - Assertions.assertEquals(1030847893, model.scripts().get(0).parameters().get(0).size()); - Assertions.assertEquals(ScriptActivityLogDestination.EXTERNAL_STORE, model.logSettings().logDestination()); - Assertions.assertEquals("kfipxolpu", + Assertions.assertEquals(2077560759, model.scripts().get(0).parameters().get(0).size()); + Assertions.assertEquals(ScriptActivityLogDestination.ACTIVITY_OUTPUT, model.logSettings().logDestination()); + Assertions.assertEquals("ojzjryp", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ScriptActivity model = new ScriptActivity().withName("kpscmtccz").withDescription("la") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn( - Arrays.asList( - new ActivityDependency().withActivity("hi") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("pojmgkeoqrxhdsu") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("hiegkpukv") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("vtfkdz") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("kdsbp").withValue("databdajc"), - new UserProperty().withName("rlnxjucoj").withValue("datajryppvdhklcczg"), - new UserProperty().withName("logzstc").withValue("datawb"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("mnkg") - .withParameters(mapOf("sfu", "dataayfyvodctpczzq"))) - .withPolicy(new ActivityPolicy().withTimeout("datarrrgijolyspkyswy").withRetry("datajffvfkkhgqsjec") - .withRetryIntervalInSeconds(1874344477).withSecureInput(true).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withScriptBlockExecutionTimeout("datanmdeimagmw") - .withScripts(Arrays.asList(new ScriptActivityScriptBlock().withText("datae").withType(ScriptType.QUERY) - .withParameters(Arrays.asList( - new ScriptActivityParameter().withName("datakllbfnn").withType(ScriptActivityParameterType.INT64) - .withValue("datagq").withDirection(ScriptActivityParameterDirection.INPUT).withSize(1030847893), - new ScriptActivityParameter().withName("datayrhgeuvujy").withType(ScriptActivityParameterType.INT32) - .withValue("datajayiexpcxy").withDirection(ScriptActivityParameterDirection.INPUT) - .withSize(279017132), - new ScriptActivityParameter().withName("datanwactjpgwse") - .withType(ScriptActivityParameterType.SINGLE).withValue("dataxghaylrvpau") - .withDirection(ScriptActivityParameterDirection.INPUT).withSize(864680631))))) - .withLogSettings(new ScriptActivityTypePropertiesLogSettings() - .withLogDestination(ScriptActivityLogDestination.EXTERNAL_STORE) - .withLogLocationSettings(new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("kfipxolpu") - .withParameters(mapOf("vouxcdenthgpw", "databpjnalsb", "iryhlarohwq", "datamevkckocexhlvf"))) - .withPath("datahzwsjqrmxpyzn"))); + ScriptActivity model = new ScriptActivity().withName("bjsrisfcc").withDescription("u") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("kyvu") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("yljw") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, + DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("apsypgmmieheq").withValue("datatetnywgme"), + new UserProperty().withName("ihaboj").withValue("datargetnc"), + new UserProperty().withName("ljwjrpljkc").withValue("dataed"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("mtzlcvokvo") + .withParameters(mapOf("ccnlvyhietezn", "datajdy", "zulkk", "datalsqymvihhgpe", "rysanifcfrtijz", + "datayfiuxdadc", "zebqbdewepoan", "dataum"))) + .withPolicy( + new ActivityPolicy().withTimeout("dataspde").withRetry("datajgjdqlazune") + .withRetryIntervalInSeconds(2015861080).withSecureInput( + false) + .withSecureOutput(true).withAdditionalProperties(mapOf())) + .withScriptBlockExecutionTimeout( + "dataikiogdtde") + .withScripts( + Arrays + .asList(new ScriptActivityScriptBlock().withText("datazsmya").withType(ScriptType.NON_QUERY) + .withParameters(Arrays.asList( + new ScriptActivityParameter().withName("datamkhkuknccdbs") + .withType(ScriptActivityParameterType.STRING).withValue("datadmbnqyswpnog") + .withDirection(ScriptActivityParameterDirection.INPUT).withSize(2077560759), + new ScriptActivityParameter().withName("datansduwttrvg") + .withType(ScriptActivityParameterType.TIMESPAN).withValue("datatprfqttz").withDirection( + ScriptActivityParameterDirection.OUTPUT) + .withSize(1804249174))), + new ScriptActivityScriptBlock().withText("datazvtfkdzqtkxiyjq").withType(ScriptType.QUERY) + .withParameters(Arrays.asList(new ScriptActivityParameter().withName("dataebdhpizk") + .withType(ScriptActivityParameterType.DATE_TIME_OFFSET).withValue("datalmfydiodcgwbk") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(1540037039), + new ScriptActivityParameter().withName("datazduqthqgngrf") + .withType(ScriptActivityParameterType.DOUBLE).withValue("dataewftq") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(563491415), + new ScriptActivityParameter().withName("dataaqahmeskd") + .withType(ScriptActivityParameterType.DECIMAL).withValue("databdajc") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(130816507))))) + .withLogSettings( + new ScriptActivityTypePropertiesLogSettings() + .withLogDestination( + ScriptActivityLogDestination.ACTIVITY_OUTPUT) + .withLogLocationSettings(new LogLocationSettings() + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ojzjryp") + .withParameters(mapOf("zst", "datahklcczgflo", "gwwrclxhveso", "datafwbftafrbuvw", "xuiod", + "datadxmmtyumejp", "aujhox", "datab"))) + .withPath("datavceuyw"))); model = BinaryData.fromObject(model).toObject(ScriptActivity.class); - Assertions.assertEquals("kpscmtccz", model.name()); - Assertions.assertEquals("la", model.description()); + Assertions.assertEquals("bjsrisfcc", model.name()); + Assertions.assertEquals("u", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("hi", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("kdsbp", model.userProperties().get(0).name()); - Assertions.assertEquals("mnkg", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1874344477, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("kyvu", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("apsypgmmieheq", model.userProperties().get(0).name()); + Assertions.assertEquals("mtzlcvokvo", model.linkedServiceName().referenceName()); + Assertions.assertEquals(2015861080, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(false, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); - Assertions.assertEquals(ScriptType.QUERY, model.scripts().get(0).type()); - Assertions.assertEquals(ScriptActivityParameterType.INT64, model.scripts().get(0).parameters().get(0).type()); + Assertions.assertEquals(ScriptType.NON_QUERY, model.scripts().get(0).type()); + Assertions.assertEquals(ScriptActivityParameterType.STRING, model.scripts().get(0).parameters().get(0).type()); Assertions.assertEquals(ScriptActivityParameterDirection.INPUT, model.scripts().get(0).parameters().get(0).direction()); - Assertions.assertEquals(1030847893, model.scripts().get(0).parameters().get(0).size()); - Assertions.assertEquals(ScriptActivityLogDestination.EXTERNAL_STORE, model.logSettings().logDestination()); - Assertions.assertEquals("kfipxolpu", + Assertions.assertEquals(2077560759, model.scripts().get(0).parameters().get(0).size()); + Assertions.assertEquals(ScriptActivityLogDestination.ACTIVITY_OUTPUT, model.logSettings().logDestination()); + Assertions.assertEquals("ojzjryp", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesLogSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesLogSettingsTests.java index fd35851019323..b222078c3fc67 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesLogSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesLogSettingsTests.java @@ -17,10 +17,10 @@ public final class ScriptActivityTypePropertiesLogSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScriptActivityTypePropertiesLogSettings model = BinaryData.fromString( - "{\"logDestination\":\"ActivityOutput\",\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"fbeott\",\"parameters\":{\"ut\":\"dataonejpjzqb\",\"atfalhnixo\":\"datanlow\"}},\"path\":\"dataqobngjbeihcaxkiv\"}}") + "{\"logDestination\":\"ActivityOutput\",\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"yyxhzgxkwc\",\"parameters\":{\"fylbokbriy\":\"datarrmlkrroqsdvxdd\",\"buravswnnsb\":\"datarxae\"}},\"path\":\"dataum\"}}") .toObject(ScriptActivityTypePropertiesLogSettings.class); Assertions.assertEquals(ScriptActivityLogDestination.ACTIVITY_OUTPUT, model.logDestination()); - Assertions.assertEquals("fbeott", model.logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("yyxhzgxkwc", model.logLocationSettings().linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test @@ -28,12 +28,12 @@ public void testSerialize() throws Exception { ScriptActivityTypePropertiesLogSettings model = new ScriptActivityTypePropertiesLogSettings() .withLogDestination(ScriptActivityLogDestination.ACTIVITY_OUTPUT) .withLogLocationSettings(new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fbeott") - .withParameters(mapOf("ut", "dataonejpjzqb", "atfalhnixo", "datanlow"))) - .withPath("dataqobngjbeihcaxkiv")); + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("yyxhzgxkwc") + .withParameters(mapOf("fylbokbriy", "datarrmlkrroqsdvxdd", "buravswnnsb", "datarxae"))) + .withPath("dataum")); model = BinaryData.fromObject(model).toObject(ScriptActivityTypePropertiesLogSettings.class); Assertions.assertEquals(ScriptActivityLogDestination.ACTIVITY_OUTPUT, model.logDestination()); - Assertions.assertEquals("fbeott", model.logLocationSettings().linkedServiceName().referenceName()); + Assertions.assertEquals("yyxhzgxkwc", model.logLocationSettings().linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesTests.java index a345118f1c288..e550e2a4a3b0d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ScriptActivityTypePropertiesTests.java @@ -24,68 +24,51 @@ public final class ScriptActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ScriptActivityTypeProperties model = BinaryData.fromString( - "{\"scriptBlockExecutionTimeout\":\"datat\",\"scripts\":[{\"text\":\"datavokvoqjbjdy\",\"type\":\"NonQuery\",\"parameters\":[{\"name\":\"datavyhieteznnlsqym\",\"type\":\"DateTimeOffset\",\"value\":\"datag\",\"direction\":\"InputOutput\",\"size\":1931524739}]},{\"text\":\"datalkkhyf\",\"type\":\"Query\",\"parameters\":[{\"name\":\"datadcdrysanifcf\",\"type\":\"Decimal\",\"value\":\"datazsumgzeb\",\"direction\":\"Output\",\"size\":1582956476},{\"name\":\"datapoantsrspdee\",\"type\":\"Int32\",\"value\":\"datadqlazunedsozj\",\"direction\":\"Input\",\"size\":2031000573},{\"name\":\"dataswxcvwh\",\"type\":\"Decimal\",\"value\":\"dataqzqixsvo\",\"direction\":\"Input\",\"size\":1465142989},{\"name\":\"datasfccfzuzmihifrky\",\"type\":\"Int64\",\"value\":\"datamc\",\"direction\":\"InputOutput\",\"size\":199145977}]},{\"text\":\"dataj\",\"type\":\"NonQuery\",\"parameters\":[{\"name\":\"datadtladfc\",\"type\":\"Int16\",\"value\":\"dataizuegyljw\",\"direction\":\"Input\",\"size\":2086951314},{\"name\":\"datawrvbuphbwais\",\"type\":\"Timespan\",\"value\":\"datagrysjgz\",\"direction\":\"InputOutput\",\"size\":1620731574}]}],\"logSettings\":{\"logDestination\":\"ActivityOutput\",\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"mpfwfdcpeduxydd\",\"parameters\":{\"iapsypgmmi\":\"datadhfpvnkhgga\",\"wg\":\"dataheqmtetn\",\"jwjrpljkcqed\":\"dataeiihabojargetncf\",\"obaaccglvix\":\"dataaeefzlwo\"}},\"path\":\"datacnoeiqhbr\"}}}") + "{\"scriptBlockExecutionTimeout\":\"dataandmusud\",\"scripts\":[{\"text\":\"datas\",\"type\":\"Query\",\"parameters\":[{\"name\":\"datatcpffmi\",\"type\":\"Timespan\",\"value\":\"datafbeott\",\"direction\":\"InputOutput\",\"size\":1855489224}]},{\"text\":\"dataejpjzqbdutvnlowv\",\"type\":\"Query\",\"parameters\":[{\"name\":\"datah\",\"type\":\"Boolean\",\"value\":\"databdqobngjbeihcaxk\",\"direction\":\"Output\",\"size\":1878476180},{\"name\":\"datacxnnirnfuvesmepq\",\"type\":\"Int64\",\"value\":\"dataptsvn\",\"direction\":\"InputOutput\",\"size\":473285132}]}],\"logSettings\":{\"logDestination\":\"ActivityOutput\",\"logLocationSettings\":{\"linkedServiceName\":{\"referenceName\":\"fmwbte\",\"parameters\":{\"cyvmsduodpmtisg\":\"datamndt\"}},\"path\":\"datae\"}}}") .toObject(ScriptActivityTypeProperties.class); - Assertions.assertEquals(ScriptType.NON_QUERY, model.scripts().get(0).type()); - Assertions.assertEquals(ScriptActivityParameterType.DATE_TIME_OFFSET, + Assertions.assertEquals(ScriptType.QUERY, model.scripts().get(0).type()); + Assertions.assertEquals(ScriptActivityParameterType.TIMESPAN, model.scripts().get(0).parameters().get(0).type()); Assertions.assertEquals(ScriptActivityParameterDirection.INPUT_OUTPUT, model.scripts().get(0).parameters().get(0).direction()); - Assertions.assertEquals(1931524739, model.scripts().get(0).parameters().get(0).size()); + Assertions.assertEquals(1855489224, model.scripts().get(0).parameters().get(0).size()); Assertions.assertEquals(ScriptActivityLogDestination.ACTIVITY_OUTPUT, model.logSettings().logDestination()); - Assertions.assertEquals("mpfwfdcpeduxydd", + Assertions.assertEquals("fmwbte", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ScriptActivityTypeProperties model - = new ScriptActivityTypeProperties().withScriptBlockExecutionTimeout("datat") - .withScripts( - Arrays.asList( - new ScriptActivityScriptBlock().withText("datavokvoqjbjdy").withType(ScriptType.NON_QUERY) - .withParameters(Arrays.asList(new ScriptActivityParameter().withName("datavyhieteznnlsqym") - .withType(ScriptActivityParameterType.DATE_TIME_OFFSET).withValue("datag") - .withDirection(ScriptActivityParameterDirection.INPUT_OUTPUT).withSize(1931524739))), - new ScriptActivityScriptBlock().withText("datalkkhyf").withType(ScriptType.QUERY) - .withParameters(Arrays.asList(new ScriptActivityParameter().withName("datadcdrysanifcf") - .withType(ScriptActivityParameterType.DECIMAL).withValue("datazsumgzeb") - .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(1582956476), - new ScriptActivityParameter().withName("datapoantsrspdee") - .withType(ScriptActivityParameterType.INT32).withValue("datadqlazunedsozj") - .withDirection(ScriptActivityParameterDirection.INPUT).withSize(2031000573), - new ScriptActivityParameter().withName("dataswxcvwh") - .withType(ScriptActivityParameterType.DECIMAL).withValue("dataqzqixsvo") - .withDirection(ScriptActivityParameterDirection.INPUT).withSize(1465142989), - new ScriptActivityParameter().withName("datasfccfzuzmihifrky") - .withType(ScriptActivityParameterType.INT64).withValue("datamc") - .withDirection(ScriptActivityParameterDirection.INPUT_OUTPUT).withSize(199145977))), - new ScriptActivityScriptBlock().withText("dataj").withType(ScriptType.NON_QUERY) - .withParameters(Arrays.asList( - new ScriptActivityParameter().withName("datadtladfc") - .withType(ScriptActivityParameterType.INT16).withValue("dataizuegyljw") - .withDirection(ScriptActivityParameterDirection.INPUT).withSize(2086951314), - new ScriptActivityParameter().withName("datawrvbuphbwais") - .withType(ScriptActivityParameterType.TIMESPAN).withValue("datagrysjgz") - .withDirection(ScriptActivityParameterDirection.INPUT_OUTPUT) - .withSize(1620731574))))) - .withLogSettings( - new ScriptActivityTypePropertiesLogSettings() - .withLogDestination(ScriptActivityLogDestination.ACTIVITY_OUTPUT) - .withLogLocationSettings(new LogLocationSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("mpfwfdcpeduxydd") - .withParameters(mapOf("iapsypgmmi", "datadhfpvnkhgga", "wg", "dataheqmtetn", - "jwjrpljkcqed", "dataeiihabojargetncf", "obaaccglvix", "dataaeefzlwo"))) - .withPath("datacnoeiqhbr"))); + = new ScriptActivityTypeProperties().withScriptBlockExecutionTimeout("dataandmusud") + .withScripts(Arrays.asList( + new ScriptActivityScriptBlock().withText("datas").withType(ScriptType.QUERY) + .withParameters(Arrays.asList(new ScriptActivityParameter().withName("datatcpffmi") + .withType(ScriptActivityParameterType.TIMESPAN).withValue("datafbeott") + .withDirection(ScriptActivityParameterDirection.INPUT_OUTPUT).withSize(1855489224))), + new ScriptActivityScriptBlock().withText("dataejpjzqbdutvnlowv").withType(ScriptType.QUERY) + .withParameters(Arrays.asList( + new ScriptActivityParameter().withName("datah") + .withType(ScriptActivityParameterType.BOOLEAN).withValue("databdqobngjbeihcaxk") + .withDirection(ScriptActivityParameterDirection.OUTPUT).withSize(1878476180), + new ScriptActivityParameter().withName("datacxnnirnfuvesmepq") + .withType(ScriptActivityParameterType.INT64).withValue("dataptsvn") + .withDirection(ScriptActivityParameterDirection.INPUT_OUTPUT).withSize(473285132))))) + .withLogSettings(new ScriptActivityTypePropertiesLogSettings() + .withLogDestination(ScriptActivityLogDestination.ACTIVITY_OUTPUT) + .withLogLocationSettings(new LogLocationSettings() + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fmwbte") + .withParameters(mapOf("cyvmsduodpmtisg", "datamndt"))) + .withPath("datae"))); model = BinaryData.fromObject(model).toObject(ScriptActivityTypeProperties.class); - Assertions.assertEquals(ScriptType.NON_QUERY, model.scripts().get(0).type()); - Assertions.assertEquals(ScriptActivityParameterType.DATE_TIME_OFFSET, + Assertions.assertEquals(ScriptType.QUERY, model.scripts().get(0).type()); + Assertions.assertEquals(ScriptActivityParameterType.TIMESPAN, model.scripts().get(0).parameters().get(0).type()); Assertions.assertEquals(ScriptActivityParameterDirection.INPUT_OUTPUT, model.scripts().get(0).parameters().get(0).direction()); - Assertions.assertEquals(1931524739, model.scripts().get(0).parameters().get(0).size()); + Assertions.assertEquals(1855489224, model.scripts().get(0).parameters().get(0).size()); Assertions.assertEquals(ScriptActivityLogDestination.ACTIVITY_OUTPUT, model.logSettings().logDestination()); - Assertions.assertEquals("mpfwfdcpeduxydd", + Assertions.assertEquals("fmwbte", model.logSettings().logLocationSettings().linkedServiceName().referenceName()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SecureInputOutputPolicyTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SecureInputOutputPolicyTests.java index ee3cc9e105d20..f5c60df18322a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SecureInputOutputPolicyTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SecureInputOutputPolicyTests.java @@ -11,17 +11,17 @@ public final class SecureInputOutputPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SecureInputOutputPolicy model = BinaryData.fromString("{\"secureInput\":true,\"secureOutput\":true}") + SecureInputOutputPolicy model = BinaryData.fromString("{\"secureInput\":true,\"secureOutput\":false}") .toObject(SecureInputOutputPolicy.class); Assertions.assertEquals(true, model.secureInput()); - Assertions.assertEquals(true, model.secureOutput()); + Assertions.assertEquals(false, model.secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SecureInputOutputPolicy model = new SecureInputOutputPolicy().withSecureInput(true).withSecureOutput(true); + SecureInputOutputPolicy model = new SecureInputOutputPolicy().withSecureInput(true).withSecureOutput(false); model = BinaryData.fromObject(model).toObject(SecureInputOutputPolicy.class); Assertions.assertEquals(true, model.secureInput()); - Assertions.assertEquals(true, model.secureOutput()); + Assertions.assertEquals(false, model.secureOutput()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfDependencyTumblingWindowTriggerReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfDependencyTumblingWindowTriggerReferenceTests.java index c19c7ebf4fa9c..3effbe53e062d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfDependencyTumblingWindowTriggerReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfDependencyTumblingWindowTriggerReferenceTests.java @@ -11,20 +11,19 @@ public final class SelfDependencyTumblingWindowTriggerReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SelfDependencyTumblingWindowTriggerReference model = BinaryData - .fromString( - "{\"type\":\"SelfDependencyTumblingWindowTriggerReference\",\"offset\":\"mny\",\"size\":\"mctn\"}") + SelfDependencyTumblingWindowTriggerReference model = BinaryData.fromString( + "{\"type\":\"SelfDependencyTumblingWindowTriggerReference\",\"offset\":\"vpoyhvfcwed\",\"size\":\"eroezgibfisfmcx\"}") .toObject(SelfDependencyTumblingWindowTriggerReference.class); - Assertions.assertEquals("mny", model.offset()); - Assertions.assertEquals("mctn", model.size()); + Assertions.assertEquals("vpoyhvfcwed", model.offset()); + Assertions.assertEquals("eroezgibfisfmcx", model.size()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SelfDependencyTumblingWindowTriggerReference model - = new SelfDependencyTumblingWindowTriggerReference().withOffset("mny").withSize("mctn"); + = new SelfDependencyTumblingWindowTriggerReference().withOffset("vpoyhvfcwed").withSize("eroezgibfisfmcx"); model = BinaryData.fromObject(model).toObject(SelfDependencyTumblingWindowTriggerReference.class); - Assertions.assertEquals("mny", model.offset()); - Assertions.assertEquals("mctn", model.size()); + Assertions.assertEquals("vpoyhvfcwed", model.offset()); + Assertions.assertEquals("eroezgibfisfmcx", model.size()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTests.java index e61b8ea8c1500..ea1872e2dc90e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTests.java @@ -18,23 +18,62 @@ public final class SelfHostedIntegrationRuntimeStatusTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SelfHostedIntegrationRuntimeStatus model = BinaryData.fromString( - "{\"type\":\"SelfHosted\",\"typeProperties\":{\"createTime\":\"2021-05-27T17:25:40Z\",\"taskQueueId\":\"exqzaffzqodoks\",\"internalChannelEncryption\":\"NotEncrypted\",\"version\":\"mdqxnoyzqipapifc\",\"nodes\":[{\"nodeName\":\"jghunqnt\",\"machineName\":\"wocb\",\"hostServiceUri\":\"vxdvphxmw\",\"status\":\"NeedRegistration\",\"capabilities\":{\"k\":\"icbicbogsfo\"},\"versionStatus\":\"miy\",\"version\":\"un\",\"registerTime\":\"2021-12-06T23:34:48Z\",\"lastConnectTime\":\"2021-03-17T19:59:30Z\",\"expiryTime\":\"2021-01-31T02:37:30Z\",\"lastStartTime\":\"2021-08-02T03:26:08Z\",\"lastStopTime\":\"2021-09-03T01:49:37Z\",\"lastUpdateResult\":\"Fail\",\"lastStartUpdateTime\":\"2021-03-31T07:19:51Z\",\"lastEndUpdateTime\":\"2021-06-26T12:31:16Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":992750702,\"maxConcurrentJobs\":447712322,\"\":{\"pumzuh\":\"datakifqbxmnnidot\",\"pva\":\"dataodttq\",\"avzlhjgmrodbl\":\"dataumvuwjyulqfpqql\"}}],\"scheduledUpdateDate\":\"2021-03-23T10:47:05Z\",\"updateDelayOffset\":\"acz\",\"localTimeZoneOffset\":\"niwfc\",\"capabilities\":{\"kcegct\":\"hxmyib\"},\"serviceUrls\":[\"kxtcxb\",\"jbeyqohviawp\",\"fkrarer\",\"lgbvtpxowgoww\"],\"autoUpdate\":\"On\",\"versionStatus\":\"asu\",\"links\":[{\"name\":\"jkfiszhexumfav\",\"subscriptionId\":\"sloblitxr\",\"dataFactoryName\":\"jscosanjsoutrzt\",\"dataFactoryLocation\":\"ujgbnclxhwkz\",\"createTime\":\"2021-11-21T22:45:06Z\"},{\"name\":\"uz\",\"subscriptionId\":\"d\",\"dataFactoryName\":\"tpzu\",\"dataFactoryLocation\":\"azoabthutc\",\"createTime\":\"2021-11-23T00:57:26Z\"},{\"name\":\"cpwabzfihszfkpo\",\"subscriptionId\":\"fzwegvuojuwgwe\",\"dataFactoryName\":\"vufjqvfcfsssm\",\"dataFactoryLocation\":\"emkrhbsdgktluifi\",\"createTime\":\"2021-02-02T18:13:56Z\"},{\"name\":\"cpenobqysbees\",\"subscriptionId\":\"bvvaerszsufzsa\",\"dataFactoryName\":\"bric\",\"dataFactoryLocation\":\"ofenin\",\"createTime\":\"2021-04-27T16:12:17Z\"}],\"pushedVersion\":\"hyyqxckdlxjp\",\"latestVersion\":\"rdnowincbes\",\"autoUpdateETA\":\"2021-07-24T02:05:35Z\",\"selfContainedInteractiveAuthoringEnabled\":false},\"dataFactoryName\":\"bxfiiytqxewj\",\"state\":\"Stopping\",\"\":{\"rxkivbkutogecyqo\":\"datazlghkvoxdp\",\"ya\":\"datatwssbvqnpwdwdmu\",\"blzrmiukothyfjbp\":\"datarbqpwx\"}}") + "{\"type\":\"SelfHosted\",\"typeProperties\":{\"createTime\":\"2021-07-01T23:20:22Z\",\"taskQueueId\":\"s\",\"internalChannelEncryption\":\"SslEncrypted\",\"version\":\"emkrhbsdgktluifi\",\"nodes\":[{\"nodeName\":\"cpenobqysbees\",\"machineName\":\"bvvaerszsufzsa\",\"hostServiceUri\":\"bric\",\"status\":\"Limited\",\"capabilities\":{\"ntzunhyyqxckdlx\":\"n\",\"fvijnu\":\"pisrdnowincbe\"},\"versionStatus\":\"fiiytqxewjsyu\",\"version\":\"zlghkvoxdp\",\"registerTime\":\"2021-05-27T22:13:23Z\",\"lastConnectTime\":\"2021-12-03T10:30:18Z\",\"expiryTime\":\"2021-08-14T12:28:51Z\",\"lastStartTime\":\"2021-09-05T12:26:50Z\",\"lastStopTime\":\"2021-10-25T10:44:43Z\",\"lastUpdateResult\":\"Succeed\",\"lastStartUpdateTime\":\"2021-10-18T15:20:22Z\",\"lastEndUpdateTime\":\"2021-02-18T11:20:26Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":1098696529,\"maxConcurrentJobs\":1386786245,\"\":{\"pwxnblzrmi\":\"datanpwdwdmuvyakrb\"}},{\"nodeName\":\"othyfjbp\",\"machineName\":\"dhfrvsi\",\"hostServiceUri\":\"wgnpcjnia\",\"status\":\"NeedRegistration\",\"capabilities\":{\"uuogdkpnm\":\"jjioq\",\"xqucnbgibkls\":\"rfuqjdeb\",\"evbfvxmtsmgkret\":\"wdkouzyvi\",\"ceulbyz\":\"ny\"},\"versionStatus\":\"xsygaoymyckd\",\"version\":\"brxbmljrjyfjl\",\"registerTime\":\"2021-08-29T14:58:42Z\",\"lastConnectTime\":\"2021-10-18T07:17:24Z\",\"expiryTime\":\"2021-03-14T00:44:48Z\",\"lastStartTime\":\"2021-04-09T05:29:06Z\",\"lastStopTime\":\"2021-08-10T05:40:55Z\",\"lastUpdateResult\":\"Succeed\",\"lastStartUpdateTime\":\"2021-08-17T06:45Z\",\"lastEndUpdateTime\":\"2021-02-26T23:11:52Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":1855197747,\"maxConcurrentJobs\":582722035,\"\":{\"csotwqtkpdcdefqo\":\"datai\",\"nddaxaogsk\":\"datarmgm\",\"fjcvmkkbpgdiwd\":\"datacmmmbipysehyybo\",\"cehzrtrgpd\":\"datayhdtiembrwqwvc\"}},{\"nodeName\":\"t\",\"machineName\":\"hyfwjfqktuzr\",\"hostServiceUri\":\"pecsdk\",\"status\":\"Online\",\"capabilities\":{\"hjlugcupcyfrhoo\":\"vttqjntvhnjp\",\"vuxyeeafdxsuwly\":\"v\",\"hj\":\"xzhgbspdx\"},\"versionStatus\":\"xkzxqomzdfa\",\"version\":\"qve\",\"registerTime\":\"2021-04-19T01:11:47Z\",\"lastConnectTime\":\"2021-01-05T03:49:30Z\",\"expiryTime\":\"2021-12-04T14:35:50Z\",\"lastStartTime\":\"2021-09-24T04:45:33Z\",\"lastStopTime\":\"2021-08-21T16:12:51Z\",\"lastUpdateResult\":\"Succeed\",\"lastStartUpdateTime\":\"2021-01-08T15:40:51Z\",\"lastEndUpdateTime\":\"2021-07-23T22:08:33Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":1876720469,\"maxConcurrentJobs\":988799296,\"\":{\"wqlbxmynslcvyn\":\"dataxapew\",\"czroddc\":\"datavwttts\",\"k\":\"dataqimodnbjmj\",\"fja\":\"databucmzkqt\"}},{\"nodeName\":\"tavc\",\"machineName\":\"godjfyplavb\",\"hostServiceUri\":\"ecedsoqwexie\",\"status\":\"Online\",\"capabilities\":{\"fadgywylavetq\":\"eikffjq\",\"tdgj\":\"vohy\"},\"versionStatus\":\"btkogfggyl\",\"version\":\"olrvwsgseqjt\",\"registerTime\":\"2021-12-09T01:57:30Z\",\"lastConnectTime\":\"2021-08-03T22:33:15Z\",\"expiryTime\":\"2021-01-29T17:19:57Z\",\"lastStartTime\":\"2021-04-07T03:22:50Z\",\"lastStopTime\":\"2021-02-27T15:10:16Z\",\"lastUpdateResult\":\"None\",\"lastStartUpdateTime\":\"2021-01-22T08:52:14Z\",\"lastEndUpdateTime\":\"2021-06-26T15:14:55Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":99010631,\"maxConcurrentJobs\":1381935048,\"\":{\"oflzuk\":\"datairdzdgv\",\"sxblmnxrxkuly\":\"datagougxpypbmz\",\"qon\":\"dataivvi\"}}],\"scheduledUpdateDate\":\"2021-11-20T22:58:39Z\",\"updateDelayOffset\":\"fozbgodywxj\",\"localTimeZoneOffset\":\"frxvlusedpnkz\",\"capabilities\":{\"vvyjehy\":\"axtmvmycvjpaxjd\",\"v\":\"nfjngoqmr\",\"xunwenbphyl\":\"bgtuhw\"},\"serviceUrls\":[\"gzrxxdusebkcfet\"],\"autoUpdate\":\"Off\",\"versionStatus\":\"terma\",\"links\":[{\"name\":\"rsnmftubqwxvsxr\",\"subscriptionId\":\"yzjlgrwjbsycukb\",\"dataFactoryName\":\"sitsxhvs\",\"dataFactoryLocation\":\"pwqieyxjkctyqst\",\"createTime\":\"2021-02-25T12:41:44Z\"},{\"name\":\"qepeftmub\",\"subscriptionId\":\"zoepeqlhbtysyiz\",\"dataFactoryName\":\"lctpqnofkwh\",\"dataFactoryLocation\":\"zwfyfdbvoo\",\"createTime\":\"2020-12-20T02:18:07Z\"},{\"name\":\"kd\",\"subscriptionId\":\"qykgjjsmvsi\",\"dataFactoryName\":\"mlmwj\",\"dataFactoryLocation\":\"m\",\"createTime\":\"2021-08-14T08:35:22Z\"},{\"name\":\"acvemmriyz\",\"subscriptionId\":\"quesxplcsinbulo\",\"dataFactoryName\":\"xhcynnmv\",\"dataFactoryLocation\":\"zvkwqqpw\",\"createTime\":\"2021-02-12T08:54:34Z\"}],\"pushedVersion\":\"jqcqyzmrtfdlgpr\",\"latestVersion\":\"jl\",\"autoUpdateETA\":\"2021-02-22T05:22Z\",\"selfContainedInteractiveAuthoringEnabled\":true},\"dataFactoryName\":\"yuwa\",\"state\":\"Stopping\",\"\":{\"nuhgy\":\"datavaidzcephn\",\"mrwpe\":\"datazkhi\",\"rjbpertjpair\":\"datai\"}}") .toObject(SelfHostedIntegrationRuntimeStatus.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SelfHostedIntegrationRuntimeStatus model = new SelfHostedIntegrationRuntimeStatus() - .withNodes(Arrays.asList(new SelfHostedIntegrationRuntimeNodeInner().withAdditionalProperties(mapOf( - "nodeName", "jghunqnt", "lastStartUpdateTime", "2021-03-31T07:19:51Z", "lastConnectTime", - "2021-03-17T19:59:30Z", "capabilities", - JacksonAdapter.createDefaultSerializerAdapter().deserialize("{\"k\":\"icbicbogsfo\"}", Object.class, - SerializerEncoding.JSON), - "hostServiceUri", "vxdvphxmw", "registerTime", "2021-12-06T23:34:48Z", "maxConcurrentJobs", 447712322, - "lastStopTime", "2021-09-03T01:49:37Z", "version", "un", "machineName", "wocb", "versionStatus", "miy", - "concurrentJobsLimit", 992750702, "lastEndUpdateTime", "2021-06-26T12:31:16Z", "expiryTime", - "2021-01-31T02:37:30Z", "lastStartTime", "2021-08-02T03:26:08Z", "lastUpdateResult", "Fail", - "isActiveDispatcher", true, "status", "NeedRegistration")))) + .withNodes(Arrays.asList( + new SelfHostedIntegrationRuntimeNodeInner() + .withAdditionalProperties(mapOf("nodeName", "cpenobqysbees", "lastStartUpdateTime", + "2021-10-18T15:20:22Z", "lastConnectTime", "2021-12-03T10:30:18Z", "capabilities", + JacksonAdapter + .createDefaultSerializerAdapter() + .deserialize("{\"ntzunhyyqxckdlx\":\"n\",\"fvijnu\":\"pisrdnowincbe\"}", Object.class, + SerializerEncoding.JSON), + "hostServiceUri", "bric", "registerTime", "2021-05-27T22:13:23Z", "maxConcurrentJobs", + 1386786245, "lastStopTime", "2021-10-25T10:44:43Z", "version", "zlghkvoxdp", "machineName", + "bvvaerszsufzsa", "versionStatus", "fiiytqxewjsyu", "concurrentJobsLimit", 1098696529, + "lastEndUpdateTime", "2021-02-18T11:20:26Z", "expiryTime", "2021-08-14T12:28:51Z", + "lastStartTime", "2021-09-05T12:26:50Z", "lastUpdateResult", "Succeed", "isActiveDispatcher", + true, "status", "Limited")), + new SelfHostedIntegrationRuntimeNodeInner().withAdditionalProperties(mapOf("nodeName", "othyfjbp", + "lastStartUpdateTime", "2021-08-17T06:45Z", "lastConnectTime", "2021-10-18T07:17:24Z", + "capabilities", + JacksonAdapter.createDefaultSerializerAdapter().deserialize( + "{\"uuogdkpnm\":\"jjioq\",\"xqucnbgibkls\":\"rfuqjdeb\",\"evbfvxmtsmgkret\":\"wdkouzyvi\",\"ceulbyz\":\"ny\"}", + Object.class, SerializerEncoding.JSON), + "hostServiceUri", "wgnpcjnia", "registerTime", "2021-08-29T14:58:42Z", "maxConcurrentJobs", + 582722035, "lastStopTime", "2021-08-10T05:40:55Z", "version", "brxbmljrjyfjl", "machineName", + "dhfrvsi", "versionStatus", "xsygaoymyckd", "concurrentJobsLimit", 1855197747, "lastEndUpdateTime", + "2021-02-26T23:11:52Z", "expiryTime", "2021-03-14T00:44:48Z", "lastStartTime", + "2021-04-09T05:29:06Z", "lastUpdateResult", "Succeed", "isActiveDispatcher", true, "status", + "NeedRegistration")), + new SelfHostedIntegrationRuntimeNodeInner().withAdditionalProperties(mapOf("nodeName", "t", + "lastStartUpdateTime", "2021-01-08T15:40:51Z", "lastConnectTime", "2021-01-05T03:49:30Z", + "capabilities", + JacksonAdapter.createDefaultSerializerAdapter().deserialize( + "{\"hjlugcupcyfrhoo\":\"vttqjntvhnjp\",\"vuxyeeafdxsuwly\":\"v\",\"hj\":\"xzhgbspdx\"}", + Object.class, SerializerEncoding.JSON), + "hostServiceUri", "pecsdk", "registerTime", "2021-04-19T01:11:47Z", "maxConcurrentJobs", 988799296, + "lastStopTime", "2021-08-21T16:12:51Z", "version", "qve", "machineName", "hyfwjfqktuzr", + "versionStatus", "xkzxqomzdfa", "concurrentJobsLimit", 1876720469, "lastEndUpdateTime", + "2021-07-23T22:08:33Z", "expiryTime", "2021-12-04T14:35:50Z", "lastStartTime", + "2021-09-24T04:45:33Z", "lastUpdateResult", "Succeed", "isActiveDispatcher", true, "status", + "Online")), + new SelfHostedIntegrationRuntimeNodeInner() + .withAdditionalProperties(mapOf("nodeName", "tavc", "lastStartUpdateTime", "2021-01-22T08:52:14Z", + "lastConnectTime", "2021-08-03T22:33:15Z", "capabilities", + JacksonAdapter.createDefaultSerializerAdapter().deserialize( + "{\"fadgywylavetq\":\"eikffjq\",\"tdgj\":\"vohy\"}", Object.class, SerializerEncoding.JSON), + "hostServiceUri", "ecedsoqwexie", "registerTime", "2021-12-09T01:57:30Z", "maxConcurrentJobs", + 1381935048, "lastStopTime", "2021-02-27T15:10:16Z", "version", "olrvwsgseqjt", "machineName", + "godjfyplavb", "versionStatus", "btkogfggyl", "concurrentJobsLimit", 99010631, + "lastEndUpdateTime", "2021-06-26T15:14:55Z", "expiryTime", "2021-01-29T17:19:57Z", + "lastStartTime", "2021-04-07T03:22:50Z", "lastUpdateResult", "None", "isActiveDispatcher", true, + "status", "Online")))) .withLinks(Arrays.asList(new LinkedIntegrationRuntime(), new LinkedIntegrationRuntime(), new LinkedIntegrationRuntime(), new LinkedIntegrationRuntime())); model = BinaryData.fromObject(model).toObject(SelfHostedIntegrationRuntimeStatus.class); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTypePropertiesTests.java index e33ff517ca5c3..0fb0cdcc02f42 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SelfHostedIntegrationRuntimeStatusTypePropertiesTests.java @@ -18,7 +18,7 @@ public final class SelfHostedIntegrationRuntimeStatusTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SelfHostedIntegrationRuntimeStatusTypeProperties model = BinaryData.fromString( - "{\"createTime\":\"2021-07-18T03:37:51Z\",\"taskQueueId\":\"frvsizfwgnp\",\"internalChannelEncryption\":\"SslEncrypted\",\"version\":\"affwcgjjioqw\",\"nodes\":[{\"nodeName\":\"dkpnmwrfuqjdeby\",\"machineName\":\"ucnbgibklscw\",\"hostServiceUri\":\"ouzyv\",\"status\":\"Online\",\"capabilities\":{\"xm\":\"f\",\"tsnyr\":\"smgkr\"},\"versionStatus\":\"ulbyzzcxs\",\"version\":\"aoymyckdpzb\",\"registerTime\":\"2021-08-26T04:54:27Z\",\"lastConnectTime\":\"2021-04-27T18:45:48Z\",\"expiryTime\":\"2021-03-06T04:59:11Z\",\"lastStartTime\":\"2020-12-23T10:34:23Z\",\"lastStopTime\":\"2021-04-09T22:30:41Z\",\"lastUpdateResult\":\"Succeed\",\"lastStartUpdateTime\":\"2021-08-29T14:58:42Z\",\"lastEndUpdateTime\":\"2021-10-18T07:17:24Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":1193471045,\"maxConcurrentJobs\":415008463,\"\":{\"twqtkpd\":\"dataimqoacfqzxjziqcs\",\"nddaxaogsk\":\"datadefqoermgm\"}},{\"nodeName\":\"mmmbipyse\",\"machineName\":\"ybobfjcvmkk\",\"hostServiceUri\":\"gdiwdyyhdtie\",\"status\":\"Upgrading\",\"capabilities\":{\"lbtbxhy\":\"wvcwcehzrtrgp\",\"e\":\"wjfqktuzrlt\"},\"versionStatus\":\"dkbqfzbvttqj\",\"version\":\"vhnjpbhjlug\",\"registerTime\":\"2021-01-18T14:59:38Z\",\"lastConnectTime\":\"2021-06-12T17:39:08Z\",\"expiryTime\":\"2021-02-16T08:11:21Z\",\"lastStartTime\":\"2021-05-12T22:34:43Z\",\"lastStopTime\":\"2021-03-22T08:29:26Z\",\"lastUpdateResult\":\"None\",\"lastStartUpdateTime\":\"2021-04-06T11:32:41Z\",\"lastEndUpdateTime\":\"2021-05-05T20:56:27Z\",\"isActiveDispatcher\":false,\"concurrentJobsLimit\":1100917041,\"maxConcurrentJobs\":982181883,\"\":{\"qu\":\"datawlynxzhgbspdxbh\",\"xqomzdfaupqve\":\"datak\",\"modwhqu\":\"datadzwnkbjqp\"}}],\"scheduledUpdateDate\":\"2021-05-17T05:14:18Z\",\"updateDelayOffset\":\"htux\",\"localTimeZoneOffset\":\"e\",\"capabilities\":{\"bx\":\"q\",\"z\":\"ynslcvynavwtttsm\",\"o\":\"oddcaqi\",\"qtkfjacktav\":\"nbjmjxkhbucmz\"},\"serviceUrls\":[\"godjfyplavb\",\"secedsoqwexi\",\"bnzot\",\"ikf\"],\"autoUpdate\":\"Off\",\"versionStatus\":\"fadgywylavetq\",\"links\":[{\"name\":\"ys\",\"subscriptionId\":\"gjtpbtkogf\",\"dataFactoryName\":\"ylyz\",\"dataFactoryLocation\":\"rvwsgseq\",\"createTime\":\"2021-11-02T03:12:25Z\"},{\"name\":\"axir\",\"subscriptionId\":\"isw\",\"dataFactoryName\":\"y\",\"dataFactoryLocation\":\"vsobamtarirdzdgv\",\"createTime\":\"2021-07-25T07:36:54Z\"},{\"name\":\"zukego\",\"subscriptionId\":\"xpypbmzqs\",\"dataFactoryName\":\"lmnxrxkuly\",\"dataFactoryLocation\":\"vviyqonbxxyf\",\"createTime\":\"2021-03-06T16:14:23Z\"},{\"name\":\"od\",\"subscriptionId\":\"xjikfrxvlusedpn\",\"dataFactoryName\":\"imqaxtmvmycvjpa\",\"dataFactoryLocation\":\"dqvv\",\"createTime\":\"2021-05-23T17:50:07Z\"}],\"pushedVersion\":\"yvn\",\"latestVersion\":\"ngoqmr\",\"autoUpdateETA\":\"2021-09-10T09:59:19Z\",\"selfContainedInteractiveAuthoringEnabled\":false}") + "{\"createTime\":\"2021-08-23T11:38:34Z\",\"taskQueueId\":\"cgiwsywpejtvqop\",\"internalChannelEncryption\":\"NotSet\",\"version\":\"er\",\"nodes\":[{\"nodeName\":\"tqe\",\"machineName\":\"ypul\",\"hostServiceUri\":\"mcbcen\",\"status\":\"NeedRegistration\",\"capabilities\":{\"wc\":\"epi\",\"kqmki\":\"oasg\"},\"versionStatus\":\"bfvkiwmqnwm\",\"version\":\"cct\",\"registerTime\":\"2021-06-23T08:30:31Z\",\"lastConnectTime\":\"2021-12-02T13:41:46Z\",\"expiryTime\":\"2021-11-23T20:55:31Z\",\"lastStartTime\":\"2021-07-18T21:27:37Z\",\"lastStopTime\":\"2021-04-24T22:04:41Z\",\"lastUpdateResult\":\"Succeed\",\"lastStartUpdateTime\":\"2021-11-19T09:35:38Z\",\"lastEndUpdateTime\":\"2021-01-27T13:52:03Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":601104812,\"maxConcurrentJobs\":298117911,\"\":{\"hqnzcbjfpxoyg\":\"datahkoqcudnwmoyhdpj\",\"q\":\"datamjn\"}},{\"nodeName\":\"qxyxfknjqss\",\"machineName\":\"wvravn\",\"hostServiceUri\":\"klkwqisnlpa\",\"status\":\"NeedRegistration\",\"capabilities\":{\"mpgqqdhtct\":\"otktdmewwlkry\",\"zjohdhczh\":\"xregykjmpad\"},\"versionStatus\":\"itydljgrpq\",\"version\":\"fxg\",\"registerTime\":\"2021-03-01T06:28:30Z\",\"lastConnectTime\":\"2021-01-05T20:00:31Z\",\"expiryTime\":\"2021-05-06T21:01:56Z\",\"lastStartTime\":\"2021-09-13T21:29:57Z\",\"lastStopTime\":\"2021-08-09T19:17:55Z\",\"lastUpdateResult\":\"None\",\"lastStartUpdateTime\":\"2021-05-19T23:25:50Z\",\"lastEndUpdateTime\":\"2021-10-05T17:19:03Z\",\"isActiveDispatcher\":true,\"concurrentJobsLimit\":1187362021,\"maxConcurrentJobs\":264432221,\"\":{\"qihvbkufqiqddjy\":\"datagljopiz\",\"mkjpajlfp\":\"datapgom\"}},{\"nodeName\":\"rljlhejcccpxbnwi\",\"machineName\":\"fjjevptv\",\"hostServiceUri\":\"ghzqwvkparskpl\",\"status\":\"InitializeFailed\",\"capabilities\":{\"cykgulellfwrmsux\":\"xqmbsu\"},\"versionStatus\":\"b\",\"version\":\"oubxlpkdsnbqoyms\",\"registerTime\":\"2021-07-23T03:20:47Z\",\"lastConnectTime\":\"2021-11-28T19:08:48Z\",\"expiryTime\":\"2021-03-05T01:15:54Z\",\"lastStartTime\":\"2021-11-08T10:38:08Z\",\"lastStopTime\":\"2021-09-14T21:07:48Z\",\"lastUpdateResult\":\"None\",\"lastStartUpdateTime\":\"2021-11-05T01:02:23Z\",\"lastEndUpdateTime\":\"2021-06-29T18:24:37Z\",\"isActiveDispatcher\":false,\"concurrentJobsLimit\":1884274645,\"maxConcurrentJobs\":1000136471,\"\":{\"ksribmbtmorik\":\"dataskkop\"}}],\"scheduledUpdateDate\":\"2021-05-31T21:34:47Z\",\"updateDelayOffset\":\"qtiqxgwsfrlyvzl\",\"localTimeZoneOffset\":\"ydgmdoqgruc\",\"capabilities\":{\"nrruwsqp\":\"yjudhgwjqegly\",\"mgexqcilfiulgnn\":\"yxkm\",\"wqvvzxgbamqrbb\":\"xwdpmc\",\"mvdpavcjkb\":\"npt\"},\"serviceUrls\":[\"w\",\"izcbqjxmhwdtunpc\",\"kd\",\"rfouwkoltjdauujj\"],\"autoUpdate\":\"Off\",\"versionStatus\":\"d\",\"links\":[{\"name\":\"kudjzr\",\"subscriptionId\":\"ye\",\"dataFactoryName\":\"lm\",\"dataFactoryLocation\":\"oduk\",\"createTime\":\"2021-08-19T15:28:18Z\"},{\"name\":\"nqqkqxkd\",\"subscriptionId\":\"seuqkrbyyakrjgb\",\"dataFactoryName\":\"gokrllbecannvxo\",\"dataFactoryLocation\":\"dcftptfc\",\"createTime\":\"2021-05-09T13:46:34Z\"},{\"name\":\"fwxmcxhwkgihif\",\"subscriptionId\":\"ocmgadhmqyufp\",\"dataFactoryName\":\"wweylszrt\",\"dataFactoryLocation\":\"iqcypmonfcorcn\",\"createTime\":\"2021-04-07T10:45:05Z\"}],\"pushedVersion\":\"igc\",\"latestVersion\":\"ayhdt\",\"autoUpdateETA\":\"2021-02-21T20:09:01Z\",\"selfContainedInteractiveAuthoringEnabled\":true}") .toObject(SelfHostedIntegrationRuntimeStatusTypeProperties.class); } @@ -26,30 +26,41 @@ public void testDeserialize() throws Exception { public void testSerialize() throws Exception { SelfHostedIntegrationRuntimeStatusTypeProperties model = new SelfHostedIntegrationRuntimeStatusTypeProperties() .withNodes(Arrays.asList( - new SelfHostedIntegrationRuntimeNodeInner().withAdditionalProperties(mapOf("nodeName", - "dkpnmwrfuqjdeby", "lastStartUpdateTime", "2021-08-29T14:58:42Z", "lastConnectTime", - "2021-04-27T18:45:48Z", "capabilities", - JacksonAdapter.createDefaultSerializerAdapter().deserialize("{\"xm\":\"f\",\"tsnyr\":\"smgkr\"}", - Object.class, SerializerEncoding.JSON), - "hostServiceUri", "ouzyv", "registerTime", "2021-08-26T04:54:27Z", "maxConcurrentJobs", 415008463, - "lastStopTime", "2021-04-09T22:30:41Z", "version", "aoymyckdpzb", "machineName", "ucnbgibklscw", - "versionStatus", "ulbyzzcxs", "concurrentJobsLimit", 1193471045, "lastEndUpdateTime", - "2021-10-18T07:17:24Z", "expiryTime", "2021-03-06T04:59:11Z", "lastStartTime", - "2020-12-23T10:34:23Z", "lastUpdateResult", "Succeed", "isActiveDispatcher", true, "status", - "Online")), - new SelfHostedIntegrationRuntimeNodeInner().withAdditionalProperties(mapOf("nodeName", "mmmbipyse", - "lastStartUpdateTime", "2021-04-06T11:32:41Z", "lastConnectTime", "2021-06-12T17:39:08Z", + new SelfHostedIntegrationRuntimeNodeInner().withAdditionalProperties(mapOf("nodeName", "tqe", + "lastStartUpdateTime", "2021-11-19T09:35:38Z", "lastConnectTime", "2021-12-02T13:41:46Z", "capabilities", - JacksonAdapter.createDefaultSerializerAdapter().deserialize( - "{\"lbtbxhy\":\"wvcwcehzrtrgp\",\"e\":\"wjfqktuzrlt\"}", Object.class, SerializerEncoding.JSON), - "hostServiceUri", "gdiwdyyhdtie", "registerTime", "2021-01-18T14:59:38Z", "maxConcurrentJobs", - 982181883, "lastStopTime", "2021-03-22T08:29:26Z", "version", "vhnjpbhjlug", "machineName", - "ybobfjcvmkk", "versionStatus", "dkbqfzbvttqj", "concurrentJobsLimit", 1100917041, - "lastEndUpdateTime", "2021-05-05T20:56:27Z", "expiryTime", "2021-02-16T08:11:21Z", "lastStartTime", - "2021-05-12T22:34:43Z", "lastUpdateResult", "None", "isActiveDispatcher", false, "status", - "Upgrading")))) + JacksonAdapter.createDefaultSerializerAdapter().deserialize("{\"wc\":\"epi\",\"kqmki\":\"oasg\"}", + Object.class, SerializerEncoding.JSON), + "hostServiceUri", "mcbcen", "registerTime", "2021-06-23T08:30:31Z", "maxConcurrentJobs", 298117911, + "lastStopTime", "2021-04-24T22:04:41Z", "version", "cct", "machineName", "ypul", "versionStatus", + "bfvkiwmqnwm", "concurrentJobsLimit", 601104812, "lastEndUpdateTime", "2021-01-27T13:52:03Z", + "expiryTime", "2021-11-23T20:55:31Z", "lastStartTime", "2021-07-18T21:27:37Z", "lastUpdateResult", + "Succeed", "isActiveDispatcher", true, "status", "NeedRegistration")), + new SelfHostedIntegrationRuntimeNodeInner() + .withAdditionalProperties(mapOf("nodeName", "qxyxfknjqss", "lastStartUpdateTime", + "2021-05-19T23:25:50Z", "lastConnectTime", "2021-01-05T20:00:31Z", "capabilities", + JacksonAdapter.createDefaultSerializerAdapter().deserialize( + "{\"mpgqqdhtct\":\"otktdmewwlkry\",\"zjohdhczh\":\"xregykjmpad\"}", Object.class, + SerializerEncoding.JSON), + "hostServiceUri", "klkwqisnlpa", "registerTime", "2021-03-01T06:28:30Z", "maxConcurrentJobs", + 264432221, "lastStopTime", "2021-08-09T19:17:55Z", "version", "fxg", "machineName", "wvravn", + "versionStatus", "itydljgrpq", "concurrentJobsLimit", 1187362021, "lastEndUpdateTime", + "2021-10-05T17:19:03Z", "expiryTime", "2021-05-06T21:01:56Z", "lastStartTime", + "2021-09-13T21:29:57Z", "lastUpdateResult", "None", "isActiveDispatcher", true, "status", + "NeedRegistration")), + new SelfHostedIntegrationRuntimeNodeInner() + .withAdditionalProperties(mapOf("nodeName", "rljlhejcccpxbnwi", "lastStartUpdateTime", + "2021-11-05T01:02:23Z", "lastConnectTime", "2021-11-28T19:08:48Z", "capabilities", + JacksonAdapter.createDefaultSerializerAdapter().deserialize("{\"cykgulellfwrmsux\":\"xqmbsu\"}", + Object.class, SerializerEncoding.JSON), + "hostServiceUri", "ghzqwvkparskpl", "registerTime", "2021-07-23T03:20:47Z", "maxConcurrentJobs", + 1000136471, "lastStopTime", "2021-09-14T21:07:48Z", "version", "oubxlpkdsnbqoyms", + "machineName", "fjjevptv", "versionStatus", "b", "concurrentJobsLimit", 1884274645, + "lastEndUpdateTime", "2021-06-29T18:24:37Z", "expiryTime", "2021-03-05T01:15:54Z", + "lastStartTime", "2021-11-08T10:38:08Z", "lastUpdateResult", "None", "isActiveDispatcher", + false, "status", "InitializeFailed")))) .withLinks(Arrays.asList(new LinkedIntegrationRuntime(), new LinkedIntegrationRuntime(), - new LinkedIntegrationRuntime(), new LinkedIntegrationRuntime())); + new LinkedIntegrationRuntime())); model = BinaryData.fromObject(model).toObject(SelfHostedIntegrationRuntimeStatusTypeProperties.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowSourceTests.java index 1e84e866bf8fe..284e370128593 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowSourceTests.java @@ -11,16 +11,16 @@ public final class ServiceNowSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ServiceNowSource model = BinaryData.fromString( - "{\"type\":\"ServiceNowSource\",\"query\":\"dataqmdxcwxfmrmwd\",\"queryTimeout\":\"datapsypmthfv\",\"additionalColumns\":\"datalafcxdldhhkde\",\"sourceRetryCount\":\"datawpzhfxvl\",\"sourceRetryWait\":\"dataurhsmg\",\"maxConcurrentConnections\":\"databtuujcuavctxyrm\",\"disableMetricsCollection\":\"datahrzmy\",\"\":{\"xvotidlwmewrgu\":\"datanwa\"}}") + "{\"type\":\"ServiceNowSource\",\"query\":\"datadtq\",\"queryTimeout\":\"datajbxol\",\"additionalColumns\":\"datahquqihgibog\",\"sourceRetryCount\":\"datajupenoupcolxc\",\"sourceRetryWait\":\"dataszwadesisd\",\"maxConcurrentConnections\":\"datauhqts\",\"disableMetricsCollection\":\"datab\",\"\":{\"bymrgelgoduexx\":\"dataeeucvv\",\"fr\":\"datad\",\"wqzvqtnozwphka\":\"dataenvkqtvtq\",\"bzbbjxkami\":\"dataracvcbrtltpo\"}}") .toObject(ServiceNowSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServiceNowSource model = new ServiceNowSource().withSourceRetryCount("datawpzhfxvl") - .withSourceRetryWait("dataurhsmg").withMaxConcurrentConnections("databtuujcuavctxyrm") - .withDisableMetricsCollection("datahrzmy").withQueryTimeout("datapsypmthfv") - .withAdditionalColumns("datalafcxdldhhkde").withQuery("dataqmdxcwxfmrmwd"); + ServiceNowSource model + = new ServiceNowSource().withSourceRetryCount("datajupenoupcolxc").withSourceRetryWait("dataszwadesisd") + .withMaxConcurrentConnections("datauhqts").withDisableMetricsCollection("datab") + .withQueryTimeout("datajbxol").withAdditionalColumns("datahquqihgibog").withQuery("datadtq"); model = BinaryData.fromObject(model).toObject(ServiceNowSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2ObjectDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2ObjectDatasetTests.java new file mode 100644 index 0000000000000..27defae2a3ca9 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2ObjectDatasetTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.DatasetFolder; +import com.azure.resourcemanager.datafactory.models.LinkedServiceReference; +import com.azure.resourcemanager.datafactory.models.ParameterSpecification; +import com.azure.resourcemanager.datafactory.models.ParameterType; +import com.azure.resourcemanager.datafactory.models.ServiceNowV2ObjectDataset; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class ServiceNowV2ObjectDatasetTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ServiceNowV2ObjectDataset model = BinaryData.fromString( + "{\"type\":\"ServiceNowV2Object\",\"typeProperties\":{\"tableName\":\"dataadzglm\"},\"description\":\"zpsuhsypxmul\",\"structure\":\"datafrerkqp\",\"schema\":\"datajxkbywsbu\",\"linkedServiceName\":{\"referenceName\":\"fmxbdjkm\",\"parameters\":{\"vghbtycvl\":\"dataggnowxhyvdbrdv\",\"xshmrdisc\":\"datausgiikhrcthype\"}},\"parameters\":{\"laf\":{\"type\":\"Int\",\"defaultValue\":\"datamktcwmivoxgzegn\"},\"aghddcozwxuxorru\":{\"type\":\"String\",\"defaultValue\":\"dataa\"},\"gxlssolqypvw\":{\"type\":\"SecureString\",\"defaultValue\":\"datave\"},\"fcm\":{\"type\":\"String\",\"defaultValue\":\"dataohvrkqvrvgdojcv\"}},\"annotations\":[\"datapjakjdtuodoc\"],\"folder\":{\"name\":\"qxnyxpmqdsqnii\"},\"\":{\"oih\":\"dataqikdipkxsqkuzabr\"}}") + .toObject(ServiceNowV2ObjectDataset.class); + Assertions.assertEquals("zpsuhsypxmul", model.description()); + Assertions.assertEquals("fmxbdjkm", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("laf").type()); + Assertions.assertEquals("qxnyxpmqdsqnii", model.folder().name()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ServiceNowV2ObjectDataset model = new ServiceNowV2ObjectDataset().withDescription("zpsuhsypxmul") + .withStructure("datafrerkqp").withSchema("datajxkbywsbu") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("fmxbdjkm") + .withParameters(mapOf("vghbtycvl", "dataggnowxhyvdbrdv", "xshmrdisc", "datausgiikhrcthype"))) + .withParameters(mapOf("laf", + new ParameterSpecification().withType(ParameterType.INT).withDefaultValue("datamktcwmivoxgzegn"), + "aghddcozwxuxorru", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataa"), "gxlssolqypvw", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datave"), "fcm", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataohvrkqvrvgdojcv"))) + .withAnnotations(Arrays.asList("datapjakjdtuodoc")) + .withFolder(new DatasetFolder().withName("qxnyxpmqdsqnii")).withTableName("dataadzglm"); + model = BinaryData.fromObject(model).toObject(ServiceNowV2ObjectDataset.class); + Assertions.assertEquals("zpsuhsypxmul", model.description()); + Assertions.assertEquals("fmxbdjkm", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.INT, model.parameters().get("laf").type()); + Assertions.assertEquals("qxnyxpmqdsqnii", model.folder().name()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2SourceTests.java new file mode 100644 index 0000000000000..63f3837be9b72 --- /dev/null +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ServiceNowV2SourceTests.java @@ -0,0 +1,69 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.datafactory.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.datafactory.models.ExpressionV2; +import com.azure.resourcemanager.datafactory.models.ExpressionV2Type; +import com.azure.resourcemanager.datafactory.models.ServiceNowV2Source; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ServiceNowV2SourceTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ServiceNowV2Source model = BinaryData.fromString( + "{\"type\":\"ServiceNowV2Source\",\"expression\":{\"type\":\"Constant\",\"value\":\"avbteaegyojy\",\"operator\":\"epcdhqjczh\",\"operands\":[{\"type\":\"Constant\",\"value\":\"jbhwoszrhfwcihk\",\"operator\":\"jbitmuriizyrgzxp\",\"operands\":[{\"type\":\"Unary\",\"value\":\"sod\",\"operator\":\"jn\",\"operands\":[{},{},{},{}]},{\"type\":\"Constant\",\"value\":\"hr\",\"operator\":\"i\",\"operands\":[{},{}]},{\"type\":\"Constant\",\"value\":\"huwcqnaspjdahi\",\"operator\":\"kliyfgkzwkyqa\",\"operands\":[{}]}]},{\"type\":\"Unary\",\"value\":\"oyykhidia\",\"operator\":\"ktwijoxkkynppqt\",\"operands\":[{\"type\":\"Unary\",\"value\":\"jat\",\"operator\":\"kqq\",\"operands\":[{},{},{}]}]}]},\"queryTimeout\":\"dataoinqzv\",\"additionalColumns\":\"datafpu\",\"sourceRetryCount\":\"datayhs\",\"sourceRetryWait\":\"dataa\",\"maxConcurrentConnections\":\"dataofqntt\",\"disableMetricsCollection\":\"datajcgupxnuvsh\",\"\":{\"ogvafbd\":\"databpmvppgui\",\"deqntbl\":\"dataokplolcalyvcxvcp\",\"afqr\":\"datadqsqbyubsw\"}}") + .toObject(ServiceNowV2Source.class); + Assertions.assertEquals(ExpressionV2Type.CONSTANT, model.expression().type()); + Assertions.assertEquals("avbteaegyojy", model.expression().value()); + Assertions.assertEquals("epcdhqjczh", model.expression().operator()); + Assertions.assertEquals(ExpressionV2Type.CONSTANT, model.expression().operands().get(0).type()); + Assertions.assertEquals("jbhwoszrhfwcihk", model.expression().operands().get(0).value()); + Assertions.assertEquals("jbitmuriizyrgzxp", model.expression().operands().get(0).operator()); + Assertions.assertEquals(ExpressionV2Type.UNARY, model.expression().operands().get(0).operands().get(0).type()); + Assertions.assertEquals("sod", model.expression().operands().get(0).operands().get(0).value()); + Assertions.assertEquals("jn", model.expression().operands().get(0).operands().get(0).operator()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ServiceNowV2Source model + = new ServiceNowV2Source().withSourceRetryCount("datayhs").withSourceRetryWait("dataa") + .withMaxConcurrentConnections( + "dataofqntt") + .withDisableMetricsCollection( + "datajcgupxnuvsh") + .withQueryTimeout("dataoinqzv").withAdditionalColumns("datafpu") + .withExpression(new ExpressionV2().withType(ExpressionV2Type.CONSTANT).withValue("avbteaegyojy") + .withOperator("epcdhqjczh") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.CONSTANT).withValue("jbhwoszrhfwcihk") + .withOperator("jbitmuriizyrgzxp") + .withOperands(Arrays.asList( + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("sod").withOperator("jn") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2(), + new ExpressionV2(), new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.CONSTANT).withValue("hr").withOperator("i") + .withOperands(Arrays.asList(new ExpressionV2(), new ExpressionV2())), + new ExpressionV2().withType(ExpressionV2Type.CONSTANT).withValue("huwcqnaspjdahi") + .withOperator("kliyfgkzwkyqa").withOperands(Arrays.asList(new ExpressionV2())))), + new ExpressionV2().withType(ExpressionV2Type.UNARY).withValue("oyykhidia") + .withOperator("ktwijoxkkynppqt") + .withOperands(Arrays.asList(new ExpressionV2().withType(ExpressionV2Type.UNARY) + .withValue("jat").withOperator("kqq").withOperands( + Arrays.asList(new ExpressionV2(), new ExpressionV2(), new ExpressionV2()))))))); + model = BinaryData.fromObject(model).toObject(ServiceNowV2Source.class); + Assertions.assertEquals(ExpressionV2Type.CONSTANT, model.expression().type()); + Assertions.assertEquals("avbteaegyojy", model.expression().value()); + Assertions.assertEquals("epcdhqjczh", model.expression().operator()); + Assertions.assertEquals(ExpressionV2Type.CONSTANT, model.expression().operands().get(0).type()); + Assertions.assertEquals("jbhwoszrhfwcihk", model.expression().operands().get(0).value()); + Assertions.assertEquals("jbitmuriizyrgzxp", model.expression().operands().get(0).operator()); + Assertions.assertEquals(ExpressionV2Type.UNARY, model.expression().operands().get(0).operands().get(0).type()); + Assertions.assertEquals("sod", model.expression().operands().get(0).operands().get(0).value()); + Assertions.assertEquals("jn", model.expression().operands().get(0).operands().get(0).operator()); + } +} diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTests.java index f5ac0d03ddbb3..dc867df8301e5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTests.java @@ -21,59 +21,56 @@ public final class SetVariableActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SetVariableActivity model = BinaryData.fromString( - "{\"type\":\"SetVariable\",\"typeProperties\":{\"variableName\":\"qwogpe\",\"value\":\"datamyfg\",\"setSystemVariable\":false},\"policy\":{\"secureInput\":false,\"secureOutput\":false},\"name\":\"utiodkypckhq\",\"description\":\"qniqv\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"kmgcipvrkng\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Failed\"],\"\":{\"gf\":\"datamkgh\",\"tjtiidozfrgvqurr\":\"dataugqhmouekoxylcbp\",\"vohjg\":\"datanijdr\"}},{\"activity\":\"oiikr\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Completed\",\"Skipped\"],\"\":{\"omjajdmtzkqnlz\":\"dataccbzq\",\"twgbfiosdizpg\":\"datatazqsucttp\",\"gwvvenmuenoq\":\"dataqnglzfgepblhe\"}},{\"activity\":\"amrytrny\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Succeeded\",\"Skipped\"],\"\":{\"hgyjwmbjv\":\"datard\",\"olxlxlezzy\":\"datawlxkxlrung\",\"yxaj\":\"datamzzoiudelmxbx\"}},{\"activity\":\"ayc\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Succeeded\"],\"\":{\"dsjbhuzybm\":\"datadnygtsjafv\",\"zuvbnmzjwhybsgz\":\"datay\",\"qmwmwoggbxiasfi\":\"datawyfhkznl\",\"lfedwhvhlzpvpix\":\"dataucnp\"}}],\"userProperties\":[{\"name\":\"y\",\"value\":\"dataodfubnvdibbnvct\"},{\"name\":\"tmhanmp\",\"value\":\"dataxlrvklypauqyais\"},{\"name\":\"iwokgv\",\"value\":\"datasoxykrmalen\"},{\"name\":\"bvahj\",\"value\":\"datavbnlxe\"}],\"\":{\"tgntlfd\":\"datajopsgedsyykueif\",\"zhbhelgwlr\":\"dataqzvazf\",\"cy\":\"datagpudbimehd\"}}") + "{\"type\":\"SetVariable\",\"typeProperties\":{\"variableName\":\"cjawqhpijurqoi\",\"value\":\"dataibjivmvjtkvf\",\"setSystemVariable\":false},\"policy\":{\"secureInput\":true,\"secureOutput\":false},\"name\":\"jmsaacnblu\",\"description\":\"mzgqj\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"xfkxnevciwuiopgy\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Succeeded\"],\"\":{\"hmpmfakinode\":\"dataycgdkikpqmdi\"}},{\"activity\":\"ppcpwcxfn\",\"dependencyConditions\":[\"Failed\"],\"\":{\"xldorqprj\":\"dataxay\",\"ssog\":\"datavueyzg\",\"vir\":\"dataeqv\"}},{\"activity\":\"bguewtcq\",\"dependencyConditions\":[\"Completed\"],\"\":{\"sgauwepojmxbyv\":\"databn\",\"r\":\"dataykwrffx\",\"qlcvtdy\":\"dataxhcxpzjewo\",\"quyatvsn\":\"dataozmtsjgnp\"}},{\"activity\":\"rxhv\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Skipped\",\"Succeeded\"],\"\":{\"wzpauwhfh\":\"datarzlrnuyhldtjzid\",\"lojca\":\"datanh\",\"idqlghrc\":\"databukjywgs\"}}],\"userProperties\":[{\"name\":\"ln\",\"value\":\"datakvdr\"},{\"name\":\"ekxvlejh\",\"value\":\"databqzxqid\"},{\"name\":\"u\",\"value\":\"datawrwjbanteeu\"}],\"\":{\"fvjktfpo\":\"datacai\",\"q\":\"dataalrrqjioltdlppyk\"}}") .toObject(SetVariableActivity.class); - Assertions.assertEquals("utiodkypckhq", model.name()); - Assertions.assertEquals("qniqv", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals("jmsaacnblu", model.name()); + Assertions.assertEquals("mzgqj", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("kmgcipvrkng", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("y", model.userProperties().get(0).name()); - Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals("xfkxnevciwuiopgy", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ln", model.userProperties().get(0).name()); + Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("qwogpe", model.variableName()); + Assertions.assertEquals("cjawqhpijurqoi", model.variableName()); Assertions.assertEquals(false, model.setSystemVariable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SetVariableActivity model = new SetVariableActivity().withName("utiodkypckhq").withDescription("qniqv") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + SetVariableActivity model = new SetVariableActivity().withName("jmsaacnblu").withDescription("mzgqj") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("kmgcipvrkng") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("oiikr") + new ActivityDependency().withActivity("xfkxnevciwuiopgy") .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("ppcpwcxfn") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("amrytrny") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("bguewtcq") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ayc") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("rxhv") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, + DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("y").withValue("dataodfubnvdibbnvct"), - new UserProperty().withName("tmhanmp").withValue("dataxlrvklypauqyais"), - new UserProperty().withName("iwokgv").withValue("datasoxykrmalen"), - new UserProperty().withName("bvahj").withValue("datavbnlxe"))) - .withPolicy(new SecureInputOutputPolicy().withSecureInput(false).withSecureOutput(false)) - .withVariableName("qwogpe").withValue("datamyfg").withSetSystemVariable(false); + .withUserProperties(Arrays.asList(new UserProperty().withName("ln").withValue("datakvdr"), + new UserProperty().withName("ekxvlejh").withValue("databqzxqid"), + new UserProperty().withName("u").withValue("datawrwjbanteeu"))) + .withPolicy(new SecureInputOutputPolicy().withSecureInput(true).withSecureOutput(false)) + .withVariableName("cjawqhpijurqoi").withValue("dataibjivmvjtkvf").withSetSystemVariable(false); model = BinaryData.fromObject(model).toObject(SetVariableActivity.class); - Assertions.assertEquals("utiodkypckhq", model.name()); - Assertions.assertEquals("qniqv", model.description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.state()); + Assertions.assertEquals("jmsaacnblu", model.name()); + Assertions.assertEquals("mzgqj", model.description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); - Assertions.assertEquals("kmgcipvrkng", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("y", model.userProperties().get(0).name()); - Assertions.assertEquals(false, model.policy().secureInput()); + Assertions.assertEquals("xfkxnevciwuiopgy", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ln", model.userProperties().get(0).name()); + Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(false, model.policy().secureOutput()); - Assertions.assertEquals("qwogpe", model.variableName()); + Assertions.assertEquals("cjawqhpijurqoi", model.variableName()); Assertions.assertEquals(false, model.setSystemVariable()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTypePropertiesTests.java index d10b60068ace2..698811f251086 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SetVariableActivityTypePropertiesTests.java @@ -12,18 +12,18 @@ public final class SetVariableActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SetVariableActivityTypeProperties model = BinaryData - .fromString("{\"variableName\":\"fhwkbhapfny\",\"value\":\"dataut\",\"setSystemVariable\":false}") + .fromString("{\"variableName\":\"rvghvfodrqmcgeqy\",\"value\":\"datard\",\"setSystemVariable\":true}") .toObject(SetVariableActivityTypeProperties.class); - Assertions.assertEquals("fhwkbhapfny", model.variableName()); - Assertions.assertEquals(false, model.setSystemVariable()); + Assertions.assertEquals("rvghvfodrqmcgeqy", model.variableName()); + Assertions.assertEquals(true, model.setSystemVariable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SetVariableActivityTypeProperties model = new SetVariableActivityTypeProperties() - .withVariableName("fhwkbhapfny").withValue("dataut").withSetSystemVariable(false); + .withVariableName("rvghvfodrqmcgeqy").withValue("datard").withSetSystemVariable(true); model = BinaryData.fromObject(model).toObject(SetVariableActivityTypeProperties.class); - Assertions.assertEquals("fhwkbhapfny", model.variableName()); - Assertions.assertEquals(false, model.setSystemVariable()); + Assertions.assertEquals("rvghvfodrqmcgeqy", model.variableName()); + Assertions.assertEquals(true, model.setSystemVariable()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpReadSettingsTests.java index 491a400d523dc..afae147a92626 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpReadSettingsTests.java @@ -11,19 +11,18 @@ public final class SftpReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SftpReadSettings model = BinaryData.fromString( - "{\"type\":\"SftpReadSettings\",\"recursive\":\"databci\",\"wildcardFolderPath\":\"dataz\",\"wildcardFileName\":\"dataoaksjwiswzn\",\"enablePartitionDiscovery\":\"databhtleiwfiz\",\"partitionRootPath\":\"datahvq\",\"fileListPath\":\"datahltnds\",\"deleteFilesAfterCompletion\":\"datagnyfhqylia\",\"modifiedDatetimeStart\":\"databhzteninafyaga\",\"modifiedDatetimeEnd\":\"datavetzkcolbclspq\",\"disableChunking\":\"datazicurufng\",\"maxConcurrentConnections\":\"datavdlkpzdkiywwenvx\",\"disableMetricsCollection\":\"datazixranbtqejfqgh\",\"\":{\"iqpjlakamhdqluic\":\"datarvxbcyehajbcbr\",\"ppfgtnsxdj\":\"dataqxqjzmosmlh\",\"hdeml\":\"dataztnjpkpmdlttm\"}}") + "{\"type\":\"SftpReadSettings\",\"recursive\":\"datapbukou\",\"wildcardFolderPath\":\"dataviyfksegwezgfqou\",\"wildcardFileName\":\"datacpdtdzlyog\",\"enablePartitionDiscovery\":\"dataasjckakikkkajm\",\"partitionRootPath\":\"databigmnkrqdyc\",\"fileListPath\":\"databgkxxpklq\",\"deleteFilesAfterCompletion\":\"databcgsa\",\"modifiedDatetimeStart\":\"dataay\",\"modifiedDatetimeEnd\":\"dataeqedft\",\"disableChunking\":\"datagm\",\"maxConcurrentConnections\":\"datakt\",\"disableMetricsCollection\":\"datazyvzixmusiidivbb\",\"\":{\"pucl\":\"datafliqntnoegxo\",\"ygi\":\"datadytwdaiexisa\",\"ivfiypfvwyzjsi\":\"dataoukaffzzf\"}}") .toObject(SftpReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SftpReadSettings model = new SftpReadSettings().withMaxConcurrentConnections("datavdlkpzdkiywwenvx") - .withDisableMetricsCollection("datazixranbtqejfqgh").withRecursive("databci") - .withWildcardFolderPath("dataz").withWildcardFileName("dataoaksjwiswzn") - .withEnablePartitionDiscovery("databhtleiwfiz").withPartitionRootPath("datahvq") - .withFileListPath("datahltnds").withDeleteFilesAfterCompletion("datagnyfhqylia") - .withModifiedDatetimeStart("databhzteninafyaga").withModifiedDatetimeEnd("datavetzkcolbclspq") - .withDisableChunking("datazicurufng"); + SftpReadSettings model = new SftpReadSettings().withMaxConcurrentConnections("datakt") + .withDisableMetricsCollection("datazyvzixmusiidivbb").withRecursive("datapbukou") + .withWildcardFolderPath("dataviyfksegwezgfqou").withWildcardFileName("datacpdtdzlyog") + .withEnablePartitionDiscovery("dataasjckakikkkajm").withPartitionRootPath("databigmnkrqdyc") + .withFileListPath("databgkxxpklq").withDeleteFilesAfterCompletion("databcgsa") + .withModifiedDatetimeStart("dataay").withModifiedDatetimeEnd("dataeqedft").withDisableChunking("datagm"); model = BinaryData.fromObject(model).toObject(SftpReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpWriteSettingsTests.java index 1933b4aba39c1..e287adf3f738a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SftpWriteSettingsTests.java @@ -13,16 +13,19 @@ public final class SftpWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SftpWriteSettings model = BinaryData.fromString( - "{\"type\":\"SftpWriteSettings\",\"operationTimeout\":\"datafvxnisoorwfdtj\",\"useTempFileRename\":\"datajwlpcxljzzcdrgtu\",\"maxConcurrentConnections\":\"dataouoca\",\"disableMetricsCollection\":\"dataxvhjrpbnr\",\"copyBehavior\":\"datagelsgnenjtxuu\",\"metadata\":[{\"name\":\"dataqahlby\",\"value\":\"datahb\"}],\"\":{\"lrl\":\"datalmacbwmvphmjyzic\",\"qgjibrxxiaocr\":\"datazcgwnibnd\",\"pugnvhtgwadu\":\"datauhumgw\"}}") + "{\"type\":\"SftpWriteSettings\",\"operationTimeout\":\"dataj\",\"useTempFileRename\":\"datarorjbyyxkcvahvb\",\"maxConcurrentConnections\":\"dataxtjivwveng\",\"disableMetricsCollection\":\"datayct\",\"copyBehavior\":\"datahgjyholsmahbjc\",\"metadata\":[{\"name\":\"dataqxgbigozrvlkla\",\"value\":\"datalysse\"},{\"name\":\"datappgsfj\",\"value\":\"datagmogmcjn\"},{\"name\":\"dataukbwypcvqfz\",\"value\":\"dataoxgerizflwl\"},{\"name\":\"datavopqpfcdpy\",\"value\":\"datamwpm\"}],\"\":{\"sxfptv\":\"datajj\",\"rdwfhh\":\"datamvrejkvcimq\"}}") .toObject(SftpWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SftpWriteSettings model = new SftpWriteSettings().withMaxConcurrentConnections("dataouoca") - .withDisableMetricsCollection("dataxvhjrpbnr").withCopyBehavior("datagelsgnenjtxuu") - .withMetadata(Arrays.asList(new MetadataItem().withName("dataqahlby").withValue("datahb"))) - .withOperationTimeout("datafvxnisoorwfdtj").withUseTempFileRename("datajwlpcxljzzcdrgtu"); + SftpWriteSettings model = new SftpWriteSettings().withMaxConcurrentConnections("dataxtjivwveng") + .withDisableMetricsCollection("datayct").withCopyBehavior("datahgjyholsmahbjc") + .withMetadata(Arrays.asList(new MetadataItem().withName("dataqxgbigozrvlkla").withValue("datalysse"), + new MetadataItem().withName("datappgsfj").withValue("datagmogmcjn"), + new MetadataItem().withName("dataukbwypcvqfz").withValue("dataoxgerizflwl"), + new MetadataItem().withName("datavopqpfcdpy").withValue("datamwpm"))) + .withOperationTimeout("dataj").withUseTempFileRename("datarorjbyyxkcvahvb"); model = BinaryData.fromObject(model).toObject(SftpWriteSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SharePointOnlineListSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SharePointOnlineListSourceTests.java index 42e79df8a7290..8a94072cd83b2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SharePointOnlineListSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SharePointOnlineListSourceTests.java @@ -11,16 +11,16 @@ public final class SharePointOnlineListSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SharePointOnlineListSource model = BinaryData.fromString( - "{\"type\":\"SharePointOnlineListSource\",\"query\":\"datavbcxnni\",\"httpRequestTimeout\":\"datafuvesmepqrkjyp\",\"sourceRetryCount\":\"datavnotbenfshf\",\"sourceRetryWait\":\"databte\",\"maxConcurrentConnections\":\"datag\",\"disableMetricsCollection\":\"datadtjcyvmsduodpm\",\"\":{\"ocqaejlebcy\":\"dataggn\",\"g\":\"datapqwucprpw\",\"w\":\"datazdkcike\",\"bcto\":\"dataqial\"}}") + "{\"type\":\"SharePointOnlineListSource\",\"query\":\"datacnkojy\",\"httpRequestTimeout\":\"datahbtycfj\",\"sourceRetryCount\":\"dataxiapts\",\"sourceRetryWait\":\"datadoybpwzniekedx\",\"maxConcurrentConnections\":\"dataevip\",\"disableMetricsCollection\":\"datazcxqdrqsuve\",\"\":{\"oxqwcusls\":\"datayb\",\"zwybbewjvyrd\":\"datatzq\",\"bwr\":\"dataw\"}}") .toObject(SharePointOnlineListSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SharePointOnlineListSource model - = new SharePointOnlineListSource().withSourceRetryCount("datavnotbenfshf").withSourceRetryWait("databte") - .withMaxConcurrentConnections("datag").withDisableMetricsCollection("datadtjcyvmsduodpm") - .withQuery("datavbcxnni").withHttpRequestTimeout("datafuvesmepqrkjyp"); + SharePointOnlineListSource model = new SharePointOnlineListSource().withSourceRetryCount("dataxiapts") + .withSourceRetryWait("datadoybpwzniekedx").withMaxConcurrentConnections("dataevip") + .withDisableMetricsCollection("datazcxqdrqsuve").withQuery("datacnkojy") + .withHttpRequestTimeout("datahbtycfj"); model = BinaryData.fromObject(model).toObject(SharePointOnlineListSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ShopifySourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ShopifySourceTests.java index 5a7a70482bc3f..e199d1365bc3f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ShopifySourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ShopifySourceTests.java @@ -11,16 +11,15 @@ public final class ShopifySourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ShopifySource model = BinaryData.fromString( - "{\"type\":\"ShopifySource\",\"query\":\"dataugpkunvygupgnnvm\",\"queryTimeout\":\"datazqmxwwmekms\",\"additionalColumns\":\"datafjbefszfrxfy\",\"sourceRetryCount\":\"dataypxcqmdeecd\",\"sourceRetryWait\":\"datajsizyhp\",\"maxConcurrentConnections\":\"dataykgrtwhmad\",\"disableMetricsCollection\":\"datasmwjkqtfyuygy\",\"\":{\"btkrhc\":\"datachxueaitzgewwqw\",\"pkrrppmwozwji\":\"databzrlfsewusq\",\"vzslttkpirku\":\"dataxyrtec\",\"zhsdpfoabmahuwxo\":\"datacee\"}}") + "{\"type\":\"ShopifySource\",\"query\":\"datav\",\"queryTimeout\":\"datapdv\",\"additionalColumns\":\"datayelrteunkwypu\",\"sourceRetryCount\":\"datafmsygt\",\"sourceRetryWait\":\"dataqlfdml\",\"maxConcurrentConnections\":\"datazdbrw\",\"disableMetricsCollection\":\"datawft\",\"\":{\"jsfgkwrcbgxypr\":\"dataxwi\",\"izabjb\":\"databpywecz\"}}") .toObject(ShopifySource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ShopifySource model = new ShopifySource().withSourceRetryCount("dataypxcqmdeecd") - .withSourceRetryWait("datajsizyhp").withMaxConcurrentConnections("dataykgrtwhmad") - .withDisableMetricsCollection("datasmwjkqtfyuygy").withQueryTimeout("datazqmxwwmekms") - .withAdditionalColumns("datafjbefszfrxfy").withQuery("dataugpkunvygupgnnvm"); + ShopifySource model = new ShopifySource().withSourceRetryCount("datafmsygt").withSourceRetryWait("dataqlfdml") + .withMaxConcurrentConnections("datazdbrw").withDisableMetricsCollection("datawft") + .withQueryTimeout("datapdv").withAdditionalColumns("datayelrteunkwypu").withQuery("datav"); model = BinaryData.fromObject(model).toObject(ShopifySource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SkipErrorFileTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SkipErrorFileTests.java index ac540c12c8fcf..9b8b5a1f25628 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SkipErrorFileTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SkipErrorFileTests.java @@ -11,13 +11,14 @@ public final class SkipErrorFileTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SkipErrorFile model - = BinaryData.fromString("{\"fileMissing\":\"datas\",\"dataInconsistency\":\"dataksltunrwxsqvx\"}") + = BinaryData.fromString("{\"fileMissing\":\"dataalh\",\"dataInconsistency\":\"dataneclphmjsqcubyjr\"}") .toObject(SkipErrorFile.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SkipErrorFile model = new SkipErrorFile().withFileMissing("datas").withDataInconsistency("dataksltunrwxsqvx"); + SkipErrorFile model + = new SkipErrorFile().withFileMissing("dataalh").withDataInconsistency("dataneclphmjsqcubyjr"); model = BinaryData.fromObject(model).toObject(SkipErrorFile.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeExportCopyCommandTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeExportCopyCommandTests.java index 797478994fca1..9037e93320e6c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeExportCopyCommandTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeExportCopyCommandTests.java @@ -13,16 +13,15 @@ public final class SnowflakeExportCopyCommandTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnowflakeExportCopyCommand model = BinaryData.fromString( - "{\"type\":\"SnowflakeExportCopyCommand\",\"additionalCopyOptions\":{\"xoyllx\":\"dataahlqmcboxgpmmz\"},\"additionalFormatOptions\":{\"yrhynl\":\"datahzylspz\",\"yhqrasxeomjq\":\"datatriaecvagudrhad\",\"bhzdjvdyrzijggb\":\"datahbkxiuhia\",\"icrutyhmf\":\"datadpzgvqfznxzal\"},\"\":{\"ghqdlj\":\"datailaytjywfqzwn\"}}") + "{\"type\":\"SnowflakeExportCopyCommand\",\"additionalCopyOptions\":{\"eas\":\"datakmgp\",\"wdosfgbvsoz\":\"datadqpwhp\",\"tlb\":\"datafnpwxcjci\",\"oyl\":\"datauemqetmotuvhhed\"},\"additionalFormatOptions\":{\"wcca\":\"datal\"},\"\":{\"lxqlzzkbx\":\"databdvsorvhbygw\",\"lnkkghlexvqhbn\":\"datancggamxbtqizydai\",\"ltbpqjfoujeiagny\":\"datamokzx\",\"jssay\":\"datae\"}}") .toObject(SnowflakeExportCopyCommand.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnowflakeExportCopyCommand model - = new SnowflakeExportCopyCommand().withAdditionalCopyOptions(mapOf("xoyllx", "dataahlqmcboxgpmmz")) - .withAdditionalFormatOptions(mapOf("yrhynl", "datahzylspz", "yhqrasxeomjq", "datatriaecvagudrhad", - "bhzdjvdyrzijggb", "datahbkxiuhia", "icrutyhmf", "datadpzgvqfznxzal")); + SnowflakeExportCopyCommand model = new SnowflakeExportCopyCommand().withAdditionalCopyOptions( + mapOf("eas", "datakmgp", "wdosfgbvsoz", "datadqpwhp", "tlb", "datafnpwxcjci", "oyl", "datauemqetmotuvhhed")) + .withAdditionalFormatOptions(mapOf("wcca", "datal")); model = BinaryData.fromObject(model).toObject(SnowflakeExportCopyCommand.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeImportCopyCommandTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeImportCopyCommandTests.java index 492d8f709e39c..7be88e45574e7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeImportCopyCommandTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeImportCopyCommandTests.java @@ -13,17 +13,16 @@ public final class SnowflakeImportCopyCommandTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnowflakeImportCopyCommand model = BinaryData.fromString( - "{\"type\":\"SnowflakeImportCopyCommand\",\"additionalCopyOptions\":{\"amoyxdigk\":\"datayxmpmsacbamtoqs\",\"ylqhqeosxdsxil\":\"datagz\",\"ttyh\":\"datafiottdawgkaohh\",\"kxoicbxsmfvltbo\":\"datapidzjjjfcyskpn\"},\"additionalFormatOptions\":{\"g\":\"datavtpmvppvgr\",\"lgkoqbzrcla\":\"dataeg\",\"moramsh\":\"datartfmfkuvyb\",\"zvzqhvzjdsn\":\"datavkuq\"},\"\":{\"ani\":\"datab\",\"aq\":\"dataozjrc\",\"almzpfylqevwwvz\":\"datag\",\"gjl\":\"datapdxcizrop\"}}") + "{\"type\":\"SnowflakeImportCopyCommand\",\"additionalCopyOptions\":{\"lvktjbmce\":\"databe\",\"ukdawgz\":\"datal\",\"vygx\":\"databwhjvonu\"},\"additionalFormatOptions\":{\"jcjvdajxebm\":\"datatjoxocothsg\",\"nd\":\"dataiyrctfaabkukra\",\"dhjdwfnbiyxqr\":\"datahwdicntqsrhacjsb\"},\"\":{\"yovljtrmlxr\":\"dataffk\",\"wbqukjithxlpvp\":\"datallugnxmbwdkzi\",\"shhcktbfmtbprt\":\"datavceiwc\"}}") .toObject(SnowflakeImportCopyCommand.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SnowflakeImportCopyCommand model = new SnowflakeImportCopyCommand() - .withAdditionalCopyOptions(mapOf("amoyxdigk", "datayxmpmsacbamtoqs", "ylqhqeosxdsxil", "datagz", "ttyh", - "datafiottdawgkaohh", "kxoicbxsmfvltbo", "datapidzjjjfcyskpn")) - .withAdditionalFormatOptions(mapOf("g", "datavtpmvppvgr", "lgkoqbzrcla", "dataeg", "moramsh", - "datartfmfkuvyb", "zvzqhvzjdsn", "datavkuq")); + .withAdditionalCopyOptions(mapOf("lvktjbmce", "databe", "ukdawgz", "datal", "vygx", "databwhjvonu")) + .withAdditionalFormatOptions(mapOf("jcjvdajxebm", "datatjoxocothsg", "nd", "dataiyrctfaabkukra", + "dhjdwfnbiyxqr", "datahwdicntqsrhacjsb")); model = BinaryData.fromObject(model).toObject(SnowflakeImportCopyCommand.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSinkTests.java index 72ac34d75f87f..3ee66d7cf7ed2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSinkTests.java @@ -14,23 +14,21 @@ public final class SnowflakeSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnowflakeSink model = BinaryData.fromString( - "{\"type\":\"SnowflakeSink\",\"preCopyScript\":\"datamuj\",\"importSettings\":{\"type\":\"SnowflakeImportCopyCommand\",\"additionalCopyOptions\":{\"mumuc\":\"datahepfjdiwzg\",\"xgelnjgftqkgavgo\":\"datavqwhscvaq\",\"kskkziebm\":\"datallxpaylkrast\",\"gdxdtxbr\":\"datayodfmp\"},\"additionalFormatOptions\":{\"pf\":\"datawqt\",\"kdoukqsc\":\"datafrfvhbbnoevkkr\",\"lgu\":\"datadsjgows\"},\"\":{\"opqy\":\"dataqlhhmbyfacex\"}},\"writeBatchSize\":\"datacesqpvmoxil\",\"writeBatchTimeout\":\"datakqiqsriubem\",\"sinkRetryCount\":\"datauygmrenrbn\",\"sinkRetryWait\":\"dataafmophtky\",\"maxConcurrentConnections\":\"datagayngmowvcnvfgqx\",\"disableMetricsCollection\":\"dataysuapdns\",\"\":{\"ycjdnio\":\"dataqxr\"}}") + "{\"type\":\"SnowflakeSink\",\"preCopyScript\":\"datastofdedlmfwabf\",\"importSettings\":{\"type\":\"SnowflakeImportCopyCommand\",\"additionalCopyOptions\":{\"lwh\":\"databiwxmcsxidaz\",\"fperheiplzms\":\"datayikhdcilinbuok\",\"u\":\"datahqrdvqvalo\",\"fjgklmyomav\":\"datawoigofumbpmzed\"},\"additionalFormatOptions\":{\"tqvjn\":\"dataaoaixipcwyinfy\",\"yaeyyiwraowpdm\":\"dataemqwut\",\"dmwcxvcron\":\"dataxugd\",\"z\":\"dataidhwoyznjddsh\"},\"\":{\"puotniqzqmpgvyy\":\"datavxv\",\"w\":\"datajwwbrhjhcwcfftsz\"}},\"writeBatchSize\":\"dataighjgvhgyeoikxjp\",\"writeBatchTimeout\":\"dataggpsaqfn\",\"sinkRetryCount\":\"datauwawmbusleg\",\"sinkRetryWait\":\"dataqzkzworuhhvdeodc\",\"maxConcurrentConnections\":\"datahfxb\",\"disableMetricsCollection\":\"datapn\",\"\":{\"lhwutyuvuzqtrf\":\"dataylgpznb\"}}") .toObject(SnowflakeSink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnowflakeSink model - = new SnowflakeSink().withWriteBatchSize("datacesqpvmoxil").withWriteBatchTimeout("datakqiqsriubem") - .withSinkRetryCount("datauygmrenrbn").withSinkRetryWait("dataafmophtky") - .withMaxConcurrentConnections("datagayngmowvcnvfgqx").withDisableMetricsCollection("dataysuapdns") - .withPreCopyScript( - "datamuj") - .withImportSettings(new SnowflakeImportCopyCommand() - .withAdditionalCopyOptions(mapOf("mumuc", "datahepfjdiwzg", "xgelnjgftqkgavgo", "datavqwhscvaq", - "kskkziebm", "datallxpaylkrast", "gdxdtxbr", "datayodfmp")) - .withAdditionalFormatOptions( - mapOf("pf", "datawqt", "kdoukqsc", "datafrfvhbbnoevkkr", "lgu", "datadsjgows"))); + SnowflakeSink model = new SnowflakeSink().withWriteBatchSize("dataighjgvhgyeoikxjp") + .withWriteBatchTimeout("dataggpsaqfn").withSinkRetryCount("datauwawmbusleg") + .withSinkRetryWait("dataqzkzworuhhvdeodc").withMaxConcurrentConnections("datahfxb") + .withDisableMetricsCollection("datapn").withPreCopyScript("datastofdedlmfwabf") + .withImportSettings(new SnowflakeImportCopyCommand() + .withAdditionalCopyOptions(mapOf("lwh", "databiwxmcsxidaz", "fperheiplzms", "datayikhdcilinbuok", "u", + "datahqrdvqvalo", "fjgklmyomav", "datawoigofumbpmzed")) + .withAdditionalFormatOptions(mapOf("tqvjn", "dataaoaixipcwyinfy", "yaeyyiwraowpdm", "dataemqwut", + "dmwcxvcron", "dataxugd", "z", "dataidhwoyznjddsh"))); model = BinaryData.fromObject(model).toObject(SnowflakeSink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSourceTests.java index b5424d9805349..04ba1162faa29 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeSourceTests.java @@ -14,18 +14,21 @@ public final class SnowflakeSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnowflakeSource model = BinaryData.fromString( - "{\"type\":\"SnowflakeSource\",\"query\":\"datafqntak\",\"exportSettings\":{\"type\":\"SnowflakeExportCopyCommand\",\"additionalCopyOptions\":{\"fsuwcmzpwkca\":\"dataku\",\"zq\":\"datafq\"},\"additionalFormatOptions\":{\"blonlhtgexwjhicu\":\"datajprdp\",\"ogtvlrbgd\":\"dataqavimxnhyl\"},\"\":{\"k\":\"datanhdxlfntdc\"}},\"sourceRetryCount\":\"datag\",\"sourceRetryWait\":\"datalfyxaiaf\",\"maxConcurrentConnections\":\"datamxekfvycvhwduo\",\"disableMetricsCollection\":\"dataapzzcxk\",\"\":{\"bdnz\":\"databahcassqe\",\"rkohfqm\":\"datayak\",\"arboxaluoadmcv\":\"datazkicxtumqinawct\",\"xwevl\":\"datakjnpe\"}}") + "{\"type\":\"SnowflakeSource\",\"query\":\"datagpcrvv\",\"exportSettings\":{\"type\":\"SnowflakeExportCopyCommand\",\"additionalCopyOptions\":{\"bwpai\":\"datalckpznove\",\"mrpdxnr\":\"dataikzysdharswh\",\"dfplk\":\"datavtvtyqlthn\",\"ltsnqjcmkpxb\":\"datadiehrajbatgmxk\"},\"additionalFormatOptions\":{\"qmyowddhtwaxob\":\"datarfkwc\",\"vsclwpsteuvjdnh\":\"datazatqocvrdj\"},\"\":{\"bfomo\":\"datavymvnlaehitxo\",\"rhhbvbqxtktkeuap\":\"datacyn\",\"bhptraljcq\":\"datamoofb\"}},\"sourceRetryCount\":\"datapmathiydmkyvsx\",\"sourceRetryWait\":\"dataivghajpddgfozn\",\"maxConcurrentConnections\":\"datamkpjoesozcuhunm\",\"disableMetricsCollection\":\"databmwptdrrruy\",\"\":{\"ckxhmxzsmpoiuta\":\"dataiumuxnabivg\",\"vo\":\"datatvpbiojncgjog\"}}") .toObject(SnowflakeSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnowflakeSource model = new SnowflakeSource().withSourceRetryCount("datag").withSourceRetryWait("datalfyxaiaf") - .withMaxConcurrentConnections("datamxekfvycvhwduo").withDisableMetricsCollection("dataapzzcxk") - .withQuery("datafqntak") - .withExportSettings(new SnowflakeExportCopyCommand() - .withAdditionalCopyOptions(mapOf("fsuwcmzpwkca", "dataku", "zq", "datafq")) - .withAdditionalFormatOptions(mapOf("blonlhtgexwjhicu", "datajprdp", "ogtvlrbgd", "dataqavimxnhyl"))); + SnowflakeSource model + = new SnowflakeSource().withSourceRetryCount("datapmathiydmkyvsx").withSourceRetryWait("dataivghajpddgfozn") + .withMaxConcurrentConnections("datamkpjoesozcuhunm").withDisableMetricsCollection("databmwptdrrruy") + .withQuery("datagpcrvv").withExportSettings( + new SnowflakeExportCopyCommand() + .withAdditionalCopyOptions(mapOf("bwpai", "datalckpznove", "mrpdxnr", "dataikzysdharswh", + "dfplk", "datavtvtyqlthn", "ltsnqjcmkpxb", "datadiehrajbatgmxk")) + .withAdditionalFormatOptions( + mapOf("qmyowddhtwaxob", "datarfkwc", "vsclwpsteuvjdnh", "datazatqocvrdj"))); model = BinaryData.fromObject(model).toObject(SnowflakeSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SinkTests.java index 5ae37e511eb45..56d3797d5d127 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SinkTests.java @@ -14,19 +14,18 @@ public final class SnowflakeV2SinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnowflakeV2Sink model = BinaryData.fromString( - "{\"type\":\"SnowflakeV2Sink\",\"preCopyScript\":\"datacf\",\"importSettings\":{\"type\":\"SnowflakeImportCopyCommand\",\"additionalCopyOptions\":{\"nstqwnpeg\":\"datakvb\"},\"additionalFormatOptions\":{\"beekzyebpatwbbf\":\"datadqeflvdfaqcqlex\",\"nwohlcahhfuydgd\":\"datadfl\",\"bpduzeebde\":\"dataitavgayuspzlcv\"},\"\":{\"gbbozivfoyq\":\"datawkhruzz\",\"vvscbpkmo\":\"datanlhsxeasxsqq\",\"qyibwu\":\"datadukp\"}},\"writeBatchSize\":\"datamorsyirfkxazuboi\",\"writeBatchTimeout\":\"datarw\",\"sinkRetryCount\":\"databjzdvaqoilgkz\",\"sinkRetryWait\":\"datazpvjwego\",\"maxConcurrentConnections\":\"dataceqyrajdvvs\",\"disableMetricsCollection\":\"datatyypercazcchvww\",\"\":{\"tfhhay\":\"datazztv\",\"liguwqoszcm\":\"dataxkfgxxef\"}}") + "{\"type\":\"SnowflakeV2Sink\",\"preCopyScript\":\"datauxwuepjcug\",\"importSettings\":{\"type\":\"SnowflakeImportCopyCommand\",\"additionalCopyOptions\":{\"hqkie\":\"dataqbwgmznvlwcn\"},\"additionalFormatOptions\":{\"xetxzcjrb\":\"dataubwz\",\"kbeoofox\":\"dataqcwnbx\"},\"\":{\"prjcfyvkkgxil\":\"dataunewkssxpnhhl\",\"kgwdng\":\"datalonzifdfs\",\"xdvrajoghgxgzb\":\"datadnrtydhqkariatxh\"}},\"writeBatchSize\":\"dataasgam\",\"writeBatchTimeout\":\"dataetxnsgcwad\",\"sinkRetryCount\":\"dataqbageltffqalmcq\",\"sinkRetryWait\":\"datapzwwtcwbgmx\",\"maxConcurrentConnections\":\"dataynslcty\",\"disableMetricsCollection\":\"datai\",\"\":{\"tarxtdgpmhzll\":\"datamhhqw\",\"zsdnfp\":\"datah\",\"ycyfubgnml\":\"datakyks\",\"qvcutwngfd\":\"datajdaxiohulvp\"}}") .toObject(SnowflakeV2Sink.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnowflakeV2Sink model = new SnowflakeV2Sink().withWriteBatchSize("datamorsyirfkxazuboi") - .withWriteBatchTimeout("datarw").withSinkRetryCount("databjzdvaqoilgkz").withSinkRetryWait("datazpvjwego") - .withMaxConcurrentConnections("dataceqyrajdvvs").withDisableMetricsCollection("datatyypercazcchvww") - .withPreCopyScript("datacf").withImportSettings( - new SnowflakeImportCopyCommand().withAdditionalCopyOptions(mapOf("nstqwnpeg", "datakvb")) - .withAdditionalFormatOptions(mapOf("beekzyebpatwbbf", "datadqeflvdfaqcqlex", "nwohlcahhfuydgd", - "datadfl", "bpduzeebde", "dataitavgayuspzlcv"))); + SnowflakeV2Sink model = new SnowflakeV2Sink().withWriteBatchSize("dataasgam") + .withWriteBatchTimeout("dataetxnsgcwad").withSinkRetryCount("dataqbageltffqalmcq") + .withSinkRetryWait("datapzwwtcwbgmx").withMaxConcurrentConnections("dataynslcty") + .withDisableMetricsCollection("datai").withPreCopyScript("datauxwuepjcug").withImportSettings( + new SnowflakeImportCopyCommand().withAdditionalCopyOptions(mapOf("hqkie", "dataqbwgmznvlwcn")) + .withAdditionalFormatOptions(mapOf("xetxzcjrb", "dataubwz", "kbeoofox", "dataqcwnbx"))); model = BinaryData.fromObject(model).toObject(SnowflakeV2Sink.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SourceTests.java index bc69d92031c5a..42ff8f322dc71 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SnowflakeV2SourceTests.java @@ -14,18 +14,19 @@ public final class SnowflakeV2SourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnowflakeV2Source model = BinaryData.fromString( - "{\"type\":\"SnowflakeV2Source\",\"query\":\"datatetzvqtcesvcsby\",\"exportSettings\":{\"type\":\"SnowflakeExportCopyCommand\",\"additionalCopyOptions\":{\"hm\":\"datagswduzpdnbzqwe\"},\"additionalFormatOptions\":{\"rqtm\":\"datahxadmauanx\",\"sh\":\"dataaerhx\",\"b\":\"databrkhjjbwelicr\"},\"\":{\"kddxljjzdbzk\":\"dataoguypmselwszqve\",\"nkiygpqxqevtfdv\":\"datawgwvkujshc\",\"gsa\":\"dataypjkjirvj\"}},\"sourceRetryCount\":\"datajlhn\",\"sourceRetryWait\":\"datauaeokympwquutl\",\"maxConcurrentConnections\":\"datafuzthszjy\",\"disableMetricsCollection\":\"datahsliqeft\",\"\":{\"dpufqsdt\":\"dataro\",\"xolmehquqi\":\"datazxj\",\"ojupenoupc\":\"datagibog\",\"esisdkcuhq\":\"datalxcwrszwa\"}}") + "{\"type\":\"SnowflakeV2Source\",\"query\":\"datav\",\"exportSettings\":{\"type\":\"SnowflakeExportCopyCommand\",\"additionalCopyOptions\":{\"qlshg\":\"dataexwhoscinpmvcvnm\",\"elazvcfhiayro\":\"dataym\"},\"additionalFormatOptions\":{\"wzttkhyrwdsnpuoa\":\"dataezlqwbgly\",\"pizybpjypqatkzgh\":\"datar\",\"wrbmxw\":\"datac\"},\"\":{\"wffjiezbmhsqyj\":\"datakf\",\"ntghjmmjmmjnxh\":\"datawbzhafcoayuq\"}},\"sourceRetryCount\":\"datajtsemnidbaykvlrs\",\"sourceRetryWait\":\"dataniocyo\",\"maxConcurrentConnections\":\"dataimbchi\",\"disableMetricsCollection\":\"datawaffsjqn\",\"\":{\"vaagazryyjjwggp\":\"dataybugojzcargsxmaw\",\"csjw\":\"datadugwddob\",\"hzugwkqnmh\":\"dataxedzmmcgqifh\"}}") .toObject(SnowflakeV2Source.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnowflakeV2Source model = new SnowflakeV2Source().withSourceRetryCount("datajlhn") - .withSourceRetryWait("datauaeokympwquutl").withMaxConcurrentConnections("datafuzthszjy") - .withDisableMetricsCollection("datahsliqeft").withQuery("datatetzvqtcesvcsby") + SnowflakeV2Source model = new SnowflakeV2Source().withSourceRetryCount("datajtsemnidbaykvlrs") + .withSourceRetryWait("dataniocyo").withMaxConcurrentConnections("dataimbchi") + .withDisableMetricsCollection("datawaffsjqn").withQuery("datav") .withExportSettings(new SnowflakeExportCopyCommand() - .withAdditionalCopyOptions(mapOf("hm", "datagswduzpdnbzqwe")).withAdditionalFormatOptions( - mapOf("rqtm", "datahxadmauanx", "sh", "dataaerhx", "b", "databrkhjjbwelicr"))); + .withAdditionalCopyOptions(mapOf("qlshg", "dataexwhoscinpmvcvnm", "elazvcfhiayro", "dataym")) + .withAdditionalFormatOptions( + mapOf("wzttkhyrwdsnpuoa", "dataezlqwbgly", "pizybpjypqatkzgh", "datar", "wrbmxw", "datac"))); model = BinaryData.fromObject(model).toObject(SnowflakeV2Source.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkConfigurationParametrizationReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkConfigurationParametrizationReferenceTests.java index d9ff1205980af..9703d45fa93ed 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkConfigurationParametrizationReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkConfigurationParametrizationReferenceTests.java @@ -13,7 +13,7 @@ public final class SparkConfigurationParametrizationReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SparkConfigurationParametrizationReference model - = BinaryData.fromString("{\"type\":\"SparkConfigurationReference\",\"referenceName\":\"datalvea\"}") + = BinaryData.fromString("{\"type\":\"SparkConfigurationReference\",\"referenceName\":\"datacry\"}") .toObject(SparkConfigurationParametrizationReference.class); Assertions.assertEquals(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE, model.type()); } @@ -21,7 +21,7 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SparkConfigurationParametrizationReference model = new SparkConfigurationParametrizationReference() - .withType(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE).withReferenceName("datalvea"); + .withType(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE).withReferenceName("datacry"); model = BinaryData.fromObject(model).toObject(SparkConfigurationParametrizationReference.class); Assertions.assertEquals(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE, model.type()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkSourceTests.java index b9c6799c443c5..9f466b1ce7e23 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SparkSourceTests.java @@ -11,15 +11,15 @@ public final class SparkSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SparkSource model = BinaryData.fromString( - "{\"type\":\"SparkSource\",\"query\":\"datadqzewr\",\"queryTimeout\":\"datatgsocqkdclbzqnao\",\"additionalColumns\":\"datagmpdcbheza\",\"sourceRetryCount\":\"datauaysxh\",\"sourceRetryWait\":\"datapvq\",\"maxConcurrentConnections\":\"dataqlafi\",\"disableMetricsCollection\":\"dataw\",\"\":{\"mgaifgy\":\"datapuyefhhd\",\"hxpcxqc\":\"datakgqwmp\",\"dhx\":\"datankxhc\"}}") + "{\"type\":\"SparkSource\",\"query\":\"datayddijfkktigisee\",\"queryTimeout\":\"datazrerxyds\",\"additionalColumns\":\"datapn\",\"sourceRetryCount\":\"dataarkjt\",\"sourceRetryWait\":\"dataaczkjkfakgrwt\",\"maxConcurrentConnections\":\"datasfanmjmpce\",\"disableMetricsCollection\":\"datamfdylvpyhhgqysz\",\"\":{\"jekolnylpyk\":\"datajzhvej\",\"aouyaanfxai\":\"datapa\"}}") .toObject(SparkSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SparkSource model = new SparkSource().withSourceRetryCount("datauaysxh").withSourceRetryWait("datapvq") - .withMaxConcurrentConnections("dataqlafi").withDisableMetricsCollection("dataw") - .withQueryTimeout("datatgsocqkdclbzqnao").withAdditionalColumns("datagmpdcbheza").withQuery("datadqzewr"); + SparkSource model = new SparkSource().withSourceRetryCount("dataarkjt").withSourceRetryWait("dataaczkjkfakgrwt") + .withMaxConcurrentConnections("datasfanmjmpce").withDisableMetricsCollection("datamfdylvpyhhgqysz") + .withQueryTimeout("datazrerxyds").withAdditionalColumns("datapn").withQuery("datayddijfkktigisee"); model = BinaryData.fromObject(model).toObject(SparkSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlDWSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlDWSourceTests.java index 2b956ba98a27c..1900a67d55043 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlDWSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlDWSourceTests.java @@ -12,19 +12,19 @@ public final class SqlDWSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlDWSource model = BinaryData.fromString( - "{\"type\":\"SqlDWSource\",\"sqlReaderQuery\":\"datagabsfjrjzdq\",\"sqlReaderStoredProcedureName\":\"datagorvgdibepg\",\"storedProcedureParameters\":\"databijoehhqwwsgqzi\",\"isolationLevel\":\"dataoetwjssyazm\",\"partitionOption\":\"dataux\",\"partitionSettings\":{\"partitionColumnName\":\"datakckxfkf\",\"partitionUpperBound\":\"datakbyruheawucmqfu\",\"partitionLowerBound\":\"datatbogxlyveb\"},\"queryTimeout\":\"datajguwts\",\"additionalColumns\":\"datajwiz\",\"sourceRetryCount\":\"dataifz\",\"sourceRetryWait\":\"dataxtykjrdxlximvr\",\"maxConcurrentConnections\":\"datajja\",\"disableMetricsCollection\":\"dataaskullvtsauj\",\"\":{\"pfhzxkjygkuidgwd\":\"datatzidzq\"}}") + "{\"type\":\"SqlDWSource\",\"sqlReaderQuery\":\"dataahijbjjlx\",\"sqlReaderStoredProcedureName\":\"datajzbggsnanojty\",\"storedProcedureParameters\":\"datahzxzazofr\",\"isolationLevel\":\"datasxjdgaimk\",\"partitionOption\":\"datasowszb\",\"partitionSettings\":{\"partitionColumnName\":\"datahxikrgokyngarwz\",\"partitionUpperBound\":\"datazjxgassmna\",\"partitionLowerBound\":\"datapolueylqysgmiix\"},\"queryTimeout\":\"dataekcwec\",\"additionalColumns\":\"datatkdginm\",\"sourceRetryCount\":\"datagp\",\"sourceRetryWait\":\"dataqqcceyowrwvbqv\",\"maxConcurrentConnections\":\"dataqgqrsopqgiqf\",\"disableMetricsCollection\":\"datatl\",\"\":{\"hfaabibvslo\":\"datazcgugslpvyktf\",\"jzashhiztfmibwzu\":\"datadkpvvkqlkhdxn\",\"wtbfxxsfjn\":\"dataydajck\"}}") .toObject(SqlDWSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SqlDWSource model = new SqlDWSource().withSourceRetryCount("dataifz").withSourceRetryWait("dataxtykjrdxlximvr") - .withMaxConcurrentConnections("datajja").withDisableMetricsCollection("dataaskullvtsauj") - .withQueryTimeout("datajguwts").withAdditionalColumns("datajwiz").withSqlReaderQuery("datagabsfjrjzdq") - .withSqlReaderStoredProcedureName("datagorvgdibepg").withStoredProcedureParameters("databijoehhqwwsgqzi") - .withIsolationLevel("dataoetwjssyazm").withPartitionOption("dataux") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datakckxfkf") - .withPartitionUpperBound("datakbyruheawucmqfu").withPartitionLowerBound("datatbogxlyveb")); + SqlDWSource model = new SqlDWSource().withSourceRetryCount("datagp").withSourceRetryWait("dataqqcceyowrwvbqv") + .withMaxConcurrentConnections("dataqgqrsopqgiqf").withDisableMetricsCollection("datatl") + .withQueryTimeout("dataekcwec").withAdditionalColumns("datatkdginm").withSqlReaderQuery("dataahijbjjlx") + .withSqlReaderStoredProcedureName("datajzbggsnanojty").withStoredProcedureParameters("datahzxzazofr") + .withIsolationLevel("datasxjdgaimk").withPartitionOption("datasowszb") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datahxikrgokyngarwz") + .withPartitionUpperBound("datazjxgassmna").withPartitionLowerBound("datapolueylqysgmiix")); model = BinaryData.fromObject(model).toObject(SqlDWSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlMISourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlMISourceTests.java index 22069a538b66c..048758c511a80 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlMISourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlMISourceTests.java @@ -12,20 +12,20 @@ public final class SqlMISourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlMISource model = BinaryData.fromString( - "{\"type\":\"SqlMISource\",\"sqlReaderQuery\":\"datad\",\"sqlReaderStoredProcedureName\":\"datalyhx\",\"storedProcedureParameters\":\"dataqoe\",\"isolationLevel\":\"databoqozxnuxamxikh\",\"produceAdditionalTypes\":\"dataikglynbqpeojecb\",\"partitionOption\":\"datagw\",\"partitionSettings\":{\"partitionColumnName\":\"datatnywgtsodnxeirjt\",\"partitionUpperBound\":\"dataimcfrhtzgd\",\"partitionLowerBound\":\"datao\"},\"queryTimeout\":\"dataqo\",\"additionalColumns\":\"datalp\",\"sourceRetryCount\":\"datapbzyqbggxcyra\",\"sourceRetryWait\":\"datazuaxtbr\",\"maxConcurrentConnections\":\"datayurxlpuwxslzq\",\"disableMetricsCollection\":\"datax\",\"\":{\"bm\":\"datarurtnwbjjysupckh\",\"aoofltb\":\"dataemohlshm\",\"ddcftnxyrt\":\"dataayvmwaejxzkqc\"}}") + "{\"type\":\"SqlMISource\",\"sqlReaderQuery\":\"dataxhfttm\",\"sqlReaderStoredProcedureName\":\"datamuwl\",\"storedProcedureParameters\":\"datajwkpznsfbi\",\"isolationLevel\":\"datafzgpvdlx\",\"produceAdditionalTypes\":\"dataotclcuxzllnwmgqc\",\"partitionOption\":\"datagjequox\",\"partitionSettings\":{\"partitionColumnName\":\"datafspwhfhdguuvga\",\"partitionUpperBound\":\"datazvd\",\"partitionLowerBound\":\"datatqzxemqnwpwr\"},\"queryTimeout\":\"datat\",\"additionalColumns\":\"datag\",\"sourceRetryCount\":\"dataeufhkoernrjm\",\"sourceRetryWait\":\"dataha\",\"maxConcurrentConnections\":\"dataen\",\"disableMetricsCollection\":\"dataqjvdde\",\"\":{\"wk\":\"datarjhtpxydiuviu\",\"suhozihd\":\"datansyrrybdyqivk\",\"hwcpijgas\":\"dataqdjw\",\"vzbdhrcepanhy\":\"dataafdjinwgirnjgs\"}}") .toObject(SqlMISource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SqlMISource model = new SqlMISource().withSourceRetryCount("datapbzyqbggxcyra") - .withSourceRetryWait("datazuaxtbr").withMaxConcurrentConnections("datayurxlpuwxslzq") - .withDisableMetricsCollection("datax").withQueryTimeout("dataqo").withAdditionalColumns("datalp") - .withSqlReaderQuery("datad").withSqlReaderStoredProcedureName("datalyhx") - .withStoredProcedureParameters("dataqoe").withIsolationLevel("databoqozxnuxamxikh") - .withProduceAdditionalTypes("dataikglynbqpeojecb").withPartitionOption("datagw") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datatnywgtsodnxeirjt") - .withPartitionUpperBound("dataimcfrhtzgd").withPartitionLowerBound("datao")); + SqlMISource model = new SqlMISource().withSourceRetryCount("dataeufhkoernrjm").withSourceRetryWait("dataha") + .withMaxConcurrentConnections("dataen").withDisableMetricsCollection("dataqjvdde").withQueryTimeout("datat") + .withAdditionalColumns("datag").withSqlReaderQuery("dataxhfttm") + .withSqlReaderStoredProcedureName("datamuwl").withStoredProcedureParameters("datajwkpznsfbi") + .withIsolationLevel("datafzgpvdlx").withProduceAdditionalTypes("dataotclcuxzllnwmgqc") + .withPartitionOption("datagjequox") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datafspwhfhdguuvga") + .withPartitionUpperBound("datazvd").withPartitionLowerBound("datatqzxemqnwpwr")); model = BinaryData.fromObject(model).toObject(SqlMISource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlPartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlPartitionSettingsTests.java index d25bd7504ee61..1c0d4ec7a2473 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlPartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlPartitionSettingsTests.java @@ -11,14 +11,14 @@ public final class SqlPartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlPartitionSettings model = BinaryData.fromString( - "{\"partitionColumnName\":\"dataeqkuozarr\",\"partitionUpperBound\":\"datapyzryjb\",\"partitionLowerBound\":\"databcvoyqnrjdrc\"}") + "{\"partitionColumnName\":\"dataokqeuzslny\",\"partitionUpperBound\":\"datauywijnlpeczq\",\"partitionLowerBound\":\"datamzkqydthf\"}") .toObject(SqlPartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SqlPartitionSettings model = new SqlPartitionSettings().withPartitionColumnName("dataeqkuozarr") - .withPartitionUpperBound("datapyzryjb").withPartitionLowerBound("databcvoyqnrjdrc"); + SqlPartitionSettings model = new SqlPartitionSettings().withPartitionColumnName("dataokqeuzslny") + .withPartitionUpperBound("datauywijnlpeczq").withPartitionLowerBound("datamzkqydthf"); model = BinaryData.fromObject(model).toObject(SqlPartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerSourceTests.java index 66fd5eff24581..e2e5e955bfadc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerSourceTests.java @@ -12,21 +12,21 @@ public final class SqlServerSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlServerSource model = BinaryData.fromString( - "{\"type\":\"SqlServerSource\",\"sqlReaderQuery\":\"datarvzewogh\",\"sqlReaderStoredProcedureName\":\"datazxkjqecj\",\"storedProcedureParameters\":\"dataromeawthycbigpi\",\"isolationLevel\":\"datapxhzjnparsulmuwl\",\"produceAdditionalTypes\":\"dataakheox\",\"partitionOption\":\"datagoavzycxpzat\",\"partitionSettings\":{\"partitionColumnName\":\"dataoomf\",\"partitionUpperBound\":\"dataxwtwzslrp\",\"partitionLowerBound\":\"datatqhfvouyqzho\"},\"queryTimeout\":\"dataemhohxabmxoow\",\"additionalColumns\":\"dataogozercczvpbnkgk\",\"sourceRetryCount\":\"datajeqqjqa\",\"sourceRetryWait\":\"datakajlogv\",\"maxConcurrentConnections\":\"datawqzolva\",\"disableMetricsCollection\":\"dataqkyc\",\"\":{\"ehjyb\":\"datag\",\"xiycd\":\"dataoq\",\"myd\":\"datadgem\",\"vgslm\":\"datazjtx\"}}") + "{\"type\":\"SqlServerSource\",\"sqlReaderQuery\":\"dataycmwvphrwuf\",\"sqlReaderStoredProcedureName\":\"dataov\",\"storedProcedureParameters\":\"dataisqlekc\",\"isolationLevel\":\"datadhlskeifwqtcownx\",\"produceAdditionalTypes\":\"datapptv\",\"partitionOption\":\"datadbnu\",\"partitionSettings\":{\"partitionColumnName\":\"datallyjelnhmuzhxk\",\"partitionUpperBound\":\"datazxk\",\"partitionLowerBound\":\"datal\"},\"queryTimeout\":\"datakbamahnwgccgblep\",\"additionalColumns\":\"datavl\",\"sourceRetryCount\":\"dataxdaoj\",\"sourceRetryWait\":\"datalqoxwqlnxvnmrl\",\"maxConcurrentConnections\":\"datajzya\",\"disableMetricsCollection\":\"datafecwnufldzjc\",\"\":{\"hfejgpef\":\"datajbzp\",\"shtujaqpkupnr\":\"databoxvwtlnv\"}}") .toObject(SqlServerSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SqlServerSource model - = new SqlServerSource().withSourceRetryCount("datajeqqjqa").withSourceRetryWait("datakajlogv") - .withMaxConcurrentConnections("datawqzolva").withDisableMetricsCollection("dataqkyc") - .withQueryTimeout("dataemhohxabmxoow").withAdditionalColumns("dataogozercczvpbnkgk") - .withSqlReaderQuery("datarvzewogh").withSqlReaderStoredProcedureName("datazxkjqecj") - .withStoredProcedureParameters("dataromeawthycbigpi").withIsolationLevel("datapxhzjnparsulmuwl") - .withProduceAdditionalTypes("dataakheox").withPartitionOption("datagoavzycxpzat") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("dataoomf") - .withPartitionUpperBound("dataxwtwzslrp").withPartitionLowerBound("datatqhfvouyqzho")); + = new SqlServerSource().withSourceRetryCount("dataxdaoj").withSourceRetryWait("datalqoxwqlnxvnmrl") + .withMaxConcurrentConnections("datajzya").withDisableMetricsCollection("datafecwnufldzjc") + .withQueryTimeout("datakbamahnwgccgblep").withAdditionalColumns("datavl") + .withSqlReaderQuery("dataycmwvphrwuf").withSqlReaderStoredProcedureName("dataov") + .withStoredProcedureParameters("dataisqlekc").withIsolationLevel("datadhlskeifwqtcownx") + .withProduceAdditionalTypes("datapptv").withPartitionOption("datadbnu") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datallyjelnhmuzhxk") + .withPartitionUpperBound("datazxk").withPartitionLowerBound("datal")); model = BinaryData.fromObject(model).toObject(SqlServerSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTests.java index 778f8c5db4496..85d71d0b6f39d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTests.java @@ -22,61 +22,52 @@ public final class SqlServerStoredProcedureActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlServerStoredProcedureActivity model = BinaryData.fromString( - "{\"type\":\"SqlServerStoredProcedure\",\"typeProperties\":{\"storedProcedureName\":\"datazuapkhfhuuizyeyf\",\"storedProcedureParameters\":\"datanidyjffpuuy\"},\"linkedServiceName\":{\"referenceName\":\"vbpneymlctnns\",\"parameters\":{\"zzp\":\"datafijvaxuv\",\"bnqyewinlenht\":\"dataldaaxglx\",\"hntoivi\":\"datakebtvnedcclp\",\"oxqkcayy\":\"dataerrieh\"}},\"policy\":{\"timeout\":\"datallkyiqj\",\"retry\":\"datavxgr\",\"retryIntervalInSeconds\":1235521806,\"secureInput\":false,\"secureOutput\":true,\"\":{\"vii\":\"datahivvoczsryp\",\"usnhnnekhfdlbc\":\"datajjqpsbbxkeygmqnu\",\"cccydldavozmibtk\":\"datacwfcbug\"}},\"name\":\"ftp\",\"description\":\"lsrrankxxwtnr\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"czjwizrulrkw\",\"dependencyConditions\":[\"Failed\",\"Failed\"],\"\":{\"myc\":\"datagcpqmkpobenaahdj\",\"qhpphjimo\":\"datatvpeirhstwpbvw\"}},{\"activity\":\"cqpqkpnvsuaizxdl\",\"dependencyConditions\":[\"Failed\"],\"\":{\"jiurldsftmllcdq\":\"dataotyjgxugf\",\"ruwqbe\":\"dataunvnggqacf\"}},{\"activity\":\"dzruuscbs\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Succeeded\",\"Succeeded\"],\"\":{\"ppphwvduuzpiooa\":\"dataifzqqsvofo\",\"qp\":\"datajxsofsiritp\",\"oux\":\"datanrl\"}}],\"userProperties\":[{\"name\":\"z\",\"value\":\"datacjnfyubc\"}],\"\":{\"navfpkleie\":\"datafnqqnum\",\"nbofeucctppbgzf\":\"datafpvbsllyoriad\"}}") + "{\"type\":\"SqlServerStoredProcedure\",\"typeProperties\":{\"storedProcedureName\":\"datazclnqexlnpwpw\",\"storedProcedureParameters\":\"datajsjkondrkncfoq\"},\"linkedServiceName\":{\"referenceName\":\"pslc\",\"parameters\":{\"ppwoli\":\"datawrsfdpikxsggaeg\",\"ophcwzdwvy\":\"dataflj\",\"vy\":\"datazo\",\"prnqjxsexzxbiwn\":\"datainmywjcfkmfoztwm\"}},\"policy\":{\"timeout\":\"dataqtbztogihpy\",\"retry\":\"datadryesgalspar\",\"retryIntervalInSeconds\":1336549807,\"secureInput\":true,\"secureOutput\":false,\"\":{\"nosblczctwacbn\":\"dataemkzeol\",\"mufhz\":\"datakpdcvjhykptcijun\"}},\"name\":\"cqhtlqr\",\"description\":\"rfxrg\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"acavzadybhydlqfx\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Failed\"],\"\":{\"haoviwuttlmfcn\":\"datahnsaespzwgpjri\",\"pihqmmmbokdqkf\":\"datatgai\",\"uwrvg\":\"datapfzxniv\",\"rlkgpipwtrt\":\"datal\"}}],\"userProperties\":[{\"name\":\"ipbddhfkjsqq\",\"value\":\"dataunoa\"},{\"name\":\"zkefz\",\"value\":\"datauyhvaovoqonqjlpc\"},{\"name\":\"yqiytrhhmld\",\"value\":\"datatyz\"}],\"\":{\"lkfg\":\"datast\",\"fe\":\"dataovbbcsb\",\"chfssbqwvr\":\"datamcprg\",\"qipfrrvngill\":\"datagvxhw\"}}") .toObject(SqlServerStoredProcedureActivity.class); - Assertions.assertEquals("ftp", model.name()); - Assertions.assertEquals("lsrrankxxwtnr", model.description()); + Assertions.assertEquals("cqhtlqr", model.name()); + Assertions.assertEquals("rfxrg", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("czjwizrulrkw", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("z", model.userProperties().get(0).name()); - Assertions.assertEquals("vbpneymlctnns", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1235521806, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals("acavzadybhydlqfx", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ipbddhfkjsqq", model.userProperties().get(0).name()); + Assertions.assertEquals("pslc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1336549807, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SqlServerStoredProcedureActivity model - = new SqlServerStoredProcedureActivity().withName("ftp").withDescription("lsrrankxxwtnr") + = new SqlServerStoredProcedureActivity().withName("cqhtlqr").withDescription("rfxrg") .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn( - Arrays.asList( - new ActivityDependency().withActivity("czjwizrulrkw") - .withDependencyConditions(Arrays.asList( - DependencyCondition.FAILED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency() - .withActivity("cqpqkpnvsuaizxdl") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dzruuscbs") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("z").withValue("datacjnfyubc"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("vbpneymlctnns") - .withParameters(mapOf("zzp", "datafijvaxuv", "bnqyewinlenht", "dataldaaxglx", "hntoivi", - "datakebtvnedcclp", "oxqkcayy", "dataerrieh"))) - .withPolicy(new ActivityPolicy().withTimeout("datallkyiqj").withRetry("datavxgr") - .withRetryIntervalInSeconds(1235521806).withSecureInput(false).withSecureOutput(true) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("acavzadybhydlqfx") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("ipbddhfkjsqq").withValue("dataunoa"), + new UserProperty().withName("zkefz").withValue("datauyhvaovoqonqjlpc"), + new UserProperty().withName("yqiytrhhmld").withValue("datatyz"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("pslc") + .withParameters(mapOf("ppwoli", "datawrsfdpikxsggaeg", "ophcwzdwvy", "dataflj", "vy", "datazo", + "prnqjxsexzxbiwn", "datainmywjcfkmfoztwm"))) + .withPolicy(new ActivityPolicy().withTimeout("dataqtbztogihpy").withRetry("datadryesgalspar") + .withRetryIntervalInSeconds(1336549807).withSecureInput(true).withSecureOutput(false) .withAdditionalProperties(mapOf())) - .withStoredProcedureName("datazuapkhfhuuizyeyf").withStoredProcedureParameters("datanidyjffpuuy"); + .withStoredProcedureName("datazclnqexlnpwpw").withStoredProcedureParameters("datajsjkondrkncfoq"); model = BinaryData.fromObject(model).toObject(SqlServerStoredProcedureActivity.class); - Assertions.assertEquals("ftp", model.name()); - Assertions.assertEquals("lsrrankxxwtnr", model.description()); + Assertions.assertEquals("cqhtlqr", model.name()); + Assertions.assertEquals("rfxrg", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("czjwizrulrkw", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("z", model.userProperties().get(0).name()); - Assertions.assertEquals("vbpneymlctnns", model.linkedServiceName().referenceName()); - Assertions.assertEquals(1235521806, model.policy().retryIntervalInSeconds()); - Assertions.assertEquals(false, model.policy().secureInput()); - Assertions.assertEquals(true, model.policy().secureOutput()); + Assertions.assertEquals("acavzadybhydlqfx", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("ipbddhfkjsqq", model.userProperties().get(0).name()); + Assertions.assertEquals("pslc", model.linkedServiceName().referenceName()); + Assertions.assertEquals(1336549807, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(true, model.policy().secureInput()); + Assertions.assertEquals(false, model.policy().secureOutput()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTypePropertiesTests.java index 139ba8d732699..ebcc3f11d9c99 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerStoredProcedureActivityTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class SqlServerStoredProcedureActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlServerStoredProcedureActivityTypeProperties model = BinaryData - .fromString("{\"storedProcedureName\":\"datauylsk\",\"storedProcedureParameters\":\"datavwdftrqso\"}") + .fromString("{\"storedProcedureName\":\"datamfbl\",\"storedProcedureParameters\":\"dataekoux\"}") .toObject(SqlServerStoredProcedureActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SqlServerStoredProcedureActivityTypeProperties model = new SqlServerStoredProcedureActivityTypeProperties() - .withStoredProcedureName("datauylsk").withStoredProcedureParameters("datavwdftrqso"); + .withStoredProcedureName("datamfbl").withStoredProcedureParameters("dataekoux"); model = BinaryData.fromObject(model).toObject(SqlServerStoredProcedureActivityTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTests.java index edbdf53751a4f..3d75ed9c13f41 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTests.java @@ -19,29 +19,31 @@ public final class SqlServerTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlServerTableDataset model = BinaryData.fromString( - "{\"type\":\"SqlServerTable\",\"typeProperties\":{\"tableName\":\"datazcq\",\"schema\":\"datavnkyakck\",\"table\":\"datah\"},\"description\":\"nsddjkkd\",\"structure\":\"dataesu\",\"schema\":\"dataogfcnxcxgxum\",\"linkedServiceName\":{\"referenceName\":\"cqxmyvkxixy\",\"parameters\":{\"g\":\"dataifjc\"}},\"parameters\":{\"chmxczbyfkoc\":{\"type\":\"Bool\",\"defaultValue\":\"datauw\"}},\"annotations\":[\"datadctsnlwscrngt\"],\"folder\":{\"name\":\"rolwv\"},\"\":{\"cucti\":\"datasdksut\",\"ux\":\"dataavishbvjhxvpmqqu\",\"lexoweorocr\":\"dataphngr\",\"gbq\":\"dataicgym\"}}") + "{\"type\":\"SqlServerTable\",\"typeProperties\":{\"tableName\":\"dataszfutgpbygbnbc\",\"schema\":\"dataiqgtzpv\",\"table\":\"datawfl\"},\"description\":\"hxzuxerxhyw\",\"structure\":\"datakqsqvvdkfp\",\"schema\":\"datadajdqxymxxyfr\",\"linkedServiceName\":{\"referenceName\":\"j\",\"parameters\":{\"ld\":\"dataetfvgwfw\",\"rsnxfrp\":\"datagwouppvyddqsvc\",\"xzxlcqzfxa\":\"datawwqclmdmtfxxepz\"}},\"parameters\":{\"smkir\":{\"type\":\"SecureString\",\"defaultValue\":\"datacj\"},\"hkcomeobw\":{\"type\":\"Bool\",\"defaultValue\":\"dataipud\"}},\"annotations\":[\"datazltenlbfxl\",\"dataxozesn\"],\"folder\":{\"name\":\"uomtxj\"},\"\":{\"wis\":\"dataxymckikkqyvur\",\"ktehognsdd\":\"datayfmrzcqfevnkyak\"}}") .toObject(SqlServerTableDataset.class); - Assertions.assertEquals("nsddjkkd", model.description()); - Assertions.assertEquals("cqxmyvkxixy", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("chmxczbyfkoc").type()); - Assertions.assertEquals("rolwv", model.folder().name()); + Assertions.assertEquals("hxzuxerxhyw", model.description()); + Assertions.assertEquals("j", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("smkir").type()); + Assertions.assertEquals("uomtxj", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SqlServerTableDataset model = new SqlServerTableDataset().withDescription("nsddjkkd").withStructure("dataesu") - .withSchema("dataogfcnxcxgxum") - .withLinkedServiceName( - new LinkedServiceReference().withReferenceName("cqxmyvkxixy").withParameters(mapOf("g", "dataifjc"))) - .withParameters(mapOf("chmxczbyfkoc", - new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datauw"))) - .withAnnotations(Arrays.asList("datadctsnlwscrngt")).withFolder(new DatasetFolder().withName("rolwv")) - .withTableName("datazcq").withSchemaTypePropertiesSchema("datavnkyakck").withTable("datah"); + SqlServerTableDataset model = new SqlServerTableDataset().withDescription("hxzuxerxhyw") + .withStructure("datakqsqvvdkfp").withSchema("datadajdqxymxxyfr") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("j").withParameters( + mapOf("ld", "dataetfvgwfw", "rsnxfrp", "datagwouppvyddqsvc", "xzxlcqzfxa", "datawwqclmdmtfxxepz"))) + .withParameters(mapOf("smkir", + new ParameterSpecification().withType(ParameterType.SECURE_STRING).withDefaultValue("datacj"), + "hkcomeobw", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("dataipud"))) + .withAnnotations(Arrays.asList("datazltenlbfxl", "dataxozesn")) + .withFolder(new DatasetFolder().withName("uomtxj")).withTableName("dataszfutgpbygbnbc") + .withSchemaTypePropertiesSchema("dataiqgtzpv").withTable("datawfl"); model = BinaryData.fromObject(model).toObject(SqlServerTableDataset.class); - Assertions.assertEquals("nsddjkkd", model.description()); - Assertions.assertEquals("cqxmyvkxixy", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.BOOL, model.parameters().get("chmxczbyfkoc").type()); - Assertions.assertEquals("rolwv", model.folder().name()); + Assertions.assertEquals("hxzuxerxhyw", model.description()); + Assertions.assertEquals("j", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.SECURE_STRING, model.parameters().get("smkir").type()); + Assertions.assertEquals("uomtxj", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTypePropertiesTests.java index 7401378cccd40..9209c8ae57240 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlServerTableDatasetTypePropertiesTests.java @@ -10,15 +10,15 @@ public final class SqlServerTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SqlServerTableDatasetTypeProperties model = BinaryData - .fromString("{\"tableName\":\"datayrvhtv\",\"schema\":\"datavwmrgcnzhrplc\",\"table\":\"datambzquu\"}") - .toObject(SqlServerTableDatasetTypeProperties.class); + SqlServerTableDatasetTypeProperties model + = BinaryData.fromString("{\"tableName\":\"datakde\",\"schema\":\"datasuaz\",\"table\":\"datafcnxc\"}") + .toObject(SqlServerTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SqlServerTableDatasetTypeProperties model = new SqlServerTableDatasetTypeProperties() - .withTableName("datayrvhtv").withSchema("datavwmrgcnzhrplc").withTable("datambzquu"); + SqlServerTableDatasetTypeProperties model = new SqlServerTableDatasetTypeProperties().withTableName("datakde") + .withSchema("datasuaz").withTable("datafcnxc"); model = BinaryData.fromObject(model).toObject(SqlServerTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlSourceTests.java index f269b5308588b..90699870d26ba 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SqlSourceTests.java @@ -12,19 +12,19 @@ public final class SqlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SqlSource model = BinaryData.fromString( - "{\"type\":\"SqlSource\",\"sqlReaderQuery\":\"dataepkrncjrqhugu\",\"sqlReaderStoredProcedureName\":\"databqq\",\"storedProcedureParameters\":\"datahcmvdowlqcyhfub\",\"isolationLevel\":\"dataxqxxgrawbftznxfr\",\"partitionOption\":\"datajefiutbrnr\",\"partitionSettings\":{\"partitionColumnName\":\"datajucodrb\",\"partitionUpperBound\":\"dataieismdkvfi\",\"partitionLowerBound\":\"dataecij\"},\"queryTimeout\":\"datamnkvpafoe\",\"additionalColumns\":\"datacsk\",\"sourceRetryCount\":\"datawzmji\",\"sourceRetryWait\":\"dataqyllcckgfo\",\"maxConcurrentConnections\":\"datarbfyjmenq\",\"disableMetricsCollection\":\"datajfxqtvsfsvqy\",\"\":{\"zwhdaczixf\":\"dataweixnobl\",\"s\":\"datauw\",\"hmiu\":\"datavitpcsmaxzdxu\",\"zajtbnekhjzbfb\":\"dataypvu\"}}") + "{\"type\":\"SqlSource\",\"sqlReaderQuery\":\"dataplo\",\"sqlReaderStoredProcedureName\":\"datahimvlocdxvhkobi\",\"storedProcedureParameters\":\"datahipntrddyr\",\"isolationLevel\":\"dataanv\",\"partitionOption\":\"datagmqscijlfulxgnza\",\"partitionSettings\":{\"partitionColumnName\":\"datamwsooq\",\"partitionUpperBound\":\"datavplmyzebvgh\",\"partitionLowerBound\":\"dataydehbvbexrbynnl\"},\"queryTimeout\":\"datad\",\"additionalColumns\":\"datak\",\"sourceRetryCount\":\"datazzsi\",\"sourceRetryWait\":\"databosacrnpscfkef\",\"maxConcurrentConnections\":\"datatxe\",\"disableMetricsCollection\":\"datamimgjuvjvtgece\",\"\":{\"oukfjwkctdn\":\"datanled\"}}") .toObject(SqlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SqlSource model = new SqlSource().withSourceRetryCount("datawzmji").withSourceRetryWait("dataqyllcckgfo") - .withMaxConcurrentConnections("datarbfyjmenq").withDisableMetricsCollection("datajfxqtvsfsvqy") - .withQueryTimeout("datamnkvpafoe").withAdditionalColumns("datacsk").withSqlReaderQuery("dataepkrncjrqhugu") - .withSqlReaderStoredProcedureName("databqq").withStoredProcedureParameters("datahcmvdowlqcyhfub") - .withIsolationLevel("dataxqxxgrawbftznxfr").withPartitionOption("datajefiutbrnr") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datajucodrb") - .withPartitionUpperBound("dataieismdkvfi").withPartitionLowerBound("dataecij")); + SqlSource model = new SqlSource().withSourceRetryCount("datazzsi").withSourceRetryWait("databosacrnpscfkef") + .withMaxConcurrentConnections("datatxe").withDisableMetricsCollection("datamimgjuvjvtgece") + .withQueryTimeout("datad").withAdditionalColumns("datak").withSqlReaderQuery("dataplo") + .withSqlReaderStoredProcedureName("datahimvlocdxvhkobi").withStoredProcedureParameters("datahipntrddyr") + .withIsolationLevel("dataanv").withPartitionOption("datagmqscijlfulxgnza") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datamwsooq") + .withPartitionUpperBound("datavplmyzebvgh").withPartitionLowerBound("dataydehbvbexrbynnl")); model = BinaryData.fromObject(model).toObject(SqlSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SquareSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SquareSourceTests.java index 7c7d4d1b0ccdc..e1ba19606fc67 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SquareSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SquareSourceTests.java @@ -11,15 +11,15 @@ public final class SquareSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SquareSource model = BinaryData.fromString( - "{\"type\":\"SquareSource\",\"query\":\"dataxllfwxdou\",\"queryTimeout\":\"datazpaqjahjxgedtmzh\",\"additionalColumns\":\"datahktywtac\",\"sourceRetryCount\":\"datakie\",\"sourceRetryWait\":\"dataqrfassiiil\",\"maxConcurrentConnections\":\"datargahscay\",\"disableMetricsCollection\":\"datagc\",\"\":{\"vqopxun\":\"dataieqonsbukznxd\"}}") + "{\"type\":\"SquareSource\",\"query\":\"dataeycakkon\",\"queryTimeout\":\"datadpd\",\"additionalColumns\":\"datahadzyxaanhwuqewc\",\"sourceRetryCount\":\"datasksfbkxfkeeqo\",\"sourceRetryWait\":\"databek\",\"maxConcurrentConnections\":\"dataerwss\",\"disableMetricsCollection\":\"datamrpdjrylfpdudx\",\"\":{\"tqssngeviyffg\":\"dataeuriehxbanfsqfh\",\"hdapynpvgyaf\":\"datahrhjsps\"}}") .toObject(SquareSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SquareSource model = new SquareSource().withSourceRetryCount("datakie").withSourceRetryWait("dataqrfassiiil") - .withMaxConcurrentConnections("datargahscay").withDisableMetricsCollection("datagc") - .withQueryTimeout("datazpaqjahjxgedtmzh").withAdditionalColumns("datahktywtac").withQuery("dataxllfwxdou"); + SquareSource model = new SquareSource().withSourceRetryCount("datasksfbkxfkeeqo").withSourceRetryWait("databek") + .withMaxConcurrentConnections("dataerwss").withDisableMetricsCollection("datamrpdjrylfpdudx") + .withQueryTimeout("datadpd").withAdditionalColumns("datahadzyxaanhwuqewc").withQuery("dataeycakkon"); model = BinaryData.fromObject(model).toObject(SquareSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisChildPackageTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisChildPackageTests.java index dbc9e457e02f8..b6898898f7b34 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisChildPackageTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisChildPackageTests.java @@ -12,18 +12,18 @@ public final class SsisChildPackageTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisChildPackage model = BinaryData.fromString( - "{\"packagePath\":\"datavkuvykiyrfo\",\"packageName\":\"lcoikstap\",\"packageContent\":\"databdh\",\"packageLastModifiedDate\":\"mugkugwtg\"}") + "{\"packagePath\":\"datadflckumjjpx\",\"packageName\":\"xabvx\",\"packageContent\":\"dataoagoeills\",\"packageLastModifiedDate\":\"gy\"}") .toObject(SsisChildPackage.class); - Assertions.assertEquals("lcoikstap", model.packageName()); - Assertions.assertEquals("mugkugwtg", model.packageLastModifiedDate()); + Assertions.assertEquals("xabvx", model.packageName()); + Assertions.assertEquals("gy", model.packageLastModifiedDate()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisChildPackage model = new SsisChildPackage().withPackagePath("datavkuvykiyrfo").withPackageName("lcoikstap") - .withPackageContent("databdh").withPackageLastModifiedDate("mugkugwtg"); + SsisChildPackage model = new SsisChildPackage().withPackagePath("datadflckumjjpx").withPackageName("xabvx") + .withPackageContent("dataoagoeills").withPackageLastModifiedDate("gy"); model = BinaryData.fromObject(model).toObject(SsisChildPackage.class); - Assertions.assertEquals("lcoikstap", model.packageName()); - Assertions.assertEquals("mugkugwtg", model.packageLastModifiedDate()); + Assertions.assertEquals("xabvx", model.packageName()); + Assertions.assertEquals("gy", model.packageLastModifiedDate()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentReferenceTests.java index a31877d552403..ebab0da2548f0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentReferenceTests.java @@ -12,23 +12,22 @@ public final class SsisEnvironmentReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisEnvironmentReference model = BinaryData.fromString( - "{\"id\":2367681337627846678,\"environmentFolderName\":\"stnwvravntvklkwq\",\"environmentName\":\"nlpaymketotk\",\"referenceType\":\"mewwlkryz\"}") + "{\"id\":858241301939195859,\"environmentFolderName\":\"dulymk\",\"environmentName\":\"s\",\"referenceType\":\"h\"}") .toObject(SsisEnvironmentReference.class); - Assertions.assertEquals(2367681337627846678L, model.id()); - Assertions.assertEquals("stnwvravntvklkwq", model.environmentFolderName()); - Assertions.assertEquals("nlpaymketotk", model.environmentName()); - Assertions.assertEquals("mewwlkryz", model.referenceType()); + Assertions.assertEquals(858241301939195859L, model.id()); + Assertions.assertEquals("dulymk", model.environmentFolderName()); + Assertions.assertEquals("s", model.environmentName()); + Assertions.assertEquals("h", model.referenceType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisEnvironmentReference model - = new SsisEnvironmentReference().withId(2367681337627846678L).withEnvironmentFolderName("stnwvravntvklkwq") - .withEnvironmentName("nlpaymketotk").withReferenceType("mewwlkryz"); + SsisEnvironmentReference model = new SsisEnvironmentReference().withId(858241301939195859L) + .withEnvironmentFolderName("dulymk").withEnvironmentName("s").withReferenceType("h"); model = BinaryData.fromObject(model).toObject(SsisEnvironmentReference.class); - Assertions.assertEquals(2367681337627846678L, model.id()); - Assertions.assertEquals("stnwvravntvklkwq", model.environmentFolderName()); - Assertions.assertEquals("nlpaymketotk", model.environmentName()); - Assertions.assertEquals("mewwlkryz", model.referenceType()); + Assertions.assertEquals(858241301939195859L, model.id()); + Assertions.assertEquals("dulymk", model.environmentFolderName()); + Assertions.assertEquals("s", model.environmentName()); + Assertions.assertEquals("h", model.referenceType()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentTests.java index a2c927032a9a5..3b668e83335df 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisEnvironmentTests.java @@ -14,48 +14,45 @@ public final class SsisEnvironmentTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisEnvironment model = BinaryData.fromString( - "{\"type\":\"Environment\",\"folderId\":5803007966104299513,\"variables\":[{\"id\":3760635159453315925,\"name\":\"fpfowweylszrtngi\",\"description\":\"yp\",\"dataType\":\"nfcorcnhf\",\"sensitive\":false,\"value\":\"cbmayhdtjfrponaj\",\"sensitiveValue\":\"jqzbrwrf\"},{\"id\":4176721555817426691,\"name\":\"fokrbgvojcks\",\"description\":\"snqunnikl\",\"dataType\":\"deggnzadqmvpe\",\"sensitive\":true,\"value\":\"vkyqhrplfqqnw\",\"sensitiveValue\":\"srgclxnsvbkhh\"},{\"id\":4951180645110169456,\"name\":\"uecmgukywdpu\",\"description\":\"lcex\",\"dataType\":\"pit\",\"sensitive\":false,\"value\":\"bseu\",\"sensitiveValue\":\"mtcidca\"},{\"id\":7501570445950128623,\"name\":\"cxctshxoeftfor\",\"description\":\"xaknwkjzvqpsym\",\"dataType\":\"pyjtrxxzwd\",\"sensitive\":false,\"value\":\"yefnakdmtpjksdlu\",\"sensitiveValue\":\"tjxhxwt\"}],\"id\":8367398360175666390,\"name\":\"vukvupuplug\",\"description\":\"ynv\"}") + "{\"type\":\"Environment\",\"folderId\":832771683225896076,\"variables\":[{\"id\":3326667507931302810,\"name\":\"ejdhrodyiitredd\",\"description\":\"goppybs\",\"dataType\":\"kgaxmhaszjietfst\",\"sensitive\":true,\"value\":\"vzcnlk\",\"sensitiveValue\":\"rjtkreiso\"},{\"id\":8273566367613483537,\"name\":\"mgrbkobmgwa\",\"description\":\"mqpaalwidt\",\"dataType\":\"wedj\",\"sensitive\":true,\"value\":\"ppgijn\",\"sensitiveValue\":\"ba\"},{\"id\":8583337090822703542,\"name\":\"gaxpy\",\"description\":\"mccqdss\",\"dataType\":\"gersdud\",\"sensitive\":false,\"value\":\"mnfgzmxtxfuhxy\",\"sensitiveValue\":\"fyzevcknglf\"}],\"id\":926387397922110716,\"name\":\"fysffrpjf\",\"description\":\"yx\"}") .toObject(SsisEnvironment.class); - Assertions.assertEquals(8367398360175666390L, model.id()); - Assertions.assertEquals("vukvupuplug", model.name()); - Assertions.assertEquals("ynv", model.description()); - Assertions.assertEquals(5803007966104299513L, model.folderId()); - Assertions.assertEquals(3760635159453315925L, model.variables().get(0).id()); - Assertions.assertEquals("fpfowweylszrtngi", model.variables().get(0).name()); - Assertions.assertEquals("yp", model.variables().get(0).description()); - Assertions.assertEquals("nfcorcnhf", model.variables().get(0).dataType()); - Assertions.assertEquals(false, model.variables().get(0).sensitive()); - Assertions.assertEquals("cbmayhdtjfrponaj", model.variables().get(0).value()); - Assertions.assertEquals("jqzbrwrf", model.variables().get(0).sensitiveValue()); + Assertions.assertEquals(926387397922110716L, model.id()); + Assertions.assertEquals("fysffrpjf", model.name()); + Assertions.assertEquals("yx", model.description()); + Assertions.assertEquals(832771683225896076L, model.folderId()); + Assertions.assertEquals(3326667507931302810L, model.variables().get(0).id()); + Assertions.assertEquals("ejdhrodyiitredd", model.variables().get(0).name()); + Assertions.assertEquals("goppybs", model.variables().get(0).description()); + Assertions.assertEquals("kgaxmhaszjietfst", model.variables().get(0).dataType()); + Assertions.assertEquals(true, model.variables().get(0).sensitive()); + Assertions.assertEquals("vzcnlk", model.variables().get(0).value()); + Assertions.assertEquals("rjtkreiso", model.variables().get(0).sensitiveValue()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisEnvironment model = new SsisEnvironment().withId(8367398360175666390L).withName("vukvupuplug") - .withDescription("ynv").withFolderId(5803007966104299513L) + SsisEnvironment model = new SsisEnvironment().withId(926387397922110716L).withName("fysffrpjf") + .withDescription("yx").withFolderId(832771683225896076L) .withVariables(Arrays.asList( - new SsisVariable().withId(3760635159453315925L).withName("fpfowweylszrtngi").withDescription("yp") - .withDataType("nfcorcnhf").withSensitive(false).withValue("cbmayhdtjfrponaj") - .withSensitiveValue("jqzbrwrf"), - new SsisVariable().withId(4176721555817426691L).withName("fokrbgvojcks").withDescription("snqunnikl") - .withDataType("deggnzadqmvpe").withSensitive(true).withValue("vkyqhrplfqqnw") - .withSensitiveValue("srgclxnsvbkhh"), - new SsisVariable().withId(4951180645110169456L).withName("uecmgukywdpu").withDescription("lcex") - .withDataType("pit").withSensitive(false).withValue("bseu").withSensitiveValue("mtcidca"), - new SsisVariable().withId(7501570445950128623L).withName("cxctshxoeftfor") - .withDescription("xaknwkjzvqpsym").withDataType("pyjtrxxzwd").withSensitive(false) - .withValue("yefnakdmtpjksdlu").withSensitiveValue("tjxhxwt"))); + new SsisVariable().withId(3326667507931302810L).withName("ejdhrodyiitredd").withDescription("goppybs") + .withDataType("kgaxmhaszjietfst").withSensitive(true).withValue("vzcnlk") + .withSensitiveValue("rjtkreiso"), + new SsisVariable().withId(8273566367613483537L).withName("mgrbkobmgwa").withDescription("mqpaalwidt") + .withDataType("wedj").withSensitive(true).withValue("ppgijn").withSensitiveValue("ba"), + new SsisVariable().withId(8583337090822703542L).withName("gaxpy").withDescription("mccqdss") + .withDataType("gersdud").withSensitive(false).withValue("mnfgzmxtxfuhxy") + .withSensitiveValue("fyzevcknglf"))); model = BinaryData.fromObject(model).toObject(SsisEnvironment.class); - Assertions.assertEquals(8367398360175666390L, model.id()); - Assertions.assertEquals("vukvupuplug", model.name()); - Assertions.assertEquals("ynv", model.description()); - Assertions.assertEquals(5803007966104299513L, model.folderId()); - Assertions.assertEquals(3760635159453315925L, model.variables().get(0).id()); - Assertions.assertEquals("fpfowweylszrtngi", model.variables().get(0).name()); - Assertions.assertEquals("yp", model.variables().get(0).description()); - Assertions.assertEquals("nfcorcnhf", model.variables().get(0).dataType()); - Assertions.assertEquals(false, model.variables().get(0).sensitive()); - Assertions.assertEquals("cbmayhdtjfrponaj", model.variables().get(0).value()); - Assertions.assertEquals("jqzbrwrf", model.variables().get(0).sensitiveValue()); + Assertions.assertEquals(926387397922110716L, model.id()); + Assertions.assertEquals("fysffrpjf", model.name()); + Assertions.assertEquals("yx", model.description()); + Assertions.assertEquals(832771683225896076L, model.folderId()); + Assertions.assertEquals(3326667507931302810L, model.variables().get(0).id()); + Assertions.assertEquals("ejdhrodyiitredd", model.variables().get(0).name()); + Assertions.assertEquals("goppybs", model.variables().get(0).description()); + Assertions.assertEquals("kgaxmhaszjietfst", model.variables().get(0).dataType()); + Assertions.assertEquals(true, model.variables().get(0).sensitive()); + Assertions.assertEquals("vzcnlk", model.variables().get(0).value()); + Assertions.assertEquals("rjtkreiso", model.variables().get(0).sensitiveValue()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisExecutionParameterTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisExecutionParameterTests.java index bd8f8287520e7..a364ceefc7880 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisExecutionParameterTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisExecutionParameterTests.java @@ -11,12 +11,12 @@ public final class SsisExecutionParameterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisExecutionParameter model - = BinaryData.fromString("{\"value\":\"dataghtkdcuf\"}").toObject(SsisExecutionParameter.class); + = BinaryData.fromString("{\"value\":\"datavwmybokqpfhs\"}").toObject(SsisExecutionParameter.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisExecutionParameter model = new SsisExecutionParameter().withValue("dataghtkdcuf"); + SsisExecutionParameter model = new SsisExecutionParameter().withValue("datavwmybokqpfhs"); model = BinaryData.fromObject(model).toObject(SsisExecutionParameter.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisFolderTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisFolderTests.java index 398faa36a32e6..dd4f0210e6a03 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisFolderTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisFolderTests.java @@ -12,20 +12,20 @@ public final class SsisFolderTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisFolder model = BinaryData.fromString( - "{\"type\":\"Folder\",\"id\":4550200017563706482,\"name\":\"terma\",\"description\":\"iqrsnmftubqwxv\"}") + "{\"type\":\"Folder\",\"id\":9119871534508040877,\"name\":\"nzadqmvpehp\",\"description\":\"vkyqhrplfqqnw\"}") .toObject(SsisFolder.class); - Assertions.assertEquals(4550200017563706482L, model.id()); - Assertions.assertEquals("terma", model.name()); - Assertions.assertEquals("iqrsnmftubqwxv", model.description()); + Assertions.assertEquals(9119871534508040877L, model.id()); + Assertions.assertEquals("nzadqmvpehp", model.name()); + Assertions.assertEquals("vkyqhrplfqqnw", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SsisFolder model - = new SsisFolder().withId(4550200017563706482L).withName("terma").withDescription("iqrsnmftubqwxv"); + = new SsisFolder().withId(9119871534508040877L).withName("nzadqmvpehp").withDescription("vkyqhrplfqqnw"); model = BinaryData.fromObject(model).toObject(SsisFolder.class); - Assertions.assertEquals(4550200017563706482L, model.id()); - Assertions.assertEquals("terma", model.name()); - Assertions.assertEquals("iqrsnmftubqwxv", model.description()); + Assertions.assertEquals(9119871534508040877L, model.id()); + Assertions.assertEquals("nzadqmvpehp", model.name()); + Assertions.assertEquals("vkyqhrplfqqnw", model.description()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPackageTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPackageTests.java index eeb61ea5d37d3..0c217943183a6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPackageTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPackageTests.java @@ -14,69 +14,55 @@ public final class SsisPackageTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisPackage model = BinaryData.fromString( - "{\"type\":\"Package\",\"folderId\":6916090180367931254,\"projectVersion\":902719333920741034,\"projectId\":1331588668258189103,\"parameters\":[{\"id\":5812566392029071194,\"name\":\"ejcccpxb\",\"description\":\"itafjjevptvit\",\"dataType\":\"zqwvkpar\",\"required\":true,\"sensitive\":false,\"designDefaultValue\":\"tqc\",\"defaultValue\":\"mbsudcykg\",\"sensitiveDefaultValue\":\"ellfwrmsuxyqbm\",\"valueType\":\"ubxlpkdsnbqoy\",\"valueSet\":false,\"variable\":\"ra\"},{\"id\":2697828338162056747,\"name\":\"axzyrp\",\"description\":\"y\",\"dataType\":\"jribeskkopbksrib\",\"required\":true,\"sensitive\":true,\"designDefaultValue\":\"ikczerq\",\"defaultValue\":\"qxgwsfr\",\"sensitiveDefaultValue\":\"vzlljydgmdoq\",\"valueType\":\"ucwzcyjudhgwjqe\",\"valueSet\":true,\"variable\":\"nrruwsqp\"},{\"id\":4606170167698003019,\"name\":\"qm\",\"description\":\"xqcil\",\"dataType\":\"ulgnnyxwdpm\",\"required\":true,\"sensitive\":false,\"designDefaultValue\":\"z\",\"defaultValue\":\"bamqrbbsnptm\",\"sensitiveDefaultValue\":\"dpavcjkbyjuwhizc\",\"valueType\":\"jxm\",\"valueSet\":false,\"variable\":\"unpcskdlrf\"},{\"id\":4968103143818459144,\"name\":\"ltjdauuj\",\"description\":\"tyhdd\",\"dataType\":\"ykudjzrimy\",\"required\":false,\"sensitive\":false,\"designDefaultValue\":\"y\",\"defaultValue\":\"uksaxznqqkqxkd\",\"sensitiveDefaultValue\":\"seuqkrbyyakrjgb\",\"valueType\":\"gokrllbecannvxo\",\"valueSet\":false,\"variable\":\"ftp\"}],\"id\":5255929803700515581,\"name\":\"pmfw\",\"description\":\"cxhwkgihifoyo\"}") + "{\"type\":\"Package\",\"folderId\":3556959886109984896,\"projectVersion\":4692852653848740418,\"projectId\":668264090121943233,\"parameters\":[{\"id\":525854716862013224,\"name\":\"ifywxjjylaqhx\",\"description\":\"fdarvjhwgkynxlw\",\"dataType\":\"gotdt\",\"required\":false,\"sensitive\":true,\"designDefaultValue\":\"lh\",\"defaultValue\":\"ybfnkylzrignqlwo\",\"sensitiveDefaultValue\":\"nbjuaiu\",\"valueType\":\"mqcbnk\",\"valueSet\":true,\"variable\":\"i\"}],\"id\":4578072156366697994,\"name\":\"ryywyfcenkbfxqc\",\"description\":\"ggeciradmxokbutb\"}") .toObject(SsisPackage.class); - Assertions.assertEquals(5255929803700515581L, model.id()); - Assertions.assertEquals("pmfw", model.name()); - Assertions.assertEquals("cxhwkgihifoyo", model.description()); - Assertions.assertEquals(6916090180367931254L, model.folderId()); - Assertions.assertEquals(902719333920741034L, model.projectVersion()); - Assertions.assertEquals(1331588668258189103L, model.projectId()); - Assertions.assertEquals(5812566392029071194L, model.parameters().get(0).id()); - Assertions.assertEquals("ejcccpxb", model.parameters().get(0).name()); - Assertions.assertEquals("itafjjevptvit", model.parameters().get(0).description()); - Assertions.assertEquals("zqwvkpar", model.parameters().get(0).dataType()); - Assertions.assertEquals(true, model.parameters().get(0).required()); - Assertions.assertEquals(false, model.parameters().get(0).sensitive()); - Assertions.assertEquals("tqc", model.parameters().get(0).designDefaultValue()); - Assertions.assertEquals("mbsudcykg", model.parameters().get(0).defaultValue()); - Assertions.assertEquals("ellfwrmsuxyqbm", model.parameters().get(0).sensitiveDefaultValue()); - Assertions.assertEquals("ubxlpkdsnbqoy", model.parameters().get(0).valueType()); - Assertions.assertEquals(false, model.parameters().get(0).valueSet()); - Assertions.assertEquals("ra", model.parameters().get(0).variable()); + Assertions.assertEquals(4578072156366697994L, model.id()); + Assertions.assertEquals("ryywyfcenkbfxqc", model.name()); + Assertions.assertEquals("ggeciradmxokbutb", model.description()); + Assertions.assertEquals(3556959886109984896L, model.folderId()); + Assertions.assertEquals(4692852653848740418L, model.projectVersion()); + Assertions.assertEquals(668264090121943233L, model.projectId()); + Assertions.assertEquals(525854716862013224L, model.parameters().get(0).id()); + Assertions.assertEquals("ifywxjjylaqhx", model.parameters().get(0).name()); + Assertions.assertEquals("fdarvjhwgkynxlw", model.parameters().get(0).description()); + Assertions.assertEquals("gotdt", model.parameters().get(0).dataType()); + Assertions.assertEquals(false, model.parameters().get(0).required()); + Assertions.assertEquals(true, model.parameters().get(0).sensitive()); + Assertions.assertEquals("lh", model.parameters().get(0).designDefaultValue()); + Assertions.assertEquals("ybfnkylzrignqlwo", model.parameters().get(0).defaultValue()); + Assertions.assertEquals("nbjuaiu", model.parameters().get(0).sensitiveDefaultValue()); + Assertions.assertEquals("mqcbnk", model.parameters().get(0).valueType()); + Assertions.assertEquals(true, model.parameters().get(0).valueSet()); + Assertions.assertEquals("i", model.parameters().get(0).variable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisPackage model = new SsisPackage().withId(5255929803700515581L).withName("pmfw") - .withDescription("cxhwkgihifoyo").withFolderId(6916090180367931254L).withProjectVersion(902719333920741034L) - .withProjectId(1331588668258189103L) - .withParameters(Arrays.asList( - new SsisParameter().withId(5812566392029071194L).withName("ejcccpxb").withDescription("itafjjevptvit") - .withDataType("zqwvkpar").withRequired(true).withSensitive(false).withDesignDefaultValue("tqc") - .withDefaultValue("mbsudcykg").withSensitiveDefaultValue("ellfwrmsuxyqbm") - .withValueType("ubxlpkdsnbqoy").withValueSet(false).withVariable("ra"), - new SsisParameter().withId(2697828338162056747L).withName("axzyrp").withDescription("y") - .withDataType("jribeskkopbksrib").withRequired(true).withSensitive(true) - .withDesignDefaultValue("ikczerq").withDefaultValue("qxgwsfr") - .withSensitiveDefaultValue("vzlljydgmdoq").withValueType("ucwzcyjudhgwjqe").withValueSet(true) - .withVariable("nrruwsqp"), - new SsisParameter().withId(4606170167698003019L).withName("qm").withDescription("xqcil") - .withDataType("ulgnnyxwdpm").withRequired(true).withSensitive(false).withDesignDefaultValue("z") - .withDefaultValue("bamqrbbsnptm").withSensitiveDefaultValue("dpavcjkbyjuwhizc").withValueType("jxm") - .withValueSet(false).withVariable("unpcskdlrf"), - new SsisParameter().withId(4968103143818459144L).withName("ltjdauuj").withDescription("tyhdd") - .withDataType("ykudjzrimy").withRequired(false).withSensitive(false).withDesignDefaultValue("y") - .withDefaultValue("uksaxznqqkqxkd").withSensitiveDefaultValue("seuqkrbyyakrjgb") - .withValueType("gokrllbecannvxo").withValueSet(false).withVariable("ftp"))); + SsisPackage model = new SsisPackage().withId(4578072156366697994L).withName("ryywyfcenkbfxqc") + .withDescription("ggeciradmxokbutb").withFolderId(3556959886109984896L) + .withProjectVersion(4692852653848740418L).withProjectId(668264090121943233L) + .withParameters(Arrays.asList(new SsisParameter().withId(525854716862013224L).withName("ifywxjjylaqhx") + .withDescription("fdarvjhwgkynxlw").withDataType("gotdt").withRequired(false).withSensitive(true) + .withDesignDefaultValue("lh").withDefaultValue("ybfnkylzrignqlwo").withSensitiveDefaultValue("nbjuaiu") + .withValueType("mqcbnk").withValueSet(true).withVariable("i"))); model = BinaryData.fromObject(model).toObject(SsisPackage.class); - Assertions.assertEquals(5255929803700515581L, model.id()); - Assertions.assertEquals("pmfw", model.name()); - Assertions.assertEquals("cxhwkgihifoyo", model.description()); - Assertions.assertEquals(6916090180367931254L, model.folderId()); - Assertions.assertEquals(902719333920741034L, model.projectVersion()); - Assertions.assertEquals(1331588668258189103L, model.projectId()); - Assertions.assertEquals(5812566392029071194L, model.parameters().get(0).id()); - Assertions.assertEquals("ejcccpxb", model.parameters().get(0).name()); - Assertions.assertEquals("itafjjevptvit", model.parameters().get(0).description()); - Assertions.assertEquals("zqwvkpar", model.parameters().get(0).dataType()); - Assertions.assertEquals(true, model.parameters().get(0).required()); - Assertions.assertEquals(false, model.parameters().get(0).sensitive()); - Assertions.assertEquals("tqc", model.parameters().get(0).designDefaultValue()); - Assertions.assertEquals("mbsudcykg", model.parameters().get(0).defaultValue()); - Assertions.assertEquals("ellfwrmsuxyqbm", model.parameters().get(0).sensitiveDefaultValue()); - Assertions.assertEquals("ubxlpkdsnbqoy", model.parameters().get(0).valueType()); - Assertions.assertEquals(false, model.parameters().get(0).valueSet()); - Assertions.assertEquals("ra", model.parameters().get(0).variable()); + Assertions.assertEquals(4578072156366697994L, model.id()); + Assertions.assertEquals("ryywyfcenkbfxqc", model.name()); + Assertions.assertEquals("ggeciradmxokbutb", model.description()); + Assertions.assertEquals(3556959886109984896L, model.folderId()); + Assertions.assertEquals(4692852653848740418L, model.projectVersion()); + Assertions.assertEquals(668264090121943233L, model.projectId()); + Assertions.assertEquals(525854716862013224L, model.parameters().get(0).id()); + Assertions.assertEquals("ifywxjjylaqhx", model.parameters().get(0).name()); + Assertions.assertEquals("fdarvjhwgkynxlw", model.parameters().get(0).description()); + Assertions.assertEquals("gotdt", model.parameters().get(0).dataType()); + Assertions.assertEquals(false, model.parameters().get(0).required()); + Assertions.assertEquals(true, model.parameters().get(0).sensitive()); + Assertions.assertEquals("lh", model.parameters().get(0).designDefaultValue()); + Assertions.assertEquals("ybfnkylzrignqlwo", model.parameters().get(0).defaultValue()); + Assertions.assertEquals("nbjuaiu", model.parameters().get(0).sensitiveDefaultValue()); + Assertions.assertEquals("mqcbnk", model.parameters().get(0).valueType()); + Assertions.assertEquals(true, model.parameters().get(0).valueSet()); + Assertions.assertEquals("i", model.parameters().get(0).variable()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisParameterTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisParameterTests.java index 4b31fe193a7aa..e299d91ab6ec7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisParameterTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisParameterTests.java @@ -12,41 +12,40 @@ public final class SsisParameterTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisParameter model = BinaryData.fromString( - "{\"id\":3047901925060502269,\"name\":\"dhtct\",\"description\":\"regykjmpad\",\"dataType\":\"joh\",\"required\":false,\"sensitive\":false,\"designDefaultValue\":\"xitydljgrpqua\",\"defaultValue\":\"gjqqbactffxdb\",\"sensitiveDefaultValue\":\"ihumw\",\"valueType\":\"apctgljopizqqihv\",\"valueSet\":true,\"variable\":\"qiqddjynpgomz\"}") + "{\"id\":3304004875475532508,\"name\":\"enpftk\",\"description\":\"bmvxbi\",\"dataType\":\"zghpsotbameir\",\"required\":true,\"sensitive\":true,\"designDefaultValue\":\"svr\",\"defaultValue\":\"hyncppmmwhje\",\"sensitiveDefaultValue\":\"urgipv\",\"valueType\":\"xlepsmck\",\"valueSet\":false,\"variable\":\"xgcqmguv\"}") .toObject(SsisParameter.class); - Assertions.assertEquals(3047901925060502269L, model.id()); - Assertions.assertEquals("dhtct", model.name()); - Assertions.assertEquals("regykjmpad", model.description()); - Assertions.assertEquals("joh", model.dataType()); - Assertions.assertEquals(false, model.required()); - Assertions.assertEquals(false, model.sensitive()); - Assertions.assertEquals("xitydljgrpqua", model.designDefaultValue()); - Assertions.assertEquals("gjqqbactffxdb", model.defaultValue()); - Assertions.assertEquals("ihumw", model.sensitiveDefaultValue()); - Assertions.assertEquals("apctgljopizqqihv", model.valueType()); - Assertions.assertEquals(true, model.valueSet()); - Assertions.assertEquals("qiqddjynpgomz", model.variable()); + Assertions.assertEquals(3304004875475532508L, model.id()); + Assertions.assertEquals("enpftk", model.name()); + Assertions.assertEquals("bmvxbi", model.description()); + Assertions.assertEquals("zghpsotbameir", model.dataType()); + Assertions.assertEquals(true, model.required()); + Assertions.assertEquals(true, model.sensitive()); + Assertions.assertEquals("svr", model.designDefaultValue()); + Assertions.assertEquals("hyncppmmwhje", model.defaultValue()); + Assertions.assertEquals("urgipv", model.sensitiveDefaultValue()); + Assertions.assertEquals("xlepsmck", model.valueType()); + Assertions.assertEquals(false, model.valueSet()); + Assertions.assertEquals("xgcqmguv", model.variable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisParameter model - = new SsisParameter().withId(3047901925060502269L).withName("dhtct").withDescription("regykjmpad") - .withDataType("joh").withRequired(false).withSensitive(false).withDesignDefaultValue("xitydljgrpqua") - .withDefaultValue("gjqqbactffxdb").withSensitiveDefaultValue("ihumw").withValueType("apctgljopizqqihv") - .withValueSet(true).withVariable("qiqddjynpgomz"); + SsisParameter model = new SsisParameter().withId(3304004875475532508L).withName("enpftk") + .withDescription("bmvxbi").withDataType("zghpsotbameir").withRequired(true).withSensitive(true) + .withDesignDefaultValue("svr").withDefaultValue("hyncppmmwhje").withSensitiveDefaultValue("urgipv") + .withValueType("xlepsmck").withValueSet(false).withVariable("xgcqmguv"); model = BinaryData.fromObject(model).toObject(SsisParameter.class); - Assertions.assertEquals(3047901925060502269L, model.id()); - Assertions.assertEquals("dhtct", model.name()); - Assertions.assertEquals("regykjmpad", model.description()); - Assertions.assertEquals("joh", model.dataType()); - Assertions.assertEquals(false, model.required()); - Assertions.assertEquals(false, model.sensitive()); - Assertions.assertEquals("xitydljgrpqua", model.designDefaultValue()); - Assertions.assertEquals("gjqqbactffxdb", model.defaultValue()); - Assertions.assertEquals("ihumw", model.sensitiveDefaultValue()); - Assertions.assertEquals("apctgljopizqqihv", model.valueType()); - Assertions.assertEquals(true, model.valueSet()); - Assertions.assertEquals("qiqddjynpgomz", model.variable()); + Assertions.assertEquals(3304004875475532508L, model.id()); + Assertions.assertEquals("enpftk", model.name()); + Assertions.assertEquals("bmvxbi", model.description()); + Assertions.assertEquals("zghpsotbameir", model.dataType()); + Assertions.assertEquals(true, model.required()); + Assertions.assertEquals(true, model.sensitive()); + Assertions.assertEquals("svr", model.designDefaultValue()); + Assertions.assertEquals("hyncppmmwhje", model.defaultValue()); + Assertions.assertEquals("urgipv", model.sensitiveDefaultValue()); + Assertions.assertEquals("xlepsmck", model.valueType()); + Assertions.assertEquals(false, model.valueSet()); + Assertions.assertEquals("xgcqmguv", model.variable()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisProjectTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisProjectTests.java index 02cbe472f44c9..3768267ffbcfc 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisProjectTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisProjectTests.java @@ -15,76 +15,80 @@ public final class SsisProjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisProject model = BinaryData.fromString( - "{\"type\":\"Project\",\"folderId\":2958352264813914291,\"version\":7021570955207116544,\"environmentRefs\":[{\"id\":9167880799790341338,\"environmentFolderName\":\"b\",\"environmentName\":\"cukboc\",\"referenceType\":\"t\"}],\"parameters\":[{\"id\":5036820104472780225,\"name\":\"pwqieyxjkctyqst\",\"description\":\"m\",\"dataType\":\"epeftmubpl\",\"required\":true,\"sensitive\":false,\"designDefaultValue\":\"lhb\",\"defaultValue\":\"syizeqlctpqno\",\"sensitiveDefaultValue\":\"whgyzwfyfdb\",\"valueType\":\"oelmikdsq\",\"valueSet\":false,\"variable\":\"jjsmvs\"},{\"id\":3473208369725969980,\"name\":\"mwjwsmnwbmac\",\"description\":\"mmriyzyvques\",\"dataType\":\"lcsinbulolxxh\",\"required\":false,\"sensitive\":false,\"designDefaultValue\":\"aizvk\",\"defaultValue\":\"qpw\",\"sensitiveDefaultValue\":\"dsjqcqy\",\"valueType\":\"rtfdlgp\",\"valueSet\":false,\"variable\":\"likalbcyuwahw\"},{\"id\":3859919096178828653,\"name\":\"idzcephn\",\"description\":\"u\",\"dataType\":\"y\",\"required\":false,\"sensitive\":false,\"designDefaultValue\":\"mrwpe\",\"defaultValue\":\"prjbpertjpairpw\",\"sensitiveDefaultValue\":\"cgiwsywpejtvqop\",\"valueType\":\"rsergiuztqefzy\",\"valueSet\":true,\"variable\":\"dmcbc\"},{\"id\":3071010166095133135,\"name\":\"huepikwcxoasgukq\",\"description\":\"iy\",\"dataType\":\"fvkiw\",\"required\":false,\"sensitive\":true,\"designDefaultValue\":\"tcctirgyu\",\"defaultValue\":\"x\",\"sensitiveDefaultValue\":\"hdmcgvjbrybfa\",\"valueType\":\"hkoqcudnwmoyhdpj\",\"valueSet\":true,\"variable\":\"zcbjfpxoy\"}],\"id\":2333701757231657721,\"name\":\"iqw\",\"description\":\"xyxf\"}") + "{\"type\":\"Project\",\"folderId\":4812651773635303684,\"version\":229346380706684388,\"environmentRefs\":[{\"id\":6689352297389894599,\"environmentFolderName\":\"hhaq\",\"environmentName\":\"tuecmguk\",\"referenceType\":\"dpuowlcexkr\"},{\"id\":7044331808895170259,\"environmentFolderName\":\"qnbs\",\"environmentName\":\"jcmtcidcab\",\"referenceType\":\"xhcxct\"},{\"id\":3640487153181540727,\"environmentFolderName\":\"ftforylxaknwkjzv\",\"environmentName\":\"symtupyjt\",\"referenceType\":\"xzwdsnqhyefnakd\"}],\"parameters\":[{\"id\":8635897262903643245,\"name\":\"lulytjx\",\"description\":\"wtittlnv\",\"dataType\":\"vupuplugulynv\",\"required\":false,\"sensitive\":false,\"designDefaultValue\":\"dapydsfpz\",\"defaultValue\":\"bsilbnrucqehyrn\",\"sensitiveDefaultValue\":\"jrgfbmpszwkbcstz\",\"valueType\":\"bgaesm\",\"valueSet\":true,\"variable\":\"xrwqt\"},{\"id\":1353396779166802372,\"name\":\"takx\",\"description\":\"lkgjhomywlypghhu\",\"dataType\":\"qyfvgpqwg\",\"required\":false,\"sensitive\":true,\"designDefaultValue\":\"skmbuihtqfvyqmm\",\"defaultValue\":\"uguvlieegjnqwh\",\"sensitiveDefaultValue\":\"o\",\"valueType\":\"ehjscgqcrwaucft\",\"valueSet\":false,\"variable\":\"hjxdlmuhf\"},{\"id\":8128896989412760226,\"name\":\"jyqmpmsknaxr\",\"description\":\"jwqufudpypboql\",\"dataType\":\"xfpwmajvwfijf\",\"required\":false,\"sensitive\":true,\"designDefaultValue\":\"vhms\",\"defaultValue\":\"ihddnbwl\",\"sensitiveDefaultValue\":\"ntdde\",\"valueType\":\"xyiwuzpsvcmz\",\"valueSet\":false,\"variable\":\"yyysqnwnl\"}],\"id\":4034013017830873122,\"name\":\"dzkfthsyd\",\"description\":\"dbzzetfgkz\"}") .toObject(SsisProject.class); - Assertions.assertEquals(2333701757231657721L, model.id()); - Assertions.assertEquals("iqw", model.name()); - Assertions.assertEquals("xyxf", model.description()); - Assertions.assertEquals(2958352264813914291L, model.folderId()); - Assertions.assertEquals(7021570955207116544L, model.version()); - Assertions.assertEquals(9167880799790341338L, model.environmentRefs().get(0).id()); - Assertions.assertEquals("b", model.environmentRefs().get(0).environmentFolderName()); - Assertions.assertEquals("cukboc", model.environmentRefs().get(0).environmentName()); - Assertions.assertEquals("t", model.environmentRefs().get(0).referenceType()); - Assertions.assertEquals(5036820104472780225L, model.parameters().get(0).id()); - Assertions.assertEquals("pwqieyxjkctyqst", model.parameters().get(0).name()); - Assertions.assertEquals("m", model.parameters().get(0).description()); - Assertions.assertEquals("epeftmubpl", model.parameters().get(0).dataType()); - Assertions.assertEquals(true, model.parameters().get(0).required()); + Assertions.assertEquals(4034013017830873122L, model.id()); + Assertions.assertEquals("dzkfthsyd", model.name()); + Assertions.assertEquals("dbzzetfgkz", model.description()); + Assertions.assertEquals(4812651773635303684L, model.folderId()); + Assertions.assertEquals(229346380706684388L, model.version()); + Assertions.assertEquals(6689352297389894599L, model.environmentRefs().get(0).id()); + Assertions.assertEquals("hhaq", model.environmentRefs().get(0).environmentFolderName()); + Assertions.assertEquals("tuecmguk", model.environmentRefs().get(0).environmentName()); + Assertions.assertEquals("dpuowlcexkr", model.environmentRefs().get(0).referenceType()); + Assertions.assertEquals(8635897262903643245L, model.parameters().get(0).id()); + Assertions.assertEquals("lulytjx", model.parameters().get(0).name()); + Assertions.assertEquals("wtittlnv", model.parameters().get(0).description()); + Assertions.assertEquals("vupuplugulynv", model.parameters().get(0).dataType()); + Assertions.assertEquals(false, model.parameters().get(0).required()); Assertions.assertEquals(false, model.parameters().get(0).sensitive()); - Assertions.assertEquals("lhb", model.parameters().get(0).designDefaultValue()); - Assertions.assertEquals("syizeqlctpqno", model.parameters().get(0).defaultValue()); - Assertions.assertEquals("whgyzwfyfdb", model.parameters().get(0).sensitiveDefaultValue()); - Assertions.assertEquals("oelmikdsq", model.parameters().get(0).valueType()); - Assertions.assertEquals(false, model.parameters().get(0).valueSet()); - Assertions.assertEquals("jjsmvs", model.parameters().get(0).variable()); + Assertions.assertEquals("dapydsfpz", model.parameters().get(0).designDefaultValue()); + Assertions.assertEquals("bsilbnrucqehyrn", model.parameters().get(0).defaultValue()); + Assertions.assertEquals("jrgfbmpszwkbcstz", model.parameters().get(0).sensitiveDefaultValue()); + Assertions.assertEquals("bgaesm", model.parameters().get(0).valueType()); + Assertions.assertEquals(true, model.parameters().get(0).valueSet()); + Assertions.assertEquals("xrwqt", model.parameters().get(0).variable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisProject model = new SsisProject().withId(2333701757231657721L).withName("iqw").withDescription("xyxf") - .withFolderId(2958352264813914291L).withVersion(7021570955207116544L) - .withEnvironmentRefs(Arrays.asList(new SsisEnvironmentReference().withId(9167880799790341338L) - .withEnvironmentFolderName("b").withEnvironmentName("cukboc").withReferenceType("t"))) + SsisProject model = new SsisProject().withId(4034013017830873122L).withName("dzkfthsyd") + .withDescription("dbzzetfgkz").withFolderId(4812651773635303684L).withVersion(229346380706684388L) + .withEnvironmentRefs(Arrays.asList( + new SsisEnvironmentReference().withId(6689352297389894599L).withEnvironmentFolderName("hhaq") + .withEnvironmentName("tuecmguk").withReferenceType("dpuowlcexkr"), + new SsisEnvironmentReference().withId(7044331808895170259L).withEnvironmentFolderName("qnbs") + .withEnvironmentName("jcmtcidcab").withReferenceType("xhcxct"), + new SsisEnvironmentReference().withId(3640487153181540727L) + .withEnvironmentFolderName("ftforylxaknwkjzv").withEnvironmentName("symtupyjt") + .withReferenceType("xzwdsnqhyefnakd"))) .withParameters(Arrays.asList( - new SsisParameter().withId(5036820104472780225L).withName("pwqieyxjkctyqst").withDescription("m") - .withDataType("epeftmubpl").withRequired(true).withSensitive(false).withDesignDefaultValue("lhb") - .withDefaultValue("syizeqlctpqno").withSensitiveDefaultValue("whgyzwfyfdb") - .withValueType("oelmikdsq").withValueSet(false).withVariable("jjsmvs"), - new SsisParameter().withId(3473208369725969980L).withName("mwjwsmnwbmac") - .withDescription("mmriyzyvques").withDataType("lcsinbulolxxh").withRequired(false) - .withSensitive(false).withDesignDefaultValue("aizvk").withDefaultValue("qpw") - .withSensitiveDefaultValue("dsjqcqy").withValueType("rtfdlgp").withValueSet(false) - .withVariable("likalbcyuwahw"), - new SsisParameter().withId(3859919096178828653L).withName("idzcephn").withDescription("u") - .withDataType("y").withRequired(false).withSensitive(false).withDesignDefaultValue("mrwpe") - .withDefaultValue("prjbpertjpairpw").withSensitiveDefaultValue("cgiwsywpejtvqop") - .withValueType("rsergiuztqefzy").withValueSet(true).withVariable("dmcbc"), - new SsisParameter().withId(3071010166095133135L).withName("huepikwcxoasgukq").withDescription("iy") - .withDataType("fvkiw").withRequired(false).withSensitive(true).withDesignDefaultValue("tcctirgyu") - .withDefaultValue("x").withSensitiveDefaultValue("hdmcgvjbrybfa").withValueType("hkoqcudnwmoyhdpj") - .withValueSet(true).withVariable("zcbjfpxoy"))); + new SsisParameter().withId(8635897262903643245L).withName("lulytjx").withDescription("wtittlnv") + .withDataType("vupuplugulynv").withRequired(false).withSensitive(false) + .withDesignDefaultValue("dapydsfpz").withDefaultValue("bsilbnrucqehyrn") + .withSensitiveDefaultValue("jrgfbmpszwkbcstz").withValueType("bgaesm").withValueSet(true) + .withVariable("xrwqt"), + new SsisParameter().withId(1353396779166802372L).withName("takx").withDescription("lkgjhomywlypghhu") + .withDataType("qyfvgpqwg").withRequired(false).withSensitive(true) + .withDesignDefaultValue("skmbuihtqfvyqmm").withDefaultValue("uguvlieegjnqwh") + .withSensitiveDefaultValue("o").withValueType("ehjscgqcrwaucft").withValueSet(false) + .withVariable("hjxdlmuhf"), + new SsisParameter().withId(8128896989412760226L).withName("jyqmpmsknaxr") + .withDescription("jwqufudpypboql").withDataType("xfpwmajvwfijf").withRequired(false) + .withSensitive(true).withDesignDefaultValue("vhms").withDefaultValue("ihddnbwl") + .withSensitiveDefaultValue("ntdde").withValueType("xyiwuzpsvcmz").withValueSet(false) + .withVariable("yyysqnwnl"))); model = BinaryData.fromObject(model).toObject(SsisProject.class); - Assertions.assertEquals(2333701757231657721L, model.id()); - Assertions.assertEquals("iqw", model.name()); - Assertions.assertEquals("xyxf", model.description()); - Assertions.assertEquals(2958352264813914291L, model.folderId()); - Assertions.assertEquals(7021570955207116544L, model.version()); - Assertions.assertEquals(9167880799790341338L, model.environmentRefs().get(0).id()); - Assertions.assertEquals("b", model.environmentRefs().get(0).environmentFolderName()); - Assertions.assertEquals("cukboc", model.environmentRefs().get(0).environmentName()); - Assertions.assertEquals("t", model.environmentRefs().get(0).referenceType()); - Assertions.assertEquals(5036820104472780225L, model.parameters().get(0).id()); - Assertions.assertEquals("pwqieyxjkctyqst", model.parameters().get(0).name()); - Assertions.assertEquals("m", model.parameters().get(0).description()); - Assertions.assertEquals("epeftmubpl", model.parameters().get(0).dataType()); - Assertions.assertEquals(true, model.parameters().get(0).required()); + Assertions.assertEquals(4034013017830873122L, model.id()); + Assertions.assertEquals("dzkfthsyd", model.name()); + Assertions.assertEquals("dbzzetfgkz", model.description()); + Assertions.assertEquals(4812651773635303684L, model.folderId()); + Assertions.assertEquals(229346380706684388L, model.version()); + Assertions.assertEquals(6689352297389894599L, model.environmentRefs().get(0).id()); + Assertions.assertEquals("hhaq", model.environmentRefs().get(0).environmentFolderName()); + Assertions.assertEquals("tuecmguk", model.environmentRefs().get(0).environmentName()); + Assertions.assertEquals("dpuowlcexkr", model.environmentRefs().get(0).referenceType()); + Assertions.assertEquals(8635897262903643245L, model.parameters().get(0).id()); + Assertions.assertEquals("lulytjx", model.parameters().get(0).name()); + Assertions.assertEquals("wtittlnv", model.parameters().get(0).description()); + Assertions.assertEquals("vupuplugulynv", model.parameters().get(0).dataType()); + Assertions.assertEquals(false, model.parameters().get(0).required()); Assertions.assertEquals(false, model.parameters().get(0).sensitive()); - Assertions.assertEquals("lhb", model.parameters().get(0).designDefaultValue()); - Assertions.assertEquals("syizeqlctpqno", model.parameters().get(0).defaultValue()); - Assertions.assertEquals("whgyzwfyfdb", model.parameters().get(0).sensitiveDefaultValue()); - Assertions.assertEquals("oelmikdsq", model.parameters().get(0).valueType()); - Assertions.assertEquals(false, model.parameters().get(0).valueSet()); - Assertions.assertEquals("jjsmvs", model.parameters().get(0).variable()); + Assertions.assertEquals("dapydsfpz", model.parameters().get(0).designDefaultValue()); + Assertions.assertEquals("bsilbnrucqehyrn", model.parameters().get(0).defaultValue()); + Assertions.assertEquals("jrgfbmpszwkbcstz", model.parameters().get(0).sensitiveDefaultValue()); + Assertions.assertEquals("bgaesm", model.parameters().get(0).valueType()); + Assertions.assertEquals(true, model.parameters().get(0).valueSet()); + Assertions.assertEquals("xrwqt", model.parameters().get(0).variable()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPropertyOverrideTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPropertyOverrideTests.java index c712c51dff70c..2053990a4e2fe 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPropertyOverrideTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisPropertyOverrideTests.java @@ -11,15 +11,15 @@ public final class SsisPropertyOverrideTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SsisPropertyOverride model = BinaryData.fromString("{\"value\":\"dataxxq\",\"isSensitive\":false}") + SsisPropertyOverride model = BinaryData.fromString("{\"value\":\"databpjzoyzy\",\"isSensitive\":true}") .toObject(SsisPropertyOverride.class); - Assertions.assertEquals(false, model.isSensitive()); + Assertions.assertEquals(true, model.isSensitive()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisPropertyOverride model = new SsisPropertyOverride().withValue("dataxxq").withIsSensitive(false); + SsisPropertyOverride model = new SsisPropertyOverride().withValue("databpjzoyzy").withIsSensitive(true); model = BinaryData.fromObject(model).toObject(SsisPropertyOverride.class); - Assertions.assertEquals(false, model.isSensitive()); + Assertions.assertEquals(true, model.isSensitive()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisVariableTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisVariableTests.java index 896bf745b875d..b7b0ca8f30418 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisVariableTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SsisVariableTests.java @@ -12,29 +12,29 @@ public final class SsisVariableTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SsisVariable model = BinaryData.fromString( - "{\"id\":6310282971839998498,\"name\":\"qdapydsfpzpjbs\",\"description\":\"bnrucq\",\"dataType\":\"yrnzmjrgfbmps\",\"sensitive\":true,\"value\":\"cstzuwbgaesmaxd\",\"sensitiveValue\":\"rwqtb\"}") + "{\"id\":1109572290980895504,\"name\":\"hhwpufrspreyilq\",\"description\":\"kxkteoykqrqtxqog\",\"dataType\":\"dimnacklyrbv\",\"sensitive\":true,\"value\":\"expnphtqwfp\",\"sensitiveValue\":\"sbcxqiy\"}") .toObject(SsisVariable.class); - Assertions.assertEquals(6310282971839998498L, model.id()); - Assertions.assertEquals("qdapydsfpzpjbs", model.name()); - Assertions.assertEquals("bnrucq", model.description()); - Assertions.assertEquals("yrnzmjrgfbmps", model.dataType()); + Assertions.assertEquals(1109572290980895504L, model.id()); + Assertions.assertEquals("hhwpufrspreyilq", model.name()); + Assertions.assertEquals("kxkteoykqrqtxqog", model.description()); + Assertions.assertEquals("dimnacklyrbv", model.dataType()); Assertions.assertEquals(true, model.sensitive()); - Assertions.assertEquals("cstzuwbgaesmaxd", model.value()); - Assertions.assertEquals("rwqtb", model.sensitiveValue()); + Assertions.assertEquals("expnphtqwfp", model.value()); + Assertions.assertEquals("sbcxqiy", model.sensitiveValue()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SsisVariable model = new SsisVariable().withId(6310282971839998498L).withName("qdapydsfpzpjbs") - .withDescription("bnrucq").withDataType("yrnzmjrgfbmps").withSensitive(true).withValue("cstzuwbgaesmaxd") - .withSensitiveValue("rwqtb"); + SsisVariable model = new SsisVariable().withId(1109572290980895504L).withName("hhwpufrspreyilq") + .withDescription("kxkteoykqrqtxqog").withDataType("dimnacklyrbv").withSensitive(true) + .withValue("expnphtqwfp").withSensitiveValue("sbcxqiy"); model = BinaryData.fromObject(model).toObject(SsisVariable.class); - Assertions.assertEquals(6310282971839998498L, model.id()); - Assertions.assertEquals("qdapydsfpzpjbs", model.name()); - Assertions.assertEquals("bnrucq", model.description()); - Assertions.assertEquals("yrnzmjrgfbmps", model.dataType()); + Assertions.assertEquals(1109572290980895504L, model.id()); + Assertions.assertEquals("hhwpufrspreyilq", model.name()); + Assertions.assertEquals("kxkteoykqrqtxqog", model.description()); + Assertions.assertEquals("dimnacklyrbv", model.dataType()); Assertions.assertEquals(true, model.sensitive()); - Assertions.assertEquals("cstzuwbgaesmaxd", model.value()); - Assertions.assertEquals("rwqtb", model.sensitiveValue()); + Assertions.assertEquals("expnphtqwfp", model.value()); + Assertions.assertEquals("sbcxqiy", model.sensitiveValue()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StagingSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StagingSettingsTests.java index c40e236206368..b4306fcee9cfb 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StagingSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StagingSettingsTests.java @@ -15,20 +15,19 @@ public final class StagingSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { StagingSettings model = BinaryData.fromString( - "{\"linkedServiceName\":{\"referenceName\":\"h\",\"parameters\":{\"unicgrxce\":\"datatgwerbpobvj\",\"mztrnniarje\":\"datavvmdtkllqhznutrx\",\"xiqfoqwesqykqfs\":\"datajh\",\"wmzgvnojgmobkali\":\"datarlsaipshheta\"}},\"path\":\"dataikkehpdssvlubd\",\"enableCompression\":\"dataowxsxbxd\",\"\":{\"tghmtb\":\"dataxurcekcqmjqqau\",\"shlhe\":\"datafkcnkghkrbi\"}}") + "{\"linkedServiceName\":{\"referenceName\":\"cjkarggvyuewg\",\"parameters\":{\"li\":\"datalvxwlqlugnbudjy\",\"qdoxooxuaufqoo\":\"datatgtlansykvlxsyc\"}},\"path\":\"dataxctkveqvpedwmhqc\",\"enableCompression\":\"dataery\",\"\":{\"clxvaovssibnvq\":\"datayqxeyzqnupsi\",\"q\":\"datavi\"}}") .toObject(StagingSettings.class); - Assertions.assertEquals("h", model.linkedServiceName().referenceName()); + Assertions.assertEquals("cjkarggvyuewg", model.linkedServiceName().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { StagingSettings model = new StagingSettings() - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("h") - .withParameters(mapOf("unicgrxce", "datatgwerbpobvj", "mztrnniarje", "datavvmdtkllqhznutrx", - "xiqfoqwesqykqfs", "datajh", "wmzgvnojgmobkali", "datarlsaipshheta"))) - .withPath("dataikkehpdssvlubd").withEnableCompression("dataowxsxbxd").withAdditionalProperties(mapOf()); + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("cjkarggvyuewg") + .withParameters(mapOf("li", "datalvxwlqlugnbudjy", "qdoxooxuaufqoo", "datatgtlansykvlxsyc"))) + .withPath("dataxctkveqvpedwmhqc").withEnableCompression("dataery").withAdditionalProperties(mapOf()); model = BinaryData.fromObject(model).toObject(StagingSettings.class); - Assertions.assertEquals("h", model.linkedServiceName().referenceName()); + Assertions.assertEquals("cjkarggvyuewg", model.linkedServiceName().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreReadSettingsTests.java index 08b13b51372eb..8ad619b184d9a 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreReadSettingsTests.java @@ -13,14 +13,15 @@ public final class StoreReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { StoreReadSettings model = BinaryData.fromString( - "{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"dataukqurrtcf\",\"disableMetricsCollection\":\"datat\",\"\":{\"leghozs\":\"datar\",\"ozryyyvlxmsp\":\"datajjsvy\",\"quazo\":\"dataqafsxv\",\"sdolodfodokhaog\":\"datalxxksputizpvvihg\"}}") + "{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datafotaaqyx\",\"disableMetricsCollection\":\"dataoabcoxqaavjkre\",\"\":{\"ivianklqclftp\":\"datasviysbvo\"}}") .toObject(StoreReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - StoreReadSettings model = new StoreReadSettings().withMaxConcurrentConnections("dataukqurrtcf") - .withDisableMetricsCollection("datat").withAdditionalProperties(mapOf("type", "StoreReadSettings")); + StoreReadSettings model = new StoreReadSettings().withMaxConcurrentConnections("datafotaaqyx") + .withDisableMetricsCollection("dataoabcoxqaavjkre") + .withAdditionalProperties(mapOf("type", "StoreReadSettings")); model = BinaryData.fromObject(model).toObject(StoreReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreWriteSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreWriteSettingsTests.java index 0d6b60c323c85..fc7d4d1aa684e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreWriteSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/StoreWriteSettingsTests.java @@ -15,19 +15,15 @@ public final class StoreWriteSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { StoreWriteSettings model = BinaryData.fromString( - "{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datakoxqbozezx\",\"disableMetricsCollection\":\"datainrguk\",\"copyBehavior\":\"databov\",\"metadata\":[{\"name\":\"dataqlqufkrnrbnjkco\",\"value\":\"datazqlyputawdmdikuf\"},{\"name\":\"datav\",\"value\":\"dataujzofyldxk\"},{\"name\":\"datavfojcvnhpebuiy\",\"value\":\"dataysgq\"},{\"name\":\"databeauvldb\",\"value\":\"datamguifqjtoxz\"}],\"\":{\"aii\":\"datajpzauugdarfumit\",\"wbgmjrvrsq\":\"datamokfdybv\",\"cttvxkxgffpvv\":\"datajcozrwry\"}}") + "{\"type\":\"StoreWriteSettings\",\"maxConcurrentConnections\":\"datagcgefay\",\"disableMetricsCollection\":\"datavgotbjnxozi\",\"copyBehavior\":\"dataxnpov\",\"metadata\":[{\"name\":\"datauvmsgdis\",\"value\":\"datanxthu\"}],\"\":{\"dbqeahgsibldxya\":\"datavokxuyhhrdi\",\"h\":\"datadaaznzaxz\"}}") .toObject(StoreWriteSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - StoreWriteSettings model = new StoreWriteSettings().withMaxConcurrentConnections("datakoxqbozezx") - .withDisableMetricsCollection("datainrguk").withCopyBehavior("databov") - .withMetadata( - Arrays.asList(new MetadataItem().withName("dataqlqufkrnrbnjkco").withValue("datazqlyputawdmdikuf"), - new MetadataItem().withName("datav").withValue("dataujzofyldxk"), - new MetadataItem().withName("datavfojcvnhpebuiy").withValue("dataysgq"), - new MetadataItem().withName("databeauvldb").withValue("datamguifqjtoxz"))) + StoreWriteSettings model = new StoreWriteSettings().withMaxConcurrentConnections("datagcgefay") + .withDisableMetricsCollection("datavgotbjnxozi").withCopyBehavior("dataxnpov") + .withMetadata(Arrays.asList(new MetadataItem().withName("datauvmsgdis").withValue("datanxthu"))) .withAdditionalProperties(mapOf("type", "StoreWriteSettings")); model = BinaryData.fromObject(model).toObject(StoreWriteSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTests.java index e36c43f9228b7..1863690c63d0c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTests.java @@ -23,209 +23,193 @@ public final class SwitchActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SwitchActivity model = BinaryData.fromString( - "{\"type\":\"Switch\",\"typeProperties\":{\"on\":{\"value\":\"cguwyuzhkefownc\"},\"cases\":[{\"value\":\"woiqsrqebjgo\",\"activities\":[{\"type\":\"Activity\",\"name\":\"cahdagchk\",\"description\":\"f\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"q\",\"dependencyConditions\":[]},{\"activity\":\"bslwxcf\",\"dependencyConditions\":[]},{\"activity\":\"vedxyeba\",\"dependencyConditions\":[]},{\"activity\":\"wnmnxppgfep\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"djva\",\"value\":\"dataae\"},{\"name\":\"oqknz\",\"value\":\"datanvvkfbmrppjf\"},{\"name\":\"eabgpw\",\"value\":\"datas\"}],\"\":{\"n\":\"datai\",\"telimqxwih\":\"datavdjmvzcycg\",\"hz\":\"datapyexjrguziglr\",\"isklotwnppstpq\":\"datamrvgcbf\"}}]},{\"value\":\"deawolhl\",\"activities\":[{\"type\":\"Activity\",\"name\":\"eznbmhqylrsywi\",\"description\":\"oqtvx\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"de\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"p\",\"value\":\"dataqkcikfeshxo\"},{\"name\":\"tvkxpsxlpypzgdet\",\"value\":\"datad\"}],\"\":{\"pocrskkraapcz\":\"datahuysut\",\"jigtqyzoc\":\"dataziif\",\"lciooxybmktbwdfj\":\"datayywc\"}},{\"type\":\"Activity\",\"name\":\"epycpw\",\"description\":\"oku\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"gcnicknsbbccbqx\",\"dependencyConditions\":[]},{\"activity\":\"ojvejxhfeo\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"tfyjcenkidl\",\"value\":\"datamlxhzwyyy\"},{\"name\":\"rzbmlhg\",\"value\":\"datatkthevodddne\"},{\"name\":\"wsyx\",\"value\":\"datafdjftcr\"},{\"name\":\"ohdifbhtxtcqjg\",\"value\":\"datad\"}],\"\":{\"aq\":\"datalpuurjxkp\",\"vuvh\":\"datanejufljqzbixlzaa\",\"bneepfjibtsp\":\"dataerjrcxyxepl\",\"eigywj\":\"dataiwfqj\"}},{\"type\":\"Activity\",\"name\":\"qpgncscw\",\"description\":\"fdq\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"nutyjdcvnanej\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"llec\",\"value\":\"datauofgyhk\"},{\"name\":\"tofxkelwvcyprpo\",\"value\":\"datanqvuftki\"}],\"\":{\"kigoch\":\"datacmp\",\"bpwwo\":\"datazcgsapklfnstv\",\"w\":\"datakzesfdrsgfpdsh\",\"mkcjiyoc\":\"datavepmttrfu\"}}]},{\"value\":\"idukrjjfn\",\"activities\":[{\"type\":\"Activity\",\"name\":\"qalwjcqbnvbzem\",\"description\":\"fyvjdcehlgryvku\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"porioki\",\"dependencyConditions\":[]},{\"activity\":\"x\",\"dependencyConditions\":[]},{\"activity\":\"fftt\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"nqlggtrjzi\",\"value\":\"dataxzbu\"},{\"name\":\"cinjej\",\"value\":\"datainlysguladd\"},{\"name\":\"jzenagmhhmgtbqzf\",\"value\":\"datampgibmngb\"}],\"\":{\"loldxmgyjvjyxue\":\"dataprbwsn\",\"lzbmdn\":\"dataqcbgbscgxaktkbry\",\"ymsbhd\":\"dataajggzuyokc\",\"jfuwt\":\"dataxzaobsenxg\"}},{\"type\":\"Activity\",\"name\":\"ukbpwwfeixm\",\"description\":\"uwpivsltly\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"cjrdqcmsrzrcd\",\"dependencyConditions\":[]},{\"activity\":\"lzgaopt\",\"dependencyConditions\":[]},{\"activity\":\"qfgqccondaoptru\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"ezlloflcilrafkr\",\"value\":\"datavdwknymqzmu\"},{\"name\":\"nuvtgjgp\",\"value\":\"datavdjinuos\"}],\"\":{\"rbjgdvwamcvcne\":\"databzzfevwc\",\"kmenawsvximqkuy\":\"datak\",\"wcomqyqvyw\":\"datalzxgsuuapktfvemw\",\"jxpkpsqkvcsn\":\"datahbyklwc\"}},{\"type\":\"Activity\",\"name\":\"ey\",\"description\":\"hyqouzzpgnldzsnv\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"shgfdvwshc\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"qnfsjnrfpzlvae\",\"value\":\"datajnskekhmo\"}],\"\":{\"qfazsiizcwhax\":\"datarkd\",\"e\":\"datahaet\",\"yajyiwvqlr\":\"datafjlismacac\"}},{\"type\":\"Activity\",\"name\":\"obvkg\",\"description\":\"jbjavnk\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"zf\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"ukuypyeofsa\",\"value\":\"dataabbxkldtwrrycl\"},{\"name\":\"rld\",\"value\":\"databguaxilcdbu\"},{\"name\":\"fwlp\",\"value\":\"datapxyrfkslg\"}],\"\":{\"cmkdhgpzqibqilc\":\"datad\",\"emex\":\"datatmu\"}}]}],\"defaultActivities\":[{\"type\":\"Activity\",\"name\":\"xaawentkokndjwpx\",\"description\":\"n\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"bv\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Succeeded\",\"Completed\"],\"\":{\"eayowzp\":\"datalgrkfwof\",\"thpsycasx\":\"dataxpeodbmuzpd\"}},{\"activity\":\"hi\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Skipped\",\"Succeeded\"],\"\":{\"fp\":\"datafjgjmys\",\"yfzadeui\":\"datarzqagmcivsqawia\",\"bsuxgnwuykulo\":\"datakfvdjgwzakqgab\"}},{\"activity\":\"doilhrxjiwjivyor\",\"dependencyConditions\":[\"Failed\"],\"\":{\"lau\":\"datawnb\",\"ah\":\"dataazyrisciokbvft\",\"dlmag\":\"datansllfkcroviim\",\"gfx\":\"dataw\"}},{\"activity\":\"endfpdoxtif\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"l\":\"dataktklo\",\"jgsvllefliriqb\":\"datasbystzn\",\"owuzasdztufmujad\":\"datatvkj\"}}],\"userProperties\":[{\"name\":\"dntunbpeeprmebv\",\"value\":\"datamaacrqrovbozjce\"},{\"name\":\"mrcxugatv\",\"value\":\"dataxyvxdbuzdphog\"}],\"\":{\"lrasd\":\"datamgu\",\"vygolzjnjkb\":\"datarfoz\",\"ysvca\":\"datafc\"}},{\"type\":\"Activity\",\"name\":\"rkrjmzqnbwnl\",\"description\":\"z\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"utcoqclypbr\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Succeeded\",\"Failed\"],\"\":{\"it\":\"datarj\",\"ict\":\"datapebuvxxloqrdefhb\"}},{\"activity\":\"dwg\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"wvmw\":\"dataesxzukl\"}},{\"activity\":\"uqchcooty\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Skipped\",\"Skipped\"],\"\":{\"ysk\":\"dataewvc\"}},{\"activity\":\"kvkwdtbvy\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Skipped\",\"Skipped\"],\"\":{\"akglygeuoolywjvd\":\"datatprxtfwvngwcsn\",\"rjtyhth\":\"datajlgwzbrggntqp\",\"vkhkubpojhdxcha\":\"datacpzdn\",\"w\":\"datag\"}}],\"userProperties\":[{\"name\":\"nwxolfhiq\",\"value\":\"dataiulfxgzyr\"},{\"name\":\"qux\",\"value\":\"datatekixouhca\"},{\"name\":\"ozsodp\",\"value\":\"datasqcwjxatghuixc\"},{\"name\":\"ycifdr\",\"value\":\"datarywribmeuukkonw\"}],\"\":{\"bwmiap\":\"datamhpjmnxlfkmdwzgb\",\"p\":\"datamrpbmxmxshfh\",\"oqnytuc\":\"dataqimjnxpfvxyt\"}}]},\"name\":\"zyr\",\"description\":\"de\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"b\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\",\"Skipped\",\"Skipped\"],\"\":{\"gxrozcfcxks\":\"datahlhludimqjc\",\"bteakdrh\":\"dataj\"}}],\"userProperties\":[{\"name\":\"pmnxva\",\"value\":\"dataq\"},{\"name\":\"cnlphlkx\",\"value\":\"dataanlyccdmkp\"}],\"\":{\"qzdedizdmwndnsg\":\"dataa\"}}") + "{\"type\":\"Switch\",\"typeProperties\":{\"on\":{\"value\":\"suuapktfvemwfwc\"},\"cases\":[{\"value\":\"qv\",\"activities\":[{\"type\":\"Activity\",\"name\":\"hbyklwc\",\"description\":\"xpkpsqk\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"bkhyqouzz\",\"dependencyConditions\":[]},{\"activity\":\"gnldz\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"tlhnm\",\"value\":\"datads\"},{\"name\":\"gfdvwshcc\",\"value\":\"datayqn\"},{\"name\":\"sjnrfpzlvaeo\",\"value\":\"datanskekhmomv\"}],\"\":{\"aetyeafjlismaca\":\"datadsqfazsiizcwhaxj\",\"kgfpjbj\":\"datadyajyiwvqlrzob\",\"rzfh\":\"datavnkyqrjb\"}},{\"type\":\"Activity\",\"name\":\"hukuypyeo\",\"description\":\"aeabbxkldtw\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ldgbgua\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"cdbudfwlpgpxyrfk\",\"value\":\"datalgpladqc\"}],\"\":{\"emex\":\"datahgpzqibqilcntmu\",\"djwp\":\"datarjxaawentkok\",\"jqfwxicbvwnnvt\":\"dataea\",\"yowzptxpe\":\"databclgrkfwofwe\"}}]},{\"value\":\"bmuzpdjt\",\"activities\":[{\"type\":\"Activity\",\"name\":\"ycasxuhi\",\"description\":\"mmiipf\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"nfpxrzqagmcivs\",\"dependencyConditions\":[]},{\"activity\":\"awiabyfzadeu\",\"dependencyConditions\":[]},{\"activity\":\"tkfvdjgw\",\"dependencyConditions\":[]},{\"activity\":\"akqgabrbsuxgn\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"kulozdoilhrxji\",\"value\":\"datajiv\"},{\"name\":\"orqlkycwnb\",\"value\":\"datalau\"}],\"\":{\"bvftqahjnsllfkcr\":\"datayriscio\",\"fxtendfp\":\"dataviimhdlmagdwi\",\"tklojlgsbystznwj\":\"dataoxtifosxxk\",\"ptvkjdowuzasd\":\"datasvllefliriq\"}},{\"type\":\"Activity\",\"name\":\"tufmujadippdntun\",\"description\":\"eeprmebvxmaacr\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"jcesm\",\"dependencyConditions\":[]},{\"activity\":\"cxugatvjxyvxd\",\"dependencyConditions\":[]},{\"activity\":\"uzdphogmrcmgu\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"asdrrfozz\",\"value\":\"dataygolz\"},{\"name\":\"njkbmfcrysvcab\",\"value\":\"datak\"},{\"name\":\"jmzqnbwnlo\",\"value\":\"dataz\"}],\"\":{\"njorpcrg\":\"datapvdxutcoqclypb\"}}]}],\"defaultActivities\":[{\"type\":\"Activity\",\"name\":\"it\",\"description\":\"ebuvxxl\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"bzictf\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Succeeded\",\"Skipped\"],\"\":{\"wvmw\":\"dataesxzukl\"}}],\"userProperties\":[{\"name\":\"chcootyscar\",\"value\":\"datamhiewvcpyskh\"},{\"name\":\"vkw\",\"value\":\"datatbvyclg\"},{\"name\":\"zbyxtprxt\",\"value\":\"datawvng\"},{\"name\":\"csno\",\"value\":\"datakglygeuo\"}],\"\":{\"ggntqptrjtyhth\":\"datawjvdrjlgwzb\",\"vkhkubpojhdxcha\":\"datacpzdn\",\"w\":\"datag\",\"iulfxgzyr\":\"datavrnwxolfhiq\"}},{\"type\":\"Activity\",\"name\":\"qux\",\"description\":\"ekixouhcatozs\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"wjxatghuixczycif\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Succeeded\",\"Completed\"],\"\":{\"euukko\":\"datab\",\"wzgb\":\"datawtucmhpjmnxlfkm\",\"mrpbmxmxshfh\":\"databwmiap\"}},{\"activity\":\"p\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Failed\"],\"\":{\"vxytmoqnytucuzy\":\"datap\",\"e\":\"dataigdebsinsoybe\",\"mqjcagxrozcfcxk\":\"datarpouhlhlud\",\"kgepmnxvahqvc\":\"datahjxbteakdr\"}}],\"userProperties\":[{\"name\":\"hlkx\",\"value\":\"dataanlyccdmkp\"}],\"\":{\"qzdedizdmwndnsg\":\"dataa\"}},{\"type\":\"Activity\",\"name\":\"fzp\",\"description\":\"wmdmwsflrdyrxl\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"m\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"ldhwr\":\"dataynlhsdtcgflevn\"}},{\"activity\":\"cflhwfrjyuhuthqd\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"uqve\":\"dataloyqjrkted\",\"zxcf\":\"datajsogesrmah\",\"vupnd\":\"datapyrelbzwxxsowdnu\"}}],\"userProperties\":[{\"name\":\"faeisboeap\",\"value\":\"dataraydlpu\"},{\"name\":\"kmakkwqrkaym\",\"value\":\"datagzbkliokuwhrpam\"},{\"name\":\"vx\",\"value\":\"datarl\"}],\"\":{\"dw\":\"databbacixlirolaoo\"}},{\"type\":\"Activity\",\"name\":\"jerm\",\"description\":\"kikgp\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"kcczb\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"iqvqbvfihna\":\"datakwjhkjvsvywnz\"}}],\"userProperties\":[{\"name\":\"ukegkludfdh\",\"value\":\"dataorihqzfjyqadtq\"},{\"name\":\"tsa\",\"value\":\"datajjfa\"},{\"name\":\"plywtgilhxaa\",\"value\":\"datanuufenp\"}],\"\":{\"exqyroqklgvyce\":\"dataktnfeghcmxi\",\"twhyznlhak\":\"dataywuioi\"}}]},\"name\":\"fskgxfmdpsreqor\",\"description\":\"ulzqjqbw\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"juakdsmwajalsen\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Completed\",\"Failed\"],\"\":{\"unz\":\"datavbslrhcceyobjs\"}},{\"activity\":\"bbyvxk\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"jvgftmpj\":\"datatdvxidmitmjcc\",\"scngdu\":\"datanrqgliqxahpy\",\"vhcwt\":\"dataw\",\"xigpmc\":\"dataqires\"}},{\"activity\":\"equocawcb\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Skipped\"],\"\":{\"aaav\":\"datapwhybtx\",\"dxdxrkrvmhhgvrxv\":\"dataiadygoad\"}},{\"activity\":\"uwbvrbwafw\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Skipped\"],\"\":{\"bvf\":\"datagwfgvpftbwmuxcpy\",\"ptkbe\":\"datamghhzm\",\"qvxzqwcmmolpfcv\":\"datapywvgfdsrng\"}}],\"userProperties\":[{\"name\":\"nnyksskuscdnn\",\"value\":\"dataoftapyrh\"},{\"name\":\"tjtqww\",\"value\":\"dataaxhsjw\"},{\"name\":\"c\",\"value\":\"datatwywhrzntmzzzavx\"}],\"\":{\"teaisywopko\":\"dataexspoiqvuky\",\"cbyldsmyq\":\"datalwmaigd\"}}") .toObject(SwitchActivity.class); - Assertions.assertEquals("zyr", model.name()); - Assertions.assertEquals("de", model.description()); + Assertions.assertEquals("fskgxfmdpsreqor", model.name()); + Assertions.assertEquals("ulzqjqbw", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("b", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("pmnxva", model.userProperties().get(0).name()); - Assertions.assertEquals("cguwyuzhkefownc", model.on().value()); - Assertions.assertEquals("woiqsrqebjgo", model.cases().get(0).value()); - Assertions.assertEquals("cahdagchk", model.cases().get(0).activities().get(0).name()); - Assertions.assertEquals("f", model.cases().get(0).activities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.cases().get(0).activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("juakdsmwajalsen", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("nnyksskuscdnn", model.userProperties().get(0).name()); + Assertions.assertEquals("suuapktfvemwfwc", model.on().value()); + Assertions.assertEquals("qv", model.cases().get(0).value()); + Assertions.assertEquals("hbyklwc", model.cases().get(0).activities().get(0).name()); + Assertions.assertEquals("xpkpsqk", model.cases().get(0).activities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.cases().get(0).activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.cases().get(0).activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("q", model.cases().get(0).activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("djva", model.cases().get(0).activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("xaawentkokndjwpx", model.defaultActivities().get(0).name()); - Assertions.assertEquals("n", model.defaultActivities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.defaultActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, - model.defaultActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("bv", model.defaultActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, + Assertions.assertEquals("bkhyqouzz", model.cases().get(0).activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("tlhnm", model.cases().get(0).activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("it", model.defaultActivities().get(0).name()); + Assertions.assertEquals("ebuvxxl", model.defaultActivities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.defaultActivities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.defaultActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("bzictf", model.defaultActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.defaultActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("dntunbpeeprmebv", model.defaultActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("chcootyscar", model.defaultActivities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SwitchActivity model = new SwitchActivity().withName("zyr").withDescription("de") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("b") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("pmnxva").withValue("dataq"), - new UserProperty().withName("cnlphlkx").withValue("dataanlyccdmkp"))) - .withOn(new Expression().withValue("cguwyuzhkefownc")) - .withCases(Arrays.asList( - new SwitchCase().withValue("woiqsrqebjgo") - .withActivities(Arrays.asList(new Activity().withName("cahdagchk").withDescription("f") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("q").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("bslwxcf").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("vedxyeba").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("wnmnxppgfep") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("djva").withValue("dataae"), - new UserProperty().withName("oqknz").withValue("datanvvkfbmrppjf"), - new UserProperty().withName("eabgpw").withValue("datas"))) - .withAdditionalProperties(mapOf("type", "Activity")))), - new SwitchCase().withValue("deawolhl").withActivities(Arrays.asList( - new Activity() - .withName("eznbmhqylrsywi").withDescription("oqtvx").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("de") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("p").withValue("dataqkcikfeshxo"), - new UserProperty().withName("tvkxpsxlpypzgdet").withValue("datad"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("epycpw").withDescription("oku").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("gcnicknsbbccbqx") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ojvejxhfeo") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("tfyjcenkidl").withValue("datamlxhzwyyy"), - new UserProperty().withName("rzbmlhg").withValue("datatkthevodddne"), - new UserProperty().withName("wsyx").withValue("datafdjftcr"), - new UserProperty().withName("ohdifbhtxtcqjg").withValue("datad"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("qpgncscw").withDescription("fdq").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("nutyjdcvnanej") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("llec").withValue("datauofgyhk"), - new UserProperty().withName("tofxkelwvcyprpo").withValue("datanqvuftki"))) - .withAdditionalProperties(mapOf("type", "Activity")))), - new SwitchCase().withValue("idukrjjfn") - .withActivities(Arrays.asList( - new Activity() - .withName("qalwjcqbnvbzem").withDescription("fyvjdcehlgryvku") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + SwitchActivity model = new SwitchActivity().withName("fskgxfmdpsreqor").withDescription("ulzqjqbw") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency() + .withActivity("juakdsmwajalsen") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, + DependencyCondition.COMPLETED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("bbyvxk") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("equocawcb") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, + DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("uwbvrbwafw") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays + .asList(new UserProperty().withName("nnyksskuscdnn").withValue("dataoftapyrh"), + new UserProperty().withName("tjtqww").withValue("dataaxhsjw"), new UserProperty().withName( + "c").withValue( + "datatwywhrzntmzzzavx"))) + .withOn(new Expression().withValue("suuapktfvemwfwc")) + .withCases( + Arrays.asList( + new SwitchCase().withValue("qv") + .withActivities(Arrays.asList( + new Activity().withName("hbyklwc").withDescription("xpkpsqk") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("bkhyqouzz") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("gnldz") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("tlhnm").withValue("datads"), + new UserProperty().withName("gfdvwshcc").withValue("datayqn"), + new UserProperty().withName("sjnrfpzlvaeo").withValue("datanskekhmomv"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("hukuypyeo").withDescription("aeabbxkldtw") + .withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("ldgbgua") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays + .asList(new UserProperty().withName("cdbudfwlpgpxyrfk").withValue("datalgpladqc"))) + .withAdditionalProperties(mapOf("type", "Activity")))), + new SwitchCase().withValue("bmuzpdjt").withActivities(Arrays.asList( + new Activity().withName("ycasxuhi").withDescription("mmiipf").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("porioki") + new ActivityDependency().withActivity("nfpxrzqagmcivs") .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("x").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fftt").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()))) + new ActivityDependency().withActivity("awiabyfzadeu") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("tkfvdjgw") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("akqgabrbsuxgn") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("nqlggtrjzi").withValue("dataxzbu"), - new UserProperty().withName("cinjej").withValue("datainlysguladd"), - new UserProperty().withName("jzenagmhhmgtbqzf").withValue("datampgibmngb"))) + Arrays.asList(new UserProperty().withName("kulozdoilhrxji").withValue("datajiv"), + new UserProperty().withName("orqlkycwnb").withValue("datalau"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("ukbpwwfeixm").withDescription("uwpivsltly") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + new Activity().withName("tufmujadippdntun").withDescription("eeprmebvxmaacr") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("cjrdqcmsrzrcd") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("lzgaopt") + new ActivityDependency().withActivity("jcesm").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("cxugatvjxyvxd") .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("qfgqccondaoptru") + new ActivityDependency().withActivity("uzdphogmrcmgu") .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("ezlloflcilrafkr").withValue("datavdwknymqzmu"), - new UserProperty().withName("nuvtgjgp").withValue("datavdjinuos"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("ey").withDescription("hyqouzzpgnldzsnv") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("shgfdvwshc") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("qnfsjnrfpzlvae").withValue("datajnskekhmo"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("obvkg").withDescription("jbjavnk").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("zf") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("ukuypyeofsa").withValue("dataabbxkldtwrrycl"), - new UserProperty().withName("rld").withValue("databguaxilcdbu"), - new UserProperty().withName("fwlp").withValue("datapxyrfkslg"))) + Arrays.asList(new UserProperty().withName("asdrrfozz").withValue("dataygolz"), + new UserProperty().withName("njkbmfcrysvcab").withValue("datak"), + new UserProperty().withName("jmzqnbwnlo").withValue("dataz"))) .withAdditionalProperties(mapOf("type", "Activity")))))) - .withDefaultActivities(Arrays.asList(new Activity().withName("xaawentkokndjwpx").withDescription("n") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("bv") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("hi") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("doilhrxjiwjivyor") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("endfpdoxtif") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + .withDefaultActivities(Arrays.asList( + new Activity().withName("it").withDescription("ebuvxxl").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("bzictf") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("dntunbpeeprmebv").withValue("datamaacrqrovbozjce"), - new UserProperty().withName("mrcxugatv").withValue("dataxyvxdbuzdphog"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("rkrjmzqnbwnl").withDescription("z").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withUserProperties( + Arrays.asList(new UserProperty().withName("chcootyscar").withValue("datamhiewvcpyskh"), + new UserProperty().withName("vkw").withValue("datatbvyclg"), + new UserProperty().withName("zbyxtprxt").withValue("datawvng"), new UserProperty() + .withName("csno").withValue("datakglygeuo"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("qux").withDescription("ekixouhcatozs").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs( + ActivityOnInactiveMarkAs.SKIPPED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("utcoqclypbr") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dwg") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("wjxatghuixczycif") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("uqchcooty") + new ActivityDependency().withActivity("p") .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("hlkx").withValue("dataanlyccdmkp"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("fzp").withDescription("wmdmwsflrdyrxl").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("m") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kvkwdtbvy") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("cflhwfrjyuhuthqd") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("nwxolfhiq").withValue("dataiulfxgzyr"), - new UserProperty().withName("qux").withValue("datatekixouhca"), - new UserProperty().withName("ozsodp").withValue("datasqcwjxatghuixc"), - new UserProperty().withName("ycifdr").withValue("datarywribmeuukkonw"))) + Arrays.asList(new UserProperty().withName("faeisboeap").withValue("dataraydlpu"), + new UserProperty().withName("kmakkwqrkaym").withValue("datagzbkliokuwhrpam"), + new UserProperty().withName("vx").withValue("datarl"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("jerm").withDescription("kikgp").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("kcczb") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("ukegkludfdh").withValue("dataorihqzfjyqadtq"), + new UserProperty().withName("tsa").withValue("datajjfa"), + new UserProperty().withName("plywtgilhxaa").withValue("datanuufenp"))) .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(SwitchActivity.class); - Assertions.assertEquals("zyr", model.name()); - Assertions.assertEquals("de", model.description()); + Assertions.assertEquals("fskgxfmdpsreqor", model.name()); + Assertions.assertEquals("ulzqjqbw", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); - Assertions.assertEquals("b", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("pmnxva", model.userProperties().get(0).name()); - Assertions.assertEquals("cguwyuzhkefownc", model.on().value()); - Assertions.assertEquals("woiqsrqebjgo", model.cases().get(0).value()); - Assertions.assertEquals("cahdagchk", model.cases().get(0).activities().get(0).name()); - Assertions.assertEquals("f", model.cases().get(0).activities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.cases().get(0).activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("juakdsmwajalsen", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("nnyksskuscdnn", model.userProperties().get(0).name()); + Assertions.assertEquals("suuapktfvemwfwc", model.on().value()); + Assertions.assertEquals("qv", model.cases().get(0).value()); + Assertions.assertEquals("hbyklwc", model.cases().get(0).activities().get(0).name()); + Assertions.assertEquals("xpkpsqk", model.cases().get(0).activities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.cases().get(0).activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.cases().get(0).activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("q", model.cases().get(0).activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("djva", model.cases().get(0).activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("xaawentkokndjwpx", model.defaultActivities().get(0).name()); - Assertions.assertEquals("n", model.defaultActivities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.defaultActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, - model.defaultActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("bv", model.defaultActivities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.FAILED, + Assertions.assertEquals("bkhyqouzz", model.cases().get(0).activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("tlhnm", model.cases().get(0).activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("it", model.defaultActivities().get(0).name()); + Assertions.assertEquals("ebuvxxl", model.defaultActivities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.defaultActivities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.defaultActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("bzictf", model.defaultActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.defaultActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("dntunbpeeprmebv", model.defaultActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("chcootyscar", model.defaultActivities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTypePropertiesTests.java index 1bb782b52f691..27f8f82022404 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchActivityTypePropertiesTests.java @@ -23,305 +23,173 @@ public final class SwitchActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SwitchActivityTypeProperties model = BinaryData.fromString( - "{\"on\":{\"value\":\"fzp\"},\"cases\":[{\"value\":\"dmwsflrdyr\",\"activities\":[{\"type\":\"Activity\",\"name\":\"x\",\"description\":\"mxnmx\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"hsdtcgflevndldhw\",\"dependencyConditions\":[]},{\"activity\":\"fcfl\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"rjyuhu\",\"value\":\"datahqdfibiz\"},{\"name\":\"oyqjrktedvu\",\"value\":\"dataveqjsogesr\"},{\"name\":\"ahszxcfbpyrelbz\",\"value\":\"dataxxsowdnuwvup\"},{\"name\":\"dnhq\",\"value\":\"dataaeisboeapsrayd\"}],\"\":{\"kliokuwhrpama\":\"dataokmakkwqrkaymdgz\",\"irolaooldwd\":\"dataxorldubbbacix\",\"dzkikgpo\":\"dataer\"}},{\"type\":\"Activity\",\"name\":\"qgku\",\"description\":\"cczbuobe\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"vsvywnzliqvqbv\",\"dependencyConditions\":[]},{\"activity\":\"ihnas\",\"dependencyConditions\":[]},{\"activity\":\"qukegkludfdh\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"ihqzfjyqa\",\"value\":\"datatqwtsatjjfav\"},{\"name\":\"lywtgilhx\",\"value\":\"dataa\"}],\"\":{\"toktnfeghc\":\"dataufenp\",\"lgvyceuywuioim\":\"dataxigexqyroq\"}},{\"type\":\"Activity\",\"name\":\"whyznlhakif\",\"description\":\"gxfmdpsreqorp\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"bwjiqrukbj\",\"dependencyConditions\":[]},{\"activity\":\"akdsmwajalsens\",\"dependencyConditions\":[]},{\"activity\":\"oslvf\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"slr\",\"value\":\"datacceyobjsju\"},{\"name\":\"zwbb\",\"value\":\"datavx\"},{\"name\":\"vfukuhtd\",\"value\":\"dataxidmit\"},{\"name\":\"jc\",\"value\":\"datanjvgftmp\"}],\"\":{\"ahpyss\":\"datarqgliq\",\"iresixigpmcmequo\":\"datangduewevhcwtt\",\"ybtxzaaaveiad\":\"dataawcbknyljycpw\"}},{\"type\":\"Activity\",\"name\":\"goadtdxdxrkrvmh\",\"description\":\"vrxvsuwbvrbwafw\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"gwfgvpftbwmuxcpy\",\"dependencyConditions\":[]},{\"activity\":\"bvf\",\"dependencyConditions\":[]},{\"activity\":\"mghhzm\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"kbehpywvgfdsrng\",\"value\":\"dataqvxzqwcmmolpfcv\"},{\"name\":\"ksnnykss\",\"value\":\"datauscdnneofta\"},{\"name\":\"yrhttjtqwwlaxh\",\"value\":\"datajwpc\"}],\"\":{\"vxidkexspoi\":\"dataywhrzntmzzz\",\"tteaisywopkovl\":\"datavuk\"}}]},{\"value\":\"aigdvcbylds\",\"activities\":[{\"type\":\"Activity\",\"name\":\"mgvqthlimvyzrdq\",\"description\":\"yonkrpxwldktphni\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"tgfitpx\",\"dependencyConditions\":[]},{\"activity\":\"kb\",\"dependencyConditions\":[]},{\"activity\":\"ngj\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"yqvbdlfzkujg\",\"value\":\"datapp\"}],\"\":{\"eacjoaix\":\"dataovghalmscogga\"}},{\"type\":\"Activity\",\"name\":\"maokkgvwvlqq\",\"description\":\"d\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"lwjhpib\",\"dependencyConditions\":[]},{\"activity\":\"alefjsgnxrgmv\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"b\",\"value\":\"dataypqnshnbfd\"},{\"name\":\"xs\",\"value\":\"datap\"}],\"\":{\"bowqmfh\":\"dataprgztzc\",\"g\":\"datahnbsxoebephohjo\",\"bmngkqej\":\"dataifchvr\",\"wcfxbywpwjvpg\":\"datahwyyzzdlfayic\"}},{\"type\":\"Activity\",\"name\":\"stxznkbjkjezunrd\",\"description\":\"gpdnnvepbo\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"bxd\",\"dependencyConditions\":[]},{\"activity\":\"opummphbfp\",\"dependencyConditions\":[]},{\"activity\":\"ive\",\"dependencyConditions\":[]},{\"activity\":\"l\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"izyenajjx\",\"value\":\"datamdp\"},{\"name\":\"ersmevhgsuqolj\",\"value\":\"datavrjqakbhzsyqp\"}],\"\":{\"y\":\"databxgrgygu\",\"yivgzeiocacngi\":\"datav\",\"zhyceteidf\":\"dataadgx\"}},{\"type\":\"Activity\",\"name\":\"ofmcnnicmlomlnpr\",\"description\":\"kyn\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"hqnuitumxhvemo\",\"dependencyConditions\":[]},{\"activity\":\"abcwvibjfkcmzan\",\"dependencyConditions\":[]},{\"activity\":\"utjdmd\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"dtrqiuoh\",\"value\":\"datajjlaxeqehg\"}],\"\":{\"om\":\"datavraw\",\"dwfyagvhe\":\"datagb\",\"ndapxxgvcsvtf\":\"dataptcuqzdwpcupejzo\",\"xmteexapfyp\":\"datac\"}}]},{\"value\":\"ierutced\",\"activities\":[{\"type\":\"Activity\",\"name\":\"srrgdi\",\"description\":\"qy\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"librwomdwzzac\",\"dependencyConditions\":[]},{\"activity\":\"rkcdokkagkait\",\"dependencyConditions\":[]},{\"activity\":\"h\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"sadjlqk\",\"value\":\"dataraqt\"},{\"name\":\"i\",\"value\":\"datakkcebjaj\"},{\"name\":\"ptydve\",\"value\":\"dataipkeonixx\"},{\"name\":\"ukghxde\",\"value\":\"dataqptvxibpzhkn\"}],\"\":{\"ss\":\"datavzqawjnw\"}},{\"type\":\"Activity\",\"name\":\"ubpfe\",\"description\":\"jghfaldx\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"kjseftvwuhfmakn\",\"dependencyConditions\":[]},{\"activity\":\"naqyeswinoecwabu\",\"dependencyConditions\":[]},{\"activity\":\"qflwskb\",\"dependencyConditions\":[]},{\"activity\":\"eqayvkmp\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"qxiyllamdz\",\"value\":\"datazjrlm\"}],\"\":{\"ahbcygg\":\"dataoesxpcbglb\",\"clbdcglimac\":\"datalosxtbdgpy\"}},{\"type\":\"Activity\",\"name\":\"tkypyvztdhoya\",\"description\":\"umpxdyyohbbtwpkg\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"jejalybvxumtxu\",\"dependencyConditions\":[]},{\"activity\":\"doteidc\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"dqqgtwegqmlviy\",\"value\":\"datacyfszluzmz\"},{\"name\":\"attagroejsa\",\"value\":\"datarzckmcukzwz\"}],\"\":{\"wbvyraazscxi\":\"datazzrxgqxddvuiur\",\"kmybohax\":\"datajyjcshmtpdvuix\"}},{\"type\":\"Activity\",\"name\":\"vckfivia\",\"description\":\"nnmcdqzgepjyppk\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"eyfsvuyuyq\",\"dependencyConditions\":[]},{\"activity\":\"uzpjnakqc\",\"dependencyConditions\":[]},{\"activity\":\"goozyxu\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"itpo\",\"value\":\"datajyjhkrklzni\"},{\"name\":\"pxiqpjnqy\",\"value\":\"datalkcbkljjiuirmc\"},{\"name\":\"pbehqbmhqihlxdho\",\"value\":\"datag\"}],\"\":{\"haw\":\"dataocvctmpxnbnhogb\",\"zlokttpmbxn\":\"dataefgett\",\"gnbcwfpgvmixfqqm\":\"datakbvhd\"}}]},{\"value\":\"cu\",\"activities\":[{\"type\":\"Activity\",\"name\":\"r\",\"description\":\"pdtzugwurvpcwy\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ravdq\",\"dependencyConditions\":[]},{\"activity\":\"uewgpmademloima\",\"dependencyConditions\":[]},{\"activity\":\"kbmkkun\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"dvgcgunqitzw\",\"value\":\"datadpkevto\"},{\"name\":\"vqjrdydzq\",\"value\":\"datauppxdzpjewp\"}],\"\":{\"wkrk\":\"datas\",\"tfgbxiao\":\"datakgrnxly\",\"n\":\"datazrouwkkwtoxl\"}},{\"type\":\"Activity\",\"name\":\"vealwdltstxronbz\",\"description\":\"ee\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"krxajta\",\"dependencyConditions\":[]},{\"activity\":\"dfqc\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"kdwpcmy\",\"value\":\"datadrrkfhlaygwxo\"}],\"\":{\"trsppucxigk\":\"datadoruiycvourqdmzs\",\"d\":\"dataevtblmrjlaldggwa\",\"czhiradklzgiq\":\"dataelnvcfume\"}},{\"type\":\"Activity\",\"name\":\"mapxnoogmfujecis\",\"description\":\"meze\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"nljy\",\"dependencyConditions\":[]},{\"activity\":\"umpydk\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"ufh\",\"value\":\"datarvxxz\"},{\"name\":\"qo\",\"value\":\"datao\"},{\"name\":\"sczcksjwdwzfdfkg\",\"value\":\"datala\"},{\"name\":\"vmbsmxhsqdotbnf\",\"value\":\"dataniybotuq\"}],\"\":{\"erbsgwoy\":\"datakuqv\",\"xwg\":\"datacvwqyf\",\"z\":\"datamxmiw\"}}]}],\"defaultActivities\":[{\"type\":\"Activity\",\"name\":\"lypuxbnv\",\"description\":\"xutkwb\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"jnz\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Skipped\",\"Succeeded\"],\"\":{\"qov\":\"databgjhmyz\",\"ycyyajl\":\"datatid\",\"ipnclnbfxme\":\"dataotmir\"}},{\"activity\":\"y\",\"dependencyConditions\":[\"Succeeded\",\"Completed\"],\"\":{\"jaqzdzkyqqbqb\":\"datazgh\",\"ba\":\"databwvtwmm\",\"eyoxtlqytxfte\":\"dataftkceyjt\"}},{\"activity\":\"zrcq\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Failed\"],\"\":{\"kl\":\"dataapawpzphkmwbtr\",\"lqtpebaa\":\"datandb\",\"rbjtjvqdwz\":\"datazsxp\"}}],\"userProperties\":[{\"name\":\"dgt\",\"value\":\"datanieqlikyc\"}],\"\":{\"vktlrcauad\":\"datafukeh\",\"wnrqfi\":\"dataprjs\",\"pftw\":\"dataqamxxpfyl\",\"uortgwwtaolfdgjr\":\"datae\"}},{\"type\":\"Activity\",\"name\":\"phpvohvcaqar\",\"description\":\"kzznarnjueq\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"zoihtncadrmthh\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Succeeded\",\"Completed\"],\"\":{\"rqotig\":\"datazb\",\"wqyhklhossc\":\"datandfrxn\"}},{\"activity\":\"jtiungjbfmrsjgm\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Skipped\",\"Skipped\"],\"\":{\"gxskghdadgqpbgzy\":\"datajhfl\"}},{\"activity\":\"fazwiei\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Skipped\"],\"\":{\"inrsrri\":\"dataox\",\"t\":\"datacwnthtqtbc\",\"dzjlkocjuajclr\":\"dataqjsvlzdustzb\",\"jnqiprjahgqzb\":\"datassbkzdgwpy\"}}],\"userProperties\":[{\"name\":\"yufnumfeah\",\"value\":\"datar\"}],\"\":{\"ossqm\":\"datamuryajppuflvazpi\",\"ntsfyntkfziitbw\":\"dataihbvfallpobzv\",\"wwhml\":\"datah\",\"bfg\":\"datas\"}}]}") + "{\"on\":{\"value\":\"gvqthlimvyzrdqpg\"},\"cases\":[{\"value\":\"rpxwldktphnis\",\"activities\":[{\"type\":\"Activity\",\"name\":\"jnbt\",\"description\":\"itpxpkbangjxbbyq\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"ujgeppxiyovgha\",\"dependencyConditions\":[]},{\"activity\":\"m\",\"dependencyConditions\":[]},{\"activity\":\"cog\",\"dependencyConditions\":[]},{\"activity\":\"ameacjoaixhma\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"gvwvlqqnf\",\"value\":\"datawrws\"},{\"name\":\"yblwjhpibgalefjs\",\"value\":\"datanxrgmvzcibqy\"},{\"name\":\"qnshnbfd\",\"value\":\"dataxs\"}],\"\":{\"wqmfhg\":\"dataskprgztzcib\",\"ephohjorguif\":\"datanbsxoe\",\"gbmngkqejr\":\"datahv\"}},{\"type\":\"Activity\",\"name\":\"w\",\"description\":\"zzdlfayich\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"pwjvpglstxznk\",\"dependencyConditions\":[]},{\"activity\":\"jkjezunr\",\"dependencyConditions\":[]},{\"activity\":\"dygpdnnvepbocw\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"wdbxdko\",\"value\":\"dataummphbfpriveilx\"},{\"name\":\"pizyen\",\"value\":\"datajj\"},{\"name\":\"zmdpnersm\",\"value\":\"datavhgsu\"}],\"\":{\"zsyqpkpvb\":\"datajlvrjqakb\"}},{\"type\":\"Activity\",\"name\":\"g\",\"description\":\"ygu\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"vg\",\"dependencyConditions\":[]},{\"activity\":\"eiocacngiaa\",\"dependencyConditions\":[]},{\"activity\":\"gxdzhyc\",\"dependencyConditions\":[]},{\"activity\":\"teidfzof\",\"dependencyConditions\":[]}],\"userProperties\":[{\"name\":\"nicmlomlnp\",\"value\":\"datakikyn\"}],\"\":{\"ogabcwvibjfkc\":\"datagquphqnuitumxhve\"}}]}],\"defaultActivities\":[{\"type\":\"Activity\",\"name\":\"n\",\"description\":\"tjdmdbtbdtrqiu\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"xeqehgrjgvr\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\"],\"\":{\"dwfyagvhe\":\"datagb\"}},{\"activity\":\"ptcuqzdwpcupejzo\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Completed\",\"Succeeded\"],\"\":{\"exapfypdfierut\":\"datavcsvtflcjxmt\",\"sahv\":\"dataedeygsrrgdimaqy\",\"cyrkcdo\":\"datawlibrwomdwzz\"}}],\"userProperties\":[{\"name\":\"g\",\"value\":\"dataaitihncysa\"},{\"name\":\"jlq\",\"value\":\"dataora\"},{\"name\":\"tbiskkceb\",\"value\":\"dataajlptydvebipkeo\"}],\"\":{\"tvxibpzh\":\"dataxiukghxdekq\",\"uevzqawjnwj\":\"datan\"}},{\"type\":\"Activity\",\"name\":\"siubp\",\"description\":\"isjghfal\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"bpkjseft\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"noecwabuf\":\"datamaknonaqyesw\",\"ayvkmptgpqx\":\"dataflwskbbe\"}},{\"activity\":\"y\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\"],\"\":{\"kdbo\":\"dataozjrl\",\"bc\":\"datasxpcbglbpa\",\"o\":\"dataggf\"}},{\"activity\":\"xtbdgp\",\"dependencyConditions\":[\"Skipped\",\"Completed\"],\"\":{\"ztdho\":\"datacglimacztkypy\",\"yohbbtwpkgc\":\"dataarcumpxd\"}}],\"userProperties\":[{\"name\":\"mjcjejalybvxumt\",\"value\":\"datauvdoteidcwrmdq\"}],\"\":{\"ym\":\"datawegqmlv\",\"agroejsaer\":\"datayfszluzmzgat\"}},{\"type\":\"Activity\",\"name\":\"ckmcukzwzgio\",\"description\":\"rxgq\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"riwbvyraa\",\"dependencyConditions\":[\"Failed\"],\"\":{\"hmtpdvuixmkmyboh\":\"datakjyjc\",\"viadqnnmcdqzgep\":\"dataxmvckf\",\"fsvuyuyqc\":\"datayppkfraohiye\",\"ieitp\":\"datazpjnakqcsgoozyxu\"}},{\"activity\":\"kjyjhkrk\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Failed\",\"Skipped\"],\"\":{\"jjiuirmcupbehq\":\"dataqpjnqyylkcbk\"}},{\"activity\":\"mhqihlxdh\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Failed\",\"Skipped\"],\"\":{\"bwhawref\":\"datavctmpxnbnho\"}}],\"userProperties\":[{\"name\":\"ttzlo\",\"value\":\"datat\"}],\"\":{\"bcwfp\":\"databxnqkbvhdbg\",\"tcucfbr\":\"datavmixfqqm\"}},{\"type\":\"Activity\",\"name\":\"m\",\"description\":\"tzugwurvpcwy\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ravdq\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Failed\",\"Failed\"],\"\":{\"oimaykbmkkunfh\":\"datadem\",\"dpkevto\":\"datadvgcgunqitzw\",\"uppxdzpjewp\":\"datavqjrdydzq\"}},{\"activity\":\"lyszw\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Failed\"],\"\":{\"tfgbxiao\":\"datanxly\",\"n\":\"datazrouwkkwtoxl\",\"keeeakzys\":\"datavealwdltstxronbz\"}},{\"activity\":\"krxajta\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"q\":\"datahjwkdwpcm\"}}],\"userProperties\":[{\"name\":\"kfhlayg\",\"value\":\"dataxoreed\"},{\"name\":\"ruiycvourq\",\"value\":\"datamzsitrspp\"},{\"name\":\"cxigkpevtblmrjl\",\"value\":\"dataldggwaldte\"},{\"name\":\"nvcfumezczh\",\"value\":\"dataradklzgiqm\"}],\"\":{\"icmezexwzpgy\":\"dataxnoogmfujeci\"}}]}") .toObject(SwitchActivityTypeProperties.class); - Assertions.assertEquals("fzp", model.on().value()); - Assertions.assertEquals("dmwsflrdyr", model.cases().get(0).value()); - Assertions.assertEquals("x", model.cases().get(0).activities().get(0).name()); - Assertions.assertEquals("mxnmx", model.cases().get(0).activities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.cases().get(0).activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, + Assertions.assertEquals("gvqthlimvyzrdqpg", model.on().value()); + Assertions.assertEquals("rpxwldktphnis", model.cases().get(0).value()); + Assertions.assertEquals("jnbt", model.cases().get(0).activities().get(0).name()); + Assertions.assertEquals("itpxpkbangjxbbyq", model.cases().get(0).activities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.cases().get(0).activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.cases().get(0).activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("hsdtcgflevndldhw", + Assertions.assertEquals("ujgeppxiyovgha", model.cases().get(0).activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("rjyuhu", model.cases().get(0).activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("lypuxbnv", model.defaultActivities().get(0).name()); - Assertions.assertEquals("xutkwb", model.defaultActivities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.defaultActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.defaultActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("jnz", model.defaultActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("gvwvlqqnf", model.cases().get(0).activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("n", model.defaultActivities().get(0).name()); + Assertions.assertEquals("tjdmdbtbdtrqiu", model.defaultActivities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.defaultActivities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.defaultActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("xeqehgrjgvr", model.defaultActivities().get(0).dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.SKIPPED, model.defaultActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("dgt", model.defaultActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("g", model.defaultActivities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SwitchActivityTypeProperties model - = new SwitchActivityTypeProperties().withOn(new Expression().withValue("fzp")) - .withCases(Arrays.asList( - new SwitchCase().withValue("dmwsflrdyr") - .withActivities(Arrays.asList( - new Activity().withName("x").withDescription("mxnmx").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("hsdtcgflevndldhw") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fcfl") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("rjyuhu").withValue("datahqdfibiz"), - new UserProperty().withName("oyqjrktedvu").withValue("dataveqjsogesr"), - new UserProperty().withName("ahszxcfbpyrelbz").withValue("dataxxsowdnuwvup"), - new UserProperty().withName("dnhq").withValue("dataaeisboeapsrayd"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("qgku").withDescription("cczbuobe") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("vsvywnzliqvqbv") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ihnas") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("qukegkludfdh") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("ihqzfjyqa").withValue("datatqwtsatjjfav"), - new UserProperty().withName("lywtgilhx").withValue("dataa"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("whyznlhakif").withDescription("gxfmdpsreqorp") - .withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("bwjiqrukbj") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("akdsmwajalsens") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("oslvf") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("slr").withValue("datacceyobjsju"), - new UserProperty().withName("zwbb").withValue("datavx"), - new UserProperty() - .withName("vfukuhtd").withValue("dataxidmit"), - new UserProperty().withName("jc").withValue("datanjvgftmp"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("goadtdxdxrkrvmh").withDescription("vrxvsuwbvrbwafw") - .withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("gwfgvpftbwmuxcpy") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("bvf") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("mghhzm") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("kbehpywvgfdsrng").withValue("dataqvxzqwcmmolpfcv"), - new UserProperty().withName("ksnnykss") - .withValue("datauscdnneofta"), - new UserProperty().withName("yrhttjtqwwlaxh").withValue("datajwpc"))) - .withAdditionalProperties(mapOf("type", "Activity")))), - new SwitchCase().withValue("aigdvcbylds").withActivities(Arrays.asList( + SwitchActivityTypeProperties model = new SwitchActivityTypeProperties() + .withOn(new Expression().withValue("gvqthlimvyzrdqpg")) + .withCases(Arrays.asList(new SwitchCase().withValue("rpxwldktphnis").withActivities(Arrays.asList( + new Activity().withName("jnbt").withDescription("itpxpkbangjxbbyq").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("ujgeppxiyovgha") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("m").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("cog").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("ameacjoaixhma").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("gvwvlqqnf").withValue("datawrws"), + new UserProperty().withName("yblwjhpibgalefjs").withValue("datanxrgmvzcibqy"), + new UserProperty().withName("qnshnbfd").withValue("dataxs"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("w").withDescription("zzdlfayich").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("pwjvpglstxznk").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("jkjezunr").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("dygpdnnvepbocw") + .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("wdbxdko").withValue("dataummphbfpriveilx"), + new UserProperty().withName("pizyen").withValue("datajj"), + new UserProperty().withName("zmdpnersm").withValue("datavhgsu"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("g").withDescription("ygu").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("vg").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("eiocacngiaa").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("gxdzhyc").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("teidfzof").withDependencyConditions(Arrays.asList()) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("nicmlomlnp").withValue("datakikyn"))) + .withAdditionalProperties(mapOf("type", "Activity")))))) + .withDefaultActivities( + Arrays + .asList( new Activity() - .withName("mgvqthlimvyzrdq").withDescription("yonkrpxwldktphni") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("tgfitpx") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kb").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ngj").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("yqvbdlfzkujg").withValue("datapp"))) + .withName("n").withDescription("tjdmdbtbdtrqiu").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED).withDependsOn( + Arrays + .asList( + new ActivityDependency().withActivity("xeqehgrjgvr") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, + DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("ptcuqzdwpcupejzo") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, + DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays + .asList(new UserProperty().withName("g").withValue( + "dataaitihncysa"), new UserProperty().withName("jlq").withValue("dataora"), + new UserProperty().withName("tbiskkceb").withValue("dataajlptydvebipkeo"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("maokkgvwvlqq").withDescription("d").withState(ActivityState.ACTIVE) + new Activity().withName("siubp").withDescription("isjghfal").withState(ActivityState.INACTIVE) .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("lwjhpib") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("alefjsgnxrgmv") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("b").withValue("dataypqnshnbfd"), - new UserProperty().withName("xs").withValue("datap"))) + .withDependsOn( + Arrays.asList( + new ActivityDependency().withActivity("bpkjseft") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("y") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("xtbdgp") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays + .asList(new UserProperty().withName("mjcjejalybvxumt").withValue("datauvdoteidcwrmdq"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("stxznkbjkjezunrd").withDescription("gpdnnvepbo") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("bxd").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("opummphbfp") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ive").withDependencyConditions(Arrays.asList()) + new Activity().withName("ckmcukzwzgio").withDescription("rxgq").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED).withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("riwbvyraa") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("l").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("izyenajjx").withValue("datamdp"), - new UserProperty().withName("ersmevhgsuqolj").withValue("datavrjqakbhzsyqp"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("ofmcnnicmlomlnpr").withDescription("kyn") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("hqnuitumxhvemo") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("abcwvibjfkcmzan") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("utjdmd") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("dtrqiuoh").withValue("datajjlaxeqehg"))) - .withAdditionalProperties(mapOf("type", "Activity")))), - new SwitchCase().withValue("ierutced") - .withActivities(Arrays.asList( - new Activity().withName("srrgdi").withDescription("qy").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("librwomdwzzac") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("rkcdokkagkait") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("h").withDependencyConditions(Arrays.asList()) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("sadjlqk").withValue("dataraqt"), - new UserProperty().withName("i").withValue("datakkcebjaj"), - new UserProperty().withName("ptydve") - .withValue("dataipkeonixx"), - new UserProperty().withName("ukghxde").withValue("dataqptvxibpzhkn"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("ubpfe").withDescription("jghfaldx") - .withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("kjseftvwuhfmakn") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("naqyeswinoecwabu") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("qflwskb") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("eqayvkmp") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("qxiyllamdz").withValue("datazjrlm"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("tkypyvztdhoya").withDescription("umpxdyyohbbtwpkg") - .withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("jejalybvxumtxu") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("doteidc") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList( - new UserProperty().withName("dqqgtwegqmlviy").withValue("datacyfszluzmz"), - new UserProperty().withName("attagroejsa").withValue("datarzckmcukzwz"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("vckfivia").withDescription("nnmcdqzgepjyppk") - .withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("eyfsvuyuyq") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("uzpjnakqc") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("goozyxu") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("itpo").withValue("datajyjhkrklzni"), - new UserProperty().withName("pxiqpjnqy").withValue("datalkcbkljjiuirmc"), - new UserProperty().withName("pbehqbmhqihlxdho").withValue("datag"))) - .withAdditionalProperties(mapOf("type", "Activity")))), - new SwitchCase().withValue("cu").withActivities(Arrays.asList( - new Activity().withName("r").withDescription("pdtzugwurvpcwy").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("ravdq").withDependencyConditions(Arrays.asList()) + new ActivityDependency().withActivity("kjyjhkrk") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("uewgpmademloima") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kbmkkun") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("dvgcgunqitzw").withValue("datadpkevto"), - new UserProperty().withName("vqjrdydzq").withValue("datauppxdzpjewp"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("vealwdltstxronbz").withDescription("ee") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("krxajta") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dfqc").withDependencyConditions(Arrays.asList()) + new ActivityDependency().withActivity("mhqihlxdh") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, + DependencyCondition.FAILED, DependencyCondition.SKIPPED)) .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("kdwpcmy").withValue("datadrrkfhlaygwxo"))) + .withUserProperties(Arrays.asList(new UserProperty().withName("ttzlo").withValue("datat"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("mapxnoogmfujecis").withDescription("meze") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + new Activity().withName("m").withDescription("tzugwurvpcwy").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs( + ActivityOnInactiveMarkAs.SUCCEEDED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("nljy").withDependencyConditions(Arrays.asList()) + new ActivityDependency().withActivity("ravdq") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, + DependencyCondition.FAILED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("umpydk") - .withDependencyConditions(Arrays.asList()).withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("ufh").withValue("datarvxxz"), - new UserProperty().withName("qo").withValue("datao"), - new UserProperty().withName("sczcksjwdwzfdfkg").withValue("datala"), - new UserProperty().withName("vmbsmxhsqdotbnf").withValue("dataniybotuq"))) - .withAdditionalProperties(mapOf("type", "Activity")))))) - .withDefaultActivities( - Arrays.asList( - new Activity().withName("lypuxbnv").withDescription("xutkwb").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("jnz") + new ActivityDependency().withActivity("lyszw") .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED, - DependencyCondition.SUCCEEDED)) + DependencyCondition.FAILED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("y") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("zrcq") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED)) + new ActivityDependency().withActivity("krxajta") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("dgt") - .withValue("datanieqlikyc"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("phpvohvcaqar").withDescription("kzznarnjueq") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs( - ActivityOnInactiveMarkAs.FAILED) - .withDependsOn( - Arrays - .asList( - new ActivityDependency().withActivity("zoihtncadrmthh") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jtiungjbfmrsjgm") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fazwiei") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("yufnumfeah").withValue("datar"))) + Arrays.asList(new UserProperty().withName("kfhlayg").withValue("dataxoreed"), + new UserProperty().withName("ruiycvourq").withValue("datamzsitrspp"), + new UserProperty().withName("cxigkpevtblmrjl").withValue("dataldggwaldte"), + new UserProperty().withName("nvcfumezczh").withValue("dataradklzgiqm"))) .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(SwitchActivityTypeProperties.class); - Assertions.assertEquals("fzp", model.on().value()); - Assertions.assertEquals("dmwsflrdyr", model.cases().get(0).value()); - Assertions.assertEquals("x", model.cases().get(0).activities().get(0).name()); - Assertions.assertEquals("mxnmx", model.cases().get(0).activities().get(0).description()); - Assertions.assertEquals(ActivityState.ACTIVE, model.cases().get(0).activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, + Assertions.assertEquals("gvqthlimvyzrdqpg", model.on().value()); + Assertions.assertEquals("rpxwldktphnis", model.cases().get(0).value()); + Assertions.assertEquals("jnbt", model.cases().get(0).activities().get(0).name()); + Assertions.assertEquals("itpxpkbangjxbbyq", model.cases().get(0).activities().get(0).description()); + Assertions.assertEquals(ActivityState.INACTIVE, model.cases().get(0).activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.cases().get(0).activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("hsdtcgflevndldhw", + Assertions.assertEquals("ujgeppxiyovgha", model.cases().get(0).activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals("rjyuhu", model.cases().get(0).activities().get(0).userProperties().get(0).name()); - Assertions.assertEquals("lypuxbnv", model.defaultActivities().get(0).name()); - Assertions.assertEquals("xutkwb", model.defaultActivities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.defaultActivities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.defaultActivities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("jnz", model.defaultActivities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals("gvwvlqqnf", model.cases().get(0).activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("n", model.defaultActivities().get(0).name()); + Assertions.assertEquals("tjdmdbtbdtrqiu", model.defaultActivities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.defaultActivities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.defaultActivities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("xeqehgrjgvr", model.defaultActivities().get(0).dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.SKIPPED, model.defaultActivities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("dgt", model.defaultActivities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("g", model.defaultActivities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchCaseTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchCaseTests.java index ede000010301d..fddb70523eee5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchCaseTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SwitchCaseTests.java @@ -21,90 +21,66 @@ public final class SwitchCaseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SwitchCase model = BinaryData.fromString( - "{\"value\":\"kkraenzuufpd\",\"activities\":[{\"type\":\"Activity\",\"name\":\"xephwxd\",\"description\":\"ow\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"jca\",\"dependencyConditions\":[\"Succeeded\",\"Failed\"],\"\":{\"mnqdjfldzvgog\":\"datakvpleoo\",\"aofobjlqnaxfvsy\":\"datauc\",\"bjbknpzhfhi\":\"datastrbje\"}},{\"activity\":\"hgwbslrqbd\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Completed\",\"Completed\"],\"\":{\"ahbzdgwkim\":\"dataijwpskneprumhik\",\"ujxdnia\":\"datavatrvjkxcrxqpen\",\"qytppjdyikdykxh\":\"dataeterjerhwgiuduw\",\"gmuk\":\"datarkdtucyutrp\"}}],\"userProperties\":[{\"name\":\"vftij\",\"value\":\"datas\"},{\"name\":\"lcrjynefx\",\"value\":\"dataednczvn\"},{\"name\":\"yfzavs\",\"value\":\"datajbahshyxhfe\"},{\"name\":\"tywluxysmq\",\"value\":\"dataeo\"}],\"\":{\"qmcaofxg\":\"datalwfsmpbwwphj\"}},{\"type\":\"Activity\",\"name\":\"yvjefnlxqmtedzxu\",\"description\":\"kxjrtt\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"qynwqcov\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Skipped\",\"Skipped\"],\"\":{\"rcmayqasdve\":\"datahhbddxkoj\"}},{\"activity\":\"ldafx\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Completed\",\"Succeeded\"],\"\":{\"dzyjugdkn\":\"datasbcfoguubqcqnc\",\"vpru\":\"datalbrixvcpi\"}},{\"activity\":\"ttrvk\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Completed\"],\"\":{\"jbanlmpmvegxgymx\":\"dataw\",\"lnjeybgpjy\":\"datalrtuegqhq\"}}],\"userProperties\":[{\"name\":\"uowkt\",\"value\":\"databpv\"},{\"name\":\"dkydqcgedip\",\"value\":\"datanzmvttttjm\"},{\"name\":\"tfuwx\",\"value\":\"dataee\"}],\"\":{\"c\":\"dataiesgur\"}},{\"type\":\"Activity\",\"name\":\"pb\",\"description\":\"mttmixewparbazgo\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"wjhjutixggnldfl\",\"dependencyConditions\":[\"Succeeded\",\"Failed\",\"Skipped\"],\"\":{\"klepetsxetne\":\"datacmuvf\",\"lmsxnx\":\"dataerqb\"}},{\"activity\":\"kzcdnipj\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"izkwpo\":\"datavgydtdtomknzotm\"}},{\"activity\":\"askflrqw\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\",\"Succeeded\",\"Completed\"],\"\":{\"udiyfix\":\"datarzvnouthbvvcb\",\"rqivqzqcmrxh\":\"dataw\",\"fhijcetcystrs\":\"datalozg\"}}],\"userProperties\":[{\"name\":\"txspaafseqoyo\",\"value\":\"datarlrq\"}],\"\":{\"lgt\":\"dataxh\",\"qky\":\"dataveumwbmqpbfjbsol\",\"xbkckam\":\"datajzbxmg\"}}]}") + "{\"value\":\"lj\",\"activities\":[{\"type\":\"Activity\",\"name\":\"mpydkgbcufhk\",\"description\":\"xxzhqouon\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"wdwzf\",\"dependencyConditions\":[\"Completed\"],\"\":{\"vmbsmxhsqdotbnf\":\"datala\",\"jfkuqvt\":\"dataniybotuq\",\"gqmxmiwfzrhilyp\":\"datarbsgwoykcvwqyfix\",\"n\":\"datax\"}},{\"activity\":\"quxut\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Skipped\",\"Completed\"],\"\":{\"vbgjhmyzsq\":\"dataaijnzqnqwka\",\"ir\":\"datavmtidmycyyajlnot\"}},{\"activity\":\"ipnclnbfxme\",\"dependencyConditions\":[\"Failed\"],\"\":{\"bqbwbw\":\"datafrfzghnjaqzdzkyq\"}},{\"activity\":\"twmmvbahftkcey\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Skipped\"],\"\":{\"pzrcq\":\"datatlqytxft\",\"phkmwbtrqklondbv\":\"datasjqrgtapawp\",\"rbjtjvqdwz\":\"dataqtpebaawzsxp\",\"eqlikyctun\":\"datavxdgten\"}}],\"userProperties\":[{\"name\":\"ehxvktlrc\",\"value\":\"datauad\"}],\"\":{\"n\":\"datajsu\",\"amxxpfyl\":\"dataqfiz\"}},{\"type\":\"Activity\",\"name\":\"pftw\",\"description\":\"pu\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"aolfdgjrgp\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Skipped\"],\"\":{\"kzznarnjueq\":\"dataaqarp\"}},{\"activity\":\"uzjgv\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Completed\",\"Succeeded\"],\"\":{\"hfxy\":\"dataadrmt\",\"azbfrqo\":\"datann\"}}],\"userProperties\":[{\"name\":\"xndfrxnvwqyhklho\",\"value\":\"datascpj\"}],\"\":{\"jbfmrsjgm\":\"datan\",\"kghd\":\"datasamhxkjjhflrgx\"}}]}") .toObject(SwitchCase.class); - Assertions.assertEquals("kkraenzuufpd", model.value()); - Assertions.assertEquals("xephwxd", model.activities().get(0).name()); - Assertions.assertEquals("ow", model.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("jca", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, + Assertions.assertEquals("lj", model.value()); + Assertions.assertEquals("mpydkgbcufhk", model.activities().get(0).name()); + Assertions.assertEquals("xxzhqouon", model.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.activities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("wdwzf", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("vftij", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("ehxvktlrc", model.activities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SwitchCase model = new SwitchCase().withValue("kkraenzuufpd") - .withActivities(Arrays.asList( - new Activity() - .withName("xephwxd").withDescription("ow").withState( - ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("jca") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("hgwbslrqbd") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("vftij").withValue("datas"), - new UserProperty().withName("lcrjynefx").withValue("dataednczvn"), - new UserProperty().withName("yfzavs").withValue("datajbahshyxhfe"), - new UserProperty().withName("tywluxysmq").withValue("dataeo"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("yvjefnlxqmtedzxu").withDescription("kxjrtt").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("qynwqcov") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.FAILED, DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ldafx") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ttrvk") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("uowkt").withValue("databpv"), - new UserProperty().withName("dkydqcgedip").withValue("datanzmvttttjm"), new UserProperty() - .withName("tfuwx").withValue("dataee"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("pb").withDescription("mttmixewparbazgo").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("wjhjutixggnldfl") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.FAILED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kzcdnipj") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("askflrqw") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("txspaafseqoyo").withValue("datarlrq"))) - .withAdditionalProperties(mapOf("type", "Activity")))); + SwitchCase model = new SwitchCase().withValue("lj").withActivities(Arrays.asList( + new Activity().withName("mpydkgbcufhk").withDescription("xxzhqouon").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("wdwzf") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("quxut") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, + DependencyCondition.SKIPPED, DependencyCondition.COMPLETED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency() + .withActivity("ipnclnbfxme").withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("twmmvbahftkcey") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList( + new UserProperty().withName("ehxvktlrc").withValue("datauad"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("pftw").withDescription("pu").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("aolfdgjrgp") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("uzjgv") + .withDependencyConditions( + Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, + DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties( + Arrays.asList(new UserProperty().withName("xndfrxnvwqyhklho").withValue("datascpj"))) + .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(SwitchCase.class); - Assertions.assertEquals("kkraenzuufpd", model.value()); - Assertions.assertEquals("xephwxd", model.activities().get(0).name()); - Assertions.assertEquals("ow", model.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("jca", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, + Assertions.assertEquals("lj", model.value()); + Assertions.assertEquals("mpydkgbcufhk", model.activities().get(0).name()); + Assertions.assertEquals("xxzhqouon", model.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.activities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("wdwzf", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("vftij", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("ehxvktlrc", model.activities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseSourceTests.java index cd3969e3ba268..2a1e7295b3250 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseSourceTests.java @@ -11,15 +11,16 @@ public final class SybaseSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SybaseSource model = BinaryData.fromString( - "{\"type\":\"SybaseSource\",\"query\":\"databro\",\"queryTimeout\":\"datamufaz\",\"additionalColumns\":\"datawzbew\",\"sourceRetryCount\":\"datasspe\",\"sourceRetryWait\":\"datajhwpn\",\"maxConcurrentConnections\":\"dataoucjzbovuvmdzdqt\",\"disableMetricsCollection\":\"dataguajsrdecbowkh\",\"\":{\"sujx\":\"datafllpdn\",\"aeykueat\":\"dataueqljzkhn\",\"ahvlzgsqwiubgb\":\"datatnprnshln\",\"cuxuxaihheg\":\"datatjyisjscuwylukt\"}}") + "{\"type\":\"SybaseSource\",\"query\":\"datasyszdtgwmqcutkkp\",\"queryTimeout\":\"dataurtmccdejtoypl\",\"additionalColumns\":\"datavjutckfhmdcvlb\",\"sourceRetryCount\":\"dataezvujpbmz\",\"sourceRetryWait\":\"datalgm\",\"maxConcurrentConnections\":\"dataxwkkbnhmdtj\",\"disableMetricsCollection\":\"datapfoispchhvvmvs\",\"\":{\"ggtsovozy\":\"dataqdhazmc\",\"ugubob\":\"datapkrncjrq\"}}") .toObject(SybaseSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SybaseSource model = new SybaseSource().withSourceRetryCount("datasspe").withSourceRetryWait("datajhwpn") - .withMaxConcurrentConnections("dataoucjzbovuvmdzdqt").withDisableMetricsCollection("dataguajsrdecbowkh") - .withQueryTimeout("datamufaz").withAdditionalColumns("datawzbew").withQuery("databro"); + SybaseSource model = new SybaseSource().withSourceRetryCount("dataezvujpbmz").withSourceRetryWait("datalgm") + .withMaxConcurrentConnections("dataxwkkbnhmdtj").withDisableMetricsCollection("datapfoispchhvvmvs") + .withQueryTimeout("dataurtmccdejtoypl").withAdditionalColumns("datavjutckfhmdcvlb") + .withQuery("datasyszdtgwmqcutkkp"); model = BinaryData.fromObject(model).toObject(SybaseSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTests.java index d8e7f66914c05..2a32674d3a749 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTests.java @@ -19,30 +19,30 @@ public final class SybaseTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SybaseTableDataset model = BinaryData.fromString( - "{\"type\":\"SybaseTable\",\"typeProperties\":{\"tableName\":\"dataokez\"},\"description\":\"ezknfzqnzbflbqmh\",\"structure\":\"datayxxvwedhagqbbse\",\"schema\":\"dataayuflmsyz\",\"linkedServiceName\":{\"referenceName\":\"dcrolrze\",\"parameters\":{\"ivt\":\"datamphzkymunw\",\"wdalisd\":\"datauszbdjrdfeuj\",\"dz\":\"dataqngca\",\"p\":\"datanloou\"}},\"parameters\":{\"iaffj\":{\"type\":\"Object\",\"defaultValue\":\"datahyclxrsidoebldp\"},\"x\":{\"type\":\"Bool\",\"defaultValue\":\"datanhrevimxm\"}},\"annotations\":[\"datapitygv\",\"datawdsoqtbfkvuozbzc\"],\"folder\":{\"name\":\"ekwanklp\"},\"\":{\"kjse\":\"datacydjh\",\"rdonkgobx\":\"datawiynd\",\"olenrswknpdr\":\"datalr\"}}") + "{\"type\":\"SybaseTable\",\"typeProperties\":{\"tableName\":\"datacrolrzesbomp\"},\"description\":\"kymunwjivtb\",\"structure\":\"datazbdjrdfeujywdal\",\"schema\":\"datadeqngc\",\"linkedServiceName\":{\"referenceName\":\"ydzin\",\"parameters\":{\"xrsi\":\"dataulpozmdahyc\",\"oiaf\":\"dataoebld\",\"x\":\"datajkrtnhrevimxm\"}},\"parameters\":{\"oqtbfkvuozbzc\":{\"type\":\"Object\",\"defaultValue\":\"datatygvdwd\"},\"rlcydjht\":{\"type\":\"Object\",\"defaultValue\":\"dataekwanklp\"}},\"annotations\":[\"dataerwi\",\"datandurdonkgobxbl\",\"datadolenrsw\",\"datanpdrgnmzaofroe\"],\"folder\":{\"name\":\"kievyrej\"},\"\":{\"ftusdwmnrt\":\"databk\",\"nrovome\":\"datavbuc\"}}") .toObject(SybaseTableDataset.class); - Assertions.assertEquals("ezknfzqnzbflbqmh", model.description()); - Assertions.assertEquals("dcrolrze", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("iaffj").type()); - Assertions.assertEquals("ekwanklp", model.folder().name()); + Assertions.assertEquals("kymunwjivtb", model.description()); + Assertions.assertEquals("ydzin", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("oqtbfkvuozbzc").type()); + Assertions.assertEquals("kievyrej", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SybaseTableDataset model = new SybaseTableDataset().withDescription("ezknfzqnzbflbqmh") - .withStructure("datayxxvwedhagqbbse").withSchema("dataayuflmsyz") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("dcrolrze").withParameters( - mapOf("ivt", "datamphzkymunw", "wdalisd", "datauszbdjrdfeuj", "dz", "dataqngca", "p", "datanloou"))) - .withParameters(mapOf("iaffj", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datahyclxrsidoebldp"), - "x", new ParameterSpecification().withType(ParameterType.BOOL).withDefaultValue("datanhrevimxm"))) - .withAnnotations(Arrays.asList("datapitygv", "datawdsoqtbfkvuozbzc")) - .withFolder(new DatasetFolder().withName("ekwanklp")).withTableName("dataokez"); + SybaseTableDataset model = new SybaseTableDataset().withDescription("kymunwjivtb") + .withStructure("datazbdjrdfeujywdal").withSchema("datadeqngc") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ydzin") + .withParameters(mapOf("xrsi", "dataulpozmdahyc", "oiaf", "dataoebld", "x", "datajkrtnhrevimxm"))) + .withParameters(mapOf("oqtbfkvuozbzc", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datatygvdwd"), "rlcydjht", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("dataekwanklp"))) + .withAnnotations(Arrays.asList("dataerwi", "datandurdonkgobxbl", "datadolenrsw", "datanpdrgnmzaofroe")) + .withFolder(new DatasetFolder().withName("kievyrej")).withTableName("datacrolrzesbomp"); model = BinaryData.fromObject(model).toObject(SybaseTableDataset.class); - Assertions.assertEquals("ezknfzqnzbflbqmh", model.description()); - Assertions.assertEquals("dcrolrze", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("iaffj").type()); - Assertions.assertEquals("ekwanklp", model.folder().name()); + Assertions.assertEquals("kymunwjivtb", model.description()); + Assertions.assertEquals("ydzin", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("oqtbfkvuozbzc").type()); + Assertions.assertEquals("kievyrej", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTypePropertiesTests.java index a488a0166d1a2..56e850d0e9774 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SybaseTableDatasetTypePropertiesTests.java @@ -10,13 +10,13 @@ public final class SybaseTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SybaseTableDatasetTypeProperties model - = BinaryData.fromString("{\"tableName\":\"datamzaof\"}").toObject(SybaseTableDatasetTypeProperties.class); + SybaseTableDatasetTypeProperties model = BinaryData.fromString("{\"tableName\":\"datawsicvwqzoc\"}") + .toObject(SybaseTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SybaseTableDatasetTypeProperties model = new SybaseTableDatasetTypeProperties().withTableName("datamzaof"); + SybaseTableDatasetTypeProperties model = new SybaseTableDatasetTypeProperties().withTableName("datawsicvwqzoc"); model = BinaryData.fromObject(model).toObject(SybaseTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTests.java index 74384791d5e70..2a98a0bc1f025 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTests.java @@ -31,22 +31,22 @@ public final class SynapseNotebookActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SynapseNotebookActivity model = BinaryData.fromString( - "{\"type\":\"SynapseNotebook\",\"typeProperties\":{\"notebook\":{\"type\":\"NotebookReference\",\"referenceName\":\"dataocepjsfhxhulrekr\"},\"sparkPool\":{\"type\":\"BigDataPoolReference\",\"referenceName\":\"datadnzrcjokgthy\"},\"parameters\":{\"vyceks\":{\"value\":\"datarwlguesoivaoryef\",\"type\":\"bool\"}},\"executorSize\":\"datat\",\"conf\":\"datagmfjzqv\",\"driverSize\":\"datadhixd\",\"numExecutors\":\"dataocsmcqs\",\"configurationType\":\"Customized\",\"targetSparkConfiguration\":{\"type\":\"SparkConfigurationReference\",\"referenceName\":\"dataa\"},\"sparkConfig\":{\"heoxs\":\"datazjdrkc\",\"tlxqhyy\":\"dataf\"}},\"linkedServiceName\":{\"referenceName\":\"zgxkwcqpv\",\"parameters\":{\"fylbokbriy\":\"datalkrroqsdvxdd\",\"buravswnnsb\":\"datarxae\",\"ojyn\":\"datakumxbcn\",\"fxiaptsukdoy\":\"datahbtycfj\"}},\"policy\":{\"timeout\":\"datazniekedxvw\",\"retry\":\"dataipxzzcxqd\",\"retryIntervalInSeconds\":2074445681,\"secureInput\":true,\"secureOutput\":true,\"\":{\"oxqwcusls\":\"datayb\",\"zwybbewjvyrd\":\"datatzq\",\"bwr\":\"dataw\"}},\"name\":\"bm\",\"description\":\"lmzaruosmpcajxua\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"lwrqhehnazck\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\"],\"\":{\"oahektwgiumcco\":\"datazgi\",\"dhrkhfyaxi\":\"datajxxjaafr\"}},{\"activity\":\"cnzsimbgv\",\"dependencyConditions\":[\"Failed\",\"Succeeded\"],\"\":{\"wwyubkppocjyj\":\"dataqparbog\"}},{\"activity\":\"emgbkjxuxm\",\"dependencyConditions\":[\"Failed\"],\"\":{\"fekpgllezvrvjws\":\"datajfeanbn\",\"jbsvk\":\"datafkzlv\"}},{\"activity\":\"jynvguhqugnqs\",\"dependencyConditions\":[\"Skipped\",\"Succeeded\"],\"\":{\"xxsybtpqgxz\":\"datawhmncewcfinsoi\",\"sicnckdxflgj\":\"datagcl\",\"ilcerrpalxm\":\"databtce\",\"u\":\"datasbgj\"}}],\"userProperties\":[{\"name\":\"j\",\"value\":\"datanabyvmch\"},{\"name\":\"kwlmittpbivhkdxh\",\"value\":\"datavybxplbdaz\"}],\"\":{\"zu\":\"datagvd\"}}") + "{\"type\":\"SynapseNotebook\",\"typeProperties\":{\"notebook\":{\"type\":\"NotebookReference\",\"referenceName\":\"datamcblmza\"},\"sparkPool\":{\"type\":\"BigDataPoolReference\",\"referenceName\":\"datasmpcajx\"},\"parameters\":{\"pysg\":{\"value\":\"datapdkrjlwrqhehnazc\",\"type\":\"int\"},\"iumccomjxx\":{\"value\":\"dataivoahek\",\"type\":\"bool\"},\"hrkhfyaxiw\":{\"value\":\"dataaf\",\"type\":\"float\"},\"rbogzwwyub\":{\"value\":\"datazsimbgvrksjjq\",\"type\":\"string\"}},\"executorSize\":\"datapocjyjqem\",\"conf\":\"datakjxuxmkkgbyjfe\",\"driverSize\":\"databnwfekpgllezvr\",\"numExecutors\":\"datawsffk\",\"configurationType\":\"Customized\",\"targetSparkConfiguration\":{\"type\":\"SparkConfigurationReference\",\"referenceName\":\"databsvkijynvguh\"},\"sparkConfig\":{\"sclrvquwhmncewcf\":\"datan\",\"xsybtpqgxzogclu\":\"datansoim\",\"i\":\"dataicnckdxflg\"}},\"linkedServiceName\":{\"referenceName\":\"ce\",\"parameters\":{\"mrsbgjjuhzf\":\"datacerrpal\"}},\"policy\":{\"timeout\":\"dataabyvmchhkwlmit\",\"retry\":\"databivhkdxhnv\",\"retryIntervalInSeconds\":559455759,\"secureInput\":true,\"secureOutput\":true,\"\":{\"bgv\":\"datas\",\"vvhovkadmih\":\"datazzukhl\",\"nl\":\"datab\",\"fjqobbpjlrvxryjx\":\"datajzdahckijvikpgz\"}},\"name\":\"dlgignja\",\"description\":\"cixwtwzgbuhcrwqr\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Skipped\",\"dependsOn\":[{\"activity\":\"mfufs\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Skipped\",\"Skipped\"],\"\":{\"lcqaafuwxeho\":\"datae\",\"q\":\"dataazbgcbd\",\"vtimyccdognhw\":\"datay\"}},{\"activity\":\"vgowkak\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"dv\":\"datajiykwbytuzhcpx\",\"y\":\"datafxvrfez\"}},{\"activity\":\"iyovcrmoalvea\",\"dependencyConditions\":[\"Failed\"],\"\":{\"jpedowmhgzrri\":\"datazr\",\"qqfycw\":\"datavyugxnopd\",\"ki\":\"dataupxf\"}},{\"activity\":\"mhvpxptq\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Failed\",\"Failed\"],\"\":{\"spnsbbhdjee\":\"datamxnjk\"}}],\"userProperties\":[{\"name\":\"cykihym\",\"value\":\"datagukf\"},{\"name\":\"kqok\",\"value\":\"datavxknygimoh\"},{\"name\":\"llxjyxhwv\",\"value\":\"datayupszch\"},{\"name\":\"wnudd\",\"value\":\"dataazvsmnxblc\"}],\"\":{\"yn\":\"dataymgfwdxukmeoxe\",\"uqm\":\"datarbwvai\",\"ztorvwgpjxdii\":\"dataaqoqjnvmfmrymk\"}}") .toObject(SynapseNotebookActivity.class); - Assertions.assertEquals("bm", model.name()); - Assertions.assertEquals("lmzaruosmpcajxua", model.description()); + Assertions.assertEquals("dlgignja", model.name()); + Assertions.assertEquals("cixwtwzgbuhcrwqr", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("lwrqhehnazck", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("j", model.userProperties().get(0).name()); - Assertions.assertEquals("zgxkwcqpv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(2074445681, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("mfufs", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("cykihym", model.userProperties().get(0).name()); + Assertions.assertEquals("ce", model.linkedServiceName().referenceName()); + Assertions.assertEquals(559455759, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); Assertions.assertEquals(NotebookReferenceType.NOTEBOOK_REFERENCE, model.notebook().type()); Assertions.assertEquals(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE, model.sparkPool().type()); - Assertions.assertEquals(NotebookParameterType.BOOL, model.parameters().get("vyceks").type()); + Assertions.assertEquals(NotebookParameterType.INT, model.parameters().get("pysg").type()); Assertions.assertEquals(ConfigurationType.CUSTOMIZED, model.configurationType()); Assertions.assertEquals(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE, model.targetSparkConfiguration().type()); @@ -54,59 +54,64 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SynapseNotebookActivity model - = new SynapseNotebookActivity().withName("bm").withDescription("lmzaruosmpcajxua") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("lwrqhehnazck") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("cnzsimbgv") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("emgbkjxuxm") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jynvguhqugnqs") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("j").withValue("datanabyvmch"), - new UserProperty().withName("kwlmittpbivhkdxh").withValue("datavybxplbdaz"))) - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("zgxkwcqpv") - .withParameters(mapOf("fylbokbriy", "datalkrroqsdvxdd", "buravswnnsb", "datarxae", "ojyn", - "datakumxbcn", "fxiaptsukdoy", "datahbtycfj"))) - .withPolicy(new ActivityPolicy().withTimeout("datazniekedxvw").withRetry("dataipxzzcxqd") - .withRetryIntervalInSeconds(2074445681).withSecureInput(true).withSecureOutput(true) - .withAdditionalProperties(mapOf())) - .withNotebook(new SynapseNotebookReference().withType(NotebookReferenceType.NOTEBOOK_REFERENCE) - .withReferenceName("dataocepjsfhxhulrekr")) - .withSparkPool(new BigDataPoolParametrizationReference() - .withType(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE).withReferenceName("datadnzrcjokgthy")) - .withParameters(mapOf("vyceks", - new NotebookParameter().withValue("datarwlguesoivaoryef").withType(NotebookParameterType.BOOL))) - .withExecutorSize("datat").withConf("datagmfjzqv").withDriverSize("datadhixd") - .withNumExecutors("dataocsmcqs").withConfigurationType(ConfigurationType.CUSTOMIZED) - .withTargetSparkConfiguration(new SparkConfigurationParametrizationReference() - .withType(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE).withReferenceName("dataa")) - .withSparkConfig(mapOf("heoxs", "datazjdrkc", "tlxqhyy", "dataf")); + SynapseNotebookActivity model = new SynapseNotebookActivity().withName("dlgignja") + .withDescription("cixwtwzgbuhcrwqr").withState(ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SKIPPED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("mfufs") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, + DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("vgowkak") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("iyovcrmoalvea") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("mhvpxptq") + .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, + DependencyCondition.FAILED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("cykihym").withValue("datagukf"), + new UserProperty().withName("kqok").withValue("datavxknygimoh"), + new UserProperty().withName("llxjyxhwv").withValue("datayupszch"), + new UserProperty().withName("wnudd").withValue("dataazvsmnxblc"))) + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("ce") + .withParameters(mapOf("mrsbgjjuhzf", "datacerrpal"))) + .withPolicy(new ActivityPolicy().withTimeout("dataabyvmchhkwlmit").withRetry("databivhkdxhnv") + .withRetryIntervalInSeconds(559455759).withSecureInput(true).withSecureOutput(true) + .withAdditionalProperties(mapOf())) + .withNotebook(new SynapseNotebookReference().withType(NotebookReferenceType.NOTEBOOK_REFERENCE) + .withReferenceName("datamcblmza")) + .withSparkPool(new BigDataPoolParametrizationReference() + .withType(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE).withReferenceName("datasmpcajx")) + .withParameters(mapOf("pysg", + new NotebookParameter().withValue("datapdkrjlwrqhehnazc").withType(NotebookParameterType.INT), + "iumccomjxx", new NotebookParameter().withValue("dataivoahek").withType(NotebookParameterType.BOOL), + "hrkhfyaxiw", new NotebookParameter().withValue("dataaf").withType(NotebookParameterType.FLOAT), + "rbogzwwyub", + new NotebookParameter().withValue("datazsimbgvrksjjq").withType(NotebookParameterType.STRING))) + .withExecutorSize("datapocjyjqem").withConf("datakjxuxmkkgbyjfe").withDriverSize("databnwfekpgllezvr") + .withNumExecutors("datawsffk").withConfigurationType(ConfigurationType.CUSTOMIZED) + .withTargetSparkConfiguration(new SparkConfigurationParametrizationReference() + .withType(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE) + .withReferenceName("databsvkijynvguh")) + .withSparkConfig(mapOf("sclrvquwhmncewcf", "datan", "xsybtpqgxzogclu", "datansoim", "i", "dataicnckdxflg")); model = BinaryData.fromObject(model).toObject(SynapseNotebookActivity.class); - Assertions.assertEquals("bm", model.name()); - Assertions.assertEquals("lmzaruosmpcajxua", model.description()); + Assertions.assertEquals("dlgignja", model.name()); + Assertions.assertEquals("cixwtwzgbuhcrwqr", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("lwrqhehnazck", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("j", model.userProperties().get(0).name()); - Assertions.assertEquals("zgxkwcqpv", model.linkedServiceName().referenceName()); - Assertions.assertEquals(2074445681, model.policy().retryIntervalInSeconds()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SKIPPED, model.onInactiveMarkAs()); + Assertions.assertEquals("mfufs", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("cykihym", model.userProperties().get(0).name()); + Assertions.assertEquals("ce", model.linkedServiceName().referenceName()); + Assertions.assertEquals(559455759, model.policy().retryIntervalInSeconds()); Assertions.assertEquals(true, model.policy().secureInput()); Assertions.assertEquals(true, model.policy().secureOutput()); Assertions.assertEquals(NotebookReferenceType.NOTEBOOK_REFERENCE, model.notebook().type()); Assertions.assertEquals(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE, model.sparkPool().type()); - Assertions.assertEquals(NotebookParameterType.BOOL, model.parameters().get("vyceks").type()); + Assertions.assertEquals(NotebookParameterType.INT, model.parameters().get("pysg").type()); Assertions.assertEquals(ConfigurationType.CUSTOMIZED, model.configurationType()); Assertions.assertEquals(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE, model.targetSparkConfiguration().type()); diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTypePropertiesTests.java index 187e0d9ab7a82..25929a5e01ef8 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookActivityTypePropertiesTests.java @@ -23,12 +23,12 @@ public final class SynapseNotebookActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SynapseNotebookActivityTypeProperties model = BinaryData.fromString( - "{\"notebook\":{\"type\":\"NotebookReference\",\"referenceName\":\"datalwvvho\"},\"sparkPool\":{\"type\":\"BigDataPoolReference\",\"referenceName\":\"datad\"},\"parameters\":{\"ejzd\":{\"value\":\"datab\",\"type\":\"float\"},\"gzkfjq\":{\"value\":\"datackijv\",\"type\":\"int\"}},\"executorSize\":\"databpjlr\",\"conf\":\"dataryjxj\",\"driverSize\":\"datagi\",\"numExecutors\":\"datajalccixwtwzg\",\"configurationType\":\"Default\",\"targetSparkConfiguration\":{\"type\":\"SparkConfigurationReference\",\"referenceName\":\"datawqr\"},\"sparkConfig\":{\"mfufs\":\"dataznz\",\"uwxehozazb\":\"datavsjiojvetlcqaa\",\"imyccdog\":\"datacbdkqrywv\",\"kakdjnszj\":\"datahwhvgo\"}}") + "{\"notebook\":{\"type\":\"NotebookReference\",\"referenceName\":\"datatdzhkbcouavotfm\"},\"sparkPool\":{\"type\":\"BigDataPoolReference\",\"referenceName\":\"datazvydzqmlk\"},\"parameters\":{\"ukjirtiu\":{\"value\":\"datapbbjcznxdhiwaa\",\"type\":\"string\"},\"msexaejb\":{\"value\":\"datayudkgonrrarzn\",\"type\":\"string\"}},\"executorSize\":\"dataoune\",\"conf\":\"datafhclssedxiig\",\"driverSize\":\"datazwqjpudupishcvsj\",\"numExecutors\":\"dataedsqfdulndywghn\",\"configurationType\":\"Artifact\",\"targetSparkConfiguration\":{\"type\":\"SparkConfigurationReference\",\"referenceName\":\"dataljnromhsia\"},\"sparkConfig\":{\"pmsyhrvifurg\":\"datahpelqckwc\"}}") .toObject(SynapseNotebookActivityTypeProperties.class); Assertions.assertEquals(NotebookReferenceType.NOTEBOOK_REFERENCE, model.notebook().type()); Assertions.assertEquals(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE, model.sparkPool().type()); - Assertions.assertEquals(NotebookParameterType.FLOAT, model.parameters().get("ejzd").type()); - Assertions.assertEquals(ConfigurationType.DEFAULT, model.configurationType()); + Assertions.assertEquals(NotebookParameterType.STRING, model.parameters().get("ukjirtiu").type()); + Assertions.assertEquals(ConfigurationType.ARTIFACT, model.configurationType()); Assertions.assertEquals(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE, model.targetSparkConfiguration().type()); } @@ -37,23 +37,24 @@ public void testDeserialize() throws Exception { public void testSerialize() throws Exception { SynapseNotebookActivityTypeProperties model = new SynapseNotebookActivityTypeProperties() .withNotebook(new SynapseNotebookReference().withType(NotebookReferenceType.NOTEBOOK_REFERENCE) - .withReferenceName("datalwvvho")) + .withReferenceName("datatdzhkbcouavotfm")) .withSparkPool(new BigDataPoolParametrizationReference() - .withType(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE).withReferenceName("datad")) - .withParameters( - mapOf("ejzd", new NotebookParameter().withValue("datab").withType(NotebookParameterType.FLOAT), - "gzkfjq", new NotebookParameter().withValue("datackijv").withType(NotebookParameterType.INT))) - .withExecutorSize("databpjlr").withConf("dataryjxj").withDriverSize("datagi") - .withNumExecutors("datajalccixwtwzg").withConfigurationType(ConfigurationType.DEFAULT) + .withType(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE).withReferenceName("datazvydzqmlk")) + .withParameters(mapOf("ukjirtiu", + new NotebookParameter().withValue("datapbbjcznxdhiwaa").withType(NotebookParameterType.STRING), + "msexaejb", + new NotebookParameter().withValue("datayudkgonrrarzn").withType(NotebookParameterType.STRING))) + .withExecutorSize("dataoune").withConf("datafhclssedxiig").withDriverSize("datazwqjpudupishcvsj") + .withNumExecutors("dataedsqfdulndywghn").withConfigurationType(ConfigurationType.ARTIFACT) .withTargetSparkConfiguration(new SparkConfigurationParametrizationReference() - .withType(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE).withReferenceName("datawqr")) - .withSparkConfig(mapOf("mfufs", "dataznz", "uwxehozazb", "datavsjiojvetlcqaa", "imyccdog", "datacbdkqrywv", - "kakdjnszj", "datahwhvgo")); + .withType(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE) + .withReferenceName("dataljnromhsia")) + .withSparkConfig(mapOf("pmsyhrvifurg", "datahpelqckwc")); model = BinaryData.fromObject(model).toObject(SynapseNotebookActivityTypeProperties.class); Assertions.assertEquals(NotebookReferenceType.NOTEBOOK_REFERENCE, model.notebook().type()); Assertions.assertEquals(BigDataPoolReferenceType.BIG_DATA_POOL_REFERENCE, model.sparkPool().type()); - Assertions.assertEquals(NotebookParameterType.FLOAT, model.parameters().get("ejzd").type()); - Assertions.assertEquals(ConfigurationType.DEFAULT, model.configurationType()); + Assertions.assertEquals(NotebookParameterType.STRING, model.parameters().get("ukjirtiu").type()); + Assertions.assertEquals(ConfigurationType.ARTIFACT, model.configurationType()); Assertions.assertEquals(SparkConfigurationReferenceType.SPARK_CONFIGURATION_REFERENCE, model.targetSparkConfiguration().type()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookReferenceTests.java index 0b4e13f344e5f..9a62bed492f9b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseNotebookReferenceTests.java @@ -13,7 +13,7 @@ public final class SynapseNotebookReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SynapseNotebookReference model - = BinaryData.fromString("{\"type\":\"NotebookReference\",\"referenceName\":\"datakwbytuzhcpxtdvy\"}") + = BinaryData.fromString("{\"type\":\"NotebookReference\",\"referenceName\":\"datahoqfvuqimdgkvf\"}") .toObject(SynapseNotebookReference.class); Assertions.assertEquals(NotebookReferenceType.NOTEBOOK_REFERENCE, model.type()); } @@ -21,7 +21,7 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SynapseNotebookReference model = new SynapseNotebookReference() - .withType(NotebookReferenceType.NOTEBOOK_REFERENCE).withReferenceName("datakwbytuzhcpxtdvy"); + .withType(NotebookReferenceType.NOTEBOOK_REFERENCE).withReferenceName("datahoqfvuqimdgkvf"); model = BinaryData.fromObject(model).toObject(SynapseNotebookReference.class); Assertions.assertEquals(NotebookReferenceType.NOTEBOOK_REFERENCE, model.type()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseSparkJobReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseSparkJobReferenceTests.java index 902ac684d21d6..0fe416e1cac30 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseSparkJobReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/SynapseSparkJobReferenceTests.java @@ -13,7 +13,7 @@ public final class SynapseSparkJobReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SynapseSparkJobReference model - = BinaryData.fromString("{\"type\":\"SparkJobDefinitionReference\",\"referenceName\":\"dataonrrarznlr\"}") + = BinaryData.fromString("{\"type\":\"SparkJobDefinitionReference\",\"referenceName\":\"databow\"}") .toObject(SynapseSparkJobReference.class); Assertions.assertEquals(SparkJobReferenceType.SPARK_JOB_DEFINITION_REFERENCE, model.type()); } @@ -21,7 +21,7 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SynapseSparkJobReference model = new SynapseSparkJobReference() - .withType(SparkJobReferenceType.SPARK_JOB_DEFINITION_REFERENCE).withReferenceName("dataonrrarznlr"); + .withType(SparkJobReferenceType.SPARK_JOB_DEFINITION_REFERENCE).withReferenceName("databow"); model = BinaryData.fromObject(model).toObject(SynapseSparkJobReference.class); Assertions.assertEquals(SparkJobReferenceType.SPARK_JOB_DEFINITION_REFERENCE, model.type()); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularSourceTests.java index 6c136c4a5bddc..117bd97783740 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularSourceTests.java @@ -11,16 +11,16 @@ public final class TabularSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TabularSource model = BinaryData.fromString( - "{\"type\":\"TabularSource\",\"queryTimeout\":\"datanycchpcjztz\",\"additionalColumns\":\"datauuyvpcfvinjxc\",\"sourceRetryCount\":\"datanetcxgdgqkletlw\",\"sourceRetryWait\":\"datatzbphxxvftj\",\"maxConcurrentConnections\":\"dataqg\",\"disableMetricsCollection\":\"datanmok\",\"\":{\"eeyskbqlcakle\":\"datatdnym\",\"okhbqmx\":\"datajgxdhgezyhph\",\"tyx\":\"dataglkqitpbyn\"}}") + "{\"type\":\"TabularSource\",\"queryTimeout\":\"dataqdswfno\",\"additionalColumns\":\"dataiwhumngihfndsj\",\"sourceRetryCount\":\"datailfvrpbcgd\",\"sourceRetryWait\":\"datafxoffckejxomngu\",\"maxConcurrentConnections\":\"dataxxynt\",\"disableMetricsCollection\":\"datanksvximgn\",\"\":{\"mwfoummdomvditp\":\"datacxuyzrnngnmf\",\"sfnoczefgfqxejj\":\"dataqalwlirapqhsidf\",\"fcrb\":\"datatiqbxzeiudog\"}}") .toObject(TabularSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { TabularSource model - = new TabularSource().withSourceRetryCount("datanetcxgdgqkletlw").withSourceRetryWait("datatzbphxxvftj") - .withMaxConcurrentConnections("dataqg").withDisableMetricsCollection("datanmok") - .withQueryTimeout("datanycchpcjztz").withAdditionalColumns("datauuyvpcfvinjxc"); + = new TabularSource().withSourceRetryCount("datailfvrpbcgd").withSourceRetryWait("datafxoffckejxomngu") + .withMaxConcurrentConnections("dataxxynt").withDisableMetricsCollection("datanksvximgn") + .withQueryTimeout("dataqdswfno").withAdditionalColumns("dataiwhumngihfndsj"); model = BinaryData.fromObject(model).toObject(TabularSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularTranslatorTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularTranslatorTests.java index 33f7557589e7d..15005558262bf 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularTranslatorTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TabularTranslatorTests.java @@ -12,18 +12,18 @@ public final class TabularTranslatorTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TabularTranslator model = BinaryData.fromString( - "{\"type\":\"TabularTranslator\",\"columnMappings\":\"datadqilzogilgrqzwy\",\"schemaMapping\":\"datahfybflr\",\"collectionReference\":\"datacgqq\",\"mapComplexValuesToString\":\"dataksghpsqvuisedeqr\",\"mappings\":\"dataj\",\"typeConversion\":\"dataxnqrq\",\"typeConversionSettings\":{\"allowDataTruncation\":\"databtpvwx\",\"treatBooleanAsNumber\":\"datas\",\"dateTimeFormat\":\"datacvpagwohkro\",\"dateTimeOffsetFormat\":\"datasse\",\"timeSpanFormat\":\"dataozjyovrll\",\"culture\":\"databgkgjp\"},\"\":{\"pfbiqjrzfrxizor\":\"datathflgpsalynanmw\"}}") + "{\"type\":\"TabularTranslator\",\"columnMappings\":\"dataqvdivzjyx\",\"schemaMapping\":\"datablblxjbrqbutmacn\",\"collectionReference\":\"datadmyduvawea\",\"mapComplexValuesToString\":\"datafbvbvkw\",\"mappings\":\"datarz\",\"typeConversion\":\"datayymh\",\"typeConversionSettings\":{\"allowDataTruncation\":\"dataobhltmpay\",\"treatBooleanAsNumber\":\"dataqgrsytto\",\"dateTimeFormat\":\"datazbbxifacrhpuzcag\",\"dateTimeOffsetFormat\":\"datavpbwt\",\"timeSpanFormat\":\"datauiguo\",\"culture\":\"datao\"},\"\":{\"gdv\":\"databuexrkoxwyxodp\"}}") .toObject(TabularTranslator.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TabularTranslator model = new TabularTranslator().withColumnMappings("datadqilzogilgrqzwy") - .withSchemaMapping("datahfybflr").withCollectionReference("datacgqq") - .withMapComplexValuesToString("dataksghpsqvuisedeqr").withMappings("dataj").withTypeConversion("dataxnqrq") - .withTypeConversionSettings(new TypeConversionSettings().withAllowDataTruncation("databtpvwx") - .withTreatBooleanAsNumber("datas").withDateTimeFormat("datacvpagwohkro") - .withDateTimeOffsetFormat("datasse").withTimeSpanFormat("dataozjyovrll").withCulture("databgkgjp")); + TabularTranslator model = new TabularTranslator().withColumnMappings("dataqvdivzjyx") + .withSchemaMapping("datablblxjbrqbutmacn").withCollectionReference("datadmyduvawea") + .withMapComplexValuesToString("datafbvbvkw").withMappings("datarz").withTypeConversion("datayymh") + .withTypeConversionSettings(new TypeConversionSettings().withAllowDataTruncation("dataobhltmpay") + .withTreatBooleanAsNumber("dataqgrsytto").withDateTimeFormat("datazbbxifacrhpuzcag") + .withDateTimeOffsetFormat("datavpbwt").withTimeSpanFormat("datauiguo").withCulture("datao")); model = BinaryData.fromObject(model).toObject(TabularTranslator.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarGZipReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarGZipReadSettingsTests.java index f672c75b914c3..84dfe83112c9e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarGZipReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarGZipReadSettingsTests.java @@ -11,13 +11,14 @@ public final class TarGZipReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TarGZipReadSettings model = BinaryData.fromString( - "{\"type\":\"TarGZipReadSettings\",\"preserveCompressionFileNameAsFolder\":\"datazijpyyve\",\"\":{\"dsthktsaljkhlpg\":\"datahqym\",\"xkkoypxwlvthiva\":\"datap\"}}") + "{\"type\":\"TarGZipReadSettings\",\"preserveCompressionFileNameAsFolder\":\"dataiaizsglavdttty\",\"\":{\"qkjqcsh\":\"dataomz\"}}") .toObject(TarGZipReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TarGZipReadSettings model = new TarGZipReadSettings().withPreserveCompressionFileNameAsFolder("datazijpyyve"); + TarGZipReadSettings model + = new TarGZipReadSettings().withPreserveCompressionFileNameAsFolder("dataiaizsglavdttty"); model = BinaryData.fromObject(model).toObject(TarGZipReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarReadSettingsTests.java index 31ea0305ebb4d..6f953272918f2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TarReadSettingsTests.java @@ -11,13 +11,13 @@ public final class TarReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TarReadSettings model = BinaryData.fromString( - "{\"type\":\"TarReadSettings\",\"preserveCompressionFileNameAsFolder\":\"dataekbpqghxdpg\",\"\":{\"l\":\"datai\",\"qoucmfuvuslvbujw\":\"dataxdmixjudbiac\"}}") + "{\"type\":\"TarReadSettings\",\"preserveCompressionFileNameAsFolder\":\"datapp\",\"\":{\"zjplaxxfn\":\"dataikktretutsy\",\"rotg\":\"dataltanvb\",\"rpmgd\":\"dataxk\"}}") .toObject(TarReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TarReadSettings model = new TarReadSettings().withPreserveCompressionFileNameAsFolder("dataekbpqghxdpg"); + TarReadSettings model = new TarReadSettings().withPreserveCompressionFileNameAsFolder("datapp"); model = BinaryData.fromObject(model).toObject(TarReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataPartitionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataPartitionSettingsTests.java index 2627a13e40a60..b03dc39b437ed 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataPartitionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataPartitionSettingsTests.java @@ -11,14 +11,14 @@ public final class TeradataPartitionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TeradataPartitionSettings model = BinaryData.fromString( - "{\"partitionColumnName\":\"datanihgo\",\"partitionUpperBound\":\"datayebwgesov\",\"partitionLowerBound\":\"datajxnsorsolarhlyhg\"}") + "{\"partitionColumnName\":\"datajwrhubgaaaxi\",\"partitionUpperBound\":\"datafahtt\",\"partitionLowerBound\":\"dataggzdoblpdtcyv\"}") .toObject(TeradataPartitionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TeradataPartitionSettings model = new TeradataPartitionSettings().withPartitionColumnName("datanihgo") - .withPartitionUpperBound("datayebwgesov").withPartitionLowerBound("datajxnsorsolarhlyhg"); + TeradataPartitionSettings model = new TeradataPartitionSettings().withPartitionColumnName("datajwrhubgaaaxi") + .withPartitionUpperBound("datafahtt").withPartitionLowerBound("dataggzdoblpdtcyv"); model = BinaryData.fromObject(model).toObject(TeradataPartitionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataSourceTests.java index d8b9a001bc529..f40cbd8c44b61 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TeradataSourceTests.java @@ -12,18 +12,18 @@ public final class TeradataSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TeradataSource model = BinaryData.fromString( - "{\"type\":\"TeradataSource\",\"query\":\"datamllyjelnhm\",\"partitionOption\":\"datahxkofzxkqsle\",\"partitionSettings\":{\"partitionColumnName\":\"dataamahnwg\",\"partitionUpperBound\":\"datagble\",\"partitionLowerBound\":\"datamvlbaxdaojahl\"},\"queryTimeout\":\"dataxwqlnxvnmrljm\",\"additionalColumns\":\"datayadafecwnufldzjc\",\"sourceRetryCount\":\"datahjbzpoh\",\"sourceRetryWait\":\"datajgpe\",\"maxConcurrentConnections\":\"databoxvwtlnv\",\"disableMetricsCollection\":\"datahtujaqpkup\",\"\":{\"pdkyscxzsyn\":\"datarje\",\"q\":\"datad\"}}") + "{\"type\":\"TeradataSource\",\"query\":\"datamnjtfplgxcjr\",\"partitionOption\":\"datab\",\"partitionSettings\":{\"partitionColumnName\":\"dataisfjamgnpeosu\",\"partitionUpperBound\":\"datayycofljabdmwal\",\"partitionLowerBound\":\"databuqkdieuop\"},\"queryTimeout\":\"dataaknhmi\",\"additionalColumns\":\"dataf\",\"sourceRetryCount\":\"datalfm\",\"sourceRetryWait\":\"datan\",\"maxConcurrentConnections\":\"dataiahoygzkdb\",\"disableMetricsCollection\":\"datazobcdvbbuuipe\",\"\":{\"x\":\"dataptteojxhwgja\",\"sl\":\"datarpwjgkxvkjd\",\"qi\":\"dataqmmwwtzxsv\"}}") .toObject(TeradataSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TeradataSource model = new TeradataSource().withSourceRetryCount("datahjbzpoh").withSourceRetryWait("datajgpe") - .withMaxConcurrentConnections("databoxvwtlnv").withDisableMetricsCollection("datahtujaqpkup") - .withQueryTimeout("dataxwqlnxvnmrljm").withAdditionalColumns("datayadafecwnufldzjc") - .withQuery("datamllyjelnhm").withPartitionOption("datahxkofzxkqsle") - .withPartitionSettings(new TeradataPartitionSettings().withPartitionColumnName("dataamahnwg") - .withPartitionUpperBound("datagble").withPartitionLowerBound("datamvlbaxdaojahl")); + TeradataSource model = new TeradataSource().withSourceRetryCount("datalfm").withSourceRetryWait("datan") + .withMaxConcurrentConnections("dataiahoygzkdb").withDisableMetricsCollection("datazobcdvbbuuipe") + .withQueryTimeout("dataaknhmi").withAdditionalColumns("dataf").withQuery("datamnjtfplgxcjr") + .withPartitionOption("datab") + .withPartitionSettings(new TeradataPartitionSettings().withPartitionColumnName("dataisfjamgnpeosu") + .withPartitionUpperBound("datayycofljabdmwal").withPartitionLowerBound("databuqkdieuop")); model = BinaryData.fromObject(model).toObject(TeradataSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerDependencyReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerDependencyReferenceTests.java index bc5ba1d1a84a6..0003706466092 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerDependencyReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerDependencyReferenceTests.java @@ -14,18 +14,18 @@ public final class TriggerDependencyReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TriggerDependencyReference model = BinaryData.fromString( - "{\"type\":\"TriggerDependencyReference\",\"referenceTrigger\":{\"type\":\"TriggerReference\",\"referenceName\":\"uj\"}}") + "{\"type\":\"TriggerDependencyReference\",\"referenceTrigger\":{\"type\":\"TriggerReference\",\"referenceName\":\"rxlgz\"}}") .toObject(TriggerDependencyReference.class); Assertions.assertEquals(TriggerReferenceType.TRIGGER_REFERENCE, model.referenceTrigger().type()); - Assertions.assertEquals("uj", model.referenceTrigger().referenceName()); + Assertions.assertEquals("rxlgz", model.referenceTrigger().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { TriggerDependencyReference model = new TriggerDependencyReference().withReferenceTrigger( - new TriggerReference().withType(TriggerReferenceType.TRIGGER_REFERENCE).withReferenceName("uj")); + new TriggerReference().withType(TriggerReferenceType.TRIGGER_REFERENCE).withReferenceName("rxlgz")); model = BinaryData.fromObject(model).toObject(TriggerDependencyReference.class); Assertions.assertEquals(TriggerReferenceType.TRIGGER_REFERENCE, model.referenceTrigger().type()); - Assertions.assertEquals("uj", model.referenceTrigger().referenceName()); + Assertions.assertEquals("rxlgz", model.referenceTrigger().referenceName()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerReferenceTests.java index 90a94919bc609..51244474c8a7b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerReferenceTests.java @@ -13,18 +13,18 @@ public final class TriggerReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TriggerReference model - = BinaryData.fromString("{\"type\":\"TriggerReference\",\"referenceName\":\"uekmzubdmcdfvwrl\"}") + = BinaryData.fromString("{\"type\":\"TriggerReference\",\"referenceName\":\"eacpwsdir\"}") .toObject(TriggerReference.class); Assertions.assertEquals(TriggerReferenceType.TRIGGER_REFERENCE, model.type()); - Assertions.assertEquals("uekmzubdmcdfvwrl", model.referenceName()); + Assertions.assertEquals("eacpwsdir", model.referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TriggerReference model = new TriggerReference().withType(TriggerReferenceType.TRIGGER_REFERENCE) - .withReferenceName("uekmzubdmcdfvwrl"); + TriggerReference model + = new TriggerReference().withType(TriggerReferenceType.TRIGGER_REFERENCE).withReferenceName("eacpwsdir"); model = BinaryData.fromObject(model).toObject(TriggerReference.class); Assertions.assertEquals(TriggerReferenceType.TRIGGER_REFERENCE, model.type()); - Assertions.assertEquals("uekmzubdmcdfvwrl", model.referenceName()); + Assertions.assertEquals("eacpwsdir", model.referenceName()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsCancelWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsCancelWithResponseMockTests.java index 6da130e3af105..c37de529c582c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsCancelWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsCancelWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testCancelWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.triggerRuns().cancelWithResponse("gebqhbbqodyvvp", "oiaaagvaecwwdqg", "o", "bhfrg", + manager.triggerRuns().cancelWithResponse("rwdfo", "rdytsgypvidzo", "qzdoy", "npkjpcgtgnhzufhw", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsRerunWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsRerunWithResponseMockTests.java index b1d0fb3aeaf86..1ca0739624a22 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsRerunWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggerRunsRerunWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testRerunWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.triggerRuns().rerunWithResponse("lkmwvgdfutds", "jtuq", "ojpauiccja", "a", + manager.triggerRuns().rerunWithResponse("edtwtukkhuusrm", "tonpgtaz", "upkebwses", "ls", com.azure.core.util.Context.NONE); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersCreateOrUpdateWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersCreateOrUpdateWithResponseMockTests.java index 28dce05fcd142..339b08d7d645e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersCreateOrUpdateWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersCreateOrUpdateWithResponseMockTests.java @@ -35,7 +35,7 @@ public void testCreateOrUpdateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"Trigger\",\"description\":\"twstqgc\",\"runtimeState\":\"Stopped\",\"annotations\":[\"datarzoafxoyddus\",\"datavyjh\",\"dataynlmxzdwpdw\"],\"\":{\"yznyeghm\":\"datau\",\"xdnckgdcszz\":\"datam\",\"ldeq\":\"dataedfdzleaz\",\"q\":\"datafzyhikhnwseftlj\"}},\"name\":\"fkdy\",\"type\":\"zaxithppjxtobe\",\"etag\":\"zcadoqijfll\",\"id\":\"uzeolcgqjtvp\"}"; + = "{\"properties\":{\"type\":\"Trigger\",\"description\":\"pbln\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datacij\",\"datan\",\"datavjskuw\",\"datarcxtyfbbomug\"],\"\":{\"ebpl\":\"datajvvdafbtozxvko\"}},\"name\":\"xmljnseaogqiybf\",\"type\":\"xuyo\",\"etag\":\"dryeucl\",\"id\":\"cwpgipttp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,14 +52,13 @@ public void testCreateOrUpdateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - TriggerResource response = manager.triggers().define("a").withExistingFactory("rklxnbbk", "jnnzqz") - .withProperties(new Trigger().withDescription("fxvlac") - .withAnnotations(Arrays.asList("dataksguccotgqge", "dataielyhow")) + TriggerResource response = manager.triggers().define("mdjlxzttgvawy").withExistingFactory("xrm", "ljqhoiqvk") + .withProperties(new Trigger().withDescription("xoh").withAnnotations(Arrays.asList("datatheukclayqipwkx")) .withAdditionalProperties(mapOf("type", "Trigger", "runtimeState", "Stopped"))) - .withIfMatch("lpiccx").create(); + .withIfMatch("mxlnt").create(); - Assertions.assertEquals("uzeolcgqjtvp", response.id()); - Assertions.assertEquals("twstqgc", response.properties().description()); + Assertions.assertEquals("cwpgipttp", response.id()); + Assertions.assertEquals("pbln", response.properties().description()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersDeleteWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersDeleteWithResponseMockTests.java index 9decc091c666c..5e7017357dc04 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersDeleteWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersDeleteWithResponseMockTests.java @@ -45,7 +45,7 @@ public void testDeleteWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.triggers().deleteWithResponse("rhhxlibdn", "eamslvpxsywn", "fvwrdy", com.azure.core.util.Context.NONE); + manager.triggers().deleteWithResponse("yki", "q", "gmlhszipih", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetEventSubscriptionStatusWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetEventSubscriptionStatusWithResponseMockTests.java index e3691f2b074b9..da613262a004e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetEventSubscriptionStatusWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetEventSubscriptionStatusWithResponseMockTests.java @@ -29,7 +29,7 @@ public void testGetEventSubscriptionStatusWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"triggerName\":\"lqioqtafmbxtncxb\",\"status\":\"Unknown\"}"; + String responseStr = "{\"triggerName\":\"oqgutr\",\"status\":\"Enabled\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,7 +47,7 @@ public void testGetEventSubscriptionStatusWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); TriggerSubscriptionOperationStatus response - = manager.triggers().getEventSubscriptionStatusWithResponse("wievjndvaf", "cvn", "yxlcgycvcspcfx", + = manager.triggers().getEventSubscriptionStatusWithResponse("cmmpjdrhxf", "swyafdlf", "yirjbfwrqivi", com.azure.core.util.Context.NONE).getValue(); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetWithResponseMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetWithResponseMockTests.java index 30d02f4a82ab9..9b6813f8b3a17 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetWithResponseMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"type\":\"Trigger\",\"description\":\"t\",\"runtimeState\":\"Stopped\",\"annotations\":[\"datamtyfgswpqunvxt\",\"datambwydqoqpqyjebg\",\"dataeuazwkzedifw\",\"datarrgzguupwqoohz\"],\"\":{\"dlxqjshyyrcr\":\"databsncorini\"}},\"name\":\"z\",\"type\":\"faurmqpk\",\"etag\":\"fb\",\"id\":\"fxh\"}"; + = "{\"properties\":{\"type\":\"Trigger\",\"description\":\"xuvc\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datawnayfkvwlfcf\",\"datayh\",\"datayn\"],\"\":{\"lbjccjorovr\":\"datauqnvnxsawicou\"}},\"name\":\"fgdvifoxoz\",\"type\":\"ymdjpkoj\",\"etag\":\"ytpyirctda\",\"id\":\"huk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,9 +49,10 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); TriggerResource response = manager.triggers() - .getWithResponse("zglgx", "qdlwuzkz", "hbieeswb", "bijtepr", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("btnvxomhkfk", "dmjjiqjvuf", "omfkiopkkhbf", "hsp", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("fxh", response.id()); - Assertions.assertEquals("t", response.properties().description()); + Assertions.assertEquals("huk", response.id()); + Assertions.assertEquals("xuvc", response.properties().description()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersListByFactoryMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersListByFactoryMockTests.java index 957b60f8d9829..fa28799b764f6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersListByFactoryMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersListByFactoryMockTests.java @@ -32,7 +32,7 @@ public void testListByFactory() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"type\":\"Trigger\",\"description\":\"nq\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datadztdjsxwuk\",\"datacwymbpredclvcl\"],\"\":{\"swasvey\":\"dataqsvtcrk\"}},\"name\":\"rbmff\",\"type\":\"yyykwwhs\",\"etag\":\"bgwzmanplzbzcgzh\",\"id\":\"vk\"}]}"; + = "{\"value\":[{\"properties\":{\"type\":\"Trigger\",\"description\":\"mxctkwx\",\"runtimeState\":\"Disabled\",\"annotations\":[\"databfeucdq\",\"datajazhtetimcjkh\",\"dataxx\"],\"\":{\"ssvrhcjhszmymfri\":\"datalcekonmcxriqfrrx\"}},\"name\":\"m\",\"type\":\"jkizqqdawmrk\",\"etag\":\"ixbbhjgnjlzdj\",\"id\":\"xwobxs\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,9 +50,9 @@ public void testListByFactory() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.triggers().listByFactory("wojt", "yrdzogtrycb", com.azure.core.util.Context.NONE); + = manager.triggers().listByFactory("ymsqnliou", "av", com.azure.core.util.Context.NONE); - Assertions.assertEquals("vk", response.iterator().next().id()); - Assertions.assertEquals("nq", response.iterator().next().properties().description()); + Assertions.assertEquals("xwobxs", response.iterator().next().id()); + Assertions.assertEquals("mxctkwx", response.iterator().next().properties().description()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStartMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStartMockTests.java index 6c65c7c35db13..cced69f0e5df9 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStartMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStartMockTests.java @@ -45,7 +45,7 @@ public void testStart() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.triggers().start("lflwqdjz", "og", "byks", com.azure.core.util.Context.NONE); + manager.triggers().start("sxlm", "lqtnngw", "gbfrtxbgaafttv", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStopMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStopMockTests.java index 796da3461cdd8..1679e7c19517f 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStopMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersStopMockTests.java @@ -45,7 +45,7 @@ public void testStop() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.triggers().stop("qxxy", "rrixkobmrrnkdmn", "qhkju", com.azure.core.util.Context.NONE); + manager.triggers().stop("mdnigajbxjnrlfdq", "afrwmxmd", "ezhutcaqqdch", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersSubscribeToEventsMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersSubscribeToEventsMockTests.java index 3b3f8634077c0..304263c549a69 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersSubscribeToEventsMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersSubscribeToEventsMockTests.java @@ -29,7 +29,7 @@ public void testSubscribeToEvents() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"triggerName\":\"bhkyas\",\"status\":\"Deprovisioning\"}"; + String responseStr = "{\"triggerName\":\"wfytnvcjhjrwnb\",\"status\":\"Unknown\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -46,8 +46,8 @@ public void testSubscribeToEvents() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - TriggerSubscriptionOperationStatus response - = manager.triggers().subscribeToEvents("aulsmqohvcvv", "bxgxgr", "h", com.azure.core.util.Context.NONE); + TriggerSubscriptionOperationStatus response = manager.triggers().subscribeToEvents("nvhlpuobhaomaow", + "mwjsvuziog", "oaimw", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersUnsubscribeFromEventsMockTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersUnsubscribeFromEventsMockTests.java index 9cb90aa0afaf2..5efdd11aba169 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersUnsubscribeFromEventsMockTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TriggersUnsubscribeFromEventsMockTests.java @@ -29,7 +29,7 @@ public void testUnsubscribeFromEvents() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"triggerName\":\"ofdmlrtlhpfucfi\",\"status\":\"Disabled\"}"; + String responseStr = "{\"triggerName\":\"eiy\",\"status\":\"Deprovisioning\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -46,8 +46,8 @@ public void testUnsubscribeFromEvents() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - TriggerSubscriptionOperationStatus response = manager.triggers().unsubscribeFromEvents("utsabuvuuweq", - "eygnetu", "sqvgjvumdznb", com.azure.core.util.Context.NONE); + TriggerSubscriptionOperationStatus response = manager.triggers().unsubscribeFromEvents("wmkgzsqrirlcjmha", + "rkhlayer", "shiuwnefe", com.azure.core.util.Context.NONE); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerDependencyReferenceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerDependencyReferenceTests.java index ca5880fbafee3..366d759b2519e 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerDependencyReferenceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerDependencyReferenceTests.java @@ -14,24 +14,24 @@ public final class TumblingWindowTriggerDependencyReferenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TumblingWindowTriggerDependencyReference model = BinaryData.fromString( - "{\"type\":\"TumblingWindowTriggerDependencyReference\",\"offset\":\"mffaqolnfeywbp\",\"size\":\"q\",\"referenceTrigger\":{\"type\":\"TriggerReference\",\"referenceName\":\"pzwwsfrpbwvfjdg\"}}") + "{\"type\":\"TumblingWindowTriggerDependencyReference\",\"offset\":\"rmmjyvmxtj\",\"size\":\"obwskyjlteiulvrp\",\"referenceTrigger\":{\"type\":\"TriggerReference\",\"referenceName\":\"ivvlmz\"}}") .toObject(TumblingWindowTriggerDependencyReference.class); Assertions.assertEquals(TriggerReferenceType.TRIGGER_REFERENCE, model.referenceTrigger().type()); - Assertions.assertEquals("pzwwsfrpbwvfjdg", model.referenceTrigger().referenceName()); - Assertions.assertEquals("mffaqolnfeywbp", model.offset()); - Assertions.assertEquals("q", model.size()); + Assertions.assertEquals("ivvlmz", model.referenceTrigger().referenceName()); + Assertions.assertEquals("rmmjyvmxtj", model.offset()); + Assertions.assertEquals("obwskyjlteiulvrp", model.size()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { TumblingWindowTriggerDependencyReference model = new TumblingWindowTriggerDependencyReference() - .withReferenceTrigger(new TriggerReference().withType(TriggerReferenceType.TRIGGER_REFERENCE) - .withReferenceName("pzwwsfrpbwvfjdg")) - .withOffset("mffaqolnfeywbp").withSize("q"); + .withReferenceTrigger( + new TriggerReference().withType(TriggerReferenceType.TRIGGER_REFERENCE).withReferenceName("ivvlmz")) + .withOffset("rmmjyvmxtj").withSize("obwskyjlteiulvrp"); model = BinaryData.fromObject(model).toObject(TumblingWindowTriggerDependencyReference.class); Assertions.assertEquals(TriggerReferenceType.TRIGGER_REFERENCE, model.referenceTrigger().type()); - Assertions.assertEquals("pzwwsfrpbwvfjdg", model.referenceTrigger().referenceName()); - Assertions.assertEquals("mffaqolnfeywbp", model.offset()); - Assertions.assertEquals("q", model.size()); + Assertions.assertEquals("ivvlmz", model.referenceTrigger().referenceName()); + Assertions.assertEquals("rmmjyvmxtj", model.offset()); + Assertions.assertEquals("obwskyjlteiulvrp", model.size()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTests.java index 70cf66db29816..a8e98b74d2db5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTests.java @@ -21,43 +21,43 @@ public final class TumblingWindowTriggerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TumblingWindowTrigger model = BinaryData.fromString( - "{\"type\":\"TumblingWindowTrigger\",\"pipeline\":{\"pipelineReference\":{\"referenceName\":\"j\",\"name\":\"sfvyv\"},\"parameters\":{\"jkczkcd\":\"dataclqtdcasjnzeckpg\",\"abzy\":\"datazicltwa\",\"zspw\":\"dataxvif\",\"inlqkzee\":\"datal\"}},\"typeProperties\":{\"frequency\":\"Month\",\"interval\":1329395575,\"startTime\":\"2021-08-16T22:36:54Z\",\"endTime\":\"2021-07-20T02:17:02Z\",\"delay\":\"datamdhmdvk\",\"maxConcurrency\":300833021,\"retryPolicy\":{\"count\":\"datavmc\",\"intervalInSeconds\":874325874},\"dependsOn\":[{\"type\":\"DependencyReference\"}]},\"description\":\"jfmztpwujmu\",\"runtimeState\":\"Disabled\",\"annotations\":[\"datayebwzrfonqj\"],\"\":{\"rf\":\"dataofjfusneyuirrr\",\"cflp\":\"datafamozyvjm\",\"cowmukzcrp\":\"dataqtq\"}}") + "{\"type\":\"TumblingWindowTrigger\",\"pipeline\":{\"pipelineReference\":{\"referenceName\":\"tx\",\"name\":\"uxmegrix\"},\"parameters\":{\"tcoibiodfy\":\"datazjlqrpsqpj\",\"oemcajjazo\":\"dataafenwvvxcah\"}},\"typeProperties\":{\"frequency\":\"Month\",\"interval\":838287368,\"startTime\":\"2021-04-15T18:57:46Z\",\"endTime\":\"2021-07-24T14:06:28Z\",\"delay\":\"datahxydx\",\"maxConcurrency\":2006064884,\"retryPolicy\":{\"count\":\"datailgtbslagtmkii\",\"intervalInSeconds\":1595340480},\"dependsOn\":[{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"}]},\"description\":\"jpvgvbz\",\"runtimeState\":\"Stopped\",\"annotations\":[\"datazgrfaq\",\"datatqmcszdptoyt\",\"datanxgqovfrtm\"],\"\":{\"jiocvjmyi\":\"datazrexmcawpbifzw\"}}") .toObject(TumblingWindowTrigger.class); - Assertions.assertEquals("jfmztpwujmu", model.description()); - Assertions.assertEquals("j", model.pipeline().pipelineReference().referenceName()); - Assertions.assertEquals("sfvyv", model.pipeline().pipelineReference().name()); + Assertions.assertEquals("jpvgvbz", model.description()); + Assertions.assertEquals("tx", model.pipeline().pipelineReference().referenceName()); + Assertions.assertEquals("uxmegrix", model.pipeline().pipelineReference().name()); Assertions.assertEquals(TumblingWindowFrequency.MONTH, model.frequency()); - Assertions.assertEquals(1329395575, model.interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-16T22:36:54Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-20T02:17:02Z"), model.endTime()); - Assertions.assertEquals(300833021, model.maxConcurrency()); - Assertions.assertEquals(874325874, model.retryPolicy().intervalInSeconds()); + Assertions.assertEquals(838287368, model.interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-15T18:57:46Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-24T14:06:28Z"), model.endTime()); + Assertions.assertEquals(2006064884, model.maxConcurrency()); + Assertions.assertEquals(1595340480, model.retryPolicy().intervalInSeconds()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TumblingWindowTrigger model = new TumblingWindowTrigger().withDescription("jfmztpwujmu") - .withAnnotations(Arrays.asList("datayebwzrfonqj")) + TumblingWindowTrigger model = new TumblingWindowTrigger().withDescription("jpvgvbz") + .withAnnotations(Arrays.asList("datazgrfaq", "datatqmcszdptoyt", "datanxgqovfrtm")) .withPipeline(new TriggerPipelineReference() - .withPipelineReference(new PipelineReference().withReferenceName("j").withName("sfvyv")) - .withParameters(mapOf("jkczkcd", "dataclqtdcasjnzeckpg", "abzy", "datazicltwa", "zspw", "dataxvif", - "inlqkzee", "datal"))) - .withFrequency(TumblingWindowFrequency.MONTH).withInterval(1329395575) - .withStartTime(OffsetDateTime.parse("2021-08-16T22:36:54Z")) - .withEndTime(OffsetDateTime.parse("2021-07-20T02:17:02Z")).withDelay("datamdhmdvk") - .withMaxConcurrency(300833021) - .withRetryPolicy(new RetryPolicy().withCount("datavmc").withIntervalInSeconds(874325874)) - .withDependsOn(Arrays.asList(new DependencyReference())); + .withPipelineReference(new PipelineReference().withReferenceName("tx").withName("uxmegrix")) + .withParameters(mapOf("tcoibiodfy", "datazjlqrpsqpj", "oemcajjazo", "dataafenwvvxcah"))) + .withFrequency(TumblingWindowFrequency.MONTH).withInterval(838287368) + .withStartTime(OffsetDateTime.parse("2021-04-15T18:57:46Z")) + .withEndTime(OffsetDateTime.parse("2021-07-24T14:06:28Z")).withDelay("datahxydx") + .withMaxConcurrency(2006064884) + .withRetryPolicy(new RetryPolicy().withCount("datailgtbslagtmkii").withIntervalInSeconds(1595340480)) + .withDependsOn(Arrays.asList(new DependencyReference(), new DependencyReference(), + new DependencyReference(), new DependencyReference())); model = BinaryData.fromObject(model).toObject(TumblingWindowTrigger.class); - Assertions.assertEquals("jfmztpwujmu", model.description()); - Assertions.assertEquals("j", model.pipeline().pipelineReference().referenceName()); - Assertions.assertEquals("sfvyv", model.pipeline().pipelineReference().name()); + Assertions.assertEquals("jpvgvbz", model.description()); + Assertions.assertEquals("tx", model.pipeline().pipelineReference().referenceName()); + Assertions.assertEquals("uxmegrix", model.pipeline().pipelineReference().name()); Assertions.assertEquals(TumblingWindowFrequency.MONTH, model.frequency()); - Assertions.assertEquals(1329395575, model.interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-16T22:36:54Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-20T02:17:02Z"), model.endTime()); - Assertions.assertEquals(300833021, model.maxConcurrency()); - Assertions.assertEquals(874325874, model.retryPolicy().intervalInSeconds()); + Assertions.assertEquals(838287368, model.interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-15T18:57:46Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-24T14:06:28Z"), model.endTime()); + Assertions.assertEquals(2006064884, model.maxConcurrency()); + Assertions.assertEquals(1595340480, model.retryPolicy().intervalInSeconds()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTypePropertiesTests.java index 0a6a168ad8c7d..258770ae4a6a0 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TumblingWindowTriggerTypePropertiesTests.java @@ -17,31 +17,32 @@ public final class TumblingWindowTriggerTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TumblingWindowTriggerTypeProperties model = BinaryData.fromString( - "{\"frequency\":\"Minute\",\"interval\":1685716291,\"startTime\":\"2021-03-02T03:51:14Z\",\"endTime\":\"2021-10-15T07:32:04Z\",\"delay\":\"datanlaklszbeutqfx\",\"maxConcurrency\":1561097036,\"retryPolicy\":{\"count\":\"datai\",\"intervalInSeconds\":1537689259},\"dependsOn\":[{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"}]}") + "{\"frequency\":\"Hour\",\"interval\":1467815834,\"startTime\":\"2021-01-26T15:42:35Z\",\"endTime\":\"2021-06-18T21:24:42Z\",\"delay\":\"datahxfjlecbbabih\",\"maxConcurrency\":945617288,\"retryPolicy\":{\"count\":\"datarxzatlzwrpj\",\"intervalInSeconds\":526431444},\"dependsOn\":[{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"},{\"type\":\"DependencyReference\"}]}") .toObject(TumblingWindowTriggerTypeProperties.class); - Assertions.assertEquals(TumblingWindowFrequency.MINUTE, model.frequency()); - Assertions.assertEquals(1685716291, model.interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-03-02T03:51:14Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-15T07:32:04Z"), model.endTime()); - Assertions.assertEquals(1561097036, model.maxConcurrency()); - Assertions.assertEquals(1537689259, model.retryPolicy().intervalInSeconds()); + Assertions.assertEquals(TumblingWindowFrequency.HOUR, model.frequency()); + Assertions.assertEquals(1467815834, model.interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-26T15:42:35Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-18T21:24:42Z"), model.endTime()); + Assertions.assertEquals(945617288, model.maxConcurrency()); + Assertions.assertEquals(526431444, model.retryPolicy().intervalInSeconds()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { TumblingWindowTriggerTypeProperties model - = new TumblingWindowTriggerTypeProperties().withFrequency(TumblingWindowFrequency.MINUTE) - .withInterval(1685716291).withStartTime(OffsetDateTime.parse("2021-03-02T03:51:14Z")) - .withEndTime(OffsetDateTime.parse("2021-10-15T07:32:04Z")).withDelay("datanlaklszbeutqfx") - .withMaxConcurrency(1561097036) - .withRetryPolicy(new RetryPolicy().withCount("datai").withIntervalInSeconds(1537689259)).withDependsOn( + = new TumblingWindowTriggerTypeProperties().withFrequency(TumblingWindowFrequency.HOUR) + .withInterval(1467815834).withStartTime(OffsetDateTime.parse("2021-01-26T15:42:35Z")) + .withEndTime(OffsetDateTime.parse("2021-06-18T21:24:42Z")).withDelay("datahxfjlecbbabih") + .withMaxConcurrency(945617288) + .withRetryPolicy(new RetryPolicy().withCount("datarxzatlzwrpj").withIntervalInSeconds(526431444)) + .withDependsOn( Arrays.asList(new DependencyReference(), new DependencyReference(), new DependencyReference())); model = BinaryData.fromObject(model).toObject(TumblingWindowTriggerTypeProperties.class); - Assertions.assertEquals(TumblingWindowFrequency.MINUTE, model.frequency()); - Assertions.assertEquals(1685716291, model.interval()); - Assertions.assertEquals(OffsetDateTime.parse("2021-03-02T03:51:14Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-15T07:32:04Z"), model.endTime()); - Assertions.assertEquals(1561097036, model.maxConcurrency()); - Assertions.assertEquals(1537689259, model.retryPolicy().intervalInSeconds()); + Assertions.assertEquals(TumblingWindowFrequency.HOUR, model.frequency()); + Assertions.assertEquals(1467815834, model.interval()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-26T15:42:35Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-18T21:24:42Z"), model.endTime()); + Assertions.assertEquals(945617288, model.maxConcurrency()); + Assertions.assertEquals(526431444, model.retryPolicy().intervalInSeconds()); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TypeConversionSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TypeConversionSettingsTests.java index 3736db1c8ae75..b274876ed5598 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TypeConversionSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/TypeConversionSettingsTests.java @@ -11,15 +11,16 @@ public final class TypeConversionSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { TypeConversionSettings model = BinaryData.fromString( - "{\"allowDataTruncation\":\"dataiblybx\",\"treatBooleanAsNumber\":\"datazknkffzdy\",\"dateTimeFormat\":\"datani\",\"dateTimeOffsetFormat\":\"datat\",\"timeSpanFormat\":\"datadedlmfw\",\"culture\":\"datafg\"}") + "{\"allowDataTruncation\":\"datan\",\"treatBooleanAsNumber\":\"datavxdafilaizcdugn\",\"dateTimeFormat\":\"datamljgayka\",\"dateTimeOffsetFormat\":\"datankxoqecjznuqg\",\"timeSpanFormat\":\"datavmuewshhq\",\"culture\":\"datavchliezfbtczzjf\"}") .toObject(TypeConversionSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TypeConversionSettings model = new TypeConversionSettings().withAllowDataTruncation("dataiblybx") - .withTreatBooleanAsNumber("datazknkffzdy").withDateTimeFormat("datani").withDateTimeOffsetFormat("datat") - .withTimeSpanFormat("datadedlmfw").withCulture("datafg"); + TypeConversionSettings model = new TypeConversionSettings().withAllowDataTruncation("datan") + .withTreatBooleanAsNumber("datavxdafilaizcdugn").withDateTimeFormat("datamljgayka") + .withDateTimeOffsetFormat("datankxoqecjznuqg").withTimeSpanFormat("datavmuewshhq") + .withCulture("datavchliezfbtczzjf"); model = BinaryData.fromObject(model).toObject(TypeConversionSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTests.java index 5a6d97f6651e7..e87eaa9590ea7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTests.java @@ -22,131 +22,119 @@ public final class UntilActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { UntilActivity model = BinaryData.fromString( - "{\"type\":\"Until\",\"typeProperties\":{\"expression\":{\"value\":\"jusqhrvadffdr\"},\"timeout\":\"datakhtsyc\",\"activities\":[{\"type\":\"Activity\",\"name\":\"krvnksiemb\",\"description\":\"tzmldw\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"zhacorqbmkfoala\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\"],\"\":{\"saskgiyrilbi\":\"datawdohvinvzdnu\",\"ccvtb\":\"dataacxldhoqcdpwx\",\"mfzdlhp\":\"dataznpxaxcshtlqhi\"}},{\"activity\":\"obhnuziaz\",\"dependencyConditions\":[\"Completed\",\"Completed\",\"Skipped\"],\"\":{\"hwdirt\":\"dataevwidnjpfku\",\"aqya\":\"datay\"}},{\"activity\":\"dykxgcfhv\",\"dependencyConditions\":[\"Succeeded\",\"Skipped\"],\"\":{\"ycraryxrtt\":\"datazysuoqf\",\"stlg\":\"dataujhjbfoemmj\",\"jm\":\"datavvpxhdefydit\"}}],\"userProperties\":[{\"name\":\"jyqhcowouoih\",\"value\":\"datatnyvigjbxhjpsgpr\"},{\"name\":\"mpzb\",\"value\":\"dataiakyflryhvph\"},{\"name\":\"dciyidzbpfwlxxwp\",\"value\":\"dataz\"}],\"\":{\"gcazyni\":\"datatmlhrzi\",\"hmxmjm\":\"datakkengowcut\",\"kqdqn\":\"dataouichoiimennxvqj\"}},{\"type\":\"Activity\",\"name\":\"efszuu\",\"description\":\"zesfggheqllr\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"krvmvdqhag\",\"dependencyConditions\":[\"Skipped\",\"Failed\"],\"\":{\"upkmvxe\":\"dataqtlsipedg\",\"hmcmfvyh\":\"databngwidgxypdovl\",\"o\":\"dataivybl\",\"tvoprgcsjycorxib\":\"datak\"}},{\"activity\":\"sfxkudic\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Failed\"],\"\":{\"qassclgol\":\"dataue\"}},{\"activity\":\"pwegz\",\"dependencyConditions\":[\"Completed\",\"Skipped\"],\"\":{\"zskvp\":\"datanespkxnhfd\",\"hlbxrqbi\":\"dataqxnd\",\"zkehfkpoczxm\":\"datajhaafvxxi\"}}],\"userProperties\":[{\"name\":\"rwihbyu\",\"value\":\"datamuinhqpqfow\"},{\"name\":\"dihuxz\",\"value\":\"datagoto\"},{\"name\":\"n\",\"value\":\"dataduirjqxknaeuhxnp\"},{\"name\":\"d\",\"value\":\"datajaeqaolfyqjgob\"}],\"\":{\"rwvlzbdfmhzgti\":\"datanaxtbnjmjw\",\"et\":\"dataybimitgx\",\"egkwabzr\":\"datawloo\"}},{\"type\":\"Activity\",\"name\":\"reftwhiivxytvje\",\"description\":\"kuzlfnbz\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"kwrv\",\"dependencyConditions\":[\"Failed\",\"Failed\"],\"\":{\"g\":\"dataqy\",\"wxmqyhtlnnpftay\":\"datavpxsdtnxg\",\"gxamhmqexyoy\":\"datao\",\"pvvelcrwhrpxs\":\"datacwzkcreuf\"}},{\"activity\":\"ybalsmiar\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Succeeded\",\"Failed\"],\"\":{\"obyyv\":\"datapv\"}},{\"activity\":\"jelsjh\",\"dependencyConditions\":[\"Failed\",\"Skipped\",\"Succeeded\"],\"\":{\"ujjdoelawdbkez\":\"datahkhiycddonqi\"}},{\"activity\":\"kotvoszgcy\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Succeeded\"],\"\":{\"j\":\"dataqwvvferlqhfzzqqs\",\"skjqejkm\":\"datashwxy\",\"utcyjjbdgfrl\":\"datatwftlhsmtkxzio\",\"egqvusffzvpwzvh\":\"datah\"}}],\"userProperties\":[{\"name\":\"rvmpiw\",\"value\":\"dataoorrnssthninza\"},{\"name\":\"dmnc\",\"value\":\"dataltrxwab\"},{\"name\":\"d\",\"value\":\"dataclqgteoepdpx\"}],\"\":{\"qikeamymalvoy\":\"dataqwfpqixomonq\"}}]},\"name\":\"qgelcccccc\",\"description\":\"nljzqv\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"lhf\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Completed\"],\"\":{\"pzq\":\"datahvxtxuihydwkdvy\",\"yaudellb\":\"datapyquyqydtllpwz\",\"hrene\":\"datafp\"}},{\"activity\":\"snubirus\",\"dependencyConditions\":[\"Completed\",\"Completed\"],\"\":{\"wnqfdgcrf\":\"dataptoktr\",\"ebwqz\":\"datagktmzooszvungkkf\",\"scjhay\":\"dataiqtyeqeasia\",\"cowbxpvm\":\"dataidbbqvip\"}},{\"activity\":\"dqmzcgqe\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Completed\",\"Succeeded\"],\"\":{\"ahn\":\"datahvqkeuiyme\",\"vus\":\"datantldddkn\",\"ozksood\":\"datagwsnqnxrrjihgig\"}}],\"userProperties\":[{\"name\":\"uwxkeqlbm\",\"value\":\"dataoyapxnq\"}],\"\":{\"qhtgtadtootkgxx\":\"datafixhxlcv\",\"qwxskl\":\"datamen\"}}") + "{\"type\":\"Until\",\"typeProperties\":{\"expression\":{\"value\":\"oggzppufu\"},\"timeout\":\"dataaiecexy\",\"activities\":[{\"type\":\"Activity\",\"name\":\"hjqdwlxa\",\"description\":\"l\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"aipfqnxjko\",\"dependencyConditions\":[\"Completed\",\"Succeeded\"],\"\":{\"mhabzjemqvlouuc\":\"datajbpt\"}}],\"userProperties\":[{\"name\":\"bqsj\",\"value\":\"datancgqhpqgivyx\"},{\"name\":\"jimussvur\",\"value\":\"datalwdxnx\"},{\"name\":\"valvkdaql\",\"value\":\"datasoqrhwla\"}],\"\":{\"zis\":\"datatwsxliwpzuce\",\"ubh\":\"datarvtrwswbm\",\"ivusehyvqxjbqfcl\":\"databtthzfgpzy\"}},{\"type\":\"Activity\",\"name\":\"jecajtuo\",\"description\":\"dlzxuakbavpk\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"kfzbqvsgxe\",\"dependencyConditions\":[\"Completed\",\"Succeeded\"],\"\":{\"dhhqsfht\":\"datagn\"}}],\"userProperties\":[{\"name\":\"axdyxjicikzmvdd\",\"value\":\"datafjmi\"}],\"\":{\"svpfspfdfrymrfpq\":\"datafcqls\",\"w\":\"dataxln\",\"qhzotkowi\":\"datagi\"}},{\"type\":\"Activity\",\"name\":\"uerhzyl\",\"description\":\"ym\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"ieemslhztvry\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Succeeded\"],\"\":{\"gryolbqcftrywdg\":\"datawnewmpwj\"}},{\"activity\":\"skdl\",\"dependencyConditions\":[\"Failed\",\"Succeeded\",\"Succeeded\"],\"\":{\"djgcuew\":\"datanxvmcxljlpyhdx\",\"qgfqivmsxwev\":\"datanq\",\"oubjnmoid\":\"datajmxvvtuk\",\"cgmfklqswwdbs\":\"datanbfbkwyvw\"}}],\"userProperties\":[{\"name\":\"ysedqrbevobqrwng\",\"value\":\"datayjfquzxmtmsyiby\"}],\"\":{\"jkrosqxvffrnc\":\"datapksas\",\"lzoi\":\"datawvjgyjoklngjs\",\"gakkszz\":\"datawsqdnasjup\"}},{\"type\":\"Activity\",\"name\":\"dtvrgyebvq\",\"description\":\"ikeuqvqiotvfcb\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"fatyqa\",\"dependencyConditions\":[\"Succeeded\"],\"\":{\"teap\":\"dataqopglixhapvwacwr\"}},{\"activity\":\"cnknzncoxeop\",\"dependencyConditions\":[\"Completed\"],\"\":{\"vr\":\"dataaxrsyxeqwgaeice\",\"ztssqbclaeciwz\":\"datadcidcxkywy\"}},{\"activity\":\"vttkha\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Failed\"],\"\":{\"jkhqaxpicza\":\"datamqj\",\"dv\":\"datagevsnnqvkufezwgw\",\"dkjusqhr\":\"dataskffqqaobbq\"}},{\"activity\":\"adffdr\",\"dependencyConditions\":[\"Failed\",\"Completed\",\"Completed\"],\"\":{\"iembc\":\"datacctkrvnk\",\"ixjkxvz\":\"datatzmldw\",\"orqbmkfo\":\"dataa\"}}],\"userProperties\":[{\"name\":\"aqfqgmwdo\",\"value\":\"datavinvzdnubs\"},{\"name\":\"skgiy\",\"value\":\"datailbiwacxldho\"},{\"name\":\"cdpwxh\",\"value\":\"datacvtbgznpx\"},{\"name\":\"xcshtlqhikmfzdlh\",\"value\":\"datazo\"}],\"\":{\"dn\":\"datauziaztmxwmjaevw\",\"hwdirt\":\"datapfku\",\"aqya\":\"datay\"}}]},\"name\":\"dykxgcfhv\",\"description\":\"ns\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"qfbycra\",\"dependencyConditions\":[\"Failed\"],\"\":{\"f\":\"datatnujhj\",\"gdv\":\"dataemmjtst\",\"mvavjyqhcowou\":\"datapxhdefyditb\",\"nyv\":\"dataihl\"}},{\"activity\":\"gjb\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Failed\"],\"\":{\"mpzb\":\"datar\",\"dciyidzbpfwlxxwp\":\"dataiakyflryhvph\",\"gstmlhrziggc\":\"dataz\",\"n\":\"datazynimkk\"}},{\"activity\":\"owcutohmxmj\",\"dependencyConditions\":[\"Completed\",\"Failed\"],\"\":{\"qnnef\":\"datahoiimennxvqjakq\"}}],\"userProperties\":[{\"name\":\"uguzesfgg\",\"value\":\"dataeqllrpcqyxqf\"},{\"name\":\"rvmvdqhageho\",\"value\":\"dataqeqtlsipedgtup\"},{\"name\":\"mvxeubngwidgxy\",\"value\":\"datadovlp\"}],\"\":{\"lt\":\"datamfvyhmivy\",\"s\":\"dataakmtvoprg\",\"orxibw\":\"datay\",\"cwfo\":\"datafxkud\"}}") .toObject(UntilActivity.class); - Assertions.assertEquals("qgelcccccc", model.name()); - Assertions.assertEquals("nljzqv", model.description()); + Assertions.assertEquals("dykxgcfhv", model.name()); + Assertions.assertEquals("ns", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("lhf", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("uwxkeqlbm", model.userProperties().get(0).name()); - Assertions.assertEquals("jusqhrvadffdr", model.expression().value()); - Assertions.assertEquals("krvnksiemb", model.activities().get(0).name()); - Assertions.assertEquals("tzmldw", model.activities().get(0).description()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("qfbycra", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("uguzesfgg", model.userProperties().get(0).name()); + Assertions.assertEquals("oggzppufu", model.expression().value()); + Assertions.assertEquals("hjqdwlxa", model.activities().get(0).name()); + Assertions.assertEquals("l", model.activities().get(0).description()); Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("zhacorqbmkfoala", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, + Assertions.assertEquals("aipfqnxjko", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("jyqhcowouoih", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("bqsj", model.activities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - UntilActivity model = new UntilActivity().withName("qgelcccccc").withDescription("nljzqv") - .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("lhf") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) + UntilActivity model = new UntilActivity().withName("dykxgcfhv").withDescription("ns") + .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("qfbycra") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)).withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("gjb") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.FAILED, + DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("snubirus") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dqmzcgqe") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("owcutohmxmj") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("uwxkeqlbm").withValue("dataoyapxnq"))) - .withExpression(new Expression().withValue("jusqhrvadffdr")).withTimeout("datakhtsyc") + .withUserProperties(Arrays.asList(new UserProperty().withName("uguzesfgg").withValue("dataeqllrpcqyxqf"), + new UserProperty().withName("rvmvdqhageho").withValue("dataqeqtlsipedgtup"), + new UserProperty().withName("mvxeubngwidgxy").withValue("datadovlp"))) + .withExpression( + new Expression().withValue("oggzppufu")) + .withTimeout("dataaiecexy") .withActivities(Arrays.asList( - new Activity() - .withName("krvnksiemb").withDescription("tzmldw").withState( - ActivityState.INACTIVE) + new Activity().withName("hjqdwlxa").withDescription("l").withState(ActivityState.INACTIVE) .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("zhacorqbmkfoala") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("obhnuziaz") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("dykxgcfhv") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()))) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("aipfqnxjko") + .withDependencyConditions( + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("bqsj").withValue("datancgqhpqgivyx"), + new UserProperty().withName("jimussvur").withValue("datalwdxnx"), + new UserProperty().withName("valvkdaql").withValue("datasoqrhwla"))) + .withAdditionalProperties(mapOf("type", "Activity")), + new Activity().withName("jecajtuo").withDescription("dlzxuakbavpk").withState(ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("kfzbqvsgxe") + .withDependencyConditions( + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("jyqhcowouoih").withValue("datatnyvigjbxhjpsgpr"), - new UserProperty().withName("mpzb").withValue("dataiakyflryhvph"), new UserProperty() - .withName("dciyidzbpfwlxxwp").withValue("dataz"))) + Arrays.asList(new UserProperty().withName("axdyxjicikzmvdd").withValue("datafjmi"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("efszuu").withDescription("zesfggheqllr").withState(ActivityState.ACTIVE) + new Activity().withName("uerhzyl").withDescription("ym").withState(ActivityState.ACTIVE) .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("krvmvdqhag") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("sfxkudic") + new ActivityDependency().withActivity("ieemslhztvry") .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.FAILED, DependencyCondition.FAILED)) + DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("pwegz") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("skdl") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("rwihbyu").withValue("datamuinhqpqfow"), - new UserProperty().withName("dihuxz").withValue("datagoto"), - new UserProperty().withName("n").withValue("dataduirjqxknaeuhxnp"), - new UserProperty().withName("d").withValue("datajaeqaolfyqjgob"))) + Arrays.asList(new UserProperty().withName("ysedqrbevobqrwng").withValue("datayjfquzxmtmsyiby"))) .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("reftwhiivxytvje").withDescription("kuzlfnbz").withState(ActivityState.ACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + new Activity().withName("dtvrgyebvq").withDescription("ikeuqvqiotvfcb") + .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency() - .withActivity("kwrv") - .withDependencyConditions( - Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED)) + new ActivityDependency().withActivity("fatyqa") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ybalsmiar") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.FAILED, DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) + new ActivityDependency().withActivity("cnknzncoxeop") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("jelsjh") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, - DependencyCondition.SKIPPED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("vttkha") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kotvoszgcy") - .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, - DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) + new ActivityDependency().withActivity("adffdr") + .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, + DependencyCondition.COMPLETED, DependencyCondition.COMPLETED)) .withAdditionalProperties(mapOf()))) .withUserProperties( - Arrays.asList(new UserProperty().withName("rvmpiw").withValue("dataoorrnssthninza"), - new UserProperty().withName("dmnc").withValue("dataltrxwab"), - new UserProperty().withName("d").withValue("dataclqgteoepdpx"))) + Arrays.asList(new UserProperty().withName("aqfqgmwdo").withValue("datavinvzdnubs"), + new UserProperty().withName("skgiy").withValue("datailbiwacxldho"), + new UserProperty().withName("cdpwxh").withValue("datacvtbgznpx"), + new UserProperty().withName("xcshtlqhikmfzdlh").withValue("datazo"))) .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(UntilActivity.class); - Assertions.assertEquals("qgelcccccc", model.name()); - Assertions.assertEquals("nljzqv", model.description()); + Assertions.assertEquals("dykxgcfhv", model.name()); + Assertions.assertEquals("ns", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("lhf", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("uwxkeqlbm", model.userProperties().get(0).name()); - Assertions.assertEquals("jusqhrvadffdr", model.expression().value()); - Assertions.assertEquals("krvnksiemb", model.activities().get(0).name()); - Assertions.assertEquals("tzmldw", model.activities().get(0).description()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.onInactiveMarkAs()); + Assertions.assertEquals("qfbycra", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("uguzesfgg", model.userProperties().get(0).name()); + Assertions.assertEquals("oggzppufu", model.expression().value()); + Assertions.assertEquals("hjqdwlxa", model.activities().get(0).name()); + Assertions.assertEquals("l", model.activities().get(0).description()); Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("zhacorqbmkfoala", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, + Assertions.assertEquals("aipfqnxjko", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("jyqhcowouoih", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("bqsj", model.activities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTypePropertiesTests.java index d86155d9b9240..5e3fb692d90c6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/UntilActivityTypePropertiesTests.java @@ -22,92 +22,57 @@ public final class UntilActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { UntilActivityTypeProperties model = BinaryData.fromString( - "{\"expression\":{\"value\":\"zzp\"},\"timeout\":\"datawgtmpytomftubh\",\"activities\":[{\"type\":\"Activity\",\"name\":\"wgbvpyjpaihh\",\"description\":\"azxfz\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"srdaoixgq\",\"dependencyConditions\":[\"Skipped\",\"Skipped\"],\"\":{\"yoimtfkj\":\"dataekotqhdgixknc\"}},{\"activity\":\"djs\",\"dependencyConditions\":[\"Skipped\"],\"\":{\"clg\":\"datahvccxuntghwcb\"}},{\"activity\":\"yfcbcakcq\",\"dependencyConditions\":[\"Skipped\",\"Completed\",\"Failed\"],\"\":{\"gvcm\":\"dataumlnfxboqvgwaiyw\"}}],\"userProperties\":[{\"name\":\"irjsznxzez\",\"value\":\"datanqmxirsp\"},{\"name\":\"hakrbewvzis\",\"value\":\"datanbourwv\"},{\"name\":\"nsdluq\",\"value\":\"datayxgmzyqftla\"}],\"\":{\"z\":\"dataoufnxtynusqzai\"}},{\"type\":\"Activity\",\"name\":\"w\",\"description\":\"lkwbwxcjfjuzwiw\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Succeeded\",\"dependsOn\":[{\"activity\":\"hpuqlsdwt\",\"dependencyConditions\":[\"Completed\",\"Succeeded\",\"Failed\",\"Succeeded\"],\"\":{\"obs\":\"datalyxujqpccpvyc\"}},{\"activity\":\"ogjaubplfzjgl\",\"dependencyConditions\":[\"Skipped\",\"Failed\",\"Succeeded\",\"Failed\"],\"\":{\"tyqbtijyb\":\"datatnndiswycwplj\"}},{\"activity\":\"fwgclppwdfxhzvr\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Completed\",\"Succeeded\"],\"\":{\"rrevuzbxuubwjo\":\"datazakisipjgvmrb\",\"oniydgnxsgy\":\"datakldubqfbwffg\"}}],\"userProperties\":[{\"name\":\"mhlankosdfvmfy\",\"value\":\"datalefkh\"},{\"name\":\"iayb\",\"value\":\"dataivxv\"},{\"name\":\"wdfmqzndlgqtuq\",\"value\":\"dataczco\"},{\"name\":\"ctcwtxa\",\"value\":\"datafc\"}],\"\":{\"hbmwlfo\":\"datamsdodmrzsnin\",\"jphzxmcpsepkrdge\":\"datayt\"}},{\"type\":\"Activity\",\"name\":\"qzxkpxrful\",\"description\":\"hmnd\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"jmel\",\"dependencyConditions\":[\"Skipped\",\"Skipped\",\"Skipped\",\"Failed\"],\"\":{\"gfaiyvmpfebsummy\":\"dataoyliopbo\",\"ckuhgbrvh\":\"datarxnneqxsdupmr\",\"zvclzutvqkoi\":\"dataskos\",\"fskqwjlohkaffyny\":\"datamv\"}}],\"userProperties\":[{\"name\":\"qbyty\",\"value\":\"dataj\"}],\"\":{\"j\":\"datacxpqpaxkayvxegiu\",\"wbdmunuv\":\"datajgupjjppbalcft\"}}]}") + "{\"expression\":{\"value\":\"enmuevq\"},\"timeout\":\"datas\",\"activities\":[{\"type\":\"Activity\",\"name\":\"golbpwegzdion\",\"description\":\"nespkxnhfd\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"qxnd\",\"dependencyConditions\":[\"Completed\",\"Succeeded\"],\"\":{\"afvxxiizkehfk\":\"dataqbicjh\",\"krwihbyufmuinhq\":\"dataoczxmw\",\"ih\":\"dataqfowx\",\"qxk\":\"dataxzdgotoonsduir\"}},{\"activity\":\"aeuhxnpa\",\"dependencyConditions\":[\"Completed\",\"Failed\",\"Failed\"],\"\":{\"gobu\":\"dataolfyq\",\"rwvlzbdfmhzgti\":\"datavnaxtbnjmjw\",\"et\":\"dataybimitgx\"}},{\"activity\":\"wloo\",\"dependencyConditions\":[\"Failed\",\"Succeeded\"],\"\":{\"xytvjezi\":\"databzrrreftwhii\"}}],\"userProperties\":[{\"name\":\"lfnbzekwoajb\",\"value\":\"datawrvckyhncqy\"},{\"name\":\"g\",\"value\":\"datavpxsdtnxg\"},{\"name\":\"wxmqyhtlnnpftay\",\"value\":\"datao\"},{\"name\":\"gxamhmqexyoy\",\"value\":\"datacwzkcreuf\"}],\"\":{\"sxybalsmia\":\"datavelcrwhrp\",\"qzypvc\":\"datauvbo\",\"luhkhiycddon\":\"databyyvxjelsjhgrvy\",\"bkez\":\"dataikujjdoelaw\"}}]}") .toObject(UntilActivityTypeProperties.class); - Assertions.assertEquals("zzp", model.expression().value()); - Assertions.assertEquals("wgbvpyjpaihh", model.activities().get(0).name()); - Assertions.assertEquals("azxfz", model.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("srdaoixgq", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, + Assertions.assertEquals("enmuevq", model.expression().value()); + Assertions.assertEquals("golbpwegzdion", model.activities().get(0).name()); + Assertions.assertEquals("nespkxnhfd", model.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.activities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("qxnd", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("irjsznxzez", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("lfnbzekwoajb", model.activities().get(0).userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { UntilActivityTypeProperties model - = new UntilActivityTypeProperties().withExpression(new Expression().withValue("zzp")) - .withTimeout("datawgtmpytomftubh") - .withActivities(Arrays.asList( - new Activity() - .withName("wgbvpyjpaihh").withDescription("azxfz").withState( - ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("srdaoixgq") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("djs") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("yfcbcakcq") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.COMPLETED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("irjsznxzez").withValue("datanqmxirsp"), - new UserProperty().withName("hakrbewvzis") - .withValue("datanbourwv"), - new UserProperty().withName("nsdluq").withValue("datayxgmzyqftla"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("w").withDescription("lkwbwxcjfjuzwiw").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.SUCCEEDED) - .withDependsOn(Arrays - .asList( - new ActivityDependency() - .withActivity("hpuqlsdwt") - .withDependencyConditions( - Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED, - DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("ogjaubplfzjgl") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED, - DependencyCondition.SUCCEEDED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("fwgclppwdfxhzvr") - .withDependencyConditions( - Arrays.asList(DependencyCondition.SUCCEEDED, DependencyCondition.SUCCEEDED, - DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("mhlankosdfvmfy").withValue("datalefkh"), - new UserProperty().withName("iayb").withValue("dataivxv"), - new UserProperty() - .withName("wdfmqzndlgqtuq").withValue("dataczco"), - new UserProperty().withName("ctcwtxa").withValue("datafc"))) - .withAdditionalProperties(mapOf("type", "Activity")), - new Activity().withName("qzxkpxrful").withDescription("hmnd").withState(ActivityState.INACTIVE) - .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList(new ActivityDependency().withActivity("jmel") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, - DependencyCondition.SKIPPED, DependencyCondition.SKIPPED, DependencyCondition.FAILED)) + = new UntilActivityTypeProperties().withExpression(new Expression().withValue("enmuevq")) + .withTimeout("datas") + .withActivities(Arrays.asList(new Activity() + .withName("golbpwegzdion").withDescription("nespkxnhfd").withState( + ActivityState.ACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn(Arrays.asList( + new ActivityDependency().withActivity("qxnd") + .withDependencyConditions( + Arrays.asList(DependencyCondition.COMPLETED, DependencyCondition.SUCCEEDED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("aeuhxnpa") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.FAILED, DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("wloo") + .withDependencyConditions( + Arrays.asList(DependencyCondition.FAILED, DependencyCondition.SUCCEEDED)) .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("qbyty").withValue("dataj"))) - .withAdditionalProperties(mapOf("type", "Activity")))); + .withUserProperties( + Arrays.asList(new UserProperty().withName("lfnbzekwoajb").withValue("datawrvckyhncqy"), + new UserProperty().withName("g").withValue("datavpxsdtnxg"), + new UserProperty().withName("wxmqyhtlnnpftay").withValue("datao"), + new UserProperty().withName("gxamhmqexyoy").withValue("datacwzkcreuf"))) + .withAdditionalProperties(mapOf("type", "Activity")))); model = BinaryData.fromObject(model).toObject(UntilActivityTypeProperties.class); - Assertions.assertEquals("zzp", model.expression().value()); - Assertions.assertEquals("wgbvpyjpaihh", model.activities().get(0).name()); - Assertions.assertEquals("azxfz", model.activities().get(0).description()); - Assertions.assertEquals(ActivityState.INACTIVE, model.activities().get(0).state()); - Assertions.assertEquals(ActivityOnInactiveMarkAs.SUCCEEDED, model.activities().get(0).onInactiveMarkAs()); - Assertions.assertEquals("srdaoixgq", model.activities().get(0).dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SKIPPED, + Assertions.assertEquals("enmuevq", model.expression().value()); + Assertions.assertEquals("golbpwegzdion", model.activities().get(0).name()); + Assertions.assertEquals("nespkxnhfd", model.activities().get(0).description()); + Assertions.assertEquals(ActivityState.ACTIVE, model.activities().get(0).state()); + Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.activities().get(0).onInactiveMarkAs()); + Assertions.assertEquals("qxnd", model.activities().get(0).dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.COMPLETED, model.activities().get(0).dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("irjsznxzez", model.activities().get(0).userProperties().get(0).name()); + Assertions.assertEquals("lfnbzekwoajb", model.activities().get(0).userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTests.java index ac9a52c645a0d..cf243deabef7c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTests.java @@ -21,44 +21,53 @@ public final class ValidationActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ValidationActivity model = BinaryData.fromString( - "{\"type\":\"Validation\",\"typeProperties\":{\"timeout\":\"datamxxgocpzqrbtyz\",\"sleep\":\"datawrufiouafxp\",\"minimumSize\":\"datazmwx\",\"childItems\":\"datagi\",\"dataset\":{\"referenceName\":\"gnplzbtvpuigtnjy\",\"parameters\":{\"extlyyvebpykzhr\":\"datavvitxoitnqmiwlri\"}}},\"name\":\"usbtwpvmzgnxepa\",\"description\":\"v\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"lu\",\"dependencyConditions\":[\"Succeeded\",\"Succeeded\",\"Completed\"],\"\":{\"kl\":\"datasxsnbttefbb\",\"vdohocsgktfzs\":\"datafkvshozjkwj\",\"ekcxe\":\"datayac\",\"vbzyhexlh\":\"datai\"}},{\"activity\":\"kpiedcrtvdcbzpyn\",\"dependencyConditions\":[\"Skipped\",\"Failed\"],\"\":{\"amzfxocuvjbp\":\"datatdmgwxowaawehxs\",\"swgkbzrmeftg\":\"datav\",\"agsyvzghnqed\":\"datafuuu\",\"ihotzygqdc\":\"dataykvgfh\"}}],\"userProperties\":[{\"name\":\"srrytkmf\",\"value\":\"databpcr\"},{\"name\":\"ynunrajtbumaid\",\"value\":\"datasn\"}],\"\":{\"vcgtlttnjpgxux\":\"datautcvumvgtt\",\"aq\":\"datacenxkdqqombia\",\"htqbh\":\"datawoixnuffraursqt\",\"kreanakkg\":\"datadpnzqqti\"}}") + "{\"type\":\"Validation\",\"typeProperties\":{\"timeout\":\"dataotvoszgcyhwj\",\"sleep\":\"dataahqwvvferl\",\"minimumSize\":\"datafz\",\"childItems\":\"dataqsbjbshwxyfsk\",\"dataset\":{\"referenceName\":\"qejkmltwft\",\"parameters\":{\"jjbdg\":\"datamtkxziowutc\",\"vpwzv\":\"datarlohregqvusff\",\"orrnssthn\":\"datammirvmpiwo\",\"rxwabwdbclq\":\"datanzatdmncyl\"}}},\"name\":\"teoepdpxuzpqwfp\",\"description\":\"xomonqqqike\",\"state\":\"Inactive\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"oydqgelccccccojn\",\"dependencyConditions\":[\"Skipped\",\"Skipped\"],\"\":{\"h\":\"datavmzpoi\",\"hvxtxuihydwkdvy\":\"datavipnbd\",\"pyquyqydtllpwz\":\"datapzq\"}},{\"activity\":\"yaudellb\",\"dependencyConditions\":[\"Failed\"],\"\":{\"b\":\"datarenedsn\",\"ptoktr\":\"datarusknpyf\"}},{\"activity\":\"wnqfdgcrf\",\"dependencyConditions\":[\"Completed\",\"Skipped\",\"Completed\",\"Skipped\"],\"\":{\"pebwqzvi\":\"dataszvungkk\",\"pidb\":\"datatyeqeasiadscjha\",\"lcowb\":\"dataqvi\",\"w\":\"datapvmndqmzcgqedono\"}}],\"userProperties\":[{\"name\":\"vq\",\"value\":\"datae\"},{\"name\":\"iy\",\"value\":\"dataeaahnkntldddk\"},{\"name\":\"pvusigw\",\"value\":\"datanq\"}],\"\":{\"gigcozksoodq\":\"datarji\",\"ke\":\"dataouw\",\"yapxnqvudfi\":\"datalbmw\"}}") .toObject(ValidationActivity.class); - Assertions.assertEquals("usbtwpvmzgnxepa", model.name()); - Assertions.assertEquals("v", model.description()); + Assertions.assertEquals("teoepdpxuzpqwfp", model.name()); + Assertions.assertEquals("xomonqqqike", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("lu", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("srrytkmf", model.userProperties().get(0).name()); - Assertions.assertEquals("gnplzbtvpuigtnjy", model.dataset().referenceName()); + Assertions.assertEquals("oydqgelccccccojn", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("vq", model.userProperties().get(0).name()); + Assertions.assertEquals("qejkmltwft", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ValidationActivity model = new ValidationActivity().withName("usbtwpvmzgnxepa").withDescription("v") - .withState(ActivityState.INACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) - .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("lu") - .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, - DependencyCondition.SUCCEEDED, DependencyCondition.COMPLETED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("kpiedcrtvdcbzpyn") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()))) - .withUserProperties(Arrays.asList(new UserProperty().withName("srrytkmf").withValue("databpcr"), - new UserProperty().withName("ynunrajtbumaid").withValue("datasn"))) - .withTimeout("datamxxgocpzqrbtyz").withSleep("datawrufiouafxp").withMinimumSize("datazmwx") - .withChildItems("datagi").withDataset(new DatasetReference().withReferenceName("gnplzbtvpuigtnjy") - .withParameters(mapOf("extlyyvebpykzhr", "datavvitxoitnqmiwlri"))); + ValidationActivity model = new ValidationActivity() + .withName("teoepdpxuzpqwfp").withDescription("xomonqqqike").withState( + ActivityState.INACTIVE) + .withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) + .withDependsOn( + Arrays.asList( + new ActivityDependency().withActivity("oydqgelccccccojn") + .withDependencyConditions( + Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency() + .withActivity("yaudellb").withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) + .withAdditionalProperties(mapOf()), + new ActivityDependency().withActivity("wnqfdgcrf") + .withDependencyConditions(Arrays.asList(DependencyCondition.COMPLETED, + DependencyCondition.SKIPPED, DependencyCondition.COMPLETED, DependencyCondition.SKIPPED)) + .withAdditionalProperties(mapOf()))) + .withUserProperties(Arrays.asList(new UserProperty().withName("vq").withValue("datae"), + new UserProperty().withName("iy").withValue("dataeaahnkntldddk"), + new UserProperty().withName("pvusigw").withValue("datanq"))) + .withTimeout("dataotvoszgcyhwj").withSleep("dataahqwvvferl").withMinimumSize("datafz") + .withChildItems("dataqsbjbshwxyfsk").withDataset( + new DatasetReference().withReferenceName("qejkmltwft").withParameters(mapOf("jjbdg", "datamtkxziowutc", + "vpwzv", "datarlohregqvusff", "orrnssthn", "datammirvmpiwo", "rxwabwdbclq", "datanzatdmncyl"))); model = BinaryData.fromObject(model).toObject(ValidationActivity.class); - Assertions.assertEquals("usbtwpvmzgnxepa", model.name()); - Assertions.assertEquals("v", model.description()); + Assertions.assertEquals("teoepdpxuzpqwfp", model.name()); + Assertions.assertEquals("xomonqqqike", model.description()); Assertions.assertEquals(ActivityState.INACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("lu", model.dependsOn().get(0).activity()); - Assertions.assertEquals(DependencyCondition.SUCCEEDED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("srrytkmf", model.userProperties().get(0).name()); - Assertions.assertEquals("gnplzbtvpuigtnjy", model.dataset().referenceName()); + Assertions.assertEquals("oydqgelccccccojn", model.dependsOn().get(0).activity()); + Assertions.assertEquals(DependencyCondition.SKIPPED, model.dependsOn().get(0).dependencyConditions().get(0)); + Assertions.assertEquals("vq", model.userProperties().get(0).name()); + Assertions.assertEquals("qejkmltwft", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTypePropertiesTests.java index e5689e8b661ee..47d0f53cd94d2 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ValidationActivityTypePropertiesTests.java @@ -15,21 +15,19 @@ public final class ValidationActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ValidationActivityTypeProperties model = BinaryData.fromString( - "{\"timeout\":\"datakigukfximwinwa\",\"sleep\":\"datarlvhlferiqend\",\"minimumSize\":\"dataccnghszgau\",\"childItems\":\"databizjbwufjogswf\",\"dataset\":{\"referenceName\":\"qeebpyp\",\"parameters\":{\"gxy\":\"datanveetaydh\",\"mawfzyvxkrtgofp\":\"dataobsxshjsra\",\"ngt\":\"datarhoujkcpyerf\",\"gsurfnktxht\":\"dataijbolksehtyx\"}}}") + "{\"timeout\":\"dataxlc\",\"sleep\":\"dataqhtgtadtootkgxx\",\"minimumSize\":\"dataenlqwxskltzzp\",\"childItems\":\"datawgtmpytomftubh\",\"dataset\":{\"referenceName\":\"bwgbvpyjpai\",\"parameters\":{\"llihwpsrdaoixgqt\":\"datadazxfz\",\"ix\":\"datasjnlekotqhd\"}}}") .toObject(ValidationActivityTypeProperties.class); - Assertions.assertEquals("qeebpyp", model.dataset().referenceName()); + Assertions.assertEquals("bwgbvpyjpai", model.dataset().referenceName()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ValidationActivityTypeProperties model - = new ValidationActivityTypeProperties().withTimeout("datakigukfximwinwa").withSleep("datarlvhlferiqend") - .withMinimumSize("dataccnghszgau").withChildItems("databizjbwufjogswf") - .withDataset(new DatasetReference().withReferenceName("qeebpyp") - .withParameters(mapOf("gxy", "datanveetaydh", "mawfzyvxkrtgofp", "dataobsxshjsra", "ngt", - "datarhoujkcpyerf", "gsurfnktxht", "dataijbolksehtyx"))); + ValidationActivityTypeProperties model = new ValidationActivityTypeProperties().withTimeout("dataxlc") + .withSleep("dataqhtgtadtootkgxx").withMinimumSize("dataenlqwxskltzzp").withChildItems("datawgtmpytomftubh") + .withDataset(new DatasetReference().withReferenceName("bwgbvpyjpai") + .withParameters(mapOf("llihwpsrdaoixgqt", "datadazxfz", "ix", "datasjnlekotqhd"))); model = BinaryData.fromObject(model).toObject(ValidationActivityTypeProperties.class); - Assertions.assertEquals("qeebpyp", model.dataset().referenceName()); + Assertions.assertEquals("bwgbvpyjpai", model.dataset().referenceName()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/VerticaSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/VerticaSourceTests.java index 7ea91ae459111..57f547e8f2ac7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/VerticaSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/VerticaSourceTests.java @@ -11,16 +11,16 @@ public final class VerticaSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VerticaSource model = BinaryData.fromString( - "{\"type\":\"VerticaSource\",\"query\":\"datahsodofsxjik\",\"queryTimeout\":\"datasquhu\",\"additionalColumns\":\"dataqwoggwcxdm\",\"sourceRetryCount\":\"datauwldfahkyft\",\"sourceRetryWait\":\"datakbgsgopyckmncru\",\"maxConcurrentConnections\":\"dataudjmdayqkgix\",\"disableMetricsCollection\":\"datarne\",\"\":{\"plzliizbwfjumulh\":\"dataldovpwrqcfzo\"}}") + "{\"type\":\"VerticaSource\",\"query\":\"dataddqxhegcolhqz\",\"queryTimeout\":\"datalqrunqwcrk\",\"additionalColumns\":\"databyxxyfnipy\",\"sourceRetryCount\":\"datajgfbsfsv\",\"sourceRetryWait\":\"datagejypok\",\"maxConcurrentConnections\":\"dataptnwpwskck\",\"disableMetricsCollection\":\"dataymf\",\"\":{\"pemxcdreqaqvsp\":\"datagvqioqrebwarljpl\",\"rbsc\":\"datayvearwt\"}}") .toObject(VerticaSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VerticaSource model - = new VerticaSource().withSourceRetryCount("datauwldfahkyft").withSourceRetryWait("datakbgsgopyckmncru") - .withMaxConcurrentConnections("dataudjmdayqkgix").withDisableMetricsCollection("datarne") - .withQueryTimeout("datasquhu").withAdditionalColumns("dataqwoggwcxdm").withQuery("datahsodofsxjik"); + VerticaSource model = new VerticaSource().withSourceRetryCount("datajgfbsfsv") + .withSourceRetryWait("datagejypok").withMaxConcurrentConnections("dataptnwpwskck") + .withDisableMetricsCollection("dataymf").withQueryTimeout("datalqrunqwcrk") + .withAdditionalColumns("databyxxyfnipy").withQuery("dataddqxhegcolhqz"); model = BinaryData.fromObject(model).toObject(VerticaSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTests.java index 1db7cb18f9615..038407f4bcba5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTests.java @@ -20,50 +20,40 @@ public final class WaitActivityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WaitActivity model = BinaryData.fromString( - "{\"type\":\"Wait\",\"typeProperties\":{\"waitTimeInSeconds\":\"databckfzbqvsg\"},\"name\":\"eijnvsjgnbdhh\",\"description\":\"f\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"dyxjicikzm\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Succeeded\"],\"\":{\"isvpfspfdf\":\"datairbnfcql\",\"hzotko\":\"dataymrfpqyxlncwagia\",\"uerhzyl\":\"datai\"}},{\"activity\":\"wymrmuioepi\",\"dependencyConditions\":[\"Failed\"],\"\":{\"szqzvemwnewm\":\"datahztvr\",\"ywdgrskdlt\":\"datawjcgryolbqcft\",\"vmcxljlpyh\":\"datafzyijn\",\"wtnqbqgfqi\":\"dataxvdjgcu\"}},{\"activity\":\"msxwe\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Failed\",\"Skipped\"],\"\":{\"bjn\":\"dataukyo\"}},{\"activity\":\"o\",\"dependencyConditions\":[\"Skipped\",\"Skipped\"],\"\":{\"wdbsvghyse\":\"databkwyvwvcgmfklqs\",\"vo\":\"dataqrb\",\"yibycoupksa\":\"dataqrwngfyjfquzxmtm\",\"sw\":\"datadjkrosqxvffrn\"}}],\"userProperties\":[{\"name\":\"yjoklngjsglzoirw\",\"value\":\"dataqdnasjupbgakks\"},{\"name\":\"zbdtvrg\",\"value\":\"dataebvqslikeuqv\"},{\"name\":\"iotvfcbgffd\",\"value\":\"dataff\"},{\"name\":\"atyqawtfyzqo\",\"value\":\"dataglixhapvwacwrcte\"}],\"\":{\"ble\":\"datacnknzncoxeop\",\"vr\":\"dataaxrsyxeqwgaeice\",\"ztssqbclaeciwz\":\"datadcidcxkywy\",\"qyinfdmqjqjk\":\"datavttkha\"}}") + "{\"type\":\"Wait\",\"typeProperties\":{\"waitTimeInSeconds\":\"datalcz\"},\"name\":\"lkmtrrcbu\",\"description\":\"auxkgklqucxew\",\"state\":\"Active\",\"onInactiveMarkAs\":\"Failed\",\"dependsOn\":[{\"activity\":\"mhkqzvarq\",\"dependencyConditions\":[\"Failed\",\"Failed\",\"Failed\"],\"\":{\"xhmdorxb\":\"dataimvnvxhxza\"}},{\"activity\":\"aprksoeq\",\"dependencyConditions\":[\"Succeeded\",\"Completed\",\"Skipped\",\"Failed\"],\"\":{\"aktgtwvzps\":\"datarcyexb\",\"gjidrag\":\"datagho\",\"yimbqdsu\":\"datacwcdbtop\",\"xwr\":\"dataazkouvvgcwsimhj\"}}],\"userProperties\":[{\"name\":\"ofwopzqxpk\",\"value\":\"datanxjmlys\"}],\"\":{\"lfahryuz\":\"databuxjhquzirhcghn\",\"ptpq\":\"dataeuegrdit\",\"xzfy\":\"dataajggmmiwoisql\"}}") .toObject(WaitActivity.class); - Assertions.assertEquals("eijnvsjgnbdhh", model.name()); - Assertions.assertEquals("f", model.description()); + Assertions.assertEquals("lkmtrrcbu", model.name()); + Assertions.assertEquals("auxkgklqucxew", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("dyxjicikzm", model.dependsOn().get(0).activity()); + Assertions.assertEquals("mhkqzvarq", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("yjoklngjsglzoirw", model.userProperties().get(0).name()); + Assertions.assertEquals("ofwopzqxpk", model.userProperties().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WaitActivity model = new WaitActivity().withName("eijnvsjgnbdhh").withDescription("f") + WaitActivity model = new WaitActivity().withName("lkmtrrcbu").withDescription("auxkgklqucxew") .withState(ActivityState.ACTIVE).withOnInactiveMarkAs(ActivityOnInactiveMarkAs.FAILED) .withDependsOn(Arrays.asList( - new ActivityDependency().withActivity("dyxjicikzm") + new ActivityDependency().withActivity("mhkqzvarq") .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.SUCCEEDED)) + DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("wymrmuioepi") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("msxwe") - .withDependencyConditions(Arrays.asList(DependencyCondition.FAILED, DependencyCondition.FAILED, - DependencyCondition.FAILED, DependencyCondition.SKIPPED)) - .withAdditionalProperties(mapOf()), - new ActivityDependency().withActivity("o") - .withDependencyConditions(Arrays.asList(DependencyCondition.SKIPPED, DependencyCondition.SKIPPED)) + new ActivityDependency().withActivity("aprksoeq") + .withDependencyConditions(Arrays.asList(DependencyCondition.SUCCEEDED, + DependencyCondition.COMPLETED, DependencyCondition.SKIPPED, DependencyCondition.FAILED)) .withAdditionalProperties(mapOf()))) - .withUserProperties( - Arrays.asList(new UserProperty().withName("yjoklngjsglzoirw").withValue("dataqdnasjupbgakks"), - new UserProperty().withName("zbdtvrg").withValue("dataebvqslikeuqv"), - new UserProperty().withName("iotvfcbgffd").withValue("dataff"), - new UserProperty().withName("atyqawtfyzqo").withValue("dataglixhapvwacwrcte"))) - .withWaitTimeInSeconds("databckfzbqvsg"); + .withUserProperties(Arrays.asList(new UserProperty().withName("ofwopzqxpk").withValue("datanxjmlys"))) + .withWaitTimeInSeconds("datalcz"); model = BinaryData.fromObject(model).toObject(WaitActivity.class); - Assertions.assertEquals("eijnvsjgnbdhh", model.name()); - Assertions.assertEquals("f", model.description()); + Assertions.assertEquals("lkmtrrcbu", model.name()); + Assertions.assertEquals("auxkgklqucxew", model.description()); Assertions.assertEquals(ActivityState.ACTIVE, model.state()); Assertions.assertEquals(ActivityOnInactiveMarkAs.FAILED, model.onInactiveMarkAs()); - Assertions.assertEquals("dyxjicikzm", model.dependsOn().get(0).activity()); + Assertions.assertEquals("mhkqzvarq", model.dependsOn().get(0).activity()); Assertions.assertEquals(DependencyCondition.FAILED, model.dependsOn().get(0).dependencyConditions().get(0)); - Assertions.assertEquals("yjoklngjsglzoirw", model.userProperties().get(0).name()); + Assertions.assertEquals("ofwopzqxpk", model.userProperties().get(0).name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTypePropertiesTests.java index 760cd30d5bd77..22a3e4f28f94c 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WaitActivityTypePropertiesTests.java @@ -11,12 +11,12 @@ public final class WaitActivityTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WaitActivityTypeProperties model - = BinaryData.fromString("{\"waitTimeInSeconds\":\"dataq\"}").toObject(WaitActivityTypeProperties.class); + = BinaryData.fromString("{\"waitTimeInSeconds\":\"datan\"}").toObject(WaitActivityTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WaitActivityTypeProperties model = new WaitActivityTypeProperties().withWaitTimeInSeconds("dataq"); + WaitActivityTypeProperties model = new WaitActivityTypeProperties().withWaitTimeInSeconds("datan"); model = BinaryData.fromObject(model).toObject(WaitActivityTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSinkTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSinkTests.java index 0b336777f2c2a..3c52e7a4cf335 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSinkTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSinkTests.java @@ -17,29 +17,27 @@ public final class WarehouseSinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WarehouseSink model = BinaryData.fromString( - "{\"type\":\"WarehouseSink\",\"preCopyScript\":\"datainfauytmqvsdyqyj\",\"allowCopyCommand\":\"datafotwmxedlcxmyxt\",\"copyCommandSettings\":{\"defaultValues\":[{\"columnName\":\"datajk\",\"defaultValue\":\"dataxantlpspi\"},{\"columnName\":\"dataf\",\"defaultValue\":\"datanqpkvvrhoqyv\"},{\"columnName\":\"datavnruodu\",\"defaultValue\":\"datahskh\"}],\"additionalOptions\":{\"h\":\"vocrddqxhegco\"}},\"tableOption\":\"datacklqrunqwcrkkaby\",\"writeBehavior\":\"datay\",\"writeBatchSize\":\"dataipywgjgfbsfsva\",\"writeBatchTimeout\":\"dataejypokk\",\"sinkRetryCount\":\"datatnwpwskckc\",\"sinkRetryWait\":\"datamfyxpgvqioqrebwa\",\"maxConcurrentConnections\":\"datajplk\",\"disableMetricsCollection\":\"datamxcdreqaqvspsy\",\"\":{\"vwmcrhyoestplmy\":\"datarwtkrbscwbi\",\"vyjzferhxg\":\"datan\",\"qxujxbdyczplmlj\":\"datatiawywp\"}}") + "{\"type\":\"WarehouseSink\",\"preCopyScript\":\"dataioqwmhcpujygnt\",\"allowCopyCommand\":\"datae\",\"copyCommandSettings\":{\"defaultValues\":[{\"columnName\":\"datath\",\"defaultValue\":\"datawy\"},{\"columnName\":\"dataxprsocfxlrzjjffl\",\"defaultValue\":\"datamuljfaulwlz\"},{\"columnName\":\"dataygnepj\",\"defaultValue\":\"dataxqdrphiyxjq\"}],\"additionalOptions\":{\"l\":\"pz\",\"wvio\":\"chpfzsfutaapb\"}},\"tableOption\":\"datayhsorcavkfhyoig\",\"writeBehavior\":\"dataedfterat\",\"writeBatchSize\":\"datakg\",\"writeBatchTimeout\":\"datarmujizd\",\"sinkRetryCount\":\"dataepfjdiwz\",\"sinkRetryWait\":\"datamumuc\",\"maxConcurrentConnections\":\"dataq\",\"disableMetricsCollection\":\"datascva\",\"\":{\"avgoullxpaylkras\":\"datagelnjgftqk\",\"skkziebmwyod\":\"datab\",\"qtxpf\":\"datamplgdxdtxbrdbw\",\"kdoukqsc\":\"datafrfvhbbnoevkkr\"}}") .toObject(WarehouseSink.class); - Assertions.assertEquals("vocrddqxhegco", model.copyCommandSettings().additionalOptions().get("h")); + Assertions.assertEquals("pz", model.copyCommandSettings().additionalOptions().get("l")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WarehouseSink model - = new WarehouseSink().withWriteBatchSize("dataipywgjgfbsfsva").withWriteBatchTimeout("dataejypokk") - .withSinkRetryCount("datatnwpwskckc").withSinkRetryWait("datamfyxpgvqioqrebwa") - .withMaxConcurrentConnections("datajplk").withDisableMetricsCollection("datamxcdreqaqvspsy") - .withPreCopyScript("datainfauytmqvsdyqyj").withAllowCopyCommand("datafotwmxedlcxmyxt") - .withCopyCommandSettings( - new DWCopyCommandSettings() - .withDefaultValues(Arrays.asList( - new DWCopyCommandDefaultValue().withColumnName("datajk").withDefaultValue("dataxantlpspi"), - new DWCopyCommandDefaultValue().withColumnName("dataf") - .withDefaultValue("datanqpkvvrhoqyv"), - new DWCopyCommandDefaultValue().withColumnName("datavnruodu").withDefaultValue("datahskh"))) - .withAdditionalOptions(mapOf("h", "vocrddqxhegco"))) - .withTableOption("datacklqrunqwcrkkaby").withWriteBehavior("datay"); + WarehouseSink model = new WarehouseSink().withWriteBatchSize("datakg").withWriteBatchTimeout("datarmujizd") + .withSinkRetryCount("dataepfjdiwz").withSinkRetryWait("datamumuc").withMaxConcurrentConnections("dataq") + .withDisableMetricsCollection("datascva").withPreCopyScript("dataioqwmhcpujygnt") + .withAllowCopyCommand("datae") + .withCopyCommandSettings(new DWCopyCommandSettings() + .withDefaultValues(Arrays.asList( + new DWCopyCommandDefaultValue().withColumnName("datath").withDefaultValue("datawy"), + new DWCopyCommandDefaultValue().withColumnName("dataxprsocfxlrzjjffl") + .withDefaultValue("datamuljfaulwlz"), + new DWCopyCommandDefaultValue().withColumnName("dataygnepj").withDefaultValue("dataxqdrphiyxjq"))) + .withAdditionalOptions(mapOf("l", "pz", "wvio", "chpfzsfutaapb"))) + .withTableOption("datayhsorcavkfhyoig").withWriteBehavior("dataedfterat"); model = BinaryData.fromObject(model).toObject(WarehouseSink.class); - Assertions.assertEquals("vocrddqxhegco", model.copyCommandSettings().additionalOptions().get("h")); + Assertions.assertEquals("pz", model.copyCommandSettings().additionalOptions().get("l")); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSourceTests.java index e0120c9a04e8d..b7f1f789703ce 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WarehouseSourceTests.java @@ -12,20 +12,20 @@ public final class WarehouseSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WarehouseSource model = BinaryData.fromString( - "{\"type\":\"WarehouseSource\",\"sqlReaderQuery\":\"datarzfppopwxxdgzhn\",\"sqlReaderStoredProcedureName\":\"databuwauytq\",\"storedProcedureParameters\":\"datagaxloafws\",\"isolationLevel\":\"dataxqrokw\",\"partitionOption\":\"dataipn\",\"partitionSettings\":{\"partitionColumnName\":\"datalxjrhctbrv\",\"partitionUpperBound\":\"datadamoyvqfjpkez\",\"partitionLowerBound\":\"dataizbyczmepcacgvln\"},\"queryTimeout\":\"datajb\",\"additionalColumns\":\"datayrktuvdestarulnh\",\"sourceRetryCount\":\"datatvyhsxhcrf\",\"sourceRetryWait\":\"dataxexupcuizvx\",\"maxConcurrentConnections\":\"datavzhlkeotdscqkxzr\",\"disableMetricsCollection\":\"dataoqzmvemli\",\"\":{\"nftrrhhgwawi\":\"datafq\",\"nvcf\":\"datah\",\"ctxu\":\"datadvkdnf\",\"xnepub\":\"datatxhqqvdhdyyad\"}}") + "{\"type\":\"WarehouseSource\",\"sqlReaderQuery\":\"dataycvcksz\",\"sqlReaderStoredProcedureName\":\"datagguucpytsxnuj\",\"storedProcedureParameters\":\"datafth\",\"isolationLevel\":\"dataiieoxlbcccc\",\"partitionOption\":\"datauabde\",\"partitionSettings\":{\"partitionColumnName\":\"databgcdxqgsteek\",\"partitionUpperBound\":\"dataksvvyvoib\",\"partitionLowerBound\":\"datauqwljmzp\"},\"queryTimeout\":\"datakrwvvhcgtct\",\"additionalColumns\":\"datadcgobkcebzrt\",\"sourceRetryCount\":\"datatmtjsklkwtnqqiqc\",\"sourceRetryWait\":\"datafxldqtma\",\"maxConcurrentConnections\":\"dataejnemrfqjhc\",\"disableMetricsCollection\":\"datawlezbfgullq\",\"\":{\"gksrorxejf\":\"datajyxcmqc\",\"ray\":\"dataarphltlf\",\"wbkxdhavegy\":\"dataxzdujpuhbaog\",\"pdatvndvwwejvqpw\":\"dataqsmlbzi\"}}") .toObject(WarehouseSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WarehouseSource model = new WarehouseSource().withSourceRetryCount("datatvyhsxhcrf") - .withSourceRetryWait("dataxexupcuizvx").withMaxConcurrentConnections("datavzhlkeotdscqkxzr") - .withDisableMetricsCollection("dataoqzmvemli").withQueryTimeout("datajb") - .withAdditionalColumns("datayrktuvdestarulnh").withSqlReaderQuery("datarzfppopwxxdgzhn") - .withSqlReaderStoredProcedureName("databuwauytq").withStoredProcedureParameters("datagaxloafws") - .withIsolationLevel("dataxqrokw").withPartitionOption("dataipn") - .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("datalxjrhctbrv") - .withPartitionUpperBound("datadamoyvqfjpkez").withPartitionLowerBound("dataizbyczmepcacgvln")); + WarehouseSource model = new WarehouseSource().withSourceRetryCount("datatmtjsklkwtnqqiqc") + .withSourceRetryWait("datafxldqtma").withMaxConcurrentConnections("dataejnemrfqjhc") + .withDisableMetricsCollection("datawlezbfgullq").withQueryTimeout("datakrwvvhcgtct") + .withAdditionalColumns("datadcgobkcebzrt").withSqlReaderQuery("dataycvcksz") + .withSqlReaderStoredProcedureName("datagguucpytsxnuj").withStoredProcedureParameters("datafth") + .withIsolationLevel("dataiieoxlbcccc").withPartitionOption("datauabde") + .withPartitionSettings(new SqlPartitionSettings().withPartitionColumnName("databgcdxqgsteek") + .withPartitionUpperBound("dataksvvyvoib").withPartitionLowerBound("datauqwljmzp")); model = BinaryData.fromObject(model).toObject(WarehouseSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebAnonymousAuthenticationTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebAnonymousAuthenticationTests.java index cb21f7b55ca20..03e20f15516d7 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebAnonymousAuthenticationTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebAnonymousAuthenticationTests.java @@ -11,13 +11,13 @@ public final class WebAnonymousAuthenticationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WebAnonymousAuthentication model - = BinaryData.fromString("{\"authenticationType\":\"Anonymous\",\"url\":\"datatwnqtjcem\"}") + = BinaryData.fromString("{\"authenticationType\":\"Anonymous\",\"url\":\"datahjkxg\"}") .toObject(WebAnonymousAuthentication.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WebAnonymousAuthentication model = new WebAnonymousAuthentication().withUrl("datatwnqtjcem"); + WebAnonymousAuthentication model = new WebAnonymousAuthentication().withUrl("datahjkxg"); model = BinaryData.fromObject(model).toObject(WebAnonymousAuthentication.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTests.java index 2c0934b6add7b..10cbe8860a9cf 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTests.java @@ -19,29 +19,36 @@ public final class WebLinkedServiceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WebLinkedService model = BinaryData.fromString( - "{\"type\":\"Web\",\"typeProperties\":{\"authenticationType\":\"WebLinkedServiceTypeProperties\",\"url\":\"databkeeohpfedx\"},\"connectVia\":{\"referenceName\":\"judekmxhwz\",\"parameters\":{\"ztu\":\"datacdfdxthq\",\"edijdhzkwvuewe\":\"datair\"}},\"description\":\"ajelswlxdknxctg\",\"parameters\":{\"igyi\":{\"type\":\"Float\",\"defaultValue\":\"datadfsg\"},\"c\":{\"type\":\"String\",\"defaultValue\":\"datajudxzfvna\"}},\"annotations\":[\"datahnqiyevjehts\",\"dataaoxxsehf\",\"datafmabpyzxc\",\"datalphfstvurtkc\"],\"\":{\"disxsrrhr\":\"datavaxwzbwxx\",\"jzzuimpiybmybtih\":\"datarbbedrn\",\"uvqqo\":\"datagxmfbik\"}}") + "{\"type\":\"Web\",\"typeProperties\":{\"authenticationType\":\"WebLinkedServiceTypeProperties\",\"url\":\"dataalkqlvbkz\"},\"connectVia\":{\"referenceName\":\"yrnww\",\"parameters\":{\"crigbjbelnqalbs\":\"datainiyoizuww\",\"bwxuypcuriwll\":\"dataqxajsiueaibqjbi\",\"lwjxrymiysgh\":\"datatclveqdqtzsh\",\"vdulmka\":\"datacmqdlkkwbdek\"}},\"description\":\"n\",\"parameters\":{\"fvoz\":{\"type\":\"Float\",\"defaultValue\":\"datatpialrqhwcxxccf\"},\"nosclujywwumbus\":{\"type\":\"SecureString\",\"defaultValue\":\"datavrexitpzri\"},\"uahokiclrmmudv\":{\"type\":\"Array\",\"defaultValue\":\"databncljkhmso\"},\"tlizdoys\":{\"type\":\"Array\",\"defaultValue\":\"databscidkwznw\"}},\"annotations\":[\"databogdj\",\"datauybcp\",\"datadvuotkvkbpmk\",\"datapbnkcwauylkbdsk\"],\"\":{\"zylemp\":\"datafrtlukaf\",\"npw\":\"datahyuxxeike\"}}") .toObject(WebLinkedService.class); - Assertions.assertEquals("judekmxhwz", model.connectVia().referenceName()); - Assertions.assertEquals("ajelswlxdknxctg", model.description()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("igyi").type()); + Assertions.assertEquals("yrnww", model.connectVia().referenceName()); + Assertions.assertEquals("n", model.description()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("fvoz").type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { WebLinkedService model = new WebLinkedService() - .withConnectVia(new IntegrationRuntimeReference().withReferenceName("judekmxhwz") - .withParameters(mapOf("ztu", "datacdfdxthq", "edijdhzkwvuewe", "datair"))) - .withDescription("ajelswlxdknxctg") - .withParameters(mapOf("igyi", - new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datadfsg"), "c", - new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("datajudxzfvna"))) - .withAnnotations(Arrays.asList("datahnqiyevjehts", "dataaoxxsehf", "datafmabpyzxc", "datalphfstvurtkc")) - .withTypeProperties(new WebLinkedServiceTypeProperties().withUrl("databkeeohpfedx")); + .withConnectVia(new IntegrationRuntimeReference().withReferenceName("yrnww") + .withParameters(mapOf("crigbjbelnqalbs", "datainiyoizuww", "bwxuypcuriwll", "dataqxajsiueaibqjbi", + "lwjxrymiysgh", "datatclveqdqtzsh", "vdulmka", "datacmqdlkkwbdek"))) + .withDescription("n") + .withParameters(mapOf("fvoz", + new ParameterSpecification().withType(ParameterType.FLOAT).withDefaultValue("datatpialrqhwcxxccf"), + "nosclujywwumbus", + new ParameterSpecification().withType(ParameterType.SECURE_STRING) + .withDefaultValue("datavrexitpzri"), + "uahokiclrmmudv", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("databncljkhmso"), + "tlizdoys", + new ParameterSpecification().withType(ParameterType.ARRAY).withDefaultValue("databscidkwznw"))) + .withAnnotations(Arrays.asList("databogdj", "datauybcp", "datadvuotkvkbpmk", "datapbnkcwauylkbdsk")) + .withTypeProperties(new WebLinkedServiceTypeProperties().withUrl("dataalkqlvbkz")); model = BinaryData.fromObject(model).toObject(WebLinkedService.class); - Assertions.assertEquals("judekmxhwz", model.connectVia().referenceName()); - Assertions.assertEquals("ajelswlxdknxctg", model.description()); - Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("igyi").type()); + Assertions.assertEquals("yrnww", model.connectVia().referenceName()); + Assertions.assertEquals("n", model.description()); + Assertions.assertEquals(ParameterType.FLOAT, model.parameters().get("fvoz").type()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTypePropertiesTests.java index 58350c0b04f1d..fe47b057a2fe5 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebLinkedServiceTypePropertiesTests.java @@ -10,14 +10,14 @@ public final class WebLinkedServiceTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WebLinkedServiceTypeProperties model - = BinaryData.fromString("{\"authenticationType\":\"WebLinkedServiceTypeProperties\",\"url\":\"dataht\"}") - .toObject(WebLinkedServiceTypeProperties.class); + WebLinkedServiceTypeProperties model = BinaryData + .fromString("{\"authenticationType\":\"WebLinkedServiceTypeProperties\",\"url\":\"datanhqafuvvys\"}") + .toObject(WebLinkedServiceTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WebLinkedServiceTypeProperties model = new WebLinkedServiceTypeProperties().withUrl("dataht"); + WebLinkedServiceTypeProperties model = new WebLinkedServiceTypeProperties().withUrl("datanhqafuvvys"); model = BinaryData.fromObject(model).toObject(WebLinkedServiceTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebSourceTests.java index 957602c18db2c..761da82b38e94 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebSourceTests.java @@ -11,15 +11,15 @@ public final class WebSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WebSource model = BinaryData.fromString( - "{\"type\":\"WebSource\",\"additionalColumns\":\"datasnfaxtoboafp\",\"sourceRetryCount\":\"datalojwczar\",\"sourceRetryWait\":\"datatauskish\",\"maxConcurrentConnections\":\"datatypgrkd\",\"disableMetricsCollection\":\"datazaunbwcqti\",\"\":{\"epplrdkcazujvhu\":\"datazdsnrjhjlplo\"}}") + "{\"type\":\"WebSource\",\"additionalColumns\":\"datahboplavgfbvro\",\"sourceRetryCount\":\"datauexqweyslwlppoh\",\"sourceRetryWait\":\"datafgalexy\",\"maxConcurrentConnections\":\"datagkadtwd\",\"disableMetricsCollection\":\"databjx\",\"\":{\"jkwltnsnhuvmok\":\"dataxcjdobsgvz\",\"dnlodkqrqnkptixa\":\"datahsclpnb\",\"zmhoplqtzgt\":\"dataoyzgaevrygggcc\"}}") .toObject(WebSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WebSource model = new WebSource().withSourceRetryCount("datalojwczar").withSourceRetryWait("datatauskish") - .withMaxConcurrentConnections("datatypgrkd").withDisableMetricsCollection("datazaunbwcqti") - .withAdditionalColumns("datasnfaxtoboafp"); + WebSource model = new WebSource().withSourceRetryCount("datauexqweyslwlppoh").withSourceRetryWait("datafgalexy") + .withMaxConcurrentConnections("datagkadtwd").withDisableMetricsCollection("databjx") + .withAdditionalColumns("datahboplavgfbvro"); model = BinaryData.fromObject(model).toObject(WebSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTests.java index 3d737ba9b691b..59c9a280cc296 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTests.java @@ -19,31 +19,32 @@ public final class WebTableDatasetTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WebTableDataset model = BinaryData.fromString( - "{\"type\":\"WebTable\",\"typeProperties\":{\"index\":\"datajxnavpyxqbkxdtb\",\"path\":\"dataihainzkefkzlxvc\"},\"description\":\"cgoeozlibcbnu\",\"structure\":\"datau\",\"schema\":\"dataajvvq\",\"linkedServiceName\":{\"referenceName\":\"honyonelivgtibt\",\"parameters\":{\"fytkhhkemrv\":\"dataqjcajg\",\"dyulglhelwr\":\"dataxeoj\",\"px\":\"dataklfqfx\",\"skvctvu\":\"dataogypbztgaexj\"}},\"parameters\":{\"cyxrn\":{\"type\":\"Object\",\"defaultValue\":\"datattmhlvr\"}},\"annotations\":[\"datafajnpdw\",\"datajggkwdepem\",\"dataiayfiqiidxco\",\"datajvudyhgtrttcuayi\"],\"folder\":{\"name\":\"nkmm\"},\"\":{\"qgqexowqzrtgqr\":\"dataf\",\"obothx\":\"datakkvfygkuobpwainp\",\"qgzyvextc\":\"dataewhpnyjt\",\"whdlrifioz\":\"dataslroldow\"}}") + "{\"type\":\"WebTable\",\"typeProperties\":{\"index\":\"datavapeakfdmc\",\"path\":\"datal\"},\"description\":\"lxkyoddoq\",\"structure\":\"datanqtrkicwhqyr\",\"schema\":\"datamndkrwwmurhvif\",\"linkedServiceName\":{\"referenceName\":\"eqfsrnackitl\",\"parameters\":{\"juniln\":\"databylpzjeldaq\",\"tnpkbvzpk\":\"datajhwcbrdsyp\",\"ldxuczlhvbqyczn\":\"datadngvnqdjgsbtwg\"}},\"parameters\":{\"g\":{\"type\":\"String\",\"defaultValue\":\"dataiuvcqoqkqwucqs\"},\"myvwp\":{\"type\":\"Object\",\"defaultValue\":\"datawtvmijccpkkjl\"}},\"annotations\":[\"dataaih\",\"datannlb\",\"dataxjppcbqetfzfppv\",\"datalzayjwdun\"],\"folder\":{\"name\":\"prklatwiuujxsuj\"},\"\":{\"cymgbfmdquyyaes\":\"datagxeegxbnjnczepd\",\"kihai\":\"datajxnavpyxqbkxdtb\",\"ozcgoeozlib\":\"datazkefkzlxv\"}}") .toObject(WebTableDataset.class); - Assertions.assertEquals("cgoeozlibcbnu", model.description()); - Assertions.assertEquals("honyonelivgtibt", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("cyxrn").type()); - Assertions.assertEquals("nkmm", model.folder().name()); + Assertions.assertEquals("lxkyoddoq", model.description()); + Assertions.assertEquals("eqfsrnackitl", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("g").type()); + Assertions.assertEquals("prklatwiuujxsuj", model.folder().name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WebTableDataset model = new WebTableDataset().withDescription("cgoeozlibcbnu").withStructure("datau") - .withSchema("dataajvvq") - .withLinkedServiceName(new LinkedServiceReference().withReferenceName("honyonelivgtibt") - .withParameters(mapOf("fytkhhkemrv", "dataqjcajg", "dyulglhelwr", "dataxeoj", "px", "dataklfqfx", - "skvctvu", "dataogypbztgaexj"))) - .withParameters(mapOf("cyxrn", - new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datattmhlvr"))) - .withAnnotations(Arrays.asList("datafajnpdw", "datajggkwdepem", "dataiayfiqiidxco", "datajvudyhgtrttcuayi")) - .withFolder(new DatasetFolder().withName("nkmm")).withIndex("datajxnavpyxqbkxdtb") - .withPath("dataihainzkefkzlxvc"); + WebTableDataset model = new WebTableDataset().withDescription("lxkyoddoq").withStructure("datanqtrkicwhqyr") + .withSchema("datamndkrwwmurhvif") + .withLinkedServiceName(new LinkedServiceReference().withReferenceName("eqfsrnackitl") + .withParameters(mapOf("juniln", "databylpzjeldaq", "tnpkbvzpk", "datajhwcbrdsyp", "ldxuczlhvbqyczn", + "datadngvnqdjgsbtwg"))) + .withParameters(mapOf("g", + new ParameterSpecification().withType(ParameterType.STRING).withDefaultValue("dataiuvcqoqkqwucqs"), + "myvwp", + new ParameterSpecification().withType(ParameterType.OBJECT).withDefaultValue("datawtvmijccpkkjl"))) + .withAnnotations(Arrays.asList("dataaih", "datannlb", "dataxjppcbqetfzfppv", "datalzayjwdun")) + .withFolder(new DatasetFolder().withName("prklatwiuujxsuj")).withIndex("datavapeakfdmc").withPath("datal"); model = BinaryData.fromObject(model).toObject(WebTableDataset.class); - Assertions.assertEquals("cgoeozlibcbnu", model.description()); - Assertions.assertEquals("honyonelivgtibt", model.linkedServiceName().referenceName()); - Assertions.assertEquals(ParameterType.OBJECT, model.parameters().get("cyxrn").type()); - Assertions.assertEquals("nkmm", model.folder().name()); + Assertions.assertEquals("lxkyoddoq", model.description()); + Assertions.assertEquals("eqfsrnackitl", model.linkedServiceName().referenceName()); + Assertions.assertEquals(ParameterType.STRING, model.parameters().get("g").type()); + Assertions.assertEquals("prklatwiuujxsuj", model.folder().name()); } // Use "Map.of" if available diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTypePropertiesTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTypePropertiesTests.java index df812f3a3ed6d..31fd776282132 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTypePropertiesTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/WebTableDatasetTypePropertiesTests.java @@ -11,14 +11,14 @@ public final class WebTableDatasetTypePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WebTableDatasetTypeProperties model - = BinaryData.fromString("{\"index\":\"datatcbiich\",\"path\":\"dataudsozodwjcfqoy\"}") + = BinaryData.fromString("{\"index\":\"databnunzuysajvvqlho\",\"path\":\"dataon\"}") .toObject(WebTableDatasetTypeProperties.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { WebTableDatasetTypeProperties model - = new WebTableDatasetTypeProperties().withIndex("datatcbiich").withPath("dataudsozodwjcfqoy"); + = new WebTableDatasetTypeProperties().withIndex("databnunzuysajvvqlho").withPath("dataon"); model = BinaryData.fromObject(model).toObject(WebTableDatasetTypeProperties.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XeroSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XeroSourceTests.java index 086999003a323..f220f1b448e2b 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XeroSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XeroSourceTests.java @@ -11,15 +11,15 @@ public final class XeroSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { XeroSource model = BinaryData.fromString( - "{\"type\":\"XeroSource\",\"query\":\"dataxtkmknacnfzcy\",\"queryTimeout\":\"datahdjpagwszm\",\"additionalColumns\":\"datagzfeyexbg\",\"sourceRetryCount\":\"datayo\",\"sourceRetryWait\":\"datawigvqgc\",\"maxConcurrentConnections\":\"datacqjg\",\"disableMetricsCollection\":\"dataxpbpj\",\"\":{\"ohehhtl\":\"datanvdabaodiytxq\"}}") + "{\"type\":\"XeroSource\",\"query\":\"databetzydtgpvnczf\",\"queryTimeout\":\"dataybjku\",\"additionalColumns\":\"dataajkyrhucbfkaqlp\",\"sourceRetryCount\":\"dataptero\",\"sourceRetryWait\":\"dataqaktao\",\"maxConcurrentConnections\":\"datagefobcqvzmyw\",\"disableMetricsCollection\":\"datayns\",\"\":{\"kklzabauvncln\":\"dataosqvojgol\",\"ikireetvjfizafd\":\"dataaoidjhoykgtyvrn\",\"csipfwlye\":\"datajhnuvndgrolgxa\",\"rzfppopwxxdgzhn\":\"dataajdpjmqteirrjjm\"}}") .toObject(XeroSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - XeroSource model = new XeroSource().withSourceRetryCount("datayo").withSourceRetryWait("datawigvqgc") - .withMaxConcurrentConnections("datacqjg").withDisableMetricsCollection("dataxpbpj") - .withQueryTimeout("datahdjpagwszm").withAdditionalColumns("datagzfeyexbg").withQuery("dataxtkmknacnfzcy"); + XeroSource model = new XeroSource().withSourceRetryCount("dataptero").withSourceRetryWait("dataqaktao") + .withMaxConcurrentConnections("datagefobcqvzmyw").withDisableMetricsCollection("datayns") + .withQueryTimeout("dataybjku").withAdditionalColumns("dataajkyrhucbfkaqlp").withQuery("databetzydtgpvnczf"); model = BinaryData.fromObject(model).toObject(XeroSource.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlReadSettingsTests.java index 4aadd3dc2037c..3d962e7b777c6 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlReadSettingsTests.java @@ -14,7 +14,7 @@ public final class XmlReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { XmlReadSettings model = BinaryData.fromString( - "{\"type\":\"XmlReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"fkdsclqwkddge\":\"datayxbatrviynlslgx\"}},\"validationMode\":\"datavam\",\"detectDataType\":\"datafxgsha\",\"namespaces\":\"dataueq\",\"namespacePrefixes\":\"dataia\",\"\":{\"jtunlllmtiy\":\"datayxsoxqa\"}}") + "{\"type\":\"XmlReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"cbuzudkqoeoukvi\":\"dataubziibuabpvdwhvn\",\"fhdyasklmy\":\"datakgbkzqbo\",\"awljatvfddq\":\"datahclxwede\",\"xcqcwbxxvcvek\":\"datapduttqjtszq\"}},\"validationMode\":\"datadrum\",\"detectDataType\":\"datakdwwq\",\"namespaces\":\"datalfvmwuyar\",\"namespacePrefixes\":\"datasvtzotmwxq\",\"\":{\"wrtmjskb\":\"datadanfexlawkeqjhz\",\"mvounbyvsfqu\":\"dataenjnady\",\"xqbknoxjhedwh\":\"datar\",\"rpajbiig\":\"datamwb\"}}") .toObject(XmlReadSettings.class); } @@ -23,8 +23,8 @@ public void testSerialize() throws Exception { XmlReadSettings model = new XmlReadSettings() .withCompressionProperties( new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings"))) - .withValidationMode("datavam").withDetectDataType("datafxgsha").withNamespaces("dataueq") - .withNamespacePrefixes("dataia"); + .withValidationMode("datadrum").withDetectDataType("datakdwwq").withNamespaces("datalfvmwuyar") + .withNamespacePrefixes("datasvtzotmwxq"); model = BinaryData.fromObject(model).toObject(XmlReadSettings.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlSourceTests.java index 67e591dfcfbcc..86fe70d75b106 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/XmlSourceTests.java @@ -16,22 +16,23 @@ public final class XmlSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { XmlSource model = BinaryData.fromString( - "{\"type\":\"XmlSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datavcv\",\"disableMetricsCollection\":\"datalsb\",\"\":{\"teklgs\":\"datadjvyclasdprknup\",\"oaol\":\"datapvqczpk\",\"wcs\":\"datah\",\"euimtxmd\":\"dataebtjg\"}},\"formatSettings\":{\"type\":\"XmlReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"irnivf\":\"dataguk\"}},\"validationMode\":\"databskinmxanjgua\",\"detectDataType\":\"dataipgfxstw\",\"namespaces\":\"datazeqjno\",\"namespacePrefixes\":\"dataujliti\",\"\":{\"pbxspvkcng\":\"datamqxgjzslhopyqx\",\"klkhhjxww\":\"dataocnwnjmiitlamf\",\"wznwc\":\"datazsyetbffrhqz\",\"purv\":\"dataoapdtjkltetf\"}},\"additionalColumns\":\"datatjb\",\"sourceRetryCount\":\"datafume\",\"sourceRetryWait\":\"dataaafkvqhbw\",\"maxConcurrentConnections\":\"dataivqkyaghf\",\"disableMetricsCollection\":\"datablszsvegu\",\"\":{\"iufqxrlzij\":\"dataijpnymbjrs\"}}") + "{\"type\":\"XmlSource\",\"storeSettings\":{\"type\":\"StoreReadSettings\",\"maxConcurrentConnections\":\"datanlihbku\",\"disableMetricsCollection\":\"dataeywyftvy\",\"\":{\"scyzvv\":\"dataqzjfvbnyyjvz\",\"ssgbscq\":\"dataxmy\",\"qiparctshe\":\"dataeixazebmmjaigax\",\"fawhoosrsol\":\"datagtdvhokx\"}},\"formatSettings\":{\"type\":\"XmlReadSettings\",\"compressionProperties\":{\"type\":\"CompressionReadSettings\",\"\":{\"nudifierxx\":\"dataoejbgiqhjpe\"}},\"validationMode\":\"datasdvuirqfks\",\"detectDataType\":\"datalfgmdoaihlvrsqc\",\"namespaces\":\"datamirybwga\",\"namespacePrefixes\":\"datavwkynemazgtbynx\",\"\":{\"zvuzxx\":\"dataawexgeqo\",\"hbobuovsvwnpcx\":\"dataojj\"}},\"additionalColumns\":\"datamtvparyubny\",\"sourceRetryCount\":\"datalpz\",\"sourceRetryWait\":\"datavotuc\",\"maxConcurrentConnections\":\"databp\",\"disableMetricsCollection\":\"dataumqzftzoe\",\"\":{\"nuqsgertxicemgs\":\"datanvnooklgr\",\"edbsl\":\"datacbbdokph\",\"k\":\"datanunpxswmcc\"}}") .toObject(XmlSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - XmlSource model = new XmlSource().withSourceRetryCount("datafume").withSourceRetryWait("dataaafkvqhbw") - .withMaxConcurrentConnections("dataivqkyaghf").withDisableMetricsCollection("datablszsvegu") - .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datavcv") - .withDisableMetricsCollection("datalsb").withAdditionalProperties(mapOf("type", "StoreReadSettings"))) + XmlSource model = new XmlSource().withSourceRetryCount("datalpz").withSourceRetryWait("datavotuc") + .withMaxConcurrentConnections("databp").withDisableMetricsCollection("dataumqzftzoe") + .withStoreSettings(new StoreReadSettings().withMaxConcurrentConnections("datanlihbku") + .withDisableMetricsCollection("dataeywyftvy") + .withAdditionalProperties(mapOf("type", "StoreReadSettings"))) .withFormatSettings(new XmlReadSettings() .withCompressionProperties( new CompressionReadSettings().withAdditionalProperties(mapOf("type", "CompressionReadSettings"))) - .withValidationMode("databskinmxanjgua").withDetectDataType("dataipgfxstw").withNamespaces("datazeqjno") - .withNamespacePrefixes("dataujliti")) - .withAdditionalColumns("datatjb"); + .withValidationMode("datasdvuirqfks").withDetectDataType("datalfgmdoaihlvrsqc") + .withNamespaces("datamirybwga").withNamespacePrefixes("datavwkynemazgtbynx")) + .withAdditionalColumns("datamtvparyubny"); model = BinaryData.fromObject(model).toObject(XmlSource.class); } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZipDeflateReadSettingsTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZipDeflateReadSettingsTests.java index 2beffaa836e21..61a6b69235d0d 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZipDeflateReadSettingsTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZipDeflateReadSettingsTests.java @@ -11,14 +11,13 @@ public final class ZipDeflateReadSettingsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ZipDeflateReadSettings model = BinaryData.fromString( - "{\"type\":\"ZipDeflateReadSettings\",\"preserveZipFileNameAsFolder\":\"datavzixmusiidivbbrt\",\"\":{\"oegxoqpuclidy\":\"dataiqnt\",\"ukaffzzfgivfiypf\":\"datawdaiexisapygii\",\"u\":\"datawyzjsixorvigrxmp\"}}") + "{\"type\":\"ZipDeflateReadSettings\",\"preserveZipFileNameAsFolder\":\"databq\",\"\":{\"wiavmqutgxd\":\"dataxzyj\",\"rskyl\":\"dataznfokcb\"}}") .toObject(ZipDeflateReadSettings.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ZipDeflateReadSettings model - = new ZipDeflateReadSettings().withPreserveZipFileNameAsFolder("datavzixmusiidivbbrt"); + ZipDeflateReadSettings model = new ZipDeflateReadSettings().withPreserveZipFileNameAsFolder("databq"); model = BinaryData.fromObject(model).toObject(ZipDeflateReadSettings.class); } } diff --git a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZohoSourceTests.java b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZohoSourceTests.java index 078cf21643d59..9d59cb25bced3 100644 --- a/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZohoSourceTests.java +++ b/sdk/datafactory/azure-resourcemanager-datafactory/src/test/java/com/azure/resourcemanager/datafactory/generated/ZohoSourceTests.java @@ -11,16 +11,15 @@ public final class ZohoSourceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ZohoSource model = BinaryData.fromString( - "{\"type\":\"ZohoSource\",\"query\":\"datavijdr\",\"queryTimeout\":\"datayqvhz\",\"additionalColumns\":\"datayvhrenozl\",\"sourceRetryCount\":\"dataqfghlosho\",\"sourceRetryWait\":\"datakpcmtsbandesalv\",\"maxConcurrentConnections\":\"datawrljmlo\",\"disableMetricsCollection\":\"datatzvtfyqe\",\"\":{\"xhcygfg\":\"databsyni\",\"aosttbwap\":\"datamdbazggr\"}}") + "{\"type\":\"ZohoSource\",\"query\":\"databuwauytq\",\"queryTimeout\":\"datagaxloafws\",\"additionalColumns\":\"dataxqrokw\",\"sourceRetryCount\":\"dataipn\",\"sourceRetryWait\":\"dataql\",\"maxConcurrentConnections\":\"datarhctbrvegdamoy\",\"disableMetricsCollection\":\"datafjpkezqjizbyczme\",\"\":{\"destarulnhbqt\":\"datacgvlnpjjbyryrktu\",\"xhcrffj\":\"datayh\",\"svzhlkeot\":\"dataexupcuizvx\"}}") .toObject(ZohoSource.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ZohoSource model - = new ZohoSource().withSourceRetryCount("dataqfghlosho").withSourceRetryWait("datakpcmtsbandesalv") - .withMaxConcurrentConnections("datawrljmlo").withDisableMetricsCollection("datatzvtfyqe") - .withQueryTimeout("datayqvhz").withAdditionalColumns("datayvhrenozl").withQuery("datavijdr"); + ZohoSource model = new ZohoSource().withSourceRetryCount("dataipn").withSourceRetryWait("dataql") + .withMaxConcurrentConnections("datarhctbrvegdamoy").withDisableMetricsCollection("datafjpkezqjizbyczme") + .withQueryTimeout("datagaxloafws").withAdditionalColumns("dataxqrokw").withQuery("databuwauytq"); model = BinaryData.fromObject(model).toObject(ZohoSource.class); } } diff --git a/sdk/datafactory/tests.mgmt.yml b/sdk/datafactory/tests.mgmt.yml index 00ba8877cd40d..13282d8c8b956 100644 --- a/sdk/datafactory/tests.mgmt.yml +++ b/sdk/datafactory/tests.mgmt.yml @@ -2,12 +2,12 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: datafactory - Artifacts: - - name: azure-resourcemanager-datafactory - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdatafactory - Clouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: datafactory + Artifacts: + - name: azure-resourcemanager-datafactory + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdatafactory + Clouds: 'Public' diff --git a/sdk/datalakeanalytics/azure-resourcemanager-datalakeanalytics/pom.xml b/sdk/datalakeanalytics/azure-resourcemanager-datalakeanalytics/pom.xml index c16e35bacc1ed..e194903aba1b0 100644 --- a/sdk/datalakeanalytics/azure-resourcemanager-datalakeanalytics/pom.xml +++ b/sdk/datalakeanalytics/azure-resourcemanager-datalakeanalytics/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/datalakestore/azure-resourcemanager-datalakestore/pom.xml b/sdk/datalakestore/azure-resourcemanager-datalakestore/pom.xml index f85bd8a93eecf..6f045ae20b2cf 100644 --- a/sdk/datalakestore/azure-resourcemanager-datalakestore/pom.xml +++ b/sdk/datalakestore/azure-resourcemanager-datalakestore/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/datamigration/azure-resourcemanager-datamigration/pom.xml b/sdk/datamigration/azure-resourcemanager-datamigration/pom.xml index c16a497bc3f07..30f739386ae00 100644 --- a/sdk/datamigration/azure-resourcemanager-datamigration/pom.xml +++ b/sdk/datamigration/azure-resourcemanager-datamigration/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/dataprotection/azure-resourcemanager-dataprotection/pom.xml b/sdk/dataprotection/azure-resourcemanager-dataprotection/pom.xml index ec385b21fdc84..aa92a4dceb93f 100644 --- a/sdk/dataprotection/azure-resourcemanager-dataprotection/pom.xml +++ b/sdk/dataprotection/azure-resourcemanager-dataprotection/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/dataprotection/tests.mgmt.yml b/sdk/dataprotection/tests.mgmt.yml index 71e7d6ceda90b..cc4c47b72418e 100644 --- a/sdk/dataprotection/tests.mgmt.yml +++ b/sdk/dataprotection/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: dataprotection - Artifacts: - - name: azure-resourcemanager-dataprotection - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdataprotection - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: dataprotection + Artifacts: + - name: azure-resourcemanager-dataprotection + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdataprotection + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/defendereasm/azure-resourcemanager-defendereasm/pom.xml b/sdk/defendereasm/azure-resourcemanager-defendereasm/pom.xml index 5cffac5aa7cc1..cb7e04ad7f89f 100644 --- a/sdk/defendereasm/azure-resourcemanager-defendereasm/pom.xml +++ b/sdk/defendereasm/azure-resourcemanager-defendereasm/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/delegatednetwork/azure-resourcemanager-delegatednetwork/pom.xml b/sdk/delegatednetwork/azure-resourcemanager-delegatednetwork/pom.xml index 3f9e14b5a2d2d..f514edbf309c4 100644 --- a/sdk/delegatednetwork/azure-resourcemanager-delegatednetwork/pom.xml +++ b/sdk/delegatednetwork/azure-resourcemanager-delegatednetwork/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/deploymentmanager/azure-resourcemanager-deploymentmanager/pom.xml b/sdk/deploymentmanager/azure-resourcemanager-deploymentmanager/pom.xml index c11e56670b4d4..b6929e84a50c1 100644 --- a/sdk/deploymentmanager/azure-resourcemanager-deploymentmanager/pom.xml +++ b/sdk/deploymentmanager/azure-resourcemanager-deploymentmanager/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/desktopvirtualization/azure-resourcemanager-desktopvirtualization/pom.xml b/sdk/desktopvirtualization/azure-resourcemanager-desktopvirtualization/pom.xml index 6a3282ece5793..c096bf3dfa303 100644 --- a/sdk/desktopvirtualization/azure-resourcemanager-desktopvirtualization/pom.xml +++ b/sdk/desktopvirtualization/azure-resourcemanager-desktopvirtualization/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/devcenter/azure-developer-devcenter/pom.xml b/sdk/devcenter/azure-developer-devcenter/pom.xml index d4b84fb925e69..702f2cc624652 100644 --- a/sdk/devcenter/azure-developer-devcenter/pom.xml +++ b/sdk/devcenter/azure-developer-devcenter/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/devcenter/azure-resourcemanager-devcenter/pom.xml b/sdk/devcenter/azure-resourcemanager-devcenter/pom.xml index 1b64203ce232c..9e402e9f43b8c 100644 --- a/sdk/devcenter/azure-resourcemanager-devcenter/pom.xml +++ b/sdk/devcenter/azure-resourcemanager-devcenter/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/devhub/azure-resourcemanager-devhub/pom.xml b/sdk/devhub/azure-resourcemanager-devhub/pom.xml index 9270075ce9eaa..39a36eca463d8 100644 --- a/sdk/devhub/azure-resourcemanager-devhub/pom.xml +++ b/sdk/devhub/azure-resourcemanager-devhub/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/deviceprovisioningservices/azure-resourcemanager-deviceprovisioningservices/pom.xml b/sdk/deviceprovisioningservices/azure-resourcemanager-deviceprovisioningservices/pom.xml index e5993a359807f..0119b8277b756 100644 --- a/sdk/deviceprovisioningservices/azure-resourcemanager-deviceprovisioningservices/pom.xml +++ b/sdk/deviceprovisioningservices/azure-resourcemanager-deviceprovisioningservices/pom.xml @@ -61,13 +61,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/deviceprovisioningservices/tests.mgmt.yml b/sdk/deviceprovisioningservices/tests.mgmt.yml index 3a896b5ea23b6..536d49e31d9ac 100644 --- a/sdk/deviceprovisioningservices/tests.mgmt.yml +++ b/sdk/deviceprovisioningservices/tests.mgmt.yml @@ -2,12 +2,12 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: deviceprovisioningservices - Artifacts: - - name: azure-resourcemanager-deviceprovisioningservices - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdeviceprovisioningservices - Clouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: deviceprovisioningservices + Artifacts: + - name: azure-resourcemanager-deviceprovisioningservices + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdeviceprovisioningservices + Clouds: 'Public' diff --git a/sdk/deviceupdate/azure-iot-deviceupdate/CHANGELOG.md b/sdk/deviceupdate/azure-iot-deviceupdate/CHANGELOG.md index a706842fe74cc..d556a7bf9e004 100644 --- a/sdk/deviceupdate/azure-iot-deviceupdate/CHANGELOG.md +++ b/sdk/deviceupdate/azure-iot-deviceupdate/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.0.16 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.0.15 (2024-02-20) ### Other Changes diff --git a/sdk/deviceupdate/azure-iot-deviceupdate/pom.xml b/sdk/deviceupdate/azure-iot-deviceupdate/pom.xml index a4baa413fc68a..8cf2b127b27a7 100644 --- a/sdk/deviceupdate/azure-iot-deviceupdate/pom.xml +++ b/sdk/deviceupdate/azure-iot-deviceupdate/pom.xml @@ -49,7 +49,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/deviceupdate/azure-resourcemanager-deviceupdate/pom.xml b/sdk/deviceupdate/azure-resourcemanager-deviceupdate/pom.xml index 1b6a86d2c7836..85efb0e014c92 100644 --- a/sdk/deviceupdate/azure-resourcemanager-deviceupdate/pom.xml +++ b/sdk/deviceupdate/azure-resourcemanager-deviceupdate/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/deviceupdate/tests.mgmt.yml b/sdk/deviceupdate/tests.mgmt.yml index e4ef73a6be61b..cf6f6e094eeb3 100644 --- a/sdk/deviceupdate/tests.mgmt.yml +++ b/sdk/deviceupdate/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: deviceupdate - Artifacts: - - name: azure-resourcemanager-deviceupdate - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdeviceupdate - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: deviceupdate + Artifacts: + - name: azure-resourcemanager-deviceupdate + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdeviceupdate + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/devspaces/azure-resourcemanager-devspaces/pom.xml b/sdk/devspaces/azure-resourcemanager-devspaces/pom.xml index 23e48e748116c..56901ab2c2925 100644 --- a/sdk/devspaces/azure-resourcemanager-devspaces/pom.xml +++ b/sdk/devspaces/azure-resourcemanager-devspaces/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/devtestlabs/azure-resourcemanager-devtestlabs/pom.xml b/sdk/devtestlabs/azure-resourcemanager-devtestlabs/pom.xml index 05216aabe5429..10a066cf61746 100644 --- a/sdk/devtestlabs/azure-resourcemanager-devtestlabs/pom.xml +++ b/sdk/devtestlabs/azure-resourcemanager-devtestlabs/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/devtestlabs/tests.mgmt.yml b/sdk/devtestlabs/tests.mgmt.yml index f8ba09ff24b13..ca3526493ca71 100644 --- a/sdk/devtestlabs/tests.mgmt.yml +++ b/sdk/devtestlabs/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: devtestlabs - Artifacts: - - name: azure-resourcemanager-devtestlabs - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerdevtestlabs - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: devtestlabs + Artifacts: + - name: azure-resourcemanager-devtestlabs + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerdevtestlabs + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/digitaltwins/azure-digitaltwins-core/CHANGELOG.md b/sdk/digitaltwins/azure-digitaltwins-core/CHANGELOG.md index 0377427ca9211..29d8ed7ad4b3f 100644 --- a/sdk/digitaltwins/azure-digitaltwins-core/CHANGELOG.md +++ b/sdk/digitaltwins/azure-digitaltwins-core/CHANGELOG.md @@ -10,6 +10,18 @@ ### Other Changes + +## 1.3.18 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-serializer-json-jackson` from `1.4.9` to version `1.4.10`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.3.17 (2024-02-20) ### Other Changes diff --git a/sdk/digitaltwins/azure-digitaltwins-core/pom.xml b/sdk/digitaltwins/azure-digitaltwins-core/pom.xml index 530f1cb7a1c54..0c2c1d0f15909 100644 --- a/sdk/digitaltwins/azure-digitaltwins-core/pom.xml +++ b/sdk/digitaltwins/azure-digitaltwins-core/pom.xml @@ -76,7 +76,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/digitaltwins/azure-resourcemanager-digitaltwins/pom.xml b/sdk/digitaltwins/azure-resourcemanager-digitaltwins/pom.xml index e9a783a8fe1a8..fdacd0ad9c80a 100644 --- a/sdk/digitaltwins/azure-resourcemanager-digitaltwins/pom.xml +++ b/sdk/digitaltwins/azure-resourcemanager-digitaltwins/pom.xml @@ -70,7 +70,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -96,13 +96,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -114,7 +114,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/digitaltwins/tests.yml b/sdk/digitaltwins/tests.yml index 9e6266a5fc0ea..3993f08f4162d 100644 --- a/sdk/digitaltwins/tests.yml +++ b/sdk/digitaltwins/tests.yml @@ -1,12 +1,12 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - Artifacts: - - name: azure-digitaltwins-core - groupId: com.azure - safeName: azuredigitaltwinscore - ServiceDirectory: digitaltwins - Clouds: 'Preview' - Location: westus2 +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + Artifacts: + - name: azure-digitaltwins-core + groupId: com.azure + safeName: azuredigitaltwinscore + ServiceDirectory: digitaltwins + Clouds: 'Preview' + Location: westus2 diff --git a/sdk/dnsresolver/azure-resourcemanager-dnsresolver/pom.xml b/sdk/dnsresolver/azure-resourcemanager-dnsresolver/pom.xml index 551eb33b4e06d..a29329979aaa9 100644 --- a/sdk/dnsresolver/azure-resourcemanager-dnsresolver/pom.xml +++ b/sdk/dnsresolver/azure-resourcemanager-dnsresolver/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -80,13 +80,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/documentintelligence/azure-ai-documentintelligence/CHANGELOG.md b/sdk/documentintelligence/azure-ai-documentintelligence/CHANGELOG.md index a6fff526cd514..37a5738eca7db 100644 --- a/sdk/documentintelligence/azure-ai-documentintelligence/CHANGELOG.md +++ b/sdk/documentintelligence/azure-ai-documentintelligence/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 1.0.0-beta.3 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 1.0.0-beta.2 (2024-03-06) ### Features Added diff --git a/sdk/documentintelligence/azure-ai-documentintelligence/pom.xml b/sdk/documentintelligence/azure-ai-documentintelligence/pom.xml index 4ace7c0aa9750..57ccc71e79458 100644 --- a/sdk/documentintelligence/azure-ai-documentintelligence/pom.xml +++ b/sdk/documentintelligence/azure-ai-documentintelligence/pom.xml @@ -13,7 +13,7 @@ com.azure azure-ai-documentintelligence - 1.0.0-beta.2 + 1.0.0-beta.3 Microsoft Azure client library for Document Intelligence This package contains the Microsoft Azure AI Document Intelligence SDK. @@ -95,7 +95,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/documentintelligence/tests.native.yml b/sdk/documentintelligence/tests.native.yml index 4fcec3def8e6b..d2abc732b068c 100644 --- a/sdk/documentintelligence/tests.native.yml +++ b/sdk/documentintelligence/tests.native.yml @@ -1,16 +1,16 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-native-tests.yml - parameters: - ServiceDirectory: documentintelligence - timeoutInMinutes: 150 # how long to run the job before automatically cancelling - Artifacts: - - name: azure-ai-documentintelligence - groupId: com.azure - safeName: azureaidocumentintelligence - CloudConfig: - Public: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - Location: 'eastus' - SupportedClouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-native-tests.yml + parameters: + ServiceDirectory: documentintelligence + timeoutInMinutes: 150 # how long to run the job before automatically cancelling + Artifacts: + - name: azure-ai-documentintelligence + groupId: com.azure + safeName: azureaidocumentintelligence + CloudConfig: + Public: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + Location: 'eastus' + SupportedClouds: 'Public' diff --git a/sdk/documentintelligence/tests.yml b/sdk/documentintelligence/tests.yml index dd63b52a854ca..2a1e7923fc0cb 100644 --- a/sdk/documentintelligence/tests.yml +++ b/sdk/documentintelligence/tests.yml @@ -1,16 +1,16 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: documentintelligence - timeoutInMinutes: 150 # how long to run the job before automatically cancelling - Artifacts: - - name: azure-ai-documentintelligence - groupId: com.azure - safeName: azureaidocumentintelligence - CloudConfig: - Public: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - Location: 'centraluseuap' - SupportedClouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: documentintelligence + timeoutInMinutes: 150 # how long to run the job before automatically cancelling + Artifacts: + - name: azure-ai-documentintelligence + groupId: com.azure + safeName: azureaidocumentintelligence + CloudConfig: + Public: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + Location: 'centraluseuap' + SupportedClouds: 'Public' diff --git a/sdk/dynatrace/azure-resourcemanager-dynatrace/pom.xml b/sdk/dynatrace/azure-resourcemanager-dynatrace/pom.xml index 134867e67d16d..cbac47b060e0c 100644 --- a/sdk/dynatrace/azure-resourcemanager-dynatrace/pom.xml +++ b/sdk/dynatrace/azure-resourcemanager-dynatrace/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/e2e/pom.xml b/sdk/e2e/pom.xml index 061612f294e5b..8626abd61d7a0 100644 --- a/sdk/e2e/pom.xml +++ b/sdk/e2e/pom.xml @@ -82,7 +82,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/easm/azure-analytics-defender-easm/pom.xml b/sdk/easm/azure-analytics-defender-easm/pom.xml index 7176d375935ce..dc146927a0374 100644 --- a/sdk/easm/azure-analytics-defender-easm/pom.xml +++ b/sdk/easm/azure-analytics-defender-easm/pom.xml @@ -78,13 +78,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -96,7 +96,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/education/azure-resourcemanager-education/pom.xml b/sdk/education/azure-resourcemanager-education/pom.xml index 3b7982ed959a8..1c6f0415952fb 100644 --- a/sdk/education/azure-resourcemanager-education/pom.xml +++ b/sdk/education/azure-resourcemanager-education/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -80,13 +80,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/elastic/azure-resourcemanager-elastic/pom.xml b/sdk/elastic/azure-resourcemanager-elastic/pom.xml index 4a7883e7abe10..82349f42d5f2e 100644 --- a/sdk/elastic/azure-resourcemanager-elastic/pom.xml +++ b/sdk/elastic/azure-resourcemanager-elastic/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/elasticsan/azure-resourcemanager-elasticsan/pom.xml b/sdk/elasticsan/azure-resourcemanager-elasticsan/pom.xml index 92b848c96b853..e30ed6ed2c50f 100644 --- a/sdk/elasticsan/azure-resourcemanager-elasticsan/pom.xml +++ b/sdk/elasticsan/azure-resourcemanager-elasticsan/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -96,13 +96,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/elasticsan/tests.mgmt.yml b/sdk/elasticsan/tests.mgmt.yml index a24b736d1f959..877b18934a555 100644 --- a/sdk/elasticsan/tests.mgmt.yml +++ b/sdk/elasticsan/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: elasticsan - Artifacts: - - name: azure-resourcemanager-elasticsan - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerelasticsan - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: elasticsan + Artifacts: + - name: azure-resourcemanager-elasticsan + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerelasticsan + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/pom.xml b/sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/pom.xml index 30da011e6384c..70ffacc44cac9 100644 --- a/sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/pom.xml +++ b/sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/pom.xml @@ -88,7 +88,7 @@ com.azure azure-messaging-eventgrid - 4.21.0 + 4.23.0-beta.1 io.cloudevents diff --git a/sdk/eventgrid/azure-messaging-eventgrid/CHANGELOG.md b/sdk/eventgrid/azure-messaging-eventgrid/CHANGELOG.md index ad545476221db..690b9dd40f7b5 100644 --- a/sdk/eventgrid/azure-messaging-eventgrid/CHANGELOG.md +++ b/sdk/eventgrid/azure-messaging-eventgrid/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 4.23.0-beta.1 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 4.22.0 (2024-03-11) ### Features Added diff --git a/sdk/eventgrid/azure-messaging-eventgrid/pom.xml b/sdk/eventgrid/azure-messaging-eventgrid/pom.xml index fcfb726735ebe..ab7f76ce104be 100644 --- a/sdk/eventgrid/azure-messaging-eventgrid/pom.xml +++ b/sdk/eventgrid/azure-messaging-eventgrid/pom.xml @@ -12,7 +12,7 @@ com.azure azure-messaging-eventgrid - 4.22.0 + 4.23.0-beta.1 jar Microsoft Azure SDK for eventgrid @@ -95,13 +95,13 @@ com.azure azure-storage-queue - 12.20.2 + 12.20.3 test com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -131,7 +131,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/eventgrid/azure-resourcemanager-eventgrid/ci.yml b/sdk/eventgrid/azure-resourcemanager-eventgrid/ci.yml new file mode 100644 index 0000000000000..956bd97cd6ab2 --- /dev/null +++ b/sdk/eventgrid/azure-resourcemanager-eventgrid/ci.yml @@ -0,0 +1,44 @@ +# NOTE: Please refer to https://aka.ms/azsdk/engsys/ci-yaml before editing this file. + +trigger: + branches: + include: + - main + - hotfix/* + - release/* + paths: + include: + - sdk/eventgrid/azure-resourcemanager-eventgrid/ + exclude: + - sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml + +pr: + branches: + include: + - main + - feature/* + - hotfix/* + - release/* + paths: + include: + - sdk/eventgrid/azure-resourcemanager-eventgrid/ + exclude: + - sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml + +parameters: +- name: release_azureresourcemanagereventgrid + displayName: 'azure-resourcemanager-eventgrid' + type: boolean + default: true + +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-client.yml + parameters: + ServiceDirectory: eventgrid + JavadocSafeJavaBuildVersion: '1.21' + EnableBatchRelease: true + Artifacts: + - name: azure-resourcemanager-eventgrid + groupId: com.azure.resourcemanager + safeName: azureresourcemanagereventgrid + releaseInBatch: ${{ parameters.release_azureresourcemanagereventgrid }} diff --git a/sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml b/sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml index 0a00b15ad44b4..703e2eb705683 100644 --- a/sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml +++ b/sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,19 +111,19 @@ com.azure.resourcemanager azure-resourcemanager - 2.36.0 + 2.37.0 test com.azure azure-messaging-eventgrid - 4.21.0 + 4.22.0 test com.azure azure-messaging-eventhubs - 5.18.1 + 5.18.2 test diff --git a/sdk/eventgrid/ci.yml b/sdk/eventgrid/ci.yml index e2fcdea366237..792ea80340139 100644 --- a/sdk/eventgrid/ci.yml +++ b/sdk/eventgrid/ci.yml @@ -11,12 +11,11 @@ trigger: - sdk/eventgrid/ci.yml - sdk/eventgrid/azure-messaging-eventgrid/ - sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/ - - sdk/eventgrid/azure-resourcemanager-eventgrid/ exclude: - sdk/eventgrid/pom.xml - sdk/eventgrid/azure-messaging-eventgrid/pom.xml - sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/pom.xml - - sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml + - sdk/eventgrid/azure-resourcemanager-eventgrid/ pr: branches: @@ -30,12 +29,11 @@ pr: - sdk/eventgrid/ci.yml - sdk/eventgrid/azure-messaging-eventgrid/ - sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/ - - sdk/eventgrid/azure-resourcemanager-eventgrid/ exclude: - sdk/eventgrid/pom.xml - sdk/eventgrid/azure-messaging-eventgrid/pom.xml - sdk/eventgrid/azure-messaging-eventgrid-cloudnative-cloudevents/pom.xml - - sdk/eventgrid/azure-resourcemanager-eventgrid/pom.xml + - sdk/eventgrid/azure-resourcemanager-eventgrid/ parameters: - name: release_azuremessagingeventgrid @@ -46,10 +44,6 @@ parameters: displayName: 'azure-messaging-eventgrid-cloudnative-cloudevents' type: boolean default: true -- name: release_azureresourcemanagereventgrid - displayName: 'azure-resourcemanager-eventgrid' - type: boolean - default: false extends: template: ../../eng/pipelines/templates/stages/archetype-sdk-client.yml @@ -66,8 +60,3 @@ extends: groupId: com.azure safeName: azuremessagingeventgridcloudnativecloudevents releaseInBatch: ${{ parameters.release_azuremessagingeventgridcloudnativecloudevents }} - - name: azure-resourcemanager-eventgrid - groupId: com.azure.resourcemanager - safeName: azureresourcemanagereventgrid - releaseInBatch: ${{ parameters.release_azureresourcemanagereventgrid }} - diff --git a/sdk/eventgrid/tests.yml b/sdk/eventgrid/tests.yml index 1647b3a0a9340..b01374d9d57bc 100644 --- a/sdk/eventgrid/tests.yml +++ b/sdk/eventgrid/tests.yml @@ -1,15 +1,15 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: eventgrid - TimeoutInMinutes: 60 - MaxParallel: 7 - Artifacts: - - name: azure-messaging-eventgrid - groupId: com.azure - safeName: azuremessagingeventgrid - - name: azure-messaging-eventgrid-cloudnative-cloudevents - groupId: com.azure - safeName: azuremessagingeventgridcloudnativecloudevents +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: eventgrid + TimeoutInMinutes: 60 + MaxParallel: 7 + Artifacts: + - name: azure-messaging-eventgrid + groupId: com.azure + safeName: azuremessagingeventgrid + - name: azure-messaging-eventgrid-cloudnative-cloudevents + groupId: com.azure + safeName: azuremessagingeventgridcloudnativecloudevents diff --git a/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/CHANGELOG.md b/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/CHANGELOG.md index 438f93039221d..aa119b7f7ab23 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/CHANGELOG.md +++ b/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.19.2 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-messaging-eventhubs` from `5.18.1` to version `5.18.2`. +- Upgraded `azure-storage-blob` from `12.25.1` to version `12.25.2`. + + ## 1.19.1 (2024-02-16) ### Other Changes diff --git a/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/pom.xml b/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/pom.xml index e93edcbd22dc0..9bd3d2ef99713 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/pom.xml +++ b/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-blob/pom.xml @@ -54,7 +54,7 @@ com.azure azure-storage-blob - 12.25.2 + 12.25.3 @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,7 +91,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -105,13 +105,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-jedis/pom.xml b/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-jedis/pom.xml index bbc9a2a4287df..058f7be838a66 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-jedis/pom.xml +++ b/sdk/eventhubs/azure-messaging-eventhubs-checkpointstore-jedis/pom.xml @@ -53,7 +53,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -77,7 +77,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -97,13 +97,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/eventhubs/azure-messaging-eventhubs-stress/pom.xml b/sdk/eventhubs/azure-messaging-eventhubs-stress/pom.xml index 2d3f85884370a..c922dc449c2ff 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs-stress/pom.xml +++ b/sdk/eventhubs/azure-messaging-eventhubs-stress/pom.xml @@ -50,17 +50,17 @@ com.azure azure-monitor-opentelemetry-exporter - 1.0.0-beta.20 + 1.0.0-beta.21 io.opentelemetry.instrumentation opentelemetry-runtime-telemetry-java8 - 2.1.0-alpha + 2.2.0-alpha io.opentelemetry.instrumentation opentelemetry-logback-appender-1.0 - 2.1.0-alpha + 2.2.0-alpha com.azure @@ -97,8 +97,8 @@ org.springframework.boot:spring-boot-starter:[2.7.18] - io.opentelemetry.instrumentation:opentelemetry-runtime-telemetry-java8:[2.1.0-alpha] - io.opentelemetry.instrumentation:opentelemetry-logback-appender-1.0:[2.1.0-alpha] + io.opentelemetry.instrumentation:opentelemetry-runtime-telemetry-java8:[2.2.0-alpha] + io.opentelemetry.instrumentation:opentelemetry-logback-appender-1.0:[2.2.0-alpha] diff --git a/sdk/eventhubs/azure-messaging-eventhubs-track1-perf/pom.xml b/sdk/eventhubs/azure-messaging-eventhubs-track1-perf/pom.xml index f314107e71544..11161f71f8276 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs-track1-perf/pom.xml +++ b/sdk/eventhubs/azure-messaging-eventhubs-track1-perf/pom.xml @@ -23,7 +23,7 @@ com.microsoft.azure azure-eventhubs-eph - 3.3.0 + 3.3.0 com.azure diff --git a/sdk/eventhubs/azure-messaging-eventhubs/CHANGELOG.md b/sdk/eventhubs/azure-messaging-eventhubs/CHANGELOG.md index c381a53d5f99c..e9e60f829713e 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs/CHANGELOG.md +++ b/sdk/eventhubs/azure-messaging-eventhubs/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 5.18.2 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-amqp` from `2.9.1` to version `2.9.2`. + + ## 5.18.1 (2024-02-16) ### Bugs Fixed diff --git a/sdk/eventhubs/azure-messaging-eventhubs/docs/pom.xml b/sdk/eventhubs/azure-messaging-eventhubs/docs/pom.xml index 7ec9348d86dc9..9c5b411b17fcb 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs/docs/pom.xml +++ b/sdk/eventhubs/azure-messaging-eventhubs/docs/pom.xml @@ -25,7 +25,7 @@ io.projectreactor reactor-core - 3.4.34 + 3.4.36 @@ -55,7 +55,7 @@ ch.qos.logback logback-classic - 1.3.12 + 1.3.14 org.codehaus.janino diff --git a/sdk/eventhubs/azure-messaging-eventhubs/pom.xml b/sdk/eventhubs/azure-messaging-eventhubs/pom.xml index a26721217e224..773c7add0b35b 100644 --- a/sdk/eventhubs/azure-messaging-eventhubs/pom.xml +++ b/sdk/eventhubs/azure-messaging-eventhubs/pom.xml @@ -56,7 +56,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,7 +87,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -101,13 +101,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -121,21 +121,21 @@ io.opentelemetry opentelemetry-api - 1.35.0 + 1.36.0 test io.opentelemetry opentelemetry-sdk - 1.35.0 + 1.36.0 test io.opentelemetry opentelemetry-exporter-logging - 1.35.0 + 1.36.0 test diff --git a/sdk/eventhubs/ci.yml b/sdk/eventhubs/ci.yml index 1c954d7debecc..168c1327f6704 100644 --- a/sdk/eventhubs/ci.yml +++ b/sdk/eventhubs/ci.yml @@ -87,8 +87,6 @@ extends: safeName: azuremessagingeventhubscheckpointstorejedis releaseInBatch: ${{ parameters.release_azuremessagingeventhubscheckpointstorejedis }} AdditionalModules: - - name: azure-messaging-eventhubs-track1-perf - groupId: com.azure - name: azure-messaging-eventhubs-track2-perf groupId: com.azure # required by the above perf libraries diff --git a/sdk/eventhubs/microsoft-azure-eventhubs-eph/pom.xml b/sdk/eventhubs/microsoft-azure-eventhubs-eph/pom.xml index 2c11f2030fa75..d8eb8eaa3af8a 100644 --- a/sdk/eventhubs/microsoft-azure-eventhubs-eph/pom.xml +++ b/sdk/eventhubs/microsoft-azure-eventhubs-eph/pom.xml @@ -46,7 +46,7 @@ com.google.code.gson gson - 2.10 + 2.10.1 @@ -64,7 +64,7 @@ com.microsoft.azure msal4j - 1.14.0 + 1.14.3 test diff --git a/sdk/eventhubs/microsoft-azure-eventhubs-extensions/pom.xml b/sdk/eventhubs/microsoft-azure-eventhubs-extensions/pom.xml index 8e851f68adb3a..f32be6abb9cd9 100644 --- a/sdk/eventhubs/microsoft-azure-eventhubs-extensions/pom.xml +++ b/sdk/eventhubs/microsoft-azure-eventhubs-extensions/pom.xml @@ -40,7 +40,7 @@ com.microsoft.azure azure-eventhubs - 3.3.0 + 3.4.0-beta.1 org.apache.logging.log4j @@ -68,7 +68,7 @@ com.microsoft.azure msal4j - 1.14.0 + 1.14.3 test diff --git a/sdk/eventhubs/microsoft-azure-eventhubs/pom.xml b/sdk/eventhubs/microsoft-azure-eventhubs/pom.xml index e0bed0cfbd573..9b0793bdb63d1 100644 --- a/sdk/eventhubs/microsoft-azure-eventhubs/pom.xml +++ b/sdk/eventhubs/microsoft-azure-eventhubs/pom.xml @@ -77,7 +77,7 @@ com.microsoft.azure msal4j - 1.14.0 + 1.14.3 test @@ -97,13 +97,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/eventhubs/tests.yml b/sdk/eventhubs/tests.yml index 528ddc654094e..e00a46dc1d6cb 100644 --- a/sdk/eventhubs/tests.yml +++ b/sdk/eventhubs/tests.yml @@ -1,25 +1,25 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: eventhubs - Artifacts: - - name: azure-messaging-eventhubs - groupId: com.azure - safeName: azuremessagingeventhubs - - name: azure-messaging-eventhubs-checkpointstore-blob - groupId: com.azure - safeName: azuremessagingeventhubscheckpointstoreblob - - name: azure-messaging-eventhubs-checkpointstore-jedis - groupId: com.azure - safeName: azuremessagingeventhubscheckpointstorejedis - TimeoutInMinutes: 120 - SupportedClouds: 'Public,UsGov,China' - EnvVars: - AZURE_LOG_LEVEL: 1 - AdditionalMatrixConfigs: - - Name: version_overrides_tests - Path: sdk/eventhubs/version-overrides-matrix.json - Selection: all - GenerateVMJobs: true +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: eventhubs + Artifacts: + - name: azure-messaging-eventhubs + groupId: com.azure + safeName: azuremessagingeventhubs + - name: azure-messaging-eventhubs-checkpointstore-blob + groupId: com.azure + safeName: azuremessagingeventhubscheckpointstoreblob + - name: azure-messaging-eventhubs-checkpointstore-jedis + groupId: com.azure + safeName: azuremessagingeventhubscheckpointstorejedis + TimeoutInMinutes: 120 + SupportedClouds: 'Public,UsGov,China' + EnvVars: + AZURE_LOG_LEVEL: 1 + AdditionalMatrixConfigs: + - Name: version_overrides_tests + Path: sdk/eventhubs/version-overrides-matrix.json + Selection: all + GenerateVMJobs: true diff --git a/sdk/eventhubs/version-overrides-matrix.json b/sdk/eventhubs/version-overrides-matrix.json index 8ca1fe9995b60..e596b93e7b0a8 100644 --- a/sdk/eventhubs/version-overrides-matrix.json +++ b/sdk/eventhubs/version-overrides-matrix.json @@ -1,7 +1,7 @@ { "matrix": { "Agent": { - "ubuntu-20.04": { "OSVmImage": "MMSUbuntu20.04", "Pool": "azsdk-pool-mms-ubuntu-2004-general" } + "ubuntu-20.04": { "OSVmImage": "env:LINUXVMIMAGE", "Pool": "env:LINUXPOOL" } }, "JavaTestVersion": "1.21", "TestGoals": "surefire:test", diff --git a/sdk/fluidrelay/azure-resourcemanager-fluidrelay/pom.xml b/sdk/fluidrelay/azure-resourcemanager-fluidrelay/pom.xml index d868f2f12e224..f64245399d4da 100644 --- a/sdk/fluidrelay/azure-resourcemanager-fluidrelay/pom.xml +++ b/sdk/fluidrelay/azure-resourcemanager-fluidrelay/pom.xml @@ -54,7 +54,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/fluidrelay/tests.mgmt.yml b/sdk/fluidrelay/tests.mgmt.yml index a422277f5c4b5..f1bc3251f9ec6 100644 --- a/sdk/fluidrelay/tests.mgmt.yml +++ b/sdk/fluidrelay/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: fluidrelay - Artifacts: - - name: azure-resourcemanager-fluidrelay - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerfluidrelay - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: fluidrelay + Artifacts: + - name: azure-resourcemanager-fluidrelay + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerfluidrelay + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/formrecognizer/azure-ai-formrecognizer-perf/pom.xml b/sdk/formrecognizer/azure-ai-formrecognizer-perf/pom.xml index e689d96c6f3f4..0a3d365a3577d 100644 --- a/sdk/formrecognizer/azure-ai-formrecognizer-perf/pom.xml +++ b/sdk/formrecognizer/azure-ai-formrecognizer-perf/pom.xml @@ -105,7 +105,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 diff --git a/sdk/formrecognizer/azure-ai-formrecognizer/CHANGELOG.md b/sdk/formrecognizer/azure-ai-formrecognizer/CHANGELOG.md index 74f25366a472c..9d882cd0788ad 100644 --- a/sdk/formrecognizer/azure-ai-formrecognizer/CHANGELOG.md +++ b/sdk/formrecognizer/azure-ai-formrecognizer/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 4.1.6 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 4.1.5 (2024-02-20) ### Other Changes diff --git a/sdk/formrecognizer/azure-ai-formrecognizer/pom.xml b/sdk/formrecognizer/azure-ai-formrecognizer/pom.xml index 4842077655afc..6f2ed9a93c2e7 100644 --- a/sdk/formrecognizer/azure-ai-formrecognizer/pom.xml +++ b/sdk/formrecognizer/azure-ai-formrecognizer/pom.xml @@ -94,7 +94,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/formrecognizer/ci.yml b/sdk/formrecognizer/ci.yml index 92a0b6893be23..1bd887d1c3be5 100644 --- a/sdk/formrecognizer/ci.yml +++ b/sdk/formrecognizer/ci.yml @@ -48,6 +48,8 @@ extends: AdditionalModules: - name: azure-ai-formrecognizer-perf groupId: com.azure - # required by the above perf library + # both of these are required by the above perf library - name: perf-test-core groupId: com.azure + - name: azure-aot-graalvm-perf + groupId: com.azure diff --git a/sdk/formrecognizer/tests.native.yml b/sdk/formrecognizer/tests.native.yml index 8ac9a4e9b862f..b8b66d4bc304b 100644 --- a/sdk/formrecognizer/tests.native.yml +++ b/sdk/formrecognizer/tests.native.yml @@ -1,16 +1,16 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-native-tests.yml - parameters: - ServiceDirectory: formrecognizer - timeoutInMinutes: 150 # how long to run the job before automatically cancelling - Artifacts: - - name: azure-ai-formrecognizer - groupId: com.azure - safeName: azureaiformrecognizer - CloudConfig: - Public: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - Location: 'eastus' - SupportedClouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-native-tests.yml + parameters: + ServiceDirectory: formrecognizer + timeoutInMinutes: 150 # how long to run the job before automatically cancelling + Artifacts: + - name: azure-ai-formrecognizer + groupId: com.azure + safeName: azureaiformrecognizer + CloudConfig: + Public: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + Location: 'eastus' + SupportedClouds: 'Public' diff --git a/sdk/formrecognizer/tests.yml b/sdk/formrecognizer/tests.yml index 2c37988b803b6..ac79c1dbee0b8 100644 --- a/sdk/formrecognizer/tests.yml +++ b/sdk/formrecognizer/tests.yml @@ -1,25 +1,25 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: formrecognizer - timeoutInMinutes: 150 # how long to run the job before automatically cancelling - Artifacts: - - name: azure-ai-formrecognizer - groupId: com.azure - safeName: azureaiformrecognizer - CloudConfig: - Public: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - Location: 'centraluseuap' - Canary: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - Location: 'centraluseuap' - UsGov: - SubscriptionConfiguration: $(sub-config-gov-test-resources) - Location: 'usgovvirginia' - China: - SubscriptionConfiguration: $(sub-config-cn-test-resources) - Location: 'chinaeast2' - SupportedClouds: 'Public,Canary,UsGov,China' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: formrecognizer + timeoutInMinutes: 150 # how long to run the job before automatically cancelling + Artifacts: + - name: azure-ai-formrecognizer + groupId: com.azure + safeName: azureaiformrecognizer + CloudConfig: + Public: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + Location: 'centraluseuap' + Canary: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + Location: 'centraluseuap' + UsGov: + SubscriptionConfiguration: $(sub-config-gov-test-resources) + Location: 'usgovvirginia' + China: + SubscriptionConfiguration: $(sub-config-cn-test-resources) + Location: 'chinaeast2' + SupportedClouds: 'Public,Canary,UsGov,China' diff --git a/sdk/frontdoor/azure-resourcemanager-frontdoor/pom.xml b/sdk/frontdoor/azure-resourcemanager-frontdoor/pom.xml index dca884a1500e1..ed3cf1bacc018 100644 --- a/sdk/frontdoor/azure-resourcemanager-frontdoor/pom.xml +++ b/sdk/frontdoor/azure-resourcemanager-frontdoor/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 test diff --git a/sdk/graphservices/azure-resourcemanager-graphservices/pom.xml b/sdk/graphservices/azure-resourcemanager-graphservices/pom.xml index dcfa3c828f0e3..e33437e1d56cb 100644 --- a/sdk/graphservices/azure-resourcemanager-graphservices/pom.xml +++ b/sdk/graphservices/azure-resourcemanager-graphservices/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hanaonazure/azure-resourcemanager-hanaonazure/pom.xml b/sdk/hanaonazure/azure-resourcemanager-hanaonazure/pom.xml index fd1c9e5410508..1be60c9eec78e 100644 --- a/sdk/hanaonazure/azure-resourcemanager-hanaonazure/pom.xml +++ b/sdk/hanaonazure/azure-resourcemanager-hanaonazure/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hardwaresecuritymodules/azure-resourcemanager-hardwaresecuritymodules/pom.xml b/sdk/hardwaresecuritymodules/azure-resourcemanager-hardwaresecuritymodules/pom.xml index 1af81a00ab0db..cb0a5dd85baaa 100644 --- a/sdk/hardwaresecuritymodules/azure-resourcemanager-hardwaresecuritymodules/pom.xml +++ b/sdk/hardwaresecuritymodules/azure-resourcemanager-hardwaresecuritymodules/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hdinsight/azure-resourcemanager-hdinsight-containers/pom.xml b/sdk/hdinsight/azure-resourcemanager-hdinsight-containers/pom.xml index c275b1b11a2af..b68f6fb267779 100644 --- a/sdk/hdinsight/azure-resourcemanager-hdinsight-containers/pom.xml +++ b/sdk/hdinsight/azure-resourcemanager-hdinsight-containers/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hdinsight/azure-resourcemanager-hdinsight/pom.xml b/sdk/hdinsight/azure-resourcemanager-hdinsight/pom.xml index 14e644f4a33bb..28fa8565e7d9f 100644 --- a/sdk/hdinsight/azure-resourcemanager-hdinsight/pom.xml +++ b/sdk/hdinsight/azure-resourcemanager-hdinsight/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,19 +111,19 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 test com.azure.resourcemanager azure-resourcemanager-network - 2.36.0 + 2.37.0 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/hdinsight/tests.mgmt.yml b/sdk/hdinsight/tests.mgmt.yml index 8c58bb1a0103b..e06a285949fad 100644 --- a/sdk/hdinsight/tests.mgmt.yml +++ b/sdk/hdinsight/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: hdinsight - Artifacts: - - name: azure-resourcemanager-hdinsight - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerhdinsight - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: hdinsight + Artifacts: + - name: azure-resourcemanager-hdinsight + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerhdinsight + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/healthcareapis/azure-resourcemanager-healthcareapis/pom.xml b/sdk/healthcareapis/azure-resourcemanager-healthcareapis/pom.xml index c078e21cca6e1..3827660a752a2 100644 --- a/sdk/healthcareapis/azure-resourcemanager-healthcareapis/pom.xml +++ b/sdk/healthcareapis/azure-resourcemanager-healthcareapis/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/healthcareapis/tests.mgmt.yml b/sdk/healthcareapis/tests.mgmt.yml index 8352ee8f96b05..a325c0ffe0a7f 100644 --- a/sdk/healthcareapis/tests.mgmt.yml +++ b/sdk/healthcareapis/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: healthcareapis - Artifacts: - - name: azure-resourcemanager-healthcareapis - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerhealthcareapis - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: healthcareapis + Artifacts: + - name: azure-resourcemanager-healthcareapis + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerhealthcareapis + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/healthinsights/azure-health-insights-cancerprofiling/pom.xml b/sdk/healthinsights/azure-health-insights-cancerprofiling/pom.xml index c74e82f8042e5..700b4c1ba076b 100644 --- a/sdk/healthinsights/azure-health-insights-cancerprofiling/pom.xml +++ b/sdk/healthinsights/azure-health-insights-cancerprofiling/pom.xml @@ -80,7 +80,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/healthinsights/azure-health-insights-clinicalmatching/pom.xml b/sdk/healthinsights/azure-health-insights-clinicalmatching/pom.xml index 5381abf324b4d..740f7033ea8d1 100644 --- a/sdk/healthinsights/azure-health-insights-clinicalmatching/pom.xml +++ b/sdk/healthinsights/azure-health-insights-clinicalmatching/pom.xml @@ -80,7 +80,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/healthinsights/azure-health-insights-radiologyinsights/pom.xml b/sdk/healthinsights/azure-health-insights-radiologyinsights/pom.xml index e437ad1376232..aed4f7d53d5af 100644 --- a/sdk/healthinsights/azure-health-insights-radiologyinsights/pom.xml +++ b/sdk/healthinsights/azure-health-insights-radiologyinsights/pom.xml @@ -84,7 +84,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/healthinsights/tests.yml b/sdk/healthinsights/tests.yml index 8bd83f1851743..299fac654cce5 100644 --- a/sdk/healthinsights/tests.yml +++ b/sdk/healthinsights/tests.yml @@ -4,18 +4,18 @@ parameters: type: string default: eastus -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: healthinsights - Artifacts: - - name: azure-health-insights-cancerprofiling - groupId: com.azure - safeName: azurehealthinsightscancerprofiling - - name: azure-health-insights-clinicalmatching - groupId: com.azure - safeName: azurehealthinsightsclinicalmatching - - name: azure-health-insights-radiologyinsights - groupId: com.azure - safeName: azurehealthinsightsradiologyinsights - Location: '${{ parameters.Location }}' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: healthinsights + Artifacts: + - name: azure-health-insights-cancerprofiling + groupId: com.azure + safeName: azurehealthinsightscancerprofiling + - name: azure-health-insights-clinicalmatching + groupId: com.azure + safeName: azurehealthinsightsclinicalmatching + - name: azure-health-insights-radiologyinsights + groupId: com.azure + safeName: azurehealthinsightsradiologyinsights + Location: '${{ parameters.Location }}' diff --git a/sdk/hybridcompute/azure-resourcemanager-hybridcompute/pom.xml b/sdk/hybridcompute/azure-resourcemanager-hybridcompute/pom.xml index c2ca5c48acde4..6f57d51d3b4c5 100644 --- a/sdk/hybridcompute/azure-resourcemanager-hybridcompute/pom.xml +++ b/sdk/hybridcompute/azure-resourcemanager-hybridcompute/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hybridconnectivity/azure-resourcemanager-hybridconnectivity/pom.xml b/sdk/hybridconnectivity/azure-resourcemanager-hybridconnectivity/pom.xml index 01b06a58acdb5..6cc5f17939f4e 100644 --- a/sdk/hybridconnectivity/azure-resourcemanager-hybridconnectivity/pom.xml +++ b/sdk/hybridconnectivity/azure-resourcemanager-hybridconnectivity/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hybridcontainerservice/azure-resourcemanager-hybridcontainerservice/pom.xml b/sdk/hybridcontainerservice/azure-resourcemanager-hybridcontainerservice/pom.xml index abb9b5b9faeec..94296b417e6f0 100644 --- a/sdk/hybridcontainerservice/azure-resourcemanager-hybridcontainerservice/pom.xml +++ b/sdk/hybridcontainerservice/azure-resourcemanager-hybridcontainerservice/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -90,13 +90,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hybridkubernetes/azure-resourcemanager-hybridkubernetes/pom.xml b/sdk/hybridkubernetes/azure-resourcemanager-hybridkubernetes/pom.xml index cd4c3ac7bca6f..e888672130612 100644 --- a/sdk/hybridkubernetes/azure-resourcemanager-hybridkubernetes/pom.xml +++ b/sdk/hybridkubernetes/azure-resourcemanager-hybridkubernetes/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/hybridnetwork/azure-resourcemanager-hybridnetwork/pom.xml b/sdk/hybridnetwork/azure-resourcemanager-hybridnetwork/pom.xml index 48252fabfc7fa..c7c5f0e21ffa8 100644 --- a/sdk/hybridnetwork/azure-resourcemanager-hybridnetwork/pom.xml +++ b/sdk/hybridnetwork/azure-resourcemanager-hybridnetwork/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/identity/azure-identity-broker-samples/src/samples/java/com/azure/identity/broker/JavaDocCodeSnippets.java b/sdk/identity/azure-identity-broker-samples/src/samples/java/com/azure/identity/broker/JavaDocCodeSnippets.java index de949729d7450..b08c5b654cf4f 100644 --- a/sdk/identity/azure-identity-broker-samples/src/samples/java/com/azure/identity/broker/JavaDocCodeSnippets.java +++ b/sdk/identity/azure-identity-broker-samples/src/samples/java/com/azure/identity/broker/JavaDocCodeSnippets.java @@ -40,6 +40,14 @@ public void configureCredentialForWindows() { // END: com.azure.identity.broker.interactivebrowserbrokercredentialbuilder.useinteractivebrowserbroker.windows } + public void configureCredentialForDefaultAccount() { + // BEGIN: com.azure.identity.broker.interactivebrowserbrokercredentialbuilder.useinteractivebrowserbroker.defaultaccount + InteractiveBrowserCredential cred = new InteractiveBrowserBrokerCredentialBuilder() + .useDefaultBrokerAccount() + .build(); + // END: com.azure.identity.broker.interactivebrowserbrokercredentialbuilder.useinteractivebrowserbroker.defaultaccount + } + private long getWindowHandle() { return 0; } diff --git a/sdk/identity/azure-identity-broker/CHANGELOG.md b/sdk/identity/azure-identity-broker/CHANGELOG.md index 34814f86063b6..4ad9cba6b8c05 100644 --- a/sdk/identity/azure-identity-broker/CHANGELOG.md +++ b/sdk/identity/azure-identity-broker/CHANGELOG.md @@ -4,12 +4,23 @@ ### Features Added +- Added support for using the default broker account + ### Breaking Changes ### Bugs Fixed ### Other Changes +## 1.0.4 (2024-03-14) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-identity` from `1.11.3` to version `1.11.4`. +- Upgraded `msal4j` from `1.14.0` to version `1.14.3`. + ## 1.0.2 (2024-02-05) ### Other Changes diff --git a/sdk/identity/azure-identity-broker/README.md b/sdk/identity/azure-identity-broker/README.md index 11f496207516e..f08a3d3c81279 100644 --- a/sdk/identity/azure-identity-broker/README.md +++ b/sdk/identity/azure-identity-broker/README.md @@ -89,6 +89,16 @@ InteractiveBrowserCredential cred = new InteractiveBrowserBrokerCredentialBuilde .build(); ``` +#### Use the default account for sign-in + +When this option is enabled, the credential will attempt to silently use the default broker account. If using the default account fails, the credential will fall back to interactive authentication. + +```java com.azure.identity.broker.interactivebrowserbrokercredentialbuilder.useinteractivebrowserbroker.defaultaccount +InteractiveBrowserCredential cred = new InteractiveBrowserBrokerCredentialBuilder() + .useDefaultBrokerAccount() + .build(); +``` + #### Obtain a window handle ##### JavaFX diff --git a/sdk/identity/azure-identity-broker/pom.xml b/sdk/identity/azure-identity-broker/pom.xml index 667101f5c1461..ab7d7d239af90 100644 --- a/sdk/identity/azure-identity-broker/pom.xml +++ b/sdk/identity/azure-identity-broker/pom.xml @@ -39,7 +39,7 @@ com.microsoft.azure msal4j - 1.14.0 + 1.14.3 com.microsoft.azure @@ -64,7 +64,7 @@ - com.microsoft.azure:msal4j:[1.14.0] + com.microsoft.azure:msal4j:[1.14.3] com.microsoft.azure:msal4j-brokers:[1.0.0] diff --git a/sdk/identity/azure-identity-broker/src/main/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilder.java b/sdk/identity/azure-identity-broker/src/main/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilder.java index a1bc9c1f9f0db..a5c395f444c5f 100644 --- a/sdk/identity/azure-identity-broker/src/main/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilder.java +++ b/sdk/identity/azure-identity-broker/src/main/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilder.java @@ -62,15 +62,13 @@ public InteractiveBrowserBrokerCredentialBuilder enableLegacyMsaPassthrough() { } /** - * Enables automatically using the signed-in user's account for brokered authentication instead of - * of prompting the user with a login dialog. + * Enables automatically using the default broker account for authentication instead + * of prompting the user with an account picker. * - * @param useOperatingSystemAccount Boolean value to determine if the operating system account should be used. - * @return An updated instance of this builder with useOperatingSystemAccount set. + * @return An updated instance of this builder with useDefaultBrokerAccount set. */ - public InteractiveBrowserCredentialBuilder useOperatingSystemAccount(boolean useOperatingSystemAccount) { - CredentialBuilderBaseHelper.getClientOptions(this). - setUseOperatingSystemAccount(useOperatingSystemAccount); + public InteractiveBrowserCredentialBuilder useDefaultBrokerAccount() { + CredentialBuilderBaseHelper.getClientOptions(this).setUseDefaultBrokerAccount(true); return this; } diff --git a/sdk/identity/azure-identity-broker/src/test/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilderTest.java b/sdk/identity/azure-identity-broker/src/test/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilderTest.java index 070622c138d50..d6fd468b3fb3b 100644 --- a/sdk/identity/azure-identity-broker/src/test/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilderTest.java +++ b/sdk/identity/azure-identity-broker/src/test/java/com/azure/identity/broker/InteractiveBrowserBrokerCredentialBuilderTest.java @@ -242,10 +242,10 @@ void proxyOptions() { } @Test - void setUseOperatingSystemAccount() { + void setDefaultBrokerAccount() { assertDoesNotThrow(() -> { InteractiveBrowserBrokerCredentialBuilder builder = new InteractiveBrowserBrokerCredentialBuilder(); - builder.useOperatingSystemAccount(true); + builder.useDefaultBrokerAccount(); builder.build(); }); } diff --git a/sdk/identity/azure-identity-extensions/CHANGELOG.md b/sdk/identity/azure-identity-extensions/CHANGELOG.md index 36af09f62651c..be418f400d98d 100644 --- a/sdk/identity/azure-identity-extensions/CHANGELOG.md +++ b/sdk/identity/azure-identity-extensions/CHANGELOG.md @@ -11,6 +11,25 @@ ### Other Changes +## 1.1.14 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-identity` from `1.11.2` to version `1.11.4`. +- Upgraded `postgresql` from `42.3.8` to version `42.3.9`. + + +## 1.1.13 (2024-02-22) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-identity` from `1.11.1` to version `1.11.2`. + + ## 1.1.12 (2024-02-05) ### Other Changes diff --git a/sdk/identity/azure-identity-extensions/ci.yml b/sdk/identity/azure-identity-extensions/ci.yml new file mode 100644 index 0000000000000..48954f23fcf84 --- /dev/null +++ b/sdk/identity/azure-identity-extensions/ci.yml @@ -0,0 +1,43 @@ +# NOTE: Please refer to https://aka.ms/azsdk/engsys/ci-yaml before editing this file. + +trigger: + branches: + include: + - main + - hotfix/* + - release/* + paths: + include: + - sdk/identity/azure-identity-extensions/ + exclude: + - sdk/identity/azure-identity-extensions/pom.xml + +pr: + branches: + include: + - main + - feature/* + - hotfix/* + - release/* + paths: + include: + - sdk/identity/azure-identity-extensions/ + exclude: + - sdk/identity/azure-identity-extensions/pom.xml + +parameters: +- name: release_azureidentityextensions + displayName: 'azure-identity-extensions' + type: boolean + default: true + +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-client.yml + parameters: + ServiceDirectory: identity + EnableBatchRelease: true + Artifacts: + - name: azure-identity-extensions + groupId: com.azure + safeName: azureidentityextensions + releaseInBatch: ${{ parameters.release_azureidentityextensions }} diff --git a/sdk/identity/azure-identity-extensions/pom.xml b/sdk/identity/azure-identity-extensions/pom.xml index c099f5b314d77..2a1c9e164c782 100644 --- a/sdk/identity/azure-identity-extensions/pom.xml +++ b/sdk/identity/azure-identity-extensions/pom.xml @@ -34,7 +34,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 @@ -56,13 +56,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/identity/azure-identity/CHANGELOG.md b/sdk/identity/azure-identity/CHANGELOG.md index 91472f6087c9d..a35606f5ac928 100644 --- a/sdk/identity/azure-identity/CHANGELOG.md +++ b/sdk/identity/azure-identity/CHANGELOG.md @@ -10,6 +10,13 @@ ### Other Changes +## 1.11.4 (2024-03-14) + +### Other Changes + +#### Dependency Updates +- Upgraded `msal4j` from `1.14.0` to version `1.14.3`. + ## 1.11.3 (2024-03-01) ### Bugs fixed diff --git a/sdk/identity/azure-identity/pom.xml b/sdk/identity/azure-identity/pom.xml index 8406340284c20..95a37d9dacc2c 100644 --- a/sdk/identity/azure-identity/pom.xml +++ b/sdk/identity/azure-identity/pom.xml @@ -48,7 +48,7 @@ com.microsoft.azure msal4j - 1.14.0 + 1.14.3 com.microsoft.azure @@ -87,14 +87,14 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test com.google.code.gson gson - 2.10 + 2.10.1 test @@ -106,13 +106,13 @@ io.lettuce lettuce-core - 6.3.1.RELEASE + 6.3.2.RELEASE test org.redisson redisson - 3.27.0 + 3.27.2 test @@ -132,13 +132,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -153,7 +153,7 @@ - com.microsoft.azure:msal4j:[1.14.0] + com.microsoft.azure:msal4j:[1.14.3] com.microsoft.azure:msal4j-persistence-extension:[1.2.0] net.java.dev.jna:jna-platform:[5.6.0] org.linguafranca.pwdb:KeePassJava2:[2.1.4] diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClient.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClient.java index 920c7a4e740eb..1557d07d0d81c 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClient.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClient.java @@ -823,24 +823,36 @@ public Mono authenticateWithBrowserInteraction(TokenRequestContext re return Mono.error(LOGGER.logExceptionAsError(new RuntimeException(e))); } - if (options.isBrokerEnabled() && options.useOperatingSystemAccount()) { - return getPublicClientInstance(request).getValue().flatMap(pc -> - Mono.fromFuture(() -> - acquireTokenFromPublicClientSilently(request, pc, null, false)). - map(MsalToken::new)); - } else { - + // If the broker is enabled, try to get the token for the default account by passing + // a null account to MSAL. If that fails, show the dialog. + + return getPublicClientInstance(request).getValue().flatMap(pc -> { + if (options.isBrokerEnabled() && options.useDefaultBrokerAccount()) { + return Mono.fromFuture(() -> + acquireTokenFromPublicClientSilently(request, pc, null, false)) + // The error case here represents the silent acquisition failing. There's nothing actionable and + // in this case the fallback path of showing the dialog will capture any meaningful error and share it. + .onErrorResume(e -> Mono.empty()); + } else { + return Mono.empty(); + } + }) + .switchIfEmpty(Mono.defer(() -> { InteractiveRequestParameters.InteractiveRequestParametersBuilder builder = buildInteractiveRequestParameters(request, loginHint, redirectUri); SynchronizedAccessor publicClient = getPublicClientInstance(request); - Mono acquireToken = publicClient.getValue() + return publicClient.getValue() .flatMap(pc -> Mono.fromFuture(() -> pc.acquireToken(builder.build()))); - return acquireToken.onErrorMap(t -> new ClientAuthenticationException( - "Failed to acquire token with Interactive Browser Authentication.", null, t)).map(MsalToken::new); - } + })) + // If we're already throwing a ClientAuthenticationException we don't need to wrap it again. + .onErrorMap(t -> !(t instanceof ClientAuthenticationException), + t -> { + throw new ClientAuthenticationException("Failed to acquire token with Interactive Browser Authentication.", null, t); + }) + .map(MsalToken::new); } /** diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClientOptions.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClientOptions.java index 4350bb5e9ebe1..9087bf8e0687f 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClientOptions.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentityClientOptions.java @@ -79,7 +79,7 @@ public final class IdentityClientOptions implements Cloneable { private long brokerWindowHandle; private boolean brokerEnabled; private boolean enableMsaPassthrough; - private boolean useOperatingSystemAccount; + private boolean useDefaultBrokerAccount; /** * Creates an instance of IdentityClientOptions with default settings. @@ -784,11 +784,11 @@ public IdentityClientOptions setEnableLegacyMsaPassthrough(boolean enableMsaPass /** * Sets whether to use the logged-in user's account for broker authentication. - * @param useOperatingSystemAccount + * @param useDefaultBrokerAccount * @return the updated client options */ - public IdentityClientOptions setUseOperatingSystemAccount(boolean useOperatingSystemAccount) { - this.useOperatingSystemAccount = useOperatingSystemAccount; + public IdentityClientOptions setUseDefaultBrokerAccount(boolean useDefaultBrokerAccount) { + this.useDefaultBrokerAccount = useDefaultBrokerAccount; return this; } @@ -804,8 +804,8 @@ public boolean isMsaPassthroughEnabled() { * Gets the status whether to use the logged-in user's account for broker authentication. * @return the flag indicating if the logged-in user's account should be used for broker authentication. */ - public boolean useOperatingSystemAccount() { - return this.useOperatingSystemAccount; + public boolean useDefaultBrokerAccount() { + return this.useDefaultBrokerAccount; } public IdentityClientOptions clone() { diff --git a/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentitySyncClient.java b/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentitySyncClient.java index 6402042cc0349..37949583c002b 100644 --- a/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentitySyncClient.java +++ b/sdk/identity/azure-identity/src/main/java/com/azure/identity/implementation/IdentitySyncClient.java @@ -337,12 +337,21 @@ public MsalToken authenticateWithBrowserInteraction(TokenRequestContext request, } PublicClientApplication pc = getPublicClientInstance(request).getValue(); - if (options.isBrokerEnabled() && options.useOperatingSystemAccount()) { - return acquireTokenFromPublicClientSilently(request, - pc, - null, - false); - } else { + // If the broker is enabled, try to get the token for the default account by passing + // a null account to MSAL. If that fails, show the dialog. + MsalToken token = null; + if (options.isBrokerEnabled() && options.useDefaultBrokerAccount()) { + try { + token = acquireTokenFromPublicClientSilently(request, + pc, + null, + false); + } catch (Exception e) { + // The error case here represents the silent acquisition failing. There's nothing actionable and + // in this case the fallback path of showing the dialog will capture any meaningful error and share it. + } + } + if (token == null) { InteractiveRequestParameters.InteractiveRequestParametersBuilder builder = buildInteractiveRequestParameters(request, loginHint, redirectUri); @@ -353,6 +362,7 @@ public MsalToken authenticateWithBrowserInteraction(TokenRequestContext request, "Failed to acquire token with Interactive Browser Authentication.", null, e)); } } + return token; } /** diff --git a/sdk/identity/ci.yml b/sdk/identity/ci.yml index ac7f73684797c..f49b0103f6e0f 100644 --- a/sdk/identity/ci.yml +++ b/sdk/identity/ci.yml @@ -14,6 +14,7 @@ trigger: - sdk/identity/azure-identity/pom.xml - sdk/identity/azure-identity-perf/pom.xml - sdk/identity/azure-identity-broker/pom.xml + - sdk/identity/azure-identity-extensions/ pr: branches: @@ -30,6 +31,7 @@ pr: - sdk/identity/azure-identity/pom.xml - sdk/identity/azure-identity-perf/pom.xml - sdk/identity/azure-identity-broker/pom.xml + - sdk/identity/azure-identity-extensions/ parameters: - name: release_dependsonlivetests @@ -40,19 +42,16 @@ parameters: displayName: 'azure-identity' type: boolean default: true -- name: release_azureidentityextensions - displayName: 'azure-identity-extensions' - type: boolean - default: true - name: release_azureidentitybroker displayName: 'azure-identity-broker' type: boolean default: true extends: - template: ../../eng/pipelines/templates/stages/archetype-sdk-client.yml + template: /eng/pipelines/templates/stages/archetype-sdk-client.yml parameters: ServiceDirectory: identity + TimeoutInMinutes: 90 EnableBatchRelease: true ReleaseDependsOnLiveTests: ${{ parameters.release_dependsonlivetests }} Artifacts: @@ -60,10 +59,6 @@ extends: groupId: com.azure safeName: azureidentity releaseInBatch: ${{ parameters.release_azureidentity }} - - name: azure-identity-extensions - groupId: com.azure - safeName: azureidentityextensions - releaseInBatch: ${{ parameters.release_azureidentityextensions }} - name: azure-identity-broker groupId: com.azure safeName: azureidentitybroker @@ -75,8 +70,9 @@ extends: - name: perf-test-core groupId: com.azure LiveTestStages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + - template: /eng/pipelines/templates/stages/archetype-sdk-tests-isolated.yml parameters: + CalledFromClient: true ServiceDirectory: identity SupportedClouds: 'Public,UsGov,China' Artifacts: diff --git a/sdk/imagebuilder/azure-resourcemanager-imagebuilder/pom.xml b/sdk/imagebuilder/azure-resourcemanager-imagebuilder/pom.xml index fada6968328dd..9efac8fad751e 100644 --- a/sdk/imagebuilder/azure-resourcemanager-imagebuilder/pom.xml +++ b/sdk/imagebuilder/azure-resourcemanager-imagebuilder/pom.xml @@ -66,19 +66,19 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test com.azure.resourcemanager azure-resourcemanager-msi - 2.36.0 + 2.37.0 test com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -102,13 +102,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/imagebuilder/tests.mgmt.yml b/sdk/imagebuilder/tests.mgmt.yml index d9496b1a60573..1e835b3296fa0 100644 --- a/sdk/imagebuilder/tests.mgmt.yml +++ b/sdk/imagebuilder/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: imagebuilder - Artifacts: - - name: azure-resourcemanager-imagebuilder - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerimagebuilder - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: imagebuilder + Artifacts: + - name: azure-resourcemanager-imagebuilder + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerimagebuilder + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/iotcentral/azure-resourcemanager-iotcentral/pom.xml b/sdk/iotcentral/azure-resourcemanager-iotcentral/pom.xml index 1aa6f6a82a172..e22b8390bc583 100644 --- a/sdk/iotcentral/azure-resourcemanager-iotcentral/pom.xml +++ b/sdk/iotcentral/azure-resourcemanager-iotcentral/pom.xml @@ -54,7 +54,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/iotcentral/tests.mgmt.yml b/sdk/iotcentral/tests.mgmt.yml index dcdc2e0d16bb2..d4ec351bf6504 100644 --- a/sdk/iotcentral/tests.mgmt.yml +++ b/sdk/iotcentral/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: iotcentral - Artifacts: - - name: azure-resourcemanager-iotcentral - groupId: com.azure.resourcemanager - safeName: azureresourcemanageriotcentral - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: iotcentral + Artifacts: + - name: azure-resourcemanager-iotcentral + groupId: com.azure.resourcemanager + safeName: azureresourcemanageriotcentral + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/CHANGELOG.md b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/CHANGELOG.md index 3af45b0c69953..3eb2e20a1d75e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/CHANGELOG.md +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.0.0-beta.2 (Unreleased) +## 1.1.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,244 @@ ### Other Changes +## 1.0.0 (2024-03-18) + +- Azure Resource Manager IoT Firmware Defense client library for Java. This package contains Microsoft Azure SDK for IoT Firmware Defense Management SDK. Firmware & IoT Security REST API. Package tag package-2024-01-10. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Breaking Changes + +* `models.Cve` was removed + +* `models.CryptoKeyList` was removed + +* `models.PasswordHashList` was removed + +* `models.NxFlag` was removed + +* `models.CveList` was removed + +* `models.CryptoKey` was removed + +* `models.CryptoCertificateList` was removed + +* `models.CryptoKeySummary` was removed + +* `models.BinaryHardeningSummary` was removed + +* `models.BinaryHardeningList` was removed + +* `models.CanaryFlag` was removed + +* `models.IsExpired` was removed + +* `models.IsUpdateAvailable` was removed + +* `models.CryptoCertificateSummary` was removed + +* `models.PasswordHash` was removed + +* `models.PieFlag` was removed + +* `models.StrippedFlag` was removed + +* `models.IsSelfSigned` was removed + +* `models.CryptoCertificate` was removed + +* `models.IsWeakSignature` was removed + +* `models.IsShortKeySize` was removed + +* `models.ComponentList` was removed + +* `models.RelroFlag` was removed + +* `models.Component` was removed + +* `models.BinaryHardening` was removed + +#### `models.PairedKey` was modified + +* `withAdditionalProperties(java.lang.Object)` was removed +* `additionalProperties()` was removed + +#### `models.UrlToken` was modified + +* `uploadUrl()` was removed + +#### `IoTFirmwareDefenseManager` was modified + +* `fluent.Fist serviceClient()` -> `fluent.IoTFirmwareDefense serviceClient()` + +#### `models.FirmwareSummary` was modified + +* `java.lang.Long extractedFileCount()` -> `java.lang.Long extractedFileCount()` +* `java.lang.Long fileSize()` -> `java.lang.Long fileSize()` +* `innerModel()` was removed +* `java.lang.Long componentCount()` -> `java.lang.Long componentCount()` +* `java.lang.Long rootFileSystems()` -> `java.lang.Long rootFileSystems()` +* `java.lang.Long analysisTimeSeconds()` -> `java.lang.Long analysisTimeSeconds()` +* `java.lang.Long binaryCount()` -> `java.lang.Long binaryCount()` +* `java.lang.Long extractedSize()` -> `java.lang.Long extractedSize()` + +#### `models.Firmwares` was modified + +* `generateCveSummary(java.lang.String,java.lang.String,java.lang.String)` was removed +* `listGenerateCveList(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateSummaryWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `listGenerateCryptoKeyList(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `listGeneratePasswordHashList(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `listGenerateCryptoCertificateList(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `listGenerateBinaryHardeningList(java.lang.String,java.lang.String,java.lang.String)` was removed +* `listGenerateCveList(java.lang.String,java.lang.String,java.lang.String)` was removed +* `listGenerateComponentList(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateBinaryHardeningDetails(java.lang.String,java.lang.String,java.lang.String)` was removed +* `generateCryptoKeySummary(java.lang.String,java.lang.String,java.lang.String)` was removed +* `generateBinaryHardeningDetailsWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateBinaryHardeningSummary(java.lang.String,java.lang.String,java.lang.String)` was removed +* `listGenerateCryptoKeyList(java.lang.String,java.lang.String,java.lang.String)` was removed +* `generateCryptoCertificateSummary(java.lang.String,java.lang.String,java.lang.String)` was removed +* `generateCveSummaryWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `listGenerateComponentList(java.lang.String,java.lang.String,java.lang.String)` was removed +* `generateComponentDetails(java.lang.String,java.lang.String,java.lang.String)` was removed +* `listGeneratePasswordHashList(java.lang.String,java.lang.String,java.lang.String)` was removed +* `listGenerateBinaryHardeningList(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateBinaryHardeningSummaryWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateComponentDetailsWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateCryptoKeySummaryWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `generateSummary(java.lang.String,java.lang.String,java.lang.String)` was removed +* `generateCryptoCertificateSummaryWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `listGenerateCryptoCertificateList(java.lang.String,java.lang.String,java.lang.String)` was removed + +#### `models.CveSummary` was modified + +* `java.lang.Long critical()` -> `java.lang.Long critical()` +* `java.lang.Long low()` -> `java.lang.Long low()` +* `java.lang.Long high()` -> `java.lang.Long high()` +* `undefined()` was removed +* `innerModel()` was removed +* `java.lang.Long unknown()` -> `java.lang.Long unknown()` +* `java.lang.Long medium()` -> `java.lang.Long medium()` + +#### `models.Firmware` was modified + +* `listGeneratePasswordHashList()` was removed +* `listGenerateCveList(com.azure.core.util.Context)` was removed +* `generateComponentDetailsWithResponse(com.azure.core.util.Context)` was removed +* `generateCryptoKeySummaryWithResponse(com.azure.core.util.Context)` was removed +* `listGenerateCveList()` was removed +* `generateCveSummary()` was removed +* `listGenerateComponentList()` was removed +* `generateComponentDetails()` was removed +* `generateBinaryHardeningDetails()` was removed +* `listGenerateComponentList(com.azure.core.util.Context)` was removed +* `listGenerateBinaryHardeningList(com.azure.core.util.Context)` was removed +* `listGenerateCryptoKeyList(com.azure.core.util.Context)` was removed +* `listGenerateBinaryHardeningList()` was removed +* `listGenerateCryptoCertificateList(com.azure.core.util.Context)` was removed +* `generateCryptoKeySummary()` was removed +* `generateBinaryHardeningSummary()` was removed +* `listGenerateCryptoKeyList()` was removed +* `generateSummaryWithResponse(com.azure.core.util.Context)` was removed +* `listGeneratePasswordHashList(com.azure.core.util.Context)` was removed +* `generateSummary()` was removed +* `generateCryptoCertificateSummary()` was removed +* `generateBinaryHardeningDetailsWithResponse(com.azure.core.util.Context)` was removed +* `generateCryptoCertificateSummaryWithResponse(com.azure.core.util.Context)` was removed +* `listGenerateCryptoCertificateList()` was removed +* `generateCveSummaryWithResponse(com.azure.core.util.Context)` was removed +* `generateBinaryHardeningSummaryWithResponse(com.azure.core.util.Context)` was removed + +### Features Added + +* `models.SbomComponentResource` was added + +* `models.CveResource` was added + +* `models.SbomComponentListResult` was added + +* `models.BinaryHardenings` was added + +* `models.CryptoCertificateSummaryResource` was added + +* `models.BinaryHardeningListResult` was added + +* `models.CryptoKeyListResult` was added + +* `models.SummaryListResult` was added + +* `models.BinaryHardeningSummaryResource` was added + +* `models.CveListResult` was added + +* `models.CryptoKeys` was added + +* `models.Summaries` was added + +* `models.SbomComponents` was added + +* `models.SummaryType` was added + +* `models.SummaryResource` was added + +* `models.StatusMessage` was added + +* `models.CveComponent` was added + +* `models.CryptoCertificateListResult` was added + +* `models.SummaryResourceProperties` was added + +* `models.CryptoCertificates` was added + +* `models.CryptoKeyResource` was added + +* `models.SummaryName` was added + +* `models.CryptoCertificateResource` was added + +* `models.CryptoKeySummaryResource` was added + +* `models.PasswordHashes` was added + +* `models.Cves` was added + +* `models.PasswordHashResource` was added + +* `models.PasswordHashListResult` was added + +* `models.BinaryHardeningResource` was added + +#### `IoTFirmwareDefenseManager` was modified + +* `passwordHashes()` was added +* `binaryHardenings()` was added +* `cryptoCertificates()` was added +* `sbomComponents()` was added +* `summaries()` was added +* `cves()` was added +* `cryptoKeys()` was added + +#### `models.FirmwareSummary` was modified + +* `withExtractedFileCount(java.lang.Long)` was added +* `withFileSize(java.lang.Long)` was added +* `withExtractedSize(java.lang.Long)` was added +* `withComponentCount(java.lang.Long)` was added +* `withBinaryCount(java.lang.Long)` was added +* `withAnalysisTimeSeconds(java.lang.Long)` was added +* `withRootFileSystems(java.lang.Long)` was added +* `validate()` was added + +#### `models.CveSummary` was modified + +* `withUnknown(java.lang.Long)` was added +* `withLow(java.lang.Long)` was added +* `withMedium(java.lang.Long)` was added +* `validate()` was added +* `withHigh(java.lang.Long)` was added +* `withCritical(java.lang.Long)` was added + ## 1.0.0-beta.1 (2023-07-20) - Azure Resource Manager IoT Firmware Defense client library for Java. This package contains Microsoft Azure SDK for IoT Firmware Defense Management SDK. The definitions and parameters in this swagger specification will be used to manage the IoT Firmware Defense resources. Package tag package-2023-02-08-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/README.md b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/README.md index c59231da6be3f..2444da68044d9 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/README.md +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/README.md @@ -2,7 +2,7 @@ Azure Resource Manager IoT Firmware Defense client library for Java. -This package contains Microsoft Azure SDK for IoT Firmware Defense Management SDK. The definitions and parameters in this swagger specification will be used to manage the IoT Firmware Defense resources. Package tag package-2023-02-08-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for IoT Firmware Defense Management SDK. Firmware & IoT Security REST API. Package tag package-2024-01-10. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-iotfirmwaredefense - 1.0.0-beta.1 + 1.0.0 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -94,7 +94,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/SAMPLE.md b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/SAMPLE.md index 4ac2d3af1ba18..f061a9295800d 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/SAMPLE.md +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/SAMPLE.md @@ -1,890 +1,690 @@ # Code snippets and samples -## Firmware - -- [Create](#firmware_create) -- [Delete](#firmware_delete) -- [GenerateBinaryHardeningDetails](#firmware_generatebinaryhardeningdetails) -- [GenerateBinaryHardeningSummary](#firmware_generatebinaryhardeningsummary) -- [GenerateComponentDetails](#firmware_generatecomponentdetails) -- [GenerateCryptoCertificateSummary](#firmware_generatecryptocertificatesummary) -- [GenerateCryptoKeySummary](#firmware_generatecryptokeysummary) -- [GenerateCveSummary](#firmware_generatecvesummary) -- [GenerateDownloadUrl](#firmware_generatedownloadurl) -- [GenerateFilesystemDownloadUrl](#firmware_generatefilesystemdownloadurl) -- [GenerateSummary](#firmware_generatesummary) -- [Get](#firmware_get) -- [ListByWorkspace](#firmware_listbyworkspace) -- [ListGenerateBinaryHardeningList](#firmware_listgeneratebinaryhardeninglist) -- [ListGenerateComponentList](#firmware_listgeneratecomponentlist) -- [ListGenerateCryptoCertificateList](#firmware_listgeneratecryptocertificatelist) -- [ListGenerateCryptoKeyList](#firmware_listgeneratecryptokeylist) -- [ListGenerateCveList](#firmware_listgeneratecvelist) -- [ListGeneratePasswordHashList](#firmware_listgeneratepasswordhashlist) -- [Update](#firmware_update) +## BinaryHardening -## Operations +- [ListByFirmware](#binaryhardening_listbyfirmware) -- [List](#operations_list) +## CryptoCertificates -## Workspaces +- [ListByFirmware](#cryptocertificates_listbyfirmware) -- [Create](#workspaces_create) -- [Delete](#workspaces_delete) -- [GenerateUploadUrl](#workspaces_generateuploadurl) -- [GetByResourceGroup](#workspaces_getbyresourcegroup) -- [List](#workspaces_list) -- [ListByResourceGroup](#workspaces_listbyresourcegroup) -- [Update](#workspaces_update) -### Firmware_Create +## CryptoKeys -```java -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.serializer.SerializerEncoding; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.io.IOException; -import java.util.Arrays; +- [ListByFirmware](#cryptokeys_listbyfirmware) -/** Samples for Firmware Create. */ -public final class FirmwareCreateSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Create_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_Create_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareCreateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager.firmwares().define("umrkdttp").withExistingWorkspace("rgworkspaces-firmwares", "A7").create(); - } +## Cves - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Create_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_Create_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareCreateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) throws IOException { - manager - .firmwares() - .define("umrkdttp") - .withExistingWorkspace("rgworkspaces-firmwares", "A7") - .withFileName("wresexxulcdsdd") - .withVendor("vycmdhgtmepcptyoubztiuudpkcpd") - .withModel("f") - .withVersion("s") - .withDescription("uz") - .withFileSize(17L) - .withStatus(Status.PENDING) - .withStatusMessages( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"message\":\"ulvhmhokezathzzauiitu\"}", Object.class, SerializerEncoding.JSON))) - .create(); - } -} -``` +- [ListByFirmware](#cves_listbyfirmware) -### Firmware_Delete +## Firmwares -```java -/** Samples for Firmware Delete. */ -public final class FirmwareDeleteSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Delete_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_Delete_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareDeleteMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Delete_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_Delete_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareDeleteMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} -``` +- [Create](#firmwares_create) +- [Delete](#firmwares_delete) +- [GenerateDownloadUrl](#firmwares_generatedownloadurl) +- [GenerateFilesystemDownloadUrl](#firmwares_generatefilesystemdownloadurl) +- [Get](#firmwares_get) +- [ListByWorkspace](#firmwares_listbyworkspace) +- [Update](#firmwares_update) -### Firmware_GenerateBinaryHardeningDetails +## Operations -```java -/** Samples for Firmware GenerateBinaryHardeningDetails. */ -public final class FirmwareGenerateBinaryHardeningDetailsSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningDetails_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningDetails_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningDetailsMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } +- [List](#operations_list) - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningDetails_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningDetails_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningDetailsMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} -``` +## PasswordHashes -### Firmware_GenerateBinaryHardeningSummary +- [ListByFirmware](#passwordhashes_listbyfirmware) -```java -/** Samples for Firmware GenerateBinaryHardeningSummary. */ -public final class FirmwareGenerateBinaryHardeningSummarySamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningSummary_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningSummary_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningSummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } +## SbomComponents - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningSummary_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningSummary_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningSummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} -``` +- [ListByFirmware](#sbomcomponents_listbyfirmware) -### Firmware_GenerateComponentDetails +## Summaries -```java -/** Samples for Firmware GenerateComponentDetails. */ -public final class FirmwareGenerateComponentDetailsSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateComponentDetails_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateComponentDetails_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateComponentDetailsMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateComponentDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } +- [Get](#summaries_get) +- [ListByFirmware](#summaries_listbyfirmware) - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateComponentDetails_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateComponentDetails_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateComponentDetailsMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateComponentDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} -``` +## Workspaces -### Firmware_GenerateCryptoCertificateSummary +- [Create](#workspaces_create) +- [Delete](#workspaces_delete) +- [GenerateUploadUrl](#workspaces_generateuploadurl) +- [GetByResourceGroup](#workspaces_getbyresourcegroup) +- [List](#workspaces_list) +- [ListByResourceGroup](#workspaces_listbyresourcegroup) +- [Update](#workspaces_update) +### BinaryHardening_ListByFirmware ```java -/** Samples for Firmware GenerateCryptoCertificateSummary. */ -public final class FirmwareGenerateCryptoCertificateSummarySamples { +/** + * Samples for BinaryHardening ListByFirmware. + */ +public final class BinaryHardeningListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoCertificateSummary_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * BinaryHardening_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Firmware_GenerateCryptoCertificateSummary_MaximumSet_Gen. - * + * Sample code: BinaryHardening_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateCryptoCertificateSummaryMaximumSetGen( + public static void binaryHardeningListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoCertificateSummaryWithResponse( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); + manager.binaryHardenings().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoCertificateSummary_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * BinaryHardening_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Firmware_GenerateCryptoCertificateSummary_MinimumSet_Gen. - * + * Sample code: BinaryHardening_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateCryptoCertificateSummaryMinimumSetGen( + public static void binaryHardeningListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoCertificateSummaryWithResponse( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); + manager.binaryHardenings().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); } } ``` -### Firmware_GenerateCryptoKeySummary +### CryptoCertificates_ListByFirmware ```java -/** Samples for Firmware GenerateCryptoKeySummary. */ -public final class FirmwareGenerateCryptoKeySummarySamples { +/** + * Samples for CryptoCertificates ListByFirmware. + */ +public final class CryptoCertificatesListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoKeySummary_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoCertificates_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Firmware_GenerateCryptoKeySummary_MaximumSet_Gen. - * + * Sample code: CryptoCertificates_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateCryptoKeySummaryMaximumSetGen( + public static void cryptoCertificatesListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoKeySummaryWithResponse( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); + manager.cryptoCertificates().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoKeySummary_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoCertificates_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Firmware_GenerateCryptoKeySummary_MinimumSet_Gen. - * + * Sample code: CryptoCertificates_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateCryptoKeySummaryMinimumSetGen( + public static void cryptoCertificatesListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoKeySummaryWithResponse( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); + manager.cryptoCertificates().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); } } ``` -### Firmware_GenerateCveSummary +### CryptoKeys_ListByFirmware ```java -/** Samples for Firmware GenerateCveSummary. */ -public final class FirmwareGenerateCveSummarySamples { +/** + * Samples for CryptoKeys ListByFirmware. + */ +public final class CryptoKeysListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCveSummary_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoKeys_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Firmware_GenerateCveSummary_MinimumSet_Gen. - * + * Sample code: CryptoKeys_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateCveSummaryMinimumSetGen( + public static void cryptoKeysListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCveSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.cryptoKeys().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCveSummary_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoKeys_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Firmware_GenerateCveSummary_MaximumSet_Gen. - * + * Sample code: CryptoKeys_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateCveSummaryMaximumSetGen( + public static void cryptoKeysListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCveSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.cryptoKeys().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_GenerateDownloadUrl +### Cves_ListByFirmware ```java -/** Samples for Firmware GenerateDownloadUrl. */ -public final class FirmwareGenerateDownloadUrlSamples { +/** + * Samples for Cves ListByFirmware. + */ +public final class CvesListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateDownloadUrl_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Cves_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Firmware_GenerateDownloadUrl_MaximumSet_Gen. - * + * Sample code: Cves_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateDownloadUrlMaximumSetGen( + public static void cvesListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.cves().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateDownloadUrl_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Cves_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Firmware_GenerateDownloadUrl_MinimumSet_Gen. - * + * Sample code: Cves_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateDownloadUrlMinimumSetGen( + public static void cvesListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.cves().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_GenerateFilesystemDownloadUrl +### Firmwares_Create ```java -/** Samples for Firmware GenerateFilesystemDownloadUrl. */ -public final class FirmwareGenerateFilesystemDownloadUrlSamples { +import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; +import java.util.Arrays; + +/** + * Samples for Firmwares Create. + */ +public final class FirmwaresCreateSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateFilesystemDownloadUrl_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Create_MaximumSet_Gen.json */ /** - * Sample code: Firmware_GenerateFilesystemDownloadUrl_MaximumSet_Gen. - * + * Sample code: Firmwares_Create_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateFilesystemDownloadUrlMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateFilesystemDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + public static void + firmwaresCreateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().define("umrkdttp").withExistingWorkspace("rgworkspaces-firmwares", "A7") + .withFileName("wresexxulcdsdd").withVendor("vycmdhgtmepcptyoubztiuudpkcpd").withModel("f").withVersion("s") + .withDescription("uz").withFileSize(17L).withStatus(Status.PENDING) + .withStatusMessages(Arrays.asList(new StatusMessage().withMessage("ulvhmhokezathzzauiitu"))).create(); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateFilesystemDownloadUrl_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Create_MinimumSet_Gen.json */ /** - * Sample code: Firmware_GenerateFilesystemDownloadUrl_MinimumSet_Gen. - * + * Sample code: Firmwares_Create_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateFilesystemDownloadUrlMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateFilesystemDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + public static void + firmwaresCreateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().define("umrkdttp").withExistingWorkspace("rgworkspaces-firmwares", "A7").create(); } } ``` -### Firmware_GenerateSummary +### Firmwares_Delete ```java -/** Samples for Firmware GenerateSummary. */ -public final class FirmwareGenerateSummarySamples { +/** + * Samples for Firmwares Delete. + */ +public final class FirmwaresDeleteSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateSummary_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Delete_MinimumSet_Gen.json */ /** - * Sample code: Firmware_GenerateSummary_MinimumSet_Gen. - * + * Sample code: Firmwares_Delete_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateSummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateSummaryWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + public static void + firmwaresDeleteMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateSummary_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Delete_MaximumSet_Gen.json */ /** - * Sample code: Firmware_GenerateSummary_MaximumSet_Gen. - * + * Sample code: Firmwares_Delete_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGenerateSummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateSummaryWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + public static void + firmwaresDeleteMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_Get +### Firmwares_GenerateDownloadUrl ```java -/** Samples for Firmware Get. */ -public final class FirmwareGetSamples { +/** + * Samples for Firmwares GenerateDownloadUrl. + */ +public final class FirmwaresGenerateDownloadUrlSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Get_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateDownloadUrl_MaximumSet_Gen.json */ /** - * Sample code: Firmware_Get_MinimumSet_Gen. - * + * Sample code: Firmwares_GenerateDownloadUrl_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGetMinimumSetGen( + public static void firmwaresGenerateDownloadUrlMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.firmwares().generateDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateDownloadUrl_MinimumSet_Gen.json */ /** - * Sample code: Firmware_Get_MaximumSet_Gen. - * + * Sample code: Firmwares_GenerateDownloadUrl_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareGetMaximumSetGen( + public static void firmwaresGenerateDownloadUrlMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.firmwares().generateDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_ListByWorkspace +### Firmwares_GenerateFilesystemDownloadUrl ```java -/** Samples for Firmware ListByWorkspace. */ -public final class FirmwareListByWorkspaceSamples { +/** + * Samples for Firmwares GenerateFilesystemDownloadUrl. + */ +public final class FirmwaresGenerateFilesystemDownloadUrlSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListByWorkspace_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateFilesystemDownloadUrl_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListByWorkspace_MaximumSet_Gen. - * + * Sample code: Firmwares_GenerateFilesystemDownloadUrl_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListByWorkspaceMaximumSetGen( + public static void firmwaresGenerateFilesystemDownloadUrlMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager.firmwares().listByWorkspace("rgworkspaces-firmwares", "A7", com.azure.core.util.Context.NONE); + manager.firmwares().generateFilesystemDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListByWorkspace_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateFilesystemDownloadUrl_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListByWorkspace_MinimumSet_Gen. - * + * Sample code: Firmwares_GenerateFilesystemDownloadUrl_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListByWorkspaceMinimumSetGen( + public static void firmwaresGenerateFilesystemDownloadUrlMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager.firmwares().listByWorkspace("rgworkspaces-firmwares", "A7", com.azure.core.util.Context.NONE); + manager.firmwares().generateFilesystemDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_ListGenerateBinaryHardeningList +### Firmwares_Get ```java -/** Samples for Firmware ListGenerateBinaryHardeningList. */ -public final class FirmwareListGenerateBinaryHardeningListSamples { +/** + * Samples for Firmwares Get. + */ +public final class FirmwaresGetSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateBinaryHardeningList_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Get_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateBinaryHardeningList_MinimumSet_Gen. - * + * Sample code: Firmwares_Get_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateBinaryHardeningListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateBinaryHardeningList( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + public static void + firmwaresGetMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateBinaryHardeningList_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Get_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateBinaryHardeningList_MaximumSet_Gen. - * + * Sample code: Firmwares_Get_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateBinaryHardeningListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateBinaryHardeningList( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + public static void + firmwaresGetMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_ListGenerateComponentList +### Firmwares_ListByWorkspace ```java -/** Samples for Firmware ListGenerateComponentList. */ -public final class FirmwareListGenerateComponentListSamples { +/** + * Samples for Firmwares ListByWorkspace. + */ +public final class FirmwaresListByWorkspaceSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateComponentList_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_ListByWorkspace_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateComponentList_MaximumSet_Gen. - * + * Sample code: Firmwares_ListByWorkspace_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateComponentListMaximumSetGen( + public static void firmwaresListByWorkspaceMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateComponentList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.firmwares().listByWorkspace("rgworkspaces-firmwares", "A7", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateComponentList_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_ListByWorkspace_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateComponentList_MinimumSet_Gen. - * + * Sample code: Firmwares_ListByWorkspace_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateComponentListMinimumSetGen( + public static void firmwaresListByWorkspaceMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateComponentList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.firmwares().listByWorkspace("rgworkspaces-firmwares", "A7", com.azure.core.util.Context.NONE); } } ``` -### Firmware_ListGenerateCryptoCertificateList +### Firmwares_Update ```java -/** Samples for Firmware ListGenerateCryptoCertificateList. */ -public final class FirmwareListGenerateCryptoCertificateListSamples { +import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; +import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; +import java.util.Arrays; + +/** + * Samples for Firmwares Update. + */ +public final class FirmwaresUpdateSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoCertificateList_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Update_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateCryptoCertificateList_MaximumSet_Gen. - * + * Sample code: Firmwares_Update_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateCryptoCertificateListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoCertificateList( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); + public static void + firmwaresUpdateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Firmware resource = manager.firmwares() + .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE).getValue(); + resource.update().withFileName("wresexxulcdsdd").withVendor("vycmdhgtmepcptyoubztiuudpkcpd").withModel("f") + .withVersion("s").withDescription("uz").withFileSize(17L).withStatus(Status.PENDING) + .withStatusMessages(Arrays.asList(new StatusMessage().withMessage("ulvhmhokezathzzauiitu"))).apply(); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoCertificateList_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Update_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateCryptoCertificateList_MinimumSet_Gen. - * + * Sample code: Firmwares_Update_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateCryptoCertificateListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoCertificateList( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); + public static void + firmwaresUpdateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Firmware resource = manager.firmwares() + .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE).getValue(); + resource.update().apply(); } } ``` -### Firmware_ListGenerateCryptoKeyList +### Operations_List ```java -/** Samples for Firmware ListGenerateCryptoKeyList. */ -public final class FirmwareListGenerateCryptoKeyListSamples { +/** + * Samples for Operations List. + */ +public final class OperationsListSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoKeyList_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Operations_List_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateCryptoKeyList_MinimumSet_Gen. - * + * Sample code: Operations_List_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateCryptoKeyListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoKeyList( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); + public static void + operationsListMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.operations().list(com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoKeyList_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Operations_List_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateCryptoKeyList_MaximumSet_Gen. - * + * Sample code: Operations_List_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateCryptoKeyListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoKeyList( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); + public static void + operationsListMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.operations().list(com.azure.core.util.Context.NONE); } } ``` -### Firmware_ListGenerateCveList +### PasswordHashes_ListByFirmware ```java -/** Samples for Firmware ListGenerateCveList. */ -public final class FirmwareListGenerateCveListSamples { +/** + * Samples for PasswordHashes ListByFirmware. + */ +public final class PasswordHashesListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCveList_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * PasswordHashes_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateCveList_MaximumSet_Gen. - * + * Sample code: PasswordHashes_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateCveListMaximumSetGen( + public static void passwordHashesListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCveList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.passwordHashes().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCveList_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * PasswordHashes_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListGenerateCveList_MinimumSet_Gen. - * + * Sample code: PasswordHashes_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGenerateCveListMinimumSetGen( + public static void passwordHashesListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCveList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.passwordHashes().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_ListGeneratePasswordHashList +### SbomComponents_ListByFirmware ```java -/** Samples for Firmware ListGeneratePasswordHashList. */ -public final class FirmwareListGeneratePasswordHashListSamples { +/** + * Samples for SbomComponents ListByFirmware. + */ +public final class SbomComponentsListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGeneratePasswordHashList_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * SbomComponents_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListGeneratePasswordHashList_MinimumSet_Gen. - * + * Sample code: SbomComponents_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGeneratePasswordHashListMinimumSetGen( + public static void sbomComponentsListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGeneratePasswordHashList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.sbomComponents().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGeneratePasswordHashList_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * SbomComponents_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListGeneratePasswordHashList_MaximumSet_Gen. - * + * Sample code: SbomComponents_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListGeneratePasswordHashListMaximumSetGen( + public static void sbomComponentsListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGeneratePasswordHashList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); + manager.sbomComponents().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } } ``` -### Firmware_Update +### Summaries_Get ```java -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.serializer.SerializerEncoding; -import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.io.IOException; -import java.util.Arrays; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName; -/** Samples for Firmware Update. */ -public final class FirmwareUpdateSamples { +/** + * Samples for Summaries Get. + */ +public final class SummariesGetSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_Get_MaximumSet_Gen.json */ /** - * Sample code: Firmware_Update_MaximumSet_Gen. - * + * Sample code: Summaries_Get_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareUpdateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) throws IOException { - Firmware resource = - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withFileName("wresexxulcdsdd") - .withVendor("vycmdhgtmepcptyoubztiuudpkcpd") - .withModel("f") - .withVersion("s") - .withDescription("uz") - .withFileSize(17L) - .withStatus(Status.PENDING) - .withStatusMessages( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"message\":\"ulvhmhokezathzzauiitu\"}", Object.class, SerializerEncoding.JSON))) - .apply(); + public static void + summariesGetMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.summaries().getWithResponse("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + SummaryName.FIRMWARE, com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Update_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_Get_MinimumSet_Gen.json */ /** - * Sample code: Firmware_Update_MinimumSet_Gen. - * + * Sample code: Summaries_Get_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareUpdateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - Firmware resource = - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE) - .getValue(); - resource.update().apply(); + public static void + summariesGetMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.summaries().getWithResponse("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + SummaryName.FIRMWARE, com.azure.core.util.Context.NONE); } } ``` -### Operations_List +### Summaries_ListByFirmware ```java -/** Samples for Operations List. */ -public final class OperationsListSamples { +/** + * Samples for Summaries ListByFirmware. + */ +public final class SummariesListByFirmwareSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Operations_List_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_ListByFirmware_MinimumSet_Gen.json */ /** - * Sample code: Operations_List_MinimumSet_Gen. - * + * Sample code: Summaries_ListByFirmware_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void operationsListMinimumSetGen( + public static void summariesListByFirmwareMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager.operations().list(com.azure.core.util.Context.NONE); + manager.summaries().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Operations_List_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_ListByFirmware_MaximumSet_Gen.json */ /** - * Sample code: Operations_List_MaximumSet_Gen. - * + * Sample code: Summaries_ListByFirmware_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void operationsListMaximumSetGen( + public static void summariesListByFirmwareMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager.operations().list(com.azure.core.util.Context.NONE); + manager.summaries().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); } } ``` @@ -895,45 +695,41 @@ public final class OperationsListSamples { import java.util.HashMap; import java.util.Map; -/** Samples for Workspaces Create. */ +/** + * Samples for Workspaces Create. + */ public final class WorkspacesCreateSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Create_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Create_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Create_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesCreateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .define("E___-3") - .withRegion("jjwbseilitjgdrhbvvkwviqj") - .withExistingResourceGroup("rgworkspaces") - .create(); + public static void + workspacesCreateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().define("E___-3").withRegion("jjwbseilitjgdrhbvvkwviqj") + .withExistingResourceGroup("rgworkspaces").create(); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Create_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Create_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Create_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesCreateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .define("E___-3") - .withRegion("jjwbseilitjgdrhbvvkwviqj") - .withExistingResourceGroup("rgworkspaces") - .withTags(mapOf("key450", "fakeTokenPlaceholder")) - .create(); + public static void + workspacesCreateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().define("E___-3").withRegion("jjwbseilitjgdrhbvvkwviqj") + .withExistingResourceGroup("rgworkspaces").withTags(mapOf("key450", "fakeTokenPlaceholder")).create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -950,36 +746,38 @@ public final class WorkspacesCreateSamples { ### Workspaces_Delete ```java -/** Samples for Workspaces Delete. */ +/** + * Samples for Workspaces Delete. + */ public final class WorkspacesDeleteSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Delete_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Delete_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Delete_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesDeleteMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE); + public static void + workspacesDeleteMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Delete_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Delete_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Delete_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesDeleteMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE); + public static void + workspacesDeleteMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", + com.azure.core.util.Context.NONE); } } ``` @@ -989,41 +787,38 @@ public final class WorkspacesDeleteSamples { ```java import com.azure.resourcemanager.iotfirmwaredefense.models.GenerateUploadUrlRequest; -/** Samples for Workspaces GenerateUploadUrl. */ +/** + * Samples for Workspaces GenerateUploadUrl. + */ public final class WorkspacesGenerateUploadUrlSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_GenerateUploadUrl_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_GenerateUploadUrl_MinimumSet_Gen.json */ /** * Sample code: Workspaces_GenerateUploadUrl_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesGenerateUploadUrlMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .generateUploadUrlWithResponse( - "rgworkspaces", "E___-3", new GenerateUploadUrlRequest(), com.azure.core.util.Context.NONE); + manager.workspaces().generateUploadUrlWithResponse("rgworkspaces", "E___-3", new GenerateUploadUrlRequest(), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_GenerateUploadUrl_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_GenerateUploadUrl_MaximumSet_Gen.json */ /** * Sample code: Workspaces_GenerateUploadUrl_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesGenerateUploadUrlMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .generateUploadUrlWithResponse( - "rgworkspaces", - "E___-3", - new GenerateUploadUrlRequest().withFirmwareId("ytsfprbywi"), - com.azure.core.util.Context.NONE); + manager.workspaces().generateUploadUrlWithResponse("rgworkspaces", "E___-3", + new GenerateUploadUrlRequest().withFirmwareId("ytsfprbywi"), com.azure.core.util.Context.NONE); } } ``` @@ -1031,31 +826,35 @@ public final class WorkspacesGenerateUploadUrlSamples { ### Workspaces_GetByResourceGroup ```java -/** Samples for Workspaces GetByResourceGroup. */ +/** + * Samples for Workspaces GetByResourceGroup. + */ public final class WorkspacesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Get_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Get_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesGetMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + public static void + workspacesGetMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.workspaces().getByResourceGroupWithResponse("rgworkspaces", "E_US", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Get_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Get_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Get_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesGetMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + public static void + workspacesGetMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.workspaces().getByResourceGroupWithResponse("rgworkspaces", "E_US", com.azure.core.util.Context.NONE); } } @@ -1064,14 +863,17 @@ public final class WorkspacesGetByResourceGroupSamples { ### Workspaces_List ```java -/** Samples for Workspaces List. */ +/** + * Samples for Workspaces List. + */ public final class WorkspacesListSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListBySubscription_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListBySubscription_MaximumSet_Gen.json */ /** * Sample code: Workspaces_ListBySubscription_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListBySubscriptionMaximumSetGen( @@ -1080,11 +882,12 @@ public final class WorkspacesListSamples { } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListBySubscription_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListBySubscription_MinimumSet_Gen.json */ /** * Sample code: Workspaces_ListBySubscription_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListBySubscriptionMinimumSetGen( @@ -1097,14 +900,17 @@ public final class WorkspacesListSamples { ### Workspaces_ListByResourceGroup ```java -/** Samples for Workspaces ListByResourceGroup. */ +/** + * Samples for Workspaces ListByResourceGroup. + */ public final class WorkspacesListByResourceGroupSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListByResourceGroup_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListByResourceGroup_MaximumSet_Gen.json */ /** * Sample code: Workspaces_ListByResourceGroup_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListByResourceGroupMaximumSetGen( @@ -1113,11 +919,12 @@ public final class WorkspacesListByResourceGroupSamples { } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListByResourceGroup_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListByResourceGroup_MinimumSet_Gen.json */ /** * Sample code: Workspaces_ListByResourceGroup_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListByResourceGroupMinimumSetGen( @@ -1132,41 +939,39 @@ public final class WorkspacesListByResourceGroupSamples { ```java import com.azure.resourcemanager.iotfirmwaredefense.models.Workspace; -/** Samples for Workspaces Update. */ +/** + * Samples for Workspaces Update. + */ public final class WorkspacesUpdateSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Update_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Update_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesUpdateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - Workspace resource = - manager - .workspaces() - .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE) - .getValue(); + public static void + workspacesUpdateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Workspace resource = manager.workspaces() + .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Update_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Update_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Update_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesUpdateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - Workspace resource = - manager - .workspaces() - .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE) - .getValue(); + public static void + workspacesUpdateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Workspace resource = manager.workspaces() + .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/pom.xml b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/pom.xml index 465815f8137e7..674404aec23e7 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/pom.xml +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-iotfirmwaredefense - 1.0.0-beta.2 + 1.1.0-beta.1 jar Microsoft Azure SDK for IoT Firmware Defense Management - This package contains Microsoft Azure SDK for IoT Firmware Defense Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. The definitions and parameters in this swagger specification will be used to manage the IoT Firmware Defense resources. Package tag package-2023-02-08-preview. + This package contains Microsoft Azure SDK for IoT Firmware Defense Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Firmware & IoT Security REST API. Package tag package-2024-01-10. https://github.com/Azure/azure-sdk-for-java @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/IoTFirmwareDefenseManager.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/IoTFirmwareDefenseManager.java index 25e0d19ed8069..c3f1c3a689e7b 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/IoTFirmwareDefenseManager.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/IoTFirmwareDefenseManager.java @@ -23,13 +23,27 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.core.util.Configuration; import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.Fist; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.IoTFirmwareDefense; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.BinaryHardeningsImpl; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.CryptoCertificatesImpl; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.CryptoKeysImpl; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.CvesImpl; import com.azure.resourcemanager.iotfirmwaredefense.implementation.FirmwaresImpl; -import com.azure.resourcemanager.iotfirmwaredefense.implementation.FistBuilder; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.IoTFirmwareDefenseBuilder; import com.azure.resourcemanager.iotfirmwaredefense.implementation.OperationsImpl; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.PasswordHashesImpl; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.SbomComponentsImpl; +import com.azure.resourcemanager.iotfirmwaredefense.implementation.SummariesImpl; import com.azure.resourcemanager.iotfirmwaredefense.implementation.WorkspacesImpl; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardenings; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificates; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeys; +import com.azure.resourcemanager.iotfirmwaredefense.models.Cves; import com.azure.resourcemanager.iotfirmwaredefense.models.Firmwares; import com.azure.resourcemanager.iotfirmwaredefense.models.Operations; +import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashes; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponents; +import com.azure.resourcemanager.iotfirmwaredefense.models.Summaries; import com.azure.resourcemanager.iotfirmwaredefense.models.Workspaces; import java.time.Duration; import java.time.temporal.ChronoUnit; @@ -39,33 +53,43 @@ import java.util.stream.Collectors; /** - * Entry point to IoTFirmwareDefenseManager. The definitions and parameters in this swagger specification will be used - * to manage the IoT Firmware Defense resources. + * Entry point to IoTFirmwareDefenseManager. + * Firmware & IoT Security REST API. */ public final class IoTFirmwareDefenseManager { - private Firmwares firmwares; + private BinaryHardenings binaryHardenings; - private Workspaces workspaces; + private CryptoCertificates cryptoCertificates; + + private CryptoKeys cryptoKeys; + + private Cves cves; + + private Firmwares firmwares; private Operations operations; - private final Fist clientObject; + private PasswordHashes passwordHashes; + + private SbomComponents sbomComponents; + + private Summaries summaries; + + private Workspaces workspaces; + + private final IoTFirmwareDefense clientObject; private IoTFirmwareDefenseManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new FistBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new IoTFirmwareDefenseBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of IoT Firmware Defense service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the IoT Firmware Defense service API instance. @@ -78,7 +102,7 @@ public static IoTFirmwareDefenseManager authenticate(TokenCredential credential, /** * Creates an instance of IoT Firmware Defense service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the IoT Firmware Defense service API instance. @@ -91,14 +115,16 @@ public static IoTFirmwareDefenseManager authenticate(HttpPipeline httpPipeline, /** * Gets a Configurable instance that can be used to create IoTFirmwareDefenseManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new IoTFirmwareDefenseManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -170,8 +196,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -188,8 +214,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -209,21 +235,12 @@ public IoTFirmwareDefenseManager authenticate(TokenCredential credential, AzureP Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.iotfirmwaredefense") - .append("/") - .append("1.0.0-beta.1"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.iotfirmwaredefense") + .append("/").append("1.0.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -242,38 +259,73 @@ public IoTFirmwareDefenseManager authenticate(TokenCredential credential, AzureP policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new IoTFirmwareDefenseManager(httpPipeline, profile, defaultPollInterval); } } + /** + * Gets the resource collection API of BinaryHardenings. + * + * @return Resource collection API of BinaryHardenings. + */ + public BinaryHardenings binaryHardenings() { + if (this.binaryHardenings == null) { + this.binaryHardenings = new BinaryHardeningsImpl(clientObject.getBinaryHardenings(), this); + } + return binaryHardenings; + } + + /** + * Gets the resource collection API of CryptoCertificates. + * + * @return Resource collection API of CryptoCertificates. + */ + public CryptoCertificates cryptoCertificates() { + if (this.cryptoCertificates == null) { + this.cryptoCertificates = new CryptoCertificatesImpl(clientObject.getCryptoCertificates(), this); + } + return cryptoCertificates; + } + + /** + * Gets the resource collection API of CryptoKeys. + * + * @return Resource collection API of CryptoKeys. + */ + public CryptoKeys cryptoKeys() { + if (this.cryptoKeys == null) { + this.cryptoKeys = new CryptoKeysImpl(clientObject.getCryptoKeys(), this); + } + return cryptoKeys; + } + + /** + * Gets the resource collection API of Cves. + * + * @return Resource collection API of Cves. + */ + public Cves cves() { + if (this.cves == null) { + this.cves = new CvesImpl(clientObject.getCves(), this); + } + return cves; + } + /** * Gets the resource collection API of Firmwares. It manages Firmware. - * + * * @return Resource collection API of Firmwares. */ public Firmwares firmwares() { @@ -283,21 +335,9 @@ public Firmwares firmwares() { return firmwares; } - /** - * Gets the resource collection API of Workspaces. It manages Workspace. - * - * @return Resource collection API of Workspaces. - */ - public Workspaces workspaces() { - if (this.workspaces == null) { - this.workspaces = new WorkspacesImpl(clientObject.getWorkspaces(), this); - } - return workspaces; - } - /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -308,10 +348,60 @@ public Operations operations() { } /** - * @return Wrapped service client Fist providing direct access to the underlying auto-generated API implementation, - * based on Azure REST API. + * Gets the resource collection API of PasswordHashes. + * + * @return Resource collection API of PasswordHashes. + */ + public PasswordHashes passwordHashes() { + if (this.passwordHashes == null) { + this.passwordHashes = new PasswordHashesImpl(clientObject.getPasswordHashes(), this); + } + return passwordHashes; + } + + /** + * Gets the resource collection API of SbomComponents. + * + * @return Resource collection API of SbomComponents. + */ + public SbomComponents sbomComponents() { + if (this.sbomComponents == null) { + this.sbomComponents = new SbomComponentsImpl(clientObject.getSbomComponents(), this); + } + return sbomComponents; + } + + /** + * Gets the resource collection API of Summaries. + * + * @return Resource collection API of Summaries. + */ + public Summaries summaries() { + if (this.summaries == null) { + this.summaries = new SummariesImpl(clientObject.getSummaries(), this); + } + return summaries; + } + + /** + * Gets the resource collection API of Workspaces. It manages Workspace. + * + * @return Resource collection API of Workspaces. + */ + public Workspaces workspaces() { + if (this.workspaces == null) { + this.workspaces = new WorkspacesImpl(clientObject.getWorkspaces(), this); + } + return workspaces; + } + + /** + * Gets wrapped service client IoTFirmwareDefense providing direct access to the underlying auto-generated API + * implementation, based on Azure REST API. + * + * @return Wrapped service client IoTFirmwareDefense. */ - public Fist serviceClient() { + public IoTFirmwareDefense serviceClient() { return this.clientObject; } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/BinaryHardeningsClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/BinaryHardeningsClient.java new file mode 100644 index 0000000000000..01834d11246ec --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/BinaryHardeningsClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; + +/** + * An instance of this class provides access to all the operations defined in BinaryHardeningsClient. + */ +public interface BinaryHardeningsClient { + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoCertificatesClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoCertificatesClient.java new file mode 100644 index 0000000000000..a06def7392ec8 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoCertificatesClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner; + +/** + * An instance of this class provides access to all the operations defined in CryptoCertificatesClient. + */ +public interface CryptoCertificatesClient { + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoKeysClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoKeysClient.java new file mode 100644 index 0000000000000..2fb6b18bac965 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CryptoKeysClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner; + +/** + * An instance of this class provides access to all the operations defined in CryptoKeysClient. + */ +public interface CryptoKeysClient { + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CvesClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CvesClient.java new file mode 100644 index 0000000000000..1f30566ff7fa6 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/CvesClient.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; + +/** + * An instance of this class provides access to all the operations defined in CvesClient. + */ +public interface CvesClient { + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId); + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId, + Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/FirmwaresClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/FirmwaresClient.java index 2b52cc81adea3..3966be2042cbf 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/FirmwaresClient.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/FirmwaresClient.java @@ -9,26 +9,17 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeySummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.UrlTokenInner; import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareUpdateDefinition; -/** An instance of this class provides access to all the operations defined in FirmwaresClient. */ +/** + * An instance of this class provides access to all the operations defined in FirmwaresClient. + */ public interface FirmwaresClient { /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -41,7 +32,7 @@ public interface FirmwaresClient { /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -55,7 +46,7 @@ public interface FirmwaresClient { /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -67,12 +58,12 @@ public interface FirmwaresClient { * @return firmware definition along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareInner firmware, Context context); + Response createWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareInner firmware, Context context); /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -87,7 +78,7 @@ Response createWithResponse( /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -99,16 +90,12 @@ Response createWithResponse( * @return firmware definition along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String workspaceName, - String firmwareId, - FirmwareUpdateDefinition firmware, - Context context); + Response updateWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareUpdateDefinition firmware, Context context); /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -119,12 +106,12 @@ Response updateWithResponse( * @return firmware definition. */ @ServiceMethod(returns = ReturnType.SINGLE) - FirmwareInner update( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareUpdateDefinition firmware); + FirmwareInner update(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareUpdateDefinition firmware); /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -135,12 +122,12 @@ FirmwareInner update( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response deleteWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context); /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -153,7 +140,7 @@ Response deleteWithResponse( /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -164,12 +151,12 @@ Response deleteWithResponse( * @return firmware along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context); /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -183,7 +170,7 @@ Response getWithResponse( /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -194,12 +181,12 @@ Response getWithResponse( * @return url data for creating or accessing a blob file along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response generateDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response generateDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context); /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -213,7 +200,7 @@ Response generateDownloadUrlWithResponse( /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -224,12 +211,12 @@ Response generateDownloadUrlWithResponse( * @return url data for creating or accessing a blob file along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response generateFilesystemDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response generateFilesystemDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context); /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -240,406 +227,4 @@ Response generateFilesystemDownloadUrlWithResponse( */ @ServiceMethod(returns = ReturnType.SINGLE) UrlTokenInner generateFilesystemDownloadUrl(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - FirmwareSummaryInner generateSummary(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateComponentDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - ComponentInner generateComponentDetails(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateBinaryHardeningSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BinaryHardeningSummaryInner generateBinaryHardeningSummary( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateBinaryHardeningDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BinaryHardeningInner generateBinaryHardeningDetails( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateCveList(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateCveList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateCveSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - CveSummaryInner generateCveSummary(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateCryptoCertificateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - CryptoCertificateSummaryInner generateCryptoCertificateSummary( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response generateCryptoKeySummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - CryptoKeySummaryInner generateCryptoKeySummary(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/Fist.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/Fist.java deleted file mode 100644 index dd7e8b60df716..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/Fist.java +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.fluent; - -import com.azure.core.http.HttpPipeline; -import java.time.Duration; - -/** The interface for Fist class. */ -public interface Fist { - /** - * Gets The ID of the target subscription. - * - * @return the subscriptionId value. - */ - String getSubscriptionId(); - - /** - * Gets server parameter. - * - * @return the endpoint value. - */ - String getEndpoint(); - - /** - * Gets Api Version. - * - * @return the apiVersion value. - */ - String getApiVersion(); - - /** - * Gets The HTTP pipeline to send requests through. - * - * @return the httpPipeline value. - */ - HttpPipeline getHttpPipeline(); - - /** - * Gets The default poll interval for long-running operation. - * - * @return the defaultPollInterval value. - */ - Duration getDefaultPollInterval(); - - /** - * Gets the FirmwaresClient object to access its operations. - * - * @return the FirmwaresClient object. - */ - FirmwaresClient getFirmwares(); - - /** - * Gets the WorkspacesClient object to access its operations. - * - * @return the WorkspacesClient object. - */ - WorkspacesClient getWorkspaces(); - - /** - * Gets the OperationsClient object to access its operations. - * - * @return the OperationsClient object. - */ - OperationsClient getOperations(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/IoTFirmwareDefense.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/IoTFirmwareDefense.java new file mode 100644 index 0000000000000..8910eeb3e362b --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/IoTFirmwareDefense.java @@ -0,0 +1,118 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.http.HttpPipeline; +import java.time.Duration; + +/** + * The interface for IoTFirmwareDefense class. + */ +public interface IoTFirmwareDefense { + /** + * Gets The ID of the target subscription. The value must be an UUID. + * + * @return the subscriptionId value. + */ + String getSubscriptionId(); + + /** + * Gets server parameter. + * + * @return the endpoint value. + */ + String getEndpoint(); + + /** + * Gets Api Version. + * + * @return the apiVersion value. + */ + String getApiVersion(); + + /** + * Gets The HTTP pipeline to send requests through. + * + * @return the httpPipeline value. + */ + HttpPipeline getHttpPipeline(); + + /** + * Gets The default poll interval for long-running operation. + * + * @return the defaultPollInterval value. + */ + Duration getDefaultPollInterval(); + + /** + * Gets the BinaryHardeningsClient object to access its operations. + * + * @return the BinaryHardeningsClient object. + */ + BinaryHardeningsClient getBinaryHardenings(); + + /** + * Gets the CryptoCertificatesClient object to access its operations. + * + * @return the CryptoCertificatesClient object. + */ + CryptoCertificatesClient getCryptoCertificates(); + + /** + * Gets the CryptoKeysClient object to access its operations. + * + * @return the CryptoKeysClient object. + */ + CryptoKeysClient getCryptoKeys(); + + /** + * Gets the CvesClient object to access its operations. + * + * @return the CvesClient object. + */ + CvesClient getCves(); + + /** + * Gets the FirmwaresClient object to access its operations. + * + * @return the FirmwaresClient object. + */ + FirmwaresClient getFirmwares(); + + /** + * Gets the OperationsClient object to access its operations. + * + * @return the OperationsClient object. + */ + OperationsClient getOperations(); + + /** + * Gets the PasswordHashesClient object to access its operations. + * + * @return the PasswordHashesClient object. + */ + PasswordHashesClient getPasswordHashes(); + + /** + * Gets the SbomComponentsClient object to access its operations. + * + * @return the SbomComponentsClient object. + */ + SbomComponentsClient getSbomComponents(); + + /** + * Gets the SummariesClient object to access its operations. + * + * @return the SummariesClient object. + */ + SummariesClient getSummaries(); + + /** + * Gets the WorkspacesClient object to access its operations. + * + * @return the WorkspacesClient object. + */ + WorkspacesClient getWorkspaces(); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/OperationsClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/OperationsClient.java index a6f466c735d01..2c1ecd598b94b 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/OperationsClient.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/OperationsClient.java @@ -10,28 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Lists the operations for this resource provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Lists the operations for this resource provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/PasswordHashesClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/PasswordHashesClient.java new file mode 100644 index 0000000000000..816f0abf61bf7 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/PasswordHashesClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner; + +/** + * An instance of this class provides access to all the operations defined in PasswordHashesClient. + */ +public interface PasswordHashesClient { + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SbomComponentsClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SbomComponentsClient.java new file mode 100644 index 0000000000000..109ea4fc1b370 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SbomComponentsClient.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; + +/** + * An instance of this class provides access to all the operations defined in SbomComponentsClient. + */ +public interface SbomComponentsClient { + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SummariesClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SummariesClient.java new file mode 100644 index 0000000000000..cb65fe78869a6 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/SummariesClient.java @@ -0,0 +1,82 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName; + +/** + * An instance of this class provides access to all the operations defined in SummariesClient. + */ +public interface SummariesClient { + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName, Context context); + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + SummaryResourceInner get(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/WorkspacesClient.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/WorkspacesClient.java index ca7c6040a3d61..a69ea8da5bbb1 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/WorkspacesClient.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/WorkspacesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.iotfirmwaredefense.models.GenerateUploadUrlRequest; import com.azure.resourcemanager.iotfirmwaredefense.models.WorkspaceUpdateDefinition; -/** An instance of this class provides access to all the operations defined in WorkspacesClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspacesClient. + */ public interface WorkspacesClient { /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces as paginated response with {@link PagedIterable}. @@ -28,7 +30,7 @@ public interface WorkspacesClient { /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -40,7 +42,7 @@ public interface WorkspacesClient { /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -52,7 +54,7 @@ public interface WorkspacesClient { /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -65,7 +67,7 @@ public interface WorkspacesClient { /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -76,12 +78,12 @@ public interface WorkspacesClient { * @return firmware analysis workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String resourceGroupName, String workspaceName, WorkspaceInner workspace, Context context); + Response createWithResponse(String resourceGroupName, String workspaceName, + WorkspaceInner workspace, Context context); /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -95,7 +97,7 @@ Response createWithResponse( /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -106,12 +108,12 @@ Response createWithResponse( * @return firmware analysis workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, String workspaceName, WorkspaceUpdateDefinition workspace, Context context); + Response updateWithResponse(String resourceGroupName, String workspaceName, + WorkspaceUpdateDefinition workspace, Context context); /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -125,7 +127,7 @@ Response updateWithResponse( /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -139,7 +141,7 @@ Response updateWithResponse( /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -151,7 +153,7 @@ Response updateWithResponse( /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -161,12 +163,12 @@ Response updateWithResponse( * @return firmware analysis workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context); /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -179,7 +181,7 @@ Response getByResourceGroupWithResponse( /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -190,12 +192,12 @@ Response getByResourceGroupWithResponse( * @return url data for creating or accessing a blob file along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response generateUploadUrlWithResponse( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl, Context context); + Response generateUploadUrlWithResponse(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl, Context context); /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -205,6 +207,6 @@ Response generateUploadUrlWithResponse( * @return url data for creating or accessing a blob file. */ @ServiceMethod(returns = ReturnType.SINGLE) - UrlTokenInner generateUploadUrl( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl); + UrlTokenInner generateUploadUrl(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningFeatures.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningFeatures.java index fe437801507c7..e9f07a37166f1 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningFeatures.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningFeatures.java @@ -5,153 +5,152 @@ package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; import com.fasterxml.jackson.annotation.JsonProperty; -/** Binary hardening features. */ +/** + * Binary hardening features. + */ @Fluent public final class BinaryHardeningFeatures { /* - * NX flag. + * NX (no-execute) flag. */ @JsonProperty(value = "nx") - private NxFlag nx; + private Boolean nx; /* - * PIE flag. + * PIE (position independent executable) flag. */ @JsonProperty(value = "pie") - private PieFlag pie; + private Boolean pie; /* - * RELRO flag. + * RELRO (relocation read-only) flag. */ @JsonProperty(value = "relro") - private RelroFlag relro; + private Boolean relro; /* - * Canary flag. + * Canary (stack canaries) flag. */ @JsonProperty(value = "canary") - private CanaryFlag canary; + private Boolean canary; /* * Stripped flag. */ @JsonProperty(value = "stripped") - private StrippedFlag stripped; + private Boolean stripped; - /** Creates an instance of BinaryHardeningFeatures class. */ + /** + * Creates an instance of BinaryHardeningFeatures class. + */ public BinaryHardeningFeatures() { } /** - * Get the nx property: NX flag. - * + * Get the nx property: NX (no-execute) flag. + * * @return the nx value. */ - public NxFlag nx() { + public Boolean nx() { return this.nx; } /** - * Set the nx property: NX flag. - * + * Set the nx property: NX (no-execute) flag. + * * @param nx the nx value to set. * @return the BinaryHardeningFeatures object itself. */ - public BinaryHardeningFeatures withNx(NxFlag nx) { + public BinaryHardeningFeatures withNx(Boolean nx) { this.nx = nx; return this; } /** - * Get the pie property: PIE flag. - * + * Get the pie property: PIE (position independent executable) flag. + * * @return the pie value. */ - public PieFlag pie() { + public Boolean pie() { return this.pie; } /** - * Set the pie property: PIE flag. - * + * Set the pie property: PIE (position independent executable) flag. + * * @param pie the pie value to set. * @return the BinaryHardeningFeatures object itself. */ - public BinaryHardeningFeatures withPie(PieFlag pie) { + public BinaryHardeningFeatures withPie(Boolean pie) { this.pie = pie; return this; } /** - * Get the relro property: RELRO flag. - * + * Get the relro property: RELRO (relocation read-only) flag. + * * @return the relro value. */ - public RelroFlag relro() { + public Boolean relro() { return this.relro; } /** - * Set the relro property: RELRO flag. - * + * Set the relro property: RELRO (relocation read-only) flag. + * * @param relro the relro value to set. * @return the BinaryHardeningFeatures object itself. */ - public BinaryHardeningFeatures withRelro(RelroFlag relro) { + public BinaryHardeningFeatures withRelro(Boolean relro) { this.relro = relro; return this; } /** - * Get the canary property: Canary flag. - * + * Get the canary property: Canary (stack canaries) flag. + * * @return the canary value. */ - public CanaryFlag canary() { + public Boolean canary() { return this.canary; } /** - * Set the canary property: Canary flag. - * + * Set the canary property: Canary (stack canaries) flag. + * * @param canary the canary value to set. * @return the BinaryHardeningFeatures object itself. */ - public BinaryHardeningFeatures withCanary(CanaryFlag canary) { + public BinaryHardeningFeatures withCanary(Boolean canary) { this.canary = canary; return this; } /** * Get the stripped property: Stripped flag. - * + * * @return the stripped value. */ - public StrippedFlag stripped() { + public Boolean stripped() { return this.stripped; } /** * Set the stripped property: Stripped flag. - * + * * @param stripped the stripped value to set. * @return the BinaryHardeningFeatures object itself. */ - public BinaryHardeningFeatures withStripped(StrippedFlag stripped) { + public BinaryHardeningFeatures withStripped(Boolean stripped) { this.stripped = stripped; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResourceInner.java new file mode 100644 index 0000000000000..443c1b36cedd8 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResourceInner.java @@ -0,0 +1,316 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * binary hardening analysis result resource. + */ +@Fluent +public final class BinaryHardeningResourceInner extends ProxyResource { + /* + * The properties of a binary hardening result found within a firmware image + */ + @JsonProperty(value = "properties") + private BinaryHardeningResult innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of BinaryHardeningResourceInner class. + */ + public BinaryHardeningResourceInner() { + } + + /** + * Get the innerProperties property: The properties of a binary hardening result found within a firmware image. + * + * @return the innerProperties value. + */ + private BinaryHardeningResult innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the binaryHardeningId property: ID for the binary hardening result. + * + * @return the binaryHardeningId value. + */ + public String binaryHardeningId() { + return this.innerProperties() == null ? null : this.innerProperties().binaryHardeningId(); + } + + /** + * Set the binaryHardeningId property: ID for the binary hardening result. + * + * @param binaryHardeningId the binaryHardeningId value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withBinaryHardeningId(String binaryHardeningId) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withBinaryHardeningId(binaryHardeningId); + return this; + } + + /** + * Get the architecture property: The architecture of the uploaded firmware. + * + * @return the architecture value. + */ + public String architecture() { + return this.innerProperties() == null ? null : this.innerProperties().architecture(); + } + + /** + * Set the architecture property: The architecture of the uploaded firmware. + * + * @param architecture the architecture value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withArchitecture(String architecture) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withArchitecture(architecture); + return this; + } + + /** + * Get the filePath property: The executable path. + * + * @return the filePath value. + */ + public String filePath() { + return this.innerProperties() == null ? null : this.innerProperties().filePath(); + } + + /** + * Set the filePath property: The executable path. + * + * @param filePath the filePath value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withFilePath(String filePath) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withFilePath(filePath); + return this; + } + + /** + * Get the classProperty property: The executable class to indicate 32 or 64 bit. + * + * @return the classProperty value. + */ + public String classProperty() { + return this.innerProperties() == null ? null : this.innerProperties().classProperty(); + } + + /** + * Set the classProperty property: The executable class to indicate 32 or 64 bit. + * + * @param classProperty the classProperty value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withClassProperty(String classProperty) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withClassProperty(classProperty); + return this; + } + + /** + * Get the runpath property: The runpath of the uploaded firmware. + * + * @return the runpath value. + */ + public String runpath() { + return this.innerProperties() == null ? null : this.innerProperties().runpath(); + } + + /** + * Set the runpath property: The runpath of the uploaded firmware. + * + * @param runpath the runpath value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withRunpath(String runpath) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withRunpath(runpath); + return this; + } + + /** + * Get the rpath property: The rpath of the uploaded firmware. + * + * @return the rpath value. + */ + public String rpath() { + return this.innerProperties() == null ? null : this.innerProperties().rpath(); + } + + /** + * Set the rpath property: The rpath of the uploaded firmware. + * + * @param rpath the rpath value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withRpath(String rpath) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withRpath(rpath); + return this; + } + + /** + * Get the nx property: NX (no-execute) flag. + * + * @return the nx value. + */ + public Boolean nx() { + return this.innerProperties() == null ? null : this.innerProperties().nx(); + } + + /** + * Set the nx property: NX (no-execute) flag. + * + * @param nx the nx value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withNx(Boolean nx) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withNx(nx); + return this; + } + + /** + * Get the pie property: PIE (position independent executable) flag. + * + * @return the pie value. + */ + public Boolean pie() { + return this.innerProperties() == null ? null : this.innerProperties().pie(); + } + + /** + * Set the pie property: PIE (position independent executable) flag. + * + * @param pie the pie value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withPie(Boolean pie) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withPie(pie); + return this; + } + + /** + * Get the relro property: RELRO (relocation read-only) flag. + * + * @return the relro value. + */ + public Boolean relro() { + return this.innerProperties() == null ? null : this.innerProperties().relro(); + } + + /** + * Set the relro property: RELRO (relocation read-only) flag. + * + * @param relro the relro value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withRelro(Boolean relro) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withRelro(relro); + return this; + } + + /** + * Get the canary property: Canary (stack canaries) flag. + * + * @return the canary value. + */ + public Boolean canary() { + return this.innerProperties() == null ? null : this.innerProperties().canary(); + } + + /** + * Set the canary property: Canary (stack canaries) flag. + * + * @param canary the canary value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withCanary(Boolean canary) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withCanary(canary); + return this; + } + + /** + * Get the stripped property: Stripped flag. + * + * @return the stripped value. + */ + public Boolean stripped() { + return this.innerProperties() == null ? null : this.innerProperties().stripped(); + } + + /** + * Set the stripped property: Stripped flag. + * + * @param stripped the stripped value to set. + * @return the BinaryHardeningResourceInner object itself. + */ + public BinaryHardeningResourceInner withStripped(Boolean stripped) { + if (this.innerProperties() == null) { + this.innerProperties = new BinaryHardeningResult(); + } + this.innerProperties().withStripped(stripped); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResult.java similarity index 65% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResult.java index cd64564164d06..dc31bef3f0972 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningResult.java @@ -5,16 +5,13 @@ package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; import com.fasterxml.jackson.annotation.JsonProperty; -/** Binary hardening of a firmware. */ +/** + * Binary hardening of a firmware. + */ @Fluent -public final class BinaryHardeningInner { +public final class BinaryHardeningResult { /* * ID for the binary hardening result. */ @@ -34,13 +31,13 @@ public final class BinaryHardeningInner { private String architecture; /* - * path for binary hardening. + * The executable path. */ - @JsonProperty(value = "path") - private String path; + @JsonProperty(value = "filePath") + private String filePath; /* - * class for binary hardening. + * The executable class to indicate 32 or 64 bit. */ @JsonProperty(value = "class") private String classProperty; @@ -57,13 +54,15 @@ public final class BinaryHardeningInner { @JsonProperty(value = "rpath") private String rpath; - /** Creates an instance of BinaryHardeningInner class. */ - public BinaryHardeningInner() { + /** + * Creates an instance of BinaryHardeningResult class. + */ + public BinaryHardeningResult() { } /** * Get the binaryHardeningId property: ID for the binary hardening result. - * + * * @return the binaryHardeningId value. */ public String binaryHardeningId() { @@ -72,18 +71,18 @@ public String binaryHardeningId() { /** * Set the binaryHardeningId property: ID for the binary hardening result. - * + * * @param binaryHardeningId the binaryHardeningId value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withBinaryHardeningId(String binaryHardeningId) { + public BinaryHardeningResult withBinaryHardeningId(String binaryHardeningId) { this.binaryHardeningId = binaryHardeningId; return this; } /** * Get the innerFeatures property: Binary hardening features. - * + * * @return the innerFeatures value. */ private BinaryHardeningFeatures innerFeatures() { @@ -92,7 +91,7 @@ private BinaryHardeningFeatures innerFeatures() { /** * Get the architecture property: The architecture of the uploaded firmware. - * + * * @return the architecture value. */ public String architecture() { @@ -101,38 +100,38 @@ public String architecture() { /** * Set the architecture property: The architecture of the uploaded firmware. - * + * * @param architecture the architecture value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withArchitecture(String architecture) { + public BinaryHardeningResult withArchitecture(String architecture) { this.architecture = architecture; return this; } /** - * Get the path property: path for binary hardening. - * - * @return the path value. + * Get the filePath property: The executable path. + * + * @return the filePath value. */ - public String path() { - return this.path; + public String filePath() { + return this.filePath; } /** - * Set the path property: path for binary hardening. - * - * @param path the path value to set. - * @return the BinaryHardeningInner object itself. + * Set the filePath property: The executable path. + * + * @param filePath the filePath value to set. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withPath(String path) { - this.path = path; + public BinaryHardeningResult withFilePath(String filePath) { + this.filePath = filePath; return this; } /** - * Get the classProperty property: class for binary hardening. - * + * Get the classProperty property: The executable class to indicate 32 or 64 bit. + * * @return the classProperty value. */ public String classProperty() { @@ -140,19 +139,19 @@ public String classProperty() { } /** - * Set the classProperty property: class for binary hardening. - * + * Set the classProperty property: The executable class to indicate 32 or 64 bit. + * * @param classProperty the classProperty value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withClassProperty(String classProperty) { + public BinaryHardeningResult withClassProperty(String classProperty) { this.classProperty = classProperty; return this; } /** * Get the runpath property: The runpath of the uploaded firmware. - * + * * @return the runpath value. */ public String runpath() { @@ -161,18 +160,18 @@ public String runpath() { /** * Set the runpath property: The runpath of the uploaded firmware. - * + * * @param runpath the runpath value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withRunpath(String runpath) { + public BinaryHardeningResult withRunpath(String runpath) { this.runpath = runpath; return this; } /** * Get the rpath property: The rpath of the uploaded firmware. - * + * * @return the rpath value. */ public String rpath() { @@ -181,31 +180,31 @@ public String rpath() { /** * Set the rpath property: The rpath of the uploaded firmware. - * + * * @param rpath the rpath value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withRpath(String rpath) { + public BinaryHardeningResult withRpath(String rpath) { this.rpath = rpath; return this; } /** - * Get the nx property: NX flag. - * + * Get the nx property: NX (no-execute) flag. + * * @return the nx value. */ - public NxFlag nx() { + public Boolean nx() { return this.innerFeatures() == null ? null : this.innerFeatures().nx(); } /** - * Set the nx property: NX flag. - * + * Set the nx property: NX (no-execute) flag. + * * @param nx the nx value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withNx(NxFlag nx) { + public BinaryHardeningResult withNx(Boolean nx) { if (this.innerFeatures() == null) { this.innerFeatures = new BinaryHardeningFeatures(); } @@ -214,21 +213,21 @@ public BinaryHardeningInner withNx(NxFlag nx) { } /** - * Get the pie property: PIE flag. - * + * Get the pie property: PIE (position independent executable) flag. + * * @return the pie value. */ - public PieFlag pie() { + public Boolean pie() { return this.innerFeatures() == null ? null : this.innerFeatures().pie(); } /** - * Set the pie property: PIE flag. - * + * Set the pie property: PIE (position independent executable) flag. + * * @param pie the pie value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withPie(PieFlag pie) { + public BinaryHardeningResult withPie(Boolean pie) { if (this.innerFeatures() == null) { this.innerFeatures = new BinaryHardeningFeatures(); } @@ -237,21 +236,21 @@ public BinaryHardeningInner withPie(PieFlag pie) { } /** - * Get the relro property: RELRO flag. - * + * Get the relro property: RELRO (relocation read-only) flag. + * * @return the relro value. */ - public RelroFlag relro() { + public Boolean relro() { return this.innerFeatures() == null ? null : this.innerFeatures().relro(); } /** - * Set the relro property: RELRO flag. - * + * Set the relro property: RELRO (relocation read-only) flag. + * * @param relro the relro value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withRelro(RelroFlag relro) { + public BinaryHardeningResult withRelro(Boolean relro) { if (this.innerFeatures() == null) { this.innerFeatures = new BinaryHardeningFeatures(); } @@ -260,21 +259,21 @@ public BinaryHardeningInner withRelro(RelroFlag relro) { } /** - * Get the canary property: Canary flag. - * + * Get the canary property: Canary (stack canaries) flag. + * * @return the canary value. */ - public CanaryFlag canary() { + public Boolean canary() { return this.innerFeatures() == null ? null : this.innerFeatures().canary(); } /** - * Set the canary property: Canary flag. - * + * Set the canary property: Canary (stack canaries) flag. + * * @param canary the canary value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withCanary(CanaryFlag canary) { + public BinaryHardeningResult withCanary(Boolean canary) { if (this.innerFeatures() == null) { this.innerFeatures = new BinaryHardeningFeatures(); } @@ -284,20 +283,20 @@ public BinaryHardeningInner withCanary(CanaryFlag canary) { /** * Get the stripped property: Stripped flag. - * + * * @return the stripped value. */ - public StrippedFlag stripped() { + public Boolean stripped() { return this.innerFeatures() == null ? null : this.innerFeatures().stripped(); } /** * Set the stripped property: Stripped flag. - * + * * @param stripped the stripped value to set. - * @return the BinaryHardeningInner object itself. + * @return the BinaryHardeningResult object itself. */ - public BinaryHardeningInner withStripped(StrippedFlag stripped) { + public BinaryHardeningResult withStripped(Boolean stripped) { if (this.innerFeatures() == null) { this.innerFeatures = new BinaryHardeningFeatures(); } @@ -307,7 +306,7 @@ public BinaryHardeningInner withStripped(StrippedFlag stripped) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/ComponentInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/ComponentInner.java deleted file mode 100644 index a489bcdc2363a..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/ComponentInner.java +++ /dev/null @@ -1,209 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsUpdateAvailable; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; -import java.util.List; - -/** Component of a firmware. */ -@Fluent -public final class ComponentInner { - /* - * ID for the component. - */ - @JsonProperty(value = "componentId") - private String componentId; - - /* - * Name for the component. - */ - @JsonProperty(value = "componentName") - private String componentName; - - /* - * Version for the component. - */ - @JsonProperty(value = "version") - private String version; - - /* - * License for the component. - */ - @JsonProperty(value = "license") - private String license; - - /* - * Release date for the component. - */ - @JsonProperty(value = "releaseDate") - private OffsetDateTime releaseDate; - - /* - * Paths of the component. - */ - @JsonProperty(value = "paths") - private List paths; - - /* - * Flag if new update is available for the component. - */ - @JsonProperty(value = "isUpdateAvailable") - private IsUpdateAvailable isUpdateAvailable; - - /** Creates an instance of ComponentInner class. */ - public ComponentInner() { - } - - /** - * Get the componentId property: ID for the component. - * - * @return the componentId value. - */ - public String componentId() { - return this.componentId; - } - - /** - * Set the componentId property: ID for the component. - * - * @param componentId the componentId value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withComponentId(String componentId) { - this.componentId = componentId; - return this; - } - - /** - * Get the componentName property: Name for the component. - * - * @return the componentName value. - */ - public String componentName() { - return this.componentName; - } - - /** - * Set the componentName property: Name for the component. - * - * @param componentName the componentName value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withComponentName(String componentName) { - this.componentName = componentName; - return this; - } - - /** - * Get the version property: Version for the component. - * - * @return the version value. - */ - public String version() { - return this.version; - } - - /** - * Set the version property: Version for the component. - * - * @param version the version value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withVersion(String version) { - this.version = version; - return this; - } - - /** - * Get the license property: License for the component. - * - * @return the license value. - */ - public String license() { - return this.license; - } - - /** - * Set the license property: License for the component. - * - * @param license the license value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withLicense(String license) { - this.license = license; - return this; - } - - /** - * Get the releaseDate property: Release date for the component. - * - * @return the releaseDate value. - */ - public OffsetDateTime releaseDate() { - return this.releaseDate; - } - - /** - * Set the releaseDate property: Release date for the component. - * - * @param releaseDate the releaseDate value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withReleaseDate(OffsetDateTime releaseDate) { - this.releaseDate = releaseDate; - return this; - } - - /** - * Get the paths property: Paths of the component. - * - * @return the paths value. - */ - public List paths() { - return this.paths; - } - - /** - * Set the paths property: Paths of the component. - * - * @param paths the paths value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withPaths(List paths) { - this.paths = paths; - return this; - } - - /** - * Get the isUpdateAvailable property: Flag if new update is available for the component. - * - * @return the isUpdateAvailable value. - */ - public IsUpdateAvailable isUpdateAvailable() { - return this.isUpdateAvailable; - } - - /** - * Set the isUpdateAvailable property: Flag if new update is available for the component. - * - * @param isUpdateAvailable the isUpdateAvailable value to set. - * @return the ComponentInner object itself. - */ - public ComponentInner withIsUpdateAvailable(IsUpdateAvailable isUpdateAvailable) { - this.isUpdateAvailable = isUpdateAvailable; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificate.java similarity index 73% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificate.java index 9d8783501c8f1..996ccc9a5663f 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificate.java @@ -6,20 +6,18 @@ import com.azure.core.annotation.Fluent; import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateEntity; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsExpired; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsSelfSigned; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsShortKeySize; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsWeakSignature; import com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; import java.util.List; -/** Crypto certificate properties. */ +/** + * Crypto certificate properties. + */ @Fluent -public final class CryptoCertificateInner { +public final class CryptoCertificate { /* - * ID for the certificate. + * ID for the certificate result. */ @JsonProperty(value = "cryptoCertId") private String cryptoCertId; @@ -103,7 +101,7 @@ public final class CryptoCertificateInner { private List usage; /* - * List of files paths for this certificate + * List of files where this certificate was found. */ @JsonProperty(value = "filePaths", access = JsonProperty.Access.WRITE_ONLY) private List filePaths; @@ -118,33 +116,35 @@ public final class CryptoCertificateInner { * Indicates if the certificate is expired. */ @JsonProperty(value = "isExpired") - private IsExpired isExpired; + private Boolean isExpired; /* - * Indicates if the certificate was self-signed. + * Indicates if the certificate is self-signed. */ @JsonProperty(value = "isSelfSigned") - private IsSelfSigned isSelfSigned; + private Boolean isSelfSigned; /* * Indicates the signature algorithm used is insecure. */ @JsonProperty(value = "isWeakSignature") - private IsWeakSignature isWeakSignature; + private Boolean isWeakSignature; /* * Indicates the certificate's key size is considered too small to be secure for the key algorithm. */ @JsonProperty(value = "isShortKeySize") - private IsShortKeySize isShortKeySize; + private Boolean isShortKeySize; - /** Creates an instance of CryptoCertificateInner class. */ - public CryptoCertificateInner() { + /** + * Creates an instance of CryptoCertificate class. + */ + public CryptoCertificate() { } /** - * Get the cryptoCertId property: ID for the certificate. - * + * Get the cryptoCertId property: ID for the certificate result. + * * @return the cryptoCertId value. */ public String cryptoCertId() { @@ -152,19 +152,19 @@ public String cryptoCertId() { } /** - * Set the cryptoCertId property: ID for the certificate. - * + * Set the cryptoCertId property: ID for the certificate result. + * * @param cryptoCertId the cryptoCertId value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withCryptoCertId(String cryptoCertId) { + public CryptoCertificate withCryptoCertId(String cryptoCertId) { this.cryptoCertId = cryptoCertId; return this; } /** * Get the name property: Name of the certificate. - * + * * @return the name value. */ public String name() { @@ -173,18 +173,18 @@ public String name() { /** * Set the name property: Name of the certificate. - * + * * @param name the name value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withName(String name) { + public CryptoCertificate withName(String name) { this.name = name; return this; } /** * Get the subject property: Subject information of the certificate. - * + * * @return the subject value. */ public CryptoCertificateEntity subject() { @@ -193,18 +193,18 @@ public CryptoCertificateEntity subject() { /** * Set the subject property: Subject information of the certificate. - * + * * @param subject the subject value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withSubject(CryptoCertificateEntity subject) { + public CryptoCertificate withSubject(CryptoCertificateEntity subject) { this.subject = subject; return this; } /** * Get the issuer property: Issuer information of the certificate. - * + * * @return the issuer value. */ public CryptoCertificateEntity issuer() { @@ -213,18 +213,18 @@ public CryptoCertificateEntity issuer() { /** * Set the issuer property: Issuer information of the certificate. - * + * * @param issuer the issuer value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withIssuer(CryptoCertificateEntity issuer) { + public CryptoCertificate withIssuer(CryptoCertificateEntity issuer) { this.issuer = issuer; return this; } /** * Get the issuedDate property: Issue date for the certificate. - * + * * @return the issuedDate value. */ public OffsetDateTime issuedDate() { @@ -233,18 +233,18 @@ public OffsetDateTime issuedDate() { /** * Set the issuedDate property: Issue date for the certificate. - * + * * @param issuedDate the issuedDate value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withIssuedDate(OffsetDateTime issuedDate) { + public CryptoCertificate withIssuedDate(OffsetDateTime issuedDate) { this.issuedDate = issuedDate; return this; } /** * Get the expirationDate property: Expiration date for the certificate. - * + * * @return the expirationDate value. */ public OffsetDateTime expirationDate() { @@ -253,18 +253,18 @@ public OffsetDateTime expirationDate() { /** * Set the expirationDate property: Expiration date for the certificate. - * + * * @param expirationDate the expirationDate value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withExpirationDate(OffsetDateTime expirationDate) { + public CryptoCertificate withExpirationDate(OffsetDateTime expirationDate) { this.expirationDate = expirationDate; return this; } /** * Get the role property: Role of the certificate (Root CA, etc). - * + * * @return the role value. */ public String role() { @@ -273,18 +273,18 @@ public String role() { /** * Set the role property: Role of the certificate (Root CA, etc). - * + * * @param role the role value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withRole(String role) { + public CryptoCertificate withRole(String role) { this.role = role; return this; } /** * Get the signatureAlgorithm property: The signature algorithm used in the certificate. - * + * * @return the signatureAlgorithm value. */ public String signatureAlgorithm() { @@ -293,18 +293,18 @@ public String signatureAlgorithm() { /** * Set the signatureAlgorithm property: The signature algorithm used in the certificate. - * + * * @param signatureAlgorithm the signatureAlgorithm value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withSignatureAlgorithm(String signatureAlgorithm) { + public CryptoCertificate withSignatureAlgorithm(String signatureAlgorithm) { this.signatureAlgorithm = signatureAlgorithm; return this; } /** * Get the keySize property: Size of the certificate's key in bits. - * + * * @return the keySize value. */ public Long keySize() { @@ -313,18 +313,18 @@ public Long keySize() { /** * Set the keySize property: Size of the certificate's key in bits. - * + * * @param keySize the keySize value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withKeySize(Long keySize) { + public CryptoCertificate withKeySize(Long keySize) { this.keySize = keySize; return this; } /** * Get the keyAlgorithm property: Key algorithm used in the certificate. - * + * * @return the keyAlgorithm value. */ public String keyAlgorithm() { @@ -333,18 +333,18 @@ public String keyAlgorithm() { /** * Set the keyAlgorithm property: Key algorithm used in the certificate. - * + * * @param keyAlgorithm the keyAlgorithm value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withKeyAlgorithm(String keyAlgorithm) { + public CryptoCertificate withKeyAlgorithm(String keyAlgorithm) { this.keyAlgorithm = keyAlgorithm; return this; } /** * Get the encoding property: Encoding used for the certificate. - * + * * @return the encoding value. */ public String encoding() { @@ -353,18 +353,18 @@ public String encoding() { /** * Set the encoding property: Encoding used for the certificate. - * + * * @param encoding the encoding value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withEncoding(String encoding) { + public CryptoCertificate withEncoding(String encoding) { this.encoding = encoding; return this; } /** * Get the serialNumber property: Serial number of the certificate. - * + * * @return the serialNumber value. */ public String serialNumber() { @@ -373,18 +373,18 @@ public String serialNumber() { /** * Set the serialNumber property: Serial number of the certificate. - * + * * @param serialNumber the serialNumber value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withSerialNumber(String serialNumber) { + public CryptoCertificate withSerialNumber(String serialNumber) { this.serialNumber = serialNumber; return this; } /** * Get the fingerprint property: Fingerprint of the certificate. - * + * * @return the fingerprint value. */ public String fingerprint() { @@ -393,18 +393,18 @@ public String fingerprint() { /** * Set the fingerprint property: Fingerprint of the certificate. - * + * * @param fingerprint the fingerprint value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withFingerprint(String fingerprint) { + public CryptoCertificate withFingerprint(String fingerprint) { this.fingerprint = fingerprint; return this; } /** * Get the usage property: List of functions the certificate can fulfill. - * + * * @return the usage value. */ public List usage() { @@ -413,18 +413,18 @@ public List usage() { /** * Set the usage property: List of functions the certificate can fulfill. - * + * * @param usage the usage value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withUsage(List usage) { + public CryptoCertificate withUsage(List usage) { this.usage = usage; return this; } /** - * Get the filePaths property: List of files paths for this certificate. - * + * Get the filePaths property: List of files where this certificate was found. + * * @return the filePaths value. */ public List filePaths() { @@ -433,7 +433,7 @@ public List filePaths() { /** * Get the pairedKey property: A matching paired private key. - * + * * @return the pairedKey value. */ public PairedKey pairedKey() { @@ -442,71 +442,71 @@ public PairedKey pairedKey() { /** * Set the pairedKey property: A matching paired private key. - * + * * @param pairedKey the pairedKey value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withPairedKey(PairedKey pairedKey) { + public CryptoCertificate withPairedKey(PairedKey pairedKey) { this.pairedKey = pairedKey; return this; } /** * Get the isExpired property: Indicates if the certificate is expired. - * + * * @return the isExpired value. */ - public IsExpired isExpired() { + public Boolean isExpired() { return this.isExpired; } /** * Set the isExpired property: Indicates if the certificate is expired. - * + * * @param isExpired the isExpired value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withIsExpired(IsExpired isExpired) { + public CryptoCertificate withIsExpired(Boolean isExpired) { this.isExpired = isExpired; return this; } /** - * Get the isSelfSigned property: Indicates if the certificate was self-signed. - * + * Get the isSelfSigned property: Indicates if the certificate is self-signed. + * * @return the isSelfSigned value. */ - public IsSelfSigned isSelfSigned() { + public Boolean isSelfSigned() { return this.isSelfSigned; } /** - * Set the isSelfSigned property: Indicates if the certificate was self-signed. - * + * Set the isSelfSigned property: Indicates if the certificate is self-signed. + * * @param isSelfSigned the isSelfSigned value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withIsSelfSigned(IsSelfSigned isSelfSigned) { + public CryptoCertificate withIsSelfSigned(Boolean isSelfSigned) { this.isSelfSigned = isSelfSigned; return this; } /** * Get the isWeakSignature property: Indicates the signature algorithm used is insecure. - * + * * @return the isWeakSignature value. */ - public IsWeakSignature isWeakSignature() { + public Boolean isWeakSignature() { return this.isWeakSignature; } /** * Set the isWeakSignature property: Indicates the signature algorithm used is insecure. - * + * * @param isWeakSignature the isWeakSignature value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withIsWeakSignature(IsWeakSignature isWeakSignature) { + public CryptoCertificate withIsWeakSignature(Boolean isWeakSignature) { this.isWeakSignature = isWeakSignature; return this; } @@ -514,28 +514,28 @@ public CryptoCertificateInner withIsWeakSignature(IsWeakSignature isWeakSignatur /** * Get the isShortKeySize property: Indicates the certificate's key size is considered too small to be secure for * the key algorithm. - * + * * @return the isShortKeySize value. */ - public IsShortKeySize isShortKeySize() { + public Boolean isShortKeySize() { return this.isShortKeySize; } /** * Set the isShortKeySize property: Indicates the certificate's key size is considered too small to be secure for * the key algorithm. - * + * * @param isShortKeySize the isShortKeySize value to set. - * @return the CryptoCertificateInner object itself. + * @return the CryptoCertificate object itself. */ - public CryptoCertificateInner withIsShortKeySize(IsShortKeySize isShortKeySize) { + public CryptoCertificate withIsShortKeySize(Boolean isShortKeySize) { this.isShortKeySize = isShortKeySize; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateResourceInner.java new file mode 100644 index 0000000000000..2b1d5af950e3f --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateResourceInner.java @@ -0,0 +1,515 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateEntity; +import com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; +import java.util.List; + +/** + * Crypto certificate resource. + */ +@Fluent +public final class CryptoCertificateResourceInner extends ProxyResource { + /* + * The properties of a crypto certificate found within a firmware image + */ + @JsonProperty(value = "properties") + private CryptoCertificate innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of CryptoCertificateResourceInner class. + */ + public CryptoCertificateResourceInner() { + } + + /** + * Get the innerProperties property: The properties of a crypto certificate found within a firmware image. + * + * @return the innerProperties value. + */ + private CryptoCertificate innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the cryptoCertId property: ID for the certificate result. + * + * @return the cryptoCertId value. + */ + public String cryptoCertId() { + return this.innerProperties() == null ? null : this.innerProperties().cryptoCertId(); + } + + /** + * Set the cryptoCertId property: ID for the certificate result. + * + * @param cryptoCertId the cryptoCertId value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withCryptoCertId(String cryptoCertId) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withCryptoCertId(cryptoCertId); + return this; + } + + /** + * Get the name property: Name of the certificate. + * + * @return the name value. + */ + public String namePropertiesName() { + return this.innerProperties() == null ? null : this.innerProperties().name(); + } + + /** + * Set the name property: Name of the certificate. + * + * @param name the name value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withNamePropertiesName(String name) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withName(name); + return this; + } + + /** + * Get the subject property: Subject information of the certificate. + * + * @return the subject value. + */ + public CryptoCertificateEntity subject() { + return this.innerProperties() == null ? null : this.innerProperties().subject(); + } + + /** + * Set the subject property: Subject information of the certificate. + * + * @param subject the subject value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withSubject(CryptoCertificateEntity subject) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withSubject(subject); + return this; + } + + /** + * Get the issuer property: Issuer information of the certificate. + * + * @return the issuer value. + */ + public CryptoCertificateEntity issuer() { + return this.innerProperties() == null ? null : this.innerProperties().issuer(); + } + + /** + * Set the issuer property: Issuer information of the certificate. + * + * @param issuer the issuer value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withIssuer(CryptoCertificateEntity issuer) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withIssuer(issuer); + return this; + } + + /** + * Get the issuedDate property: Issue date for the certificate. + * + * @return the issuedDate value. + */ + public OffsetDateTime issuedDate() { + return this.innerProperties() == null ? null : this.innerProperties().issuedDate(); + } + + /** + * Set the issuedDate property: Issue date for the certificate. + * + * @param issuedDate the issuedDate value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withIssuedDate(OffsetDateTime issuedDate) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withIssuedDate(issuedDate); + return this; + } + + /** + * Get the expirationDate property: Expiration date for the certificate. + * + * @return the expirationDate value. + */ + public OffsetDateTime expirationDate() { + return this.innerProperties() == null ? null : this.innerProperties().expirationDate(); + } + + /** + * Set the expirationDate property: Expiration date for the certificate. + * + * @param expirationDate the expirationDate value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withExpirationDate(OffsetDateTime expirationDate) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withExpirationDate(expirationDate); + return this; + } + + /** + * Get the role property: Role of the certificate (Root CA, etc). + * + * @return the role value. + */ + public String role() { + return this.innerProperties() == null ? null : this.innerProperties().role(); + } + + /** + * Set the role property: Role of the certificate (Root CA, etc). + * + * @param role the role value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withRole(String role) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withRole(role); + return this; + } + + /** + * Get the signatureAlgorithm property: The signature algorithm used in the certificate. + * + * @return the signatureAlgorithm value. + */ + public String signatureAlgorithm() { + return this.innerProperties() == null ? null : this.innerProperties().signatureAlgorithm(); + } + + /** + * Set the signatureAlgorithm property: The signature algorithm used in the certificate. + * + * @param signatureAlgorithm the signatureAlgorithm value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withSignatureAlgorithm(String signatureAlgorithm) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withSignatureAlgorithm(signatureAlgorithm); + return this; + } + + /** + * Get the keySize property: Size of the certificate's key in bits. + * + * @return the keySize value. + */ + public Long keySize() { + return this.innerProperties() == null ? null : this.innerProperties().keySize(); + } + + /** + * Set the keySize property: Size of the certificate's key in bits. + * + * @param keySize the keySize value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withKeySize(Long keySize) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withKeySize(keySize); + return this; + } + + /** + * Get the keyAlgorithm property: Key algorithm used in the certificate. + * + * @return the keyAlgorithm value. + */ + public String keyAlgorithm() { + return this.innerProperties() == null ? null : this.innerProperties().keyAlgorithm(); + } + + /** + * Set the keyAlgorithm property: Key algorithm used in the certificate. + * + * @param keyAlgorithm the keyAlgorithm value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withKeyAlgorithm(String keyAlgorithm) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withKeyAlgorithm(keyAlgorithm); + return this; + } + + /** + * Get the encoding property: Encoding used for the certificate. + * + * @return the encoding value. + */ + public String encoding() { + return this.innerProperties() == null ? null : this.innerProperties().encoding(); + } + + /** + * Set the encoding property: Encoding used for the certificate. + * + * @param encoding the encoding value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withEncoding(String encoding) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withEncoding(encoding); + return this; + } + + /** + * Get the serialNumber property: Serial number of the certificate. + * + * @return the serialNumber value. + */ + public String serialNumber() { + return this.innerProperties() == null ? null : this.innerProperties().serialNumber(); + } + + /** + * Set the serialNumber property: Serial number of the certificate. + * + * @param serialNumber the serialNumber value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withSerialNumber(String serialNumber) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withSerialNumber(serialNumber); + return this; + } + + /** + * Get the fingerprint property: Fingerprint of the certificate. + * + * @return the fingerprint value. + */ + public String fingerprint() { + return this.innerProperties() == null ? null : this.innerProperties().fingerprint(); + } + + /** + * Set the fingerprint property: Fingerprint of the certificate. + * + * @param fingerprint the fingerprint value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withFingerprint(String fingerprint) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withFingerprint(fingerprint); + return this; + } + + /** + * Get the usage property: List of functions the certificate can fulfill. + * + * @return the usage value. + */ + public List usage() { + return this.innerProperties() == null ? null : this.innerProperties().usage(); + } + + /** + * Set the usage property: List of functions the certificate can fulfill. + * + * @param usage the usage value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withUsage(List usage) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withUsage(usage); + return this; + } + + /** + * Get the filePaths property: List of files where this certificate was found. + * + * @return the filePaths value. + */ + public List filePaths() { + return this.innerProperties() == null ? null : this.innerProperties().filePaths(); + } + + /** + * Get the pairedKey property: A matching paired private key. + * + * @return the pairedKey value. + */ + public PairedKey pairedKey() { + return this.innerProperties() == null ? null : this.innerProperties().pairedKey(); + } + + /** + * Set the pairedKey property: A matching paired private key. + * + * @param pairedKey the pairedKey value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withPairedKey(PairedKey pairedKey) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withPairedKey(pairedKey); + return this; + } + + /** + * Get the isExpired property: Indicates if the certificate is expired. + * + * @return the isExpired value. + */ + public Boolean isExpired() { + return this.innerProperties() == null ? null : this.innerProperties().isExpired(); + } + + /** + * Set the isExpired property: Indicates if the certificate is expired. + * + * @param isExpired the isExpired value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withIsExpired(Boolean isExpired) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withIsExpired(isExpired); + return this; + } + + /** + * Get the isSelfSigned property: Indicates if the certificate is self-signed. + * + * @return the isSelfSigned value. + */ + public Boolean isSelfSigned() { + return this.innerProperties() == null ? null : this.innerProperties().isSelfSigned(); + } + + /** + * Set the isSelfSigned property: Indicates if the certificate is self-signed. + * + * @param isSelfSigned the isSelfSigned value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withIsSelfSigned(Boolean isSelfSigned) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withIsSelfSigned(isSelfSigned); + return this; + } + + /** + * Get the isWeakSignature property: Indicates the signature algorithm used is insecure. + * + * @return the isWeakSignature value. + */ + public Boolean isWeakSignature() { + return this.innerProperties() == null ? null : this.innerProperties().isWeakSignature(); + } + + /** + * Set the isWeakSignature property: Indicates the signature algorithm used is insecure. + * + * @param isWeakSignature the isWeakSignature value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withIsWeakSignature(Boolean isWeakSignature) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withIsWeakSignature(isWeakSignature); + return this; + } + + /** + * Get the isShortKeySize property: Indicates the certificate's key size is considered too small to be secure for + * the key algorithm. + * + * @return the isShortKeySize value. + */ + public Boolean isShortKeySize() { + return this.innerProperties() == null ? null : this.innerProperties().isShortKeySize(); + } + + /** + * Set the isShortKeySize property: Indicates the certificate's key size is considered too small to be secure for + * the key algorithm. + * + * @param isShortKeySize the isShortKeySize value to set. + * @return the CryptoCertificateResourceInner object itself. + */ + public CryptoCertificateResourceInner withIsShortKeySize(Boolean isShortKeySize) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoCertificate(); + } + this.innerProperties().withIsShortKeySize(isShortKeySize); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKey.java similarity index 75% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKey.java index 63eebaee2a8b5..c29fae1a42fdf 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKey.java @@ -5,16 +5,17 @@ package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsShortKeySize; import com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Crypto key properties. */ +/** + * Crypto key properties. + */ @Fluent -public final class CryptoKeyInner { +public final class CryptoKey { /* - * ID for the key. + * ID for the key result. */ @JsonProperty(value = "cryptoKeyId") private String cryptoKeyId; @@ -44,7 +45,7 @@ public final class CryptoKeyInner { private List usage; /* - * List of files paths for this key. + * List of files where this key was found. */ @JsonProperty(value = "filePaths", access = JsonProperty.Access.WRITE_ONLY) private List filePaths; @@ -59,15 +60,17 @@ public final class CryptoKeyInner { * Indicates the key size is considered too small to be secure for the algorithm. */ @JsonProperty(value = "isShortKeySize") - private IsShortKeySize isShortKeySize; + private Boolean isShortKeySize; - /** Creates an instance of CryptoKeyInner class. */ - public CryptoKeyInner() { + /** + * Creates an instance of CryptoKey class. + */ + public CryptoKey() { } /** - * Get the cryptoKeyId property: ID for the key. - * + * Get the cryptoKeyId property: ID for the key result. + * * @return the cryptoKeyId value. */ public String cryptoKeyId() { @@ -75,19 +78,19 @@ public String cryptoKeyId() { } /** - * Set the cryptoKeyId property: ID for the key. - * + * Set the cryptoKeyId property: ID for the key result. + * * @param cryptoKeyId the cryptoKeyId value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withCryptoKeyId(String cryptoKeyId) { + public CryptoKey withCryptoKeyId(String cryptoKeyId) { this.cryptoKeyId = cryptoKeyId; return this; } /** * Get the keyType property: Type of the key (public or private). - * + * * @return the keyType value. */ public String keyType() { @@ -96,18 +99,18 @@ public String keyType() { /** * Set the keyType property: Type of the key (public or private). - * + * * @param keyType the keyType value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withKeyType(String keyType) { + public CryptoKey withKeyType(String keyType) { this.keyType = keyType; return this; } /** * Get the keySize property: Size of the key in bits. - * + * * @return the keySize value. */ public Long keySize() { @@ -116,18 +119,18 @@ public Long keySize() { /** * Set the keySize property: Size of the key in bits. - * + * * @param keySize the keySize value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withKeySize(Long keySize) { + public CryptoKey withKeySize(Long keySize) { this.keySize = keySize; return this; } /** * Get the keyAlgorithm property: Key algorithm name. - * + * * @return the keyAlgorithm value. */ public String keyAlgorithm() { @@ -136,18 +139,18 @@ public String keyAlgorithm() { /** * Set the keyAlgorithm property: Key algorithm name. - * + * * @param keyAlgorithm the keyAlgorithm value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withKeyAlgorithm(String keyAlgorithm) { + public CryptoKey withKeyAlgorithm(String keyAlgorithm) { this.keyAlgorithm = keyAlgorithm; return this; } /** * Get the usage property: Functions the key can fulfill. - * + * * @return the usage value. */ public List usage() { @@ -156,18 +159,18 @@ public List usage() { /** * Set the usage property: Functions the key can fulfill. - * + * * @param usage the usage value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withUsage(List usage) { + public CryptoKey withUsage(List usage) { this.usage = usage; return this; } /** - * Get the filePaths property: List of files paths for this key. - * + * Get the filePaths property: List of files where this key was found. + * * @return the filePaths value. */ public List filePaths() { @@ -176,7 +179,7 @@ public List filePaths() { /** * Get the pairedKey property: A matching paired key or certificate. - * + * * @return the pairedKey value. */ public PairedKey pairedKey() { @@ -185,38 +188,38 @@ public PairedKey pairedKey() { /** * Set the pairedKey property: A matching paired key or certificate. - * + * * @param pairedKey the pairedKey value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withPairedKey(PairedKey pairedKey) { + public CryptoKey withPairedKey(PairedKey pairedKey) { this.pairedKey = pairedKey; return this; } /** * Get the isShortKeySize property: Indicates the key size is considered too small to be secure for the algorithm. - * + * * @return the isShortKeySize value. */ - public IsShortKeySize isShortKeySize() { + public Boolean isShortKeySize() { return this.isShortKeySize; } /** * Set the isShortKeySize property: Indicates the key size is considered too small to be secure for the algorithm. - * + * * @param isShortKeySize the isShortKeySize value to set. - * @return the CryptoKeyInner object itself. + * @return the CryptoKey object itself. */ - public CryptoKeyInner withIsShortKeySize(IsShortKeySize isShortKeySize) { + public CryptoKey withIsShortKeySize(Boolean isShortKeySize) { this.isShortKeySize = isShortKeySize; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyResourceInner.java new file mode 100644 index 0000000000000..723308e310454 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeyResourceInner.java @@ -0,0 +1,235 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Crypto key resource. + */ +@Fluent +public final class CryptoKeyResourceInner extends ProxyResource { + /* + * The properties of a crypto key found within a firmware image + */ + @JsonProperty(value = "properties") + private CryptoKey innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of CryptoKeyResourceInner class. + */ + public CryptoKeyResourceInner() { + } + + /** + * Get the innerProperties property: The properties of a crypto key found within a firmware image. + * + * @return the innerProperties value. + */ + private CryptoKey innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the cryptoKeyId property: ID for the key result. + * + * @return the cryptoKeyId value. + */ + public String cryptoKeyId() { + return this.innerProperties() == null ? null : this.innerProperties().cryptoKeyId(); + } + + /** + * Set the cryptoKeyId property: ID for the key result. + * + * @param cryptoKeyId the cryptoKeyId value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withCryptoKeyId(String cryptoKeyId) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withCryptoKeyId(cryptoKeyId); + return this; + } + + /** + * Get the keyType property: Type of the key (public or private). + * + * @return the keyType value. + */ + public String keyType() { + return this.innerProperties() == null ? null : this.innerProperties().keyType(); + } + + /** + * Set the keyType property: Type of the key (public or private). + * + * @param keyType the keyType value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withKeyType(String keyType) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withKeyType(keyType); + return this; + } + + /** + * Get the keySize property: Size of the key in bits. + * + * @return the keySize value. + */ + public Long keySize() { + return this.innerProperties() == null ? null : this.innerProperties().keySize(); + } + + /** + * Set the keySize property: Size of the key in bits. + * + * @param keySize the keySize value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withKeySize(Long keySize) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withKeySize(keySize); + return this; + } + + /** + * Get the keyAlgorithm property: Key algorithm name. + * + * @return the keyAlgorithm value. + */ + public String keyAlgorithm() { + return this.innerProperties() == null ? null : this.innerProperties().keyAlgorithm(); + } + + /** + * Set the keyAlgorithm property: Key algorithm name. + * + * @param keyAlgorithm the keyAlgorithm value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withKeyAlgorithm(String keyAlgorithm) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withKeyAlgorithm(keyAlgorithm); + return this; + } + + /** + * Get the usage property: Functions the key can fulfill. + * + * @return the usage value. + */ + public List usage() { + return this.innerProperties() == null ? null : this.innerProperties().usage(); + } + + /** + * Set the usage property: Functions the key can fulfill. + * + * @param usage the usage value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withUsage(List usage) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withUsage(usage); + return this; + } + + /** + * Get the filePaths property: List of files where this key was found. + * + * @return the filePaths value. + */ + public List filePaths() { + return this.innerProperties() == null ? null : this.innerProperties().filePaths(); + } + + /** + * Get the pairedKey property: A matching paired key or certificate. + * + * @return the pairedKey value. + */ + public PairedKey pairedKey() { + return this.innerProperties() == null ? null : this.innerProperties().pairedKey(); + } + + /** + * Set the pairedKey property: A matching paired key or certificate. + * + * @param pairedKey the pairedKey value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withPairedKey(PairedKey pairedKey) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withPairedKey(pairedKey); + return this; + } + + /** + * Get the isShortKeySize property: Indicates the key size is considered too small to be secure for the algorithm. + * + * @return the isShortKeySize value. + */ + public Boolean isShortKeySize() { + return this.innerProperties() == null ? null : this.innerProperties().isShortKeySize(); + } + + /** + * Set the isShortKeySize property: Indicates the key size is considered too small to be secure for the algorithm. + * + * @param isShortKeySize the isShortKeySize value to set. + * @return the CryptoKeyResourceInner object itself. + */ + public CryptoKeyResourceInner withIsShortKeySize(Boolean isShortKeySize) { + if (this.innerProperties() == null) { + this.innerProperties = new CryptoKey(); + } + this.innerProperties().withIsShortKeySize(isShortKeySize); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResourceInner.java new file mode 100644 index 0000000000000..a062d632d7d75 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResourceInner.java @@ -0,0 +1,284 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveLink; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * CVE analysis result resource. + */ +@Fluent +public final class CveResourceInner extends ProxyResource { + /* + * The properties of a CVE result found within a firmware image + */ + @JsonProperty(value = "properties") + private CveResult innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of CveResourceInner class. + */ + public CveResourceInner() { + } + + /** + * Get the innerProperties property: The properties of a CVE result found within a firmware image. + * + * @return the innerProperties value. + */ + private CveResult innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the cveId property: ID of the CVE result. + * + * @return the cveId value. + */ + public String cveId() { + return this.innerProperties() == null ? null : this.innerProperties().cveId(); + } + + /** + * Set the cveId property: ID of the CVE result. + * + * @param cveId the cveId value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withCveId(String cveId) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withCveId(cveId); + return this; + } + + /** + * Get the component property: The SBOM component for the CVE. + * + * @return the component value. + */ + public CveComponent component() { + return this.innerProperties() == null ? null : this.innerProperties().component(); + } + + /** + * Set the component property: The SBOM component for the CVE. + * + * @param component the component value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withComponent(CveComponent component) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withComponent(component); + return this; + } + + /** + * Get the severity property: Severity of the CVE. + * + * @return the severity value. + */ + public String severity() { + return this.innerProperties() == null ? null : this.innerProperties().severity(); + } + + /** + * Set the severity property: Severity of the CVE. + * + * @param severity the severity value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withSeverity(String severity) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withSeverity(severity); + return this; + } + + /** + * Get the name property: Name of the CVE. + * + * @return the name value. + */ + public String namePropertiesName() { + return this.innerProperties() == null ? null : this.innerProperties().name(); + } + + /** + * Set the name property: Name of the CVE. + * + * @param name the name value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withNamePropertiesName(String name) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withName(name); + return this; + } + + /** + * Get the cvssScore property: A single CVSS score to represent the CVE. If a V3 score is specified, then it will + * use the V3 score. Otherwise if the V2 score is specified it will be the V2 score. + * + * @return the cvssScore value. + */ + public String cvssScore() { + return this.innerProperties() == null ? null : this.innerProperties().cvssScore(); + } + + /** + * Set the cvssScore property: A single CVSS score to represent the CVE. If a V3 score is specified, then it will + * use the V3 score. Otherwise if the V2 score is specified it will be the V2 score. + * + * @param cvssScore the cvssScore value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withCvssScore(String cvssScore) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withCvssScore(cvssScore); + return this; + } + + /** + * Get the cvssVersion property: CVSS version of the CVE. + * + * @return the cvssVersion value. + */ + public String cvssVersion() { + return this.innerProperties() == null ? null : this.innerProperties().cvssVersion(); + } + + /** + * Set the cvssVersion property: CVSS version of the CVE. + * + * @param cvssVersion the cvssVersion value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withCvssVersion(String cvssVersion) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withCvssVersion(cvssVersion); + return this; + } + + /** + * Get the cvssV2Score property: CVSS V2 score of the CVE. + * + * @return the cvssV2Score value. + */ + public String cvssV2Score() { + return this.innerProperties() == null ? null : this.innerProperties().cvssV2Score(); + } + + /** + * Set the cvssV2Score property: CVSS V2 score of the CVE. + * + * @param cvssV2Score the cvssV2Score value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withCvssV2Score(String cvssV2Score) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withCvssV2Score(cvssV2Score); + return this; + } + + /** + * Get the cvssV3Score property: CVSS V3 score of the CVE. + * + * @return the cvssV3Score value. + */ + public String cvssV3Score() { + return this.innerProperties() == null ? null : this.innerProperties().cvssV3Score(); + } + + /** + * Set the cvssV3Score property: CVSS V3 score of the CVE. + * + * @param cvssV3Score the cvssV3Score value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withCvssV3Score(String cvssV3Score) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withCvssV3Score(cvssV3Score); + return this; + } + + /** + * Get the links property: The list of reference links for the CVE. + * + * @return the links value. + */ + public List links() { + return this.innerProperties() == null ? null : this.innerProperties().links(); + } + + /** + * Get the description property: The CVE description. + * + * @return the description value. + */ + public String description() { + return this.innerProperties() == null ? null : this.innerProperties().description(); + } + + /** + * Set the description property: The CVE description. + * + * @param description the description value to set. + * @return the CveResourceInner object itself. + */ + public CveResourceInner withDescription(String description) { + if (this.innerProperties() == null) { + this.innerProperties = new CveResult(); + } + this.innerProperties().withDescription(description); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResult.java similarity index 54% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResult.java index 37fe0eaf8be7c..65e29e3421a0d 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveResult.java @@ -5,94 +5,86 @@ package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent; import com.azure.resourcemanager.iotfirmwaredefense.models.CveLink; import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; import java.util.List; -/** Known CVEs of a firmware. */ +/** + * Details of a CVE detected in firmware. + */ @Fluent -public final class CveInner { +public final class CveResult { /* - * ID of CVE + * ID of the CVE result. */ @JsonProperty(value = "cveId") private String cveId; /* - * Component of CVE + * The SBOM component for the CVE. */ @JsonProperty(value = "component") - private Object component; + private CveComponent component; /* - * Severity of CVE + * Severity of the CVE. */ @JsonProperty(value = "severity") private String severity; /* - * Name of CVE + * Name of the CVE. */ @JsonProperty(value = "name") private String name; /* * A single CVSS score to represent the CVE. If a V3 score is specified, then it will use the V3 score. Otherwise - * if the V2 score is specified it will be the V2 score + * if the V2 score is specified it will be the V2 score. */ @JsonProperty(value = "cvssScore") private String cvssScore; /* - * Cvss version of CVE + * CVSS version of the CVE. */ @JsonProperty(value = "cvssVersion") private String cvssVersion; /* - * Cvss V2 score of CVE + * CVSS V2 score of the CVE. */ @JsonProperty(value = "cvssV2Score") private String cvssV2Score; /* - * Cvss V3 score of CVE + * CVSS V3 score of the CVE. */ @JsonProperty(value = "cvssV3Score") private String cvssV3Score; /* - * Publish date of CVE - */ - @JsonProperty(value = "publishDate") - private OffsetDateTime publishDate; - - /* - * Updated date of CVE - */ - @JsonProperty(value = "updatedDate") - private OffsetDateTime updatedDate; - - /* - * The list of CVE links. + * The list of reference links for the CVE. */ @JsonProperty(value = "links", access = JsonProperty.Access.WRITE_ONLY) private List links; /* - * Description of CVE + * The CVE description. */ @JsonProperty(value = "description") private String description; - /** Creates an instance of CveInner class. */ - public CveInner() { + /** + * Creates an instance of CveResult class. + */ + public CveResult() { } /** - * Get the cveId property: ID of CVE. - * + * Get the cveId property: ID of the CVE result. + * * @return the cveId value. */ public String cveId() { @@ -100,39 +92,39 @@ public String cveId() { } /** - * Set the cveId property: ID of CVE. - * + * Set the cveId property: ID of the CVE result. + * * @param cveId the cveId value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withCveId(String cveId) { + public CveResult withCveId(String cveId) { this.cveId = cveId; return this; } /** - * Get the component property: Component of CVE. - * + * Get the component property: The SBOM component for the CVE. + * * @return the component value. */ - public Object component() { + public CveComponent component() { return this.component; } /** - * Set the component property: Component of CVE. - * + * Set the component property: The SBOM component for the CVE. + * * @param component the component value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withComponent(Object component) { + public CveResult withComponent(CveComponent component) { this.component = component; return this; } /** - * Get the severity property: Severity of CVE. - * + * Get the severity property: Severity of the CVE. + * * @return the severity value. */ public String severity() { @@ -140,19 +132,19 @@ public String severity() { } /** - * Set the severity property: Severity of CVE. - * + * Set the severity property: Severity of the CVE. + * * @param severity the severity value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withSeverity(String severity) { + public CveResult withSeverity(String severity) { this.severity = severity; return this; } /** - * Get the name property: Name of CVE. - * + * Get the name property: Name of the CVE. + * * @return the name value. */ public String name() { @@ -160,12 +152,12 @@ public String name() { } /** - * Set the name property: Name of CVE. - * + * Set the name property: Name of the CVE. + * * @param name the name value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withName(String name) { + public CveResult withName(String name) { this.name = name; return this; } @@ -173,7 +165,7 @@ public CveInner withName(String name) { /** * Get the cvssScore property: A single CVSS score to represent the CVE. If a V3 score is specified, then it will * use the V3 score. Otherwise if the V2 score is specified it will be the V2 score. - * + * * @return the cvssScore value. */ public String cvssScore() { @@ -183,18 +175,18 @@ public String cvssScore() { /** * Set the cvssScore property: A single CVSS score to represent the CVE. If a V3 score is specified, then it will * use the V3 score. Otherwise if the V2 score is specified it will be the V2 score. - * + * * @param cvssScore the cvssScore value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withCvssScore(String cvssScore) { + public CveResult withCvssScore(String cvssScore) { this.cvssScore = cvssScore; return this; } /** - * Get the cvssVersion property: Cvss version of CVE. - * + * Get the cvssVersion property: CVSS version of the CVE. + * * @return the cvssVersion value. */ public String cvssVersion() { @@ -202,19 +194,19 @@ public String cvssVersion() { } /** - * Set the cvssVersion property: Cvss version of CVE. - * + * Set the cvssVersion property: CVSS version of the CVE. + * * @param cvssVersion the cvssVersion value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withCvssVersion(String cvssVersion) { + public CveResult withCvssVersion(String cvssVersion) { this.cvssVersion = cvssVersion; return this; } /** - * Get the cvssV2Score property: Cvss V2 score of CVE. - * + * Get the cvssV2Score property: CVSS V2 score of the CVE. + * * @return the cvssV2Score value. */ public String cvssV2Score() { @@ -222,19 +214,19 @@ public String cvssV2Score() { } /** - * Set the cvssV2Score property: Cvss V2 score of CVE. - * + * Set the cvssV2Score property: CVSS V2 score of the CVE. + * * @param cvssV2Score the cvssV2Score value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withCvssV2Score(String cvssV2Score) { + public CveResult withCvssV2Score(String cvssV2Score) { this.cvssV2Score = cvssV2Score; return this; } /** - * Get the cvssV3Score property: Cvss V3 score of CVE. - * + * Get the cvssV3Score property: CVSS V3 score of the CVE. + * * @return the cvssV3Score value. */ public String cvssV3Score() { @@ -242,59 +234,19 @@ public String cvssV3Score() { } /** - * Set the cvssV3Score property: Cvss V3 score of CVE. - * + * Set the cvssV3Score property: CVSS V3 score of the CVE. + * * @param cvssV3Score the cvssV3Score value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withCvssV3Score(String cvssV3Score) { + public CveResult withCvssV3Score(String cvssV3Score) { this.cvssV3Score = cvssV3Score; return this; } /** - * Get the publishDate property: Publish date of CVE. - * - * @return the publishDate value. - */ - public OffsetDateTime publishDate() { - return this.publishDate; - } - - /** - * Set the publishDate property: Publish date of CVE. - * - * @param publishDate the publishDate value to set. - * @return the CveInner object itself. - */ - public CveInner withPublishDate(OffsetDateTime publishDate) { - this.publishDate = publishDate; - return this; - } - - /** - * Get the updatedDate property: Updated date of CVE. - * - * @return the updatedDate value. - */ - public OffsetDateTime updatedDate() { - return this.updatedDate; - } - - /** - * Set the updatedDate property: Updated date of CVE. - * - * @param updatedDate the updatedDate value to set. - * @return the CveInner object itself. - */ - public CveInner withUpdatedDate(OffsetDateTime updatedDate) { - this.updatedDate = updatedDate; - return this; - } - - /** - * Get the links property: The list of CVE links. - * + * Get the links property: The list of reference links for the CVE. + * * @return the links value. */ public List links() { @@ -302,8 +254,8 @@ public List links() { } /** - * Get the description property: Description of CVE. - * + * Get the description property: The CVE description. + * * @return the description value. */ public String description() { @@ -311,22 +263,25 @@ public String description() { } /** - * Set the description property: Description of CVE. - * + * Set the description property: The CVE description. + * * @param description the description value to set. - * @return the CveInner object itself. + * @return the CveResult object itself. */ - public CveInner withDescription(String description) { + public CveResult withDescription(String description) { this.description = description; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (component() != null) { + component().validate(); + } if (links() != null) { links().forEach(e -> e.validate()); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveSummaryInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveSummaryInner.java deleted file mode 100644 index 7b30c7deb57e9..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CveSummaryInner.java +++ /dev/null @@ -1,180 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** CVE summary values. */ -@Fluent -public final class CveSummaryInner { - /* - * The total number of critical severity CVEs detected - */ - @JsonProperty(value = "critical") - private Long critical; - - /* - * The total number of high severity CVEs detected - */ - @JsonProperty(value = "high") - private Long high; - - /* - * The total number of medium severity CVEs detected - */ - @JsonProperty(value = "medium") - private Long medium; - - /* - * The total number of low severity CVEs detected - */ - @JsonProperty(value = "low") - private Long low; - - /* - * The total number of unknown severity CVEs detected - */ - @JsonProperty(value = "unknown") - private Long unknown; - - /* - * The total number of undefined severity CVEs detected - */ - @JsonProperty(value = "undefined") - private Long undefined; - - /** Creates an instance of CveSummaryInner class. */ - public CveSummaryInner() { - } - - /** - * Get the critical property: The total number of critical severity CVEs detected. - * - * @return the critical value. - */ - public Long critical() { - return this.critical; - } - - /** - * Set the critical property: The total number of critical severity CVEs detected. - * - * @param critical the critical value to set. - * @return the CveSummaryInner object itself. - */ - public CveSummaryInner withCritical(Long critical) { - this.critical = critical; - return this; - } - - /** - * Get the high property: The total number of high severity CVEs detected. - * - * @return the high value. - */ - public Long high() { - return this.high; - } - - /** - * Set the high property: The total number of high severity CVEs detected. - * - * @param high the high value to set. - * @return the CveSummaryInner object itself. - */ - public CveSummaryInner withHigh(Long high) { - this.high = high; - return this; - } - - /** - * Get the medium property: The total number of medium severity CVEs detected. - * - * @return the medium value. - */ - public Long medium() { - return this.medium; - } - - /** - * Set the medium property: The total number of medium severity CVEs detected. - * - * @param medium the medium value to set. - * @return the CveSummaryInner object itself. - */ - public CveSummaryInner withMedium(Long medium) { - this.medium = medium; - return this; - } - - /** - * Get the low property: The total number of low severity CVEs detected. - * - * @return the low value. - */ - public Long low() { - return this.low; - } - - /** - * Set the low property: The total number of low severity CVEs detected. - * - * @param low the low value to set. - * @return the CveSummaryInner object itself. - */ - public CveSummaryInner withLow(Long low) { - this.low = low; - return this; - } - - /** - * Get the unknown property: The total number of unknown severity CVEs detected. - * - * @return the unknown value. - */ - public Long unknown() { - return this.unknown; - } - - /** - * Set the unknown property: The total number of unknown severity CVEs detected. - * - * @param unknown the unknown value to set. - * @return the CveSummaryInner object itself. - */ - public CveSummaryInner withUnknown(Long unknown) { - this.unknown = unknown; - return this; - } - - /** - * Get the undefined property: The total number of undefined severity CVEs detected. - * - * @return the undefined value. - */ - public Long undefined() { - return this.undefined; - } - - /** - * Set the undefined property: The total number of undefined severity CVEs detected. - * - * @param undefined the undefined value to set. - * @return the CveSummaryInner object itself. - */ - public CveSummaryInner withUndefined(Long undefined) { - this.undefined = undefined; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareInner.java index 7377f620d360a..eea703ddf639a 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareInner.java @@ -9,10 +9,13 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.iotfirmwaredefense.models.ProvisioningState; import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Firmware definition. */ +/** + * Firmware definition. + */ @Fluent public final class FirmwareInner extends ProxyResource { /* @@ -27,13 +30,15 @@ public final class FirmwareInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of FirmwareInner class. */ + /** + * Creates an instance of FirmwareInner class. + */ public FirmwareInner() { } /** * Get the innerProperties property: The properties of a firmware. - * + * * @return the innerProperties value. */ private FirmwareProperties innerProperties() { @@ -42,7 +47,7 @@ private FirmwareProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -51,7 +56,7 @@ public SystemData systemData() { /** * Get the fileName property: File name for a firmware that user uploaded. - * + * * @return the fileName value. */ public String fileName() { @@ -60,7 +65,7 @@ public String fileName() { /** * Set the fileName property: File name for a firmware that user uploaded. - * + * * @param fileName the fileName value to set. * @return the FirmwareInner object itself. */ @@ -74,7 +79,7 @@ public FirmwareInner withFileName(String fileName) { /** * Get the vendor property: Firmware vendor. - * + * * @return the vendor value. */ public String vendor() { @@ -83,7 +88,7 @@ public String vendor() { /** * Set the vendor property: Firmware vendor. - * + * * @param vendor the vendor value to set. * @return the FirmwareInner object itself. */ @@ -97,7 +102,7 @@ public FirmwareInner withVendor(String vendor) { /** * Get the model property: Firmware model. - * + * * @return the model value. */ public String model() { @@ -106,7 +111,7 @@ public String model() { /** * Set the model property: Firmware model. - * + * * @param model the model value to set. * @return the FirmwareInner object itself. */ @@ -120,7 +125,7 @@ public FirmwareInner withModel(String model) { /** * Get the version property: Firmware version. - * + * * @return the version value. */ public String version() { @@ -129,7 +134,7 @@ public String version() { /** * Set the version property: Firmware version. - * + * * @param version the version value to set. * @return the FirmwareInner object itself. */ @@ -143,7 +148,7 @@ public FirmwareInner withVersion(String version) { /** * Get the description property: User-specified description of the firmware. - * + * * @return the description value. */ public String description() { @@ -152,7 +157,7 @@ public String description() { /** * Set the description property: User-specified description of the firmware. - * + * * @param description the description value to set. * @return the FirmwareInner object itself. */ @@ -166,7 +171,7 @@ public FirmwareInner withDescription(String description) { /** * Get the fileSize property: File size of the uploaded firmware image. - * + * * @return the fileSize value. */ public Long fileSize() { @@ -175,7 +180,7 @@ public Long fileSize() { /** * Set the fileSize property: File size of the uploaded firmware image. - * + * * @param fileSize the fileSize value to set. * @return the FirmwareInner object itself. */ @@ -189,7 +194,7 @@ public FirmwareInner withFileSize(Long fileSize) { /** * Get the status property: The status of firmware scan. - * + * * @return the status value. */ public Status status() { @@ -198,7 +203,7 @@ public Status status() { /** * Set the status property: The status of firmware scan. - * + * * @param status the status value to set. * @return the FirmwareInner object itself. */ @@ -212,20 +217,20 @@ public FirmwareInner withStatus(Status status) { /** * Get the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @return the statusMessages value. */ - public List statusMessages() { + public List statusMessages() { return this.innerProperties() == null ? null : this.innerProperties().statusMessages(); } /** * Set the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @param statusMessages the statusMessages value to set. * @return the FirmwareInner object itself. */ - public FirmwareInner withStatusMessages(List statusMessages) { + public FirmwareInner withStatusMessages(List statusMessages) { if (this.innerProperties() == null) { this.innerProperties = new FirmwareProperties(); } @@ -235,7 +240,7 @@ public FirmwareInner withStatusMessages(List statusMessages) { /** * Get the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -244,7 +249,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareProperties.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareProperties.java index 6b84b20da58d7..b063e7e44f0a9 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareProperties.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareProperties.java @@ -7,10 +7,13 @@ import com.azure.core.annotation.Fluent; import com.azure.resourcemanager.iotfirmwaredefense.models.ProvisioningState; import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Firmware properties. */ +/** + * Firmware properties. + */ @Fluent public final class FirmwareProperties { /* @@ -59,7 +62,7 @@ public final class FirmwareProperties { * A list of errors or other messages generated during firmware analysis */ @JsonProperty(value = "statusMessages") - private List statusMessages; + private List statusMessages; /* * Provisioning state of the resource. @@ -67,13 +70,15 @@ public final class FirmwareProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of FirmwareProperties class. */ + /** + * Creates an instance of FirmwareProperties class. + */ public FirmwareProperties() { } /** * Get the fileName property: File name for a firmware that user uploaded. - * + * * @return the fileName value. */ public String fileName() { @@ -82,7 +87,7 @@ public String fileName() { /** * Set the fileName property: File name for a firmware that user uploaded. - * + * * @param fileName the fileName value to set. * @return the FirmwareProperties object itself. */ @@ -93,7 +98,7 @@ public FirmwareProperties withFileName(String fileName) { /** * Get the vendor property: Firmware vendor. - * + * * @return the vendor value. */ public String vendor() { @@ -102,7 +107,7 @@ public String vendor() { /** * Set the vendor property: Firmware vendor. - * + * * @param vendor the vendor value to set. * @return the FirmwareProperties object itself. */ @@ -113,7 +118,7 @@ public FirmwareProperties withVendor(String vendor) { /** * Get the model property: Firmware model. - * + * * @return the model value. */ public String model() { @@ -122,7 +127,7 @@ public String model() { /** * Set the model property: Firmware model. - * + * * @param model the model value to set. * @return the FirmwareProperties object itself. */ @@ -133,7 +138,7 @@ public FirmwareProperties withModel(String model) { /** * Get the version property: Firmware version. - * + * * @return the version value. */ public String version() { @@ -142,7 +147,7 @@ public String version() { /** * Set the version property: Firmware version. - * + * * @param version the version value to set. * @return the FirmwareProperties object itself. */ @@ -153,7 +158,7 @@ public FirmwareProperties withVersion(String version) { /** * Get the description property: User-specified description of the firmware. - * + * * @return the description value. */ public String description() { @@ -162,7 +167,7 @@ public String description() { /** * Set the description property: User-specified description of the firmware. - * + * * @param description the description value to set. * @return the FirmwareProperties object itself. */ @@ -173,7 +178,7 @@ public FirmwareProperties withDescription(String description) { /** * Get the fileSize property: File size of the uploaded firmware image. - * + * * @return the fileSize value. */ public Long fileSize() { @@ -182,7 +187,7 @@ public Long fileSize() { /** * Set the fileSize property: File size of the uploaded firmware image. - * + * * @param fileSize the fileSize value to set. * @return the FirmwareProperties object itself. */ @@ -193,7 +198,7 @@ public FirmwareProperties withFileSize(Long fileSize) { /** * Get the status property: The status of firmware scan. - * + * * @return the status value. */ public Status status() { @@ -202,7 +207,7 @@ public Status status() { /** * Set the status property: The status of firmware scan. - * + * * @param status the status value to set. * @return the FirmwareProperties object itself. */ @@ -213,27 +218,27 @@ public FirmwareProperties withStatus(Status status) { /** * Get the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @return the statusMessages value. */ - public List statusMessages() { + public List statusMessages() { return this.statusMessages; } /** * Set the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @param statusMessages the statusMessages value to set. * @return the FirmwareProperties object itself. */ - public FirmwareProperties withStatusMessages(List statusMessages) { + public FirmwareProperties withStatusMessages(List statusMessages) { this.statusMessages = statusMessages; return this; } /** * Get the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -242,9 +247,12 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (statusMessages() != null) { + statusMessages().forEach(e -> e.validate()); + } } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareSummaryInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareSummaryInner.java deleted file mode 100644 index ddcc63f2c8930..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/FirmwareSummaryInner.java +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** Summary result after scanning the firmware. */ -@Fluent -public final class FirmwareSummaryInner { - /* - * Total extracted size of the firmware in bytes. - */ - @JsonProperty(value = "extractedSize") - private Long extractedSize; - - /* - * Firmware file size in bytes. - */ - @JsonProperty(value = "fileSize") - private Long fileSize; - - /* - * Extracted file count. - */ - @JsonProperty(value = "extractedFileCount") - private Long extractedFileCount; - - /* - * Components count. - */ - @JsonProperty(value = "componentCount") - private Long componentCount; - - /* - * Binary count - */ - @JsonProperty(value = "binaryCount") - private Long binaryCount; - - /* - * Time used for analysis - */ - @JsonProperty(value = "analysisTimeSeconds") - private Long analysisTimeSeconds; - - /* - * The number of root file systems found. - */ - @JsonProperty(value = "rootFileSystems") - private Long rootFileSystems; - - /** Creates an instance of FirmwareSummaryInner class. */ - public FirmwareSummaryInner() { - } - - /** - * Get the extractedSize property: Total extracted size of the firmware in bytes. - * - * @return the extractedSize value. - */ - public Long extractedSize() { - return this.extractedSize; - } - - /** - * Set the extractedSize property: Total extracted size of the firmware in bytes. - * - * @param extractedSize the extractedSize value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withExtractedSize(Long extractedSize) { - this.extractedSize = extractedSize; - return this; - } - - /** - * Get the fileSize property: Firmware file size in bytes. - * - * @return the fileSize value. - */ - public Long fileSize() { - return this.fileSize; - } - - /** - * Set the fileSize property: Firmware file size in bytes. - * - * @param fileSize the fileSize value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withFileSize(Long fileSize) { - this.fileSize = fileSize; - return this; - } - - /** - * Get the extractedFileCount property: Extracted file count. - * - * @return the extractedFileCount value. - */ - public Long extractedFileCount() { - return this.extractedFileCount; - } - - /** - * Set the extractedFileCount property: Extracted file count. - * - * @param extractedFileCount the extractedFileCount value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withExtractedFileCount(Long extractedFileCount) { - this.extractedFileCount = extractedFileCount; - return this; - } - - /** - * Get the componentCount property: Components count. - * - * @return the componentCount value. - */ - public Long componentCount() { - return this.componentCount; - } - - /** - * Set the componentCount property: Components count. - * - * @param componentCount the componentCount value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withComponentCount(Long componentCount) { - this.componentCount = componentCount; - return this; - } - - /** - * Get the binaryCount property: Binary count. - * - * @return the binaryCount value. - */ - public Long binaryCount() { - return this.binaryCount; - } - - /** - * Set the binaryCount property: Binary count. - * - * @param binaryCount the binaryCount value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withBinaryCount(Long binaryCount) { - this.binaryCount = binaryCount; - return this; - } - - /** - * Get the analysisTimeSeconds property: Time used for analysis. - * - * @return the analysisTimeSeconds value. - */ - public Long analysisTimeSeconds() { - return this.analysisTimeSeconds; - } - - /** - * Set the analysisTimeSeconds property: Time used for analysis. - * - * @param analysisTimeSeconds the analysisTimeSeconds value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withAnalysisTimeSeconds(Long analysisTimeSeconds) { - this.analysisTimeSeconds = analysisTimeSeconds; - return this; - } - - /** - * Get the rootFileSystems property: The number of root file systems found. - * - * @return the rootFileSystems value. - */ - public Long rootFileSystems() { - return this.rootFileSystems; - } - - /** - * Set the rootFileSystems property: The number of root file systems found. - * - * @param rootFileSystems the rootFileSystems value to set. - * @return the FirmwareSummaryInner object itself. - */ - public FirmwareSummaryInner withRootFileSystems(Long rootFileSystems) { - this.rootFileSystems = rootFileSystems; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/OperationInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/OperationInner.java index d45e50cd6ff04..9e665e7a378de 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/OperationInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/OperationInner.java @@ -12,8 +12,8 @@ /** * REST API Operation - * - *

Details of a REST API operation, returned from the Resource Provider Operations API. + * + * Details of a REST API operation, returned from the Resource Provider Operations API. */ @Fluent public final class OperationInner { @@ -50,14 +50,16 @@ public final class OperationInner { @JsonProperty(value = "actionType", access = JsonProperty.Access.WRITE_ONLY) private ActionType actionType; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ public String name() { @@ -67,7 +69,7 @@ public String name() { /** * Get the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -76,7 +78,7 @@ public Boolean isDataAction() { /** * Get the display property: Localized display information for this particular operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -85,7 +87,7 @@ public OperationDisplay display() { /** * Set the display property: Localized display information for this particular operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -97,7 +99,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Get the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ public Origin origin() { @@ -107,7 +109,7 @@ public Origin origin() { /** * Get the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ public ActionType actionType() { @@ -116,7 +118,7 @@ public ActionType actionType() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHash.java similarity index 78% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHash.java index a7486b889bffd..64fb63f701bfb 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHash.java @@ -7,9 +7,11 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Password hash properties. */ +/** + * Password hash properties. + */ @Fluent -public final class PasswordHashInner { +public final class PasswordHash { /* * ID for password hash */ @@ -52,13 +54,15 @@ public final class PasswordHashInner { @JsonProperty(value = "algorithm") private String algorithm; - /** Creates an instance of PasswordHashInner class. */ - public PasswordHashInner() { + /** + * Creates an instance of PasswordHash class. + */ + public PasswordHash() { } /** * Get the passwordHashId property: ID for password hash. - * + * * @return the passwordHashId value. */ public String passwordHashId() { @@ -67,18 +71,18 @@ public String passwordHashId() { /** * Set the passwordHashId property: ID for password hash. - * + * * @param passwordHashId the passwordHashId value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withPasswordHashId(String passwordHashId) { + public PasswordHash withPasswordHashId(String passwordHashId) { this.passwordHashId = passwordHashId; return this; } /** * Get the filePath property: File path of the password hash. - * + * * @return the filePath value. */ public String filePath() { @@ -87,18 +91,18 @@ public String filePath() { /** * Set the filePath property: File path of the password hash. - * + * * @param filePath the filePath value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withFilePath(String filePath) { + public PasswordHash withFilePath(String filePath) { this.filePath = filePath; return this; } /** * Get the salt property: Salt of the password hash. - * + * * @return the salt value. */ public String salt() { @@ -107,18 +111,18 @@ public String salt() { /** * Set the salt property: Salt of the password hash. - * + * * @param salt the salt value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withSalt(String salt) { + public PasswordHash withSalt(String salt) { this.salt = salt; return this; } /** * Get the hash property: Hash of the password. - * + * * @return the hash value. */ public String hash() { @@ -127,18 +131,18 @@ public String hash() { /** * Set the hash property: Hash of the password. - * + * * @param hash the hash value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withHash(String hash) { + public PasswordHash withHash(String hash) { this.hash = hash; return this; } /** * Get the context property: Context of password hash. - * + * * @return the context value. */ public String context() { @@ -147,18 +151,18 @@ public String context() { /** * Set the context property: Context of password hash. - * + * * @param context the context value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withContext(String context) { + public PasswordHash withContext(String context) { this.context = context; return this; } /** * Get the username property: User name of password hash. - * + * * @return the username value. */ public String username() { @@ -167,18 +171,18 @@ public String username() { /** * Set the username property: User name of password hash. - * + * * @param username the username value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withUsername(String username) { + public PasswordHash withUsername(String username) { this.username = username; return this; } /** * Get the algorithm property: Algorithm of the password hash. - * + * * @return the algorithm value. */ public String algorithm() { @@ -187,18 +191,18 @@ public String algorithm() { /** * Set the algorithm property: Algorithm of the password hash. - * + * * @param algorithm the algorithm value to set. - * @return the PasswordHashInner object itself. + * @return the PasswordHash object itself. */ - public PasswordHashInner withAlgorithm(String algorithm) { + public PasswordHash withAlgorithm(String algorithm) { this.algorithm = algorithm; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashResourceInner.java new file mode 100644 index 0000000000000..003fe8e9adb53 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/PasswordHashResourceInner.java @@ -0,0 +1,224 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Password hash resource. + */ +@Fluent +public final class PasswordHashResourceInner extends ProxyResource { + /* + * The properties of a password hash found within a firmware image + */ + @JsonProperty(value = "properties") + private PasswordHash innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of PasswordHashResourceInner class. + */ + public PasswordHashResourceInner() { + } + + /** + * Get the innerProperties property: The properties of a password hash found within a firmware image. + * + * @return the innerProperties value. + */ + private PasswordHash innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the passwordHashId property: ID for password hash. + * + * @return the passwordHashId value. + */ + public String passwordHashId() { + return this.innerProperties() == null ? null : this.innerProperties().passwordHashId(); + } + + /** + * Set the passwordHashId property: ID for password hash. + * + * @param passwordHashId the passwordHashId value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withPasswordHashId(String passwordHashId) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withPasswordHashId(passwordHashId); + return this; + } + + /** + * Get the filePath property: File path of the password hash. + * + * @return the filePath value. + */ + public String filePath() { + return this.innerProperties() == null ? null : this.innerProperties().filePath(); + } + + /** + * Set the filePath property: File path of the password hash. + * + * @param filePath the filePath value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withFilePath(String filePath) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withFilePath(filePath); + return this; + } + + /** + * Get the salt property: Salt of the password hash. + * + * @return the salt value. + */ + public String salt() { + return this.innerProperties() == null ? null : this.innerProperties().salt(); + } + + /** + * Set the salt property: Salt of the password hash. + * + * @param salt the salt value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withSalt(String salt) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withSalt(salt); + return this; + } + + /** + * Get the hash property: Hash of the password. + * + * @return the hash value. + */ + public String hash() { + return this.innerProperties() == null ? null : this.innerProperties().hash(); + } + + /** + * Set the hash property: Hash of the password. + * + * @param hash the hash value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withHash(String hash) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withHash(hash); + return this; + } + + /** + * Get the context property: Context of password hash. + * + * @return the context value. + */ + public String context() { + return this.innerProperties() == null ? null : this.innerProperties().context(); + } + + /** + * Set the context property: Context of password hash. + * + * @param context the context value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withContext(String context) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withContext(context); + return this; + } + + /** + * Get the username property: User name of password hash. + * + * @return the username value. + */ + public String username() { + return this.innerProperties() == null ? null : this.innerProperties().username(); + } + + /** + * Set the username property: User name of password hash. + * + * @param username the username value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withUsername(String username) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withUsername(username); + return this; + } + + /** + * Get the algorithm property: Algorithm of the password hash. + * + * @return the algorithm value. + */ + public String algorithm() { + return this.innerProperties() == null ? null : this.innerProperties().algorithm(); + } + + /** + * Set the algorithm property: Algorithm of the password hash. + * + * @param algorithm the algorithm value to set. + * @return the PasswordHashResourceInner object itself. + */ + public PasswordHashResourceInner withAlgorithm(String algorithm) { + if (this.innerProperties() == null) { + this.innerProperties = new PasswordHash(); + } + this.innerProperties().withAlgorithm(algorithm); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponent.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponent.java new file mode 100644 index 0000000000000..46d4a5dee2d72 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponent.java @@ -0,0 +1,159 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * SBOM component of a firmware. + */ +@Fluent +public final class SbomComponent { + /* + * ID for the component. + */ + @JsonProperty(value = "componentId") + private String componentId; + + /* + * Name for the component. + */ + @JsonProperty(value = "componentName") + private String componentName; + + /* + * Version for the component. + */ + @JsonProperty(value = "version") + private String version; + + /* + * License for the component. + */ + @JsonProperty(value = "license") + private String license; + + /* + * File paths related to the component. + */ + @JsonProperty(value = "filePaths") + private List filePaths; + + /** + * Creates an instance of SbomComponent class. + */ + public SbomComponent() { + } + + /** + * Get the componentId property: ID for the component. + * + * @return the componentId value. + */ + public String componentId() { + return this.componentId; + } + + /** + * Set the componentId property: ID for the component. + * + * @param componentId the componentId value to set. + * @return the SbomComponent object itself. + */ + public SbomComponent withComponentId(String componentId) { + this.componentId = componentId; + return this; + } + + /** + * Get the componentName property: Name for the component. + * + * @return the componentName value. + */ + public String componentName() { + return this.componentName; + } + + /** + * Set the componentName property: Name for the component. + * + * @param componentName the componentName value to set. + * @return the SbomComponent object itself. + */ + public SbomComponent withComponentName(String componentName) { + this.componentName = componentName; + return this; + } + + /** + * Get the version property: Version for the component. + * + * @return the version value. + */ + public String version() { + return this.version; + } + + /** + * Set the version property: Version for the component. + * + * @param version the version value to set. + * @return the SbomComponent object itself. + */ + public SbomComponent withVersion(String version) { + this.version = version; + return this; + } + + /** + * Get the license property: License for the component. + * + * @return the license value. + */ + public String license() { + return this.license; + } + + /** + * Set the license property: License for the component. + * + * @param license the license value to set. + * @return the SbomComponent object itself. + */ + public SbomComponent withLicense(String license) { + this.license = license; + return this; + } + + /** + * Get the filePaths property: File paths related to the component. + * + * @return the filePaths value. + */ + public List filePaths() { + return this.filePaths; + } + + /** + * Set the filePaths property: File paths related to the component. + * + * @param filePaths the filePaths value to set. + * @return the SbomComponent object itself. + */ + public SbomComponent withFilePaths(List filePaths) { + this.filePaths = filePaths; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponentResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponentResourceInner.java new file mode 100644 index 0000000000000..7704186e4ce87 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SbomComponentResourceInner.java @@ -0,0 +1,179 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * SBOM analysis result resource. + */ +@Fluent +public final class SbomComponentResourceInner extends ProxyResource { + /* + * The properties of an SBOM component found within a firmware image + */ + @JsonProperty(value = "properties") + private SbomComponent innerProperties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of SbomComponentResourceInner class. + */ + public SbomComponentResourceInner() { + } + + /** + * Get the innerProperties property: The properties of an SBOM component found within a firmware image. + * + * @return the innerProperties value. + */ + private SbomComponent innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the componentId property: ID for the component. + * + * @return the componentId value. + */ + public String componentId() { + return this.innerProperties() == null ? null : this.innerProperties().componentId(); + } + + /** + * Set the componentId property: ID for the component. + * + * @param componentId the componentId value to set. + * @return the SbomComponentResourceInner object itself. + */ + public SbomComponentResourceInner withComponentId(String componentId) { + if (this.innerProperties() == null) { + this.innerProperties = new SbomComponent(); + } + this.innerProperties().withComponentId(componentId); + return this; + } + + /** + * Get the componentName property: Name for the component. + * + * @return the componentName value. + */ + public String componentName() { + return this.innerProperties() == null ? null : this.innerProperties().componentName(); + } + + /** + * Set the componentName property: Name for the component. + * + * @param componentName the componentName value to set. + * @return the SbomComponentResourceInner object itself. + */ + public SbomComponentResourceInner withComponentName(String componentName) { + if (this.innerProperties() == null) { + this.innerProperties = new SbomComponent(); + } + this.innerProperties().withComponentName(componentName); + return this; + } + + /** + * Get the version property: Version for the component. + * + * @return the version value. + */ + public String version() { + return this.innerProperties() == null ? null : this.innerProperties().version(); + } + + /** + * Set the version property: Version for the component. + * + * @param version the version value to set. + * @return the SbomComponentResourceInner object itself. + */ + public SbomComponentResourceInner withVersion(String version) { + if (this.innerProperties() == null) { + this.innerProperties = new SbomComponent(); + } + this.innerProperties().withVersion(version); + return this; + } + + /** + * Get the license property: License for the component. + * + * @return the license value. + */ + public String license() { + return this.innerProperties() == null ? null : this.innerProperties().license(); + } + + /** + * Set the license property: License for the component. + * + * @param license the license value to set. + * @return the SbomComponentResourceInner object itself. + */ + public SbomComponentResourceInner withLicense(String license) { + if (this.innerProperties() == null) { + this.innerProperties = new SbomComponent(); + } + this.innerProperties().withLicense(license); + return this; + } + + /** + * Get the filePaths property: File paths related to the component. + * + * @return the filePaths value. + */ + public List filePaths() { + return this.innerProperties() == null ? null : this.innerProperties().filePaths(); + } + + /** + * Set the filePaths property: File paths related to the component. + * + * @param filePaths the filePaths value to set. + * @return the SbomComponentResourceInner object itself. + */ + public SbomComponentResourceInner withFilePaths(List filePaths) { + if (this.innerProperties() == null) { + this.innerProperties = new SbomComponent(); + } + this.innerProperties().withFilePaths(filePaths); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SummaryResourceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SummaryResourceInner.java new file mode 100644 index 0000000000000..7f7bfdda32081 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/SummaryResourceInner.java @@ -0,0 +1,64 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; + +import com.azure.core.annotation.Immutable; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResourceProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The object representing a firmware analysis summary resource. + */ +@Immutable +public final class SummaryResourceInner extends ProxyResource { + /* + * Properties of an analysis summary. + */ + @JsonProperty(value = "properties", access = JsonProperty.Access.WRITE_ONLY) + private SummaryResourceProperties properties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of SummaryResourceInner class. + */ + public SummaryResourceInner() { + } + + /** + * Get the properties property: Properties of an analysis summary. + * + * @return the properties value. + */ + public SummaryResourceProperties properties() { + return this.properties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/UrlTokenInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/UrlTokenInner.java index 40164c117d837..4aafb2c90a322 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/UrlTokenInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/UrlTokenInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Url data for creating or accessing a blob file. */ +/** + * Url data for creating or accessing a blob file. + */ @Immutable public final class UrlTokenInner { /* @@ -16,37 +18,24 @@ public final class UrlTokenInner { @JsonProperty(value = "url", access = JsonProperty.Access.WRITE_ONLY) private String url; - /* - * SAS URL for file uploading. Kept for backwards compatibility + /** + * Creates an instance of UrlTokenInner class. */ - @JsonProperty(value = "uploadUrl", access = JsonProperty.Access.WRITE_ONLY) - private String uploadUrl; - - /** Creates an instance of UrlTokenInner class. */ public UrlTokenInner() { } /** * Get the url property: SAS URL for creating or accessing a blob file. - * + * * @return the url value. */ public String url() { return this.url; } - /** - * Get the uploadUrl property: SAS URL for file uploading. Kept for backwards compatibility. - * - * @return the uploadUrl value. - */ - public String uploadUrl() { - return this.uploadUrl; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceInner.java index 89c0c0cd5e3e0..0edab0172f15a 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Firmware analysis workspace. */ +/** + * Firmware analysis workspace. + */ @Fluent public final class WorkspaceInner extends Resource { /* @@ -26,13 +28,15 @@ public final class WorkspaceInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of WorkspaceInner class. */ + /** + * Creates an instance of WorkspaceInner class. + */ public WorkspaceInner() { } /** * Get the innerProperties property: Workspace properties. - * + * * @return the innerProperties value. */ private WorkspaceProperties innerProperties() { @@ -41,21 +45,25 @@ private WorkspaceProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public WorkspaceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public WorkspaceInner withTags(Map tags) { super.withTags(tags); @@ -64,7 +72,7 @@ public WorkspaceInner withTags(Map tags) { /** * Get the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -73,7 +81,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceProperties.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceProperties.java index 0550db5d17909..505cdbda358d1 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceProperties.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/WorkspaceProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.iotfirmwaredefense.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Workspace properties. */ +/** + * Workspace properties. + */ @Immutable public final class WorkspaceProperties { /* @@ -17,13 +19,15 @@ public final class WorkspaceProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private ProvisioningState provisioningState; - /** Creates an instance of WorkspaceProperties class. */ + /** + * Creates an instance of WorkspaceProperties class. + */ public WorkspaceProperties() { } /** * Get the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -32,7 +36,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/package-info.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/package-info.java index 3a80a61295751..1e6f7f0c41e31 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/package-info.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the inner data models for Fist. The definitions and parameters in this swagger specification will - * be used to manage the IoT Firmware Defense resources. + * Package containing the inner data models for IoTFirmwareDefense. + * Firmware & IoT Security REST API. */ package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/package-info.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/package-info.java index b8362def140f3..13851f086a9f9 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/package-info.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the service clients for Fist. The definitions and parameters in this swagger specification will be - * used to manage the IoT Firmware Defense resources. + * Package containing the service clients for IoTFirmwareDefense. + * Firmware & IoT Security REST API. */ package com.azure.resourcemanager.iotfirmwaredefense.fluent; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningResourceImpl.java similarity index 63% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningResourceImpl.java index c30ed498a1263..ff952ebb7544f 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningResourceImpl.java @@ -4,26 +4,37 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardening; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningResource; -public final class BinaryHardeningImpl implements BinaryHardening { - private BinaryHardeningInner innerObject; +public final class BinaryHardeningResourceImpl implements BinaryHardeningResource { + private BinaryHardeningResourceInner innerObject; private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - BinaryHardeningImpl( - BinaryHardeningInner innerObject, + BinaryHardeningResourceImpl(BinaryHardeningResourceInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String binaryHardeningId() { return this.innerModel().binaryHardeningId(); } @@ -32,8 +43,8 @@ public String architecture() { return this.innerModel().architecture(); } - public String path() { - return this.innerModel().path(); + public String filePath() { + return this.innerModel().filePath(); } public String classProperty() { @@ -48,27 +59,27 @@ public String rpath() { return this.innerModel().rpath(); } - public NxFlag nx() { + public Boolean nx() { return this.innerModel().nx(); } - public PieFlag pie() { + public Boolean pie() { return this.innerModel().pie(); } - public RelroFlag relro() { + public Boolean relro() { return this.innerModel().relro(); } - public CanaryFlag canary() { + public Boolean canary() { return this.innerModel().canary(); } - public StrippedFlag stripped() { + public Boolean stripped() { return this.innerModel().stripped(); } - public BinaryHardeningInner innerModel() { + public BinaryHardeningResourceInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningSummaryImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningSummaryImpl.java deleted file mode 100644 index 435686164972f..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningSummaryImpl.java +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.implementation; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningSummary; - -public final class BinaryHardeningSummaryImpl implements BinaryHardeningSummary { - private BinaryHardeningSummaryInner innerObject; - - private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - - BinaryHardeningSummaryImpl( - BinaryHardeningSummaryInner innerObject, - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public Long totalFiles() { - return this.innerModel().totalFiles(); - } - - public Integer nx() { - return this.innerModel().nx(); - } - - public Integer pie() { - return this.innerModel().pie(); - } - - public Integer relro() { - return this.innerModel().relro(); - } - - public Integer canary() { - return this.innerModel().canary(); - } - - public Integer stripped() { - return this.innerModel().stripped(); - } - - public BinaryHardeningSummaryInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsClientImpl.java new file mode 100644 index 0000000000000..d681358f1eb9f --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsClientImpl.java @@ -0,0 +1,300 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.BinaryHardeningsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningListResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in BinaryHardeningsClient. + */ +public final class BinaryHardeningsClientImpl implements BinaryHardeningsClient { + /** + * The proxy service used to perform REST calls. + */ + private final BinaryHardeningsService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of BinaryHardeningsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + BinaryHardeningsClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(BinaryHardeningsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefenseBinaryHardenings to be used by the proxy service + * to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefenseBi") + public interface BinaryHardeningsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/binaryHardeningResults") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsImpl.java new file mode 100644 index 0000000000000..8de1d800e527f --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/BinaryHardeningsImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.BinaryHardeningsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardenings; + +public final class BinaryHardeningsImpl implements BinaryHardenings { + private static final ClientLogger LOGGER = new ClientLogger(BinaryHardeningsImpl.class); + + private final BinaryHardeningsClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public BinaryHardeningsImpl(BinaryHardeningsClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BinaryHardeningResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BinaryHardeningResourceImpl(inner1, this.manager())); + } + + private BinaryHardeningsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateResourceImpl.java similarity index 77% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateResourceImpl.java index 7d4de06b6ae1a..175d3e5890074 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateResourceImpl.java @@ -4,38 +4,50 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificate; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner; import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateEntity; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsExpired; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsSelfSigned; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsShortKeySize; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsWeakSignature; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateResource; import com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey; import java.time.OffsetDateTime; import java.util.Collections; import java.util.List; -public final class CryptoCertificateImpl implements CryptoCertificate { - private CryptoCertificateInner innerObject; +public final class CryptoCertificateResourceImpl implements CryptoCertificateResource { + private CryptoCertificateResourceInner innerObject; private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - CryptoCertificateImpl( - CryptoCertificateInner innerObject, + CryptoCertificateResourceImpl(CryptoCertificateResourceInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } - public String cryptoCertId() { - return this.innerModel().cryptoCertId(); + public String id() { + return this.innerModel().id(); } public String name() { return this.innerModel().name(); } + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public String cryptoCertId() { + return this.innerModel().cryptoCertId(); + } + + public String namePropertiesName() { + return this.innerModel().namePropertiesName(); + } + public CryptoCertificateEntity subject() { return this.innerModel().subject(); } @@ -102,23 +114,23 @@ public PairedKey pairedKey() { return this.innerModel().pairedKey(); } - public IsExpired isExpired() { + public Boolean isExpired() { return this.innerModel().isExpired(); } - public IsSelfSigned isSelfSigned() { + public Boolean isSelfSigned() { return this.innerModel().isSelfSigned(); } - public IsWeakSignature isWeakSignature() { + public Boolean isWeakSignature() { return this.innerModel().isWeakSignature(); } - public IsShortKeySize isShortKeySize() { + public Boolean isShortKeySize() { return this.innerModel().isShortKeySize(); } - public CryptoCertificateInner innerModel() { + public CryptoCertificateResourceInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateSummaryImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateSummaryImpl.java deleted file mode 100644 index e18e7d7a66e68..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificateSummaryImpl.java +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.implementation; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateSummary; - -public final class CryptoCertificateSummaryImpl implements CryptoCertificateSummary { - private CryptoCertificateSummaryInner innerObject; - - private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - - CryptoCertificateSummaryImpl( - CryptoCertificateSummaryInner innerObject, - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public Long totalCertificates() { - return this.innerModel().totalCertificates(); - } - - public Long pairedKeys() { - return this.innerModel().pairedKeys(); - } - - public Long expired() { - return this.innerModel().expired(); - } - - public Long expiringSoon() { - return this.innerModel().expiringSoon(); - } - - public Long weakSignature() { - return this.innerModel().weakSignature(); - } - - public Long selfSigned() { - return this.innerModel().selfSigned(); - } - - public Long shortKeySize() { - return this.innerModel().shortKeySize(); - } - - public CryptoCertificateSummaryInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesClientImpl.java new file mode 100644 index 0000000000000..1471fa83acfae --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesClientImpl.java @@ -0,0 +1,296 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CryptoCertificatesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateListResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in CryptoCertificatesClient. + */ +public final class CryptoCertificatesClientImpl implements CryptoCertificatesClient { + /** + * The proxy service used to perform REST calls. + */ + private final CryptoCertificatesService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of CryptoCertificatesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + CryptoCertificatesClientImpl(IoTFirmwareDefenseImpl client) { + this.service = RestProxy.create(CryptoCertificatesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefenseCryptoCertificates to be used by the proxy service + * to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefenseCr") + public interface CryptoCertificatesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/cryptoCertificates") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesImpl.java new file mode 100644 index 0000000000000..5f331c5d53e37 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoCertificatesImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CryptoCertificatesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificates; + +public final class CryptoCertificatesImpl implements CryptoCertificates { + private static final ClientLogger LOGGER = new ClientLogger(CryptoCertificatesImpl.class); + + private final CryptoCertificatesClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public CryptoCertificatesImpl(CryptoCertificatesClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CryptoCertificateResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CryptoCertificateResourceImpl(inner1, this.manager())); + } + + private CryptoCertificatesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeyImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeyResourceImpl.java similarity index 74% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeyImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeyResourceImpl.java index a1aa8febbaf67..eeca5b7a60328 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeyImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeyResourceImpl.java @@ -4,25 +4,40 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKey; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsShortKeySize; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeyResource; import com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey; import java.util.Collections; import java.util.List; -public final class CryptoKeyImpl implements CryptoKey { - private CryptoKeyInner innerObject; +public final class CryptoKeyResourceImpl implements CryptoKeyResource { + private CryptoKeyResourceInner innerObject; private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - CryptoKeyImpl( - CryptoKeyInner innerObject, + CryptoKeyResourceImpl(CryptoKeyResourceInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String cryptoKeyId() { return this.innerModel().cryptoKeyId(); } @@ -61,11 +76,11 @@ public PairedKey pairedKey() { return this.innerModel().pairedKey(); } - public IsShortKeySize isShortKeySize() { + public Boolean isShortKeySize() { return this.innerModel().isShortKeySize(); } - public CryptoKeyInner innerModel() { + public CryptoKeyResourceInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeySummaryImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeySummaryImpl.java deleted file mode 100644 index bed579f3265fc..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeySummaryImpl.java +++ /dev/null @@ -1,49 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.implementation; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeySummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeySummary; - -public final class CryptoKeySummaryImpl implements CryptoKeySummary { - private CryptoKeySummaryInner innerObject; - - private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - - CryptoKeySummaryImpl( - CryptoKeySummaryInner innerObject, - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public Long totalKeys() { - return this.innerModel().totalKeys(); - } - - public Long publicKeys() { - return this.innerModel().publicKeys(); - } - - public Long privateKeys() { - return this.innerModel().privateKeys(); - } - - public Long pairedKeys() { - return this.innerModel().pairedKeys(); - } - - public Long shortKeySize() { - return this.innerModel().shortKeySize(); - } - - public CryptoKeySummaryInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysClientImpl.java new file mode 100644 index 0000000000000..b9cf4dc2b54d9 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysClientImpl.java @@ -0,0 +1,296 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CryptoKeysClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeyListResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in CryptoKeysClient. + */ +public final class CryptoKeysClientImpl implements CryptoKeysClient { + /** + * The proxy service used to perform REST calls. + */ + private final CryptoKeysService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of CryptoKeysClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + CryptoKeysClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(CryptoKeysService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefenseCryptoKeys to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefenseCr") + public interface CryptoKeysService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/cryptoKeys") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysImpl.java new file mode 100644 index 0000000000000..4f87e68e9cf5a --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CryptoKeysImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CryptoKeysClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeyResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeys; + +public final class CryptoKeysImpl implements CryptoKeys { + private static final ClientLogger LOGGER = new ClientLogger(CryptoKeysImpl.class); + + private final CryptoKeysClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public CryptoKeysImpl(CryptoKeysClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CryptoKeyResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CryptoKeyResourceImpl(inner1, this.manager())); + } + + private CryptoKeysClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveResourceImpl.java similarity index 67% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveResourceImpl.java index e22c8c30c96d9..e0e5641e2c635 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveResourceImpl.java @@ -4,29 +4,46 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.Cve; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent; import com.azure.resourcemanager.iotfirmwaredefense.models.CveLink; -import java.time.OffsetDateTime; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveResource; import java.util.Collections; import java.util.List; -public final class CveImpl implements Cve { - private CveInner innerObject; +public final class CveResourceImpl implements CveResource { + private CveResourceInner innerObject; private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - CveImpl( - CveInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + CveResourceImpl(CveResourceInner innerObject, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String cveId() { return this.innerModel().cveId(); } - public Object component() { + public CveComponent component() { return this.innerModel().component(); } @@ -34,8 +51,8 @@ public String severity() { return this.innerModel().severity(); } - public String name() { - return this.innerModel().name(); + public String namePropertiesName() { + return this.innerModel().namePropertiesName(); } public String cvssScore() { @@ -54,14 +71,6 @@ public String cvssV3Score() { return this.innerModel().cvssV3Score(); } - public OffsetDateTime publishDate() { - return this.innerModel().publishDate(); - } - - public OffsetDateTime updatedDate() { - return this.innerModel().updatedDate(); - } - public List links() { List inner = this.innerModel().links(); if (inner != null) { @@ -75,7 +84,7 @@ public String description() { return this.innerModel().description(); } - public CveInner innerModel() { + public CveResourceInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveSummaryImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveSummaryImpl.java deleted file mode 100644 index 0a81bc74acb2f..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CveSummaryImpl.java +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.implementation; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.CveSummary; - -public final class CveSummaryImpl implements CveSummary { - private CveSummaryInner innerObject; - - private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - - CveSummaryImpl( - CveSummaryInner innerObject, - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public Long critical() { - return this.innerModel().critical(); - } - - public Long high() { - return this.innerModel().high(); - } - - public Long medium() { - return this.innerModel().medium(); - } - - public Long low() { - return this.innerModel().low(); - } - - public Long unknown() { - return this.innerModel().unknown(); - } - - public Long undefined() { - return this.innerModel().undefined(); - } - - public CveSummaryInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesClientImpl.java new file mode 100644 index 0000000000000..b58d6f12227a7 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesClientImpl.java @@ -0,0 +1,293 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CvesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveListResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in CvesClient. + */ +public final class CvesClientImpl implements CvesClient { + /** + * The proxy service used to perform REST calls. + */ + private final CvesService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of CvesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + CvesClientImpl(IoTFirmwareDefenseImpl client) { + this.service = RestProxy.create(CvesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefenseCves to be used by the proxy service to perform + * REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefenseCv") + public interface CvesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/cves") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesImpl.java new file mode 100644 index 0000000000000..16f828535a7e2 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/CvesImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CvesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.Cves; + +public final class CvesImpl implements Cves { + private static final ClientLogger LOGGER = new ClientLogger(CvesImpl.class); + + private final CvesClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public CvesImpl(CvesClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CveResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId, + Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CveResourceImpl(inner1, this.manager())); + } + + private CvesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareImpl.java index bb01cfd4ac50d..eff8f9d8e7048 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareImpl.java @@ -4,26 +4,15 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardening; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningSummary; -import com.azure.resourcemanager.iotfirmwaredefense.models.Component; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificate; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateSummary; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKey; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeySummary; -import com.azure.resourcemanager.iotfirmwaredefense.models.Cve; -import com.azure.resourcemanager.iotfirmwaredefense.models.CveSummary; import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareSummary; import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareUpdateDefinition; -import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHash; import com.azure.resourcemanager.iotfirmwaredefense.models.ProvisioningState; import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; import com.azure.resourcemanager.iotfirmwaredefense.models.UrlToken; import java.util.Collections; import java.util.List; @@ -77,8 +66,8 @@ public Status status() { return this.innerModel().status(); } - public List statusMessages() { - List inner = this.innerModel().statusMessages(); + public List statusMessages() { + List inner = this.innerModel().statusMessages(); if (inner != null) { return Collections.unmodifiableList(inner); } else { @@ -117,22 +106,15 @@ public FirmwareImpl withExistingWorkspace(String resourceGroupName, String works } public Firmware create() { - this.innerObject = - serviceManager - .serviceClient() - .getFirmwares() - .createWithResponse(resourceGroupName, workspaceName, firmwareId, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirmwares() + .createWithResponse(resourceGroupName, workspaceName, firmwareId, this.innerModel(), Context.NONE) + .getValue(); return this; } public Firmware create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getFirmwares() - .createWithResponse(resourceGroupName, workspaceName, firmwareId, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirmwares() + .createWithResponse(resourceGroupName, workspaceName, firmwareId, this.innerModel(), context).getValue(); return this; } @@ -148,59 +130,41 @@ public FirmwareImpl update() { } public Firmware apply() { - this.innerObject = - serviceManager - .serviceClient() - .getFirmwares() - .updateWithResponse(resourceGroupName, workspaceName, firmwareId, updateFirmware, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirmwares() + .updateWithResponse(resourceGroupName, workspaceName, firmwareId, updateFirmware, Context.NONE).getValue(); return this; } public Firmware apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getFirmwares() - .updateWithResponse(resourceGroupName, workspaceName, firmwareId, updateFirmware, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirmwares() + .updateWithResponse(resourceGroupName, workspaceName, firmwareId, updateFirmware, context).getValue(); return this; } - FirmwareImpl( - FirmwareInner innerObject, + FirmwareImpl(FirmwareInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.workspaceName = Utils.getValueFromIdByName(innerObject.id(), "workspaces"); - this.firmwareId = Utils.getValueFromIdByName(innerObject.id(), "firmwares"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.workspaceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "workspaces"); + this.firmwareId = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "firmwares"); } public Firmware refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getFirmwares() - .getWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirmwares() + .getWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE).getValue(); return this; } public Firmware refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getFirmwares() - .getWithResponse(resourceGroupName, workspaceName, firmwareId, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getFirmwares() + .getWithResponse(resourceGroupName, workspaceName, firmwareId, context).getValue(); return this; } public Response generateDownloadUrlWithResponse(Context context) { - return serviceManager - .firmwares() - .generateDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, context); + return serviceManager.firmwares().generateDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, + context); } public UrlToken generateDownloadUrl() { @@ -208,147 +172,14 @@ public UrlToken generateDownloadUrl() { } public Response generateFilesystemDownloadUrlWithResponse(Context context) { - return serviceManager - .firmwares() - .generateFilesystemDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, context); + return serviceManager.firmwares().generateFilesystemDownloadUrlWithResponse(resourceGroupName, workspaceName, + firmwareId, context); } public UrlToken generateFilesystemDownloadUrl() { return serviceManager.firmwares().generateFilesystemDownloadUrl(resourceGroupName, workspaceName, firmwareId); } - public Response generateSummaryWithResponse(Context context) { - return serviceManager - .firmwares() - .generateSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public FirmwareSummary generateSummary() { - return serviceManager.firmwares().generateSummary(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateComponentList() { - return serviceManager.firmwares().listGenerateComponentList(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateComponentList(Context context) { - return serviceManager - .firmwares() - .listGenerateComponentList(resourceGroupName, workspaceName, firmwareId, context); - } - - public Response generateComponentDetailsWithResponse(Context context) { - return serviceManager - .firmwares() - .generateComponentDetailsWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public Component generateComponentDetails() { - return serviceManager.firmwares().generateComponentDetails(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateBinaryHardeningList() { - return serviceManager.firmwares().listGenerateBinaryHardeningList(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateBinaryHardeningList(Context context) { - return serviceManager - .firmwares() - .listGenerateBinaryHardeningList(resourceGroupName, workspaceName, firmwareId, context); - } - - public Response generateBinaryHardeningSummaryWithResponse(Context context) { - return serviceManager - .firmwares() - .generateBinaryHardeningSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public BinaryHardeningSummary generateBinaryHardeningSummary() { - return serviceManager.firmwares().generateBinaryHardeningSummary(resourceGroupName, workspaceName, firmwareId); - } - - public Response generateBinaryHardeningDetailsWithResponse(Context context) { - return serviceManager - .firmwares() - .generateBinaryHardeningDetailsWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public BinaryHardening generateBinaryHardeningDetails() { - return serviceManager.firmwares().generateBinaryHardeningDetails(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGeneratePasswordHashList() { - return serviceManager.firmwares().listGeneratePasswordHashList(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGeneratePasswordHashList(Context context) { - return serviceManager - .firmwares() - .listGeneratePasswordHashList(resourceGroupName, workspaceName, firmwareId, context); - } - - public PagedIterable listGenerateCveList() { - return serviceManager.firmwares().listGenerateCveList(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateCveList(Context context) { - return serviceManager.firmwares().listGenerateCveList(resourceGroupName, workspaceName, firmwareId, context); - } - - public Response generateCveSummaryWithResponse(Context context) { - return serviceManager - .firmwares() - .generateCveSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public CveSummary generateCveSummary() { - return serviceManager.firmwares().generateCveSummary(resourceGroupName, workspaceName, firmwareId); - } - - public Response generateCryptoCertificateSummaryWithResponse(Context context) { - return serviceManager - .firmwares() - .generateCryptoCertificateSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public CryptoCertificateSummary generateCryptoCertificateSummary() { - return serviceManager - .firmwares() - .generateCryptoCertificateSummary(resourceGroupName, workspaceName, firmwareId); - } - - public Response generateCryptoKeySummaryWithResponse(Context context) { - return serviceManager - .firmwares() - .generateCryptoKeySummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - } - - public CryptoKeySummary generateCryptoKeySummary() { - return serviceManager.firmwares().generateCryptoKeySummary(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateCryptoCertificateList() { - return serviceManager - .firmwares() - .listGenerateCryptoCertificateList(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateCryptoCertificateList(Context context) { - return serviceManager - .firmwares() - .listGenerateCryptoCertificateList(resourceGroupName, workspaceName, firmwareId, context); - } - - public PagedIterable listGenerateCryptoKeyList() { - return serviceManager.firmwares().listGenerateCryptoKeyList(resourceGroupName, workspaceName, firmwareId); - } - - public PagedIterable listGenerateCryptoKeyList(Context context) { - return serviceManager - .firmwares() - .listGenerateCryptoKeyList(resourceGroupName, workspaceName, firmwareId, context); - } - public FirmwareImpl withFileName(String fileName) { if (isInCreateMode()) { this.innerModel().withFileName(fileName); @@ -419,7 +250,7 @@ public FirmwareImpl withStatus(Status status) { } } - public FirmwareImpl withStatusMessages(List statusMessages) { + public FirmwareImpl withStatusMessages(List statusMessages) { if (isInCreateMode()) { this.innerModel().withStatusMessages(statusMessages); return this; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareSummaryImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareSummaryImpl.java deleted file mode 100644 index 0ccafa291c6a5..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwareSummaryImpl.java +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.implementation; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareSummary; - -public final class FirmwareSummaryImpl implements FirmwareSummary { - private FirmwareSummaryInner innerObject; - - private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - - FirmwareSummaryImpl( - FirmwareSummaryInner innerObject, - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public Long extractedSize() { - return this.innerModel().extractedSize(); - } - - public Long fileSize() { - return this.innerModel().fileSize(); - } - - public Long extractedFileCount() { - return this.innerModel().extractedFileCount(); - } - - public Long componentCount() { - return this.innerModel().componentCount(); - } - - public Long binaryCount() { - return this.innerModel().binaryCount(); - } - - public Long analysisTimeSeconds() { - return this.innerModel().analysisTimeSeconds(); - } - - public Long rootFileSystems() { - return this.innerModel().rootFileSystems(); - } - - public FirmwareSummaryInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresClientImpl.java index ca8c56889598c..80177869933ab 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresClientImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresClientImpl.java @@ -31,429 +31,126 @@ import com.azure.core.util.Context; import com.azure.core.util.FluxUtil; import com.azure.resourcemanager.iotfirmwaredefense.fluent.FirmwaresClient; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeySummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.UrlTokenInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningList; -import com.azure.resourcemanager.iotfirmwaredefense.models.ComponentList; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateList; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeyList; -import com.azure.resourcemanager.iotfirmwaredefense.models.CveList; import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareList; import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareUpdateDefinition; -import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in FirmwaresClient. */ +/** + * An instance of this class provides access to all the operations defined in FirmwaresClient. + */ public final class FirmwaresClientImpl implements FirmwaresClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final FirmwaresService service; - /** The service client containing this operation class. */ - private final FistImpl client; + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; /** * Initializes an instance of FirmwaresClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - FirmwaresClientImpl(FistImpl client) { - this.service = - RestProxy.create(FirmwaresService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + FirmwaresClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(FirmwaresService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for FistFirmwares to be used by the proxy service to perform REST calls. + * The interface defining all the services for IoTFirmwareDefenseFirmwares to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "FistFirmwares") + @ServiceInterface(name = "IoTFirmwareDefenseFi") public interface FirmwaresService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByWorkspace( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") - @ExpectedResponses({200, 201}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") FirmwareInner firmware, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") - @ExpectedResponses({200, 201}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") FirmwareUpdateDefinition firmware, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") - @ExpectedResponses({200, 204}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateDownloadUrl") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateDownloadUrl( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateFilesystemDownloadUrl") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateFilesystemDownloadUrl( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateSummary") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateSummary( - @HostParam("$host") String endpoint, + Mono> listByWorkspace(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateComponentList") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateComponentList( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateComponentDetails") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateComponentDetails( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateBinaryHardeningList") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateBinaryHardeningList( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateBinaryHardeningSummary") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateBinaryHardeningSummary( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateBinaryHardeningDetails") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateBinaryHardeningDetails( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generatePasswordHashList") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGeneratePasswordHashList( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateCveList") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateCveList( - @HostParam("$host") String endpoint, + Mono> create(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") FirmwareInner firmware, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateCveSummary") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateCveSummary( - @HostParam("$host") String endpoint, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") FirmwareUpdateDefinition firmware, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateCryptoCertificateSummary") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateCryptoCertificateSummary( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateCryptoKeySummary") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateCryptoKeySummary( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateCryptoCertificateList") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateDownloadUrl") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateCryptoCertificateList( - @HostParam("$host") String endpoint, + Mono> generateDownloadUrl(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateCryptoKeyList") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/generateFilesystemDownloadUrl") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateCryptoKeyList( - @HostParam("$host") String endpoint, + Mono> generateFilesystemDownloadUrl(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @PathParam("firmwareId") String firmwareId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByWorkspaceNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateComponentListNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateBinaryHardeningListNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGeneratePasswordHashListNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateCveListNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateCryptoCertificateListNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listGenerateCryptoKeyListNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByWorkspaceNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -462,19 +159,15 @@ Mono> listGenerateCryptoKeyListNext( * @return list of firmwares along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByWorkspaceSinglePageAsync( - String resourceGroupName, String workspaceName) { + private Mono> listByWorkspaceSinglePageAsync(String resourceGroupName, + String workspaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -485,32 +178,16 @@ private Mono> listByWorkspaceSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByWorkspace( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByWorkspace(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -520,19 +197,15 @@ private Mono> listByWorkspaceSinglePageAsync( * @return list of firmwares along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByWorkspaceSinglePageAsync( - String resourceGroupName, String workspaceName, Context context) { + private Mono> listByWorkspaceSinglePageAsync(String resourceGroupName, + String workspaceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -544,28 +217,15 @@ private Mono> listByWorkspaceSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByWorkspace( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByWorkspace(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -575,14 +235,13 @@ private Mono> listByWorkspaceSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByWorkspaceAsync(String resourceGroupName, String workspaceName) { - return new PagedFlux<>( - () -> listByWorkspaceSinglePageAsync(resourceGroupName, workspaceName), + return new PagedFlux<>(() -> listByWorkspaceSinglePageAsync(resourceGroupName, workspaceName), nextLink -> listByWorkspaceNextSinglePageAsync(nextLink)); } /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -592,16 +251,15 @@ private PagedFlux listByWorkspaceAsync(String resourceGroupName, * @return list of firmwares as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByWorkspaceAsync( - String resourceGroupName, String workspaceName, Context context) { - return new PagedFlux<>( - () -> listByWorkspaceSinglePageAsync(resourceGroupName, workspaceName, context), + private PagedFlux listByWorkspaceAsync(String resourceGroupName, String workspaceName, + Context context) { + return new PagedFlux<>(() -> listByWorkspaceSinglePageAsync(resourceGroupName, workspaceName, context), nextLink -> listByWorkspaceNextSinglePageAsync(nextLink, context)); } /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -616,7 +274,7 @@ public PagedIterable listByWorkspace(String resourceGroupName, St /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -626,14 +284,14 @@ public PagedIterable listByWorkspace(String resourceGroupName, St * @return list of firmwares as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByWorkspace( - String resourceGroupName, String workspaceName, Context context) { + public PagedIterable listByWorkspace(String resourceGroupName, String workspaceName, + Context context) { return new PagedIterable<>(listByWorkspaceAsync(resourceGroupName, workspaceName, context)); } /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -644,19 +302,15 @@ public PagedIterable listByWorkspace( * @return firmware definition along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareInner firmware) { + private Mono> createWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, FirmwareInner firmware) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -675,25 +329,14 @@ private Mono> createWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - firmware, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), firmware, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -705,19 +348,15 @@ private Mono> createWithResponseAsync( * @return firmware definition along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareInner firmware, Context context) { + private Mono> createWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, FirmwareInner firmware, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -736,22 +375,13 @@ private Mono> createWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - firmware, - accept, - context); + return service.create(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), firmware, accept, context); } /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -762,15 +392,15 @@ private Mono> createWithResponseAsync( * @return firmware definition on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareInner firmware) { + private Mono createAsync(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareInner firmware) { return createWithResponseAsync(resourceGroupName, workspaceName, firmwareId, firmware) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -782,14 +412,14 @@ private Mono createAsync( * @return firmware definition along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareInner firmware, Context context) { + public Response createWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareInner firmware, Context context) { return createWithResponseAsync(resourceGroupName, workspaceName, firmwareId, firmware, context).block(); } /** * The operation to create a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -800,14 +430,14 @@ public Response createWithResponse( * @return firmware definition. */ @ServiceMethod(returns = ReturnType.SINGLE) - public FirmwareInner create( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareInner firmware) { + public FirmwareInner create(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareInner firmware) { return createWithResponse(resourceGroupName, workspaceName, firmwareId, firmware, Context.NONE).getValue(); } /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -818,19 +448,15 @@ public FirmwareInner create( * @return firmware definition along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareUpdateDefinition firmware) { + private Mono> updateWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, FirmwareUpdateDefinition firmware) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -849,25 +475,14 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - firmware, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), firmware, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -879,23 +494,15 @@ private Mono> updateWithResponseAsync( * @return firmware definition along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String workspaceName, - String firmwareId, - FirmwareUpdateDefinition firmware, - Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, FirmwareUpdateDefinition firmware, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -914,22 +521,13 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - firmware, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), firmware, accept, context); } /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -940,15 +538,15 @@ private Mono> updateWithResponseAsync( * @return firmware definition on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareUpdateDefinition firmware) { + private Mono updateAsync(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareUpdateDefinition firmware) { return updateWithResponseAsync(resourceGroupName, workspaceName, firmwareId, firmware) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -960,18 +558,14 @@ private Mono updateAsync( * @return firmware definition along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String workspaceName, - String firmwareId, - FirmwareUpdateDefinition firmware, - Context context) { + public Response updateWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareUpdateDefinition firmware, Context context) { return updateWithResponseAsync(resourceGroupName, workspaceName, firmwareId, firmware, context).block(); } /** * The operation to update firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -982,14 +576,14 @@ public Response updateWithResponse( * @return firmware definition. */ @ServiceMethod(returns = ReturnType.SINGLE) - public FirmwareInner update( - String resourceGroupName, String workspaceName, String firmwareId, FirmwareUpdateDefinition firmware) { + public FirmwareInner update(String resourceGroupName, String workspaceName, String firmwareId, + FirmwareUpdateDefinition firmware) { return updateWithResponse(resourceGroupName, workspaceName, firmwareId, firmware, Context.NONE).getValue(); } /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -999,19 +593,15 @@ public FirmwareInner update( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1025,24 +615,14 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1053,19 +633,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1079,21 +655,13 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context); } /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1109,7 +677,7 @@ private Mono deleteAsync(String resourceGroupName, String workspaceName, S /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1120,14 +688,14 @@ private Mono deleteAsync(String resourceGroupName, String workspaceName, S * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + public Response deleteWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context) { return deleteWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); } /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1142,7 +710,7 @@ public void delete(String resourceGroupName, String workspaceName, String firmwa /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1152,19 +720,15 @@ public void delete(String resourceGroupName, String workspaceName, String firmwa * @return firmware along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { + private Mono> getWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1178,24 +742,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1206,19 +760,15 @@ private Mono> getWithResponseAsync( * @return firmware along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1232,21 +782,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, workspaceName, + firmwareId, this.client.getApiVersion(), accept, context); } /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1263,7 +805,7 @@ private Mono getAsync(String resourceGroupName, String workspaceN /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1274,14 +816,14 @@ private Mono getAsync(String resourceGroupName, String workspaceN * @return firmware along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + public Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context) { return getWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); } /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1297,7 +839,7 @@ public FirmwareInner get(String resourceGroupName, String workspaceName, String /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1305,22 +847,18 @@ public FirmwareInner get(String resourceGroupName, String workspaceName, String * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateDownloadUrlWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { + private Mono> generateDownloadUrlWithResponseAsync(String resourceGroupName, + String workspaceName, String firmwareId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1335,23 +873,14 @@ private Mono> generateDownloadUrlWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .generateDownloadUrl( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) + context -> service.generateDownloadUrl(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1360,22 +889,18 @@ private Mono> generateDownloadUrlWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateDownloadUrlWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + private Mono> generateDownloadUrlWithResponseAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1389,21 +914,13 @@ private Mono> generateDownloadUrlWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .generateDownloadUrl( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); + return service.generateDownloadUrl(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context); } /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1413,15 +930,15 @@ private Mono> generateDownloadUrlWithResponseAsync( * @return url data for creating or accessing a blob file on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateDownloadUrlAsync( - String resourceGroupName, String workspaceName, String firmwareId) { + private Mono generateDownloadUrlAsync(String resourceGroupName, String workspaceName, + String firmwareId) { return generateDownloadUrlWithResponseAsync(resourceGroupName, workspaceName, firmwareId) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1432,14 +949,14 @@ private Mono generateDownloadUrlAsync( * @return url data for creating or accessing a blob file along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + public Response generateDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { return generateDownloadUrlWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); } /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1455,7 +972,7 @@ public UrlTokenInner generateDownloadUrl(String resourceGroupName, String worksp /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1463,22 +980,18 @@ public UrlTokenInner generateDownloadUrl(String resourceGroupName, String worksp * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateFilesystemDownloadUrlWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { + private Mono> generateFilesystemDownloadUrlWithResponseAsync(String resourceGroupName, + String workspaceName, String firmwareId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1492,24 +1005,15 @@ private Mono> generateFilesystemDownloadUrlWithResponseA } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .generateFilesystemDownloadUrl( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.generateFilesystemDownloadUrl(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, workspaceName, firmwareId, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1518,22 +1022,18 @@ private Mono> generateFilesystemDownloadUrlWithResponseA * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateFilesystemDownloadUrlWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + private Mono> generateFilesystemDownloadUrlWithResponseAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1547,21 +1047,13 @@ private Mono> generateFilesystemDownloadUrlWithResponseA } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .generateFilesystemDownloadUrl( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); + return service.generateFilesystemDownloadUrl(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context); } /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1571,15 +1063,15 @@ private Mono> generateFilesystemDownloadUrlWithResponseA * @return url data for creating or accessing a blob file on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateFilesystemDownloadUrlAsync( - String resourceGroupName, String workspaceName, String firmwareId) { + private Mono generateFilesystemDownloadUrlAsync(String resourceGroupName, String workspaceName, + String firmwareId) { return generateFilesystemDownloadUrlWithResponseAsync(resourceGroupName, workspaceName, firmwareId) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1590,15 +1082,15 @@ private Mono generateFilesystemDownloadUrlAsync( * @return url data for creating or accessing a blob file along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateFilesystemDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + public Response generateFilesystemDownloadUrlWithResponse(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { return generateFilesystemDownloadUrlWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context) .block(); } /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -1608,2843 +1100,65 @@ public Response generateFilesystemDownloadUrlWithResponse( * @return url data for creating or accessing a blob file. */ @ServiceMethod(returns = ReturnType.SINGLE) - public UrlTokenInner generateFilesystemDownloadUrl( - String resourceGroupName, String workspaceName, String firmwareId) { + public UrlTokenInner generateFilesystemDownloadUrl(String resourceGroupName, String workspaceName, + String firmwareId) { return generateFilesystemDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) .getValue(); } /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware along with {@link Response} on successful completion of {@link - * Mono}. + * @return list of firmwares along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + private Mono> listByWorkspaceNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .generateSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.listByWorkspaceNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware along with {@link Response} on successful completion of {@link - * Mono}. + * @return list of firmwares along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + private Mono> listByWorkspaceNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateSummaryAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); - } - - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public FirmwareSummaryInner generateSummary(String resourceGroupName, String workspaceName, String firmwareId) { - return generateSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE).getValue(); - } - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateComponentListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listGenerateComponentList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateComponentListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateComponentList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateComponentListAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedFlux<>( - () -> listGenerateComponentListSinglePageAsync(resourceGroupName, workspaceName, firmwareId), - nextLink -> listGenerateComponentListNextSinglePageAsync(nextLink)); - } - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateComponentListAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedFlux<>( - () -> listGenerateComponentListSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), - nextLink -> listGenerateComponentListNextSinglePageAsync(nextLink, context)); - } - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedIterable<>(listGenerateComponentListAsync(resourceGroupName, workspaceName, firmwareId)); - } - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedIterable<>( - listGenerateComponentListAsync(resourceGroupName, workspaceName, firmwareId, context)); - } - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateComponentDetailsWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .generateComponentDetails( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateComponentDetailsWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateComponentDetails( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateComponentDetailsAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateComponentDetailsWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateComponentDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateComponentDetailsWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); - } - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public ComponentInner generateComponentDetails(String resourceGroupName, String workspaceName, String firmwareId) { - return generateComponentDetailsWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) - .getValue(); - } - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening along with {@link PagedResponse} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateBinaryHardeningListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listGenerateBinaryHardeningList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening along with {@link PagedResponse} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateBinaryHardeningListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateBinaryHardeningList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateBinaryHardeningListAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedFlux<>( - () -> listGenerateBinaryHardeningListSinglePageAsync(resourceGroupName, workspaceName, firmwareId), - nextLink -> listGenerateBinaryHardeningListNextSinglePageAsync(nextLink)); - } - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateBinaryHardeningListAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedFlux<>( - () -> listGenerateBinaryHardeningListSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), - nextLink -> listGenerateBinaryHardeningListNextSinglePageAsync(nextLink, context)); - } - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedIterable<>(listGenerateBinaryHardeningListAsync(resourceGroupName, workspaceName, firmwareId)); - } - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedIterable<>( - listGenerateBinaryHardeningListAsync(resourceGroupName, workspaceName, firmwareId, context)); - } - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages along with {@link Response} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateBinaryHardeningSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .generateBinaryHardeningSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages along with {@link Response} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateBinaryHardeningSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateBinaryHardeningSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateBinaryHardeningSummaryAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateBinaryHardeningSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateBinaryHardeningSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateBinaryHardeningSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context) - .block(); - } - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BinaryHardeningSummaryInner generateBinaryHardeningSummary( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateBinaryHardeningSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) - .getValue(); - } - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateBinaryHardeningDetailsWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .generateBinaryHardeningDetails( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateBinaryHardeningDetailsWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateBinaryHardeningDetails( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateBinaryHardeningDetailsAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateBinaryHardeningDetailsWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateBinaryHardeningDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateBinaryHardeningDetailsWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context) - .block(); - } - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BinaryHardeningInner generateBinaryHardeningDetails( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateBinaryHardeningDetailsWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) - .getValue(); - } - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGeneratePasswordHashListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listGeneratePasswordHashList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGeneratePasswordHashListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGeneratePasswordHashList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGeneratePasswordHashListAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedFlux<>( - () -> listGeneratePasswordHashListSinglePageAsync(resourceGroupName, workspaceName, firmwareId), - nextLink -> listGeneratePasswordHashListNextSinglePageAsync(nextLink)); - } - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGeneratePasswordHashListAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedFlux<>( - () -> listGeneratePasswordHashListSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), - nextLink -> listGeneratePasswordHashListNextSinglePageAsync(nextLink, context)); - } - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedIterable<>(listGeneratePasswordHashListAsync(resourceGroupName, workspaceName, firmwareId)); - } - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedIterable<>( - listGeneratePasswordHashListAsync(resourceGroupName, workspaceName, firmwareId, context)); - } - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCveListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listGenerateCveList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCveListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateCveList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateCveListAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedFlux<>( - () -> listGenerateCveListSinglePageAsync(resourceGroupName, workspaceName, firmwareId), - nextLink -> listGenerateCveListNextSinglePageAsync(nextLink)); - } - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateCveListAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedFlux<>( - () -> listGenerateCveListSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), - nextLink -> listGenerateCveListNextSinglePageAsync(nextLink, context)); - } - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateCveList( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedIterable<>(listGenerateCveListAsync(resourceGroupName, workspaceName, firmwareId)); - } - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateCveList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedIterable<>(listGenerateCveListAsync(resourceGroupName, workspaceName, firmwareId, context)); - } - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateCveSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .generateCveSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateCveSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateCveSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateCveSummaryAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateCveSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateCveSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateCveSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); - } - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public CveSummaryInner generateCveSummary(String resourceGroupName, String workspaceName, String firmwareId) { - return generateCveSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE).getValue(); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values along with {@link Response} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateCryptoCertificateSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .generateCryptoCertificateSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values along with {@link Response} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateCryptoCertificateSummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateCryptoCertificateSummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateCryptoCertificateSummaryAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateCryptoCertificateSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateCryptoCertificateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateCryptoCertificateSummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context) - .block(); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public CryptoCertificateSummaryInner generateCryptoCertificateSummary( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateCryptoCertificateSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) - .getValue(); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateCryptoKeySummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .generateCryptoKeySummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateCryptoKeySummaryWithResponseAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .generateCryptoKeySummary( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateCryptoKeySummaryAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateCryptoKeySummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateCryptoKeySummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return generateCryptoKeySummaryWithResponseAsync(resourceGroupName, workspaceName, firmwareId, context).block(); - } - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public CryptoKeySummaryInner generateCryptoKeySummary( - String resourceGroupName, String workspaceName, String firmwareId) { - return generateCryptoKeySummaryWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE) - .getValue(); - } - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoCertificateListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listGenerateCryptoCertificateList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoCertificateListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateCryptoCertificateList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateCryptoCertificateListAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedFlux<>( - () -> listGenerateCryptoCertificateListSinglePageAsync(resourceGroupName, workspaceName, firmwareId), - nextLink -> listGenerateCryptoCertificateListNextSinglePageAsync(nextLink)); - } - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateCryptoCertificateListAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedFlux<>( - () -> - listGenerateCryptoCertificateListSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), - nextLink -> listGenerateCryptoCertificateListNextSinglePageAsync(nextLink, context)); - } - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedIterable<>( - listGenerateCryptoCertificateListAsync(resourceGroupName, workspaceName, firmwareId)); - } - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedIterable<>( - listGenerateCryptoCertificateListAsync(resourceGroupName, workspaceName, firmwareId, context)); - } - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoKeyListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listGenerateCryptoKeyList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoKeyListSinglePageAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (workspaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); - } - if (firmwareId == null) { - return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateCryptoKeyList( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - firmwareId, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateCryptoKeyListAsync( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedFlux<>( - () -> listGenerateCryptoKeyListSinglePageAsync(resourceGroupName, workspaceName, firmwareId), - nextLink -> listGenerateCryptoKeyListNextSinglePageAsync(nextLink)); - } - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listGenerateCryptoKeyListAsync( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedFlux<>( - () -> listGenerateCryptoKeyListSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), - nextLink -> listGenerateCryptoKeyListNextSinglePageAsync(nextLink, context)); - } - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId) { - return new PagedIterable<>(listGenerateCryptoKeyListAsync(resourceGroupName, workspaceName, firmwareId)); - } - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - return new PagedIterable<>( - listGenerateCryptoKeyListAsync(resourceGroupName, workspaceName, firmwareId, context)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of firmwares along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByWorkspaceNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listByWorkspaceNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of firmwares along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByWorkspaceNextSinglePageAsync(String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listByWorkspaceNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateComponentListNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.listGenerateComponentListNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateComponentListNextSinglePageAsync( - String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateComponentListNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening along with {@link PagedResponse} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateBinaryHardeningListNextSinglePageAsync( - String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listGenerateBinaryHardeningListNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening along with {@link PagedResponse} on successful completion of {@link - * Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateBinaryHardeningListNextSinglePageAsync( - String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateBinaryHardeningListNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGeneratePasswordHashListNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listGeneratePasswordHashListNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGeneratePasswordHashListNextSinglePageAsync( - String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGeneratePasswordHashListNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCveListNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.listGenerateCveListNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCveListNextSinglePageAsync(String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateCveListNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoCertificateListNextSinglePageAsync( - String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listGenerateCryptoCertificateListNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoCertificateListNextSinglePageAsync( - String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listGenerateCryptoCertificateListNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoKeyListNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.listGenerateCryptoKeyListNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listGenerateCryptoKeyListNextSinglePageAsync( - String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listGenerateCryptoKeyListNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByWorkspaceNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresImpl.java index f63dcadf1b934..9260ad59c715e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FirmwaresImpl.java @@ -10,32 +10,10 @@ import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.iotfirmwaredefense.fluent.FirmwaresClient; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeySummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.UrlTokenInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardening; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningSummary; -import com.azure.resourcemanager.iotfirmwaredefense.models.Component; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificate; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateSummary; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKey; -import com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeySummary; -import com.azure.resourcemanager.iotfirmwaredefense.models.Cve; -import com.azure.resourcemanager.iotfirmwaredefense.models.CveSummary; import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareSummary; import com.azure.resourcemanager.iotfirmwaredefense.models.Firmwares; -import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHash; import com.azure.resourcemanager.iotfirmwaredefense.models.UrlToken; public final class FirmwaresImpl implements Firmwares { @@ -45,8 +23,7 @@ public final class FirmwaresImpl implements Firmwares { private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - public FirmwaresImpl( - FirmwaresClient innerClient, + public FirmwaresImpl(FirmwaresClient innerClient, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -54,17 +31,17 @@ public FirmwaresImpl( public PagedIterable listByWorkspace(String resourceGroupName, String workspaceName) { PagedIterable inner = this.serviceClient().listByWorkspace(resourceGroupName, workspaceName); - return Utils.mapPage(inner, inner1 -> new FirmwareImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new FirmwareImpl(inner1, this.manager())); } public PagedIterable listByWorkspace(String resourceGroupName, String workspaceName, Context context) { - PagedIterable inner = - this.serviceClient().listByWorkspace(resourceGroupName, workspaceName, context); - return Utils.mapPage(inner, inner1 -> new FirmwareImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByWorkspace(resourceGroupName, workspaceName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new FirmwareImpl(inner1, this.manager())); } - public Response deleteWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { + public Response deleteWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, workspaceName, firmwareId, context); } @@ -72,15 +49,12 @@ public void delete(String resourceGroupName, String workspaceName, String firmwa this.serviceClient().delete(resourceGroupName, workspaceName, firmwareId); } - public Response getWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, workspaceName, firmwareId, context); + public Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, workspaceName, firmwareId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new FirmwareImpl(inner.getValue(), this.manager())); } else { return null; @@ -96,15 +70,12 @@ public Firmware get(String resourceGroupName, String workspaceName, String firmw } } - public Response generateDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this.serviceClient().generateDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, context); + public Response generateDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + Response inner = this.serviceClient().generateDownloadUrlWithResponse(resourceGroupName, + workspaceName, firmwareId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new UrlTokenImpl(inner.getValue(), this.manager())); } else { return null; @@ -120,17 +91,12 @@ public UrlToken generateDownloadUrl(String resourceGroupName, String workspaceNa } } - public Response generateFilesystemDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this - .serviceClient() - .generateFilesystemDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, context); + public Response generateFilesystemDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + Response inner = this.serviceClient() + .generateFilesystemDownloadUrlWithResponse(resourceGroupName, workspaceName, firmwareId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new UrlTokenImpl(inner.getValue(), this.manager())); } else { return null; @@ -138,8 +104,8 @@ public Response generateFilesystemDownloadUrlWithResponse( } public UrlToken generateFilesystemDownloadUrl(String resourceGroupName, String workspaceName, String firmwareId) { - UrlTokenInner inner = - this.serviceClient().generateFilesystemDownloadUrl(resourceGroupName, workspaceName, firmwareId); + UrlTokenInner inner + = this.serviceClient().generateFilesystemDownloadUrl(resourceGroupName, workspaceName, firmwareId); if (inner != null) { return new UrlTokenImpl(inner, this.manager()); } else { @@ -147,378 +113,78 @@ public UrlToken generateFilesystemDownloadUrl(String resourceGroupName, String w } } - public Response generateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this.serviceClient().generateSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new FirmwareSummaryImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public FirmwareSummary generateSummary(String resourceGroupName, String workspaceName, String firmwareId) { - FirmwareSummaryInner inner = this.serviceClient().generateSummary(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new FirmwareSummaryImpl(inner, this.manager()); - } else { - return null; - } - } - - public PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId) { - PagedIterable inner = - this.serviceClient().listGenerateComponentList(resourceGroupName, workspaceName, firmwareId); - return Utils.mapPage(inner, inner1 -> new ComponentImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - PagedIterable inner = - this.serviceClient().listGenerateComponentList(resourceGroupName, workspaceName, firmwareId, context); - return Utils.mapPage(inner, inner1 -> new ComponentImpl(inner1, this.manager())); - } - - public Response generateComponentDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this - .serviceClient() - .generateComponentDetailsWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ComponentImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public Component generateComponentDetails(String resourceGroupName, String workspaceName, String firmwareId) { - ComponentInner inner = - this.serviceClient().generateComponentDetails(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new ComponentImpl(inner, this.manager()); - } else { - return null; - } - } - - public PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId) { - PagedIterable inner = - this.serviceClient().listGenerateBinaryHardeningList(resourceGroupName, workspaceName, firmwareId); - return Utils.mapPage(inner, inner1 -> new BinaryHardeningImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - PagedIterable inner = - this.serviceClient().listGenerateBinaryHardeningList(resourceGroupName, workspaceName, firmwareId, context); - return Utils.mapPage(inner, inner1 -> new BinaryHardeningImpl(inner1, this.manager())); - } - - public Response generateBinaryHardeningSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this - .serviceClient() - .generateBinaryHardeningSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new BinaryHardeningSummaryImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public BinaryHardeningSummary generateBinaryHardeningSummary( - String resourceGroupName, String workspaceName, String firmwareId) { - BinaryHardeningSummaryInner inner = - this.serviceClient().generateBinaryHardeningSummary(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new BinaryHardeningSummaryImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response generateBinaryHardeningDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this - .serviceClient() - .generateBinaryHardeningDetailsWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new BinaryHardeningImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public BinaryHardening generateBinaryHardeningDetails( - String resourceGroupName, String workspaceName, String firmwareId) { - BinaryHardeningInner inner = - this.serviceClient().generateBinaryHardeningDetails(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new BinaryHardeningImpl(inner, this.manager()); - } else { - return null; - } - } - - public PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId) { - PagedIterable inner = - this.serviceClient().listGeneratePasswordHashList(resourceGroupName, workspaceName, firmwareId); - return Utils.mapPage(inner, inner1 -> new PasswordHashImpl(inner1, this.manager())); - } - - public PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - PagedIterable inner = - this.serviceClient().listGeneratePasswordHashList(resourceGroupName, workspaceName, firmwareId, context); - return Utils.mapPage(inner, inner1 -> new PasswordHashImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateCveList(String resourceGroupName, String workspaceName, String firmwareId) { - PagedIterable inner = - this.serviceClient().listGenerateCveList(resourceGroupName, workspaceName, firmwareId); - return Utils.mapPage(inner, inner1 -> new CveImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateCveList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - PagedIterable inner = - this.serviceClient().listGenerateCveList(resourceGroupName, workspaceName, firmwareId, context); - return Utils.mapPage(inner, inner1 -> new CveImpl(inner1, this.manager())); - } - - public Response generateCveSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this.serviceClient().generateCveSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CveSummaryImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public CveSummary generateCveSummary(String resourceGroupName, String workspaceName, String firmwareId) { - CveSummaryInner inner = this.serviceClient().generateCveSummary(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new CveSummaryImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response generateCryptoCertificateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this - .serviceClient() - .generateCryptoCertificateSummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CryptoCertificateSummaryImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public CryptoCertificateSummary generateCryptoCertificateSummary( - String resourceGroupName, String workspaceName, String firmwareId) { - CryptoCertificateSummaryInner inner = - this.serviceClient().generateCryptoCertificateSummary(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new CryptoCertificateSummaryImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response generateCryptoKeySummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - Response inner = - this - .serviceClient() - .generateCryptoKeySummaryWithResponse(resourceGroupName, workspaceName, firmwareId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CryptoKeySummaryImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public CryptoKeySummary generateCryptoKeySummary( - String resourceGroupName, String workspaceName, String firmwareId) { - CryptoKeySummaryInner inner = - this.serviceClient().generateCryptoKeySummary(resourceGroupName, workspaceName, firmwareId); - if (inner != null) { - return new CryptoKeySummaryImpl(inner, this.manager()); - } else { - return null; - } - } - - public PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId) { - PagedIterable inner = - this.serviceClient().listGenerateCryptoCertificateList(resourceGroupName, workspaceName, firmwareId); - return Utils.mapPage(inner, inner1 -> new CryptoCertificateImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - PagedIterable inner = - this - .serviceClient() - .listGenerateCryptoCertificateList(resourceGroupName, workspaceName, firmwareId, context); - return Utils.mapPage(inner, inner1 -> new CryptoCertificateImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId) { - PagedIterable inner = - this.serviceClient().listGenerateCryptoKeyList(resourceGroupName, workspaceName, firmwareId); - return Utils.mapPage(inner, inner1 -> new CryptoKeyImpl(inner1, this.manager())); - } - - public PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId, Context context) { - PagedIterable inner = - this.serviceClient().listGenerateCryptoKeyList(resourceGroupName, workspaceName, firmwareId, context); - return Utils.mapPage(inner, inner1 -> new CryptoKeyImpl(inner1, this.manager())); - } - public Firmware getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } - String firmwareId = Utils.getValueFromIdByName(id, "firmwares"); + String firmwareId = ResourceManagerUtils.getValueFromIdByName(id, "firmwares"); if (firmwareId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); } return this.getWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } - String firmwareId = Utils.getValueFromIdByName(id, "firmwares"); + String firmwareId = ResourceManagerUtils.getValueFromIdByName(id, "firmwares"); if (firmwareId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); } return this.getWithResponse(resourceGroupName, workspaceName, firmwareId, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } - String firmwareId = Utils.getValueFromIdByName(id, "firmwares"); + String firmwareId = ResourceManagerUtils.getValueFromIdByName(id, "firmwares"); if (firmwareId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); } this.deleteWithResponse(resourceGroupName, workspaceName, firmwareId, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } - String firmwareId = Utils.getValueFromIdByName(id, "firmwares"); + String firmwareId = ResourceManagerUtils.getValueFromIdByName(id, "firmwares"); if (firmwareId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'firmwares'.", id))); } return this.deleteWithResponse(resourceGroupName, workspaceName, firmwareId, context); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FistBuilder.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/IoTFirmwareDefenseBuilder.java similarity index 56% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FistBuilder.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/IoTFirmwareDefenseBuilder.java index e0b079f1fc627..ca8bc5661dd61 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FistBuilder.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/IoTFirmwareDefenseBuilder.java @@ -14,21 +14,23 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the FistImpl type. */ -@ServiceClientBuilder(serviceClients = {FistImpl.class}) -public final class FistBuilder { +/** + * A builder for creating a new instance of the IoTFirmwareDefenseImpl type. + */ +@ServiceClientBuilder(serviceClients = { IoTFirmwareDefenseImpl.class }) +public final class IoTFirmwareDefenseBuilder { /* - * The ID of the target subscription. + * The ID of the target subscription. The value must be an UUID. */ private String subscriptionId; /** - * Sets The ID of the target subscription. - * + * Sets The ID of the target subscription. The value must be an UUID. + * * @param subscriptionId the subscriptionId value. - * @return the FistBuilder. + * @return the IoTFirmwareDefenseBuilder. */ - public FistBuilder subscriptionId(String subscriptionId) { + public IoTFirmwareDefenseBuilder subscriptionId(String subscriptionId) { this.subscriptionId = subscriptionId; return this; } @@ -40,11 +42,11 @@ public FistBuilder subscriptionId(String subscriptionId) { /** * Sets server parameter. - * + * * @param endpoint the endpoint value. - * @return the FistBuilder. + * @return the IoTFirmwareDefenseBuilder. */ - public FistBuilder endpoint(String endpoint) { + public IoTFirmwareDefenseBuilder endpoint(String endpoint) { this.endpoint = endpoint; return this; } @@ -56,11 +58,11 @@ public FistBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. - * @return the FistBuilder. + * @return the IoTFirmwareDefenseBuilder. */ - public FistBuilder environment(AzureEnvironment environment) { + public IoTFirmwareDefenseBuilder environment(AzureEnvironment environment) { this.environment = environment; return this; } @@ -72,11 +74,11 @@ public FistBuilder environment(AzureEnvironment environment) { /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. - * @return the FistBuilder. + * @return the IoTFirmwareDefenseBuilder. */ - public FistBuilder pipeline(HttpPipeline pipeline) { + public IoTFirmwareDefenseBuilder pipeline(HttpPipeline pipeline) { this.pipeline = pipeline; return this; } @@ -88,11 +90,11 @@ public FistBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. - * @return the FistBuilder. + * @return the IoTFirmwareDefenseBuilder. */ - public FistBuilder defaultPollInterval(Duration defaultPollInterval) { + public IoTFirmwareDefenseBuilder defaultPollInterval(Duration defaultPollInterval) { this.defaultPollInterval = defaultPollInterval; return this; } @@ -104,41 +106,31 @@ public FistBuilder defaultPollInterval(Duration defaultPollInterval) { /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. - * @return the FistBuilder. + * @return the IoTFirmwareDefenseBuilder. */ - public FistBuilder serializerAdapter(SerializerAdapter serializerAdapter) { + public IoTFirmwareDefenseBuilder serializerAdapter(SerializerAdapter serializerAdapter) { this.serializerAdapter = serializerAdapter; return this; } /** - * Builds an instance of FistImpl with the provided parameters. - * - * @return an instance of FistImpl. + * Builds an instance of IoTFirmwareDefenseImpl with the provided parameters. + * + * @return an instance of IoTFirmwareDefenseImpl. */ - public FistImpl buildClient() { + public IoTFirmwareDefenseImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - FistImpl client = - new FistImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + IoTFirmwareDefenseImpl client = new IoTFirmwareDefenseImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FistImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/IoTFirmwareDefenseImpl.java similarity index 60% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FistImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/IoTFirmwareDefenseImpl.java index 8dc1d97ec132a..a5afad0368ff6 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/FistImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/IoTFirmwareDefenseImpl.java @@ -22,9 +22,16 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.serializer.SerializerAdapter; import com.azure.core.util.serializer.SerializerEncoding; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.BinaryHardeningsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CryptoCertificatesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CryptoKeysClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.CvesClient; import com.azure.resourcemanager.iotfirmwaredefense.fluent.FirmwaresClient; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.Fist; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.IoTFirmwareDefense; import com.azure.resourcemanager.iotfirmwaredefense.fluent.OperationsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.PasswordHashesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.SbomComponentsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.SummariesClient; import com.azure.resourcemanager.iotfirmwaredefense.fluent.WorkspacesClient; import java.io.IOException; import java.lang.reflect.Type; @@ -35,111 +42,173 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the FistImpl type. */ -@ServiceClient(builder = FistBuilder.class) -public final class FistImpl implements Fist { - /** The ID of the target subscription. */ +/** + * Initializes a new instance of the IoTFirmwareDefenseImpl type. + */ +@ServiceClient(builder = IoTFirmwareDefenseBuilder.class) +public final class IoTFirmwareDefenseImpl implements IoTFirmwareDefense { + /** + * The ID of the target subscription. The value must be an UUID. + */ private final String subscriptionId; /** - * Gets The ID of the target subscription. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The FirmwaresClient object to access its operations. */ + /** + * The BinaryHardeningsClient object to access its operations. + */ + private final BinaryHardeningsClient binaryHardenings; + + /** + * Gets the BinaryHardeningsClient object to access its operations. + * + * @return the BinaryHardeningsClient object. + */ + public BinaryHardeningsClient getBinaryHardenings() { + return this.binaryHardenings; + } + + /** + * The CryptoCertificatesClient object to access its operations. + */ + private final CryptoCertificatesClient cryptoCertificates; + + /** + * Gets the CryptoCertificatesClient object to access its operations. + * + * @return the CryptoCertificatesClient object. + */ + public CryptoCertificatesClient getCryptoCertificates() { + return this.cryptoCertificates; + } + + /** + * The CryptoKeysClient object to access its operations. + */ + private final CryptoKeysClient cryptoKeys; + + /** + * Gets the CryptoKeysClient object to access its operations. + * + * @return the CryptoKeysClient object. + */ + public CryptoKeysClient getCryptoKeys() { + return this.cryptoKeys; + } + + /** + * The CvesClient object to access its operations. + */ + private final CvesClient cves; + + /** + * Gets the CvesClient object to access its operations. + * + * @return the CvesClient object. + */ + public CvesClient getCves() { + return this.cves; + } + + /** + * The FirmwaresClient object to access its operations. + */ private final FirmwaresClient firmwares; /** * Gets the FirmwaresClient object to access its operations. - * + * * @return the FirmwaresClient object. */ public FirmwaresClient getFirmwares() { return this.firmwares; } - /** The WorkspacesClient object to access its operations. */ - private final WorkspacesClient workspaces; - /** - * Gets the WorkspacesClient object to access its operations. - * - * @return the WorkspacesClient object. + * The OperationsClient object to access its operations. */ - public WorkspacesClient getWorkspaces() { - return this.workspaces; - } - - /** The OperationsClient object to access its operations. */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { @@ -147,36 +216,94 @@ public OperationsClient getOperations() { } /** - * Initializes an instance of Fist client. - * + * The PasswordHashesClient object to access its operations. + */ + private final PasswordHashesClient passwordHashes; + + /** + * Gets the PasswordHashesClient object to access its operations. + * + * @return the PasswordHashesClient object. + */ + public PasswordHashesClient getPasswordHashes() { + return this.passwordHashes; + } + + /** + * The SbomComponentsClient object to access its operations. + */ + private final SbomComponentsClient sbomComponents; + + /** + * Gets the SbomComponentsClient object to access its operations. + * + * @return the SbomComponentsClient object. + */ + public SbomComponentsClient getSbomComponents() { + return this.sbomComponents; + } + + /** + * The SummariesClient object to access its operations. + */ + private final SummariesClient summaries; + + /** + * Gets the SummariesClient object to access its operations. + * + * @return the SummariesClient object. + */ + public SummariesClient getSummaries() { + return this.summaries; + } + + /** + * The WorkspacesClient object to access its operations. + */ + private final WorkspacesClient workspaces; + + /** + * Gets the WorkspacesClient object to access its operations. + * + * @return the WorkspacesClient object. + */ + public WorkspacesClient getWorkspaces() { + return this.workspaces; + } + + /** + * Initializes an instance of IoTFirmwareDefense client. + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. * @param environment The Azure environment. - * @param subscriptionId The ID of the target subscription. + * @param subscriptionId The ID of the target subscription. The value must be an UUID. * @param endpoint server parameter. */ - FistImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + IoTFirmwareDefenseImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, Duration defaultPollInterval, + AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-02-08-preview"; + this.apiVersion = "2024-01-10"; + this.binaryHardenings = new BinaryHardeningsClientImpl(this); + this.cryptoCertificates = new CryptoCertificatesClientImpl(this); + this.cryptoKeys = new CryptoKeysClientImpl(this); + this.cves = new CvesClientImpl(this); this.firmwares = new FirmwaresClientImpl(this); - this.workspaces = new WorkspacesClientImpl(this); this.operations = new OperationsClientImpl(this); + this.passwordHashes = new PasswordHashesClientImpl(this); + this.sbomComponents = new SbomComponentsClientImpl(this); + this.summaries = new SummariesClientImpl(this); + this.workspaces = new WorkspacesClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -185,7 +312,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -195,7 +322,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -205,26 +332,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -237,19 +353,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } @@ -314,5 +427,5 @@ public Mono getBodyAsString(Charset charset) { } } - private static final ClientLogger LOGGER = new ClientLogger(FistImpl.class); + private static final ClientLogger LOGGER = new ClientLogger(IoTFirmwareDefenseImpl.class); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationImpl.java index a45d8c1bc2e25..352f62e462e3a 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationImpl.java @@ -15,8 +15,7 @@ public final class OperationImpl implements Operation { private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - OperationImpl( - OperationInner innerObject, + OperationImpl(OperationInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsClientImpl.java index 6e3ade631ead6..cdeb210ecf183 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsClientImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsClientImpl.java @@ -30,124 +30,106 @@ import com.azure.resourcemanager.iotfirmwaredefense.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ - private final FistImpl client; + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - OperationsClientImpl(FistImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + OperationsClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for FistOperations to be used by the proxy service to perform REST calls. + * The interface defining all the services for IoTFirmwareDefenseOperations to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "FistOperations") + @ServiceInterface(name = "IoTFirmwareDefenseOp") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.IoTFirmwareDefense/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Lists the operations for this resource provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists the operations for this resource provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists the operations for this resource provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -156,27 +138,27 @@ private PagedFlux listAsync() { /** * Lists the operations for this resource provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists the operations for this resource provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -185,13 +167,13 @@ public PagedIterable list() { /** * Lists the operations for this resource provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -200,14 +182,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -215,37 +198,28 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -253,23 +227,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsImpl.java index 30c5f7cc91449..1e3554b1d91e0 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/OperationsImpl.java @@ -19,8 +19,7 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, + public OperationsImpl(OperationsClient innerClient, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public OperationsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashResourceImpl.java similarity index 68% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashResourceImpl.java index 8b0632f6d6739..189bc19352ec7 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashResourceImpl.java @@ -4,21 +4,37 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHash; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashResource; -public final class PasswordHashImpl implements PasswordHash { - private PasswordHashInner innerObject; +public final class PasswordHashResourceImpl implements PasswordHashResource { + private PasswordHashResourceInner innerObject; private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - PasswordHashImpl( - PasswordHashInner innerObject, + PasswordHashResourceImpl(PasswordHashResourceInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String passwordHashId() { return this.innerModel().passwordHashId(); } @@ -47,7 +63,7 @@ public String algorithm() { return this.innerModel().algorithm(); } - public PasswordHashInner innerModel() { + public PasswordHashResourceInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesClientImpl.java new file mode 100644 index 0000000000000..a826d2f39aaae --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesClientImpl.java @@ -0,0 +1,296 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.PasswordHashesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashListResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in PasswordHashesClient. + */ +public final class PasswordHashesClientImpl implements PasswordHashesClient { + /** + * The proxy service used to perform REST calls. + */ + private final PasswordHashesService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of PasswordHashesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + PasswordHashesClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(PasswordHashesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefensePasswordHashes to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefensePa") + public interface PasswordHashesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/passwordHashes") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesImpl.java new file mode 100644 index 0000000000000..2e19ab6dee412 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/PasswordHashesImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.PasswordHashesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashes; + +public final class PasswordHashesImpl implements PasswordHashes { + private static final ClientLogger LOGGER = new ClientLogger(PasswordHashesImpl.class); + + private final PasswordHashesClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public PasswordHashesImpl(PasswordHashesClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PasswordHashResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PasswordHashResourceImpl(inner1, this.manager())); + } + + private PasswordHashesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ResourceManagerUtils.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..791a676d80b31 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ComponentImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentResourceImpl.java similarity index 62% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ComponentImpl.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentResourceImpl.java index 31d4d11874322..4561517b7a5a3 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/ComponentImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentResourceImpl.java @@ -4,25 +4,39 @@ package com.azure.resourcemanager.iotfirmwaredefense.implementation; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.Component; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsUpdateAvailable; -import java.time.OffsetDateTime; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponentResource; import java.util.Collections; import java.util.List; -public final class ComponentImpl implements Component { - private ComponentInner innerObject; +public final class SbomComponentResourceImpl implements SbomComponentResource { + private SbomComponentResourceInner innerObject; private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - ComponentImpl( - ComponentInner innerObject, + SbomComponentResourceImpl(SbomComponentResourceInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String componentId() { return this.innerModel().componentId(); } @@ -39,12 +53,8 @@ public String license() { return this.innerModel().license(); } - public OffsetDateTime releaseDate() { - return this.innerModel().releaseDate(); - } - - public List paths() { - List inner = this.innerModel().paths(); + public List filePaths() { + List inner = this.innerModel().filePaths(); if (inner != null) { return Collections.unmodifiableList(inner); } else { @@ -52,11 +62,7 @@ public List paths() { } } - public IsUpdateAvailable isUpdateAvailable() { - return this.innerModel().isUpdateAvailable(); - } - - public ComponentInner innerModel() { + public SbomComponentResourceInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsClientImpl.java new file mode 100644 index 0000000000000..10f28eedcfa7e --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsClientImpl.java @@ -0,0 +1,296 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.SbomComponentsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponentListResult; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in SbomComponentsClient. + */ +public final class SbomComponentsClientImpl implements SbomComponentsClient { + /** + * The proxy service used to perform REST calls. + */ + private final SbomComponentsService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of SbomComponentsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + SbomComponentsClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(SbomComponentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefenseSbomComponents to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefenseSb") + public interface SbomComponentsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/sbomComponents") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsImpl.java new file mode 100644 index 0000000000000..e1100da23655a --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SbomComponentsImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.SbomComponentsClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponentResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponents; + +public final class SbomComponentsImpl implements SbomComponents { + private static final ClientLogger LOGGER = new ClientLogger(SbomComponentsImpl.class); + + private final SbomComponentsClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public SbomComponentsImpl(SbomComponentsClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SbomComponentResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SbomComponentResourceImpl(inner1, this.manager())); + } + + private SbomComponentsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesClientImpl.java new file mode 100644 index 0000000000000..4b4525cb41f39 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesClientImpl.java @@ -0,0 +1,452 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.SummariesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryListResult; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in SummariesClient. + */ +public final class SummariesClientImpl implements SummariesClient { + /** + * The proxy service used to perform REST calls. + */ + private final SummariesService service; + + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; + + /** + * Initializes an instance of SummariesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + SummariesClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(SummariesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for IoTFirmwareDefenseSummaries to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "IoTFirmwareDefenseSu") + public interface SummariesService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/summaries") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmware(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/firmwares/{firmwareId}/summaries/{summaryName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @PathParam("firmwareId") String firmwareId, @PathParam("summaryName") SummaryName summaryName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByFirmwareNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, firmwareId, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareSinglePageAsync(String resourceGroupName, + String workspaceName, String firmwareId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByFirmware(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedFlux<>(() -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink)); + } + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByFirmwareAsync(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedFlux<>( + () -> listByFirmwareSinglePageAsync(resourceGroupName, workspaceName, firmwareId, context), + nextLink -> listByFirmwareNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId)); + } + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + return new PagedIterable<>(listByFirmwareAsync(resourceGroupName, workspaceName, firmwareId, context)); + } + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, SummaryName summaryName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + if (summaryName == null) { + return Mono.error(new IllegalArgumentException("Parameter summaryName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, firmwareId, summaryName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String workspaceName, + String firmwareId, SummaryName summaryName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (firmwareId == null) { + return Mono.error(new IllegalArgumentException("Parameter firmwareId is required and cannot be null.")); + } + if (summaryName == null) { + return Mono.error(new IllegalArgumentException("Parameter summaryName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, workspaceName, + firmwareId, summaryName, this.client.getApiVersion(), accept, context); + } + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName) { + return getWithResponseAsync(resourceGroupName, workspaceName, firmwareId, summaryName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, SummaryName summaryName, Context context) { + return getWithResponseAsync(resourceGroupName, workspaceName, firmwareId, summaryName, context).block(); + } + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public SummaryResourceInner get(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName) { + return getWithResponse(resourceGroupName, workspaceName, firmwareId, summaryName, Context.NONE).getValue(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByFirmwareNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByFirmwareNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesImpl.java new file mode 100644 index 0000000000000..aad4b3986a5ba --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummariesImpl.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.SummariesClient; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.Summaries; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResource; + +public final class SummariesImpl implements Summaries { + private static final ClientLogger LOGGER = new ClientLogger(SummariesImpl.class); + + private final SummariesClient innerClient; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + public SummariesImpl(SummariesClient innerClient, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SummaryResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context) { + PagedIterable inner + = this.serviceClient().listByFirmware(resourceGroupName, workspaceName, firmwareId, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SummaryResourceImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, workspaceName, firmwareId, summaryName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SummaryResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public SummaryResource get(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName) { + SummaryResourceInner inner + = this.serviceClient().get(resourceGroupName, workspaceName, firmwareId, summaryName); + if (inner != null) { + return new SummaryResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + private SummariesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummaryResourceImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummaryResourceImpl.java new file mode 100644 index 0000000000000..0dec1fd0b03ab --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/SummaryResourceImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResource; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResourceProperties; + +public final class SummaryResourceImpl implements SummaryResource { + private SummaryResourceInner innerObject; + + private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; + + SummaryResourceImpl(SummaryResourceInner innerObject, + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SummaryResourceProperties properties() { + return this.innerModel().properties(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public SummaryResourceInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/UrlTokenImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/UrlTokenImpl.java index 6323de0e4c769..435c4e2a24e58 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/UrlTokenImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/UrlTokenImpl.java @@ -12,8 +12,7 @@ public final class UrlTokenImpl implements UrlToken { private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - UrlTokenImpl( - UrlTokenInner innerObject, + UrlTokenImpl(UrlTokenInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -23,10 +22,6 @@ public String url() { return this.innerModel().url(); } - public String uploadUrl() { - return this.innerModel().uploadUrl(); - } - public UrlTokenInner innerModel() { return this.innerObject; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/Utils.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/Utils.java deleted file mode 100644 index 8a9e03e5e56fb..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspaceImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspaceImpl.java index dfd556d71c417..fa5a1ec84c4ea 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspaceImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspaceImpl.java @@ -87,22 +87,14 @@ public WorkspaceImpl withExistingResourceGroup(String resourceGroupName) { } public Workspace create() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .createWithResponse(resourceGroupName, workspaceName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .createWithResponse(resourceGroupName, workspaceName, this.innerModel(), Context.NONE).getValue(); return this; } public Workspace create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .createWithResponse(resourceGroupName, workspaceName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .createWithResponse(resourceGroupName, workspaceName, this.innerModel(), context).getValue(); return this; } @@ -118,59 +110,41 @@ public WorkspaceImpl update() { } public Workspace apply() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .updateWithResponse(resourceGroupName, workspaceName, updateWorkspace, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .updateWithResponse(resourceGroupName, workspaceName, updateWorkspace, Context.NONE).getValue(); return this; } public Workspace apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .updateWithResponse(resourceGroupName, workspaceName, updateWorkspace, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .updateWithResponse(resourceGroupName, workspaceName, updateWorkspace, context).getValue(); return this; } - WorkspaceImpl( - WorkspaceInner innerObject, + WorkspaceImpl(WorkspaceInner innerObject, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.workspaceName = Utils.getValueFromIdByName(innerObject.id(), "workspaces"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.workspaceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "workspaces"); } public Workspace refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .getByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .getByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE).getValue(); return this; } public Workspace refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .getByResourceGroupWithResponse(resourceGroupName, workspaceName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .getByResourceGroupWithResponse(resourceGroupName, workspaceName, context).getValue(); return this; } - public Response generateUploadUrlWithResponse( - GenerateUploadUrlRequest generateUploadUrl, Context context) { - return serviceManager - .workspaces() - .generateUploadUrlWithResponse(resourceGroupName, workspaceName, generateUploadUrl, context); + public Response generateUploadUrlWithResponse(GenerateUploadUrlRequest generateUploadUrl, + Context context) { + return serviceManager.workspaces().generateUploadUrlWithResponse(resourceGroupName, workspaceName, + generateUploadUrl, context); } public UrlToken generateUploadUrl(GenerateUploadUrlRequest generateUploadUrl) { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesClientImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesClientImpl.java index 3ba7887b471e6..b0ec189e1ba82 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesClientImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesClientImpl.java @@ -38,254 +38,194 @@ import com.azure.resourcemanager.iotfirmwaredefense.models.WorkspaceUpdateDefinition; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in WorkspacesClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspacesClient. + */ public final class WorkspacesClientImpl implements WorkspacesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final WorkspacesService service; - /** The service client containing this operation class. */ - private final FistImpl client; + /** + * The service client containing this operation class. + */ + private final IoTFirmwareDefenseImpl client; /** * Initializes an instance of WorkspacesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - WorkspacesClientImpl(FistImpl client) { - this.service = - RestProxy.create(WorkspacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + WorkspacesClientImpl(IoTFirmwareDefenseImpl client) { + this.service + = RestProxy.create(WorkspacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for FistWorkspaces to be used by the proxy service to perform REST calls. + * The interface defining all the services for IoTFirmwareDefenseWorkspaces to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "FistWorkspaces") + @ServiceInterface(name = "IoTFirmwareDefenseWo") public interface WorkspacesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.IoTFirmwareDefense/workspaces") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, + Mono> create(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") WorkspaceInner workspace, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") WorkspaceInner workspace, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") WorkspaceUpdateDefinition workspace, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") WorkspaceUpdateDefinition workspace, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") - @ExpectedResponses({200, 304}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/generateUploadUrl") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.IoTFirmwareDefense/workspaces/{workspaceName}/generateUploadUrl") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> generateUploadUrl( - @HostParam("$host") String endpoint, + Mono> generateUploadUrl(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("workspaceName") String workspaceName, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") GenerateUploadUrlRequest generateUploadUrl, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -294,13 +234,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces as paginated response with {@link PagedIterable}. @@ -312,7 +252,7 @@ public PagedIterable list() { /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -326,27 +266,23 @@ public PagedIterable list(Context context) { /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -354,53 +290,34 @@ private Mono> listByResourceGroupSinglePageAsync(S } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -409,27 +326,15 @@ private Mono> listByResourceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -438,14 +343,13 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -455,14 +359,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupN */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -476,7 +379,7 @@ public PagedIterable listByResourceGroup(String resourceGroupNam /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -491,7 +394,7 @@ public PagedIterable listByResourceGroup(String resourceGroupNam /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -501,19 +404,15 @@ public PagedIterable listByResourceGroup(String resourceGroupNam * @return firmware analysis workspace along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, String workspaceName, WorkspaceInner workspace) { + private Mono> createWithResponseAsync(String resourceGroupName, String workspaceName, + WorkspaceInner workspace) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -529,24 +428,14 @@ private Mono> createWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - workspace, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, this.client.getApiVersion(), workspace, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -557,19 +446,15 @@ private Mono> createWithResponseAsync( * @return firmware analysis workspace along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, String workspaceName, WorkspaceInner workspace, Context context) { + private Mono> createWithResponseAsync(String resourceGroupName, String workspaceName, + WorkspaceInner workspace, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -585,21 +470,13 @@ private Mono> createWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - workspace, - accept, - context); + return service.create(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, this.client.getApiVersion(), workspace, accept, context); } /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -616,7 +493,7 @@ private Mono createAsync(String resourceGroupName, String worksp /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -627,14 +504,14 @@ private Mono createAsync(String resourceGroupName, String worksp * @return firmware analysis workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String resourceGroupName, String workspaceName, WorkspaceInner workspace, Context context) { + public Response createWithResponse(String resourceGroupName, String workspaceName, + WorkspaceInner workspace, Context context) { return createWithResponseAsync(resourceGroupName, workspaceName, workspace, context).block(); } /** * The operation to create or update a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when creating a firmware analysis workspace. @@ -650,7 +527,7 @@ public WorkspaceInner create(String resourceGroupName, String workspaceName, Wor /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -660,19 +537,15 @@ public WorkspaceInner create(String resourceGroupName, String workspaceName, Wor * @return firmware analysis workspace along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String workspaceName, WorkspaceUpdateDefinition workspace) { + private Mono> updateWithResponseAsync(String resourceGroupName, String workspaceName, + WorkspaceUpdateDefinition workspace) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -688,24 +561,14 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - workspace, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, this.client.getApiVersion(), workspace, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -716,19 +579,15 @@ private Mono> updateWithResponseAsync( * @return firmware analysis workspace along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String workspaceName, WorkspaceUpdateDefinition workspace, Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, String workspaceName, + WorkspaceUpdateDefinition workspace, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -744,21 +603,13 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - workspace, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, this.client.getApiVersion(), workspace, accept, context); } /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -768,15 +619,15 @@ private Mono> updateWithResponseAsync( * @return firmware analysis workspace on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String workspaceName, WorkspaceUpdateDefinition workspace) { + private Mono updateAsync(String resourceGroupName, String workspaceName, + WorkspaceUpdateDefinition workspace) { return updateWithResponseAsync(resourceGroupName, workspaceName, workspace) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -787,14 +638,14 @@ private Mono updateAsync( * @return firmware analysis workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, String workspaceName, WorkspaceUpdateDefinition workspace, Context context) { + public Response updateWithResponse(String resourceGroupName, String workspaceName, + WorkspaceUpdateDefinition workspace, Context context) { return updateWithResponseAsync(resourceGroupName, workspaceName, workspace, context).block(); } /** * The operation to update a firmware analysis workspaces. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param workspace Parameters when updating a firmware analysis workspace. @@ -810,7 +661,7 @@ public WorkspaceInner update(String resourceGroupName, String workspaceName, Wor /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -821,16 +672,12 @@ public WorkspaceInner update(String resourceGroupName, String workspaceName, Wor @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceGroupName, String workspaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -841,23 +688,14 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -867,19 +705,15 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String workspaceName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String workspaceName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -890,20 +724,13 @@ private Mono> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, this.client.getApiVersion(), accept, context); } /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -918,7 +745,7 @@ private Mono deleteAsync(String resourceGroupName, String workspaceName) { /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -934,7 +761,7 @@ public Response deleteWithResponse(String resourceGroupName, String worksp /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -948,7 +775,7 @@ public void delete(String resourceGroupName, String workspaceName) { /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -957,19 +784,15 @@ public void delete(String resourceGroupName, String workspaceName) { * @return firmware analysis workspace along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String workspaceName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String workspaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -981,22 +804,14 @@ private Mono> getByResourceGroupWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -1006,19 +821,15 @@ private Mono> getByResourceGroupWithResponseAsync( * @return firmware analysis workspace along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String workspaceName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String workspaceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1029,20 +840,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, this.client.getApiVersion(), accept, context); } /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1058,7 +862,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, S /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -1068,14 +872,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName, S * @return firmware analysis workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, workspaceName, context).block(); } /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1090,7 +894,7 @@ public WorkspaceInner getByResourceGroup(String resourceGroupName, String worksp /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -1098,22 +902,18 @@ public WorkspaceInner getByResourceGroup(String resourceGroupName, String worksp * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateUploadUrlWithResponseAsync( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl) { + private Mono> generateUploadUrlWithResponseAsync(String resourceGroupName, + String workspaceName, GenerateUploadUrlRequest generateUploadUrl) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1131,23 +931,14 @@ private Mono> generateUploadUrlWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .generateUploadUrl( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - generateUploadUrl, - accept, - context)) + context -> service.generateUploadUrl(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, workspaceName, this.client.getApiVersion(), generateUploadUrl, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -1156,22 +947,18 @@ private Mono> generateUploadUrlWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> generateUploadUrlWithResponseAsync( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl, Context context) { + private Mono> generateUploadUrlWithResponseAsync(String resourceGroupName, + String workspaceName, GenerateUploadUrlRequest generateUploadUrl, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1188,21 +975,13 @@ private Mono> generateUploadUrlWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .generateUploadUrl( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - workspaceName, - this.client.getApiVersion(), - generateUploadUrl, - accept, - context); + return service.generateUploadUrl(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + workspaceName, this.client.getApiVersion(), generateUploadUrl, accept, context); } /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -1212,15 +991,15 @@ private Mono> generateUploadUrlWithResponseAsync( * @return url data for creating or accessing a blob file on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono generateUploadUrlAsync( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl) { + private Mono generateUploadUrlAsync(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl) { return generateUploadUrlWithResponseAsync(resourceGroupName, workspaceName, generateUploadUrl) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -1231,14 +1010,14 @@ private Mono generateUploadUrlAsync( * @return url data for creating or accessing a blob file along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response generateUploadUrlWithResponse( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl, Context context) { + public Response generateUploadUrlWithResponse(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl, Context context) { return generateUploadUrlWithResponseAsync(resourceGroupName, workspaceName, generateUploadUrl, context).block(); } /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -1248,22 +1027,23 @@ public Response generateUploadUrlWithResponse( * @return url data for creating or accessing a blob file. */ @ServiceMethod(returns = ReturnType.SINGLE) - public UrlTokenInner generateUploadUrl( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl) { + public UrlTokenInner generateUploadUrl(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl) { return generateUploadUrlWithResponse(resourceGroupName, workspaceName, generateUploadUrl, Context.NONE) .getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -1271,76 +1051,59 @@ private Mono> listBySubscriptionNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1348,63 +1111,45 @@ private Mono> listByResourceGroupNextSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesImpl.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesImpl.java index 17c358ca97aaa..c7a2f6d3cd357 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesImpl.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/WorkspacesImpl.java @@ -24,8 +24,7 @@ public final class WorkspacesImpl implements Workspaces { private final com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager; - public WorkspacesImpl( - WorkspacesClient innerClient, + public WorkspacesImpl(WorkspacesClient innerClient, com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -33,26 +32,26 @@ public WorkspacesImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkspaceImpl(inner1, this.manager())); } - public Response deleteByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, workspaceName, context); } @@ -60,15 +59,12 @@ public void deleteByResourceGroup(String resourceGroupName, String workspaceName this.serviceClient().delete(resourceGroupName, workspaceName); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, workspaceName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, workspaceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new WorkspaceImpl(inner.getValue(), this.manager())); } else { return null; @@ -84,27 +80,22 @@ public Workspace getByResourceGroup(String resourceGroupName, String workspaceNa } } - public Response generateUploadUrlWithResponse( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl, Context context) { - Response inner = - this - .serviceClient() - .generateUploadUrlWithResponse(resourceGroupName, workspaceName, generateUploadUrl, context); + public Response generateUploadUrlWithResponse(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl, Context context) { + Response inner = this.serviceClient().generateUploadUrlWithResponse(resourceGroupName, + workspaceName, generateUploadUrl, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new UrlTokenImpl(inner.getValue(), this.manager())); } else { return null; } } - public UrlToken generateUploadUrl( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl) { - UrlTokenInner inner = - this.serviceClient().generateUploadUrl(resourceGroupName, workspaceName, generateUploadUrl); + public UrlToken generateUploadUrl(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl) { + UrlTokenInner inner + = this.serviceClient().generateUploadUrl(resourceGroupName, workspaceName, generateUploadUrl); if (inner != null) { return new UrlTokenImpl(inner, this.manager()); } else { @@ -113,77 +104,57 @@ public UrlToken generateUploadUrl( } public Workspace getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, workspaceName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } this.deleteByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } return this.deleteByResourceGroupWithResponse(resourceGroupName, workspaceName, context); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/package-info.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/package-info.java index 0e164a564928e..b4c7025b41e02 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/package-info.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/implementation/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the implementations for Fist. The definitions and parameters in this swagger specification will be - * used to manage the IoT Firmware Defense resources. + * Package containing the implementations for IoTFirmwareDefense. + * Firmware & IoT Security REST API. */ package com.azure.resourcemanager.iotfirmwaredefense.implementation; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ActionType.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ActionType.java index 28952be1ea48f..fa6342ff6b034 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ActionType.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ActionType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. */ +/** + * Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. + */ public final class ActionType extends ExpandableStringEnum { - /** Static value Internal for ActionType. */ + /** + * Static value Internal for ActionType. + */ public static final ActionType INTERNAL = fromString("Internal"); /** * Creates a new instance of ActionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ActionType() { /** * Creates or finds a ActionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActionType. */ @@ -35,7 +39,7 @@ public static ActionType fromString(String name) { /** * Gets known ActionType values. - * + * * @return known ActionType values. */ public static Collection values() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardening.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardening.java deleted file mode 100644 index 75f15a6d249b6..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardening.java +++ /dev/null @@ -1,94 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; - -/** An immutable client-side representation of BinaryHardening. */ -public interface BinaryHardening { - /** - * Gets the binaryHardeningId property: ID for the binary hardening result. - * - * @return the binaryHardeningId value. - */ - String binaryHardeningId(); - - /** - * Gets the architecture property: The architecture of the uploaded firmware. - * - * @return the architecture value. - */ - String architecture(); - - /** - * Gets the path property: path for binary hardening. - * - * @return the path value. - */ - String path(); - - /** - * Gets the classProperty property: class for binary hardening. - * - * @return the classProperty value. - */ - String classProperty(); - - /** - * Gets the runpath property: The runpath of the uploaded firmware. - * - * @return the runpath value. - */ - String runpath(); - - /** - * Gets the rpath property: The rpath of the uploaded firmware. - * - * @return the rpath value. - */ - String rpath(); - - /** - * Gets the nx property: NX flag. - * - * @return the nx value. - */ - NxFlag nx(); - - /** - * Gets the pie property: PIE flag. - * - * @return the pie value. - */ - PieFlag pie(); - - /** - * Gets the relro property: RELRO flag. - * - * @return the relro value. - */ - RelroFlag relro(); - - /** - * Gets the canary property: Canary flag. - * - * @return the canary value. - */ - CanaryFlag canary(); - - /** - * Gets the stripped property: Stripped flag. - * - * @return the stripped value. - */ - StrippedFlag stripped(); - - /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner object. - * - * @return the inner object. - */ - BinaryHardeningInner innerModel(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningListResult.java similarity index 70% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningList.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningListResult.java index 118ca6fad6a17..f4060cffcb44f 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningListResult.java @@ -5,41 +5,45 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List result for binary hardening. */ +/** + * List of binary hardening results. + */ @Fluent -public final class BinaryHardeningList { +public final class BinaryHardeningListResult { /* * The list of binary hardening results. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; + private List value; /* - * The uri to fetch the next page of asset. + * The uri to fetch the next page of resources. */ @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of BinaryHardeningList class. */ - public BinaryHardeningList() { + /** + * Creates an instance of BinaryHardeningListResult class. + */ + public BinaryHardeningListResult() { } /** * Get the value property: The list of binary hardening results. - * + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Get the nextLink property: The uri to fetch the next page of asset. - * + * Get the nextLink property: The uri to fetch the next page of resources. + * * @return the nextLink value. */ public String nextLink() { @@ -47,19 +51,19 @@ public String nextLink() { } /** - * Set the nextLink property: The uri to fetch the next page of asset. - * + * Set the nextLink property: The uri to fetch the next page of resources. + * * @param nextLink the nextLink value to set. - * @return the BinaryHardeningList object itself. + * @return the BinaryHardeningListResult object itself. */ - public BinaryHardeningList withNextLink(String nextLink) { + public BinaryHardeningListResult withNextLink(String nextLink) { this.nextLink = nextLink; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningResource.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningResource.java new file mode 100644 index 0000000000000..65236de3721cb --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningResource.java @@ -0,0 +1,125 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; + +/** + * An immutable client-side representation of BinaryHardeningResource. + */ +public interface BinaryHardeningResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the binaryHardeningId property: ID for the binary hardening result. + * + * @return the binaryHardeningId value. + */ + String binaryHardeningId(); + + /** + * Gets the architecture property: The architecture of the uploaded firmware. + * + * @return the architecture value. + */ + String architecture(); + + /** + * Gets the filePath property: The executable path. + * + * @return the filePath value. + */ + String filePath(); + + /** + * Gets the classProperty property: The executable class to indicate 32 or 64 bit. + * + * @return the classProperty value. + */ + String classProperty(); + + /** + * Gets the runpath property: The runpath of the uploaded firmware. + * + * @return the runpath value. + */ + String runpath(); + + /** + * Gets the rpath property: The rpath of the uploaded firmware. + * + * @return the rpath value. + */ + String rpath(); + + /** + * Gets the nx property: NX (no-execute) flag. + * + * @return the nx value. + */ + Boolean nx(); + + /** + * Gets the pie property: PIE (position independent executable) flag. + * + * @return the pie value. + */ + Boolean pie(); + + /** + * Gets the relro property: RELRO (relocation read-only) flag. + * + * @return the relro value. + */ + Boolean relro(); + + /** + * Gets the canary property: Canary (stack canaries) flag. + * + * @return the canary value. + */ + Boolean canary(); + + /** + * Gets the stripped property: Stripped flag. + * + * @return the stripped value. + */ + Boolean stripped(); + + /** + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner object. + * + * @return the inner object. + */ + BinaryHardeningResourceInner innerModel(); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummary.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummary.java deleted file mode 100644 index 0d35b620758dd..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummary.java +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner; - -/** An immutable client-side representation of BinaryHardeningSummary. */ -public interface BinaryHardeningSummary { - /** - * Gets the totalFiles property: Total number of binaries that were analyzed. - * - * @return the totalFiles value. - */ - Long totalFiles(); - - /** - * Gets the nx property: NX summary percentage. - * - * @return the nx value. - */ - Integer nx(); - - /** - * Gets the pie property: PIE summary percentage. - * - * @return the pie value. - */ - Integer pie(); - - /** - * Gets the relro property: RELRO summary percentage. - * - * @return the relro value. - */ - Integer relro(); - - /** - * Gets the canary property: Canary summary percentage. - * - * @return the canary value. - */ - Integer canary(); - - /** - * Gets the stripped property: Stripped summary percentage. - * - * @return the stripped value. - */ - Integer stripped(); - - /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner object. - * - * @return the inner object. - */ - BinaryHardeningSummaryInner innerModel(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningSummaryInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummaryResource.java similarity index 67% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningSummaryInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummaryResource.java index 7eca7366b64b2..8c8cec0d54a67 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/BinaryHardeningSummaryInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardeningSummaryResource.java @@ -2,14 +2,20 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; +package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; - -/** Binary hardening summary percentages. */ +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * Properties for a binary hardening analysis summary. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "summaryType") +@JsonTypeName("BinaryHardening") @Fluent -public final class BinaryHardeningSummaryInner { +public final class BinaryHardeningSummaryResource extends SummaryResourceProperties { /* * Total number of binaries that were analyzed */ @@ -46,13 +52,15 @@ public final class BinaryHardeningSummaryInner { @JsonProperty(value = "stripped") private Integer stripped; - /** Creates an instance of BinaryHardeningSummaryInner class. */ - public BinaryHardeningSummaryInner() { + /** + * Creates an instance of BinaryHardeningSummaryResource class. + */ + public BinaryHardeningSummaryResource() { } /** * Get the totalFiles property: Total number of binaries that were analyzed. - * + * * @return the totalFiles value. */ public Long totalFiles() { @@ -61,18 +69,18 @@ public Long totalFiles() { /** * Set the totalFiles property: Total number of binaries that were analyzed. - * + * * @param totalFiles the totalFiles value to set. - * @return the BinaryHardeningSummaryInner object itself. + * @return the BinaryHardeningSummaryResource object itself. */ - public BinaryHardeningSummaryInner withTotalFiles(Long totalFiles) { + public BinaryHardeningSummaryResource withTotalFiles(Long totalFiles) { this.totalFiles = totalFiles; return this; } /** * Get the nx property: NX summary percentage. - * + * * @return the nx value. */ public Integer nx() { @@ -81,18 +89,18 @@ public Integer nx() { /** * Set the nx property: NX summary percentage. - * + * * @param nx the nx value to set. - * @return the BinaryHardeningSummaryInner object itself. + * @return the BinaryHardeningSummaryResource object itself. */ - public BinaryHardeningSummaryInner withNx(Integer nx) { + public BinaryHardeningSummaryResource withNx(Integer nx) { this.nx = nx; return this; } /** * Get the pie property: PIE summary percentage. - * + * * @return the pie value. */ public Integer pie() { @@ -101,18 +109,18 @@ public Integer pie() { /** * Set the pie property: PIE summary percentage. - * + * * @param pie the pie value to set. - * @return the BinaryHardeningSummaryInner object itself. + * @return the BinaryHardeningSummaryResource object itself. */ - public BinaryHardeningSummaryInner withPie(Integer pie) { + public BinaryHardeningSummaryResource withPie(Integer pie) { this.pie = pie; return this; } /** * Get the relro property: RELRO summary percentage. - * + * * @return the relro value. */ public Integer relro() { @@ -121,18 +129,18 @@ public Integer relro() { /** * Set the relro property: RELRO summary percentage. - * + * * @param relro the relro value to set. - * @return the BinaryHardeningSummaryInner object itself. + * @return the BinaryHardeningSummaryResource object itself. */ - public BinaryHardeningSummaryInner withRelro(Integer relro) { + public BinaryHardeningSummaryResource withRelro(Integer relro) { this.relro = relro; return this; } /** * Get the canary property: Canary summary percentage. - * + * * @return the canary value. */ public Integer canary() { @@ -141,18 +149,18 @@ public Integer canary() { /** * Set the canary property: Canary summary percentage. - * + * * @param canary the canary value to set. - * @return the BinaryHardeningSummaryInner object itself. + * @return the BinaryHardeningSummaryResource object itself. */ - public BinaryHardeningSummaryInner withCanary(Integer canary) { + public BinaryHardeningSummaryResource withCanary(Integer canary) { this.canary = canary; return this; } /** * Get the stripped property: Stripped summary percentage. - * + * * @return the stripped value. */ public Integer stripped() { @@ -161,20 +169,22 @@ public Integer stripped() { /** * Set the stripped property: Stripped summary percentage. - * + * * @param stripped the stripped value to set. - * @return the BinaryHardeningSummaryInner object itself. + * @return the BinaryHardeningSummaryResource object itself. */ - public BinaryHardeningSummaryInner withStripped(Integer stripped) { + public BinaryHardeningSummaryResource withStripped(Integer stripped) { this.stripped = stripped; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ + @Override public void validate() { + super.validate(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardenings.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardenings.java new file mode 100644 index 0000000000000..74394382b1866 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/BinaryHardenings.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of BinaryHardenings. + */ +public interface BinaryHardenings { + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists binary hardening analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of binary hardening results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CanaryFlag.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CanaryFlag.java deleted file mode 100644 index 8fe0b7d5f33a7..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CanaryFlag.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Canary flag. */ -public final class CanaryFlag extends ExpandableStringEnum { - /** Static value True for CanaryFlag. */ - public static final CanaryFlag TRUE = fromString("True"); - - /** Static value False for CanaryFlag. */ - public static final CanaryFlag FALSE = fromString("False"); - - /** - * Creates a new instance of CanaryFlag value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public CanaryFlag() { - } - - /** - * Creates or finds a CanaryFlag from its string representation. - * - * @param name a name to look for. - * @return the corresponding CanaryFlag. - */ - @JsonCreator - public static CanaryFlag fromString(String name) { - return fromString(name, CanaryFlag.class); - } - - /** - * Gets known CanaryFlag values. - * - * @return known CanaryFlag values. - */ - public static Collection values() { - return values(CanaryFlag.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Component.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Component.java deleted file mode 100644 index 21ed187a90866..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Component.java +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; -import java.time.OffsetDateTime; -import java.util.List; - -/** An immutable client-side representation of Component. */ -public interface Component { - /** - * Gets the componentId property: ID for the component. - * - * @return the componentId value. - */ - String componentId(); - - /** - * Gets the componentName property: Name for the component. - * - * @return the componentName value. - */ - String componentName(); - - /** - * Gets the version property: Version for the component. - * - * @return the version value. - */ - String version(); - - /** - * Gets the license property: License for the component. - * - * @return the license value. - */ - String license(); - - /** - * Gets the releaseDate property: Release date for the component. - * - * @return the releaseDate value. - */ - OffsetDateTime releaseDate(); - - /** - * Gets the paths property: Paths of the component. - * - * @return the paths value. - */ - List paths(); - - /** - * Gets the isUpdateAvailable property: Flag if new update is available for the component. - * - * @return the isUpdateAvailable value. - */ - IsUpdateAvailable isUpdateAvailable(); - - /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner object. - * - * @return the inner object. - */ - ComponentInner innerModel(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateEntity.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateEntity.java index a1c82c50e7641..97b345391b684 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateEntity.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateEntity.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Information on an entity (distinguished name) in a cryptographic certificate. */ +/** + * Information on an entity (distinguished name) in a cryptographic certificate. + */ @Fluent public final class CryptoCertificateEntity { /* @@ -40,13 +42,15 @@ public final class CryptoCertificateEntity { @JsonProperty(value = "country") private String country; - /** Creates an instance of CryptoCertificateEntity class. */ + /** + * Creates an instance of CryptoCertificateEntity class. + */ public CryptoCertificateEntity() { } /** * Get the commonName property: Common name of the certificate entity. - * + * * @return the commonName value. */ public String commonName() { @@ -55,7 +59,7 @@ public String commonName() { /** * Set the commonName property: Common name of the certificate entity. - * + * * @param commonName the commonName value to set. * @return the CryptoCertificateEntity object itself. */ @@ -66,7 +70,7 @@ public CryptoCertificateEntity withCommonName(String commonName) { /** * Get the organization property: Organization of the certificate entity. - * + * * @return the organization value. */ public String organization() { @@ -75,7 +79,7 @@ public String organization() { /** * Set the organization property: Organization of the certificate entity. - * + * * @param organization the organization value to set. * @return the CryptoCertificateEntity object itself. */ @@ -86,7 +90,7 @@ public CryptoCertificateEntity withOrganization(String organization) { /** * Get the organizationalUnit property: The organizational unit of the certificate entity. - * + * * @return the organizationalUnit value. */ public String organizationalUnit() { @@ -95,7 +99,7 @@ public String organizationalUnit() { /** * Set the organizationalUnit property: The organizational unit of the certificate entity. - * + * * @param organizationalUnit the organizationalUnit value to set. * @return the CryptoCertificateEntity object itself. */ @@ -106,7 +110,7 @@ public CryptoCertificateEntity withOrganizationalUnit(String organizationalUnit) /** * Get the state property: Geographical state or province of the certificate entity. - * + * * @return the state value. */ public String state() { @@ -115,7 +119,7 @@ public String state() { /** * Set the state property: Geographical state or province of the certificate entity. - * + * * @param state the state value to set. * @return the CryptoCertificateEntity object itself. */ @@ -126,7 +130,7 @@ public CryptoCertificateEntity withState(String state) { /** * Get the country property: Country code of the certificate entity. - * + * * @return the country value. */ public String country() { @@ -135,7 +139,7 @@ public String country() { /** * Set the country property: Country code of the certificate entity. - * + * * @param country the country value to set. * @return the CryptoCertificateEntity object itself. */ @@ -146,7 +150,7 @@ public CryptoCertificateEntity withCountry(String country) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateListResult.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateListResult.java new file mode 100644 index 0000000000000..3ab0bd380485a --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateListResult.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of crypto certificates. + */ +@Fluent +public final class CryptoCertificateListResult { + /* + * The list of crypto certificate results. + */ + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) + private List value; + + /* + * The uri to fetch the next page of resources. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of CryptoCertificateListResult class. + */ + public CryptoCertificateListResult() { + } + + /** + * Get the value property: The list of crypto certificate results. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Get the nextLink property: The uri to fetch the next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: The uri to fetch the next page of resources. + * + * @param nextLink the nextLink value to set. + * @return the CryptoCertificateListResult object itself. + */ + public CryptoCertificateListResult withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificate.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateResource.java similarity index 69% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificate.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateResource.java index 1d5cb76279560..64162fbf0b1d1 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificate.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateResource.java @@ -4,157 +4,188 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner; import java.time.OffsetDateTime; import java.util.List; -/** An immutable client-side representation of CryptoCertificate. */ -public interface CryptoCertificate { +/** + * An immutable client-side representation of CryptoCertificateResource. + */ +public interface CryptoCertificateResource { /** - * Gets the cryptoCertId property: ID for the certificate. - * - * @return the cryptoCertId value. + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. */ - String cryptoCertId(); + String id(); /** - * Gets the name property: Name of the certificate. - * + * Gets the name property: The name of the resource. + * * @return the name value. */ String name(); + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the cryptoCertId property: ID for the certificate result. + * + * @return the cryptoCertId value. + */ + String cryptoCertId(); + + /** + * Gets the namePropertiesName property: Name of the certificate. + * + * @return the namePropertiesName value. + */ + String namePropertiesName(); + /** * Gets the subject property: Subject information of the certificate. - * + * * @return the subject value. */ CryptoCertificateEntity subject(); /** * Gets the issuer property: Issuer information of the certificate. - * + * * @return the issuer value. */ CryptoCertificateEntity issuer(); /** * Gets the issuedDate property: Issue date for the certificate. - * + * * @return the issuedDate value. */ OffsetDateTime issuedDate(); /** * Gets the expirationDate property: Expiration date for the certificate. - * + * * @return the expirationDate value. */ OffsetDateTime expirationDate(); /** * Gets the role property: Role of the certificate (Root CA, etc). - * + * * @return the role value. */ String role(); /** * Gets the signatureAlgorithm property: The signature algorithm used in the certificate. - * + * * @return the signatureAlgorithm value. */ String signatureAlgorithm(); /** * Gets the keySize property: Size of the certificate's key in bits. - * + * * @return the keySize value. */ Long keySize(); /** * Gets the keyAlgorithm property: Key algorithm used in the certificate. - * + * * @return the keyAlgorithm value. */ String keyAlgorithm(); /** * Gets the encoding property: Encoding used for the certificate. - * + * * @return the encoding value. */ String encoding(); /** * Gets the serialNumber property: Serial number of the certificate. - * + * * @return the serialNumber value. */ String serialNumber(); /** * Gets the fingerprint property: Fingerprint of the certificate. - * + * * @return the fingerprint value. */ String fingerprint(); /** * Gets the usage property: List of functions the certificate can fulfill. - * + * * @return the usage value. */ List usage(); /** - * Gets the filePaths property: List of files paths for this certificate. - * + * Gets the filePaths property: List of files where this certificate was found. + * * @return the filePaths value. */ List filePaths(); /** * Gets the pairedKey property: A matching paired private key. - * + * * @return the pairedKey value. */ PairedKey pairedKey(); /** * Gets the isExpired property: Indicates if the certificate is expired. - * + * * @return the isExpired value. */ - IsExpired isExpired(); + Boolean isExpired(); /** - * Gets the isSelfSigned property: Indicates if the certificate was self-signed. - * + * Gets the isSelfSigned property: Indicates if the certificate is self-signed. + * * @return the isSelfSigned value. */ - IsSelfSigned isSelfSigned(); + Boolean isSelfSigned(); /** * Gets the isWeakSignature property: Indicates the signature algorithm used is insecure. - * + * * @return the isWeakSignature value. */ - IsWeakSignature isWeakSignature(); + Boolean isWeakSignature(); /** * Gets the isShortKeySize property: Indicates the certificate's key size is considered too small to be secure for * the key algorithm. - * + * * @return the isShortKeySize value. */ - IsShortKeySize isShortKeySize(); + Boolean isShortKeySize(); /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner object. - * + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner object. + * * @return the inner object. */ - CryptoCertificateInner innerModel(); + CryptoCertificateResourceInner innerModel(); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummary.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummary.java deleted file mode 100644 index 92b4a15278695..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummary.java +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateSummaryInner; - -/** An immutable client-side representation of CryptoCertificateSummary. */ -public interface CryptoCertificateSummary { - /** - * Gets the totalCertificates property: Total number of certificates found. - * - * @return the totalCertificates value. - */ - Long totalCertificates(); - - /** - * Gets the pairedKeys property: Total number of paired private keys found for the certificates. - * - * @return the pairedKeys value. - */ - Long pairedKeys(); - - /** - * Gets the expired property: Total number of expired certificates found. - * - * @return the expired value. - */ - Long expired(); - - /** - * Gets the expiringSoon property: Total number of nearly expired certificates found. - * - * @return the expiringSoon value. - */ - Long expiringSoon(); - - /** - * Gets the weakSignature property: Total number of certificates found using a weak signature algorithm. - * - * @return the weakSignature value. - */ - Long weakSignature(); - - /** - * Gets the selfSigned property: Total number of certificates found that are self-signed. - * - * @return the selfSigned value. - */ - Long selfSigned(); - - /** - * Gets the shortKeySize property: Total number of certificates found that have an insecure key size for the key - * algorithm. - * - * @return the shortKeySize value. - */ - Long shortKeySize(); - - /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateSummaryInner object. - * - * @return the inner object. - */ - CryptoCertificateSummaryInner innerModel(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateSummaryInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummaryResource.java similarity index 72% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateSummaryInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummaryResource.java index 5c43bf47c9628..c27eea0d3a3a2 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoCertificateSummaryInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateSummaryResource.java @@ -2,14 +2,20 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; +package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; - -/** Cryptographic certificate summary values. */ +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * Properties for cryptographic certificate summary. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "summaryType") +@JsonTypeName("CryptoCertificate") @Fluent -public final class CryptoCertificateSummaryInner { +public final class CryptoCertificateSummaryResource extends SummaryResourceProperties { /* * Total number of certificates found. */ @@ -52,13 +58,15 @@ public final class CryptoCertificateSummaryInner { @JsonProperty(value = "shortKeySize") private Long shortKeySize; - /** Creates an instance of CryptoCertificateSummaryInner class. */ - public CryptoCertificateSummaryInner() { + /** + * Creates an instance of CryptoCertificateSummaryResource class. + */ + public CryptoCertificateSummaryResource() { } /** * Get the totalCertificates property: Total number of certificates found. - * + * * @return the totalCertificates value. */ public Long totalCertificates() { @@ -67,18 +75,18 @@ public Long totalCertificates() { /** * Set the totalCertificates property: Total number of certificates found. - * + * * @param totalCertificates the totalCertificates value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withTotalCertificates(Long totalCertificates) { + public CryptoCertificateSummaryResource withTotalCertificates(Long totalCertificates) { this.totalCertificates = totalCertificates; return this; } /** * Get the pairedKeys property: Total number of paired private keys found for the certificates. - * + * * @return the pairedKeys value. */ public Long pairedKeys() { @@ -87,18 +95,18 @@ public Long pairedKeys() { /** * Set the pairedKeys property: Total number of paired private keys found for the certificates. - * + * * @param pairedKeys the pairedKeys value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withPairedKeys(Long pairedKeys) { + public CryptoCertificateSummaryResource withPairedKeys(Long pairedKeys) { this.pairedKeys = pairedKeys; return this; } /** * Get the expired property: Total number of expired certificates found. - * + * * @return the expired value. */ public Long expired() { @@ -107,18 +115,18 @@ public Long expired() { /** * Set the expired property: Total number of expired certificates found. - * + * * @param expired the expired value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withExpired(Long expired) { + public CryptoCertificateSummaryResource withExpired(Long expired) { this.expired = expired; return this; } /** * Get the expiringSoon property: Total number of nearly expired certificates found. - * + * * @return the expiringSoon value. */ public Long expiringSoon() { @@ -127,18 +135,18 @@ public Long expiringSoon() { /** * Set the expiringSoon property: Total number of nearly expired certificates found. - * + * * @param expiringSoon the expiringSoon value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withExpiringSoon(Long expiringSoon) { + public CryptoCertificateSummaryResource withExpiringSoon(Long expiringSoon) { this.expiringSoon = expiringSoon; return this; } /** * Get the weakSignature property: Total number of certificates found using a weak signature algorithm. - * + * * @return the weakSignature value. */ public Long weakSignature() { @@ -147,18 +155,18 @@ public Long weakSignature() { /** * Set the weakSignature property: Total number of certificates found using a weak signature algorithm. - * + * * @param weakSignature the weakSignature value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withWeakSignature(Long weakSignature) { + public CryptoCertificateSummaryResource withWeakSignature(Long weakSignature) { this.weakSignature = weakSignature; return this; } /** * Get the selfSigned property: Total number of certificates found that are self-signed. - * + * * @return the selfSigned value. */ public Long selfSigned() { @@ -167,11 +175,11 @@ public Long selfSigned() { /** * Set the selfSigned property: Total number of certificates found that are self-signed. - * + * * @param selfSigned the selfSigned value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withSelfSigned(Long selfSigned) { + public CryptoCertificateSummaryResource withSelfSigned(Long selfSigned) { this.selfSigned = selfSigned; return this; } @@ -179,7 +187,7 @@ public CryptoCertificateSummaryInner withSelfSigned(Long selfSigned) { /** * Get the shortKeySize property: Total number of certificates found that have an insecure key size for the key * algorithm. - * + * * @return the shortKeySize value. */ public Long shortKeySize() { @@ -189,20 +197,22 @@ public Long shortKeySize() { /** * Set the shortKeySize property: Total number of certificates found that have an insecure key size for the key * algorithm. - * + * * @param shortKeySize the shortKeySize value to set. - * @return the CryptoCertificateSummaryInner object itself. + * @return the CryptoCertificateSummaryResource object itself. */ - public CryptoCertificateSummaryInner withShortKeySize(Long shortKeySize) { + public CryptoCertificateSummaryResource withShortKeySize(Long shortKeySize) { this.shortKeySize = shortKeySize; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ + @Override public void validate() { + super.validate(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificates.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificates.java new file mode 100644 index 0000000000000..53e1a7467d663 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificates.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of CryptoCertificates. + */ +public interface CryptoCertificates { + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists cryptographic certificate analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto certificates as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyListResult.java similarity index 67% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyList.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyListResult.java index 182da0e5cddc8..cd6656fde7f90 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyListResult.java @@ -5,41 +5,45 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Crypto keys list. */ +/** + * List of crypto keys. + */ @Fluent -public final class CryptoKeyList { +public final class CryptoKeyListResult { /* - * Crypto keys list + * The list of crypto key results. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; + private List value; /* - * The uri to fetch the next page of asset. + * The uri to fetch the next page of resources. */ @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of CryptoKeyList class. */ - public CryptoKeyList() { + /** + * Creates an instance of CryptoKeyListResult class. + */ + public CryptoKeyListResult() { } /** - * Get the value property: Crypto keys list. - * + * Get the value property: The list of crypto key results. + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Get the nextLink property: The uri to fetch the next page of asset. - * + * Get the nextLink property: The uri to fetch the next page of resources. + * * @return the nextLink value. */ public String nextLink() { @@ -47,19 +51,19 @@ public String nextLink() { } /** - * Set the nextLink property: The uri to fetch the next page of asset. - * + * Set the nextLink property: The uri to fetch the next page of resources. + * * @param nextLink the nextLink value to set. - * @return the CryptoKeyList object itself. + * @return the CryptoKeyListResult object itself. */ - public CryptoKeyList withNextLink(String nextLink) { + public CryptoKeyListResult withNextLink(String nextLink) { this.nextLink = nextLink; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKey.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyResource.java similarity index 56% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKey.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyResource.java index 37f106d18c521..63cbb418bec24 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKey.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeyResource.java @@ -4,71 +4,102 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner; import java.util.List; -/** An immutable client-side representation of CryptoKey. */ -public interface CryptoKey { +/** + * An immutable client-side representation of CryptoKeyResource. + */ +public interface CryptoKeyResource { /** - * Gets the cryptoKeyId property: ID for the key. - * + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the cryptoKeyId property: ID for the key result. + * * @return the cryptoKeyId value. */ String cryptoKeyId(); /** * Gets the keyType property: Type of the key (public or private). - * + * * @return the keyType value. */ String keyType(); /** * Gets the keySize property: Size of the key in bits. - * + * * @return the keySize value. */ Long keySize(); /** * Gets the keyAlgorithm property: Key algorithm name. - * + * * @return the keyAlgorithm value. */ String keyAlgorithm(); /** * Gets the usage property: Functions the key can fulfill. - * + * * @return the usage value. */ List usage(); /** - * Gets the filePaths property: List of files paths for this key. - * + * Gets the filePaths property: List of files where this key was found. + * * @return the filePaths value. */ List filePaths(); /** * Gets the pairedKey property: A matching paired key or certificate. - * + * * @return the pairedKey value. */ PairedKey pairedKey(); /** * Gets the isShortKeySize property: Indicates the key size is considered too small to be secure for the algorithm. - * + * * @return the isShortKeySize value. */ - IsShortKeySize isShortKeySize(); + Boolean isShortKeySize(); /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyInner object. - * + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner object. + * * @return the inner object. */ - CryptoKeyInner innerModel(); + CryptoKeyResourceInner innerModel(); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummary.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummary.java deleted file mode 100644 index fe71e0febe2dc..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummary.java +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeySummaryInner; - -/** An immutable client-side representation of CryptoKeySummary. */ -public interface CryptoKeySummary { - /** - * Gets the totalKeys property: Total number of cryptographic keys found. - * - * @return the totalKeys value. - */ - Long totalKeys(); - - /** - * Gets the publicKeys property: Total number of (non-certificate) public keys found. - * - * @return the publicKeys value. - */ - Long publicKeys(); - - /** - * Gets the privateKeys property: Total number of private keys found. - * - * @return the privateKeys value. - */ - Long privateKeys(); - - /** - * Gets the pairedKeys property: Total number of keys found that have a matching paired key or certificate. - * - * @return the pairedKeys value. - */ - Long pairedKeys(); - - /** - * Gets the shortKeySize property: Total number of keys found that have an insecure key size for the algorithm. - * - * @return the shortKeySize value. - */ - Long shortKeySize(); - - /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeySummaryInner object. - * - * @return the inner object. - */ - CryptoKeySummaryInner innerModel(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeySummaryInner.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummaryResource.java similarity index 72% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeySummaryInner.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummaryResource.java index 8015e4e5e020f..6a558ea9ff908 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/fluent/models/CryptoKeySummaryInner.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeySummaryResource.java @@ -2,14 +2,20 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.iotfirmwaredefense.fluent.models; +package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; - -/** Cryptographic key summary values. */ +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * Properties for cryptographic key summary. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "summaryType") +@JsonTypeName("CryptoKey") @Fluent -public final class CryptoKeySummaryInner { +public final class CryptoKeySummaryResource extends SummaryResourceProperties { /* * Total number of cryptographic keys found. */ @@ -40,13 +46,15 @@ public final class CryptoKeySummaryInner { @JsonProperty(value = "shortKeySize") private Long shortKeySize; - /** Creates an instance of CryptoKeySummaryInner class. */ - public CryptoKeySummaryInner() { + /** + * Creates an instance of CryptoKeySummaryResource class. + */ + public CryptoKeySummaryResource() { } /** * Get the totalKeys property: Total number of cryptographic keys found. - * + * * @return the totalKeys value. */ public Long totalKeys() { @@ -55,18 +63,18 @@ public Long totalKeys() { /** * Set the totalKeys property: Total number of cryptographic keys found. - * + * * @param totalKeys the totalKeys value to set. - * @return the CryptoKeySummaryInner object itself. + * @return the CryptoKeySummaryResource object itself. */ - public CryptoKeySummaryInner withTotalKeys(Long totalKeys) { + public CryptoKeySummaryResource withTotalKeys(Long totalKeys) { this.totalKeys = totalKeys; return this; } /** * Get the publicKeys property: Total number of (non-certificate) public keys found. - * + * * @return the publicKeys value. */ public Long publicKeys() { @@ -75,18 +83,18 @@ public Long publicKeys() { /** * Set the publicKeys property: Total number of (non-certificate) public keys found. - * + * * @param publicKeys the publicKeys value to set. - * @return the CryptoKeySummaryInner object itself. + * @return the CryptoKeySummaryResource object itself. */ - public CryptoKeySummaryInner withPublicKeys(Long publicKeys) { + public CryptoKeySummaryResource withPublicKeys(Long publicKeys) { this.publicKeys = publicKeys; return this; } /** * Get the privateKeys property: Total number of private keys found. - * + * * @return the privateKeys value. */ public Long privateKeys() { @@ -95,18 +103,18 @@ public Long privateKeys() { /** * Set the privateKeys property: Total number of private keys found. - * + * * @param privateKeys the privateKeys value to set. - * @return the CryptoKeySummaryInner object itself. + * @return the CryptoKeySummaryResource object itself. */ - public CryptoKeySummaryInner withPrivateKeys(Long privateKeys) { + public CryptoKeySummaryResource withPrivateKeys(Long privateKeys) { this.privateKeys = privateKeys; return this; } /** * Get the pairedKeys property: Total number of keys found that have a matching paired key or certificate. - * + * * @return the pairedKeys value. */ public Long pairedKeys() { @@ -115,18 +123,18 @@ public Long pairedKeys() { /** * Set the pairedKeys property: Total number of keys found that have a matching paired key or certificate. - * + * * @param pairedKeys the pairedKeys value to set. - * @return the CryptoKeySummaryInner object itself. + * @return the CryptoKeySummaryResource object itself. */ - public CryptoKeySummaryInner withPairedKeys(Long pairedKeys) { + public CryptoKeySummaryResource withPairedKeys(Long pairedKeys) { this.pairedKeys = pairedKeys; return this; } /** * Get the shortKeySize property: Total number of keys found that have an insecure key size for the algorithm. - * + * * @return the shortKeySize value. */ public Long shortKeySize() { @@ -135,20 +143,22 @@ public Long shortKeySize() { /** * Set the shortKeySize property: Total number of keys found that have an insecure key size for the algorithm. - * + * * @param shortKeySize the shortKeySize value to set. - * @return the CryptoKeySummaryInner object itself. + * @return the CryptoKeySummaryResource object itself. */ - public CryptoKeySummaryInner withShortKeySize(Long shortKeySize) { + public CryptoKeySummaryResource withShortKeySize(Long shortKeySize) { this.shortKeySize = shortKeySize; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ + @Override public void validate() { + super.validate(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeys.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeys.java new file mode 100644 index 0000000000000..cee25c52467e6 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoKeys.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of CryptoKeys. + */ +public interface CryptoKeys { + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId); + + /** + * Lists cryptographic key analysis results found in a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of crypto keys as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId, + Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cve.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cve.java deleted file mode 100644 index 86387b3b180ec..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cve.java +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; -import java.time.OffsetDateTime; -import java.util.List; - -/** An immutable client-side representation of Cve. */ -public interface Cve { - /** - * Gets the cveId property: ID of CVE. - * - * @return the cveId value. - */ - String cveId(); - - /** - * Gets the component property: Component of CVE. - * - * @return the component value. - */ - Object component(); - - /** - * Gets the severity property: Severity of CVE. - * - * @return the severity value. - */ - String severity(); - - /** - * Gets the name property: Name of CVE. - * - * @return the name value. - */ - String name(); - - /** - * Gets the cvssScore property: A single CVSS score to represent the CVE. If a V3 score is specified, then it will - * use the V3 score. Otherwise if the V2 score is specified it will be the V2 score. - * - * @return the cvssScore value. - */ - String cvssScore(); - - /** - * Gets the cvssVersion property: Cvss version of CVE. - * - * @return the cvssVersion value. - */ - String cvssVersion(); - - /** - * Gets the cvssV2Score property: Cvss V2 score of CVE. - * - * @return the cvssV2Score value. - */ - String cvssV2Score(); - - /** - * Gets the cvssV3Score property: Cvss V3 score of CVE. - * - * @return the cvssV3Score value. - */ - String cvssV3Score(); - - /** - * Gets the publishDate property: Publish date of CVE. - * - * @return the publishDate value. - */ - OffsetDateTime publishDate(); - - /** - * Gets the updatedDate property: Updated date of CVE. - * - * @return the updatedDate value. - */ - OffsetDateTime updatedDate(); - - /** - * Gets the links property: The list of CVE links. - * - * @return the links value. - */ - List links(); - - /** - * Gets the description property: Description of CVE. - * - * @return the description value. - */ - String description(); - - /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner object. - * - * @return the inner object. - */ - CveInner innerModel(); -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveComponent.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveComponent.java new file mode 100644 index 0000000000000..78d7643b31128 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveComponent.java @@ -0,0 +1,106 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Properties of the SBOM component for a CVE. + */ +@Fluent +public final class CveComponent { + /* + * ID of the SBOM component + */ + @JsonProperty(value = "componentId") + private String componentId; + + /* + * Name of the SBOM component + */ + @JsonProperty(value = "name") + private String name; + + /* + * Version of the SBOM component. + */ + @JsonProperty(value = "version") + private String version; + + /** + * Creates an instance of CveComponent class. + */ + public CveComponent() { + } + + /** + * Get the componentId property: ID of the SBOM component. + * + * @return the componentId value. + */ + public String componentId() { + return this.componentId; + } + + /** + * Set the componentId property: ID of the SBOM component. + * + * @param componentId the componentId value to set. + * @return the CveComponent object itself. + */ + public CveComponent withComponentId(String componentId) { + this.componentId = componentId; + return this; + } + + /** + * Get the name property: Name of the SBOM component. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the SBOM component. + * + * @param name the name value to set. + * @return the CveComponent object itself. + */ + public CveComponent withName(String name) { + this.name = name; + return this; + } + + /** + * Get the version property: Version of the SBOM component. + * + * @return the version value. + */ + public String version() { + return this.version; + } + + /** + * Set the version property: Version of the SBOM component. + * + * @param version the version value to set. + * @return the CveComponent object itself. + */ + public CveComponent withVersion(String version) { + this.version = version; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveLink.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveLink.java index 0869414371857..bb851d0e37ec6 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveLink.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveLink.java @@ -7,28 +7,32 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Link for CVE. */ +/** + * Properties of a reference link for a CVE. + */ @Fluent public final class CveLink { /* - * Href of CVE link + * The destination of the reference link. */ @JsonProperty(value = "href") private String href; /* - * Label of CVE link + * The label of the reference link. */ @JsonProperty(value = "label") private String label; - /** Creates an instance of CveLink class. */ + /** + * Creates an instance of CveLink class. + */ public CveLink() { } /** - * Get the href property: Href of CVE link. - * + * Get the href property: The destination of the reference link. + * * @return the href value. */ public String href() { @@ -36,8 +40,8 @@ public String href() { } /** - * Set the href property: Href of CVE link. - * + * Set the href property: The destination of the reference link. + * * @param href the href value to set. * @return the CveLink object itself. */ @@ -47,8 +51,8 @@ public CveLink withHref(String href) { } /** - * Get the label property: Label of CVE link. - * + * Get the label property: The label of the reference link. + * * @return the label value. */ public String label() { @@ -56,8 +60,8 @@ public String label() { } /** - * Set the label property: Label of CVE link. - * + * Set the label property: The label of the reference link. + * * @param label the label value to set. * @return the CveLink object itself. */ @@ -68,7 +72,7 @@ public CveLink withLabel(String label) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveListResult.java similarity index 73% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveList.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveListResult.java index 89eba0be3a5b0..90c2817d91097 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveListResult.java @@ -5,41 +5,45 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List result for CVE. */ +/** + * List of CVE results. + */ @Fluent -public final class CveList { +public final class CveListResult { /* * The list of CVE results. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; + private List value; /* - * The uri to fetch the next page of asset. + * The uri to fetch the next page of resources. */ @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of CveList class. */ - public CveList() { + /** + * Creates an instance of CveListResult class. + */ + public CveListResult() { } /** * Get the value property: The list of CVE results. - * + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Get the nextLink property: The uri to fetch the next page of asset. - * + * Get the nextLink property: The uri to fetch the next page of resources. + * * @return the nextLink value. */ public String nextLink() { @@ -47,19 +51,19 @@ public String nextLink() { } /** - * Set the nextLink property: The uri to fetch the next page of asset. - * + * Set the nextLink property: The uri to fetch the next page of resources. + * * @param nextLink the nextLink value to set. - * @return the CveList object itself. + * @return the CveListResult object itself. */ - public CveList withNextLink(String nextLink) { + public CveListResult withNextLink(String nextLink) { this.nextLink = nextLink; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveResource.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveResource.java new file mode 100644 index 0000000000000..e23dafc37dbbf --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveResource.java @@ -0,0 +1,120 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; +import java.util.List; + +/** + * An immutable client-side representation of CveResource. + */ +public interface CveResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the cveId property: ID of the CVE result. + * + * @return the cveId value. + */ + String cveId(); + + /** + * Gets the component property: The SBOM component for the CVE. + * + * @return the component value. + */ + CveComponent component(); + + /** + * Gets the severity property: Severity of the CVE. + * + * @return the severity value. + */ + String severity(); + + /** + * Gets the namePropertiesName property: Name of the CVE. + * + * @return the namePropertiesName value. + */ + String namePropertiesName(); + + /** + * Gets the cvssScore property: A single CVSS score to represent the CVE. If a V3 score is specified, then it will + * use the V3 score. Otherwise if the V2 score is specified it will be the V2 score. + * + * @return the cvssScore value. + */ + String cvssScore(); + + /** + * Gets the cvssVersion property: CVSS version of the CVE. + * + * @return the cvssVersion value. + */ + String cvssVersion(); + + /** + * Gets the cvssV2Score property: CVSS V2 score of the CVE. + * + * @return the cvssV2Score value. + */ + String cvssV2Score(); + + /** + * Gets the cvssV3Score property: CVSS V3 score of the CVE. + * + * @return the cvssV3Score value. + */ + String cvssV3Score(); + + /** + * Gets the links property: The list of reference links for the CVE. + * + * @return the links value. + */ + List links(); + + /** + * Gets the description property: The CVE description. + * + * @return the description value. + */ + String description(); + + /** + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner object. + * + * @return the inner object. + */ + CveResourceInner innerModel(); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveSummary.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveSummary.java index 7927a0b3f22a5..e73a215b54063 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveSummary.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CveSummary.java @@ -4,56 +4,161 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner; +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * Properties for a CVE analysis summary. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "summaryType") +@JsonTypeName("CVE") +@Fluent +public final class CveSummary extends SummaryResourceProperties { + /* + * The total number of critical severity CVEs detected + */ + @JsonProperty(value = "critical") + private Long critical; + + /* + * The total number of high severity CVEs detected + */ + @JsonProperty(value = "high") + private Long high; + + /* + * The total number of medium severity CVEs detected + */ + @JsonProperty(value = "medium") + private Long medium; + + /* + * The total number of low severity CVEs detected + */ + @JsonProperty(value = "low") + private Long low; + + /* + * The total number of unknown severity CVEs detected + */ + @JsonProperty(value = "unknown") + private Long unknown; + + /** + * Creates an instance of CveSummary class. + */ + public CveSummary() { + } -/** An immutable client-side representation of CveSummary. */ -public interface CveSummary { /** - * Gets the critical property: The total number of critical severity CVEs detected. - * + * Get the critical property: The total number of critical severity CVEs detected. + * * @return the critical value. */ - Long critical(); + public Long critical() { + return this.critical; + } /** - * Gets the high property: The total number of high severity CVEs detected. - * + * Set the critical property: The total number of critical severity CVEs detected. + * + * @param critical the critical value to set. + * @return the CveSummary object itself. + */ + public CveSummary withCritical(Long critical) { + this.critical = critical; + return this; + } + + /** + * Get the high property: The total number of high severity CVEs detected. + * * @return the high value. */ - Long high(); + public Long high() { + return this.high; + } + + /** + * Set the high property: The total number of high severity CVEs detected. + * + * @param high the high value to set. + * @return the CveSummary object itself. + */ + public CveSummary withHigh(Long high) { + this.high = high; + return this; + } /** - * Gets the medium property: The total number of medium severity CVEs detected. - * + * Get the medium property: The total number of medium severity CVEs detected. + * * @return the medium value. */ - Long medium(); + public Long medium() { + return this.medium; + } /** - * Gets the low property: The total number of low severity CVEs detected. - * + * Set the medium property: The total number of medium severity CVEs detected. + * + * @param medium the medium value to set. + * @return the CveSummary object itself. + */ + public CveSummary withMedium(Long medium) { + this.medium = medium; + return this; + } + + /** + * Get the low property: The total number of low severity CVEs detected. + * * @return the low value. */ - Long low(); + public Long low() { + return this.low; + } + + /** + * Set the low property: The total number of low severity CVEs detected. + * + * @param low the low value to set. + * @return the CveSummary object itself. + */ + public CveSummary withLow(Long low) { + this.low = low; + return this; + } /** - * Gets the unknown property: The total number of unknown severity CVEs detected. - * + * Get the unknown property: The total number of unknown severity CVEs detected. + * * @return the unknown value. */ - Long unknown(); + public Long unknown() { + return this.unknown; + } /** - * Gets the undefined property: The total number of undefined severity CVEs detected. - * - * @return the undefined value. + * Set the unknown property: The total number of unknown severity CVEs detected. + * + * @param unknown the unknown value to set. + * @return the CveSummary object itself. */ - Long undefined(); + public CveSummary withUnknown(Long unknown) { + this.unknown = unknown; + return this; + } /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner object. - * - * @return the inner object. + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. */ - CveSummaryInner innerModel(); + @Override + public void validate() { + super.validate(); + } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cves.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cves.java new file mode 100644 index 0000000000000..a3f1a3213e0d1 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Cves.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of Cves. + */ +public interface Cves { + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId); + + /** + * Lists CVE analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of CVE results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId, + Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmware.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmware.java index 4af43d6957b75..825bf6bf8da39 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmware.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmware.java @@ -4,136 +4,145 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner; import java.util.List; -/** An immutable client-side representation of Firmware. */ +/** + * An immutable client-side representation of Firmware. + */ public interface Firmware { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the fileName property: File name for a firmware that user uploaded. - * + * * @return the fileName value. */ String fileName(); /** * Gets the vendor property: Firmware vendor. - * + * * @return the vendor value. */ String vendor(); /** * Gets the model property: Firmware model. - * + * * @return the model value. */ String model(); /** * Gets the version property: Firmware version. - * + * * @return the version value. */ String version(); /** * Gets the description property: User-specified description of the firmware. - * + * * @return the description value. */ String description(); /** * Gets the fileSize property: File size of the uploaded firmware image. - * + * * @return the fileSize value. */ Long fileSize(); /** * Gets the status property: The status of firmware scan. - * + * * @return the status value. */ Status status(); /** * Gets the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @return the statusMessages value. */ - List statusMessages(); + List statusMessages(); /** * Gets the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner object. - * + * * @return the inner object. */ FirmwareInner innerModel(); - /** The entirety of the Firmware definition. */ + /** + * The entirety of the Firmware definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Firmware definition stages. */ + /** + * The Firmware definition stages. + */ interface DefinitionStages { - /** The first stage of the Firmware definition. */ + /** + * The first stage of the Firmware definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Firmware definition allowing to specify parent resource. */ + /** + * The stage of the Firmware definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, workspaceName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @return the next definition stage. @@ -145,256 +154,280 @@ interface WithParentResource { * The stage of the Firmware definition which contains all the minimum required properties for the resource to * be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithFileName, - DefinitionStages.WithVendor, - DefinitionStages.WithModel, - DefinitionStages.WithVersion, - DefinitionStages.WithDescription, - DefinitionStages.WithFileSize, - DefinitionStages.WithStatus, - DefinitionStages.WithStatusMessages { + interface WithCreate extends DefinitionStages.WithFileName, DefinitionStages.WithVendor, + DefinitionStages.WithModel, DefinitionStages.WithVersion, DefinitionStages.WithDescription, + DefinitionStages.WithFileSize, DefinitionStages.WithStatus, DefinitionStages.WithStatusMessages { /** * Executes the create request. - * + * * @return the created resource. */ Firmware create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Firmware create(Context context); } - /** The stage of the Firmware definition allowing to specify fileName. */ + /** + * The stage of the Firmware definition allowing to specify fileName. + */ interface WithFileName { /** * Specifies the fileName property: File name for a firmware that user uploaded.. - * + * * @param fileName File name for a firmware that user uploaded. * @return the next definition stage. */ WithCreate withFileName(String fileName); } - /** The stage of the Firmware definition allowing to specify vendor. */ + /** + * The stage of the Firmware definition allowing to specify vendor. + */ interface WithVendor { /** * Specifies the vendor property: Firmware vendor.. - * + * * @param vendor Firmware vendor. * @return the next definition stage. */ WithCreate withVendor(String vendor); } - /** The stage of the Firmware definition allowing to specify model. */ + /** + * The stage of the Firmware definition allowing to specify model. + */ interface WithModel { /** * Specifies the model property: Firmware model.. - * + * * @param model Firmware model. * @return the next definition stage. */ WithCreate withModel(String model); } - /** The stage of the Firmware definition allowing to specify version. */ + /** + * The stage of the Firmware definition allowing to specify version. + */ interface WithVersion { /** * Specifies the version property: Firmware version.. - * + * * @param version Firmware version. * @return the next definition stage. */ WithCreate withVersion(String version); } - /** The stage of the Firmware definition allowing to specify description. */ + /** + * The stage of the Firmware definition allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: User-specified description of the firmware.. - * + * * @param description User-specified description of the firmware. * @return the next definition stage. */ WithCreate withDescription(String description); } - /** The stage of the Firmware definition allowing to specify fileSize. */ + /** + * The stage of the Firmware definition allowing to specify fileSize. + */ interface WithFileSize { /** * Specifies the fileSize property: File size of the uploaded firmware image.. - * + * * @param fileSize File size of the uploaded firmware image. * @return the next definition stage. */ WithCreate withFileSize(Long fileSize); } - /** The stage of the Firmware definition allowing to specify status. */ + /** + * The stage of the Firmware definition allowing to specify status. + */ interface WithStatus { /** * Specifies the status property: The status of firmware scan.. - * + * * @param status The status of firmware scan. * @return the next definition stage. */ WithCreate withStatus(Status status); } - /** The stage of the Firmware definition allowing to specify statusMessages. */ + /** + * The stage of the Firmware definition allowing to specify statusMessages. + */ interface WithStatusMessages { /** * Specifies the statusMessages property: A list of errors or other messages generated during firmware * analysis. - * + * * @param statusMessages A list of errors or other messages generated during firmware analysis. * @return the next definition stage. */ - WithCreate withStatusMessages(List statusMessages); + WithCreate withStatusMessages(List statusMessages); } } /** * Begins update for the Firmware resource. - * + * * @return the stage of resource update. */ Firmware.Update update(); - /** The template for Firmware update. */ - interface Update - extends UpdateStages.WithFileName, - UpdateStages.WithVendor, - UpdateStages.WithModel, - UpdateStages.WithVersion, - UpdateStages.WithDescription, - UpdateStages.WithFileSize, - UpdateStages.WithStatus, - UpdateStages.WithStatusMessages { + /** + * The template for Firmware update. + */ + interface Update extends UpdateStages.WithFileName, UpdateStages.WithVendor, UpdateStages.WithModel, + UpdateStages.WithVersion, UpdateStages.WithDescription, UpdateStages.WithFileSize, UpdateStages.WithStatus, + UpdateStages.WithStatusMessages { /** * Executes the update request. - * + * * @return the updated resource. */ Firmware apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Firmware apply(Context context); } - /** The Firmware update stages. */ + /** + * The Firmware update stages. + */ interface UpdateStages { - /** The stage of the Firmware update allowing to specify fileName. */ + /** + * The stage of the Firmware update allowing to specify fileName. + */ interface WithFileName { /** * Specifies the fileName property: File name for a firmware that user uploaded.. - * + * * @param fileName File name for a firmware that user uploaded. * @return the next definition stage. */ Update withFileName(String fileName); } - /** The stage of the Firmware update allowing to specify vendor. */ + /** + * The stage of the Firmware update allowing to specify vendor. + */ interface WithVendor { /** * Specifies the vendor property: Firmware vendor.. - * + * * @param vendor Firmware vendor. * @return the next definition stage. */ Update withVendor(String vendor); } - /** The stage of the Firmware update allowing to specify model. */ + /** + * The stage of the Firmware update allowing to specify model. + */ interface WithModel { /** * Specifies the model property: Firmware model.. - * + * * @param model Firmware model. * @return the next definition stage. */ Update withModel(String model); } - /** The stage of the Firmware update allowing to specify version. */ + /** + * The stage of the Firmware update allowing to specify version. + */ interface WithVersion { /** * Specifies the version property: Firmware version.. - * + * * @param version Firmware version. * @return the next definition stage. */ Update withVersion(String version); } - /** The stage of the Firmware update allowing to specify description. */ + /** + * The stage of the Firmware update allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: User-specified description of the firmware.. - * + * * @param description User-specified description of the firmware. * @return the next definition stage. */ Update withDescription(String description); } - /** The stage of the Firmware update allowing to specify fileSize. */ + /** + * The stage of the Firmware update allowing to specify fileSize. + */ interface WithFileSize { /** * Specifies the fileSize property: File size of the uploaded firmware image.. - * + * * @param fileSize File size of the uploaded firmware image. * @return the next definition stage. */ Update withFileSize(Long fileSize); } - /** The stage of the Firmware update allowing to specify status. */ + /** + * The stage of the Firmware update allowing to specify status. + */ interface WithStatus { /** * Specifies the status property: The status of firmware scan.. - * + * * @param status The status of firmware scan. * @return the next definition stage. */ Update withStatus(Status status); } - /** The stage of the Firmware update allowing to specify statusMessages. */ + /** + * The stage of the Firmware update allowing to specify statusMessages. + */ interface WithStatusMessages { /** * Specifies the statusMessages property: A list of errors or other messages generated during firmware * analysis. - * + * * @param statusMessages A list of errors or other messages generated during firmware analysis. * @return the next definition stage. */ - Update withStatusMessages(List statusMessages); + Update withStatusMessages(List statusMessages); } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Firmware refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -402,7 +435,7 @@ interface WithStatusMessages { /** * The operation to a url for file download. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -413,7 +446,7 @@ interface WithStatusMessages { /** * The operation to a url for file download. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file. @@ -422,7 +455,7 @@ interface WithStatusMessages { /** * The operation to a url for tar file download. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -433,274 +466,10 @@ interface WithStatusMessages { /** * The operation to a url for tar file download. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file. */ UrlToken generateFilesystemDownloadUrl(); - - /** - * The operation to get a scan summary. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware along with {@link Response}. - */ - Response generateSummaryWithResponse(Context context); - - /** - * The operation to get a scan summary. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware. - */ - FirmwareSummary generateSummary(); - - /** - * The operation to list all components result for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateComponentList(); - - /** - * The operation to list all components result for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateComponentList(Context context); - - /** - * The operation to get component details for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware along with {@link Response}. - */ - Response generateComponentDetailsWithResponse(Context context); - - /** - * The operation to get component details for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware. - */ - Component generateComponentDetails(); - - /** - * The operation to list all binary hardening result for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateBinaryHardeningList(); - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateBinaryHardeningList(Context context); - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages along with {@link Response}. - */ - Response generateBinaryHardeningSummaryWithResponse(Context context); - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages. - */ - BinaryHardeningSummary generateBinaryHardeningSummary(); - - /** - * The operation to get binary hardening details for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware along with {@link Response}. - */ - Response generateBinaryHardeningDetailsWithResponse(Context context); - - /** - * The operation to get binary hardening details for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware. - */ - BinaryHardening generateBinaryHardeningDetails(); - - /** - * The operation to list all password hashes for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGeneratePasswordHashList(); - - /** - * The operation to list all password hashes for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGeneratePasswordHashList(Context context); - - /** - * The operation to list all cve results for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCveList(); - - /** - * The operation to list all cve results for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCveList(Context context); - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values along with {@link Response}. - */ - Response generateCveSummaryWithResponse(Context context); - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values. - */ - CveSummary generateCveSummary(); - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values along with {@link Response}. - */ - Response generateCryptoCertificateSummaryWithResponse(Context context); - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values. - */ - CryptoCertificateSummary generateCryptoCertificateSummary(); - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values along with {@link Response}. - */ - Response generateCryptoKeySummaryWithResponse(Context context); - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values. - */ - CryptoKeySummary generateCryptoKeySummary(); - - /** - * The operation to list all crypto certificates for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoCertificateList(); - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoCertificateList(Context context); - - /** - * The operation to list all crypto keys for a firmware. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoKeyList(); - - /** - * The operation to list all crypto keys for a firmware. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoKeyList(Context context); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareList.java index 1467872db56e9..00617e6002cdd 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of firmwares. */ +/** + * List of firmwares. + */ @Fluent public final class FirmwareList { /* @@ -24,13 +26,15 @@ public final class FirmwareList { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of FirmwareList class. */ + /** + * Creates an instance of FirmwareList class. + */ public FirmwareList() { } /** * Get the value property: The list of firmwares. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The uri to fetch the next page of asset. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Set the nextLink property: The uri to fetch the next page of asset. - * + * * @param nextLink the nextLink value to set. * @return the FirmwareList object itself. */ @@ -59,7 +63,7 @@ public FirmwareList withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareSummary.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareSummary.java index 51ba31089a98f..7a6c0208f8c09 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareSummary.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareSummary.java @@ -4,63 +4,213 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner; +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * Properties for high level summary of firmware analysis results. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "summaryType") +@JsonTypeName("Firmware") +@Fluent +public final class FirmwareSummary extends SummaryResourceProperties { + /* + * Total extracted size of the firmware in bytes. + */ + @JsonProperty(value = "extractedSize") + private Long extractedSize; + + /* + * Firmware file size in bytes. + */ + @JsonProperty(value = "fileSize") + private Long fileSize; + + /* + * Extracted file count. + */ + @JsonProperty(value = "extractedFileCount") + private Long extractedFileCount; + + /* + * Components count. + */ + @JsonProperty(value = "componentCount") + private Long componentCount; + + /* + * Binary count + */ + @JsonProperty(value = "binaryCount") + private Long binaryCount; + + /* + * Time used for analysis + */ + @JsonProperty(value = "analysisTimeSeconds") + private Long analysisTimeSeconds; + + /* + * The number of root file systems found. + */ + @JsonProperty(value = "rootFileSystems") + private Long rootFileSystems; -/** An immutable client-side representation of FirmwareSummary. */ -public interface FirmwareSummary { /** - * Gets the extractedSize property: Total extracted size of the firmware in bytes. - * + * Creates an instance of FirmwareSummary class. + */ + public FirmwareSummary() { + } + + /** + * Get the extractedSize property: Total extracted size of the firmware in bytes. + * * @return the extractedSize value. */ - Long extractedSize(); + public Long extractedSize() { + return this.extractedSize; + } + + /** + * Set the extractedSize property: Total extracted size of the firmware in bytes. + * + * @param extractedSize the extractedSize value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withExtractedSize(Long extractedSize) { + this.extractedSize = extractedSize; + return this; + } /** - * Gets the fileSize property: Firmware file size in bytes. - * + * Get the fileSize property: Firmware file size in bytes. + * * @return the fileSize value. */ - Long fileSize(); + public Long fileSize() { + return this.fileSize; + } /** - * Gets the extractedFileCount property: Extracted file count. - * + * Set the fileSize property: Firmware file size in bytes. + * + * @param fileSize the fileSize value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withFileSize(Long fileSize) { + this.fileSize = fileSize; + return this; + } + + /** + * Get the extractedFileCount property: Extracted file count. + * * @return the extractedFileCount value. */ - Long extractedFileCount(); + public Long extractedFileCount() { + return this.extractedFileCount; + } + + /** + * Set the extractedFileCount property: Extracted file count. + * + * @param extractedFileCount the extractedFileCount value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withExtractedFileCount(Long extractedFileCount) { + this.extractedFileCount = extractedFileCount; + return this; + } /** - * Gets the componentCount property: Components count. - * + * Get the componentCount property: Components count. + * * @return the componentCount value. */ - Long componentCount(); + public Long componentCount() { + return this.componentCount; + } + + /** + * Set the componentCount property: Components count. + * + * @param componentCount the componentCount value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withComponentCount(Long componentCount) { + this.componentCount = componentCount; + return this; + } /** - * Gets the binaryCount property: Binary count. - * + * Get the binaryCount property: Binary count. + * * @return the binaryCount value. */ - Long binaryCount(); + public Long binaryCount() { + return this.binaryCount; + } /** - * Gets the analysisTimeSeconds property: Time used for analysis. - * + * Set the binaryCount property: Binary count. + * + * @param binaryCount the binaryCount value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withBinaryCount(Long binaryCount) { + this.binaryCount = binaryCount; + return this; + } + + /** + * Get the analysisTimeSeconds property: Time used for analysis. + * * @return the analysisTimeSeconds value. */ - Long analysisTimeSeconds(); + public Long analysisTimeSeconds() { + return this.analysisTimeSeconds; + } /** - * Gets the rootFileSystems property: The number of root file systems found. - * + * Set the analysisTimeSeconds property: Time used for analysis. + * + * @param analysisTimeSeconds the analysisTimeSeconds value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withAnalysisTimeSeconds(Long analysisTimeSeconds) { + this.analysisTimeSeconds = analysisTimeSeconds; + return this; + } + + /** + * Get the rootFileSystems property: The number of root file systems found. + * * @return the rootFileSystems value. */ - Long rootFileSystems(); + public Long rootFileSystems() { + return this.rootFileSystems; + } + + /** + * Set the rootFileSystems property: The number of root file systems found. + * + * @param rootFileSystems the rootFileSystems value to set. + * @return the FirmwareSummary object itself. + */ + public FirmwareSummary withRootFileSystems(Long rootFileSystems) { + this.rootFileSystems = rootFileSystems; + return this; + } /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner object. - * - * @return the inner object. + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. */ - FirmwareSummaryInner innerModel(); + @Override + public void validate() { + super.validate(); + } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareUpdateDefinition.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareUpdateDefinition.java index 8a00b46ef265b..6eb4015d2c16d 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareUpdateDefinition.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/FirmwareUpdateDefinition.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Firmware definition. */ +/** + * Firmware definition. + */ @Fluent public final class FirmwareUpdateDefinition { /* @@ -18,13 +20,15 @@ public final class FirmwareUpdateDefinition { @JsonProperty(value = "properties") private FirmwareProperties innerProperties; - /** Creates an instance of FirmwareUpdateDefinition class. */ + /** + * Creates an instance of FirmwareUpdateDefinition class. + */ public FirmwareUpdateDefinition() { } /** * Get the innerProperties property: The editable properties of a firmware. - * + * * @return the innerProperties value. */ private FirmwareProperties innerProperties() { @@ -33,7 +37,7 @@ private FirmwareProperties innerProperties() { /** * Get the fileName property: File name for a firmware that user uploaded. - * + * * @return the fileName value. */ public String fileName() { @@ -42,7 +46,7 @@ public String fileName() { /** * Set the fileName property: File name for a firmware that user uploaded. - * + * * @param fileName the fileName value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -56,7 +60,7 @@ public FirmwareUpdateDefinition withFileName(String fileName) { /** * Get the vendor property: Firmware vendor. - * + * * @return the vendor value. */ public String vendor() { @@ -65,7 +69,7 @@ public String vendor() { /** * Set the vendor property: Firmware vendor. - * + * * @param vendor the vendor value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -79,7 +83,7 @@ public FirmwareUpdateDefinition withVendor(String vendor) { /** * Get the model property: Firmware model. - * + * * @return the model value. */ public String model() { @@ -88,7 +92,7 @@ public String model() { /** * Set the model property: Firmware model. - * + * * @param model the model value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -102,7 +106,7 @@ public FirmwareUpdateDefinition withModel(String model) { /** * Get the version property: Firmware version. - * + * * @return the version value. */ public String version() { @@ -111,7 +115,7 @@ public String version() { /** * Set the version property: Firmware version. - * + * * @param version the version value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -125,7 +129,7 @@ public FirmwareUpdateDefinition withVersion(String version) { /** * Get the description property: User-specified description of the firmware. - * + * * @return the description value. */ public String description() { @@ -134,7 +138,7 @@ public String description() { /** * Set the description property: User-specified description of the firmware. - * + * * @param description the description value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -148,7 +152,7 @@ public FirmwareUpdateDefinition withDescription(String description) { /** * Get the fileSize property: File size of the uploaded firmware image. - * + * * @return the fileSize value. */ public Long fileSize() { @@ -157,7 +161,7 @@ public Long fileSize() { /** * Set the fileSize property: File size of the uploaded firmware image. - * + * * @param fileSize the fileSize value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -171,7 +175,7 @@ public FirmwareUpdateDefinition withFileSize(Long fileSize) { /** * Get the status property: The status of firmware scan. - * + * * @return the status value. */ public Status status() { @@ -180,7 +184,7 @@ public Status status() { /** * Set the status property: The status of firmware scan. - * + * * @param status the status value to set. * @return the FirmwareUpdateDefinition object itself. */ @@ -194,20 +198,20 @@ public FirmwareUpdateDefinition withStatus(Status status) { /** * Get the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @return the statusMessages value. */ - public List statusMessages() { + public List statusMessages() { return this.innerProperties() == null ? null : this.innerProperties().statusMessages(); } /** * Set the statusMessages property: A list of errors or other messages generated during firmware analysis. - * + * * @param statusMessages the statusMessages value to set. * @return the FirmwareUpdateDefinition object itself. */ - public FirmwareUpdateDefinition withStatusMessages(List statusMessages) { + public FirmwareUpdateDefinition withStatusMessages(List statusMessages) { if (this.innerProperties() == null) { this.innerProperties = new FirmwareProperties(); } @@ -217,7 +221,7 @@ public FirmwareUpdateDefinition withStatusMessages(List statusMessages) /** * Get the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -226,7 +230,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmwares.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmwares.java index 093ff7015c12b..2b7295d4febde 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmwares.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Firmwares.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Firmwares. */ +/** + * Resource collection API of Firmwares. + */ public interface Firmwares { /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface Firmwares { /** * Lists all of firmwares inside a workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -37,7 +39,7 @@ public interface Firmwares { /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -47,12 +49,12 @@ public interface Firmwares { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response deleteWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context); /** * The operation to delete a firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -64,7 +66,7 @@ Response deleteWithResponse( /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -74,12 +76,12 @@ Response deleteWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return firmware along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + Context context); /** * Get firmware. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -92,7 +94,7 @@ Response getWithResponse( /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -102,12 +104,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response}. */ - Response generateDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response generateDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context); /** * The operation to a url for file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -120,7 +122,7 @@ Response generateDownloadUrlWithResponse( /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -130,12 +132,12 @@ Response generateDownloadUrlWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response}. */ - Response generateFilesystemDownloadUrlWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); + Response generateFilesystemDownloadUrlWithResponse(String resourceGroupName, String workspaceName, + String firmwareId, Context context); /** * The operation to a url for tar file download. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param firmwareId The id of the firmware. @@ -146,384 +148,9 @@ Response generateFilesystemDownloadUrlWithResponse( */ UrlToken generateFilesystemDownloadUrl(String resourceGroupName, String workspaceName, String firmwareId); - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware along with {@link Response}. - */ - Response generateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get a scan summary. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return summary result after scanning the firmware. - */ - FirmwareSummary generateSummary(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all components result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for components as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateComponentList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware along with {@link Response}. - */ - Response generateComponentDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get component details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return component of a firmware. - */ - Component generateComponentDetails(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all binary hardening result for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for binary hardening as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateBinaryHardeningList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages along with {@link Response}. - */ - Response generateBinaryHardeningSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list the binary hardening summary percentages for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening summary percentages. - */ - BinaryHardeningSummary generateBinaryHardeningSummary( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware along with {@link Response}. - */ - Response generateBinaryHardeningDetailsWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to get binary hardening details for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return binary hardening of a firmware. - */ - BinaryHardening generateBinaryHardeningDetails(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all password hashes for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return password hashes list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGeneratePasswordHashList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCveList(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all cve results for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list result for CVE as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCveList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values along with {@link Response}. - */ - Response generateCveSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the CVEs reported for the firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cVE summary values. - */ - CveSummary generateCveSummary(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values along with {@link Response}. - */ - Response generateCryptoCertificateSummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the discovered cryptographic certificates reported for the - * firmware image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic certificate summary values. - */ - CryptoCertificateSummary generateCryptoCertificateSummary( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values along with {@link Response}. - */ - Response generateCryptoKeySummaryWithResponse( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to provide a high level summary of the discovered cryptographic keys reported for the firmware - * image. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return cryptographic key summary values. - */ - CryptoKeySummary generateCryptoKeySummary(String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all crypto certificates for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto certificates list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoCertificateList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId); - - /** - * The operation to list all crypto keys for a firmware. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param workspaceName The name of the firmware analysis workspace. - * @param firmwareId The id of the firmware. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return crypto keys list as paginated response with {@link PagedIterable}. - */ - PagedIterable listGenerateCryptoKeyList( - String resourceGroupName, String workspaceName, String firmwareId, Context context); - /** * Get firmware. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -534,7 +161,7 @@ PagedIterable listGenerateCryptoKeyList( /** * Get firmware. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -546,7 +173,7 @@ PagedIterable listGenerateCryptoKeyList( /** * The operation to delete a firmware. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -556,7 +183,7 @@ PagedIterable listGenerateCryptoKeyList( /** * The operation to delete a firmware. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -568,7 +195,7 @@ PagedIterable listGenerateCryptoKeyList( /** * Begins definition for a new Firmware resource. - * + * * @param name resource name. * @return the first stage of the new Firmware definition. */ diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/GenerateUploadUrlRequest.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/GenerateUploadUrlRequest.java index 85f8fad0c1a74..bdcfd6a4d151b 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/GenerateUploadUrlRequest.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/GenerateUploadUrlRequest.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Properties for generating an upload URL. */ +/** + * Properties for generating an upload URL. + */ @Fluent public final class GenerateUploadUrlRequest { /* @@ -16,13 +18,15 @@ public final class GenerateUploadUrlRequest { @JsonProperty(value = "firmwareId") private String firmwareId; - /** Creates an instance of GenerateUploadUrlRequest class. */ + /** + * Creates an instance of GenerateUploadUrlRequest class. + */ public GenerateUploadUrlRequest() { } /** * Get the firmwareId property: A unique ID for the firmware to be uploaded. - * + * * @return the firmwareId value. */ public String firmwareId() { @@ -31,7 +35,7 @@ public String firmwareId() { /** * Set the firmwareId property: A unique ID for the firmware to be uploaded. - * + * * @param firmwareId the firmwareId value to set. * @return the GenerateUploadUrlRequest object itself. */ @@ -42,7 +46,7 @@ public GenerateUploadUrlRequest withFirmwareId(String firmwareId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsExpired.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsExpired.java deleted file mode 100644 index 691c16e148aca..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsExpired.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Indicates if the certificate is expired. */ -public final class IsExpired extends ExpandableStringEnum { - /** Static value True for IsExpired. */ - public static final IsExpired TRUE = fromString("True"); - - /** Static value False for IsExpired. */ - public static final IsExpired FALSE = fromString("False"); - - /** - * Creates a new instance of IsExpired value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public IsExpired() { - } - - /** - * Creates or finds a IsExpired from its string representation. - * - * @param name a name to look for. - * @return the corresponding IsExpired. - */ - @JsonCreator - public static IsExpired fromString(String name) { - return fromString(name, IsExpired.class); - } - - /** - * Gets known IsExpired values. - * - * @return known IsExpired values. - */ - public static Collection values() { - return values(IsExpired.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsSelfSigned.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsSelfSigned.java deleted file mode 100644 index b19757e483b9a..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsSelfSigned.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Indicates if the certificate was self-signed. */ -public final class IsSelfSigned extends ExpandableStringEnum { - /** Static value True for IsSelfSigned. */ - public static final IsSelfSigned TRUE = fromString("True"); - - /** Static value False for IsSelfSigned. */ - public static final IsSelfSigned FALSE = fromString("False"); - - /** - * Creates a new instance of IsSelfSigned value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public IsSelfSigned() { - } - - /** - * Creates or finds a IsSelfSigned from its string representation. - * - * @param name a name to look for. - * @return the corresponding IsSelfSigned. - */ - @JsonCreator - public static IsSelfSigned fromString(String name) { - return fromString(name, IsSelfSigned.class); - } - - /** - * Gets known IsSelfSigned values. - * - * @return known IsSelfSigned values. - */ - public static Collection values() { - return values(IsSelfSigned.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsShortKeySize.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsShortKeySize.java deleted file mode 100644 index 41e04e546c4c9..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsShortKeySize.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Indicates the key size is considered too small to be secure for the algorithm. */ -public final class IsShortKeySize extends ExpandableStringEnum { - /** Static value True for IsShortKeySize. */ - public static final IsShortKeySize TRUE = fromString("True"); - - /** Static value False for IsShortKeySize. */ - public static final IsShortKeySize FALSE = fromString("False"); - - /** - * Creates a new instance of IsShortKeySize value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public IsShortKeySize() { - } - - /** - * Creates or finds a IsShortKeySize from its string representation. - * - * @param name a name to look for. - * @return the corresponding IsShortKeySize. - */ - @JsonCreator - public static IsShortKeySize fromString(String name) { - return fromString(name, IsShortKeySize.class); - } - - /** - * Gets known IsShortKeySize values. - * - * @return known IsShortKeySize values. - */ - public static Collection values() { - return values(IsShortKeySize.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsUpdateAvailable.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsUpdateAvailable.java deleted file mode 100644 index 65d0dc353b270..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsUpdateAvailable.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Flag if new update is available for the component. */ -public final class IsUpdateAvailable extends ExpandableStringEnum { - /** Static value True for IsUpdateAvailable. */ - public static final IsUpdateAvailable TRUE = fromString("True"); - - /** Static value False for IsUpdateAvailable. */ - public static final IsUpdateAvailable FALSE = fromString("False"); - - /** - * Creates a new instance of IsUpdateAvailable value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public IsUpdateAvailable() { - } - - /** - * Creates or finds a IsUpdateAvailable from its string representation. - * - * @param name a name to look for. - * @return the corresponding IsUpdateAvailable. - */ - @JsonCreator - public static IsUpdateAvailable fromString(String name) { - return fromString(name, IsUpdateAvailable.class); - } - - /** - * Gets known IsUpdateAvailable values. - * - * @return known IsUpdateAvailable values. - */ - public static Collection values() { - return values(IsUpdateAvailable.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsWeakSignature.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsWeakSignature.java deleted file mode 100644 index 3bf6c0e7eebc4..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/IsWeakSignature.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Indicates the signature algorithm used is insecure. */ -public final class IsWeakSignature extends ExpandableStringEnum { - /** Static value True for IsWeakSignature. */ - public static final IsWeakSignature TRUE = fromString("True"); - - /** Static value False for IsWeakSignature. */ - public static final IsWeakSignature FALSE = fromString("False"); - - /** - * Creates a new instance of IsWeakSignature value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public IsWeakSignature() { - } - - /** - * Creates or finds a IsWeakSignature from its string representation. - * - * @param name a name to look for. - * @return the corresponding IsWeakSignature. - */ - @JsonCreator - public static IsWeakSignature fromString(String name) { - return fromString(name, IsWeakSignature.class); - } - - /** - * Gets known IsWeakSignature values. - * - * @return known IsWeakSignature values. - */ - public static Collection values() { - return values(IsWeakSignature.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/NxFlag.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/NxFlag.java deleted file mode 100644 index b5d188482ce95..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/NxFlag.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** NX flag. */ -public final class NxFlag extends ExpandableStringEnum { - /** Static value True for NxFlag. */ - public static final NxFlag TRUE = fromString("True"); - - /** Static value False for NxFlag. */ - public static final NxFlag FALSE = fromString("False"); - - /** - * Creates a new instance of NxFlag value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public NxFlag() { - } - - /** - * Creates or finds a NxFlag from its string representation. - * - * @param name a name to look for. - * @return the corresponding NxFlag. - */ - @JsonCreator - public static NxFlag fromString(String name) { - return fromString(name, NxFlag.class); - } - - /** - * Gets known NxFlag values. - * - * @return known NxFlag values. - */ - public static Collection values() { - return values(NxFlag.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operation.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operation.java index 39a1f12261227..2cdb4b7cbba9f 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operation.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operation.java @@ -6,12 +6,14 @@ import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ String name(); @@ -19,14 +21,14 @@ public interface Operation { /** * Gets the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Localized display information for this particular operation. - * + * * @return the display value. */ OperationDisplay display(); @@ -34,7 +36,7 @@ public interface Operation { /** * Gets the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ Origin origin(); @@ -42,14 +44,14 @@ public interface Operation { /** * Gets the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ ActionType actionType(); /** * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationDisplay.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationDisplay.java index 1e43a50036cf0..ac4d6d0797883 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationDisplay.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Localized display information for this particular operation. */ +/** + * Localized display information for this particular operation. + */ @Immutable public final class OperationDisplay { /* @@ -37,14 +39,16 @@ public final class OperationDisplay { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: The localized friendly form of the resource provider name, e.g. "Microsoft Monitoring * Insights" or "Microsoft Compute". - * + * * @return the provider value. */ public String provider() { @@ -54,7 +58,7 @@ public String provider() { /** * Get the resource property: The localized friendly name of the resource type related to this operation. E.g. * "Virtual Machines" or "Job Schedule Collections". - * + * * @return the resource value. */ public String resource() { @@ -64,7 +68,7 @@ public String resource() { /** * Get the operation property: The concise, localized friendly name for the operation; suitable for dropdowns. E.g. * "Create or Update Virtual Machine", "Restart Virtual Machine". - * + * * @return the operation value. */ public String operation() { @@ -74,7 +78,7 @@ public String operation() { /** * Get the description property: The short, localized friendly description of the operation; suitable for tool tips * and detailed views. - * + * * @return the description value. */ public String description() { @@ -83,7 +87,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationListResult.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationListResult.java index 17cf6aedeb8c5..594bf6c944685 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationListResult.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/OperationListResult.java @@ -10,8 +10,8 @@ import java.util.List; /** - * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set of - * results. + * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set + * of results. */ @Immutable public final class OperationListResult { @@ -27,13 +27,15 @@ public final class OperationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationListResult class. */ + /** + * Creates an instance of OperationListResult class. + */ public OperationListResult() { } /** * Get the value property: List of operations supported by the resource provider. - * + * * @return the value value. */ public List value() { @@ -42,7 +44,7 @@ public List value() { /** * Get the nextLink property: URL to get the next set of operation list results (if there are any). - * + * * @return the nextLink value. */ public String nextLink() { @@ -51,7 +53,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operations.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operations.java index a82b53b187868..d31c5a67564d0 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operations.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Operations.java @@ -7,27 +7,29 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Lists the operations for this resource provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Lists the operations for this resource provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Origin.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Origin.java index f27c1fb3e4aed..b2737993d57bb 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Origin.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Origin.java @@ -13,18 +13,24 @@ * is "user,system". */ public final class Origin extends ExpandableStringEnum { - /** Static value user for Origin. */ + /** + * Static value user for Origin. + */ public static final Origin USER = fromString("user"); - /** Static value system for Origin. */ + /** + * Static value system for Origin. + */ public static final Origin SYSTEM = fromString("system"); - /** Static value user,system for Origin. */ + /** + * Static value user,system for Origin. + */ public static final Origin USER_SYSTEM = fromString("user,system"); /** * Creates a new instance of Origin value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +39,7 @@ public Origin() { /** * Creates or finds a Origin from its string representation. - * + * * @param name a name to look for. * @return the corresponding Origin. */ @@ -44,7 +50,7 @@ public static Origin fromString(String name) { /** * Gets known Origin values. - * + * * @return known Origin values. */ public static Collection values() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PairedKey.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PairedKey.java index bd643914f449e..8c578015d2a3f 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PairedKey.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PairedKey.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of a matching paired key or certificate. */ +/** + * Details of a matching paired key or certificate. + */ @Fluent public final class PairedKey { /* @@ -22,19 +24,15 @@ public final class PairedKey { @JsonProperty(value = "type") private String type; - /* - * Additional paired key properties + /** + * Creates an instance of PairedKey class. */ - @JsonProperty(value = "additionalProperties") - private Object additionalProperties; - - /** Creates an instance of PairedKey class. */ public PairedKey() { } /** * Get the id property: ID of the paired key or certificate. - * + * * @return the id value. */ public String id() { @@ -43,7 +41,7 @@ public String id() { /** * Set the id property: ID of the paired key or certificate. - * + * * @param id the id value to set. * @return the PairedKey object itself. */ @@ -54,7 +52,7 @@ public PairedKey withId(String id) { /** * Get the type property: The type indicating whether the paired object is a key or certificate. - * + * * @return the type value. */ public String type() { @@ -63,7 +61,7 @@ public String type() { /** * Set the type property: The type indicating whether the paired object is a key or certificate. - * + * * @param type the type value to set. * @return the PairedKey object itself. */ @@ -72,29 +70,9 @@ public PairedKey withType(String type) { return this; } - /** - * Get the additionalProperties property: Additional paired key properties. - * - * @return the additionalProperties value. - */ - public Object additionalProperties() { - return this.additionalProperties; - } - - /** - * Set the additionalProperties property: Additional paired key properties. - * - * @param additionalProperties the additionalProperties value to set. - * @return the PairedKey object itself. - */ - public PairedKey withAdditionalProperties(Object additionalProperties) { - this.additionalProperties = additionalProperties; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashList.java deleted file mode 100644 index 0086caee3e8cf..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashList.java +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** Password hashes list. */ -@Fluent -public final class PasswordHashList { - /* - * Password hashes list - */ - @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; - - /* - * The uri to fetch the next page of asset. - */ - @JsonProperty(value = "nextLink") - private String nextLink; - - /** Creates an instance of PasswordHashList class. */ - public PasswordHashList() { - } - - /** - * Get the value property: Password hashes list. - * - * @return the value value. - */ - public List value() { - return this.value; - } - - /** - * Get the nextLink property: The uri to fetch the next page of asset. - * - * @return the nextLink value. - */ - public String nextLink() { - return this.nextLink; - } - - /** - * Set the nextLink property: The uri to fetch the next page of asset. - * - * @param nextLink the nextLink value to set. - * @return the PasswordHashList object itself. - */ - public PasswordHashList withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (value() != null) { - value().forEach(e -> e.validate()); - } - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashListResult.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashListResult.java new file mode 100644 index 0000000000000..1630c7323f419 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashListResult.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of password hash results. + */ +@Fluent +public final class PasswordHashListResult { + /* + * The list of password hash results. + */ + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) + private List value; + + /* + * The uri to fetch the next page of resources. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of PasswordHashListResult class. + */ + public PasswordHashListResult() { + } + + /** + * Get the value property: The list of password hash results. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Get the nextLink property: The uri to fetch the next page of resources. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: The uri to fetch the next page of resources. + * + * @param nextLink the nextLink value to set. + * @return the PasswordHashListResult object itself. + */ + public PasswordHashListResult withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHash.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashResource.java similarity index 58% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHash.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashResource.java index d53611186f21a..740ea5503e3bc 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHash.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashResource.java @@ -4,63 +4,94 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner; + +/** + * An immutable client-side representation of PasswordHashResource. + */ +public interface PasswordHashResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); -/** An immutable client-side representation of PasswordHash. */ -public interface PasswordHash { /** * Gets the passwordHashId property: ID for password hash. - * + * * @return the passwordHashId value. */ String passwordHashId(); /** * Gets the filePath property: File path of the password hash. - * + * * @return the filePath value. */ String filePath(); /** * Gets the salt property: Salt of the password hash. - * + * * @return the salt value. */ String salt(); /** * Gets the hash property: Hash of the password. - * + * * @return the hash value. */ String hash(); /** * Gets the context property: Context of password hash. - * + * * @return the context value. */ String context(); /** * Gets the username property: User name of password hash. - * + * * @return the username value. */ String username(); /** * Gets the algorithm property: Algorithm of the password hash. - * + * * @return the algorithm value. */ String algorithm(); /** - * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashInner object. - * + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner object. + * * @return the inner object. */ - PasswordHashInner innerModel(); + PasswordHashResourceInner innerModel(); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashes.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashes.java new file mode 100644 index 0000000000000..1f03e53d8d7d5 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PasswordHashes.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of PasswordHashes. + */ +public interface PasswordHashes { + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists password hash analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of password hash results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PieFlag.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PieFlag.java deleted file mode 100644 index abfe7add20cf1..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/PieFlag.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** PIE flag. */ -public final class PieFlag extends ExpandableStringEnum { - /** Static value True for PieFlag. */ - public static final PieFlag TRUE = fromString("True"); - - /** Static value False for PieFlag. */ - public static final PieFlag FALSE = fromString("False"); - - /** - * Creates a new instance of PieFlag value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public PieFlag() { - } - - /** - * Creates or finds a PieFlag from its string representation. - * - * @param name a name to look for. - * @return the corresponding PieFlag. - */ - @JsonCreator - public static PieFlag fromString(String name) { - return fromString(name, PieFlag.class); - } - - /** - * Gets known PieFlag values. - * - * @return known PieFlag values. - */ - public static Collection values() { - return values(PieFlag.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ProvisioningState.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ProvisioningState.java index a010a27772506..2352f14898812 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ProvisioningState.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ProvisioningState.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning state of the resource. */ +/** + * Provisioning state of the resource. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Accepted for ProvisioningState. */ + /** + * Static value Accepted for ProvisioningState. + */ public static final ProvisioningState ACCEPTED = fromString("Accepted"); - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -44,7 +54,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/RelroFlag.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/RelroFlag.java deleted file mode 100644 index fe19992c8b295..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/RelroFlag.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** RELRO flag. */ -public final class RelroFlag extends ExpandableStringEnum { - /** Static value True for RelroFlag. */ - public static final RelroFlag TRUE = fromString("True"); - - /** Static value False for RelroFlag. */ - public static final RelroFlag FALSE = fromString("False"); - - /** - * Creates a new instance of RelroFlag value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public RelroFlag() { - } - - /** - * Creates or finds a RelroFlag from its string representation. - * - * @param name a name to look for. - * @return the corresponding RelroFlag. - */ - @JsonCreator - public static RelroFlag fromString(String name) { - return fromString(name, RelroFlag.class); - } - - /** - * Gets known RelroFlag values. - * - * @return known RelroFlag values. - */ - public static Collection values() { - return values(RelroFlag.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ComponentList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentListResult.java similarity index 66% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ComponentList.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentListResult.java index 61ca6c2e752ae..3da5ea803f773 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/ComponentList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentListResult.java @@ -5,41 +5,45 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List result for components. */ +/** + * List of SBOM results. + */ @Fluent -public final class ComponentList { +public final class SbomComponentListResult { /* - * The list of components. + * The list of SBOM components. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; + private List value; /* - * The uri to fetch the next page of asset. + * The uri to fetch the next page of resources. */ @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of ComponentList class. */ - public ComponentList() { + /** + * Creates an instance of SbomComponentListResult class. + */ + public SbomComponentListResult() { } /** - * Get the value property: The list of components. - * + * Get the value property: The list of SBOM components. + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Get the nextLink property: The uri to fetch the next page of asset. - * + * Get the nextLink property: The uri to fetch the next page of resources. + * * @return the nextLink value. */ public String nextLink() { @@ -47,19 +51,19 @@ public String nextLink() { } /** - * Set the nextLink property: The uri to fetch the next page of asset. - * + * Set the nextLink property: The uri to fetch the next page of resources. + * * @param nextLink the nextLink value to set. - * @return the ComponentList object itself. + * @return the SbomComponentListResult object itself. */ - public ComponentList withNextLink(String nextLink) { + public SbomComponentListResult withNextLink(String nextLink) { this.nextLink = nextLink; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentResource.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentResource.java new file mode 100644 index 0000000000000..b99bbba46f5e2 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponentResource.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; +import java.util.List; + +/** + * An immutable client-side representation of SbomComponentResource. + */ +public interface SbomComponentResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the componentId property: ID for the component. + * + * @return the componentId value. + */ + String componentId(); + + /** + * Gets the componentName property: Name for the component. + * + * @return the componentName value. + */ + String componentName(); + + /** + * Gets the version property: Version for the component. + * + * @return the version value. + */ + String version(); + + /** + * Gets the license property: License for the component. + * + * @return the license value. + */ + String license(); + + /** + * Gets the filePaths property: File paths related to the component. + * + * @return the filePaths value. + */ + List filePaths(); + + /** + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner object. + * + * @return the inner object. + */ + SbomComponentResourceInner innerModel(); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponents.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponents.java new file mode 100644 index 0000000000000..58c9762e44729 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SbomComponents.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of SbomComponents. + */ +public interface SbomComponents { + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId); + + /** + * Lists SBOM analysis results of a firmware. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of SBOM results as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, + String firmwareId, Context context); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Status.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Status.java index 6889c079b9ca1..8ac35b1a5ddb6 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Status.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Status.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The status of firmware scan. */ +/** + * The status of firmware scan. + */ public final class Status extends ExpandableStringEnum { - /** Static value Pending for Status. */ + /** + * Static value Pending for Status. + */ public static final Status PENDING = fromString("Pending"); - /** Static value Extracting for Status. */ + /** + * Static value Extracting for Status. + */ public static final Status EXTRACTING = fromString("Extracting"); - /** Static value Analyzing for Status. */ + /** + * Static value Analyzing for Status. + */ public static final Status ANALYZING = fromString("Analyzing"); - /** Static value Ready for Status. */ + /** + * Static value Ready for Status. + */ public static final Status READY = fromString("Ready"); - /** Static value Error for Status. */ + /** + * Static value Error for Status. + */ public static final Status ERROR = fromString("Error"); /** * Creates a new instance of Status value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public Status() { /** * Creates or finds a Status from its string representation. - * + * * @param name a name to look for. * @return the corresponding Status. */ @@ -47,7 +59,7 @@ public static Status fromString(String name) { /** * Gets known Status values. - * + * * @return known Status values. */ public static Collection values() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StatusMessage.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StatusMessage.java new file mode 100644 index 0000000000000..e94fac77c0a7e --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StatusMessage.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Error and status message. + */ +@Fluent +public final class StatusMessage { + /* + * The error code + */ + @JsonProperty(value = "errorCode") + private Long errorCode; + + /* + * The error or status message + */ + @JsonProperty(value = "message") + private String message; + + /** + * Creates an instance of StatusMessage class. + */ + public StatusMessage() { + } + + /** + * Get the errorCode property: The error code. + * + * @return the errorCode value. + */ + public Long errorCode() { + return this.errorCode; + } + + /** + * Set the errorCode property: The error code. + * + * @param errorCode the errorCode value to set. + * @return the StatusMessage object itself. + */ + public StatusMessage withErrorCode(Long errorCode) { + this.errorCode = errorCode; + return this; + } + + /** + * Get the message property: The error or status message. + * + * @return the message value. + */ + public String message() { + return this.message; + } + + /** + * Set the message property: The error or status message. + * + * @param message the message value to set. + * @return the StatusMessage object itself. + */ + public StatusMessage withMessage(String message) { + this.message = message; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StrippedFlag.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StrippedFlag.java deleted file mode 100644 index aa0cb1c78750c..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/StrippedFlag.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** Stripped flag. */ -public final class StrippedFlag extends ExpandableStringEnum { - /** Static value True for StrippedFlag. */ - public static final StrippedFlag TRUE = fromString("True"); - - /** Static value False for StrippedFlag. */ - public static final StrippedFlag FALSE = fromString("False"); - - /** - * Creates a new instance of StrippedFlag value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public StrippedFlag() { - } - - /** - * Creates or finds a StrippedFlag from its string representation. - * - * @param name a name to look for. - * @return the corresponding StrippedFlag. - */ - @JsonCreator - public static StrippedFlag fromString(String name) { - return fromString(name, StrippedFlag.class); - } - - /** - * Gets known StrippedFlag values. - * - * @return known StrippedFlag values. - */ - public static Collection values() { - return values(StrippedFlag.class); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Summaries.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Summaries.java new file mode 100644 index 0000000000000..d38ef7b2b510b --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Summaries.java @@ -0,0 +1,72 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of Summaries. + */ +public interface Summaries { + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId); + + /** + * Lists analysis result summary names of a firmware. To fetch the full summary data, get that summary by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of analysis summaries as paginated response with {@link PagedIterable}. + */ + PagedIterable listByFirmware(String resourceGroupName, String workspaceName, String firmwareId, + Context context); + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String workspaceName, String firmwareId, + SummaryName summaryName, Context context); + + /** + * Get an analysis result summary of a firmware by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the firmware analysis workspace. + * @param firmwareId The id of the firmware. + * @param summaryName The Firmware analysis summary name describing the type of summary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an analysis result summary of a firmware by name. + */ + SummaryResource get(String resourceGroupName, String workspaceName, String firmwareId, SummaryName summaryName); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryListResult.java similarity index 68% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateList.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryListResult.java index 65484a36c6410..0cff22b0e120c 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/CryptoCertificateList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryListResult.java @@ -5,41 +5,45 @@ package com.azure.resourcemanager.iotfirmwaredefense.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateInner; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Crypto certificates list. */ +/** + * List of analysis summaries. + */ @Fluent -public final class CryptoCertificateList { +public final class SummaryListResult { /* - * Crypto certificates list + * The list of summaries. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; + private List value; /* - * The uri to fetch the next page of asset. + * The uri to fetch the next page of resources. */ @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of CryptoCertificateList class. */ - public CryptoCertificateList() { + /** + * Creates an instance of SummaryListResult class. + */ + public SummaryListResult() { } /** - * Get the value property: Crypto certificates list. - * + * Get the value property: The list of summaries. + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Get the nextLink property: The uri to fetch the next page of asset. - * + * Get the nextLink property: The uri to fetch the next page of resources. + * * @return the nextLink value. */ public String nextLink() { @@ -47,19 +51,19 @@ public String nextLink() { } /** - * Set the nextLink property: The uri to fetch the next page of asset. - * + * Set the nextLink property: The uri to fetch the next page of resources. + * * @param nextLink the nextLink value to set. - * @return the CryptoCertificateList object itself. + * @return the SummaryListResult object itself. */ - public CryptoCertificateList withNextLink(String nextLink) { + public SummaryListResult withNextLink(String nextLink) { this.nextLink = nextLink; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryName.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryName.java new file mode 100644 index 0000000000000..8357178f0e4d5 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryName.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for SummaryName. + */ +public final class SummaryName extends ExpandableStringEnum { + /** + * Static value Firmware for SummaryName. + */ + public static final SummaryName FIRMWARE = fromString("Firmware"); + + /** + * Static value CVE for SummaryName. + */ + public static final SummaryName CVE = fromString("CVE"); + + /** + * Static value BinaryHardening for SummaryName. + */ + public static final SummaryName BINARY_HARDENING = fromString("BinaryHardening"); + + /** + * Static value CryptoCertificate for SummaryName. + */ + public static final SummaryName CRYPTO_CERTIFICATE = fromString("CryptoCertificate"); + + /** + * Static value CryptoKey for SummaryName. + */ + public static final SummaryName CRYPTO_KEY = fromString("CryptoKey"); + + /** + * Creates a new instance of SummaryName value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public SummaryName() { + } + + /** + * Creates or finds a SummaryName from its string representation. + * + * @param name a name to look for. + * @return the corresponding SummaryName. + */ + @JsonCreator + public static SummaryName fromString(String name) { + return fromString(name, SummaryName.class); + } + + /** + * Gets known SummaryName values. + * + * @return known SummaryName values. + */ + public static Collection values() { + return values(SummaryName.class); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResource.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResource.java new file mode 100644 index 0000000000000..0a5dad26fbafe --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResource.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; + +/** + * An immutable client-side representation of SummaryResource. + */ +public interface SummaryResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: Properties of an analysis summary. + * + * @return the properties value. + */ + SummaryResourceProperties properties(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner object. + * + * @return the inner object. + */ + SummaryResourceInner innerModel(); +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResourceProperties.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResourceProperties.java new file mode 100644 index 0000000000000..678a5fb697354 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryResourceProperties.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * Properties of an analysis summary. + */ +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "summaryType", + defaultImpl = SummaryResourceProperties.class) +@JsonTypeName("SummaryResourceProperties") +@JsonSubTypes({ + @JsonSubTypes.Type(name = "Firmware", value = FirmwareSummary.class), + @JsonSubTypes.Type(name = "CVE", value = CveSummary.class), + @JsonSubTypes.Type(name = "BinaryHardening", value = BinaryHardeningSummaryResource.class), + @JsonSubTypes.Type(name = "CryptoCertificate", value = CryptoCertificateSummaryResource.class), + @JsonSubTypes.Type(name = "CryptoKey", value = CryptoKeySummaryResource.class) }) +@Immutable +public class SummaryResourceProperties { + /** + * Creates an instance of SummaryResourceProperties class. + */ + public SummaryResourceProperties() { + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryType.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryType.java new file mode 100644 index 0000000000000..fc1daadefba34 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/SummaryType.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Describes the type of summary. + */ +public final class SummaryType extends ExpandableStringEnum { + /** + * Static value Firmware for SummaryType. + */ + public static final SummaryType FIRMWARE = fromString("Firmware"); + + /** + * Static value CVE for SummaryType. + */ + public static final SummaryType CVE = fromString("CVE"); + + /** + * Static value BinaryHardening for SummaryType. + */ + public static final SummaryType BINARY_HARDENING = fromString("BinaryHardening"); + + /** + * Static value CryptoCertificate for SummaryType. + */ + public static final SummaryType CRYPTO_CERTIFICATE = fromString("CryptoCertificate"); + + /** + * Static value CryptoKey for SummaryType. + */ + public static final SummaryType CRYPTO_KEY = fromString("CryptoKey"); + + /** + * Creates a new instance of SummaryType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public SummaryType() { + } + + /** + * Creates or finds a SummaryType from its string representation. + * + * @param name a name to look for. + * @return the corresponding SummaryType. + */ + @JsonCreator + public static SummaryType fromString(String name) { + return fromString(name, SummaryType.class); + } + + /** + * Gets known SummaryType values. + * + * @return known SummaryType values. + */ + public static Collection values() { + return values(SummaryType.class); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/UrlToken.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/UrlToken.java index 0dfd6fe48e003..20c9fafe69cec 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/UrlToken.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/UrlToken.java @@ -6,25 +6,20 @@ import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.UrlTokenInner; -/** An immutable client-side representation of UrlToken. */ +/** + * An immutable client-side representation of UrlToken. + */ public interface UrlToken { /** * Gets the url property: SAS URL for creating or accessing a blob file. - * + * * @return the url value. */ String url(); - /** - * Gets the uploadUrl property: SAS URL for file uploading. Kept for backwards compatibility. - * - * @return the uploadUrl value. - */ - String uploadUrl(); - /** * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.UrlTokenInner object. - * + * * @return the inner object. */ UrlTokenInner innerModel(); diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspace.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspace.java index 19e1332e5267e..52511306f9259 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspace.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspace.java @@ -11,104 +11,111 @@ import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.WorkspaceInner; import java.util.Map; -/** An immutable client-side representation of Workspace. */ +/** + * An immutable client-side representation of Workspace. + */ public interface Workspace { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.iotfirmwaredefense.fluent.models.WorkspaceInner object. - * + * * @return the inner object. */ WorkspaceInner innerModel(); - /** The entirety of the Workspace definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the Workspace definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The Workspace definition stages. */ + /** + * The Workspace definition stages. + */ interface DefinitionStages { - /** The first stage of the Workspace definition. */ + /** + * The first stage of the Workspace definition. + */ interface Blank extends WithLocation { } - /** The stage of the Workspace definition allowing to specify location. */ + /** + * The stage of the Workspace definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -116,18 +123,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Workspace definition allowing to specify parent resource. */ + /** + * The stage of the Workspace definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -141,25 +150,27 @@ interface WithResourceGroup { interface WithCreate extends DefinitionStages.WithTags { /** * Executes the create request. - * + * * @return the created resource. */ Workspace create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Workspace create(Context context); } - /** The stage of the Workspace definition allowing to specify tags. */ + /** + * The stage of the Workspace definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -169,43 +180,47 @@ interface WithTags { /** * Begins update for the Workspace resource. - * + * * @return the stage of resource update. */ Workspace.Update update(); - /** The template for Workspace update. */ + /** + * The template for Workspace update. + */ interface Update { /** * Executes the update request. - * + * * @return the updated resource. */ Workspace apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Workspace apply(Context context); } - /** The Workspace update stages. */ + /** + * The Workspace update stages. + */ interface UpdateStages { } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Workspace refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -213,7 +228,7 @@ interface UpdateStages { /** * The operation to get a url for file upload. - * + * * @param generateUploadUrl Parameters when requesting a URL to upload firmware. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -225,7 +240,7 @@ interface UpdateStages { /** * The operation to get a url for file upload. - * + * * @param generateUploadUrl Parameters when requesting a URL to upload firmware. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceList.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceList.java index 0d29f6a2a125d..cefa79a509f29 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceList.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Return a list of firmware analysis workspaces. */ +/** + * Return a list of firmware analysis workspaces. + */ @Fluent public final class WorkspaceList { /* @@ -24,13 +26,15 @@ public final class WorkspaceList { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of WorkspaceList class. */ + /** + * Creates an instance of WorkspaceList class. + */ public WorkspaceList() { } /** * Get the value property: The list of firmware analysis workspaces. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The uri to fetch the next page of asset. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Set the nextLink property: The uri to fetch the next page of asset. - * + * * @param nextLink the nextLink value to set. * @return the WorkspaceList object itself. */ @@ -59,7 +63,7 @@ public WorkspaceList withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceUpdateDefinition.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceUpdateDefinition.java index 2a69be24b9974..7d021ea923fd4 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceUpdateDefinition.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/WorkspaceUpdateDefinition.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.WorkspaceProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Firmware analysis workspace. */ +/** + * Firmware analysis workspace. + */ @Immutable public final class WorkspaceUpdateDefinition { /* @@ -17,13 +19,15 @@ public final class WorkspaceUpdateDefinition { @JsonProperty(value = "properties") private WorkspaceProperties innerProperties; - /** Creates an instance of WorkspaceUpdateDefinition class. */ + /** + * Creates an instance of WorkspaceUpdateDefinition class. + */ public WorkspaceUpdateDefinition() { } /** * Get the innerProperties property: The editable workspace properties. - * + * * @return the innerProperties value. */ private WorkspaceProperties innerProperties() { @@ -32,7 +36,7 @@ private WorkspaceProperties innerProperties() { /** * Get the provisioningState property: Provisioning state of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -41,7 +45,7 @@ public ProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspaces.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspaces.java index 1a4f91415054c..602f762fd8edb 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspaces.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/Workspaces.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Workspaces. */ +/** + * Resource collection API of Workspaces. + */ public interface Workspaces { /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return return a list of firmware analysis workspaces as paginated response with {@link PagedIterable}. @@ -21,7 +23,7 @@ public interface Workspaces { /** * Lists all of the firmware analysis workspaces in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,7 +34,7 @@ public interface Workspaces { /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -43,7 +45,7 @@ public interface Workspaces { /** * Lists all of the firmware analysis workspaces in the specified resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -55,7 +57,7 @@ public interface Workspaces { /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -68,7 +70,7 @@ public interface Workspaces { /** * The operation to delete a firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,7 +81,7 @@ public interface Workspaces { /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param context The context to associate with this operation. @@ -92,7 +94,7 @@ public interface Workspaces { /** * Get firmware analysis workspace. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -104,7 +106,7 @@ public interface Workspaces { /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -114,12 +116,12 @@ public interface Workspaces { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file along with {@link Response}. */ - Response generateUploadUrlWithResponse( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl, Context context); + Response generateUploadUrlWithResponse(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl, Context context); /** * The operation to get a url for file upload. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the firmware analysis workspace. * @param generateUploadUrl Parameters when requesting a URL to upload firmware. @@ -128,12 +130,12 @@ Response generateUploadUrlWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return url data for creating or accessing a blob file. */ - UrlToken generateUploadUrl( - String resourceGroupName, String workspaceName, GenerateUploadUrlRequest generateUploadUrl); + UrlToken generateUploadUrl(String resourceGroupName, String workspaceName, + GenerateUploadUrlRequest generateUploadUrl); /** * Get firmware analysis workspace. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -144,7 +146,7 @@ UrlToken generateUploadUrl( /** * Get firmware analysis workspace. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -156,7 +158,7 @@ UrlToken generateUploadUrl( /** * The operation to delete a firmware analysis workspace. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -166,7 +168,7 @@ UrlToken generateUploadUrl( /** * The operation to delete a firmware analysis workspace. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -178,7 +180,7 @@ UrlToken generateUploadUrl( /** * Begins definition for a new Workspace resource. - * + * * @param name resource name. * @return the first stage of the new Workspace definition. */ diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/package-info.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/package-info.java index 2c614579ee6b7..59a8f8f9bdb21 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/package-info.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the data models for Fist. The definitions and parameters in this swagger specification will be - * used to manage the IoT Firmware Defense resources. + * Package containing the data models for IoTFirmwareDefense. + * Firmware & IoT Security REST API. */ package com.azure.resourcemanager.iotfirmwaredefense.models; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/package-info.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/package-info.java index 7f678d3e0d000..adbe3ccf3e75d 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/package-info.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/com/azure/resourcemanager/iotfirmwaredefense/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the classes for Fist. The definitions and parameters in this swagger specification will be used to - * manage the IoT Firmware Defense resources. + * Package containing the classes for IoTFirmwareDefense. + * Firmware & IoT Security REST API. */ package com.azure.resourcemanager.iotfirmwaredefense; diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/module-info.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/module-info.java index 8d30e276bd315..9a691f3ca849a 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/module-info.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.iotfirmwaredefense { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.iotfirmwaredefense; exports com.azure.resourcemanager.iotfirmwaredefense.fluent; exports com.azure.resourcemanager.iotfirmwaredefense.fluent.models; exports com.azure.resourcemanager.iotfirmwaredefense.models; - - opens com.azure.resourcemanager.iotfirmwaredefense.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.iotfirmwaredefense.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.iotfirmwaredefense.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.iotfirmwaredefense.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/proxy-config.json b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/proxy-config.json new file mode 100644 index 0000000000000..18ede4d0367b3 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.BinaryHardeningsClientImpl$BinaryHardeningsService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.CryptoCertificatesClientImpl$CryptoCertificatesService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.CryptoKeysClientImpl$CryptoKeysService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.CvesClientImpl$CvesService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.FirmwaresClientImpl$FirmwaresService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.PasswordHashesClientImpl$PasswordHashesService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.SbomComponentsClientImpl$SbomComponentsService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.SummariesClientImpl$SummariesService" ], [ "com.azure.resourcemanager.iotfirmwaredefense.implementation.WorkspacesClientImpl$WorkspacesService" ] ] \ No newline at end of file diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/reflect-config.json b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/reflect-config.json new file mode 100644 index 0000000000000..b5bbd45f95a28 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-iotfirmwaredefense/reflect-config.json @@ -0,0 +1,256 @@ +[ { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningFeatures", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificateResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoCertificate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.PairedKey", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeyListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKeyResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CryptoKey", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CveListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CveLink", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareUpdateDefinition", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.UrlTokenInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.OperationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.PasswordHashListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHashResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.PasswordHash", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponentListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponent", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.SummaryListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResourceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.WorkspaceList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.WorkspaceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.fluent.models.WorkspaceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.WorkspaceUpdateDefinition", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.GenerateUploadUrlRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareSummary", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CveSummary", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningSummaryResource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CryptoCertificateSummaryResource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.CryptoKeySummaryResource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.Status", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.ProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.Origin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.ActionType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.SummaryType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListByFirmwareSamples.java new file mode 100644 index 0000000000000..0048470863a7f --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for BinaryHardening ListByFirmware. + */ +public final class BinaryHardeningListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * BinaryHardening_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: BinaryHardening_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void binaryHardeningListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.binaryHardenings().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * BinaryHardening_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: BinaryHardening_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void binaryHardeningListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.binaryHardenings().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificatesListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificatesListByFirmwareSamples.java new file mode 100644 index 0000000000000..30ebad1861044 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificatesListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for CryptoCertificates ListByFirmware. + */ +public final class CryptoCertificatesListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoCertificates_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: CryptoCertificates_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void cryptoCertificatesListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.cryptoCertificates().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoCertificates_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: CryptoCertificates_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void cryptoCertificatesListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.cryptoCertificates().listByFirmware("FirmwareAnalysisRG", "default", + "109a9886-50bf-85a8-9d75-000000000000", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoKeysListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoKeysListByFirmwareSamples.java new file mode 100644 index 0000000000000..24b31df3d1f42 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoKeysListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for CryptoKeys ListByFirmware. + */ +public final class CryptoKeysListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoKeys_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: CryptoKeys_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void cryptoKeysListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.cryptoKeys().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * CryptoKeys_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: CryptoKeys_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void cryptoKeysListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.cryptoKeys().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareSamples.java new file mode 100644 index 0000000000000..88c909fc21bb2 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for Cves ListByFirmware. + */ +public final class CvesListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Cves_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: Cves_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void cvesListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.cves().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Cves_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: Cves_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void cvesListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.cves().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareCreateSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareCreateSamples.java deleted file mode 100644 index c8269ee45957b..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareCreateSamples.java +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.serializer.SerializerEncoding; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.io.IOException; -import java.util.Arrays; - -/** Samples for Firmware Create. */ -public final class FirmwareCreateSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Create_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_Create_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareCreateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager.firmwares().define("umrkdttp").withExistingWorkspace("rgworkspaces-firmwares", "A7").create(); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Create_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_Create_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareCreateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) throws IOException { - manager - .firmwares() - .define("umrkdttp") - .withExistingWorkspace("rgworkspaces-firmwares", "A7") - .withFileName("wresexxulcdsdd") - .withVendor("vycmdhgtmepcptyoubztiuudpkcpd") - .withModel("f") - .withVersion("s") - .withDescription("uz") - .withFileSize(17L) - .withStatus(Status.PENDING) - .withStatusMessages( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"message\":\"ulvhmhokezathzzauiitu\"}", Object.class, SerializerEncoding.JSON))) - .create(); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareDeleteSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareDeleteSamples.java deleted file mode 100644 index 3cab11d210d40..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareDeleteSamples.java +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware Delete. */ -public final class FirmwareDeleteSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Delete_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_Delete_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareDeleteMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Delete_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_Delete_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareDeleteMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningDetailsSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningDetailsSamples.java deleted file mode 100644 index d4f8107675fc4..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningDetailsSamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateBinaryHardeningDetails. */ -public final class FirmwareGenerateBinaryHardeningDetailsSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningDetails_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningDetails_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningDetailsMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningDetails_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningDetails_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningDetailsMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningSummarySamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningSummarySamples.java deleted file mode 100644 index 20b0d9b246277..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateBinaryHardeningSummarySamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateBinaryHardeningSummary. */ -public final class FirmwareGenerateBinaryHardeningSummarySamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningSummary_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningSummary_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningSummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateBinaryHardeningSummary_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateBinaryHardeningSummary_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateBinaryHardeningSummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateBinaryHardeningSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateComponentDetailsSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateComponentDetailsSamples.java deleted file mode 100644 index faa549618b2de..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateComponentDetailsSamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateComponentDetails. */ -public final class FirmwareGenerateComponentDetailsSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateComponentDetails_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateComponentDetails_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateComponentDetailsMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateComponentDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateComponentDetails_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateComponentDetails_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateComponentDetailsMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateComponentDetailsWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoCertificateSummarySamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoCertificateSummarySamples.java deleted file mode 100644 index c56f1f5990005..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoCertificateSummarySamples.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateCryptoCertificateSummary. */ -public final class FirmwareGenerateCryptoCertificateSummarySamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoCertificateSummary_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateCryptoCertificateSummary_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateCryptoCertificateSummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoCertificateSummaryWithResponse( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoCertificateSummary_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateCryptoCertificateSummary_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateCryptoCertificateSummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoCertificateSummaryWithResponse( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoKeySummarySamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoKeySummarySamples.java deleted file mode 100644 index 06e5973aaa209..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCryptoKeySummarySamples.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateCryptoKeySummary. */ -public final class FirmwareGenerateCryptoKeySummarySamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoKeySummary_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateCryptoKeySummary_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateCryptoKeySummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoKeySummaryWithResponse( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCryptoKeySummary_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateCryptoKeySummary_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateCryptoKeySummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCryptoKeySummaryWithResponse( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCveSummarySamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCveSummarySamples.java deleted file mode 100644 index 1555d82dd3db3..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateCveSummarySamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateCveSummary. */ -public final class FirmwareGenerateCveSummarySamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCveSummary_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateCveSummary_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateCveSummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCveSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateCveSummary_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateCveSummary_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateCveSummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateCveSummaryWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateDownloadUrlSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateDownloadUrlSamples.java deleted file mode 100644 index 2131b705806ef..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateDownloadUrlSamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateDownloadUrl. */ -public final class FirmwareGenerateDownloadUrlSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateDownloadUrl_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateDownloadUrl_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateDownloadUrlMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateDownloadUrl_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateDownloadUrl_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateDownloadUrlMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateFilesystemDownloadUrlSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateFilesystemDownloadUrlSamples.java deleted file mode 100644 index 9e5eb41afdbd7..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateFilesystemDownloadUrlSamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateFilesystemDownloadUrl. */ -public final class FirmwareGenerateFilesystemDownloadUrlSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateFilesystemDownloadUrl_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateFilesystemDownloadUrl_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateFilesystemDownloadUrlMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateFilesystemDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateFilesystemDownloadUrl_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateFilesystemDownloadUrl_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateFilesystemDownloadUrlMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateFilesystemDownloadUrlWithResponse( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateSummarySamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateSummarySamples.java deleted file mode 100644 index 83908a76241ab..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGenerateSummarySamples.java +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware GenerateSummary. */ -public final class FirmwareGenerateSummarySamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateSummary_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateSummary_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateSummaryMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateSummaryWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_GenerateSummary_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_GenerateSummary_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGenerateSummaryMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .generateSummaryWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGetSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGetSamples.java deleted file mode 100644 index 15b1e702fa283..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareGetSamples.java +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware Get. */ -public final class FirmwareGetSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Get_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_Get_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGetMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Get_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_Get_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareGetMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateBinaryHardeningListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateBinaryHardeningListSamples.java deleted file mode 100644 index 2954146f65cc4..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateBinaryHardeningListSamples.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware ListGenerateBinaryHardeningList. */ -public final class FirmwareListGenerateBinaryHardeningListSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateBinaryHardeningList_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateBinaryHardeningList_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateBinaryHardeningListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateBinaryHardeningList( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateBinaryHardeningList_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateBinaryHardeningList_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateBinaryHardeningListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateBinaryHardeningList( - "rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateComponentListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateComponentListSamples.java deleted file mode 100644 index 616b2e82ba6ef..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateComponentListSamples.java +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware ListGenerateComponentList. */ -public final class FirmwareListGenerateComponentListSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateComponentList_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateComponentList_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateComponentListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateComponentList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateComponentList_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateComponentList_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateComponentListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateComponentList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoCertificateListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoCertificateListSamples.java deleted file mode 100644 index d7881e705f373..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoCertificateListSamples.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware ListGenerateCryptoCertificateList. */ -public final class FirmwareListGenerateCryptoCertificateListSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoCertificateList_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateCryptoCertificateList_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateCryptoCertificateListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoCertificateList( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoCertificateList_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateCryptoCertificateList_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateCryptoCertificateListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoCertificateList( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoKeyListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoKeyListSamples.java deleted file mode 100644 index 0c01feaf5924c..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCryptoKeyListSamples.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware ListGenerateCryptoKeyList. */ -public final class FirmwareListGenerateCryptoKeyListSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoKeyList_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateCryptoKeyList_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateCryptoKeyListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoKeyList( - "rgworkspaces-firmwares", "j5QE_", "wujtpcgypfpqseyrsebolarkspy", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCryptoKeyList_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateCryptoKeyList_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateCryptoKeyListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCryptoKeyList( - "FirmwareAnalysisRG", - "default", - "DECAFBAD-0000-0000-0000-BADBADBADBAD", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCveListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCveListSamples.java deleted file mode 100644 index ba8a9880a30e9..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGenerateCveListSamples.java +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware ListGenerateCveList. */ -public final class FirmwareListGenerateCveListSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCveList_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateCveList_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateCveListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCveList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGenerateCveList_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGenerateCveList_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGenerateCveListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGenerateCveList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGeneratePasswordHashListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGeneratePasswordHashListSamples.java deleted file mode 100644 index a207f4ba6e5b3..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListGeneratePasswordHashListSamples.java +++ /dev/null @@ -1,38 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -/** Samples for Firmware ListGeneratePasswordHashList. */ -public final class FirmwareListGeneratePasswordHashListSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGeneratePasswordHashList_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGeneratePasswordHashList_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGeneratePasswordHashListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGeneratePasswordHashList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListGeneratePasswordHashList_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_ListGeneratePasswordHashList_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareListGeneratePasswordHashListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .firmwares() - .listGeneratePasswordHashList("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateSamples.java deleted file mode 100644 index 110f00bd708b9..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateSamples.java +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.serializer.SerializerEncoding; -import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.io.IOException; -import java.util.Arrays; - -/** Samples for Firmware Update. */ -public final class FirmwareUpdateSamples { - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Update_MaximumSet_Gen.json - */ - /** - * Sample code: Firmware_Update_MaximumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareUpdateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) throws IOException { - Firmware resource = - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withFileName("wresexxulcdsdd") - .withVendor("vycmdhgtmepcptyoubztiuudpkcpd") - .withModel("f") - .withVersion("s") - .withDescription("uz") - .withFileSize(17L) - .withStatus(Status.PENDING) - .withStatusMessages( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"message\":\"ulvhmhokezathzzauiitu\"}", Object.class, SerializerEncoding.JSON))) - .apply(); - } - - /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_Update_MinimumSet_Gen.json - */ - /** - * Sample code: Firmware_Update_MinimumSet_Gen. - * - * @param manager Entry point to IoTFirmwareDefenseManager. - */ - public static void firmwareUpdateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - Firmware resource = - manager - .firmwares() - .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE) - .getValue(); - resource.update().apply(); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateSamples.java new file mode 100644 index 0000000000000..6e4d7434616ad --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateSamples.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; +import java.util.Arrays; + +/** + * Samples for Firmwares Create. + */ +public final class FirmwaresCreateSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Create_MaximumSet_Gen.json + */ + /** + * Sample code: Firmwares_Create_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresCreateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().define("umrkdttp").withExistingWorkspace("rgworkspaces-firmwares", "A7") + .withFileName("wresexxulcdsdd").withVendor("vycmdhgtmepcptyoubztiuudpkcpd").withModel("f").withVersion("s") + .withDescription("uz").withFileSize(17L).withStatus(Status.PENDING) + .withStatusMessages(Arrays.asList(new StatusMessage().withMessage("ulvhmhokezathzzauiitu"))).create(); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Create_MinimumSet_Gen.json + */ + /** + * Sample code: Firmwares_Create_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresCreateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().define("umrkdttp").withExistingWorkspace("rgworkspaces-firmwares", "A7").create(); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteSamples.java new file mode 100644 index 0000000000000..dc3b28777b04d --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for Firmwares Delete. + */ +public final class FirmwaresDeleteSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Delete_MinimumSet_Gen.json + */ + /** + * Sample code: Firmwares_Delete_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresDeleteMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Delete_MaximumSet_Gen.json + */ + /** + * Sample code: Firmwares_Delete_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresDeleteMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().deleteWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlSamples.java new file mode 100644 index 0000000000000..0aa7d926d898b --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for Firmwares GenerateDownloadUrl. + */ +public final class FirmwaresGenerateDownloadUrlSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateDownloadUrl_MaximumSet_Gen.json + */ + /** + * Sample code: Firmwares_GenerateDownloadUrl_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void firmwaresGenerateDownloadUrlMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().generateDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateDownloadUrl_MinimumSet_Gen.json + */ + /** + * Sample code: Firmwares_GenerateDownloadUrl_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void firmwaresGenerateDownloadUrlMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().generateDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlSamples.java new file mode 100644 index 0000000000000..7f5cb01b37609 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for Firmwares GenerateFilesystemDownloadUrl. + */ +public final class FirmwaresGenerateFilesystemDownloadUrlSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateFilesystemDownloadUrl_MaximumSet_Gen.json + */ + /** + * Sample code: Firmwares_GenerateFilesystemDownloadUrl_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void firmwaresGenerateFilesystemDownloadUrlMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().generateFilesystemDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_GenerateFilesystemDownloadUrl_MinimumSet_Gen.json + */ + /** + * Sample code: Firmwares_GenerateFilesystemDownloadUrl_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void firmwaresGenerateFilesystemDownloadUrlMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().generateFilesystemDownloadUrlWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetSamples.java new file mode 100644 index 0000000000000..0eaa1e2ee2510 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for Firmwares Get. + */ +public final class FirmwaresGetSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Get_MinimumSet_Gen.json + */ + /** + * Sample code: Firmwares_Get_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresGetMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Get_MaximumSet_Gen.json + */ + /** + * Sample code: Firmwares_Get_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresGetMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.firmwares().getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListByWorkspaceSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceSamples.java similarity index 62% rename from sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListByWorkspaceSamples.java rename to sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceSamples.java index af4e73bbedbb1..5ecab64bd5018 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListByWorkspaceSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceSamples.java @@ -4,30 +4,34 @@ package com.azure.resourcemanager.iotfirmwaredefense.generated; -/** Samples for Firmware ListByWorkspace. */ -public final class FirmwareListByWorkspaceSamples { +/** + * Samples for Firmwares ListByWorkspace. + */ +public final class FirmwaresListByWorkspaceSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListByWorkspace_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_ListByWorkspace_MaximumSet_Gen.json */ /** - * Sample code: Firmware_ListByWorkspace_MaximumSet_Gen. - * + * Sample code: Firmwares_ListByWorkspace_MaximumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListByWorkspaceMaximumSetGen( + public static void firmwaresListByWorkspaceMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.firmwares().listByWorkspace("rgworkspaces-firmwares", "A7", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Firmware_ListByWorkspace_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_ListByWorkspace_MinimumSet_Gen.json */ /** - * Sample code: Firmware_ListByWorkspace_MinimumSet_Gen. - * + * Sample code: Firmwares_ListByWorkspace_MinimumSet_Gen. + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void firmwareListByWorkspaceMinimumSetGen( + public static void firmwaresListByWorkspaceMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.firmwares().listByWorkspace("rgworkspaces-firmwares", "A7", com.azure.core.util.Context.NONE); } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresUpdateSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresUpdateSamples.java new file mode 100644 index 0000000000000..1b9e31910bf82 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresUpdateSamples.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; +import com.azure.resourcemanager.iotfirmwaredefense.models.Status; +import com.azure.resourcemanager.iotfirmwaredefense.models.StatusMessage; +import java.util.Arrays; + +/** + * Samples for Firmwares Update. + */ +public final class FirmwaresUpdateSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Update_MaximumSet_Gen.json + */ + /** + * Sample code: Firmwares_Update_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresUpdateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Firmware resource = manager.firmwares() + .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE).getValue(); + resource.update().withFileName("wresexxulcdsdd").withVendor("vycmdhgtmepcptyoubztiuudpkcpd").withModel("f") + .withVersion("s").withDescription("uz").withFileSize(17L).withStatus(Status.PENDING) + .withStatusMessages(Arrays.asList(new StatusMessage().withMessage("ulvhmhokezathzzauiitu"))).apply(); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Firmwares_Update_MinimumSet_Gen.json + */ + /** + * Sample code: Firmwares_Update_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + firmwaresUpdateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Firmware resource = manager.firmwares() + .getWithResponse("rgworkspaces-firmwares", "A7", "umrkdttp", com.azure.core.util.Context.NONE).getValue(); + resource.update().apply(); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListSamples.java index 07e7ab015ddf7..4d86e03e18ca9 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListSamples.java @@ -4,31 +4,35 @@ package com.azure.resourcemanager.iotfirmwaredefense.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Operations_List_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Operations_List_MinimumSet_Gen.json */ /** * Sample code: Operations_List_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void operationsListMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + public static void + operationsListMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.operations().list(com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Operations_List_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Operations_List_MaximumSet_Gen.json */ /** * Sample code: Operations_List_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void operationsListMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + public static void + operationsListMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.operations().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PasswordHashesListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PasswordHashesListByFirmwareSamples.java new file mode 100644 index 0000000000000..387faa2b4674e --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PasswordHashesListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for PasswordHashes ListByFirmware. + */ +public final class PasswordHashesListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * PasswordHashes_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: PasswordHashes_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void passwordHashesListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.passwordHashes().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * PasswordHashes_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: PasswordHashes_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void passwordHashesListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.passwordHashes().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareSamples.java new file mode 100644 index 0000000000000..82db2c0317ddf --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for SbomComponents ListByFirmware. + */ +public final class SbomComponentsListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * SbomComponents_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: SbomComponents_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void sbomComponentsListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.sbomComponents().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * SbomComponents_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: SbomComponents_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void sbomComponentsListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.sbomComponents().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetSamples.java new file mode 100644 index 0000000000000..0df6c700a4314 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetSamples.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName; + +/** + * Samples for Summaries Get. + */ +public final class SummariesGetSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_Get_MaximumSet_Gen.json + */ + /** + * Sample code: Summaries_Get_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + summariesGetMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.summaries().getWithResponse("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + SummaryName.FIRMWARE, com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_Get_MinimumSet_Gen.json + */ + /** + * Sample code: Summaries_Get_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void + summariesGetMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.summaries().getWithResponse("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + SummaryName.FIRMWARE, com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareSamples.java new file mode 100644 index 0000000000000..f69b4bd27c403 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +/** + * Samples for Summaries ListByFirmware. + */ +public final class SummariesListByFirmwareSamples { + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_ListByFirmware_MinimumSet_Gen.json + */ + /** + * Sample code: Summaries_ListByFirmware_MinimumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void summariesListByFirmwareMinimumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.summaries().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Summaries_ListByFirmware_MaximumSet_Gen.json + */ + /** + * Sample code: Summaries_ListByFirmware_MaximumSet_Gen. + * + * @param manager Entry point to IoTFirmwareDefenseManager. + */ + public static void summariesListByFirmwareMaximumSetGen( + com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.summaries().listByFirmware("FirmwareAnalysisRG", "default", "109a9886-50bf-85a8-9d75-000000000000", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateSamples.java index 92ce135e5cfe4..8296dbf56b92f 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateSamples.java @@ -7,45 +7,41 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Workspaces Create. */ +/** + * Samples for Workspaces Create. + */ public final class WorkspacesCreateSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Create_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Create_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Create_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesCreateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .define("E___-3") - .withRegion("jjwbseilitjgdrhbvvkwviqj") - .withExistingResourceGroup("rgworkspaces") - .create(); + public static void + workspacesCreateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().define("E___-3").withRegion("jjwbseilitjgdrhbvvkwviqj") + .withExistingResourceGroup("rgworkspaces").create(); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Create_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Create_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Create_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesCreateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .define("E___-3") - .withRegion("jjwbseilitjgdrhbvvkwviqj") - .withExistingResourceGroup("rgworkspaces") - .withTags(mapOf("key450", "fakeTokenPlaceholder")) - .create(); + public static void + workspacesCreateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().define("E___-3").withRegion("jjwbseilitjgdrhbvvkwviqj") + .withExistingResourceGroup("rgworkspaces").withTags(mapOf("key450", "fakeTokenPlaceholder")).create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteSamples.java index 0102ed79b2bee..a72c6353a4aa0 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteSamples.java @@ -4,35 +4,37 @@ package com.azure.resourcemanager.iotfirmwaredefense.generated; -/** Samples for Workspaces Delete. */ +/** + * Samples for Workspaces Delete. + */ public final class WorkspacesDeleteSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Delete_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Delete_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Delete_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesDeleteMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE); + public static void + workspacesDeleteMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Delete_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Delete_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Delete_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesDeleteMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE); + public static void + workspacesDeleteMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + manager.workspaces().deleteByResourceGroupWithResponse("rgworkspaces", "E___-3", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlSamples.java index b516728f10144..41b565f14d2b6 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlSamples.java @@ -6,40 +6,37 @@ import com.azure.resourcemanager.iotfirmwaredefense.models.GenerateUploadUrlRequest; -/** Samples for Workspaces GenerateUploadUrl. */ +/** + * Samples for Workspaces GenerateUploadUrl. + */ public final class WorkspacesGenerateUploadUrlSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_GenerateUploadUrl_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_GenerateUploadUrl_MinimumSet_Gen.json */ /** * Sample code: Workspaces_GenerateUploadUrl_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesGenerateUploadUrlMinimumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .generateUploadUrlWithResponse( - "rgworkspaces", "E___-3", new GenerateUploadUrlRequest(), com.azure.core.util.Context.NONE); + manager.workspaces().generateUploadUrlWithResponse("rgworkspaces", "E___-3", new GenerateUploadUrlRequest(), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_GenerateUploadUrl_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_GenerateUploadUrl_MaximumSet_Gen.json */ /** * Sample code: Workspaces_GenerateUploadUrl_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesGenerateUploadUrlMaximumSetGen( com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - manager - .workspaces() - .generateUploadUrlWithResponse( - "rgworkspaces", - "E___-3", - new GenerateUploadUrlRequest().withFirmwareId("ytsfprbywi"), - com.azure.core.util.Context.NONE); + manager.workspaces().generateUploadUrlWithResponse("rgworkspaces", "E___-3", + new GenerateUploadUrlRequest().withFirmwareId("ytsfprbywi"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupSamples.java index 304c6c1076f0d..e76c0567e2aca 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupSamples.java @@ -4,31 +4,35 @@ package com.azure.resourcemanager.iotfirmwaredefense.generated; -/** Samples for Workspaces GetByResourceGroup. */ +/** + * Samples for Workspaces GetByResourceGroup. + */ public final class WorkspacesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Get_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Get_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesGetMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + public static void + workspacesGetMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.workspaces().getByResourceGroupWithResponse("rgworkspaces", "E_US", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Get_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Get_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Get_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesGetMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + public static void + workspacesGetMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { manager.workspaces().getByResourceGroupWithResponse("rgworkspaces", "E_US", com.azure.core.util.Context.NONE); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupSamples.java index e249b83391d18..8f0e62efc31b5 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.iotfirmwaredefense.generated; -/** Samples for Workspaces ListByResourceGroup. */ +/** + * Samples for Workspaces ListByResourceGroup. + */ public final class WorkspacesListByResourceGroupSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListByResourceGroup_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListByResourceGroup_MaximumSet_Gen.json */ /** * Sample code: Workspaces_ListByResourceGroup_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListByResourceGroupMaximumSetGen( @@ -20,11 +23,12 @@ public static void workspacesListByResourceGroupMaximumSetGen( } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListByResourceGroup_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListByResourceGroup_MinimumSet_Gen.json */ /** * Sample code: Workspaces_ListByResourceGroup_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListByResourceGroupMinimumSetGen( diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListSamples.java index ebb6535a46ca8..afef1db42652d 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.iotfirmwaredefense.generated; -/** Samples for Workspaces List. */ +/** + * Samples for Workspaces List. + */ public final class WorkspacesListSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListBySubscription_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListBySubscription_MaximumSet_Gen.json */ /** * Sample code: Workspaces_ListBySubscription_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListBySubscriptionMaximumSetGen( @@ -20,11 +23,12 @@ public static void workspacesListBySubscriptionMaximumSetGen( } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_ListBySubscription_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_ListBySubscription_MinimumSet_Gen.json */ /** * Sample code: Workspaces_ListBySubscription_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ public static void workspacesListBySubscriptionMinimumSetGen( diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesUpdateSamples.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesUpdateSamples.java index e7ae262821b8f..bb5e8c8e5d868 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesUpdateSamples.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/samples/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesUpdateSamples.java @@ -6,41 +6,39 @@ import com.azure.resourcemanager.iotfirmwaredefense.models.Workspace; -/** Samples for Workspaces Update. */ +/** + * Samples for Workspaces Update. + */ public final class WorkspacesUpdateSamples { /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Update_MaximumSet_Gen.json */ /** * Sample code: Workspaces_Update_MaximumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesUpdateMaximumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - Workspace resource = - manager - .workspaces() - .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE) - .getValue(); + public static void + workspacesUpdateMaximumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Workspace resource = manager.workspaces() + .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } /* - * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/preview/2023-02-08-preview/examples/Workspaces_Update_MinimumSet_Gen.json + * x-ms-original-file: specification/fist/resource-manager/Microsoft.IoTFirmwareDefense/stable/2024-01-10/examples/ + * Workspaces_Update_MinimumSet_Gen.json */ /** * Sample code: Workspaces_Update_MinimumSet_Gen. - * + * * @param manager Entry point to IoTFirmwareDefenseManager. */ - public static void workspacesUpdateMinimumSetGen( - com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { - Workspace resource = - manager - .workspaces() - .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE) - .getValue(); + public static void + workspacesUpdateMinimumSetGen(com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager manager) { + Workspace resource = manager.workspaces() + .getByResourceGroupWithResponse("rgworkspaces", "E___-3", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningFeaturesTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningFeaturesTests.java index e0dc62975d6c4..99c2006ddf1ba 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningFeaturesTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningFeaturesTests.java @@ -6,42 +6,30 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningFeatures; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; import org.junit.jupiter.api.Assertions; public final class BinaryHardeningFeaturesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - BinaryHardeningFeatures model = - BinaryData - .fromString( - "{\"nx\":\"True\",\"pie\":\"True\",\"relro\":\"True\",\"canary\":\"True\",\"stripped\":\"True\"}") + BinaryHardeningFeatures model + = BinaryData.fromString("{\"nx\":false,\"pie\":true,\"relro\":false,\"canary\":true,\"stripped\":false}") .toObject(BinaryHardeningFeatures.class); - Assertions.assertEquals(NxFlag.TRUE, model.nx()); - Assertions.assertEquals(PieFlag.TRUE, model.pie()); - Assertions.assertEquals(RelroFlag.TRUE, model.relro()); - Assertions.assertEquals(CanaryFlag.TRUE, model.canary()); - Assertions.assertEquals(StrippedFlag.TRUE, model.stripped()); + Assertions.assertEquals(false, model.nx()); + Assertions.assertEquals(true, model.pie()); + Assertions.assertEquals(false, model.relro()); + Assertions.assertEquals(true, model.canary()); + Assertions.assertEquals(false, model.stripped()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BinaryHardeningFeatures model = - new BinaryHardeningFeatures() - .withNx(NxFlag.TRUE) - .withPie(PieFlag.TRUE) - .withRelro(RelroFlag.TRUE) - .withCanary(CanaryFlag.TRUE) - .withStripped(StrippedFlag.TRUE); + BinaryHardeningFeatures model = new BinaryHardeningFeatures().withNx(false).withPie(true).withRelro(false) + .withCanary(true).withStripped(false); model = BinaryData.fromObject(model).toObject(BinaryHardeningFeatures.class); - Assertions.assertEquals(NxFlag.TRUE, model.nx()); - Assertions.assertEquals(PieFlag.TRUE, model.pie()); - Assertions.assertEquals(RelroFlag.TRUE, model.relro()); - Assertions.assertEquals(CanaryFlag.TRUE, model.canary()); - Assertions.assertEquals(StrippedFlag.TRUE, model.stripped()); + Assertions.assertEquals(false, model.nx()); + Assertions.assertEquals(true, model.pie()); + Assertions.assertEquals(false, model.relro()); + Assertions.assertEquals(true, model.canary()); + Assertions.assertEquals(false, model.stripped()); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningInnerTests.java deleted file mode 100644 index 90879d02b8b11..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningInnerTests.java +++ /dev/null @@ -1,65 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; -import org.junit.jupiter.api.Assertions; - -public final class BinaryHardeningInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BinaryHardeningInner model = - BinaryData - .fromString( - "{\"binaryHardeningId\":\"qedqytbciqfoufl\",\"features\":{\"nx\":\"True\",\"pie\":\"False\",\"relro\":\"True\",\"canary\":\"False\",\"stripped\":\"True\"},\"architecture\":\"ugpbkw\",\"path\":\"utduqktapspwgcu\",\"class\":\"tumkdosvqwhbm\",\"runpath\":\"bbjfddgmbmbexp\",\"rpath\":\"htqqrolfp\"}") - .toObject(BinaryHardeningInner.class); - Assertions.assertEquals("qedqytbciqfoufl", model.binaryHardeningId()); - Assertions.assertEquals("ugpbkw", model.architecture()); - Assertions.assertEquals("utduqktapspwgcu", model.path()); - Assertions.assertEquals("tumkdosvqwhbm", model.classProperty()); - Assertions.assertEquals("bbjfddgmbmbexp", model.runpath()); - Assertions.assertEquals("htqqrolfp", model.rpath()); - Assertions.assertEquals(NxFlag.TRUE, model.nx()); - Assertions.assertEquals(PieFlag.FALSE, model.pie()); - Assertions.assertEquals(RelroFlag.TRUE, model.relro()); - Assertions.assertEquals(CanaryFlag.FALSE, model.canary()); - Assertions.assertEquals(StrippedFlag.TRUE, model.stripped()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BinaryHardeningInner model = - new BinaryHardeningInner() - .withBinaryHardeningId("qedqytbciqfoufl") - .withArchitecture("ugpbkw") - .withPath("utduqktapspwgcu") - .withClassProperty("tumkdosvqwhbm") - .withRunpath("bbjfddgmbmbexp") - .withRpath("htqqrolfp") - .withNx(NxFlag.TRUE) - .withPie(PieFlag.FALSE) - .withRelro(RelroFlag.TRUE) - .withCanary(CanaryFlag.FALSE) - .withStripped(StrippedFlag.TRUE); - model = BinaryData.fromObject(model).toObject(BinaryHardeningInner.class); - Assertions.assertEquals("qedqytbciqfoufl", model.binaryHardeningId()); - Assertions.assertEquals("ugpbkw", model.architecture()); - Assertions.assertEquals("utduqktapspwgcu", model.path()); - Assertions.assertEquals("tumkdosvqwhbm", model.classProperty()); - Assertions.assertEquals("bbjfddgmbmbexp", model.runpath()); - Assertions.assertEquals("htqqrolfp", model.rpath()); - Assertions.assertEquals(NxFlag.TRUE, model.nx()); - Assertions.assertEquals(PieFlag.FALSE, model.pie()); - Assertions.assertEquals(RelroFlag.TRUE, model.relro()); - Assertions.assertEquals(CanaryFlag.FALSE, model.canary()); - Assertions.assertEquals(StrippedFlag.TRUE, model.stripped()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListResultTests.java new file mode 100644 index 0000000000000..d4dec7dc4b561 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListResultTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningListResult; +import org.junit.jupiter.api.Assertions; + +public final class BinaryHardeningListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BinaryHardeningListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"binaryHardeningId\":\"uv\",\"features\":{\"nx\":true,\"pie\":false,\"relro\":false,\"canary\":false,\"stripped\":false},\"architecture\":\"zopbsphrupidgs\",\"filePath\":\"bejhphoycmsxa\",\"class\":\"hdxbmtqio\",\"runpath\":\"zehtbmu\",\"rpath\":\"ownoizhw\"},\"id\":\"xybqsoqij\",\"name\":\"k\",\"type\":\"mbpazlobcufpdzn\"},{\"properties\":{\"binaryHardeningId\":\"cqqjnqglhqgn\",\"features\":{\"nx\":false,\"pie\":false,\"relro\":false,\"canary\":true,\"stripped\":false},\"architecture\":\"saagdf\",\"filePath\":\"lzl\",\"class\":\"xrifkwmrvkts\",\"runpath\":\"nt\",\"rpath\":\"ipa\"},\"id\":\"ajpsquc\",\"name\":\"poyfdkfogkn\",\"type\":\"gjofjd\"},{\"properties\":{\"binaryHardeningId\":\"s\",\"features\":{\"nx\":true,\"pie\":true,\"relro\":true,\"canary\":true,\"stripped\":false},\"architecture\":\"jzyflu\",\"filePath\":\"rh\",\"class\":\"fcqhsmyurkd\",\"runpath\":\"lx\",\"rpath\":\"kuksjtxukcdm\"},\"id\":\"rcryuanzwuxzdxta\",\"name\":\"rlhm\",\"type\":\"hfpmrqobmtukknr\"}],\"nextLink\":\"tihfx\"}") + .toObject(BinaryHardeningListResult.class); + Assertions.assertEquals("tihfx", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BinaryHardeningListResult model = new BinaryHardeningListResult().withNextLink("tihfx"); + model = BinaryData.fromObject(model).toObject(BinaryHardeningListResult.class); + Assertions.assertEquals("tihfx", model.nextLink()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListTests.java deleted file mode 100644 index ea1f6e846123d..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningListTests.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningList; -import org.junit.jupiter.api.Assertions; - -public final class BinaryHardeningListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BinaryHardeningList model = - BinaryData - .fromString( - "{\"value\":[{\"binaryHardeningId\":\"gsntnbybkzgcwr\",\"features\":{\"nx\":\"True\",\"pie\":\"False\",\"relro\":\"True\",\"canary\":\"False\",\"stripped\":\"False\"},\"architecture\":\"kcqvkocrc\",\"path\":\"kwt\",\"class\":\"xbnjbiksq\",\"runpath\":\"lssai\",\"rpath\":\"p\"},{\"binaryHardeningId\":\"nzl\",\"features\":{\"nx\":\"False\",\"pie\":\"True\",\"relro\":\"False\",\"canary\":\"True\",\"stripped\":\"True\"},\"architecture\":\"sabkyqduujitcjcz\",\"path\":\"evndh\",\"class\":\"wpdappdsbdkv\",\"runpath\":\"wjfeusnhutjel\",\"rpath\":\"rl\"},{\"binaryHardeningId\":\"ugjzzdatqxhocdge\",\"features\":{\"nx\":\"False\",\"pie\":\"False\",\"relro\":\"True\",\"canary\":\"False\",\"stripped\":\"True\"},\"architecture\":\"kao\",\"path\":\"yiftyhxhuro\",\"class\":\"tyxolniwpwc\",\"runpath\":\"jfkgiawxk\",\"rpath\":\"ypl\"},{\"binaryHardeningId\":\"kbasyypn\",\"features\":{\"nx\":\"True\",\"pie\":\"False\",\"relro\":\"False\",\"canary\":\"False\",\"stripped\":\"True\"},\"architecture\":\"koty\",\"path\":\"gou\",\"class\":\"ndlik\",\"runpath\":\"qkgfgibma\",\"rpath\":\"akeqs\"}],\"nextLink\":\"yb\"}") - .toObject(BinaryHardeningList.class); - Assertions.assertEquals("yb", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BinaryHardeningList model = new BinaryHardeningList().withNextLink("yb"); - model = BinaryData.fromObject(model).toObject(BinaryHardeningList.class); - Assertions.assertEquals("yb", model.nextLink()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResourceInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResourceInnerTests.java new file mode 100644 index 0000000000000..65195374f2a00 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResourceInnerTests.java @@ -0,0 +1,48 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResourceInner; +import org.junit.jupiter.api.Assertions; + +public final class BinaryHardeningResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BinaryHardeningResourceInner model = BinaryData.fromString( + "{\"properties\":{\"binaryHardeningId\":\"bpzvgn\",\"features\":{\"nx\":true,\"pie\":false,\"relro\":true,\"canary\":true,\"stripped\":true},\"architecture\":\"kohdbiha\",\"filePath\":\"fhfcb\",\"class\":\"s\",\"runpath\":\"ithxqhabifpi\",\"rpath\":\"wczbys\"},\"id\":\"pqxu\",\"name\":\"ivyqniwbybrkxvd\",\"type\":\"mjgr\"}") + .toObject(BinaryHardeningResourceInner.class); + Assertions.assertEquals("bpzvgn", model.binaryHardeningId()); + Assertions.assertEquals("kohdbiha", model.architecture()); + Assertions.assertEquals("fhfcb", model.filePath()); + Assertions.assertEquals("s", model.classProperty()); + Assertions.assertEquals("ithxqhabifpi", model.runpath()); + Assertions.assertEquals("wczbys", model.rpath()); + Assertions.assertEquals(true, model.nx()); + Assertions.assertEquals(false, model.pie()); + Assertions.assertEquals(true, model.relro()); + Assertions.assertEquals(true, model.canary()); + Assertions.assertEquals(true, model.stripped()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BinaryHardeningResourceInner model = new BinaryHardeningResourceInner().withBinaryHardeningId("bpzvgn") + .withArchitecture("kohdbiha").withFilePath("fhfcb").withClassProperty("s").withRunpath("ithxqhabifpi") + .withRpath("wczbys").withNx(true).withPie(false).withRelro(true).withCanary(true).withStripped(true); + model = BinaryData.fromObject(model).toObject(BinaryHardeningResourceInner.class); + Assertions.assertEquals("bpzvgn", model.binaryHardeningId()); + Assertions.assertEquals("kohdbiha", model.architecture()); + Assertions.assertEquals("fhfcb", model.filePath()); + Assertions.assertEquals("s", model.classProperty()); + Assertions.assertEquals("ithxqhabifpi", model.runpath()); + Assertions.assertEquals("wczbys", model.rpath()); + Assertions.assertEquals(true, model.nx()); + Assertions.assertEquals(false, model.pie()); + Assertions.assertEquals(true, model.relro()); + Assertions.assertEquals(true, model.canary()); + Assertions.assertEquals(true, model.stripped()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResultTests.java new file mode 100644 index 0000000000000..ed31cd0668006 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningResultTests.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningResult; +import org.junit.jupiter.api.Assertions; + +public final class BinaryHardeningResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BinaryHardeningResult model = BinaryData.fromString( + "{\"binaryHardeningId\":\"wvukx\",\"features\":{\"nx\":false,\"pie\":false,\"relro\":true,\"canary\":false,\"stripped\":true},\"architecture\":\"yejhk\",\"filePath\":\"htnapczwlokjyem\",\"class\":\"vnipjox\",\"runpath\":\"nchgej\",\"rpath\":\"odmailzyd\"}") + .toObject(BinaryHardeningResult.class); + Assertions.assertEquals("wvukx", model.binaryHardeningId()); + Assertions.assertEquals("yejhk", model.architecture()); + Assertions.assertEquals("htnapczwlokjyem", model.filePath()); + Assertions.assertEquals("vnipjox", model.classProperty()); + Assertions.assertEquals("nchgej", model.runpath()); + Assertions.assertEquals("odmailzyd", model.rpath()); + Assertions.assertEquals(false, model.nx()); + Assertions.assertEquals(false, model.pie()); + Assertions.assertEquals(true, model.relro()); + Assertions.assertEquals(false, model.canary()); + Assertions.assertEquals(true, model.stripped()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BinaryHardeningResult model = new BinaryHardeningResult().withBinaryHardeningId("wvukx") + .withArchitecture("yejhk").withFilePath("htnapczwlokjyem").withClassProperty("vnipjox") + .withRunpath("nchgej").withRpath("odmailzyd").withNx(false).withPie(false).withRelro(true).withCanary(false) + .withStripped(true); + model = BinaryData.fromObject(model).toObject(BinaryHardeningResult.class); + Assertions.assertEquals("wvukx", model.binaryHardeningId()); + Assertions.assertEquals("yejhk", model.architecture()); + Assertions.assertEquals("htnapczwlokjyem", model.filePath()); + Assertions.assertEquals("vnipjox", model.classProperty()); + Assertions.assertEquals("nchgej", model.runpath()); + Assertions.assertEquals("odmailzyd", model.rpath()); + Assertions.assertEquals(false, model.nx()); + Assertions.assertEquals(false, model.pie()); + Assertions.assertEquals(true, model.relro()); + Assertions.assertEquals(false, model.canary()); + Assertions.assertEquals(true, model.stripped()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryInnerTests.java deleted file mode 100644 index e6d1314b00b38..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryInnerTests.java +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.BinaryHardeningSummaryInner; -import org.junit.jupiter.api.Assertions; - -public final class BinaryHardeningSummaryInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BinaryHardeningSummaryInner model = - BinaryData - .fromString( - "{\"totalFiles\":2563543164314318183,\"nx\":1739804762,\"pie\":161079646,\"relro\":243879376,\"canary\":2040036621,\"stripped\":1987698558}") - .toObject(BinaryHardeningSummaryInner.class); - Assertions.assertEquals(2563543164314318183L, model.totalFiles()); - Assertions.assertEquals(1739804762, model.nx()); - Assertions.assertEquals(161079646, model.pie()); - Assertions.assertEquals(243879376, model.relro()); - Assertions.assertEquals(2040036621, model.canary()); - Assertions.assertEquals(1987698558, model.stripped()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BinaryHardeningSummaryInner model = - new BinaryHardeningSummaryInner() - .withTotalFiles(2563543164314318183L) - .withNx(1739804762) - .withPie(161079646) - .withRelro(243879376) - .withCanary(2040036621) - .withStripped(1987698558); - model = BinaryData.fromObject(model).toObject(BinaryHardeningSummaryInner.class); - Assertions.assertEquals(2563543164314318183L, model.totalFiles()); - Assertions.assertEquals(1739804762, model.nx()); - Assertions.assertEquals(161079646, model.pie()); - Assertions.assertEquals(243879376, model.relro()); - Assertions.assertEquals(2040036621, model.canary()); - Assertions.assertEquals(1987698558, model.stripped()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryResourceTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryResourceTests.java new file mode 100644 index 0000000000000..7c91a0f7e738b --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningSummaryResourceTests.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningSummaryResource; +import org.junit.jupiter.api.Assertions; + +public final class BinaryHardeningSummaryResourceTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BinaryHardeningSummaryResource model = BinaryData.fromString( + "{\"summaryType\":\"BinaryHardening\",\"totalFiles\":1588366635739120038,\"nx\":1840512089,\"pie\":903805755,\"relro\":1405180794,\"canary\":142305050,\"stripped\":1098465372}") + .toObject(BinaryHardeningSummaryResource.class); + Assertions.assertEquals(1588366635739120038L, model.totalFiles()); + Assertions.assertEquals(1840512089, model.nx()); + Assertions.assertEquals(903805755, model.pie()); + Assertions.assertEquals(1405180794, model.relro()); + Assertions.assertEquals(142305050, model.canary()); + Assertions.assertEquals(1098465372, model.stripped()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BinaryHardeningSummaryResource model = new BinaryHardeningSummaryResource().withTotalFiles(1588366635739120038L) + .withNx(1840512089).withPie(903805755).withRelro(1405180794).withCanary(142305050).withStripped(1098465372); + model = BinaryData.fromObject(model).toObject(BinaryHardeningSummaryResource.class); + Assertions.assertEquals(1588366635739120038L, model.totalFiles()); + Assertions.assertEquals(1840512089, model.nx()); + Assertions.assertEquals(903805755, model.pie()); + Assertions.assertEquals(1405180794, model.relro()); + Assertions.assertEquals(142305050, model.canary()); + Assertions.assertEquals(1098465372, model.stripped()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningsListByFirmwareMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningsListByFirmwareMockTests.java new file mode 100644 index 0000000000000..46a9b135218d8 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/BinaryHardeningsListByFirmwareMockTests.java @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; +import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningResource; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class BinaryHardeningsListByFirmwareMockTests { + @Test + public void testListByFirmware() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"binaryHardeningId\":\"rzqlveu\",\"features\":{\"nx\":true,\"pie\":false,\"relro\":true,\"canary\":true,\"stripped\":true},\"architecture\":\"cswsrtjri\",\"filePath\":\"rbpbewtghfgblcg\",\"class\":\"zvlvqhjkbegib\",\"runpath\":\"mxiebw\",\"rpath\":\"loayqcgw\"},\"id\":\"zjuzgwyz\",\"name\":\"htxongmtsavjc\",\"type\":\"pwxqp\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.binaryHardenings().listByFirmware("auwhvylwzbtdhx", + "jznb", "pow", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("rzqlveu", response.iterator().next().binaryHardeningId()); + Assertions.assertEquals("cswsrtjri", response.iterator().next().architecture()); + Assertions.assertEquals("rbpbewtghfgblcg", response.iterator().next().filePath()); + Assertions.assertEquals("zvlvqhjkbegib", response.iterator().next().classProperty()); + Assertions.assertEquals("mxiebw", response.iterator().next().runpath()); + Assertions.assertEquals("loayqcgw", response.iterator().next().rpath()); + Assertions.assertEquals(true, response.iterator().next().nx()); + Assertions.assertEquals(false, response.iterator().next().pie()); + Assertions.assertEquals(true, response.iterator().next().relro()); + Assertions.assertEquals(true, response.iterator().next().canary()); + Assertions.assertEquals(true, response.iterator().next().stripped()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentInnerTests.java deleted file mode 100644 index d2429ca415aaa..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentInnerTests.java +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.ComponentInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsUpdateAvailable; -import java.time.OffsetDateTime; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class ComponentInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - ComponentInner model = - BinaryData - .fromString( - "{\"componentId\":\"iheogna\",\"componentName\":\"zxtheotusivyevcc\",\"version\":\"ihnhun\",\"license\":\"wjzrnfygxgisp\",\"releaseDate\":\"2021-05-03T11:09:18Z\",\"paths\":[\"fkufublj\",\"fxqeof\",\"aeqjhqjbasvms\"],\"isUpdateAvailable\":\"False\"}") - .toObject(ComponentInner.class); - Assertions.assertEquals("iheogna", model.componentId()); - Assertions.assertEquals("zxtheotusivyevcc", model.componentName()); - Assertions.assertEquals("ihnhun", model.version()); - Assertions.assertEquals("wjzrnfygxgisp", model.license()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-03T11:09:18Z"), model.releaseDate()); - Assertions.assertEquals("fkufublj", model.paths().get(0)); - Assertions.assertEquals(IsUpdateAvailable.FALSE, model.isUpdateAvailable()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - ComponentInner model = - new ComponentInner() - .withComponentId("iheogna") - .withComponentName("zxtheotusivyevcc") - .withVersion("ihnhun") - .withLicense("wjzrnfygxgisp") - .withReleaseDate(OffsetDateTime.parse("2021-05-03T11:09:18Z")) - .withPaths(Arrays.asList("fkufublj", "fxqeof", "aeqjhqjbasvms")) - .withIsUpdateAvailable(IsUpdateAvailable.FALSE); - model = BinaryData.fromObject(model).toObject(ComponentInner.class); - Assertions.assertEquals("iheogna", model.componentId()); - Assertions.assertEquals("zxtheotusivyevcc", model.componentName()); - Assertions.assertEquals("ihnhun", model.version()); - Assertions.assertEquals("wjzrnfygxgisp", model.license()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-03T11:09:18Z"), model.releaseDate()); - Assertions.assertEquals("fkufublj", model.paths().get(0)); - Assertions.assertEquals(IsUpdateAvailable.FALSE, model.isUpdateAvailable()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentListTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentListTests.java deleted file mode 100644 index 6fbeca1c7cae9..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/ComponentListTests.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.models.ComponentList; -import org.junit.jupiter.api.Assertions; - -public final class ComponentListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - ComponentList model = - BinaryData - .fromString( - "{\"value\":[{\"componentId\":\"f\",\"componentName\":\"s\",\"version\":\"zgvfcjrwz\",\"license\":\"xjtfelluwfzit\",\"releaseDate\":\"2021-10-17T22:50:15Z\",\"paths\":[\"fpjkjlxofp\",\"vhpfxxypininmay\"],\"isUpdateAvailable\":\"True\"}],\"nextLink\":\"bkpodepooginuv\"}") - .toObject(ComponentList.class); - Assertions.assertEquals("bkpodepooginuv", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - ComponentList model = new ComponentList().withNextLink("bkpodepooginuv"); - model = BinaryData.fromObject(model).toObject(ComponentList.class); - Assertions.assertEquals("bkpodepooginuv", model.nextLink()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificateEntityTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificateEntityTests.java index f15795f868a17..2d860a6df3546 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificateEntityTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CryptoCertificateEntityTests.java @@ -11,32 +11,26 @@ public final class CryptoCertificateEntityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CryptoCertificateEntity model = - BinaryData - .fromString( - "{\"commonName\":\"daj\",\"organization\":\"ysou\",\"organizationalUnit\":\"e\",\"state\":\"noae\",\"country\":\"fhyhltrpmopjmcma\"}") - .toObject(CryptoCertificateEntity.class); - Assertions.assertEquals("daj", model.commonName()); - Assertions.assertEquals("ysou", model.organization()); - Assertions.assertEquals("e", model.organizationalUnit()); - Assertions.assertEquals("noae", model.state()); - Assertions.assertEquals("fhyhltrpmopjmcma", model.country()); + CryptoCertificateEntity model = BinaryData.fromString( + "{\"commonName\":\"xklrypl\",\"organization\":\"kbasyypn\",\"organizationalUnit\":\"hsgcbacphejkot\",\"state\":\"qgoulznd\",\"country\":\"kwy\"}") + .toObject(CryptoCertificateEntity.class); + Assertions.assertEquals("xklrypl", model.commonName()); + Assertions.assertEquals("kbasyypn", model.organization()); + Assertions.assertEquals("hsgcbacphejkot", model.organizationalUnit()); + Assertions.assertEquals("qgoulznd", model.state()); + Assertions.assertEquals("kwy", model.country()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CryptoCertificateEntity model = - new CryptoCertificateEntity() - .withCommonName("daj") - .withOrganization("ysou") - .withOrganizationalUnit("e") - .withState("noae") - .withCountry("fhyhltrpmopjmcma"); + CryptoCertificateEntity model + = new CryptoCertificateEntity().withCommonName("xklrypl").withOrganization("kbasyypn") + .withOrganizationalUnit("hsgcbacphejkot").withState("qgoulznd").withCountry("kwy"); model = BinaryData.fromObject(model).toObject(CryptoCertificateEntity.class); - Assertions.assertEquals("daj", model.commonName()); - Assertions.assertEquals("ysou", model.organization()); - Assertions.assertEquals("e", model.organizationalUnit()); - Assertions.assertEquals("noae", model.state()); - Assertions.assertEquals("fhyhltrpmopjmcma", model.country()); + Assertions.assertEquals("xklrypl", model.commonName()); + Assertions.assertEquals("kbasyypn", model.organization()); + Assertions.assertEquals("hsgcbacphejkot", model.organizationalUnit()); + Assertions.assertEquals("qgoulznd", model.state()); + Assertions.assertEquals("kwy", model.country()); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveComponentTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveComponentTests.java new file mode 100644 index 0000000000000..91bd3877b0180 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveComponentTests.java @@ -0,0 +1,31 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent; +import org.junit.jupiter.api.Assertions; + +public final class CveComponentTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CveComponent model + = BinaryData.fromString("{\"componentId\":\"zmyzydagf\",\"name\":\"xbezyiuokktwh\",\"version\":\"xw\"}") + .toObject(CveComponent.class); + Assertions.assertEquals("zmyzydagf", model.componentId()); + Assertions.assertEquals("xbezyiuokktwh", model.name()); + Assertions.assertEquals("xw", model.version()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CveComponent model + = new CveComponent().withComponentId("zmyzydagf").withName("xbezyiuokktwh").withVersion("xw"); + model = BinaryData.fromObject(model).toObject(CveComponent.class); + Assertions.assertEquals("zmyzydagf", model.componentId()); + Assertions.assertEquals("xbezyiuokktwh", model.name()); + Assertions.assertEquals("xw", model.version()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveInnerTests.java deleted file mode 100644 index 7bddcd88fb7b3..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveInnerTests.java +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveInner; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; - -public final class CveInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - CveInner model = - BinaryData - .fromString( - "{\"cveId\":\"rzayv\",\"component\":\"datapgvdf\",\"severity\":\"otkftutqxlngx\",\"name\":\"fgugnxkrxdqmid\",\"cvssScore\":\"hzrvqd\",\"cvssVersion\":\"bhj\",\"cvssV2Score\":\"igeho\",\"cvssV3Score\":\"bowsk\",\"publishDate\":\"2021-10-18T00:51:47Z\",\"updatedDate\":\"2020-12-21T07:17:40Z\",\"links\":[{\"href\":\"u\",\"label\":\"wgqyw\"},{\"href\":\"drvyn\",\"label\":\"gpphrcgyn\"},{\"href\":\"cpecfvmmcoofs\",\"label\":\"zevgb\"},{\"href\":\"jqabcypmivkwlzuv\",\"label\":\"fwnfnb\"}],\"description\":\"fionl\"}") - .toObject(CveInner.class); - Assertions.assertEquals("rzayv", model.cveId()); - Assertions.assertEquals("otkftutqxlngx", model.severity()); - Assertions.assertEquals("fgugnxkrxdqmid", model.name()); - Assertions.assertEquals("hzrvqd", model.cvssScore()); - Assertions.assertEquals("bhj", model.cvssVersion()); - Assertions.assertEquals("igeho", model.cvssV2Score()); - Assertions.assertEquals("bowsk", model.cvssV3Score()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-18T00:51:47Z"), model.publishDate()); - Assertions.assertEquals(OffsetDateTime.parse("2020-12-21T07:17:40Z"), model.updatedDate()); - Assertions.assertEquals("fionl", model.description()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - CveInner model = - new CveInner() - .withCveId("rzayv") - .withComponent("datapgvdf") - .withSeverity("otkftutqxlngx") - .withName("fgugnxkrxdqmid") - .withCvssScore("hzrvqd") - .withCvssVersion("bhj") - .withCvssV2Score("igeho") - .withCvssV3Score("bowsk") - .withPublishDate(OffsetDateTime.parse("2021-10-18T00:51:47Z")) - .withUpdatedDate(OffsetDateTime.parse("2020-12-21T07:17:40Z")) - .withDescription("fionl"); - model = BinaryData.fromObject(model).toObject(CveInner.class); - Assertions.assertEquals("rzayv", model.cveId()); - Assertions.assertEquals("otkftutqxlngx", model.severity()); - Assertions.assertEquals("fgugnxkrxdqmid", model.name()); - Assertions.assertEquals("hzrvqd", model.cvssScore()); - Assertions.assertEquals("bhj", model.cvssVersion()); - Assertions.assertEquals("igeho", model.cvssV2Score()); - Assertions.assertEquals("bowsk", model.cvssV3Score()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-18T00:51:47Z"), model.publishDate()); - Assertions.assertEquals(OffsetDateTime.parse("2020-12-21T07:17:40Z"), model.updatedDate()); - Assertions.assertEquals("fionl", model.description()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveLinkTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveLinkTests.java index f7a7bbd100099..8e763e0b579d3 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveLinkTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveLinkTests.java @@ -11,16 +11,17 @@ public final class CveLinkTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CveLink model = BinaryData.fromString("{\"href\":\"x\",\"label\":\"qgtz\"}").toObject(CveLink.class); - Assertions.assertEquals("x", model.href()); - Assertions.assertEquals("qgtz", model.label()); + CveLink model + = BinaryData.fromString("{\"href\":\"wqsmbsur\",\"label\":\"imoryocfsfksym\"}").toObject(CveLink.class); + Assertions.assertEquals("wqsmbsur", model.href()); + Assertions.assertEquals("imoryocfsfksym", model.label()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CveLink model = new CveLink().withHref("x").withLabel("qgtz"); + CveLink model = new CveLink().withHref("wqsmbsur").withLabel("imoryocfsfksym"); model = BinaryData.fromObject(model).toObject(CveLink.class); - Assertions.assertEquals("x", model.href()); - Assertions.assertEquals("qgtz", model.label()); + Assertions.assertEquals("wqsmbsur", model.href()); + Assertions.assertEquals("imoryocfsfksym", model.label()); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListResultTests.java new file mode 100644 index 0000000000000..e20fba94be910 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListResultTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveListResult; +import org.junit.jupiter.api.Assertions; + +public final class CveListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CveListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"cveId\":\"qktapspwgcuert\",\"component\":{\"componentId\":\"dosvqwhbmdgbbjf\",\"name\":\"gmbmbexppbh\",\"version\":\"qrolfpf\"},\"severity\":\"algbquxigjyjg\",\"name\":\"aoyfhrtxilnerkuj\",\"cvssScore\":\"vlejuvfqa\",\"cvssVersion\":\"lyxwjkcprbnwbx\",\"cvssV2Score\":\"vtb\",\"cvssV3Score\":\"ysszdnrujqguh\",\"links\":[{\"href\":\"qfprwzwbn\",\"label\":\"itnwuizgazxufi\"},{\"href\":\"ckyfih\",\"label\":\"idf\"}],\"description\":\"wdzuhtymwisd\"},\"id\":\"thwxmnteiwaopvkm\",\"name\":\"jcmmxdcufufsrp\",\"type\":\"mzidnsezcxtb\"},{\"properties\":{\"cveId\":\"fycc\",\"component\":{\"componentId\":\"wmdwzjeiachboo\",\"name\":\"lnrosfqp\",\"version\":\"ehzzvypyqrim\"},\"severity\":\"npvswjdkirso\",\"name\":\"qxhcrmn\",\"cvssScore\":\"jtckwhdso\",\"cvssVersion\":\"iy\",\"cvssV2Score\":\"jxsqwpgrjbz\",\"cvssV3Score\":\"rcjxvsnbyxqabn\",\"links\":[{\"href\":\"cyshurzafbljjgp\",\"label\":\"oq\"}],\"description\":\"mkljavb\"},\"id\":\"dtqajzyulpkudj\",\"name\":\"rlkhbzhfepgzgq\",\"type\":\"xzlocxscp\"},{\"properties\":{\"cveId\":\"rhhbcs\",\"component\":{\"componentId\":\"mmajtjaodx\",\"name\":\"nbdxk\",\"version\":\"xo\"},\"severity\":\"jionpimexgstxgc\",\"name\":\"dg\",\"cvssScore\":\"ajrmvdjwzrlovmc\",\"cvssVersion\":\"hijco\",\"cvssV2Score\":\"ctbzaq\",\"cvssV3Score\":\"sycbkbfk\",\"links\":[{\"href\":\"kexxppof\",\"label\":\"axcfjpgddtocjjx\"},{\"href\":\"pmouexhdz\",\"label\":\"bqe\"}],\"description\":\"nxqbzvddn\"},\"id\":\"ndei\",\"name\":\"btwnpzaoqvuhrhcf\",\"type\":\"cyddglmjthjqk\"},{\"properties\":{\"cveId\":\"eicxmqciwqvhkhi\",\"component\":{\"componentId\":\"gdtopbobjogh\",\"name\":\"w\",\"version\":\"m\"},\"severity\":\"hrzayvvtpgvdf\",\"name\":\"otkftutqxlngx\",\"cvssScore\":\"fgugnxkrxdqmid\",\"cvssVersion\":\"hzrvqd\",\"cvssV2Score\":\"bhj\",\"cvssV3Score\":\"igeho\",\"links\":[{\"href\":\"wska\",\"label\":\"ktzlcuiywg\"},{\"href\":\"wgndrvynhzgpp\",\"label\":\"cgyncocpecf\"}],\"description\":\"mcoo\"},\"id\":\"xlzevgbmqjqabcy\",\"name\":\"mivkwlzuvcc\",\"type\":\"wnfnbacf\"}],\"nextLink\":\"nlebxetqgtzxd\"}") + .toObject(CveListResult.class); + Assertions.assertEquals("nlebxetqgtzxd", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CveListResult model = new CveListResult().withNextLink("nlebxetqgtzxd"); + model = BinaryData.fromObject(model).toObject(CveListResult.class); + Assertions.assertEquals("nlebxetqgtzxd", model.nextLink()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListTests.java deleted file mode 100644 index 1b946e963e266..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveListTests.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.models.CveList; -import org.junit.jupiter.api.Assertions; - -public final class CveListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - CveList model = - BinaryData - .fromString( - "{\"value\":[{\"cveId\":\"bnwbxgjvtbvpyssz\",\"component\":\"datarujqg\",\"severity\":\"muouqfp\",\"name\":\"zw\",\"cvssScore\":\"g\",\"cvssVersion\":\"tnwu\",\"cvssV2Score\":\"gazxuf\",\"cvssV3Score\":\"uckyf\",\"publishDate\":\"2021-05-30T20:21:31Z\",\"updatedDate\":\"2021-03-02T12:22:29Z\",\"links\":[{\"href\":\"zwdzuh\",\"label\":\"mwisdkfthwxmnt\"},{\"href\":\"waopvkmijcmmxd\",\"label\":\"fufsrpymzi\"},{\"href\":\"sezcxtb\",\"label\":\"gfycc\"}],\"description\":\"ewmdw\"},{\"cveId\":\"eiachboosflnr\",\"component\":\"datafqpte\",\"severity\":\"zzvypyqrimzinp\",\"name\":\"wjdk\",\"cvssScore\":\"soodqxhcrmnoh\",\"cvssVersion\":\"ckwhds\",\"cvssV2Score\":\"fiyipjxsqwpgrj\",\"cvssV3Score\":\"norcjxvsnbyxqab\",\"publishDate\":\"2021-01-31T04:10:32Z\",\"updatedDate\":\"2021-11-28T11:46Z\",\"links\":[{\"href\":\"hurzafblj\",\"label\":\"pbtoqcjmkl\"}],\"description\":\"vbqid\"},{\"cveId\":\"ajzyul\",\"component\":\"datau\",\"severity\":\"krlkhbzhfepg\",\"name\":\"qex\",\"cvssScore\":\"ocxscpaierhhbcs\",\"cvssVersion\":\"ummajtjaod\",\"cvssV2Score\":\"bnbdxkqpxokajion\",\"cvssV3Score\":\"mexgstxgcp\",\"publishDate\":\"2021-04-23T14:47:55Z\",\"updatedDate\":\"2020-12-20T13:07:36Z\",\"links\":[{\"href\":\"mvdjwzrlovmc\",\"label\":\"hijco\"},{\"href\":\"ctbzaq\",\"label\":\"sycbkbfk\"},{\"href\":\"kdkexxp\",\"label\":\"fmxa\"}],\"description\":\"fjpgddtocjjxhvp\"},{\"cveId\":\"uexhdzx\",\"component\":\"dataqeojnxqbzvddntw\",\"severity\":\"eic\",\"name\":\"w\",\"cvssScore\":\"zao\",\"cvssVersion\":\"uhrhcffcyddgl\",\"cvssV2Score\":\"t\",\"cvssV3Score\":\"qkwpyeicxmqc\",\"publishDate\":\"2021-08-11T13:08:26Z\",\"updatedDate\":\"2021-01-16T07:23:11Z\",\"links\":[{\"href\":\"xuigdtopbobj\",\"label\":\"hm\"}],\"description\":\"u\"}],\"nextLink\":\"a\"}") - .toObject(CveList.class); - Assertions.assertEquals("a", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - CveList model = new CveList().withNextLink("a"); - model = BinaryData.fromObject(model).toObject(CveList.class); - Assertions.assertEquals("a", model.nextLink()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResourceInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResourceInnerTests.java new file mode 100644 index 0000000000000..957cbc143e830 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResourceInnerTests.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResourceInner; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent; +import org.junit.jupiter.api.Assertions; + +public final class CveResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CveResourceInner model = BinaryData.fromString( + "{\"properties\":{\"cveId\":\"bqqwxrj\",\"component\":{\"componentId\":\"llnwsubi\",\"name\":\"jampmngnzscxaqw\",\"version\":\"chcbonqvpkvlrxnj\"},\"severity\":\"seiphe\",\"name\":\"lokeyy\",\"cvssScore\":\"nj\",\"cvssVersion\":\"lwtgrhpdj\",\"cvssV2Score\":\"umasxazjpq\",\"cvssV3Score\":\"gual\",\"links\":[{\"href\":\"hejjz\",\"label\":\"dudgwdslfhot\"},{\"href\":\"cynpwlbjnp\",\"label\":\"cftadeh\"},{\"href\":\"ltyfsop\",\"label\":\"suesnzw\"}],\"description\":\"jbavorxzdm\"},\"id\":\"ctbqvudwx\",\"name\":\"ndnvo\",\"type\":\"gujjugwdkcglh\"}") + .toObject(CveResourceInner.class); + Assertions.assertEquals("bqqwxrj", model.cveId()); + Assertions.assertEquals("llnwsubi", model.component().componentId()); + Assertions.assertEquals("jampmngnzscxaqw", model.component().name()); + Assertions.assertEquals("chcbonqvpkvlrxnj", model.component().version()); + Assertions.assertEquals("seiphe", model.severity()); + Assertions.assertEquals("lokeyy", model.namePropertiesName()); + Assertions.assertEquals("nj", model.cvssScore()); + Assertions.assertEquals("lwtgrhpdj", model.cvssVersion()); + Assertions.assertEquals("umasxazjpq", model.cvssV2Score()); + Assertions.assertEquals("gual", model.cvssV3Score()); + Assertions.assertEquals("jbavorxzdm", model.description()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CveResourceInner model = new CveResourceInner().withCveId("bqqwxrj") + .withComponent(new CveComponent().withComponentId("llnwsubi").withName("jampmngnzscxaqw") + .withVersion("chcbonqvpkvlrxnj")) + .withSeverity("seiphe").withNamePropertiesName("lokeyy").withCvssScore("nj").withCvssVersion("lwtgrhpdj") + .withCvssV2Score("umasxazjpq").withCvssV3Score("gual").withDescription("jbavorxzdm"); + model = BinaryData.fromObject(model).toObject(CveResourceInner.class); + Assertions.assertEquals("bqqwxrj", model.cveId()); + Assertions.assertEquals("llnwsubi", model.component().componentId()); + Assertions.assertEquals("jampmngnzscxaqw", model.component().name()); + Assertions.assertEquals("chcbonqvpkvlrxnj", model.component().version()); + Assertions.assertEquals("seiphe", model.severity()); + Assertions.assertEquals("lokeyy", model.namePropertiesName()); + Assertions.assertEquals("nj", model.cvssScore()); + Assertions.assertEquals("lwtgrhpdj", model.cvssVersion()); + Assertions.assertEquals("umasxazjpq", model.cvssV2Score()); + Assertions.assertEquals("gual", model.cvssV3Score()); + Assertions.assertEquals("jbavorxzdm", model.description()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResultTests.java new file mode 100644 index 0000000000000..b305f68cf8172 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveResultTests.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveResult; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveComponent; +import org.junit.jupiter.api.Assertions; + +public final class CveResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CveResult model = BinaryData.fromString( + "{\"cveId\":\"azjdyggd\",\"component\":{\"componentId\":\"xhbkuofq\",\"name\":\"yk\",\"version\":\"enevfyexfwhybci\"},\"severity\":\"yvdcsitynnaa\",\"name\":\"ectehf\",\"cvssScore\":\"scjeypv\",\"cvssVersion\":\"zrkgqhcjrefovg\",\"cvssV2Score\":\"qsl\",\"cvssV3Score\":\"yvxyqjp\",\"links\":[{\"href\":\"tpngjcrcczsqpjh\",\"label\":\"daj\"},{\"href\":\"ysou\",\"label\":\"e\"},{\"href\":\"noae\",\"label\":\"fhyhltrpmopjmcma\"},{\"href\":\"okth\",\"label\":\"iuaod\"}],\"description\":\"cpkvxodp\"}") + .toObject(CveResult.class); + Assertions.assertEquals("azjdyggd", model.cveId()); + Assertions.assertEquals("xhbkuofq", model.component().componentId()); + Assertions.assertEquals("yk", model.component().name()); + Assertions.assertEquals("enevfyexfwhybci", model.component().version()); + Assertions.assertEquals("yvdcsitynnaa", model.severity()); + Assertions.assertEquals("ectehf", model.name()); + Assertions.assertEquals("scjeypv", model.cvssScore()); + Assertions.assertEquals("zrkgqhcjrefovg", model.cvssVersion()); + Assertions.assertEquals("qsl", model.cvssV2Score()); + Assertions.assertEquals("yvxyqjp", model.cvssV3Score()); + Assertions.assertEquals("cpkvxodp", model.description()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CveResult model = new CveResult().withCveId("azjdyggd") + .withComponent(new CveComponent().withComponentId("xhbkuofq").withName("yk").withVersion("enevfyexfwhybci")) + .withSeverity("yvdcsitynnaa").withName("ectehf").withCvssScore("scjeypv").withCvssVersion("zrkgqhcjrefovg") + .withCvssV2Score("qsl").withCvssV3Score("yvxyqjp").withDescription("cpkvxodp"); + model = BinaryData.fromObject(model).toObject(CveResult.class); + Assertions.assertEquals("azjdyggd", model.cveId()); + Assertions.assertEquals("xhbkuofq", model.component().componentId()); + Assertions.assertEquals("yk", model.component().name()); + Assertions.assertEquals("enevfyexfwhybci", model.component().version()); + Assertions.assertEquals("yvdcsitynnaa", model.severity()); + Assertions.assertEquals("ectehf", model.name()); + Assertions.assertEquals("scjeypv", model.cvssScore()); + Assertions.assertEquals("zrkgqhcjrefovg", model.cvssVersion()); + Assertions.assertEquals("qsl", model.cvssV2Score()); + Assertions.assertEquals("yvxyqjp", model.cvssV3Score()); + Assertions.assertEquals("cpkvxodp", model.description()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryInnerTests.java deleted file mode 100644 index aa285aa876f01..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryInnerTests.java +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.CveSummaryInner; -import org.junit.jupiter.api.Assertions; - -public final class CveSummaryInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - CveSummaryInner model = - BinaryData - .fromString( - "{\"critical\":4368619677721418751,\"high\":7743628540054408847,\"medium\":5911991287657256269,\"low\":2526678259667108543,\"unknown\":1458668924048164836,\"undefined\":383995801038376266}") - .toObject(CveSummaryInner.class); - Assertions.assertEquals(4368619677721418751L, model.critical()); - Assertions.assertEquals(7743628540054408847L, model.high()); - Assertions.assertEquals(5911991287657256269L, model.medium()); - Assertions.assertEquals(2526678259667108543L, model.low()); - Assertions.assertEquals(1458668924048164836L, model.unknown()); - Assertions.assertEquals(383995801038376266L, model.undefined()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - CveSummaryInner model = - new CveSummaryInner() - .withCritical(4368619677721418751L) - .withHigh(7743628540054408847L) - .withMedium(5911991287657256269L) - .withLow(2526678259667108543L) - .withUnknown(1458668924048164836L) - .withUndefined(383995801038376266L); - model = BinaryData.fromObject(model).toObject(CveSummaryInner.class); - Assertions.assertEquals(4368619677721418751L, model.critical()); - Assertions.assertEquals(7743628540054408847L, model.high()); - Assertions.assertEquals(5911991287657256269L, model.medium()); - Assertions.assertEquals(2526678259667108543L, model.low()); - Assertions.assertEquals(1458668924048164836L, model.unknown()); - Assertions.assertEquals(383995801038376266L, model.undefined()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryTests.java new file mode 100644 index 0000000000000..338aa5fe26582 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CveSummaryTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveSummary; +import org.junit.jupiter.api.Assertions; + +public final class CveSummaryTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CveSummary model = BinaryData.fromString( + "{\"summaryType\":\"CVE\",\"critical\":2851326765545975201,\"high\":3602555477170687906,\"medium\":7412584900785787675,\"low\":7022663417329031305,\"unknown\":6962157826226813076}") + .toObject(CveSummary.class); + Assertions.assertEquals(2851326765545975201L, model.critical()); + Assertions.assertEquals(3602555477170687906L, model.high()); + Assertions.assertEquals(7412584900785787675L, model.medium()); + Assertions.assertEquals(7022663417329031305L, model.low()); + Assertions.assertEquals(6962157826226813076L, model.unknown()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CveSummary model = new CveSummary().withCritical(2851326765545975201L).withHigh(3602555477170687906L) + .withMedium(7412584900785787675L).withLow(7022663417329031305L).withUnknown(6962157826226813076L); + model = BinaryData.fromObject(model).toObject(CveSummary.class); + Assertions.assertEquals(2851326765545975201L, model.critical()); + Assertions.assertEquals(3602555477170687906L, model.high()); + Assertions.assertEquals(7412584900785787675L, model.medium()); + Assertions.assertEquals(7022663417329031305L, model.low()); + Assertions.assertEquals(6962157826226813076L, model.unknown()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareMockTests.java new file mode 100644 index 0000000000000..c7eb32dba12fd --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/CvesListByFirmwareMockTests.java @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; +import com.azure.resourcemanager.iotfirmwaredefense.models.CveResource; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class CvesListByFirmwareMockTests { + @Test + public void testListByFirmware() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"cveId\":\"pnapnyiropuh\",\"component\":{\"componentId\":\"vpgylgqgitxmed\",\"name\":\"c\",\"version\":\"ynqwwncwzzhxgk\"},\"severity\":\"mgucna\",\"name\":\"t\",\"cvssScore\":\"ellwptfdy\",\"cvssVersion\":\"fqbuaceopzf\",\"cvssV2Score\":\"hhuao\",\"cvssV3Score\":\"pcqeqx\",\"links\":[{\"href\":\"ahzxctobgbk\",\"label\":\"oizpostmgrcfbun\"}],\"description\":\"fqjhhkxbpvjymj\"},\"id\":\"xjyngudivk\",\"name\":\"tswb\",\"type\":\"qzvszjf\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.cves().listByFirmware("o", "hb", "xknalaulppg", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("pnapnyiropuh", response.iterator().next().cveId()); + Assertions.assertEquals("vpgylgqgitxmed", response.iterator().next().component().componentId()); + Assertions.assertEquals("c", response.iterator().next().component().name()); + Assertions.assertEquals("ynqwwncwzzhxgk", response.iterator().next().component().version()); + Assertions.assertEquals("mgucna", response.iterator().next().severity()); + Assertions.assertEquals("t", response.iterator().next().namePropertiesName()); + Assertions.assertEquals("ellwptfdy", response.iterator().next().cvssScore()); + Assertions.assertEquals("fqbuaceopzf", response.iterator().next().cvssVersion()); + Assertions.assertEquals("hhuao", response.iterator().next().cvssV2Score()); + Assertions.assertEquals("pcqeqx", response.iterator().next().cvssV3Score()); + Assertions.assertEquals("fqjhhkxbpvjymj", response.iterator().next().description()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareInnerTests.java deleted file mode 100644 index 31ad01139a11f..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareInnerTests.java +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareInner; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class FirmwareInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - FirmwareInner model = - BinaryData - .fromString( - "{\"properties\":{\"fileName\":\"fcyzkohdbihanufh\",\"vendor\":\"bj\",\"model\":\"a\",\"version\":\"th\",\"description\":\"hab\",\"fileSize\":8979181371698395815,\"status\":\"Analyzing\",\"statusMessages\":[\"datazb\",\"datascnpqxuhivy\"],\"provisioningState\":\"Failed\"},\"id\":\"b\",\"name\":\"br\",\"type\":\"xvd\"}") - .toObject(FirmwareInner.class); - Assertions.assertEquals("fcyzkohdbihanufh", model.fileName()); - Assertions.assertEquals("bj", model.vendor()); - Assertions.assertEquals("a", model.model()); - Assertions.assertEquals("th", model.version()); - Assertions.assertEquals("hab", model.description()); - Assertions.assertEquals(8979181371698395815L, model.fileSize()); - Assertions.assertEquals(Status.ANALYZING, model.status()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - FirmwareInner model = - new FirmwareInner() - .withFileName("fcyzkohdbihanufh") - .withVendor("bj") - .withModel("a") - .withVersion("th") - .withDescription("hab") - .withFileSize(8979181371698395815L) - .withStatus(Status.ANALYZING) - .withStatusMessages(Arrays.asList("datazb", "datascnpqxuhivy")); - model = BinaryData.fromObject(model).toObject(FirmwareInner.class); - Assertions.assertEquals("fcyzkohdbihanufh", model.fileName()); - Assertions.assertEquals("bj", model.vendor()); - Assertions.assertEquals("a", model.model()); - Assertions.assertEquals("th", model.version()); - Assertions.assertEquals("hab", model.description()); - Assertions.assertEquals(8979181371698395815L, model.fileSize()); - Assertions.assertEquals(Status.ANALYZING, model.status()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListTests.java deleted file mode 100644 index 3fa2511a5d891..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareListTests.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareList; -import org.junit.jupiter.api.Assertions; - -public final class FirmwareListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - FirmwareList model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"fileName\":\"uv\",\"vendor\":\"xpyb\",\"model\":\"m\",\"version\":\"mtz\",\"description\":\"bsphrupidgsybbe\",\"fileSize\":1367507172981204915,\"status\":\"Analyzing\",\"statusMessages\":[\"datasx\",\"dataobhdxbmtqioqjze\"],\"provisioningState\":\"Succeeded\"},\"id\":\"ufpo\",\"name\":\"noi\",\"type\":\"hwlrx\"},{\"properties\":{\"fileName\":\"soqijg\",\"vendor\":\"mbpazlobcufpdzn\",\"model\":\"t\",\"version\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\",\"fileSize\":5657078495382185794,\"status\":\"Error\",\"statusMessages\":[\"dataf\",\"datawmrvktsizntocipa\",\"datauajpsquc\",\"datapoyfdkfogkn\"],\"provisioningState\":\"Succeeded\"},\"id\":\"fjddeqs\",\"name\":\"deupewnwrei\",\"type\":\"jzyflu\"},{\"properties\":{\"fileName\":\"hmofc\",\"vendor\":\"smy\",\"model\":\"kdtmlxhekuk\",\"version\":\"txukcdmp\",\"description\":\"cryuan\",\"fileSize\":8444265625651342832,\"status\":\"Error\",\"statusMessages\":[\"dataa\"],\"provisioningState\":\"Failed\"},\"id\":\"m\",\"name\":\"hfpmrqobmtukknr\",\"type\":\"rtihfxtijbpz\"}],\"nextLink\":\"nwzsymg\"}") - .toObject(FirmwareList.class); - Assertions.assertEquals("nwzsymg", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - FirmwareList model = new FirmwareList().withNextLink("nwzsymg"); - model = BinaryData.fromObject(model).toObject(FirmwareList.class); - Assertions.assertEquals("nwzsymg", model.nextLink()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwarePropertiesTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwarePropertiesTests.java deleted file mode 100644 index 35ff158065a55..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwarePropertiesTests.java +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareProperties; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class FirmwarePropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - FirmwareProperties model = - BinaryData - .fromString( - "{\"fileName\":\"jgrtfwvukxga\",\"vendor\":\"ccsnhsjc\",\"model\":\"ejhkry\",\"version\":\"napczwlokjy\",\"description\":\"kkvnipjox\",\"fileSize\":6610645791355850354,\"status\":\"Extracting\",\"statusMessages\":[\"dataspodmail\",\"dataydehoj\",\"datayahux\",\"datanpmqnjaqwixjspro\"],\"provisioningState\":\"Failed\"}") - .toObject(FirmwareProperties.class); - Assertions.assertEquals("jgrtfwvukxga", model.fileName()); - Assertions.assertEquals("ccsnhsjc", model.vendor()); - Assertions.assertEquals("ejhkry", model.model()); - Assertions.assertEquals("napczwlokjy", model.version()); - Assertions.assertEquals("kkvnipjox", model.description()); - Assertions.assertEquals(6610645791355850354L, model.fileSize()); - Assertions.assertEquals(Status.EXTRACTING, model.status()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - FirmwareProperties model = - new FirmwareProperties() - .withFileName("jgrtfwvukxga") - .withVendor("ccsnhsjc") - .withModel("ejhkry") - .withVersion("napczwlokjy") - .withDescription("kkvnipjox") - .withFileSize(6610645791355850354L) - .withStatus(Status.EXTRACTING) - .withStatusMessages(Arrays.asList("dataspodmail", "dataydehoj", "datayahux", "datanpmqnjaqwixjspro")); - model = BinaryData.fromObject(model).toObject(FirmwareProperties.class); - Assertions.assertEquals("jgrtfwvukxga", model.fileName()); - Assertions.assertEquals("ccsnhsjc", model.vendor()); - Assertions.assertEquals("ejhkry", model.model()); - Assertions.assertEquals("napczwlokjy", model.version()); - Assertions.assertEquals("kkvnipjox", model.description()); - Assertions.assertEquals(6610645791355850354L, model.fileSize()); - Assertions.assertEquals(Status.EXTRACTING, model.status()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryInnerTests.java deleted file mode 100644 index 1e785b6f7fe78..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryInnerTests.java +++ /dev/null @@ -1,48 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.FirmwareSummaryInner; -import org.junit.jupiter.api.Assertions; - -public final class FirmwareSummaryInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - FirmwareSummaryInner model = - BinaryData - .fromString( - "{\"extractedSize\":2192126563466196128,\"fileSize\":1776439872109058737,\"extractedFileCount\":5524797481405641879,\"componentCount\":6443293507210712059,\"binaryCount\":3252772488790490647,\"analysisTimeSeconds\":8622826594592540845,\"rootFileSystems\":2343470507309759072}") - .toObject(FirmwareSummaryInner.class); - Assertions.assertEquals(2192126563466196128L, model.extractedSize()); - Assertions.assertEquals(1776439872109058737L, model.fileSize()); - Assertions.assertEquals(5524797481405641879L, model.extractedFileCount()); - Assertions.assertEquals(6443293507210712059L, model.componentCount()); - Assertions.assertEquals(3252772488790490647L, model.binaryCount()); - Assertions.assertEquals(8622826594592540845L, model.analysisTimeSeconds()); - Assertions.assertEquals(2343470507309759072L, model.rootFileSystems()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - FirmwareSummaryInner model = - new FirmwareSummaryInner() - .withExtractedSize(2192126563466196128L) - .withFileSize(1776439872109058737L) - .withExtractedFileCount(5524797481405641879L) - .withComponentCount(6443293507210712059L) - .withBinaryCount(3252772488790490647L) - .withAnalysisTimeSeconds(8622826594592540845L) - .withRootFileSystems(2343470507309759072L); - model = BinaryData.fromObject(model).toObject(FirmwareSummaryInner.class); - Assertions.assertEquals(2192126563466196128L, model.extractedSize()); - Assertions.assertEquals(1776439872109058737L, model.fileSize()); - Assertions.assertEquals(5524797481405641879L, model.extractedFileCount()); - Assertions.assertEquals(6443293507210712059L, model.componentCount()); - Assertions.assertEquals(3252772488790490647L, model.binaryCount()); - Assertions.assertEquals(8622826594592540845L, model.analysisTimeSeconds()); - Assertions.assertEquals(2343470507309759072L, model.rootFileSystems()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryTests.java new file mode 100644 index 0000000000000..f87354e3032d4 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareSummaryTests.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareSummary; +import org.junit.jupiter.api.Assertions; + +public final class FirmwareSummaryTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + FirmwareSummary model = BinaryData.fromString( + "{\"summaryType\":\"Firmware\",\"extractedSize\":5328487592567211708,\"fileSize\":8973268749707616780,\"extractedFileCount\":4408315824184510745,\"componentCount\":4532260686626458912,\"binaryCount\":3431496164141244398,\"analysisTimeSeconds\":7532014789770741104,\"rootFileSystems\":2762302346763758082}") + .toObject(FirmwareSummary.class); + Assertions.assertEquals(5328487592567211708L, model.extractedSize()); + Assertions.assertEquals(8973268749707616780L, model.fileSize()); + Assertions.assertEquals(4408315824184510745L, model.extractedFileCount()); + Assertions.assertEquals(4532260686626458912L, model.componentCount()); + Assertions.assertEquals(3431496164141244398L, model.binaryCount()); + Assertions.assertEquals(7532014789770741104L, model.analysisTimeSeconds()); + Assertions.assertEquals(2762302346763758082L, model.rootFileSystems()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + FirmwareSummary model = new FirmwareSummary().withExtractedSize(5328487592567211708L) + .withFileSize(8973268749707616780L).withExtractedFileCount(4408315824184510745L) + .withComponentCount(4532260686626458912L).withBinaryCount(3431496164141244398L) + .withAnalysisTimeSeconds(7532014789770741104L).withRootFileSystems(2762302346763758082L); + model = BinaryData.fromObject(model).toObject(FirmwareSummary.class); + Assertions.assertEquals(5328487592567211708L, model.extractedSize()); + Assertions.assertEquals(8973268749707616780L, model.fileSize()); + Assertions.assertEquals(4408315824184510745L, model.extractedFileCount()); + Assertions.assertEquals(4532260686626458912L, model.componentCount()); + Assertions.assertEquals(3431496164141244398L, model.binaryCount()); + Assertions.assertEquals(7532014789770741104L, model.analysisTimeSeconds()); + Assertions.assertEquals(2762302346763758082L, model.rootFileSystems()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateDefinitionTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateDefinitionTests.java deleted file mode 100644 index 57ae64886d1f7..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwareUpdateDefinitionTests.java +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareUpdateDefinition; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class FirmwareUpdateDefinitionTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - FirmwareUpdateDefinition model = - BinaryData - .fromString( - "{\"properties\":{\"fileName\":\"tegjvwmf\",\"vendor\":\"t\",\"model\":\"mdvpjhulsu\",\"version\":\"mkjozkrwf\",\"description\":\"iodjp\",\"fileSize\":548063352355670740,\"status\":\"Extracting\",\"statusMessages\":[\"datawryoqpsoacc\",\"dataazakl\",\"datalahbcryff\"],\"provisioningState\":\"Succeeded\"}}") - .toObject(FirmwareUpdateDefinition.class); - Assertions.assertEquals("tegjvwmf", model.fileName()); - Assertions.assertEquals("t", model.vendor()); - Assertions.assertEquals("mdvpjhulsu", model.model()); - Assertions.assertEquals("mkjozkrwf", model.version()); - Assertions.assertEquals("iodjp", model.description()); - Assertions.assertEquals(548063352355670740L, model.fileSize()); - Assertions.assertEquals(Status.EXTRACTING, model.status()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - FirmwareUpdateDefinition model = - new FirmwareUpdateDefinition() - .withFileName("tegjvwmf") - .withVendor("t") - .withModel("mdvpjhulsu") - .withVersion("mkjozkrwf") - .withDescription("iodjp") - .withFileSize(548063352355670740L) - .withStatus(Status.EXTRACTING) - .withStatusMessages(Arrays.asList("datawryoqpsoacc", "dataazakl", "datalahbcryff")); - model = BinaryData.fromObject(model).toObject(FirmwareUpdateDefinition.class); - Assertions.assertEquals("tegjvwmf", model.fileName()); - Assertions.assertEquals("t", model.vendor()); - Assertions.assertEquals("mdvpjhulsu", model.model()); - Assertions.assertEquals("mkjozkrwf", model.version()); - Assertions.assertEquals("iodjp", model.description()); - Assertions.assertEquals(548063352355670740L, model.fileSize()); - Assertions.assertEquals(Status.EXTRACTING, model.status()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateWithResponseMockTests.java deleted file mode 100644 index 9778857458e26..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresCreateWithResponseMockTests.java +++ /dev/null @@ -1,87 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresCreateWithResponseMockTests { - @Test - public void testCreateWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"fileName\":\"cfbu\",\"vendor\":\"mfqjhhkxbp\",\"model\":\"ymjhxxjyngudivkr\",\"version\":\"wbxqzvszjfau\",\"description\":\"fdxxivetvtcqaqtd\",\"fileSize\":2614581750064023217,\"status\":\"Ready\",\"statusMessages\":[\"datavxysl\",\"databhsfxob\"],\"provisioningState\":\"Canceled\"},\"id\":\"blmpewww\",\"name\":\"bkrvrnsvshqj\",\"type\":\"hxcr\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Firmware response = - manager - .firmwares() - .define("gvpgy") - .withExistingWorkspace("xknalaulppg", "dtpnapnyiropuhp") - .withFileName("git") - .withVendor("edjvcslynqw") - .withModel("cwzzhxgktr") - .withVersion("ucnapkteoellwp") - .withDescription("d") - .withFileSize(4494990539994529771L) - .withStatus(Status.EXTRACTING) - .withStatusMessages(Arrays.asList("dataceopzfqrhhuaopp")) - .create(); - - Assertions.assertEquals("cfbu", response.fileName()); - Assertions.assertEquals("mfqjhhkxbp", response.vendor()); - Assertions.assertEquals("ymjhxxjyngudivkr", response.model()); - Assertions.assertEquals("wbxqzvszjfau", response.version()); - Assertions.assertEquals("fdxxivetvtcqaqtd", response.description()); - Assertions.assertEquals(2614581750064023217L, response.fileSize()); - Assertions.assertEquals(Status.READY, response.status()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteWithResponseMockTests.java index fbf88d6ece8a5..d7634d6178d1b 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresDeleteWithResponseMockTests.java @@ -32,32 +32,20 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .firmwares() - .deleteWithResponse("hibnuqqkpika", "rgvtqag", "buynhijggm", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.firmwares().deleteWithResponse("bfovasrruvwbhsq", "sub", "gjb", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningDetailsWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningDetailsWithResponseMockTests.java deleted file mode 100644 index c36c475655416..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningDetailsWithResponseMockTests.java +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardening; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresGenerateBinaryHardeningDetailsWithResponseMockTests { - @Test - public void testGenerateBinaryHardeningDetailsWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"binaryHardeningId\":\"fm\",\"features\":{\"nx\":\"True\",\"pie\":\"True\",\"relro\":\"True\",\"canary\":\"False\",\"stripped\":\"True\"},\"architecture\":\"hahvljuahaq\",\"path\":\"c\",\"class\":\"mdua\",\"runpath\":\"exq\",\"rpath\":\"fadmws\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - BinaryHardening response = - manager - .firmwares() - .generateBinaryHardeningDetailsWithResponse("v", "us", "tslhspkdeem", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("fm", response.binaryHardeningId()); - Assertions.assertEquals("hahvljuahaq", response.architecture()); - Assertions.assertEquals("c", response.path()); - Assertions.assertEquals("mdua", response.classProperty()); - Assertions.assertEquals("exq", response.runpath()); - Assertions.assertEquals("fadmws", response.rpath()); - Assertions.assertEquals(NxFlag.TRUE, response.nx()); - Assertions.assertEquals(PieFlag.TRUE, response.pie()); - Assertions.assertEquals(RelroFlag.TRUE, response.relro()); - Assertions.assertEquals(CanaryFlag.FALSE, response.canary()); - Assertions.assertEquals(StrippedFlag.TRUE, response.stripped()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningSummaryWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningSummaryWithResponseMockTests.java deleted file mode 100644 index 9e6243546bc23..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateBinaryHardeningSummaryWithResponseMockTests.java +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardeningSummary; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresGenerateBinaryHardeningSummaryWithResponseMockTests { - @Test - public void testGenerateBinaryHardeningSummaryWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"totalFiles\":8812234948991108432,\"nx\":946722582,\"pie\":1666097242,\"relro\":1542253292,\"canary\":1198251199,\"stripped\":849293677}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - BinaryHardeningSummary response = - manager - .firmwares() - .generateBinaryHardeningSummaryWithResponse( - "bkyvp", "ca", "uzbpzkafku", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals(8812234948991108432L, response.totalFiles()); - Assertions.assertEquals(946722582, response.nx()); - Assertions.assertEquals(1666097242, response.pie()); - Assertions.assertEquals(1542253292, response.relro()); - Assertions.assertEquals(1198251199, response.canary()); - Assertions.assertEquals(849293677, response.stripped()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateComponentDetailsWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateComponentDetailsWithResponseMockTests.java deleted file mode 100644 index 2ed3d2271e7b3..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateComponentDetailsWithResponseMockTests.java +++ /dev/null @@ -1,77 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.Component; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsUpdateAvailable; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresGenerateComponentDetailsWithResponseMockTests { - @Test - public void testGenerateComponentDetailsWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"componentId\":\"ilovnot\",\"componentName\":\"jfcn\",\"version\":\"k\",\"license\":\"x\",\"releaseDate\":\"2021-07-21T19:51:01Z\",\"paths\":[\"kphywpnvjto\",\"nermcl\",\"plpho\",\"uscrpabgyepsb\"],\"isUpdateAvailable\":\"False\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Component response = - manager - .firmwares() - .generateComponentDetailsWithResponse("g", "wrupqsxvnmicykvc", "o", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("ilovnot", response.componentId()); - Assertions.assertEquals("jfcn", response.componentName()); - Assertions.assertEquals("k", response.version()); - Assertions.assertEquals("x", response.license()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-21T19:51:01Z"), response.releaseDate()); - Assertions.assertEquals("kphywpnvjto", response.paths().get(0)); - Assertions.assertEquals(IsUpdateAvailable.FALSE, response.isUpdateAvailable()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateCveSummaryWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateCveSummaryWithResponseMockTests.java deleted file mode 100644 index 1f09798ac7195..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateCveSummaryWithResponseMockTests.java +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.CveSummary; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresGenerateCveSummaryWithResponseMockTests { - @Test - public void testGenerateCveSummaryWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"critical\":6671653333153914115,\"high\":6973940601441794238,\"medium\":7890354998634829119,\"low\":7832013398863943919,\"unknown\":9129367680420401865,\"undefined\":7757966335492850327}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - CveSummary response = - manager - .firmwares() - .generateCveSummaryWithResponse( - "hfxobbcswsrtj", "iplrbpbewtghfgb", "c", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals(6671653333153914115L, response.critical()); - Assertions.assertEquals(6973940601441794238L, response.high()); - Assertions.assertEquals(7890354998634829119L, response.medium()); - Assertions.assertEquals(7832013398863943919L, response.low()); - Assertions.assertEquals(9129367680420401865L, response.unknown()); - Assertions.assertEquals(7757966335492850327L, response.undefined()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlWithResponseMockTests.java index 4cd30cdf0bf80..33eff4e77ab4e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateDownloadUrlWithResponseMockTests.java @@ -29,38 +29,25 @@ public void testGenerateDownloadUrlWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"url\":\"zjancuxr\",\"uploadUrl\":\"wbavxbniwdj\"}"; + String responseStr = "{\"url\":\"hqlkthumaqo\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + UrlToken response = manager.firmwares().generateDownloadUrlWithResponse("naenqpehindo", "gmifthnzd", + "dslgnayqigynduh", com.azure.core.util.Context.NONE).getValue(); - UrlToken response = - manager - .firmwares() - .generateDownloadUrlWithResponse("bxu", "wbhqwal", "uzyoxaep", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlWithResponseMockTests.java index 42b75bfd21391..19644d66dd70e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateFilesystemDownloadUrlWithResponseMockTests.java @@ -29,38 +29,25 @@ public void testGenerateFilesystemDownloadUrlWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"url\":\"ytxhp\",\"uploadUrl\":\"bzpfzab\"}"; + String responseStr = "{\"url\":\"n\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + UrlToken response = manager.firmwares().generateFilesystemDownloadUrlWithResponse("bgycduiertgccym", "aolps", + "lqlfm", com.azure.core.util.Context.NONE).getValue(); - UrlToken response = - manager - .firmwares() - .generateFilesystemDownloadUrlWithResponse("wz", "s", "bpg", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateSummaryWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateSummaryWithResponseMockTests.java deleted file mode 100644 index 924d011e2f37d..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGenerateSummaryWithResponseMockTests.java +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.FirmwareSummary; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresGenerateSummaryWithResponseMockTests { - @Test - public void testGenerateSummaryWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"extractedSize\":6050958993805304885,\"fileSize\":988487328349657078,\"extractedFileCount\":4353962267137944263,\"componentCount\":4045271831667825207,\"binaryCount\":5126500575476383860,\"analysisTimeSeconds\":319753834170561939,\"rootFileSystems\":1866971331588926084}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - FirmwareSummary response = - manager - .firmwares() - .generateSummaryWithResponse("lcuhxwtctyqiklb", "ovplw", "bhvgy", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals(6050958993805304885L, response.extractedSize()); - Assertions.assertEquals(988487328349657078L, response.fileSize()); - Assertions.assertEquals(4353962267137944263L, response.extractedFileCount()); - Assertions.assertEquals(4045271831667825207L, response.componentCount()); - Assertions.assertEquals(5126500575476383860L, response.binaryCount()); - Assertions.assertEquals(319753834170561939L, response.analysisTimeSeconds()); - Assertions.assertEquals(1866971331588926084L, response.rootFileSystems()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetWithResponseMockTests.java deleted file mode 100644 index 7ce768dbd9f96..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresGetWithResponseMockTests.java +++ /dev/null @@ -1,77 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresGetWithResponseMockTests { - @Test - public void testGetWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"fileName\":\"dbhrbnlankxm\",\"vendor\":\"k\",\"model\":\"henbtkcxywnytn\",\"version\":\"yn\",\"description\":\"idybyxczf\",\"fileSize\":3227528533061557739,\"status\":\"Extracting\",\"statusMessages\":[\"dataabphlw\",\"dataqlfktsths\",\"datacocmnyyaztt\"],\"provisioningState\":\"Accepted\"},\"id\":\"rq\",\"name\":\"uedck\",\"type\":\"ywbiexzfeyueax\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Firmware response = - manager - .firmwares() - .getWithResponse("bfs", "arbu", "rcvpnazzmhjrunmp", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("dbhrbnlankxm", response.fileName()); - Assertions.assertEquals("k", response.vendor()); - Assertions.assertEquals("henbtkcxywnytn", response.model()); - Assertions.assertEquals("yn", response.version()); - Assertions.assertEquals("idybyxczf", response.description()); - Assertions.assertEquals(3227528533061557739L, response.fileSize()); - Assertions.assertEquals(Status.EXTRACTING, response.status()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceMockTests.java deleted file mode 100644 index f6ffd5805a187..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListByWorkspaceMockTests.java +++ /dev/null @@ -1,75 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.Firmware; -import com.azure.resourcemanager.iotfirmwaredefense.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresListByWorkspaceMockTests { - @Test - public void testListByWorkspace() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"fileName\":\"nljky\",\"vendor\":\"j\",\"model\":\"ujqgidok\",\"version\":\"ljyoxgvcltb\",\"description\":\"ncghkje\",\"fileSize\":1198495964069172235,\"status\":\"Error\",\"statusMessages\":[\"datatxfvgx\",\"datafsm\",\"datanehmpvecx\",\"dataodebfqkkrbmpu\"],\"provisioningState\":\"Canceled\"},\"id\":\"wflzlfbxzpuzy\",\"name\":\"ispnqzahmgkbrp\",\"type\":\"y\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.firmwares().listByWorkspace("fuwutttxf", "jrbirphxepcyv", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("nljky", response.iterator().next().fileName()); - Assertions.assertEquals("j", response.iterator().next().vendor()); - Assertions.assertEquals("ujqgidok", response.iterator().next().model()); - Assertions.assertEquals("ljyoxgvcltb", response.iterator().next().version()); - Assertions.assertEquals("ncghkje", response.iterator().next().description()); - Assertions.assertEquals(1198495964069172235L, response.iterator().next().fileSize()); - Assertions.assertEquals(Status.ERROR, response.iterator().next().status()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateBinaryHardeningListMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateBinaryHardeningListMockTests.java deleted file mode 100644 index 704bc51f3ae72..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateBinaryHardeningListMockTests.java +++ /dev/null @@ -1,85 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.BinaryHardening; -import com.azure.resourcemanager.iotfirmwaredefense.models.CanaryFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.NxFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.PieFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.RelroFlag; -import com.azure.resourcemanager.iotfirmwaredefense.models.StrippedFlag; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresListGenerateBinaryHardeningListMockTests { - @Test - public void testListGenerateBinaryHardeningList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"binaryHardeningId\":\"qkqujidsu\",\"features\":{\"nx\":\"False\",\"pie\":\"True\",\"relro\":\"False\",\"canary\":\"True\",\"stripped\":\"True\"},\"architecture\":\"ccm\",\"path\":\"udxytlmoyrx\",\"class\":\"fudwpznt\",\"runpath\":\"dzhlrq\",\"rpath\":\"hckfrlhrx\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .firmwares() - .listGenerateBinaryHardeningList("zq", "gxywpmue", "fjz", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("qkqujidsu", response.iterator().next().binaryHardeningId()); - Assertions.assertEquals("ccm", response.iterator().next().architecture()); - Assertions.assertEquals("udxytlmoyrx", response.iterator().next().path()); - Assertions.assertEquals("fudwpznt", response.iterator().next().classProperty()); - Assertions.assertEquals("dzhlrq", response.iterator().next().runpath()); - Assertions.assertEquals("hckfrlhrx", response.iterator().next().rpath()); - Assertions.assertEquals(NxFlag.FALSE, response.iterator().next().nx()); - Assertions.assertEquals(PieFlag.TRUE, response.iterator().next().pie()); - Assertions.assertEquals(RelroFlag.FALSE, response.iterator().next().relro()); - Assertions.assertEquals(CanaryFlag.TRUE, response.iterator().next().canary()); - Assertions.assertEquals(StrippedFlag.TRUE, response.iterator().next().stripped()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateComponentListMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateComponentListMockTests.java deleted file mode 100644 index f45e8ffc5c047..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateComponentListMockTests.java +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.Component; -import com.azure.resourcemanager.iotfirmwaredefense.models.IsUpdateAvailable; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresListGenerateComponentListMockTests { - @Test - public void testListGenerateComponentList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"componentId\":\"ped\",\"componentName\":\"jn\",\"version\":\"ckhsmtxpsieb\",\"license\":\"hvpesapskrdqm\",\"releaseDate\":\"2021-01-31T10:46:35Z\",\"paths\":[\"tldwkyzxuutk\",\"cwscwsvlx\"],\"isUpdateAvailable\":\"False\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .firmwares() - .listGenerateComponentList( - "sxnkjzkdeslpvlo", "wiyighxpkdw", "baiuebbaumny", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("ped", response.iterator().next().componentId()); - Assertions.assertEquals("jn", response.iterator().next().componentName()); - Assertions.assertEquals("ckhsmtxpsieb", response.iterator().next().version()); - Assertions.assertEquals("hvpesapskrdqm", response.iterator().next().license()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-31T10:46:35Z"), response.iterator().next().releaseDate()); - Assertions.assertEquals("tldwkyzxuutk", response.iterator().next().paths().get(0)); - Assertions.assertEquals(IsUpdateAvailable.FALSE, response.iterator().next().isUpdateAvailable()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateCveListMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateCveListMockTests.java deleted file mode 100644 index 8521278be0df0..0000000000000 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/FirmwaresListGenerateCveListMockTests.java +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.iotfirmwaredefense.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; -import com.azure.resourcemanager.iotfirmwaredefense.models.Cve; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class FirmwaresListGenerateCveListMockTests { - @Test - public void testListGenerateCveList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"cveId\":\"cwwfvovbvme\",\"component\":\"datacivyhzceuo\",\"severity\":\"jrwjueiotwm\",\"name\":\"ytdxwit\",\"cvssScore\":\"rjaw\",\"cvssVersion\":\"wgxhn\",\"cvssV2Score\":\"kxfbkpycgklwndn\",\"cvssV3Score\":\"dauwhvylwzbtd\",\"publishDate\":\"2021-05-11T10:23:35Z\",\"updatedDate\":\"2021-06-17T22:36:31Z\",\"links\":[{\"href\":\"pow\",\"label\":\"przqlveu\"}],\"description\":\"upjm\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .firmwares() - .listGenerateCveList("uhashsfwx", "sowzxcugi", "jooxdjebw", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("cwwfvovbvme", response.iterator().next().cveId()); - Assertions.assertEquals("jrwjueiotwm", response.iterator().next().severity()); - Assertions.assertEquals("ytdxwit", response.iterator().next().name()); - Assertions.assertEquals("rjaw", response.iterator().next().cvssScore()); - Assertions.assertEquals("wgxhn", response.iterator().next().cvssVersion()); - Assertions.assertEquals("kxfbkpycgklwndn", response.iterator().next().cvssV2Score()); - Assertions.assertEquals("dauwhvylwzbtd", response.iterator().next().cvssV3Score()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-11T10:23:35Z"), response.iterator().next().publishDate()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-17T22:36:31Z"), response.iterator().next().updatedDate()); - Assertions.assertEquals("upjm", response.iterator().next().description()); - } -} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/GenerateUploadUrlRequestTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/GenerateUploadUrlRequestTests.java index e2cdd97100687..efab46290700a 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/GenerateUploadUrlRequestTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/GenerateUploadUrlRequestTests.java @@ -11,15 +11,15 @@ public final class GenerateUploadUrlRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GenerateUploadUrlRequest model = - BinaryData.fromString("{\"firmwareId\":\"u\"}").toObject(GenerateUploadUrlRequest.class); - Assertions.assertEquals("u", model.firmwareId()); + GenerateUploadUrlRequest model + = BinaryData.fromString("{\"firmwareId\":\"uecivyhz\"}").toObject(GenerateUploadUrlRequest.class); + Assertions.assertEquals("uecivyhz", model.firmwareId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GenerateUploadUrlRequest model = new GenerateUploadUrlRequest().withFirmwareId("u"); + GenerateUploadUrlRequest model = new GenerateUploadUrlRequest().withFirmwareId("uecivyhz"); model = BinaryData.fromObject(model).toObject(GenerateUploadUrlRequest.class); - Assertions.assertEquals("u", model.firmwareId()); + Assertions.assertEquals("uecivyhz", model.firmwareId()); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationDisplayTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationDisplayTests.java index 0f85e1fe85152..e3fafb56d330c 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationDisplayTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationDisplayTests.java @@ -10,11 +10,9 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"wlrsffrzpwv\",\"resource\":\"dqgbiqylihkaetc\",\"operation\":\"vfcivfsnkymuc\",\"description\":\"hjfbebrjcxe\"}") - .toObject(OperationDisplay.class); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"mpvecxgodebfqk\",\"resource\":\"bmpukgriwflz\",\"operation\":\"bxzpuzycisp\",\"description\":\"zahmgkbrpyydhibn\"}") + .toObject(OperationDisplay.class); } @org.junit.jupiter.api.Test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationInnerTests.java index d0b243ccc2188..e43d95e091a05 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationInnerTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationInnerTests.java @@ -11,11 +11,9 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"w\",\"isDataAction\":false,\"display\":{\"provider\":\"mmqhgyxzkonocuk\",\"resource\":\"lyaxuc\",\"operation\":\"uqszfk\",\"description\":\"ypewrmjmwvvjekt\"},\"origin\":\"system\",\"actionType\":\"Internal\"}") - .toObject(OperationInner.class); + OperationInner model = BinaryData.fromString( + "{\"name\":\"pcyvahfnljkyqx\",\"isDataAction\":true,\"display\":{\"provider\":\"qgidokgjljyo\",\"resource\":\"vcltbgsncgh\",\"operation\":\"esz\",\"description\":\"bijhtxfvgxbf\"},\"origin\":\"system\",\"actionType\":\"Internal\"}") + .toObject(OperationInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationListResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationListResultTests.java index 678e5018f2723..2b42fe75b5b2a 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationListResultTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationListResultTests.java @@ -10,11 +10,9 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"faxkffeii\",\"isDataAction\":true,\"display\":{\"provider\":\"ez\",\"resource\":\"shxmzsbbzoggigrx\",\"operation\":\"ur\",\"description\":\"xxjnspydptk\"},\"origin\":\"system\",\"actionType\":\"Internal\"},{\"name\":\"knvudwtiukb\",\"isDataAction\":true,\"display\":{\"provider\":\"pocipazyxoegu\",\"resource\":\"jnpiucgyg\",\"operation\":\"qzntypm\",\"description\":\"p\"},\"origin\":\"user\",\"actionType\":\"Internal\"},{\"name\":\"j\",\"isDataAction\":false,\"display\":{\"provider\":\"nfyhx\",\"resource\":\"oejzi\",\"operation\":\"ifsjttgzfbishcb\",\"description\":\"ajdeyeamdphaga\"},\"origin\":\"user\",\"actionType\":\"Internal\"},{\"name\":\"gipwhonowkg\",\"isDataAction\":true,\"display\":{\"provider\":\"kix\",\"resource\":\"injep\",\"operation\":\"tmryw\",\"description\":\"zoqftiyqzrnkcqvy\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"sicohoqqnwvlry\"}") - .toObject(OperationListResult.class); + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"drqjsdpy\",\"isDataAction\":false,\"display\":{\"provider\":\"xdeoejzic\",\"resource\":\"fsj\",\"operation\":\"gzfbishcbk\",\"description\":\"jdeyeamdpha\"},\"origin\":\"system\",\"actionType\":\"Internal\"},{\"name\":\"xw\",\"isDataAction\":true,\"display\":{\"provider\":\"onowk\",\"resource\":\"hwankixzbinjepu\",\"operation\":\"mryw\",\"description\":\"zoqftiyqzrnkcqvy\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"sicohoqqnwvlry\",\"isDataAction\":false,\"display\":{\"provider\":\"eun\",\"resource\":\"qhgyxzkonocukok\",\"operation\":\"axuconuq\",\"description\":\"fkbey\"},\"origin\":\"user\",\"actionType\":\"Internal\"},{\"name\":\"mwvvjektcxsenhw\",\"isDataAction\":false,\"display\":{\"provider\":\"rzpwvlqdqgbiq\",\"resource\":\"ihkaetcktvfc\",\"operation\":\"fsnkymuctq\",\"description\":\"fbebrjcxer\"},\"origin\":\"system\",\"actionType\":\"Internal\"}],\"nextLink\":\"txfvjrbirph\"}") + .toObject(OperationListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListMockTests.java index 84d410e06774b..1ffab7290ce3e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/OperationsListMockTests.java @@ -30,35 +30,25 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"fwdsj\",\"isDataAction\":true,\"display\":{\"provider\":\"utiiswacf\",\"resource\":\"dkzzewkfvhqcrail\",\"operation\":\"n\",\"description\":\"fuflrwdmhdlx\"},\"origin\":\"user\",\"actionType\":\"Internal\"}]}"; + String responseStr + = "{\"value\":[{\"name\":\"dsslswt\",\"isDataAction\":true,\"display\":{\"provider\":\"ofz\",\"resource\":\"qsemwabne\",\"operation\":\"hhszh\",\"description\":\"plvwiwubmwmbes\"},\"origin\":\"user\",\"actionType\":\"Internal\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); + } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PairedKeyTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PairedKeyTests.java index 159d9915eea4a..6dbfc8ebe00ee 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PairedKeyTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/PairedKeyTests.java @@ -11,19 +11,16 @@ public final class PairedKeyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PairedKey model = - BinaryData - .fromString("{\"id\":\"okth\",\"type\":\"iuaod\",\"additionalProperties\":\"datacpkvxodp\"}") - .toObject(PairedKey.class); - Assertions.assertEquals("okth", model.id()); - Assertions.assertEquals("iuaod", model.type()); + PairedKey model = BinaryData.fromString("{\"id\":\"gfgibm\",\"type\":\"gakeqsr\"}").toObject(PairedKey.class); + Assertions.assertEquals("gfgibm", model.id()); + Assertions.assertEquals("gakeqsr", model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PairedKey model = new PairedKey().withId("okth").withType("iuaod").withAdditionalProperties("datacpkvxodp"); + PairedKey model = new PairedKey().withId("gfgibm").withType("gakeqsr"); model = BinaryData.fromObject(model).toObject(PairedKey.class); - Assertions.assertEquals("okth", model.id()); - Assertions.assertEquals("iuaod", model.type()); + Assertions.assertEquals("gfgibm", model.id()); + Assertions.assertEquals("gakeqsr", model.type()); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentListResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentListResultTests.java new file mode 100644 index 0000000000000..5f62da781aab3 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentListResultTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponentListResult; +import org.junit.jupiter.api.Assertions; + +public final class SbomComponentListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SbomComponentListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"componentId\":\"iarbutrcvpna\",\"componentName\":\"mhjrunmpxttdbhr\",\"version\":\"l\",\"license\":\"kx\",\"filePaths\":[\"k\",\"bhenbtkcxywnyt\",\"rsyn\",\"qidybyx\"]},\"id\":\"fclhaaxdbabphlwr\",\"name\":\"lfktsths\",\"type\":\"cocmnyyaztt\"}],\"nextLink\":\"wwrq\"}") + .toObject(SbomComponentListResult.class); + Assertions.assertEquals("wwrq", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SbomComponentListResult model = new SbomComponentListResult().withNextLink("wwrq"); + model = BinaryData.fromObject(model).toObject(SbomComponentListResult.class); + Assertions.assertEquals("wwrq", model.nextLink()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentResourceInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentResourceInnerTests.java new file mode 100644 index 0000000000000..9adbac5bfabc4 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentResourceInnerTests.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponentResourceInner; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class SbomComponentResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SbomComponentResourceInner model = BinaryData.fromString( + "{\"properties\":{\"componentId\":\"dckzywbiexz\",\"componentName\":\"yueaxibxujwb\",\"version\":\"walm\",\"license\":\"yoxa\",\"filePaths\":[\"kzjancuxrhdwbav\",\"bniwdj\",\"wz\"]},\"id\":\"dbpgnxytxhp\",\"name\":\"xbzpfzab\",\"type\":\"lcuhxwtctyqiklb\"}") + .toObject(SbomComponentResourceInner.class); + Assertions.assertEquals("dckzywbiexz", model.componentId()); + Assertions.assertEquals("yueaxibxujwb", model.componentName()); + Assertions.assertEquals("walm", model.version()); + Assertions.assertEquals("yoxa", model.license()); + Assertions.assertEquals("kzjancuxrhdwbav", model.filePaths().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SbomComponentResourceInner model = new SbomComponentResourceInner().withComponentId("dckzywbiexz") + .withComponentName("yueaxibxujwb").withVersion("walm").withLicense("yoxa") + .withFilePaths(Arrays.asList("kzjancuxrhdwbav", "bniwdj", "wz")); + model = BinaryData.fromObject(model).toObject(SbomComponentResourceInner.class); + Assertions.assertEquals("dckzywbiexz", model.componentId()); + Assertions.assertEquals("yueaxibxujwb", model.componentName()); + Assertions.assertEquals("walm", model.version()); + Assertions.assertEquals("yoxa", model.license()); + Assertions.assertEquals("kzjancuxrhdwbav", model.filePaths().get(0)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentTests.java new file mode 100644 index 0000000000000..4fe2b10e886db --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentTests.java @@ -0,0 +1,36 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SbomComponent; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class SbomComponentTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SbomComponent model = BinaryData.fromString( + "{\"componentId\":\"vplwzbhv\",\"componentName\":\"u\",\"version\":\"osvmk\",\"license\":\"sxqu\",\"filePaths\":[\"plgmgsxnk\"]}") + .toObject(SbomComponent.class); + Assertions.assertEquals("vplwzbhv", model.componentId()); + Assertions.assertEquals("u", model.componentName()); + Assertions.assertEquals("osvmk", model.version()); + Assertions.assertEquals("sxqu", model.license()); + Assertions.assertEquals("plgmgsxnk", model.filePaths().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SbomComponent model = new SbomComponent().withComponentId("vplwzbhv").withComponentName("u") + .withVersion("osvmk").withLicense("sxqu").withFilePaths(Arrays.asList("plgmgsxnk")); + model = BinaryData.fromObject(model).toObject(SbomComponent.class); + Assertions.assertEquals("vplwzbhv", model.componentId()); + Assertions.assertEquals("u", model.componentName()); + Assertions.assertEquals("osvmk", model.version()); + Assertions.assertEquals("sxqu", model.license()); + Assertions.assertEquals("plgmgsxnk", model.filePaths().get(0)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareMockTests.java new file mode 100644 index 0000000000000..2aafd8062f2b5 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SbomComponentsListByFirmwareMockTests.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; +import com.azure.resourcemanager.iotfirmwaredefense.models.SbomComponentResource; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class SbomComponentsListByFirmwareMockTests { + @Test + public void testListByFirmware() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"componentId\":\"lbpodxunk\",\"componentName\":\"bxmubyynt\",\"version\":\"rbqtkoie\",\"license\":\"eotg\",\"filePaths\":[\"ltmuwlauwzizx\"]},\"id\":\"pgcjefuzmuvp\",\"name\":\"ttdumorppxebmnzb\",\"type\":\"bhjpglkfgohdne\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.sbomComponents().listByFirmware("kwobdagxtibq", "xbxwa", + "bogqxndlkzgxhu", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("lbpodxunk", response.iterator().next().componentId()); + Assertions.assertEquals("bxmubyynt", response.iterator().next().componentName()); + Assertions.assertEquals("rbqtkoie", response.iterator().next().version()); + Assertions.assertEquals("eotg", response.iterator().next().license()); + Assertions.assertEquals("ltmuwlauwzizx", response.iterator().next().filePaths().get(0)); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetWithResponseMockTests.java new file mode 100644 index 0000000000000..cb9329a96fc82 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesGetWithResponseMockTests.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryName; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResource; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class SummariesGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"summaryType\":\"SummaryResourceProperties\"},\"id\":\"uflrwd\",\"name\":\"hdlxyjrxsagafcn\",\"type\":\"hgw\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + SummaryResource response = manager.summaries().getWithResponse("ljuti", "swacffgdkzz", "wkfvhqcrailvp", + SummaryName.CRYPTO_CERTIFICATE, com.azure.core.util.Context.NONE).getValue(); + + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareMockTests.java new file mode 100644 index 0000000000000..3bb64271f04e8 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummariesListByFirmwareMockTests.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.iotfirmwaredefense.IoTFirmwareDefenseManager; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResource; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class SummariesListByFirmwareMockTests { + @Test + public void testListByFirmware() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"summaryType\":\"SummaryResourceProperties\"},\"id\":\"zx\",\"name\":\"lvithhqzonosgg\",\"type\":\"hcohfwdsjnk\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.summaries().listByFirmware("el", "phsdyhto", "fikdowwqu", com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryListResultTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryListResultTests.java new file mode 100644 index 0000000000000..8d95b23f74c88 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryListResultTests.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryListResult; +import org.junit.jupiter.api.Assertions; + +public final class SummaryListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SummaryListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"summaryType\":\"SummaryResourceProperties\"},\"id\":\"slpvlop\",\"name\":\"i\",\"type\":\"ighxpk\"}],\"nextLink\":\"zb\"}") + .toObject(SummaryListResult.class); + Assertions.assertEquals("zb", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SummaryListResult model = new SummaryListResult().withNextLink("zb"); + model = BinaryData.fromObject(model).toObject(SummaryListResult.class); + Assertions.assertEquals("zb", model.nextLink()); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourceInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourceInnerTests.java new file mode 100644 index 0000000000000..fc9637018c044 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourceInnerTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.fluent.models.SummaryResourceInner; + +public final class SummaryResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SummaryResourceInner model = BinaryData.fromString( + "{\"properties\":{\"summaryType\":\"SummaryResourceProperties\"},\"id\":\"ebb\",\"name\":\"umnyqu\",\"type\":\"edeojnabc\"}") + .toObject(SummaryResourceInner.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SummaryResourceInner model = new SummaryResourceInner(); + model = BinaryData.fromObject(model).toObject(SummaryResourceInner.class); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourcePropertiesTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourcePropertiesTests.java new file mode 100644 index 0000000000000..792256684f7a3 --- /dev/null +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/SummaryResourcePropertiesTests.java @@ -0,0 +1,22 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.iotfirmwaredefense.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.iotfirmwaredefense.models.SummaryResourceProperties; + +public final class SummaryResourcePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SummaryResourceProperties model = BinaryData.fromString("{\"summaryType\":\"SummaryResourceProperties\"}") + .toObject(SummaryResourceProperties.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SummaryResourceProperties model = new SummaryResourceProperties(); + model = BinaryData.fromObject(model).toObject(SummaryResourceProperties.class); + } +} diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/UrlTokenInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/UrlTokenInnerTests.java index a72ab7c9f37d9..c20174395e07e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/UrlTokenInnerTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/UrlTokenInnerTests.java @@ -10,8 +10,7 @@ public final class UrlTokenInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UrlTokenInner model = - BinaryData.fromString("{\"url\":\"sy\",\"uploadUrl\":\"xpaojakhmsbz\"}").toObject(UrlTokenInner.class); + UrlTokenInner model = BinaryData.fromString("{\"url\":\"zntypmrb\"}").toObject(UrlTokenInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceInnerTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceInnerTests.java index ba984efbb8843..1f5b7b0eca55e 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceInnerTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceInnerTests.java @@ -13,24 +13,23 @@ public final class WorkspaceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkspaceInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\"},\"location\":\"ofoskghs\",\"tags\":{\"xieduugidyjrr\":\"imjm\",\"v\":\"byao\"},\"id\":\"xc\",\"name\":\"onpc\",\"type\":\"hocohslkev\"}") - .toObject(WorkspaceInner.class); - Assertions.assertEquals("ofoskghs", model.location()); - Assertions.assertEquals("imjm", model.tags().get("xieduugidyjrr")); + WorkspaceInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Accepted\"},\"location\":\"ualaexqpvfadmw\",\"tags\":{\"vxpvgomz\":\"r\",\"bnbbeldawkz\":\"fmisg\",\"urqhaka\":\"ali\"},\"id\":\"hashsfwxosow\",\"name\":\"xcug\",\"type\":\"cjooxdjebwpucwwf\"}") + .toObject(WorkspaceInner.class); + Assertions.assertEquals("ualaexqpvfadmw", model.location()); + Assertions.assertEquals("r", model.tags().get("vxpvgomz")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WorkspaceInner model = - new WorkspaceInner().withLocation("ofoskghs").withTags(mapOf("xieduugidyjrr", "imjm", "v", "byao")); + WorkspaceInner model = new WorkspaceInner().withLocation("ualaexqpvfadmw") + .withTags(mapOf("vxpvgomz", "r", "bnbbeldawkz", "fmisg", "urqhaka", "ali")); model = BinaryData.fromObject(model).toObject(WorkspaceInner.class); - Assertions.assertEquals("ofoskghs", model.location()); - Assertions.assertEquals("imjm", model.tags().get("xieduugidyjrr")); + Assertions.assertEquals("ualaexqpvfadmw", model.location()); + Assertions.assertEquals("r", model.tags().get("vxpvgomz")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceListTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceListTests.java index 2759db0292d5e..e395e25a91fa9 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceListTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceListTests.java @@ -11,18 +11,16 @@ public final class WorkspaceListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkspaceList model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\"},\"location\":\"surex\",\"tags\":{\"stkiiuxhqyud\":\"ryocfsfksymdd\"},\"id\":\"o\",\"name\":\"rq\",\"type\":\"b\"},{\"properties\":{\"provisioningState\":\"Canceled\"},\"location\":\"yifqrvkdvjsllrmv\",\"tags\":{\"zwtruwiqzbqjvsov\":\"watkpnpulexxb\"},\"id\":\"yokacspkw\",\"name\":\"hzdobpxjmflbvvnc\",\"type\":\"rkcciwwzjuqk\"}],\"nextLink\":\"sa\"}") - .toObject(WorkspaceList.class); - Assertions.assertEquals("sa", model.nextLink()); + WorkspaceList model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\"},\"location\":\"siebtfhvpesapskr\",\"tags\":{\"ncwscwsvlxoto\":\"hjjdhtldwkyzxuut\",\"cykvceo\":\"twrupqsxvnm\",\"vnotyfjfcnj\":\"eil\",\"nxdhbt\":\"k\"},\"id\":\"kphywpnvjto\",\"name\":\"nermcl\",\"type\":\"plpho\"},{\"properties\":{\"provisioningState\":\"Succeeded\"},\"location\":\"pabgyeps\",\"tags\":{\"wfqkquj\":\"azqugxywpmueefj\",\"cq\":\"dsuyonobgla\",\"g\":\"tcc\",\"wfudwpzntxhdzhl\":\"udxytlmoyrx\"},\"id\":\"qj\",\"name\":\"hckfrlhrx\",\"type\":\"bkyvp\"},{\"properties\":{\"provisioningState\":\"Accepted\"},\"location\":\"z\",\"tags\":{\"hhseyv\":\"kafkuwbcrnwbm\"},\"id\":\"us\",\"name\":\"tslhspkdeem\",\"type\":\"ofmxagkvtmelmqkr\"}],\"nextLink\":\"hvljuahaquh\"}") + .toObject(WorkspaceList.class); + Assertions.assertEquals("hvljuahaquh", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WorkspaceList model = new WorkspaceList().withNextLink("sa"); + WorkspaceList model = new WorkspaceList().withNextLink("hvljuahaquh"); model = BinaryData.fromObject(model).toObject(WorkspaceList.class); - Assertions.assertEquals("sa", model.nextLink()); + Assertions.assertEquals("hvljuahaquh", model.nextLink()); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacePropertiesTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacePropertiesTests.java index 9925075901ee4..fb5aced72989b 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacePropertiesTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacePropertiesTests.java @@ -10,8 +10,8 @@ public final class WorkspacePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkspaceProperties model = - BinaryData.fromString("{\"provisioningState\":\"Succeeded\"}").toObject(WorkspaceProperties.class); + WorkspaceProperties model + = BinaryData.fromString("{\"provisioningState\":\"Succeeded\"}").toObject(WorkspaceProperties.class); } @org.junit.jupiter.api.Test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceUpdateDefinitionTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceUpdateDefinitionTests.java index 206fd908f3d58..34d2134308d8d 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceUpdateDefinitionTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspaceUpdateDefinitionTests.java @@ -10,10 +10,8 @@ public final class WorkspaceUpdateDefinitionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - WorkspaceUpdateDefinition model = - BinaryData - .fromString("{\"properties\":{\"provisioningState\":\"Failed\"}}") - .toObject(WorkspaceUpdateDefinition.class); + WorkspaceUpdateDefinition model = BinaryData.fromString("{\"properties\":{\"provisioningState\":\"Accepted\"}}") + .toObject(WorkspaceUpdateDefinition.class); } @org.junit.jupiter.api.Test diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateWithResponseMockTests.java index 5900d666c64d3..ae35515729d39 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesCreateWithResponseMockTests.java @@ -32,48 +32,33 @@ public void testCreateWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Canceled\"},\"location\":\"ebmnzbtbhjpglk\",\"tags\":{\"dyhtozfikdowwquu\":\"hdneuelfph\",\"ithhqzon\":\"xzxcl\"},\"id\":\"sg\",\"name\":\"b\",\"type\":\"c\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Failed\"},\"location\":\"rpzbchckqqzqi\",\"tags\":{\"rwyhqmibzyhwitsm\":\"ysuiizynkedya\",\"pcdpumnz\":\"pyy\"},\"id\":\"mwzn\",\"name\":\"abikns\",\"type\":\"rgjhxb\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Workspace response = - manager - .workspaces() - .define("xbxwa") - .withRegion("xndlkzgxhu") - .withExistingResourceGroup("kwobdagxtibq") - .withTags(mapOf("ebxmubyynt", "lbpodxunk", "tkoievseotgq", "lrb", "tmuwlauwzi", "l")) - .create(); + Workspace response = manager.workspaces().define("aeneqnzarrwl").withRegion("fqka") + .withExistingResourceGroup("syrsndsytgadgvra") + .withTags(mapOf("bwwift", "iipfpubj", "ynfs", "hqkvpuvksgplsak")).create(); - Assertions.assertEquals("ebmnzbtbhjpglk", response.location()); - Assertions.assertEquals("hdneuelfph", response.tags().get("dyhtozfikdowwquu")); + Assertions.assertEquals("rpzbchckqqzqi", response.location()); + Assertions.assertEquals("ysuiizynkedya", response.tags().get("rwyhqmibzyhwitsm")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteByResourceGroupWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteByResourceGroupWithResponseMockTests.java index 05a71df8e7f95..7af9430e6b598 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteByResourceGroupWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesDeleteByResourceGroupWithResponseMockTests.java @@ -32,30 +32,21 @@ public void testDeleteWithResponse() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.workspaces().deleteByResourceGroupWithResponse("adbzmnvdfznud", "od", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.workspaces().deleteByResourceGroupWithResponse("qibrhosxsdqrhzoy", "i", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlWithResponseMockTests.java index ad9c61f0b862d..81b20f7c0e8a1 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGenerateUploadUrlWithResponseMockTests.java @@ -30,42 +30,27 @@ public void testGenerateUploadUrlWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"url\":\"sikvmkqzeqqkdlt\",\"uploadUrl\":\"xmhhvhgureo\"}"; + String responseStr = "{\"url\":\"ovvqfovljxywsu\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + UrlToken response = manager.workspaces() + .generateUploadUrlWithResponse("hykojoxafnndlpic", "koymkcd", + new GenerateUploadUrlRequest().withFirmwareId("bpkkpwdre"), com.azure.core.util.Context.NONE) + .getValue(); - UrlToken response = - manager - .workspaces() - .generateUploadUrlWithResponse( - "pp", - "flcxoga", - new GenerateUploadUrlRequest().withFirmwareId("onz"), - com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java index bb11440ab1c5c..a73bd9acedd45 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java @@ -30,42 +30,28 @@ public void testGetByResourceGroupWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Failed\"},\"location\":\"erscdntne\",\"tags\":{\"tdss\":\"wjmy\",\"tmweriofzpyq\":\"s\",\"hhszh\":\"emwabnet\"},\"id\":\"d\",\"name\":\"lvwiwubmwmbesl\",\"type\":\"nkww\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\"},\"location\":\"uszdtmhrkwof\",\"tags\":{\"iexpbtgiwbwo\":\"oqac\",\"kcnqxwbpo\":\"nwashrtd\",\"aasipqi\":\"ulpiuj\"},\"id\":\"obyu\",\"name\":\"erpqlpqwcciuqg\",\"type\":\"dbutauvfbtkuwhh\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Workspace response = - manager - .workspaces() - .getByResourceGroupWithResponse("xzb", "cblylpstdbhhxsr", com.azure.core.util.Context.NONE) - .getValue(); + Workspace response = manager.workspaces() + .getByResourceGroupWithResponse("m", "qyib", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("erscdntne", response.location()); - Assertions.assertEquals("wjmy", response.tags().get("tdss")); + Assertions.assertEquals("uszdtmhrkwof", response.location()); + Assertions.assertEquals("oqac", response.tags().get("iexpbtgiwbwo")); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupMockTests.java index c6e6f4f6c1702..feee6583d6f49 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListByResourceGroupMockTests.java @@ -31,39 +31,28 @@ public void testListByResourceGroup() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\"},\"location\":\"havhqlkthumaqolb\",\"tags\":{\"mvaolps\":\"duiertgcc\"},\"id\":\"lqlfm\",\"name\":\"dnbbglzps\",\"type\":\"iydmcwyhzdxs\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\"},\"location\":\"tmnubexkpzksmon\",\"tags\":{\"whojvp\":\"quxvypomgkop\",\"ysmocmbqfqvmkcxo\":\"jqg\"},\"id\":\"apvhelxprgly\",\"name\":\"tddckcb\",\"type\":\"uejrjxgc\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.workspaces().listByResourceGroup("nayqi", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.workspaces().listByResourceGroup("yhrfouyftaakcpw", com.azure.core.util.Context.NONE); - Assertions.assertEquals("havhqlkthumaqolb", response.iterator().next().location()); - Assertions.assertEquals("duiertgcc", response.iterator().next().tags().get("mvaolps")); + Assertions.assertEquals("tmnubexkpzksmon", response.iterator().next().location()); + Assertions.assertEquals("quxvypomgkop", response.iterator().next().tags().get("whojvp")); } } diff --git a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListMockTests.java b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListMockTests.java index b057dc6b5531f..86c9a1b616821 100644 --- a/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListMockTests.java +++ b/sdk/iotfirmwaredefense/azure-resourcemanager-iotfirmwaredefense/src/test/java/com/azure/resourcemanager/iotfirmwaredefense/generated/WorkspacesListMockTests.java @@ -31,38 +31,27 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\"},\"location\":\"asrru\",\"tags\":{\"bsrfbj\":\"hsqfsubcgjbirxbp\",\"otftpvjzbexilz\":\"dtws\",\"qtaruoujmkcjhwq\":\"nfqqnvwp\",\"ervnaenqpehi\":\"tjrybnwjewgdr\"},\"id\":\"doy\",\"name\":\"mifthnzdnd\",\"type\":\"l\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\"},\"location\":\"dgfbcvkcvqvpke\",\"tags\":{\"sotbob\":\"vdrhvoo\",\"ld\":\"dopcjwvnh\",\"twuoegrpkhjwni\":\"mgxcxrslpm\"},\"id\":\"qsluicp\",\"name\":\"ggkzzlvmbmpa\",\"type\":\"modfvuefywsbpfvm\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - IoTFirmwareDefenseManager manager = - IoTFirmwareDefenseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + IoTFirmwareDefenseManager manager = IoTFirmwareDefenseManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.workspaces().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("asrru", response.iterator().next().location()); - Assertions.assertEquals("hsqfsubcgjbirxbp", response.iterator().next().tags().get("bsrfbj")); + Assertions.assertEquals("dgfbcvkcvqvpke", response.iterator().next().location()); + Assertions.assertEquals("vdrhvoo", response.iterator().next().tags().get("sotbob")); } } diff --git a/sdk/iothub/azure-resourcemanager-iothub/pom.xml b/sdk/iothub/azure-resourcemanager-iothub/pom.xml index 65d2779ceed46..9d999f7705636 100644 --- a/sdk/iothub/azure-resourcemanager-iothub/pom.xml +++ b/sdk/iothub/azure-resourcemanager-iothub/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/iothub/tests.mgmt.yml b/sdk/iothub/tests.mgmt.yml index 49ef6ca11a4c6..9a35f4b5b2ada 100644 --- a/sdk/iothub/tests.mgmt.yml +++ b/sdk/iothub/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: iothub - Artifacts: - - name: azure-resourcemanager-iothub - groupId: com.azure.resourcemanager - safeName: azureresourcemanageriothub - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: iothub + Artifacts: + - name: azure-resourcemanager-iothub + groupId: com.azure.resourcemanager + safeName: azureresourcemanageriothub + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/keyvault/azure-security-keyvault-administration/CHANGELOG.md b/sdk/keyvault/azure-security-keyvault-administration/CHANGELOG.md index 01c0df8d8d2ba..0fb3f76bb31d8 100644 --- a/sdk/keyvault/azure-security-keyvault-administration/CHANGELOG.md +++ b/sdk/keyvault/azure-security-keyvault-administration/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 4.5.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 4.5.0 (2024-02-22) Changes when compared to the last stable release (`4.4.3`) include: diff --git a/sdk/keyvault/azure-security-keyvault-administration/pom.xml b/sdk/keyvault/azure-security-keyvault-administration/pom.xml index 6b252422ecde1..7cc8340b504c8 100644 --- a/sdk/keyvault/azure-security-keyvault-administration/pom.xml +++ b/sdk/keyvault/azure-security-keyvault-administration/pom.xml @@ -88,7 +88,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -106,19 +106,19 @@ com.azure azure-security-keyvault-keys - 4.8.0 + 4.9.0-beta.1 test com.azure azure-storage-blob - 12.25.2 + 12.25.3 test com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/keyvault/azure-security-keyvault-certificates/CHANGELOG.md b/sdk/keyvault/azure-security-keyvault-certificates/CHANGELOG.md index e4378b51ac2ea..9d1c966788b9b 100644 --- a/sdk/keyvault/azure-security-keyvault-certificates/CHANGELOG.md +++ b/sdk/keyvault/azure-security-keyvault-certificates/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 4.6.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 4.6.0 (2024-02-22) Changes when compared to the last stable release (`4.5.9`) include: diff --git a/sdk/keyvault/azure-security-keyvault-certificates/pom.xml b/sdk/keyvault/azure-security-keyvault-certificates/pom.xml index 42d70ac075df6..c9a9907557599 100644 --- a/sdk/keyvault/azure-security-keyvault-certificates/pom.xml +++ b/sdk/keyvault/azure-security-keyvault-certificates/pom.xml @@ -83,7 +83,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -101,7 +101,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/keyvault/azure-security-keyvault-jca/pom.xml b/sdk/keyvault/azure-security-keyvault-jca/pom.xml index 328ef00a79a28..c2ae8e3bab750 100644 --- a/sdk/keyvault/azure-security-keyvault-jca/pom.xml +++ b/sdk/keyvault/azure-security-keyvault-jca/pom.xml @@ -67,13 +67,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/keyvault/azure-security-keyvault-keys/CHANGELOG.md b/sdk/keyvault/azure-security-keyvault-keys/CHANGELOG.md index c8e129a5a8fcd..f2bdc31b36925 100644 --- a/sdk/keyvault/azure-security-keyvault-keys/CHANGELOG.md +++ b/sdk/keyvault/azure-security-keyvault-keys/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 4.8.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 4.8.0 (2024-02-22) Changes when compared to the last stable release (`4.7.3`) include: diff --git a/sdk/keyvault/azure-security-keyvault-keys/pom.xml b/sdk/keyvault/azure-security-keyvault-keys/pom.xml index 9dc0194075cb6..0c0458d0b0ecb 100644 --- a/sdk/keyvault/azure-security-keyvault-keys/pom.xml +++ b/sdk/keyvault/azure-security-keyvault-keys/pom.xml @@ -87,7 +87,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -105,7 +105,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -113,13 +113,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/keyvault/azure-security-keyvault-perf/pom.xml b/sdk/keyvault/azure-security-keyvault-perf/pom.xml index 108e1be76ed91..ca8d9db65a656 100644 --- a/sdk/keyvault/azure-security-keyvault-perf/pom.xml +++ b/sdk/keyvault/azure-security-keyvault-perf/pom.xml @@ -41,7 +41,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 com.azure @@ -126,7 +126,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 diff --git a/sdk/keyvault/azure-security-keyvault-secrets/CHANGELOG.md b/sdk/keyvault/azure-security-keyvault-secrets/CHANGELOG.md index af0c68dc5fe07..cf7aab890fba7 100644 --- a/sdk/keyvault/azure-security-keyvault-secrets/CHANGELOG.md +++ b/sdk/keyvault/azure-security-keyvault-secrets/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 4.8.1 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 4.8.0 (2024-02-22) Changes when compared to the last stable release (`4.7.3`) include: diff --git a/sdk/keyvault/azure-security-keyvault-secrets/pom.xml b/sdk/keyvault/azure-security-keyvault-secrets/pom.xml index 46b68c8c9bb23..af2e1baee6438 100644 --- a/sdk/keyvault/azure-security-keyvault-secrets/pom.xml +++ b/sdk/keyvault/azure-security-keyvault-secrets/pom.xml @@ -96,7 +96,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -110,7 +110,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/keyvault/azure-security-test-keyvault-jca/pom.xml b/sdk/keyvault/azure-security-test-keyvault-jca/pom.xml index cb364dd08d266..c542b165e3d67 100644 --- a/sdk/keyvault/azure-security-test-keyvault-jca/pom.xml +++ b/sdk/keyvault/azure-security-test-keyvault-jca/pom.xml @@ -76,7 +76,7 @@ org.springframework spring-core - 5.3.31 + 5.3.32 test diff --git a/sdk/keyvault/ci.yml b/sdk/keyvault/ci.yml index 4a9b9f53dafcb..d546c87ee1426 100644 --- a/sdk/keyvault/ci.yml +++ b/sdk/keyvault/ci.yml @@ -110,3 +110,5 @@ extends: # required by the above perf library - name: perf-test-core groupId: com.azure + - name: azure-aot-graalvm-perf + groupId: com.azure diff --git a/sdk/keyvault/microsoft-azure-keyvault-core/pom.xml b/sdk/keyvault/microsoft-azure-keyvault-core/pom.xml index 57adaccd1c8de..6dc9934c906ce 100644 --- a/sdk/keyvault/microsoft-azure-keyvault-core/pom.xml +++ b/sdk/keyvault/microsoft-azure-keyvault-core/pom.xml @@ -51,7 +51,7 @@ com.google.guava guava - 33.0.0-jre + 33.1.0-jre diff --git a/sdk/keyvault/microsoft-azure-keyvault-cryptography/pom.xml b/sdk/keyvault/microsoft-azure-keyvault-cryptography/pom.xml index c667fffa4c0ff..0c7749c96b1b2 100644 --- a/sdk/keyvault/microsoft-azure-keyvault-cryptography/pom.xml +++ b/sdk/keyvault/microsoft-azure-keyvault-cryptography/pom.xml @@ -76,7 +76,7 @@ com.google.guava guava - 33.0.0-jre + 33.1.0-jre diff --git a/sdk/keyvault/microsoft-azure-keyvault-extensions/pom.xml b/sdk/keyvault/microsoft-azure-keyvault-extensions/pom.xml index 2c4a60cddb85e..c8bef9a52949e 100644 --- a/sdk/keyvault/microsoft-azure-keyvault-extensions/pom.xml +++ b/sdk/keyvault/microsoft-azure-keyvault-extensions/pom.xml @@ -83,7 +83,7 @@ com.google.guava guava - 33.0.0-jre + 33.1.0-jre org.apache.commons diff --git a/sdk/keyvault/microsoft-azure-keyvault-webkey/pom.xml b/sdk/keyvault/microsoft-azure-keyvault-webkey/pom.xml index b101ffb66b4d5..17df774b3a277 100644 --- a/sdk/keyvault/microsoft-azure-keyvault-webkey/pom.xml +++ b/sdk/keyvault/microsoft-azure-keyvault-webkey/pom.xml @@ -67,7 +67,7 @@ com.google.guava guava - 33.0.0-jre + 33.1.0-jre diff --git a/sdk/keyvault/platform-matrix.json b/sdk/keyvault/platform-matrix.json index dec96e6956065..6fbf3f7ee4b4a 100644 --- a/sdk/keyvault/platform-matrix.json +++ b/sdk/keyvault/platform-matrix.json @@ -20,8 +20,8 @@ { "Agent": { "ubuntu-20.04_TestFromSource": { - "OSVmImage": "MMSUbuntu20.04", - "Pool": "azsdk-pool-mms-ubuntu-2004-general", + "OSVmImage": "env:LINUXVMIMAGE", + "Pool": "env:LINUXPOOL", "TestGoals": "surefire:test", "TestFromSource": true } @@ -34,8 +34,8 @@ { "Agent": { "ubuntu-20.04_TestFromSource": { - "OSVmImage": "MMSUbuntu20.04", - "Pool": "azsdk-pool-mms-ubuntu-2004-general", + "OSVmImage": "env:LINUXVMIMAGE", + "Pool": "env:LINUXPOOL", "TestGoals": "surefire:test", "TestFromSource": true } diff --git a/sdk/keyvault/tests.yml b/sdk/keyvault/tests.yml index eadd2585abe7c..673d1d7de23b3 100644 --- a/sdk/keyvault/tests.yml +++ b/sdk/keyvault/tests.yml @@ -1,58 +1,58 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: keyvault - TimeoutInMinutes: 240 - SupportedClouds: 'Public,UsGov,China,Canary' - EnvVars: - AZURE_LOG_LEVEL: 2 - CloudConfig: - Public: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - ${{ if not(contains(variables['Build.DefinitionName'], 'tests-weekly')) }}: - MatrixFilters: - - ArmTemplateParameters=^(?!.*enableHsm.*true) - UsGov: - SubscriptionConfiguration: $(sub-config-gov-test-resources) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: keyvault + TimeoutInMinutes: 240 + SupportedClouds: 'Public,UsGov,China,Canary' + EnvVars: + AZURE_LOG_LEVEL: 2 + CloudConfig: + Public: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + ${{ if not(contains(variables['Build.DefinitionName'], 'tests-weekly')) }}: MatrixFilters: - ArmTemplateParameters=^(?!.*enableHsm.*true) - China: - SubscriptionConfiguration: $(sub-config-cn-test-resources) - MatrixFilters: - - ArmTemplateParameters=^(?!.*enableHsm.*true) - Canary: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) - Location: 'centraluseuap' - # Managed HSM test resources are expensive and provisioning has not been reliable. - # Given test coverage of non-canary regions we probably don't need to test in canary. - MatrixFilters: - - ArmTemplateParameters=^(?!.*enableHsm.*true) - # Some resource providers required for attestation are not supported in canary. - MatrixReplace: - - 'ArmTemplateParameters=(.*)enableAttestation.*?\$true(.*)/$1enableAttestation \= $false$2' - MatrixConfigs: - - Name: Key_Vault_live_test - Path: sdk/keyvault/platform-matrix.json - Selection: sparse - GenerateVMJobs: true - Artifacts: - - name: azure-security-keyvault-administration - groupId: com.azure - safeName: azuresecuritykeyvaultadministration - - name: azure-security-keyvault-certificates - groupId: com.azure - safeName: azuresecuritykeyvaultcertificates - - name: azure-security-keyvault-keys - groupId: com.azure - safeName: azuresecuritykeyvaultkeys - - name: azure-security-keyvault-secrets - groupId: com.azure - safeName: azuresecuritykeyvaultsecrets - - name: azure-security-test-keyvault-jca - groupId: com.azure - safeName: azuresecuritytestkeyvaultjca - - name: azure-security-keyvault-jca - groupId: com.azure - safeName: azuresecuritykeyvaultjca + UsGov: + SubscriptionConfiguration: $(sub-config-gov-test-resources) + MatrixFilters: + - ArmTemplateParameters=^(?!.*enableHsm.*true) + China: + SubscriptionConfiguration: $(sub-config-cn-test-resources) + MatrixFilters: + - ArmTemplateParameters=^(?!.*enableHsm.*true) + Canary: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources) + Location: 'centraluseuap' + # Managed HSM test resources are expensive and provisioning has not been reliable. + # Given test coverage of non-canary regions we probably don't need to test in canary. + MatrixFilters: + - ArmTemplateParameters=^(?!.*enableHsm.*true) + # Some resource providers required for attestation are not supported in canary. + MatrixReplace: + - 'ArmTemplateParameters=(.*)enableAttestation.*?\$true(.*)/$1enableAttestation \= $false$2' + MatrixConfigs: + - Name: Key_Vault_live_test + Path: sdk/keyvault/platform-matrix.json + Selection: sparse + GenerateVMJobs: true + Artifacts: + - name: azure-security-keyvault-administration + groupId: com.azure + safeName: azuresecuritykeyvaultadministration + - name: azure-security-keyvault-certificates + groupId: com.azure + safeName: azuresecuritykeyvaultcertificates + - name: azure-security-keyvault-keys + groupId: com.azure + safeName: azuresecuritykeyvaultkeys + - name: azure-security-keyvault-secrets + groupId: com.azure + safeName: azuresecuritykeyvaultsecrets + - name: azure-security-test-keyvault-jca + groupId: com.azure + safeName: azuresecuritytestkeyvaultjca + - name: azure-security-keyvault-jca + groupId: com.azure + safeName: azuresecuritykeyvaultjca diff --git a/sdk/kubernetesconfiguration/azure-resourcemanager-kubernetesconfiguration/pom.xml b/sdk/kubernetesconfiguration/azure-resourcemanager-kubernetesconfiguration/pom.xml index 8b91f4ae878ee..8f9bc5748c9ee 100644 --- a/sdk/kubernetesconfiguration/azure-resourcemanager-kubernetesconfiguration/pom.xml +++ b/sdk/kubernetesconfiguration/azure-resourcemanager-kubernetesconfiguration/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/kusto/azure-resourcemanager-kusto/pom.xml b/sdk/kusto/azure-resourcemanager-kusto/pom.xml index 45a18d864d0bf..3110f7e953719 100644 --- a/sdk/kusto/azure-resourcemanager-kusto/pom.xml +++ b/sdk/kusto/azure-resourcemanager-kusto/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/kusto/tests.mgmt.yml b/sdk/kusto/tests.mgmt.yml index 6b8ab4a587024..ff142b6234dae 100644 --- a/sdk/kusto/tests.mgmt.yml +++ b/sdk/kusto/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: kusto - Artifacts: - - name: azure-resourcemanager-kusto - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerkusto - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: kusto + Artifacts: + - name: azure-resourcemanager-kusto + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerkusto + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/largeinstance/azure-resourcemanager-largeinstance/pom.xml b/sdk/largeinstance/azure-resourcemanager-largeinstance/pom.xml index 2b168daf3f17a..858a3d1902444 100644 --- a/sdk/largeinstance/azure-resourcemanager-largeinstance/pom.xml +++ b/sdk/largeinstance/azure-resourcemanager-largeinstance/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/loadtesting/azure-developer-loadtesting/CHANGELOG.md b/sdk/loadtesting/azure-developer-loadtesting/CHANGELOG.md index a1a9e227b26f8..942276d9b22b4 100644 --- a/sdk/loadtesting/azure-developer-loadtesting/CHANGELOG.md +++ b/sdk/loadtesting/azure-developer-loadtesting/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.0.11 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.0.10 (2024-02-20) ### Other Changes diff --git a/sdk/loadtesting/azure-developer-loadtesting/pom.xml b/sdk/loadtesting/azure-developer-loadtesting/pom.xml index ddb68e8a2a3d8..591acd9e52d54 100644 --- a/sdk/loadtesting/azure-developer-loadtesting/pom.xml +++ b/sdk/loadtesting/azure-developer-loadtesting/pom.xml @@ -65,7 +65,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/loadtesting/azure-resourcemanager-loadtesting/pom.xml b/sdk/loadtesting/azure-resourcemanager-loadtesting/pom.xml index 2cebfa0300d59..26100c91d80b0 100644 --- a/sdk/loadtesting/azure-resourcemanager-loadtesting/pom.xml +++ b/sdk/loadtesting/azure-resourcemanager-loadtesting/pom.xml @@ -61,13 +61,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/loadtesting/tests.mgmt.yml b/sdk/loadtesting/tests.mgmt.yml index 69b429b01f1e1..7133a120350c1 100644 --- a/sdk/loadtesting/tests.mgmt.yml +++ b/sdk/loadtesting/tests.mgmt.yml @@ -2,12 +2,12 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: loadtesting - Artifacts: - - name: azure-resourcemanager-loadtesting - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerloadtesting - Clouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: loadtesting + Artifacts: + - name: azure-resourcemanager-loadtesting + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerloadtesting + Clouds: 'Public' diff --git a/sdk/loganalytics/azure-resourcemanager-loganalytics/pom.xml b/sdk/loganalytics/azure-resourcemanager-loganalytics/pom.xml index ac1e088bc069f..f5906373ac891 100644 --- a/sdk/loganalytics/azure-resourcemanager-loganalytics/pom.xml +++ b/sdk/loganalytics/azure-resourcemanager-loganalytics/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/loganalytics/tests.mgmt.yml b/sdk/loganalytics/tests.mgmt.yml index bd19c384f431c..d37ca9270580e 100644 --- a/sdk/loganalytics/tests.mgmt.yml +++ b/sdk/loganalytics/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: loganalytics - Artifacts: - - name: azure-resourcemanager-loganalytics - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerloganalytics - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: loganalytics + Artifacts: + - name: azure-resourcemanager-loganalytics + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerloganalytics + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/logic/azure-resourcemanager-logic/pom.xml b/sdk/logic/azure-resourcemanager-logic/pom.xml index f50aae5abbe5e..859519e75f86a 100644 --- a/sdk/logic/azure-resourcemanager-logic/pom.xml +++ b/sdk/logic/azure-resourcemanager-logic/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/logz/azure-resourcemanager-logz/pom.xml b/sdk/logz/azure-resourcemanager-logz/pom.xml index fe8d4f157c330..3436a67a464b8 100644 --- a/sdk/logz/azure-resourcemanager-logz/pom.xml +++ b/sdk/logz/azure-resourcemanager-logz/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/machinelearning/azure-resourcemanager-machinelearning/pom.xml b/sdk/machinelearning/azure-resourcemanager-machinelearning/pom.xml index 0f7cb90ef099a..920aacfb490fb 100644 --- a/sdk/machinelearning/azure-resourcemanager-machinelearning/pom.xml +++ b/sdk/machinelearning/azure-resourcemanager-machinelearning/pom.xml @@ -54,13 +54,13 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 test com.azure.resourcemanager azure-resourcemanager-keyvault - 2.36.0 + 2.37.0 test @@ -72,7 +72,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -84,7 +84,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -110,13 +110,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/machinelearning/tests.mgmt.yml b/sdk/machinelearning/tests.mgmt.yml index a09b4631739f2..99305c40144a8 100644 --- a/sdk/machinelearning/tests.mgmt.yml +++ b/sdk/machinelearning/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: machinelearning - Artifacts: - - name: azure-resourcemanager-machinelearning - groupId: com.azure.resourcemanager - safeName: azureresourcemanagermachinelearning - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: machinelearning + Artifacts: + - name: azure-resourcemanager-machinelearning + groupId: com.azure.resourcemanager + safeName: azureresourcemanagermachinelearning + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/maintenance/azure-resourcemanager-maintenance/pom.xml b/sdk/maintenance/azure-resourcemanager-maintenance/pom.xml index d02b8a8159b3e..b416d950b0efa 100644 --- a/sdk/maintenance/azure-resourcemanager-maintenance/pom.xml +++ b/sdk/maintenance/azure-resourcemanager-maintenance/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/managedapplications/azure-resourcemanager-managedapplications/pom.xml b/sdk/managedapplications/azure-resourcemanager-managedapplications/pom.xml index d18edb4ea600f..8a1c772856815 100644 --- a/sdk/managedapplications/azure-resourcemanager-managedapplications/pom.xml +++ b/sdk/managedapplications/azure-resourcemanager-managedapplications/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/managednetworkfabric/azure-resourcemanager-managednetworkfabric/pom.xml b/sdk/managednetworkfabric/azure-resourcemanager-managednetworkfabric/pom.xml index 207b1875f5527..cf24a26735415 100644 --- a/sdk/managednetworkfabric/azure-resourcemanager-managednetworkfabric/pom.xml +++ b/sdk/managednetworkfabric/azure-resourcemanager-managednetworkfabric/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/managednetworkfabric/tests.mgmt.yml b/sdk/managednetworkfabric/tests.mgmt.yml index 2d5fd1d9b76a6..fd2e81561a07f 100644 --- a/sdk/managednetworkfabric/tests.mgmt.yml +++ b/sdk/managednetworkfabric/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: managednetworkfabric - Artifacts: - - name: azure-resourcemanager-managednetworkfabric - groupId: com.azure.resourcemanager - safeName: azureresourcemanagermanagednetworkfabric - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: managednetworkfabric + Artifacts: + - name: azure-resourcemanager-managednetworkfabric + groupId: com.azure.resourcemanager + safeName: azureresourcemanagermanagednetworkfabric + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/managementgroups/azure-resourcemanager-managementgroups/pom.xml b/sdk/managementgroups/azure-resourcemanager-managementgroups/pom.xml index 7798a51b35e66..b9f23716448ee 100644 --- a/sdk/managementgroups/azure-resourcemanager-managementgroups/pom.xml +++ b/sdk/managementgroups/azure-resourcemanager-managementgroups/pom.xml @@ -69,7 +69,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -95,13 +95,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/maps/azure-maps-elevation/pom.xml b/sdk/maps/azure-maps-elevation/pom.xml index d2b13150abe33..9848b47c7e36d 100644 --- a/sdk/maps/azure-maps-elevation/pom.xml +++ b/sdk/maps/azure-maps-elevation/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -97,7 +97,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-elevation/tests.yml b/sdk/maps/azure-maps-elevation/tests.yml index 8deb3721c7238..8fb4a7a76c8a4 100644 --- a/sdk/maps/azure-maps-elevation/tests.yml +++ b/sdk/maps/azure-maps-elevation/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-elevation - SafeName: azuremapselevation +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-elevation + SafeName: azuremapselevation diff --git a/sdk/maps/azure-maps-geolocation/pom.xml b/sdk/maps/azure-maps-geolocation/pom.xml index 197685f43c77b..23b09858ac127 100644 --- a/sdk/maps/azure-maps-geolocation/pom.xml +++ b/sdk/maps/azure-maps-geolocation/pom.xml @@ -71,7 +71,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -101,7 +101,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-geolocation/tests.yml b/sdk/maps/azure-maps-geolocation/tests.yml index a4f716b2ae46d..af2595761bc7d 100644 --- a/sdk/maps/azure-maps-geolocation/tests.yml +++ b/sdk/maps/azure-maps-geolocation/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-geolocation - SafeName: azuremapsgeolocation +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-geolocation + SafeName: azuremapsgeolocation diff --git a/sdk/maps/azure-maps-render/pom.xml b/sdk/maps/azure-maps-render/pom.xml index c6c75c4a8530d..3d4e7164a1bd7 100644 --- a/sdk/maps/azure-maps-render/pom.xml +++ b/sdk/maps/azure-maps-render/pom.xml @@ -80,7 +80,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -110,7 +110,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-render/tests.yml b/sdk/maps/azure-maps-render/tests.yml index ffa6e233a87c1..042abef2335c8 100644 --- a/sdk/maps/azure-maps-render/tests.yml +++ b/sdk/maps/azure-maps-render/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-render - SafeName: azuremapsrender +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-render + SafeName: azuremapsrender diff --git a/sdk/maps/azure-maps-route/pom.xml b/sdk/maps/azure-maps-route/pom.xml index 71c46f4604bad..219a1f1330409 100644 --- a/sdk/maps/azure-maps-route/pom.xml +++ b/sdk/maps/azure-maps-route/pom.xml @@ -80,7 +80,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -116,7 +116,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-route/tests.yml b/sdk/maps/azure-maps-route/tests.yml index 2317bdc0f3915..4c6fdfce50ec8 100644 --- a/sdk/maps/azure-maps-route/tests.yml +++ b/sdk/maps/azure-maps-route/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-route - SafeName: azuremapsroute +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-route + SafeName: azuremapsroute diff --git a/sdk/maps/azure-maps-search/pom.xml b/sdk/maps/azure-maps-search/pom.xml index 8d2ac98f98ccc..9bd1b2edd105d 100644 --- a/sdk/maps/azure-maps-search/pom.xml +++ b/sdk/maps/azure-maps-search/pom.xml @@ -81,7 +81,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -111,7 +111,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-search/tests.yml b/sdk/maps/azure-maps-search/tests.yml index 49a7f7e6a1df0..0c88d001324e1 100644 --- a/sdk/maps/azure-maps-search/tests.yml +++ b/sdk/maps/azure-maps-search/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-search - SafeName: azuremapssearch +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-search + SafeName: azuremapssearch diff --git a/sdk/maps/azure-maps-timezone/pom.xml b/sdk/maps/azure-maps-timezone/pom.xml index 72899d6bb6ccb..4c4b5e2eaaecf 100644 --- a/sdk/maps/azure-maps-timezone/pom.xml +++ b/sdk/maps/azure-maps-timezone/pom.xml @@ -77,7 +77,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -107,7 +107,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-timezone/tests.yml b/sdk/maps/azure-maps-timezone/tests.yml index 0aba0d781bdb1..b0b8d050fc9f3 100644 --- a/sdk/maps/azure-maps-timezone/tests.yml +++ b/sdk/maps/azure-maps-timezone/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-timezone - SafeName: azuremapstimezone +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-timezone + SafeName: azuremapstimezone diff --git a/sdk/maps/azure-maps-traffic/pom.xml b/sdk/maps/azure-maps-traffic/pom.xml index bc62215347b4b..177c33248fd59 100644 --- a/sdk/maps/azure-maps-traffic/pom.xml +++ b/sdk/maps/azure-maps-traffic/pom.xml @@ -68,7 +68,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -98,7 +98,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-traffic/tests.yml b/sdk/maps/azure-maps-traffic/tests.yml index 23cc68f9004ab..89d51f32db669 100644 --- a/sdk/maps/azure-maps-traffic/tests.yml +++ b/sdk/maps/azure-maps-traffic/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-traffic - SafeName: azuremapstraffic +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-traffic + SafeName: azuremapstraffic diff --git a/sdk/maps/azure-maps-weather/pom.xml b/sdk/maps/azure-maps-weather/pom.xml index a363067ff3d51..d46fec7ee3bcc 100644 --- a/sdk/maps/azure-maps-weather/pom.xml +++ b/sdk/maps/azure-maps-weather/pom.xml @@ -78,7 +78,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -108,7 +108,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/maps/azure-maps-weather/tests.yml b/sdk/maps/azure-maps-weather/tests.yml index e1cb420ac1a2d..b8c7e387037f1 100644 --- a/sdk/maps/azure-maps-weather/tests.yml +++ b/sdk/maps/azure-maps-weather/tests.yml @@ -1,7 +1,7 @@ trigger: none -stages: - - template: /sdk/maps/maps-tests-template.yml - parameters: - PackageName: azure-maps-weather - SafeName: azuremapsweather +extends: + template: /sdk/maps/maps-tests-template.yml + parameters: + PackageName: azure-maps-weather + SafeName: azuremapsweather diff --git a/sdk/maps/azure-resourcemanager-maps/pom.xml b/sdk/maps/azure-resourcemanager-maps/pom.xml index ef6c21b26eef6..e62037412facf 100644 --- a/sdk/maps/azure-resourcemanager-maps/pom.xml +++ b/sdk/maps/azure-resourcemanager-maps/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/maps/maps-tests-template.yml b/sdk/maps/maps-tests-template.yml index 853ebbf9eb5e5..3120a262b197e 100644 --- a/sdk/maps/maps-tests-template.yml +++ b/sdk/maps/maps-tests-template.yml @@ -10,24 +10,24 @@ parameters: Selection: sparse GenerateVMJobs: true -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - CloudConfig: - Public: - SubscriptionConfigurations: - - $(sub-config-azure-cloud-test-resources) - Clouds: ${{ parameters.Clouds }} - TestMode: ${{ parameters.TestMode }} - Artifacts: - - name: ${{ parameters.PackageName }} - groupId: com.azure - safeName: ${{ parameters.SafeName }} - ServiceDirectory: maps - EnvVars: - #SKIP_LIVE_TEST: TRUE - ${{ each var in parameters.EnVars }}: - ${{ var.key }}: ${{ var.value }} - MatrixConfigs: - - ${{ each config in parameters.MatrixConfigs }}: - - ${{ config }} +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + CloudConfig: + Public: + SubscriptionConfigurations: + - $(sub-config-azure-cloud-test-resources) + Clouds: ${{ parameters.Clouds }} + TestMode: ${{ parameters.TestMode }} + Artifacts: + - name: ${{ parameters.PackageName }} + groupId: com.azure + safeName: ${{ parameters.SafeName }} + ServiceDirectory: maps + EnvVars: + #SKIP_LIVE_TEST: TRUE + ${{ each var in parameters.EnVars }}: + ${{ var.key }}: ${{ var.value }} + MatrixConfigs: + - ${{ each config in parameters.MatrixConfigs }}: + - ${{ config }} diff --git a/sdk/mariadb/azure-resourcemanager-mariadb/pom.xml b/sdk/mariadb/azure-resourcemanager-mariadb/pom.xml index 8bcbc23130f77..ed1d0883d6450 100644 --- a/sdk/mariadb/azure-resourcemanager-mariadb/pom.xml +++ b/sdk/mariadb/azure-resourcemanager-mariadb/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/mediaservices/azure-resourcemanager-mediaservices/pom.xml b/sdk/mediaservices/azure-resourcemanager-mediaservices/pom.xml index f251ff66068fb..4a33ce06a1506 100644 --- a/sdk/mediaservices/azure-resourcemanager-mediaservices/pom.xml +++ b/sdk/mediaservices/azure-resourcemanager-mediaservices/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 test diff --git a/sdk/mediaservices/tests.mgmt.yml b/sdk/mediaservices/tests.mgmt.yml index 395bfe77e57f9..5f20007f51f38 100644 --- a/sdk/mediaservices/tests.mgmt.yml +++ b/sdk/mediaservices/tests.mgmt.yml @@ -2,12 +2,12 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: mediaservices - Artifacts: - - name: azure-resourcemanager-mediaservices - groupId: com.azure.resourcemanager - safeName: azureresourcemanagermediaservices - Clouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: mediaservices + Artifacts: + - name: azure-resourcemanager-mediaservices + groupId: com.azure.resourcemanager + safeName: azureresourcemanagermediaservices + Clouds: 'Public' diff --git a/sdk/metricsadvisor/azure-ai-metricsadvisor/CHANGELOG.md b/sdk/metricsadvisor/azure-ai-metricsadvisor/CHANGELOG.md index 011f0e5675075..0aa955648e8cb 100644 --- a/sdk/metricsadvisor/azure-ai-metricsadvisor/CHANGELOG.md +++ b/sdk/metricsadvisor/azure-ai-metricsadvisor/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.1.23 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.1.22 (2024-02-20) ### Other Changes diff --git a/sdk/metricsadvisor/azure-ai-metricsadvisor/pom.xml b/sdk/metricsadvisor/azure-ai-metricsadvisor/pom.xml index d7d52eb7eb8f2..a67f389a77c6e 100644 --- a/sdk/metricsadvisor/azure-ai-metricsadvisor/pom.xml +++ b/sdk/metricsadvisor/azure-ai-metricsadvisor/pom.xml @@ -92,7 +92,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/metricsadvisor/tests.yml b/sdk/metricsadvisor/tests.yml index d27e25df4f9a0..5b79b59a23452 100644 --- a/sdk/metricsadvisor/tests.yml +++ b/sdk/metricsadvisor/tests.yml @@ -1,35 +1,35 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: metricsadvisor - Artifacts: - - name: azure-ai-metricsadvisor - groupId: com.azure - safeName: azureaimetricsadvisor - EnvVars: - AZURE_TENANT_ID: $(aad-azure-sdk-test-tenant-id) - AZURE_CLIENT_ID: $(aad-azure-sdk-test-client-id) - AZURE_CLIENT_SECRET: $(aad-azure-sdk-test-client-secret) - AZURE_METRICS_ADVISOR_ENDPOINT: $(metricsadvisor-test-service-endpoint) - AZURE_METRICS_ADVISOR_SUBSCRIPTION_KEY: $(metricsadvisor-test-subscription-key) - AZURE_METRICS_ADVISOR_API_KEY: $(java-metricsadvisor-test-api-key) - AZURE_METRICS_ADVISOR_SQL_SERVER_CONNECTION_STRING: $(metricsadvisor-test-sql-server-connection-string) - AZURE_METRICS_ADVISOR_AZURE_TABLE_CONNECTION_STRING: $(metricsadvisor-test-azure-table-connection-string) - AZURE_METRICS_ADVISOR_AZURE_BLOB_CONNECTION_STRING: $(metricsadvisor-test-azure-blob-connection-string) - AZURE_METRICS_ADVISOR_COSMOS_DB_CONNECTION_STRING: $(metricsadvisor-test-cosmos-db-connection-string) - AZURE_METRICS_ADVISOR_HTTP_GET_URL: $(metricsadvisor-test-http-get-url) - AZURE_METRICS_ADVISOR_HTTP_POST_URL: $(metricsadvisor-test-http-post-url) - AZURE_METRICS_ADVISOR_APPLICATION_INSIGHTS_API_KEY: $(metricsadvisor-test-application-insights-api-key) - AZURE_METRICS_ADVISOR_APPLICATION_INSIGHTS_APPLICATION_ID: $(metricsadvisor-test-application-insights-application-id) - AZURE_METRICS_ADVISOR_AZURE_DATA_EXPLORER_CONNECTION_STRING: $(metricsadvisor-test-azure-data-explorer-connection-string) - AZURE_METRICS_ADVISOR_INFLUX_DB_CONNECTION_STRING: $(metricsadvisor-test-influx-db-connection-string) - AZURE_METRICS_ADVISOR_INFLUX_DB_PASSWORD: $(metricsadvisor-test-influx-db-password) - AZURE_METRICS_ADVISOR_AZURE_DATALAKE_ACCOUNT_KEY: $(metricsadvisor-test-azure-datalake-account-key) - AZURE_METRICS_ADVISOR_AZURE_MONGODB_CONNECTION_STRING: $(metricsadvisor-test-azure-mongodb-connection-string) - AZURE_METRICS_ADVISOR_MYSQL_CONNECTION_STRING: $(metricsadvisor-test-mysql-connection-string) - AZURE_METRICS_ADVISOR_POSTGRESQL_CONNECTION_STRING: $(metricsadvisor-test-postgresql-connection-string) - AZURE_METRICS_ADVISOR_LOG_ANALYTICS_WORKSPACE_ID: $(metricsadvisor-test-loganalytics-workspace-id) - AZURE_METRICS_ADVISOR_LOG_ANALYTICS_CLIENT_ID: $(metricsadvisor-test-loganalytics-client-id) - AZURE_METRICS_ADVISOR_LOG_ANALYTICS_CLIENT_SECRET: $(metricsadvisor-test-loganalytics-client-secret) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: metricsadvisor + Artifacts: + - name: azure-ai-metricsadvisor + groupId: com.azure + safeName: azureaimetricsadvisor + EnvVars: + AZURE_TENANT_ID: $(aad-azure-sdk-test-tenant-id) + AZURE_CLIENT_ID: $(aad-azure-sdk-test-client-id) + AZURE_CLIENT_SECRET: $(aad-azure-sdk-test-client-secret) + AZURE_METRICS_ADVISOR_ENDPOINT: $(metricsadvisor-test-service-endpoint) + AZURE_METRICS_ADVISOR_SUBSCRIPTION_KEY: $(metricsadvisor-test-subscription-key) + AZURE_METRICS_ADVISOR_API_KEY: $(java-metricsadvisor-test-api-key) + AZURE_METRICS_ADVISOR_SQL_SERVER_CONNECTION_STRING: $(metricsadvisor-test-sql-server-connection-string) + AZURE_METRICS_ADVISOR_AZURE_TABLE_CONNECTION_STRING: $(metricsadvisor-test-azure-table-connection-string) + AZURE_METRICS_ADVISOR_AZURE_BLOB_CONNECTION_STRING: $(metricsadvisor-test-azure-blob-connection-string) + AZURE_METRICS_ADVISOR_COSMOS_DB_CONNECTION_STRING: $(metricsadvisor-test-cosmos-db-connection-string) + AZURE_METRICS_ADVISOR_HTTP_GET_URL: $(metricsadvisor-test-http-get-url) + AZURE_METRICS_ADVISOR_HTTP_POST_URL: $(metricsadvisor-test-http-post-url) + AZURE_METRICS_ADVISOR_APPLICATION_INSIGHTS_API_KEY: $(metricsadvisor-test-application-insights-api-key) + AZURE_METRICS_ADVISOR_APPLICATION_INSIGHTS_APPLICATION_ID: $(metricsadvisor-test-application-insights-application-id) + AZURE_METRICS_ADVISOR_AZURE_DATA_EXPLORER_CONNECTION_STRING: $(metricsadvisor-test-azure-data-explorer-connection-string) + AZURE_METRICS_ADVISOR_INFLUX_DB_CONNECTION_STRING: $(metricsadvisor-test-influx-db-connection-string) + AZURE_METRICS_ADVISOR_INFLUX_DB_PASSWORD: $(metricsadvisor-test-influx-db-password) + AZURE_METRICS_ADVISOR_AZURE_DATALAKE_ACCOUNT_KEY: $(metricsadvisor-test-azure-datalake-account-key) + AZURE_METRICS_ADVISOR_AZURE_MONGODB_CONNECTION_STRING: $(metricsadvisor-test-azure-mongodb-connection-string) + AZURE_METRICS_ADVISOR_MYSQL_CONNECTION_STRING: $(metricsadvisor-test-mysql-connection-string) + AZURE_METRICS_ADVISOR_POSTGRESQL_CONNECTION_STRING: $(metricsadvisor-test-postgresql-connection-string) + AZURE_METRICS_ADVISOR_LOG_ANALYTICS_WORKSPACE_ID: $(metricsadvisor-test-loganalytics-workspace-id) + AZURE_METRICS_ADVISOR_LOG_ANALYTICS_CLIENT_ID: $(metricsadvisor-test-loganalytics-client-id) + AZURE_METRICS_ADVISOR_LOG_ANALYTICS_CLIENT_SECRET: $(metricsadvisor-test-loganalytics-client-secret) diff --git a/sdk/mixedreality/azure-mixedreality-authentication/CHANGELOG.md b/sdk/mixedreality/azure-mixedreality-authentication/CHANGELOG.md index fac6c80d0e416..2ea9fdad1aaf2 100644 --- a/sdk/mixedreality/azure-mixedreality-authentication/CHANGELOG.md +++ b/sdk/mixedreality/azure-mixedreality-authentication/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.2.22 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.2.21 (2024-02-20) ### Other Changes diff --git a/sdk/mixedreality/azure-mixedreality-authentication/pom.xml b/sdk/mixedreality/azure-mixedreality-authentication/pom.xml index 8060abf26465d..2f45b30461680 100644 --- a/sdk/mixedreality/azure-mixedreality-authentication/pom.xml +++ b/sdk/mixedreality/azure-mixedreality-authentication/pom.xml @@ -82,13 +82,13 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/mixedreality/azure-resourcemanager-mixedreality/pom.xml b/sdk/mixedreality/azure-resourcemanager-mixedreality/pom.xml index edc5a9c770fce..3fbc096c95be3 100644 --- a/sdk/mixedreality/azure-resourcemanager-mixedreality/pom.xml +++ b/sdk/mixedreality/azure-resourcemanager-mixedreality/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/mixedreality/tests.yml b/sdk/mixedreality/tests.yml index 62a57c9c3a1cf..043dbfa9ecddf 100644 --- a/sdk/mixedreality/tests.yml +++ b/sdk/mixedreality/tests.yml @@ -1,11 +1,11 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: mixedreality - Location: eastus2 - Artifacts: - - name: azure-mixedreality-authentication - groupId: com.azure - safeName: azuremixedrealityauthentication +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: mixedreality + Location: eastus2 + Artifacts: + - name: azure-mixedreality-authentication + groupId: com.azure + safeName: azuremixedrealityauthentication diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/CHANGELOG.md b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/CHANGELOG.md index 7fa96b2b0266d..c4f9dd45f1068 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/CHANGELOG.md +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.1 (Unreleased) +## 1.2.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,105 @@ ### Other Changes +## 1.1.0 (2024-03-19) + +- Azure Resource Manager MobileNetwork client library for Java. This package contains Microsoft Azure SDK for MobileNetwork Management SDK. The resources in this API specification will be used to manage attached data network resources in mobile network attached to a particular packet core instance. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +* `models.GlobalRanNodeId` was added + +* `models.UeInformations` was added + +* `models.HomeNetworkPrivateKeysProvisioning` was added + +* `models.UeIpAddress` was added + +* `models.PublicLandMobileNetworkHomeNetworkPublicKeys` was added + +* `models.UeInfo` was added + +* `models.RatType` was added + +* `models.AmfId` was added + +* `models.HomeNetworkPublicKey` was added + +* `models.ExtendedUeInfoProperties` was added + +* `models.UeInfo4GProperties` was added + +* `models.UeUsageSetting` was added + +* `models.UeSessionInfo5G` was added + +* `models.MmeId` was added + +* `models.UeSessionInfo4G` was added + +* `models.Guti5G` was added + +* `models.GNbId` was added + +* `models.ExtendedUeInformations` was added + +* `models.Guti4G` was added + +* `models.UeConnectionInfo5G` was added + +* `models.UeInfoList` was added + +* `models.UeInfo4G` was added + +* `models.UeConnectionInfo4G` was added + +* `models.UeInfo5G` was added + +* `models.UeQosFlow` was added + +* `models.UeState` was added + +* `models.UeInfo5GProperties` was added + +* `models.PublicLandMobileNetwork` was added + +* `models.PdnType` was added + +* `models.UeInfoPropertiesFormat` was added + +* `models.ExtendedUeInfo` was added + +* `models.HomeNetworkPrivateKeysProvisioningState` was added + +* `models.UeLocationInfo` was added + +* `models.RrcEstablishmentCause` was added + +* `models.DnnIpPair` was added + +#### `models.PacketCoreControlPlane` was modified + +* `homeNetworkPrivateKeysProvisioning()` was added + +#### `models.MobileNetwork$Definition` was modified + +* `withPublicLandMobileNetworks(java.util.List)` was added +* `withIdentity(models.ManagedServiceIdentity)` was added + +#### `models.MobileNetwork$Update` was modified + +* `withIdentity(models.ManagedServiceIdentity)` was added + +#### `models.MobileNetwork` was modified + +* `publicLandMobileNetworks()` was added +* `identity()` was added + +#### `MobileNetworkManager` was modified + +* `ueInformations()` was added +* `extendedUeInformations()` was added + ## 1.0.0 (2023-11-15) - Azure Resource Manager MobileNetwork client library for Java. This package contains Microsoft Azure SDK for MobileNetwork Management SDK. The resources in this API specification will be used to manage attached data network resources in mobile network attached to a particular packet core instance. Package tag package-2023-09. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/README.md b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/README.md index ff507146efeb3..7346c3d3cb703 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/README.md +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/README.md @@ -2,7 +2,7 @@ Azure Resource Manager MobileNetwork client library for Java. -This package contains Microsoft Azure SDK for MobileNetwork Management SDK. The resources in this API specification will be used to manage attached data network resources in mobile network attached to a particular packet core instance. Package tag package-2023-09. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for MobileNetwork Management SDK. The resources in this API specification will be used to manage attached data network resources in mobile network attached to a particular packet core instance. Package tag package-2024-02. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-mobilenetwork - 1.0.0 + 1.1.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/SAMPLE.md b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/SAMPLE.md index 2598fee9eba69..37332948e8a7c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/SAMPLE.md +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/SAMPLE.md @@ -24,6 +24,10 @@ - [Get](#diagnosticspackages_get) - [ListByPacketCoreControlPlane](#diagnosticspackages_listbypacketcorecontrolplane) +## ExtendedUeInformation + +- [Get](#extendedueinformation_get) + ## MobileNetworks - [CreateOrUpdate](#mobilenetworks_createorupdate) @@ -123,6 +127,10 @@ - [Get](#slices_get) - [ListByMobileNetwork](#slices_listbymobilenetwork) - [UpdateTags](#slices_updatetags) + +## UeInformation + +- [List](#ueinformation_list) ### AttachedDataNetworks_CreateOrUpdate ```java @@ -134,34 +142,31 @@ import com.azure.resourcemanager.mobilenetwork.models.PortRange; import com.azure.resourcemanager.mobilenetwork.models.PortReuseHoldTimes; import java.util.Arrays; -/** Samples for AttachedDataNetworks CreateOrUpdate. */ +/** + * Samples for AttachedDataNetworks CreateOrUpdate. + */ public final class AttachedDataNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkCreate.json */ /** * Sample code: Create attached data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createAttachedDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .define("TestAttachedDataNetwork") - .withRegion("eastus") + manager.attachedDataNetworks().define("TestAttachedDataNetwork").withRegion("eastus") .withExistingPacketCoreDataPlane("rg1", "TestPacketCoreCP", "TestPacketCoreDP") .withUserPlaneDataInterface(new InterfaceProperties().withName("N6")) .withDnsAddresses(Arrays.asList("1.1.1.1")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.ENABLED) - .withPortRange(new PortRange().withMinPort(1024).withMaxPort(49999)) - .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(120).withUdp(60)) - .withPinholeLimits(65536) - .withPinholeTimeouts(new PinholeTimeouts().withTcp(180).withUdp(30).withIcmp(30))) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.ENABLED) + .withPortRange(new PortRange().withMinPort(1024).withMaxPort(49999)) + .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(120).withUdp(60)).withPinholeLimits(65536) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(180).withUdp(30).withIcmp(30))) .withUserEquipmentAddressPoolPrefix(Arrays.asList("2.2.0.0/16")) - .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("2.4.0.0/16")) - .create(); + .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("2.4.0.0/16")).create(); } } ``` @@ -169,26 +174,24 @@ public final class AttachedDataNetworksCreateOrUpdateSamples { ### AttachedDataNetworks_Delete ```java -/** Samples for AttachedDataNetworks Delete. */ +/** + * Samples for AttachedDataNetworks Delete. + */ public final class AttachedDataNetworksDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkDelete.json */ /** * Sample code: Delete attached data network resource. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void deleteAttachedDataNetworkResource( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .delete( - "rg1", - "TestPacketCoreCP", - "TestPacketCoreDP", - "TestAttachedDataNetwork", - com.azure.core.util.Context.NONE); + public static void + deleteAttachedDataNetworkResource(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.attachedDataNetworks().delete("rg1", "TestPacketCoreCP", "TestPacketCoreDP", "TestAttachedDataNetwork", + com.azure.core.util.Context.NONE); } } ``` @@ -196,25 +199,23 @@ public final class AttachedDataNetworksDeleteSamples { ### AttachedDataNetworks_Get ```java -/** Samples for AttachedDataNetworks Get. */ +/** + * Samples for AttachedDataNetworks Get. + */ public final class AttachedDataNetworksGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkGet.json */ /** * Sample code: Get attached data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getAttachedDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .getWithResponse( - "rg1", - "TestPacketCoreCP", - "TestPacketCoreDP", - "TestAttachedDataNetwork", - com.azure.core.util.Context.NONE); + manager.attachedDataNetworks().getWithResponse("rg1", "TestPacketCoreCP", "TestPacketCoreDP", + "TestAttachedDataNetwork", com.azure.core.util.Context.NONE); } } ``` @@ -222,21 +223,24 @@ public final class AttachedDataNetworksGetSamples { ### AttachedDataNetworks_ListByPacketCoreDataPlane ```java -/** Samples for AttachedDataNetworks ListByPacketCoreDataPlane. */ +/** + * Samples for AttachedDataNetworks ListByPacketCoreDataPlane. + */ public final class AttachedDataNetworksListByPacketCoreDataPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkListByPacketCoreDataPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkListByPacketCoreDataPlane.json */ /** * Sample code: List attached data networks in a data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listAttachedDataNetworksInADataPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .listByPacketCoreDataPlane("rg1", "TestPacketCoreCP", "TestPacketCoreDP", com.azure.core.util.Context.NONE); + public static void + listAttachedDataNetworksInADataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.attachedDataNetworks().listByPacketCoreDataPlane("rg1", "TestPacketCoreCP", "TestPacketCoreDP", + com.azure.core.util.Context.NONE); } } ``` @@ -248,28 +252,24 @@ import com.azure.resourcemanager.mobilenetwork.models.AttachedDataNetwork; import java.util.HashMap; import java.util.Map; -/** Samples for AttachedDataNetworks UpdateTags. */ +/** + * Samples for AttachedDataNetworks UpdateTags. + */ public final class AttachedDataNetworksUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkUpdateTags.json */ /** * Sample code: Update attached data network tags. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void updateAttachedDataNetworkTags( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - AttachedDataNetwork resource = - manager - .attachedDataNetworks() - .getWithResponse( - "rg1", - "TestPacketCoreCP", - "TestPacketCoreDP", - "TestAttachedDataNetwork", - com.azure.core.util.Context.NONE) - .getValue(); + public static void + updateAttachedDataNetworkTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + AttachedDataNetwork resource = manager.attachedDataNetworks().getWithResponse("rg1", "TestPacketCoreCP", + "TestPacketCoreDP", "TestAttachedDataNetwork", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -290,24 +290,23 @@ public final class AttachedDataNetworksUpdateTagsSamples { ### DataNetworks_CreateOrUpdate ```java -/** Samples for DataNetworks CreateOrUpdate. */ +/** + * Samples for DataNetworks CreateOrUpdate. + */ public final class DataNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/DataNetworkCreate + * .json */ /** * Sample code: Create data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .dataNetworks() - .define("testDataNetwork") - .withRegion("eastus") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .withDescription("myFavouriteDataNetwork") - .create(); + manager.dataNetworks().define("testDataNetwork").withRegion("eastus") + .withExistingMobileNetwork("rg1", "testMobileNetwork").withDescription("myFavouriteDataNetwork").create(); } } ``` @@ -315,14 +314,18 @@ public final class DataNetworksCreateOrUpdateSamples { ### DataNetworks_Delete ```java -/** Samples for DataNetworks Delete. */ +/** + * Samples for DataNetworks Delete. + */ public final class DataNetworksDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/DataNetworkDelete + * .json */ /** * Sample code: Delete data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -334,20 +337,23 @@ public final class DataNetworksDeleteSamples { ### DataNetworks_Get ```java -/** Samples for DataNetworks Get. */ +/** + * Samples for DataNetworks Get. + */ public final class DataNetworksGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/DataNetworkGet. + * json */ /** * Sample code: Get data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .dataNetworks() - .getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", com.azure.core.util.Context.NONE); + manager.dataNetworks().getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", + com.azure.core.util.Context.NONE); } } ``` @@ -355,18 +361,22 @@ public final class DataNetworksGetSamples { ### DataNetworks_ListByMobileNetwork ```java -/** Samples for DataNetworks ListByMobileNetwork. */ +/** + * Samples for DataNetworks ListByMobileNetwork. + */ public final class DataNetworksListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DataNetworkListByMobileNetwork.json */ /** * Sample code: List data networks in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listDataNetworksInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listDataNetworksInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.dataNetworks().listByMobileNetwork("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); } } @@ -379,22 +389,24 @@ import com.azure.resourcemanager.mobilenetwork.models.DataNetwork; import java.util.HashMap; import java.util.Map; -/** Samples for DataNetworks UpdateTags. */ +/** + * Samples for DataNetworks UpdateTags. + */ public final class DataNetworksUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DataNetworkUpdateTags.json */ /** * Sample code: Update data network tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateDataNetworkTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - DataNetwork resource = - manager - .dataNetworks() - .getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", com.azure.core.util.Context.NONE) - .getValue(); + DataNetwork resource = manager.dataNetworks() + .getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -415,20 +427,23 @@ public final class DataNetworksUpdateTagsSamples { ### DiagnosticsPackages_CreateOrUpdate ```java -/** Samples for DiagnosticsPackages CreateOrUpdate. */ +/** + * Samples for DiagnosticsPackages CreateOrUpdate. + */ public final class DiagnosticsPackagesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageCreate.json */ /** * Sample code: Create diagnostics package. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createDiagnosticsPackage(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .diagnosticsPackages() - .createOrUpdate("rg1", "TestPacketCoreCP", "dp1", com.azure.core.util.Context.NONE); + manager.diagnosticsPackages().createOrUpdate("rg1", "TestPacketCoreCP", "dp1", + com.azure.core.util.Context.NONE); } } ``` @@ -436,14 +451,18 @@ public final class DiagnosticsPackagesCreateOrUpdateSamples { ### DiagnosticsPackages_Delete ```java -/** Samples for DiagnosticsPackages Delete. */ +/** + * Samples for DiagnosticsPackages Delete. + */ public final class DiagnosticsPackagesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageDelete.json */ /** * Sample code: Delete diagnostics package. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteDiagnosticsPackage(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -455,20 +474,23 @@ public final class DiagnosticsPackagesDeleteSamples { ### DiagnosticsPackages_Get ```java -/** Samples for DiagnosticsPackages Get. */ +/** + * Samples for DiagnosticsPackages Get. + */ public final class DiagnosticsPackagesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageGet.json */ /** * Sample code: Get diagnostics package. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getDiagnosticsPackage(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .diagnosticsPackages() - .getWithResponse("rg1", "TestPacketCoreCP", "dp1", com.azure.core.util.Context.NONE); + manager.diagnosticsPackages().getWithResponse("rg1", "TestPacketCoreCP", "dp1", + com.azure.core.util.Context.NONE); } } ``` @@ -476,21 +498,63 @@ public final class DiagnosticsPackagesGetSamples { ### DiagnosticsPackages_ListByPacketCoreControlPlane ```java -/** Samples for DiagnosticsPackages ListByPacketCoreControlPlane. */ +/** + * Samples for DiagnosticsPackages ListByPacketCoreControlPlane. + */ public final class DiagnosticsPackagesListByPacketCoreControlPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageListByPacketCoreControlPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageListByPacketCoreControlPlane.json */ /** * Sample code: List diagnostics packages under a packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listDiagnosticsPackagesUnderAPacketCoreControlPlane( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .diagnosticsPackages() - .listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + manager.diagnosticsPackages().listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", + com.azure.core.util.Context.NONE); + } +} +``` + +### ExtendedUeInformation_Get + +```java +/** + * Samples for ExtendedUeInformation Get. + */ +public final class ExtendedUeInformationGetSamples { + /* + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * ExtendedUeInfo4GGet.json + */ + /** + * Sample code: Get UE Information 4G. + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void getUEInformation4G(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.extendedUeInformations().getWithResponse("rg1", "TestPacketCoreCP", "84449105622", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * ExtendedUeInfo5GGet.json + */ + /** + * Sample code: Get UE Information 5G. + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void getUEInformation5G(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.extendedUeInformations().getWithResponse("rg1", "TestPacketCoreCP", "84449105622", + com.azure.core.util.Context.NONE); } } ``` @@ -498,25 +562,38 @@ public final class DiagnosticsPackagesListByPacketCoreControlPlaneSamples { ### MobileNetworks_CreateOrUpdate ```java +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPublicKey; import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetwork; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetworkHomeNetworkPublicKeys; +import java.util.Arrays; -/** Samples for MobileNetworks CreateOrUpdate. */ +/** + * Samples for MobileNetworks CreateOrUpdate. + */ public final class MobileNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkCreate.json */ /** * Sample code: Create mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .mobileNetworks() - .define("testMobileNetwork") - .withRegion("eastus") - .withExistingResourceGroup("rg1") + manager.mobileNetworks().define("testMobileNetwork").withRegion("eastus").withExistingResourceGroup("rg1") .withPublicLandMobileNetworkIdentifier(new PlmnId().withMcc("001").withMnc("01")) + .withPublicLandMobileNetworks(Arrays.asList(new PublicLandMobileNetwork().withMcc("001").withMnc("01") + .withHomeNetworkPublicKeys(new PublicLandMobileNetworkHomeNetworkPublicKeys() + .withProfileA(Arrays.asList( + new HomeNetworkPublicKey().withId(1) + .withUrl("https://contosovault.vault.azure.net/secrets/exampleHnpk"), + new HomeNetworkPublicKey().withId(2).withUrl( + "https://contosovault.vault.azure.net/secrets/exampleHnpk2/5e4876e9140e4e16bfe6e2cf92e0cbd2"))) + .withProfileB(Arrays.asList(new HomeNetworkPublicKey().withId(1) + .withUrl("https://contosovault.vault.azure.net/secrets/exampleHnpkProfileB")))))) .create(); } } @@ -525,14 +602,18 @@ public final class MobileNetworksCreateOrUpdateSamples { ### MobileNetworks_Delete ```java -/** Samples for MobileNetworks Delete. */ +/** + * Samples for MobileNetworks Delete. + */ public final class MobileNetworksDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkDelete.json */ /** * Sample code: Delete mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -544,20 +625,23 @@ public final class MobileNetworksDeleteSamples { ### MobileNetworks_GetByResourceGroup ```java -/** Samples for MobileNetworks GetByResourceGroup. */ +/** + * Samples for MobileNetworks GetByResourceGroup. + */ public final class MobileNetworksGetByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/MobileNetworkGet. + * json */ /** * Sample code: Get mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .mobileNetworks() - .getByResourceGroupWithResponse("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); + manager.mobileNetworks().getByResourceGroupWithResponse("rg1", "testMobileNetwork", + com.azure.core.util.Context.NONE); } } ``` @@ -565,18 +649,22 @@ public final class MobileNetworksGetByResourceGroupSamples { ### MobileNetworks_List ```java -/** Samples for MobileNetworks List. */ +/** + * Samples for MobileNetworks List. + */ public final class MobileNetworksListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkListBySubscription.json */ /** * Sample code: List mobile networks in a subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listMobileNetworksInASubscription( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listMobileNetworksInASubscription(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.mobileNetworks().list(com.azure.core.util.Context.NONE); } } @@ -585,18 +673,22 @@ public final class MobileNetworksListSamples { ### MobileNetworks_ListByResourceGroup ```java -/** Samples for MobileNetworks ListByResourceGroup. */ +/** + * Samples for MobileNetworks ListByResourceGroup. + */ public final class MobileNetworksListByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkListByResourceGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkListByResourceGroup.json */ /** * Sample code: List mobile networks in resource group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listMobileNetworksInResourceGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listMobileNetworksInResourceGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.mobileNetworks().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); } } @@ -609,22 +701,23 @@ import com.azure.resourcemanager.mobilenetwork.models.MobileNetwork; import java.util.HashMap; import java.util.Map; -/** Samples for MobileNetworks UpdateTags. */ +/** + * Samples for MobileNetworks UpdateTags. + */ public final class MobileNetworksUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkUpdateTags.json */ /** * Sample code: Update mobile network tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateMobileNetworkTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - MobileNetwork resource = - manager - .mobileNetworks() - .getByResourceGroupWithResponse("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE) - .getValue(); + MobileNetwork resource = manager.mobileNetworks() + .getByResourceGroupWithResponse("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -645,14 +738,18 @@ public final class MobileNetworksUpdateTagsSamples { ### Operations_List ```java -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/OperationList.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/OperationList. + * json */ /** * Sample code: Get Registration Operations. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getRegistrationOperations(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -666,31 +763,27 @@ public final class OperationsListSamples { ```java import java.util.Arrays; -/** Samples for PacketCaptures CreateOrUpdate. */ +/** + * Samples for PacketCaptures CreateOrUpdate. + */ public final class PacketCapturesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCaptureCreate.json */ /** * Sample code: Create packet capture. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createPacketCapture(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCaptures() - .define("pc1") - .withExistingPacketCoreControlPlane("rg1", "TestPacketCoreCP") - .withNetworkInterfaces( - Arrays - .asList( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP", - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP", - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestADN")) - .withBytesToCapturePerPacket(10000L) - .withTotalBytesPerSession(100000L) - .withTimeLimitInSeconds(100) - .create(); + manager.packetCaptures().define("pc1").withExistingPacketCoreControlPlane("rg1", "TestPacketCoreCP") + .withNetworkInterfaces(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP", + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP", + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestADN")) + .withBytesToCapturePerPacket(10000L).withTotalBytesPerSession(100000L).withTimeLimitInSeconds(100).create(); } } ``` @@ -698,14 +791,18 @@ public final class PacketCapturesCreateOrUpdateSamples { ### PacketCaptures_Delete ```java -/** Samples for PacketCaptures Delete. */ +/** + * Samples for PacketCaptures Delete. + */ public final class PacketCapturesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCaptureDelete.json */ /** * Sample code: Delete packet capture. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deletePacketCapture(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -717,14 +814,18 @@ public final class PacketCapturesDeleteSamples { ### PacketCaptures_Get ```java -/** Samples for PacketCaptures Get. */ +/** + * Samples for PacketCaptures Get. + */ public final class PacketCapturesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/PacketCaptureGet. + * json */ /** * Sample code: Get packet capture session. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCaptureSession(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -736,21 +837,24 @@ public final class PacketCapturesGetSamples { ### PacketCaptures_ListByPacketCoreControlPlane ```java -/** Samples for PacketCaptures ListByPacketCoreControlPlane. */ +/** + * Samples for PacketCaptures ListByPacketCoreControlPlane. + */ public final class PacketCapturesListByPacketCoreControlPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureListByPacketCoreControlPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCaptureListByPacketCoreControlPlane.json */ /** * Sample code: List packet capture sessions under a packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listPacketCaptureSessionsUnderAPacketCoreControlPlane( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCaptures() - .listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + manager.packetCaptures().listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", + com.azure.core.util.Context.NONE); } } ``` @@ -758,14 +862,18 @@ public final class PacketCapturesListByPacketCoreControlPlaneSamples { ### PacketCaptures_Stop ```java -/** Samples for PacketCaptures Stop. */ +/** + * Samples for PacketCaptures Stop. + */ public final class PacketCapturesStopSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureStop.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/PacketCaptureStop + * .json */ /** * Sample code: Stop packet capture session. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void stopPacketCaptureSession(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -777,18 +885,22 @@ public final class PacketCapturesStopSamples { ### PacketCoreControlPlaneVersions_Get ```java -/** Samples for PacketCoreControlPlaneVersions Get. */ +/** + * Samples for PacketCoreControlPlaneVersions Get. + */ public final class PacketCoreControlPlaneVersionsGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionGet.json */ /** * Sample code: Get packet core control plane version. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void getPacketCoreControlPlaneVersion( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + getPacketCoreControlPlaneVersion(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlaneVersions().getWithResponse("PMN-4-11-1", com.azure.core.util.Context.NONE); } } @@ -797,21 +909,24 @@ public final class PacketCoreControlPlaneVersionsGetSamples { ### PacketCoreControlPlaneVersions_GetBySubscription ```java -/** Samples for PacketCoreControlPlaneVersions GetBySubscription. */ +/** + * Samples for PacketCoreControlPlaneVersions GetBySubscription. + */ public final class PacketCoreControlPlaneVersionsGetBySubscriptionSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionGetBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionGetBySubscription.json */ /** * Sample code: Get packet core control plane version by subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCoreControlPlaneVersionBySubscription( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlaneVersions() - .getBySubscriptionWithResponse("PMN-4-11-1", com.azure.core.util.Context.NONE); + manager.packetCoreControlPlaneVersions().getBySubscriptionWithResponse("PMN-4-11-1", + com.azure.core.util.Context.NONE); } } ``` @@ -819,14 +934,18 @@ public final class PacketCoreControlPlaneVersionsGetBySubscriptionSamples { ### PacketCoreControlPlaneVersions_List ```java -/** Samples for PacketCoreControlPlaneVersions List. */ +/** + * Samples for PacketCoreControlPlaneVersions List. + */ public final class PacketCoreControlPlaneVersionsListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionList.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionList.json */ /** * Sample code: Get supported packet core control plane versions. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSupportedPacketCoreControlPlaneVersions( @@ -839,14 +958,18 @@ public final class PacketCoreControlPlaneVersionsListSamples { ### PacketCoreControlPlaneVersions_ListBySubscription ```java -/** Samples for PacketCoreControlPlaneVersions ListBySubscription. */ +/** + * Samples for PacketCoreControlPlaneVersions ListBySubscription. + */ public final class PacketCoreControlPlaneVersionsListBySubscriptionSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionListBySubscription.json */ /** * Sample code: Get supported packet core control plane versions by subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSupportedPacketCoreControlPlaneVersionsBySubscription( @@ -861,27 +984,26 @@ public final class PacketCoreControlPlaneVersionsListBySubscriptionSamples { ```java import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneCollectDiagnosticsPackage; -/** Samples for PacketCoreControlPlanes CollectDiagnosticsPackage. */ +/** + * Samples for PacketCoreControlPlanes CollectDiagnosticsPackage. + */ public final class PacketCoreControlPlanesCollectDiagnosticsPackageSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneCollectDiagnosticsPackage.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneCollectDiagnosticsPackage.json */ /** * Sample code: Collect diagnostics package from packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void collectDiagnosticsPackageFromPacketCoreControlPlane( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlanes() - .collectDiagnosticsPackage( - "rg1", - "TestPacketCoreCP", - new PacketCoreControlPlaneCollectDiagnosticsPackage() - .withStorageAccountBlobUrl( - "https://contosoaccount.blob.core.windows.net/container/diagnosticsPackage.zip"), - com.azure.core.util.Context.NONE); + manager.packetCoreControlPlanes().collectDiagnosticsPackage("rg1", "TestPacketCoreCP", + new PacketCoreControlPlaneCollectDiagnosticsPackage().withStorageAccountBlobUrl( + "https://contosoaccount.blob.core.windows.net/container/diagnosticsPackage.zip"), + com.azure.core.util.Context.NONE); } } ``` @@ -908,61 +1030,43 @@ import com.azure.resourcemanager.mobilenetwork.models.SignalingConfiguration; import com.azure.resourcemanager.mobilenetwork.models.SiteResourceId; import java.util.Arrays; -/** Samples for PacketCoreControlPlanes CreateOrUpdate. */ +/** + * Samples for PacketCoreControlPlanes CreateOrUpdate. + */ public final class PacketCoreControlPlanesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneCreate.json */ /** * Sample code: Create packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void createPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlanes() - .define("TestPacketCoreCP") - .withRegion("eastus") + public static void + createPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.packetCoreControlPlanes().define("TestPacketCoreCP").withRegion("eastus") .withExistingResourceGroup("rg1") - .withSites( - Arrays - .asList( - new SiteResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/sites/testSite"))) - .withPlatform( - new PlatformConfiguration() - .withType(PlatformType.AKS_HCI) - .withAzureStackEdgeDevice( - new AzureStackEdgeDeviceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.DataBoxEdge/dataBoxEdgeDevices/TestAzureStackEdgeDevice")) - .withConnectedCluster( - new ConnectedClusterResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.Kubernetes/connectedClusters/TestConnectedCluster")) - .withCustomLocation( - new CustomLocationResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ExtendedLocation/customLocations/TestCustomLocation"))) - .withControlPlaneAccessInterface(new InterfaceProperties().withName("N2")) - .withSku(BillingSku.G0) + .withSites(Arrays.asList(new SiteResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/sites/testSite"))) + .withPlatform(new PlatformConfiguration().withType(PlatformType.AKS_HCI) + .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.DataBoxEdge/dataBoxEdgeDevices/TestAzureStackEdgeDevice")) + .withConnectedCluster(new ConnectedClusterResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.Kubernetes/connectedClusters/TestConnectedCluster")) + .withCustomLocation(new CustomLocationResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ExtendedLocation/customLocations/TestCustomLocation"))) + .withControlPlaneAccessInterface(new InterfaceProperties().withName("N2")).withSku(BillingSku.G0) .withLocalDiagnosticsAccess( - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.AAD) - .withHttpsServerCertificate( - new HttpsServerCertificate() - .withCertificateUrl("https://contosovault.vault.azure.net/certificates/ingress"))) + new LocalDiagnosticsAccessConfiguration().withAuthenticationType(AuthenticationType.AAD) + .withHttpsServerCertificate(new HttpsServerCertificate() + .withCertificateUrl("https://contosovault.vault.azure.net/certificates/ingress"))) .withInstallation(new Installation().withDesiredState(DesiredInstallationState.INSTALLED)) - .withCoreNetworkTechnology(CoreNetworkType.FIVE_GC) - .withVersion("0.2.0") - .withUeMtu(1600) - .withEventHub( - new EventHubConfiguration() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.EventHub/namespaces/contosoNamespace/eventHubs/contosoHub") - .withReportingInterval(60)) + .withCoreNetworkTechnology(CoreNetworkType.FIVE_GC).withVersion("0.2.0").withUeMtu(1600) + .withEventHub(new EventHubConfiguration().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.EventHub/namespaces/contosoNamespace/eventHubs/contosoHub") + .withReportingInterval(60)) .withSignaling( new SignalingConfiguration().withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(1024))) .create(); @@ -973,18 +1077,22 @@ public final class PacketCoreControlPlanesCreateOrUpdateSamples { ### PacketCoreControlPlanes_Delete ```java -/** Samples for PacketCoreControlPlanes Delete. */ +/** + * Samples for PacketCoreControlPlanes Delete. + */ public final class PacketCoreControlPlanesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneDelete.json */ /** * Sample code: Delete packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void deletePacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + deletePacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlanes().delete("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); } } @@ -993,20 +1101,23 @@ public final class PacketCoreControlPlanesDeleteSamples { ### PacketCoreControlPlanes_GetByResourceGroup ```java -/** Samples for PacketCoreControlPlanes GetByResourceGroup. */ +/** + * Samples for PacketCoreControlPlanes GetByResourceGroup. + */ public final class PacketCoreControlPlanesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneGet.json */ /** * Sample code: Get packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlanes() - .getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + manager.packetCoreControlPlanes().getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", + com.azure.core.util.Context.NONE); } } ``` @@ -1014,14 +1125,18 @@ public final class PacketCoreControlPlanesGetByResourceGroupSamples { ### PacketCoreControlPlanes_List ```java -/** Samples for PacketCoreControlPlanes List. */ +/** + * Samples for PacketCoreControlPlanes List. + */ public final class PacketCoreControlPlanesListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneListBySubscription.json */ /** * Sample code: List packet core control planes in a subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listPacketCoreControlPlanesInASubscription( @@ -1034,14 +1149,18 @@ public final class PacketCoreControlPlanesListSamples { ### PacketCoreControlPlanes_ListByResourceGroup ```java -/** Samples for PacketCoreControlPlanes ListByResourceGroup. */ +/** + * Samples for PacketCoreControlPlanes ListByResourceGroup. + */ public final class PacketCoreControlPlanesListByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneListByResourceGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneListByResourceGroup.json */ /** * Sample code: List packet core control planes in resource group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listPacketCoreControlPlanesInResourceGroup( @@ -1054,18 +1173,22 @@ public final class PacketCoreControlPlanesListByResourceGroupSamples { ### PacketCoreControlPlanes_Reinstall ```java -/** Samples for PacketCoreControlPlanes Reinstall. */ +/** + * Samples for PacketCoreControlPlanes Reinstall. + */ public final class PacketCoreControlPlanesReinstallSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneReinstall.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneReinstall.json */ /** * Sample code: Reinstall packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void reinstallPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + reinstallPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlanes().reinstall("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); } } @@ -1074,18 +1197,22 @@ public final class PacketCoreControlPlanesReinstallSamples { ### PacketCoreControlPlanes_Rollback ```java -/** Samples for PacketCoreControlPlanes Rollback. */ +/** + * Samples for PacketCoreControlPlanes Rollback. + */ public final class PacketCoreControlPlanesRollbackSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneRollback.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneRollback.json */ /** * Sample code: Rollback packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void rollbackPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + rollbackPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlanes().rollback("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); } } @@ -1101,33 +1228,29 @@ import com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity; import java.util.HashMap; import java.util.Map; -/** Samples for PacketCoreControlPlanes UpdateTags. */ +/** + * Samples for PacketCoreControlPlanes UpdateTags. + */ public final class PacketCoreControlPlanesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlanePatch.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlanePatch.json */ /** * Sample code: Patch packet core control plane. - * - * @param manager Entry point to MobileNetworkManager. - */ - public static void patchPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - PacketCoreControlPlane resource = - manager - .packetCoreControlPlanes() - .getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", - new UserAssignedIdentity()))) + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void + patchPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + PacketCoreControlPlane resource = manager.packetCoreControlPlanes() + .getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", + new UserAssignedIdentity()))) .apply(); } @@ -1150,24 +1273,24 @@ public final class PacketCoreControlPlanesUpdateTagsSamples { ```java import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; -/** Samples for PacketCoreDataPlanes CreateOrUpdate. */ +/** + * Samples for PacketCoreDataPlanes CreateOrUpdate. + */ public final class PacketCoreDataPlanesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneCreate.json */ /** * Sample code: Create packet core data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createPacketCoreDataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .define("testPacketCoreDP") - .withRegion("eastus") + manager.packetCoreDataPlanes().define("testPacketCoreDP").withRegion("eastus") .withExistingPacketCoreControlPlane("rg1", "testPacketCoreCP") - .withUserPlaneAccessInterface(new InterfaceProperties().withName("N3")) - .create(); + .withUserPlaneAccessInterface(new InterfaceProperties().withName("N3")).create(); } } ``` @@ -1175,20 +1298,23 @@ public final class PacketCoreDataPlanesCreateOrUpdateSamples { ### PacketCoreDataPlanes_Delete ```java -/** Samples for PacketCoreDataPlanes Delete. */ +/** + * Samples for PacketCoreDataPlanes Delete. + */ public final class PacketCoreDataPlanesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneDelete.json */ /** * Sample code: Delete packet core data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deletePacketCoreDataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .delete("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE); + manager.packetCoreDataPlanes().delete("rg1", "testPacketCoreCP", "testPacketCoreDP", + com.azure.core.util.Context.NONE); } } ``` @@ -1196,20 +1322,23 @@ public final class PacketCoreDataPlanesDeleteSamples { ### PacketCoreDataPlanes_Get ```java -/** Samples for PacketCoreDataPlanes Get. */ +/** + * Samples for PacketCoreDataPlanes Get. + */ public final class PacketCoreDataPlanesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneGet.json */ /** * Sample code: Get packet core data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCoreDataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE); + manager.packetCoreDataPlanes().getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", + com.azure.core.util.Context.NONE); } } ``` @@ -1217,21 +1346,24 @@ public final class PacketCoreDataPlanesGetSamples { ### PacketCoreDataPlanes_ListByPacketCoreControlPlane ```java -/** Samples for PacketCoreDataPlanes ListByPacketCoreControlPlane. */ +/** + * Samples for PacketCoreDataPlanes ListByPacketCoreControlPlane. + */ public final class PacketCoreDataPlanesListByPacketCoreControlPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneListByPacketCoreControlPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneListByPacketCoreControlPlane.json */ /** * Sample code: List packet core data planes in a control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listPacketCoreDataPlanesInAControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .listByPacketCoreControlPlane("rg1", "testPacketCoreCP", com.azure.core.util.Context.NONE); + public static void + listPacketCoreDataPlanesInAControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.packetCoreDataPlanes().listByPacketCoreControlPlane("rg1", "testPacketCoreCP", + com.azure.core.util.Context.NONE); } } ``` @@ -1243,23 +1375,25 @@ import com.azure.resourcemanager.mobilenetwork.models.PacketCoreDataPlane; import java.util.HashMap; import java.util.Map; -/** Samples for PacketCoreDataPlanes UpdateTags. */ +/** + * Samples for PacketCoreDataPlanes UpdateTags. + */ public final class PacketCoreDataPlanesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneUpdateTags.json */ /** * Sample code: Update packet core data plane tags. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void updatePacketCoreDataPlaneTags( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - PacketCoreDataPlane resource = - manager - .packetCoreDataPlanes() - .getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE) - .getValue(); + public static void + updatePacketCoreDataPlaneTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + PacketCoreDataPlane resource = manager.packetCoreDataPlanes() + .getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -1291,53 +1425,36 @@ import com.azure.resourcemanager.mobilenetwork.models.ServiceDataFlowTemplate; import com.azure.resourcemanager.mobilenetwork.models.TrafficControlPermission; import java.util.Arrays; -/** Samples for Services CreateOrUpdate. */ +/** + * Samples for Services CreateOrUpdate. + */ public final class ServicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceCreate. + * json */ /** * Sample code: Create service. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createService(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .services() - .define("TestService") - .withRegion("eastus") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .withServicePrecedence(255) - .withPccRules( - Arrays - .asList( - new PccRuleConfiguration() - .withRuleName("default-rule") - .withRulePrecedence(255) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(9) - .withAllocationAndRetentionPriorityLevel(9) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("IP-to-server") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("ip")) - .withRemoteIpList(Arrays.asList("10.3.4.0/24")) - .withPorts(Arrays.asList()))))) - .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(9) - .withAllocationAndRetentionPriorityLevel(9) + manager.services().define("TestService").withRegion("eastus") + .withExistingMobileNetwork("rg1", "testMobileNetwork").withServicePrecedence(255) + .withPccRules(Arrays.asList(new PccRuleConfiguration().withRuleName("default-rule").withRulePrecedence(255) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(9).withAllocationAndRetentionPriorityLevel(9) .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) .withMaximumBitRate(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps"))) + .withTrafficControl(TrafficControlPermission.ENABLED).withServiceDataFlowTemplates( + Arrays.asList(new ServiceDataFlowTemplate().withTemplateName("IP-to-server") + .withDirection(SdfDirection.UPLINK).withProtocol(Arrays.asList("ip")) + .withRemoteIpList(Arrays.asList("10.3.4.0/24")).withPorts(Arrays.asList()))))) + .withServiceQosPolicy(new QosPolicy().withFiveQi(9).withAllocationAndRetentionPriorityLevel(9) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps"))) .create(); } } @@ -1346,14 +1463,18 @@ public final class ServicesCreateOrUpdateSamples { ### Services_Delete ```java -/** Samples for Services Delete. */ +/** + * Samples for Services Delete. + */ public final class ServicesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceDelete. + * json */ /** * Sample code: Delete service. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteService(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -1365,14 +1486,17 @@ public final class ServicesDeleteSamples { ### Services_Get ```java -/** Samples for Services Get. */ +/** + * Samples for Services Get. + */ public final class ServicesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceGet.json */ /** * Sample code: Get service. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getService(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -1384,21 +1508,24 @@ public final class ServicesGetSamples { ### Services_ListByMobileNetwork ```java -/** Samples for Services ListByMobileNetwork. */ +/** + * Samples for Services ListByMobileNetwork. + */ public final class ServicesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * ServiceListByMobileNetwork.json */ /** * Sample code: List services in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listServicesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .services() - .listByMobileNetwork("testResourceGroupName", "testMobileNetwork", com.azure.core.util.Context.NONE); + public static void + listServicesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.services().listByMobileNetwork("testResourceGroupName", "testMobileNetwork", + com.azure.core.util.Context.NONE); } } ``` @@ -1410,22 +1537,23 @@ import com.azure.resourcemanager.mobilenetwork.models.Service; import java.util.HashMap; import java.util.Map; -/** Samples for Services UpdateTags. */ +/** + * Samples for Services UpdateTags. + */ public final class ServicesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceUpdateTags + * .json */ /** * Sample code: Update service tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateServiceTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - Service resource = - manager - .services() - .getWithResponse("rg1", "testMobileNetwork", "TestService", com.azure.core.util.Context.NONE) - .getValue(); + Service resource = manager.services() + .getWithResponse("rg1", "testMobileNetwork", "TestService", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -1454,34 +1582,29 @@ import com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity; import java.util.HashMap; import java.util.Map; -/** Samples for SimGroups CreateOrUpdate. */ +/** + * Samples for SimGroups CreateOrUpdate. + */ public final class SimGroupsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupCreate. + * json */ /** * Sample code: Create SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simGroups() - .define("testSimGroup") - .withRegion("eastus") - .withExistingResourceGroup("rg1") - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", - new UserAssignedIdentity()))) + manager.simGroups().define("testSimGroup").withRegion("eastus").withExistingResourceGroup("rg1") + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", + new UserAssignedIdentity()))) .withEncryptionKey(new KeyVaultKey().withKeyUrl("fakeTokenPlaceholder")) - .withMobileNetwork( - new MobileNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork")) + .withMobileNetwork(new MobileNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork")) .create(); } @@ -1502,14 +1625,18 @@ public final class SimGroupsCreateOrUpdateSamples { ### SimGroups_Delete ```java -/** Samples for SimGroups Delete. */ +/** + * Samples for SimGroups Delete. + */ public final class SimGroupsDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupDelete. + * json */ /** * Sample code: Delete SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -1521,21 +1648,22 @@ public final class SimGroupsDeleteSamples { ### SimGroups_GetByResourceGroup ```java -/** Samples for SimGroups GetByResourceGroup. */ +/** + * Samples for SimGroups GetByResourceGroup. + */ public final class SimGroupsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupGet.json */ /** * Sample code: Get SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simGroups() - .getByResourceGroupWithResponse( - "testResourceGroupName", "testSimGroupName", com.azure.core.util.Context.NONE); + manager.simGroups().getByResourceGroupWithResponse("testResourceGroupName", "testSimGroupName", + com.azure.core.util.Context.NONE); } } ``` @@ -1543,18 +1671,22 @@ public final class SimGroupsGetByResourceGroupSamples { ### SimGroups_List ```java -/** Samples for SimGroups List. */ +/** + * Samples for SimGroups List. + */ public final class SimGroupsListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimGroupListBySubscription.json */ /** * Sample code: List SIM groups in a subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listSIMGroupsInASubscription( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listSIMGroupsInASubscription(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.simGroups().list(com.azure.core.util.Context.NONE); } } @@ -1563,18 +1695,22 @@ public final class SimGroupsListSamples { ### SimGroups_ListByResourceGroup ```java -/** Samples for SimGroups ListByResourceGroup. */ +/** + * Samples for SimGroups ListByResourceGroup. + */ public final class SimGroupsListByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupListByResourceGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimGroupListByResourceGroup.json */ /** * Sample code: List SIM groups in a resource group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listSIMGroupsInAResourceGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listSIMGroupsInAResourceGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.simGroups().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); } } @@ -1590,32 +1726,28 @@ import com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity; import java.util.HashMap; import java.util.Map; -/** Samples for SimGroups UpdateTags. */ +/** + * Samples for SimGroups UpdateTags. + */ public final class SimGroupsUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupPatch.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupPatch. + * json */ /** * Sample code: Patch SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void patchSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - SimGroup resource = - manager - .simGroups() - .getByResourceGroupWithResponse("rg1", "testSimGroup", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", - new UserAssignedIdentity()))) + SimGroup resource = manager.simGroups() + .getByResourceGroupWithResponse("rg1", "testSimGroup", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", + new UserAssignedIdentity()))) .apply(); } @@ -1647,63 +1779,42 @@ import com.azure.resourcemanager.mobilenetwork.models.SliceConfiguration; import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for SimPolicies CreateOrUpdate. */ +/** + * Samples for SimPolicies CreateOrUpdate. + */ public final class SimPoliciesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimPolicyCreate. + * json */ /** * Sample code: Create SIM policy. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createSIMPolicy(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simPolicies() - .define("testPolicy") - .withRegion("eastus") + manager.simPolicies().define("testPolicy").withRegion("eastus") .withExistingMobileNetwork("rg1", "testMobileNetwork") .withUeAmbr(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps")) - .withDefaultSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withSliceConfigurations( - Arrays - .asList( - new SliceConfiguration() - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withDefaultDataNetwork( - new DataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) - .withSessionAmbr(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps")) - .withFiveQi(9) - .withAllocationAndRetentionPriorityLevel(9) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes(Arrays.asList()) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/services/testService"))) - .withMaximumNumberOfBufferedPackets(200))))) - .withRegistrationTimer(3240) - .create(); + .withDefaultSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withSliceConfigurations(Arrays.asList(new SliceConfiguration().withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) + .withDataNetworkConfigurations(Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) + .withSessionAmbr(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps")).withFiveQi(9) + .withAllocationAndRetentionPriorityLevel(9) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4).withAdditionalAllowedSessionTypes(Arrays.asList()) + .withAllowedServices(Arrays.asList(new ServiceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/services/testService"))) + .withMaximumNumberOfBufferedPackets(200))))) + .withRegistrationTimer(3240).create(); } } ``` @@ -1711,14 +1822,18 @@ public final class SimPoliciesCreateOrUpdateSamples { ### SimPolicies_Delete ```java -/** Samples for SimPolicies Delete. */ +/** + * Samples for SimPolicies Delete. + */ public final class SimPoliciesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimPolicyDelete. + * json */ /** * Sample code: Delete SIM policy. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteSIMPolicy(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -1730,20 +1845,22 @@ public final class SimPoliciesDeleteSamples { ### SimPolicies_Get ```java -/** Samples for SimPolicies Get. */ +/** + * Samples for SimPolicies Get. + */ public final class SimPoliciesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimPolicyGet.json */ /** * Sample code: Get SIM policy. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSIMPolicy(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simPolicies() - .getWithResponse("rg1", "testMobileNetwork", "testPolicy", com.azure.core.util.Context.NONE); + manager.simPolicies().getWithResponse("rg1", "testMobileNetwork", "testPolicy", + com.azure.core.util.Context.NONE); } } ``` @@ -1751,21 +1868,24 @@ public final class SimPoliciesGetSamples { ### SimPolicies_ListByMobileNetwork ```java -/** Samples for SimPolicies ListByMobileNetwork. */ +/** + * Samples for SimPolicies ListByMobileNetwork. + */ public final class SimPoliciesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimPolicyListByMobileNetwork.json */ /** * Sample code: List SIM policies in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listSIMPoliciesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simPolicies() - .listByMobileNetwork("testResourceGroupName", "testMobileNetwork", com.azure.core.util.Context.NONE); + public static void + listSIMPoliciesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.simPolicies().listByMobileNetwork("testResourceGroupName", "testMobileNetwork", + com.azure.core.util.Context.NONE); } } ``` @@ -1777,22 +1897,23 @@ import com.azure.resourcemanager.mobilenetwork.models.SimPolicy; import java.util.HashMap; import java.util.Map; -/** Samples for SimPolicies UpdateTags. */ +/** + * Samples for SimPolicies UpdateTags. + */ public final class SimPoliciesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimPolicyUpdateTags.json */ /** * Sample code: Update SIM policy tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateSIMPolicyTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - SimPolicy resource = - manager - .simPolicies() - .getWithResponse("rg1", "testMobileNetwork", "testPolicy", com.azure.core.util.Context.NONE) - .getValue(); + SimPolicy resource = manager.simPolicies() + .getWithResponse("rg1", "testMobileNetwork", "testPolicy", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -1816,25 +1937,24 @@ public final class SimPoliciesUpdateTagsSamples { import com.azure.resourcemanager.mobilenetwork.models.SimDeleteList; import java.util.Arrays; -/** Samples for Sims BulkDelete. */ +/** + * Samples for Sims BulkDelete. + */ public final class SimsBulkDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimBulkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimBulkDelete. + * json */ /** * Sample code: Bulk delete SIMs from a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void bulkDeleteSIMsFromASIMGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .bulkDelete( - "testResourceGroupName", - "testSimGroup", - new SimDeleteList().withSims(Arrays.asList("testSim", "testSim2")), - com.azure.core.util.Context.NONE); + public static void + bulkDeleteSIMsFromASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.sims().bulkDelete("testResourceGroupName", "testSimGroup", + new SimDeleteList().withSims(Arrays.asList("testSim", "testSim2")), com.azure.core.util.Context.NONE); } } ``` @@ -1851,79 +1971,45 @@ import com.azure.resourcemanager.mobilenetwork.models.SimUploadList; import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for Sims BulkUpload. */ +/** + * Samples for Sims BulkUpload. + */ public final class SimsBulkUploadSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimBulkUpload.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimBulkUpload. + * json */ /** * Sample code: Bulk upload SIMs in a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void bulkUploadSIMsInASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .bulkUpload( - "rg1", - "testSimGroup", - new SimUploadList() - .withSims( - Arrays - .asList( - new SimNameAndProperties() - .withName("testSim") - .withAuthenticationKey("fakeTokenPlaceholder") - .withOperatorKeyCode("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000000") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.1")))), - new SimNameAndProperties() - .withName("testSim2") - .withAuthenticationKey("fakeTokenPlaceholder") - .withOperatorKeyCode("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000001") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.2")))))), - com.azure.core.util.Context.NONE); + manager.sims().bulkUpload("rg1", "testSimGroup", new SimUploadList().withSims(Arrays.asList( + new SimNameAndProperties().withName("testSim").withAuthenticationKey("fakeTokenPlaceholder") + .withOperatorKeyCode("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000000").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))), + new SimNameAndProperties().withName("testSim2").withAuthenticationKey("fakeTokenPlaceholder") + .withOperatorKeyCode("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000001").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.2")))))), + com.azure.core.util.Context.NONE); } } ``` @@ -1940,83 +2026,48 @@ import com.azure.resourcemanager.mobilenetwork.models.SimStaticIpPropertiesStati import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for Sims BulkUploadEncrypted. */ +/** + * Samples for Sims BulkUploadEncrypted. + */ public final class SimsBulkUploadEncryptedSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimBulkUploadEncrypted.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimBulkUploadEncrypted.json */ /** * Sample code: Bulk upload encrypted SIMs to a SIM group. - * - * @param manager Entry point to MobileNetworkManager. - */ - public static void bulkUploadEncryptedSIMsToASIMGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .bulkUploadEncrypted( - "rg1", - "testSimGroup", - new EncryptedSimUploadList() - .withVersion(1) - .withAzureKeyIdentifier(1) - .withVendorKeyFingerprint("fakeTokenPlaceholder") - .withEncryptedTransportKey("fakeTokenPlaceholder") - .withSignedTransportKey("fakeTokenPlaceholder") - .withSims( - Arrays - .asList( - new SimNameAndEncryptedProperties() - .withName("testSim") - .withEncryptedCredentials("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000000") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.1")))), - new SimNameAndEncryptedProperties() - .withName("testSim2") - .withEncryptedCredentials("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000001") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.2")))))), - com.azure.core.util.Context.NONE); + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void + bulkUploadEncryptedSIMsToASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.sims().bulkUploadEncrypted("rg1", "testSimGroup", new EncryptedSimUploadList().withVersion(1) + .withAzureKeyIdentifier(1).withVendorKeyFingerprint("fakeTokenPlaceholder") + .withEncryptedTransportKey("fakeTokenPlaceholder").withSignedTransportKey("fakeTokenPlaceholder") + .withSims(Arrays.asList(new SimNameAndEncryptedProperties().withName("testSim") + .withEncryptedCredentials("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000000").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))), + new SimNameAndEncryptedProperties().withName("testSim2") + .withEncryptedCredentials("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000001").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.2")))))), + com.azure.core.util.Context.NONE); } } ``` @@ -2031,43 +2082,33 @@ import com.azure.resourcemanager.mobilenetwork.models.SimStaticIpPropertiesStati import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for Sims CreateOrUpdate. */ +/** + * Samples for Sims CreateOrUpdate. + */ public final class SimsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimCreate.json */ /** * Sample code: Create SIM. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createSIM(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .define("testSim") - .withExistingSimGroup("rg1", "testSimGroup") + manager.sims().define("testSim").withExistingSimGroup("rg1", "testSimGroup") .withInternationalMobileSubscriberIdentity("00000") .withAuthenticationKey("00000000000000000000000000000000") .withOperatorKeyCode("00000000000000000000000000000000") - .withIntegratedCircuitCardIdentifier("8900000000000000000") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))) + .withIntegratedCircuitCardIdentifier("8900000000000000000").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))) .create(); } } @@ -2076,14 +2117,17 @@ public final class SimsCreateOrUpdateSamples { ### Sims_Delete ```java -/** Samples for Sims Delete. */ +/** + * Samples for Sims Delete. + */ public final class SimsDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimDelete.json */ /** * Sample code: Delete SIM. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteSIM(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -2095,20 +2139,22 @@ public final class SimsDeleteSamples { ### Sims_Get ```java -/** Samples for Sims Get. */ +/** + * Samples for Sims Get. + */ public final class SimsGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGet.json */ /** * Sample code: Get SIM. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSIM(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .getWithResponse("testResourceGroupName", "testSimGroup", "testSimName", com.azure.core.util.Context.NONE); + manager.sims().getWithResponse("testResourceGroupName", "testSimGroup", "testSimName", + com.azure.core.util.Context.NONE); } } ``` @@ -2116,14 +2162,18 @@ public final class SimsGetSamples { ### Sims_ListByGroup ```java -/** Samples for Sims ListByGroup. */ +/** + * Samples for Sims ListByGroup. + */ public final class SimsListByGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimListBySimGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimListBySimGroup + * .json */ /** * Sample code: List SIMs in a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listSIMsInASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -2135,23 +2185,22 @@ public final class SimsListByGroupSamples { ### Sites_CreateOrUpdate ```java -/** Samples for Sites CreateOrUpdate. */ +/** + * Samples for Sites CreateOrUpdate. + */ public final class SitesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteCreate.json */ /** * Sample code: Create mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sites() - .define("testSite") - .withRegion("testLocation") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .create(); + manager.sites().define("testSite").withRegion("testLocation") + .withExistingMobileNetwork("rg1", "testMobileNetwork").create(); } } ``` @@ -2159,14 +2208,17 @@ public final class SitesCreateOrUpdateSamples { ### Sites_Delete ```java -/** Samples for Sites Delete. */ +/** + * Samples for Sites Delete. + */ public final class SitesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteDelete.json */ /** * Sample code: Delete mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -2181,30 +2233,26 @@ public final class SitesDeleteSamples { import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneResourceId; import com.azure.resourcemanager.mobilenetwork.models.SiteDeletePacketCore; -/** Samples for Sites DeletePacketCore. */ +/** + * Samples for Sites DeletePacketCore. + */ public final class SitesDeletePacketCoreSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteDeletePacketCore.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SiteDeletePacketCore.json */ /** * Sample code: Delete packet core in mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void deletePacketCoreInMobileNetworkSite( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sites() - .deletePacketCore( - "rg1", - "testMobileNetwork", - "testSite", - new SiteDeletePacketCore() - .withPacketCore( - new PacketCoreControlPlaneResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP")), - com.azure.core.util.Context.NONE); + public static void + deletePacketCoreInMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.sites().deletePacketCore("rg1", "testMobileNetwork", "testSite", + new SiteDeletePacketCore().withPacketCore(new PacketCoreControlPlaneResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP")), + com.azure.core.util.Context.NONE); } } ``` @@ -2212,14 +2260,17 @@ public final class SitesDeletePacketCoreSamples { ### Sites_Get ```java -/** Samples for Sites Get. */ +/** + * Samples for Sites Get. + */ public final class SitesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteGet.json */ /** * Sample code: Get mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -2231,18 +2282,22 @@ public final class SitesGetSamples { ### Sites_ListByMobileNetwork ```java -/** Samples for Sites ListByMobileNetwork. */ +/** + * Samples for Sites ListByMobileNetwork. + */ public final class SitesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SiteListByMobileNetwork.json */ /** * Sample code: List mobile network sites in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listMobileNetworkSitesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listMobileNetworkSitesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.sites().listByMobileNetwork("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); } } @@ -2255,23 +2310,24 @@ import com.azure.resourcemanager.mobilenetwork.models.Site; import java.util.HashMap; import java.util.Map; -/** Samples for Sites UpdateTags. */ +/** + * Samples for Sites UpdateTags. + */ public final class SitesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteUpdateTags. + * json */ /** * Sample code: Update mobile network site tags. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void updateMobileNetworkSiteTags( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - Site resource = - manager - .sites() - .getWithResponse("rg1", "testMobileNetwork", "testSite", com.azure.core.util.Context.NONE) - .getValue(); + public static void + updateMobileNetworkSiteTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + Site resource = manager.sites() + .getWithResponse("rg1", "testMobileNetwork", "testSite", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -2294,25 +2350,22 @@ public final class SitesUpdateTagsSamples { ```java import com.azure.resourcemanager.mobilenetwork.models.Snssai; -/** Samples for Slices CreateOrUpdate. */ +/** + * Samples for Slices CreateOrUpdate. + */ public final class SlicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceCreate.json */ /** * Sample code: Create network slice. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createNetworkSlice(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .slices() - .define("testSlice") - .withRegion("eastus") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .withSnssai(new Snssai().withSst(1).withSd("1abcde")) - .withDescription("myFavouriteSlice") - .create(); + manager.slices().define("testSlice").withRegion("eastus").withExistingMobileNetwork("rg1", "testMobileNetwork") + .withSnssai(new Snssai().withSst(1).withSd("1abcde")).withDescription("myFavouriteSlice").create(); } } ``` @@ -2320,14 +2373,17 @@ public final class SlicesCreateOrUpdateSamples { ### Slices_Delete ```java -/** Samples for Slices Delete. */ +/** + * Samples for Slices Delete. + */ public final class SlicesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceDelete.json */ /** * Sample code: Delete network slice. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteNetworkSlice(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -2339,14 +2395,17 @@ public final class SlicesDeleteSamples { ### Slices_Get ```java -/** Samples for Slices Get. */ +/** + * Samples for Slices Get. + */ public final class SlicesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceGet.json */ /** * Sample code: Get network slice. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getNetworkSlice(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { @@ -2358,18 +2417,22 @@ public final class SlicesGetSamples { ### Slices_ListByMobileNetwork ```java -/** Samples for Slices ListByMobileNetwork. */ +/** + * Samples for Slices ListByMobileNetwork. + */ public final class SlicesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SliceListByMobileNetwork.json */ /** * Sample code: List network slices in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listNetworkSlicesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listNetworkSlicesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.slices().listByMobileNetwork("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); } } @@ -2382,22 +2445,23 @@ import com.azure.resourcemanager.mobilenetwork.models.Slice; import java.util.HashMap; import java.util.Map; -/** Samples for Slices UpdateTags. */ +/** + * Samples for Slices UpdateTags. + */ public final class SlicesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceUpdateTags. + * json */ /** * Sample code: Update network slice tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateNetworkSliceTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - Slice resource = - manager - .slices() - .getWithResponse("rg1", "testMobileNetwork", "testSlice", com.azure.core.util.Context.NONE) - .getValue(); + Slice resource = manager.slices() + .getWithResponse("rg1", "testMobileNetwork", "testSlice", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } @@ -2415,3 +2479,25 @@ public final class SlicesUpdateTagsSamples { } ``` +### UeInformation_List + +```java +/** + * Samples for UeInformation List. + */ +public final class UeInformationListSamples { + /* + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/UeInfoList.json + */ + /** + * Sample code: Get UE Information. + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void getUEInformation(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.ueInformations().list("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + } +} +``` + diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/pom.xml b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/pom.xml index 44d748140fc5b..da5d2d54448cd 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/pom.xml +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-mobilenetwork - 1.1.0-beta.1 + 1.2.0-beta.1 jar Microsoft Azure SDK for MobileNetwork Management - This package contains Microsoft Azure SDK for MobileNetwork Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. The resources in this API specification will be used to manage attached data network resources in mobile network attached to a particular packet core instance. Package tag package-2023-09. + This package contains Microsoft Azure SDK for MobileNetwork Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. The resources in this API specification will be used to manage attached data network resources in mobile network attached to a particular packet core instance. Package tag package-2024-02. https://github.com/Azure/azure-sdk-for-java @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/MobileNetworkManager.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/MobileNetworkManager.java index 5c6a0940479ea..91aec47ceb114 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/MobileNetworkManager.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/MobileNetworkManager.java @@ -27,6 +27,7 @@ import com.azure.resourcemanager.mobilenetwork.implementation.AttachedDataNetworksImpl; import com.azure.resourcemanager.mobilenetwork.implementation.DataNetworksImpl; import com.azure.resourcemanager.mobilenetwork.implementation.DiagnosticsPackagesImpl; +import com.azure.resourcemanager.mobilenetwork.implementation.ExtendedUeInformationsImpl; import com.azure.resourcemanager.mobilenetwork.implementation.MobileNetworkManagementClientBuilder; import com.azure.resourcemanager.mobilenetwork.implementation.MobileNetworksImpl; import com.azure.resourcemanager.mobilenetwork.implementation.OperationsImpl; @@ -40,9 +41,11 @@ import com.azure.resourcemanager.mobilenetwork.implementation.SimsImpl; import com.azure.resourcemanager.mobilenetwork.implementation.SitesImpl; import com.azure.resourcemanager.mobilenetwork.implementation.SlicesImpl; +import com.azure.resourcemanager.mobilenetwork.implementation.UeInformationsImpl; import com.azure.resourcemanager.mobilenetwork.models.AttachedDataNetworks; import com.azure.resourcemanager.mobilenetwork.models.DataNetworks; import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsPackages; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInformations; import com.azure.resourcemanager.mobilenetwork.models.MobileNetworks; import com.azure.resourcemanager.mobilenetwork.models.Operations; import com.azure.resourcemanager.mobilenetwork.models.PacketCaptures; @@ -55,6 +58,7 @@ import com.azure.resourcemanager.mobilenetwork.models.Sims; import com.azure.resourcemanager.mobilenetwork.models.Sites; import com.azure.resourcemanager.mobilenetwork.models.Slices; +import com.azure.resourcemanager.mobilenetwork.models.UeInformations; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -63,8 +67,9 @@ import java.util.stream.Collectors; /** - * Entry point to MobileNetworkManager. The resources in this API specification will be used to manage attached data - * network resources in mobile network attached to a particular packet core instance. + * Entry point to MobileNetworkManager. + * The resources in this API specification will be used to manage attached data network resources in mobile network + * attached to a particular packet core instance. */ public final class MobileNetworkManager { private AttachedDataNetworks attachedDataNetworks; @@ -97,23 +102,23 @@ public final class MobileNetworkManager { private Slices slices; + private ExtendedUeInformations extendedUeInformations; + + private UeInformations ueInformations; + private final MobileNetworkManagementClient clientObject; private MobileNetworkManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new MobileNetworkManagementClientBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new MobileNetworkManagementClientBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of MobileNetwork service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the MobileNetwork service API instance. @@ -126,7 +131,7 @@ public static MobileNetworkManager authenticate(TokenCredential credential, Azur /** * Creates an instance of MobileNetwork service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the MobileNetwork service API instance. @@ -139,14 +144,16 @@ public static MobileNetworkManager authenticate(HttpPipeline httpPipeline, Azure /** * Gets a Configurable instance that can be used to create MobileNetworkManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new MobileNetworkManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -218,8 +225,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -236,8 +243,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -257,21 +264,12 @@ public MobileNetworkManager authenticate(TokenCredential credential, AzureProfil Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.mobilenetwork") - .append("/") - .append("1.0.0"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.mobilenetwork") + .append("/").append("1.1.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -290,38 +288,25 @@ public MobileNetworkManager authenticate(TokenCredential credential, AzureProfil policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new MobileNetworkManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of AttachedDataNetworks. It manages AttachedDataNetwork. - * + * * @return Resource collection API of AttachedDataNetworks. */ public AttachedDataNetworks attachedDataNetworks() { @@ -333,7 +318,7 @@ public AttachedDataNetworks attachedDataNetworks() { /** * Gets the resource collection API of DataNetworks. It manages DataNetwork. - * + * * @return Resource collection API of DataNetworks. */ public DataNetworks dataNetworks() { @@ -345,7 +330,7 @@ public DataNetworks dataNetworks() { /** * Gets the resource collection API of DiagnosticsPackages. - * + * * @return Resource collection API of DiagnosticsPackages. */ public DiagnosticsPackages diagnosticsPackages() { @@ -357,7 +342,7 @@ public DiagnosticsPackages diagnosticsPackages() { /** * Gets the resource collection API of MobileNetworks. It manages MobileNetwork. - * + * * @return Resource collection API of MobileNetworks. */ public MobileNetworks mobileNetworks() { @@ -369,7 +354,7 @@ public MobileNetworks mobileNetworks() { /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -381,7 +366,7 @@ public Operations operations() { /** * Gets the resource collection API of PacketCaptures. It manages PacketCapture. - * + * * @return Resource collection API of PacketCaptures. */ public PacketCaptures packetCaptures() { @@ -393,33 +378,33 @@ public PacketCaptures packetCaptures() { /** * Gets the resource collection API of PacketCoreControlPlanes. It manages PacketCoreControlPlane. - * + * * @return Resource collection API of PacketCoreControlPlanes. */ public PacketCoreControlPlanes packetCoreControlPlanes() { if (this.packetCoreControlPlanes == null) { - this.packetCoreControlPlanes = - new PacketCoreControlPlanesImpl(clientObject.getPacketCoreControlPlanes(), this); + this.packetCoreControlPlanes + = new PacketCoreControlPlanesImpl(clientObject.getPacketCoreControlPlanes(), this); } return packetCoreControlPlanes; } /** * Gets the resource collection API of PacketCoreControlPlaneVersions. - * + * * @return Resource collection API of PacketCoreControlPlaneVersions. */ public PacketCoreControlPlaneVersions packetCoreControlPlaneVersions() { if (this.packetCoreControlPlaneVersions == null) { - this.packetCoreControlPlaneVersions = - new PacketCoreControlPlaneVersionsImpl(clientObject.getPacketCoreControlPlaneVersions(), this); + this.packetCoreControlPlaneVersions + = new PacketCoreControlPlaneVersionsImpl(clientObject.getPacketCoreControlPlaneVersions(), this); } return packetCoreControlPlaneVersions; } /** * Gets the resource collection API of PacketCoreDataPlanes. It manages PacketCoreDataPlane. - * + * * @return Resource collection API of PacketCoreDataPlanes. */ public PacketCoreDataPlanes packetCoreDataPlanes() { @@ -431,7 +416,7 @@ public PacketCoreDataPlanes packetCoreDataPlanes() { /** * Gets the resource collection API of Services. It manages Service. - * + * * @return Resource collection API of Services. */ public Services services() { @@ -443,7 +428,7 @@ public Services services() { /** * Gets the resource collection API of Sims. It manages Sim. - * + * * @return Resource collection API of Sims. */ public Sims sims() { @@ -455,7 +440,7 @@ public Sims sims() { /** * Gets the resource collection API of SimGroups. It manages SimGroup. - * + * * @return Resource collection API of SimGroups. */ public SimGroups simGroups() { @@ -467,7 +452,7 @@ public SimGroups simGroups() { /** * Gets the resource collection API of SimPolicies. It manages SimPolicy. - * + * * @return Resource collection API of SimPolicies. */ public SimPolicies simPolicies() { @@ -479,7 +464,7 @@ public SimPolicies simPolicies() { /** * Gets the resource collection API of Sites. It manages Site. - * + * * @return Resource collection API of Sites. */ public Sites sites() { @@ -491,7 +476,7 @@ public Sites sites() { /** * Gets the resource collection API of Slices. It manages Slice. - * + * * @return Resource collection API of Slices. */ public Slices slices() { @@ -501,10 +486,35 @@ public Slices slices() { return slices; } + /** + * Gets the resource collection API of ExtendedUeInformations. + * + * @return Resource collection API of ExtendedUeInformations. + */ + public ExtendedUeInformations extendedUeInformations() { + if (this.extendedUeInformations == null) { + this.extendedUeInformations + = new ExtendedUeInformationsImpl(clientObject.getExtendedUeInformations(), this); + } + return extendedUeInformations; + } + + /** + * Gets the resource collection API of UeInformations. + * + * @return Resource collection API of UeInformations. + */ + public UeInformations ueInformations() { + if (this.ueInformations == null) { + this.ueInformations = new UeInformationsImpl(clientObject.getUeInformations(), this); + } + return ueInformations; + } + /** * Gets wrapped service client MobileNetworkManagementClient providing direct access to the underlying * auto-generated API implementation, based on Azure REST API. - * + * * @return Wrapped service client MobileNetworkManagementClient. */ public MobileNetworkManagementClient serviceClient() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/AttachedDataNetworksClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/AttachedDataNetworksClient.java index 4daa6d05ff5f3..793272bf2d240 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/AttachedDataNetworksClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/AttachedDataNetworksClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.AttachedDataNetworkInner; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in AttachedDataNetworksClient. */ +/** + * An instance of this class provides access to all the operations defined in AttachedDataNetworksClient. + */ public interface AttachedDataNetworksClient { /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -29,15 +31,12 @@ public interface AttachedDataNetworksClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName); /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -49,16 +48,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, Context context); /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -68,15 +63,12 @@ SyncPoller, Void> beginDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName); /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -87,16 +79,12 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context); + void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, Context context); /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -108,16 +96,12 @@ void delete( * @return information about the specified attached data network along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, Context context); /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -128,16 +112,13 @@ Response getWithResponse( * @return information about the specified attached data network. */ @ServiceMethod(returns = ReturnType.SINGLE) - AttachedDataNetworkInner get( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName); + AttachedDataNetworkInner get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName); /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -150,16 +131,13 @@ AttachedDataNetworkInner get( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, AttachedDataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, AttachedDataNetworkInner parameters); /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -173,17 +151,13 @@ SyncPoller, AttachedDataNetworkInner> begin */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, AttachedDataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, - Context context); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, AttachedDataNetworkInner parameters, Context context); /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -195,17 +169,13 @@ SyncPoller, AttachedDataNetworkInner> begin * @return attached data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - AttachedDataNetworkInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters); + AttachedDataNetworkInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, AttachedDataNetworkInner parameters); /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -218,17 +188,13 @@ AttachedDataNetworkInner createOrUpdate( * @return attached data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - AttachedDataNetworkInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, + AttachedDataNetworkInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, AttachedDataNetworkInner parameters, Context context); /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -241,17 +207,13 @@ AttachedDataNetworkInner createOrUpdate( * @return attached data network resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - TagsObject parameters, - Context context); + Response updateTagsWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, + TagsObject parameters, Context context); /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -263,32 +225,28 @@ Response updateTagsWithResponse( * @return attached data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - AttachedDataNetworkInner updateTags( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - TagsObject parameters); + AttachedDataNetworkInner updateTags(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, TagsObject parameters); /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedIterable}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName); + PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName); /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -296,10 +254,10 @@ PagedIterable listByPacketCoreDataPlane( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedIterable}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DataNetworksClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DataNetworksClient.java index f11264eb54147..4e83b325463d5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DataNetworksClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DataNetworksClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.DataNetworkInner; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in DataNetworksClient. */ +/** + * An instance of this class provides access to all the operations defined in DataNetworksClient. + */ public interface DataNetworksClient { /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -28,12 +30,12 @@ public interface DataNetworksClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String dataNetworkName); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String dataNetworkName); /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -44,12 +46,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context); /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -62,7 +64,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -76,7 +78,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -87,12 +89,12 @@ SyncPoller, Void> beginDelete( * @return information about the specified data network along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context); /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -106,7 +108,7 @@ Response getWithResponse( /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -117,12 +119,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of data network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters); + SyncPoller, DataNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters); /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -134,16 +136,12 @@ SyncPoller, DataNetworkInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of data network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, - Context context); + SyncPoller, DataNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters, Context context); /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -154,12 +152,12 @@ SyncPoller, DataNetworkInner> beginCreateOrUpdate( * @return data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DataNetworkInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters); + DataNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + DataNetworkInner parameters); /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -171,16 +169,12 @@ DataNetworkInner createOrUpdate( * @return data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DataNetworkInner createOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, - Context context); + DataNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + DataNetworkInner parameters, Context context); /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -192,16 +186,12 @@ DataNetworkInner createOrUpdate( * @return data network resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - TagsObject parameters, - Context context); + Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, TagsObject parameters, Context context); /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -212,12 +202,12 @@ Response updateTagsWithResponse( * @return data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DataNetworkInner updateTags( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, TagsObject parameters); + DataNetworkInner updateTags(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + TagsObject parameters); /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -230,7 +220,7 @@ DataNetworkInner updateTags( /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -240,6 +230,6 @@ DataNetworkInner updateTags( * @return response for data network API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context); + PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DiagnosticsPackagesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DiagnosticsPackagesClient.java index f7cbfd5a3fa4d..39eb4255f92d5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DiagnosticsPackagesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/DiagnosticsPackagesClient.java @@ -13,11 +13,13 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.mobilenetwork.fluent.models.DiagnosticsPackageInner; -/** An instance of this class provides access to all the operations defined in DiagnosticsPackagesClient. */ +/** + * An instance of this class provides access to all the operations defined in DiagnosticsPackagesClient. + */ public interface DiagnosticsPackagesClient { /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -27,12 +29,12 @@ public interface DiagnosticsPackagesClient { * @return the {@link SyncPoller} for polling of diagnostics package resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DiagnosticsPackageInner> beginCreateOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName); + SyncPoller, DiagnosticsPackageInner> + beginCreateOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName); /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -48,7 +50,7 @@ SyncPoller, DiagnosticsPackageInner> beginCr /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -58,12 +60,12 @@ SyncPoller, DiagnosticsPackageInner> beginCr * @return diagnostics package resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DiagnosticsPackageInner createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName); + DiagnosticsPackageInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName); /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -74,12 +76,12 @@ DiagnosticsPackageInner createOrUpdate( * @return diagnostics package resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DiagnosticsPackageInner createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + DiagnosticsPackageInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context); /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -90,12 +92,12 @@ DiagnosticsPackageInner createOrUpdate( * @return information about the specified diagnostics package along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context); /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -105,12 +107,12 @@ Response getWithResponse( * @return information about the specified diagnostics package. */ @ServiceMethod(returns = ReturnType.SINGLE) - DiagnosticsPackageInner get( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName); + DiagnosticsPackageInner get(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName); /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -120,12 +122,12 @@ DiagnosticsPackageInner get( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName); /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -136,12 +138,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context); /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -154,7 +156,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -164,12 +166,12 @@ SyncPoller, Void> beginDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + void delete(String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, + Context context); /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -178,12 +180,12 @@ void delete( * @return response for diagnostics package API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -193,6 +195,6 @@ PagedIterable listByPacketCoreControlPlane( * @return response for diagnostics package API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ExtendedUeInformationsClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ExtendedUeInformationsClient.java new file mode 100644 index 0000000000000..17ee5153b9aa5 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ExtendedUeInformationsClient.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner; + +/** + * An instance of this class provides access to all the operations defined in ExtendedUeInformationsClient. + */ +public interface ExtendedUeInformationsClient { + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String ueId, Context context); + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ExtendedUeInfoInner get(String resourceGroupName, String packetCoreControlPlaneName, String ueId); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworkManagementClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworkManagementClient.java index f54ea636bd3b1..aa6b05de469ab 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworkManagementClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworkManagementClient.java @@ -7,145 +7,161 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for MobileNetworkManagementClient class. */ +/** + * The interface for MobileNetworkManagementClient class. + */ public interface MobileNetworkManagementClient { /** * Gets The ID of the target subscription. The value must be an UUID. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the AttachedDataNetworksClient object to access its operations. - * + * * @return the AttachedDataNetworksClient object. */ AttachedDataNetworksClient getAttachedDataNetworks(); /** * Gets the DataNetworksClient object to access its operations. - * + * * @return the DataNetworksClient object. */ DataNetworksClient getDataNetworks(); /** * Gets the DiagnosticsPackagesClient object to access its operations. - * + * * @return the DiagnosticsPackagesClient object. */ DiagnosticsPackagesClient getDiagnosticsPackages(); /** * Gets the MobileNetworksClient object to access its operations. - * + * * @return the MobileNetworksClient object. */ MobileNetworksClient getMobileNetworks(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the PacketCapturesClient object to access its operations. - * + * * @return the PacketCapturesClient object. */ PacketCapturesClient getPacketCaptures(); /** * Gets the PacketCoreControlPlanesClient object to access its operations. - * + * * @return the PacketCoreControlPlanesClient object. */ PacketCoreControlPlanesClient getPacketCoreControlPlanes(); /** * Gets the PacketCoreControlPlaneVersionsClient object to access its operations. - * + * * @return the PacketCoreControlPlaneVersionsClient object. */ PacketCoreControlPlaneVersionsClient getPacketCoreControlPlaneVersions(); /** * Gets the PacketCoreDataPlanesClient object to access its operations. - * + * * @return the PacketCoreDataPlanesClient object. */ PacketCoreDataPlanesClient getPacketCoreDataPlanes(); /** * Gets the ServicesClient object to access its operations. - * + * * @return the ServicesClient object. */ ServicesClient getServices(); /** * Gets the SimsClient object to access its operations. - * + * * @return the SimsClient object. */ SimsClient getSims(); /** * Gets the SimGroupsClient object to access its operations. - * + * * @return the SimGroupsClient object. */ SimGroupsClient getSimGroups(); /** * Gets the SimPoliciesClient object to access its operations. - * + * * @return the SimPoliciesClient object. */ SimPoliciesClient getSimPolicies(); /** * Gets the SitesClient object to access its operations. - * + * * @return the SitesClient object. */ SitesClient getSites(); /** * Gets the SlicesClient object to access its operations. - * + * * @return the SlicesClient object. */ SlicesClient getSlices(); + + /** + * Gets the ExtendedUeInformationsClient object to access its operations. + * + * @return the ExtendedUeInformationsClient object. + */ + ExtendedUeInformationsClient getExtendedUeInformations(); + + /** + * Gets the UeInformationsClient object to access its operations. + * + * @return the UeInformationsClient object. + */ + UeInformationsClient getUeInformations(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworksClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworksClient.java index 62743246b2088..afe882b5d6195 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworksClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/MobileNetworksClient.java @@ -12,13 +12,15 @@ import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.mobilenetwork.fluent.models.MobileNetworkInner; -import com.azure.resourcemanager.mobilenetwork.models.TagsObject; +import com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject; -/** An instance of this class provides access to all the operations defined in MobileNetworksClient. */ +/** + * An instance of this class provides access to all the operations defined in MobileNetworksClient. + */ public interface MobileNetworksClient { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -31,7 +33,7 @@ public interface MobileNetworksClient { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -45,7 +47,7 @@ public interface MobileNetworksClient { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -57,7 +59,7 @@ public interface MobileNetworksClient { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -70,7 +72,7 @@ public interface MobileNetworksClient { /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -80,12 +82,12 @@ public interface MobileNetworksClient { * @return information about the specified mobile network along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String mobileNetworkName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String mobileNetworkName, + Context context); /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -98,7 +100,7 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -108,12 +110,12 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of mobile network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, MobileNetworkInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters); + SyncPoller, MobileNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, MobileNetworkInner parameters); /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -124,12 +126,12 @@ SyncPoller, MobileNetworkInner> beginCreateOrUpda * @return the {@link SyncPoller} for polling of mobile network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, MobileNetworkInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context); + SyncPoller, MobileNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, MobileNetworkInner parameters, Context context); /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -139,12 +141,12 @@ SyncPoller, MobileNetworkInner> beginCreateOrUpda * @return mobile network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - MobileNetworkInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters); + MobileNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, + MobileNetworkInner parameters); /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -155,15 +157,15 @@ MobileNetworkInner createOrUpdate( * @return mobile network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - MobileNetworkInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context); + MobileNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, + Context context); /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -171,26 +173,26 @@ MobileNetworkInner createOrUpdate( * @return mobile network resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, TagsObject parameters, Context context); + Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + IdentityAndTagsObject parameters, Context context); /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return mobile network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - MobileNetworkInner updateTags(String resourceGroupName, String mobileNetworkName, TagsObject parameters); + MobileNetworkInner updateTags(String resourceGroupName, String mobileNetworkName, IdentityAndTagsObject parameters); /** * Lists all the mobile networks in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call as paginated response with {@link PagedIterable}. @@ -200,7 +202,7 @@ Response updateTagsWithResponse( /** * Lists all the mobile networks in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -212,7 +214,7 @@ Response updateTagsWithResponse( /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -224,7 +226,7 @@ Response updateTagsWithResponse( /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/OperationsClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/OperationsClient.java index 1178375ea3c2b..06fde2dbc79fe 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/OperationsClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/OperationsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.mobilenetwork.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Gets a list of the operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of the operations as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface OperationsClient { /** * Gets a list of the operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCapturesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCapturesClient.java index 15aad7ac98cda..83ab40a6ab1a3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCapturesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCapturesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.AsyncOperationStatusInner; import com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCaptureInner; -/** An instance of this class provides access to all the operations defined in PacketCapturesClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCapturesClient. + */ public interface PacketCapturesClient { /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -29,15 +31,12 @@ public interface PacketCapturesClient { * @return the {@link SyncPoller} for polling of packet capture session resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, PacketCaptureInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters); + SyncPoller, PacketCaptureInner> beginCreateOrUpdate(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters); /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -49,16 +48,12 @@ SyncPoller, PacketCaptureInner> beginCreateOrUpda * @return the {@link SyncPoller} for polling of packet capture session resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, PacketCaptureInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context); + SyncPoller, PacketCaptureInner> beginCreateOrUpdate(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters, Context context); /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -69,15 +64,12 @@ SyncPoller, PacketCaptureInner> beginCreateOrUpda * @return packet capture session resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCaptureInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters); + PacketCaptureInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, PacketCaptureInner parameters); /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -89,16 +81,12 @@ PacketCaptureInner createOrUpdate( * @return packet capture session resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCaptureInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context); + PacketCaptureInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, PacketCaptureInner parameters, Context context); /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -109,12 +97,12 @@ PacketCaptureInner createOrUpdate( * @return information about the specified packet capture session along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context); /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -128,7 +116,7 @@ Response getWithResponse( /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -138,12 +126,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName); /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -154,12 +142,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context); /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -172,7 +160,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -186,7 +174,7 @@ SyncPoller, Void> beginDelete( /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -196,12 +184,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginStop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName); + SyncPoller, AsyncOperationStatusInner> beginStop(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName); /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -212,12 +200,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginStop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context); + SyncPoller, AsyncOperationStatusInner> beginStop(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, Context context); /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -227,12 +215,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName); + AsyncOperationStatusInner stop(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName); /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -243,12 +231,12 @@ AsyncOperationStatusInner stop( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context); + AsyncOperationStatusInner stop(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context); /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -257,12 +245,12 @@ AsyncOperationStatusInner stop( * @return response for packet capture API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -272,6 +260,6 @@ PagedIterable listByPacketCoreControlPlane( * @return response for packet capture API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlaneVersionsClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlaneVersionsClient.java index 5592f4a53e402..8e5c93cc61ef3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlaneVersionsClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlaneVersionsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreControlPlaneVersionInner; -/** An instance of this class provides access to all the operations defined in PacketCoreControlPlaneVersionsClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCoreControlPlaneVersionsClient. + */ public interface PacketCoreControlPlaneVersionsClient { /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -28,7 +30,7 @@ public interface PacketCoreControlPlaneVersionsClient { /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -40,31 +42,31 @@ public interface PacketCoreControlPlaneVersionsClient { /** * Lists all supported packet core control planes versions. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -77,7 +79,7 @@ public interface PacketCoreControlPlaneVersionsClient { /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -89,24 +91,24 @@ public interface PacketCoreControlPlaneVersionsClient { /** * Lists all supported packet core control planes versions. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listBySubscription(); /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listBySubscription(Context context); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlanesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlanesClient.java index 4df1a5d532d31..1335db721884e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlanesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreControlPlanesClient.java @@ -16,11 +16,13 @@ import com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject; import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneCollectDiagnosticsPackage; -/** An instance of this class provides access to all the operations defined in PacketCoreControlPlanesClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCoreControlPlanesClient. + */ public interface PacketCoreControlPlanesClient { /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -33,7 +35,7 @@ public interface PacketCoreControlPlanesClient { /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -43,12 +45,12 @@ public interface PacketCoreControlPlanesClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + Context context); /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -60,7 +62,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -73,7 +75,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -83,12 +85,12 @@ SyncPoller, Void> beginDelete( * @return information about the specified packet core control plane along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, Context context); /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -101,7 +103,7 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -116,7 +118,7 @@ SyncPoller, PacketCoreControlPlaneInner> /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -128,14 +130,12 @@ SyncPoller, PacketCoreControlPlaneInner> */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PacketCoreControlPlaneInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, + String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters, Context context); /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -145,12 +145,12 @@ SyncPoller, PacketCoreControlPlaneInner> * @return packet core control plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreControlPlaneInner createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters); + PacketCoreControlPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneInner parameters); /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -161,15 +161,12 @@ PacketCoreControlPlaneInner createOrUpdate( * @return packet core control plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreControlPlaneInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, - Context context); + PacketCoreControlPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneInner parameters, Context context); /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -180,12 +177,12 @@ PacketCoreControlPlaneInner createOrUpdate( * @return packet core control plane resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, IdentityAndTagsObject parameters, Context context); + Response updateTagsWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, IdentityAndTagsObject parameters, Context context); /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -195,56 +192,56 @@ Response updateTagsWithResponse( * @return packet core control plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreControlPlaneInner updateTags( - String resourceGroupName, String packetCoreControlPlaneName, IdentityAndTagsObject parameters); + PacketCoreControlPlaneInner updateTags(String resourceGroupName, String packetCoreControlPlaneName, + IdentityAndTagsObject parameters); /** * Lists all the packet core control planes in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Lists all the packet core control planes in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByResourceGroup(String resourceGroupName); /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByResourceGroup(String resourceGroupName, Context context); @@ -252,7 +249,7 @@ PacketCoreControlPlaneInner updateTags( /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -261,13 +258,13 @@ PacketCoreControlPlaneInner updateTags( * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginRollback( - String resourceGroupName, String packetCoreControlPlaneName); + SyncPoller, AsyncOperationStatusInner> beginRollback(String resourceGroupName, + String packetCoreControlPlaneName); /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -277,13 +274,13 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginRollback( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + SyncPoller, AsyncOperationStatusInner> beginRollback(String resourceGroupName, + String packetCoreControlPlaneName, Context context); /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -297,7 +294,7 @@ SyncPoller, AsyncOperationStatusInner> beg /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -310,9 +307,9 @@ SyncPoller, AsyncOperationStatusInner> beg AsyncOperationStatusInner rollback(String resourceGroupName, String packetCoreControlPlaneName, Context context); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -321,13 +318,13 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginReinstall( - String resourceGroupName, String packetCoreControlPlaneName); + SyncPoller, AsyncOperationStatusInner> + beginReinstall(String resourceGroupName, String packetCoreControlPlaneName); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -337,13 +334,13 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginReinstall( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + SyncPoller, AsyncOperationStatusInner> + beginReinstall(String resourceGroupName, String packetCoreControlPlaneName, Context context); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -355,9 +352,9 @@ SyncPoller, AsyncOperationStatusInner> beg AsyncOperationStatusInner reinstall(String resourceGroupName, String packetCoreControlPlaneName); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -372,7 +369,7 @@ SyncPoller, AsyncOperationStatusInner> beg /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -383,14 +380,13 @@ SyncPoller, AsyncOperationStatusInner> beg */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, AsyncOperationStatusInner> beginCollectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, + String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters); /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -402,15 +398,13 @@ SyncPoller, AsyncOperationStatusInner> beg */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, AsyncOperationStatusInner> beginCollectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - Context context); + String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context); /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -420,15 +414,13 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, + AsyncOperationStatusInner collectDiagnosticsPackage(String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters); /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -439,9 +431,6 @@ AsyncOperationStatusInner collectDiagnosticsPackage( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - Context context); + AsyncOperationStatusInner collectDiagnosticsPackage(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreDataPlanesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreDataPlanesClient.java index a10e26848f585..e442bdc055576 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreDataPlanesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/PacketCoreDataPlanesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreDataPlaneInner; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in PacketCoreDataPlanesClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCoreDataPlanesClient. + */ public interface PacketCoreDataPlanesClient { /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -28,12 +30,12 @@ public interface PacketCoreDataPlanesClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName); /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -44,12 +46,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, Context context); /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -62,7 +64,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -72,12 +74,12 @@ SyncPoller, Void> beginDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + Context context); /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -88,12 +90,12 @@ void delete( * @return information about the specified packet core data plane along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, Context context); /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -103,13 +105,13 @@ Response getWithResponse( * @return information about the specified packet core data plane. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreDataPlaneInner get( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName); + PacketCoreDataPlaneInner get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName); /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -121,15 +123,13 @@ PacketCoreDataPlaneInner get( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PacketCoreDataPlaneInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters); /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -142,16 +142,13 @@ SyncPoller, PacketCoreDataPlaneInner> begin */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PacketCoreDataPlaneInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, - Context context); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + PacketCoreDataPlaneInner parameters, Context context); /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -162,16 +159,13 @@ SyncPoller, PacketCoreDataPlaneInner> begin * @return packet core data plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreDataPlaneInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters); + PacketCoreDataPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters); /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -183,16 +177,12 @@ PacketCoreDataPlaneInner createOrUpdate( * @return packet core data plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreDataPlaneInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, - Context context); + PacketCoreDataPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters, Context context); /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -204,16 +194,12 @@ PacketCoreDataPlaneInner createOrUpdate( * @return packet core data plane resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters, - Context context); + Response updateTagsWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, TagsObject parameters, Context context); /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -224,15 +210,12 @@ Response updateTagsWithResponse( * @return packet core data plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PacketCoreDataPlaneInner updateTags( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters); + PacketCoreDataPlaneInner updateTags(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, TagsObject parameters); /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -241,12 +224,12 @@ PacketCoreDataPlaneInner updateTags( * @return response for packet core data planes API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -256,6 +239,6 @@ PagedIterable listByPacketCoreControlPlane( * @return response for packet core data planes API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ServicesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ServicesClient.java index fec9f99fccf72..462b57fad5c9a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ServicesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/ServicesClient.java @@ -14,31 +14,33 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.ServiceInner; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in ServicesClient. */ +/** + * An instance of this class provides access to all the operations defined in ServicesClient. + */ public interface ServicesClient { /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String serviceName); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String serviceName); /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -46,16 +48,16 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String serviceName, Context context); /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -65,11 +67,11 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -80,11 +82,11 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -92,16 +94,16 @@ SyncPoller, Void> beginDelete( * @return information about the specified service along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String serviceName, + Context context); /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -112,11 +114,11 @@ Response getWithResponse( /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -124,16 +126,16 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of service resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ServiceInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters); + SyncPoller, ServiceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters); /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -142,20 +144,16 @@ SyncPoller, ServiceInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of service resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ServiceInner> beginCreateOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context); + SyncPoller, ServiceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters, Context context); /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -163,16 +161,16 @@ SyncPoller, ServiceInner> beginCreateOrUpdate( * @return service resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServiceInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters); + ServiceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String serviceName, + ServiceInner parameters); /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -181,20 +179,16 @@ ServiceInner createOrUpdate( * @return service resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServiceInner createOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context); + ServiceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String serviceName, + ServiceInner parameters, Context context); /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -203,16 +197,16 @@ ServiceInner createOrUpdate( * @return service resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, String serviceName, TagsObject parameters, Context context); + Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String serviceName, TagsObject parameters, Context context); /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -220,12 +214,12 @@ Response updateTagsWithResponse( * @return service resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServiceInner updateTags( - String resourceGroupName, String mobileNetworkName, String serviceName, TagsObject parameters); + ServiceInner updateTags(String resourceGroupName, String mobileNetworkName, String serviceName, + TagsObject parameters); /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -238,7 +232,7 @@ ServiceInner updateTags( /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -248,6 +242,6 @@ ServiceInner updateTags( * @return all the services in a mobile network as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context); + PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimGroupsClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimGroupsClient.java index 7fed75af24f81..4eb1106ef7627 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimGroupsClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimGroupsClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.SimGroupInner; import com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject; -/** An instance of this class provides access to all the operations defined in SimGroupsClient. */ +/** + * An instance of this class provides access to all the operations defined in SimGroupsClient. + */ public interface SimGroupsClient { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -31,7 +33,7 @@ public interface SimGroupsClient { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -45,7 +47,7 @@ public interface SimGroupsClient { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -57,7 +59,7 @@ public interface SimGroupsClient { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -70,7 +72,7 @@ public interface SimGroupsClient { /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -80,12 +82,12 @@ public interface SimGroupsClient { * @return information about the specified SIM group along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String simGroupName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String simGroupName, + Context context); /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -98,7 +100,7 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -108,12 +110,12 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of sIM group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SimGroupInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, SimGroupInner parameters); + SyncPoller, SimGroupInner> beginCreateOrUpdate(String resourceGroupName, + String simGroupName, SimGroupInner parameters); /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -124,12 +126,12 @@ SyncPoller, SimGroupInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of sIM group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SimGroupInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context); + SyncPoller, SimGroupInner> beginCreateOrUpdate(String resourceGroupName, + String simGroupName, SimGroupInner parameters, Context context); /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -143,7 +145,7 @@ SyncPoller, SimGroupInner> beginCreateOrUpdate( /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -154,12 +156,12 @@ SyncPoller, SimGroupInner> beginCreateOrUpdate( * @return sIM group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SimGroupInner createOrUpdate( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context); + SimGroupInner createOrUpdate(String resourceGroupName, String simGroupName, SimGroupInner parameters, + Context context); /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -170,12 +172,12 @@ SimGroupInner createOrUpdate( * @return sIM group resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String simGroupName, IdentityAndTagsObject parameters, Context context); + Response updateTagsWithResponse(String resourceGroupName, String simGroupName, + IdentityAndTagsObject parameters, Context context); /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -189,7 +191,7 @@ Response updateTagsWithResponse( /** * Gets all the SIM groups in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM groups in a subscription as paginated response with {@link PagedIterable}. @@ -199,7 +201,7 @@ Response updateTagsWithResponse( /** * Gets all the SIM groups in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -211,7 +213,7 @@ Response updateTagsWithResponse( /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -223,7 +225,7 @@ Response updateTagsWithResponse( /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimPoliciesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimPoliciesClient.java index d14ddbe59abdc..23427f54f8e58 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimPoliciesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimPoliciesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.SimPolicyInner; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in SimPoliciesClient. */ +/** + * An instance of this class provides access to all the operations defined in SimPoliciesClient. + */ public interface SimPoliciesClient { /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -28,12 +30,12 @@ public interface SimPoliciesClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String simPolicyName); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String simPolicyName); /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -44,12 +46,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String simPolicyName, Context context); /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -62,7 +64,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -76,7 +78,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -87,12 +89,12 @@ SyncPoller, Void> beginDelete( * @return information about the specified SIM policy along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String simPolicyName, + Context context); /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -106,7 +108,7 @@ Response getWithResponse( /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -117,12 +119,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of sIM policy resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SimPolicyInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters); + SyncPoller, SimPolicyInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters); /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -134,16 +136,12 @@ SyncPoller, SimPolicyInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of sIM policy resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SimPolicyInner> beginCreateOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context); + SyncPoller, SimPolicyInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters, Context context); /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -154,12 +152,12 @@ SyncPoller, SimPolicyInner> beginCreateOrUpdate( * @return sIM policy resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SimPolicyInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters); + SimPolicyInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String simPolicyName, + SimPolicyInner parameters); /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -171,16 +169,12 @@ SimPolicyInner createOrUpdate( * @return sIM policy resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SimPolicyInner createOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context); + SimPolicyInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String simPolicyName, + SimPolicyInner parameters, Context context); /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -192,16 +186,12 @@ SimPolicyInner createOrUpdate( * @return sIM policy resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - TagsObject parameters, - Context context); + Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String simPolicyName, TagsObject parameters, Context context); /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -212,12 +202,12 @@ Response updateTagsWithResponse( * @return sIM policy resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SimPolicyInner updateTags( - String resourceGroupName, String mobileNetworkName, String simPolicyName, TagsObject parameters); + SimPolicyInner updateTags(String resourceGroupName, String mobileNetworkName, String simPolicyName, + TagsObject parameters); /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -230,7 +220,7 @@ SimPolicyInner updateTags( /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -240,6 +230,6 @@ SimPolicyInner updateTags( * @return all the SIM policies in a mobile network as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context); + PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimsClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimsClient.java index c2bb2940ad272..6155d2c7b91ff 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimsClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SimsClient.java @@ -17,11 +17,13 @@ import com.azure.resourcemanager.mobilenetwork.models.SimDeleteList; import com.azure.resourcemanager.mobilenetwork.models.SimUploadList; -/** An instance of this class provides access to all the operations defined in SimsClient. */ +/** + * An instance of this class provides access to all the operations defined in SimsClient. + */ public interface SimsClient { /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -35,7 +37,7 @@ public interface SimsClient { /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -46,12 +48,12 @@ public interface SimsClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String simGroupName, String simName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String simGroupName, String simName, + Context context); /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -64,7 +66,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -78,7 +80,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -93,7 +95,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -107,7 +109,7 @@ SyncPoller, Void> beginDelete( /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -118,12 +120,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of sIM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SimInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, String simName, SimInner parameters); + SyncPoller, SimInner> beginCreateOrUpdate(String resourceGroupName, String simGroupName, + String simName, SimInner parameters); /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -135,12 +137,12 @@ SyncPoller, SimInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of sIM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SimInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context); + SyncPoller, SimInner> beginCreateOrUpdate(String resourceGroupName, String simGroupName, + String simName, SimInner parameters, Context context); /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -155,7 +157,7 @@ SyncPoller, SimInner> beginCreateOrUpdate( /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -167,12 +169,12 @@ SyncPoller, SimInner> beginCreateOrUpdate( * @return sIM resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SimInner createOrUpdate( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context); + SimInner createOrUpdate(String resourceGroupName, String simGroupName, String simName, SimInner parameters, + Context context); /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -185,7 +187,7 @@ SimInner createOrUpdate( /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -199,7 +201,7 @@ SimInner createOrUpdate( /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -209,12 +211,12 @@ SimInner createOrUpdate( * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginBulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters); + SyncPoller, AsyncOperationStatusInner> + beginBulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters); /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -225,12 +227,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginBulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context); + SyncPoller, AsyncOperationStatusInner> + beginBulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters, Context context); /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -244,7 +246,7 @@ SyncPoller, AsyncOperationStatusInner> beg /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -255,12 +257,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner bulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context); + AsyncOperationStatusInner bulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters, + Context context); /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -270,12 +272,12 @@ AsyncOperationStatusInner bulkUpload( * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginBulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters); + SyncPoller, AsyncOperationStatusInner> + beginBulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters); /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -286,12 +288,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginBulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context); + SyncPoller, AsyncOperationStatusInner> + beginBulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context); /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -305,7 +307,7 @@ SyncPoller, AsyncOperationStatusInner> beg /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -316,12 +318,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner bulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context); + AsyncOperationStatusInner bulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters, + Context context); /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -331,12 +333,12 @@ AsyncOperationStatusInner bulkDelete( * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AsyncOperationStatusInner> beginBulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters); + SyncPoller, AsyncOperationStatusInner> + beginBulkUploadEncrypted(String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters); /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -352,7 +354,7 @@ SyncPoller, AsyncOperationStatusInner> beg /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -362,12 +364,12 @@ SyncPoller, AsyncOperationStatusInner> beg * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters); + AsyncOperationStatusInner bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters); /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -378,6 +380,6 @@ AsyncOperationStatusInner bulkUploadEncrypted( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - AsyncOperationStatusInner bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context); + AsyncOperationStatusInner bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SitesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SitesClient.java index 38855ed351ad4..34b2a967b712b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SitesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SitesClient.java @@ -15,12 +15,14 @@ import com.azure.resourcemanager.mobilenetwork.models.SiteDeletePacketCore; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in SitesClient. */ +/** + * An instance of this class provides access to all the operations defined in SitesClient. + */ public interface SitesClient { /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -35,7 +37,7 @@ public interface SitesClient { /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -46,13 +48,13 @@ public interface SitesClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String siteName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, String siteName, + Context context); /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -66,7 +68,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -80,7 +82,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -91,12 +93,12 @@ SyncPoller, Void> beginDelete( * @return information about the specified mobile network site along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String siteName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String siteName, + Context context); /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -110,7 +112,7 @@ Response getWithResponse( /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -121,12 +123,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of site resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SiteInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters); + SyncPoller, SiteInner> beginCreateOrUpdate(String resourceGroupName, String mobileNetworkName, + String siteName, SiteInner parameters); /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -138,12 +140,12 @@ SyncPoller, SiteInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of site resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SiteInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context); + SyncPoller, SiteInner> beginCreateOrUpdate(String resourceGroupName, String mobileNetworkName, + String siteName, SiteInner parameters, Context context); /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -158,7 +160,7 @@ SyncPoller, SiteInner> beginCreateOrUpdate( /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -170,12 +172,12 @@ SyncPoller, SiteInner> beginCreateOrUpdate( * @return site resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SiteInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context); + SiteInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, + Context context); /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -187,12 +189,12 @@ SiteInner createOrUpdate( * @return site resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, String siteName, TagsObject parameters, Context context); + Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, String siteName, + TagsObject parameters, Context context); /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -207,7 +209,7 @@ Response updateTagsWithResponse( /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -220,7 +222,7 @@ Response updateTagsWithResponse( /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -234,7 +236,7 @@ Response updateTagsWithResponse( /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -245,12 +247,12 @@ Response updateTagsWithResponse( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDeletePacketCore( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters); + SyncPoller, Void> beginDeletePacketCore(String resourceGroupName, String mobileNetworkName, + String siteName, SiteDeletePacketCore parameters); /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -262,16 +264,12 @@ SyncPoller, Void> beginDeletePacketCore( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDeletePacketCore( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context); + SyncPoller, Void> beginDeletePacketCore(String resourceGroupName, String mobileNetworkName, + String siteName, SiteDeletePacketCore parameters, Context context); /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -281,12 +279,12 @@ SyncPoller, Void> beginDeletePacketCore( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void deletePacketCore( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters); + void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters); /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -297,10 +295,6 @@ void deletePacketCore( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void deletePacketCore( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context); + void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SlicesClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SlicesClient.java index 0fad1d899bbca..a72dbc9031513 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SlicesClient.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/SlicesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.SliceInner; import com.azure.resourcemanager.mobilenetwork.models.TagsObject; -/** An instance of this class provides access to all the operations defined in SlicesClient. */ +/** + * An instance of this class provides access to all the operations defined in SlicesClient. + */ public interface SlicesClient { /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -28,12 +30,12 @@ public interface SlicesClient { * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String sliceName); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String sliceName); /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -44,12 +46,12 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, String sliceName, + Context context); /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -62,7 +64,7 @@ SyncPoller, Void> beginDelete( /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -76,7 +78,7 @@ SyncPoller, Void> beginDelete( /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -87,12 +89,12 @@ SyncPoller, Void> beginDelete( * @return information about the specified network slice along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String sliceName, + Context context); /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -106,7 +108,7 @@ Response getWithResponse( /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -117,12 +119,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of network slice resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SliceInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters); + SyncPoller, SliceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters); /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -134,12 +136,12 @@ SyncPoller, SliceInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of network slice resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, SliceInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context); + SyncPoller, SliceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters, Context context); /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -150,12 +152,12 @@ SyncPoller, SliceInner> beginCreateOrUpdate( * @return network slice resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SliceInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters); + SliceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String sliceName, + SliceInner parameters); /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -167,12 +169,12 @@ SliceInner createOrUpdate( * @return network slice resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SliceInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context); + SliceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String sliceName, + SliceInner parameters, Context context); /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -184,12 +186,12 @@ SliceInner createOrUpdate( * @return network slice resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, String sliceName, TagsObject parameters, Context context); + Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, String sliceName, + TagsObject parameters, Context context); /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -204,7 +206,7 @@ Response updateTagsWithResponse( /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -217,7 +219,7 @@ Response updateTagsWithResponse( /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/UeInformationsClient.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/UeInformationsClient.java new file mode 100644 index 0000000000000..d5f26dffa06a3 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/UeInformationsClient.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; + +/** + * An instance of this class provides access to all the operations defined in UeInformationsClient. + */ +public interface UeInformationsClient { + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName); + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName, Context context); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AsyncOperationStatusInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AsyncOperationStatusInner.java index c28654d049610..6801b3e928cd6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AsyncOperationStatusInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AsyncOperationStatusInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** The current status of an async operation. */ +/** + * The current status of an async operation. + */ @Fluent public final class AsyncOperationStatusInner { /* @@ -67,13 +69,15 @@ public final class AsyncOperationStatusInner { @JsonProperty(value = "error") private ManagementError error; - /** Creates an instance of AsyncOperationStatusInner class. */ + /** + * Creates an instance of AsyncOperationStatusInner class. + */ public AsyncOperationStatusInner() { } /** * Get the id property: Fully qualified ID for the async operation. - * + * * @return the id value. */ public String id() { @@ -82,7 +86,7 @@ public String id() { /** * Set the id property: Fully qualified ID for the async operation. - * + * * @param id the id value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -93,7 +97,7 @@ public AsyncOperationStatusInner withId(String id) { /** * Get the name property: Name of the async operation. - * + * * @return the name value. */ public String name() { @@ -102,7 +106,7 @@ public String name() { /** * Set the name property: Name of the async operation. - * + * * @param name the name value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -113,7 +117,7 @@ public AsyncOperationStatusInner withName(String name) { /** * Get the status property: The operation status. - * + * * @return the status value. */ public String status() { @@ -122,7 +126,7 @@ public String status() { /** * Set the status property: The operation status. - * + * * @param status the status value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -133,7 +137,7 @@ public AsyncOperationStatusInner withStatus(String status) { /** * Get the resourceId property: Fully qualified ID for the resource that this async operation status relates to. - * + * * @return the resourceId value. */ public String resourceId() { @@ -142,7 +146,7 @@ public String resourceId() { /** * Set the resourceId property: Fully qualified ID for the resource that this async operation status relates to. - * + * * @param resourceId the resourceId value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -153,7 +157,7 @@ public AsyncOperationStatusInner withResourceId(String resourceId) { /** * Get the startTime property: The start time of the operation. - * + * * @return the startTime value. */ public OffsetDateTime startTime() { @@ -162,7 +166,7 @@ public OffsetDateTime startTime() { /** * Set the startTime property: The start time of the operation. - * + * * @param startTime the startTime value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -173,7 +177,7 @@ public AsyncOperationStatusInner withStartTime(OffsetDateTime startTime) { /** * Get the endTime property: The end time of the operation. - * + * * @return the endTime value. */ public OffsetDateTime endTime() { @@ -182,7 +186,7 @@ public OffsetDateTime endTime() { /** * Set the endTime property: The end time of the operation. - * + * * @param endTime the endTime value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -193,7 +197,7 @@ public AsyncOperationStatusInner withEndTime(OffsetDateTime endTime) { /** * Get the percentComplete property: Percentage of the operation that is complete. - * + * * @return the percentComplete value. */ public Double percentComplete() { @@ -202,7 +206,7 @@ public Double percentComplete() { /** * Set the percentComplete property: Percentage of the operation that is complete. - * + * * @param percentComplete the percentComplete value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -213,7 +217,7 @@ public AsyncOperationStatusInner withPercentComplete(Double percentComplete) { /** * Get the properties property: Properties returned by the resource provider on a successful operation. - * + * * @return the properties value. */ public Object properties() { @@ -222,7 +226,7 @@ public Object properties() { /** * Set the properties property: Properties returned by the resource provider on a successful operation. - * + * * @param properties the properties value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -233,7 +237,7 @@ public AsyncOperationStatusInner withProperties(Object properties) { /** * Get the error property: If present, details of the operation error. - * + * * @return the error value. */ public ManagementError error() { @@ -242,7 +246,7 @@ public ManagementError error() { /** * Set the error property: If present, details of the operation error. - * + * * @param error the error value to set. * @return the AsyncOperationStatusInner object itself. */ @@ -253,15 +257,13 @@ public AsyncOperationStatusInner withError(ManagementError error) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (status() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property status in model AsyncOperationStatusInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property status in model AsyncOperationStatusInner")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkInner.java index 2350b81bf6814..86eb3a00d6ec8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkInner.java @@ -15,7 +15,9 @@ import java.util.List; import java.util.Map; -/** Attached data network resource. Must be created in the same location as its parent packet core data plane. */ +/** + * Attached data network resource. Must be created in the same location as its parent packet core data plane. + */ @Fluent public final class AttachedDataNetworkInner extends Resource { /* @@ -30,13 +32,15 @@ public final class AttachedDataNetworkInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of AttachedDataNetworkInner class. */ + /** + * Creates an instance of AttachedDataNetworkInner class. + */ public AttachedDataNetworkInner() { } /** * Get the innerProperties property: Attached data network properties. - * + * * @return the innerProperties value. */ private AttachedDataNetworkPropertiesFormat innerProperties() { @@ -45,21 +49,25 @@ private AttachedDataNetworkPropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AttachedDataNetworkInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AttachedDataNetworkInner withTags(Map tags) { super.withTags(tags); @@ -68,7 +76,7 @@ public AttachedDataNetworkInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the attached data network resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -78,7 +86,7 @@ public ProvisioningState provisioningState() { /** * Get the userPlaneDataInterface property: The user plane interface on the data network. For 5G networks, this is * the N6 interface. For 4G networks, this is the SGi interface. - * + * * @return the userPlaneDataInterface value. */ public InterfaceProperties userPlaneDataInterface() { @@ -88,7 +96,7 @@ public InterfaceProperties userPlaneDataInterface() { /** * Set the userPlaneDataInterface property: The user plane interface on the data network. For 5G networks, this is * the N6 interface. For 4G networks, this is the SGi interface. - * + * * @param userPlaneDataInterface the userPlaneDataInterface value to set. * @return the AttachedDataNetworkInner object itself. */ @@ -103,7 +111,7 @@ public AttachedDataNetworkInner withUserPlaneDataInterface(InterfaceProperties u /** * Get the dnsAddresses property: The DNS servers to signal to UEs to use for this attached data network. This * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. - * + * * @return the dnsAddresses value. */ public List dnsAddresses() { @@ -113,7 +121,7 @@ public List dnsAddresses() { /** * Set the dnsAddresses property: The DNS servers to signal to UEs to use for this attached data network. This * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. - * + * * @param dnsAddresses the dnsAddresses value to set. * @return the AttachedDataNetworkInner object itself. */ @@ -126,9 +134,9 @@ public AttachedDataNetworkInner withDnsAddresses(List dnsAddresses) { } /** - * Get the naptConfiguration property: The network address and port translation (NAPT) configuration. If this is not - * specified, the attached data network will use a default NAPT configuration with NAPT enabled. - * + * Get the naptConfiguration property: The network address and port translation (NAPT) configuration. + * If this is not specified, the attached data network will use a default NAPT configuration with NAPT enabled. + * * @return the naptConfiguration value. */ public NaptConfiguration naptConfiguration() { @@ -136,9 +144,9 @@ public NaptConfiguration naptConfiguration() { } /** - * Set the naptConfiguration property: The network address and port translation (NAPT) configuration. If this is not - * specified, the attached data network will use a default NAPT configuration with NAPT enabled. - * + * Set the naptConfiguration property: The network address and port translation (NAPT) configuration. + * If this is not specified, the attached data network will use a default NAPT configuration with NAPT enabled. + * * @param naptConfiguration the naptConfiguration value to set. * @return the AttachedDataNetworkInner object itself. */ @@ -152,11 +160,11 @@ public AttachedDataNetworkInner withNaptConfiguration(NaptConfiguration naptConf /** * Get the userEquipmentAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the attached - * data network from which the packet core instance will dynamically assign IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. You must define at least one of - * userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you define both, they must be of the - * same size. - * + * data network from which the packet core instance will dynamically assign IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you + * define both, they must be of the same size. + * * @return the userEquipmentAddressPoolPrefix value. */ public List userEquipmentAddressPoolPrefix() { @@ -165,11 +173,11 @@ public List userEquipmentAddressPoolPrefix() { /** * Set the userEquipmentAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the attached - * data network from which the packet core instance will dynamically assign IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. You must define at least one of - * userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you define both, they must be of the - * same size. - * + * data network from which the packet core instance will dynamically assign IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you + * define both, they must be of the same size. + * * @param userEquipmentAddressPoolPrefix the userEquipmentAddressPoolPrefix value to set. * @return the AttachedDataNetworkInner object itself. */ @@ -183,11 +191,12 @@ public AttachedDataNetworkInner withUserEquipmentAddressPoolPrefix(List /** * Get the userEquipmentStaticAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the - * attached data network from which the packet core instance will assign static IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address for a specific UE - * is set in StaticIPConfiguration on the corresponding SIM resource. At least one of userEquipmentAddressPoolPrefix - * and userEquipmentStaticAddressPoolPrefix must be defined. If both are defined, they must be of the same size. - * + * attached data network from which the packet core instance will assign static IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address + * for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. If both + * are defined, they must be of the same size. + * * @return the userEquipmentStaticAddressPoolPrefix value. */ public List userEquipmentStaticAddressPoolPrefix() { @@ -196,16 +205,17 @@ public List userEquipmentStaticAddressPoolPrefix() { /** * Set the userEquipmentStaticAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the - * attached data network from which the packet core instance will assign static IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address for a specific UE - * is set in StaticIPConfiguration on the corresponding SIM resource. At least one of userEquipmentAddressPoolPrefix - * and userEquipmentStaticAddressPoolPrefix must be defined. If both are defined, they must be of the same size. - * + * attached data network from which the packet core instance will assign static IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address + * for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. If both + * are defined, they must be of the same size. + * * @param userEquipmentStaticAddressPoolPrefix the userEquipmentStaticAddressPoolPrefix value to set. * @return the AttachedDataNetworkInner object itself. */ - public AttachedDataNetworkInner withUserEquipmentStaticAddressPoolPrefix( - List userEquipmentStaticAddressPoolPrefix) { + public AttachedDataNetworkInner + withUserEquipmentStaticAddressPoolPrefix(List userEquipmentStaticAddressPoolPrefix) { if (this.innerProperties() == null) { this.innerProperties = new AttachedDataNetworkPropertiesFormat(); } @@ -215,15 +225,13 @@ public AttachedDataNetworkInner withUserEquipmentStaticAddressPoolPrefix( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model AttachedDataNetworkInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerProperties in model AttachedDataNetworkInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkPropertiesFormat.java index d2e3f88ed0e78..ac31268da2586 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/AttachedDataNetworkPropertiesFormat.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Data network properties. */ +/** + * Data network properties. + */ @Fluent public final class AttachedDataNetworkPropertiesFormat { /* @@ -63,13 +65,15 @@ public final class AttachedDataNetworkPropertiesFormat { @JsonProperty(value = "userEquipmentStaticAddressPoolPrefix") private List userEquipmentStaticAddressPoolPrefix; - /** Creates an instance of AttachedDataNetworkPropertiesFormat class. */ + /** + * Creates an instance of AttachedDataNetworkPropertiesFormat class. + */ public AttachedDataNetworkPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the attached data network resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -79,7 +83,7 @@ public ProvisioningState provisioningState() { /** * Get the userPlaneDataInterface property: The user plane interface on the data network. For 5G networks, this is * the N6 interface. For 4G networks, this is the SGi interface. - * + * * @return the userPlaneDataInterface value. */ public InterfaceProperties userPlaneDataInterface() { @@ -89,7 +93,7 @@ public InterfaceProperties userPlaneDataInterface() { /** * Set the userPlaneDataInterface property: The user plane interface on the data network. For 5G networks, this is * the N6 interface. For 4G networks, this is the SGi interface. - * + * * @param userPlaneDataInterface the userPlaneDataInterface value to set. * @return the AttachedDataNetworkPropertiesFormat object itself. */ @@ -101,7 +105,7 @@ public AttachedDataNetworkPropertiesFormat withUserPlaneDataInterface(InterfaceP /** * Get the dnsAddresses property: The DNS servers to signal to UEs to use for this attached data network. This * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. - * + * * @return the dnsAddresses value. */ public List dnsAddresses() { @@ -111,7 +115,7 @@ public List dnsAddresses() { /** * Set the dnsAddresses property: The DNS servers to signal to UEs to use for this attached data network. This * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. - * + * * @param dnsAddresses the dnsAddresses value to set. * @return the AttachedDataNetworkPropertiesFormat object itself. */ @@ -121,9 +125,9 @@ public AttachedDataNetworkPropertiesFormat withDnsAddresses(List dnsAddr } /** - * Get the naptConfiguration property: The network address and port translation (NAPT) configuration. If this is not - * specified, the attached data network will use a default NAPT configuration with NAPT enabled. - * + * Get the naptConfiguration property: The network address and port translation (NAPT) configuration. + * If this is not specified, the attached data network will use a default NAPT configuration with NAPT enabled. + * * @return the naptConfiguration value. */ public NaptConfiguration naptConfiguration() { @@ -131,9 +135,9 @@ public NaptConfiguration naptConfiguration() { } /** - * Set the naptConfiguration property: The network address and port translation (NAPT) configuration. If this is not - * specified, the attached data network will use a default NAPT configuration with NAPT enabled. - * + * Set the naptConfiguration property: The network address and port translation (NAPT) configuration. + * If this is not specified, the attached data network will use a default NAPT configuration with NAPT enabled. + * * @param naptConfiguration the naptConfiguration value to set. * @return the AttachedDataNetworkPropertiesFormat object itself. */ @@ -144,11 +148,11 @@ public AttachedDataNetworkPropertiesFormat withNaptConfiguration(NaptConfigurati /** * Get the userEquipmentAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the attached - * data network from which the packet core instance will dynamically assign IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. You must define at least one of - * userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you define both, they must be of the - * same size. - * + * data network from which the packet core instance will dynamically assign IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you + * define both, they must be of the same size. + * * @return the userEquipmentAddressPoolPrefix value. */ public List userEquipmentAddressPoolPrefix() { @@ -157,27 +161,28 @@ public List userEquipmentAddressPoolPrefix() { /** * Set the userEquipmentAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the attached - * data network from which the packet core instance will dynamically assign IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. You must define at least one of - * userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you define both, they must be of the - * same size. - * + * data network from which the packet core instance will dynamically assign IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you + * define both, they must be of the same size. + * * @param userEquipmentAddressPoolPrefix the userEquipmentAddressPoolPrefix value to set. * @return the AttachedDataNetworkPropertiesFormat object itself. */ - public AttachedDataNetworkPropertiesFormat withUserEquipmentAddressPoolPrefix( - List userEquipmentAddressPoolPrefix) { + public AttachedDataNetworkPropertiesFormat + withUserEquipmentAddressPoolPrefix(List userEquipmentAddressPoolPrefix) { this.userEquipmentAddressPoolPrefix = userEquipmentAddressPoolPrefix; return this; } /** * Get the userEquipmentStaticAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the - * attached data network from which the packet core instance will assign static IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address for a specific UE - * is set in StaticIPConfiguration on the corresponding SIM resource. At least one of userEquipmentAddressPoolPrefix - * and userEquipmentStaticAddressPoolPrefix must be defined. If both are defined, they must be of the same size. - * + * attached data network from which the packet core instance will assign static IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address + * for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. If both + * are defined, they must be of the same size. + * * @return the userEquipmentStaticAddressPoolPrefix value. */ public List userEquipmentStaticAddressPoolPrefix() { @@ -186,40 +191,36 @@ public List userEquipmentStaticAddressPoolPrefix() { /** * Set the userEquipmentStaticAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the - * attached data network from which the packet core instance will assign static IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address for a specific UE - * is set in StaticIPConfiguration on the corresponding SIM resource. At least one of userEquipmentAddressPoolPrefix - * and userEquipmentStaticAddressPoolPrefix must be defined. If both are defined, they must be of the same size. - * + * attached data network from which the packet core instance will assign static IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address + * for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. If both + * are defined, they must be of the same size. + * * @param userEquipmentStaticAddressPoolPrefix the userEquipmentStaticAddressPoolPrefix value to set. * @return the AttachedDataNetworkPropertiesFormat object itself. */ - public AttachedDataNetworkPropertiesFormat withUserEquipmentStaticAddressPoolPrefix( - List userEquipmentStaticAddressPoolPrefix) { + public AttachedDataNetworkPropertiesFormat + withUserEquipmentStaticAddressPoolPrefix(List userEquipmentStaticAddressPoolPrefix) { this.userEquipmentStaticAddressPoolPrefix = userEquipmentStaticAddressPoolPrefix; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (userPlaneDataInterface() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property userPlaneDataInterface in model" - + " AttachedDataNetworkPropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property userPlaneDataInterface in model AttachedDataNetworkPropertiesFormat")); } else { userPlaneDataInterface().validate(); } if (dnsAddresses() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property dnsAddresses in model AttachedDataNetworkPropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property dnsAddresses in model AttachedDataNetworkPropertiesFormat")); } if (naptConfiguration() != null) { naptConfiguration().validate(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkInner.java index 3524789b12004..8118c0362c56c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Data network resource. Must be created in the same location as its parent mobile network. */ +/** + * Data network resource. Must be created in the same location as its parent mobile network. + */ @Fluent public final class DataNetworkInner extends Resource { /* @@ -26,13 +28,15 @@ public final class DataNetworkInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of DataNetworkInner class. */ + /** + * Creates an instance of DataNetworkInner class. + */ public DataNetworkInner() { } /** * Get the innerProperties property: Data network properties. - * + * * @return the innerProperties value. */ private DataNetworkPropertiesFormat innerProperties() { @@ -41,21 +45,25 @@ private DataNetworkPropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DataNetworkInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DataNetworkInner withTags(Map tags) { super.withTags(tags); @@ -64,7 +72,7 @@ public DataNetworkInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the data network resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -73,7 +81,7 @@ public ProvisioningState provisioningState() { /** * Get the description property: An optional description for this data network. - * + * * @return the description value. */ public String description() { @@ -82,7 +90,7 @@ public String description() { /** * Set the description property: An optional description for this data network. - * + * * @param description the description value to set. * @return the DataNetworkInner object itself. */ @@ -96,7 +104,7 @@ public DataNetworkInner withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkPropertiesFormat.java index 0b49db76fcd46..4e9f152aabef4 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DataNetworkPropertiesFormat.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.mobilenetwork.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Data network properties. */ +/** + * Data network properties. + */ @Fluent public final class DataNetworkPropertiesFormat { /* @@ -23,13 +25,15 @@ public final class DataNetworkPropertiesFormat { @JsonProperty(value = "description") private String description; - /** Creates an instance of DataNetworkPropertiesFormat class. */ + /** + * Creates an instance of DataNetworkPropertiesFormat class. + */ public DataNetworkPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the data network resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -38,7 +42,7 @@ public ProvisioningState provisioningState() { /** * Get the description property: An optional description for this data network. - * + * * @return the description value. */ public String description() { @@ -47,7 +51,7 @@ public String description() { /** * Set the description property: An optional description for this data network. - * + * * @param description the description value to set. * @return the DataNetworkPropertiesFormat object itself. */ @@ -58,7 +62,7 @@ public DataNetworkPropertiesFormat withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackageInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackageInner.java index e0f61d74700ea..6a810623f7397 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackageInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackageInner.java @@ -12,7 +12,9 @@ import com.azure.resourcemanager.mobilenetwork.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Diagnostics package resource. */ +/** + * Diagnostics package resource. + */ @Immutable public final class DiagnosticsPackageInner extends ProxyResource { /* @@ -28,7 +30,9 @@ public final class DiagnosticsPackageInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of DiagnosticsPackageInner class. */ + /** + * Creates an instance of DiagnosticsPackageInner class. + */ public DiagnosticsPackageInner() { } @@ -36,7 +40,7 @@ public DiagnosticsPackageInner() { * Get the innerProperties property: Diagnostics package properties. A diagnostics package file derived from the * name of this resource will be uploaded to the Storage Account Container URL in the packet core control plane * properties. - * + * * @return the innerProperties value. */ private DiagnosticsPackagePropertiesFormat innerProperties() { @@ -45,7 +49,7 @@ private DiagnosticsPackagePropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -54,7 +58,7 @@ public SystemData systemData() { /** * Get the provisioningState property: The provisioning state of the diagnostics package resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -63,7 +67,7 @@ public ProvisioningState provisioningState() { /** * Get the status property: The status of the diagnostics package collection. - * + * * @return the status value. */ public DiagnosticsPackageStatus status() { @@ -72,7 +76,7 @@ public DiagnosticsPackageStatus status() { /** * Get the reason property: The reason for the current state of the diagnostics package collection. - * + * * @return the reason value. */ public String reason() { @@ -81,15 +85,13 @@ public String reason() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model DiagnosticsPackageInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerProperties in model DiagnosticsPackageInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackagePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackagePropertiesFormat.java index 51084e98aef57..c239d39f61c76 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackagePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/DiagnosticsPackagePropertiesFormat.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.mobilenetwork.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Diagnostics package properties. */ +/** + * Diagnostics package properties. + */ @Immutable public final class DiagnosticsPackagePropertiesFormat { /* @@ -30,13 +32,15 @@ public final class DiagnosticsPackagePropertiesFormat { @JsonProperty(value = "reason", access = JsonProperty.Access.WRITE_ONLY) private String reason; - /** Creates an instance of DiagnosticsPackagePropertiesFormat class. */ + /** + * Creates an instance of DiagnosticsPackagePropertiesFormat class. + */ public DiagnosticsPackagePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the diagnostics package resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -45,7 +49,7 @@ public ProvisioningState provisioningState() { /** * Get the status property: The status of the diagnostics package collection. - * + * * @return the status value. */ public DiagnosticsPackageStatus status() { @@ -54,7 +58,7 @@ public DiagnosticsPackageStatus status() { /** * Get the reason property: The reason for the current state of the diagnostics package collection. - * + * * @return the reason value. */ public String reason() { @@ -63,7 +67,7 @@ public String reason() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/EncryptedSimPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/EncryptedSimPropertiesFormat.java index 2138fcb2beace..f42c7905e3788 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/EncryptedSimPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/EncryptedSimPropertiesFormat.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Encrypted SIM properties. */ +/** + * Encrypted SIM properties. + */ @Fluent public final class EncryptedSimPropertiesFormat extends CommonSimPropertiesFormat { /* @@ -20,13 +22,15 @@ public final class EncryptedSimPropertiesFormat extends CommonSimPropertiesForma @JsonProperty(value = "encryptedCredentials") private String encryptedCredentials; - /** Creates an instance of EncryptedSimPropertiesFormat class. */ + /** + * Creates an instance of EncryptedSimPropertiesFormat class. + */ public EncryptedSimPropertiesFormat() { } /** * Get the encryptedCredentials property: The encrypted SIM credentials. - * + * * @return the encryptedCredentials value. */ public String encryptedCredentials() { @@ -35,7 +39,7 @@ public String encryptedCredentials() { /** * Set the encryptedCredentials property: The encrypted SIM credentials. - * + * * @param encryptedCredentials the encryptedCredentials value to set. * @return the EncryptedSimPropertiesFormat object itself. */ @@ -44,36 +48,46 @@ public EncryptedSimPropertiesFormat withEncryptedCredentials(String encryptedCre return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override - public EncryptedSimPropertiesFormat withInternationalMobileSubscriberIdentity( - String internationalMobileSubscriberIdentity) { + public EncryptedSimPropertiesFormat + withInternationalMobileSubscriberIdentity(String internationalMobileSubscriberIdentity) { super.withInternationalMobileSubscriberIdentity(internationalMobileSubscriberIdentity); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public EncryptedSimPropertiesFormat withIntegratedCircuitCardIdentifier(String integratedCircuitCardIdentifier) { super.withIntegratedCircuitCardIdentifier(integratedCircuitCardIdentifier); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public EncryptedSimPropertiesFormat withDeviceType(String deviceType) { super.withDeviceType(deviceType); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public EncryptedSimPropertiesFormat withSimPolicy(SimPolicyResourceId simPolicy) { super.withSimPolicy(simPolicy); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public EncryptedSimPropertiesFormat withStaticIpConfiguration(List staticIpConfiguration) { super.withStaticIpConfiguration(staticIpConfiguration); @@ -82,7 +96,7 @@ public EncryptedSimPropertiesFormat withStaticIpConfiguration(List tags) { super.withTags(tags); @@ -66,7 +105,7 @@ public MobileNetworkInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the mobile network resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -75,10 +114,9 @@ public ProvisioningState provisioningState() { /** * Get the publicLandMobileNetworkIdentifier property: The unique public land mobile network identifier for the - * network. This is made up of the mobile country code and mobile network code, as defined in - * https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the values 999-99 - * and 999-999 can be used on internal private networks. - * + * network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * * @return the publicLandMobileNetworkIdentifier value. */ public PlmnId publicLandMobileNetworkIdentifier() { @@ -87,10 +125,9 @@ public PlmnId publicLandMobileNetworkIdentifier() { /** * Set the publicLandMobileNetworkIdentifier property: The unique public land mobile network identifier for the - * network. This is made up of the mobile country code and mobile network code, as defined in - * https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the values 999-99 - * and 999-999 can be used on internal private networks. - * + * network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * * @param publicLandMobileNetworkIdentifier the publicLandMobileNetworkIdentifier value to set. * @return the MobileNetworkInner object itself. */ @@ -102,9 +139,36 @@ public MobileNetworkInner withPublicLandMobileNetworkIdentifier(PlmnId publicLan return this; } + /** + * Get the publicLandMobileNetworks property: A list of public land mobile networks including their identifiers. If + * both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * + * @return the publicLandMobileNetworks value. + */ + public List publicLandMobileNetworks() { + return this.innerProperties() == null ? null : this.innerProperties().publicLandMobileNetworks(); + } + + /** + * Set the publicLandMobileNetworks property: A list of public land mobile networks including their identifiers. If + * both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * + * @param publicLandMobileNetworks the publicLandMobileNetworks value to set. + * @return the MobileNetworkInner object itself. + */ + public MobileNetworkInner withPublicLandMobileNetworks(List publicLandMobileNetworks) { + if (this.innerProperties() == null) { + this.innerProperties = new MobileNetworkPropertiesFormat(); + } + this.innerProperties().withPublicLandMobileNetworks(publicLandMobileNetworks); + return this; + } + /** * Get the serviceKey property: The mobile network resource identifier. - * + * * @return the serviceKey value. */ public String serviceKey() { @@ -113,18 +177,19 @@ public String serviceKey() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model MobileNetworkInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model MobileNetworkInner")); } else { innerProperties().validate(); } + if (identity() != null) { + identity().validate(); + } } private static final ClientLogger LOGGER = new ClientLogger(MobileNetworkInner.class); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/MobileNetworkPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/MobileNetworkPropertiesFormat.java index faa33eec404ea..3dcdd537e7aa5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/MobileNetworkPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/MobileNetworkPropertiesFormat.java @@ -8,9 +8,13 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.mobilenetwork.models.PlmnId; import com.azure.resourcemanager.mobilenetwork.models.ProvisioningState; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetwork; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; -/** Mobile network properties. */ +/** + * Mobile network properties. + */ @Fluent public final class MobileNetworkPropertiesFormat { /* @@ -20,26 +24,34 @@ public final class MobileNetworkPropertiesFormat { private ProvisioningState provisioningState; /* - * The unique public land mobile network identifier for the network. This is made up of the mobile country code and - * mobile network code, as defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be - * used for testing and the values 999-99 and 999-999 can be used on internal private networks. + * The unique public land mobile network identifier for the network. If both 'publicLandMobileNetworks' and + * 'publicLandMobileNetworkIdentifier' are specified, then the 'publicLandMobileNetworks' will take precedence. */ @JsonProperty(value = "publicLandMobileNetworkIdentifier", required = true) private PlmnId publicLandMobileNetworkIdentifier; + /* + * A list of public land mobile networks including their identifiers. If both 'publicLandMobileNetworks' and + * 'publicLandMobileNetworkIdentifier' are specified, then the 'publicLandMobileNetworks' will take precedence. + */ + @JsonProperty(value = "publicLandMobileNetworks") + private List publicLandMobileNetworks; + /* * The mobile network resource identifier */ @JsonProperty(value = "serviceKey", access = JsonProperty.Access.WRITE_ONLY) private String serviceKey; - /** Creates an instance of MobileNetworkPropertiesFormat class. */ + /** + * Creates an instance of MobileNetworkPropertiesFormat class. + */ public MobileNetworkPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the mobile network resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -48,10 +60,9 @@ public ProvisioningState provisioningState() { /** * Get the publicLandMobileNetworkIdentifier property: The unique public land mobile network identifier for the - * network. This is made up of the mobile country code and mobile network code, as defined in - * https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the values 999-99 - * and 999-999 can be used on internal private networks. - * + * network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * * @return the publicLandMobileNetworkIdentifier value. */ public PlmnId publicLandMobileNetworkIdentifier() { @@ -60,22 +71,46 @@ public PlmnId publicLandMobileNetworkIdentifier() { /** * Set the publicLandMobileNetworkIdentifier property: The unique public land mobile network identifier for the - * network. This is made up of the mobile country code and mobile network code, as defined in - * https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the values 999-99 - * and 999-999 can be used on internal private networks. - * + * network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * * @param publicLandMobileNetworkIdentifier the publicLandMobileNetworkIdentifier value to set. * @return the MobileNetworkPropertiesFormat object itself. */ - public MobileNetworkPropertiesFormat withPublicLandMobileNetworkIdentifier( - PlmnId publicLandMobileNetworkIdentifier) { + public MobileNetworkPropertiesFormat + withPublicLandMobileNetworkIdentifier(PlmnId publicLandMobileNetworkIdentifier) { this.publicLandMobileNetworkIdentifier = publicLandMobileNetworkIdentifier; return this; } + /** + * Get the publicLandMobileNetworks property: A list of public land mobile networks including their identifiers. If + * both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * + * @return the publicLandMobileNetworks value. + */ + public List publicLandMobileNetworks() { + return this.publicLandMobileNetworks; + } + + /** + * Set the publicLandMobileNetworks property: A list of public land mobile networks including their identifiers. If + * both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * + * @param publicLandMobileNetworks the publicLandMobileNetworks value to set. + * @return the MobileNetworkPropertiesFormat object itself. + */ + public MobileNetworkPropertiesFormat + withPublicLandMobileNetworks(List publicLandMobileNetworks) { + this.publicLandMobileNetworks = publicLandMobileNetworks; + return this; + } + /** * Get the serviceKey property: The mobile network resource identifier. - * + * * @return the serviceKey value. */ public String serviceKey() { @@ -84,19 +119,19 @@ public String serviceKey() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (publicLandMobileNetworkIdentifier() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property publicLandMobileNetworkIdentifier in model" - + " MobileNetworkPropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property publicLandMobileNetworkIdentifier in model MobileNetworkPropertiesFormat")); } else { publicLandMobileNetworkIdentifier().validate(); } + if (publicLandMobileNetworks() != null) { + publicLandMobileNetworks().forEach(e -> e.validate()); + } } private static final ClientLogger LOGGER = new ClientLogger(MobileNetworkPropertiesFormat.class); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/OperationInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/OperationInner.java index 8408ffc83e97b..193cbf6644746 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/OperationInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/OperationInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.mobilenetwork.models.OperationDisplay; import com.fasterxml.jackson.annotation.JsonProperty; -/** Object that describes a single Microsoft.MobileNetwork operation. */ +/** + * Object that describes a single Microsoft.MobileNetwork operation. + */ @Fluent public final class OperationInner { /* @@ -29,13 +31,15 @@ public final class OperationInner { @JsonProperty(value = "display", access = JsonProperty.Access.WRITE_ONLY) private OperationDisplay display; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the isDataAction property: Indicates whether the operation applies to data-plane. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -44,7 +48,7 @@ public Boolean isDataAction() { /** * Set the isDataAction property: Indicates whether the operation applies to data-plane. - * + * * @param isDataAction the isDataAction value to set. * @return the OperationInner object itself. */ @@ -55,7 +59,7 @@ public OperationInner withIsDataAction(Boolean isDataAction) { /** * Get the name property: Operation name: {provider}/{resource}/{operation}. - * + * * @return the name value. */ public String name() { @@ -64,7 +68,7 @@ public String name() { /** * Get the display property: The object that represents the operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -73,7 +77,7 @@ public OperationDisplay display() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCaptureInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCaptureInner.java index cdeaeee4fe3e0..ac887003c1181 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCaptureInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCaptureInner.java @@ -14,7 +14,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Packet capture session resource. */ +/** + * Packet capture session resource. + */ @Fluent public final class PacketCaptureInner extends ProxyResource { /* @@ -30,15 +32,17 @@ public final class PacketCaptureInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of PacketCaptureInner class. */ + /** + * Creates an instance of PacketCaptureInner class. + */ public PacketCaptureInner() { } /** - * Get the innerProperties property: Packet capture session properties. Packet capture file(s) derived from the name - * of this session will be uploaded to the Storage Account Container URL in the packet core control plane + * Get the innerProperties property: Packet capture session properties. Packet capture file(s) derived from the + * name of this session will be uploaded to the Storage Account Container URL in the packet core control plane * properties. - * + * * @return the innerProperties value. */ private PacketCapturePropertiesFormat innerProperties() { @@ -47,7 +51,7 @@ private PacketCapturePropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -56,7 +60,7 @@ public SystemData systemData() { /** * Get the provisioningState property: The provisioning state of the packet capture session resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -65,7 +69,7 @@ public ProvisioningState provisioningState() { /** * Get the status property: The status of the packet capture session. - * + * * @return the status value. */ public PacketCaptureStatus status() { @@ -74,7 +78,7 @@ public PacketCaptureStatus status() { /** * Get the reason property: The reason the current packet capture session state. - * + * * @return the reason value. */ public String reason() { @@ -83,7 +87,7 @@ public String reason() { /** * Get the captureStartTime property: The start time of the packet capture session. - * + * * @return the captureStartTime value. */ public OffsetDateTime captureStartTime() { @@ -92,7 +96,7 @@ public OffsetDateTime captureStartTime() { /** * Get the networkInterfaces property: List of network interfaces to capture on. - * + * * @return the networkInterfaces value. */ public List networkInterfaces() { @@ -101,7 +105,7 @@ public List networkInterfaces() { /** * Set the networkInterfaces property: List of network interfaces to capture on. - * + * * @param networkInterfaces the networkInterfaces value to set. * @return the PacketCaptureInner object itself. */ @@ -114,9 +118,9 @@ public PacketCaptureInner withNetworkInterfaces(List networkInterfaces) } /** - * Get the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are truncated. - * The default "0" means the entire packet is captured. - * + * Get the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are + * truncated. The default "0" means the entire packet is captured. + * * @return the bytesToCapturePerPacket value. */ public Long bytesToCapturePerPacket() { @@ -124,9 +128,9 @@ public Long bytesToCapturePerPacket() { } /** - * Set the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are truncated. - * The default "0" means the entire packet is captured. - * + * Set the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are + * truncated. The default "0" means the entire packet is captured. + * * @param bytesToCapturePerPacket the bytesToCapturePerPacket value to set. * @return the PacketCaptureInner object itself. */ @@ -140,7 +144,7 @@ public PacketCaptureInner withBytesToCapturePerPacket(Long bytesToCapturePerPack /** * Get the totalBytesPerSession property: Maximum size of the capture output. - * + * * @return the totalBytesPerSession value. */ public Long totalBytesPerSession() { @@ -149,7 +153,7 @@ public Long totalBytesPerSession() { /** * Set the totalBytesPerSession property: Maximum size of the capture output. - * + * * @param totalBytesPerSession the totalBytesPerSession value to set. * @return the PacketCaptureInner object itself. */ @@ -163,7 +167,7 @@ public PacketCaptureInner withTotalBytesPerSession(Long totalBytesPerSession) { /** * Get the timeLimitInSeconds property: Maximum duration of the capture session in seconds. - * + * * @return the timeLimitInSeconds value. */ public Integer timeLimitInSeconds() { @@ -172,7 +176,7 @@ public Integer timeLimitInSeconds() { /** * Set the timeLimitInSeconds property: Maximum duration of the capture session in seconds. - * + * * @param timeLimitInSeconds the timeLimitInSeconds value to set. * @return the PacketCaptureInner object itself. */ @@ -186,7 +190,7 @@ public PacketCaptureInner withTimeLimitInSeconds(Integer timeLimitInSeconds) { /** * Get the outputFiles property: The list of output files of a packet capture session. - * + * * @return the outputFiles value. */ public List outputFiles() { @@ -195,15 +199,13 @@ public List outputFiles() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model PacketCaptureInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model PacketCaptureInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCapturePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCapturePropertiesFormat.java index 71b45858e3da9..f63f970d4f13a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCapturePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCapturePropertiesFormat.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Packet capture session properties. */ +/** + * Packet capture session properties. + */ @Fluent public final class PacketCapturePropertiesFormat { /* @@ -69,13 +71,15 @@ public final class PacketCapturePropertiesFormat { @JsonProperty(value = "outputFiles", access = JsonProperty.Access.WRITE_ONLY) private List outputFiles; - /** Creates an instance of PacketCapturePropertiesFormat class. */ + /** + * Creates an instance of PacketCapturePropertiesFormat class. + */ public PacketCapturePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the packet capture session resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -84,7 +88,7 @@ public ProvisioningState provisioningState() { /** * Get the status property: The status of the packet capture session. - * + * * @return the status value. */ public PacketCaptureStatus status() { @@ -93,7 +97,7 @@ public PacketCaptureStatus status() { /** * Get the reason property: The reason the current packet capture session state. - * + * * @return the reason value. */ public String reason() { @@ -102,7 +106,7 @@ public String reason() { /** * Get the captureStartTime property: The start time of the packet capture session. - * + * * @return the captureStartTime value. */ public OffsetDateTime captureStartTime() { @@ -111,7 +115,7 @@ public OffsetDateTime captureStartTime() { /** * Get the networkInterfaces property: List of network interfaces to capture on. - * + * * @return the networkInterfaces value. */ public List networkInterfaces() { @@ -120,7 +124,7 @@ public List networkInterfaces() { /** * Set the networkInterfaces property: List of network interfaces to capture on. - * + * * @param networkInterfaces the networkInterfaces value to set. * @return the PacketCapturePropertiesFormat object itself. */ @@ -130,9 +134,9 @@ public PacketCapturePropertiesFormat withNetworkInterfaces(List networkI } /** - * Get the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are truncated. - * The default "0" means the entire packet is captured. - * + * Get the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are + * truncated. The default "0" means the entire packet is captured. + * * @return the bytesToCapturePerPacket value. */ public Long bytesToCapturePerPacket() { @@ -140,9 +144,9 @@ public Long bytesToCapturePerPacket() { } /** - * Set the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are truncated. - * The default "0" means the entire packet is captured. - * + * Set the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are + * truncated. The default "0" means the entire packet is captured. + * * @param bytesToCapturePerPacket the bytesToCapturePerPacket value to set. * @return the PacketCapturePropertiesFormat object itself. */ @@ -153,7 +157,7 @@ public PacketCapturePropertiesFormat withBytesToCapturePerPacket(Long bytesToCap /** * Get the totalBytesPerSession property: Maximum size of the capture output. - * + * * @return the totalBytesPerSession value. */ public Long totalBytesPerSession() { @@ -162,7 +166,7 @@ public Long totalBytesPerSession() { /** * Set the totalBytesPerSession property: Maximum size of the capture output. - * + * * @param totalBytesPerSession the totalBytesPerSession value to set. * @return the PacketCapturePropertiesFormat object itself. */ @@ -173,7 +177,7 @@ public PacketCapturePropertiesFormat withTotalBytesPerSession(Long totalBytesPer /** * Get the timeLimitInSeconds property: Maximum duration of the capture session in seconds. - * + * * @return the timeLimitInSeconds value. */ public Integer timeLimitInSeconds() { @@ -182,7 +186,7 @@ public Integer timeLimitInSeconds() { /** * Set the timeLimitInSeconds property: Maximum duration of the capture session in seconds. - * + * * @param timeLimitInSeconds the timeLimitInSeconds value to set. * @return the PacketCapturePropertiesFormat object itself. */ @@ -193,7 +197,7 @@ public PacketCapturePropertiesFormat withTimeLimitInSeconds(Integer timeLimitInS /** * Get the outputFiles property: The list of output files of a packet capture session. - * + * * @return the outputFiles value. */ public List outputFiles() { @@ -202,7 +206,7 @@ public List outputFiles() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneInner.java index d79e9daa39c76..b58caad56ed1d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneInner.java @@ -12,6 +12,7 @@ import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPrivateKeysProvisioning; import com.azure.resourcemanager.mobilenetwork.models.Installation; import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; import com.azure.resourcemanager.mobilenetwork.models.LocalDiagnosticsAccessConfiguration; @@ -24,7 +25,9 @@ import java.util.List; import java.util.Map; -/** Packet core control plane resource. */ +/** + * Packet core control plane resource. + */ @Fluent public final class PacketCoreControlPlaneInner extends Resource { /* @@ -45,13 +48,15 @@ public final class PacketCoreControlPlaneInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of PacketCoreControlPlaneInner class. */ + /** + * Creates an instance of PacketCoreControlPlaneInner class. + */ public PacketCoreControlPlaneInner() { } /** * Get the innerProperties property: Packet core control plane Properties. - * + * * @return the innerProperties value. */ private PacketCoreControlPlanePropertiesFormat innerProperties() { @@ -60,7 +65,7 @@ private PacketCoreControlPlanePropertiesFormat innerProperties() { /** * Get the identity property: The identity used to retrieve the ingress certificate from Azure key vault. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -69,7 +74,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The identity used to retrieve the ingress certificate from Azure key vault. - * + * * @param identity the identity value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -80,21 +85,25 @@ public PacketCoreControlPlaneInner withIdentity(ManagedServiceIdentity identity) /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PacketCoreControlPlaneInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PacketCoreControlPlaneInner withTags(Map tags) { super.withTags(tags); @@ -103,7 +112,7 @@ public PacketCoreControlPlaneInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the packet core control plane resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -112,7 +121,7 @@ public ProvisioningState provisioningState() { /** * Get the installation property: The installation state of the packet core control plane resource. - * + * * @return the installation value. */ public Installation installation() { @@ -121,7 +130,7 @@ public Installation installation() { /** * Set the installation property: The installation state of the packet core control plane resource. - * + * * @param installation the installation value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -136,7 +145,7 @@ public PacketCoreControlPlaneInner withInstallation(Installation installation) { /** * Get the sites property: Site(s) under which this packet core control plane should be deployed. The sites must be * in the same location as the packet core control plane. - * + * * @return the sites value. */ public List sites() { @@ -146,7 +155,7 @@ public List sites() { /** * Set the sites property: Site(s) under which this packet core control plane should be deployed. The sites must be * in the same location as the packet core control plane. - * + * * @param sites the sites value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -160,7 +169,7 @@ public PacketCoreControlPlaneInner withSites(List sites) { /** * Get the platform property: The platform where the packet core is deployed. - * + * * @return the platform value. */ public PlatformConfiguration platform() { @@ -169,7 +178,7 @@ public PlatformConfiguration platform() { /** * Set the platform property: The platform where the packet core is deployed. - * + * * @param platform the platform value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -183,7 +192,7 @@ public PacketCoreControlPlaneInner withPlatform(PlatformConfiguration platform) /** * Get the coreNetworkTechnology property: The core network technology generation (5G core or EPC / 4G core). - * + * * @return the coreNetworkTechnology value. */ public CoreNetworkType coreNetworkTechnology() { @@ -192,7 +201,7 @@ public CoreNetworkType coreNetworkTechnology() { /** * Set the coreNetworkTechnology property: The core network technology generation (5G core or EPC / 4G core). - * + * * @param coreNetworkTechnology the coreNetworkTechnology value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -206,7 +215,7 @@ public PacketCoreControlPlaneInner withCoreNetworkTechnology(CoreNetworkType cor /** * Get the version property: The desired version of the packet core software. - * + * * @return the version value. */ public String version() { @@ -215,7 +224,7 @@ public String version() { /** * Set the version property: The desired version of the packet core software. - * + * * @param version the version value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -229,7 +238,7 @@ public PacketCoreControlPlaneInner withVersion(String version) { /** * Get the installedVersion property: The currently installed version of the packet core software. - * + * * @return the installedVersion value. */ public String installedVersion() { @@ -239,7 +248,7 @@ public String installedVersion() { /** * Get the rollbackVersion property: The previous version of the packet core software that was deployed. Used when * performing the rollback action. - * + * * @return the rollbackVersion value. */ public String rollbackVersion() { @@ -247,9 +256,9 @@ public String rollbackVersion() { } /** - * Get the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G networks, - * this is the N2 interface. For 4G networks, this is the S1-MME interface. - * + * Get the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G + * networks, this is the N2 interface. For 4G networks, this is the S1-MME interface. + * * @return the controlPlaneAccessInterface value. */ public InterfaceProperties controlPlaneAccessInterface() { @@ -257,14 +266,14 @@ public InterfaceProperties controlPlaneAccessInterface() { } /** - * Set the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G networks, - * this is the N2 interface. For 4G networks, this is the S1-MME interface. - * + * Set the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G + * networks, this is the N2 interface. For 4G networks, this is the S1-MME interface. + * * @param controlPlaneAccessInterface the controlPlaneAccessInterface value to set. * @return the PacketCoreControlPlaneInner object itself. */ - public PacketCoreControlPlaneInner withControlPlaneAccessInterface( - InterfaceProperties controlPlaneAccessInterface) { + public PacketCoreControlPlaneInner + withControlPlaneAccessInterface(InterfaceProperties controlPlaneAccessInterface) { if (this.innerProperties() == null) { this.innerProperties = new PacketCoreControlPlanePropertiesFormat(); } @@ -277,7 +286,7 @@ public PacketCoreControlPlaneInner withControlPlaneAccessInterface( * access network in a High Availability (HA) system. In an HA deployment the access network router should be * configured to anycast traffic for this address to the control plane access interfaces on the active and standby * nodes. In non-HA system this list should be omitted or empty. - * + * * @return the controlPlaneAccessVirtualIpv4Addresses value. */ public List controlPlaneAccessVirtualIpv4Addresses() { @@ -289,12 +298,12 @@ public List controlPlaneAccessVirtualIpv4Addresses() { * access network in a High Availability (HA) system. In an HA deployment the access network router should be * configured to anycast traffic for this address to the control plane access interfaces on the active and standby * nodes. In non-HA system this list should be omitted or empty. - * + * * @param controlPlaneAccessVirtualIpv4Addresses the controlPlaneAccessVirtualIpv4Addresses value to set. * @return the PacketCoreControlPlaneInner object itself. */ - public PacketCoreControlPlaneInner withControlPlaneAccessVirtualIpv4Addresses( - List controlPlaneAccessVirtualIpv4Addresses) { + public PacketCoreControlPlaneInner + withControlPlaneAccessVirtualIpv4Addresses(List controlPlaneAccessVirtualIpv4Addresses) { if (this.innerProperties() == null) { this.innerProperties = new PacketCoreControlPlanePropertiesFormat(); } @@ -305,7 +314,7 @@ public PacketCoreControlPlaneInner withControlPlaneAccessVirtualIpv4Addresses( /** * Get the sku property: The SKU defining the throughput and SIM allowances for this packet core control plane * deployment. - * + * * @return the sku value. */ public BillingSku sku() { @@ -315,7 +324,7 @@ public BillingSku sku() { /** * Set the sku property: The SKU defining the throughput and SIM allowances for this packet core control plane * deployment. - * + * * @param sku the sku value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -331,7 +340,7 @@ public PacketCoreControlPlaneInner withSku(BillingSku sku) { * Get the ueMtu property: The MTU (in bytes) signaled to the UE. The same MTU is set on the user plane data links * for all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater than this * value to allow for GTP encapsulation. - * + * * @return the ueMtu value. */ public Integer ueMtu() { @@ -342,7 +351,7 @@ public Integer ueMtu() { * Set the ueMtu property: The MTU (in bytes) signaled to the UE. The same MTU is set on the user plane data links * for all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater than this * value to allow for GTP encapsulation. - * + * * @param ueMtu the ueMtu value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -357,7 +366,7 @@ public PacketCoreControlPlaneInner withUeMtu(Integer ueMtu) { /** * Get the localDiagnosticsAccess property: The kubernetes ingress configuration to control access to packet core * diagnostics over local APIs. - * + * * @return the localDiagnosticsAccess value. */ public LocalDiagnosticsAccessConfiguration localDiagnosticsAccess() { @@ -367,12 +376,12 @@ public LocalDiagnosticsAccessConfiguration localDiagnosticsAccess() { /** * Set the localDiagnosticsAccess property: The kubernetes ingress configuration to control access to packet core * diagnostics over local APIs. - * + * * @param localDiagnosticsAccess the localDiagnosticsAccess value to set. * @return the PacketCoreControlPlaneInner object itself. */ - public PacketCoreControlPlaneInner withLocalDiagnosticsAccess( - LocalDiagnosticsAccessConfiguration localDiagnosticsAccess) { + public PacketCoreControlPlaneInner + withLocalDiagnosticsAccess(LocalDiagnosticsAccessConfiguration localDiagnosticsAccess) { if (this.innerProperties() == null) { this.innerProperties = new PacketCoreControlPlanePropertiesFormat(); } @@ -382,7 +391,7 @@ public PacketCoreControlPlaneInner withLocalDiagnosticsAccess( /** * Get the diagnosticsUpload property: Configuration for uploading packet core diagnostics. - * + * * @return the diagnosticsUpload value. */ public DiagnosticsUploadConfiguration diagnosticsUpload() { @@ -391,7 +400,7 @@ public DiagnosticsUploadConfiguration diagnosticsUpload() { /** * Set the diagnosticsUpload property: Configuration for uploading packet core diagnostics. - * + * * @param diagnosticsUpload the diagnosticsUpload value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -405,7 +414,7 @@ public PacketCoreControlPlaneInner withDiagnosticsUpload(DiagnosticsUploadConfig /** * Get the eventHub property: Configuration for sending packet core events to an Azure Event Hub. - * + * * @return the eventHub value. */ public EventHubConfiguration eventHub() { @@ -414,7 +423,7 @@ public EventHubConfiguration eventHub() { /** * Set the eventHub property: Configuration for sending packet core events to an Azure Event Hub. - * + * * @param eventHub the eventHub value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -428,7 +437,7 @@ public PacketCoreControlPlaneInner withEventHub(EventHubConfiguration eventHub) /** * Get the signaling property: Signaling configuration for the packet core. - * + * * @return the signaling value. */ public SignalingConfiguration signaling() { @@ -437,7 +446,7 @@ public SignalingConfiguration signaling() { /** * Set the signaling property: Signaling configuration for the packet core. - * + * * @param signaling the signaling value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -452,7 +461,7 @@ public PacketCoreControlPlaneInner withSignaling(SignalingConfiguration signalin /** * Get the interopSettings property: Settings to allow interoperability with third party components e.g. RANs and * UEs. - * + * * @return the interopSettings value. */ public Object interopSettings() { @@ -462,7 +471,7 @@ public Object interopSettings() { /** * Set the interopSettings property: Settings to allow interoperability with third party components e.g. RANs and * UEs. - * + * * @param interopSettings the interopSettings value to set. * @return the PacketCoreControlPlaneInner object itself. */ @@ -474,17 +483,25 @@ public PacketCoreControlPlaneInner withInteropSettings(Object interopSettings) { return this; } + /** + * Get the homeNetworkPrivateKeysProvisioning property: The provisioning state of the secret containing private + * keys and keyIds for SUPI concealment. + * + * @return the homeNetworkPrivateKeysProvisioning value. + */ + public HomeNetworkPrivateKeysProvisioning homeNetworkPrivateKeysProvisioning() { + return this.innerProperties() == null ? null : this.innerProperties().homeNetworkPrivateKeysProvisioning(); + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model PacketCoreControlPlaneInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerProperties in model PacketCoreControlPlaneInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlanePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlanePropertiesFormat.java index 71036e0a06d8e..a28384612bc8d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlanePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlanePropertiesFormat.java @@ -10,6 +10,7 @@ import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPrivateKeysProvisioning; import com.azure.resourcemanager.mobilenetwork.models.Installation; import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; import com.azure.resourcemanager.mobilenetwork.models.LocalDiagnosticsAccessConfiguration; @@ -20,7 +21,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Packet core control plane properties. */ +/** + * Packet core control plane properties. + */ @Fluent public final class PacketCoreControlPlanePropertiesFormat { /* @@ -131,13 +134,21 @@ public final class PacketCoreControlPlanePropertiesFormat { @JsonProperty(value = "interopSettings") private Object interopSettings; - /** Creates an instance of PacketCoreControlPlanePropertiesFormat class. */ + /* + * The provisioning state of the secret containing private keys and keyIds for SUPI concealment. + */ + @JsonProperty(value = "homeNetworkPrivateKeysProvisioning", access = JsonProperty.Access.WRITE_ONLY) + private HomeNetworkPrivateKeysProvisioning homeNetworkPrivateKeysProvisioning; + + /** + * Creates an instance of PacketCoreControlPlanePropertiesFormat class. + */ public PacketCoreControlPlanePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the packet core control plane resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -146,7 +157,7 @@ public ProvisioningState provisioningState() { /** * Get the installation property: The installation state of the packet core control plane resource. - * + * * @return the installation value. */ public Installation installation() { @@ -155,7 +166,7 @@ public Installation installation() { /** * Set the installation property: The installation state of the packet core control plane resource. - * + * * @param installation the installation value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -167,7 +178,7 @@ public PacketCoreControlPlanePropertiesFormat withInstallation(Installation inst /** * Get the sites property: Site(s) under which this packet core control plane should be deployed. The sites must be * in the same location as the packet core control plane. - * + * * @return the sites value. */ public List sites() { @@ -177,7 +188,7 @@ public List sites() { /** * Set the sites property: Site(s) under which this packet core control plane should be deployed. The sites must be * in the same location as the packet core control plane. - * + * * @param sites the sites value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -188,7 +199,7 @@ public PacketCoreControlPlanePropertiesFormat withSites(List sit /** * Get the platform property: The platform where the packet core is deployed. - * + * * @return the platform value. */ public PlatformConfiguration platform() { @@ -197,7 +208,7 @@ public PlatformConfiguration platform() { /** * Set the platform property: The platform where the packet core is deployed. - * + * * @param platform the platform value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -208,7 +219,7 @@ public PacketCoreControlPlanePropertiesFormat withPlatform(PlatformConfiguration /** * Get the coreNetworkTechnology property: The core network technology generation (5G core or EPC / 4G core). - * + * * @return the coreNetworkTechnology value. */ public CoreNetworkType coreNetworkTechnology() { @@ -217,7 +228,7 @@ public CoreNetworkType coreNetworkTechnology() { /** * Set the coreNetworkTechnology property: The core network technology generation (5G core or EPC / 4G core). - * + * * @param coreNetworkTechnology the coreNetworkTechnology value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -228,7 +239,7 @@ public PacketCoreControlPlanePropertiesFormat withCoreNetworkTechnology(CoreNetw /** * Get the version property: The desired version of the packet core software. - * + * * @return the version value. */ public String version() { @@ -237,7 +248,7 @@ public String version() { /** * Set the version property: The desired version of the packet core software. - * + * * @param version the version value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -248,7 +259,7 @@ public PacketCoreControlPlanePropertiesFormat withVersion(String version) { /** * Get the installedVersion property: The currently installed version of the packet core software. - * + * * @return the installedVersion value. */ public String installedVersion() { @@ -258,7 +269,7 @@ public String installedVersion() { /** * Get the rollbackVersion property: The previous version of the packet core software that was deployed. Used when * performing the rollback action. - * + * * @return the rollbackVersion value. */ public String rollbackVersion() { @@ -266,9 +277,9 @@ public String rollbackVersion() { } /** - * Get the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G networks, - * this is the N2 interface. For 4G networks, this is the S1-MME interface. - * + * Get the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G + * networks, this is the N2 interface. For 4G networks, this is the S1-MME interface. + * * @return the controlPlaneAccessInterface value. */ public InterfaceProperties controlPlaneAccessInterface() { @@ -276,14 +287,14 @@ public InterfaceProperties controlPlaneAccessInterface() { } /** - * Set the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G networks, - * this is the N2 interface. For 4G networks, this is the S1-MME interface. - * + * Set the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G + * networks, this is the N2 interface. For 4G networks, this is the S1-MME interface. + * * @param controlPlaneAccessInterface the controlPlaneAccessInterface value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ - public PacketCoreControlPlanePropertiesFormat withControlPlaneAccessInterface( - InterfaceProperties controlPlaneAccessInterface) { + public PacketCoreControlPlanePropertiesFormat + withControlPlaneAccessInterface(InterfaceProperties controlPlaneAccessInterface) { this.controlPlaneAccessInterface = controlPlaneAccessInterface; return this; } @@ -293,7 +304,7 @@ public PacketCoreControlPlanePropertiesFormat withControlPlaneAccessInterface( * access network in a High Availability (HA) system. In an HA deployment the access network router should be * configured to anycast traffic for this address to the control plane access interfaces on the active and standby * nodes. In non-HA system this list should be omitted or empty. - * + * * @return the controlPlaneAccessVirtualIpv4Addresses value. */ public List controlPlaneAccessVirtualIpv4Addresses() { @@ -305,12 +316,12 @@ public List controlPlaneAccessVirtualIpv4Addresses() { * access network in a High Availability (HA) system. In an HA deployment the access network router should be * configured to anycast traffic for this address to the control plane access interfaces on the active and standby * nodes. In non-HA system this list should be omitted or empty. - * + * * @param controlPlaneAccessVirtualIpv4Addresses the controlPlaneAccessVirtualIpv4Addresses value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ - public PacketCoreControlPlanePropertiesFormat withControlPlaneAccessVirtualIpv4Addresses( - List controlPlaneAccessVirtualIpv4Addresses) { + public PacketCoreControlPlanePropertiesFormat + withControlPlaneAccessVirtualIpv4Addresses(List controlPlaneAccessVirtualIpv4Addresses) { this.controlPlaneAccessVirtualIpv4Addresses = controlPlaneAccessVirtualIpv4Addresses; return this; } @@ -318,7 +329,7 @@ public PacketCoreControlPlanePropertiesFormat withControlPlaneAccessVirtualIpv4A /** * Get the sku property: The SKU defining the throughput and SIM allowances for this packet core control plane * deployment. - * + * * @return the sku value. */ public BillingSku sku() { @@ -328,7 +339,7 @@ public BillingSku sku() { /** * Set the sku property: The SKU defining the throughput and SIM allowances for this packet core control plane * deployment. - * + * * @param sku the sku value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -341,7 +352,7 @@ public PacketCoreControlPlanePropertiesFormat withSku(BillingSku sku) { * Get the ueMtu property: The MTU (in bytes) signaled to the UE. The same MTU is set on the user plane data links * for all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater than this * value to allow for GTP encapsulation. - * + * * @return the ueMtu value. */ public Integer ueMtu() { @@ -352,7 +363,7 @@ public Integer ueMtu() { * Set the ueMtu property: The MTU (in bytes) signaled to the UE. The same MTU is set on the user plane data links * for all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater than this * value to allow for GTP encapsulation. - * + * * @param ueMtu the ueMtu value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -364,7 +375,7 @@ public PacketCoreControlPlanePropertiesFormat withUeMtu(Integer ueMtu) { /** * Get the localDiagnosticsAccess property: The kubernetes ingress configuration to control access to packet core * diagnostics over local APIs. - * + * * @return the localDiagnosticsAccess value. */ public LocalDiagnosticsAccessConfiguration localDiagnosticsAccess() { @@ -374,19 +385,19 @@ public LocalDiagnosticsAccessConfiguration localDiagnosticsAccess() { /** * Set the localDiagnosticsAccess property: The kubernetes ingress configuration to control access to packet core * diagnostics over local APIs. - * + * * @param localDiagnosticsAccess the localDiagnosticsAccess value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ - public PacketCoreControlPlanePropertiesFormat withLocalDiagnosticsAccess( - LocalDiagnosticsAccessConfiguration localDiagnosticsAccess) { + public PacketCoreControlPlanePropertiesFormat + withLocalDiagnosticsAccess(LocalDiagnosticsAccessConfiguration localDiagnosticsAccess) { this.localDiagnosticsAccess = localDiagnosticsAccess; return this; } /** * Get the diagnosticsUpload property: Configuration for uploading packet core diagnostics. - * + * * @return the diagnosticsUpload value. */ public DiagnosticsUploadConfiguration diagnosticsUpload() { @@ -395,19 +406,19 @@ public DiagnosticsUploadConfiguration diagnosticsUpload() { /** * Set the diagnosticsUpload property: Configuration for uploading packet core diagnostics. - * + * * @param diagnosticsUpload the diagnosticsUpload value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ - public PacketCoreControlPlanePropertiesFormat withDiagnosticsUpload( - DiagnosticsUploadConfiguration diagnosticsUpload) { + public PacketCoreControlPlanePropertiesFormat + withDiagnosticsUpload(DiagnosticsUploadConfiguration diagnosticsUpload) { this.diagnosticsUpload = diagnosticsUpload; return this; } /** * Get the eventHub property: Configuration for sending packet core events to an Azure Event Hub. - * + * * @return the eventHub value. */ public EventHubConfiguration eventHub() { @@ -416,7 +427,7 @@ public EventHubConfiguration eventHub() { /** * Set the eventHub property: Configuration for sending packet core events to an Azure Event Hub. - * + * * @param eventHub the eventHub value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -427,7 +438,7 @@ public PacketCoreControlPlanePropertiesFormat withEventHub(EventHubConfiguration /** * Get the signaling property: Signaling configuration for the packet core. - * + * * @return the signaling value. */ public SignalingConfiguration signaling() { @@ -436,7 +447,7 @@ public SignalingConfiguration signaling() { /** * Set the signaling property: Signaling configuration for the packet core. - * + * * @param signaling the signaling value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -448,7 +459,7 @@ public PacketCoreControlPlanePropertiesFormat withSignaling(SignalingConfigurati /** * Get the interopSettings property: Settings to allow interoperability with third party components e.g. RANs and * UEs. - * + * * @return the interopSettings value. */ public Object interopSettings() { @@ -458,7 +469,7 @@ public Object interopSettings() { /** * Set the interopSettings property: Settings to allow interoperability with third party components e.g. RANs and * UEs. - * + * * @param interopSettings the interopSettings value to set. * @return the PacketCoreControlPlanePropertiesFormat object itself. */ @@ -467,9 +478,19 @@ public PacketCoreControlPlanePropertiesFormat withInteropSettings(Object interop return this; } + /** + * Get the homeNetworkPrivateKeysProvisioning property: The provisioning state of the secret containing private + * keys and keyIds for SUPI concealment. + * + * @return the homeNetworkPrivateKeysProvisioning value. + */ + public HomeNetworkPrivateKeysProvisioning homeNetworkPrivateKeysProvisioning() { + return this.homeNetworkPrivateKeysProvisioning; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -477,42 +498,30 @@ public void validate() { installation().validate(); } if (sites() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property sites in model PacketCoreControlPlanePropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sites in model PacketCoreControlPlanePropertiesFormat")); } else { sites().forEach(e -> e.validate()); } if (platform() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property platform in model PacketCoreControlPlanePropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property platform in model PacketCoreControlPlanePropertiesFormat")); } else { platform().validate(); } if (controlPlaneAccessInterface() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property controlPlaneAccessInterface in model" - + " PacketCoreControlPlanePropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property controlPlaneAccessInterface in model PacketCoreControlPlanePropertiesFormat")); } else { controlPlaneAccessInterface().validate(); } if (sku() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property sku in model PacketCoreControlPlanePropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sku in model PacketCoreControlPlanePropertiesFormat")); } if (localDiagnosticsAccess() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property localDiagnosticsAccess in model" - + " PacketCoreControlPlanePropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property localDiagnosticsAccess in model PacketCoreControlPlanePropertiesFormat")); } else { localDiagnosticsAccess().validate(); } @@ -525,6 +534,9 @@ public void validate() { if (signaling() != null) { signaling().validate(); } + if (homeNetworkPrivateKeysProvisioning() != null) { + homeNetworkPrivateKeysProvisioning().validate(); + } } private static final ClientLogger LOGGER = new ClientLogger(PacketCoreControlPlanePropertiesFormat.class); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionInner.java index ed6e014f5f6fd..e06af7aa548c9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionInner.java @@ -13,15 +13,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Packet core control plane version resource. */ +/** + * Packet core control plane version resource. + */ @Fluent public final class PacketCoreControlPlaneVersionInner extends ProxyResource { /* * Packet core control plane version properties. */ @JsonProperty(value = "properties", required = true) - private PacketCoreControlPlaneVersionPropertiesFormat innerProperties = - new PacketCoreControlPlaneVersionPropertiesFormat(); + private PacketCoreControlPlaneVersionPropertiesFormat innerProperties + = new PacketCoreControlPlaneVersionPropertiesFormat(); /* * Azure Resource Manager metadata containing createdBy and modifiedBy information. @@ -29,13 +31,15 @@ public final class PacketCoreControlPlaneVersionInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of PacketCoreControlPlaneVersionInner class. */ + /** + * Creates an instance of PacketCoreControlPlaneVersionInner class. + */ public PacketCoreControlPlaneVersionInner() { } /** * Get the innerProperties property: Packet core control plane version properties. - * + * * @return the innerProperties value. */ private PacketCoreControlPlaneVersionPropertiesFormat innerProperties() { @@ -44,7 +48,7 @@ private PacketCoreControlPlaneVersionPropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -53,7 +57,7 @@ public SystemData systemData() { /** * Get the provisioningState property: The provisioning state of the packet core control plane version resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -62,7 +66,7 @@ public ProvisioningState provisioningState() { /** * Get the platforms property: Platform specific packet core control plane version properties. - * + * * @return the platforms value. */ public List platforms() { @@ -71,7 +75,7 @@ public List platforms() { /** * Set the platforms property: Platform specific packet core control plane version properties. - * + * * @param platforms the platforms value to set. * @return the PacketCoreControlPlaneVersionInner object itself. */ @@ -85,15 +89,13 @@ public PacketCoreControlPlaneVersionInner withPlatforms(List platforms /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model PacketCoreControlPlaneVersionInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerProperties in model PacketCoreControlPlaneVersionInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionPropertiesFormat.java index 750ab5238437e..e9428a79590d2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreControlPlaneVersionPropertiesFormat.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Packet core control plane version properties. */ +/** + * Packet core control plane version properties. + */ @Fluent public final class PacketCoreControlPlaneVersionPropertiesFormat { /* @@ -25,13 +27,15 @@ public final class PacketCoreControlPlaneVersionPropertiesFormat { @JsonProperty(value = "platforms") private List platforms; - /** Creates an instance of PacketCoreControlPlaneVersionPropertiesFormat class. */ + /** + * Creates an instance of PacketCoreControlPlaneVersionPropertiesFormat class. + */ public PacketCoreControlPlaneVersionPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the packet core control plane version resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -40,7 +44,7 @@ public ProvisioningState provisioningState() { /** * Get the platforms property: Platform specific packet core control plane version properties. - * + * * @return the platforms value. */ public List platforms() { @@ -49,7 +53,7 @@ public List platforms() { /** * Set the platforms property: Platform specific packet core control plane version properties. - * + * * @param platforms the platforms value to set. * @return the PacketCoreControlPlaneVersionPropertiesFormat object itself. */ @@ -60,7 +64,7 @@ public PacketCoreControlPlaneVersionPropertiesFormat withPlatforms(List tags) { super.withTags(tags); @@ -67,7 +75,7 @@ public PacketCoreDataPlaneInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the packet core data plane resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -77,7 +85,7 @@ public ProvisioningState provisioningState() { /** * Get the userPlaneAccessInterface property: The user plane interface on the access network. For 5G networks, this * is the N3 interface. For 4G networks, this is the S1-U interface. - * + * * @return the userPlaneAccessInterface value. */ public InterfaceProperties userPlaneAccessInterface() { @@ -87,7 +95,7 @@ public InterfaceProperties userPlaneAccessInterface() { /** * Set the userPlaneAccessInterface property: The user plane interface on the access network. For 5G networks, this * is the N3 interface. For 4G networks, this is the S1-U interface. - * + * * @param userPlaneAccessInterface the userPlaneAccessInterface value to set. * @return the PacketCoreDataPlaneInner object itself. */ @@ -100,11 +108,11 @@ public PacketCoreDataPlaneInner withUserPlaneAccessInterface(InterfaceProperties } /** - * Get the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the access - * network in a High Availability (HA) system. In an HA deployment the access network router should be configured to - * forward traffic for this address to the control plane access interface on the active or standby node. In non-HA - * system this list should be omitted or empty. - * + * Get the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the + * access network in a High Availability (HA) system. In an HA deployment the access network router should be + * configured to forward traffic for this address to the control plane access interface on the active or standby + * node. In non-HA system this list should be omitted or empty. + * * @return the userPlaneAccessVirtualIpv4Addresses value. */ public List userPlaneAccessVirtualIpv4Addresses() { @@ -112,16 +120,16 @@ public List userPlaneAccessVirtualIpv4Addresses() { } /** - * Set the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the access - * network in a High Availability (HA) system. In an HA deployment the access network router should be configured to - * forward traffic for this address to the control plane access interface on the active or standby node. In non-HA - * system this list should be omitted or empty. - * + * Set the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the + * access network in a High Availability (HA) system. In an HA deployment the access network router should be + * configured to forward traffic for this address to the control plane access interface on the active or standby + * node. In non-HA system this list should be omitted or empty. + * * @param userPlaneAccessVirtualIpv4Addresses the userPlaneAccessVirtualIpv4Addresses value to set. * @return the PacketCoreDataPlaneInner object itself. */ - public PacketCoreDataPlaneInner withUserPlaneAccessVirtualIpv4Addresses( - List userPlaneAccessVirtualIpv4Addresses) { + public PacketCoreDataPlaneInner + withUserPlaneAccessVirtualIpv4Addresses(List userPlaneAccessVirtualIpv4Addresses) { if (this.innerProperties() == null) { this.innerProperties = new PacketCoreDataPlanePropertiesFormat(); } @@ -131,15 +139,13 @@ public PacketCoreDataPlaneInner withUserPlaneAccessVirtualIpv4Addresses( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model PacketCoreDataPlaneInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerProperties in model PacketCoreDataPlaneInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreDataPlanePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreDataPlanePropertiesFormat.java index 70860b884fc2e..975b491f978e9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreDataPlanePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/PacketCoreDataPlanePropertiesFormat.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Packet core data plane properties. */ +/** + * Packet core data plane properties. + */ @Fluent public final class PacketCoreDataPlanePropertiesFormat { /* @@ -35,13 +37,15 @@ public final class PacketCoreDataPlanePropertiesFormat { @JsonProperty(value = "userPlaneAccessVirtualIpv4Addresses") private List userPlaneAccessVirtualIpv4Addresses; - /** Creates an instance of PacketCoreDataPlanePropertiesFormat class. */ + /** + * Creates an instance of PacketCoreDataPlanePropertiesFormat class. + */ public PacketCoreDataPlanePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the packet core data plane resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -51,7 +55,7 @@ public ProvisioningState provisioningState() { /** * Get the userPlaneAccessInterface property: The user plane interface on the access network. For 5G networks, this * is the N3 interface. For 4G networks, this is the S1-U interface. - * + * * @return the userPlaneAccessInterface value. */ public InterfaceProperties userPlaneAccessInterface() { @@ -61,22 +65,22 @@ public InterfaceProperties userPlaneAccessInterface() { /** * Set the userPlaneAccessInterface property: The user plane interface on the access network. For 5G networks, this * is the N3 interface. For 4G networks, this is the S1-U interface. - * + * * @param userPlaneAccessInterface the userPlaneAccessInterface value to set. * @return the PacketCoreDataPlanePropertiesFormat object itself. */ - public PacketCoreDataPlanePropertiesFormat withUserPlaneAccessInterface( - InterfaceProperties userPlaneAccessInterface) { + public PacketCoreDataPlanePropertiesFormat + withUserPlaneAccessInterface(InterfaceProperties userPlaneAccessInterface) { this.userPlaneAccessInterface = userPlaneAccessInterface; return this; } /** - * Get the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the access - * network in a High Availability (HA) system. In an HA deployment the access network router should be configured to - * forward traffic for this address to the control plane access interface on the active or standby node. In non-HA - * system this list should be omitted or empty. - * + * Get the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the + * access network in a High Availability (HA) system. In an HA deployment the access network router should be + * configured to forward traffic for this address to the control plane access interface on the active or standby + * node. In non-HA system this list should be omitted or empty. + * * @return the userPlaneAccessVirtualIpv4Addresses value. */ public List userPlaneAccessVirtualIpv4Addresses() { @@ -84,32 +88,29 @@ public List userPlaneAccessVirtualIpv4Addresses() { } /** - * Set the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the access - * network in a High Availability (HA) system. In an HA deployment the access network router should be configured to - * forward traffic for this address to the control plane access interface on the active or standby node. In non-HA - * system this list should be omitted or empty. - * + * Set the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane on the + * access network in a High Availability (HA) system. In an HA deployment the access network router should be + * configured to forward traffic for this address to the control plane access interface on the active or standby + * node. In non-HA system this list should be omitted or empty. + * * @param userPlaneAccessVirtualIpv4Addresses the userPlaneAccessVirtualIpv4Addresses value to set. * @return the PacketCoreDataPlanePropertiesFormat object itself. */ - public PacketCoreDataPlanePropertiesFormat withUserPlaneAccessVirtualIpv4Addresses( - List userPlaneAccessVirtualIpv4Addresses) { + public PacketCoreDataPlanePropertiesFormat + withUserPlaneAccessVirtualIpv4Addresses(List userPlaneAccessVirtualIpv4Addresses) { this.userPlaneAccessVirtualIpv4Addresses = userPlaneAccessVirtualIpv4Addresses; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (userPlaneAccessInterface() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property userPlaneAccessInterface in model" - + " PacketCoreDataPlanePropertiesFormat")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property userPlaneAccessInterface in model PacketCoreDataPlanePropertiesFormat")); } else { userPlaneAccessInterface().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServiceInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServiceInner.java index 6c90b2e68528f..99d0c3d1828af 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServiceInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServiceInner.java @@ -15,7 +15,9 @@ import java.util.List; import java.util.Map; -/** Service resource. Must be created in the same location as its parent mobile network. */ +/** + * Service resource. Must be created in the same location as its parent mobile network. + */ @Fluent public final class ServiceInner extends Resource { /* @@ -30,13 +32,15 @@ public final class ServiceInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of ServiceInner class. */ + /** + * Creates an instance of ServiceInner class. + */ public ServiceInner() { } /** * Get the innerProperties property: Service Properties. - * + * * @return the innerProperties value. */ private ServicePropertiesFormat innerProperties() { @@ -45,21 +49,25 @@ private ServicePropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ServiceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ServiceInner withTags(Map tags) { super.withTags(tags); @@ -68,7 +76,7 @@ public ServiceInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the service resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -79,7 +87,7 @@ public ProvisioningState provisioningState() { * Get the servicePrecedence property: A precedence value that is used to decide between services when identifying * the QoS values to use for a particular SIM. A lower value means a higher priority. This value should be unique * among all services configured in the mobile network. - * + * * @return the servicePrecedence value. */ public int servicePrecedence() { @@ -90,7 +98,7 @@ public int servicePrecedence() { * Set the servicePrecedence property: A precedence value that is used to decide between services when identifying * the QoS values to use for a particular SIM. A lower value means a higher priority. This value should be unique * among all services configured in the mobile network. - * + * * @param servicePrecedence the servicePrecedence value to set. * @return the ServiceInner object itself. */ @@ -106,7 +114,7 @@ public ServiceInner withServicePrecedence(int servicePrecedence) { * Get the serviceQosPolicy property: The QoS policy to use for packets matching this service. This can be * overridden for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null * then the UE's SIM policy will define the QoS settings. - * + * * @return the serviceQosPolicy value. */ public QosPolicy serviceQosPolicy() { @@ -117,7 +125,7 @@ public QosPolicy serviceQosPolicy() { * Set the serviceQosPolicy property: The QoS policy to use for packets matching this service. This can be * overridden for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null * then the UE's SIM policy will define the QoS settings. - * + * * @param serviceQosPolicy the serviceQosPolicy value to set. * @return the ServiceInner object itself. */ @@ -131,7 +139,7 @@ public ServiceInner withServiceQosPolicy(QosPolicy serviceQosPolicy) { /** * Get the pccRules property: The set of data flow policy rules that make up this service. - * + * * @return the pccRules value. */ public List pccRules() { @@ -140,7 +148,7 @@ public List pccRules() { /** * Set the pccRules property: The set of data flow policy rules that make up this service. - * + * * @param pccRules the pccRules value to set. * @return the ServiceInner object itself. */ @@ -154,14 +162,13 @@ public ServiceInner withPccRules(List pccRules) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model ServiceInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model ServiceInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServicePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServicePropertiesFormat.java index 654a4cc236114..af4da8514913d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServicePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/ServicePropertiesFormat.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Service properties. */ +/** + * Service properties. + */ @Fluent public final class ServicePropertiesFormat { /* @@ -43,13 +45,15 @@ public final class ServicePropertiesFormat { @JsonProperty(value = "pccRules", required = true) private List pccRules; - /** Creates an instance of ServicePropertiesFormat class. */ + /** + * Creates an instance of ServicePropertiesFormat class. + */ public ServicePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the service resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -60,7 +64,7 @@ public ProvisioningState provisioningState() { * Get the servicePrecedence property: A precedence value that is used to decide between services when identifying * the QoS values to use for a particular SIM. A lower value means a higher priority. This value should be unique * among all services configured in the mobile network. - * + * * @return the servicePrecedence value. */ public int servicePrecedence() { @@ -71,7 +75,7 @@ public int servicePrecedence() { * Set the servicePrecedence property: A precedence value that is used to decide between services when identifying * the QoS values to use for a particular SIM. A lower value means a higher priority. This value should be unique * among all services configured in the mobile network. - * + * * @param servicePrecedence the servicePrecedence value to set. * @return the ServicePropertiesFormat object itself. */ @@ -84,7 +88,7 @@ public ServicePropertiesFormat withServicePrecedence(int servicePrecedence) { * Get the serviceQosPolicy property: The QoS policy to use for packets matching this service. This can be * overridden for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null * then the UE's SIM policy will define the QoS settings. - * + * * @return the serviceQosPolicy value. */ public QosPolicy serviceQosPolicy() { @@ -95,7 +99,7 @@ public QosPolicy serviceQosPolicy() { * Set the serviceQosPolicy property: The QoS policy to use for packets matching this service. This can be * overridden for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null * then the UE's SIM policy will define the QoS settings. - * + * * @param serviceQosPolicy the serviceQosPolicy value to set. * @return the ServicePropertiesFormat object itself. */ @@ -106,7 +110,7 @@ public ServicePropertiesFormat withServiceQosPolicy(QosPolicy serviceQosPolicy) /** * Get the pccRules property: The set of data flow policy rules that make up this service. - * + * * @return the pccRules value. */ public List pccRules() { @@ -115,7 +119,7 @@ public List pccRules() { /** * Set the pccRules property: The set of data flow policy rules that make up this service. - * + * * @param pccRules the pccRules value to set. * @return the ServicePropertiesFormat object itself. */ @@ -126,7 +130,7 @@ public ServicePropertiesFormat withPccRules(List pccRules) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -134,10 +138,8 @@ public void validate() { serviceQosPolicy().validate(); } if (pccRules() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property pccRules in model ServicePropertiesFormat")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property pccRules in model ServicePropertiesFormat")); } else { pccRules().forEach(e -> e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupInner.java index 7cac6a0e55210..3ba49621d535e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupInner.java @@ -15,7 +15,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** SIM group resource. */ +/** + * SIM group resource. + */ @Fluent public final class SimGroupInner extends Resource { /* @@ -36,13 +38,15 @@ public final class SimGroupInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of SimGroupInner class. */ + /** + * Creates an instance of SimGroupInner class. + */ public SimGroupInner() { } /** * Get the innerProperties property: SIM group Properties. - * + * * @return the innerProperties value. */ private SimGroupPropertiesFormat innerProperties() { @@ -51,7 +55,7 @@ private SimGroupPropertiesFormat innerProperties() { /** * Get the identity property: The identity used to retrieve the encryption key from Azure key vault. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -60,7 +64,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The identity used to retrieve the encryption key from Azure key vault. - * + * * @param identity the identity value to set. * @return the SimGroupInner object itself. */ @@ -71,21 +75,25 @@ public SimGroupInner withIdentity(ManagedServiceIdentity identity) { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimGroupInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimGroupInner withTags(Map tags) { super.withTags(tags); @@ -94,7 +102,7 @@ public SimGroupInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the SIM group resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -103,7 +111,7 @@ public ProvisioningState provisioningState() { /** * Get the encryptionKey property: A key to encrypt the SIM data that belongs to this SIM group. - * + * * @return the encryptionKey value. */ public KeyVaultKey encryptionKey() { @@ -112,7 +120,7 @@ public KeyVaultKey encryptionKey() { /** * Set the encryptionKey property: A key to encrypt the SIM data that belongs to this SIM group. - * + * * @param encryptionKey the encryptionKey value to set. * @return the SimGroupInner object itself. */ @@ -127,7 +135,7 @@ public SimGroupInner withEncryptionKey(KeyVaultKey encryptionKey) { /** * Get the mobileNetwork property: Mobile network that this SIM group belongs to. The mobile network must be in the * same location as the SIM group. - * + * * @return the mobileNetwork value. */ public MobileNetworkResourceId mobileNetwork() { @@ -137,7 +145,7 @@ public MobileNetworkResourceId mobileNetwork() { /** * Set the mobileNetwork property: Mobile network that this SIM group belongs to. The mobile network must be in the * same location as the SIM group. - * + * * @param mobileNetwork the mobileNetwork value to set. * @return the SimGroupInner object itself. */ @@ -151,14 +159,13 @@ public SimGroupInner withMobileNetwork(MobileNetworkResourceId mobileNetwork) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model SimGroupInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model SimGroupInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupPropertiesFormat.java index d838802e973b5..bc4a0e8532cfe 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimGroupPropertiesFormat.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.mobilenetwork.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** SIM group properties. */ +/** + * SIM group properties. + */ @Fluent public final class SimGroupPropertiesFormat { /* @@ -31,13 +33,15 @@ public final class SimGroupPropertiesFormat { @JsonProperty(value = "mobileNetwork") private MobileNetworkResourceId mobileNetwork; - /** Creates an instance of SimGroupPropertiesFormat class. */ + /** + * Creates an instance of SimGroupPropertiesFormat class. + */ public SimGroupPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the SIM group resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -46,7 +50,7 @@ public ProvisioningState provisioningState() { /** * Get the encryptionKey property: A key to encrypt the SIM data that belongs to this SIM group. - * + * * @return the encryptionKey value. */ public KeyVaultKey encryptionKey() { @@ -55,7 +59,7 @@ public KeyVaultKey encryptionKey() { /** * Set the encryptionKey property: A key to encrypt the SIM data that belongs to this SIM group. - * + * * @param encryptionKey the encryptionKey value to set. * @return the SimGroupPropertiesFormat object itself. */ @@ -67,7 +71,7 @@ public SimGroupPropertiesFormat withEncryptionKey(KeyVaultKey encryptionKey) { /** * Get the mobileNetwork property: Mobile network that this SIM group belongs to. The mobile network must be in the * same location as the SIM group. - * + * * @return the mobileNetwork value. */ public MobileNetworkResourceId mobileNetwork() { @@ -77,7 +81,7 @@ public MobileNetworkResourceId mobileNetwork() { /** * Set the mobileNetwork property: Mobile network that this SIM group belongs to. The mobile network must be in the * same location as the SIM group. - * + * * @param mobileNetwork the mobileNetwork value to set. * @return the SimGroupPropertiesFormat object itself. */ @@ -88,7 +92,7 @@ public SimGroupPropertiesFormat withMobileNetwork(MobileNetworkResourceId mobile /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimInner.java index f22ec09be992d..8cd3b53c9e9d4 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimInner.java @@ -17,7 +17,9 @@ import java.util.List; import java.util.Map; -/** SIM resource. */ +/** + * SIM resource. + */ @Fluent public final class SimInner extends ProxyResource { /* @@ -32,13 +34,15 @@ public final class SimInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of SimInner class. */ + /** + * Creates an instance of SimInner class. + */ public SimInner() { } /** * Get the innerProperties property: SIM Properties. - * + * * @return the innerProperties value. */ private SimPropertiesFormat innerProperties() { @@ -47,7 +51,7 @@ private SimPropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -56,7 +60,7 @@ public SystemData systemData() { /** * Get the authenticationKey property: The Ki value for the SIM. - * + * * @return the authenticationKey value. */ public String authenticationKey() { @@ -65,7 +69,7 @@ public String authenticationKey() { /** * Set the authenticationKey property: The Ki value for the SIM. - * + * * @param authenticationKey the authenticationKey value to set. * @return the SimInner object itself. */ @@ -79,7 +83,7 @@ public SimInner withAuthenticationKey(String authenticationKey) { /** * Get the operatorKeyCode property: The Opc value for the SIM. - * + * * @return the operatorKeyCode value. */ public String operatorKeyCode() { @@ -88,7 +92,7 @@ public String operatorKeyCode() { /** * Set the operatorKeyCode property: The Opc value for the SIM. - * + * * @param operatorKeyCode the operatorKeyCode value to set. * @return the SimInner object itself. */ @@ -102,7 +106,7 @@ public SimInner withOperatorKeyCode(String operatorKeyCode) { /** * Get the provisioningState property: The provisioning state of the SIM resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -111,7 +115,7 @@ public ProvisioningState provisioningState() { /** * Get the simState property: The state of the SIM resource. - * + * * @return the simState value. */ public SimState simState() { @@ -119,8 +123,9 @@ public SimState simState() { } /** - * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that site. - * + * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that + * site. + * * @return the siteProvisioningState value. */ public Map siteProvisioningState() { @@ -130,7 +135,7 @@ public Map siteProvisioningState() { /** * Get the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @return the internationalMobileSubscriberIdentity value. */ public String internationalMobileSubscriberIdentity() { @@ -140,7 +145,7 @@ public String internationalMobileSubscriberIdentity() { /** * Set the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @param internationalMobileSubscriberIdentity the internationalMobileSubscriberIdentity value to set. * @return the SimInner object itself. */ @@ -154,7 +159,7 @@ public SimInner withInternationalMobileSubscriberIdentity(String internationalMo /** * Get the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @return the integratedCircuitCardIdentifier value. */ public String integratedCircuitCardIdentifier() { @@ -163,7 +168,7 @@ public String integratedCircuitCardIdentifier() { /** * Set the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @param integratedCircuitCardIdentifier the integratedCircuitCardIdentifier value to set. * @return the SimInner object itself. */ @@ -176,10 +181,10 @@ public SimInner withIntegratedCircuitCardIdentifier(String integratedCircuitCard } /** - * Get the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Get the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @return the deviceType value. */ public String deviceType() { @@ -187,10 +192,10 @@ public String deviceType() { } /** - * Set the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Set the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @param deviceType the deviceType value to set. * @return the SimInner object itself. */ @@ -205,7 +210,7 @@ public SimInner withDeviceType(String deviceType) { /** * Get the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @return the simPolicy value. */ public SimPolicyResourceId simPolicy() { @@ -215,7 +220,7 @@ public SimPolicyResourceId simPolicy() { /** * Set the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @param simPolicy the simPolicy value to set. * @return the SimInner object itself. */ @@ -230,7 +235,7 @@ public SimInner withSimPolicy(SimPolicyResourceId simPolicy) { /** * Get the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @return the staticIpConfiguration value. */ public List staticIpConfiguration() { @@ -240,7 +245,7 @@ public List staticIpConfiguration() { /** * Set the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @param staticIpConfiguration the staticIpConfiguration value to set. * @return the SimInner object itself. */ @@ -254,7 +259,7 @@ public SimInner withStaticIpConfiguration(List staticIpCo /** * Get the vendorName property: The name of the SIM vendor who provided this SIM, if any. - * + * * @return the vendorName value. */ public String vendorName() { @@ -264,7 +269,7 @@ public String vendorName() { /** * Get the vendorKeyFingerprint property: The public key fingerprint of the SIM vendor who provided this SIM, if * any. - * + * * @return the vendorKeyFingerprint value. */ public String vendorKeyFingerprint() { @@ -273,14 +278,13 @@ public String vendorKeyFingerprint() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model SimInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model SimInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyInner.java index 8f514be7225d8..928b713337f30 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyInner.java @@ -17,7 +17,9 @@ import java.util.List; import java.util.Map; -/** SIM policy resource. */ +/** + * SIM policy resource. + */ @Fluent public final class SimPolicyInner extends Resource { /* @@ -32,13 +34,15 @@ public final class SimPolicyInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of SimPolicyInner class. */ + /** + * Creates an instance of SimPolicyInner class. + */ public SimPolicyInner() { } /** * Get the innerProperties property: SIM policy Properties. - * + * * @return the innerProperties value. */ private SimPolicyPropertiesFormat innerProperties() { @@ -47,21 +51,25 @@ private SimPolicyPropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPolicyInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPolicyInner withTags(Map tags) { super.withTags(tags); @@ -70,7 +78,7 @@ public SimPolicyInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the SIM policy resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -80,7 +88,7 @@ public ProvisioningState provisioningState() { /** * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM policy on * that site. - * + * * @return the siteProvisioningState value. */ public Map siteProvisioningState() { @@ -90,7 +98,7 @@ public Map siteProvisioningState() { /** * Get the ueAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of all PDU sessions of a given * UE. See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. - * + * * @return the ueAmbr value. */ public Ambr ueAmbr() { @@ -100,7 +108,7 @@ public Ambr ueAmbr() { /** * Set the ueAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of all PDU sessions of a given * UE. See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. - * + * * @param ueAmbr the ueAmbr value to set. * @return the SimPolicyInner object itself. */ @@ -113,9 +121,9 @@ public SimPolicyInner withUeAmbr(Ambr ueAmbr) { } /** - * Get the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice must - * exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. - * + * Get the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice + * must exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. + * * @return the defaultSlice value. */ public SliceResourceId defaultSlice() { @@ -123,9 +131,9 @@ public SliceResourceId defaultSlice() { } /** - * Set the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice must - * exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. - * + * Set the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice + * must exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. + * * @param defaultSlice the defaultSlice value to set. * @return the SimPolicyInner object itself. */ @@ -140,7 +148,7 @@ public SimPolicyInner withDefaultSlice(SliceResourceId defaultSlice) { /** * Get the rfspIndex property: RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This is an * optional setting and by default is unspecified. - * + * * @return the rfspIndex value. */ public Integer rfspIndex() { @@ -150,7 +158,7 @@ public Integer rfspIndex() { /** * Set the rfspIndex property: RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This is an * optional setting and by default is unspecified. - * + * * @param rfspIndex the rfspIndex value to set. * @return the SimPolicyInner object itself. */ @@ -165,7 +173,7 @@ public SimPolicyInner withRfspIndex(Integer rfspIndex) { /** * Get the registrationTimer property: UE periodic registration update timer (5G) or UE periodic tracking area * update timer (4G), in seconds. - * + * * @return the registrationTimer value. */ public Integer registrationTimer() { @@ -175,7 +183,7 @@ public Integer registrationTimer() { /** * Set the registrationTimer property: UE periodic registration update timer (5G) or UE periodic tracking area * update timer (4G), in seconds. - * + * * @param registrationTimer the registrationTimer value to set. * @return the SimPolicyInner object itself. */ @@ -190,7 +198,7 @@ public SimPolicyInner withRegistrationTimer(Integer registrationTimer) { /** * Get the sliceConfigurations property: The allowed slices and the settings to use for them. The list must not * contain duplicate items and must contain at least one item. - * + * * @return the sliceConfigurations value. */ public List sliceConfigurations() { @@ -200,7 +208,7 @@ public List sliceConfigurations() { /** * Set the sliceConfigurations property: The allowed slices and the settings to use for them. The list must not * contain duplicate items and must contain at least one item. - * + * * @param sliceConfigurations the sliceConfigurations value to set. * @return the SimPolicyInner object itself. */ @@ -214,14 +222,13 @@ public SimPolicyInner withSliceConfigurations(List sliceConf /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model SimPolicyInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model SimPolicyInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyPropertiesFormat.java index 2ed0c17cf8610..96518ebcf26d4 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPolicyPropertiesFormat.java @@ -16,7 +16,9 @@ import java.util.List; import java.util.Map; -/** SIM policy properties. Must be created in the same location as its parent mobile network. */ +/** + * SIM policy properties. Must be created in the same location as its parent mobile network. + */ @Fluent public final class SimPolicyPropertiesFormat { /* @@ -66,13 +68,15 @@ public final class SimPolicyPropertiesFormat { @JsonProperty(value = "sliceConfigurations", required = true) private List sliceConfigurations; - /** Creates an instance of SimPolicyPropertiesFormat class. */ + /** + * Creates an instance of SimPolicyPropertiesFormat class. + */ public SimPolicyPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the SIM policy resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -82,7 +86,7 @@ public ProvisioningState provisioningState() { /** * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM policy on * that site. - * + * * @return the siteProvisioningState value. */ public Map siteProvisioningState() { @@ -92,7 +96,7 @@ public Map siteProvisioningState() { /** * Get the ueAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of all PDU sessions of a given * UE. See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. - * + * * @return the ueAmbr value. */ public Ambr ueAmbr() { @@ -102,7 +106,7 @@ public Ambr ueAmbr() { /** * Set the ueAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of all PDU sessions of a given * UE. See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. - * + * * @param ueAmbr the ueAmbr value to set. * @return the SimPolicyPropertiesFormat object itself. */ @@ -112,9 +116,9 @@ public SimPolicyPropertiesFormat withUeAmbr(Ambr ueAmbr) { } /** - * Get the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice must - * exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. - * + * Get the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice + * must exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. + * * @return the defaultSlice value. */ public SliceResourceId defaultSlice() { @@ -122,9 +126,9 @@ public SliceResourceId defaultSlice() { } /** - * Set the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice must - * exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. - * + * Set the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice + * must exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. + * * @param defaultSlice the defaultSlice value to set. * @return the SimPolicyPropertiesFormat object itself. */ @@ -136,7 +140,7 @@ public SimPolicyPropertiesFormat withDefaultSlice(SliceResourceId defaultSlice) /** * Get the rfspIndex property: RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This is an * optional setting and by default is unspecified. - * + * * @return the rfspIndex value. */ public Integer rfspIndex() { @@ -146,7 +150,7 @@ public Integer rfspIndex() { /** * Set the rfspIndex property: RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This is an * optional setting and by default is unspecified. - * + * * @param rfspIndex the rfspIndex value to set. * @return the SimPolicyPropertiesFormat object itself. */ @@ -158,7 +162,7 @@ public SimPolicyPropertiesFormat withRfspIndex(Integer rfspIndex) { /** * Get the registrationTimer property: UE periodic registration update timer (5G) or UE periodic tracking area * update timer (4G), in seconds. - * + * * @return the registrationTimer value. */ public Integer registrationTimer() { @@ -168,7 +172,7 @@ public Integer registrationTimer() { /** * Set the registrationTimer property: UE periodic registration update timer (5G) or UE periodic tracking area * update timer (4G), in seconds. - * + * * @param registrationTimer the registrationTimer value to set. * @return the SimPolicyPropertiesFormat object itself. */ @@ -180,7 +184,7 @@ public SimPolicyPropertiesFormat withRegistrationTimer(Integer registrationTimer /** * Get the sliceConfigurations property: The allowed slices and the settings to use for them. The list must not * contain duplicate items and must contain at least one item. - * + * * @return the sliceConfigurations value. */ public List sliceConfigurations() { @@ -190,7 +194,7 @@ public List sliceConfigurations() { /** * Set the sliceConfigurations property: The allowed slices and the settings to use for them. The list must not * contain duplicate items and must contain at least one item. - * + * * @param sliceConfigurations the sliceConfigurations value to set. * @return the SimPolicyPropertiesFormat object itself. */ @@ -201,31 +205,25 @@ public SimPolicyPropertiesFormat withSliceConfigurations(List e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPropertiesFormat.java index cee914dedcef4..66fe830a6af0c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SimPropertiesFormat.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** SIM properties. */ +/** + * SIM properties. + */ @Fluent public final class SimPropertiesFormat extends CommonSimPropertiesFormat { /* @@ -26,13 +28,15 @@ public final class SimPropertiesFormat extends CommonSimPropertiesFormat { @JsonProperty(value = "operatorKeyCode") private String operatorKeyCode; - /** Creates an instance of SimPropertiesFormat class. */ + /** + * Creates an instance of SimPropertiesFormat class. + */ public SimPropertiesFormat() { } /** * Get the authenticationKey property: The Ki value for the SIM. - * + * * @return the authenticationKey value. */ public String authenticationKey() { @@ -41,7 +45,7 @@ public String authenticationKey() { /** * Set the authenticationKey property: The Ki value for the SIM. - * + * * @param authenticationKey the authenticationKey value to set. * @return the SimPropertiesFormat object itself. */ @@ -52,7 +56,7 @@ public SimPropertiesFormat withAuthenticationKey(String authenticationKey) { /** * Get the operatorKeyCode property: The Opc value for the SIM. - * + * * @return the operatorKeyCode value. */ public String operatorKeyCode() { @@ -61,7 +65,7 @@ public String operatorKeyCode() { /** * Set the operatorKeyCode property: The Opc value for the SIM. - * + * * @param operatorKeyCode the operatorKeyCode value to set. * @return the SimPropertiesFormat object itself. */ @@ -70,35 +74,45 @@ public SimPropertiesFormat withOperatorKeyCode(String operatorKeyCode) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPropertiesFormat withInternationalMobileSubscriberIdentity(String internationalMobileSubscriberIdentity) { super.withInternationalMobileSubscriberIdentity(internationalMobileSubscriberIdentity); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPropertiesFormat withIntegratedCircuitCardIdentifier(String integratedCircuitCardIdentifier) { super.withIntegratedCircuitCardIdentifier(integratedCircuitCardIdentifier); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPropertiesFormat withDeviceType(String deviceType) { super.withDeviceType(deviceType); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPropertiesFormat withSimPolicy(SimPolicyResourceId simPolicy) { super.withSimPolicy(simPolicy); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SimPropertiesFormat withStaticIpConfiguration(List staticIpConfiguration) { super.withStaticIpConfiguration(staticIpConfiguration); @@ -107,7 +121,7 @@ public SimPropertiesFormat withStaticIpConfiguration(List /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SiteInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SiteInner.java index 2f23bcc491d66..975eff841488e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SiteInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SiteInner.java @@ -13,7 +13,9 @@ import java.util.List; import java.util.Map; -/** Site resource. Must be created in the same location as its parent mobile network. */ +/** + * Site resource. Must be created in the same location as its parent mobile network. + */ @Fluent public final class SiteInner extends Resource { /* @@ -28,13 +30,15 @@ public final class SiteInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of SiteInner class. */ + /** + * Creates an instance of SiteInner class. + */ public SiteInner() { } /** * Get the innerProperties property: Site properties. - * + * * @return the innerProperties value. */ private SitePropertiesFormat innerProperties() { @@ -43,21 +47,25 @@ private SitePropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SiteInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SiteInner withTags(Map tags) { super.withTags(tags); @@ -66,7 +74,7 @@ public SiteInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the site resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -76,7 +84,7 @@ public ProvisioningState provisioningState() { /** * Get the networkFunctions property: An array of IDs of the network functions deployed in the site. Deleting the * site will delete any network functions that are deployed in the site. - * + * * @return the networkFunctions value. */ public List networkFunctions() { @@ -85,7 +93,7 @@ public List networkFunctions() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SitePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SitePropertiesFormat.java index 72744498a4e96..3ee9bca1f7443 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SitePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SitePropertiesFormat.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Site properties. */ +/** + * Site properties. + */ @Immutable public final class SitePropertiesFormat { /* @@ -26,13 +28,15 @@ public final class SitePropertiesFormat { @JsonProperty(value = "networkFunctions", access = JsonProperty.Access.WRITE_ONLY) private List networkFunctions; - /** Creates an instance of SitePropertiesFormat class. */ + /** + * Creates an instance of SitePropertiesFormat class. + */ public SitePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the site resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -42,7 +46,7 @@ public ProvisioningState provisioningState() { /** * Get the networkFunctions property: An array of IDs of the network functions deployed in the site. Deleting the * site will delete any network functions that are deployed in the site. - * + * * @return the networkFunctions value. */ public List networkFunctions() { @@ -51,7 +55,7 @@ public List networkFunctions() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SliceInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SliceInner.java index e4a8cd69b7c12..da808950ec548 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SliceInner.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SliceInner.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Network slice resource. Must be created in the same location as its parent mobile network. */ +/** + * Network slice resource. Must be created in the same location as its parent mobile network. + */ @Fluent public final class SliceInner extends Resource { /* @@ -28,13 +30,15 @@ public final class SliceInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of SliceInner class. */ + /** + * Creates an instance of SliceInner class. + */ public SliceInner() { } /** * Get the innerProperties property: Slice properties. - * + * * @return the innerProperties value. */ private SlicePropertiesFormat innerProperties() { @@ -43,21 +47,25 @@ private SlicePropertiesFormat innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SliceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SliceInner withTags(Map tags) { super.withTags(tags); @@ -66,7 +74,7 @@ public SliceInner withTags(Map tags) { /** * Get the provisioningState property: The provisioning state of the network slice resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -76,7 +84,7 @@ public ProvisioningState provisioningState() { /** * Get the snssai property: Single-network slice selection assistance information (S-NSSAI). Unique at the scope of * a mobile network. - * + * * @return the snssai value. */ public Snssai snssai() { @@ -86,7 +94,7 @@ public Snssai snssai() { /** * Set the snssai property: Single-network slice selection assistance information (S-NSSAI). Unique at the scope of * a mobile network. - * + * * @param snssai the snssai value to set. * @return the SliceInner object itself. */ @@ -100,7 +108,7 @@ public SliceInner withSnssai(Snssai snssai) { /** * Get the description property: An optional description for this network slice. - * + * * @return the description value. */ public String description() { @@ -109,7 +117,7 @@ public String description() { /** * Set the description property: An optional description for this network slice. - * + * * @param description the description value to set. * @return the SliceInner object itself. */ @@ -123,14 +131,13 @@ public SliceInner withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model SliceInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model SliceInner")); } else { innerProperties().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SlicePropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SlicePropertiesFormat.java index b33adc3b3f972..6e59b0240de66 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SlicePropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/SlicePropertiesFormat.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.mobilenetwork.models.Snssai; import com.fasterxml.jackson.annotation.JsonProperty; -/** Network slice properties. */ +/** + * Network slice properties. + */ @Fluent public final class SlicePropertiesFormat { /* @@ -31,13 +33,15 @@ public final class SlicePropertiesFormat { @JsonProperty(value = "description") private String description; - /** Creates an instance of SlicePropertiesFormat class. */ + /** + * Creates an instance of SlicePropertiesFormat class. + */ public SlicePropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the network slice resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -47,7 +51,7 @@ public ProvisioningState provisioningState() { /** * Get the snssai property: Single-network slice selection assistance information (S-NSSAI). Unique at the scope of * a mobile network. - * + * * @return the snssai value. */ public Snssai snssai() { @@ -57,7 +61,7 @@ public Snssai snssai() { /** * Set the snssai property: Single-network slice selection assistance information (S-NSSAI). Unique at the scope of * a mobile network. - * + * * @param snssai the snssai value to set. * @return the SlicePropertiesFormat object itself. */ @@ -68,7 +72,7 @@ public SlicePropertiesFormat withSnssai(Snssai snssai) { /** * Get the description property: An optional description for this network slice. - * + * * @return the description value. */ public String description() { @@ -77,7 +81,7 @@ public String description() { /** * Set the description property: An optional description for this network slice. - * + * * @param description the description value to set. * @return the SlicePropertiesFormat object itself. */ @@ -88,14 +92,13 @@ public SlicePropertiesFormat withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (snssai() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property snssai in model SlicePropertiesFormat")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property snssai in model SlicePropertiesFormat")); } else { snssai().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/UeInfoInner.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/UeInfoInner.java new file mode 100644 index 0000000000000..ac78c1b36fb57 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/UeInfoInner.java @@ -0,0 +1,81 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoPropertiesFormat; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Basic UE Information. + */ +@Fluent +public final class UeInfoInner extends ProxyResource { + /* + * Basic UE Information Properties. + */ + @JsonProperty(value = "properties", required = true) + private UeInfoPropertiesFormat properties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of UeInfoInner class. + */ + public UeInfoInner() { + } + + /** + * Get the properties property: Basic UE Information Properties. + * + * @return the properties value. + */ + public UeInfoPropertiesFormat properties() { + return this.properties; + } + + /** + * Set the properties property: Basic UE Information Properties. + * + * @param properties the properties value to set. + * @return the UeInfoInner object itself. + */ + public UeInfoInner withProperties(UeInfoPropertiesFormat properties) { + this.properties = properties; + return this; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property properties in model UeInfoInner")); + } else { + properties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeInfoInner.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/package-info.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/package-info.java index 762580a15592a..041cf7835826a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/package-info.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/models/package-info.java @@ -3,8 +3,8 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the inner data models for MobileNetworkManagementClient. The resources in this API specification - * will be used to manage attached data network resources in mobile network attached to a particular packet core - * instance. + * Package containing the inner data models for MobileNetworkManagementClient. + * The resources in this API specification will be used to manage attached data network resources in mobile network + * attached to a particular packet core instance. */ package com.azure.resourcemanager.mobilenetwork.fluent.models; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/package-info.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/package-info.java index ccaa823e7a113..d9ab88114860a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/package-info.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/fluent/package-info.java @@ -3,8 +3,8 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the service clients for MobileNetworkManagementClient. The resources in this API specification - * will be used to manage attached data network resources in mobile network attached to a particular packet core - * instance. + * Package containing the service clients for MobileNetworkManagementClient. + * The resources in this API specification will be used to manage attached data network resources in mobile network + * attached to a particular packet core instance. */ package com.azure.resourcemanager.mobilenetwork.fluent; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AsyncOperationStatusImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AsyncOperationStatusImpl.java index 482cb076415f9..3678e7eacb1b1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AsyncOperationStatusImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AsyncOperationStatusImpl.java @@ -14,8 +14,7 @@ public final class AsyncOperationStatusImpl implements AsyncOperationStatus { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - AsyncOperationStatusImpl( - AsyncOperationStatusInner innerObject, + AsyncOperationStatusImpl(AsyncOperationStatusInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworkImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworkImpl.java index 8ad6ffd456c1e..4844078ed1b7b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworkImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworkImpl.java @@ -121,8 +121,8 @@ private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { private TagsObject updateParameters; - public AttachedDataNetworkImpl withExistingPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + public AttachedDataNetworkImpl withExistingPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { this.resourceGroupName = resourceGroupName; this.packetCoreControlPlaneName = packetCoreControlPlaneName; this.packetCoreDataPlaneName = packetCoreDataPlaneName; @@ -130,32 +130,15 @@ public AttachedDataNetworkImpl withExistingPacketCoreDataPlane( } public AttachedDataNetwork create() { - this.innerObject = - serviceManager - .serviceClient() - .getAttachedDataNetworks() - .createOrUpdate( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.innerModel(), - Context.NONE); + this.innerObject = serviceManager.serviceClient().getAttachedDataNetworks().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, this.innerModel(), + Context.NONE); return this; } public AttachedDataNetwork create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAttachedDataNetworks() - .createOrUpdate( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.innerModel(), - context); + this.innerObject = serviceManager.serviceClient().getAttachedDataNetworks().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, this.innerModel(), context); return this; } @@ -171,75 +154,46 @@ public AttachedDataNetworkImpl update() { } public AttachedDataNetwork apply() { - this.innerObject = - serviceManager - .serviceClient() - .getAttachedDataNetworks() - .updateTagsWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - updateParameters, - Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAttachedDataNetworks() + .updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, updateParameters, Context.NONE) + .getValue(); return this; } public AttachedDataNetwork apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAttachedDataNetworks() - .updateTagsWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - updateParameters, - context) - .getValue(); + this.innerObject = serviceManager + .serviceClient().getAttachedDataNetworks().updateTagsWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, updateParameters, context) + .getValue(); return this; } - AttachedDataNetworkImpl( - AttachedDataNetworkInner innerObject, + AttachedDataNetworkImpl(AttachedDataNetworkInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.packetCoreControlPlaneName = Utils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); - this.packetCoreDataPlaneName = Utils.getValueFromIdByName(innerObject.id(), "packetCoreDataPlanes"); - this.attachedDataNetworkName = Utils.getValueFromIdByName(innerObject.id(), "attachedDataNetworks"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.packetCoreControlPlaneName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); + this.packetCoreDataPlaneName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCoreDataPlanes"); + this.attachedDataNetworkName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "attachedDataNetworks"); } public AttachedDataNetwork refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getAttachedDataNetworks() - .getWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - Context.NONE) + this.innerObject + = serviceManager + .serviceClient().getAttachedDataNetworks().getWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, Context.NONE) .getValue(); return this; } public AttachedDataNetwork refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAttachedDataNetworks() - .getWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAttachedDataNetworks().getWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, context).getValue(); return this; } @@ -283,8 +237,8 @@ public AttachedDataNetworkImpl withUserEquipmentAddressPoolPrefix(List u return this; } - public AttachedDataNetworkImpl withUserEquipmentStaticAddressPoolPrefix( - List userEquipmentStaticAddressPoolPrefix) { + public AttachedDataNetworkImpl + withUserEquipmentStaticAddressPoolPrefix(List userEquipmentStaticAddressPoolPrefix) { this.innerModel().withUserEquipmentStaticAddressPoolPrefix(userEquipmentStaticAddressPoolPrefix); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksClientImpl.java index 705c26c90891c..6aa816e925800 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksClientImpl.java @@ -40,23 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AttachedDataNetworksClient. */ +/** + * An instance of this class provides access to all the operations defined in AttachedDataNetworksClient. + */ public final class AttachedDataNetworksClientImpl implements AttachedDataNetworksClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AttachedDataNetworksService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of AttachedDataNetworksClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AttachedDataNetworksClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy - .create(AttachedDataNetworksService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AttachedDataNetworksService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -67,101 +72,80 @@ public final class AttachedDataNetworksClientImpl implements AttachedDataNetwork @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface AttachedDataNetworksService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, @PathParam("attachedDataNetworkName") String attachedDataNetworkName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, @PathParam("attachedDataNetworkName") String attachedDataNetworkName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, @PathParam("attachedDataNetworkName") String attachedDataNetworkName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") AttachedDataNetworkInner parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") AttachedDataNetworkInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks/{attachedDataNetworkName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, @PathParam("attachedDataNetworkName") String attachedDataNetworkName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") TagsObject parameters, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}/attachedDataNetworks") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByPacketCoreDataPlane( - @HostParam("$host") String endpoint, + Mono> listByPacketCoreDataPlane(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByPacketCoreDataPlaneNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -172,64 +156,43 @@ Mono> listByPacketCoreDataPlaneNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -241,62 +204,43 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, this.client.getApiVersion(), + accept, context); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -307,23 +251,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { - Mono>> mono = - deleteWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName) { + Mono>> mono = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, attachedDataNetworkName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -335,28 +273,19 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, attachedDataNetworkName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -367,20 +296,15 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { - return this - .beginDeleteAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName) - .getSyncPoller(); + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName).getSyncPoller(); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -392,25 +316,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context) { - return this - .beginDeleteAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context) - .getSyncPoller(); + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, context).getSyncPoller(); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -421,20 +335,15 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { - return beginDeleteAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName).last().flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -446,25 +355,15 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context) { - return beginDeleteAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -474,10 +373,7 @@ private Mono deleteAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName) { deleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName) .block(); @@ -485,7 +381,7 @@ public void delete( /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -496,24 +392,15 @@ public void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context) { - deleteAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context) - .block(); + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, Context context) { + deleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + context).block(); } /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -522,67 +409,46 @@ public void delete( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified attached data network along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -592,65 +458,46 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified attached data network along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, this.client.getApiVersion(), + accept, context); } /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -661,19 +508,15 @@ private Mono> getWithResponseAsync( * @return information about the specified attached data network on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { - return getWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono getAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName) { + return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -685,24 +528,16 @@ private Mono getAsync( * @return information about the specified attached data network along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + public Response getWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { - return getWithResponseAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context) - .block(); + return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, context).block(); } /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -713,24 +548,16 @@ public Response getWithResponse( * @return information about the specified attached data network. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AttachedDataNetworkInner get( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { - return getWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - Context.NONE) - .getValue(); + public AttachedDataNetworkInner get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName) { + return getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, Context.NONE).getValue(); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -742,43 +569,32 @@ public AttachedDataNetworkInner get( * @return attached data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, AttachedDataNetworkInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -787,27 +603,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -820,44 +625,32 @@ private Mono>> createOrUpdateWithResponseAsync( * @return attached data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, + AttachedDataNetworkInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -866,24 +659,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, this.client.getApiVersion(), + parameters, accept, context); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -896,32 +680,19 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, AttachedDataNetworkInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AttachedDataNetworkInner.class, - AttachedDataNetworkInner.class, - this.client.getContext()); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, AttachedDataNetworkInner parameters) { + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AttachedDataNetworkInner.class, AttachedDataNetworkInner.class, + this.client.getContext()); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -935,35 +706,19 @@ private PollerFlux, AttachedDataNetworkInne */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, AttachedDataNetworkInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, - Context context) { + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, AttachedDataNetworkInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters, - context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AttachedDataNetworkInner.class, - AttachedDataNetworkInner.class, - context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AttachedDataNetworkInner.class, AttachedDataNetworkInner.class, context); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -976,25 +731,16 @@ private PollerFlux, AttachedDataNetworkInne */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, AttachedDataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters) - .getSyncPoller(); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, AttachedDataNetworkInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters).getSyncPoller(); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1008,27 +754,16 @@ public SyncPoller, AttachedDataNetworkInner */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, AttachedDataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters, - context) - .getSyncPoller(); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, AttachedDataNetworkInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters, context).getSyncPoller(); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1040,26 +775,17 @@ public SyncPoller, AttachedDataNetworkInner * @return attached data network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, AttachedDataNetworkInner parameters) { - return beginCreateOrUpdateAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters) - .last() - .flatMap(this.client::getLroFinalResultOrError); + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters).last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1072,28 +798,17 @@ private Mono createOrUpdateAsync( * @return attached data network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters, - context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, + AttachedDataNetworkInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1105,25 +820,16 @@ private Mono createOrUpdateAsync( * @return attached data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AttachedDataNetworkInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters) { - return createOrUpdateAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters) - .block(); + public AttachedDataNetworkInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, AttachedDataNetworkInner parameters) { + return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters).block(); } /** * Creates or updates an attached data network. Must be created in the same location as its parent packet core data * plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1136,26 +842,16 @@ public AttachedDataNetworkInner createOrUpdate( * @return attached data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AttachedDataNetworkInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - AttachedDataNetworkInner parameters, + public AttachedDataNetworkInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, AttachedDataNetworkInner parameters, Context context) { - return createOrUpdateAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters, - context) - .block(); + return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters, context).block(); } /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1167,43 +863,32 @@ public AttachedDataNetworkInner createOrUpdate( * @return attached data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1212,26 +897,15 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1244,44 +918,32 @@ private Mono> updateTagsWithResponseAsync( * @return attached data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - TagsObject parameters, - Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, + TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (attachedDataNetworkName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter attachedDataNetworkName is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1290,23 +952,14 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - this.client.getApiVersion(), - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, this.client.getApiVersion(), + parameters, accept, context); } /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1318,24 +971,15 @@ private Mono> updateTagsWithResponseAsync( * @return attached data network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - TagsObject parameters) { - return updateTagsWithResponseAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono updateTagsAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, TagsObject parameters) { + return updateTagsWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1348,26 +992,16 @@ private Mono updateTagsAsync( * @return attached data network resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - TagsObject parameters, - Context context) { - return updateTagsWithResponseAsync( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters, - context) - .block(); + public Response updateTagsWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName, + TagsObject parameters, Context context) { + return updateTagsWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters, context).block(); } /** * Updates an attached data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1379,25 +1013,15 @@ public Response updateTagsWithResponse( * @return attached data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AttachedDataNetworkInner updateTags( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - TagsObject parameters) { - return updateTagsWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - parameters, - Context.NONE) - .getValue(); + public AttachedDataNetworkInner updateTags(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, TagsObject parameters) { + return updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, parameters, Context.NONE).getValue(); } /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1405,67 +1029,44 @@ public AttachedDataNetworkInner updateTags( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the attached data networks associated with a packet core data plane along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByPacketCoreDataPlaneSinglePageAsync( String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByPacketCoreDataPlane( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByPacketCoreDataPlane(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1474,86 +1075,63 @@ private Mono> listByPacketCoreDataPlaneS * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the attached data networks associated with a packet core data plane along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByPacketCoreDataPlaneSinglePageAsync( String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByPacketCoreDataPlane( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByPacketCoreDataPlane(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedFlux}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreDataPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { - return new PagedFlux<>( - () -> - listByPacketCoreDataPlaneSinglePageAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName), + private PagedFlux listByPacketCoreDataPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + return new PagedFlux<>(() -> listByPacketCoreDataPlaneSinglePageAsync(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName), nextLink -> listByPacketCoreDataPlaneNextSinglePageAsync(nextLink)); } /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1561,41 +1139,40 @@ private PagedFlux listByPacketCoreDataPlaneAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedFlux}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreDataPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + private PagedFlux listByPacketCoreDataPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { return new PagedFlux<>( - () -> - listByPacketCoreDataPlaneSinglePageAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context), + () -> listByPacketCoreDataPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, context), nextLink -> listByPacketCoreDataPlaneNextSinglePageAsync(nextLink, context)); } /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedIterable}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + public PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { return new PagedIterable<>( listByPacketCoreDataPlaneAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName)); } /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1603,92 +1180,74 @@ public PagedIterable listByPacketCoreDataPlane( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedIterable}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { - return new PagedIterable<>( - listByPacketCoreDataPlaneAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context)); + public PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + return new PagedIterable<>(listByPacketCoreDataPlaneAsync(resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for attached data network API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreDataPlaneNextSinglePageAsync( - String nextLink) { + private Mono> + listByPacketCoreDataPlaneNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByPacketCoreDataPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for attached data network API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreDataPlaneNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByPacketCoreDataPlaneNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByPacketCoreDataPlaneNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByPacketCoreDataPlaneNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksImpl.java index 4f7ea67e7257a..88f93f4edb01e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/AttachedDataNetworksImpl.java @@ -21,74 +21,40 @@ public final class AttachedDataNetworksImpl implements AttachedDataNetworks { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public AttachedDataNetworksImpl( - AttachedDataNetworksClient innerClient, + public AttachedDataNetworksImpl(AttachedDataNetworksClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName) { - this - .serviceClient() - .delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName); + this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName); } - public void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context) { - this - .serviceClient() - .delete( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context); + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, Context context) { + this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, context); } - public Response getWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context); + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AttachedDataNetworkImpl(inner.getValue(), this.manager())); } else { return null; } } - public AttachedDataNetwork get( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName) { - AttachedDataNetworkInner inner = - this - .serviceClient() - .get(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName); + public AttachedDataNetwork get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName) { + AttachedDataNetworkInner inner = this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, attachedDataNetworkName); if (inner != null) { return new AttachedDataNetworkImpl(inner, this.manager()); } else { @@ -96,216 +62,118 @@ public AttachedDataNetwork get( } } - public PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { - PagedIterable inner = - this - .serviceClient() - .listByPacketCoreDataPlane(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); - return Utils.mapPage(inner, inner1 -> new AttachedDataNetworkImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + PagedIterable inner = this.serviceClient() + .listByPacketCoreDataPlane(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AttachedDataNetworkImpl(inner1, this.manager())); } - public PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { - PagedIterable inner = - this - .serviceClient() - .listByPacketCoreDataPlane( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); - return Utils.mapPage(inner, inner1 -> new AttachedDataNetworkImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + PagedIterable inner = this.serviceClient() + .listByPacketCoreDataPlane(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AttachedDataNetworkImpl(inner1, this.manager())); } public AttachedDataNetwork getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } - String attachedDataNetworkName = Utils.getValueFromIdByName(id, "attachedDataNetworks"); + String attachedDataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "attachedDataNetworks"); if (attachedDataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", id))); } - return this - .getWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - Context.NONE) - .getValue(); + return this.getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } - String attachedDataNetworkName = Utils.getValueFromIdByName(id, "attachedDataNetworks"); + String attachedDataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "attachedDataNetworks"); if (attachedDataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", id))); } - return this - .getWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context); + return this.getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + attachedDataNetworkName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } - String attachedDataNetworkName = Utils.getValueFromIdByName(id, "attachedDataNetworks"); + String attachedDataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "attachedDataNetworks"); if (attachedDataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", id))); } - this - .delete( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - Context.NONE); + this.delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } - String attachedDataNetworkName = Utils.getValueFromIdByName(id, "attachedDataNetworks"); + String attachedDataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "attachedDataNetworks"); if (attachedDataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'attachedDataNetworks'.", id))); } - this - .delete( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - attachedDataNetworkName, - context); + this.delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, attachedDataNetworkName, + context); } private AttachedDataNetworksClient serviceClient() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworkImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworkImpl.java index 74f931a569238..d319d4238a10e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworkImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworkImpl.java @@ -91,20 +91,14 @@ public DataNetworkImpl withExistingMobileNetwork(String resourceGroupName, Strin } public DataNetwork create() { - this.innerObject = - serviceManager - .serviceClient() - .getDataNetworks() - .createOrUpdate(resourceGroupName, mobileNetworkName, dataNetworkName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getDataNetworks().createOrUpdate(resourceGroupName, + mobileNetworkName, dataNetworkName, this.innerModel(), Context.NONE); return this; } public DataNetwork create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDataNetworks() - .createOrUpdate(resourceGroupName, mobileNetworkName, dataNetworkName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getDataNetworks().createOrUpdate(resourceGroupName, + mobileNetworkName, dataNetworkName, this.innerModel(), context); return this; } @@ -120,53 +114,36 @@ public DataNetworkImpl update() { } public DataNetwork apply() { - this.innerObject = - serviceManager - .serviceClient() - .getDataNetworks() - .updateTagsWithResponse( - resourceGroupName, mobileNetworkName, dataNetworkName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDataNetworks().updateTagsWithResponse(resourceGroupName, + mobileNetworkName, dataNetworkName, updateParameters, Context.NONE).getValue(); return this; } public DataNetwork apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDataNetworks() - .updateTagsWithResponse( - resourceGroupName, mobileNetworkName, dataNetworkName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDataNetworks() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, updateParameters, context) + .getValue(); return this; } - DataNetworkImpl( - DataNetworkInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + DataNetworkImpl(DataNetworkInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.mobileNetworkName = Utils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); - this.dataNetworkName = Utils.getValueFromIdByName(innerObject.id(), "dataNetworks"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); + this.dataNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "dataNetworks"); } public DataNetwork refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getDataNetworks() - .getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDataNetworks() + .getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, Context.NONE).getValue(); return this; } public DataNetwork refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDataNetworks() - .getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDataNetworks() + .getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksClientImpl.java index e8239faa0008c..3e3cd0e603465 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksClientImpl.java @@ -40,22 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DataNetworksClient. */ +/** + * An instance of this class provides access to all the operations defined in DataNetworksClient. + */ public final class DataNetworksClientImpl implements DataNetworksClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DataNetworksService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of DataNetworksClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ DataNetworksClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(DataNetworksService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(DataNetworksService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,96 +72,73 @@ public final class DataNetworksClientImpl implements DataNetworksClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface DataNetworksService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("dataNetworkName") String dataNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("dataNetworkName") String dataNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("dataNetworkName") String dataNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("dataNetworkName") String dataNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("dataNetworkName") String dataNetworkName, - @BodyParam("application/json") DataNetworkInner parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") DataNetworkInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks/{dataNetworkName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("dataNetworkName") String dataNetworkName, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("dataNetworkName") String dataNetworkName, @BodyParam("application/json") TagsObject parameters, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/dataNetworks") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByMobileNetwork( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> listByMobileNetwork(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("mobileNetworkName") String mobileNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByMobileNetworkNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -165,19 +148,15 @@ Mono> listByMobileNetworkNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -193,24 +172,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, dataNetworkName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -221,19 +190,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -249,21 +214,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, dataNetworkName, accept, context); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -273,19 +230,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -296,19 +251,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -318,14 +272,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String dataNetworkName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String dataNetworkName) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, dataNetworkName).getSyncPoller(); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -336,14 +290,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, dataNetworkName, context).getSyncPoller(); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -354,14 +308,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String dataNetworkName) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, dataNetworkName) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName, dataNetworkName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -372,16 +325,15 @@ private Mono deleteAsync(String resourceGroupName, String mobileNetworkNam * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, dataNetworkName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + Context context) { + return beginDeleteAsync(resourceGroupName, mobileNetworkName, dataNetworkName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -396,7 +348,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String da /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -412,7 +364,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String da /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -420,22 +372,18 @@ public void delete(String resourceGroupName, String mobileNetworkName, String da * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified data network along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -451,24 +399,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, dataNetworkName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -477,22 +415,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified data network along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -508,21 +442,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, dataNetworkName, accept, context); } /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -532,15 +458,15 @@ private Mono> getWithResponseAsync( * @return information about the specified data network on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName) { + private Mono getAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName) { return getWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -551,14 +477,14 @@ private Mono getAsync( * @return information about the specified data network along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context) { return getWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, context).block(); } /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -574,7 +500,7 @@ public DataNetworkInner get(String resourceGroupName, String mobileNetworkName, /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -585,19 +511,15 @@ public DataNetworkInner get(String resourceGroupName, String mobileNetworkName, * @return data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -618,25 +540,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, dataNetworkName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -648,23 +560,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -685,22 +589,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, dataNetworkName, parameters, accept, context); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -713,21 +608,15 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, DataNetworkInner> beginCreateOrUpdateAsync( String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DataNetworkInner.class, - DataNetworkInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DataNetworkInner.class, DataNetworkInner.class, this.client.getContext()); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -740,23 +629,18 @@ private PollerFlux, DataNetworkInner> beginCreateOr */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, DataNetworkInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, + String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DataNetworkInner.class, DataNetworkInner.class, context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, + dataNetworkName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DataNetworkInner.class, DataNetworkInner.class, context); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -767,16 +651,15 @@ private PollerFlux, DataNetworkInner> beginCreateOr * @return the {@link SyncPoller} for polling of data network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters) + public SyncPoller, DataNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters) .getSyncPoller(); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -788,20 +671,15 @@ public SyncPoller, DataNetworkInner> beginCreateOrU * @return the {@link SyncPoller} for polling of data network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DataNetworkInner> beginCreateOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, context) + public SyncPoller, DataNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, context) .getSyncPoller(); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -812,16 +690,15 @@ public SyncPoller, DataNetworkInner> beginCreateOrU * @return data network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, DataNetworkInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -833,20 +710,15 @@ private Mono createOrUpdateAsync( * @return data network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, - Context context) { + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, DataNetworkInner parameters, Context context) { return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -857,14 +729,14 @@ private Mono createOrUpdateAsync( * @return data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DataNetworkInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, DataNetworkInner parameters) { + public DataNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + DataNetworkInner parameters) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters).block(); } /** * Creates or updates a data network. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -876,18 +748,14 @@ public DataNetworkInner createOrUpdate( * @return data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DataNetworkInner createOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - DataNetworkInner parameters, - Context context) { + public DataNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + DataNetworkInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, context).block(); } /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -898,19 +766,15 @@ public DataNetworkInner createOrUpdate( * @return data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -931,25 +795,15 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, dataNetworkName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -961,23 +815,15 @@ private Mono> updateTagsWithResponseAsync( * @return data network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - TagsObject parameters, - Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String dataNetworkName, TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -998,22 +844,13 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - dataNetworkName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, dataNetworkName, parameters, accept, context); } /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -1024,15 +861,15 @@ private Mono> updateTagsWithResponseAsync( * @return data network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, TagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, TagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -1044,19 +881,15 @@ private Mono updateTagsAsync( * @return data network resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, - String mobileNetworkName, - String dataNetworkName, - TagsObject parameters, - Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, TagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, context) .block(); } /** * Updates data network tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -1067,37 +900,33 @@ public Response updateTagsWithResponse( * @return data network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DataNetworkInner updateTags( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, TagsObject parameters) { + public DataNetworkInner updateTags(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + TagsObject parameters) { return updateTagsWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, parameters, Context.NONE) .getValue(); } /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for data network API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1109,32 +938,16 @@ private Mono> listByMobileNetworkSinglePageAsync } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByMobileNetwork( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - mobileNetworkName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByMobileNetwork(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, mobileNetworkName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1142,22 +955,18 @@ private Mono> listByMobileNetworkSinglePageAsync * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for data network API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1170,28 +979,15 @@ private Mono> listByMobileNetworkSinglePageAsync final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByMobileNetwork( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - mobileNetworkName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByMobileNetwork(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, mobileNetworkName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1201,14 +997,13 @@ private Mono> listByMobileNetworkSinglePageAsync */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink)); } /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1218,16 +1013,15 @@ private PagedFlux listByMobileNetworkAsync(String resourceGrou * @return response for data network API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByMobileNetworkAsync( - String resourceGroupName, String mobileNetworkName, Context context) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), + private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName, + Context context) { + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink, context)); } /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1242,7 +1036,7 @@ public PagedIterable listByMobileNetwork(String resourceGroupN /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1252,21 +1046,22 @@ public PagedIterable listByMobileNetwork(String resourceGroupN * @return response for data network API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { return new PagedIterable<>(listByMobileNetworkAsync(resourceGroupName, mobileNetworkName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for data network API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink) { @@ -1274,63 +1069,45 @@ private Mono> listByMobileNetworkNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for data network API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksImpl.java index af59b942e0d3e..7bc8e8588ad2b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DataNetworksImpl.java @@ -21,8 +21,8 @@ public final class DataNetworksImpl implements DataNetworks { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public DataNetworksImpl( - DataNetworksClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public DataNetworksImpl(DataNetworksClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -35,15 +35,12 @@ public void delete(String resourceGroupName, String mobileNetworkName, String da this.serviceClient().delete(resourceGroupName, mobileNetworkName, dataNetworkName, context); } - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, context); + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, + String dataNetworkName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DataNetworkImpl(inner.getValue(), this.manager())); } else { return null; @@ -60,122 +57,90 @@ public DataNetwork get(String resourceGroupName, String mobileNetworkName, Strin } public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); - return Utils.mapPage(inner, inner1 -> new DataNetworkImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DataNetworkImpl(inner1, this.manager())); } - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); - return Utils.mapPage(inner, inner1 -> new DataNetworkImpl(inner1, this.manager())); + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DataNetworkImpl(inner1, this.manager())); } public DataNetwork getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String dataNetworkName = Utils.getValueFromIdByName(id, "dataNetworks"); + String dataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "dataNetworks"); if (dataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String dataNetworkName = Utils.getValueFromIdByName(id, "dataNetworks"); + String dataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "dataNetworks"); if (dataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, dataNetworkName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String dataNetworkName = Utils.getValueFromIdByName(id, "dataNetworks"); + String dataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "dataNetworks"); if (dataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); } this.delete(resourceGroupName, mobileNetworkName, dataNetworkName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String dataNetworkName = Utils.getValueFromIdByName(id, "dataNetworks"); + String dataNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "dataNetworks"); if (dataNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'dataNetworks'.", id))); } this.delete(resourceGroupName, mobileNetworkName, dataNetworkName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackageImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackageImpl.java index 9330e86db0a93..f7682194510f0 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackageImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackageImpl.java @@ -15,8 +15,7 @@ public final class DiagnosticsPackageImpl implements DiagnosticsPackage { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - DiagnosticsPackageImpl( - DiagnosticsPackageInner innerObject, + DiagnosticsPackageImpl(DiagnosticsPackageInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesClientImpl.java index 2e0c9fe5ad25b..1c3b80bb74c63 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesClientImpl.java @@ -37,22 +37,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DiagnosticsPackagesClient. */ +/** + * An instance of this class provides access to all the operations defined in DiagnosticsPackagesClient. + */ public final class DiagnosticsPackagesClientImpl implements DiagnosticsPackagesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DiagnosticsPackagesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of DiagnosticsPackagesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ DiagnosticsPackagesClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(DiagnosticsPackagesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(DiagnosticsPackagesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -63,79 +69,61 @@ public final class DiagnosticsPackagesClientImpl implements DiagnosticsPackagesC @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface DiagnosticsPackagesService { - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages/{diagnosticsPackageName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages/{diagnosticsPackageName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("diagnosticsPackageName") String diagnosticsPackageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages/{diagnosticsPackageName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages/{diagnosticsPackageName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("diagnosticsPackageName") String diagnosticsPackageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages/{diagnosticsPackageName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages/{diagnosticsPackageName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("diagnosticsPackageName") String diagnosticsPackageName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/diagnosticsPackages") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByPacketCoreControlPlane( - @HostParam("$host") String endpoint, + Mono> listByPacketCoreControlPlane(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByPacketCoreControlPlaneNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -145,55 +133,39 @@ Mono> listByPacketCoreControlPlaneNext( * @return diagnostics package resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (diagnosticsPackageName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - diagnosticsPackageName, - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, this.client.getApiVersion(), + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -204,52 +176,37 @@ private Mono>> createOrUpdateWithResponseAsync( * @return diagnostics package resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (diagnosticsPackageName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - diagnosticsPackageName, - this.client.getApiVersion(), - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, this.client.getApiVersion(), accept, context); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -261,21 +218,16 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, DiagnosticsPackageInner> beginCreateOrUpdateAsync( String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DiagnosticsPackageInner.class, - DiagnosticsPackageInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), DiagnosticsPackageInner.class, DiagnosticsPackageInner.class, + this.client.getContext()); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -289,22 +241,15 @@ private PollerFlux, DiagnosticsPackageInner> private PollerFlux, DiagnosticsPackageInner> beginCreateOrUpdateAsync( String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DiagnosticsPackageInner.class, - DiagnosticsPackageInner.class, - context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), DiagnosticsPackageInner.class, DiagnosticsPackageInner.class, context); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -316,14 +261,13 @@ private PollerFlux, DiagnosticsPackageInner> @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, DiagnosticsPackageInner> beginCreateOrUpdate( String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) .getSyncPoller(); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -343,7 +287,7 @@ public SyncPoller, DiagnosticsPackageInner> /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -353,16 +297,15 @@ public SyncPoller, DiagnosticsPackageInner> * @return diagnostics package resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName) { + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -373,16 +316,15 @@ private Mono createOrUpdateAsync( * @return diagnostics package resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -392,14 +334,14 @@ private Mono createOrUpdateAsync( * @return diagnostics package resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DiagnosticsPackageInner createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { + public DiagnosticsPackageInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName).block(); } /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -410,15 +352,15 @@ public DiagnosticsPackageInner createOrUpdate( * @return diagnostics package resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DiagnosticsPackageInner createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + public DiagnosticsPackageInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context) { return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context) .block(); } /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -426,58 +368,42 @@ public DiagnosticsPackageInner createOrUpdate( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified diagnostics package along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (diagnosticsPackageName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - diagnosticsPackageName, - this.client.getApiVersion(), - accept, - context)) + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -486,55 +412,40 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified diagnostics package along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (diagnosticsPackageName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - diagnosticsPackageName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, this.client.getApiVersion(), accept, context); } /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -544,15 +455,15 @@ private Mono> getWithResponseAsync( * @return information about the specified diagnostics package on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { + private Mono getAsync(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -563,15 +474,15 @@ private Mono getAsync( * @return information about the specified diagnostics package along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + public Response getWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context) .block(); } /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -581,15 +492,15 @@ public Response getWithResponse( * @return information about the specified diagnostics package. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DiagnosticsPackageInner get( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { + public DiagnosticsPackageInner get(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { return getWithResponse(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, Context.NONE) .getValue(); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -599,55 +510,39 @@ public DiagnosticsPackageInner get( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (diagnosticsPackageName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - diagnosticsPackageName, - this.client.getApiVersion(), - accept, - context)) + context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -658,52 +553,37 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (diagnosticsPackageName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter diagnosticsPackageName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - diagnosticsPackageName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, this.client.getApiVersion(), accept, context); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -713,19 +593,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -736,19 +614,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -758,16 +635,15 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - return this - .beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) .getSyncPoller(); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -778,16 +654,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { - return this - .beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context) .getSyncPoller(); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -797,16 +672,15 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName) - .last() + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -817,16 +691,15 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -841,7 +714,7 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -851,75 +724,54 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, + Context context) { deleteAsync(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context).block(); } /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for diagnostics package API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneSinglePageAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono> + listByPacketCoreControlPlaneSinglePageAsync(String resourceGroupName, String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByPacketCoreControlPlane( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByPacketCoreControlPlane(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -927,58 +779,39 @@ private Mono> listByPacketCoreControlPlan * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for diagnostics package API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByPacketCoreControlPlaneSinglePageAsync( String resourceGroupName, String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByPacketCoreControlPlane( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByPacketCoreControlPlane(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -987,8 +820,8 @@ private Mono> listByPacketCoreControlPlan * @return response for diagnostics package API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreControlPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private PagedFlux listByPacketCoreControlPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName) { return new PagedFlux<>( () -> listByPacketCoreControlPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName), nextLink -> listByPacketCoreControlPlaneNextSinglePageAsync(nextLink)); @@ -996,7 +829,7 @@ private PagedFlux listByPacketCoreControlPlaneAsync( /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1006,8 +839,8 @@ private PagedFlux listByPacketCoreControlPlaneAsync( * @return response for diagnostics package API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreControlPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private PagedFlux listByPacketCoreControlPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return new PagedFlux<>( () -> listByPacketCoreControlPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName, context), nextLink -> listByPacketCoreControlPlaneNextSinglePageAsync(nextLink, context)); @@ -1015,7 +848,7 @@ private PagedFlux listByPacketCoreControlPlaneAsync( /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1024,14 +857,14 @@ private PagedFlux listByPacketCoreControlPlaneAsync( * @return response for diagnostics package API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { return new PagedIterable<>(listByPacketCoreControlPlaneAsync(resourceGroupName, packetCoreControlPlaneName)); } /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1041,88 +874,69 @@ public PagedIterable listByPacketCoreControlPlane( * @return response for diagnostics package API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return new PagedIterable<>( listByPacketCoreControlPlaneAsync(resourceGroupName, packetCoreControlPlaneName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for diagnostics package API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneNextSinglePageAsync( - String nextLink) { + private Mono> + listByPacketCoreControlPlaneNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for diagnostics package API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listByPacketCoreControlPlaneNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesImpl.java index 8a630f85c1122..a1153979a83f5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/DiagnosticsPackagesImpl.java @@ -21,17 +21,16 @@ public final class DiagnosticsPackagesImpl implements DiagnosticsPackages { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public DiagnosticsPackagesImpl( - DiagnosticsPackagesClient innerClient, + public DiagnosticsPackagesImpl(DiagnosticsPackagesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public DiagnosticsPackage createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - DiagnosticsPackageInner inner = - this.serviceClient().createOrUpdate(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); + public DiagnosticsPackage createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { + DiagnosticsPackageInner inner = this.serviceClient().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName); if (inner != null) { return new DiagnosticsPackageImpl(inner, this.manager()); } else { @@ -39,12 +38,10 @@ public DiagnosticsPackage createOrUpdate( } } - public DiagnosticsPackage createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { - DiagnosticsPackageInner inner = - this - .serviceClient() - .createOrUpdate(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context); + public DiagnosticsPackage createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context) { + DiagnosticsPackageInner inner = this.serviceClient().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, context); if (inner != null) { return new DiagnosticsPackageImpl(inner, this.manager()); } else { @@ -52,27 +49,22 @@ public DiagnosticsPackage createOrUpdate( } } - public Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context); + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + packetCoreControlPlaneName, diagnosticsPackageName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DiagnosticsPackageImpl(inner.getValue(), this.manager())); } else { return null; } } - public DiagnosticsPackage get( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName) { - DiagnosticsPackageInner inner = - this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); + public DiagnosticsPackage get(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName) { + DiagnosticsPackageInner inner + = this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); if (inner != null) { return new DiagnosticsPackageImpl(inner, this.manager()); } else { @@ -84,23 +76,23 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName); } - public void delete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context) { + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, + Context context) { this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, diagnosticsPackageName, context); } - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { - PagedIterable inner = - this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName); - return Utils.mapPage(inner, inner1 -> new DiagnosticsPackageImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { + PagedIterable inner + = this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DiagnosticsPackageImpl(inner1, this.manager())); } - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - PagedIterable inner = - this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName, context); - return Utils.mapPage(inner, inner1 -> new DiagnosticsPackageImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { + PagedIterable inner + = this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DiagnosticsPackageImpl(inner1, this.manager())); } private DiagnosticsPackagesClient serviceClient() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInfoImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInfoImpl.java new file mode 100644 index 0000000000000..2ee0e6e3710d3 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInfoImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfo; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfoProperties; + +public final class ExtendedUeInfoImpl implements ExtendedUeInfo { + private ExtendedUeInfoInner innerObject; + + private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; + + ExtendedUeInfoImpl(ExtendedUeInfoInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public ExtendedUeInfoProperties properties() { + return this.innerModel().properties(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public ExtendedUeInfoInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsClientImpl.java new file mode 100644 index 0000000000000..cae55b99fcda5 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsClientImpl.java @@ -0,0 +1,205 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.mobilenetwork.fluent.ExtendedUeInformationsClient; +import com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in ExtendedUeInformationsClient. + */ +public final class ExtendedUeInformationsClientImpl implements ExtendedUeInformationsClient { + /** + * The proxy service used to perform REST calls. + */ + private final ExtendedUeInformationsService service; + + /** + * The service client containing this operation class. + */ + private final MobileNetworkManagementClientImpl client; + + /** + * Initializes an instance of ExtendedUeInformationsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + ExtendedUeInformationsClientImpl(MobileNetworkManagementClientImpl client) { + this.service = RestProxy.create(ExtendedUeInformationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for MobileNetworkManagementClientExtendedUeInformations to be used by + * the proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "MobileNetworkManagem") + public interface ExtendedUeInformationsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/ues/{ueId}/extendedInformation/default") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("ueId") String ueId, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String ueId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (packetCoreControlPlaneName == null) { + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); + } + if (ueId == null) { + return Mono.error(new IllegalArgumentException("Parameter ueId is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), packetCoreControlPlaneName, ueId, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String ueId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (packetCoreControlPlaneName == null) { + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); + } + if (ueId == null) { + return Mono.error(new IllegalArgumentException("Parameter ueId is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), packetCoreControlPlaneName, ueId, accept, context); + } + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String packetCoreControlPlaneName, + String ueId) { + return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, ueId) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String ueId, Context context) { + return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, ueId, context).block(); + } + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ExtendedUeInfoInner get(String resourceGroupName, String packetCoreControlPlaneName, String ueId) { + return getWithResponse(resourceGroupName, packetCoreControlPlaneName, ueId, Context.NONE).getValue(); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsImpl.java new file mode 100644 index 0000000000000..e1f1f8bae3e9f --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ExtendedUeInformationsImpl.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.mobilenetwork.fluent.ExtendedUeInformationsClient; +import com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfo; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInformations; + +public final class ExtendedUeInformationsImpl implements ExtendedUeInformations { + private static final ClientLogger LOGGER = new ClientLogger(ExtendedUeInformationsImpl.class); + + private final ExtendedUeInformationsClient innerClient; + + private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; + + public ExtendedUeInformationsImpl(ExtendedUeInformationsClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String ueId, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, packetCoreControlPlaneName, ueId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ExtendedUeInfoImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ExtendedUeInfo get(String resourceGroupName, String packetCoreControlPlaneName, String ueId) { + ExtendedUeInfoInner inner = this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, ueId); + if (inner != null) { + return new ExtendedUeInfoImpl(inner, this.manager()); + } else { + return null; + } + } + + private ExtendedUeInformationsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkImpl.java index e9da6c24a48ae..f339a7ebfe3e3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkImpl.java @@ -8,11 +8,14 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.mobilenetwork.fluent.models.MobileNetworkInner; +import com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject; +import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentity; import com.azure.resourcemanager.mobilenetwork.models.MobileNetwork; import com.azure.resourcemanager.mobilenetwork.models.PlmnId; import com.azure.resourcemanager.mobilenetwork.models.ProvisioningState; -import com.azure.resourcemanager.mobilenetwork.models.TagsObject; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetwork; import java.util.Collections; +import java.util.List; import java.util.Map; public final class MobileNetworkImpl implements MobileNetwork, MobileNetwork.Definition, MobileNetwork.Update { @@ -45,6 +48,10 @@ public Map tags() { } } + public ManagedServiceIdentity identity() { + return this.innerModel().identity(); + } + public SystemData systemData() { return this.innerModel().systemData(); } @@ -57,6 +64,15 @@ public PlmnId publicLandMobileNetworkIdentifier() { return this.innerModel().publicLandMobileNetworkIdentifier(); } + public List publicLandMobileNetworks() { + List inner = this.innerModel().publicLandMobileNetworks(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + public String serviceKey() { return this.innerModel().serviceKey(); } @@ -85,7 +101,7 @@ private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { private String mobileNetworkName; - private TagsObject updateParameters; + private IdentityAndTagsObject updateParameters; public MobileNetworkImpl withExistingResourceGroup(String resourceGroupName) { this.resourceGroupName = resourceGroupName; @@ -93,20 +109,14 @@ public MobileNetworkImpl withExistingResourceGroup(String resourceGroupName) { } public MobileNetwork create() { - this.innerObject = - serviceManager - .serviceClient() - .getMobileNetworks() - .createOrUpdate(resourceGroupName, mobileNetworkName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getMobileNetworks().createOrUpdate(resourceGroupName, + mobileNetworkName, this.innerModel(), Context.NONE); return this; } public MobileNetwork create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getMobileNetworks() - .createOrUpdate(resourceGroupName, mobileNetworkName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getMobileNetworks().createOrUpdate(resourceGroupName, + mobileNetworkName, this.innerModel(), context); return this; } @@ -117,55 +127,39 @@ public MobileNetwork create(Context context) { } public MobileNetworkImpl update() { - this.updateParameters = new TagsObject(); + this.updateParameters = new IdentityAndTagsObject(); return this; } public MobileNetwork apply() { - this.innerObject = - serviceManager - .serviceClient() - .getMobileNetworks() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMobileNetworks() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, updateParameters, Context.NONE).getValue(); return this; } public MobileNetwork apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getMobileNetworks() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMobileNetworks() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, updateParameters, context).getValue(); return this; } - MobileNetworkImpl( - MobileNetworkInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + MobileNetworkImpl(MobileNetworkInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.mobileNetworkName = Utils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); } public MobileNetwork refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getMobileNetworks() - .getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMobileNetworks() + .getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, Context.NONE).getValue(); return this; } public MobileNetwork refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getMobileNetworks() - .getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMobileNetworks() + .getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, context).getValue(); return this; } @@ -194,6 +188,21 @@ public MobileNetworkImpl withTags(Map tags) { } } + public MobileNetworkImpl withIdentity(ManagedServiceIdentity identity) { + if (isInCreateMode()) { + this.innerModel().withIdentity(identity); + return this; + } else { + this.updateParameters.withIdentity(identity); + return this; + } + } + + public MobileNetworkImpl withPublicLandMobileNetworks(List publicLandMobileNetworks) { + this.innerModel().withPublicLandMobileNetworks(publicLandMobileNetworks); + return this; + } + private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientBuilder.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientBuilder.java index c6b2505709daa..e714706f1d7b1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientBuilder.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientBuilder.java @@ -14,8 +14,10 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the MobileNetworkManagementClientImpl type. */ -@ServiceClientBuilder(serviceClients = {MobileNetworkManagementClientImpl.class}) +/** + * A builder for creating a new instance of the MobileNetworkManagementClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { MobileNetworkManagementClientImpl.class }) public final class MobileNetworkManagementClientBuilder { /* * The ID of the target subscription. The value must be an UUID. @@ -24,7 +26,7 @@ public final class MobileNetworkManagementClientBuilder { /** * Sets The ID of the target subscription. The value must be an UUID. - * + * * @param subscriptionId the subscriptionId value. * @return the MobileNetworkManagementClientBuilder. */ @@ -40,7 +42,7 @@ public MobileNetworkManagementClientBuilder subscriptionId(String subscriptionId /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the MobileNetworkManagementClientBuilder. */ @@ -56,7 +58,7 @@ public MobileNetworkManagementClientBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the MobileNetworkManagementClientBuilder. */ @@ -72,7 +74,7 @@ public MobileNetworkManagementClientBuilder environment(AzureEnvironment environ /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the MobileNetworkManagementClientBuilder. */ @@ -88,7 +90,7 @@ public MobileNetworkManagementClientBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the MobileNetworkManagementClientBuilder. */ @@ -104,7 +106,7 @@ public MobileNetworkManagementClientBuilder defaultPollInterval(Duration default /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the MobileNetworkManagementClientBuilder. */ @@ -115,30 +117,20 @@ public MobileNetworkManagementClientBuilder serializerAdapter(SerializerAdapter /** * Builds an instance of MobileNetworkManagementClientImpl with the provided parameters. - * + * * @return an instance of MobileNetworkManagementClientImpl. */ public MobileNetworkManagementClientImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - MobileNetworkManagementClientImpl client = - new MobileNetworkManagementClientImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - this.subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + MobileNetworkManagementClientImpl client = new MobileNetworkManagementClientImpl(localPipeline, + localSerializerAdapter, localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientImpl.java index 36bf3c648c072..5b1b80c4902c5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworkManagementClientImpl.java @@ -25,6 +25,7 @@ import com.azure.resourcemanager.mobilenetwork.fluent.AttachedDataNetworksClient; import com.azure.resourcemanager.mobilenetwork.fluent.DataNetworksClient; import com.azure.resourcemanager.mobilenetwork.fluent.DiagnosticsPackagesClient; +import com.azure.resourcemanager.mobilenetwork.fluent.ExtendedUeInformationsClient; import com.azure.resourcemanager.mobilenetwork.fluent.MobileNetworkManagementClient; import com.azure.resourcemanager.mobilenetwork.fluent.MobileNetworksClient; import com.azure.resourcemanager.mobilenetwork.fluent.OperationsClient; @@ -38,6 +39,7 @@ import com.azure.resourcemanager.mobilenetwork.fluent.SimsClient; import com.azure.resourcemanager.mobilenetwork.fluent.SitesClient; import com.azure.resourcemanager.mobilenetwork.fluent.SlicesClient; +import com.azure.resourcemanager.mobilenetwork.fluent.UeInformationsClient; import java.io.IOException; import java.lang.reflect.Type; import java.nio.ByteBuffer; @@ -47,264 +49,336 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the MobileNetworkManagementClientImpl type. */ +/** + * Initializes a new instance of the MobileNetworkManagementClientImpl type. + */ @ServiceClient(builder = MobileNetworkManagementClientBuilder.class) public final class MobileNetworkManagementClientImpl implements MobileNetworkManagementClient { - /** The ID of the target subscription. The value must be an UUID. */ + /** + * The ID of the target subscription. The value must be an UUID. + */ private final String subscriptionId; /** * Gets The ID of the target subscription. The value must be an UUID. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The AttachedDataNetworksClient object to access its operations. */ + /** + * The AttachedDataNetworksClient object to access its operations. + */ private final AttachedDataNetworksClient attachedDataNetworks; /** * Gets the AttachedDataNetworksClient object to access its operations. - * + * * @return the AttachedDataNetworksClient object. */ public AttachedDataNetworksClient getAttachedDataNetworks() { return this.attachedDataNetworks; } - /** The DataNetworksClient object to access its operations. */ + /** + * The DataNetworksClient object to access its operations. + */ private final DataNetworksClient dataNetworks; /** * Gets the DataNetworksClient object to access its operations. - * + * * @return the DataNetworksClient object. */ public DataNetworksClient getDataNetworks() { return this.dataNetworks; } - /** The DiagnosticsPackagesClient object to access its operations. */ + /** + * The DiagnosticsPackagesClient object to access its operations. + */ private final DiagnosticsPackagesClient diagnosticsPackages; /** * Gets the DiagnosticsPackagesClient object to access its operations. - * + * * @return the DiagnosticsPackagesClient object. */ public DiagnosticsPackagesClient getDiagnosticsPackages() { return this.diagnosticsPackages; } - /** The MobileNetworksClient object to access its operations. */ + /** + * The MobileNetworksClient object to access its operations. + */ private final MobileNetworksClient mobileNetworks; /** * Gets the MobileNetworksClient object to access its operations. - * + * * @return the MobileNetworksClient object. */ public MobileNetworksClient getMobileNetworks() { return this.mobileNetworks; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The PacketCapturesClient object to access its operations. */ + /** + * The PacketCapturesClient object to access its operations. + */ private final PacketCapturesClient packetCaptures; /** * Gets the PacketCapturesClient object to access its operations. - * + * * @return the PacketCapturesClient object. */ public PacketCapturesClient getPacketCaptures() { return this.packetCaptures; } - /** The PacketCoreControlPlanesClient object to access its operations. */ + /** + * The PacketCoreControlPlanesClient object to access its operations. + */ private final PacketCoreControlPlanesClient packetCoreControlPlanes; /** * Gets the PacketCoreControlPlanesClient object to access its operations. - * + * * @return the PacketCoreControlPlanesClient object. */ public PacketCoreControlPlanesClient getPacketCoreControlPlanes() { return this.packetCoreControlPlanes; } - /** The PacketCoreControlPlaneVersionsClient object to access its operations. */ + /** + * The PacketCoreControlPlaneVersionsClient object to access its operations. + */ private final PacketCoreControlPlaneVersionsClient packetCoreControlPlaneVersions; /** * Gets the PacketCoreControlPlaneVersionsClient object to access its operations. - * + * * @return the PacketCoreControlPlaneVersionsClient object. */ public PacketCoreControlPlaneVersionsClient getPacketCoreControlPlaneVersions() { return this.packetCoreControlPlaneVersions; } - /** The PacketCoreDataPlanesClient object to access its operations. */ + /** + * The PacketCoreDataPlanesClient object to access its operations. + */ private final PacketCoreDataPlanesClient packetCoreDataPlanes; /** * Gets the PacketCoreDataPlanesClient object to access its operations. - * + * * @return the PacketCoreDataPlanesClient object. */ public PacketCoreDataPlanesClient getPacketCoreDataPlanes() { return this.packetCoreDataPlanes; } - /** The ServicesClient object to access its operations. */ + /** + * The ServicesClient object to access its operations. + */ private final ServicesClient services; /** * Gets the ServicesClient object to access its operations. - * + * * @return the ServicesClient object. */ public ServicesClient getServices() { return this.services; } - /** The SimsClient object to access its operations. */ + /** + * The SimsClient object to access its operations. + */ private final SimsClient sims; /** * Gets the SimsClient object to access its operations. - * + * * @return the SimsClient object. */ public SimsClient getSims() { return this.sims; } - /** The SimGroupsClient object to access its operations. */ + /** + * The SimGroupsClient object to access its operations. + */ private final SimGroupsClient simGroups; /** * Gets the SimGroupsClient object to access its operations. - * + * * @return the SimGroupsClient object. */ public SimGroupsClient getSimGroups() { return this.simGroups; } - /** The SimPoliciesClient object to access its operations. */ + /** + * The SimPoliciesClient object to access its operations. + */ private final SimPoliciesClient simPolicies; /** * Gets the SimPoliciesClient object to access its operations. - * + * * @return the SimPoliciesClient object. */ public SimPoliciesClient getSimPolicies() { return this.simPolicies; } - /** The SitesClient object to access its operations. */ + /** + * The SitesClient object to access its operations. + */ private final SitesClient sites; /** * Gets the SitesClient object to access its operations. - * + * * @return the SitesClient object. */ public SitesClient getSites() { return this.sites; } - /** The SlicesClient object to access its operations. */ + /** + * The SlicesClient object to access its operations. + */ private final SlicesClient slices; /** * Gets the SlicesClient object to access its operations. - * + * * @return the SlicesClient object. */ public SlicesClient getSlices() { return this.slices; } + /** + * The ExtendedUeInformationsClient object to access its operations. + */ + private final ExtendedUeInformationsClient extendedUeInformations; + + /** + * Gets the ExtendedUeInformationsClient object to access its operations. + * + * @return the ExtendedUeInformationsClient object. + */ + public ExtendedUeInformationsClient getExtendedUeInformations() { + return this.extendedUeInformations; + } + + /** + * The UeInformationsClient object to access its operations. + */ + private final UeInformationsClient ueInformations; + + /** + * Gets the UeInformationsClient object to access its operations. + * + * @return the UeInformationsClient object. + */ + public UeInformationsClient getUeInformations() { + return this.ueInformations; + } + /** * Initializes an instance of MobileNetworkManagementClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -312,19 +386,14 @@ public SlicesClient getSlices() { * @param subscriptionId The ID of the target subscription. The value must be an UUID. * @param endpoint server parameter. */ - MobileNetworkManagementClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + MobileNetworkManagementClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-09-01"; + this.apiVersion = "2024-02-01"; this.attachedDataNetworks = new AttachedDataNetworksClientImpl(this); this.dataNetworks = new DataNetworksClientImpl(this); this.diagnosticsPackages = new DiagnosticsPackagesClientImpl(this); @@ -340,11 +409,13 @@ public SlicesClient getSlices() { this.simPolicies = new SimPoliciesClientImpl(this); this.sites = new SitesClientImpl(this); this.slices = new SlicesClientImpl(this); + this.extendedUeInformations = new ExtendedUeInformationsClientImpl(this); + this.ueInformations = new UeInformationsClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -353,7 +424,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -363,7 +434,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -373,26 +444,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -405,19 +465,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksClientImpl.java index a1157096c6a3a..b5fc562ef23af 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksClientImpl.java @@ -34,28 +34,34 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.mobilenetwork.fluent.MobileNetworksClient; import com.azure.resourcemanager.mobilenetwork.fluent.models.MobileNetworkInner; +import com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject; import com.azure.resourcemanager.mobilenetwork.models.MobileNetworkListResult; -import com.azure.resourcemanager.mobilenetwork.models.TagsObject; import java.nio.ByteBuffer; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in MobileNetworksClient. */ +/** + * An instance of this class provides access to all the operations defined in MobileNetworksClient. + */ public final class MobileNetworksClientImpl implements MobileNetworksClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final MobileNetworksService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of MobileNetworksClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ MobileNetworksClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(MobileNetworksService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(MobileNetworksService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,112 +72,85 @@ public final class MobileNetworksClientImpl implements MobileNetworksClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface MobileNetworksService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, @PathParam("mobileNetworkName") String mobileNetworkName, - @BodyParam("application/json") MobileNetworkInner parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") MobileNetworkInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, @PathParam("mobileNetworkName") String mobileNetworkName, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") IdentityAndTagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.MobileNetwork/mobileNetworks") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -180,19 +159,15 @@ Mono> listByResourceGroupNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -204,23 +179,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -230,19 +196,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -254,20 +216,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, accept, context); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -278,15 +233,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, mobileNetworkName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -296,18 +249,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -322,7 +274,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -332,14 +284,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + Context context) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, context).getSyncPoller(); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -349,14 +301,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -367,14 +318,13 @@ private Mono deleteAsync(String resourceGroupName, String mobileNetworkNam */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, Context context) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, context) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -388,7 +338,7 @@ public void delete(String resourceGroupName, String mobileNetworkName) { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -403,29 +353,25 @@ public void delete(String resourceGroupName, String mobileNetworkName, Context c /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified mobile network along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -438,22 +384,14 @@ private Mono> getByResourceGroupWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -461,22 +399,18 @@ private Mono> getByResourceGroupWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified mobile network along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String mobileNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -488,20 +422,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, accept, context); } /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -517,7 +444,7 @@ private Mono getByResourceGroupAsync(String resourceGroupNam /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -527,14 +454,14 @@ private Mono getByResourceGroupAsync(String resourceGroupNam * @return information about the specified mobile network along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String mobileNetworkName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String mobileNetworkName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, mobileNetworkName, context).block(); } /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -549,7 +476,7 @@ public MobileNetworkInner getByResourceGroup(String resourceGroupName, String mo /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -559,19 +486,15 @@ public MobileNetworkInner getByResourceGroup(String resourceGroupName, String mo * @return mobile network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, MobileNetworkInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -588,24 +511,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -616,19 +529,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return mobile network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, MobileNetworkInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -645,21 +554,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, parameters, accept, context); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -669,23 +570,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of mobile network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, MobileNetworkInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - MobileNetworkInner.class, - MobileNetworkInner.class, - this.client.getContext()); + private PollerFlux, MobileNetworkInner> + beginCreateOrUpdateAsync(String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + MobileNetworkInner.class, MobileNetworkInner.class, this.client.getContext()); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -699,17 +594,15 @@ private PollerFlux, MobileNetworkInner> beginCrea private PollerFlux, MobileNetworkInner> beginCreateOrUpdateAsync( String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), MobileNetworkInner.class, MobileNetworkInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + MobileNetworkInner.class, MobileNetworkInner.class, context); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -719,14 +612,14 @@ private PollerFlux, MobileNetworkInner> beginCrea * @return the {@link SyncPoller} for polling of mobile network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, MobileNetworkInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters) { + public SyncPoller, MobileNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, MobileNetworkInner parameters) { return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters).getSyncPoller(); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -737,14 +630,14 @@ public SyncPoller, MobileNetworkInner> beginCreat * @return the {@link SyncPoller} for polling of mobile network resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, MobileNetworkInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context) { + public SyncPoller, MobileNetworkInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, MobileNetworkInner parameters, Context context) { return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters, context).getSyncPoller(); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -754,16 +647,15 @@ public SyncPoller, MobileNetworkInner> beginCreat * @return mobile network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + MobileNetworkInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -774,16 +666,15 @@ private Mono createOrUpdateAsync( * @return mobile network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + MobileNetworkInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -793,14 +684,14 @@ private Mono createOrUpdateAsync( * @return mobile network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public MobileNetworkInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters) { + public MobileNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, + MobileNetworkInner parameters) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters).block(); } /** * Creates or updates a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param parameters Parameters supplied to the create or update mobile network operation. @@ -811,36 +702,32 @@ public MobileNetworkInner createOrUpdate( * @return mobile network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public MobileNetworkInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, MobileNetworkInner parameters, Context context) { + public MobileNetworkInner createOrUpdate(String resourceGroupName, String mobileNetworkName, + MobileNetworkInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, parameters, context).block(); } /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return mobile network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String mobileNetworkName, IdentityAndTagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -857,27 +744,17 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -885,19 +762,15 @@ private Mono> updateTagsWithResponseAsync( * @return mobile network resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, TagsObject parameters, Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String mobileNetworkName, IdentityAndTagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -914,42 +787,34 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, parameters, accept, context); } /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return mobile network resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String mobileNetworkName, TagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String mobileNetworkName, + IdentityAndTagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -957,132 +822,100 @@ private Mono updateTagsAsync( * @return mobile network resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, TagsObject parameters, Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + IdentityAndTagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, parameters, context).block(); } /** - * Updates mobile network tags. - * + * Updates mobile network tags and managed identity. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. - * @param parameters Parameters supplied to update mobile network tags. + * @param parameters Parameters supplied to update mobile network tags and/or identity. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return mobile network resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public MobileNetworkInner updateTags(String resourceGroupName, String mobileNetworkName, TagsObject parameters) { + public MobileNetworkInner updateTags(String resourceGroupName, String mobileNetworkName, + IdentityAndTagsObject parameters) { return updateTagsWithResponse(resourceGroupName, mobileNetworkName, parameters, Context.NONE).getValue(); } /** * Lists all the mobile networks in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the mobile networks in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the mobile networks in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Lists all the mobile networks in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1091,13 +924,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Lists all the mobile networks in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call as paginated response with {@link PagedIterable}. @@ -1109,7 +942,7 @@ public PagedIterable list() { /** * Lists all the mobile networks in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1123,27 +956,23 @@ public PagedIterable list(Context context) { /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1151,53 +980,34 @@ private Mono> listByResourceGroupSinglePageAsy } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1206,27 +1016,15 @@ private Mono> listByResourceGroupSinglePageAsy final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1235,14 +1033,13 @@ private Mono> listByResourceGroupSinglePageAsy */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1252,14 +1049,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGr */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1273,7 +1069,7 @@ public PagedIterable listByResourceGroup(String resourceGrou /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1288,14 +1084,15 @@ public PagedIterable listByResourceGroup(String resourceGrou /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -1303,76 +1100,59 @@ private Mono> listBySubscriptionNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1380,63 +1160,45 @@ private Mono> listByResourceGroupNextSinglePag return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksImpl.java index f38d239ee364d..fc08239497f5b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/MobileNetworksImpl.java @@ -21,8 +21,8 @@ public final class MobileNetworksImpl implements MobileNetworks { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public MobileNetworksImpl( - MobileNetworksClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public MobileNetworksImpl(MobileNetworksClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -35,15 +35,12 @@ public void delete(String resourceGroupName, String mobileNetworkName, Context c this.serviceClient().delete(resourceGroupName, mobileNetworkName, context); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String mobileNetworkName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String mobileNetworkName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new MobileNetworkImpl(inner.getValue(), this.manager())); } else { return null; @@ -61,100 +58,76 @@ public MobileNetwork getByResourceGroup(String resourceGroupName, String mobileN public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new MobileNetworkImpl(inner1, this.manager())); } public MobileNetwork getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, mobileNetworkName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } this.delete(resourceGroupName, mobileNetworkName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } this.delete(resourceGroupName, mobileNetworkName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationImpl.java index 5fd1a99ccc229..dbdb0cd8f2eb3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationImpl.java @@ -13,8 +13,8 @@ public final class OperationImpl implements Operation { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - OperationImpl( - OperationInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + OperationImpl(OperationInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsClientImpl.java index c5d9424f22b31..c15980e0cef79 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.mobilenetwork.models.OperationList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,30 +62,24 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.MobileNetwork/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of the operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of the operations along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -87,30 +87,21 @@ Mono> listNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of the operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -120,29 +111,19 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of the operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of the operations as paginated response with {@link PagedFlux}. @@ -154,7 +135,7 @@ private PagedFlux listAsync() { /** * Gets a list of the operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -163,13 +144,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets a list of the operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of the operations as paginated response with {@link PagedIterable}. @@ -181,7 +162,7 @@ public PagedIterable list() { /** * Gets a list of the operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -195,9 +176,10 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -209,31 +191,22 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -246,23 +219,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsImpl.java index 13978c7b8c39d..0552ecf68e5ff 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/OperationsImpl.java @@ -19,20 +19,20 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public OperationsImpl(OperationsClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCaptureImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCaptureImpl.java index 5fcfac47edb13..9e4d737c2d3a7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCaptureImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCaptureImpl.java @@ -100,30 +100,22 @@ private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { private String packetCaptureName; - public PacketCaptureImpl withExistingPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { + public PacketCaptureImpl withExistingPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { this.resourceGroupName = resourceGroupName; this.packetCoreControlPlaneName = packetCoreControlPlaneName; return this; } public PacketCapture create() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCaptures() - .createOrUpdate( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPacketCaptures().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.innerModel(), Context.NONE); return this; } public PacketCapture create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCaptures() - .createOrUpdate( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPacketCaptures().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.innerModel(), context); return this; } @@ -138,51 +130,36 @@ public PacketCaptureImpl update() { } public PacketCapture apply() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCaptures() - .createOrUpdate( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPacketCaptures().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.innerModel(), Context.NONE); return this; } public PacketCapture apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCaptures() - .createOrUpdate( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPacketCaptures().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.innerModel(), context); return this; } - PacketCaptureImpl( - PacketCaptureInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + PacketCaptureImpl(PacketCaptureInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.packetCoreControlPlaneName = Utils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); - this.packetCaptureName = Utils.getValueFromIdByName(innerObject.id(), "packetCaptures"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.packetCoreControlPlaneName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); + this.packetCaptureName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCaptures"); } public PacketCapture refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCaptures() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCaptures() + .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, Context.NONE).getValue(); return this; } public PacketCapture refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCaptures() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCaptures() + .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context).getValue(); return this; } @@ -191,9 +168,8 @@ public AsyncOperationStatus stop() { } public AsyncOperationStatus stop(Context context) { - return serviceManager - .packetCaptures() - .stop(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); + return serviceManager.packetCaptures().stop(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, + context); } public PacketCaptureImpl withNetworkInterfaces(List networkInterfaces) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesClientImpl.java index aafef56354b6c..6d21a88673c64 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesClientImpl.java @@ -40,22 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PacketCapturesClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCapturesClient. + */ public final class PacketCapturesClientImpl implements PacketCapturesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PacketCapturesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of PacketCapturesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PacketCapturesClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(PacketCapturesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(PacketCapturesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,95 +72,73 @@ public final class PacketCapturesClientImpl implements PacketCapturesClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface PacketCapturesService { - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @PathParam("packetCaptureName") String packetCaptureName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") PacketCaptureInner parameters, - @HeaderParam("Accept") String accept, + @PathParam("packetCaptureName") String packetCaptureName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") PacketCaptureInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @PathParam("packetCaptureName") String packetCaptureName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("packetCaptureName") String packetCaptureName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @PathParam("packetCaptureName") String packetCaptureName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("packetCaptureName") String packetCaptureName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}/stop") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures/{packetCaptureName}/stop") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> stop( - @HostParam("$host") String endpoint, + Mono>> stop(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @PathParam("packetCaptureName") String packetCaptureName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("packetCaptureName") String packetCaptureName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCaptures") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByPacketCoreControlPlane( - @HostParam("$host") String endpoint, + Mono> listByPacketCoreControlPlane(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByPacketCoreControlPlaneNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -165,32 +149,23 @@ Mono> listByPacketCoreControlPlaneNext( * @return packet capture session resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -203,25 +178,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), + parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -233,33 +198,23 @@ private Mono>> createOrUpdateWithResponseAsync( * @return packet capture session resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -272,22 +227,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), parameters, accept, context); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -299,26 +245,17 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PacketCaptureInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, + String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PacketCaptureInner.class, - PacketCaptureInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + PacketCaptureInner.class, PacketCaptureInner.class, this.client.getContext()); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -331,24 +268,18 @@ private PollerFlux, PacketCaptureInner> beginCrea */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PacketCaptureInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context) { + String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, + PacketCaptureInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), PacketCaptureInner.class, PacketCaptureInner.class, context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + PacketCaptureInner.class, PacketCaptureInner.class, context); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -359,11 +290,8 @@ private PollerFlux, PacketCaptureInner> beginCrea * @return the {@link SyncPoller} for polling of packet capture session resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, PacketCaptureInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters) { + public SyncPoller, PacketCaptureInner> beginCreateOrUpdate(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters) { return this .beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters) .getSyncPoller(); @@ -371,7 +299,7 @@ public SyncPoller, PacketCaptureInner> beginCreat /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -383,21 +311,15 @@ public SyncPoller, PacketCaptureInner> beginCreat * @return the {@link SyncPoller} for polling of packet capture session resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, PacketCaptureInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters, context) - .getSyncPoller(); + public SyncPoller, PacketCaptureInner> beginCreateOrUpdate(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, PacketCaptureInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, + parameters, context).getSyncPoller(); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -408,19 +330,15 @@ public SyncPoller, PacketCaptureInner> beginCreat * @return packet capture session resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters) { + private Mono createOrUpdateAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, PacketCaptureInner parameters) { return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -432,21 +350,15 @@ private Mono createOrUpdateAsync( * @return packet capture session resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, PacketCaptureInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters, + context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -457,18 +369,15 @@ private Mono createOrUpdateAsync( * @return packet capture session resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCaptureInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters) { + public PacketCaptureInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, PacketCaptureInner parameters) { return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters) .block(); } /** * Creates or updates a packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -480,20 +389,15 @@ public PacketCaptureInner createOrUpdate( * @return packet capture session resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCaptureInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCaptureName, - PacketCaptureInner parameters, - Context context) { - return createOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters, context) - .block(); + public PacketCaptureInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, PacketCaptureInner parameters, Context context) { + return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, parameters, + context).block(); } /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -501,32 +405,26 @@ public PacketCaptureInner createOrUpdate( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet capture session along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -535,23 +433,14 @@ private Mono> getWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - accept, - context)) + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -560,32 +449,26 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet capture session along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -593,21 +476,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), accept, context); } /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -617,15 +492,15 @@ private Mono> getWithResponseAsync( * @return information about the specified packet capture session on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + private Mono getAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -636,14 +511,14 @@ private Mono getAsync( * @return information about the specified packet capture session along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context).block(); } /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -653,15 +528,15 @@ public Response getWithResponse( * @return information about the specified packet capture session. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCaptureInner get( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + public PacketCaptureInner get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { return getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, Context.NONE) .getValue(); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -671,29 +546,23 @@ public PacketCaptureInner get( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -702,23 +571,14 @@ private Mono>> deleteWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - accept, - context)) + context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -729,29 +589,23 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -759,21 +613,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), accept, context); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -783,19 +629,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -806,19 +650,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -828,14 +671,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName).getSyncPoller(); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -846,16 +689,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { - return this - .beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) .getSyncPoller(); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -865,16 +707,15 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName) - .last() + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -885,16 +726,15 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -909,7 +749,7 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -919,47 +759,41 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, + Context context) { deleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context).block(); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + private Mono>> stopWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -968,23 +802,14 @@ private Mono>> stopWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .stop( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - accept, - context)) + context -> service.stop(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -992,33 +817,27 @@ private Mono>> stopWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> stopWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + private Mono>> stopWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCaptureName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCaptureName == null) { return Mono @@ -1026,21 +845,13 @@ private Mono>> stopWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .stop( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - packetCaptureName, - this.client.getApiVersion(), - accept, - context); + return service.stop(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, this.client.getApiVersion(), accept, context); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1050,23 +861,18 @@ private Mono>> stopWithResponseAsync( * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginStopAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { - Mono>> mono = - stopWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + private PollerFlux, AsyncOperationStatusInner> + beginStopAsync(String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + Mono>> mono + = stopWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1080,21 +886,15 @@ private PollerFlux, AsyncOperationStatusIn private PollerFlux, AsyncOperationStatusInner> beginStopAsync( String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - stopWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono + = stopWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1104,14 +904,14 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginStop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + public SyncPoller, AsyncOperationStatusInner> + beginStop(String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { return this.beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName).getSyncPoller(); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1124,14 +924,13 @@ public SyncPoller, AsyncOperationStatusInn @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, AsyncOperationStatusInner> beginStop( String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { - return this - .beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) + return this.beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) .getSyncPoller(); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1141,16 +940,15 @@ public SyncPoller, AsyncOperationStatusInn * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono stopAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { - return beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName) - .last() + private Mono stopAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { + return beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1161,16 +959,15 @@ private Mono stopAsync( * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono stopAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { - return beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context) - .last() + private Mono stopAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { + return beginStopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1180,14 +977,14 @@ private Mono stopAsync( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { + public AsyncOperationStatusInner stop(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { return stopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName).block(); } /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -1198,75 +995,54 @@ public AsyncOperationStatusInner stop( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + public AsyncOperationStatusInner stop(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { return stopAsync(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context).block(); } /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet capture API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneSinglePageAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono> + listByPacketCoreControlPlaneSinglePageAsync(String resourceGroupName, String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByPacketCoreControlPlane( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByPacketCoreControlPlane(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1274,58 +1050,39 @@ private Mono> listByPacketCoreControlPlaneSing * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet capture API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByPacketCoreControlPlaneSinglePageAsync( String resourceGroupName, String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByPacketCoreControlPlane( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByPacketCoreControlPlane(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1334,8 +1091,8 @@ private Mono> listByPacketCoreControlPlaneSing * @return response for packet capture API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreControlPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private PagedFlux listByPacketCoreControlPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName) { return new PagedFlux<>( () -> listByPacketCoreControlPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName), nextLink -> listByPacketCoreControlPlaneNextSinglePageAsync(nextLink)); @@ -1343,7 +1100,7 @@ private PagedFlux listByPacketCoreControlPlaneAsync( /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1353,8 +1110,8 @@ private PagedFlux listByPacketCoreControlPlaneAsync( * @return response for packet capture API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreControlPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private PagedFlux listByPacketCoreControlPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return new PagedFlux<>( () -> listByPacketCoreControlPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName, context), nextLink -> listByPacketCoreControlPlaneNextSinglePageAsync(nextLink, context)); @@ -1362,7 +1119,7 @@ private PagedFlux listByPacketCoreControlPlaneAsync( /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1371,14 +1128,14 @@ private PagedFlux listByPacketCoreControlPlaneAsync( * @return response for packet capture API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { return new PagedIterable<>(listByPacketCoreControlPlaneAsync(resourceGroupName, packetCoreControlPlaneName)); } /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1388,22 +1145,23 @@ public PagedIterable listByPacketCoreControlPlane( * @return response for packet capture API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return new PagedIterable<>( listByPacketCoreControlPlaneAsync(resourceGroupName, packetCoreControlPlaneName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet capture API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByPacketCoreControlPlaneNextSinglePageAsync(String nextLink) { @@ -1411,64 +1169,44 @@ private Mono> listByPacketCoreControlPlaneNext return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet capture API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByPacketCoreControlPlaneNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesImpl.java index 7f7ac064b5df6..1d370b43b988d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCapturesImpl.java @@ -23,23 +23,18 @@ public final class PacketCapturesImpl implements PacketCaptures { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public PacketCapturesImpl( - PacketCapturesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public PacketCapturesImpl(PacketCapturesClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCaptureName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PacketCaptureImpl(inner.getValue(), this.manager())); } else { return null; @@ -47,8 +42,8 @@ public Response getWithResponse( } public PacketCapture get(String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { - PacketCaptureInner inner = - this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); + PacketCaptureInner inner + = this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); if (inner != null) { return new PacketCaptureImpl(inner, this.manager()); } else { @@ -60,15 +55,15 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); } - public void delete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, + Context context) { this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); } - public AsyncOperationStatus stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName) { - AsyncOperationStatusInner inner = - this.serviceClient().stop(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); + public AsyncOperationStatus stop(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName) { + AsyncOperationStatusInner inner + = this.serviceClient().stop(resourceGroupName, packetCoreControlPlaneName, packetCaptureName); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -76,10 +71,10 @@ public AsyncOperationStatus stop( } } - public AsyncOperationStatus stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context) { - AsyncOperationStatusInner inner = - this.serviceClient().stop(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); + public AsyncOperationStatus stop(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context) { + AsyncOperationStatusInner inner + = this.serviceClient().stop(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -87,138 +82,93 @@ public AsyncOperationStatus stop( } } - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { - PagedIterable inner = - this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName); - return Utils.mapPage(inner, inner1 -> new PacketCaptureImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { + PagedIterable inner + = this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCaptureImpl(inner1, this.manager())); } - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - PagedIterable inner = - this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName, context); - return Utils.mapPage(inner, inner1 -> new PacketCaptureImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { + PagedIterable inner + = this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCaptureImpl(inner1, this.manager())); } public PacketCapture getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); - } - String packetCaptureName = Utils.getValueFromIdByName(id, "packetCaptures"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); + } + String packetCaptureName = ResourceManagerUtils.getValueFromIdByName(id, "packetCaptures"); if (packetCaptureName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); - } - return this - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, Context.NONE) + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); + } + return this.getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); - } - String packetCaptureName = Utils.getValueFromIdByName(id, "packetCaptures"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); + } + String packetCaptureName = ResourceManagerUtils.getValueFromIdByName(id, "packetCaptures"); if (packetCaptureName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); } return this.getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); - } - String packetCaptureName = Utils.getValueFromIdByName(id, "packetCaptures"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); + } + String packetCaptureName = ResourceManagerUtils.getValueFromIdByName(id, "packetCaptures"); if (packetCaptureName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); } this.delete(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); - } - String packetCaptureName = Utils.getValueFromIdByName(id, "packetCaptures"); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); + } + String packetCaptureName = ResourceManagerUtils.getValueFromIdByName(id, "packetCaptures"); if (packetCaptureName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCaptures'.", id))); } this.delete(resourceGroupName, packetCoreControlPlaneName, packetCaptureName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneImpl.java index f220e960e5091..820325e2bef57 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneImpl.java @@ -13,6 +13,7 @@ import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPrivateKeysProvisioning; import com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject; import com.azure.resourcemanager.mobilenetwork.models.Installation; import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; @@ -145,6 +146,10 @@ public Object interopSettings() { return this.innerModel().interopSettings(); } + public HomeNetworkPrivateKeysProvisioning homeNetworkPrivateKeysProvisioning() { + return this.innerModel().homeNetworkPrivateKeysProvisioning(); + } + public Region region() { return Region.fromName(this.regionName()); } @@ -177,25 +182,19 @@ public PacketCoreControlPlaneImpl withExistingResourceGroup(String resourceGroup } public PacketCoreControlPlane create() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreControlPlanes() - .createOrUpdate(resourceGroupName, packetCoreControlPlaneName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPacketCoreControlPlanes().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, this.innerModel(), Context.NONE); return this; } public PacketCoreControlPlane create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreControlPlanes() - .createOrUpdate(resourceGroupName, packetCoreControlPlaneName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPacketCoreControlPlanes().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, this.innerModel(), context); return this; } - PacketCoreControlPlaneImpl( - String name, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + PacketCoreControlPlaneImpl(String name, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = new PacketCoreControlPlaneInner(); this.serviceManager = serviceManager; this.packetCoreControlPlaneName = name; @@ -207,51 +206,37 @@ public PacketCoreControlPlaneImpl update() { } public PacketCoreControlPlane apply() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreControlPlanes() - .updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCoreControlPlanes() + .updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, updateParameters, Context.NONE) + .getValue(); return this; } public PacketCoreControlPlane apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreControlPlanes() - .updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCoreControlPlanes() + .updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, updateParameters, context) + .getValue(); return this; } - PacketCoreControlPlaneImpl( - PacketCoreControlPlaneInner innerObject, + PacketCoreControlPlaneImpl(PacketCoreControlPlaneInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.packetCoreControlPlaneName = Utils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.packetCoreControlPlaneName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); } public PacketCoreControlPlane refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreControlPlanes() - .getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCoreControlPlanes() + .getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, Context.NONE).getValue(); return this; } public PacketCoreControlPlane refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreControlPlanes() - .getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCoreControlPlanes() + .getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, context).getValue(); return this; } @@ -260,9 +245,8 @@ public AsyncOperationStatus rollback() { } public AsyncOperationStatus rollback(Context context) { - return serviceManager - .packetCoreControlPlanes() - .rollback(resourceGroupName, packetCoreControlPlaneName, context); + return serviceManager.packetCoreControlPlanes().rollback(resourceGroupName, packetCoreControlPlaneName, + context); } public AsyncOperationStatus reinstall() { @@ -270,22 +254,19 @@ public AsyncOperationStatus reinstall() { } public AsyncOperationStatus reinstall(Context context) { - return serviceManager - .packetCoreControlPlanes() - .reinstall(resourceGroupName, packetCoreControlPlaneName, context); + return serviceManager.packetCoreControlPlanes().reinstall(resourceGroupName, packetCoreControlPlaneName, + context); } public AsyncOperationStatus collectDiagnosticsPackage(PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { - return serviceManager - .packetCoreControlPlanes() - .collectDiagnosticsPackage(resourceGroupName, packetCoreControlPlaneName, parameters); + return serviceManager.packetCoreControlPlanes().collectDiagnosticsPackage(resourceGroupName, + packetCoreControlPlaneName, parameters); } - public AsyncOperationStatus collectDiagnosticsPackage( - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { - return serviceManager - .packetCoreControlPlanes() - .collectDiagnosticsPackage(resourceGroupName, packetCoreControlPlaneName, parameters, context); + public AsyncOperationStatus collectDiagnosticsPackage(PacketCoreControlPlaneCollectDiagnosticsPackage parameters, + Context context) { + return serviceManager.packetCoreControlPlanes().collectDiagnosticsPackage(resourceGroupName, + packetCoreControlPlaneName, parameters, context); } public PacketCoreControlPlaneImpl withRegion(Region location) { @@ -318,8 +299,8 @@ public PacketCoreControlPlaneImpl withSku(BillingSku sku) { return this; } - public PacketCoreControlPlaneImpl withLocalDiagnosticsAccess( - LocalDiagnosticsAccessConfiguration localDiagnosticsAccess) { + public PacketCoreControlPlaneImpl + withLocalDiagnosticsAccess(LocalDiagnosticsAccessConfiguration localDiagnosticsAccess) { this.innerModel().withLocalDiagnosticsAccess(localDiagnosticsAccess); return this; } @@ -359,8 +340,8 @@ public PacketCoreControlPlaneImpl withVersion(String version) { return this; } - public PacketCoreControlPlaneImpl withControlPlaneAccessVirtualIpv4Addresses( - List controlPlaneAccessVirtualIpv4Addresses) { + public PacketCoreControlPlaneImpl + withControlPlaneAccessVirtualIpv4Addresses(List controlPlaneAccessVirtualIpv4Addresses) { this.innerModel().withControlPlaneAccessVirtualIpv4Addresses(controlPlaneAccessVirtualIpv4Addresses); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionImpl.java index a84ff5b2c0d58..e39bafdbd38e9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionImpl.java @@ -17,8 +17,7 @@ public final class PacketCoreControlPlaneVersionImpl implements PacketCoreContro private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - PacketCoreControlPlaneVersionImpl( - PacketCoreControlPlaneVersionInner innerObject, + PacketCoreControlPlaneVersionImpl(PacketCoreControlPlaneVersionInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsClientImpl.java index 904b49531d23d..09ffadd8ab757 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsClientImpl.java @@ -30,26 +30,28 @@ import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneVersionListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PacketCoreControlPlaneVersionsClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCoreControlPlaneVersionsClient. + */ public final class PacketCoreControlPlaneVersionsClientImpl implements PacketCoreControlPlaneVersionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PacketCoreControlPlaneVersionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of PacketCoreControlPlaneVersionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PacketCoreControlPlaneVersionsClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy - .create( - PacketCoreControlPlaneVersionsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(PacketCoreControlPlaneVersionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,120 +62,97 @@ public final class PacketCoreControlPlaneVersionsClientImpl implements PacketCor @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface PacketCoreControlPlaneVersionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.MobileNetwork/packetCoreControlPlaneVersions/{versionName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("versionName") String versionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("versionName") String versionName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.MobileNetwork/packetCoreControlPlaneVersions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.MobileNetwork/packetCoreControlPlaneVersions/{versionName}") - @ExpectedResponses({200}) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.MobileNetwork/packetCoreControlPlaneVersions/{versionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getBySubscription( - @HostParam("$host") String endpoint, - @PathParam("versionName") String versionName, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> getBySubscription(@HostParam("$host") String endpoint, + @PathParam("versionName") String versionName, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.MobileNetwork/packetCoreControlPlaneVersions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listBySubscription( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> listBySubscription(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane version along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String versionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (versionName == null) { return Mono.error(new IllegalArgumentException("Parameter versionName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service.get(this.client.getEndpoint(), versionName, this.client.getApiVersion(), accept, context)) + .withContext(context -> service.get(this.client.getEndpoint(), versionName, this.client.getApiVersion(), + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane version along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String versionName, Context context) { + private Mono> getWithResponseAsync(String versionName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (versionName == null) { return Mono.error(new IllegalArgumentException("Parameter versionName is required and cannot be null.")); @@ -185,13 +164,13 @@ private Mono> getWithResponseAsync( /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified packet core control plane version on successful completion of {@link - * Mono}. + * @return information about the specified packet core control plane version on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync(String versionName) { @@ -200,7 +179,7 @@ private Mono getAsync(String versionName) { /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -215,7 +194,7 @@ public Response getWithResponse(String versi /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -229,76 +208,57 @@ public PacketCoreControlPlaneVersionInner get(String versionName) { /** * Lists all supported packet core control planes versions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all supported packet core control planes versions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedFlux}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -307,27 +267,27 @@ private PagedFlux listAsync() { /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedFlux}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists all supported packet core control planes versions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -336,13 +296,13 @@ public PagedIterable list() { /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -351,96 +311,74 @@ public PagedIterable list(Context context) { /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane version along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getBySubscriptionWithResponseAsync(String versionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (versionName == null) { return Mono.error(new IllegalArgumentException("Parameter versionName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getBySubscription( - this.client.getEndpoint(), - versionName, - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) + .withContext(context -> service.getBySubscription(this.client.getEndpoint(), versionName, + this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane version along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getBySubscriptionWithResponseAsync( - String versionName, Context context) { + private Mono> getBySubscriptionWithResponseAsync(String versionName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (versionName == null) { return Mono.error(new IllegalArgumentException("Parameter versionName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getBySubscription( - this.client.getEndpoint(), - versionName, - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context); + return service.getBySubscription(this.client.getEndpoint(), versionName, this.client.getSubscriptionId(), + this.client.getApiVersion(), accept, context); } /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified packet core control plane version on successful completion of {@link - * Mono}. + * @return information about the specified packet core control plane version on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getBySubscriptionAsync(String versionName) { @@ -449,7 +387,7 @@ private Mono getBySubscriptionAsync(String v /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -458,14 +396,14 @@ private Mono getBySubscriptionAsync(String v * @return information about the specified packet core control plane version along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getBySubscriptionWithResponse( - String versionName, Context context) { + public Response getBySubscriptionWithResponse(String versionName, + Context context) { return getBySubscriptionWithResponseAsync(versionName, context).block(); } /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -479,131 +417,97 @@ public PacketCoreControlPlaneVersionInner getBySubscription(String versionName) /** * Lists all supported packet core control planes versions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listBySubscription( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listBySubscription(this.client.getEndpoint(), + this.client.getSubscriptionId(), this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listBySubscription( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listBySubscription(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all supported packet core control planes versions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedFlux}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySubscriptionAsync() { - return new PagedFlux<>( - () -> listBySubscriptionSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listBySubscriptionSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedFlux}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySubscriptionAsync(Context context) { - return new PagedFlux<>( - () -> listBySubscriptionSinglePageAsync(context), + return new PagedFlux<>(() -> listBySubscriptionSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Lists all supported packet core control planes versions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listBySubscription() { @@ -612,13 +516,13 @@ public PagedIterable listBySubscription() { /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listBySubscription(Context context) { @@ -627,14 +531,15 @@ public PagedIterable listBySubscription(Cont /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -642,140 +547,104 @@ private Mono> listNextSinglePa return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control plane version API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsImpl.java index 1a6bedf3e0b99..23425b30ab804 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlaneVersionsImpl.java @@ -21,8 +21,7 @@ public final class PacketCoreControlPlaneVersionsImpl implements PacketCoreContr private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public PacketCoreControlPlaneVersionsImpl( - PacketCoreControlPlaneVersionsClient innerClient, + public PacketCoreControlPlaneVersionsImpl(PacketCoreControlPlaneVersionsClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -31,10 +30,7 @@ public PacketCoreControlPlaneVersionsImpl( public Response getWithResponse(String versionName, Context context) { Response inner = this.serviceClient().getWithResponse(versionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PacketCoreControlPlaneVersionImpl(inner.getValue(), this.manager())); } else { return null; @@ -52,22 +48,21 @@ public PacketCoreControlPlaneVersion get(String versionName) { public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); } public Response getBySubscriptionWithResponse(String versionName, Context context) { - Response inner = - this.serviceClient().getBySubscriptionWithResponse(versionName, context); + Response inner + = this.serviceClient().getBySubscriptionWithResponse(versionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PacketCoreControlPlaneVersionImpl(inner.getValue(), this.manager())); } else { return null; @@ -85,12 +80,14 @@ public PacketCoreControlPlaneVersion getBySubscription(String versionName) { public PagedIterable listBySubscription() { PagedIterable inner = this.serviceClient().listBySubscription(); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); } public PagedIterable listBySubscription(Context context) { PagedIterable inner = this.serviceClient().listBySubscription(context); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new PacketCoreControlPlaneVersionImpl(inner1, this.manager())); } private PacketCoreControlPlaneVersionsClient serviceClient() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesClientImpl.java index 838a28c07ce30..b2f4ab2c60e3d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesClientImpl.java @@ -43,23 +43,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PacketCoreControlPlanesClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCoreControlPlanesClient. + */ public final class PacketCoreControlPlanesClientImpl implements PacketCoreControlPlanesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PacketCoreControlPlanesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of PacketCoreControlPlanesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PacketCoreControlPlanesClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy - .create(PacketCoreControlPlanesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(PacketCoreControlPlanesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -70,155 +75,115 @@ public final class PacketCoreControlPlanesClientImpl implements PacketCoreContro @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface PacketCoreControlPlanesService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") PacketCoreControlPlaneInner parameters, - @HeaderParam("Accept") String accept, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") PacketCoreControlPlaneInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") IdentityAndTagsObject parameters, - @HeaderParam("Accept") String accept, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") IdentityAndTagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/rollback") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/rollback") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> rollback( - @HostParam("$host") String endpoint, + Mono>> rollback(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/reinstall") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/reinstall") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> reinstall( - @HostParam("$host") String endpoint, + Mono>> reinstall(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/collectDiagnosticsPackage") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/collectDiagnosticsPackage") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> collectDiagnosticsPackage( - @HostParam("$host") String endpoint, + Mono>> collectDiagnosticsPackage(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @BodyParam("application/json") PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -227,49 +192,35 @@ Mono> listByResourceGroupNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + context -> service.delete(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -279,46 +230,33 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -327,18 +265,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -348,19 +284,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -375,7 +310,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -385,14 +320,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, context).getSyncPoller(); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -402,14 +337,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName) - .last() + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -420,14 +354,13 @@ private Mono deleteAsync(String resourceGroupName, String packetCoreContro */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, Context context) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, context) - .last() + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -441,7 +374,7 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName) /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -456,59 +389,43 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + return FluxUtil.withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -516,49 +433,36 @@ private Mono> getByResourceGroupWithRespon * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -567,15 +471,15 @@ private Mono> getByResourceGroupWithRespon * @return information about the specified packet core control plane on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getByResourceGroupAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono getByResourceGroupAsync(String resourceGroupName, + String packetCoreControlPlaneName) { return getByResourceGroupWithResponseAsync(resourceGroupName, packetCoreControlPlaneName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -585,14 +489,14 @@ private Mono getByResourceGroupAsync( * @return information about the specified packet core control plane along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context).block(); } /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -607,7 +511,7 @@ public PacketCoreControlPlaneInner getByResourceGroup(String resourceGroupName, /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -617,29 +521,23 @@ public PacketCoreControlPlaneInner getByResourceGroup(String resourceGroupName, * @return packet core control plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -648,24 +546,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -676,32 +565,23 @@ private Mono>> createOrUpdateWithResponseAsync( * @return packet core control plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -710,21 +590,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -736,21 +608,16 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PacketCoreControlPlaneInner> beginCreateOrUpdateAsync( String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PacketCoreControlPlaneInner.class, - PacketCoreControlPlaneInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PacketCoreControlPlaneInner.class, PacketCoreControlPlaneInner.class, + this.client.getContext()); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -762,26 +629,19 @@ private PollerFlux, PacketCoreControlPla */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PacketCoreControlPlaneInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, + String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PacketCoreControlPlaneInner.class, - PacketCoreControlPlaneInner.class, - context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PacketCoreControlPlaneInner.class, PacketCoreControlPlaneInner.class, + context); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -798,7 +658,7 @@ public SyncPoller, PacketCoreControlPlan /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -810,18 +670,15 @@ public SyncPoller, PacketCoreControlPlan */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PacketCoreControlPlaneInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, + String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context) + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context) .getSyncPoller(); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -831,16 +688,15 @@ public SyncPoller, PacketCoreControlPlan * @return packet core control plane resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -851,19 +707,15 @@ private Mono createOrUpdateAsync( * @return packet core control plane resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, - Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -873,14 +725,14 @@ private Mono createOrUpdateAsync( * @return packet core control plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreControlPlaneInner createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneInner parameters) { + public PacketCoreControlPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneInner parameters) { return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters).block(); } /** * Creates or updates a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the create or update packet core control plane operation. @@ -891,17 +743,14 @@ public PacketCoreControlPlaneInner createOrUpdate( * @return packet core control plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreControlPlaneInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneInner parameters, - Context context) { + public PacketCoreControlPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context).block(); } /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -911,29 +760,23 @@ public PacketCoreControlPlaneInner createOrUpdate( * @return packet core control plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, IdentityAndTagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, IdentityAndTagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -943,23 +786,14 @@ private Mono> updateTagsWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.updateTags(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -970,32 +804,23 @@ private Mono> updateTagsWithResponseAsync( * @return packet core control plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - IdentityAndTagsObject parameters, - Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, IdentityAndTagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1004,21 +829,13 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -1028,15 +845,15 @@ private Mono> updateTagsWithResponseAsync( * @return packet core control plane resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String packetCoreControlPlaneName, IdentityAndTagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, + String packetCoreControlPlaneName, IdentityAndTagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -1047,17 +864,14 @@ private Mono updateTagsAsync( * @return packet core control plane resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - IdentityAndTagsObject parameters, - Context context) { + public Response updateTagsWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, IdentityAndTagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context).block(); } /** * Patch packet core control plane resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to patch packet core control plane resource. @@ -1067,117 +881,84 @@ public Response updateTagsWithResponse( * @return packet core control plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreControlPlaneInner updateTags( - String resourceGroupName, String packetCoreControlPlaneName, IdentityAndTagsObject parameters) { + public PacketCoreControlPlaneInner updateTags(String resourceGroupName, String packetCoreControlPlaneName, + IdentityAndTagsObject parameters) { return updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, parameters, Context.NONE) .getValue(); } /** * Lists all the packet core control planes in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the packet core control planes in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the packet core control planes in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Lists all the packet core control planes in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1186,17 +967,17 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Lists all the packet core control planes in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -1205,13 +986,13 @@ public PagedIterable list() { /** * Lists all the packet core control planes in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -1220,111 +1001,76 @@ public PagedIterable list(Context context) { /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1333,14 +1079,13 @@ private Mono> listByResourceGroupSing */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1350,20 +1095,19 @@ private PagedFlux listByResourceGroupAsync(String r */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByResourceGroup(String resourceGroupName) { @@ -1372,14 +1116,14 @@ public PagedIterable listByResourceGroup(String res /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { @@ -1389,111 +1133,84 @@ public PagedIterable listByResourceGroup(String res /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> rollbackWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono>> rollbackWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .rollback( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + context -> service.rollback(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> rollbackWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private Mono>> rollbackWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .rollback( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.rollback(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1502,24 +1219,19 @@ private Mono>> rollbackWithResponseAsync( * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginRollbackAsync( - String resourceGroupName, String packetCoreControlPlaneName) { - Mono>> mono = - rollbackWithResponseAsync(resourceGroupName, packetCoreControlPlaneName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + private PollerFlux, AsyncOperationStatusInner> + beginRollbackAsync(String resourceGroupName, String packetCoreControlPlaneName) { + Mono>> mono + = rollbackWithResponseAsync(resourceGroupName, packetCoreControlPlaneName); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1529,25 +1241,19 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginRollbackAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private PollerFlux, AsyncOperationStatusInner> + beginRollbackAsync(String resourceGroupName, String packetCoreControlPlaneName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - rollbackWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono + = rollbackWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1556,15 +1262,15 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginRollback( - String resourceGroupName, String packetCoreControlPlaneName) { + public SyncPoller, AsyncOperationStatusInner> + beginRollback(String resourceGroupName, String packetCoreControlPlaneName) { return this.beginRollbackAsync(resourceGroupName, packetCoreControlPlaneName).getSyncPoller(); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1574,15 +1280,15 @@ public SyncPoller, AsyncOperationStatusInn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginRollback( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public SyncPoller, AsyncOperationStatusInner> + beginRollback(String resourceGroupName, String packetCoreControlPlaneName, Context context) { return this.beginRollbackAsync(resourceGroupName, packetCoreControlPlaneName, context).getSyncPoller(); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1592,15 +1298,14 @@ public SyncPoller, AsyncOperationStatusInn */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono rollbackAsync(String resourceGroupName, String packetCoreControlPlaneName) { - return beginRollbackAsync(resourceGroupName, packetCoreControlPlaneName) - .last() + return beginRollbackAsync(resourceGroupName, packetCoreControlPlaneName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1610,17 +1315,16 @@ private Mono rollbackAsync(String resourceGroupName, * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono rollbackAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - return beginRollbackAsync(resourceGroupName, packetCoreControlPlaneName, context) - .last() + private Mono rollbackAsync(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { + return beginRollbackAsync(resourceGroupName, packetCoreControlPlaneName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1636,7 +1340,7 @@ public AsyncOperationStatusInner rollback(String resourceGroupName, String packe /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1646,119 +1350,92 @@ public AsyncOperationStatusInner rollback(String resourceGroupName, String packe * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner rollback( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public AsyncOperationStatusInner rollback(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { return rollbackAsync(resourceGroupName, packetCoreControlPlaneName, context).block(); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> reinstallWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono>> reinstallWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .reinstall( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + context -> service.reinstall(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> reinstallWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private Mono>> reinstallWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .reinstall( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.reinstall(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1767,24 +1444,19 @@ private Mono>> reinstallWithResponseAsync( * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginReinstallAsync( - String resourceGroupName, String packetCoreControlPlaneName) { - Mono>> mono = - reinstallWithResponseAsync(resourceGroupName, packetCoreControlPlaneName); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + private PollerFlux, AsyncOperationStatusInner> + beginReinstallAsync(String resourceGroupName, String packetCoreControlPlaneName) { + Mono>> mono + = reinstallWithResponseAsync(resourceGroupName, packetCoreControlPlaneName); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1794,25 +1466,19 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginReinstallAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private PollerFlux, AsyncOperationStatusInner> + beginReinstallAsync(String resourceGroupName, String packetCoreControlPlaneName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - reinstallWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono + = reinstallWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1821,15 +1487,15 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginReinstall( - String resourceGroupName, String packetCoreControlPlaneName) { + public SyncPoller, AsyncOperationStatusInner> + beginReinstall(String resourceGroupName, String packetCoreControlPlaneName) { return this.beginReinstallAsync(resourceGroupName, packetCoreControlPlaneName).getSyncPoller(); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1839,15 +1505,15 @@ public SyncPoller, AsyncOperationStatusInn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginReinstall( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public SyncPoller, AsyncOperationStatusInner> + beginReinstall(String resourceGroupName, String packetCoreControlPlaneName, Context context) { return this.beginReinstallAsync(resourceGroupName, packetCoreControlPlaneName, context).getSyncPoller(); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1856,17 +1522,16 @@ public SyncPoller, AsyncOperationStatusInn * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono reinstallAsync( - String resourceGroupName, String packetCoreControlPlaneName) { - return beginReinstallAsync(resourceGroupName, packetCoreControlPlaneName) - .last() + private Mono reinstallAsync(String resourceGroupName, + String packetCoreControlPlaneName) { + return beginReinstallAsync(resourceGroupName, packetCoreControlPlaneName).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1876,17 +1541,16 @@ private Mono reinstallAsync( * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono reinstallAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - return beginReinstallAsync(resourceGroupName, packetCoreControlPlaneName, context) - .last() + private Mono reinstallAsync(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { + return beginReinstallAsync(resourceGroupName, packetCoreControlPlaneName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1900,9 +1564,9 @@ public AsyncOperationStatusInner reinstall(String resourceGroupName, String pack } /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1912,50 +1576,42 @@ public AsyncOperationStatusInner reinstall(String resourceGroupName, String pack * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner reinstall( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public AsyncOperationStatusInner reinstall(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { return reinstallAsync(resourceGroupName, packetCoreControlPlaneName, context).block(); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> collectDiagnosticsPackageWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { + private Mono>> collectDiagnosticsPackageWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1964,25 +1620,16 @@ private Mono>> collectDiagnosticsPackageWithResponseAs } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .collectDiagnosticsPackage( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.collectDiagnosticsPackage(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -1990,36 +1637,28 @@ private Mono>> collectDiagnosticsPackageWithResponseAs * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> collectDiagnosticsPackageWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, + private Mono>> collectDiagnosticsPackageWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2028,22 +1667,15 @@ private Mono>> collectDiagnosticsPackageWithResponseAs } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .collectDiagnosticsPackage( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.collectDiagnosticsPackage(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, + accept, context); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2054,26 +1686,19 @@ private Mono>> collectDiagnosticsPackageWithResponseAs */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, AsyncOperationStatusInner> - beginCollectDiagnosticsPackageAsync( - String resourceGroupName, - String packetCoreControlPlaneName, + beginCollectDiagnosticsPackageAsync(String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { - Mono>> mono = - collectDiagnosticsPackageWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + Mono>> mono + = collectDiagnosticsPackageWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2085,29 +1710,19 @@ private Mono>> collectDiagnosticsPackageWithResponseAs */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, AsyncOperationStatusInner> - beginCollectDiagnosticsPackageAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - Context context) { + beginCollectDiagnosticsPackageAsync(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - collectDiagnosticsPackageWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono = collectDiagnosticsPackageWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2118,18 +1733,16 @@ private Mono>> collectDiagnosticsPackageWithResponseAs */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, AsyncOperationStatusInner> beginCollectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, + String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { - return this - .beginCollectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters) + return this.beginCollectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters) .getSyncPoller(); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2141,10 +1754,8 @@ public SyncPoller, AsyncOperationStatusInn */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, AsyncOperationStatusInner> beginCollectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - Context context) { + String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { return this .beginCollectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context) .getSyncPoller(); @@ -2153,7 +1764,7 @@ public SyncPoller, AsyncOperationStatusInn /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2163,19 +1774,16 @@ public SyncPoller, AsyncOperationStatusInn * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono collectDiagnosticsPackageAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { - return beginCollectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters) - .last() + private Mono collectDiagnosticsPackageAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { + return beginCollectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2186,20 +1794,17 @@ private Mono collectDiagnosticsPackageAsync( * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono collectDiagnosticsPackageAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, + private Mono collectDiagnosticsPackageAsync(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { return beginCollectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + .last().flatMap(this.client::getLroFinalResultOrError); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2209,17 +1814,15 @@ private Mono collectDiagnosticsPackageAsync( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { + public AsyncOperationStatusInner collectDiagnosticsPackage(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { return collectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters).block(); } /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -2230,10 +1833,8 @@ public AsyncOperationStatusInner collectDiagnosticsPackage( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, + public AsyncOperationStatusInner collectDiagnosticsPackage(String resourceGroupName, + String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { return collectDiagnosticsPackageAsync(resourceGroupName, packetCoreControlPlaneName, parameters, context) .block(); @@ -2241,14 +1842,15 @@ public AsyncOperationStatusInner collectDiagnosticsPackage( /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -2256,76 +1858,59 @@ private Mono> listBySubscriptionNextS return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -2333,63 +1918,45 @@ private Mono> listByResourceGroupNext return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core control planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesImpl.java index 435a95b407a70..664524f23d1af 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreControlPlanesImpl.java @@ -24,8 +24,7 @@ public final class PacketCoreControlPlanesImpl implements PacketCoreControlPlane private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public PacketCoreControlPlanesImpl( - PacketCoreControlPlanesClient innerClient, + public PacketCoreControlPlanesImpl(PacketCoreControlPlanesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -39,15 +38,12 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, context); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { + Response inner = this.serviceClient() + .getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PacketCoreControlPlaneImpl(inner.getValue(), this.manager())); } else { return null; @@ -55,8 +51,8 @@ public Response getByResourceGroupWithResponse( } public PacketCoreControlPlane getByResourceGroup(String resourceGroupName, String packetCoreControlPlaneName) { - PacketCoreControlPlaneInner inner = - this.serviceClient().getByResourceGroup(resourceGroupName, packetCoreControlPlaneName); + PacketCoreControlPlaneInner inner + = this.serviceClient().getByResourceGroup(resourceGroupName, packetCoreControlPlaneName); if (inner != null) { return new PacketCoreControlPlaneImpl(inner, this.manager()); } else { @@ -66,23 +62,23 @@ public PacketCoreControlPlane getByResourceGroup(String resourceGroupName, Strin public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCoreControlPlaneImpl(inner1, this.manager())); } public AsyncOperationStatus rollback(String resourceGroupName, String packetCoreControlPlaneName) { @@ -95,8 +91,8 @@ public AsyncOperationStatus rollback(String resourceGroupName, String packetCore } public AsyncOperationStatus rollback(String resourceGroupName, String packetCoreControlPlaneName, Context context) { - AsyncOperationStatusInner inner = - this.serviceClient().rollback(resourceGroupName, packetCoreControlPlaneName, context); + AsyncOperationStatusInner inner + = this.serviceClient().rollback(resourceGroupName, packetCoreControlPlaneName, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -113,10 +109,10 @@ public AsyncOperationStatus reinstall(String resourceGroupName, String packetCor } } - public AsyncOperationStatus reinstall( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - AsyncOperationStatusInner inner = - this.serviceClient().reinstall(resourceGroupName, packetCoreControlPlaneName, context); + public AsyncOperationStatus reinstall(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { + AsyncOperationStatusInner inner + = this.serviceClient().reinstall(resourceGroupName, packetCoreControlPlaneName, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -124,12 +120,10 @@ public AsyncOperationStatus reinstall( } } - public AsyncOperationStatus collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, + public AsyncOperationStatus collectDiagnosticsPackage(String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters) { - AsyncOperationStatusInner inner = - this.serviceClient().collectDiagnosticsPackage(resourceGroupName, packetCoreControlPlaneName, parameters); + AsyncOperationStatusInner inner + = this.serviceClient().collectDiagnosticsPackage(resourceGroupName, packetCoreControlPlaneName, parameters); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -137,15 +131,10 @@ public AsyncOperationStatus collectDiagnosticsPackage( } } - public AsyncOperationStatus collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - Context context) { - AsyncOperationStatusInner inner = - this - .serviceClient() - .collectDiagnosticsPackage(resourceGroupName, packetCoreControlPlaneName, parameters, context); + public AsyncOperationStatus collectDiagnosticsPackage(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context) { + AsyncOperationStatusInner inner = this.serviceClient().collectDiagnosticsPackage(resourceGroupName, + packetCoreControlPlaneName, parameters, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -154,91 +143,58 @@ public AsyncOperationStatus collectDiagnosticsPackage( } public PacketCoreControlPlane getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - return this - .getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, Context.NONE) + return this.getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, packetCoreControlPlaneName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } this.delete(resourceGroupName, packetCoreControlPlaneName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } this.delete(resourceGroupName, packetCoreControlPlaneName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlaneImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlaneImpl.java index 2a35152c7a5dd..76d22f86571da 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlaneImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlaneImpl.java @@ -96,34 +96,22 @@ private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { private TagsObject updateParameters; - public PacketCoreDataPlaneImpl withExistingPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { + public PacketCoreDataPlaneImpl withExistingPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { this.resourceGroupName = resourceGroupName; this.packetCoreControlPlaneName = packetCoreControlPlaneName; return this; } public PacketCoreDataPlane create() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreDataPlanes() - .createOrUpdate( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.innerModel(), - Context.NONE); + this.innerObject = serviceManager.serviceClient().getPacketCoreDataPlanes().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.innerModel(), Context.NONE); return this; } public PacketCoreDataPlane create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreDataPlanes() - .createOrUpdate( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPacketCoreDataPlanes().createOrUpdate(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.innerModel(), context); return this; } @@ -139,58 +127,41 @@ public PacketCoreDataPlaneImpl update() { } public PacketCoreDataPlane apply() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreDataPlanes() - .updateTagsWithResponse( - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - updateParameters, - Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getPacketCoreDataPlanes().updateTagsWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, updateParameters, Context.NONE).getValue(); return this; } public PacketCoreDataPlane apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreDataPlanes() - .updateTagsWithResponse( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, updateParameters, context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getPacketCoreDataPlanes().updateTagsWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, updateParameters, context).getValue(); return this; } - PacketCoreDataPlaneImpl( - PacketCoreDataPlaneInner innerObject, + PacketCoreDataPlaneImpl(PacketCoreDataPlaneInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.packetCoreControlPlaneName = Utils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); - this.packetCoreDataPlaneName = Utils.getValueFromIdByName(innerObject.id(), "packetCoreDataPlanes"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.packetCoreControlPlaneName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCoreControlPlanes"); + this.packetCoreDataPlaneName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "packetCoreDataPlanes"); } public PacketCoreDataPlane refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreDataPlanes() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCoreDataPlanes() + .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, Context.NONE) + .getValue(); return this; } public PacketCoreDataPlane refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPacketCoreDataPlanes() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPacketCoreDataPlanes() + .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context) + .getValue(); return this; } @@ -219,8 +190,8 @@ public PacketCoreDataPlaneImpl withTags(Map tags) { } } - public PacketCoreDataPlaneImpl withUserPlaneAccessVirtualIpv4Addresses( - List userPlaneAccessVirtualIpv4Addresses) { + public PacketCoreDataPlaneImpl + withUserPlaneAccessVirtualIpv4Addresses(List userPlaneAccessVirtualIpv4Addresses) { this.innerModel().withUserPlaneAccessVirtualIpv4Addresses(userPlaneAccessVirtualIpv4Addresses); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesClientImpl.java index e8e25d72bba6b..c7f4bb6c0ac4d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesClientImpl.java @@ -40,23 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PacketCoreDataPlanesClient. */ +/** + * An instance of this class provides access to all the operations defined in PacketCoreDataPlanesClient. + */ public final class PacketCoreDataPlanesClientImpl implements PacketCoreDataPlanesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PacketCoreDataPlanesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of PacketCoreDataPlanesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PacketCoreDataPlanesClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy - .create(PacketCoreDataPlanesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(PacketCoreDataPlanesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -67,96 +72,74 @@ public final class PacketCoreDataPlanesClientImpl implements PacketCoreDataPlane @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface PacketCoreDataPlanesService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") PacketCoreDataPlaneInner parameters, - @HeaderParam("Accept") String accept, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") PacketCoreDataPlaneInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes/{packetCoreDataPlaneName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, @PathParam("packetCoreDataPlaneName") String packetCoreDataPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") TagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/packetCoreDataPlanes") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByPacketCoreControlPlane( - @HostParam("$host") String endpoint, + Mono> listByPacketCoreControlPlane(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByPacketCoreControlPlaneNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -166,55 +149,39 @@ Mono> listByPacketCoreControlPlaneNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -225,52 +192,37 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -280,19 +232,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -303,19 +253,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -325,16 +274,15 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { - return this - .beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName) + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName) .getSyncPoller(); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -345,16 +293,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { - return this - .beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, Context context) { + return this.beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context) .getSyncPoller(); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -364,16 +311,15 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName) - .last() + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -384,16 +330,15 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { - return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, Context context) { + return beginDeleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -408,7 +353,7 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -418,14 +363,14 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + Context context) { deleteAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context).block(); } /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -433,58 +378,42 @@ public void delete( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core data plane along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -493,55 +422,40 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core data plane along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -551,15 +465,15 @@ private Mono> getWithResponseAsync( * @return information about the specified packet core data plane on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + private Mono getAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName) { return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -570,15 +484,15 @@ private Mono getAsync( * @return information about the specified packet core data plane along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + public Response getWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { return getWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context) .block(); } /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -588,8 +502,8 @@ public Response getWithResponse( * @return information about the specified packet core data plane. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreDataPlaneInner get( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { + public PacketCoreDataPlaneInner get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName) { return getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, Context.NONE) .getValue(); } @@ -597,7 +511,7 @@ public PacketCoreDataPlaneInner get( /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -608,37 +522,27 @@ public PacketCoreDataPlaneInner get( * @return packet core data plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -647,26 +551,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.client.getApiVersion(), + this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -678,38 +572,28 @@ private Mono>> createOrUpdateWithResponseAsync( * @return packet core data plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -718,23 +602,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, + context); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -746,27 +622,19 @@ private Mono>> createOrUpdateWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PacketCoreDataPlaneInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PacketCoreDataPlaneInner.class, - PacketCoreDataPlaneInner.class, - this.client.getContext()); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PacketCoreDataPlaneInner.class, PacketCoreDataPlaneInner.class, + this.client.getContext()); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -779,29 +647,19 @@ private PollerFlux, PacketCoreDataPlaneInne */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PacketCoreDataPlaneInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, - Context context) { + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + PacketCoreDataPlaneInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PacketCoreDataPlaneInner.class, - PacketCoreDataPlaneInner.class, - context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PacketCoreDataPlaneInner.class, PacketCoreDataPlaneInner.class, context); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -813,20 +671,16 @@ private PollerFlux, PacketCoreDataPlaneInne */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PacketCoreDataPlaneInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters) - .getSyncPoller(); + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters).getSyncPoller(); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -839,21 +693,16 @@ public SyncPoller, PacketCoreDataPlaneInner */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PacketCoreDataPlaneInner> beginCreateOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, context) - .getSyncPoller(); + String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + PacketCoreDataPlaneInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters, context).getSyncPoller(); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -864,21 +713,16 @@ public SyncPoller, PacketCoreDataPlaneInner * @return packet core data plane resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters) { - return beginCreateOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters).last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -890,22 +734,17 @@ private Mono createOrUpdateAsync( * @return packet core data plane resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, + private Mono createOrUpdateAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters, Context context) { - return beginCreateOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + return beginCreateOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -916,11 +755,8 @@ private Mono createOrUpdateAsync( * @return packet core data plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreDataPlaneInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters) { + public PacketCoreDataPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters) { return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters) .block(); } @@ -928,7 +764,7 @@ public PacketCoreDataPlaneInner createOrUpdate( /** * Creates or updates a packet core data plane. Must be created in the same location as its parent packet core * control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -940,20 +776,15 @@ public PacketCoreDataPlaneInner createOrUpdate( * @return packet core data plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreDataPlaneInner createOrUpdate( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - PacketCoreDataPlaneInner parameters, - Context context) { - return createOrUpdateAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, context) - .block(); + public PacketCoreDataPlaneInner createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, PacketCoreDataPlaneInner parameters, Context context) { + return createOrUpdateAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, + context).block(); } /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -964,37 +795,27 @@ public PacketCoreDataPlaneInner createOrUpdate( * @return packet core data plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1003,25 +824,15 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, this.client.getApiVersion(), + this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1033,38 +844,27 @@ private Mono> updateTagsWithResponseAsync( * @return packet core data plane resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters, - Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (packetCoreDataPlaneName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreDataPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1073,22 +873,14 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - packetCoreDataPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + packetCoreDataPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, + context); } /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1099,19 +891,15 @@ private Mono> updateTagsWithResponseAsync( * @return packet core data plane resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters) { - return updateTagsWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono updateTagsAsync(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, TagsObject parameters) { + return updateTagsWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1123,20 +911,15 @@ private Mono updateTagsAsync( * @return packet core data plane resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters, - Context context) { - return updateTagsWithResponseAsync( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, context) - .block(); + public Response updateTagsWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, TagsObject parameters, Context context) { + return updateTagsWithResponseAsync(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters, context).block(); } /** * Updates packet core data planes tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -1147,80 +930,55 @@ public Response updateTagsWithResponse( * @return packet core data plane resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PacketCoreDataPlaneInner updateTags( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - TagsObject parameters) { - return updateTagsWithResponse( - resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, parameters, Context.NONE) - .getValue(); + public PacketCoreDataPlaneInner updateTags(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, TagsObject parameters) { + return updateTagsWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, + parameters, Context.NONE).getValue(); } /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core data planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneSinglePageAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private Mono> + listByPacketCoreControlPlaneSinglePageAsync(String resourceGroupName, String packetCoreControlPlaneName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByPacketCoreControlPlane( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByPacketCoreControlPlane(this.client.getEndpoint(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, + context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1228,58 +986,39 @@ private Mono> listByPacketCoreControlPla * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core data planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByPacketCoreControlPlaneSinglePageAsync( String resourceGroupName, String packetCoreControlPlaneName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (packetCoreControlPlaneName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter packetCoreControlPlaneName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByPacketCoreControlPlane( - this.client.getEndpoint(), - resourceGroupName, - packetCoreControlPlaneName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByPacketCoreControlPlane(this.client.getEndpoint(), resourceGroupName, packetCoreControlPlaneName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1288,8 +1027,8 @@ private Mono> listByPacketCoreControlPla * @return response for packet core data planes API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreControlPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName) { + private PagedFlux listByPacketCoreControlPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName) { return new PagedFlux<>( () -> listByPacketCoreControlPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName), nextLink -> listByPacketCoreControlPlaneNextSinglePageAsync(nextLink)); @@ -1297,7 +1036,7 @@ private PagedFlux listByPacketCoreControlPlaneAsync( /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1307,8 +1046,8 @@ private PagedFlux listByPacketCoreControlPlaneAsync( * @return response for packet core data planes API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByPacketCoreControlPlaneAsync( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + private PagedFlux listByPacketCoreControlPlaneAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return new PagedFlux<>( () -> listByPacketCoreControlPlaneSinglePageAsync(resourceGroupName, packetCoreControlPlaneName, context), nextLink -> listByPacketCoreControlPlaneNextSinglePageAsync(nextLink, context)); @@ -1316,7 +1055,7 @@ private PagedFlux listByPacketCoreControlPlaneAsync( /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1325,14 +1064,14 @@ private PagedFlux listByPacketCoreControlPlaneAsync( * @return response for packet core data planes API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { return new PagedIterable<>(listByPacketCoreControlPlaneAsync(resourceGroupName, packetCoreControlPlaneName)); } /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -1342,88 +1081,69 @@ public PagedIterable listByPacketCoreControlPlane( * @return response for packet core data planes API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { return new PagedIterable<>( listByPacketCoreControlPlaneAsync(resourceGroupName, packetCoreControlPlaneName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core data planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneNextSinglePageAsync( - String nextLink) { + private Mono> + listByPacketCoreControlPlaneNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core data planes API service call along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByPacketCoreControlPlaneNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listByPacketCoreControlPlaneNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByPacketCoreControlPlaneNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesImpl.java index 6e5aba515def1..670e8f0accaf9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/PacketCoreDataPlanesImpl.java @@ -21,8 +21,7 @@ public final class PacketCoreDataPlanesImpl implements PacketCoreDataPlanes { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public PacketCoreDataPlanesImpl( - PacketCoreDataPlanesClient innerClient, + public PacketCoreDataPlanesImpl(PacketCoreDataPlanesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -32,32 +31,27 @@ public void delete(String resourceGroupName, String packetCoreControlPlaneName, this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); } - public void delete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { + public void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + Context context) { this.serviceClient().delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); } - public Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); + public Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + packetCoreControlPlaneName, packetCoreDataPlaneName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PacketCoreDataPlaneImpl(inner.getValue(), this.manager())); } else { return null; } } - public PacketCoreDataPlane get( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName) { - PacketCoreDataPlaneInner inner = - this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); + public PacketCoreDataPlane get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName) { + PacketCoreDataPlaneInner inner + = this.serviceClient().get(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName); if (inner != null) { return new PacketCoreDataPlaneImpl(inner, this.manager()); } else { @@ -65,48 +59,35 @@ public PacketCoreDataPlane get( } } - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName) { - PagedIterable inner = - this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName); - return Utils.mapPage(inner, inner1 -> new PacketCoreDataPlaneImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName) { + PagedIterable inner + = this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCoreDataPlaneImpl(inner1, this.manager())); } - public PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context) { - PagedIterable inner = - this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName, context); - return Utils.mapPage(inner, inner1 -> new PacketCoreDataPlaneImpl(inner1, this.manager())); + public PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { + PagedIterable inner + = this.serviceClient().listByPacketCoreControlPlane(resourceGroupName, packetCoreControlPlaneName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PacketCoreDataPlaneImpl(inner1, this.manager())); } public PacketCoreDataPlane getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } return this .getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, Context.NONE) @@ -114,97 +95,58 @@ public PacketCoreDataPlane getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } return this.getWithResponse(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } this.delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String packetCoreControlPlaneName = Utils.getValueFromIdByName(id, "packetCoreControlPlanes"); + String packetCoreControlPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreControlPlanes"); if (packetCoreControlPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'packetCoreControlPlanes'.", id))); } - String packetCoreDataPlaneName = Utils.getValueFromIdByName(id, "packetCoreDataPlanes"); + String packetCoreDataPlaneName = ResourceManagerUtils.getValueFromIdByName(id, "packetCoreDataPlanes"); if (packetCoreDataPlaneName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'packetCoreDataPlanes'.", id))); } this.delete(resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/Utils.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ResourceManagerUtils.java similarity index 90% rename from sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/Utils.java rename to sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ResourceManagerUtils.java index c7c58d648d4af..e224f97ad364e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/Utils.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ResourceManagerUtils.java @@ -19,8 +19,8 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { - private Utils() { +final class ResourceManagerUtils { + private ResourceManagerUtils() { } static String getValueFromIdByName(String id, String name) { @@ -41,6 +41,7 @@ static String getValueFromIdByName(String id, String name) { } } return null; + } static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { @@ -74,6 +75,7 @@ static String getValueFromIdByParameterName(String id, String pathTemplate, Stri } } return null; + } static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { @@ -87,26 +89,17 @@ private static final class PagedIterableImpl extends PagedIterable { private final Function, PagedResponse> pageMapper; private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); this.pagedIterable = pagedIterable; this.mapper = mapper; this.pageMapper = getPageMapper(mapper); } private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); } @Override diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServiceImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServiceImpl.java index 064b36226d443..49a0dbcec5dfb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServiceImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServiceImpl.java @@ -107,20 +107,14 @@ public ServiceImpl withExistingMobileNetwork(String resourceGroupName, String mo } public Service create() { - this.innerObject = - serviceManager - .serviceClient() - .getServices() - .createOrUpdate(resourceGroupName, mobileNetworkName, serviceName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getServices().createOrUpdate(resourceGroupName, + mobileNetworkName, serviceName, this.innerModel(), Context.NONE); return this; } public Service create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getServices() - .createOrUpdate(resourceGroupName, mobileNetworkName, serviceName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getServices().createOrUpdate(resourceGroupName, + mobileNetworkName, serviceName, this.innerModel(), context); return this; } @@ -136,51 +130,36 @@ public ServiceImpl update() { } public Service apply() { - this.innerObject = - serviceManager - .serviceClient() - .getServices() - .updateTagsWithResponse( - resourceGroupName, mobileNetworkName, serviceName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getServices() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, serviceName, updateParameters, Context.NONE) + .getValue(); return this; } public Service apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getServices() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, serviceName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getServices() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, serviceName, updateParameters, context) + .getValue(); return this; } ServiceImpl(ServiceInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.mobileNetworkName = Utils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); - this.serviceName = Utils.getValueFromIdByName(innerObject.id(), "services"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); + this.serviceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "services"); } public Service refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getServices() - .getWithResponse(resourceGroupName, mobileNetworkName, serviceName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getServices() + .getWithResponse(resourceGroupName, mobileNetworkName, serviceName, Context.NONE).getValue(); return this; } public Service refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getServices() - .getWithResponse(resourceGroupName, mobileNetworkName, serviceName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getServices() + .getWithResponse(resourceGroupName, mobileNetworkName, serviceName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesClientImpl.java index 121bb8b46b872..819d498f06e18 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesClientImpl.java @@ -40,17 +40,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ServicesClient. */ +/** + * An instance of this class provides access to all the operations defined in ServicesClient. + */ public final class ServicesClientImpl implements ServicesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ServicesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of ServicesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ServicesClientImpl(MobileNetworkManagementClientImpl client) { @@ -59,119 +65,90 @@ public final class ServicesClientImpl implements ServicesClient { } /** - * The interface defining all the services for MobileNetworkManagementClientServices to be used by the proxy service - * to perform REST calls. + * The interface defining all the services for MobileNetworkManagementClientServices to be used by the proxy + * service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface ServicesService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("serviceName") String serviceName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("serviceName") String serviceName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("serviceName") String serviceName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("serviceName") String serviceName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("serviceName") String serviceName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") ServiceInner parameters, - @HeaderParam("Accept") String accept, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("serviceName") String serviceName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") ServiceInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services/{serviceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("serviceName") String serviceName, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("serviceName") String serviceName, + @BodyParam("application/json") TagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/services") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByMobileNetwork( - @HostParam("$host") String endpoint, + Mono> listByMobileNetwork(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("mobileNetworkName") String mobileNetworkName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByMobileNetworkNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String serviceName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String serviceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -185,35 +162,23 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - serviceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, + serviceName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -221,13 +186,11 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -241,55 +204,43 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - serviceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, serviceName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String serviceName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String serviceName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -297,41 +248,40 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String serviceName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String serviceName) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, serviceName).getSyncPoller(); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -339,18 +289,18 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String serviceName, Context context) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, serviceName, context).getSyncPoller(); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -358,18 +308,17 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String serviceName) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, serviceName) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName, serviceName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -377,20 +326,19 @@ private Mono deleteAsync(String resourceGroupName, String mobileNetworkNam * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, serviceName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String serviceName, + Context context) { + return beginDeleteAsync(resourceGroupName, mobileNetworkName, serviceName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -402,11 +350,11 @@ public void delete(String resourceGroupName, String mobileNetworkName, String se /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -419,25 +367,23 @@ public void delete(String resourceGroupName, String mobileNetworkName, String se /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified service along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about the specified service along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String serviceName) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String serviceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -451,50 +397,36 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - serviceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, + serviceName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified service along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about the specified service along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -508,32 +440,22 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - serviceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, serviceName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -547,11 +469,11 @@ private Mono getAsync(String resourceGroupName, String mobileNetwo /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -559,18 +481,18 @@ private Mono getAsync(String resourceGroupName, String mobileNetwo * @return information about the specified service along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, + String serviceName, Context context) { return getWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, context).block(); } /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -583,11 +505,11 @@ public ServiceInner get(String resourceGroupName, String mobileNetworkName, Stri /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -595,13 +517,11 @@ public ServiceInner get(String resourceGroupName, String mobileNetworkName, Stri * @return service resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -615,10 +535,8 @@ private Mono>> createOrUpdateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -627,29 +545,19 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - serviceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, + mobileNetworkName, serviceName, this.client.getApiVersion(), this.client.getSubscriptionId(), + parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -658,17 +566,11 @@ private Mono>> createOrUpdateWithResponseAsync( * @return service resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -682,10 +584,8 @@ private Mono>> createOrUpdateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -694,26 +594,17 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - serviceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, serviceName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -721,23 +612,21 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of service resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ServiceInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ServiceInner.class, ServiceInner.class, this.client.getContext()); + private PollerFlux, ServiceInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ServiceInner.class, ServiceInner.class, this.client.getContext()); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -746,28 +635,22 @@ private PollerFlux, ServiceInner> beginCreateOrUpdateAs * @return the {@link PollerFlux} for polling of service resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ServiceInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context) { + private PollerFlux, ServiceInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ServiceInner.class, ServiceInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ServiceInner.class, ServiceInner.class, context); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -775,20 +658,19 @@ private PollerFlux, ServiceInner> beginCreateOrUpdateAs * @return the {@link SyncPoller} for polling of service resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ServiceInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters) + public SyncPoller, ServiceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters) .getSyncPoller(); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -797,24 +679,19 @@ public SyncPoller, ServiceInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of service resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ServiceInner> beginCreateOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context) + public SyncPoller, ServiceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String serviceName, ServiceInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context) .getSyncPoller(); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -822,20 +699,19 @@ public SyncPoller, ServiceInner> beginCreateOrUpdate( * @return service resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, ServiceInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -844,24 +720,19 @@ private Mono createOrUpdateAsync( * @return service resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, ServiceInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -869,18 +740,18 @@ private Mono createOrUpdateAsync( * @return service resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServiceInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String serviceName, ServiceInner parameters) { + public ServiceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String serviceName, + ServiceInner parameters) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters).block(); } /** * Creates or updates a service. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to the create or update service operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -889,22 +760,18 @@ public ServiceInner createOrUpdate( * @return service resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServiceInner createOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - ServiceInner parameters, - Context context) { + public ServiceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String serviceName, + ServiceInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context).block(); } /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -912,19 +779,15 @@ public ServiceInner createOrUpdate( * @return service resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -944,29 +807,19 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - serviceName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, serviceName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -975,23 +828,15 @@ private Mono> updateTagsWithResponseAsync( * @return service resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - TagsObject parameters, - Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String serviceName, TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1011,26 +856,17 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - serviceName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, serviceName, parameters, accept, context); } /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1038,19 +874,19 @@ private Mono> updateTagsWithResponseAsync( * @return service resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String mobileNetworkName, String serviceName, TagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String mobileNetworkName, String serviceName, + TagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1059,23 +895,19 @@ private Mono updateTagsAsync( * @return service resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, - String mobileNetworkName, - String serviceName, - TagsObject parameters, - Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String serviceName, TagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, serviceName, parameters, context) .block(); } /** * Updates service tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param parameters Parameters supplied to update service tags. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1083,31 +915,29 @@ public Response updateTagsWithResponse( * @return service resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServiceInner updateTags( - String resourceGroupName, String mobileNetworkName, String serviceName, TagsObject parameters) { + public ServiceInner updateTags(String resourceGroupName, String mobileNetworkName, String serviceName, + TagsObject parameters) { return updateTagsWithResponse(resourceGroupName, mobileNetworkName, serviceName, parameters, Context.NONE) .getValue(); } /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the services in a mobile network along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return all the services in a mobile network along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1118,56 +948,36 @@ private Mono> listByMobileNetworkSinglePageAsync( .error(new IllegalArgumentException("Parameter mobileNetworkName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByMobileNetwork( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByMobileNetwork(this.client.getEndpoint(), resourceGroupName, + mobileNetworkName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the services in a mobile network along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return all the services in a mobile network along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1178,36 +988,21 @@ private Mono> listByMobileNetworkSinglePageAsync( .error(new IllegalArgumentException("Parameter mobileNetworkName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByMobileNetwork( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByMobileNetwork(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1217,14 +1012,13 @@ private Mono> listByMobileNetworkSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink)); } /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1234,16 +1028,15 @@ private PagedFlux listByMobileNetworkAsync(String resourceGroupNam * @return all the services in a mobile network as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByMobileNetworkAsync( - String resourceGroupName, String mobileNetworkName, Context context) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), + private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName, + Context context) { + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink, context)); } /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1258,7 +1051,7 @@ public PagedIterable listByMobileNetwork(String resourceGroupName, /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1268,21 +1061,22 @@ public PagedIterable listByMobileNetwork(String resourceGroupName, * @return all the services in a mobile network as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { return new PagedIterable<>(listByMobileNetworkAsync(resourceGroupName, mobileNetworkName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for services API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink) { @@ -1290,38 +1084,30 @@ private Mono> listByMobileNetworkNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for services API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink, Context context) { @@ -1329,23 +1115,13 @@ private Mono> listByMobileNetworkNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesImpl.java index bc5741ce8fb67..6791e63d4ecf8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/ServicesImpl.java @@ -21,8 +21,8 @@ public final class ServicesImpl implements Services { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public ServicesImpl( - ServicesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public ServicesImpl(ServicesClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -35,15 +35,12 @@ public void delete(String resourceGroupName, String mobileNetworkName, String se this.serviceClient().delete(resourceGroupName, mobileNetworkName, serviceName, context); } - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, serviceName, context); + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, String serviceName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, serviceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ServiceImpl(inner.getValue(), this.manager())); } else { return null; @@ -60,122 +57,90 @@ public Service get(String resourceGroupName, String mobileNetworkName, String se } public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); - return Utils.mapPage(inner, inner1 -> new ServiceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ServiceImpl(inner1, this.manager())); } - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); - return Utils.mapPage(inner, inner1 -> new ServiceImpl(inner1, this.manager())); + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ServiceImpl(inner1, this.manager())); } public Service getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String serviceName = Utils.getValueFromIdByName(id, "services"); + String serviceName = ResourceManagerUtils.getValueFromIdByName(id, "services"); if (serviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, serviceName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String serviceName = Utils.getValueFromIdByName(id, "services"); + String serviceName = ResourceManagerUtils.getValueFromIdByName(id, "services"); if (serviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, serviceName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String serviceName = Utils.getValueFromIdByName(id, "services"); + String serviceName = ResourceManagerUtils.getValueFromIdByName(id, "services"); if (serviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); } this.delete(resourceGroupName, mobileNetworkName, serviceName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String serviceName = Utils.getValueFromIdByName(id, "services"); + String serviceName = ResourceManagerUtils.getValueFromIdByName(id, "services"); if (serviceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'services'.", id))); } this.delete(resourceGroupName, mobileNetworkName, serviceName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupImpl.java index cd87285de9d49..ad78073d76b8b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupImpl.java @@ -99,20 +99,14 @@ public SimGroupImpl withExistingResourceGroup(String resourceGroupName) { } public SimGroup create() { - this.innerObject = - serviceManager - .serviceClient() - .getSimGroups() - .createOrUpdate(resourceGroupName, simGroupName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getSimGroups().createOrUpdate(resourceGroupName, simGroupName, + this.innerModel(), Context.NONE); return this; } public SimGroup create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSimGroups() - .createOrUpdate(resourceGroupName, simGroupName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getSimGroups().createOrUpdate(resourceGroupName, simGroupName, + this.innerModel(), context); return this; } @@ -128,50 +122,34 @@ public SimGroupImpl update() { } public SimGroup apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSimGroups() - .updateTagsWithResponse(resourceGroupName, simGroupName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimGroups() + .updateTagsWithResponse(resourceGroupName, simGroupName, updateParameters, Context.NONE).getValue(); return this; } public SimGroup apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSimGroups() - .updateTagsWithResponse(resourceGroupName, simGroupName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimGroups() + .updateTagsWithResponse(resourceGroupName, simGroupName, updateParameters, context).getValue(); return this; } - SimGroupImpl( - SimGroupInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + SimGroupImpl(SimGroupInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.simGroupName = Utils.getValueFromIdByName(innerObject.id(), "simGroups"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.simGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "simGroups"); } public SimGroup refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSimGroups() - .getByResourceGroupWithResponse(resourceGroupName, simGroupName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimGroups() + .getByResourceGroupWithResponse(resourceGroupName, simGroupName, Context.NONE).getValue(); return this; } public SimGroup refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSimGroups() - .getByResourceGroupWithResponse(resourceGroupName, simGroupName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimGroups() + .getByResourceGroupWithResponse(resourceGroupName, simGroupName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsClientImpl.java index e184516b02705..509b6f00876d5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsClientImpl.java @@ -40,22 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SimGroupsClient. */ +/** + * An instance of this class provides access to all the operations defined in SimGroupsClient. + */ public final class SimGroupsClientImpl implements SimGroupsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SimGroupsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of SimGroupsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SimGroupsClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(SimGroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SimGroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,112 +72,80 @@ public final class SimGroupsClientImpl implements SimGroupsClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface SimGroupsService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @HeaderParam("Accept") String accept, - Context context); + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @HeaderParam("Accept") String accept, - Context context); + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @BodyParam("application/json") SimGroupInner parameters, - @HeaderParam("Accept") String accept, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @BodyParam("application/json") SimGroupInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @BodyParam("application/json") IdentityAndTagsObject parameters, - @HeaderParam("Accept") String accept, + Mono> updateTags(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @BodyParam("application/json") IdentityAndTagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.MobileNetwork/simGroups") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -182,16 +156,12 @@ Mono> listByResourceGroupNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String simGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -202,23 +172,14 @@ private Mono>> deleteWithResponseAsync(String resource } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -228,19 +189,15 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String simGroupName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String simGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -251,20 +208,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, simGroupName, accept, context); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -275,15 +225,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String simGroupName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, simGroupName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -293,18 +241,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String simGroupName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String simGroupName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, simGroupName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -319,7 +266,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -329,14 +276,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String simGroupName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String simGroupName, + Context context) { return this.beginDeleteAsync(resourceGroupName, simGroupName, context).getSyncPoller(); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -351,7 +298,7 @@ private Mono deleteAsync(String resourceGroupName, String simGroupName) { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -362,14 +309,13 @@ private Mono deleteAsync(String resourceGroupName, String simGroupName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String simGroupName, Context context) { - return beginDeleteAsync(resourceGroupName, simGroupName, context) - .last() + return beginDeleteAsync(resourceGroupName, simGroupName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -383,7 +329,7 @@ public void delete(String resourceGroupName, String simGroupName) { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -398,29 +344,25 @@ public void delete(String resourceGroupName, String simGroupName, Context contex /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified SIM group along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about the specified SIM group along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String simGroupName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String simGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -431,46 +373,33 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified SIM group along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about the specified SIM group along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String simGroupName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String simGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -481,20 +410,13 @@ private Mono> getByResourceGroupWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, accept, context); } /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -510,7 +432,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, St /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -520,14 +442,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName, St * @return information about the specified SIM group along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String simGroupName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String simGroupName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, simGroupName, context).block(); } /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -542,7 +464,7 @@ public SimGroupInner getByResourceGroup(String resourceGroupName, String simGrou /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -552,19 +474,15 @@ public SimGroupInner getByResourceGroup(String resourceGroupName, String simGrou * @return sIM group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String simGroupName, SimGroupInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String simGroupName, SimGroupInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -580,24 +498,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -608,19 +516,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return sIM group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String simGroupName, SimGroupInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -636,21 +540,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, parameters, accept, context); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -660,23 +556,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of sIM group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SimGroupInner> beginCreateOrUpdateAsync( - String resourceGroupName, String simGroupName, SimGroupInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - SimGroupInner.class, - SimGroupInner.class, - this.client.getContext()); + private PollerFlux, SimGroupInner> beginCreateOrUpdateAsync(String resourceGroupName, + String simGroupName, SimGroupInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + SimGroupInner.class, SimGroupInner.class, this.client.getContext()); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -687,20 +577,18 @@ private PollerFlux, SimGroupInner> beginCreateOrUpdate * @return the {@link PollerFlux} for polling of sIM group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SimGroupInner> beginCreateOrUpdateAsync( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context) { + private PollerFlux, SimGroupInner> beginCreateOrUpdateAsync(String resourceGroupName, + String simGroupName, SimGroupInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SimGroupInner.class, SimGroupInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + SimGroupInner.class, SimGroupInner.class, context); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -710,14 +598,14 @@ private PollerFlux, SimGroupInner> beginCreateOrUpdate * @return the {@link SyncPoller} for polling of sIM group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SimGroupInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, SimGroupInner parameters) { + public SyncPoller, SimGroupInner> beginCreateOrUpdate(String resourceGroupName, + String simGroupName, SimGroupInner parameters) { return this.beginCreateOrUpdateAsync(resourceGroupName, simGroupName, parameters).getSyncPoller(); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -728,14 +616,14 @@ public SyncPoller, SimGroupInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of sIM group resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SimGroupInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context) { + public SyncPoller, SimGroupInner> beginCreateOrUpdate(String resourceGroupName, + String simGroupName, SimGroupInner parameters, Context context) { return this.beginCreateOrUpdateAsync(resourceGroupName, simGroupName, parameters, context).getSyncPoller(); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -745,16 +633,15 @@ public SyncPoller, SimGroupInner> beginCreateOrUpdate( * @return sIM group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String simGroupName, SimGroupInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String simGroupName, + SimGroupInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -765,16 +652,15 @@ private Mono createOrUpdateAsync( * @return sIM group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String simGroupName, + SimGroupInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -790,7 +676,7 @@ public SimGroupInner createOrUpdate(String resourceGroupName, String simGroupNam /** * Creates or updates a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the create or update SIM group operation. @@ -801,14 +687,14 @@ public SimGroupInner createOrUpdate(String resourceGroupName, String simGroupNam * @return sIM group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SimGroupInner createOrUpdate( - String resourceGroupName, String simGroupName, SimGroupInner parameters, Context context) { + public SimGroupInner createOrUpdate(String resourceGroupName, String simGroupName, SimGroupInner parameters, + Context context) { return createOrUpdateAsync(resourceGroupName, simGroupName, parameters, context).block(); } /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -818,19 +704,15 @@ public SimGroupInner createOrUpdate( * @return sIM group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String simGroupName, IdentityAndTagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String simGroupName, + IdentityAndTagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -846,24 +728,14 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -874,19 +746,15 @@ private Mono> updateTagsWithResponseAsync( * @return sIM group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String simGroupName, IdentityAndTagsObject parameters, Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String simGroupName, + IdentityAndTagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -902,21 +770,13 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, parameters, accept, context); } /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -926,15 +786,15 @@ private Mono> updateTagsWithResponseAsync( * @return sIM group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String simGroupName, IdentityAndTagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String simGroupName, + IdentityAndTagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, simGroupName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -945,14 +805,14 @@ private Mono updateTagsAsync( * @return sIM group resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, String simGroupName, IdentityAndTagsObject parameters, Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String simGroupName, + IdentityAndTagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, simGroupName, parameters, context).block(); } /** * Patch SIM group resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to patch SIM group resource. @@ -968,109 +828,76 @@ public SimGroupInner updateTags(String resourceGroupName, String simGroupName, I /** * Gets all the SIM groups in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the SIM groups in a subscription along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return all the SIM groups in a subscription along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all the SIM groups in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the SIM groups in a subscription along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return all the SIM groups in a subscription along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all the SIM groups in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM groups in a subscription as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Gets all the SIM groups in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1079,13 +906,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Gets all the SIM groups in a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM groups in a subscription as paginated response with {@link PagedIterable}. @@ -1097,7 +924,7 @@ public PagedIterable list() { /** * Gets all the SIM groups in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1111,110 +938,75 @@ public PagedIterable list(Context context) { /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM groups in a resource group along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM groups in a resource group along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1223,14 +1015,13 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1240,14 +1031,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupNa */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1261,7 +1051,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1276,14 +1066,15 @@ public PagedIterable listByResourceGroup(String resourceGroupName /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for list SIM groups API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -1291,38 +1082,30 @@ private Mono> listBySubscriptionNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for list SIM groups API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { @@ -1330,36 +1113,27 @@ private Mono> listBySubscriptionNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for list SIM groups API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1367,63 +1141,45 @@ private Mono> listByResourceGroupNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for list SIM groups API service call along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsImpl.java index 841479709ba3d..f9f847f864459 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimGroupsImpl.java @@ -21,8 +21,8 @@ public final class SimGroupsImpl implements SimGroups { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public SimGroupsImpl( - SimGroupsClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public SimGroupsImpl(SimGroupsClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -35,15 +35,12 @@ public void delete(String resourceGroupName, String simGroupName, Context contex this.serviceClient().delete(resourceGroupName, simGroupName, context); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String simGroupName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, simGroupName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String simGroupName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, simGroupName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SimGroupImpl(inner.getValue(), this.manager())); } else { return null; @@ -61,96 +58,76 @@ public SimGroup getByResourceGroup(String resourceGroupName, String simGroupName public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimGroupImpl(inner1, this.manager())); } public SimGroup getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, simGroupName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, simGroupName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } this.delete(resourceGroupName, simGroupName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } this.delete(resourceGroupName, simGroupName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimImpl.java index 43732cb8ecbd0..5939f4250d34c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimImpl.java @@ -121,20 +121,14 @@ public SimImpl withExistingSimGroup(String resourceGroupName, String simGroupNam } public Sim create() { - this.innerObject = - serviceManager - .serviceClient() - .getSims() - .createOrUpdate(resourceGroupName, simGroupName, simName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getSims().createOrUpdate(resourceGroupName, simGroupName, + simName, this.innerModel(), Context.NONE); return this; } public Sim create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSims() - .createOrUpdate(resourceGroupName, simGroupName, simName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getSims().createOrUpdate(resourceGroupName, simGroupName, + simName, this.innerModel(), context); return this; } @@ -149,48 +143,34 @@ public SimImpl update() { } public Sim apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSims() - .createOrUpdate(resourceGroupName, simGroupName, simName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getSims().createOrUpdate(resourceGroupName, simGroupName, + simName, this.innerModel(), Context.NONE); return this; } public Sim apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSims() - .createOrUpdate(resourceGroupName, simGroupName, simName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getSims().createOrUpdate(resourceGroupName, simGroupName, + simName, this.innerModel(), context); return this; } SimImpl(SimInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.simGroupName = Utils.getValueFromIdByName(innerObject.id(), "simGroups"); - this.simName = Utils.getValueFromIdByName(innerObject.id(), "sims"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.simGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "simGroups"); + this.simName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "sims"); } public Sim refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSims() - .getWithResponse(resourceGroupName, simGroupName, simName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSims() + .getWithResponse(resourceGroupName, simGroupName, simName, Context.NONE).getValue(); return this; } public Sim refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSims() - .getWithResponse(resourceGroupName, simGroupName, simName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSims() + .getWithResponse(resourceGroupName, simGroupName, simName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesClientImpl.java index 72ad144774aaa..b70b87cf6df7f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesClientImpl.java @@ -40,22 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SimPoliciesClient. */ +/** + * An instance of this class provides access to all the operations defined in SimPoliciesClient. + */ public final class SimPoliciesClientImpl implements SimPoliciesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SimPoliciesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of SimPoliciesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SimPoliciesClientImpl(MobileNetworkManagementClientImpl client) { - this.service = - RestProxy.create(SimPoliciesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SimPoliciesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,96 +72,69 @@ public final class SimPoliciesClientImpl implements SimPoliciesClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface SimPoliciesService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("simPolicyName") String simPolicyName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("simPolicyName") String simPolicyName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("simPolicyName") String simPolicyName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("simPolicyName") String simPolicyName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("simPolicyName") String simPolicyName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") SimPolicyInner parameters, - @HeaderParam("Accept") String accept, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("simPolicyName") String simPolicyName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") SimPolicyInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies/{simPolicyName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("simPolicyName") String simPolicyName, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("simPolicyName") String simPolicyName, + @BodyParam("application/json") TagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/simPolicies") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByMobileNetwork( - @HostParam("$host") String endpoint, + Mono> listByMobileNetwork(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("mobileNetworkName") String mobileNetworkName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByMobileNetworkNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -165,13 +144,11 @@ Mono> listByMobileNetworkNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -185,31 +162,19 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter simPolicyName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - simPolicyName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, + simPolicyName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -220,13 +185,11 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -240,28 +203,18 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter simPolicyName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - simPolicyName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, simPolicyName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -271,19 +224,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -294,19 +245,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -316,14 +266,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String simPolicyName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String simPolicyName) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, simPolicyName).getSyncPoller(); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -334,14 +284,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String simPolicyName, Context context) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, simPolicyName, context).getSyncPoller(); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -352,14 +302,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String simPolicyName) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, simPolicyName) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName, simPolicyName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -370,16 +319,15 @@ private Mono deleteAsync(String resourceGroupName, String mobileNetworkNam * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, simPolicyName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String simPolicyName, + Context context) { + return beginDeleteAsync(resourceGroupName, mobileNetworkName, simPolicyName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -394,7 +342,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si /** * Deletes the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -410,24 +358,22 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified SIM policy along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about the specified SIM policy along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -441,31 +387,19 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter simPolicyName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - simPolicyName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, + simPolicyName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -473,17 +407,15 @@ private Mono> getWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about the specified SIM policy along with {@link Response} on successful completion of {@link - * Mono}. + * @return information about the specified SIM policy along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -497,28 +429,18 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter simPolicyName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - simPolicyName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, simPolicyName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -535,7 +457,7 @@ private Mono getAsync(String resourceGroupName, String mobileNet /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -546,14 +468,14 @@ private Mono getAsync(String resourceGroupName, String mobileNet * @return information about the specified SIM policy along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, + String simPolicyName, Context context) { return getWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, context).block(); } /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -569,7 +491,7 @@ public SimPolicyInner get(String resourceGroupName, String mobileNetworkName, St /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -580,13 +502,11 @@ public SimPolicyInner get(String resourceGroupName, String mobileNetworkName, St * @return sIM policy resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -600,10 +520,8 @@ private Mono>> createOrUpdateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter simPolicyName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -612,25 +530,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - simPolicyName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, + mobileNetworkName, simPolicyName, this.client.getApiVersion(), this.client.getSubscriptionId(), + parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -642,17 +550,11 @@ private Mono>> createOrUpdateWithResponseAsync( * @return sIM policy resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -666,10 +568,8 @@ private Mono>> createOrUpdateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter simPolicyName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -678,22 +578,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - simPolicyName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, simPolicyName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -704,23 +595,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of sIM policy resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SimPolicyInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - SimPolicyInner.class, - SimPolicyInner.class, - this.client.getContext()); + private PollerFlux, SimPolicyInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + SimPolicyInner.class, SimPolicyInner.class, this.client.getContext()); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -732,24 +617,18 @@ private PollerFlux, SimPolicyInner> beginCreateOrUpda * @return the {@link PollerFlux} for polling of sIM policy resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SimPolicyInner> beginCreateOrUpdateAsync( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context) { + private PollerFlux, SimPolicyInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SimPolicyInner.class, SimPolicyInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + SimPolicyInner.class, SimPolicyInner.class, context); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -760,16 +639,15 @@ private PollerFlux, SimPolicyInner> beginCreateOrUpda * @return the {@link SyncPoller} for polling of sIM policy resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SimPolicyInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters) + public SyncPoller, SimPolicyInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters) .getSyncPoller(); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -781,20 +659,15 @@ public SyncPoller, SimPolicyInner> beginCreateOrUpdat * @return the {@link SyncPoller} for polling of sIM policy resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SimPolicyInner> beginCreateOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context) + public SyncPoller, SimPolicyInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String simPolicyName, SimPolicyInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context) .getSyncPoller(); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -805,16 +678,15 @@ public SyncPoller, SimPolicyInner> beginCreateOrUpdat * @return sIM policy resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName, SimPolicyInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -826,20 +698,15 @@ private Mono createOrUpdateAsync( * @return sIM policy resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName, SimPolicyInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -850,14 +717,14 @@ private Mono createOrUpdateAsync( * @return sIM policy resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SimPolicyInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String simPolicyName, SimPolicyInner parameters) { + public SimPolicyInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String simPolicyName, + SimPolicyInner parameters) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters).block(); } /** * Creates or updates a SIM policy. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -869,18 +736,14 @@ public SimPolicyInner createOrUpdate( * @return sIM policy resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SimPolicyInner createOrUpdate( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - SimPolicyInner parameters, - Context context) { + public SimPolicyInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String simPolicyName, + SimPolicyInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context).block(); } /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -891,19 +754,15 @@ public SimPolicyInner createOrUpdate( * @return sIM policy resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String simPolicyName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -923,25 +782,15 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - simPolicyName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, simPolicyName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -953,23 +802,15 @@ private Mono> updateTagsWithResponseAsync( * @return sIM policy resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - TagsObject parameters, - Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String simPolicyName, TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -989,22 +830,13 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - simPolicyName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, simPolicyName, parameters, accept, context); } /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -1015,15 +847,15 @@ private Mono> updateTagsWithResponseAsync( * @return sIM policy resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String mobileNetworkName, String simPolicyName, TagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String mobileNetworkName, + String simPolicyName, TagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -1035,19 +867,15 @@ private Mono updateTagsAsync( * @return sIM policy resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, - String mobileNetworkName, - String simPolicyName, - TagsObject parameters, - Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String simPolicyName, TagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, simPolicyName, parameters, context) .block(); } /** * Updates SIM policy tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -1058,31 +886,29 @@ public Response updateTagsWithResponse( * @return sIM policy resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SimPolicyInner updateTags( - String resourceGroupName, String mobileNetworkName, String simPolicyName, TagsObject parameters) { + public SimPolicyInner updateTags(String resourceGroupName, String mobileNetworkName, String simPolicyName, + TagsObject parameters) { return updateTagsWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, parameters, Context.NONE) .getValue(); } /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM policies in a mobile network along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1093,39 +919,21 @@ private Mono> listByMobileNetworkSinglePageAsync( .error(new IllegalArgumentException("Parameter mobileNetworkName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByMobileNetwork( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByMobileNetwork(this.client.getEndpoint(), resourceGroupName, + mobileNetworkName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1133,16 +941,14 @@ private Mono> listByMobileNetworkSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM policies in a mobile network along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1153,36 +959,21 @@ private Mono> listByMobileNetworkSinglePageAsync( .error(new IllegalArgumentException("Parameter mobileNetworkName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByMobileNetwork( - this.client.getEndpoint(), - resourceGroupName, - mobileNetworkName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByMobileNetwork(this.client.getEndpoint(), resourceGroupName, mobileNetworkName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1192,14 +983,13 @@ private Mono> listByMobileNetworkSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink)); } /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1209,16 +999,15 @@ private PagedFlux listByMobileNetworkAsync(String resourceGroupN * @return all the SIM policies in a mobile network as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByMobileNetworkAsync( - String resourceGroupName, String mobileNetworkName, Context context) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), + private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName, + Context context) { + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink, context)); } /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1233,7 +1022,7 @@ public PagedIterable listByMobileNetwork(String resourceGroupNam /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1243,21 +1032,22 @@ public PagedIterable listByMobileNetwork(String resourceGroupNam * @return all the SIM policies in a mobile network as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { return new PagedIterable<>(listByMobileNetworkAsync(resourceGroupName, mobileNetworkName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for SIM policies API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink) { @@ -1265,63 +1055,45 @@ private Mono> listByMobileNetworkNextSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for SIM policies API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesImpl.java index a12aa3b2cf90b..97e682151b2ff 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPoliciesImpl.java @@ -21,8 +21,8 @@ public final class SimPoliciesImpl implements SimPolicies { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public SimPoliciesImpl( - SimPoliciesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public SimPoliciesImpl(SimPoliciesClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -35,15 +35,12 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si this.serviceClient().delete(resourceGroupName, mobileNetworkName, simPolicyName, context); } - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, context); + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, String simPolicyName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SimPolicyImpl(inner.getValue(), this.manager())); } else { return null; @@ -60,122 +57,90 @@ public SimPolicy get(String resourceGroupName, String mobileNetworkName, String } public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); - return Utils.mapPage(inner, inner1 -> new SimPolicyImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimPolicyImpl(inner1, this.manager())); } - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); - return Utils.mapPage(inner, inner1 -> new SimPolicyImpl(inner1, this.manager())); + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimPolicyImpl(inner1, this.manager())); } public SimPolicy getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String simPolicyName = Utils.getValueFromIdByName(id, "simPolicies"); + String simPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "simPolicies"); if (simPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String simPolicyName = Utils.getValueFromIdByName(id, "simPolicies"); + String simPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "simPolicies"); if (simPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String simPolicyName = Utils.getValueFromIdByName(id, "simPolicies"); + String simPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "simPolicies"); if (simPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); } this.delete(resourceGroupName, mobileNetworkName, simPolicyName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String simPolicyName = Utils.getValueFromIdByName(id, "simPolicies"); + String simPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "simPolicies"); if (simPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simPolicies'.", id))); } this.delete(resourceGroupName, mobileNetworkName, simPolicyName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPolicyImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPolicyImpl.java index ffadd129c57c4..07e54bc233c3c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPolicyImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimPolicyImpl.java @@ -126,20 +126,14 @@ public SimPolicyImpl withExistingMobileNetwork(String resourceGroupName, String } public SimPolicy create() { - this.innerObject = - serviceManager - .serviceClient() - .getSimPolicies() - .createOrUpdate(resourceGroupName, mobileNetworkName, simPolicyName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getSimPolicies().createOrUpdate(resourceGroupName, + mobileNetworkName, simPolicyName, this.innerModel(), Context.NONE); return this; } public SimPolicy create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSimPolicies() - .createOrUpdate(resourceGroupName, mobileNetworkName, simPolicyName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getSimPolicies().createOrUpdate(resourceGroupName, + mobileNetworkName, simPolicyName, this.innerModel(), context); return this; } @@ -155,52 +149,37 @@ public SimPolicyImpl update() { } public SimPolicy apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSimPolicies() - .updateTagsWithResponse( - resourceGroupName, mobileNetworkName, simPolicyName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimPolicies() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, updateParameters, Context.NONE) + .getValue(); return this; } public SimPolicy apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSimPolicies() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimPolicies() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, updateParameters, context) + .getValue(); return this; } - SimPolicyImpl( - SimPolicyInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + SimPolicyImpl(SimPolicyInner innerObject, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.mobileNetworkName = Utils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); - this.simPolicyName = Utils.getValueFromIdByName(innerObject.id(), "simPolicies"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); + this.simPolicyName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "simPolicies"); } public SimPolicy refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSimPolicies() - .getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimPolicies() + .getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, Context.NONE).getValue(); return this; } public SimPolicy refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSimPolicies() - .getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSimPolicies() + .getWithResponse(resourceGroupName, mobileNetworkName, simPolicyName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsClientImpl.java index 3874ba161d7a3..3d38d55908bff 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsClientImpl.java @@ -43,17 +43,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SimsClient. */ +/** + * An instance of this class provides access to all the operations defined in SimsClient. + */ public final class SimsClientImpl implements SimsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SimsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of SimsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SimsClientImpl(MobileNetworkManagementClientImpl client) { @@ -68,125 +74,84 @@ public final class SimsClientImpl implements SimsClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface SimsService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims/{simName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims/{simName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @PathParam("simName") String simName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims/{simName}") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @PathParam("simName") String simName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims/{simName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @PathParam("simName") String simName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @PathParam("simName") String simName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims/{simName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims/{simName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("simGroupName") String simGroupName, - @PathParam("simName") String simName, - @BodyParam("application/json") SimInner parameters, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims") - @ExpectedResponses({200}) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("simGroupName") String simGroupName, + @PathParam("simName") String simName, @BodyParam("application/json") SimInner parameters, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/sims") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("simGroupName") String simGroupName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/uploadSims") - @ExpectedResponses({200, 202}) + Mono> listByGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("simGroupName") String simGroupName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/uploadSims") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> bulkUpload( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("simGroupName") String simGroupName, - @BodyParam("application/json") SimUploadList parameters, - @HeaderParam("Accept") String accept, + Mono>> bulkUpload(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("simGroupName") String simGroupName, + @BodyParam("application/json") SimUploadList parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/deleteSims") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/deleteSims") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> bulkDelete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("simGroupName") String simGroupName, - @BodyParam("application/json") SimDeleteList parameters, - @HeaderParam("Accept") String accept, + Mono>> bulkDelete(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("simGroupName") String simGroupName, + @BodyParam("application/json") SimDeleteList parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/uploadEncryptedSims") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/simGroups/{simGroupName}/uploadEncryptedSims") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> bulkUploadEncrypted( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("simGroupName") String simGroupName, - @BodyParam("application/json") EncryptedSimUploadList parameters, - @HeaderParam("Accept") String accept, + Mono>> bulkUploadEncrypted(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("simGroupName") String simGroupName, + @BodyParam("application/json") EncryptedSimUploadList parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByGroupNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -196,19 +161,15 @@ Mono> listByGroupNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String simGroupName, String simName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String simGroupName, + String simName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -222,24 +183,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - simName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, simName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -250,19 +201,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String simGroupName, String simName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String simGroupName, + String simName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -276,21 +223,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - simName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, simGroupName, simName, accept, context); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -300,18 +239,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String simGroupName, String simName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String simGroupName, + String simName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, simGroupName, simName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -322,19 +259,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String simGroupName, String simName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String simGroupName, + String simName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, simGroupName, simName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, simGroupName, simName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -344,14 +280,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String simGroupName, String simName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String simGroupName, + String simName) { return this.beginDeleteAsync(resourceGroupName, simGroupName, simName).getSyncPoller(); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -362,14 +298,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String simGroupName, String simName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String simGroupName, String simName, + Context context) { return this.beginDeleteAsync(resourceGroupName, simGroupName, simName, context).getSyncPoller(); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -380,14 +316,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String simGroupName, String simName) { - return beginDeleteAsync(resourceGroupName, simGroupName, simName) - .last() + return beginDeleteAsync(resourceGroupName, simGroupName, simName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -399,14 +334,13 @@ private Mono deleteAsync(String resourceGroupName, String simGroupName, St */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String simGroupName, String simName, Context context) { - return beginDeleteAsync(resourceGroupName, simGroupName, simName, context) - .last() + return beginDeleteAsync(resourceGroupName, simGroupName, simName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -421,7 +355,7 @@ public void delete(String resourceGroupName, String simGroupName, String simName /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -437,7 +371,7 @@ public void delete(String resourceGroupName, String simGroupName, String simName /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -447,19 +381,15 @@ public void delete(String resourceGroupName, String simGroupName, String simName * @return information about the specified SIM along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String simGroupName, String simName) { + private Mono> getWithResponseAsync(String resourceGroupName, String simGroupName, + String simName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -473,24 +403,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - simName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, simName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -501,19 +421,15 @@ private Mono> getWithResponseAsync( * @return information about the specified SIM along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String simGroupName, String simName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String simGroupName, String simName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -527,21 +443,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - simName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, simGroupName, simName, accept, context); } /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -558,7 +466,7 @@ private Mono getAsync(String resourceGroupName, String simGroupName, S /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -569,14 +477,14 @@ private Mono getAsync(String resourceGroupName, String simGroupName, S * @return information about the specified SIM along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String simGroupName, String simName, Context context) { + public Response getWithResponse(String resourceGroupName, String simGroupName, String simName, + Context context) { return getWithResponseAsync(resourceGroupName, simGroupName, simName, context).block(); } /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -592,7 +500,7 @@ public SimInner get(String resourceGroupName, String simGroupName, String simNam /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -603,19 +511,15 @@ public SimInner get(String resourceGroupName, String simGroupName, String simNam * @return sIM resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String simGroupName, String simName, SimInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String simGroupName, String simName, SimInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -634,25 +538,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - simName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, simName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -664,19 +557,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return sIM resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String simGroupName, String simName, SimInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -695,22 +584,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - simGroupName, - simName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, simGroupName, simName, parameters, accept, context); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -721,19 +601,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of sIM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SimInner> beginCreateOrUpdateAsync( - String resourceGroupName, String simGroupName, String simName, SimInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, simName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SimInner.class, SimInner.class, this.client.getContext()); + private PollerFlux, SimInner> beginCreateOrUpdateAsync(String resourceGroupName, + String simGroupName, String simName, SimInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, simName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), SimInner.class, + SimInner.class, this.client.getContext()); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -745,20 +623,18 @@ private PollerFlux, SimInner> beginCreateOrUpdateAsync( * @return the {@link PollerFlux} for polling of sIM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SimInner> beginCreateOrUpdateAsync( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context) { + private PollerFlux, SimInner> beginCreateOrUpdateAsync(String resourceGroupName, + String simGroupName, String simName, SimInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, simName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SimInner.class, SimInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, simGroupName, simName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), SimInner.class, + SimInner.class, context); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -769,14 +645,14 @@ private PollerFlux, SimInner> beginCreateOrUpdateAsync( * @return the {@link SyncPoller} for polling of sIM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SimInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, String simName, SimInner parameters) { + public SyncPoller, SimInner> beginCreateOrUpdate(String resourceGroupName, String simGroupName, + String simName, SimInner parameters) { return this.beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters).getSyncPoller(); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -788,16 +664,15 @@ public SyncPoller, SimInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of sIM resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SimInner> beginCreateOrUpdate( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters, context) + public SyncPoller, SimInner> beginCreateOrUpdate(String resourceGroupName, String simGroupName, + String simName, SimInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters, context) .getSyncPoller(); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -808,16 +683,15 @@ public SyncPoller, SimInner> beginCreateOrUpdate( * @return sIM resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String simGroupName, String simName, SimInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String simGroupName, String simName, + SimInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -829,16 +703,15 @@ private Mono createOrUpdateAsync( * @return sIM resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String simGroupName, String simName, + SimInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -855,7 +728,7 @@ public SimInner createOrUpdate(String resourceGroupName, String simGroupName, St /** * Creates or updates a SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -867,14 +740,14 @@ public SimInner createOrUpdate(String resourceGroupName, String simGroupName, St * @return sIM resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SimInner createOrUpdate( - String resourceGroupName, String simGroupName, String simName, SimInner parameters, Context context) { + public SimInner createOrUpdate(String resourceGroupName, String simGroupName, String simName, SimInner parameters, + Context context) { return createOrUpdateAsync(resourceGroupName, simGroupName, simName, parameters, context).block(); } /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -885,52 +758,32 @@ public SimInner createOrUpdate( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByGroupSinglePageAsync(String resourceGroupName, String simGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByGroup(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), simGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -940,23 +793,19 @@ private Mono> listByGroupSinglePageAsync(String resource * @return all the SIMs in a SIM group along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByGroupSinglePageAsync( - String resourceGroupName, String simGroupName, Context context) { + private Mono> listByGroupSinglePageAsync(String resourceGroupName, String simGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -964,28 +813,15 @@ private Mono> listByGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByGroup(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), simGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -995,14 +831,13 @@ private Mono> listByGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByGroupAsync(String resourceGroupName, String simGroupName) { - return new PagedFlux<>( - () -> listByGroupSinglePageAsync(resourceGroupName, simGroupName), + return new PagedFlux<>(() -> listByGroupSinglePageAsync(resourceGroupName, simGroupName), nextLink -> listByGroupNextSinglePageAsync(nextLink)); } /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -1013,14 +848,13 @@ private PagedFlux listByGroupAsync(String resourceGroupName, String si */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByGroupAsync(String resourceGroupName, String simGroupName, Context context) { - return new PagedFlux<>( - () -> listByGroupSinglePageAsync(resourceGroupName, simGroupName, context), + return new PagedFlux<>(() -> listByGroupSinglePageAsync(resourceGroupName, simGroupName, context), nextLink -> listByGroupNextSinglePageAsync(nextLink, context)); } /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1035,7 +869,7 @@ public PagedIterable listByGroup(String resourceGroupName, String simG /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -1051,34 +885,30 @@ public PagedIterable listByGroup(String resourceGroupName, String simG /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> bulkUploadWithResponseAsync( - String resourceGroupName, String simGroupName, SimUploadList parameters) { + private Mono>> bulkUploadWithResponseAsync(String resourceGroupName, String simGroupName, + SimUploadList parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -1091,23 +921,14 @@ private Mono>> bulkUploadWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .bulkUpload( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - parameters, - accept, - context)) + context -> service.bulkUpload(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), simGroupName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1115,27 +936,23 @@ private Mono>> bulkUploadWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> bulkUploadWithResponseAsync( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { + private Mono>> bulkUploadWithResponseAsync(String resourceGroupName, String simGroupName, + SimUploadList parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -1147,21 +964,13 @@ private Mono>> bulkUploadWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .bulkUpload( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - parameters, - accept, - context); + return service.bulkUpload(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), simGroupName, parameters, accept, context); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1171,23 +980,18 @@ private Mono>> bulkUploadWithResponseAsync( * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginBulkUploadAsync( - String resourceGroupName, String simGroupName, SimUploadList parameters) { - Mono>> mono = - bulkUploadWithResponseAsync(resourceGroupName, simGroupName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + private PollerFlux, AsyncOperationStatusInner> + beginBulkUploadAsync(String resourceGroupName, String simGroupName, SimUploadList parameters) { + Mono>> mono + = bulkUploadWithResponseAsync(resourceGroupName, simGroupName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1198,24 +1002,18 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginBulkUploadAsync( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { + private PollerFlux, AsyncOperationStatusInner> + beginBulkUploadAsync(String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - bulkUploadWithResponseAsync(resourceGroupName, simGroupName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono + = bulkUploadWithResponseAsync(resourceGroupName, simGroupName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1225,14 +1023,14 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginBulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters) { + public SyncPoller, AsyncOperationStatusInner> + beginBulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters) { return this.beginBulkUploadAsync(resourceGroupName, simGroupName, parameters).getSyncPoller(); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1243,14 +1041,14 @@ public SyncPoller, AsyncOperationStatusInn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginBulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { + public SyncPoller, AsyncOperationStatusInner> + beginBulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { return this.beginBulkUploadAsync(resourceGroupName, simGroupName, parameters, context).getSyncPoller(); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1260,16 +1058,15 @@ public SyncPoller, AsyncOperationStatusInn * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono bulkUploadAsync( - String resourceGroupName, String simGroupName, SimUploadList parameters) { - return beginBulkUploadAsync(resourceGroupName, simGroupName, parameters) - .last() + private Mono bulkUploadAsync(String resourceGroupName, String simGroupName, + SimUploadList parameters) { + return beginBulkUploadAsync(resourceGroupName, simGroupName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1280,16 +1077,15 @@ private Mono bulkUploadAsync( * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono bulkUploadAsync( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { - return beginBulkUploadAsync(resourceGroupName, simGroupName, parameters, context) - .last() + private Mono bulkUploadAsync(String resourceGroupName, String simGroupName, + SimUploadList parameters, Context context) { + return beginBulkUploadAsync(resourceGroupName, simGroupName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1299,14 +1095,14 @@ private Mono bulkUploadAsync( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner bulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters) { + public AsyncOperationStatusInner bulkUpload(String resourceGroupName, String simGroupName, + SimUploadList parameters) { return bulkUploadAsync(resourceGroupName, simGroupName, parameters).block(); } /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -1317,41 +1113,37 @@ public AsyncOperationStatusInner bulkUpload( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner bulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { + public AsyncOperationStatusInner bulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters, + Context context) { return bulkUploadAsync(resourceGroupName, simGroupName, parameters, context).block(); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> bulkDeleteWithResponseAsync( - String resourceGroupName, String simGroupName, SimDeleteList parameters) { + private Mono>> bulkDeleteWithResponseAsync(String resourceGroupName, String simGroupName, + SimDeleteList parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -1364,23 +1156,14 @@ private Mono>> bulkDeleteWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .bulkDelete( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - parameters, - accept, - context)) + context -> service.bulkDelete(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), simGroupName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1388,27 +1171,23 @@ private Mono>> bulkDeleteWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> bulkDeleteWithResponseAsync( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { + private Mono>> bulkDeleteWithResponseAsync(String resourceGroupName, String simGroupName, + SimDeleteList parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -1420,21 +1199,13 @@ private Mono>> bulkDeleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .bulkDelete( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - parameters, - accept, - context); + return service.bulkDelete(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), simGroupName, parameters, accept, context); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1444,23 +1215,18 @@ private Mono>> bulkDeleteWithResponseAsync( * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginBulkDeleteAsync( - String resourceGroupName, String simGroupName, SimDeleteList parameters) { - Mono>> mono = - bulkDeleteWithResponseAsync(resourceGroupName, simGroupName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + private PollerFlux, AsyncOperationStatusInner> + beginBulkDeleteAsync(String resourceGroupName, String simGroupName, SimDeleteList parameters) { + Mono>> mono + = bulkDeleteWithResponseAsync(resourceGroupName, simGroupName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1471,24 +1237,18 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link PollerFlux} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AsyncOperationStatusInner> beginBulkDeleteAsync( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { + private PollerFlux, AsyncOperationStatusInner> + beginBulkDeleteAsync(String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - bulkDeleteWithResponseAsync(resourceGroupName, simGroupName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono + = bulkDeleteWithResponseAsync(resourceGroupName, simGroupName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1498,14 +1258,14 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginBulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters) { + public SyncPoller, AsyncOperationStatusInner> + beginBulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters) { return this.beginBulkDeleteAsync(resourceGroupName, simGroupName, parameters).getSyncPoller(); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1516,14 +1276,14 @@ public SyncPoller, AsyncOperationStatusInn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginBulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { + public SyncPoller, AsyncOperationStatusInner> + beginBulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { return this.beginBulkDeleteAsync(resourceGroupName, simGroupName, parameters, context).getSyncPoller(); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1533,16 +1293,15 @@ public SyncPoller, AsyncOperationStatusInn * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono bulkDeleteAsync( - String resourceGroupName, String simGroupName, SimDeleteList parameters) { - return beginBulkDeleteAsync(resourceGroupName, simGroupName, parameters) - .last() + private Mono bulkDeleteAsync(String resourceGroupName, String simGroupName, + SimDeleteList parameters) { + return beginBulkDeleteAsync(resourceGroupName, simGroupName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1553,16 +1312,15 @@ private Mono bulkDeleteAsync( * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono bulkDeleteAsync( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { - return beginBulkDeleteAsync(resourceGroupName, simGroupName, parameters, context) - .last() + private Mono bulkDeleteAsync(String resourceGroupName, String simGroupName, + SimDeleteList parameters, Context context) { + return beginBulkDeleteAsync(resourceGroupName, simGroupName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1572,14 +1330,14 @@ private Mono bulkDeleteAsync( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner bulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters) { + public AsyncOperationStatusInner bulkDelete(String resourceGroupName, String simGroupName, + SimDeleteList parameters) { return bulkDeleteAsync(resourceGroupName, simGroupName, parameters).block(); } /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -1590,41 +1348,37 @@ public AsyncOperationStatusInner bulkDelete( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner bulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { + public AsyncOperationStatusInner bulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters, + Context context) { return bulkDeleteAsync(resourceGroupName, simGroupName, parameters, context).block(); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> bulkUploadEncryptedWithResponseAsync( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { + private Mono>> bulkUploadEncryptedWithResponseAsync(String resourceGroupName, + String simGroupName, EncryptedSimUploadList parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -1635,25 +1389,14 @@ private Mono>> bulkUploadEncryptedWithResponseAsync( parameters.validate(); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .bulkUploadEncrypted( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - parameters, - accept, - context)) + return FluxUtil.withContext(context -> service.bulkUploadEncrypted(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), simGroupName, parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1661,27 +1404,23 @@ private Mono>> bulkUploadEncryptedWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the current status of an async operation along with {@link Response} on successful completion of {@link - * Mono}. + * @return the current status of an async operation along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> bulkUploadEncryptedWithResponseAsync( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context) { + private Mono>> bulkUploadEncryptedWithResponseAsync(String resourceGroupName, + String simGroupName, EncryptedSimUploadList parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (simGroupName == null) { return Mono.error(new IllegalArgumentException("Parameter simGroupName is required and cannot be null.")); @@ -1693,21 +1432,13 @@ private Mono>> bulkUploadEncryptedWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .bulkUploadEncrypted( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - simGroupName, - parameters, - accept, - context); + return service.bulkUploadEncrypted(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), simGroupName, parameters, accept, context); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1719,21 +1450,16 @@ private Mono>> bulkUploadEncryptedWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, AsyncOperationStatusInner> beginBulkUploadEncryptedAsync( String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { - Mono>> mono = - bulkUploadEncryptedWithResponseAsync(resourceGroupName, simGroupName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - this.client.getContext()); + Mono>> mono + = bulkUploadEncryptedWithResponseAsync(resourceGroupName, simGroupName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, + this.client.getContext()); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1747,21 +1473,15 @@ private PollerFlux, AsyncOperationStatusIn private PollerFlux, AsyncOperationStatusInner> beginBulkUploadEncryptedAsync( String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - bulkUploadEncryptedWithResponseAsync(resourceGroupName, simGroupName, parameters, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AsyncOperationStatusInner.class, - AsyncOperationStatusInner.class, - context); + Mono>> mono + = bulkUploadEncryptedWithResponseAsync(resourceGroupName, simGroupName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AsyncOperationStatusInner.class, AsyncOperationStatusInner.class, context); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1771,14 +1491,14 @@ private PollerFlux, AsyncOperationStatusIn * @return the {@link SyncPoller} for polling of the current status of an async operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AsyncOperationStatusInner> beginBulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { + public SyncPoller, AsyncOperationStatusInner> + beginBulkUploadEncrypted(String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { return this.beginBulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters).getSyncPoller(); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1796,7 +1516,7 @@ public SyncPoller, AsyncOperationStatusInn /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1806,16 +1526,15 @@ public SyncPoller, AsyncOperationStatusInn * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono bulkUploadEncryptedAsync( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { - return beginBulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters) - .last() + private Mono bulkUploadEncryptedAsync(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters) { + return beginBulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1826,16 +1545,15 @@ private Mono bulkUploadEncryptedAsync( * @return the current status of an async operation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono bulkUploadEncryptedAsync( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context) { - return beginBulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters, context) - .last() + private Mono bulkUploadEncryptedAsync(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters, Context context) { + return beginBulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1845,14 +1563,14 @@ private Mono bulkUploadEncryptedAsync( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { + public AsyncOperationStatusInner bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters) { return bulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters).block(); } /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -1863,21 +1581,22 @@ public AsyncOperationStatusInner bulkUploadEncrypted( * @return the current status of an async operation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AsyncOperationStatusInner bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context) { + public AsyncOperationStatusInner bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters, Context context) { return bulkUploadEncryptedAsync(resourceGroupName, simGroupName, parameters, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for list SIMs API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByGroupNextSinglePageAsync(String nextLink) { @@ -1885,37 +1604,29 @@ private Mono> listByGroupNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for list SIMs API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByGroupNextSinglePageAsync(String nextLink, Context context) { @@ -1923,23 +1634,13 @@ private Mono> listByGroupNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsImpl.java index 8f852aa004ed7..5f8bf2b69b44c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SimsImpl.java @@ -26,8 +26,8 @@ public final class SimsImpl implements Sims { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public SimsImpl( - SimsClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public SimsImpl(SimsClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -40,15 +40,12 @@ public void delete(String resourceGroupName, String simGroupName, String simName this.serviceClient().delete(resourceGroupName, simGroupName, simName, context); } - public Response getWithResponse( - String resourceGroupName, String simGroupName, String simName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, simGroupName, simName, context); + public Response getWithResponse(String resourceGroupName, String simGroupName, String simName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, simGroupName, simName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SimImpl(inner.getValue(), this.manager())); } else { return null; @@ -66,12 +63,12 @@ public Sim get(String resourceGroupName, String simGroupName, String simName) { public PagedIterable listByGroup(String resourceGroupName, String simGroupName) { PagedIterable inner = this.serviceClient().listByGroup(resourceGroupName, simGroupName); - return Utils.mapPage(inner, inner1 -> new SimImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimImpl(inner1, this.manager())); } public PagedIterable listByGroup(String resourceGroupName, String simGroupName, Context context) { PagedIterable inner = this.serviceClient().listByGroup(resourceGroupName, simGroupName, context); - return Utils.mapPage(inner, inner1 -> new SimImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SimImpl(inner1, this.manager())); } public AsyncOperationStatus bulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters) { @@ -83,10 +80,10 @@ public AsyncOperationStatus bulkUpload(String resourceGroupName, String simGroup } } - public AsyncOperationStatus bulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context) { - AsyncOperationStatusInner inner = - this.serviceClient().bulkUpload(resourceGroupName, simGroupName, parameters, context); + public AsyncOperationStatus bulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters, + Context context) { + AsyncOperationStatusInner inner + = this.serviceClient().bulkUpload(resourceGroupName, simGroupName, parameters, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -103,10 +100,10 @@ public AsyncOperationStatus bulkDelete(String resourceGroupName, String simGroup } } - public AsyncOperationStatus bulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context) { - AsyncOperationStatusInner inner = - this.serviceClient().bulkDelete(resourceGroupName, simGroupName, parameters, context); + public AsyncOperationStatus bulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters, + Context context) { + AsyncOperationStatusInner inner + = this.serviceClient().bulkDelete(resourceGroupName, simGroupName, parameters, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -114,10 +111,10 @@ public AsyncOperationStatus bulkDelete( } } - public AsyncOperationStatus bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters) { - AsyncOperationStatusInner inner = - this.serviceClient().bulkUploadEncrypted(resourceGroupName, simGroupName, parameters); + public AsyncOperationStatus bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters) { + AsyncOperationStatusInner inner + = this.serviceClient().bulkUploadEncrypted(resourceGroupName, simGroupName, parameters); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -125,10 +122,10 @@ public AsyncOperationStatus bulkUploadEncrypted( } } - public AsyncOperationStatus bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context) { - AsyncOperationStatusInner inner = - this.serviceClient().bulkUploadEncrypted(resourceGroupName, simGroupName, parameters, context); + public AsyncOperationStatus bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters, Context context) { + AsyncOperationStatusInner inner + = this.serviceClient().bulkUploadEncrypted(resourceGroupName, simGroupName, parameters, context); if (inner != null) { return new AsyncOperationStatusImpl(inner, this.manager()); } else { @@ -137,105 +134,77 @@ public AsyncOperationStatus bulkUploadEncrypted( } public Sim getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } - String simName = Utils.getValueFromIdByName(id, "sims"); + String simName = ResourceManagerUtils.getValueFromIdByName(id, "sims"); if (simName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); } return this.getWithResponse(resourceGroupName, simGroupName, simName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } - String simName = Utils.getValueFromIdByName(id, "sims"); + String simName = ResourceManagerUtils.getValueFromIdByName(id, "sims"); if (simName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); } return this.getWithResponse(resourceGroupName, simGroupName, simName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } - String simName = Utils.getValueFromIdByName(id, "sims"); + String simName = ResourceManagerUtils.getValueFromIdByName(id, "sims"); if (simName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); } this.delete(resourceGroupName, simGroupName, simName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String simGroupName = Utils.getValueFromIdByName(id, "simGroups"); + String simGroupName = ResourceManagerUtils.getValueFromIdByName(id, "simGroups"); if (simGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'simGroups'.", id))); } - String simName = Utils.getValueFromIdByName(id, "sims"); + String simName = ResourceManagerUtils.getValueFromIdByName(id, "sims"); if (simName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sims'.", id))); } this.delete(resourceGroupName, simGroupName, simName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SiteImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SiteImpl.java index 1f8cd6814b77a..ae60a32fcf6c9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SiteImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SiteImpl.java @@ -99,20 +99,14 @@ public SiteImpl withExistingMobileNetwork(String resourceGroupName, String mobil } public Site create() { - this.innerObject = - serviceManager - .serviceClient() - .getSites() - .createOrUpdate(resourceGroupName, mobileNetworkName, siteName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getSites().createOrUpdate(resourceGroupName, + mobileNetworkName, siteName, this.innerModel(), Context.NONE); return this; } public Site create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSites() - .createOrUpdate(resourceGroupName, mobileNetworkName, siteName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getSites().createOrUpdate(resourceGroupName, + mobileNetworkName, siteName, this.innerModel(), context); return this; } @@ -128,50 +122,36 @@ public SiteImpl update() { } public Site apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSites() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, siteName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSites() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, siteName, updateParameters, Context.NONE) + .getValue(); return this; } public Site apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSites() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, siteName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSites() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, siteName, updateParameters, context) + .getValue(); return this; } SiteImpl(SiteInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.mobileNetworkName = Utils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); - this.siteName = Utils.getValueFromIdByName(innerObject.id(), "sites"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); + this.siteName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "sites"); } public Site refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSites() - .getWithResponse(resourceGroupName, mobileNetworkName, siteName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSites() + .getWithResponse(resourceGroupName, mobileNetworkName, siteName, Context.NONE).getValue(); return this; } public Site refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSites() - .getWithResponse(resourceGroupName, mobileNetworkName, siteName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSites() + .getWithResponse(resourceGroupName, mobileNetworkName, siteName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesClientImpl.java index 06b880f8e23c7..6591a68b986bc 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesClientImpl.java @@ -42,17 +42,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SitesClient. */ +/** + * An instance of this class provides access to all the operations defined in SitesClient. + */ public final class SitesClientImpl implements SitesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SitesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of SitesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SitesClientImpl(MobileNetworkManagementClientImpl client) { @@ -61,119 +67,87 @@ public final class SitesClientImpl implements SitesClient { } /** - * The interface defining all the services for MobileNetworkManagementClientSites to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for MobileNetworkManagementClientSites to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface SitesService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("siteName") String siteName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("siteName") String siteName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("siteName") String siteName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("siteName") String siteName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("siteName") String siteName, - @BodyParam("application/json") SiteInner parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("siteName") String siteName, + @BodyParam("application/json") SiteInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("siteName") String siteName, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("siteName") String siteName, + @BodyParam("application/json") TagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByMobileNetwork( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> listByMobileNetwork(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("mobileNetworkName") String mobileNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}/deletePacketCore") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/sites/{siteName}/deletePacketCore") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> deletePacketCore( - @HostParam("$host") String endpoint, + Mono>> deletePacketCore(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("siteName") String siteName, - @BodyParam("application/json") SiteDeletePacketCore parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("siteName") String siteName, + @BodyParam("application/json") SiteDeletePacketCore parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByMobileNetworkNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -183,19 +157,15 @@ Mono> listByMobileNetworkNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String siteName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -210,25 +180,15 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, siteName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -239,19 +199,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String siteName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -266,22 +222,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, siteName, accept, context); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -291,19 +239,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String siteName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String siteName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, siteName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -314,20 +260,19 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String siteName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -337,15 +282,15 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String siteName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String siteName) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, siteName).getSyncPoller(); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -356,15 +301,15 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String siteName, Context context) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, siteName, context).getSyncPoller(); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -375,15 +320,14 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String siteName) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, siteName) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName, siteName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -394,17 +338,16 @@ private Mono deleteAsync(String resourceGroupName, String mobileNetworkNam * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, siteName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String siteName, + Context context) { + return beginDeleteAsync(resourceGroupName, mobileNetworkName, siteName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -420,7 +363,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -436,7 +379,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -444,22 +387,18 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified mobile network site along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String siteName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -474,24 +413,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, siteName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -500,22 +429,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified mobile network site along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String siteName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -530,21 +455,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, siteName, accept, context); } /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -561,7 +478,7 @@ private Mono getAsync(String resourceGroupName, String mobileNetworkN /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -572,14 +489,14 @@ private Mono getAsync(String resourceGroupName, String mobileNetworkN * @return information about the specified mobile network site along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, String siteName, + Context context) { return getWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, context).block(); } /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -595,7 +512,7 @@ public SiteInner get(String resourceGroupName, String mobileNetworkName, String /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -606,19 +523,15 @@ public SiteInner get(String resourceGroupName, String mobileNetworkName, String * @return site resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -638,25 +551,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, siteName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -668,19 +571,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return site resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -700,22 +599,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, siteName, parameters, accept, context); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -726,19 +616,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of site resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SiteInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SiteInner.class, SiteInner.class, this.client.getContext()); + private PollerFlux, SiteInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), SiteInner.class, + SiteInner.class, this.client.getContext()); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -750,20 +638,18 @@ private PollerFlux, SiteInner> beginCreateOrUpdateAsync( * @return the {@link PollerFlux} for polling of site resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SiteInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context) { + private PollerFlux, SiteInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SiteInner.class, SiteInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), SiteInner.class, + SiteInner.class, context); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -774,16 +660,15 @@ private PollerFlux, SiteInner> beginCreateOrUpdateAsync( * @return the {@link SyncPoller} for polling of site resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SiteInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters) + public SyncPoller, SiteInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String siteName, SiteInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters) .getSyncPoller(); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -795,16 +680,15 @@ public SyncPoller, SiteInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of site resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SiteInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context) + public SyncPoller, SiteInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String siteName, SiteInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context) .getSyncPoller(); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -815,16 +699,15 @@ public SyncPoller, SiteInner> beginCreateOrUpdate( * @return site resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, String siteName, + SiteInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -836,16 +719,15 @@ private Mono createOrUpdateAsync( * @return site resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, String siteName, + SiteInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -856,14 +738,14 @@ private Mono createOrUpdateAsync( * @return site resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SiteInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters) { + public SiteInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String siteName, + SiteInner parameters) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters).block(); } /** * Creates or updates a mobile network site. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -875,14 +757,14 @@ public SiteInner createOrUpdate( * @return site resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SiteInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String siteName, SiteInner parameters, Context context) { + public SiteInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String siteName, + SiteInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context).block(); } /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -893,19 +775,15 @@ public SiteInner createOrUpdate( * @return site resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String siteName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -925,25 +803,15 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, siteName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -955,19 +823,15 @@ private Mono> updateTagsWithResponseAsync( * @return site resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, TagsObject parameters, Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String siteName, TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -987,22 +851,13 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, siteName, parameters, accept, context); } /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1013,15 +868,15 @@ private Mono> updateTagsWithResponseAsync( * @return site resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String mobileNetworkName, String siteName, TagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String mobileNetworkName, String siteName, + TagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1033,14 +888,14 @@ private Mono updateTagsAsync( * @return site resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, String siteName, TagsObject parameters, Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String siteName, TagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context).block(); } /** * Updates site tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1051,37 +906,33 @@ public Response updateTagsWithResponse( * @return site resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SiteInner updateTags( - String resourceGroupName, String mobileNetworkName, String siteName, TagsObject parameters) { + public SiteInner updateTags(String resourceGroupName, String mobileNetworkName, String siteName, + TagsObject parameters) { return updateTagsWithResponse(resourceGroupName, mobileNetworkName, siteName, parameters, Context.NONE) .getValue(); } /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for sites API service call along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response for sites API service call along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1093,55 +944,35 @@ private Mono> listByMobileNetworkSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByMobileNetwork( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - mobileNetworkName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByMobileNetwork(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, mobileNetworkName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for sites API service call along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response for sites API service call along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1154,28 +985,15 @@ private Mono> listByMobileNetworkSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByMobileNetwork( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - mobileNetworkName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByMobileNetwork(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, mobileNetworkName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1185,14 +1003,13 @@ private Mono> listByMobileNetworkSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink)); } /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1202,16 +1019,15 @@ private PagedFlux listByMobileNetworkAsync(String resourceGroupName, * @return response for sites API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByMobileNetworkAsync( - String resourceGroupName, String mobileNetworkName, Context context) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), + private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName, + Context context) { + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink, context)); } /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1226,7 +1042,7 @@ public PagedIterable listByMobileNetwork(String resourceGroupName, St /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1236,14 +1052,14 @@ public PagedIterable listByMobileNetwork(String resourceGroupName, St * @return response for sites API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { return new PagedIterable<>(listByMobileNetworkAsync(resourceGroupName, mobileNetworkName, context)); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1254,19 +1070,15 @@ public PagedIterable listByMobileNetwork( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deletePacketCoreWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { + private Mono>> deletePacketCoreWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1286,25 +1098,15 @@ private Mono>> deletePacketCoreWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .deletePacketCore( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - parameters, - accept, - context)) + .withContext(context -> service.deletePacketCore(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, siteName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1316,23 +1118,15 @@ private Mono>> deletePacketCoreWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deletePacketCoreWithResponseAsync( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context) { + private Mono>> deletePacketCoreWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteDeletePacketCore parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1352,22 +1146,13 @@ private Mono>> deletePacketCoreWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .deletePacketCore( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - siteName, - parameters, - accept, - context); + return service.deletePacketCore(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, siteName, parameters, accept, context); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1378,19 +1163,17 @@ private Mono>> deletePacketCoreWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeletePacketCoreAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { - Mono>> mono = - deletePacketCoreWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeletePacketCoreAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { + Mono>> mono + = deletePacketCoreWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1402,23 +1185,18 @@ private PollerFlux, Void> beginDeletePacketCoreAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeletePacketCoreAsync( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context) { + private PollerFlux, Void> beginDeletePacketCoreAsync(String resourceGroupName, + String mobileNetworkName, String siteName, SiteDeletePacketCore parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deletePacketCoreWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deletePacketCoreWithResponseAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1429,16 +1207,15 @@ private PollerFlux, Void> beginDeletePacketCoreAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDeletePacketCore( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { - return this - .beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters) + public SyncPoller, Void> beginDeletePacketCore(String resourceGroupName, String mobileNetworkName, + String siteName, SiteDeletePacketCore parameters) { + return this.beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters) .getSyncPoller(); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1450,20 +1227,15 @@ public SyncPoller, Void> beginDeletePacketCore( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDeletePacketCore( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context) { - return this - .beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context) + public SyncPoller, Void> beginDeletePacketCore(String resourceGroupName, String mobileNetworkName, + String siteName, SiteDeletePacketCore parameters, Context context) { + return this.beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context) .getSyncPoller(); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1474,16 +1246,15 @@ public SyncPoller, Void> beginDeletePacketCore( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deletePacketCoreAsync( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { - return beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters) - .last() + private Mono deletePacketCoreAsync(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters) { + return beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1495,20 +1266,15 @@ private Mono deletePacketCoreAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deletePacketCoreAsync( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context) { - return beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context) - .last() + private Mono deletePacketCoreAsync(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters, Context context) { + return beginDeletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1518,14 +1284,14 @@ private Mono deletePacketCoreAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void deletePacketCore( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { + public void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters) { deletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters).block(); } /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -1536,25 +1302,22 @@ public void deletePacketCore( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void deletePacketCore( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context) { + public void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters, Context context) { deletePacketCoreAsync(resourceGroupName, mobileNetworkName, siteName, parameters, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for sites API service call along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response for sites API service call along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink) { @@ -1562,38 +1325,30 @@ private Mono> listByMobileNetworkNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for sites API service call along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response for sites API service call along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink, Context context) { @@ -1601,23 +1356,13 @@ private Mono> listByMobileNetworkNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesImpl.java index 1d729731c58cf..92d4e473daa89 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SitesImpl.java @@ -22,8 +22,8 @@ public final class SitesImpl implements Sites { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public SitesImpl( - SitesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public SitesImpl(SitesClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -36,15 +36,12 @@ public void delete(String resourceGroupName, String mobileNetworkName, String si this.serviceClient().delete(resourceGroupName, mobileNetworkName, siteName, context); } - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String siteName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, siteName, context); + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, String siteName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, siteName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SiteImpl(inner.getValue(), this.manager())); } else { return null; @@ -62,134 +59,98 @@ public Site get(String resourceGroupName, String mobileNetworkName, String siteN public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName) { PagedIterable inner = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); - return Utils.mapPage(inner, inner1 -> new SiteImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SiteImpl(inner1, this.manager())); } - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); - return Utils.mapPage(inner, inner1 -> new SiteImpl(inner1, this.manager())); + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SiteImpl(inner1, this.manager())); } - public void deletePacketCore( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters) { + public void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters) { this.serviceClient().deletePacketCore(resourceGroupName, mobileNetworkName, siteName, parameters); } - public void deletePacketCore( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context) { + public void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters, Context context) { this.serviceClient().deletePacketCore(resourceGroupName, mobileNetworkName, siteName, parameters, context); } public Site getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String siteName = Utils.getValueFromIdByName(id, "sites"); + String siteName = ResourceManagerUtils.getValueFromIdByName(id, "sites"); if (siteName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, siteName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String siteName = Utils.getValueFromIdByName(id, "sites"); + String siteName = ResourceManagerUtils.getValueFromIdByName(id, "sites"); if (siteName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, siteName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String siteName = Utils.getValueFromIdByName(id, "sites"); + String siteName = ResourceManagerUtils.getValueFromIdByName(id, "sites"); if (siteName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); } this.delete(resourceGroupName, mobileNetworkName, siteName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String siteName = Utils.getValueFromIdByName(id, "sites"); + String siteName = ResourceManagerUtils.getValueFromIdByName(id, "sites"); if (siteName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'sites'.", id))); } this.delete(resourceGroupName, mobileNetworkName, siteName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SliceImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SliceImpl.java index 52b33fd8e3215..b10390ef90f4f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SliceImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SliceImpl.java @@ -96,20 +96,14 @@ public SliceImpl withExistingMobileNetwork(String resourceGroupName, String mobi } public Slice create() { - this.innerObject = - serviceManager - .serviceClient() - .getSlices() - .createOrUpdate(resourceGroupName, mobileNetworkName, sliceName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getSlices().createOrUpdate(resourceGroupName, + mobileNetworkName, sliceName, this.innerModel(), Context.NONE); return this; } public Slice create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSlices() - .createOrUpdate(resourceGroupName, mobileNetworkName, sliceName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getSlices().createOrUpdate(resourceGroupName, + mobileNetworkName, sliceName, this.innerModel(), context); return this; } @@ -125,50 +119,36 @@ public SliceImpl update() { } public Slice apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSlices() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, sliceName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSlices() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, sliceName, updateParameters, Context.NONE) + .getValue(); return this; } public Slice apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSlices() - .updateTagsWithResponse(resourceGroupName, mobileNetworkName, sliceName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSlices() + .updateTagsWithResponse(resourceGroupName, mobileNetworkName, sliceName, updateParameters, context) + .getValue(); return this; } SliceImpl(SliceInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.mobileNetworkName = Utils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); - this.sliceName = Utils.getValueFromIdByName(innerObject.id(), "slices"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "mobileNetworks"); + this.sliceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "slices"); } public Slice refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSlices() - .getWithResponse(resourceGroupName, mobileNetworkName, sliceName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSlices() + .getWithResponse(resourceGroupName, mobileNetworkName, sliceName, Context.NONE).getValue(); return this; } public Slice refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSlices() - .getWithResponse(resourceGroupName, mobileNetworkName, sliceName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSlices() + .getWithResponse(resourceGroupName, mobileNetworkName, sliceName, context).getValue(); return this; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesClientImpl.java index f45a2b3828c2a..36c69fa8cafb3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesClientImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesClientImpl.java @@ -40,17 +40,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SlicesClient. */ +/** + * An instance of this class provides access to all the operations defined in SlicesClient. + */ public final class SlicesClientImpl implements SlicesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SlicesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final MobileNetworkManagementClientImpl client; /** * Initializes an instance of SlicesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SlicesClientImpl(MobileNetworkManagementClientImpl client) { @@ -65,96 +71,70 @@ public final class SlicesClientImpl implements SlicesClient { @Host("{$host}") @ServiceInterface(name = "MobileNetworkManagem") public interface SlicesService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("sliceName") String sliceName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("sliceName") String sliceName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("sliceName") String sliceName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("sliceName") String sliceName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("sliceName") String sliceName, - @BodyParam("application/json") SliceInner parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("sliceName") String sliceName, + @BodyParam("application/json") SliceInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices/{sliceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, + Mono> updateTags(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("mobileNetworkName") String mobileNetworkName, - @PathParam("sliceName") String sliceName, - @BodyParam("application/json") TagsObject parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("mobileNetworkName") String mobileNetworkName, @PathParam("sliceName") String sliceName, + @BodyParam("application/json") TagsObject parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/mobileNetworks/{mobileNetworkName}/slices") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByMobileNetwork( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> listByMobileNetwork(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("mobileNetworkName") String mobileNetworkName, - @HeaderParam("Accept") String accept, + @PathParam("mobileNetworkName") String mobileNetworkName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByMobileNetworkNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -164,19 +144,15 @@ Mono> listByMobileNetworkNext( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String sliceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -191,24 +167,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, sliceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -219,19 +185,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String sliceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -246,21 +208,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, sliceName, accept, context); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -270,19 +224,17 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String sliceName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String sliceName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -293,19 +245,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String mobileNetworkName, + String sliceName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -315,14 +266,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String sliceName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String sliceName) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, sliceName).getSyncPoller(); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -333,14 +284,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String mobileNetworkName, + String sliceName, Context context) { return this.beginDeleteAsync(resourceGroupName, mobileNetworkName, sliceName, context).getSyncPoller(); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -351,14 +302,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String sliceName) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, sliceName) - .last() + return beginDeleteAsync(resourceGroupName, mobileNetworkName, sliceName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -369,16 +319,15 @@ private Mono deleteAsync(String resourceGroupName, String mobileNetworkNam * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { - return beginDeleteAsync(resourceGroupName, mobileNetworkName, sliceName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String mobileNetworkName, String sliceName, + Context context) { + return beginDeleteAsync(resourceGroupName, mobileNetworkName, sliceName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -393,7 +342,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String sl /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -409,7 +358,7 @@ public void delete(String resourceGroupName, String mobileNetworkName, String sl /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -417,22 +366,18 @@ public void delete(String resourceGroupName, String mobileNetworkName, String sl * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified network slice along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String sliceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -447,24 +392,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, sliceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -473,22 +408,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified network slice along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String sliceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -503,21 +434,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, sliceName, accept, context); } /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -534,7 +457,7 @@ private Mono getAsync(String resourceGroupName, String mobileNetwork /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -545,14 +468,14 @@ private Mono getAsync(String resourceGroupName, String mobileNetwork * @return information about the specified network slice along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, String sliceName, + Context context) { return getWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, context).block(); } /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -568,7 +491,7 @@ public SliceInner get(String resourceGroupName, String mobileNetworkName, String /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -579,19 +502,15 @@ public SliceInner get(String resourceGroupName, String mobileNetworkName, String * @return network slice resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -611,25 +530,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, sliceName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -641,19 +550,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return network slice resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -673,22 +578,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, sliceName, parameters, accept, context); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -699,19 +595,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of network slice resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SliceInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SliceInner.class, SliceInner.class, this.client.getContext()); + private PollerFlux, SliceInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), SliceInner.class, + SliceInner.class, this.client.getContext()); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -723,20 +617,18 @@ private PollerFlux, SliceInner> beginCreateOrUpdateAsync( * @return the {@link PollerFlux} for polling of network slice resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, SliceInner> beginCreateOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { + private PollerFlux, SliceInner> beginCreateOrUpdateAsync(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), SliceInner.class, SliceInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), SliceInner.class, + SliceInner.class, context); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -747,16 +639,15 @@ private PollerFlux, SliceInner> beginCreateOrUpdateAsync( * @return the {@link SyncPoller} for polling of network slice resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SliceInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters) + public SyncPoller, SliceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters) .getSyncPoller(); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -768,16 +659,15 @@ public SyncPoller, SliceInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of network slice resource. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, SliceInner> beginCreateOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context) + public SyncPoller, SliceInner> beginCreateOrUpdate(String resourceGroupName, + String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context) .getSyncPoller(); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -788,16 +678,15 @@ public SyncPoller, SliceInner> beginCreateOrUpdate( * @return network slice resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, String sliceName, + SliceInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -809,16 +698,15 @@ private Mono createOrUpdateAsync( * @return network slice resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String mobileNetworkName, String sliceName, + SliceInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -829,14 +717,14 @@ private Mono createOrUpdateAsync( * @return network slice resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SliceInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters) { + public SliceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String sliceName, + SliceInner parameters) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters).block(); } /** * Creates or updates a network slice. Must be created in the same location as its parent mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -848,14 +736,14 @@ public SliceInner createOrUpdate( * @return network slice resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SliceInner createOrUpdate( - String resourceGroupName, String mobileNetworkName, String sliceName, SliceInner parameters, Context context) { + public SliceInner createOrUpdate(String resourceGroupName, String mobileNetworkName, String sliceName, + SliceInner parameters, Context context) { return createOrUpdateAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context).block(); } /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -866,19 +754,15 @@ public SliceInner createOrUpdate( * @return network slice resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, TagsObject parameters) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String sliceName, TagsObject parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -898,25 +782,15 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - parameters, - accept, - context)) + .withContext(context -> service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, this.client.getApiVersion(), mobileNetworkName, sliceName, parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -928,19 +802,15 @@ private Mono> updateTagsWithResponseAsync( * @return network slice resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, TagsObject parameters, Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, String mobileNetworkName, + String sliceName, TagsObject parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -960,22 +830,13 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - this.client.getApiVersion(), - mobileNetworkName, - sliceName, - parameters, - accept, - context); + return service.updateTags(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + this.client.getApiVersion(), mobileNetworkName, sliceName, parameters, accept, context); } /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -986,15 +847,15 @@ private Mono> updateTagsWithResponseAsync( * @return network slice resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String mobileNetworkName, String sliceName, TagsObject parameters) { + private Mono updateTagsAsync(String resourceGroupName, String mobileNetworkName, String sliceName, + TagsObject parameters) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, parameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -1006,15 +867,15 @@ private Mono updateTagsAsync( * @return network slice resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, String mobileNetworkName, String sliceName, TagsObject parameters, Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String mobileNetworkName, + String sliceName, TagsObject parameters, Context context) { return updateTagsWithResponseAsync(resourceGroupName, mobileNetworkName, sliceName, parameters, context) .block(); } /** * Updates slice tags. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -1025,37 +886,33 @@ public Response updateTagsWithResponse( * @return network slice resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SliceInner updateTags( - String resourceGroupName, String mobileNetworkName, String sliceName, TagsObject parameters) { + public SliceInner updateTags(String resourceGroupName, String mobileNetworkName, String sliceName, + TagsObject parameters) { return updateTagsWithResponse(resourceGroupName, mobileNetworkName, sliceName, parameters, Context.NONE) .getValue(); } /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for network slice API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1067,32 +924,16 @@ private Mono> listByMobileNetworkSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByMobileNetwork( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - mobileNetworkName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByMobileNetwork(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, mobileNetworkName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1100,22 +941,18 @@ private Mono> listByMobileNetworkSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for network slice API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByMobileNetworkSinglePageAsync( - String resourceGroupName, String mobileNetworkName, Context context) { + private Mono> listByMobileNetworkSinglePageAsync(String resourceGroupName, + String mobileNetworkName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1128,28 +965,15 @@ private Mono> listByMobileNetworkSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByMobileNetwork( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - mobileNetworkName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByMobileNetwork(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, mobileNetworkName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1159,14 +983,13 @@ private Mono> listByMobileNetworkSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink)); } /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1176,16 +999,15 @@ private PagedFlux listByMobileNetworkAsync(String resourceGroupName, * @return response for network slice API service call as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByMobileNetworkAsync( - String resourceGroupName, String mobileNetworkName, Context context) { - return new PagedFlux<>( - () -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), + private PagedFlux listByMobileNetworkAsync(String resourceGroupName, String mobileNetworkName, + Context context) { + return new PagedFlux<>(() -> listByMobileNetworkSinglePageAsync(resourceGroupName, mobileNetworkName, context), nextLink -> listByMobileNetworkNextSinglePageAsync(nextLink, context)); } /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1200,7 +1022,7 @@ public PagedIterable listByMobileNetwork(String resourceGroupName, S /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -1210,21 +1032,22 @@ public PagedIterable listByMobileNetwork(String resourceGroupName, S * @return response for network slice API service call as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { return new PagedIterable<>(listByMobileNetworkAsync(resourceGroupName, mobileNetworkName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for network slice API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink) { @@ -1232,38 +1055,30 @@ private Mono> listByMobileNetworkNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for network slice API service call along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByMobileNetworkNextSinglePageAsync(String nextLink, Context context) { @@ -1271,23 +1086,13 @@ private Mono> listByMobileNetworkNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByMobileNetworkNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesImpl.java index 85b45de2775a0..a20bc4a8eba44 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesImpl.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/SlicesImpl.java @@ -21,8 +21,8 @@ public final class SlicesImpl implements Slices { private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; - public SlicesImpl( - SlicesClient innerClient, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + public SlicesImpl(SlicesClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -35,15 +35,12 @@ public void delete(String resourceGroupName, String mobileNetworkName, String sl this.serviceClient().delete(resourceGroupName, mobileNetworkName, sliceName, context); } - public Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, sliceName, context); + public Response getWithResponse(String resourceGroupName, String mobileNetworkName, String sliceName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, mobileNetworkName, sliceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SliceImpl(inner.getValue(), this.manager())); } else { return null; @@ -60,122 +57,90 @@ public Slice get(String resourceGroupName, String mobileNetworkName, String slic } public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); - return Utils.mapPage(inner, inner1 -> new SliceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SliceImpl(inner1, this.manager())); } - public PagedIterable listByMobileNetwork( - String resourceGroupName, String mobileNetworkName, Context context) { - PagedIterable inner = - this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); - return Utils.mapPage(inner, inner1 -> new SliceImpl(inner1, this.manager())); + public PagedIterable listByMobileNetwork(String resourceGroupName, String mobileNetworkName, + Context context) { + PagedIterable inner + = this.serviceClient().listByMobileNetwork(resourceGroupName, mobileNetworkName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SliceImpl(inner1, this.manager())); } public Slice getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String sliceName = Utils.getValueFromIdByName(id, "slices"); + String sliceName = ResourceManagerUtils.getValueFromIdByName(id, "slices"); if (sliceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, sliceName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String sliceName = Utils.getValueFromIdByName(id, "slices"); + String sliceName = ResourceManagerUtils.getValueFromIdByName(id, "slices"); if (sliceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); } return this.getWithResponse(resourceGroupName, mobileNetworkName, sliceName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String sliceName = Utils.getValueFromIdByName(id, "slices"); + String sliceName = ResourceManagerUtils.getValueFromIdByName(id, "slices"); if (sliceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); } this.delete(resourceGroupName, mobileNetworkName, sliceName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String mobileNetworkName = Utils.getValueFromIdByName(id, "mobileNetworks"); + String mobileNetworkName = ResourceManagerUtils.getValueFromIdByName(id, "mobileNetworks"); if (mobileNetworkName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'mobileNetworks'.", id))); } - String sliceName = Utils.getValueFromIdByName(id, "slices"); + String sliceName = ResourceManagerUtils.getValueFromIdByName(id, "slices"); if (sliceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'slices'.", id))); } this.delete(resourceGroupName, mobileNetworkName, sliceName, context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInfoImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInfoImpl.java new file mode 100644 index 0000000000000..ec6db49c23663 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInfoImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.UeInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoPropertiesFormat; + +public final class UeInfoImpl implements UeInfo { + private UeInfoInner innerObject; + + private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; + + UeInfoImpl(UeInfoInner innerObject, com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public UeInfoPropertiesFormat properties() { + return this.innerModel().properties(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public UeInfoInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsClientImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsClientImpl.java new file mode 100644 index 0000000000000..3afa4d70f0c7c --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsClientImpl.java @@ -0,0 +1,284 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.mobilenetwork.fluent.UeInformationsClient; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoList; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in UeInformationsClient. + */ +public final class UeInformationsClientImpl implements UeInformationsClient { + /** + * The proxy service used to perform REST calls. + */ + private final UeInformationsService service; + + /** + * The service client containing this operation class. + */ + private final MobileNetworkManagementClientImpl client; + + /** + * Initializes an instance of UeInformationsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + UeInformationsClientImpl(MobileNetworkManagementClientImpl client) { + this.service + = RestProxy.create(UeInformationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for MobileNetworkManagementClientUeInformations to be used by the proxy + * service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "MobileNetworkManagem") + public interface UeInformationsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/{packetCoreControlPlaneName}/ues") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("packetCoreControlPlaneName") String packetCoreControlPlaneName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); + } + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String packetCoreControlPlaneName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (packetCoreControlPlaneName == null) { + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, packetCoreControlPlaneName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String packetCoreControlPlaneName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (packetCoreControlPlaneName == null) { + return Mono.error( + new IllegalArgumentException("Parameter packetCoreControlPlaneName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + packetCoreControlPlaneName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String packetCoreControlPlaneName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, packetCoreControlPlaneName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, packetCoreControlPlaneName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName) { + return new PagedIterable<>(listAsync(resourceGroupName, packetCoreControlPlaneName)); + } + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, packetCoreControlPlaneName, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsImpl.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsImpl.java new file mode 100644 index 0000000000000..f6c04d1674777 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/UeInformationsImpl.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.mobilenetwork.fluent.UeInformationsClient; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.UeInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeInformations; + +public final class UeInformationsImpl implements UeInformations { + private static final ClientLogger LOGGER = new ClientLogger(UeInformationsImpl.class); + + private final UeInformationsClient innerClient; + + private final com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager; + + public UeInformationsImpl(UeInformationsClient innerClient, + com.azure.resourcemanager.mobilenetwork.MobileNetworkManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, packetCoreControlPlaneName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new UeInfoImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, packetCoreControlPlaneName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new UeInfoImpl(inner1, this.manager())); + } + + private UeInformationsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/package-info.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/package-info.java index c5a0e7f3b350f..6a06af8b7f841 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/package-info.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/implementation/package-info.java @@ -3,8 +3,8 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the implementations for MobileNetworkManagementClient. The resources in this API specification - * will be used to manage attached data network resources in mobile network attached to a particular packet core - * instance. + * Package containing the implementations for MobileNetworkManagementClient. + * The resources in this API specification will be used to manage attached data network resources in mobile network + * attached to a particular packet core instance. */ package com.azure.resourcemanager.mobilenetwork.implementation; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Ambr.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Ambr.java index 5b997fe1c7671..7554fb499ce7b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Ambr.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Ambr.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Aggregate maximum bit rate. */ +/** + * Aggregate maximum bit rate. + */ @Fluent public final class Ambr { /* @@ -23,13 +25,15 @@ public final class Ambr { @JsonProperty(value = "downlink", required = true) private String downlink; - /** Creates an instance of Ambr class. */ + /** + * Creates an instance of Ambr class. + */ public Ambr() { } /** * Get the uplink property: Uplink bit rate. - * + * * @return the uplink value. */ public String uplink() { @@ -38,7 +42,7 @@ public String uplink() { /** * Set the uplink property: Uplink bit rate. - * + * * @param uplink the uplink value to set. * @return the Ambr object itself. */ @@ -49,7 +53,7 @@ public Ambr withUplink(String uplink) { /** * Get the downlink property: Downlink bit rate. - * + * * @return the downlink value. */ public String downlink() { @@ -58,7 +62,7 @@ public String downlink() { /** * Set the downlink property: Downlink bit rate. - * + * * @param downlink the downlink value to set. * @return the Ambr object itself. */ @@ -69,7 +73,7 @@ public Ambr withDownlink(String downlink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AmfId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AmfId.java new file mode 100644 index 0000000000000..a081713651acd --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AmfId.java @@ -0,0 +1,106 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * AMF identifier. + */ +@Fluent +public final class AmfId { + /* + * AMF region identifier + */ + @JsonProperty(value = "regionId", required = true) + private int regionId; + + /* + * AMF set identifier + */ + @JsonProperty(value = "setId", required = true) + private int setId; + + /* + * AMF pointer + */ + @JsonProperty(value = "pointer", required = true) + private int pointer; + + /** + * Creates an instance of AmfId class. + */ + public AmfId() { + } + + /** + * Get the regionId property: AMF region identifier. + * + * @return the regionId value. + */ + public int regionId() { + return this.regionId; + } + + /** + * Set the regionId property: AMF region identifier. + * + * @param regionId the regionId value to set. + * @return the AmfId object itself. + */ + public AmfId withRegionId(int regionId) { + this.regionId = regionId; + return this; + } + + /** + * Get the setId property: AMF set identifier. + * + * @return the setId value. + */ + public int setId() { + return this.setId; + } + + /** + * Set the setId property: AMF set identifier. + * + * @param setId the setId value to set. + * @return the AmfId object itself. + */ + public AmfId withSetId(int setId) { + this.setId = setId; + return this; + } + + /** + * Get the pointer property: AMF pointer. + * + * @return the pointer value. + */ + public int pointer() { + return this.pointer; + } + + /** + * Set the pointer property: AMF pointer. + * + * @param pointer the pointer value to set. + * @return the AmfId object itself. + */ + public AmfId withPointer(int pointer) { + this.pointer = pointer; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationId.java index b0420279bfb84..fe7d38807b1f7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to an Azure Async Operation ID. */ +/** + * Reference to an Azure Async Operation ID. + */ @Fluent public final class AsyncOperationId { /* @@ -17,13 +19,15 @@ public final class AsyncOperationId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of AsyncOperationId class. */ + /** + * Creates an instance of AsyncOperationId class. + */ public AsyncOperationId() { } /** * Get the id property: Azure Async Operation ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Azure Async Operation ID. - * + * * @param id the id value to set. * @return the AsyncOperationId object itself. */ @@ -43,14 +47,13 @@ public AsyncOperationId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model AsyncOperationId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model AsyncOperationId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationStatus.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationStatus.java index 262b0aefc1dbe..984ccc042c4de 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationStatus.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AsyncOperationStatus.java @@ -8,74 +8,76 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.AsyncOperationStatusInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of AsyncOperationStatus. */ +/** + * An immutable client-side representation of AsyncOperationStatus. + */ public interface AsyncOperationStatus { /** * Gets the id property: Fully qualified ID for the async operation. - * + * * @return the id value. */ String id(); /** * Gets the name property: Name of the async operation. - * + * * @return the name value. */ String name(); /** * Gets the status property: The operation status. - * + * * @return the status value. */ String status(); /** * Gets the resourceId property: Fully qualified ID for the resource that this async operation status relates to. - * + * * @return the resourceId value. */ String resourceId(); /** * Gets the startTime property: The start time of the operation. - * + * * @return the startTime value. */ OffsetDateTime startTime(); /** * Gets the endTime property: The end time of the operation. - * + * * @return the endTime value. */ OffsetDateTime endTime(); /** * Gets the percentComplete property: Percentage of the operation that is complete. - * + * * @return the percentComplete value. */ Double percentComplete(); /** * Gets the properties property: Properties returned by the resource provider on a successful operation. - * + * * @return the properties value. */ Object properties(); /** * Gets the error property: If present, details of the operation error. - * + * * @return the error value. */ ManagementError error(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.AsyncOperationStatusInner object. - * + * * @return the inner object. */ AsyncOperationStatusInner innerModel(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetwork.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetwork.java index 04611c6fbf6f1..6d275ab179381 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetwork.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetwork.java @@ -11,53 +11,55 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of AttachedDataNetwork. */ +/** + * An immutable client-side representation of AttachedDataNetwork. + */ public interface AttachedDataNetwork { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the attached data network resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); @@ -65,7 +67,7 @@ public interface AttachedDataNetwork { /** * Gets the userPlaneDataInterface property: The user plane interface on the data network. For 5G networks, this is * the N6 interface. For 4G networks, this is the SGi interface. - * + * * @return the userPlaneDataInterface value. */ InterfaceProperties userPlaneDataInterface(); @@ -73,90 +75,95 @@ public interface AttachedDataNetwork { /** * Gets the dnsAddresses property: The DNS servers to signal to UEs to use for this attached data network. This * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. - * + * * @return the dnsAddresses value. */ List dnsAddresses(); /** - * Gets the naptConfiguration property: The network address and port translation (NAPT) configuration. If this is - * not specified, the attached data network will use a default NAPT configuration with NAPT enabled. - * + * Gets the naptConfiguration property: The network address and port translation (NAPT) configuration. + * If this is not specified, the attached data network will use a default NAPT configuration with NAPT enabled. + * * @return the naptConfiguration value. */ NaptConfiguration naptConfiguration(); /** * Gets the userEquipmentAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the attached - * data network from which the packet core instance will dynamically assign IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. You must define at least one of - * userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you define both, they must be of the - * same size. - * + * data network from which the packet core instance will dynamically assign IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you + * define both, they must be of the same size. + * * @return the userEquipmentAddressPoolPrefix value. */ List userEquipmentAddressPoolPrefix(); /** * Gets the userEquipmentStaticAddressPoolPrefix property: The user equipment (UE) address pool prefixes for the - * attached data network from which the packet core instance will assign static IP addresses to UEs. The packet core - * instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address for a specific UE - * is set in StaticIPConfiguration on the corresponding SIM resource. At least one of userEquipmentAddressPoolPrefix - * and userEquipmentStaticAddressPoolPrefix must be defined. If both are defined, they must be of the same size. - * + * attached data network from which the packet core instance will assign static IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP address + * for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. If both + * are defined, they must be of the same size. + * * @return the userEquipmentStaticAddressPoolPrefix value. */ List userEquipmentStaticAddressPoolPrefix(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.AttachedDataNetworkInner object. - * + * * @return the inner object. */ AttachedDataNetworkInner innerModel(); - /** The entirety of the AttachedDataNetwork definition. */ + /** + * The entirety of the AttachedDataNetwork definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithUserPlaneDataInterface, - DefinitionStages.WithDnsAddresses, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithLocation, DefinitionStages.WithParentResource, + DefinitionStages.WithUserPlaneDataInterface, DefinitionStages.WithDnsAddresses, DefinitionStages.WithCreate { } - /** The AttachedDataNetwork definition stages. */ + /** + * The AttachedDataNetwork definition stages. + */ interface DefinitionStages { - /** The first stage of the AttachedDataNetwork definition. */ + /** + * The first stage of the AttachedDataNetwork definition. + */ interface Blank extends WithLocation { } - /** The stage of the AttachedDataNetwork definition allowing to specify location. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -164,49 +171,55 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the AttachedDataNetwork definition allowing to specify parent resource. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, packetCoreControlPlaneName, packetCoreDataPlaneName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. * @return the next definition stage. */ - WithUserPlaneDataInterface withExistingPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName); + WithUserPlaneDataInterface withExistingPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName); } - /** The stage of the AttachedDataNetwork definition allowing to specify userPlaneDataInterface. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify userPlaneDataInterface. + */ interface WithUserPlaneDataInterface { /** * Specifies the userPlaneDataInterface property: The user plane interface on the data network. For 5G * networks, this is the N6 interface. For 4G networks, this is the SGi interface.. - * + * * @param userPlaneDataInterface The user plane interface on the data network. For 5G networks, this is the - * N6 interface. For 4G networks, this is the SGi interface. + * N6 interface. For 4G networks, this is the SGi interface. * @return the next definition stage. */ WithDnsAddresses withUserPlaneDataInterface(InterfaceProperties userPlaneDataInterface); } - /** The stage of the AttachedDataNetwork definition allowing to specify dnsAddresses. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify dnsAddresses. + */ interface WithDnsAddresses { /** * Specifies the dnsAddresses property: The DNS servers to signal to UEs to use for this attached data * network. This configuration is mandatory - if you don't want DNS servers, you must provide an empty * array.. - * + * * @param dnsAddresses The DNS servers to signal to UEs to use for this attached data network. This - * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. + * configuration is mandatory - if you don't want DNS servers, you must provide an empty array. * @return the next definition stage. */ WithCreate withDnsAddresses(List dnsAddresses); @@ -216,87 +229,96 @@ interface WithDnsAddresses { * The stage of the AttachedDataNetwork definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithNaptConfiguration, - DefinitionStages.WithUserEquipmentAddressPoolPrefix, - DefinitionStages.WithUserEquipmentStaticAddressPoolPrefix { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithNaptConfiguration, + DefinitionStages.WithUserEquipmentAddressPoolPrefix, + DefinitionStages.WithUserEquipmentStaticAddressPoolPrefix { /** * Executes the create request. - * + * * @return the created resource. */ AttachedDataNetwork create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ AttachedDataNetwork create(Context context); } - /** The stage of the AttachedDataNetwork definition allowing to specify tags. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the AttachedDataNetwork definition allowing to specify naptConfiguration. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify naptConfiguration. + */ interface WithNaptConfiguration { /** * Specifies the naptConfiguration property: The network address and port translation (NAPT) configuration. * If this is not specified, the attached data network will use a default NAPT configuration with NAPT * enabled.. - * - * @param naptConfiguration The network address and port translation (NAPT) configuration. If this is not - * specified, the attached data network will use a default NAPT configuration with NAPT enabled. + * + * @param naptConfiguration The network address and port translation (NAPT) configuration. + * If this is not specified, the attached data network will use a default NAPT configuration with NAPT + * enabled. * @return the next definition stage. */ WithCreate withNaptConfiguration(NaptConfiguration naptConfiguration); } - /** The stage of the AttachedDataNetwork definition allowing to specify userEquipmentAddressPoolPrefix. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify userEquipmentAddressPoolPrefix. + */ interface WithUserEquipmentAddressPoolPrefix { /** * Specifies the userEquipmentAddressPoolPrefix property: The user equipment (UE) address pool prefixes for * the attached data network from which the packet core instance will dynamically assign IP addresses to - * UEs. The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. You must - * define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you - * define both, they must be of the same size.. - * + * UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. + * If you define both, they must be of the same size.. + * * @param userEquipmentAddressPoolPrefix The user equipment (UE) address pool prefixes for the attached data - * network from which the packet core instance will dynamically assign IP addresses to UEs. The packet - * core instance assigns an IP address to a UE when the UE sets up a PDU session. You must define at - * least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. If you define - * both, they must be of the same size. + * network from which the packet core instance will dynamically assign IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. + * You must define at least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix. + * If you define both, they must be of the same size. * @return the next definition stage. */ WithCreate withUserEquipmentAddressPoolPrefix(List userEquipmentAddressPoolPrefix); } - /** The stage of the AttachedDataNetwork definition allowing to specify userEquipmentStaticAddressPoolPrefix. */ + /** + * The stage of the AttachedDataNetwork definition allowing to specify userEquipmentStaticAddressPoolPrefix. + */ interface WithUserEquipmentStaticAddressPoolPrefix { /** * Specifies the userEquipmentStaticAddressPoolPrefix property: The user equipment (UE) address pool * prefixes for the attached data network from which the packet core instance will assign static IP - * addresses to UEs. The packet core instance assigns an IP address to a UE when the UE sets up a PDU - * session. The static IP address for a specific UE is set in StaticIPConfiguration on the corresponding SIM - * resource. At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be - * defined. If both are defined, they must be of the same size.. - * + * addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP + * address for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. + * If both are defined, they must be of the same size.. + * * @param userEquipmentStaticAddressPoolPrefix The user equipment (UE) address pool prefixes for the - * attached data network from which the packet core instance will assign static IP addresses to UEs. The - * packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP - * address for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. At least - * one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. If - * both are defined, they must be of the same size. + * attached data network from which the packet core instance will assign static IP addresses to UEs. + * The packet core instance assigns an IP address to a UE when the UE sets up a PDU session. The static IP + * address for a specific UE is set in StaticIPConfiguration on the corresponding SIM resource. + * At least one of userEquipmentAddressPoolPrefix and userEquipmentStaticAddressPoolPrefix must be defined. + * If both are defined, they must be of the same size. * @return the next definition stage. */ WithCreate withUserEquipmentStaticAddressPoolPrefix(List userEquipmentStaticAddressPoolPrefix); @@ -305,36 +327,42 @@ interface WithUserEquipmentStaticAddressPoolPrefix { /** * Begins update for the AttachedDataNetwork resource. - * + * * @return the stage of resource update. */ AttachedDataNetwork.Update update(); - /** The template for AttachedDataNetwork update. */ + /** + * The template for AttachedDataNetwork update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ AttachedDataNetwork apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ AttachedDataNetwork apply(Context context); } - /** The AttachedDataNetwork update stages. */ + /** + * The AttachedDataNetwork update stages. + */ interface UpdateStages { - /** The stage of the AttachedDataNetwork update allowing to specify tags. */ + /** + * The stage of the AttachedDataNetwork update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -344,14 +372,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ AttachedDataNetwork refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkListResult.java index f2375c4366743..9ff574a065872 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for attached data network API service call. */ +/** + * Response for attached data network API service call. + */ @Fluent public final class AttachedDataNetworkListResult { /* @@ -24,13 +26,15 @@ public final class AttachedDataNetworkListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AttachedDataNetworkListResult class. */ + /** + * Creates an instance of AttachedDataNetworkListResult class. + */ public AttachedDataNetworkListResult() { } /** * Get the value property: A list of data networks in a resource group. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of data networks in a resource group. - * + * * @param value the value value to set. * @return the AttachedDataNetworkListResult object itself. */ @@ -50,7 +54,7 @@ public AttachedDataNetworkListResult withValue(List va /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkResourceId.java index 043d21f68f0b1..fe9f8c5d3d010 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworkResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to an attached data network resource. */ +/** + * Reference to an attached data network resource. + */ @Fluent public final class AttachedDataNetworkResourceId { /* @@ -17,13 +19,15 @@ public final class AttachedDataNetworkResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of AttachedDataNetworkResourceId class. */ + /** + * Creates an instance of AttachedDataNetworkResourceId class. + */ public AttachedDataNetworkResourceId() { } /** * Get the id property: Attached data network resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Attached data network resource ID. - * + * * @param id the id value to set. * @return the AttachedDataNetworkResourceId object itself. */ @@ -43,15 +47,13 @@ public AttachedDataNetworkResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property id in model AttachedDataNetworkResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model AttachedDataNetworkResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworks.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworks.java index 1707ab2e488f2..aab7fdd8f8045 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworks.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AttachedDataNetworks.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of AttachedDataNetworks. */ +/** + * Resource collection API of AttachedDataNetworks. + */ public interface AttachedDataNetworks { /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -21,15 +23,12 @@ public interface AttachedDataNetworks { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName); /** * Deletes the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -39,16 +38,12 @@ void delete( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context); + void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + String attachedDataNetworkName, Context context); /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -59,16 +54,12 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified attached data network along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, - String attachedDataNetworkName, - Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, String attachedDataNetworkName, Context context); /** * Gets information about the specified attached data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -78,30 +69,27 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified attached data network. */ - AttachedDataNetwork get( - String resourceGroupName, - String packetCoreControlPlaneName, - String packetCoreDataPlaneName, + AttachedDataNetwork get(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, String attachedDataNetworkName); /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedIterable}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedIterable}. */ - PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName); + PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName); /** * Gets all the attached data networks associated with a packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -109,15 +97,15 @@ PagedIterable listByPacketCoreDataPlane( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all the attached data networks associated with a packet core data plane as paginated response with {@link - * PagedIterable}. + * @return all the attached data networks associated with a packet core data plane as paginated response with + * {@link PagedIterable}. */ - PagedIterable listByPacketCoreDataPlane( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + PagedIterable listByPacketCoreDataPlane(String resourceGroupName, + String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); /** * Gets information about the specified attached data network. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -128,7 +116,7 @@ PagedIterable listByPacketCoreDataPlane( /** * Gets information about the specified attached data network. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -140,7 +128,7 @@ PagedIterable listByPacketCoreDataPlane( /** * Deletes the specified attached data network. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -150,7 +138,7 @@ PagedIterable listByPacketCoreDataPlane( /** * Deletes the specified attached data network. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -161,7 +149,7 @@ PagedIterable listByPacketCoreDataPlane( /** * Begins definition for a new AttachedDataNetwork resource. - * + * * @param name resource name. * @return the first stage of the new AttachedDataNetwork definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AuthenticationType.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AuthenticationType.java index 030c0612bb4dd..eaf3365846f17 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AuthenticationType.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AuthenticationType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** How to authenticate users who access local diagnostics APIs. */ +/** + * How to authenticate users who access local diagnostics APIs. + */ public final class AuthenticationType extends ExpandableStringEnum { - /** Static value AAD for AuthenticationType. */ + /** + * Static value AAD for AuthenticationType. + */ public static final AuthenticationType AAD = fromString("AAD"); - /** Static value Password for AuthenticationType. */ + /** + * Static value Password for AuthenticationType. + */ public static final AuthenticationType PASSWORD = fromString("Password"); /** * Creates a new instance of AuthenticationType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AuthenticationType() { /** * Creates or finds a AuthenticationType from its string representation. - * + * * @param name a name to look for. * @return the corresponding AuthenticationType. */ @@ -38,7 +44,7 @@ public static AuthenticationType fromString(String name) { /** * Gets known AuthenticationType values. - * + * * @return known AuthenticationType values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackEdgeDeviceResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackEdgeDeviceResourceId.java index b143a49a2ba10..60ae53310b410 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackEdgeDeviceResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackEdgeDeviceResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to an Azure Stack Edge device resource. */ +/** + * Reference to an Azure Stack Edge device resource. + */ @Fluent public final class AzureStackEdgeDeviceResourceId { /* @@ -17,13 +19,15 @@ public final class AzureStackEdgeDeviceResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of AzureStackEdgeDeviceResourceId class. */ + /** + * Creates an instance of AzureStackEdgeDeviceResourceId class. + */ public AzureStackEdgeDeviceResourceId() { } /** * Get the id property: Azure Stack Edge device resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Azure Stack Edge device resource ID. - * + * * @param id the id value to set. * @return the AzureStackEdgeDeviceResourceId object itself. */ @@ -43,15 +47,13 @@ public AzureStackEdgeDeviceResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property id in model AzureStackEdgeDeviceResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model AzureStackEdgeDeviceResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackHciClusterResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackHciClusterResourceId.java index d764dfafd44af..e077dc0f5b69f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackHciClusterResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/AzureStackHciClusterResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to an Azure Stack HCI cluster resource. */ +/** + * Reference to an Azure Stack HCI cluster resource. + */ @Fluent public final class AzureStackHciClusterResourceId { /* @@ -17,13 +19,15 @@ public final class AzureStackHciClusterResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of AzureStackHciClusterResourceId class. */ + /** + * Creates an instance of AzureStackHciClusterResourceId class. + */ public AzureStackHciClusterResourceId() { } /** * Get the id property: Azure Stack HCI cluster resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Azure Stack HCI cluster resource ID. - * + * * @param id the id value to set. * @return the AzureStackHciClusterResourceId object itself. */ @@ -43,15 +47,13 @@ public AzureStackHciClusterResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property id in model AzureStackHciClusterResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model AzureStackHciClusterResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/BillingSku.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/BillingSku.java index 7ac42c30e9e8b..31661db0cee15 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/BillingSku.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/BillingSku.java @@ -13,24 +13,34 @@ * exiting SKU gets removed. */ public final class BillingSku extends ExpandableStringEnum { - /** Static value G0 for BillingSku. */ + /** + * Static value G0 for BillingSku. + */ public static final BillingSku G0 = fromString("G0"); - /** Static value G1 for BillingSku. */ + /** + * Static value G1 for BillingSku. + */ public static final BillingSku G1 = fromString("G1"); - /** Static value G2 for BillingSku. */ + /** + * Static value G2 for BillingSku. + */ public static final BillingSku G2 = fromString("G2"); - /** Static value G5 for BillingSku. */ + /** + * Static value G5 for BillingSku. + */ public static final BillingSku G5 = fromString("G5"); - /** Static value G10 for BillingSku. */ + /** + * Static value G10 for BillingSku. + */ public static final BillingSku G10 = fromString("G10"); /** * Creates a new instance of BillingSku value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +49,7 @@ public BillingSku() { /** * Creates or finds a BillingSku from its string representation. - * + * * @param name a name to look for. * @return the corresponding BillingSku. */ @@ -50,7 +60,7 @@ public static BillingSku fromString(String name) { /** * Gets known BillingSku values. - * + * * @return known BillingSku values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioning.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioning.java index 07bdc31561f93..aa3ed285c5a8d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioning.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Certificate provisioning state. */ +/** + * Certificate provisioning state. + */ @Immutable public final class CertificateProvisioning { /* @@ -22,13 +24,15 @@ public final class CertificateProvisioning { @JsonProperty(value = "reason", access = JsonProperty.Access.WRITE_ONLY) private String reason; - /** Creates an instance of CertificateProvisioning class. */ + /** + * Creates an instance of CertificateProvisioning class. + */ public CertificateProvisioning() { } /** * Get the state property: The certificate's provisioning state. - * + * * @return the state value. */ public CertificateProvisioningState state() { @@ -37,7 +41,7 @@ public CertificateProvisioningState state() { /** * Get the reason property: Reason for certificate provisioning failure. - * + * * @return the reason value. */ public String reason() { @@ -46,7 +50,7 @@ public String reason() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioningState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioningState.java index 3ba99df9c2a7f..981c32a886133 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioningState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CertificateProvisioningState.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The certificate's provisioning state. */ +/** + * The certificate's provisioning state. + */ public final class CertificateProvisioningState extends ExpandableStringEnum { - /** Static value NotProvisioned for CertificateProvisioningState. */ + /** + * Static value NotProvisioned for CertificateProvisioningState. + */ public static final CertificateProvisioningState NOT_PROVISIONED = fromString("NotProvisioned"); - /** Static value Provisioned for CertificateProvisioningState. */ + /** + * Static value Provisioned for CertificateProvisioningState. + */ public static final CertificateProvisioningState PROVISIONED = fromString("Provisioned"); - /** Static value Failed for CertificateProvisioningState. */ + /** + * Static value Failed for CertificateProvisioningState. + */ public static final CertificateProvisioningState FAILED = fromString("Failed"); /** * Creates a new instance of CertificateProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public CertificateProvisioningState() { /** * Creates or finds a CertificateProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding CertificateProvisioningState. */ @@ -41,7 +49,7 @@ public static CertificateProvisioningState fromString(String name) { /** * Gets known CertificateProvisioningState values. - * + * * @return known CertificateProvisioningState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CommonSimPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CommonSimPropertiesFormat.java index 3ea7ce98369ce..3556a706c2c6f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CommonSimPropertiesFormat.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/CommonSimPropertiesFormat.java @@ -11,7 +11,9 @@ import java.util.List; import java.util.Map; -/** Common SIM properties. */ +/** + * Common SIM properties. + */ @Fluent public class CommonSimPropertiesFormat { /* @@ -77,13 +79,15 @@ public class CommonSimPropertiesFormat { @JsonProperty(value = "vendorKeyFingerprint", access = JsonProperty.Access.WRITE_ONLY) private String vendorKeyFingerprint; - /** Creates an instance of CommonSimPropertiesFormat class. */ + /** + * Creates an instance of CommonSimPropertiesFormat class. + */ public CommonSimPropertiesFormat() { } /** * Get the provisioningState property: The provisioning state of the SIM resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -92,7 +96,7 @@ public ProvisioningState provisioningState() { /** * Get the simState property: The state of the SIM resource. - * + * * @return the simState value. */ public SimState simState() { @@ -100,8 +104,9 @@ public SimState simState() { } /** - * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that site. - * + * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that + * site. + * * @return the siteProvisioningState value. */ public Map siteProvisioningState() { @@ -111,7 +116,7 @@ public Map siteProvisioningState() { /** * Get the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @return the internationalMobileSubscriberIdentity value. */ public String internationalMobileSubscriberIdentity() { @@ -121,19 +126,19 @@ public String internationalMobileSubscriberIdentity() { /** * Set the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @param internationalMobileSubscriberIdentity the internationalMobileSubscriberIdentity value to set. * @return the CommonSimPropertiesFormat object itself. */ - public CommonSimPropertiesFormat withInternationalMobileSubscriberIdentity( - String internationalMobileSubscriberIdentity) { + public CommonSimPropertiesFormat + withInternationalMobileSubscriberIdentity(String internationalMobileSubscriberIdentity) { this.internationalMobileSubscriberIdentity = internationalMobileSubscriberIdentity; return this; } /** * Get the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @return the integratedCircuitCardIdentifier value. */ public String integratedCircuitCardIdentifier() { @@ -142,7 +147,7 @@ public String integratedCircuitCardIdentifier() { /** * Set the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @param integratedCircuitCardIdentifier the integratedCircuitCardIdentifier value to set. * @return the CommonSimPropertiesFormat object itself. */ @@ -152,10 +157,10 @@ public CommonSimPropertiesFormat withIntegratedCircuitCardIdentifier(String inte } /** - * Get the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Get the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @return the deviceType value. */ public String deviceType() { @@ -163,10 +168,10 @@ public String deviceType() { } /** - * Set the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Set the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @param deviceType the deviceType value to set. * @return the CommonSimPropertiesFormat object itself. */ @@ -178,7 +183,7 @@ public CommonSimPropertiesFormat withDeviceType(String deviceType) { /** * Get the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @return the simPolicy value. */ public SimPolicyResourceId simPolicy() { @@ -188,7 +193,7 @@ public SimPolicyResourceId simPolicy() { /** * Set the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @param simPolicy the simPolicy value to set. * @return the CommonSimPropertiesFormat object itself. */ @@ -200,7 +205,7 @@ public CommonSimPropertiesFormat withSimPolicy(SimPolicyResourceId simPolicy) { /** * Get the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @return the staticIpConfiguration value. */ public List staticIpConfiguration() { @@ -210,7 +215,7 @@ public List staticIpConfiguration() { /** * Set the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @param staticIpConfiguration the staticIpConfiguration value to set. * @return the CommonSimPropertiesFormat object itself. */ @@ -221,7 +226,7 @@ public CommonSimPropertiesFormat withStaticIpConfiguration(List tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the data network resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the description property: An optional description for this data network. - * + * * @return the description value. */ String description(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.DataNetworkInner object. - * + * * @return the inner object. */ DataNetworkInner innerModel(); - /** The entirety of the DataNetwork definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithCreate { + /** + * The entirety of the DataNetwork definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The DataNetwork definition stages. */ + /** + * The DataNetwork definition stages. + */ interface DefinitionStages { - /** The first stage of the DataNetwork definition. */ + /** + * The first stage of the DataNetwork definition. + */ interface Blank extends WithLocation { } - /** The stage of the DataNetwork definition allowing to specify location. */ + /** + * The stage of the DataNetwork definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -122,18 +129,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the DataNetwork definition allowing to specify parent resource. */ + /** + * The stage of the DataNetwork definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, mobileNetworkName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @return the next definition stage. @@ -148,36 +157,40 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithDescription { /** * Executes the create request. - * + * * @return the created resource. */ DataNetwork create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ DataNetwork create(Context context); } - /** The stage of the DataNetwork definition allowing to specify tags. */ + /** + * The stage of the DataNetwork definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the DataNetwork definition allowing to specify description. */ + /** + * The stage of the DataNetwork definition allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: An optional description for this data network.. - * + * * @param description An optional description for this data network. * @return the next definition stage. */ @@ -187,36 +200,42 @@ interface WithDescription { /** * Begins update for the DataNetwork resource. - * + * * @return the stage of resource update. */ DataNetwork.Update update(); - /** The template for DataNetwork update. */ + /** + * The template for DataNetwork update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ DataNetwork apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ DataNetwork apply(Context context); } - /** The DataNetwork update stages. */ + /** + * The DataNetwork update stages. + */ interface UpdateStages { - /** The stage of the DataNetwork update allowing to specify tags. */ + /** + * The stage of the DataNetwork update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -226,14 +245,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ DataNetwork refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkConfiguration.java index 95a8f865ac010..c071424405c16 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Settings controlling data network use. */ +/** + * Settings controlling data network use. + */ @Fluent public final class DataNetworkConfiguration { /* @@ -86,14 +88,16 @@ public final class DataNetworkConfiguration { @JsonProperty(value = "maximumNumberOfBufferedPackets") private Integer maximumNumberOfBufferedPackets; - /** Creates an instance of DataNetworkConfiguration class. */ + /** + * Creates an instance of DataNetworkConfiguration class. + */ public DataNetworkConfiguration() { } /** - * Get the dataNetwork property: A reference to the data network that these settings apply to. The data network must - * be in the same location as the SIM policy. - * + * Get the dataNetwork property: A reference to the data network that these settings apply to. The data network + * must be in the same location as the SIM policy. + * * @return the dataNetwork value. */ public DataNetworkResourceId dataNetwork() { @@ -101,9 +105,9 @@ public DataNetworkResourceId dataNetwork() { } /** - * Set the dataNetwork property: A reference to the data network that these settings apply to. The data network must - * be in the same location as the SIM policy. - * + * Set the dataNetwork property: A reference to the data network that these settings apply to. The data network + * must be in the same location as the SIM policy. + * * @param dataNetwork the dataNetwork value to set. * @return the DataNetworkConfiguration object itself. */ @@ -113,9 +117,9 @@ public DataNetworkConfiguration withDataNetwork(DataNetworkResourceId dataNetwor } /** - * Get the sessionAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of a given PDU session. See - * 3GPP TS23.501 section 5.7.2.6 for a full description of the Session-AMBR. - * + * Get the sessionAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of a given PDU session. + * See 3GPP TS23.501 section 5.7.2.6 for a full description of the Session-AMBR. + * * @return the sessionAmbr value. */ public Ambr sessionAmbr() { @@ -123,9 +127,9 @@ public Ambr sessionAmbr() { } /** - * Set the sessionAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of a given PDU session. See - * 3GPP TS23.501 section 5.7.2.6 for a full description of the Session-AMBR. - * + * Set the sessionAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of a given PDU session. + * See 3GPP TS23.501 section 5.7.2.6 for a full description of the Session-AMBR. + * * @param sessionAmbr the sessionAmbr value to set. * @return the DataNetworkConfiguration object itself. */ @@ -138,7 +142,7 @@ public DataNetworkConfiguration withSessionAmbr(Ambr sessionAmbr) { * Get the fiveQi property: Default 5G QoS Flow Indicator value. The 5QI identifies a specific QoS forwarding * treatment to be provided to a flow. See 3GPP TS23.501 section 5.7.2.1 for a full description of the 5QI * parameter, and table 5.7.4-1 for the definition the 5QI values. - * + * * @return the fiveQi value. */ public Integer fiveQi() { @@ -149,7 +153,7 @@ public Integer fiveQi() { * Set the fiveQi property: Default 5G QoS Flow Indicator value. The 5QI identifies a specific QoS forwarding * treatment to be provided to a flow. See 3GPP TS23.501 section 5.7.2.1 for a full description of the 5QI * parameter, and table 5.7.4-1 for the definition the 5QI values. - * + * * @param fiveQi the fiveQi value to set. * @return the DataNetworkConfiguration object itself. */ @@ -164,7 +168,7 @@ public DataNetworkConfiguration withFiveQi(Integer fiveQi) { * and `preemptionVulnerability` allow it. 1 is the highest level of priority. If this field is not specified then * `5qi` is used to derive the ARP value. See 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP * parameters. - * + * * @return the allocationAndRetentionPriorityLevel value. */ public Integer allocationAndRetentionPriorityLevel() { @@ -177,21 +181,21 @@ public Integer allocationAndRetentionPriorityLevel() { * and `preemptionVulnerability` allow it. 1 is the highest level of priority. If this field is not specified then * `5qi` is used to derive the ARP value. See 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP * parameters. - * + * * @param allocationAndRetentionPriorityLevel the allocationAndRetentionPriorityLevel value to set. * @return the DataNetworkConfiguration object itself. */ - public DataNetworkConfiguration withAllocationAndRetentionPriorityLevel( - Integer allocationAndRetentionPriorityLevel) { + public DataNetworkConfiguration + withAllocationAndRetentionPriorityLevel(Integer allocationAndRetentionPriorityLevel) { this.allocationAndRetentionPriorityLevel = allocationAndRetentionPriorityLevel; return this; } /** - * Get the preemptionCapability property: Default QoS Flow preemption capability. The preemption capability of a QoS - * Flow controls whether it can preempt another QoS Flow with a lower priority level. See 3GPP TS23.501 section + * Get the preemptionCapability property: Default QoS Flow preemption capability. The preemption capability of a + * QoS Flow controls whether it can preempt another QoS Flow with a lower priority level. See 3GPP TS23.501 section * 5.7.2.2 for a full description of the ARP parameters. - * + * * @return the preemptionCapability value. */ public PreemptionCapability preemptionCapability() { @@ -199,10 +203,10 @@ public PreemptionCapability preemptionCapability() { } /** - * Set the preemptionCapability property: Default QoS Flow preemption capability. The preemption capability of a QoS - * Flow controls whether it can preempt another QoS Flow with a lower priority level. See 3GPP TS23.501 section + * Set the preemptionCapability property: Default QoS Flow preemption capability. The preemption capability of a + * QoS Flow controls whether it can preempt another QoS Flow with a lower priority level. See 3GPP TS23.501 section * 5.7.2.2 for a full description of the ARP parameters. - * + * * @param preemptionCapability the preemptionCapability value to set. * @return the DataNetworkConfiguration object itself. */ @@ -212,10 +216,10 @@ public DataNetworkConfiguration withPreemptionCapability(PreemptionCapability pr } /** - * Get the preemptionVulnerability property: Default QoS Flow preemption vulnerability. The preemption vulnerability - * of a QoS Flow controls whether it can be preempted by a QoS Flow with a higher priority level. See 3GPP TS23.501 - * section 5.7.2.2 for a full description of the ARP parameters. - * + * Get the preemptionVulnerability property: Default QoS Flow preemption vulnerability. The preemption + * vulnerability of a QoS Flow controls whether it can be preempted by a QoS Flow with a higher priority level. See + * 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP parameters. + * * @return the preemptionVulnerability value. */ public PreemptionVulnerability preemptionVulnerability() { @@ -223,10 +227,10 @@ public PreemptionVulnerability preemptionVulnerability() { } /** - * Set the preemptionVulnerability property: Default QoS Flow preemption vulnerability. The preemption vulnerability - * of a QoS Flow controls whether it can be preempted by a QoS Flow with a higher priority level. See 3GPP TS23.501 - * section 5.7.2.2 for a full description of the ARP parameters. - * + * Set the preemptionVulnerability property: Default QoS Flow preemption vulnerability. The preemption + * vulnerability of a QoS Flow controls whether it can be preempted by a QoS Flow with a higher priority level. See + * 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP parameters. + * * @param preemptionVulnerability the preemptionVulnerability value to set. * @return the DataNetworkConfiguration object itself. */ @@ -238,7 +242,7 @@ public DataNetworkConfiguration withPreemptionVulnerability(PreemptionVulnerabil /** * Get the defaultSessionType property: The default PDU session type, which is used if the UE does not request a * specific session type. - * + * * @return the defaultSessionType value. */ public PduSessionType defaultSessionType() { @@ -248,7 +252,7 @@ public PduSessionType defaultSessionType() { /** * Set the defaultSessionType property: The default PDU session type, which is used if the UE does not request a * specific session type. - * + * * @param defaultSessionType the defaultSessionType value to set. * @return the DataNetworkConfiguration object itself. */ @@ -260,7 +264,7 @@ public DataNetworkConfiguration withDefaultSessionType(PduSessionType defaultSes /** * Get the additionalAllowedSessionTypes property: Allowed session types in addition to the default session type. * Must not duplicate the default session type. - * + * * @return the additionalAllowedSessionTypes value. */ public List additionalAllowedSessionTypes() { @@ -270,21 +274,21 @@ public List additionalAllowedSessionTypes() { /** * Set the additionalAllowedSessionTypes property: Allowed session types in addition to the default session type. * Must not duplicate the default session type. - * + * * @param additionalAllowedSessionTypes the additionalAllowedSessionTypes value to set. * @return the DataNetworkConfiguration object itself. */ - public DataNetworkConfiguration withAdditionalAllowedSessionTypes( - List additionalAllowedSessionTypes) { + public DataNetworkConfiguration + withAdditionalAllowedSessionTypes(List additionalAllowedSessionTypes) { this.additionalAllowedSessionTypes = additionalAllowedSessionTypes; return this; } /** - * Get the allowedServices property: List of services that can be used as part of this SIM policy. The list must not - * contain duplicate items and must contain at least one item. The services must be in the same location as the SIM - * policy. - * + * Get the allowedServices property: List of services that can be used as part of this SIM policy. The list must + * not contain duplicate items and must contain at least one item. The services must be in the same location as the + * SIM policy. + * * @return the allowedServices value. */ public List allowedServices() { @@ -292,10 +296,10 @@ public List allowedServices() { } /** - * Set the allowedServices property: List of services that can be used as part of this SIM policy. The list must not - * contain duplicate items and must contain at least one item. The services must be in the same location as the SIM - * policy. - * + * Set the allowedServices property: List of services that can be used as part of this SIM policy. The list must + * not contain duplicate items and must contain at least one item. The services must be in the same location as the + * SIM policy. + * * @param allowedServices the allowedServices value to set. * @return the DataNetworkConfiguration object itself. */ @@ -309,7 +313,7 @@ public DataNetworkConfiguration withAllowedServices(List allo * plane for High Latency Communication - Extended Buffering. See 3GPP TS29.272 v15.10.0 section 7.3.188 for a full * description. This maximum is not guaranteed because there is a internal limit on buffered packets across all PDU * sessions. - * + * * @return the maximumNumberOfBufferedPackets value. */ public Integer maximumNumberOfBufferedPackets() { @@ -321,7 +325,7 @@ public Integer maximumNumberOfBufferedPackets() { * plane for High Latency Communication - Extended Buffering. See 3GPP TS29.272 v15.10.0 section 7.3.188 for a full * description. This maximum is not guaranteed because there is a internal limit on buffered packets across all PDU * sessions. - * + * * @param maximumNumberOfBufferedPackets the maximumNumberOfBufferedPackets value to set. * @return the DataNetworkConfiguration object itself. */ @@ -332,31 +336,25 @@ public DataNetworkConfiguration withMaximumNumberOfBufferedPackets(Integer maxim /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (dataNetwork() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property dataNetwork in model DataNetworkConfiguration")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property dataNetwork in model DataNetworkConfiguration")); } else { dataNetwork().validate(); } if (sessionAmbr() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property sessionAmbr in model DataNetworkConfiguration")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sessionAmbr in model DataNetworkConfiguration")); } else { sessionAmbr().validate(); } if (allowedServices() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property allowedServices in model DataNetworkConfiguration")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property allowedServices in model DataNetworkConfiguration")); } else { allowedServices().forEach(e -> e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkListResult.java index 35108f339966b..33612f6f3ccbd 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for data network API service call. */ +/** + * Response for data network API service call. + */ @Fluent public final class DataNetworkListResult { /* @@ -24,13 +26,15 @@ public final class DataNetworkListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DataNetworkListResult class. */ + /** + * Creates an instance of DataNetworkListResult class. + */ public DataNetworkListResult() { } /** * Get the value property: A list of data networks. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of data networks. - * + * * @param value the value value to set. * @return the DataNetworkListResult object itself. */ @@ -50,7 +54,7 @@ public DataNetworkListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkResourceId.java index 5439c458542fb..4df7a643ea7ba 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworkResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to a data network resource. */ +/** + * Reference to a data network resource. + */ @Fluent public final class DataNetworkResourceId { /* @@ -17,13 +19,15 @@ public final class DataNetworkResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of DataNetworkResourceId class. */ + /** + * Creates an instance of DataNetworkResourceId class. + */ public DataNetworkResourceId() { } /** * Get the id property: Data network resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Data network resource ID. - * + * * @param id the id value to set. * @return the DataNetworkResourceId object itself. */ @@ -43,14 +47,13 @@ public DataNetworkResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model DataNetworkResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model DataNetworkResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworks.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworks.java index 04ca55f158dc7..a8b34be2c268d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworks.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DataNetworks.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of DataNetworks. */ +/** + * Resource collection API of DataNetworks. + */ public interface DataNetworks { /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -24,7 +26,7 @@ public interface DataNetworks { /** * Deletes the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -37,7 +39,7 @@ public interface DataNetworks { /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -47,12 +49,12 @@ public interface DataNetworks { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified data network along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String dataNetworkName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String dataNetworkName, + Context context); /** * Gets information about the specified data network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param dataNetworkName The name of the data network. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Lists all data networks in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -90,7 +92,7 @@ Response getWithResponse( /** * Gets information about the specified data network. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -101,7 +103,7 @@ Response getWithResponse( /** * Gets information about the specified data network. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,7 +115,7 @@ Response getWithResponse( /** * Deletes the specified data network. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -123,7 +125,7 @@ Response getWithResponse( /** * Deletes the specified data network. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -134,7 +136,7 @@ Response getWithResponse( /** * Begins definition for a new DataNetwork resource. - * + * * @param name resource name. * @return the first stage of the new DataNetwork definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DesiredInstallationState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DesiredInstallationState.java index b8514f8864332..aab8921018045 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DesiredInstallationState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DesiredInstallationState.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The desired installation state of the packet core. */ +/** + * The desired installation state of the packet core. + */ public final class DesiredInstallationState extends ExpandableStringEnum { - /** Static value Uninstalled for DesiredInstallationState. */ + /** + * Static value Uninstalled for DesiredInstallationState. + */ public static final DesiredInstallationState UNINSTALLED = fromString("Uninstalled"); - /** Static value Installed for DesiredInstallationState. */ + /** + * Static value Installed for DesiredInstallationState. + */ public static final DesiredInstallationState INSTALLED = fromString("Installed"); /** * Creates a new instance of DesiredInstallationState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public DesiredInstallationState() { /** * Creates or finds a DesiredInstallationState from its string representation. - * + * * @param name a name to look for. * @return the corresponding DesiredInstallationState. */ @@ -38,7 +44,7 @@ public static DesiredInstallationState fromString(String name) { /** * Gets known DesiredInstallationState values. - * + * * @return known DesiredInstallationState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackage.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackage.java index d6599bf4ec7cd..752a351e70b7c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackage.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackage.java @@ -7,60 +7,62 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.mobilenetwork.fluent.models.DiagnosticsPackageInner; -/** An immutable client-side representation of DiagnosticsPackage. */ +/** + * An immutable client-side representation of DiagnosticsPackage. + */ public interface DiagnosticsPackage { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the diagnostics package resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the status property: The status of the diagnostics package collection. - * + * * @return the status value. */ DiagnosticsPackageStatus status(); /** * Gets the reason property: The reason for the current state of the diagnostics package collection. - * + * * @return the reason value. */ String reason(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.DiagnosticsPackageInner object. - * + * * @return the inner object. */ DiagnosticsPackageInner innerModel(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageListResult.java index d13439dd51175..f57cf188454b5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for diagnostics package API service call. */ +/** + * Response for diagnostics package API service call. + */ @Fluent public final class DiagnosticsPackageListResult { /* @@ -24,13 +26,15 @@ public final class DiagnosticsPackageListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DiagnosticsPackageListResult class. */ + /** + * Creates an instance of DiagnosticsPackageListResult class. + */ public DiagnosticsPackageListResult() { } /** * Get the value property: A list of diagnostics packages under a packet core control plane. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of diagnostics packages under a packet core control plane. - * + * * @param value the value value to set. * @return the DiagnosticsPackageListResult object itself. */ @@ -50,7 +54,7 @@ public DiagnosticsPackageListResult withValue(List valu /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageStatus.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageStatus.java index da6be4bf0e44c..2234bdeb6a941 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageStatus.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackageStatus.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The status of the diagnostics package collection. */ +/** + * The status of the diagnostics package collection. + */ public final class DiagnosticsPackageStatus extends ExpandableStringEnum { - /** Static value NotStarted for DiagnosticsPackageStatus. */ + /** + * Static value NotStarted for DiagnosticsPackageStatus. + */ public static final DiagnosticsPackageStatus NOT_STARTED = fromString("NotStarted"); - /** Static value Collecting for DiagnosticsPackageStatus. */ + /** + * Static value Collecting for DiagnosticsPackageStatus. + */ public static final DiagnosticsPackageStatus COLLECTING = fromString("Collecting"); - /** Static value Collected for DiagnosticsPackageStatus. */ + /** + * Static value Collected for DiagnosticsPackageStatus. + */ public static final DiagnosticsPackageStatus COLLECTED = fromString("Collected"); - /** Static value Error for DiagnosticsPackageStatus. */ + /** + * Static value Error for DiagnosticsPackageStatus. + */ public static final DiagnosticsPackageStatus ERROR = fromString("Error"); /** * Creates a new instance of DiagnosticsPackageStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public DiagnosticsPackageStatus() { /** * Creates or finds a DiagnosticsPackageStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding DiagnosticsPackageStatus. */ @@ -44,7 +54,7 @@ public static DiagnosticsPackageStatus fromString(String name) { /** * Gets known DiagnosticsPackageStatus values. - * + * * @return known DiagnosticsPackageStatus values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackages.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackages.java index f13577bfaf432..cec9bb571af5e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackages.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsPackages.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of DiagnosticsPackages. */ +/** + * Resource collection API of DiagnosticsPackages. + */ public interface DiagnosticsPackages { /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -21,12 +23,12 @@ public interface DiagnosticsPackages { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return diagnostics package resource. */ - DiagnosticsPackage createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName); + DiagnosticsPackage createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName); /** * Creates or updates a diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -36,12 +38,12 @@ DiagnosticsPackage createOrUpdate( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return diagnostics package resource. */ - DiagnosticsPackage createOrUpdate( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + DiagnosticsPackage createOrUpdate(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context); /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -51,12 +53,12 @@ DiagnosticsPackage createOrUpdate( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified diagnostics package along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String diagnosticsPackageName, Context context); /** * Gets information about the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -69,7 +71,7 @@ Response getWithResponse( /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -81,7 +83,7 @@ Response getWithResponse( /** * Deletes the specified diagnostics package. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param diagnosticsPackageName The name of the diagnostics package. @@ -90,12 +92,12 @@ Response getWithResponse( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, Context context); + void delete(String resourceGroupName, String packetCoreControlPlaneName, String diagnosticsPackageName, + Context context); /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -103,12 +105,12 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for diagnostics package API service call as paginated response with {@link PagedIterable}. */ - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); /** * Lists all the diagnostics packages under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -117,6 +119,6 @@ PagedIterable listByPacketCoreControlPlane( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for diagnostics package API service call as paginated response with {@link PagedIterable}. */ - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsUploadConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsUploadConfiguration.java index 37f81897f1cfd..f27fab35f3fff 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsUploadConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DiagnosticsUploadConfiguration.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration for uploading packet core diagnostics. */ +/** + * Configuration for uploading packet core diagnostics. + */ @Fluent public final class DiagnosticsUploadConfiguration { /* @@ -17,13 +19,15 @@ public final class DiagnosticsUploadConfiguration { @JsonProperty(value = "storageAccountContainerUrl", required = true) private String storageAccountContainerUrl; - /** Creates an instance of DiagnosticsUploadConfiguration class. */ + /** + * Creates an instance of DiagnosticsUploadConfiguration class. + */ public DiagnosticsUploadConfiguration() { } /** * Get the storageAccountContainerUrl property: The Storage Account Container URL to upload diagnostics to. - * + * * @return the storageAccountContainerUrl value. */ public String storageAccountContainerUrl() { @@ -32,7 +36,7 @@ public String storageAccountContainerUrl() { /** * Set the storageAccountContainerUrl property: The Storage Account Container URL to upload diagnostics to. - * + * * @param storageAccountContainerUrl the storageAccountContainerUrl value to set. * @return the DiagnosticsUploadConfiguration object itself. */ @@ -43,16 +47,13 @@ public DiagnosticsUploadConfiguration withStorageAccountContainerUrl(String stor /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (storageAccountContainerUrl() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property storageAccountContainerUrl in model" - + " DiagnosticsUploadConfiguration")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property storageAccountContainerUrl in model DiagnosticsUploadConfiguration")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DnnIpPair.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DnnIpPair.java new file mode 100644 index 0000000000000..79c44508fadb5 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/DnnIpPair.java @@ -0,0 +1,83 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * DNN and UE IP address. + */ +@Fluent +public final class DnnIpPair { + /* + * Data network name + */ + @JsonProperty(value = "dnn") + private String dnn; + + /* + * UE IP address + */ + @JsonProperty(value = "ueIpAddress") + private UeIpAddress ueIpAddress; + + /** + * Creates an instance of DnnIpPair class. + */ + public DnnIpPair() { + } + + /** + * Get the dnn property: Data network name. + * + * @return the dnn value. + */ + public String dnn() { + return this.dnn; + } + + /** + * Set the dnn property: Data network name. + * + * @param dnn the dnn value to set. + * @return the DnnIpPair object itself. + */ + public DnnIpPair withDnn(String dnn) { + this.dnn = dnn; + return this; + } + + /** + * Get the ueIpAddress property: UE IP address. + * + * @return the ueIpAddress value. + */ + public UeIpAddress ueIpAddress() { + return this.ueIpAddress; + } + + /** + * Set the ueIpAddress property: UE IP address. + * + * @param ueIpAddress the ueIpAddress value to set. + * @return the DnnIpPair object itself. + */ + public DnnIpPair withUeIpAddress(UeIpAddress ueIpAddress) { + this.ueIpAddress = ueIpAddress; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (ueIpAddress() != null) { + ueIpAddress().validate(); + } + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EncryptedSimUploadList.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EncryptedSimUploadList.java index ec9c661480a0f..21cc4045d9d18 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EncryptedSimUploadList.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EncryptedSimUploadList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The SIMs to upload. The SIM credentials must be encrypted. */ +/** + * The SIMs to upload. The SIM credentials must be encrypted. + */ @Fluent public final class EncryptedSimUploadList { /* @@ -49,13 +51,15 @@ public final class EncryptedSimUploadList { @JsonProperty(value = "sims", required = true) private List sims; - /** Creates an instance of EncryptedSimUploadList class. */ + /** + * Creates an instance of EncryptedSimUploadList class. + */ public EncryptedSimUploadList() { } /** * Get the version property: The upload file format version. - * + * * @return the version value. */ public int version() { @@ -64,7 +68,7 @@ public int version() { /** * Set the version property: The upload file format version. - * + * * @param version the version value to set. * @return the EncryptedSimUploadList object itself. */ @@ -76,7 +80,7 @@ public EncryptedSimUploadList withVersion(int version) { /** * Get the azureKeyIdentifier property: An identifier for the Azure SIM onboarding public key used for encrypted * upload. - * + * * @return the azureKeyIdentifier value. */ public int azureKeyIdentifier() { @@ -86,7 +90,7 @@ public int azureKeyIdentifier() { /** * Set the azureKeyIdentifier property: An identifier for the Azure SIM onboarding public key used for encrypted * upload. - * + * * @param azureKeyIdentifier the azureKeyIdentifier value to set. * @return the EncryptedSimUploadList object itself. */ @@ -98,7 +102,7 @@ public EncryptedSimUploadList withAzureKeyIdentifier(int azureKeyIdentifier) { /** * Get the vendorKeyFingerprint property: The fingerprint of the SIM vendor public key. The private counterpart is * used for signing the encrypted transport key. - * + * * @return the vendorKeyFingerprint value. */ public String vendorKeyFingerprint() { @@ -108,7 +112,7 @@ public String vendorKeyFingerprint() { /** * Set the vendorKeyFingerprint property: The fingerprint of the SIM vendor public key. The private counterpart is * used for signing the encrypted transport key. - * + * * @param vendorKeyFingerprint the vendorKeyFingerprint value to set. * @return the EncryptedSimUploadList object itself. */ @@ -120,7 +124,7 @@ public EncryptedSimUploadList withVendorKeyFingerprint(String vendorKeyFingerpri /** * Get the encryptedTransportKey property: The transport key used for encrypting SIM credentials, encrypted using * the SIM onboarding public key. - * + * * @return the encryptedTransportKey value. */ public String encryptedTransportKey() { @@ -130,7 +134,7 @@ public String encryptedTransportKey() { /** * Set the encryptedTransportKey property: The transport key used for encrypting SIM credentials, encrypted using * the SIM onboarding public key. - * + * * @param encryptedTransportKey the encryptedTransportKey value to set. * @return the EncryptedSimUploadList object itself. */ @@ -141,7 +145,7 @@ public EncryptedSimUploadList withEncryptedTransportKey(String encryptedTranspor /** * Get the signedTransportKey property: The encrypted transport key, signed using the SIM vendor private key. - * + * * @return the signedTransportKey value. */ public String signedTransportKey() { @@ -150,7 +154,7 @@ public String signedTransportKey() { /** * Set the signedTransportKey property: The encrypted transport key, signed using the SIM vendor private key. - * + * * @param signedTransportKey the signedTransportKey value to set. * @return the EncryptedSimUploadList object itself. */ @@ -161,7 +165,7 @@ public EncryptedSimUploadList withSignedTransportKey(String signedTransportKey) /** * Get the sims property: A list of SIMs to upload, with encrypted properties. - * + * * @return the sims value. */ public List sims() { @@ -170,7 +174,7 @@ public List sims() { /** * Set the sims property: A list of SIMs to upload, with encrypted properties. - * + * * @param sims the sims value to set. * @return the EncryptedSimUploadList object itself. */ @@ -181,32 +185,25 @@ public EncryptedSimUploadList withSims(List sims) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (vendorKeyFingerprint() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property vendorKeyFingerprint in model EncryptedSimUploadList")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property vendorKeyFingerprint in model EncryptedSimUploadList")); } if (encryptedTransportKey() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property encryptedTransportKey in model EncryptedSimUploadList")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property encryptedTransportKey in model EncryptedSimUploadList")); } if (signedTransportKey() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property signedTransportKey in model EncryptedSimUploadList")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property signedTransportKey in model EncryptedSimUploadList")); } if (sims() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property sims in model EncryptedSimUploadList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sims in model EncryptedSimUploadList")); } else { sims().forEach(e -> e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EventHubConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EventHubConfiguration.java index c83b71a6421cf..4c0bb69642f32 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EventHubConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/EventHubConfiguration.java @@ -8,11 +8,13 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration for sending packet core events to Azure Event Hub. */ +/** + * Configuration for sending packet core events to Azure Event Hub. + */ @Fluent public final class EventHubConfiguration { /* - * Resource ID of Azure Event Hub to send packet core events to. + * Resource ID of Azure Event Hub to send packet core events to. */ @JsonProperty(value = "id", required = true) private String id; @@ -23,13 +25,15 @@ public final class EventHubConfiguration { @JsonProperty(value = "reportingInterval") private Integer reportingInterval; - /** Creates an instance of EventHubConfiguration class. */ + /** + * Creates an instance of EventHubConfiguration class. + */ public EventHubConfiguration() { } /** * Get the id property: Resource ID of Azure Event Hub to send packet core events to. - * + * * @return the id value. */ public String id() { @@ -38,7 +42,7 @@ public String id() { /** * Set the id property: Resource ID of Azure Event Hub to send packet core events to. - * + * * @param id the id value to set. * @return the EventHubConfiguration object itself. */ @@ -49,7 +53,7 @@ public EventHubConfiguration withId(String id) { /** * Get the reportingInterval property: The duration (in seconds) between UE usage reports. - * + * * @return the reportingInterval value. */ public Integer reportingInterval() { @@ -58,7 +62,7 @@ public Integer reportingInterval() { /** * Set the reportingInterval property: The duration (in seconds) between UE usage reports. - * + * * @param reportingInterval the reportingInterval value to set. * @return the EventHubConfiguration object itself. */ @@ -69,14 +73,13 @@ public EventHubConfiguration withReportingInterval(Integer reportingInterval) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model EventHubConfiguration")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model EventHubConfiguration")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfo.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfo.java new file mode 100644 index 0000000000000..6ceebe2d700b6 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfo.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner; + +/** + * An immutable client-side representation of ExtendedUeInfo. + */ +public interface ExtendedUeInfo { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: Extended UE Information Properties. + * + * @return the properties value. + */ + ExtendedUeInfoProperties properties(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner object. + * + * @return the inner object. + */ + ExtendedUeInfoInner innerModel(); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfoProperties.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfoProperties.java new file mode 100644 index 0000000000000..565075ff32372 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInfoProperties.java @@ -0,0 +1,67 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.time.OffsetDateTime; + +/** + * Extended UE Information Properties. + */ +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "ratType", + defaultImpl = ExtendedUeInfoProperties.class) +@JsonTypeName("ExtendedUeInfoProperties") +@JsonSubTypes({ + @JsonSubTypes.Type(name = "5G", value = UeInfo5G.class), + @JsonSubTypes.Type(name = "4G", value = UeInfo4G.class) }) +@Fluent +public class ExtendedUeInfoProperties { + /* + * The timestamp of last UE info read from the packet core (UTC). + */ + @JsonProperty(value = "lastReadAt") + private OffsetDateTime lastReadAt; + + /** + * Creates an instance of ExtendedUeInfoProperties class. + */ + public ExtendedUeInfoProperties() { + } + + /** + * Get the lastReadAt property: The timestamp of last UE info read from the packet core (UTC). + * + * @return the lastReadAt value. + */ + public OffsetDateTime lastReadAt() { + return this.lastReadAt; + } + + /** + * Set the lastReadAt property: The timestamp of last UE info read from the packet core (UTC). + * + * @param lastReadAt the lastReadAt value to set. + * @return the ExtendedUeInfoProperties object itself. + */ + public ExtendedUeInfoProperties withLastReadAt(OffsetDateTime lastReadAt) { + this.lastReadAt = lastReadAt; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInformations.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInformations.java new file mode 100644 index 0000000000000..130d01097be59 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ExtendedUeInformations.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of ExtendedUeInformations. + */ +public interface ExtendedUeInformations { + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, String ueId, + Context context); + + /** + * Gets extended information about the specified UE from the packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param ueId IMSI of a UE. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return extended information about the specified UE from the packet core. + */ + ExtendedUeInfo get(String resourceGroupName, String packetCoreControlPlaneName, String ueId); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GNbId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GNbId.java new file mode 100644 index 0000000000000..97053bfab17f8 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GNbId.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * gNodeB identifier. + */ +@Fluent +public final class GNbId { + /* + * The bitLength property. + */ + @JsonProperty(value = "bitLength") + private Integer bitLength; + + /* + * The gNBValue property. + */ + @JsonProperty(value = "gNBValue") + private String gNBValue; + + /** + * Creates an instance of GNbId class. + */ + public GNbId() { + } + + /** + * Get the bitLength property: The bitLength property. + * + * @return the bitLength value. + */ + public Integer bitLength() { + return this.bitLength; + } + + /** + * Set the bitLength property: The bitLength property. + * + * @param bitLength the bitLength value to set. + * @return the GNbId object itself. + */ + public GNbId withBitLength(Integer bitLength) { + this.bitLength = bitLength; + return this; + } + + /** + * Get the gNBValue property: The gNBValue property. + * + * @return the gNBValue value. + */ + public String gNBValue() { + return this.gNBValue; + } + + /** + * Set the gNBValue property: The gNBValue property. + * + * @param gNBValue the gNBValue value to set. + * @return the GNbId object itself. + */ + public GNbId withGNBValue(String gNBValue) { + this.gNBValue = gNBValue; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GlobalRanNodeId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GlobalRanNodeId.java new file mode 100644 index 0000000000000..d72f7f6e703cd --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/GlobalRanNodeId.java @@ -0,0 +1,248 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Global RAN Node ID. + */ +@Fluent +public final class GlobalRanNodeId { + /* + * PLMN Identifier + */ + @JsonProperty(value = "plmnId", required = true) + private PlmnId plmnId; + + /* + * gNodeB identifier + */ + @JsonProperty(value = "gNbId") + private GNbId gNbId; + + /* + * NG-eNodeB identifier + */ + @JsonProperty(value = "ngeNbId") + private String ngeNbId; + + /* + * eNodeB identifier + */ + @JsonProperty(value = "eNbId") + private String eNbId; + + /* + * N3 IWF identifier + */ + @JsonProperty(value = "n3IwfId") + private String n3IwfId; + + /* + * W-AGF identifier + */ + @JsonProperty(value = "wagfId") + private String wagfId; + + /* + * TNGF identifier + */ + @JsonProperty(value = "tngfId") + private String tngfId; + + /* + * Network identifier + */ + @JsonProperty(value = "nid") + private String nid; + + /** + * Creates an instance of GlobalRanNodeId class. + */ + public GlobalRanNodeId() { + } + + /** + * Get the plmnId property: PLMN Identifier. + * + * @return the plmnId value. + */ + public PlmnId plmnId() { + return this.plmnId; + } + + /** + * Set the plmnId property: PLMN Identifier. + * + * @param plmnId the plmnId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withPlmnId(PlmnId plmnId) { + this.plmnId = plmnId; + return this; + } + + /** + * Get the gNbId property: gNodeB identifier. + * + * @return the gNbId value. + */ + public GNbId gNbId() { + return this.gNbId; + } + + /** + * Set the gNbId property: gNodeB identifier. + * + * @param gNbId the gNbId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withGNbId(GNbId gNbId) { + this.gNbId = gNbId; + return this; + } + + /** + * Get the ngeNbId property: NG-eNodeB identifier. + * + * @return the ngeNbId value. + */ + public String ngeNbId() { + return this.ngeNbId; + } + + /** + * Set the ngeNbId property: NG-eNodeB identifier. + * + * @param ngeNbId the ngeNbId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withNgeNbId(String ngeNbId) { + this.ngeNbId = ngeNbId; + return this; + } + + /** + * Get the eNbId property: eNodeB identifier. + * + * @return the eNbId value. + */ + public String eNbId() { + return this.eNbId; + } + + /** + * Set the eNbId property: eNodeB identifier. + * + * @param eNbId the eNbId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withENbId(String eNbId) { + this.eNbId = eNbId; + return this; + } + + /** + * Get the n3IwfId property: N3 IWF identifier. + * + * @return the n3IwfId value. + */ + public String n3IwfId() { + return this.n3IwfId; + } + + /** + * Set the n3IwfId property: N3 IWF identifier. + * + * @param n3IwfId the n3IwfId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withN3IwfId(String n3IwfId) { + this.n3IwfId = n3IwfId; + return this; + } + + /** + * Get the wagfId property: W-AGF identifier. + * + * @return the wagfId value. + */ + public String wagfId() { + return this.wagfId; + } + + /** + * Set the wagfId property: W-AGF identifier. + * + * @param wagfId the wagfId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withWagfId(String wagfId) { + this.wagfId = wagfId; + return this; + } + + /** + * Get the tngfId property: TNGF identifier. + * + * @return the tngfId value. + */ + public String tngfId() { + return this.tngfId; + } + + /** + * Set the tngfId property: TNGF identifier. + * + * @param tngfId the tngfId value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withTngfId(String tngfId) { + this.tngfId = tngfId; + return this; + } + + /** + * Get the nid property: Network identifier. + * + * @return the nid value. + */ + public String nid() { + return this.nid; + } + + /** + * Set the nid property: Network identifier. + * + * @param nid the nid value to set. + * @return the GlobalRanNodeId object itself. + */ + public GlobalRanNodeId withNid(String nid) { + this.nid = nid; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (plmnId() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property plmnId in model GlobalRanNodeId")); + } else { + plmnId().validate(); + } + if (gNbId() != null) { + gNbId().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(GlobalRanNodeId.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti4G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti4G.java new file mode 100644 index 0000000000000..353158fcfd3c8 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti4G.java @@ -0,0 +1,127 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Globally Unique Temporary Identifier (4G). + */ +@Fluent +public final class Guti4G { + /* + * Public land mobile network (PLMN) ID. This is made up of the mobile country code and mobile network code, as + * defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the + * values 999-99 and 999-999 can be used on internal private networks. + */ + @JsonProperty(value = "plmn", required = true) + private PlmnId plmn; + + /* + * MME identifier + */ + @JsonProperty(value = "mmeId", required = true) + private MmeId mmeId; + + /* + * MME Temporary Mobile Subscriber Identity + */ + @JsonProperty(value = "mTmsi", required = true) + private int mTmsi; + + /** + * Creates an instance of Guti4G class. + */ + public Guti4G() { + } + + /** + * Get the plmn property: Public land mobile network (PLMN) ID. This is made up of the mobile country code and + * mobile network code, as defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be + * used for testing and the values 999-99 and 999-999 can be used on internal private networks. + * + * @return the plmn value. + */ + public PlmnId plmn() { + return this.plmn; + } + + /** + * Set the plmn property: Public land mobile network (PLMN) ID. This is made up of the mobile country code and + * mobile network code, as defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be + * used for testing and the values 999-99 and 999-999 can be used on internal private networks. + * + * @param plmn the plmn value to set. + * @return the Guti4G object itself. + */ + public Guti4G withPlmn(PlmnId plmn) { + this.plmn = plmn; + return this; + } + + /** + * Get the mmeId property: MME identifier. + * + * @return the mmeId value. + */ + public MmeId mmeId() { + return this.mmeId; + } + + /** + * Set the mmeId property: MME identifier. + * + * @param mmeId the mmeId value to set. + * @return the Guti4G object itself. + */ + public Guti4G withMmeId(MmeId mmeId) { + this.mmeId = mmeId; + return this; + } + + /** + * Get the mTmsi property: MME Temporary Mobile Subscriber Identity. + * + * @return the mTmsi value. + */ + public int mTmsi() { + return this.mTmsi; + } + + /** + * Set the mTmsi property: MME Temporary Mobile Subscriber Identity. + * + * @param mTmsi the mTmsi value to set. + * @return the Guti4G object itself. + */ + public Guti4G withMTmsi(int mTmsi) { + this.mTmsi = mTmsi; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (plmn() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property plmn in model Guti4G")); + } else { + plmn().validate(); + } + if (mmeId() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property mmeId in model Guti4G")); + } else { + mmeId().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(Guti4G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti5G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti5G.java new file mode 100644 index 0000000000000..0725cad0bdcb0 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Guti5G.java @@ -0,0 +1,127 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * 5G GUTI. + */ +@Fluent +public final class Guti5G { + /* + * Public land mobile network (PLMN) ID. This is made up of the mobile country code and mobile network code, as + * defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the + * values 999-99 and 999-999 can be used on internal private networks. + */ + @JsonProperty(value = "plmn", required = true) + private PlmnId plmn; + + /* + * AMF identifier + */ + @JsonProperty(value = "amfId", required = true) + private AmfId amfId; + + /* + * 5G Temporary Mobile Subscriber Identity + */ + @JsonProperty(value = "fivegTmsi", required = true) + private int fivegTmsi; + + /** + * Creates an instance of Guti5G class. + */ + public Guti5G() { + } + + /** + * Get the plmn property: Public land mobile network (PLMN) ID. This is made up of the mobile country code and + * mobile network code, as defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be + * used for testing and the values 999-99 and 999-999 can be used on internal private networks. + * + * @return the plmn value. + */ + public PlmnId plmn() { + return this.plmn; + } + + /** + * Set the plmn property: Public land mobile network (PLMN) ID. This is made up of the mobile country code and + * mobile network code, as defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be + * used for testing and the values 999-99 and 999-999 can be used on internal private networks. + * + * @param plmn the plmn value to set. + * @return the Guti5G object itself. + */ + public Guti5G withPlmn(PlmnId plmn) { + this.plmn = plmn; + return this; + } + + /** + * Get the amfId property: AMF identifier. + * + * @return the amfId value. + */ + public AmfId amfId() { + return this.amfId; + } + + /** + * Set the amfId property: AMF identifier. + * + * @param amfId the amfId value to set. + * @return the Guti5G object itself. + */ + public Guti5G withAmfId(AmfId amfId) { + this.amfId = amfId; + return this; + } + + /** + * Get the fivegTmsi property: 5G Temporary Mobile Subscriber Identity. + * + * @return the fivegTmsi value. + */ + public int fivegTmsi() { + return this.fivegTmsi; + } + + /** + * Set the fivegTmsi property: 5G Temporary Mobile Subscriber Identity. + * + * @param fivegTmsi the fivegTmsi value to set. + * @return the Guti5G object itself. + */ + public Guti5G withFivegTmsi(int fivegTmsi) { + this.fivegTmsi = fivegTmsi; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (plmn() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property plmn in model Guti5G")); + } else { + plmn().validate(); + } + if (amfId() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property amfId in model Guti5G")); + } else { + amfId().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(Guti5G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioning.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioning.java new file mode 100644 index 0000000000000..b6ce96903b531 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioning.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The HomeNetworkPrivateKeysProvisioning model. + */ +@Immutable +public final class HomeNetworkPrivateKeysProvisioning { + /* + * The provisioning state of the private keys for SUPI concealment. + */ + @JsonProperty(value = "state", required = true, access = JsonProperty.Access.WRITE_ONLY) + private HomeNetworkPrivateKeysProvisioningState state; + + /** + * Creates an instance of HomeNetworkPrivateKeysProvisioning class. + */ + public HomeNetworkPrivateKeysProvisioning() { + } + + /** + * Get the state property: The provisioning state of the private keys for SUPI concealment. + * + * @return the state value. + */ + public HomeNetworkPrivateKeysProvisioningState state() { + return this.state; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioningState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioningState.java new file mode 100644 index 0000000000000..eb82635abadda --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPrivateKeysProvisioningState.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The provisioning state of the private keys for SUPI concealment. + */ +public final class HomeNetworkPrivateKeysProvisioningState + extends ExpandableStringEnum { + /** + * Static value NotProvisioned for HomeNetworkPrivateKeysProvisioningState. + */ + public static final HomeNetworkPrivateKeysProvisioningState NOT_PROVISIONED = fromString("NotProvisioned"); + + /** + * Static value Provisioned for HomeNetworkPrivateKeysProvisioningState. + */ + public static final HomeNetworkPrivateKeysProvisioningState PROVISIONED = fromString("Provisioned"); + + /** + * Static value Failed for HomeNetworkPrivateKeysProvisioningState. + */ + public static final HomeNetworkPrivateKeysProvisioningState FAILED = fromString("Failed"); + + /** + * Creates a new instance of HomeNetworkPrivateKeysProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public HomeNetworkPrivateKeysProvisioningState() { + } + + /** + * Creates or finds a HomeNetworkPrivateKeysProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding HomeNetworkPrivateKeysProvisioningState. + */ + @JsonCreator + public static HomeNetworkPrivateKeysProvisioningState fromString(String name) { + return fromString(name, HomeNetworkPrivateKeysProvisioningState.class); + } + + /** + * Gets known HomeNetworkPrivateKeysProvisioningState values. + * + * @return known HomeNetworkPrivateKeysProvisioningState values. + */ + public static Collection values() { + return values(HomeNetworkPrivateKeysProvisioningState.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPublicKey.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPublicKey.java new file mode 100644 index 0000000000000..d240b97abfc88 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HomeNetworkPublicKey.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * A key used for SUPI concealment. + */ +@Fluent +public final class HomeNetworkPublicKey { + /* + * The Home Network Public Key Identifier determines which public key was used to generate the SUCI sent to the + * AMF. See TS 23.003 Section 2.2B Section 5. + */ + @JsonProperty(value = "id", required = true) + private int id; + + /* + * The URL of Azure Key Vault secret containing the private key, versioned or unversioned. For example: + * https://contosovault.vault.azure.net/secrets/mySuciPrivateKey/562a4bb76b524a1493a6afe8e536ee78. + */ + @JsonProperty(value = "url") + private String url; + + /** + * Creates an instance of HomeNetworkPublicKey class. + */ + public HomeNetworkPublicKey() { + } + + /** + * Get the id property: The Home Network Public Key Identifier determines which public key was used to generate the + * SUCI sent to the AMF. See TS 23.003 Section 2.2B Section 5. + * + * @return the id value. + */ + public int id() { + return this.id; + } + + /** + * Set the id property: The Home Network Public Key Identifier determines which public key was used to generate the + * SUCI sent to the AMF. See TS 23.003 Section 2.2B Section 5. + * + * @param id the id value to set. + * @return the HomeNetworkPublicKey object itself. + */ + public HomeNetworkPublicKey withId(int id) { + this.id = id; + return this; + } + + /** + * Get the url property: The URL of Azure Key Vault secret containing the private key, versioned or unversioned. + * For example: https://contosovault.vault.azure.net/secrets/mySuciPrivateKey/562a4bb76b524a1493a6afe8e536ee78. + * + * @return the url value. + */ + public String url() { + return this.url; + } + + /** + * Set the url property: The URL of Azure Key Vault secret containing the private key, versioned or unversioned. + * For example: https://contosovault.vault.azure.net/secrets/mySuciPrivateKey/562a4bb76b524a1493a6afe8e536ee78. + * + * @param url the url value to set. + * @return the HomeNetworkPublicKey object itself. + */ + public HomeNetworkPublicKey withUrl(String url) { + this.url = url; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HttpsServerCertificate.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HttpsServerCertificate.java index 3e0270c295e9e..739831bfbc113 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HttpsServerCertificate.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/HttpsServerCertificate.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** HTTPS server certificate configuration. */ +/** + * HTTPS server certificate configuration. + */ @Fluent public final class HttpsServerCertificate { /* @@ -23,14 +25,16 @@ public final class HttpsServerCertificate { @JsonProperty(value = "provisioning", access = JsonProperty.Access.WRITE_ONLY) private CertificateProvisioning provisioning; - /** Creates an instance of HttpsServerCertificate class. */ + /** + * Creates an instance of HttpsServerCertificate class. + */ public HttpsServerCertificate() { } /** * Get the certificateUrl property: The certificate URL, unversioned. For example: * https://contosovault.vault.azure.net/certificates/ingress. - * + * * @return the certificateUrl value. */ public String certificateUrl() { @@ -40,7 +44,7 @@ public String certificateUrl() { /** * Set the certificateUrl property: The certificate URL, unversioned. For example: * https://contosovault.vault.azure.net/certificates/ingress. - * + * * @param certificateUrl the certificateUrl value to set. * @return the HttpsServerCertificate object itself. */ @@ -51,7 +55,7 @@ public HttpsServerCertificate withCertificateUrl(String certificateUrl) { /** * Get the provisioning property: The provisioning state of the certificate. - * + * * @return the provisioning value. */ public CertificateProvisioning provisioning() { @@ -60,15 +64,13 @@ public CertificateProvisioning provisioning() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (certificateUrl() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property certificateUrl in model HttpsServerCertificate")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property certificateUrl in model HttpsServerCertificate")); } if (provisioning() != null) { provisioning().validate(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/IdentityAndTagsObject.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/IdentityAndTagsObject.java index 750ee955fc8be..0aeb5870eb351 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/IdentityAndTagsObject.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/IdentityAndTagsObject.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Identity and Tags object for patch operations. */ +/** + * Identity and Tags object for patch operations. + */ @Fluent public final class IdentityAndTagsObject { /* @@ -25,13 +27,15 @@ public final class IdentityAndTagsObject { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of IdentityAndTagsObject class. */ + /** + * Creates an instance of IdentityAndTagsObject class. + */ public IdentityAndTagsObject() { } /** * Get the identity property: The managed service identity associated with this resource. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -40,7 +44,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The managed service identity associated with this resource. - * + * * @param identity the identity value to set. * @return the IdentityAndTagsObject object itself. */ @@ -51,7 +55,7 @@ public IdentityAndTagsObject withIdentity(ManagedServiceIdentity identity) { /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -60,7 +64,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the IdentityAndTagsObject object itself. */ @@ -71,7 +75,7 @@ public IdentityAndTagsObject withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Installation.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Installation.java index 1700f0845ba44..db161c9c6ba7f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Installation.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Installation.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The installation state of the packet core. */ +/** + * The installation state of the packet core. + */ @Fluent public final class Installation { /* @@ -41,13 +43,15 @@ public final class Installation { @JsonProperty(value = "operation", access = JsonProperty.Access.WRITE_ONLY) private AsyncOperationId operation; - /** Creates an instance of Installation class. */ + /** + * Creates an instance of Installation class. + */ public Installation() { } /** * Get the desiredState property: The desired installation state. - * + * * @return the desiredState value. */ public DesiredInstallationState desiredState() { @@ -56,7 +60,7 @@ public DesiredInstallationState desiredState() { /** * Set the desiredState property: The desired installation state. - * + * * @param desiredState the desiredState value to set. * @return the Installation object itself. */ @@ -67,7 +71,7 @@ public Installation withDesiredState(DesiredInstallationState desiredState) { /** * Get the state property: Installation state. - * + * * @return the state value. */ public InstallationState state() { @@ -77,7 +81,7 @@ public InstallationState state() { /** * Get the reinstallRequired property: Whether a reinstall of the packet core is required to pick up the latest * configuration changes. - * + * * @return the reinstallRequired value. */ public ReinstallRequired reinstallRequired() { @@ -86,7 +90,7 @@ public ReinstallRequired reinstallRequired() { /** * Get the reasons property: Reason(s) for the current installation state of the packet core. - * + * * @return the reasons value. */ public List reasons() { @@ -95,7 +99,7 @@ public List reasons() { /** * Get the operation property: A reference to an in-progress installation operation. - * + * * @return the operation value. */ public AsyncOperationId operation() { @@ -104,7 +108,7 @@ public AsyncOperationId operation() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationReason.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationReason.java index 960d101a58c58..92e83f9417bc9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationReason.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationReason.java @@ -8,44 +8,64 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The reason or list of reasons why a packet core has not been installed or requires a reinstall. */ +/** + * The reason or list of reasons why a packet core has not been installed or requires a reinstall. + */ public final class InstallationReason extends ExpandableStringEnum { - /** Static value NoSlices for InstallationReason. */ + /** + * Static value NoSlices for InstallationReason. + */ public static final InstallationReason NO_SLICES = fromString("NoSlices"); - /** Static value NoPacketCoreDataPlane for InstallationReason. */ + /** + * Static value NoPacketCoreDataPlane for InstallationReason. + */ public static final InstallationReason NO_PACKET_CORE_DATA_PLANE = fromString("NoPacketCoreDataPlane"); - /** Static value NoAttachedDataNetworks for InstallationReason. */ + /** + * Static value NoAttachedDataNetworks for InstallationReason. + */ public static final InstallationReason NO_ATTACHED_DATA_NETWORKS = fromString("NoAttachedDataNetworks"); - /** Static value PublicLandMobileNetworkIdentifierHasChanged for InstallationReason. */ - public static final InstallationReason PUBLIC_LAND_MOBILE_NETWORK_IDENTIFIER_HAS_CHANGED = - fromString("PublicLandMobileNetworkIdentifierHasChanged"); + /** + * Static value PublicLandMobileNetworkIdentifierHasChanged for InstallationReason. + */ + public static final InstallationReason PUBLIC_LAND_MOBILE_NETWORK_IDENTIFIER_HAS_CHANGED + = fromString("PublicLandMobileNetworkIdentifierHasChanged"); - /** Static value ControlPlaneAccessInterfaceHasChanged for InstallationReason. */ - public static final InstallationReason CONTROL_PLANE_ACCESS_INTERFACE_HAS_CHANGED = - fromString("ControlPlaneAccessInterfaceHasChanged"); + /** + * Static value ControlPlaneAccessInterfaceHasChanged for InstallationReason. + */ + public static final InstallationReason CONTROL_PLANE_ACCESS_INTERFACE_HAS_CHANGED + = fromString("ControlPlaneAccessInterfaceHasChanged"); - /** Static value UserPlaneAccessInterfaceHasChanged for InstallationReason. */ - public static final InstallationReason USER_PLANE_ACCESS_INTERFACE_HAS_CHANGED = - fromString("UserPlaneAccessInterfaceHasChanged"); + /** + * Static value UserPlaneAccessInterfaceHasChanged for InstallationReason. + */ + public static final InstallationReason USER_PLANE_ACCESS_INTERFACE_HAS_CHANGED + = fromString("UserPlaneAccessInterfaceHasChanged"); - /** Static value UserPlaneDataInterfaceHasChanged for InstallationReason. */ - public static final InstallationReason USER_PLANE_DATA_INTERFACE_HAS_CHANGED = - fromString("UserPlaneDataInterfaceHasChanged"); + /** + * Static value UserPlaneDataInterfaceHasChanged for InstallationReason. + */ + public static final InstallationReason USER_PLANE_DATA_INTERFACE_HAS_CHANGED + = fromString("UserPlaneDataInterfaceHasChanged"); - /** Static value ControlPlaneAccessVirtualIpv4AddressesHasChanged for InstallationReason. */ - public static final InstallationReason CONTROL_PLANE_ACCESS_VIRTUAL_IPV4ADDRESSES_HAS_CHANGED = - fromString("ControlPlaneAccessVirtualIpv4AddressesHasChanged"); + /** + * Static value ControlPlaneAccessVirtualIpv4AddressesHasChanged for InstallationReason. + */ + public static final InstallationReason CONTROL_PLANE_ACCESS_VIRTUAL_IPV4ADDRESSES_HAS_CHANGED + = fromString("ControlPlaneAccessVirtualIpv4AddressesHasChanged"); - /** Static value UserPlaneAccessVirtualIpv4AddressesHasChanged for InstallationReason. */ - public static final InstallationReason USER_PLANE_ACCESS_VIRTUAL_IPV4ADDRESSES_HAS_CHANGED = - fromString("UserPlaneAccessVirtualIpv4AddressesHasChanged"); + /** + * Static value UserPlaneAccessVirtualIpv4AddressesHasChanged for InstallationReason. + */ + public static final InstallationReason USER_PLANE_ACCESS_VIRTUAL_IPV4ADDRESSES_HAS_CHANGED + = fromString("UserPlaneAccessVirtualIpv4AddressesHasChanged"); /** * Creates a new instance of InstallationReason value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -54,7 +74,7 @@ public InstallationReason() { /** * Creates or finds a InstallationReason from its string representation. - * + * * @param name a name to look for. * @return the corresponding InstallationReason. */ @@ -65,7 +85,7 @@ public static InstallationReason fromString(String name) { /** * Gets known InstallationReason values. - * + * * @return known InstallationReason values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationState.java index 73784c8db8437..c0c06a21cd8a7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InstallationState.java @@ -8,38 +8,58 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The installation state of the packet core. */ +/** + * The installation state of the packet core. + */ public final class InstallationState extends ExpandableStringEnum { - /** Static value Uninstalled for InstallationState. */ + /** + * Static value Uninstalled for InstallationState. + */ public static final InstallationState UNINSTALLED = fromString("Uninstalled"); - /** Static value Installing for InstallationState. */ + /** + * Static value Installing for InstallationState. + */ public static final InstallationState INSTALLING = fromString("Installing"); - /** Static value Installed for InstallationState. */ + /** + * Static value Installed for InstallationState. + */ public static final InstallationState INSTALLED = fromString("Installed"); - /** Static value Updating for InstallationState. */ + /** + * Static value Updating for InstallationState. + */ public static final InstallationState UPDATING = fromString("Updating"); - /** Static value Upgrading for InstallationState. */ + /** + * Static value Upgrading for InstallationState. + */ public static final InstallationState UPGRADING = fromString("Upgrading"); - /** Static value Uninstalling for InstallationState. */ + /** + * Static value Uninstalling for InstallationState. + */ public static final InstallationState UNINSTALLING = fromString("Uninstalling"); - /** Static value Reinstalling for InstallationState. */ + /** + * Static value Reinstalling for InstallationState. + */ public static final InstallationState REINSTALLING = fromString("Reinstalling"); - /** Static value RollingBack for InstallationState. */ + /** + * Static value RollingBack for InstallationState. + */ public static final InstallationState ROLLING_BACK = fromString("RollingBack"); - /** Static value Failed for InstallationState. */ + /** + * Static value Failed for InstallationState. + */ public static final InstallationState FAILED = fromString("Failed"); /** * Creates a new instance of InstallationState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -48,7 +68,7 @@ public InstallationState() { /** * Creates or finds a InstallationState from its string representation. - * + * * @param name a name to look for. * @return the corresponding InstallationState. */ @@ -59,7 +79,7 @@ public static InstallationState fromString(String name) { /** * Gets known InstallationState values. - * + * * @return known InstallationState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InterfaceProperties.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InterfaceProperties.java index 37e04f353f253..6c07eb9107f1d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InterfaceProperties.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/InterfaceProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Interface properties. */ +/** + * Interface properties. + */ @Fluent public final class InterfaceProperties { /* @@ -35,14 +37,16 @@ public final class InterfaceProperties { @JsonProperty(value = "ipv4Gateway") private String ipv4Gateway; - /** Creates an instance of InterfaceProperties class. */ + /** + * Creates an instance of InterfaceProperties class. + */ public InterfaceProperties() { } /** - * Get the name property: The logical name for this interface. This should match one of the interfaces configured on - * your Azure Stack Edge device. - * + * Get the name property: The logical name for this interface. This should match one of the interfaces configured + * on your Azure Stack Edge device. + * * @return the name value. */ public String name() { @@ -50,9 +54,9 @@ public String name() { } /** - * Set the name property: The logical name for this interface. This should match one of the interfaces configured on - * your Azure Stack Edge device. - * + * Set the name property: The logical name for this interface. This should match one of the interfaces configured + * on your Azure Stack Edge device. + * * @param name the name value to set. * @return the InterfaceProperties object itself. */ @@ -63,7 +67,7 @@ public InterfaceProperties withName(String name) { /** * Get the ipv4Address property: The IPv4 address. - * + * * @return the ipv4Address value. */ public String ipv4Address() { @@ -72,7 +76,7 @@ public String ipv4Address() { /** * Set the ipv4Address property: The IPv4 address. - * + * * @param ipv4Address the ipv4Address value to set. * @return the InterfaceProperties object itself. */ @@ -83,7 +87,7 @@ public InterfaceProperties withIpv4Address(String ipv4Address) { /** * Get the ipv4Subnet property: The IPv4 subnet. - * + * * @return the ipv4Subnet value. */ public String ipv4Subnet() { @@ -92,7 +96,7 @@ public String ipv4Subnet() { /** * Set the ipv4Subnet property: The IPv4 subnet. - * + * * @param ipv4Subnet the ipv4Subnet value to set. * @return the InterfaceProperties object itself. */ @@ -103,7 +107,7 @@ public InterfaceProperties withIpv4Subnet(String ipv4Subnet) { /** * Get the ipv4Gateway property: The default IPv4 gateway (router). - * + * * @return the ipv4Gateway value. */ public String ipv4Gateway() { @@ -112,7 +116,7 @@ public String ipv4Gateway() { /** * Set the ipv4Gateway property: The default IPv4 gateway (router). - * + * * @param ipv4Gateway the ipv4Gateway value to set. * @return the InterfaceProperties object itself. */ @@ -123,7 +127,7 @@ public InterfaceProperties withIpv4Gateway(String ipv4Gateway) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/KeyVaultKey.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/KeyVaultKey.java index a23c12886a6df..b65113ddba8c3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/KeyVaultKey.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/KeyVaultKey.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** An Azure key vault key. */ +/** + * An Azure key vault key. + */ @Fluent public final class KeyVaultKey { /* @@ -16,14 +18,16 @@ public final class KeyVaultKey { @JsonProperty(value = "keyUrl") private String keyUrl; - /** Creates an instance of KeyVaultKey class. */ + /** + * Creates an instance of KeyVaultKey class. + */ public KeyVaultKey() { } /** * Get the keyUrl property: The key URL, unversioned. For example: * https://contosovault.vault.azure.net/keys/azureKey. - * + * * @return the keyUrl value. */ public String keyUrl() { @@ -33,7 +37,7 @@ public String keyUrl() { /** * Set the keyUrl property: The key URL, unversioned. For example: * https://contosovault.vault.azure.net/keys/azureKey. - * + * * @param keyUrl the keyUrl value to set. * @return the KeyVaultKey object itself. */ @@ -44,7 +48,7 @@ public KeyVaultKey withKeyUrl(String keyUrl) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/LocalDiagnosticsAccessConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/LocalDiagnosticsAccessConfiguration.java index 71c283c7cf4a6..99c52eda152ed 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/LocalDiagnosticsAccessConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/LocalDiagnosticsAccessConfiguration.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** The kubernetes ingress configuration to control access to packet core diagnostics over local APIs. */ +/** + * The kubernetes ingress configuration to control access to packet core diagnostics over local APIs. + */ @Fluent public final class LocalDiagnosticsAccessConfiguration { /* @@ -23,13 +25,15 @@ public final class LocalDiagnosticsAccessConfiguration { @JsonProperty(value = "httpsServerCertificate") private HttpsServerCertificate httpsServerCertificate; - /** Creates an instance of LocalDiagnosticsAccessConfiguration class. */ + /** + * Creates an instance of LocalDiagnosticsAccessConfiguration class. + */ public LocalDiagnosticsAccessConfiguration() { } /** * Get the authenticationType property: How to authenticate users who access local diagnostics APIs. - * + * * @return the authenticationType value. */ public AuthenticationType authenticationType() { @@ -38,7 +42,7 @@ public AuthenticationType authenticationType() { /** * Set the authenticationType property: How to authenticate users who access local diagnostics APIs. - * + * * @param authenticationType the authenticationType value to set. * @return the LocalDiagnosticsAccessConfiguration object itself. */ @@ -50,7 +54,7 @@ public LocalDiagnosticsAccessConfiguration withAuthenticationType(Authentication /** * Get the httpsServerCertificate property: The HTTPS server TLS certificate used to secure local access to * diagnostics. - * + * * @return the httpsServerCertificate value. */ public HttpsServerCertificate httpsServerCertificate() { @@ -60,27 +64,25 @@ public HttpsServerCertificate httpsServerCertificate() { /** * Set the httpsServerCertificate property: The HTTPS server TLS certificate used to secure local access to * diagnostics. - * + * * @param httpsServerCertificate the httpsServerCertificate value to set. * @return the LocalDiagnosticsAccessConfiguration object itself. */ - public LocalDiagnosticsAccessConfiguration withHttpsServerCertificate( - HttpsServerCertificate httpsServerCertificate) { + public LocalDiagnosticsAccessConfiguration + withHttpsServerCertificate(HttpsServerCertificate httpsServerCertificate) { this.httpsServerCertificate = httpsServerCertificate; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (authenticationType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property authenticationType in model LocalDiagnosticsAccessConfiguration")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property authenticationType in model LocalDiagnosticsAccessConfiguration")); } if (httpsServerCertificate() != null) { httpsServerCertificate().validate(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentity.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentity.java index 98c1b7f260486..31cc67a18fb39 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentity.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentity.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Managed service identity (User assigned identity). */ +/** + * Managed service identity (User assigned identity). + */ @Fluent public final class ManagedServiceIdentity { /* @@ -22,20 +24,23 @@ public final class ManagedServiceIdentity { /* * The set of user assigned identities associated with the resource. The userAssignedIdentities dictionary keys * will be ARM resource ids in the form: - * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. + * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/ + * userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. */ @JsonProperty(value = "userAssignedIdentities") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map userAssignedIdentities; - /** Creates an instance of ManagedServiceIdentity class. */ + /** + * Creates an instance of ManagedServiceIdentity class. + */ public ManagedServiceIdentity() { } /** * Get the type property: Type of managed service identity (currently only UserAssigned allowed). - * + * * @return the type value. */ public ManagedServiceIdentityType type() { @@ -44,7 +49,7 @@ public ManagedServiceIdentityType type() { /** * Set the type property: Type of managed service identity (currently only UserAssigned allowed). - * + * * @param type the type value to set. * @return the ManagedServiceIdentity object itself. */ @@ -58,7 +63,7 @@ public ManagedServiceIdentity withType(ManagedServiceIdentityType type) { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @return the userAssignedIdentities value. */ public Map userAssignedIdentities() { @@ -70,7 +75,7 @@ public Map userAssignedIdentities() { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @param userAssignedIdentities the userAssignedIdentities value to set. * @return the ManagedServiceIdentity object itself. */ @@ -81,24 +86,20 @@ public ManagedServiceIdentity withUserAssignedIdentities(Map { - if (e != null) { - e.validate(); - } - }); + userAssignedIdentities().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentityType.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentityType.java index 51c1ae4ccab88..e3228954bf944 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentityType.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ManagedServiceIdentityType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Type of managed service identity (currently only UserAssigned allowed). */ +/** + * Type of managed service identity (currently only UserAssigned allowed). + */ public final class ManagedServiceIdentityType extends ExpandableStringEnum { - /** Static value None for ManagedServiceIdentityType. */ + /** + * Static value None for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType NONE = fromString("None"); - /** Static value UserAssigned for ManagedServiceIdentityType. */ + /** + * Static value UserAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType USER_ASSIGNED = fromString("UserAssigned"); /** * Creates a new instance of ManagedServiceIdentityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ManagedServiceIdentityType() { /** * Creates or finds a ManagedServiceIdentityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ManagedServiceIdentityType. */ @@ -38,7 +44,7 @@ public static ManagedServiceIdentityType fromString(String name) { /** * Gets known ManagedServiceIdentityType values. - * + * * @return known ManagedServiceIdentityType values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MmeId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MmeId.java new file mode 100644 index 0000000000000..9b2b4e62e4e72 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MmeId.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * MME identifier. + */ +@Fluent +public final class MmeId { + /* + * MME group identifier + */ + @JsonProperty(value = "groupId", required = true) + private int groupId; + + /* + * MME code + */ + @JsonProperty(value = "code", required = true) + private int code; + + /** + * Creates an instance of MmeId class. + */ + public MmeId() { + } + + /** + * Get the groupId property: MME group identifier. + * + * @return the groupId value. + */ + public int groupId() { + return this.groupId; + } + + /** + * Set the groupId property: MME group identifier. + * + * @param groupId the groupId value to set. + * @return the MmeId object itself. + */ + public MmeId withGroupId(int groupId) { + this.groupId = groupId; + return this; + } + + /** + * Get the code property: MME code. + * + * @return the code value. + */ + public int code() { + return this.code; + } + + /** + * Set the code property: MME code. + * + * @param code the code value to set. + * @return the MmeId object itself. + */ + public MmeId withCode(int code) { + this.code = code; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetwork.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetwork.java index a866ce05f67ce..345057167c52b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetwork.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetwork.java @@ -8,124 +8,148 @@ import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.mobilenetwork.fluent.models.MobileNetworkInner; +import java.util.List; import java.util.Map; -/** An immutable client-side representation of MobileNetwork. */ +/** + * An immutable client-side representation of MobileNetwork. + */ public interface MobileNetwork { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); + /** + * Gets the identity property: The identity used to retrieve any private keys used for SUPI concealment from Azure + * key vault. + * + * @return the identity value. + */ + ManagedServiceIdentity identity(); + /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the mobile network resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the publicLandMobileNetworkIdentifier property: The unique public land mobile network identifier for the - * network. This is made up of the mobile country code and mobile network code, as defined in - * https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the values 999-99 - * and 999-999 can be used on internal private networks. - * + * network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * * @return the publicLandMobileNetworkIdentifier value. */ PlmnId publicLandMobileNetworkIdentifier(); + /** + * Gets the publicLandMobileNetworks property: A list of public land mobile networks including their identifiers. If + * both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * + * @return the publicLandMobileNetworks value. + */ + List publicLandMobileNetworks(); + /** * Gets the serviceKey property: The mobile network resource identifier. - * + * * @return the serviceKey value. */ String serviceKey(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.MobileNetworkInner object. - * + * * @return the inner object. */ MobileNetworkInner innerModel(); - /** The entirety of the MobileNetwork definition. */ + /** + * The entirety of the MobileNetwork definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithPublicLandMobileNetworkIdentifier, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithLocation, DefinitionStages.WithResourceGroup, + DefinitionStages.WithPublicLandMobileNetworkIdentifier, DefinitionStages.WithCreate { } - /** The MobileNetwork definition stages. */ + /** + * The MobileNetwork definition stages. + */ interface DefinitionStages { - /** The first stage of the MobileNetwork definition. */ + /** + * The first stage of the MobileNetwork definition. + */ interface Blank extends WithLocation { } - /** The stage of the MobileNetwork definition allowing to specify location. */ + /** + * The stage of the MobileNetwork definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -133,36 +157,38 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the MobileNetwork definition allowing to specify parent resource. */ + /** + * The stage of the MobileNetwork definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ WithPublicLandMobileNetworkIdentifier withExistingResourceGroup(String resourceGroupName); } - /** The stage of the MobileNetwork definition allowing to specify publicLandMobileNetworkIdentifier. */ + /** + * The stage of the MobileNetwork definition allowing to specify publicLandMobileNetworkIdentifier. + */ interface WithPublicLandMobileNetworkIdentifier { /** * Specifies the publicLandMobileNetworkIdentifier property: The unique public land mobile network - * identifier for the network. This is made up of the mobile country code and mobile network code, as - * defined in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and - * the values 999-99 and 999-999 can be used on internal private networks.. - * + * identifier for the network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' + * are specified, then the 'publicLandMobileNetworks' will take precedence.. + * * @param publicLandMobileNetworkIdentifier The unique public land mobile network identifier for the - * network. This is made up of the mobile country code and mobile network code, as defined in - * https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the - * values 999-99 and 999-999 can be used on internal private networks. + * network. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then + * the 'publicLandMobileNetworks' will take precedence. * @return the next definition stage. */ WithCreate withPublicLandMobileNetworkIdentifier(PlmnId publicLandMobileNetworkIdentifier); @@ -172,84 +198,138 @@ interface WithPublicLandMobileNetworkIdentifier { * The stage of the MobileNetwork definition which contains all the minimum required properties for the resource * to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate extends DefinitionStages.WithTags { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithPublicLandMobileNetworks { /** * Executes the create request. - * + * * @return the created resource. */ MobileNetwork create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ MobileNetwork create(Context context); } - /** The stage of the MobileNetwork definition allowing to specify tags. */ + /** + * The stage of the MobileNetwork definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } + + /** + * The stage of the MobileNetwork definition allowing to specify identity. + */ + interface WithIdentity { + /** + * Specifies the identity property: The identity used to retrieve any private keys used for SUPI concealment + * from Azure key vault.. + * + * @param identity The identity used to retrieve any private keys used for SUPI concealment from Azure key + * vault. + * @return the next definition stage. + */ + WithCreate withIdentity(ManagedServiceIdentity identity); + } + + /** + * The stage of the MobileNetwork definition allowing to specify publicLandMobileNetworks. + */ + interface WithPublicLandMobileNetworks { + /** + * Specifies the publicLandMobileNetworks property: A list of public land mobile networks including their + * identifiers. If both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, + * then the 'publicLandMobileNetworks' will take precedence.. + * + * @param publicLandMobileNetworks A list of public land mobile networks including their identifiers. If + * both 'publicLandMobileNetworks' and 'publicLandMobileNetworkIdentifier' are specified, then the + * 'publicLandMobileNetworks' will take precedence. + * @return the next definition stage. + */ + WithCreate withPublicLandMobileNetworks(List publicLandMobileNetworks); + } } /** * Begins update for the MobileNetwork resource. - * + * * @return the stage of resource update. */ MobileNetwork.Update update(); - /** The template for MobileNetwork update. */ - interface Update extends UpdateStages.WithTags { + /** + * The template for MobileNetwork update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity { /** * Executes the update request. - * + * * @return the updated resource. */ MobileNetwork apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ MobileNetwork apply(Context context); } - /** The MobileNetwork update stages. */ + /** + * The MobileNetwork update stages. + */ interface UpdateStages { - /** The stage of the MobileNetwork update allowing to specify tags. */ + /** + * The stage of the MobileNetwork update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } + + /** + * The stage of the MobileNetwork update allowing to specify identity. + */ + interface WithIdentity { + /** + * Specifies the identity property: The managed service identity associated with this resource.. + * + * @param identity The managed service identity associated with this resource. + * @return the next definition stage. + */ + Update withIdentity(ManagedServiceIdentity identity); + } } /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ MobileNetwork refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkListResult.java index 3d83e4571369d..7552645b5e888 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for mobile networks API service call. */ +/** + * Response for mobile networks API service call. + */ @Fluent public final class MobileNetworkListResult { /* @@ -24,13 +26,15 @@ public final class MobileNetworkListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of MobileNetworkListResult class. */ + /** + * Creates an instance of MobileNetworkListResult class. + */ public MobileNetworkListResult() { } /** * Get the value property: A list of mobile networks in a resource group. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of mobile networks in a resource group. - * + * * @param value the value value to set. * @return the MobileNetworkListResult object itself. */ @@ -50,7 +54,7 @@ public MobileNetworkListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkResourceId.java index 8d093d0eae8ef..4a407130b9bae 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworkResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to a mobile network resource. */ +/** + * Reference to a mobile network resource. + */ @Fluent public final class MobileNetworkResourceId { /* @@ -17,13 +19,15 @@ public final class MobileNetworkResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of MobileNetworkResourceId class. */ + /** + * Creates an instance of MobileNetworkResourceId class. + */ public MobileNetworkResourceId() { } /** * Get the id property: Mobile network resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Mobile network resource ID. - * + * * @param id the id value to set. * @return the MobileNetworkResourceId object itself. */ @@ -43,14 +47,13 @@ public MobileNetworkResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model MobileNetworkResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model MobileNetworkResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworks.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworks.java index 2fdaec6262f4e..ff0f179666dba 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworks.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/MobileNetworks.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of MobileNetworks. */ +/** + * Resource collection API of MobileNetworks. + */ public interface MobileNetworks { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface MobileNetworks { /** * Deletes the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -35,7 +37,7 @@ public interface MobileNetworks { /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -44,12 +46,12 @@ public interface MobileNetworks { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified mobile network along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String mobileNetworkName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String mobileNetworkName, + Context context); /** * Gets information about the specified mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -61,7 +63,7 @@ Response getByResourceGroupWithResponse( /** * Lists all the mobile networks in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for mobile networks API service call as paginated response with {@link PagedIterable}. @@ -70,7 +72,7 @@ Response getByResourceGroupWithResponse( /** * Lists all the mobile networks in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -81,7 +83,7 @@ Response getByResourceGroupWithResponse( /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -92,7 +94,7 @@ Response getByResourceGroupWithResponse( /** * Lists all the mobile networks in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -104,7 +106,7 @@ Response getByResourceGroupWithResponse( /** * Gets information about the specified mobile network. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -115,7 +117,7 @@ Response getByResourceGroupWithResponse( /** * Gets information about the specified mobile network. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -127,7 +129,7 @@ Response getByResourceGroupWithResponse( /** * Deletes the specified mobile network. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -137,7 +139,7 @@ Response getByResourceGroupWithResponse( /** * Deletes the specified mobile network. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -148,7 +150,7 @@ Response getByResourceGroupWithResponse( /** * Begins definition for a new MobileNetwork resource. - * + * * @param name resource name. * @return the first stage of the new MobileNetwork definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptConfiguration.java index eb447fe8da512..14f38a49eda24 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The network address and port translation settings to use for the attached data network. */ +/** + * The network address and port translation settings to use for the attached data network. + */ @Fluent public final class NaptConfiguration { /* @@ -45,13 +47,15 @@ public final class NaptConfiguration { @JsonProperty(value = "pinholeTimeouts") private PinholeTimeouts pinholeTimeouts; - /** Creates an instance of NaptConfiguration class. */ + /** + * Creates an instance of NaptConfiguration class. + */ public NaptConfiguration() { } /** * Get the enabled property: Whether NAPT is enabled for connections to this attached data network. - * + * * @return the enabled value. */ public NaptEnabled enabled() { @@ -60,7 +64,7 @@ public NaptEnabled enabled() { /** * Set the enabled property: Whether NAPT is enabled for connections to this attached data network. - * + * * @param enabled the enabled value to set. * @return the NaptConfiguration object itself. */ @@ -70,10 +74,11 @@ public NaptConfiguration withEnabled(NaptEnabled enabled) { } /** - * Get the portRange property: Range of port numbers to use as translated ports on each translated address. If not - * specified and NAPT is enabled, this range defaults to 1,024 - 49,999. (Ports under 1,024 should not be used - * because these are special purpose ports reserved by IANA. Ports 50,000 and above are reserved for non-NAPT use.). - * + * Get the portRange property: Range of port numbers to use as translated ports on each translated address. + * If not specified and NAPT is enabled, this range defaults to 1,024 - 49,999. + * (Ports under 1,024 should not be used because these are special purpose ports reserved by IANA. Ports 50,000 and + * above are reserved for non-NAPT use.). + * * @return the portRange value. */ public PortRange portRange() { @@ -81,10 +86,11 @@ public PortRange portRange() { } /** - * Set the portRange property: Range of port numbers to use as translated ports on each translated address. If not - * specified and NAPT is enabled, this range defaults to 1,024 - 49,999. (Ports under 1,024 should not be used - * because these are special purpose ports reserved by IANA. Ports 50,000 and above are reserved for non-NAPT use.). - * + * Set the portRange property: Range of port numbers to use as translated ports on each translated address. + * If not specified and NAPT is enabled, this range defaults to 1,024 - 49,999. + * (Ports under 1,024 should not be used because these are special purpose ports reserved by IANA. Ports 50,000 and + * above are reserved for non-NAPT use.). + * * @param portRange the portRange value to set. * @return the NaptConfiguration object itself. */ @@ -94,9 +100,9 @@ public NaptConfiguration withPortRange(PortRange portRange) { } /** - * Get the portReuseHoldTime property: The minimum time (in seconds) that will pass before a port that was used by a - * closed pinhole can be recycled for use by another pinhole. All hold times must be at least 1 second. - * + * Get the portReuseHoldTime property: The minimum time (in seconds) that will pass before a port that was used by + * a closed pinhole can be recycled for use by another pinhole. All hold times must be at least 1 second. + * * @return the portReuseHoldTime value. */ public PortReuseHoldTimes portReuseHoldTime() { @@ -104,9 +110,9 @@ public PortReuseHoldTimes portReuseHoldTime() { } /** - * Set the portReuseHoldTime property: The minimum time (in seconds) that will pass before a port that was used by a - * closed pinhole can be recycled for use by another pinhole. All hold times must be at least 1 second. - * + * Set the portReuseHoldTime property: The minimum time (in seconds) that will pass before a port that was used by + * a closed pinhole can be recycled for use by another pinhole. All hold times must be at least 1 second. + * * @param portReuseHoldTime the portReuseHoldTime value to set. * @return the NaptConfiguration object itself. */ @@ -118,7 +124,7 @@ public NaptConfiguration withPortReuseHoldTime(PortReuseHoldTimes portReuseHoldT /** * Get the pinholeLimits property: Maximum number of UDP and TCP pinholes that can be open simultaneously on the * core interface. For 5G networks, this is the N6 interface. For 4G networks, this is the SGi interface. - * + * * @return the pinholeLimits value. */ public Integer pinholeLimits() { @@ -128,7 +134,7 @@ public Integer pinholeLimits() { /** * Set the pinholeLimits property: Maximum number of UDP and TCP pinholes that can be open simultaneously on the * core interface. For 5G networks, this is the N6 interface. For 4G networks, this is the SGi interface. - * + * * @param pinholeLimits the pinholeLimits value to set. * @return the NaptConfiguration object itself. */ @@ -138,9 +144,9 @@ public NaptConfiguration withPinholeLimits(Integer pinholeLimits) { } /** - * Get the pinholeTimeouts property: Expiry times of inactive NAPT pinholes, in seconds. All timers must be at least - * 1 second. - * + * Get the pinholeTimeouts property: Expiry times of inactive NAPT pinholes, in seconds. All timers must be at + * least 1 second. + * * @return the pinholeTimeouts value. */ public PinholeTimeouts pinholeTimeouts() { @@ -148,9 +154,9 @@ public PinholeTimeouts pinholeTimeouts() { } /** - * Set the pinholeTimeouts property: Expiry times of inactive NAPT pinholes, in seconds. All timers must be at least - * 1 second. - * + * Set the pinholeTimeouts property: Expiry times of inactive NAPT pinholes, in seconds. All timers must be at + * least 1 second. + * * @param pinholeTimeouts the pinholeTimeouts value to set. * @return the NaptConfiguration object itself. */ @@ -161,7 +167,7 @@ public NaptConfiguration withPinholeTimeouts(PinholeTimeouts pinholeTimeouts) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptEnabled.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptEnabled.java index 6ff0b942e9436..0f456811debc9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptEnabled.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NaptEnabled.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Whether network address and port translation is enabled. */ +/** + * Whether network address and port translation is enabled. + */ public final class NaptEnabled extends ExpandableStringEnum { - /** Static value Enabled for NaptEnabled. */ + /** + * Static value Enabled for NaptEnabled. + */ public static final NaptEnabled ENABLED = fromString("Enabled"); - /** Static value Disabled for NaptEnabled. */ + /** + * Static value Disabled for NaptEnabled. + */ public static final NaptEnabled DISABLED = fromString("Disabled"); /** * Creates a new instance of NaptEnabled value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public NaptEnabled() { /** * Creates or finds a NaptEnabled from its string representation. - * + * * @param name a name to look for. * @return the corresponding NaptEnabled. */ @@ -38,7 +44,7 @@ public static NaptEnabled fromString(String name) { /** * Gets known NaptEnabled values. - * + * * @return known NaptEnabled values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NasRerouteConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NasRerouteConfiguration.java index 10de2ef053ed1..9a7fbd562ea36 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NasRerouteConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/NasRerouteConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration enabling NAS reroute. */ +/** + * Configuration enabling NAS reroute. + */ @Fluent public final class NasRerouteConfiguration { /* @@ -16,14 +18,16 @@ public final class NasRerouteConfiguration { @JsonProperty(value = "macroMmeGroupId", required = true) private int macroMmeGroupId; - /** Creates an instance of NasRerouteConfiguration class. */ + /** + * Creates an instance of NasRerouteConfiguration class. + */ public NasRerouteConfiguration() { } /** - * Get the macroMmeGroupId property: The macro network's MME group ID. This is where unknown UEs are sent to via NAS - * reroute. - * + * Get the macroMmeGroupId property: The macro network's MME group ID. This is where unknown UEs are sent to via + * NAS reroute. + * * @return the macroMmeGroupId value. */ public int macroMmeGroupId() { @@ -31,9 +35,9 @@ public int macroMmeGroupId() { } /** - * Set the macroMmeGroupId property: The macro network's MME group ID. This is where unknown UEs are sent to via NAS - * reroute. - * + * Set the macroMmeGroupId property: The macro network's MME group ID. This is where unknown UEs are sent to via + * NAS reroute. + * * @param macroMmeGroupId the macroMmeGroupId value to set. * @return the NasRerouteConfiguration object itself. */ @@ -44,7 +48,7 @@ public NasRerouteConfiguration withMacroMmeGroupId(int macroMmeGroupId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ObsoleteVersion.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ObsoleteVersion.java index 5b999a0c0211f..bfe493e7dfbe5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ObsoleteVersion.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ObsoleteVersion.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether this version is obsolete. */ +/** + * Indicates whether this version is obsolete. + */ public final class ObsoleteVersion extends ExpandableStringEnum { - /** Static value Obsolete for ObsoleteVersion. */ + /** + * Static value Obsolete for ObsoleteVersion. + */ public static final ObsoleteVersion OBSOLETE = fromString("Obsolete"); - /** Static value NotObsolete for ObsoleteVersion. */ + /** + * Static value NotObsolete for ObsoleteVersion. + */ public static final ObsoleteVersion NOT_OBSOLETE = fromString("NotObsolete"); /** * Creates a new instance of ObsoleteVersion value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ObsoleteVersion() { /** * Creates or finds a ObsoleteVersion from its string representation. - * + * * @param name a name to look for. * @return the corresponding ObsoleteVersion. */ @@ -38,7 +44,7 @@ public static ObsoleteVersion fromString(String name) { /** * Gets known ObsoleteVersion values. - * + * * @return known ObsoleteVersion values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operation.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operation.java index 4820611cffdbb..2fc4cd91e1d51 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operation.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operation.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the isDataAction property: Indicates whether the operation applies to data-plane. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the name property: Operation name: {provider}/{resource}/{operation}. - * + * * @return the name value. */ String name(); /** * Gets the display property: The object that represents the operation. - * + * * @return the display value. */ OperationDisplay display(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationDisplay.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationDisplay.java index ce91ca12f94d1..47519a47c6dab 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationDisplay.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The object that represents the operation. */ +/** + * The object that represents the operation. + */ @Fluent public final class OperationDisplay { /* @@ -34,13 +36,15 @@ public final class OperationDisplay { @JsonProperty(value = "description") private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: Service provider: Microsoft.MobileNetwork. - * + * * @return the provider value. */ public String provider() { @@ -49,7 +53,7 @@ public String provider() { /** * Set the provider property: Service provider: Microsoft.MobileNetwork. - * + * * @param provider the provider value to set. * @return the OperationDisplay object itself. */ @@ -61,7 +65,7 @@ public OperationDisplay withProvider(String provider) { /** * Get the resource property: Resource on which the operation is performed: Registration definition, registration * assignment etc. - * + * * @return the resource value. */ public String resource() { @@ -71,7 +75,7 @@ public String resource() { /** * Set the resource property: Resource on which the operation is performed: Registration definition, registration * assignment etc. - * + * * @param resource the resource value to set. * @return the OperationDisplay object itself. */ @@ -82,7 +86,7 @@ public OperationDisplay withResource(String resource) { /** * Get the operation property: Operation type: Read, write, delete, etc. - * + * * @return the operation value. */ public String operation() { @@ -91,7 +95,7 @@ public String operation() { /** * Set the operation property: Operation type: Read, write, delete, etc. - * + * * @param operation the operation value to set. * @return the OperationDisplay object itself. */ @@ -102,7 +106,7 @@ public OperationDisplay withOperation(String operation) { /** * Get the description property: Description of the operation. - * + * * @return the description value. */ public String description() { @@ -111,7 +115,7 @@ public String description() { /** * Set the description property: Description of the operation. - * + * * @param description the description value to set. * @return the OperationDisplay object itself. */ @@ -122,7 +126,7 @@ public OperationDisplay withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationList.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationList.java index 95fab3006121e..58c5147d58620 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationList.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/OperationList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of the operations. */ +/** + * List of the operations. + */ @Immutable public final class OperationList { /* @@ -24,13 +26,15 @@ public final class OperationList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationList class. */ + /** + * Creates an instance of OperationList class. + */ public OperationList() { } /** * Get the value property: List of Microsoft.MobileNetwork operations. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operations.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operations.java index 17f569e98df7e..08438cffc0551 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operations.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Operations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Gets a list of the operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of the operations as paginated response with {@link PagedIterable}. @@ -20,7 +22,7 @@ public interface Operations { /** * Gets a list of the operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCapture.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCapture.java index 7d65b91da21d5..221a195d05fc7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCapture.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCapture.java @@ -10,67 +10,69 @@ import java.time.OffsetDateTime; import java.util.List; -/** An immutable client-side representation of PacketCapture. */ +/** + * An immutable client-side representation of PacketCapture. + */ public interface PacketCapture { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the packet capture session resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the status property: The status of the packet capture session. - * + * * @return the status value. */ PacketCaptureStatus status(); /** * Gets the reason property: The reason the current packet capture session state. - * + * * @return the reason value. */ String reason(); /** * Gets the captureStartTime property: The start time of the packet capture session. - * + * * @return the captureStartTime value. */ OffsetDateTime captureStartTime(); /** * Gets the networkInterfaces property: List of network interfaces to capture on. - * + * * @return the networkInterfaces value. */ List networkInterfaces(); @@ -78,62 +80,70 @@ public interface PacketCapture { /** * Gets the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes are * truncated. The default "0" means the entire packet is captured. - * + * * @return the bytesToCapturePerPacket value. */ Long bytesToCapturePerPacket(); /** * Gets the totalBytesPerSession property: Maximum size of the capture output. - * + * * @return the totalBytesPerSession value. */ Long totalBytesPerSession(); /** * Gets the timeLimitInSeconds property: Maximum duration of the capture session in seconds. - * + * * @return the timeLimitInSeconds value. */ Integer timeLimitInSeconds(); /** * Gets the outputFiles property: The list of output files of a packet capture session. - * + * * @return the outputFiles value. */ List outputFiles(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCaptureInner object. - * + * * @return the inner object. */ PacketCaptureInner innerModel(); - /** The entirety of the PacketCapture definition. */ + /** + * The entirety of the PacketCapture definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The PacketCapture definition stages. */ + /** + * The PacketCapture definition stages. + */ interface DefinitionStages { - /** The first stage of the PacketCapture definition. */ + /** + * The first stage of the PacketCapture definition. + */ interface Blank extends WithParentResource { } - /** The stage of the PacketCapture definition allowing to specify parent resource. */ + /** + * The stage of the PacketCapture definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, packetCoreControlPlaneName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @return the next definition stage. @@ -146,66 +156,72 @@ interface WithParentResource { * to be created, but also allows for any other optional properties to be specified. */ interface WithCreate - extends DefinitionStages.WithNetworkInterfaces, - DefinitionStages.WithBytesToCapturePerPacket, - DefinitionStages.WithTotalBytesPerSession, - DefinitionStages.WithTimeLimitInSeconds { + extends DefinitionStages.WithNetworkInterfaces, DefinitionStages.WithBytesToCapturePerPacket, + DefinitionStages.WithTotalBytesPerSession, DefinitionStages.WithTimeLimitInSeconds { /** * Executes the create request. - * + * * @return the created resource. */ PacketCapture create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ PacketCapture create(Context context); } - /** The stage of the PacketCapture definition allowing to specify networkInterfaces. */ + /** + * The stage of the PacketCapture definition allowing to specify networkInterfaces. + */ interface WithNetworkInterfaces { /** * Specifies the networkInterfaces property: List of network interfaces to capture on.. - * + * * @param networkInterfaces List of network interfaces to capture on. * @return the next definition stage. */ WithCreate withNetworkInterfaces(List networkInterfaces); } - /** The stage of the PacketCapture definition allowing to specify bytesToCapturePerPacket. */ + /** + * The stage of the PacketCapture definition allowing to specify bytesToCapturePerPacket. + */ interface WithBytesToCapturePerPacket { /** * Specifies the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes * are truncated. The default "0" means the entire packet is captured.. - * + * * @param bytesToCapturePerPacket Number of bytes captured per packet, the remaining bytes are truncated. - * The default "0" means the entire packet is captured. + * The default "0" means the entire packet is captured. * @return the next definition stage. */ WithCreate withBytesToCapturePerPacket(Long bytesToCapturePerPacket); } - /** The stage of the PacketCapture definition allowing to specify totalBytesPerSession. */ + /** + * The stage of the PacketCapture definition allowing to specify totalBytesPerSession. + */ interface WithTotalBytesPerSession { /** * Specifies the totalBytesPerSession property: Maximum size of the capture output.. - * + * * @param totalBytesPerSession Maximum size of the capture output. * @return the next definition stage. */ WithCreate withTotalBytesPerSession(Long totalBytesPerSession); } - /** The stage of the PacketCapture definition allowing to specify timeLimitInSeconds. */ + /** + * The stage of the PacketCapture definition allowing to specify timeLimitInSeconds. + */ interface WithTimeLimitInSeconds { /** * Specifies the timeLimitInSeconds property: Maximum duration of the capture session in seconds.. - * + * * @param timeLimitInSeconds Maximum duration of the capture session in seconds. * @return the next definition stage. */ @@ -215,75 +231,84 @@ interface WithTimeLimitInSeconds { /** * Begins update for the PacketCapture resource. - * + * * @return the stage of resource update. */ PacketCapture.Update update(); - /** The template for PacketCapture update. */ - interface Update - extends UpdateStages.WithNetworkInterfaces, - UpdateStages.WithBytesToCapturePerPacket, - UpdateStages.WithTotalBytesPerSession, - UpdateStages.WithTimeLimitInSeconds { + /** + * The template for PacketCapture update. + */ + interface Update extends UpdateStages.WithNetworkInterfaces, UpdateStages.WithBytesToCapturePerPacket, + UpdateStages.WithTotalBytesPerSession, UpdateStages.WithTimeLimitInSeconds { /** * Executes the update request. - * + * * @return the updated resource. */ PacketCapture apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ PacketCapture apply(Context context); } - /** The PacketCapture update stages. */ + /** + * The PacketCapture update stages. + */ interface UpdateStages { - /** The stage of the PacketCapture update allowing to specify networkInterfaces. */ + /** + * The stage of the PacketCapture update allowing to specify networkInterfaces. + */ interface WithNetworkInterfaces { /** * Specifies the networkInterfaces property: List of network interfaces to capture on.. - * + * * @param networkInterfaces List of network interfaces to capture on. * @return the next definition stage. */ Update withNetworkInterfaces(List networkInterfaces); } - /** The stage of the PacketCapture update allowing to specify bytesToCapturePerPacket. */ + /** + * The stage of the PacketCapture update allowing to specify bytesToCapturePerPacket. + */ interface WithBytesToCapturePerPacket { /** * Specifies the bytesToCapturePerPacket property: Number of bytes captured per packet, the remaining bytes * are truncated. The default "0" means the entire packet is captured.. - * + * * @param bytesToCapturePerPacket Number of bytes captured per packet, the remaining bytes are truncated. - * The default "0" means the entire packet is captured. + * The default "0" means the entire packet is captured. * @return the next definition stage. */ Update withBytesToCapturePerPacket(Long bytesToCapturePerPacket); } - /** The stage of the PacketCapture update allowing to specify totalBytesPerSession. */ + /** + * The stage of the PacketCapture update allowing to specify totalBytesPerSession. + */ interface WithTotalBytesPerSession { /** * Specifies the totalBytesPerSession property: Maximum size of the capture output.. - * + * * @param totalBytesPerSession Maximum size of the capture output. * @return the next definition stage. */ Update withTotalBytesPerSession(Long totalBytesPerSession); } - /** The stage of the PacketCapture update allowing to specify timeLimitInSeconds. */ + /** + * The stage of the PacketCapture update allowing to specify timeLimitInSeconds. + */ interface WithTimeLimitInSeconds { /** * Specifies the timeLimitInSeconds property: Maximum duration of the capture session in seconds.. - * + * * @param timeLimitInSeconds Maximum duration of the capture session in seconds. * @return the next definition stage. */ @@ -293,14 +318,14 @@ interface WithTimeLimitInSeconds { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ PacketCapture refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -308,7 +333,7 @@ interface WithTimeLimitInSeconds { /** * Stop a packet capture session. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. @@ -317,7 +342,7 @@ interface WithTimeLimitInSeconds { /** * Stop a packet capture session. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureListResult.java index 01a819aa1ecb5..79a26d822b58b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for packet capture API service call. */ +/** + * Response for packet capture API service call. + */ @Fluent public final class PacketCaptureListResult { /* @@ -24,13 +26,15 @@ public final class PacketCaptureListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of PacketCaptureListResult class. */ + /** + * Creates an instance of PacketCaptureListResult class. + */ public PacketCaptureListResult() { } /** * Get the value property: A list of packet capture sessions under a packet core control plane. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of packet capture sessions under a packet core control plane. - * + * * @param value the value value to set. * @return the PacketCaptureListResult object itself. */ @@ -50,7 +54,7 @@ public PacketCaptureListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureStatus.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureStatus.java index 6eb052273b248..aa60c50fb0798 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureStatus.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptureStatus.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The status of the packet capture session. */ +/** + * The status of the packet capture session. + */ public final class PacketCaptureStatus extends ExpandableStringEnum { - /** Static value NotStarted for PacketCaptureStatus. */ + /** + * Static value NotStarted for PacketCaptureStatus. + */ public static final PacketCaptureStatus NOT_STARTED = fromString("NotStarted"); - /** Static value Running for PacketCaptureStatus. */ + /** + * Static value Running for PacketCaptureStatus. + */ public static final PacketCaptureStatus RUNNING = fromString("Running"); - /** Static value Stopped for PacketCaptureStatus. */ + /** + * Static value Stopped for PacketCaptureStatus. + */ public static final PacketCaptureStatus STOPPED = fromString("Stopped"); - /** Static value Error for PacketCaptureStatus. */ + /** + * Static value Error for PacketCaptureStatus. + */ public static final PacketCaptureStatus ERROR = fromString("Error"); /** * Creates a new instance of PacketCaptureStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public PacketCaptureStatus() { /** * Creates or finds a PacketCaptureStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding PacketCaptureStatus. */ @@ -44,7 +54,7 @@ public static PacketCaptureStatus fromString(String name) { /** * Gets known PacketCaptureStatus values. - * + * * @return known PacketCaptureStatus values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptures.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptures.java index 5f981700a792a..8cb11ea79a8e8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptures.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCaptures.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of PacketCaptures. */ +/** + * Resource collection API of PacketCaptures. + */ public interface PacketCaptures { /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -22,12 +24,12 @@ public interface PacketCaptures { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet capture session along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCaptureName, Context context); /** * Gets information about the specified packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -40,7 +42,7 @@ Response getWithResponse( /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -52,7 +54,7 @@ Response getWithResponse( /** * Deletes the specified packet capture. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -78,7 +80,7 @@ Response getWithResponse( /** * Stop a packet capture session. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCaptureName The name of the packet capture session. @@ -88,12 +90,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus stop( - String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, Context context); + AsyncOperationStatus stop(String resourceGroupName, String packetCoreControlPlaneName, String packetCaptureName, + Context context); /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -101,12 +103,12 @@ AsyncOperationStatus stop( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet capture API service call as paginated response with {@link PagedIterable}. */ - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); /** * Lists all the packet capture sessions under a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -115,12 +117,12 @@ PagedIterable listByPacketCoreControlPlane( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet capture API service call as paginated response with {@link PagedIterable}. */ - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context); /** * Gets information about the specified packet capture session. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -131,7 +133,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Gets information about the specified packet capture session. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -143,7 +145,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Deletes the specified packet capture. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -153,7 +155,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Deletes the specified packet capture. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -164,7 +166,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Begins definition for a new PacketCapture resource. - * + * * @param name resource name. * @return the first stage of the new PacketCapture definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlane.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlane.java index 49177a8fee825..d4f94e9cb9a8a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlane.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlane.java @@ -11,67 +11,69 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of PacketCoreControlPlane. */ +/** + * An immutable client-side representation of PacketCoreControlPlane. + */ public interface PacketCoreControlPlane { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the identity property: The identity used to retrieve the ingress certificate from Azure key vault. - * + * * @return the identity value. */ ManagedServiceIdentity identity(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the packet core control plane resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the installation property: The installation state of the packet core control plane resource. - * + * * @return the installation value. */ Installation installation(); @@ -79,35 +81,35 @@ public interface PacketCoreControlPlane { /** * Gets the sites property: Site(s) under which this packet core control plane should be deployed. The sites must be * in the same location as the packet core control plane. - * + * * @return the sites value. */ List sites(); /** * Gets the platform property: The platform where the packet core is deployed. - * + * * @return the platform value. */ PlatformConfiguration platform(); /** * Gets the coreNetworkTechnology property: The core network technology generation (5G core or EPC / 4G core). - * + * * @return the coreNetworkTechnology value. */ CoreNetworkType coreNetworkTechnology(); /** * Gets the version property: The desired version of the packet core software. - * + * * @return the version value. */ String version(); /** * Gets the installedVersion property: The currently installed version of the packet core software. - * + * * @return the installedVersion value. */ String installedVersion(); @@ -115,7 +117,7 @@ public interface PacketCoreControlPlane { /** * Gets the rollbackVersion property: The previous version of the packet core software that was deployed. Used when * performing the rollback action. - * + * * @return the rollbackVersion value. */ String rollbackVersion(); @@ -123,7 +125,7 @@ public interface PacketCoreControlPlane { /** * Gets the controlPlaneAccessInterface property: The control plane interface on the access network. For 5G * networks, this is the N2 interface. For 4G networks, this is the S1-MME interface. - * + * * @return the controlPlaneAccessInterface value. */ InterfaceProperties controlPlaneAccessInterface(); @@ -133,7 +135,7 @@ public interface PacketCoreControlPlane { * access network in a High Availability (HA) system. In an HA deployment the access network router should be * configured to anycast traffic for this address to the control plane access interfaces on the active and standby * nodes. In non-HA system this list should be omitted or empty. - * + * * @return the controlPlaneAccessVirtualIpv4Addresses value. */ List controlPlaneAccessVirtualIpv4Addresses(); @@ -141,7 +143,7 @@ public interface PacketCoreControlPlane { /** * Gets the sku property: The SKU defining the throughput and SIM allowances for this packet core control plane * deployment. - * + * * @return the sku value. */ BillingSku sku(); @@ -150,7 +152,7 @@ public interface PacketCoreControlPlane { * Gets the ueMtu property: The MTU (in bytes) signaled to the UE. The same MTU is set on the user plane data links * for all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater than this * value to allow for GTP encapsulation. - * + * * @return the ueMtu value. */ Integer ueMtu(); @@ -158,28 +160,28 @@ public interface PacketCoreControlPlane { /** * Gets the localDiagnosticsAccess property: The kubernetes ingress configuration to control access to packet core * diagnostics over local APIs. - * + * * @return the localDiagnosticsAccess value. */ LocalDiagnosticsAccessConfiguration localDiagnosticsAccess(); /** * Gets the diagnosticsUpload property: Configuration for uploading packet core diagnostics. - * + * * @return the diagnosticsUpload value. */ DiagnosticsUploadConfiguration diagnosticsUpload(); /** * Gets the eventHub property: Configuration for sending packet core events to an Azure Event Hub. - * + * * @return the eventHub value. */ EventHubConfiguration eventHub(); /** * Gets the signaling property: Signaling configuration for the packet core. - * + * * @return the signaling value. */ SignalingConfiguration signaling(); @@ -187,63 +189,73 @@ public interface PacketCoreControlPlane { /** * Gets the interopSettings property: Settings to allow interoperability with third party components e.g. RANs and * UEs. - * + * * @return the interopSettings value. */ Object interopSettings(); + /** + * Gets the homeNetworkPrivateKeysProvisioning property: The provisioning state of the secret containing private + * keys and keyIds for SUPI concealment. + * + * @return the homeNetworkPrivateKeysProvisioning value. + */ + HomeNetworkPrivateKeysProvisioning homeNetworkPrivateKeysProvisioning(); + /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreControlPlaneInner object. - * + * * @return the inner object. */ PacketCoreControlPlaneInner innerModel(); - /** The entirety of the PacketCoreControlPlane definition. */ + /** + * The entirety of the PacketCoreControlPlane definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithSites, - DefinitionStages.WithPlatform, - DefinitionStages.WithControlPlaneAccessInterface, - DefinitionStages.WithSku, - DefinitionStages.WithLocalDiagnosticsAccess, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithLocation, DefinitionStages.WithResourceGroup, + DefinitionStages.WithSites, DefinitionStages.WithPlatform, DefinitionStages.WithControlPlaneAccessInterface, + DefinitionStages.WithSku, DefinitionStages.WithLocalDiagnosticsAccess, DefinitionStages.WithCreate { } - /** The PacketCoreControlPlane definition stages. */ + /** + * The PacketCoreControlPlane definition stages. + */ interface DefinitionStages { - /** The first stage of the PacketCoreControlPlane definition. */ + /** + * The first stage of the PacketCoreControlPlane definition. + */ interface Blank extends WithLocation { } - /** The stage of the PacketCoreControlPlane definition allowing to specify location. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -251,82 +263,94 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the PacketCoreControlPlane definition allowing to specify parent resource. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ WithSites withExistingResourceGroup(String resourceGroupName); } - /** The stage of the PacketCoreControlPlane definition allowing to specify sites. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify sites. + */ interface WithSites { /** * Specifies the sites property: Site(s) under which this packet core control plane should be deployed. The * sites must be in the same location as the packet core control plane.. - * + * * @param sites Site(s) under which this packet core control plane should be deployed. The sites must be in - * the same location as the packet core control plane. + * the same location as the packet core control plane. * @return the next definition stage. */ WithPlatform withSites(List sites); } - /** The stage of the PacketCoreControlPlane definition allowing to specify platform. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify platform. + */ interface WithPlatform { /** * Specifies the platform property: The platform where the packet core is deployed.. - * + * * @param platform The platform where the packet core is deployed. * @return the next definition stage. */ WithControlPlaneAccessInterface withPlatform(PlatformConfiguration platform); } - /** The stage of the PacketCoreControlPlane definition allowing to specify controlPlaneAccessInterface. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify controlPlaneAccessInterface. + */ interface WithControlPlaneAccessInterface { /** * Specifies the controlPlaneAccessInterface property: The control plane interface on the access network. * For 5G networks, this is the N2 interface. For 4G networks, this is the S1-MME interface.. - * + * * @param controlPlaneAccessInterface The control plane interface on the access network. For 5G networks, - * this is the N2 interface. For 4G networks, this is the S1-MME interface. + * this is the N2 interface. For 4G networks, this is the S1-MME interface. * @return the next definition stage. */ WithSku withControlPlaneAccessInterface(InterfaceProperties controlPlaneAccessInterface); } - /** The stage of the PacketCoreControlPlane definition allowing to specify sku. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify sku. + */ interface WithSku { /** * Specifies the sku property: The SKU defining the throughput and SIM allowances for this packet core * control plane deployment.. - * + * * @param sku The SKU defining the throughput and SIM allowances for this packet core control plane - * deployment. + * deployment. * @return the next definition stage. */ WithLocalDiagnosticsAccess withSku(BillingSku sku); } - /** The stage of the PacketCoreControlPlane definition allowing to specify localDiagnosticsAccess. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify localDiagnosticsAccess. + */ interface WithLocalDiagnosticsAccess { /** * Specifies the localDiagnosticsAccess property: The kubernetes ingress configuration to control access to * packet core diagnostics over local APIs.. - * + * * @param localDiagnosticsAccess The kubernetes ingress configuration to control access to packet core - * diagnostics over local APIs. + * diagnostics over local APIs. * @return the next definition stage. */ WithCreate withLocalDiagnosticsAccess(LocalDiagnosticsAccessConfiguration localDiagnosticsAccess); @@ -336,85 +360,88 @@ interface WithLocalDiagnosticsAccess { * The stage of the PacketCoreControlPlane definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithIdentity, - DefinitionStages.WithInstallation, - DefinitionStages.WithCoreNetworkTechnology, - DefinitionStages.WithVersion, - DefinitionStages.WithControlPlaneAccessVirtualIpv4Addresses, - DefinitionStages.WithUeMtu, - DefinitionStages.WithDiagnosticsUpload, - DefinitionStages.WithEventHub, - DefinitionStages.WithSignaling, - DefinitionStages.WithInteropSettings { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithInstallation, DefinitionStages.WithCoreNetworkTechnology, DefinitionStages.WithVersion, + DefinitionStages.WithControlPlaneAccessVirtualIpv4Addresses, DefinitionStages.WithUeMtu, + DefinitionStages.WithDiagnosticsUpload, DefinitionStages.WithEventHub, DefinitionStages.WithSignaling, + DefinitionStages.WithInteropSettings { /** * Executes the create request. - * + * * @return the created resource. */ PacketCoreControlPlane create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ PacketCoreControlPlane create(Context context); } - /** The stage of the PacketCoreControlPlane definition allowing to specify tags. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the PacketCoreControlPlane definition allowing to specify identity. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The identity used to retrieve the ingress certificate from Azure key * vault.. - * + * * @param identity The identity used to retrieve the ingress certificate from Azure key vault. * @return the next definition stage. */ WithCreate withIdentity(ManagedServiceIdentity identity); } - /** The stage of the PacketCoreControlPlane definition allowing to specify installation. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify installation. + */ interface WithInstallation { /** * Specifies the installation property: The installation state of the packet core control plane resource.. - * + * * @param installation The installation state of the packet core control plane resource. * @return the next definition stage. */ WithCreate withInstallation(Installation installation); } - /** The stage of the PacketCoreControlPlane definition allowing to specify coreNetworkTechnology. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify coreNetworkTechnology. + */ interface WithCoreNetworkTechnology { /** * Specifies the coreNetworkTechnology property: The core network technology generation (5G core or EPC / 4G * core).. - * + * * @param coreNetworkTechnology The core network technology generation (5G core or EPC / 4G core). * @return the next definition stage. */ WithCreate withCoreNetworkTechnology(CoreNetworkType coreNetworkTechnology); } - /** The stage of the PacketCoreControlPlane definition allowing to specify version. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify version. + */ interface WithVersion { /** * Specifies the version property: The desired version of the packet core software.. - * + * * @param version The desired version of the packet core software. * @return the next definition stage. */ @@ -431,70 +458,80 @@ interface WithControlPlaneAccessVirtualIpv4Addresses { * plane on the access network in a High Availability (HA) system. In an HA deployment the access network * router should be configured to anycast traffic for this address to the control plane access interfaces on * the active and standby nodes. In non-HA system this list should be omitted or empty.. - * + * * @param controlPlaneAccessVirtualIpv4Addresses The virtual IP address(es) for the control plane on the - * access network in a High Availability (HA) system. In an HA deployment the access network router - * should be configured to anycast traffic for this address to the control plane access interfaces on - * the active and standby nodes. In non-HA system this list should be omitted or empty. + * access network in a High Availability (HA) system. In an HA deployment the access network router should + * be configured to anycast traffic for this address to the control plane access interfaces on the active + * and standby nodes. In non-HA system this list should be omitted or empty. * @return the next definition stage. */ WithCreate withControlPlaneAccessVirtualIpv4Addresses(List controlPlaneAccessVirtualIpv4Addresses); } - /** The stage of the PacketCoreControlPlane definition allowing to specify ueMtu. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify ueMtu. + */ interface WithUeMtu { /** * Specifies the ueMtu property: The MTU (in bytes) signaled to the UE. The same MTU is set on the user * plane data links for all data networks. The MTU set on the user plane access link is calculated to be 60 * bytes greater than this value to allow for GTP encapsulation.. - * + * * @param ueMtu The MTU (in bytes) signaled to the UE. The same MTU is set on the user plane data links for - * all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater - * than this value to allow for GTP encapsulation. + * all data networks. The MTU set on the user plane access link is calculated to be 60 bytes greater than + * this value to allow for GTP encapsulation. * @return the next definition stage. */ WithCreate withUeMtu(Integer ueMtu); } - /** The stage of the PacketCoreControlPlane definition allowing to specify diagnosticsUpload. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify diagnosticsUpload. + */ interface WithDiagnosticsUpload { /** * Specifies the diagnosticsUpload property: Configuration for uploading packet core diagnostics. - * + * * @param diagnosticsUpload Configuration for uploading packet core diagnostics. * @return the next definition stage. */ WithCreate withDiagnosticsUpload(DiagnosticsUploadConfiguration diagnosticsUpload); } - /** The stage of the PacketCoreControlPlane definition allowing to specify eventHub. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify eventHub. + */ interface WithEventHub { /** * Specifies the eventHub property: Configuration for sending packet core events to an Azure Event Hub.. - * + * * @param eventHub Configuration for sending packet core events to an Azure Event Hub. * @return the next definition stage. */ WithCreate withEventHub(EventHubConfiguration eventHub); } - /** The stage of the PacketCoreControlPlane definition allowing to specify signaling. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify signaling. + */ interface WithSignaling { /** * Specifies the signaling property: Signaling configuration for the packet core.. - * + * * @param signaling Signaling configuration for the packet core. * @return the next definition stage. */ WithCreate withSignaling(SignalingConfiguration signaling); } - /** The stage of the PacketCoreControlPlane definition allowing to specify interopSettings. */ + /** + * The stage of the PacketCoreControlPlane definition allowing to specify interopSettings. + */ interface WithInteropSettings { /** * Specifies the interopSettings property: Settings to allow interoperability with third party components * e.g. RANs and UEs.. - * + * * @param interopSettings Settings to allow interoperability with third party components e.g. RANs and UEs. * @return the next definition stage. */ @@ -504,47 +541,55 @@ interface WithInteropSettings { /** * Begins update for the PacketCoreControlPlane resource. - * + * * @return the stage of resource update. */ PacketCoreControlPlane.Update update(); - /** The template for PacketCoreControlPlane update. */ + /** + * The template for PacketCoreControlPlane update. + */ interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity { /** * Executes the update request. - * + * * @return the updated resource. */ PacketCoreControlPlane apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ PacketCoreControlPlane apply(Context context); } - /** The PacketCoreControlPlane update stages. */ + /** + * The PacketCoreControlPlane update stages. + */ interface UpdateStages { - /** The stage of the PacketCoreControlPlane update allowing to specify tags. */ + /** + * The stage of the PacketCoreControlPlane update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the PacketCoreControlPlane update allowing to specify identity. */ + /** + * The stage of the PacketCoreControlPlane update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The managed service identity associated with this resource.. - * + * * @param identity The managed service identity associated with this resource. * @return the next definition stage. */ @@ -554,14 +599,14 @@ interface WithIdentity { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ PacketCoreControlPlane refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -570,7 +615,7 @@ interface WithIdentity { /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. @@ -580,7 +625,7 @@ interface WithIdentity { /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -590,9 +635,9 @@ interface WithIdentity { AsyncOperationStatus rollback(Context context); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. @@ -600,9 +645,9 @@ interface WithIdentity { AsyncOperationStatus reinstall(); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -614,7 +659,7 @@ interface WithIdentity { /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -626,7 +671,7 @@ interface WithIdentity { /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -634,6 +679,6 @@ interface WithIdentity { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus collectDiagnosticsPackage( - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context); + AsyncOperationStatus collectDiagnosticsPackage(PacketCoreControlPlaneCollectDiagnosticsPackage parameters, + Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneCollectDiagnosticsPackage.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneCollectDiagnosticsPackage.java index e18ea1fd796ce..abeb752a79ab7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneCollectDiagnosticsPackage.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneCollectDiagnosticsPackage.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Packet core control plane collect diagnostics package options. */ +/** + * Packet core control plane collect diagnostics package options. + */ @Fluent public final class PacketCoreControlPlaneCollectDiagnosticsPackage { /* @@ -17,13 +19,15 @@ public final class PacketCoreControlPlaneCollectDiagnosticsPackage { @JsonProperty(value = "storageAccountBlobUrl", required = true) private String storageAccountBlobUrl; - /** Creates an instance of PacketCoreControlPlaneCollectDiagnosticsPackage class. */ + /** + * Creates an instance of PacketCoreControlPlaneCollectDiagnosticsPackage class. + */ public PacketCoreControlPlaneCollectDiagnosticsPackage() { } /** * Get the storageAccountBlobUrl property: The Storage Account Blob URL to upload the diagnostics package to. - * + * * @return the storageAccountBlobUrl value. */ public String storageAccountBlobUrl() { @@ -32,7 +36,7 @@ public String storageAccountBlobUrl() { /** * Set the storageAccountBlobUrl property: The Storage Account Blob URL to upload the diagnostics package to. - * + * * @param storageAccountBlobUrl the storageAccountBlobUrl value to set. * @return the PacketCoreControlPlaneCollectDiagnosticsPackage object itself. */ @@ -43,16 +47,13 @@ public PacketCoreControlPlaneCollectDiagnosticsPackage withStorageAccountBlobUrl /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (storageAccountBlobUrl() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property storageAccountBlobUrl in model" - + " PacketCoreControlPlaneCollectDiagnosticsPackage")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property storageAccountBlobUrl in model PacketCoreControlPlaneCollectDiagnosticsPackage")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneListResult.java index 05b50b5d91468..17df308fda411 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for packet core control planes API service call. */ +/** + * Response for packet core control planes API service call. + */ @Fluent public final class PacketCoreControlPlaneListResult { /* @@ -24,13 +26,15 @@ public final class PacketCoreControlPlaneListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of PacketCoreControlPlaneListResult class. */ + /** + * Creates an instance of PacketCoreControlPlaneListResult class. + */ public PacketCoreControlPlaneListResult() { } /** * Get the value property: A list of packet core control planes in a resource group. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of packet core control planes in a resource group. - * + * * @param value the value value to set. * @return the PacketCoreControlPlaneListResult object itself. */ @@ -50,7 +54,7 @@ public PacketCoreControlPlaneListResult withValue(List platforms(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreControlPlaneVersionInner object. - * + * * @return the inner object. */ PacketCoreControlPlaneVersionInner innerModel(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneVersionListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneVersionListResult.java index a7ce419c2a719..6790bc0271e3d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneVersionListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlaneVersionListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for packet core control plane version API service call. */ +/** + * Response for packet core control plane version API service call. + */ @Fluent public final class PacketCoreControlPlaneVersionListResult { /* @@ -24,13 +26,15 @@ public final class PacketCoreControlPlaneVersionListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of PacketCoreControlPlaneVersionListResult class. */ + /** + * Creates an instance of PacketCoreControlPlaneVersionListResult class. + */ public PacketCoreControlPlaneVersionListResult() { } /** * Get the value property: A list of supported packet core control plane versions. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of supported packet core control plane versions. - * + * * @param value the value value to set. * @return the PacketCoreControlPlaneVersionListResult object itself. */ @@ -50,7 +54,7 @@ public PacketCoreControlPlaneVersionListResult withValue(List list(); /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -69,7 +71,7 @@ public interface PacketCoreControlPlaneVersions { /** * Gets information about the specified packet core control plane version. - * + * * @param versionName The name of the packet core control plane version. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -80,23 +82,23 @@ public interface PacketCoreControlPlaneVersions { /** * Lists all supported packet core control planes versions. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable listBySubscription(); /** * Lists all supported packet core control planes versions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control plane version API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control plane version API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable listBySubscription(Context context); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlanes.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlanes.java index 791ce53777ca6..cd3b2885bf771 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlanes.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreControlPlanes.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of PacketCoreControlPlanes. */ +/** + * Resource collection API of PacketCoreControlPlanes. + */ public interface PacketCoreControlPlanes { /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface PacketCoreControlPlanes { /** * Deletes the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -35,7 +37,7 @@ public interface PacketCoreControlPlanes { /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -44,12 +46,12 @@ public interface PacketCoreControlPlanes { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core control plane along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String packetCoreControlPlaneName, Context context); /** * Gets information about the specified packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -61,55 +63,55 @@ Response getByResourceGroupWithResponse( /** * Lists all the packet core control planes in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Lists all the packet core control planes in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable listByResourceGroup(String resourceGroupName); /** * Lists all the packet core control planes in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response for packet core control planes API service call as paginated response with {@link - * PagedIterable}. + * @return response for packet core control planes API service call as paginated response with + * {@link PagedIterable}. */ PagedIterable listByResourceGroup(String resourceGroupName, Context context); /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -122,7 +124,7 @@ Response getByResourceGroupWithResponse( /** * Roll back the specified packet core control plane to the previous version, "rollbackVersion". Multiple * consecutive rollbacks are not possible. This action may cause a service outage. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -134,9 +136,9 @@ Response getByResourceGroupWithResponse( AsyncOperationStatus rollback(String resourceGroupName, String packetCoreControlPlaneName, Context context); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -147,9 +149,9 @@ Response getByResourceGroupWithResponse( AsyncOperationStatus reinstall(String resourceGroupName, String packetCoreControlPlaneName); /** - * Reinstall the specified packet core control plane. This action will remove any transaction state from the packet - * core to return it to a known state. This action will cause a service outage. - * + * Reinstall the specified packet core control plane. This action will try to restore the packet core to the + * installed state that was disrupted by a transient failure. This action will cause a service outage. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -163,7 +165,7 @@ Response getByResourceGroupWithResponse( /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -172,15 +174,13 @@ Response getByResourceGroupWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, + AsyncOperationStatus collectDiagnosticsPackage(String resourceGroupName, String packetCoreControlPlaneName, PacketCoreControlPlaneCollectDiagnosticsPackage parameters); /** * Collect a diagnostics package for the specified packet core control plane. This action will upload the * diagnostics to a storage account. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param parameters Parameters supplied to the packet core control plane collect diagnostics package operation. @@ -190,15 +190,12 @@ AsyncOperationStatus collectDiagnosticsPackage( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus collectDiagnosticsPackage( - String resourceGroupName, - String packetCoreControlPlaneName, - PacketCoreControlPlaneCollectDiagnosticsPackage parameters, - Context context); + AsyncOperationStatus collectDiagnosticsPackage(String resourceGroupName, String packetCoreControlPlaneName, + PacketCoreControlPlaneCollectDiagnosticsPackage parameters, Context context); /** * Gets information about the specified packet core control plane. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -209,7 +206,7 @@ AsyncOperationStatus collectDiagnosticsPackage( /** * Gets information about the specified packet core control plane. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -221,7 +218,7 @@ AsyncOperationStatus collectDiagnosticsPackage( /** * Deletes the specified packet core control plane. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -231,7 +228,7 @@ AsyncOperationStatus collectDiagnosticsPackage( /** * Deletes the specified packet core control plane. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -242,7 +239,7 @@ AsyncOperationStatus collectDiagnosticsPackage( /** * Begins definition for a new PacketCoreControlPlane resource. - * + * * @param name resource name. * @return the first stage of the new PacketCoreControlPlane definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlane.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlane.java index 86571d713d2c8..6bc8e157f9c58 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlane.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlane.java @@ -11,53 +11,55 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of PacketCoreDataPlane. */ +/** + * An immutable client-side representation of PacketCoreDataPlane. + */ public interface PacketCoreDataPlane { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the packet core data plane resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); @@ -65,7 +67,7 @@ public interface PacketCoreDataPlane { /** * Gets the userPlaneAccessInterface property: The user plane interface on the access network. For 5G networks, this * is the N3 interface. For 4G networks, this is the S1-U interface. - * + * * @return the userPlaneAccessInterface value. */ InterfaceProperties userPlaneAccessInterface(); @@ -75,59 +77,64 @@ public interface PacketCoreDataPlane { * access network in a High Availability (HA) system. In an HA deployment the access network router should be * configured to forward traffic for this address to the control plane access interface on the active or standby * node. In non-HA system this list should be omitted or empty. - * + * * @return the userPlaneAccessVirtualIpv4Addresses value. */ List userPlaneAccessVirtualIpv4Addresses(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreDataPlaneInner object. - * + * * @return the inner object. */ PacketCoreDataPlaneInner innerModel(); - /** The entirety of the PacketCoreDataPlane definition. */ + /** + * The entirety of the PacketCoreDataPlane definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithUserPlaneAccessInterface, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithLocation, DefinitionStages.WithParentResource, + DefinitionStages.WithUserPlaneAccessInterface, DefinitionStages.WithCreate { } - /** The PacketCoreDataPlane definition stages. */ + /** + * The PacketCoreDataPlane definition stages. + */ interface DefinitionStages { - /** The first stage of the PacketCoreDataPlane definition. */ + /** + * The first stage of the PacketCoreDataPlane definition. + */ interface Blank extends WithLocation { } - /** The stage of the PacketCoreDataPlane definition allowing to specify location. */ + /** + * The stage of the PacketCoreDataPlane definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -135,34 +142,38 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the PacketCoreDataPlane definition allowing to specify parent resource. */ + /** + * The stage of the PacketCoreDataPlane definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, packetCoreControlPlaneName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @return the next definition stage. */ - WithUserPlaneAccessInterface withExistingPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + WithUserPlaneAccessInterface withExistingPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); } - /** The stage of the PacketCoreDataPlane definition allowing to specify userPlaneAccessInterface. */ + /** + * The stage of the PacketCoreDataPlane definition allowing to specify userPlaneAccessInterface. + */ interface WithUserPlaneAccessInterface { /** * Specifies the userPlaneAccessInterface property: The user plane interface on the access network. For 5G * networks, this is the N3 interface. For 4G networks, this is the S1-U interface.. - * + * * @param userPlaneAccessInterface The user plane interface on the access network. For 5G networks, this is - * the N3 interface. For 4G networks, this is the S1-U interface. + * the N3 interface. For 4G networks, this is the S1-U interface. * @return the next definition stage. */ WithCreate withUserPlaneAccessInterface(InterfaceProperties userPlaneAccessInterface); @@ -176,43 +187,47 @@ interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithUserPlaneAccessVirtualIpv4Addresses { /** * Executes the create request. - * + * * @return the created resource. */ PacketCoreDataPlane create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ PacketCoreDataPlane create(Context context); } - /** The stage of the PacketCoreDataPlane definition allowing to specify tags. */ + /** + * The stage of the PacketCoreDataPlane definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the PacketCoreDataPlane definition allowing to specify userPlaneAccessVirtualIpv4Addresses. */ + /** + * The stage of the PacketCoreDataPlane definition allowing to specify userPlaneAccessVirtualIpv4Addresses. + */ interface WithUserPlaneAccessVirtualIpv4Addresses { /** * Specifies the userPlaneAccessVirtualIpv4Addresses property: The virtual IP address(es) for the user plane * on the access network in a High Availability (HA) system. In an HA deployment the access network router * should be configured to forward traffic for this address to the control plane access interface on the * active or standby node. In non-HA system this list should be omitted or empty.. - * + * * @param userPlaneAccessVirtualIpv4Addresses The virtual IP address(es) for the user plane on the access - * network in a High Availability (HA) system. In an HA deployment the access network router should be - * configured to forward traffic for this address to the control plane access interface on the active or - * standby node. In non-HA system this list should be omitted or empty. + * network in a High Availability (HA) system. In an HA deployment the access network router should be + * configured to forward traffic for this address to the control plane access interface on the active or + * standby node. In non-HA system this list should be omitted or empty. * @return the next definition stage. */ WithCreate withUserPlaneAccessVirtualIpv4Addresses(List userPlaneAccessVirtualIpv4Addresses); @@ -221,36 +236,42 @@ interface WithUserPlaneAccessVirtualIpv4Addresses { /** * Begins update for the PacketCoreDataPlane resource. - * + * * @return the stage of resource update. */ PacketCoreDataPlane.Update update(); - /** The template for PacketCoreDataPlane update. */ + /** + * The template for PacketCoreDataPlane update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ PacketCoreDataPlane apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ PacketCoreDataPlane apply(Context context); } - /** The PacketCoreDataPlane update stages. */ + /** + * The PacketCoreDataPlane update stages. + */ interface UpdateStages { - /** The stage of the PacketCoreDataPlane update allowing to specify tags. */ + /** + * The stage of the PacketCoreDataPlane update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -260,14 +281,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ PacketCoreDataPlane refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlaneListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlaneListResult.java index fae3b1bf513e9..7076c983e3f30 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlaneListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlaneListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for packet core data planes API service call. */ +/** + * Response for packet core data planes API service call. + */ @Fluent public final class PacketCoreDataPlaneListResult { /* @@ -24,13 +26,15 @@ public final class PacketCoreDataPlaneListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of PacketCoreDataPlaneListResult class. */ + /** + * Creates an instance of PacketCoreDataPlaneListResult class. + */ public PacketCoreDataPlaneListResult() { } /** * Get the value property: A list of packet core data planes in a resource group. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of packet core data planes in a resource group. - * + * * @param value the value value to set. * @return the PacketCoreDataPlaneListResult object itself. */ @@ -50,7 +54,7 @@ public PacketCoreDataPlaneListResult withValue(List va /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlanes.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlanes.java index 21d38f9b6fa6d..0d5d29fdbe920 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlanes.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PacketCoreDataPlanes.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of PacketCoreDataPlanes. */ +/** + * Resource collection API of PacketCoreDataPlanes. + */ public interface PacketCoreDataPlanes { /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -24,7 +26,7 @@ public interface PacketCoreDataPlanes { /** * Deletes the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -33,12 +35,12 @@ public interface PacketCoreDataPlanes { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + void delete(String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, + Context context); /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -48,12 +50,12 @@ void delete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core data plane along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName, Context context); + Response getWithResponse(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName, Context context); /** * Gets information about the specified packet core data plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param packetCoreDataPlaneName The name of the packet core data plane. @@ -62,12 +64,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified packet core data plane. */ - PacketCoreDataPlane get( - String resourceGroupName, String packetCoreControlPlaneName, String packetCoreDataPlaneName); + PacketCoreDataPlane get(String resourceGroupName, String packetCoreControlPlaneName, + String packetCoreDataPlaneName); /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -75,12 +77,12 @@ PacketCoreDataPlane get( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core data planes API service call as paginated response with {@link PagedIterable}. */ - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName); /** * Lists all the packet core data planes associated with a packet core control plane. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param packetCoreControlPlaneName The name of the packet core control plane. * @param context The context to associate with this operation. @@ -89,12 +91,12 @@ PagedIterable listByPacketCoreControlPlane( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for packet core data planes API service call as paginated response with {@link PagedIterable}. */ - PagedIterable listByPacketCoreControlPlane( - String resourceGroupName, String packetCoreControlPlaneName, Context context); + PagedIterable listByPacketCoreControlPlane(String resourceGroupName, + String packetCoreControlPlaneName, Context context); /** * Gets information about the specified packet core data plane. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -105,7 +107,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Gets information about the specified packet core data plane. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -117,7 +119,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Deletes the specified packet core data plane. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -127,7 +129,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Deletes the specified packet core data plane. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -138,7 +140,7 @@ PagedIterable listByPacketCoreControlPlane( /** * Begins definition for a new PacketCoreDataPlane resource. - * + * * @param name resource name. * @return the first stage of the new PacketCoreDataPlane definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleConfiguration.java index 8edc8af7de9af..5dd953279b5d0 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Data flow policy rule configuration. */ +/** + * Data flow policy rule configuration. + */ @Fluent public final class PccRuleConfiguration { /* @@ -46,14 +48,16 @@ public final class PccRuleConfiguration { @JsonProperty(value = "serviceDataFlowTemplates", required = true) private List serviceDataFlowTemplates; - /** Creates an instance of PccRuleConfiguration class. */ + /** + * Creates an instance of PccRuleConfiguration class. + */ public PccRuleConfiguration() { } /** * Get the ruleName property: The name of the rule. This must be unique within the parent service. You must not use * any of the following reserved strings - `default`, `requested` or `service`. - * + * * @return the ruleName value. */ public String ruleName() { @@ -63,7 +67,7 @@ public String ruleName() { /** * Set the ruleName property: The name of the rule. This must be unique within the parent service. You must not use * any of the following reserved strings - `default`, `requested` or `service`. - * + * * @param ruleName the ruleName value to set. * @return the PccRuleConfiguration object itself. */ @@ -76,7 +80,7 @@ public PccRuleConfiguration withRuleName(String ruleName) { * Get the rulePrecedence property: A precedence value that is used to decide between data flow policy rules when * identifying the QoS values to use for a particular SIM. A lower value means a higher priority. This value should * be unique among all data flow policy rules configured in the mobile network. - * + * * @return the rulePrecedence value. */ public int rulePrecedence() { @@ -87,7 +91,7 @@ public int rulePrecedence() { * Set the rulePrecedence property: A precedence value that is used to decide between data flow policy rules when * identifying the QoS values to use for a particular SIM. A lower value means a higher priority. This value should * be unique among all data flow policy rules configured in the mobile network. - * + * * @param rulePrecedence the rulePrecedence value to set. * @return the PccRuleConfiguration object itself. */ @@ -99,7 +103,7 @@ public PccRuleConfiguration withRulePrecedence(int rulePrecedence) { /** * Get the ruleQosPolicy property: The QoS policy to use for packets matching this rule. If this field is null then * the parent service will define the QoS settings. - * + * * @return the ruleQosPolicy value. */ public PccRuleQosPolicy ruleQosPolicy() { @@ -109,7 +113,7 @@ public PccRuleQosPolicy ruleQosPolicy() { /** * Set the ruleQosPolicy property: The QoS policy to use for packets matching this rule. If this field is null then * the parent service will define the QoS settings. - * + * * @param ruleQosPolicy the ruleQosPolicy value to set. * @return the PccRuleConfiguration object itself. */ @@ -120,7 +124,7 @@ public PccRuleConfiguration withRuleQosPolicy(PccRuleQosPolicy ruleQosPolicy) { /** * Get the trafficControl property: Determines whether flows that match this data flow policy rule are permitted. - * + * * @return the trafficControl value. */ public TrafficControlPermission trafficControl() { @@ -129,7 +133,7 @@ public TrafficControlPermission trafficControl() { /** * Set the trafficControl property: Determines whether flows that match this data flow policy rule are permitted. - * + * * @param trafficControl the trafficControl value to set. * @return the PccRuleConfiguration object itself. */ @@ -140,7 +144,7 @@ public PccRuleConfiguration withTrafficControl(TrafficControlPermission trafficC /** * Get the serviceDataFlowTemplates property: The set of data flow templates to use for this data flow policy rule. - * + * * @return the serviceDataFlowTemplates value. */ public List serviceDataFlowTemplates() { @@ -149,7 +153,7 @@ public List serviceDataFlowTemplates() { /** * Set the serviceDataFlowTemplates property: The set of data flow templates to use for this data flow policy rule. - * + * * @param serviceDataFlowTemplates the serviceDataFlowTemplates value to set. * @return the PccRuleConfiguration object itself. */ @@ -160,23 +164,20 @@ public PccRuleConfiguration withServiceDataFlowTemplates(List e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleQosPolicy.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleQosPolicy.java index e58f4c89ecea4..4af7ce6190375 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleQosPolicy.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PccRuleQosPolicy.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Data flow policy rule QoS policy. */ +/** + * Data flow policy rule QoS policy. + */ @Fluent public final class PccRuleQosPolicy extends QosPolicy { /* @@ -18,7 +20,9 @@ public final class PccRuleQosPolicy extends QosPolicy { @JsonProperty(value = "guaranteedBitRate") private Ambr guaranteedBitRate; - /** Creates an instance of PccRuleQosPolicy class. */ + /** + * Creates an instance of PccRuleQosPolicy class. + */ public PccRuleQosPolicy() { } @@ -26,7 +30,7 @@ public PccRuleQosPolicy() { * Get the guaranteedBitRate property: The guaranteed bit rate (GBR) for all service data flows that use this data * flow policy rule. This is an optional setting. If you do not provide a value, there will be no GBR set for the * data flow policy rule that uses this QoS definition. - * + * * @return the guaranteedBitRate value. */ public Ambr guaranteedBitRate() { @@ -37,7 +41,7 @@ public Ambr guaranteedBitRate() { * Set the guaranteedBitRate property: The guaranteed bit rate (GBR) for all service data flows that use this data * flow policy rule. This is an optional setting. If you do not provide a value, there will be no GBR set for the * data flow policy rule that uses this QoS definition. - * + * * @param guaranteedBitRate the guaranteedBitRate value to set. * @return the PccRuleQosPolicy object itself. */ @@ -46,35 +50,45 @@ public PccRuleQosPolicy withGuaranteedBitRate(Ambr guaranteedBitRate) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PccRuleQosPolicy withFiveQi(Integer fiveQi) { super.withFiveQi(fiveQi); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PccRuleQosPolicy withAllocationAndRetentionPriorityLevel(Integer allocationAndRetentionPriorityLevel) { super.withAllocationAndRetentionPriorityLevel(allocationAndRetentionPriorityLevel); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PccRuleQosPolicy withPreemptionCapability(PreemptionCapability preemptionCapability) { super.withPreemptionCapability(preemptionCapability); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PccRuleQosPolicy withPreemptionVulnerability(PreemptionVulnerability preemptionVulnerability) { super.withPreemptionVulnerability(preemptionVulnerability); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public PccRuleQosPolicy withMaximumBitRate(Ambr maximumBitRate) { super.withMaximumBitRate(maximumBitRate); @@ -83,7 +97,7 @@ public PccRuleQosPolicy withMaximumBitRate(Ambr maximumBitRate) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PdnType.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PdnType.java new file mode 100644 index 0000000000000..563326bf3f85b --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PdnType.java @@ -0,0 +1,48 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Packet Data Network Type. + */ +public final class PdnType extends ExpandableStringEnum { + /** + * Static value IPV4 for PdnType. + */ + public static final PdnType IPV4 = fromString("IPV4"); + + /** + * Creates a new instance of PdnType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PdnType() { + } + + /** + * Creates or finds a PdnType from its string representation. + * + * @param name a name to look for. + * @return the corresponding PdnType. + */ + @JsonCreator + public static PdnType fromString(String name) { + return fromString(name, PdnType.class); + } + + /** + * Gets known PdnType values. + * + * @return known PdnType values. + */ + public static Collection values() { + return values(PdnType.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PduSessionType.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PduSessionType.java index c9ff674d970b4..e4b5e3636f575 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PduSessionType.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PduSessionType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** PDU session type (IPv4/IPv6). */ +/** + * PDU session type (IPv4/IPv6). + */ public final class PduSessionType extends ExpandableStringEnum { - /** Static value IPv4 for PduSessionType. */ + /** + * Static value IPv4 for PduSessionType. + */ public static final PduSessionType IPV4 = fromString("IPv4"); - /** Static value IPv6 for PduSessionType. */ + /** + * Static value IPv6 for PduSessionType. + */ public static final PduSessionType IPV6 = fromString("IPv6"); /** * Creates a new instance of PduSessionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public PduSessionType() { /** * Creates or finds a PduSessionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding PduSessionType. */ @@ -38,7 +44,7 @@ public static PduSessionType fromString(String name) { /** * Gets known PduSessionType values. - * + * * @return known PduSessionType values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PinholeTimeouts.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PinholeTimeouts.java index 149cb771724da..19fc4cc1fc85d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PinholeTimeouts.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PinholeTimeouts.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Expiry times of inactive NAPT pinholes, in seconds. All timers must be at least 1 second. */ +/** + * Expiry times of inactive NAPT pinholes, in seconds. All timers must be at least 1 second. + */ @Fluent public final class PinholeTimeouts { /* @@ -28,13 +30,15 @@ public final class PinholeTimeouts { @JsonProperty(value = "icmp") private Integer icmp; - /** Creates an instance of PinholeTimeouts class. */ + /** + * Creates an instance of PinholeTimeouts class. + */ public PinholeTimeouts() { } /** * Get the tcp property: Pinhole timeout for TCP pinholes in seconds. Default for TCP is 3 minutes. - * + * * @return the tcp value. */ public Integer tcp() { @@ -43,7 +47,7 @@ public Integer tcp() { /** * Set the tcp property: Pinhole timeout for TCP pinholes in seconds. Default for TCP is 3 minutes. - * + * * @param tcp the tcp value to set. * @return the PinholeTimeouts object itself. */ @@ -54,7 +58,7 @@ public PinholeTimeouts withTcp(Integer tcp) { /** * Get the udp property: Pinhole timeout for UDP pinholes in seconds. Default for UDP is 30 seconds. - * + * * @return the udp value. */ public Integer udp() { @@ -63,7 +67,7 @@ public Integer udp() { /** * Set the udp property: Pinhole timeout for UDP pinholes in seconds. Default for UDP is 30 seconds. - * + * * @param udp the udp value to set. * @return the PinholeTimeouts object itself. */ @@ -74,7 +78,7 @@ public PinholeTimeouts withUdp(Integer udp) { /** * Get the icmp property: Pinhole timeout for ICMP pinholes in seconds. Default for ICMP Echo is 30 seconds. - * + * * @return the icmp value. */ public Integer icmp() { @@ -83,7 +87,7 @@ public Integer icmp() { /** * Set the icmp property: Pinhole timeout for ICMP pinholes in seconds. Default for ICMP Echo is 30 seconds. - * + * * @param icmp the icmp value to set. * @return the PinholeTimeouts object itself. */ @@ -94,7 +98,7 @@ public PinholeTimeouts withIcmp(Integer icmp) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Platform.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Platform.java index 2ee3a5b2f8fa9..0c9ee396f4da1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Platform.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Platform.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Platform specific packet core control plane version properties. */ +/** + * Platform specific packet core control plane version properties. + */ @Fluent public final class Platform { /* @@ -46,13 +48,15 @@ public final class Platform { @JsonProperty(value = "obsoleteVersion") private ObsoleteVersion obsoleteVersion; - /** Creates an instance of Platform class. */ + /** + * Creates an instance of Platform class. + */ public Platform() { } /** * Get the platformType property: The platform type where this version can be deployed. - * + * * @return the platformType value. */ public PlatformType platformType() { @@ -61,7 +65,7 @@ public PlatformType platformType() { /** * Set the platformType property: The platform type where this version can be deployed. - * + * * @param platformType the platformType value to set. * @return the Platform object itself. */ @@ -72,7 +76,7 @@ public Platform withPlatformType(PlatformType platformType) { /** * Get the versionState property: The state of this packet core control plane version on this platform. - * + * * @return the versionState value. */ public VersionState versionState() { @@ -81,7 +85,7 @@ public VersionState versionState() { /** * Set the versionState property: The state of this packet core control plane version on this platform. - * + * * @param versionState the versionState value to set. * @return the Platform object itself. */ @@ -93,7 +97,7 @@ public Platform withVersionState(VersionState versionState) { /** * Get the minimumPlatformSoftwareVersion property: The minimum software version of the platform where this packet * core version can be deployed. - * + * * @return the minimumPlatformSoftwareVersion value. */ public String minimumPlatformSoftwareVersion() { @@ -103,7 +107,7 @@ public String minimumPlatformSoftwareVersion() { /** * Set the minimumPlatformSoftwareVersion property: The minimum software version of the platform where this packet * core version can be deployed. - * + * * @param minimumPlatformSoftwareVersion the minimumPlatformSoftwareVersion value to set. * @return the Platform object itself. */ @@ -115,7 +119,7 @@ public Platform withMinimumPlatformSoftwareVersion(String minimumPlatformSoftwar /** * Get the maximumPlatformSoftwareVersion property: The maximum software version of the platform where this packet * core version can be deployed. - * + * * @return the maximumPlatformSoftwareVersion value. */ public String maximumPlatformSoftwareVersion() { @@ -125,7 +129,7 @@ public String maximumPlatformSoftwareVersion() { /** * Set the maximumPlatformSoftwareVersion property: The maximum software version of the platform where this packet * core version can be deployed. - * + * * @param maximumPlatformSoftwareVersion the maximumPlatformSoftwareVersion value to set. * @return the Platform object itself. */ @@ -136,7 +140,7 @@ public Platform withMaximumPlatformSoftwareVersion(String maximumPlatformSoftwar /** * Get the recommendedVersion property: Indicates whether this is the recommended version for this platform. - * + * * @return the recommendedVersion value. */ public RecommendedVersion recommendedVersion() { @@ -145,7 +149,7 @@ public RecommendedVersion recommendedVersion() { /** * Set the recommendedVersion property: Indicates whether this is the recommended version for this platform. - * + * * @param recommendedVersion the recommendedVersion value to set. * @return the Platform object itself. */ @@ -156,7 +160,7 @@ public Platform withRecommendedVersion(RecommendedVersion recommendedVersion) { /** * Get the obsoleteVersion property: Indicates whether this version is obsoleted for this platform. - * + * * @return the obsoleteVersion value. */ public ObsoleteVersion obsoleteVersion() { @@ -165,7 +169,7 @@ public ObsoleteVersion obsoleteVersion() { /** * Set the obsoleteVersion property: Indicates whether this version is obsoleted for this platform. - * + * * @param obsoleteVersion the obsoleteVersion value to set. * @return the Platform object itself. */ @@ -176,7 +180,7 @@ public Platform withObsoleteVersion(ObsoleteVersion obsoleteVersion) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformConfiguration.java index 8d102d598f63d..051de0e279417 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The platform where the packet core is deployed. */ +/** + * The platform where the packet core is deployed. + */ @Fluent public final class PlatformConfiguration { /* @@ -50,13 +52,15 @@ public final class PlatformConfiguration { @JsonProperty(value = "customLocation") private CustomLocationResourceId customLocation; - /** Creates an instance of PlatformConfiguration class. */ + /** + * Creates an instance of PlatformConfiguration class. + */ public PlatformConfiguration() { } /** * Get the type property: The platform type where packet core is deployed. - * + * * @return the type value. */ public PlatformType type() { @@ -65,7 +69,7 @@ public PlatformType type() { /** * Set the type property: The platform type where packet core is deployed. - * + * * @param type the type value to set. * @return the PlatformConfiguration object itself. */ @@ -77,7 +81,7 @@ public PlatformConfiguration withType(PlatformType type) { /** * Get the azureStackEdgeDevice property: The Azure Stack Edge device where the packet core is deployed. If the * device is part of a fault tolerant pair, either device in the pair can be specified. - * + * * @return the azureStackEdgeDevice value. */ public AzureStackEdgeDeviceResourceId azureStackEdgeDevice() { @@ -87,7 +91,7 @@ public AzureStackEdgeDeviceResourceId azureStackEdgeDevice() { /** * Set the azureStackEdgeDevice property: The Azure Stack Edge device where the packet core is deployed. If the * device is part of a fault tolerant pair, either device in the pair can be specified. - * + * * @param azureStackEdgeDevice the azureStackEdgeDevice value to set. * @return the PlatformConfiguration object itself. */ @@ -99,7 +103,7 @@ public PlatformConfiguration withAzureStackEdgeDevice(AzureStackEdgeDeviceResour /** * Get the azureStackEdgeDevices property: The Azure Stack Edge devices where the packet core is deployed. If the * packet core is deployed across multiple devices, all devices will appear in this list. - * + * * @return the azureStackEdgeDevices value. */ public List azureStackEdgeDevices() { @@ -108,7 +112,7 @@ public List azureStackEdgeDevices() { /** * Get the azureStackHciCluster property: The Azure Stack HCI cluster where the packet core is deployed. - * + * * @return the azureStackHciCluster value. */ public AzureStackHciClusterResourceId azureStackHciCluster() { @@ -117,7 +121,7 @@ public AzureStackHciClusterResourceId azureStackHciCluster() { /** * Set the azureStackHciCluster property: The Azure Stack HCI cluster where the packet core is deployed. - * + * * @param azureStackHciCluster the azureStackHciCluster value to set. * @return the PlatformConfiguration object itself. */ @@ -128,7 +132,7 @@ public PlatformConfiguration withAzureStackHciCluster(AzureStackHciClusterResour /** * Get the connectedCluster property: Azure Arc connected cluster where the packet core is deployed. - * + * * @return the connectedCluster value. */ public ConnectedClusterResourceId connectedCluster() { @@ -137,7 +141,7 @@ public ConnectedClusterResourceId connectedCluster() { /** * Set the connectedCluster property: Azure Arc connected cluster where the packet core is deployed. - * + * * @param connectedCluster the connectedCluster value to set. * @return the PlatformConfiguration object itself. */ @@ -148,7 +152,7 @@ public PlatformConfiguration withConnectedCluster(ConnectedClusterResourceId con /** * Get the customLocation property: Azure Arc custom location where the packet core is deployed. - * + * * @return the customLocation value. */ public CustomLocationResourceId customLocation() { @@ -157,7 +161,7 @@ public CustomLocationResourceId customLocation() { /** * Set the customLocation property: Azure Arc custom location where the packet core is deployed. - * + * * @param customLocation the customLocation value to set. * @return the PlatformConfiguration object itself. */ @@ -168,14 +172,13 @@ public PlatformConfiguration withCustomLocation(CustomLocationResourceId customL /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (type() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property type in model PlatformConfiguration")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property type in model PlatformConfiguration")); } if (azureStackEdgeDevice() != null) { azureStackEdgeDevice().validate(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformType.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformType.java index 63e3adeb4b244..1fdb7a11c5bce 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformType.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlatformType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The platform type where packet core is deployed. The contents of this enum can change. */ +/** + * The platform type where packet core is deployed. The contents of this enum can change. + */ public final class PlatformType extends ExpandableStringEnum { - /** Static value AKS-HCI for PlatformType. */ + /** + * Static value AKS-HCI for PlatformType. + */ public static final PlatformType AKS_HCI = fromString("AKS-HCI"); - /** Static value 3P-AZURE-STACK-HCI for PlatformType. */ + /** + * Static value 3P-AZURE-STACK-HCI for PlatformType. + */ public static final PlatformType THREE_P_AZURE_STACK_HCI = fromString("3P-AZURE-STACK-HCI"); /** * Creates a new instance of PlatformType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public PlatformType() { /** * Creates or finds a PlatformType from its string representation. - * + * * @param name a name to look for. * @return the corresponding PlatformType. */ @@ -38,7 +44,7 @@ public static PlatformType fromString(String name) { /** * Gets known PlatformType values. - * + * * @return known PlatformType values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlmnId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlmnId.java index 9f9b72ed99e47..aed9be1b064b9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlmnId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PlmnId.java @@ -8,9 +8,13 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Public land mobile network (PLMN) ID. */ +/** + * Public land mobile network (PLMN) ID. This is made up of the mobile country code and mobile network code, as defined + * in https://www.itu.int/rec/T-REC-E.212. The values 001-01 and 001-001 can be used for testing and the values 999-99 + * and 999-999 can be used on internal private networks. + */ @Fluent -public final class PlmnId { +public class PlmnId { /* * Mobile country code (MCC). */ @@ -23,13 +27,15 @@ public final class PlmnId { @JsonProperty(value = "mnc", required = true) private String mnc; - /** Creates an instance of PlmnId class. */ + /** + * Creates an instance of PlmnId class. + */ public PlmnId() { } /** * Get the mcc property: Mobile country code (MCC). - * + * * @return the mcc value. */ public String mcc() { @@ -38,7 +44,7 @@ public String mcc() { /** * Set the mcc property: Mobile country code (MCC). - * + * * @param mcc the mcc value to set. * @return the PlmnId object itself. */ @@ -49,7 +55,7 @@ public PlmnId withMcc(String mcc) { /** * Get the mnc property: Mobile network code (MNC). - * + * * @return the mnc value. */ public String mnc() { @@ -58,7 +64,7 @@ public String mnc() { /** * Set the mnc property: Mobile network code (MNC). - * + * * @param mnc the mnc value to set. * @return the PlmnId object itself. */ @@ -69,7 +75,7 @@ public PlmnId withMnc(String mnc) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortRange.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortRange.java index b27026aa97683..e94f1a42940aa 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortRange.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortRange.java @@ -8,9 +8,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * Range of port numbers to use as translated ports on each translated address. If not specified and NAPT is enabled, - * this range defaults to 1,024 - 49,999. (Ports under 1,024 should not be used because these are special purpose ports - * reserved by IANA. Ports 50,000 and above are reserved for non-NAPT use.). + * Range of port numbers to use as translated ports on each translated address. + * If not specified and NAPT is enabled, this range defaults to 1,024 - 49,999. + * (Ports under 1,024 should not be used because these are special purpose ports reserved by IANA. Ports 50,000 and + * above are reserved for non-NAPT use.). */ @Fluent public final class PortRange { @@ -26,13 +27,15 @@ public final class PortRange { @JsonProperty(value = "maxPort") private Integer maxPort; - /** Creates an instance of PortRange class. */ + /** + * Creates an instance of PortRange class. + */ public PortRange() { } /** * Get the minPort property: The minimum port number. - * + * * @return the minPort value. */ public Integer minPort() { @@ -41,7 +44,7 @@ public Integer minPort() { /** * Set the minPort property: The minimum port number. - * + * * @param minPort the minPort value to set. * @return the PortRange object itself. */ @@ -52,7 +55,7 @@ public PortRange withMinPort(Integer minPort) { /** * Get the maxPort property: The maximum port number. - * + * * @return the maxPort value. */ public Integer maxPort() { @@ -61,7 +64,7 @@ public Integer maxPort() { /** * Set the maxPort property: The maximum port number. - * + * * @param maxPort the maxPort value to set. * @return the PortRange object itself. */ @@ -72,7 +75,7 @@ public PortRange withMaxPort(Integer maxPort) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortReuseHoldTimes.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortReuseHoldTimes.java index ede12c4968f8d..f4e63dbadaa20 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortReuseHoldTimes.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PortReuseHoldTimes.java @@ -27,14 +27,16 @@ public final class PortReuseHoldTimes { @JsonProperty(value = "udp") private Integer udp; - /** Creates an instance of PortReuseHoldTimes class. */ + /** + * Creates an instance of PortReuseHoldTimes class. + */ public PortReuseHoldTimes() { } /** * Get the tcp property: Minimum time in seconds that will pass before a TCP port that was used by a closed pinhole * can be reused. Default for TCP is 2 minutes. - * + * * @return the tcp value. */ public Integer tcp() { @@ -44,7 +46,7 @@ public Integer tcp() { /** * Set the tcp property: Minimum time in seconds that will pass before a TCP port that was used by a closed pinhole * can be reused. Default for TCP is 2 minutes. - * + * * @param tcp the tcp value to set. * @return the PortReuseHoldTimes object itself. */ @@ -56,7 +58,7 @@ public PortReuseHoldTimes withTcp(Integer tcp) { /** * Get the udp property: Minimum time in seconds that will pass before a UDP port that was used by a closed pinhole * can be reused. Default for UDP is 1 minute. - * + * * @return the udp value. */ public Integer udp() { @@ -66,7 +68,7 @@ public Integer udp() { /** * Set the udp property: Minimum time in seconds that will pass before a UDP port that was used by a closed pinhole * can be reused. Default for UDP is 1 minute. - * + * * @param udp the udp value to set. * @return the PortReuseHoldTimes object itself. */ @@ -77,7 +79,7 @@ public PortReuseHoldTimes withUdp(Integer udp) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionCapability.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionCapability.java index 402ffa77eb22a..2ca94ee41915e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionCapability.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionCapability.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Preemption capability. */ +/** + * Preemption capability. + */ public final class PreemptionCapability extends ExpandableStringEnum { - /** Static value NotPreempt for PreemptionCapability. */ + /** + * Static value NotPreempt for PreemptionCapability. + */ public static final PreemptionCapability NOT_PREEMPT = fromString("NotPreempt"); - /** Static value MayPreempt for PreemptionCapability. */ + /** + * Static value MayPreempt for PreemptionCapability. + */ public static final PreemptionCapability MAY_PREEMPT = fromString("MayPreempt"); /** * Creates a new instance of PreemptionCapability value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public PreemptionCapability() { /** * Creates or finds a PreemptionCapability from its string representation. - * + * * @param name a name to look for. * @return the corresponding PreemptionCapability. */ @@ -38,7 +44,7 @@ public static PreemptionCapability fromString(String name) { /** * Gets known PreemptionCapability values. - * + * * @return known PreemptionCapability values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionVulnerability.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionVulnerability.java index 462179a992101..0afeda753699c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionVulnerability.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PreemptionVulnerability.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Preemption vulnerability. */ +/** + * Preemption vulnerability. + */ public final class PreemptionVulnerability extends ExpandableStringEnum { - /** Static value NotPreemptable for PreemptionVulnerability. */ + /** + * Static value NotPreemptable for PreemptionVulnerability. + */ public static final PreemptionVulnerability NOT_PREEMPTABLE = fromString("NotPreemptable"); - /** Static value Preemptable for PreemptionVulnerability. */ + /** + * Static value Preemptable for PreemptionVulnerability. + */ public static final PreemptionVulnerability PREEMPTABLE = fromString("Preemptable"); /** * Creates a new instance of PreemptionVulnerability value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public PreemptionVulnerability() { /** * Creates or finds a PreemptionVulnerability from its string representation. - * + * * @param name a name to look for. * @return the corresponding PreemptionVulnerability. */ @@ -38,7 +44,7 @@ public static PreemptionVulnerability fromString(String name) { /** * Gets known PreemptionVulnerability values. - * + * * @return known PreemptionVulnerability values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ProvisioningState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ProvisioningState.java index 090cf77c1a7d4..14d1c2619c8dc 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ProvisioningState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ProvisioningState.java @@ -8,32 +8,48 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The current provisioning state. */ +/** + * The current provisioning state. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Unknown for ProvisioningState. */ + /** + * Static value Unknown for ProvisioningState. + */ public static final ProvisioningState UNKNOWN = fromString("Unknown"); - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Accepted for ProvisioningState. */ + /** + * Static value Accepted for ProvisioningState. + */ public static final ProvisioningState ACCEPTED = fromString("Accepted"); - /** Static value Deleting for ProvisioningState. */ + /** + * Static value Deleting for ProvisioningState. + */ public static final ProvisioningState DELETING = fromString("Deleting"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value Deleted for ProvisioningState. */ + /** + * Static value Deleted for ProvisioningState. + */ public static final ProvisioningState DELETED = fromString("Deleted"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -42,7 +58,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -53,7 +69,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetwork.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetwork.java new file mode 100644 index 0000000000000..648132e384671 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetwork.java @@ -0,0 +1,78 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Configuration relating to a particular PLMN. + */ +@Fluent +public final class PublicLandMobileNetwork extends PlmnId { + /* + * Configuration relating to SUPI concealment. + */ + @JsonProperty(value = "homeNetworkPublicKeys") + private PublicLandMobileNetworkHomeNetworkPublicKeys homeNetworkPublicKeys; + + /** + * Creates an instance of PublicLandMobileNetwork class. + */ + public PublicLandMobileNetwork() { + } + + /** + * Get the homeNetworkPublicKeys property: Configuration relating to SUPI concealment. + * + * @return the homeNetworkPublicKeys value. + */ + public PublicLandMobileNetworkHomeNetworkPublicKeys homeNetworkPublicKeys() { + return this.homeNetworkPublicKeys; + } + + /** + * Set the homeNetworkPublicKeys property: Configuration relating to SUPI concealment. + * + * @param homeNetworkPublicKeys the homeNetworkPublicKeys value to set. + * @return the PublicLandMobileNetwork object itself. + */ + public PublicLandMobileNetwork + withHomeNetworkPublicKeys(PublicLandMobileNetworkHomeNetworkPublicKeys homeNetworkPublicKeys) { + this.homeNetworkPublicKeys = homeNetworkPublicKeys; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PublicLandMobileNetwork withMcc(String mcc) { + super.withMcc(mcc); + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public PublicLandMobileNetwork withMnc(String mnc) { + super.withMnc(mnc); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (homeNetworkPublicKeys() != null) { + homeNetworkPublicKeys().validate(); + } + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetworkHomeNetworkPublicKeys.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetworkHomeNetworkPublicKeys.java new file mode 100644 index 0000000000000..cc580a58e6d20 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/PublicLandMobileNetworkHomeNetworkPublicKeys.java @@ -0,0 +1,93 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration relating to SUPI concealment. + */ +@Fluent +public final class PublicLandMobileNetworkHomeNetworkPublicKeys { + /* + * This provides a mapping to identify which public key has been used for SUPI concealment using the Profile A + * Protection Scheme. + */ + @JsonProperty(value = "profileA") + private List profileA; + + /* + * This provides a mapping to identify which public key has been used for SUPI concealment using the Profile B + * Protection Scheme. + */ + @JsonProperty(value = "profileB") + private List profileB; + + /** + * Creates an instance of PublicLandMobileNetworkHomeNetworkPublicKeys class. + */ + public PublicLandMobileNetworkHomeNetworkPublicKeys() { + } + + /** + * Get the profileA property: This provides a mapping to identify which public key has been used for SUPI + * concealment using the Profile A Protection Scheme. + * + * @return the profileA value. + */ + public List profileA() { + return this.profileA; + } + + /** + * Set the profileA property: This provides a mapping to identify which public key has been used for SUPI + * concealment using the Profile A Protection Scheme. + * + * @param profileA the profileA value to set. + * @return the PublicLandMobileNetworkHomeNetworkPublicKeys object itself. + */ + public PublicLandMobileNetworkHomeNetworkPublicKeys withProfileA(List profileA) { + this.profileA = profileA; + return this; + } + + /** + * Get the profileB property: This provides a mapping to identify which public key has been used for SUPI + * concealment using the Profile B Protection Scheme. + * + * @return the profileB value. + */ + public List profileB() { + return this.profileB; + } + + /** + * Set the profileB property: This provides a mapping to identify which public key has been used for SUPI + * concealment using the Profile B Protection Scheme. + * + * @param profileB the profileB value to set. + * @return the PublicLandMobileNetworkHomeNetworkPublicKeys object itself. + */ + public PublicLandMobileNetworkHomeNetworkPublicKeys withProfileB(List profileB) { + this.profileB = profileB; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (profileA() != null) { + profileA().forEach(e -> e.validate()); + } + if (profileB() != null) { + profileB().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/QosPolicy.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/QosPolicy.java index b8068397c6657..8ec27a2278824 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/QosPolicy.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/QosPolicy.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** QoS policy. */ +/** + * QoS policy. + */ @Fluent public class QosPolicy { /* @@ -50,7 +52,9 @@ public class QosPolicy { @JsonProperty(value = "maximumBitRate", required = true) private Ambr maximumBitRate; - /** Creates an instance of QosPolicy class. */ + /** + * Creates an instance of QosPolicy class. + */ public QosPolicy() { } @@ -58,7 +62,7 @@ public QosPolicy() { * Get the fiveQi property: 5G QoS Flow Indicator value. The 5QI identifies a specific QoS forwarding treatment to * be provided to a flow. See 3GPP TS23.501 section 5.7.2.1 for a full description of the 5QI parameter, and table * 5.7.4-1 for the definition the 5QI values. - * + * * @return the fiveQi value. */ public Integer fiveQi() { @@ -69,7 +73,7 @@ public Integer fiveQi() { * Set the fiveQi property: 5G QoS Flow Indicator value. The 5QI identifies a specific QoS forwarding treatment to * be provided to a flow. See 3GPP TS23.501 section 5.7.2.1 for a full description of the 5QI parameter, and table * 5.7.4-1 for the definition the 5QI values. - * + * * @param fiveQi the fiveQi value to set. * @return the QosPolicy object itself. */ @@ -81,9 +85,10 @@ public QosPolicy withFiveQi(Integer fiveQi) { /** * Get the allocationAndRetentionPriorityLevel property: QoS Flow allocation and retention priority (ARP) level. * Flows with higher priority preempt flows with lower priority, if the settings of `preemptionCapability` and - * `preemptionVulnerability` allow it. 1 is the highest level of priority. If this field is not specified then `5qi` - * is used to derive the ARP value. See 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP parameters. - * + * `preemptionVulnerability` allow it. 1 is the highest level of priority. If this field is not specified then + * `5qi` is used to derive the ARP value. See 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP + * parameters. + * * @return the allocationAndRetentionPriorityLevel value. */ public Integer allocationAndRetentionPriorityLevel() { @@ -93,9 +98,10 @@ public Integer allocationAndRetentionPriorityLevel() { /** * Set the allocationAndRetentionPriorityLevel property: QoS Flow allocation and retention priority (ARP) level. * Flows with higher priority preempt flows with lower priority, if the settings of `preemptionCapability` and - * `preemptionVulnerability` allow it. 1 is the highest level of priority. If this field is not specified then `5qi` - * is used to derive the ARP value. See 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP parameters. - * + * `preemptionVulnerability` allow it. 1 is the highest level of priority. If this field is not specified then + * `5qi` is used to derive the ARP value. See 3GPP TS23.501 section 5.7.2.2 for a full description of the ARP + * parameters. + * * @param allocationAndRetentionPriorityLevel the allocationAndRetentionPriorityLevel value to set. * @return the QosPolicy object itself. */ @@ -108,7 +114,7 @@ public QosPolicy withAllocationAndRetentionPriorityLevel(Integer allocationAndRe * Get the preemptionCapability property: QoS Flow preemption capability. The preemption capability of a QoS Flow * controls whether it can preempt another QoS Flow with a lower priority level. See 3GPP TS23.501 section 5.7.2.2 * for a full description of the ARP parameters. - * + * * @return the preemptionCapability value. */ public PreemptionCapability preemptionCapability() { @@ -119,7 +125,7 @@ public PreemptionCapability preemptionCapability() { * Set the preemptionCapability property: QoS Flow preemption capability. The preemption capability of a QoS Flow * controls whether it can preempt another QoS Flow with a lower priority level. See 3GPP TS23.501 section 5.7.2.2 * for a full description of the ARP parameters. - * + * * @param preemptionCapability the preemptionCapability value to set. * @return the QosPolicy object itself. */ @@ -132,7 +138,7 @@ public QosPolicy withPreemptionCapability(PreemptionCapability preemptionCapabil * Get the preemptionVulnerability property: QoS Flow preemption vulnerability. The preemption vulnerability of a * QoS Flow controls whether it can be preempted by a QoS Flow with a higher priority level. See 3GPP TS23.501 * section 5.7.2.2 for a full description of the ARP parameters. - * + * * @return the preemptionVulnerability value. */ public PreemptionVulnerability preemptionVulnerability() { @@ -143,7 +149,7 @@ public PreemptionVulnerability preemptionVulnerability() { * Set the preemptionVulnerability property: QoS Flow preemption vulnerability. The preemption vulnerability of a * QoS Flow controls whether it can be preempted by a QoS Flow with a higher priority level. See 3GPP TS23.501 * section 5.7.2.2 for a full description of the ARP parameters. - * + * * @param preemptionVulnerability the preemptionVulnerability value to set. * @return the QosPolicy object itself. */ @@ -155,7 +161,7 @@ public QosPolicy withPreemptionVulnerability(PreemptionVulnerability preemptionV /** * Get the maximumBitRate property: The maximum bit rate (MBR) for all service data flows that use this data flow * policy rule or service. - * + * * @return the maximumBitRate value. */ public Ambr maximumBitRate() { @@ -165,7 +171,7 @@ public Ambr maximumBitRate() { /** * Set the maximumBitRate property: The maximum bit rate (MBR) for all service data flows that use this data flow * policy rule or service. - * + * * @param maximumBitRate the maximumBitRate value to set. * @return the QosPolicy object itself. */ @@ -176,14 +182,13 @@ public QosPolicy withMaximumBitRate(Ambr maximumBitRate) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (maximumBitRate() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property maximumBitRate in model QosPolicy")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property maximumBitRate in model QosPolicy")); } else { maximumBitRate().validate(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RatType.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RatType.java new file mode 100644 index 0000000000000..45ec715996990 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RatType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * RAT Type. + */ +public final class RatType extends ExpandableStringEnum { + /** + * Static value 4G for RatType. + */ + public static final RatType FOURG = fromString("4G"); + + /** + * Static value 5G for RatType. + */ + public static final RatType FIVEG = fromString("5G"); + + /** + * Creates a new instance of RatType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public RatType() { + } + + /** + * Creates or finds a RatType from its string representation. + * + * @param name a name to look for. + * @return the corresponding RatType. + */ + @JsonCreator + public static RatType fromString(String name) { + return fromString(name, RatType.class); + } + + /** + * Gets known RatType values. + * + * @return known RatType values. + */ + public static Collection values() { + return values(RatType.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RecommendedVersion.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RecommendedVersion.java index d9c5e366f484b..e2f8e5a17a110 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RecommendedVersion.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RecommendedVersion.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether this is the recommended version to use for new packet core control plane deployments. */ +/** + * Indicates whether this is the recommended version to use for new packet core control plane deployments. + */ public final class RecommendedVersion extends ExpandableStringEnum { - /** Static value Recommended for RecommendedVersion. */ + /** + * Static value Recommended for RecommendedVersion. + */ public static final RecommendedVersion RECOMMENDED = fromString("Recommended"); - /** Static value NotRecommended for RecommendedVersion. */ + /** + * Static value NotRecommended for RecommendedVersion. + */ public static final RecommendedVersion NOT_RECOMMENDED = fromString("NotRecommended"); /** * Creates a new instance of RecommendedVersion value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public RecommendedVersion() { /** * Creates or finds a RecommendedVersion from its string representation. - * + * * @param name a name to look for. * @return the corresponding RecommendedVersion. */ @@ -38,7 +44,7 @@ public static RecommendedVersion fromString(String name) { /** * Gets known RecommendedVersion values. - * + * * @return known RecommendedVersion values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ReinstallRequired.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ReinstallRequired.java index 816db9d503c6a..ca140c6108f6a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ReinstallRequired.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ReinstallRequired.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Whether a reinstall of the packet core is required to pick up the latest configuration changes. */ +/** + * Whether a reinstall of the packet core is required to pick up the latest configuration changes. + */ public final class ReinstallRequired extends ExpandableStringEnum { - /** Static value Required for ReinstallRequired. */ + /** + * Static value Required for ReinstallRequired. + */ public static final ReinstallRequired REQUIRED = fromString("Required"); - /** Static value NotRequired for ReinstallRequired. */ + /** + * Static value NotRequired for ReinstallRequired. + */ public static final ReinstallRequired NOT_REQUIRED = fromString("NotRequired"); /** * Creates a new instance of ReinstallRequired value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ReinstallRequired() { /** * Creates or finds a ReinstallRequired from its string representation. - * + * * @param name a name to look for. * @return the corresponding ReinstallRequired. */ @@ -38,7 +44,7 @@ public static ReinstallRequired fromString(String name) { /** * Gets known ReinstallRequired values. - * + * * @return known ReinstallRequired values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RrcEstablishmentCause.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RrcEstablishmentCause.java new file mode 100644 index 0000000000000..2f71608c5c0ca --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/RrcEstablishmentCause.java @@ -0,0 +1,73 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Radio connection establishment cause. + */ +public final class RrcEstablishmentCause extends ExpandableStringEnum { + /** + * Static value Emergency for RrcEstablishmentCause. + */ + public static final RrcEstablishmentCause EMERGENCY = fromString("Emergency"); + + /** + * Static value MobileOriginatedSignaling for RrcEstablishmentCause. + */ + public static final RrcEstablishmentCause MOBILE_ORIGINATED_SIGNALING = fromString("MobileOriginatedSignaling"); + + /** + * Static value MobileTerminatedSignaling for RrcEstablishmentCause. + */ + public static final RrcEstablishmentCause MOBILE_TERMINATED_SIGNALING = fromString("MobileTerminatedSignaling"); + + /** + * Static value MobileOriginatedData for RrcEstablishmentCause. + */ + public static final RrcEstablishmentCause MOBILE_ORIGINATED_DATA = fromString("MobileOriginatedData"); + + /** + * Static value MobileTerminatedData for RrcEstablishmentCause. + */ + public static final RrcEstablishmentCause MOBILE_TERMINATED_DATA = fromString("MobileTerminatedData"); + + /** + * Static value SMS for RrcEstablishmentCause. + */ + public static final RrcEstablishmentCause SMS = fromString("SMS"); + + /** + * Creates a new instance of RrcEstablishmentCause value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public RrcEstablishmentCause() { + } + + /** + * Creates or finds a RrcEstablishmentCause from its string representation. + * + * @param name a name to look for. + * @return the corresponding RrcEstablishmentCause. + */ + @JsonCreator + public static RrcEstablishmentCause fromString(String name) { + return fromString(name, RrcEstablishmentCause.class); + } + + /** + * Gets known RrcEstablishmentCause values. + * + * @return known RrcEstablishmentCause values. + */ + public static Collection values() { + return values(RrcEstablishmentCause.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SdfDirection.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SdfDirection.java index 3e768cafe7fe7..a7aa94a1491f2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SdfDirection.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SdfDirection.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Service data flow direction. */ +/** + * Service data flow direction. + */ public final class SdfDirection extends ExpandableStringEnum { - /** Static value Uplink for SdfDirection. */ + /** + * Static value Uplink for SdfDirection. + */ public static final SdfDirection UPLINK = fromString("Uplink"); - /** Static value Downlink for SdfDirection. */ + /** + * Static value Downlink for SdfDirection. + */ public static final SdfDirection DOWNLINK = fromString("Downlink"); - /** Static value Bidirectional for SdfDirection. */ + /** + * Static value Bidirectional for SdfDirection. + */ public static final SdfDirection BIDIRECTIONAL = fromString("Bidirectional"); /** * Creates a new instance of SdfDirection value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public SdfDirection() { /** * Creates or finds a SdfDirection from its string representation. - * + * * @param name a name to look for. * @return the corresponding SdfDirection. */ @@ -41,7 +49,7 @@ public static SdfDirection fromString(String name) { /** * Gets known SdfDirection values. - * + * * @return known SdfDirection values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Service.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Service.java index 348368119efea..a33ae1502de1f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Service.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Service.java @@ -11,53 +11,55 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Service. */ +/** + * An immutable client-side representation of Service. + */ public interface Service { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the service resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); @@ -66,7 +68,7 @@ public interface Service { * Gets the servicePrecedence property: A precedence value that is used to decide between services when identifying * the QoS values to use for a particular SIM. A lower value means a higher priority. This value should be unique * among all services configured in the mobile network. - * + * * @return the servicePrecedence value. */ int servicePrecedence(); @@ -75,67 +77,71 @@ public interface Service { * Gets the serviceQosPolicy property: The QoS policy to use for packets matching this service. This can be * overridden for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null * then the UE's SIM policy will define the QoS settings. - * + * * @return the serviceQosPolicy value. */ QosPolicy serviceQosPolicy(); /** * Gets the pccRules property: The set of data flow policy rules that make up this service. - * + * * @return the pccRules value. */ List pccRules(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.ServiceInner object. - * + * * @return the inner object. */ ServiceInner innerModel(); - /** The entirety of the Service definition. */ + /** + * The entirety of the Service definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithServicePrecedence, - DefinitionStages.WithPccRules, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithLocation, DefinitionStages.WithParentResource, + DefinitionStages.WithServicePrecedence, DefinitionStages.WithPccRules, DefinitionStages.WithCreate { } - /** The Service definition stages. */ + /** + * The Service definition stages. + */ interface DefinitionStages { - /** The first stage of the Service definition. */ + /** + * The first stage of the Service definition. + */ interface Blank extends WithLocation { } - /** The stage of the Service definition allowing to specify location. */ + /** + * The stage of the Service definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -143,18 +149,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the Service definition allowing to specify parent resource. */ + /** + * The stage of the Service definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, mobileNetworkName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @return the next definition stage. @@ -162,26 +170,30 @@ interface WithParentResource { WithServicePrecedence withExistingMobileNetwork(String resourceGroupName, String mobileNetworkName); } - /** The stage of the Service definition allowing to specify servicePrecedence. */ + /** + * The stage of the Service definition allowing to specify servicePrecedence. + */ interface WithServicePrecedence { /** * Specifies the servicePrecedence property: A precedence value that is used to decide between services when * identifying the QoS values to use for a particular SIM. A lower value means a higher priority. This value * should be unique among all services configured in the mobile network.. - * + * * @param servicePrecedence A precedence value that is used to decide between services when identifying the - * QoS values to use for a particular SIM. A lower value means a higher priority. This value should be - * unique among all services configured in the mobile network. + * QoS values to use for a particular SIM. A lower value means a higher priority. This value should be + * unique among all services configured in the mobile network. * @return the next definition stage. */ WithPccRules withServicePrecedence(int servicePrecedence); } - /** The stage of the Service definition allowing to specify pccRules. */ + /** + * The stage of the Service definition allowing to specify pccRules. + */ interface WithPccRules { /** * Specifies the pccRules property: The set of data flow policy rules that make up this service.. - * + * * @param pccRules The set of data flow policy rules that make up this service. * @return the next definition stage. */ @@ -195,41 +207,45 @@ interface WithPccRules { interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithServiceQosPolicy { /** * Executes the create request. - * + * * @return the created resource. */ Service create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Service create(Context context); } - /** The stage of the Service definition allowing to specify tags. */ + /** + * The stage of the Service definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Service definition allowing to specify serviceQosPolicy. */ + /** + * The stage of the Service definition allowing to specify serviceQosPolicy. + */ interface WithServiceQosPolicy { /** * Specifies the serviceQosPolicy property: The QoS policy to use for packets matching this service. This * can be overridden for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this * field is null then the UE's SIM policy will define the QoS settings.. - * + * * @param serviceQosPolicy The QoS policy to use for packets matching this service. This can be overridden - * for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null - * then the UE's SIM policy will define the QoS settings. + * for particular flows using the ruleQosPolicy field in a PccRuleConfiguration. If this field is null then + * the UE's SIM policy will define the QoS settings. * @return the next definition stage. */ WithCreate withServiceQosPolicy(QosPolicy serviceQosPolicy); @@ -238,36 +254,42 @@ interface WithServiceQosPolicy { /** * Begins update for the Service resource. - * + * * @return the stage of resource update. */ Service.Update update(); - /** The template for Service update. */ + /** + * The template for Service update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ Service apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Service apply(Context context); } - /** The Service update stages. */ + /** + * The Service update stages. + */ interface UpdateStages { - /** The stage of the Service update allowing to specify tags. */ + /** + * The stage of the Service update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -277,14 +299,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Service refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceDataFlowTemplate.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceDataFlowTemplate.java index f799ab4a407cd..bee0007db3b23 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceDataFlowTemplate.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceDataFlowTemplate.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Data flow template. */ +/** + * Data flow template. + */ @Fluent public final class ServiceDataFlowTemplate { /* @@ -53,14 +55,16 @@ public final class ServiceDataFlowTemplate { @JsonProperty(value = "ports") private List ports; - /** Creates an instance of ServiceDataFlowTemplate class. */ + /** + * Creates an instance of ServiceDataFlowTemplate class. + */ public ServiceDataFlowTemplate() { } /** * Get the templateName property: The name of the data flow template. This must be unique within the parent data * flow policy rule. You must not use any of the following reserved strings - `default`, `requested` or `service`. - * + * * @return the templateName value. */ public String templateName() { @@ -70,7 +74,7 @@ public String templateName() { /** * Set the templateName property: The name of the data flow template. This must be unique within the parent data * flow policy rule. You must not use any of the following reserved strings - `default`, `requested` or `service`. - * + * * @param templateName the templateName value to set. * @return the ServiceDataFlowTemplate object itself. */ @@ -81,7 +85,7 @@ public ServiceDataFlowTemplate withTemplateName(String templateName) { /** * Get the direction property: The direction of this flow. - * + * * @return the direction value. */ public SdfDirection direction() { @@ -90,7 +94,7 @@ public SdfDirection direction() { /** * Set the direction property: The direction of this flow. - * + * * @param direction the direction value to set. * @return the ServiceDataFlowTemplate object itself. */ @@ -101,11 +105,11 @@ public ServiceDataFlowTemplate withDirection(SdfDirection direction) { /** * Get the protocol property: A list of the allowed protocol(s) for this flow. If you want this flow to be able to - * use any protocol within the internet protocol suite, use the value `ip`. If you only want to allow a selection of - * protocols, you must use the corresponding IANA Assigned Internet Protocol Number for each protocol, as described - * in https://www.iana.org/assignments/protocol-numbers/protocol-numbers.xhtml. For example, for UDP, you must use - * 17. If you use the value `ip` then you must leave the field `port` unspecified. - * + * use any protocol within the internet protocol suite, use the value `ip`. If you only want to allow a selection + * of protocols, you must use the corresponding IANA Assigned Internet Protocol Number for each protocol, as + * described in https://www.iana.org/assignments/protocol-numbers/protocol-numbers.xhtml. For example, for UDP, you + * must use 17. If you use the value `ip` then you must leave the field `port` unspecified. + * * @return the protocol value. */ public List protocol() { @@ -114,11 +118,11 @@ public List protocol() { /** * Set the protocol property: A list of the allowed protocol(s) for this flow. If you want this flow to be able to - * use any protocol within the internet protocol suite, use the value `ip`. If you only want to allow a selection of - * protocols, you must use the corresponding IANA Assigned Internet Protocol Number for each protocol, as described - * in https://www.iana.org/assignments/protocol-numbers/protocol-numbers.xhtml. For example, for UDP, you must use - * 17. If you use the value `ip` then you must leave the field `port` unspecified. - * + * use any protocol within the internet protocol suite, use the value `ip`. If you only want to allow a selection + * of protocols, you must use the corresponding IANA Assigned Internet Protocol Number for each protocol, as + * described in https://www.iana.org/assignments/protocol-numbers/protocol-numbers.xhtml. For example, for UDP, you + * must use 17. If you use the value `ip` then you must leave the field `port` unspecified. + * * @param protocol the protocol value to set. * @return the ServiceDataFlowTemplate object itself. */ @@ -132,7 +136,7 @@ public ServiceDataFlowTemplate withProtocol(List protocol) { * allow connections on any IP address, use the value `any`. Otherwise, you must provide each of the remote IP * addresses to which the packet core instance will connect for this flow. You must provide each IP address in CIDR * notation, including the netmask (for example, 192.0.2.54/24). - * + * * @return the remoteIpList value. */ public List remoteIpList() { @@ -144,7 +148,7 @@ public List remoteIpList() { * allow connections on any IP address, use the value `any`. Otherwise, you must provide each of the remote IP * addresses to which the packet core instance will connect for this flow. You must provide each IP address in CIDR * notation, including the netmask (for example, 192.0.2.54/24). - * + * * @param remoteIpList the remoteIpList value to set. * @return the ServiceDataFlowTemplate object itself. */ @@ -159,7 +163,7 @@ public ServiceDataFlowTemplate withRemoteIpList(List remoteIpList) { * the `protocol` field. This is an optional setting. If you do not specify it then connections will be allowed on * all ports. Port ranges must be specified as <FirstPort>-<LastPort>. For example: [`8080`, * `8082-8085`]. - * + * * @return the ports value. */ public List ports() { @@ -172,7 +176,7 @@ public List ports() { * the `protocol` field. This is an optional setting. If you do not specify it then connections will be allowed on * all ports. Port ranges must be specified as <FirstPort>-<LastPort>. For example: [`8080`, * `8082-8085`]. - * + * * @param ports the ports value to set. * @return the ServiceDataFlowTemplate object itself. */ @@ -183,33 +187,25 @@ public ServiceDataFlowTemplate withPorts(List ports) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (templateName() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property templateName in model ServiceDataFlowTemplate")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property templateName in model ServiceDataFlowTemplate")); } if (direction() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property direction in model ServiceDataFlowTemplate")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property direction in model ServiceDataFlowTemplate")); } if (protocol() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property protocol in model ServiceDataFlowTemplate")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property protocol in model ServiceDataFlowTemplate")); } if (remoteIpList() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property remoteIpList in model ServiceDataFlowTemplate")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property remoteIpList in model ServiceDataFlowTemplate")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceListResult.java index 698f7372324b5..665de5b0f51a8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for services API service call. */ +/** + * Response for services API service call. + */ @Fluent public final class ServiceListResult { /* @@ -24,13 +26,15 @@ public final class ServiceListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ServiceListResult class. */ + /** + * Creates an instance of ServiceListResult class. + */ public ServiceListResult() { } /** * Get the value property: A list of services. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of services. - * + * * @param value the value value to set. * @return the ServiceListResult object itself. */ @@ -50,7 +54,7 @@ public ServiceListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceResourceId.java index 4dcf7913220b4..2e02efddb651e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/ServiceResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to a service resource. */ +/** + * Reference to a service resource. + */ @Fluent public final class ServiceResourceId { /* @@ -17,13 +19,15 @@ public final class ServiceResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of ServiceResourceId class. */ + /** + * Creates an instance of ServiceResourceId class. + */ public ServiceResourceId() { } /** * Get the id property: Service resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Service resource ID. - * + * * @param id the id value to set. * @return the ServiceResourceId object itself. */ @@ -43,14 +47,13 @@ public ServiceResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model ServiceResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model ServiceResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Services.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Services.java index f1453562d65bb..507d7d56777ed 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Services.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Services.java @@ -8,15 +8,17 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Services. */ +/** + * Resource collection API of Services. + */ public interface Services { /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -25,11 +27,11 @@ public interface Services { /** * Deletes the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -39,27 +41,27 @@ public interface Services { /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified service along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String serviceName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String serviceName, + Context context); /** * Gets information about the specified service. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param serviceName The name of the service. You must not use any of the following reserved strings - `default`, - * `requested` or `service`. + * `requested` or `service`. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -69,7 +71,7 @@ Response getWithResponse( /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -81,7 +83,7 @@ Response getWithResponse( /** * Gets all the services in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -94,7 +96,7 @@ Response getWithResponse( /** * Gets information about the specified service. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -105,7 +107,7 @@ Response getWithResponse( /** * Gets information about the specified service. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -117,7 +119,7 @@ Response getWithResponse( /** * Deletes the specified service. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -127,7 +129,7 @@ Response getWithResponse( /** * Deletes the specified service. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -138,7 +140,7 @@ Response getWithResponse( /** * Begins definition for a new Service resource. - * + * * @param name resource name. * @return the first stage of the new Service definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SignalingConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SignalingConfiguration.java index 29b68e1886e46..25d891ef685c8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SignalingConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SignalingConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Signaling configuration for the packet core. */ +/** + * Signaling configuration for the packet core. + */ @Fluent public final class SignalingConfiguration { /* @@ -16,13 +18,15 @@ public final class SignalingConfiguration { @JsonProperty(value = "nasReroute") private NasRerouteConfiguration nasReroute; - /** Creates an instance of SignalingConfiguration class. */ + /** + * Creates an instance of SignalingConfiguration class. + */ public SignalingConfiguration() { } /** * Get the nasReroute property: Configuration enabling 4G NAS reroute. - * + * * @return the nasReroute value. */ public NasRerouteConfiguration nasReroute() { @@ -31,7 +35,7 @@ public NasRerouteConfiguration nasReroute() { /** * Set the nasReroute property: Configuration enabling 4G NAS reroute. - * + * * @param nasReroute the nasReroute value to set. * @return the SignalingConfiguration object itself. */ @@ -42,7 +46,7 @@ public SignalingConfiguration withNasReroute(NasRerouteConfiguration nasReroute) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sim.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sim.java index b47624df45835..55a42dd6a5654 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sim.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sim.java @@ -10,60 +10,62 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Sim. */ +/** + * An immutable client-side representation of Sim. + */ public interface Sim { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the authenticationKey property: The Ki value for the SIM. - * + * * @return the authenticationKey value. */ String authenticationKey(); /** * Gets the operatorKeyCode property: The Opc value for the SIM. - * + * * @return the operatorKeyCode value. */ String operatorKeyCode(); /** * Gets the provisioningState property: The provisioning state of the SIM resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the simState property: The state of the SIM resource. - * + * * @return the simState value. */ SimState simState(); @@ -71,7 +73,7 @@ public interface Sim { /** * Gets the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that * site. - * + * * @return the siteProvisioningState value. */ Map siteProvisioningState(); @@ -79,14 +81,14 @@ public interface Sim { /** * Gets the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @return the internationalMobileSubscriberIdentity value. */ String internationalMobileSubscriberIdentity(); /** * Gets the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @return the integratedCircuitCardIdentifier value. */ String integratedCircuitCardIdentifier(); @@ -95,7 +97,7 @@ public interface Sim { * Gets the deviceType property: An optional free-form text field that can be used to record the device type this * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based * on this value. - * + * * @return the deviceType value. */ String deviceType(); @@ -103,7 +105,7 @@ public interface Sim { /** * Gets the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @return the simPolicy value. */ SimPolicyResourceId simPolicy(); @@ -111,14 +113,14 @@ public interface Sim { /** * Gets the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @return the staticIpConfiguration value. */ List staticIpConfiguration(); /** * Gets the vendorName property: The name of the SIM vendor who provided this SIM, if any. - * + * * @return the vendorName value. */ String vendorName(); @@ -126,60 +128,67 @@ public interface Sim { /** * Gets the vendorKeyFingerprint property: The public key fingerprint of the SIM vendor who provided this SIM, if * any. - * + * * @return the vendorKeyFingerprint value. */ String vendorKeyFingerprint(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.SimInner object. - * + * * @return the inner object. */ SimInner innerModel(); - /** The entirety of the Sim definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithParentResource, - DefinitionStages.WithInternationalMobileSubscriberIdentity, - DefinitionStages.WithCreate { + /** + * The entirety of the Sim definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, + DefinitionStages.WithInternationalMobileSubscriberIdentity, DefinitionStages.WithCreate { } - /** The Sim definition stages. */ + /** + * The Sim definition stages. + */ interface DefinitionStages { - /** The first stage of the Sim definition. */ + /** + * The first stage of the Sim definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Sim definition allowing to specify parent resource. */ + /** + * The stage of the Sim definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, simGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @return the next definition stage. */ - WithInternationalMobileSubscriberIdentity withExistingSimGroup( - String resourceGroupName, String simGroupName); + WithInternationalMobileSubscriberIdentity withExistingSimGroup(String resourceGroupName, + String simGroupName); } - /** The stage of the Sim definition allowing to specify internationalMobileSubscriberIdentity. */ + /** + * The stage of the Sim definition allowing to specify internationalMobileSubscriberIdentity. + */ interface WithInternationalMobileSubscriberIdentity { /** * Specifies the internationalMobileSubscriberIdentity property: The international mobile subscriber * identity (IMSI) for the SIM.. - * + * * @param internationalMobileSubscriberIdentity The international mobile subscriber identity (IMSI) for the - * SIM. + * SIM. * @return the next definition stage. */ WithCreate withInternationalMobileSubscriberIdentity(String internationalMobileSubscriberIdentity); @@ -189,98 +198,106 @@ interface WithInternationalMobileSubscriberIdentity { * The stage of the Sim definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithAuthenticationKey, - DefinitionStages.WithOperatorKeyCode, - DefinitionStages.WithIntegratedCircuitCardIdentifier, - DefinitionStages.WithDeviceType, - DefinitionStages.WithSimPolicy, - DefinitionStages.WithStaticIpConfiguration { + interface WithCreate extends DefinitionStages.WithAuthenticationKey, DefinitionStages.WithOperatorKeyCode, + DefinitionStages.WithIntegratedCircuitCardIdentifier, DefinitionStages.WithDeviceType, + DefinitionStages.WithSimPolicy, DefinitionStages.WithStaticIpConfiguration { /** * Executes the create request. - * + * * @return the created resource. */ Sim create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Sim create(Context context); } - /** The stage of the Sim definition allowing to specify authenticationKey. */ + /** + * The stage of the Sim definition allowing to specify authenticationKey. + */ interface WithAuthenticationKey { /** * Specifies the authenticationKey property: The Ki value for the SIM.. - * + * * @param authenticationKey The Ki value for the SIM. * @return the next definition stage. */ WithCreate withAuthenticationKey(String authenticationKey); } - /** The stage of the Sim definition allowing to specify operatorKeyCode. */ + /** + * The stage of the Sim definition allowing to specify operatorKeyCode. + */ interface WithOperatorKeyCode { /** * Specifies the operatorKeyCode property: The Opc value for the SIM.. - * + * * @param operatorKeyCode The Opc value for the SIM. * @return the next definition stage. */ WithCreate withOperatorKeyCode(String operatorKeyCode); } - /** The stage of the Sim definition allowing to specify integratedCircuitCardIdentifier. */ + /** + * The stage of the Sim definition allowing to specify integratedCircuitCardIdentifier. + */ interface WithIntegratedCircuitCardIdentifier { /** * Specifies the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the * SIM.. - * + * * @param integratedCircuitCardIdentifier The integrated circuit card ID (ICCID) for the SIM. * @return the next definition stage. */ WithCreate withIntegratedCircuitCardIdentifier(String integratedCircuitCardIdentifier); } - /** The stage of the Sim definition allowing to specify deviceType. */ + /** + * The stage of the Sim definition allowing to specify deviceType. + */ interface WithDeviceType { /** * Specifies the deviceType property: An optional free-form text field that can be used to record the device * type this SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped * and filtered based on this value.. - * + * * @param deviceType An optional free-form text field that can be used to record the device type this SIM is - * associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered - * based on this value. + * associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. * @return the next definition stage. */ WithCreate withDeviceType(String deviceType); } - /** The stage of the Sim definition allowing to specify simPolicy. */ + /** + * The stage of the Sim definition allowing to specify simPolicy. + */ interface WithSimPolicy { /** * Specifies the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same * location as the SIM.. - * + * * @param simPolicy The SIM policy used by this SIM. The SIM policy must be in the same location as the SIM. * @return the next definition stage. */ WithCreate withSimPolicy(SimPolicyResourceId simPolicy); } - /** The stage of the Sim definition allowing to specify staticIpConfiguration. */ + /** + * The stage of the Sim definition allowing to specify staticIpConfiguration. + */ interface WithStaticIpConfiguration { /** * Specifies the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each * address is assigned at a defined network scope, made up of {attached data network, slice}.. - * + * * @param staticIpConfiguration A list of static IP addresses assigned to this SIM. Each address is assigned - * at a defined network scope, made up of {attached data network, slice}. + * at a defined network scope, made up of {attached data network, slice}. * @return the next definition stage. */ WithCreate withStaticIpConfiguration(List staticIpConfiguration); @@ -289,106 +306,118 @@ interface WithStaticIpConfiguration { /** * Begins update for the Sim resource. - * + * * @return the stage of resource update. */ Sim.Update update(); - /** The template for Sim update. */ - interface Update - extends UpdateStages.WithAuthenticationKey, - UpdateStages.WithOperatorKeyCode, - UpdateStages.WithIntegratedCircuitCardIdentifier, - UpdateStages.WithDeviceType, - UpdateStages.WithSimPolicy, - UpdateStages.WithStaticIpConfiguration { + /** + * The template for Sim update. + */ + interface Update extends UpdateStages.WithAuthenticationKey, UpdateStages.WithOperatorKeyCode, + UpdateStages.WithIntegratedCircuitCardIdentifier, UpdateStages.WithDeviceType, UpdateStages.WithSimPolicy, + UpdateStages.WithStaticIpConfiguration { /** * Executes the update request. - * + * * @return the updated resource. */ Sim apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Sim apply(Context context); } - /** The Sim update stages. */ + /** + * The Sim update stages. + */ interface UpdateStages { - /** The stage of the Sim update allowing to specify authenticationKey. */ + /** + * The stage of the Sim update allowing to specify authenticationKey. + */ interface WithAuthenticationKey { /** * Specifies the authenticationKey property: The Ki value for the SIM.. - * + * * @param authenticationKey The Ki value for the SIM. * @return the next definition stage. */ Update withAuthenticationKey(String authenticationKey); } - /** The stage of the Sim update allowing to specify operatorKeyCode. */ + /** + * The stage of the Sim update allowing to specify operatorKeyCode. + */ interface WithOperatorKeyCode { /** * Specifies the operatorKeyCode property: The Opc value for the SIM.. - * + * * @param operatorKeyCode The Opc value for the SIM. * @return the next definition stage. */ Update withOperatorKeyCode(String operatorKeyCode); } - /** The stage of the Sim update allowing to specify integratedCircuitCardIdentifier. */ + /** + * The stage of the Sim update allowing to specify integratedCircuitCardIdentifier. + */ interface WithIntegratedCircuitCardIdentifier { /** * Specifies the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the * SIM.. - * + * * @param integratedCircuitCardIdentifier The integrated circuit card ID (ICCID) for the SIM. * @return the next definition stage. */ Update withIntegratedCircuitCardIdentifier(String integratedCircuitCardIdentifier); } - /** The stage of the Sim update allowing to specify deviceType. */ + /** + * The stage of the Sim update allowing to specify deviceType. + */ interface WithDeviceType { /** * Specifies the deviceType property: An optional free-form text field that can be used to record the device * type this SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped * and filtered based on this value.. - * + * * @param deviceType An optional free-form text field that can be used to record the device type this SIM is - * associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered - * based on this value. + * associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. * @return the next definition stage. */ Update withDeviceType(String deviceType); } - /** The stage of the Sim update allowing to specify simPolicy. */ + /** + * The stage of the Sim update allowing to specify simPolicy. + */ interface WithSimPolicy { /** * Specifies the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same * location as the SIM.. - * + * * @param simPolicy The SIM policy used by this SIM. The SIM policy must be in the same location as the SIM. * @return the next definition stage. */ Update withSimPolicy(SimPolicyResourceId simPolicy); } - /** The stage of the Sim update allowing to specify staticIpConfiguration. */ + /** + * The stage of the Sim update allowing to specify staticIpConfiguration. + */ interface WithStaticIpConfiguration { /** * Specifies the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each * address is assigned at a defined network scope, made up of {attached data network, slice}.. - * + * * @param staticIpConfiguration A list of static IP addresses assigned to this SIM. Each address is assigned - * at a defined network scope, made up of {attached data network, slice}. + * at a defined network scope, made up of {attached data network, slice}. * @return the next definition stage. */ Update withStaticIpConfiguration(List staticIpConfiguration); @@ -397,14 +426,14 @@ interface WithStaticIpConfiguration { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Sim refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimDeleteList.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimDeleteList.java index a138b4013e184..cd10d3a786a6c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimDeleteList.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimDeleteList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The SIMs to delete. */ +/** + * The SIMs to delete. + */ @Fluent public final class SimDeleteList { /* @@ -18,13 +20,15 @@ public final class SimDeleteList { @JsonProperty(value = "sims", required = true) private List sims; - /** Creates an instance of SimDeleteList class. */ + /** + * Creates an instance of SimDeleteList class. + */ public SimDeleteList() { } /** * Get the sims property: A list of SIM resource names to delete. - * + * * @return the sims value. */ public List sims() { @@ -33,7 +37,7 @@ public List sims() { /** * Set the sims property: A list of SIM resource names to delete. - * + * * @param sims the sims value to set. * @return the SimDeleteList object itself. */ @@ -44,14 +48,13 @@ public SimDeleteList withSims(List sims) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sims() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property sims in model SimDeleteList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sims in model SimDeleteList")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroup.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroup.java index b747e16877519..f669998423dba 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroup.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroup.java @@ -10,67 +10,69 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.SimGroupInner; import java.util.Map; -/** An immutable client-side representation of SimGroup. */ +/** + * An immutable client-side representation of SimGroup. + */ public interface SimGroup { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the identity property: The identity used to retrieve the encryption key from Azure key vault. - * + * * @return the identity value. */ ManagedServiceIdentity identity(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the SIM group resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the encryptionKey property: A key to encrypt the SIM data that belongs to this SIM group. - * + * * @return the encryptionKey value. */ KeyVaultKey encryptionKey(); @@ -78,58 +80,63 @@ public interface SimGroup { /** * Gets the mobileNetwork property: Mobile network that this SIM group belongs to. The mobile network must be in the * same location as the SIM group. - * + * * @return the mobileNetwork value. */ MobileNetworkResourceId mobileNetwork(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.SimGroupInner object. - * + * * @return the inner object. */ SimGroupInner innerModel(); - /** The entirety of the SimGroup definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the SimGroup definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The SimGroup definition stages. */ + /** + * The SimGroup definition stages. + */ interface DefinitionStages { - /** The first stage of the SimGroup definition. */ + /** + * The first stage of the SimGroup definition. + */ interface Blank extends WithLocation { } - /** The stage of the SimGroup definition allowing to specify location. */ + /** + * The stage of the SimGroup definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -137,18 +144,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the SimGroup definition allowing to specify parent resource. */ + /** + * The stage of the SimGroup definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ @@ -159,68 +168,73 @@ interface WithResourceGroup { * The stage of the SimGroup definition which contains all the minimum required properties for the resource to * be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithIdentity, - DefinitionStages.WithEncryptionKey, - DefinitionStages.WithMobileNetwork { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithEncryptionKey, DefinitionStages.WithMobileNetwork { /** * Executes the create request. - * + * * @return the created resource. */ SimGroup create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ SimGroup create(Context context); } - /** The stage of the SimGroup definition allowing to specify tags. */ + /** + * The stage of the SimGroup definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the SimGroup definition allowing to specify identity. */ + /** + * The stage of the SimGroup definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The identity used to retrieve the encryption key from Azure key vault.. - * + * * @param identity The identity used to retrieve the encryption key from Azure key vault. * @return the next definition stage. */ WithCreate withIdentity(ManagedServiceIdentity identity); } - /** The stage of the SimGroup definition allowing to specify encryptionKey. */ + /** + * The stage of the SimGroup definition allowing to specify encryptionKey. + */ interface WithEncryptionKey { /** * Specifies the encryptionKey property: A key to encrypt the SIM data that belongs to this SIM group.. - * + * * @param encryptionKey A key to encrypt the SIM data that belongs to this SIM group. * @return the next definition stage. */ WithCreate withEncryptionKey(KeyVaultKey encryptionKey); } - /** The stage of the SimGroup definition allowing to specify mobileNetwork. */ + /** + * The stage of the SimGroup definition allowing to specify mobileNetwork. + */ interface WithMobileNetwork { /** * Specifies the mobileNetwork property: Mobile network that this SIM group belongs to. The mobile network * must be in the same location as the SIM group.. - * + * * @param mobileNetwork Mobile network that this SIM group belongs to. The mobile network must be in the - * same location as the SIM group. + * same location as the SIM group. * @return the next definition stage. */ WithCreate withMobileNetwork(MobileNetworkResourceId mobileNetwork); @@ -229,47 +243,55 @@ interface WithMobileNetwork { /** * Begins update for the SimGroup resource. - * + * * @return the stage of resource update. */ SimGroup.Update update(); - /** The template for SimGroup update. */ + /** + * The template for SimGroup update. + */ interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity { /** * Executes the update request. - * + * * @return the updated resource. */ SimGroup apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ SimGroup apply(Context context); } - /** The SimGroup update stages. */ + /** + * The SimGroup update stages. + */ interface UpdateStages { - /** The stage of the SimGroup update allowing to specify tags. */ + /** + * The stage of the SimGroup update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the SimGroup update allowing to specify identity. */ + /** + * The stage of the SimGroup update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The managed service identity associated with this resource.. - * + * * @param identity The managed service identity associated with this resource. * @return the next definition stage. */ @@ -279,14 +301,14 @@ interface WithIdentity { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ SimGroup refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroupListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroupListResult.java index 8118d3c9c4629..ce95c85bc37b9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroupListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroupListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for list SIM groups API service call. */ +/** + * Response for list SIM groups API service call. + */ @Fluent public final class SimGroupListResult { /* @@ -24,13 +26,15 @@ public final class SimGroupListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of SimGroupListResult class. */ + /** + * Creates an instance of SimGroupListResult class. + */ public SimGroupListResult() { } /** * Get the value property: A list of SIM groups in a resource group. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of SIM groups in a resource group. - * + * * @param value the value value to set. * @return the SimGroupListResult object itself. */ @@ -50,7 +54,7 @@ public SimGroupListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroups.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroups.java index d4685510ca64d..313d34cb8127d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroups.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimGroups.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of SimGroups. */ +/** + * Resource collection API of SimGroups. + */ public interface SimGroups { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface SimGroups { /** * Deletes the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -35,7 +37,7 @@ public interface SimGroups { /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -48,7 +50,7 @@ public interface SimGroups { /** * Gets information about the specified SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -60,7 +62,7 @@ public interface SimGroups { /** * Gets all the SIM groups in a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all the SIM groups in a subscription as paginated response with {@link PagedIterable}. @@ -69,7 +71,7 @@ public interface SimGroups { /** * Gets all the SIM groups in a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -80,7 +82,7 @@ public interface SimGroups { /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -91,7 +93,7 @@ public interface SimGroups { /** * Gets all the SIM groups in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -103,7 +105,7 @@ public interface SimGroups { /** * Gets information about the specified SIM group. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -114,7 +116,7 @@ public interface SimGroups { /** * Gets information about the specified SIM group. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -126,7 +128,7 @@ public interface SimGroups { /** * Deletes the specified SIM group. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -136,7 +138,7 @@ public interface SimGroups { /** * Deletes the specified SIM group. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -147,7 +149,7 @@ public interface SimGroups { /** * Begins definition for a new SimGroup resource. - * + * * @param name resource name. * @return the first stage of the new SimGroup definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimListResult.java index 373a57b8cca47..8b3d0d60a7151 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for list SIMs API service call. */ +/** + * Response for list SIMs API service call. + */ @Fluent public final class SimListResult { /* @@ -24,13 +26,15 @@ public final class SimListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of SimListResult class. */ + /** + * Creates an instance of SimListResult class. + */ public SimListResult() { } /** * Get the value property: A list of SIMs in a resource group. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of SIMs in a resource group. - * + * * @param value the value value to set. * @return the SimListResult object itself. */ @@ -50,7 +54,7 @@ public SimListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimNameAndEncryptedProperties.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimNameAndEncryptedProperties.java index de85d2df8b4f1..96faff45bc76a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimNameAndEncryptedProperties.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimNameAndEncryptedProperties.java @@ -11,7 +11,9 @@ import java.util.List; import java.util.Map; -/** SIM name and encrypted properties. */ +/** + * SIM name and encrypted properties. + */ @Fluent public final class SimNameAndEncryptedProperties { /* @@ -26,13 +28,15 @@ public final class SimNameAndEncryptedProperties { @JsonProperty(value = "properties", required = true) private EncryptedSimPropertiesFormat innerProperties = new EncryptedSimPropertiesFormat(); - /** Creates an instance of SimNameAndEncryptedProperties class. */ + /** + * Creates an instance of SimNameAndEncryptedProperties class. + */ public SimNameAndEncryptedProperties() { } /** * Get the name property: The name of the SIM. - * + * * @return the name value. */ public String name() { @@ -41,7 +45,7 @@ public String name() { /** * Set the name property: The name of the SIM. - * + * * @param name the name value to set. * @return the SimNameAndEncryptedProperties object itself. */ @@ -52,7 +56,7 @@ public SimNameAndEncryptedProperties withName(String name) { /** * Get the innerProperties property: Encrypted SIM Properties. - * + * * @return the innerProperties value. */ private EncryptedSimPropertiesFormat innerProperties() { @@ -61,7 +65,7 @@ private EncryptedSimPropertiesFormat innerProperties() { /** * Get the encryptedCredentials property: The encrypted SIM credentials. - * + * * @return the encryptedCredentials value. */ public String encryptedCredentials() { @@ -70,7 +74,7 @@ public String encryptedCredentials() { /** * Set the encryptedCredentials property: The encrypted SIM credentials. - * + * * @param encryptedCredentials the encryptedCredentials value to set. * @return the SimNameAndEncryptedProperties object itself. */ @@ -84,7 +88,7 @@ public SimNameAndEncryptedProperties withEncryptedCredentials(String encryptedCr /** * Get the provisioningState property: The provisioning state of the SIM resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -93,7 +97,7 @@ public ProvisioningState provisioningState() { /** * Get the simState property: The state of the SIM resource. - * + * * @return the simState value. */ public SimState simState() { @@ -101,8 +105,9 @@ public SimState simState() { } /** - * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that site. - * + * Get the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM on that + * site. + * * @return the siteProvisioningState value. */ public Map siteProvisioningState() { @@ -112,7 +117,7 @@ public Map siteProvisioningState() { /** * Get the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @return the internationalMobileSubscriberIdentity value. */ public String internationalMobileSubscriberIdentity() { @@ -122,12 +127,12 @@ public String internationalMobileSubscriberIdentity() { /** * Set the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @param internationalMobileSubscriberIdentity the internationalMobileSubscriberIdentity value to set. * @return the SimNameAndEncryptedProperties object itself. */ - public SimNameAndEncryptedProperties withInternationalMobileSubscriberIdentity( - String internationalMobileSubscriberIdentity) { + public SimNameAndEncryptedProperties + withInternationalMobileSubscriberIdentity(String internationalMobileSubscriberIdentity) { if (this.innerProperties() == null) { this.innerProperties = new EncryptedSimPropertiesFormat(); } @@ -137,7 +142,7 @@ public SimNameAndEncryptedProperties withInternationalMobileSubscriberIdentity( /** * Get the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @return the integratedCircuitCardIdentifier value. */ public String integratedCircuitCardIdentifier() { @@ -146,7 +151,7 @@ public String integratedCircuitCardIdentifier() { /** * Set the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @param integratedCircuitCardIdentifier the integratedCircuitCardIdentifier value to set. * @return the SimNameAndEncryptedProperties object itself. */ @@ -159,10 +164,10 @@ public SimNameAndEncryptedProperties withIntegratedCircuitCardIdentifier(String } /** - * Get the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Get the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @return the deviceType value. */ public String deviceType() { @@ -170,10 +175,10 @@ public String deviceType() { } /** - * Set the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Set the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @param deviceType the deviceType value to set. * @return the SimNameAndEncryptedProperties object itself. */ @@ -188,7 +193,7 @@ public SimNameAndEncryptedProperties withDeviceType(String deviceType) { /** * Get the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @return the simPolicy value. */ public SimPolicyResourceId simPolicy() { @@ -198,7 +203,7 @@ public SimPolicyResourceId simPolicy() { /** * Set the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @param simPolicy the simPolicy value to set. * @return the SimNameAndEncryptedProperties object itself. */ @@ -213,7 +218,7 @@ public SimNameAndEncryptedProperties withSimPolicy(SimPolicyResourceId simPolicy /** * Get the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @return the staticIpConfiguration value. */ public List staticIpConfiguration() { @@ -223,7 +228,7 @@ public List staticIpConfiguration() { /** * Set the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @param staticIpConfiguration the staticIpConfiguration value to set. * @return the SimNameAndEncryptedProperties object itself. */ @@ -237,7 +242,7 @@ public SimNameAndEncryptedProperties withStaticIpConfiguration(List siteProvisioningState() { @@ -135,7 +140,7 @@ public Map siteProvisioningState() { /** * Get the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @return the internationalMobileSubscriberIdentity value. */ public String internationalMobileSubscriberIdentity() { @@ -145,12 +150,12 @@ public String internationalMobileSubscriberIdentity() { /** * Set the internationalMobileSubscriberIdentity property: The international mobile subscriber identity (IMSI) for * the SIM. - * + * * @param internationalMobileSubscriberIdentity the internationalMobileSubscriberIdentity value to set. * @return the SimNameAndProperties object itself. */ - public SimNameAndProperties withInternationalMobileSubscriberIdentity( - String internationalMobileSubscriberIdentity) { + public SimNameAndProperties + withInternationalMobileSubscriberIdentity(String internationalMobileSubscriberIdentity) { if (this.innerProperties() == null) { this.innerProperties = new SimPropertiesFormat(); } @@ -160,7 +165,7 @@ public SimNameAndProperties withInternationalMobileSubscriberIdentity( /** * Get the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @return the integratedCircuitCardIdentifier value. */ public String integratedCircuitCardIdentifier() { @@ -169,7 +174,7 @@ public String integratedCircuitCardIdentifier() { /** * Set the integratedCircuitCardIdentifier property: The integrated circuit card ID (ICCID) for the SIM. - * + * * @param integratedCircuitCardIdentifier the integratedCircuitCardIdentifier value to set. * @return the SimNameAndProperties object itself. */ @@ -182,10 +187,10 @@ public SimNameAndProperties withIntegratedCircuitCardIdentifier(String integrate } /** - * Get the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Get the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @return the deviceType value. */ public String deviceType() { @@ -193,10 +198,10 @@ public String deviceType() { } /** - * Set the deviceType property: An optional free-form text field that can be used to record the device type this SIM - * is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered based on - * this value. - * + * Set the deviceType property: An optional free-form text field that can be used to record the device type this + * SIM is associated with, for example 'Video camera'. The Azure portal allows SIMs to be grouped and filtered + * based on this value. + * * @param deviceType the deviceType value to set. * @return the SimNameAndProperties object itself. */ @@ -211,7 +216,7 @@ public SimNameAndProperties withDeviceType(String deviceType) { /** * Get the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @return the simPolicy value. */ public SimPolicyResourceId simPolicy() { @@ -221,7 +226,7 @@ public SimPolicyResourceId simPolicy() { /** * Set the simPolicy property: The SIM policy used by this SIM. The SIM policy must be in the same location as the * SIM. - * + * * @param simPolicy the simPolicy value to set. * @return the SimNameAndProperties object itself. */ @@ -236,7 +241,7 @@ public SimNameAndProperties withSimPolicy(SimPolicyResourceId simPolicy) { /** * Get the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @return the staticIpConfiguration value. */ public List staticIpConfiguration() { @@ -246,7 +251,7 @@ public List staticIpConfiguration() { /** * Set the staticIpConfiguration property: A list of static IP addresses assigned to this SIM. Each address is * assigned at a defined network scope, made up of {attached data network, slice}. - * + * * @param staticIpConfiguration the staticIpConfiguration value to set. * @return the SimNameAndProperties object itself. */ @@ -260,7 +265,7 @@ public SimNameAndProperties withStaticIpConfiguration(List getWithResponse( - String resourceGroupName, String mobileNetworkName, String simPolicyName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String simPolicyName, + Context context); /** * Gets information about the specified SIM policy. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param simPolicyName The name of the SIM policy. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Gets all the SIM policies in a mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -90,7 +92,7 @@ Response getWithResponse( /** * Gets information about the specified SIM policy. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -101,7 +103,7 @@ Response getWithResponse( /** * Gets information about the specified SIM policy. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,7 +115,7 @@ Response getWithResponse( /** * Deletes the specified SIM policy. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -123,7 +125,7 @@ Response getWithResponse( /** * Deletes the specified SIM policy. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -134,7 +136,7 @@ Response getWithResponse( /** * Begins definition for a new SimPolicy resource. - * + * * @param name resource name. * @return the first stage of the new SimPolicy definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicy.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicy.java index 19464b46735e9..17849a0d4628e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicy.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicy.java @@ -11,53 +11,55 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of SimPolicy. */ +/** + * An immutable client-side representation of SimPolicy. + */ public interface SimPolicy { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the SIM policy resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); @@ -65,7 +67,7 @@ public interface SimPolicy { /** * Gets the siteProvisioningState property: A dictionary of sites to the provisioning state of this SIM policy on * that site. - * + * * @return the siteProvisioningState value. */ Map siteProvisioningState(); @@ -73,7 +75,7 @@ public interface SimPolicy { /** * Gets the ueAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of all PDU sessions of a given * UE. See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. - * + * * @return the ueAmbr value. */ Ambr ueAmbr(); @@ -81,7 +83,7 @@ public interface SimPolicy { /** * Gets the defaultSlice property: The default slice to use if the UE does not explicitly specify it. This slice * must exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. - * + * * @return the defaultSlice value. */ SliceResourceId defaultSlice(); @@ -89,7 +91,7 @@ public interface SimPolicy { /** * Gets the rfspIndex property: RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This is an * optional setting and by default is unspecified. - * + * * @return the rfspIndex value. */ Integer rfspIndex(); @@ -97,7 +99,7 @@ public interface SimPolicy { /** * Gets the registrationTimer property: UE periodic registration update timer (5G) or UE periodic tracking area * update timer (4G), in seconds. - * + * * @return the registrationTimer value. */ Integer registrationTimer(); @@ -105,61 +107,64 @@ public interface SimPolicy { /** * Gets the sliceConfigurations property: The allowed slices and the settings to use for them. The list must not * contain duplicate items and must contain at least one item. - * + * * @return the sliceConfigurations value. */ List sliceConfigurations(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.SimPolicyInner object. - * + * * @return the inner object. */ SimPolicyInner innerModel(); - /** The entirety of the SimPolicy definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithUeAmbr, - DefinitionStages.WithDefaultSlice, - DefinitionStages.WithSliceConfigurations, - DefinitionStages.WithCreate { + /** + * The entirety of the SimPolicy definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithUeAmbr, DefinitionStages.WithDefaultSlice, + DefinitionStages.WithSliceConfigurations, DefinitionStages.WithCreate { } - /** The SimPolicy definition stages. */ + /** + * The SimPolicy definition stages. + */ interface DefinitionStages { - /** The first stage of the SimPolicy definition. */ + /** + * The first stage of the SimPolicy definition. + */ interface Blank extends WithLocation { } - /** The stage of the SimPolicy definition allowing to specify location. */ + /** + * The stage of the SimPolicy definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -167,18 +172,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the SimPolicy definition allowing to specify parent resource. */ + /** + * The stage of the SimPolicy definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, mobileNetworkName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @return the next definition stage. @@ -186,41 +193,47 @@ interface WithParentResource { WithUeAmbr withExistingMobileNetwork(String resourceGroupName, String mobileNetworkName); } - /** The stage of the SimPolicy definition allowing to specify ueAmbr. */ + /** + * The stage of the SimPolicy definition allowing to specify ueAmbr. + */ interface WithUeAmbr { /** * Specifies the ueAmbr property: Aggregate maximum bit rate across all non-GBR QoS flows of all PDU * sessions of a given UE. See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR.. - * + * * @param ueAmbr Aggregate maximum bit rate across all non-GBR QoS flows of all PDU sessions of a given UE. - * See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. + * See 3GPP TS23.501 section 5.7.2.6 for a full description of the UE-AMBR. * @return the next definition stage. */ WithDefaultSlice withUeAmbr(Ambr ueAmbr); } - /** The stage of the SimPolicy definition allowing to specify defaultSlice. */ + /** + * The stage of the SimPolicy definition allowing to specify defaultSlice. + */ interface WithDefaultSlice { /** * Specifies the defaultSlice property: The default slice to use if the UE does not explicitly specify it. * This slice must exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM * policy.. - * + * * @param defaultSlice The default slice to use if the UE does not explicitly specify it. This slice must - * exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. + * exist in the `sliceConfigurations` map. The slice must be in the same location as the SIM policy. * @return the next definition stage. */ WithSliceConfigurations withDefaultSlice(SliceResourceId defaultSlice); } - /** The stage of the SimPolicy definition allowing to specify sliceConfigurations. */ + /** + * The stage of the SimPolicy definition allowing to specify sliceConfigurations. + */ interface WithSliceConfigurations { /** * Specifies the sliceConfigurations property: The allowed slices and the settings to use for them. The list * must not contain duplicate items and must contain at least one item.. - * + * * @param sliceConfigurations The allowed slices and the settings to use for them. The list must not contain - * duplicate items and must contain at least one item. + * duplicate items and must contain at least one item. * @return the next definition stage. */ WithCreate withSliceConfigurations(List sliceConfigurations); @@ -234,52 +247,58 @@ interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithRfspIndex, DefinitionStages.WithRegistrationTimer { /** * Executes the create request. - * + * * @return the created resource. */ SimPolicy create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ SimPolicy create(Context context); } - /** The stage of the SimPolicy definition allowing to specify tags. */ + /** + * The stage of the SimPolicy definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the SimPolicy definition allowing to specify rfspIndex. */ + /** + * The stage of the SimPolicy definition allowing to specify rfspIndex. + */ interface WithRfspIndex { /** * Specifies the rfspIndex property: RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This * is an optional setting and by default is unspecified.. - * + * * @param rfspIndex RAT/Frequency Selection Priority Index, defined in 3GPP TS 36.413. This is an optional - * setting and by default is unspecified. + * setting and by default is unspecified. * @return the next definition stage. */ WithCreate withRfspIndex(Integer rfspIndex); } - /** The stage of the SimPolicy definition allowing to specify registrationTimer. */ + /** + * The stage of the SimPolicy definition allowing to specify registrationTimer. + */ interface WithRegistrationTimer { /** * Specifies the registrationTimer property: UE periodic registration update timer (5G) or UE periodic * tracking area update timer (4G), in seconds.. - * + * * @param registrationTimer UE periodic registration update timer (5G) or UE periodic tracking area update - * timer (4G), in seconds. + * timer (4G), in seconds. * @return the next definition stage. */ WithCreate withRegistrationTimer(Integer registrationTimer); @@ -288,36 +307,42 @@ interface WithRegistrationTimer { /** * Begins update for the SimPolicy resource. - * + * * @return the stage of resource update. */ SimPolicy.Update update(); - /** The template for SimPolicy update. */ + /** + * The template for SimPolicy update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ SimPolicy apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ SimPolicy apply(Context context); } - /** The SimPolicy update stages. */ + /** + * The SimPolicy update stages. + */ interface UpdateStages { - /** The stage of the SimPolicy update allowing to specify tags. */ + /** + * The stage of the SimPolicy update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -327,14 +352,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ SimPolicy refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyListResult.java index 13bc194cfa0e8..7b114bf954dc3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for SIM policies API service call. */ +/** + * Response for SIM policies API service call. + */ @Fluent public final class SimPolicyListResult { /* @@ -24,13 +26,15 @@ public final class SimPolicyListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of SimPolicyListResult class. */ + /** + * Creates an instance of SimPolicyListResult class. + */ public SimPolicyListResult() { } /** * Get the value property: A list of SIM policies. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of SIM policies. - * + * * @param value the value value to set. * @return the SimPolicyListResult object itself. */ @@ -50,7 +54,7 @@ public SimPolicyListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyResourceId.java index 86b82e4bc4220..9473ee37e8457 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimPolicyResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to a SIM policy resource. */ +/** + * Reference to a SIM policy resource. + */ @Fluent public final class SimPolicyResourceId { /* @@ -17,13 +19,15 @@ public final class SimPolicyResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of SimPolicyResourceId class. */ + /** + * Creates an instance of SimPolicyResourceId class. + */ public SimPolicyResourceId() { } /** * Get the id property: SIM policy resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: SIM policy resource ID. - * + * * @param id the id value to set. * @return the SimPolicyResourceId object itself. */ @@ -43,14 +47,13 @@ public SimPolicyResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model SimPolicyResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model SimPolicyResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimState.java index 72813dd494102..cb589b72537a8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimState.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The state of the SIM resource. */ +/** + * The state of the SIM resource. + */ public final class SimState extends ExpandableStringEnum { - /** Static value Disabled for SimState. */ + /** + * Static value Disabled for SimState. + */ public static final SimState DISABLED = fromString("Disabled"); - /** Static value Enabled for SimState. */ + /** + * Static value Enabled for SimState. + */ public static final SimState ENABLED = fromString("Enabled"); - /** Static value Invalid for SimState. */ + /** + * Static value Invalid for SimState. + */ public static final SimState INVALID = fromString("Invalid"); /** * Creates a new instance of SimState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public SimState() { /** * Creates or finds a SimState from its string representation. - * + * * @param name a name to look for. * @return the corresponding SimState. */ @@ -41,7 +49,7 @@ public static SimState fromString(String name) { /** * Gets known SimState values. - * + * * @return known SimState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpProperties.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpProperties.java index 0029802d1c5f4..b40d156f4e1fb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpProperties.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Static IP configuration for a SIM, scoped to a particular attached data network and slice. */ +/** + * Static IP configuration for a SIM, scoped to a particular attached data network and slice. + */ @Fluent public final class SimStaticIpProperties { /* @@ -31,7 +33,9 @@ public final class SimStaticIpProperties { @JsonProperty(value = "staticIp") private SimStaticIpPropertiesStaticIp staticIp; - /** Creates an instance of SimStaticIpProperties class. */ + /** + * Creates an instance of SimStaticIpProperties class. + */ public SimStaticIpProperties() { } @@ -39,7 +43,7 @@ public SimStaticIpProperties() { * Get the attachedDataNetwork property: The attached data network on which the static IP address will be used. The * combination of attached data network and slice defines the network scope of the IP address. The attached data * network must be in the same location as the SIM. - * + * * @return the attachedDataNetwork value. */ public AttachedDataNetworkResourceId attachedDataNetwork() { @@ -50,7 +54,7 @@ public AttachedDataNetworkResourceId attachedDataNetwork() { * Set the attachedDataNetwork property: The attached data network on which the static IP address will be used. The * combination of attached data network and slice defines the network scope of the IP address. The attached data * network must be in the same location as the SIM. - * + * * @param attachedDataNetwork the attachedDataNetwork value to set. * @return the SimStaticIpProperties object itself. */ @@ -63,7 +67,7 @@ public SimStaticIpProperties withAttachedDataNetwork(AttachedDataNetworkResource * Get the slice property: The network slice on which the static IP address will be used. The combination of * attached data network and slice defines the network scope of the IP address. The slice must be in the same * location as the SIM. - * + * * @return the slice value. */ public SliceResourceId slice() { @@ -74,7 +78,7 @@ public SliceResourceId slice() { * Set the slice property: The network slice on which the static IP address will be used. The combination of * attached data network and slice defines the network scope of the IP address. The slice must be in the same * location as the SIM. - * + * * @param slice the slice value to set. * @return the SimStaticIpProperties object itself. */ @@ -85,7 +89,7 @@ public SimStaticIpProperties withSlice(SliceResourceId slice) { /** * Get the staticIp property: The static IP configuration for the SIM to use at the defined network scope. - * + * * @return the staticIp value. */ public SimStaticIpPropertiesStaticIp staticIp() { @@ -94,7 +98,7 @@ public SimStaticIpPropertiesStaticIp staticIp() { /** * Set the staticIp property: The static IP configuration for the SIM to use at the defined network scope. - * + * * @param staticIp the staticIp value to set. * @return the SimStaticIpProperties object itself. */ @@ -105,7 +109,7 @@ public SimStaticIpProperties withStaticIp(SimStaticIpPropertiesStaticIp staticIp /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpPropertiesStaticIp.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpPropertiesStaticIp.java index 687e174facd24..4688dfd76cbef 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpPropertiesStaticIp.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimStaticIpPropertiesStaticIp.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The static IP configuration for the SIM to use at the defined network scope. */ +/** + * The static IP configuration for the SIM to use at the defined network scope. + */ @Fluent public final class SimStaticIpPropertiesStaticIp { /* @@ -17,14 +19,16 @@ public final class SimStaticIpPropertiesStaticIp { @JsonProperty(value = "ipv4Address") private String ipv4Address; - /** Creates an instance of SimStaticIpPropertiesStaticIp class. */ + /** + * Creates an instance of SimStaticIpPropertiesStaticIp class. + */ public SimStaticIpPropertiesStaticIp() { } /** - * Get the ipv4Address property: The IPv4 address assigned to the SIM at this network scope. This address must be in - * the userEquipmentStaticAddressPoolPrefix defined in the attached data network. - * + * Get the ipv4Address property: The IPv4 address assigned to the SIM at this network scope. This address must be + * in the userEquipmentStaticAddressPoolPrefix defined in the attached data network. + * * @return the ipv4Address value. */ public String ipv4Address() { @@ -32,9 +36,9 @@ public String ipv4Address() { } /** - * Set the ipv4Address property: The IPv4 address assigned to the SIM at this network scope. This address must be in - * the userEquipmentStaticAddressPoolPrefix defined in the attached data network. - * + * Set the ipv4Address property: The IPv4 address assigned to the SIM at this network scope. This address must be + * in the userEquipmentStaticAddressPoolPrefix defined in the attached data network. + * * @param ipv4Address the ipv4Address value to set. * @return the SimStaticIpPropertiesStaticIp object itself. */ @@ -45,7 +49,7 @@ public SimStaticIpPropertiesStaticIp withIpv4Address(String ipv4Address) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimUploadList.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimUploadList.java index ca3425f054ff6..c52bce31f9a1e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimUploadList.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SimUploadList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The SIMs to upload. */ +/** + * The SIMs to upload. + */ @Fluent public final class SimUploadList { /* @@ -18,13 +20,15 @@ public final class SimUploadList { @JsonProperty(value = "sims", required = true) private List sims; - /** Creates an instance of SimUploadList class. */ + /** + * Creates an instance of SimUploadList class. + */ public SimUploadList() { } /** * Get the sims property: A list of SIMs to upload. - * + * * @return the sims value. */ public List sims() { @@ -33,7 +37,7 @@ public List sims() { /** * Set the sims property: A list of SIMs to upload. - * + * * @param sims the sims value to set. * @return the SimUploadList object itself. */ @@ -44,14 +48,13 @@ public SimUploadList withSims(List sims) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sims() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property sims in model SimUploadList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sims in model SimUploadList")); } else { sims().forEach(e -> e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sims.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sims.java index 229ddb5e8a7a9..f6d9a346c5e07 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sims.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sims.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Sims. */ +/** + * Resource collection API of Sims. + */ public interface Sims { /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -24,7 +26,7 @@ public interface Sims { /** * Deletes the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -37,7 +39,7 @@ public interface Sims { /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -51,7 +53,7 @@ public interface Sims { /** * Gets information about the specified SIM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param simName The name of the SIM. @@ -64,7 +66,7 @@ public interface Sims { /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -76,7 +78,7 @@ public interface Sims { /** * Gets all the SIMs in a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param context The context to associate with this operation. @@ -89,7 +91,7 @@ public interface Sims { /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -102,7 +104,7 @@ public interface Sims { /** * Bulk upload SIMs to a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM upload operation. @@ -112,12 +114,12 @@ public interface Sims { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus bulkUpload( - String resourceGroupName, String simGroupName, SimUploadList parameters, Context context); + AsyncOperationStatus bulkUpload(String resourceGroupName, String simGroupName, SimUploadList parameters, + Context context); /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -130,7 +132,7 @@ AsyncOperationStatus bulkUpload( /** * Bulk delete SIMs from a SIM group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the bulk SIM delete operation. @@ -140,12 +142,12 @@ AsyncOperationStatus bulkUpload( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus bulkDelete( - String resourceGroupName, String simGroupName, SimDeleteList parameters, Context context); + AsyncOperationStatus bulkDelete(String resourceGroupName, String simGroupName, SimDeleteList parameters, + Context context); /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -154,12 +156,12 @@ AsyncOperationStatus bulkDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters); + AsyncOperationStatus bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters); /** * Bulk upload SIMs in encrypted form to a SIM group. The SIM credentials must be encrypted. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param simGroupName The name of the SIM Group. * @param parameters Parameters supplied to the encrypted SIMs upload operation. @@ -169,12 +171,12 @@ AsyncOperationStatus bulkUploadEncrypted( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the current status of an async operation. */ - AsyncOperationStatus bulkUploadEncrypted( - String resourceGroupName, String simGroupName, EncryptedSimUploadList parameters, Context context); + AsyncOperationStatus bulkUploadEncrypted(String resourceGroupName, String simGroupName, + EncryptedSimUploadList parameters, Context context); /** * Gets information about the specified SIM. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -185,7 +187,7 @@ AsyncOperationStatus bulkUploadEncrypted( /** * Gets information about the specified SIM. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -197,7 +199,7 @@ AsyncOperationStatus bulkUploadEncrypted( /** * Deletes the specified SIM. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -207,7 +209,7 @@ AsyncOperationStatus bulkUploadEncrypted( /** * Deletes the specified SIM. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -218,7 +220,7 @@ AsyncOperationStatus bulkUploadEncrypted( /** * Begins definition for a new Sim resource. - * + * * @param name resource name. * @return the first stage of the new Sim definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Site.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Site.java index 203c36acbc16a..2e24838b1f54f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Site.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Site.java @@ -12,53 +12,55 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Site. */ +/** + * An immutable client-side representation of Site. + */ public interface Site { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the site resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); @@ -66,58 +68,63 @@ public interface Site { /** * Gets the networkFunctions property: An array of IDs of the network functions deployed in the site. Deleting the * site will delete any network functions that are deployed in the site. - * + * * @return the networkFunctions value. */ List networkFunctions(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.SiteInner object. - * + * * @return the inner object. */ SiteInner innerModel(); - /** The entirety of the Site definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithCreate { + /** + * The entirety of the Site definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Site definition stages. */ + /** + * The Site definition stages. + */ interface DefinitionStages { - /** The first stage of the Site definition. */ + /** + * The first stage of the Site definition. + */ interface Blank extends WithLocation { } - /** The stage of the Site definition allowing to specify location. */ + /** + * The stage of the Site definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -125,18 +132,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the Site definition allowing to specify parent resource. */ + /** + * The stage of the Site definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, mobileNetworkName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @return the next definition stage. @@ -151,25 +160,27 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithTags { /** * Executes the create request. - * + * * @return the created resource. */ Site create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Site create(Context context); } - /** The stage of the Site definition allowing to specify tags. */ + /** + * The stage of the Site definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -179,36 +190,42 @@ interface WithTags { /** * Begins update for the Site resource. - * + * * @return the stage of resource update. */ Site.Update update(); - /** The template for Site update. */ + /** + * The template for Site update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ Site apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Site apply(Context context); } - /** The Site update stages. */ + /** + * The Site update stages. + */ interface UpdateStages { - /** The stage of the Site update allowing to specify tags. */ + /** + * The stage of the Site update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -218,14 +235,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Site refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -233,7 +250,7 @@ interface WithTags { /** * Deletes a packet core under the specified mobile network site. - * + * * @param parameters Parameters supplied to delete a packet core under a site. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -243,7 +260,7 @@ interface WithTags { /** * Deletes a packet core under the specified mobile network site. - * + * * @param parameters Parameters supplied to delete a packet core under a site. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteDeletePacketCore.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteDeletePacketCore.java index a8b7f891b5163..802c43380bcdf 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteDeletePacketCore.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteDeletePacketCore.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The packet core to delete under a site. */ +/** + * The packet core to delete under a site. + */ @Fluent public final class SiteDeletePacketCore { /* @@ -16,13 +18,15 @@ public final class SiteDeletePacketCore { @JsonProperty(value = "packetCore") private PacketCoreControlPlaneResourceId packetCore; - /** Creates an instance of SiteDeletePacketCore class. */ + /** + * Creates an instance of SiteDeletePacketCore class. + */ public SiteDeletePacketCore() { } /** * Get the packetCore property: Reference to an packet core control plane resource. - * + * * @return the packetCore value. */ public PacketCoreControlPlaneResourceId packetCore() { @@ -31,7 +35,7 @@ public PacketCoreControlPlaneResourceId packetCore() { /** * Set the packetCore property: Reference to an packet core control plane resource. - * + * * @param packetCore the packetCore value to set. * @return the SiteDeletePacketCore object itself. */ @@ -42,7 +46,7 @@ public SiteDeletePacketCore withPacketCore(PacketCoreControlPlaneResourceId pack /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteListResult.java index 038a899947650..bb8475b6272e1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for sites API service call. */ +/** + * Response for sites API service call. + */ @Fluent public final class SiteListResult { /* @@ -24,13 +26,15 @@ public final class SiteListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of SiteListResult class. */ + /** + * Creates an instance of SiteListResult class. + */ public SiteListResult() { } /** * Get the value property: A list of sites in a mobile network. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of sites in a mobile network. - * + * * @param value the value value to set. * @return the SiteListResult object itself. */ @@ -50,7 +54,7 @@ public SiteListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteProvisioningState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteProvisioningState.java index e00f84ecf1553..37509350431b6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteProvisioningState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteProvisioningState.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The provisioning state of a resource e.g. SIM/SIM policy on a site. */ +/** + * The provisioning state of a resource e.g. SIM/SIM policy on a site. + */ public final class SiteProvisioningState extends ExpandableStringEnum { - /** Static value NotApplicable for SiteProvisioningState. */ + /** + * Static value NotApplicable for SiteProvisioningState. + */ public static final SiteProvisioningState NOT_APPLICABLE = fromString("NotApplicable"); - /** Static value Adding for SiteProvisioningState. */ + /** + * Static value Adding for SiteProvisioningState. + */ public static final SiteProvisioningState ADDING = fromString("Adding"); - /** Static value Updating for SiteProvisioningState. */ + /** + * Static value Updating for SiteProvisioningState. + */ public static final SiteProvisioningState UPDATING = fromString("Updating"); - /** Static value Deleting for SiteProvisioningState. */ + /** + * Static value Deleting for SiteProvisioningState. + */ public static final SiteProvisioningState DELETING = fromString("Deleting"); - /** Static value Provisioned for SiteProvisioningState. */ + /** + * Static value Provisioned for SiteProvisioningState. + */ public static final SiteProvisioningState PROVISIONED = fromString("Provisioned"); - /** Static value Failed for SiteProvisioningState. */ + /** + * Static value Failed for SiteProvisioningState. + */ public static final SiteProvisioningState FAILED = fromString("Failed"); /** * Creates a new instance of SiteProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public SiteProvisioningState() { /** * Creates or finds a SiteProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding SiteProvisioningState. */ @@ -50,7 +64,7 @@ public static SiteProvisioningState fromString(String name) { /** * Gets known SiteProvisioningState values. - * + * * @return known SiteProvisioningState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteResourceId.java index f6d5042754238..f893d47e921bb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SiteResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to a site resource. */ +/** + * Reference to a site resource. + */ @Fluent public final class SiteResourceId { /* @@ -17,13 +19,15 @@ public final class SiteResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of SiteResourceId class. */ + /** + * Creates an instance of SiteResourceId class. + */ public SiteResourceId() { } /** * Get the id property: Site resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Site resource ID. - * + * * @param id the id value to set. * @return the SiteResourceId object itself. */ @@ -43,14 +47,13 @@ public SiteResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model SiteResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model SiteResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sites.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sites.java index b6a7d4c50fc2d..32251a28c1f2a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sites.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Sites.java @@ -8,12 +8,14 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Sites. */ +/** + * Resource collection API of Sites. + */ public interface Sites { /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -26,7 +28,7 @@ public interface Sites { /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -39,7 +41,7 @@ public interface Sites { /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -49,12 +51,12 @@ public interface Sites { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified mobile network site along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String siteName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String siteName, + Context context); /** * Gets information about the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -67,7 +69,7 @@ Response getWithResponse( /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,7 +81,7 @@ Response getWithResponse( /** * Lists all sites in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -92,7 +94,7 @@ Response getWithResponse( /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -101,12 +103,12 @@ Response getWithResponse( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void deletePacketCore( - String resourceGroupName, String mobileNetworkName, String siteName, SiteDeletePacketCore parameters); + void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters); /** * Deletes a packet core under the specified mobile network site. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param siteName The name of the mobile network site. @@ -116,16 +118,12 @@ void deletePacketCore( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void deletePacketCore( - String resourceGroupName, - String mobileNetworkName, - String siteName, - SiteDeletePacketCore parameters, - Context context); + void deletePacketCore(String resourceGroupName, String mobileNetworkName, String siteName, + SiteDeletePacketCore parameters, Context context); /** * Gets information about the specified mobile network site. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -136,7 +134,7 @@ void deletePacketCore( /** * Gets information about the specified mobile network site. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -149,7 +147,7 @@ void deletePacketCore( /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -160,7 +158,7 @@ void deletePacketCore( /** * Deletes the specified mobile network site. This will also delete any network functions that are a part of this * site. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -171,7 +169,7 @@ void deletePacketCore( /** * Begins definition for a new Site resource. - * + * * @param name resource name. * @return the first stage of the new Site definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slice.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slice.java index 81dd2889fed1e..1418db914f014 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slice.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slice.java @@ -10,53 +10,55 @@ import com.azure.resourcemanager.mobilenetwork.fluent.models.SliceInner; import java.util.Map; -/** An immutable client-side representation of Slice. */ +/** + * An immutable client-side representation of Slice. + */ public interface Slice { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: The provisioning state of the network slice resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); @@ -64,66 +66,70 @@ public interface Slice { /** * Gets the snssai property: Single-network slice selection assistance information (S-NSSAI). Unique at the scope of * a mobile network. - * + * * @return the snssai value. */ Snssai snssai(); /** * Gets the description property: An optional description for this network slice. - * + * * @return the description value. */ String description(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.SliceInner object. - * + * * @return the inner object. */ SliceInner innerModel(); - /** The entirety of the Slice definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithSnssai, - DefinitionStages.WithCreate { + /** + * The entirety of the Slice definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithSnssai, DefinitionStages.WithCreate { } - /** The Slice definition stages. */ + /** + * The Slice definition stages. + */ interface DefinitionStages { - /** The first stage of the Slice definition. */ + /** + * The first stage of the Slice definition. + */ interface Blank extends WithLocation { } - /** The stage of the Slice definition allowing to specify location. */ + /** + * The stage of the Slice definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -131,18 +137,20 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the Slice definition allowing to specify parent resource. */ + /** + * The stage of the Slice definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, mobileNetworkName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @return the next definition stage. @@ -150,14 +158,16 @@ interface WithParentResource { WithSnssai withExistingMobileNetwork(String resourceGroupName, String mobileNetworkName); } - /** The stage of the Slice definition allowing to specify snssai. */ + /** + * The stage of the Slice definition allowing to specify snssai. + */ interface WithSnssai { /** * Specifies the snssai property: Single-network slice selection assistance information (S-NSSAI). Unique at * the scope of a mobile network.. - * + * * @param snssai Single-network slice selection assistance information (S-NSSAI). Unique at the scope of a - * mobile network. + * mobile network. * @return the next definition stage. */ WithCreate withSnssai(Snssai snssai); @@ -170,36 +180,40 @@ interface WithSnssai { interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithDescription { /** * Executes the create request. - * + * * @return the created resource. */ Slice create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Slice create(Context context); } - /** The stage of the Slice definition allowing to specify tags. */ + /** + * The stage of the Slice definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Slice definition allowing to specify description. */ + /** + * The stage of the Slice definition allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: An optional description for this network slice.. - * + * * @param description An optional description for this network slice. * @return the next definition stage. */ @@ -209,36 +223,42 @@ interface WithDescription { /** * Begins update for the Slice resource. - * + * * @return the stage of resource update. */ Slice.Update update(); - /** The template for Slice update. */ + /** + * The template for Slice update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ Slice apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Slice apply(Context context); } - /** The Slice update stages. */ + /** + * The Slice update stages. + */ interface UpdateStages { - /** The stage of the Slice update allowing to specify tags. */ + /** + * The stage of the Slice update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -248,14 +268,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Slice refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceConfiguration.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceConfiguration.java index dc9865d6948c7..1fc768375c15a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceConfiguration.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Per-slice settings. */ +/** + * Per-slice settings. + */ @Fluent public final class SliceConfiguration { /* @@ -32,14 +34,16 @@ public final class SliceConfiguration { @JsonProperty(value = "dataNetworkConfigurations", required = true) private List dataNetworkConfigurations; - /** Creates an instance of SliceConfiguration class. */ + /** + * Creates an instance of SliceConfiguration class. + */ public SliceConfiguration() { } /** * Get the slice property: A reference to the slice that these settings apply to. The slice must be in the same * location as the SIM policy. - * + * * @return the slice value. */ public SliceResourceId slice() { @@ -49,7 +53,7 @@ public SliceResourceId slice() { /** * Set the slice property: A reference to the slice that these settings apply to. The slice must be in the same * location as the SIM policy. - * + * * @param slice the slice value to set. * @return the SliceConfiguration object itself. */ @@ -62,7 +66,7 @@ public SliceConfiguration withSlice(SliceResourceId slice) { * Get the defaultDataNetwork property: The default data network to use if the UE does not explicitly specify it. * Configuration for this object must exist in the `dataNetworkConfigurations` map. The data network must be in the * same location as the SIM policy. - * + * * @return the defaultDataNetwork value. */ public DataNetworkResourceId defaultDataNetwork() { @@ -73,7 +77,7 @@ public DataNetworkResourceId defaultDataNetwork() { * Set the defaultDataNetwork property: The default data network to use if the UE does not explicitly specify it. * Configuration for this object must exist in the `dataNetworkConfigurations` map. The data network must be in the * same location as the SIM policy. - * + * * @param defaultDataNetwork the defaultDataNetwork value to set. * @return the SliceConfiguration object itself. */ @@ -85,7 +89,7 @@ public SliceConfiguration withDefaultDataNetwork(DataNetworkResourceId defaultDa /** * Get the dataNetworkConfigurations property: The allowed data networks and the settings to use for them. The list * must not contain duplicate items and must contain at least one item. - * + * * @return the dataNetworkConfigurations value. */ public List dataNetworkConfigurations() { @@ -95,7 +99,7 @@ public List dataNetworkConfigurations() { /** * Set the dataNetworkConfigurations property: The allowed data networks and the settings to use for them. The list * must not contain duplicate items and must contain at least one item. - * + * * @param dataNetworkConfigurations the dataNetworkConfigurations value to set. * @return the SliceConfiguration object itself. */ @@ -106,30 +110,25 @@ public SliceConfiguration withDataNetworkConfigurations(List e.validate()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceListResult.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceListResult.java index f289b6cb569f7..47d2becf4e27c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceListResult.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for network slice API service call. */ +/** + * Response for network slice API service call. + */ @Fluent public final class SliceListResult { /* @@ -24,13 +26,15 @@ public final class SliceListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of SliceListResult class. */ + /** + * Creates an instance of SliceListResult class. + */ public SliceListResult() { } /** * Get the value property: A list of network slices in a mobile network. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of network slices in a mobile network. - * + * * @param value the value value to set. * @return the SliceListResult object itself. */ @@ -50,7 +54,7 @@ public SliceListResult withValue(List value) { /** * Get the nextLink property: The URL to get the next set of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceResourceId.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceResourceId.java index fcec10ac96e30..03fa156ddeae8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceResourceId.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/SliceResourceId.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Reference to a slice resource. */ +/** + * Reference to a slice resource. + */ @Fluent public final class SliceResourceId { /* @@ -17,13 +19,15 @@ public final class SliceResourceId { @JsonProperty(value = "id", required = true) private String id; - /** Creates an instance of SliceResourceId class. */ + /** + * Creates an instance of SliceResourceId class. + */ public SliceResourceId() { } /** * Get the id property: Slice resource ID. - * + * * @return the id value. */ public String id() { @@ -32,7 +36,7 @@ public String id() { /** * Set the id property: Slice resource ID. - * + * * @param id the id value to set. * @return the SliceResourceId object itself. */ @@ -43,14 +47,13 @@ public SliceResourceId withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (id() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property id in model SliceResourceId")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property id in model SliceResourceId")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slices.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slices.java index 4ebd1b32cc2fa..946acd0c2584e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slices.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Slices.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Slices. */ +/** + * Resource collection API of Slices. + */ public interface Slices { /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -24,7 +26,7 @@ public interface Slices { /** * Deletes the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -37,7 +39,7 @@ public interface Slices { /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -47,12 +49,12 @@ public interface Slices { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about the specified network slice along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String mobileNetworkName, String sliceName, Context context); + Response getWithResponse(String resourceGroupName, String mobileNetworkName, String sliceName, + Context context); /** * Gets information about the specified network slice. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param sliceName The name of the network slice. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Lists all slices in the mobile network. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param mobileNetworkName The name of the mobile network. * @param context The context to associate with this operation. @@ -90,7 +92,7 @@ Response getWithResponse( /** * Gets information about the specified network slice. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -101,7 +103,7 @@ Response getWithResponse( /** * Gets information about the specified network slice. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,7 +115,7 @@ Response getWithResponse( /** * Deletes the specified network slice. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -123,7 +125,7 @@ Response getWithResponse( /** * Deletes the specified network slice. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -134,7 +136,7 @@ Response getWithResponse( /** * Begins definition for a new Slice resource. - * + * * @param name resource name. * @return the first stage of the new Slice definition. */ diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Snssai.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Snssai.java index 2729c9cf95d78..c392d6963ba8c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Snssai.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/Snssai.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Single-network slice selection assistance information (S-NSSAI). */ +/** + * Single-network slice selection assistance information (S-NSSAI). + */ @Fluent public final class Snssai { /* @@ -22,13 +24,15 @@ public final class Snssai { @JsonProperty(value = "sd") private String sd; - /** Creates an instance of Snssai class. */ + /** + * Creates an instance of Snssai class. + */ public Snssai() { } /** * Get the sst property: Slice/service type (SST). - * + * * @return the sst value. */ public int sst() { @@ -37,7 +41,7 @@ public int sst() { /** * Set the sst property: Slice/service type (SST). - * + * * @param sst the sst value to set. * @return the Snssai object itself. */ @@ -48,7 +52,7 @@ public Snssai withSst(int sst) { /** * Get the sd property: Slice differentiator (SD). - * + * * @return the sd value. */ public String sd() { @@ -57,7 +61,7 @@ public String sd() { /** * Set the sd property: Slice differentiator (SD). - * + * * @param sd the sd value to set. * @return the Snssai object itself. */ @@ -68,7 +72,7 @@ public Snssai withSd(String sd) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TagsObject.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TagsObject.java index 2564390164d83..d9508d4d50dc9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TagsObject.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TagsObject.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Tags object for patch operations. */ +/** + * Tags object for patch operations. + */ @Fluent public final class TagsObject { /* @@ -19,13 +21,15 @@ public final class TagsObject { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of TagsObject class. */ + /** + * Creates an instance of TagsObject class. + */ public TagsObject() { } /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -34,7 +38,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the TagsObject object itself. */ @@ -45,7 +49,7 @@ public TagsObject withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TrafficControlPermission.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TrafficControlPermission.java index f96e4079effa8..d9ed67d686245 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TrafficControlPermission.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/TrafficControlPermission.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Traffic control permission. */ +/** + * Traffic control permission. + */ public final class TrafficControlPermission extends ExpandableStringEnum { - /** Static value Enabled for TrafficControlPermission. */ + /** + * Static value Enabled for TrafficControlPermission. + */ public static final TrafficControlPermission ENABLED = fromString("Enabled"); - /** Static value Blocked for TrafficControlPermission. */ + /** + * Static value Blocked for TrafficControlPermission. + */ public static final TrafficControlPermission BLOCKED = fromString("Blocked"); /** * Creates a new instance of TrafficControlPermission value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public TrafficControlPermission() { /** * Creates or finds a TrafficControlPermission from its string representation. - * + * * @param name a name to look for. * @return the corresponding TrafficControlPermission. */ @@ -38,7 +44,7 @@ public static TrafficControlPermission fromString(String name) { /** * Gets known TrafficControlPermission values. - * + * * @return known TrafficControlPermission values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo4G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo4G.java new file mode 100644 index 0000000000000..a5db5a5a0a5e6 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo4G.java @@ -0,0 +1,309 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * UE Connection Info for 4G. + */ +@Fluent +public final class UeConnectionInfo4G { + /* + * UE Location Info properties + */ + @JsonProperty(value = "locationInfo") + private UeLocationInfo locationInfo; + + /* + * Global RAN Node ID + */ + @JsonProperty(value = "globalRanNodeId", required = true) + private GlobalRanNodeId globalRanNodeId; + + /* + * Per-UE transport network layer association + */ + @JsonProperty(value = "perUeTnla") + private String perUeTnla; + + /* + * MME S1AP identifier + */ + @JsonProperty(value = "mmeS1apId", required = true) + private int mmeS1ApId; + + /* + * eNodeB S1AP identifier + */ + @JsonProperty(value = "enbS1apId", required = true) + private int enbS1ApId; + + /* + * Last Visited TAI + */ + @JsonProperty(value = "lastVisitedTai") + private String lastVisitedTai; + + /* + * State of the UE. + */ + @JsonProperty(value = "ueState", required = true) + private UeState ueState; + + /* + * Radio connection establishment cause + */ + @JsonProperty(value = "rrcEstablishmentCause", required = true) + private RrcEstablishmentCause rrcEstablishmentCause; + + /* + * The UE's usage setting + */ + @JsonProperty(value = "ueUsageSetting") + private UeUsageSetting ueUsageSetting; + + /* + * The timestamp of last activity of UE (UTC). + */ + @JsonProperty(value = "lastActivityTime") + private OffsetDateTime lastActivityTime; + + /** + * Creates an instance of UeConnectionInfo4G class. + */ + public UeConnectionInfo4G() { + } + + /** + * Get the locationInfo property: UE Location Info properties. + * + * @return the locationInfo value. + */ + public UeLocationInfo locationInfo() { + return this.locationInfo; + } + + /** + * Set the locationInfo property: UE Location Info properties. + * + * @param locationInfo the locationInfo value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withLocationInfo(UeLocationInfo locationInfo) { + this.locationInfo = locationInfo; + return this; + } + + /** + * Get the globalRanNodeId property: Global RAN Node ID. + * + * @return the globalRanNodeId value. + */ + public GlobalRanNodeId globalRanNodeId() { + return this.globalRanNodeId; + } + + /** + * Set the globalRanNodeId property: Global RAN Node ID. + * + * @param globalRanNodeId the globalRanNodeId value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withGlobalRanNodeId(GlobalRanNodeId globalRanNodeId) { + this.globalRanNodeId = globalRanNodeId; + return this; + } + + /** + * Get the perUeTnla property: Per-UE transport network layer association. + * + * @return the perUeTnla value. + */ + public String perUeTnla() { + return this.perUeTnla; + } + + /** + * Set the perUeTnla property: Per-UE transport network layer association. + * + * @param perUeTnla the perUeTnla value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withPerUeTnla(String perUeTnla) { + this.perUeTnla = perUeTnla; + return this; + } + + /** + * Get the mmeS1ApId property: MME S1AP identifier. + * + * @return the mmeS1ApId value. + */ + public int mmeS1ApId() { + return this.mmeS1ApId; + } + + /** + * Set the mmeS1ApId property: MME S1AP identifier. + * + * @param mmeS1ApId the mmeS1ApId value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withMmeS1ApId(int mmeS1ApId) { + this.mmeS1ApId = mmeS1ApId; + return this; + } + + /** + * Get the enbS1ApId property: eNodeB S1AP identifier. + * + * @return the enbS1ApId value. + */ + public int enbS1ApId() { + return this.enbS1ApId; + } + + /** + * Set the enbS1ApId property: eNodeB S1AP identifier. + * + * @param enbS1ApId the enbS1ApId value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withEnbS1ApId(int enbS1ApId) { + this.enbS1ApId = enbS1ApId; + return this; + } + + /** + * Get the lastVisitedTai property: Last Visited TAI. + * + * @return the lastVisitedTai value. + */ + public String lastVisitedTai() { + return this.lastVisitedTai; + } + + /** + * Set the lastVisitedTai property: Last Visited TAI. + * + * @param lastVisitedTai the lastVisitedTai value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withLastVisitedTai(String lastVisitedTai) { + this.lastVisitedTai = lastVisitedTai; + return this; + } + + /** + * Get the ueState property: State of the UE. + * + * @return the ueState value. + */ + public UeState ueState() { + return this.ueState; + } + + /** + * Set the ueState property: State of the UE. + * + * @param ueState the ueState value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withUeState(UeState ueState) { + this.ueState = ueState; + return this; + } + + /** + * Get the rrcEstablishmentCause property: Radio connection establishment cause. + * + * @return the rrcEstablishmentCause value. + */ + public RrcEstablishmentCause rrcEstablishmentCause() { + return this.rrcEstablishmentCause; + } + + /** + * Set the rrcEstablishmentCause property: Radio connection establishment cause. + * + * @param rrcEstablishmentCause the rrcEstablishmentCause value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withRrcEstablishmentCause(RrcEstablishmentCause rrcEstablishmentCause) { + this.rrcEstablishmentCause = rrcEstablishmentCause; + return this; + } + + /** + * Get the ueUsageSetting property: The UE's usage setting. + * + * @return the ueUsageSetting value. + */ + public UeUsageSetting ueUsageSetting() { + return this.ueUsageSetting; + } + + /** + * Set the ueUsageSetting property: The UE's usage setting. + * + * @param ueUsageSetting the ueUsageSetting value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withUeUsageSetting(UeUsageSetting ueUsageSetting) { + this.ueUsageSetting = ueUsageSetting; + return this; + } + + /** + * Get the lastActivityTime property: The timestamp of last activity of UE (UTC). + * + * @return the lastActivityTime value. + */ + public OffsetDateTime lastActivityTime() { + return this.lastActivityTime; + } + + /** + * Set the lastActivityTime property: The timestamp of last activity of UE (UTC). + * + * @param lastActivityTime the lastActivityTime value to set. + * @return the UeConnectionInfo4G object itself. + */ + public UeConnectionInfo4G withLastActivityTime(OffsetDateTime lastActivityTime) { + this.lastActivityTime = lastActivityTime; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (locationInfo() != null) { + locationInfo().validate(); + } + if (globalRanNodeId() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property globalRanNodeId in model UeConnectionInfo4G")); + } else { + globalRanNodeId().validate(); + } + if (ueState() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ueState in model UeConnectionInfo4G")); + } + if (rrcEstablishmentCause() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property rrcEstablishmentCause in model UeConnectionInfo4G")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeConnectionInfo4G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo5G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo5G.java new file mode 100644 index 0000000000000..8bdf2de8f4aa8 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeConnectionInfo5G.java @@ -0,0 +1,339 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; +import java.util.List; + +/** + * UE Connection Info for 5G. + */ +@Fluent +public final class UeConnectionInfo5G { + /* + * UE Location Info properties + */ + @JsonProperty(value = "locationInfo") + private UeLocationInfo locationInfo; + + /* + * Global RAN Node ID + */ + @JsonProperty(value = "globalRanNodeId", required = true) + private GlobalRanNodeId globalRanNodeId; + + /* + * Per-UE transport network layer association + */ + @JsonProperty(value = "perUeTnla") + private String perUeTnla; + + /* + * The AMF UE NGAP ID + */ + @JsonProperty(value = "amfUeNgapId", required = true) + private long amfUeNgapId; + + /* + * The RAN UE NGAP ID + */ + @JsonProperty(value = "ranUeNgapId", required = true) + private int ranUeNgapId; + + /* + * Last Visited TAI + */ + @JsonProperty(value = "lastVisitedTai") + private String lastVisitedTai; + + /* + * Allowed Network Slice Selection Assistance Information + */ + @JsonProperty(value = "allowedNssai") + private List allowedNssai; + + /* + * State of the UE. + */ + @JsonProperty(value = "ueState", required = true) + private UeState ueState; + + /* + * Radio connection establishment cause + */ + @JsonProperty(value = "rrcEstablishmentCause", required = true) + private RrcEstablishmentCause rrcEstablishmentCause; + + /* + * The UE's usage setting + */ + @JsonProperty(value = "ueUsageSetting") + private UeUsageSetting ueUsageSetting; + + /* + * The timestamp of last activity of UE (UTC). + */ + @JsonProperty(value = "lastActivityTime") + private OffsetDateTime lastActivityTime; + + /** + * Creates an instance of UeConnectionInfo5G class. + */ + public UeConnectionInfo5G() { + } + + /** + * Get the locationInfo property: UE Location Info properties. + * + * @return the locationInfo value. + */ + public UeLocationInfo locationInfo() { + return this.locationInfo; + } + + /** + * Set the locationInfo property: UE Location Info properties. + * + * @param locationInfo the locationInfo value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withLocationInfo(UeLocationInfo locationInfo) { + this.locationInfo = locationInfo; + return this; + } + + /** + * Get the globalRanNodeId property: Global RAN Node ID. + * + * @return the globalRanNodeId value. + */ + public GlobalRanNodeId globalRanNodeId() { + return this.globalRanNodeId; + } + + /** + * Set the globalRanNodeId property: Global RAN Node ID. + * + * @param globalRanNodeId the globalRanNodeId value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withGlobalRanNodeId(GlobalRanNodeId globalRanNodeId) { + this.globalRanNodeId = globalRanNodeId; + return this; + } + + /** + * Get the perUeTnla property: Per-UE transport network layer association. + * + * @return the perUeTnla value. + */ + public String perUeTnla() { + return this.perUeTnla; + } + + /** + * Set the perUeTnla property: Per-UE transport network layer association. + * + * @param perUeTnla the perUeTnla value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withPerUeTnla(String perUeTnla) { + this.perUeTnla = perUeTnla; + return this; + } + + /** + * Get the amfUeNgapId property: The AMF UE NGAP ID. + * + * @return the amfUeNgapId value. + */ + public long amfUeNgapId() { + return this.amfUeNgapId; + } + + /** + * Set the amfUeNgapId property: The AMF UE NGAP ID. + * + * @param amfUeNgapId the amfUeNgapId value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withAmfUeNgapId(long amfUeNgapId) { + this.amfUeNgapId = amfUeNgapId; + return this; + } + + /** + * Get the ranUeNgapId property: The RAN UE NGAP ID. + * + * @return the ranUeNgapId value. + */ + public int ranUeNgapId() { + return this.ranUeNgapId; + } + + /** + * Set the ranUeNgapId property: The RAN UE NGAP ID. + * + * @param ranUeNgapId the ranUeNgapId value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withRanUeNgapId(int ranUeNgapId) { + this.ranUeNgapId = ranUeNgapId; + return this; + } + + /** + * Get the lastVisitedTai property: Last Visited TAI. + * + * @return the lastVisitedTai value. + */ + public String lastVisitedTai() { + return this.lastVisitedTai; + } + + /** + * Set the lastVisitedTai property: Last Visited TAI. + * + * @param lastVisitedTai the lastVisitedTai value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withLastVisitedTai(String lastVisitedTai) { + this.lastVisitedTai = lastVisitedTai; + return this; + } + + /** + * Get the allowedNssai property: Allowed Network Slice Selection Assistance Information. + * + * @return the allowedNssai value. + */ + public List allowedNssai() { + return this.allowedNssai; + } + + /** + * Set the allowedNssai property: Allowed Network Slice Selection Assistance Information. + * + * @param allowedNssai the allowedNssai value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withAllowedNssai(List allowedNssai) { + this.allowedNssai = allowedNssai; + return this; + } + + /** + * Get the ueState property: State of the UE. + * + * @return the ueState value. + */ + public UeState ueState() { + return this.ueState; + } + + /** + * Set the ueState property: State of the UE. + * + * @param ueState the ueState value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withUeState(UeState ueState) { + this.ueState = ueState; + return this; + } + + /** + * Get the rrcEstablishmentCause property: Radio connection establishment cause. + * + * @return the rrcEstablishmentCause value. + */ + public RrcEstablishmentCause rrcEstablishmentCause() { + return this.rrcEstablishmentCause; + } + + /** + * Set the rrcEstablishmentCause property: Radio connection establishment cause. + * + * @param rrcEstablishmentCause the rrcEstablishmentCause value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withRrcEstablishmentCause(RrcEstablishmentCause rrcEstablishmentCause) { + this.rrcEstablishmentCause = rrcEstablishmentCause; + return this; + } + + /** + * Get the ueUsageSetting property: The UE's usage setting. + * + * @return the ueUsageSetting value. + */ + public UeUsageSetting ueUsageSetting() { + return this.ueUsageSetting; + } + + /** + * Set the ueUsageSetting property: The UE's usage setting. + * + * @param ueUsageSetting the ueUsageSetting value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withUeUsageSetting(UeUsageSetting ueUsageSetting) { + this.ueUsageSetting = ueUsageSetting; + return this; + } + + /** + * Get the lastActivityTime property: The timestamp of last activity of UE (UTC). + * + * @return the lastActivityTime value. + */ + public OffsetDateTime lastActivityTime() { + return this.lastActivityTime; + } + + /** + * Set the lastActivityTime property: The timestamp of last activity of UE (UTC). + * + * @param lastActivityTime the lastActivityTime value to set. + * @return the UeConnectionInfo5G object itself. + */ + public UeConnectionInfo5G withLastActivityTime(OffsetDateTime lastActivityTime) { + this.lastActivityTime = lastActivityTime; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (locationInfo() != null) { + locationInfo().validate(); + } + if (globalRanNodeId() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property globalRanNodeId in model UeConnectionInfo5G")); + } else { + globalRanNodeId().validate(); + } + if (allowedNssai() != null) { + allowedNssai().forEach(e -> e.validate()); + } + if (ueState() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ueState in model UeConnectionInfo5G")); + } + if (rrcEstablishmentCause() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property rrcEstablishmentCause in model UeConnectionInfo5G")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeConnectionInfo5G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo.java new file mode 100644 index 0000000000000..90063c5718c52 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; + +/** + * An immutable client-side representation of UeInfo. + */ +public interface UeInfo { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: Basic UE Information Properties. + * + * @return the properties value. + */ + UeInfoPropertiesFormat properties(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the inner com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner object. + * + * @return the inner object. + */ + UeInfoInner innerModel(); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4G.java new file mode 100644 index 0000000000000..07f8f6c211c8b --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4G.java @@ -0,0 +1,79 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.time.OffsetDateTime; + +/** + * UE Information for 4G. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ratType") +@JsonTypeName("4G") +@Fluent +public final class UeInfo4G extends ExtendedUeInfoProperties { + /* + * UE Information properties for 4G. + */ + @JsonProperty(value = "info", required = true) + private UeInfo4GProperties info; + + /** + * Creates an instance of UeInfo4G class. + */ + public UeInfo4G() { + } + + /** + * Get the info property: UE Information properties for 4G. + * + * @return the info value. + */ + public UeInfo4GProperties info() { + return this.info; + } + + /** + * Set the info property: UE Information properties for 4G. + * + * @param info the info value to set. + * @return the UeInfo4G object itself. + */ + public UeInfo4G withInfo(UeInfo4GProperties info) { + this.info = info; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public UeInfo4G withLastReadAt(OffsetDateTime lastReadAt) { + super.withLastReadAt(lastReadAt); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (info() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property info in model UeInfo4G")); + } else { + info().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeInfo4G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4GProperties.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4GProperties.java new file mode 100644 index 0000000000000..7b934a12fbb65 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo4GProperties.java @@ -0,0 +1,204 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * UE Information properties for 4G. + */ +@Fluent +public final class UeInfo4GProperties { + /* + * International mobile subscriber identifier + */ + @JsonProperty(value = "imsi", required = true) + private String imsi; + + /* + * International mobile equipment identity + */ + @JsonProperty(value = "imei") + private String imei; + + /* + * International mobile equipment identity – software version + */ + @JsonProperty(value = "imeisv") + private String imeisv; + + /* + * Globally Unique Temporary Identifier (4G) + */ + @JsonProperty(value = "guti", required = true) + private Guti4G guti; + + /* + * UE Connection Info for 4G + */ + @JsonProperty(value = "connectionInfo") + private UeConnectionInfo4G connectionInfo; + + /* + * The sessionInfo property. + */ + @JsonProperty(value = "sessionInfo") + private List sessionInfo; + + /** + * Creates an instance of UeInfo4GProperties class. + */ + public UeInfo4GProperties() { + } + + /** + * Get the imsi property: International mobile subscriber identifier. + * + * @return the imsi value. + */ + public String imsi() { + return this.imsi; + } + + /** + * Set the imsi property: International mobile subscriber identifier. + * + * @param imsi the imsi value to set. + * @return the UeInfo4GProperties object itself. + */ + public UeInfo4GProperties withImsi(String imsi) { + this.imsi = imsi; + return this; + } + + /** + * Get the imei property: International mobile equipment identity. + * + * @return the imei value. + */ + public String imei() { + return this.imei; + } + + /** + * Set the imei property: International mobile equipment identity. + * + * @param imei the imei value to set. + * @return the UeInfo4GProperties object itself. + */ + public UeInfo4GProperties withImei(String imei) { + this.imei = imei; + return this; + } + + /** + * Get the imeisv property: International mobile equipment identity – software version. + * + * @return the imeisv value. + */ + public String imeisv() { + return this.imeisv; + } + + /** + * Set the imeisv property: International mobile equipment identity – software version. + * + * @param imeisv the imeisv value to set. + * @return the UeInfo4GProperties object itself. + */ + public UeInfo4GProperties withImeisv(String imeisv) { + this.imeisv = imeisv; + return this; + } + + /** + * Get the guti property: Globally Unique Temporary Identifier (4G). + * + * @return the guti value. + */ + public Guti4G guti() { + return this.guti; + } + + /** + * Set the guti property: Globally Unique Temporary Identifier (4G). + * + * @param guti the guti value to set. + * @return the UeInfo4GProperties object itself. + */ + public UeInfo4GProperties withGuti(Guti4G guti) { + this.guti = guti; + return this; + } + + /** + * Get the connectionInfo property: UE Connection Info for 4G. + * + * @return the connectionInfo value. + */ + public UeConnectionInfo4G connectionInfo() { + return this.connectionInfo; + } + + /** + * Set the connectionInfo property: UE Connection Info for 4G. + * + * @param connectionInfo the connectionInfo value to set. + * @return the UeInfo4GProperties object itself. + */ + public UeInfo4GProperties withConnectionInfo(UeConnectionInfo4G connectionInfo) { + this.connectionInfo = connectionInfo; + return this; + } + + /** + * Get the sessionInfo property: The sessionInfo property. + * + * @return the sessionInfo value. + */ + public List sessionInfo() { + return this.sessionInfo; + } + + /** + * Set the sessionInfo property: The sessionInfo property. + * + * @param sessionInfo the sessionInfo value to set. + * @return the UeInfo4GProperties object itself. + */ + public UeInfo4GProperties withSessionInfo(List sessionInfo) { + this.sessionInfo = sessionInfo; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (imsi() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property imsi in model UeInfo4GProperties")); + } + if (guti() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property guti in model UeInfo4GProperties")); + } else { + guti().validate(); + } + if (connectionInfo() != null) { + connectionInfo().validate(); + } + if (sessionInfo() != null) { + sessionInfo().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeInfo4GProperties.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5G.java new file mode 100644 index 0000000000000..755a38338584b --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5G.java @@ -0,0 +1,79 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import java.time.OffsetDateTime; + +/** + * UE Information for 5G. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ratType") +@JsonTypeName("5G") +@Fluent +public final class UeInfo5G extends ExtendedUeInfoProperties { + /* + * UE Information properties for 5G. + */ + @JsonProperty(value = "info", required = true) + private UeInfo5GProperties info; + + /** + * Creates an instance of UeInfo5G class. + */ + public UeInfo5G() { + } + + /** + * Get the info property: UE Information properties for 5G. + * + * @return the info value. + */ + public UeInfo5GProperties info() { + return this.info; + } + + /** + * Set the info property: UE Information properties for 5G. + * + * @param info the info value to set. + * @return the UeInfo5G object itself. + */ + public UeInfo5G withInfo(UeInfo5GProperties info) { + this.info = info; + return this; + } + + /** + * {@inheritDoc} + */ + @Override + public UeInfo5G withLastReadAt(OffsetDateTime lastReadAt) { + super.withLastReadAt(lastReadAt); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (info() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property info in model UeInfo5G")); + } else { + info().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeInfo5G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5GProperties.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5GProperties.java new file mode 100644 index 0000000000000..1a5d1ffd929ff --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfo5GProperties.java @@ -0,0 +1,178 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * UE Information properties for 5G. + */ +@Fluent +public final class UeInfo5GProperties { + /* + * Subscription Permanent Identifier + */ + @JsonProperty(value = "supi", required = true) + private String supi; + + /* + * Permanent Equipment Identifier + */ + @JsonProperty(value = "pei") + private String pei; + + /* + * 5G GUTI + */ + @JsonProperty(value = "fivegGuti", required = true) + private Guti5G fivegGuti; + + /* + * UE Connection Info for 5G. + */ + @JsonProperty(value = "connectionInfo") + private UeConnectionInfo5G connectionInfo; + + /* + * The sessionInfo property. + */ + @JsonProperty(value = "sessionInfo") + private List sessionInfo; + + /** + * Creates an instance of UeInfo5GProperties class. + */ + public UeInfo5GProperties() { + } + + /** + * Get the supi property: Subscription Permanent Identifier. + * + * @return the supi value. + */ + public String supi() { + return this.supi; + } + + /** + * Set the supi property: Subscription Permanent Identifier. + * + * @param supi the supi value to set. + * @return the UeInfo5GProperties object itself. + */ + public UeInfo5GProperties withSupi(String supi) { + this.supi = supi; + return this; + } + + /** + * Get the pei property: Permanent Equipment Identifier. + * + * @return the pei value. + */ + public String pei() { + return this.pei; + } + + /** + * Set the pei property: Permanent Equipment Identifier. + * + * @param pei the pei value to set. + * @return the UeInfo5GProperties object itself. + */ + public UeInfo5GProperties withPei(String pei) { + this.pei = pei; + return this; + } + + /** + * Get the fivegGuti property: 5G GUTI. + * + * @return the fivegGuti value. + */ + public Guti5G fivegGuti() { + return this.fivegGuti; + } + + /** + * Set the fivegGuti property: 5G GUTI. + * + * @param fivegGuti the fivegGuti value to set. + * @return the UeInfo5GProperties object itself. + */ + public UeInfo5GProperties withFivegGuti(Guti5G fivegGuti) { + this.fivegGuti = fivegGuti; + return this; + } + + /** + * Get the connectionInfo property: UE Connection Info for 5G. + * + * @return the connectionInfo value. + */ + public UeConnectionInfo5G connectionInfo() { + return this.connectionInfo; + } + + /** + * Set the connectionInfo property: UE Connection Info for 5G. + * + * @param connectionInfo the connectionInfo value to set. + * @return the UeInfo5GProperties object itself. + */ + public UeInfo5GProperties withConnectionInfo(UeConnectionInfo5G connectionInfo) { + this.connectionInfo = connectionInfo; + return this; + } + + /** + * Get the sessionInfo property: The sessionInfo property. + * + * @return the sessionInfo value. + */ + public List sessionInfo() { + return this.sessionInfo; + } + + /** + * Set the sessionInfo property: The sessionInfo property. + * + * @param sessionInfo the sessionInfo value to set. + * @return the UeInfo5GProperties object itself. + */ + public UeInfo5GProperties withSessionInfo(List sessionInfo) { + this.sessionInfo = sessionInfo; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (supi() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property supi in model UeInfo5GProperties")); + } + if (fivegGuti() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property fivegGuti in model UeInfo5GProperties")); + } else { + fivegGuti().validate(); + } + if (connectionInfo() != null) { + connectionInfo().validate(); + } + if (sessionInfo() != null) { + sessionInfo().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeInfo5GProperties.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoList.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoList.java new file mode 100644 index 0000000000000..c5361a25c1ff1 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoList.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Response for packet core list UEs API call. + */ +@Fluent +public final class UeInfoList { + /* + * A list of UEs in a packet core and their basic information. + */ + @JsonProperty(value = "value") + private List value; + + /* + * The URL to get the next set of results. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of UeInfoList class. + */ + public UeInfoList() { + } + + /** + * Get the value property: A list of UEs in a packet core and their basic information. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: A list of UEs in a packet core and their basic information. + * + * @param value the value value to set. + * @return the UeInfoList object itself. + */ + public UeInfoList withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: The URL to get the next set of results. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoPropertiesFormat.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoPropertiesFormat.java new file mode 100644 index 0000000000000..9d89799cff251 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInfoPropertiesFormat.java @@ -0,0 +1,148 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; +import java.util.List; + +/** + * Basic UE Information Properties. + */ +@Fluent +public final class UeInfoPropertiesFormat { + /* + * RAT Type + */ + @JsonProperty(value = "ratType", required = true) + private RatType ratType; + + /* + * State of the UE. + */ + @JsonProperty(value = "ueState", required = true) + private UeState ueState; + + /* + * The ueIpAddresses property. + */ + @JsonProperty(value = "ueIpAddresses") + private List ueIpAddresses; + + /* + * The timestamp of last list UEs call to the packet core (UTC). + */ + @JsonProperty(value = "lastReadAt") + private OffsetDateTime lastReadAt; + + /** + * Creates an instance of UeInfoPropertiesFormat class. + */ + public UeInfoPropertiesFormat() { + } + + /** + * Get the ratType property: RAT Type. + * + * @return the ratType value. + */ + public RatType ratType() { + return this.ratType; + } + + /** + * Set the ratType property: RAT Type. + * + * @param ratType the ratType value to set. + * @return the UeInfoPropertiesFormat object itself. + */ + public UeInfoPropertiesFormat withRatType(RatType ratType) { + this.ratType = ratType; + return this; + } + + /** + * Get the ueState property: State of the UE. + * + * @return the ueState value. + */ + public UeState ueState() { + return this.ueState; + } + + /** + * Set the ueState property: State of the UE. + * + * @param ueState the ueState value to set. + * @return the UeInfoPropertiesFormat object itself. + */ + public UeInfoPropertiesFormat withUeState(UeState ueState) { + this.ueState = ueState; + return this; + } + + /** + * Get the ueIpAddresses property: The ueIpAddresses property. + * + * @return the ueIpAddresses value. + */ + public List ueIpAddresses() { + return this.ueIpAddresses; + } + + /** + * Set the ueIpAddresses property: The ueIpAddresses property. + * + * @param ueIpAddresses the ueIpAddresses value to set. + * @return the UeInfoPropertiesFormat object itself. + */ + public UeInfoPropertiesFormat withUeIpAddresses(List ueIpAddresses) { + this.ueIpAddresses = ueIpAddresses; + return this; + } + + /** + * Get the lastReadAt property: The timestamp of last list UEs call to the packet core (UTC). + * + * @return the lastReadAt value. + */ + public OffsetDateTime lastReadAt() { + return this.lastReadAt; + } + + /** + * Set the lastReadAt property: The timestamp of last list UEs call to the packet core (UTC). + * + * @param lastReadAt the lastReadAt value to set. + * @return the UeInfoPropertiesFormat object itself. + */ + public UeInfoPropertiesFormat withLastReadAt(OffsetDateTime lastReadAt) { + this.lastReadAt = lastReadAt; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (ratType() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ratType in model UeInfoPropertiesFormat")); + } + if (ueState() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ueState in model UeInfoPropertiesFormat")); + } + if (ueIpAddresses() != null) { + ueIpAddresses().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeInfoPropertiesFormat.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInformations.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInformations.java new file mode 100644 index 0000000000000..1a4a50ec6d7d1 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeInformations.java @@ -0,0 +1,38 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of UeInformations. + */ +public interface UeInformations { + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName); + + /** + * List all UEs and their state in a packet core. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param packetCoreControlPlaneName The name of the packet core control plane. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for packet core list UEs API call as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String packetCoreControlPlaneName, Context context); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeIpAddress.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeIpAddress.java new file mode 100644 index 0000000000000..7aee6d56e7b4e --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeIpAddress.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * UE IP address. + */ +@Fluent +public final class UeIpAddress { + /* + * IPv4 address. + */ + @JsonProperty(value = "ipV4Addr") + private String ipV4Addr; + + /** + * Creates an instance of UeIpAddress class. + */ + public UeIpAddress() { + } + + /** + * Get the ipV4Addr property: IPv4 address. + * + * @return the ipV4Addr value. + */ + public String ipV4Addr() { + return this.ipV4Addr; + } + + /** + * Set the ipV4Addr property: IPv4 address. + * + * @param ipV4Addr the ipV4Addr value to set. + * @return the UeIpAddress object itself. + */ + public UeIpAddress withIpV4Addr(String ipV4Addr) { + this.ipV4Addr = ipV4Addr; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeLocationInfo.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeLocationInfo.java new file mode 100644 index 0000000000000..55392d7dd79e3 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeLocationInfo.java @@ -0,0 +1,123 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * UE Location Info properties. + */ +@Fluent +public final class UeLocationInfo { + /* + * Location Type + */ + @JsonProperty(value = "locationType", required = true) + private String locationType; + + /* + * Type Allocation Code of UE + */ + @JsonProperty(value = "tac", required = true) + private String tac; + + /* + * PLMN Identifier + */ + @JsonProperty(value = "plmn", required = true) + private PlmnId plmn; + + /** + * Creates an instance of UeLocationInfo class. + */ + public UeLocationInfo() { + } + + /** + * Get the locationType property: Location Type. + * + * @return the locationType value. + */ + public String locationType() { + return this.locationType; + } + + /** + * Set the locationType property: Location Type. + * + * @param locationType the locationType value to set. + * @return the UeLocationInfo object itself. + */ + public UeLocationInfo withLocationType(String locationType) { + this.locationType = locationType; + return this; + } + + /** + * Get the tac property: Type Allocation Code of UE. + * + * @return the tac value. + */ + public String tac() { + return this.tac; + } + + /** + * Set the tac property: Type Allocation Code of UE. + * + * @param tac the tac value to set. + * @return the UeLocationInfo object itself. + */ + public UeLocationInfo withTac(String tac) { + this.tac = tac; + return this; + } + + /** + * Get the plmn property: PLMN Identifier. + * + * @return the plmn value. + */ + public PlmnId plmn() { + return this.plmn; + } + + /** + * Set the plmn property: PLMN Identifier. + * + * @param plmn the plmn value to set. + * @return the UeLocationInfo object itself. + */ + public UeLocationInfo withPlmn(PlmnId plmn) { + this.plmn = plmn; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (locationType() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property locationType in model UeLocationInfo")); + } + if (tac() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property tac in model UeLocationInfo")); + } + if (plmn() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property plmn in model UeLocationInfo")); + } else { + plmn().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeLocationInfo.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeQosFlow.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeQosFlow.java new file mode 100644 index 0000000000000..7086b3b5c2fa7 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeQosFlow.java @@ -0,0 +1,138 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * QoS Flow. + */ +@Fluent +public final class UeQosFlow { + /* + * Qos Flow Identifier + */ + @JsonProperty(value = "qfi", required = true) + private int qfi; + + /* + * 5G QoS Identifier. + */ + @JsonProperty(value = "fiveqi", required = true) + private int fiveqi; + + /* + * Maximum Bit Rate + */ + @JsonProperty(value = "mbr") + private Ambr mbr; + + /* + * Guaranteed Bit Rate + */ + @JsonProperty(value = "gbr") + private Ambr gbr; + + /** + * Creates an instance of UeQosFlow class. + */ + public UeQosFlow() { + } + + /** + * Get the qfi property: Qos Flow Identifier. + * + * @return the qfi value. + */ + public int qfi() { + return this.qfi; + } + + /** + * Set the qfi property: Qos Flow Identifier. + * + * @param qfi the qfi value to set. + * @return the UeQosFlow object itself. + */ + public UeQosFlow withQfi(int qfi) { + this.qfi = qfi; + return this; + } + + /** + * Get the fiveqi property: 5G QoS Identifier. + * + * @return the fiveqi value. + */ + public int fiveqi() { + return this.fiveqi; + } + + /** + * Set the fiveqi property: 5G QoS Identifier. + * + * @param fiveqi the fiveqi value to set. + * @return the UeQosFlow object itself. + */ + public UeQosFlow withFiveqi(int fiveqi) { + this.fiveqi = fiveqi; + return this; + } + + /** + * Get the mbr property: Maximum Bit Rate. + * + * @return the mbr value. + */ + public Ambr mbr() { + return this.mbr; + } + + /** + * Set the mbr property: Maximum Bit Rate. + * + * @param mbr the mbr value to set. + * @return the UeQosFlow object itself. + */ + public UeQosFlow withMbr(Ambr mbr) { + this.mbr = mbr; + return this; + } + + /** + * Get the gbr property: Guaranteed Bit Rate. + * + * @return the gbr value. + */ + public Ambr gbr() { + return this.gbr; + } + + /** + * Set the gbr property: Guaranteed Bit Rate. + * + * @param gbr the gbr value to set. + * @return the UeQosFlow object itself. + */ + public UeQosFlow withGbr(Ambr gbr) { + this.gbr = gbr; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (mbr() != null) { + mbr().validate(); + } + if (gbr() != null) { + gbr().validate(); + } + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo4G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo4G.java new file mode 100644 index 0000000000000..ffdc79a165b87 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo4G.java @@ -0,0 +1,149 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * UE Session Info for 4G. + */ +@Fluent +public final class UeSessionInfo4G { + /* + * EPS bearer identifier + */ + @JsonProperty(value = "ebi", required = true) + private int ebi; + + /* + * Access point name + */ + @JsonProperty(value = "apn", required = true) + private String apn; + + /* + * UE IP address + */ + @JsonProperty(value = "ueIpAddress", required = true) + private UeIpAddress ueIpAddress; + + /* + * Packet Data Network Type + */ + @JsonProperty(value = "pdnType", required = true) + private PdnType pdnType; + + /** + * Creates an instance of UeSessionInfo4G class. + */ + public UeSessionInfo4G() { + } + + /** + * Get the ebi property: EPS bearer identifier. + * + * @return the ebi value. + */ + public int ebi() { + return this.ebi; + } + + /** + * Set the ebi property: EPS bearer identifier. + * + * @param ebi the ebi value to set. + * @return the UeSessionInfo4G object itself. + */ + public UeSessionInfo4G withEbi(int ebi) { + this.ebi = ebi; + return this; + } + + /** + * Get the apn property: Access point name. + * + * @return the apn value. + */ + public String apn() { + return this.apn; + } + + /** + * Set the apn property: Access point name. + * + * @param apn the apn value to set. + * @return the UeSessionInfo4G object itself. + */ + public UeSessionInfo4G withApn(String apn) { + this.apn = apn; + return this; + } + + /** + * Get the ueIpAddress property: UE IP address. + * + * @return the ueIpAddress value. + */ + public UeIpAddress ueIpAddress() { + return this.ueIpAddress; + } + + /** + * Set the ueIpAddress property: UE IP address. + * + * @param ueIpAddress the ueIpAddress value to set. + * @return the UeSessionInfo4G object itself. + */ + public UeSessionInfo4G withUeIpAddress(UeIpAddress ueIpAddress) { + this.ueIpAddress = ueIpAddress; + return this; + } + + /** + * Get the pdnType property: Packet Data Network Type. + * + * @return the pdnType value. + */ + public PdnType pdnType() { + return this.pdnType; + } + + /** + * Set the pdnType property: Packet Data Network Type. + * + * @param pdnType the pdnType value to set. + * @return the UeSessionInfo4G object itself. + */ + public UeSessionInfo4G withPdnType(PdnType pdnType) { + this.pdnType = pdnType; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (apn() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property apn in model UeSessionInfo4G")); + } + if (ueIpAddress() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ueIpAddress in model UeSessionInfo4G")); + } else { + ueIpAddress().validate(); + } + if (pdnType() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property pdnType in model UeSessionInfo4G")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeSessionInfo4G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo5G.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo5G.java new file mode 100644 index 0000000000000..8ec8922f4563d --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeSessionInfo5G.java @@ -0,0 +1,246 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * UE Session Info for 5G. + */ +@Fluent +public final class UeSessionInfo5G { + /* + * PDU session identifier + */ + @JsonProperty(value = "pduSessionId", required = true) + private int pduSessionId; + + /* + * Data network name + */ + @JsonProperty(value = "dnn", required = true) + private String dnn; + + /* + * Single-network slice selection assistance information (S-NSSAI). + */ + @JsonProperty(value = "snssai", required = true) + private Snssai snssai; + + /* + * UE IP address + */ + @JsonProperty(value = "ueIpAddress", required = true) + private UeIpAddress ueIpAddress; + + /* + * Packet Data Network Type + */ + @JsonProperty(value = "pdnType", required = true) + private PdnType pdnType; + + /* + * Aggregate maximum bit rate. + */ + @JsonProperty(value = "ambr", required = true) + private Ambr ambr; + + /* + * The qosFlow property. + */ + @JsonProperty(value = "qosFlow", required = true) + private List qosFlow; + + /** + * Creates an instance of UeSessionInfo5G class. + */ + public UeSessionInfo5G() { + } + + /** + * Get the pduSessionId property: PDU session identifier. + * + * @return the pduSessionId value. + */ + public int pduSessionId() { + return this.pduSessionId; + } + + /** + * Set the pduSessionId property: PDU session identifier. + * + * @param pduSessionId the pduSessionId value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withPduSessionId(int pduSessionId) { + this.pduSessionId = pduSessionId; + return this; + } + + /** + * Get the dnn property: Data network name. + * + * @return the dnn value. + */ + public String dnn() { + return this.dnn; + } + + /** + * Set the dnn property: Data network name. + * + * @param dnn the dnn value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withDnn(String dnn) { + this.dnn = dnn; + return this; + } + + /** + * Get the snssai property: Single-network slice selection assistance information (S-NSSAI). + * + * @return the snssai value. + */ + public Snssai snssai() { + return this.snssai; + } + + /** + * Set the snssai property: Single-network slice selection assistance information (S-NSSAI). + * + * @param snssai the snssai value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withSnssai(Snssai snssai) { + this.snssai = snssai; + return this; + } + + /** + * Get the ueIpAddress property: UE IP address. + * + * @return the ueIpAddress value. + */ + public UeIpAddress ueIpAddress() { + return this.ueIpAddress; + } + + /** + * Set the ueIpAddress property: UE IP address. + * + * @param ueIpAddress the ueIpAddress value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withUeIpAddress(UeIpAddress ueIpAddress) { + this.ueIpAddress = ueIpAddress; + return this; + } + + /** + * Get the pdnType property: Packet Data Network Type. + * + * @return the pdnType value. + */ + public PdnType pdnType() { + return this.pdnType; + } + + /** + * Set the pdnType property: Packet Data Network Type. + * + * @param pdnType the pdnType value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withPdnType(PdnType pdnType) { + this.pdnType = pdnType; + return this; + } + + /** + * Get the ambr property: Aggregate maximum bit rate. + * + * @return the ambr value. + */ + public Ambr ambr() { + return this.ambr; + } + + /** + * Set the ambr property: Aggregate maximum bit rate. + * + * @param ambr the ambr value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withAmbr(Ambr ambr) { + this.ambr = ambr; + return this; + } + + /** + * Get the qosFlow property: The qosFlow property. + * + * @return the qosFlow value. + */ + public List qosFlow() { + return this.qosFlow; + } + + /** + * Set the qosFlow property: The qosFlow property. + * + * @param qosFlow the qosFlow value to set. + * @return the UeSessionInfo5G object itself. + */ + public UeSessionInfo5G withQosFlow(List qosFlow) { + this.qosFlow = qosFlow; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (dnn() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property dnn in model UeSessionInfo5G")); + } + if (snssai() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property snssai in model UeSessionInfo5G")); + } else { + snssai().validate(); + } + if (ueIpAddress() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ueIpAddress in model UeSessionInfo5G")); + } else { + ueIpAddress().validate(); + } + if (pdnType() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property pdnType in model UeSessionInfo5G")); + } + if (ambr() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ambr in model UeSessionInfo5G")); + } else { + ambr().validate(); + } + if (qosFlow() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property qosFlow in model UeSessionInfo5G")); + } else { + qosFlow().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(UeSessionInfo5G.class); +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeState.java new file mode 100644 index 0000000000000..25829be3b05ee --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeState.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * State of the UE. + */ +public final class UeState extends ExpandableStringEnum { + /** + * Static value Connected for UeState. + */ + public static final UeState CONNECTED = fromString("Connected"); + + /** + * Static value Idle for UeState. + */ + public static final UeState IDLE = fromString("Idle"); + + /** + * Static value Detached for UeState. + */ + public static final UeState DETACHED = fromString("Detached"); + + /** + * Static value Deregistered for UeState. + */ + public static final UeState DEREGISTERED = fromString("Deregistered"); + + /** + * Static value Unknown for UeState. + */ + public static final UeState UNKNOWN = fromString("Unknown"); + + /** + * Creates a new instance of UeState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public UeState() { + } + + /** + * Creates or finds a UeState from its string representation. + * + * @param name a name to look for. + * @return the corresponding UeState. + */ + @JsonCreator + public static UeState fromString(String name) { + return fromString(name, UeState.class); + } + + /** + * Gets known UeState values. + * + * @return known UeState values. + */ + public static Collection values() { + return values(UeState.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeUsageSetting.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeUsageSetting.java new file mode 100644 index 0000000000000..315cc07a9ec8f --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UeUsageSetting.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The UE's usage setting. + */ +public final class UeUsageSetting extends ExpandableStringEnum { + /** + * Static value VoiceCentric for UeUsageSetting. + */ + public static final UeUsageSetting VOICE_CENTRIC = fromString("VoiceCentric"); + + /** + * Static value DataCentric for UeUsageSetting. + */ + public static final UeUsageSetting DATA_CENTRIC = fromString("DataCentric"); + + /** + * Creates a new instance of UeUsageSetting value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public UeUsageSetting() { + } + + /** + * Creates or finds a UeUsageSetting from its string representation. + * + * @param name a name to look for. + * @return the corresponding UeUsageSetting. + */ + @JsonCreator + public static UeUsageSetting fromString(String name) { + return fromString(name, UeUsageSetting.class); + } + + /** + * Gets known UeUsageSetting values. + * + * @return known UeUsageSetting values. + */ + public static Collection values() { + return values(UeUsageSetting.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UserAssignedIdentity.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UserAssignedIdentity.java index 16a3586e13418..4368d01cfb87d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UserAssignedIdentity.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/UserAssignedIdentity.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.UUID; -/** User assigned identity properties. */ +/** + * User assigned identity properties. + */ @Immutable public final class UserAssignedIdentity { /* @@ -23,13 +25,15 @@ public final class UserAssignedIdentity { @JsonProperty(value = "clientId", access = JsonProperty.Access.WRITE_ONLY) private UUID clientId; - /** Creates an instance of UserAssignedIdentity class. */ + /** + * Creates an instance of UserAssignedIdentity class. + */ public UserAssignedIdentity() { } /** * Get the principalId property: The principal ID of the assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -38,7 +42,7 @@ public UUID principalId() { /** * Get the clientId property: The client ID of the assigned identity. - * + * * @return the clientId value. */ public UUID clientId() { @@ -47,7 +51,7 @@ public UUID clientId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/VersionState.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/VersionState.java index 0efdef95d4d81..b7208a33b2f84 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/VersionState.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/VersionState.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The state of this packet core control plane version. */ +/** + * The state of this packet core control plane version. + */ public final class VersionState extends ExpandableStringEnum { - /** Static value Unknown for VersionState. */ + /** + * Static value Unknown for VersionState. + */ public static final VersionState UNKNOWN = fromString("Unknown"); - /** Static value Preview for VersionState. */ + /** + * Static value Preview for VersionState. + */ public static final VersionState PREVIEW = fromString("Preview"); - /** Static value Validating for VersionState. */ + /** + * Static value Validating for VersionState. + */ public static final VersionState VALIDATING = fromString("Validating"); - /** Static value ValidationFailed for VersionState. */ + /** + * Static value ValidationFailed for VersionState. + */ public static final VersionState VALIDATION_FAILED = fromString("ValidationFailed"); - /** Static value Active for VersionState. */ + /** + * Static value Active for VersionState. + */ public static final VersionState ACTIVE = fromString("Active"); - /** Static value Deprecated for VersionState. */ + /** + * Static value Deprecated for VersionState. + */ public static final VersionState DEPRECATED = fromString("Deprecated"); /** * Creates a new instance of VersionState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public VersionState() { /** * Creates or finds a VersionState from its string representation. - * + * * @param name a name to look for. * @return the corresponding VersionState. */ @@ -50,7 +64,7 @@ public static VersionState fromString(String name) { /** * Gets known VersionState values. - * + * * @return known VersionState values. */ public static Collection values() { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/package-info.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/package-info.java index 3b85eecc41cbf..f2d09ec04acaa 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/package-info.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/models/package-info.java @@ -3,7 +3,8 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the data models for MobileNetworkManagementClient. The resources in this API specification will be - * used to manage attached data network resources in mobile network attached to a particular packet core instance. + * Package containing the data models for MobileNetworkManagementClient. + * The resources in this API specification will be used to manage attached data network resources in mobile network + * attached to a particular packet core instance. */ package com.azure.resourcemanager.mobilenetwork.models; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/package-info.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/package-info.java index 1761cf6b8158c..bae4e24a43aae 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/package-info.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/com/azure/resourcemanager/mobilenetwork/package-info.java @@ -3,7 +3,8 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the classes for MobileNetworkManagementClient. The resources in this API specification will be - * used to manage attached data network resources in mobile network attached to a particular packet core instance. + * Package containing the classes for MobileNetworkManagementClient. + * The resources in this API specification will be used to manage attached data network resources in mobile network + * attached to a particular packet core instance. */ package com.azure.resourcemanager.mobilenetwork; diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/module-info.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/module-info.java index 33e8683e643f1..c396f5fd62a24 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/module-info.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.mobilenetwork { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.mobilenetwork; exports com.azure.resourcemanager.mobilenetwork.fluent; exports com.azure.resourcemanager.mobilenetwork.fluent.models; exports com.azure.resourcemanager.mobilenetwork.models; - - opens com.azure.resourcemanager.mobilenetwork.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.mobilenetwork.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.mobilenetwork.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.mobilenetwork.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/proxy-config.json b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/proxy-config.json index 0bd97c4226076..3dd101ba3fcbe 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/proxy-config.json +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/proxy-config.json @@ -1 +1 @@ -[ [ "com.azure.resourcemanager.mobilenetwork.implementation.AttachedDataNetworksClientImpl$AttachedDataNetworksService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.DataNetworksClientImpl$DataNetworksService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.DiagnosticsPackagesClientImpl$DiagnosticsPackagesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.MobileNetworksClientImpl$MobileNetworksService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCapturesClientImpl$PacketCapturesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCoreControlPlanesClientImpl$PacketCoreControlPlanesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCoreControlPlaneVersionsClientImpl$PacketCoreControlPlaneVersionsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCoreDataPlanesClientImpl$PacketCoreDataPlanesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.ServicesClientImpl$ServicesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SimsClientImpl$SimsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SimGroupsClientImpl$SimGroupsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SimPoliciesClientImpl$SimPoliciesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SitesClientImpl$SitesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SlicesClientImpl$SlicesService" ] ] \ No newline at end of file +[ [ "com.azure.resourcemanager.mobilenetwork.implementation.AttachedDataNetworksClientImpl$AttachedDataNetworksService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.DataNetworksClientImpl$DataNetworksService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.DiagnosticsPackagesClientImpl$DiagnosticsPackagesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.MobileNetworksClientImpl$MobileNetworksService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCapturesClientImpl$PacketCapturesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCoreControlPlanesClientImpl$PacketCoreControlPlanesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCoreControlPlaneVersionsClientImpl$PacketCoreControlPlaneVersionsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.PacketCoreDataPlanesClientImpl$PacketCoreDataPlanesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.ServicesClientImpl$ServicesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SimsClientImpl$SimsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SimGroupsClientImpl$SimGroupsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SimPoliciesClientImpl$SimPoliciesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SitesClientImpl$SitesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.SlicesClientImpl$SlicesService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.ExtendedUeInformationsClientImpl$ExtendedUeInformationsService" ], [ "com.azure.resourcemanager.mobilenetwork.implementation.UeInformationsClientImpl$UeInformationsService" ] ] \ No newline at end of file diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/reflect-config.json b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/reflect-config.json index e7bde877e8b74..56b1ec71346ce 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/reflect-config.json +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-mobilenetwork/reflect-config.json @@ -88,6 +88,36 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetwork", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetworkHomeNetworkPublicKeys", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPublicKey", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.mobilenetwork.models.MobileNetworkListResult", "allDeclaredConstructors" : true, @@ -214,17 +244,7 @@ "allDeclaredFields" : true, "allDeclaredMethods" : true }, { - "name" : "com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentity", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.mobilenetwork.models.IdentityAndTagsObject", + "name" : "com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPrivateKeysProvisioning", "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true @@ -493,6 +513,121 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfoProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeInfoList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeInfoPropertiesFormat", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.DnnIpPair", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeIpAddress", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.GNbId", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.GlobalRanNodeId", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeInfo5G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeInfo5GProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.Guti5G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.AmfId", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeConnectionInfo5G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeLocationInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeSessionInfo5G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeQosFlow", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeInfo4G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeInfo4GProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.Guti4G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.MmeId", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeConnectionInfo4G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeSessionInfo4G", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.mobilenetwork.models.ProvisioningState", "allDeclaredConstructors" : true, @@ -508,6 +643,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.mobilenetwork.models.PacketCaptureStatus", "allDeclaredConstructors" : true, @@ -554,7 +694,7 @@ "allDeclaredFields" : true, "allDeclaredMethods" : true }, { - "name" : "com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType", + "name" : "com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPrivateKeysProvisioningState", "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true @@ -608,6 +748,31 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.RatType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.RrcEstablishmentCause", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.UeUsageSetting", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.mobilenetwork.models.PdnType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType", "allDeclaredConstructors" : true, diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateSamples.java index 01ec60498648e..2a850bb64549d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateSamples.java @@ -12,33 +12,30 @@ import com.azure.resourcemanager.mobilenetwork.models.PortReuseHoldTimes; import java.util.Arrays; -/** Samples for AttachedDataNetworks CreateOrUpdate. */ +/** + * Samples for AttachedDataNetworks CreateOrUpdate. + */ public final class AttachedDataNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkCreate.json */ /** * Sample code: Create attached data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createAttachedDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .define("TestAttachedDataNetwork") - .withRegion("eastus") + manager.attachedDataNetworks().define("TestAttachedDataNetwork").withRegion("eastus") .withExistingPacketCoreDataPlane("rg1", "TestPacketCoreCP", "TestPacketCoreDP") .withUserPlaneDataInterface(new InterfaceProperties().withName("N6")) .withDnsAddresses(Arrays.asList("1.1.1.1")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.ENABLED) - .withPortRange(new PortRange().withMinPort(1024).withMaxPort(49999)) - .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(120).withUdp(60)) - .withPinholeLimits(65536) - .withPinholeTimeouts(new PinholeTimeouts().withTcp(180).withUdp(30).withIcmp(30))) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.ENABLED) + .withPortRange(new PortRange().withMinPort(1024).withMaxPort(49999)) + .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(120).withUdp(60)).withPinholeLimits(65536) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(180).withUdp(30).withIcmp(30))) .withUserEquipmentAddressPoolPrefix(Arrays.asList("2.2.0.0/16")) - .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("2.4.0.0/16")) - .create(); + .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("2.4.0.0/16")).create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteSamples.java index a04069a63f9ae..0f956d4a2ff34 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteSamples.java @@ -4,25 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for AttachedDataNetworks Delete. */ +/** + * Samples for AttachedDataNetworks Delete. + */ public final class AttachedDataNetworksDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkDelete.json */ /** * Sample code: Delete attached data network resource. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void deleteAttachedDataNetworkResource( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .delete( - "rg1", - "TestPacketCoreCP", - "TestPacketCoreDP", - "TestAttachedDataNetwork", - com.azure.core.util.Context.NONE); + public static void + deleteAttachedDataNetworkResource(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.attachedDataNetworks().delete("rg1", "TestPacketCoreCP", "TestPacketCoreDP", "TestAttachedDataNetwork", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetSamples.java index a8f4e56b9a449..be299045f0648 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetSamples.java @@ -4,24 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for AttachedDataNetworks Get. */ +/** + * Samples for AttachedDataNetworks Get. + */ public final class AttachedDataNetworksGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkGet.json */ /** * Sample code: Get attached data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getAttachedDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .getWithResponse( - "rg1", - "TestPacketCoreCP", - "TestPacketCoreDP", - "TestAttachedDataNetwork", - com.azure.core.util.Context.NONE); + manager.attachedDataNetworks().getWithResponse("rg1", "TestPacketCoreCP", "TestPacketCoreDP", + "TestAttachedDataNetwork", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneSamples.java index c9e88f9126037..f46204494c5c6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for AttachedDataNetworks ListByPacketCoreDataPlane. */ +/** + * Samples for AttachedDataNetworks ListByPacketCoreDataPlane. + */ public final class AttachedDataNetworksListByPacketCoreDataPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkListByPacketCoreDataPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkListByPacketCoreDataPlane.json */ /** * Sample code: List attached data networks in a data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listAttachedDataNetworksInADataPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .attachedDataNetworks() - .listByPacketCoreDataPlane("rg1", "TestPacketCoreCP", "TestPacketCoreDP", com.azure.core.util.Context.NONE); + public static void + listAttachedDataNetworksInADataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.attachedDataNetworks().listByPacketCoreDataPlane("rg1", "TestPacketCoreCP", "TestPacketCoreDP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksUpdateTagsSamples.java index dae6cde48c7e9..366e92d3dab19 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksUpdateTagsSamples.java @@ -8,28 +8,24 @@ import java.util.HashMap; import java.util.Map; -/** Samples for AttachedDataNetworks UpdateTags. */ +/** + * Samples for AttachedDataNetworks UpdateTags. + */ public final class AttachedDataNetworksUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/AttachedDataNetworkUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * AttachedDataNetworkUpdateTags.json */ /** * Sample code: Update attached data network tags. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void updateAttachedDataNetworkTags( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - AttachedDataNetwork resource = - manager - .attachedDataNetworks() - .getWithResponse( - "rg1", - "TestPacketCoreCP", - "TestPacketCoreDP", - "TestAttachedDataNetwork", - com.azure.core.util.Context.NONE) - .getValue(); + public static void + updateAttachedDataNetworkTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + AttachedDataNetwork resource = manager.attachedDataNetworks().getWithResponse("rg1", "TestPacketCoreCP", + "TestPacketCoreDP", "TestAttachedDataNetwork", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateSamples.java index 82eed0ec9b40c..8b8ef3cdde7da 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateSamples.java @@ -4,23 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DataNetworks CreateOrUpdate. */ +/** + * Samples for DataNetworks CreateOrUpdate. + */ public final class DataNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/DataNetworkCreate + * .json */ /** * Sample code: Create data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .dataNetworks() - .define("testDataNetwork") - .withRegion("eastus") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .withDescription("myFavouriteDataNetwork") - .create(); + manager.dataNetworks().define("testDataNetwork").withRegion("eastus") + .withExistingMobileNetwork("rg1", "testMobileNetwork").withDescription("myFavouriteDataNetwork").create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteSamples.java index e18369681ca6a..3d47cf3ad492b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DataNetworks Delete. */ +/** + * Samples for DataNetworks Delete. + */ public final class DataNetworksDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/DataNetworkDelete + * .json */ /** * Sample code: Delete data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetSamples.java index 6b9c599acadec..f09ceb8a5c41b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DataNetworks Get. */ +/** + * Samples for DataNetworks Get. + */ public final class DataNetworksGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/DataNetworkGet. + * json */ /** * Sample code: Get data network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getDataNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .dataNetworks() - .getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", com.azure.core.util.Context.NONE); + manager.dataNetworks().getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkSamples.java index 540f3ce5bf03e..b4b7c650ba286 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DataNetworks ListByMobileNetwork. */ +/** + * Samples for DataNetworks ListByMobileNetwork. + */ public final class DataNetworksListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DataNetworkListByMobileNetwork.json */ /** * Sample code: List data networks in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listDataNetworksInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listDataNetworksInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.dataNetworks().listByMobileNetwork("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksUpdateTagsSamples.java index 9ed7f1f3e62b6..0f9cdf5cf12d8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksUpdateTagsSamples.java @@ -8,22 +8,24 @@ import java.util.HashMap; import java.util.Map; -/** Samples for DataNetworks UpdateTags. */ +/** + * Samples for DataNetworks UpdateTags. + */ public final class DataNetworksUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DataNetworkUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DataNetworkUpdateTags.json */ /** * Sample code: Update data network tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateDataNetworkTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - DataNetwork resource = - manager - .dataNetworks() - .getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", com.azure.core.util.Context.NONE) - .getValue(); + DataNetwork resource = manager.dataNetworks() + .getWithResponse("rg1", "testMobileNetwork", "testDataNetwork", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateSamples.java index e674b4f7b08d3..6221d13d46a66 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DiagnosticsPackages CreateOrUpdate. */ +/** + * Samples for DiagnosticsPackages CreateOrUpdate. + */ public final class DiagnosticsPackagesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageCreate.json */ /** * Sample code: Create diagnostics package. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createDiagnosticsPackage(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .diagnosticsPackages() - .createOrUpdate("rg1", "TestPacketCoreCP", "dp1", com.azure.core.util.Context.NONE); + manager.diagnosticsPackages().createOrUpdate("rg1", "TestPacketCoreCP", "dp1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteSamples.java index 38ea3a52e479d..0a88400dff2fd 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DiagnosticsPackages Delete. */ +/** + * Samples for DiagnosticsPackages Delete. + */ public final class DiagnosticsPackagesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageDelete.json */ /** * Sample code: Delete diagnostics package. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteDiagnosticsPackage(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetSamples.java index a7d0455bd28b6..15285188cb552 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DiagnosticsPackages Get. */ +/** + * Samples for DiagnosticsPackages Get. + */ public final class DiagnosticsPackagesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageGet.json */ /** * Sample code: Get diagnostics package. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getDiagnosticsPackage(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .diagnosticsPackages() - .getWithResponse("rg1", "TestPacketCoreCP", "dp1", com.azure.core.util.Context.NONE); + manager.diagnosticsPackages().getWithResponse("rg1", "TestPacketCoreCP", "dp1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneSamples.java index b9605c1502b6c..eb5d3481688dc 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for DiagnosticsPackages ListByPacketCoreControlPlane. */ +/** + * Samples for DiagnosticsPackages ListByPacketCoreControlPlane. + */ public final class DiagnosticsPackagesListByPacketCoreControlPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/DiagnosticsPackageListByPacketCoreControlPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * DiagnosticsPackageListByPacketCoreControlPlane.json */ /** * Sample code: List diagnostics packages under a packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listDiagnosticsPackagesUnderAPacketCoreControlPlane( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .diagnosticsPackages() - .listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + manager.diagnosticsPackages().listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationGetSamples.java new file mode 100644 index 0000000000000..961508a88b74a --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationGetSamples.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +/** + * Samples for ExtendedUeInformation Get. + */ +public final class ExtendedUeInformationGetSamples { + /* + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * ExtendedUeInfo4GGet.json + */ + /** + * Sample code: Get UE Information 4G. + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void getUEInformation4G(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.extendedUeInformations().getWithResponse("rg1", "TestPacketCoreCP", "84449105622", + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * ExtendedUeInfo5GGet.json + */ + /** + * Sample code: Get UE Information 5G. + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void getUEInformation5G(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.extendedUeInformations().getWithResponse("rg1", "TestPacketCoreCP", "84449105622", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksCreateOrUpdateSamples.java index 76c163d83477b..42d20904dac23 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksCreateOrUpdateSamples.java @@ -4,25 +4,38 @@ package com.azure.resourcemanager.mobilenetwork.generated; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPublicKey; import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetwork; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetworkHomeNetworkPublicKeys; +import java.util.Arrays; -/** Samples for MobileNetworks CreateOrUpdate. */ +/** + * Samples for MobileNetworks CreateOrUpdate. + */ public final class MobileNetworksCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkCreate.json */ /** * Sample code: Create mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .mobileNetworks() - .define("testMobileNetwork") - .withRegion("eastus") - .withExistingResourceGroup("rg1") + manager.mobileNetworks().define("testMobileNetwork").withRegion("eastus").withExistingResourceGroup("rg1") .withPublicLandMobileNetworkIdentifier(new PlmnId().withMcc("001").withMnc("01")) + .withPublicLandMobileNetworks(Arrays.asList(new PublicLandMobileNetwork().withMcc("001").withMnc("01") + .withHomeNetworkPublicKeys(new PublicLandMobileNetworkHomeNetworkPublicKeys() + .withProfileA(Arrays.asList( + new HomeNetworkPublicKey().withId(1) + .withUrl("https://contosovault.vault.azure.net/secrets/exampleHnpk"), + new HomeNetworkPublicKey().withId(2).withUrl( + "https://contosovault.vault.azure.net/secrets/exampleHnpk2/5e4876e9140e4e16bfe6e2cf92e0cbd2"))) + .withProfileB(Arrays.asList(new HomeNetworkPublicKey().withId(1) + .withUrl("https://contosovault.vault.azure.net/secrets/exampleHnpkProfileB")))))) .create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteSamples.java index b3a8b57e23117..b05dc344565d0 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for MobileNetworks Delete. */ +/** + * Samples for MobileNetworks Delete. + */ public final class MobileNetworksDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkDelete.json */ /** * Sample code: Delete mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksGetByResourceGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksGetByResourceGroupSamples.java index c90f63a76de5b..492f70904ec10 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksGetByResourceGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksGetByResourceGroupSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for MobileNetworks GetByResourceGroup. */ +/** + * Samples for MobileNetworks GetByResourceGroup. + */ public final class MobileNetworksGetByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/MobileNetworkGet. + * json */ /** * Sample code: Get mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .mobileNetworks() - .getByResourceGroupWithResponse("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); + manager.mobileNetworks().getByResourceGroupWithResponse("rg1", "testMobileNetwork", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListByResourceGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListByResourceGroupSamples.java index 9db602557a9f0..b7e3ff8f60668 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListByResourceGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListByResourceGroupSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for MobileNetworks ListByResourceGroup. */ +/** + * Samples for MobileNetworks ListByResourceGroup. + */ public final class MobileNetworksListByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkListByResourceGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkListByResourceGroup.json */ /** * Sample code: List mobile networks in resource group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listMobileNetworksInResourceGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listMobileNetworksInResourceGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.mobileNetworks().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListSamples.java index f582a2b1ccc4d..1320663daed55 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksListSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for MobileNetworks List. */ +/** + * Samples for MobileNetworks List. + */ public final class MobileNetworksListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkListBySubscription.json */ /** * Sample code: List mobile networks in a subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listMobileNetworksInASubscription( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listMobileNetworksInASubscription(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.mobileNetworks().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksUpdateTagsSamples.java index 7a751342a0105..389018ac20f1e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksUpdateTagsSamples.java @@ -8,22 +8,23 @@ import java.util.HashMap; import java.util.Map; -/** Samples for MobileNetworks UpdateTags. */ +/** + * Samples for MobileNetworks UpdateTags. + */ public final class MobileNetworksUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/MobileNetworkUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * MobileNetworkUpdateTags.json */ /** * Sample code: Update mobile network tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateMobileNetworkTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - MobileNetwork resource = - manager - .mobileNetworks() - .getByResourceGroupWithResponse("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE) - .getValue(); + MobileNetwork resource = manager.mobileNetworks() + .getByResourceGroupWithResponse("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListSamples.java index 1c1ea89f95e51..c3ab13ee876f2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/OperationList.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/OperationList. + * json */ /** * Sample code: Get Registration Operations. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getRegistrationOperations(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateSamples.java index 437cf9bdd216d..f6da0620a6522 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateSamples.java @@ -6,30 +6,26 @@ import java.util.Arrays; -/** Samples for PacketCaptures CreateOrUpdate. */ +/** + * Samples for PacketCaptures CreateOrUpdate. + */ public final class PacketCapturesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCaptureCreate.json */ /** * Sample code: Create packet capture. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createPacketCapture(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCaptures() - .define("pc1") - .withExistingPacketCoreControlPlane("rg1", "TestPacketCoreCP") - .withNetworkInterfaces( - Arrays - .asList( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP", - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP", - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestADN")) - .withBytesToCapturePerPacket(10000L) - .withTotalBytesPerSession(100000L) - .withTimeLimitInSeconds(100) - .create(); + manager.packetCaptures().define("pc1").withExistingPacketCoreControlPlane("rg1", "TestPacketCoreCP") + .withNetworkInterfaces(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP", + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP", + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestADN")) + .withBytesToCapturePerPacket(10000L).withTotalBytesPerSession(100000L).withTimeLimitInSeconds(100).create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteSamples.java index 488d0916ebdc1..b7a75e2adcfb7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCaptures Delete. */ +/** + * Samples for PacketCaptures Delete. + */ public final class PacketCapturesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCaptureDelete.json */ /** * Sample code: Delete packet capture. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deletePacketCapture(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetSamples.java index c0540c098c24c..f49b4716ba37f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCaptures Get. */ +/** + * Samples for PacketCaptures Get. + */ public final class PacketCapturesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/PacketCaptureGet. + * json */ /** * Sample code: Get packet capture session. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCaptureSession(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneSamples.java index f871aeaaf07cf..b511d7c469c4d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCaptures ListByPacketCoreControlPlane. */ +/** + * Samples for PacketCaptures ListByPacketCoreControlPlane. + */ public final class PacketCapturesListByPacketCoreControlPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureListByPacketCoreControlPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCaptureListByPacketCoreControlPlane.json */ /** * Sample code: List packet capture sessions under a packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listPacketCaptureSessionsUnderAPacketCoreControlPlane( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCaptures() - .listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + manager.packetCaptures().listByPacketCoreControlPlane("rg1", "TestPacketCoreCP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopSamples.java index 8890f6fb883f0..d8b12438d982c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCaptures Stop. */ +/** + * Samples for PacketCaptures Stop. + */ public final class PacketCapturesStopSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCaptureStop.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/PacketCaptureStop + * .json */ /** * Sample code: Stop packet capture session. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void stopPacketCaptureSession(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionSamples.java index bda60f73fc112..c9bafdf887678 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlaneVersions GetBySubscription. */ +/** + * Samples for PacketCoreControlPlaneVersions GetBySubscription. + */ public final class PacketCoreControlPlaneVersionsGetBySubscriptionSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionGetBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionGetBySubscription.json */ /** * Sample code: Get packet core control plane version by subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCoreControlPlaneVersionBySubscription( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlaneVersions() - .getBySubscriptionWithResponse("PMN-4-11-1", com.azure.core.util.Context.NONE); + manager.packetCoreControlPlaneVersions().getBySubscriptionWithResponse("PMN-4-11-1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetSamples.java index 753a3cf92d936..7911ed0349841 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlaneVersions Get. */ +/** + * Samples for PacketCoreControlPlaneVersions Get. + */ public final class PacketCoreControlPlaneVersionsGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionGet.json */ /** * Sample code: Get packet core control plane version. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void getPacketCoreControlPlaneVersion( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + getPacketCoreControlPlaneVersion(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlaneVersions().getWithResponse("PMN-4-11-1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionSamples.java index 816d29e20698d..02055b836d804 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlaneVersions ListBySubscription. */ +/** + * Samples for PacketCoreControlPlaneVersions ListBySubscription. + */ public final class PacketCoreControlPlaneVersionsListBySubscriptionSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionListBySubscription.json */ /** * Sample code: Get supported packet core control plane versions by subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSupportedPacketCoreControlPlaneVersionsBySubscription( diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListSamples.java index d9474bc7ae47f..7e0d2d6087586 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlaneVersions List. */ +/** + * Samples for PacketCoreControlPlaneVersions List. + */ public final class PacketCoreControlPlaneVersionsListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneVersionList.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneVersionList.json */ /** * Sample code: Get supported packet core control plane versions. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSupportedPacketCoreControlPlaneVersions( diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageSamples.java index 6f9e6c3dc63bc..040f3bcdb0ea8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageSamples.java @@ -6,26 +6,25 @@ import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneCollectDiagnosticsPackage; -/** Samples for PacketCoreControlPlanes CollectDiagnosticsPackage. */ +/** + * Samples for PacketCoreControlPlanes CollectDiagnosticsPackage. + */ public final class PacketCoreControlPlanesCollectDiagnosticsPackageSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneCollectDiagnosticsPackage.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneCollectDiagnosticsPackage.json */ /** * Sample code: Collect diagnostics package from packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void collectDiagnosticsPackageFromPacketCoreControlPlane( com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlanes() - .collectDiagnosticsPackage( - "rg1", - "TestPacketCoreCP", - new PacketCoreControlPlaneCollectDiagnosticsPackage() - .withStorageAccountBlobUrl( - "https://contosoaccount.blob.core.windows.net/container/diagnosticsPackage.zip"), - com.azure.core.util.Context.NONE); + manager.packetCoreControlPlanes().collectDiagnosticsPackage("rg1", "TestPacketCoreCP", + new PacketCoreControlPlaneCollectDiagnosticsPackage().withStorageAccountBlobUrl( + "https://contosoaccount.blob.core.windows.net/container/diagnosticsPackage.zip"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateSamples.java index aa9518a336bf1..ea427716201d5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateSamples.java @@ -23,61 +23,43 @@ import com.azure.resourcemanager.mobilenetwork.models.SiteResourceId; import java.util.Arrays; -/** Samples for PacketCoreControlPlanes CreateOrUpdate. */ +/** + * Samples for PacketCoreControlPlanes CreateOrUpdate. + */ public final class PacketCoreControlPlanesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneCreate.json */ /** * Sample code: Create packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void createPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlanes() - .define("TestPacketCoreCP") - .withRegion("eastus") + public static void + createPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.packetCoreControlPlanes().define("TestPacketCoreCP").withRegion("eastus") .withExistingResourceGroup("rg1") - .withSites( - Arrays - .asList( - new SiteResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/sites/testSite"))) - .withPlatform( - new PlatformConfiguration() - .withType(PlatformType.AKS_HCI) - .withAzureStackEdgeDevice( - new AzureStackEdgeDeviceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.DataBoxEdge/dataBoxEdgeDevices/TestAzureStackEdgeDevice")) - .withConnectedCluster( - new ConnectedClusterResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.Kubernetes/connectedClusters/TestConnectedCluster")) - .withCustomLocation( - new CustomLocationResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ExtendedLocation/customLocations/TestCustomLocation"))) - .withControlPlaneAccessInterface(new InterfaceProperties().withName("N2")) - .withSku(BillingSku.G0) + .withSites(Arrays.asList(new SiteResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/sites/testSite"))) + .withPlatform(new PlatformConfiguration().withType(PlatformType.AKS_HCI) + .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.DataBoxEdge/dataBoxEdgeDevices/TestAzureStackEdgeDevice")) + .withConnectedCluster(new ConnectedClusterResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.Kubernetes/connectedClusters/TestConnectedCluster")) + .withCustomLocation(new CustomLocationResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.ExtendedLocation/customLocations/TestCustomLocation"))) + .withControlPlaneAccessInterface(new InterfaceProperties().withName("N2")).withSku(BillingSku.G0) .withLocalDiagnosticsAccess( - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.AAD) - .withHttpsServerCertificate( - new HttpsServerCertificate() - .withCertificateUrl("https://contosovault.vault.azure.net/certificates/ingress"))) + new LocalDiagnosticsAccessConfiguration().withAuthenticationType(AuthenticationType.AAD) + .withHttpsServerCertificate(new HttpsServerCertificate() + .withCertificateUrl("https://contosovault.vault.azure.net/certificates/ingress"))) .withInstallation(new Installation().withDesiredState(DesiredInstallationState.INSTALLED)) - .withCoreNetworkTechnology(CoreNetworkType.FIVE_GC) - .withVersion("0.2.0") - .withUeMtu(1600) - .withEventHub( - new EventHubConfiguration() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.EventHub/namespaces/contosoNamespace/eventHubs/contosoHub") - .withReportingInterval(60)) + .withCoreNetworkTechnology(CoreNetworkType.FIVE_GC).withVersion("0.2.0").withUeMtu(1600) + .withEventHub(new EventHubConfiguration().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.EventHub/namespaces/contosoNamespace/eventHubs/contosoHub") + .withReportingInterval(60)) .withSignaling( new SignalingConfiguration().withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(1024))) .create(); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteSamples.java index 9da26ac9cec8e..606eb48d270f9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlanes Delete. */ +/** + * Samples for PacketCoreControlPlanes Delete. + */ public final class PacketCoreControlPlanesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneDelete.json */ /** * Sample code: Delete packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void deletePacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + deletePacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlanes().delete("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupSamples.java index 5dc8c9039b97d..2d06b26a7c935 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlanes GetByResourceGroup. */ +/** + * Samples for PacketCoreControlPlanes GetByResourceGroup. + */ public final class PacketCoreControlPlanesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneGet.json */ /** * Sample code: Get packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreControlPlanes() - .getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + manager.packetCoreControlPlanes().getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupSamples.java index 30515dde2bb00..577c3e44aca58 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlanes ListByResourceGroup. */ +/** + * Samples for PacketCoreControlPlanes ListByResourceGroup. + */ public final class PacketCoreControlPlanesListByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneListByResourceGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneListByResourceGroup.json */ /** * Sample code: List packet core control planes in resource group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listPacketCoreControlPlanesInResourceGroup( diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListSamples.java index 0d2b1a764bae2..f8e041a34181e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlanes List. */ +/** + * Samples for PacketCoreControlPlanes List. + */ public final class PacketCoreControlPlanesListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneListBySubscription.json */ /** * Sample code: List packet core control planes in a subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listPacketCoreControlPlanesInASubscription( diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallSamples.java index a4123ddf253cb..fa4eb456ceaf3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlanes Reinstall. */ +/** + * Samples for PacketCoreControlPlanes Reinstall. + */ public final class PacketCoreControlPlanesReinstallSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneReinstall.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneReinstall.json */ /** * Sample code: Reinstall packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void reinstallPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + reinstallPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlanes().reinstall("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackSamples.java index 7ae97ae127f2d..5540e6dc204a3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreControlPlanes Rollback. */ +/** + * Samples for PacketCoreControlPlanes Rollback. + */ public final class PacketCoreControlPlanesRollbackSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlaneRollback.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlaneRollback.json */ /** * Sample code: Rollback packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void rollbackPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + rollbackPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.packetCoreControlPlanes().rollback("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesUpdateTagsSamples.java index 21902cd01d58d..035a552f5cbbc 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesUpdateTagsSamples.java @@ -11,33 +11,29 @@ import java.util.HashMap; import java.util.Map; -/** Samples for PacketCoreControlPlanes UpdateTags. */ +/** + * Samples for PacketCoreControlPlanes UpdateTags. + */ public final class PacketCoreControlPlanesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreControlPlanePatch.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreControlPlanePatch.json */ /** * Sample code: Patch packet core control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void patchPacketCoreControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - PacketCoreControlPlane resource = - manager - .packetCoreControlPlanes() - .getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", - new UserAssignedIdentity()))) + public static void + patchPacketCoreControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + PacketCoreControlPlane resource = manager.packetCoreControlPlanes() + .getByResourceGroupWithResponse("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", + new UserAssignedIdentity()))) .apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateSamples.java index e6992079f4096..e705f1990b344 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateSamples.java @@ -6,23 +6,23 @@ import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; -/** Samples for PacketCoreDataPlanes CreateOrUpdate. */ +/** + * Samples for PacketCoreDataPlanes CreateOrUpdate. + */ public final class PacketCoreDataPlanesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneCreate.json */ /** * Sample code: Create packet core data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createPacketCoreDataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .define("testPacketCoreDP") - .withRegion("eastus") + manager.packetCoreDataPlanes().define("testPacketCoreDP").withRegion("eastus") .withExistingPacketCoreControlPlane("rg1", "testPacketCoreCP") - .withUserPlaneAccessInterface(new InterfaceProperties().withName("N3")) - .create(); + .withUserPlaneAccessInterface(new InterfaceProperties().withName("N3")).create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteSamples.java index 5a24b6babdc4a..f48827db950bf 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreDataPlanes Delete. */ +/** + * Samples for PacketCoreDataPlanes Delete. + */ public final class PacketCoreDataPlanesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneDelete.json */ /** * Sample code: Delete packet core data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deletePacketCoreDataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .delete("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE); + manager.packetCoreDataPlanes().delete("rg1", "testPacketCoreCP", "testPacketCoreDP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetSamples.java index e8bc96e8fdb24..23ccd710255f6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreDataPlanes Get. */ +/** + * Samples for PacketCoreDataPlanes Get. + */ public final class PacketCoreDataPlanesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneGet.json */ /** * Sample code: Get packet core data plane. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getPacketCoreDataPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE); + manager.packetCoreDataPlanes().getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneSamples.java index 04e2918b945df..4f3fe69565a2a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for PacketCoreDataPlanes ListByPacketCoreControlPlane. */ +/** + * Samples for PacketCoreDataPlanes ListByPacketCoreControlPlane. + */ public final class PacketCoreDataPlanesListByPacketCoreControlPlaneSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneListByPacketCoreControlPlane.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneListByPacketCoreControlPlane.json */ /** * Sample code: List packet core data planes in a control plane. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listPacketCoreDataPlanesInAControlPlane( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .packetCoreDataPlanes() - .listByPacketCoreControlPlane("rg1", "testPacketCoreCP", com.azure.core.util.Context.NONE); + public static void + listPacketCoreDataPlanesInAControlPlane(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.packetCoreDataPlanes().listByPacketCoreControlPlane("rg1", "testPacketCoreCP", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesUpdateTagsSamples.java index cddadab0ac4ea..8cc3d44b5279f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesUpdateTagsSamples.java @@ -8,23 +8,25 @@ import java.util.HashMap; import java.util.Map; -/** Samples for PacketCoreDataPlanes UpdateTags. */ +/** + * Samples for PacketCoreDataPlanes UpdateTags. + */ public final class PacketCoreDataPlanesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/PacketCoreDataPlaneUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * PacketCoreDataPlaneUpdateTags.json */ /** * Sample code: Update packet core data plane tags. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void updatePacketCoreDataPlaneTags( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - PacketCoreDataPlane resource = - manager - .packetCoreDataPlanes() - .getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE) - .getValue(); + public static void + updatePacketCoreDataPlaneTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + PacketCoreDataPlane resource = manager.packetCoreDataPlanes() + .getWithResponse("rg1", "testPacketCoreCP", "testPacketCoreDP", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateSamples.java index 210be9f832c50..033db7ced5ed3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateSamples.java @@ -15,53 +15,36 @@ import com.azure.resourcemanager.mobilenetwork.models.TrafficControlPermission; import java.util.Arrays; -/** Samples for Services CreateOrUpdate. */ +/** + * Samples for Services CreateOrUpdate. + */ public final class ServicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceCreate. + * json */ /** * Sample code: Create service. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createService(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .services() - .define("TestService") - .withRegion("eastus") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .withServicePrecedence(255) - .withPccRules( - Arrays - .asList( - new PccRuleConfiguration() - .withRuleName("default-rule") - .withRulePrecedence(255) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(9) - .withAllocationAndRetentionPriorityLevel(9) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("IP-to-server") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("ip")) - .withRemoteIpList(Arrays.asList("10.3.4.0/24")) - .withPorts(Arrays.asList()))))) - .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(9) - .withAllocationAndRetentionPriorityLevel(9) + manager.services().define("TestService").withRegion("eastus") + .withExistingMobileNetwork("rg1", "testMobileNetwork").withServicePrecedence(255) + .withPccRules(Arrays.asList(new PccRuleConfiguration().withRuleName("default-rule").withRulePrecedence(255) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(9).withAllocationAndRetentionPriorityLevel(9) .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) .withMaximumBitRate(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps"))) + .withTrafficControl(TrafficControlPermission.ENABLED).withServiceDataFlowTemplates( + Arrays.asList(new ServiceDataFlowTemplate().withTemplateName("IP-to-server") + .withDirection(SdfDirection.UPLINK).withProtocol(Arrays.asList("ip")) + .withRemoteIpList(Arrays.asList("10.3.4.0/24")).withPorts(Arrays.asList()))))) + .withServiceQosPolicy(new QosPolicy().withFiveQi(9).withAllocationAndRetentionPriorityLevel(9) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps"))) .create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteSamples.java index 1f7d0f4954470..432973f27eb30 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Services Delete. */ +/** + * Samples for Services Delete. + */ public final class ServicesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceDelete. + * json */ /** * Sample code: Delete service. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteService(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetSamples.java index d668a35d58a30..ed2e3659f9f85 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Services Get. */ +/** + * Samples for Services Get. + */ public final class ServicesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceGet.json */ /** * Sample code: Get service. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getService(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkSamples.java index cf4b15d7ef2e6..b9f9badfcb581 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Services ListByMobileNetwork. */ +/** + * Samples for Services ListByMobileNetwork. + */ public final class ServicesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * ServiceListByMobileNetwork.json */ /** * Sample code: List services in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listServicesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .services() - .listByMobileNetwork("testResourceGroupName", "testMobileNetwork", com.azure.core.util.Context.NONE); + public static void + listServicesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.services().listByMobileNetwork("testResourceGroupName", "testMobileNetwork", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesUpdateTagsSamples.java index c59c8034e28ff..a3442b95b746f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesUpdateTagsSamples.java @@ -8,22 +8,23 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Services UpdateTags. */ +/** + * Samples for Services UpdateTags. + */ public final class ServicesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/ServiceUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ServiceUpdateTags + * .json */ /** * Sample code: Update service tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateServiceTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - Service resource = - manager - .services() - .getWithResponse("rg1", "testMobileNetwork", "TestService", com.azure.core.util.Context.NONE) - .getValue(); + Service resource = manager.services() + .getWithResponse("rg1", "testMobileNetwork", "TestService", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsCreateOrUpdateSamples.java index ad851f93f7c0b..839f6140240d4 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsCreateOrUpdateSamples.java @@ -12,34 +12,29 @@ import java.util.HashMap; import java.util.Map; -/** Samples for SimGroups CreateOrUpdate. */ +/** + * Samples for SimGroups CreateOrUpdate. + */ public final class SimGroupsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupCreate. + * json */ /** * Sample code: Create SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simGroups() - .define("testSimGroup") - .withRegion("eastus") - .withExistingResourceGroup("rg1") - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", - new UserAssignedIdentity()))) + manager.simGroups().define("testSimGroup").withRegion("eastus").withExistingResourceGroup("rg1") + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", + new UserAssignedIdentity()))) .withEncryptionKey(new KeyVaultKey().withKeyUrl("fakeTokenPlaceholder")) - .withMobileNetwork( - new MobileNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork")) + .withMobileNetwork(new MobileNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork")) .create(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteSamples.java index 92e73507d1072..5bd79f5ea3f3c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimGroups Delete. */ +/** + * Samples for SimGroups Delete. + */ public final class SimGroupsDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupDelete. + * json */ /** * Sample code: Delete SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsGetByResourceGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsGetByResourceGroupSamples.java index 975c39a36fd39..1090217dcb510 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsGetByResourceGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsGetByResourceGroupSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimGroups GetByResourceGroup. */ +/** + * Samples for SimGroups GetByResourceGroup. + */ public final class SimGroupsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupGet.json */ /** * Sample code: Get SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simGroups() - .getByResourceGroupWithResponse( - "testResourceGroupName", "testSimGroupName", com.azure.core.util.Context.NONE); + manager.simGroups().getByResourceGroupWithResponse("testResourceGroupName", "testSimGroupName", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListByResourceGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListByResourceGroupSamples.java index 33440b6250414..1a3034754b60a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListByResourceGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListByResourceGroupSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimGroups ListByResourceGroup. */ +/** + * Samples for SimGroups ListByResourceGroup. + */ public final class SimGroupsListByResourceGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupListByResourceGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimGroupListByResourceGroup.json */ /** * Sample code: List SIM groups in a resource group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listSIMGroupsInAResourceGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listSIMGroupsInAResourceGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.simGroups().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListSamples.java index 79273568b061d..c2b5672ecdf1e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsListSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimGroups List. */ +/** + * Samples for SimGroups List. + */ public final class SimGroupsListSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupListBySubscription.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimGroupListBySubscription.json */ /** * Sample code: List SIM groups in a subscription. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listSIMGroupsInASubscription( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listSIMGroupsInASubscription(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.simGroups().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsUpdateTagsSamples.java index d748b2f00fa51..b265bee0a29a0 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsUpdateTagsSamples.java @@ -11,32 +11,28 @@ import java.util.HashMap; import java.util.Map; -/** Samples for SimGroups UpdateTags. */ +/** + * Samples for SimGroups UpdateTags. + */ public final class SimGroupsUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGroupPatch.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGroupPatch. + * json */ /** * Sample code: Patch SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void patchSIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - SimGroup resource = - manager - .simGroups() - .getByResourceGroupWithResponse("rg1", "testSimGroup", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", - new UserAssignedIdentity()))) + SimGroup resource = manager.simGroups() + .getByResourceGroupWithResponse("rg1", "testSimGroup", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).withIdentity(new ManagedServiceIdentity() + .withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourcegroups/rg1/providers/Microsoft.ManagedIdentity/userAssignedIdentities/testUserAssignedManagedIdentity", + new UserAssignedIdentity()))) .apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateSamples.java index 6ddd649e5f536..b97245f1f1a16 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateSamples.java @@ -15,62 +15,41 @@ import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for SimPolicies CreateOrUpdate. */ +/** + * Samples for SimPolicies CreateOrUpdate. + */ public final class SimPoliciesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimPolicyCreate. + * json */ /** * Sample code: Create SIM policy. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createSIMPolicy(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simPolicies() - .define("testPolicy") - .withRegion("eastus") + manager.simPolicies().define("testPolicy").withRegion("eastus") .withExistingMobileNetwork("rg1", "testMobileNetwork") .withUeAmbr(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps")) - .withDefaultSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withSliceConfigurations( - Arrays - .asList( - new SliceConfiguration() - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withDefaultDataNetwork( - new DataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) - .withSessionAmbr(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps")) - .withFiveQi(9) - .withAllocationAndRetentionPriorityLevel(9) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes(Arrays.asList()) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/services/testService"))) - .withMaximumNumberOfBufferedPackets(200))))) - .withRegistrationTimer(3240) - .create(); + .withDefaultSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withSliceConfigurations(Arrays.asList(new SliceConfiguration().withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) + .withDataNetworkConfigurations(Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/dataNetworks/testdataNetwork")) + .withSessionAmbr(new Ambr().withUplink("500 Mbps").withDownlink("1 Gbps")).withFiveQi(9) + .withAllocationAndRetentionPriorityLevel(9) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4).withAdditionalAllowedSessionTypes(Arrays.asList()) + .withAllowedServices(Arrays.asList(new ServiceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/services/testService"))) + .withMaximumNumberOfBufferedPackets(200))))) + .withRegistrationTimer(3240).create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteSamples.java index 5adedc537387b..7a872cb0817ac 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimPolicies Delete. */ +/** + * Samples for SimPolicies Delete. + */ public final class SimPoliciesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimPolicyDelete. + * json */ /** * Sample code: Delete SIM policy. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteSIMPolicy(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetSamples.java index 04350aacdfe38..37e1306aa8636 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimPolicies Get. */ +/** + * Samples for SimPolicies Get. + */ public final class SimPoliciesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimPolicyGet.json */ /** * Sample code: Get SIM policy. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSIMPolicy(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simPolicies() - .getWithResponse("rg1", "testMobileNetwork", "testPolicy", com.azure.core.util.Context.NONE); + manager.simPolicies().getWithResponse("rg1", "testMobileNetwork", "testPolicy", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkSamples.java index 1d2995d452997..54134db758ac8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for SimPolicies ListByMobileNetwork. */ +/** + * Samples for SimPolicies ListByMobileNetwork. + */ public final class SimPoliciesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimPolicyListByMobileNetwork.json */ /** * Sample code: List SIM policies in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listSIMPoliciesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .simPolicies() - .listByMobileNetwork("testResourceGroupName", "testMobileNetwork", com.azure.core.util.Context.NONE); + public static void + listSIMPoliciesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.simPolicies().listByMobileNetwork("testResourceGroupName", "testMobileNetwork", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesUpdateTagsSamples.java index 43d2728d95e78..c52b87d4bb3c9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesUpdateTagsSamples.java @@ -8,22 +8,23 @@ import java.util.HashMap; import java.util.Map; -/** Samples for SimPolicies UpdateTags. */ +/** + * Samples for SimPolicies UpdateTags. + */ public final class SimPoliciesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimPolicyUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimPolicyUpdateTags.json */ /** * Sample code: Update SIM policy tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateSIMPolicyTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - SimPolicy resource = - manager - .simPolicies() - .getWithResponse("rg1", "testMobileNetwork", "testPolicy", com.azure.core.util.Context.NONE) - .getValue(); + SimPolicy resource = manager.simPolicies() + .getWithResponse("rg1", "testMobileNetwork", "testPolicy", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteSamples.java index 0ef96cbd78454..cf0bebe419d3b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteSamples.java @@ -7,24 +7,23 @@ import com.azure.resourcemanager.mobilenetwork.models.SimDeleteList; import java.util.Arrays; -/** Samples for Sims BulkDelete. */ +/** + * Samples for Sims BulkDelete. + */ public final class SimsBulkDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimBulkDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimBulkDelete. + * json */ /** * Sample code: Bulk delete SIMs from a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void bulkDeleteSIMsFromASIMGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .bulkDelete( - "testResourceGroupName", - "testSimGroup", - new SimDeleteList().withSims(Arrays.asList("testSim", "testSim2")), - com.azure.core.util.Context.NONE); + public static void + bulkDeleteSIMsFromASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.sims().bulkDelete("testResourceGroupName", "testSimGroup", + new SimDeleteList().withSims(Arrays.asList("testSim", "testSim2")), com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadEncryptedSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadEncryptedSamples.java index 06c1d7573b4de..fe69b05a45cea 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadEncryptedSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadEncryptedSamples.java @@ -13,82 +13,47 @@ import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for Sims BulkUploadEncrypted. */ +/** + * Samples for Sims BulkUploadEncrypted. + */ public final class SimsBulkUploadEncryptedSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimBulkUploadEncrypted.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SimBulkUploadEncrypted.json */ /** * Sample code: Bulk upload encrypted SIMs to a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void bulkUploadEncryptedSIMsToASIMGroup( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .bulkUploadEncrypted( - "rg1", - "testSimGroup", - new EncryptedSimUploadList() - .withVersion(1) - .withAzureKeyIdentifier(1) - .withVendorKeyFingerprint("fakeTokenPlaceholder") - .withEncryptedTransportKey("fakeTokenPlaceholder") - .withSignedTransportKey("fakeTokenPlaceholder") - .withSims( - Arrays - .asList( - new SimNameAndEncryptedProperties() - .withName("testSim") - .withEncryptedCredentials("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000000") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.1")))), - new SimNameAndEncryptedProperties() - .withName("testSim2") - .withEncryptedCredentials("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000001") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.2")))))), - com.azure.core.util.Context.NONE); + public static void + bulkUploadEncryptedSIMsToASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.sims().bulkUploadEncrypted("rg1", "testSimGroup", new EncryptedSimUploadList().withVersion(1) + .withAzureKeyIdentifier(1).withVendorKeyFingerprint("fakeTokenPlaceholder") + .withEncryptedTransportKey("fakeTokenPlaceholder").withSignedTransportKey("fakeTokenPlaceholder") + .withSims(Arrays.asList(new SimNameAndEncryptedProperties().withName("testSim") + .withEncryptedCredentials("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000000").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))), + new SimNameAndEncryptedProperties().withName("testSim2") + .withEncryptedCredentials("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000001").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.2")))))), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadSamples.java index e6750b8013f45..02280ee679957 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkUploadSamples.java @@ -13,78 +13,44 @@ import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for Sims BulkUpload. */ +/** + * Samples for Sims BulkUpload. + */ public final class SimsBulkUploadSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimBulkUpload.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimBulkUpload. + * json */ /** * Sample code: Bulk upload SIMs in a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void bulkUploadSIMsInASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .bulkUpload( - "rg1", - "testSimGroup", - new SimUploadList() - .withSims( - Arrays - .asList( - new SimNameAndProperties() - .withName("testSim") - .withAuthenticationKey("fakeTokenPlaceholder") - .withOperatorKeyCode("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000000") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.1")))), - new SimNameAndProperties() - .withName("testSim2") - .withAuthenticationKey("fakeTokenPlaceholder") - .withOperatorKeyCode("fakeTokenPlaceholder") - .withInternationalMobileSubscriberIdentity("00000") - .withIntegratedCircuitCardIdentifier("8900000000000000001") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp( - new SimStaticIpPropertiesStaticIp() - .withIpv4Address("2.4.0.2")))))), - com.azure.core.util.Context.NONE); + manager.sims().bulkUpload("rg1", "testSimGroup", new SimUploadList().withSims(Arrays.asList( + new SimNameAndProperties().withName("testSim").withAuthenticationKey("fakeTokenPlaceholder") + .withOperatorKeyCode("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000000").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))), + new SimNameAndProperties().withName("testSim2").withAuthenticationKey("fakeTokenPlaceholder") + .withOperatorKeyCode("fakeTokenPlaceholder").withInternationalMobileSubscriberIdentity("00000") + .withIntegratedCircuitCardIdentifier("8900000000000000001").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.2")))))), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsCreateOrUpdateSamples.java index cea7e2e4e5f36..6d8b4b1f60461 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsCreateOrUpdateSamples.java @@ -11,43 +11,33 @@ import com.azure.resourcemanager.mobilenetwork.models.SliceResourceId; import java.util.Arrays; -/** Samples for Sims CreateOrUpdate. */ +/** + * Samples for Sims CreateOrUpdate. + */ public final class SimsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimCreate.json */ /** * Sample code: Create SIM. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createSIM(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .define("testSim") - .withExistingSimGroup("rg1", "testSimGroup") + manager.sims().define("testSim").withExistingSimGroup("rg1", "testSimGroup") .withInternationalMobileSubscriberIdentity("00000") .withAuthenticationKey("00000000000000000000000000000000") .withOperatorKeyCode("00000000000000000000000000000000") - .withIntegratedCircuitCardIdentifier("8900000000000000000") - .withDeviceType("Video camera") - .withSimPolicy( - new SimPolicyResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) - .withStaticIpConfiguration( - Arrays - .asList( - new SimStaticIpProperties() - .withAttachedDataNetwork( - new AttachedDataNetworkResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) - .withSlice( - new SliceResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) - .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))) + .withIntegratedCircuitCardIdentifier("8900000000000000000").withDeviceType("Video camera") + .withSimPolicy(new SimPolicyResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/simPolicies/MySimPolicy")) + .withStaticIpConfiguration(Arrays.asList(new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP/packetCoreDataPlanes/TestPacketCoreDP/attachedDataNetworks/TestAttachedDataNetwork")) + .withSlice(new SliceResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/mobileNetworks/testMobileNetwork/slices/testSlice")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("2.4.0.1")))) .create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteSamples.java index 96d221ce10b95..054467252c785 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sims Delete. */ +/** + * Samples for Sims Delete. + */ public final class SimsDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimDelete.json */ /** * Sample code: Delete SIM. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteSIM(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsGetSamples.java index 7f64a0fe4ce97..dc5f80e3b0648 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsGetSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sims Get. */ +/** + * Samples for Sims Get. + */ public final class SimsGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimGet.json */ /** * Sample code: Get SIM. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getSIM(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sims() - .getWithResponse("testResourceGroupName", "testSimGroup", "testSimName", com.azure.core.util.Context.NONE); + manager.sims().getWithResponse("testResourceGroupName", "testSimGroup", "testSimName", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsListByGroupSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsListByGroupSamples.java index 1bdea1af36a0d..db096dfe74f9b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsListByGroupSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SimsListByGroupSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sims ListByGroup. */ +/** + * Samples for Sims ListByGroup. + */ public final class SimsListByGroupSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SimListBySimGroup.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SimListBySimGroup + * .json */ /** * Sample code: List SIMs in a SIM group. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void listSIMsInASIMGroup(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateSamples.java index 3d7091a2112bf..65cd5c42c79d9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateSamples.java @@ -4,22 +4,21 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sites CreateOrUpdate. */ +/** + * Samples for Sites CreateOrUpdate. + */ public final class SitesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteCreate.json */ /** * Sample code: Create mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sites() - .define("testSite") - .withRegion("testLocation") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .create(); + manager.sites().define("testSite").withRegion("testLocation") + .withExistingMobileNetwork("rg1", "testMobileNetwork").create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreSamples.java index 96aafd6f7d4f8..c215ab35693fb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreSamples.java @@ -7,29 +7,25 @@ import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneResourceId; import com.azure.resourcemanager.mobilenetwork.models.SiteDeletePacketCore; -/** Samples for Sites DeletePacketCore. */ +/** + * Samples for Sites DeletePacketCore. + */ public final class SitesDeletePacketCoreSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteDeletePacketCore.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SiteDeletePacketCore.json */ /** * Sample code: Delete packet core in mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void deletePacketCoreInMobileNetworkSite( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .sites() - .deletePacketCore( - "rg1", - "testMobileNetwork", - "testSite", - new SiteDeletePacketCore() - .withPacketCore( - new PacketCoreControlPlaneResourceId() - .withId( - "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP")), - com.azure.core.util.Context.NONE); + public static void + deletePacketCoreInMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.sites().deletePacketCore("rg1", "testMobileNetwork", "testSite", + new SiteDeletePacketCore().withPacketCore(new PacketCoreControlPlaneResourceId().withId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rg1/providers/Microsoft.MobileNetwork/packetCoreControlPlanes/TestPacketCoreCP")), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteSamples.java index 67a7e1613a29a..079c5ff4ef7a5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sites Delete. */ +/** + * Samples for Sites Delete. + */ public final class SitesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteDelete.json */ /** * Sample code: Delete mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetSamples.java index bd8934628967a..39fae0915e1a0 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sites Get. */ +/** + * Samples for Sites Get. + */ public final class SitesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteGet.json */ /** * Sample code: Get mobile network site. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getMobileNetworkSite(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkSamples.java index c0ac3140863c9..8a4d4fde624b4 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Sites ListByMobileNetwork. */ +/** + * Samples for Sites ListByMobileNetwork. + */ public final class SitesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SiteListByMobileNetwork.json */ /** * Sample code: List mobile network sites in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listMobileNetworkSitesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listMobileNetworkSitesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.sites().listByMobileNetwork("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesUpdateTagsSamples.java index 46498511946dc..a3aaa91b17590 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SitesUpdateTagsSamples.java @@ -8,23 +8,24 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Sites UpdateTags. */ +/** + * Samples for Sites UpdateTags. + */ public final class SitesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SiteUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SiteUpdateTags. + * json */ /** * Sample code: Update mobile network site tags. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void updateMobileNetworkSiteTags( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - Site resource = - manager - .sites() - .getWithResponse("rg1", "testMobileNetwork", "testSite", com.azure.core.util.Context.NONE) - .getValue(); + public static void + updateMobileNetworkSiteTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + Site resource = manager.sites() + .getWithResponse("rg1", "testMobileNetwork", "testSite", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateSamples.java index cb2e93fc2ef1f..6ddca629d5ab1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateSamples.java @@ -6,24 +6,21 @@ import com.azure.resourcemanager.mobilenetwork.models.Snssai; -/** Samples for Slices CreateOrUpdate. */ +/** + * Samples for Slices CreateOrUpdate. + */ public final class SlicesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceCreate.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceCreate.json */ /** * Sample code: Create network slice. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void createNetworkSlice(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - manager - .slices() - .define("testSlice") - .withRegion("eastus") - .withExistingMobileNetwork("rg1", "testMobileNetwork") - .withSnssai(new Snssai().withSst(1).withSd("1abcde")) - .withDescription("myFavouriteSlice") - .create(); + manager.slices().define("testSlice").withRegion("eastus").withExistingMobileNetwork("rg1", "testMobileNetwork") + .withSnssai(new Snssai().withSst(1).withSd("1abcde")).withDescription("myFavouriteSlice").create(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteSamples.java index c22a126ef2b5e..df20703847324 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Slices Delete. */ +/** + * Samples for Slices Delete. + */ public final class SlicesDeleteSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceDelete.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceDelete.json */ /** * Sample code: Delete network slice. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void deleteNetworkSlice(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetSamples.java index b4ec34febdb82..46cb1cd37d26e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Slices Get. */ +/** + * Samples for Slices Get. + */ public final class SlicesGetSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceGet.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceGet.json */ /** * Sample code: Get network slice. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void getNetworkSlice(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkSamples.java index 3323f94b3eada..626b50e12c1fc 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkSamples.java @@ -4,18 +4,22 @@ package com.azure.resourcemanager.mobilenetwork.generated; -/** Samples for Slices ListByMobileNetwork. */ +/** + * Samples for Slices ListByMobileNetwork. + */ public final class SlicesListByMobileNetworkSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceListByMobileNetwork.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/ + * SliceListByMobileNetwork.json */ /** * Sample code: List network slices in a mobile network. - * + * * @param manager Entry point to MobileNetworkManager. */ - public static void listNetworkSlicesInAMobileNetwork( - com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + public static void + listNetworkSlicesInAMobileNetwork(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { manager.slices().listByMobileNetwork("rg1", "testMobileNetwork", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesUpdateTagsSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesUpdateTagsSamples.java index 588e976cf654e..807d930874b45 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesUpdateTagsSamples.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesUpdateTagsSamples.java @@ -8,22 +8,23 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Slices UpdateTags. */ +/** + * Samples for Slices UpdateTags. + */ public final class SlicesUpdateTagsSamples { /* - * x-ms-original-file: specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2023-09-01/examples/SliceUpdateTags.json + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/SliceUpdateTags. + * json */ /** * Sample code: Update network slice tags. - * + * * @param manager Entry point to MobileNetworkManager. */ public static void updateNetworkSliceTags(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { - Slice resource = - manager - .slices() - .getWithResponse("rg1", "testMobileNetwork", "testSlice", com.azure.core.util.Context.NONE) - .getValue(); + Slice resource = manager.slices() + .getWithResponse("rg1", "testMobileNetwork", "testSlice", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationListSamples.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationListSamples.java new file mode 100644 index 0000000000000..e6c35edbb0f90 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/samples/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationListSamples.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +/** + * Samples for UeInformation List. + */ +public final class UeInformationListSamples { + /* + * x-ms-original-file: + * specification/mobilenetwork/resource-manager/Microsoft.MobileNetwork/stable/2024-02-01/examples/UeInfoList.json + */ + /** + * Sample code: Get UE Information. + * + * @param manager Entry point to MobileNetworkManager. + */ + public static void getUEInformation(com.azure.resourcemanager.mobilenetwork.MobileNetworkManager manager) { + manager.ueInformations().list("rg1", "TestPacketCoreCP", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmbrTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmbrTests.java index afa01d91b1a80..3a7511bc71868 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmbrTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmbrTests.java @@ -11,17 +11,16 @@ public final class AmbrTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Ambr model = - BinaryData.fromString("{\"uplink\":\"lbjbsyb\",\"downlink\":\"qwrvtldgmfp\"}").toObject(Ambr.class); - Assertions.assertEquals("lbjbsyb", model.uplink()); - Assertions.assertEquals("qwrvtldgmfp", model.downlink()); + Ambr model = BinaryData.fromString("{\"uplink\":\"stkwqqtch\",\"downlink\":\"alm\"}").toObject(Ambr.class); + Assertions.assertEquals("stkwqqtch", model.uplink()); + Assertions.assertEquals("alm", model.downlink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Ambr model = new Ambr().withUplink("lbjbsyb").withDownlink("qwrvtldgmfp"); + Ambr model = new Ambr().withUplink("stkwqqtch").withDownlink("alm"); model = BinaryData.fromObject(model).toObject(Ambr.class); - Assertions.assertEquals("lbjbsyb", model.uplink()); - Assertions.assertEquals("qwrvtldgmfp", model.downlink()); + Assertions.assertEquals("stkwqqtch", model.uplink()); + Assertions.assertEquals("alm", model.downlink()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmfIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmfIdTests.java new file mode 100644 index 0000000000000..1b526ec8d1373 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AmfIdTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.AmfId; +import org.junit.jupiter.api.Assertions; + +public final class AmfIdTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + AmfId model = BinaryData.fromString("{\"regionId\":161082288,\"setId\":2027553854,\"pointer\":1407254135}") + .toObject(AmfId.class); + Assertions.assertEquals(161082288, model.regionId()); + Assertions.assertEquals(2027553854, model.setId()); + Assertions.assertEquals(1407254135, model.pointer()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + AmfId model = new AmfId().withRegionId(161082288).withSetId(2027553854).withPointer(1407254135); + model = BinaryData.fromObject(model).toObject(AmfId.class); + Assertions.assertEquals(161082288, model.regionId()); + Assertions.assertEquals(2027553854, model.setId()); + Assertions.assertEquals(1407254135, model.pointer()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationIdTests.java index 06bbd73da6eee..49cab2abc52d2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationIdTests.java @@ -11,14 +11,14 @@ public final class AsyncOperationIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AsyncOperationId model = BinaryData.fromString("{\"id\":\"bhvgy\"}").toObject(AsyncOperationId.class); - Assertions.assertEquals("bhvgy", model.id()); + AsyncOperationId model = BinaryData.fromString("{\"id\":\"lrqjbhckfr\"}").toObject(AsyncOperationId.class); + Assertions.assertEquals("lrqjbhckfr", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AsyncOperationId model = new AsyncOperationId().withId("bhvgy"); + AsyncOperationId model = new AsyncOperationId().withId("lrqjbhckfr"); model = BinaryData.fromObject(model).toObject(AsyncOperationId.class); - Assertions.assertEquals("bhvgy", model.id()); + Assertions.assertEquals("lrqjbhckfr", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationStatusInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationStatusInnerTests.java index 0c87d24e78d7c..40bf3b8d1b5c7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationStatusInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AsyncOperationStatusInnerTests.java @@ -12,39 +12,32 @@ public final class AsyncOperationStatusInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AsyncOperationStatusInner model = - BinaryData - .fromString( - "{\"id\":\"s\",\"name\":\"ddystkiiuxhqy\",\"status\":\"dxorrqnbpoczv\",\"resourceId\":\"fqrvkdvjsllrmvvd\",\"startTime\":\"2021-07-25T18:56:56Z\",\"endTime\":\"2021-08-07T11:35:41Z\",\"percentComplete\":4.838999534427835,\"properties\":\"datalexxbczwtru\"}") - .toObject(AsyncOperationStatusInner.class); - Assertions.assertEquals("s", model.id()); - Assertions.assertEquals("ddystkiiuxhqy", model.name()); - Assertions.assertEquals("dxorrqnbpoczv", model.status()); - Assertions.assertEquals("fqrvkdvjsllrmvvd", model.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-25T18:56:56Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-07T11:35:41Z"), model.endTime()); - Assertions.assertEquals(4.838999534427835D, model.percentComplete()); + AsyncOperationStatusInner model = BinaryData.fromString( + "{\"id\":\"egukgjnpiucg\",\"name\":\"evqzntypmrbp\",\"status\":\"zcdrqjsdpydnfyhx\",\"resourceId\":\"oejzi\",\"startTime\":\"2021-10-16T23:09:14Z\",\"endTime\":\"2021-04-16T21:39:11Z\",\"percentComplete\":44.46839055655801,\"properties\":\"datazfbishcbkhaj\"}") + .toObject(AsyncOperationStatusInner.class); + Assertions.assertEquals("egukgjnpiucg", model.id()); + Assertions.assertEquals("evqzntypmrbp", model.name()); + Assertions.assertEquals("zcdrqjsdpydnfyhx", model.status()); + Assertions.assertEquals("oejzi", model.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-16T23:09:14Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-16T21:39:11Z"), model.endTime()); + Assertions.assertEquals(44.46839055655801D, model.percentComplete()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AsyncOperationStatusInner model = - new AsyncOperationStatusInner() - .withId("s") - .withName("ddystkiiuxhqy") - .withStatus("dxorrqnbpoczv") - .withResourceId("fqrvkdvjsllrmvvd") - .withStartTime(OffsetDateTime.parse("2021-07-25T18:56:56Z")) - .withEndTime(OffsetDateTime.parse("2021-08-07T11:35:41Z")) - .withPercentComplete(4.838999534427835D) - .withProperties("datalexxbczwtru"); + AsyncOperationStatusInner model = new AsyncOperationStatusInner().withId("egukgjnpiucg") + .withName("evqzntypmrbp").withStatus("zcdrqjsdpydnfyhx").withResourceId("oejzi") + .withStartTime(OffsetDateTime.parse("2021-10-16T23:09:14Z")) + .withEndTime(OffsetDateTime.parse("2021-04-16T21:39:11Z")).withPercentComplete(44.46839055655801D) + .withProperties("datazfbishcbkhaj"); model = BinaryData.fromObject(model).toObject(AsyncOperationStatusInner.class); - Assertions.assertEquals("s", model.id()); - Assertions.assertEquals("ddystkiiuxhqy", model.name()); - Assertions.assertEquals("dxorrqnbpoczv", model.status()); - Assertions.assertEquals("fqrvkdvjsllrmvvd", model.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-25T18:56:56Z"), model.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-07T11:35:41Z"), model.endTime()); - Assertions.assertEquals(4.838999534427835D, model.percentComplete()); + Assertions.assertEquals("egukgjnpiucg", model.id()); + Assertions.assertEquals("evqzntypmrbp", model.name()); + Assertions.assertEquals("zcdrqjsdpydnfyhx", model.status()); + Assertions.assertEquals("oejzi", model.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-16T23:09:14Z"), model.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-16T21:39:11Z"), model.endTime()); + Assertions.assertEquals(44.46839055655801D, model.percentComplete()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkInnerTests.java index a219ebad027b9..fb0dd08f86da2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkInnerTests.java @@ -20,11 +20,9 @@ public final class AttachedDataNetworkInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AttachedDataNetworkInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Canceled\",\"userPlaneDataInterface\":{\"name\":\"quvgjxpybczme\",\"ipv4Address\":\"tzopbsphrupidgsy\",\"ipv4Subnet\":\"ejhphoyc\",\"ipv4Gateway\":\"xaobhdxbmtqioqjz\"},\"dnsAddresses\":[\"tbmufpo\",\"noi\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":167840319,\"maxPort\":201473614},\"portReuseHoldTime\":{\"tcp\":1372882397,\"udp\":1472149737},\"pinholeLimits\":231072361,\"pinholeTimeouts\":{\"tcp\":1450276464,\"udp\":1792128890,\"icmp\":1864680039}},\"userEquipmentAddressPoolPrefix\":[\"c\"],\"userEquipmentStaticAddressPoolPrefix\":[\"dznrbtcqq\",\"nq\",\"lhqgnufooojy\",\"ifsqesaagdfmg\"]},\"location\":\"lhjxr\",\"tags\":{\"uajpsquc\":\"wmrvktsizntocipa\"},\"id\":\"poyfdkfogkn\",\"name\":\"gjofjd\",\"type\":\"eqsrdeupewnwreit\"}") - .toObject(AttachedDataNetworkInner.class); + AttachedDataNetworkInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Canceled\",\"userPlaneDataInterface\":{\"name\":\"quvgjxpybczme\",\"ipv4Address\":\"tzopbsphrupidgsy\",\"ipv4Subnet\":\"ejhphoyc\",\"ipv4Gateway\":\"xaobhdxbmtqioqjz\"},\"dnsAddresses\":[\"tbmufpo\",\"noi\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":167840319,\"maxPort\":201473614},\"portReuseHoldTime\":{\"tcp\":1372882397,\"udp\":1472149737},\"pinholeLimits\":231072361,\"pinholeTimeouts\":{\"tcp\":1450276464,\"udp\":1792128890,\"icmp\":1864680039}},\"userEquipmentAddressPoolPrefix\":[\"c\"],\"userEquipmentStaticAddressPoolPrefix\":[\"dznrbtcqq\",\"nq\",\"lhqgnufooojy\",\"ifsqesaagdfmg\"]},\"location\":\"lhjxr\",\"tags\":{\"uajpsquc\":\"wmrvktsizntocipa\"},\"id\":\"poyfdkfogkn\",\"name\":\"gjofjd\",\"type\":\"eqsrdeupewnwreit\"}") + .toObject(AttachedDataNetworkInner.class); Assertions.assertEquals("lhjxr", model.location()); Assertions.assertEquals("wmrvktsizntocipa", model.tags().get("uajpsquc")); Assertions.assertEquals("quvgjxpybczme", model.userPlaneDataInterface().name()); @@ -47,27 +45,17 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AttachedDataNetworkInner model = - new AttachedDataNetworkInner() - .withLocation("lhjxr") - .withTags(mapOf("uajpsquc", "wmrvktsizntocipa")) - .withUserPlaneDataInterface( - new InterfaceProperties() - .withName("quvgjxpybczme") - .withIpv4Address("tzopbsphrupidgsy") - .withIpv4Subnet("ejhphoyc") - .withIpv4Gateway("xaobhdxbmtqioqjz")) + AttachedDataNetworkInner model + = new AttachedDataNetworkInner().withLocation("lhjxr").withTags(mapOf("uajpsquc", "wmrvktsizntocipa")) + .withUserPlaneDataInterface(new InterfaceProperties().withName("quvgjxpybczme") + .withIpv4Address("tzopbsphrupidgsy").withIpv4Subnet("ejhphoyc").withIpv4Gateway("xaobhdxbmtqioqjz")) .withDnsAddresses(Arrays.asList("tbmufpo", "noi")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.DISABLED) - .withPortRange(new PortRange().withMinPort(167840319).withMaxPort(201473614)) - .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(1372882397).withUdp(1472149737)) - .withPinholeLimits(231072361) - .withPinholeTimeouts( - new PinholeTimeouts().withTcp(1450276464).withUdp(1792128890).withIcmp(1864680039))) - .withUserEquipmentAddressPoolPrefix(Arrays.asList("c")) - .withUserEquipmentStaticAddressPoolPrefix( + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.DISABLED) + .withPortRange(new PortRange().withMinPort(167840319).withMaxPort(201473614)) + .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(1372882397).withUdp(1472149737)) + .withPinholeLimits(231072361).withPinholeTimeouts( + new PinholeTimeouts().withTcp(1450276464).withUdp(1792128890).withIcmp(1864680039))) + .withUserEquipmentAddressPoolPrefix(Arrays.asList("c")).withUserEquipmentStaticAddressPoolPrefix( Arrays.asList("dznrbtcqq", "nq", "lhqgnufooojy", "ifsqesaagdfmg")); model = BinaryData.fromObject(model).toObject(AttachedDataNetworkInner.class); Assertions.assertEquals("lhjxr", model.location()); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkListResultTests.java index d326310c5fa71..2738a9945fc8c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkListResultTests.java @@ -21,11 +21,9 @@ public final class AttachedDataNetworkListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AttachedDataNetworkListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"userPlaneDataInterface\":{\"name\":\"t\",\"ipv4Address\":\"mdvpjhulsu\",\"ipv4Subnet\":\"mkjozkrwf\",\"ipv4Gateway\":\"iodjp\"},\"dnsAddresses\":[\"w\",\"jdpvwryo\"],\"naptConfiguration\":{\"enabled\":\"Enabled\",\"portRange\":{\"minPort\":2018987029,\"maxPort\":1285377118},\"portReuseHoldTime\":{\"tcp\":1137960460,\"udp\":124042122},\"pinholeLimits\":1063449943,\"pinholeTimeouts\":{\"tcp\":276706615,\"udp\":2140975130,\"icmp\":1060950423}},\"userEquipmentAddressPoolPrefix\":[\"dosyg\",\"xpaojakhmsbz\"],\"userEquipmentStaticAddressPoolPrefix\":[\"rzevdphlxaol\",\"hqtrgqjbpf\",\"fsinzgvfcjrwzoxx\"]},\"location\":\"felluwfzitonpe\",\"tags\":{\"ypininm\":\"jkjlxofpdvhpfx\",\"po\":\"yhuybbkpod\",\"ognarxzxtheotus\":\"ginuvamih\"},\"id\":\"vyevcciqi\",\"name\":\"nhungbw\",\"type\":\"zrnf\"},{\"properties\":{\"provisioningState\":\"Failed\",\"userPlaneDataInterface\":{\"name\":\"ispe\",\"ipv4Address\":\"tzfkufubl\",\"ipv4Subnet\":\"fxqeof\",\"ipv4Gateway\":\"e\"},\"dnsAddresses\":[\"hqjbasvmsmj\",\"ulngsntn\",\"ybkzgcwr\",\"clxxwrljdo\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":1132832602,\"maxPort\":422670277},\"portReuseHoldTime\":{\"tcp\":1066142513,\"udp\":747491790},\"pinholeLimits\":1343054120,\"pinholeTimeouts\":{\"tcp\":24386691,\"udp\":1728330738,\"icmp\":1272736991}},\"userEquipmentAddressPoolPrefix\":[\"sqrglssainq\",\"jwnzlljfmp\",\"eebvmgxsab\"],\"userEquipmentStaticAddressPoolPrefix\":[\"duuji\",\"c\",\"czdzev\",\"dhkrwpdappdsbdk\"]},\"location\":\"rwjfe\",\"tags\":{\"tmrldhugjzzdatq\":\"hutje\",\"gphuticndvka\":\"hocdgeab\",\"k\":\"zwyiftyhxhur\",\"kjfkg\":\"tyxolniwpwc\"},\"id\":\"awxklr\",\"name\":\"plwckbas\",\"type\":\"ypnddhsgcb\"},{\"properties\":{\"provisioningState\":\"Unknown\",\"userPlaneDataInterface\":{\"name\":\"ejk\",\"ipv4Address\":\"ynqgoulzndlikwyq\",\"ipv4Subnet\":\"fgibmadgakeq\",\"ipv4Gateway\":\"xybz\"},\"dnsAddresses\":[\"e\",\"qytbciq\",\"ouf\",\"mmnkzsmodmgl\"],\"naptConfiguration\":{\"enabled\":\"Enabled\",\"portRange\":{\"minPort\":1951665736,\"maxPort\":542784283},\"portReuseHoldTime\":{\"tcp\":1183608022,\"udp\":1581209984},\"pinholeLimits\":289494577,\"pinholeTimeouts\":{\"tcp\":1911925190,\"udp\":1035420183,\"icmp\":1593608704}},\"userEquipmentAddressPoolPrefix\":[\"rtumkdosvq\",\"hbmdgbbjfdd\"],\"userEquipmentStaticAddressPoolPrefix\":[\"mbe\",\"ppbhtqqrolfp\",\"psalgbqux\",\"gjyjgzjaoyfhrtxi\"]},\"location\":\"erkujys\",\"tags\":{\"awrlyx\":\"juvf\"},\"id\":\"jkcpr\",\"name\":\"nwbxgjvtbvpyssz\",\"type\":\"nruj\"}],\"nextLink\":\"uhmuouqfprwzwbn\"}") - .toObject(AttachedDataNetworkListResult.class); + AttachedDataNetworkListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"userPlaneDataInterface\":{\"name\":\"t\",\"ipv4Address\":\"mdvpjhulsu\",\"ipv4Subnet\":\"mkjozkrwf\",\"ipv4Gateway\":\"iodjp\"},\"dnsAddresses\":[\"w\",\"jdpvwryo\"],\"naptConfiguration\":{\"enabled\":\"Enabled\",\"portRange\":{\"minPort\":2018987029,\"maxPort\":1285377118},\"portReuseHoldTime\":{\"tcp\":1137960460,\"udp\":124042122},\"pinholeLimits\":1063449943,\"pinholeTimeouts\":{\"tcp\":276706615,\"udp\":2140975130,\"icmp\":1060950423}},\"userEquipmentAddressPoolPrefix\":[\"dosyg\",\"xpaojakhmsbz\"],\"userEquipmentStaticAddressPoolPrefix\":[\"rzevdphlxaol\",\"hqtrgqjbpf\",\"fsinzgvfcjrwzoxx\"]},\"location\":\"felluwfzitonpe\",\"tags\":{\"ypininm\":\"jkjlxofpdvhpfx\",\"po\":\"yhuybbkpod\",\"ognarxzxtheotus\":\"ginuvamih\"},\"id\":\"vyevcciqi\",\"name\":\"nhungbw\",\"type\":\"zrnf\"},{\"properties\":{\"provisioningState\":\"Failed\",\"userPlaneDataInterface\":{\"name\":\"ispe\",\"ipv4Address\":\"tzfkufubl\",\"ipv4Subnet\":\"fxqeof\",\"ipv4Gateway\":\"e\"},\"dnsAddresses\":[\"hqjbasvmsmj\",\"ulngsntn\",\"ybkzgcwr\",\"clxxwrljdo\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":1132832602,\"maxPort\":422670277},\"portReuseHoldTime\":{\"tcp\":1066142513,\"udp\":747491790},\"pinholeLimits\":1343054120,\"pinholeTimeouts\":{\"tcp\":24386691,\"udp\":1728330738,\"icmp\":1272736991}},\"userEquipmentAddressPoolPrefix\":[\"sqrglssainq\",\"jwnzlljfmp\",\"eebvmgxsab\"],\"userEquipmentStaticAddressPoolPrefix\":[\"duuji\",\"c\",\"czdzev\",\"dhkrwpdappdsbdk\"]},\"location\":\"rwjfe\",\"tags\":{\"tmrldhugjzzdatq\":\"hutje\",\"gphuticndvka\":\"hocdgeab\",\"k\":\"zwyiftyhxhur\",\"kjfkg\":\"tyxolniwpwc\"},\"id\":\"awxklr\",\"name\":\"plwckbas\",\"type\":\"ypnddhsgcb\"},{\"properties\":{\"provisioningState\":\"Unknown\",\"userPlaneDataInterface\":{\"name\":\"ejk\",\"ipv4Address\":\"ynqgoulzndlikwyq\",\"ipv4Subnet\":\"fgibmadgakeq\",\"ipv4Gateway\":\"xybz\"},\"dnsAddresses\":[\"e\",\"qytbciq\",\"ouf\",\"mmnkzsmodmgl\"],\"naptConfiguration\":{\"enabled\":\"Enabled\",\"portRange\":{\"minPort\":1951665736,\"maxPort\":542784283},\"portReuseHoldTime\":{\"tcp\":1183608022,\"udp\":1581209984},\"pinholeLimits\":289494577,\"pinholeTimeouts\":{\"tcp\":1911925190,\"udp\":1035420183,\"icmp\":1593608704}},\"userEquipmentAddressPoolPrefix\":[\"rtumkdosvq\",\"hbmdgbbjfdd\"],\"userEquipmentStaticAddressPoolPrefix\":[\"mbe\",\"ppbhtqqrolfp\",\"psalgbqux\",\"gjyjgzjaoyfhrtxi\"]},\"location\":\"erkujys\",\"tags\":{\"awrlyx\":\"juvf\"},\"id\":\"jkcpr\",\"name\":\"nwbxgjvtbvpyssz\",\"type\":\"nruj\"}],\"nextLink\":\"uhmuouqfprwzwbn\"}") + .toObject(AttachedDataNetworkListResult.class); Assertions.assertEquals("felluwfzitonpe", model.value().get(0).location()); Assertions.assertEquals("jkjlxofpdvhpfx", model.value().get(0).tags().get("ypininm")); Assertions.assertEquals("t", model.value().get(0).userPlaneDataInterface().name()); @@ -48,100 +46,58 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AttachedDataNetworkListResult model = - new AttachedDataNetworkListResult() + AttachedDataNetworkListResult model + = new AttachedDataNetworkListResult() .withValue( Arrays .asList( - new AttachedDataNetworkInner() - .withLocation("felluwfzitonpe") - .withTags( - mapOf( - "ypininm", - "jkjlxofpdvhpfx", - "po", - "yhuybbkpod", - "ognarxzxtheotus", - "ginuvamih")) - .withUserPlaneDataInterface( - new InterfaceProperties() - .withName("t") - .withIpv4Address("mdvpjhulsu") - .withIpv4Subnet("mkjozkrwf") - .withIpv4Gateway("iodjp")) + new AttachedDataNetworkInner().withLocation("felluwfzitonpe") + .withTags(mapOf("ypininm", "jkjlxofpdvhpfx", "po", "yhuybbkpod", "ognarxzxtheotus", + "ginuvamih")) + .withUserPlaneDataInterface(new InterfaceProperties().withName("t") + .withIpv4Address("mdvpjhulsu").withIpv4Subnet("mkjozkrwf").withIpv4Gateway("iodjp")) .withDnsAddresses(Arrays.asList("w", "jdpvwryo")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.ENABLED) - .withPortRange(new PortRange().withMinPort(2018987029).withMaxPort(1285377118)) - .withPortReuseHoldTime( - new PortReuseHoldTimes().withTcp(1137960460).withUdp(124042122)) - .withPinholeLimits(1063449943) - .withPinholeTimeouts( - new PinholeTimeouts() - .withTcp(276706615) - .withUdp(2140975130) - .withIcmp(1060950423))) - .withUserEquipmentAddressPoolPrefix(Arrays.asList("dosyg", "xpaojakhmsbz")) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.ENABLED) + .withPortRange(new PortRange().withMinPort(2018987029).withMaxPort(1285377118)) + .withPortReuseHoldTime( + new PortReuseHoldTimes().withTcp(1137960460).withUdp(124042122)) + .withPinholeLimits(1063449943) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(276706615).withUdp(2140975130) + .withIcmp(1060950423))) + .withUserEquipmentAddressPoolPrefix( + Arrays.asList("dosyg", "xpaojakhmsbz")) .withUserEquipmentStaticAddressPoolPrefix( Arrays.asList("rzevdphlxaol", "hqtrgqjbpf", "fsinzgvfcjrwzoxx")), new AttachedDataNetworkInner() .withLocation("rwjfe") - .withTags( - mapOf( - "tmrldhugjzzdatq", - "hutje", - "gphuticndvka", - "hocdgeab", - "k", - "zwyiftyhxhur", - "kjfkg", - "tyxolniwpwc")) - .withUserPlaneDataInterface( - new InterfaceProperties() - .withName("ispe") - .withIpv4Address("tzfkufubl") - .withIpv4Subnet("fxqeof") - .withIpv4Gateway("e")) + .withTags(mapOf("tmrldhugjzzdatq", "hutje", "gphuticndvka", "hocdgeab", "k", + "zwyiftyhxhur", "kjfkg", "tyxolniwpwc")) + .withUserPlaneDataInterface(new InterfaceProperties().withName("ispe") + .withIpv4Address("tzfkufubl").withIpv4Subnet("fxqeof").withIpv4Gateway("e")) .withDnsAddresses(Arrays.asList("hqjbasvmsmj", "ulngsntn", "ybkzgcwr", "clxxwrljdo")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.DISABLED) - .withPortRange(new PortRange().withMinPort(1132832602).withMaxPort(422670277)) - .withPortReuseHoldTime( - new PortReuseHoldTimes().withTcp(1066142513).withUdp(747491790)) - .withPinholeLimits(1343054120) - .withPinholeTimeouts( - new PinholeTimeouts() - .withTcp(24386691) - .withUdp(1728330738) - .withIcmp(1272736991))) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.DISABLED) + .withPortRange(new PortRange().withMinPort(1132832602).withMaxPort(422670277)) + .withPortReuseHoldTime( + new PortReuseHoldTimes().withTcp(1066142513).withUdp(747491790)) + .withPinholeLimits(1343054120) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(24386691).withUdp(1728330738) + .withIcmp(1272736991))) .withUserEquipmentAddressPoolPrefix( Arrays.asList("sqrglssainq", "jwnzlljfmp", "eebvmgxsab")) .withUserEquipmentStaticAddressPoolPrefix( Arrays.asList("duuji", "c", "czdzev", "dhkrwpdappdsbdk")), - new AttachedDataNetworkInner() - .withLocation("erkujys") - .withTags(mapOf("awrlyx", "juvf")) + new AttachedDataNetworkInner().withLocation("erkujys").withTags(mapOf("awrlyx", "juvf")) .withUserPlaneDataInterface( - new InterfaceProperties() - .withName("ejk") - .withIpv4Address("ynqgoulzndlikwyq") - .withIpv4Subnet("fgibmadgakeq") - .withIpv4Gateway("xybz")) + new InterfaceProperties().withName("ejk").withIpv4Address("ynqgoulzndlikwyq") + .withIpv4Subnet("fgibmadgakeq").withIpv4Gateway("xybz")) .withDnsAddresses(Arrays.asList("e", "qytbciq", "ouf", "mmnkzsmodmgl")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.ENABLED) - .withPortRange(new PortRange().withMinPort(1951665736).withMaxPort(542784283)) - .withPortReuseHoldTime( - new PortReuseHoldTimes().withTcp(1183608022).withUdp(1581209984)) - .withPinholeLimits(289494577) - .withPinholeTimeouts( - new PinholeTimeouts() - .withTcp(1911925190) - .withUdp(1035420183) - .withIcmp(1593608704))) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.ENABLED) + .withPortRange(new PortRange().withMinPort(1951665736).withMaxPort(542784283)) + .withPortReuseHoldTime( + new PortReuseHoldTimes().withTcp(1183608022).withUdp(1581209984)) + .withPinholeLimits(289494577) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(1911925190).withUdp(1035420183) + .withIcmp(1593608704))) .withUserEquipmentAddressPoolPrefix(Arrays.asList("rtumkdosvq", "hbmdgbbjfdd")) .withUserEquipmentStaticAddressPoolPrefix( Arrays.asList("mbe", "ppbhtqqrolfp", "psalgbqux", "gjyjgzjaoyfhrtxi")))); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkPropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkPropertiesFormatTests.java index bf30aa1aa6ee7..7bc0096fe4321 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkPropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkPropertiesFormatTests.java @@ -18,11 +18,9 @@ public final class AttachedDataNetworkPropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AttachedDataNetworkPropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Failed\",\"userPlaneDataInterface\":{\"name\":\"lusarh\",\"ipv4Address\":\"fcqhsmyurkd\",\"ipv4Subnet\":\"lx\",\"ipv4Gateway\":\"kuksjtxukcdm\"},\"dnsAddresses\":[\"rcryuanzwuxzdxta\",\"rlhm\",\"hfpmrqobmtukknr\",\"rtihfxtijbpz\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":477742160,\"maxPort\":1531004720},\"portReuseHoldTime\":{\"tcp\":271043498,\"udp\":1526885086},\"pinholeLimits\":504117688,\"pinholeTimeouts\":{\"tcp\":1760407196,\"udp\":1757371466,\"icmp\":1730607260}},\"userEquipmentAddressPoolPrefix\":[\"ufhfcbjysa\",\"ithxqhabifpi\"],\"userEquipmentStaticAddressPoolPrefix\":[\"czbysc\",\"pqxu\"]}") - .toObject(AttachedDataNetworkPropertiesFormat.class); + AttachedDataNetworkPropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Failed\",\"userPlaneDataInterface\":{\"name\":\"lusarh\",\"ipv4Address\":\"fcqhsmyurkd\",\"ipv4Subnet\":\"lx\",\"ipv4Gateway\":\"kuksjtxukcdm\"},\"dnsAddresses\":[\"rcryuanzwuxzdxta\",\"rlhm\",\"hfpmrqobmtukknr\",\"rtihfxtijbpz\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":477742160,\"maxPort\":1531004720},\"portReuseHoldTime\":{\"tcp\":271043498,\"udp\":1526885086},\"pinholeLimits\":504117688,\"pinholeTimeouts\":{\"tcp\":1760407196,\"udp\":1757371466,\"icmp\":1730607260}},\"userEquipmentAddressPoolPrefix\":[\"ufhfcbjysa\",\"ithxqhabifpi\"],\"userEquipmentStaticAddressPoolPrefix\":[\"czbysc\",\"pqxu\"]}") + .toObject(AttachedDataNetworkPropertiesFormat.class); Assertions.assertEquals("lusarh", model.userPlaneDataInterface().name()); Assertions.assertEquals("fcqhsmyurkd", model.userPlaneDataInterface().ipv4Address()); Assertions.assertEquals("lx", model.userPlaneDataInterface().ipv4Subnet()); @@ -43,25 +41,17 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AttachedDataNetworkPropertiesFormat model = - new AttachedDataNetworkPropertiesFormat() - .withUserPlaneDataInterface( - new InterfaceProperties() - .withName("lusarh") - .withIpv4Address("fcqhsmyurkd") - .withIpv4Subnet("lx") - .withIpv4Gateway("kuksjtxukcdm")) - .withDnsAddresses(Arrays.asList("rcryuanzwuxzdxta", "rlhm", "hfpmrqobmtukknr", "rtihfxtijbpz")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.DISABLED) - .withPortRange(new PortRange().withMinPort(477742160).withMaxPort(1531004720)) - .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(271043498).withUdp(1526885086)) - .withPinholeLimits(504117688) - .withPinholeTimeouts( - new PinholeTimeouts().withTcp(1760407196).withUdp(1757371466).withIcmp(1730607260))) - .withUserEquipmentAddressPoolPrefix(Arrays.asList("ufhfcbjysa", "ithxqhabifpi")) - .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("czbysc", "pqxu")); + AttachedDataNetworkPropertiesFormat model = new AttachedDataNetworkPropertiesFormat() + .withUserPlaneDataInterface(new InterfaceProperties().withName("lusarh").withIpv4Address("fcqhsmyurkd") + .withIpv4Subnet("lx").withIpv4Gateway("kuksjtxukcdm")) + .withDnsAddresses(Arrays.asList("rcryuanzwuxzdxta", "rlhm", "hfpmrqobmtukknr", "rtihfxtijbpz")) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.DISABLED) + .withPortRange(new PortRange().withMinPort(477742160).withMaxPort(1531004720)) + .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(271043498).withUdp(1526885086)) + .withPinholeLimits(504117688).withPinholeTimeouts( + new PinholeTimeouts().withTcp(1760407196).withUdp(1757371466).withIcmp(1730607260))) + .withUserEquipmentAddressPoolPrefix(Arrays.asList("ufhfcbjysa", "ithxqhabifpi")) + .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("czbysc", "pqxu")); model = BinaryData.fromObject(model).toObject(AttachedDataNetworkPropertiesFormat.class); Assertions.assertEquals("lusarh", model.userPlaneDataInterface().name()); Assertions.assertEquals("fcqhsmyurkd", model.userPlaneDataInterface().ipv4Address()); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkResourceIdTests.java index f029898bfb20c..42177507bf76b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworkResourceIdTests.java @@ -11,15 +11,15 @@ public final class AttachedDataNetworkResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AttachedDataNetworkResourceId model = - BinaryData.fromString("{\"id\":\"imwkslircizj\"}").toObject(AttachedDataNetworkResourceId.class); - Assertions.assertEquals("imwkslircizj", model.id()); + AttachedDataNetworkResourceId model + = BinaryData.fromString("{\"id\":\"bdeibqipqk\"}").toObject(AttachedDataNetworkResourceId.class); + Assertions.assertEquals("bdeibqipqk", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AttachedDataNetworkResourceId model = new AttachedDataNetworkResourceId().withId("imwkslircizj"); + AttachedDataNetworkResourceId model = new AttachedDataNetworkResourceId().withId("bdeibqipqk"); model = BinaryData.fromObject(model).toObject(AttachedDataNetworkResourceId.class); - Assertions.assertEquals("imwkslircizj", model.id()); + Assertions.assertEquals("bdeibqipqk", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateMockTests.java index a0a9f7bbeb99c..febb72e939728 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksCreateOrUpdateMockTests.java @@ -39,80 +39,57 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneDataInterface\":{\"name\":\"ywakoihk\",\"ipv4Address\":\"mjblmljhlny\",\"ipv4Subnet\":\"otqyry\",\"ipv4Gateway\":\"cbm\"},\"dnsAddresses\":[\"vxmvw\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":414360047,\"maxPort\":1016884622},\"portReuseHoldTime\":{\"tcp\":98251742,\"udp\":1744580177},\"pinholeLimits\":593939862,\"pinholeTimeouts\":{\"tcp\":559578866,\"udp\":1503247909,\"icmp\":489682268}},\"userEquipmentAddressPoolPrefix\":[\"nzoibgsxgnx\",\"yqo\",\"mpqoxw\",\"o\"],\"userEquipmentStaticAddressPoolPrefix\":[\"xiqxeiiqbimh\",\"mwwinhehfqpofv\",\"bcblemb\",\"kbwvqvxkdiv\"]},\"location\":\"heb\",\"tags\":{\"izvcjfe\":\"wbzuwfmdurage\",\"ggbqi\":\"isdju\",\"m\":\"kxkbsazgakgacyr\",\"pv\":\"dmspof\"},\"id\":\"hryl\",\"name\":\"iofrzgbzjedmstk\",\"type\":\"nlvxbcuii\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneDataInterface\":{\"name\":\"uwm\",\"ipv4Address\":\"pkcdqzhlct\",\"ipv4Subnet\":\"unqndyfpchrqb\",\"ipv4Gateway\":\"jrcg\"},\"dnsAddresses\":[\"ydcwboxjumv\",\"qoli\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":1564106365,\"maxPort\":1508462488},\"portReuseHoldTime\":{\"tcp\":1002121154,\"udp\":1218728383},\"pinholeLimits\":617532608,\"pinholeTimeouts\":{\"tcp\":465765494,\"udp\":1541278105,\"icmp\":1492295803}},\"userEquipmentAddressPoolPrefix\":[\"flrzpas\",\"cb\",\"uimzdlyjd\",\"qwmkyoquf\"],\"userEquipmentStaticAddressPoolPrefix\":[\"uzslzojhpctfnmdx\",\"tngfdgugeyzihgr\"]},\"location\":\"ui\",\"tags\":{\"yhsgz\":\"snmfpphojeevy\",\"qgleohibetnluank\":\"czbgomfgbeg\"},\"id\":\"rfxeeebtij\",\"name\":\"acvbmqz\",\"type\":\"qqxlajr\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - AttachedDataNetwork response = - manager - .attachedDataNetworks() - .define("zsvtuikzhajqgl") - .withRegion("glka") - .withExistingPacketCoreDataPlane("jng", "qdqx", "bjwgnyfus") - .withUserPlaneDataInterface( - new InterfaceProperties() - .withName("l") - .withIpv4Address("ryxynqnzrd") - .withIpv4Subnet("ovw") - .withIpv4Gateway("nptgoeiybba")) - .withDnsAddresses(Arrays.asList("fhvfsl", "vntjlrigjk")) - .withTags(mapOf("juj", "n", "cpopmxel", "ickpz", "xm", "wcltyjede", "kqscazuawxtzx", "f")) - .withNaptConfiguration( - new NaptConfiguration() - .withEnabled(NaptEnabled.ENABLED) - .withPortRange(new PortRange().withMinPort(1610623953).withMaxPort(714440003)) - .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(2075190734).withUdp(1304895406)) - .withPinholeLimits(2100581785) - .withPinholeTimeouts( - new PinholeTimeouts().withTcp(664139919).withUdp(300950097).withIcmp(969722323))) - .withUserEquipmentAddressPoolPrefix( - Arrays.asList("nmmaxrizkzob", "opxlhslnelxieixy", "llxecwc", "ojphslhc")) - .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("u", "i", "dwfmvigorqjb", "tzh")) - .create(); + AttachedDataNetwork response = manager.attachedDataNetworks().define("erbdk").withRegion("zsrzpgepq") + .withExistingPacketCoreDataPlane("reljeamur", "zmlovuanash", "xlpm") + .withUserPlaneDataInterface(new InterfaceProperties().withName("di").withIpv4Address("zsdbccxjmon") + .withIpv4Subnet("gnwncypuuwwlt").withIpv4Gateway("qjctzenkeif")) + .withDnsAddresses(Arrays.asList("hmkdasvfl", "hbxcu", "chxgs")) + .withTags(mapOf("lixqnrkcxkjibn", "bwwpgdakchzy")) + .withNaptConfiguration(new NaptConfiguration().withEnabled(NaptEnabled.DISABLED) + .withPortRange(new PortRange().withMinPort(477372699).withMaxPort(82251270)) + .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(1131661980).withUdp(2116230034)) + .withPinholeLimits(447073477) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(842423855).withUdp(65114282).withIcmp(1417596146))) + .withUserEquipmentAddressPoolPrefix(Arrays.asList("qpbtuodxes", "abbelawumuaslzk")) + .withUserEquipmentStaticAddressPoolPrefix(Arrays.asList("woycqucwyha", "nomdrkywuhpsv", "uurutlwexxwlalni")) + .create(); - Assertions.assertEquals("heb", response.location()); - Assertions.assertEquals("wbzuwfmdurage", response.tags().get("izvcjfe")); - Assertions.assertEquals("ywakoihk", response.userPlaneDataInterface().name()); - Assertions.assertEquals("mjblmljhlny", response.userPlaneDataInterface().ipv4Address()); - Assertions.assertEquals("otqyry", response.userPlaneDataInterface().ipv4Subnet()); - Assertions.assertEquals("cbm", response.userPlaneDataInterface().ipv4Gateway()); - Assertions.assertEquals("vxmvw", response.dnsAddresses().get(0)); + Assertions.assertEquals("ui", response.location()); + Assertions.assertEquals("snmfpphojeevy", response.tags().get("yhsgz")); + Assertions.assertEquals("uwm", response.userPlaneDataInterface().name()); + Assertions.assertEquals("pkcdqzhlct", response.userPlaneDataInterface().ipv4Address()); + Assertions.assertEquals("unqndyfpchrqb", response.userPlaneDataInterface().ipv4Subnet()); + Assertions.assertEquals("jrcg", response.userPlaneDataInterface().ipv4Gateway()); + Assertions.assertEquals("ydcwboxjumv", response.dnsAddresses().get(0)); Assertions.assertEquals(NaptEnabled.DISABLED, response.naptConfiguration().enabled()); - Assertions.assertEquals(414360047, response.naptConfiguration().portRange().minPort()); - Assertions.assertEquals(1016884622, response.naptConfiguration().portRange().maxPort()); - Assertions.assertEquals(98251742, response.naptConfiguration().portReuseHoldTime().tcp()); - Assertions.assertEquals(1744580177, response.naptConfiguration().portReuseHoldTime().udp()); - Assertions.assertEquals(593939862, response.naptConfiguration().pinholeLimits()); - Assertions.assertEquals(559578866, response.naptConfiguration().pinholeTimeouts().tcp()); - Assertions.assertEquals(1503247909, response.naptConfiguration().pinholeTimeouts().udp()); - Assertions.assertEquals(489682268, response.naptConfiguration().pinholeTimeouts().icmp()); - Assertions.assertEquals("nzoibgsxgnx", response.userEquipmentAddressPoolPrefix().get(0)); - Assertions.assertEquals("xiqxeiiqbimh", response.userEquipmentStaticAddressPoolPrefix().get(0)); + Assertions.assertEquals(1564106365, response.naptConfiguration().portRange().minPort()); + Assertions.assertEquals(1508462488, response.naptConfiguration().portRange().maxPort()); + Assertions.assertEquals(1002121154, response.naptConfiguration().portReuseHoldTime().tcp()); + Assertions.assertEquals(1218728383, response.naptConfiguration().portReuseHoldTime().udp()); + Assertions.assertEquals(617532608, response.naptConfiguration().pinholeLimits()); + Assertions.assertEquals(465765494, response.naptConfiguration().pinholeTimeouts().tcp()); + Assertions.assertEquals(1541278105, response.naptConfiguration().pinholeTimeouts().udp()); + Assertions.assertEquals(1492295803, response.naptConfiguration().pinholeTimeouts().icmp()); + Assertions.assertEquals("flrzpas", response.userEquipmentAddressPoolPrefix().get(0)); + Assertions.assertEquals("uzslzojhpctfnmdx", response.userEquipmentStaticAddressPoolPrefix().get(0)); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteMockTests.java index 1982c740a9dd6..ab6725278bfd5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksDeleteMockTests.java @@ -32,32 +32,21 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .attachedDataNetworks() - .delete("yhsgz", "czbgomfgbeg", "qgleohibetnluank", "rfxeeebtij", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.attachedDataNetworks().delete("dclxgc", "knfnwmbtmvpdv", "dhttzaefedxihchr", "hk", + com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetWithResponseMockTests.java index fc69b6d8129e4..e02f6b658f242 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksGetWithResponseMockTests.java @@ -31,58 +31,44 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneDataInterface\":{\"name\":\"lt\",\"ipv4Address\":\"eyl\",\"ipv4Subnet\":\"mfgvxirpghriypo\",\"ipv4Gateway\":\"yhlqhykprlpyznu\"},\"dnsAddresses\":[\"qdsmexiit\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":1550177536,\"maxPort\":1265824648},\"portReuseHoldTime\":{\"tcp\":1535598222,\"udp\":2086565224},\"pinholeLimits\":1192000578,\"pinholeTimeouts\":{\"tcp\":1246730356,\"udp\":1143037859,\"icmp\":1300683178}},\"userEquipmentAddressPoolPrefix\":[\"nmgixh\",\"mavmq\",\"oudorhcgyyp\"],\"userEquipmentStaticAddressPoolPrefix\":[\"wy\",\"undmbx\",\"ugcmjkavlgorb\"]},\"location\":\"tp\",\"tags\":{\"p\":\"zfjltfvnzcyjto\",\"bdb\":\"opv\"},\"id\":\"qgqqihedsvqwthmk\",\"name\":\"ibcysihsgqc\",\"type\":\"dhohsdtmcdzsuf\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneDataInterface\":{\"name\":\"ppipifhpfeoa\",\"ipv4Address\":\"gcxtxj\",\"ipv4Subnet\":\"heafidlt\",\"ipv4Gateway\":\"sr\"},\"dnsAddresses\":[\"mks\",\"jhoiftxfkfweg\"],\"naptConfiguration\":{\"enabled\":\"Enabled\",\"portRange\":{\"minPort\":756597053,\"maxPort\":721482602},\"portReuseHoldTime\":{\"tcp\":2086555538,\"udp\":1656639409},\"pinholeLimits\":1467633348,\"pinholeTimeouts\":{\"tcp\":343258165,\"udp\":1742754649,\"icmp\":1560006884}},\"userEquipmentAddressPoolPrefix\":[\"riz\",\"tpwb\",\"a\"],\"userEquipmentStaticAddressPoolPrefix\":[\"ibph\",\"qzmiza\"]},\"location\":\"kan\",\"tags\":{\"joylh\":\"dnjzh\"},\"id\":\"lmuoyxprimrsopte\",\"name\":\"cjmeislstvasy\",\"type\":\"wxdzaumweoohgu\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - AttachedDataNetwork response = - manager - .attachedDataNetworks() - .getWithResponse("acvbmqz", "qqxlajr", "wxacevehj", "uyxoaf", com.azure.core.util.Context.NONE) - .getValue(); + AttachedDataNetwork response = manager.attachedDataNetworks().getWithResponse("crjdqnsdfzpbgtg", "ylkdghrje", + "utlwxezwzhok", "bwnhhtql", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("tp", response.location()); - Assertions.assertEquals("zfjltfvnzcyjto", response.tags().get("p")); - Assertions.assertEquals("lt", response.userPlaneDataInterface().name()); - Assertions.assertEquals("eyl", response.userPlaneDataInterface().ipv4Address()); - Assertions.assertEquals("mfgvxirpghriypo", response.userPlaneDataInterface().ipv4Subnet()); - Assertions.assertEquals("yhlqhykprlpyznu", response.userPlaneDataInterface().ipv4Gateway()); - Assertions.assertEquals("qdsmexiit", response.dnsAddresses().get(0)); - Assertions.assertEquals(NaptEnabled.DISABLED, response.naptConfiguration().enabled()); - Assertions.assertEquals(1550177536, response.naptConfiguration().portRange().minPort()); - Assertions.assertEquals(1265824648, response.naptConfiguration().portRange().maxPort()); - Assertions.assertEquals(1535598222, response.naptConfiguration().portReuseHoldTime().tcp()); - Assertions.assertEquals(2086565224, response.naptConfiguration().portReuseHoldTime().udp()); - Assertions.assertEquals(1192000578, response.naptConfiguration().pinholeLimits()); - Assertions.assertEquals(1246730356, response.naptConfiguration().pinholeTimeouts().tcp()); - Assertions.assertEquals(1143037859, response.naptConfiguration().pinholeTimeouts().udp()); - Assertions.assertEquals(1300683178, response.naptConfiguration().pinholeTimeouts().icmp()); - Assertions.assertEquals("nmgixh", response.userEquipmentAddressPoolPrefix().get(0)); - Assertions.assertEquals("wy", response.userEquipmentStaticAddressPoolPrefix().get(0)); + Assertions.assertEquals("kan", response.location()); + Assertions.assertEquals("dnjzh", response.tags().get("joylh")); + Assertions.assertEquals("ppipifhpfeoa", response.userPlaneDataInterface().name()); + Assertions.assertEquals("gcxtxj", response.userPlaneDataInterface().ipv4Address()); + Assertions.assertEquals("heafidlt", response.userPlaneDataInterface().ipv4Subnet()); + Assertions.assertEquals("sr", response.userPlaneDataInterface().ipv4Gateway()); + Assertions.assertEquals("mks", response.dnsAddresses().get(0)); + Assertions.assertEquals(NaptEnabled.ENABLED, response.naptConfiguration().enabled()); + Assertions.assertEquals(756597053, response.naptConfiguration().portRange().minPort()); + Assertions.assertEquals(721482602, response.naptConfiguration().portRange().maxPort()); + Assertions.assertEquals(2086555538, response.naptConfiguration().portReuseHoldTime().tcp()); + Assertions.assertEquals(1656639409, response.naptConfiguration().portReuseHoldTime().udp()); + Assertions.assertEquals(1467633348, response.naptConfiguration().pinholeLimits()); + Assertions.assertEquals(343258165, response.naptConfiguration().pinholeTimeouts().tcp()); + Assertions.assertEquals(1742754649, response.naptConfiguration().pinholeTimeouts().udp()); + Assertions.assertEquals(1560006884, response.naptConfiguration().pinholeTimeouts().icmp()); + Assertions.assertEquals("riz", response.userEquipmentAddressPoolPrefix().get(0)); + Assertions.assertEquals("ibph", response.userEquipmentStaticAddressPoolPrefix().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneMockTests.java index a3da3e0719ff9..dcc9bf9c67251 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AttachedDataNetworksListByPacketCoreDataPlaneMockTests.java @@ -32,58 +32,44 @@ public void testListByPacketCoreDataPlane() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"userPlaneDataInterface\":{\"name\":\"clj\",\"ipv4Address\":\"gvkzqkjjeokbze\",\"ipv4Subnet\":\"zrxcczurt\",\"ipv4Gateway\":\"ipqxbkwvzgnzv\"},\"dnsAddresses\":[\"bzdixzmq\",\"noda\",\"opqhewjptmc\",\"sbostzel\"],\"naptConfiguration\":{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":492133944,\"maxPort\":1611840724},\"portReuseHoldTime\":{\"tcp\":204672914,\"udp\":1115327100},\"pinholeLimits\":1010329526,\"pinholeTimeouts\":{\"tcp\":1281899778,\"udp\":1426924958,\"icmp\":1171808306}},\"userEquipmentAddressPoolPrefix\":[\"vcwwyyurmochppr\",\"rsnm\"],\"userEquipmentStaticAddressPoolPrefix\":[\"yzejnhlbk\",\"bzpcpiljhahzvec\",\"ndbnwieh\",\"lewjwiuubwef\"]},\"location\":\"fapaqtfer\",\"tags\":{\"kmfx\":\"ex\",\"pud\":\"pjwogqqno\",\"yawbzasqbu\":\"dabtqwp\",\"oguyaip\":\"ljgkyex\"},\"id\":\"dsdaultxijjumf\",\"name\":\"waz\",\"type\":\"nqnm\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"userPlaneDataInterface\":{\"name\":\"nmwmqtibx\",\"ipv4Address\":\"jddtvqct\",\"ipv4Subnet\":\"dija\",\"ipv4Gateway\":\"kmr\"},\"dnsAddresses\":[\"eekpndz\",\"apm\",\"dqmeqwigpibudq\"],\"naptConfiguration\":{\"enabled\":\"Enabled\",\"portRange\":{\"minPort\":1568036168,\"maxPort\":650775767},\"portReuseHoldTime\":{\"tcp\":606462246,\"udp\":619135714},\"pinholeLimits\":310350754,\"pinholeTimeouts\":{\"tcp\":639795933,\"udp\":1808729813,\"icmp\":547449174}},\"userEquipmentAddressPoolPrefix\":[\"eioqaqhvse\",\"fuqyrxpdlcgqlsi\"],\"userEquipmentStaticAddressPoolPrefix\":[\"qfrddgam\",\"uhiosrsju\",\"vfcdisyirn\"]},\"location\":\"hcz\",\"tags\":{\"vk\":\"xzbujrtrhqvwr\",\"zonzlrpiqywnc\":\"gnl\"},\"id\":\"jtszcof\",\"name\":\"zehtdhgb\",\"type\":\"k\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .attachedDataNetworks() - .listByPacketCoreDataPlane( - "ohdxbzlmcmu", "pcvhdbevwqqxeys", "onqzinkfkbgbzbow", com.azure.core.util.Context.NONE); + PagedIterable response = manager.attachedDataNetworks().listByPacketCoreDataPlane( + "fuzboyjathwtzolb", "emwmdxmebwjs", "jpahlxvea", com.azure.core.util.Context.NONE); - Assertions.assertEquals("fapaqtfer", response.iterator().next().location()); - Assertions.assertEquals("ex", response.iterator().next().tags().get("kmfx")); - Assertions.assertEquals("clj", response.iterator().next().userPlaneDataInterface().name()); - Assertions.assertEquals("gvkzqkjjeokbze", response.iterator().next().userPlaneDataInterface().ipv4Address()); - Assertions.assertEquals("zrxcczurt", response.iterator().next().userPlaneDataInterface().ipv4Subnet()); - Assertions.assertEquals("ipqxbkwvzgnzv", response.iterator().next().userPlaneDataInterface().ipv4Gateway()); - Assertions.assertEquals("bzdixzmq", response.iterator().next().dnsAddresses().get(0)); - Assertions.assertEquals(NaptEnabled.DISABLED, response.iterator().next().naptConfiguration().enabled()); - Assertions.assertEquals(492133944, response.iterator().next().naptConfiguration().portRange().minPort()); - Assertions.assertEquals(1611840724, response.iterator().next().naptConfiguration().portRange().maxPort()); - Assertions.assertEquals(204672914, response.iterator().next().naptConfiguration().portReuseHoldTime().tcp()); - Assertions.assertEquals(1115327100, response.iterator().next().naptConfiguration().portReuseHoldTime().udp()); - Assertions.assertEquals(1010329526, response.iterator().next().naptConfiguration().pinholeLimits()); - Assertions.assertEquals(1281899778, response.iterator().next().naptConfiguration().pinholeTimeouts().tcp()); - Assertions.assertEquals(1426924958, response.iterator().next().naptConfiguration().pinholeTimeouts().udp()); - Assertions.assertEquals(1171808306, response.iterator().next().naptConfiguration().pinholeTimeouts().icmp()); - Assertions.assertEquals("vcwwyyurmochppr", response.iterator().next().userEquipmentAddressPoolPrefix().get(0)); - Assertions.assertEquals("yzejnhlbk", response.iterator().next().userEquipmentStaticAddressPoolPrefix().get(0)); + Assertions.assertEquals("hcz", response.iterator().next().location()); + Assertions.assertEquals("xzbujrtrhqvwr", response.iterator().next().tags().get("vk")); + Assertions.assertEquals("nmwmqtibx", response.iterator().next().userPlaneDataInterface().name()); + Assertions.assertEquals("jddtvqct", response.iterator().next().userPlaneDataInterface().ipv4Address()); + Assertions.assertEquals("dija", response.iterator().next().userPlaneDataInterface().ipv4Subnet()); + Assertions.assertEquals("kmr", response.iterator().next().userPlaneDataInterface().ipv4Gateway()); + Assertions.assertEquals("eekpndz", response.iterator().next().dnsAddresses().get(0)); + Assertions.assertEquals(NaptEnabled.ENABLED, response.iterator().next().naptConfiguration().enabled()); + Assertions.assertEquals(1568036168, response.iterator().next().naptConfiguration().portRange().minPort()); + Assertions.assertEquals(650775767, response.iterator().next().naptConfiguration().portRange().maxPort()); + Assertions.assertEquals(606462246, response.iterator().next().naptConfiguration().portReuseHoldTime().tcp()); + Assertions.assertEquals(619135714, response.iterator().next().naptConfiguration().portReuseHoldTime().udp()); + Assertions.assertEquals(310350754, response.iterator().next().naptConfiguration().pinholeLimits()); + Assertions.assertEquals(639795933, response.iterator().next().naptConfiguration().pinholeTimeouts().tcp()); + Assertions.assertEquals(1808729813, response.iterator().next().naptConfiguration().pinholeTimeouts().udp()); + Assertions.assertEquals(547449174, response.iterator().next().naptConfiguration().pinholeTimeouts().icmp()); + Assertions.assertEquals("eioqaqhvse", response.iterator().next().userEquipmentAddressPoolPrefix().get(0)); + Assertions.assertEquals("qfrddgam", response.iterator().next().userEquipmentStaticAddressPoolPrefix().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackEdgeDeviceResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackEdgeDeviceResourceIdTests.java index 8c54ad3393f55..1ffcc3405f420 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackEdgeDeviceResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackEdgeDeviceResourceIdTests.java @@ -11,15 +11,15 @@ public final class AzureStackEdgeDeviceResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzureStackEdgeDeviceResourceId model = - BinaryData.fromString("{\"id\":\"hsmtxpsiebtfhvp\"}").toObject(AzureStackEdgeDeviceResourceId.class); - Assertions.assertEquals("hsmtxpsiebtfhvp", model.id()); + AzureStackEdgeDeviceResourceId model + = BinaryData.fromString("{\"id\":\"dhmdua\"}").toObject(AzureStackEdgeDeviceResourceId.class); + Assertions.assertEquals("dhmdua", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureStackEdgeDeviceResourceId model = new AzureStackEdgeDeviceResourceId().withId("hsmtxpsiebtfhvp"); + AzureStackEdgeDeviceResourceId model = new AzureStackEdgeDeviceResourceId().withId("dhmdua"); model = BinaryData.fromObject(model).toObject(AzureStackEdgeDeviceResourceId.class); - Assertions.assertEquals("hsmtxpsiebtfhvp", model.id()); + Assertions.assertEquals("dhmdua", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackHciClusterResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackHciClusterResourceIdTests.java index c147a26970139..96a8aeb840253 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackHciClusterResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/AzureStackHciClusterResourceIdTests.java @@ -11,15 +11,15 @@ public final class AzureStackHciClusterResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AzureStackHciClusterResourceId model = - BinaryData.fromString("{\"id\":\"sapskr\"}").toObject(AzureStackHciClusterResourceId.class); - Assertions.assertEquals("sapskr", model.id()); + AzureStackHciClusterResourceId model + = BinaryData.fromString("{\"id\":\"aex\"}").toObject(AzureStackHciClusterResourceId.class); + Assertions.assertEquals("aex", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AzureStackHciClusterResourceId model = new AzureStackHciClusterResourceId().withId("sapskr"); + AzureStackHciClusterResourceId model = new AzureStackHciClusterResourceId().withId("aex"); model = BinaryData.fromObject(model).toObject(AzureStackHciClusterResourceId.class); - Assertions.assertEquals("sapskr", model.id()); + Assertions.assertEquals("aex", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CertificateProvisioningTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CertificateProvisioningTests.java index 365679c5697e8..1e0a60928d186 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CertificateProvisioningTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CertificateProvisioningTests.java @@ -10,10 +10,8 @@ public final class CertificateProvisioningTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CertificateProvisioning model = - BinaryData - .fromString("{\"state\":\"NotProvisioned\",\"reason\":\"yfjfcnjbkcn\"}") - .toObject(CertificateProvisioning.class); + CertificateProvisioning model = BinaryData.fromString("{\"state\":\"Provisioned\",\"reason\":\"xdje\"}") + .toObject(CertificateProvisioning.class); } @org.junit.jupiter.api.Test diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ConnectedClusterResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ConnectedClusterResourceIdTests.java index 6e99f7ab0dcbf..c4bdd8c69a2e9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ConnectedClusterResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ConnectedClusterResourceIdTests.java @@ -11,15 +11,15 @@ public final class ConnectedClusterResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ConnectedClusterResourceId model = - BinaryData.fromString("{\"id\":\"qmhjjdhtld\"}").toObject(ConnectedClusterResourceId.class); - Assertions.assertEquals("qmhjjdhtld", model.id()); + ConnectedClusterResourceId model + = BinaryData.fromString("{\"id\":\"pvfadmwsrcr\"}").toObject(ConnectedClusterResourceId.class); + Assertions.assertEquals("pvfadmwsrcr", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ConnectedClusterResourceId model = new ConnectedClusterResourceId().withId("qmhjjdhtld"); + ConnectedClusterResourceId model = new ConnectedClusterResourceId().withId("pvfadmwsrcr"); model = BinaryData.fromObject(model).toObject(ConnectedClusterResourceId.class); - Assertions.assertEquals("qmhjjdhtld", model.id()); + Assertions.assertEquals("pvfadmwsrcr", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CustomLocationResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CustomLocationResourceIdTests.java index 72496d783e134..a7987759290f9 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CustomLocationResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/CustomLocationResourceIdTests.java @@ -11,15 +11,15 @@ public final class CustomLocationResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CustomLocationResourceId model = - BinaryData.fromString("{\"id\":\"kyzxuutk\"}").toObject(CustomLocationResourceId.class); - Assertions.assertEquals("kyzxuutk", model.id()); + CustomLocationResourceId model + = BinaryData.fromString("{\"id\":\"vxpvgomz\"}").toObject(CustomLocationResourceId.class); + Assertions.assertEquals("vxpvgomz", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CustomLocationResourceId model = new CustomLocationResourceId().withId("kyzxuutk"); + CustomLocationResourceId model = new CustomLocationResourceId().withId("vxpvgomz"); model = BinaryData.fromObject(model).toObject(CustomLocationResourceId.class); - Assertions.assertEquals("kyzxuutk", model.id()); + Assertions.assertEquals("vxpvgomz", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkConfigurationTests.java index fb572421141fc..af05274b4ab4a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkConfigurationTests.java @@ -18,53 +18,47 @@ public final class DataNetworkConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DataNetworkConfiguration model = - BinaryData - .fromString( - "{\"dataNetwork\":{\"id\":\"hihihlhzdsqtzbsr\"},\"sessionAmbr\":{\"uplink\":\"nowc\",\"downlink\":\"hfgmvecactxm\"},\"5qi\":2043767833,\"allocationAndRetentionPriorityLevel\":966273213,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"vgqouwifzmpj\"},{\"id\":\"yivqikfxcvhrfsp\"}],\"maximumNumberOfBufferedPackets\":754601836}") - .toObject(DataNetworkConfiguration.class); - Assertions.assertEquals("hihihlhzdsqtzbsr", model.dataNetwork().id()); - Assertions.assertEquals("nowc", model.sessionAmbr().uplink()); - Assertions.assertEquals("hfgmvecactxm", model.sessionAmbr().downlink()); - Assertions.assertEquals(2043767833, model.fiveQi()); - Assertions.assertEquals(966273213, model.allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.preemptionCapability()); - Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, model.preemptionVulnerability()); - Assertions.assertEquals(PduSessionType.IPV6, model.defaultSessionType()); - Assertions.assertEquals(PduSessionType.IPV6, model.additionalAllowedSessionTypes().get(0)); - Assertions.assertEquals("vgqouwifzmpj", model.allowedServices().get(0).id()); - Assertions.assertEquals(754601836, model.maximumNumberOfBufferedPackets()); + DataNetworkConfiguration model = BinaryData.fromString( + "{\"dataNetwork\":{\"id\":\"xcbihw\"},\"sessionAmbr\":{\"uplink\":\"knfd\",\"downlink\":\"twjchrdg\"},\"5qi\":916840280,\"allocationAndRetentionPriorityLevel\":144072003,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv4\"],\"allowedServices\":[{\"id\":\"uu\"},{\"id\":\"fdlwg\"},{\"id\":\"ytsbwtovv\"},{\"id\":\"gseinq\"}],\"maximumNumberOfBufferedPackets\":1894270396}") + .toObject(DataNetworkConfiguration.class); + Assertions.assertEquals("xcbihw", model.dataNetwork().id()); + Assertions.assertEquals("knfd", model.sessionAmbr().uplink()); + Assertions.assertEquals("twjchrdg", model.sessionAmbr().downlink()); + Assertions.assertEquals(916840280, model.fiveQi()); + Assertions.assertEquals(144072003, model.allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.preemptionVulnerability()); + Assertions.assertEquals(PduSessionType.IPV4, model.defaultSessionType()); + Assertions.assertEquals(PduSessionType.IPV4, model.additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("uu", model.allowedServices().get(0).id()); + Assertions.assertEquals(1894270396, model.maximumNumberOfBufferedPackets()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DataNetworkConfiguration model = - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("hihihlhzdsqtzbsr")) - .withSessionAmbr(new Ambr().withUplink("nowc").withDownlink("hfgmvecactxm")) - .withFiveQi(2043767833) - .withAllocationAndRetentionPriorityLevel(966273213) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV6, PduSessionType.IPV4)) + DataNetworkConfiguration model + = new DataNetworkConfiguration().withDataNetwork(new DataNetworkResourceId().withId("xcbihw")) + .withSessionAmbr(new Ambr().withUplink("knfd").withDownlink("twjchrdg")).withFiveQi(916840280) + .withAllocationAndRetentionPriorityLevel(144072003) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4) + .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV4)) .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("vgqouwifzmpj"), - new ServiceResourceId().withId("yivqikfxcvhrfsp"))) - .withMaximumNumberOfBufferedPackets(754601836); + Arrays.asList(new ServiceResourceId().withId("uu"), new ServiceResourceId().withId("fdlwg"), + new ServiceResourceId().withId("ytsbwtovv"), new ServiceResourceId().withId("gseinq"))) + .withMaximumNumberOfBufferedPackets(1894270396); model = BinaryData.fromObject(model).toObject(DataNetworkConfiguration.class); - Assertions.assertEquals("hihihlhzdsqtzbsr", model.dataNetwork().id()); - Assertions.assertEquals("nowc", model.sessionAmbr().uplink()); - Assertions.assertEquals("hfgmvecactxm", model.sessionAmbr().downlink()); - Assertions.assertEquals(2043767833, model.fiveQi()); - Assertions.assertEquals(966273213, model.allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.preemptionCapability()); - Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, model.preemptionVulnerability()); - Assertions.assertEquals(PduSessionType.IPV6, model.defaultSessionType()); - Assertions.assertEquals(PduSessionType.IPV6, model.additionalAllowedSessionTypes().get(0)); - Assertions.assertEquals("vgqouwifzmpj", model.allowedServices().get(0).id()); - Assertions.assertEquals(754601836, model.maximumNumberOfBufferedPackets()); + Assertions.assertEquals("xcbihw", model.dataNetwork().id()); + Assertions.assertEquals("knfd", model.sessionAmbr().uplink()); + Assertions.assertEquals("twjchrdg", model.sessionAmbr().downlink()); + Assertions.assertEquals(916840280, model.fiveQi()); + Assertions.assertEquals(144072003, model.allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.preemptionVulnerability()); + Assertions.assertEquals(PduSessionType.IPV4, model.defaultSessionType()); + Assertions.assertEquals(PduSessionType.IPV4, model.additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("uu", model.allowedServices().get(0).id()); + Assertions.assertEquals(1894270396, model.maximumNumberOfBufferedPackets()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkInnerTests.java index 1c6b238aaa676..371a8538e410b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkInnerTests.java @@ -13,11 +13,9 @@ public final class DataNetworkInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DataNetworkInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Canceled\",\"description\":\"wu\"},\"location\":\"gazxuf\",\"tags\":{\"fidfvzw\":\"ckyfih\",\"nteiwaopv\":\"zuhtymwisdkfthwx\"},\"id\":\"mijcmmxdcufufs\",\"name\":\"pymzidnsezcxtbzs\",\"type\":\"fycc\"}") - .toObject(DataNetworkInner.class); + DataNetworkInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Canceled\",\"description\":\"wu\"},\"location\":\"gazxuf\",\"tags\":{\"fidfvzw\":\"ckyfih\",\"nteiwaopv\":\"zuhtymwisdkfthwx\"},\"id\":\"mijcmmxdcufufs\",\"name\":\"pymzidnsezcxtbzs\",\"type\":\"fycc\"}") + .toObject(DataNetworkInner.class); Assertions.assertEquals("gazxuf", model.location()); Assertions.assertEquals("ckyfih", model.tags().get("fidfvzw")); Assertions.assertEquals("wu", model.description()); @@ -25,11 +23,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DataNetworkInner model = - new DataNetworkInner() - .withLocation("gazxuf") - .withTags(mapOf("fidfvzw", "ckyfih", "nteiwaopv", "zuhtymwisdkfthwx")) - .withDescription("wu"); + DataNetworkInner model = new DataNetworkInner().withLocation("gazxuf") + .withTags(mapOf("fidfvzw", "ckyfih", "nteiwaopv", "zuhtymwisdkfthwx")).withDescription("wu"); model = BinaryData.fromObject(model).toObject(DataNetworkInner.class); Assertions.assertEquals("gazxuf", model.location()); Assertions.assertEquals("ckyfih", model.tags().get("fidfvzw")); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkListResultTests.java index 8f2f0df54c466..580366f6906e8 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkListResultTests.java @@ -15,11 +15,9 @@ public final class DataNetworkListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DataNetworkListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"description\":\"fqpte\"},\"location\":\"zzvypyqrimzinp\",\"tags\":{\"crmnohjtckwhds\":\"jdkirsoodqx\"},\"id\":\"ifiyipjxsqwpgrj\",\"name\":\"znorcj\",\"type\":\"vsnb\"},{\"properties\":{\"provisioningState\":\"Canceled\",\"description\":\"nmoc\"},\"location\":\"ysh\",\"tags\":{\"javbqidtqajz\":\"afbljjgpbtoqcjmk\"},\"id\":\"ulpkudjkrl\",\"name\":\"hbzhfepg\",\"type\":\"gqexzlocxs\"}],\"nextLink\":\"aierhhb\"}") - .toObject(DataNetworkListResult.class); + DataNetworkListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"description\":\"fqpte\"},\"location\":\"zzvypyqrimzinp\",\"tags\":{\"crmnohjtckwhds\":\"jdkirsoodqx\"},\"id\":\"ifiyipjxsqwpgrj\",\"name\":\"znorcj\",\"type\":\"vsnb\"},{\"properties\":{\"provisioningState\":\"Canceled\",\"description\":\"nmoc\"},\"location\":\"ysh\",\"tags\":{\"javbqidtqajz\":\"afbljjgpbtoqcjmk\"},\"id\":\"ulpkudjkrl\",\"name\":\"hbzhfepg\",\"type\":\"gqexzlocxs\"}],\"nextLink\":\"aierhhb\"}") + .toObject(DataNetworkListResult.class); Assertions.assertEquals("zzvypyqrimzinp", model.value().get(0).location()); Assertions.assertEquals("jdkirsoodqx", model.value().get(0).tags().get("crmnohjtckwhds")); Assertions.assertEquals("fqpte", model.value().get(0).description()); @@ -27,19 +25,11 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DataNetworkListResult model = - new DataNetworkListResult() - .withValue( - Arrays - .asList( - new DataNetworkInner() - .withLocation("zzvypyqrimzinp") - .withTags(mapOf("crmnohjtckwhds", "jdkirsoodqx")) - .withDescription("fqpte"), - new DataNetworkInner() - .withLocation("ysh") - .withTags(mapOf("javbqidtqajz", "afbljjgpbtoqcjmk")) - .withDescription("nmoc"))); + DataNetworkListResult model = new DataNetworkListResult().withValue(Arrays.asList( + new DataNetworkInner().withLocation("zzvypyqrimzinp").withTags(mapOf("crmnohjtckwhds", "jdkirsoodqx")) + .withDescription("fqpte"), + new DataNetworkInner().withLocation("ysh").withTags(mapOf("javbqidtqajz", "afbljjgpbtoqcjmk")) + .withDescription("nmoc"))); model = BinaryData.fromObject(model).toObject(DataNetworkListResult.class); Assertions.assertEquals("zzvypyqrimzinp", model.value().get(0).location()); Assertions.assertEquals("jdkirsoodqx", model.value().get(0).tags().get("crmnohjtckwhds")); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkPropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkPropertiesFormatTests.java index 339b8caea99bc..0d0556220315f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkPropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkPropertiesFormatTests.java @@ -11,9 +11,8 @@ public final class DataNetworkPropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DataNetworkPropertiesFormat model = - BinaryData - .fromString("{\"provisioningState\":\"Failed\",\"description\":\"mdwzjeiachboo\"}") + DataNetworkPropertiesFormat model + = BinaryData.fromString("{\"provisioningState\":\"Failed\",\"description\":\"mdwzjeiachboo\"}") .toObject(DataNetworkPropertiesFormat.class); Assertions.assertEquals("mdwzjeiachboo", model.description()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkResourceIdTests.java index ca73ad749d266..ab9c9fc8d9396 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworkResourceIdTests.java @@ -11,15 +11,15 @@ public final class DataNetworkResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DataNetworkResourceId model = - BinaryData.fromString("{\"id\":\"yqdhcuplcplcw\"}").toObject(DataNetworkResourceId.class); - Assertions.assertEquals("yqdhcuplcplcw", model.id()); + DataNetworkResourceId model + = BinaryData.fromString("{\"id\":\"iithtywu\"}").toObject(DataNetworkResourceId.class); + Assertions.assertEquals("iithtywu", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DataNetworkResourceId model = new DataNetworkResourceId().withId("yqdhcuplcplcw"); + DataNetworkResourceId model = new DataNetworkResourceId().withId("iithtywu"); model = BinaryData.fromObject(model).toObject(DataNetworkResourceId.class); - Assertions.assertEquals("yqdhcuplcplcw", model.id()); + Assertions.assertEquals("iithtywu", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateMockTests.java index 110f1d87015e9..6d1f5280b2575 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksCreateOrUpdateMockTests.java @@ -32,50 +32,33 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"description\":\"nthjtwkjaosrxuzv\"},\"location\":\"mktcqiosmgbza\",\"tags\":{\"laprlt\":\"qdlyrtl\",\"nnbsoqeqa\":\"katbhjm\",\"febwlnbmhyreeudz\":\"arvlagunbt\",\"pdqmjxlyyzglgouw\":\"av\"},\"id\":\"lmjjyuo\",\"name\":\"qtobaxkjeyt\",\"type\":\"nlb\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"description\":\"piljhahzvech\"},\"location\":\"bnwieholew\",\"tags\":{\"fqsfa\":\"uubw\",\"wexjkmfxapjwogq\":\"aqtferr\",\"awbzasqb\":\"nobpudcdabtqwpw\"},\"id\":\"clj\",\"name\":\"kyexaoguyaipi\",\"type\":\"sdaultxij\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - DataNetwork response = - manager - .dataNetworks() - .define("ycjsx") - .withRegion("mwmxqhndvnoamld") - .withExistingMobileNetwork("kzruswh", "hczznvf") - .withTags( - mapOf( - "flzokxco", "aohdjh", "tsxoatftgz", "pelnjetag", "vefloccsrmozihmi", "npbs", "wtxxpkyjcx", "g")) - .withDescription("z") - .create(); + DataNetwork response + = manager.dataNetworks().define("zinkfkbgbzbowxeq").withRegion("kbzefezrxcczurt") + .withExistingMobileNetwork("v", "qqxeyskon").withTags(mapOf("zdix", "pqxbkwvzgnzvdf", "opqhewjptmc", + "mqpnoda", "dlat", "sbostzel", "hrbbpneqvcwwyy", "tmzlbiojlv")) + .withDescription("ygvkzqkjj").create(); - Assertions.assertEquals("mktcqiosmgbza", response.location()); - Assertions.assertEquals("qdlyrtl", response.tags().get("laprlt")); - Assertions.assertEquals("nthjtwkjaosrxuzv", response.description()); + Assertions.assertEquals("bnwieholew", response.location()); + Assertions.assertEquals("uubw", response.tags().get("fqsfa")); + Assertions.assertEquals("piljhahzvech", response.description()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteMockTests.java index af207a2fb7c1f..239a07d3fc520 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksDeleteMockTests.java @@ -32,32 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .dataNetworks() - .delete("nktwfansnvpdibmi", "ostbzbkiwb", "qnyophzfyls", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.dataNetworks().delete("wxacevehj", "uyxoaf", "aoqltfaey", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetWithResponseMockTests.java index c8c20d49e2746..e9d81f0ab1e74 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksGetWithResponseMockTests.java @@ -30,43 +30,29 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Failed\",\"description\":\"lafzvaylptr\"},\"location\":\"qwztcmwqkc\",\"tags\":{\"xfdeqvhpsyl\":\"waxfewzjkj\",\"bffmbmxz\":\"ksh\",\"jx\":\"rgywwp\"},\"id\":\"nptfujgi\",\"name\":\"gaao\",\"type\":\"pttaqutd\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Deleted\",\"description\":\"qdsmexiit\"},\"location\":\"uxtyasiibmi\",\"tags\":{\"mgixhcmavmqfou\":\"nustgnljh\",\"cgyypro\":\"or\"},\"id\":\"wy\",\"name\":\"undmbx\",\"type\":\"ugcmjkavlgorb\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - DataNetwork response = - manager - .dataNetworks() - .getWithResponse("crpfbcunez", "cez", "lfwyfwlwxjwetn", com.azure.core.util.Context.NONE) - .getValue(); + DataNetwork response = manager.dataNetworks() + .getWithResponse("inmfgvxirp", "hriypoqeyhlqhy", "prlpy", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("qwztcmwqkc", response.location()); - Assertions.assertEquals("waxfewzjkj", response.tags().get("xfdeqvhpsyl")); - Assertions.assertEquals("lafzvaylptr", response.description()); + Assertions.assertEquals("uxtyasiibmi", response.location()); + Assertions.assertEquals("nustgnljh", response.tags().get("mgixhcmavmqfou")); + Assertions.assertEquals("qdsmexiit", response.description()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkMockTests.java index 8265fce106b3b..041609c135482 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DataNetworksListByMobileNetworkMockTests.java @@ -31,42 +31,29 @@ public void testListByMobileNetwork() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"description\":\"jqepqwhi\"},\"location\":\"onsts\",\"tags\":{\"vuwcobiegstmnin\":\"xgvelfclduccbird\",\"ejjtbxqmul\":\"jizcilnghgs\",\"rsbycucrwn\":\"xlxqzvn\"},\"id\":\"mikzeb\",\"name\":\"qbsms\",\"type\":\"ziqgfuh\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\",\"description\":\"pbdbzqgqqi\"},\"location\":\"dsvqwt\",\"tags\":{\"ihsgq\":\"yibcy\"},\"id\":\"wdhohsdtmcdzsu\",\"name\":\"cohdx\",\"type\":\"zlmcmuapcvhdb\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager - .dataNetworks() - .listByMobileNetwork("wemxswvruunzz", "gehkfkimrtixokff", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.dataNetworks().listByMobileNetwork("ftpmdtzfjltfv", "zcyjtot", com.azure.core.util.Context.NONE); - Assertions.assertEquals("onsts", response.iterator().next().location()); - Assertions.assertEquals("xgvelfclduccbird", response.iterator().next().tags().get("vuwcobiegstmnin")); - Assertions.assertEquals("jqepqwhi", response.iterator().next().description()); + Assertions.assertEquals("dsvqwt", response.iterator().next().location()); + Assertions.assertEquals("yibcy", response.iterator().next().tags().get("ihsgq")); + Assertions.assertEquals("pbdbzqgqqi", response.iterator().next().description()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageInnerTests.java index 3d96ff0923f19..f8a391dad85d1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageInnerTests.java @@ -10,11 +10,9 @@ public final class DiagnosticsPackageInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsPackageInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"Collected\",\"reason\":\"majtjaod\"},\"id\":\"bnbdxkqpxokajion\",\"name\":\"imexgstxgcpodgma\",\"type\":\"jrmvdjwzrlo\"}") - .toObject(DiagnosticsPackageInner.class); + DiagnosticsPackageInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"Collected\",\"reason\":\"majtjaod\"},\"id\":\"bnbdxkqpxokajion\",\"name\":\"imexgstxgcpodgma\",\"type\":\"jrmvdjwzrlo\"}") + .toObject(DiagnosticsPackageInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageListResultTests.java index 3d9a33a817c94..52762e376df7f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackageListResultTests.java @@ -12,18 +12,15 @@ public final class DiagnosticsPackageListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsPackageListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"NotStarted\",\"reason\":\"bkbfkgukdkex\"},\"id\":\"pofm\",\"name\":\"axcfjpgddtocjjx\",\"type\":\"vpmouexhdzxib\"},{\"properties\":{\"provisioningState\":\"Deleting\",\"status\":\"Collected\",\"reason\":\"qbzvddntwnd\"},\"id\":\"cbtwnpzaoqvuh\",\"name\":\"hcffcyddglmjthjq\",\"type\":\"wpyeicxmqciwqvh\"}],\"nextLink\":\"ixuigdtopbobj\"}") - .toObject(DiagnosticsPackageListResult.class); + DiagnosticsPackageListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"NotStarted\",\"reason\":\"bkbfkgukdkex\"},\"id\":\"pofm\",\"name\":\"axcfjpgddtocjjx\",\"type\":\"vpmouexhdzxib\"},{\"properties\":{\"provisioningState\":\"Deleting\",\"status\":\"Collected\",\"reason\":\"qbzvddntwnd\"},\"id\":\"cbtwnpzaoqvuh\",\"name\":\"hcffcyddglmjthjq\",\"type\":\"wpyeicxmqciwqvh\"}],\"nextLink\":\"ixuigdtopbobj\"}") + .toObject(DiagnosticsPackageListResult.class); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsPackageListResult model = - new DiagnosticsPackageListResult() - .withValue(Arrays.asList(new DiagnosticsPackageInner(), new DiagnosticsPackageInner())); + DiagnosticsPackageListResult model = new DiagnosticsPackageListResult() + .withValue(Arrays.asList(new DiagnosticsPackageInner(), new DiagnosticsPackageInner())); model = BinaryData.fromObject(model).toObject(DiagnosticsPackageListResult.class); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagePropertiesFormatTests.java index fa5d6d4502506..8c5df98ebe507 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagePropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagePropertiesFormatTests.java @@ -10,10 +10,9 @@ public final class DiagnosticsPackagePropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsPackagePropertiesFormat model = - BinaryData - .fromString("{\"provisioningState\":\"Canceled\",\"status\":\"Error\",\"reason\":\"ijcoejctb\"}") - .toObject(DiagnosticsPackagePropertiesFormat.class); + DiagnosticsPackagePropertiesFormat model = BinaryData + .fromString("{\"provisioningState\":\"Canceled\",\"status\":\"Error\",\"reason\":\"ijcoejctb\"}") + .toObject(DiagnosticsPackagePropertiesFormat.class); } @org.junit.jupiter.api.Test diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateMockTests.java index 65fa4d71e4d2d..57aa0edd84259 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesCreateOrUpdateMockTests.java @@ -29,38 +29,26 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"Collecting\",\"reason\":\"fmjnnawtqa\"},\"id\":\"xuckpggqoweyir\",\"name\":\"hlisngw\",\"type\":\"lqqmpiz\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"Collecting\",\"reason\":\"t\"},\"id\":\"kzhajqglcfhm\",\"name\":\"rqryxynqn\",\"type\":\"rd\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DiagnosticsPackage response = manager.diagnosticsPackages().createOrUpdate("um", "qwazlnqnmcjngzq", + "qxtbjwgnyf", com.azure.core.util.Context.NONE); - DiagnosticsPackage response = - manager - .diagnosticsPackages() - .createOrUpdate("jkwrusnkq", "hsyrqunj", "hdenxaulk", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteMockTests.java index ea9732673f649..0e53df5b54f63 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesDeleteMockTests.java @@ -32,32 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .diagnosticsPackages() - .delete("rgxffmshkw", "bkgozxwopdbydpi", "qaclnapxbiy", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.diagnosticsPackages().delete("llxecwc", "ojphslhc", "wjutifdwfmv", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetWithResponseMockTests.java index fb84452a0b5a2..00ed434feccf3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesGetWithResponseMockTests.java @@ -29,39 +29,27 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"Collecting\",\"reason\":\"grwsdp\"},\"id\":\"atzv\",\"name\":\"bglbyvict\",\"type\":\"tbrxkjz\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"Error\",\"reason\":\"sxwaabzm\"},\"id\":\"rygznmmaxriz\",\"name\":\"zob\",\"type\":\"opxlhslnelxieixy\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DiagnosticsPackage response = manager.diagnosticsPackages() + .getWithResponse("sovwxznptgoeiyb", "abpfhvfs", "kvntjlrigjkskyri", com.azure.core.util.Context.NONE) + .getValue(); - DiagnosticsPackage response = - manager - .diagnosticsPackages() - .getWithResponse("uwnpqxpxiwfcng", "saasiixtmkzj", "kv", com.azure.core.util.Context.NONE) - .getValue(); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneMockTests.java index 24647bb97397f..be9cf5da5c9ca 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsPackagesListByPacketCoreControlPlaneMockTests.java @@ -30,38 +30,26 @@ public void testListByPacketCoreControlPlane() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"status\":\"Collected\",\"reason\":\"uuyilfl\"},\"id\":\"iquvrehm\",\"name\":\"njhvsujztc\",\"type\":\"ytqj\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"status\":\"Collecting\",\"reason\":\"ckpzvcpopmxeln\"},\"id\":\"ltyjedexxmlfmk\",\"name\":\"scazuawxtzxpu\",\"type\":\"mwabzxrvxc\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.diagnosticsPackages() + .listByPacketCoreControlPlane("gorqjbttzh", "aglkafhon", com.azure.core.util.Context.NONE); - PagedIterable response = - manager - .diagnosticsPackages() - .listByPacketCoreControlPlane("nugj", "nfsm", com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsUploadConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsUploadConfigurationTests.java index e75f1ee82b64c..ebb2fdc95783d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsUploadConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DiagnosticsUploadConfigurationTests.java @@ -11,18 +11,17 @@ public final class DiagnosticsUploadConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DiagnosticsUploadConfiguration model = - BinaryData - .fromString("{\"storageAccountContainerUrl\":\"dhbt\"}") + DiagnosticsUploadConfiguration model + = BinaryData.fromString("{\"storageAccountContainerUrl\":\"wpucwwfvovbv\"}") .toObject(DiagnosticsUploadConfiguration.class); - Assertions.assertEquals("dhbt", model.storageAccountContainerUrl()); + Assertions.assertEquals("wpucwwfvovbv", model.storageAccountContainerUrl()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DiagnosticsUploadConfiguration model = - new DiagnosticsUploadConfiguration().withStorageAccountContainerUrl("dhbt"); + DiagnosticsUploadConfiguration model + = new DiagnosticsUploadConfiguration().withStorageAccountContainerUrl("wpucwwfvovbv"); model = BinaryData.fromObject(model).toObject(DiagnosticsUploadConfiguration.class); - Assertions.assertEquals("dhbt", model.storageAccountContainerUrl()); + Assertions.assertEquals("wpucwwfvovbv", model.storageAccountContainerUrl()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DnnIpPairTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DnnIpPairTests.java new file mode 100644 index 0000000000000..8b38ee7b75dc5 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/DnnIpPairTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.DnnIpPair; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import org.junit.jupiter.api.Assertions; + +public final class DnnIpPairTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DnnIpPair model = BinaryData.fromString("{\"dnn\":\"poekrsgsgb\",\"ueIpAddress\":{\"ipV4Addr\":\"zq\"}}") + .toObject(DnnIpPair.class); + Assertions.assertEquals("poekrsgsgb", model.dnn()); + Assertions.assertEquals("zq", model.ueIpAddress().ipV4Addr()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DnnIpPair model = new DnnIpPair().withDnn("poekrsgsgb").withUeIpAddress(new UeIpAddress().withIpV4Addr("zq")); + model = BinaryData.fromObject(model).toObject(DnnIpPair.class); + Assertions.assertEquals("poekrsgsgb", model.dnn()); + Assertions.assertEquals("zq", model.ueIpAddress().ipV4Addr()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/EventHubConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/EventHubConfigurationTests.java index 029f0d0ba2886..c447b15065101 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/EventHubConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/EventHubConfigurationTests.java @@ -11,20 +11,19 @@ public final class EventHubConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - EventHubConfiguration model = - BinaryData - .fromString("{\"id\":\"kphywpnvjto\",\"reportingInterval\":1979421186}") + EventHubConfiguration model + = BinaryData.fromString("{\"id\":\"euecivyhzceuoj\",\"reportingInterval\":347281810}") .toObject(EventHubConfiguration.class); - Assertions.assertEquals("kphywpnvjto", model.id()); - Assertions.assertEquals(1979421186, model.reportingInterval()); + Assertions.assertEquals("euecivyhzceuoj", model.id()); + Assertions.assertEquals(347281810, model.reportingInterval()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - EventHubConfiguration model = - new EventHubConfiguration().withId("kphywpnvjto").withReportingInterval(1979421186); + EventHubConfiguration model + = new EventHubConfiguration().withId("euecivyhzceuoj").withReportingInterval(347281810); model = BinaryData.fromObject(model).toObject(EventHubConfiguration.class); - Assertions.assertEquals("kphywpnvjto", model.id()); - Assertions.assertEquals(1979421186, model.reportingInterval()); + Assertions.assertEquals("euecivyhzceuoj", model.id()); + Assertions.assertEquals(347281810, model.reportingInterval()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoInnerTests.java new file mode 100644 index 0000000000000..2a6687496c0cc --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoInnerTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.ExtendedUeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfoProperties; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; + +public final class ExtendedUeInfoInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ExtendedUeInfoInner model = BinaryData.fromString( + "{\"properties\":{\"ratType\":\"ExtendedUeInfoProperties\",\"lastReadAt\":\"2021-02-12T04:30:40Z\"},\"id\":\"zdmovzv\",\"name\":\"va\",\"type\":\"wzqa\"}") + .toObject(ExtendedUeInfoInner.class); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-12T04:30:40Z"), model.properties().lastReadAt()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ExtendedUeInfoInner model = new ExtendedUeInfoInner().withProperties( + new ExtendedUeInfoProperties().withLastReadAt(OffsetDateTime.parse("2021-02-12T04:30:40Z"))); + model = BinaryData.fromObject(model).toObject(ExtendedUeInfoInner.class); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-12T04:30:40Z"), model.properties().lastReadAt()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoPropertiesTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoPropertiesTests.java new file mode 100644 index 0000000000000..c7e3c9397999a --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInfoPropertiesTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfoProperties; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; + +public final class ExtendedUeInfoPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ExtendedUeInfoProperties model = BinaryData + .fromString("{\"ratType\":\"ExtendedUeInfoProperties\",\"lastReadAt\":\"2021-01-08T21:19:01Z\"}") + .toObject(ExtendedUeInfoProperties.class); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-08T21:19:01Z"), model.lastReadAt()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ExtendedUeInfoProperties model + = new ExtendedUeInfoProperties().withLastReadAt(OffsetDateTime.parse("2021-01-08T21:19:01Z")); + model = BinaryData.fromObject(model).toObject(ExtendedUeInfoProperties.class); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-08T21:19:01Z"), model.lastReadAt()); + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationsGetWithResponseMockTests.java similarity index 70% rename from sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetWithResponseMockTests.java rename to sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationsGetWithResponseMockTests.java index 89686aaee7114..704178601e042 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ExtendedUeInformationsGetWithResponseMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.generated; +package com.azure.resourcemanager.mobilenetwork.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -11,8 +11,8 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.communication.CommunicationManager; -import com.azure.resourcemanager.communication.models.SuppressionListResource; +import com.azure.resourcemanager.mobilenetwork.MobileNetworkManager; +import com.azure.resourcemanager.mobilenetwork.models.ExtendedUeInfo; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -23,7 +23,7 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SuppressionListsGetWithResponseMockTests { +public final class ExtendedUeInformationsGetWithResponseMockTests { @Test public void testGetWithResponse() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"listName\":\"atpxl\",\"lastUpdatedTimeStamp\":\"xcyjmoadsuvarmy\",\"createdTimeStamp\":\"mjsjqb\",\"dataLocation\":\"hyxxrwlycoduhpk\"},\"id\":\"gymare\",\"name\":\"n\",\"type\":\"jxqugjhky\"}"; + = "{\"properties\":{\"ratType\":\"ExtendedUeInfoProperties\",\"lastReadAt\":\"2021-10-15T00:22:29Z\"},\"id\":\"vmwfauxxep\",\"name\":\"ywbo\",\"type\":\"mcqmiciijqp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -44,13 +44,13 @@ public void testGetWithResponse() throws Exception { return Mono.just(httpResponse); })); - CommunicationManager manager = CommunicationManager.configure().withHttpClient(httpClient).authenticate( + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - SuppressionListResource response = manager.suppressionLists().getWithResponse("rvimjwosytxitcsk", - "cktqumiekkezzi", "hlyfjhdgqgg", "bdunygaeqid", com.azure.core.util.Context.NONE).getValue(); + ExtendedUeInfo response = manager.extendedUeInformations() + .getWithResponse("atbwbqam", "e", "liys", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("atpxl", response.listName()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-15T00:22:29Z"), response.properties().lastReadAt()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GNbIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GNbIdTests.java new file mode 100644 index 0000000000000..063812e5666a6 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GNbIdTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.GNbId; +import org.junit.jupiter.api.Assertions; + +public final class GNbIdTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + GNbId model + = BinaryData.fromString("{\"bitLength\":598099932,\"gNBValue\":\"nscliqhzvhxnk\"}").toObject(GNbId.class); + Assertions.assertEquals(598099932, model.bitLength()); + Assertions.assertEquals("nscliqhzvhxnk", model.gNBValue()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + GNbId model = new GNbId().withBitLength(598099932).withGNBValue("nscliqhzvhxnk"); + model = BinaryData.fromObject(model).toObject(GNbId.class); + Assertions.assertEquals(598099932, model.bitLength()); + Assertions.assertEquals("nscliqhzvhxnk", model.gNBValue()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GlobalRanNodeIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GlobalRanNodeIdTests.java new file mode 100644 index 0000000000000..6926cbd21f24e --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/GlobalRanNodeIdTests.java @@ -0,0 +1,48 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.GNbId; +import com.azure.resourcemanager.mobilenetwork.models.GlobalRanNodeId; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import org.junit.jupiter.api.Assertions; + +public final class GlobalRanNodeIdTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + GlobalRanNodeId model = BinaryData.fromString( + "{\"plmnId\":{\"mcc\":\"mtk\",\"mnc\":\"bo\"},\"gNbId\":{\"bitLength\":1558773691,\"gNBValue\":\"dxzxhi\"},\"ngeNbId\":\"rbbcevq\",\"eNbId\":\"tltdhlfkqojpy\",\"n3IwfId\":\"gtrd\",\"wagfId\":\"ifmzzsd\",\"tngfId\":\"brn\",\"nid\":\"u\"}") + .toObject(GlobalRanNodeId.class); + Assertions.assertEquals("mtk", model.plmnId().mcc()); + Assertions.assertEquals("bo", model.plmnId().mnc()); + Assertions.assertEquals(1558773691, model.gNbId().bitLength()); + Assertions.assertEquals("dxzxhi", model.gNbId().gNBValue()); + Assertions.assertEquals("rbbcevq", model.ngeNbId()); + Assertions.assertEquals("tltdhlfkqojpy", model.eNbId()); + Assertions.assertEquals("gtrd", model.n3IwfId()); + Assertions.assertEquals("ifmzzsd", model.wagfId()); + Assertions.assertEquals("brn", model.tngfId()); + Assertions.assertEquals("u", model.nid()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + GlobalRanNodeId model = new GlobalRanNodeId().withPlmnId(new PlmnId().withMcc("mtk").withMnc("bo")) + .withGNbId(new GNbId().withBitLength(1558773691).withGNBValue("dxzxhi")).withNgeNbId("rbbcevq") + .withENbId("tltdhlfkqojpy").withN3IwfId("gtrd").withWagfId("ifmzzsd").withTngfId("brn").withNid("u"); + model = BinaryData.fromObject(model).toObject(GlobalRanNodeId.class); + Assertions.assertEquals("mtk", model.plmnId().mcc()); + Assertions.assertEquals("bo", model.plmnId().mnc()); + Assertions.assertEquals(1558773691, model.gNbId().bitLength()); + Assertions.assertEquals("dxzxhi", model.gNbId().gNBValue()); + Assertions.assertEquals("rbbcevq", model.ngeNbId()); + Assertions.assertEquals("tltdhlfkqojpy", model.eNbId()); + Assertions.assertEquals("gtrd", model.n3IwfId()); + Assertions.assertEquals("ifmzzsd", model.wagfId()); + Assertions.assertEquals("brn", model.tngfId()); + Assertions.assertEquals("u", model.nid()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/Guti5GTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/Guti5GTests.java new file mode 100644 index 0000000000000..182d9e95fc95a --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/Guti5GTests.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.AmfId; +import com.azure.resourcemanager.mobilenetwork.models.Guti5G; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import org.junit.jupiter.api.Assertions; + +public final class Guti5GTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + Guti5G model = BinaryData.fromString( + "{\"plmn\":{\"mcc\":\"zflbqvg\",\"mnc\":\"qvlgafcqusrdvetn\"},\"amfId\":{\"regionId\":999562740,\"setId\":89870573,\"pointer\":2059814466},\"fivegTmsi\":1022807330}") + .toObject(Guti5G.class); + Assertions.assertEquals("zflbqvg", model.plmn().mcc()); + Assertions.assertEquals("qvlgafcqusrdvetn", model.plmn().mnc()); + Assertions.assertEquals(999562740, model.amfId().regionId()); + Assertions.assertEquals(89870573, model.amfId().setId()); + Assertions.assertEquals(2059814466, model.amfId().pointer()); + Assertions.assertEquals(1022807330, model.fivegTmsi()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + Guti5G model = new Guti5G().withPlmn(new PlmnId().withMcc("zflbqvg").withMnc("qvlgafcqusrdvetn")) + .withAmfId(new AmfId().withRegionId(999562740).withSetId(89870573).withPointer(2059814466)) + .withFivegTmsi(1022807330); + model = BinaryData.fromObject(model).toObject(Guti5G.class); + Assertions.assertEquals("zflbqvg", model.plmn().mcc()); + Assertions.assertEquals("qvlgafcqusrdvetn", model.plmn().mnc()); + Assertions.assertEquals(999562740, model.amfId().regionId()); + Assertions.assertEquals(89870573, model.amfId().setId()); + Assertions.assertEquals(2059814466, model.amfId().pointer()); + Assertions.assertEquals(1022807330, model.fivegTmsi()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPrivateKeysProvisioningTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPrivateKeysProvisioningTests.java new file mode 100644 index 0000000000000..00cbdc0c68407 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPrivateKeysProvisioningTests.java @@ -0,0 +1,22 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPrivateKeysProvisioning; + +public final class HomeNetworkPrivateKeysProvisioningTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HomeNetworkPrivateKeysProvisioning model + = BinaryData.fromString("{\"state\":\"Provisioned\"}").toObject(HomeNetworkPrivateKeysProvisioning.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HomeNetworkPrivateKeysProvisioning model = new HomeNetworkPrivateKeysProvisioning(); + model = BinaryData.fromObject(model).toObject(HomeNetworkPrivateKeysProvisioning.class); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPublicKeyTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPublicKeyTests.java new file mode 100644 index 0000000000000..6dc34dc92e2da --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HomeNetworkPublicKeyTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPublicKey; +import org.junit.jupiter.api.Assertions; + +public final class HomeNetworkPublicKeyTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + HomeNetworkPublicKey model + = BinaryData.fromString("{\"id\":184362773,\"url\":\"ualhbxxhejj\"}").toObject(HomeNetworkPublicKey.class); + Assertions.assertEquals(184362773, model.id()); + Assertions.assertEquals("ualhbxxhejj", model.url()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + HomeNetworkPublicKey model = new HomeNetworkPublicKey().withId(184362773).withUrl("ualhbxxhejj"); + model = BinaryData.fromObject(model).toObject(HomeNetworkPublicKey.class); + Assertions.assertEquals(184362773, model.id()); + Assertions.assertEquals("ualhbxxhejj", model.url()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HttpsServerCertificateTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HttpsServerCertificateTests.java index f40ad5d97877e..c93e05cf5ea99 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HttpsServerCertificateTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/HttpsServerCertificateTests.java @@ -11,18 +11,17 @@ public final class HttpsServerCertificateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - HttpsServerCertificate model = - BinaryData - .fromString( - "{\"certificateUrl\":\"wrupqsxvnmicykvc\",\"provisioning\":{\"state\":\"NotProvisioned\",\"reason\":\"lo\"}}") - .toObject(HttpsServerCertificate.class); - Assertions.assertEquals("wrupqsxvnmicykvc", model.certificateUrl()); + HttpsServerCertificate model = BinaryData + .fromString( + "{\"certificateUrl\":\"hashsfwxosow\",\"provisioning\":{\"state\":\"Provisioned\",\"reason\":\"i\"}}") + .toObject(HttpsServerCertificate.class); + Assertions.assertEquals("hashsfwxosow", model.certificateUrl()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HttpsServerCertificate model = new HttpsServerCertificate().withCertificateUrl("wrupqsxvnmicykvc"); + HttpsServerCertificate model = new HttpsServerCertificate().withCertificateUrl("hashsfwxosow"); model = BinaryData.fromObject(model).toObject(HttpsServerCertificate.class); - Assertions.assertEquals("wrupqsxvnmicykvc", model.certificateUrl()); + Assertions.assertEquals("hashsfwxosow", model.certificateUrl()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/IdentityAndTagsObjectTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/IdentityAndTagsObjectTests.java index a0688c0ab3017..879a682cac625 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/IdentityAndTagsObjectTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/IdentityAndTagsObjectTests.java @@ -16,36 +16,24 @@ public final class IdentityAndTagsObjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - IdentityAndTagsObject model = - BinaryData - .fromString( - "{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"ugxywpmueef\":{\"principalId\":\"50a92351-5a87-4b17-b650-81a112491712\",\"clientId\":\"b523b088-53bb-4a07-a297-9b7ed86bfafa\"},\"fqkquj\":{\"principalId\":\"5f016837-7506-4398-9752-3a2516c8bba9\",\"clientId\":\"6495bd3d-8c39-429f-bd86-20a832fe5c83\"},\"uyonobglaoc\":{\"principalId\":\"523838f9-3940-4490-87b7-0b2129aaf35e\",\"clientId\":\"b27875f6-bce4-4392-91ed-6e2eb997aa6c\"},\"ccm\":{\"principalId\":\"5e3686b1-f064-4c00-9295-ea6c013e5dcf\",\"clientId\":\"11f43685-4ca6-468c-9769-2ede339c8a4e\"}}},\"tags\":{\"moyrxvwfudwpz\":\"dxyt\",\"rqjbhckfrl\":\"txhdzh\",\"ca\":\"rxsbkyvp\"}}") - .toObject(IdentityAndTagsObject.class); + IdentityAndTagsObject model = BinaryData.fromString( + "{\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"ej\":{\"principalId\":\"93ba844f-51a2-4cf1-a1a0-adff1fc878ed\",\"clientId\":\"07b594cc-6127-454b-9e97-3a39b96d70da\"},\"orxzdmohctbqvud\":{\"principalId\":\"da7330a6-1c41-4510-b455-035718204891\",\"clientId\":\"8eb90e3a-4ff4-48c5-bd22-5c5ff3ecfe66\"},\"ndnvo\":{\"principalId\":\"f12a39a2-ab2d-433f-90f9-1164fb80612a\",\"clientId\":\"9bfb80f5-1b67-4891-91b8-adef5e306f20\"}}},\"tags\":{\"dyggdtjixhbku\":\"jjugwdkcglhslaz\",\"fyexfwhy\":\"fqweykhmene\",\"amdecte\":\"cibvyvdcsitynn\"}}") + .toObject(IdentityAndTagsObject.class); Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("dxyt", model.tags().get("moyrxvwfudwpz")); + Assertions.assertEquals("jjugwdkcglhslaz", model.tags().get("dyggdtjixhbku")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - IdentityAndTagsObject model = - new IdentityAndTagsObject() - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities( - mapOf( - "ugxywpmueef", - new UserAssignedIdentity(), - "fqkquj", - new UserAssignedIdentity(), - "uyonobglaoc", - new UserAssignedIdentity(), - "ccm", - new UserAssignedIdentity()))) - .withTags(mapOf("moyrxvwfudwpz", "dxyt", "rqjbhckfrl", "txhdzh", "ca", "rxsbkyvp")); + IdentityAndTagsObject model = new IdentityAndTagsObject() + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("ej", new UserAssignedIdentity(), "orxzdmohctbqvud", + new UserAssignedIdentity(), "ndnvo", new UserAssignedIdentity()))) + .withTags( + mapOf("dyggdtjixhbku", "jjugwdkcglhslaz", "fyexfwhy", "fqweykhmene", "amdecte", "cibvyvdcsitynn")); model = BinaryData.fromObject(model).toObject(IdentityAndTagsObject.class); Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("dxyt", model.tags().get("moyrxvwfudwpz")); + Assertions.assertEquals("jjugwdkcglhslaz", model.tags().get("dyggdtjixhbku")); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InstallationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InstallationTests.java index 52c1f9f73dce8..6ae1396591005 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InstallationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InstallationTests.java @@ -12,18 +12,16 @@ public final class InstallationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Installation model = - BinaryData - .fromString( - "{\"desiredState\":\"Installed\",\"state\":\"RollingBack\",\"reinstallRequired\":\"Required\",\"reasons\":[\"UserPlaneAccessVirtualIpv4AddressesHasChanged\"],\"operation\":{\"id\":\"lbbovplw\"}}") - .toObject(Installation.class); - Assertions.assertEquals(DesiredInstallationState.INSTALLED, model.desiredState()); + Installation model = BinaryData.fromString( + "{\"desiredState\":\"Uninstalled\",\"state\":\"Reinstalling\",\"reinstallRequired\":\"Required\",\"reasons\":[\"NoAttachedDataNetworks\"],\"operation\":{\"id\":\"zntxhdz\"}}") + .toObject(Installation.class); + Assertions.assertEquals(DesiredInstallationState.UNINSTALLED, model.desiredState()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Installation model = new Installation().withDesiredState(DesiredInstallationState.INSTALLED); + Installation model = new Installation().withDesiredState(DesiredInstallationState.UNINSTALLED); model = BinaryData.fromObject(model).toObject(Installation.class); - Assertions.assertEquals(DesiredInstallationState.INSTALLED, model.desiredState()); + Assertions.assertEquals(DesiredInstallationState.UNINSTALLED, model.desiredState()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InterfacePropertiesTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InterfacePropertiesTests.java index 83c63d72b723d..d6ce12b8d387a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InterfacePropertiesTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/InterfacePropertiesTests.java @@ -11,11 +11,10 @@ public final class InterfacePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - InterfaceProperties model = - BinaryData - .fromString( - "{\"name\":\"vyq\",\"ipv4Address\":\"wby\",\"ipv4Subnet\":\"k\",\"ipv4Gateway\":\"dumjgrtfwvuk\"}") - .toObject(InterfaceProperties.class); + InterfaceProperties model = BinaryData + .fromString( + "{\"name\":\"vyq\",\"ipv4Address\":\"wby\",\"ipv4Subnet\":\"k\",\"ipv4Gateway\":\"dumjgrtfwvuk\"}") + .toObject(InterfaceProperties.class); Assertions.assertEquals("vyq", model.name()); Assertions.assertEquals("wby", model.ipv4Address()); Assertions.assertEquals("k", model.ipv4Subnet()); @@ -24,12 +23,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - InterfaceProperties model = - new InterfaceProperties() - .withName("vyq") - .withIpv4Address("wby") - .withIpv4Subnet("k") - .withIpv4Gateway("dumjgrtfwvuk"); + InterfaceProperties model = new InterfaceProperties().withName("vyq").withIpv4Address("wby").withIpv4Subnet("k") + .withIpv4Gateway("dumjgrtfwvuk"); model = BinaryData.fromObject(model).toObject(InterfaceProperties.class); Assertions.assertEquals("vyq", model.name()); Assertions.assertEquals("wby", model.ipv4Address()); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/LocalDiagnosticsAccessConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/LocalDiagnosticsAccessConfigurationTests.java index a012f8de98a23..45053ccaa318f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/LocalDiagnosticsAccessConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/LocalDiagnosticsAccessConfigurationTests.java @@ -13,23 +13,20 @@ public final class LocalDiagnosticsAccessConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - LocalDiagnosticsAccessConfiguration model = - BinaryData - .fromString( - "{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"scwsv\",\"provisioning\":{\"state\":\"NotProvisioned\",\"reason\":\"g\"}}}") - .toObject(LocalDiagnosticsAccessConfiguration.class); - Assertions.assertEquals(AuthenticationType.PASSWORD, model.authenticationType()); - Assertions.assertEquals("scwsv", model.httpsServerCertificate().certificateUrl()); + LocalDiagnosticsAccessConfiguration model = BinaryData.fromString( + "{\"authenticationType\":\"AAD\",\"httpsServerCertificate\":{\"certificateUrl\":\"isgwbnbbeldawkz\",\"provisioning\":{\"state\":\"Failed\",\"reason\":\"urqhaka\"}}}") + .toObject(LocalDiagnosticsAccessConfiguration.class); + Assertions.assertEquals(AuthenticationType.AAD, model.authenticationType()); + Assertions.assertEquals("isgwbnbbeldawkz", model.httpsServerCertificate().certificateUrl()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LocalDiagnosticsAccessConfiguration model = - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.PASSWORD) - .withHttpsServerCertificate(new HttpsServerCertificate().withCertificateUrl("scwsv")); + LocalDiagnosticsAccessConfiguration model + = new LocalDiagnosticsAccessConfiguration().withAuthenticationType(AuthenticationType.AAD) + .withHttpsServerCertificate(new HttpsServerCertificate().withCertificateUrl("isgwbnbbeldawkz")); model = BinaryData.fromObject(model).toObject(LocalDiagnosticsAccessConfiguration.class); - Assertions.assertEquals(AuthenticationType.PASSWORD, model.authenticationType()); - Assertions.assertEquals("scwsv", model.httpsServerCertificate().certificateUrl()); + Assertions.assertEquals(AuthenticationType.AAD, model.authenticationType()); + Assertions.assertEquals("isgwbnbbeldawkz", model.httpsServerCertificate().certificateUrl()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ManagedServiceIdentityTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ManagedServiceIdentityTests.java index 92a08cce735e9..e298a82d71a3f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ManagedServiceIdentityTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ManagedServiceIdentityTests.java @@ -15,23 +15,19 @@ public final class ManagedServiceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ManagedServiceIdentity model = - BinaryData - .fromString( - "{\"type\":\"None\",\"userAssignedIdentities\":{\"hoxus\":{\"principalId\":\"d8ec4b94-42f5-4d75-a814-48a9d2955920\",\"clientId\":\"79fb29f8-425a-4e38-8a70-0b576086a2dd\"},\"abgy\":{\"principalId\":\"3355358b-96f4-4c03-8f78-aa731062614b\",\"clientId\":\"c7b242d7-68d7-40b9-bc95-b8afb8c67e44\"}}}") - .toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); + ManagedServiceIdentity model = BinaryData.fromString( + "{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"gwdslfhotwm\":{\"principalId\":\"0c3d0632-d783-4ea3-ab2b-1a57dd7b9e2c\",\"clientId\":\"8cb779c1-f44c-4e50-b2c5-1b7d9dac147e\"},\"pwlbjnpg\":{\"principalId\":\"7efb2d8e-4c01-4c1d-92fb-116102c79d8c\",\"clientId\":\"aa044b1d-7311-4e7c-89ce-5bb7b175ecfc\"},\"tadehxnltyfsopp\":{\"principalId\":\"3a80ce23-0918-443b-afab-b9e68b8764cc\",\"clientId\":\"7d53a280-2690-417a-8378-3b1d0a79be8a\"}}}") + .toObject(ManagedServiceIdentity.class); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedServiceIdentity model = - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities( - mapOf("hoxus", new UserAssignedIdentity(), "abgy", new UserAssignedIdentity())); + ManagedServiceIdentity model = new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf("gwdslfhotwm", new UserAssignedIdentity(), "pwlbjnpg", + new UserAssignedIdentity(), "tadehxnltyfsopp", new UserAssignedIdentity())); model = BinaryData.fromObject(model).toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.type()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworkResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworkResourceIdTests.java index a3d3140fb49a8..c9e9ef1cc303e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworkResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworkResourceIdTests.java @@ -11,15 +11,15 @@ public final class MobileNetworkResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MobileNetworkResourceId model = - BinaryData.fromString("{\"id\":\"sv\"}").toObject(MobileNetworkResourceId.class); - Assertions.assertEquals("sv", model.id()); + MobileNetworkResourceId model + = BinaryData.fromString("{\"id\":\"sibircgpi\"}").toObject(MobileNetworkResourceId.class); + Assertions.assertEquals("sibircgpi", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MobileNetworkResourceId model = new MobileNetworkResourceId().withId("sv"); + MobileNetworkResourceId model = new MobileNetworkResourceId().withId("sibircgpi"); model = BinaryData.fromObject(model).toObject(MobileNetworkResourceId.class); - Assertions.assertEquals("sv", model.id()); + Assertions.assertEquals("sibircgpi", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteMockTests.java index 4190d24aec34a..e6ee4f06739a2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/MobileNetworksDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.mobileNetworks().delete("w", "auunfprnjletlx", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.mobileNetworks().delete("s", "sphaivmxyasflvg", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NaptConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NaptConfigurationTests.java index 701b9b9c45085..0c68cf1eae238 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NaptConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NaptConfigurationTests.java @@ -15,11 +15,9 @@ public final class NaptConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NaptConfiguration model = - BinaryData - .fromString( - "{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":1991761310,\"maxPort\":1674535960},\"portReuseHoldTime\":{\"tcp\":14804050,\"udp\":2045600331},\"pinholeLimits\":1132741350,\"pinholeTimeouts\":{\"tcp\":1617618462,\"udp\":1833864756,\"icmp\":1656493513}}") - .toObject(NaptConfiguration.class); + NaptConfiguration model = BinaryData.fromString( + "{\"enabled\":\"Disabled\",\"portRange\":{\"minPort\":1991761310,\"maxPort\":1674535960},\"portReuseHoldTime\":{\"tcp\":14804050,\"udp\":2045600331},\"pinholeLimits\":1132741350,\"pinholeTimeouts\":{\"tcp\":1617618462,\"udp\":1833864756,\"icmp\":1656493513}}") + .toObject(NaptConfiguration.class); Assertions.assertEquals(NaptEnabled.DISABLED, model.enabled()); Assertions.assertEquals(1991761310, model.portRange().minPort()); Assertions.assertEquals(1674535960, model.portRange().maxPort()); @@ -33,14 +31,11 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NaptConfiguration model = - new NaptConfiguration() - .withEnabled(NaptEnabled.DISABLED) - .withPortRange(new PortRange().withMinPort(1991761310).withMaxPort(1674535960)) - .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(14804050).withUdp(2045600331)) - .withPinholeLimits(1132741350) - .withPinholeTimeouts( - new PinholeTimeouts().withTcp(1617618462).withUdp(1833864756).withIcmp(1656493513)); + NaptConfiguration model = new NaptConfiguration().withEnabled(NaptEnabled.DISABLED) + .withPortRange(new PortRange().withMinPort(1991761310).withMaxPort(1674535960)) + .withPortReuseHoldTime(new PortReuseHoldTimes().withTcp(14804050).withUdp(2045600331)) + .withPinholeLimits(1132741350) + .withPinholeTimeouts(new PinholeTimeouts().withTcp(1617618462).withUdp(1833864756).withIcmp(1656493513)); model = BinaryData.fromObject(model).toObject(NaptConfiguration.class); Assertions.assertEquals(NaptEnabled.DISABLED, model.enabled()); Assertions.assertEquals(1991761310, model.portRange().minPort()); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NasRerouteConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NasRerouteConfigurationTests.java index f9234b71bc761..602c9a1253790 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NasRerouteConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/NasRerouteConfigurationTests.java @@ -11,15 +11,15 @@ public final class NasRerouteConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NasRerouteConfiguration model = - BinaryData.fromString("{\"macroMmeGroupId\":927719856}").toObject(NasRerouteConfiguration.class); - Assertions.assertEquals(927719856, model.macroMmeGroupId()); + NasRerouteConfiguration model + = BinaryData.fromString("{\"macroMmeGroupId\":737115334}").toObject(NasRerouteConfiguration.class); + Assertions.assertEquals(737115334, model.macroMmeGroupId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NasRerouteConfiguration model = new NasRerouteConfiguration().withMacroMmeGroupId(927719856); + NasRerouteConfiguration model = new NasRerouteConfiguration().withMacroMmeGroupId(737115334); model = BinaryData.fromObject(model).toObject(NasRerouteConfiguration.class); - Assertions.assertEquals(927719856, model.macroMmeGroupId()); + Assertions.assertEquals(737115334, model.macroMmeGroupId()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationDisplayTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationDisplayTests.java index 9ff96fca1107a..b95cdcef8e934 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationDisplayTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationDisplayTests.java @@ -11,29 +11,23 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"vo\",\"resource\":\"ujjugwdkcglh\",\"operation\":\"azjdyggd\",\"description\":\"ixhbkuofqweykhm\"}") - .toObject(OperationDisplay.class); - Assertions.assertEquals("vo", model.provider()); - Assertions.assertEquals("ujjugwdkcglh", model.resource()); - Assertions.assertEquals("azjdyggd", model.operation()); - Assertions.assertEquals("ixhbkuofqweykhm", model.description()); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"rrqnbpoczvyifqrv\",\"resource\":\"vjsllrmvvdfw\",\"operation\":\"kpnpulexxbczwtr\",\"description\":\"iqzbq\"}") + .toObject(OperationDisplay.class); + Assertions.assertEquals("rrqnbpoczvyifqrv", model.provider()); + Assertions.assertEquals("vjsllrmvvdfw", model.resource()); + Assertions.assertEquals("kpnpulexxbczwtr", model.operation()); + Assertions.assertEquals("iqzbq", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationDisplay model = - new OperationDisplay() - .withProvider("vo") - .withResource("ujjugwdkcglh") - .withOperation("azjdyggd") - .withDescription("ixhbkuofqweykhm"); + OperationDisplay model = new OperationDisplay().withProvider("rrqnbpoczvyifqrv").withResource("vjsllrmvvdfw") + .withOperation("kpnpulexxbczwtr").withDescription("iqzbq"); model = BinaryData.fromObject(model).toObject(OperationDisplay.class); - Assertions.assertEquals("vo", model.provider()); - Assertions.assertEquals("ujjugwdkcglh", model.resource()); - Assertions.assertEquals("azjdyggd", model.operation()); - Assertions.assertEquals("ixhbkuofqweykhm", model.description()); + Assertions.assertEquals("rrqnbpoczvyifqrv", model.provider()); + Assertions.assertEquals("vjsllrmvvdfw", model.resource()); + Assertions.assertEquals("kpnpulexxbczwtr", model.operation()); + Assertions.assertEquals("iqzbq", model.description()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationInnerTests.java index f8dc3709a04af..34ff5e5fc09db 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationInnerTests.java @@ -11,18 +11,16 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"isDataAction\":false,\"name\":\"yfsoppu\",\"display\":{\"provider\":\"snzwd\",\"resource\":\"bavo\",\"operation\":\"zdmohctbqvu\",\"description\":\"xdn\"}}") - .toObject(OperationInner.class); - Assertions.assertEquals(false, model.isDataAction()); + OperationInner model = BinaryData.fromString( + "{\"isDataAction\":true,\"name\":\"hrdxwzywqsmbs\",\"display\":{\"provider\":\"xim\",\"resource\":\"yocf\",\"operation\":\"ksymd\",\"description\":\"stkiiuxhqyud\"}}") + .toObject(OperationInner.class); + Assertions.assertEquals(true, model.isDataAction()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationInner model = new OperationInner().withIsDataAction(false); + OperationInner model = new OperationInner().withIsDataAction(true); model = BinaryData.fromObject(model).toObject(OperationInner.class); - Assertions.assertEquals(false, model.isDataAction()); + Assertions.assertEquals(true, model.isDataAction()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationListTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationListTests.java index 3797dc89f439c..32ffab4fac0f1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationListTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationListTests.java @@ -10,11 +10,9 @@ public final class OperationListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationList model = - BinaryData - .fromString( - "{\"value\":[{\"isDataAction\":true,\"name\":\"rvynhzgpph\",\"display\":{\"provider\":\"yncocpecfvmmcoo\",\"resource\":\"xlzevgbmqjqabcy\",\"operation\":\"ivkwlzuvccfwnfnb\",\"description\":\"fionl\"}},{\"isDataAction\":false,\"name\":\"tqgtzxdpnqbqq\",\"display\":{\"provider\":\"jfeallnwsub\",\"resource\":\"njampm\",\"operation\":\"nzscxa\",\"description\":\"ooch\"}},{\"isDataAction\":false,\"name\":\"qvpkvlrxnjeaseip\",\"display\":{\"provider\":\"f\",\"resource\":\"keyyi\",\"operation\":\"jbdlwtgrhpdjpju\",\"description\":\"sxazjpq\"}},{\"isDataAction\":false,\"name\":\"alhbx\",\"display\":{\"provider\":\"jj\",\"resource\":\"v\",\"operation\":\"dgwdslfhot\",\"description\":\"cynpwlbjnp\"}}],\"nextLink\":\"cftadeh\"}") - .toObject(OperationList.class); + OperationList model = BinaryData.fromString( + "{\"value\":[{\"isDataAction\":false,\"name\":\"gm\",\"display\":{\"provider\":\"leyyvx\",\"resource\":\"jpkcattpng\",\"operation\":\"rcczsqpjhvmd\",\"description\":\"v\"}},{\"isDataAction\":true,\"name\":\"unqecanoae\",\"display\":{\"provider\":\"hy\",\"resource\":\"trpmo\",\"operation\":\"mcmatuokthfuiu\",\"description\":\"dsfcpkvxodpuoz\"}}],\"nextLink\":\"zydagfuaxbezyiuo\"}") + .toObject(OperationList.class); } @org.junit.jupiter.api.Test diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListMockTests.java index 6f7516e20d194..eb2cf6c456cee 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/OperationsListMockTests.java @@ -31,34 +31,23 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"isDataAction\":true,\"name\":\"zgp\",\"display\":{\"provider\":\"vhjknidi\",\"resource\":\"qjxgpnrhgovfgp\",\"operation\":\"qmhhaowj\",\"description\":\"zvuporqzdfuydz\"}}]}"; + String responseStr + = "{\"value\":[{\"isDataAction\":true,\"name\":\"ggbqi\",\"display\":{\"provider\":\"kbsazgak\",\"resource\":\"cyrcmjdmspo\",\"operation\":\"pv\",\"description\":\"rylniofrzg\"}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureInnerTests.java index b7ac6244a10ea..56aad205a605b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureInnerTests.java @@ -12,29 +12,25 @@ public final class PacketCaptureInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCaptureInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"Running\",\"reason\":\"exfwhy\",\"captureStartTime\":\"2021-05-17T05:51:50Z\",\"networkInterfaces\":[\"yvdcsitynnaa\",\"dectehfiqsc\"],\"bytesToCapturePerPacket\":991412043845588361,\"totalBytesPerSession\":7720678337601187559,\"timeLimitInSeconds\":732207919,\"outputFiles\":[\"q\",\"c\",\"refovgmkqsleyyvx\"]},\"id\":\"jpkcattpng\",\"name\":\"cr\",\"type\":\"czsqpjhvm\"}") - .toObject(PacketCaptureInner.class); - Assertions.assertEquals("yvdcsitynnaa", model.networkInterfaces().get(0)); - Assertions.assertEquals(991412043845588361L, model.bytesToCapturePerPacket()); - Assertions.assertEquals(7720678337601187559L, model.totalBytesPerSession()); - Assertions.assertEquals(732207919, model.timeLimitInSeconds()); + PacketCaptureInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Unknown\",\"status\":\"Stopped\",\"reason\":\"yokacspkw\",\"captureStartTime\":\"2021-04-27T23:18:31Z\",\"networkInterfaces\":[\"pxjmflbvvnchr\",\"cciw\"],\"bytesToCapturePerPacket\":4127972279090027602,\"totalBytesPerSession\":4133391501410189772,\"timeLimitInSeconds\":330822059,\"outputFiles\":[\"iwkuofos\",\"ghsauuimjmvxied\",\"ugidyjrr\",\"byao\"]},\"id\":\"e\",\"name\":\"csonpclhoco\",\"type\":\"slkevle\"}") + .toObject(PacketCaptureInner.class); + Assertions.assertEquals("pxjmflbvvnchr", model.networkInterfaces().get(0)); + Assertions.assertEquals(4127972279090027602L, model.bytesToCapturePerPacket()); + Assertions.assertEquals(4133391501410189772L, model.totalBytesPerSession()); + Assertions.assertEquals(330822059, model.timeLimitInSeconds()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCaptureInner model = - new PacketCaptureInner() - .withNetworkInterfaces(Arrays.asList("yvdcsitynnaa", "dectehfiqsc")) - .withBytesToCapturePerPacket(991412043845588361L) - .withTotalBytesPerSession(7720678337601187559L) - .withTimeLimitInSeconds(732207919); + PacketCaptureInner model + = new PacketCaptureInner().withNetworkInterfaces(Arrays.asList("pxjmflbvvnchr", "cciw")) + .withBytesToCapturePerPacket(4127972279090027602L).withTotalBytesPerSession(4133391501410189772L) + .withTimeLimitInSeconds(330822059); model = BinaryData.fromObject(model).toObject(PacketCaptureInner.class); - Assertions.assertEquals("yvdcsitynnaa", model.networkInterfaces().get(0)); - Assertions.assertEquals(991412043845588361L, model.bytesToCapturePerPacket()); - Assertions.assertEquals(7720678337601187559L, model.totalBytesPerSession()); - Assertions.assertEquals(732207919, model.timeLimitInSeconds()); + Assertions.assertEquals("pxjmflbvvnchr", model.networkInterfaces().get(0)); + Assertions.assertEquals(4127972279090027602L, model.bytesToCapturePerPacket()); + Assertions.assertEquals(4133391501410189772L, model.totalBytesPerSession()); + Assertions.assertEquals(330822059, model.timeLimitInSeconds()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureListResultTests.java index 9d5ea4768244a..34c28ea96e099 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCaptureListResultTests.java @@ -13,33 +13,34 @@ public final class PacketCaptureListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCaptureListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"Stopped\",\"reason\":\"ovm\",\"captureStartTime\":\"2020-12-31T07:20:30Z\",\"networkInterfaces\":[\"spkwlhzdobpxjm\",\"lb\",\"vnchrkcci\",\"wzjuqk\"],\"bytesToCapturePerPacket\":1420869925986942860,\"totalBytesPerSession\":9083271895139408409,\"timeLimitInSeconds\":481973669,\"outputFiles\":[\"oskg\",\"sauuimj\",\"vxieduugidyj\",\"rfbyaosvexcso\"]},\"id\":\"clhocohsl\",\"name\":\"ev\",\"type\":\"eggzfb\"}],\"nextLink\":\"fmvfaxkffeiit\"}") - .toObject(PacketCaptureListResult.class); - Assertions.assertEquals("spkwlhzdobpxjm", model.value().get(0).networkInterfaces().get(0)); - Assertions.assertEquals(1420869925986942860L, model.value().get(0).bytesToCapturePerPacket()); - Assertions.assertEquals(9083271895139408409L, model.value().get(0).totalBytesPerSession()); - Assertions.assertEquals(481973669, model.value().get(0).timeLimitInSeconds()); + PacketCaptureListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"NotStarted\",\"reason\":\"agalpbuxwgipwhon\",\"captureStartTime\":\"2021-08-18T16:08:42Z\",\"networkInterfaces\":[\"hwankixzbinjepu\",\"tmryw\",\"uzoqft\",\"yqzrnkcqvyxlw\"],\"bytesToCapturePerPacket\":4276936142044234939,\"totalBytesPerSession\":5175708823620809129,\"timeLimitInSeconds\":601015315,\"outputFiles\":[\"nwvlryavwhheunmm\",\"hgyxzkonoc\"]},\"id\":\"oklyaxuconuq\",\"name\":\"zf\",\"type\":\"beypewrmjmw\"},{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"NotStarted\",\"reason\":\"cxsenhwlrsff\",\"captureStartTime\":\"2021-03-02T08:14:37Z\",\"networkInterfaces\":[\"lqdqgbiqylihka\"],\"bytesToCapturePerPacket\":9095359841321899444,\"totalBytesPerSession\":4997259722046744559,\"timeLimitInSeconds\":1351604805,\"outputFiles\":[\"snkymuctq\",\"jf\",\"ebrjcxe\"]},\"id\":\"uwutttxfvjrbi\",\"name\":\"phxepcyvahf\",\"type\":\"ljkyqxjvuuj\"}],\"nextLink\":\"idokgjlj\"}") + .toObject(PacketCaptureListResult.class); + Assertions.assertEquals("hwankixzbinjepu", model.value().get(0).networkInterfaces().get(0)); + Assertions.assertEquals(4276936142044234939L, model.value().get(0).bytesToCapturePerPacket()); + Assertions.assertEquals(5175708823620809129L, model.value().get(0).totalBytesPerSession()); + Assertions.assertEquals(601015315, model.value().get(0).timeLimitInSeconds()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCaptureListResult model = - new PacketCaptureListResult() + PacketCaptureListResult model + = new PacketCaptureListResult() .withValue( Arrays .asList( new PacketCaptureInner() - .withNetworkInterfaces(Arrays.asList("spkwlhzdobpxjm", "lb", "vnchrkcci", "wzjuqk")) - .withBytesToCapturePerPacket(1420869925986942860L) - .withTotalBytesPerSession(9083271895139408409L) - .withTimeLimitInSeconds(481973669))); + .withNetworkInterfaces( + Arrays.asList("hwankixzbinjepu", "tmryw", "uzoqft", "yqzrnkcqvyxlw")) + .withBytesToCapturePerPacket(4276936142044234939L) + .withTotalBytesPerSession(5175708823620809129L).withTimeLimitInSeconds(601015315), + new PacketCaptureInner().withNetworkInterfaces(Arrays.asList("lqdqgbiqylihka")) + .withBytesToCapturePerPacket(9095359841321899444L) + .withTotalBytesPerSession(4997259722046744559L).withTimeLimitInSeconds(1351604805))); model = BinaryData.fromObject(model).toObject(PacketCaptureListResult.class); - Assertions.assertEquals("spkwlhzdobpxjm", model.value().get(0).networkInterfaces().get(0)); - Assertions.assertEquals(1420869925986942860L, model.value().get(0).bytesToCapturePerPacket()); - Assertions.assertEquals(9083271895139408409L, model.value().get(0).totalBytesPerSession()); - Assertions.assertEquals(481973669, model.value().get(0).timeLimitInSeconds()); + Assertions.assertEquals("hwankixzbinjepu", model.value().get(0).networkInterfaces().get(0)); + Assertions.assertEquals(4276936142044234939L, model.value().get(0).bytesToCapturePerPacket()); + Assertions.assertEquals(5175708823620809129L, model.value().get(0).totalBytesPerSession()); + Assertions.assertEquals(601015315, model.value().get(0).timeLimitInSeconds()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturePropertiesFormatTests.java index 7638afc692f52..36337a145516a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturePropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturePropertiesFormatTests.java @@ -12,29 +12,25 @@ public final class PacketCapturePropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCapturePropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Canceled\",\"status\":\"Running\",\"reason\":\"sounqecanoaeu\",\"captureStartTime\":\"2021-05-02T19:00:13Z\",\"networkInterfaces\":[\"ltrpmopj\",\"cma\",\"u\",\"kthfui\"],\"bytesToCapturePerPacket\":8411741247205328781,\"totalBytesPerSession\":2922940373355602570,\"timeLimitInSeconds\":1099680690,\"outputFiles\":[\"odpuozmyzydag\",\"uaxbezyiuokkt\",\"hrdxwzywqsmbs\",\"reximoryocfs\"]}") - .toObject(PacketCapturePropertiesFormat.class); - Assertions.assertEquals("ltrpmopj", model.networkInterfaces().get(0)); - Assertions.assertEquals(8411741247205328781L, model.bytesToCapturePerPacket()); - Assertions.assertEquals(2922940373355602570L, model.totalBytesPerSession()); - Assertions.assertEquals(1099680690, model.timeLimitInSeconds()); + PacketCapturePropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Accepted\",\"status\":\"Error\",\"reason\":\"hfmvfaxkffe\",\"captureStartTime\":\"2021-05-15T04:16:32Z\",\"networkInterfaces\":[\"vmezy\",\"shxmzsbbzoggigrx\",\"burvjxxjnspy\",\"ptkoenkoukn\"],\"bytesToCapturePerPacket\":5351704758336650999,\"totalBytesPerSession\":5050642493378346293,\"timeLimitInSeconds\":133898645,\"outputFiles\":[\"ngkpocipazy\"]}") + .toObject(PacketCapturePropertiesFormat.class); + Assertions.assertEquals("vmezy", model.networkInterfaces().get(0)); + Assertions.assertEquals(5351704758336650999L, model.bytesToCapturePerPacket()); + Assertions.assertEquals(5050642493378346293L, model.totalBytesPerSession()); + Assertions.assertEquals(133898645, model.timeLimitInSeconds()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCapturePropertiesFormat model = - new PacketCapturePropertiesFormat() - .withNetworkInterfaces(Arrays.asList("ltrpmopj", "cma", "u", "kthfui")) - .withBytesToCapturePerPacket(8411741247205328781L) - .withTotalBytesPerSession(2922940373355602570L) - .withTimeLimitInSeconds(1099680690); + PacketCapturePropertiesFormat model = new PacketCapturePropertiesFormat() + .withNetworkInterfaces(Arrays.asList("vmezy", "shxmzsbbzoggigrx", "burvjxxjnspy", "ptkoenkoukn")) + .withBytesToCapturePerPacket(5351704758336650999L).withTotalBytesPerSession(5050642493378346293L) + .withTimeLimitInSeconds(133898645); model = BinaryData.fromObject(model).toObject(PacketCapturePropertiesFormat.class); - Assertions.assertEquals("ltrpmopj", model.networkInterfaces().get(0)); - Assertions.assertEquals(8411741247205328781L, model.bytesToCapturePerPacket()); - Assertions.assertEquals(2922940373355602570L, model.totalBytesPerSession()); - Assertions.assertEquals(1099680690, model.timeLimitInSeconds()); + Assertions.assertEquals("vmezy", model.networkInterfaces().get(0)); + Assertions.assertEquals(5351704758336650999L, model.bytesToCapturePerPacket()); + Assertions.assertEquals(5050642493378346293L, model.totalBytesPerSession()); + Assertions.assertEquals(133898645, model.timeLimitInSeconds()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateMockTests.java index f1a48fe110277..f5332a84e79c7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesCreateOrUpdateMockTests.java @@ -31,49 +31,33 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"Stopped\",\"reason\":\"owolbaui\",\"captureStartTime\":\"2021-04-01T11:02:42Z\",\"networkInterfaces\":[\"nszonwpngaj\"],\"bytesToCapturePerPacket\":6898929909926119274,\"totalBytesPerSession\":3652679001065312018,\"timeLimitInSeconds\":1472417276,\"outputFiles\":[\"jfjmyccxlzh\",\"oxovnekhenlusf\",\"rd\",\"jxtxrdc\"]},\"id\":\"jvidttge\",\"name\":\"uslvyjtcvuwkasi\",\"type\":\"iesfuug\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"status\":\"NotStarted\",\"reason\":\"qeqala\",\"captureStartTime\":\"2021-09-27T21:31:23Z\",\"networkInterfaces\":[\"un\",\"tgfebwln\",\"mhyreeudz\",\"av\"],\"bytesToCapturePerPacket\":7319159798879999243,\"totalBytesPerSession\":7257741408349868875,\"timeLimitInSeconds\":1758881694,\"outputFiles\":[\"glgou\"]},\"id\":\"lmjjyuo\",\"name\":\"qtobaxkjeyt\",\"type\":\"nlb\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PacketCapture response = - manager - .packetCaptures() - .define("herngb") - .withExistingPacketCoreControlPlane("taboidvmf", "hppubowsepdfgkmt") - .withNetworkInterfaces(Arrays.asList("sh", "phwpnulaiywzej")) - .withBytesToCapturePerPacket(2237489036169813862L) - .withTotalBytesPerSession(3858468610675521337L) - .withTimeLimitInSeconds(1142499440) - .create(); - - Assertions.assertEquals("nszonwpngaj", response.networkInterfaces().get(0)); - Assertions.assertEquals(6898929909926119274L, response.bytesToCapturePerPacket()); - Assertions.assertEquals(3652679001065312018L, response.totalBytesPerSession()); - Assertions.assertEquals(1472417276, response.timeLimitInSeconds()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PacketCapture response + = manager.packetCaptures().define("g").withExistingPacketCoreControlPlane("npbs", "vefloccsrmozihmi") + .withNetworkInterfaces(Arrays.asList("xgrytfmp", "ycilrmcaykggnox", "ztrksxwpndf")) + .withBytesToCapturePerPacket(6903617454605367538L).withTotalBytesPerSession(8428956603599349243L) + .withTimeLimitInSeconds(1956660642).create(); + + Assertions.assertEquals("un", response.networkInterfaces().get(0)); + Assertions.assertEquals(7319159798879999243L, response.bytesToCapturePerPacket()); + Assertions.assertEquals(7257741408349868875L, response.totalBytesPerSession()); + Assertions.assertEquals(1758881694, response.timeLimitInSeconds()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteMockTests.java index 75097f05b8a9e..1ed72957357d5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.packetCaptures().delete("izvu", "mmkjsvthnwpztek", "vmribiat", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.packetCaptures().delete("ylkkshkbffmbm", "zjrgyww", "gjxsnptfu", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetWithResponseMockTests.java index 4fbc59dc2eee1..12c41582904e1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesGetWithResponseMockTests.java @@ -30,44 +30,30 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Accepted\",\"status\":\"NotStarted\",\"reason\":\"wiwtglxxhl\",\"captureStartTime\":\"2021-02-24T17:36:23Z\",\"networkInterfaces\":[\"icrmnzh\",\"gmqgjs\",\"vpqcb\",\"rmbodt\"],\"bytesToCapturePerPacket\":4761446188291703076,\"totalBytesPerSession\":6402554888202206387,\"timeLimitInSeconds\":26841896,\"outputFiles\":[\"kclacjfrn\"]},\"id\":\"usx\",\"name\":\"uzlwvsgmw\",\"type\":\"hqf\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Deleted\",\"status\":\"Error\",\"reason\":\"ibmikostbzb\",\"captureStartTime\":\"2021-09-02T20:21Z\",\"networkInterfaces\":[\"qnyophzfyls\"],\"bytesToCapturePerPacket\":1557828548687606223,\"totalBytesPerSession\":2804347969541145358,\"timeLimitInSeconds\":1928749436,\"outputFiles\":[\"zcezelfwyfwl\",\"xjwet\",\"psihcla\",\"zvaylptrsqqw\"]},\"id\":\"cmwqkchcxwa\",\"name\":\"fewz\",\"type\":\"kjexfdeqvhp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PacketCapture response = - manager - .packetCaptures() - .getWithResponse("kfvxcnq", "xqpswok", "vkhlggdhbemz", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("icrmnzh", response.networkInterfaces().get(0)); - Assertions.assertEquals(4761446188291703076L, response.bytesToCapturePerPacket()); - Assertions.assertEquals(6402554888202206387L, response.totalBytesPerSession()); - Assertions.assertEquals(26841896, response.timeLimitInSeconds()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PacketCapture response = manager.packetCaptures() + .getWithResponse("zjedmstkvnlv", "bcuiiz", "ktwfa", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("qnyophzfyls", response.networkInterfaces().get(0)); + Assertions.assertEquals(1557828548687606223L, response.bytesToCapturePerPacket()); + Assertions.assertEquals(2804347969541145358L, response.totalBytesPerSession()); + Assertions.assertEquals(1928749436, response.timeLimitInSeconds()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneMockTests.java index 8e0d759caacef..c028bd582f8e7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesListByPacketCoreControlPlaneMockTests.java @@ -31,43 +31,30 @@ public void testListByPacketCoreControlPlane() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\",\"status\":\"Error\",\"reason\":\"wgcloxoebqinji\",\"captureStartTime\":\"2021-03-04T16:18:21Z\",\"networkInterfaces\":[\"ujqlafcbahh\",\"zpofoiyjwpfilk\",\"kkholvdndvia\"],\"bytesToCapturePerPacket\":3026958489306582785,\"totalBytesPerSession\":6998870719021995460,\"timeLimitInSeconds\":626252727,\"outputFiles\":[\"iukyefchnmna\",\"mnxhkxjqirwrweo\"]},\"id\":\"ffifhx\",\"name\":\"rsnewmozqvbubqma\",\"type\":\"hsycxhxzgaz\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"status\":\"Stopped\",\"reason\":\"ejjtbxqmul\",\"captureStartTime\":\"2021-12-05T08:42:23Z\",\"networkInterfaces\":[\"zvners\",\"ycucrwnamikzeb\",\"qbsms\",\"ziqgfuh\"],\"bytesToCapturePerPacket\":6886133591151042666,\"totalBytesPerSession\":7144106503918371496,\"timeLimitInSeconds\":302343226,\"outputFiles\":[\"zznvfbyc\",\"sxjwwixz\",\"umwmxqhnd\",\"noamldsehaohdj\"]},\"id\":\"flzokxco\",\"name\":\"pelnjetag\",\"type\":\"tsxoatftgz\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .packetCaptures() - .listByPacketCoreControlPlane("wjedmurrxxgew", "ktvqylkmqpzoy", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("ujqlafcbahh", response.iterator().next().networkInterfaces().get(0)); - Assertions.assertEquals(3026958489306582785L, response.iterator().next().bytesToCapturePerPacket()); - Assertions.assertEquals(6998870719021995460L, response.iterator().next().totalBytesPerSession()); - Assertions.assertEquals(626252727, response.iterator().next().timeLimitInSeconds()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.packetCaptures().listByPacketCoreControlPlane("tmninw", "izcil", + com.azure.core.util.Context.NONE); + + Assertions.assertEquals("zvners", response.iterator().next().networkInterfaces().get(0)); + Assertions.assertEquals(6886133591151042666L, response.iterator().next().bytesToCapturePerPacket()); + Assertions.assertEquals(7144106503918371496L, response.iterator().next().totalBytesPerSession()); + Assertions.assertEquals(302343226, response.iterator().next().timeLimitInSeconds()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopMockTests.java index 6c8fe5e4201ca..ab9c7e6368e24 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCapturesStopMockTests.java @@ -30,46 +30,33 @@ public void testStop() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"vclx\",\"name\":\"pdkvg\",\"status\":\"abuiy\",\"resourceId\":\"buzphdugn\",\"startTime\":\"2021-09-24T01:09:52Z\",\"endTime\":\"2021-08-29T15:14:45Z\",\"percentComplete\":65.76420280046689,\"properties\":\"datajiuqhibtozi\"}"; + String responseStr + = "{\"id\":\"tixo\",\"name\":\"fqyinljqepqw\",\"status\":\"ixmonstshiyxg\",\"resourceId\":\"lfcldu\",\"startTime\":\"2021-03-30T02:54:32Z\",\"endTime\":\"2021-10-18T23:01:08Z\",\"percentComplete\":18.709508916398665,\"properties\":\"datawcobie\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - AsyncOperationStatus response = - manager - .packetCaptures() - .stop("gplucfotangcfhny", "zcugswvxwlmzqw", "vtxnjmxmcuqud", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("vclx", response.id()); - Assertions.assertEquals("pdkvg", response.name()); - Assertions.assertEquals("abuiy", response.status()); - Assertions.assertEquals("buzphdugn", response.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-24T01:09:52Z"), response.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-29T15:14:45Z"), response.endTime()); - Assertions.assertEquals(65.76420280046689D, response.percentComplete()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + AsyncOperationStatus response = manager.packetCaptures().stop("gicgaaoepttaq", "tdewemxswvruu", "zzjgehkfki", + com.azure.core.util.Context.NONE); + + Assertions.assertEquals("tixo", response.id()); + Assertions.assertEquals("fqyinljqepqw", response.name()); + Assertions.assertEquals("ixmonstshiyxg", response.status()); + Assertions.assertEquals("lfcldu", response.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-30T02:54:32Z"), response.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-18T23:01:08Z"), response.endTime()); + Assertions.assertEquals(18.709508916398665D, response.percentComplete()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneCollectDiagnosticsPackageTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneCollectDiagnosticsPackageTests.java index a5f1a9137696f..517ed0a4a8eb5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneCollectDiagnosticsPackageTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneCollectDiagnosticsPackageTests.java @@ -11,18 +11,17 @@ public final class PacketCoreControlPlaneCollectDiagnosticsPackageTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreControlPlaneCollectDiagnosticsPackage model = - BinaryData - .fromString("{\"storageAccountBlobUrl\":\"tzjuzgwyzmhtxo\"}") + PacketCoreControlPlaneCollectDiagnosticsPackage model + = BinaryData.fromString("{\"storageAccountBlobUrl\":\"kfpbs\"}") .toObject(PacketCoreControlPlaneCollectDiagnosticsPackage.class); - Assertions.assertEquals("tzjuzgwyzmhtxo", model.storageAccountBlobUrl()); + Assertions.assertEquals("kfpbs", model.storageAccountBlobUrl()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreControlPlaneCollectDiagnosticsPackage model = - new PacketCoreControlPlaneCollectDiagnosticsPackage().withStorageAccountBlobUrl("tzjuzgwyzmhtxo"); + PacketCoreControlPlaneCollectDiagnosticsPackage model + = new PacketCoreControlPlaneCollectDiagnosticsPackage().withStorageAccountBlobUrl("kfpbs"); model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneCollectDiagnosticsPackage.class); - Assertions.assertEquals("tzjuzgwyzmhtxo", model.storageAccountBlobUrl()); + Assertions.assertEquals("kfpbs", model.storageAccountBlobUrl()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneInnerTests.java deleted file mode 100644 index 62e8d9b973c13..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneInnerTests.java +++ /dev/null @@ -1,162 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreControlPlaneInner; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackEdgeDeviceResourceId; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackHciClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.ConnectedClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.CustomLocationResourceId; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.HttpsServerCertificate; -import com.azure.resourcemanager.mobilenetwork.models.Installation; -import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; -import com.azure.resourcemanager.mobilenetwork.models.LocalDiagnosticsAccessConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentity; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.mobilenetwork.models.NasRerouteConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PlatformConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import com.azure.resourcemanager.mobilenetwork.models.SignalingConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.SiteResourceId; -import com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class PacketCoreControlPlaneInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - PacketCoreControlPlaneInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Deleting\",\"installation\":{\"desiredState\":\"Uninstalled\",\"state\":\"RollingBack\",\"reinstallRequired\":\"NotRequired\",\"reasons\":[\"NoPacketCoreDataPlane\"],\"operation\":{\"id\":\"bbzoggig\"}},\"sites\":[{\"id\":\"wburvjxxjnspydpt\"},{\"id\":\"oenkouknvudwti\"},{\"id\":\"kbldngkpocipa\"}],\"platform\":{\"type\":\"3P-AZURE-STACK-HCI\",\"azureStackEdgeDevice\":{\"id\":\"o\"},\"azureStackEdgeDevices\":[{\"id\":\"kgjn\"},{\"id\":\"iucgygevqzn\"},{\"id\":\"yp\"},{\"id\":\"rbpizc\"}],\"azureStackHciCluster\":{\"id\":\"qjsdpydnfyhxdeo\"},\"connectedCluster\":{\"id\":\"zi\"},\"customLocation\":{\"id\":\"ifsjttgzfbishcb\"}},\"coreNetworkTechnology\":\"EPC\",\"version\":\"deyeamdphagalpbu\",\"installedVersion\":\"gipwhonowkg\",\"rollbackVersion\":\"wankixzbi\",\"controlPlaneAccessInterface\":{\"name\":\"eputtmrywnuzoqf\",\"ipv4Address\":\"yqzrnkcqvyxlw\",\"ipv4Subnet\":\"lsicohoqqnwv\",\"ipv4Gateway\":\"yav\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"eun\"],\"sku\":\"G0\",\"ueMtu\":375848936,\"localDiagnosticsAccess\":{\"authenticationType\":\"AAD\",\"httpsServerCertificate\":{\"certificateUrl\":\"xzko\",\"provisioning\":{\"state\":\"NotProvisioned\",\"reason\":\"oklyaxuconuq\"}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"fkbey\"},\"eventHub\":{\"id\":\"wrmjmwvvjektc\",\"reportingInterval\":1803945661},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":621929714}},\"interopSettings\":\"datarsffrzpwvlqdqgbi\"},\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"etcktvfcivfsnk\":{\"principalId\":\"1a6a960c-6430-4ee4-876b-d65e7e574635\",\"clientId\":\"89d561cc-4761-425e-8e03-a45bca5d550e\"},\"ctq\":{\"principalId\":\"910a99b8-84ee-436d-b1e8-28a8e8dd9c04\",\"clientId\":\"17b1a85e-8bc1-4025-a802-13c7a50121e3\"},\"bebrjcxerfuwuttt\":{\"principalId\":\"635cac69-5999-41ca-a3b3-aad100487103\",\"clientId\":\"b7a93a18-aa98-4868-a3b7-ebf0822c7c60\"}}},\"location\":\"vjrbirphxepcyvah\",\"tags\":{\"xjvuujqgidokg\":\"jky\",\"sncghkjeszz\":\"ljyoxgvcltb\",\"mxnehmp\":\"bijhtxfvgxbf\"},\"id\":\"ec\",\"name\":\"godebfqkkrbmpu\",\"type\":\"gr\"}") - .toObject(PacketCoreControlPlaneInner.class); - Assertions.assertEquals("vjrbirphxepcyvah", model.location()); - Assertions.assertEquals("jky", model.tags().get("xjvuujqgidokg")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.identity().type()); - Assertions.assertEquals(DesiredInstallationState.UNINSTALLED, model.installation().desiredState()); - Assertions.assertEquals("wburvjxxjnspydpt", model.sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platform().type()); - Assertions.assertEquals("o", model.platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("qjsdpydnfyhxdeo", model.platform().azureStackHciCluster().id()); - Assertions.assertEquals("zi", model.platform().connectedCluster().id()); - Assertions.assertEquals("ifsjttgzfbishcb", model.platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.EPC, model.coreNetworkTechnology()); - Assertions.assertEquals("deyeamdphagalpbu", model.version()); - Assertions.assertEquals("eputtmrywnuzoqf", model.controlPlaneAccessInterface().name()); - Assertions.assertEquals("yqzrnkcqvyxlw", model.controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("lsicohoqqnwv", model.controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("yav", model.controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("eun", model.controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G0, model.sku()); - Assertions.assertEquals(375848936, model.ueMtu()); - Assertions.assertEquals(AuthenticationType.AAD, model.localDiagnosticsAccess().authenticationType()); - Assertions.assertEquals("xzko", model.localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("fkbey", model.diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("wrmjmwvvjektc", model.eventHub().id()); - Assertions.assertEquals(1803945661, model.eventHub().reportingInterval()); - Assertions.assertEquals(621929714, model.signaling().nasReroute().macroMmeGroupId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - PacketCoreControlPlaneInner model = - new PacketCoreControlPlaneInner() - .withLocation("vjrbirphxepcyvah") - .withTags(mapOf("xjvuujqgidokg", "jky", "sncghkjeszz", "ljyoxgvcltb", "mxnehmp", "bijhtxfvgxbf")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "etcktvfcivfsnk", - new UserAssignedIdentity(), - "ctq", - new UserAssignedIdentity(), - "bebrjcxerfuwuttt", - new UserAssignedIdentity()))) - .withInstallation(new Installation().withDesiredState(DesiredInstallationState.UNINSTALLED)) - .withSites( - Arrays - .asList( - new SiteResourceId().withId("wburvjxxjnspydpt"), - new SiteResourceId().withId("oenkouknvudwti"), - new SiteResourceId().withId("kbldngkpocipa"))) - .withPlatform( - new PlatformConfiguration() - .withType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId("o")) - .withAzureStackHciCluster(new AzureStackHciClusterResourceId().withId("qjsdpydnfyhxdeo")) - .withConnectedCluster(new ConnectedClusterResourceId().withId("zi")) - .withCustomLocation(new CustomLocationResourceId().withId("ifsjttgzfbishcb"))) - .withCoreNetworkTechnology(CoreNetworkType.EPC) - .withVersion("deyeamdphagalpbu") - .withControlPlaneAccessInterface( - new InterfaceProperties() - .withName("eputtmrywnuzoqf") - .withIpv4Address("yqzrnkcqvyxlw") - .withIpv4Subnet("lsicohoqqnwv") - .withIpv4Gateway("yav")) - .withControlPlaneAccessVirtualIpv4Addresses(Arrays.asList("eun")) - .withSku(BillingSku.G0) - .withUeMtu(375848936) - .withLocalDiagnosticsAccess( - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.AAD) - .withHttpsServerCertificate(new HttpsServerCertificate().withCertificateUrl("xzko"))) - .withDiagnosticsUpload(new DiagnosticsUploadConfiguration().withStorageAccountContainerUrl("fkbey")) - .withEventHub(new EventHubConfiguration().withId("wrmjmwvvjektc").withReportingInterval(1803945661)) - .withSignaling( - new SignalingConfiguration() - .withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(621929714))) - .withInteropSettings("datarsffrzpwvlqdqgbi"); - model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneInner.class); - Assertions.assertEquals("vjrbirphxepcyvah", model.location()); - Assertions.assertEquals("jky", model.tags().get("xjvuujqgidokg")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.identity().type()); - Assertions.assertEquals(DesiredInstallationState.UNINSTALLED, model.installation().desiredState()); - Assertions.assertEquals("wburvjxxjnspydpt", model.sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platform().type()); - Assertions.assertEquals("o", model.platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("qjsdpydnfyhxdeo", model.platform().azureStackHciCluster().id()); - Assertions.assertEquals("zi", model.platform().connectedCluster().id()); - Assertions.assertEquals("ifsjttgzfbishcb", model.platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.EPC, model.coreNetworkTechnology()); - Assertions.assertEquals("deyeamdphagalpbu", model.version()); - Assertions.assertEquals("eputtmrywnuzoqf", model.controlPlaneAccessInterface().name()); - Assertions.assertEquals("yqzrnkcqvyxlw", model.controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("lsicohoqqnwv", model.controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("yav", model.controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("eun", model.controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G0, model.sku()); - Assertions.assertEquals(375848936, model.ueMtu()); - Assertions.assertEquals(AuthenticationType.AAD, model.localDiagnosticsAccess().authenticationType()); - Assertions.assertEquals("xzko", model.localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("fkbey", model.diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("wrmjmwvvjektc", model.eventHub().id()); - Assertions.assertEquals(1803945661, model.eventHub().reportingInterval()); - Assertions.assertEquals(621929714, model.signaling().nasReroute().macroMmeGroupId()); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneListResultTests.java deleted file mode 100644 index 8330682e2f445..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneListResultTests.java +++ /dev/null @@ -1,181 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreControlPlaneInner; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackEdgeDeviceResourceId; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackHciClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.ConnectedClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.CustomLocationResourceId; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.HttpsServerCertificate; -import com.azure.resourcemanager.mobilenetwork.models.Installation; -import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; -import com.azure.resourcemanager.mobilenetwork.models.LocalDiagnosticsAccessConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentity; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.mobilenetwork.models.NasRerouteConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlaneListResult; -import com.azure.resourcemanager.mobilenetwork.models.PlatformConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import com.azure.resourcemanager.mobilenetwork.models.SignalingConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.SiteResourceId; -import com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class PacketCoreControlPlaneListResultTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - PacketCoreControlPlaneListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"installation\":{\"desiredState\":\"Uninstalled\",\"state\":\"Installed\",\"reinstallRequired\":\"NotRequired\",\"reasons\":[\"NoPacketCoreDataPlane\",\"UserPlaneAccessInterfaceHasChanged\"],\"operation\":{\"id\":\"bmehh\"}},\"sites\":[{\"id\":\"yvjusrtslhsp\"},{\"id\":\"deemao\"},{\"id\":\"mx\"},{\"id\":\"gkvtmelmqkrhah\"}],\"platform\":{\"type\":\"3P-AZURE-STACK-HCI\",\"azureStackEdgeDevice\":{\"id\":\"uahaquhcdhmd\"},\"azureStackEdgeDevices\":[{\"id\":\"aex\"},{\"id\":\"pvfadmwsrcr\"}],\"azureStackHciCluster\":{\"id\":\"xpvgo\"},\"connectedCluster\":{\"id\":\"lf\"},\"customLocation\":{\"id\":\"sgwbnbbeld\"}},\"coreNetworkTechnology\":\"5GC\",\"version\":\"baliourqhakauha\",\"installedVersion\":\"sfwxosowzxc\",\"rollbackVersion\":\"i\",\"controlPlaneAccessInterface\":{\"name\":\"ooxdjebwpuc\",\"ipv4Address\":\"fvovbvmeuecivy\",\"ipv4Subnet\":\"ce\",\"ipv4Gateway\":\"jgjrwjueiotwm\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"tdx\",\"it\"],\"sku\":\"G5\",\"ueMtu\":1725876914,\"localDiagnosticsAccess\":{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"gqwgxhniskxfbkp\",\"provisioning\":{}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"klwndnhjdauwhv\"},\"eventHub\":{\"id\":\"wzbtdhxu\",\"reportingInterval\":872540735},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":751877073}},\"interopSettings\":\"datawuwprzqlv\"},\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"mkh\":{\"principalId\":\"b6f169da-75c7-4ffa-a715-e0ff1dfe86f3\",\"clientId\":\"65b3f096-3213-4257-a5da-fd6f7bb2237e\"}}},\"location\":\"obbc\",\"tags\":{\"ghfg\":\"rtjriplrbpbew\"},\"id\":\"lcgwxzvlvqh\",\"name\":\"kbegibt\",\"type\":\"mxiebw\"}],\"nextLink\":\"loayqcgw\"}") - .toObject(PacketCoreControlPlaneListResult.class); - Assertions.assertEquals("obbc", model.value().get(0).location()); - Assertions.assertEquals("rtjriplrbpbew", model.value().get(0).tags().get("ghfg")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.value().get(0).identity().type()); - Assertions - .assertEquals(DesiredInstallationState.UNINSTALLED, model.value().get(0).installation().desiredState()); - Assertions.assertEquals("yvjusrtslhsp", model.value().get(0).sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.value().get(0).platform().type()); - Assertions.assertEquals("uahaquhcdhmd", model.value().get(0).platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("xpvgo", model.value().get(0).platform().azureStackHciCluster().id()); - Assertions.assertEquals("lf", model.value().get(0).platform().connectedCluster().id()); - Assertions.assertEquals("sgwbnbbeld", model.value().get(0).platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.FIVE_GC, model.value().get(0).coreNetworkTechnology()); - Assertions.assertEquals("baliourqhakauha", model.value().get(0).version()); - Assertions.assertEquals("ooxdjebwpuc", model.value().get(0).controlPlaneAccessInterface().name()); - Assertions.assertEquals("fvovbvmeuecivy", model.value().get(0).controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("ce", model.value().get(0).controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("jgjrwjueiotwm", model.value().get(0).controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("tdx", model.value().get(0).controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G5, model.value().get(0).sku()); - Assertions.assertEquals(1725876914, model.value().get(0).ueMtu()); - Assertions - .assertEquals( - AuthenticationType.PASSWORD, model.value().get(0).localDiagnosticsAccess().authenticationType()); - Assertions - .assertEquals( - "gqwgxhniskxfbkp", - model.value().get(0).localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions - .assertEquals("klwndnhjdauwhv", model.value().get(0).diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("wzbtdhxu", model.value().get(0).eventHub().id()); - Assertions.assertEquals(872540735, model.value().get(0).eventHub().reportingInterval()); - Assertions.assertEquals(751877073, model.value().get(0).signaling().nasReroute().macroMmeGroupId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - PacketCoreControlPlaneListResult model = - new PacketCoreControlPlaneListResult() - .withValue( - Arrays - .asList( - new PacketCoreControlPlaneInner() - .withLocation("obbc") - .withTags(mapOf("ghfg", "rtjriplrbpbew")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf("mkh", new UserAssignedIdentity()))) - .withInstallation( - new Installation().withDesiredState(DesiredInstallationState.UNINSTALLED)) - .withSites( - Arrays - .asList( - new SiteResourceId().withId("yvjusrtslhsp"), - new SiteResourceId().withId("deemao"), - new SiteResourceId().withId("mx"), - new SiteResourceId().withId("gkvtmelmqkrhah"))) - .withPlatform( - new PlatformConfiguration() - .withType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withAzureStackEdgeDevice( - new AzureStackEdgeDeviceResourceId().withId("uahaquhcdhmd")) - .withAzureStackHciCluster(new AzureStackHciClusterResourceId().withId("xpvgo")) - .withConnectedCluster(new ConnectedClusterResourceId().withId("lf")) - .withCustomLocation(new CustomLocationResourceId().withId("sgwbnbbeld"))) - .withCoreNetworkTechnology(CoreNetworkType.FIVE_GC) - .withVersion("baliourqhakauha") - .withControlPlaneAccessInterface( - new InterfaceProperties() - .withName("ooxdjebwpuc") - .withIpv4Address("fvovbvmeuecivy") - .withIpv4Subnet("ce") - .withIpv4Gateway("jgjrwjueiotwm")) - .withControlPlaneAccessVirtualIpv4Addresses(Arrays.asList("tdx", "it")) - .withSku(BillingSku.G5) - .withUeMtu(1725876914) - .withLocalDiagnosticsAccess( - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.PASSWORD) - .withHttpsServerCertificate( - new HttpsServerCertificate().withCertificateUrl("gqwgxhniskxfbkp"))) - .withDiagnosticsUpload( - new DiagnosticsUploadConfiguration() - .withStorageAccountContainerUrl("klwndnhjdauwhv")) - .withEventHub( - new EventHubConfiguration().withId("wzbtdhxu").withReportingInterval(872540735)) - .withSignaling( - new SignalingConfiguration() - .withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(751877073))) - .withInteropSettings("datawuwprzqlv"))); - model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneListResult.class); - Assertions.assertEquals("obbc", model.value().get(0).location()); - Assertions.assertEquals("rtjriplrbpbew", model.value().get(0).tags().get("ghfg")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.value().get(0).identity().type()); - Assertions - .assertEquals(DesiredInstallationState.UNINSTALLED, model.value().get(0).installation().desiredState()); - Assertions.assertEquals("yvjusrtslhsp", model.value().get(0).sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.value().get(0).platform().type()); - Assertions.assertEquals("uahaquhcdhmd", model.value().get(0).platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("xpvgo", model.value().get(0).platform().azureStackHciCluster().id()); - Assertions.assertEquals("lf", model.value().get(0).platform().connectedCluster().id()); - Assertions.assertEquals("sgwbnbbeld", model.value().get(0).platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.FIVE_GC, model.value().get(0).coreNetworkTechnology()); - Assertions.assertEquals("baliourqhakauha", model.value().get(0).version()); - Assertions.assertEquals("ooxdjebwpuc", model.value().get(0).controlPlaneAccessInterface().name()); - Assertions.assertEquals("fvovbvmeuecivy", model.value().get(0).controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("ce", model.value().get(0).controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("jgjrwjueiotwm", model.value().get(0).controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("tdx", model.value().get(0).controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G5, model.value().get(0).sku()); - Assertions.assertEquals(1725876914, model.value().get(0).ueMtu()); - Assertions - .assertEquals( - AuthenticationType.PASSWORD, model.value().get(0).localDiagnosticsAccess().authenticationType()); - Assertions - .assertEquals( - "gqwgxhniskxfbkp", - model.value().get(0).localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions - .assertEquals("klwndnhjdauwhv", model.value().get(0).diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("wzbtdhxu", model.value().get(0).eventHub().id()); - Assertions.assertEquals(872540735, model.value().get(0).eventHub().reportingInterval()); - Assertions.assertEquals(751877073, model.value().get(0).signaling().nasReroute().macroMmeGroupId()); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanePropertiesFormatTests.java deleted file mode 100644 index 2c74cf21d5312..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanePropertiesFormatTests.java +++ /dev/null @@ -1,121 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.mobilenetwork.fluent.models.PacketCoreControlPlanePropertiesFormat; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackEdgeDeviceResourceId; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackHciClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.ConnectedClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.CustomLocationResourceId; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.HttpsServerCertificate; -import com.azure.resourcemanager.mobilenetwork.models.Installation; -import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; -import com.azure.resourcemanager.mobilenetwork.models.LocalDiagnosticsAccessConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.NasRerouteConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PlatformConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import com.azure.resourcemanager.mobilenetwork.models.SignalingConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.SiteResourceId; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class PacketCoreControlPlanePropertiesFormatTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - PacketCoreControlPlanePropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Deleted\",\"installation\":{\"desiredState\":\"Installed\",\"state\":\"Failed\",\"reinstallRequired\":\"NotRequired\",\"reasons\":[\"UserPlaneAccessVirtualIpv4AddressesHasChanged\",\"NoSlices\",\"NoAttachedDataNetworks\"],\"operation\":{\"id\":\"spnqzahmgkb\"}},\"sites\":[{\"id\":\"yydhibnuqqk\"},{\"id\":\"ik\"}],\"platform\":{\"type\":\"3P-AZURE-STACK-HCI\",\"azureStackEdgeDevice\":{\"id\":\"gvtqagnbuynh\"},\"azureStackEdgeDevices\":[{\"id\":\"gmebfsiarbutrcv\"}],\"azureStackHciCluster\":{\"id\":\"azzmhjrunmpxt\"},\"connectedCluster\":{\"id\":\"bh\"},\"customLocation\":{\"id\":\"nlankxmyskpb\"}},\"coreNetworkTechnology\":\"5GC\",\"version\":\"tkcxywnytnrsy\",\"installedVersion\":\"qidybyx\",\"rollbackVersion\":\"fclhaaxdbabphlwr\",\"controlPlaneAccessInterface\":{\"name\":\"fkts\",\"ipv4Address\":\"sucocmnyyazttbtw\",\"ipv4Subnet\":\"qpuedckzywbiex\",\"ipv4Gateway\":\"eyueaxibxujwb\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"almuzyoxaepdkzja\"],\"sku\":\"G10\",\"ueMtu\":1987726952,\"localDiagnosticsAccess\":{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"d\",\"provisioning\":{\"state\":\"NotProvisioned\",\"reason\":\"bniwdj\"}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"zt\"},\"eventHub\":{\"id\":\"bpg\",\"reportingInterval\":1127167490},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":686373242}},\"interopSettings\":\"datazxbzpfzabglc\"}") - .toObject(PacketCoreControlPlanePropertiesFormat.class); - Assertions.assertEquals(DesiredInstallationState.INSTALLED, model.installation().desiredState()); - Assertions.assertEquals("yydhibnuqqk", model.sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platform().type()); - Assertions.assertEquals("gvtqagnbuynh", model.platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("azzmhjrunmpxt", model.platform().azureStackHciCluster().id()); - Assertions.assertEquals("bh", model.platform().connectedCluster().id()); - Assertions.assertEquals("nlankxmyskpb", model.platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.FIVE_GC, model.coreNetworkTechnology()); - Assertions.assertEquals("tkcxywnytnrsy", model.version()); - Assertions.assertEquals("fkts", model.controlPlaneAccessInterface().name()); - Assertions.assertEquals("sucocmnyyazttbtw", model.controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("qpuedckzywbiex", model.controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("eyueaxibxujwb", model.controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("almuzyoxaepdkzja", model.controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G10, model.sku()); - Assertions.assertEquals(1987726952, model.ueMtu()); - Assertions.assertEquals(AuthenticationType.PASSWORD, model.localDiagnosticsAccess().authenticationType()); - Assertions.assertEquals("d", model.localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("zt", model.diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("bpg", model.eventHub().id()); - Assertions.assertEquals(1127167490, model.eventHub().reportingInterval()); - Assertions.assertEquals(686373242, model.signaling().nasReroute().macroMmeGroupId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - PacketCoreControlPlanePropertiesFormat model = - new PacketCoreControlPlanePropertiesFormat() - .withInstallation(new Installation().withDesiredState(DesiredInstallationState.INSTALLED)) - .withSites(Arrays.asList(new SiteResourceId().withId("yydhibnuqqk"), new SiteResourceId().withId("ik"))) - .withPlatform( - new PlatformConfiguration() - .withType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId("gvtqagnbuynh")) - .withAzureStackHciCluster(new AzureStackHciClusterResourceId().withId("azzmhjrunmpxt")) - .withConnectedCluster(new ConnectedClusterResourceId().withId("bh")) - .withCustomLocation(new CustomLocationResourceId().withId("nlankxmyskpb"))) - .withCoreNetworkTechnology(CoreNetworkType.FIVE_GC) - .withVersion("tkcxywnytnrsy") - .withControlPlaneAccessInterface( - new InterfaceProperties() - .withName("fkts") - .withIpv4Address("sucocmnyyazttbtw") - .withIpv4Subnet("qpuedckzywbiex") - .withIpv4Gateway("eyueaxibxujwb")) - .withControlPlaneAccessVirtualIpv4Addresses(Arrays.asList("almuzyoxaepdkzja")) - .withSku(BillingSku.G10) - .withUeMtu(1987726952) - .withLocalDiagnosticsAccess( - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.PASSWORD) - .withHttpsServerCertificate(new HttpsServerCertificate().withCertificateUrl("d"))) - .withDiagnosticsUpload(new DiagnosticsUploadConfiguration().withStorageAccountContainerUrl("zt")) - .withEventHub(new EventHubConfiguration().withId("bpg").withReportingInterval(1127167490)) - .withSignaling( - new SignalingConfiguration() - .withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(686373242))) - .withInteropSettings("datazxbzpfzabglc"); - model = BinaryData.fromObject(model).toObject(PacketCoreControlPlanePropertiesFormat.class); - Assertions.assertEquals(DesiredInstallationState.INSTALLED, model.installation().desiredState()); - Assertions.assertEquals("yydhibnuqqk", model.sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platform().type()); - Assertions.assertEquals("gvtqagnbuynh", model.platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("azzmhjrunmpxt", model.platform().azureStackHciCluster().id()); - Assertions.assertEquals("bh", model.platform().connectedCluster().id()); - Assertions.assertEquals("nlankxmyskpb", model.platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.FIVE_GC, model.coreNetworkTechnology()); - Assertions.assertEquals("tkcxywnytnrsy", model.version()); - Assertions.assertEquals("fkts", model.controlPlaneAccessInterface().name()); - Assertions.assertEquals("sucocmnyyazttbtw", model.controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("qpuedckzywbiex", model.controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("eyueaxibxujwb", model.controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("almuzyoxaepdkzja", model.controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G10, model.sku()); - Assertions.assertEquals(1987726952, model.ueMtu()); - Assertions.assertEquals(AuthenticationType.PASSWORD, model.localDiagnosticsAccess().authenticationType()); - Assertions.assertEquals("d", model.localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("zt", model.diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("bpg", model.eventHub().id()); - Assertions.assertEquals(1127167490, model.eventHub().reportingInterval()); - Assertions.assertEquals(686373242, model.signaling().nasReroute().macroMmeGroupId()); - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneResourceIdTests.java index 5bdb99c685c7a..56d45d88fce34 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneResourceIdTests.java @@ -11,15 +11,15 @@ public final class PacketCoreControlPlaneResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreControlPlaneResourceId model = - BinaryData.fromString("{\"id\":\"rxpdlcgqlsi\"}").toObject(PacketCoreControlPlaneResourceId.class); - Assertions.assertEquals("rxpdlcgqlsi", model.id()); + PacketCoreControlPlaneResourceId model + = BinaryData.fromString("{\"id\":\"jub\"}").toObject(PacketCoreControlPlaneResourceId.class); + Assertions.assertEquals("jub", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreControlPlaneResourceId model = new PacketCoreControlPlaneResourceId().withId("rxpdlcgqlsi"); + PacketCoreControlPlaneResourceId model = new PacketCoreControlPlaneResourceId().withId("jub"); model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneResourceId.class); - Assertions.assertEquals("rxpdlcgqlsi", model.id()); + Assertions.assertEquals("jub", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionInnerTests.java index df0d11494c8ae..ecc1031348ece 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionInnerTests.java @@ -17,53 +17,34 @@ public final class PacketCoreControlPlaneVersionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreControlPlaneVersionInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Deprecated\",\"minimumPlatformSoftwareVersion\":\"pwxqp\",\"maximumPlatformSoftwareVersion\":\"knftguvriuh\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"ayriwwroyqbexrm\",\"maximumPlatformSoftwareVersion\":\"ibycno\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"zvahapjy\",\"maximumPlatformSoftwareVersion\":\"pvgqzcjrvxdjzlm\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"gfhzovawjvzunlut\",\"name\":\"nnprn\",\"type\":\"i\"}") - .toObject(PacketCoreControlPlaneVersionInner.class); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.DEPRECATED, model.platforms().get(0).versionState()); - Assertions.assertEquals("pwxqp", model.platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions.assertEquals("knftguvriuh", model.platforms().get(0).maximumPlatformSoftwareVersion()); + PacketCoreControlPlaneVersionInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Succeeded\",\"platforms\":[{\"platformType\":\"AKS-HCI\",\"versionState\":\"Unknown\",\"minimumPlatformSoftwareVersion\":\"dttouwaboekqvkel\",\"maximumPlatformSoftwareVersion\":\"mvb\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Deprecated\",\"minimumPlatformSoftwareVersion\":\"aalnjixi\",\"maximumPlatformSoftwareVersion\":\"yaw\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"}]},\"id\":\"slyjpkiid\",\"name\":\"yexz\",\"type\":\"eli\"}") + .toObject(PacketCoreControlPlaneVersionInner.class); + Assertions.assertEquals(PlatformType.AKS_HCI, model.platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.UNKNOWN, model.platforms().get(0).versionState()); + Assertions.assertEquals("dttouwaboekqvkel", model.platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("mvb", model.platforms().get(0).maximumPlatformSoftwareVersion()); Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, model.platforms().get(0).recommendedVersion()); - Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, model.platforms().get(0).obsoleteVersion()); + Assertions.assertEquals(ObsoleteVersion.OBSOLETE, model.platforms().get(0).obsoleteVersion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreControlPlaneVersionInner model = - new PacketCoreControlPlaneVersionInner() - .withPlatforms( - Arrays - .asList( - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.DEPRECATED) - .withMinimumPlatformSoftwareVersion("pwxqp") - .withMaximumPlatformSoftwareVersion("knftguvriuh") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.PREVIEW) - .withMinimumPlatformSoftwareVersion("ayriwwroyqbexrm") - .withMaximumPlatformSoftwareVersion("ibycno") - .withRecommendedVersion(RecommendedVersion.RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.ACTIVE) - .withMinimumPlatformSoftwareVersion("zvahapjy") - .withMaximumPlatformSoftwareVersion("pvgqzcjrvxdjzlm") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE))); + PacketCoreControlPlaneVersionInner model = new PacketCoreControlPlaneVersionInner().withPlatforms(Arrays.asList( + new Platform().withPlatformType(PlatformType.AKS_HCI).withVersionState(VersionState.UNKNOWN) + .withMinimumPlatformSoftwareVersion("dttouwaboekqvkel").withMaximumPlatformSoftwareVersion("mvb") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE), + new Platform().withPlatformType(PlatformType.AKS_HCI).withVersionState(VersionState.DEPRECATED) + .withMinimumPlatformSoftwareVersion("aalnjixi").withMaximumPlatformSoftwareVersion("yaw") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE))); model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneVersionInner.class); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.DEPRECATED, model.platforms().get(0).versionState()); - Assertions.assertEquals("pwxqp", model.platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions.assertEquals("knftguvriuh", model.platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(PlatformType.AKS_HCI, model.platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.UNKNOWN, model.platforms().get(0).versionState()); + Assertions.assertEquals("dttouwaboekqvkel", model.platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("mvb", model.platforms().get(0).maximumPlatformSoftwareVersion()); Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, model.platforms().get(0).recommendedVersion()); - Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, model.platforms().get(0).obsoleteVersion()); + Assertions.assertEquals(ObsoleteVersion.OBSOLETE, model.platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionListResultTests.java index e3d7190f3830e..2d78e48ee61f5 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionListResultTests.java @@ -18,115 +18,62 @@ public final class PacketCoreControlPlaneVersionListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreControlPlaneVersionListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Validating\",\"minimumPlatformSoftwareVersion\":\"smv\",\"maximumPlatformSoftwareVersion\":\"wyjsflhhcaalnjix\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"qcslyjpkiid\",\"maximumPlatformSoftwareVersion\":\"exznelixhnr\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"bnxknalaulppg\",\"name\":\"dtpnapnyiropuhp\",\"type\":\"gvpgy\"},{\"properties\":{\"provisioningState\":\"Deleted\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"jvc\",\"maximumPlatformSoftwareVersion\":\"ynqwwncwzzhxgk\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"t\",\"maximumPlatformSoftwareVersion\":\"ellwptfdy\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"pzfqrhhuaoppp\",\"maximumPlatformSoftwareVersion\":\"eqx\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"obgbkdmoizp\",\"maximumPlatformSoftwareVersion\":\"tmgrcfbun\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"hkxbpv\",\"name\":\"ymjhxxjyngudivkr\",\"type\":\"swbxqz\"},{\"properties\":{\"provisioningState\":\"Succeeded\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"dxxiv\",\"maximumPlatformSoftwareVersion\":\"vtcqaqtdo\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Deprecated\",\"minimumPlatformSoftwareVersion\":\"slqb\",\"maximumPlatformSoftwareVersion\":\"fxoblytkb\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"rvrnsvshqjohxc\",\"maximumPlatformSoftwareVersion\":\"bfovasrruvwbhsq\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"}]},\"id\":\"jbi\",\"name\":\"xb\",\"type\":\"ybsrfbjfdtwss\"}],\"nextLink\":\"ftpvjzbexil\"}") - .toObject(PacketCoreControlPlaneVersionListResult.class); - Assertions - .assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.value().get(0).platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.VALIDATING, model.value().get(0).platforms().get(0).versionState()); - Assertions.assertEquals("smv", model.value().get(0).platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions - .assertEquals("wyjsflhhcaalnjix", model.value().get(0).platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions - .assertEquals(RecommendedVersion.RECOMMENDED, model.value().get(0).platforms().get(0).recommendedVersion()); - Assertions - .assertEquals(ObsoleteVersion.NOT_OBSOLETE, model.value().get(0).platforms().get(0).obsoleteVersion()); + PacketCoreControlPlaneVersionListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"moizpos\",\"maximumPlatformSoftwareVersion\":\"grcfb\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Deprecated\",\"minimumPlatformSoftwareVersion\":\"xbpvjymjhx\",\"maximumPlatformSoftwareVersion\":\"yngudivk\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"ValidationFailed\",\"minimumPlatformSoftwareVersion\":\"zjf\",\"maximumPlatformSoftwareVersion\":\"vjfdx\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"t\",\"name\":\"qaqtdoqmcbxvwvxy\",\"type\":\"lqbhsf\"},{\"properties\":{\"provisioningState\":\"Accepted\",\"platforms\":[{\"platformType\":\"AKS-HCI\",\"versionState\":\"Deprecated\",\"minimumPlatformSoftwareVersion\":\"pe\",\"maximumPlatformSoftwareVersion\":\"wfbkrvrns\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"ValidationFailed\",\"minimumPlatformSoftwareVersion\":\"sbfov\",\"maximumPlatformSoftwareVersion\":\"rruvwbhsq\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"ValidationFailed\",\"minimumPlatformSoftwareVersion\":\"xb\",\"maximumPlatformSoftwareVersion\":\"bsrfbj\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"otftpvjzbexilz\",\"name\":\"nfqqnvwp\",\"type\":\"qtaruoujmkcjhwq\"}],\"nextLink\":\"jrybnwjewgdrjer\"}") + .toObject(PacketCoreControlPlaneVersionListResult.class); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, + model.value().get(0).platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.PREVIEW, model.value().get(0).platforms().get(0).versionState()); + Assertions.assertEquals("moizpos", model.value().get(0).platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("grcfb", model.value().get(0).platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, + model.value().get(0).platforms().get(0).recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, + model.value().get(0).platforms().get(0).obsoleteVersion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreControlPlaneVersionListResult model = - new PacketCoreControlPlaneVersionListResult() - .withValue( - Arrays - .asList( - new PacketCoreControlPlaneVersionInner() - .withPlatforms( - Arrays - .asList( - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.VALIDATING) - .withMinimumPlatformSoftwareVersion("smv") - .withMaximumPlatformSoftwareVersion("wyjsflhhcaalnjix") - .withRecommendedVersion(RecommendedVersion.RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), - new Platform() - .withPlatformType(PlatformType.AKS_HCI) - .withVersionState(VersionState.ACTIVE) - .withMinimumPlatformSoftwareVersion("qcslyjpkiid") - .withMaximumPlatformSoftwareVersion("exznelixhnr") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE))), - new PacketCoreControlPlaneVersionInner() - .withPlatforms( - Arrays - .asList( - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.ACTIVE) - .withMinimumPlatformSoftwareVersion("jvc") - .withMaximumPlatformSoftwareVersion("ynqwwncwzzhxgk") - .withRecommendedVersion(RecommendedVersion.RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.ACTIVE) - .withMinimumPlatformSoftwareVersion("t") - .withMaximumPlatformSoftwareVersion("ellwptfdy") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE), - new Platform() - .withPlatformType(PlatformType.AKS_HCI) - .withVersionState(VersionState.ACTIVE) - .withMinimumPlatformSoftwareVersion("pzfqrhhuaoppp") - .withMaximumPlatformSoftwareVersion("eqx") - .withRecommendedVersion(RecommendedVersion.RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), - new Platform() - .withPlatformType(PlatformType.AKS_HCI) - .withVersionState(VersionState.PREVIEW) - .withMinimumPlatformSoftwareVersion("obgbkdmoizp") - .withMaximumPlatformSoftwareVersion("tmgrcfbun") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE))), - new PacketCoreControlPlaneVersionInner() - .withPlatforms( - Arrays - .asList( - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.PREVIEW) - .withMinimumPlatformSoftwareVersion("dxxiv") - .withMaximumPlatformSoftwareVersion("vtcqaqtdo") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE), - new Platform() - .withPlatformType(PlatformType.AKS_HCI) - .withVersionState(VersionState.DEPRECATED) - .withMinimumPlatformSoftwareVersion("slqb") - .withMaximumPlatformSoftwareVersion("fxoblytkb") - .withRecommendedVersion(RecommendedVersion.RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE), - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.PREVIEW) - .withMinimumPlatformSoftwareVersion("rvrnsvshqjohxc") - .withMaximumPlatformSoftwareVersion("bfovasrruvwbhsq") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE))))); + PacketCoreControlPlaneVersionListResult model + = new PacketCoreControlPlaneVersionListResult().withValue(Arrays.asList( + new PacketCoreControlPlaneVersionInner().withPlatforms(Arrays.asList(new Platform() + .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI).withVersionState(VersionState.PREVIEW) + .withMinimumPlatformSoftwareVersion("moizpos").withMaximumPlatformSoftwareVersion("grcfb") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), + new Platform().withPlatformType(PlatformType.AKS_HCI).withVersionState(VersionState.DEPRECATED) + .withMinimumPlatformSoftwareVersion("xbpvjymjhx").withMaximumPlatformSoftwareVersion("yngudivk") + .withRecommendedVersion(RecommendedVersion.RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE), + new Platform().withPlatformType(PlatformType.AKS_HCI) + .withVersionState(VersionState.VALIDATION_FAILED).withMinimumPlatformSoftwareVersion("zjf") + .withMaximumPlatformSoftwareVersion("vjfdx") + .withRecommendedVersion(RecommendedVersion.RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE))), + new PacketCoreControlPlaneVersionInner().withPlatforms(Arrays.asList( + new Platform().withPlatformType(PlatformType.AKS_HCI).withVersionState(VersionState.DEPRECATED) + .withMinimumPlatformSoftwareVersion("pe").withMaximumPlatformSoftwareVersion("wfbkrvrns") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE), + new Platform().withPlatformType(PlatformType.AKS_HCI) + .withVersionState(VersionState.VALIDATION_FAILED).withMinimumPlatformSoftwareVersion("sbfov") + .withMaximumPlatformSoftwareVersion("rruvwbhsq") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), + new Platform().withPlatformType(PlatformType.AKS_HCI) + .withVersionState(VersionState.VALIDATION_FAILED).withMinimumPlatformSoftwareVersion("xb") + .withMaximumPlatformSoftwareVersion("bsrfbj") + .withRecommendedVersion(RecommendedVersion.RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE))))); model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneVersionListResult.class); - Assertions - .assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.value().get(0).platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.VALIDATING, model.value().get(0).platforms().get(0).versionState()); - Assertions.assertEquals("smv", model.value().get(0).platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions - .assertEquals("wyjsflhhcaalnjix", model.value().get(0).platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions - .assertEquals(RecommendedVersion.RECOMMENDED, model.value().get(0).platforms().get(0).recommendedVersion()); - Assertions - .assertEquals(ObsoleteVersion.NOT_OBSOLETE, model.value().get(0).platforms().get(0).obsoleteVersion()); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, + model.value().get(0).platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.PREVIEW, model.value().get(0).platforms().get(0).versionState()); + Assertions.assertEquals("moizpos", model.value().get(0).platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("grcfb", model.value().get(0).platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, + model.value().get(0).platforms().get(0).recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, + model.value().get(0).platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionPropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionPropertiesFormatTests.java index 2f7c195d5b7bf..638eb328b757f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionPropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionPropertiesFormatTests.java @@ -17,39 +17,41 @@ public final class PacketCoreControlPlaneVersionPropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreControlPlaneVersionPropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Deleting\",\"platforms\":[{\"platformType\":\"AKS-HCI\",\"versionState\":\"Validating\",\"minimumPlatformSoftwareVersion\":\"jxdultskzbbtdzu\",\"maximumPlatformSoftwareVersion\":\"e\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"}]}") - .toObject(PacketCoreControlPlaneVersionPropertiesFormat.class); - Assertions.assertEquals(PlatformType.AKS_HCI, model.platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.VALIDATING, model.platforms().get(0).versionState()); - Assertions.assertEquals("jxdultskzbbtdzu", model.platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions.assertEquals("e", model.platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions.assertEquals(RecommendedVersion.RECOMMENDED, model.platforms().get(0).recommendedVersion()); + PacketCoreControlPlaneVersionPropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Deleted\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Unknown\",\"minimumPlatformSoftwareVersion\":\"bnxknalaulppg\",\"maximumPlatformSoftwareVersion\":\"tpnapnyiropuhpig\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Unknown\",\"minimumPlatformSoftwareVersion\":\"xmedjvcsly\",\"maximumPlatformSoftwareVersion\":\"wwncwzzhxgk\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"t\",\"maximumPlatformSoftwareVersion\":\"ellwptfdy\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"}]}") + .toObject(PacketCoreControlPlaneVersionPropertiesFormat.class); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.UNKNOWN, model.platforms().get(0).versionState()); + Assertions.assertEquals("bnxknalaulppg", model.platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("tpnapnyiropuhpig", model.platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, model.platforms().get(0).recommendedVersion()); Assertions.assertEquals(ObsoleteVersion.OBSOLETE, model.platforms().get(0).obsoleteVersion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreControlPlaneVersionPropertiesFormat model = - new PacketCoreControlPlaneVersionPropertiesFormat() - .withPlatforms( - Arrays - .asList( - new Platform() - .withPlatformType(PlatformType.AKS_HCI) - .withVersionState(VersionState.VALIDATING) - .withMinimumPlatformSoftwareVersion("jxdultskzbbtdzu") - .withMaximumPlatformSoftwareVersion("e") - .withRecommendedVersion(RecommendedVersion.RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE))); + PacketCoreControlPlaneVersionPropertiesFormat model + = new PacketCoreControlPlaneVersionPropertiesFormat().withPlatforms(Arrays.asList( + new Platform().withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) + .withVersionState(VersionState.UNKNOWN).withMinimumPlatformSoftwareVersion("bnxknalaulppg") + .withMaximumPlatformSoftwareVersion("tpnapnyiropuhpig") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE), + new Platform().withPlatformType(PlatformType.AKS_HCI).withVersionState(VersionState.UNKNOWN) + .withMinimumPlatformSoftwareVersion("xmedjvcsly").withMaximumPlatformSoftwareVersion("wwncwzzhxgk") + .withRecommendedVersion(RecommendedVersion.RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE), + new Platform().withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) + .withVersionState(VersionState.ACTIVE).withMinimumPlatformSoftwareVersion("t") + .withMaximumPlatformSoftwareVersion("ellwptfdy") + .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) + .withObsoleteVersion(ObsoleteVersion.OBSOLETE))); model = BinaryData.fromObject(model).toObject(PacketCoreControlPlaneVersionPropertiesFormat.class); - Assertions.assertEquals(PlatformType.AKS_HCI, model.platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.VALIDATING, model.platforms().get(0).versionState()); - Assertions.assertEquals("jxdultskzbbtdzu", model.platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions.assertEquals("e", model.platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions.assertEquals(RecommendedVersion.RECOMMENDED, model.platforms().get(0).recommendedVersion()); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.UNKNOWN, model.platforms().get(0).versionState()); + Assertions.assertEquals("bnxknalaulppg", model.platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("tpnapnyiropuhpig", model.platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, model.platforms().get(0).recommendedVersion()); Assertions.assertEquals(ObsoleteVersion.OBSOLETE, model.platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionWithResponseMockTests.java index 0a2309eef0538..2cb65da9f1e2b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetBySubscriptionWithResponseMockTests.java @@ -34,46 +34,32 @@ public void testGetBySubscriptionWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Deleting\",\"platforms\":[{\"platformType\":\"AKS-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"pd\",\"maximumPlatformSoftwareVersion\":\"mkoisqcssf\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Unknown\",\"minimumPlatformSoftwareVersion\":\"obkdqzrdzsyl\",\"maximumPlatformSoftwareVersion\":\"lgtrczzy\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"jpvuaurkihcirld\",\"name\":\"fx\",\"type\":\"dcoxnbk\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Deleting\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Validating\",\"minimumPlatformSoftwareVersion\":\"mgbro\",\"maximumPlatformSoftwareVersion\":\"ddbhf\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"wjxhpdulon\",\"maximumPlatformSoftwareVersion\":\"cnpqwteht\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"ogwxhnsduugwb\",\"maximumPlatformSoftwareVersion\":\"e\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"}]},\"id\":\"uarenlv\",\"name\":\"htkln\",\"type\":\"nafvvkyfedev\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PacketCoreControlPlaneVersion response = - manager - .packetCoreControlPlaneVersions() - .getBySubscriptionWithResponse("jjkhvyomaclu", com.azure.core.util.Context.NONE) - .getValue(); + PacketCoreControlPlaneVersion response = manager.packetCoreControlPlaneVersions() + .getBySubscriptionWithResponse("arfdlpukhpyrnei", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(PlatformType.AKS_HCI, response.platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.PREVIEW, response.platforms().get(0).versionState()); - Assertions.assertEquals("pd", response.platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions.assertEquals("mkoisqcssf", response.platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions.assertEquals(RecommendedVersion.RECOMMENDED, response.platforms().get(0).recommendedVersion()); - Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, response.platforms().get(0).obsoleteVersion()); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, response.platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.VALIDATING, response.platforms().get(0).versionState()); + Assertions.assertEquals("mgbro", response.platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("ddbhf", response.platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, response.platforms().get(0).recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.OBSOLETE, response.platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetWithResponseMockTests.java index b806f1949eb4a..8e8da15b64a32 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsGetWithResponseMockTests.java @@ -34,46 +34,32 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Canceled\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Validating\",\"minimumPlatformSoftwareVersion\":\"rjgeih\",\"maximumPlatformSoftwareVersion\":\"lg\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"ValidationFailed\",\"minimumPlatformSoftwareVersion\":\"pbyephmgtvljvrcm\",\"maximumPlatformSoftwareVersion\":\"qipgx\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Validating\",\"minimumPlatformSoftwareVersion\":\"bvnuile\",\"maximumPlatformSoftwareVersion\":\"aswlp\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"fjlrxwtoauk\",\"name\":\"fkvcisi\",\"type\":\"moaedsxj\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Canceled\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Validating\",\"minimumPlatformSoftwareVersion\":\"bnchrsziz\",\"maximumPlatformSoftwareVersion\":\"uelyetndn\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"agfl\",\"name\":\"lgmtrwahzjmucf\",\"type\":\"byrplrohkpig\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PacketCoreControlPlaneVersion response = - manager - .packetCoreControlPlaneVersions() - .getWithResponse("nssqyzqed", com.azure.core.util.Context.NONE) - .getValue(); + PacketCoreControlPlaneVersion response = manager.packetCoreControlPlaneVersions() + .getWithResponse("hlwntsjgq", com.azure.core.util.Context.NONE).getValue(); Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, response.platforms().get(0).platformType()); Assertions.assertEquals(VersionState.VALIDATING, response.platforms().get(0).versionState()); - Assertions.assertEquals("rjgeih", response.platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions.assertEquals("lg", response.platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, response.platforms().get(0).recommendedVersion()); - Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, response.platforms().get(0).obsoleteVersion()); + Assertions.assertEquals("bnchrsziz", response.platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("uelyetndn", response.platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.RECOMMENDED, response.platforms().get(0).recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.OBSOLETE, response.platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionMockTests.java index b6b749f4f4809..4320ec0253887 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListBySubscriptionMockTests.java @@ -35,49 +35,37 @@ public void testListBySubscription() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"platforms\":[{\"platformType\":\"AKS-HCI\",\"versionState\":\"ValidationFailed\",\"minimumPlatformSoftwareVersion\":\"nqbpi\",\"maximumPlatformSoftwareVersion\":\"qltgrd\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"Obsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"ihwuhvctafsrbxrb\",\"maximumPlatformSoftwareVersion\":\"li\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"qagnepzwakl\",\"maximumPlatformSoftwareVersion\":\"sbq\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"}]},\"id\":\"rxaomzisglrrcze\",\"name\":\"k\",\"type\":\"hltnjadhqoawjq\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"okkhminq\",\"maximumPlatformSoftwareVersion\":\"mczngn\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"AKS-HCI\",\"versionState\":\"ValidationFailed\",\"minimumPlatformSoftwareVersion\":\"vudb\",\"maximumPlatformSoftwareVersion\":\"aqdtvqecrqctmxxd\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"xzvtzna\",\"maximumPlatformSoftwareVersion\":\"bannovvoxczytp\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"}]},\"id\":\"oevytlyokr\",\"name\":\"rouuxvnsasbcry\",\"type\":\"o\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.packetCoreControlPlaneVersions().listBySubscription(com.azure.core.util.Context.NONE); + PagedIterable response + = manager.packetCoreControlPlaneVersions().listBySubscription(com.azure.core.util.Context.NONE); - Assertions.assertEquals(PlatformType.AKS_HCI, response.iterator().next().platforms().get(0).platformType()); - Assertions - .assertEquals(VersionState.VALIDATION_FAILED, response.iterator().next().platforms().get(0).versionState()); - Assertions - .assertEquals("nqbpi", response.iterator().next().platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions - .assertEquals("qltgrd", response.iterator().next().platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions - .assertEquals( - RecommendedVersion.RECOMMENDED, response.iterator().next().platforms().get(0).recommendedVersion()); - Assertions - .assertEquals(ObsoleteVersion.OBSOLETE, response.iterator().next().platforms().get(0).obsoleteVersion()); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, + response.iterator().next().platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.PREVIEW, response.iterator().next().platforms().get(0).versionState()); + Assertions.assertEquals("okkhminq", + response.iterator().next().platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("mczngn", + response.iterator().next().platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.RECOMMENDED, + response.iterator().next().platforms().get(0).recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, + response.iterator().next().platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListMockTests.java index b9f49527f7bc0..6d149534a388d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlaneVersionsListMockTests.java @@ -35,52 +35,37 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Unknown\",\"minimumPlatformSoftwareVersion\":\"eewxeiqbpsmg\",\"maximumPlatformSoftwareVersion\":\"guamlj\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"},{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Preview\",\"minimumPlatformSoftwareVersion\":\"auf\",\"maximumPlatformSoftwareVersion\":\"hhvnewgnxkymp\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"}]},\"id\":\"kixtwbtaoy\",\"name\":\"nyghshxcylhkgmn\",\"type\":\"ghpxycphdr\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\",\"platforms\":[{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Deprecated\",\"minimumPlatformSoftwareVersion\":\"wklsnoxaxmqe\",\"maximumPlatformSoftwareVersion\":\"lhhjnh\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"NotObsolete\"}]},\"id\":\"nfsvk\",\"name\":\"gbv\",\"type\":\"ta\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.packetCoreControlPlaneVersions().list(com.azure.core.util.Context.NONE); + PagedIterable response + = manager.packetCoreControlPlaneVersions().list(com.azure.core.util.Context.NONE); - Assertions - .assertEquals( - PlatformType.THREE_P_AZURE_STACK_HCI, response.iterator().next().platforms().get(0).platformType()); - Assertions.assertEquals(VersionState.UNKNOWN, response.iterator().next().platforms().get(0).versionState()); - Assertions - .assertEquals( - "eewxeiqbpsmg", response.iterator().next().platforms().get(0).minimumPlatformSoftwareVersion()); - Assertions - .assertEquals("guamlj", response.iterator().next().platforms().get(0).maximumPlatformSoftwareVersion()); - Assertions - .assertEquals( - RecommendedVersion.NOT_RECOMMENDED, response.iterator().next().platforms().get(0).recommendedVersion()); - Assertions - .assertEquals( - ObsoleteVersion.NOT_OBSOLETE, response.iterator().next().platforms().get(0).obsoleteVersion()); + Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, + response.iterator().next().platforms().get(0).platformType()); + Assertions.assertEquals(VersionState.DEPRECATED, response.iterator().next().platforms().get(0).versionState()); + Assertions.assertEquals("wklsnoxaxmqe", + response.iterator().next().platforms().get(0).minimumPlatformSoftwareVersion()); + Assertions.assertEquals("lhhjnh", + response.iterator().next().platforms().get(0).maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, + response.iterator().next().platforms().get(0).recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, + response.iterator().next().platforms().get(0).obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageMockTests.java index 70c5f1ab36019..333d36be66fd6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCollectDiagnosticsPackageMockTests.java @@ -31,50 +31,34 @@ public void testCollectDiagnosticsPackage() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"kujrllfojui\",\"name\":\"puuyjucejik\",\"status\":\"oeo\",\"resourceId\":\"tzejetjklnt\",\"startTime\":\"2021-05-29T06:35:10Z\",\"endTime\":\"2021-10-12T06:41:45Z\",\"percentComplete\":4.448023219350572,\"properties\":\"dataqzolxrzvhqjw\"}"; + String responseStr + = "{\"id\":\"ohzjqatucoigeb\",\"name\":\"cnwfepbnwgfmxjg\",\"status\":\"g\",\"resourceId\":\"bgdlfgtdysna\",\"startTime\":\"2021-11-27T06:47:30Z\",\"endTime\":\"2021-11-21T00:03:39Z\",\"percentComplete\":69.11632649551352,\"properties\":\"datahamzjrwdkqze\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - AsyncOperationStatus response = - manager - .packetCoreControlPlanes() - .collectDiagnosticsPackage( - "w", - "dqlvhukoveof", - new PacketCoreControlPlaneCollectDiagnosticsPackage().withStorageAccountBlobUrl("zrvjfnmjmvlwyzgi"), - com.azure.core.util.Context.NONE); + AsyncOperationStatus response = manager.packetCoreControlPlanes().collectDiagnosticsPackage("jlxuz", "hwpusxj", + new PacketCoreControlPlaneCollectDiagnosticsPackage().withStorageAccountBlobUrl("aqehg"), + com.azure.core.util.Context.NONE); - Assertions.assertEquals("kujrllfojui", response.id()); - Assertions.assertEquals("puuyjucejik", response.name()); - Assertions.assertEquals("oeo", response.status()); - Assertions.assertEquals("tzejetjklnt", response.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-29T06:35:10Z"), response.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-12T06:41:45Z"), response.endTime()); - Assertions.assertEquals(4.448023219350572D, response.percentComplete()); + Assertions.assertEquals("ohzjqatucoigeb", response.id()); + Assertions.assertEquals("cnwfepbnwgfmxjg", response.name()); + Assertions.assertEquals("g", response.status()); + Assertions.assertEquals("bgdlfgtdysna", response.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-27T06:47:30Z"), response.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-11-21T00:03:39Z"), response.endTime()); + Assertions.assertEquals(69.11632649551352D, response.percentComplete()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateMockTests.java deleted file mode 100644 index ea1d619edfc87..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesCreateOrUpdateMockTests.java +++ /dev/null @@ -1,173 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.mobilenetwork.MobileNetworkManager; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackEdgeDeviceResourceId; -import com.azure.resourcemanager.mobilenetwork.models.AzureStackHciClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.ConnectedClusterResourceId; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.CustomLocationResourceId; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.DiagnosticsUploadConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.EventHubConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.HttpsServerCertificate; -import com.azure.resourcemanager.mobilenetwork.models.Installation; -import com.azure.resourcemanager.mobilenetwork.models.InterfaceProperties; -import com.azure.resourcemanager.mobilenetwork.models.LocalDiagnosticsAccessConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentity; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.mobilenetwork.models.NasRerouteConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlane; -import com.azure.resourcemanager.mobilenetwork.models.PlatformConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import com.azure.resourcemanager.mobilenetwork.models.SignalingConfiguration; -import com.azure.resourcemanager.mobilenetwork.models.SiteResourceId; -import com.azure.resourcemanager.mobilenetwork.models.UserAssignedIdentity; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class PacketCoreControlPlanesCreateOrUpdateMockTests { - @Test - public void testCreateOrUpdate() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"installation\":{\"desiredState\":\"Uninstalled\",\"state\":\"RollingBack\",\"reinstallRequired\":\"Required\",\"reasons\":[\"UserPlaneDataInterfaceHasChanged\",\"NoAttachedDataNetworks\"],\"operation\":{\"id\":\"im\"}},\"sites\":[{\"id\":\"zovgkkumuikj\"},{\"id\":\"jcazt\"}],\"platform\":{\"type\":\"AKS-HCI\",\"azureStackEdgeDevice\":{\"id\":\"nsq\"},\"azureStackEdgeDevices\":[{\"id\":\"wcoml\"}],\"azureStackHciCluster\":{\"id\":\"ytwvczcswkacve\"},\"connectedCluster\":{\"id\":\"fdv\"},\"customLocation\":{\"id\":\"hbwrnfxtgddpqth\"}},\"coreNetworkTechnology\":\"5GC\",\"version\":\"naoyank\",\"installedVersion\":\"eqswanklty\",\"rollbackVersion\":\"hdroznnh\",\"controlPlaneAccessInterface\":{\"name\":\"lktgjc\",\"ipv4Address\":\"guxhem\",\"ipv4Subnet\":\"ywaeeczgf\",\"ipv4Gateway\":\"kklelssxblycs\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"jks\",\"lsmdesqplpvmjc\",\"oewbid\",\"vteo\"],\"sku\":\"G1\",\"ueMtu\":2010112978,\"localDiagnosticsAccess\":{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"udeugfsxz\",\"provisioning\":{\"state\":\"NotProvisioned\",\"reason\":\"wkufykhvuhxepm\"}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"tznabao\"},\"eventHub\":{\"id\":\"slujdjltym\",\"reportingInterval\":101256689},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":778668452}},\"interopSettings\":\"dataywart\"},\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"ky\":{\"principalId\":\"42df32a6-1a25-4857-bcde-898855f270c3\",\"clientId\":\"eaf83d74-0265-4997-9b6e-0ac3b96edceb\"},\"ssjpemmucf\":{\"principalId\":\"5e56cb7f-d07b-457b-af59-1ec9cf685728\",\"clientId\":\"9fec39e1-2aa2-4e98-ab4b-cf92a32090df\"},\"kkflrmymyincqlhr\":{\"principalId\":\"139b3838-81c9-4388-9318-291f2d1799bb\",\"clientId\":\"7cb851c4-3d24-4b4d-8b65-4fdeb1e13153\"},\"sl\":{\"principalId\":\"eae513a5-9c69-4b1d-996d-59f0d91dfa35\",\"clientId\":\"a4a69244-1cb3-41ae-b21f-e056ab9726ea\"}}},\"location\":\"iiovgqcgxu\",\"tags\":{\"ptjgwdt\":\"kctotiowlxteq\",\"hqlkccuzgygqwaho\":\"ukranblw\",\"d\":\"ulwgniiprglvawuw\",\"sbbjpmcu\":\"fypiv\"},\"id\":\"kmifoxxkub\",\"name\":\"phavpmhbrb\",\"type\":\"gvgovpbbttefjo\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PacketCoreControlPlane response = - manager - .packetCoreControlPlanes() - .define("zpcrrkolawj") - .withRegion("ljqobbpihehcecyb") - .withExistingResourceGroup("htgv") - .withSites( - Arrays - .asList( - new SiteResourceId().withId("hguynuchlgmltxdw"), - new SiteResourceId().withId("mozusgzvlnsnnjz"))) - .withPlatform( - new PlatformConfiguration() - .withType(PlatformType.AKS_HCI) - .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId("folpymwamxqzra")) - .withAzureStackHciCluster(new AzureStackHciClusterResourceId().withId("zlanrupdwvnph")) - .withConnectedCluster(new ConnectedClusterResourceId().withId("zqtpjhmq")) - .withCustomLocation(new CustomLocationResourceId().withId("v"))) - .withControlPlaneAccessInterface( - new InterfaceProperties() - .withName("lbyulidwcwvmze") - .withIpv4Address("o") - .withIpv4Subnet("hj") - .withIpv4Gateway("wgdnqzbr")) - .withSku(BillingSku.G0) - .withLocalDiagnosticsAccess( - new LocalDiagnosticsAccessConfiguration() - .withAuthenticationType(AuthenticationType.AAD) - .withHttpsServerCertificate(new HttpsServerCertificate().withCertificateUrl("chtomflrytsw"))) - .withTags(mapOf("frexcrseqw", "brjbbmpxdlvy")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf("nu", new UserAssignedIdentity()))) - .withInstallation(new Installation().withDesiredState(DesiredInstallationState.INSTALLED)) - .withCoreNetworkTechnology(CoreNetworkType.EPC_5GC) - .withVersion("iwdcxsmlzzhzd") - .withControlPlaneAccessVirtualIpv4Addresses(Arrays.asList("pzhz", "tk", "jcitdigsxcdglj")) - .withUeMtu(409338172) - .withDiagnosticsUpload(new DiagnosticsUploadConfiguration().withStorageAccountContainerUrl("rlpshhkv")) - .withEventHub(new EventHubConfiguration().withId("dwqslsrhmpqvw").withReportingInterval(1592039992)) - .withSignaling( - new SignalingConfiguration() - .withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(375889283))) - .withInteropSettings("databrwimuvqejosovyr") - .create(); - - Assertions.assertEquals("iiovgqcgxu", response.location()); - Assertions.assertEquals("kctotiowlxteq", response.tags().get("ptjgwdt")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.identity().type()); - Assertions.assertEquals(DesiredInstallationState.UNINSTALLED, response.installation().desiredState()); - Assertions.assertEquals("zovgkkumuikj", response.sites().get(0).id()); - Assertions.assertEquals(PlatformType.AKS_HCI, response.platform().type()); - Assertions.assertEquals("nsq", response.platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("ytwvczcswkacve", response.platform().azureStackHciCluster().id()); - Assertions.assertEquals("fdv", response.platform().connectedCluster().id()); - Assertions.assertEquals("hbwrnfxtgddpqth", response.platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.FIVE_GC, response.coreNetworkTechnology()); - Assertions.assertEquals("naoyank", response.version()); - Assertions.assertEquals("lktgjc", response.controlPlaneAccessInterface().name()); - Assertions.assertEquals("guxhem", response.controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("ywaeeczgf", response.controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("kklelssxblycs", response.controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("jks", response.controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G1, response.sku()); - Assertions.assertEquals(2010112978, response.ueMtu()); - Assertions.assertEquals(AuthenticationType.PASSWORD, response.localDiagnosticsAccess().authenticationType()); - Assertions - .assertEquals("udeugfsxz", response.localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("tznabao", response.diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("slujdjltym", response.eventHub().id()); - Assertions.assertEquals(101256689, response.eventHub().reportingInterval()); - Assertions.assertEquals(778668452, response.signaling().nasReroute().macroMmeGroupId()); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteMockTests.java index 56fd6504af953..f24ef0731e34f 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.packetCoreControlPlanes().delete("tuqfecjxeygtu", "xu", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.packetCoreControlPlanes().delete("jkwrusnkq", "hsyrqunj", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupWithResponseMockTests.java deleted file mode 100644 index b16117aba2c30..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesGetByResourceGroupWithResponseMockTests.java +++ /dev/null @@ -1,101 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.mobilenetwork.MobileNetworkManager; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlane; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class PacketCoreControlPlanesGetByResourceGroupWithResponseMockTests { - @Test - public void testGetByResourceGroupWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"provisioningState\":\"Deleting\",\"installation\":{\"desiredState\":\"Uninstalled\",\"state\":\"Failed\",\"reinstallRequired\":\"NotRequired\",\"reasons\":[\"PublicLandMobileNetworkIdentifierHasChanged\",\"UserPlaneDataInterfaceHasChanged\",\"PublicLandMobileNetworkIdentifierHasChanged\",\"UserPlaneAccessInterfaceHasChanged\"],\"operation\":{\"id\":\"o\"}},\"sites\":[{\"id\":\"e\"}],\"platform\":{\"type\":\"AKS-HCI\",\"azureStackEdgeDevice\":{\"id\":\"cnwfepbnwgfmxjg\"},\"azureStackEdgeDevices\":[{\"id\":\"jbgdlfgtdysnaquf\"}],\"azureStackHciCluster\":{\"id\":\"bctqhamzjrwd\"},\"connectedCluster\":{\"id\":\"zeqyjleziun\"},\"customLocation\":{\"id\":\"dfzantkwcegy\"}},\"coreNetworkTechnology\":\"EPC" - + " + 5GC\",\"version\":\"nseqacjjvp\",\"installedVersion\":\"guooqjagmdit\",\"rollbackVersion\":\"eiookjbsah\",\"controlPlaneAccessInterface\":{\"name\":\"dt\",\"ipv4Address\":\"elqacslmot\",\"ipv4Subnet\":\"bnfxofvc\",\"ipv4Gateway\":\"gdirazf\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"jwabmd\",\"jtmvc\",\"pexcmjurbuhh\",\"kyqltqsrogt\"],\"sku\":\"G5\",\"ueMtu\":2102252305,\"localDiagnosticsAccess\":{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"jk\",\"provisioning\":{\"state\":\"Provisioned\",\"reason\":\"dfvclglxnfu\"}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"tkbu\"},\"eventHub\":{\"id\":\"ogsf\",\"reportingInterval\":1510144840},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":1717573357}},\"interopSettings\":\"datasharujtj\"},\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"qttv\":{\"principalId\":\"6ec2415b-3db2-4eea-a430-87e774f21137\",\"clientId\":\"50f50c1a-4d12-4e78-ae85-0665d3c64fa1\"}}},\"location\":\"pqhjpenuygbqeqq\",\"tags\":{\"qvcdlguaucmfdjw\":\"wv\",\"ikczvvitacgxmf\":\"laxpunj\",\"sjgqrsxyp\":\"sserxhtvsoxhlwn\"},\"id\":\"uuuybnchrsziz\",\"name\":\"yuel\",\"type\":\"etndnbfqyggagf\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PacketCoreControlPlane response = - manager - .packetCoreControlPlanes() - .getByResourceGroupWithResponse("cbuewmrswnjlxuz", "hwpusxj", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("pqhjpenuygbqeqq", response.location()); - Assertions.assertEquals("wv", response.tags().get("qvcdlguaucmfdjw")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.identity().type()); - Assertions.assertEquals(DesiredInstallationState.UNINSTALLED, response.installation().desiredState()); - Assertions.assertEquals("e", response.sites().get(0).id()); - Assertions.assertEquals(PlatformType.AKS_HCI, response.platform().type()); - Assertions.assertEquals("cnwfepbnwgfmxjg", response.platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("bctqhamzjrwd", response.platform().azureStackHciCluster().id()); - Assertions.assertEquals("zeqyjleziun", response.platform().connectedCluster().id()); - Assertions.assertEquals("dfzantkwcegy", response.platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.EPC_5GC, response.coreNetworkTechnology()); - Assertions.assertEquals("nseqacjjvp", response.version()); - Assertions.assertEquals("dt", response.controlPlaneAccessInterface().name()); - Assertions.assertEquals("elqacslmot", response.controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("bnfxofvc", response.controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("gdirazf", response.controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("jwabmd", response.controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G5, response.sku()); - Assertions.assertEquals(2102252305, response.ueMtu()); - Assertions.assertEquals(AuthenticationType.PASSWORD, response.localDiagnosticsAccess().authenticationType()); - Assertions.assertEquals("jk", response.localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("tkbu", response.diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("ogsf", response.eventHub().id()); - Assertions.assertEquals(1510144840, response.eventHub().reportingInterval()); - Assertions.assertEquals(1717573357, response.signaling().nasReroute().macroMmeGroupId()); - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupMockTests.java deleted file mode 100644 index c23c32c05d1ec..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListByResourceGroupMockTests.java +++ /dev/null @@ -1,106 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.mobilenetwork.MobileNetworkManager; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlane; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class PacketCoreControlPlanesListByResourceGroupMockTests { - @Test - public void testListByResourceGroup() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"installation\":{\"desiredState\":\"Installed\",\"state\":\"Updating\",\"reinstallRequired\":\"NotRequired\",\"reasons\":[\"UserPlaneAccessVirtualIpv4AddressesHasChanged\",\"UserPlaneAccessInterfaceHasChanged\"],\"operation\":{\"id\":\"m\"}},\"sites\":[{\"id\":\"f\"},{\"id\":\"lfmu\"},{\"id\":\"apckccrrvw\"},{\"id\":\"yoxoy\"}],\"platform\":{\"type\":\"3P-AZURE-STACK-HCI\",\"azureStackEdgeDevice\":{\"id\":\"phaimmoi\"},\"azureStackEdgeDevices\":[{\"id\":\"boshbragapyyrmfs\"},{\"id\":\"bpav\"}],\"azureStackHciCluster\":{\"id\":\"pfppd\"},\"connectedCluster\":{\"id\":\"nupgahxku\"},\"customLocation\":{\"id\":\"sjcaacfdmmcpugm\"}},\"coreNetworkTechnology\":\"5GC\",\"version\":\"pvufhbze\",\"installedVersion\":\"whoqhnlbqnbldxe\",\"rollbackVersion\":\"lgsc\",\"controlPlaneAccessInterface\":{\"name\":\"ri\",\"ipv4Address\":\"rsrrmoucsofldp\",\"ipv4Subnet\":\"iyfc\",\"ipv4Gateway\":\"beolh\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"vbmxuqibsx\",\"kcudfbsfarfsiowl\",\"jxnqp\",\"wgfstmhqykizm\"],\"sku\":\"G2\",\"ueMtu\":930902009,\"localDiagnosticsAccess\":{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"fcluqvo\",\"provisioning\":{\"state\":\"NotProvisioned\",\"reason\":\"im\"}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"vwg\"},\"eventHub\":{\"id\":\"wpbmzgwesydsxwef\",\"reportingInterval\":799767919},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":1456843419}},\"interopSettings\":\"datapwndyqlea\"},\"identity\":{\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"hlowkx\":{\"principalId\":\"ac27ebc1-d10f-4d58-a29b-db568d53c45c\",\"clientId\":\"762b8217-0501-42d4-9ecb-13152373f1ba\"},\"br\":{\"principalId\":\"6b8d85a4-7927-4662-8133-9fabb93a94d3\",\"clientId\":\"ab790f53-4860-4509-92c0-8028300e5ffb\"},\"mzsyzfhotl\":{\"principalId\":\"924dcd90-ed35-4071-b3a3-9ecf42cfe02c\",\"clientId\":\"aa69bca4-16ec-41b3-9ee1-7562a833e71a\"},\"cyychunsjlp\":{\"principalId\":\"4612a64e-10da-4180-acbe-28f0b4d7a4b0\",\"clientId\":\"4efb5923-42b6-4033-8ead-a506433de7cf\"}}},\"location\":\"twszhvvuic\",\"tags\":{\"dpyflubhv\":\"trrmhwrb\",\"lw\":\"glrocuy\",\"ooclutnp\":\"hmem\",\"ujxsglhsr\":\"memczjkmmyk\"},\"id\":\"rye\",\"name\":\"ylmbkzudni\",\"type\":\"rfih\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.packetCoreControlPlanes().listByResourceGroup("zb", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("twszhvvuic", response.iterator().next().location()); - Assertions.assertEquals("trrmhwrb", response.iterator().next().tags().get("dpyflubhv")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.iterator().next().identity().type()); - Assertions - .assertEquals(DesiredInstallationState.INSTALLED, response.iterator().next().installation().desiredState()); - Assertions.assertEquals("f", response.iterator().next().sites().get(0).id()); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, response.iterator().next().platform().type()); - Assertions.assertEquals("phaimmoi", response.iterator().next().platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("pfppd", response.iterator().next().platform().azureStackHciCluster().id()); - Assertions.assertEquals("nupgahxku", response.iterator().next().platform().connectedCluster().id()); - Assertions.assertEquals("sjcaacfdmmcpugm", response.iterator().next().platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.FIVE_GC, response.iterator().next().coreNetworkTechnology()); - Assertions.assertEquals("pvufhbze", response.iterator().next().version()); - Assertions.assertEquals("ri", response.iterator().next().controlPlaneAccessInterface().name()); - Assertions - .assertEquals("rsrrmoucsofldp", response.iterator().next().controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("iyfc", response.iterator().next().controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("beolh", response.iterator().next().controlPlaneAccessInterface().ipv4Gateway()); - Assertions - .assertEquals("vbmxuqibsx", response.iterator().next().controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G2, response.iterator().next().sku()); - Assertions.assertEquals(930902009, response.iterator().next().ueMtu()); - Assertions - .assertEquals( - AuthenticationType.PASSWORD, response.iterator().next().localDiagnosticsAccess().authenticationType()); - Assertions - .assertEquals( - "fcluqvo", - response.iterator().next().localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions.assertEquals("vwg", response.iterator().next().diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("wpbmzgwesydsxwef", response.iterator().next().eventHub().id()); - Assertions.assertEquals(799767919, response.iterator().next().eventHub().reportingInterval()); - Assertions.assertEquals(1456843419, response.iterator().next().signaling().nasReroute().macroMmeGroupId()); - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListMockTests.java deleted file mode 100644 index 68f3b22538d1c..0000000000000 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesListMockTests.java +++ /dev/null @@ -1,105 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.mobilenetwork.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.mobilenetwork.MobileNetworkManager; -import com.azure.resourcemanager.mobilenetwork.models.AuthenticationType; -import com.azure.resourcemanager.mobilenetwork.models.BillingSku; -import com.azure.resourcemanager.mobilenetwork.models.CoreNetworkType; -import com.azure.resourcemanager.mobilenetwork.models.DesiredInstallationState; -import com.azure.resourcemanager.mobilenetwork.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.mobilenetwork.models.PacketCoreControlPlane; -import com.azure.resourcemanager.mobilenetwork.models.PlatformType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class PacketCoreControlPlanesListMockTests { - @Test - public void testList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"installation\":{\"desiredState\":\"Uninstalled\",\"state\":\"Uninstalling\",\"reinstallRequired\":\"NotRequired\",\"reasons\":[\"ControlPlaneAccessInterfaceHasChanged\",\"PublicLandMobileNetworkIdentifierHasChanged\"],\"operation\":{\"id\":\"ftbyrplro\"}},\"sites\":[{\"id\":\"pigqfusuckzmkw\"},{\"id\":\"lsnoxaxmqeqalh\"},{\"id\":\"jnhgwydyyn\"}],\"platform\":{\"type\":\"AKS-HCI\",\"azureStackEdgeDevice\":{\"id\":\"khgb\"},\"azureStackEdgeDevices\":[{\"id\":\"anarfdlpukhpyrne\"}],\"azureStackHciCluster\":{\"id\":\"jcpeogkhnmg\"},\"connectedCluster\":{\"id\":\"ouxddbhfhpfpazj\"},\"customLocation\":{\"id\":\"ywjxh\"}},\"coreNetworkTechnology\":\"EPC\",\"version\":\"ontacnpq\",\"installedVersion\":\"ehtuevrhr\",\"rollbackVersion\":\"yoogw\",\"controlPlaneAccessInterface\":{\"name\":\"nsduugwbsre\",\"ipv4Address\":\"fqkfuarenl\",\"ipv4Subnet\":\"htkln\",\"ipv4Gateway\":\"afvvk\"},\"controlPlaneAccessVirtualIpv4Addresses\":[\"devjb\"],\"sku\":\"G5\",\"ueMtu\":1265182150,\"localDiagnosticsAccess\":{\"authenticationType\":\"Password\",\"httpsServerCertificate\":{\"certificateUrl\":\"y\",\"provisioning\":{\"state\":\"Failed\",\"reason\":\"minqcym\"}}},\"diagnosticsUpload\":{\"storageAccountContainerUrl\":\"ngnbdxxew\"},\"eventHub\":{\"id\":\"invudbch\",\"reportingInterval\":1149033945},\"signaling\":{\"nasReroute\":{\"macroMmeGroupId\":529081403}},\"interopSettings\":\"datacrqctmxxdtddmflh\"},\"identity\":{\"type\":\"None\",\"userAssignedIdentities\":{\"zna\":{\"principalId\":\"7c4bab26-15ba-4f97-9865-71054339e390\",\"clientId\":\"53e38b90-0915-4c3e-bf75-45c7231ff445\"}}},\"location\":\"bannovvoxczytp\",\"tags\":{\"uuxvnsasbcry\":\"wvroevytlyokrrr\",\"izrxklob\":\"o\",\"vevfxz\":\"xnazpmkml\",\"bzxliohrdddtfgxq\":\"pj\"},\"id\":\"awpcbbnzqcykn\",\"name\":\"p\",\"type\":\"ofyuicd\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.packetCoreControlPlanes().list(com.azure.core.util.Context.NONE); - - Assertions.assertEquals("bannovvoxczytp", response.iterator().next().location()); - Assertions.assertEquals("wvroevytlyokrrr", response.iterator().next().tags().get("uuxvnsasbcry")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.iterator().next().identity().type()); - Assertions - .assertEquals( - DesiredInstallationState.UNINSTALLED, response.iterator().next().installation().desiredState()); - Assertions.assertEquals("pigqfusuckzmkw", response.iterator().next().sites().get(0).id()); - Assertions.assertEquals(PlatformType.AKS_HCI, response.iterator().next().platform().type()); - Assertions.assertEquals("khgb", response.iterator().next().platform().azureStackEdgeDevice().id()); - Assertions.assertEquals("jcpeogkhnmg", response.iterator().next().platform().azureStackHciCluster().id()); - Assertions.assertEquals("ouxddbhfhpfpazj", response.iterator().next().platform().connectedCluster().id()); - Assertions.assertEquals("ywjxh", response.iterator().next().platform().customLocation().id()); - Assertions.assertEquals(CoreNetworkType.EPC, response.iterator().next().coreNetworkTechnology()); - Assertions.assertEquals("ontacnpq", response.iterator().next().version()); - Assertions.assertEquals("nsduugwbsre", response.iterator().next().controlPlaneAccessInterface().name()); - Assertions.assertEquals("fqkfuarenl", response.iterator().next().controlPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("htkln", response.iterator().next().controlPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("afvvk", response.iterator().next().controlPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("devjb", response.iterator().next().controlPlaneAccessVirtualIpv4Addresses().get(0)); - Assertions.assertEquals(BillingSku.G5, response.iterator().next().sku()); - Assertions.assertEquals(1265182150, response.iterator().next().ueMtu()); - Assertions - .assertEquals( - AuthenticationType.PASSWORD, response.iterator().next().localDiagnosticsAccess().authenticationType()); - Assertions - .assertEquals( - "y", response.iterator().next().localDiagnosticsAccess().httpsServerCertificate().certificateUrl()); - Assertions - .assertEquals("ngnbdxxew", response.iterator().next().diagnosticsUpload().storageAccountContainerUrl()); - Assertions.assertEquals("invudbch", response.iterator().next().eventHub().id()); - Assertions.assertEquals(1149033945, response.iterator().next().eventHub().reportingInterval()); - Assertions.assertEquals(529081403, response.iterator().next().signaling().nasReroute().macroMmeGroupId()); - } -} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallMockTests.java index ee77d522c337a..fabf437ca704a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesReinstallMockTests.java @@ -30,44 +30,33 @@ public void testReinstall() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"ju\",\"name\":\"lrvkmjc\",\"status\":\"mjvlgfgg\",\"resourceId\":\"kyylizr\",\"startTime\":\"2021-07-07T04:53:59Z\",\"endTime\":\"2021-08-01T10:31:29Z\",\"percentComplete\":50.1159498396177,\"properties\":\"datauztlvtmva\"}"; + String responseStr + = "{\"id\":\"vidttgepuslvyjt\",\"name\":\"uwkasiz\",\"status\":\"esfuught\",\"resourceId\":\"fecjxeygtuhx\",\"startTime\":\"2021-01-12T18:06:14Z\",\"endTime\":\"2021-07-12T08:32:17Z\",\"percentComplete\":80.28171521825273,\"properties\":\"datas\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - AsyncOperationStatus response = - manager.packetCoreControlPlanes().reinstall("zviqsowsaae", "cat", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("ju", response.id()); - Assertions.assertEquals("lrvkmjc", response.name()); - Assertions.assertEquals("mjvlgfgg", response.status()); - Assertions.assertEquals("kyylizr", response.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-07T04:53:59Z"), response.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-01T10:31:29Z"), response.endTime()); - Assertions.assertEquals(50.1159498396177D, response.percentComplete()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + AsyncOperationStatus response + = manager.packetCoreControlPlanes().reinstall("fnrdtjxtxr", "cq", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("vidttgepuslvyjt", response.id()); + Assertions.assertEquals("uwkasiz", response.name()); + Assertions.assertEquals("esfuught", response.status()); + Assertions.assertEquals("fecjxeygtuhx", response.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-12T18:06:14Z"), response.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-12T08:32:17Z"), response.endTime()); + Assertions.assertEquals(80.28171521825273D, response.percentComplete()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackMockTests.java index 38db3b2948619..cc592d0503a98 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreControlPlanesRollbackMockTests.java @@ -30,44 +30,33 @@ public void testRollback() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"toihiqakydi\",\"name\":\"brkwpzdqt\",\"status\":\"hcspo\",\"resourceId\":\"qaxsipietgbebjf\",\"startTime\":\"2021-08-07T23:47:59Z\",\"endTime\":\"2021-12-06T17:55:42Z\",\"percentComplete\":65.3142900105031,\"properties\":\"datalpnfpubntnb\"}"; + String responseStr + = "{\"id\":\"ropions\",\"name\":\"nw\",\"status\":\"ngajinnixjawrtmj\",\"resourceId\":\"myccx\",\"startTime\":\"2021-07-27T15:44:21Z\",\"endTime\":\"2021-03-20T00:37:44Z\",\"percentComplete\":22.430662745783934,\"properties\":\"dataekhenl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - AsyncOperationStatus response = - manager.packetCoreControlPlanes().rollback("tjewlpxuzzj", "nrefqyh", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("toihiqakydi", response.id()); - Assertions.assertEquals("brkwpzdqt", response.name()); - Assertions.assertEquals("hcspo", response.status()); - Assertions.assertEquals("qaxsipietgbebjf", response.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-08-07T23:47:59Z"), response.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2021-12-06T17:55:42Z"), response.endTime()); - Assertions.assertEquals(65.3142900105031D, response.percentComplete()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + AsyncOperationStatus response + = manager.packetCoreControlPlanes().rollback("vzuyturmlmu", "wolba", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("ropions", response.id()); + Assertions.assertEquals("nw", response.name()); + Assertions.assertEquals("ngajinnixjawrtmj", response.status()); + Assertions.assertEquals("myccx", response.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-27T15:44:21Z"), response.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-20T00:37:44Z"), response.endTime()); + Assertions.assertEquals(22.430662745783934D, response.percentComplete()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneInnerTests.java index 22b6cebe8831b..c452de9280771 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneInnerTests.java @@ -15,42 +15,34 @@ public final class PacketCoreDataPlaneInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreDataPlaneInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\",\"userPlaneAccessInterface\":{\"name\":\"qqnvwpmq\",\"ipv4Address\":\"ruoujmk\",\"ipv4Subnet\":\"hwqytj\",\"ipv4Gateway\":\"bnw\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"gdrjervnaenqpe\",\"indoygmifthnzd\",\"dslgnayqigynduh\",\"vhqlkthumaqo\"]},\"location\":\"gycdu\",\"tags\":{\"qlfmmdnbb\":\"tgccymvaolpss\"},\"id\":\"lzpswiydm\",\"name\":\"wyhzdx\",\"type\":\"sadbz\"}") - .toObject(PacketCoreDataPlaneInner.class); - Assertions.assertEquals("gycdu", model.location()); - Assertions.assertEquals("tgccymvaolpss", model.tags().get("qlfmmdnbb")); - Assertions.assertEquals("qqnvwpmq", model.userPlaneAccessInterface().name()); - Assertions.assertEquals("ruoujmk", model.userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("hwqytj", model.userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("bnw", model.userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("gdrjervnaenqpe", model.userPlaneAccessVirtualIpv4Addresses().get(0)); + PacketCoreDataPlaneInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Deleted\",\"userPlaneAccessInterface\":{\"name\":\"nqpeh\",\"ipv4Address\":\"doy\",\"ipv4Subnet\":\"ifthnz\",\"ipv4Gateway\":\"dslgnayqigynduh\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"qlkth\",\"maqolbgycduie\",\"tgccymvaolpss\"]},\"location\":\"lfmmdnbbglzpswi\",\"tags\":{\"bzmnvdfznud\":\"cwyhzdxssa\",\"xzb\":\"od\",\"dzu\":\"cblylpstdbhhxsr\",\"fiwjmygtdssls\":\"erscdntne\"},\"id\":\"tmweriofzpyq\",\"name\":\"emwabnet\",\"type\":\"hhszh\"}") + .toObject(PacketCoreDataPlaneInner.class); + Assertions.assertEquals("lfmmdnbbglzpswi", model.location()); + Assertions.assertEquals("cwyhzdxssa", model.tags().get("bzmnvdfznud")); + Assertions.assertEquals("nqpeh", model.userPlaneAccessInterface().name()); + Assertions.assertEquals("doy", model.userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("ifthnz", model.userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("dslgnayqigynduh", model.userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("qlkth", model.userPlaneAccessVirtualIpv4Addresses().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreDataPlaneInner model = - new PacketCoreDataPlaneInner() - .withLocation("gycdu") - .withTags(mapOf("qlfmmdnbb", "tgccymvaolpss")) - .withUserPlaneAccessInterface( - new InterfaceProperties() - .withName("qqnvwpmq") - .withIpv4Address("ruoujmk") - .withIpv4Subnet("hwqytj") - .withIpv4Gateway("bnw")) - .withUserPlaneAccessVirtualIpv4Addresses( - Arrays.asList("gdrjervnaenqpe", "indoygmifthnzd", "dslgnayqigynduh", "vhqlkthumaqo")); + PacketCoreDataPlaneInner model = new PacketCoreDataPlaneInner().withLocation("lfmmdnbbglzpswi") + .withTags( + mapOf("bzmnvdfznud", "cwyhzdxssa", "xzb", "od", "dzu", "cblylpstdbhhxsr", "fiwjmygtdssls", "erscdntne")) + .withUserPlaneAccessInterface(new InterfaceProperties().withName("nqpeh").withIpv4Address("doy") + .withIpv4Subnet("ifthnz").withIpv4Gateway("dslgnayqigynduh")) + .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("qlkth", "maqolbgycduie", "tgccymvaolpss")); model = BinaryData.fromObject(model).toObject(PacketCoreDataPlaneInner.class); - Assertions.assertEquals("gycdu", model.location()); - Assertions.assertEquals("tgccymvaolpss", model.tags().get("qlfmmdnbb")); - Assertions.assertEquals("qqnvwpmq", model.userPlaneAccessInterface().name()); - Assertions.assertEquals("ruoujmk", model.userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("hwqytj", model.userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("bnw", model.userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("gdrjervnaenqpe", model.userPlaneAccessVirtualIpv4Addresses().get(0)); + Assertions.assertEquals("lfmmdnbbglzpswi", model.location()); + Assertions.assertEquals("cwyhzdxssa", model.tags().get("bzmnvdfznud")); + Assertions.assertEquals("nqpeh", model.userPlaneAccessInterface().name()); + Assertions.assertEquals("doy", model.userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("ifthnz", model.userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("dslgnayqigynduh", model.userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("qlkth", model.userPlaneAccessVirtualIpv4Addresses().get(0)); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneListResultTests.java index b7413e05b5710..4b5c90b73387a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlaneListResultTests.java @@ -16,55 +16,45 @@ public final class PacketCoreDataPlaneListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreDataPlaneListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"userPlaneAccessInterface\":{\"name\":\"bmwmbesldnkw\",\"ipv4Address\":\"pp\",\"ipv4Subnet\":\"lcxog\",\"ipv4Gateway\":\"konzmnsik\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"qzeqqkdltfzxm\"]},\"location\":\"v\",\"tags\":{\"odkwobd\":\"r\"},\"id\":\"gxtibqdxbxw\",\"name\":\"kbogqxndlkzgx\",\"type\":\"uriplbpodxunkb\"},{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneAccessInterface\":{\"name\":\"ubyyntw\",\"ipv4Address\":\"bqtkoievseotgqr\",\"ipv4Subnet\":\"tmuwlauwzi\",\"ipv4Gateway\":\"bm\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"jefuzmuvpbttdumo\",\"p\"]},\"location\":\"ebmnzbtbhjpglk\",\"tags\":{\"dyhtozfikdowwquu\":\"hdneuelfph\",\"ithhqzon\":\"xzxcl\"},\"id\":\"sg\",\"name\":\"b\",\"type\":\"c\"}],\"nextLink\":\"fwdsj\"}") - .toObject(PacketCoreDataPlaneListResult.class); - Assertions.assertEquals("v", model.value().get(0).location()); - Assertions.assertEquals("r", model.value().get(0).tags().get("odkwobd")); - Assertions.assertEquals("bmwmbesldnkw", model.value().get(0).userPlaneAccessInterface().name()); - Assertions.assertEquals("pp", model.value().get(0).userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("lcxog", model.value().get(0).userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("konzmnsik", model.value().get(0).userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("qzeqqkdltfzxm", model.value().get(0).userPlaneAccessVirtualIpv4Addresses().get(0)); + PacketCoreDataPlaneListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"userPlaneAccessInterface\":{\"name\":\"tibqdxbxwakb\",\"ipv4Address\":\"qxn\",\"ipv4Subnet\":\"kzgxhurip\",\"ipv4Gateway\":\"podxunkb\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"mubyynt\",\"lrb\",\"tkoievseotgq\",\"l\"]},\"location\":\"muwlauwzizxbm\",\"tags\":{\"p\":\"jefuzmuvpbttdumo\",\"mnzb\":\"xe\"},\"id\":\"bhjpglkfgohdne\",\"name\":\"el\",\"type\":\"phsdyhto\"},{\"properties\":{\"provisioningState\":\"Accepted\",\"userPlaneAccessInterface\":{\"name\":\"dowwquuvxzxclvit\",\"ipv4Address\":\"qzonosggbhcohf\",\"ipv4Subnet\":\"sjnkal\",\"ipv4Gateway\":\"tiiswacffg\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"zewk\",\"vhqcrail\",\"pnppfuf\",\"rwdmhdlxyjrxsa\"]},\"location\":\"fcnihgwq\",\"tags\":{\"pkeqdcvdrhvoo\":\"edgfbcvkcvq\",\"dopcjwvnh\":\"sotbob\"},\"id\":\"ld\",\"name\":\"mgxcxrslpm\",\"type\":\"twuoegrpkhjwni\"},{\"properties\":{\"provisioningState\":\"Canceled\",\"userPlaneAccessInterface\":{\"name\":\"uicpd\",\"ipv4Address\":\"kzzlvmbmpaxmodf\",\"ipv4Subnet\":\"efyw\",\"ipv4Gateway\":\"pfvmwyhrfou\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"aakc\",\"wiyzvqtmnubexkp\"]},\"location\":\"smond\",\"tags\":{\"jvp\":\"uxvypomgkopkwh\",\"ysmocmbqfqvmkcxo\":\"jqg\",\"tddckcb\":\"apvhelxprgly\"},\"id\":\"uejrjxgc\",\"name\":\"qibrhosxsdqrhzoy\",\"type\":\"i\"}],\"nextLink\":\"rq\"}") + .toObject(PacketCoreDataPlaneListResult.class); + Assertions.assertEquals("muwlauwzizxbm", model.value().get(0).location()); + Assertions.assertEquals("jefuzmuvpbttdumo", model.value().get(0).tags().get("p")); + Assertions.assertEquals("tibqdxbxwakb", model.value().get(0).userPlaneAccessInterface().name()); + Assertions.assertEquals("qxn", model.value().get(0).userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("kzgxhurip", model.value().get(0).userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("podxunkb", model.value().get(0).userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("mubyynt", model.value().get(0).userPlaneAccessVirtualIpv4Addresses().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreDataPlaneListResult model = - new PacketCoreDataPlaneListResult() - .withValue( - Arrays - .asList( - new PacketCoreDataPlaneInner() - .withLocation("v") - .withTags(mapOf("odkwobd", "r")) - .withUserPlaneAccessInterface( - new InterfaceProperties() - .withName("bmwmbesldnkw") - .withIpv4Address("pp") - .withIpv4Subnet("lcxog") - .withIpv4Gateway("konzmnsik")) - .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("qzeqqkdltfzxm")), - new PacketCoreDataPlaneInner() - .withLocation("ebmnzbtbhjpglk") - .withTags(mapOf("dyhtozfikdowwquu", "hdneuelfph", "ithhqzon", "xzxcl")) - .withUserPlaneAccessInterface( - new InterfaceProperties() - .withName("ubyyntw") - .withIpv4Address("bqtkoievseotgqr") - .withIpv4Subnet("tmuwlauwzi") - .withIpv4Gateway("bm")) - .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("jefuzmuvpbttdumo", "p")))); + PacketCoreDataPlaneListResult model = new PacketCoreDataPlaneListResult().withValue(Arrays.asList( + new PacketCoreDataPlaneInner().withLocation("muwlauwzizxbm") + .withTags(mapOf("p", "jefuzmuvpbttdumo", "mnzb", "xe")) + .withUserPlaneAccessInterface(new InterfaceProperties().withName("tibqdxbxwakb").withIpv4Address("qxn") + .withIpv4Subnet("kzgxhurip").withIpv4Gateway("podxunkb")) + .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("mubyynt", "lrb", "tkoievseotgq", "l")), + new PacketCoreDataPlaneInner().withLocation("fcnihgwq") + .withTags(mapOf("pkeqdcvdrhvoo", "edgfbcvkcvq", "dopcjwvnh", "sotbob")) + .withUserPlaneAccessInterface(new InterfaceProperties().withName("dowwquuvxzxclvit") + .withIpv4Address("qzonosggbhcohf").withIpv4Subnet("sjnkal").withIpv4Gateway("tiiswacffg")) + .withUserPlaneAccessVirtualIpv4Addresses( + Arrays.asList("zewk", "vhqcrail", "pnppfuf", "rwdmhdlxyjrxsa")), + new PacketCoreDataPlaneInner().withLocation("smond") + .withTags(mapOf("jvp", "uxvypomgkopkwh", "ysmocmbqfqvmkcxo", "jqg", "tddckcb", "apvhelxprgly")) + .withUserPlaneAccessInterface(new InterfaceProperties().withName("uicpd") + .withIpv4Address("kzzlvmbmpaxmodf").withIpv4Subnet("efyw").withIpv4Gateway("pfvmwyhrfou")) + .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("aakc", "wiyzvqtmnubexkp")))); model = BinaryData.fromObject(model).toObject(PacketCoreDataPlaneListResult.class); - Assertions.assertEquals("v", model.value().get(0).location()); - Assertions.assertEquals("r", model.value().get(0).tags().get("odkwobd")); - Assertions.assertEquals("bmwmbesldnkw", model.value().get(0).userPlaneAccessInterface().name()); - Assertions.assertEquals("pp", model.value().get(0).userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("lcxog", model.value().get(0).userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("konzmnsik", model.value().get(0).userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("qzeqqkdltfzxm", model.value().get(0).userPlaneAccessVirtualIpv4Addresses().get(0)); + Assertions.assertEquals("muwlauwzizxbm", model.value().get(0).location()); + Assertions.assertEquals("jefuzmuvpbttdumo", model.value().get(0).tags().get("p")); + Assertions.assertEquals("tibqdxbxwakb", model.value().get(0).userPlaneAccessInterface().name()); + Assertions.assertEquals("qxn", model.value().get(0).userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("kzgxhurip", model.value().get(0).userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("podxunkb", model.value().get(0).userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("mubyynt", model.value().get(0).userPlaneAccessVirtualIpv4Addresses().get(0)); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanePropertiesFormatTests.java index 189ebcaafb23c..7e421d0f58548 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanePropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanePropertiesFormatTests.java @@ -13,34 +13,27 @@ public final class PacketCoreDataPlanePropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PacketCoreDataPlanePropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Canceled\",\"userPlaneAccessInterface\":{\"name\":\"fznudaodvxzb\",\"ipv4Address\":\"blylpstdbh\",\"ipv4Subnet\":\"srzdzucerscdn\",\"ipv4Gateway\":\"evfiwjmygt\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"lswtmweriofzpyqs\",\"mwabnetshhszhedp\"]}") - .toObject(PacketCoreDataPlanePropertiesFormat.class); - Assertions.assertEquals("fznudaodvxzb", model.userPlaneAccessInterface().name()); - Assertions.assertEquals("blylpstdbh", model.userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("srzdzucerscdn", model.userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("evfiwjmygt", model.userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("lswtmweriofzpyqs", model.userPlaneAccessVirtualIpv4Addresses().get(0)); + PacketCoreDataPlanePropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Deleted\",\"userPlaneAccessInterface\":{\"name\":\"vwiwubmwmbesld\",\"ipv4Address\":\"wwtppj\",\"ipv4Subnet\":\"cxogaokonzm\",\"ipv4Gateway\":\"ikvmkqzeqqk\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"fzxmhhvhgureodkw\"]}") + .toObject(PacketCoreDataPlanePropertiesFormat.class); + Assertions.assertEquals("vwiwubmwmbesld", model.userPlaneAccessInterface().name()); + Assertions.assertEquals("wwtppj", model.userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("cxogaokonzm", model.userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("ikvmkqzeqqk", model.userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("fzxmhhvhgureodkw", model.userPlaneAccessVirtualIpv4Addresses().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PacketCoreDataPlanePropertiesFormat model = - new PacketCoreDataPlanePropertiesFormat() - .withUserPlaneAccessInterface( - new InterfaceProperties() - .withName("fznudaodvxzb") - .withIpv4Address("blylpstdbh") - .withIpv4Subnet("srzdzucerscdn") - .withIpv4Gateway("evfiwjmygt")) - .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("lswtmweriofzpyqs", "mwabnetshhszhedp")); + PacketCoreDataPlanePropertiesFormat model = new PacketCoreDataPlanePropertiesFormat() + .withUserPlaneAccessInterface(new InterfaceProperties().withName("vwiwubmwmbesld").withIpv4Address("wwtppj") + .withIpv4Subnet("cxogaokonzm").withIpv4Gateway("ikvmkqzeqqk")) + .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("fzxmhhvhgureodkw")); model = BinaryData.fromObject(model).toObject(PacketCoreDataPlanePropertiesFormat.class); - Assertions.assertEquals("fznudaodvxzb", model.userPlaneAccessInterface().name()); - Assertions.assertEquals("blylpstdbh", model.userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("srzdzucerscdn", model.userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("evfiwjmygt", model.userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("lswtmweriofzpyqs", model.userPlaneAccessVirtualIpv4Addresses().get(0)); + Assertions.assertEquals("vwiwubmwmbesld", model.userPlaneAccessInterface().name()); + Assertions.assertEquals("wwtppj", model.userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("cxogaokonzm", model.userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("ikvmkqzeqqk", model.userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("fzxmhhvhgureodkw", model.userPlaneAccessVirtualIpv4Addresses().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateMockTests.java index 2339cd6c81e19..23c319e7e2c7c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesCreateOrUpdateMockTests.java @@ -34,58 +34,39 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneAccessInterface\":{\"name\":\"azlp\",\"ipv4Address\":\"wex\",\"ipv4Subnet\":\"zvlazipbhpwvqsgn\",\"ipv4Gateway\":\"uuzivensrpmeyyvp\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"tlbijpzg\"]},\"location\":\"srfhf\",\"tags\":{\"qa\":\"mknbnxwcdommpv\",\"hajlfn\":\"zfgbrttuiaclkie\",\"b\":\"hiqfyuttdiy\"},\"id\":\"vnwsw\",\"name\":\"txkyctwwgzwxjlm\",\"type\":\"cvogygzyvne\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"userPlaneAccessInterface\":{\"name\":\"zbj\",\"ipv4Address\":\"fxsfuztlvtmv\",\"ipv4Subnet\":\"bwidql\",\"ipv4Gateway\":\"ukoveofi\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"jfnmjmvlwyz\"]},\"location\":\"blkujrllfojuidjp\",\"tags\":{\"je\":\"jucejikzoeovvtz\",\"zkdbqzolx\":\"jklntikyj\",\"pcrrk\":\"zvhqjwtrhtgvg\"},\"id\":\"lawjmjsmwrok\",\"name\":\"dxfzzzwyjaf\",\"type\":\"tlhguynuchl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PacketCoreDataPlane response = - manager - .packetCoreDataPlanes() - .define("v") - .withRegion("whc") - .withExistingPacketCoreControlPlane("yvk", "kmrocxne") - .withUserPlaneAccessInterface( - new InterfaceProperties() - .withName("dlfp") - .withIpv4Address("pucygvoavyunss") - .withIpv4Subnet("ghiee") - .withIpv4Gateway("lgvvpaseksgbu")) - .withTags(mapOf("uuerctatoyi", "ibkeph", "duczkgof", "tqpbrlcy", "srucvcrrpcjtt", "y")) - .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("tu", "gdhg", "qipir", "iwrqofulopmjnl")) - .create(); + PacketCoreDataPlane response = manager.packetCoreDataPlanes().define("mykyujxsglhs").withRegion("nfpubntnbatz") + .withExistingPacketCoreControlPlane("emc", "jk") + .withUserPlaneAccessInterface(new InterfaceProperties().withName("ejylmbkzu") + .withIpv4Address("igrfihotjewl").withIpv4Subnet("uzzjgnrefq").withIpv4Gateway("qotoihiqakydiwfb")) + .withTags(mapOf("attcju", "sowsaael")) + .withUserPlaneAccessVirtualIpv4Addresses(Arrays.asList("pzdqtvhcspod", "qaxsipietgbebjf", "lbmoichd")) + .create(); - Assertions.assertEquals("srfhf", response.location()); - Assertions.assertEquals("mknbnxwcdommpv", response.tags().get("qa")); - Assertions.assertEquals("azlp", response.userPlaneAccessInterface().name()); - Assertions.assertEquals("wex", response.userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("zvlazipbhpwvqsgn", response.userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("uuzivensrpmeyyvp", response.userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("tlbijpzg", response.userPlaneAccessVirtualIpv4Addresses().get(0)); + Assertions.assertEquals("blkujrllfojuidjp", response.location()); + Assertions.assertEquals("jucejikzoeovvtz", response.tags().get("je")); + Assertions.assertEquals("zbj", response.userPlaneAccessInterface().name()); + Assertions.assertEquals("fxsfuztlvtmv", response.userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("bwidql", response.userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("ukoveofi", response.userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("jfnmjmvlwyz", response.userPlaneAccessVirtualIpv4Addresses().get(0)); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteMockTests.java index 50706282961f4..08098df00663e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.packetCoreDataPlanes().delete("yueayfbpcmsp", "byrrueqth", "mg", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.packetCoreDataPlanes().delete("izrxklob", "xnazpmkml", "vevfxz", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetWithResponseMockTests.java index 49d1518c32714..717bd771a78c7 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesGetWithResponseMockTests.java @@ -30,47 +30,33 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Deleting\",\"userPlaneAccessInterface\":{\"name\":\"dbwdpyqyybxubmdn\",\"ipv4Address\":\"cbq\",\"ipv4Subnet\":\"e\",\"ipv4Gateway\":\"ela\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"igeleohd\",\"vqvwzkjopwbe\",\"nrlkwzdqy\",\"x\"]},\"location\":\"akxcptsoqfyiaseq\",\"tags\":{\"mvanbwzo\":\"rttzrazisgykiu\"},\"id\":\"mnrxxbsojkl\",\"name\":\"nhmdptysprqs\",\"type\":\"nzxojpslsvjgpli\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Deleting\",\"userPlaneAccessInterface\":{\"name\":\"fyuicdh\",\"ipv4Address\":\"dyb\",\"ipv4Subnet\":\"gbdvibidmhmwff\",\"ipv4Gateway\":\"fmuvapckccr\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"eyoxoyyukphai\",\"moiroqb\"]},\"location\":\"hbragapyyr\",\"tags\":{\"upgahxkum\":\"vbpavbopfppdbw\",\"hqepvufhbzehewh\":\"sjcaacfdmmcpugm\",\"eaclgschorimk\":\"qhnlbqnbld\",\"ucsofldpuviyf\":\"srrm\"},\"id\":\"aabeolhbhlvbmxuq\",\"name\":\"bsxtkcudfbsfarfs\",\"type\":\"owlkjxnqpv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PacketCoreDataPlane response = - manager - .packetCoreDataPlanes() - .getWithResponse("mbscbbx", "gdhxi", "d", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("akxcptsoqfyiaseq", response.location()); - Assertions.assertEquals("rttzrazisgykiu", response.tags().get("mvanbwzo")); - Assertions.assertEquals("dbwdpyqyybxubmdn", response.userPlaneAccessInterface().name()); - Assertions.assertEquals("cbq", response.userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("e", response.userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("ela", response.userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("igeleohd", response.userPlaneAccessVirtualIpv4Addresses().get(0)); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PacketCoreDataPlane response = manager.packetCoreDataPlanes() + .getWithResponse("pj", "bzxliohrdddtfgxq", "awpcbbnzqcykn", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("hbragapyyr", response.location()); + Assertions.assertEquals("vbpavbopfppdbw", response.tags().get("upgahxkum")); + Assertions.assertEquals("fyuicdh", response.userPlaneAccessInterface().name()); + Assertions.assertEquals("dyb", response.userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("gbdvibidmhmwff", response.userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("fmuvapckccr", response.userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("eyoxoyyukphai", response.userPlaneAccessVirtualIpv4Addresses().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneMockTests.java index 312560ed11690..fc9d728220c5c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PacketCoreDataPlanesListByPacketCoreControlPlaneMockTests.java @@ -31,46 +31,33 @@ public void testListByPacketCoreControlPlane() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Canceled\",\"userPlaneAccessInterface\":{\"name\":\"xcxzrzdcgdzben\",\"ipv4Address\":\"bcawetz\",\"ipv4Subnet\":\"dtjwfljhznamt\",\"ipv4Gateway\":\"tmzwcjjn\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"jzmizv\",\"bgatzu\"]},\"location\":\"bxn\",\"tags\":{\"lswva\":\"bwggahtt\",\"zasunwqrjzfrgq\":\"qfutlxjo\"},\"id\":\"aohcmbuocn\",\"name\":\"r\",\"type\":\"hmbpyryxamebly\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"userPlaneAccessInterface\":{\"name\":\"aoaf\",\"ipv4Address\":\"uqvoxmycjimryv\",\"ipv4Subnet\":\"cwwp\",\"ipv4Gateway\":\"zgwe\"},\"userPlaneAccessVirtualIpv4Addresses\":[\"sxwef\",\"hecbvopwndyq\"]},\"location\":\"allklmtkhlowk\",\"tags\":{\"lhikcyychunsj\":\"vbrdfjmzsyzfho\",\"hv\":\"pjrtws\",\"hvtrrmhwrbfdpyf\":\"uic\"},\"id\":\"ubhvj\",\"name\":\"lrocuyzlwh\",\"type\":\"memhooclutnpq\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .packetCoreDataPlanes() - .listByPacketCoreControlPlane("fiqwoy", "qvapcohhoucq", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("bxn", response.iterator().next().location()); - Assertions.assertEquals("bwggahtt", response.iterator().next().tags().get("lswva")); - Assertions.assertEquals("xcxzrzdcgdzben", response.iterator().next().userPlaneAccessInterface().name()); - Assertions.assertEquals("bcawetz", response.iterator().next().userPlaneAccessInterface().ipv4Address()); - Assertions.assertEquals("dtjwfljhznamt", response.iterator().next().userPlaneAccessInterface().ipv4Subnet()); - Assertions.assertEquals("tmzwcjjn", response.iterator().next().userPlaneAccessInterface().ipv4Gateway()); - Assertions.assertEquals("jzmizv", response.iterator().next().userPlaneAccessVirtualIpv4Addresses().get(0)); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.packetCoreDataPlanes().listByPacketCoreControlPlane("gf", + "tmhqykiz", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("allklmtkhlowk", response.iterator().next().location()); + Assertions.assertEquals("vbrdfjmzsyzfho", response.iterator().next().tags().get("lhikcyychunsj")); + Assertions.assertEquals("aoaf", response.iterator().next().userPlaneAccessInterface().name()); + Assertions.assertEquals("uqvoxmycjimryv", response.iterator().next().userPlaneAccessInterface().ipv4Address()); + Assertions.assertEquals("cwwp", response.iterator().next().userPlaneAccessInterface().ipv4Subnet()); + Assertions.assertEquals("zgwe", response.iterator().next().userPlaneAccessInterface().ipv4Gateway()); + Assertions.assertEquals("sxwef", response.iterator().next().userPlaneAccessVirtualIpv4Addresses().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleConfigurationTests.java index 9249dfab6ca0c..b5da854e90024 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleConfigurationTests.java @@ -19,75 +19,58 @@ public final class PccRuleConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PccRuleConfiguration model = - BinaryData - .fromString( - "{\"ruleName\":\"vm\",\"rulePrecedence\":1907297138,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"slthaq\",\"downlink\":\"x\"},\"5qi\":400254725,\"allocationAndRetentionPriorityLevel\":236380813,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"srezp\",\"downlink\":\"rhneuyowq\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"t\",\"direction\":\"Bidirectional\",\"protocol\":[\"bi\"],\"remoteIpList\":[\"gpikpzimejza\",\"lfzxiavrmbzonoki\"],\"ports\":[\"q\"]},{\"templateName\":\"irgzp\",\"direction\":\"Bidirectional\",\"protocol\":[\"azszrnwoiindfpw\",\"jylwbtlhflsj\"],\"remoteIpList\":[\"hszfjvfb\",\"ofel\"],\"ports\":[\"rqmq\",\"ldvriiiojnalghfk\",\"tvsexsowuel\"]}]}") - .toObject(PccRuleConfiguration.class); - Assertions.assertEquals("vm", model.ruleName()); - Assertions.assertEquals(1907297138, model.rulePrecedence()); - Assertions.assertEquals(400254725, model.ruleQosPolicy().fiveQi()); - Assertions.assertEquals(236380813, model.ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.ruleQosPolicy().preemptionCapability()); + PccRuleConfiguration model = BinaryData.fromString( + "{\"ruleName\":\"mtdaa\",\"rulePrecedence\":687186912,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"wvgpiohg\",\"downlink\":\"xrtfudxep\"},\"5qi\":1902888428,\"allocationAndRetentionPriorityLevel\":682770513,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"mnpkukghimdblxg\",\"downlink\":\"imfnjhfjx\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"kkfoqr\",\"direction\":\"Bidirectional\",\"protocol\":[\"kzikfjawneaivxwc\",\"elpcirelsfeaenwa\"],\"remoteIpList\":[\"atklddxbjhwuaa\",\"oz\"],\"ports\":[\"ph\",\"oulpjrv\",\"ag\",\"rvimjwosytxitcsk\"]}]}") + .toObject(PccRuleConfiguration.class); + Assertions.assertEquals("mtdaa", model.ruleName()); + Assertions.assertEquals(687186912, model.rulePrecedence()); + Assertions.assertEquals(1902888428, model.ruleQosPolicy().fiveQi()); + Assertions.assertEquals(682770513, model.ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.ruleQosPolicy().preemptionCapability()); Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, model.ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("srezp", model.ruleQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("rhneuyowq", model.ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("slthaq", model.ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("x", model.ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.ENABLED, model.trafficControl()); - Assertions.assertEquals("t", model.serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals("mnpkukghimdblxg", model.ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("imfnjhfjx", model.ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("wvgpiohg", model.ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("xrtfudxep", model.ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.BLOCKED, model.trafficControl()); + Assertions.assertEquals("kkfoqr", model.serviceDataFlowTemplates().get(0).templateName()); Assertions.assertEquals(SdfDirection.BIDIRECTIONAL, model.serviceDataFlowTemplates().get(0).direction()); - Assertions.assertEquals("bi", model.serviceDataFlowTemplates().get(0).protocol().get(0)); - Assertions.assertEquals("gpikpzimejza", model.serviceDataFlowTemplates().get(0).remoteIpList().get(0)); - Assertions.assertEquals("q", model.serviceDataFlowTemplates().get(0).ports().get(0)); + Assertions.assertEquals("kzikfjawneaivxwc", model.serviceDataFlowTemplates().get(0).protocol().get(0)); + Assertions.assertEquals("atklddxbjhwuaa", model.serviceDataFlowTemplates().get(0).remoteIpList().get(0)); + Assertions.assertEquals("ph", model.serviceDataFlowTemplates().get(0).ports().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PccRuleConfiguration model = - new PccRuleConfiguration() - .withRuleName("vm") - .withRulePrecedence(1907297138) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(400254725) - .withAllocationAndRetentionPriorityLevel(236380813) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("srezp").withDownlink("rhneuyowq")) - .withGuaranteedBitRate(new Ambr().withUplink("slthaq").withDownlink("x"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("t") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("bi")) - .withRemoteIpList(Arrays.asList("gpikpzimejza", "lfzxiavrmbzonoki")) - .withPorts(Arrays.asList("q")), - new ServiceDataFlowTemplate() - .withTemplateName("irgzp") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("azszrnwoiindfpw", "jylwbtlhflsj")) - .withRemoteIpList(Arrays.asList("hszfjvfb", "ofel")) - .withPorts(Arrays.asList("rqmq", "ldvriiiojnalghfk", "tvsexsowuel")))); + PccRuleConfiguration model = new PccRuleConfiguration().withRuleName("mtdaa").withRulePrecedence(687186912) + .withRuleQosPolicy( + new PccRuleQosPolicy().withFiveQi(1902888428).withAllocationAndRetentionPriorityLevel(682770513) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("mnpkukghimdblxg").withDownlink("imfnjhfjx")) + .withGuaranteedBitRate(new Ambr().withUplink("wvgpiohg").withDownlink("xrtfudxep"))) + .withTrafficControl(TrafficControlPermission.BLOCKED) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("kkfoqr").withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList("kzikfjawneaivxwc", "elpcirelsfeaenwa")) + .withRemoteIpList(Arrays.asList("atklddxbjhwuaa", "oz")) + .withPorts(Arrays.asList("ph", "oulpjrv", "ag", "rvimjwosytxitcsk")))); model = BinaryData.fromObject(model).toObject(PccRuleConfiguration.class); - Assertions.assertEquals("vm", model.ruleName()); - Assertions.assertEquals(1907297138, model.rulePrecedence()); - Assertions.assertEquals(400254725, model.ruleQosPolicy().fiveQi()); - Assertions.assertEquals(236380813, model.ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals("mtdaa", model.ruleName()); + Assertions.assertEquals(687186912, model.rulePrecedence()); + Assertions.assertEquals(1902888428, model.ruleQosPolicy().fiveQi()); + Assertions.assertEquals(682770513, model.ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.ruleQosPolicy().preemptionCapability()); Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, model.ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("srezp", model.ruleQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("rhneuyowq", model.ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("slthaq", model.ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("x", model.ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.ENABLED, model.trafficControl()); - Assertions.assertEquals("t", model.serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals("mnpkukghimdblxg", model.ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("imfnjhfjx", model.ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("wvgpiohg", model.ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("xrtfudxep", model.ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.BLOCKED, model.trafficControl()); + Assertions.assertEquals("kkfoqr", model.serviceDataFlowTemplates().get(0).templateName()); Assertions.assertEquals(SdfDirection.BIDIRECTIONAL, model.serviceDataFlowTemplates().get(0).direction()); - Assertions.assertEquals("bi", model.serviceDataFlowTemplates().get(0).protocol().get(0)); - Assertions.assertEquals("gpikpzimejza", model.serviceDataFlowTemplates().get(0).remoteIpList().get(0)); - Assertions.assertEquals("q", model.serviceDataFlowTemplates().get(0).ports().get(0)); + Assertions.assertEquals("kzikfjawneaivxwc", model.serviceDataFlowTemplates().get(0).protocol().get(0)); + Assertions.assertEquals("atklddxbjhwuaa", model.serviceDataFlowTemplates().get(0).remoteIpList().get(0)); + Assertions.assertEquals("ph", model.serviceDataFlowTemplates().get(0).ports().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleQosPolicyTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleQosPolicyTests.java index 5e1b66dc5ef2c..38307b6a3613b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleQosPolicyTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PccRuleQosPolicyTests.java @@ -14,39 +14,35 @@ public final class PccRuleQosPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PccRuleQosPolicy model = - BinaryData - .fromString( - "{\"guaranteedBitRate\":{\"uplink\":\"hhahhxvrhmzkwpjg\",\"downlink\":\"wspughftqsxhqx\"},\"5qi\":318834407,\"allocationAndRetentionPriorityLevel\":649615996,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"igrjguufzdm\",\"downlink\":\"yqtfihwh\"}}") - .toObject(PccRuleQosPolicy.class); - Assertions.assertEquals(318834407, model.fiveQi()); - Assertions.assertEquals(649615996, model.allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.preemptionCapability()); - Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, model.preemptionVulnerability()); - Assertions.assertEquals("igrjguufzdm", model.maximumBitRate().uplink()); - Assertions.assertEquals("yqtfihwh", model.maximumBitRate().downlink()); - Assertions.assertEquals("hhahhxvrhmzkwpjg", model.guaranteedBitRate().uplink()); - Assertions.assertEquals("wspughftqsxhqx", model.guaranteedBitRate().downlink()); + PccRuleQosPolicy model = BinaryData.fromString( + "{\"guaranteedBitRate\":{\"uplink\":\"k\",\"downlink\":\"qumiek\"},\"5qi\":1035067368,\"allocationAndRetentionPriorityLevel\":697538658,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"fjhdg\",\"downlink\":\"gge\"}}") + .toObject(PccRuleQosPolicy.class); + Assertions.assertEquals(1035067368, model.fiveQi()); + Assertions.assertEquals(697538658, model.allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.preemptionVulnerability()); + Assertions.assertEquals("fjhdg", model.maximumBitRate().uplink()); + Assertions.assertEquals("gge", model.maximumBitRate().downlink()); + Assertions.assertEquals("k", model.guaranteedBitRate().uplink()); + Assertions.assertEquals("qumiek", model.guaranteedBitRate().downlink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PccRuleQosPolicy model = - new PccRuleQosPolicy() - .withFiveQi(318834407) - .withAllocationAndRetentionPriorityLevel(649615996) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("igrjguufzdm").withDownlink("yqtfihwh")) - .withGuaranteedBitRate(new Ambr().withUplink("hhahhxvrhmzkwpjg").withDownlink("wspughftqsxhqx")); + PccRuleQosPolicy model + = new PccRuleQosPolicy().withFiveQi(1035067368).withAllocationAndRetentionPriorityLevel(697538658) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("fjhdg").withDownlink("gge")) + .withGuaranteedBitRate(new Ambr().withUplink("k").withDownlink("qumiek")); model = BinaryData.fromObject(model).toObject(PccRuleQosPolicy.class); - Assertions.assertEquals(318834407, model.fiveQi()); - Assertions.assertEquals(649615996, model.allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.preemptionCapability()); - Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, model.preemptionVulnerability()); - Assertions.assertEquals("igrjguufzdm", model.maximumBitRate().uplink()); - Assertions.assertEquals("yqtfihwh", model.maximumBitRate().downlink()); - Assertions.assertEquals("hhahhxvrhmzkwpjg", model.guaranteedBitRate().uplink()); - Assertions.assertEquals("wspughftqsxhqx", model.guaranteedBitRate().downlink()); + Assertions.assertEquals(1035067368, model.fiveQi()); + Assertions.assertEquals(697538658, model.allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.preemptionVulnerability()); + Assertions.assertEquals("fjhdg", model.maximumBitRate().uplink()); + Assertions.assertEquals("gge", model.maximumBitRate().downlink()); + Assertions.assertEquals("k", model.guaranteedBitRate().uplink()); + Assertions.assertEquals("qumiek", model.guaranteedBitRate().downlink()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PinholeTimeoutsTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PinholeTimeoutsTests.java index dc07b59a0743d..9b37fd94ec696 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PinholeTimeoutsTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PinholeTimeoutsTests.java @@ -11,10 +11,8 @@ public final class PinholeTimeoutsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PinholeTimeouts model = - BinaryData - .fromString("{\"tcp\":1863276924,\"udp\":253959415,\"icmp\":1793570177}") - .toObject(PinholeTimeouts.class); + PinholeTimeouts model = BinaryData.fromString("{\"tcp\":1863276924,\"udp\":253959415,\"icmp\":1793570177}") + .toObject(PinholeTimeouts.class); Assertions.assertEquals(1863276924, model.tcp()); Assertions.assertEquals(253959415, model.udp()); Assertions.assertEquals(1793570177, model.icmp()); diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformConfigurationTests.java index 6c3ae3363fbe7..7944a11bf2cd1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformConfigurationTests.java @@ -16,32 +16,28 @@ public final class PlatformConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PlatformConfiguration model = - BinaryData - .fromString( - "{\"type\":\"3P-AZURE-STACK-HCI\",\"azureStackEdgeDevice\":{\"id\":\"mkfssxqukkfplgm\"},\"azureStackEdgeDevices\":[{\"id\":\"nkjzkdeslpvlop\"},{\"id\":\"i\"},{\"id\":\"ighxpk\"}],\"azureStackHciCluster\":{\"id\":\"zb\"},\"connectedCluster\":{\"id\":\"uebbaumnyqup\"},\"customLocation\":{\"id\":\"eojnabc\"}}") - .toObject(PlatformConfiguration.class); + PlatformConfiguration model = BinaryData.fromString( + "{\"type\":\"3P-AZURE-STACK-HCI\",\"azureStackEdgeDevice\":{\"id\":\"ycanuzbpzkafku\"},\"azureStackEdgeDevices\":[{\"id\":\"rnwb\"},{\"id\":\"ehhseyvjusrts\"}],\"azureStackHciCluster\":{\"id\":\"spkdee\"},\"connectedCluster\":{\"id\":\"ofmxagkvtmelmqkr\"},\"customLocation\":{\"id\":\"hvljuahaquh\"}}") + .toObject(PlatformConfiguration.class); Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.type()); - Assertions.assertEquals("mkfssxqukkfplgm", model.azureStackEdgeDevice().id()); - Assertions.assertEquals("zb", model.azureStackHciCluster().id()); - Assertions.assertEquals("uebbaumnyqup", model.connectedCluster().id()); - Assertions.assertEquals("eojnabc", model.customLocation().id()); + Assertions.assertEquals("ycanuzbpzkafku", model.azureStackEdgeDevice().id()); + Assertions.assertEquals("spkdee", model.azureStackHciCluster().id()); + Assertions.assertEquals("ofmxagkvtmelmqkr", model.connectedCluster().id()); + Assertions.assertEquals("hvljuahaquh", model.customLocation().id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PlatformConfiguration model = - new PlatformConfiguration() - .withType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId("mkfssxqukkfplgm")) - .withAzureStackHciCluster(new AzureStackHciClusterResourceId().withId("zb")) - .withConnectedCluster(new ConnectedClusterResourceId().withId("uebbaumnyqup")) - .withCustomLocation(new CustomLocationResourceId().withId("eojnabc")); + PlatformConfiguration model = new PlatformConfiguration().withType(PlatformType.THREE_P_AZURE_STACK_HCI) + .withAzureStackEdgeDevice(new AzureStackEdgeDeviceResourceId().withId("ycanuzbpzkafku")) + .withAzureStackHciCluster(new AzureStackHciClusterResourceId().withId("spkdee")) + .withConnectedCluster(new ConnectedClusterResourceId().withId("ofmxagkvtmelmqkr")) + .withCustomLocation(new CustomLocationResourceId().withId("hvljuahaquh")); model = BinaryData.fromObject(model).toObject(PlatformConfiguration.class); Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.type()); - Assertions.assertEquals("mkfssxqukkfplgm", model.azureStackEdgeDevice().id()); - Assertions.assertEquals("zb", model.azureStackHciCluster().id()); - Assertions.assertEquals("uebbaumnyqup", model.connectedCluster().id()); - Assertions.assertEquals("eojnabc", model.customLocation().id()); + Assertions.assertEquals("ycanuzbpzkafku", model.azureStackEdgeDevice().id()); + Assertions.assertEquals("spkdee", model.azureStackHciCluster().id()); + Assertions.assertEquals("ofmxagkvtmelmqkr", model.connectedCluster().id()); + Assertions.assertEquals("hvljuahaquh", model.customLocation().id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformTests.java index 3e37918869d57..0d71172219e95 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlatformTests.java @@ -15,35 +15,28 @@ public final class PlatformTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Platform model = - BinaryData - .fromString( - "{\"platformType\":\"3P-AZURE-STACK-HCI\",\"versionState\":\"Unknown\",\"minimumPlatformSoftwareVersion\":\"fpbsjyofdxl\",\"maximumPlatformSoftwareVersion\":\"sd\",\"recommendedVersion\":\"NotRecommended\",\"obsoleteVersion\":\"Obsolete\"}") - .toObject(Platform.class); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platformType()); - Assertions.assertEquals(VersionState.UNKNOWN, model.versionState()); - Assertions.assertEquals("fpbsjyofdxl", model.minimumPlatformSoftwareVersion()); - Assertions.assertEquals("sd", model.maximumPlatformSoftwareVersion()); - Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, model.recommendedVersion()); - Assertions.assertEquals(ObsoleteVersion.OBSOLETE, model.obsoleteVersion()); + Platform model = BinaryData.fromString( + "{\"platformType\":\"AKS-HCI\",\"versionState\":\"Active\",\"minimumPlatformSoftwareVersion\":\"pzfqrhhuaoppp\",\"maximumPlatformSoftwareVersion\":\"eqx\",\"recommendedVersion\":\"Recommended\",\"obsoleteVersion\":\"NotObsolete\"}") + .toObject(Platform.class); + Assertions.assertEquals(PlatformType.AKS_HCI, model.platformType()); + Assertions.assertEquals(VersionState.ACTIVE, model.versionState()); + Assertions.assertEquals("pzfqrhhuaoppp", model.minimumPlatformSoftwareVersion()); + Assertions.assertEquals("eqx", model.maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.RECOMMENDED, model.recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, model.obsoleteVersion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Platform model = - new Platform() - .withPlatformType(PlatformType.THREE_P_AZURE_STACK_HCI) - .withVersionState(VersionState.UNKNOWN) - .withMinimumPlatformSoftwareVersion("fpbsjyofdxl") - .withMaximumPlatformSoftwareVersion("sd") - .withRecommendedVersion(RecommendedVersion.NOT_RECOMMENDED) - .withObsoleteVersion(ObsoleteVersion.OBSOLETE); + Platform model = new Platform().withPlatformType(PlatformType.AKS_HCI).withVersionState(VersionState.ACTIVE) + .withMinimumPlatformSoftwareVersion("pzfqrhhuaoppp").withMaximumPlatformSoftwareVersion("eqx") + .withRecommendedVersion(RecommendedVersion.RECOMMENDED).withObsoleteVersion(ObsoleteVersion.NOT_OBSOLETE); model = BinaryData.fromObject(model).toObject(Platform.class); - Assertions.assertEquals(PlatformType.THREE_P_AZURE_STACK_HCI, model.platformType()); - Assertions.assertEquals(VersionState.UNKNOWN, model.versionState()); - Assertions.assertEquals("fpbsjyofdxl", model.minimumPlatformSoftwareVersion()); - Assertions.assertEquals("sd", model.maximumPlatformSoftwareVersion()); - Assertions.assertEquals(RecommendedVersion.NOT_RECOMMENDED, model.recommendedVersion()); - Assertions.assertEquals(ObsoleteVersion.OBSOLETE, model.obsoleteVersion()); + Assertions.assertEquals(PlatformType.AKS_HCI, model.platformType()); + Assertions.assertEquals(VersionState.ACTIVE, model.versionState()); + Assertions.assertEquals("pzfqrhhuaoppp", model.minimumPlatformSoftwareVersion()); + Assertions.assertEquals("eqx", model.maximumPlatformSoftwareVersion()); + Assertions.assertEquals(RecommendedVersion.RECOMMENDED, model.recommendedVersion()); + Assertions.assertEquals(ObsoleteVersion.NOT_OBSOLETE, model.obsoleteVersion()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlmnIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlmnIdTests.java index 7d85a6d0ee59e..0b82bddc6f292 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlmnIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PlmnIdTests.java @@ -11,17 +11,16 @@ public final class PlmnIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PlmnId model = - BinaryData.fromString("{\"mcc\":\"tutqxlngxlefgug\",\"mnc\":\"xkrxdqmi\"}").toObject(PlmnId.class); - Assertions.assertEquals("tutqxlngxlefgug", model.mcc()); - Assertions.assertEquals("xkrxdqmi", model.mnc()); + PlmnId model = BinaryData.fromString("{\"mcc\":\"cgyncocpecf\",\"mnc\":\"m\"}").toObject(PlmnId.class); + Assertions.assertEquals("cgyncocpecf", model.mcc()); + Assertions.assertEquals("m", model.mnc()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PlmnId model = new PlmnId().withMcc("tutqxlngxlefgug").withMnc("xkrxdqmi"); + PlmnId model = new PlmnId().withMcc("cgyncocpecf").withMnc("m"); model = BinaryData.fromObject(model).toObject(PlmnId.class); - Assertions.assertEquals("tutqxlngxlefgug", model.mcc()); - Assertions.assertEquals("xkrxdqmi", model.mnc()); + Assertions.assertEquals("cgyncocpecf", model.mcc()); + Assertions.assertEquals("m", model.mnc()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortRangeTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortRangeTests.java index 2da938b904ea7..06b7f681eea91 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortRangeTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortRangeTests.java @@ -11,8 +11,8 @@ public final class PortRangeTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PortRange model = - BinaryData.fromString("{\"minPort\":761147414,\"maxPort\":1410896813}").toObject(PortRange.class); + PortRange model + = BinaryData.fromString("{\"minPort\":761147414,\"maxPort\":1410896813}").toObject(PortRange.class); Assertions.assertEquals(761147414, model.minPort()); Assertions.assertEquals(1410896813, model.maxPort()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortReuseHoldTimesTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortReuseHoldTimesTests.java index a31d3560a7092..d09d8c4a9d9be 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortReuseHoldTimesTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PortReuseHoldTimesTests.java @@ -11,8 +11,8 @@ public final class PortReuseHoldTimesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PortReuseHoldTimes model = - BinaryData.fromString("{\"tcp\":464271708,\"udp\":48641393}").toObject(PortReuseHoldTimes.class); + PortReuseHoldTimes model + = BinaryData.fromString("{\"tcp\":464271708,\"udp\":48641393}").toObject(PortReuseHoldTimes.class); Assertions.assertEquals(464271708, model.tcp()); Assertions.assertEquals(48641393, model.udp()); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PublicLandMobileNetworkHomeNetworkPublicKeysTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PublicLandMobileNetworkHomeNetworkPublicKeysTests.java new file mode 100644 index 0000000000000..dab60ac149575 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/PublicLandMobileNetworkHomeNetworkPublicKeysTests.java @@ -0,0 +1,42 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.HomeNetworkPublicKey; +import com.azure.resourcemanager.mobilenetwork.models.PublicLandMobileNetworkHomeNetworkPublicKeys; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class PublicLandMobileNetworkHomeNetworkPublicKeysTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PublicLandMobileNetworkHomeNetworkPublicKeys model = BinaryData.fromString( + "{\"profileA\":[{\"id\":588288126,\"url\":\"eallnwsubisnj\"},{\"id\":1708150837,\"url\":\"mngnzscxaqw\"},{\"id\":1219986824,\"url\":\"hcbonqvpkvlr\"},{\"id\":1945684802,\"url\":\"ea\"}],\"profileB\":[{\"id\":625674029,\"url\":\"eo\"},{\"id\":1371791574,\"url\":\"keyyi\"},{\"id\":354708232,\"url\":\"bdlwtgrhpdjpj\"},{\"id\":1270874920,\"url\":\"sxazjpq\"}]}") + .toObject(PublicLandMobileNetworkHomeNetworkPublicKeys.class); + Assertions.assertEquals(588288126, model.profileA().get(0).id()); + Assertions.assertEquals("eallnwsubisnj", model.profileA().get(0).url()); + Assertions.assertEquals(625674029, model.profileB().get(0).id()); + Assertions.assertEquals("eo", model.profileB().get(0).url()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PublicLandMobileNetworkHomeNetworkPublicKeys model = new PublicLandMobileNetworkHomeNetworkPublicKeys() + .withProfileA(Arrays.asList(new HomeNetworkPublicKey().withId(588288126).withUrl("eallnwsubisnj"), + new HomeNetworkPublicKey().withId(1708150837).withUrl("mngnzscxaqw"), + new HomeNetworkPublicKey().withId(1219986824).withUrl("hcbonqvpkvlr"), + new HomeNetworkPublicKey().withId(1945684802).withUrl("ea"))) + .withProfileB(Arrays.asList(new HomeNetworkPublicKey().withId(625674029).withUrl("eo"), + new HomeNetworkPublicKey().withId(1371791574).withUrl("keyyi"), + new HomeNetworkPublicKey().withId(354708232).withUrl("bdlwtgrhpdjpj"), + new HomeNetworkPublicKey().withId(1270874920).withUrl("sxazjpq"))); + model = BinaryData.fromObject(model).toObject(PublicLandMobileNetworkHomeNetworkPublicKeys.class); + Assertions.assertEquals(588288126, model.profileA().get(0).id()); + Assertions.assertEquals("eallnwsubisnj", model.profileA().get(0).url()); + Assertions.assertEquals(625674029, model.profileB().get(0).id()); + Assertions.assertEquals("eo", model.profileB().get(0).url()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/QosPolicyTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/QosPolicyTests.java index fbcad6853cf03..eefe8dbe64037 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/QosPolicyTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/QosPolicyTests.java @@ -14,34 +14,29 @@ public final class QosPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - QosPolicy model = - BinaryData - .fromString( - "{\"5qi\":1698371946,\"allocationAndRetentionPriorityLevel\":523614486,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"hmouwqlgzrfze\",\"downlink\":\"yebizikayuh\"}}") - .toObject(QosPolicy.class); - Assertions.assertEquals(1698371946, model.fiveQi()); - Assertions.assertEquals(523614486, model.allocationAndRetentionPriorityLevel()); + QosPolicy model = BinaryData.fromString( + "{\"5qi\":192957128,\"allocationAndRetentionPriorityLevel\":2075344266,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"xmueed\",\"downlink\":\"drd\"}}") + .toObject(QosPolicy.class); + Assertions.assertEquals(192957128, model.fiveQi()); + Assertions.assertEquals(2075344266, model.allocationAndRetentionPriorityLevel()); Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.preemptionCapability()); Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.preemptionVulnerability()); - Assertions.assertEquals("hmouwqlgzrfze", model.maximumBitRate().uplink()); - Assertions.assertEquals("yebizikayuh", model.maximumBitRate().downlink()); + Assertions.assertEquals("xmueed", model.maximumBitRate().uplink()); + Assertions.assertEquals("drd", model.maximumBitRate().downlink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - QosPolicy model = - new QosPolicy() - .withFiveQi(1698371946) - .withAllocationAndRetentionPriorityLevel(523614486) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("hmouwqlgzrfze").withDownlink("yebizikayuh")); + QosPolicy model = new QosPolicy().withFiveQi(192957128).withAllocationAndRetentionPriorityLevel(2075344266) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("xmueed").withDownlink("drd")); model = BinaryData.fromObject(model).toObject(QosPolicy.class); - Assertions.assertEquals(1698371946, model.fiveQi()); - Assertions.assertEquals(523614486, model.allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(192957128, model.fiveQi()); + Assertions.assertEquals(2075344266, model.allocationAndRetentionPriorityLevel()); Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.preemptionCapability()); Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.preemptionVulnerability()); - Assertions.assertEquals("hmouwqlgzrfze", model.maximumBitRate().uplink()); - Assertions.assertEquals("yebizikayuh", model.maximumBitRate().downlink()); + Assertions.assertEquals("xmueed", model.maximumBitRate().uplink()); + Assertions.assertEquals("drd", model.maximumBitRate().downlink()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceDataFlowTemplateTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceDataFlowTemplateTests.java index 68788bf3dc917..9ec697b9889b3 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceDataFlowTemplateTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceDataFlowTemplateTests.java @@ -13,32 +13,27 @@ public final class ServiceDataFlowTemplateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServiceDataFlowTemplate model = - BinaryData - .fromString( - "{\"templateName\":\"otzi\",\"direction\":\"Uplink\",\"protocol\":[\"mvpp\",\"o\",\"zqzudph\",\"amvdkfwynwcvtbv\"],\"remoteIpList\":[\"yhmtnvyqiat\",\"zwpcnpwzcjaesg\",\"v\",\"ccyajg\"],\"ports\":[\"hwyg\",\"lvdnkfx\",\"semdwzrmu\",\"apfcqdpsq\"]}") - .toObject(ServiceDataFlowTemplate.class); - Assertions.assertEquals("otzi", model.templateName()); - Assertions.assertEquals(SdfDirection.UPLINK, model.direction()); - Assertions.assertEquals("mvpp", model.protocol().get(0)); - Assertions.assertEquals("yhmtnvyqiat", model.remoteIpList().get(0)); - Assertions.assertEquals("hwyg", model.ports().get(0)); + ServiceDataFlowTemplate model = BinaryData.fromString( + "{\"templateName\":\"dunyg\",\"direction\":\"Downlink\",\"protocol\":[\"idb\",\"fatpxllrxcyjmoa\",\"su\",\"arm\"],\"remoteIpList\":[\"dmjsjqb\",\"hhyxxrw\",\"yc\",\"duhpk\"],\"ports\":[\"ymareqnajxqugj\",\"ky\"]}") + .toObject(ServiceDataFlowTemplate.class); + Assertions.assertEquals("dunyg", model.templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, model.direction()); + Assertions.assertEquals("idb", model.protocol().get(0)); + Assertions.assertEquals("dmjsjqb", model.remoteIpList().get(0)); + Assertions.assertEquals("ymareqnajxqugj", model.ports().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServiceDataFlowTemplate model = - new ServiceDataFlowTemplate() - .withTemplateName("otzi") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("mvpp", "o", "zqzudph", "amvdkfwynwcvtbv")) - .withRemoteIpList(Arrays.asList("yhmtnvyqiat", "zwpcnpwzcjaesg", "v", "ccyajg")) - .withPorts(Arrays.asList("hwyg", "lvdnkfx", "semdwzrmu", "apfcqdpsq")); + ServiceDataFlowTemplate model = new ServiceDataFlowTemplate().withTemplateName("dunyg") + .withDirection(SdfDirection.DOWNLINK).withProtocol(Arrays.asList("idb", "fatpxllrxcyjmoa", "su", "arm")) + .withRemoteIpList(Arrays.asList("dmjsjqb", "hhyxxrw", "yc", "duhpk")) + .withPorts(Arrays.asList("ymareqnajxqugj", "ky")); model = BinaryData.fromObject(model).toObject(ServiceDataFlowTemplate.class); - Assertions.assertEquals("otzi", model.templateName()); - Assertions.assertEquals(SdfDirection.UPLINK, model.direction()); - Assertions.assertEquals("mvpp", model.protocol().get(0)); - Assertions.assertEquals("yhmtnvyqiat", model.remoteIpList().get(0)); - Assertions.assertEquals("hwyg", model.ports().get(0)); + Assertions.assertEquals("dunyg", model.templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, model.direction()); + Assertions.assertEquals("idb", model.protocol().get(0)); + Assertions.assertEquals("dmjsjqb", model.remoteIpList().get(0)); + Assertions.assertEquals("ymareqnajxqugj", model.ports().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceInnerTests.java index 7f5819226a14f..ed76ae8293cbc 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceInnerTests.java @@ -23,175 +23,100 @@ public final class ServiceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServiceInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Deleted\",\"servicePrecedence\":1077119628,\"serviceQosPolicy\":{\"5qi\":391766197,\"allocationAndRetentionPriorityLevel\":1151274297,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"ffgdkz\",\"downlink\":\"ewkfvhqcrai\"}},\"pccRules\":[{\"ruleName\":\"pnppfuf\",\"rulePrecedence\":1374740690,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"mh\",\"downlink\":\"lxyjr\"},\"5qi\":1271427468,\"allocationAndRetentionPriorityLevel\":1591764317,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"hgw\",\"downlink\":\"apnedgfbcvkc\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"keqdcvdrhvoods\",\"direction\":\"Downlink\",\"protocol\":[\"obzdopcjwvnhdl\",\"wmgxcxrsl\",\"mutwuoe\",\"rpkhjwn\"],\"remoteIpList\":[\"qsluicp\",\"ggkzzlvmbmpa\"],\"ports\":[\"dfvue\",\"yw\",\"bpfvm\"]},{\"templateName\":\"yhrfouyftaakcpw\",\"direction\":\"Uplink\",\"protocol\":[\"vqtmnub\",\"xkp\",\"ksmond\"],\"remoteIpList\":[\"quxvypomgkop\",\"whojvp\"],\"ports\":[\"gxysmocmbqfqvm\"]},{\"templateName\":\"cxozapvhelxp\",\"direction\":\"Bidirectional\",\"protocol\":[\"yat\"],\"remoteIpList\":[\"ckcb\",\"uejrjxgc\"],\"ports\":[\"brh\",\"sxsdqrhzoymibm\",\"qyib\"]},{\"templateName\":\"hwflu\",\"direction\":\"Uplink\",\"protocol\":[\"tmhrkwofyyvoqacp\",\"expbtg\",\"wbwo\",\"nwashrtd\"],\"remoteIpList\":[\"cnqxwbpokulpi\",\"jwaa\",\"ipqiiobyuqerpq\"],\"ports\":[\"wcciuqgbdbu\",\"auvfbtkuwhhmhyk\"]}]},{\"ruleName\":\"joxafnndlpi\",\"rulePrecedence\":906567669,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"ymkcdyhb\",\"downlink\":\"kkpwdreqnovvq\"},\"5qi\":1985211204,\"allocationAndRetentionPriorityLevel\":1902041203,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"uwsyrsndsytgadg\",\"downlink\":\"r\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"neqn\",\"direction\":\"Downlink\",\"protocol\":[\"rwlqu\",\"ijfqkacewiipfp\",\"bjibwwiftohq\",\"vpuvks\"],\"remoteIpList\":[\"lsa\",\"nynfsynljphuo\",\"xodlqiyntorzih\",\"eosjswsr\"],\"ports\":[\"yzrpzbchckqqzq\",\"ox\",\"ysuiizynkedya\",\"rwyhqmibzyhwitsm\"]}]}]},\"location\":\"yynpcdpumnzgmwz\",\"tags\":{\"lwwrl\":\"biknsorgjhxbld\",\"ok\":\"dmtnc\"},\"id\":\"tllxdyhgsyocogj\",\"name\":\"tdtbnnhadooc\",\"type\":\"kvci\"}") - .toObject(ServiceInner.class); - Assertions.assertEquals("yynpcdpumnzgmwz", model.location()); - Assertions.assertEquals("biknsorgjhxbld", model.tags().get("lwwrl")); - Assertions.assertEquals(1077119628, model.servicePrecedence()); - Assertions.assertEquals(391766197, model.serviceQosPolicy().fiveQi()); - Assertions.assertEquals(1151274297, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals(PreemptionVulnerability.PREEMPTABLE, model.serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("ffgdkz", model.serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("ewkfvhqcrai", model.serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("pnppfuf", model.pccRules().get(0).ruleName()); - Assertions.assertEquals(1374740690, model.pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(1271427468, model.pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals(1591764317, model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.MAY_PREEMPT, model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("hgw", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("apnedgfbcvkc", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("mh", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("lxyjr", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.ENABLED, model.pccRules().get(0).trafficControl()); - Assertions - .assertEquals("keqdcvdrhvoods", model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals(SdfDirection.DOWNLINK, model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); - Assertions - .assertEquals( - "obzdopcjwvnhdl", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); - Assertions - .assertEquals("qsluicp", model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); - Assertions.assertEquals("dfvue", model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); + ServiceInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Canceled\",\"servicePrecedence\":1370014910,\"serviceQosPolicy\":{\"5qi\":2064087160,\"allocationAndRetentionPriorityLevel\":898647722,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"mhrkwofyyvoqacp\",\"downlink\":\"expbtg\"}},\"pccRules\":[{\"ruleName\":\"bwoenwashrt\",\"rulePrecedence\":2130378751,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"nqxwbp\",\"downlink\":\"kulpiujwaasi\"},\"5qi\":172575084,\"allocationAndRetentionPriorityLevel\":1035814224,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"erpqlpqwcciuqg\",\"downlink\":\"dbutauvfbtkuwhh\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"ojoxafnndlp\",\"direction\":\"Downlink\",\"protocol\":[\"koymkcd\",\"h\",\"pkkpw\"],\"remoteIpList\":[\"eqnovvqfovl\",\"xywsuws\",\"rsndsytgadgvra\"],\"ports\":[\"neqn\"]}]}]},\"location\":\"rrwlquuijfqkace\",\"tags\":{\"f\":\"pfpubjibww\",\"kn\":\"ohqkvpuvksgpls\"},\"id\":\"n\",\"name\":\"synljphuopxodl\",\"type\":\"iyntorzihle\"}") + .toObject(ServiceInner.class); + Assertions.assertEquals("rrwlquuijfqkace", model.location()); + Assertions.assertEquals("pfpubjibww", model.tags().get("f")); + Assertions.assertEquals(1370014910, model.servicePrecedence()); + Assertions.assertEquals(2064087160, model.serviceQosPolicy().fiveQi()); + Assertions.assertEquals(898647722, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("mhrkwofyyvoqacp", model.serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("expbtg", model.serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("bwoenwashrt", model.pccRules().get(0).ruleName()); + Assertions.assertEquals(2130378751, model.pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(172575084, model.pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(1035814224, + model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("erpqlpqwcciuqg", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("dbutauvfbtkuwhh", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("nqxwbp", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("kulpiujwaasi", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.BLOCKED, model.pccRules().get(0).trafficControl()); + Assertions.assertEquals("ojoxafnndlp", + model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, + model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("koymkcd", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); + Assertions.assertEquals("eqnovvqfovl", + model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); + Assertions.assertEquals("neqn", model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServiceInner model = - new ServiceInner() - .withLocation("yynpcdpumnzgmwz") - .withTags(mapOf("lwwrl", "biknsorgjhxbld", "ok", "dmtnc")) - .withServicePrecedence(1077119628) - .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(391766197) - .withAllocationAndRetentionPriorityLevel(1151274297) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("ffgdkz").withDownlink("ewkfvhqcrai"))) - .withPccRules( - Arrays - .asList( - new PccRuleConfiguration() - .withRuleName("pnppfuf") - .withRulePrecedence(1374740690) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(1271427468) - .withAllocationAndRetentionPriorityLevel(1591764317) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("hgw").withDownlink("apnedgfbcvkc")) - .withGuaranteedBitRate(new Ambr().withUplink("mh").withDownlink("lxyjr"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("keqdcvdrhvoods") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol( - Arrays.asList("obzdopcjwvnhdl", "wmgxcxrsl", "mutwuoe", "rpkhjwn")) - .withRemoteIpList(Arrays.asList("qsluicp", "ggkzzlvmbmpa")) - .withPorts(Arrays.asList("dfvue", "yw", "bpfvm")), - new ServiceDataFlowTemplate() - .withTemplateName("yhrfouyftaakcpw") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("vqtmnub", "xkp", "ksmond")) - .withRemoteIpList(Arrays.asList("quxvypomgkop", "whojvp")) - .withPorts(Arrays.asList("gxysmocmbqfqvm")), - new ServiceDataFlowTemplate() - .withTemplateName("cxozapvhelxp") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("yat")) - .withRemoteIpList(Arrays.asList("ckcb", "uejrjxgc")) - .withPorts(Arrays.asList("brh", "sxsdqrhzoymibm", "qyib")), - new ServiceDataFlowTemplate() - .withTemplateName("hwflu") - .withDirection(SdfDirection.UPLINK) - .withProtocol( - Arrays.asList("tmhrkwofyyvoqacp", "expbtg", "wbwo", "nwashrtd")) - .withRemoteIpList( - Arrays.asList("cnqxwbpokulpi", "jwaa", "ipqiiobyuqerpq")) - .withPorts(Arrays.asList("wcciuqgbdbu", "auvfbtkuwhhmhyk")))), - new PccRuleConfiguration() - .withRuleName("joxafnndlpi") - .withRulePrecedence(906567669) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(1985211204) - .withAllocationAndRetentionPriorityLevel(1902041203) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("uwsyrsndsytgadg").withDownlink("r")) - .withGuaranteedBitRate( - new Ambr().withUplink("ymkcdyhb").withDownlink("kkpwdreqnovvq"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("neqn") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol( - Arrays.asList("rwlqu", "ijfqkacewiipfp", "bjibwwiftohq", "vpuvks")) - .withRemoteIpList( - Arrays.asList("lsa", "nynfsynljphuo", "xodlqiyntorzih", "eosjswsr")) - .withPorts( - Arrays - .asList( - "yzrpzbchckqqzq", - "ox", - "ysuiizynkedya", - "rwyhqmibzyhwitsm")))))); + ServiceInner model = new ServiceInner().withLocation("rrwlquuijfqkace") + .withTags(mapOf("f", "pfpubjibww", "kn", "ohqkvpuvksgpls")).withServicePrecedence(1370014910) + .withServiceQosPolicy(new QosPolicy().withFiveQi(2064087160) + .withAllocationAndRetentionPriorityLevel(898647722) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("mhrkwofyyvoqacp").withDownlink("expbtg"))) + .withPccRules( + Arrays + .asList(new PccRuleConfiguration().withRuleName("bwoenwashrt").withRulePrecedence(2130378751) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(172575084) + .withAllocationAndRetentionPriorityLevel(1035814224) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("erpqlpqwcciuqg").withDownlink("dbutauvfbtkuwhh")) + .withGuaranteedBitRate(new Ambr().withUplink("nqxwbp").withDownlink("kulpiujwaasi"))) + .withTrafficControl(TrafficControlPermission.BLOCKED) + .withServiceDataFlowTemplates(Arrays.asList(new ServiceDataFlowTemplate() + .withTemplateName("ojoxafnndlp").withDirection(SdfDirection.DOWNLINK) + .withProtocol(Arrays.asList("koymkcd", "h", "pkkpw")) + .withRemoteIpList(Arrays.asList("eqnovvqfovl", "xywsuws", "rsndsytgadgvra")) + .withPorts(Arrays.asList("neqn")))))); model = BinaryData.fromObject(model).toObject(ServiceInner.class); - Assertions.assertEquals("yynpcdpumnzgmwz", model.location()); - Assertions.assertEquals("biknsorgjhxbld", model.tags().get("lwwrl")); - Assertions.assertEquals(1077119628, model.servicePrecedence()); - Assertions.assertEquals(391766197, model.serviceQosPolicy().fiveQi()); - Assertions.assertEquals(1151274297, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals(PreemptionVulnerability.PREEMPTABLE, model.serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("ffgdkz", model.serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("ewkfvhqcrai", model.serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("pnppfuf", model.pccRules().get(0).ruleName()); - Assertions.assertEquals(1374740690, model.pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(1271427468, model.pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals(1591764317, model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.MAY_PREEMPT, model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("hgw", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("apnedgfbcvkc", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("mh", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("lxyjr", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.ENABLED, model.pccRules().get(0).trafficControl()); - Assertions - .assertEquals("keqdcvdrhvoods", model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals(SdfDirection.DOWNLINK, model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); - Assertions - .assertEquals( - "obzdopcjwvnhdl", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); - Assertions - .assertEquals("qsluicp", model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); - Assertions.assertEquals("dfvue", model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); + Assertions.assertEquals("rrwlquuijfqkace", model.location()); + Assertions.assertEquals("pfpubjibww", model.tags().get("f")); + Assertions.assertEquals(1370014910, model.servicePrecedence()); + Assertions.assertEquals(2064087160, model.serviceQosPolicy().fiveQi()); + Assertions.assertEquals(898647722, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("mhrkwofyyvoqacp", model.serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("expbtg", model.serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("bwoenwashrt", model.pccRules().get(0).ruleName()); + Assertions.assertEquals(2130378751, model.pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(172575084, model.pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(1035814224, + model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("erpqlpqwcciuqg", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("dbutauvfbtkuwhh", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("nqxwbp", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("kulpiujwaasi", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.BLOCKED, model.pccRules().get(0).trafficControl()); + Assertions.assertEquals("ojoxafnndlp", + model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, + model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("koymkcd", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); + Assertions.assertEquals("eqnovvqfovl", + model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); + Assertions.assertEquals("neqn", model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceListResultTests.java index 699681ee58693..99fff9e04838d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceListResultTests.java @@ -24,375 +24,220 @@ public final class ServiceListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServiceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"servicePrecedence\":1014865004,\"serviceQosPolicy\":{\"5qi\":1021951816,\"allocationAndRetentionPriorityLevel\":434335068,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"vezrypqlmfeo\",\"downlink\":\"erqwkyhkobopg\"}},\"pccRules\":[{\"ruleName\":\"dkow\",\"rulePrecedence\":1472948449,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"qpcrf\",\"downlink\":\"bwccsnjvcdwxlpqe\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"khtj\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"i\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"wfqatmtd\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"mdvy\",\"rulePrecedence\":1700439966,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"kdgszywkbirr\",\"downlink\":\"uzhlhkjoqrv\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"t\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"nrvgoupmfiibfgg\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"ool\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"wxkvtkkgll\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"jygvjayvbl\",\"rulePrecedence\":1371446568,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"k\",\"downlink\":\"uhbxvvy\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"pbyrqufegxu\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"zfbn\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"mctlpdngitv\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"mhrixkwmyijejve\",\"rulePrecedence\":520387101,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"bpnaixexccbdre\",\"downlink\":\"xhcexdrrvqahq\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"pwijnhy\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"vfycxzb\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"owvrvmtgjqppyos\",\"tags\":{\"hgfipnsxkmcw\":\"nzm\"},\"id\":\"ekrrjr\",\"name\":\"afxtsgum\",\"type\":\"jglikkxwslolb\"},{\"properties\":{\"provisioningState\":\"Deleted\",\"servicePrecedence\":878184103,\"serviceQosPolicy\":{\"5qi\":182360383,\"allocationAndRetentionPriorityLevel\":1828575207,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"tgp\",\"downlink\":\"crpw\"}},\"pccRules\":[{\"ruleName\":\"eznoig\",\"rulePrecedence\":1406843727,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"jwmwkpnbs\",\"downlink\":\"zejjoqk\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"sxtta\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"zxnfaaz\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"dtnkdmkq\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"lwuenvrkp\",\"rulePrecedence\":438145341,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"aibrebqaaysjkixq\",\"downlink\":\"nqttezl\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"iakp\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"qqmtedltmmji\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"eozphv\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"y\",\"tags\":{\"p\":\"ygupkv\"},\"id\":\"dscwxqupevzhf\",\"name\":\"totxhojujb\",\"type\":\"pelmcuvhixbjxyf\"},{\"properties\":{\"provisioningState\":\"Failed\",\"servicePrecedence\":1015582593,\"serviceQosPolicy\":{\"5qi\":729990408,\"allocationAndRetentionPriorityLevel\":1448571025,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"kiwkkbnujr\",\"downlink\":\"wvtylbfpncurdo\"}},\"pccRules\":[{\"ruleName\":\"iithtywu\",\"rulePrecedence\":1466459543,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"bihwqknfdnt\",\"downlink\":\"jchrdgoihxumw\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"dzjlu\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"fdlwg\",\"rulePrecedence\":89081605,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"sb\",\"downlink\":\"tov\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"einqf\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"fxqknpirgneptt\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"msniffc\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"qnrojlpijnkrxfrd\",\"rulePrecedence\":1333681706,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"ratiz\",\"downlink\":\"ronasxift\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"zh\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"wesgogczh\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"nxkrlgnyhmossxkk\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"h\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"hxjbdhqxvc\",\"tags\":{\"shrnsvbuswdvz\":\"rpdsof\",\"rtkfawnopq\":\"ybycnunvj\",\"x\":\"ikyzirtxdy\",\"sewgioilqukr\":\"ejnt\"},\"id\":\"dxtqmieoxo\",\"name\":\"ggufhyaomtb\",\"type\":\"hhavgrvkffovjz\"}],\"nextLink\":\"jbibg\"}") - .toObject(ServiceListResult.class); - Assertions.assertEquals("owvrvmtgjqppyos", model.value().get(0).location()); - Assertions.assertEquals("nzm", model.value().get(0).tags().get("hgfipnsxkmcw")); - Assertions.assertEquals(1014865004, model.value().get(0).servicePrecedence()); - Assertions.assertEquals(1021951816, model.value().get(0).serviceQosPolicy().fiveQi()); - Assertions - .assertEquals(434335068, model.value().get(0).serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, model.value().get(0).serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - model.value().get(0).serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("vezrypqlmfeo", model.value().get(0).serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("erqwkyhkobopg", model.value().get(0).serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("dkow", model.value().get(0).pccRules().get(0).ruleName()); - Assertions.assertEquals(1472948449, model.value().get(0).pccRules().get(0).rulePrecedence()); - Assertions - .assertEquals("qpcrf", model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions - .assertEquals( - "bwccsnjvcdwxlpqe", model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions - .assertEquals(TrafficControlPermission.ENABLED, model.value().get(0).pccRules().get(0).trafficControl()); - Assertions - .assertEquals( - "khtj", model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals( - SdfDirection.BIDIRECTIONAL, - model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + ServiceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"servicePrecedence\":1237082530,\"serviceQosPolicy\":{\"5qi\":2056337569,\"allocationAndRetentionPriorityLevel\":1063119881,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"qal\",\"downlink\":\"rmnjijpx\"}},\"pccRules\":[{\"ruleName\":\"q\",\"rulePrecedence\":208789733,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"fn\",\"downlink\":\"yxbaaabjyvayf\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"zrtuzq\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"sexnevfdnw\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"mewzsyyc\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"zsoibjudpfrxtr\",\"rulePrecedence\":552907355,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"vaytdwkqbrq\",\"downlink\":\"bpaxhexiilivpdt\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"tdqoaxoruzfgsq\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"xrxxlep\",\"tags\":{\"wxuqlcvydypatdoo\":\"mxjezwlw\",\"kooebwnu\":\"ojknio\",\"vdkcrodtj\":\"hemms\"},\"id\":\"nfwjlfltkacjvefk\",\"name\":\"lfoakg\",\"type\":\"kfpagao\"},{\"properties\":{\"provisioningState\":\"Unknown\",\"servicePrecedence\":597345805,\"serviceQosPolicy\":{\"5qi\":1177485669,\"allocationAndRetentionPriorityLevel\":985513982,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"k\",\"downlink\":\"jnsjervtiagxsd\"}},\"pccRules\":[{\"ruleName\":\"uem\",\"rulePrecedence\":1236978734,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"zkfzbeyv\",\"downlink\":\"nqicvinvkjjxdxrb\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"clewyhm\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"paztzpofncck\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"fz\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"hxx\",\"rulePrecedence\":1513228319,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"qa\",\"downlink\":\"zfeqztppri\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"rjaltolmncw\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"bqwcsdbnwdcf\",\"rulePrecedence\":1062647603,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"qdpfuvglsbjjca\",\"downlink\":\"vxb\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"dut\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"ormrlxqtvcofudfl\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"jub\",\"tags\":{\"nqntorudsgsahm\":\"nnqvsa\",\"rauwjuetaebu\":\"yc\",\"dmovsm\":\"u\"},\"id\":\"l\",\"name\":\"wabm\",\"type\":\"oefki\"},{\"properties\":{\"provisioningState\":\"Deleted\",\"servicePrecedence\":2086656667,\"serviceQosPolicy\":{\"5qi\":188593097,\"allocationAndRetentionPriorityLevel\":2034144205,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"kfbtndoaongbjc\",\"downlink\":\"tujitcjedft\"}},\"pccRules\":[{\"ruleName\":\"ae\",\"rulePrecedence\":1170623374,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"jvdcpzfoqouic\",\"downlink\":\"bxarzgszufoxci\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"doamciodhkha\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"khnzbonlw\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"oegokdwbwh\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"z\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"mrv\",\"rulePrecedence\":2100538893,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"tvb\",\"downlink\":\"qgsfraoyzkoow\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"guxawqaldsyuuxi\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"rqf\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"wyznkbyku\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"pfhpagmhrskdsnfd\",\"rulePrecedence\":2086518628,\"ruleQosPolicy\":{\"maximumBitRate\":{\"uplink\":\"akgtdlmkkzevdlh\",\"downlink\":\"wpusdsttwvogv\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"dcngqqmoakufgmj\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"wr\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"twaenuuzko\",\"tags\":{\"hziuiefozbhdms\":\"inrfdwoyu\",\"zqhof\":\"l\"},\"id\":\"rmaequ\",\"name\":\"ah\",\"type\":\"icslfaoq\"}],\"nextLink\":\"iyylhalnswhccsp\"}") + .toObject(ServiceListResult.class); + Assertions.assertEquals("xrxxlep", model.value().get(0).location()); + Assertions.assertEquals("mxjezwlw", model.value().get(0).tags().get("wxuqlcvydypatdoo")); + Assertions.assertEquals(1237082530, model.value().get(0).servicePrecedence()); + Assertions.assertEquals(2056337569, model.value().get(0).serviceQosPolicy().fiveQi()); + Assertions.assertEquals(1063119881, + model.value().get(0).serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + model.value().get(0).serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.value().get(0).serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("qal", model.value().get(0).serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("rmnjijpx", model.value().get(0).serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("q", model.value().get(0).pccRules().get(0).ruleName()); + Assertions.assertEquals(208789733, model.value().get(0).pccRules().get(0).rulePrecedence()); + Assertions.assertEquals("fn", model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("yxbaaabjyvayf", + model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.ENABLED, + model.value().get(0).pccRules().get(0).trafficControl()); + Assertions.assertEquals("zrtuzq", + model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, + model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServiceListResult model = - new ServiceListResult() + ServiceListResult model + = new ServiceListResult() .withValue( Arrays .asList( - new ServiceInner() - .withLocation("owvrvmtgjqppyos") - .withTags(mapOf("hgfipnsxkmcw", "nzm")) - .withServicePrecedence(1014865004) - .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(1021951816) - .withAllocationAndRetentionPriorityLevel(434335068) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withMaximumBitRate( - new Ambr().withUplink("vezrypqlmfeo").withDownlink("erqwkyhkobopg"))) + new ServiceInner().withLocation("xrxxlep") + .withTags( + mapOf("wxuqlcvydypatdoo", "mxjezwlw", "kooebwnu", "ojknio", "vdkcrodtj", "hemms")) + .withServicePrecedence(1237082530) + .withServiceQosPolicy(new QosPolicy().withFiveQi(2056337569) + .withAllocationAndRetentionPriorityLevel(1063119881) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability( + PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("qal").withDownlink("rmnjijpx"))) .withPccRules( Arrays .asList( new PccRuleConfiguration() - .withRuleName("dkow") - .withRulePrecedence(1472948449) + .withRuleName("q").withRulePrecedence(208789733) .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr() - .withUplink("qpcrf") - .withDownlink("bwccsnjvcdwxlpqe"))) + new PccRuleQosPolicy().withMaximumBitRate( + new Ambr().withUplink("fn").withDownlink("yxbaaabjyvayf"))) .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("khtj") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("i") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("wfqatmtd") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("mdvy") - .withRulePrecedence(1700439966) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr() - .withUplink("kdgszywkbirr") - .withDownlink("uzhlhkjoqrv"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("t") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("nrvgoupmfiibfgg") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("ool") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("wxkvtkkgll") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("jygvjayvbl") - .withRulePrecedence(1371446568) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr().withUplink("k").withDownlink("uhbxvvy"))) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("zrtuzq") + .withDirection(SdfDirection.DOWNLINK) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("sexnevfdnw") + .withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("mewzsyyc") + .withDirection(SdfDirection.UPLINK) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("zsoibjudpfrxtr") + .withRulePrecedence(552907355) + .withRuleQosPolicy(new PccRuleQosPolicy().withMaximumBitRate(new Ambr() + .withUplink("vaytdwkqbrq").withDownlink("bpaxhexiilivpdt"))) .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("pbyrqufegxu") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("zfbn") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("mctlpdngitv") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("mhrixkwmyijejve") - .withRulePrecedence(520387101) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr() - .withUplink("bpnaixexccbdre") - .withDownlink("xhcexdrrvqahq"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("pwijnhy") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("vfycxzb") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))))), - new ServiceInner() - .withLocation("y") - .withTags(mapOf("p", "ygupkv")) - .withServicePrecedence(878184103) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("tdqoaxoruzfgsq") + .withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))))), + new ServiceInner().withLocation("jub") + .withTags(mapOf("nqntorudsgsahm", "nnqvsa", "rauwjuetaebu", "yc", "dmovsm", "u")) + .withServicePrecedence(597345805) + .withServiceQosPolicy(new QosPolicy().withFiveQi(1177485669) + .withAllocationAndRetentionPriorityLevel(985513982) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("k").withDownlink("jnsjervtiagxsd"))) + .withPccRules(Arrays.asList( + new PccRuleConfiguration().withRuleName("uem").withRulePrecedence(1236978734) + .withRuleQosPolicy(new PccRuleQosPolicy().withMaximumBitRate( + new Ambr().withUplink("zkfzbeyv").withDownlink("nqicvinvkjjxdxrb"))) + .withTrafficControl( + TrafficControlPermission.ENABLED) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("clewyhm") + .withDirection(SdfDirection.UPLINK).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("paztzpofncck") + .withDirection(SdfDirection.BIDIRECTIONAL).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("fz") + .withDirection(SdfDirection.UPLINK).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("hxx").withRulePrecedence(1513228319) + .withRuleQosPolicy(new PccRuleQosPolicy().withMaximumBitRate( + new Ambr().withUplink("qa").withDownlink("zfeqztppri"))) + .withTrafficControl(TrafficControlPermission.BLOCKED) + .withServiceDataFlowTemplates( + Arrays.asList(new ServiceDataFlowTemplate().withTemplateName("rjaltolmncw") + .withDirection(SdfDirection.BIDIRECTIONAL).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("bqwcsdbnwdcf") + .withRulePrecedence(1062647603) + .withRuleQosPolicy(new PccRuleQosPolicy().withMaximumBitRate( + new Ambr().withUplink("qdpfuvglsbjjca").withDownlink("vxb"))) + .withTrafficControl(TrafficControlPermission.BLOCKED) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("dut") + .withDirection(SdfDirection.DOWNLINK).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("ormrlxqtvcofudfl") + .withDirection(SdfDirection.BIDIRECTIONAL).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))))), + new ServiceInner().withLocation("twaenuuzko") + .withTags(mapOf("hziuiefozbhdms", "inrfdwoyu", "zqhof", "l")) + .withServicePrecedence( + 2086656667) .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(182360383) - .withAllocationAndRetentionPriorityLevel(1828575207) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("tgp").withDownlink("crpw"))) - .withPccRules( - Arrays - .asList( - new PccRuleConfiguration() - .withRuleName("eznoig") - .withRulePrecedence(1406843727) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr().withUplink("jwmwkpnbs").withDownlink("zejjoqk"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("sxtta") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("zxnfaaz") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("dtnkdmkq") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("lwuenvrkp") - .withRulePrecedence(438145341) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr() - .withUplink("aibrebqaaysjkixq") - .withDownlink("nqttezl"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("iakp") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("qqmtedltmmji") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("eozphv") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))))), - new ServiceInner() - .withLocation("hxjbdhqxvc") - .withTags( - mapOf( - "shrnsvbuswdvz", - "rpdsof", - "rtkfawnopq", - "ybycnunvj", - "x", - "ikyzirtxdy", - "sewgioilqukr", - "ejnt")) - .withServicePrecedence(1015582593) - .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(729990408) - .withAllocationAndRetentionPriorityLevel(1448571025) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + new QosPolicy().withFiveQi(188593097) + .withAllocationAndRetentionPriorityLevel(2034144205) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) .withMaximumBitRate( - new Ambr().withUplink("kiwkkbnujr").withDownlink("wvtylbfpncurdo"))) + new Ambr().withUplink("kfbtndoaongbjc").withDownlink("tujitcjedft"))) .withPccRules( Arrays .asList( - new PccRuleConfiguration() - .withRuleName("iithtywu") - .withRulePrecedence(1466459543) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr() - .withUplink("bihwqknfdnt") - .withDownlink("jchrdgoihxumw"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("dzjlu") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("fdlwg") - .withRulePrecedence(89081605) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr().withUplink("sb").withDownlink("tov"))) + new PccRuleConfiguration().withRuleName("ae").withRulePrecedence(1170623374) + .withRuleQosPolicy(new PccRuleQosPolicy().withMaximumBitRate(new Ambr() + .withUplink("jvdcpzfoqouic").withDownlink("bxarzgszufoxci"))) .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("einqf") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("fxqknpirgneptt") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("msniffc") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("qnrojlpijnkrxfrd") - .withRulePrecedence(1333681706) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("doamciodhkha") + .withDirection(SdfDirection.BIDIRECTIONAL).withProtocol( + Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("khnzbonlw") + .withDirection(SdfDirection.DOWNLINK).withProtocol( + Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("oegokdwbwh") + .withDirection(SdfDirection.UPLINK).withProtocol( + Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("z") + .withDirection(SdfDirection.DOWNLINK) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("mrv") + .withRulePrecedence(2100538893) .withRuleQosPolicy( - new PccRuleQosPolicy() - .withMaximumBitRate( - new Ambr().withUplink("ratiz").withDownlink("ronasxift"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("zh") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("wesgogczh") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("nxkrlgnyhmossxkk") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("h") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))))))); + new PccRuleQosPolicy().withMaximumBitRate( + new Ambr().withUplink("tvb").withDownlink("qgsfraoyzkoow"))) + .withTrafficControl(TrafficControlPermission.BLOCKED) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("guxawqaldsyuuxi") + .withDirection(SdfDirection.DOWNLINK) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("rqf") + .withDirection(SdfDirection.DOWNLINK) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate() + .withTemplateName("wyznkbyku") + .withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("pfhpagmhrskdsnfd") + .withRulePrecedence(2086518628) + .withRuleQosPolicy(new PccRuleQosPolicy().withMaximumBitRate(new Ambr() + .withUplink("akgtdlmkkzevdlh").withDownlink("wpusdsttwvogv"))) + .withTrafficControl(TrafficControlPermission.BLOCKED) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("dcngqqmoakufgmj") + .withDirection(SdfDirection.UPLINK) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("wr") + .withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()))))))); model = BinaryData.fromObject(model).toObject(ServiceListResult.class); - Assertions.assertEquals("owvrvmtgjqppyos", model.value().get(0).location()); - Assertions.assertEquals("nzm", model.value().get(0).tags().get("hgfipnsxkmcw")); - Assertions.assertEquals(1014865004, model.value().get(0).servicePrecedence()); - Assertions.assertEquals(1021951816, model.value().get(0).serviceQosPolicy().fiveQi()); - Assertions - .assertEquals(434335068, model.value().get(0).serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, model.value().get(0).serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - model.value().get(0).serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("vezrypqlmfeo", model.value().get(0).serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("erqwkyhkobopg", model.value().get(0).serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("dkow", model.value().get(0).pccRules().get(0).ruleName()); - Assertions.assertEquals(1472948449, model.value().get(0).pccRules().get(0).rulePrecedence()); - Assertions - .assertEquals("qpcrf", model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions - .assertEquals( - "bwccsnjvcdwxlpqe", model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions - .assertEquals(TrafficControlPermission.ENABLED, model.value().get(0).pccRules().get(0).trafficControl()); - Assertions - .assertEquals( - "khtj", model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals( - SdfDirection.BIDIRECTIONAL, - model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("xrxxlep", model.value().get(0).location()); + Assertions.assertEquals("mxjezwlw", model.value().get(0).tags().get("wxuqlcvydypatdoo")); + Assertions.assertEquals(1237082530, model.value().get(0).servicePrecedence()); + Assertions.assertEquals(2056337569, model.value().get(0).serviceQosPolicy().fiveQi()); + Assertions.assertEquals(1063119881, + model.value().get(0).serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + model.value().get(0).serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.value().get(0).serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("qal", model.value().get(0).serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("rmnjijpx", model.value().get(0).serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("q", model.value().get(0).pccRules().get(0).ruleName()); + Assertions.assertEquals(208789733, model.value().get(0).pccRules().get(0).rulePrecedence()); + Assertions.assertEquals("fn", model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("yxbaaabjyvayf", + model.value().get(0).pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.ENABLED, + model.value().get(0).pccRules().get(0).trafficControl()); + Assertions.assertEquals("zrtuzq", + model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, + model.value().get(0).pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicePropertiesFormatTests.java index 3abb7f2fc975e..6e7b0036dcecf 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicePropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicePropertiesFormatTests.java @@ -21,271 +21,92 @@ public final class ServicePropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServicePropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Accepted\",\"servicePrecedence\":600474862,\"serviceQosPolicy\":{\"5qi\":703916773,\"allocationAndRetentionPriorityLevel\":254959284,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"ezikywggxkal\",\"downlink\":\"atmelwui\"}},\"pccRules\":[{\"ruleName\":\"ccjzkzivgvv\",\"rulePrecedence\":1892001157,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"rhyrnxxmueed\",\"downlink\":\"drd\"},\"5qi\":817024636,\"allocationAndRetentionPriorityLevel\":467483193,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"healmfmtda\",\"downlink\":\"ygdvwv\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"hgwxrtfudx\",\"direction\":\"Downlink\",\"protocol\":[\"gyqagvrvmnpkuk\"],\"remoteIpList\":[\"i\",\"dblx\"],\"ports\":[\"mfnjh\",\"j\",\"wmszkk\"]},{\"templateName\":\"oqreyfkzikfjawn\",\"direction\":\"Downlink\",\"protocol\":[\"vxwc\"],\"remoteIpList\":[\"lpcirelsf\",\"aenwabf\",\"tkl\",\"dxbjhwuaanozj\"],\"ports\":[\"hyoulpjr\"]},{\"templateName\":\"xagl\",\"direction\":\"Uplink\",\"protocol\":[\"mjwosytx\",\"tcs\"],\"remoteIpList\":[\"cktqumiekkezzi\",\"hlyfjhdgqgg\",\"bdunygaeqid\"],\"ports\":[\"atpxl\",\"rxcyjmoad\",\"uvarmywdmjsjq\",\"jhhyxxrwlycoduhp\"]},{\"templateName\":\"xkgymareqnajxqu\",\"direction\":\"Bidirectional\",\"protocol\":[\"ky\"],\"remoteIpList\":[\"beddgssofw\",\"mzqa\"],\"ports\":[\"mnjijpxacqqudf\",\"byxbaaabjy\"]}]},{\"ruleName\":\"ayffim\",\"rulePrecedence\":1578513386,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"uzqogsexnevf\",\"downlink\":\"nwnwme\"},\"5qi\":1440499034,\"allocationAndRetentionPriorityLevel\":10547913,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"soibjudpfrx\",\"downlink\":\"rthzvaytdwkqbrqu\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"h\",\"direction\":\"Uplink\",\"protocol\":[\"i\",\"ivpdtiir\",\"tdqoaxoruzfgsq\",\"yfxrx\"],\"remoteIpList\":[\"eptra\"],\"ports\":[\"ezw\",\"wnwxuqlcvyd\"]},{\"templateName\":\"patdooaojkniodko\",\"direction\":\"Bidirectional\",\"protocol\":[\"wnujhemmsbvdk\"],\"remoteIpList\":[\"odtji\",\"fw\",\"lfltka\",\"jvefkdlfoakggkfp\"],\"ports\":[\"owpulpq\"]},{\"templateName\":\"lyls\",\"direction\":\"Downlink\",\"protocol\":[\"qjnsjervtia\"],\"remoteIpList\":[\"sdszue\",\"psbzkfzbeyvpn\",\"icvi\"],\"ports\":[\"jjxd\"]},{\"templateName\":\"rbuukzclewyhmlwp\",\"direction\":\"Bidirectional\",\"protocol\":[\"zpof\",\"cckwyfzqwhxxbu\",\"qa\",\"zfeqztppri\"],\"remoteIpList\":[\"xorjaltolmncwsob\",\"wcsdbnwdcfhucq\",\"pfuvglsbjjca\",\"vxb\"],\"ports\":[\"udutnco\",\"mr\",\"xqtvcofu\"]}]},{\"ruleName\":\"f\",\"rulePrecedence\":573023383,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"jub\",\"downlink\":\"dknnqvsazn\"},\"5qi\":764676926,\"allocationAndRetentionPriorityLevel\":1165613569,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"sahmky\",\"downlink\":\"grauwjuetaebur\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"ovsm\",\"direction\":\"Bidirectional\",\"protocol\":[\"wabm\",\"oefki\"],\"remoteIpList\":[\"vtpuqujmqlgk\"],\"ports\":[\"ndo\",\"ongbjcnt\",\"jitcjedftwwaez\"]},{\"templateName\":\"ojvdcpzfoqo\",\"direction\":\"Bidirectional\",\"protocol\":[\"ybxarzgszu\",\"oxciqopidoamcio\"],\"remoteIpList\":[\"khazxkhnzbonlwn\"],\"ports\":[\"gokdwbwhks\",\"zcmrvexztvb\",\"qgsfraoyzkoow\",\"lmnguxaw\"]},{\"templateName\":\"aldsy\",\"direction\":\"Downlink\",\"protocol\":[\"imerqfobwyznk\",\"ykutwpf\",\"pagmhrskdsnf\",\"sd\"],\"remoteIpList\":[\"kgtdlmkkze\",\"dlhewp\",\"sdsttwvog\",\"bbejdcngqqm\"],\"ports\":[\"ufgmjzrwrdg\",\"twaenuuzko\"]},{\"templateName\":\"bminrfdwoyuhhzi\",\"direction\":\"Bidirectional\",\"protocol\":[\"fozbhdmsmlmzqhof\",\"rmaequ\",\"ah\"],\"remoteIpList\":[\"cslfaoqzpiyylha\",\"nswhcc\"],\"ports\":[\"kaivwit\"]}]},{\"ruleName\":\"scywuggwoluhc\",\"rulePrecedence\":219242582,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"mhairsbrgzdwmsw\",\"downlink\":\"ypqwdxggiccc\"},\"5qi\":1451810874,\"allocationAndRetentionPriorityLevel\":356490202,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"ttlstvlzywemhz\",\"downlink\":\"ncsdtclusiyp\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"ytguslf\",\"direction\":\"Bidirectional\",\"protocol\":[\"cy\",\"qukyhejhzi\",\"xgfpelolppv\",\"srp\"],\"remoteIpList\":[\"ujzra\"],\"ports\":[\"w\"]},{\"templateName\":\"wrft\",\"direction\":\"Bidirectional\",\"protocol\":[\"byrcdlbhshfwp\"],\"remoteIpList\":[\"cstwity\",\"hevxcced\",\"pnmdyodnwzxltjcv\"],\"ports\":[\"tiugcxnav\",\"wxqibyq\",\"nyowxwlmdjrkvfg\"]},{\"templateName\":\"vfvpdbodaciz\",\"direction\":\"Uplink\",\"protocol\":[\"lhkrribdeibqipqk\"],\"remoteIpList\":[\"vxndz\",\"mkrefajpjorwkq\",\"yhgbijtjivfx\",\"sjabibs\"],\"ports\":[\"awfsdjpvkvpbjxbk\",\"bzkdvn\",\"jabudurgkakmo\"]}]}]}") - .toObject(ServicePropertiesFormat.class); - Assertions.assertEquals(600474862, model.servicePrecedence()); - Assertions.assertEquals(703916773, model.serviceQosPolicy().fiveQi()); - Assertions.assertEquals(254959284, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("ezikywggxkal", model.serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("atmelwui", model.serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("ccjzkzivgvv", model.pccRules().get(0).ruleName()); - Assertions.assertEquals(1892001157, model.pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(817024636, model.pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals(467483193, model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("healmfmtda", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("ygdvwv", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("rhyrnxxmueed", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("drd", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.BLOCKED, model.pccRules().get(0).trafficControl()); - Assertions.assertEquals("hgwxrtfudx", model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals(SdfDirection.DOWNLINK, model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); - Assertions - .assertEquals( - "gyqagvrvmnpkuk", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); - Assertions.assertEquals("i", model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); - Assertions.assertEquals("mfnjh", model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); + ServicePropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Canceled\",\"servicePrecedence\":1517246970,\"serviceQosPolicy\":{\"5qi\":2014449644,\"allocationAndRetentionPriorityLevel\":1407927145,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"pzbchck\",\"downlink\":\"qzqioxiysuiizy\"}},\"pccRules\":[{\"ruleName\":\"edyatrwyhqmibzyh\",\"rulePrecedence\":248570264,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"mypyynpcdpu\",\"downlink\":\"nzgmwznmabik\"},\"5qi\":438726197,\"allocationAndRetentionPriorityLevel\":284256630,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"ldtlwwr\",\"downlink\":\"kdmtncvokotll\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"gsyocogj\",\"direction\":\"Bidirectional\",\"protocol\":[\"tbnnha\",\"oocrkvcikhnv\"],\"remoteIpList\":[\"mqg\",\"qqu\",\"zikywgg\",\"kallatmel\"],\"ports\":[\"piccjzkzivgv\",\"c\"]}]}]}") + .toObject(ServicePropertiesFormat.class); + Assertions.assertEquals(1517246970, model.servicePrecedence()); + Assertions.assertEquals(2014449644, model.serviceQosPolicy().fiveQi()); + Assertions.assertEquals(1407927145, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("pzbchck", model.serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("qzqioxiysuiizy", model.serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("edyatrwyhqmibzyh", model.pccRules().get(0).ruleName()); + Assertions.assertEquals(248570264, model.pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(438726197, model.pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(284256630, + model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("ldtlwwr", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("kdmtncvokotll", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("mypyynpcdpu", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("nzgmwznmabik", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.ENABLED, model.pccRules().get(0).trafficControl()); + Assertions.assertEquals("gsyocogj", model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.BIDIRECTIONAL, + model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("tbnnha", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); + Assertions.assertEquals("mqg", model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); + Assertions.assertEquals("piccjzkzivgv", + model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServicePropertiesFormat model = - new ServicePropertiesFormat() - .withServicePrecedence(600474862) - .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(703916773) - .withAllocationAndRetentionPriorityLevel(254959284) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("ezikywggxkal").withDownlink("atmelwui"))) - .withPccRules( - Arrays - .asList( - new PccRuleConfiguration() - .withRuleName("ccjzkzivgvv") - .withRulePrecedence(1892001157) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(817024636) - .withAllocationAndRetentionPriorityLevel(467483193) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("healmfmtda").withDownlink("ygdvwv")) - .withGuaranteedBitRate( - new Ambr().withUplink("rhyrnxxmueed").withDownlink("drd"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("hgwxrtfudx") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList("gyqagvrvmnpkuk")) - .withRemoteIpList(Arrays.asList("i", "dblx")) - .withPorts(Arrays.asList("mfnjh", "j", "wmszkk")), - new ServiceDataFlowTemplate() - .withTemplateName("oqreyfkzikfjawn") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList("vxwc")) - .withRemoteIpList( - Arrays.asList("lpcirelsf", "aenwabf", "tkl", "dxbjhwuaanozj")) - .withPorts(Arrays.asList("hyoulpjr")), - new ServiceDataFlowTemplate() - .withTemplateName("xagl") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("mjwosytx", "tcs")) - .withRemoteIpList( - Arrays.asList("cktqumiekkezzi", "hlyfjhdgqgg", "bdunygaeqid")) - .withPorts( - Arrays - .asList( - "atpxl", "rxcyjmoad", "uvarmywdmjsjq", "jhhyxxrwlycoduhp")), - new ServiceDataFlowTemplate() - .withTemplateName("xkgymareqnajxqu") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("ky")) - .withRemoteIpList(Arrays.asList("beddgssofw", "mzqa")) - .withPorts(Arrays.asList("mnjijpxacqqudf", "byxbaaabjy")))), - new PccRuleConfiguration() - .withRuleName("ayffim") - .withRulePrecedence(1578513386) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(1440499034) - .withAllocationAndRetentionPriorityLevel(10547913) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate( - new Ambr().withUplink("soibjudpfrx").withDownlink("rthzvaytdwkqbrqu")) - .withGuaranteedBitRate( - new Ambr().withUplink("uzqogsexnevf").withDownlink("nwnwme"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("h") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("i", "ivpdtiir", "tdqoaxoruzfgsq", "yfxrx")) - .withRemoteIpList(Arrays.asList("eptra")) - .withPorts(Arrays.asList("ezw", "wnwxuqlcvyd")), - new ServiceDataFlowTemplate() - .withTemplateName("patdooaojkniodko") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("wnujhemmsbvdk")) - .withRemoteIpList( - Arrays.asList("odtji", "fw", "lfltka", "jvefkdlfoakggkfp")) - .withPorts(Arrays.asList("owpulpq")), - new ServiceDataFlowTemplate() - .withTemplateName("lyls") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList("qjnsjervtia")) - .withRemoteIpList(Arrays.asList("sdszue", "psbzkfzbeyvpn", "icvi")) - .withPorts(Arrays.asList("jjxd")), - new ServiceDataFlowTemplate() - .withTemplateName("rbuukzclewyhmlwp") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol( - Arrays.asList("zpof", "cckwyfzqwhxxbu", "qa", "zfeqztppri")) - .withRemoteIpList( - Arrays - .asList( - "xorjaltolmncwsob", - "wcsdbnwdcfhucq", - "pfuvglsbjjca", - "vxb")) - .withPorts(Arrays.asList("udutnco", "mr", "xqtvcofu")))), - new PccRuleConfiguration() - .withRuleName("f") - .withRulePrecedence(573023383) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(764676926) - .withAllocationAndRetentionPriorityLevel(1165613569) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate( - new Ambr().withUplink("sahmky").withDownlink("grauwjuetaebur")) - .withGuaranteedBitRate(new Ambr().withUplink("jub").withDownlink("dknnqvsazn"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("ovsm") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("wabm", "oefki")) - .withRemoteIpList(Arrays.asList("vtpuqujmqlgk")) - .withPorts(Arrays.asList("ndo", "ongbjcnt", "jitcjedftwwaez")), - new ServiceDataFlowTemplate() - .withTemplateName("ojvdcpzfoqo") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("ybxarzgszu", "oxciqopidoamcio")) - .withRemoteIpList(Arrays.asList("khazxkhnzbonlwn")) - .withPorts( - Arrays - .asList( - "gokdwbwhks", "zcmrvexztvb", "qgsfraoyzkoow", "lmnguxaw")), - new ServiceDataFlowTemplate() - .withTemplateName("aldsy") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol( - Arrays.asList("imerqfobwyznk", "ykutwpf", "pagmhrskdsnf", "sd")) - .withRemoteIpList( - Arrays.asList("kgtdlmkkze", "dlhewp", "sdsttwvog", "bbejdcngqqm")) - .withPorts(Arrays.asList("ufgmjzrwrdg", "twaenuuzko")), - new ServiceDataFlowTemplate() - .withTemplateName("bminrfdwoyuhhzi") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("fozbhdmsmlmzqhof", "rmaequ", "ah")) - .withRemoteIpList(Arrays.asList("cslfaoqzpiyylha", "nswhcc")) - .withPorts(Arrays.asList("kaivwit")))), - new PccRuleConfiguration() - .withRuleName("scywuggwoluhc") - .withRulePrecedence(219242582) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(1451810874) - .withAllocationAndRetentionPriorityLevel(356490202) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withMaximumBitRate( - new Ambr().withUplink("ttlstvlzywemhz").withDownlink("ncsdtclusiyp")) - .withGuaranteedBitRate( - new Ambr().withUplink("mhairsbrgzdwmsw").withDownlink("ypqwdxggiccc"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("ytguslf") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("cy", "qukyhejhzi", "xgfpelolppv", "srp")) - .withRemoteIpList(Arrays.asList("ujzra")) - .withPorts(Arrays.asList("w")), - new ServiceDataFlowTemplate() - .withTemplateName("wrft") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList("byrcdlbhshfwp")) - .withRemoteIpList( - Arrays.asList("cstwity", "hevxcced", "pnmdyodnwzxltjcv")) - .withPorts(Arrays.asList("tiugcxnav", "wxqibyq", "nyowxwlmdjrkvfg")), - new ServiceDataFlowTemplate() - .withTemplateName("vfvpdbodaciz") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList("lhkrribdeibqipqk")) - .withRemoteIpList( - Arrays.asList("vxndz", "mkrefajpjorwkq", "yhgbijtjivfx", "sjabibs")) - .withPorts( - Arrays.asList("awfsdjpvkvpbjxbk", "bzkdvn", "jabudurgkakmo")))))); + ServicePropertiesFormat model = new ServicePropertiesFormat().withServicePrecedence(1517246970) + .withServiceQosPolicy( + new QosPolicy().withFiveQi(2014449644).withAllocationAndRetentionPriorityLevel(1407927145) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("pzbchck").withDownlink("qzqioxiysuiizy"))) + .withPccRules( + Arrays + .asList(new PccRuleConfiguration().withRuleName("edyatrwyhqmibzyh").withRulePrecedence(248570264) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(438726197) + .withAllocationAndRetentionPriorityLevel(284256630) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("ldtlwwr").withDownlink("kdmtncvokotll")) + .withGuaranteedBitRate(new Ambr().withUplink("mypyynpcdpu").withDownlink("nzgmwznmabik"))) + .withTrafficControl(TrafficControlPermission.ENABLED) + .withServiceDataFlowTemplates(Arrays.asList(new ServiceDataFlowTemplate() + .withTemplateName("gsyocogj").withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList("tbnnha", "oocrkvcikhnv")) + .withRemoteIpList(Arrays.asList("mqg", "qqu", "zikywgg", "kallatmel")) + .withPorts(Arrays.asList("piccjzkzivgv", "c")))))); model = BinaryData.fromObject(model).toObject(ServicePropertiesFormat.class); - Assertions.assertEquals(600474862, model.servicePrecedence()); - Assertions.assertEquals(703916773, model.serviceQosPolicy().fiveQi()); - Assertions.assertEquals(254959284, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, model.serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, model.serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("ezikywggxkal", model.serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("atmelwui", model.serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("ccjzkzivgvv", model.pccRules().get(0).ruleName()); - Assertions.assertEquals(1892001157, model.pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(817024636, model.pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals(467483193, model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("healmfmtda", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("ygdvwv", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("rhyrnxxmueed", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("drd", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.BLOCKED, model.pccRules().get(0).trafficControl()); - Assertions.assertEquals("hgwxrtfudx", model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals(SdfDirection.DOWNLINK, model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); - Assertions - .assertEquals( - "gyqagvrvmnpkuk", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); - Assertions.assertEquals("i", model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); - Assertions.assertEquals("mfnjh", model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); + Assertions.assertEquals(1517246970, model.servicePrecedence()); + Assertions.assertEquals(2014449644, model.serviceQosPolicy().fiveQi()); + Assertions.assertEquals(1407927145, model.serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, model.serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + model.serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("pzbchck", model.serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("qzqioxiysuiizy", model.serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("edyatrwyhqmibzyh", model.pccRules().get(0).ruleName()); + Assertions.assertEquals(248570264, model.pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(438726197, model.pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(284256630, + model.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("ldtlwwr", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("kdmtncvokotll", model.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("mypyynpcdpu", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("nzgmwznmabik", model.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.ENABLED, model.pccRules().get(0).trafficControl()); + Assertions.assertEquals("gsyocogj", model.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.BIDIRECTIONAL, + model.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("tbnnha", model.pccRules().get(0).serviceDataFlowTemplates().get(0).protocol().get(0)); + Assertions.assertEquals("mqg", model.pccRules().get(0).serviceDataFlowTemplates().get(0).remoteIpList().get(0)); + Assertions.assertEquals("piccjzkzivgv", + model.pccRules().get(0).serviceDataFlowTemplates().get(0).ports().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceResourceIdTests.java index 94f461398baff..0320d74ab76cb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServiceResourceIdTests.java @@ -11,15 +11,15 @@ public final class ServiceResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ServiceResourceId model = - BinaryData.fromString("{\"id\":\"grttikteusqczk\"}").toObject(ServiceResourceId.class); - Assertions.assertEquals("grttikteusqczk", model.id()); + ServiceResourceId model + = BinaryData.fromString("{\"id\":\"fxqknpirgneptt\"}").toObject(ServiceResourceId.class); + Assertions.assertEquals("fxqknpirgneptt", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ServiceResourceId model = new ServiceResourceId().withId("grttikteusqczk"); + ServiceResourceId model = new ServiceResourceId().withId("fxqknpirgneptt"); model = BinaryData.fromObject(model).toObject(ServiceResourceId.class); - Assertions.assertEquals("grttikteusqczk", model.id()); + Assertions.assertEquals("fxqknpirgneptt", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateMockTests.java index 163bb98ed9bd9..a710e151c4081 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesCreateOrUpdateMockTests.java @@ -42,174 +42,109 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"servicePrecedence\":1020817609,\"serviceQosPolicy\":{\"5qi\":624775029,\"allocationAndRetentionPriorityLevel\":162312753,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"y\",\"downlink\":\"leseyqrhvyeldo\"}},\"pccRules\":[{\"ruleName\":\"v\",\"rulePrecedence\":1278495487,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"swskuk\",\"downlink\":\"tasbvw\"},\"5qi\":179725168,\"allocationAndRetentionPriorityLevel\":2105599753,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"fkndl\",\"downlink\":\"vtwknvgmmbugt\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"mqaqkueatgroes\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"gzcbyf\",\"tags\":{\"mvqmtdwckygroej\":\"faoytehqpuv\",\"eqy\":\"ndljdjuskbr\",\"ysh\":\"kceysfaqegplw\"},\"id\":\"ddkvbxgkqu\",\"name\":\"ybwptda\",\"type\":\"arvvlfnt\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"servicePrecedence\":1320210433,\"serviceQosPolicy\":{\"5qi\":287367613,\"allocationAndRetentionPriorityLevel\":1401099475,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"rxvbfihwuh\",\"downlink\":\"ctafsrbxrblm\"}},\"pccRules\":[{\"ruleName\":\"owxih\",\"rulePrecedence\":131708236,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"wqagnepzwa\",\"downlink\":\"lsbs\"},\"5qi\":1761348556,\"allocationAndRetentionPriorityLevel\":171530224,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"xaomzisglrrc\",\"downlink\":\"ezkhhltnjadhqo\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"oyueayfbpcmsp\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"yrrueqth\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"gnmbscbbxigdhx\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"d\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"pedbwdpyqy\",\"rulePrecedence\":1980640945,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"bmdnafcbqwre\",\"downlink\":\"jelaqacigele\"},\"5qi\":175855539,\"allocationAndRetentionPriorityLevel\":883742006,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"kjopwbeon\",\"downlink\":\"lkwzdqybxceakxcp\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"fyiaseqch\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"ttzrazisgyki\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"mvanbwzo\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"rx\",\"tags\":{\"mdptys\":\"ojklin\",\"jp\":\"rqsgnzx\",\"iqwoyxqvapcoh\":\"lsvjgpliu\",\"x\":\"oucqpqojx\"},\"id\":\"rzdcgdzbenribcaw\",\"name\":\"tzqdd\",\"type\":\"jwfljhznamtua\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Service response = - manager - .services() - .define("lzijiufehgmvflnw") - .withRegion("yxkyxlz") - .withExistingMobileNetwork("zusjsz", "bscm") - .withServicePrecedence(1938856045) - .withPccRules( - Arrays - .asList( - new PccRuleConfiguration() - .withRuleName("twhghmupgxyjtc") - .withRulePrecedence(1034131495) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(650123324) - .withAllocationAndRetentionPriorityLevel(161273746) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate( - new Ambr().withUplink("nlzafwxudgnh").withDownlink("ookrtalvnbw")) - .withGuaranteedBitRate( - new Ambr().withUplink("bujftaben").withDownlink("bklqpxz"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("meluclvdjj") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("yrdnqod") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("hhxhq") - .withRulePrecedence(1931668946) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(2069987266) - .withAllocationAndRetentionPriorityLevel(244263827) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate( - new Ambr().withUplink("homewjjstliu").withDownlink("qawmoaianc")) - .withGuaranteedBitRate( - new Ambr().withUplink("vzoqgyipemchgavs").withDownlink("zuejd"))) - .withTrafficControl(TrafficControlPermission.BLOCKED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("drrslblxydk") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("xvv") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("iwkgfbqljnq") - .withDirection(SdfDirection.DOWNLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))), - new PccRuleConfiguration() - .withRuleName("ychocokulehu") - .withRulePrecedence(1204793627) - .withRuleQosPolicy( - new PccRuleQosPolicy() - .withFiveQi(1871401315) - .withAllocationAndRetentionPriorityLevel(257563193) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("mr").withDownlink("chbapxkiy")) - .withGuaranteedBitRate( - new Ambr().withUplink("qffawey").withDownlink("rkphyjdxr"))) - .withTrafficControl(TrafficControlPermission.ENABLED) - .withServiceDataFlowTemplates( - Arrays - .asList( - new ServiceDataFlowTemplate() - .withTemplateName("bajbuscgduusi") - .withDirection(SdfDirection.BIDIRECTIONAL) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()), - new ServiceDataFlowTemplate() - .withTemplateName("cblevpmc") - .withDirection(SdfDirection.UPLINK) - .withProtocol(Arrays.asList()) - .withRemoteIpList(Arrays.asList()))))) - .withTags(mapOf("taf", "gkzz")) + Service response + = manager.services().define("kmifoxxkub").withRegion("rdzsylollgtrczzy") + .withExistingMobileNetwork("fypiv", + "sbbjpmcu") + .withServicePrecedence(749203483) + .withPccRules(Arrays.asList( + new PccRuleConfiguration().withRuleName("iqmrjgeihfqlggw").withRulePrecedence(1124077019) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(1005208720) + .withAllocationAndRetentionPriorityLevel(152052348) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("npomy").withDownlink("wcabvnuileeya")) + .withGuaranteedBitRate(new Ambr().withUplink("cxmjpbyephmg").withDownlink("vljvrc"))) + .withTrafficControl(TrafficControlPermission.ENABLED).withServiceDataFlowTemplates(Arrays + .asList( + new ServiceDataFlowTemplate() + .withTemplateName("augmrmfjlr").withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList()).withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("toaukhfkvcisiz") + .withDirection(SdfDirection.UPLINK).withProtocol(Arrays.asList()) + .withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate().withTemplateName("a").withDirection(SdfDirection.DOWNLINK) + .withProtocol(Arrays.asList()).withRemoteIpList(Arrays.asList()), + new ServiceDataFlowTemplate() + .withTemplateName("sx").withDirection(SdfDirection.BIDIRECTIONAL) + .withProtocol(Arrays.asList()).withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("uivedwcgyeewxeiq").withRulePrecedence(1318685891) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(179076410) + .withAllocationAndRetentionPriorityLevel(1866886383) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("zgaufcshhvn").withDownlink("wgnxkympqanxrj")) + .withGuaranteedBitRate(new Ambr().withUplink("g").withDownlink("mguaml"))) + .withTrafficControl(TrafficControlPermission.ENABLED) + .withServiceDataFlowTemplates(Arrays.asList( + new ServiceDataFlowTemplate().withTemplateName("wbtaoyp").withDirection(SdfDirection.UPLINK) + .withProtocol(Arrays.asList()).withRemoteIpList(Arrays.asList()))), + new PccRuleConfiguration().withRuleName("ghshxcylhkgmnsg").withRulePrecedence(1963006150) + .withRuleQosPolicy(new PccRuleQosPolicy().withFiveQi(217556161) + .withAllocationAndRetentionPriorityLevel(1280541827) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withMaximumBitRate(new Ambr().withUplink("rpqpd").withDownlink("wmkoisq")) + .withGuaranteedBitRate(new Ambr().withUplink("cphdrwjjkhvyo").withDownlink("ac"))) + .withTrafficControl(TrafficControlPermission.ENABLED) + .withServiceDataFlowTemplates(Arrays.asList(new ServiceDataFlowTemplate() + .withTemplateName("fxuifmcsypobk").withDirection(SdfDirection.UPLINK) + .withProtocol(Arrays.asList()).withRemoteIpList(Arrays.asList()))))) + .withTags(mapOf("defxrdc", "zjijpvuaurkihcir")) .withServiceQosPolicy( - new QosPolicy() - .withFiveQi(925629911) - .withAllocationAndRetentionPriorityLevel(813643959) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + new QosPolicy().withFiveQi(43138947).withAllocationAndRetentionPriorityLevel(301475138) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withMaximumBitRate(new Ambr().withUplink("yl").withDownlink("fwxzutg"))) + .withMaximumBitRate(new Ambr().withUplink("vgovpbbttefjokn").withDownlink("sqyzqedikdfr"))) .create(); - Assertions.assertEquals("gzcbyf", response.location()); - Assertions.assertEquals("faoytehqpuv", response.tags().get("mvqmtdwckygroej")); - Assertions.assertEquals(1020817609, response.servicePrecedence()); - Assertions.assertEquals(624775029, response.serviceQosPolicy().fiveQi()); - Assertions.assertEquals(162312753, response.serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, response.serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals(PreemptionVulnerability.PREEMPTABLE, response.serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("y", response.serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("leseyqrhvyeldo", response.serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("v", response.pccRules().get(0).ruleName()); - Assertions.assertEquals(1278495487, response.pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(179725168, response.pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals(2105599753, response.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.MAY_PREEMPT, response.pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - response.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("fkndl", response.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions - .assertEquals("vtwknvgmmbugt", response.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("swskuk", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions.assertEquals("tasbvw", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals("rx", response.location()); + Assertions.assertEquals("ojklin", response.tags().get("mdptys")); + Assertions.assertEquals(1320210433, response.servicePrecedence()); + Assertions.assertEquals(287367613, response.serviceQosPolicy().fiveQi()); + Assertions.assertEquals(1401099475, response.serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, response.serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + response.serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("rxvbfihwuh", response.serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("ctafsrbxrblm", response.serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("owxih", response.pccRules().get(0).ruleName()); + Assertions.assertEquals(131708236, response.pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(1761348556, response.pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(171530224, + response.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + response.pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + response.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("xaomzisglrrc", response.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("ezkhhltnjadhqo", + response.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("wqagnepzwa", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("lsbs", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); Assertions.assertEquals(TrafficControlPermission.BLOCKED, response.pccRules().get(0).trafficControl()); - Assertions - .assertEquals( - "mqaqkueatgroes", response.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals( - SdfDirection.DOWNLINK, response.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("oyueayfbpcmsp", + response.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, + response.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteMockTests.java index 048316877e28a..f9637716aa01a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.services().delete("zaifghtmoqqtlff", "zbkr", "jjjavfqnvhnq", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.services().delete("mltx", "whmozusgzvlnsnnj", "fpafolpymwamxq", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetWithResponseMockTests.java index 03d401e93dbda..8a05f256e153c 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesGetWithResponseMockTests.java @@ -34,74 +34,56 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"servicePrecedence\":835275806,\"serviceQosPolicy\":{\"5qi\":1368462904,\"allocationAndRetentionPriorityLevel\":998586271,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"yaupia\",\"downlink\":\"cxnafbwqrooh\"}},\"pccRules\":[{\"ruleName\":\"ovmaonurjtum\",\"rulePrecedence\":318506853,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"pv\",\"downlink\":\"cmslclblyjxltbs\"},\"5qi\":727856496,\"allocationAndRetentionPriorityLevel\":2065272697,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"gctmgxuupbezq\",\"downlink\":\"cydrtceukdqkk\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"tg\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"mgqzgwldoyc\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"llcecfehuwaoa\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"h\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"qllizstac\",\"rulePrecedence\":1921879633,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"rweft\",\"downlink\":\"wqejpmvsse\"},\"5qi\":432783393,\"allocationAndRetentionPriorityLevel\":731110699,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"tczhupeuknijd\",\"downlink\":\"y\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"ydjfb\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"yv\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"ulrtywikdmh\",\"rulePrecedence\":1891952866,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"flgbhgau\",\"downlink\":\"cdixmx\"},\"5qi\":467497378,\"allocationAndRetentionPriorityLevel\":2084049172,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"dkfnozoeoqbvj\",\"downlink\":\"vefgwbmqjchntas\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"xbulpzealbmqkyo\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"vf\",\"tags\":{\"hxsgxj\":\"tsu\",\"wsdrnpxqwodif\":\"mmzrrscub\"},\"id\":\"jxcjrmmuabwib\",\"name\":\"jogjonmc\",\"type\":\"e\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Canceled\",\"servicePrecedence\":748558129,\"serviceQosPolicy\":{\"5qi\":1011670124,\"allocationAndRetentionPriorityLevel\":876103382,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"lanrupdwvnp\",\"downlink\":\"cnzqtpjhmqr\"}},\"pccRules\":[{\"ruleName\":\"thlaiwdcxs\",\"rulePrecedence\":457713777,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"hzdtxetlgydlh\",\"downlink\":\"vlnnpx\"},\"5qi\":1628791507,\"allocationAndRetentionPriorityLevel\":821138198,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"arbgjekgl\",\"downlink\":\"lbyulidwcwvmze\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"fhjirwgdnqzbrfk\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"zhzmtksjci\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"igsxcdgljplk\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"acht\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"lryts\",\"tags\":{\"whqjjyslurlpshhk\":\"fmdgycxnms\",\"rhmpqvwwsk\":\"pedwqsl\"},\"id\":\"ndcbrwi\",\"name\":\"uvqejosovyrrle\",\"type\":\"esi\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Service response = - manager - .services() - .getWithResponse("ewdogiyetesy", "vidbztjhqtfb", "vnynkb", com.azure.core.util.Context.NONE) - .getValue(); + Service response = manager.services() + .getWithResponse("rag", "gdphtvdula", "vl", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("vf", response.location()); - Assertions.assertEquals("tsu", response.tags().get("hxsgxj")); - Assertions.assertEquals(835275806, response.servicePrecedence()); - Assertions.assertEquals(1368462904, response.serviceQosPolicy().fiveQi()); - Assertions.assertEquals(998586271, response.serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, response.serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, response.serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("yaupia", response.serviceQosPolicy().maximumBitRate().uplink()); - Assertions.assertEquals("cxnafbwqrooh", response.serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("ovmaonurjtum", response.pccRules().get(0).ruleName()); - Assertions.assertEquals(318506853, response.pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(727856496, response.pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals(2065272697, response.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, response.pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - response.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("gctmgxuupbezq", response.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions - .assertEquals("cydrtceukdqkk", response.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("pv", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions - .assertEquals("cmslclblyjxltbs", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions.assertEquals(TrafficControlPermission.BLOCKED, response.pccRules().get(0).trafficControl()); - Assertions.assertEquals("tg", response.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals( - SdfDirection.BIDIRECTIONAL, response.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("lryts", response.location()); + Assertions.assertEquals("fmdgycxnms", response.tags().get("whqjjyslurlpshhk")); + Assertions.assertEquals(748558129, response.servicePrecedence()); + Assertions.assertEquals(1011670124, response.serviceQosPolicy().fiveQi()); + Assertions.assertEquals(876103382, response.serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, response.serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + response.serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("lanrupdwvnp", response.serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("cnzqtpjhmqr", response.serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("thlaiwdcxs", response.pccRules().get(0).ruleName()); + Assertions.assertEquals(457713777, response.pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(1628791507, response.pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(821138198, + response.pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + response.pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + response.pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("arbgjekgl", response.pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("lbyulidwcwvmze", + response.pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("hzdtxetlgydlh", + response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("vlnnpx", response.pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.ENABLED, response.pccRules().get(0).trafficControl()); + Assertions.assertEquals("fhjirwgdnqzbrfk", + response.pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.UPLINK, + response.pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkMockTests.java index b89c05bb87d70..fc937b62d4eed 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/ServicesListByMobileNetworkMockTests.java @@ -35,94 +35,61 @@ public void testListByMobileNetwork() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"servicePrecedence\":1307344323,\"serviceQosPolicy\":{\"5qi\":909885231,\"allocationAndRetentionPriorityLevel\":1517914791,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"ttxpnrupza\",\"downlink\":\"mrdixtreki\"}},\"pccRules\":[{\"ruleName\":\"wyskbruffgll\",\"rulePrecedence\":190739056,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"tvlxhrpqh\",\"downlink\":\"mblcouqehbhbcds\"},\"5qi\":1639777013,\"allocationAndRetentionPriorityLevel\":23531273,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"ypmbltoormkfq\",\"downlink\":\"wxldykalsygaoln\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"bmjk\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"bjgsjjxxahmrn\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"zyqegxyiv\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"nbm\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"bjijkgqxnh\",\"rulePrecedence\":1269742784,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"znj\",\"downlink\":\"ujvaannggi\"},\"5qi\":280719484,\"allocationAndRetentionPriorityLevel\":2110011560,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"xw\",\"downlink\":\"ekaum\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"bzmqkratbn\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"bjsidb\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"kfpksokdgo\",\"rulePrecedence\":998196669,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"jymrhbg\",\"downlink\":\"zozk\"},\"5qi\":617785229,\"allocationAndRetentionPriorityLevel\":254711315,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"hqosmf\",\"downlink\":\"jkutycyarnr\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"uabzoghkt\",\"direction\":\"Bidirectional\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"czhcoeocnh\",\"tags\":{\"jzcfyjzptwr\":\"ot\",\"yglqdhmrjzral\":\"ohapqinfsz\",\"ypsjoq\":\"xpjb\",\"qzvs\":\"jenkyh\"},\"id\":\"xfxjelgcmpzqj\",\"name\":\"hhqxuwyvcacoyviv\",\"type\":\"s\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"servicePrecedence\":1856476390,\"serviceQosPolicy\":{\"5qi\":72464986,\"allocationAndRetentionPriorityLevel\":643452158,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"e\",\"downlink\":\"crse\"}},\"pccRules\":[{\"ruleName\":\"jksghudg\",\"rulePrecedence\":1842910187,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"gjggsv\",\"downlink\":\"u\"},\"5qi\":1202802169,\"allocationAndRetentionPriorityLevel\":2119883340,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"rkmdyom\",\"downlink\":\"xfbvfb\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"rhpw\",\"direction\":\"Downlink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"ddei\",\"rulePrecedence\":2101219561,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"o\",\"downlink\":\"gkkumuikjcj\"},\"5qi\":1257655777,\"allocationAndRetentionPriorityLevel\":2099639555,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"qowxwcom\",\"downlink\":\"ikytwvczcswka\"}},\"trafficControl\":\"Blocked\",\"serviceDataFlowTemplates\":[{\"templateName\":\"yfdvlvhbwrnfxtgd\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"qthe\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"mnaoy\",\"rulePrecedence\":1839346369,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"oe\",\"downlink\":\"swankltytmh\"},\"5qi\":986610623,\"allocationAndRetentionPriorityLevel\":1515443122,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"maximumBitRate\":{\"uplink\":\"lktgjc\",\"downlink\":\"gguxhemlwyw\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"zgfbukklelssx\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]},{\"ruleName\":\"ycsxzu\",\"rulePrecedence\":967931451,\"ruleQosPolicy\":{\"guaranteedBitRate\":{\"uplink\":\"lsmdesqplpvmjc\",\"downlink\":\"oewbid\"},\"5qi\":15124139,\"allocationAndRetentionPriorityLevel\":542120184,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"maximumBitRate\":{\"uplink\":\"piudeugfsxzecpa\",\"downlink\":\"wkufykhvuhxepm\"}},\"trafficControl\":\"Enabled\",\"serviceDataFlowTemplates\":[{\"templateName\":\"nabaobnslujd\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"tymkmvguihywart\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]},{\"templateName\":\"phkixkykxdssjpe\",\"direction\":\"Uplink\",\"protocol\":[],\"remoteIpList\":[]}]}]},\"location\":\"cfxhikkflrmy\",\"tags\":{\"lhrisw\":\"nc\",\"cgxuugqkctotiowl\":\"lmiiiovg\",\"wdtgukranblw\":\"teqdptj\"},\"id\":\"hqlkccuzgygqwaho\",\"name\":\"ulwgniiprglvawuw\",\"type\":\"d\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.services().listByMobileNetwork("oyzbamwineofvf", "akpoldtvevbo", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.services().listByMobileNetwork("uqtljqobbpih", "hcecybmrqbr", com.azure.core.util.Context.NONE); - Assertions.assertEquals("czhcoeocnh", response.iterator().next().location()); - Assertions.assertEquals("ot", response.iterator().next().tags().get("jzcfyjzptwr")); - Assertions.assertEquals(1307344323, response.iterator().next().servicePrecedence()); - Assertions.assertEquals(909885231, response.iterator().next().serviceQosPolicy().fiveQi()); - Assertions - .assertEquals( - 1517914791, response.iterator().next().serviceQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, response.iterator().next().serviceQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - response.iterator().next().serviceQosPolicy().preemptionVulnerability()); - Assertions.assertEquals("ttxpnrupza", response.iterator().next().serviceQosPolicy().maximumBitRate().uplink()); - Assertions - .assertEquals("mrdixtreki", response.iterator().next().serviceQosPolicy().maximumBitRate().downlink()); - Assertions.assertEquals("wyskbruffgll", response.iterator().next().pccRules().get(0).ruleName()); - Assertions.assertEquals(190739056, response.iterator().next().pccRules().get(0).rulePrecedence()); - Assertions.assertEquals(1639777013, response.iterator().next().pccRules().get(0).ruleQosPolicy().fiveQi()); - Assertions - .assertEquals( - 23531273, - response.iterator().next().pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, - response.iterator().next().pccRules().get(0).ruleQosPolicy().preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - response.iterator().next().pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); - Assertions - .assertEquals( - "ypmbltoormkfq", - response.iterator().next().pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); - Assertions - .assertEquals( - "wxldykalsygaoln", - response.iterator().next().pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); - Assertions - .assertEquals( - "tvlxhrpqh", response.iterator().next().pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); - Assertions - .assertEquals( - "mblcouqehbhbcds", - response.iterator().next().pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); - Assertions - .assertEquals( - TrafficControlPermission.BLOCKED, response.iterator().next().pccRules().get(0).trafficControl()); - Assertions - .assertEquals( - "bmjk", response.iterator().next().pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); - Assertions - .assertEquals( - SdfDirection.DOWNLINK, - response.iterator().next().pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); + Assertions.assertEquals("cfxhikkflrmy", response.iterator().next().location()); + Assertions.assertEquals("nc", response.iterator().next().tags().get("lhrisw")); + Assertions.assertEquals(1856476390, response.iterator().next().servicePrecedence()); + Assertions.assertEquals(72464986, response.iterator().next().serviceQosPolicy().fiveQi()); + Assertions.assertEquals(643452158, + response.iterator().next().serviceQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + response.iterator().next().serviceQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + response.iterator().next().serviceQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("e", response.iterator().next().serviceQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("crse", response.iterator().next().serviceQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("jksghudg", response.iterator().next().pccRules().get(0).ruleName()); + Assertions.assertEquals(1842910187, response.iterator().next().pccRules().get(0).rulePrecedence()); + Assertions.assertEquals(1202802169, response.iterator().next().pccRules().get(0).ruleQosPolicy().fiveQi()); + Assertions.assertEquals(2119883340, + response.iterator().next().pccRules().get(0).ruleQosPolicy().allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + response.iterator().next().pccRules().get(0).ruleQosPolicy().preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.PREEMPTABLE, + response.iterator().next().pccRules().get(0).ruleQosPolicy().preemptionVulnerability()); + Assertions.assertEquals("rkmdyom", + response.iterator().next().pccRules().get(0).ruleQosPolicy().maximumBitRate().uplink()); + Assertions.assertEquals("xfbvfb", + response.iterator().next().pccRules().get(0).ruleQosPolicy().maximumBitRate().downlink()); + Assertions.assertEquals("gjggsv", + response.iterator().next().pccRules().get(0).ruleQosPolicy().guaranteedBitRate().uplink()); + Assertions.assertEquals("u", + response.iterator().next().pccRules().get(0).ruleQosPolicy().guaranteedBitRate().downlink()); + Assertions.assertEquals(TrafficControlPermission.ENABLED, + response.iterator().next().pccRules().get(0).trafficControl()); + Assertions.assertEquals("rhpw", + response.iterator().next().pccRules().get(0).serviceDataFlowTemplates().get(0).templateName()); + Assertions.assertEquals(SdfDirection.DOWNLINK, + response.iterator().next().pccRules().get(0).serviceDataFlowTemplates().get(0).direction()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SignalingConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SignalingConfigurationTests.java index 49b69fdf39567..c658736254159 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SignalingConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SignalingConfigurationTests.java @@ -12,18 +12,16 @@ public final class SignalingConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SignalingConfiguration model = - BinaryData - .fromString("{\"nasReroute\":{\"macroMmeGroupId\":1468269047}}") - .toObject(SignalingConfiguration.class); - Assertions.assertEquals(1468269047, model.nasReroute().macroMmeGroupId()); + SignalingConfiguration model = BinaryData.fromString("{\"nasReroute\":{\"macroMmeGroupId\":1178618137}}") + .toObject(SignalingConfiguration.class); + Assertions.assertEquals(1178618137, model.nasReroute().macroMmeGroupId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SignalingConfiguration model = - new SignalingConfiguration().withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(1468269047)); + SignalingConfiguration model = new SignalingConfiguration() + .withNasReroute(new NasRerouteConfiguration().withMacroMmeGroupId(1178618137)); model = BinaryData.fromObject(model).toObject(SignalingConfiguration.class); - Assertions.assertEquals(1468269047, model.nasReroute().macroMmeGroupId()); + Assertions.assertEquals(1178618137, model.nasReroute().macroMmeGroupId()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimDeleteListTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimDeleteListTests.java index f6a979202c9df..9b101af8337f6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimDeleteListTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimDeleteListTests.java @@ -12,14 +12,15 @@ public final class SimDeleteListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimDeleteList model = BinaryData.fromString("{\"sims\":[\"dvk\"]}").toObject(SimDeleteList.class); - Assertions.assertEquals("dvk", model.sims().get(0)); + SimDeleteList model = BinaryData.fromString("{\"sims\":[\"ouw\",\"lgzrfzeeyeb\",\"zi\",\"ayuhqlbjbsy\"]}") + .toObject(SimDeleteList.class); + Assertions.assertEquals("ouw", model.sims().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimDeleteList model = new SimDeleteList().withSims(Arrays.asList("dvk")); + SimDeleteList model = new SimDeleteList().withSims(Arrays.asList("ouw", "lgzrfzeeyeb", "zi", "ayuhqlbjbsy")); model = BinaryData.fromObject(model).toObject(SimDeleteList.class); - Assertions.assertEquals("dvk", model.sims().get(0)); + Assertions.assertEquals("ouw", model.sims().get(0)); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteMockTests.java index a9547139207da..de553f5213c03 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimGroupsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.simGroups().delete("avnwqj", "g", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.simGroups().delete("mzvlazipbh", "wvqsgny", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateMockTests.java index ad6c4b75dc3fc..91ac99ff7afaa 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesCreateOrUpdateMockTests.java @@ -38,110 +38,83 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"siteProvisioningState\":{\"kduoiqtamtyvs\":\"Updating\",\"xrwzawnvsbcf\":\"NotApplicable\",\"agxnvhycvdimw\":\"Failed\"},\"ueAmbr\":{\"uplink\":\"zregzgyufutrwpw\",\"downlink\":\"ryekzkd\"},\"defaultSlice\":{\"id\":\"meottawj\"},\"rfspIndex\":375034092,\"registrationTimer\":543459562,\"sliceConfigurations\":[{\"slice\":{\"id\":\"hnhjtfvpndpmi\"},\"defaultDataNetwork\":{\"id\":\"jpnwynudql\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"sauzpjlx\"},\"sessionAmbr\":{\"uplink\":\"ehuxiqhzlraym\",\"downlink\":\"zxlskihmxr\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"dsajrednwyysh\"},\"sessionAmbr\":{\"uplink\":\"uwg\",\"downlink\":\"evuafpwzyifrk\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"wltxeqipxgzdyims\"},\"defaultDataNetwork\":{\"id\":\"ayorprav\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"oge\"},\"sessionAmbr\":{\"uplink\":\"labnsmjkwynq\",\"downlink\":\"aekqsykvwj\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"qpkevmyltjc\"},\"sessionAmbr\":{\"uplink\":\"spxklu\",\"downlink\":\"cclfgxannn\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"ytzpo\"},\"sessionAmbr\":{\"uplink\":\"ewxigpxvk\",\"downlink\":\"maupxvpi\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"dfaifyzyzeyuube\"},\"sessionAmbr\":{\"uplink\":\"ds\",\"downlink\":\"l\"},\"allowedServices\":[]}]}]},\"location\":\"toi\",\"tags\":{\"rxwe\":\"ygvfltgvdihoyn\",\"cy\":\"wkd\",\"dnaienhqhskndnel\":\"ucpcunnuzdqumoen\",\"lknwfoanniyop\":\"kaa\"},\"id\":\"txiv\",\"name\":\"nrlyxnuc\",\"type\":\"ephblkwqpatvbqs\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"siteProvisioningState\":{\"zzlt\":\"Updating\"},\"ueAmbr\":{\"uplink\":\"fhbzffovwmbjlzq\",\"downlink\":\"czpgvdwnapfdq\"},\"defaultSlice\":{\"id\":\"owftptnuwjtks\"},\"rfspIndex\":2008228134,\"registrationTimer\":588054641,\"sliceConfigurations\":[{\"slice\":{\"id\":\"y\"},\"defaultDataNetwork\":{\"id\":\"leseyqrhvyeldo\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"v\"},\"sessionAmbr\":{\"uplink\":\"kwiswskukjtas\",\"downlink\":\"vwisp\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"xkdtxfkndlqvtwkn\"},\"defaultDataNetwork\":{\"id\":\"gmmbu\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"yw\"},\"sessionAmbr\":{\"uplink\":\"tmqa\",\"downlink\":\"kueatgroeshoy\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"zcbyfq\"},\"sessionAmbr\":{\"uplink\":\"kfaoytehq\",\"downlink\":\"uvjmv\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"mtdwcky\"},\"sessionAmbr\":{\"uplink\":\"roejnndl\",\"downlink\":\"djus\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"br\"},\"defaultDataNetwork\":{\"id\":\"eqy\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"ceysfaqeg\"},\"sessionAmbr\":{\"uplink\":\"lwryshwddkvbxgkq\",\"downlink\":\"sybwptdaca\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"vvlfntymtp\"},\"sessionAmbr\":{\"uplink\":\"iwenazero\",\"downlink\":\"zrsq\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"lsxkd\"},\"sessionAmbr\":{\"uplink\":\"wqapfgsdp\",\"downlink\":\"vessm\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"hhkuuip\"},\"sessionAmbr\":{\"uplink\":\"dqq\",\"downlink\":\"tekva\"},\"allowedServices\":[]}]}]},\"location\":\"l\",\"tags\":{\"emxhzzy\":\"qvqyvweht\"},\"id\":\"ev\",\"name\":\"sxivzrrryvei\",\"type\":\"ipsk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - SimPolicy response = - manager - .simPolicies() - .define("hza") - .withRegion("f") - .withExistingMobileNetwork("sgftipwc", "byubhiqdxyurnpn") - .withUeAmbr(new Ambr().withUplink("vatvcrkd").withDownlink("b")) - .withDefaultSlice(new SliceResourceId().withId("bqxvhcsyhzlwxae")) - .withSliceConfigurations( - Arrays - .asList( - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("xdndsbdweaderzm")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("ntopa")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("tmvmmagoaqylkjz")) - .withSessionAmbr( - new Ambr().withUplink("jiuazjc").withDownlink("mxitpfinzcpd")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("tkrlgjmtbd")) - .withSessionAmbr( - new Ambr().withUplink("vcqguefzh").withDownlink("mpheqdur")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("lyujlfyoump")) - .withSessionAmbr( - new Ambr() - .withUplink("kyeclcdigpta") - .withDownlink("brzmqxucycijoclx")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("utgjcyz")) - .withSessionAmbr(new Ambr().withUplink("zjd").withDownlink("r")) - .withAllowedServices(Arrays.asList()))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("jb")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("xjeaoqaqbzgyh")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("wvua")) - .withSessionAmbr( - new Ambr().withUplink("bwbqamt").withDownlink("uliyslpkcv")) - .withAllowedServices(Arrays.asList()))))) - .withTags(mapOf("myw", "xe")) - .withRfspIndex(1277865813) - .withRegistrationTimer(92922352) + SimPolicy response + = manager + .simPolicies().define( + "elgcmpzqjhhhqx") + .withRegion("apx").withExistingMobileNetwork("zv", "qxfx") + .withUeAmbr( + new Ambr().withUplink("zusjsz").withDownlink("bscm")) + .withDefaultSlice( + new SliceResourceId().withId("lzijiufehgmvflnw")) + .withSliceConfigurations(Arrays.asList(new SliceConfiguration() + .withSlice(new SliceResourceId().withId("erlniylylyfw")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("zutgqztwhghmupg")) + .withDataNetworkConfigurations(Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("jtcdxabbujftaben")) + .withSessionAmbr(new Ambr().withUplink("bklqpxz").withDownlink("cafeddw")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("nlzafwxudgnh")) + .withSessionAmbr(new Ambr().withUplink("ookrtalvnbw").withDownlink("pbeme")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration().withDataNetwork(new DataNetworkResourceId().withId("uclvdjj")) + .withSessionAmbr(new Ambr().withUplink("kyrdnqodx").withDownlink("hhxhq")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("aqnvzoqgyipemchg")) + .withSessionAmbr(new Ambr().withUplink("v").withDownlink("czuejdtxptl")) + .withAllowedServices(Arrays.asList()))), + new SliceConfiguration().withSlice(new SliceResourceId().withId("h")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("zhomewjjstliu")) + .withDataNetworkConfigurations(Arrays.asList( + new DataNetworkConfiguration().withDataNetwork(new DataNetworkResourceId().withId("awmo")) + .withSessionAmbr(new Ambr().withUplink("ia").withDownlink("cz")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("vodrrslblxydkxr")) + .withSessionAmbr(new Ambr().withUplink("vvbxiwkgfbqljnq").withDownlink("hychocokuleh")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("rqlrqffawe")) + .withSessionAmbr(new Ambr().withUplink("urkphyjdxravju").withDownlink("dbrxmrgc")) + .withAllowedServices(Arrays.asList()))))) + .withTags(mapOf("scgdu", "fjjkbajb")).withRfspIndex(1717154026).withRegistrationTimer(1205812916) .create(); - Assertions.assertEquals("toi", response.location()); - Assertions.assertEquals("ygvfltgvdihoyn", response.tags().get("rxwe")); - Assertions.assertEquals("zregzgyufutrwpw", response.ueAmbr().uplink()); - Assertions.assertEquals("ryekzkd", response.ueAmbr().downlink()); - Assertions.assertEquals("meottawj", response.defaultSlice().id()); - Assertions.assertEquals(375034092, response.rfspIndex()); - Assertions.assertEquals(543459562, response.registrationTimer()); - Assertions.assertEquals("hnhjtfvpndpmi", response.sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("jpnwynudql", response.sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "sauzpjlx", - response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions - .assertEquals( - "ehuxiqhzlraym", - response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions - .assertEquals( - "zxlskihmxr", - response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals("l", response.location()); + Assertions.assertEquals("qvqyvweht", response.tags().get("emxhzzy")); + Assertions.assertEquals("fhbzffovwmbjlzq", response.ueAmbr().uplink()); + Assertions.assertEquals("czpgvdwnapfdq", response.ueAmbr().downlink()); + Assertions.assertEquals("owftptnuwjtks", response.defaultSlice().id()); + Assertions.assertEquals(2008228134, response.rfspIndex()); + Assertions.assertEquals(588054641, response.registrationTimer()); + Assertions.assertEquals("y", response.sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("leseyqrhvyeldo", response.sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("v", + response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("kwiswskukjtas", + response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("vwisp", + response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteMockTests.java index 4bb506a3e9dde..a6860f4cbefdb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.simPolicies().delete("bfw", "rzx", "dew", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.simPolicies().delete("cvogygzyvne", "zaifghtmoqqtlff", "zbkr", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetWithResponseMockTests.java index fecb15ed836c4..d78e178afdb2e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesGetWithResponseMockTests.java @@ -30,61 +30,42 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Canceled\",\"siteProvisioningState\":{\"qkbiwet\":\"Deleting\",\"zycyqiqyhg\":\"Deleting\",\"etzlexbsfledynoj\":\"Adding\",\"iuwfbzkkdtnhqsy\":\"Failed\"},\"ueAmbr\":{\"uplink\":\"ljselp\",\"downlink\":\"pbafvafhlbylc\"},\"defaultSlice\":{\"id\":\"bevxrhyzd\"},\"rfspIndex\":1616798113,\"registrationTimer\":2025630449,\"sliceConfigurations\":[{\"slice\":{\"id\":\"pl\"},\"defaultDataNetwork\":{\"id\":\"dbmairrhvhfnr\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"wnpqigtuujwouhd\"},\"sessionAmbr\":{\"uplink\":\"wsigrbjbxsj\",\"downlink\":\"bvitvqkjya\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"nu\"},\"sessionAmbr\":{\"uplink\":\"tggmuwdchozfnkfe\",\"downlink\":\"lv\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"noakiz\"},\"sessionAmbr\":{\"uplink\":\"oaiknaqlnuwig\",\"downlink\":\"yxl\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"kwph\"},\"sessionAmbr\":{\"uplink\":\"xzcwxhmpejt\",\"downlink\":\"ke\"},\"allowedServices\":[]}]}]},\"location\":\"onwivkcqhrxhxkn\",\"tags\":{\"yqjf\":\"rmmkyupiju\",\"jascowvfdjkpd\":\"akfqfrkemyildud\",\"zvyfijdkzuqnwsi\":\"phlkksnm\",\"ah\":\"huqol\"},\"id\":\"uqwqulsutrjbhxyk\",\"name\":\"hyqezvqq\",\"type\":\"g\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Deleting\",\"siteProvisioningState\":{\"bwetnjuhpsp\":\"Failed\"},\"ueAmbr\":{\"uplink\":\"kzyaup\",\"downlink\":\"accxnafbwqroohtu\"},\"defaultSlice\":{\"id\":\"vmaonurjt\"},\"rfspIndex\":399456929,\"registrationTimer\":1804037238,\"sliceConfigurations\":[{\"slice\":{\"id\":\"pv\"},\"defaultDataNetwork\":{\"id\":\"cmslclblyjxltbs\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"scvsfxigctm\"},\"sessionAmbr\":{\"uplink\":\"xu\",\"downlink\":\"pbezqccydrtceu\"},\"allowedServices\":[]}]}]},\"location\":\"qkkyihztgeqmg\",\"tags\":{\"lcecfeh\":\"wldoychi\",\"qllizstac\":\"waoaguhi\"},\"id\":\"jvhrweft\",\"name\":\"wqejpmvsse\",\"type\":\"aepwamcxtcz\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - SimPolicy response = - manager - .simPolicies() - .getWithResponse("rsxkr", "lbjazejww", "iyoypsuhbrnnhj", com.azure.core.util.Context.NONE) - .getValue(); + SimPolicy response = manager.simPolicies() + .getWithResponse("jjjavfqnvhnq", "ewdogiyetesy", "vidbztjhqtfb", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("onwivkcqhrxhxkn", response.location()); - Assertions.assertEquals("rmmkyupiju", response.tags().get("yqjf")); - Assertions.assertEquals("ljselp", response.ueAmbr().uplink()); - Assertions.assertEquals("pbafvafhlbylc", response.ueAmbr().downlink()); - Assertions.assertEquals("bevxrhyzd", response.defaultSlice().id()); - Assertions.assertEquals(1616798113, response.rfspIndex()); - Assertions.assertEquals(2025630449, response.registrationTimer()); - Assertions.assertEquals("pl", response.sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("dbmairrhvhfnr", response.sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "wnpqigtuujwouhd", - response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions - .assertEquals( - "wsigrbjbxsj", - response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions - .assertEquals( - "bvitvqkjya", - response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals("qkkyihztgeqmg", response.location()); + Assertions.assertEquals("wldoychi", response.tags().get("lcecfeh")); + Assertions.assertEquals("kzyaup", response.ueAmbr().uplink()); + Assertions.assertEquals("accxnafbwqroohtu", response.ueAmbr().downlink()); + Assertions.assertEquals("vmaonurjt", response.defaultSlice().id()); + Assertions.assertEquals(399456929, response.rfspIndex()); + Assertions.assertEquals(1804037238, response.registrationTimer()); + Assertions.assertEquals("pv", response.sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("cmslclblyjxltbs", response.sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("scvsfxigctm", + response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("xu", + response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("pbezqccydrtceu", + response.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkMockTests.java index 096ffb4d9b770..cd5de2e13fa53 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPoliciesListByMobileNetworkMockTests.java @@ -31,83 +31,42 @@ public void testListByMobileNetwork() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"siteProvisioningState\":{\"owtljvfw\":\"NotApplicable\",\"eagk\":\"Failed\",\"xv\":\"Adding\",\"tvbczsulm\":\"Adding\"},\"ueAmbr\":{\"uplink\":\"gglmepjpfsey\",\"downlink\":\"g\"},\"defaultSlice\":{\"id\":\"angpszngafpgyl\"},\"rfspIndex\":2068016278,\"registrationTimer\":73150877,\"sliceConfigurations\":[{\"slice\":{\"id\":\"jcngoadyed\"},\"defaultDataNetwork\":{\"id\":\"zrgjfok\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"bnoit\"},\"sessionAmbr\":{\"uplink\":\"kpztrgd\",\"downlink\":\"x\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"coqra\"},\"defaultDataNetwork\":{\"id\":\"wugyx\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"itweialwvskbuh\"},\"sessionAmbr\":{\"uplink\":\"acaqtyltcoqcu\",\"downlink\":\"pdsxzakuejkm\"},\"allowedServices\":[]}]}]},\"location\":\"iztjof\",\"tags\":{\"yeji\":\"ovjufycsjmlbe\",\"rtudawlpjfel\":\"iuxegth\",\"bgqnz\":\"erppt\",\"ialwc\":\"nhii\"},\"id\":\"gckbb\",\"name\":\"ccgzpraoxnyu\",\"type\":\"fa\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"siteProvisioningState\":{\"rtywi\":\"Updating\",\"mhlaku\":\"Updating\",\"gbhgau\":\"Failed\"},\"ueAmbr\":{\"uplink\":\"cdixmx\",\"downlink\":\"f\"},\"defaultSlice\":{\"id\":\"sryjqgdkfno\"},\"rfspIndex\":2036068845,\"registrationTimer\":374644844,\"sliceConfigurations\":[{\"slice\":{\"id\":\"vjhvefgwbm\"},\"defaultDataNetwork\":{\"id\":\"jchnta\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"ay\"},\"sessionAmbr\":{\"uplink\":\"xbulpzealbmqkyo\",\"downlink\":\"wyvf\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"mbtsuahxsg\"},\"sessionAmbr\":{\"uplink\":\"jcmmzrrscub\",\"downlink\":\"wsdrnpxqwodif\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"jxcjrmmuabwib\"},\"sessionAmbr\":{\"uplink\":\"jogjonmc\",\"downlink\":\"e\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"oyzbamwineofvf\"},\"defaultDataNetwork\":{\"id\":\"akpoldtvevbo\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"zhzjknyuxg\"},\"sessionAmbr\":{\"uplink\":\"ttxpnrupza\",\"downlink\":\"mrdixtreki\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"swyskbruffg\"},\"defaultDataNetwork\":{\"id\":\"lukkutvlxhrpqhvm\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"couqehb\"},\"sessionAmbr\":{\"uplink\":\"bcdsziryrandoyp\",\"downlink\":\"blto\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"rmkfqlwxldykals\"},\"sessionAmbr\":{\"uplink\":\"g\",\"downlink\":\"olnjpnnbmjksibj\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"sjjx\"},\"sessionAmbr\":{\"uplink\":\"ahmrnadzyqegxyi\",\"downlink\":\"pinbmhwbjijkgqxn\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"mbk\"},\"sessionAmbr\":{\"uplink\":\"znj\",\"downlink\":\"ujvaannggi\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"cwkdtaaw\"},\"defaultDataNetwork\":{\"id\":\"wfekaumrrqmb\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"qkratbnxwbjsid\"},\"sessionAmbr\":{\"uplink\":\"irkfpksokdgo\",\"downlink\":\"ewijymrhbguz\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"zkye\"},\"sessionAmbr\":{\"uplink\":\"nfnzhhh\",\"downlink\":\"o\"},\"allowedServices\":[]}]}]},\"location\":\"ffjkutycyarnroo\",\"tags\":{\"ocnhzqrottjzcfyj\":\"abzoghktdpyczhco\",\"rl\":\"pt\",\"pqinf\":\"h\"},\"id\":\"zpyglqdhmrj\",\"name\":\"ralcxpjbyypsj\",\"type\":\"qcjenkyhf\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.simPolicies().listByMobileNetwork("rftb", "ve", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.simPolicies().listByMobileNetwork("upeuknijduyye", "pydjfboc", com.azure.core.util.Context.NONE); - Assertions.assertEquals("iztjof", response.iterator().next().location()); - Assertions.assertEquals("ovjufycsjmlbe", response.iterator().next().tags().get("yeji")); - Assertions.assertEquals("gglmepjpfsey", response.iterator().next().ueAmbr().uplink()); - Assertions.assertEquals("g", response.iterator().next().ueAmbr().downlink()); - Assertions.assertEquals("angpszngafpgyl", response.iterator().next().defaultSlice().id()); - Assertions.assertEquals(2068016278, response.iterator().next().rfspIndex()); - Assertions.assertEquals(73150877, response.iterator().next().registrationTimer()); - Assertions.assertEquals("jcngoadyed", response.iterator().next().sliceConfigurations().get(0).slice().id()); - Assertions - .assertEquals("zrgjfok", response.iterator().next().sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "bnoit", - response - .iterator() - .next() - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .dataNetwork() - .id()); - Assertions - .assertEquals( - "kpztrgd", - response - .iterator() - .next() - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .sessionAmbr() - .uplink()); - Assertions - .assertEquals( - "x", - response - .iterator() - .next() - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .sessionAmbr() - .downlink()); + Assertions.assertEquals("ffjkutycyarnroo", response.iterator().next().location()); + Assertions.assertEquals("abzoghktdpyczhco", response.iterator().next().tags().get("ocnhzqrottjzcfyj")); + Assertions.assertEquals("cdixmx", response.iterator().next().ueAmbr().uplink()); + Assertions.assertEquals("f", response.iterator().next().ueAmbr().downlink()); + Assertions.assertEquals("sryjqgdkfno", response.iterator().next().defaultSlice().id()); + Assertions.assertEquals(2036068845, response.iterator().next().rfspIndex()); + Assertions.assertEquals(374644844, response.iterator().next().registrationTimer()); + Assertions.assertEquals("vjhvefgwbm", response.iterator().next().sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("jchnta", + response.iterator().next().sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("ay", response.iterator().next().sliceConfigurations().get(0) + .dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("xbulpzealbmqkyo", response.iterator().next().sliceConfigurations().get(0) + .dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("wyvf", response.iterator().next().sliceConfigurations().get(0) + .dataNetworkConfigurations().get(0).sessionAmbr().downlink()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyInnerTests.java index 59c776f89bd0b..20acaad8eee43 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyInnerTests.java @@ -23,312 +23,133 @@ public final class SimPolicyInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimPolicyInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Unknown\",\"siteProvisioningState\":{\"bsdshmkxmaehvbbx\":\"Provisioned\",\"iplt\":\"NotApplicable\"},\"ueAmbr\":{\"uplink\":\"n\",\"downlink\":\"tbaxk\"},\"defaultSlice\":{\"id\":\"xywr\"},\"rfspIndex\":1619197472,\"registrationTimer\":479149992,\"sliceConfigurations\":[{\"slice\":{\"id\":\"yhpluodpvruudlgz\"},\"defaultDataNetwork\":{\"id\":\"bth\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"tgk\"},\"sessionAmbr\":{\"uplink\":\"stvdxeclz\",\"downlink\":\"dqbcvhzlhplod\"},\"5qi\":1140864864,\"allocationAndRetentionPriorityLevel\":1592538634,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv6\"],\"allowedServices\":[{\"id\":\"qjfsmlmbtxhw\"},{\"id\":\"fwsrtawcoezbrhu\"}],\"maximumNumberOfBufferedPackets\":518235624},{\"dataNetwork\":{\"id\":\"hud\"},\"sessionAmbr\":{\"uplink\":\"goo\",\"downlink\":\"kkqfqjbvle\"},\"5qi\":1730138654,\"allocationAndRetentionPriorityLevel\":1618097313,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv6\",\"IPv6\"],\"allowedServices\":[{\"id\":\"qqybarye\"},{\"id\":\"ayjkqa\"}],\"maximumNumberOfBufferedPackets\":2002865497}]},{\"slice\":{\"id\":\"zslesjcbher\"},\"defaultDataNetwork\":{\"id\":\"ntiew\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"cv\"},\"sessionAmbr\":{\"uplink\":\"quwrbehwag\",\"downlink\":\"hbuffkmrq\"},\"5qi\":226189444,\"allocationAndRetentionPriorityLevel\":1393141923,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv4\"],\"allowedServices\":[{\"id\":\"oe\"},{\"id\":\"jvewzcjznmwcp\"},{\"id\":\"guaadraufactkahz\"}],\"maximumNumberOfBufferedPackets\":1219742698}]},{\"slice\":{\"id\":\"jjziuxxpsh\"},\"defaultDataNetwork\":{\"id\":\"eekulfgslqubkwd\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"nrdsutujbazpjuoh\"},\"sessionAmbr\":{\"uplink\":\"inyflnorwmduvwp\",\"downlink\":\"lvxwmyg\"},\"5qi\":1521698279,\"allocationAndRetentionPriorityLevel\":181907688,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv4\"],\"allowedServices\":[{\"id\":\"jcrxgibbdaxcon\"},{\"id\":\"ozauorsukokwb\"},{\"id\":\"plhlvnuuepzlrp\"},{\"id\":\"wzsoldweyuqdunv\"}],\"maximumNumberOfBufferedPackets\":221136034}]},{\"slice\":{\"id\":\"rwrbi\"},\"defaultDataNetwork\":{\"id\":\"rk\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"lywjhh\"},\"sessionAmbr\":{\"uplink\":\"dnhxmsi\",\"downlink\":\"fomiloxgg\"},\"5qi\":524951220,\"allocationAndRetentionPriorityLevel\":75081100,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv4\",\"IPv6\"],\"allowedServices\":[{\"id\":\"vcyy\"},{\"id\":\"s\"},{\"id\":\"gdotcubiipuipwo\"}],\"maximumNumberOfBufferedPackets\":1120696384},{\"dataNetwork\":{\"id\":\"macjekn\"},\"sessionAmbr\":{\"uplink\":\"zshq\",\"downlink\":\"cimpevfg\"},\"5qi\":910650574,\"allocationAndRetentionPriorityLevel\":1947593422,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv4\",\"IPv4\"],\"allowedServices\":[{\"id\":\"wrwfscjfnyns\"},{\"id\":\"qujizdvo\"}],\"maximumNumberOfBufferedPackets\":918297012},{\"dataNetwork\":{\"id\":\"ibyowbblgyavutp\"},\"sessionAmbr\":{\"uplink\":\"hjoxo\",\"downlink\":\"smsks\"},\"5qi\":791689891,\"allocationAndRetentionPriorityLevel\":1015646968,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv4\",\"IPv4\",\"IPv6\"],\"allowedServices\":[{\"id\":\"ffgcvizqz\"},{\"id\":\"wlvwlyoupf\"},{\"id\":\"fbkjubdyhgkfmi\"}],\"maximumNumberOfBufferedPackets\":87584157}]}]},\"location\":\"wzf\",\"tags\":{\"ctxtgzukxi\":\"ttktlahbq\",\"xrxc\":\"mmqtgqqqxhr\",\"azivjlfrqttbajl\":\"juisavokqdzf\"},\"id\":\"atnwxyiopi\",\"name\":\"kqqfk\",\"type\":\"vscx\"}") - .toObject(SimPolicyInner.class); - Assertions.assertEquals("wzf", model.location()); - Assertions.assertEquals("ttktlahbq", model.tags().get("ctxtgzukxi")); - Assertions.assertEquals("n", model.ueAmbr().uplink()); - Assertions.assertEquals("tbaxk", model.ueAmbr().downlink()); - Assertions.assertEquals("xywr", model.defaultSlice().id()); - Assertions.assertEquals(1619197472, model.rfspIndex()); - Assertions.assertEquals(479149992, model.registrationTimer()); - Assertions.assertEquals("yhpluodpvruudlgz", model.sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("bth", model.sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "tgk", model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions - .assertEquals( - "stvdxeclz", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions - .assertEquals( - "dqbcvhzlhplod", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); - Assertions - .assertEquals(1140864864, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); - Assertions - .assertEquals( - 1592538634, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.MAY_PREEMPT, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); - Assertions - .assertEquals( - PduSessionType.IPV6, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); - Assertions - .assertEquals( - PduSessionType.IPV4, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .additionalAllowedSessionTypes() - .get(0)); - Assertions - .assertEquals( - "qjfsmlmbtxhw", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); - Assertions - .assertEquals( - 518235624, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); + SimPolicyInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Deleted\",\"siteProvisioningState\":{\"rmbzo\":\"Updating\"},\"ueAmbr\":{\"uplink\":\"okixrjqcir\",\"downlink\":\"zpfrla\"},\"defaultSlice\":{\"id\":\"szrnwo\"},\"rfspIndex\":1723984044,\"registrationTimer\":427413462,\"sliceConfigurations\":[{\"slice\":{\"id\":\"wp\"},\"defaultDataNetwork\":{\"id\":\"ylwbtlhflsjcdhsz\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"vfbgofeljagrqmqh\"},\"sessionAmbr\":{\"uplink\":\"dvriiiojnal\",\"downlink\":\"hfkvtvsexsowuel\"},\"5qi\":1968234640,\"allocationAndRetentionPriorityLevel\":1709132634,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\"],\"allowedServices\":[{\"id\":\"pjgwwspug\"},{\"id\":\"ftqsxhqxujxuk\"}],\"maximumNumberOfBufferedPackets\":1615725858},{\"dataNetwork\":{\"id\":\"digrjguufzdmsyqt\"},\"sessionAmbr\":{\"uplink\":\"ihwhbotzingamvpp\",\"downlink\":\"o\"},\"5qi\":1799279870,\"allocationAndRetentionPriorityLevel\":1534302042,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv4\"],\"allowedServices\":[{\"id\":\"wynwcvtbvkayhm\"},{\"id\":\"nvyq\"},{\"id\":\"atkzwpcnpw\"},{\"id\":\"cjaesgvvs\"}],\"maximumNumberOfBufferedPackets\":1469869368},{\"dataNetwork\":{\"id\":\"ajguqf\"},\"sessionAmbr\":{\"uplink\":\"wygzlvdnkfxusem\",\"downlink\":\"wzrmuh\"},\"5qi\":1693108330,\"allocationAndRetentionPriorityLevel\":857980637,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv4\",\"IPv6\"],\"allowedServices\":[{\"id\":\"ymgc\"},{\"id\":\"elvezrypq\"},{\"id\":\"mfe\"}],\"maximumNumberOfBufferedPackets\":1181848977}]}]},\"location\":\"qwkyhkobop\",\"tags\":{\"pbqpcrfkbwccsn\":\"dkow\",\"htjsying\":\"vcdwxlpqekftn\",\"tmtdhtmdvypgik\":\"fq\",\"irryuzhlh\":\"gszywk\"},\"id\":\"joqrvqqaatj\",\"name\":\"nrvgoupmfiibfgg\",\"type\":\"ioolvrwxkvtkkgll\"}") + .toObject(SimPolicyInner.class); + Assertions.assertEquals("qwkyhkobop", model.location()); + Assertions.assertEquals("dkow", model.tags().get("pbqpcrfkbwccsn")); + Assertions.assertEquals("okixrjqcir", model.ueAmbr().uplink()); + Assertions.assertEquals("zpfrla", model.ueAmbr().downlink()); + Assertions.assertEquals("szrnwo", model.defaultSlice().id()); + Assertions.assertEquals(1723984044, model.rfspIndex()); + Assertions.assertEquals(427413462, model.registrationTimer()); + Assertions.assertEquals("wp", model.sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("ylwbtlhflsjcdhsz", model.sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("vfbgofeljagrqmqh", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("dvriiiojnal", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("hfkvtvsexsowuel", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals(1968234640, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); + Assertions.assertEquals(1709132634, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0) + .allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); + Assertions.assertEquals(PduSessionType.IPV4, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); + Assertions.assertEquals(PduSessionType.IPV6, model.sliceConfigurations().get(0).dataNetworkConfigurations() + .get(0).additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("pjgwwspug", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); + Assertions.assertEquals(1615725858, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimPolicyInner model = - new SimPolicyInner() - .withLocation("wzf") - .withTags(mapOf("ctxtgzukxi", "ttktlahbq", "xrxc", "mmqtgqqqxhr", "azivjlfrqttbajl", "juisavokqdzf")) - .withUeAmbr(new Ambr().withUplink("n").withDownlink("tbaxk")) - .withDefaultSlice(new SliceResourceId().withId("xywr")) - .withRfspIndex(1619197472) - .withRegistrationTimer(479149992) + SimPolicyInner model + = new SimPolicyInner().withLocation("qwkyhkobop") + .withTags(mapOf("pbqpcrfkbwccsn", "dkow", "htjsying", "vcdwxlpqekftn", "tmtdhtmdvypgik", "fq", + "irryuzhlh", "gszywk")) + .withUeAmbr(new Ambr().withUplink("okixrjqcir").withDownlink("zpfrla")) + .withDefaultSlice( + new SliceResourceId().withId("szrnwo")) + .withRfspIndex( + 1723984044) + .withRegistrationTimer( + 427413462) .withSliceConfigurations( Arrays - .asList( - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("yhpluodpvruudlgz")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("bth")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("tgk")) - .withSessionAmbr( - new Ambr().withUplink("stvdxeclz").withDownlink("dqbcvhzlhplod")) - .withFiveQi(1140864864) - .withAllocationAndRetentionPriorityLevel(1592538634) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays.asList(PduSessionType.IPV4, PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("qjfsmlmbtxhw"), - new ServiceResourceId().withId("fwsrtawcoezbrhu"))) - .withMaximumNumberOfBufferedPackets(518235624), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("hud")) - .withSessionAmbr( - new Ambr().withUplink("goo").withDownlink("kkqfqjbvle")) - .withFiveQi(1730138654) - .withAllocationAndRetentionPriorityLevel(1618097313) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV4, - PduSessionType.IPV6, - PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("qqybarye"), - new ServiceResourceId().withId("ayjkqa"))) - .withMaximumNumberOfBufferedPackets(2002865497))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("zslesjcbher")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("ntiew")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("cv")) - .withSessionAmbr( - new Ambr().withUplink("quwrbehwag").withDownlink("hbuffkmrq")) - .withFiveQi(226189444) - .withAllocationAndRetentionPriorityLevel(1393141923) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays.asList(PduSessionType.IPV4, PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("oe"), - new ServiceResourceId().withId("jvewzcjznmwcp"), - new ServiceResourceId().withId("guaadraufactkahz"))) - .withMaximumNumberOfBufferedPackets(1219742698))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("jjziuxxpsh")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("eekulfgslqubkwd")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("nrdsutujbazpjuoh")) - .withSessionAmbr( - new Ambr().withUplink("inyflnorwmduvwp").withDownlink("lvxwmyg")) - .withFiveQi(1521698279) - .withAllocationAndRetentionPriorityLevel(181907688) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays.asList(PduSessionType.IPV4, PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("jcrxgibbdaxcon"), - new ServiceResourceId().withId("ozauorsukokwb"), - new ServiceResourceId().withId("plhlvnuuepzlrp"), - new ServiceResourceId().withId("wzsoldweyuqdunv"))) - .withMaximumNumberOfBufferedPackets(221136034))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("rwrbi")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("rk")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("lywjhh")) - .withSessionAmbr( - new Ambr().withUplink("dnhxmsi").withDownlink("fomiloxgg")) - .withFiveQi(524951220) - .withAllocationAndRetentionPriorityLevel(75081100) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV4, - PduSessionType.IPV4, - PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("vcyy"), - new ServiceResourceId().withId("s"), - new ServiceResourceId().withId("gdotcubiipuipwo"))) - .withMaximumNumberOfBufferedPackets(1120696384), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("macjekn")) - .withSessionAmbr(new Ambr().withUplink("zshq").withDownlink("cimpevfg")) - .withFiveQi(910650574) - .withAllocationAndRetentionPriorityLevel(1947593422) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV4, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("wrwfscjfnyns"), - new ServiceResourceId().withId("qujizdvo"))) - .withMaximumNumberOfBufferedPackets(918297012), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("ibyowbblgyavutp")) - .withSessionAmbr(new Ambr().withUplink("hjoxo").withDownlink("smsks")) - .withFiveQi(791689891) - .withAllocationAndRetentionPriorityLevel(1015646968) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV4, - PduSessionType.IPV4, - PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("ffgcvizqz"), - new ServiceResourceId().withId("wlvwlyoupf"), - new ServiceResourceId().withId("fbkjubdyhgkfmi"))) - .withMaximumNumberOfBufferedPackets(87584157))))); + .asList(new SliceConfiguration().withSlice(new SliceResourceId().withId("wp")) + .withDefaultDataNetwork( + new DataNetworkResourceId().withId("ylwbtlhflsjcdhsz")) + .withDataNetworkConfigurations(Arrays.asList( + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("vfbgofeljagrqmqh")) + .withSessionAmbr( + new Ambr().withUplink("dvriiiojnal").withDownlink("hfkvtvsexsowuel")) + .withFiveQi(1968234640).withAllocationAndRetentionPriorityLevel(1709132634) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4) + .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV6)) + .withAllowedServices( + Arrays.asList(new ServiceResourceId().withId("pjgwwspug"), + new ServiceResourceId().withId("ftqsxhqxujxuk"))) + .withMaximumNumberOfBufferedPackets(1615725858), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("digrjguufzdmsyqt")) + .withSessionAmbr(new Ambr().withUplink("ihwhbotzingamvpp").withDownlink("o")) + .withFiveQi(1799279870).withAllocationAndRetentionPriorityLevel(1534302042) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4) + .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV4)) + .withAllowedServices( + Arrays.asList(new ServiceResourceId().withId("wynwcvtbvkayhm"), + new ServiceResourceId().withId("nvyq"), new ServiceResourceId() + .withId("atkzwpcnpw"), + new ServiceResourceId().withId("cjaesgvvs"))) + .withMaximumNumberOfBufferedPackets(1469869368), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("ajguqf")) + .withSessionAmbr(new Ambr().withUplink("wygzlvdnkfxusem").withDownlink("wzrmuh")) + .withFiveQi(1693108330).withAllocationAndRetentionPriorityLevel(857980637) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV6) + .withAdditionalAllowedSessionTypes( + Arrays.asList(PduSessionType.IPV4, PduSessionType.IPV4, PduSessionType.IPV6)) + .withAllowedServices(Arrays.asList(new ServiceResourceId().withId("ymgc"), + new ServiceResourceId().withId("elvezrypq"), + new ServiceResourceId().withId("mfe"))) + .withMaximumNumberOfBufferedPackets(1181848977))))); model = BinaryData.fromObject(model).toObject(SimPolicyInner.class); - Assertions.assertEquals("wzf", model.location()); - Assertions.assertEquals("ttktlahbq", model.tags().get("ctxtgzukxi")); - Assertions.assertEquals("n", model.ueAmbr().uplink()); - Assertions.assertEquals("tbaxk", model.ueAmbr().downlink()); - Assertions.assertEquals("xywr", model.defaultSlice().id()); - Assertions.assertEquals(1619197472, model.rfspIndex()); - Assertions.assertEquals(479149992, model.registrationTimer()); - Assertions.assertEquals("yhpluodpvruudlgz", model.sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("bth", model.sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "tgk", model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions - .assertEquals( - "stvdxeclz", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions - .assertEquals( - "dqbcvhzlhplod", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); - Assertions - .assertEquals(1140864864, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); - Assertions - .assertEquals( - 1592538634, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.MAY_PREEMPT, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.NOT_PREEMPTABLE, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); - Assertions - .assertEquals( - PduSessionType.IPV6, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); - Assertions - .assertEquals( - PduSessionType.IPV4, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .additionalAllowedSessionTypes() - .get(0)); - Assertions - .assertEquals( - "qjfsmlmbtxhw", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); - Assertions - .assertEquals( - 518235624, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); + Assertions.assertEquals("qwkyhkobop", model.location()); + Assertions.assertEquals("dkow", model.tags().get("pbqpcrfkbwccsn")); + Assertions.assertEquals("okixrjqcir", model.ueAmbr().uplink()); + Assertions.assertEquals("zpfrla", model.ueAmbr().downlink()); + Assertions.assertEquals("szrnwo", model.defaultSlice().id()); + Assertions.assertEquals(1723984044, model.rfspIndex()); + Assertions.assertEquals(427413462, model.registrationTimer()); + Assertions.assertEquals("wp", model.sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("ylwbtlhflsjcdhsz", model.sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("vfbgofeljagrqmqh", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("dvriiiojnal", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("hfkvtvsexsowuel", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals(1968234640, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); + Assertions.assertEquals(1709132634, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0) + .allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.MAY_PREEMPT, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); + Assertions.assertEquals(PduSessionType.IPV4, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); + Assertions.assertEquals(PduSessionType.IPV6, model.sliceConfigurations().get(0).dataNetworkConfigurations() + .get(0).additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("pjgwwspug", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); + Assertions.assertEquals(1615725858, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyListResultTests.java index 14599233cd1e7..ee8e28cb4a94d 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyListResultTests.java @@ -20,319 +20,165 @@ public final class SimPolicyListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimPolicyListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"siteProvisioningState\":{\"jaffmmfblcqcuu\":\"Adding\",\"qibrtalme\":\"Failed\",\"t\":\"Failed\"},\"ueAmbr\":{\"uplink\":\"gdslqxihhrmoo\",\"downlink\":\"z\"},\"defaultSlice\":{\"id\":\"seypxiutcxapz\"},\"rfspIndex\":1318130137,\"registrationTimer\":1111738970,\"sliceConfigurations\":[{\"slice\":{\"id\":\"o\"},\"defaultDataNetwork\":{\"id\":\"e\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"oxslh\"},\"sessionAmbr\":{\"uplink\":\"nhl\",\"downlink\":\"brqnkkzjcjb\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"rgaehvvibrxjj\"},\"sessionAmbr\":{\"uplink\":\"toqbeitpkxztmoob\",\"downlink\":\"lftidgfcwqmpim\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"qxzhem\"},\"sessionAmbr\":{\"uplink\":\"yhohujswtwkozzwc\",\"downlink\":\"lkb\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"wpfaj\"},\"defaultDataNetwork\":{\"id\":\"jwltlwtjjgu\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"alhsnvkc\"},\"sessionAmbr\":{\"uplink\":\"mxzrpoa\",\"downlink\":\"mlnwiaaomylweazu\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"cse\"},\"sessionAmbr\":{\"uplink\":\"hwwn\",\"downlink\":\"jhlfzswpchwahf\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"ousnfepgfewe\"},\"sessionAmbr\":{\"uplink\":\"wlyxgncxyk\",\"downlink\":\"hdjhlimmbcx\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"h\"},\"sessionAmbr\":{\"uplink\":\"cporxvxcjz\",\"downlink\":\"qizxfpxtgqscjavf\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"juhdqazkmtgguwpi\"},\"defaultDataNetwork\":{\"id\":\"r\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"civmmg\"},\"sessionAmbr\":{\"uplink\":\"f\",\"downlink\":\"fiwrxgkn\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"uvyinzqodfvpgs\"},\"sessionAmbr\":{\"uplink\":\"oxgsgbpfgzdjtx\",\"downlink\":\"zflbqvg\"},\"allowedServices\":[]}]}]},\"location\":\"vl\",\"tags\":{\"vetnwsdtutn\":\"cqusr\"},\"id\":\"lduycv\",\"name\":\"zhyrmewipmve\",\"type\":\"dxukuqgsjjxundxg\"},{\"properties\":{\"provisioningState\":\"Accepted\",\"siteProvisioningState\":{\"hzjhf\":\"Deleting\",\"hvvmuvgpmun\":\"Failed\",\"sx\":\"NotApplicable\",\"hfbuzjyihsasbhud\":\"Adding\"},\"ueAmbr\":{\"uplink\":\"pohyuemslynsqyr\",\"downlink\":\"foobrlttyms\"},\"defaultSlice\":{\"id\":\"nygq\"},\"rfspIndex\":1366289055,\"registrationTimer\":986277752,\"sliceConfigurations\":[{\"slice\":{\"id\":\"dzgtil\"},\"defaultDataNetwork\":{\"id\":\"xhnfhqly\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"jouw\"},\"sessionAmbr\":{\"uplink\":\"vk\",\"downlink\":\"oyzunbixxr\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"ikvcpwp\"},\"sessionAmbr\":{\"uplink\":\"clrcivtsox\",\"downlink\":\"rkenx\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"m\"},\"sessionAmbr\":{\"uplink\":\"yefrpmpdnqqska\",\"downlink\":\"ao\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"vmm\"},\"sessionAmbr\":{\"uplink\":\"npqfrtqlkzmeg\",\"downlink\":\"itgvkx\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"zyqdrfegcealzx\"},\"defaultDataNetwork\":{\"id\":\"hcans\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"oyqhlwig\"},\"sessionAmbr\":{\"uplink\":\"ivbkbxgomfaj\",\"downlink\":\"wasqvdaeyyg\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"xakjsqzhzb\"},\"defaultDataNetwork\":{\"id\":\"zkgimsid\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"sicddyvv\"},\"sessionAmbr\":{\"uplink\":\"skgfmocwahpq\",\"downlink\":\"atjeaahh\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"jhhn\"},\"defaultDataNetwork\":{\"id\":\"kzyb\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"jid\"},\"sessionAmbr\":{\"uplink\":\"ksyxkyxvxevblbje\",\"downlink\":\"nljlageuaulx\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"nsmjbnkppxynen\"},\"sessionAmbr\":{\"uplink\":\"svxeizzgwklnsr\",\"downlink\":\"ffeycx\"},\"allowedServices\":[]}]}]},\"location\":\"tpiymerteea\",\"tags\":{\"g\":\"qiekkkzddrt\",\"vrefdeesv\":\"ojbmxv\",\"s\":\"cuijpxt\",\"wsawddjibabxvi\":\"wprtu\"},\"id\":\"itvtzeexavo\",\"name\":\"tfgle\",\"type\":\"dmdqb\"}],\"nextLink\":\"ypq\"}") - .toObject(SimPolicyListResult.class); - Assertions.assertEquals("vl", model.value().get(0).location()); - Assertions.assertEquals("cqusr", model.value().get(0).tags().get("vetnwsdtutn")); - Assertions.assertEquals("gdslqxihhrmoo", model.value().get(0).ueAmbr().uplink()); - Assertions.assertEquals("z", model.value().get(0).ueAmbr().downlink()); - Assertions.assertEquals("seypxiutcxapz", model.value().get(0).defaultSlice().id()); - Assertions.assertEquals(1318130137, model.value().get(0).rfspIndex()); - Assertions.assertEquals(1111738970, model.value().get(0).registrationTimer()); - Assertions.assertEquals("o", model.value().get(0).sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("e", model.value().get(0).sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "oxslh", - model - .value() - .get(0) - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .dataNetwork() - .id()); - Assertions - .assertEquals( - "nhl", - model - .value() - .get(0) - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .sessionAmbr() - .uplink()); - Assertions - .assertEquals( - "brqnkkzjcjb", - model - .value() - .get(0) - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .sessionAmbr() - .downlink()); + SimPolicyListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"siteProvisioningState\":{\"cdm\":\"Deleting\",\"r\":\"NotApplicable\",\"lpijnkrxfrd\":\"NotApplicable\"},\"ueAmbr\":{\"uplink\":\"hcrat\",\"downlink\":\"zzronasx\"},\"defaultSlice\":{\"id\":\"ft\"},\"rfspIndex\":1206467391,\"registrationTimer\":418200584,\"sliceConfigurations\":[{\"slice\":{\"id\":\"f\"},\"defaultDataNetwork\":{\"id\":\"wesgogczh\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"nxkrlgnyhmossxkk\"},\"sessionAmbr\":{\"uplink\":\"thrrgh\",\"downlink\":\"jbdhqxvc\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"gf\"},\"defaultDataNetwork\":{\"id\":\"pdso\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"shrnsvbuswdvz\"},\"sessionAmbr\":{\"uplink\":\"ybycnunvj\",\"downlink\":\"rtkfawnopq\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"ikyzirtxdy\"},\"sessionAmbr\":{\"uplink\":\"x\",\"downlink\":\"ejnt\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"sewgioilqukr\"},\"sessionAmbr\":{\"uplink\":\"dxtqmieoxo\",\"downlink\":\"ggufhyaomtb\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"hhavgrvkffovjz\"},\"defaultDataNetwork\":{\"id\":\"pjbi\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"jmfxumvf\"},\"sessionAmbr\":{\"uplink\":\"luyovwxnbkfezzx\",\"downlink\":\"cy\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"wzdgirujbzbo\"},\"sessionAmbr\":{\"uplink\":\"vzzbtdcq\",\"downlink\":\"pniyujviyl\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"dshf\"},\"sessionAmbr\":{\"uplink\":\"snrbgyefrymsgao\",\"downlink\":\"fmwncotmrfh\"},\"allowedServices\":[]}]}]},\"location\":\"ctymoxoftp\",\"tags\":{\"hxacpqjlihhyu\":\"wycz\",\"x\":\"pskasdvlmfwdg\"},\"id\":\"lucvpam\",\"name\":\"sreuzvxurisjnh\",\"type\":\"ytxifqjzgxmrh\"},{\"properties\":{\"provisioningState\":\"Failed\",\"siteProvisioningState\":{\"esutrgjupauutpw\":\"Deleting\",\"hihejqgwzpnfqn\":\"NotApplicable\"},\"ueAmbr\":{\"uplink\":\"cypsxjv\",\"downlink\":\"oimwkslirc\"},\"defaultSlice\":{\"id\":\"zjxvydfcea\"},\"rfspIndex\":532238686,\"registrationTimer\":1877895410,\"sliceConfigurations\":[{\"slice\":{\"id\":\"gdyftumrtwna\"},\"defaultDataNetwork\":{\"id\":\"jslb\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"kojgcyzts\"},\"sessionAmbr\":{\"uplink\":\"mznbaeqphch\",\"downlink\":\"nrnrp\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"ehuwrykqgaifmvik\"},\"sessionAmbr\":{\"uplink\":\"bydvkhbejdz\",\"downlink\":\"xcv\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"srhnjivo\"},\"sessionAmbr\":{\"uplink\":\"v\",\"downlink\":\"novqfzge\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"jdftuljltd\"},\"sessionAmbr\":{\"uplink\":\"ceamtm\",\"downlink\":\"zuo\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"ejwcwwqiok\"},\"defaultDataNetwork\":{\"id\":\"ssxmojms\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"kjprvk\"},\"sessionAmbr\":{\"uplink\":\"cfzq\",\"downlink\":\"jyxgtczh\"},\"allowedServices\":[]}]},{\"slice\":{\"id\":\"ydbsd\"},\"defaultDataNetwork\":{\"id\":\"hmkxmaehvbb\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"ripltf\"},\"sessionAmbr\":{\"uplink\":\"htba\",\"downlink\":\"kgxywr\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"kpyklyhp\"},\"sessionAmbr\":{\"uplink\":\"uodpv\",\"downlink\":\"uudl\"},\"allowedServices\":[]},{\"dataNetwork\":{\"id\":\"zibt\"},\"sessionAmbr\":{\"uplink\":\"ostgkts\",\"downlink\":\"vdxec\"},\"allowedServices\":[]}]}]},\"location\":\"edqbc\",\"tags\":{\"dqkdlwwqfbu\":\"lhpl\"},\"id\":\"lkxt\",\"name\":\"qjfsmlmbtxhw\",\"type\":\"fwsrtawcoezbrhu\"}],\"nextLink\":\"kh\"}") + .toObject(SimPolicyListResult.class); + Assertions.assertEquals("ctymoxoftp", model.value().get(0).location()); + Assertions.assertEquals("wycz", model.value().get(0).tags().get("hxacpqjlihhyu")); + Assertions.assertEquals("hcrat", model.value().get(0).ueAmbr().uplink()); + Assertions.assertEquals("zzronasx", model.value().get(0).ueAmbr().downlink()); + Assertions.assertEquals("ft", model.value().get(0).defaultSlice().id()); + Assertions.assertEquals(1206467391, model.value().get(0).rfspIndex()); + Assertions.assertEquals(418200584, model.value().get(0).registrationTimer()); + Assertions.assertEquals("f", model.value().get(0).sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("wesgogczh", + model.value().get(0).sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("nxkrlgnyhmossxkk", + model.value().get(0).sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("thrrgh", model.value().get(0).sliceConfigurations().get(0).dataNetworkConfigurations() + .get(0).sessionAmbr().uplink()); + Assertions.assertEquals("jbdhqxvc", model.value().get(0).sliceConfigurations().get(0) + .dataNetworkConfigurations().get(0).sessionAmbr().downlink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimPolicyListResult model = - new SimPolicyListResult() + SimPolicyListResult model + = new SimPolicyListResult() .withValue( Arrays .asList( - new SimPolicyInner() - .withLocation("vl") - .withTags(mapOf("vetnwsdtutn", "cqusr")) - .withUeAmbr(new Ambr().withUplink("gdslqxihhrmoo").withDownlink("z")) - .withDefaultSlice(new SliceResourceId().withId("seypxiutcxapz")) - .withRfspIndex(1318130137) - .withRegistrationTimer(1111738970) + new SimPolicyInner().withLocation("ctymoxoftp") + .withTags(mapOf("hxacpqjlihhyu", "wycz", "x", "pskasdvlmfwdg")) + .withUeAmbr(new Ambr().withUplink("hcrat").withDownlink("zzronasx")) + .withDefaultSlice(new SliceResourceId().withId("ft")).withRfspIndex( + 1206467391) + .withRegistrationTimer(418200584).withSliceConfigurations( + Arrays.asList( + new SliceConfiguration().withSlice(new SliceResourceId().withId("f")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("wesgogczh")) + .withDataNetworkConfigurations(Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("nxkrlgnyhmossxkk")) + .withSessionAmbr( + new Ambr().withUplink("thrrgh").withDownlink("jbdhqxvc")) + .withAllowedServices(Arrays.asList()))), + new SliceConfiguration().withSlice(new SliceResourceId().withId("gf")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("pdso")) + .withDataNetworkConfigurations(Arrays.asList( + new DataNetworkConfiguration() + .withDataNetwork( + new DataNetworkResourceId().withId("shrnsvbuswdvz")) + .withSessionAmbr( + new Ambr().withUplink("ybycnunvj").withDownlink("rtkfawnopq")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("ikyzirtxdy")) + .withSessionAmbr(new Ambr().withUplink("x").withDownlink("ejnt")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("sewgioilqukr")) + .withSessionAmbr( + new Ambr().withUplink("dxtqmieoxo").withDownlink("ggufhyaomtb")) + .withAllowedServices(Arrays.asList()))), + new SliceConfiguration() + .withSlice(new SliceResourceId().withId("hhavgrvkffovjz")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("pjbi")) + .withDataNetworkConfigurations(Arrays.asList( + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("jmfxumvf")) + .withSessionAmbr( + new Ambr().withUplink("luyovwxnbkfezzx").withDownlink("cy")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("wzdgirujbzbo")) + .withSessionAmbr( + new Ambr().withUplink("vzzbtdcq").withDownlink("pniyujviyl")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("dshf")) + .withSessionAmbr(new Ambr().withUplink("snrbgyefrymsgao") + .withDownlink("fmwncotmrfh")) + .withAllowedServices(Arrays.asList()))))), + new SimPolicyInner().withLocation("edqbc").withTags(mapOf("dqkdlwwqfbu", "lhpl")) + .withUeAmbr(new Ambr().withUplink("cypsxjv").withDownlink("oimwkslirc")) + .withDefaultSlice(new SliceResourceId().withId("zjxvydfcea")).withRfspIndex(532238686) + .withRegistrationTimer( + 1877895410) .withSliceConfigurations( Arrays .asList( new SliceConfiguration() - .withSlice(new SliceResourceId().withId("o")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("e")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("oxslh")) - .withSessionAmbr( - new Ambr() - .withUplink("nhl") - .withDownlink("brqnkkzjcjb")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("rgaehvvibrxjj")) - .withSessionAmbr( - new Ambr() - .withUplink("toqbeitpkxztmoob") - .withDownlink("lftidgfcwqmpim")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("qxzhem")) - .withSessionAmbr( - new Ambr() - .withUplink("yhohujswtwkozzwc") - .withDownlink("lkb")) - .withAllowedServices(Arrays.asList()))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("wpfaj")) - .withDefaultDataNetwork( - new DataNetworkResourceId().withId("jwltlwtjjgu")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("alhsnvkc")) - .withSessionAmbr( - new Ambr() - .withUplink("mxzrpoa") - .withDownlink("mlnwiaaomylweazu")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("cse")) - .withSessionAmbr( - new Ambr() - .withUplink("hwwn") - .withDownlink("jhlfzswpchwahf")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("ousnfepgfewe")) - .withSessionAmbr( - new Ambr() - .withUplink("wlyxgncxyk") - .withDownlink("hdjhlimmbcx")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("h")) - .withSessionAmbr( - new Ambr() - .withUplink("cporxvxcjz") - .withDownlink("qizxfpxtgqscjavf")) - .withAllowedServices(Arrays.asList()))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("juhdqazkmtgguwpi")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("r")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("civmmg")) - .withSessionAmbr( - new Ambr().withUplink("f").withDownlink("fiwrxgkn")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId() - .withId("uvyinzqodfvpgs")) - .withSessionAmbr( - new Ambr() - .withUplink("oxgsgbpfgzdjtx") - .withDownlink("zflbqvg")) - .withAllowedServices(Arrays.asList()))))), - new SimPolicyInner() - .withLocation("tpiymerteea") - .withTags( - mapOf( - "g", - "qiekkkzddrt", - "vrefdeesv", - "ojbmxv", - "s", - "cuijpxt", - "wsawddjibabxvi", - "wprtu")) - .withUeAmbr(new Ambr().withUplink("pohyuemslynsqyr").withDownlink("foobrlttyms")) - .withDefaultSlice(new SliceResourceId().withId("nygq")) - .withRfspIndex(1366289055) - .withRegistrationTimer(986277752) - .withSliceConfigurations( - Arrays - .asList( - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("dzgtil")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("xhnfhqly")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("jouw")) - .withSessionAmbr( - new Ambr() - .withUplink("vk") - .withDownlink("oyzunbixxr")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("ikvcpwp")) - .withSessionAmbr( - new Ambr() - .withUplink("clrcivtsox") - .withDownlink("rkenx")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("m")) - .withSessionAmbr( - new Ambr() - .withUplink("yefrpmpdnqqska") - .withDownlink("ao")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("vmm")) - .withSessionAmbr( - new Ambr() - .withUplink("npqfrtqlkzmeg") - .withDownlink("itgvkx")) - .withAllowedServices(Arrays.asList()))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("zyqdrfegcealzx")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("hcans")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("oyqhlwig")) - .withSessionAmbr( - new Ambr() - .withUplink("ivbkbxgomfaj") - .withDownlink("wasqvdaeyyg")) - .withAllowedServices(Arrays.asList()))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("xakjsqzhzb")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("zkgimsid")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("sicddyvv")) - .withSessionAmbr( - new Ambr() - .withUplink("skgfmocwahpq") - .withDownlink("atjeaahh")) - .withAllowedServices(Arrays.asList()))), + .withSlice(new SliceResourceId().withId("gdyftumrtwna")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("jslb")) + .withDataNetworkConfigurations(Arrays.asList( + new DataNetworkConfiguration() + .withDataNetwork( + new DataNetworkResourceId().withId("kojgcyzts")) + .withSessionAmbr( + new Ambr().withUplink("mznbaeqphch").withDownlink("nrnrp")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork( + new DataNetworkResourceId().withId("ehuwrykqgaifmvik")) + .withSessionAmbr( + new Ambr().withUplink("bydvkhbejdz").withDownlink("xcv")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("srhnjivo")) + .withSessionAmbr( + new Ambr().withUplink("v").withDownlink("novqfzge")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork( + new DataNetworkResourceId().withId("jdftuljltd")) + .withSessionAmbr( + new Ambr().withUplink("ceamtm").withDownlink("zuo")) + .withAllowedServices(Arrays.asList()))), new SliceConfiguration() - .withSlice(new SliceResourceId().withId("jhhn")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("kzyb")) + .withSlice(new SliceResourceId().withId("ejwcwwqiok")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("ssxmojms")) .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId().withId("jid")) - .withSessionAmbr( - new Ambr() - .withUplink("ksyxkyxvxevblbje") - .withDownlink("nljlageuaulx")) - .withAllowedServices(Arrays.asList()), - new DataNetworkConfiguration() - .withDataNetwork( - new DataNetworkResourceId() - .withId("nsmjbnkppxynen")) - .withSessionAmbr( - new Ambr() - .withUplink("svxeizzgwklnsr") - .withDownlink("ffeycx")) - .withAllowedServices(Arrays.asList()))))))); + Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("kjprvk")) + .withSessionAmbr( + new Ambr().withUplink("cfzq").withDownlink("jyxgtczh")) + .withAllowedServices(Arrays.asList()))), + new SliceConfiguration().withSlice(new SliceResourceId().withId("ydbsd")) + .withDefaultDataNetwork( + new DataNetworkResourceId().withId("hmkxmaehvbb")) + .withDataNetworkConfigurations(Arrays.asList( + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("ripltf")) + .withSessionAmbr( + new Ambr().withUplink("htba").withDownlink("kgxywr")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("kpyklyhp")) + .withSessionAmbr( + new Ambr().withUplink("uodpv").withDownlink("uudl")) + .withAllowedServices(Arrays.asList()), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("zibt")) + .withSessionAmbr( + new Ambr().withUplink("ostgkts").withDownlink("vdxec")) + .withAllowedServices(Arrays.asList()))))))); model = BinaryData.fromObject(model).toObject(SimPolicyListResult.class); - Assertions.assertEquals("vl", model.value().get(0).location()); - Assertions.assertEquals("cqusr", model.value().get(0).tags().get("vetnwsdtutn")); - Assertions.assertEquals("gdslqxihhrmoo", model.value().get(0).ueAmbr().uplink()); - Assertions.assertEquals("z", model.value().get(0).ueAmbr().downlink()); - Assertions.assertEquals("seypxiutcxapz", model.value().get(0).defaultSlice().id()); - Assertions.assertEquals(1318130137, model.value().get(0).rfspIndex()); - Assertions.assertEquals(1111738970, model.value().get(0).registrationTimer()); - Assertions.assertEquals("o", model.value().get(0).sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("e", model.value().get(0).sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "oxslh", - model - .value() - .get(0) - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .dataNetwork() - .id()); - Assertions - .assertEquals( - "nhl", - model - .value() - .get(0) - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .sessionAmbr() - .uplink()); - Assertions - .assertEquals( - "brqnkkzjcjb", - model - .value() - .get(0) - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .sessionAmbr() - .downlink()); + Assertions.assertEquals("ctymoxoftp", model.value().get(0).location()); + Assertions.assertEquals("wycz", model.value().get(0).tags().get("hxacpqjlihhyu")); + Assertions.assertEquals("hcrat", model.value().get(0).ueAmbr().uplink()); + Assertions.assertEquals("zzronasx", model.value().get(0).ueAmbr().downlink()); + Assertions.assertEquals("ft", model.value().get(0).defaultSlice().id()); + Assertions.assertEquals(1206467391, model.value().get(0).rfspIndex()); + Assertions.assertEquals(418200584, model.value().get(0).registrationTimer()); + Assertions.assertEquals("f", model.value().get(0).sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("wesgogczh", + model.value().get(0).sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("nxkrlgnyhmossxkk", + model.value().get(0).sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("thrrgh", model.value().get(0).sliceConfigurations().get(0).dataNetworkConfigurations() + .get(0).sessionAmbr().uplink()); + Assertions.assertEquals("jbdhqxvc", model.value().get(0).sliceConfigurations().get(0) + .dataNetworkConfigurations().get(0).sessionAmbr().downlink()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyPropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyPropertiesFormatTests.java index ef9dad5da8cf2..f0c61ae21175e 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyPropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyPropertiesFormatTests.java @@ -21,359 +21,127 @@ public final class SimPolicyPropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimPolicyPropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Unknown\",\"siteProvisioningState\":{\"ovibrxkp\":\"Updating\",\"oaz\":\"Provisioned\"},\"ueAmbr\":{\"uplink\":\"ruocbgo\",\"downlink\":\"rb\"},\"defaultSlice\":{\"id\":\"eoybfhjxakvvjgs\"},\"rfspIndex\":36886384,\"registrationTimer\":417144058,\"sliceConfigurations\":[{\"slice\":{\"id\":\"mywwtkgkxnyed\"},\"defaultDataNetwork\":{\"id\":\"b\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"vudtjuewbcihx\"},\"sessionAmbr\":{\"uplink\":\"uwhcjyxccybv\",\"downlink\":\"ayakkudzpx\"},\"5qi\":431593989,\"allocationAndRetentionPriorityLevel\":710163355,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv6\",\"IPv4\",\"IPv4\"],\"allowedServices\":[{\"id\":\"rkdbdgiogsjkmnwq\"},{\"id\":\"nobaiyhddviacegf\"},{\"id\":\"m\"},{\"id\":\"tfpmvmemfnczdw\"}],\"maximumNumberOfBufferedPackets\":201856710},{\"dataNetwork\":{\"id\":\"alxlllchp\"},\"sessionAmbr\":{\"uplink\":\"db\",\"downlink\":\"evwrdnhfuk\"},\"5qi\":1934709471,\"allocationAndRetentionPriorityLevel\":437386763,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv6\"],\"allowedServices\":[{\"id\":\"ypfcvlerchpqbmf\"},{\"id\":\"jbabwidf\"}],\"maximumNumberOfBufferedPackets\":1264081959},{\"dataNetwork\":{\"id\":\"spuunnoxyhkxgq\"},\"sessionAmbr\":{\"uplink\":\"drihpfhoqcaaewda\",\"downlink\":\"mdjvlpj\"},\"5qi\":1119660278,\"allocationAndRetentionPriorityLevel\":785056478,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv6\",\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"dncj\"},{\"id\":\"xonbzoggculapz\"}],\"maximumNumberOfBufferedPackets\":350819792}]},{\"slice\":{\"id\":\"pgogtqxepny\"},\"defaultDataNetwork\":{\"id\":\"b\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"ajlyjtlvofqzhv\"},\"sessionAmbr\":{\"uplink\":\"cib\",\"downlink\":\"fmo\"},\"5qi\":1885560912,\"allocationAndRetentionPriorityLevel\":2073572107,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv6\",\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"zjb\"},{\"id\":\"yzsxjrkambtrne\"}],\"maximumNumberOfBufferedPackets\":1525920075},{\"dataNetwork\":{\"id\":\"nvuqeqvldspa\"},\"sessionAmbr\":{\"uplink\":\"tjb\",\"downlink\":\"kdmflvestmjlx\"},\"5qi\":1811256055,\"allocationAndRetentionPriorityLevel\":1713301534,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv6\",\"IPv6\",\"IPv6\"],\"allowedServices\":[{\"id\":\"wk\"},{\"id\":\"ziycslevufuztck\"},{\"id\":\"yhjtqedcgzu\"}],\"maximumNumberOfBufferedPackets\":1160577956}]},{\"slice\":{\"id\":\"mrqzzrrjvpgl\"},\"defaultDataNetwork\":{\"id\":\"dzgkr\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"eevt\"},\"sessionAmbr\":{\"uplink\":\"epr\",\"downlink\":\"utnwytpzdmovzvf\"},\"5qi\":1925116648,\"allocationAndRetentionPriorityLevel\":1341644400,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\"],\"allowedServices\":[{\"id\":\"glae\"},{\"id\":\"xndticokpvzmlqtm\"}],\"maximumNumberOfBufferedPackets\":763019330},{\"dataNetwork\":{\"id\":\"x\"},\"sessionAmbr\":{\"uplink\":\"bf\",\"downlink\":\"rclnpkc\"},\"5qi\":1931804941,\"allocationAndRetentionPriorityLevel\":1808660907,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv4\",\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"xqvkjlmxhomdyn\"},{\"id\":\"dwdigumb\"},{\"id\":\"raauzzpt\"},{\"id\":\"a\"}],\"maximumNumberOfBufferedPackets\":2034990044}]},{\"slice\":{\"id\":\"dz\"},\"defaultDataNetwork\":{\"id\":\"ezwwv\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"qyuvvfonkp\"},\"sessionAmbr\":{\"uplink\":\"hqyikvy\",\"downlink\":\"auy\"},\"5qi\":2048419194,\"allocationAndRetentionPriorityLevel\":712290061,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv4\",\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"o\"},{\"id\":\"krsgsgb\"},{\"id\":\"huzqgn\"}],\"maximumNumberOfBufferedPackets\":308026039},{\"dataNetwork\":{\"id\":\"kynscliqhzv\"},\"sessionAmbr\":{\"uplink\":\"xnkomtkubo\",\"downlink\":\"ppnvdxz\"},\"5qi\":685064266,\"allocationAndRetentionPriorityLevel\":1207165403,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv6\",\"IPv4\",\"IPv4\"],\"allowedServices\":[{\"id\":\"hlfkqojpy\"},{\"id\":\"vgtrdcnifmzzs\"},{\"id\":\"ymbrnysuxmpraf\"},{\"id\":\"g\"}],\"maximumNumberOfBufferedPackets\":1986449760}]}]}") - .toObject(SimPolicyPropertiesFormat.class); - Assertions.assertEquals("ruocbgo", model.ueAmbr().uplink()); - Assertions.assertEquals("rb", model.ueAmbr().downlink()); - Assertions.assertEquals("eoybfhjxakvvjgs", model.defaultSlice().id()); - Assertions.assertEquals(36886384, model.rfspIndex()); - Assertions.assertEquals(417144058, model.registrationTimer()); - Assertions.assertEquals("mywwtkgkxnyed", model.sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("b", model.sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "vudtjuewbcihx", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions - .assertEquals( - "uwhcjyxccybv", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions - .assertEquals( - "ayakkudzpx", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); - Assertions - .assertEquals(431593989, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); - Assertions - .assertEquals( - 710163355, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); - Assertions - .assertEquals( - PduSessionType.IPV6, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); - Assertions - .assertEquals( - PduSessionType.IPV6, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .additionalAllowedSessionTypes() - .get(0)); - Assertions - .assertEquals( - "rkdbdgiogsjkmnwq", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); - Assertions - .assertEquals( - 201856710, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); + SimPolicyPropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Unknown\",\"siteProvisioningState\":{\"jayvblmhv\":\"NotApplicable\",\"uhbxvvy\":\"Updating\",\"s\":\"Failed\"},\"ueAmbr\":{\"uplink\":\"pbyrqufegxu\",\"downlink\":\"wz\"},\"defaultSlice\":{\"id\":\"bnhlmc\"},\"rfspIndex\":25336970,\"registrationTimer\":1814176762,\"sliceConfigurations\":[{\"slice\":{\"id\":\"itvgbmhrixkwm\"},\"defaultDataNetwork\":{\"id\":\"ijejvegrhbpn\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"xexccbdreaxhcexd\"},\"sessionAmbr\":{\"uplink\":\"rvqahqkghtpwi\",\"downlink\":\"nhyjsv\"},\"5qi\":758254768,\"allocationAndRetentionPriorityLevel\":146536566,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"gjqppy\"}],\"maximumNumberOfBufferedPackets\":768184156},{\"dataNetwork\":{\"id\":\"ronzmyhgfip\"},\"sessionAmbr\":{\"uplink\":\"sxkm\",\"downlink\":\"waekrrjreafxtsgu\"},\"5qi\":1263462710,\"allocationAndRetentionPriorityLevel\":610202176,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\"],\"allowedServices\":[{\"id\":\"qpvuzlmvfelf\"}],\"maximumNumberOfBufferedPackets\":2091794134}]},{\"slice\":{\"id\":\"plcrpwjxeznoig\"},\"defaultDataNetwork\":{\"id\":\"rnjwmw\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"nbsazejjoqkag\"},\"sessionAmbr\":{\"uplink\":\"hsxttaugzxnf\",\"downlink\":\"azpxdtnkdmkqjjl\"},\"5qi\":1899473333,\"allocationAndRetentionPriorityLevel\":1206033090,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv6\"],\"allowedServices\":[{\"id\":\"bqaays\"},{\"id\":\"kixqtnqtt\"},{\"id\":\"zlwfffiakp\"},{\"id\":\"pqqmted\"}],\"maximumNumberOfBufferedPackets\":1028485886}]}]}") + .toObject(SimPolicyPropertiesFormat.class); + Assertions.assertEquals("pbyrqufegxu", model.ueAmbr().uplink()); + Assertions.assertEquals("wz", model.ueAmbr().downlink()); + Assertions.assertEquals("bnhlmc", model.defaultSlice().id()); + Assertions.assertEquals(25336970, model.rfspIndex()); + Assertions.assertEquals(1814176762, model.registrationTimer()); + Assertions.assertEquals("itvgbmhrixkwm", model.sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("ijejvegrhbpn", model.sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("xexccbdreaxhcexd", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("rvqahqkghtpwi", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("nhyjsv", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals(758254768, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); + Assertions.assertEquals(146536566, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0) + .allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); + Assertions.assertEquals(PduSessionType.IPV4, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); + Assertions.assertEquals(PduSessionType.IPV6, model.sliceConfigurations().get(0).dataNetworkConfigurations() + .get(0).additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("gjqppy", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); + Assertions.assertEquals(768184156, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimPolicyPropertiesFormat model = - new SimPolicyPropertiesFormat() - .withUeAmbr(new Ambr().withUplink("ruocbgo").withDownlink("rb")) - .withDefaultSlice(new SliceResourceId().withId("eoybfhjxakvvjgs")) - .withRfspIndex(36886384) - .withRegistrationTimer(417144058) - .withSliceConfigurations( + SimPolicyPropertiesFormat model + = new SimPolicyPropertiesFormat() + .withUeAmbr( + new Ambr().withUplink("pbyrqufegxu").withDownlink("wz")) + .withDefaultSlice(new SliceResourceId().withId("bnhlmc")).withRfspIndex( + 25336970) + .withRegistrationTimer(1814176762).withSliceConfigurations( Arrays .asList( - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("mywwtkgkxnyed")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("b")) + new SliceConfiguration().withSlice(new SliceResourceId().withId("itvgbmhrixkwm")) + .withDefaultDataNetwork( + new DataNetworkResourceId().withId("ijejvegrhbpn")) .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("vudtjuewbcihx")) - .withSessionAmbr( - new Ambr().withUplink("uwhcjyxccybv").withDownlink("ayakkudzpx")) - .withFiveQi(431593989) - .withAllocationAndRetentionPriorityLevel(710163355) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV4, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("rkdbdgiogsjkmnwq"), - new ServiceResourceId().withId("nobaiyhddviacegf"), - new ServiceResourceId().withId("m"), - new ServiceResourceId().withId("tfpmvmemfnczdw"))) - .withMaximumNumberOfBufferedPackets(201856710), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("alxlllchp")) - .withSessionAmbr(new Ambr().withUplink("db").withDownlink("evwrdnhfuk")) - .withFiveQi(1934709471) - .withAllocationAndRetentionPriorityLevel(437386763) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays.asList(PduSessionType.IPV6, PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("ypfcvlerchpqbmf"), - new ServiceResourceId().withId("jbabwidf"))) - .withMaximumNumberOfBufferedPackets(1264081959), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("spuunnoxyhkxgq")) - .withSessionAmbr( - new Ambr().withUplink("drihpfhoqcaaewda").withDownlink("mdjvlpj")) - .withFiveQi(1119660278) - .withAllocationAndRetentionPriorityLevel(785056478) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("dncj"), - new ServiceResourceId().withId("xonbzoggculapz"))) - .withMaximumNumberOfBufferedPackets(350819792))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("pgogtqxepny")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("b")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("ajlyjtlvofqzhv")) - .withSessionAmbr(new Ambr().withUplink("cib").withDownlink("fmo")) - .withFiveQi(1885560912) - .withAllocationAndRetentionPriorityLevel(2073572107) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("zjb"), - new ServiceResourceId().withId("yzsxjrkambtrne"))) - .withMaximumNumberOfBufferedPackets(1525920075), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("nvuqeqvldspa")) - .withSessionAmbr( - new Ambr().withUplink("tjb").withDownlink("kdmflvestmjlx")) - .withFiveQi(1811256055) - .withAllocationAndRetentionPriorityLevel(1713301534) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV6, - PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("wk"), - new ServiceResourceId().withId("ziycslevufuztck"), - new ServiceResourceId().withId("yhjtqedcgzu"))) - .withMaximumNumberOfBufferedPackets(1160577956))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("mrqzzrrjvpgl")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("dzgkr")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("eevt")) - .withSessionAmbr( - new Ambr().withUplink("epr").withDownlink("utnwytpzdmovzvf")) - .withFiveQi(1925116648) - .withAllocationAndRetentionPriorityLevel(1341644400) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("glae"), - new ServiceResourceId().withId("xndticokpvzmlqtm"))) - .withMaximumNumberOfBufferedPackets(763019330), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("x")) - .withSessionAmbr(new Ambr().withUplink("bf").withDownlink("rclnpkc")) - .withFiveQi(1931804941) - .withAllocationAndRetentionPriorityLevel(1808660907) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV6, - PduSessionType.IPV4, - PduSessionType.IPV6, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("xqvkjlmxhomdyn"), - new ServiceResourceId().withId("dwdigumb"), - new ServiceResourceId().withId("raauzzpt"), - new ServiceResourceId().withId("a"))) - .withMaximumNumberOfBufferedPackets(2034990044))), - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("dz")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("ezwwv")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("qyuvvfonkp")) - .withSessionAmbr(new Ambr().withUplink("hqyikvy").withDownlink("auy")) - .withFiveQi(2048419194) - .withAllocationAndRetentionPriorityLevel(712290061) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV4, - PduSessionType.IPV4, - PduSessionType.IPV6, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("o"), - new ServiceResourceId().withId("krsgsgb"), - new ServiceResourceId().withId("huzqgn"))) - .withMaximumNumberOfBufferedPackets(308026039), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("kynscliqhzv")) - .withSessionAmbr( - new Ambr().withUplink("xnkomtkubo").withDownlink("ppnvdxz")) - .withFiveQi(685064266) - .withAllocationAndRetentionPriorityLevel(1207165403) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV4, - PduSessionType.IPV6, - PduSessionType.IPV4, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("hlfkqojpy"), - new ServiceResourceId().withId("vgtrdcnifmzzs"), - new ServiceResourceId().withId("ymbrnysuxmpraf"), - new ServiceResourceId().withId("g"))) - .withMaximumNumberOfBufferedPackets(1986449760))))); + Arrays.asList( + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("xexccbdreaxhcexd")) + .withSessionAmbr( + new Ambr().withUplink("rvqahqkghtpwi").withDownlink("nhyjsv")) + .withFiveQi(758254768).withAllocationAndRetentionPriorityLevel(146536566) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4) + .withAdditionalAllowedSessionTypes( + Arrays.asList(PduSessionType.IPV6, PduSessionType.IPV4)) + .withAllowedServices( + Arrays.asList(new ServiceResourceId().withId("gjqppy"))) + .withMaximumNumberOfBufferedPackets(768184156), + new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("ronzmyhgfip")) + .withSessionAmbr( + new Ambr().withUplink("sxkm").withDownlink("waekrrjreafxtsgu")) + .withFiveQi(1263462710).withAllocationAndRetentionPriorityLevel(610202176) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4) + .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV6)) + .withAllowedServices(Arrays + .asList(new ServiceResourceId().withId("qpvuzlmvfelf"))) + .withMaximumNumberOfBufferedPackets(2091794134))), + new SliceConfiguration().withSlice(new SliceResourceId().withId("plcrpwjxeznoig")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("rnjwmw")) + .withDataNetworkConfigurations(Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("nbsazejjoqkag")) + .withSessionAmbr( + new Ambr().withUplink("hsxttaugzxnf").withDownlink("azpxdtnkdmkqjjl")) + .withFiveQi(1899473333).withAllocationAndRetentionPriorityLevel(1206033090) + .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV6) + .withAdditionalAllowedSessionTypes( + Arrays.asList(PduSessionType.IPV4, PduSessionType.IPV6)) + .withAllowedServices(Arrays.asList(new ServiceResourceId().withId("bqaays"), + new ServiceResourceId().withId("kixqtnqtt"), + new ServiceResourceId().withId("zlwfffiakp"), + new ServiceResourceId().withId("pqqmted"))) + .withMaximumNumberOfBufferedPackets(1028485886))))); model = BinaryData.fromObject(model).toObject(SimPolicyPropertiesFormat.class); - Assertions.assertEquals("ruocbgo", model.ueAmbr().uplink()); - Assertions.assertEquals("rb", model.ueAmbr().downlink()); - Assertions.assertEquals("eoybfhjxakvvjgs", model.defaultSlice().id()); - Assertions.assertEquals(36886384, model.rfspIndex()); - Assertions.assertEquals(417144058, model.registrationTimer()); - Assertions.assertEquals("mywwtkgkxnyed", model.sliceConfigurations().get(0).slice().id()); - Assertions.assertEquals("b", model.sliceConfigurations().get(0).defaultDataNetwork().id()); - Assertions - .assertEquals( - "vudtjuewbcihx", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions - .assertEquals( - "uwhcjyxccybv", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions - .assertEquals( - "ayakkudzpx", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); - Assertions - .assertEquals(431593989, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); - Assertions - .assertEquals( - 710163355, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); - Assertions - .assertEquals( - PduSessionType.IPV6, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); - Assertions - .assertEquals( - PduSessionType.IPV6, - model - .sliceConfigurations() - .get(0) - .dataNetworkConfigurations() - .get(0) - .additionalAllowedSessionTypes() - .get(0)); - Assertions - .assertEquals( - "rkdbdgiogsjkmnwq", - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); - Assertions - .assertEquals( - 201856710, - model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); + Assertions.assertEquals("pbyrqufegxu", model.ueAmbr().uplink()); + Assertions.assertEquals("wz", model.ueAmbr().downlink()); + Assertions.assertEquals("bnhlmc", model.defaultSlice().id()); + Assertions.assertEquals(25336970, model.rfspIndex()); + Assertions.assertEquals(1814176762, model.registrationTimer()); + Assertions.assertEquals("itvgbmhrixkwm", model.sliceConfigurations().get(0).slice().id()); + Assertions.assertEquals("ijejvegrhbpn", model.sliceConfigurations().get(0).defaultDataNetwork().id()); + Assertions.assertEquals("xexccbdreaxhcexd", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("rvqahqkghtpwi", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("nhyjsv", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals(758254768, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).fiveQi()); + Assertions.assertEquals(146536566, model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0) + .allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).preemptionVulnerability()); + Assertions.assertEquals(PduSessionType.IPV4, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).defaultSessionType()); + Assertions.assertEquals(PduSessionType.IPV6, model.sliceConfigurations().get(0).dataNetworkConfigurations() + .get(0).additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("gjqppy", + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).allowedServices().get(0).id()); + Assertions.assertEquals(768184156, + model.sliceConfigurations().get(0).dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyResourceIdTests.java index df10c660dfe9e..acfd0348a1a2b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimPolicyResourceIdTests.java @@ -11,14 +11,14 @@ public final class SimPolicyResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimPolicyResourceId model = BinaryData.fromString("{\"id\":\"hej\"}").toObject(SimPolicyResourceId.class); - Assertions.assertEquals("hej", model.id()); + SimPolicyResourceId model = BinaryData.fromString("{\"id\":\"vvwxqi\"}").toObject(SimPolicyResourceId.class); + Assertions.assertEquals("vvwxqi", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimPolicyResourceId model = new SimPolicyResourceId().withId("hej"); + SimPolicyResourceId model = new SimPolicyResourceId().withId("vvwxqi"); model = BinaryData.fromObject(model).toObject(SimPolicyResourceId.class); - Assertions.assertEquals("hej", model.id()); + Assertions.assertEquals("vvwxqi", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesStaticIpTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesStaticIpTests.java index 96ba4fb67fca7..f0ca8ff3d9812 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesStaticIpTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesStaticIpTests.java @@ -11,15 +11,15 @@ public final class SimStaticIpPropertiesStaticIpTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimStaticIpPropertiesStaticIp model = - BinaryData.fromString("{\"ipv4Address\":\"umrtwnawjsl\"}").toObject(SimStaticIpPropertiesStaticIp.class); - Assertions.assertEquals("umrtwnawjsl", model.ipv4Address()); + SimStaticIpPropertiesStaticIp model + = BinaryData.fromString("{\"ipv4Address\":\"wkqnyhg\"}").toObject(SimStaticIpPropertiesStaticIp.class); + Assertions.assertEquals("wkqnyhg", model.ipv4Address()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimStaticIpPropertiesStaticIp model = new SimStaticIpPropertiesStaticIp().withIpv4Address("umrtwnawjsl"); + SimStaticIpPropertiesStaticIp model = new SimStaticIpPropertiesStaticIp().withIpv4Address("wkqnyhg"); model = BinaryData.fromObject(model).toObject(SimStaticIpPropertiesStaticIp.class); - Assertions.assertEquals("umrtwnawjsl", model.ipv4Address()); + Assertions.assertEquals("wkqnyhg", model.ipv4Address()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesTests.java index 7af1169fc088d..b3f304c7775aa 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimStaticIpPropertiesTests.java @@ -14,26 +14,23 @@ public final class SimStaticIpPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SimStaticIpProperties model = - BinaryData - .fromString( - "{\"attachedDataNetwork\":{\"id\":\"w\"},\"slice\":{\"id\":\"nfqn\"},\"staticIp\":{\"ipv4Address\":\"psxjvf\"}}") - .toObject(SimStaticIpProperties.class); - Assertions.assertEquals("w", model.attachedDataNetwork().id()); - Assertions.assertEquals("nfqn", model.slice().id()); - Assertions.assertEquals("psxjvf", model.staticIp().ipv4Address()); + SimStaticIpProperties model = BinaryData.fromString( + "{\"attachedDataNetwork\":{\"id\":\"qunyowxwlmdjr\"},\"slice\":{\"id\":\"fgbvfvpdbo\"},\"staticIp\":{\"ipv4Address\":\"izsjqlhkrr\"}}") + .toObject(SimStaticIpProperties.class); + Assertions.assertEquals("qunyowxwlmdjr", model.attachedDataNetwork().id()); + Assertions.assertEquals("fgbvfvpdbo", model.slice().id()); + Assertions.assertEquals("izsjqlhkrr", model.staticIp().ipv4Address()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SimStaticIpProperties model = - new SimStaticIpProperties() - .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId("w")) - .withSlice(new SliceResourceId().withId("nfqn")) - .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("psxjvf")); + SimStaticIpProperties model = new SimStaticIpProperties() + .withAttachedDataNetwork(new AttachedDataNetworkResourceId().withId("qunyowxwlmdjr")) + .withSlice(new SliceResourceId().withId("fgbvfvpdbo")) + .withStaticIp(new SimStaticIpPropertiesStaticIp().withIpv4Address("izsjqlhkrr")); model = BinaryData.fromObject(model).toObject(SimStaticIpProperties.class); - Assertions.assertEquals("w", model.attachedDataNetwork().id()); - Assertions.assertEquals("nfqn", model.slice().id()); - Assertions.assertEquals("psxjvf", model.staticIp().ipv4Address()); + Assertions.assertEquals("qunyowxwlmdjr", model.attachedDataNetwork().id()); + Assertions.assertEquals("fgbvfvpdbo", model.slice().id()); + Assertions.assertEquals("izsjqlhkrr", model.staticIp().ipv4Address()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteMockTests.java index 4c5acd9bab7e8..d63f4998928ff 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsBulkDeleteMockTests.java @@ -32,50 +32,34 @@ public void testBulkDelete() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"hwtbbaedorvvm\",\"name\":\"loyg\",\"status\":\"dgwumgxdgdhp\",\"resourceId\":\"gdexjd\",\"startTime\":\"2021-02-07T21:21:29Z\",\"endTime\":\"2020-12-20T10:31:54Z\",\"percentComplete\":95.73648016931433,\"properties\":\"datammwllc\"}"; + String responseStr + = "{\"id\":\"i\",\"name\":\"rpiwrqofulo\",\"status\":\"mjnlexwhcb\",\"resourceId\":\"ibkeph\",\"startTime\":\"2021-05-19T15:44:44Z\",\"endTime\":\"2021-01-05T02:55:46Z\",\"percentComplete\":3.1507250902145123,\"properties\":\"dataoyin\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - AsyncOperationStatus response = - manager - .sims() - .bulkDelete( - "atvfuzkaftj", - "vru", - new SimDeleteList().withSims(Arrays.asList("igsyeipqdsmjt", "rqgdgkkil", "plkcsm")), - com.azure.core.util.Context.NONE); + AsyncOperationStatus response = manager.sims().bulkDelete("ucygvo", "vyuns", + new SimDeleteList().withSims(Arrays.asList("lghieegjlgvvpase", "sgb", "xantuygdhg")), + com.azure.core.util.Context.NONE); - Assertions.assertEquals("hwtbbaedorvvm", response.id()); - Assertions.assertEquals("loyg", response.name()); - Assertions.assertEquals("dgwumgxdgdhp", response.status()); - Assertions.assertEquals("gdexjd", response.resourceId()); - Assertions.assertEquals(OffsetDateTime.parse("2021-02-07T21:21:29Z"), response.startTime()); - Assertions.assertEquals(OffsetDateTime.parse("2020-12-20T10:31:54Z"), response.endTime()); - Assertions.assertEquals(95.73648016931433D, response.percentComplete()); + Assertions.assertEquals("i", response.id()); + Assertions.assertEquals("rpiwrqofulo", response.name()); + Assertions.assertEquals("mjnlexwhcb", response.status()); + Assertions.assertEquals("ibkeph", response.resourceId()); + Assertions.assertEquals(OffsetDateTime.parse("2021-05-19T15:44:44Z"), response.startTime()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-05T02:55:46Z"), response.endTime()); + Assertions.assertEquals(3.1507250902145123D, response.percentComplete()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteMockTests.java index 654109a365493..66fbff851b459 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SimsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.sims().delete("mt", "oiwenazerohzrsq", "lsxkd", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.sims().delete("mzwcjjncqt", "z", "izvg", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteDeletePacketCoreTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteDeletePacketCoreTests.java index 25eb823eca802..d3987bfa84cfa 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteDeletePacketCoreTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteDeletePacketCoreTests.java @@ -12,16 +12,16 @@ public final class SiteDeletePacketCoreTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SiteDeletePacketCore model = - BinaryData.fromString("{\"packetCore\":{\"id\":\"oqaqhvseufuq\"}}").toObject(SiteDeletePacketCore.class); - Assertions.assertEquals("oqaqhvseufuq", model.packetCore().id()); + SiteDeletePacketCore model = BinaryData.fromString("{\"packetCore\":{\"id\":\"zdwlvwlyoupfgfb\"}}") + .toObject(SiteDeletePacketCore.class); + Assertions.assertEquals("zdwlvwlyoupfgfb", model.packetCore().id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SiteDeletePacketCore model = - new SiteDeletePacketCore().withPacketCore(new PacketCoreControlPlaneResourceId().withId("oqaqhvseufuq")); + SiteDeletePacketCore model = new SiteDeletePacketCore() + .withPacketCore(new PacketCoreControlPlaneResourceId().withId("zdwlvwlyoupfgfb")); model = BinaryData.fromObject(model).toObject(SiteDeletePacketCore.class); - Assertions.assertEquals("oqaqhvseufuq", model.packetCore().id()); + Assertions.assertEquals("zdwlvwlyoupfgfb", model.packetCore().id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteInnerTests.java index dc3c4fadce1e4..20eae3d3c06f2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteInnerTests.java @@ -13,26 +13,19 @@ public final class SiteInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SiteInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Unknown\",\"networkFunctions\":[{\"id\":\"b\"},{\"id\":\"hhxud\"},{\"id\":\"vodhtn\"}]},\"location\":\"rudhzmmesckdl\",\"tags\":{\"dfgsftufqobrj\":\"zrcxfailcfxwmdbo\",\"cc\":\"nac\",\"kizvytn\":\"knh\",\"aaeranokqgukk\":\"zvulj\"},\"id\":\"qnvb\",\"name\":\"oylaxxul\",\"type\":\"disdosfjbjsvgj\"}") - .toObject(SiteInner.class); - Assertions.assertEquals("rudhzmmesckdl", model.location()); - Assertions.assertEquals("zrcxfailcfxwmdbo", model.tags().get("dfgsftufqobrj")); + SiteInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Deleted\",\"networkFunctions\":[{\"id\":\"kkqfqjbvle\"}]},\"location\":\"fmluiqtqzfavyvn\",\"tags\":{\"euayjkqabqgzsles\":\"bar\"},\"id\":\"cbhernntiewdj\",\"name\":\"vbquwr\",\"type\":\"ehwagoh\"}") + .toObject(SiteInner.class); + Assertions.assertEquals("fmluiqtqzfavyvn", model.location()); + Assertions.assertEquals("bar", model.tags().get("euayjkqabqgzsles")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SiteInner model = - new SiteInner() - .withLocation("rudhzmmesckdl") - .withTags( - mapOf( - "dfgsftufqobrj", "zrcxfailcfxwmdbo", "cc", "nac", "kizvytn", "knh", "aaeranokqgukk", "zvulj")); + SiteInner model = new SiteInner().withLocation("fmluiqtqzfavyvn").withTags(mapOf("euayjkqabqgzsles", "bar")); model = BinaryData.fromObject(model).toObject(SiteInner.class); - Assertions.assertEquals("rudhzmmesckdl", model.location()); - Assertions.assertEquals("zrcxfailcfxwmdbo", model.tags().get("dfgsftufqobrj")); + Assertions.assertEquals("fmluiqtqzfavyvn", model.location()); + Assertions.assertEquals("bar", model.tags().get("euayjkqabqgzsles")); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteListResultTests.java index f1f7e4544685e..b7966dbea26f0 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteListResultTests.java @@ -15,50 +15,25 @@ public final class SiteListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SiteListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Unknown\",\"networkFunctions\":[{\"id\":\"dhttzaefedxihchr\"},{\"id\":\"k\"},{\"id\":\"rjdqnsdfzp\"},{\"id\":\"tg\"}]},\"location\":\"lkdghr\",\"tags\":{\"bwnhhtql\":\"utlwxezwzhok\",\"pifhpfeoajvgcxtx\":\"ehgpp\",\"smkss\":\"csheafidltugsr\",\"iftxfkf\":\"h\"},\"id\":\"egprhptil\",\"name\":\"ucb\",\"type\":\"qtgdqohmcwsl\"},{\"properties\":{\"provisioningState\":\"Succeeded\",\"networkFunctions\":[{\"id\":\"wbralllibphbqzm\"}]},\"location\":\"aka\",\"tags\":{\"lhjlmuoyxprimr\":\"kjpdnjzhajo\",\"eislstvasylwx\":\"opteecj\",\"eoohguufuzboyj\":\"zaum\"},\"id\":\"thwtzol\",\"name\":\"a\",\"type\":\"mwmdxmebwjscjpa\"},{\"properties\":{\"provisioningState\":\"Canceled\",\"networkFunctions\":[{\"id\":\"f\"}]},\"location\":\"nmwmqtibx\",\"tags\":{\"dija\":\"ddtvqctt\",\"sieekpndzaapm\":\"ukm\"},\"id\":\"dqmeqwigpibudq\",\"name\":\"yxeb\",\"type\":\"ybpmzznrtffyaq\"}],\"nextLink\":\"mhh\"}") - .toObject(SiteListResult.class); - Assertions.assertEquals("lkdghr", model.value().get(0).location()); - Assertions.assertEquals("utlwxezwzhok", model.value().get(0).tags().get("bwnhhtql")); + SiteListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"networkFunctions\":[{\"id\":\"ebjvewzcjzn\"},{\"id\":\"cpmguaadraufact\"},{\"id\":\"hzovaj\"}]},\"location\":\"iuxxpshneekulfg\",\"tags\":{\"rds\":\"ubkwdle\",\"hminyflnorwmduv\":\"tujbazpju\"},\"id\":\"pklvxw\",\"name\":\"ygdxpgpqchis\",\"type\":\"epn\"},{\"properties\":{\"provisioningState\":\"Deleted\",\"networkFunctions\":[{\"id\":\"ibbdaxconfoza\"},{\"id\":\"rsukokwbqplh\"}]},\"location\":\"nuuepzlrp\",\"tags\":{\"nnrwrbiork\":\"soldweyuqdunv\",\"xmsivfomiloxggdu\":\"alywjhhgdn\"},\"id\":\"iqndieuzaofj\",\"name\":\"hvcyyysfg\",\"type\":\"otcubi\"},{\"properties\":{\"provisioningState\":\"Deleted\",\"networkFunctions\":[{\"id\":\"qonmacj\"}]},\"location\":\"nizshqvcim\",\"tags\":{\"ccwr\":\"fgmblrrilbywdxsm\",\"dvoqyt\":\"fscjfnynszquji\",\"hjoxo\":\"byowbblgyavutp\"},\"id\":\"smsks\",\"name\":\"pi\",\"type\":\"lqol\"}],\"nextLink\":\"kcgxxlxsffgcvi\"}") + .toObject(SiteListResult.class); + Assertions.assertEquals("iuxxpshneekulfg", model.value().get(0).location()); + Assertions.assertEquals("ubkwdle", model.value().get(0).tags().get("rds")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SiteListResult model = - new SiteListResult() - .withValue( - Arrays - .asList( - new SiteInner() - .withLocation("lkdghr") - .withTags( - mapOf( - "bwnhhtql", - "utlwxezwzhok", - "pifhpfeoajvgcxtx", - "ehgpp", - "smkss", - "csheafidltugsr", - "iftxfkf", - "h")), - new SiteInner() - .withLocation("aka") - .withTags( - mapOf( - "lhjlmuoyxprimr", - "kjpdnjzhajo", - "eislstvasylwx", - "opteecj", - "eoohguufuzboyj", - "zaum")), - new SiteInner() - .withLocation("nmwmqtibx") - .withTags(mapOf("dija", "ddtvqctt", "sieekpndzaapm", "ukm")))); + SiteListResult model = new SiteListResult().withValue(Arrays.asList( + new SiteInner().withLocation("iuxxpshneekulfg") + .withTags(mapOf("rds", "ubkwdle", "hminyflnorwmduv", "tujbazpju")), + new SiteInner().withLocation("nuuepzlrp") + .withTags(mapOf("nnrwrbiork", "soldweyuqdunv", "xmsivfomiloxggdu", "alywjhhgdn")), + new SiteInner().withLocation("nizshqvcim") + .withTags(mapOf("ccwr", "fgmblrrilbywdxsm", "dvoqyt", "fscjfnynszquji", "hjoxo", "byowbblgyavutp")))); model = BinaryData.fromObject(model).toObject(SiteListResult.class); - Assertions.assertEquals("lkdghr", model.value().get(0).location()); - Assertions.assertEquals("utlwxezwzhok", model.value().get(0).tags().get("bwnhhtql")); + Assertions.assertEquals("iuxxpshneekulfg", model.value().get(0).location()); + Assertions.assertEquals("ubkwdle", model.value().get(0).tags().get("rds")); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitePropertiesFormatTests.java index a405d309c0e4b..87e3a31d97eca 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitePropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitePropertiesFormatTests.java @@ -10,11 +10,9 @@ public final class SitePropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SitePropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Unknown\",\"networkFunctions\":[{\"id\":\"ycy\"},{\"id\":\"c\"},{\"id\":\"gc\"},{\"id\":\"nfnw\"}]}") - .toObject(SitePropertiesFormat.class); + SitePropertiesFormat model = BinaryData + .fromString("{\"provisioningState\":\"Deleted\",\"networkFunctions\":[{\"id\":\"rqemvvhmxt\"}]}") + .toObject(SitePropertiesFormat.class); } @org.junit.jupiter.api.Test diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteResourceIdTests.java index f628f2a2b2d24..c0c0c78507460 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SiteResourceIdTests.java @@ -11,14 +11,14 @@ public final class SiteResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SiteResourceId model = BinaryData.fromString("{\"id\":\"gu\"}").toObject(SiteResourceId.class); - Assertions.assertEquals("gu", model.id()); + SiteResourceId model = BinaryData.fromString("{\"id\":\"hrxsbk\"}").toObject(SiteResourceId.class); + Assertions.assertEquals("hrxsbk", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SiteResourceId model = new SiteResourceId().withId("gu"); + SiteResourceId model = new SiteResourceId().withId("hrxsbk"); model = BinaryData.fromObject(model).toObject(SiteResourceId.class); - Assertions.assertEquals("gu", model.id()); + Assertions.assertEquals("hrxsbk", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateMockTests.java index d42e60c1ca66a..df7fcd8e287eb 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesCreateOrUpdateMockTests.java @@ -32,46 +32,29 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"networkFunctions\":[{\"id\":\"eyguq\"}]},\"location\":\"ijiitns\",\"tags\":{\"jwaiuf\":\"zdesygr\"},\"id\":\"n\",\"name\":\"aybfu\",\"type\":\"qfrojsydgrhyd\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"networkFunctions\":[{\"id\":\"noakiz\"}]},\"location\":\"aikn\",\"tags\":{\"xzcwxhmpejt\":\"nuwiguyxlykwph\",\"aonwivkcqhrxh\":\"ke\"},\"id\":\"knlccrmmkyup\",\"name\":\"jubyqjfkakfq\",\"type\":\"rke\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Site response = - manager - .sites() - .define("tujwjju") - .withRegion("ytiq") - .withExistingMobileNetwork("enwphpzfngq", "clid") - .withTags(mapOf("w", "qer", "gukvlbpkt", "iytxt")) - .create(); + Site response = manager.sites().define("mairrh").withRegion("s") + .withExistingMobileNetwork("cbevxrhyzdfw", "sofpltd").withTags(mapOf("sjybvitv", "bjb")).create(); - Assertions.assertEquals("ijiitns", response.location()); - Assertions.assertEquals("zdesygr", response.tags().get("jwaiuf")); + Assertions.assertEquals("aikn", response.location()); + Assertions.assertEquals("nuwiguyxlykwph", response.tags().get("xzcwxhmpejt")); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteMockTests.java index 9185300c35a8a..280fda7c4c66a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.sites().delete("tcjb", "tvivuzqym", "uo", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.sites().delete("yzatvfuzkaft", "vvruxwi", "syeipqd", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreMockTests.java index 6e23972520e06..c3a5ef91dd865 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesDeletePacketCoreMockTests.java @@ -34,38 +34,21 @@ public void testDeletePacketCore() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.sites().deletePacketCore("jpziu", "fbzkk", "tnhqsycl", new SiteDeletePacketCore().withPacketCore( + new PacketCoreControlPlaneResourceId().withId("elpkpbafvafhlbyl")), com.azure.core.util.Context.NONE); - manager - .sites() - .deletePacketCore( - "tp", - "wxqcsehchkhufmpq", - "mqyjgy", - new SiteDeletePacketCore() - .withPacketCore(new PacketCoreControlPlaneResourceId().withId("ulodsaeuzanhsfnh")), - com.azure.core.util.Context.NONE); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetWithResponseMockTests.java index 365baef3cbfa5..f22fecf997e2b 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesGetWithResponseMockTests.java @@ -30,39 +30,29 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Canceled\",\"networkFunctions\":[{\"id\":\"xeatkd\"},{\"id\":\"wnrdjyibqbnaom\"},{\"id\":\"rmkuh\"}]},\"location\":\"xljalfihc\",\"tags\":{\"nc\":\"bc\",\"xf\":\"exxqcwg\",\"r\":\"vaknokzwjj\",\"x\":\"tixldzyyfytpqs\"},\"id\":\"mmpuj\",\"name\":\"vyqlkjuvsmbmslzo\",\"type\":\"ovwzdbpqvybefg\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Canceled\",\"networkFunctions\":[{\"id\":\"wumgx\"},{\"id\":\"dhp\"},{\"id\":\"gdexjd\"}]},\"location\":\"jsaqwotmmwllcols\",\"tags\":{\"hexcgjokj\":\"apte\",\"bksdqhjvyklxesl\":\"jnhvlqjbekpeeks\",\"cpoq\":\"hhus\",\"g\":\"avnwqj\"},\"id\":\"knlejjjkxybwfd\",\"name\":\"kjbztensvkzykj\",\"type\":\"jknsxfwu\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Site response = - manager.sites().getWithResponse("o", "tgitsqhzvbrzc", "banf", com.azure.core.util.Context.NONE).getValue(); + Site response = manager.sites() + .getWithResponse("mjtgrqg", "gkkileplkcsmkn", "wtbbaedorvvmqf", com.azure.core.util.Context.NONE) + .getValue(); - Assertions.assertEquals("xljalfihc", response.location()); - Assertions.assertEquals("bc", response.tags().get("nc")); + Assertions.assertEquals("jsaqwotmmwllcols", response.location()); + Assertions.assertEquals("apte", response.tags().get("hexcgjokj")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkMockTests.java index 2e9055558af88..a94f31360cb36 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SitesListByMobileNetworkMockTests.java @@ -31,39 +31,28 @@ public void testListByMobileNetwork() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"networkFunctions\":[{\"id\":\"rhunlp\"}]},\"location\":\"ykycndzfqi\",\"tags\":{\"bbmnwagltb\":\"euy\",\"ymvqdbpbhfckdvez\":\"oeeonqlnfwm\",\"b\":\"rcssbzhddubbnq\",\"alehpav\":\"h\"},\"id\":\"wugiqjti\",\"name\":\"gqgdminictteajoh\",\"type\":\"ygspnbonhpczykm\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"networkFunctions\":[{\"id\":\"kjzwfbcyayk\"}]},\"location\":\"fz\",\"tags\":{\"bjazejwwviy\":\"wxrzxmdewsrsxkrp\",\"suhbrnn\":\"y\",\"zycyqiqyhg\":\"jxsqwjhqkbiwetp\"},\"id\":\"se\",\"name\":\"zlex\",\"type\":\"sfledyn\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.sites().listByMobileNetwork("mx", "okcvtlubses", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.sites().listByMobileNetwork("hcdpkupnqrmgj", "bpkuwxeoio", com.azure.core.util.Context.NONE); - Assertions.assertEquals("ykycndzfqi", response.iterator().next().location()); - Assertions.assertEquals("euy", response.iterator().next().tags().get("bbmnwagltb")); + Assertions.assertEquals("fz", response.iterator().next().location()); + Assertions.assertEquals("wxrzxmdewsrsxkrp", response.iterator().next().tags().get("bjazejwwviy")); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceConfigurationTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceConfigurationTests.java index 9bc362b7df1a2..71009493f2c45 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceConfigurationTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceConfigurationTests.java @@ -20,120 +20,61 @@ public final class SliceConfigurationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SliceConfiguration model = - BinaryData - .fromString( - "{\"slice\":{\"id\":\"ocxvdfffwafqr\"},\"defaultDataNetwork\":{\"id\":\"udaspavehh\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"kbunzoz\"},\"sessionAmbr\":{\"uplink\":\"dhcxgkmoy\",\"downlink\":\"cdyuibhmfdnbzyd\"},\"5qi\":1872796730,\"allocationAndRetentionPriorityLevel\":1709322147,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\",\"IPv6\"],\"allowedServices\":[{\"id\":\"gorf\"},{\"id\":\"ukiscvwmzhw\"},{\"id\":\"lefaxvxilcbtgn\"}],\"maximumNumberOfBufferedPackets\":2012963266},{\"dataNetwork\":{\"id\":\"eyqxtjjfzql\"},\"sessionAmbr\":{\"uplink\":\"hycav\",\"downlink\":\"dggxdbeesmi\"},\"5qi\":1356434972,\"allocationAndRetentionPriorityLevel\":420094918,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"Preemptable\",\"defaultSessionType\":\"IPv6\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv4\",\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"qfby\"},{\"id\":\"yr\"},{\"id\":\"giagtcojo\"},{\"id\":\"qwogfnzjvus\"}],\"maximumNumberOfBufferedPackets\":359002746},{\"dataNetwork\":{\"id\":\"dmozu\"},\"sessionAmbr\":{\"uplink\":\"ylfsbtk\",\"downlink\":\"dp\"},\"5qi\":737232948,\"allocationAndRetentionPriorityLevel\":1774852320,\"preemptionCapability\":\"MayPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv6\"],\"allowedServices\":[{\"id\":\"cto\"},{\"id\":\"cmisofie\"},{\"id\":\"pe\"}],\"maximumNumberOfBufferedPackets\":419692653}]}") - .toObject(SliceConfiguration.class); - Assertions.assertEquals("ocxvdfffwafqr", model.slice().id()); - Assertions.assertEquals("udaspavehh", model.defaultDataNetwork().id()); - Assertions.assertEquals("kbunzoz", model.dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions.assertEquals("dhcxgkmoy", model.dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions.assertEquals("cdyuibhmfdnbzyd", model.dataNetworkConfigurations().get(0).sessionAmbr().downlink()); - Assertions.assertEquals(1872796730, model.dataNetworkConfigurations().get(0).fiveQi()); - Assertions - .assertEquals(1709322147, model.dataNetworkConfigurations().get(0).allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, model.dataNetworkConfigurations().get(0).preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - model.dataNetworkConfigurations().get(0).preemptionVulnerability()); + SliceConfiguration model = BinaryData.fromString( + "{\"slice\":{\"id\":\"mjihyeozphv\"},\"defaultDataNetwork\":{\"id\":\"auyqncygupkv\"},\"dataNetworkConfigurations\":[{\"dataNetwork\":{\"id\":\"mdscwxqupev\"},\"sessionAmbr\":{\"uplink\":\"hfstotxhojujbyp\",\"downlink\":\"lmcuvhixb\"},\"5qi\":1696706590,\"allocationAndRetentionPriorityLevel\":12838706,\"preemptionCapability\":\"NotPreempt\",\"preemptionVulnerability\":\"NotPreemptable\",\"defaultSessionType\":\"IPv4\",\"additionalAllowedSessionTypes\":[\"IPv4\",\"IPv6\",\"IPv6\",\"IPv4\"],\"allowedServices\":[{\"id\":\"iwkkbn\"},{\"id\":\"jrywvtylbfpnc\"},{\"id\":\"rd\"}],\"maximumNumberOfBufferedPackets\":287864608}]}") + .toObject(SliceConfiguration.class); + Assertions.assertEquals("mjihyeozphv", model.slice().id()); + Assertions.assertEquals("auyqncygupkv", model.defaultDataNetwork().id()); + Assertions.assertEquals("mdscwxqupev", model.dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("hfstotxhojujbyp", model.dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("lmcuvhixb", model.dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals(1696706590, model.dataNetworkConfigurations().get(0).fiveQi()); + Assertions.assertEquals(12838706, + model.dataNetworkConfigurations().get(0).allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.dataNetworkConfigurations().get(0).preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.dataNetworkConfigurations().get(0).preemptionVulnerability()); Assertions.assertEquals(PduSessionType.IPV4, model.dataNetworkConfigurations().get(0).defaultSessionType()); - Assertions - .assertEquals( - PduSessionType.IPV6, model.dataNetworkConfigurations().get(0).additionalAllowedSessionTypes().get(0)); - Assertions.assertEquals("gorf", model.dataNetworkConfigurations().get(0).allowedServices().get(0).id()); - Assertions.assertEquals(2012963266, model.dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); + Assertions.assertEquals(PduSessionType.IPV4, + model.dataNetworkConfigurations().get(0).additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("iwkkbn", model.dataNetworkConfigurations().get(0).allowedServices().get(0).id()); + Assertions.assertEquals(287864608, model.dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SliceConfiguration model = - new SliceConfiguration() - .withSlice(new SliceResourceId().withId("ocxvdfffwafqr")) - .withDefaultDataNetwork(new DataNetworkResourceId().withId("udaspavehh")) - .withDataNetworkConfigurations( - Arrays - .asList( - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("kbunzoz")) - .withSessionAmbr(new Ambr().withUplink("dhcxgkmoy").withDownlink("cdyuibhmfdnbzyd")) - .withFiveQi(1872796730) - .withAllocationAndRetentionPriorityLevel(1709322147) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes( - Arrays.asList(PduSessionType.IPV6, PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("gorf"), - new ServiceResourceId().withId("ukiscvwmzhw"), - new ServiceResourceId().withId("lefaxvxilcbtgn"))) - .withMaximumNumberOfBufferedPackets(2012963266), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("eyqxtjjfzql")) - .withSessionAmbr(new Ambr().withUplink("hycav").withDownlink("dggxdbeesmi")) - .withFiveQi(1356434972) - .withAllocationAndRetentionPriorityLevel(420094918) - .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV6) - .withAdditionalAllowedSessionTypes( - Arrays - .asList( - PduSessionType.IPV4, - PduSessionType.IPV4, - PduSessionType.IPV6, - PduSessionType.IPV4)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("qfby"), - new ServiceResourceId().withId("yr"), - new ServiceResourceId().withId("giagtcojo"), - new ServiceResourceId().withId("qwogfnzjvus"))) - .withMaximumNumberOfBufferedPackets(359002746), - new DataNetworkConfiguration() - .withDataNetwork(new DataNetworkResourceId().withId("dmozu")) - .withSessionAmbr(new Ambr().withUplink("ylfsbtk").withDownlink("dp")) - .withFiveQi(737232948) - .withAllocationAndRetentionPriorityLevel(1774852320) - .withPreemptionCapability(PreemptionCapability.MAY_PREEMPT) - .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) - .withDefaultSessionType(PduSessionType.IPV4) - .withAdditionalAllowedSessionTypes(Arrays.asList(PduSessionType.IPV6)) - .withAllowedServices( - Arrays - .asList( - new ServiceResourceId().withId("cto"), - new ServiceResourceId().withId("cmisofie"), - new ServiceResourceId().withId("pe"))) - .withMaximumNumberOfBufferedPackets(419692653))); + SliceConfiguration model = new SliceConfiguration().withSlice(new SliceResourceId().withId("mjihyeozphv")) + .withDefaultDataNetwork(new DataNetworkResourceId().withId("auyqncygupkv")) + .withDataNetworkConfigurations(Arrays.asList(new DataNetworkConfiguration() + .withDataNetwork(new DataNetworkResourceId().withId("mdscwxqupev")) + .withSessionAmbr(new Ambr().withUplink("hfstotxhojujbyp").withDownlink("lmcuvhixb")) + .withFiveQi(1696706590).withAllocationAndRetentionPriorityLevel(12838706) + .withPreemptionCapability(PreemptionCapability.NOT_PREEMPT) + .withPreemptionVulnerability(PreemptionVulnerability.NOT_PREEMPTABLE) + .withDefaultSessionType(PduSessionType.IPV4) + .withAdditionalAllowedSessionTypes( + Arrays.asList(PduSessionType.IPV4, PduSessionType.IPV6, PduSessionType.IPV6, PduSessionType.IPV4)) + .withAllowedServices(Arrays.asList(new ServiceResourceId().withId("iwkkbn"), + new ServiceResourceId().withId("jrywvtylbfpnc"), new ServiceResourceId().withId("rd"))) + .withMaximumNumberOfBufferedPackets(287864608))); model = BinaryData.fromObject(model).toObject(SliceConfiguration.class); - Assertions.assertEquals("ocxvdfffwafqr", model.slice().id()); - Assertions.assertEquals("udaspavehh", model.defaultDataNetwork().id()); - Assertions.assertEquals("kbunzoz", model.dataNetworkConfigurations().get(0).dataNetwork().id()); - Assertions.assertEquals("dhcxgkmoy", model.dataNetworkConfigurations().get(0).sessionAmbr().uplink()); - Assertions.assertEquals("cdyuibhmfdnbzyd", model.dataNetworkConfigurations().get(0).sessionAmbr().downlink()); - Assertions.assertEquals(1872796730, model.dataNetworkConfigurations().get(0).fiveQi()); - Assertions - .assertEquals(1709322147, model.dataNetworkConfigurations().get(0).allocationAndRetentionPriorityLevel()); - Assertions - .assertEquals( - PreemptionCapability.NOT_PREEMPT, model.dataNetworkConfigurations().get(0).preemptionCapability()); - Assertions - .assertEquals( - PreemptionVulnerability.PREEMPTABLE, - model.dataNetworkConfigurations().get(0).preemptionVulnerability()); + Assertions.assertEquals("mjihyeozphv", model.slice().id()); + Assertions.assertEquals("auyqncygupkv", model.defaultDataNetwork().id()); + Assertions.assertEquals("mdscwxqupev", model.dataNetworkConfigurations().get(0).dataNetwork().id()); + Assertions.assertEquals("hfstotxhojujbyp", model.dataNetworkConfigurations().get(0).sessionAmbr().uplink()); + Assertions.assertEquals("lmcuvhixb", model.dataNetworkConfigurations().get(0).sessionAmbr().downlink()); + Assertions.assertEquals(1696706590, model.dataNetworkConfigurations().get(0).fiveQi()); + Assertions.assertEquals(12838706, + model.dataNetworkConfigurations().get(0).allocationAndRetentionPriorityLevel()); + Assertions.assertEquals(PreemptionCapability.NOT_PREEMPT, + model.dataNetworkConfigurations().get(0).preemptionCapability()); + Assertions.assertEquals(PreemptionVulnerability.NOT_PREEMPTABLE, + model.dataNetworkConfigurations().get(0).preemptionVulnerability()); Assertions.assertEquals(PduSessionType.IPV4, model.dataNetworkConfigurations().get(0).defaultSessionType()); - Assertions - .assertEquals( - PduSessionType.IPV6, model.dataNetworkConfigurations().get(0).additionalAllowedSessionTypes().get(0)); - Assertions.assertEquals("gorf", model.dataNetworkConfigurations().get(0).allowedServices().get(0).id()); - Assertions.assertEquals(2012963266, model.dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); + Assertions.assertEquals(PduSessionType.IPV4, + model.dataNetworkConfigurations().get(0).additionalAllowedSessionTypes().get(0)); + Assertions.assertEquals("iwkkbn", model.dataNetworkConfigurations().get(0).allowedServices().get(0).id()); + Assertions.assertEquals(287864608, model.dataNetworkConfigurations().get(0).maximumNumberOfBufferedPackets()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceInnerTests.java index a4229b5e08713..b59ec45dff3c1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceInnerTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceInnerTests.java @@ -14,32 +14,27 @@ public final class SliceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SliceInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Canceled\",\"snssai\":{\"sst\":2043816183,\"sd\":\"rddga\"},\"description\":\"uhiosrsju\"},\"location\":\"fcdis\",\"tags\":{\"rxzbujr\":\"nxzhcze\",\"nlnzonzlrpi\":\"rhqvwrevkh\"},\"id\":\"yw\",\"name\":\"cvjtszcofiz\",\"type\":\"htd\"}") - .toObject(SliceInner.class); - Assertions.assertEquals("fcdis", model.location()); - Assertions.assertEquals("nxzhcze", model.tags().get("rxzbujr")); - Assertions.assertEquals(2043816183, model.snssai().sst()); - Assertions.assertEquals("rddga", model.snssai().sd()); - Assertions.assertEquals("uhiosrsju", model.description()); + SliceInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Deleted\",\"snssai\":{\"sst\":1622525022,\"sd\":\"fminsgowzf\"},\"description\":\"sttktlahbqa\"},\"location\":\"xtgzukxitmmqtgqq\",\"tags\":{\"isavok\":\"rnxrxcpj\",\"azivjlfrqttbajl\":\"dzf\"},\"id\":\"atnwxyiopi\",\"name\":\"kqqfk\",\"type\":\"vscx\"}") + .toObject(SliceInner.class); + Assertions.assertEquals("xtgzukxitmmqtgqq", model.location()); + Assertions.assertEquals("rnxrxcpj", model.tags().get("isavok")); + Assertions.assertEquals(1622525022, model.snssai().sst()); + Assertions.assertEquals("fminsgowzf", model.snssai().sd()); + Assertions.assertEquals("sttktlahbqa", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SliceInner model = - new SliceInner() - .withLocation("fcdis") - .withTags(mapOf("rxzbujr", "nxzhcze", "nlnzonzlrpi", "rhqvwrevkh")) - .withSnssai(new Snssai().withSst(2043816183).withSd("rddga")) - .withDescription("uhiosrsju"); + SliceInner model = new SliceInner().withLocation("xtgzukxitmmqtgqq") + .withTags(mapOf("isavok", "rnxrxcpj", "azivjlfrqttbajl", "dzf")) + .withSnssai(new Snssai().withSst(1622525022).withSd("fminsgowzf")).withDescription("sttktlahbqa"); model = BinaryData.fromObject(model).toObject(SliceInner.class); - Assertions.assertEquals("fcdis", model.location()); - Assertions.assertEquals("nxzhcze", model.tags().get("rxzbujr")); - Assertions.assertEquals(2043816183, model.snssai().sst()); - Assertions.assertEquals("rddga", model.snssai().sd()); - Assertions.assertEquals("uhiosrsju", model.description()); + Assertions.assertEquals("xtgzukxitmmqtgqq", model.location()); + Assertions.assertEquals("rnxrxcpj", model.tags().get("isavok")); + Assertions.assertEquals(1622525022, model.snssai().sst()); + Assertions.assertEquals("fminsgowzf", model.snssai().sd()); + Assertions.assertEquals("sttktlahbqa", model.description()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceListResultTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceListResultTests.java index 65e2a1cfae556..95b5ec472fe36 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceListResultTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceListResultTests.java @@ -16,71 +16,37 @@ public final class SliceListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SliceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"snssai\":{\"sst\":1098245078,\"sd\":\"lvidizozs\"},\"description\":\"ccxjm\"},\"location\":\"fdgnwncypuuwwlt\",\"tags\":{\"zzhmkd\":\"jctzenkei\",\"gsrboldforobw\":\"svflyhbxcudch\",\"hfovvacqpbtu\":\"lvizb\"},\"id\":\"dxe\",\"name\":\"zab\",\"type\":\"elawumu\"},{\"properties\":{\"provisioningState\":\"Failed\",\"snssai\":{\"sst\":46527363,\"sd\":\"wrrwoycqucw\"},\"description\":\"ahnomdrkywu\"},\"location\":\"svfuurutlwexxwl\",\"tags\":{\"q\":\"iexzsrzpge\",\"wwpgdakchzyvlixq\":\"yb\",\"bn\":\"rkcxkj\",\"swqrntvlwijp\":\"mysu\"},\"id\":\"ttexoqqpwcyyufmh\",\"name\":\"uncuw\",\"type\":\"qspkcdqzhlctd\"},{\"properties\":{\"provisioningState\":\"Deleting\",\"snssai\":{\"sst\":302400307,\"sd\":\"dyfpchrqbnj\"},\"description\":\"cgegydcwbo\"},\"location\":\"umvq\",\"tags\":{\"jtlo\":\"ihrraiouaub\",\"iflrzpasccbiu\":\"xfuojrn\",\"ufdvruz\":\"mzdlyjdfqwmkyo\",\"hpc\":\"lzo\"},\"id\":\"fnmdxotn\",\"name\":\"fdgugeyzi\",\"type\":\"grkyuizabsnmfpph\"}],\"nextLink\":\"eevy\"}") - .toObject(SliceListResult.class); - Assertions.assertEquals("fdgnwncypuuwwlt", model.value().get(0).location()); - Assertions.assertEquals("jctzenkei", model.value().get(0).tags().get("zzhmkd")); - Assertions.assertEquals(1098245078, model.value().get(0).snssai().sst()); - Assertions.assertEquals("lvidizozs", model.value().get(0).snssai().sd()); - Assertions.assertEquals("ccxjm", model.value().get(0).description()); + SliceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"snssai\":{\"sst\":1094227624,\"sd\":\"t\"},\"description\":\"kxn\"},\"location\":\"dabg\",\"tags\":{\"ewbcihxuuw\":\"dtj\",\"yxccyb\":\"c\",\"px\":\"payakkud\"},\"id\":\"wjplma\",\"name\":\"stcyohpfkyrkdbd\",\"type\":\"iogsjkmnwq\"},{\"properties\":{\"provisioningState\":\"Succeeded\",\"snssai\":{\"sst\":1313812453,\"sd\":\"iyhddvi\"},\"description\":\"egfnmntfpmvmemfn\"},\"location\":\"dwvvba\",\"tags\":{\"chp\":\"l\",\"evwrdnhfuk\":\"db\",\"fcvlerch\":\"vsjcswsmystuluqy\"},\"id\":\"qbmfpjbabwidf\",\"name\":\"xsspuunnoxyhk\",\"type\":\"g\"},{\"properties\":{\"provisioningState\":\"Deleted\",\"snssai\":{\"sst\":2067421779,\"sd\":\"hp\"},\"description\":\"oqcaaewdaomdj\"},\"location\":\"pjxxkzb\",\"tags\":{\"xonbzoggculapz\":\"geivsiykzkdncj\",\"pgogtqxepny\":\"y\"},\"id\":\"b\",\"name\":\"uajlyj\",\"type\":\"lvofqzhvfcibyfmo\"},{\"properties\":{\"provisioningState\":\"Accepted\",\"snssai\":{\"sst\":1105067830,\"sd\":\"jpvd\"},\"description\":\"fzwiivwzjbhyz\"},\"location\":\"jrkambtrnegvmnv\",\"tags\":{\"flvestmjlxrrilo\":\"qvldspastjbkkd\",\"kuziycsle\":\"apeewchpxlkt\",\"dcgzul\":\"ufuztcktyhjtq\"},\"id\":\"mmrqz\",\"name\":\"rr\",\"type\":\"vpglydz\"}],\"nextLink\":\"rvqeevtoepryutn\"}") + .toObject(SliceListResult.class); + Assertions.assertEquals("dabg", model.value().get(0).location()); + Assertions.assertEquals("dtj", model.value().get(0).tags().get("ewbcihxuuw")); + Assertions.assertEquals(1094227624, model.value().get(0).snssai().sst()); + Assertions.assertEquals("t", model.value().get(0).snssai().sd()); + Assertions.assertEquals("kxn", model.value().get(0).description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SliceListResult model = - new SliceListResult() - .withValue( - Arrays - .asList( - new SliceInner() - .withLocation("fdgnwncypuuwwlt") - .withTags( - mapOf( - "zzhmkd", - "jctzenkei", - "gsrboldforobw", - "svflyhbxcudch", - "hfovvacqpbtu", - "lvizb")) - .withSnssai(new Snssai().withSst(1098245078).withSd("lvidizozs")) - .withDescription("ccxjm"), - new SliceInner() - .withLocation("svfuurutlwexxwl") - .withTags( - mapOf( - "q", - "iexzsrzpge", - "wwpgdakchzyvlixq", - "yb", - "bn", - "rkcxkj", - "swqrntvlwijp", - "mysu")) - .withSnssai(new Snssai().withSst(46527363).withSd("wrrwoycqucw")) - .withDescription("ahnomdrkywu"), - new SliceInner() - .withLocation("umvq") - .withTags( - mapOf( - "jtlo", - "ihrraiouaub", - "iflrzpasccbiu", - "xfuojrn", - "ufdvruz", - "mzdlyjdfqwmkyo", - "hpc", - "lzo")) - .withSnssai(new Snssai().withSst(302400307).withSd("dyfpchrqbnj")) - .withDescription("cgegydcwbo"))); + SliceListResult model = new SliceListResult().withValue(Arrays.asList( + new SliceInner().withLocation("dabg").withTags(mapOf("ewbcihxuuw", "dtj", "yxccyb", "c", "px", "payakkud")) + .withSnssai(new Snssai().withSst(1094227624).withSd("t")).withDescription("kxn"), + new SliceInner().withLocation("dwvvba") + .withTags(mapOf("chp", "l", "evwrdnhfuk", "db", "fcvlerch", "vsjcswsmystuluqy")) + .withSnssai(new Snssai().withSst(1313812453).withSd("iyhddvi")).withDescription("egfnmntfpmvmemfn"), + new SliceInner().withLocation("pjxxkzb") + .withTags(mapOf("xonbzoggculapz", "geivsiykzkdncj", "pgogtqxepny", "y")) + .withSnssai(new Snssai().withSst(2067421779).withSd("hp")).withDescription("oqcaaewdaomdj"), + new SliceInner().withLocation("jrkambtrnegvmnv") + .withTags( + mapOf("flvestmjlxrrilo", "qvldspastjbkkd", "kuziycsle", "apeewchpxlkt", "dcgzul", "ufuztcktyhjtq")) + .withSnssai(new Snssai().withSst(1105067830).withSd("jpvd")).withDescription("fzwiivwzjbhyz"))); model = BinaryData.fromObject(model).toObject(SliceListResult.class); - Assertions.assertEquals("fdgnwncypuuwwlt", model.value().get(0).location()); - Assertions.assertEquals("jctzenkei", model.value().get(0).tags().get("zzhmkd")); - Assertions.assertEquals(1098245078, model.value().get(0).snssai().sst()); - Assertions.assertEquals("lvidizozs", model.value().get(0).snssai().sd()); - Assertions.assertEquals("ccxjm", model.value().get(0).description()); + Assertions.assertEquals("dabg", model.value().get(0).location()); + Assertions.assertEquals("dtj", model.value().get(0).tags().get("ewbcihxuuw")); + Assertions.assertEquals(1094227624, model.value().get(0).snssai().sst()); + Assertions.assertEquals("t", model.value().get(0).snssai().sd()); + Assertions.assertEquals("kxn", model.value().get(0).description()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicePropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicePropertiesFormatTests.java index 9f7261c6cfe91..fb8adae77e615 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicePropertiesFormatTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicePropertiesFormatTests.java @@ -12,25 +12,21 @@ public final class SlicePropertiesFormatTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SlicePropertiesFormat model = - BinaryData - .fromString( - "{\"provisioningState\":\"Accepted\",\"snssai\":{\"sst\":673531978,\"sd\":\"vreljea\"},\"description\":\"rvzmlovuana\"}") - .toObject(SlicePropertiesFormat.class); - Assertions.assertEquals(673531978, model.snssai().sst()); - Assertions.assertEquals("vreljea", model.snssai().sd()); - Assertions.assertEquals("rvzmlovuana", model.description()); + SlicePropertiesFormat model = BinaryData.fromString( + "{\"provisioningState\":\"Unknown\",\"snssai\":{\"sst\":902196516,\"sd\":\"govibrxkpmloazu\"},\"description\":\"ocbgoorbteoyb\"}") + .toObject(SlicePropertiesFormat.class); + Assertions.assertEquals(902196516, model.snssai().sst()); + Assertions.assertEquals("govibrxkpmloazu", model.snssai().sd()); + Assertions.assertEquals("ocbgoorbteoyb", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SlicePropertiesFormat model = - new SlicePropertiesFormat() - .withSnssai(new Snssai().withSst(673531978).withSd("vreljea")) - .withDescription("rvzmlovuana"); + SlicePropertiesFormat model = new SlicePropertiesFormat() + .withSnssai(new Snssai().withSst(902196516).withSd("govibrxkpmloazu")).withDescription("ocbgoorbteoyb"); model = BinaryData.fromObject(model).toObject(SlicePropertiesFormat.class); - Assertions.assertEquals(673531978, model.snssai().sst()); - Assertions.assertEquals("vreljea", model.snssai().sd()); - Assertions.assertEquals("rvzmlovuana", model.description()); + Assertions.assertEquals(902196516, model.snssai().sst()); + Assertions.assertEquals("govibrxkpmloazu", model.snssai().sd()); + Assertions.assertEquals("ocbgoorbteoyb", model.description()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceResourceIdTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceResourceIdTests.java index c26f9f98c49cb..e37aedfeac5e2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceResourceIdTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SliceResourceIdTests.java @@ -11,14 +11,14 @@ public final class SliceResourceIdTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SliceResourceId model = BinaryData.fromString("{\"id\":\"vydfceacvlhvygdy\"}").toObject(SliceResourceId.class); - Assertions.assertEquals("vydfceacvlhvygdy", model.id()); + SliceResourceId model = BinaryData.fromString("{\"id\":\"hvxndzwmkrefajpj\"}").toObject(SliceResourceId.class); + Assertions.assertEquals("hvxndzwmkrefajpj", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SliceResourceId model = new SliceResourceId().withId("vydfceacvlhvygdy"); + SliceResourceId model = new SliceResourceId().withId("hvxndzwmkrefajpj"); model = BinaryData.fromObject(model).toObject(SliceResourceId.class); - Assertions.assertEquals("vydfceacvlhvygdy", model.id()); + Assertions.assertEquals("hvxndzwmkrefajpj", model.id()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateMockTests.java index 97ea58231dc32..9271573931780 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesCreateOrUpdateMockTests.java @@ -33,51 +33,35 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"snssai\":{\"sst\":1368337074,\"sd\":\"izhyhnep\"},\"description\":\"etiarxqiubxdukec\"},\"location\":\"dazvdhctm\",\"tags\":{\"iofkbtfmh\":\"szudblnsntrpcaq\"},\"id\":\"lbnld\",\"name\":\"vcb\",\"type\":\"hez\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"snssai\":{\"sst\":631894274,\"sd\":\"gmxitpfi\"},\"description\":\"cpdltkrlg\"},\"location\":\"tbdrvcqgue\",\"tags\":{\"lyujlfyoump\":\"ompheqdur\",\"brzmqxucycijoclx\":\"kyeclcdigpta\",\"zjd\":\"utgjcyz\",\"jb\":\"r\"},\"id\":\"xjeaoqaqbzgyh\",\"name\":\"w\",\"type\":\"v\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Slice response = - manager - .slices() - .define("kqevadrmmw") - .withRegion("iidisczskoswoqiq") - .withExistingMobileNetwork("c", "equygdjboqgrmtq") - .withSnssai(new Snssai().withSst(1097578040).withSd("c")) - .withTags(mapOf("amnppcce", "gamxzkrrcoiis")) - .withDescription("z") - .create(); + Slice response + = manager.slices().define("sgftipwc").withRegion("byl").withExistingMobileNetwork("ccgzpraoxnyu", "fa") + .withSnssai(new Snssai().withSst(437350600).withSd("hiqdxy")) + .withTags(mapOf("rkdlb", "gvxvatv", "ov", "bqxvhcsyhzlwxae", "ead", "rexdndsbd")) + .withDescription("npnuhzafccnuhi").create(); - Assertions.assertEquals("dazvdhctm", response.location()); - Assertions.assertEquals("szudblnsntrpcaq", response.tags().get("iofkbtfmh")); - Assertions.assertEquals(1368337074, response.snssai().sst()); - Assertions.assertEquals("izhyhnep", response.snssai().sd()); - Assertions.assertEquals("etiarxqiubxdukec", response.description()); + Assertions.assertEquals("tbdrvcqgue", response.location()); + Assertions.assertEquals("ompheqdur", response.tags().get("lyujlfyoump")); + Assertions.assertEquals(631894274, response.snssai().sst()); + Assertions.assertEquals("gmxitpfi", response.snssai().sd()); + Assertions.assertEquals("cpdltkrlg", response.description()); } // Use "Map.of" if available diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteMockTests.java index 266df1e7ce7c3..c78023914d98a 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.slices().delete("ygywe", "skieca", "ygzmxieqvdsmak", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.slices().delete("yildudxjascow", "f", "jkpdxphlkksnm", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetWithResponseMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetWithResponseMockTests.java index 1061a67e0770d..847a118e368e6 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetWithResponseMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesGetWithResponseMockTests.java @@ -30,45 +30,31 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Failed\",\"snssai\":{\"sst\":704277704,\"sd\":\"llizhce\"},\"description\":\"oqodkadpp\"},\"location\":\"bngqladywrx\",\"tags\":{\"yemlowuowhlxln\":\"dtluvvadswzsn\"},\"id\":\"y\",\"name\":\"mouvbl\",\"type\":\"mo\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Accepted\",\"snssai\":{\"sst\":1782311657,\"sd\":\"ulsutrjbhxy\"},\"description\":\"hyqezvqq\"},\"location\":\"drftbcvexreuquo\",\"tags\":{\"whreagkhyxvrq\":\"jv\",\"lmepjpf\":\"vbczsulmdg\",\"g\":\"eykgsangpszngaf\"},\"id\":\"lkvec\",\"name\":\"ujcngo\",\"type\":\"dyedmzrgjfo\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Slice response = - manager - .slices() - .getWithResponse("ixqcahyhxalybxaw", "ijpodtblxpkkwj", "jodqhykincn", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("bngqladywrx", response.location()); - Assertions.assertEquals("dtluvvadswzsn", response.tags().get("yemlowuowhlxln")); - Assertions.assertEquals(704277704, response.snssai().sst()); - Assertions.assertEquals("llizhce", response.snssai().sd()); - Assertions.assertEquals("oqodkadpp", response.description()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + Slice response = manager.slices() + .getWithResponse("zvyfijdkzuqnwsi", "huqol", "ah", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("drftbcvexreuquo", response.location()); + Assertions.assertEquals("jv", response.tags().get("whreagkhyxvrq")); + Assertions.assertEquals(1782311657, response.snssai().sst()); + Assertions.assertEquals("ulsutrjbhxy", response.snssai().sd()); + Assertions.assertEquals("hyqezvqq", response.description()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkMockTests.java index d10c98d0d7ea8..ce163bdfc6ec1 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkMockTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SlicesListByMobileNetworkMockTests.java @@ -31,42 +31,31 @@ public void testListByMobileNetwork() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleted\",\"snssai\":{\"sst\":867213638,\"sd\":\"qcrj\"},\"description\":\"h\"},\"location\":\"ukvhd\",\"tags\":{\"jyixhaf\":\"yojbfqzdkf\",\"al\":\"atqxmbjroumzzn\",\"on\":\"jrhuzgf\",\"tiotzb\":\"tpusllywp\"},\"id\":\"d\",\"name\":\"ollgry\",\"type\":\"qiuasigrows\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"snssai\":{\"sst\":1967343356,\"sd\":\"aswugyxpqitw\"},\"description\":\"alwvskbu\"},\"location\":\"acaqtyltcoqcu\",\"tags\":{\"bizt\":\"sxzakuejkm\",\"bemyeji\":\"ofqcvovjufycsjm\",\"rtudawlpjfel\":\"iuxegth\",\"bgqnz\":\"erppt\"},\"id\":\"nhii\",\"name\":\"ialwc\",\"type\":\"gckbb\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - MobileNetworkManager manager = - MobileNetworkManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.slices().listByMobileNetwork("zkltrfowtdvrfmv", "ih", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("ukvhd", response.iterator().next().location()); - Assertions.assertEquals("yojbfqzdkf", response.iterator().next().tags().get("jyixhaf")); - Assertions.assertEquals(867213638, response.iterator().next().snssai().sst()); - Assertions.assertEquals("qcrj", response.iterator().next().snssai().sd()); - Assertions.assertEquals("h", response.iterator().next().description()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.slices().listByMobileNetwork("nub", "oitpkpztrgdgx", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("acaqtyltcoqcu", response.iterator().next().location()); + Assertions.assertEquals("sxzakuejkm", response.iterator().next().tags().get("bizt")); + Assertions.assertEquals(1967343356, response.iterator().next().snssai().sst()); + Assertions.assertEquals("aswugyxpqitw", response.iterator().next().snssai().sd()); + Assertions.assertEquals("alwvskbu", response.iterator().next().description()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SnssaiTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SnssaiTests.java index 5a0dbbec5b174..afc04d8b24720 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SnssaiTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/SnssaiTests.java @@ -11,16 +11,16 @@ public final class SnssaiTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Snssai model = BinaryData.fromString("{\"sst\":1372249565,\"sd\":\"xlpm\"}").toObject(Snssai.class); - Assertions.assertEquals(1372249565, model.sst()); - Assertions.assertEquals("xlpm", model.sd()); + Snssai model = BinaryData.fromString("{\"sst\":1010160642,\"sd\":\"xakvvjgslor\"}").toObject(Snssai.class); + Assertions.assertEquals(1010160642, model.sst()); + Assertions.assertEquals("xakvvjgslor", model.sd()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Snssai model = new Snssai().withSst(1372249565).withSd("xlpm"); + Snssai model = new Snssai().withSst(1010160642).withSd("xakvvjgslor"); model = BinaryData.fromObject(model).toObject(Snssai.class); - Assertions.assertEquals(1372249565, model.sst()); - Assertions.assertEquals("xlpm", model.sd()); + Assertions.assertEquals(1010160642, model.sst()); + Assertions.assertEquals("xakvvjgslor", model.sd()); } } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/TagsObjectTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/TagsObjectTests.java index 8623a0bfdc8ef..164948c29a5c2 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/TagsObjectTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/TagsObjectTests.java @@ -13,19 +13,17 @@ public final class TagsObjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TagsObject model = - BinaryData - .fromString( - "{\"tags\":{\"jnchgej\":\"vnipjox\",\"huxinpmqnj\":\"podmailzydehojwy\",\"vcputegj\":\"qwixjspro\"}}") - .toObject(TagsObject.class); + TagsObject model = BinaryData + .fromString( + "{\"tags\":{\"jnchgej\":\"vnipjox\",\"huxinpmqnj\":\"podmailzydehojwy\",\"vcputegj\":\"qwixjspro\"}}") + .toObject(TagsObject.class); Assertions.assertEquals("vnipjox", model.tags().get("jnchgej")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TagsObject model = - new TagsObject() - .withTags(mapOf("jnchgej", "vnipjox", "huxinpmqnj", "podmailzydehojwy", "vcputegj", "qwixjspro")); + TagsObject model = new TagsObject() + .withTags(mapOf("jnchgej", "vnipjox", "huxinpmqnj", "podmailzydehojwy", "vcputegj", "qwixjspro")); model = BinaryData.fromObject(model).toObject(TagsObject.class); Assertions.assertEquals("vnipjox", model.tags().get("jnchgej")); } diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo4GTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo4GTests.java new file mode 100644 index 0000000000000..016c7b673e04f --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo4GTests.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.GNbId; +import com.azure.resourcemanager.mobilenetwork.models.GlobalRanNodeId; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.RrcEstablishmentCause; +import com.azure.resourcemanager.mobilenetwork.models.UeConnectionInfo4G; +import com.azure.resourcemanager.mobilenetwork.models.UeLocationInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import com.azure.resourcemanager.mobilenetwork.models.UeUsageSetting; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; + +public final class UeConnectionInfo4GTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeConnectionInfo4G model = BinaryData.fromString( + "{\"locationInfo\":{\"locationType\":\"wpypqtgsfj\",\"tac\":\"cbslhhx\",\"plmn\":{\"mcc\":\"db\",\"mnc\":\"vodhtn\"}},\"globalRanNodeId\":{\"plmnId\":{\"mcc\":\"irudh\",\"mnc\":\"m\"},\"gNbId\":{\"bitLength\":1714425112,\"gNBValue\":\"dlpagzrcxfail\"},\"ngeNbId\":\"xwmdboxd\",\"eNbId\":\"sftufqobrjlna\",\"n3IwfId\":\"cc\",\"wagfId\":\"nhxk\",\"tngfId\":\"v\",\"nid\":\"nrzvuljraaer\"},\"perUeTnla\":\"okqgukkjq\",\"mmeS1apId\":380919763,\"enbS1apId\":1534220522,\"lastVisitedTai\":\"oylaxxul\",\"ueState\":\"Detached\",\"rrcEstablishmentCause\":\"SMS\",\"ueUsageSetting\":\"DataCentric\",\"lastActivityTime\":\"2021-04-25T17:14:26Z\"}") + .toObject(UeConnectionInfo4G.class); + Assertions.assertEquals("wpypqtgsfj", model.locationInfo().locationType()); + Assertions.assertEquals("cbslhhx", model.locationInfo().tac()); + Assertions.assertEquals("db", model.locationInfo().plmn().mcc()); + Assertions.assertEquals("vodhtn", model.locationInfo().plmn().mnc()); + Assertions.assertEquals("irudh", model.globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("m", model.globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(1714425112, model.globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("dlpagzrcxfail", model.globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("xwmdboxd", model.globalRanNodeId().ngeNbId()); + Assertions.assertEquals("sftufqobrjlna", model.globalRanNodeId().eNbId()); + Assertions.assertEquals("cc", model.globalRanNodeId().n3IwfId()); + Assertions.assertEquals("nhxk", model.globalRanNodeId().wagfId()); + Assertions.assertEquals("v", model.globalRanNodeId().tngfId()); + Assertions.assertEquals("nrzvuljraaer", model.globalRanNodeId().nid()); + Assertions.assertEquals("okqgukkjq", model.perUeTnla()); + Assertions.assertEquals(380919763, model.mmeS1ApId()); + Assertions.assertEquals(1534220522, model.enbS1ApId()); + Assertions.assertEquals("oylaxxul", model.lastVisitedTai()); + Assertions.assertEquals(UeState.DETACHED, model.ueState()); + Assertions.assertEquals(RrcEstablishmentCause.SMS, model.rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.DATA_CENTRIC, model.ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-25T17:14:26Z"), model.lastActivityTime()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeConnectionInfo4G model = new UeConnectionInfo4G() + .withLocationInfo(new UeLocationInfo().withLocationType("wpypqtgsfj").withTac("cbslhhx") + .withPlmn(new PlmnId().withMcc("db").withMnc("vodhtn"))) + .withGlobalRanNodeId(new GlobalRanNodeId().withPlmnId(new PlmnId().withMcc("irudh").withMnc("m")) + .withGNbId(new GNbId().withBitLength(1714425112).withGNBValue("dlpagzrcxfail")).withNgeNbId("xwmdboxd") + .withENbId("sftufqobrjlna").withN3IwfId("cc").withWagfId("nhxk").withTngfId("v") + .withNid("nrzvuljraaer")) + .withPerUeTnla("okqgukkjq").withMmeS1ApId(380919763).withEnbS1ApId(1534220522) + .withLastVisitedTai("oylaxxul").withUeState(UeState.DETACHED) + .withRrcEstablishmentCause(RrcEstablishmentCause.SMS).withUeUsageSetting(UeUsageSetting.DATA_CENTRIC) + .withLastActivityTime(OffsetDateTime.parse("2021-04-25T17:14:26Z")); + model = BinaryData.fromObject(model).toObject(UeConnectionInfo4G.class); + Assertions.assertEquals("wpypqtgsfj", model.locationInfo().locationType()); + Assertions.assertEquals("cbslhhx", model.locationInfo().tac()); + Assertions.assertEquals("db", model.locationInfo().plmn().mcc()); + Assertions.assertEquals("vodhtn", model.locationInfo().plmn().mnc()); + Assertions.assertEquals("irudh", model.globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("m", model.globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(1714425112, model.globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("dlpagzrcxfail", model.globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("xwmdboxd", model.globalRanNodeId().ngeNbId()); + Assertions.assertEquals("sftufqobrjlna", model.globalRanNodeId().eNbId()); + Assertions.assertEquals("cc", model.globalRanNodeId().n3IwfId()); + Assertions.assertEquals("nhxk", model.globalRanNodeId().wagfId()); + Assertions.assertEquals("v", model.globalRanNodeId().tngfId()); + Assertions.assertEquals("nrzvuljraaer", model.globalRanNodeId().nid()); + Assertions.assertEquals("okqgukkjq", model.perUeTnla()); + Assertions.assertEquals(380919763, model.mmeS1ApId()); + Assertions.assertEquals(1534220522, model.enbS1ApId()); + Assertions.assertEquals("oylaxxul", model.lastVisitedTai()); + Assertions.assertEquals(UeState.DETACHED, model.ueState()); + Assertions.assertEquals(RrcEstablishmentCause.SMS, model.rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.DATA_CENTRIC, model.ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-25T17:14:26Z"), model.lastActivityTime()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo5GTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo5GTests.java new file mode 100644 index 0000000000000..4c7d3afb968ea --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeConnectionInfo5GTests.java @@ -0,0 +1,94 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.GNbId; +import com.azure.resourcemanager.mobilenetwork.models.GlobalRanNodeId; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.RrcEstablishmentCause; +import com.azure.resourcemanager.mobilenetwork.models.Snssai; +import com.azure.resourcemanager.mobilenetwork.models.UeConnectionInfo5G; +import com.azure.resourcemanager.mobilenetwork.models.UeLocationInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import com.azure.resourcemanager.mobilenetwork.models.UeUsageSetting; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeConnectionInfo5GTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeConnectionInfo5G model = BinaryData.fromString( + "{\"locationInfo\":{\"locationType\":\"duy\",\"tac\":\"vuzhyr\",\"plmn\":{\"mcc\":\"ewipm\",\"mnc\":\"ekdxuku\"}},\"globalRanNodeId\":{\"plmnId\":{\"mcc\":\"gsjj\",\"mnc\":\"undxgketw\"},\"gNbId\":{\"bitLength\":84465670,\"gNBValue\":\"hfjmhvv\"},\"ngeNbId\":\"vgpmun\",\"eNbId\":\"sx\",\"n3IwfId\":\"hfbuzjyihsasbhud\",\"wagfId\":\"ohyuemslynsq\",\"tngfId\":\"pfoobr\",\"nid\":\"tyms\"},\"perUeTnla\":\"ygqdnfwqzdz\",\"amfUeNgapId\":6824838811661161914,\"ranUeNgapId\":362294520,\"lastVisitedTai\":\"xhnfhqly\",\"allowedNssai\":[{\"sst\":2137536799,\"sd\":\"wivkxo\"}],\"ueState\":\"Connected\",\"rrcEstablishmentCause\":\"SMS\",\"ueUsageSetting\":\"VoiceCentric\",\"lastActivityTime\":\"2021-01-31T15:44:08Z\"}") + .toObject(UeConnectionInfo5G.class); + Assertions.assertEquals("duy", model.locationInfo().locationType()); + Assertions.assertEquals("vuzhyr", model.locationInfo().tac()); + Assertions.assertEquals("ewipm", model.locationInfo().plmn().mcc()); + Assertions.assertEquals("ekdxuku", model.locationInfo().plmn().mnc()); + Assertions.assertEquals("gsjj", model.globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("undxgketw", model.globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(84465670, model.globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("hfjmhvv", model.globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("vgpmun", model.globalRanNodeId().ngeNbId()); + Assertions.assertEquals("sx", model.globalRanNodeId().eNbId()); + Assertions.assertEquals("hfbuzjyihsasbhud", model.globalRanNodeId().n3IwfId()); + Assertions.assertEquals("ohyuemslynsq", model.globalRanNodeId().wagfId()); + Assertions.assertEquals("pfoobr", model.globalRanNodeId().tngfId()); + Assertions.assertEquals("tyms", model.globalRanNodeId().nid()); + Assertions.assertEquals("ygqdnfwqzdz", model.perUeTnla()); + Assertions.assertEquals(6824838811661161914L, model.amfUeNgapId()); + Assertions.assertEquals(362294520, model.ranUeNgapId()); + Assertions.assertEquals("xhnfhqly", model.lastVisitedTai()); + Assertions.assertEquals(2137536799, model.allowedNssai().get(0).sst()); + Assertions.assertEquals("wivkxo", model.allowedNssai().get(0).sd()); + Assertions.assertEquals(UeState.CONNECTED, model.ueState()); + Assertions.assertEquals(RrcEstablishmentCause.SMS, model.rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.VOICE_CENTRIC, model.ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-31T15:44:08Z"), model.lastActivityTime()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeConnectionInfo5G model = new UeConnectionInfo5G() + .withLocationInfo(new UeLocationInfo().withLocationType("duy").withTac("vuzhyr") + .withPlmn(new PlmnId().withMcc("ewipm").withMnc("ekdxuku"))) + .withGlobalRanNodeId(new GlobalRanNodeId().withPlmnId(new PlmnId().withMcc("gsjj").withMnc("undxgketw")) + .withGNbId(new GNbId().withBitLength(84465670).withGNBValue("hfjmhvv")).withNgeNbId("vgpmun") + .withENbId("sx").withN3IwfId("hfbuzjyihsasbhud").withWagfId("ohyuemslynsq").withTngfId("pfoobr") + .withNid("tyms")) + .withPerUeTnla("ygqdnfwqzdz").withAmfUeNgapId(6824838811661161914L).withRanUeNgapId(362294520) + .withLastVisitedTai("xhnfhqly") + .withAllowedNssai(Arrays.asList(new Snssai().withSst(2137536799).withSd("wivkxo"))) + .withUeState(UeState.CONNECTED).withRrcEstablishmentCause(RrcEstablishmentCause.SMS) + .withUeUsageSetting(UeUsageSetting.VOICE_CENTRIC) + .withLastActivityTime(OffsetDateTime.parse("2021-01-31T15:44:08Z")); + model = BinaryData.fromObject(model).toObject(UeConnectionInfo5G.class); + Assertions.assertEquals("duy", model.locationInfo().locationType()); + Assertions.assertEquals("vuzhyr", model.locationInfo().tac()); + Assertions.assertEquals("ewipm", model.locationInfo().plmn().mcc()); + Assertions.assertEquals("ekdxuku", model.locationInfo().plmn().mnc()); + Assertions.assertEquals("gsjj", model.globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("undxgketw", model.globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(84465670, model.globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("hfjmhvv", model.globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("vgpmun", model.globalRanNodeId().ngeNbId()); + Assertions.assertEquals("sx", model.globalRanNodeId().eNbId()); + Assertions.assertEquals("hfbuzjyihsasbhud", model.globalRanNodeId().n3IwfId()); + Assertions.assertEquals("ohyuemslynsq", model.globalRanNodeId().wagfId()); + Assertions.assertEquals("pfoobr", model.globalRanNodeId().tngfId()); + Assertions.assertEquals("tyms", model.globalRanNodeId().nid()); + Assertions.assertEquals("ygqdnfwqzdz", model.perUeTnla()); + Assertions.assertEquals(6824838811661161914L, model.amfUeNgapId()); + Assertions.assertEquals(362294520, model.ranUeNgapId()); + Assertions.assertEquals("xhnfhqly", model.lastVisitedTai()); + Assertions.assertEquals(2137536799, model.allowedNssai().get(0).sst()); + Assertions.assertEquals("wivkxo", model.allowedNssai().get(0).sd()); + Assertions.assertEquals(UeState.CONNECTED, model.ueState()); + Assertions.assertEquals(RrcEstablishmentCause.SMS, model.rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.VOICE_CENTRIC, model.ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-31T15:44:08Z"), model.lastActivityTime()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GPropertiesTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GPropertiesTests.java new file mode 100644 index 0000000000000..aab5f0d47380d --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GPropertiesTests.java @@ -0,0 +1,169 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.Ambr; +import com.azure.resourcemanager.mobilenetwork.models.AmfId; +import com.azure.resourcemanager.mobilenetwork.models.GNbId; +import com.azure.resourcemanager.mobilenetwork.models.GlobalRanNodeId; +import com.azure.resourcemanager.mobilenetwork.models.Guti5G; +import com.azure.resourcemanager.mobilenetwork.models.PdnType; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.RrcEstablishmentCause; +import com.azure.resourcemanager.mobilenetwork.models.Snssai; +import com.azure.resourcemanager.mobilenetwork.models.UeConnectionInfo5G; +import com.azure.resourcemanager.mobilenetwork.models.UeInfo5GProperties; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeLocationInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeQosFlow; +import com.azure.resourcemanager.mobilenetwork.models.UeSessionInfo5G; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import com.azure.resourcemanager.mobilenetwork.models.UeUsageSetting; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeInfo5GPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeInfo5GProperties model = BinaryData.fromString( + "{\"supi\":\"qikfxcv\",\"pei\":\"fsphuagrttikt\",\"fivegGuti\":{\"plmn\":{\"mcc\":\"usqczkvy\",\"mnc\":\"lxubyj\"},\"amfId\":{\"regionId\":490127820,\"setId\":472498299,\"pointer\":1774375239},\"fivegTmsi\":716445965},\"connectionInfo\":{\"locationInfo\":{\"locationType\":\"lcqcuubgqibrt\",\"tac\":\"lmetttwgdslqxi\",\"plmn\":{\"mcc\":\"hrmooi\",\"mnc\":\"qseypxiutcxa\"}},\"globalRanNodeId\":{\"plmnId\":{\"mcc\":\"zhyrpeto\",\"mnc\":\"e\"},\"gNbId\":{\"bitLength\":1642454169,\"gNBValue\":\"lhvnhlab\"},\"ngeNbId\":\"nk\",\"eNbId\":\"jcjbt\",\"n3IwfId\":\"aehvvibrxjjstoq\",\"wagfId\":\"it\",\"tngfId\":\"xztmo\",\"nid\":\"klf\"},\"perUeTnla\":\"dgfcwqmp\",\"amfUeNgapId\":2829270092150598933,\"ranUeNgapId\":1479549397,\"lastVisitedTai\":\"zhe\",\"allowedNssai\":[{\"sst\":107725068,\"sd\":\"hujswtwkozzwcul\"},{\"sst\":1771791056,\"sd\":\"wpfaj\"},{\"sst\":654985035,\"sd\":\"ltlwtjjguktalhs\"},{\"sst\":2060151419,\"sd\":\"cdmxzrpoaiml\"}],\"ueState\":\"Detached\",\"rrcEstablishmentCause\":\"Emergency\",\"ueUsageSetting\":\"VoiceCentric\",\"lastActivityTime\":\"2021-12-05T18:05:26Z\"},\"sessionInfo\":[{\"pduSessionId\":1405503758,\"dnn\":\"eazulcs\",\"snssai\":{\"sst\":1226042956,\"sd\":\"wwnpj\"},\"ueIpAddress\":{\"ipV4Addr\":\"fz\"},\"pdnType\":\"IPV4\",\"ambr\":{\"uplink\":\"pchwa\",\"downlink\":\"fb\"},\"qosFlow\":[{\"qfi\":1658070092,\"fiveqi\":1497864768,\"mbr\":{\"uplink\":\"epgfew\",\"downlink\":\"twly\"},\"gbr\":{\"uplink\":\"ncxykxhdjhlimm\",\"downlink\":\"cxfhbcporxv\"}},{\"qfi\":1548715838,\"fiveqi\":637127144,\"mbr\":{\"uplink\":\"hqizxfpxtgqscj\",\"downlink\":\"vftjuhd\"},\"gbr\":{\"uplink\":\"zkmtgguwpijrajci\",\"downlink\":\"m\"}},{\"qfi\":1977689221,\"fiveqi\":1010868676,\"mbr\":{\"uplink\":\"cf\",\"downlink\":\"wrxgkneuvy\"},\"gbr\":{\"uplink\":\"zqodfvpgshox\",\"downlink\":\"sgbpfgzdjtx\"}}]}]}") + .toObject(UeInfo5GProperties.class); + Assertions.assertEquals("qikfxcv", model.supi()); + Assertions.assertEquals("fsphuagrttikt", model.pei()); + Assertions.assertEquals("usqczkvy", model.fivegGuti().plmn().mcc()); + Assertions.assertEquals("lxubyj", model.fivegGuti().plmn().mnc()); + Assertions.assertEquals(490127820, model.fivegGuti().amfId().regionId()); + Assertions.assertEquals(472498299, model.fivegGuti().amfId().setId()); + Assertions.assertEquals(1774375239, model.fivegGuti().amfId().pointer()); + Assertions.assertEquals(716445965, model.fivegGuti().fivegTmsi()); + Assertions.assertEquals("lcqcuubgqibrt", model.connectionInfo().locationInfo().locationType()); + Assertions.assertEquals("lmetttwgdslqxi", model.connectionInfo().locationInfo().tac()); + Assertions.assertEquals("hrmooi", model.connectionInfo().locationInfo().plmn().mcc()); + Assertions.assertEquals("qseypxiutcxa", model.connectionInfo().locationInfo().plmn().mnc()); + Assertions.assertEquals("zhyrpeto", model.connectionInfo().globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("e", model.connectionInfo().globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(1642454169, model.connectionInfo().globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("lhvnhlab", model.connectionInfo().globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("nk", model.connectionInfo().globalRanNodeId().ngeNbId()); + Assertions.assertEquals("jcjbt", model.connectionInfo().globalRanNodeId().eNbId()); + Assertions.assertEquals("aehvvibrxjjstoq", model.connectionInfo().globalRanNodeId().n3IwfId()); + Assertions.assertEquals("it", model.connectionInfo().globalRanNodeId().wagfId()); + Assertions.assertEquals("xztmo", model.connectionInfo().globalRanNodeId().tngfId()); + Assertions.assertEquals("klf", model.connectionInfo().globalRanNodeId().nid()); + Assertions.assertEquals("dgfcwqmp", model.connectionInfo().perUeTnla()); + Assertions.assertEquals(2829270092150598933L, model.connectionInfo().amfUeNgapId()); + Assertions.assertEquals(1479549397, model.connectionInfo().ranUeNgapId()); + Assertions.assertEquals("zhe", model.connectionInfo().lastVisitedTai()); + Assertions.assertEquals(107725068, model.connectionInfo().allowedNssai().get(0).sst()); + Assertions.assertEquals("hujswtwkozzwcul", model.connectionInfo().allowedNssai().get(0).sd()); + Assertions.assertEquals(UeState.DETACHED, model.connectionInfo().ueState()); + Assertions.assertEquals(RrcEstablishmentCause.EMERGENCY, model.connectionInfo().rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.VOICE_CENTRIC, model.connectionInfo().ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-12-05T18:05:26Z"), + model.connectionInfo().lastActivityTime()); + Assertions.assertEquals(1405503758, model.sessionInfo().get(0).pduSessionId()); + Assertions.assertEquals("eazulcs", model.sessionInfo().get(0).dnn()); + Assertions.assertEquals(1226042956, model.sessionInfo().get(0).snssai().sst()); + Assertions.assertEquals("wwnpj", model.sessionInfo().get(0).snssai().sd()); + Assertions.assertEquals("fz", model.sessionInfo().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.sessionInfo().get(0).pdnType()); + Assertions.assertEquals("pchwa", model.sessionInfo().get(0).ambr().uplink()); + Assertions.assertEquals("fb", model.sessionInfo().get(0).ambr().downlink()); + Assertions.assertEquals(1658070092, model.sessionInfo().get(0).qosFlow().get(0).qfi()); + Assertions.assertEquals(1497864768, model.sessionInfo().get(0).qosFlow().get(0).fiveqi()); + Assertions.assertEquals("epgfew", model.sessionInfo().get(0).qosFlow().get(0).mbr().uplink()); + Assertions.assertEquals("twly", model.sessionInfo().get(0).qosFlow().get(0).mbr().downlink()); + Assertions.assertEquals("ncxykxhdjhlimm", model.sessionInfo().get(0).qosFlow().get(0).gbr().uplink()); + Assertions.assertEquals("cxfhbcporxv", model.sessionInfo().get(0).qosFlow().get(0).gbr().downlink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeInfo5GProperties model = new UeInfo5GProperties().withSupi("qikfxcv").withPei("fsphuagrttikt") + .withFivegGuti(new Guti5G().withPlmn(new PlmnId().withMcc("usqczkvy").withMnc("lxubyj")) + .withAmfId(new AmfId().withRegionId(490127820).withSetId(472498299).withPointer(1774375239)) + .withFivegTmsi(716445965)) + .withConnectionInfo(new UeConnectionInfo5G() + .withLocationInfo(new UeLocationInfo().withLocationType("lcqcuubgqibrt").withTac("lmetttwgdslqxi") + .withPlmn(new PlmnId().withMcc("hrmooi").withMnc("qseypxiutcxa"))) + .withGlobalRanNodeId(new GlobalRanNodeId().withPlmnId(new PlmnId().withMcc("zhyrpeto").withMnc("e")) + .withGNbId(new GNbId().withBitLength(1642454169).withGNBValue("lhvnhlab")).withNgeNbId("nk") + .withENbId("jcjbt").withN3IwfId("aehvvibrxjjstoq").withWagfId("it").withTngfId("xztmo") + .withNid("klf")) + .withPerUeTnla("dgfcwqmp").withAmfUeNgapId(2829270092150598933L).withRanUeNgapId(1479549397) + .withLastVisitedTai("zhe") + .withAllowedNssai(Arrays.asList(new Snssai().withSst(107725068).withSd("hujswtwkozzwcul"), + new Snssai().withSst(1771791056).withSd("wpfaj"), + new Snssai().withSst(654985035).withSd("ltlwtjjguktalhs"), + new Snssai().withSst(2060151419).withSd("cdmxzrpoaiml"))) + .withUeState(UeState.DETACHED).withRrcEstablishmentCause(RrcEstablishmentCause.EMERGENCY) + .withUeUsageSetting(UeUsageSetting.VOICE_CENTRIC).withLastActivityTime( + OffsetDateTime.parse("2021-12-05T18:05:26Z"))) + .withSessionInfo(Arrays.asList(new UeSessionInfo5G().withPduSessionId(1405503758).withDnn("eazulcs") + .withSnssai(new Snssai().withSst(1226042956).withSd("wwnpj")) + .withUeIpAddress(new UeIpAddress().withIpV4Addr("fz")).withPdnType(PdnType.IPV4) + .withAmbr(new Ambr().withUplink("pchwa").withDownlink("fb")) + .withQosFlow(Arrays.asList( + new UeQosFlow().withQfi(1658070092).withFiveqi(1497864768) + .withMbr(new Ambr().withUplink("epgfew").withDownlink("twly")) + .withGbr(new Ambr().withUplink("ncxykxhdjhlimm").withDownlink("cxfhbcporxv")), + new UeQosFlow().withQfi(1548715838).withFiveqi(637127144) + .withMbr(new Ambr().withUplink("hqizxfpxtgqscj").withDownlink("vftjuhd")) + .withGbr(new Ambr().withUplink("zkmtgguwpijrajci").withDownlink("m")), + new UeQosFlow().withQfi(1977689221).withFiveqi(1010868676) + .withMbr(new Ambr().withUplink("cf").withDownlink("wrxgkneuvy")) + .withGbr(new Ambr().withUplink("zqodfvpgshox").withDownlink("sgbpfgzdjtx")))))); + model = BinaryData.fromObject(model).toObject(UeInfo5GProperties.class); + Assertions.assertEquals("qikfxcv", model.supi()); + Assertions.assertEquals("fsphuagrttikt", model.pei()); + Assertions.assertEquals("usqczkvy", model.fivegGuti().plmn().mcc()); + Assertions.assertEquals("lxubyj", model.fivegGuti().plmn().mnc()); + Assertions.assertEquals(490127820, model.fivegGuti().amfId().regionId()); + Assertions.assertEquals(472498299, model.fivegGuti().amfId().setId()); + Assertions.assertEquals(1774375239, model.fivegGuti().amfId().pointer()); + Assertions.assertEquals(716445965, model.fivegGuti().fivegTmsi()); + Assertions.assertEquals("lcqcuubgqibrt", model.connectionInfo().locationInfo().locationType()); + Assertions.assertEquals("lmetttwgdslqxi", model.connectionInfo().locationInfo().tac()); + Assertions.assertEquals("hrmooi", model.connectionInfo().locationInfo().plmn().mcc()); + Assertions.assertEquals("qseypxiutcxa", model.connectionInfo().locationInfo().plmn().mnc()); + Assertions.assertEquals("zhyrpeto", model.connectionInfo().globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("e", model.connectionInfo().globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(1642454169, model.connectionInfo().globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("lhvnhlab", model.connectionInfo().globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("nk", model.connectionInfo().globalRanNodeId().ngeNbId()); + Assertions.assertEquals("jcjbt", model.connectionInfo().globalRanNodeId().eNbId()); + Assertions.assertEquals("aehvvibrxjjstoq", model.connectionInfo().globalRanNodeId().n3IwfId()); + Assertions.assertEquals("it", model.connectionInfo().globalRanNodeId().wagfId()); + Assertions.assertEquals("xztmo", model.connectionInfo().globalRanNodeId().tngfId()); + Assertions.assertEquals("klf", model.connectionInfo().globalRanNodeId().nid()); + Assertions.assertEquals("dgfcwqmp", model.connectionInfo().perUeTnla()); + Assertions.assertEquals(2829270092150598933L, model.connectionInfo().amfUeNgapId()); + Assertions.assertEquals(1479549397, model.connectionInfo().ranUeNgapId()); + Assertions.assertEquals("zhe", model.connectionInfo().lastVisitedTai()); + Assertions.assertEquals(107725068, model.connectionInfo().allowedNssai().get(0).sst()); + Assertions.assertEquals("hujswtwkozzwcul", model.connectionInfo().allowedNssai().get(0).sd()); + Assertions.assertEquals(UeState.DETACHED, model.connectionInfo().ueState()); + Assertions.assertEquals(RrcEstablishmentCause.EMERGENCY, model.connectionInfo().rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.VOICE_CENTRIC, model.connectionInfo().ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-12-05T18:05:26Z"), + model.connectionInfo().lastActivityTime()); + Assertions.assertEquals(1405503758, model.sessionInfo().get(0).pduSessionId()); + Assertions.assertEquals("eazulcs", model.sessionInfo().get(0).dnn()); + Assertions.assertEquals(1226042956, model.sessionInfo().get(0).snssai().sst()); + Assertions.assertEquals("wwnpj", model.sessionInfo().get(0).snssai().sd()); + Assertions.assertEquals("fz", model.sessionInfo().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.sessionInfo().get(0).pdnType()); + Assertions.assertEquals("pchwa", model.sessionInfo().get(0).ambr().uplink()); + Assertions.assertEquals("fb", model.sessionInfo().get(0).ambr().downlink()); + Assertions.assertEquals(1658070092, model.sessionInfo().get(0).qosFlow().get(0).qfi()); + Assertions.assertEquals(1497864768, model.sessionInfo().get(0).qosFlow().get(0).fiveqi()); + Assertions.assertEquals("epgfew", model.sessionInfo().get(0).qosFlow().get(0).mbr().uplink()); + Assertions.assertEquals("twly", model.sessionInfo().get(0).qosFlow().get(0).mbr().downlink()); + Assertions.assertEquals("ncxykxhdjhlimm", model.sessionInfo().get(0).qosFlow().get(0).gbr().uplink()); + Assertions.assertEquals("cxfhbcporxv", model.sessionInfo().get(0).qosFlow().get(0).gbr().downlink()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GTests.java new file mode 100644 index 0000000000000..ead8809552682 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfo5GTests.java @@ -0,0 +1,166 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.Ambr; +import com.azure.resourcemanager.mobilenetwork.models.AmfId; +import com.azure.resourcemanager.mobilenetwork.models.GNbId; +import com.azure.resourcemanager.mobilenetwork.models.GlobalRanNodeId; +import com.azure.resourcemanager.mobilenetwork.models.Guti5G; +import com.azure.resourcemanager.mobilenetwork.models.PdnType; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.RrcEstablishmentCause; +import com.azure.resourcemanager.mobilenetwork.models.Snssai; +import com.azure.resourcemanager.mobilenetwork.models.UeConnectionInfo5G; +import com.azure.resourcemanager.mobilenetwork.models.UeInfo5G; +import com.azure.resourcemanager.mobilenetwork.models.UeInfo5GProperties; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeLocationInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeQosFlow; +import com.azure.resourcemanager.mobilenetwork.models.UeSessionInfo5G; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import com.azure.resourcemanager.mobilenetwork.models.UeUsageSetting; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeInfo5GTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeInfo5G model = BinaryData.fromString( + "{\"ratType\":\"5G\",\"info\":{\"supi\":\"m\",\"pei\":\"a\",\"fivegGuti\":{\"plmn\":{\"mcc\":\"wgck\",\"mnc\":\"ocxvdfffwafqr\"},\"amfId\":{\"regionId\":478345716,\"setId\":1019115621,\"pointer\":113317343},\"fivegTmsi\":1181286444},\"connectionInfo\":{\"locationInfo\":{\"locationType\":\"vehhrvkbunzozu\",\"tac\":\"hcxgkmoyx\",\"plmn\":{\"mcc\":\"dyuib\",\"mnc\":\"mfdn\"}},\"globalRanNodeId\":{\"plmnId\":{\"mcc\":\"zydvfvf\",\"mnc\":\"jnaeois\"},\"gNbId\":{\"bitLength\":716191050,\"gNBValue\":\"orffukiscv\"},\"ngeNbId\":\"zhwplefaxvxilc\",\"eNbId\":\"gnhnzeyq\",\"n3IwfId\":\"jjfzqlqhycavo\",\"wagfId\":\"gxdbeesmie\",\"tngfId\":\"lrariaawiu\",\"nid\":\"ydwqfbylyrf\"},\"perUeTnla\":\"agt\",\"amfUeNgapId\":143948817696336233,\"ranUeNgapId\":1125599847,\"lastVisitedTai\":\"qwogfnzjvus\",\"allowedNssai\":[{\"sst\":1215599551,\"sd\":\"ozuxylfsbtkadpys\"}],\"ueState\":\"Idle\",\"rrcEstablishmentCause\":\"MobileTerminatedData\",\"ueUsageSetting\":\"DataCentric\",\"lastActivityTime\":\"2021-04-26T21:54:32Z\"},\"sessionInfo\":[{\"pduSessionId\":792325056,\"dnn\":\"rj\",\"snssai\":{\"sst\":587108165,\"sd\":\"ojcmisofieyp\"},\"ueIpAddress\":{\"ipV4Addr\":\"ojyqdhcuplcplcw\"},\"pdnType\":\"IPV4\",\"ambr\":{\"uplink\":\"ihih\",\"downlink\":\"hzdsqtzbsrgnow\"},\"qosFlow\":[{\"qfi\":994844547,\"fiveqi\":1143830782,\"mbr\":{\"uplink\":\"mvec\",\"downlink\":\"ctxmwoteyowcluq\"},\"gbr\":{\"uplink\":\"ekqvgqouwif\",\"downlink\":\"mpjw\"}}]}]},\"lastReadAt\":\"2021-08-17T21:45:28Z\"}") + .toObject(UeInfo5G.class); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-17T21:45:28Z"), model.lastReadAt()); + Assertions.assertEquals("m", model.info().supi()); + Assertions.assertEquals("a", model.info().pei()); + Assertions.assertEquals("wgck", model.info().fivegGuti().plmn().mcc()); + Assertions.assertEquals("ocxvdfffwafqr", model.info().fivegGuti().plmn().mnc()); + Assertions.assertEquals(478345716, model.info().fivegGuti().amfId().regionId()); + Assertions.assertEquals(1019115621, model.info().fivegGuti().amfId().setId()); + Assertions.assertEquals(113317343, model.info().fivegGuti().amfId().pointer()); + Assertions.assertEquals(1181286444, model.info().fivegGuti().fivegTmsi()); + Assertions.assertEquals("vehhrvkbunzozu", model.info().connectionInfo().locationInfo().locationType()); + Assertions.assertEquals("hcxgkmoyx", model.info().connectionInfo().locationInfo().tac()); + Assertions.assertEquals("dyuib", model.info().connectionInfo().locationInfo().plmn().mcc()); + Assertions.assertEquals("mfdn", model.info().connectionInfo().locationInfo().plmn().mnc()); + Assertions.assertEquals("zydvfvf", model.info().connectionInfo().globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("jnaeois", model.info().connectionInfo().globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(716191050, model.info().connectionInfo().globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("orffukiscv", model.info().connectionInfo().globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("zhwplefaxvxilc", model.info().connectionInfo().globalRanNodeId().ngeNbId()); + Assertions.assertEquals("gnhnzeyq", model.info().connectionInfo().globalRanNodeId().eNbId()); + Assertions.assertEquals("jjfzqlqhycavo", model.info().connectionInfo().globalRanNodeId().n3IwfId()); + Assertions.assertEquals("gxdbeesmie", model.info().connectionInfo().globalRanNodeId().wagfId()); + Assertions.assertEquals("lrariaawiu", model.info().connectionInfo().globalRanNodeId().tngfId()); + Assertions.assertEquals("ydwqfbylyrf", model.info().connectionInfo().globalRanNodeId().nid()); + Assertions.assertEquals("agt", model.info().connectionInfo().perUeTnla()); + Assertions.assertEquals(143948817696336233L, model.info().connectionInfo().amfUeNgapId()); + Assertions.assertEquals(1125599847, model.info().connectionInfo().ranUeNgapId()); + Assertions.assertEquals("qwogfnzjvus", model.info().connectionInfo().lastVisitedTai()); + Assertions.assertEquals(1215599551, model.info().connectionInfo().allowedNssai().get(0).sst()); + Assertions.assertEquals("ozuxylfsbtkadpys", model.info().connectionInfo().allowedNssai().get(0).sd()); + Assertions.assertEquals(UeState.IDLE, model.info().connectionInfo().ueState()); + Assertions.assertEquals(RrcEstablishmentCause.MOBILE_TERMINATED_DATA, + model.info().connectionInfo().rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.DATA_CENTRIC, model.info().connectionInfo().ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-26T21:54:32Z"), + model.info().connectionInfo().lastActivityTime()); + Assertions.assertEquals(792325056, model.info().sessionInfo().get(0).pduSessionId()); + Assertions.assertEquals("rj", model.info().sessionInfo().get(0).dnn()); + Assertions.assertEquals(587108165, model.info().sessionInfo().get(0).snssai().sst()); + Assertions.assertEquals("ojcmisofieyp", model.info().sessionInfo().get(0).snssai().sd()); + Assertions.assertEquals("ojyqdhcuplcplcw", model.info().sessionInfo().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.info().sessionInfo().get(0).pdnType()); + Assertions.assertEquals("ihih", model.info().sessionInfo().get(0).ambr().uplink()); + Assertions.assertEquals("hzdsqtzbsrgnow", model.info().sessionInfo().get(0).ambr().downlink()); + Assertions.assertEquals(994844547, model.info().sessionInfo().get(0).qosFlow().get(0).qfi()); + Assertions.assertEquals(1143830782, model.info().sessionInfo().get(0).qosFlow().get(0).fiveqi()); + Assertions.assertEquals("mvec", model.info().sessionInfo().get(0).qosFlow().get(0).mbr().uplink()); + Assertions.assertEquals("ctxmwoteyowcluq", model.info().sessionInfo().get(0).qosFlow().get(0).mbr().downlink()); + Assertions.assertEquals("ekqvgqouwif", model.info().sessionInfo().get(0).qosFlow().get(0).gbr().uplink()); + Assertions.assertEquals("mpjw", model.info().sessionInfo().get(0).qosFlow().get(0).gbr().downlink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeInfo5G model = new UeInfo5G().withLastReadAt(OffsetDateTime.parse("2021-08-17T21:45:28Z")) + .withInfo(new UeInfo5GProperties().withSupi("m").withPei("a") + .withFivegGuti(new Guti5G().withPlmn(new PlmnId().withMcc("wgck").withMnc("ocxvdfffwafqr")) + .withAmfId(new AmfId().withRegionId(478345716).withSetId(1019115621).withPointer(113317343)) + .withFivegTmsi(1181286444)) + .withConnectionInfo(new UeConnectionInfo5G() + .withLocationInfo(new UeLocationInfo().withLocationType("vehhrvkbunzozu").withTac("hcxgkmoyx") + .withPlmn(new PlmnId().withMcc("dyuib").withMnc("mfdn"))) + .withGlobalRanNodeId( + new GlobalRanNodeId().withPlmnId(new PlmnId().withMcc("zydvfvf").withMnc("jnaeois")) + .withGNbId(new GNbId().withBitLength(716191050).withGNBValue("orffukiscv")) + .withNgeNbId("zhwplefaxvxilc").withENbId("gnhnzeyq").withN3IwfId("jjfzqlqhycavo") + .withWagfId("gxdbeesmie").withTngfId("lrariaawiu").withNid("ydwqfbylyrf")) + .withPerUeTnla("agt").withAmfUeNgapId(143948817696336233L).withRanUeNgapId(1125599847) + .withLastVisitedTai("qwogfnzjvus") + .withAllowedNssai(Arrays.asList(new Snssai().withSst(1215599551).withSd("ozuxylfsbtkadpys"))) + .withUeState(UeState.IDLE).withRrcEstablishmentCause(RrcEstablishmentCause.MOBILE_TERMINATED_DATA) + .withUeUsageSetting(UeUsageSetting.DATA_CENTRIC) + .withLastActivityTime(OffsetDateTime.parse("2021-04-26T21:54:32Z"))) + .withSessionInfo(Arrays.asList(new UeSessionInfo5G().withPduSessionId(792325056).withDnn("rj") + .withSnssai(new Snssai().withSst(587108165).withSd("ojcmisofieyp")) + .withUeIpAddress(new UeIpAddress().withIpV4Addr("ojyqdhcuplcplcw")).withPdnType(PdnType.IPV4) + .withAmbr(new Ambr().withUplink("ihih").withDownlink("hzdsqtzbsrgnow")) + .withQosFlow(Arrays.asList(new UeQosFlow().withQfi(994844547).withFiveqi(1143830782) + .withMbr(new Ambr().withUplink("mvec").withDownlink("ctxmwoteyowcluq")) + .withGbr(new Ambr().withUplink("ekqvgqouwif").withDownlink("mpjw"))))))); + model = BinaryData.fromObject(model).toObject(UeInfo5G.class); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-17T21:45:28Z"), model.lastReadAt()); + Assertions.assertEquals("m", model.info().supi()); + Assertions.assertEquals("a", model.info().pei()); + Assertions.assertEquals("wgck", model.info().fivegGuti().plmn().mcc()); + Assertions.assertEquals("ocxvdfffwafqr", model.info().fivegGuti().plmn().mnc()); + Assertions.assertEquals(478345716, model.info().fivegGuti().amfId().regionId()); + Assertions.assertEquals(1019115621, model.info().fivegGuti().amfId().setId()); + Assertions.assertEquals(113317343, model.info().fivegGuti().amfId().pointer()); + Assertions.assertEquals(1181286444, model.info().fivegGuti().fivegTmsi()); + Assertions.assertEquals("vehhrvkbunzozu", model.info().connectionInfo().locationInfo().locationType()); + Assertions.assertEquals("hcxgkmoyx", model.info().connectionInfo().locationInfo().tac()); + Assertions.assertEquals("dyuib", model.info().connectionInfo().locationInfo().plmn().mcc()); + Assertions.assertEquals("mfdn", model.info().connectionInfo().locationInfo().plmn().mnc()); + Assertions.assertEquals("zydvfvf", model.info().connectionInfo().globalRanNodeId().plmnId().mcc()); + Assertions.assertEquals("jnaeois", model.info().connectionInfo().globalRanNodeId().plmnId().mnc()); + Assertions.assertEquals(716191050, model.info().connectionInfo().globalRanNodeId().gNbId().bitLength()); + Assertions.assertEquals("orffukiscv", model.info().connectionInfo().globalRanNodeId().gNbId().gNBValue()); + Assertions.assertEquals("zhwplefaxvxilc", model.info().connectionInfo().globalRanNodeId().ngeNbId()); + Assertions.assertEquals("gnhnzeyq", model.info().connectionInfo().globalRanNodeId().eNbId()); + Assertions.assertEquals("jjfzqlqhycavo", model.info().connectionInfo().globalRanNodeId().n3IwfId()); + Assertions.assertEquals("gxdbeesmie", model.info().connectionInfo().globalRanNodeId().wagfId()); + Assertions.assertEquals("lrariaawiu", model.info().connectionInfo().globalRanNodeId().tngfId()); + Assertions.assertEquals("ydwqfbylyrf", model.info().connectionInfo().globalRanNodeId().nid()); + Assertions.assertEquals("agt", model.info().connectionInfo().perUeTnla()); + Assertions.assertEquals(143948817696336233L, model.info().connectionInfo().amfUeNgapId()); + Assertions.assertEquals(1125599847, model.info().connectionInfo().ranUeNgapId()); + Assertions.assertEquals("qwogfnzjvus", model.info().connectionInfo().lastVisitedTai()); + Assertions.assertEquals(1215599551, model.info().connectionInfo().allowedNssai().get(0).sst()); + Assertions.assertEquals("ozuxylfsbtkadpys", model.info().connectionInfo().allowedNssai().get(0).sd()); + Assertions.assertEquals(UeState.IDLE, model.info().connectionInfo().ueState()); + Assertions.assertEquals(RrcEstablishmentCause.MOBILE_TERMINATED_DATA, + model.info().connectionInfo().rrcEstablishmentCause()); + Assertions.assertEquals(UeUsageSetting.DATA_CENTRIC, model.info().connectionInfo().ueUsageSetting()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-26T21:54:32Z"), + model.info().connectionInfo().lastActivityTime()); + Assertions.assertEquals(792325056, model.info().sessionInfo().get(0).pduSessionId()); + Assertions.assertEquals("rj", model.info().sessionInfo().get(0).dnn()); + Assertions.assertEquals(587108165, model.info().sessionInfo().get(0).snssai().sst()); + Assertions.assertEquals("ojcmisofieyp", model.info().sessionInfo().get(0).snssai().sd()); + Assertions.assertEquals("ojyqdhcuplcplcw", model.info().sessionInfo().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.info().sessionInfo().get(0).pdnType()); + Assertions.assertEquals("ihih", model.info().sessionInfo().get(0).ambr().uplink()); + Assertions.assertEquals("hzdsqtzbsrgnow", model.info().sessionInfo().get(0).ambr().downlink()); + Assertions.assertEquals(994844547, model.info().sessionInfo().get(0).qosFlow().get(0).qfi()); + Assertions.assertEquals(1143830782, model.info().sessionInfo().get(0).qosFlow().get(0).fiveqi()); + Assertions.assertEquals("mvec", model.info().sessionInfo().get(0).qosFlow().get(0).mbr().uplink()); + Assertions.assertEquals("ctxmwoteyowcluq", model.info().sessionInfo().get(0).qosFlow().get(0).mbr().downlink()); + Assertions.assertEquals("ekqvgqouwif", model.info().sessionInfo().get(0).qosFlow().get(0).gbr().uplink()); + Assertions.assertEquals("mpjw", model.info().sessionInfo().get(0).qosFlow().get(0).gbr().downlink()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoInnerTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoInnerTests.java new file mode 100644 index 0000000000000..ee8b6486863ae --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoInnerTests.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.DnnIpPair; +import com.azure.resourcemanager.mobilenetwork.models.RatType; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoPropertiesFormat; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeInfoInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeInfoInner model = BinaryData.fromString( + "{\"properties\":{\"ratType\":\"5G\",\"ueState\":\"Connected\",\"ueIpAddresses\":[{\"dnn\":\"mdy\",\"ueIpAddress\":{\"ipV4Addr\":\"wdigumbnraauz\"}}],\"lastReadAt\":\"2021-06-11T04:36:53Z\"},\"id\":\"a\",\"name\":\"ysdzhez\",\"type\":\"wva\"}") + .toObject(UeInfoInner.class); + Assertions.assertEquals(RatType.FIVEG, model.properties().ratType()); + Assertions.assertEquals(UeState.CONNECTED, model.properties().ueState()); + Assertions.assertEquals("mdy", model.properties().ueIpAddresses().get(0).dnn()); + Assertions.assertEquals("wdigumbnraauz", model.properties().ueIpAddresses().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-11T04:36:53Z"), model.properties().lastReadAt()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeInfoInner model = new UeInfoInner() + .withProperties(new UeInfoPropertiesFormat().withRatType(RatType.FIVEG).withUeState(UeState.CONNECTED) + .withUeIpAddresses(Arrays.asList( + new DnnIpPair().withDnn("mdy").withUeIpAddress(new UeIpAddress().withIpV4Addr("wdigumbnraauz")))) + .withLastReadAt(OffsetDateTime.parse("2021-06-11T04:36:53Z"))); + model = BinaryData.fromObject(model).toObject(UeInfoInner.class); + Assertions.assertEquals(RatType.FIVEG, model.properties().ratType()); + Assertions.assertEquals(UeState.CONNECTED, model.properties().ueState()); + Assertions.assertEquals("mdy", model.properties().ueIpAddresses().get(0).dnn()); + Assertions.assertEquals("wdigumbnraauz", model.properties().ueIpAddresses().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-11T04:36:53Z"), model.properties().lastReadAt()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoListTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoListTests.java new file mode 100644 index 0000000000000..4feb19baf85aa --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoListTests.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.fluent.models.UeInfoInner; +import com.azure.resourcemanager.mobilenetwork.models.DnnIpPair; +import com.azure.resourcemanager.mobilenetwork.models.RatType; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoList; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoPropertiesFormat; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeInfoListTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeInfoList model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"ratType\":\"4G\",\"ueState\":\"Connected\",\"ueIpAddresses\":[{\"dnn\":\"aecxndtic\",\"ueIpAddress\":{}}],\"lastReadAt\":\"2021-12-05T04:32:03Z\"},\"id\":\"mlqtmldgxob\",\"name\":\"irclnpk\",\"type\":\"iayz\"}],\"nextLink\":\"ykhyawfvjlboxqvk\"}") + .toObject(UeInfoList.class); + Assertions.assertEquals(RatType.FOURG, model.value().get(0).properties().ratType()); + Assertions.assertEquals(UeState.CONNECTED, model.value().get(0).properties().ueState()); + Assertions.assertEquals("aecxndtic", model.value().get(0).properties().ueIpAddresses().get(0).dnn()); + Assertions.assertEquals(OffsetDateTime.parse("2021-12-05T04:32:03Z"), + model.value().get(0).properties().lastReadAt()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeInfoList model = new UeInfoList().withValue(Arrays.asList(new UeInfoInner() + .withProperties(new UeInfoPropertiesFormat().withRatType(RatType.FOURG).withUeState(UeState.CONNECTED) + .withUeIpAddresses( + Arrays.asList(new DnnIpPair().withDnn("aecxndtic").withUeIpAddress(new UeIpAddress()))) + .withLastReadAt(OffsetDateTime.parse("2021-12-05T04:32:03Z"))))); + model = BinaryData.fromObject(model).toObject(UeInfoList.class); + Assertions.assertEquals(RatType.FOURG, model.value().get(0).properties().ratType()); + Assertions.assertEquals(UeState.CONNECTED, model.value().get(0).properties().ueState()); + Assertions.assertEquals("aecxndtic", model.value().get(0).properties().ueIpAddresses().get(0).dnn()); + Assertions.assertEquals(OffsetDateTime.parse("2021-12-05T04:32:03Z"), + model.value().get(0).properties().lastReadAt()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoPropertiesFormatTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoPropertiesFormatTests.java new file mode 100644 index 0000000000000..b94cb149f0aef --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInfoPropertiesFormatTests.java @@ -0,0 +1,44 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.DnnIpPair; +import com.azure.resourcemanager.mobilenetwork.models.RatType; +import com.azure.resourcemanager.mobilenetwork.models.UeInfoPropertiesFormat; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeInfoPropertiesFormatTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeInfoPropertiesFormat model = BinaryData.fromString( + "{\"ratType\":\"5G\",\"ueState\":\"Deregistered\",\"ueIpAddresses\":[{\"dnn\":\"fonkphhqyikvyla\",\"ueIpAddress\":{\"ipV4Addr\":\"vluwmncsttij\"}}],\"lastReadAt\":\"2021-04-05T16:52:36Z\"}") + .toObject(UeInfoPropertiesFormat.class); + Assertions.assertEquals(RatType.FIVEG, model.ratType()); + Assertions.assertEquals(UeState.DEREGISTERED, model.ueState()); + Assertions.assertEquals("fonkphhqyikvyla", model.ueIpAddresses().get(0).dnn()); + Assertions.assertEquals("vluwmncsttij", model.ueIpAddresses().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-05T16:52:36Z"), model.lastReadAt()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeInfoPropertiesFormat model + = new UeInfoPropertiesFormat().withRatType(RatType.FIVEG).withUeState(UeState.DEREGISTERED) + .withUeIpAddresses(Arrays.asList(new DnnIpPair().withDnn("fonkphhqyikvyla") + .withUeIpAddress(new UeIpAddress().withIpV4Addr("vluwmncsttij")))) + .withLastReadAt(OffsetDateTime.parse("2021-04-05T16:52:36Z")); + model = BinaryData.fromObject(model).toObject(UeInfoPropertiesFormat.class); + Assertions.assertEquals(RatType.FIVEG, model.ratType()); + Assertions.assertEquals(UeState.DEREGISTERED, model.ueState()); + Assertions.assertEquals("fonkphhqyikvyla", model.ueIpAddresses().get(0).dnn()); + Assertions.assertEquals("vluwmncsttij", model.ueIpAddresses().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-05T16:52:36Z"), model.lastReadAt()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationsListMockTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationsListMockTests.java new file mode 100644 index 0000000000000..a41e7e47830ca --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeInformationsListMockTests.java @@ -0,0 +1,66 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.mobilenetwork.MobileNetworkManager; +import com.azure.resourcemanager.mobilenetwork.models.RatType; +import com.azure.resourcemanager.mobilenetwork.models.UeInfo; +import com.azure.resourcemanager.mobilenetwork.models.UeState; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class UeInformationsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"ratType\":\"4G\",\"ueState\":\"Deregistered\",\"ueIpAddresses\":[{\"dnn\":\"ofalickduoiqtam\",\"ueIpAddress\":{\"ipV4Addr\":\"sknxrwzawnvsbcf\"}},{\"dnn\":\"agxnvhycvdimw\",\"ueIpAddress\":{\"ipV4Addr\":\"e\"}},{\"dnn\":\"gy\",\"ueIpAddress\":{\"ipV4Addr\":\"trwpw\"}}],\"lastReadAt\":\"2021-07-22T04:45:11Z\"},\"id\":\"kzkdhmeott\",\"name\":\"w\",\"type\":\"yos\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + MobileNetworkManager manager = MobileNetworkManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.ueInformations().list("zfboj", "jmcsmyqwixvcpwn", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(RatType.FOURG, response.iterator().next().properties().ratType()); + Assertions.assertEquals(UeState.DEREGISTERED, response.iterator().next().properties().ueState()); + Assertions.assertEquals("ofalickduoiqtam", + response.iterator().next().properties().ueIpAddresses().get(0).dnn()); + Assertions.assertEquals("sknxrwzawnvsbcf", + response.iterator().next().properties().ueIpAddresses().get(0).ueIpAddress().ipV4Addr()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-22T04:45:11Z"), + response.iterator().next().properties().lastReadAt()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeIpAddressTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeIpAddressTests.java new file mode 100644 index 0000000000000..9fc874180278e --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeIpAddressTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import org.junit.jupiter.api.Assertions; + +public final class UeIpAddressTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeIpAddress model = BinaryData.fromString("{\"ipV4Addr\":\"j\"}").toObject(UeIpAddress.class); + Assertions.assertEquals("j", model.ipV4Addr()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeIpAddress model = new UeIpAddress().withIpV4Addr("j"); + model = BinaryData.fromObject(model).toObject(UeIpAddress.class); + Assertions.assertEquals("j", model.ipV4Addr()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeLocationInfoTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeLocationInfoTests.java new file mode 100644 index 0000000000000..bb0d6430b17c1 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeLocationInfoTests.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.PlmnId; +import com.azure.resourcemanager.mobilenetwork.models.UeLocationInfo; +import org.junit.jupiter.api.Assertions; + +public final class UeLocationInfoTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeLocationInfo model = BinaryData.fromString( + "{\"locationType\":\"xrtikvcpwpgclr\",\"tac\":\"ivtsoxfrkenxpm\",\"plmn\":{\"mcc\":\"yefrpmpdnqqska\",\"mnc\":\"ao\"}}") + .toObject(UeLocationInfo.class); + Assertions.assertEquals("xrtikvcpwpgclr", model.locationType()); + Assertions.assertEquals("ivtsoxfrkenxpm", model.tac()); + Assertions.assertEquals("yefrpmpdnqqska", model.plmn().mcc()); + Assertions.assertEquals("ao", model.plmn().mnc()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeLocationInfo model = new UeLocationInfo().withLocationType("xrtikvcpwpgclr").withTac("ivtsoxfrkenxpm") + .withPlmn(new PlmnId().withMcc("yefrpmpdnqqska").withMnc("ao")); + model = BinaryData.fromObject(model).toObject(UeLocationInfo.class); + Assertions.assertEquals("xrtikvcpwpgclr", model.locationType()); + Assertions.assertEquals("ivtsoxfrkenxpm", model.tac()); + Assertions.assertEquals("yefrpmpdnqqska", model.plmn().mcc()); + Assertions.assertEquals("ao", model.plmn().mnc()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeQosFlowTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeQosFlowTests.java new file mode 100644 index 0000000000000..74adab474b046 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeQosFlowTests.java @@ -0,0 +1,39 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.Ambr; +import com.azure.resourcemanager.mobilenetwork.models.UeQosFlow; +import org.junit.jupiter.api.Assertions; + +public final class UeQosFlowTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeQosFlow model = BinaryData.fromString( + "{\"qfi\":995606751,\"fiveqi\":901132598,\"mbr\":{\"uplink\":\"lageuaulxun\",\"downlink\":\"mjbnk\"},\"gbr\":{\"uplink\":\"xynenl\",\"downlink\":\"vxei\"}}") + .toObject(UeQosFlow.class); + Assertions.assertEquals(995606751, model.qfi()); + Assertions.assertEquals(901132598, model.fiveqi()); + Assertions.assertEquals("lageuaulxun", model.mbr().uplink()); + Assertions.assertEquals("mjbnk", model.mbr().downlink()); + Assertions.assertEquals("xynenl", model.gbr().uplink()); + Assertions.assertEquals("vxei", model.gbr().downlink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeQosFlow model = new UeQosFlow().withQfi(995606751).withFiveqi(901132598) + .withMbr(new Ambr().withUplink("lageuaulxun").withDownlink("mjbnk")) + .withGbr(new Ambr().withUplink("xynenl").withDownlink("vxei")); + model = BinaryData.fromObject(model).toObject(UeQosFlow.class); + Assertions.assertEquals(995606751, model.qfi()); + Assertions.assertEquals(901132598, model.fiveqi()); + Assertions.assertEquals("lageuaulxun", model.mbr().uplink()); + Assertions.assertEquals("mjbnk", model.mbr().downlink()); + Assertions.assertEquals("xynenl", model.gbr().uplink()); + Assertions.assertEquals("vxei", model.gbr().downlink()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo4GTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo4GTests.java new file mode 100644 index 0000000000000..c391264e8d106 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo4GTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.PdnType; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeSessionInfo4G; +import org.junit.jupiter.api.Assertions; + +public final class UeSessionInfo4GTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeSessionInfo4G model = BinaryData.fromString( + "{\"ebi\":499643337,\"apn\":\"jbjsvgjrwh\",\"ueIpAddress\":{\"ipV4Addr\":\"vyc\"},\"pdnType\":\"IPV4\"}") + .toObject(UeSessionInfo4G.class); + Assertions.assertEquals(499643337, model.ebi()); + Assertions.assertEquals("jbjsvgjrwh", model.apn()); + Assertions.assertEquals("vyc", model.ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.pdnType()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeSessionInfo4G model = new UeSessionInfo4G().withEbi(499643337).withApn("jbjsvgjrwh") + .withUeIpAddress(new UeIpAddress().withIpV4Addr("vyc")).withPdnType(PdnType.IPV4); + model = BinaryData.fromObject(model).toObject(UeSessionInfo4G.class); + Assertions.assertEquals(499643337, model.ebi()); + Assertions.assertEquals("jbjsvgjrwh", model.apn()); + Assertions.assertEquals("vyc", model.ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.pdnType()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo5GTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo5GTests.java new file mode 100644 index 0000000000000..a1561414ff6e0 --- /dev/null +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UeSessionInfo5GTests.java @@ -0,0 +1,71 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.mobilenetwork.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.mobilenetwork.models.Ambr; +import com.azure.resourcemanager.mobilenetwork.models.PdnType; +import com.azure.resourcemanager.mobilenetwork.models.Snssai; +import com.azure.resourcemanager.mobilenetwork.models.UeIpAddress; +import com.azure.resourcemanager.mobilenetwork.models.UeQosFlow; +import com.azure.resourcemanager.mobilenetwork.models.UeSessionInfo5G; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class UeSessionInfo5GTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + UeSessionInfo5G model = BinaryData.fromString( + "{\"pduSessionId\":791881876,\"dnn\":\"mmbnpqfrtql\",\"snssai\":{\"sst\":460730501,\"sd\":\"egnitg\"},\"ueIpAddress\":{\"ipV4Addr\":\"xlzyqd\"},\"pdnType\":\"IPV4\",\"ambr\":{\"uplink\":\"eg\",\"downlink\":\"ealzxwhcansymoyq\"},\"qosFlow\":[{\"qfi\":1329054502,\"fiveqi\":434694281,\"mbr\":{\"uplink\":\"divb\",\"downlink\":\"bxgomfaj\"},\"gbr\":{\"uplink\":\"asqvd\",\"downlink\":\"e\"}},{\"qfi\":13307941,\"fiveqi\":1169594553,\"mbr\":{\"uplink\":\"xakjsqzhzb\",\"downlink\":\"zkgimsid\"},\"gbr\":{\"uplink\":\"sicddyvv\",\"downlink\":\"skgfmocwahpq\"}},{\"qfi\":2049829405,\"fiveqi\":789499857,\"mbr\":{\"uplink\":\"eaahhvjhhn\",\"downlink\":\"kzyb\"},\"gbr\":{\"uplink\":\"jid\",\"downlink\":\"ksyxkyxvxevblbje\"}}]}") + .toObject(UeSessionInfo5G.class); + Assertions.assertEquals(791881876, model.pduSessionId()); + Assertions.assertEquals("mmbnpqfrtql", model.dnn()); + Assertions.assertEquals(460730501, model.snssai().sst()); + Assertions.assertEquals("egnitg", model.snssai().sd()); + Assertions.assertEquals("xlzyqd", model.ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.pdnType()); + Assertions.assertEquals("eg", model.ambr().uplink()); + Assertions.assertEquals("ealzxwhcansymoyq", model.ambr().downlink()); + Assertions.assertEquals(1329054502, model.qosFlow().get(0).qfi()); + Assertions.assertEquals(434694281, model.qosFlow().get(0).fiveqi()); + Assertions.assertEquals("divb", model.qosFlow().get(0).mbr().uplink()); + Assertions.assertEquals("bxgomfaj", model.qosFlow().get(0).mbr().downlink()); + Assertions.assertEquals("asqvd", model.qosFlow().get(0).gbr().uplink()); + Assertions.assertEquals("e", model.qosFlow().get(0).gbr().downlink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + UeSessionInfo5G model = new UeSessionInfo5G().withPduSessionId(791881876).withDnn("mmbnpqfrtql") + .withSnssai(new Snssai().withSst(460730501).withSd("egnitg")) + .withUeIpAddress(new UeIpAddress().withIpV4Addr("xlzyqd")).withPdnType(PdnType.IPV4) + .withAmbr(new Ambr().withUplink("eg").withDownlink("ealzxwhcansymoyq")) + .withQosFlow(Arrays.asList( + new UeQosFlow().withQfi(1329054502).withFiveqi(434694281) + .withMbr(new Ambr().withUplink("divb").withDownlink("bxgomfaj")) + .withGbr(new Ambr().withUplink("asqvd").withDownlink("e")), + new UeQosFlow().withQfi(13307941).withFiveqi(1169594553) + .withMbr(new Ambr().withUplink("xakjsqzhzb").withDownlink("zkgimsid")) + .withGbr(new Ambr().withUplink("sicddyvv").withDownlink("skgfmocwahpq")), + new UeQosFlow().withQfi(2049829405).withFiveqi(789499857) + .withMbr(new Ambr().withUplink("eaahhvjhhn").withDownlink("kzyb")) + .withGbr(new Ambr().withUplink("jid").withDownlink("ksyxkyxvxevblbje")))); + model = BinaryData.fromObject(model).toObject(UeSessionInfo5G.class); + Assertions.assertEquals(791881876, model.pduSessionId()); + Assertions.assertEquals("mmbnpqfrtql", model.dnn()); + Assertions.assertEquals(460730501, model.snssai().sst()); + Assertions.assertEquals("egnitg", model.snssai().sd()); + Assertions.assertEquals("xlzyqd", model.ueIpAddress().ipV4Addr()); + Assertions.assertEquals(PdnType.IPV4, model.pdnType()); + Assertions.assertEquals("eg", model.ambr().uplink()); + Assertions.assertEquals("ealzxwhcansymoyq", model.ambr().downlink()); + Assertions.assertEquals(1329054502, model.qosFlow().get(0).qfi()); + Assertions.assertEquals(434694281, model.qosFlow().get(0).fiveqi()); + Assertions.assertEquals("divb", model.qosFlow().get(0).mbr().uplink()); + Assertions.assertEquals("bxgomfaj", model.qosFlow().get(0).mbr().downlink()); + Assertions.assertEquals("asqvd", model.qosFlow().get(0).gbr().uplink()); + Assertions.assertEquals("e", model.qosFlow().get(0).gbr().downlink()); + } +} diff --git a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UserAssignedIdentityTests.java b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UserAssignedIdentityTests.java index 5e474d2ce3e5b..bca70d6de9f84 100644 --- a/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UserAssignedIdentityTests.java +++ b/sdk/mobilenetwork/azure-resourcemanager-mobilenetwork/src/test/java/com/azure/resourcemanager/mobilenetwork/generated/UserAssignedIdentityTests.java @@ -10,11 +10,9 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UserAssignedIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"7a0e5f96-c7c3-4a30-90fc-23a18c4433c7\",\"clientId\":\"31aa0b31-740f-4961-9ce3-6259efcf6fca\"}") - .toObject(UserAssignedIdentity.class); + UserAssignedIdentity model = BinaryData.fromString( + "{\"principalId\":\"b1517fdd-83c4-4151-84ba-cf3a4e3a9d0f\",\"clientId\":\"0d38b7b8-5f80-4ccf-b893-5e94d9da8b1a\"}") + .toObject(UserAssignedIdentity.class); } @org.junit.jupiter.api.Test diff --git a/sdk/modelsrepository/azure-iot-modelsrepository/pom.xml b/sdk/modelsrepository/azure-iot-modelsrepository/pom.xml index 4a859283206a3..885eefb52fd97 100644 --- a/sdk/modelsrepository/azure-iot-modelsrepository/pom.xml +++ b/sdk/modelsrepository/azure-iot-modelsrepository/pom.xml @@ -64,7 +64,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/modelsrepository/tests.yml b/sdk/modelsrepository/tests.yml index 186861f3dfa2b..6aa619a04e9e3 100644 --- a/sdk/modelsrepository/tests.yml +++ b/sdk/modelsrepository/tests.yml @@ -1,11 +1,11 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - Artifacts: - - name: azure-iot-modelsrepository - groupId: com.azure - safeName: azureiotmodelsrepository - ServiceDirectory: modelsrepository - Clouds: 'Preview' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + Artifacts: + - name: azure-iot-modelsrepository + groupId: com.azure + safeName: azureiotmodelsrepository + ServiceDirectory: modelsrepository + Clouds: 'Preview' diff --git a/sdk/monitor/azure-monitor-ingestion-perf/pom.xml b/sdk/monitor/azure-monitor-ingestion-perf/pom.xml index d5ec3b414e47a..48413bede7155 100644 --- a/sdk/monitor/azure-monitor-ingestion-perf/pom.xml +++ b/sdk/monitor/azure-monitor-ingestion-perf/pom.xml @@ -31,12 +31,12 @@ com.azure azure-monitor-ingestion - 1.1.5 + 1.2.0-beta.1 com.azure azure-identity - 1.11.3 + 1.11.4 diff --git a/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md b/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md index 1f42795d3a156..53d412567e5e5 100644 --- a/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md +++ b/sdk/monitor/azure-monitor-ingestion/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 1.2.0-beta.1 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 1.1.5 (2024-03-11) ### Other Changes diff --git a/sdk/monitor/azure-monitor-ingestion/pom.xml b/sdk/monitor/azure-monitor-ingestion/pom.xml index 171983fcbba84..86631662e0150 100644 --- a/sdk/monitor/azure-monitor-ingestion/pom.xml +++ b/sdk/monitor/azure-monitor-ingestion/pom.xml @@ -7,7 +7,7 @@ com.azure azure-monitor-ingestion - 1.1.5 + 1.2.0-beta.1 jar Microsoft Azure SDK for Azure Monitor Data Ingestion @@ -100,13 +100,13 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/monitor/azure-monitor-opentelemetry-exporter/CHANGELOG.md b/sdk/monitor/azure-monitor-opentelemetry-exporter/CHANGELOG.md index 050071f84af81..c5f9897de936d 100644 --- a/sdk/monitor/azure-monitor-opentelemetry-exporter/CHANGELOG.md +++ b/sdk/monitor/azure-monitor-opentelemetry-exporter/CHANGELOG.md @@ -1,5 +1,15 @@ # Release History +## 1.0.0-beta.22 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + ## 1.0.0-beta.21 (2024-03-11) ### Bugs Fixed diff --git a/sdk/monitor/azure-monitor-opentelemetry-exporter/pom.xml b/sdk/monitor/azure-monitor-opentelemetry-exporter/pom.xml index b4ebfa319d4b8..3b1cd9b5c1d86 100644 --- a/sdk/monitor/azure-monitor-opentelemetry-exporter/pom.xml +++ b/sdk/monitor/azure-monitor-opentelemetry-exporter/pom.xml @@ -14,7 +14,7 @@ com.azure azure-monitor-opentelemetry-exporter - 1.0.0-beta.21 + 1.0.0-beta.22 Microsoft Azure SDK for OpenTelemetry Azure Monitor Exporter This package contains Microsoft Azure SDK for OpenTelemetry Azure Monitor Exporter. @@ -54,32 +54,32 @@ io.opentelemetry opentelemetry-api - 1.35.0 + 1.36.0 io.opentelemetry opentelemetry-sdk - 1.35.0 + 1.36.0 io.opentelemetry opentelemetry-sdk-metrics - 1.35.0 + 1.36.0 io.opentelemetry opentelemetry-sdk-logs - 1.35.0 + 1.36.0 io.opentelemetry opentelemetry-sdk-extension-autoconfigure-spi - 1.35.0 + 1.36.0 io.opentelemetry opentelemetry-sdk-extension-autoconfigure - 1.35.0 + 1.36.0 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test io.opentelemetry opentelemetry-sdk-testing - 1.35.0 + 1.36.0 test com.azure azure-data-appconfiguration - 1.5.2 + 1.5.3 test com.azure azure-messaging-eventhubs - 5.18.1 + 5.18.2 test @@ -176,13 +176,13 @@ com.azure azure-messaging-eventhubs-checkpointstore-blob - 1.19.1 + 1.19.2 test com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -200,7 +200,7 @@ com.squareup.okio okio - 3.6.0 + 3.9.0 test @@ -215,13 +215,13 @@ - io.opentelemetry:opentelemetry-api:[1.35.0] - io.opentelemetry:opentelemetry-sdk:[1.35.0] - io.opentelemetry:opentelemetry-sdk-metrics:[1.35.0] - io.opentelemetry:opentelemetry-sdk-logs:[1.35.0] - io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi:[1.35.0] - io.opentelemetry:opentelemetry-sdk-extension-autoconfigure:[1.35.0] - com.squareup.okio:okio:[3.6.0] + io.opentelemetry:opentelemetry-api:[1.36.0] + io.opentelemetry:opentelemetry-sdk:[1.36.0] + io.opentelemetry:opentelemetry-sdk-metrics:[1.36.0] + io.opentelemetry:opentelemetry-sdk-logs:[1.36.0] + io.opentelemetry:opentelemetry-sdk-extension-autoconfigure-spi:[1.36.0] + io.opentelemetry:opentelemetry-sdk-extension-autoconfigure:[1.36.0] + com.squareup.okio:okio:[3.9.0] diff --git a/sdk/monitor/azure-monitor-opentelemetry-exporter/tests.yml b/sdk/monitor/azure-monitor-opentelemetry-exporter/tests.yml index f155a9dedffd3..f70dcec7c140f 100644 --- a/sdk/monitor/azure-monitor-opentelemetry-exporter/tests.yml +++ b/sdk/monitor/azure-monitor-opentelemetry-exporter/tests.yml @@ -1,14 +1,14 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: monitor - Artifacts: - - name: azure-monitor-opentelemetry-exporter - groupId: com.azure - safeName: azuremonitoropentelemetryexporter - EnvVars: - AZURE_CLIENT_ID: $(MONITOR_CLIENT_ID) - AZURE_TENANT_ID: $(MONITOR_TENANT_ID) - AZURE_CLIENT_SECRET: $(MONITOR_CLIENT_SECRET) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: monitor + Artifacts: + - name: azure-monitor-opentelemetry-exporter + groupId: com.azure + safeName: azuremonitoropentelemetryexporter + EnvVars: + AZURE_CLIENT_ID: $(MONITOR_CLIENT_ID) + AZURE_TENANT_ID: $(MONITOR_TENANT_ID) + AZURE_CLIENT_SECRET: $(MONITOR_CLIENT_SECRET) diff --git a/sdk/monitor/azure-monitor-query-perf/pom.xml b/sdk/monitor/azure-monitor-query-perf/pom.xml index a6c71e842a92c..796f03fd124c7 100644 --- a/sdk/monitor/azure-monitor-query-perf/pom.xml +++ b/sdk/monitor/azure-monitor-query-perf/pom.xml @@ -36,7 +36,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 diff --git a/sdk/monitor/azure-monitor-query/CHANGELOG.md b/sdk/monitor/azure-monitor-query/CHANGELOG.md index 4d6dcaaeb3190..4502c7202cbfe 100644 --- a/sdk/monitor/azure-monitor-query/CHANGELOG.md +++ b/sdk/monitor/azure-monitor-query/CHANGELOG.md @@ -13,6 +13,17 @@ ### Other Changes + +## 1.2.10 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 1.2.9 (2024-02-20) ### Other Changes diff --git a/sdk/monitor/azure-monitor-query/README.md b/sdk/monitor/azure-monitor-query/README.md index 073d888869543..e506e2f9c2cc0 100644 --- a/sdk/monitor/azure-monitor-query/README.md +++ b/sdk/monitor/azure-monitor-query/README.md @@ -108,8 +108,8 @@ MetricsQueryClient metricsQueryClient = new MetricsQueryClientBuilder() .buildClient(); ``` -```java readme-sample-createMetricsBatchQueryClient -MetricsClient metricsBatchQueryClient = new MetricsClientBuilder() +```java readme-sample-createMetricsClient +MetricsClient metricsClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("{endpoint}") .buildClient(); @@ -129,7 +129,7 @@ MetricsQueryAsyncClient metricsQueryAsyncClient = new MetricsQueryClientBuilder( .buildAsyncClient(); ``` -```java readme-sample-createMetricsBatchQueryAsyncClient +```java readme-sample-createMetricsAsyncClient MetricsAsyncClient metricsAsyncClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("{endpoint}") @@ -196,8 +196,8 @@ Each set of metric values is a time series with the following characteristics: - [Handle metrics query response](#handle-metrics-query-response) - [Get average and count metrics](#get-average-and-count-metrics) - [Create a metrics client for non-public Azure clouds](#configure-clients-for-non-public-azure-clouds) -- [Metrics batch query](#metrics-batch-query) - - [Handle metrics batch query response](#handle-metrics-batch-query-response) +- [Metrics query resources](#metrics-query-resources) + - [Handle metrics query resources response](#handle-metrics-query-resources-response) ### Logs query ```java readme-sample-logsquery @@ -529,22 +529,22 @@ for (MetricResult metric : metricsQueryResult.getMetrics()) { } ``` -### Metrics batch query +### Metrics query resources -#### Handle metrics batch query response +#### Handle metrics query resources response -```java readme-sample-metricsquerybatch -MetricsClient metricsBatchQueryClient = new MetricsClientBuilder() +```java readme-sample-metricsquerymultipleresources +MetricsClient metricsClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("{endpoint}") .buildClient(); -MetricsBatchQueryResult metricsBatchQueryResult = metricsBatchQueryClient.queryBatch( +MetricsQueryResourcesResult metricsQueryResourcesResult = metricsClient.queryResources( Arrays.asList("{resourceId1}", "{resourceId2}"), Arrays.asList("{metric1}", "{metric2}"), "{metricNamespace}"); -for (MetricsQueryResult metricsQueryResult : metricsBatchQueryResult.getMetricsQueryResults()) { +for (MetricsQueryResult metricsQueryResult : metricsQueryResourcesResult.getMetricsQueryResults()) { // Each MetricsQueryResult corresponds to one of the resourceIds in the batch request. List metrics = metricsQueryResult.getMetrics(); metrics.forEach(metric -> { diff --git a/sdk/monitor/azure-monitor-query/pom.xml b/sdk/monitor/azure-monitor-query/pom.xml index 78ae391c45530..f053812ffff10 100644 --- a/sdk/monitor/azure-monitor-query/pom.xml +++ b/sdk/monitor/azure-monitor-query/pom.xml @@ -70,7 +70,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -88,7 +88,7 @@ com.azure azure-data-appconfiguration - 1.5.2 + 1.5.3 test diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryAsyncClient.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryAsyncClient.java index 360cf0fa21c4a..1a56141c47203 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryAsyncClient.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryAsyncClient.java @@ -37,8 +37,16 @@ import static com.azure.monitor.query.implementation.logs.models.LogsQueryHelper.mapLogsQueryError; /** - * The asynchronous client for querying Azure Monitor logs. - *

Instantiating an asynchronous Logs query Client

+ *

Provides an asynchronous service client for querying logs in the Azure Monitor Service.

+ * + *

The LogsQueryClient is an asynchronous client that provides methods to execute Kusto queries against + * Azure Monitor logs. It provides methods to query logs in a specific workspace, execute a batch of queries, and + * query logs for a specific Azure resource.

+ * + *

Getting Started

+ * + *

Authenticating and building instances of this client are handled by {@link LogsQueryClientBuilder}. + * This sample shows how to authenticate and build a LogQueryAsyncClient instance using LogQueryClientBuilder.

* * *
@@ -47,6 +55,38 @@
  *         .buildAsyncClient();
  * 
* + * + *

For more information on building and authenticating, see the {@link LogsQueryClientBuilder} documentation.

+ * + *

Client Usage

+ * + *

+ * For more information on how to use this client, see the following method documentation: + *

+ * + *
    + *
  • + * {@link LogsQueryAsyncClient#queryWorkspace(String, String, QueryTimeInterval) queryWorkspace(String, String, QueryTimeInterval)} - Query logs from a workspace. + *
  • + *
  • + * {@link LogsQueryAsyncClient#queryWorkspaceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions) queryWorkspaceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions)} - Query logs from a workspace using query options with service response returned. + *
  • + *
  • + * {@link LogsQueryAsyncClient#queryBatch(LogsBatchQuery) queryBatch(LogsBatchQuery)} - Execute a batch of logs queries. + *
  • + *
  • + * {@link LogsQueryAsyncClient#queryBatchWithResponse(LogsBatchQuery) queryBatchWithResponse(LogsBatchQuery)} - Execute a batch of logs queries with service response returned. + *
  • + *
  • + * {@link LogsQueryAsyncClient#queryResource(String, String, QueryTimeInterval) queryResource(String, String, QueryTimeInterval)} - Query logs for an Azure resource. + *
  • + *
  • + * {@link LogsQueryAsyncClient#queryResourceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions) queryResourceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions)} - Query logs for an Azure resource using query options with service response returned. + *
  • + *
+ * + * @see com.azure.monitor.query + * @see LogsQueryClientBuilder */ @ServiceClient(builder = LogsQueryClientBuilder.class, isAsync = true) public final class LogsQueryAsyncClient { diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClient.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClient.java index 08b73749048f9..06595ff25426d 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClient.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClient.java @@ -35,9 +35,16 @@ import static com.azure.monitor.query.implementation.logs.models.LogsQueryHelper.updateContext; /** - * The synchronous client for querying Azure Monitor logs. + *

Provides a synchronous service client for querying logs in the Azure Monitor Service.

* - *

Instantiating a synchronous Logs query Client

+ *

The LogsQueryClient is a synchronous client that provides methods to execute Kusto queries against + * Azure Monitor logs. It provides methods to query logs in a specific workspace, execute a batch of queries, and + * query logs for a specific Azure resource.

+ * + *

Getting Started

+ * + *

Authenticating and building instances of this client are handled by {@link LogsQueryClientBuilder}. + * his sample shows how to authenticate and build a LogsQueryClient instance using LogQueryClientBuilder.

* * *
@@ -46,6 +53,38 @@
  *         .buildClient();
  * 
* + * + *

For more information on building and authenticating, see the {@link LogsQueryClientBuilder} documentation.

+ * + *

Client Usage

+ * + *

+ * For more information on how to use this client, see the following method documentation: + *

+ * + *
    + *
  • + * {@link LogsQueryClient#queryWorkspace(String, String, QueryTimeInterval) queryWorkspace(String, String, QueryTimeInterval)} - Query logs from a workspace. + *
  • + *
  • + * {@link LogsQueryClient#queryWorkspaceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions, Context) queryWorkspaceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions, Context)} - Query logs from a workspace using query options and context with service response returned. + *
  • + *
  • + * {@link LogsQueryClient#queryBatch(LogsBatchQuery) queryBatch(LogsBatchQuery)} - Execute a batch of logs queries. + *
  • + *
  • + * {@link LogsQueryClient#queryBatchWithResponse(LogsBatchQuery, Context) queryBatchWithResponse(LogsBatchQuery, Context)} - Execute a batch of logs queries with context and service response returned. + *
  • + *
  • + * {@link LogsQueryClient#queryResource(String, String, QueryTimeInterval) queryResource(String, String, QueryTimeInterval)} - Query logs for an Azure resource. + *
  • + *
  • + * {@link LogsQueryClient#queryResourceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions, Context) queryResourceWithResponse(String, String, QueryTimeInterval, LogsQueryOptions, Context)} - Query logs for an Azure resource with query options and context with service response returned. + *
  • + *
+ * + * @see com.azure.monitor.query + * @see LogsQueryClientBuilder */ @ServiceClient(builder = LogsQueryClientBuilder.class) public final class LogsQueryClient { diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClientBuilder.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClientBuilder.java index 5859275c13179..2bc4e72086afa 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClientBuilder.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/LogsQueryClientBuilder.java @@ -21,9 +21,23 @@ import com.azure.monitor.query.implementation.logs.AzureLogAnalyticsImplBuilder; /** - * Fluent builder for creating instances of {@link LogsQueryClient} and {@link LogsQueryAsyncClient}. + *

Fluent builder for creating instances of {@link LogsQueryClient} and {@link LogsQueryAsyncClient}.

* - *

Instantiating an asynchronous Logs query Client

+ *

The LogsQueryClientBuilder is responsible for authenticating a building instances of {@link LogsQueryClient} and + * {@link LogsQueryAsyncClient}. Customizations can be applied to clients through the builder using the various options + * available.

+ * + *

Getting Started

+ * + *

+ * To create instances of the clients, sufficient authentication credentials are required. {@link TokenCredential} is + * a common form of authentication. The resource / workspace is not required for client creation, but the authentication + * credentials must have access to the resources / workspaces utilized by the client. + *

+ * + *

Client Builder Usage

+ * + *

The following sample shows instantiating an asynchronous Logs query Client using Token Credential

* * *
@@ -33,7 +47,7 @@
  * 
* * - *

Instantiating a synchronous Logs query Client

+ *

The following sample shows instantiating a synchronous Logs query Client using Token Credential

* * *
@@ -42,6 +56,15 @@
  *         .buildClient();
  * 
* + * + *

+ * For more information about the other types of credentials that can be used to authenticate your client, please see + * this documentation: Azure Identity + *

+ * + * @see com.azure.monitor.query + * @see LogsQueryClient + * @see LogsQueryAsyncClient */ @ServiceClientBuilder(serviceClients = {LogsQueryClient.class, LogsQueryAsyncClient.class}) public final class LogsQueryClientBuilder implements EndpointTrait, @@ -50,6 +73,11 @@ public final class LogsQueryClientBuilder implements EndpointTrait queryBatch(List resourceUris, List metricsNames, String metricsNamespace) { - return this.queryBatchWithResponse(resourceUris, metricsNames, metricsNamespace, new MetricsBatchQueryOptions()) + public Mono queryResources(List resourceIds, List metricsNames, String metricsNamespace) { + return this.queryResourcesWithResponse(resourceIds, metricsNames, metricsNamespace, new MetricsQueryResourcesOptions()) .map(Response::getValue); } /** * Returns all the Azure Monitor metrics requested for the batch of resources. * - * @param resourceUris The resource URIs for which the metrics is requested. + * @param resourceIds The resource ids for which the metrics is requested. * @param metricsNames The names of the metrics to query. * @param metricsNamespace The namespace of the metrics to query. - * @param options The {@link MetricsBatchQueryOptions} to include for the request. + * @param options The {@link MetricsQueryResourcesOptions} to include for the request. * @return A time-series metrics result for the requested metric names. - * @throws IllegalArgumentException thrown if {@code resourceUris}, {@code metricsNames} or {@code metricsNamespace} are empty. - * @throws NullPointerException thrown if {@code resourceUris}, {@code metricsNames} or {@code metricsNamespace} are null. + * @throws IllegalArgumentException thrown if {@code resourceIds}, {@code metricsNames} or {@code metricsNamespace} are empty. + * @throws NullPointerException thrown if {@code resourceIds}, {@code metricsNames} or {@code metricsNamespace} are null. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Mono> queryBatchWithResponse(List resourceUris, List metricsNames, - String metricsNamespace, MetricsBatchQueryOptions options) { + public Mono> queryResourcesWithResponse(List resourceIds, List metricsNames, + String metricsNamespace, MetricsQueryResourcesOptions options) { - if (CoreUtils.isNullOrEmpty(Objects.requireNonNull(resourceUris, "'resourceUris cannot be null."))) { - return monoError(LOGGER, new IllegalArgumentException("resourceUris cannot be empty")); + if (CoreUtils.isNullOrEmpty(Objects.requireNonNull(resourceIds, "'resourceIds cannot be null."))) { + return monoError(LOGGER, new IllegalArgumentException("resourceIds cannot be empty")); } if (CoreUtils.isNullOrEmpty(Objects.requireNonNull(metricsNames, "metricsNames cannot be null"))) { @@ -119,9 +119,9 @@ public Mono> queryBatchWithResponse(List> responseMono = this.serviceClient.getMetricsBatches() .batchWithResponseAsync(subscriptionId, metricsNamespace, metricsNames, resourceIdList, startTime, endTime, granularity, aggregations, top, orderBy, filter, rollupBy); @@ -133,9 +133,9 @@ public Mono> queryBatchWithResponse(List metricsQueryResults = values.stream() .map(result -> mapToMetricsQueryResult(result)) .collect(Collectors.toList()); - MetricsBatchQueryResult metricsBatchQueryResult = new MetricsBatchQueryResult(metricsQueryResults); + MetricsQueryResourcesResult metricsQueryResourcesResult = new MetricsQueryResourcesResult(metricsQueryResults); return new SimpleResponse<>(response.getRequest(), response.getStatusCode(), - response.getHeaders(), metricsBatchQueryResult); + response.getHeaders(), metricsQueryResourcesResult); }); } } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClient.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClient.java index eb46e7f34ee52..02863c20bb6d7 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClient.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClient.java @@ -16,8 +16,8 @@ import com.azure.monitor.query.implementation.metricsbatch.models.MetricResultsResponseValuesItem; import com.azure.monitor.query.implementation.metricsbatch.models.ResourceIdList; import com.azure.monitor.query.models.AggregationType; -import com.azure.monitor.query.models.MetricsBatchQueryResult; -import com.azure.monitor.query.models.MetricsBatchQueryOptions; +import com.azure.monitor.query.models.MetricsQueryResourcesOptions; +import com.azure.monitor.query.models.MetricsQueryResourcesResult; import com.azure.monitor.query.models.MetricsQueryResult; import java.time.Duration; @@ -45,34 +45,34 @@ public final class MetricsClient { /** * Returns all the Azure Monitor metrics requested for the batch of resources. * - * @param resourceUris The resource URIs for which the metrics is requested. + * @param resourceIds The resource ids for which the metrics is requested. * @param metricsNames The names of the metrics to query. * @param metricsNamespace The namespace of the metrics to query. * @return A time-series metrics result for the requested metric names. */ @ServiceMethod(returns = ReturnType.SINGLE) - public MetricsBatchQueryResult queryBatch(List resourceUris, List metricsNames, String metricsNamespace) { - return this.queryBatchWithResponse(resourceUris, metricsNames, metricsNamespace, new MetricsBatchQueryOptions(), + public MetricsQueryResourcesResult queryResources(List resourceIds, List metricsNames, String metricsNamespace) { + return this.queryResourcesWithResponse(resourceIds, metricsNames, metricsNamespace, new MetricsQueryResourcesOptions(), Context.NONE).getValue(); } /** * Returns all the Azure Monitor metrics requested for the batch of resources. * - * @param resourceUris The resource URIs for which the metrics is requested. + * @param resourceIds The resource ids for which the metrics is requested. * @param metricsNames The names of the metrics to query. * @param metricsNamespace The namespace of the metrics to query. - * @param options The {@link MetricsBatchQueryOptions} to include for the request. + * @param options The {@link MetricsQueryResourcesOptions} to include for the request. * @param context The context to associate with this operation. * @return A time-series metrics result for the requested metric names. - * @throws IllegalArgumentException thrown if {@code resourceUris}, {@code metricsNames} or {@code metricsNamespace} are empty. - * @throws NullPointerException thrown if {@code resourceUris}, {@code metricsNames} or {@code metricsNamespace} are null. + * @throws IllegalArgumentException thrown if {@code resourceIds}, {@code metricsNames} or {@code metricsNamespace} are empty. + * @throws NullPointerException thrown if {@code resourceIds}, {@code metricsNames} or {@code metricsNamespace} are null. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response queryBatchWithResponse(List resourceUris, List metricsNames, - String metricsNamespace, MetricsBatchQueryOptions options, Context context) { - if (CoreUtils.isNullOrEmpty(Objects.requireNonNull(resourceUris, "'resourceUris cannot be null."))) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException("resourceUris cannot be empty")); + public Response queryResourcesWithResponse(List resourceIds, List metricsNames, + String metricsNamespace, MetricsQueryResourcesOptions options, Context context) { + if (CoreUtils.isNullOrEmpty(Objects.requireNonNull(resourceIds, "'resourceIds cannot be null."))) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException("resourceIds cannot be empty")); } if (CoreUtils.isNullOrEmpty(Objects.requireNonNull(metricsNames, "metricsNames cannot be null"))) { @@ -118,9 +118,9 @@ public Response queryBatchWithResponse(List res top = options.getTop(); orderBy = options.getOrderBy(); } - String subscriptionId = getSubscriptionFromResourceId(resourceUris.get(0)); + String subscriptionId = getSubscriptionFromResourceId(resourceIds.get(0)); ResourceIdList resourceIdList = new ResourceIdList(); - resourceIdList.setResourceids(resourceUris); + resourceIdList.setResourceids(resourceIds); Response response = this.serviceClient.getMetricsBatches() .batchWithResponse(subscriptionId, metricsNamespace, metricsNames, resourceIdList, startTime, endTime, granularity, aggregations, top, orderBy, filter, rollupBy, context); @@ -129,10 +129,10 @@ public Response queryBatchWithResponse(List res List metricsQueryResults = values.stream() .map(result -> mapToMetricsQueryResult(result)) .collect(Collectors.toList()); - MetricsBatchQueryResult metricsBatchQueryResult = new MetricsBatchQueryResult(metricsQueryResults); + MetricsQueryResourcesResult metricsQueryResourcesResult = new MetricsQueryResourcesResult(metricsQueryResults); return new SimpleResponse<>(response.getRequest(), response.getStatusCode(), - response.getHeaders(), metricsBatchQueryResult); + response.getHeaders(), metricsQueryResourcesResult); } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClientBuilder.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClientBuilder.java index b83776202a1fa..666c7bfe4846d 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClientBuilder.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsClientBuilder.java @@ -29,7 +29,12 @@ public final class MetricsClientBuilder implements EndpointTraitInstantiating an asynchronous Metrics query Client

+ *

The asynchronous client for querying Azure Monitor metrics.

+ * + *

Azure Monitor Metrics is a feature of Azure Monitor that collects numeric data from monitored resources into a + * time-series database. Metrics are numerical values that are collected at regular intervals and describe some aspect + * of a system at a particular time. The MetricsQueryClient provides synchronous implementations of methods that query + * metrics from your Azure services.

+ * + *

Getting Started

+ * + *

+ * Authenticating and building MetricsQueryAsyncClient instances are done through {@link MetricsQueryClientBuilder}. + * The following sample shows how to build a new MetricsQueryClient instance. + *

* * *
@@ -43,6 +54,27 @@
  *         .buildAsyncClient();
  * 
* + * + *

+ * For more information on building and authenticating, see the {@link MetricsQueryClientBuilder} documentation. + *

+ * + *

Client Usage

+ * + *

+ * For more information on using the MetricsQueryAsyncClient, see the following method documentation: + *

+ * + *
    + *
  • + * {@link MetricsQueryAsyncClient#queryResource(String, List)} - Query metrics for an Azure resource. + * {@link MetricsQueryAsyncClient#listMetricNamespaces(String, OffsetDateTime)} - Lists all the metrics namespaces created for the resource URI. + * {@link MetricsQueryAsyncClient#listMetricDefinitions(String)} - Lists all the metrics definitions created for the resource URI. + *
  • + *
+ * + * @see com.azure.monitor.query + * @see MetricsQueryClientBuilder */ @ServiceClient(builder = MetricsQueryClientBuilder.class, isAsync = true) public final class MetricsQueryAsyncClient { diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClient.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClient.java index 10b50a41da481..340bc6ce986ea 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClient.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClient.java @@ -33,9 +33,20 @@ import static com.azure.monitor.query.implementation.metrics.models.MetricsHelper.convertToMetricsQueryResult; /** - * The synchronous client for querying Azure Monitor metrics. + *

The synchronous client for querying Azure Monitor metrics.

+ * + *

Azure Monitor Metrics is a feature of Azure Monitor that collects numeric data from monitored resources into a + * time-series database. Metrics are numerical values that are collected at regular intervals and describe some aspect + * of a system at a particular time. The MetricsQueryClient provides synchronous implementations of methods that query + * metrics from your Azure services.

+ * + *

Getting Started

+ * + *

+ * Authenticating and building MetricsQueryClient instances are done through {@link MetricsQueryClientBuilder}. + * The following sample shows how to build a new MetricsQueryClient instance. + *

* - *

Instantiating a synchronous Metrics query Client

* *
  * MetricsQueryClient metricsQueryClient = new MetricsQueryClientBuilder()
@@ -43,6 +54,27 @@
  *         .buildClient();
  * 
* + * + *

+ * For more information on building and authenticating, see the {@link MetricsQueryClientBuilder} documentation. + *

+ * + *

Client Usage

+ * + *

+ * For more information on using the MetricsQueryClient, see the following method documentation: + *

+ * + *
    + *
  • + * {@link MetricsQueryClient#queryResource(String, List)} - Query metrics for an Azure resource. + * {@link MetricsQueryClient#listMetricNamespaces(String, OffsetDateTime)} - Lists all the metrics namespaces created for the resource URI. + * {@link MetricsQueryClient#listMetricDefinitions(String)} - Lists all the metrics definitions created for the resource URI. + *
  • + *
+ * + * @see com.azure.monitor.query + * @see MetricsQueryClientBuilder */ @ServiceClient(builder = MetricsQueryClientBuilder.class) public final class MetricsQueryClient { diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClientBuilder.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClientBuilder.java index c9227ecfae538..be8a683be297d 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClientBuilder.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/MetricsQueryClientBuilder.java @@ -24,9 +24,27 @@ import com.azure.monitor.query.implementation.metricsnamespaces.MetricsNamespacesClientImplBuilder; /** - * Fluent builder for creating instances of {@link MetricsQueryClient} and {@link MetricsQueryAsyncClient}. + *

Fluent builder for creating instances of {@link MetricsQueryClient} and {@link MetricsQueryAsyncClient}.

* - *

Instantiating an asynchronous Metrics query Client

+ *

+ * The MetricsQueryClientBuilder is responsible for authenticating a building instances of {@link MetricsQueryClient} and + * {@link MetricsQueryAsyncClient}. Customizations can be applied to clients through the builder using the various options + * available. + *

+ * + *

Getting Started

+ * + *

+ * To create instances of the clients, sufficient authentication credentials are required. {@link TokenCredential} is + * a common form of authentication. The resource / workspace is not required for client creation, but the authentication + * credentials must have access to the resources / workspaces utilized by the client. + *

+ * + *

Client Builder Usage

+ * + *

+ * The following sample shows instantiating an asynchronous Metrics query Client using Token Credential + *

* * *
@@ -36,7 +54,9 @@
  * 
* * - *

Instantiating a synchronous Metrics query Client

+ *

+ * The following sample shows instantiating a synchronous Metrics query Client using Token Credential + *

* * *
@@ -45,6 +65,15 @@
  *         .buildClient();
  * 
* + * + *

+ * For more information about the other types of credentials that can be used to authenticate your client, please see + * this documentation: Azure Identity + *

+ * + * @see com.azure.monitor.query + * @see MetricsQueryClient + * @see MetricsQueryAsyncClient */ @ServiceClientBuilder(serviceClients = {MetricsQueryClient.class, MetricsQueryAsyncClient.class}) public final class MetricsQueryClientBuilder implements EndpointTrait, @@ -60,6 +89,11 @@ public final class MetricsQueryClientBuilder implements EndpointTrait mapToMetrics(metric)) .collect(Collectors.toList()); - MetricsQueryResult metricsQueryResult = new MetricsQueryResult(/* TODO (srnagar): fix this item.getCost() */ null, + MetricsQueryResult metricsQueryResult = new MetricsQueryResult(null, QueryTimeInterval.parse(item.getStarttime() + "/" + item.getEndtime()), Duration.parse(item.getInterval()), item.getNamespace(), item.getResourceregion(), metrics); return metricsQueryResult; } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsBatchQuery.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsBatchQuery.java index 9c4f1cce91b4e..94d5830a4f164 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsBatchQuery.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsBatchQuery.java @@ -41,6 +41,11 @@ public Duration getMaxServerTimeout(LogsBatchQuery query) { }); } + /** + * Creates an instance of LogsBatchQuery. + */ + public LogsBatchQuery() { } + /** * Adds a new logs query to the batch. * @param workspaceId The workspaceId on which the query is executed. diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsColumnType.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsColumnType.java index f4c1609aac602..8b3a3f84b5f78 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsColumnType.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsColumnType.java @@ -39,6 +39,11 @@ public final class LogsColumnType extends ExpandableStringEnum { /** Static value timespan for LogsColumnType. */ public static final LogsColumnType TIMESPAN = fromString("timespan"); + /** + * Creates an instance of LogsColumnType. + */ + public LogsColumnType() { } + /** * Creates or finds a LogsColumnType from its string representation. * @@ -50,7 +55,11 @@ public static LogsColumnType fromString(String name) { return fromString(name, LogsColumnType.class); } - /** @return known LogsColumnType values. */ + /** + * Returns known LogsColumnType values. + * + * @return The {@link Collection} of known LogsColumnType values. + */ public static Collection values() { return values(LogsColumnType.class); } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsQueryOptions.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsQueryOptions.java index fe1487c452d19..a560793d08247 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsQueryOptions.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/LogsQueryOptions.java @@ -20,6 +20,11 @@ public final class LogsQueryOptions { private Duration serverTimeout; private List additionalWorkspaces; + /** + * Creates an instance of LogsQueryOptions. + */ + public LogsQueryOptions() { } + /** * Returns the server timeout for this query. * @return The server timeout duration. diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricAvailability.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricAvailability.java index 8d875fd773786..75037bb2d7a97 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricAvailability.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricAvailability.java @@ -19,6 +19,11 @@ public final class MetricAvailability { MetricsHelper.setMetricAvailabilityAccessor(MetricAvailability::setMetricAvailabilityProperties); } + /** + * Creates an instance of MetricAvailability. + */ + public MetricAvailability() { } + private void setMetricAvailabilityProperties(Duration retention, Duration granularity) { this.retention = retention; this.granularity = granularity; diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricClass.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricClass.java index 15010f5b9a7b7..7916fe8762d7e 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricClass.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricClass.java @@ -24,6 +24,11 @@ public final class MetricClass extends ExpandableStringEnum { /** Static value Saturation for MetricClass. */ public static final MetricClass SATURATION = fromString("Saturation"); + /** + * Creates an instance of MetricClass. + */ + public MetricClass() { } + /** * Creates or finds a MetricClass from its string representation. * @@ -35,7 +40,11 @@ public static MetricClass fromString(String name) { return fromString(name, MetricClass.class); } - /** @return known MetricClass values. */ + /** + * Returns the known MetricClass values. + * + * @return The {@link Collection} of known MetricClass values. + */ public static Collection values() { return values(MetricClass.class); } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricDefinition.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricDefinition.java index afa2531e88d5c..6256a24c8631d 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricDefinition.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricDefinition.java @@ -30,6 +30,11 @@ public final class MetricDefinition { MetricsHelper.setMetricDefinitionAccessor(MetricDefinition::setMetricDefinitionProperties); } + /** + * Creates an instance of MetricDefinition. + */ + public MetricDefinition() { } + private void setMetricDefinitionProperties(Boolean dimensionRequired, String resourceId, String namespace, diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricNamespace.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricNamespace.java index 50ce627056404..93bb1ee5c10ec 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricNamespace.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricNamespace.java @@ -19,6 +19,11 @@ public final class MetricNamespace { MetricsHelper.setMetricNamespaceAccessor(MetricNamespace::setMetricNamespaceProperties); } + /** + * Creates an instance of MetricNamespace. + */ + public MetricNamespace() { } + private void setMetricNamespaceProperties(NamespaceClassification classification, String id, String name, String fullyQualifiedName, String type) { this.classification = classification; diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricUnit.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricUnit.java index 699088b6dc8ac..34d07f49fb611 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricUnit.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricUnit.java @@ -48,6 +48,11 @@ public final class MetricUnit extends ExpandableStringEnum { /** Static value BitsPerSecond for MetricUnit. */ public static final MetricUnit BITS_PER_SECOND = fromString("BitsPerSecond"); + /** + * Creates an instance of MetricUnit. + */ + public MetricUnit() { } + /** * Creates or finds a MetricUnit from its string representation. * @@ -59,7 +64,11 @@ public static MetricUnit fromString(String name) { return fromString(name, MetricUnit.class); } - /** @return known MetricUnit values. */ + /** + * Returns the known MetricUnit values. + * + * @return The {@link Collection} of known MetricUnit values. + */ public static Collection values() { return values(MetricUnit.class); } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryOptions.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryOptions.java index 57d6eed361949..75f7af9c279e3 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryOptions.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryOptions.java @@ -22,6 +22,11 @@ public final class MetricsQueryOptions { private String filter; private String metricNamespace; + /** + * Creates an instance of MetricsQueryOptions. + */ + public MetricsQueryOptions() { } + /** * Returns the time span for which the metrics data is queried. * @return the time span for which the metrics data is queried. diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsBatchQueryOptions.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryResourcesOptions.java similarity index 88% rename from sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsBatchQueryOptions.java rename to sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryResourcesOptions.java index 108ebb5b57f38..b5e03446eb809 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsBatchQueryOptions.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryResourcesOptions.java @@ -12,7 +12,7 @@ * The model class to configure the metrics batch query options. */ @Fluent -public final class MetricsBatchQueryOptions { +public final class MetricsQueryResourcesOptions { private QueryTimeInterval timeInterval; private Duration granularity; private List aggregations; @@ -20,8 +20,12 @@ public final class MetricsBatchQueryOptions { private String orderBy; private String filter; - private String rollupBy; + /** + * Creates an instance of MetricsQueryResourcesOptions. + */ + public MetricsQueryResourcesOptions() { } + private String rollupBy; /** * Returns the time span for which the metrics data is queried. @@ -37,7 +41,7 @@ public QueryTimeInterval getTimeInterval() { * * @return The updated options instance */ - public MetricsBatchQueryOptions setTimeInterval(QueryTimeInterval timeInterval) { + public MetricsQueryResourcesOptions setTimeInterval(QueryTimeInterval timeInterval) { this.timeInterval = timeInterval; return this; } @@ -56,7 +60,7 @@ public Duration getGranularity() { * * @return The updated options instance */ - public MetricsBatchQueryOptions setGranularity(Duration granularity) { + public MetricsQueryResourcesOptions setGranularity(Duration granularity) { this.granularity = granularity; return this; } @@ -74,7 +78,7 @@ public List getAggregations() { * @param aggregations the list of aggregations that should be applied to the metrics data. * @return The updated options instance */ - public MetricsBatchQueryOptions setAggregations(List aggregations) { + public MetricsQueryResourcesOptions setAggregations(List aggregations) { this.aggregations = aggregations; return this; } @@ -93,7 +97,7 @@ public Integer getTop() { * * @return The updated options instance */ - public MetricsBatchQueryOptions setTop(Integer top) { + public MetricsQueryResourcesOptions setTop(Integer top) { this.top = top; return this; } @@ -112,7 +116,7 @@ public String getOrderBy() { * * @return The updated options instance */ - public MetricsBatchQueryOptions setOrderBy(String orderBy) { + public MetricsQueryResourcesOptions setOrderBy(String orderBy) { this.orderBy = orderBy; return this; } @@ -131,7 +135,7 @@ public String getFilter() { * * @return The updated options instance */ - public MetricsBatchQueryOptions setFilter(String filter) { + public MetricsQueryResourcesOptions setFilter(String filter) { this.filter = filter; return this; } @@ -159,7 +163,7 @@ public String getRollupBy() { * specify 'RollUpBy=City' to see the results for Seattle and Tacoma rolled up into one timeseries. * @return The updated options instance */ - public MetricsBatchQueryOptions setRollupBy(String rollupBy) { + public MetricsQueryResourcesOptions setRollupBy(String rollupBy) { this.rollupBy = rollupBy; return this; } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsBatchQueryResult.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryResourcesResult.java similarity index 85% rename from sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsBatchQueryResult.java rename to sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryResourcesResult.java index 4de74cd34e3fb..a1e83c9a17a15 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsBatchQueryResult.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/MetricsQueryResourcesResult.java @@ -8,14 +8,14 @@ /** * The result of a metrics query batch. It contains the results of individual queries. */ -public final class MetricsBatchQueryResult { +public final class MetricsQueryResourcesResult { private final List metricsQueryResults; /** * Creates an instance of MetricsBatchResult. * @param metricsQueryResults the metrics results for individual queries */ - public MetricsBatchQueryResult(List metricsQueryResults) { + public MetricsQueryResourcesResult(List metricsQueryResults) { this.metricsQueryResults = metricsQueryResults; } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/NamespaceClassification.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/NamespaceClassification.java index e366de2d3c572..f94469cf9f6f3 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/NamespaceClassification.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/models/NamespaceClassification.java @@ -18,6 +18,11 @@ public final class NamespaceClassification extends ExpandableStringEnum values() { return values(NamespaceClassification.class); } diff --git a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/package-info.java b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/package-info.java index d4471509d8537..6f5ecd316e98b 100644 --- a/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/package-info.java +++ b/sdk/monitor/azure-monitor-query/src/main/java/com/azure/monitor/query/package-info.java @@ -2,6 +2,121 @@ // Licensed under the MIT License. /** - * Package containing clients for querying logs and metrics from Azure Monitor. + *

Azure Monitor Query service is a powerful tool that allows you to query and analyze log data from various sources + * in Azure. It is built on top of the Kusto Query Language (KQL), which is a powerful query language that allows you + * to perform complex queries on large datasets. With Azure Monitor Query, you can easily search and analyze + * log data from various sources, including virtual machines, containers, and applications.

+ * + *

Azure Monitor Query java client library is a library that allows you to execute read-only queries against + * Azure Monitor’s two data platforms: Logs and Metrics. The library provides both synchronous and asynchronous forms + * of the clients.

+ * + *
    + *
  • Logs - Collects and + * organizes log and performance data from monitored resources. Data from different sources such as platform logs + * from Azure services, log and performance data from virtual machines agents, and usage and performance data from + * apps can be consolidated into a single Azure Log Analytics workspace. + * The various data types can be analyzed together using the Kusto Query Language.
  • + * + *
  • Metrics - Collects + * numeric data from monitored resources into a time series database. Metrics are numerical values that are + * collected at regular intervals and describe some aspect of a system at a particular time. Metrics are lightweight + * and capable of supporting near real-time scenarios, making them particularly useful for alerting and fast + * detection of issues.
  • + *
+ * + *

Getting Started

+ * + *

In order to interact with the Monitor service you'll need to create an instance of the + * {@link com.azure.monitor.query.LogsQueryClient} or {@link com.azure.monitor.query.MetricsQueryClient} class. To make + * this possible you'll need to use AAD authentication via + * Azure Identity + * to connect to the service.

+ * + *

Sample: Construct Asynchronous Clients

+ * + *

The following code sample demonstrates the creation of a {@link com.azure.monitor.query.LogsQueryAsyncClient} + * using the {@link com.azure.monitor.query.LogsQueryClientBuilder}.

+ * + * + *
+ * LogsQueryAsyncClient logsQueryAsyncClient = new LogsQueryClientBuilder()
+ *         .credential(tokenCredential)
+ *         .buildAsyncClient();
+ * 
+ * + * + *

The following code sample demonstrates the creation of a {@link com.azure.monitor.query.MetricsQueryAsyncClient} + * using the {@link com.azure.monitor.query.MetricsQueryClientBuilder}.

+ * + * + *
+ * MetricsQueryAsyncClient metricsQueryAsyncClient = new MetricsQueryClientBuilder()
+ *         .credential(tokenCredential)
+ *         .buildAsyncClient();
+ * 
+ * + * + *

Sample: Construct Synchronous Clients

+ * + *

The following code sample demonstrates the creation of a {@link com.azure.monitor.query.LogsQueryClient} + * using the {@link com.azure.monitor.query.LogsQueryClientBuilder}.

+ * + * + *
+ * LogsQueryClient logsQueryClient = new LogsQueryClientBuilder()
+ *         .credential(tokenCredential)
+ *         .buildClient();
+ * 
+ * + * + *

The following code sample demonstrates the creation of a {@link com.azure.monitor.query.MetricsQueryClient} + * using the {@link com.azure.monitor.query.MetricsQueryClientBuilder}.

+ * + * + *
+ * MetricsQueryClient metricsQueryClient = new MetricsQueryClientBuilder()
+ *         .credential(tokenCredential)
+ *         .buildClient();
+ * 
+ * + * + *
+ * + *
+ * + *

Query Workspace

+ * + *

The {@link com.azure.monitor.query.LogsQueryClient#queryWorkspace(java.lang.String, java.lang.String, com.azure.monitor.query.models.QueryTimeInterval) Query Workspace API} method can be used to + * query logs from a given workspace.

+ * + *

The sample below shows how to query logs from the last 24 hours

+ * + * + *
+ * LogsQueryResult queryResult = logsQueryClient.queryWorkspace("{workspace-id}", "{kusto-query}",
+ *         QueryTimeInterval.LAST_DAY);
+ * for (LogsTableRow row : queryResult.getTable().getRows()) {
+ *     System.out.println(row.getRow()
+ *             .stream()
+ *             .map(LogsTableCell::getValueAsString)
+ *             .collect(Collectors.joining(",")));
+ * }
+ * 
+ * + * + * + *

Note: For asynchronous sample, refer to {@link com.azure.monitor.query.LogsQueryAsyncClient#queryWorkspace(java.lang.String, java.lang.String, com.azure.monitor.query.models.QueryTimeInterval) QueryWorkspace Async API}.

+ * + *
+ * + *
+ * + * @see com.azure.monitor.query.LogsQueryClientBuilder + * @see com.azure.monitor.query.LogsQueryClient + * @see com.azure.monitor.query.LogsQueryAsyncClient + * @see com.azure.monitor.query.MetricsQueryClientBuilder + * @see com.azure.monitor.query.MetricsQueryClient + * @see com.azure.monitor.query.MetricsQueryAsyncClient */ package com.azure.monitor.query; diff --git a/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/MetricsBatchQuerySample.java b/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/MetricsSample.java similarity index 83% rename from sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/MetricsBatchQuerySample.java rename to sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/MetricsSample.java index 6ef22212bcbfc..0cac473a88733 100644 --- a/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/MetricsBatchQuerySample.java +++ b/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/MetricsSample.java @@ -5,7 +5,7 @@ import com.azure.identity.DefaultAzureCredentialBuilder; import com.azure.monitor.query.models.MetricResult; -import com.azure.monitor.query.models.MetricsBatchQueryResult; +import com.azure.monitor.query.models.MetricsQueryResourcesResult; import com.azure.monitor.query.models.MetricsQueryResult; import java.util.Arrays; @@ -14,24 +14,24 @@ /** * Sample demonstrates how to synchronously query metrics for multiple resources. */ -public class MetricsBatchQuerySample { +public class MetricsSample { /** * The main method to execute the sample. * * @param args Unused. Arguments to the program. */ public static void main(String[] args) { - MetricsClient metricsBatchQueryClient = new MetricsClientBuilder() + MetricsClient metricsClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("https://westus2.monitoring.azure.com") .buildClient(); - MetricsBatchQueryResult metricsBatchQueryResult = metricsBatchQueryClient.queryBatch( + MetricsQueryResourcesResult metricsQueryResourcesResult = metricsClient.queryResources( Arrays.asList("{resourceId1}", "{resourceId2}"), Arrays.asList("{metric1}", "{metric2}"), "{metricNamespace}"); - for (MetricsQueryResult metricsQueryResult : metricsBatchQueryResult.getMetricsQueryResults()) { + for (MetricsQueryResult metricsQueryResult : metricsQueryResourcesResult.getMetricsQueryResults()) { // Each MetricsQueryResult corresponds to one of the resourceIds in the batch request. List metrics = metricsQueryResult.getMetrics(); metrics.forEach(metric -> { diff --git a/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/ReadmeSamples.java b/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/ReadmeSamples.java index 9d2ea302797a3..eda715f96f3ae 100644 --- a/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/ReadmeSamples.java +++ b/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/ReadmeSamples.java @@ -21,7 +21,7 @@ import com.azure.monitor.query.models.LogsTableRow; import com.azure.monitor.query.models.MetricResult; import com.azure.monitor.query.models.MetricValue; -import com.azure.monitor.query.models.MetricsBatchQueryResult; +import com.azure.monitor.query.models.MetricsQueryResourcesResult; import com.azure.monitor.query.models.MetricsQueryOptions; import com.azure.monitor.query.models.MetricsQueryResult; import com.azure.monitor.query.models.QueryTimeInterval; @@ -58,7 +58,7 @@ public void createLogsClients() { /** * Sample for creating sync and async clients for querying metrics. */ - public void createMetricsClients() { + public void createMetricsQueryClients() { // BEGIN: readme-sample-createMetricsQueryClient MetricsQueryClient metricsQueryClient = new MetricsQueryClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) @@ -72,20 +72,20 @@ public void createMetricsClients() { // END: readme-sample-createMetricsQueryAsyncClient } - public void createMetricsBatchClients() { - // BEGIN: readme-sample-createMetricsBatchQueryClient - MetricsClient metricsBatchQueryClient = new MetricsClientBuilder() + public void createMetricsClients() { + // BEGIN: readme-sample-createMetricsClient + MetricsClient metricsClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("{endpoint}") .buildClient(); - // END: readme-sample-createMetricsBatchQueryClient + // END: readme-sample-createMetricsClient - // BEGIN: readme-sample-createMetricsBatchQueryAsyncClient + // BEGIN: readme-sample-createMetricsAsyncClient MetricsAsyncClient metricsAsyncClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("{endpoint}") .buildAsyncClient(); - // END: readme-sample-createMetricsBatchQueryAsyncClient + // END: readme-sample-createMetricsAsyncClient } /** @@ -446,19 +446,19 @@ public void createMetricsClientWithSovereignCloud() { /** * Sample to demonstrate querying Azure Monitor for metrics. */ - public void getMetricsBatch() { - // BEGIN: readme-sample-metricsquerybatch - MetricsClient metricsBatchQueryClient = new MetricsClientBuilder() + public void getMetricsForMultipleResources() { + // BEGIN: readme-sample-metricsquerymultipleresources + MetricsClient metricsClient = new MetricsClientBuilder() .credential(new DefaultAzureCredentialBuilder().build()) .endpoint("{endpoint}") .buildClient(); - MetricsBatchQueryResult metricsBatchQueryResult = metricsBatchQueryClient.queryBatch( + MetricsQueryResourcesResult metricsQueryResourcesResult = metricsClient.queryResources( Arrays.asList("{resourceId1}", "{resourceId2}"), Arrays.asList("{metric1}", "{metric2}"), "{metricNamespace}"); - for (MetricsQueryResult metricsQueryResult : metricsBatchQueryResult.getMetricsQueryResults()) { + for (MetricsQueryResult metricsQueryResult : metricsQueryResourcesResult.getMetricsQueryResults()) { // Each MetricsQueryResult corresponds to one of the resourceIds in the batch request. List metrics = metricsQueryResult.getMetrics(); metrics.forEach(metric -> { @@ -476,6 +476,6 @@ public void getMetricsBatch() { + "; Average = " + mv.getAverage())); }); } - // END: readme-sample-metricsquerybatch + // END: readme-sample-metricsquerymultipleresources } } diff --git a/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/codesnippets/MetricsBatchQueryClientJavaDocCodeSnippets.java b/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/codesnippets/MetricsBatchQueryClientJavaDocCodeSnippets.java index 1f0d2dfc6260a..5f055a45e5722 100644 --- a/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/codesnippets/MetricsBatchQueryClientJavaDocCodeSnippets.java +++ b/sdk/monitor/azure-monitor-query/src/samples/java/com/azure/monitor/query/codesnippets/MetricsBatchQueryClientJavaDocCodeSnippets.java @@ -8,7 +8,7 @@ import com.azure.monitor.query.MetricsClient; import com.azure.monitor.query.MetricsClientBuilder; import com.azure.monitor.query.models.MetricResult; -import com.azure.monitor.query.models.MetricsBatchQueryResult; +import com.azure.monitor.query.models.MetricsQueryResourcesResult; import com.azure.monitor.query.models.MetricsQueryResult; import java.util.Arrays; @@ -41,7 +41,7 @@ public void createAsyncClient() { } /** - * Generates a code sample for using {@link MetricsClient#queryBatch(List, List, String)}. + * Generates a code sample for using {@link MetricsClient#queryResources(List, List, String)}. */ public void queryBatch() { MetricsClient metricsBatchQueryClient = new MetricsClientBuilder() @@ -50,11 +50,11 @@ public void queryBatch() { .buildClient(); // BEGIN: com.azure.monitor.query.MetricsBatchQueryClient.queryBatch#List-List-String - MetricsBatchQueryResult metricsBatchQueryResult = metricsBatchQueryClient.queryBatch( + MetricsQueryResourcesResult metricsQueryResourcesResult = metricsBatchQueryClient.queryResources( Arrays.asList("{resourceId1}", "{resourceId2}"), Arrays.asList("{metricId}"), "{metricNamespace}"); - for (MetricsQueryResult metricsQueryResult : metricsBatchQueryResult.getMetricsQueryResults()) { + for (MetricsQueryResult metricsQueryResult : metricsQueryResourcesResult.getMetricsQueryResults()) { // Each MetricsQueryResult corresponds to one of the resourceIds in the batch request. List metrics = metricsQueryResult.getMetrics(); metrics.forEach(metric -> { @@ -82,7 +82,7 @@ public void queryBatchAsync() { .buildAsyncClient(); // BEGIN: com.azure.monitor.query.MetricsBatchQueryAsyncClient.queryBatch#List-List-String - metricsAsyncClient.queryBatch( + metricsAsyncClient.queryResources( Arrays.asList("{resourceId1}", "{resourceId2}"), Arrays.asList("{metricId}"), "{metricNamespace}") .subscribe(metricsBatchResult -> { diff --git a/sdk/monitor/azure-monitor-query/src/test/java/com/azure/monitor/query/MetricsClientTest.java b/sdk/monitor/azure-monitor-query/src/test/java/com/azure/monitor/query/MetricsClientTest.java index ce1ea8d3540ec..2bf88ba61daf1 100644 --- a/sdk/monitor/azure-monitor-query/src/test/java/com/azure/monitor/query/MetricsClientTest.java +++ b/sdk/monitor/azure-monitor-query/src/test/java/com/azure/monitor/query/MetricsClientTest.java @@ -10,8 +10,8 @@ import com.azure.core.util.Context; import com.azure.core.util.CoreUtils; import com.azure.monitor.query.models.MetricResult; -import com.azure.monitor.query.models.MetricsBatchQueryOptions; -import com.azure.monitor.query.models.MetricsBatchQueryResult; +import com.azure.monitor.query.models.MetricsQueryResourcesOptions; +import com.azure.monitor.query.models.MetricsQueryResourcesResult; import com.azure.monitor.query.models.QueryTimeInterval; import org.junit.jupiter.api.Test; @@ -40,12 +40,12 @@ public void testMetricsBatchQuery() { // ignore as this is only to generate some metrics } - MetricsBatchQueryOptions options = new MetricsBatchQueryOptions() + MetricsQueryResourcesOptions options = new MetricsQueryResourcesOptions() .setGranularity(Duration.ofMinutes(15)) .setTop(10) .setTimeInterval(new QueryTimeInterval(OffsetDateTime.now().minusDays(1), OffsetDateTime.now())); - MetricsBatchQueryResult metricsQueryResults = metricsClient.queryBatchWithResponse( + MetricsQueryResourcesResult metricsQueryResults = metricsClient.queryResourcesWithResponse( Arrays.asList(resourceId), Arrays.asList("HttpIncomingRequestCount"), "microsoft.appconfiguration/configurationstores", options, Context.NONE) .getValue(); @@ -66,7 +66,7 @@ public void testMetricsBatchQueryDifferentResourceTypes() { String updatedResource1 = resourceId1.substring(resourceId1.indexOf("/subscriptions")); String updatedResource2 = resourceId2.substring(resourceId2.indexOf("/subscriptions")); - assertThrows(HttpResponseException.class, () -> metricsBatchQueryClient.queryBatch( + assertThrows(HttpResponseException.class, () -> metricsBatchQueryClient.queryResources( Arrays.asList(updatedResource1, updatedResource2), Arrays.asList("Successful Requests"), " Microsoft.Eventhub/Namespaces")); diff --git a/sdk/monitor/azure-monitor-query/tests.yml b/sdk/monitor/azure-monitor-query/tests.yml index 6f13a1cd8361b..9463d0d470a9e 100644 --- a/sdk/monitor/azure-monitor-query/tests.yml +++ b/sdk/monitor/azure-monitor-query/tests.yml @@ -1,14 +1,14 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: monitor - Artifacts: - - name: azure-monitor-query - groupId: com.azure - safeName: azuremonitorquery - EnvVars: - AZURE_CLIENT_ID: $(MONITOR_CLIENT_ID) - AZURE_TENANT_ID: $(MONITOR_TENANT_ID) - AZURE_CLIENT_SECRET: $(MONITOR_CLIENT_SECRET) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: monitor + Artifacts: + - name: azure-monitor-query + groupId: com.azure + safeName: azuremonitorquery + EnvVars: + AZURE_CLIENT_ID: $(MONITOR_CLIENT_ID) + AZURE_TENANT_ID: $(MONITOR_TENANT_ID) + AZURE_CLIENT_SECRET: $(MONITOR_CLIENT_SECRET) diff --git a/sdk/monitor/tests.yml b/sdk/monitor/tests.yml index 208844b8aee5e..2384248c37841 100644 --- a/sdk/monitor/tests.yml +++ b/sdk/monitor/tests.yml @@ -1,19 +1,19 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: monitor - SupportedClouds: 'Public,UsGov,China' - Artifacts: - - name: azure-monitor-ingestion - groupId: com.azure - safeName: azuremonitoringestion - - name: azure-monitor-query - groupId: com.azure - safeName: azuremonitorquery - EnvVars: - AZURE_CLIENT_ID: $(MONITOR_CLIENT_ID) - AZURE_TENANT_ID: $(MONITOR_TENANT_ID) - AZURE_CLIENT_SECRET: $(MONITOR_CLIENT_SECRET) - AZURE_SUBSCRIPTION_ID: $(MONITOR_SUBSCRIPTION_ID) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: monitor + SupportedClouds: 'Public,UsGov,China' + Artifacts: + - name: azure-monitor-ingestion + groupId: com.azure + safeName: azuremonitoringestion + - name: azure-monitor-query + groupId: com.azure + safeName: azuremonitorquery + EnvVars: + AZURE_CLIENT_ID: $(MONITOR_CLIENT_ID) + AZURE_TENANT_ID: $(MONITOR_TENANT_ID) + AZURE_CLIENT_SECRET: $(MONITOR_CLIENT_SECRET) + AZURE_SUBSCRIPTION_ID: $(MONITOR_SUBSCRIPTION_ID) diff --git a/sdk/mysql/azure-resourcemanager-mysql/pom.xml b/sdk/mysql/azure-resourcemanager-mysql/pom.xml index 3573d266934bd..9302ceb3f8b3d 100644 --- a/sdk/mysql/azure-resourcemanager-mysql/pom.xml +++ b/sdk/mysql/azure-resourcemanager-mysql/pom.xml @@ -61,7 +61,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -79,7 +79,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/mysql/tests.mgmt.yml b/sdk/mysql/tests.mgmt.yml index 4382b9372f28d..423a0b131cdae 100644 --- a/sdk/mysql/tests.mgmt.yml +++ b/sdk/mysql/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: mysql - Artifacts: - - name: azure-resourcemanager-mysql - groupId: com.azure.resourcemanager - safeName: azureresourcemanagermysql - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: mysql + Artifacts: + - name: azure-resourcemanager-mysql + groupId: com.azure.resourcemanager + safeName: azureresourcemanagermysql + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/mysqlflexibleserver/azure-resourcemanager-mysqlflexibleserver/pom.xml b/sdk/mysqlflexibleserver/azure-resourcemanager-mysqlflexibleserver/pom.xml index 2547efa0b4671..c4b2cab90d7ad 100644 --- a/sdk/mysqlflexibleserver/azure-resourcemanager-mysqlflexibleserver/pom.xml +++ b/sdk/mysqlflexibleserver/azure-resourcemanager-mysqlflexibleserver/pom.xml @@ -60,7 +60,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -72,7 +72,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/mysqlflexibleserver/tests.mgmt.yml b/sdk/mysqlflexibleserver/tests.mgmt.yml index 9f5deae0a60fc..ad008048e38c0 100644 --- a/sdk/mysqlflexibleserver/tests.mgmt.yml +++ b/sdk/mysqlflexibleserver/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: mysqlflexibleserver - Artifacts: - - name: azure-resourcemanager-mysqlflexibleserver - groupId: com.azure.resourcemanager - safeName: azureresourcemanagermysqlflexibleserver - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: mysqlflexibleserver + Artifacts: + - name: azure-resourcemanager-mysqlflexibleserver + groupId: com.azure.resourcemanager + safeName: azureresourcemanagermysqlflexibleserver + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/netapp/azure-resourcemanager-netapp/CHANGELOG.md b/sdk/netapp/azure-resourcemanager-netapp/CHANGELOG.md index c509152f1bd77..370553640ed69 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/CHANGELOG.md +++ b/sdk/netapp/azure-resourcemanager-netapp/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.2 (Unreleased) +## 1.2.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,16 @@ ### Other Changes +## 1.1.0 (2024-03-20) + +- Azure Resource Manager NetAppFiles client library for Java. This package contains Microsoft Azure SDK for NetAppFiles Management SDK. Microsoft NetApp Files Azure Resource Provider specification. Package tag package-netapp-2023-07-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Breaking Changes + +#### `models.VolumeGroupMetadata` was modified + +* Removed `deploymentSpecId` property as it's non-functional in service backend. + ## 1.1.0-beta.1 (2023-12-19) - Azure Resource Manager NetAppFiles client library for Java. This package contains Microsoft Azure SDK for NetAppFiles Management SDK. Microsoft NetApp Files Azure Resource Provider specification. Package tag package-preview-2023-05. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/netapp/azure-resourcemanager-netapp/README.md b/sdk/netapp/azure-resourcemanager-netapp/README.md index bdd44a8194324..e391122fca3f4 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/README.md +++ b/sdk/netapp/azure-resourcemanager-netapp/README.md @@ -2,7 +2,7 @@ Azure Resource Manager NetAppFiles client library for Java. -This package contains Microsoft Azure SDK for NetAppFiles Management SDK. Microsoft NetApp Files Azure Resource Provider specification. Package tag package-preview-2023-05. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for NetAppFiles Management SDK. Microsoft NetApp Files Azure Resource Provider specification. Package tag package-netapp-2023-07-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-netapp - 1.1.0-beta.1 + 1.1.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/netapp/azure-resourcemanager-netapp/SAMPLE.md b/sdk/netapp/azure-resourcemanager-netapp/SAMPLE.md index c1066e3f20412..013ef157b6580 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/SAMPLE.md +++ b/sdk/netapp/azure-resourcemanager-netapp/SAMPLE.md @@ -1,12 +1,6 @@ # Code snippets and samples -## AccountBackups - -- [Delete](#accountbackups_delete) -- [Get](#accountbackups_get) -- [ListByNetAppAccount](#accountbackups_listbynetappaccount) - ## Accounts - [CreateOrUpdate](#accounts_createorupdate) @@ -14,7 +8,6 @@ - [GetByResourceGroup](#accounts_getbyresourcegroup) - [List](#accounts_list) - [ListByResourceGroup](#accounts_listbyresourcegroup) -- [MigrateEncryptionKey](#accounts_migrateencryptionkey) - [RenewCredentials](#accounts_renewcredentials) - [Update](#accounts_update) @@ -26,35 +19,9 @@ - [List](#backuppolicies_list) - [Update](#backuppolicies_update) -## BackupVaults - -- [CreateOrUpdate](#backupvaults_createorupdate) -- [Delete](#backupvaults_delete) -- [Get](#backupvaults_get) -- [ListByNetAppAccount](#backupvaults_listbynetappaccount) -- [Update](#backupvaults_update) - ## Backups -- [Create](#backups_create) -- [Delete](#backups_delete) -- [Get](#backups_get) -- [GetLatestStatus](#backups_getlateststatus) - [GetVolumeRestoreStatus](#backups_getvolumerestorestatus) -- [ListByVault](#backups_listbyvault) -- [Update](#backups_update) - -## BackupsUnderAccount - -- [MigrateBackups](#backupsunderaccount_migratebackups) - -## BackupsUnderBackupVault - -- [RestoreFiles](#backupsunderbackupvault_restorefiles) - -## BackupsUnderVolume - -- [MigrateBackups](#backupsundervolume_migratebackups) ## NetAppResource @@ -70,11 +37,6 @@ - [Get](#netappresourcequotalimits_get) - [List](#netappresourcequotalimits_list) -## NetAppResourceRegionInfos - -- [Get](#netappresourceregioninfos_get) -- [List](#netappresourceregioninfos_list) - ## Operations - [List](#operations_list) @@ -152,71 +114,7 @@ - [ResyncReplication](#volumes_resyncreplication) - [Revert](#volumes_revert) - [RevertRelocation](#volumes_revertrelocation) -- [SplitCloneFromParent](#volumes_splitclonefromparent) - [Update](#volumes_update) -### AccountBackups_Delete - -```java -/** - * Samples for AccountBackups Delete. - */ -public final class AccountBackupsDeleteSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Backups_Account_Delete.json - */ - /** - * Sample code: AccountBackups_Delete. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountBackupsDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accountBackups().delete("resourceGroup", "accountName", "backupName", com.azure.core.util.Context.NONE); - } -} -``` - -### AccountBackups_Get - -```java -/** - * Samples for AccountBackups Get. - */ -public final class AccountBackupsGetSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Backups_Account_Get.json - */ - /** - * Sample code: AccountBackups_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountBackupsGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accountBackups().getWithResponse("myRG", "account1", "backup1", com.azure.core.util.Context.NONE); - } -} -``` - -### AccountBackups_ListByNetAppAccount - -```java -/** - * Samples for AccountBackups ListByNetAppAccount. - */ -public final class AccountBackupsListByNetAppAccountSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Backups_Account_List.json - */ - /** - * Sample code: AccountBackups_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountBackupsList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accountBackups().listByNetAppAccount("myRG", "account1", null, com.azure.core.util.Context.NONE); - } -} -``` - ### Accounts_CreateOrUpdate ```java @@ -228,7 +126,8 @@ import java.util.Arrays; */ public final class AccountsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_CreateOrUpdate.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_CreateOrUpdate.json */ /** * Sample code: Accounts_CreateOrUpdate. @@ -240,15 +139,22 @@ public final class AccountsCreateOrUpdateSamples { } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_CreateOrUpdateAD.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_CreateOrUpdateAD.json */ /** * Sample code: Accounts_CreateOrUpdateWithActiveDirectory. * * @param manager Entry point to NetAppFilesManager. */ - public static void accountsCreateOrUpdateWithActiveDirectory(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accounts().define("account1").withRegion("eastus").withExistingResourceGroup("myRG").withActiveDirectories(Arrays.asList(new ActiveDirectory().withUsername("ad_user_name").withPassword("fakeTokenPlaceholder").withDomain("10.10.10.3").withDns("10.10.10.3").withSmbServerName("SMBServer").withOrganizationalUnit("OU=Engineering").withSite("SiteName").withAesEncryption(true).withLdapSigning(false).withLdapOverTls(false))).create(); + public static void + accountsCreateOrUpdateWithActiveDirectory(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { + manager.accounts().define("account1").withRegion("eastus").withExistingResourceGroup("myRG") + .withActiveDirectories(Arrays.asList(new ActiveDirectory().withUsername("ad_user_name") + .withPassword("fakeTokenPlaceholder").withDomain("10.10.10.3").withDns("10.10.10.3") + .withSmbServerName("SMBServer").withOrganizationalUnit("OU=Engineering").withSite("SiteName") + .withAesEncryption(true).withLdapSigning(false).withLdapOverTls(false))) + .create(); } } ``` @@ -261,7 +167,8 @@ public final class AccountsCreateOrUpdateSamples { */ public final class AccountsDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_Delete.json */ /** * Sample code: Accounts_Delete. @@ -282,7 +189,8 @@ public final class AccountsDeleteSamples { */ public final class AccountsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_Get.json */ /** * Sample code: Accounts_Get. @@ -303,7 +211,8 @@ public final class AccountsGetByResourceGroupSamples { */ public final class AccountsListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_List.json */ /** * Sample code: Accounts_List. @@ -324,7 +233,8 @@ public final class AccountsListSamples { */ public final class AccountsListByResourceGroupSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_List.json */ /** * Sample code: Accounts_List. @@ -337,29 +247,6 @@ public final class AccountsListByResourceGroupSamples { } ``` -### Accounts_MigrateEncryptionKey - -```java -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; - -/** - * Samples for Accounts MigrateEncryptionKey. - */ -public final class AccountsMigrateEncryptionKeySamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_MigrateEncryptionKey.json - */ - /** - * Sample code: Accounts_MigrateEncryptionKey. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountsMigrateEncryptionKey(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accounts().migrateEncryptionKey("myRG", "account1", new EncryptionMigrationRequest().withVirtualNetworkId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.Network/virtualNetworks/vnet1").withPrivateEndpointId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.Network/privateEndpoints/privip1"), com.azure.core.util.Context.NONE); - } -} -``` - ### Accounts_RenewCredentials ```java @@ -368,7 +255,8 @@ public final class AccountsMigrateEncryptionKeySamples { */ public final class AccountsRenewCredentialsSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_RenewCredentials.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_RenewCredentials.json */ /** * Sample code: Accounts_RenewCredentials. @@ -393,7 +281,8 @@ import java.util.Map; */ public final class AccountsUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_Update.json */ /** * Sample code: Accounts_Update. @@ -401,7 +290,8 @@ public final class AccountsUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void accountsUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - NetAppAccount resource = manager.accounts().getByResourceGroupWithResponse("myRG", "account1", com.azure.core.util.Context.NONE).getValue(); + NetAppAccount resource = manager.accounts() + .getByResourceGroupWithResponse("myRG", "account1", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("Tag1", "Value1")).apply(); } @@ -427,7 +317,8 @@ public final class AccountsUpdateSamples { */ public final class BackupPoliciesCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Create.json */ /** * Sample code: BackupPolicies_Create. @@ -435,7 +326,9 @@ public final class BackupPoliciesCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void backupPoliciesCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupPolicies().define("backupPolicyName").withRegion("westus").withExistingNetAppAccount("myRG", "account1").withDailyBackupsToKeep(10).withWeeklyBackupsToKeep(10).withMonthlyBackupsToKeep(10).withEnabled(true).create(); + manager.backupPolicies().define("backupPolicyName").withRegion("westus") + .withExistingNetAppAccount("myRG", "account1").withDailyBackupsToKeep(10).withWeeklyBackupsToKeep(10) + .withMonthlyBackupsToKeep(10).withEnabled(true).create(); } } ``` @@ -448,7 +341,8 @@ public final class BackupPoliciesCreateSamples { */ public final class BackupPoliciesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Delete.json */ /** * Sample code: Backups_Delete. @@ -456,7 +350,8 @@ public final class BackupPoliciesDeleteSamples { * @param manager Entry point to NetAppFilesManager. */ public static void backupsDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupPolicies().delete("resourceGroup", "accountName", "backupPolicyName", com.azure.core.util.Context.NONE); + manager.backupPolicies().delete("resourceGroup", "accountName", "backupPolicyName", + com.azure.core.util.Context.NONE); } } ``` @@ -469,7 +364,8 @@ public final class BackupPoliciesDeleteSamples { */ public final class BackupPoliciesGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Get.json */ /** * Sample code: Backups_Get. @@ -477,7 +373,8 @@ public final class BackupPoliciesGetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void backupsGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupPolicies().getWithResponse("myRG", "account1", "backupPolicyName", com.azure.core.util.Context.NONE); + manager.backupPolicies().getWithResponse("myRG", "account1", "backupPolicyName", + com.azure.core.util.Context.NONE); } } ``` @@ -490,7 +387,8 @@ public final class BackupPoliciesGetSamples { */ public final class BackupPoliciesListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_List.json */ /** * Sample code: Backups_List. @@ -513,7 +411,8 @@ import com.azure.resourcemanager.netapp.models.BackupPolicy; */ public final class BackupPoliciesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Update.json */ /** * Sample code: BackupPolicies_Update. @@ -521,214 +420,10 @@ public final class BackupPoliciesUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void backupPoliciesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - BackupPolicy resource = manager.backupPolicies().getWithResponse("myRG", "account1", "backupPolicyName", com.azure.core.util.Context.NONE).getValue(); - resource.update().withDailyBackupsToKeep(5).withWeeklyBackupsToKeep(10).withMonthlyBackupsToKeep(10).withEnabled(false).apply(); - } -} -``` - -### BackupVaults_CreateOrUpdate - -```java -/** - * Samples for BackupVaults CreateOrUpdate. - */ -public final class BackupVaultsCreateOrUpdateSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Create.json - */ - /** - * Sample code: BackupVault_CreateOrUpdate. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultCreateOrUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().define("backupVault1").withRegion("eastus").withExistingNetAppAccount("myRG", "account1").create(); - } -} -``` - -### BackupVaults_Delete - -```java -/** - * Samples for BackupVaults Delete. - */ -public final class BackupVaultsDeleteSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Delete.json - */ - /** - * Sample code: BackupVaults_Delete. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().delete("resourceGroup", "account1", "backupVault1", com.azure.core.util.Context.NONE); - } -} -``` - -### BackupVaults_Get - -```java -/** - * Samples for BackupVaults Get. - */ -public final class BackupVaultsGetSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Get.json - */ - /** - * Sample code: BackupVaults_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().getWithResponse("myRG", "account1", "backupVault1", com.azure.core.util.Context.NONE); - } -} -``` - -### BackupVaults_ListByNetAppAccount - -```java -/** - * Samples for BackupVaults ListByNetAppAccount. - */ -public final class BackupVaultsListByNetAppAccountSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_List.json - */ - /** - * Sample code: BackupVaults_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().listByNetAppAccount("myRG", "account1", com.azure.core.util.Context.NONE); - } -} -``` - -### BackupVaults_Update - -```java -import com.azure.resourcemanager.netapp.models.BackupVault; -import java.util.HashMap; -import java.util.Map; - -/** - * Samples for BackupVaults Update. - */ -public final class BackupVaultsUpdateSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Update.json - */ - /** - * Sample code: BackupVaults_Update. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - BackupVault resource = manager.backupVaults().getWithResponse("myRG", "account1", "backupVault1", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("Tag1", "Value1")).apply(); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} -``` - -### Backups_Create - -```java -/** - * Samples for Backups Create. - */ -public final class BackupsCreateSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderBackupVault_Create.json - */ - /** - * Sample code: BackupsUnderBackupVault_Create. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().define("backup1").withExistingBackupVault("myRG", "account1", "backupVault1").withVolumeResourceId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPool/pool1/volumes/volume1").withLabel("myLabel").create(); - } -} -``` - -### Backups_Delete - -```java -/** - * Samples for Backups Delete. - */ -public final class BackupsDeleteSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderBackupVault_Delete.json - */ - /** - * Sample code: BackupsUnderBackupVault_Delete. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().delete("resourceGroup", "account1", "backupVault1", "backup1", com.azure.core.util.Context.NONE); - } -} -``` - -### Backups_Get - -```java -/** - * Samples for Backups Get. - */ -public final class BackupsGetSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderBackupVault_Get.json - */ - /** - * Sample code: BackupsUnderBackupVault_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().getWithResponse("myRG", "account1", "backupVault1", "backup1", com.azure.core.util.Context.NONE); - } -} -``` - -### Backups_GetLatestStatus - -```java -/** - * Samples for Backups GetLatestStatus. - */ -public final class BackupsGetLatestStatusSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_LatestBackupStatus.json - */ - /** - * Sample code: Volumes_BackupStatus. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void volumesBackupStatus(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().getLatestStatusWithResponse("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE); + BackupPolicy resource = manager.backupPolicies() + .getWithResponse("myRG", "account1", "backupPolicyName", com.azure.core.util.Context.NONE).getValue(); + resource.update().withDailyBackupsToKeep(5).withWeeklyBackupsToKeep(10).withMonthlyBackupsToKeep(10) + .withEnabled(false).apply(); } } ``` @@ -741,7 +436,8 @@ public final class BackupsGetLatestStatusSamples { */ public final class BackupsGetVolumeRestoreStatusSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_RestoreStatus.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_RestoreStatus.json */ /** * Sample code: Volumes_RestoreStatus. @@ -749,122 +445,8 @@ public final class BackupsGetVolumeRestoreStatusSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesRestoreStatus(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().getVolumeRestoreStatusWithResponse("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE); - } -} -``` - -### Backups_ListByVault - -```java -/** - * Samples for Backups ListByVault. - */ -public final class BackupsListByVaultSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderBackupVault_List.json - */ - /** - * Sample code: Backups_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().listByVault("myRG", "account1", "backupVault1", null, com.azure.core.util.Context.NONE); - } -} -``` - -### Backups_Update - -```java -import com.azure.resourcemanager.netapp.models.Backup; - -/** - * Samples for Backups Update. - */ -public final class BackupsUpdateSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderBackupVault_Update.json - */ - /** - * Sample code: BackupsUnderBackupVault_Update. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - Backup resource = manager.backups().getWithResponse("myRG", "account1", "backupVault1", "backup1", com.azure.core.util.Context.NONE).getValue(); - resource.update().apply(); - } -} -``` - -### BackupsUnderAccount_MigrateBackups - -```java -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; - -/** - * Samples for BackupsUnderAccount MigrateBackups. - */ -public final class BackupsUnderAccountMigrateBackupsSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderAccount_Migrate.json - */ - /** - * Sample code: BackupsUnderAccount_Migrate. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderAccountMigrate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupsUnderAccounts().migrateBackups("myRG", "account1", new BackupsMigrationRequest().withBackupVaultId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/backupVaults/backupVault1"), com.azure.core.util.Context.NONE); - } -} -``` - -### BackupsUnderBackupVault_RestoreFiles - -```java -import com.azure.resourcemanager.netapp.models.BackupRestoreFiles; -import java.util.Arrays; - -/** - * Samples for BackupsUnderBackupVault RestoreFiles. - */ -public final class BackupsUnderBackupVaultRestoreFilesSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderBackupVault_SingleFileRestore.json - */ - /** - * Sample code: Backups_SingleFileRestore. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsSingleFileRestore(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupsUnderBackupVaults().restoreFiles("myRG", "account1", "backupVault1", "backup1", new BackupRestoreFiles().withFileList(Arrays.asList("/dir1/customer1.db", "/dir1/customer2.db")).withDestinationVolumeId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1/volumes/volume1"), com.azure.core.util.Context.NONE); - } -} -``` - -### BackupsUnderVolume_MigrateBackups - -```java -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; - -/** - * Samples for BackupsUnderVolume MigrateBackups. - */ -public final class BackupsUnderVolumeMigrateBackupsSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupsUnderVolume_Migrate.json - */ - /** - * Sample code: BackupsUnderVolume_Migrate. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderVolumeMigrate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupsUnderVolumes().migrateBackups("myRG", "account1", "pool1", "volume1", new BackupsMigrationRequest().withBackupVaultId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/backupVaults/backupVault1"), com.azure.core.util.Context.NONE); + manager.backups().getVolumeRestoreStatusWithResponse("myRG", "account1", "pool1", "volume1", + com.azure.core.util.Context.NONE); } } ``` @@ -879,7 +461,8 @@ import com.azure.resourcemanager.netapp.models.FilePathAvailabilityRequest; */ public final class NetAppResourceCheckFilePathAvailabilitySamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/CheckFilePathAvailability.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/CheckFilePathAvailability.json */ /** * Sample code: CheckFilePathAvailability. @@ -887,7 +470,10 @@ public final class NetAppResourceCheckFilePathAvailabilitySamples { * @param manager Entry point to NetAppFilesManager. */ public static void checkFilePathAvailability(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResources().checkFilePathAvailabilityWithResponse("eastus", new FilePathAvailabilityRequest().withName("my-exact-filepth").withSubnetId("/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3"), com.azure.core.util.Context.NONE); + manager.netAppResources().checkFilePathAvailabilityWithResponse("eastus", + new FilePathAvailabilityRequest().withName("my-exact-filepth").withSubnetId( + "/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3"), + com.azure.core.util.Context.NONE); } } ``` @@ -897,14 +483,14 @@ public final class NetAppResourceCheckFilePathAvailabilitySamples { ```java import com.azure.resourcemanager.netapp.models.CheckNameResourceTypes; import com.azure.resourcemanager.netapp.models.ResourceNameAvailabilityRequest; -import java.util.stream.Collectors; /** * Samples for NetAppResource CheckNameAvailability. */ public final class NetAppResourceCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/CheckNameAvailability.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/CheckNameAvailability.json */ /** * Sample code: CheckNameAvailability. @@ -912,7 +498,10 @@ public final class NetAppResourceCheckNameAvailabilitySamples { * @param manager Entry point to NetAppFilesManager. */ public static void checkNameAvailability(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResources().checkNameAvailabilityWithResponse("eastus", new ResourceNameAvailabilityRequest().withName("accName").withType(CheckNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS).withResourceGroup("myRG"), com.azure.core.util.Context.NONE); + manager.netAppResources().checkNameAvailabilityWithResponse("eastus", + new ResourceNameAvailabilityRequest().withName("accName") + .withType(CheckNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS).withResourceGroup("myRG"), + com.azure.core.util.Context.NONE); } } ``` @@ -922,14 +511,14 @@ public final class NetAppResourceCheckNameAvailabilitySamples { ```java import com.azure.resourcemanager.netapp.models.CheckQuotaNameResourceTypes; import com.azure.resourcemanager.netapp.models.QuotaAvailabilityRequest; -import java.util.stream.Collectors; /** * Samples for NetAppResource CheckQuotaAvailability. */ public final class NetAppResourceCheckQuotaAvailabilitySamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/CheckQuotaAvailability.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/CheckQuotaAvailability.json */ /** * Sample code: CheckQuotaAvailability. @@ -937,7 +526,10 @@ public final class NetAppResourceCheckQuotaAvailabilitySamples { * @param manager Entry point to NetAppFilesManager. */ public static void checkQuotaAvailability(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResources().checkQuotaAvailabilityWithResponse("eastus", new QuotaAvailabilityRequest().withName("resource1").withType(CheckQuotaNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS).withResourceGroup("myRG"), com.azure.core.util.Context.NONE); + manager.netAppResources().checkQuotaAvailabilityWithResponse("eastus", + new QuotaAvailabilityRequest().withName("resource1") + .withType(CheckQuotaNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS).withResourceGroup("myRG"), + com.azure.core.util.Context.NONE); } } ``` @@ -952,7 +544,8 @@ import com.azure.resourcemanager.netapp.models.QueryNetworkSiblingSetRequest; */ public final class NetAppResourceQueryNetworkSiblingSetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/NetworkSiblingSet_Query.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/NetworkSiblingSet_Query.json */ /** * Sample code: NetworkSiblingSet_Query. @@ -960,7 +553,10 @@ public final class NetAppResourceQueryNetworkSiblingSetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void networkSiblingSetQuery(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResources().queryNetworkSiblingSetWithResponse("eastus", new QueryNetworkSiblingSetRequest().withNetworkSiblingSetId("9760acf5-4638-11e7-9bdb-020073ca3333").withSubnetId("/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testVnet/subnets/testSubnet"), com.azure.core.util.Context.NONE); + manager.netAppResources().queryNetworkSiblingSetWithResponse("eastus", new QueryNetworkSiblingSetRequest() + .withNetworkSiblingSetId("9760acf5-4638-11e7-9bdb-020073ca3333").withSubnetId( + "/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testVnet/subnets/testSubnet"), + com.azure.core.util.Context.NONE); } } ``` @@ -973,7 +569,8 @@ public final class NetAppResourceQueryNetworkSiblingSetSamples { */ public final class NetAppResourceQueryRegionInfoSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/RegionInfo.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/RegionInfo.json */ /** * Sample code: RegionInfo_Query. @@ -991,14 +588,14 @@ public final class NetAppResourceQueryRegionInfoSamples { ```java import com.azure.resourcemanager.netapp.models.NetworkFeatures; import com.azure.resourcemanager.netapp.models.UpdateNetworkSiblingSetRequest; -import java.util.stream.Collectors; /** * Samples for NetAppResource UpdateNetworkSiblingSet. */ public final class NetAppResourceUpdateNetworkSiblingSetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/NetworkSiblingSet_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/NetworkSiblingSet_Update.json */ /** * Sample code: NetworkFeatures_Update. @@ -1006,7 +603,12 @@ public final class NetAppResourceUpdateNetworkSiblingSetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void networkFeaturesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResources().updateNetworkSiblingSet("eastus", new UpdateNetworkSiblingSetRequest().withNetworkSiblingSetId("9760acf5-4638-11e7-9bdb-020073ca3333").withSubnetId("/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testVnet/subnets/testSubnet").withNetworkSiblingSetStateId("12345_44420.8001578125").withNetworkFeatures(NetworkFeatures.STANDARD), com.azure.core.util.Context.NONE); + manager.netAppResources().updateNetworkSiblingSet("eastus", new UpdateNetworkSiblingSetRequest() + .withNetworkSiblingSetId("9760acf5-4638-11e7-9bdb-020073ca3333") + .withSubnetId( + "/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testVnet/subnets/testSubnet") + .withNetworkSiblingSetStateId("12345_44420.8001578125").withNetworkFeatures(NetworkFeatures.STANDARD), + com.azure.core.util.Context.NONE); } } ``` @@ -1019,7 +621,8 @@ public final class NetAppResourceUpdateNetworkSiblingSetSamples { */ public final class NetAppResourceQuotaLimitsGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/QuotaLimits_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/QuotaLimits_Get.json */ /** * Sample code: QuotaLimits. @@ -1027,7 +630,8 @@ public final class NetAppResourceQuotaLimitsGetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void quotaLimits(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResourceQuotaLimits().getWithResponse("eastus", "totalCoolAccessVolumesPerSubscription", com.azure.core.util.Context.NONE); + manager.netAppResourceQuotaLimits().getWithResponse("eastus", "totalCoolAccessVolumesPerSubscription", + com.azure.core.util.Context.NONE); } } ``` @@ -1040,7 +644,8 @@ public final class NetAppResourceQuotaLimitsGetSamples { */ public final class NetAppResourceQuotaLimitsListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/QuotaLimits_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/QuotaLimits_List.json */ /** * Sample code: QuotaLimits. @@ -1053,48 +658,6 @@ public final class NetAppResourceQuotaLimitsListSamples { } ``` -### NetAppResourceRegionInfos_Get - -```java -/** - * Samples for NetAppResourceRegionInfos Get. - */ -public final class NetAppResourceRegionInfosGetSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/RegionInfos_Get.json - */ - /** - * Sample code: RegionInfos_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void regionInfosGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResourceRegionInfos().getWithResponse("eastus", com.azure.core.util.Context.NONE); - } -} -``` - -### NetAppResourceRegionInfos_List - -```java -/** - * Samples for NetAppResourceRegionInfos List. - */ -public final class NetAppResourceRegionInfosListSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/RegionInfos_List.json - */ - /** - * Sample code: RegionInfos_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void regionInfosList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResourceRegionInfos().list("eastus", com.azure.core.util.Context.NONE); - } -} -``` - ### Operations_List ```java @@ -1103,7 +666,8 @@ public final class NetAppResourceRegionInfosListSamples { */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/OperationList.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/OperationList.json */ /** * Sample code: OperationList. @@ -1121,14 +685,14 @@ public final class OperationsListSamples { ```java import com.azure.resourcemanager.netapp.models.QosType; import com.azure.resourcemanager.netapp.models.ServiceLevel; -import java.util.stream.Collectors; /** * Samples for Pools CreateOrUpdate. */ public final class PoolsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_CreateOrUpdate.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_CreateOrUpdate.json */ /** * Sample code: Pools_CreateOrUpdate. @@ -1136,7 +700,8 @@ public final class PoolsCreateOrUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void poolsCreateOrUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.pools().define("pool1").withRegion("eastus").withExistingNetAppAccount("myRG", "account1").withSize(4398046511104L).withServiceLevel(ServiceLevel.PREMIUM).withQosType(QosType.AUTO).create(); + manager.pools().define("pool1").withRegion("eastus").withExistingNetAppAccount("myRG", "account1") + .withSize(4398046511104L).withServiceLevel(ServiceLevel.PREMIUM).withQosType(QosType.AUTO).create(); } } ``` @@ -1149,7 +714,8 @@ public final class PoolsCreateOrUpdateSamples { */ public final class PoolsDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_Delete.json */ /** * Sample code: Pools_Delete. @@ -1170,7 +736,8 @@ public final class PoolsDeleteSamples { */ public final class PoolsGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_Get.json */ /** * Sample code: Pools_Get. @@ -1191,7 +758,8 @@ public final class PoolsGetSamples { */ public final class PoolsListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_List.json */ /** * Sample code: Pools_List. @@ -1214,7 +782,8 @@ import com.azure.resourcemanager.netapp.models.CapacityPool; */ public final class PoolsUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_Update.json */ /** * Sample code: Pools_Update. @@ -1222,7 +791,8 @@ public final class PoolsUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void poolsUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - CapacityPool resource = manager.pools().getWithResponse("myRG", "account1", "pool1", com.azure.core.util.Context.NONE).getValue(); + CapacityPool resource + = manager.pools().getWithResponse("myRG", "account1", "pool1", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } @@ -1241,7 +811,8 @@ import com.azure.resourcemanager.netapp.models.WeeklySchedule; */ public final class SnapshotPoliciesCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Create.json */ /** * Sample code: SnapshotPolicies_Create. @@ -1249,7 +820,15 @@ public final class SnapshotPoliciesCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotPoliciesCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshotPolicies().define("snapshotPolicyName").withRegion("eastus").withExistingNetAppAccount("myRG", "account1").withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(2).withMinute(50)).withDailySchedule(new DailySchedule().withSnapshotsToKeep(4).withHour(14).withMinute(30)).withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(3).withDay("Wednesday").withHour(14).withMinute(45)).withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(5).withDaysOfMonth("10,11,12").withHour(14).withMinute(15)).withEnabled(true).create(); + manager.snapshotPolicies().define("snapshotPolicyName").withRegion("eastus") + .withExistingNetAppAccount("myRG", "account1") + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(2).withMinute(50)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(4).withHour(14).withMinute(30)) + .withWeeklySchedule( + new WeeklySchedule().withSnapshotsToKeep(3).withDay("Wednesday").withHour(14).withMinute(45)) + .withMonthlySchedule( + new MonthlySchedule().withSnapshotsToKeep(5).withDaysOfMonth("10,11,12").withHour(14).withMinute(15)) + .withEnabled(true).create(); } } ``` @@ -1262,7 +841,8 @@ public final class SnapshotPoliciesCreateSamples { */ public final class SnapshotPoliciesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Delete.json */ /** * Sample code: SnapshotPolicies_Delete. @@ -1270,7 +850,8 @@ public final class SnapshotPoliciesDeleteSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotPoliciesDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshotPolicies().delete("resourceGroup", "accountName", "snapshotPolicyName", com.azure.core.util.Context.NONE); + manager.snapshotPolicies().delete("resourceGroup", "accountName", "snapshotPolicyName", + com.azure.core.util.Context.NONE); } } ``` @@ -1283,7 +864,8 @@ public final class SnapshotPoliciesDeleteSamples { */ public final class SnapshotPoliciesGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Get.json */ /** * Sample code: SnapshotPolicies_Get. @@ -1291,7 +873,8 @@ public final class SnapshotPoliciesGetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotPoliciesGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshotPolicies().getWithResponse("myRG", "account1", "snapshotPolicyName", com.azure.core.util.Context.NONE); + manager.snapshotPolicies().getWithResponse("myRG", "account1", "snapshotPolicyName", + com.azure.core.util.Context.NONE); } } ``` @@ -1304,7 +887,8 @@ public final class SnapshotPoliciesGetSamples { */ public final class SnapshotPoliciesListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_List.json */ /** * Sample code: SnapshotPolicies_List. @@ -1325,7 +909,9 @@ public final class SnapshotPoliciesListSamples { */ public final class SnapshotPoliciesListVolumesSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_ListVolumes.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_ListVolumes. + * json */ /** * Sample code: SnapshotPolicies_ListVolumes. @@ -1333,7 +919,8 @@ public final class SnapshotPoliciesListVolumesSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotPoliciesListVolumes(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshotPolicies().listVolumesWithResponse("myRG", "account1", "snapshotPolicyName", com.azure.core.util.Context.NONE); + manager.snapshotPolicies().listVolumesWithResponse("myRG", "account1", "snapshotPolicyName", + com.azure.core.util.Context.NONE); } } ``` @@ -1352,7 +939,8 @@ import com.azure.resourcemanager.netapp.models.WeeklySchedule; */ public final class SnapshotPoliciesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Update.json */ /** * Sample code: SnapshotPolicies_Update. @@ -1360,8 +948,15 @@ public final class SnapshotPoliciesUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotPoliciesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - SnapshotPolicy resource = manager.snapshotPolicies().getWithResponse("myRG", "account1", "snapshotPolicyName", com.azure.core.util.Context.NONE).getValue(); - resource.update().withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(2).withMinute(50)).withDailySchedule(new DailySchedule().withSnapshotsToKeep(4).withHour(14).withMinute(30)).withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(3).withDay("Wednesday").withHour(14).withMinute(45)).withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(5).withDaysOfMonth("10,11,12").withHour(14).withMinute(15)).withEnabled(true).apply(); + SnapshotPolicy resource = manager.snapshotPolicies() + .getWithResponse("myRG", "account1", "snapshotPolicyName", com.azure.core.util.Context.NONE).getValue(); + resource.update().withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(2).withMinute(50)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(4).withHour(14).withMinute(30)) + .withWeeklySchedule( + new WeeklySchedule().withSnapshotsToKeep(3).withDay("Wednesday").withHour(14).withMinute(45)) + .withMonthlySchedule( + new MonthlySchedule().withSnapshotsToKeep(5).withDaysOfMonth("10,11,12").withHour(14).withMinute(15)) + .withEnabled(true).apply(); } } ``` @@ -1374,7 +969,8 @@ public final class SnapshotPoliciesUpdateSamples { */ public final class SnapshotsCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Create.json */ /** * Sample code: Snapshots_Create. @@ -1382,7 +978,8 @@ public final class SnapshotsCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotsCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshots().define("snapshot1").withRegion("eastus").withExistingVolume("myRG", "account1", "pool1", "volume1").create(); + manager.snapshots().define("snapshot1").withRegion("eastus") + .withExistingVolume("myRG", "account1", "pool1", "volume1").create(); } } ``` @@ -1395,7 +992,8 @@ public final class SnapshotsCreateSamples { */ public final class SnapshotsDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Delete.json */ /** * Sample code: Snapshots_Delete. @@ -1403,7 +1001,8 @@ public final class SnapshotsDeleteSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotsDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshots().delete("myRG", "account1", "pool1", "volume1", "snapshot1", com.azure.core.util.Context.NONE); + manager.snapshots().delete("myRG", "account1", "pool1", "volume1", "snapshot1", + com.azure.core.util.Context.NONE); } } ``` @@ -1416,7 +1015,8 @@ public final class SnapshotsDeleteSamples { */ public final class SnapshotsGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Get.json */ /** * Sample code: Snapshots_Get. @@ -1424,7 +1024,8 @@ public final class SnapshotsGetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotsGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshots().getWithResponse("myRG", "account1", "pool1", "volume1", "snapshot1", com.azure.core.util.Context.NONE); + manager.snapshots().getWithResponse("myRG", "account1", "pool1", "volume1", "snapshot1", + com.azure.core.util.Context.NONE); } } ``` @@ -1437,7 +1038,8 @@ public final class SnapshotsGetSamples { */ public final class SnapshotsListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_List.json */ /** * Sample code: Snapshots_List. @@ -1461,7 +1063,9 @@ import java.util.Arrays; */ public final class SnapshotsRestoreFilesSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_SingleFileRestore.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_SingleFileRestore. + * json */ /** * Sample code: Snapshots_SingleFileRestore. @@ -1469,7 +1073,9 @@ public final class SnapshotsRestoreFilesSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotsSingleFileRestore(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.snapshots().restoreFiles("myRG", "account1", "pool1", "volume1", "snapshot1", new SnapshotRestoreFiles().withFilePaths(Arrays.asList("/dir1/customer1.db", "/dir1/customer2.db")), com.azure.core.util.Context.NONE); + manager.snapshots().restoreFiles("myRG", "account1", "pool1", "volume1", "snapshot1", + new SnapshotRestoreFiles().withFilePaths(Arrays.asList("/dir1/customer1.db", "/dir1/customer2.db")), + com.azure.core.util.Context.NONE); } } ``` @@ -1486,7 +1092,8 @@ import java.io.IOException; */ public final class SnapshotsUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Update.json */ /** * Sample code: Snapshots_Update. @@ -1494,7 +1101,10 @@ public final class SnapshotsUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void snapshotsUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) throws IOException { - manager.snapshots().update("myRG", "account1", "pool1", "volume1", "snapshot1", SerializerFactory.createDefaultManagementSerializerAdapter().deserialize("{}", Object.class, SerializerEncoding.JSON), com.azure.core.util.Context.NONE); + manager.snapshots() + .update("myRG", "account1", "pool1", "volume1", "snapshot1", SerializerFactory + .createDefaultManagementSerializerAdapter().deserialize("{}", Object.class, SerializerEncoding.JSON), + com.azure.core.util.Context.NONE); } } ``` @@ -1507,7 +1117,8 @@ public final class SnapshotsUpdateSamples { */ public final class SubvolumesCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Create.json */ /** * Sample code: Subvolumes_Create. @@ -1515,7 +1126,8 @@ public final class SubvolumesCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void subvolumesCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.subvolumes().define("subvolume1").withExistingVolume("myRG", "account1", "pool1", "volume1").withPath("/subvolumePath").create(); + manager.subvolumes().define("subvolume1").withExistingVolume("myRG", "account1", "pool1", "volume1") + .withPath("/subvolumePath").create(); } } ``` @@ -1528,7 +1140,8 @@ public final class SubvolumesCreateSamples { */ public final class SubvolumesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Delete.json */ /** * Sample code: Subvolumes_Delete. @@ -1536,7 +1149,8 @@ public final class SubvolumesDeleteSamples { * @param manager Entry point to NetAppFilesManager. */ public static void subvolumesDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.subvolumes().delete("myRG", "account1", "pool1", "volume1", "subvolume1", com.azure.core.util.Context.NONE); + manager.subvolumes().delete("myRG", "account1", "pool1", "volume1", "subvolume1", + com.azure.core.util.Context.NONE); } } ``` @@ -1549,7 +1163,8 @@ public final class SubvolumesDeleteSamples { */ public final class SubvolumesGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Get.json */ /** * Sample code: Subvolumes_Get. @@ -1557,7 +1172,8 @@ public final class SubvolumesGetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void subvolumesGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.subvolumes().getWithResponse("myRG", "account1", "pool1", "volume1", "subvolume1", com.azure.core.util.Context.NONE); + manager.subvolumes().getWithResponse("myRG", "account1", "pool1", "volume1", "subvolume1", + com.azure.core.util.Context.NONE); } } ``` @@ -1570,7 +1186,8 @@ public final class SubvolumesGetSamples { */ public final class SubvolumesGetMetadataSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Metadata.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Metadata.json */ /** * Sample code: Subvolumes_Metadata. @@ -1578,7 +1195,8 @@ public final class SubvolumesGetMetadataSamples { * @param manager Entry point to NetAppFilesManager. */ public static void subvolumesMetadata(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.subvolumes().getMetadata("myRG", "account1", "pool1", "volume1", "subvolume1", com.azure.core.util.Context.NONE); + manager.subvolumes().getMetadata("myRG", "account1", "pool1", "volume1", "subvolume1", + com.azure.core.util.Context.NONE); } } ``` @@ -1591,7 +1209,8 @@ public final class SubvolumesGetMetadataSamples { */ public final class SubvolumesListByVolumeSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_List.json */ /** * Sample code: Subvolumes_List. @@ -1614,7 +1233,8 @@ import com.azure.resourcemanager.netapp.models.SubvolumeInfo; */ public final class SubvolumesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Update.json */ /** * Sample code: Subvolumes_Update. @@ -1622,7 +1242,9 @@ public final class SubvolumesUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void subvolumesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - SubvolumeInfo resource = manager.subvolumes().getWithResponse("myRG", "account1", "pool1", "volume1", "subvolume1", com.azure.core.util.Context.NONE).getValue(); + SubvolumeInfo resource = manager.subvolumes() + .getWithResponse("myRG", "account1", "pool1", "volume1", "subvolume1", com.azure.core.util.Context.NONE) + .getValue(); resource.update().withPath("/subvolumePath").apply(); } } @@ -1638,14 +1260,15 @@ import com.azure.resourcemanager.netapp.models.VolumeGroupMetadata; import com.azure.resourcemanager.netapp.models.VolumeGroupVolumeProperties; import com.azure.resourcemanager.netapp.models.VolumePropertiesExportPolicy; import java.util.Arrays; -import java.util.stream.Collectors; /** * Samples for VolumeGroups Create. */ public final class VolumeGroupsCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_Create_SapHana.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Create_SapHana. + * json */ /** * Sample code: VolumeGroups_Create_SapHana. @@ -1653,11 +1276,102 @@ public final class VolumeGroupsCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeGroupsCreateSapHana(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumeGroups().define("group1").withExistingNetAppAccount("myRG", "account1").withRegion("westus").withGroupMetadata(new VolumeGroupMetadata().withGroupDescription("Volume group").withApplicationType(ApplicationType.SAP_HANA).withApplicationIdentifier("SH9")).withVolumes(Arrays.asList(new VolumeGroupVolumeProperties().withName("test-data-mnt00001").withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withProximityPlacementGroup("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg").withVolumeSpecName("data"), new VolumeGroupVolumeProperties().withName("test-log-mnt00001").withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withProximityPlacementGroup("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg").withVolumeSpecName("log"), new VolumeGroupVolumeProperties().withName("test-shared").withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withProximityPlacementGroup("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg").withVolumeSpecName("shared"), new VolumeGroupVolumeProperties().withName("test-data-backup").withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withProximityPlacementGroup("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg").withVolumeSpecName("data-backup"), new VolumeGroupVolumeProperties().withName("test-log-backup").withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withProximityPlacementGroup("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg").withVolumeSpecName("log-backup"))).create(); - } - - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_Create_Oracle.json + manager + .volumeGroups().define("group1").withExistingNetAppAccount("myRG", "account1").withRegion( + "westus") + .withGroupMetadata( + new VolumeGroupMetadata() + .withGroupDescription("Volume group").withApplicationType( + ApplicationType.SAP_HANA) + .withApplicationIdentifier("SH9")) + .withVolumes(Arrays.asList(new VolumeGroupVolumeProperties().withName("test-data-mnt00001") + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withProximityPlacementGroup( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg") + .withVolumeSpecName("data"), + new VolumeGroupVolumeProperties().withName("test-log-mnt00001") + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withProximityPlacementGroup( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg") + .withVolumeSpecName("log"), + new VolumeGroupVolumeProperties().withName("test-shared").withCreationToken("fakeTokenPlaceholder") + .withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withProximityPlacementGroup( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg") + .withVolumeSpecName("shared"), + new VolumeGroupVolumeProperties().withName("test-data-backup").withCreationToken("fakeTokenPlaceholder") + .withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withProximityPlacementGroup( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg") + .withVolumeSpecName("data-backup"), + new VolumeGroupVolumeProperties().withName("test-log-backup").withCreationToken("fakeTokenPlaceholder") + .withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withProximityPlacementGroup( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/cys_sjain_fcp_rg/providers/Microsoft.Compute/proximityPlacementGroups/svlqa_sjain_multivolume_ppg") + .withVolumeSpecName("log-backup"))) + .create(); + } + + /* + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Create_Oracle.json */ /** * Sample code: VolumeGroups_Create_Oracle. @@ -1665,7 +1379,195 @@ public final class VolumeGroupsCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeGroupsCreateOracle(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumeGroups().define("group1").withExistingNetAppAccount("myRG", "account1").withRegion("westus").withGroupMetadata(new VolumeGroupMetadata().withGroupDescription("Volume group").withApplicationType(ApplicationType.ORACLE).withApplicationIdentifier("OR2")).withVolumes(Arrays.asList(new VolumeGroupVolumeProperties().withName("test-ora-data1").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data1"), new VolumeGroupVolumeProperties().withName("test-ora-data2").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data2"), new VolumeGroupVolumeProperties().withName("test-ora-data3").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data3"), new VolumeGroupVolumeProperties().withName("test-ora-data4").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data4"), new VolumeGroupVolumeProperties().withName("test-ora-data5").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data5"), new VolumeGroupVolumeProperties().withName("test-ora-data6").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data6"), new VolumeGroupVolumeProperties().withName("test-ora-data7").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data7"), new VolumeGroupVolumeProperties().withName("test-ora-data8").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-data8"), new VolumeGroupVolumeProperties().withName("test-ora-log").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-log"), new VolumeGroupVolumeProperties().withName("test-ora-log-mirror").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-log-mirror"), new VolumeGroupVolumeProperties().withName("test-ora-binary").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-binary"), new VolumeGroupVolumeProperties().withName("test-ora-backup").withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L).withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))).withProtocolTypes(Arrays.asList("NFSv4.1")).withSubnetId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withThroughputMibps(10.0F).withCapacityPoolResourceId("/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1").withVolumeSpecName("ora-backup"))).create(); + manager + .volumeGroups().define("group1").withExistingNetAppAccount("myRG", "account1").withRegion( + "westus") + .withGroupMetadata( + new VolumeGroupMetadata() + .withGroupDescription("Volume group").withApplicationType( + ApplicationType.ORACLE) + .withApplicationIdentifier("OR2")) + .withVolumes(Arrays.asList(new VolumeGroupVolumeProperties().withName("test-ora-data1") + .withZones(Arrays.asList("1")).withCreationToken("fakeTokenPlaceholder") + .withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data1"), + new VolumeGroupVolumeProperties().withName("test-ora-data2").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data2"), + new VolumeGroupVolumeProperties().withName("test-ora-data3").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data3"), + new VolumeGroupVolumeProperties().withName("test-ora-data4").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data4"), + new VolumeGroupVolumeProperties().withName("test-ora-data5").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data5"), + new VolumeGroupVolumeProperties().withName("test-ora-data6").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data6"), + new VolumeGroupVolumeProperties().withName("test-ora-data7").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data7"), + new VolumeGroupVolumeProperties().withName("test-ora-data8").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-data8"), + new VolumeGroupVolumeProperties().withName("test-ora-log").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-log"), + new VolumeGroupVolumeProperties().withName("test-ora-log-mirror").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-log-mirror"), + new VolumeGroupVolumeProperties().withName("test-ora-binary").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-binary"), + new VolumeGroupVolumeProperties().withName("test-ora-backup").withZones(Arrays.asList("1")) + .withCreationToken("fakeTokenPlaceholder").withServiceLevel(ServiceLevel.PREMIUM) + .withUsageThreshold(107374182400L) + .withExportPolicy(new VolumePropertiesExportPolicy().withRules(Arrays.asList(new ExportPolicyRule() + .withRuleIndex(1).withUnixReadOnly(true).withUnixReadWrite(true).withKerberos5ReadOnly(false) + .withKerberos5ReadWrite(false).withKerberos5IReadOnly(false).withKerberos5IReadWrite(false) + .withKerberos5PReadOnly(false).withKerberos5PReadWrite(false).withCifs(false).withNfsv3(false) + .withNfsv41(true).withAllowedClients("0.0.0.0/0").withHasRootAccess(true)))) + .withProtocolTypes(Arrays.asList("NFSv4.1")) + .withSubnetId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withThroughputMibps(10.0F) + .withCapacityPoolResourceId( + "/subscriptions/d633cc2e-722b-4ae1-b636-bbd9e4c60ed9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1") + .withVolumeSpecName("ora-backup"))) + .create(); } } ``` @@ -1678,7 +1580,8 @@ public final class VolumeGroupsCreateSamples { */ public final class VolumeGroupsDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Delete.json */ /** * Sample code: VolumeGroups_Delete. @@ -1699,7 +1602,8 @@ public final class VolumeGroupsDeleteSamples { */ public final class VolumeGroupsGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_Get_SapHana.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Get_SapHana.json */ /** * Sample code: VolumeGroups_Get_SapHana. @@ -1711,7 +1615,8 @@ public final class VolumeGroupsGetSamples { } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_Get_Oracle.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Get_Oracle.json */ /** * Sample code: VolumeGroups_Get_Oracle. @@ -1732,7 +1637,8 @@ public final class VolumeGroupsGetSamples { */ public final class VolumeGroupsListByNetAppAccountSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_List_Oracle.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_List_Oracle.json */ /** * Sample code: VolumeGroups_List_Oracle. @@ -1744,7 +1650,8 @@ public final class VolumeGroupsListByNetAppAccountSamples { } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_List_SapHana.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_List_SapHana.json */ /** * Sample code: VolumeGroups_List_SapHana. @@ -1761,14 +1668,14 @@ public final class VolumeGroupsListByNetAppAccountSamples { ```java import com.azure.resourcemanager.netapp.models.Type; -import java.util.stream.Collectors; /** * Samples for VolumeQuotaRules Create. */ public final class VolumeQuotaRulesCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Create.json */ /** * Sample code: VolumeQuotaRules_Create. @@ -1776,7 +1683,9 @@ public final class VolumeQuotaRulesCreateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeQuotaRulesCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumeQuotaRules().define("rule-0004").withRegion("westus").withExistingVolume("myRG", "account-9957", "pool-5210", "volume-6387").withQuotaSizeInKiBs(100005L).withQuotaType(Type.INDIVIDUAL_USER_QUOTA).withQuotaTarget("1821").create(); + manager.volumeQuotaRules().define("rule-0004").withRegion("westus") + .withExistingVolume("myRG", "account-9957", "pool-5210", "volume-6387").withQuotaSizeInKiBs(100005L) + .withQuotaType(Type.INDIVIDUAL_USER_QUOTA).withQuotaTarget("1821").create(); } } ``` @@ -1789,7 +1698,8 @@ public final class VolumeQuotaRulesCreateSamples { */ public final class VolumeQuotaRulesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Delete.json */ /** * Sample code: VolumeQuotaRules_Delete. @@ -1797,7 +1707,8 @@ public final class VolumeQuotaRulesDeleteSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeQuotaRulesDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumeQuotaRules().delete("myRG", "account-9957", "pool-5210", "volume-6387", "rule-0004", com.azure.core.util.Context.NONE); + manager.volumeQuotaRules().delete("myRG", "account-9957", "pool-5210", "volume-6387", "rule-0004", + com.azure.core.util.Context.NONE); } } ``` @@ -1810,7 +1721,8 @@ public final class VolumeQuotaRulesDeleteSamples { */ public final class VolumeQuotaRulesGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Get.json */ /** * Sample code: VolumeQuotaRules_Get. @@ -1818,7 +1730,8 @@ public final class VolumeQuotaRulesGetSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeQuotaRulesGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumeQuotaRules().getWithResponse("myRG", "account-9957", "pool-5210", "volume-6387", "rule-0004", com.azure.core.util.Context.NONE); + manager.volumeQuotaRules().getWithResponse("myRG", "account-9957", "pool-5210", "volume-6387", "rule-0004", + com.azure.core.util.Context.NONE); } } ``` @@ -1831,7 +1744,8 @@ public final class VolumeQuotaRulesGetSamples { */ public final class VolumeQuotaRulesListByVolumeSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_List.json */ /** * Sample code: VolumeQuotaRules_List. @@ -1839,7 +1753,8 @@ public final class VolumeQuotaRulesListByVolumeSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeQuotaRulesList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumeQuotaRules().listByVolume("myRG", "account-9957", "pool-5210", "volume-6387", com.azure.core.util.Context.NONE); + manager.volumeQuotaRules().listByVolume("myRG", "account-9957", "pool-5210", "volume-6387", + com.azure.core.util.Context.NONE); } } ``` @@ -1854,7 +1769,8 @@ import com.azure.resourcemanager.netapp.models.VolumeQuotaRule; */ public final class VolumeQuotaRulesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Update.json */ /** * Sample code: VolumeQuotaRules_Update. @@ -1862,7 +1778,8 @@ public final class VolumeQuotaRulesUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumeQuotaRulesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - VolumeQuotaRule resource = manager.volumeQuotaRules().getWithResponse("myRG", "account-9957", "pool-5210", "volume-6387", "rule-0004", com.azure.core.util.Context.NONE).getValue(); + VolumeQuotaRule resource = manager.volumeQuotaRules().getWithResponse("myRG", "account-9957", "pool-5210", + "volume-6387", "rule-0004", com.azure.core.util.Context.NONE).getValue(); resource.update().withQuotaSizeInKiBs(100009L).apply(); } } @@ -1878,7 +1795,9 @@ import com.azure.resourcemanager.netapp.models.AuthorizeRequest; */ public final class VolumesAuthorizeReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_AuthorizeReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_AuthorizeReplication. + * json */ /** * Sample code: Volumes_AuthorizeReplication. @@ -1886,7 +1805,10 @@ public final class VolumesAuthorizeReplicationSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesAuthorizeReplication(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().authorizeReplication("myRG", "account1", "pool1", "volume1", new AuthorizeRequest().withRemoteVolumeResourceId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRemoteRG/providers/Microsoft.NetApp/netAppAccounts/remoteAccount1/capacityPools/remotePool1/volumes/remoteVolume1"), com.azure.core.util.Context.NONE); + manager.volumes().authorizeReplication("myRG", "account1", "pool1", "volume1", + new AuthorizeRequest().withRemoteVolumeResourceId( + "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRemoteRG/providers/Microsoft.NetApp/netAppAccounts/remoteAccount1/capacityPools/remotePool1/volumes/remoteVolume1"), + com.azure.core.util.Context.NONE); } } ``` @@ -1901,7 +1823,8 @@ import com.azure.resourcemanager.netapp.models.BreakFileLocksRequest; */ public final class VolumesBreakFileLocksSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_BreakFileLocks.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_BreakFileLocks.json */ /** * Sample code: Volumes_BreakFileLocks. @@ -1909,7 +1832,9 @@ public final class VolumesBreakFileLocksSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesBreakFileLocks(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().breakFileLocks("myRG", "account1", "pool1", "volume1", new BreakFileLocksRequest().withClientIp("101.102.103.104").withConfirmRunningDisruptiveOperation(true), com.azure.core.util.Context.NONE); + manager.volumes().breakFileLocks("myRG", "account1", "pool1", "volume1", + new BreakFileLocksRequest().withClientIp("101.102.103.104").withConfirmRunningDisruptiveOperation(true), + com.azure.core.util.Context.NONE); } } ``` @@ -1924,7 +1849,8 @@ import com.azure.resourcemanager.netapp.models.BreakReplicationRequest; */ public final class VolumesBreakReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_BreakReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_BreakReplication.json */ /** * Sample code: Volumes_BreakReplication. @@ -1932,7 +1858,8 @@ public final class VolumesBreakReplicationSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesBreakReplication(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().breakReplication("myRG", "account1", "pool1", "volume1", new BreakReplicationRequest().withForceBreakReplication(false), com.azure.core.util.Context.NONE); + manager.volumes().breakReplication("myRG", "account1", "pool1", "volume1", + new BreakReplicationRequest().withForceBreakReplication(false), com.azure.core.util.Context.NONE); } } ``` @@ -1940,16 +1867,15 @@ public final class VolumesBreakReplicationSamples { ### Volumes_CreateOrUpdate ```java -import com.azure.resourcemanager.netapp.models.EncryptionKeySource; import com.azure.resourcemanager.netapp.models.ServiceLevel; -import java.util.stream.Collectors; /** * Samples for Volumes CreateOrUpdate. */ public final class VolumesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_CreateOrUpdate.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_CreateOrUpdate.json */ /** * Sample code: Volumes_CreateOrUpdate. @@ -1957,7 +1883,11 @@ public final class VolumesCreateOrUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesCreateOrUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().define("volume1").withRegion("eastus").withExistingCapacityPool("myRG", "account1", "pool1").withCreationToken("my-unique-file-path").withUsageThreshold(107374182400L).withSubnetId("/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3").withServiceLevel(ServiceLevel.PREMIUM).withThroughputMibps(128.0F).withEncryptionKeySource(EncryptionKeySource.MICROSOFT_KEY_VAULT).create(); + manager.volumes().define("volume1").withRegion("eastus").withExistingCapacityPool("myRG", "account1", "pool1") + .withCreationToken("my-unique-file-path").withUsageThreshold(107374182400L) + .withSubnetId( + "/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") + .withServiceLevel(ServiceLevel.PREMIUM).create(); } } ``` @@ -1970,7 +1900,8 @@ public final class VolumesCreateOrUpdateSamples { */ public final class VolumesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Delete.json */ /** * Sample code: Volumes_Delete. @@ -1991,7 +1922,8 @@ public final class VolumesDeleteSamples { */ public final class VolumesDeleteReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_DeleteReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_DeleteReplication.json */ /** * Sample code: Volumes_DeleteReplication. @@ -2012,7 +1944,8 @@ public final class VolumesDeleteReplicationSamples { */ public final class VolumesFinalizeRelocationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_FinalizeRelocation.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_FinalizeRelocation.json */ /** * Sample code: Volumes_FinalizeRelocation. @@ -2033,7 +1966,8 @@ public final class VolumesFinalizeRelocationSamples { */ public final class VolumesGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Get.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Get.json */ /** * Sample code: Volumes_Get. @@ -2054,7 +1988,8 @@ public final class VolumesGetSamples { */ public final class VolumesListSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_List.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_List.json */ /** * Sample code: Volumes_List. @@ -2077,7 +2012,8 @@ import com.azure.resourcemanager.netapp.models.GetGroupIdListForLdapUserRequest; */ public final class VolumesListGetGroupIdListForLdapUserSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/GroupIdListForLDAPUser.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/GroupIdListForLDAPUser.json */ /** * Sample code: GetGroupIdListForUser. @@ -2085,7 +2021,8 @@ public final class VolumesListGetGroupIdListForLdapUserSamples { * @param manager Entry point to NetAppFilesManager. */ public static void getGroupIdListForUser(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().listGetGroupIdListForLdapUser("myRG", "account1", "pool1", "volume1", new GetGroupIdListForLdapUserRequest().withUsername("user1"), com.azure.core.util.Context.NONE); + manager.volumes().listGetGroupIdListForLdapUser("myRG", "account1", "pool1", "volume1", + new GetGroupIdListForLdapUserRequest().withUsername("user1"), com.azure.core.util.Context.NONE); } } ``` @@ -2098,7 +2035,8 @@ public final class VolumesListGetGroupIdListForLdapUserSamples { */ public final class VolumesListReplicationsSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_ListReplications.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ListReplications.json */ /** * Sample code: Volumes_ListReplications. @@ -2121,7 +2059,8 @@ import com.azure.resourcemanager.netapp.models.PoolChangeRequest; */ public final class VolumesPoolChangeSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_PoolChange.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_PoolChange.json */ /** * Sample code: Volumes_AuthorizeReplication. @@ -2129,7 +2068,10 @@ public final class VolumesPoolChangeSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesAuthorizeReplication(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().poolChange("myRG", "account1", "pool1", "volume1", new PoolChangeRequest().withNewPoolResourceId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1"), com.azure.core.util.Context.NONE); + manager.volumes().poolChange("myRG", "account1", "pool1", "volume1", + new PoolChangeRequest().withNewPoolResourceId( + "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1"), + com.azure.core.util.Context.NONE); } } ``` @@ -2142,7 +2084,8 @@ public final class VolumesPoolChangeSamples { */ public final class VolumesPopulateAvailabilityZoneSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_PopulateAvailabilityZones.json + * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/ + * Volumes_PopulateAvailabilityZones.json */ /** * Sample code: Volumes_PopulateAvailabilityZones. @@ -2150,7 +2093,8 @@ public final class VolumesPopulateAvailabilityZoneSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesPopulateAvailabilityZones(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().populateAvailabilityZone("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE); + manager.volumes().populateAvailabilityZone("myRG", "account1", "pool1", "volume1", + com.azure.core.util.Context.NONE); } } ``` @@ -2163,7 +2107,9 @@ public final class VolumesPopulateAvailabilityZoneSamples { */ public final class VolumesReInitializeReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_ReInitializeReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ReInitializeReplication + * .json */ /** * Sample code: Volumes_ReInitializeReplication. @@ -2171,7 +2117,8 @@ public final class VolumesReInitializeReplicationSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesReInitializeReplication(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().reInitializeReplication("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE); + manager.volumes().reInitializeReplication("myRG", "account1", "pool1", "volume1", + com.azure.core.util.Context.NONE); } } ``` @@ -2186,7 +2133,9 @@ import com.azure.resourcemanager.netapp.models.ReestablishReplicationRequest; */ public final class VolumesReestablishReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_ReestablishReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ReestablishReplication. + * json */ /** * Sample code: Volumes_ReestablishReplication. @@ -2194,7 +2143,10 @@ public final class VolumesReestablishReplicationSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesReestablishReplication(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().reestablishReplication("myRG", "account1", "pool1", "volume1", new ReestablishReplicationRequest().withSourceVolumeId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/mySourceRG/providers/Microsoft.NetApp/netAppAccounts/sourceAccount1/capacityPools/sourcePool1/volumes/sourceVolume1"), com.azure.core.util.Context.NONE); + manager.volumes().reestablishReplication("myRG", "account1", "pool1", "volume1", + new ReestablishReplicationRequest().withSourceVolumeId( + "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/mySourceRG/providers/Microsoft.NetApp/netAppAccounts/sourceAccount1/capacityPools/sourcePool1/volumes/sourceVolume1"), + com.azure.core.util.Context.NONE); } } ``` @@ -2209,7 +2161,8 @@ import com.azure.resourcemanager.netapp.models.RelocateVolumeRequest; */ public final class VolumesRelocateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Relocate.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Relocate.json */ /** * Sample code: Volumes_Relocate. @@ -2217,7 +2170,8 @@ public final class VolumesRelocateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesRelocate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().relocate("myRG", "account1", "pool1", "volume1", new RelocateVolumeRequest(), com.azure.core.util.Context.NONE); + manager.volumes().relocate("myRG", "account1", "pool1", "volume1", new RelocateVolumeRequest(), + com.azure.core.util.Context.NONE); } } ``` @@ -2230,7 +2184,8 @@ public final class VolumesRelocateSamples { */ public final class VolumesReplicationStatusSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_ReplicationStatus.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ReplicationStatus.json */ /** * Sample code: Volumes_ReplicationStatus. @@ -2238,7 +2193,8 @@ public final class VolumesReplicationStatusSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesReplicationStatus(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().replicationStatusWithResponse("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE); + manager.volumes().replicationStatusWithResponse("myRG", "account1", "pool1", "volume1", + com.azure.core.util.Context.NONE); } } ``` @@ -2251,7 +2207,8 @@ public final class VolumesReplicationStatusSamples { */ public final class VolumesResetCifsPasswordSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_ResetCifsPassword.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ResetCifsPassword.json */ /** * Sample code: Volumes_ResetCifsPassword. @@ -2272,7 +2229,8 @@ public final class VolumesResetCifsPasswordSamples { */ public final class VolumesResyncReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_ResyncReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ResyncReplication.json */ /** * Sample code: Volumes_ResyncReplication. @@ -2295,7 +2253,8 @@ import com.azure.resourcemanager.netapp.models.VolumeRevert; */ public final class VolumesRevertSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Revert.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Revert.json */ /** * Sample code: Volumes_Revert. @@ -2303,7 +2262,9 @@ public final class VolumesRevertSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesRevert(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().revert("myRG", "account1", "pool1", "volume1", new VolumeRevert().withSnapshotId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1/volumes/volume1/snapshots/snapshot1"), com.azure.core.util.Context.NONE); + manager.volumes().revert("myRG", "account1", "pool1", "volume1", new VolumeRevert().withSnapshotId( + "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1/volumes/volume1/snapshots/snapshot1"), + com.azure.core.util.Context.NONE); } } ``` @@ -2316,7 +2277,8 @@ public final class VolumesRevertSamples { */ public final class VolumesRevertRelocationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_RevertRelocation.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_RevertRelocation.json */ /** * Sample code: Volumes_RevertRelocation. @@ -2329,40 +2291,18 @@ public final class VolumesRevertRelocationSamples { } ``` -### Volumes_SplitCloneFromParent - -```java -/** - * Samples for Volumes SplitCloneFromParent. - */ -public final class VolumesSplitCloneFromParentSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_SplitClone.json - */ - /** - * Sample code: Volumes_SplitClone. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void volumesSplitClone(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().splitCloneFromParent("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE); - } -} -``` - ### Volumes_Update ```java import com.azure.resourcemanager.netapp.models.Volume; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; -import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesDataProtection; /** * Samples for Volumes Update. */ public final class VolumesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Update.json */ /** * Sample code: Volumes_Update. @@ -2370,8 +2310,9 @@ public final class VolumesUpdateSamples { * @param manager Entry point to NetAppFilesManager. */ public static void volumesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - Volume resource = manager.volumes().getWithResponse("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE).getValue(); - resource.update().withDataProtection(new VolumePatchPropertiesDataProtection().withBackup(new VolumeBackupProperties().withPolicyEnforced(false).withBackupEnabled(true).withBackupVaultId("/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRP/providers/Microsoft.NetApp/netAppAccounts/account1/backupVaults/backupVault1"))).apply(); + Volume resource = manager.volumes() + .getWithResponse("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE).getValue(); + resource.update().apply(); } } ``` diff --git a/sdk/netapp/azure-resourcemanager-netapp/pom.xml b/sdk/netapp/azure-resourcemanager-netapp/pom.xml index 9ed8f9e97f865..b9d81bcdd464e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/pom.xml +++ b/sdk/netapp/azure-resourcemanager-netapp/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-netapp - 1.1.0-beta.2 + 1.2.0-beta.1 jar Microsoft Azure SDK for NetAppFiles Management - This package contains Microsoft Azure SDK for NetAppFiles Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Microsoft NetApp Files Azure Resource Provider specification. Package tag package-preview-2023-05. + This package contains Microsoft Azure SDK for NetAppFiles Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Microsoft NetApp Files Azure Resource Provider specification. Package tag package-netapp-2023-07-01. https://github.com/Azure/azure-sdk-for-java @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -88,18 +88,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +109,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/NetAppFilesManager.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/NetAppFilesManager.java index d8661e69069cb..71c237b9770fc 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/NetAppFilesManager.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/NetAppFilesManager.java @@ -24,17 +24,11 @@ import com.azure.core.util.Configuration; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.netapp.fluent.NetAppManagementClient; -import com.azure.resourcemanager.netapp.implementation.AccountBackupsImpl; import com.azure.resourcemanager.netapp.implementation.AccountsImpl; import com.azure.resourcemanager.netapp.implementation.BackupPoliciesImpl; -import com.azure.resourcemanager.netapp.implementation.BackupVaultsImpl; import com.azure.resourcemanager.netapp.implementation.BackupsImpl; -import com.azure.resourcemanager.netapp.implementation.BackupsUnderAccountsImpl; -import com.azure.resourcemanager.netapp.implementation.BackupsUnderBackupVaultsImpl; -import com.azure.resourcemanager.netapp.implementation.BackupsUnderVolumesImpl; import com.azure.resourcemanager.netapp.implementation.NetAppManagementClientBuilder; import com.azure.resourcemanager.netapp.implementation.NetAppResourceQuotaLimitsImpl; -import com.azure.resourcemanager.netapp.implementation.NetAppResourceRegionInfosImpl; import com.azure.resourcemanager.netapp.implementation.NetAppResourcesImpl; import com.azure.resourcemanager.netapp.implementation.OperationsImpl; import com.azure.resourcemanager.netapp.implementation.PoolsImpl; @@ -44,16 +38,10 @@ import com.azure.resourcemanager.netapp.implementation.VolumeGroupsImpl; import com.azure.resourcemanager.netapp.implementation.VolumeQuotaRulesImpl; import com.azure.resourcemanager.netapp.implementation.VolumesImpl; -import com.azure.resourcemanager.netapp.models.AccountBackups; import com.azure.resourcemanager.netapp.models.Accounts; import com.azure.resourcemanager.netapp.models.BackupPolicies; -import com.azure.resourcemanager.netapp.models.BackupVaults; import com.azure.resourcemanager.netapp.models.Backups; -import com.azure.resourcemanager.netapp.models.BackupsUnderAccounts; -import com.azure.resourcemanager.netapp.models.BackupsUnderBackupVaults; -import com.azure.resourcemanager.netapp.models.BackupsUnderVolumes; import com.azure.resourcemanager.netapp.models.NetAppResourceQuotaLimits; -import com.azure.resourcemanager.netapp.models.NetAppResourceRegionInfos; import com.azure.resourcemanager.netapp.models.NetAppResources; import com.azure.resourcemanager.netapp.models.Operations; import com.azure.resourcemanager.netapp.models.Pools; @@ -81,8 +69,6 @@ public final class NetAppFilesManager { private NetAppResourceQuotaLimits netAppResourceQuotaLimits; - private NetAppResourceRegionInfos netAppResourceRegionInfos; - private Accounts accounts; private Pools pools; @@ -95,8 +81,6 @@ public final class NetAppFilesManager { private Backups backups; - private AccountBackups accountBackups; - private BackupPolicies backupPolicies; private VolumeQuotaRules volumeQuotaRules; @@ -105,14 +89,6 @@ public final class NetAppFilesManager { private Subvolumes subvolumes; - private BackupVaults backupVaults; - - private BackupsUnderBackupVaults backupsUnderBackupVaults; - - private BackupsUnderVolumes backupsUnderVolumes; - - private BackupsUnderAccounts backupsUnderAccounts; - private final NetAppManagementClient clientObject; private NetAppFilesManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { @@ -272,7 +248,7 @@ public NetAppFilesManager authenticate(TokenCredential credential, AzureProfile StringBuilder userAgentBuilder = new StringBuilder(); userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.netapp").append("/") - .append("1.1.0-beta.1"); + .append("1.1.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") @@ -348,19 +324,6 @@ public NetAppResourceQuotaLimits netAppResourceQuotaLimits() { return netAppResourceQuotaLimits; } - /** - * Gets the resource collection API of NetAppResourceRegionInfos. - * - * @return Resource collection API of NetAppResourceRegionInfos. - */ - public NetAppResourceRegionInfos netAppResourceRegionInfos() { - if (this.netAppResourceRegionInfos == null) { - this.netAppResourceRegionInfos - = new NetAppResourceRegionInfosImpl(clientObject.getNetAppResourceRegionInfos(), this); - } - return netAppResourceRegionInfos; - } - /** * Gets the resource collection API of Accounts. It manages NetAppAccount. * @@ -422,7 +385,7 @@ public SnapshotPolicies snapshotPolicies() { } /** - * Gets the resource collection API of Backups. It manages Backup. + * Gets the resource collection API of Backups. * * @return Resource collection API of Backups. */ @@ -433,18 +396,6 @@ public Backups backups() { return backups; } - /** - * Gets the resource collection API of AccountBackups. - * - * @return Resource collection API of AccountBackups. - */ - public AccountBackups accountBackups() { - if (this.accountBackups == null) { - this.accountBackups = new AccountBackupsImpl(clientObject.getAccountBackups(), this); - } - return accountBackups; - } - /** * Gets the resource collection API of BackupPolicies. It manages BackupPolicy. * @@ -493,55 +444,6 @@ public Subvolumes subvolumes() { return subvolumes; } - /** - * Gets the resource collection API of BackupVaults. It manages BackupVault. - * - * @return Resource collection API of BackupVaults. - */ - public BackupVaults backupVaults() { - if (this.backupVaults == null) { - this.backupVaults = new BackupVaultsImpl(clientObject.getBackupVaults(), this); - } - return backupVaults; - } - - /** - * Gets the resource collection API of BackupsUnderBackupVaults. - * - * @return Resource collection API of BackupsUnderBackupVaults. - */ - public BackupsUnderBackupVaults backupsUnderBackupVaults() { - if (this.backupsUnderBackupVaults == null) { - this.backupsUnderBackupVaults - = new BackupsUnderBackupVaultsImpl(clientObject.getBackupsUnderBackupVaults(), this); - } - return backupsUnderBackupVaults; - } - - /** - * Gets the resource collection API of BackupsUnderVolumes. - * - * @return Resource collection API of BackupsUnderVolumes. - */ - public BackupsUnderVolumes backupsUnderVolumes() { - if (this.backupsUnderVolumes == null) { - this.backupsUnderVolumes = new BackupsUnderVolumesImpl(clientObject.getBackupsUnderVolumes(), this); - } - return backupsUnderVolumes; - } - - /** - * Gets the resource collection API of BackupsUnderAccounts. - * - * @return Resource collection API of BackupsUnderAccounts. - */ - public BackupsUnderAccounts backupsUnderAccounts() { - if (this.backupsUnderAccounts == null) { - this.backupsUnderAccounts = new BackupsUnderAccountsImpl(clientObject.getBackupsUnderAccounts(), this); - } - return backupsUnderAccounts; - } - /** * Gets wrapped service client NetAppManagementClient providing direct access to the underlying auto-generated API * implementation, based on Azure REST API. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountsClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountsClient.java index 2e4f838e91408..b82f0b74cb0a9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountsClient.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/AccountsClient.java @@ -12,7 +12,6 @@ import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.netapp.fluent.models.NetAppAccountInner; -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; import com.azure.resourcemanager.netapp.models.NetAppAccountPatch; /** @@ -367,72 +366,4 @@ SyncPoller, Void> beginRenewCredentials(String resourceGroupNam */ @ServiceMethod(returns = ReturnType.SINGLE) void renewCredentials(String resourceGroupName, String accountName, Context context); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginMigrateEncryptionKey(String resourceGroupName, String accountName); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginMigrateEncryptionKey(String resourceGroupName, String accountName, - EncryptionMigrationRequest body, Context context); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void migrateEncryptionKey(String resourceGroupName, String accountName); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void migrateEncryptionKey(String resourceGroupName, String accountName, EncryptionMigrationRequest body, - Context context); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsClient.java index 26588b590f70b..7bc0fc8898036 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsClient.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsClient.java @@ -6,56 +6,14 @@ import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; -import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner; import com.azure.resourcemanager.netapp.fluent.models.RestoreStatusInner; -import com.azure.resourcemanager.netapp.models.BackupPatch; /** * An instance of this class provides access to all the operations defined in BackupsClient. */ public interface BackupsClient { - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getLatestStatusWithResponse(String resourceGroupName, String accountName, - String poolName, String volumeName, Context context); - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BackupStatusInner getLatestStatus(String resourceGroupName, String accountName, String poolName, String volumeName); - /** * Get volume's restore status * @@ -92,300 +50,4 @@ Response getVolumeRestoreStatusWithResponse(String resourceG @ServiceMethod(returns = ReturnType.SINGLE) RestoreStatusInner getVolumeRestoreStatus(String resourceGroupName, String accountName, String poolName, String volumeName); - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName); - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName, - String filter, Context context); - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, - String backupName, Context context); - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BackupInner get(String resourceGroupName, String accountName, String backupVaultName, String backupName); - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupInner> beginCreate(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupInner body); - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupInner> beginCreate(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupInner body, Context context); - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BackupInner create(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupInner body); - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BackupInner create(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupInner body, Context context); - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupInner> beginUpdate(String resourceGroupName, String accountName, - String backupVaultName, String backupName); - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, BackupInner> beginUpdate(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupPatch body, Context context); - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BackupInner update(String resourceGroupName, String accountName, String backupVaultName, String backupName); - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - BackupInner update(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupPatch body, Context context); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, String backupVaultName, - String backupName); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, String backupVaultName, - String backupName, Context context); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName, - Context context); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderAccountsClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderAccountsClient.java deleted file mode 100644 index c6c4122091429..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderAccountsClient.java +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.management.polling.PollResult; -import com.azure.core.util.Context; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; - -/** - * An instance of this class provides access to all the operations defined in BackupsUnderAccountsClient. - */ -public interface BackupsUnderAccountsClient { - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - BackupsMigrationRequest body); - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - BackupsMigrationRequest body, Context context); - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body); - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body, Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderBackupVaultsClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderBackupVaultsClient.java deleted file mode 100644 index 590bf356e98e1..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderBackupVaultsClient.java +++ /dev/null @@ -1,93 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.management.polling.PollResult; -import com.azure.core.util.Context; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.models.BackupRestoreFiles; - -/** - * An instance of this class provides access to all the operations defined in BackupsUnderBackupVaultsClient. - */ -public interface BackupsUnderBackupVaultsClient { - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginRestoreFiles(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body); - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginRestoreFiles(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body, Context context); - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body); - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body, Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderVolumesClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderVolumesClient.java deleted file mode 100644 index 6f8b97e2af644..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/BackupsUnderVolumesClient.java +++ /dev/null @@ -1,93 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.management.polling.PollResult; -import com.azure.core.util.Context; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; - -/** - * An instance of this class provides access to all the operations defined in BackupsUnderVolumesClient. - */ -public interface BackupsUnderVolumesClient { - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - String poolName, String volumeName, BackupsMigrationRequest body); - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - String poolName, String volumeName, BackupsMigrationRequest body, Context context); - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body); - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body, Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppManagementClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppManagementClient.java index a17af7c45614f..340aad54fa128 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppManagementClient.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppManagementClient.java @@ -67,13 +67,6 @@ public interface NetAppManagementClient { */ NetAppResourceQuotaLimitsClient getNetAppResourceQuotaLimits(); - /** - * Gets the NetAppResourceRegionInfosClient object to access its operations. - * - * @return the NetAppResourceRegionInfosClient object. - */ - NetAppResourceRegionInfosClient getNetAppResourceRegionInfos(); - /** * Gets the AccountsClient object to access its operations. * @@ -116,13 +109,6 @@ public interface NetAppManagementClient { */ BackupsClient getBackups(); - /** - * Gets the AccountBackupsClient object to access its operations. - * - * @return the AccountBackupsClient object. - */ - AccountBackupsClient getAccountBackups(); - /** * Gets the BackupPoliciesClient object to access its operations. * @@ -150,32 +136,4 @@ public interface NetAppManagementClient { * @return the SubvolumesClient object. */ SubvolumesClient getSubvolumes(); - - /** - * Gets the BackupVaultsClient object to access its operations. - * - * @return the BackupVaultsClient object. - */ - BackupVaultsClient getBackupVaults(); - - /** - * Gets the BackupsUnderBackupVaultsClient object to access its operations. - * - * @return the BackupsUnderBackupVaultsClient object. - */ - BackupsUnderBackupVaultsClient getBackupsUnderBackupVaults(); - - /** - * Gets the BackupsUnderVolumesClient object to access its operations. - * - * @return the BackupsUnderVolumesClient object. - */ - BackupsUnderVolumesClient getBackupsUnderVolumes(); - - /** - * Gets the BackupsUnderAccountsClient object to access its operations. - * - * @return the BackupsUnderAccountsClient object. - */ - BackupsUnderAccountsClient getBackupsUnderAccounts(); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/VolumesClient.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/VolumesClient.java index 8e462ed9f30d9..b6142c79bf04c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/VolumesClient.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/VolumesClient.java @@ -549,77 +549,6 @@ SyncPoller, Void> beginResetCifsPassword(String resourceGroupNa void resetCifsPassword(String resourceGroupName, String accountName, String poolName, String volumeName, Context context); - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginSplitCloneFromParent(String resourceGroupName, String accountName, - String poolName, String volumeName); - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginSplitCloneFromParent(String resourceGroupName, String accountName, - String poolName, String volumeName, Context context); - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName); - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName, - Context context); - /** * Break file locks * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/AccountProperties.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/AccountProperties.java index 20f8de0f1a7bf..710f5eea907df 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/AccountProperties.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/AccountProperties.java @@ -39,19 +39,6 @@ public final class AccountProperties { @JsonProperty(value = "disableShowmount", access = JsonProperty.Access.WRITE_ONLY) private Boolean disableShowmount; - /* - * Domain for NFSv4 user ID mapping. This property will be set for all NetApp accounts in the subscription and - * region and only affect non ldap NFSv4 volumes. - */ - @JsonProperty(value = "nfsV4IDDomain") - private String nfsV4IdDomain; - - /* - * This will have true value only if account is Multiple AD enabled. - */ - @JsonProperty(value = "isMultiAdEnabled", access = JsonProperty.Access.WRITE_ONLY) - private Boolean isMultiAdEnabled; - /** * Creates an instance of AccountProperties class. */ @@ -117,37 +104,6 @@ public Boolean disableShowmount() { return this.disableShowmount; } - /** - * Get the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @return the nfsV4IdDomain value. - */ - public String nfsV4IdDomain() { - return this.nfsV4IdDomain; - } - - /** - * Set the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @param nfsV4IdDomain the nfsV4IdDomain value to set. - * @return the AccountProperties object itself. - */ - public AccountProperties withNfsV4IdDomain(String nfsV4IdDomain) { - this.nfsV4IdDomain = nfsV4IdDomain; - return this; - } - - /** - * Get the isMultiAdEnabled property: This will have true value only if account is Multiple AD enabled. - * - * @return the isMultiAdEnabled value. - */ - public Boolean isMultiAdEnabled() { - return this.isMultiAdEnabled; - } - /** * Validates the instance. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupInner.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupInner.java deleted file mode 100644 index cf88de99c3998..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupInner.java +++ /dev/null @@ -1,234 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.models.BackupType; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; - -/** - * Backup under a Backup Vault. - */ -@Fluent -public final class BackupInner extends ProxyResource { - /* - * Backup Properties - */ - @JsonProperty(value = "properties", required = true) - private BackupProperties innerProperties = new BackupProperties(); - - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. - */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** - * Creates an instance of BackupInner class. - */ - public BackupInner() { - } - - /** - * Get the innerProperties property: Backup Properties. - * - * @return the innerProperties value. - */ - private BackupProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - - /** - * Get the backupId property: backupId - * - * UUID v4 used to identify the Backup. - * - * @return the backupId value. - */ - public String backupId() { - return this.innerProperties() == null ? null : this.innerProperties().backupId(); - } - - /** - * Get the creationDate property: creationDate - * - * The creation date of the backup. - * - * @return the creationDate value. - */ - public OffsetDateTime creationDate() { - return this.innerProperties() == null ? null : this.innerProperties().creationDate(); - } - - /** - * Get the provisioningState property: Azure lifecycle management. - * - * @return the provisioningState value. - */ - public String provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); - } - - /** - * Get the size property: Size of backup in bytes. - * - * @return the size value. - */ - public Long size() { - return this.innerProperties() == null ? null : this.innerProperties().size(); - } - - /** - * Get the label property: Label for backup. - * - * @return the label value. - */ - public String label() { - return this.innerProperties() == null ? null : this.innerProperties().label(); - } - - /** - * Set the label property: Label for backup. - * - * @param label the label value to set. - * @return the BackupInner object itself. - */ - public BackupInner withLabel(String label) { - if (this.innerProperties() == null) { - this.innerProperties = new BackupProperties(); - } - this.innerProperties().withLabel(label); - return this; - } - - /** - * Get the backupType property: backupType - * - * Type of backup Manual or Scheduled. - * - * @return the backupType value. - */ - public BackupType backupType() { - return this.innerProperties() == null ? null : this.innerProperties().backupType(); - } - - /** - * Get the failureReason property: Failure reason. - * - * @return the failureReason value. - */ - public String failureReason() { - return this.innerProperties() == null ? null : this.innerProperties().failureReason(); - } - - /** - * Get the volumeResourceId property: ResourceId used to identify the Volume. - * - * @return the volumeResourceId value. - */ - public String volumeResourceId() { - return this.innerProperties() == null ? null : this.innerProperties().volumeResourceId(); - } - - /** - * Set the volumeResourceId property: ResourceId used to identify the Volume. - * - * @param volumeResourceId the volumeResourceId value to set. - * @return the BackupInner object itself. - */ - public BackupInner withVolumeResourceId(String volumeResourceId) { - if (this.innerProperties() == null) { - this.innerProperties = new BackupProperties(); - } - this.innerProperties().withVolumeResourceId(volumeResourceId); - return this; - } - - /** - * Get the useExistingSnapshot property: Manual backup an already existing snapshot. This will always be false for - * scheduled backups and true/false for manual backups. - * - * @return the useExistingSnapshot value. - */ - public Boolean useExistingSnapshot() { - return this.innerProperties() == null ? null : this.innerProperties().useExistingSnapshot(); - } - - /** - * Set the useExistingSnapshot property: Manual backup an already existing snapshot. This will always be false for - * scheduled backups and true/false for manual backups. - * - * @param useExistingSnapshot the useExistingSnapshot value to set. - * @return the BackupInner object itself. - */ - public BackupInner withUseExistingSnapshot(Boolean useExistingSnapshot) { - if (this.innerProperties() == null) { - this.innerProperties = new BackupProperties(); - } - this.innerProperties().withUseExistingSnapshot(useExistingSnapshot); - return this; - } - - /** - * Get the snapshotName property: The name of the snapshot. - * - * @return the snapshotName value. - */ - public String snapshotName() { - return this.innerProperties() == null ? null : this.innerProperties().snapshotName(); - } - - /** - * Set the snapshotName property: The name of the snapshot. - * - * @param snapshotName the snapshotName value to set. - * @return the BackupInner object itself. - */ - public BackupInner withSnapshotName(String snapshotName) { - if (this.innerProperties() == null) { - this.innerProperties = new BackupProperties(); - } - this.innerProperties().withSnapshotName(snapshotName); - return this; - } - - /** - * Get the backupPolicyResourceId property: ResourceId used to identify the backup policy. - * - * @return the backupPolicyResourceId value. - */ - public String backupPolicyResourceId() { - return this.innerProperties() == null ? null : this.innerProperties().backupPolicyResourceId(); - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model BackupInner")); - } else { - innerProperties().validate(); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(BackupInner.class); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupPatchProperties.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupPatchProperties.java deleted file mode 100644 index c05f14b104ba0..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupPatchProperties.java +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Backup patch properties. - */ -@Fluent -public final class BackupPatchProperties { - /* - * Label for backup - */ - @JsonProperty(value = "label") - private String label; - - /** - * Creates an instance of BackupPatchProperties class. - */ - public BackupPatchProperties() { - } - - /** - * Get the label property: Label for backup. - * - * @return the label value. - */ - public String label() { - return this.label; - } - - /** - * Set the label property: Label for backup. - * - * @param label the label value to set. - * @return the BackupPatchProperties object itself. - */ - public BackupPatchProperties withLabel(String label) { - this.label = label; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupProperties.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupProperties.java deleted file mode 100644 index b776f2fd80099..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupProperties.java +++ /dev/null @@ -1,261 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.models.BackupType; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; - -/** - * Backup properties. - */ -@Fluent -public final class BackupProperties { - /* - * backupId - * - * UUID v4 used to identify the Backup - */ - @JsonProperty(value = "backupId", access = JsonProperty.Access.WRITE_ONLY) - private String backupId; - - /* - * creationDate - * - * The creation date of the backup - */ - @JsonProperty(value = "creationDate", access = JsonProperty.Access.WRITE_ONLY) - private OffsetDateTime creationDate; - - /* - * Azure lifecycle management - */ - @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) - private String provisioningState; - - /* - * Size of backup in bytes - */ - @JsonProperty(value = "size", access = JsonProperty.Access.WRITE_ONLY) - private Long size; - - /* - * Label for backup - */ - @JsonProperty(value = "label") - private String label; - - /* - * backupType - * - * Type of backup Manual or Scheduled - */ - @JsonProperty(value = "backupType", access = JsonProperty.Access.WRITE_ONLY) - private BackupType backupType; - - /* - * Failure reason - */ - @JsonProperty(value = "failureReason", access = JsonProperty.Access.WRITE_ONLY) - private String failureReason; - - /* - * ResourceId used to identify the Volume - */ - @JsonProperty(value = "volumeResourceId", required = true) - private String volumeResourceId; - - /* - * Manual backup an already existing snapshot. This will always be false for scheduled backups and true/false for - * manual backups - */ - @JsonProperty(value = "useExistingSnapshot") - private Boolean useExistingSnapshot; - - /* - * The name of the snapshot - */ - @JsonProperty(value = "snapshotName") - private String snapshotName; - - /* - * ResourceId used to identify the backup policy - */ - @JsonProperty(value = "backupPolicyResourceId", access = JsonProperty.Access.WRITE_ONLY) - private String backupPolicyResourceId; - - /** - * Creates an instance of BackupProperties class. - */ - public BackupProperties() { - } - - /** - * Get the backupId property: backupId - * - * UUID v4 used to identify the Backup. - * - * @return the backupId value. - */ - public String backupId() { - return this.backupId; - } - - /** - * Get the creationDate property: creationDate - * - * The creation date of the backup. - * - * @return the creationDate value. - */ - public OffsetDateTime creationDate() { - return this.creationDate; - } - - /** - * Get the provisioningState property: Azure lifecycle management. - * - * @return the provisioningState value. - */ - public String provisioningState() { - return this.provisioningState; - } - - /** - * Get the size property: Size of backup in bytes. - * - * @return the size value. - */ - public Long size() { - return this.size; - } - - /** - * Get the label property: Label for backup. - * - * @return the label value. - */ - public String label() { - return this.label; - } - - /** - * Set the label property: Label for backup. - * - * @param label the label value to set. - * @return the BackupProperties object itself. - */ - public BackupProperties withLabel(String label) { - this.label = label; - return this; - } - - /** - * Get the backupType property: backupType - * - * Type of backup Manual or Scheduled. - * - * @return the backupType value. - */ - public BackupType backupType() { - return this.backupType; - } - - /** - * Get the failureReason property: Failure reason. - * - * @return the failureReason value. - */ - public String failureReason() { - return this.failureReason; - } - - /** - * Get the volumeResourceId property: ResourceId used to identify the Volume. - * - * @return the volumeResourceId value. - */ - public String volumeResourceId() { - return this.volumeResourceId; - } - - /** - * Set the volumeResourceId property: ResourceId used to identify the Volume. - * - * @param volumeResourceId the volumeResourceId value to set. - * @return the BackupProperties object itself. - */ - public BackupProperties withVolumeResourceId(String volumeResourceId) { - this.volumeResourceId = volumeResourceId; - return this; - } - - /** - * Get the useExistingSnapshot property: Manual backup an already existing snapshot. This will always be false for - * scheduled backups and true/false for manual backups. - * - * @return the useExistingSnapshot value. - */ - public Boolean useExistingSnapshot() { - return this.useExistingSnapshot; - } - - /** - * Set the useExistingSnapshot property: Manual backup an already existing snapshot. This will always be false for - * scheduled backups and true/false for manual backups. - * - * @param useExistingSnapshot the useExistingSnapshot value to set. - * @return the BackupProperties object itself. - */ - public BackupProperties withUseExistingSnapshot(Boolean useExistingSnapshot) { - this.useExistingSnapshot = useExistingSnapshot; - return this; - } - - /** - * Get the snapshotName property: The name of the snapshot. - * - * @return the snapshotName value. - */ - public String snapshotName() { - return this.snapshotName; - } - - /** - * Set the snapshotName property: The name of the snapshot. - * - * @param snapshotName the snapshotName value to set. - * @return the BackupProperties object itself. - */ - public BackupProperties withSnapshotName(String snapshotName) { - this.snapshotName = snapshotName; - return this; - } - - /** - * Get the backupPolicyResourceId property: ResourceId used to identify the backup policy. - * - * @return the backupPolicyResourceId value. - */ - public String backupPolicyResourceId() { - return this.backupPolicyResourceId; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (volumeResourceId() == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Missing required property volumeResourceId in model BackupProperties")); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(BackupProperties.class); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupStatusInner.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupStatusInner.java deleted file mode 100644 index d98bfd3cfdbc4..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupStatusInner.java +++ /dev/null @@ -1,166 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Immutable; -import com.azure.resourcemanager.netapp.models.MirrorState; -import com.azure.resourcemanager.netapp.models.RelationshipStatus; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Backup status. - */ -@Immutable -public final class BackupStatusInner { - /* - * Backup health status - */ - @JsonProperty(value = "healthy", access = JsonProperty.Access.WRITE_ONLY) - private Boolean healthy; - - /* - * Status of the backup mirror relationship - */ - @JsonProperty(value = "relationshipStatus", access = JsonProperty.Access.WRITE_ONLY) - private RelationshipStatus relationshipStatus; - - /* - * The status of the backup - */ - @JsonProperty(value = "mirrorState", access = JsonProperty.Access.WRITE_ONLY) - private MirrorState mirrorState; - - /* - * Reason for the unhealthy backup relationship - */ - @JsonProperty(value = "unhealthyReason", access = JsonProperty.Access.WRITE_ONLY) - private String unhealthyReason; - - /* - * Displays error message if the backup is in an error state - */ - @JsonProperty(value = "errorMessage", access = JsonProperty.Access.WRITE_ONLY) - private String errorMessage; - - /* - * Displays the last transfer size - */ - @JsonProperty(value = "lastTransferSize", access = JsonProperty.Access.WRITE_ONLY) - private Long lastTransferSize; - - /* - * Displays the last transfer type - */ - @JsonProperty(value = "lastTransferType", access = JsonProperty.Access.WRITE_ONLY) - private String lastTransferType; - - /* - * Displays the total bytes transferred - */ - @JsonProperty(value = "totalTransferBytes", access = JsonProperty.Access.WRITE_ONLY) - private Long totalTransferBytes; - - /* - * Displays the total number of bytes transferred for the ongoing operation - */ - @JsonProperty(value = "transferProgressBytes", access = JsonProperty.Access.WRITE_ONLY) - private Long transferProgressBytes; - - /** - * Creates an instance of BackupStatusInner class. - */ - public BackupStatusInner() { - } - - /** - * Get the healthy property: Backup health status. - * - * @return the healthy value. - */ - public Boolean healthy() { - return this.healthy; - } - - /** - * Get the relationshipStatus property: Status of the backup mirror relationship. - * - * @return the relationshipStatus value. - */ - public RelationshipStatus relationshipStatus() { - return this.relationshipStatus; - } - - /** - * Get the mirrorState property: The status of the backup. - * - * @return the mirrorState value. - */ - public MirrorState mirrorState() { - return this.mirrorState; - } - - /** - * Get the unhealthyReason property: Reason for the unhealthy backup relationship. - * - * @return the unhealthyReason value. - */ - public String unhealthyReason() { - return this.unhealthyReason; - } - - /** - * Get the errorMessage property: Displays error message if the backup is in an error state. - * - * @return the errorMessage value. - */ - public String errorMessage() { - return this.errorMessage; - } - - /** - * Get the lastTransferSize property: Displays the last transfer size. - * - * @return the lastTransferSize value. - */ - public Long lastTransferSize() { - return this.lastTransferSize; - } - - /** - * Get the lastTransferType property: Displays the last transfer type. - * - * @return the lastTransferType value. - */ - public String lastTransferType() { - return this.lastTransferType; - } - - /** - * Get the totalTransferBytes property: Displays the total bytes transferred. - * - * @return the totalTransferBytes value. - */ - public Long totalTransferBytes() { - return this.totalTransferBytes; - } - - /** - * Get the transferProgressBytes property: Displays the total number of bytes transferred for the ongoing - * operation. - * - * @return the transferProgressBytes value. - */ - public Long transferProgressBytes() { - return this.transferProgressBytes; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultInner.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultInner.java deleted file mode 100644 index c8c572398b54e..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultInner.java +++ /dev/null @@ -1,91 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.management.SystemData; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Map; - -/** - * Backup Vault information. - */ -@Fluent -public final class BackupVaultInner extends Resource { - /* - * Backup Vault Properties - */ - @JsonProperty(value = "properties") - private BackupVaultProperties innerProperties; - - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. - */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** - * Creates an instance of BackupVaultInner class. - */ - public BackupVaultInner() { - } - - /** - * Get the innerProperties property: Backup Vault Properties. - * - * @return the innerProperties value. - */ - private BackupVaultProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - - /** - * {@inheritDoc} - */ - @Override - public BackupVaultInner withLocation(String location) { - super.withLocation(location); - return this; - } - - /** - * {@inheritDoc} - */ - @Override - public BackupVaultInner withTags(Map tags) { - super.withTags(tags); - return this; - } - - /** - * Get the provisioningState property: Azure lifecycle management. - * - * @return the provisioningState value. - */ - public String provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultProperties.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultProperties.java deleted file mode 100644 index c7c4463e55202..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/BackupVaultProperties.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Immutable; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Backup Vault properties. - */ -@Immutable -public final class BackupVaultProperties { - /* - * Azure lifecycle management - */ - @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) - private String provisioningState; - - /** - * Creates an instance of BackupVaultProperties class. - */ - public BackupVaultProperties() { - } - - /** - * Get the provisioningState property: Azure lifecycle management. - * - * @return the provisioningState value. - */ - public String provisioningState() { - return this.provisioningState; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/NetAppAccountInner.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/NetAppAccountInner.java index df8933d9f35d1..d9137b3416705 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/NetAppAccountInner.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/NetAppAccountInner.java @@ -179,40 +179,6 @@ public Boolean disableShowmount() { return this.innerProperties() == null ? null : this.innerProperties().disableShowmount(); } - /** - * Get the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @return the nfsV4IdDomain value. - */ - public String nfsV4IdDomain() { - return this.innerProperties() == null ? null : this.innerProperties().nfsV4IdDomain(); - } - - /** - * Set the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @param nfsV4IdDomain the nfsV4IdDomain value to set. - * @return the NetAppAccountInner object itself. - */ - public NetAppAccountInner withNfsV4IdDomain(String nfsV4IdDomain) { - if (this.innerProperties() == null) { - this.innerProperties = new AccountProperties(); - } - this.innerProperties().withNfsV4IdDomain(nfsV4IdDomain); - return this; - } - - /** - * Get the isMultiAdEnabled property: This will have true value only if account is Multiple AD enabled. - * - * @return the isMultiAdEnabled value. - */ - public Boolean isMultiAdEnabled() { - return this.innerProperties() == null ? null : this.innerProperties().isMultiAdEnabled(); - } - /** * Validates the instance. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/RegionInfoResourceInner.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/RegionInfoResourceInner.java deleted file mode 100644 index 2f692008c6fe0..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/RegionInfoResourceInner.java +++ /dev/null @@ -1,122 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; -import com.azure.resourcemanager.netapp.models.RegionInfoAvailabilityZoneMappingsItem; -import com.azure.resourcemanager.netapp.models.RegionStorageToNetworkProximity; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** - * Information regarding regionInfo Item. - */ -@Fluent -public final class RegionInfoResourceInner extends ProxyResource { - /* - * regionInfo properties - */ - @JsonProperty(value = "properties") - private RegionInfoInner innerProperties; - - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. - */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** - * Creates an instance of RegionInfoResourceInner class. - */ - public RegionInfoResourceInner() { - } - - /** - * Get the innerProperties property: regionInfo properties. - * - * @return the innerProperties value. - */ - private RegionInfoInner innerProperties() { - return this.innerProperties; - } - - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - - /** - * Get the storageToNetworkProximity property: Storage to Network Proximity - * - * Provides storage to network proximity information in the region. - * - * @return the storageToNetworkProximity value. - */ - public RegionStorageToNetworkProximity storageToNetworkProximity() { - return this.innerProperties() == null ? null : this.innerProperties().storageToNetworkProximity(); - } - - /** - * Set the storageToNetworkProximity property: Storage to Network Proximity - * - * Provides storage to network proximity information in the region. - * - * @param storageToNetworkProximity the storageToNetworkProximity value to set. - * @return the RegionInfoResourceInner object itself. - */ - public RegionInfoResourceInner - withStorageToNetworkProximity(RegionStorageToNetworkProximity storageToNetworkProximity) { - if (this.innerProperties() == null) { - this.innerProperties = new RegionInfoInner(); - } - this.innerProperties().withStorageToNetworkProximity(storageToNetworkProximity); - return this; - } - - /** - * Get the availabilityZoneMappings property: Logical availability zone mappings. - * - * Provides logical availability zone mappings for the subscription for a region. - * - * @return the availabilityZoneMappings value. - */ - public List availabilityZoneMappings() { - return this.innerProperties() == null ? null : this.innerProperties().availabilityZoneMappings(); - } - - /** - * Set the availabilityZoneMappings property: Logical availability zone mappings. - * - * Provides logical availability zone mappings for the subscription for a region. - * - * @param availabilityZoneMappings the availabilityZoneMappings value to set. - * @return the RegionInfoResourceInner object itself. - */ - public RegionInfoResourceInner - withAvailabilityZoneMappings(List availabilityZoneMappings) { - if (this.innerProperties() == null) { - this.innerProperties = new RegionInfoInner(); - } - this.innerProperties().withAvailabilityZoneMappings(availabilityZoneMappings); - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeInner.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeInner.java index 47e28ca0212e8..fc995c0e48270 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeInner.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeInner.java @@ -462,7 +462,7 @@ public List mountTargets() { /** * Get the volumeType property: What type of volume is this. For destination volumes in Cross Region Replication, - * set type to DataProtection. For creating clone volume, set type to ShortTermClone. + * set type to DataProtection. * * @return the volumeType value. */ @@ -472,7 +472,7 @@ public String volumeType() { /** * Set the volumeType property: What type of volume is this. For destination volumes in Cross Region Replication, - * set type to DataProtection. For creating clone volume, set type to ShortTermClone. + * set type to DataProtection. * * @param volumeType the volumeType value to set. * @return the VolumeInner object itself. @@ -1281,17 +1281,6 @@ public String originatingResourceId() { return this.innerProperties() == null ? null : this.innerProperties().originatingResourceId(); } - /** - * Get the inheritedSizeInBytes property: inheritedSizeInBytes - * - * Space shared by short term clone volume with parent volume in bytes. - * - * @return the inheritedSizeInBytes value. - */ - public Long inheritedSizeInBytes() { - return this.innerProperties() == null ? null : this.innerProperties().inheritedSizeInBytes(); - } - /** * Validates the instance. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeProperties.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeProperties.java index 7378b6a435f42..2d78dc5274e54 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeProperties.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/models/VolumeProperties.java @@ -154,8 +154,7 @@ public final class VolumeProperties { private List mountTargets; /* - * What type of volume is this. For destination volumes in Cross Region Replication, set type to DataProtection. - * For creating clone volume, set type to ShortTermClone + * What type of volume is this. For destination volumes in Cross Region Replication, set type to DataProtection */ @JsonProperty(value = "volumeType") private String volumeType; @@ -420,14 +419,6 @@ public final class VolumeProperties { @JsonProperty(value = "originatingResourceId", access = JsonProperty.Access.WRITE_ONLY) private String originatingResourceId; - /* - * inheritedSizeInBytes - * - * Space shared by short term clone volume with parent volume in bytes. - */ - @JsonProperty(value = "inheritedSizeInBytes", access = JsonProperty.Access.WRITE_ONLY) - private Long inheritedSizeInBytes; - /** * Creates an instance of VolumeProperties class. */ @@ -740,7 +731,7 @@ public List mountTargets() { /** * Get the volumeType property: What type of volume is this. For destination volumes in Cross Region Replication, - * set type to DataProtection. For creating clone volume, set type to ShortTermClone. + * set type to DataProtection. * * @return the volumeType value. */ @@ -750,7 +741,7 @@ public String volumeType() { /** * Set the volumeType property: What type of volume is this. For destination volumes in Cross Region Replication, - * set type to DataProtection. For creating clone volume, set type to ShortTermClone. + * set type to DataProtection. * * @param volumeType the volumeType value to set. * @return the VolumeProperties object itself. @@ -1475,17 +1466,6 @@ public String originatingResourceId() { return this.originatingResourceId; } - /** - * Get the inheritedSizeInBytes property: inheritedSizeInBytes - * - * Space shared by short term clone volume with parent volume in bytes. - * - * @return the inheritedSizeInBytes value. - */ - public Long inheritedSizeInBytes() { - return this.inheritedSizeInBytes; - } - /** * Validates the instance. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsImpl.java deleted file mode 100644 index f6579d1ea8ef6..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountBackupsImpl.java +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.fluent.AccountBackupsClient; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.models.AccountBackups; -import com.azure.resourcemanager.netapp.models.Backup; - -public final class AccountBackupsImpl implements AccountBackups { - private static final ClientLogger LOGGER = new ClientLogger(AccountBackupsImpl.class); - - private final AccountBackupsClient innerClient; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public AccountBackupsImpl(AccountBackupsClient innerClient, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName) { - PagedIterable inner = this.serviceClient().listByNetAppAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new BackupImpl(inner1, this.manager())); - } - - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, - String includeOnlyBackupsFromDeletedVolumes, Context context) { - PagedIterable inner = this.serviceClient().listByNetAppAccount(resourceGroupName, accountName, - includeOnlyBackupsFromDeletedVolumes, context); - return Utils.mapPage(inner, inner1 -> new BackupImpl(inner1, this.manager())); - } - - public Response getWithResponse(String resourceGroupName, String accountName, String backupName, - Context context) { - Response inner - = this.serviceClient().getWithResponse(resourceGroupName, accountName, backupName, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new BackupImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public Backup get(String resourceGroupName, String accountName, String backupName) { - BackupInner inner = this.serviceClient().get(resourceGroupName, accountName, backupName); - if (inner != null) { - return new BackupImpl(inner, this.manager()); - } else { - return null; - } - } - - public void delete(String resourceGroupName, String accountName, String backupName) { - this.serviceClient().delete(resourceGroupName, accountName, backupName); - } - - public void delete(String resourceGroupName, String accountName, String backupName, Context context) { - this.serviceClient().delete(resourceGroupName, accountName, backupName, context); - } - - private AccountBackupsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsClientImpl.java index d99d162dc5f7a..57e5d6d92363f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsClientImpl.java @@ -35,7 +35,6 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.netapp.fluent.AccountsClient; import com.azure.resourcemanager.netapp.fluent.models.NetAppAccountInner; -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; import com.azure.resourcemanager.netapp.models.NetAppAccountList; import com.azure.resourcemanager.netapp.models.NetAppAccountPatch; import java.nio.ByteBuffer; @@ -109,14 +108,14 @@ Mono>> createOrUpdate(@HostParam("$host") String endpo @QueryParam("api-version") String apiVersion, @BodyParam("application/json") NetAppAccountInner body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}") @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @QueryParam("api-version") String apiVersion, Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}") @@ -128,25 +127,14 @@ Mono>> update(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") NetAppAccountPatch body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/renewCredentials") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> renewCredentials(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @QueryParam("api-version") String apiVersion, Context context); - - @Headers({ "Content-Type: application/json" }) - @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/migrateEncryption") - @ExpectedResponses({ 202 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> migrateEncryptionKey(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") EncryptionMigrationRequest body, @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") @@ -833,9 +821,10 @@ private Mono>> deleteWithResponseAsync(String resource if (accountName == null) { return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -870,9 +859,10 @@ private Mono>> deleteWithResponseAsync(String resource if (accountName == null) { return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, this.client.getApiVersion(), context); + accountName, this.client.getApiVersion(), accept, context); } /** @@ -1300,9 +1290,10 @@ private Mono>> renewCredentialsWithResponseAsync(Strin if (accountName == null) { return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.renewCredentials(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1338,9 +1329,10 @@ private Mono>> renewCredentialsWithResponseAsync(Strin if (accountName == null) { return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.renewCredentials(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, this.client.getApiVersion(), context); + accountName, this.client.getApiVersion(), accept, context); } /** @@ -1500,303 +1492,6 @@ public void renewCredentials(String resourceGroupName, String accountName, Conte renewCredentialsAsync(resourceGroupName, accountName, context).block(); } - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> migrateEncryptionKeyWithResponseAsync(String resourceGroupName, - String accountName, EncryptionMigrationRequest body) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (body != null) { - body.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.migrateEncryptionKey(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, this.client.getApiVersion(), body, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> migrateEncryptionKeyWithResponseAsync(String resourceGroupName, - String accountName, EncryptionMigrationRequest body, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (body != null) { - body.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.migrateEncryptionKey(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, this.client.getApiVersion(), body, accept, context); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateEncryptionKeyAsync(String resourceGroupName, - String accountName, EncryptionMigrationRequest body) { - Mono>> mono - = migrateEncryptionKeyWithResponseAsync(resourceGroupName, accountName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateEncryptionKeyAsync(String resourceGroupName, - String accountName) { - final EncryptionMigrationRequest body = null; - Mono>> mono - = migrateEncryptionKeyWithResponseAsync(resourceGroupName, accountName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateEncryptionKeyAsync(String resourceGroupName, - String accountName, EncryptionMigrationRequest body, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = migrateEncryptionKeyWithResponseAsync(resourceGroupName, accountName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - context); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginMigrateEncryptionKey(String resourceGroupName, String accountName) { - final EncryptionMigrationRequest body = null; - return this.beginMigrateEncryptionKeyAsync(resourceGroupName, accountName, body).getSyncPoller(); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginMigrateEncryptionKey(String resourceGroupName, String accountName, - EncryptionMigrationRequest body, Context context) { - return this.beginMigrateEncryptionKeyAsync(resourceGroupName, accountName, body, context).getSyncPoller(); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateEncryptionKeyAsync(String resourceGroupName, String accountName, - EncryptionMigrationRequest body) { - return beginMigrateEncryptionKeyAsync(resourceGroupName, accountName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateEncryptionKeyAsync(String resourceGroupName, String accountName) { - final EncryptionMigrationRequest body = null; - return beginMigrateEncryptionKeyAsync(resourceGroupName, accountName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateEncryptionKeyAsync(String resourceGroupName, String accountName, - EncryptionMigrationRequest body, Context context) { - return beginMigrateEncryptionKeyAsync(resourceGroupName, accountName, body, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void migrateEncryptionKey(String resourceGroupName, String accountName) { - final EncryptionMigrationRequest body = null; - migrateEncryptionKeyAsync(resourceGroupName, accountName, body).block(); - } - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void migrateEncryptionKey(String resourceGroupName, String accountName, EncryptionMigrationRequest body, - Context context) { - migrateEncryptionKeyAsync(resourceGroupName, accountName, body, context).block(); - } - /** * Get the next page of items. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsImpl.java index 66a3e73bff4e6..15aebb81243ba 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/AccountsImpl.java @@ -12,7 +12,6 @@ import com.azure.resourcemanager.netapp.fluent.AccountsClient; import com.azure.resourcemanager.netapp.fluent.models.NetAppAccountInner; import com.azure.resourcemanager.netapp.models.Accounts; -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; import com.azure.resourcemanager.netapp.models.NetAppAccount; public final class AccountsImpl implements Accounts { @@ -30,22 +29,22 @@ public AccountsImpl(AccountsClient innerClient, public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NetAppAccountImpl(inner1, this.manager())); } public Response getByResourceGroupWithResponse(String resourceGroupName, String accountName, @@ -85,22 +84,13 @@ public void renewCredentials(String resourceGroupName, String accountName, Conte this.serviceClient().renewCredentials(resourceGroupName, accountName, context); } - public void migrateEncryptionKey(String resourceGroupName, String accountName) { - this.serviceClient().migrateEncryptionKey(resourceGroupName, accountName); - } - - public void migrateEncryptionKey(String resourceGroupName, String accountName, EncryptionMigrationRequest body, - Context context) { - this.serviceClient().migrateEncryptionKey(resourceGroupName, accountName, body, context); - } - public NetAppAccount getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); @@ -109,12 +99,12 @@ public NetAppAccount getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); @@ -123,12 +113,12 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); @@ -137,12 +127,12 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupImpl.java deleted file mode 100644 index 806ad2f4575a5..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupImpl.java +++ /dev/null @@ -1,193 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.models.Backup; -import com.azure.resourcemanager.netapp.models.BackupPatch; -import com.azure.resourcemanager.netapp.models.BackupType; -import java.time.OffsetDateTime; - -public final class BackupImpl implements Backup, Backup.Definition, Backup.Update { - private BackupInner innerObject; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public SystemData systemData() { - return this.innerModel().systemData(); - } - - public String backupId() { - return this.innerModel().backupId(); - } - - public OffsetDateTime creationDate() { - return this.innerModel().creationDate(); - } - - public String provisioningState() { - return this.innerModel().provisioningState(); - } - - public Long size() { - return this.innerModel().size(); - } - - public String label() { - return this.innerModel().label(); - } - - public BackupType backupType() { - return this.innerModel().backupType(); - } - - public String failureReason() { - return this.innerModel().failureReason(); - } - - public String volumeResourceId() { - return this.innerModel().volumeResourceId(); - } - - public Boolean useExistingSnapshot() { - return this.innerModel().useExistingSnapshot(); - } - - public String snapshotName() { - return this.innerModel().snapshotName(); - } - - public String backupPolicyResourceId() { - return this.innerModel().backupPolicyResourceId(); - } - - public String resourceGroupName() { - return resourceGroupName; - } - - public BackupInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } - - private String resourceGroupName; - - private String accountName; - - private String backupVaultName; - - private String backupName; - - private BackupPatch updateBody; - - public BackupImpl withExistingBackupVault(String resourceGroupName, String accountName, String backupVaultName) { - this.resourceGroupName = resourceGroupName; - this.accountName = accountName; - this.backupVaultName = backupVaultName; - return this; - } - - public Backup create() { - this.innerObject = serviceManager.serviceClient().getBackups().create(resourceGroupName, accountName, - backupVaultName, backupName, this.innerModel(), Context.NONE); - return this; - } - - public Backup create(Context context) { - this.innerObject = serviceManager.serviceClient().getBackups().create(resourceGroupName, accountName, - backupVaultName, backupName, this.innerModel(), context); - return this; - } - - BackupImpl(String name, com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerObject = new BackupInner(); - this.serviceManager = serviceManager; - this.backupName = name; - } - - public BackupImpl update() { - this.updateBody = new BackupPatch(); - return this; - } - - public Backup apply() { - this.innerObject = serviceManager.serviceClient().getBackups().update(resourceGroupName, accountName, - backupVaultName, backupName, updateBody, Context.NONE); - return this; - } - - public Backup apply(Context context) { - this.innerObject = serviceManager.serviceClient().getBackups().update(resourceGroupName, accountName, - backupVaultName, backupName, updateBody, context); - return this; - } - - BackupImpl(BackupInner innerObject, com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.backupVaultName = Utils.getValueFromIdByName(innerObject.id(), "backupVaults"); - this.backupName = Utils.getValueFromIdByName(innerObject.id(), "backups"); - } - - public Backup refresh() { - this.innerObject = serviceManager.serviceClient().getBackups() - .getWithResponse(resourceGroupName, accountName, backupVaultName, backupName, Context.NONE).getValue(); - return this; - } - - public Backup refresh(Context context) { - this.innerObject = serviceManager.serviceClient().getBackups() - .getWithResponse(resourceGroupName, accountName, backupVaultName, backupName, context).getValue(); - return this; - } - - public BackupImpl withVolumeResourceId(String volumeResourceId) { - this.innerModel().withVolumeResourceId(volumeResourceId); - return this; - } - - public BackupImpl withLabel(String label) { - if (isInCreateMode()) { - this.innerModel().withLabel(label); - return this; - } else { - this.updateBody.withLabel(label); - return this; - } - } - - public BackupImpl withUseExistingSnapshot(Boolean useExistingSnapshot) { - this.innerModel().withUseExistingSnapshot(useExistingSnapshot); - return this; - } - - public BackupImpl withSnapshotName(String snapshotName) { - this.innerModel().withSnapshotName(snapshotName); - return this; - } - - private boolean isInCreateMode() { - return this.innerModel().id() == null; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesClientImpl.java index 3c2efd1350e43..18528a6e502f0 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesClientImpl.java @@ -113,7 +113,7 @@ Mono>> update(@HostParam("$host") String endpoint, @BodyParam("application/json") BackupPolicyPatch body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupPolicies/{backupPolicyName}") @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -121,7 +121,7 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("backupPolicyName") String backupPolicyName, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** @@ -986,9 +986,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter backupPolicyName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupPolicyName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, backupPolicyName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1028,9 +1029,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter backupPolicyName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupPolicyName, this.client.getApiVersion(), context); + accountName, backupPolicyName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesImpl.java index 8143a6a6edac3..2c6a5a82263c4 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPoliciesImpl.java @@ -29,12 +29,12 @@ public BackupPoliciesImpl(BackupPoliciesClient innerClient, public PagedIterable list(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new BackupPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BackupPolicyImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new BackupPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new BackupPolicyImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, String backupPolicyName, @@ -67,17 +67,17 @@ public void delete(String resourceGroupName, String accountName, String backupPo } public BackupPolicy getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String backupPolicyName = Utils.getValueFromIdByName(id, "backupPolicies"); + String backupPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "backupPolicies"); if (backupPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'backupPolicies'.", id))); @@ -86,17 +86,17 @@ public BackupPolicy getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String backupPolicyName = Utils.getValueFromIdByName(id, "backupPolicies"); + String backupPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "backupPolicies"); if (backupPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'backupPolicies'.", id))); @@ -105,17 +105,17 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String backupPolicyName = Utils.getValueFromIdByName(id, "backupPolicies"); + String backupPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "backupPolicies"); if (backupPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'backupPolicies'.", id))); @@ -124,17 +124,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String backupPolicyName = Utils.getValueFromIdByName(id, "backupPolicies"); + String backupPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "backupPolicies"); if (backupPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'backupPolicies'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPolicyImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPolicyImpl.java index 31dbaf43fedce..d73207fe46165 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPolicyImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupPolicyImpl.java @@ -163,9 +163,9 @@ public BackupPolicy apply(Context context) { com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.backupPolicyName = Utils.getValueFromIdByName(innerObject.id(), "backupPolicies"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.backupPolicyName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "backupPolicies"); } public BackupPolicy refresh() { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupStatusImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupStatusImpl.java deleted file mode 100644 index df32a32dcbb76..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupStatusImpl.java +++ /dev/null @@ -1,66 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner; -import com.azure.resourcemanager.netapp.models.BackupStatus; -import com.azure.resourcemanager.netapp.models.MirrorState; -import com.azure.resourcemanager.netapp.models.RelationshipStatus; - -public final class BackupStatusImpl implements BackupStatus { - private BackupStatusInner innerObject; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - BackupStatusImpl(BackupStatusInner innerObject, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public Boolean healthy() { - return this.innerModel().healthy(); - } - - public RelationshipStatus relationshipStatus() { - return this.innerModel().relationshipStatus(); - } - - public MirrorState mirrorState() { - return this.innerModel().mirrorState(); - } - - public String unhealthyReason() { - return this.innerModel().unhealthyReason(); - } - - public String errorMessage() { - return this.innerModel().errorMessage(); - } - - public Long lastTransferSize() { - return this.innerModel().lastTransferSize(); - } - - public String lastTransferType() { - return this.innerModel().lastTransferType(); - } - - public Long totalTransferBytes() { - return this.innerModel().totalTransferBytes(); - } - - public Long transferProgressBytes() { - return this.innerModel().transferProgressBytes(); - } - - public BackupStatusInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultImpl.java deleted file mode 100644 index 4908561aadee6..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultImpl.java +++ /dev/null @@ -1,166 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.management.Region; -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import com.azure.resourcemanager.netapp.models.BackupVault; -import com.azure.resourcemanager.netapp.models.BackupVaultPatch; -import java.util.Collections; -import java.util.Map; - -public final class BackupVaultImpl implements BackupVault, BackupVault.Definition, BackupVault.Update { - private BackupVaultInner innerObject; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public String location() { - return this.innerModel().location(); - } - - public Map tags() { - Map inner = this.innerModel().tags(); - if (inner != null) { - return Collections.unmodifiableMap(inner); - } else { - return Collections.emptyMap(); - } - } - - public SystemData systemData() { - return this.innerModel().systemData(); - } - - public String provisioningState() { - return this.innerModel().provisioningState(); - } - - public Region region() { - return Region.fromName(this.regionName()); - } - - public String regionName() { - return this.location(); - } - - public String resourceGroupName() { - return resourceGroupName; - } - - public BackupVaultInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } - - private String resourceGroupName; - - private String accountName; - - private String backupVaultName; - - private BackupVaultPatch updateBody; - - public BackupVaultImpl withExistingNetAppAccount(String resourceGroupName, String accountName) { - this.resourceGroupName = resourceGroupName; - this.accountName = accountName; - return this; - } - - public BackupVault create() { - this.innerObject = serviceManager.serviceClient().getBackupVaults().createOrUpdate(resourceGroupName, - accountName, backupVaultName, this.innerModel(), Context.NONE); - return this; - } - - public BackupVault create(Context context) { - this.innerObject = serviceManager.serviceClient().getBackupVaults().createOrUpdate(resourceGroupName, - accountName, backupVaultName, this.innerModel(), context); - return this; - } - - BackupVaultImpl(String name, com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerObject = new BackupVaultInner(); - this.serviceManager = serviceManager; - this.backupVaultName = name; - } - - public BackupVaultImpl update() { - this.updateBody = new BackupVaultPatch(); - return this; - } - - public BackupVault apply() { - this.innerObject = serviceManager.serviceClient().getBackupVaults().update(resourceGroupName, accountName, - backupVaultName, updateBody, Context.NONE); - return this; - } - - public BackupVault apply(Context context) { - this.innerObject = serviceManager.serviceClient().getBackupVaults().update(resourceGroupName, accountName, - backupVaultName, updateBody, context); - return this; - } - - BackupVaultImpl(BackupVaultInner innerObject, com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.backupVaultName = Utils.getValueFromIdByName(innerObject.id(), "backupVaults"); - } - - public BackupVault refresh() { - this.innerObject = serviceManager.serviceClient().getBackupVaults() - .getWithResponse(resourceGroupName, accountName, backupVaultName, Context.NONE).getValue(); - return this; - } - - public BackupVault refresh(Context context) { - this.innerObject = serviceManager.serviceClient().getBackupVaults() - .getWithResponse(resourceGroupName, accountName, backupVaultName, context).getValue(); - return this; - } - - public BackupVaultImpl withRegion(Region location) { - this.innerModel().withLocation(location.toString()); - return this; - } - - public BackupVaultImpl withRegion(String location) { - this.innerModel().withLocation(location); - return this; - } - - public BackupVaultImpl withTags(Map tags) { - if (isInCreateMode()) { - this.innerModel().withTags(tags); - return this; - } else { - this.updateBody.withTags(tags); - return this; - } - } - - private boolean isInCreateMode() { - return this.innerModel().id() == null; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsImpl.java deleted file mode 100644 index a011175dfb83a..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupVaultsImpl.java +++ /dev/null @@ -1,159 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.fluent.BackupVaultsClient; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import com.azure.resourcemanager.netapp.models.BackupVault; -import com.azure.resourcemanager.netapp.models.BackupVaults; - -public final class BackupVaultsImpl implements BackupVaults { - private static final ClientLogger LOGGER = new ClientLogger(BackupVaultsImpl.class); - - private final BackupVaultsClient innerClient; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public BackupVaultsImpl(BackupVaultsClient innerClient, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName) { - PagedIterable inner - = this.serviceClient().listByNetAppAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new BackupVaultImpl(inner1, this.manager())); - } - - public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, - Context context) { - PagedIterable inner - = this.serviceClient().listByNetAppAccount(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new BackupVaultImpl(inner1, this.manager())); - } - - public Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, - Context context) { - Response inner - = this.serviceClient().getWithResponse(resourceGroupName, accountName, backupVaultName, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new BackupVaultImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public BackupVault get(String resourceGroupName, String accountName, String backupVaultName) { - BackupVaultInner inner = this.serviceClient().get(resourceGroupName, accountName, backupVaultName); - if (inner != null) { - return new BackupVaultImpl(inner, this.manager()); - } else { - return null; - } - } - - public void delete(String resourceGroupName, String accountName, String backupVaultName) { - this.serviceClient().delete(resourceGroupName, accountName, backupVaultName); - } - - public void delete(String resourceGroupName, String accountName, String backupVaultName, Context context) { - this.serviceClient().delete(resourceGroupName, accountName, backupVaultName, context); - } - - public BackupVault getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - return this.getWithResponse(resourceGroupName, accountName, backupVaultName, Context.NONE).getValue(); - } - - public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - return this.getWithResponse(resourceGroupName, accountName, backupVaultName, context); - } - - public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - this.delete(resourceGroupName, accountName, backupVaultName, Context.NONE); - } - - public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - this.delete(resourceGroupName, accountName, backupVaultName, context); - } - - private BackupVaultsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } - - public BackupVaultImpl define(String name) { - return new BackupVaultImpl(name, this.manager()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsClientImpl.java index 0be2250addd3e..1a4b628275469 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsClientImpl.java @@ -4,42 +4,25 @@ package com.azure.resourcemanager.netapp.implementation; -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.Delete; import com.azure.core.annotation.ExpectedResponses; import com.azure.core.annotation.Get; import com.azure.core.annotation.HeaderParam; import com.azure.core.annotation.Headers; import com.azure.core.annotation.Host; import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.Patch; import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Put; import com.azure.core.annotation.QueryParam; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceInterface; import com.azure.core.annotation.ServiceMethod; import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; import com.azure.core.http.rest.Response; import com.azure.core.http.rest.RestProxy; import com.azure.core.management.exception.ManagementException; -import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.FluxUtil; -import com.azure.core.util.polling.PollerFlux; -import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.netapp.fluent.BackupsClient; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner; import com.azure.resourcemanager.netapp.fluent.models.RestoreStatusInner; -import com.azure.resourcemanager.netapp.models.BackupPatch; -import com.azure.resourcemanager.netapp.models.BackupsList; -import java.nio.ByteBuffer; -import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; /** @@ -73,16 +56,6 @@ public final class BackupsClientImpl implements BackupsClient { @Host("{$host}") @ServiceInterface(name = "NetAppManagementClie") public interface BackupsService { - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/latestBackupStatus/current") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getLatestStatus(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/restoreStatus") @ExpectedResponses({ 200 }) @@ -92,220 +65,6 @@ Mono> getVolumeRestoreStatus(@HostParam("$host") St @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}/backups") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByVault(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @QueryParam("$filter") String filter, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}/backups/{backupName}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @PathParam("backupName") String backupName, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}/backups/{backupName}") - @ExpectedResponses({ 200, 201 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> create(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @PathParam("backupName") String backupName, - @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BackupInner body, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}/backups/{backupName}") - @ExpectedResponses({ 200, 202 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @PathParam("backupName") String backupName, - @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BackupPatch body, - @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}/backups/{backupName}") - @ExpectedResponses({ 202, 204 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @PathParam("backupName") String backupName, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("{nextLink}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByVaultNext(@PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - } - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getLatestStatusWithResponseAsync(String resourceGroupName, - String accountName, String poolName, String volumeName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (poolName == null) { - return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); - } - if (volumeName == null) { - return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.getLatestStatus(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getLatestStatusWithResponseAsync(String resourceGroupName, - String accountName, String poolName, String volumeName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (poolName == null) { - return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); - } - if (volumeName == null) { - return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.getLatestStatus(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); - } - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getLatestStatusAsync(String resourceGroupName, String accountName, String poolName, - String volumeName) { - return getLatestStatusWithResponseAsync(resourceGroupName, accountName, poolName, volumeName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getLatestStatusWithResponse(String resourceGroupName, String accountName, - String poolName, String volumeName, Context context) { - return getLatestStatusWithResponseAsync(resourceGroupName, accountName, poolName, volumeName, context).block(); - } - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BackupStatusInner getLatestStatus(String resourceGroupName, String accountName, String poolName, - String volumeName) { - return getLatestStatusWithResponse(resourceGroupName, accountName, poolName, volumeName, Context.NONE) - .getValue(); } /** @@ -464,1299 +223,4 @@ public RestoreStatusInner getVolumeRestoreStatus(String resourceGroupName, Strin return getVolumeRestoreStatusWithResponse(resourceGroupName, accountName, poolName, volumeName, Context.NONE) .getValue(); } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByVaultSinglePageAsync(String resourceGroupName, String accountName, - String backupVaultName, String filter) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listByVault(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupVaultName, filter, this.client.getApiVersion(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), - res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByVaultSinglePageAsync(String resourceGroupName, String accountName, - String backupVaultName, String filter, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listByVault(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, accountName, - backupVaultName, filter, this.client.getApiVersion(), accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByVaultAsync(String resourceGroupName, String accountName, - String backupVaultName, String filter) { - return new PagedFlux<>( - () -> listByVaultSinglePageAsync(resourceGroupName, accountName, backupVaultName, filter), - nextLink -> listByVaultNextSinglePageAsync(nextLink)); - } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByVaultAsync(String resourceGroupName, String accountName, - String backupVaultName) { - final String filter = null; - return new PagedFlux<>( - () -> listByVaultSinglePageAsync(resourceGroupName, accountName, backupVaultName, filter), - nextLink -> listByVaultNextSinglePageAsync(nextLink)); - } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByVaultAsync(String resourceGroupName, String accountName, - String backupVaultName, String filter, Context context) { - return new PagedFlux<>( - () -> listByVaultSinglePageAsync(resourceGroupName, accountName, backupVaultName, filter, context), - nextLink -> listByVaultNextSinglePageAsync(nextLink, context)); - } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByVault(String resourceGroupName, String accountName, - String backupVaultName) { - final String filter = null; - return new PagedIterable<>(listByVaultAsync(resourceGroupName, accountName, backupVaultName, filter)); - } - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName, - String filter, Context context) { - return new PagedIterable<>(listByVaultAsync(resourceGroupName, accountName, backupVaultName, filter, context)); - } - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, accountName, - backupVaultName, backupName, this.client.getApiVersion(), accept, context); - } - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName) { - return getWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, - String backupName, Context context) { - return getWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, context).block(); - } - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BackupInner get(String resourceGroupName, String accountName, String backupVaultName, String backupName) { - return getWithResponse(resourceGroupName, accountName, backupVaultName, backupName, Context.NONE).getValue(); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupInner body) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.create(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), body, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupInner body, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.create(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), body, accept, context); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupInner> beginCreateAsync(String resourceGroupName, - String accountName, String backupVaultName, String backupName, BackupInner body) { - Mono>> mono - = createWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupInner.class, BackupInner.class, this.client.getContext()); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupInner> beginCreateAsync(String resourceGroupName, - String accountName, String backupVaultName, String backupName, BackupInner body, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = createWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupInner.class, BackupInner.class, context); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupInner> beginCreate(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupInner body) { - return this.beginCreateAsync(resourceGroupName, accountName, backupVaultName, backupName, body).getSyncPoller(); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupInner> beginCreate(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupInner body, Context context) { - return this.beginCreateAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context) - .getSyncPoller(); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, BackupInner body) { - return beginCreateAsync(resourceGroupName, accountName, backupVaultName, backupName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, BackupInner body, Context context) { - return beginCreateAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BackupInner create(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupInner body) { - return createAsync(resourceGroupName, accountName, backupVaultName, backupName, body).block(); - } - - /** - * Create a backup - * - * Create a backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BackupInner create(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupInner body, Context context) { - return createAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context).block(); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupPatch body) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - if (body != null) { - body.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), body, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupPatch body, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - if (body != null) { - body.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), body, accept, context); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupInner> beginUpdateAsync(String resourceGroupName, - String accountName, String backupVaultName, String backupName, BackupPatch body) { - Mono>> mono - = updateWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupInner.class, BackupInner.class, this.client.getContext()); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupInner> beginUpdateAsync(String resourceGroupName, - String accountName, String backupVaultName, String backupName) { - final BackupPatch body = null; - Mono>> mono - = updateWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupInner.class, BackupInner.class, this.client.getContext()); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, BackupInner> beginUpdateAsync(String resourceGroupName, - String accountName, String backupVaultName, String backupName, BackupPatch body, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = updateWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), - BackupInner.class, BackupInner.class, context); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupInner> beginUpdate(String resourceGroupName, String accountName, - String backupVaultName, String backupName) { - final BackupPatch body = null; - return this.beginUpdateAsync(resourceGroupName, accountName, backupVaultName, backupName, body).getSyncPoller(); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, BackupInner> beginUpdate(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupPatch body, Context context) { - return this.beginUpdateAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context) - .getSyncPoller(); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, BackupPatch body) { - return beginUpdateAsync(resourceGroupName, accountName, backupVaultName, backupName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName) { - final BackupPatch body = null; - return beginUpdateAsync(resourceGroupName, accountName, backupVaultName, backupName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, BackupPatch body, Context context) { - return beginUpdateAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BackupInner update(String resourceGroupName, String accountName, String backupVaultName, String backupName) { - final BackupPatch body = null; - return updateAsync(resourceGroupName, accountName, backupVaultName, backupName, body).block(); - } - - /** - * Patch a backup - * - * Patch a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Backup object supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return backup under a Backup Vault. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public BackupInner update(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupPatch body, Context context) { - return updateAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context).block(); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), accept, context); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName) { - Mono>> mono - = deleteWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = deleteWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - context); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupVaultName, String backupName) { - return this.beginDeleteAsync(resourceGroupName, accountName, backupVaultName, backupName).getSyncPoller(); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String accountName, - String backupVaultName, String backupName, Context context) { - return this.beginDeleteAsync(resourceGroupName, accountName, backupVaultName, backupName, context) - .getSyncPoller(); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName) { - return beginDeleteAsync(resourceGroupName, accountName, backupVaultName, backupName).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, Context context) { - return beginDeleteAsync(resourceGroupName, accountName, backupVaultName, backupName, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName) { - deleteAsync(resourceGroupName, accountName, backupVaultName, backupName).block(); - } - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName, - Context context) { - deleteAsync(resourceGroupName, accountName, backupVaultName, backupName, context).block(); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByVaultNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listByVaultNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), - res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByVaultNextSinglePageAsync(String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.listByVaultNext(nextLink, this.client.getEndpoint(), accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsImpl.java index 62a87611d80e7..de38c88687adb 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsImpl.java @@ -4,17 +4,12 @@ package com.azure.resourcemanager.netapp.implementation; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.http.rest.SimpleResponse; import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.netapp.fluent.BackupsClient; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner; import com.azure.resourcemanager.netapp.fluent.models.RestoreStatusInner; -import com.azure.resourcemanager.netapp.models.Backup; -import com.azure.resourcemanager.netapp.models.BackupStatus; import com.azure.resourcemanager.netapp.models.Backups; import com.azure.resourcemanager.netapp.models.RestoreStatus; @@ -30,29 +25,6 @@ public BackupsImpl(BackupsClient innerClient, com.azure.resourcemanager.netapp.N this.serviceManager = serviceManager; } - public Response getLatestStatusWithResponse(String resourceGroupName, String accountName, - String poolName, String volumeName, Context context) { - Response inner = this.serviceClient().getLatestStatusWithResponse(resourceGroupName, - accountName, poolName, volumeName, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new BackupStatusImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public BackupStatus getLatestStatus(String resourceGroupName, String accountName, String poolName, - String volumeName) { - BackupStatusInner inner - = this.serviceClient().getLatestStatus(resourceGroupName, accountName, poolName, volumeName); - if (inner != null) { - return new BackupStatusImpl(inner, this.manager()); - } else { - return null; - } - } - public Response getVolumeRestoreStatusWithResponse(String resourceGroupName, String accountName, String poolName, String volumeName, Context context) { Response inner = this.serviceClient().getVolumeRestoreStatusWithResponse(resourceGroupName, @@ -76,146 +48,6 @@ public RestoreStatus getVolumeRestoreStatus(String resourceGroupName, String acc } } - public PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName) { - PagedIterable inner - = this.serviceClient().listByVault(resourceGroupName, accountName, backupVaultName); - return Utils.mapPage(inner, inner1 -> new BackupImpl(inner1, this.manager())); - } - - public PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName, - String filter, Context context) { - PagedIterable inner - = this.serviceClient().listByVault(resourceGroupName, accountName, backupVaultName, filter, context); - return Utils.mapPage(inner, inner1 -> new BackupImpl(inner1, this.manager())); - } - - public Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, - String backupName, Context context) { - Response inner = this.serviceClient().getWithResponse(resourceGroupName, accountName, - backupVaultName, backupName, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new BackupImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public Backup get(String resourceGroupName, String accountName, String backupVaultName, String backupName) { - BackupInner inner = this.serviceClient().get(resourceGroupName, accountName, backupVaultName, backupName); - if (inner != null) { - return new BackupImpl(inner, this.manager()); - } else { - return null; - } - } - - public void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName) { - this.serviceClient().delete(resourceGroupName, accountName, backupVaultName, backupName); - } - - public void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName, - Context context) { - this.serviceClient().delete(resourceGroupName, accountName, backupVaultName, backupName, context); - } - - public Backup getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - String backupName = Utils.getValueFromIdByName(id, "backups"); - if (backupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backups'.", id))); - } - return this.getWithResponse(resourceGroupName, accountName, backupVaultName, backupName, Context.NONE) - .getValue(); - } - - public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - String backupName = Utils.getValueFromIdByName(id, "backups"); - if (backupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backups'.", id))); - } - return this.getWithResponse(resourceGroupName, accountName, backupVaultName, backupName, context); - } - - public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - String backupName = Utils.getValueFromIdByName(id, "backups"); - if (backupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backups'.", id))); - } - this.delete(resourceGroupName, accountName, backupVaultName, backupName, Context.NONE); - } - - public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); - if (accountName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); - } - String backupVaultName = Utils.getValueFromIdByName(id, "backupVaults"); - if (backupVaultName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backupVaults'.", id))); - } - String backupName = Utils.getValueFromIdByName(id, "backups"); - if (backupName == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'backups'.", id))); - } - this.delete(resourceGroupName, accountName, backupVaultName, backupName, context); - } - private BackupsClient serviceClient() { return this.innerClient; } @@ -223,8 +55,4 @@ private BackupsClient serviceClient() { private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { return this.serviceManager; } - - public BackupImpl define(String name) { - return new BackupImpl(name, this.manager()); - } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsClientImpl.java deleted file mode 100644 index e9012e2f9aa64..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsClientImpl.java +++ /dev/null @@ -1,322 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Post; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.management.polling.PollResult; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.core.util.polling.PollerFlux; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderAccountsClient; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; -import java.nio.ByteBuffer; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -/** - * An instance of this class provides access to all the operations defined in BackupsUnderAccountsClient. - */ -public final class BackupsUnderAccountsClientImpl implements BackupsUnderAccountsClient { - /** - * The proxy service used to perform REST calls. - */ - private final BackupsUnderAccountsService service; - - /** - * The service client containing this operation class. - */ - private final NetAppManagementClientImpl client; - - /** - * Initializes an instance of BackupsUnderAccountsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - BackupsUnderAccountsClientImpl(NetAppManagementClientImpl client) { - this.service = RestProxy.create(BackupsUnderAccountsService.class, client.getHttpPipeline(), - client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for NetAppManagementClientBackupsUnderAccounts to be used by the proxy - * service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "NetAppManagementClie") - public interface BackupsUnderAccountsService { - @Headers({ "Content-Type: application/json" }) - @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/migrateBackups") - @ExpectedResponses({ 202 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> migrateBackups(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BackupsMigrationRequest body, - @HeaderParam("Accept") String accept, Context context); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> migrateBackupsWithResponseAsync(String resourceGroupName, - String accountName, BackupsMigrationRequest body) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.migrateBackups(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, this.client.getApiVersion(), body, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> migrateBackupsWithResponseAsync(String resourceGroupName, - String accountName, BackupsMigrationRequest body, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.migrateBackups(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, this.client.getApiVersion(), body, accept, context); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateBackupsAsync(String resourceGroupName, String accountName, - BackupsMigrationRequest body) { - Mono>> mono = migrateBackupsWithResponseAsync(resourceGroupName, accountName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateBackupsAsync(String resourceGroupName, String accountName, - BackupsMigrationRequest body, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = migrateBackupsWithResponseAsync(resourceGroupName, accountName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - context); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - BackupsMigrationRequest body) { - return this.beginMigrateBackupsAsync(resourceGroupName, accountName, body).getSyncPoller(); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - BackupsMigrationRequest body, Context context) { - return this.beginMigrateBackupsAsync(resourceGroupName, accountName, body, context).getSyncPoller(); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateBackupsAsync(String resourceGroupName, String accountName, BackupsMigrationRequest body) { - return beginMigrateBackupsAsync(resourceGroupName, accountName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateBackupsAsync(String resourceGroupName, String accountName, BackupsMigrationRequest body, - Context context) { - return beginMigrateBackupsAsync(resourceGroupName, accountName, body, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body) { - migrateBackupsAsync(resourceGroupName, accountName, body).block(); - } - - /** - * Create a new migrate request for backups under account - * - * Migrate the backups under a NetApp account to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body Migrate backups under an account payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body, - Context context) { - migrateBackupsAsync(resourceGroupName, accountName, body, context).block(); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsImpl.java deleted file mode 100644 index 573f5a53ff736..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderAccountsImpl.java +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderAccountsClient; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; -import com.azure.resourcemanager.netapp.models.BackupsUnderAccounts; - -public final class BackupsUnderAccountsImpl implements BackupsUnderAccounts { - private static final ClientLogger LOGGER = new ClientLogger(BackupsUnderAccountsImpl.class); - - private final BackupsUnderAccountsClient innerClient; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public BackupsUnderAccountsImpl(BackupsUnderAccountsClient innerClient, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body) { - this.serviceClient().migrateBackups(resourceGroupName, accountName, body); - } - - public void migrateBackups(String resourceGroupName, String accountName, BackupsMigrationRequest body, - Context context) { - this.serviceClient().migrateBackups(resourceGroupName, accountName, body, context); - } - - private BackupsUnderAccountsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsClientImpl.java deleted file mode 100644 index 031b6e07fde35..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsClientImpl.java +++ /dev/null @@ -1,363 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Post; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.management.polling.PollResult; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.core.util.polling.PollerFlux; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderBackupVaultsClient; -import com.azure.resourcemanager.netapp.models.BackupRestoreFiles; -import java.nio.ByteBuffer; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -/** - * An instance of this class provides access to all the operations defined in BackupsUnderBackupVaultsClient. - */ -public final class BackupsUnderBackupVaultsClientImpl implements BackupsUnderBackupVaultsClient { - /** - * The proxy service used to perform REST calls. - */ - private final BackupsUnderBackupVaultsService service; - - /** - * The service client containing this operation class. - */ - private final NetAppManagementClientImpl client; - - /** - * Initializes an instance of BackupsUnderBackupVaultsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - BackupsUnderBackupVaultsClientImpl(NetAppManagementClientImpl client) { - this.service = RestProxy.create(BackupsUnderBackupVaultsService.class, client.getHttpPipeline(), - client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for NetAppManagementClientBackupsUnderBackupVaults to be used by the - * proxy service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "NetAppManagementClie") - public interface BackupsUnderBackupVaultsService { - @Headers({ "Content-Type: application/json" }) - @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/backupVaults/{backupVaultName}/backups/{backupName}/restoreFiles") - @ExpectedResponses({ 202 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> restoreFiles(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("backupVaultName") String backupVaultName, @PathParam("backupName") String backupName, - @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BackupRestoreFiles body, - @HeaderParam("Accept") String accept, Context context); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> restoreFilesWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.restoreFiles(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, backupVaultName, backupName, this.client.getApiVersion(), body, accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> restoreFilesWithResponseAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (backupVaultName == null) { - return Mono - .error(new IllegalArgumentException("Parameter backupVaultName is required and cannot be null.")); - } - if (backupName == null) { - return Mono.error(new IllegalArgumentException("Parameter backupName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.restoreFiles(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, backupVaultName, backupName, this.client.getApiVersion(), body, accept, context); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginRestoreFilesAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body) { - Mono>> mono - = restoreFilesWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginRestoreFilesAsync(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = restoreFilesWithResponseAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - context); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginRestoreFiles(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body) { - return this.beginRestoreFilesAsync(resourceGroupName, accountName, backupVaultName, backupName, body) - .getSyncPoller(); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginRestoreFiles(String resourceGroupName, String accountName, - String backupVaultName, String backupName, BackupRestoreFiles body, Context context) { - return this.beginRestoreFilesAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context) - .getSyncPoller(); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono restoreFilesAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, BackupRestoreFiles body) { - return beginRestoreFilesAsync(resourceGroupName, accountName, backupVaultName, backupName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono restoreFilesAsync(String resourceGroupName, String accountName, String backupVaultName, - String backupName, BackupRestoreFiles body, Context context) { - return beginRestoreFilesAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body) { - restoreFilesAsync(resourceGroupName, accountName, backupVaultName, backupName, body).block(); - } - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body, Context context) { - restoreFilesAsync(resourceGroupName, accountName, backupVaultName, backupName, body, context).block(); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsImpl.java deleted file mode 100644 index e82e4f5723b4b..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderBackupVaultsImpl.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderBackupVaultsClient; -import com.azure.resourcemanager.netapp.models.BackupRestoreFiles; -import com.azure.resourcemanager.netapp.models.BackupsUnderBackupVaults; - -public final class BackupsUnderBackupVaultsImpl implements BackupsUnderBackupVaults { - private static final ClientLogger LOGGER = new ClientLogger(BackupsUnderBackupVaultsImpl.class); - - private final BackupsUnderBackupVaultsClient innerClient; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public BackupsUnderBackupVaultsImpl(BackupsUnderBackupVaultsClient innerClient, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body) { - this.serviceClient().restoreFiles(resourceGroupName, accountName, backupVaultName, backupName, body); - } - - public void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body, Context context) { - this.serviceClient().restoreFiles(resourceGroupName, accountName, backupVaultName, backupName, body, context); - } - - private BackupsUnderBackupVaultsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesClientImpl.java deleted file mode 100644 index d7e17e88cb6b0..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesClientImpl.java +++ /dev/null @@ -1,361 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Post; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.management.polling.PollResult; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.core.util.polling.PollerFlux; -import com.azure.core.util.polling.SyncPoller; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderVolumesClient; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; -import java.nio.ByteBuffer; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -/** - * An instance of this class provides access to all the operations defined in BackupsUnderVolumesClient. - */ -public final class BackupsUnderVolumesClientImpl implements BackupsUnderVolumesClient { - /** - * The proxy service used to perform REST calls. - */ - private final BackupsUnderVolumesService service; - - /** - * The service client containing this operation class. - */ - private final NetAppManagementClientImpl client; - - /** - * Initializes an instance of BackupsUnderVolumesClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - BackupsUnderVolumesClientImpl(NetAppManagementClientImpl client) { - this.service = RestProxy.create(BackupsUnderVolumesService.class, client.getHttpPipeline(), - client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for NetAppManagementClientBackupsUnderVolumes to be used by the proxy - * service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "NetAppManagementClie") - public interface BackupsUnderVolumesService { - @Headers({ "Content-Type: application/json" }) - @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/migrateBackups") - @ExpectedResponses({ 202 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> migrateBackups(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BackupsMigrationRequest body, - @HeaderParam("Accept") String accept, Context context); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> migrateBackupsWithResponseAsync(String resourceGroupName, - String accountName, String poolName, String volumeName, BackupsMigrationRequest body) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (poolName == null) { - return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); - } - if (volumeName == null) { - return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.migrateBackups(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> migrateBackupsWithResponseAsync(String resourceGroupName, - String accountName, String poolName, String volumeName, BackupsMigrationRequest body, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (poolName == null) { - return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); - } - if (volumeName == null) { - return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); - } - if (body == null) { - return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); - } else { - body.validate(); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.migrateBackups(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateBackupsAsync(String resourceGroupName, String accountName, - String poolName, String volumeName, BackupsMigrationRequest body) { - Mono>> mono - = migrateBackupsWithResponseAsync(resourceGroupName, accountName, poolName, volumeName, body); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginMigrateBackupsAsync(String resourceGroupName, String accountName, - String poolName, String volumeName, BackupsMigrationRequest body, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = migrateBackupsWithResponseAsync(resourceGroupName, accountName, poolName, volumeName, body, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - context); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - String poolName, String volumeName, BackupsMigrationRequest body) { - return this.beginMigrateBackupsAsync(resourceGroupName, accountName, poolName, volumeName, body) - .getSyncPoller(); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginMigrateBackups(String resourceGroupName, String accountName, - String poolName, String volumeName, BackupsMigrationRequest body, Context context) { - return this.beginMigrateBackupsAsync(resourceGroupName, accountName, poolName, volumeName, body, context) - .getSyncPoller(); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateBackupsAsync(String resourceGroupName, String accountName, String poolName, - String volumeName, BackupsMigrationRequest body) { - return beginMigrateBackupsAsync(resourceGroupName, accountName, poolName, volumeName, body).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono migrateBackupsAsync(String resourceGroupName, String accountName, String poolName, - String volumeName, BackupsMigrationRequest body, Context context) { - return beginMigrateBackupsAsync(resourceGroupName, accountName, poolName, volumeName, body, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body) { - migrateBackupsAsync(resourceGroupName, accountName, poolName, volumeName, body).block(); - } - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body, Context context) { - migrateBackupsAsync(resourceGroupName, accountName, poolName, volumeName, body, context).block(); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesImpl.java deleted file mode 100644 index 76adfeecb3110..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/BackupsUnderVolumesImpl.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderVolumesClient; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; -import com.azure.resourcemanager.netapp.models.BackupsUnderVolumes; - -public final class BackupsUnderVolumesImpl implements BackupsUnderVolumes { - private static final ClientLogger LOGGER = new ClientLogger(BackupsUnderVolumesImpl.class); - - private final BackupsUnderVolumesClient innerClient; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public BackupsUnderVolumesImpl(BackupsUnderVolumesClient innerClient, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body) { - this.serviceClient().migrateBackups(resourceGroupName, accountName, poolName, volumeName, body); - } - - public void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body, Context context) { - this.serviceClient().migrateBackups(resourceGroupName, accountName, poolName, volumeName, body, context); - } - - private BackupsUnderVolumesClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/CapacityPoolImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/CapacityPoolImpl.java index 679a2128e69de..4e78156d6621d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/CapacityPoolImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/CapacityPoolImpl.java @@ -163,9 +163,9 @@ public CapacityPool apply(Context context) { com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.poolName = Utils.getValueFromIdByName(innerObject.id(), "capacityPools"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.poolName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "capacityPools"); } public CapacityPool refresh() { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppAccountImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppAccountImpl.java index b771fc2c88887..fdb3b357eb5af 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppAccountImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppAccountImpl.java @@ -10,7 +10,6 @@ import com.azure.resourcemanager.netapp.fluent.models.NetAppAccountInner; import com.azure.resourcemanager.netapp.models.AccountEncryption; import com.azure.resourcemanager.netapp.models.ActiveDirectory; -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; import com.azure.resourcemanager.netapp.models.ManagedServiceIdentity; import com.azure.resourcemanager.netapp.models.NetAppAccount; import com.azure.resourcemanager.netapp.models.NetAppAccountPatch; @@ -81,14 +80,6 @@ public Boolean disableShowmount() { return this.innerModel().disableShowmount(); } - public String nfsV4IdDomain() { - return this.innerModel().nfsV4IdDomain(); - } - - public Boolean isMultiAdEnabled() { - return this.innerModel().isMultiAdEnabled(); - } - public Region region() { return Region.fromName(this.regionName()); } @@ -159,8 +150,8 @@ public NetAppAccount apply(Context context) { com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); } public NetAppAccount refresh() { @@ -183,14 +174,6 @@ public void renewCredentials(Context context) { serviceManager.accounts().renewCredentials(resourceGroupName, accountName, context); } - public void migrateEncryptionKey() { - serviceManager.accounts().migrateEncryptionKey(resourceGroupName, accountName); - } - - public void migrateEncryptionKey(EncryptionMigrationRequest body, Context context) { - serviceManager.accounts().migrateEncryptionKey(resourceGroupName, accountName, body, context); - } - public NetAppAccountImpl withRegion(Region location) { this.innerModel().withLocation(location.toString()); return this; @@ -241,16 +224,6 @@ public NetAppAccountImpl withEncryption(AccountEncryption encryption) { } } - public NetAppAccountImpl withNfsV4IdDomain(String nfsV4IdDomain) { - if (isInCreateMode()) { - this.innerModel().withNfsV4IdDomain(nfsV4IdDomain); - return this; - } else { - this.updateBody.withNfsV4IdDomain(nfsV4IdDomain); - return this; - } - } - private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppManagementClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppManagementClientImpl.java index 257f0e949146e..0e910a62a1aba 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppManagementClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppManagementClientImpl.java @@ -22,17 +22,11 @@ import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.serializer.SerializerAdapter; import com.azure.core.util.serializer.SerializerEncoding; -import com.azure.resourcemanager.netapp.fluent.AccountBackupsClient; import com.azure.resourcemanager.netapp.fluent.AccountsClient; import com.azure.resourcemanager.netapp.fluent.BackupPoliciesClient; -import com.azure.resourcemanager.netapp.fluent.BackupVaultsClient; import com.azure.resourcemanager.netapp.fluent.BackupsClient; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderAccountsClient; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderBackupVaultsClient; -import com.azure.resourcemanager.netapp.fluent.BackupsUnderVolumesClient; import com.azure.resourcemanager.netapp.fluent.NetAppManagementClient; import com.azure.resourcemanager.netapp.fluent.NetAppResourceQuotaLimitsClient; -import com.azure.resourcemanager.netapp.fluent.NetAppResourceRegionInfosClient; import com.azure.resourcemanager.netapp.fluent.NetAppResourcesClient; import com.azure.resourcemanager.netapp.fluent.OperationsClient; import com.azure.resourcemanager.netapp.fluent.PoolsClient; @@ -182,20 +176,6 @@ public NetAppResourceQuotaLimitsClient getNetAppResourceQuotaLimits() { return this.netAppResourceQuotaLimits; } - /** - * The NetAppResourceRegionInfosClient object to access its operations. - */ - private final NetAppResourceRegionInfosClient netAppResourceRegionInfos; - - /** - * Gets the NetAppResourceRegionInfosClient object to access its operations. - * - * @return the NetAppResourceRegionInfosClient object. - */ - public NetAppResourceRegionInfosClient getNetAppResourceRegionInfos() { - return this.netAppResourceRegionInfos; - } - /** * The AccountsClient object to access its operations. */ @@ -280,20 +260,6 @@ public BackupsClient getBackups() { return this.backups; } - /** - * The AccountBackupsClient object to access its operations. - */ - private final AccountBackupsClient accountBackups; - - /** - * Gets the AccountBackupsClient object to access its operations. - * - * @return the AccountBackupsClient object. - */ - public AccountBackupsClient getAccountBackups() { - return this.accountBackups; - } - /** * The BackupPoliciesClient object to access its operations. */ @@ -350,62 +316,6 @@ public SubvolumesClient getSubvolumes() { return this.subvolumes; } - /** - * The BackupVaultsClient object to access its operations. - */ - private final BackupVaultsClient backupVaults; - - /** - * Gets the BackupVaultsClient object to access its operations. - * - * @return the BackupVaultsClient object. - */ - public BackupVaultsClient getBackupVaults() { - return this.backupVaults; - } - - /** - * The BackupsUnderBackupVaultsClient object to access its operations. - */ - private final BackupsUnderBackupVaultsClient backupsUnderBackupVaults; - - /** - * Gets the BackupsUnderBackupVaultsClient object to access its operations. - * - * @return the BackupsUnderBackupVaultsClient object. - */ - public BackupsUnderBackupVaultsClient getBackupsUnderBackupVaults() { - return this.backupsUnderBackupVaults; - } - - /** - * The BackupsUnderVolumesClient object to access its operations. - */ - private final BackupsUnderVolumesClient backupsUnderVolumes; - - /** - * Gets the BackupsUnderVolumesClient object to access its operations. - * - * @return the BackupsUnderVolumesClient object. - */ - public BackupsUnderVolumesClient getBackupsUnderVolumes() { - return this.backupsUnderVolumes; - } - - /** - * The BackupsUnderAccountsClient object to access its operations. - */ - private final BackupsUnderAccountsClient backupsUnderAccounts; - - /** - * Gets the BackupsUnderAccountsClient object to access its operations. - * - * @return the BackupsUnderAccountsClient object. - */ - public BackupsUnderAccountsClient getBackupsUnderAccounts() { - return this.backupsUnderAccounts; - } - /** * Initializes an instance of NetAppManagementClient client. * @@ -423,26 +333,20 @@ public BackupsUnderAccountsClient getBackupsUnderAccounts() { this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-05-01-preview"; + this.apiVersion = "2023-07-01"; this.operations = new OperationsClientImpl(this); this.netAppResources = new NetAppResourcesClientImpl(this); this.netAppResourceQuotaLimits = new NetAppResourceQuotaLimitsClientImpl(this); - this.netAppResourceRegionInfos = new NetAppResourceRegionInfosClientImpl(this); this.accounts = new AccountsClientImpl(this); this.pools = new PoolsClientImpl(this); this.volumes = new VolumesClientImpl(this); this.snapshots = new SnapshotsClientImpl(this); this.snapshotPolicies = new SnapshotPoliciesClientImpl(this); this.backups = new BackupsClientImpl(this); - this.accountBackups = new AccountBackupsClientImpl(this); this.backupPolicies = new BackupPoliciesClientImpl(this); this.volumeQuotaRules = new VolumeQuotaRulesClientImpl(this); this.volumeGroups = new VolumeGroupsClientImpl(this); this.subvolumes = new SubvolumesClientImpl(this); - this.backupVaults = new BackupVaultsClientImpl(this); - this.backupsUnderBackupVaults = new BackupsUnderBackupVaultsClientImpl(this); - this.backupsUnderVolumes = new BackupsUnderVolumesClientImpl(this); - this.backupsUnderAccounts = new BackupsUnderAccountsClientImpl(this); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceQuotaLimitsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceQuotaLimitsImpl.java index 05c90866dc824..0b3b0931de89d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceQuotaLimitsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceQuotaLimitsImpl.java @@ -29,12 +29,12 @@ public NetAppResourceQuotaLimitsImpl(NetAppResourceQuotaLimitsClient innerClient public PagedIterable list(String location) { PagedIterable inner = this.serviceClient().list(location); - return Utils.mapPage(inner, inner1 -> new SubscriptionQuotaItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SubscriptionQuotaItemImpl(inner1, this.manager())); } public PagedIterable list(String location, Context context) { PagedIterable inner = this.serviceClient().list(location, context); - return Utils.mapPage(inner, inner1 -> new SubscriptionQuotaItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SubscriptionQuotaItemImpl(inner1, this.manager())); } public Response getWithResponse(String location, String quotaLimitName, Context context) { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosClientImpl.java deleted file mode 100644 index c97dfe9209b42..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosClientImpl.java +++ /dev/null @@ -1,390 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.Get; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.netapp.fluent.NetAppResourceRegionInfosClient; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; -import com.azure.resourcemanager.netapp.models.RegionInfosList; -import reactor.core.publisher.Mono; - -/** - * An instance of this class provides access to all the operations defined in NetAppResourceRegionInfosClient. - */ -public final class NetAppResourceRegionInfosClientImpl implements NetAppResourceRegionInfosClient { - /** - * The proxy service used to perform REST calls. - */ - private final NetAppResourceRegionInfosService service; - - /** - * The service client containing this operation class. - */ - private final NetAppManagementClientImpl client; - - /** - * Initializes an instance of NetAppResourceRegionInfosClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - NetAppResourceRegionInfosClientImpl(NetAppManagementClientImpl client) { - this.service = RestProxy.create(NetAppResourceRegionInfosService.class, client.getHttpPipeline(), - client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for NetAppManagementClientNetAppResourceRegionInfos to be used by the - * proxy service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "NetAppManagementClie") - public interface NetAppResourceRegionInfosService { - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.NetApp/locations/{location}/regionInfos") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, @PathParam("location") String location, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.NetApp/locations/{location}/regionInfos/default") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, @PathParam("location") String location, - @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - - @Headers({ "Content-Type: application/json" }) - @Get("{nextLink}") - @ExpectedResponses({ 200 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - } - - /** - * Describes region specific information. - * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(String location) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (location == null) { - return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), location, - this.client.getApiVersion(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Describes region specific information. - * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(String location, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (location == null) { - return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), location, this.client.getApiVersion(), - accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } - - /** - * Describes region specific information. - * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String location) { - return new PagedFlux<>(() -> listSinglePageAsync(location), nextLink -> listNextSinglePageAsync(nextLink)); - } - - /** - * Describes region specific information. - * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String location, Context context) { - return new PagedFlux<>(() -> listSinglePageAsync(location, context), - nextLink -> listNextSinglePageAsync(nextLink, context)); - } - - /** - * Describes region specific information. - * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String location) { - return new PagedIterable<>(listAsync(location)); - } - - /** - * Describes region specific information. - * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String location, Context context) { - return new PagedIterable<>(listAsync(location, context)); - } - - /** - * Describes region specific information. - * - * Provides storage to network proximity and logical zone mapping information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String location) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (location == null) { - return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), location, - this.client.getApiVersion(), accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Describes region specific information. - * - * Provides storage to network proximity and logical zone mapping information. - * - * @param location The name of the Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item along with {@link Response} on successful completion of - * {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String location, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (location == null) { - return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), location, - this.client.getApiVersion(), accept, context); - } - - /** - * Describes region specific information. - * - * Provides storage to network proximity and logical zone mapping information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String location) { - return getWithResponseAsync(location).flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Describes region specific information. - * - * Provides storage to network proximity and logical zone mapping information. - * - * @param location The name of the Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String location, Context context) { - return getWithResponseAsync(location, context).block(); - } - - /** - * Describes region specific information. - * - * Provides storage to network proximity and logical zone mapping information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public RegionInfoResourceInner get(String location) { - return getWithResponse(location, Context.NONE).getValue(); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map(res -> new PagedResponseBase<>(res.getRequest(), - res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - * - * The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.listNext(nextLink, this.client.getEndpoint(), accept, context) - .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), - res.getValue().value(), res.getValue().nextLink(), null)); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosImpl.java deleted file mode 100644 index 1f153feb2462b..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/NetAppResourceRegionInfosImpl.java +++ /dev/null @@ -1,66 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.netapp.fluent.NetAppResourceRegionInfosClient; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; -import com.azure.resourcemanager.netapp.models.NetAppResourceRegionInfos; -import com.azure.resourcemanager.netapp.models.RegionInfoResource; - -public final class NetAppResourceRegionInfosImpl implements NetAppResourceRegionInfos { - private static final ClientLogger LOGGER = new ClientLogger(NetAppResourceRegionInfosImpl.class); - - private final NetAppResourceRegionInfosClient innerClient; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - public NetAppResourceRegionInfosImpl(NetAppResourceRegionInfosClient innerClient, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable list(String location) { - PagedIterable inner = this.serviceClient().list(location); - return Utils.mapPage(inner, inner1 -> new RegionInfoResourceImpl(inner1, this.manager())); - } - - public PagedIterable list(String location, Context context) { - PagedIterable inner = this.serviceClient().list(location, context); - return Utils.mapPage(inner, inner1 -> new RegionInfoResourceImpl(inner1, this.manager())); - } - - public Response getWithResponse(String location, Context context) { - Response inner = this.serviceClient().getWithResponse(location, context); - if (inner != null) { - return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), - new RegionInfoResourceImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public RegionInfoResource get(String location) { - RegionInfoResourceInner inner = this.serviceClient().get(location); - if (inner != null) { - return new RegionInfoResourceImpl(inner, this.manager()); - } else { - return null; - } - } - - private NetAppResourceRegionInfosClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/OperationsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/OperationsImpl.java index f2395735d947f..9331c1270c6a0 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/OperationsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/OperationsImpl.java @@ -27,12 +27,12 @@ public OperationsImpl(OperationsClient innerClient, public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsClientImpl.java index 1d2f4b6bba7be..86afd4437b011 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsClientImpl.java @@ -112,14 +112,15 @@ Mono>> update(@HostParam("$host") String endpoint, @BodyParam("application/json") CapacityPoolPatch body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}") @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("poolName") String poolName, @QueryParam("api-version") String apiVersion, Context context); + @PathParam("poolName") String poolName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") @@ -989,9 +990,10 @@ private Mono>> deleteWithResponseAsync(String resource if (poolName == null) { return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1030,9 +1032,10 @@ private Mono>> deleteWithResponseAsync(String resource if (poolName == null) { return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, this.client.getApiVersion(), context); + accountName, poolName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsImpl.java index 5a98e065dcab0..c458d0655b3fe 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/PoolsImpl.java @@ -28,12 +28,12 @@ public PoolsImpl(PoolsClient innerClient, com.azure.resourcemanager.netapp.NetAp public PagedIterable list(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new CapacityPoolImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CapacityPoolImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new CapacityPoolImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new CapacityPoolImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, String poolName, @@ -66,17 +66,17 @@ public void delete(String resourceGroupName, String accountName, String poolName } public CapacityPool getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); @@ -85,17 +85,17 @@ public CapacityPool getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); @@ -104,17 +104,17 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); @@ -123,17 +123,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/RegionInfoResourceImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/RegionInfoResourceImpl.java deleted file mode 100644 index 75b06404841db..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/RegionInfoResourceImpl.java +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.implementation; - -import com.azure.core.management.SystemData; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; -import com.azure.resourcemanager.netapp.models.RegionInfoAvailabilityZoneMappingsItem; -import com.azure.resourcemanager.netapp.models.RegionInfoResource; -import com.azure.resourcemanager.netapp.models.RegionStorageToNetworkProximity; -import java.util.Collections; -import java.util.List; - -public final class RegionInfoResourceImpl implements RegionInfoResource { - private RegionInfoResourceInner innerObject; - - private final com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager; - - RegionInfoResourceImpl(RegionInfoResourceInner innerObject, - com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public SystemData systemData() { - return this.innerModel().systemData(); - } - - public RegionStorageToNetworkProximity storageToNetworkProximity() { - return this.innerModel().storageToNetworkProximity(); - } - - public List availabilityZoneMappings() { - List inner = this.innerModel().availabilityZoneMappings(); - if (inner != null) { - return Collections.unmodifiableList(inner); - } else { - return Collections.emptyList(); - } - } - - public RegionInfoResourceInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.netapp.NetAppFilesManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/Utils.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/ResourceManagerUtils.java similarity index 99% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/Utils.java rename to sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/ResourceManagerUtils.java index e57610e622a03..ee1578ed17669 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/Utils.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/ResourceManagerUtils.java @@ -19,8 +19,8 @@ import java.util.stream.Stream; import reactor.core.publisher.Flux; -final class Utils { - private Utils() { +final class ResourceManagerUtils { + private ResourceManagerUtils() { } static String getValueFromIdByName(String id, String name) { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesClientImpl.java index e01581f6b8e39..ac801c5c3ceba 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesClientImpl.java @@ -114,7 +114,7 @@ Mono>> update(@HostParam("$host") String endpoint, @BodyParam("application/json") SnapshotPolicyPatch body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/snapshotPolicies/{snapshotPolicyName}") @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -122,7 +122,7 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("snapshotPolicyName") String snapshotPolicyName, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/snapshotPolicies/{snapshotPolicyName}/volumes") @@ -833,9 +833,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter snapshotPolicyName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, snapshotPolicyName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, snapshotPolicyName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -873,9 +874,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter snapshotPolicyName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, snapshotPolicyName, this.client.getApiVersion(), context); + accountName, snapshotPolicyName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesImpl.java index dec956b3979b3..ae8a86063680b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPoliciesImpl.java @@ -31,12 +31,12 @@ public SnapshotPoliciesImpl(SnapshotPoliciesClient innerClient, public PagedIterable list(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new SnapshotPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SnapshotPolicyImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new SnapshotPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SnapshotPolicyImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, @@ -92,17 +92,17 @@ public SnapshotPolicyVolumeList listVolumes(String resourceGroupName, String acc } public SnapshotPolicy getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String snapshotPolicyName = Utils.getValueFromIdByName(id, "snapshotPolicies"); + String snapshotPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "snapshotPolicies"); if (snapshotPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshotPolicies'.", id))); @@ -111,17 +111,17 @@ public SnapshotPolicy getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String snapshotPolicyName = Utils.getValueFromIdByName(id, "snapshotPolicies"); + String snapshotPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "snapshotPolicies"); if (snapshotPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshotPolicies'.", id))); @@ -130,17 +130,17 @@ public Response getByIdWithResponse(String id, Context context) } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String snapshotPolicyName = Utils.getValueFromIdByName(id, "snapshotPolicies"); + String snapshotPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "snapshotPolicies"); if (snapshotPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshotPolicies'.", id))); @@ -149,17 +149,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String snapshotPolicyName = Utils.getValueFromIdByName(id, "snapshotPolicies"); + String snapshotPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "snapshotPolicies"); if (snapshotPolicyName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshotPolicies'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPolicyImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPolicyImpl.java index 3c2dbd88252e7..a025eaee9a966 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPolicyImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotPolicyImpl.java @@ -154,9 +154,9 @@ public SnapshotPolicy apply(Context context) { com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.snapshotPolicyName = Utils.getValueFromIdByName(innerObject.id(), "snapshotPolicies"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.snapshotPolicyName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "snapshotPolicies"); } public SnapshotPolicy refresh() { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsClientImpl.java index a5db39da2155f..9a6933c57e2ab 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsClientImpl.java @@ -116,7 +116,7 @@ Mono>> update(@HostParam("$host") String endpoint, @PathParam("snapshotName") String snapshotName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") Object body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/snapshots/{snapshotName}") @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -125,9 +125,9 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @PathParam("snapshotName") String snapshotName, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/snapshots/{snapshotName}/restoreFiles") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -136,7 +136,8 @@ Mono>> restoreFiles(@HostParam("$host") String endpoin @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @PathParam("snapshotName") String snapshotName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") SnapshotRestoreFiles body, Context context); + @BodyParam("application/json") SnapshotRestoreFiles body, @HeaderParam("Accept") String accept, + Context context); } /** @@ -1127,10 +1128,11 @@ private Mono>> deleteWithResponseAsync(String resource if (snapshotName == null) { return Mono.error(new IllegalArgumentException("Parameter snapshotName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), context)) + accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1177,9 +1179,10 @@ private Mono>> deleteWithResponseAsync(String resource if (snapshotName == null) { return Mono.error(new IllegalArgumentException("Parameter snapshotName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), accept, context); } /** @@ -1411,10 +1414,11 @@ private Mono>> restoreFilesWithResponseAsync(String re } else { body.validate(); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.restoreFiles(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), body, - context)) + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1467,9 +1471,10 @@ private Mono>> restoreFilesWithResponseAsync(String re } else { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.restoreFiles(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), body, context); + accountName, poolName, volumeName, snapshotName, this.client.getApiVersion(), body, accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsImpl.java index a7ae03c1f4711..1de9c3795758a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SnapshotsImpl.java @@ -32,14 +32,14 @@ public PagedIterable list(String resourceGroupName, String accountName String volumeName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, poolName, volumeName); - return Utils.mapPage(inner, inner1 -> new SnapshotImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SnapshotImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, String poolName, String volumeName, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, poolName, volumeName, context); - return Utils.mapPage(inner, inner1 -> new SnapshotImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SnapshotImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, String poolName, @@ -109,27 +109,27 @@ public void restoreFiles(String resourceGroupName, String accountName, String po } public Snapshot getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String snapshotName = Utils.getValueFromIdByName(id, "snapshots"); + String snapshotName = ResourceManagerUtils.getValueFromIdByName(id, "snapshots"); if (snapshotName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshots'.", id))); @@ -139,27 +139,27 @@ public Snapshot getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String snapshotName = Utils.getValueFromIdByName(id, "snapshots"); + String snapshotName = ResourceManagerUtils.getValueFromIdByName(id, "snapshots"); if (snapshotName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshots'.", id))); @@ -168,27 +168,27 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String snapshotName = Utils.getValueFromIdByName(id, "snapshots"); + String snapshotName = ResourceManagerUtils.getValueFromIdByName(id, "snapshots"); if (snapshotName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshots'.", id))); @@ -197,27 +197,27 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String snapshotName = Utils.getValueFromIdByName(id, "snapshots"); + String snapshotName = ResourceManagerUtils.getValueFromIdByName(id, "snapshots"); if (snapshotName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'snapshots'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumeInfoImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumeInfoImpl.java index e839135d0b060..e0f3f45c2165c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumeInfoImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumeInfoImpl.java @@ -120,11 +120,11 @@ public SubvolumeInfo apply(Context context) { com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.poolName = Utils.getValueFromIdByName(innerObject.id(), "capacityPools"); - this.volumeName = Utils.getValueFromIdByName(innerObject.id(), "volumes"); - this.subvolumeName = Utils.getValueFromIdByName(innerObject.id(), "subvolumes"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.poolName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "capacityPools"); + this.volumeName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "volumes"); + this.subvolumeName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "subvolumes"); } public SubvolumeInfo refresh() { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesClientImpl.java index 7f3db8910b816..b346b9d8b2bb1 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesClientImpl.java @@ -119,7 +119,7 @@ Mono>> update(@HostParam("$host") String endpoint, @BodyParam("application/json") SubvolumePatchRequest body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/subvolumes/{subvolumeName}") @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -128,7 +128,7 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @PathParam("subvolumeName") String subvolumeName, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/subvolumes/{subvolumeName}/getMetadata") @@ -1145,10 +1145,11 @@ private Mono>> deleteWithResponseAsync(String resource if (subvolumeName == null) { return Mono.error(new IllegalArgumentException("Parameter subvolumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, subvolumeName, this.client.getApiVersion(), context)) + accountName, poolName, volumeName, subvolumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1195,9 +1196,10 @@ private Mono>> deleteWithResponseAsync(String resource if (subvolumeName == null) { return Mono.error(new IllegalArgumentException("Parameter subvolumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, subvolumeName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, subvolumeName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesImpl.java index 3b06055385ef4..bf88658f1e6de 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/SubvolumesImpl.java @@ -33,14 +33,14 @@ public PagedIterable listByVolume(String resourceGroupName, Strin String volumeName) { PagedIterable inner = this.serviceClient().listByVolume(resourceGroupName, accountName, poolName, volumeName); - return Utils.mapPage(inner, inner1 -> new SubvolumeInfoImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SubvolumeInfoImpl(inner1, this.manager())); } public PagedIterable listByVolume(String resourceGroupName, String accountName, String poolName, String volumeName, Context context) { PagedIterable inner = this.serviceClient().listByVolume(resourceGroupName, accountName, poolName, volumeName, context); - return Utils.mapPage(inner, inner1 -> new SubvolumeInfoImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SubvolumeInfoImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, String poolName, @@ -99,27 +99,27 @@ public SubvolumeModel getMetadata(String resourceGroupName, String accountName, } public SubvolumeInfo getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String subvolumeName = Utils.getValueFromIdByName(id, "subvolumes"); + String subvolumeName = ResourceManagerUtils.getValueFromIdByName(id, "subvolumes"); if (subvolumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'subvolumes'.", id))); @@ -129,27 +129,27 @@ public SubvolumeInfo getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String subvolumeName = Utils.getValueFromIdByName(id, "subvolumes"); + String subvolumeName = ResourceManagerUtils.getValueFromIdByName(id, "subvolumes"); if (subvolumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'subvolumes'.", id))); @@ -158,27 +158,27 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String subvolumeName = Utils.getValueFromIdByName(id, "subvolumes"); + String subvolumeName = ResourceManagerUtils.getValueFromIdByName(id, "subvolumes"); if (subvolumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'subvolumes'.", id))); @@ -187,27 +187,27 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String subvolumeName = Utils.getValueFromIdByName(id, "subvolumes"); + String subvolumeName = ResourceManagerUtils.getValueFromIdByName(id, "subvolumes"); if (subvolumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'subvolumes'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsClientImpl.java index c08e51c9274fc..93dd5c63a1b9f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsClientImpl.java @@ -101,7 +101,7 @@ Mono>> create(@HostParam("$host") String endpoint, @BodyParam("application/json") VolumeGroupDetailsInner body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/volumeGroups/{volumeGroupName}") @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -109,7 +109,7 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("volumeGroupName") String volumeGroupName, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** @@ -735,9 +735,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter volumeGroupName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, volumeGroupName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, volumeGroupName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -777,9 +778,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter volumeGroupName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, volumeGroupName, this.client.getApiVersion(), context); + accountName, volumeGroupName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsImpl.java index 7060fa97463ea..df4a412f9cc47 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeGroupsImpl.java @@ -32,14 +32,14 @@ public VolumeGroupsImpl(VolumeGroupsClient innerClient, public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName) { PagedIterable inner = this.serviceClient().listByNetAppAccount(resourceGroupName, accountName); - return Utils.mapPage(inner, inner1 -> new VolumeGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VolumeGroupImpl(inner1, this.manager())); } public PagedIterable listByNetAppAccount(String resourceGroupName, String accountName, Context context) { PagedIterable inner = this.serviceClient().listByNetAppAccount(resourceGroupName, accountName, context); - return Utils.mapPage(inner, inner1 -> new VolumeGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VolumeGroupImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, @@ -72,17 +72,17 @@ public void delete(String resourceGroupName, String accountName, String volumeGr } public VolumeGroupDetails getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String volumeGroupName = Utils.getValueFromIdByName(id, "volumeGroups"); + String volumeGroupName = ResourceManagerUtils.getValueFromIdByName(id, "volumeGroups"); if (volumeGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeGroups'.", id))); @@ -91,17 +91,17 @@ public VolumeGroupDetails getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String volumeGroupName = Utils.getValueFromIdByName(id, "volumeGroups"); + String volumeGroupName = ResourceManagerUtils.getValueFromIdByName(id, "volumeGroups"); if (volumeGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeGroups'.", id))); @@ -110,17 +110,17 @@ public Response getByIdWithResponse(String id, Context conte } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String volumeGroupName = Utils.getValueFromIdByName(id, "volumeGroups"); + String volumeGroupName = ResourceManagerUtils.getValueFromIdByName(id, "volumeGroups"); if (volumeGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeGroups'.", id))); @@ -129,17 +129,17 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String volumeGroupName = Utils.getValueFromIdByName(id, "volumeGroups"); + String volumeGroupName = ResourceManagerUtils.getValueFromIdByName(id, "volumeGroups"); if (volumeGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeGroups'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeImpl.java index ef299b7ebe381..febbb86c09383 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeImpl.java @@ -325,10 +325,6 @@ public String originatingResourceId() { return this.innerModel().originatingResourceId(); } - public Long inheritedSizeInBytes() { - return this.innerModel().inheritedSizeInBytes(); - } - public Region region() { return Region.fromName(this.regionName()); } @@ -404,10 +400,10 @@ public Volume apply(Context context) { VolumeImpl(VolumeInner innerObject, com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.poolName = Utils.getValueFromIdByName(innerObject.id(), "capacityPools"); - this.volumeName = Utils.getValueFromIdByName(innerObject.id(), "volumes"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.poolName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "capacityPools"); + this.volumeName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "volumes"); } public Volume refresh() { @@ -447,14 +443,6 @@ public void resetCifsPassword(Context context) { serviceManager.volumes().resetCifsPassword(resourceGroupName, accountName, poolName, volumeName, context); } - public void splitCloneFromParent() { - serviceManager.volumes().splitCloneFromParent(resourceGroupName, accountName, poolName, volumeName); - } - - public void splitCloneFromParent(Context context) { - serviceManager.volumes().splitCloneFromParent(resourceGroupName, accountName, poolName, volumeName, context); - } - public void breakFileLocks() { serviceManager.volumes().breakFileLocks(resourceGroupName, accountName, poolName, volumeName); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRuleImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRuleImpl.java index 49548a6ebf170..3d531824cd660 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRuleImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRuleImpl.java @@ -145,11 +145,11 @@ public VolumeQuotaRule apply(Context context) { com.azure.resourcemanager.netapp.NetAppFilesManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.accountName = Utils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); - this.poolName = Utils.getValueFromIdByName(innerObject.id(), "capacityPools"); - this.volumeName = Utils.getValueFromIdByName(innerObject.id(), "volumes"); - this.volumeQuotaRuleName = Utils.getValueFromIdByName(innerObject.id(), "volumeQuotaRules"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.accountName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "netAppAccounts"); + this.poolName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "capacityPools"); + this.volumeName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "volumes"); + this.volumeQuotaRuleName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "volumeQuotaRules"); } public VolumeQuotaRule refresh() { diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesClientImpl.java index cbb24432561db..124f5cd8793dc 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesClientImpl.java @@ -117,7 +117,7 @@ Mono>> update(@HostParam("$host") String endpoint, @BodyParam("application/json") VolumeQuotaRulePatch body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/volumeQuotaRules/{volumeQuotaRuleName}") @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -126,7 +126,7 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @PathParam("volumeQuotaRuleName") String volumeQuotaRuleName, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** @@ -1136,10 +1136,11 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter volumeQuotaRuleName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil - .withContext( - context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, volumeQuotaRuleName, this.client.getApiVersion(), context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, accountName, poolName, volumeName, volumeQuotaRuleName, this.client.getApiVersion(), + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1187,9 +1188,10 @@ private Mono>> deleteWithResponseAsync(String resource return Mono .error(new IllegalArgumentException("Parameter volumeQuotaRuleName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, volumeQuotaRuleName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, volumeQuotaRuleName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesImpl.java index 40b328b2d45a6..41570c7d99a48 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumeQuotaRulesImpl.java @@ -31,14 +31,14 @@ public PagedIterable listByVolume(String resourceGroupName, Str String volumeName) { PagedIterable inner = this.serviceClient().listByVolume(resourceGroupName, accountName, poolName, volumeName); - return Utils.mapPage(inner, inner1 -> new VolumeQuotaRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VolumeQuotaRuleImpl(inner1, this.manager())); } public PagedIterable listByVolume(String resourceGroupName, String accountName, String poolName, String volumeName, Context context) { PagedIterable inner = this.serviceClient().listByVolume(resourceGroupName, accountName, poolName, volumeName, context); - return Utils.mapPage(inner, inner1 -> new VolumeQuotaRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VolumeQuotaRuleImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, String poolName, @@ -75,27 +75,27 @@ public void delete(String resourceGroupName, String accountName, String poolName } public VolumeQuotaRule getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String volumeQuotaRuleName = Utils.getValueFromIdByName(id, "volumeQuotaRules"); + String volumeQuotaRuleName = ResourceManagerUtils.getValueFromIdByName(id, "volumeQuotaRules"); if (volumeQuotaRuleName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeQuotaRules'.", id))); @@ -106,27 +106,27 @@ public VolumeQuotaRule getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String volumeQuotaRuleName = Utils.getValueFromIdByName(id, "volumeQuotaRules"); + String volumeQuotaRuleName = ResourceManagerUtils.getValueFromIdByName(id, "volumeQuotaRules"); if (volumeQuotaRuleName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeQuotaRules'.", id))); @@ -135,27 +135,27 @@ public Response getByIdWithResponse(String id, Context context) } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String volumeQuotaRuleName = Utils.getValueFromIdByName(id, "volumeQuotaRules"); + String volumeQuotaRuleName = ResourceManagerUtils.getValueFromIdByName(id, "volumeQuotaRules"); if (volumeQuotaRuleName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeQuotaRules'.", id))); @@ -164,27 +164,27 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); } - String volumeQuotaRuleName = Utils.getValueFromIdByName(id, "volumeQuotaRules"); + String volumeQuotaRuleName = ResourceManagerUtils.getValueFromIdByName(id, "volumeQuotaRules"); if (volumeQuotaRuleName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumeQuotaRules'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesClientImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesClientImpl.java index 9dc496a5a5acd..4713305302b15 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesClientImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesClientImpl.java @@ -126,7 +126,7 @@ Mono>> update(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") VolumePatch body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}") @ExpectedResponses({ 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -135,7 +135,7 @@ Mono>> delete(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("forceDelete") Boolean forceDelete, @QueryParam("api-version") String apiVersion, - Context context); + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/populateAvailabilityZone") @@ -147,7 +147,7 @@ Mono>> populateAvailabilityZone(@HostParam("$host") St @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/revert") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -156,29 +156,19 @@ Mono>> revert(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") VolumeRevert body, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/resetCifsPassword") @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono>> resetCifsPassword(@HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, - @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, Context context); - - @Headers({ "Content-Type: application/json" }) - @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/splitCloneFromParent") - @ExpectedResponses({ 202 }) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> splitCloneFromParent(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/breakFileLocks") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -187,7 +177,7 @@ Mono>> breakFileLocks(@HostParam("$host") String endpo @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BreakFileLocksRequest body, - Context context); + @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/getGroupIdListForLdapUser") @@ -201,7 +191,7 @@ Mono>> listGetGroupIdListForLdapUser(@HostParam("$host @BodyParam("application/json") GetGroupIdListForLdapUserRequest body, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/breakReplication") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -210,9 +200,9 @@ Mono>> breakReplication(@HostParam("$host") String end @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") BreakReplicationRequest body, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/reestablishReplication") @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -221,7 +211,8 @@ Mono>> reestablishReplication(@HostParam("$host") Stri @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") ReestablishReplicationRequest body, Context context); + @BodyParam("application/json") ReestablishReplicationRequest body, @HeaderParam("Accept") String accept, + Context context); @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/replicationStatus") @@ -243,7 +234,7 @@ Mono> listReplications(@HostParam("$host") String end @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/resyncReplication") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -251,9 +242,9 @@ Mono>> resyncReplication(@HostParam("$host") String en @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/deleteReplication") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -261,9 +252,9 @@ Mono>> deleteReplication(@HostParam("$host") String en @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/authorizeReplication") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -272,9 +263,9 @@ Mono>> authorizeReplication(@HostParam("$host") String @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") AuthorizeRequest body, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/reinitializeReplication") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -282,9 +273,9 @@ Mono>> reInitializeReplication(@HostParam("$host") Str @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/poolChange") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -293,9 +284,9 @@ Mono>> poolChange(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") PoolChangeRequest body, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/relocate") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -304,9 +295,9 @@ Mono>> relocate(@HostParam("$host") String endpoint, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") RelocateVolumeRequest body, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/finalizeRelocation") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -314,9 +305,9 @@ Mono>> finalizeRelocation(@HostParam("$host") String e @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NetApp/netAppAccounts/{accountName}/capacityPools/{poolName}/volumes/{volumeName}/revertRelocation") @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) @@ -324,7 +315,7 @@ Mono>> revertRelocation(@HostParam("$host") String end @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("accountName") String accountName, @PathParam("poolName") String poolName, @PathParam("volumeName") String volumeName, - @QueryParam("api-version") String apiVersion, Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") @@ -1261,10 +1252,11 @@ private Mono>> deleteWithResponseAsync(String resource if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, forceDelete, this.client.getApiVersion(), context)) + accountName, poolName, volumeName, forceDelete, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1309,9 +1301,10 @@ private Mono>> deleteWithResponseAsync(String resource if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, forceDelete, this.client.getApiVersion(), context); + accountName, poolName, volumeName, forceDelete, this.client.getApiVersion(), accept, context); } /** @@ -1854,9 +1847,11 @@ private Mono>> revertWithResponseAsync(String resource } else { body.validate(); } + final String accept = "application/json"; return FluxUtil - .withContext(context -> service.revert(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + .withContext( + context -> service.revert(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -1905,9 +1900,10 @@ private Mono>> revertWithResponseAsync(String resource } else { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.revert(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -2128,10 +2124,11 @@ private Mono>> resetCifsPasswordWithResponseAsync(Stri if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.resetCifsPassword(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -2174,9 +2171,10 @@ private Mono>> resetCifsPasswordWithResponseAsync(Stri if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.resetCifsPassword(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); } /** @@ -2350,269 +2348,6 @@ public void resetCifsPassword(String resourceGroupName, String accountName, Stri resetCifsPasswordAsync(resourceGroupName, accountName, poolName, volumeName, context).block(); } - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> splitCloneFromParentWithResponseAsync(String resourceGroupName, - String accountName, String poolName, String volumeName) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (poolName == null) { - return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); - } - if (volumeName == null) { - return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> service.splitCloneFromParent(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> splitCloneFromParentWithResponseAsync(String resourceGroupName, - String accountName, String poolName, String volumeName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono.error( - new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono.error(new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (accountName == null) { - return Mono.error(new IllegalArgumentException("Parameter accountName is required and cannot be null.")); - } - if (poolName == null) { - return Mono.error(new IllegalArgumentException("Parameter poolName is required and cannot be null.")); - } - if (volumeName == null) { - return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.splitCloneFromParent(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginSplitCloneFromParentAsync(String resourceGroupName, - String accountName, String poolName, String volumeName) { - Mono>> mono - = splitCloneFromParentWithResponseAsync(resourceGroupName, accountName, poolName, volumeName); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - this.client.getContext()); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link PollerFlux} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginSplitCloneFromParentAsync(String resourceGroupName, - String accountName, String poolName, String volumeName, Context context) { - context = this.client.mergeContext(context); - Mono>> mono - = splitCloneFromParentWithResponseAsync(resourceGroupName, accountName, poolName, volumeName, context); - return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, - context); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginSplitCloneFromParent(String resourceGroupName, String accountName, - String poolName, String volumeName) { - return this.beginSplitCloneFromParentAsync(resourceGroupName, accountName, poolName, volumeName) - .getSyncPoller(); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link SyncPoller} for polling of long-running operation. - */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginSplitCloneFromParent(String resourceGroupName, String accountName, - String poolName, String volumeName, Context context) { - return this.beginSplitCloneFromParentAsync(resourceGroupName, accountName, poolName, volumeName, context) - .getSyncPoller(); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono splitCloneFromParentAsync(String resourceGroupName, String accountName, String poolName, - String volumeName) { - return beginSplitCloneFromParentAsync(resourceGroupName, accountName, poolName, volumeName).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono splitCloneFromParentAsync(String resourceGroupName, String accountName, String poolName, - String volumeName, Context context) { - return beginSplitCloneFromParentAsync(resourceGroupName, accountName, poolName, volumeName, context).last() - .flatMap(this.client::getLroFinalResultOrError); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName) { - splitCloneFromParentAsync(resourceGroupName, accountName, poolName, volumeName).block(); - } - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName, - Context context) { - splitCloneFromParentAsync(resourceGroupName, accountName, poolName, volumeName, context).block(); - } - /** * Break file locks * @@ -2655,9 +2390,11 @@ private Mono>> breakFileLocksWithResponseAsync(String if (body != null) { body.validate(); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.breakFileLocks(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -2704,9 +2441,10 @@ private Mono>> breakFileLocksWithResponseAsync(String if (body != null) { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.breakFileLocks(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -3273,9 +3011,11 @@ private Mono>> breakReplicationWithResponseAsync(Strin if (body != null) { body.validate(); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.breakReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -3322,9 +3062,10 @@ private Mono>> breakReplicationWithResponseAsync(Strin if (body != null) { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.breakReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -3598,10 +3339,11 @@ private Mono>> reestablishReplicationWithResponseAsync } else { body.validate(); } + final String accept = "application/json"; return FluxUtil - .withContext( - context -> service.reestablishReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + .withContext(context -> service.reestablishReplication(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, accountName, poolName, volumeName, + this.client.getApiVersion(), body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -3651,9 +3393,10 @@ private Mono>> reestablishReplicationWithResponseAsync } else { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.reestablishReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -4220,10 +3963,11 @@ private Mono>> resyncReplicationWithResponseAsync(Stri if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.resyncReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -4267,9 +4011,10 @@ private Mono>> resyncReplicationWithResponseAsync(Stri if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.resyncReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); } /** @@ -4489,10 +4234,11 @@ private Mono>> deleteReplicationWithResponseAsync(Stri if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.deleteReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -4535,9 +4281,10 @@ private Mono>> deleteReplicationWithResponseAsync(Stri if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.deleteReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); } /** @@ -4755,10 +4502,11 @@ private Mono>> authorizeReplicationWithResponseAsync(S } else { body.validate(); } + final String accept = "application/json"; return FluxUtil - .withContext( - context -> service.authorizeReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + .withContext(context -> service.authorizeReplication(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, accountName, poolName, volumeName, + this.client.getApiVersion(), body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -4807,9 +4555,10 @@ private Mono>> authorizeReplicationWithResponseAsync(S } else { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.authorizeReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -5031,10 +4780,11 @@ private Mono>> reInitializeReplicationWithResponseAsyn if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.reInitializeReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -5077,9 +4827,10 @@ private Mono>> reInitializeReplicationWithResponseAsyn if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.reInitializeReplication(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context); + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); } /** @@ -5299,9 +5050,11 @@ private Mono>> poolChangeWithResponseAsync(String reso } else { body.validate(); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.poolChange(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -5350,9 +5103,10 @@ private Mono>> poolChangeWithResponseAsync(String reso } else { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.poolChange(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -5577,9 +5331,11 @@ private Mono>> relocateWithResponseAsync(String resour if (body != null) { body.validate(); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.relocate(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -5626,9 +5382,10 @@ private Mono>> relocateWithResponseAsync(String resour if (body != null) { body.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.relocate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), body, context); + accountName, poolName, volumeName, this.client.getApiVersion(), body, accept, context); } /** @@ -5893,10 +5650,11 @@ private Mono>> finalizeRelocationWithResponseAsync(Str if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext( context -> service.finalizeRelocation(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -5939,9 +5697,10 @@ private Mono>> finalizeRelocationWithResponseAsync(Str if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.finalizeRelocation(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); } /** @@ -6153,9 +5912,10 @@ private Mono>> revertRelocationWithResponseAsync(Strin if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; return FluxUtil .withContext(context -> service.revertRelocation(this.client.getEndpoint(), this.client.getSubscriptionId(), - resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), context)) + resourceGroupName, accountName, poolName, volumeName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -6198,9 +5958,10 @@ private Mono>> revertRelocationWithResponseAsync(Strin if (volumeName == null) { return Mono.error(new IllegalArgumentException("Parameter volumeName is required and cannot be null.")); } + final String accept = "application/json"; context = this.client.mergeContext(context); return service.revertRelocation(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, - accountName, poolName, volumeName, this.client.getApiVersion(), context); + accountName, poolName, volumeName, this.client.getApiVersion(), accept, context); } /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesImpl.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesImpl.java index 370e175b67112..87039bca03046 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesImpl.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/implementation/VolumesImpl.java @@ -42,12 +42,12 @@ public VolumesImpl(VolumesClient innerClient, com.azure.resourcemanager.netapp.N public PagedIterable list(String resourceGroupName, String accountName, String poolName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, poolName); - return Utils.mapPage(inner, inner1 -> new VolumeImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VolumeImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String accountName, String poolName, Context context) { PagedIterable inner = this.serviceClient().list(resourceGroupName, accountName, poolName, context); - return Utils.mapPage(inner, inner1 -> new VolumeImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VolumeImpl(inner1, this.manager())); } public Response getWithResponse(String resourceGroupName, String accountName, String poolName, @@ -121,15 +121,6 @@ public void resetCifsPassword(String resourceGroupName, String accountName, Stri this.serviceClient().resetCifsPassword(resourceGroupName, accountName, poolName, volumeName, context); } - public void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName) { - this.serviceClient().splitCloneFromParent(resourceGroupName, accountName, poolName, volumeName); - } - - public void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName, - Context context) { - this.serviceClient().splitCloneFromParent(resourceGroupName, accountName, poolName, volumeName, context); - } - public void breakFileLocks(String resourceGroupName, String accountName, String poolName, String volumeName) { this.serviceClient().breakFileLocks(resourceGroupName, accountName, poolName, volumeName); } @@ -208,14 +199,14 @@ public PagedIterable listReplications(String resourceGroupName, Str String volumeName) { PagedIterable inner = this.serviceClient().listReplications(resourceGroupName, accountName, poolName, volumeName); - return Utils.mapPage(inner, inner1 -> new ReplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ReplicationImpl(inner1, this.manager())); } public PagedIterable listReplications(String resourceGroupName, String accountName, String poolName, String volumeName, Context context) { PagedIterable inner = this.serviceClient().listReplications(resourceGroupName, accountName, poolName, volumeName, context); - return Utils.mapPage(inner, inner1 -> new ReplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ReplicationImpl(inner1, this.manager())); } public void resyncReplication(String resourceGroupName, String accountName, String poolName, String volumeName) { @@ -294,22 +285,22 @@ public void revertRelocation(String resourceGroupName, String accountName, Strin } public Volume getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); @@ -318,22 +309,22 @@ public Volume getById(String id) { } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); @@ -342,22 +333,22 @@ public Response getByIdWithResponse(String id, Context context) { } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); @@ -367,22 +358,22 @@ public void deleteById(String id) { } public void deleteByIdWithResponse(String id, Boolean forceDelete, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String accountName = Utils.getValueFromIdByName(id, "netAppAccounts"); + String accountName = ResourceManagerUtils.getValueFromIdByName(id, "netAppAccounts"); if (accountName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'netAppAccounts'.", id))); } - String poolName = Utils.getValueFromIdByName(id, "capacityPools"); + String poolName = ResourceManagerUtils.getValueFromIdByName(id, "capacityPools"); if (poolName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'capacityPools'.", id))); } - String volumeName = Utils.getValueFromIdByName(id, "volumes"); + String volumeName = ResourceManagerUtils.getValueFromIdByName(id, "volumes"); if (volumeName == null) { throw LOGGER.logExceptionAsError(new IllegalArgumentException( String.format("The resource ID '%s' is not valid. Missing path segment 'volumes'.", id))); diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Accounts.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Accounts.java index e989c99f25957..e7c8474d99a2b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Accounts.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Accounts.java @@ -149,37 +149,6 @@ Response getByResourceGroupWithResponse(String resourceGroupName, */ void renewCredentials(String resourceGroupName, String accountName, Context context); - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void migrateEncryptionKey(String resourceGroupName, String accountName); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void migrateEncryptionKey(String resourceGroupName, String accountName, EncryptionMigrationRequest body, - Context context); - /** * Describe a NetApp Account * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backup.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backup.java deleted file mode 100644 index b3e82fc922993..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backup.java +++ /dev/null @@ -1,311 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import java.time.OffsetDateTime; - -/** - * An immutable client-side representation of Backup. - */ -public interface Backup { - /** - * Gets the id property: Fully qualified resource Id for the resource. - * - * @return the id value. - */ - String id(); - - /** - * Gets the name property: The name of the resource. - * - * @return the name value. - */ - String name(); - - /** - * Gets the type property: The type of the resource. - * - * @return the type value. - */ - String type(); - - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - - /** - * Gets the backupId property: backupId - * - * UUID v4 used to identify the Backup. - * - * @return the backupId value. - */ - String backupId(); - - /** - * Gets the creationDate property: creationDate - * - * The creation date of the backup. - * - * @return the creationDate value. - */ - OffsetDateTime creationDate(); - - /** - * Gets the provisioningState property: Azure lifecycle management. - * - * @return the provisioningState value. - */ - String provisioningState(); - - /** - * Gets the size property: Size of backup in bytes. - * - * @return the size value. - */ - Long size(); - - /** - * Gets the label property: Label for backup. - * - * @return the label value. - */ - String label(); - - /** - * Gets the backupType property: backupType - * - * Type of backup Manual or Scheduled. - * - * @return the backupType value. - */ - BackupType backupType(); - - /** - * Gets the failureReason property: Failure reason. - * - * @return the failureReason value. - */ - String failureReason(); - - /** - * Gets the volumeResourceId property: ResourceId used to identify the Volume. - * - * @return the volumeResourceId value. - */ - String volumeResourceId(); - - /** - * Gets the useExistingSnapshot property: Manual backup an already existing snapshot. This will always be false for - * scheduled backups and true/false for manual backups. - * - * @return the useExistingSnapshot value. - */ - Boolean useExistingSnapshot(); - - /** - * Gets the snapshotName property: The name of the snapshot. - * - * @return the snapshotName value. - */ - String snapshotName(); - - /** - * Gets the backupPolicyResourceId property: ResourceId used to identify the backup policy. - * - * @return the backupPolicyResourceId value. - */ - String backupPolicyResourceId(); - - /** - * Gets the name of the resource group. - * - * @return the name of the resource group. - */ - String resourceGroupName(); - - /** - * Gets the inner com.azure.resourcemanager.netapp.fluent.models.BackupInner object. - * - * @return the inner object. - */ - BackupInner innerModel(); - - /** - * The entirety of the Backup definition. - */ - interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, - DefinitionStages.WithVolumeResourceId, DefinitionStages.WithCreate { - } - - /** - * The Backup definition stages. - */ - interface DefinitionStages { - /** - * The first stage of the Backup definition. - */ - interface Blank extends WithParentResource { - } - - /** - * The stage of the Backup definition allowing to specify parent resource. - */ - interface WithParentResource { - /** - * Specifies resourceGroupName, accountName, backupVaultName. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @return the next definition stage. - */ - WithVolumeResourceId withExistingBackupVault(String resourceGroupName, String accountName, - String backupVaultName); - } - - /** - * The stage of the Backup definition allowing to specify volumeResourceId. - */ - interface WithVolumeResourceId { - /** - * Specifies the volumeResourceId property: ResourceId used to identify the Volume. - * - * @param volumeResourceId ResourceId used to identify the Volume. - * @return the next definition stage. - */ - WithCreate withVolumeResourceId(String volumeResourceId); - } - - /** - * The stage of the Backup definition which contains all the minimum required properties for the resource to be - * created, but also allows for any other optional properties to be specified. - */ - interface WithCreate extends DefinitionStages.WithLabel, DefinitionStages.WithUseExistingSnapshot, - DefinitionStages.WithSnapshotName { - /** - * Executes the create request. - * - * @return the created resource. - */ - Backup create(); - - /** - * Executes the create request. - * - * @param context The context to associate with this operation. - * @return the created resource. - */ - Backup create(Context context); - } - - /** - * The stage of the Backup definition allowing to specify label. - */ - interface WithLabel { - /** - * Specifies the label property: Label for backup. - * - * @param label Label for backup. - * @return the next definition stage. - */ - WithCreate withLabel(String label); - } - - /** - * The stage of the Backup definition allowing to specify useExistingSnapshot. - */ - interface WithUseExistingSnapshot { - /** - * Specifies the useExistingSnapshot property: Manual backup an already existing snapshot. This will always - * be false for scheduled backups and true/false for manual backups. - * - * @param useExistingSnapshot Manual backup an already existing snapshot. This will always be false for - * scheduled backups and true/false for manual backups. - * @return the next definition stage. - */ - WithCreate withUseExistingSnapshot(Boolean useExistingSnapshot); - } - - /** - * The stage of the Backup definition allowing to specify snapshotName. - */ - interface WithSnapshotName { - /** - * Specifies the snapshotName property: The name of the snapshot. - * - * @param snapshotName The name of the snapshot. - * @return the next definition stage. - */ - WithCreate withSnapshotName(String snapshotName); - } - } - - /** - * Begins update for the Backup resource. - * - * @return the stage of resource update. - */ - Backup.Update update(); - - /** - * The template for Backup update. - */ - interface Update extends UpdateStages.WithLabel { - /** - * Executes the update request. - * - * @return the updated resource. - */ - Backup apply(); - - /** - * Executes the update request. - * - * @param context The context to associate with this operation. - * @return the updated resource. - */ - Backup apply(Context context); - } - - /** - * The Backup update stages. - */ - interface UpdateStages { - /** - * The stage of the Backup update allowing to specify label. - */ - interface WithLabel { - /** - * Specifies the label property: Label for backup. - * - * @param label Label for backup. - * @return the next definition stage. - */ - Update withLabel(String label); - } - } - - /** - * Refreshes the resource to sync with Azure. - * - * @return the refreshed resource. - */ - Backup refresh(); - - /** - * Refreshes the resource to sync with Azure. - * - * @param context The context to associate with this operation. - * @return the refreshed resource. - */ - Backup refresh(Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupPatch.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupPatch.java deleted file mode 100644 index 10b95c292fe38..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupPatch.java +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.netapp.fluent.models.BackupPatchProperties; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Backup patch. - */ -@Fluent -public final class BackupPatch { - /* - * Backup Patch Properties - */ - @JsonProperty(value = "properties") - private BackupPatchProperties innerProperties; - - /** - * Creates an instance of BackupPatch class. - */ - public BackupPatch() { - } - - /** - * Get the innerProperties property: Backup Patch Properties. - * - * @return the innerProperties value. - */ - private BackupPatchProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the label property: Label for backup. - * - * @return the label value. - */ - public String label() { - return this.innerProperties() == null ? null : this.innerProperties().label(); - } - - /** - * Set the label property: Label for backup. - * - * @param label the label value to set. - * @return the BackupPatch object itself. - */ - public BackupPatch withLabel(String label) { - if (this.innerProperties() == null) { - this.innerProperties = new BackupPatchProperties(); - } - this.innerProperties().withLabel(label); - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupRestoreFiles.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupRestoreFiles.java deleted file mode 100644 index c9e02848aff9c..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupRestoreFiles.java +++ /dev/null @@ -1,125 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** - * Restore payload for Single File Backup Restore. - */ -@Fluent -public final class BackupRestoreFiles { - /* - * List of files to be restored - */ - @JsonProperty(value = "fileList", required = true) - private List fileList; - - /* - * Destination folder where the files will be restored. The path name should start with a forward slash. If it is - * omitted from request then restore is done at the root folder of the destination volume by default - */ - @JsonProperty(value = "restoreFilePath") - private String restoreFilePath; - - /* - * Resource Id of the destination volume on which the files need to be restored - */ - @JsonProperty(value = "destinationVolumeId", required = true) - private String destinationVolumeId; - - /** - * Creates an instance of BackupRestoreFiles class. - */ - public BackupRestoreFiles() { - } - - /** - * Get the fileList property: List of files to be restored. - * - * @return the fileList value. - */ - public List fileList() { - return this.fileList; - } - - /** - * Set the fileList property: List of files to be restored. - * - * @param fileList the fileList value to set. - * @return the BackupRestoreFiles object itself. - */ - public BackupRestoreFiles withFileList(List fileList) { - this.fileList = fileList; - return this; - } - - /** - * Get the restoreFilePath property: Destination folder where the files will be restored. The path name should - * start with a forward slash. If it is omitted from request then restore is done at the root folder of the - * destination volume by default. - * - * @return the restoreFilePath value. - */ - public String restoreFilePath() { - return this.restoreFilePath; - } - - /** - * Set the restoreFilePath property: Destination folder where the files will be restored. The path name should - * start with a forward slash. If it is omitted from request then restore is done at the root folder of the - * destination volume by default. - * - * @param restoreFilePath the restoreFilePath value to set. - * @return the BackupRestoreFiles object itself. - */ - public BackupRestoreFiles withRestoreFilePath(String restoreFilePath) { - this.restoreFilePath = restoreFilePath; - return this; - } - - /** - * Get the destinationVolumeId property: Resource Id of the destination volume on which the files need to be - * restored. - * - * @return the destinationVolumeId value. - */ - public String destinationVolumeId() { - return this.destinationVolumeId; - } - - /** - * Set the destinationVolumeId property: Resource Id of the destination volume on which the files need to be - * restored. - * - * @param destinationVolumeId the destinationVolumeId value to set. - * @return the BackupRestoreFiles object itself. - */ - public BackupRestoreFiles withDestinationVolumeId(String destinationVolumeId) { - this.destinationVolumeId = destinationVolumeId; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (fileList() == null) { - throw LOGGER.logExceptionAsError( - new IllegalArgumentException("Missing required property fileList in model BackupRestoreFiles")); - } - if (destinationVolumeId() == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - "Missing required property destinationVolumeId in model BackupRestoreFiles")); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(BackupRestoreFiles.class); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupStatus.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupStatus.java deleted file mode 100644 index 7b5447a254d97..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupStatus.java +++ /dev/null @@ -1,83 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner; - -/** - * An immutable client-side representation of BackupStatus. - */ -public interface BackupStatus { - /** - * Gets the healthy property: Backup health status. - * - * @return the healthy value. - */ - Boolean healthy(); - - /** - * Gets the relationshipStatus property: Status of the backup mirror relationship. - * - * @return the relationshipStatus value. - */ - RelationshipStatus relationshipStatus(); - - /** - * Gets the mirrorState property: The status of the backup. - * - * @return the mirrorState value. - */ - MirrorState mirrorState(); - - /** - * Gets the unhealthyReason property: Reason for the unhealthy backup relationship. - * - * @return the unhealthyReason value. - */ - String unhealthyReason(); - - /** - * Gets the errorMessage property: Displays error message if the backup is in an error state. - * - * @return the errorMessage value. - */ - String errorMessage(); - - /** - * Gets the lastTransferSize property: Displays the last transfer size. - * - * @return the lastTransferSize value. - */ - Long lastTransferSize(); - - /** - * Gets the lastTransferType property: Displays the last transfer type. - * - * @return the lastTransferType value. - */ - String lastTransferType(); - - /** - * Gets the totalTransferBytes property: Displays the total bytes transferred. - * - * @return the totalTransferBytes value. - */ - Long totalTransferBytes(); - - /** - * Gets the transferProgressBytes property: Displays the total number of bytes transferred for the ongoing - * operation. - * - * @return the transferProgressBytes value. - */ - Long transferProgressBytes(); - - /** - * Gets the inner com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner object. - * - * @return the inner object. - */ - BackupStatusInner innerModel(); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupType.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupType.java deleted file mode 100644 index 840e5383705be..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupType.java +++ /dev/null @@ -1,55 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.util.ExpandableStringEnum; -import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collection; - -/** - * backupType - * - * Type of backup Manual or Scheduled. - */ -public final class BackupType extends ExpandableStringEnum { - /** - * Static value Manual for BackupType. - */ - public static final BackupType MANUAL = fromString("Manual"); - - /** - * Static value Scheduled for BackupType. - */ - public static final BackupType SCHEDULED = fromString("Scheduled"); - - /** - * Creates a new instance of BackupType value. - * - * @deprecated Use the {@link #fromString(String)} factory method. - */ - @Deprecated - public BackupType() { - } - - /** - * Creates or finds a BackupType from its string representation. - * - * @param name a name to look for. - * @return the corresponding BackupType. - */ - @JsonCreator - public static BackupType fromString(String name) { - return fromString(name, BackupType.class); - } - - /** - * Gets known BackupType values. - * - * @return known BackupType values. - */ - public static Collection values() { - return values(BackupType.class); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVault.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVault.java deleted file mode 100644 index 778124fa7ed50..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVault.java +++ /dev/null @@ -1,240 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.management.Region; -import com.azure.core.management.SystemData; -import com.azure.core.util.Context; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import java.util.Map; - -/** - * An immutable client-side representation of BackupVault. - */ -public interface BackupVault { - /** - * Gets the id property: Fully qualified resource Id for the resource. - * - * @return the id value. - */ - String id(); - - /** - * Gets the name property: The name of the resource. - * - * @return the name value. - */ - String name(); - - /** - * Gets the type property: The type of the resource. - * - * @return the type value. - */ - String type(); - - /** - * Gets the location property: The geo-location where the resource lives. - * - * @return the location value. - */ - String location(); - - /** - * Gets the tags property: Resource tags. - * - * @return the tags value. - */ - Map tags(); - - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - - /** - * Gets the provisioningState property: Azure lifecycle management. - * - * @return the provisioningState value. - */ - String provisioningState(); - - /** - * Gets the region of the resource. - * - * @return the region of the resource. - */ - Region region(); - - /** - * Gets the name of the resource region. - * - * @return the name of the resource region. - */ - String regionName(); - - /** - * Gets the name of the resource group. - * - * @return the name of the resource group. - */ - String resourceGroupName(); - - /** - * Gets the inner com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner object. - * - * @return the inner object. - */ - BackupVaultInner innerModel(); - - /** - * The entirety of the BackupVault definition. - */ - interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, DefinitionStages.WithCreate { - } - - /** - * The BackupVault definition stages. - */ - interface DefinitionStages { - /** - * The first stage of the BackupVault definition. - */ - interface Blank extends WithLocation { - } - - /** - * The stage of the BackupVault definition allowing to specify location. - */ - interface WithLocation { - /** - * Specifies the region for the resource. - * - * @param location The geo-location where the resource lives. - * @return the next definition stage. - */ - WithParentResource withRegion(Region location); - - /** - * Specifies the region for the resource. - * - * @param location The geo-location where the resource lives. - * @return the next definition stage. - */ - WithParentResource withRegion(String location); - } - - /** - * The stage of the BackupVault definition allowing to specify parent resource. - */ - interface WithParentResource { - /** - * Specifies resourceGroupName, accountName. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @return the next definition stage. - */ - WithCreate withExistingNetAppAccount(String resourceGroupName, String accountName); - } - - /** - * The stage of the BackupVault definition which contains all the minimum required properties for the resource - * to be created, but also allows for any other optional properties to be specified. - */ - interface WithCreate extends DefinitionStages.WithTags { - /** - * Executes the create request. - * - * @return the created resource. - */ - BackupVault create(); - - /** - * Executes the create request. - * - * @param context The context to associate with this operation. - * @return the created resource. - */ - BackupVault create(Context context); - } - - /** - * The stage of the BackupVault definition allowing to specify tags. - */ - interface WithTags { - /** - * Specifies the tags property: Resource tags.. - * - * @param tags Resource tags. - * @return the next definition stage. - */ - WithCreate withTags(Map tags); - } - } - - /** - * Begins update for the BackupVault resource. - * - * @return the stage of resource update. - */ - BackupVault.Update update(); - - /** - * The template for BackupVault update. - */ - interface Update extends UpdateStages.WithTags { - /** - * Executes the update request. - * - * @return the updated resource. - */ - BackupVault apply(); - - /** - * Executes the update request. - * - * @param context The context to associate with this operation. - * @return the updated resource. - */ - BackupVault apply(Context context); - } - - /** - * The BackupVault update stages. - */ - interface UpdateStages { - /** - * The stage of the BackupVault update allowing to specify tags. - */ - interface WithTags { - /** - * Specifies the tags property: Resource tags. - * - * @param tags Resource tags. - * @return the next definition stage. - */ - Update withTags(Map tags); - } - } - - /** - * Refreshes the resource to sync with Azure. - * - * @return the refreshed resource. - */ - BackupVault refresh(); - - /** - * Refreshes the resource to sync with Azure. - * - * @param context The context to associate with this operation. - * @return the refreshed resource. - */ - BackupVault refresh(Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultPatch.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultPatch.java deleted file mode 100644 index 0a25d4bb2e7f3..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultPatch.java +++ /dev/null @@ -1,57 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Map; - -/** - * Backup Vault information. - */ -@Fluent -public final class BackupVaultPatch { - /* - * Resource tags - */ - @JsonProperty(value = "tags") - @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) - private Map tags; - - /** - * Creates an instance of BackupVaultPatch class. - */ - public BackupVaultPatch() { - } - - /** - * Get the tags property: Resource tags. - * - * @return the tags value. - */ - public Map tags() { - return this.tags; - } - - /** - * Set the tags property: Resource tags. - * - * @param tags the tags value to set. - * @return the BackupVaultPatch object itself. - */ - public BackupVaultPatch withTags(Map tags) { - this.tags = tags; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backups.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backups.java index 918e26f409719..f80539951799e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backups.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Backups.java @@ -4,7 +4,6 @@ package com.azure.resourcemanager.netapp.models; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; @@ -12,40 +11,6 @@ * Resource collection API of Backups. */ public interface Backups { - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume along with {@link Response}. - */ - Response getLatestStatusWithResponse(String resourceGroupName, String accountName, String poolName, - String volumeName, Context context); - - /** - * Get the latest backup status of a volume - * - * Get the latest status of the backup for a volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the latest status of the backup for a volume. - */ - BackupStatus getLatestStatus(String resourceGroupName, String accountName, String poolName, String volumeName); - /** * Get volume's restore status * @@ -80,164 +45,4 @@ Response getVolumeRestoreStatusWithResponse(String resourceGroupN */ RestoreStatus getVolumeRestoreStatus(String resourceGroupName, String accountName, String poolName, String volumeName); - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. - */ - PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName); - - /** - * List Backups - * - * List all backups Under a Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param filter An option to specify the VolumeResourceId. If present, then only returns the backups under the - * specified volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Backups as paginated response with {@link PagedIterable}. - */ - PagedIterable listByVault(String resourceGroupName, String accountName, String backupVaultName, - String filter, Context context); - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response}. - */ - Response getWithResponse(String resourceGroupName, String accountName, String backupVaultName, - String backupName, Context context); - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault. - */ - Backup get(String resourceGroupName, String accountName, String backupVaultName, String backupName); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void delete(String resourceGroupName, String accountName, String backupVaultName, String backupName, - Context context); - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param id the resource ID. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response}. - */ - Backup getById(String id); - - /** - * Describe the Backup under Backup Vault - * - * Get the specified Backup under Backup Vault. - * - * @param id the resource ID. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified Backup under Backup Vault along with {@link Response}. - */ - Response getByIdWithResponse(String id, Context context); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param id the resource ID. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void deleteById(String id); - - /** - * Delete backup - * - * Delete a Backup under the Backup Vault. - * - * @param id the resource ID. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void deleteByIdWithResponse(String id, Context context); - - /** - * Begins definition for a new Backup resource. - * - * @param name resource name. - * @return the first stage of the new Backup definition. - */ - Backup.DefinitionStages.Blank define(String name); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsMigrationRequest.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsMigrationRequest.java deleted file mode 100644 index 6db10d82b8a08..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsMigrationRequest.java +++ /dev/null @@ -1,61 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Migrate Backups Request. - */ -@Fluent -public final class BackupsMigrationRequest { - /* - * The ResourceId of the Backup Vault - */ - @JsonProperty(value = "backupVaultId", required = true) - private String backupVaultId; - - /** - * Creates an instance of BackupsMigrationRequest class. - */ - public BackupsMigrationRequest() { - } - - /** - * Get the backupVaultId property: The ResourceId of the Backup Vault. - * - * @return the backupVaultId value. - */ - public String backupVaultId() { - return this.backupVaultId; - } - - /** - * Set the backupVaultId property: The ResourceId of the Backup Vault. - * - * @param backupVaultId the backupVaultId value to set. - * @return the BackupsMigrationRequest object itself. - */ - public BackupsMigrationRequest withBackupVaultId(String backupVaultId) { - this.backupVaultId = backupVaultId; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (backupVaultId() == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - "Missing required property backupVaultId in model BackupsMigrationRequest")); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(BackupsMigrationRequest.class); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderBackupVaults.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderBackupVaults.java deleted file mode 100644 index 45e4d720539bc..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderBackupVaults.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.util.Context; - -/** - * Resource collection API of BackupsUnderBackupVaults. - */ -public interface BackupsUnderBackupVaults { - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body); - - /** - * Create a new Backup Restore Files request - * - * Restore the specified files from the specified backup to the active filesystem. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param backupVaultName The name of the Backup Vault. - * @param backupName The name of the backup. - * @param body Restore payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void restoreFiles(String resourceGroupName, String accountName, String backupVaultName, String backupName, - BackupRestoreFiles body, Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderVolumes.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderVolumes.java deleted file mode 100644 index d277aa4b64d21..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsUnderVolumes.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.util.Context; - -/** - * Resource collection API of BackupsUnderVolumes. - */ -public interface BackupsUnderVolumes { - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body); - - /** - * Create a new migrate request for backups under volume - * - * Migrate the backups under volume to backup vault. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param body Migrate backups under volume payload supplied in the body of the operation. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void migrateBackups(String resourceGroupName, String accountName, String poolName, String volumeName, - BackupsMigrationRequest body, Context context); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/EncryptionMigrationRequest.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/EncryptionMigrationRequest.java deleted file mode 100644 index 2c849f388eeb5..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/EncryptionMigrationRequest.java +++ /dev/null @@ -1,103 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Encryption migration request. - */ -@Fluent -public final class EncryptionMigrationRequest { - /* - * virtualNetworkId - * - * Identifier for the virtual network - */ - @JsonProperty(value = "virtualNetworkId", required = true) - private String virtualNetworkId; - - /* - * privateEndpointId - * - * Identifier of the private endpoint to reach the Azure Key Vault - */ - @JsonProperty(value = "privateEndpointId", required = true) - private String privateEndpointId; - - /** - * Creates an instance of EncryptionMigrationRequest class. - */ - public EncryptionMigrationRequest() { - } - - /** - * Get the virtualNetworkId property: virtualNetworkId - * - * Identifier for the virtual network. - * - * @return the virtualNetworkId value. - */ - public String virtualNetworkId() { - return this.virtualNetworkId; - } - - /** - * Set the virtualNetworkId property: virtualNetworkId - * - * Identifier for the virtual network. - * - * @param virtualNetworkId the virtualNetworkId value to set. - * @return the EncryptionMigrationRequest object itself. - */ - public EncryptionMigrationRequest withVirtualNetworkId(String virtualNetworkId) { - this.virtualNetworkId = virtualNetworkId; - return this; - } - - /** - * Get the privateEndpointId property: privateEndpointId - * - * Identifier of the private endpoint to reach the Azure Key Vault. - * - * @return the privateEndpointId value. - */ - public String privateEndpointId() { - return this.privateEndpointId; - } - - /** - * Set the privateEndpointId property: privateEndpointId - * - * Identifier of the private endpoint to reach the Azure Key Vault. - * - * @param privateEndpointId the privateEndpointId value to set. - * @return the EncryptionMigrationRequest object itself. - */ - public EncryptionMigrationRequest withPrivateEndpointId(String privateEndpointId) { - this.privateEndpointId = privateEndpointId; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (virtualNetworkId() == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - "Missing required property virtualNetworkId in model EncryptionMigrationRequest")); - } - if (privateEndpointId() == null) { - throw LOGGER.logExceptionAsError(new IllegalArgumentException( - "Missing required property privateEndpointId in model EncryptionMigrationRequest")); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(EncryptionMigrationRequest.class); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccount.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccount.java index 3d8808ca314cc..867609d0b592a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccount.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccount.java @@ -100,21 +100,6 @@ public interface NetAppAccount { */ Boolean disableShowmount(); - /** - * Gets the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @return the nfsV4IdDomain value. - */ - String nfsV4IdDomain(); - - /** - * Gets the isMultiAdEnabled property: This will have true value only if account is Multiple AD enabled. - * - * @return the isMultiAdEnabled value. - */ - Boolean isMultiAdEnabled(); - /** * Gets the region of the resource. * @@ -198,9 +183,8 @@ interface WithResourceGroup { * The stage of the NetAppAccount definition which contains all the minimum required properties for the resource * to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, DefinitionStages.WithActiveDirectories, - DefinitionStages.WithEncryption, DefinitionStages.WithNfsV4IdDomain { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithActiveDirectories, DefinitionStages.WithEncryption { /** * Executes the create request. * @@ -268,21 +252,6 @@ interface WithEncryption { */ WithCreate withEncryption(AccountEncryption encryption); } - - /** - * The stage of the NetAppAccount definition allowing to specify nfsV4IdDomain. - */ - interface WithNfsV4IdDomain { - /** - * Specifies the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all - * NetApp accounts in the subscription and region and only affect non ldap NFSv4 volumes.. - * - * @param nfsV4IdDomain Domain for NFSv4 user ID mapping. This property will be set for all NetApp accounts - * in the subscription and region and only affect non ldap NFSv4 volumes. - * @return the next definition stage. - */ - WithCreate withNfsV4IdDomain(String nfsV4IdDomain); - } } /** @@ -296,7 +265,7 @@ interface WithNfsV4IdDomain { * The template for NetAppAccount update. */ interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity, UpdateStages.WithActiveDirectories, - UpdateStages.WithEncryption, UpdateStages.WithNfsV4IdDomain { + UpdateStages.WithEncryption { /** * Executes the update request. * @@ -368,21 +337,6 @@ interface WithEncryption { */ Update withEncryption(AccountEncryption encryption); } - - /** - * The stage of the NetAppAccount update allowing to specify nfsV4IdDomain. - */ - interface WithNfsV4IdDomain { - /** - * Specifies the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all - * NetApp accounts in the subscription and region and only affect non ldap NFSv4 volumes.. - * - * @param nfsV4IdDomain Domain for NFSv4 user ID mapping. This property will be set for all NetApp accounts - * in the subscription and region and only affect non ldap NFSv4 volumes. - * @return the next definition stage. - */ - Update withNfsV4IdDomain(String nfsV4IdDomain); - } } /** @@ -423,29 +377,4 @@ interface WithNfsV4IdDomain { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ void renewCredentials(Context context); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void migrateEncryptionKey(); - - /** - * Migrate volumes encryption key source. - * - * Migrates all volumes in a VNet to a different encryption key source (Microsoft-managed key or Azure Key Vault). - * Operation fails if targeted volumes share encryption sibling set with volumes from another account. - * - * @param body The required parameters to perform encryption migration. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void migrateEncryptionKey(EncryptionMigrationRequest body, Context context); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccountPatch.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccountPatch.java index b33a3abb2b2b8..551c73c4e8f0f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccountPatch.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/NetAppAccountPatch.java @@ -146,40 +146,6 @@ public Boolean disableShowmount() { return this.innerProperties() == null ? null : this.innerProperties().disableShowmount(); } - /** - * Get the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @return the nfsV4IdDomain value. - */ - public String nfsV4IdDomain() { - return this.innerProperties() == null ? null : this.innerProperties().nfsV4IdDomain(); - } - - /** - * Set the nfsV4IdDomain property: Domain for NFSv4 user ID mapping. This property will be set for all NetApp - * accounts in the subscription and region and only affect non ldap NFSv4 volumes. - * - * @param nfsV4IdDomain the nfsV4IdDomain value to set. - * @return the NetAppAccountPatch object itself. - */ - public NetAppAccountPatch withNfsV4IdDomain(String nfsV4IdDomain) { - if (this.innerProperties() == null) { - this.innerProperties = new AccountProperties(); - } - this.innerProperties().withNfsV4IdDomain(nfsV4IdDomain); - return this; - } - - /** - * Get the isMultiAdEnabled property: This will have true value only if account is Multiple AD enabled. - * - * @return the isMultiAdEnabled value. - */ - public Boolean isMultiAdEnabled() { - return this.innerProperties() == null ? null : this.innerProperties().isMultiAdEnabled(); - } - /** * Validates the instance. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfoResource.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfoResource.java deleted file mode 100644 index 2b38b85fc6c4c..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfoResource.java +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.models; - -import com.azure.core.management.SystemData; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; -import java.util.List; - -/** - * An immutable client-side representation of RegionInfoResource. - */ -public interface RegionInfoResource { - /** - * Gets the id property: Fully qualified resource Id for the resource. - * - * @return the id value. - */ - String id(); - - /** - * Gets the name property: The name of the resource. - * - * @return the name value. - */ - String name(); - - /** - * Gets the type property: The type of the resource. - * - * @return the type value. - */ - String type(); - - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - - /** - * Gets the storageToNetworkProximity property: Storage to Network Proximity - * - * Provides storage to network proximity information in the region. - * - * @return the storageToNetworkProximity value. - */ - RegionStorageToNetworkProximity storageToNetworkProximity(); - - /** - * Gets the availabilityZoneMappings property: Logical availability zone mappings. - * - * Provides logical availability zone mappings for the subscription for a region. - * - * @return the availabilityZoneMappings value. - */ - List availabilityZoneMappings(); - - /** - * Gets the inner com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner object. - * - * @return the inner object. - */ - RegionInfoResourceInner innerModel(); -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RelationshipStatus.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RelationshipStatus.java index f927276472843..830b2531dee9f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RelationshipStatus.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RelationshipStatus.java @@ -22,6 +22,16 @@ public final class RelationshipStatus extends ExpandableStringEnum mountTargets() { /** * Get the volumeType property: What type of volume is this. For destination volumes in Cross Region Replication, - * set type to DataProtection. For creating clone volume, set type to ShortTermClone. + * set type to DataProtection. * * @return the volumeType value. */ @@ -465,7 +465,7 @@ public String volumeType() { /** * Set the volumeType property: What type of volume is this. For destination volumes in Cross Region Replication, - * set type to DataProtection. For creating clone volume, set type to ShortTermClone. + * set type to DataProtection. * * @param volumeType the volumeType value to set. * @return the VolumeGroupVolumeProperties object itself. @@ -1276,17 +1276,6 @@ public String originatingResourceId() { return this.innerProperties() == null ? null : this.innerProperties().originatingResourceId(); } - /** - * Get the inheritedSizeInBytes property: inheritedSizeInBytes - * - * Space shared by short term clone volume with parent volume in bytes. - * - * @return the inheritedSizeInBytes value. - */ - public Long inheritedSizeInBytes() { - return this.innerProperties() == null ? null : this.innerProperties().inheritedSizeInBytes(); - } - /** * Validates the instance. * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePatchPropertiesDataProtection.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePatchPropertiesDataProtection.java index fde5158f3c885..4c88558e735d0 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePatchPropertiesDataProtection.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePatchPropertiesDataProtection.java @@ -14,12 +14,6 @@ */ @Fluent public final class VolumePatchPropertiesDataProtection { - /* - * Backup Properties - */ - @JsonProperty(value = "backup") - private VolumeBackupProperties backup; - /* * Snapshot properties. */ @@ -32,26 +26,6 @@ public final class VolumePatchPropertiesDataProtection { public VolumePatchPropertiesDataProtection() { } - /** - * Get the backup property: Backup Properties. - * - * @return the backup value. - */ - public VolumeBackupProperties backup() { - return this.backup; - } - - /** - * Set the backup property: Backup Properties. - * - * @param backup the backup value to set. - * @return the VolumePatchPropertiesDataProtection object itself. - */ - public VolumePatchPropertiesDataProtection withBackup(VolumeBackupProperties backup) { - this.backup = backup; - return this; - } - /** * Get the snapshot property: Snapshot properties. * @@ -78,9 +52,6 @@ public VolumePatchPropertiesDataProtection withSnapshot(VolumeSnapshotProperties * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (backup() != null) { - backup().validate(); - } if (snapshot() != null) { snapshot().validate(); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePropertiesDataProtection.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePropertiesDataProtection.java index 5203d2249da71..81555a2926109 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePropertiesDataProtection.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/VolumePropertiesDataProtection.java @@ -14,12 +14,6 @@ */ @Fluent public final class VolumePropertiesDataProtection { - /* - * Backup Properties - */ - @JsonProperty(value = "backup") - private VolumeBackupProperties backup; - /* * Replication properties */ @@ -44,26 +38,6 @@ public final class VolumePropertiesDataProtection { public VolumePropertiesDataProtection() { } - /** - * Get the backup property: Backup Properties. - * - * @return the backup value. - */ - public VolumeBackupProperties backup() { - return this.backup; - } - - /** - * Set the backup property: Backup Properties. - * - * @param backup the backup value to set. - * @return the VolumePropertiesDataProtection object itself. - */ - public VolumePropertiesDataProtection withBackup(VolumeBackupProperties backup) { - this.backup = backup; - return this; - } - /** * Get the replication property: Replication properties. * @@ -130,9 +104,6 @@ public VolumePropertiesDataProtection withVolumeRelocation(VolumeRelocationPrope * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (backup() != null) { - backup().validate(); - } if (replication() != null) { replication().validate(); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Volumes.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Volumes.java index 9ee261040e1c0..94599e770d461 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Volumes.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/Volumes.java @@ -211,38 +211,6 @@ void revert(String resourceGroupName, String accountName, String poolName, Strin void resetCifsPassword(String resourceGroupName, String accountName, String poolName, String volumeName, Context context); - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName); - - /** - * Split clone from parent volume - * - * Split operation to convert clone volume to an independent volume. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param accountName The name of the NetApp account. - * @param poolName The name of the capacity pool. - * @param volumeName The name of the volume. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void splitCloneFromParent(String resourceGroupName, String accountName, String poolName, String volumeName, - Context context); - /** * Break file locks * diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/module-info.java b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/module-info.java index 58a55bf7c72cf..4329b7cbfb93b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/module-info.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/java/module-info.java @@ -4,12 +4,10 @@ module com.azure.resourcemanager.netapp { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.netapp; exports com.azure.resourcemanager.netapp.fluent; exports com.azure.resourcemanager.netapp.fluent.models; exports com.azure.resourcemanager.netapp.models; - opens com.azure.resourcemanager.netapp.fluent.models to com.azure.core, com.fasterxml.jackson.databind; opens com.azure.resourcemanager.netapp.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/proxy-config.json b/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/proxy-config.json index da3590a492f28..98e17e49daaae 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/proxy-config.json +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/proxy-config.json @@ -1 +1 @@ -[ [ "com.azure.resourcemanager.netapp.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.netapp.implementation.NetAppResourcesClientImpl$NetAppResourcesService" ], [ "com.azure.resourcemanager.netapp.implementation.NetAppResourceQuotaLimitsClientImpl$NetAppResourceQuotaLimitsService" ], [ "com.azure.resourcemanager.netapp.implementation.NetAppResourceRegionInfosClientImpl$NetAppResourceRegionInfosService" ], [ "com.azure.resourcemanager.netapp.implementation.AccountsClientImpl$AccountsService" ], [ "com.azure.resourcemanager.netapp.implementation.PoolsClientImpl$PoolsService" ], [ "com.azure.resourcemanager.netapp.implementation.VolumesClientImpl$VolumesService" ], [ "com.azure.resourcemanager.netapp.implementation.SnapshotsClientImpl$SnapshotsService" ], [ "com.azure.resourcemanager.netapp.implementation.SnapshotPoliciesClientImpl$SnapshotPoliciesService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupsClientImpl$BackupsService" ], [ "com.azure.resourcemanager.netapp.implementation.AccountBackupsClientImpl$AccountBackupsService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupPoliciesClientImpl$BackupPoliciesService" ], [ "com.azure.resourcemanager.netapp.implementation.VolumeQuotaRulesClientImpl$VolumeQuotaRulesService" ], [ "com.azure.resourcemanager.netapp.implementation.VolumeGroupsClientImpl$VolumeGroupsService" ], [ "com.azure.resourcemanager.netapp.implementation.SubvolumesClientImpl$SubvolumesService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupVaultsClientImpl$BackupVaultsService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupsUnderBackupVaultsClientImpl$BackupsUnderBackupVaultsService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupsUnderVolumesClientImpl$BackupsUnderVolumesService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupsUnderAccountsClientImpl$BackupsUnderAccountsService" ] ] \ No newline at end of file +[ [ "com.azure.resourcemanager.netapp.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.netapp.implementation.NetAppResourcesClientImpl$NetAppResourcesService" ], [ "com.azure.resourcemanager.netapp.implementation.NetAppResourceQuotaLimitsClientImpl$NetAppResourceQuotaLimitsService" ], [ "com.azure.resourcemanager.netapp.implementation.AccountsClientImpl$AccountsService" ], [ "com.azure.resourcemanager.netapp.implementation.PoolsClientImpl$PoolsService" ], [ "com.azure.resourcemanager.netapp.implementation.VolumesClientImpl$VolumesService" ], [ "com.azure.resourcemanager.netapp.implementation.SnapshotsClientImpl$SnapshotsService" ], [ "com.azure.resourcemanager.netapp.implementation.SnapshotPoliciesClientImpl$SnapshotPoliciesService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupsClientImpl$BackupsService" ], [ "com.azure.resourcemanager.netapp.implementation.BackupPoliciesClientImpl$BackupPoliciesService" ], [ "com.azure.resourcemanager.netapp.implementation.VolumeQuotaRulesClientImpl$VolumeQuotaRulesService" ], [ "com.azure.resourcemanager.netapp.implementation.VolumeGroupsClientImpl$VolumeGroupsService" ], [ "com.azure.resourcemanager.netapp.implementation.SubvolumesClientImpl$SubvolumesService" ] ] \ No newline at end of file diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/reflect-config.json b/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/reflect-config.json index 853d44aed642f..87eaa157228e2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/reflect-config.json +++ b/sdk/netapp/azure-resourcemanager-netapp/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-netapp/reflect-config.json @@ -73,16 +73,6 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.RegionInfosList", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.fluent.models.RegionInfoInner", "allDeclaredConstructors" : true, @@ -168,11 +158,6 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.models.CapacityPoolList", "allDeclaredConstructors" : true, @@ -233,21 +218,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.VolumeBackupProperties", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.models.ReplicationObject", "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.RemotePath", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.models.VolumeSnapshotProperties", "allDeclaredConstructors" : true, @@ -408,41 +383,11 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.fluent.models.RestoreStatusInner", "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupsList", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.BackupInner", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.BackupProperties", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupPatch", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.BackupPatchProperties", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.models.BackupPoliciesList", "allDeclaredConstructors" : true, @@ -558,36 +503,6 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupVaultsList", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.fluent.models.BackupVaultProperties", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupVaultPatch", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupRestoreFiles", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupsMigrationRequest", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.models.MetricAggregationType", "allDeclaredConstructors" : true, @@ -728,11 +643,6 @@ "allDeclaredConstructors" : true, "allDeclaredFields" : true, "allDeclaredMethods" : true -}, { - "name" : "com.azure.resourcemanager.netapp.models.BackupType", - "allDeclaredConstructors" : true, - "allDeclaredFields" : true, - "allDeclaredMethods" : true }, { "name" : "com.azure.resourcemanager.netapp.models.Type", "allDeclaredConstructors" : true, diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsDeleteSamples.java deleted file mode 100644 index a4fbe02221ecc..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsDeleteSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for AccountBackups Delete. - */ -public final class AccountBackupsDeleteSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Backups_Account_Delete - * .json - */ - /** - * Sample code: AccountBackups_Delete. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountBackupsDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accountBackups().delete("resourceGroup", "accountName", "backupName", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsGetSamples.java deleted file mode 100644 index 15b90c2263020..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsGetSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for AccountBackups Get. - */ -public final class AccountBackupsGetSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Backups_Account_Get. - * json - */ - /** - * Sample code: AccountBackups_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountBackupsGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accountBackups().getWithResponse("myRG", "account1", "backup1", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsListByNetAppAccountSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsListByNetAppAccountSamples.java deleted file mode 100644 index 045e1c2a84711..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountBackupsListByNetAppAccountSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for AccountBackups ListByNetAppAccount. - */ -public final class AccountBackupsListByNetAppAccountSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Backups_Account_List. - * json - */ - /** - * Sample code: AccountBackups_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountBackupsList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accountBackups().listByNetAppAccount("myRG", "account1", null, com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsCreateOrUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsCreateOrUpdateSamples.java index 91bcf031ac22d..28318649eed1e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsCreateOrUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsCreateOrUpdateSamples.java @@ -12,8 +12,8 @@ */ public final class AccountsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Accounts_CreateOrUpdate.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_CreateOrUpdate.json */ /** * Sample code: Accounts_CreateOrUpdate. @@ -25,8 +25,8 @@ public static void accountsCreateOrUpdate(com.azure.resourcemanager.netapp.NetAp } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Accounts_CreateOrUpdateAD.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_CreateOrUpdateAD.json */ /** * Sample code: Accounts_CreateOrUpdateWithActiveDirectory. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsDeleteSamples.java index ea1d1b26d9ab2..314009375bc18 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsDeleteSamples.java @@ -10,7 +10,7 @@ public final class AccountsDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_Delete.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_Delete.json */ /** * Sample code: Accounts_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsGetByResourceGroupSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsGetByResourceGroupSamples.java index 8c48bc1201c20..fc0ddefcd3581 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsGetByResourceGroupSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsGetByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class AccountsGetByResourceGroupSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_Get.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_Get.json */ /** * Sample code: Accounts_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListByResourceGroupSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListByResourceGroupSamples.java index 2d2cb54c7561e..32c14bc32e1ae 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListByResourceGroupSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListByResourceGroupSamples.java @@ -10,7 +10,7 @@ public final class AccountsListByResourceGroupSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_List.json */ /** * Sample code: Accounts_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListSamples.java index d66a5cd843b9b..0e90cb3b15700 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsListSamples.java @@ -10,7 +10,7 @@ public final class AccountsListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_List.json */ /** * Sample code: Accounts_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsMigrateEncryptionKeySamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsMigrateEncryptionKeySamples.java deleted file mode 100644 index 45d252661a483..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsMigrateEncryptionKeySamples.java +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; - -/** - * Samples for Accounts MigrateEncryptionKey. - */ -public final class AccountsMigrateEncryptionKeySamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Accounts_MigrateEncryptionKey.json - */ - /** - * Sample code: Accounts_MigrateEncryptionKey. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void accountsMigrateEncryptionKey(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.accounts().migrateEncryptionKey("myRG", "account1", new EncryptionMigrationRequest() - .withVirtualNetworkId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.Network/virtualNetworks/vnet1") - .withPrivateEndpointId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.Network/privateEndpoints/privip1"), - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsSamples.java index 79efa2e75d302..5bbc4d6b5de0d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsSamples.java @@ -9,8 +9,8 @@ */ public final class AccountsRenewCredentialsSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Accounts_RenewCredentials.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_RenewCredentials.json */ /** * Sample code: Accounts_RenewCredentials. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsUpdateSamples.java index 400b099f06fe5..d3037295a9df9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/AccountsUpdateSamples.java @@ -14,7 +14,7 @@ public final class AccountsUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Accounts_Update.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Accounts_Update.json */ /** * Sample code: Accounts_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateSamples.java index e6ac5c0884262..33918be3b79d3 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateSamples.java @@ -10,8 +10,7 @@ public final class BackupPoliciesCreateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Create. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Create.json */ /** * Sample code: BackupPolicies_Create. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteSamples.java index 4cef5420c527a..a1a3cc2cbd0ae 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteSamples.java @@ -10,8 +10,7 @@ public final class BackupPoliciesDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Delete. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Delete.json */ /** * Sample code: Backups_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetSamples.java index b3a07fa1be286..b9500598b1717 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetSamples.java @@ -10,8 +10,7 @@ public final class BackupPoliciesGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Get. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Get.json */ /** * Sample code: Backups_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListSamples.java index 97dabf4282ac5..80526eeaeac36 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListSamples.java @@ -10,8 +10,7 @@ public final class BackupPoliciesListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_List. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_List.json */ /** * Sample code: Backups_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesUpdateSamples.java index 9bc3790dd1612..ad1240f60d2f3 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesUpdateSamples.java @@ -12,8 +12,7 @@ public final class BackupPoliciesUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupPolicies_Update. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/BackupPolicies_Update.json */ /** * Sample code: BackupPolicies_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateSamples.java deleted file mode 100644 index 147afd1de7e86..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsCreateOrUpdateSamples.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for BackupVaults CreateOrUpdate. - */ -public final class BackupVaultsCreateOrUpdateSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Create. - * json - */ - /** - * Sample code: BackupVault_CreateOrUpdate. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultCreateOrUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().define("backupVault1").withRegion("eastus").withExistingNetAppAccount("myRG", "account1") - .create(); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsDeleteSamples.java deleted file mode 100644 index 7a116ac9d4685..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsDeleteSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for BackupVaults Delete. - */ -public final class BackupVaultsDeleteSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Delete. - * json - */ - /** - * Sample code: BackupVaults_Delete. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().delete("resourceGroup", "account1", "backupVault1", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsGetSamples.java deleted file mode 100644 index b4d1190d8bd7e..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsGetSamples.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for BackupVaults Get. - */ -public final class BackupVaultsGetSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Get.json - */ - /** - * Sample code: BackupVaults_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().getWithResponse("myRG", "account1", "backupVault1", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListByNetAppAccountSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListByNetAppAccountSamples.java deleted file mode 100644 index 4cd6d2246b3f6..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListByNetAppAccountSamples.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for BackupVaults ListByNetAppAccount. - */ -public final class BackupVaultsListByNetAppAccountSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_List.json - */ - /** - * Sample code: BackupVaults_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupVaults().listByNetAppAccount("myRG", "account1", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsUpdateSamples.java deleted file mode 100644 index cb850971ce72c..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupVaultsUpdateSamples.java +++ /dev/null @@ -1,42 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.resourcemanager.netapp.models.BackupVault; -import java.util.HashMap; -import java.util.Map; - -/** - * Samples for BackupVaults Update. - */ -public final class BackupVaultsUpdateSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/BackupVaults_Update. - * json - */ - /** - * Sample code: BackupVaults_Update. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupVaultsUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - BackupVault resource = manager.backupVaults() - .getWithResponse("myRG", "account1", "backupVault1", com.azure.core.util.Context.NONE).getValue(); - resource.update().withTags(mapOf("Tag1", "Value1")).apply(); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsCreateSamples.java deleted file mode 100644 index 2cde03e737525..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsCreateSamples.java +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for Backups Create. - */ -public final class BackupsCreateSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderBackupVault_Create.json - */ - /** - * Sample code: BackupsUnderBackupVault_Create. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultCreate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().define("backup1").withExistingBackupVault("myRG", "account1", "backupVault1") - .withVolumeResourceId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPool/pool1/volumes/volume1") - .withLabel("myLabel").create(); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsDeleteSamples.java deleted file mode 100644 index d254bf8a1e6eb..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsDeleteSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for Backups Delete. - */ -public final class BackupsDeleteSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderBackupVault_Delete.json - */ - /** - * Sample code: BackupsUnderBackupVault_Delete. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultDelete(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().delete("resourceGroup", "account1", "backupVault1", "backup1", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusSamples.java deleted file mode 100644 index 7a4567042c132..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for Backups GetLatestStatus. - */ -public final class BackupsGetLatestStatusSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_LatestBackupStatus.json - */ - /** - * Sample code: Volumes_BackupStatus. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void volumesBackupStatus(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().getLatestStatusWithResponse("myRG", "account1", "pool1", "volume1", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetSamples.java deleted file mode 100644 index 3332175159c8e..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetSamples.java +++ /dev/null @@ -1,24 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for Backups Get. - */ -public final class BackupsGetSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderBackupVault_Get.json - */ - /** - * Sample code: BackupsUnderBackupVault_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().getWithResponse("myRG", "account1", "backupVault1", "backup1", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusSamples.java index ce7f7aac31b84..5fb2dc0dc12e2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusSamples.java @@ -10,8 +10,7 @@ public final class BackupsGetVolumeRestoreStatusSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_RestoreStatus. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_RestoreStatus.json */ /** * Sample code: Volumes_RestoreStatus. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsListByVaultSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsListByVaultSamples.java deleted file mode 100644 index 3f1cf23aaea08..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsListByVaultSamples.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for Backups ListByVault. - */ -public final class BackupsListByVaultSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderBackupVault_List.json - */ - /** - * Sample code: Backups_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backups().listByVault("myRG", "account1", "backupVault1", null, com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderAccountMigrateBackupsSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderAccountMigrateBackupsSamples.java deleted file mode 100644 index c6eeda7592f0a..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderAccountMigrateBackupsSamples.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; - -/** - * Samples for BackupsUnderAccount MigrateBackups. - */ -public final class BackupsUnderAccountMigrateBackupsSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderAccount_Migrate.json - */ - /** - * Sample code: BackupsUnderAccount_Migrate. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderAccountMigrate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupsUnderAccounts().migrateBackups("myRG", "account1", - new BackupsMigrationRequest().withBackupVaultId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/backupVaults/backupVault1"), - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderBackupVaultRestoreFilesSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderBackupVaultRestoreFilesSamples.java deleted file mode 100644 index 0aba83a038c42..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderBackupVaultRestoreFilesSamples.java +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.resourcemanager.netapp.models.BackupRestoreFiles; -import java.util.Arrays; - -/** - * Samples for BackupsUnderBackupVault RestoreFiles. - */ -public final class BackupsUnderBackupVaultRestoreFilesSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderBackupVault_SingleFileRestore.json - */ - /** - * Sample code: Backups_SingleFileRestore. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsSingleFileRestore(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupsUnderBackupVaults().restoreFiles("myRG", "account1", "backupVault1", "backup1", - new BackupRestoreFiles().withFileList(Arrays.asList("/dir1/customer1.db", "/dir1/customer2.db")) - .withDestinationVolumeId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/capacityPools/pool1/volumes/volume1"), - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderVolumeMigrateBackupsSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderVolumeMigrateBackupsSamples.java deleted file mode 100644 index c73e610df53a1..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUnderVolumeMigrateBackupsSamples.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; - -/** - * Samples for BackupsUnderVolume MigrateBackups. - */ -public final class BackupsUnderVolumeMigrateBackupsSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderVolume_Migrate.json - */ - /** - * Sample code: BackupsUnderVolume_Migrate. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderVolumeMigrate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.backupsUnderVolumes().migrateBackups("myRG", "account1", "pool1", "volume1", - new BackupsMigrationRequest().withBackupVaultId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRG/providers/Microsoft.NetApp/netAppAccounts/account1/backupVaults/backupVault1"), - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUpdateSamples.java deleted file mode 100644 index 63101b9b1529a..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/BackupsUpdateSamples.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.resourcemanager.netapp.models.Backup; - -/** - * Samples for Backups Update. - */ -public final class BackupsUpdateSamples { - /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * BackupsUnderBackupVault_Update.json - */ - /** - * Sample code: BackupsUnderBackupVault_Update. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void backupsUnderBackupVaultUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - Backup resource = manager.backups() - .getWithResponse("myRG", "account1", "backupVault1", "backup1", com.azure.core.util.Context.NONE) - .getValue(); - resource.update().apply(); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckFilePathAvailabilitySamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckFilePathAvailabilitySamples.java index fcb1cf614561a..35969ed3f9b17 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckFilePathAvailabilitySamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckFilePathAvailabilitySamples.java @@ -11,8 +11,8 @@ */ public final class NetAppResourceCheckFilePathAvailabilitySamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * CheckFilePathAvailability.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/CheckFilePathAvailability.json */ /** * Sample code: CheckFilePathAvailability. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckNameAvailabilitySamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckNameAvailabilitySamples.java index 3d0f73faf51eb..a2f88093e4b56 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckNameAvailabilitySamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckNameAvailabilitySamples.java @@ -13,8 +13,7 @@ public final class NetAppResourceCheckNameAvailabilitySamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/CheckNameAvailability. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/CheckNameAvailability.json */ /** * Sample code: CheckNameAvailability. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckQuotaAvailabilitySamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckQuotaAvailabilitySamples.java index 3b27b75f6ff30..147594cf24993 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckQuotaAvailabilitySamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceCheckQuotaAvailabilitySamples.java @@ -13,8 +13,7 @@ public final class NetAppResourceCheckQuotaAvailabilitySamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/CheckQuotaAvailability - * .json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/CheckQuotaAvailability.json */ /** * Sample code: CheckQuotaAvailability. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryNetworkSiblingSetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryNetworkSiblingSetSamples.java index 108eb1c5f2c40..9e51c370a3a18 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryNetworkSiblingSetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryNetworkSiblingSetSamples.java @@ -11,8 +11,8 @@ */ public final class NetAppResourceQueryNetworkSiblingSetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * NetworkSiblingSet_Query.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/NetworkSiblingSet_Query.json */ /** * Sample code: NetworkSiblingSet_Query. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryRegionInfoSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryRegionInfoSamples.java index c2b20645fd69e..5500bab67fbf9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryRegionInfoSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQueryRegionInfoSamples.java @@ -10,7 +10,7 @@ public final class NetAppResourceQueryRegionInfoSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/RegionInfo.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/RegionInfo.json */ /** * Sample code: RegionInfo_Query. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetSamples.java index 9dd41d5a0a89f..dca31426845d5 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetSamples.java @@ -10,7 +10,7 @@ public final class NetAppResourceQuotaLimitsGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/QuotaLimits_Get.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/QuotaLimits_Get.json */ /** * Sample code: QuotaLimits. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListSamples.java index 8d4787285e33a..66d896611a266 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListSamples.java @@ -10,7 +10,7 @@ public final class NetAppResourceQuotaLimitsListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/QuotaLimits_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/QuotaLimits_List.json */ /** * Sample code: QuotaLimits. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetSamples.java deleted file mode 100644 index 3c9c6cb38f6b2..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetSamples.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for NetAppResourceRegionInfos Get. - */ -public final class NetAppResourceRegionInfosGetSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/RegionInfos_Get.json - */ - /** - * Sample code: RegionInfos_Get. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void regionInfosGet(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResourceRegionInfos().getWithResponse("eastus", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosListSamples.java deleted file mode 100644 index 02752093ad61c..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosListSamples.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for NetAppResourceRegionInfos List. - */ -public final class NetAppResourceRegionInfosListSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/RegionInfos_List.json - */ - /** - * Sample code: RegionInfos_List. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void regionInfosList(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.netAppResourceRegionInfos().list("eastus", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceUpdateNetworkSiblingSetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceUpdateNetworkSiblingSetSamples.java index 15bbc0430173c..ed275fe11d65c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceUpdateNetworkSiblingSetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/NetAppResourceUpdateNetworkSiblingSetSamples.java @@ -12,8 +12,8 @@ */ public final class NetAppResourceUpdateNetworkSiblingSetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * NetworkSiblingSet_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/NetworkSiblingSet_Update.json */ /** * Sample code: NetworkFeatures_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/OperationsListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/OperationsListSamples.java index 463388bdd003a..5bd1d46684793 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/OperationsListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/OperationsListSamples.java @@ -10,7 +10,7 @@ public final class OperationsListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/OperationList.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/OperationList.json */ /** * Sample code: OperationList. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateSamples.java index 4869e1891f09e..a524db14bee76 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateSamples.java @@ -13,8 +13,7 @@ public final class PoolsCreateOrUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_CreateOrUpdate. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_CreateOrUpdate.json */ /** * Sample code: Pools_CreateOrUpdate. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsDeleteSamples.java index 6d8db1ddcb619..0e594238ac41e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsDeleteSamples.java @@ -10,7 +10,7 @@ public final class PoolsDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_Delete.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_Delete.json */ /** * Sample code: Pools_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsGetSamples.java index e56fddecb097b..2c51519f16ea8 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsGetSamples.java @@ -10,7 +10,7 @@ public final class PoolsGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_Get.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_Get.json */ /** * Sample code: Pools_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsListSamples.java index 2e961867fa837..8aa1eee3675fa 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsListSamples.java @@ -10,7 +10,7 @@ public final class PoolsListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_List.json */ /** * Sample code: Pools_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsUpdateSamples.java index f5fe7c55c3fa9..470901425c2c4 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/PoolsUpdateSamples.java @@ -12,7 +12,7 @@ public final class PoolsUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Pools_Update.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Pools_Update.json */ /** * Sample code: Pools_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateSamples.java index aae86f4a5b571..e5c66125722e1 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateSamples.java @@ -14,8 +14,8 @@ */ public final class SnapshotPoliciesCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * SnapshotPolicies_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Create.json */ /** * Sample code: SnapshotPolicies_Create. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteSamples.java index da7d1a44128e5..5f2a1d87cbd1f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteSamples.java @@ -9,8 +9,8 @@ */ public final class SnapshotPoliciesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * SnapshotPolicies_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Delete.json */ /** * Sample code: SnapshotPolicies_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetSamples.java index f8cde64a98e85..72db9de829206 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetSamples.java @@ -10,8 +10,7 @@ public final class SnapshotPoliciesGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_Get. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Get.json */ /** * Sample code: SnapshotPolicies_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListSamples.java index 5574a041540fe..21b5ed43e2a45 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListSamples.java @@ -10,8 +10,7 @@ public final class SnapshotPoliciesListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/SnapshotPolicies_List. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_List.json */ /** * Sample code: SnapshotPolicies_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListVolumesSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListVolumesSamples.java index a25c7e7ce5e57..a5bb38141df29 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListVolumesSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListVolumesSamples.java @@ -9,8 +9,9 @@ */ public final class SnapshotPoliciesListVolumesSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * SnapshotPolicies_ListVolumes.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_ListVolumes. + * json */ /** * Sample code: SnapshotPolicies_ListVolumes. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesUpdateSamples.java index f4a41efa5351c..812ce45e685e6 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesUpdateSamples.java @@ -15,8 +15,8 @@ */ public final class SnapshotPoliciesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * SnapshotPolicies_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/SnapshotPolicies_Update.json */ /** * Sample code: SnapshotPolicies_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsCreateSamples.java index 302ada3fab96d..cc17606d91afd 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsCreateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsCreateSamples.java @@ -10,7 +10,7 @@ public final class SnapshotsCreateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Create.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Create.json */ /** * Sample code: Snapshots_Create. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteSamples.java index 3edb41904b6d8..223702dfc220c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteSamples.java @@ -10,7 +10,7 @@ public final class SnapshotsDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Delete.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Delete.json */ /** * Sample code: Snapshots_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetSamples.java index 7585261ec6b56..e0675850165cf 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetSamples.java @@ -10,7 +10,7 @@ public final class SnapshotsGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Get.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Get.json */ /** * Sample code: Snapshots_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsListSamples.java index 63f601a28cbec..8fa136e6cbb79 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsListSamples.java @@ -10,7 +10,7 @@ public final class SnapshotsListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_List.json */ /** * Sample code: Snapshots_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesSamples.java index 74acf6151e6cb..c671af1d2dd9e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesSamples.java @@ -12,8 +12,9 @@ */ public final class SnapshotsRestoreFilesSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Snapshots_SingleFileRestore.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_SingleFileRestore. + * json */ /** * Sample code: Snapshots_SingleFileRestore. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateSamples.java index 3280c9f33ff5a..07b0fe485056e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateSamples.java @@ -14,7 +14,7 @@ public final class SnapshotsUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Snapshots_Update.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Snapshots_Update.json */ /** * Sample code: Snapshots_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateSamples.java index 83a418a35241e..cd4df6a721e78 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateSamples.java @@ -10,7 +10,7 @@ public final class SubvolumesCreateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Create.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Create.json */ /** * Sample code: Subvolumes_Create. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteSamples.java index bb11590e61ba6..2898e37d8cd25 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteSamples.java @@ -10,7 +10,7 @@ public final class SubvolumesDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Delete.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Delete.json */ /** * Sample code: Subvolumes_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataSamples.java index d9d7525de289d..a71abbde29287 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataSamples.java @@ -10,8 +10,7 @@ public final class SubvolumesGetMetadataSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Metadata. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Metadata.json */ /** * Sample code: Subvolumes_Metadata. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetSamples.java index fa28576715771..cf955d1f94858 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetSamples.java @@ -10,7 +10,7 @@ public final class SubvolumesGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Get.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Get.json */ /** * Sample code: Subvolumes_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeSamples.java index ab90eec0a574b..cdce6f7702e8c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeSamples.java @@ -10,7 +10,7 @@ public final class SubvolumesListByVolumeSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_List.json */ /** * Sample code: Subvolumes_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesUpdateSamples.java index a465d0dbca6c2..afbf8247f862b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/SubvolumesUpdateSamples.java @@ -12,7 +12,7 @@ public final class SubvolumesUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Subvolumes_Update.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Subvolumes_Update.json */ /** * Sample code: Subvolumes_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsCreateSamples.java index 0eff05d9bcf11..28accab19bf4c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsCreateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsCreateSamples.java @@ -17,8 +17,9 @@ */ public final class VolumeGroupsCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeGroups_Create_SapHana.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Create_SapHana. + * json */ /** * Sample code: VolumeGroups_Create_SapHana. @@ -120,8 +121,8 @@ public static void volumeGroupsCreateSapHana(com.azure.resourcemanager.netapp.Ne } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeGroups_Create_Oracle.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Create_Oracle.json */ /** * Sample code: VolumeGroups_Create_Oracle. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteSamples.java index bc16bf0a5d2e9..9759150243f71 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteSamples.java @@ -10,8 +10,7 @@ public final class VolumeGroupsDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeGroups_Delete. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Delete.json */ /** * Sample code: VolumeGroups_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsGetSamples.java index 2748af4fbdaa4..fc4c499282dfe 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsGetSamples.java @@ -9,8 +9,8 @@ */ public final class VolumeGroupsGetSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeGroups_Get_SapHana.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Get_SapHana.json */ /** * Sample code: VolumeGroups_Get_SapHana. @@ -22,8 +22,8 @@ public static void volumeGroupsGetSapHana(com.azure.resourcemanager.netapp.NetAp } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeGroups_Get_Oracle.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_Get_Oracle.json */ /** * Sample code: VolumeGroups_Get_Oracle. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsListByNetAppAccountSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsListByNetAppAccountSamples.java index 1a6d674bd54e2..38a07faf613e2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsListByNetAppAccountSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsListByNetAppAccountSamples.java @@ -9,8 +9,8 @@ */ public final class VolumeGroupsListByNetAppAccountSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeGroups_List_Oracle.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_List_Oracle.json */ /** * Sample code: VolumeGroups_List_Oracle. @@ -22,8 +22,8 @@ public static void volumeGroupsListOracle(com.azure.resourcemanager.netapp.NetAp } /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeGroups_List_SapHana.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeGroups_List_SapHana.json */ /** * Sample code: VolumeGroups_List_SapHana. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateSamples.java index 2da955527f378..c706c6c0ea115 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateSamples.java @@ -11,8 +11,8 @@ */ public final class VolumeQuotaRulesCreateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeQuotaRules_Create.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Create.json */ /** * Sample code: VolumeQuotaRules_Create. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteSamples.java index 5d39bd28d3555..015739da72920 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteSamples.java @@ -9,8 +9,8 @@ */ public final class VolumeQuotaRulesDeleteSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeQuotaRules_Delete.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Delete.json */ /** * Sample code: VolumeQuotaRules_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetSamples.java index 103c9a87eaaee..032e47636a299 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetSamples.java @@ -10,8 +10,7 @@ public final class VolumeQuotaRulesGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_Get. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Get.json */ /** * Sample code: VolumeQuotaRules_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeSamples.java index ee73ab65c3bd6..ebb99927e062b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeSamples.java @@ -10,8 +10,7 @@ public final class VolumeQuotaRulesListByVolumeSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/VolumeQuotaRules_List. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_List.json */ /** * Sample code: VolumeQuotaRules_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesUpdateSamples.java index 8f01c9d2011a4..b74bcd3d0e09b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesUpdateSamples.java @@ -11,8 +11,8 @@ */ public final class VolumeQuotaRulesUpdateSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * VolumeQuotaRules_Update.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/VolumeQuotaRules_Update.json */ /** * Sample code: VolumeQuotaRules_Update. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationSamples.java index e3607400b3eaf..359f224e4d5cc 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationSamples.java @@ -11,8 +11,9 @@ */ public final class VolumesAuthorizeReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_AuthorizeReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_AuthorizeReplication. + * json */ /** * Sample code: Volumes_AuthorizeReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksSamples.java index 8a9f8ddd26ac6..52c7cdebd0776 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksSamples.java @@ -12,8 +12,7 @@ public final class VolumesBreakFileLocksSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_BreakFileLocks - * .json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_BreakFileLocks.json */ /** * Sample code: Volumes_BreakFileLocks. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationSamples.java index ecbf12b4eebed..5444220d7e559 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationSamples.java @@ -11,8 +11,8 @@ */ public final class VolumesBreakReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_BreakReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_BreakReplication.json */ /** * Sample code: Volumes_BreakReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesCreateOrUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesCreateOrUpdateSamples.java index 48d98ddfd3254..cf4551de252ca 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesCreateOrUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesCreateOrUpdateSamples.java @@ -4,7 +4,6 @@ package com.azure.resourcemanager.netapp.generated; -import com.azure.resourcemanager.netapp.models.EncryptionKeySource; import com.azure.resourcemanager.netapp.models.ServiceLevel; /** @@ -13,8 +12,7 @@ public final class VolumesCreateOrUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_CreateOrUpdate - * .json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_CreateOrUpdate.json */ /** * Sample code: Volumes_CreateOrUpdate. @@ -26,7 +24,6 @@ public static void volumesCreateOrUpdate(com.azure.resourcemanager.netapp.NetApp .withCreationToken("my-unique-file-path").withUsageThreshold(107374182400L) .withSubnetId( "/subscriptions/9760acf5-4638-11e7-9bdb-020073ca7778/resourceGroups/myRP/providers/Microsoft.Network/virtualNetworks/testvnet3/subnets/testsubnet3") - .withServiceLevel(ServiceLevel.PREMIUM).withThroughputMibps(128.0F) - .withEncryptionKeySource(EncryptionKeySource.MICROSOFT_KEY_VAULT).create(); + .withServiceLevel(ServiceLevel.PREMIUM).create(); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationSamples.java index e32b1fbd4fb5f..d9ef057ff7a2d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesDeleteReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_DeleteReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_DeleteReplication.json */ /** * Sample code: Volumes_DeleteReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteSamples.java index 7f33ff0683264..2a2bf178654c2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteSamples.java @@ -10,7 +10,7 @@ public final class VolumesDeleteSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Delete.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Delete.json */ /** * Sample code: Volumes_Delete. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationSamples.java index 888a232f9fabf..dc27858c7e32d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesFinalizeRelocationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_FinalizeRelocation.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_FinalizeRelocation.json */ /** * Sample code: Volumes_FinalizeRelocation. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesGetSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesGetSamples.java index 95ed53a1fa938..d4a1c3b119043 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesGetSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesGetSamples.java @@ -10,7 +10,7 @@ public final class VolumesGetSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Get.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Get.json */ /** * Sample code: Volumes_Get. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserSamples.java index 1741052d7d826..0f738243fb301 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserSamples.java @@ -12,8 +12,7 @@ public final class VolumesListGetGroupIdListForLdapUserSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/GroupIdListForLDAPUser - * .json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/GroupIdListForLDAPUser.json */ /** * Sample code: GetGroupIdListForUser. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsSamples.java index b284ae1213290..24c115d0498e8 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesListReplicationsSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_ListReplications.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ListReplications.json */ /** * Sample code: Volumes_ListReplications. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListSamples.java index c728eff657fb1..41acf4cf99173 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesListSamples.java @@ -10,7 +10,7 @@ public final class VolumesListSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_List.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_List.json */ /** * Sample code: Volumes_List. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeSamples.java index 906e203004957..3cb52baf8bca2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeSamples.java @@ -12,8 +12,7 @@ public final class VolumesPoolChangeSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_PoolChange. - * json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_PoolChange.json */ /** * Sample code: Volumes_AuthorizeReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPopulateAvailabilityZoneSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPopulateAvailabilityZoneSamples.java index eccdee4ab423b..359a692c9dc15 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPopulateAvailabilityZoneSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesPopulateAvailabilityZoneSamples.java @@ -9,7 +9,7 @@ */ public final class VolumesPopulateAvailabilityZoneSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ + * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/ * Volumes_PopulateAvailabilityZones.json */ /** diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationSamples.java index 08a4e783cd363..6055a18362780 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationSamples.java @@ -9,8 +9,9 @@ */ public final class VolumesReInitializeReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_ReInitializeReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ReInitializeReplication + * .json */ /** * Sample code: Volumes_ReInitializeReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReestablishReplicationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReestablishReplicationSamples.java index 86f46da93b2fd..bf0f9b94b0cd9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReestablishReplicationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReestablishReplicationSamples.java @@ -11,8 +11,9 @@ */ public final class VolumesReestablishReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_ReestablishReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ReestablishReplication. + * json */ /** * Sample code: Volumes_ReestablishReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRelocateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRelocateSamples.java index 5ae1bd044b5b9..c789c755272ef 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRelocateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRelocateSamples.java @@ -12,7 +12,7 @@ public final class VolumesRelocateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Relocate.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Relocate.json */ /** * Sample code: Volumes_Relocate. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusSamples.java index e004f0c0e020f..d1c4567ff94fd 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesReplicationStatusSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_ReplicationStatus.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ReplicationStatus.json */ /** * Sample code: Volumes_ReplicationStatus. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResetCifsPasswordSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResetCifsPasswordSamples.java index 381917ba50068..978e96cba49e5 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResetCifsPasswordSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResetCifsPasswordSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesResetCifsPasswordSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_ResetCifsPassword.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ResetCifsPassword.json */ /** * Sample code: Volumes_ResetCifsPassword. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationSamples.java index d5e0c21501c71..e02ae435b63a7 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesResyncReplicationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_ResyncReplication.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_ResyncReplication.json */ /** * Sample code: Volumes_ResyncReplication. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationSamples.java index dc6f323f36d3a..c8780fbe2f064 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationSamples.java @@ -9,8 +9,8 @@ */ public final class VolumesRevertRelocationSamples { /* - * x-ms-original-file: specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/ - * Volumes_RevertRelocation.json + * x-ms-original-file: + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_RevertRelocation.json */ /** * Sample code: Volumes_RevertRelocation. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertSamples.java index d6bc8849d30fa..671610d9595d9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesRevertSamples.java @@ -12,7 +12,7 @@ public final class VolumesRevertSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Revert.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Revert.json */ /** * Sample code: Volumes_Revert. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesSplitCloneFromParentSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesSplitCloneFromParentSamples.java deleted file mode 100644 index 791dafea0bed5..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesSplitCloneFromParentSamples.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -/** - * Samples for Volumes SplitCloneFromParent. - */ -public final class VolumesSplitCloneFromParentSamples { - /* - * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_SplitClone. - * json - */ - /** - * Sample code: Volumes_SplitClone. - * - * @param manager Entry point to NetAppFilesManager. - */ - public static void volumesSplitClone(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { - manager.volumes().splitCloneFromParent("myRG", "account1", "pool1", "volume1", - com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesUpdateSamples.java b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesUpdateSamples.java index db80d91115ed8..7184d5dd6a5b7 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesUpdateSamples.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/samples/java/com/azure/resourcemanager/netapp/generated/VolumesUpdateSamples.java @@ -5,8 +5,6 @@ package com.azure.resourcemanager.netapp.generated; import com.azure.resourcemanager.netapp.models.Volume; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; -import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesDataProtection; /** * Samples for Volumes Update. @@ -14,7 +12,7 @@ public final class VolumesUpdateSamples { /* * x-ms-original-file: - * specification/netapp/resource-manager/Microsoft.NetApp/preview/2023-05-01-preview/examples/Volumes_Update.json + * specification/netapp/resource-manager/Microsoft.NetApp/stable/2023-07-01/examples/Volumes_Update.json */ /** * Sample code: Volumes_Update. @@ -24,9 +22,6 @@ public final class VolumesUpdateSamples { public static void volumesUpdate(com.azure.resourcemanager.netapp.NetAppFilesManager manager) { Volume resource = manager.volumes() .getWithResponse("myRG", "account1", "pool1", "volume1", com.azure.core.util.Context.NONE).getValue(); - resource.update().withDataProtection(new VolumePatchPropertiesDataProtection().withBackup( - new VolumeBackupProperties().withPolicyEnforced(false).withBackupEnabled(true).withBackupVaultId( - "/subscriptions/D633CC2E-722B-4AE1-B636-BBD9E4C60ED9/resourceGroups/myRP/providers/Microsoft.NetApp/netAppAccounts/account1/backupVaults/backupVault1"))) - .apply(); + resource.update().apply(); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsMockTests.java index 1922bbbcc36e6..de8dd4ae667a2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AccountsRenewCredentialsMockTests.java @@ -45,7 +45,7 @@ public void testRenewCredentials() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.accounts().renewCredentials("pdnqqskawaoqvmmb", "pqfrtqlkz", com.azure.core.util.Context.NONE); + manager.accounts().renewCredentials("fgcviz", "zdwlvwlyoupfgfb", com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AuthorizeRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AuthorizeRequestTests.java index e825f68621ac8..63d61f123c390 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AuthorizeRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/AuthorizeRequestTests.java @@ -12,14 +12,14 @@ public final class AuthorizeRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { AuthorizeRequest model - = BinaryData.fromString("{\"remoteVolumeResourceId\":\"wiipfpub\"}").toObject(AuthorizeRequest.class); - Assertions.assertEquals("wiipfpub", model.remoteVolumeResourceId()); + = BinaryData.fromString("{\"remoteVolumeResourceId\":\"koievseo\"}").toObject(AuthorizeRequest.class); + Assertions.assertEquals("koievseo", model.remoteVolumeResourceId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AuthorizeRequest model = new AuthorizeRequest().withRemoteVolumeResourceId("wiipfpub"); + AuthorizeRequest model = new AuthorizeRequest().withRemoteVolumeResourceId("koievseo"); model = BinaryData.fromObject(model).toObject(AuthorizeRequest.class); - Assertions.assertEquals("wiipfpub", model.remoteVolumeResourceId()); + Assertions.assertEquals("koievseo", model.remoteVolumeResourceId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupInnerTests.java deleted file mode 100644 index 21a9f54300f63..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupInnerTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import org.junit.jupiter.api.Assertions; - -public final class BackupInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupInner model = BinaryData.fromString( - "{\"properties\":{\"backupId\":\"shfwpracstwity\",\"creationDate\":\"2021-10-09T17:15:47Z\",\"provisioningState\":\"xccedcpnmdyodn\",\"size\":4239411735034713219,\"label\":\"jc\",\"backupType\":\"Manual\",\"failureReason\":\"tiugcxnav\",\"volumeResourceId\":\"wxqibyq\",\"useExistingSnapshot\":false,\"snapshotName\":\"wxwlmdjrkvfgb\",\"backupPolicyResourceId\":\"vpdbodaciz\"},\"id\":\"q\",\"name\":\"hkr\",\"type\":\"ibdeibq\"}") - .toObject(BackupInner.class); - Assertions.assertEquals("jc", model.label()); - Assertions.assertEquals("wxqibyq", model.volumeResourceId()); - Assertions.assertEquals(false, model.useExistingSnapshot()); - Assertions.assertEquals("wxwlmdjrkvfgb", model.snapshotName()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupInner model = new BackupInner().withLabel("jc").withVolumeResourceId("wxqibyq") - .withUseExistingSnapshot(false).withSnapshotName("wxwlmdjrkvfgb"); - model = BinaryData.fromObject(model).toObject(BackupInner.class); - Assertions.assertEquals("jc", model.label()); - Assertions.assertEquals("wxqibyq", model.volumeResourceId()); - Assertions.assertEquals(false, model.useExistingSnapshot()); - Assertions.assertEquals("wxwlmdjrkvfgb", model.snapshotName()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchPropertiesTests.java deleted file mode 100644 index bcbe923594776..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchPropertiesTests.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupPatchProperties; -import org.junit.jupiter.api.Assertions; - -public final class BackupPatchPropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupPatchProperties model - = BinaryData.fromString("{\"label\":\"fzeeyebizik\"}").toObject(BackupPatchProperties.class); - Assertions.assertEquals("fzeeyebizik", model.label()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupPatchProperties model = new BackupPatchProperties().withLabel("fzeeyebizik"); - model = BinaryData.fromObject(model).toObject(BackupPatchProperties.class); - Assertions.assertEquals("fzeeyebizik", model.label()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchTests.java deleted file mode 100644 index fa057ee02895b..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPatchTests.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.BackupPatch; -import org.junit.jupiter.api.Assertions; - -public final class BackupPatchTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupPatch model - = BinaryData.fromString("{\"properties\":{\"label\":\"lffhmouwqlg\"}}").toObject(BackupPatch.class); - Assertions.assertEquals("lffhmouwqlg", model.label()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupPatch model = new BackupPatch().withLabel("lffhmouwqlg"); - model = BinaryData.fromObject(model).toObject(BackupPatch.class); - Assertions.assertEquals("lffhmouwqlg", model.label()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateMockTests.java index 03650dc318ccd..f19ee8112b4c7 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesCreateMockTests.java @@ -33,7 +33,7 @@ public void testCreate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"etag\":\"oywjxhpdulont\",\"properties\":{\"backupPolicyId\":\"np\",\"provisioningState\":\"Succeeded\",\"dailyBackupsToKeep\":1537036510,\"weeklyBackupsToKeep\":529591375,\"monthlyBackupsToKeep\":595356327,\"volumesAssigned\":188519413,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"urfqkfuare\",\"backupsCount\":570950846,\"policyEnabled\":false}]},\"location\":\"klnvnafvvkyfede\",\"tags\":{\"xypokkhmi\":\"oslc\"},\"id\":\"q\",\"name\":\"ymc\",\"type\":\"ngnbdxxew\"}"; + = "{\"etag\":\"zinkfkbgbzbowxeq\",\"properties\":{\"backupPolicyId\":\"ljmygvkzqkjjeokb\",\"provisioningState\":\"Succeeded\",\"dailyBackupsToKeep\":1829804009,\"weeklyBackupsToKeep\":1049990644,\"monthlyBackupsToKeep\":1442604370,\"volumesAssigned\":1208921361,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"wvz\",\"backupsCount\":765217570,\"policyEnabled\":true},{\"volumeName\":\"bzdixzmq\",\"backupsCount\":1928163279,\"policyEnabled\":false}]},\"location\":\"opqhewjptmc\",\"tags\":{\"mzlbiojlvfhrb\":\"ostzelndlatu\",\"qvcww\":\"pn\"},\"id\":\"yurmochpprprs\",\"name\":\"mo\",\"type\":\"ayzejnhlbkpbz\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,16 +50,17 @@ public void testCreate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - BackupPolicy response = manager.backupPolicies().define("oxhlw").withRegion("gwydyy") - .withExistingNetAppAccount("vitac", "xmfcsserxhtv") - .withTags(mapOf("bvqt", "vkh", "izjcpeog", "narfdlpukhpyrn")).withDailyBackupsToKeep(428404939) - .withWeeklyBackupsToKeep(1979503095).withMonthlyBackupsToKeep(1699550993).withEnabled(false).create(); + BackupPolicy response = manager.backupPolicies().define("asiibmiybnnust").withRegion("nzcyjtotp") + .withExistingNetAppAccount("ciqdsme", "iitdfuxt") + .withTags(mapOf("ihed", "vpbdbzqgq", "mkyi", "vqwt", "qcwdhoh", "cysihs", "sufco", "dtmcd")) + .withDailyBackupsToKeep(677853893).withWeeklyBackupsToKeep(915134983).withMonthlyBackupsToKeep(1400929714) + .withEnabled(true).create(); - Assertions.assertEquals("klnvnafvvkyfede", response.location()); - Assertions.assertEquals("oslc", response.tags().get("xypokkhmi")); - Assertions.assertEquals(1537036510, response.dailyBackupsToKeep()); - Assertions.assertEquals(529591375, response.weeklyBackupsToKeep()); - Assertions.assertEquals(595356327, response.monthlyBackupsToKeep()); + Assertions.assertEquals("opqhewjptmc", response.location()); + Assertions.assertEquals("ostzelndlatu", response.tags().get("mzlbiojlvfhrb")); + Assertions.assertEquals(1829804009, response.dailyBackupsToKeep()); + Assertions.assertEquals(1049990644, response.weeklyBackupsToKeep()); + Assertions.assertEquals(1442604370, response.monthlyBackupsToKeep()); Assertions.assertEquals(false, response.enabled()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteMockTests.java index bc596ce7e354d..d0555ac0264e0 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.backupPolicies().delete("qqekewvnqvcdlgu", "ucmfdj", "nlaxpunjqikcz", com.azure.core.util.Context.NONE); + manager.backupPolicies().delete("gv", "irpghriypoqeyh", "qhykprlpyzn", com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetWithResponseMockTests.java index 069cf3e0fd83c..3a4a09154523b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"etag\":\"jbsahrtdtpde\",\"properties\":{\"backupPolicyId\":\"ac\",\"provisioningState\":\"m\",\"dailyBackupsToKeep\":818366731,\"weeklyBackupsToKeep\":1927909421,\"monthlyBackupsToKeep\":958421959,\"volumesAssigned\":49810118,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"gdirazf\",\"backupsCount\":1368173114,\"policyEnabled\":false}]},\"location\":\"bmdujtmvcopexc\",\"tags\":{\"ltqs\":\"rbuhhlky\",\"kffdjktsys\":\"ogtu\",\"jtkbusqogsfika\":\"dfvclglxnfu\",\"arujt\":\"ians\"},\"id\":\"iqxf\",\"name\":\"yjqtt\",\"type\":\"wkpqhjpenuygbq\"}"; + = "{\"etag\":\"pctf\",\"properties\":{\"backupPolicyId\":\"dxotng\",\"provisioningState\":\"gugey\",\"dailyBackupsToKeep\":1242659668,\"weeklyBackupsToKeep\":1204887332,\"monthlyBackupsToKeep\":692345701,\"volumesAssigned\":249369524,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"mfp\",\"backupsCount\":1515954350,\"policyEnabled\":false},{\"volumeName\":\"vyhyhsgzfc\",\"backupsCount\":396816762,\"policyEnabled\":false},{\"volumeName\":\"gbeglqgleo\",\"backupsCount\":1725930797,\"policyEnabled\":true},{\"volumeName\":\"luan\",\"backupsCount\":416708927,\"policyEnabled\":true}]},\"location\":\"eebtijvacvb\",\"tags\":{\"nw\":\"bqqxlaj\"},\"id\":\"acevehjkuyx\",\"name\":\"afgaoqlt\",\"type\":\"aeylinm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,13 +49,13 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); BackupPolicy response = manager.backupPolicies() - .getWithResponse("amlbnseqacjjvpil", "uooqjagmdit", "ueio", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("dfqwmkyoq", "fdvruz", "lzo", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("bmdujtmvcopexc", response.location()); - Assertions.assertEquals("rbuhhlky", response.tags().get("ltqs")); - Assertions.assertEquals(818366731, response.dailyBackupsToKeep()); - Assertions.assertEquals(1927909421, response.weeklyBackupsToKeep()); - Assertions.assertEquals(958421959, response.monthlyBackupsToKeep()); + Assertions.assertEquals("eebtijvacvb", response.location()); + Assertions.assertEquals("bqqxlaj", response.tags().get("nw")); + Assertions.assertEquals(1242659668, response.dailyBackupsToKeep()); + Assertions.assertEquals(1204887332, response.weeklyBackupsToKeep()); + Assertions.assertEquals(692345701, response.monthlyBackupsToKeep()); Assertions.assertEquals(false, response.enabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListMockTests.java index a61b582f11bb1..b3661822075c7 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"etag\":\"ccxlzhcoxovnek\",\"properties\":{\"backupPolicyId\":\"nlusfnrd\",\"provisioningState\":\"xtxrdcqtjvidt\",\"dailyBackupsToKeep\":695049640,\"weeklyBackupsToKeep\":1779546866,\"monthlyBackupsToKeep\":818370137,\"volumesAssigned\":2099497391,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"wkasiziesf\",\"backupsCount\":1027973509,\"policyEnabled\":true},{\"volumeName\":\"qfecjxeygtuhx\",\"backupsCount\":1670662701,\"policyEnabled\":true},{\"volumeName\":\"wmrswnjlxuzrh\",\"backupsCount\":519622226,\"policyEnabled\":true},{\"volumeName\":\"baqehgpdoh\",\"backupsCount\":1232652299,\"policyEnabled\":false}]},\"location\":\"coi\",\"tags\":{\"bnwgfmxj\":\"xncnwfe\",\"y\":\"cgbjbgdlfgt\",\"ctqhamzjrwdk\":\"naquflq\"},\"id\":\"zeqyjleziun\",\"name\":\"xdfzantkw\",\"type\":\"eg\"}]}"; + = "{\"value\":[{\"etag\":\"q\",\"properties\":{\"backupPolicyId\":\"kcxk\",\"provisioningState\":\"bn\",\"dailyBackupsToKeep\":866421840,\"weeklyBackupsToKeep\":490406344,\"monthlyBackupsToKeep\":928926099,\"volumesAssigned\":1250131057,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"wijpsttexoqqpwc\",\"backupsCount\":2114285001,\"policyEnabled\":true}]},\"location\":\"runcuwmqspkcd\",\"tags\":{\"qbnj\":\"lctddunqndyfpch\",\"qqoli\":\"rcgegydcwboxjum\",\"aiouaubrjt\":\"r\"},\"id\":\"oq\",\"name\":\"fuojrngif\",\"type\":\"rzpasccbiuimzdly\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,13 +50,13 @@ public void testList() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.backupPolicies().list("onwpnga", "innixjawrtmjfj", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("coi", response.iterator().next().location()); - Assertions.assertEquals("xncnwfe", response.iterator().next().tags().get("bnwgfmxj")); - Assertions.assertEquals(695049640, response.iterator().next().dailyBackupsToKeep()); - Assertions.assertEquals(1779546866, response.iterator().next().weeklyBackupsToKeep()); - Assertions.assertEquals(818370137, response.iterator().next().monthlyBackupsToKeep()); - Assertions.assertEquals(false, response.iterator().next().enabled()); + = manager.backupPolicies().list("gdakchz", "vl", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("runcuwmqspkcd", response.iterator().next().location()); + Assertions.assertEquals("lctddunqndyfpch", response.iterator().next().tags().get("qbnj")); + Assertions.assertEquals(866421840, response.iterator().next().dailyBackupsToKeep()); + Assertions.assertEquals(490406344, response.iterator().next().weeklyBackupsToKeep()); + Assertions.assertEquals(928926099, response.iterator().next().monthlyBackupsToKeep()); + Assertions.assertEquals(true, response.iterator().next().enabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListTests.java index 9843db0fd6731..5dd36df413595 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPoliciesListTests.java @@ -16,39 +16,30 @@ public final class BackupPoliciesListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BackupPoliciesList model = BinaryData.fromString( - "{\"value\":[{\"etag\":\"qlbjbsybbqwrvt\",\"properties\":{\"backupPolicyId\":\"gmfpgvmp\",\"provisioningState\":\"as\",\"dailyBackupsToKeep\":753407395,\"weeklyBackupsToKeep\":522033644,\"monthlyBackupsToKeep\":178976731,\"volumesAssigned\":1464941672,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"bdsrez\",\"backupsCount\":187520251,\"policyEnabled\":false},{\"volumeName\":\"uyowqkdwy\",\"backupsCount\":174588693,\"policyEnabled\":false},{\"volumeName\":\"rcgp\",\"backupsCount\":1400964444,\"policyEnabled\":false},{\"volumeName\":\"ejzanlfz\",\"backupsCount\":386879497,\"policyEnabled\":false}]},\"location\":\"bzonok\",\"tags\":{\"irgzp\":\"jq\"},\"id\":\"rlazszrnw\",\"name\":\"iin\",\"type\":\"fpwpjylwbt\"},{\"etag\":\"flsjc\",\"properties\":{\"backupPolicyId\":\"szfjvfbgofelja\",\"provisioningState\":\"qmqhldvriii\",\"dailyBackupsToKeep\":1756672367,\"weeklyBackupsToKeep\":401482588,\"monthlyBackupsToKeep\":1672429293,\"volumesAssigned\":42140994,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"xsowu\",\"backupsCount\":750613603,\"policyEnabled\":true},{\"volumeName\":\"ahhxvrh\",\"backupsCount\":2135501499,\"policyEnabled\":true}]},\"location\":\"g\",\"tags\":{\"xujxuknd\":\"pughftqsxh\",\"ihwhbotzingamvpp\":\"digrjguufzdmsyqt\",\"zqzudph\":\"o\"},\"id\":\"amvdkfwynwcvtbv\",\"name\":\"ayhmtnvyqiatkz\",\"type\":\"pcnp\"},{\"etag\":\"cjaesgvvs\",\"properties\":{\"backupPolicyId\":\"yajguqfhwygzlv\",\"provisioningState\":\"kfxu\",\"dailyBackupsToKeep\":2087511541,\"weeklyBackupsToKeep\":1547666647,\"monthlyBackupsToKeep\":252265287,\"volumesAssigned\":372421736,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"qdpsqxqvpsvu\",\"backupsCount\":1326232205,\"policyEnabled\":false},{\"volumeName\":\"elvezrypq\",\"backupsCount\":643673289,\"policyEnabled\":true}]},\"location\":\"erqwkyhkobopg\",\"tags\":{\"wep\":\"k\"},\"id\":\"qpcrf\",\"name\":\"bwccsnjvcdwxlpqe\",\"type\":\"ftnkhtj\"},{\"etag\":\"i\",\"properties\":{\"backupPolicyId\":\"wfqatmtd\",\"provisioningState\":\"mdvy\",\"dailyBackupsToKeep\":932898449,\"weeklyBackupsToKeep\":1851759881,\"monthlyBackupsToKeep\":624157964,\"volumesAssigned\":2142197435,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"ryuzh\",\"backupsCount\":524454595,\"policyEnabled\":true},{\"volumeName\":\"rvqqaatj\",\"backupsCount\":379667314,\"policyEnabled\":true},{\"volumeName\":\"upmfiibfg\",\"backupsCount\":2107533163,\"policyEnabled\":false}]},\"location\":\"vrwxkv\",\"tags\":{\"vjayvblmhvkzu\":\"gllqwjy\"},\"id\":\"bxvvyhg\",\"name\":\"opbyrqufegxu\",\"type\":\"wz\"}]}") + "{\"value\":[{\"etag\":\"ofncckwyfzqwhxxb\",\"properties\":{\"backupPolicyId\":\"qa\",\"provisioningState\":\"feqztppriol\",\"dailyBackupsToKeep\":621148688,\"weeklyBackupsToKeep\":1270431726,\"monthlyBackupsToKeep\":929982715,\"volumesAssigned\":1321856720,\"enabled\":false,\"volumeBackups\":[{\"volumeName\":\"obqwcsdbnwdcfh\",\"backupsCount\":1589501712,\"policyEnabled\":true},{\"volumeName\":\"uvglsbjjcanvx\",\"backupsCount\":235845887,\"policyEnabled\":false},{\"volumeName\":\"utncorm\",\"backupsCount\":1945766703,\"policyEnabled\":false}]},\"location\":\"cofudflvkgjub\",\"tags\":{\"nqntorudsgsahm\":\"nnqvsa\",\"rauwjuetaebu\":\"yc\",\"dmovsm\":\"u\"},\"id\":\"l\",\"name\":\"wabm\",\"type\":\"oefki\"}]}") .toObject(BackupPoliciesList.class); - Assertions.assertEquals("bzonok", model.value().get(0).location()); - Assertions.assertEquals("jq", model.value().get(0).tags().get("irgzp")); - Assertions.assertEquals(753407395, model.value().get(0).dailyBackupsToKeep()); - Assertions.assertEquals(522033644, model.value().get(0).weeklyBackupsToKeep()); - Assertions.assertEquals(178976731, model.value().get(0).monthlyBackupsToKeep()); - Assertions.assertEquals(true, model.value().get(0).enabled()); + Assertions.assertEquals("cofudflvkgjub", model.value().get(0).location()); + Assertions.assertEquals("nnqvsa", model.value().get(0).tags().get("nqntorudsgsahm")); + Assertions.assertEquals(621148688, model.value().get(0).dailyBackupsToKeep()); + Assertions.assertEquals(1270431726, model.value().get(0).weeklyBackupsToKeep()); + Assertions.assertEquals(929982715, model.value().get(0).monthlyBackupsToKeep()); + Assertions.assertEquals(false, model.value().get(0).enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BackupPoliciesList model = new BackupPoliciesList().withValue(Arrays.asList( - new BackupPolicyInner().withLocation("bzonok").withTags(mapOf("irgzp", "jq")) - .withDailyBackupsToKeep(753407395).withWeeklyBackupsToKeep(522033644) - .withMonthlyBackupsToKeep(178976731).withEnabled(true), - new BackupPolicyInner().withLocation("g") - .withTags(mapOf("xujxuknd", "pughftqsxh", "ihwhbotzingamvpp", "digrjguufzdmsyqt", "zqzudph", "o")) - .withDailyBackupsToKeep(1756672367).withWeeklyBackupsToKeep(401482588) - .withMonthlyBackupsToKeep(1672429293).withEnabled(true), - new BackupPolicyInner().withLocation("erqwkyhkobopg").withTags(mapOf("wep", "k")) - .withDailyBackupsToKeep(2087511541).withWeeklyBackupsToKeep(1547666647) - .withMonthlyBackupsToKeep(252265287).withEnabled(true), - new BackupPolicyInner().withLocation("vrwxkv").withTags(mapOf("vjayvblmhvkzu", "gllqwjy")) - .withDailyBackupsToKeep(932898449).withWeeklyBackupsToKeep(1851759881) - .withMonthlyBackupsToKeep(624157964).withEnabled(false))); + BackupPoliciesList model + = new BackupPoliciesList().withValue(Arrays.asList(new BackupPolicyInner().withLocation("cofudflvkgjub") + .withTags(mapOf("nqntorudsgsahm", "nnqvsa", "rauwjuetaebu", "yc", "dmovsm", "u")) + .withDailyBackupsToKeep(621148688).withWeeklyBackupsToKeep(1270431726) + .withMonthlyBackupsToKeep(929982715).withEnabled(false))); model = BinaryData.fromObject(model).toObject(BackupPoliciesList.class); - Assertions.assertEquals("bzonok", model.value().get(0).location()); - Assertions.assertEquals("jq", model.value().get(0).tags().get("irgzp")); - Assertions.assertEquals(753407395, model.value().get(0).dailyBackupsToKeep()); - Assertions.assertEquals(522033644, model.value().get(0).weeklyBackupsToKeep()); - Assertions.assertEquals(178976731, model.value().get(0).monthlyBackupsToKeep()); - Assertions.assertEquals(true, model.value().get(0).enabled()); + Assertions.assertEquals("cofudflvkgjub", model.value().get(0).location()); + Assertions.assertEquals("nnqvsa", model.value().get(0).tags().get("nqntorudsgsahm")); + Assertions.assertEquals(621148688, model.value().get(0).dailyBackupsToKeep()); + Assertions.assertEquals(1270431726, model.value().get(0).weeklyBackupsToKeep()); + Assertions.assertEquals(929982715, model.value().get(0).monthlyBackupsToKeep()); + Assertions.assertEquals(false, model.value().get(0).enabled()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyInnerTests.java index 57c7b02854b51..aa76ba8710b23 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyInnerTests.java @@ -14,28 +14,27 @@ public final class BackupPolicyInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BackupPolicyInner model = BinaryData.fromString( - "{\"etag\":\"nhlmctlpdng\",\"properties\":{\"backupPolicyId\":\"vgbmhr\",\"provisioningState\":\"kw\",\"dailyBackupsToKeep\":881216516,\"weeklyBackupsToKeep\":1630324691,\"monthlyBackupsToKeep\":568576807,\"volumesAssigned\":1374422252,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"aixexccbdreaxh\",\"backupsCount\":1486564388,\"policyEnabled\":false}]},\"location\":\"vqahqkghtpwi\",\"tags\":{\"z\":\"yjsvfyc\",\"rvmtgjq\":\"fvoow\",\"nsxkmcwaekrrjr\":\"pyostronzmyhgfi\"},\"id\":\"afxtsgum\",\"name\":\"jglikkxwslolb\",\"type\":\"pvuzlmv\"}") + "{\"etag\":\"vtpuqujmqlgk\",\"properties\":{\"backupPolicyId\":\"tndoaongbjc\",\"provisioningState\":\"ujitcjedftww\",\"dailyBackupsToKeep\":1066178212,\"weeklyBackupsToKeep\":565058049,\"monthlyBackupsToKeep\":229411798,\"volumesAssigned\":242462331,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"ouicybxarzgszu\",\"backupsCount\":37759464,\"policyEnabled\":true},{\"volumeName\":\"opidoamciodh\",\"backupsCount\":1741139859,\"policyEnabled\":true},{\"volumeName\":\"hnzbonl\",\"backupsCount\":1184830478,\"policyEnabled\":true},{\"volumeName\":\"okdwb\",\"backupsCount\":1161774579,\"policyEnabled\":false}]},\"location\":\"cmrvexzt\",\"tags\":{\"lmnguxaw\":\"qgsfraoyzkoow\"},\"id\":\"aldsy\",\"name\":\"uximerqfobw\",\"type\":\"znkbykutwpfhpagm\"}") .toObject(BackupPolicyInner.class); - Assertions.assertEquals("vqahqkghtpwi", model.location()); - Assertions.assertEquals("yjsvfyc", model.tags().get("z")); - Assertions.assertEquals(881216516, model.dailyBackupsToKeep()); - Assertions.assertEquals(1630324691, model.weeklyBackupsToKeep()); - Assertions.assertEquals(568576807, model.monthlyBackupsToKeep()); + Assertions.assertEquals("cmrvexzt", model.location()); + Assertions.assertEquals("qgsfraoyzkoow", model.tags().get("lmnguxaw")); + Assertions.assertEquals(1066178212, model.dailyBackupsToKeep()); + Assertions.assertEquals(565058049, model.weeklyBackupsToKeep()); + Assertions.assertEquals(229411798, model.monthlyBackupsToKeep()); Assertions.assertEquals(true, model.enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BackupPolicyInner model = new BackupPolicyInner().withLocation("vqahqkghtpwi") - .withTags(mapOf("z", "yjsvfyc", "rvmtgjq", "fvoow", "nsxkmcwaekrrjr", "pyostronzmyhgfi")) - .withDailyBackupsToKeep(881216516).withWeeklyBackupsToKeep(1630324691).withMonthlyBackupsToKeep(568576807) - .withEnabled(true); + BackupPolicyInner model = new BackupPolicyInner().withLocation("cmrvexzt") + .withTags(mapOf("lmnguxaw", "qgsfraoyzkoow")).withDailyBackupsToKeep(1066178212) + .withWeeklyBackupsToKeep(565058049).withMonthlyBackupsToKeep(229411798).withEnabled(true); model = BinaryData.fromObject(model).toObject(BackupPolicyInner.class); - Assertions.assertEquals("vqahqkghtpwi", model.location()); - Assertions.assertEquals("yjsvfyc", model.tags().get("z")); - Assertions.assertEquals(881216516, model.dailyBackupsToKeep()); - Assertions.assertEquals(1630324691, model.weeklyBackupsToKeep()); - Assertions.assertEquals(568576807, model.monthlyBackupsToKeep()); + Assertions.assertEquals("cmrvexzt", model.location()); + Assertions.assertEquals("qgsfraoyzkoow", model.tags().get("lmnguxaw")); + Assertions.assertEquals(1066178212, model.dailyBackupsToKeep()); + Assertions.assertEquals(565058049, model.weeklyBackupsToKeep()); + Assertions.assertEquals(229411798, model.monthlyBackupsToKeep()); Assertions.assertEquals(true, model.enabled()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPatchTests.java index a5198d2dcbbc1..fc3dc85544435 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPatchTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPatchTests.java @@ -14,29 +14,27 @@ public final class BackupPolicyPatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BackupPolicyPatch model = BinaryData.fromString( - "{\"properties\":{\"backupPolicyId\":\"kqjjlwuenvrkp\",\"provisioningState\":\"uaibrebqaaysj\",\"dailyBackupsToKeep\":1330578772,\"weeklyBackupsToKeep\":1662544736,\"monthlyBackupsToKeep\":2018222858,\"volumesAssigned\":1782265251,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"ffiakp\",\"backupsCount\":882438810,\"policyEnabled\":false}]},\"location\":\"tedltmmj\",\"tags\":{\"auyqncygupkv\":\"eozphv\",\"dscwxqupevzhf\":\"p\",\"pelmcuvhixbjxyf\":\"totxhojujb\",\"lrcoolsttpki\":\"n\"},\"id\":\"kkbnu\",\"name\":\"rywvtylbfpn\",\"type\":\"urdoi\"}") + "{\"properties\":{\"backupPolicyId\":\"bminrfdwoyuhhzi\",\"provisioningState\":\"efozbhdms\",\"dailyBackupsToKeep\":1531171615,\"weeklyBackupsToKeep\":1354395459,\"monthlyBackupsToKeep\":572432734,\"volumesAssigned\":1916559931,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"u\",\"backupsCount\":674060065,\"policyEnabled\":false}]},\"location\":\"cslfaoqzpiyylha\",\"tags\":{\"hka\":\"whccs\",\"ggwoluhczb\":\"vwitqscyw\",\"i\":\"emh\"},\"id\":\"sbrgz\",\"name\":\"wmsweypqwd\",\"type\":\"ggicccnxqhue\"}") .toObject(BackupPolicyPatch.class); - Assertions.assertEquals("tedltmmj", model.location()); - Assertions.assertEquals("eozphv", model.tags().get("auyqncygupkv")); - Assertions.assertEquals(1330578772, model.dailyBackupsToKeep()); - Assertions.assertEquals(1662544736, model.weeklyBackupsToKeep()); - Assertions.assertEquals(2018222858, model.monthlyBackupsToKeep()); + Assertions.assertEquals("cslfaoqzpiyylha", model.location()); + Assertions.assertEquals("whccs", model.tags().get("hka")); + Assertions.assertEquals(1531171615, model.dailyBackupsToKeep()); + Assertions.assertEquals(1354395459, model.weeklyBackupsToKeep()); + Assertions.assertEquals(572432734, model.monthlyBackupsToKeep()); Assertions.assertEquals(true, model.enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BackupPolicyPatch model = new BackupPolicyPatch().withLocation("tedltmmj") - .withTags(mapOf("auyqncygupkv", "eozphv", "dscwxqupevzhf", "p", "pelmcuvhixbjxyf", "totxhojujb", - "lrcoolsttpki", "n")) - .withDailyBackupsToKeep(1330578772).withWeeklyBackupsToKeep(1662544736).withMonthlyBackupsToKeep(2018222858) - .withEnabled(true); + BackupPolicyPatch model = new BackupPolicyPatch().withLocation("cslfaoqzpiyylha") + .withTags(mapOf("hka", "whccs", "ggwoluhczb", "vwitqscyw", "i", "emh")).withDailyBackupsToKeep(1531171615) + .withWeeklyBackupsToKeep(1354395459).withMonthlyBackupsToKeep(572432734).withEnabled(true); model = BinaryData.fromObject(model).toObject(BackupPolicyPatch.class); - Assertions.assertEquals("tedltmmj", model.location()); - Assertions.assertEquals("eozphv", model.tags().get("auyqncygupkv")); - Assertions.assertEquals(1330578772, model.dailyBackupsToKeep()); - Assertions.assertEquals(1662544736, model.weeklyBackupsToKeep()); - Assertions.assertEquals(2018222858, model.monthlyBackupsToKeep()); + Assertions.assertEquals("cslfaoqzpiyylha", model.location()); + Assertions.assertEquals("whccs", model.tags().get("hka")); + Assertions.assertEquals(1531171615, model.dailyBackupsToKeep()); + Assertions.assertEquals(1354395459, model.weeklyBackupsToKeep()); + Assertions.assertEquals(572432734, model.monthlyBackupsToKeep()); Assertions.assertEquals(true, model.enabled()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPropertiesTests.java index 23f996b654f1f..38dfd124d9f2d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPolicyPropertiesTests.java @@ -12,22 +12,22 @@ public final class BackupPolicyPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BackupPolicyProperties model = BinaryData.fromString( - "{\"backupPolicyId\":\"lfktgplcrpwjxe\",\"provisioningState\":\"oi\",\"dailyBackupsToKeep\":1406843727,\"weeklyBackupsToKeep\":76074858,\"monthlyBackupsToKeep\":509080980,\"volumesAssigned\":820542416,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"zejjoqk\",\"backupsCount\":424496113,\"policyEnabled\":false}]}") + "{\"backupPolicyId\":\"skdsnfdsdoakg\",\"provisioningState\":\"lmkk\",\"dailyBackupsToKeep\":1673981436,\"weeklyBackupsToKeep\":246193278,\"monthlyBackupsToKeep\":1837577186,\"volumesAssigned\":1686383029,\"enabled\":true,\"volumeBackups\":[{\"volumeName\":\"twvogvbbe\",\"backupsCount\":1690092543,\"policyEnabled\":true},{\"volumeName\":\"qmoa\",\"backupsCount\":840228808,\"policyEnabled\":false},{\"volumeName\":\"zr\",\"backupsCount\":1964426059,\"policyEnabled\":true}]}") .toObject(BackupPolicyProperties.class); - Assertions.assertEquals(1406843727, model.dailyBackupsToKeep()); - Assertions.assertEquals(76074858, model.weeklyBackupsToKeep()); - Assertions.assertEquals(509080980, model.monthlyBackupsToKeep()); + Assertions.assertEquals(1673981436, model.dailyBackupsToKeep()); + Assertions.assertEquals(246193278, model.weeklyBackupsToKeep()); + Assertions.assertEquals(1837577186, model.monthlyBackupsToKeep()); Assertions.assertEquals(true, model.enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - BackupPolicyProperties model = new BackupPolicyProperties().withDailyBackupsToKeep(1406843727) - .withWeeklyBackupsToKeep(76074858).withMonthlyBackupsToKeep(509080980).withEnabled(true); + BackupPolicyProperties model = new BackupPolicyProperties().withDailyBackupsToKeep(1673981436) + .withWeeklyBackupsToKeep(246193278).withMonthlyBackupsToKeep(1837577186).withEnabled(true); model = BinaryData.fromObject(model).toObject(BackupPolicyProperties.class); - Assertions.assertEquals(1406843727, model.dailyBackupsToKeep()); - Assertions.assertEquals(76074858, model.weeklyBackupsToKeep()); - Assertions.assertEquals(509080980, model.monthlyBackupsToKeep()); + Assertions.assertEquals(1673981436, model.dailyBackupsToKeep()); + Assertions.assertEquals(246193278, model.weeklyBackupsToKeep()); + Assertions.assertEquals(1837577186, model.monthlyBackupsToKeep()); Assertions.assertEquals(true, model.enabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPropertiesTests.java deleted file mode 100644 index e1956de34ce06..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupPropertiesTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupProperties; -import org.junit.jupiter.api.Assertions; - -public final class BackupPropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupProperties model = BinaryData.fromString( - "{\"backupId\":\"qkgh\",\"creationDate\":\"2021-11-17T03:58:19Z\",\"provisioningState\":\"zwmk\",\"size\":1102306209532433694,\"label\":\"pjorwkqnyhg\",\"backupType\":\"Manual\",\"failureReason\":\"jivfxzsjabib\",\"volumeResourceId\":\"ystawfsdjpvkvp\",\"useExistingSnapshot\":true,\"snapshotName\":\"kzbzkdvncjabudu\",\"backupPolicyResourceId\":\"kakmokzh\"}") - .toObject(BackupProperties.class); - Assertions.assertEquals("pjorwkqnyhg", model.label()); - Assertions.assertEquals("ystawfsdjpvkvp", model.volumeResourceId()); - Assertions.assertEquals(true, model.useExistingSnapshot()); - Assertions.assertEquals("kzbzkdvncjabudu", model.snapshotName()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupProperties model = new BackupProperties().withLabel("pjorwkqnyhg").withVolumeResourceId("ystawfsdjpvkvp") - .withUseExistingSnapshot(true).withSnapshotName("kzbzkdvncjabudu"); - model = BinaryData.fromObject(model).toObject(BackupProperties.class); - Assertions.assertEquals("pjorwkqnyhg", model.label()); - Assertions.assertEquals("ystawfsdjpvkvp", model.volumeResourceId()); - Assertions.assertEquals(true, model.useExistingSnapshot()); - Assertions.assertEquals("kzbzkdvncjabudu", model.snapshotName()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupRestoreFilesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupRestoreFilesTests.java deleted file mode 100644 index 4503cba99a2c7..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupRestoreFilesTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.BackupRestoreFiles; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class BackupRestoreFilesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupRestoreFiles model = BinaryData.fromString( - "{\"fileList\":[\"onbzoggculapzwy\",\"pgogtqxepny\",\"b\",\"uajlyj\"],\"restoreFilePath\":\"vofqzhvfc\",\"destinationVolumeId\":\"byfmowuxr\"}") - .toObject(BackupRestoreFiles.class); - Assertions.assertEquals("onbzoggculapzwy", model.fileList().get(0)); - Assertions.assertEquals("vofqzhvfc", model.restoreFilePath()); - Assertions.assertEquals("byfmowuxr", model.destinationVolumeId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupRestoreFiles model - = new BackupRestoreFiles().withFileList(Arrays.asList("onbzoggculapzwy", "pgogtqxepny", "b", "uajlyj")) - .withRestoreFilePath("vofqzhvfc").withDestinationVolumeId("byfmowuxr"); - model = BinaryData.fromObject(model).toObject(BackupRestoreFiles.class); - Assertions.assertEquals("onbzoggculapzwy", model.fileList().get(0)); - Assertions.assertEquals("vofqzhvfc", model.restoreFilePath()); - Assertions.assertEquals("byfmowuxr", model.destinationVolumeId()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupStatusInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupStatusInnerTests.java deleted file mode 100644 index a6759cfd83ee7..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupStatusInnerTests.java +++ /dev/null @@ -1,23 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupStatusInner; - -public final class BackupStatusInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupStatusInner model = BinaryData.fromString( - "{\"healthy\":false,\"relationshipStatus\":\"Idle\",\"mirrorState\":\"Uninitialized\",\"unhealthyReason\":\"oefki\",\"errorMessage\":\"vtpuqujmqlgk\",\"lastTransferSize\":2595329865796026079,\"lastTransferType\":\"oaongbjc\",\"totalTransferBytes\":3888691267660480720,\"transferProgressBytes\":4564203424204648247}") - .toObject(BackupStatusInner.class); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupStatusInner model = new BackupStatusInner(); - model = BinaryData.fromObject(model).toObject(BackupStatusInner.class); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultInnerTests.java deleted file mode 100644 index 7d8249c0573cf..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultInnerTests.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class BackupVaultInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupVaultInner model = BinaryData.fromString( - "{\"properties\":{\"provisioningState\":\"vvbalx\"},\"location\":\"lchpodbzevwrdn\",\"tags\":{\"stul\":\"kuvsjcswsm\"},\"id\":\"qypfcv\",\"name\":\"er\",\"type\":\"hp\"}") - .toObject(BackupVaultInner.class); - Assertions.assertEquals("lchpodbzevwrdn", model.location()); - Assertions.assertEquals("kuvsjcswsm", model.tags().get("stul")); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupVaultInner model - = new BackupVaultInner().withLocation("lchpodbzevwrdn").withTags(mapOf("stul", "kuvsjcswsm")); - model = BinaryData.fromObject(model).toObject(BackupVaultInner.class); - Assertions.assertEquals("lchpodbzevwrdn", model.location()); - Assertions.assertEquals("kuvsjcswsm", model.tags().get("stul")); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPatchTests.java deleted file mode 100644 index 67ddfa66972d2..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPatchTests.java +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.BackupVaultPatch; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class BackupVaultPatchTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupVaultPatch model = BinaryData.fromString( - "{\"tags\":{\"xgqddrih\":\"cxsspuunnoxyh\",\"ewda\":\"fhoqca\",\"xkzb\":\"mdjvlpj\",\"ncj\":\"msgeivsiykzk\"}}") - .toObject(BackupVaultPatch.class); - Assertions.assertEquals("cxsspuunnoxyh", model.tags().get("xgqddrih")); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupVaultPatch model = new BackupVaultPatch() - .withTags(mapOf("xgqddrih", "cxsspuunnoxyh", "ewda", "fhoqca", "xkzb", "mdjvlpj", "ncj", "msgeivsiykzk")); - model = BinaryData.fromObject(model).toObject(BackupVaultPatch.class); - Assertions.assertEquals("cxsspuunnoxyh", model.tags().get("xgqddrih")); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPropertiesTests.java deleted file mode 100644 index a8937175a7832..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultPropertiesTests.java +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultProperties; - -public final class BackupVaultPropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupVaultProperties model - = BinaryData.fromString("{\"provisioningState\":\"mfpjbabw\"}").toObject(BackupVaultProperties.class); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupVaultProperties model = new BackupVaultProperties(); - model = BinaryData.fromObject(model).toObject(BackupVaultProperties.class); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListTests.java deleted file mode 100644 index 9383657f664aa..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupVaultsListTests.java +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; -import com.azure.resourcemanager.netapp.models.BackupVaultsList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class BackupVaultsListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupVaultsList model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"bajlka\"},\"location\":\"wxyiopidkqq\",\"tags\":{\"dmligovibrxk\":\"vscx\",\"cbgoor\":\"mloazuru\",\"ybfhjxa\":\"te\",\"il\":\"vvjgslor\"},\"id\":\"yw\",\"name\":\"t\",\"type\":\"gkxnyedabg\"},{\"properties\":{\"provisioningState\":\"dtj\"},\"location\":\"wbcihxuuwh\",\"tags\":{\"akkud\":\"xccybvpa\",\"wjplma\":\"px\"},\"id\":\"stcyohpfkyrkdbd\",\"name\":\"iogsjkmnwq\",\"type\":\"nobaiyhddviacegf\"}],\"nextLink\":\"ntfpmvmemfnc\"}") - .toObject(BackupVaultsList.class); - Assertions.assertEquals("wxyiopidkqq", model.value().get(0).location()); - Assertions.assertEquals("vscx", model.value().get(0).tags().get("dmligovibrxk")); - Assertions.assertEquals("ntfpmvmemfnc", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupVaultsList model = new BackupVaultsList() - .withValue(Arrays.asList( - new BackupVaultInner().withLocation("wxyiopidkqq") - .withTags(mapOf("dmligovibrxk", "vscx", "cbgoor", "mloazuru", "ybfhjxa", "te", "il", "vvjgslor")), - new BackupVaultInner().withLocation("wbcihxuuwh").withTags(mapOf("akkud", "xccybvpa", "wjplma", "px")))) - .withNextLink("ntfpmvmemfnc"); - model = BinaryData.fromObject(model).toObject(BackupVaultsList.class); - Assertions.assertEquals("wxyiopidkqq", model.value().get(0).location()); - Assertions.assertEquals("vscx", model.value().get(0).tags().get("dmligovibrxk")); - Assertions.assertEquals("ntfpmvmemfnc", model.nextLink()); - } - - // Use "Map.of" if available - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusWithResponseMockTests.java index 17f7ef0caf2a6..8e917581c6db7 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetVolumeRestoreStatusWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetVolumeRestoreStatusWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"healthy\":false,\"relationshipStatus\":\"Transferring\",\"mirrorState\":\"Mirrored\",\"unhealthyReason\":\"glbyvi\",\"errorMessage\":\"ctbrxkjzwrgxffm\",\"totalTransferBytes\":7584093446155046981}"; + = "{\"healthy\":true,\"relationshipStatus\":\"Transferring\",\"mirrorState\":\"Mirrored\",\"unhealthyReason\":\"rzpgep\",\"errorMessage\":\"yb\",\"totalTransferBytes\":6730843668651593065}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -47,8 +47,8 @@ public void testGetVolumeRestoreStatusWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - RestoreStatus response = manager.backups().getVolumeRestoreStatusWithResponse("fcngjsa", "sii", - "tmkzjvkviirhgfgr", "sdp", com.azure.core.util.Context.NONE).getValue(); + RestoreStatus response = manager.backups().getVolumeRestoreStatusWithResponse("slzkwrrwoycqu", "wyh", + "hnomdrkywuh", "svfuurutlwexxwl", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsListTests.java deleted file mode 100644 index 2d77cf6224582..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsListTests.java +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; -import com.azure.resourcemanager.netapp.models.BackupsList; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class BackupsListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupsList model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"backupId\":\"rzgszufoxci\",\"creationDate\":\"2021-06-16T02:13:56Z\",\"provisioningState\":\"doamciodhkha\",\"size\":676590971547397849,\"label\":\"zbonlwnt\",\"backupType\":\"Scheduled\",\"failureReason\":\"kdwbwhkszz\",\"volumeResourceId\":\"mrv\",\"useExistingSnapshot\":true,\"snapshotName\":\"vbtqgsfraoyzk\",\"backupPolicyResourceId\":\"wtl\"},\"id\":\"guxawqaldsyuuxi\",\"name\":\"erqf\",\"type\":\"bw\"},{\"properties\":{\"backupId\":\"nkbykutwpfhp\",\"creationDate\":\"2021-04-26T06:25:06Z\",\"provisioningState\":\"r\",\"size\":6854590524371553523,\"label\":\"fdsd\",\"backupType\":\"Scheduled\",\"failureReason\":\"tdlmkkzevd\",\"volumeResourceId\":\"hewpusdsttwv\",\"useExistingSnapshot\":true,\"snapshotName\":\"bejdcn\",\"backupPolicyResourceId\":\"qmoa\"},\"id\":\"fgmjzrwrdgrt\",\"name\":\"aenuuz\",\"type\":\"opbminrfdw\"},{\"properties\":{\"backupId\":\"uhhziuiefozbhdm\",\"creationDate\":\"2021-02-14T10:35:20Z\",\"provisioningState\":\"zqhof\",\"size\":7787317883311090832,\"label\":\"qu\",\"backupType\":\"Scheduled\",\"failureReason\":\"icslfaoq\",\"volumeResourceId\":\"piyylhalnswhccsp\",\"useExistingSnapshot\":true,\"snapshotName\":\"vwitqscyw\",\"backupPolicyResourceId\":\"gwol\"},\"id\":\"czbwemhairsbr\",\"name\":\"z\",\"type\":\"wmsweypqwd\"},{\"properties\":{\"backupId\":\"gicccnxqhuex\",\"creationDate\":\"2021-01-10T11:07:16Z\",\"provisioningState\":\"lstvlzywe\",\"size\":7397633883241527739,\"label\":\"csdtclusiypbs\",\"backupType\":\"Manual\",\"failureReason\":\"gusl\",\"volumeResourceId\":\"eadcygqukyhejhz\",\"useExistingSnapshot\":true,\"snapshotName\":\"fpel\",\"backupPolicyResourceId\":\"p\"},\"id\":\"ksrpqv\",\"name\":\"jzraehtwdwrf\",\"type\":\"swibyr\"}],\"nextLink\":\"l\"}") - .toObject(BackupsList.class); - Assertions.assertEquals("zbonlwnt", model.value().get(0).label()); - Assertions.assertEquals("mrv", model.value().get(0).volumeResourceId()); - Assertions.assertEquals(true, model.value().get(0).useExistingSnapshot()); - Assertions.assertEquals("vbtqgsfraoyzk", model.value().get(0).snapshotName()); - Assertions.assertEquals("l", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupsList model = new BackupsList().withValue(Arrays.asList( - new BackupInner().withLabel("zbonlwnt").withVolumeResourceId("mrv").withUseExistingSnapshot(true) - .withSnapshotName("vbtqgsfraoyzk"), - new BackupInner().withLabel("fdsd").withVolumeResourceId("hewpusdsttwv").withUseExistingSnapshot(true) - .withSnapshotName("bejdcn"), - new BackupInner().withLabel("qu").withVolumeResourceId("piyylhalnswhccsp").withUseExistingSnapshot(true) - .withSnapshotName("vwitqscyw"), - new BackupInner().withLabel("csdtclusiypbs").withVolumeResourceId("eadcygqukyhejhz") - .withUseExistingSnapshot(true).withSnapshotName("fpel"))) - .withNextLink("l"); - model = BinaryData.fromObject(model).toObject(BackupsList.class); - Assertions.assertEquals("zbonlwnt", model.value().get(0).label()); - Assertions.assertEquals("mrv", model.value().get(0).volumeResourceId()); - Assertions.assertEquals(true, model.value().get(0).useExistingSnapshot()); - Assertions.assertEquals("vbtqgsfraoyzk", model.value().get(0).snapshotName()); - Assertions.assertEquals("l", model.nextLink()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsMigrationRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsMigrationRequestTests.java deleted file mode 100644 index e2cb61869ea66..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsMigrationRequestTests.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.BackupsMigrationRequest; -import org.junit.jupiter.api.Assertions; - -public final class BackupsMigrationRequestTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - BackupsMigrationRequest model - = BinaryData.fromString("{\"backupVaultId\":\"jpvd\"}").toObject(BackupsMigrationRequest.class); - Assertions.assertEquals("jpvd", model.backupVaultId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - BackupsMigrationRequest model = new BackupsMigrationRequest().withBackupVaultId("jpvd"); - model = BinaryData.fromObject(model).toObject(BackupsMigrationRequest.class); - Assertions.assertEquals("jpvd", model.backupVaultId()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BreakFileLocksRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BreakFileLocksRequestTests.java index b6130fc8319fe..e5df6fae0f895 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BreakFileLocksRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BreakFileLocksRequestTests.java @@ -12,18 +12,18 @@ public final class BreakFileLocksRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { BreakFileLocksRequest model - = BinaryData.fromString("{\"clientIp\":\"pqiiobyuqe\",\"confirmRunningDisruptiveOperation\":true}") + = BinaryData.fromString("{\"clientIp\":\"adbzmnvdfznud\",\"confirmRunningDisruptiveOperation\":true}") .toObject(BreakFileLocksRequest.class); - Assertions.assertEquals("pqiiobyuqe", model.clientIp()); + Assertions.assertEquals("adbzmnvdfznud", model.clientIp()); Assertions.assertEquals(true, model.confirmRunningDisruptiveOperation()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { BreakFileLocksRequest model - = new BreakFileLocksRequest().withClientIp("pqiiobyuqe").withConfirmRunningDisruptiveOperation(true); + = new BreakFileLocksRequest().withClientIp("adbzmnvdfznud").withConfirmRunningDisruptiveOperation(true); model = BinaryData.fromObject(model).toObject(BreakFileLocksRequest.class); - Assertions.assertEquals("pqiiobyuqe", model.clientIp()); + Assertions.assertEquals("adbzmnvdfznud", model.clientIp()); Assertions.assertEquals(true, model.confirmRunningDisruptiveOperation()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolInnerTests.java index 88e2633b9c168..4fe2803e7626c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolInnerTests.java @@ -17,30 +17,30 @@ public final class CapacityPoolInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CapacityPoolInner model = BinaryData.fromString( - "{\"etag\":\"zcjrvxdjzlmwlx\",\"properties\":{\"poolId\":\"ug\",\"size\":6340547465391917507,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"awjvzunluthnnp\",\"totalThroughputMibps\":83.22846,\"utilizedThroughputMibps\":85.84867,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Single\"},\"location\":\"aejxd\",\"tags\":{\"dzumveekg\":\"skzbb\",\"bsjyofdx\":\"wozuhkf\",\"oekqvk\":\"uusdttouwa\",\"vbxwyjsflhh\":\"lns\"},\"id\":\"aalnjixi\",\"name\":\"xyawj\",\"type\":\"yaqcslyjpkiidz\"}") + "{\"etag\":\"fqkquj\",\"properties\":{\"poolId\":\"suyonobglaocq\",\"size\":3198824190275494463,\"serviceLevel\":\"Premium\",\"provisioningState\":\"g\",\"totalThroughputMibps\":24.563444,\"utilizedThroughputMibps\":78.17242,\"qosType\":\"Auto\",\"coolAccess\":true,\"encryptionType\":\"Single\"},\"location\":\"vwfudwpzntxhd\",\"tags\":{\"rxsbkyvp\":\"rqjbhckfrl\",\"uzbpzkafku\":\"ca\",\"rnwb\":\"b\"},\"id\":\"ehhseyvjusrts\",\"name\":\"hspkdeemao\",\"type\":\"mx\"}") .toObject(CapacityPoolInner.class); - Assertions.assertEquals("aejxd", model.location()); - Assertions.assertEquals("skzbb", model.tags().get("dzumveekg")); - Assertions.assertEquals(6340547465391917507L, model.size()); - Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); - Assertions.assertEquals(QosType.MANUAL, model.qosType()); - Assertions.assertEquals(false, model.coolAccess()); + Assertions.assertEquals("vwfudwpzntxhd", model.location()); + Assertions.assertEquals("rqjbhckfrl", model.tags().get("rxsbkyvp")); + Assertions.assertEquals(3198824190275494463L, model.size()); + Assertions.assertEquals(ServiceLevel.PREMIUM, model.serviceLevel()); + Assertions.assertEquals(QosType.AUTO, model.qosType()); + Assertions.assertEquals(true, model.coolAccess()); Assertions.assertEquals(EncryptionType.SINGLE, model.encryptionType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CapacityPoolInner model = new CapacityPoolInner().withLocation("aejxd") - .withTags(mapOf("dzumveekg", "skzbb", "bsjyofdx", "wozuhkf", "oekqvk", "uusdttouwa", "vbxwyjsflhh", "lns")) - .withSize(6340547465391917507L).withServiceLevel(ServiceLevel.ULTRA).withQosType(QosType.MANUAL) - .withCoolAccess(false).withEncryptionType(EncryptionType.SINGLE); + CapacityPoolInner model = new CapacityPoolInner().withLocation("vwfudwpzntxhd") + .withTags(mapOf("rxsbkyvp", "rqjbhckfrl", "uzbpzkafku", "ca", "rnwb", "b")).withSize(3198824190275494463L) + .withServiceLevel(ServiceLevel.PREMIUM).withQosType(QosType.AUTO).withCoolAccess(true) + .withEncryptionType(EncryptionType.SINGLE); model = BinaryData.fromObject(model).toObject(CapacityPoolInner.class); - Assertions.assertEquals("aejxd", model.location()); - Assertions.assertEquals("skzbb", model.tags().get("dzumveekg")); - Assertions.assertEquals(6340547465391917507L, model.size()); - Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); - Assertions.assertEquals(QosType.MANUAL, model.qosType()); - Assertions.assertEquals(false, model.coolAccess()); + Assertions.assertEquals("vwfudwpzntxhd", model.location()); + Assertions.assertEquals("rqjbhckfrl", model.tags().get("rxsbkyvp")); + Assertions.assertEquals(3198824190275494463L, model.size()); + Assertions.assertEquals(ServiceLevel.PREMIUM, model.serviceLevel()); + Assertions.assertEquals(QosType.AUTO, model.qosType()); + Assertions.assertEquals(true, model.coolAccess()); Assertions.assertEquals(EncryptionType.SINGLE, model.encryptionType()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolListTests.java index c56007cf8f47c..133c670efc918 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolListTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolListTests.java @@ -19,43 +19,43 @@ public final class CapacityPoolListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CapacityPoolList model = BinaryData.fromString( - "{\"value\":[{\"etag\":\"uahaquhcdhmd\",\"properties\":{\"poolId\":\"laexqp\",\"size\":6851389169185127110,\"serviceLevel\":\"StandardZRS\",\"provisioningState\":\"ws\",\"totalThroughputMibps\":4.791844,\"utilizedThroughputMibps\":44.560753,\"qosType\":\"Auto\",\"coolAccess\":false,\"encryptionType\":\"Double\"},\"location\":\"lf\",\"tags\":{\"b\":\"gwb\",\"dawkzbali\":\"e\",\"hashsfwxosow\":\"urqhaka\"},\"id\":\"xcug\",\"name\":\"cjooxdjebwpucwwf\",\"type\":\"ovbvmeueciv\"},{\"etag\":\"zceuojgjrw\",\"properties\":{\"poolId\":\"eiotwmcdytdx\",\"size\":1652378653214086505,\"serviceLevel\":\"Standard\",\"provisioningState\":\"rjaw\",\"totalThroughputMibps\":30.658787,\"utilizedThroughputMibps\":90.02147,\"qosType\":\"Auto\",\"coolAccess\":false,\"encryptionType\":\"Single\"},\"location\":\"bkpyc\",\"tags\":{\"auwhvylwzbtdhx\":\"wndnhj\"},\"id\":\"jznb\",\"name\":\"pow\",\"type\":\"wpr\"},{\"etag\":\"lve\",\"properties\":{\"poolId\":\"lupj\",\"size\":4993789432354588709,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"obbc\",\"totalThroughputMibps\":4.627323,\"utilizedThroughputMibps\":29.920107,\"qosType\":\"Manual\",\"coolAccess\":true,\"encryptionType\":\"Double\"},\"location\":\"pbewtghfgblcgwx\",\"tags\":{\"hjkbegibtnmxieb\":\"v\",\"zjuzgwyz\":\"waloayqcgwr\",\"pwxqp\":\"htxongmtsavjc\",\"yvxqtayriwwroy\":\"rknftguvriuhprwm\"},\"id\":\"bexrmcq\",\"name\":\"bycnojvkn\",\"type\":\"e\"}],\"nextLink\":\"sgzvahapjyzhpv\"}") + "{\"value\":[{\"etag\":\"eaxib\",\"properties\":{\"poolId\":\"jwbhqwalmuz\",\"size\":6168274838392071541,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"pdkzjancuxr\",\"totalThroughputMibps\":66.62854,\"utilizedThroughputMibps\":93.76715,\"qosType\":\"Auto\",\"coolAccess\":false,\"encryptionType\":\"Single\"},\"location\":\"jswztsdbpg\",\"tags\":{\"xbzpfzab\":\"txhp\",\"ovplw\":\"lcuhxwtctyqiklb\"},\"id\":\"bhvgy\",\"name\":\"gu\",\"type\":\"svmkfssxquk\"},{\"etag\":\"plgmgsxnk\",\"properties\":{\"poolId\":\"kde\",\"size\":7084872588122570574,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"opwi\",\"totalThroughputMibps\":41.176914,\"utilizedThroughputMibps\":20.48598,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Double\"},\"location\":\"iuebbaumny\",\"tags\":{\"hsmtxpsiebtfhvp\":\"edeojnabc\"},\"id\":\"sapskr\",\"name\":\"qmhjjdhtld\",\"type\":\"kyzxuutk\"},{\"etag\":\"ws\",\"properties\":{\"poolId\":\"svlxotogtwrup\",\"size\":1377433518907384393,\"serviceLevel\":\"Standard\",\"provisioningState\":\"micykvceoveilo\",\"totalThroughputMibps\":15.381831,\"utilizedThroughputMibps\":64.81392,\"qosType\":\"Auto\",\"coolAccess\":true,\"encryptionType\":\"Double\"},\"location\":\"k\",\"tags\":{\"kphywpnvjto\":\"dhbt\"},\"id\":\"nermcl\",\"name\":\"plpho\",\"type\":\"uscrpabgyepsb\"}],\"nextLink\":\"azqugxywpmueefj\"}") .toObject(CapacityPoolList.class); - Assertions.assertEquals("lf", model.value().get(0).location()); - Assertions.assertEquals("gwb", model.value().get(0).tags().get("b")); - Assertions.assertEquals(6851389169185127110L, model.value().get(0).size()); - Assertions.assertEquals(ServiceLevel.STANDARD_ZRS, model.value().get(0).serviceLevel()); + Assertions.assertEquals("jswztsdbpg", model.value().get(0).location()); + Assertions.assertEquals("txhp", model.value().get(0).tags().get("xbzpfzab")); + Assertions.assertEquals(6168274838392071541L, model.value().get(0).size()); + Assertions.assertEquals(ServiceLevel.ULTRA, model.value().get(0).serviceLevel()); Assertions.assertEquals(QosType.AUTO, model.value().get(0).qosType()); Assertions.assertEquals(false, model.value().get(0).coolAccess()); - Assertions.assertEquals(EncryptionType.DOUBLE, model.value().get(0).encryptionType()); - Assertions.assertEquals("sgzvahapjyzhpv", model.nextLink()); + Assertions.assertEquals(EncryptionType.SINGLE, model.value().get(0).encryptionType()); + Assertions.assertEquals("azqugxywpmueefj", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CapacityPoolList model = new CapacityPoolList().withValue(Arrays.asList( - new CapacityPoolInner().withLocation("lf") - .withTags(mapOf("b", "gwb", "dawkzbali", "e", "hashsfwxosow", "urqhaka")).withSize(6851389169185127110L) - .withServiceLevel(ServiceLevel.STANDARD_ZRS).withQosType(QosType.AUTO).withCoolAccess(false) - .withEncryptionType(EncryptionType.DOUBLE), - new CapacityPoolInner().withLocation("bkpyc").withTags(mapOf("auwhvylwzbtdhx", "wndnhj")) - .withSize(1652378653214086505L).withServiceLevel(ServiceLevel.STANDARD).withQosType(QosType.AUTO) - .withCoolAccess(false).withEncryptionType(EncryptionType.SINGLE), - new CapacityPoolInner().withLocation("pbewtghfgblcgwx") - .withTags(mapOf("hjkbegibtnmxieb", "v", "zjuzgwyz", "waloayqcgwr", "pwxqp", "htxongmtsavjc", - "yvxqtayriwwroy", "rknftguvriuhprwm")) - .withSize(4993789432354588709L).withServiceLevel(ServiceLevel.ULTRA).withQosType(QosType.MANUAL) - .withCoolAccess(true).withEncryptionType(EncryptionType.DOUBLE))) - .withNextLink("sgzvahapjyzhpv"); + CapacityPoolList model + = new CapacityPoolList() + .withValue(Arrays.asList( + new CapacityPoolInner().withLocation("jswztsdbpg") + .withTags(mapOf("xbzpfzab", "txhp", "ovplw", "lcuhxwtctyqiklb")).withSize(6168274838392071541L) + .withServiceLevel(ServiceLevel.ULTRA).withQosType(QosType.AUTO).withCoolAccess(false) + .withEncryptionType(EncryptionType.SINGLE), + new CapacityPoolInner().withLocation("iuebbaumny").withTags(mapOf("hsmtxpsiebtfhvp", "edeojnabc")) + .withSize(7084872588122570574L).withServiceLevel(ServiceLevel.ULTRA).withQosType(QosType.MANUAL) + .withCoolAccess(false).withEncryptionType(EncryptionType.DOUBLE), + new CapacityPoolInner().withLocation("k").withTags(mapOf("kphywpnvjto", "dhbt")) + .withSize(1377433518907384393L).withServiceLevel(ServiceLevel.STANDARD) + .withQosType(QosType.AUTO).withCoolAccess(true).withEncryptionType(EncryptionType.DOUBLE))) + .withNextLink("azqugxywpmueefj"); model = BinaryData.fromObject(model).toObject(CapacityPoolList.class); - Assertions.assertEquals("lf", model.value().get(0).location()); - Assertions.assertEquals("gwb", model.value().get(0).tags().get("b")); - Assertions.assertEquals(6851389169185127110L, model.value().get(0).size()); - Assertions.assertEquals(ServiceLevel.STANDARD_ZRS, model.value().get(0).serviceLevel()); + Assertions.assertEquals("jswztsdbpg", model.value().get(0).location()); + Assertions.assertEquals("txhp", model.value().get(0).tags().get("xbzpfzab")); + Assertions.assertEquals(6168274838392071541L, model.value().get(0).size()); + Assertions.assertEquals(ServiceLevel.ULTRA, model.value().get(0).serviceLevel()); Assertions.assertEquals(QosType.AUTO, model.value().get(0).qosType()); Assertions.assertEquals(false, model.value().get(0).coolAccess()); - Assertions.assertEquals(EncryptionType.DOUBLE, model.value().get(0).encryptionType()); - Assertions.assertEquals("sgzvahapjyzhpv", model.nextLink()); + Assertions.assertEquals(EncryptionType.SINGLE, model.value().get(0).encryptionType()); + Assertions.assertEquals("azqugxywpmueefj", model.nextLink()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolPatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolPatchTests.java index 1c24f4dbc2d99..d9b991c5aaa0a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolPatchTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/CapacityPoolPatchTests.java @@ -15,27 +15,25 @@ public final class CapacityPoolPatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { CapacityPoolPatch model = BinaryData.fromString( - "{\"properties\":{\"size\":7700111059874879228,\"qosType\":\"Auto\",\"coolAccess\":true},\"location\":\"pgylg\",\"tags\":{\"c\":\"txmedj\",\"gktrmgucnapkte\":\"lynqwwncwzzh\",\"pfqbuaceopzf\":\"ellwptfdy\",\"lzdahzxctobgbkdm\":\"rhhuaopppcqeqx\"},\"id\":\"izpost\",\"name\":\"grcfb\",\"type\":\"nrmfqjhhk\"}") + "{\"properties\":{\"size\":2542526403950408941,\"qosType\":\"Auto\",\"coolAccess\":false},\"location\":\"fadmws\",\"tags\":{\"gomz\":\"gvxp\"},\"id\":\"fmisg\",\"name\":\"bnbbeldawkz\",\"type\":\"ali\"}") .toObject(CapacityPoolPatch.class); - Assertions.assertEquals("pgylg", model.location()); - Assertions.assertEquals("txmedj", model.tags().get("c")); - Assertions.assertEquals(7700111059874879228L, model.size()); + Assertions.assertEquals("fadmws", model.location()); + Assertions.assertEquals("gvxp", model.tags().get("gomz")); + Assertions.assertEquals(2542526403950408941L, model.size()); Assertions.assertEquals(QosType.AUTO, model.qosType()); - Assertions.assertEquals(true, model.coolAccess()); + Assertions.assertEquals(false, model.coolAccess()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CapacityPoolPatch model = new CapacityPoolPatch().withLocation("pgylg") - .withTags(mapOf("c", "txmedj", "gktrmgucnapkte", "lynqwwncwzzh", "pfqbuaceopzf", "ellwptfdy", - "lzdahzxctobgbkdm", "rhhuaopppcqeqx")) - .withSize(7700111059874879228L).withQosType(QosType.AUTO).withCoolAccess(true); + CapacityPoolPatch model = new CapacityPoolPatch().withLocation("fadmws").withTags(mapOf("gomz", "gvxp")) + .withSize(2542526403950408941L).withQosType(QosType.AUTO).withCoolAccess(false); model = BinaryData.fromObject(model).toObject(CapacityPoolPatch.class); - Assertions.assertEquals("pgylg", model.location()); - Assertions.assertEquals("txmedj", model.tags().get("c")); - Assertions.assertEquals(7700111059874879228L, model.size()); + Assertions.assertEquals("fadmws", model.location()); + Assertions.assertEquals("gvxp", model.tags().get("gomz")); + Assertions.assertEquals(2542526403950408941L, model.size()); Assertions.assertEquals(QosType.AUTO, model.qosType()); - Assertions.assertEquals(true, model.coolAccess()); + Assertions.assertEquals(false, model.coolAccess()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/DailyScheduleTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/DailyScheduleTests.java index 0c9d7d2fb6fde..556b5da76c64d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/DailyScheduleTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/DailyScheduleTests.java @@ -12,22 +12,22 @@ public final class DailyScheduleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { DailySchedule model = BinaryData.fromString( - "{\"snapshotsToKeep\":445064797,\"hour\":2101936696,\"minute\":1225956555,\"usedBytes\":8980911642598746949}") + "{\"snapshotsToKeep\":1081662268,\"hour\":535447173,\"minute\":88116741,\"usedBytes\":4291870996342138092}") .toObject(DailySchedule.class); - Assertions.assertEquals(445064797, model.snapshotsToKeep()); - Assertions.assertEquals(2101936696, model.hour()); - Assertions.assertEquals(1225956555, model.minute()); - Assertions.assertEquals(8980911642598746949L, model.usedBytes()); + Assertions.assertEquals(1081662268, model.snapshotsToKeep()); + Assertions.assertEquals(535447173, model.hour()); + Assertions.assertEquals(88116741, model.minute()); + Assertions.assertEquals(4291870996342138092L, model.usedBytes()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DailySchedule model = new DailySchedule().withSnapshotsToKeep(445064797).withHour(2101936696) - .withMinute(1225956555).withUsedBytes(8980911642598746949L); + DailySchedule model = new DailySchedule().withSnapshotsToKeep(1081662268).withHour(535447173) + .withMinute(88116741).withUsedBytes(4291870996342138092L); model = BinaryData.fromObject(model).toObject(DailySchedule.class); - Assertions.assertEquals(445064797, model.snapshotsToKeep()); - Assertions.assertEquals(2101936696, model.hour()); - Assertions.assertEquals(1225956555, model.minute()); - Assertions.assertEquals(8980911642598746949L, model.usedBytes()); + Assertions.assertEquals(1081662268, model.snapshotsToKeep()); + Assertions.assertEquals(535447173, model.hour()); + Assertions.assertEquals(88116741, model.minute()); + Assertions.assertEquals(4291870996342138092L, model.usedBytes()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionIdentityTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionIdentityTests.java index c1ece4bb73379..fff20576c954b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionIdentityTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionIdentityTests.java @@ -11,15 +11,16 @@ public final class EncryptionIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - EncryptionIdentity model = BinaryData.fromString("{\"principalId\":\"cq\",\"userAssignedIdentity\":\"ccm\"}") - .toObject(EncryptionIdentity.class); - Assertions.assertEquals("ccm", model.userAssignedIdentity()); + EncryptionIdentity model + = BinaryData.fromString("{\"principalId\":\"pnazzm\",\"userAssignedIdentity\":\"runmp\"}") + .toObject(EncryptionIdentity.class); + Assertions.assertEquals("runmp", model.userAssignedIdentity()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - EncryptionIdentity model = new EncryptionIdentity().withUserAssignedIdentity("ccm"); + EncryptionIdentity model = new EncryptionIdentity().withUserAssignedIdentity("runmp"); model = BinaryData.fromObject(model).toObject(EncryptionIdentity.class); - Assertions.assertEquals("ccm", model.userAssignedIdentity()); + Assertions.assertEquals("runmp", model.userAssignedIdentity()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionMigrationRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionMigrationRequestTests.java deleted file mode 100644 index 7068accb72e58..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/EncryptionMigrationRequestTests.java +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.EncryptionMigrationRequest; -import org.junit.jupiter.api.Assertions; - -public final class EncryptionMigrationRequestTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - EncryptionMigrationRequest model - = BinaryData.fromString("{\"virtualNetworkId\":\"maofmxagkv\",\"privateEndpointId\":\"melmqkrha\"}") - .toObject(EncryptionMigrationRequest.class); - Assertions.assertEquals("maofmxagkv", model.virtualNetworkId()); - Assertions.assertEquals("melmqkrha", model.privateEndpointId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - EncryptionMigrationRequest model - = new EncryptionMigrationRequest().withVirtualNetworkId("maofmxagkv").withPrivateEndpointId("melmqkrha"); - model = BinaryData.fromObject(model).toObject(EncryptionMigrationRequest.class); - Assertions.assertEquals("maofmxagkv", model.virtualNetworkId()); - Assertions.assertEquals("melmqkrha", model.privateEndpointId()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ExportPolicyRuleTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ExportPolicyRuleTests.java index 7aeff05684259..1f0056b38dff8 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ExportPolicyRuleTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ExportPolicyRuleTests.java @@ -13,47 +13,47 @@ public final class ExportPolicyRuleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ExportPolicyRule model = BinaryData.fromString( - "{\"ruleIndex\":1651928304,\"unixReadOnly\":true,\"unixReadWrite\":true,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":false,\"nfsv41\":false,\"allowedClients\":\"ewgdrjervn\",\"hasRootAccess\":false,\"chownMode\":\"Unrestricted\"}") + "{\"ruleIndex\":1115460475,\"unixReadOnly\":false,\"unixReadWrite\":true,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"srknftguv\",\"hasRootAccess\":true,\"chownMode\":\"Unrestricted\"}") .toObject(ExportPolicyRule.class); - Assertions.assertEquals(1651928304, model.ruleIndex()); - Assertions.assertEquals(true, model.unixReadOnly()); + Assertions.assertEquals(1115460475, model.ruleIndex()); + Assertions.assertEquals(false, model.unixReadOnly()); Assertions.assertEquals(true, model.unixReadWrite()); - Assertions.assertEquals(true, model.kerberos5ReadOnly()); - Assertions.assertEquals(true, model.kerberos5ReadWrite()); - Assertions.assertEquals(false, model.kerberos5IReadOnly()); + Assertions.assertEquals(false, model.kerberos5ReadOnly()); + Assertions.assertEquals(false, model.kerberos5ReadWrite()); + Assertions.assertEquals(true, model.kerberos5IReadOnly()); Assertions.assertEquals(false, model.kerberos5IReadWrite()); - Assertions.assertEquals(true, model.kerberos5PReadOnly()); + Assertions.assertEquals(false, model.kerberos5PReadOnly()); Assertions.assertEquals(true, model.kerberos5PReadWrite()); Assertions.assertEquals(false, model.cifs()); - Assertions.assertEquals(false, model.nfsv3()); - Assertions.assertEquals(false, model.nfsv41()); - Assertions.assertEquals("ewgdrjervn", model.allowedClients()); - Assertions.assertEquals(false, model.hasRootAccess()); + Assertions.assertEquals(true, model.nfsv3()); + Assertions.assertEquals(true, model.nfsv41()); + Assertions.assertEquals("srknftguv", model.allowedClients()); + Assertions.assertEquals(true, model.hasRootAccess()); Assertions.assertEquals(ChownMode.UNRESTRICTED, model.chownMode()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExportPolicyRule model = new ExportPolicyRule().withRuleIndex(1651928304).withUnixReadOnly(true) - .withUnixReadWrite(true).withKerberos5ReadOnly(true).withKerberos5ReadWrite(true) - .withKerberos5IReadOnly(false).withKerberos5IReadWrite(false).withKerberos5PReadOnly(true) - .withKerberos5PReadWrite(true).withCifs(false).withNfsv3(false).withNfsv41(false) - .withAllowedClients("ewgdrjervn").withHasRootAccess(false).withChownMode(ChownMode.UNRESTRICTED); + ExportPolicyRule model = new ExportPolicyRule().withRuleIndex(1115460475).withUnixReadOnly(false) + .withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false) + .withKerberos5IReadOnly(true).withKerberos5IReadWrite(false).withKerberos5PReadOnly(false) + .withKerberos5PReadWrite(true).withCifs(false).withNfsv3(true).withNfsv41(true) + .withAllowedClients("srknftguv").withHasRootAccess(true).withChownMode(ChownMode.UNRESTRICTED); model = BinaryData.fromObject(model).toObject(ExportPolicyRule.class); - Assertions.assertEquals(1651928304, model.ruleIndex()); - Assertions.assertEquals(true, model.unixReadOnly()); + Assertions.assertEquals(1115460475, model.ruleIndex()); + Assertions.assertEquals(false, model.unixReadOnly()); Assertions.assertEquals(true, model.unixReadWrite()); - Assertions.assertEquals(true, model.kerberos5ReadOnly()); - Assertions.assertEquals(true, model.kerberos5ReadWrite()); - Assertions.assertEquals(false, model.kerberos5IReadOnly()); + Assertions.assertEquals(false, model.kerberos5ReadOnly()); + Assertions.assertEquals(false, model.kerberos5ReadWrite()); + Assertions.assertEquals(true, model.kerberos5IReadOnly()); Assertions.assertEquals(false, model.kerberos5IReadWrite()); - Assertions.assertEquals(true, model.kerberos5PReadOnly()); + Assertions.assertEquals(false, model.kerberos5PReadOnly()); Assertions.assertEquals(true, model.kerberos5PReadWrite()); Assertions.assertEquals(false, model.cifs()); - Assertions.assertEquals(false, model.nfsv3()); - Assertions.assertEquals(false, model.nfsv41()); - Assertions.assertEquals("ewgdrjervn", model.allowedClients()); - Assertions.assertEquals(false, model.hasRootAccess()); + Assertions.assertEquals(true, model.nfsv3()); + Assertions.assertEquals(true, model.nfsv41()); + Assertions.assertEquals("srknftguv", model.allowedClients()); + Assertions.assertEquals(true, model.hasRootAccess()); Assertions.assertEquals(ChownMode.UNRESTRICTED, model.chownMode()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserRequestTests.java index 4d772e3ef686a..64b02f8137871 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserRequestTests.java @@ -12,14 +12,14 @@ public final class GetGroupIdListForLdapUserRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { GetGroupIdListForLdapUserRequest model - = BinaryData.fromString("{\"username\":\"lp\"}").toObject(GetGroupIdListForLdapUserRequest.class); - Assertions.assertEquals("lp", model.username()); + = BinaryData.fromString("{\"username\":\"vxzbncb\"}").toObject(GetGroupIdListForLdapUserRequest.class); + Assertions.assertEquals("vxzbncb", model.username()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - GetGroupIdListForLdapUserRequest model = new GetGroupIdListForLdapUserRequest().withUsername("lp"); + GetGroupIdListForLdapUserRequest model = new GetGroupIdListForLdapUserRequest().withUsername("vxzbncb"); model = BinaryData.fromObject(model).toObject(GetGroupIdListForLdapUserRequest.class); - Assertions.assertEquals("lp", model.username()); + Assertions.assertEquals("vxzbncb", model.username()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserResponseInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserResponseInnerTests.java index d0814ea85d7ac..ce63612ed5e2b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserResponseInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/GetGroupIdListForLdapUserResponseInnerTests.java @@ -12,17 +12,17 @@ public final class GetGroupIdListForLdapUserResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - GetGroupIdListForLdapUserResponseInner model - = BinaryData.fromString("{\"groupIdsForLdapUser\":[\"ciuqgbdb\",\"t\",\"uvfbtkuwh\",\"mhykojoxafnndl\"]}") - .toObject(GetGroupIdListForLdapUserResponseInner.class); - Assertions.assertEquals("ciuqgbdb", model.groupIdsForLdapUser().get(0)); + GetGroupIdListForLdapUserResponseInner model = BinaryData + .fromString("{\"groupIdsForLdapUser\":[\"pstdbhhxsrzd\",\"ucerscdntnevfi\",\"jmygtdsslswtmwer\",\"ofz\"]}") + .toObject(GetGroupIdListForLdapUserResponseInner.class); + Assertions.assertEquals("pstdbhhxsrzd", model.groupIdsForLdapUser().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { GetGroupIdListForLdapUserResponseInner model = new GetGroupIdListForLdapUserResponseInner() - .withGroupIdsForLdapUser(Arrays.asList("ciuqgbdb", "t", "uvfbtkuwh", "mhykojoxafnndl")); + .withGroupIdsForLdapUser(Arrays.asList("pstdbhhxsrzd", "ucerscdntnevfi", "jmygtdsslswtmwer", "ofz")); model = BinaryData.fromObject(model).toObject(GetGroupIdListForLdapUserResponseInner.class); - Assertions.assertEquals("ciuqgbdb", model.groupIdsForLdapUser().get(0)); + Assertions.assertEquals("pstdbhhxsrzd", model.groupIdsForLdapUser().get(0)); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/HourlyScheduleTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/HourlyScheduleTests.java index d67da8b17a362..fdf5a0ec879de 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/HourlyScheduleTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/HourlyScheduleTests.java @@ -12,20 +12,20 @@ public final class HourlyScheduleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { HourlySchedule model = BinaryData - .fromString("{\"snapshotsToKeep\":2084460640,\"minute\":124681073,\"usedBytes\":3697925526495592044}") + .fromString("{\"snapshotsToKeep\":381030924,\"minute\":1017473450,\"usedBytes\":6968746441071799209}") .toObject(HourlySchedule.class); - Assertions.assertEquals(2084460640, model.snapshotsToKeep()); - Assertions.assertEquals(124681073, model.minute()); - Assertions.assertEquals(3697925526495592044L, model.usedBytes()); + Assertions.assertEquals(381030924, model.snapshotsToKeep()); + Assertions.assertEquals(1017473450, model.minute()); + Assertions.assertEquals(6968746441071799209L, model.usedBytes()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HourlySchedule model = new HourlySchedule().withSnapshotsToKeep(2084460640).withMinute(124681073) - .withUsedBytes(3697925526495592044L); + HourlySchedule model = new HourlySchedule().withSnapshotsToKeep(381030924).withMinute(1017473450) + .withUsedBytes(6968746441071799209L); model = BinaryData.fromObject(model).toObject(HourlySchedule.class); - Assertions.assertEquals(2084460640, model.snapshotsToKeep()); - Assertions.assertEquals(124681073, model.minute()); - Assertions.assertEquals(3697925526495592044L, model.usedBytes()); + Assertions.assertEquals(381030924, model.snapshotsToKeep()); + Assertions.assertEquals(1017473450, model.minute()); + Assertions.assertEquals(6968746441071799209L, model.usedBytes()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/LdapSearchScopeOptTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/LdapSearchScopeOptTests.java index 50a9c31e22ed4..9606c84da0215 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/LdapSearchScopeOptTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/LdapSearchScopeOptTests.java @@ -12,20 +12,20 @@ public final class LdapSearchScopeOptTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { LdapSearchScopeOpt model = BinaryData - .fromString("{\"userDN\":\"psbjta\",\"groupDN\":\"ugxywpmueef\",\"groupMembershipFilter\":\"wfqkquj\"}") + .fromString("{\"userDN\":\"rgvtqag\",\"groupDN\":\"uynhijg\",\"groupMembershipFilter\":\"ebf\"}") .toObject(LdapSearchScopeOpt.class); - Assertions.assertEquals("psbjta", model.userDN()); - Assertions.assertEquals("ugxywpmueef", model.groupDN()); - Assertions.assertEquals("wfqkquj", model.groupMembershipFilter()); + Assertions.assertEquals("rgvtqag", model.userDN()); + Assertions.assertEquals("uynhijg", model.groupDN()); + Assertions.assertEquals("ebf", model.groupMembershipFilter()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LdapSearchScopeOpt model = new LdapSearchScopeOpt().withUserDN("psbjta").withGroupDN("ugxywpmueef") - .withGroupMembershipFilter("wfqkquj"); + LdapSearchScopeOpt model + = new LdapSearchScopeOpt().withUserDN("rgvtqag").withGroupDN("uynhijg").withGroupMembershipFilter("ebf"); model = BinaryData.fromObject(model).toObject(LdapSearchScopeOpt.class); - Assertions.assertEquals("psbjta", model.userDN()); - Assertions.assertEquals("ugxywpmueef", model.groupDN()); - Assertions.assertEquals("wfqkquj", model.groupMembershipFilter()); + Assertions.assertEquals("rgvtqag", model.userDN()); + Assertions.assertEquals("uynhijg", model.groupDN()); + Assertions.assertEquals("ebf", model.groupMembershipFilter()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ListReplicationsTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ListReplicationsTests.java index c78da96cdf7f0..495b71c28aeb4 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ListReplicationsTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ListReplicationsTests.java @@ -16,23 +16,29 @@ public final class ListReplicationsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ListReplications model = BinaryData.fromString( - "{\"value\":[{\"endpointType\":\"src\",\"replicationSchedule\":\"hourly\",\"remoteVolumeResourceId\":\"sytgadgvraea\",\"remoteVolumeRegion\":\"e\"}]}") + "{\"value\":[{\"endpointType\":\"src\",\"replicationSchedule\":\"_10minutely\",\"remoteVolumeResourceId\":\"xogaokonzmnsikv\",\"remoteVolumeRegion\":\"qzeqqkdltfzxm\"},{\"endpointType\":\"src\",\"replicationSchedule\":\"daily\",\"remoteVolumeResourceId\":\"ur\",\"remoteVolumeRegion\":\"dkwobdagx\"},{\"endpointType\":\"src\",\"replicationSchedule\":\"_10minutely\",\"remoteVolumeResourceId\":\"xbxwa\",\"remoteVolumeRegion\":\"ogqxndlkzgxhuri\"}]}") .toObject(ListReplications.class); Assertions.assertEquals(EndpointType.SRC, model.value().get(0).endpointType()); - Assertions.assertEquals(ReplicationSchedule.HOURLY, model.value().get(0).replicationSchedule()); - Assertions.assertEquals("sytgadgvraea", model.value().get(0).remoteVolumeResourceId()); - Assertions.assertEquals("e", model.value().get(0).remoteVolumeRegion()); + Assertions.assertEquals(ReplicationSchedule.ONE_ZEROMINUTELY, model.value().get(0).replicationSchedule()); + Assertions.assertEquals("xogaokonzmnsikv", model.value().get(0).remoteVolumeResourceId()); + Assertions.assertEquals("qzeqqkdltfzxm", model.value().get(0).remoteVolumeRegion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ListReplications model = new ListReplications().withValue(Arrays.asList(new ReplicationInner() - .withEndpointType(EndpointType.SRC).withReplicationSchedule(ReplicationSchedule.HOURLY) - .withRemoteVolumeResourceId("sytgadgvraea").withRemoteVolumeRegion("e"))); + ListReplications model = new ListReplications().withValue(Arrays.asList( + new ReplicationInner().withEndpointType(EndpointType.SRC) + .withReplicationSchedule(ReplicationSchedule.ONE_ZEROMINUTELY) + .withRemoteVolumeResourceId("xogaokonzmnsikv").withRemoteVolumeRegion("qzeqqkdltfzxm"), + new ReplicationInner().withEndpointType(EndpointType.SRC).withReplicationSchedule(ReplicationSchedule.DAILY) + .withRemoteVolumeResourceId("ur").withRemoteVolumeRegion("dkwobdagx"), + new ReplicationInner().withEndpointType(EndpointType.SRC) + .withReplicationSchedule(ReplicationSchedule.ONE_ZEROMINUTELY).withRemoteVolumeResourceId("xbxwa") + .withRemoteVolumeRegion("ogqxndlkzgxhuri"))); model = BinaryData.fromObject(model).toObject(ListReplications.class); Assertions.assertEquals(EndpointType.SRC, model.value().get(0).endpointType()); - Assertions.assertEquals(ReplicationSchedule.HOURLY, model.value().get(0).replicationSchedule()); - Assertions.assertEquals("sytgadgvraea", model.value().get(0).remoteVolumeResourceId()); - Assertions.assertEquals("e", model.value().get(0).remoteVolumeRegion()); + Assertions.assertEquals(ReplicationSchedule.ONE_ZEROMINUTELY, model.value().get(0).replicationSchedule()); + Assertions.assertEquals("xogaokonzmnsikv", model.value().get(0).remoteVolumeResourceId()); + Assertions.assertEquals("qzeqqkdltfzxm", model.value().get(0).remoteVolumeRegion()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ManagedServiceIdentityTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ManagedServiceIdentityTests.java index 22979162c4c50..8a5c74e9c6fdd 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ManagedServiceIdentityTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ManagedServiceIdentityTests.java @@ -16,19 +16,19 @@ public final class ManagedServiceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ManagedServiceIdentity model = BinaryData.fromString( - "{\"principalId\":\"b0db07e2-0d55-4a16-9043-862d6d43ce8b\",\"tenantId\":\"546cc382-9f0b-4fe7-b2d9-4dfa7e051a07\",\"type\":\"None\",\"userAssignedIdentities\":{\"moyrxvwfudwpz\":{\"principalId\":\"d68aff32-8de2-4773-a763-52f76c726951\",\"clientId\":\"dd60b7cc-a6c3-4f4d-ba11-d942001d4965\"},\"hdzhlrqj\":{\"principalId\":\"0d3e63e2-482a-425c-abe1-8b6f5e7e27ff\",\"clientId\":\"35fa09f1-945c-48a6-a4ac-96cbe97a0fe2\"},\"kfrlhrxsbky\":{\"principalId\":\"780afe23-a49e-4ec3-a63d-d7475b36aef6\",\"clientId\":\"61d91fa7-c397-4758-89ad-109b0a8c16c8\"},\"ca\":{\"principalId\":\"16f6d962-c37b-40e2-9acd-46244b8407cb\",\"clientId\":\"376332a8-28c1-4b72-96b7-a973c8e61ebd\"}}}") + "{\"principalId\":\"c4737fa3-4a9f-43d8-a563-ea2ca59172dd\",\"tenantId\":\"28cd2a9d-619f-471e-8d91-258babd14920\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"nlankxmyskpb\":{\"principalId\":\"b6186b30-9007-4479-9a71-627d4053c229\",\"clientId\":\"b922a97d-6139-4a67-9382-cad812c59f60\"},\"btkcxywnytnrsyn\":{\"principalId\":\"b9b0e7f4-0acc-4924-8d18-4279007d055b\",\"clientId\":\"cdd0c5db-f4a1-40eb-ab94-cb2d252c5933\"},\"dybyxczfclhaa\":{\"principalId\":\"dfdf6aa4-6678-446f-a0c0-f31d89329182\",\"clientId\":\"bbb6aa6b-c831-40e1-83c7-a052a8bd5bfc\"},\"abphlw\":{\"principalId\":\"2be080cf-710e-40ea-96df-3d128bbee929\",\"clientId\":\"86c401f1-b61b-4d9a-a400-630466fd5fe7\"}}}") .toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedServiceIdentity model - = new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE).withUserAssignedIdentities( - mapOf("moyrxvwfudwpz", new UserAssignedIdentity(), "hdzhlrqj", new UserAssignedIdentity(), - "kfrlhrxsbky", new UserAssignedIdentity(), "ca", new UserAssignedIdentity())); + ManagedServiceIdentity model = new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities( + mapOf("nlankxmyskpb", new UserAssignedIdentity(), "btkcxywnytnrsyn", new UserAssignedIdentity(), + "dybyxczfclhaa", new UserAssignedIdentity(), "abphlw", new UserAssignedIdentity())); model = BinaryData.fromObject(model).toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); + Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.type()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MonthlyScheduleTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MonthlyScheduleTests.java index 93a09bb25d73e..f1bbff645ba93 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MonthlyScheduleTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MonthlyScheduleTests.java @@ -12,24 +12,24 @@ public final class MonthlyScheduleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MonthlySchedule model = BinaryData.fromString( - "{\"snapshotsToKeep\":1895840108,\"daysOfMonth\":\"efkdlf\",\"hour\":1731932488,\"minute\":1662630957,\"usedBytes\":2710412149295788675}") + "{\"snapshotsToKeep\":1521931041,\"daysOfMonth\":\"krmnjijpxacqqud\",\"hour\":402319478,\"minute\":217779301,\"usedBytes\":3011567239664455561}") .toObject(MonthlySchedule.class); - Assertions.assertEquals(1895840108, model.snapshotsToKeep()); - Assertions.assertEquals("efkdlf", model.daysOfMonth()); - Assertions.assertEquals(1731932488, model.hour()); - Assertions.assertEquals(1662630957, model.minute()); - Assertions.assertEquals(2710412149295788675L, model.usedBytes()); + Assertions.assertEquals(1521931041, model.snapshotsToKeep()); + Assertions.assertEquals("krmnjijpxacqqud", model.daysOfMonth()); + Assertions.assertEquals(402319478, model.hour()); + Assertions.assertEquals(217779301, model.minute()); + Assertions.assertEquals(3011567239664455561L, model.usedBytes()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MonthlySchedule model = new MonthlySchedule().withSnapshotsToKeep(1895840108).withDaysOfMonth("efkdlf") - .withHour(1731932488).withMinute(1662630957).withUsedBytes(2710412149295788675L); + MonthlySchedule model = new MonthlySchedule().withSnapshotsToKeep(1521931041).withDaysOfMonth("krmnjijpxacqqud") + .withHour(402319478).withMinute(217779301).withUsedBytes(3011567239664455561L); model = BinaryData.fromObject(model).toObject(MonthlySchedule.class); - Assertions.assertEquals(1895840108, model.snapshotsToKeep()); - Assertions.assertEquals("efkdlf", model.daysOfMonth()); - Assertions.assertEquals(1731932488, model.hour()); - Assertions.assertEquals(1662630957, model.minute()); - Assertions.assertEquals(2710412149295788675L, model.usedBytes()); + Assertions.assertEquals(1521931041, model.snapshotsToKeep()); + Assertions.assertEquals("krmnjijpxacqqud", model.daysOfMonth()); + Assertions.assertEquals(402319478, model.hour()); + Assertions.assertEquals(217779301, model.minute()); + Assertions.assertEquals(3011567239664455561L, model.usedBytes()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MountTargetPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MountTargetPropertiesTests.java index 2581d9cda0f27..f17bc17b072f6 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MountTargetPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/MountTargetPropertiesTests.java @@ -12,18 +12,18 @@ public final class MountTargetPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { MountTargetProperties model = BinaryData.fromString( - "{\"mountTargetId\":\"hin\",\"fileSystemId\":\"oygmift\",\"ipAddress\":\"zdnds\",\"smbServerFqdn\":\"nayqi\"}") + "{\"mountTargetId\":\"wmdyvxqtay\",\"fileSystemId\":\"iwwroyqbexrmc\",\"ipAddress\":\"bycnojvkn\",\"smbServerFqdn\":\"fqsgzvahapjy\"}") .toObject(MountTargetProperties.class); - Assertions.assertEquals("oygmift", model.fileSystemId()); - Assertions.assertEquals("nayqi", model.smbServerFqdn()); + Assertions.assertEquals("iwwroyqbexrmc", model.fileSystemId()); + Assertions.assertEquals("fqsgzvahapjy", model.smbServerFqdn()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { MountTargetProperties model - = new MountTargetProperties().withFileSystemId("oygmift").withSmbServerFqdn("nayqi"); + = new MountTargetProperties().withFileSystemId("iwwroyqbexrmc").withSmbServerFqdn("fqsgzvahapjy"); model = BinaryData.fromObject(model).toObject(MountTargetProperties.class); - Assertions.assertEquals("oygmift", model.fileSystemId()); - Assertions.assertEquals("nayqi", model.smbServerFqdn()); + Assertions.assertEquals("iwwroyqbexrmc", model.fileSystemId()); + Assertions.assertEquals("fqsgzvahapjy", model.smbServerFqdn()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetWithResponseMockTests.java index 1153101e23045..47f40b6f4a047 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsGetWithResponseMockTests.java @@ -30,7 +30,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"current\":1208874486,\"default\":2137490024},\"id\":\"jhlfzswpchwahf\",\"name\":\"ousnfepgfewe\",\"type\":\"wlyxgncxyk\"}"; + = "{\"properties\":{\"current\":1272047729,\"default\":923595041},\"id\":\"bqplh\",\"name\":\"vnuuepzl\",\"type\":\"phwzsoldweyuqdu\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,7 +48,7 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SubscriptionQuotaItem response = manager.netAppResourceQuotaLimits() - .getWithResponse("poaimlnwiaaomyl", "eazulcs", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("crxgibb", "axconfozauo", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListMockTests.java index 715d1fc564fb7..78053182a5c97 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceQuotaLimitsListMockTests.java @@ -31,7 +31,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"current\":1258117952,\"default\":2128381794},\"id\":\"jnjwltlwt\",\"name\":\"j\",\"type\":\"uktalhsnvkcdmxz\"}]}"; + = "{\"value\":[{\"properties\":{\"current\":937766926,\"default\":689066944},\"id\":\"gdxpg\",\"name\":\"qchiszep\",\"type\":\"nb\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,7 +49,7 @@ public void testList() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.netAppResourceQuotaLimits().list("hujswtwkozzwcul", com.azure.core.util.Context.NONE); + = manager.netAppResourceQuotaLimits().list("inyflnorwmduvwp", com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetWithResponseMockTests.java deleted file mode 100644 index 32e293b0f9b5f..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourceRegionInfosGetWithResponseMockTests.java +++ /dev/null @@ -1,59 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.netapp.NetAppFilesManager; -import com.azure.resourcemanager.netapp.models.RegionInfoResource; -import com.azure.resourcemanager.netapp.models.RegionStorageToNetworkProximity; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class NetAppResourceRegionInfosGetWithResponseMockTests { - @Test - public void testGetWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr - = "{\"properties\":{\"storageToNetworkProximity\":\"T1AndAcrossT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"p\",\"isAvailable\":true},{\"availabilityZone\":\"jtxvzflbqvgaq\",\"isAvailable\":false}]},\"id\":\"fcq\",\"name\":\"srdvetn\",\"type\":\"sdtutnwlduyc\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito.when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito.when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NetAppFilesManager manager = NetAppFilesManager.configure().withHttpClient(httpClient).authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - RegionInfoResource response = manager.netAppResourceRegionInfos() - .getWithResponse("uvyinzqodfvpgs", com.azure.core.util.Context.NONE).getValue(); - - Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_ACROSS_T2, response.storageToNetworkProximity()); - Assertions.assertEquals("p", response.availabilityZoneMappings().get(0).availabilityZone()); - Assertions.assertEquals(true, response.availabilityZoneMappings().get(0).isAvailable()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckFilePathAvailabilityWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckFilePathAvailabilityWithResponseMockTests.java index 3ca33691617e8..92c3d0ad90533 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckFilePathAvailabilityWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckFilePathAvailabilityWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testCheckFilePathAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"isAvailable\":false,\"reason\":\"AlreadyExists\",\"message\":\"lrariaawiu\"}"; + String responseStr = "{\"isAvailable\":false,\"reason\":\"AlreadyExists\",\"message\":\"iv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,13 +50,13 @@ public void testCheckFilePathAvailabilityWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); CheckAvailabilityResponse response = manager.netAppResources() - .checkFilePathAvailabilityWithResponse("nzeyqxtjj", - new FilePathAvailabilityRequest().withName("zqlqhyc").withSubnetId("vodggxdbee"), + .checkFilePathAvailabilityWithResponse("gaifmvik", + new FilePathAvailabilityRequest().withName("bydvkhbejdz").withSubnetId("xcv"), com.azure.core.util.Context.NONE) .getValue(); Assertions.assertEquals(false, response.isAvailable()); Assertions.assertEquals(InAvailabilityReasonType.ALREADY_EXISTS, response.reason()); - Assertions.assertEquals("lrariaawiu", response.message()); + Assertions.assertEquals("iv", response.message()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckNameAvailabilityWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckNameAvailabilityWithResponseMockTests.java index 41cc00ff3f5de..a6895c8db7d0a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckNameAvailabilityWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"isAvailable\":true,\"reason\":\"AlreadyExists\",\"message\":\"lefaxvxilcbtgn\"}"; + String responseStr = "{\"isAvailable\":false,\"reason\":\"AlreadyExists\",\"message\":\"qnrnrpxehuwryk\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,13 +51,15 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); CheckAvailabilityResponse response = manager.netAppResources() - .checkNameAvailabilityWithResponse("bzydvfvfcj", new ResourceNameAvailabilityRequest().withName("aeoisrvh") - .withType(CheckNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS).withResourceGroup("orffukiscv"), + .checkNameAvailabilityWithResponse("xvydfceacvlhvygd", + new ResourceNameAvailabilityRequest().withName("ftumrtwnawjslbiw") + .withType(CheckNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS_CAPACITY_POOLS) + .withResourceGroup("jgcyztsfmznba"), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals(true, response.isAvailable()); + Assertions.assertEquals(false, response.isAvailable()); Assertions.assertEquals(InAvailabilityReasonType.ALREADY_EXISTS, response.reason()); - Assertions.assertEquals("lefaxvxilcbtgn", response.message()); + Assertions.assertEquals("qnrnrpxehuwryk", response.message()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckQuotaAvailabilityWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckQuotaAvailabilityWithResponseMockTests.java index 9aa4b930892ff..899bd86d8bd98 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckQuotaAvailabilityWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesCheckQuotaAvailabilityWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testCheckQuotaAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"isAvailable\":false,\"reason\":\"AlreadyExists\",\"message\":\"uxylfsbtkadpy\"}"; + String responseStr = "{\"isAvailable\":false,\"reason\":\"Invalid\",\"message\":\"ejwcwwqiok\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,15 +51,15 @@ public void testCheckQuotaAvailabilityWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); CheckAvailabilityResponse response = manager.netAppResources() - .checkQuotaAvailabilityWithResponse("gy", - new QuotaAvailabilityRequest().withName("wqfbylyrfgiagt") - .withType(CheckQuotaNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS_CAPACITY_POOLS_VOLUMES) - .withResourceGroup("jocqwogfnzjvusf"), + .checkQuotaAvailabilityWithResponse("lvtno", + new QuotaAvailabilityRequest().withName("qfzgemjdftul") + .withType(CheckQuotaNameResourceTypes.MICROSOFT_NET_APP_NET_APP_ACCOUNTS_CAPACITY_POOLS) + .withResourceGroup("tduceamt"), com.azure.core.util.Context.NONE) .getValue(); Assertions.assertEquals(false, response.isAvailable()); - Assertions.assertEquals(InAvailabilityReasonType.ALREADY_EXISTS, response.reason()); - Assertions.assertEquals("uxylfsbtkadpy", response.message()); + Assertions.assertEquals(InAvailabilityReasonType.INVALID, response.reason()); + Assertions.assertEquals("ejwcwwqiok", response.message()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryNetworkSiblingSetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryNetworkSiblingSetWithResponseMockTests.java index 11cce5e31465c..bd0aef8ef8475 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryNetworkSiblingSetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryNetworkSiblingSetWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testQueryNetworkSiblingSetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"networkSiblingSetId\":\"cact\",\"subnetId\":\"wotey\",\"networkSiblingSetStateId\":\"cluqovekqvgqo\",\"networkFeatures\":\"Basic_Standard\",\"provisioningState\":\"Failed\",\"nicInfoList\":[{\"ipAddress\":\"wyivqikf\",\"volumeResourceIds\":[\"hrfsphuagrtti\",\"teusqczkvyklxu\",\"yja\"]},{\"ipAddress\":\"mmfblcqcuubgqib\",\"volumeResourceIds\":[\"lmetttwgdslqxi\"]}]}"; + = "{\"networkSiblingSetId\":\"odpvruudlgzib\",\"subnetId\":\"ostgkts\",\"networkSiblingSetStateId\":\"dxeclzedqbcvh\",\"networkFeatures\":\"Basic\",\"provisioningState\":\"Updating\",\"nicInfoList\":[{\"ipAddress\":\"kdl\",\"volumeResourceIds\":[\"fbumlkx\",\"rqjfsmlm\",\"txhwgfws\",\"tawc\"]},{\"ipAddress\":\"zbrhubskhudyg\",\"volumeResourceIds\":[\"kkqfqjbvle\",\"rfmluiqtq\",\"fa\"]},{\"ipAddress\":\"vnqqybaryeua\",\"volumeResourceIds\":[\"qabqgzslesjcb\"]}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,15 +51,15 @@ public void testQueryNetworkSiblingSetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); NetworkSiblingSet response = manager.netAppResources() - .queryNetworkSiblingSetWithResponse("plcplcwkhi", - new QueryNetworkSiblingSetRequest().withNetworkSiblingSetId("ihlhzdsqtzb").withSubnetId("rgnowcjhfgm"), + .queryNetworkSiblingSetWithResponse("htba", + new QueryNetworkSiblingSetRequest().withNetworkSiblingSetId("kgxywr").withSubnetId("kpyklyhp"), com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("cact", response.networkSiblingSetId()); - Assertions.assertEquals("wotey", response.subnetId()); - Assertions.assertEquals("cluqovekqvgqo", response.networkSiblingSetStateId()); - Assertions.assertEquals(NetworkFeatures.BASIC_STANDARD, response.networkFeatures()); - Assertions.assertEquals("hrfsphuagrtti", response.nicInfoList().get(0).volumeResourceIds().get(0)); + Assertions.assertEquals("odpvruudlgzib", response.networkSiblingSetId()); + Assertions.assertEquals("ostgkts", response.subnetId()); + Assertions.assertEquals("dxeclzedqbcvh", response.networkSiblingSetStateId()); + Assertions.assertEquals(NetworkFeatures.BASIC, response.networkFeatures()); + Assertions.assertEquals("fbumlkx", response.nicInfoList().get(0).volumeResourceIds().get(0)); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryRegionInfoWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryRegionInfoWithResponseMockTests.java index 5ab2cf6e3dc1c..0f17facfd355c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryRegionInfoWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesQueryRegionInfoWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testQueryRegionInfoWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"storageToNetworkProximity\":\"T1AndT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"grjqctojcmi\",\"isAvailable\":true},{\"availabilityZone\":\"eypefojyqd\",\"isAvailable\":false}]}"; + = "{\"storageToNetworkProximity\":\"T2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"vk\",\"isAvailable\":false},{\"availabilityZone\":\"ql\",\"isAvailable\":false},{\"availabilityZone\":\"tczheydbsdshmkx\",\"isAvailable\":true},{\"availabilityZone\":\"vbbxuripl\",\"isAvailable\":false}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testQueryRegionInfoWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - RegionInfo response - = manager.netAppResources().queryRegionInfoWithResponse("own", com.azure.core.util.Context.NONE).getValue(); + RegionInfo response = manager.netAppResources() + .queryRegionInfoWithResponse("ssxmojms", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2, response.storageToNetworkProximity()); - Assertions.assertEquals("grjqctojcmi", response.availabilityZoneMappings().get(0).availabilityZone()); - Assertions.assertEquals(true, response.availabilityZoneMappings().get(0).isAvailable()); + Assertions.assertEquals(RegionStorageToNetworkProximity.T2, response.storageToNetworkProximity()); + Assertions.assertEquals("vk", response.availabilityZoneMappings().get(0).availabilityZone()); + Assertions.assertEquals(false, response.availabilityZoneMappings().get(0).isAvailable()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesUpdateNetworkSiblingSetMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesUpdateNetworkSiblingSetMockTests.java index dec14e810bcfe..409bcda9becd3 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesUpdateNetworkSiblingSetMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetAppResourcesUpdateNetworkSiblingSetMockTests.java @@ -33,7 +33,7 @@ public void testUpdateNetworkSiblingSet() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"networkSiblingSetId\":\"xslhvnhlabrqnkk\",\"subnetId\":\"cjbtrgaehvvib\",\"networkSiblingSetStateId\":\"jj\",\"networkFeatures\":\"Basic_Standard\",\"provisioningState\":\"Failed\",\"nicInfoList\":[{\"ipAddress\":\"pkxztmoobkl\",\"volumeResourceIds\":[\"dgfcwqmp\",\"maqxzhemjyh\"]}]}"; + = "{\"networkSiblingSetId\":\"vvhmxtdrj\",\"subnetId\":\"tac\",\"networkSiblingSetStateId\":\"bj\",\"networkFeatures\":\"Standard_Basic\",\"provisioningState\":\"Canceled\",\"nicInfoList\":[{\"ipAddress\":\"m\",\"volumeResourceIds\":[\"mguaadraufa\",\"tkahzo\",\"ajjziuxxpshne\",\"kulfg\"]},{\"ipAddress\":\"qubkw\",\"volumeResourceIds\":[\"nrdsutujbazpjuoh\"]}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,15 +50,15 @@ public void testUpdateNetworkSiblingSet() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - NetworkSiblingSet response = manager.netAppResources().updateNetworkSiblingSet("hrmooi", - new UpdateNetworkSiblingSetRequest().withNetworkSiblingSetId("qseypxiutcxa").withSubnetId("zhyrpeto") - .withNetworkSiblingSetStateId("e").withNetworkFeatures(NetworkFeatures.BASIC_STANDARD), + NetworkSiblingSet response = manager.netAppResources().updateNetworkSiblingSet("ernntiewdjcvbquw", + new UpdateNetworkSiblingSetRequest().withNetworkSiblingSetId("behwagohb").withSubnetId("f") + .withNetworkSiblingSetStateId("kmr").withNetworkFeatures(NetworkFeatures.STANDARD), com.azure.core.util.Context.NONE); - Assertions.assertEquals("xslhvnhlabrqnkk", response.networkSiblingSetId()); - Assertions.assertEquals("cjbtrgaehvvib", response.subnetId()); - Assertions.assertEquals("jj", response.networkSiblingSetStateId()); - Assertions.assertEquals(NetworkFeatures.BASIC_STANDARD, response.networkFeatures()); - Assertions.assertEquals("dgfcwqmp", response.nicInfoList().get(0).volumeResourceIds().get(0)); + Assertions.assertEquals("vvhmxtdrj", response.networkSiblingSetId()); + Assertions.assertEquals("tac", response.subnetId()); + Assertions.assertEquals("bj", response.networkSiblingSetStateId()); + Assertions.assertEquals(NetworkFeatures.STANDARD_BASIC, response.networkFeatures()); + Assertions.assertEquals("mguaadraufa", response.nicInfoList().get(0).volumeResourceIds().get(0)); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetworkSiblingSetInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetworkSiblingSetInnerTests.java index 07de00fb395fb..239814c1d340f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetworkSiblingSetInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NetworkSiblingSetInnerTests.java @@ -15,32 +15,26 @@ public final class NetworkSiblingSetInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { NetworkSiblingSetInner model = BinaryData.fromString( - "{\"networkSiblingSetId\":\"ynhijggme\",\"subnetId\":\"siarbutrcvpn\",\"networkSiblingSetStateId\":\"zmhjrunmp\",\"networkFeatures\":\"Basic_Standard\",\"provisioningState\":\"Succeeded\",\"nicInfoList\":[{\"ipAddress\":\"nlankxmyskpb\",\"volumeResourceIds\":[\"btkcxywnytnrsyn\",\"qidybyx\",\"zfcl\",\"aaxdbabphlwrq\"]},{\"ipAddress\":\"ktsthsucocmny\",\"volumeResourceIds\":[\"t\"]},{\"ipAddress\":\"twwrqp\",\"volumeResourceIds\":[\"ckzywbiexzfeyue\",\"xibxujwbhqwalm\",\"zyoxaepdkzjan\"]},{\"ipAddress\":\"xrhdwbavxbniwdjs\",\"volumeResourceIds\":[\"s\",\"bpg\"]}]}") + "{\"networkSiblingSetId\":\"pwhonowkg\",\"subnetId\":\"wankixzbi\",\"networkSiblingSetStateId\":\"eputtmrywnuzoqf\",\"networkFeatures\":\"Standard_Basic\",\"provisioningState\":\"Canceled\",\"nicInfoList\":[{\"ipAddress\":\"cqvyxlwhzlsico\",\"volumeResourceIds\":[\"qnwvlrya\"]}]}") .toObject(NetworkSiblingSetInner.class); - Assertions.assertEquals("ynhijggme", model.networkSiblingSetId()); - Assertions.assertEquals("siarbutrcvpn", model.subnetId()); - Assertions.assertEquals("zmhjrunmp", model.networkSiblingSetStateId()); - Assertions.assertEquals(NetworkFeatures.BASIC_STANDARD, model.networkFeatures()); - Assertions.assertEquals("btkcxywnytnrsyn", model.nicInfoList().get(0).volumeResourceIds().get(0)); + Assertions.assertEquals("pwhonowkg", model.networkSiblingSetId()); + Assertions.assertEquals("wankixzbi", model.subnetId()); + Assertions.assertEquals("eputtmrywnuzoqf", model.networkSiblingSetStateId()); + Assertions.assertEquals(NetworkFeatures.STANDARD_BASIC, model.networkFeatures()); + Assertions.assertEquals("qnwvlrya", model.nicInfoList().get(0).volumeResourceIds().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { NetworkSiblingSetInner model - = new NetworkSiblingSetInner().withNetworkSiblingSetId("ynhijggme").withSubnetId("siarbutrcvpn") - .withNetworkSiblingSetStateId("zmhjrunmp").withNetworkFeatures(NetworkFeatures.BASIC_STANDARD) - .withNicInfoList(Arrays.asList( - new NicInfo() - .withVolumeResourceIds(Arrays.asList("btkcxywnytnrsyn", "qidybyx", "zfcl", "aaxdbabphlwrq")), - new NicInfo().withVolumeResourceIds(Arrays.asList("t")), - new NicInfo() - .withVolumeResourceIds(Arrays.asList("ckzywbiexzfeyue", "xibxujwbhqwalm", "zyoxaepdkzjan")), - new NicInfo().withVolumeResourceIds(Arrays.asList("s", "bpg")))); + = new NetworkSiblingSetInner().withNetworkSiblingSetId("pwhonowkg").withSubnetId("wankixzbi") + .withNetworkSiblingSetStateId("eputtmrywnuzoqf").withNetworkFeatures(NetworkFeatures.STANDARD_BASIC) + .withNicInfoList(Arrays.asList(new NicInfo().withVolumeResourceIds(Arrays.asList("qnwvlrya")))); model = BinaryData.fromObject(model).toObject(NetworkSiblingSetInner.class); - Assertions.assertEquals("ynhijggme", model.networkSiblingSetId()); - Assertions.assertEquals("siarbutrcvpn", model.subnetId()); - Assertions.assertEquals("zmhjrunmp", model.networkSiblingSetStateId()); - Assertions.assertEquals(NetworkFeatures.BASIC_STANDARD, model.networkFeatures()); - Assertions.assertEquals("btkcxywnytnrsyn", model.nicInfoList().get(0).volumeResourceIds().get(0)); + Assertions.assertEquals("pwhonowkg", model.networkSiblingSetId()); + Assertions.assertEquals("wankixzbi", model.subnetId()); + Assertions.assertEquals("eputtmrywnuzoqf", model.networkSiblingSetStateId()); + Assertions.assertEquals(NetworkFeatures.STANDARD_BASIC, model.networkFeatures()); + Assertions.assertEquals("qnwvlrya", model.nicInfoList().get(0).volumeResourceIds().get(0)); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NicInfoTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NicInfoTests.java index a7a8503e10690..2b1bbdd171204 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NicInfoTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/NicInfoTests.java @@ -12,15 +12,16 @@ public final class NicInfoTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NicInfo model = BinaryData.fromString("{\"ipAddress\":\"ytxhp\",\"volumeResourceIds\":[\"zpfzabglc\",\"hxw\"]}") - .toObject(NicInfo.class); - Assertions.assertEquals("zpfzabglc", model.volumeResourceIds().get(0)); + NicInfo model + = BinaryData.fromString("{\"ipAddress\":\"hheunmmqhgyx\",\"volumeResourceIds\":[\"noc\",\"koklya\"]}") + .toObject(NicInfo.class); + Assertions.assertEquals("noc", model.volumeResourceIds().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NicInfo model = new NicInfo().withVolumeResourceIds(Arrays.asList("zpfzabglc", "hxw")); + NicInfo model = new NicInfo().withVolumeResourceIds(Arrays.asList("noc", "koklya")); model = BinaryData.fromObject(model).toObject(NicInfo.class); - Assertions.assertEquals("zpfzabglc", model.volumeResourceIds().get(0)); + Assertions.assertEquals("noc", model.volumeResourceIds().get(0)); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/OperationsListMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/OperationsListMockTests.java index f222eb7110e29..527f97a4a6a7c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/OperationsListMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/OperationsListMockTests.java @@ -33,7 +33,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"name\":\"fzwiivwzjbhyz\",\"display\":{\"provider\":\"rkambt\",\"resource\":\"egv\",\"operation\":\"vuqeqvldspast\",\"description\":\"kkdmfl\"},\"origin\":\"stmjlxrri\",\"properties\":{\"serviceSpecification\":{\"metricSpecifications\":[{\"name\":\"ewchpxlktwku\",\"displayName\":\"ycslevufuztcktyh\",\"displayDescription\":\"qedcgzulwm\",\"unit\":\"qzz\",\"supportedAggregationTypes\":[\"Average\",\"Average\",\"Average\"],\"supportedTimeGrainTypes\":[\"dzgkr\",\"qeevt\",\"epr\",\"utnwytpzdmovzvf\"],\"internalMetricName\":\"awzqadfl\",\"enableRegionalMdmAccount\":false,\"sourceMdmAccount\":\"iglaecx\",\"sourceMdmNamespace\":\"t\",\"dimensions\":[{},{}],\"aggregationType\":\"pvz\",\"fillGapWithZero\":false,\"category\":\"mldgxobfirc\",\"resourceIdDimensionNameOverride\":\"pkc\",\"isInternal\":false},{\"name\":\"riykhyawfvjlbox\",\"displayName\":\"kjlmx\",\"displayDescription\":\"mdy\",\"unit\":\"dwdigumb\",\"supportedAggregationTypes\":[\"Average\",\"Average\",\"Average\",\"Average\"],\"supportedTimeGrainTypes\":[\"jazysdzhezwwvaiq\"],\"internalMetricName\":\"vv\",\"enableRegionalMdmAccount\":false,\"sourceMdmAccount\":\"p\",\"sourceMdmNamespace\":\"qyikvy\",\"dimensions\":[{}],\"aggregationType\":\"avluwmncs\",\"fillGapWithZero\":false,\"category\":\"fybvpoek\",\"resourceIdDimensionNameOverride\":\"gsgbdhuzq\",\"isInternal\":false},{\"name\":\"gkynscliqh\",\"displayName\":\"h\",\"displayDescription\":\"komtkubotppn\",\"unit\":\"xz\",\"supportedAggregationTypes\":[\"Average\",\"Average\",\"Average\"],\"supportedTimeGrainTypes\":[\"cev\"],\"internalMetricName\":\"gtltdhlf\",\"enableRegionalMdmAccount\":true,\"sourceMdmAccount\":\"py\",\"sourceMdmNamespace\":\"gtrd\",\"dimensions\":[{},{}],\"aggregationType\":\"mzzs\",\"fillGapWithZero\":false,\"category\":\"rnysux\",\"resourceIdDimensionNameOverride\":\"rafwgckhocxvdf\",\"isInternal\":false}],\"logSpecifications\":[{\"name\":\"roud\",\"displayName\":\"pavehhr\"},{\"name\":\"bunzozudh\",\"displayName\":\"gkmoyxcdyuibhmfd\"}]}}}]}"; + = "{\"value\":[{\"name\":\"hixbjxyfwnyl\",\"display\":{\"provider\":\"o\",\"resource\":\"ttpkiwkkbnujrywv\",\"operation\":\"lbfpncurd\",\"description\":\"wiithtywub\"},\"origin\":\"bihwqknfdnt\",\"properties\":{\"serviceSpecification\":{\"metricSpecifications\":[{\"name\":\"goihx\",\"displayName\":\"wct\",\"displayDescription\":\"dzjlu\",\"unit\":\"fdlwg\",\"supportedAggregationTypes\":[\"Average\",\"Average\",\"Average\"],\"supportedTimeGrainTypes\":[\"vvt\",\"seinqfiuf\",\"qknp\",\"rgnepttwqmsniffc\"],\"internalMetricName\":\"qnrojlpijnkrxfrd\",\"enableRegionalMdmAccount\":false,\"sourceMdmAccount\":\"atiz\",\"sourceMdmNamespace\":\"onasxifto\",\"dimensions\":[{},{},{},{}],\"aggregationType\":\"hftwesgog\",\"fillGapWithZero\":true,\"category\":\"nnxk\",\"resourceIdDimensionNameOverride\":\"gnyhmossxkkg\",\"isInternal\":true},{\"name\":\"gh\",\"displayName\":\"bdhqxvcxgf\",\"displayDescription\":\"dsofbshrns\",\"unit\":\"uswdv\",\"supportedAggregationTypes\":[\"Average\",\"Average\",\"Average\"],\"supportedTimeGrainTypes\":[\"nvjsrtkfa\",\"nopqgikyzirtx\",\"yuxzejntpsewgi\",\"ilqu\"],\"internalMetricName\":\"ydxtqm\",\"enableRegionalMdmAccount\":false,\"sourceMdmAccount\":\"orgguf\",\"sourceMdmNamespace\":\"aomtbghhavgrvkff\",\"dimensions\":[{},{},{},{}],\"aggregationType\":\"hpjbib\",\"fillGapWithZero\":true,\"category\":\"xumvfcluyovw\",\"resourceIdDimensionNameOverride\":\"bkfezzxscyhwzdgi\",\"isInternal\":true},{\"name\":\"zbomvzzbtdcqvpni\",\"displayName\":\"jviylwdshfs\",\"displayDescription\":\"rbgyefry\",\"unit\":\"gaojf\",\"supportedAggregationTypes\":[\"Average\"],\"supportedTimeGrainTypes\":[\"mrfhirctymox\"],\"internalMetricName\":\"tpipiwyczuhx\",\"enableRegionalMdmAccount\":true,\"sourceMdmAccount\":\"jlihhyus\",\"sourceMdmNamespace\":\"kasdvlm\",\"dimensions\":[{},{},{},{}],\"aggregationType\":\"zxulucvpamrsreu\",\"fillGapWithZero\":true,\"category\":\"risjnhnytxifqjz\",\"resourceIdDimensionNameOverride\":\"mrhublwpc\",\"isInternal\":false}],\"logSpecifications\":[{\"name\":\"jupauut\",\"displayName\":\"oqh\"},{\"name\":\"ejqgw\",\"displayName\":\"nfqn\"},{\"name\":\"ypsxjvfoim\",\"displayName\":\"slirciz\"}]}}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,45 +52,45 @@ public void testList() throws Exception { PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("fzwiivwzjbhyz", response.iterator().next().name()); - Assertions.assertEquals("rkambt", response.iterator().next().display().provider()); - Assertions.assertEquals("egv", response.iterator().next().display().resource()); - Assertions.assertEquals("vuqeqvldspast", response.iterator().next().display().operation()); - Assertions.assertEquals("kkdmfl", response.iterator().next().display().description()); - Assertions.assertEquals("stmjlxrri", response.iterator().next().origin()); - Assertions.assertEquals("ewchpxlktwku", + Assertions.assertEquals("hixbjxyfwnyl", response.iterator().next().name()); + Assertions.assertEquals("o", response.iterator().next().display().provider()); + Assertions.assertEquals("ttpkiwkkbnujrywv", response.iterator().next().display().resource()); + Assertions.assertEquals("lbfpncurd", response.iterator().next().display().operation()); + Assertions.assertEquals("wiithtywub", response.iterator().next().display().description()); + Assertions.assertEquals("bihwqknfdnt", response.iterator().next().origin()); + Assertions.assertEquals("goihx", response.iterator().next().serviceSpecification().metricSpecifications().get(0).name()); - Assertions.assertEquals("ycslevufuztcktyh", + Assertions.assertEquals("wct", response.iterator().next().serviceSpecification().metricSpecifications().get(0).displayName()); - Assertions.assertEquals("qedcgzulwm", + Assertions.assertEquals("dzjlu", response.iterator().next().serviceSpecification().metricSpecifications().get(0).displayDescription()); - Assertions.assertEquals("qzz", + Assertions.assertEquals("fdlwg", response.iterator().next().serviceSpecification().metricSpecifications().get(0).unit()); Assertions.assertEquals(MetricAggregationType.AVERAGE, response.iterator().next().serviceSpecification() .metricSpecifications().get(0).supportedAggregationTypes().get(0)); - Assertions.assertEquals("dzgkr", response.iterator().next().serviceSpecification().metricSpecifications().get(0) + Assertions.assertEquals("vvt", response.iterator().next().serviceSpecification().metricSpecifications().get(0) .supportedTimeGrainTypes().get(0)); - Assertions.assertEquals("awzqadfl", + Assertions.assertEquals("qnrojlpijnkrxfrd", response.iterator().next().serviceSpecification().metricSpecifications().get(0).internalMetricName()); Assertions.assertEquals(false, response.iterator().next().serviceSpecification().metricSpecifications().get(0).enableRegionalMdmAccount()); - Assertions.assertEquals("iglaecx", + Assertions.assertEquals("atiz", response.iterator().next().serviceSpecification().metricSpecifications().get(0).sourceMdmAccount()); - Assertions.assertEquals("t", + Assertions.assertEquals("onasxifto", response.iterator().next().serviceSpecification().metricSpecifications().get(0).sourceMdmNamespace()); - Assertions.assertEquals("pvz", + Assertions.assertEquals("hftwesgog", response.iterator().next().serviceSpecification().metricSpecifications().get(0).aggregationType()); - Assertions.assertEquals(false, + Assertions.assertEquals(true, response.iterator().next().serviceSpecification().metricSpecifications().get(0).fillGapWithZero()); - Assertions.assertEquals("mldgxobfirc", + Assertions.assertEquals("nnxk", response.iterator().next().serviceSpecification().metricSpecifications().get(0).category()); - Assertions.assertEquals("pkc", response.iterator().next().serviceSpecification().metricSpecifications().get(0) - .resourceIdDimensionNameOverride()); - Assertions.assertEquals(false, + Assertions.assertEquals("gnyhmossxkkg", response.iterator().next().serviceSpecification().metricSpecifications() + .get(0).resourceIdDimensionNameOverride()); + Assertions.assertEquals(true, response.iterator().next().serviceSpecification().metricSpecifications().get(0).isInternal()); - Assertions.assertEquals("roud", + Assertions.assertEquals("jupauut", response.iterator().next().serviceSpecification().logSpecifications().get(0).name()); - Assertions.assertEquals("pavehhr", + Assertions.assertEquals("oqh", response.iterator().next().serviceSpecification().logSpecifications().get(0).displayName()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolChangeRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolChangeRequestTests.java index b708ee6fb964c..9f7b9413a38b2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolChangeRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolChangeRequestTests.java @@ -12,14 +12,14 @@ public final class PoolChangeRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PoolChangeRequest model - = BinaryData.fromString("{\"newPoolResourceId\":\"ibwwiftohqkv\"}").toObject(PoolChangeRequest.class); - Assertions.assertEquals("ibwwiftohqkv", model.newPoolResourceId()); + = BinaryData.fromString("{\"newPoolResourceId\":\"gqrlltmuwla\"}").toObject(PoolChangeRequest.class); + Assertions.assertEquals("gqrlltmuwla", model.newPoolResourceId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PoolChangeRequest model = new PoolChangeRequest().withNewPoolResourceId("ibwwiftohqkv"); + PoolChangeRequest model = new PoolChangeRequest().withNewPoolResourceId("gqrlltmuwla"); model = BinaryData.fromObject(model).toObject(PoolChangeRequest.class); - Assertions.assertEquals("ibwwiftohqkv", model.newPoolResourceId()); + Assertions.assertEquals("gqrlltmuwla", model.newPoolResourceId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPatchPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPatchPropertiesTests.java index 5445635760bbd..7f0013c0efdca 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPatchPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPatchPropertiesTests.java @@ -13,20 +13,20 @@ public final class PoolPatchPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PoolPatchProperties model - = BinaryData.fromString("{\"size\":5149673217510977863,\"qosType\":\"Manual\",\"coolAccess\":false}") + = BinaryData.fromString("{\"size\":54169882983487255,\"qosType\":\"Auto\",\"coolAccess\":true}") .toObject(PoolPatchProperties.class); - Assertions.assertEquals(5149673217510977863L, model.size()); - Assertions.assertEquals(QosType.MANUAL, model.qosType()); - Assertions.assertEquals(false, model.coolAccess()); + Assertions.assertEquals(54169882983487255L, model.size()); + Assertions.assertEquals(QosType.AUTO, model.qosType()); + Assertions.assertEquals(true, model.coolAccess()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PoolPatchProperties model = new PoolPatchProperties().withSize(5149673217510977863L).withQosType(QosType.MANUAL) - .withCoolAccess(false); + PoolPatchProperties model + = new PoolPatchProperties().withSize(54169882983487255L).withQosType(QosType.AUTO).withCoolAccess(true); model = BinaryData.fromObject(model).toObject(PoolPatchProperties.class); - Assertions.assertEquals(5149673217510977863L, model.size()); - Assertions.assertEquals(QosType.MANUAL, model.qosType()); - Assertions.assertEquals(false, model.coolAccess()); + Assertions.assertEquals(54169882983487255L, model.size()); + Assertions.assertEquals(QosType.AUTO, model.qosType()); + Assertions.assertEquals(true, model.coolAccess()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPropertiesTests.java index 452a3f6fdfcef..76e0449e36e7b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolPropertiesTests.java @@ -15,24 +15,23 @@ public final class PoolPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { PoolProperties model = BinaryData.fromString( - "{\"poolId\":\"xznelixhnrztf\",\"size\":8485001570661257107,\"serviceLevel\":\"StandardZRS\",\"provisioningState\":\"xknalaulppg\",\"totalThroughputMibps\":96.281746,\"utilizedThroughputMibps\":91.14523,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Double\"}") + "{\"poolId\":\"kv\",\"size\":989086550960883148,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"qkrhahvljua\",\"totalThroughputMibps\":69.46674,\"utilizedThroughputMibps\":50.34383,\"qosType\":\"Auto\",\"coolAccess\":false,\"encryptionType\":\"Double\"}") .toObject(PoolProperties.class); - Assertions.assertEquals(8485001570661257107L, model.size()); - Assertions.assertEquals(ServiceLevel.STANDARD_ZRS, model.serviceLevel()); - Assertions.assertEquals(QosType.MANUAL, model.qosType()); + Assertions.assertEquals(989086550960883148L, model.size()); + Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); + Assertions.assertEquals(QosType.AUTO, model.qosType()); Assertions.assertEquals(false, model.coolAccess()); Assertions.assertEquals(EncryptionType.DOUBLE, model.encryptionType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PoolProperties model - = new PoolProperties().withSize(8485001570661257107L).withServiceLevel(ServiceLevel.STANDARD_ZRS) - .withQosType(QosType.MANUAL).withCoolAccess(false).withEncryptionType(EncryptionType.DOUBLE); + PoolProperties model = new PoolProperties().withSize(989086550960883148L).withServiceLevel(ServiceLevel.ULTRA) + .withQosType(QosType.AUTO).withCoolAccess(false).withEncryptionType(EncryptionType.DOUBLE); model = BinaryData.fromObject(model).toObject(PoolProperties.class); - Assertions.assertEquals(8485001570661257107L, model.size()); - Assertions.assertEquals(ServiceLevel.STANDARD_ZRS, model.serviceLevel()); - Assertions.assertEquals(QosType.MANUAL, model.qosType()); + Assertions.assertEquals(989086550960883148L, model.size()); + Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); + Assertions.assertEquals(QosType.AUTO, model.qosType()); Assertions.assertEquals(false, model.coolAccess()); Assertions.assertEquals(EncryptionType.DOUBLE, model.encryptionType()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateMockTests.java index 93ccf5b924669..41d5402344673 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsCreateOrUpdateMockTests.java @@ -36,7 +36,7 @@ public void testCreateOrUpdate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"etag\":\"gpr\",\"properties\":{\"poolId\":\"tillucbiqtg\",\"size\":56526058003542305,\"serviceLevel\":\"Standard\",\"provisioningState\":\"Succeeded\",\"totalThroughputMibps\":21.163637,\"utilizedThroughputMibps\":90.1712,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Single\"},\"location\":\"bphbqzmizakakank\",\"tags\":{\"zhajoylhjlmuo\":\"n\",\"eecjmeis\":\"xprimrsop\"},\"id\":\"stvasylwxdzaumw\",\"name\":\"oohgu\",\"type\":\"fuzboyjathwtzolb\"}"; + = "{\"etag\":\"ajlyjtlvofqzhv\",\"properties\":{\"poolId\":\"ibyfmo\",\"size\":6727856303003361360,\"serviceLevel\":\"StandardZRS\",\"provisioningState\":\"Succeeded\",\"totalThroughputMibps\":78.12518,\"utilizedThroughputMibps\":77.284424,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Double\"},\"location\":\"hyzsxjrkamb\",\"tags\":{\"nvuqeqvldspa\":\"egv\",\"kdmflvestmjlx\":\"tjb\",\"zapeewchpx\":\"ril\"},\"id\":\"ktwkuziyc\",\"name\":\"levufuztcktyhj\",\"type\":\"qedcgzulwm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -53,19 +53,18 @@ public void testCreateOrUpdate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - CapacityPool response = manager.pools().define("whryvycytdcl").withRegion("bg") - .withExistingNetAppAccount("oylaxxul", "disdosfjbjsvgj").withSize(1887689540352270929L) - .withServiceLevel(ServiceLevel.STANDARD_ZRS) - .withTags(mapOf("utlwxezwzhok", "ylkdghrje", "ehgpp", "bwnhhtql", "csheafidltugsr", "pifhpfeoajvgcxtx")) - .withQosType(QosType.MANUAL).withCoolAccess(true).withEncryptionType(EncryptionType.DOUBLE).create(); + CapacityPool response = manager.pools().define("evwrdnhfuk").withRegion("oqcaaewdaomdj") + .withExistingNetAppAccount("chp", "db").withSize(3770519572473050056L) + .withServiceLevel(ServiceLevel.STANDARD).withTags(mapOf("siykzkdncjdxonbz", "jxxkzbrmsgei", "z", "ggcula")) + .withQosType(QosType.AUTO).withCoolAccess(false).withEncryptionType(EncryptionType.SINGLE).create(); - Assertions.assertEquals("bphbqzmizakakank", response.location()); - Assertions.assertEquals("n", response.tags().get("zhajoylhjlmuo")); - Assertions.assertEquals(56526058003542305L, response.size()); - Assertions.assertEquals(ServiceLevel.STANDARD, response.serviceLevel()); + Assertions.assertEquals("hyzsxjrkamb", response.location()); + Assertions.assertEquals("egv", response.tags().get("nvuqeqvldspa")); + Assertions.assertEquals(6727856303003361360L, response.size()); + Assertions.assertEquals(ServiceLevel.STANDARD_ZRS, response.serviceLevel()); Assertions.assertEquals(QosType.MANUAL, response.qosType()); Assertions.assertEquals(false, response.coolAccess()); - Assertions.assertEquals(EncryptionType.SINGLE, response.encryptionType()); + Assertions.assertEquals(EncryptionType.DOUBLE, response.encryptionType()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsGetWithResponseMockTests.java index 2a34a8defff7c..50d92ee86bbb5 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsGetWithResponseMockTests.java @@ -34,7 +34,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"etag\":\"pxtxsuwp\",\"properties\":{\"poolId\":\"ujwsawddjibabxvi\",\"size\":2713805626921616669,\"serviceLevel\":\"Standard\",\"provisioningState\":\"zeexavoxtfgle\",\"totalThroughputMibps\":0.8543253,\"utilizedThroughputMibps\":13.757277,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Single\"},\"location\":\"gsfjac\",\"tags\":{\"xvodhtnsir\":\"hhxud\",\"gzrcxfailcfxwmdb\":\"dhzmmesckdlp\",\"lnacgcc\":\"xdfgsftufqobr\",\"kizvytn\":\"knh\"},\"id\":\"zvulj\",\"name\":\"aaeranokqgukk\",\"type\":\"qnvb\"}"; + = "{\"etag\":\"yakk\",\"properties\":{\"poolId\":\"zp\",\"size\":8657579168665991549,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"lma\",\"totalThroughputMibps\":64.56741,\"utilizedThroughputMibps\":98.82529,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Double\"},\"location\":\"kdbdgiogsj\",\"tags\":{\"iyhddvi\":\"wqjnob\",\"vm\":\"cegfnmntfp\"},\"id\":\"mfnczd\",\"name\":\"vvbalx\",\"type\":\"l\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -52,14 +52,14 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); CapacityPool response = manager.pools() - .getWithResponse("ekkkzd", "rtkgdojbmxvavref", "eesvecu", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("dabg", "vudtjuewbcihx", "uwhcjyxccybv", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("gsfjac", response.location()); - Assertions.assertEquals("hhxud", response.tags().get("xvodhtnsir")); - Assertions.assertEquals(2713805626921616669L, response.size()); - Assertions.assertEquals(ServiceLevel.STANDARD, response.serviceLevel()); + Assertions.assertEquals("kdbdgiogsj", response.location()); + Assertions.assertEquals("wqjnob", response.tags().get("iyhddvi")); + Assertions.assertEquals(8657579168665991549L, response.size()); + Assertions.assertEquals(ServiceLevel.ULTRA, response.serviceLevel()); Assertions.assertEquals(QosType.MANUAL, response.qosType()); Assertions.assertEquals(false, response.coolAccess()); - Assertions.assertEquals(EncryptionType.SINGLE, response.encryptionType()); + Assertions.assertEquals(EncryptionType.DOUBLE, response.encryptionType()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsListMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsListMockTests.java index dd1df97f3f9dc..7c50b5520b8e0 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsListMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/PoolsListMockTests.java @@ -35,7 +35,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"etag\":\"tjeaahhvjhh\",\"properties\":{\"poolId\":\"kzyb\",\"size\":6116689524962712443,\"serviceLevel\":\"Premium\",\"provisioningState\":\"jks\",\"totalThroughputMibps\":82.007126,\"utilizedThroughputMibps\":94.716125,\"qosType\":\"Auto\",\"coolAccess\":true,\"encryptionType\":\"Single\"},\"location\":\"jednlj\",\"tags\":{\"smjbnkppxyn\":\"euaulxu\",\"gwklnsr\":\"nlsvxeiz\"},\"id\":\"ffeycx\",\"name\":\"ktp\",\"type\":\"ymerteeammxq\"}]}"; + = "{\"value\":[{\"etag\":\"uisavokq\",\"properties\":{\"poolId\":\"fvazivjlfrqttba\",\"size\":8474916357734436058,\"serviceLevel\":\"Ultra\",\"provisioningState\":\"nwxyiop\",\"totalThroughputMibps\":28.38924,\"utilizedThroughputMibps\":76.790695,\"qosType\":\"Manual\",\"coolAccess\":false,\"encryptionType\":\"Double\"},\"location\":\"kdmligovi\",\"tags\":{\"uruocbgo\":\"kpmloa\"},\"id\":\"rb\",\"name\":\"eoybfhjxakvvjgs\",\"type\":\"ordilmywwtkgkxny\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -53,14 +53,14 @@ public void testList() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.pools().list("msidxasicddyvv", "skgfmocwahpq", com.azure.core.util.Context.NONE); + = manager.pools().list("mmqtgqqqxhr", "xrxc", com.azure.core.util.Context.NONE); - Assertions.assertEquals("jednlj", response.iterator().next().location()); - Assertions.assertEquals("euaulxu", response.iterator().next().tags().get("smjbnkppxyn")); - Assertions.assertEquals(6116689524962712443L, response.iterator().next().size()); - Assertions.assertEquals(ServiceLevel.PREMIUM, response.iterator().next().serviceLevel()); - Assertions.assertEquals(QosType.AUTO, response.iterator().next().qosType()); - Assertions.assertEquals(true, response.iterator().next().coolAccess()); - Assertions.assertEquals(EncryptionType.SINGLE, response.iterator().next().encryptionType()); + Assertions.assertEquals("kdmligovi", response.iterator().next().location()); + Assertions.assertEquals("kpmloa", response.iterator().next().tags().get("uruocbgo")); + Assertions.assertEquals(8474916357734436058L, response.iterator().next().size()); + Assertions.assertEquals(ServiceLevel.ULTRA, response.iterator().next().serviceLevel()); + Assertions.assertEquals(QosType.MANUAL, response.iterator().next().qosType()); + Assertions.assertEquals(false, response.iterator().next().coolAccess()); + Assertions.assertEquals(EncryptionType.DOUBLE, response.iterator().next().encryptionType()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/QueryNetworkSiblingSetRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/QueryNetworkSiblingSetRequestTests.java index 3d1c028d4d319..500024b3b9e4d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/QueryNetworkSiblingSetRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/QueryNetworkSiblingSetRequestTests.java @@ -12,18 +12,18 @@ public final class QueryNetworkSiblingSetRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { QueryNetworkSiblingSetRequest model - = BinaryData.fromString("{\"networkSiblingSetId\":\"ik\",\"subnetId\":\"drgvtqagn\"}") + = BinaryData.fromString("{\"networkSiblingSetId\":\"yeamdphagalpb\",\"subnetId\":\"xw\"}") .toObject(QueryNetworkSiblingSetRequest.class); - Assertions.assertEquals("ik", model.networkSiblingSetId()); - Assertions.assertEquals("drgvtqagn", model.subnetId()); + Assertions.assertEquals("yeamdphagalpb", model.networkSiblingSetId()); + Assertions.assertEquals("xw", model.subnetId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { QueryNetworkSiblingSetRequest model - = new QueryNetworkSiblingSetRequest().withNetworkSiblingSetId("ik").withSubnetId("drgvtqagn"); + = new QueryNetworkSiblingSetRequest().withNetworkSiblingSetId("yeamdphagalpb").withSubnetId("xw"); model = BinaryData.fromObject(model).toObject(QueryNetworkSiblingSetRequest.class); - Assertions.assertEquals("ik", model.networkSiblingSetId()); - Assertions.assertEquals("drgvtqagn", model.subnetId()); + Assertions.assertEquals("yeamdphagalpb", model.networkSiblingSetId()); + Assertions.assertEquals("xw", model.subnetId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReestablishReplicationRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReestablishReplicationRequestTests.java index b2d3f54ba5f47..bd53d9da4b3d2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReestablishReplicationRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReestablishReplicationRequestTests.java @@ -12,14 +12,14 @@ public final class ReestablishReplicationRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ReestablishReplicationRequest model - = BinaryData.fromString("{\"sourceVolumeId\":\"koymkcd\"}").toObject(ReestablishReplicationRequest.class); - Assertions.assertEquals("koymkcd", model.sourceVolumeId()); + = BinaryData.fromString("{\"sourceVolumeId\":\"emwabnet\"}").toObject(ReestablishReplicationRequest.class); + Assertions.assertEquals("emwabnet", model.sourceVolumeId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReestablishReplicationRequest model = new ReestablishReplicationRequest().withSourceVolumeId("koymkcd"); + ReestablishReplicationRequest model = new ReestablishReplicationRequest().withSourceVolumeId("emwabnet"); model = BinaryData.fromObject(model).toObject(ReestablishReplicationRequest.class); - Assertions.assertEquals("koymkcd", model.sourceVolumeId()); + Assertions.assertEquals("emwabnet", model.sourceVolumeId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoAvailabilityZoneMappingsItemTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoAvailabilityZoneMappingsItemTests.java index cb4d123413cd2..be85868daaacc 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoAvailabilityZoneMappingsItemTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoAvailabilityZoneMappingsItemTests.java @@ -12,18 +12,18 @@ public final class RegionInfoAvailabilityZoneMappingsItemTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RegionInfoAvailabilityZoneMappingsItem model - = BinaryData.fromString("{\"availabilityZone\":\"dhibnuq\",\"isAvailable\":false}") + = BinaryData.fromString("{\"availabilityZone\":\"fbishcbkha\",\"isAvailable\":false}") .toObject(RegionInfoAvailabilityZoneMappingsItem.class); - Assertions.assertEquals("dhibnuq", model.availabilityZone()); + Assertions.assertEquals("fbishcbkha", model.availabilityZone()); Assertions.assertEquals(false, model.isAvailable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { RegionInfoAvailabilityZoneMappingsItem model - = new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("dhibnuq").withIsAvailable(false); + = new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("fbishcbkha").withIsAvailable(false); model = BinaryData.fromObject(model).toObject(RegionInfoAvailabilityZoneMappingsItem.class); - Assertions.assertEquals("dhibnuq", model.availabilityZone()); + Assertions.assertEquals("fbishcbkha", model.availabilityZone()); Assertions.assertEquals(false, model.isAvailable()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoInnerTests.java index 580f27feb7e6f..1a62697c13984 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoInnerTests.java @@ -15,25 +15,27 @@ public final class RegionInfoInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RegionInfoInner model = BinaryData.fromString( - "{\"storageToNetworkProximity\":\"T2AndAcrossT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"fqkkr\",\"isAvailable\":true},{\"availabilityZone\":\"kgriwflzlfbx\",\"isAvailable\":true},{\"availabilityZone\":\"ycispnqzahmgkb\",\"isAvailable\":true}]}") + "{\"storageToNetworkProximity\":\"T1AndT2AndAcrossT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"c\",\"isAvailable\":true},{\"availabilityZone\":\"sdpydnfyhxdeoejz\",\"isAvailable\":false},{\"availabilityZone\":\"fsj\",\"isAvailable\":true}]}") .toObject(RegionInfoInner.class); - Assertions.assertEquals(RegionStorageToNetworkProximity.T2AND_ACROSS_T2, model.storageToNetworkProximity()); - Assertions.assertEquals("fqkkr", model.availabilityZoneMappings().get(0).availabilityZone()); + Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2AND_ACROSS_T2, + model.storageToNetworkProximity()); + Assertions.assertEquals("c", model.availabilityZoneMappings().get(0).availabilityZone()); Assertions.assertEquals(true, model.availabilityZoneMappings().get(0).isAvailable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - RegionInfoInner model = new RegionInfoInner() - .withStorageToNetworkProximity(RegionStorageToNetworkProximity.T2AND_ACROSS_T2) - .withAvailabilityZoneMappings(Arrays.asList( - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("fqkkr").withIsAvailable(true), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("kgriwflzlfbx").withIsAvailable(true), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("ycispnqzahmgkb") - .withIsAvailable(true))); + RegionInfoInner model + = new RegionInfoInner().withStorageToNetworkProximity(RegionStorageToNetworkProximity.T1AND_T2AND_ACROSS_T2) + .withAvailabilityZoneMappings(Arrays.asList( + new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("c").withIsAvailable(true), + new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("sdpydnfyhxdeoejz") + .withIsAvailable(false), + new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("fsj").withIsAvailable(true))); model = BinaryData.fromObject(model).toObject(RegionInfoInner.class); - Assertions.assertEquals(RegionStorageToNetworkProximity.T2AND_ACROSS_T2, model.storageToNetworkProximity()); - Assertions.assertEquals("fqkkr", model.availabilityZoneMappings().get(0).availabilityZone()); + Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2AND_ACROSS_T2, + model.storageToNetworkProximity()); + Assertions.assertEquals("c", model.availabilityZoneMappings().get(0).availabilityZone()); Assertions.assertEquals(true, model.availabilityZoneMappings().get(0).isAvailable()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoResourceInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoResourceInnerTests.java deleted file mode 100644 index dc77d1d310aa5..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfoResourceInnerTests.java +++ /dev/null @@ -1,37 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; -import com.azure.resourcemanager.netapp.models.RegionInfoAvailabilityZoneMappingsItem; -import com.azure.resourcemanager.netapp.models.RegionStorageToNetworkProximity; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class RegionInfoResourceInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - RegionInfoResourceInner model = BinaryData.fromString( - "{\"properties\":{\"storageToNetworkProximity\":\"T1AndT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"gvcl\",\"isAvailable\":true},{\"availabilityZone\":\"ncghkje\",\"isAvailable\":true}]},\"id\":\"bijhtxfvgxbf\",\"name\":\"mxnehmp\",\"type\":\"ec\"}") - .toObject(RegionInfoResourceInner.class); - Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2, model.storageToNetworkProximity()); - Assertions.assertEquals("gvcl", model.availabilityZoneMappings().get(0).availabilityZone()); - Assertions.assertEquals(true, model.availabilityZoneMappings().get(0).isAvailable()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - RegionInfoResourceInner model = new RegionInfoResourceInner() - .withStorageToNetworkProximity(RegionStorageToNetworkProximity.T1AND_T2) - .withAvailabilityZoneMappings(Arrays.asList( - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("gvcl").withIsAvailable(true), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("ncghkje").withIsAvailable(true))); - model = BinaryData.fromObject(model).toObject(RegionInfoResourceInner.class); - Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2, model.storageToNetworkProximity()); - Assertions.assertEquals("gvcl", model.availabilityZoneMappings().get(0).availabilityZone()); - Assertions.assertEquals(true, model.availabilityZoneMappings().get(0).isAvailable()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfosListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfosListTests.java deleted file mode 100644 index 273bfbf0b71cb..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RegionInfosListTests.java +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; -import com.azure.resourcemanager.netapp.models.RegionInfoAvailabilityZoneMappingsItem; -import com.azure.resourcemanager.netapp.models.RegionInfosList; -import com.azure.resourcemanager.netapp.models.RegionStorageToNetworkProximity; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class RegionInfosListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - RegionInfosList model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"storageToNetworkProximity\":\"T1AndT2AndAcrossT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"qjsdpydnfyhxdeo\",\"isAvailable\":false},{\"availabilityZone\":\"cwif\",\"isAvailable\":false}]},\"id\":\"gzfbishcbk\",\"name\":\"ajdeyeamdphaga\",\"type\":\"pbuxwgipwhon\"},{\"properties\":{\"storageToNetworkProximity\":\"T1AndT2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"ankixzbinjeput\",\"isAvailable\":false},{\"availabilityZone\":\"wnuzoqftiyqzrnkc\",\"isAvailable\":false},{\"availabilityZone\":\"lwh\",\"isAvailable\":true},{\"availabilityZone\":\"cohoq\",\"isAvailable\":false}]},\"id\":\"lryav\",\"name\":\"hheunmmqhgyx\",\"type\":\"konocu\"},{\"properties\":{\"storageToNetworkProximity\":\"T2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"uconuqszfkbey\",\"isAvailable\":true},{\"availabilityZone\":\"mjmwvvjektcx\",\"isAvailable\":true}]},\"id\":\"wlrsffrzpwv\",\"name\":\"qdqgbi\",\"type\":\"ylihkaetckt\"},{\"properties\":{\"storageToNetworkProximity\":\"T2\",\"availabilityZoneMappings\":[{\"availabilityZone\":\"nkymuctqhjfbebrj\",\"isAvailable\":true},{\"availabilityZone\":\"fuwutttxf\",\"isAvailable\":false},{\"availabilityZone\":\"i\",\"isAvailable\":true}]},\"id\":\"epcyvahfnlj\",\"name\":\"yq\",\"type\":\"j\"}],\"nextLink\":\"ujqgidok\"}") - .toObject(RegionInfosList.class); - Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2AND_ACROSS_T2, - model.value().get(0).storageToNetworkProximity()); - Assertions.assertEquals("qjsdpydnfyhxdeo", - model.value().get(0).availabilityZoneMappings().get(0).availabilityZone()); - Assertions.assertEquals(false, model.value().get(0).availabilityZoneMappings().get(0).isAvailable()); - Assertions.assertEquals("ujqgidok", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - RegionInfosList model = new RegionInfosList() - .withValue(Arrays.asList(new RegionInfoResourceInner() - .withStorageToNetworkProximity(RegionStorageToNetworkProximity.T1AND_T2AND_ACROSS_T2) - .withAvailabilityZoneMappings(Arrays.asList( - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("qjsdpydnfyhxdeo") - .withIsAvailable(false), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("cwif").withIsAvailable(false))), - new RegionInfoResourceInner().withStorageToNetworkProximity(RegionStorageToNetworkProximity.T1AND_T2) - .withAvailabilityZoneMappings(Arrays.asList( - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("ankixzbinjeput") - .withIsAvailable(false), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("wnuzoqftiyqzrnkc") - .withIsAvailable(false), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("lwh").withIsAvailable(true), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("cohoq") - .withIsAvailable(false))), - new RegionInfoResourceInner().withStorageToNetworkProximity(RegionStorageToNetworkProximity.T2) - .withAvailabilityZoneMappings(Arrays.asList( - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("uconuqszfkbey") - .withIsAvailable(true), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("mjmwvvjektcx") - .withIsAvailable(true))), - new RegionInfoResourceInner().withStorageToNetworkProximity(RegionStorageToNetworkProximity.T2) - .withAvailabilityZoneMappings(Arrays.asList( - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("nkymuctqhjfbebrj") - .withIsAvailable(true), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("fuwutttxf") - .withIsAvailable(false), - new RegionInfoAvailabilityZoneMappingsItem().withAvailabilityZone("i").withIsAvailable(true))))) - .withNextLink("ujqgidok"); - model = BinaryData.fromObject(model).toObject(RegionInfosList.class); - Assertions.assertEquals(RegionStorageToNetworkProximity.T1AND_T2AND_ACROSS_T2, - model.value().get(0).storageToNetworkProximity()); - Assertions.assertEquals("qjsdpydnfyhxdeo", - model.value().get(0).availabilityZoneMappings().get(0).availabilityZone()); - Assertions.assertEquals(false, model.value().get(0).availabilityZoneMappings().get(0).isAvailable()); - Assertions.assertEquals("ujqgidok", model.nextLink()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RemotePathTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RemotePathTests.java deleted file mode 100644 index 0976f6eecf4dd..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RemotePathTests.java +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.RemotePath; -import org.junit.jupiter.api.Assertions; - -public final class RemotePathTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - RemotePath model = BinaryData - .fromString("{\"externalHostName\":\"wubmwmbesldn\",\"serverName\":\"wwtppj\",\"volumeName\":\"lcxog\"}") - .toObject(RemotePath.class); - Assertions.assertEquals("wubmwmbesldn", model.externalHostname()); - Assertions.assertEquals("wwtppj", model.serverName()); - Assertions.assertEquals("lcxog", model.volumeName()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - RemotePath model - = new RemotePath().withExternalHostname("wubmwmbesldn").withServerName("wwtppj").withVolumeName("lcxog"); - model = BinaryData.fromObject(model).toObject(RemotePath.class); - Assertions.assertEquals("wubmwmbesldn", model.externalHostname()); - Assertions.assertEquals("wwtppj", model.serverName()); - Assertions.assertEquals("lcxog", model.volumeName()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationInnerTests.java index 1892fbb0d044d..a7f0c2a62b520 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationInnerTests.java @@ -14,23 +14,23 @@ public final class ReplicationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ReplicationInner model = BinaryData.fromString( - "{\"endpointType\":\"src\",\"replicationSchedule\":\"hourly\",\"remoteVolumeResourceId\":\"rwlqu\",\"remoteVolumeRegion\":\"jfqka\"}") + "{\"endpointType\":\"dst\",\"replicationSchedule\":\"hourly\",\"remoteVolumeResourceId\":\"dxunkbebxmubyyn\",\"remoteVolumeRegion\":\"lrb\"}") .toObject(ReplicationInner.class); - Assertions.assertEquals(EndpointType.SRC, model.endpointType()); + Assertions.assertEquals(EndpointType.DST, model.endpointType()); Assertions.assertEquals(ReplicationSchedule.HOURLY, model.replicationSchedule()); - Assertions.assertEquals("rwlqu", model.remoteVolumeResourceId()); - Assertions.assertEquals("jfqka", model.remoteVolumeRegion()); + Assertions.assertEquals("dxunkbebxmubyyn", model.remoteVolumeResourceId()); + Assertions.assertEquals("lrb", model.remoteVolumeRegion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReplicationInner model = new ReplicationInner().withEndpointType(EndpointType.SRC) - .withReplicationSchedule(ReplicationSchedule.HOURLY).withRemoteVolumeResourceId("rwlqu") - .withRemoteVolumeRegion("jfqka"); + ReplicationInner model = new ReplicationInner().withEndpointType(EndpointType.DST) + .withReplicationSchedule(ReplicationSchedule.HOURLY).withRemoteVolumeResourceId("dxunkbebxmubyyn") + .withRemoteVolumeRegion("lrb"); model = BinaryData.fromObject(model).toObject(ReplicationInner.class); - Assertions.assertEquals(EndpointType.SRC, model.endpointType()); + Assertions.assertEquals(EndpointType.DST, model.endpointType()); Assertions.assertEquals(ReplicationSchedule.HOURLY, model.replicationSchedule()); - Assertions.assertEquals("rwlqu", model.remoteVolumeResourceId()); - Assertions.assertEquals("jfqka", model.remoteVolumeRegion()); + Assertions.assertEquals("dxunkbebxmubyyn", model.remoteVolumeResourceId()); + Assertions.assertEquals("lrb", model.remoteVolumeRegion()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationObjectTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationObjectTests.java index b8001a9a53fe9..9419b3f4e8153 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationObjectTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationObjectTests.java @@ -6,7 +6,6 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.netapp.models.EndpointType; -import com.azure.resourcemanager.netapp.models.RemotePath; import com.azure.resourcemanager.netapp.models.ReplicationObject; import com.azure.resourcemanager.netapp.models.ReplicationSchedule; import org.junit.jupiter.api.Assertions; @@ -15,31 +14,23 @@ public final class ReplicationObjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ReplicationObject model = BinaryData.fromString( - "{\"replicationId\":\"rsc\",\"endpointType\":\"src\",\"replicationSchedule\":\"hourly\",\"remoteVolumeResourceId\":\"vfiwjmygtdss\",\"remotePath\":{\"externalHostName\":\"wtmwerio\",\"serverName\":\"zpyqsemwab\",\"volumeName\":\"ets\"},\"remoteVolumeRegion\":\"szhedplvw\"}") + "{\"replicationId\":\"xdult\",\"endpointType\":\"dst\",\"replicationSchedule\":\"daily\",\"remoteVolumeResourceId\":\"tdzumveekgpw\",\"remoteVolumeRegion\":\"uh\"}") .toObject(ReplicationObject.class); - Assertions.assertEquals(EndpointType.SRC, model.endpointType()); - Assertions.assertEquals(ReplicationSchedule.HOURLY, model.replicationSchedule()); - Assertions.assertEquals("vfiwjmygtdss", model.remoteVolumeResourceId()); - Assertions.assertEquals("wtmwerio", model.remotePath().externalHostname()); - Assertions.assertEquals("zpyqsemwab", model.remotePath().serverName()); - Assertions.assertEquals("ets", model.remotePath().volumeName()); - Assertions.assertEquals("szhedplvw", model.remoteVolumeRegion()); + Assertions.assertEquals(EndpointType.DST, model.endpointType()); + Assertions.assertEquals(ReplicationSchedule.DAILY, model.replicationSchedule()); + Assertions.assertEquals("tdzumveekgpw", model.remoteVolumeResourceId()); + Assertions.assertEquals("uh", model.remoteVolumeRegion()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ReplicationObject model = new ReplicationObject().withEndpointType(EndpointType.SRC) - .withReplicationSchedule(ReplicationSchedule.HOURLY).withRemoteVolumeResourceId("vfiwjmygtdss") - .withRemotePath( - new RemotePath().withExternalHostname("wtmwerio").withServerName("zpyqsemwab").withVolumeName("ets")) - .withRemoteVolumeRegion("szhedplvw"); + ReplicationObject model = new ReplicationObject().withEndpointType(EndpointType.DST) + .withReplicationSchedule(ReplicationSchedule.DAILY).withRemoteVolumeResourceId("tdzumveekgpw") + .withRemoteVolumeRegion("uh"); model = BinaryData.fromObject(model).toObject(ReplicationObject.class); - Assertions.assertEquals(EndpointType.SRC, model.endpointType()); - Assertions.assertEquals(ReplicationSchedule.HOURLY, model.replicationSchedule()); - Assertions.assertEquals("vfiwjmygtdss", model.remoteVolumeResourceId()); - Assertions.assertEquals("wtmwerio", model.remotePath().externalHostname()); - Assertions.assertEquals("zpyqsemwab", model.remotePath().serverName()); - Assertions.assertEquals("ets", model.remotePath().volumeName()); - Assertions.assertEquals("szhedplvw", model.remoteVolumeRegion()); + Assertions.assertEquals(EndpointType.DST, model.endpointType()); + Assertions.assertEquals(ReplicationSchedule.DAILY, model.replicationSchedule()); + Assertions.assertEquals("tdzumveekgpw", model.remoteVolumeResourceId()); + Assertions.assertEquals("uh", model.remoteVolumeRegion()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationStatusInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationStatusInnerTests.java index dfedc0b9ee798..d051fcfe656ae 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationStatusInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/ReplicationStatusInnerTests.java @@ -14,25 +14,25 @@ public final class ReplicationStatusInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { ReplicationStatusInner model = BinaryData.fromString( - "{\"healthy\":true,\"relationshipStatus\":\"Transferring\",\"mirrorState\":\"Broken\",\"totalProgress\":\"dreqnovvqf\",\"errorMessage\":\"ljxywsu\"}") + "{\"healthy\":true,\"relationshipStatus\":\"Failed\",\"mirrorState\":\"Broken\",\"totalProgress\":\"plvwiwubmwmbes\",\"errorMessage\":\"nkww\"}") .toObject(ReplicationStatusInner.class); Assertions.assertEquals(true, model.healthy()); - Assertions.assertEquals(RelationshipStatus.TRANSFERRING, model.relationshipStatus()); + Assertions.assertEquals(RelationshipStatus.FAILED, model.relationshipStatus()); Assertions.assertEquals(MirrorState.BROKEN, model.mirrorState()); - Assertions.assertEquals("dreqnovvqf", model.totalProgress()); - Assertions.assertEquals("ljxywsu", model.errorMessage()); + Assertions.assertEquals("plvwiwubmwmbes", model.totalProgress()); + Assertions.assertEquals("nkww", model.errorMessage()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { ReplicationStatusInner model - = new ReplicationStatusInner().withHealthy(true).withRelationshipStatus(RelationshipStatus.TRANSFERRING) - .withMirrorState(MirrorState.BROKEN).withTotalProgress("dreqnovvqf").withErrorMessage("ljxywsu"); + = new ReplicationStatusInner().withHealthy(true).withRelationshipStatus(RelationshipStatus.FAILED) + .withMirrorState(MirrorState.BROKEN).withTotalProgress("plvwiwubmwmbes").withErrorMessage("nkww"); model = BinaryData.fromObject(model).toObject(ReplicationStatusInner.class); Assertions.assertEquals(true, model.healthy()); - Assertions.assertEquals(RelationshipStatus.TRANSFERRING, model.relationshipStatus()); + Assertions.assertEquals(RelationshipStatus.FAILED, model.relationshipStatus()); Assertions.assertEquals(MirrorState.BROKEN, model.mirrorState()); - Assertions.assertEquals("dreqnovvqf", model.totalProgress()); - Assertions.assertEquals("ljxywsu", model.errorMessage()); + Assertions.assertEquals("plvwiwubmwmbes", model.totalProgress()); + Assertions.assertEquals("nkww", model.errorMessage()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RestoreStatusInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RestoreStatusInnerTests.java index f11287d40181f..aa690d563d2da 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RestoreStatusInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/RestoreStatusInnerTests.java @@ -11,7 +11,7 @@ public final class RestoreStatusInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { RestoreStatusInner model = BinaryData.fromString( - "{\"healthy\":false,\"relationshipStatus\":\"Transferring\",\"mirrorState\":\"Mirrored\",\"unhealthyReason\":\"ae\",\"errorMessage\":\"ojvdcpzfoqo\",\"totalTransferBytes\":4892524565143747736}") + "{\"healthy\":true,\"relationshipStatus\":\"Unknown\",\"mirrorState\":\"Mirrored\",\"unhealthyReason\":\"jjxd\",\"errorMessage\":\"buukzclewyhml\",\"totalTransferBytes\":4136836492530491965}") .toObject(RestoreStatusInner.class); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotInnerTests.java index f5b6a9b4d9dbd..f3dc91ef8d806 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotInnerTests.java @@ -12,15 +12,15 @@ public final class SnapshotInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotInner model = BinaryData.fromString( - "{\"location\":\"qgxqquezikyw\",\"properties\":{\"snapshotId\":\"kallatmel\",\"created\":\"2021-06-18T06:04:26Z\",\"provisioningState\":\"iccjzkzivgvvcna\"},\"id\":\"hyrnxxmu\",\"name\":\"edndr\",\"type\":\"v\"}") + "{\"location\":\"sg\",\"properties\":{\"snapshotId\":\"c\",\"created\":\"2021-08-29T01:48:19Z\",\"provisioningState\":\"dsjnka\"},\"id\":\"utiiswacf\",\"name\":\"gdkz\",\"type\":\"ewkfvhqcrai\"}") .toObject(SnapshotInner.class); - Assertions.assertEquals("qgxqquezikyw", model.location()); + Assertions.assertEquals("sg", model.location()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnapshotInner model = new SnapshotInner().withLocation("qgxqquezikyw"); + SnapshotInner model = new SnapshotInner().withLocation("sg"); model = BinaryData.fromObject(model).toObject(SnapshotInner.class); - Assertions.assertEquals("qgxqquezikyw", model.location()); + Assertions.assertEquals("sg", model.location()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateWithResponseMockTests.java index 1eb9ebb2b6325..65ed56a34234a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesCreateWithResponseMockTests.java @@ -37,7 +37,7 @@ public void testCreateWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"etag\":\"tft\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1034709707,\"minute\":498260807,\"usedBytes\":5239547603354777125},\"dailySchedule\":{\"snapshotsToKeep\":1037799347,\"hour\":1872397233,\"minute\":505151536,\"usedBytes\":3502756066973311769},\"weeklySchedule\":{\"snapshotsToKeep\":1808572524,\"day\":\"ipgawtxx\",\"hour\":156137612,\"minute\":2059766507,\"usedBytes\":8743829512964463352},\"monthlySchedule\":{\"snapshotsToKeep\":607861165,\"daysOfMonth\":\"tfmpcycilrmcayk\",\"hour\":297687924,\"minute\":275753325,\"usedBytes\":3377592071635505398},\"enabled\":false,\"provisioningState\":\"xwpndfcpf\"},\"location\":\"nthjtwkjaosrxuzv\",\"tags\":{\"mgbzahgxqdlyrtl\":\"ktcqio\",\"katbhjm\":\"laprlt\",\"arvlagunbt\":\"nnbsoqeqa\",\"av\":\"febwlnbmhyreeudz\"},\"id\":\"pdqmjxlyyzglgouw\",\"name\":\"lmjjyuo\",\"type\":\"qtobaxkjeyt\"}"; + = "{\"etag\":\"vreljea\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1671689667,\"minute\":1202304870,\"usedBytes\":1705297672247999328},\"dailySchedule\":{\"snapshotsToKeep\":2073795724,\"hour\":1291910908,\"minute\":264997502,\"usedBytes\":7353815266805175635},\"weeklySchedule\":{\"snapshotsToKeep\":1178556723,\"day\":\"bdkelvidizo\",\"hour\":344121425,\"minute\":1181700002,\"usedBytes\":6191416292355922762},\"monthlySchedule\":{\"snapshotsToKeep\":1331407717,\"daysOfMonth\":\"dgnwncypuuwwlt\",\"hour\":740306752,\"minute\":1798680046,\"usedBytes\":5288027656858110726},\"enabled\":true,\"provisioningState\":\"if\"},\"location\":\"hmkdasvfl\",\"tags\":{\"oldforobw\":\"xcudchxgsr\",\"hfovvacqpbtu\":\"lvizb\"},\"id\":\"dxe\",\"name\":\"zab\",\"type\":\"elawumu\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -54,39 +54,38 @@ public void testCreateWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - SnapshotPolicy response = manager.snapshotPolicies().define("qwhix").withRegion("crwnamikz") - .withExistingNetAppAccount("gehkfkimrtixokff", "yinljqe") - .withTags(mapOf("ziqgfuh", "qbsms", "hczznvf", "kzruswh", "wwixzvumw", "ycjsx")) - .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(2089696921).withMinute(1519805829) - .withUsedBytes(4952257041034089193L)) - .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1065880701).withHour(1806892496) - .withMinute(667519337).withUsedBytes(3451295234524077317L)) - .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1949096000).withDay("b").withHour(1840305152) - .withMinute(1547135133).withUsedBytes(8260240583130824742L)) - .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(648351672).withDaysOfMonth("zcilnghg") - .withHour(403294934).withMinute(884114239).withUsedBytes(4618802136241516087L)) + SnapshotPolicy response = manager.snapshotPolicies().define("ieekpndzaa").withRegion("czexrxzbujrtrhqv") + .withExistingNetAppAccount("ttadijae", "kmr").withTags(mapOf("gnl", "vk")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(517735759).withMinute(161734406) + .withUsedBytes(5927921282145979607L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(606462246).withHour(619135714) + .withMinute(310350754).withUsedBytes(8944608160105267933L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(203387229).withDay("hheioqaqhvseuf") + .withHour(695023637).withMinute(703231006).withUsedBytes(6853300792215489751L)) + .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(1303034345).withDaysOfMonth("sismjqfrddga") + .withHour(15155929).withMinute(1363527870).withUsedBytes(490169068933879788L)) .withEnabled(false).create(); - Assertions.assertEquals("nthjtwkjaosrxuzv", response.location()); - Assertions.assertEquals("ktcqio", response.tags().get("mgbzahgxqdlyrtl")); - Assertions.assertEquals(1034709707, response.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(498260807, response.hourlySchedule().minute()); - Assertions.assertEquals(5239547603354777125L, response.hourlySchedule().usedBytes()); - Assertions.assertEquals(1037799347, response.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1872397233, response.dailySchedule().hour()); - Assertions.assertEquals(505151536, response.dailySchedule().minute()); - Assertions.assertEquals(3502756066973311769L, response.dailySchedule().usedBytes()); - Assertions.assertEquals(1808572524, response.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("ipgawtxx", response.weeklySchedule().day()); - Assertions.assertEquals(156137612, response.weeklySchedule().hour()); - Assertions.assertEquals(2059766507, response.weeklySchedule().minute()); - Assertions.assertEquals(8743829512964463352L, response.weeklySchedule().usedBytes()); - Assertions.assertEquals(607861165, response.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("tfmpcycilrmcayk", response.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(297687924, response.monthlySchedule().hour()); - Assertions.assertEquals(275753325, response.monthlySchedule().minute()); - Assertions.assertEquals(3377592071635505398L, response.monthlySchedule().usedBytes()); - Assertions.assertEquals(false, response.enabled()); + Assertions.assertEquals("hmkdasvfl", response.location()); + Assertions.assertEquals("xcudchxgsr", response.tags().get("oldforobw")); + Assertions.assertEquals(1671689667, response.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(1202304870, response.hourlySchedule().minute()); + Assertions.assertEquals(1705297672247999328L, response.hourlySchedule().usedBytes()); + Assertions.assertEquals(2073795724, response.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(1291910908, response.dailySchedule().hour()); + Assertions.assertEquals(264997502, response.dailySchedule().minute()); + Assertions.assertEquals(7353815266805175635L, response.dailySchedule().usedBytes()); + Assertions.assertEquals(1178556723, response.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("bdkelvidizo", response.weeklySchedule().day()); + Assertions.assertEquals(344121425, response.weeklySchedule().hour()); + Assertions.assertEquals(1181700002, response.weeklySchedule().minute()); + Assertions.assertEquals(6191416292355922762L, response.weeklySchedule().usedBytes()); + Assertions.assertEquals(1331407717, response.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("dgnwncypuuwwlt", response.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(740306752, response.monthlySchedule().hour()); + Assertions.assertEquals(1798680046, response.monthlySchedule().minute()); + Assertions.assertEquals(5288027656858110726L, response.monthlySchedule().usedBytes()); + Assertions.assertEquals(true, response.enabled()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteMockTests.java index 36d7fe2c6d176..6250fc357c588 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.snapshotPolicies().delete("zvaylptrsqqw", "tcmwqkchc", "waxfewzjkj", com.azure.core.util.Context.NONE); + manager.snapshotPolicies().delete("jpdn", "zhajoylhjlmuo", "xprimrsop", com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetWithResponseMockTests.java index cd7fb86888d7b..7f4c59a96276c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"etag\":\"qihebw\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":730785712,\"minute\":1832502698,\"usedBytes\":3351326267748259519},\"dailySchedule\":{\"snapshotsToKeep\":1527664829,\"hour\":770460161,\"minute\":307322349,\"usedBytes\":8212495412619748670},\"weeklySchedule\":{\"snapshotsToKeep\":87980458,\"day\":\"lisdjubggbq\",\"hour\":1820494144,\"minute\":2003917859,\"usedBytes\":5918202422727737393},\"monthlySchedule\":{\"snapshotsToKeep\":1063611109,\"daysOfMonth\":\"gacyrcmjdmspo\",\"hour\":361382788,\"minute\":283706298,\"usedBytes\":2234682115333576707},\"enabled\":false,\"provisioningState\":\"ofrzgb\"},\"location\":\"edm\",\"tags\":{\"nsnvpd\":\"vnlvxbcuiiznktwf\",\"z\":\"bmikost\",\"fy\":\"kiwbuqnyoph\",\"elfwy\":\"sgcrpfbcunezzce\"},\"id\":\"wl\",\"name\":\"xjwet\",\"type\":\"psihcla\"}"; + = "{\"etag\":\"edxihchrphkmcrj\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":589779398,\"minute\":531525679,\"usedBytes\":5888369763357785786},\"dailySchedule\":{\"snapshotsToKeep\":95470507,\"hour\":259124213,\"minute\":1290363462,\"usedBytes\":217902170113592881},\"weeklySchedule\":{\"snapshotsToKeep\":810731058,\"day\":\"utlwxezwzhok\",\"hour\":1775135679,\"minute\":278626218,\"usedBytes\":1768563471315587827},\"monthlySchedule\":{\"snapshotsToKeep\":1214606185,\"daysOfMonth\":\"gppipifh\",\"hour\":248320498,\"minute\":1885635288,\"usedBytes\":6741191716385680342},\"enabled\":false,\"provisioningState\":\"xjcsheafidltu\"},\"location\":\"resmkssjhoiftxfk\",\"tags\":{\"ptil\":\"gpr\",\"qtgdqohmcwsl\":\"ucb\",\"tpwb\":\"riz\"},\"id\":\"a\",\"name\":\"llibphbqzmizak\",\"type\":\"kan\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,27 +49,27 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SnapshotPolicy response = manager.snapshotPolicies() - .getWithResponse("wwinhehf", "pofvwb", "blembnkbwvqvxkd", com.azure.core.util.Context.NONE).getValue(); + .getWithResponse("nwm", "tmvpdvjdhtt", "a", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("edm", response.location()); - Assertions.assertEquals("vnlvxbcuiiznktwf", response.tags().get("nsnvpd")); - Assertions.assertEquals(730785712, response.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1832502698, response.hourlySchedule().minute()); - Assertions.assertEquals(3351326267748259519L, response.hourlySchedule().usedBytes()); - Assertions.assertEquals(1527664829, response.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(770460161, response.dailySchedule().hour()); - Assertions.assertEquals(307322349, response.dailySchedule().minute()); - Assertions.assertEquals(8212495412619748670L, response.dailySchedule().usedBytes()); - Assertions.assertEquals(87980458, response.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("lisdjubggbq", response.weeklySchedule().day()); - Assertions.assertEquals(1820494144, response.weeklySchedule().hour()); - Assertions.assertEquals(2003917859, response.weeklySchedule().minute()); - Assertions.assertEquals(5918202422727737393L, response.weeklySchedule().usedBytes()); - Assertions.assertEquals(1063611109, response.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("gacyrcmjdmspo", response.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(361382788, response.monthlySchedule().hour()); - Assertions.assertEquals(283706298, response.monthlySchedule().minute()); - Assertions.assertEquals(2234682115333576707L, response.monthlySchedule().usedBytes()); + Assertions.assertEquals("resmkssjhoiftxfk", response.location()); + Assertions.assertEquals("gpr", response.tags().get("ptil")); + Assertions.assertEquals(589779398, response.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(531525679, response.hourlySchedule().minute()); + Assertions.assertEquals(5888369763357785786L, response.hourlySchedule().usedBytes()); + Assertions.assertEquals(95470507, response.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(259124213, response.dailySchedule().hour()); + Assertions.assertEquals(1290363462, response.dailySchedule().minute()); + Assertions.assertEquals(217902170113592881L, response.dailySchedule().usedBytes()); + Assertions.assertEquals(810731058, response.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("utlwxezwzhok", response.weeklySchedule().day()); + Assertions.assertEquals(1775135679, response.weeklySchedule().hour()); + Assertions.assertEquals(278626218, response.weeklySchedule().minute()); + Assertions.assertEquals(1768563471315587827L, response.weeklySchedule().usedBytes()); + Assertions.assertEquals(1214606185, response.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("gppipifh", response.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(248320498, response.monthlySchedule().hour()); + Assertions.assertEquals(1885635288, response.monthlySchedule().minute()); + Assertions.assertEquals(6741191716385680342L, response.monthlySchedule().usedBytes()); Assertions.assertEquals(false, response.enabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListMockTests.java index 4804fc312f19e..241275148a1fb 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"etag\":\"ltyjedexxmlfmk\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1445110632,\"minute\":1734867159,\"usedBytes\":3494695317807035352},\"dailySchedule\":{\"snapshotsToKeep\":16143450,\"hour\":810465454,\"minute\":400870213,\"usedBytes\":5532317119730945052},\"weeklySchedule\":{\"snapshotsToKeep\":1757124194,\"day\":\"xcushs\",\"hour\":1333151238,\"minute\":107973649,\"usedBytes\":4154943067306073010},\"monthlySchedule\":{\"snapshotsToKeep\":666736445,\"daysOfMonth\":\"vgsgzwywakoihkn\",\"hour\":874789657,\"minute\":1233115603,\"usedBytes\":2710381116141094215},\"enabled\":true,\"provisioningState\":\"ymzotqyryuzcbmq\"},\"location\":\"xmvwfg\",\"tags\":{\"supe\":\"xo\",\"sqltnzo\":\"jlzqnhc\"},\"id\":\"bgsxgnxfyqonmpq\",\"name\":\"xwdofdb\",\"type\":\"iqxeiiqbimht\"}]}"; + = "{\"value\":[{\"etag\":\"prtujwsawdd\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1148238938,\"minute\":149022358,\"usedBytes\":6958305132617717680},\"dailySchedule\":{\"snapshotsToKeep\":177247655,\"hour\":1297733750,\"minute\":1671083084,\"usedBytes\":8033750853148480276},\"weeklySchedule\":{\"snapshotsToKeep\":1119410287,\"day\":\"glecdmdqbwpypq\",\"hour\":543854908,\"minute\":1969231562,\"usedBytes\":7692078568165207428},\"monthlySchedule\":{\"snapshotsToKeep\":1318047091,\"daysOfMonth\":\"xudb\",\"hour\":1229147679,\"minute\":1337444270,\"usedBytes\":5506706101384890254},\"enabled\":true,\"provisioningState\":\"dhzmmesckdlp\"},\"location\":\"zrcxfailcfxwmdbo\",\"tags\":{\"ckknhxkizvy\":\"gsftufqobrjlnacg\",\"nok\":\"nrzvuljraaer\",\"a\":\"gukkjqnvbroy\"},\"id\":\"xulcdisdos\",\"name\":\"jbjsvgjrwh\",\"type\":\"yvycytdclxgcckn\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,27 +50,27 @@ public void testList() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response - = manager.snapshotPolicies().list("lkafhonqjuje", "ckpzvcpopmxeln", com.azure.core.util.Context.NONE); + = manager.snapshotPolicies().list("vecuijpx", "xs", com.azure.core.util.Context.NONE); - Assertions.assertEquals("xmvwfg", response.iterator().next().location()); - Assertions.assertEquals("xo", response.iterator().next().tags().get("supe")); - Assertions.assertEquals(1445110632, response.iterator().next().hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1734867159, response.iterator().next().hourlySchedule().minute()); - Assertions.assertEquals(3494695317807035352L, response.iterator().next().hourlySchedule().usedBytes()); - Assertions.assertEquals(16143450, response.iterator().next().dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(810465454, response.iterator().next().dailySchedule().hour()); - Assertions.assertEquals(400870213, response.iterator().next().dailySchedule().minute()); - Assertions.assertEquals(5532317119730945052L, response.iterator().next().dailySchedule().usedBytes()); - Assertions.assertEquals(1757124194, response.iterator().next().weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("xcushs", response.iterator().next().weeklySchedule().day()); - Assertions.assertEquals(1333151238, response.iterator().next().weeklySchedule().hour()); - Assertions.assertEquals(107973649, response.iterator().next().weeklySchedule().minute()); - Assertions.assertEquals(4154943067306073010L, response.iterator().next().weeklySchedule().usedBytes()); - Assertions.assertEquals(666736445, response.iterator().next().monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("vgsgzwywakoihkn", response.iterator().next().monthlySchedule().daysOfMonth()); - Assertions.assertEquals(874789657, response.iterator().next().monthlySchedule().hour()); - Assertions.assertEquals(1233115603, response.iterator().next().monthlySchedule().minute()); - Assertions.assertEquals(2710381116141094215L, response.iterator().next().monthlySchedule().usedBytes()); + Assertions.assertEquals("zrcxfailcfxwmdbo", response.iterator().next().location()); + Assertions.assertEquals("gsftufqobrjlnacg", response.iterator().next().tags().get("ckknhxkizvy")); + Assertions.assertEquals(1148238938, response.iterator().next().hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(149022358, response.iterator().next().hourlySchedule().minute()); + Assertions.assertEquals(6958305132617717680L, response.iterator().next().hourlySchedule().usedBytes()); + Assertions.assertEquals(177247655, response.iterator().next().dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(1297733750, response.iterator().next().dailySchedule().hour()); + Assertions.assertEquals(1671083084, response.iterator().next().dailySchedule().minute()); + Assertions.assertEquals(8033750853148480276L, response.iterator().next().dailySchedule().usedBytes()); + Assertions.assertEquals(1119410287, response.iterator().next().weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("glecdmdqbwpypq", response.iterator().next().weeklySchedule().day()); + Assertions.assertEquals(543854908, response.iterator().next().weeklySchedule().hour()); + Assertions.assertEquals(1969231562, response.iterator().next().weeklySchedule().minute()); + Assertions.assertEquals(7692078568165207428L, response.iterator().next().weeklySchedule().usedBytes()); + Assertions.assertEquals(1318047091, response.iterator().next().monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("xudb", response.iterator().next().monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1229147679, response.iterator().next().monthlySchedule().hour()); + Assertions.assertEquals(1337444270, response.iterator().next().monthlySchedule().minute()); + Assertions.assertEquals(5506706101384890254L, response.iterator().next().monthlySchedule().usedBytes()); Assertions.assertEquals(true, response.iterator().next().enabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListTests.java index 3276353bb7154..270e828dfb2ab 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPoliciesListTests.java @@ -20,63 +20,100 @@ public final class SnapshotPoliciesListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotPoliciesList model = BinaryData.fromString( - "{\"value\":[{\"etag\":\"kghimdblxgwimfnj\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":510557599,\"minute\":1675220188,\"usedBytes\":1198847986383751038},\"dailySchedule\":{\"snapshotsToKeep\":1767053067,\"hour\":739435613,\"minute\":1563203885,\"usedBytes\":8338807266312513089},\"weeklySchedule\":{\"snapshotsToKeep\":1648570452,\"day\":\"aw\",\"hour\":113630721,\"minute\":841259449,\"usedBytes\":9051460600298746000},\"monthlySchedule\":{\"snapshotsToKeep\":220414413,\"daysOfMonth\":\"c\",\"hour\":1974208739,\"minute\":833171610,\"usedBytes\":2411008459227076557},\"enabled\":true,\"provisioningState\":\"abfatkl\"},\"location\":\"xbjhwuaanozjosph\",\"tags\":{\"jrvxaglrv\":\"l\",\"tcs\":\"mjwosytx\"},\"id\":\"fcktqumiekke\",\"name\":\"zikhl\",\"type\":\"fjhdg\"}]}") + "{\"value\":[{\"etag\":\"qdcvdrhvoo\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1193557953,\"minute\":1582782062,\"usedBytes\":2322318888299695987},\"dailySchedule\":{\"snapshotsToKeep\":1146790818,\"hour\":542490098,\"minute\":546984067,\"usedBytes\":8637599636999601474},\"weeklySchedule\":{\"snapshotsToKeep\":363113472,\"day\":\"xcxrsl\",\"hour\":1073132708,\"minute\":2107317095,\"usedBytes\":1317249364545510720},\"monthlySchedule\":{\"snapshotsToKeep\":1828084006,\"daysOfMonth\":\"hjwniyqsluic\",\"hour\":1043474675,\"minute\":1810528552,\"usedBytes\":2246531138644222618},\"enabled\":true,\"provisioningState\":\"mpaxmodfvuefywsb\"},\"location\":\"vmwy\",\"tags\":{\"wiyzvqtmnubexkp\":\"ouyftaakc\",\"mquxvypo\":\"ksmond\"},\"id\":\"gkopkwhojvpajqgx\",\"name\":\"smocmbq\",\"type\":\"qvmkcxo\"},{\"etag\":\"pvhelxprg\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1689859600,\"minute\":1462356590,\"usedBytes\":6814865152361254049},\"dailySchedule\":{\"snapshotsToKeep\":2028979165,\"hour\":1827810592,\"minute\":465544934,\"usedBytes\":8371977725942992913},\"weeklySchedule\":{\"snapshotsToKeep\":785036468,\"day\":\"rhos\",\"hour\":341443712,\"minute\":992326017,\"usedBytes\":7744101608598421858},\"monthlySchedule\":{\"snapshotsToKeep\":5318116,\"daysOfMonth\":\"m\",\"hour\":812505700,\"minute\":1391684752,\"usedBytes\":5425852562349977071},\"enabled\":true,\"provisioningState\":\"szdtmhrkwof\"},\"location\":\"voqacpiexpbt\",\"tags\":{\"tkcnqxwb\":\"bwoenwashrt\"},\"id\":\"okulpiujwa\",\"name\":\"sipqii\",\"type\":\"byuqerpqlp\"},{\"etag\":\"cciuqgbdbutau\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1305719459,\"minute\":1787121852,\"usedBytes\":3230556425991226007},\"dailySchedule\":{\"snapshotsToKeep\":1745648414,\"hour\":537605324,\"minute\":1183832331,\"usedBytes\":3465382466938026517},\"weeklySchedule\":{\"snapshotsToKeep\":2073837951,\"day\":\"pichkoymkcdy\",\"hour\":1322696494,\"minute\":556339960,\"usedBytes\":6135641576986595049},\"monthlySchedule\":{\"snapshotsToKeep\":88319304,\"daysOfMonth\":\"ovvqfovljxywsu\",\"hour\":678747249,\"minute\":188945883,\"usedBytes\":1595532838454650942},\"enabled\":true,\"provisioningState\":\"adgvraeaene\"},\"location\":\"zar\",\"tags\":{\"uijfqk\":\"q\",\"ibwwiftohqkv\":\"cewiipfpub\",\"ljphuopxodl\":\"uvksgplsaknynfsy\",\"sjswsrms\":\"iyntorzihle\"},\"id\":\"yzrpzbchckqqzq\",\"name\":\"ox\",\"type\":\"ysuiizynkedya\"},{\"etag\":\"wyhqmibzyhwits\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1679951474,\"minute\":740699984,\"usedBytes\":4085302065865673737},\"dailySchedule\":{\"snapshotsToKeep\":846796617,\"hour\":1107344942,\"minute\":998287381,\"usedBytes\":359022541227333751},\"weeklySchedule\":{\"snapshotsToKeep\":1572322989,\"day\":\"knso\",\"hour\":1498450316,\"minute\":1807799282,\"usedBytes\":7538220819901034139},\"monthlySchedule\":{\"snapshotsToKeep\":706339919,\"daysOfMonth\":\"rlkdmtncvokotl\",\"hour\":44844899,\"minute\":11976377,\"usedBytes\":8627603791684452412},\"enabled\":true,\"provisioningState\":\"gjltdtbnnhado\"},\"location\":\"rkvcikhnvpa\",\"tags\":{\"queziky\":\"x\",\"ccjzkzivgvv\":\"ggxkallatmelwuip\",\"rdvstkwqqtch\":\"nayrhyrnxxmueedn\"},\"id\":\"alm\",\"name\":\"mtdaa\",\"type\":\"gdv\"}]}") .toObject(SnapshotPoliciesList.class); - Assertions.assertEquals("xbjhwuaanozjosph", model.value().get(0).location()); - Assertions.assertEquals("l", model.value().get(0).tags().get("jrvxaglrv")); - Assertions.assertEquals(510557599, model.value().get(0).hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1675220188, model.value().get(0).hourlySchedule().minute()); - Assertions.assertEquals(1198847986383751038L, model.value().get(0).hourlySchedule().usedBytes()); - Assertions.assertEquals(1767053067, model.value().get(0).dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(739435613, model.value().get(0).dailySchedule().hour()); - Assertions.assertEquals(1563203885, model.value().get(0).dailySchedule().minute()); - Assertions.assertEquals(8338807266312513089L, model.value().get(0).dailySchedule().usedBytes()); - Assertions.assertEquals(1648570452, model.value().get(0).weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("aw", model.value().get(0).weeklySchedule().day()); - Assertions.assertEquals(113630721, model.value().get(0).weeklySchedule().hour()); - Assertions.assertEquals(841259449, model.value().get(0).weeklySchedule().minute()); - Assertions.assertEquals(9051460600298746000L, model.value().get(0).weeklySchedule().usedBytes()); - Assertions.assertEquals(220414413, model.value().get(0).monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("c", model.value().get(0).monthlySchedule().daysOfMonth()); - Assertions.assertEquals(1974208739, model.value().get(0).monthlySchedule().hour()); - Assertions.assertEquals(833171610, model.value().get(0).monthlySchedule().minute()); - Assertions.assertEquals(2411008459227076557L, model.value().get(0).monthlySchedule().usedBytes()); + Assertions.assertEquals("vmwy", model.value().get(0).location()); + Assertions.assertEquals("ouyftaakc", model.value().get(0).tags().get("wiyzvqtmnubexkp")); + Assertions.assertEquals(1193557953, model.value().get(0).hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(1582782062, model.value().get(0).hourlySchedule().minute()); + Assertions.assertEquals(2322318888299695987L, model.value().get(0).hourlySchedule().usedBytes()); + Assertions.assertEquals(1146790818, model.value().get(0).dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(542490098, model.value().get(0).dailySchedule().hour()); + Assertions.assertEquals(546984067, model.value().get(0).dailySchedule().minute()); + Assertions.assertEquals(8637599636999601474L, model.value().get(0).dailySchedule().usedBytes()); + Assertions.assertEquals(363113472, model.value().get(0).weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("xcxrsl", model.value().get(0).weeklySchedule().day()); + Assertions.assertEquals(1073132708, model.value().get(0).weeklySchedule().hour()); + Assertions.assertEquals(2107317095, model.value().get(0).weeklySchedule().minute()); + Assertions.assertEquals(1317249364545510720L, model.value().get(0).weeklySchedule().usedBytes()); + Assertions.assertEquals(1828084006, model.value().get(0).monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("hjwniyqsluic", model.value().get(0).monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1043474675, model.value().get(0).monthlySchedule().hour()); + Assertions.assertEquals(1810528552, model.value().get(0).monthlySchedule().minute()); + Assertions.assertEquals(2246531138644222618L, model.value().get(0).monthlySchedule().usedBytes()); Assertions.assertEquals(true, model.value().get(0).enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnapshotPoliciesList model = new SnapshotPoliciesList().withValue(Arrays.asList(new SnapshotPolicyInner() - .withLocation("xbjhwuaanozjosph").withTags(mapOf("jrvxaglrv", "l", "tcs", "mjwosytx")) - .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(510557599).withMinute(1675220188) - .withUsedBytes(1198847986383751038L)) - .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1767053067).withHour(739435613) - .withMinute(1563203885).withUsedBytes(8338807266312513089L)) - .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1648570452).withDay("aw").withHour(113630721) - .withMinute(841259449).withUsedBytes(9051460600298746000L)) - .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(220414413).withDaysOfMonth("c") - .withHour(1974208739).withMinute(833171610).withUsedBytes(2411008459227076557L)) - .withEnabled(true))); + SnapshotPoliciesList model = new SnapshotPoliciesList().withValue(Arrays.asList( + new SnapshotPolicyInner().withLocation("vmwy") + .withTags(mapOf("wiyzvqtmnubexkp", "ouyftaakc", "mquxvypo", "ksmond")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1193557953).withMinute(1582782062) + .withUsedBytes(2322318888299695987L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1146790818).withHour(542490098) + .withMinute(546984067).withUsedBytes(8637599636999601474L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(363113472).withDay("xcxrsl") + .withHour(1073132708).withMinute(2107317095).withUsedBytes(1317249364545510720L)) + .withMonthlySchedule( + new MonthlySchedule().withSnapshotsToKeep(1828084006).withDaysOfMonth("hjwniyqsluic") + .withHour(1043474675).withMinute(1810528552).withUsedBytes(2246531138644222618L)) + .withEnabled(true), + new SnapshotPolicyInner().withLocation("voqacpiexpbt").withTags(mapOf("tkcnqxwb", "bwoenwashrt")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1689859600).withMinute(1462356590) + .withUsedBytes(6814865152361254049L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(2028979165).withHour(1827810592) + .withMinute(465544934).withUsedBytes(8371977725942992913L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(785036468).withDay("rhos") + .withHour(341443712).withMinute(992326017).withUsedBytes(7744101608598421858L)) + .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(5318116).withDaysOfMonth("m") + .withHour(812505700).withMinute(1391684752).withUsedBytes(5425852562349977071L)) + .withEnabled(true), + new SnapshotPolicyInner().withLocation("zar") + .withTags(mapOf("uijfqk", "q", "ibwwiftohqkv", "cewiipfpub", "ljphuopxodl", "uvksgplsaknynfsy", + "sjswsrms", "iyntorzihle")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1305719459).withMinute(1787121852) + .withUsedBytes(3230556425991226007L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1745648414).withHour(537605324) + .withMinute(1183832331).withUsedBytes(3465382466938026517L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(2073837951).withDay("pichkoymkcdy") + .withHour(1322696494).withMinute(556339960).withUsedBytes(6135641576986595049L)) + .withMonthlySchedule( + new MonthlySchedule().withSnapshotsToKeep(88319304).withDaysOfMonth("ovvqfovljxywsu") + .withHour(678747249).withMinute(188945883).withUsedBytes(1595532838454650942L)) + .withEnabled(true), + new SnapshotPolicyInner().withLocation("rkvcikhnvpa") + .withTags(mapOf("queziky", "x", "ccjzkzivgvv", "ggxkallatmelwuip", "rdvstkwqqtch", "nayrhyrnxxmueedn")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1679951474).withMinute(740699984) + .withUsedBytes(4085302065865673737L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(846796617).withHour(1107344942) + .withMinute(998287381).withUsedBytes(359022541227333751L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1572322989).withDay("knso") + .withHour(1498450316).withMinute(1807799282).withUsedBytes(7538220819901034139L)) + .withMonthlySchedule( + new MonthlySchedule().withSnapshotsToKeep(706339919).withDaysOfMonth("rlkdmtncvokotl") + .withHour(44844899).withMinute(11976377).withUsedBytes(8627603791684452412L)) + .withEnabled(true))); model = BinaryData.fromObject(model).toObject(SnapshotPoliciesList.class); - Assertions.assertEquals("xbjhwuaanozjosph", model.value().get(0).location()); - Assertions.assertEquals("l", model.value().get(0).tags().get("jrvxaglrv")); - Assertions.assertEquals(510557599, model.value().get(0).hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1675220188, model.value().get(0).hourlySchedule().minute()); - Assertions.assertEquals(1198847986383751038L, model.value().get(0).hourlySchedule().usedBytes()); - Assertions.assertEquals(1767053067, model.value().get(0).dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(739435613, model.value().get(0).dailySchedule().hour()); - Assertions.assertEquals(1563203885, model.value().get(0).dailySchedule().minute()); - Assertions.assertEquals(8338807266312513089L, model.value().get(0).dailySchedule().usedBytes()); - Assertions.assertEquals(1648570452, model.value().get(0).weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("aw", model.value().get(0).weeklySchedule().day()); - Assertions.assertEquals(113630721, model.value().get(0).weeklySchedule().hour()); - Assertions.assertEquals(841259449, model.value().get(0).weeklySchedule().minute()); - Assertions.assertEquals(9051460600298746000L, model.value().get(0).weeklySchedule().usedBytes()); - Assertions.assertEquals(220414413, model.value().get(0).monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("c", model.value().get(0).monthlySchedule().daysOfMonth()); - Assertions.assertEquals(1974208739, model.value().get(0).monthlySchedule().hour()); - Assertions.assertEquals(833171610, model.value().get(0).monthlySchedule().minute()); - Assertions.assertEquals(2411008459227076557L, model.value().get(0).monthlySchedule().usedBytes()); + Assertions.assertEquals("vmwy", model.value().get(0).location()); + Assertions.assertEquals("ouyftaakc", model.value().get(0).tags().get("wiyzvqtmnubexkp")); + Assertions.assertEquals(1193557953, model.value().get(0).hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(1582782062, model.value().get(0).hourlySchedule().minute()); + Assertions.assertEquals(2322318888299695987L, model.value().get(0).hourlySchedule().usedBytes()); + Assertions.assertEquals(1146790818, model.value().get(0).dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(542490098, model.value().get(0).dailySchedule().hour()); + Assertions.assertEquals(546984067, model.value().get(0).dailySchedule().minute()); + Assertions.assertEquals(8637599636999601474L, model.value().get(0).dailySchedule().usedBytes()); + Assertions.assertEquals(363113472, model.value().get(0).weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("xcxrsl", model.value().get(0).weeklySchedule().day()); + Assertions.assertEquals(1073132708, model.value().get(0).weeklySchedule().hour()); + Assertions.assertEquals(2107317095, model.value().get(0).weeklySchedule().minute()); + Assertions.assertEquals(1317249364545510720L, model.value().get(0).weeklySchedule().usedBytes()); + Assertions.assertEquals(1828084006, model.value().get(0).monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("hjwniyqsluic", model.value().get(0).monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1043474675, model.value().get(0).monthlySchedule().hour()); + Assertions.assertEquals(1810528552, model.value().get(0).monthlySchedule().minute()); + Assertions.assertEquals(2246531138644222618L, model.value().get(0).monthlySchedule().usedBytes()); Assertions.assertEquals(true, model.value().get(0).enabled()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyInnerTests.java index 31379b3ef7955..7293f7f449e89 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyInnerTests.java @@ -18,63 +18,63 @@ public final class SnapshotPolicyInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotPolicyInner model = BinaryData.fromString( - "{\"etag\":\"gebdunygaeq\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1475802306,\"minute\":1233334035,\"usedBytes\":6620237582798297782},\"dailySchedule\":{\"snapshotsToKeep\":48002498,\"hour\":1459715111,\"minute\":2063960908,\"usedBytes\":8725801999952390480},\"weeklySchedule\":{\"snapshotsToKeep\":1120671201,\"day\":\"arm\",\"hour\":1679294493,\"minute\":522176461,\"usedBytes\":7091421984194372162},\"monthlySchedule\":{\"snapshotsToKeep\":1756269391,\"daysOfMonth\":\"yxxrwlycoduh\",\"hour\":1842821288,\"minute\":1440361385,\"usedBytes\":4645704067973725569},\"enabled\":true,\"provisioningState\":\"n\"},\"location\":\"xqugjhkycubedd\",\"tags\":{\"zqalkrmnjijpx\":\"ofwq\",\"byxbaaabjy\":\"cqqudf\",\"zrtuzq\":\"ayffim\",\"fdnw\":\"gsexne\"},\"id\":\"wmewzsyy\",\"name\":\"euzsoi\",\"type\":\"judpfrxt\"}") + "{\"etag\":\"gpiohgwxrtfudxe\",\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1902888428,\"minute\":682770513,\"usedBytes\":5596269908877645907},\"dailySchedule\":{\"snapshotsToKeep\":1104557585,\"hour\":128514806,\"minute\":1947559864,\"usedBytes\":6566741920348285642},\"weeklySchedule\":{\"snapshotsToKeep\":2068705398,\"day\":\"xgwim\",\"hour\":1697714955,\"minute\":1799565782,\"usedBytes\":2192828191882361978},\"monthlySchedule\":{\"snapshotsToKeep\":279128548,\"daysOfMonth\":\"kfoqreyfkzikfj\",\"hour\":195683064,\"minute\":113630721,\"usedBytes\":3613181819399504746},\"enabled\":true,\"provisioningState\":\"elpcirelsfeaenwa\"},\"location\":\"atklddxbjhwuaa\",\"tags\":{\"hyoulpjr\":\"jos\",\"vimjwos\":\"xagl\"},\"id\":\"tx\",\"name\":\"tcs\",\"type\":\"fcktqumiekke\"}") .toObject(SnapshotPolicyInner.class); - Assertions.assertEquals("xqugjhkycubedd", model.location()); - Assertions.assertEquals("ofwq", model.tags().get("zqalkrmnjijpx")); - Assertions.assertEquals(1475802306, model.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1233334035, model.hourlySchedule().minute()); - Assertions.assertEquals(6620237582798297782L, model.hourlySchedule().usedBytes()); - Assertions.assertEquals(48002498, model.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1459715111, model.dailySchedule().hour()); - Assertions.assertEquals(2063960908, model.dailySchedule().minute()); - Assertions.assertEquals(8725801999952390480L, model.dailySchedule().usedBytes()); - Assertions.assertEquals(1120671201, model.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("arm", model.weeklySchedule().day()); - Assertions.assertEquals(1679294493, model.weeklySchedule().hour()); - Assertions.assertEquals(522176461, model.weeklySchedule().minute()); - Assertions.assertEquals(7091421984194372162L, model.weeklySchedule().usedBytes()); - Assertions.assertEquals(1756269391, model.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("yxxrwlycoduh", model.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(1842821288, model.monthlySchedule().hour()); - Assertions.assertEquals(1440361385, model.monthlySchedule().minute()); - Assertions.assertEquals(4645704067973725569L, model.monthlySchedule().usedBytes()); + Assertions.assertEquals("atklddxbjhwuaa", model.location()); + Assertions.assertEquals("jos", model.tags().get("hyoulpjr")); + Assertions.assertEquals(1902888428, model.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(682770513, model.hourlySchedule().minute()); + Assertions.assertEquals(5596269908877645907L, model.hourlySchedule().usedBytes()); + Assertions.assertEquals(1104557585, model.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(128514806, model.dailySchedule().hour()); + Assertions.assertEquals(1947559864, model.dailySchedule().minute()); + Assertions.assertEquals(6566741920348285642L, model.dailySchedule().usedBytes()); + Assertions.assertEquals(2068705398, model.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("xgwim", model.weeklySchedule().day()); + Assertions.assertEquals(1697714955, model.weeklySchedule().hour()); + Assertions.assertEquals(1799565782, model.weeklySchedule().minute()); + Assertions.assertEquals(2192828191882361978L, model.weeklySchedule().usedBytes()); + Assertions.assertEquals(279128548, model.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("kfoqreyfkzikfj", model.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(195683064, model.monthlySchedule().hour()); + Assertions.assertEquals(113630721, model.monthlySchedule().minute()); + Assertions.assertEquals(3613181819399504746L, model.monthlySchedule().usedBytes()); Assertions.assertEquals(true, model.enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnapshotPolicyInner model = new SnapshotPolicyInner().withLocation("xqugjhkycubedd") - .withTags(mapOf("zqalkrmnjijpx", "ofwq", "byxbaaabjy", "cqqudf", "zrtuzq", "ayffim", "fdnw", "gsexne")) - .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1475802306).withMinute(1233334035) - .withUsedBytes(6620237582798297782L)) - .withDailySchedule(new DailySchedule().withSnapshotsToKeep(48002498).withHour(1459715111) - .withMinute(2063960908).withUsedBytes(8725801999952390480L)) - .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1120671201).withDay("arm").withHour(1679294493) - .withMinute(522176461).withUsedBytes(7091421984194372162L)) - .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(1756269391).withDaysOfMonth("yxxrwlycoduh") - .withHour(1842821288).withMinute(1440361385).withUsedBytes(4645704067973725569L)) + SnapshotPolicyInner model = new SnapshotPolicyInner().withLocation("atklddxbjhwuaa") + .withTags(mapOf("hyoulpjr", "jos", "vimjwos", "xagl")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1902888428).withMinute(682770513) + .withUsedBytes(5596269908877645907L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1104557585).withHour(128514806) + .withMinute(1947559864).withUsedBytes(6566741920348285642L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(2068705398).withDay("xgwim") + .withHour(1697714955).withMinute(1799565782).withUsedBytes(2192828191882361978L)) + .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(279128548).withDaysOfMonth("kfoqreyfkzikfj") + .withHour(195683064).withMinute(113630721).withUsedBytes(3613181819399504746L)) .withEnabled(true); model = BinaryData.fromObject(model).toObject(SnapshotPolicyInner.class); - Assertions.assertEquals("xqugjhkycubedd", model.location()); - Assertions.assertEquals("ofwq", model.tags().get("zqalkrmnjijpx")); - Assertions.assertEquals(1475802306, model.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1233334035, model.hourlySchedule().minute()); - Assertions.assertEquals(6620237582798297782L, model.hourlySchedule().usedBytes()); - Assertions.assertEquals(48002498, model.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1459715111, model.dailySchedule().hour()); - Assertions.assertEquals(2063960908, model.dailySchedule().minute()); - Assertions.assertEquals(8725801999952390480L, model.dailySchedule().usedBytes()); - Assertions.assertEquals(1120671201, model.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("arm", model.weeklySchedule().day()); - Assertions.assertEquals(1679294493, model.weeklySchedule().hour()); - Assertions.assertEquals(522176461, model.weeklySchedule().minute()); - Assertions.assertEquals(7091421984194372162L, model.weeklySchedule().usedBytes()); - Assertions.assertEquals(1756269391, model.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("yxxrwlycoduh", model.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(1842821288, model.monthlySchedule().hour()); - Assertions.assertEquals(1440361385, model.monthlySchedule().minute()); - Assertions.assertEquals(4645704067973725569L, model.monthlySchedule().usedBytes()); + Assertions.assertEquals("atklddxbjhwuaa", model.location()); + Assertions.assertEquals("jos", model.tags().get("hyoulpjr")); + Assertions.assertEquals(1902888428, model.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(682770513, model.hourlySchedule().minute()); + Assertions.assertEquals(5596269908877645907L, model.hourlySchedule().usedBytes()); + Assertions.assertEquals(1104557585, model.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(128514806, model.dailySchedule().hour()); + Assertions.assertEquals(1947559864, model.dailySchedule().minute()); + Assertions.assertEquals(6566741920348285642L, model.dailySchedule().usedBytes()); + Assertions.assertEquals(2068705398, model.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("xgwim", model.weeklySchedule().day()); + Assertions.assertEquals(1697714955, model.weeklySchedule().hour()); + Assertions.assertEquals(1799565782, model.weeklySchedule().minute()); + Assertions.assertEquals(2192828191882361978L, model.weeklySchedule().usedBytes()); + Assertions.assertEquals(279128548, model.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("kfoqreyfkzikfj", model.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(195683064, model.monthlySchedule().hour()); + Assertions.assertEquals(113630721, model.monthlySchedule().minute()); + Assertions.assertEquals(3613181819399504746L, model.monthlySchedule().usedBytes()); Assertions.assertEquals(true, model.enabled()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPatchTests.java index 663595a4d3fd4..6593c1948ac9c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPatchTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPatchTests.java @@ -18,64 +18,64 @@ public final class SnapshotPolicyPatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotPolicyPatch model = BinaryData.fromString( - "{\"location\":\"gaowpulpqblylsyx\",\"id\":\"jnsjervtiagxsd\",\"name\":\"uem\",\"type\":\"bzkfzbeyvpn\",\"tags\":{\"xdxr\":\"vinvkj\",\"aztz\":\"uukzclewyhmlw\",\"yq\":\"ofncckwyfzqwhxxb\",\"ztppriolxorjalto\":\"xzfe\"},\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":618891293,\"minute\":1429171965,\"usedBytes\":6564772827394590293},\"dailySchedule\":{\"snapshotsToKeep\":1051180868,\"hour\":1806206586,\"minute\":358085488,\"usedBytes\":1296933680465064139},\"weeklySchedule\":{\"snapshotsToKeep\":1589501712,\"day\":\"pfuvglsbjjca\",\"hour\":1995846414,\"minute\":1075232734,\"usedBytes\":652867049720214090},\"monthlySchedule\":{\"snapshotsToKeep\":1601872964,\"daysOfMonth\":\"cormr\",\"hour\":1360670450,\"minute\":1954508595,\"usedBytes\":2688344690599449184},\"enabled\":false,\"provisioningState\":\"lvkgju\"}}") + "{\"location\":\"bjyvay\",\"id\":\"imrzrtuzqog\",\"name\":\"xnevfdnwn\",\"type\":\"ewzsyyceuzsoib\",\"tags\":{\"dwkqbrq\":\"pfrxtrthzvay\",\"irqtdqoa\":\"bpaxhexiilivpdt\",\"eptra\":\"oruzfgsquyfxrxx\",\"wlwnwxuqlcv\":\"xje\"},\"properties\":{\"hourlySchedule\":{\"snapshotsToKeep\":1148696906,\"minute\":1248229075,\"usedBytes\":1507224315581735893},\"dailySchedule\":{\"snapshotsToKeep\":1080524607,\"hour\":1614496522,\"minute\":77495916,\"usedBytes\":1911538746456059968},\"weeklySchedule\":{\"snapshotsToKeep\":1225956555,\"day\":\"ujhemmsbvdkcrodt\",\"hour\":1997229408,\"minute\":291950475,\"usedBytes\":6410034847924566454},\"monthlySchedule\":{\"snapshotsToKeep\":896449803,\"daysOfMonth\":\"cjvefkdlfo\",\"hour\":1252976241,\"minute\":1706699033,\"usedBytes\":5309095811404897814},\"enabled\":false,\"provisioningState\":\"wpu\"}}") .toObject(SnapshotPolicyPatch.class); - Assertions.assertEquals("gaowpulpqblylsyx", model.location()); - Assertions.assertEquals("vinvkj", model.tags().get("xdxr")); - Assertions.assertEquals(618891293, model.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1429171965, model.hourlySchedule().minute()); - Assertions.assertEquals(6564772827394590293L, model.hourlySchedule().usedBytes()); - Assertions.assertEquals(1051180868, model.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1806206586, model.dailySchedule().hour()); - Assertions.assertEquals(358085488, model.dailySchedule().minute()); - Assertions.assertEquals(1296933680465064139L, model.dailySchedule().usedBytes()); - Assertions.assertEquals(1589501712, model.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("pfuvglsbjjca", model.weeklySchedule().day()); - Assertions.assertEquals(1995846414, model.weeklySchedule().hour()); - Assertions.assertEquals(1075232734, model.weeklySchedule().minute()); - Assertions.assertEquals(652867049720214090L, model.weeklySchedule().usedBytes()); - Assertions.assertEquals(1601872964, model.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("cormr", model.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(1360670450, model.monthlySchedule().hour()); - Assertions.assertEquals(1954508595, model.monthlySchedule().minute()); - Assertions.assertEquals(2688344690599449184L, model.monthlySchedule().usedBytes()); + Assertions.assertEquals("bjyvay", model.location()); + Assertions.assertEquals("pfrxtrthzvay", model.tags().get("dwkqbrq")); + Assertions.assertEquals(1148696906, model.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(1248229075, model.hourlySchedule().minute()); + Assertions.assertEquals(1507224315581735893L, model.hourlySchedule().usedBytes()); + Assertions.assertEquals(1080524607, model.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(1614496522, model.dailySchedule().hour()); + Assertions.assertEquals(77495916, model.dailySchedule().minute()); + Assertions.assertEquals(1911538746456059968L, model.dailySchedule().usedBytes()); + Assertions.assertEquals(1225956555, model.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("ujhemmsbvdkcrodt", model.weeklySchedule().day()); + Assertions.assertEquals(1997229408, model.weeklySchedule().hour()); + Assertions.assertEquals(291950475, model.weeklySchedule().minute()); + Assertions.assertEquals(6410034847924566454L, model.weeklySchedule().usedBytes()); + Assertions.assertEquals(896449803, model.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("cjvefkdlfo", model.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1252976241, model.monthlySchedule().hour()); + Assertions.assertEquals(1706699033, model.monthlySchedule().minute()); + Assertions.assertEquals(5309095811404897814L, model.monthlySchedule().usedBytes()); Assertions.assertEquals(false, model.enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnapshotPolicyPatch model = new SnapshotPolicyPatch().withLocation("gaowpulpqblylsyx") - .withTags( - mapOf("xdxr", "vinvkj", "aztz", "uukzclewyhmlw", "yq", "ofncckwyfzqwhxxb", "ztppriolxorjalto", "xzfe")) - .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(618891293).withMinute(1429171965) - .withUsedBytes(6564772827394590293L)) - .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1051180868).withHour(1806206586) - .withMinute(358085488).withUsedBytes(1296933680465064139L)) - .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1589501712).withDay("pfuvglsbjjca") - .withHour(1995846414).withMinute(1075232734).withUsedBytes(652867049720214090L)) - .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(1601872964).withDaysOfMonth("cormr") - .withHour(1360670450).withMinute(1954508595).withUsedBytes(2688344690599449184L)) + SnapshotPolicyPatch model = new SnapshotPolicyPatch().withLocation("bjyvay") + .withTags(mapOf("dwkqbrq", "pfrxtrthzvay", "irqtdqoa", "bpaxhexiilivpdt", "eptra", "oruzfgsquyfxrxx", + "wlwnwxuqlcv", "xje")) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(1148696906).withMinute(1248229075) + .withUsedBytes(1507224315581735893L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1080524607).withHour(1614496522) + .withMinute(77495916).withUsedBytes(1911538746456059968L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1225956555).withDay("ujhemmsbvdkcrodt") + .withHour(1997229408).withMinute(291950475).withUsedBytes(6410034847924566454L)) + .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(896449803).withDaysOfMonth("cjvefkdlfo") + .withHour(1252976241).withMinute(1706699033).withUsedBytes(5309095811404897814L)) .withEnabled(false); model = BinaryData.fromObject(model).toObject(SnapshotPolicyPatch.class); - Assertions.assertEquals("gaowpulpqblylsyx", model.location()); - Assertions.assertEquals("vinvkj", model.tags().get("xdxr")); - Assertions.assertEquals(618891293, model.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(1429171965, model.hourlySchedule().minute()); - Assertions.assertEquals(6564772827394590293L, model.hourlySchedule().usedBytes()); - Assertions.assertEquals(1051180868, model.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1806206586, model.dailySchedule().hour()); - Assertions.assertEquals(358085488, model.dailySchedule().minute()); - Assertions.assertEquals(1296933680465064139L, model.dailySchedule().usedBytes()); - Assertions.assertEquals(1589501712, model.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("pfuvglsbjjca", model.weeklySchedule().day()); - Assertions.assertEquals(1995846414, model.weeklySchedule().hour()); - Assertions.assertEquals(1075232734, model.weeklySchedule().minute()); - Assertions.assertEquals(652867049720214090L, model.weeklySchedule().usedBytes()); - Assertions.assertEquals(1601872964, model.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("cormr", model.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(1360670450, model.monthlySchedule().hour()); - Assertions.assertEquals(1954508595, model.monthlySchedule().minute()); - Assertions.assertEquals(2688344690599449184L, model.monthlySchedule().usedBytes()); + Assertions.assertEquals("bjyvay", model.location()); + Assertions.assertEquals("pfrxtrthzvay", model.tags().get("dwkqbrq")); + Assertions.assertEquals(1148696906, model.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(1248229075, model.hourlySchedule().minute()); + Assertions.assertEquals(1507224315581735893L, model.hourlySchedule().usedBytes()); + Assertions.assertEquals(1080524607, model.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(1614496522, model.dailySchedule().hour()); + Assertions.assertEquals(77495916, model.dailySchedule().minute()); + Assertions.assertEquals(1911538746456059968L, model.dailySchedule().usedBytes()); + Assertions.assertEquals(1225956555, model.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("ujhemmsbvdkcrodt", model.weeklySchedule().day()); + Assertions.assertEquals(1997229408, model.weeklySchedule().hour()); + Assertions.assertEquals(291950475, model.weeklySchedule().minute()); + Assertions.assertEquals(6410034847924566454L, model.weeklySchedule().usedBytes()); + Assertions.assertEquals(896449803, model.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("cjvefkdlfo", model.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1252976241, model.monthlySchedule().hour()); + Assertions.assertEquals(1706699033, model.monthlySchedule().minute()); + Assertions.assertEquals(5309095811404897814L, model.monthlySchedule().usedBytes()); Assertions.assertEquals(false, model.enabled()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPropertiesTests.java index d19ad300cea03..89d3199cca10c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPolicyPropertiesTests.java @@ -16,59 +16,58 @@ public final class SnapshotPolicyPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotPolicyProperties model = BinaryData.fromString( - "{\"hourlySchedule\":{\"snapshotsToKeep\":747190122,\"minute\":383996827,\"usedBytes\":8358866216552319675},\"dailySchedule\":{\"snapshotsToKeep\":2045472309,\"hour\":1322383691,\"minute\":1729344688,\"usedBytes\":4145471517526414946},\"weeklySchedule\":{\"snapshotsToKeep\":1389326915,\"day\":\"iilivpdtiirqtd\",\"hour\":887642500,\"minute\":1693357038,\"usedBytes\":3114947430528398041},\"monthlySchedule\":{\"snapshotsToKeep\":1061692620,\"daysOfMonth\":\"uyfxrxxleptramxj\",\"hour\":576385447,\"minute\":800812531,\"usedBytes\":9127393651345868324},\"enabled\":false,\"provisioningState\":\"cvydypatdoo\"}") + "{\"hourlySchedule\":{\"snapshotsToKeep\":885439585,\"minute\":270423154,\"usedBytes\":5614257163136424427},\"dailySchedule\":{\"snapshotsToKeep\":1081755723,\"hour\":564076169,\"minute\":1794228240,\"usedBytes\":5489756358952396230},\"weeklySchedule\":{\"snapshotsToKeep\":253317009,\"day\":\"eqidbqfatpx\",\"hour\":48002498,\"minute\":1459715111,\"usedBytes\":8864644598647393386},\"monthlySchedule\":{\"snapshotsToKeep\":2040504053,\"daysOfMonth\":\"uvarmywdmjsjq\",\"hour\":1756269391,\"minute\":1027618115,\"usedBytes\":6679760626312548131},\"enabled\":false,\"provisioningState\":\"co\"}") .toObject(SnapshotPolicyProperties.class); - Assertions.assertEquals(747190122, model.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(383996827, model.hourlySchedule().minute()); - Assertions.assertEquals(8358866216552319675L, model.hourlySchedule().usedBytes()); - Assertions.assertEquals(2045472309, model.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1322383691, model.dailySchedule().hour()); - Assertions.assertEquals(1729344688, model.dailySchedule().minute()); - Assertions.assertEquals(4145471517526414946L, model.dailySchedule().usedBytes()); - Assertions.assertEquals(1389326915, model.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("iilivpdtiirqtd", model.weeklySchedule().day()); - Assertions.assertEquals(887642500, model.weeklySchedule().hour()); - Assertions.assertEquals(1693357038, model.weeklySchedule().minute()); - Assertions.assertEquals(3114947430528398041L, model.weeklySchedule().usedBytes()); - Assertions.assertEquals(1061692620, model.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("uyfxrxxleptramxj", model.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(576385447, model.monthlySchedule().hour()); - Assertions.assertEquals(800812531, model.monthlySchedule().minute()); - Assertions.assertEquals(9127393651345868324L, model.monthlySchedule().usedBytes()); + Assertions.assertEquals(885439585, model.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(270423154, model.hourlySchedule().minute()); + Assertions.assertEquals(5614257163136424427L, model.hourlySchedule().usedBytes()); + Assertions.assertEquals(1081755723, model.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(564076169, model.dailySchedule().hour()); + Assertions.assertEquals(1794228240, model.dailySchedule().minute()); + Assertions.assertEquals(5489756358952396230L, model.dailySchedule().usedBytes()); + Assertions.assertEquals(253317009, model.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("eqidbqfatpx", model.weeklySchedule().day()); + Assertions.assertEquals(48002498, model.weeklySchedule().hour()); + Assertions.assertEquals(1459715111, model.weeklySchedule().minute()); + Assertions.assertEquals(8864644598647393386L, model.weeklySchedule().usedBytes()); + Assertions.assertEquals(2040504053, model.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("uvarmywdmjsjq", model.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1756269391, model.monthlySchedule().hour()); + Assertions.assertEquals(1027618115, model.monthlySchedule().minute()); + Assertions.assertEquals(6679760626312548131L, model.monthlySchedule().usedBytes()); Assertions.assertEquals(false, model.enabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { SnapshotPolicyProperties model = new SnapshotPolicyProperties() - .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(747190122).withMinute(383996827) - .withUsedBytes(8358866216552319675L)) - .withDailySchedule(new DailySchedule().withSnapshotsToKeep(2045472309).withHour(1322383691) - .withMinute(1729344688).withUsedBytes(4145471517526414946L)) - .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(1389326915).withDay("iilivpdtiirqtd") - .withHour(887642500).withMinute(1693357038).withUsedBytes(3114947430528398041L)) - .withMonthlySchedule( - new MonthlySchedule().withSnapshotsToKeep(1061692620).withDaysOfMonth("uyfxrxxleptramxj") - .withHour(576385447).withMinute(800812531).withUsedBytes(9127393651345868324L)) + .withHourlySchedule(new HourlySchedule().withSnapshotsToKeep(885439585).withMinute(270423154) + .withUsedBytes(5614257163136424427L)) + .withDailySchedule(new DailySchedule().withSnapshotsToKeep(1081755723).withHour(564076169) + .withMinute(1794228240).withUsedBytes(5489756358952396230L)) + .withWeeklySchedule(new WeeklySchedule().withSnapshotsToKeep(253317009).withDay("eqidbqfatpx") + .withHour(48002498).withMinute(1459715111).withUsedBytes(8864644598647393386L)) + .withMonthlySchedule(new MonthlySchedule().withSnapshotsToKeep(2040504053).withDaysOfMonth("uvarmywdmjsjq") + .withHour(1756269391).withMinute(1027618115).withUsedBytes(6679760626312548131L)) .withEnabled(false); model = BinaryData.fromObject(model).toObject(SnapshotPolicyProperties.class); - Assertions.assertEquals(747190122, model.hourlySchedule().snapshotsToKeep()); - Assertions.assertEquals(383996827, model.hourlySchedule().minute()); - Assertions.assertEquals(8358866216552319675L, model.hourlySchedule().usedBytes()); - Assertions.assertEquals(2045472309, model.dailySchedule().snapshotsToKeep()); - Assertions.assertEquals(1322383691, model.dailySchedule().hour()); - Assertions.assertEquals(1729344688, model.dailySchedule().minute()); - Assertions.assertEquals(4145471517526414946L, model.dailySchedule().usedBytes()); - Assertions.assertEquals(1389326915, model.weeklySchedule().snapshotsToKeep()); - Assertions.assertEquals("iilivpdtiirqtd", model.weeklySchedule().day()); - Assertions.assertEquals(887642500, model.weeklySchedule().hour()); - Assertions.assertEquals(1693357038, model.weeklySchedule().minute()); - Assertions.assertEquals(3114947430528398041L, model.weeklySchedule().usedBytes()); - Assertions.assertEquals(1061692620, model.monthlySchedule().snapshotsToKeep()); - Assertions.assertEquals("uyfxrxxleptramxj", model.monthlySchedule().daysOfMonth()); - Assertions.assertEquals(576385447, model.monthlySchedule().hour()); - Assertions.assertEquals(800812531, model.monthlySchedule().minute()); - Assertions.assertEquals(9127393651345868324L, model.monthlySchedule().usedBytes()); + Assertions.assertEquals(885439585, model.hourlySchedule().snapshotsToKeep()); + Assertions.assertEquals(270423154, model.hourlySchedule().minute()); + Assertions.assertEquals(5614257163136424427L, model.hourlySchedule().usedBytes()); + Assertions.assertEquals(1081755723, model.dailySchedule().snapshotsToKeep()); + Assertions.assertEquals(564076169, model.dailySchedule().hour()); + Assertions.assertEquals(1794228240, model.dailySchedule().minute()); + Assertions.assertEquals(5489756358952396230L, model.dailySchedule().usedBytes()); + Assertions.assertEquals(253317009, model.weeklySchedule().snapshotsToKeep()); + Assertions.assertEquals("eqidbqfatpx", model.weeklySchedule().day()); + Assertions.assertEquals(48002498, model.weeklySchedule().hour()); + Assertions.assertEquals(1459715111, model.weeklySchedule().minute()); + Assertions.assertEquals(8864644598647393386L, model.weeklySchedule().usedBytes()); + Assertions.assertEquals(2040504053, model.monthlySchedule().snapshotsToKeep()); + Assertions.assertEquals("uvarmywdmjsjq", model.monthlySchedule().daysOfMonth()); + Assertions.assertEquals(1756269391, model.monthlySchedule().hour()); + Assertions.assertEquals(1027618115, model.monthlySchedule().minute()); + Assertions.assertEquals(6679760626312548131L, model.monthlySchedule().usedBytes()); Assertions.assertEquals(false, model.enabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPropertiesTests.java index d479a9927931c..244ad0813739c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotPropertiesTests.java @@ -10,8 +10,9 @@ public final class SnapshotPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SnapshotProperties model = BinaryData.fromString( - "{\"snapshotId\":\"kwqqtchealmf\",\"created\":\"2021-05-23T20:32:01Z\",\"provisioningState\":\"aygdvwvgpioh\"}") + SnapshotProperties model = BinaryData + .fromString( + "{\"snapshotId\":\"pnppfuf\",\"created\":\"2021-04-27T03:55:43Z\",\"provisioningState\":\"mh\"}") .toObject(SnapshotProperties.class); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotRestoreFilesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotRestoreFilesTests.java index 06f580b79f68c..6d0bf6955135c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotRestoreFilesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotRestoreFilesTests.java @@ -13,18 +13,18 @@ public final class SnapshotRestoreFilesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotRestoreFiles model - = BinaryData.fromString("{\"filePaths\":[\"xrtfudxep\"],\"destinationPath\":\"yqagvrvm\"}") + = BinaryData.fromString("{\"filePaths\":[\"xyjrxsagafcnih\",\"wqapnedgfbcvk\"],\"destinationPath\":\"q\"}") .toObject(SnapshotRestoreFiles.class); - Assertions.assertEquals("xrtfudxep", model.filePaths().get(0)); - Assertions.assertEquals("yqagvrvm", model.destinationPath()); + Assertions.assertEquals("xyjrxsagafcnih", model.filePaths().get(0)); + Assertions.assertEquals("q", model.destinationPath()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnapshotRestoreFiles model - = new SnapshotRestoreFiles().withFilePaths(Arrays.asList("xrtfudxep")).withDestinationPath("yqagvrvm"); + SnapshotRestoreFiles model = new SnapshotRestoreFiles() + .withFilePaths(Arrays.asList("xyjrxsagafcnih", "wqapnedgfbcvk")).withDestinationPath("q"); model = BinaryData.fromObject(model).toObject(SnapshotRestoreFiles.class); - Assertions.assertEquals("xrtfudxep", model.filePaths().get(0)); - Assertions.assertEquals("yqagvrvm", model.destinationPath()); + Assertions.assertEquals("xyjrxsagafcnih", model.filePaths().get(0)); + Assertions.assertEquals("q", model.destinationPath()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteMockTests.java index 54659b7c635a3..739a843904065 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.snapshots().delete("nzrdpsovwxz", "ptgoeiybbabp", "hv", "slkvntjl", "igjkskyrio", + manager.snapshots().delete("kzyb", "jjidjk", "yxkyxvx", "vblbjednljlageua", "lxunsmjbnkppxy", com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetWithResponseMockTests.java index 66a2debc70977..9c363940afb23 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"location\":\"ap\",\"properties\":{\"snapshotId\":\"ferr\",\"created\":\"2021-05-25T15:13:52Z\",\"provisioningState\":\"jkmfxapjwo\"},\"id\":\"qnobp\",\"name\":\"dcdab\",\"type\":\"qwpwyawbzasqbuc\"}"; + = "{\"location\":\"ti\",\"properties\":{\"snapshotId\":\"pwpgclrci\",\"created\":\"2021-07-05T16:33:25Z\",\"provisioningState\":\"xfrk\"},\"id\":\"xpmyyefrpmpdnq\",\"name\":\"skawaoqvmmb\",\"type\":\"pqfrtqlkz\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,9 +48,9 @@ public void testGetWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - Snapshot response = manager.snapshots() - .getWithResponse("a", "zv", "chndbnwie", "olewjwi", "ubwefqs", com.azure.core.util.Context.NONE).getValue(); + Snapshot response = manager.snapshots().getWithResponse("ttymsjny", "qdnfwqzdz", "tilaxh", "fhqlyvi", + "ouwivkxoyzunbixx", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("ap", response.location()); + Assertions.assertEquals("ti", response.location()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListMockTests.java index fc3cde0193877..55f1527567543 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListMockTests.java @@ -32,7 +32,7 @@ public void testList() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"location\":\"utmzlbiojlvfhrbb\",\"properties\":{\"snapshotId\":\"qvcww\",\"created\":\"2021-11-09T01:20:46Z\",\"provisioningState\":\"mochpprprsnmok\"},\"id\":\"zejnhl\",\"name\":\"kpbz\",\"type\":\"cpilj\"}]}"; + = "{\"value\":[{\"location\":\"zhhzjhfjmhvvmu\",\"properties\":{\"snapshotId\":\"mun\",\"created\":\"2021-11-13T08:37:26Z\",\"provisioningState\":\"vmhfbuz\"},\"id\":\"ihsasb\",\"name\":\"udypohyuems\",\"type\":\"ynsqyrpfoobr\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,9 +49,9 @@ public void testList() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response - = manager.snapshots().list("pnodawopqhe", "jptmcgsbost", "eln", "la", com.azure.core.util.Context.NONE); + PagedIterable response = manager.snapshots().list("rdve", "nwsdtutnwlduyc", "uzhyrmewipmvekdx", + "kuqgsjjxundxgket", com.azure.core.util.Context.NONE); - Assertions.assertEquals("utmzlbiojlvfhrbb", response.iterator().next().location()); + Assertions.assertEquals("zhhzjhfjmhvvmu", response.iterator().next().location()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListTests.java index c53526e8dd345..6d7cde6d8b145 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsListTests.java @@ -14,16 +14,16 @@ public final class SnapshotsListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SnapshotsList model = BinaryData.fromString( - "{\"value\":[{\"location\":\"ph\",\"properties\":{\"snapshotId\":\"xodlqiyntorzih\",\"created\":\"2021-04-08T02:51:20Z\",\"provisioningState\":\"jswsrmslyz\"},\"id\":\"zbchckqqzqioxiy\",\"name\":\"uiizynke\",\"type\":\"yatrwy\"},{\"location\":\"q\",\"properties\":{\"snapshotId\":\"zyh\",\"created\":\"2021-05-20T17:58:30Z\",\"provisioningState\":\"mypyynpcdpu\"},\"id\":\"zgmwznmabikns\",\"name\":\"rgjhxb\",\"type\":\"dtlwwrlkd\"},{\"location\":\"tncvokot\",\"properties\":{\"snapshotId\":\"d\",\"created\":\"2021-07-23T09:43:53Z\",\"provisioningState\":\"y\"},\"id\":\"ogjltdtbnnhad\",\"name\":\"ocrkvcikh\",\"type\":\"vpa\"}]}") + "{\"value\":[{\"location\":\"zmuvpbttdumorppx\",\"properties\":{\"snapshotId\":\"nzbtbhj\",\"created\":\"2021-10-23T02:17:02Z\",\"provisioningState\":\"fgohdneuelfphs\"},\"id\":\"htozfikdow\",\"name\":\"quuvxzxcl\",\"type\":\"ithhqzon\"}]}") .toObject(SnapshotsList.class); - Assertions.assertEquals("ph", model.value().get(0).location()); + Assertions.assertEquals("zmuvpbttdumorppx", model.value().get(0).location()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SnapshotsList model = new SnapshotsList().withValue(Arrays.asList(new SnapshotInner().withLocation("ph"), - new SnapshotInner().withLocation("q"), new SnapshotInner().withLocation("tncvokot"))); + SnapshotsList model + = new SnapshotsList().withValue(Arrays.asList(new SnapshotInner().withLocation("zmuvpbttdumorppx"))); model = BinaryData.fromObject(model).toObject(SnapshotsList.class); - Assertions.assertEquals("ph", model.value().get(0).location()); + Assertions.assertEquals("zmuvpbttdumorppx", model.value().get(0).location()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesMockTests.java index 86f51040ca014..8629fecd17ed2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsRestoreFilesMockTests.java @@ -47,9 +47,8 @@ public void testRestoreFiles() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.snapshots().restoreFiles( - "vzidsxwaab", "mifrygznmma", "ri", "kzobgopxlhsln", "lxieixynllxecwcr", new SnapshotRestoreFiles() - .withFilePaths(Arrays.asList("phslhcawjutifdw", "mvi", "orq")).withDestinationPath("ttzhra"), + manager.snapshots().restoreFiles("enlsvxeizzgwkln", "rmffeyc", "ckt", "iymerteeammxqi", "kk", + new SnapshotRestoreFiles().withFilePaths(Arrays.asList("ddrtkgdojb", "xv")).withDestinationPath("refdee"), com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateMockTests.java index b92dba17b7c01..5dea340d3adcc 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SnapshotsUpdateMockTests.java @@ -31,7 +31,7 @@ public void testUpdate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"location\":\"azlnqnmcjngzqdqx\",\"properties\":{\"snapshotId\":\"wgnyfusfzsvtui\",\"created\":\"2021-03-01T07:50:01Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"fh\",\"name\":\"l\",\"type\":\"qryxyn\"}"; + = "{\"location\":\"xakjsqzhzb\",\"properties\":{\"snapshotId\":\"gims\",\"created\":\"2021-03-21T09:55:03Z\",\"provisioningState\":\"Succeeded\"},\"id\":\"kgfmocwahpq\",\"name\":\"atjeaahh\",\"type\":\"jhhn\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,9 +48,9 @@ public void testUpdate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - Snapshot response = manager.snapshots().update("jg", "yexaoguy", "i", "ids", "ault", "dataijjumfq", - com.azure.core.util.Context.NONE); + Snapshot response = manager.snapshots().update("egnitg", "kxlzyqdrfeg", "ealzxwhcansymoyq", "lwigdivbkbx", + "omfaj", "datawasqvdaeyyg", com.azure.core.util.Context.NONE); - Assertions.assertEquals("azlnqnmcjngzqdqx", response.location()); + Assertions.assertEquals("xakjsqzhzb", response.location()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeInfoInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeInfoInnerTests.java index eb82b7315141c..2f82969f935cf 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeInfoInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeInfoInnerTests.java @@ -12,20 +12,20 @@ public final class SubvolumeInfoInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SubvolumeInfoInner model = BinaryData.fromString( - "{\"properties\":{\"path\":\"rk\",\"size\":6236103366056271991,\"parentPath\":\"jhhgdnhxmsi\",\"provisioningState\":\"omi\"},\"id\":\"xggdufi\",\"name\":\"ndieuzaofj\",\"type\":\"hvcyyysfg\"}") + "{\"properties\":{\"path\":\"owvrvmtgjqppyos\",\"size\":1997162688114762581,\"parentPath\":\"myhgfipnsxkmc\",\"provisioningState\":\"ekrrjr\"},\"id\":\"fxtsgum\",\"name\":\"jglikkxwslolb\",\"type\":\"pvuzlmv\"}") .toObject(SubvolumeInfoInner.class); - Assertions.assertEquals("rk", model.path()); - Assertions.assertEquals(6236103366056271991L, model.size()); - Assertions.assertEquals("jhhgdnhxmsi", model.parentPath()); + Assertions.assertEquals("owvrvmtgjqppyos", model.path()); + Assertions.assertEquals(1997162688114762581L, model.size()); + Assertions.assertEquals("myhgfipnsxkmc", model.parentPath()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumeInfoInner model - = new SubvolumeInfoInner().withPath("rk").withSize(6236103366056271991L).withParentPath("jhhgdnhxmsi"); + SubvolumeInfoInner model = new SubvolumeInfoInner().withPath("owvrvmtgjqppyos").withSize(1997162688114762581L) + .withParentPath("myhgfipnsxkmc"); model = BinaryData.fromObject(model).toObject(SubvolumeInfoInner.class); - Assertions.assertEquals("rk", model.path()); - Assertions.assertEquals(6236103366056271991L, model.size()); - Assertions.assertEquals("jhhgdnhxmsi", model.parentPath()); + Assertions.assertEquals("owvrvmtgjqppyos", model.path()); + Assertions.assertEquals(1997162688114762581L, model.size()); + Assertions.assertEquals("myhgfipnsxkmc", model.parentPath()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelInnerTests.java index 919903e9bd92a..6ff393baa6a57 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelInnerTests.java @@ -13,38 +13,38 @@ public final class SubvolumeModelInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SubvolumeModelInner model = BinaryData.fromString( - "{\"id\":\"byowbblgyavutp\",\"name\":\"joxoism\",\"type\":\"sbpimlq\",\"properties\":{\"path\":\"xkcgxxlxsffgcvi\",\"parentPath\":\"zdwlvwlyoupfgfb\",\"size\":7653162173647164897,\"bytesUsed\":4324822470137605092,\"permissions\":\"kfm\",\"creationTimeStamp\":\"2021-07-25T09:50:13Z\",\"accessedTimeStamp\":\"2021-10-13T02:52:40Z\",\"modifiedTimeStamp\":\"2021-07-19T15:33:36Z\",\"changedTimeStamp\":\"2021-01-19T00:17:43Z\",\"provisioningState\":\"sttktlahbqa\"}}") + "{\"id\":\"wuenvr\",\"name\":\"yo\",\"type\":\"ibreb\",\"properties\":{\"path\":\"ysjkixqtnqttez\",\"parentPath\":\"fffiak\",\"size\":7989893119356760730,\"bytesUsed\":167842444741645493,\"permissions\":\"d\",\"creationTimeStamp\":\"2021-09-01T21:15:43Z\",\"accessedTimeStamp\":\"2021-03-09T02:32:12Z\",\"modifiedTimeStamp\":\"2021-08-03T11:39:14Z\",\"changedTimeStamp\":\"2021-01-23T06:09:40Z\",\"provisioningState\":\"zphv\"}}") .toObject(SubvolumeModelInner.class); - Assertions.assertEquals("xkcgxxlxsffgcvi", model.path()); - Assertions.assertEquals("zdwlvwlyoupfgfb", model.parentPath()); - Assertions.assertEquals(7653162173647164897L, model.size()); - Assertions.assertEquals(4324822470137605092L, model.bytesUsed()); - Assertions.assertEquals("kfm", model.permissions()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-25T09:50:13Z"), model.creationTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-13T02:52:40Z"), model.accessedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-19T15:33:36Z"), model.modifiedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-19T00:17:43Z"), model.changedTimestamp()); - Assertions.assertEquals("sttktlahbqa", model.provisioningState()); + Assertions.assertEquals("ysjkixqtnqttez", model.path()); + Assertions.assertEquals("fffiak", model.parentPath()); + Assertions.assertEquals(7989893119356760730L, model.size()); + Assertions.assertEquals(167842444741645493L, model.bytesUsed()); + Assertions.assertEquals("d", model.permissions()); + Assertions.assertEquals(OffsetDateTime.parse("2021-09-01T21:15:43Z"), model.creationTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-09T02:32:12Z"), model.accessedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-03T11:39:14Z"), model.modifiedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-23T06:09:40Z"), model.changedTimestamp()); + Assertions.assertEquals("zphv", model.provisioningState()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumeModelInner model = new SubvolumeModelInner().withPath("xkcgxxlxsffgcvi") - .withParentPath("zdwlvwlyoupfgfb").withSize(7653162173647164897L).withBytesUsed(4324822470137605092L) - .withPermissions("kfm").withCreationTimestamp(OffsetDateTime.parse("2021-07-25T09:50:13Z")) - .withAccessedTimestamp(OffsetDateTime.parse("2021-10-13T02:52:40Z")) - .withModifiedTimestamp(OffsetDateTime.parse("2021-07-19T15:33:36Z")) - .withChangedTimestamp(OffsetDateTime.parse("2021-01-19T00:17:43Z")).withProvisioningState("sttktlahbqa"); + SubvolumeModelInner model = new SubvolumeModelInner().withPath("ysjkixqtnqttez").withParentPath("fffiak") + .withSize(7989893119356760730L).withBytesUsed(167842444741645493L).withPermissions("d") + .withCreationTimestamp(OffsetDateTime.parse("2021-09-01T21:15:43Z")) + .withAccessedTimestamp(OffsetDateTime.parse("2021-03-09T02:32:12Z")) + .withModifiedTimestamp(OffsetDateTime.parse("2021-08-03T11:39:14Z")) + .withChangedTimestamp(OffsetDateTime.parse("2021-01-23T06:09:40Z")).withProvisioningState("zphv"); model = BinaryData.fromObject(model).toObject(SubvolumeModelInner.class); - Assertions.assertEquals("xkcgxxlxsffgcvi", model.path()); - Assertions.assertEquals("zdwlvwlyoupfgfb", model.parentPath()); - Assertions.assertEquals(7653162173647164897L, model.size()); - Assertions.assertEquals(4324822470137605092L, model.bytesUsed()); - Assertions.assertEquals("kfm", model.permissions()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-25T09:50:13Z"), model.creationTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-13T02:52:40Z"), model.accessedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-19T15:33:36Z"), model.modifiedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-19T00:17:43Z"), model.changedTimestamp()); - Assertions.assertEquals("sttktlahbqa", model.provisioningState()); + Assertions.assertEquals("ysjkixqtnqttez", model.path()); + Assertions.assertEquals("fffiak", model.parentPath()); + Assertions.assertEquals(7989893119356760730L, model.size()); + Assertions.assertEquals(167842444741645493L, model.bytesUsed()); + Assertions.assertEquals("d", model.permissions()); + Assertions.assertEquals(OffsetDateTime.parse("2021-09-01T21:15:43Z"), model.creationTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-03-09T02:32:12Z"), model.accessedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-08-03T11:39:14Z"), model.modifiedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-23T06:09:40Z"), model.changedTimestamp()); + Assertions.assertEquals("zphv", model.provisioningState()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelPropertiesTests.java index bcc8c732e7745..414ccd547d018 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumeModelPropertiesTests.java @@ -13,38 +13,38 @@ public final class SubvolumeModelPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SubvolumeModelProperties model = BinaryData.fromString( - "{\"path\":\"xtgzukxitmmqtgqq\",\"parentPath\":\"hrnxrxc\",\"size\":8912049813335593568,\"bytesUsed\":1928679362858803992,\"permissions\":\"k\",\"creationTimeStamp\":\"2021-02-23T17:29:03Z\",\"accessedTimeStamp\":\"2021-09-10T00:13:37Z\",\"modifiedTimeStamp\":\"2021-11-24T02:59:44Z\",\"changedTimeStamp\":\"2021-09-15T15:31:22Z\",\"provisioningState\":\"l\"}") + "{\"path\":\"uyqncygupkvipmd\",\"parentPath\":\"wx\",\"size\":9152224123184023671,\"bytesUsed\":7138744715268482510,\"permissions\":\"stotxh\",\"creationTimeStamp\":\"2021-02-09T01:55:57Z\",\"accessedTimeStamp\":\"2021-12-08T17:06:05Z\",\"modifiedTimeStamp\":\"2021-04-25T15:36:32Z\",\"changedTimeStamp\":\"2021-06-17T22:54:10Z\",\"provisioningState\":\"c\"}") .toObject(SubvolumeModelProperties.class); - Assertions.assertEquals("xtgzukxitmmqtgqq", model.path()); - Assertions.assertEquals("hrnxrxc", model.parentPath()); - Assertions.assertEquals(8912049813335593568L, model.size()); - Assertions.assertEquals(1928679362858803992L, model.bytesUsed()); - Assertions.assertEquals("k", model.permissions()); - Assertions.assertEquals(OffsetDateTime.parse("2021-02-23T17:29:03Z"), model.creationTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-10T00:13:37Z"), model.accessedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-11-24T02:59:44Z"), model.modifiedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-15T15:31:22Z"), model.changedTimestamp()); - Assertions.assertEquals("l", model.provisioningState()); + Assertions.assertEquals("uyqncygupkvipmd", model.path()); + Assertions.assertEquals("wx", model.parentPath()); + Assertions.assertEquals(9152224123184023671L, model.size()); + Assertions.assertEquals(7138744715268482510L, model.bytesUsed()); + Assertions.assertEquals("stotxh", model.permissions()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-09T01:55:57Z"), model.creationTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-12-08T17:06:05Z"), model.accessedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-25T15:36:32Z"), model.modifiedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-17T22:54:10Z"), model.changedTimestamp()); + Assertions.assertEquals("c", model.provisioningState()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumeModelProperties model = new SubvolumeModelProperties().withPath("xtgzukxitmmqtgqq") - .withParentPath("hrnxrxc").withSize(8912049813335593568L).withBytesUsed(1928679362858803992L) - .withPermissions("k").withCreationTimestamp(OffsetDateTime.parse("2021-02-23T17:29:03Z")) - .withAccessedTimestamp(OffsetDateTime.parse("2021-09-10T00:13:37Z")) - .withModifiedTimestamp(OffsetDateTime.parse("2021-11-24T02:59:44Z")) - .withChangedTimestamp(OffsetDateTime.parse("2021-09-15T15:31:22Z")).withProvisioningState("l"); + SubvolumeModelProperties model = new SubvolumeModelProperties().withPath("uyqncygupkvipmd").withParentPath("wx") + .withSize(9152224123184023671L).withBytesUsed(7138744715268482510L).withPermissions("stotxh") + .withCreationTimestamp(OffsetDateTime.parse("2021-02-09T01:55:57Z")) + .withAccessedTimestamp(OffsetDateTime.parse("2021-12-08T17:06:05Z")) + .withModifiedTimestamp(OffsetDateTime.parse("2021-04-25T15:36:32Z")) + .withChangedTimestamp(OffsetDateTime.parse("2021-06-17T22:54:10Z")).withProvisioningState("c"); model = BinaryData.fromObject(model).toObject(SubvolumeModelProperties.class); - Assertions.assertEquals("xtgzukxitmmqtgqq", model.path()); - Assertions.assertEquals("hrnxrxc", model.parentPath()); - Assertions.assertEquals(8912049813335593568L, model.size()); - Assertions.assertEquals(1928679362858803992L, model.bytesUsed()); - Assertions.assertEquals("k", model.permissions()); - Assertions.assertEquals(OffsetDateTime.parse("2021-02-23T17:29:03Z"), model.creationTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-10T00:13:37Z"), model.accessedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-11-24T02:59:44Z"), model.modifiedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-09-15T15:31:22Z"), model.changedTimestamp()); - Assertions.assertEquals("l", model.provisioningState()); + Assertions.assertEquals("uyqncygupkvipmd", model.path()); + Assertions.assertEquals("wx", model.parentPath()); + Assertions.assertEquals(9152224123184023671L, model.size()); + Assertions.assertEquals(7138744715268482510L, model.bytesUsed()); + Assertions.assertEquals("stotxh", model.permissions()); + Assertions.assertEquals(OffsetDateTime.parse("2021-02-09T01:55:57Z"), model.creationTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-12-08T17:06:05Z"), model.accessedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-25T15:36:32Z"), model.modifiedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-17T22:54:10Z"), model.changedTimestamp()); + Assertions.assertEquals("c", model.provisioningState()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchParamsTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchParamsTests.java index e9b06d56b747c..c3dad53f127a2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchParamsTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchParamsTests.java @@ -11,19 +11,17 @@ public final class SubvolumePatchParamsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SubvolumePatchParams model - = BinaryData.fromString("{\"size\":8527359607634175876,\"path\":\"fnynszqujizdvoqy\"}") - .toObject(SubvolumePatchParams.class); - Assertions.assertEquals(8527359607634175876L, model.size()); - Assertions.assertEquals("fnynszqujizdvoqy", model.path()); + SubvolumePatchParams model = BinaryData.fromString("{\"size\":5455709346617083840,\"path\":\"j\"}") + .toObject(SubvolumePatchParams.class); + Assertions.assertEquals(5455709346617083840L, model.size()); + Assertions.assertEquals("j", model.path()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumePatchParams model - = new SubvolumePatchParams().withSize(8527359607634175876L).withPath("fnynszqujizdvoqy"); + SubvolumePatchParams model = new SubvolumePatchParams().withSize(5455709346617083840L).withPath("j"); model = BinaryData.fromObject(model).toObject(SubvolumePatchParams.class); - Assertions.assertEquals(8527359607634175876L, model.size()); - Assertions.assertEquals("fnynszqujizdvoqy", model.path()); + Assertions.assertEquals(5455709346617083840L, model.size()); + Assertions.assertEquals("j", model.path()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchRequestTests.java index 0c840028f649d..c25d87d18596c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePatchRequestTests.java @@ -12,17 +12,18 @@ public final class SubvolumePatchRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SubvolumePatchRequest model - = BinaryData.fromString("{\"properties\":{\"size\":3659481881596773393,\"path\":\"ccwr\"}}") + = BinaryData.fromString("{\"properties\":{\"size\":4992993506097111219,\"path\":\"augzxnfaazpxdtn\"}}") .toObject(SubvolumePatchRequest.class); - Assertions.assertEquals(3659481881596773393L, model.size()); - Assertions.assertEquals("ccwr", model.path()); + Assertions.assertEquals(4992993506097111219L, model.size()); + Assertions.assertEquals("augzxnfaazpxdtn", model.path()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumePatchRequest model = new SubvolumePatchRequest().withSize(3659481881596773393L).withPath("ccwr"); + SubvolumePatchRequest model + = new SubvolumePatchRequest().withSize(4992993506097111219L).withPath("augzxnfaazpxdtn"); model = BinaryData.fromObject(model).toObject(SubvolumePatchRequest.class); - Assertions.assertEquals(3659481881596773393L, model.size()); - Assertions.assertEquals("ccwr", model.path()); + Assertions.assertEquals(4992993506097111219L, model.size()); + Assertions.assertEquals("augzxnfaazpxdtn", model.path()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePropertiesTests.java index e46448978167b..7dd148b0fa0f5 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumePropertiesTests.java @@ -12,20 +12,20 @@ public final class SubvolumePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SubvolumeProperties model = BinaryData.fromString( - "{\"path\":\"tcubiipuipwoqonm\",\"size\":941604235474814179,\"parentPath\":\"nizshqvcim\",\"provisioningState\":\"vfgmblrrilby\"}") + "{\"path\":\"lfktgplcrpwjxe\",\"size\":1213023006676271308,\"parentPath\":\"brnjwmw\",\"provisioningState\":\"nbsazejjoqkag\"}") .toObject(SubvolumeProperties.class); - Assertions.assertEquals("tcubiipuipwoqonm", model.path()); - Assertions.assertEquals(941604235474814179L, model.size()); - Assertions.assertEquals("nizshqvcim", model.parentPath()); + Assertions.assertEquals("lfktgplcrpwjxe", model.path()); + Assertions.assertEquals(1213023006676271308L, model.size()); + Assertions.assertEquals("brnjwmw", model.parentPath()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumeProperties model = new SubvolumeProperties().withPath("tcubiipuipwoqonm").withSize(941604235474814179L) - .withParentPath("nizshqvcim"); + SubvolumeProperties model = new SubvolumeProperties().withPath("lfktgplcrpwjxe").withSize(1213023006676271308L) + .withParentPath("brnjwmw"); model = BinaryData.fromObject(model).toObject(SubvolumeProperties.class); - Assertions.assertEquals("tcubiipuipwoqonm", model.path()); - Assertions.assertEquals(941604235474814179L, model.size()); - Assertions.assertEquals("nizshqvcim", model.parentPath()); + Assertions.assertEquals("lfktgplcrpwjxe", model.path()); + Assertions.assertEquals(1213023006676271308L, model.size()); + Assertions.assertEquals("brnjwmw", model.parentPath()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateMockTests.java index 362cecae378f6..ed9d984c76897 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesCreateMockTests.java @@ -31,7 +31,7 @@ public void testCreate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"path\":\"gpiudeug\",\"size\":8412472612834852479,\"parentPath\":\"c\",\"provisioningState\":\"Succeeded\"},\"id\":\"fykhvuhxepmru\",\"name\":\"znabaobns\",\"type\":\"ujdjltymkmvg\"}"; + = "{\"properties\":{\"path\":\"oweyirdhlis\",\"size\":2577921237781944905,\"parentPath\":\"qqmpizruwnpqx\",\"provisioningState\":\"Succeeded\"},\"id\":\"iixtmkzj\",\"name\":\"kv\",\"type\":\"irhgfgrwsdp\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,12 +48,12 @@ public void testCreate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - SubvolumeInfo response - = manager.subvolumes().define("sggux").withExistingVolume("naoyank", "oe", "swankltytmh", "roznnhdrlktgj") - .withPath("lwywae").withSize(4845732356948511994L).withParentPath("bukklels").create(); + SubvolumeInfo response = manager.subvolumes().define("nlb") + .withExistingVolume("ebwlnbmhyreeudzq", "vbpdqmjxlyyzglgo", "wtlmjjyuo", "qtobaxkjeyt").withPath("wrus") + .withSize(7003868836255706377L).withParentPath("syrq").create(); - Assertions.assertEquals("gpiudeug", response.path()); - Assertions.assertEquals(8412472612834852479L, response.size()); - Assertions.assertEquals("c", response.parentPath()); + Assertions.assertEquals("oweyirdhlis", response.path()); + Assertions.assertEquals(2577921237781944905L, response.size()); + Assertions.assertEquals("qqmpizruwnpqx", response.parentPath()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteMockTests.java index db940505bce9d..8d108bcbd53be 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesDeleteMockTests.java @@ -45,8 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.subvolumes().delete("uqtljqobbpih", "hcecybmrqbr", "bbmpxdlvykfre", "crse", "wjksghudgzhxo", - com.azure.core.util.Context.NONE); + manager.subvolumes().delete("g", "pnpbswveflocc", "rmozihmipgawt", "xp", "y", com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataMockTests.java index 67b348f5d0745..da1121c71cf72 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetMetadataMockTests.java @@ -31,7 +31,7 @@ public void testGetMetadata() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"id\":\"ddei\",\"name\":\"wzovgk\",\"type\":\"muikjcjcaztbws\",\"properties\":{\"path\":\"owxwcomli\",\"parentPath\":\"twvc\",\"size\":9048131341769912568,\"bytesUsed\":3791668665585819281,\"permissions\":\"ejyfdvlvhbwrnfx\",\"creationTimeStamp\":\"2021-04-12T09:19:16Z\",\"accessedTimeStamp\":\"2021-01-12T00:32:09Z\",\"modifiedTimeStamp\":\"2021-07-21T21:17:06Z\",\"changedTimeStamp\":\"2021-04-10T14:59:15Z\",\"provisioningState\":\"Succeeded\"}}"; + = "{\"id\":\"pfnznthjtwkj\",\"name\":\"srxuzvoam\",\"type\":\"cqiosmgbza\",\"properties\":{\"path\":\"qdlyrtl\",\"parentPath\":\"ap\",\"size\":1522364843172111954,\"bytesUsed\":773093768865892885,\"permissions\":\"hjm\",\"creationTimeStamp\":\"2021-01-20T21:27:07Z\",\"accessedTimeStamp\":\"2021-04-06T21:21:13Z\",\"modifiedTimeStamp\":\"2021-07-25T22:28:32Z\",\"changedTimeStamp\":\"2021-06-18T22:17:48Z\",\"provisioningState\":\"Succeeded\"}}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -48,18 +48,18 @@ public void testGetMetadata() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - SubvolumeModel response = manager.subvolumes().getMetadata("jggsvo", "jkxibda", "hrkmdyomkxfbvfbh", "y", "rhpw", - com.azure.core.util.Context.NONE); + SubvolumeModel response = manager.subvolumes().getMetadata("cxcjxgry", "fmpcycilrmca", "kggnoxu", "t", + "ksxwpnd", com.azure.core.util.Context.NONE); - Assertions.assertEquals("owxwcomli", response.path()); - Assertions.assertEquals("twvc", response.parentPath()); - Assertions.assertEquals(9048131341769912568L, response.size()); - Assertions.assertEquals(3791668665585819281L, response.bytesUsed()); - Assertions.assertEquals("ejyfdvlvhbwrnfx", response.permissions()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-12T09:19:16Z"), response.creationTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-12T00:32:09Z"), response.accessedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-21T21:17:06Z"), response.modifiedTimestamp()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-10T14:59:15Z"), response.changedTimestamp()); + Assertions.assertEquals("qdlyrtl", response.path()); + Assertions.assertEquals("ap", response.parentPath()); + Assertions.assertEquals(1522364843172111954L, response.size()); + Assertions.assertEquals(773093768865892885L, response.bytesUsed()); + Assertions.assertEquals("hjm", response.permissions()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-20T21:27:07Z"), response.creationTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-06T21:21:13Z"), response.accessedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-07-25T22:28:32Z"), response.modifiedTimestamp()); + Assertions.assertEquals(OffsetDateTime.parse("2021-06-18T22:17:48Z"), response.changedTimestamp()); Assertions.assertEquals("Succeeded", response.provisioningState()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetWithResponseMockTests.java index 12e15a541fb8a..a5b07b5cd35ec 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesGetWithResponseMockTests.java @@ -31,7 +31,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"path\":\"jyslurl\",\"size\":1165766341545693911,\"parentPath\":\"vpedwqslsrh\",\"provisioningState\":\"qvwwsko\"},\"id\":\"cbrwi\",\"name\":\"uvqejosovyrrle\",\"type\":\"esi\"}"; + = "{\"properties\":{\"path\":\"i\",\"size\":8841673596868858874,\"parentPath\":\"wmxqhndvnoamlds\",\"provisioningState\":\"aohdjh\"},\"id\":\"lzok\",\"name\":\"coxpelnjeta\",\"type\":\"ltsxoatf\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testGetWithResponse() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); SubvolumeInfo response = manager.subvolumes() - .getWithResponse("igsxcdgljplk", "ua", "htomflrytswfp", "mdgycxn", "skw", com.azure.core.util.Context.NONE) + .getWithResponse("qbsms", "ziqgfuh", "kzruswh", "hczznvf", "ycjsx", com.azure.core.util.Context.NONE) .getValue(); - Assertions.assertEquals("jyslurl", response.path()); - Assertions.assertEquals(1165766341545693911L, response.size()); - Assertions.assertEquals("vpedwqslsrh", response.parentPath()); + Assertions.assertEquals("i", response.path()); + Assertions.assertEquals(8841673596868858874L, response.size()); + Assertions.assertEquals("wmxqhndvnoamlds", response.parentPath()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeMockTests.java index 6c7afe8465d1f..8bd2fd76fd711 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListByVolumeMockTests.java @@ -32,7 +32,7 @@ public void testListByVolume() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"path\":\"tlgy\",\"size\":8015109045379432698,\"parentPath\":\"lnnpx\",\"provisioningState\":\"afiqgeaarbgjekg\"},\"id\":\"lbyulidwcwvmze\",\"name\":\"jonfhjirwgdnqzbr\",\"type\":\"kspzhzmtksjcit\"}]}"; + = "{\"value\":[{\"properties\":{\"path\":\"xgvelfclduccbird\",\"size\":3261437061903336605,\"parentPath\":\"o\",\"provisioningState\":\"egstmninwjizci\"},\"id\":\"ghgshejjtbxqmu\",\"name\":\"uxlxqzvners\",\"type\":\"ycucrwnamikzeb\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,11 +49,11 @@ public void testListByVolume() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.subvolumes().listByVolume("jchcsrlzknmzla", "rupdwvnphcnzq", - "pjhmqrhvthl", "iwdcxsmlzzhzd", com.azure.core.util.Context.NONE); + PagedIterable response = manager.subvolumes().listByVolume("rtixokff", "yinljqe", "qwhix", + "onsts", com.azure.core.util.Context.NONE); - Assertions.assertEquals("tlgy", response.iterator().next().path()); - Assertions.assertEquals(8015109045379432698L, response.iterator().next().size()); - Assertions.assertEquals("lnnpx", response.iterator().next().parentPath()); + Assertions.assertEquals("xgvelfclduccbird", response.iterator().next().path()); + Assertions.assertEquals(3261437061903336605L, response.iterator().next().size()); + Assertions.assertEquals("o", response.iterator().next().parentPath()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListTests.java index 149b0e240a560..e9813e7b8c00f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/SubvolumesListTests.java @@ -14,26 +14,23 @@ public final class SubvolumesListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { SubvolumesList model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"path\":\"tac\",\"size\":1896818914623797810,\"parentPath\":\"ewzcjznmwcp\",\"provisioningState\":\"uaadraufactkahzo\"},\"id\":\"jjziuxxpsh\",\"name\":\"eekulfgslqubkwd\",\"type\":\"enr\"},{\"properties\":{\"path\":\"tujbazpju\",\"size\":2825730842736557509,\"parentPath\":\"yfln\",\"provisioningState\":\"wmd\"},\"id\":\"wpklvxw\",\"name\":\"ygdxpgpqchis\",\"type\":\"epn\"},{\"properties\":{\"path\":\"crxgibb\",\"size\":3991598245357381450,\"parentPath\":\"nfo\",\"provisioningState\":\"uors\"},\"id\":\"okwbqplh\",\"name\":\"vnuuepzl\",\"type\":\"phwzsoldweyuqdu\"}],\"nextLink\":\"mnnrwr\"}") + "{\"value\":[{\"properties\":{\"path\":\"nhlmctlpdng\",\"size\":5842965141303303723,\"parentPath\":\"mhrixkwmyijejve\",\"provisioningState\":\"hbpnaixexccbd\"},\"id\":\"ax\",\"name\":\"cexdrrvqa\",\"type\":\"qkghtpwijnhy\"}],\"nextLink\":\"vfycxzb\"}") .toObject(SubvolumesList.class); - Assertions.assertEquals("tac", model.value().get(0).path()); - Assertions.assertEquals(1896818914623797810L, model.value().get(0).size()); - Assertions.assertEquals("ewzcjznmwcp", model.value().get(0).parentPath()); - Assertions.assertEquals("mnnrwr", model.nextLink()); + Assertions.assertEquals("nhlmctlpdng", model.value().get(0).path()); + Assertions.assertEquals(5842965141303303723L, model.value().get(0).size()); + Assertions.assertEquals("mhrixkwmyijejve", model.value().get(0).parentPath()); + Assertions.assertEquals("vfycxzb", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SubvolumesList model = new SubvolumesList() - .withValue(Arrays.asList( - new SubvolumeInfoInner().withPath("tac").withSize(1896818914623797810L).withParentPath("ewzcjznmwcp"), - new SubvolumeInfoInner().withPath("tujbazpju").withSize(2825730842736557509L).withParentPath("yfln"), - new SubvolumeInfoInner().withPath("crxgibb").withSize(3991598245357381450L).withParentPath("nfo"))) - .withNextLink("mnnrwr"); + SubvolumesList model + = new SubvolumesList().withValue(Arrays.asList(new SubvolumeInfoInner().withPath("nhlmctlpdng") + .withSize(5842965141303303723L).withParentPath("mhrixkwmyijejve"))).withNextLink("vfycxzb"); model = BinaryData.fromObject(model).toObject(SubvolumesList.class); - Assertions.assertEquals("tac", model.value().get(0).path()); - Assertions.assertEquals(1896818914623797810L, model.value().get(0).size()); - Assertions.assertEquals("ewzcjznmwcp", model.value().get(0).parentPath()); - Assertions.assertEquals("mnnrwr", model.nextLink()); + Assertions.assertEquals("nhlmctlpdng", model.value().get(0).path()); + Assertions.assertEquals(5842965141303303723L, model.value().get(0).size()); + Assertions.assertEquals("mhrixkwmyijejve", model.value().get(0).parentPath()); + Assertions.assertEquals("vfycxzb", model.nextLink()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UpdateNetworkSiblingSetRequestTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UpdateNetworkSiblingSetRequestTests.java index ddab94d505514..90894dc5b320f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UpdateNetworkSiblingSetRequestTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UpdateNetworkSiblingSetRequestTests.java @@ -13,23 +13,23 @@ public final class UpdateNetworkSiblingSetRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { UpdateNetworkSiblingSetRequest model = BinaryData.fromString( - "{\"networkSiblingSetId\":\"ctyqik\",\"subnetId\":\"bbovplwzbhvgyugu\",\"networkSiblingSetStateId\":\"svmkfssxquk\",\"networkFeatures\":\"Basic\"}") + "{\"networkSiblingSetId\":\"uconuqszfkbey\",\"subnetId\":\"ewrmjmwvvjektc\",\"networkSiblingSetStateId\":\"senhwlrs\",\"networkFeatures\":\"Basic_Standard\"}") .toObject(UpdateNetworkSiblingSetRequest.class); - Assertions.assertEquals("ctyqik", model.networkSiblingSetId()); - Assertions.assertEquals("bbovplwzbhvgyugu", model.subnetId()); - Assertions.assertEquals("svmkfssxquk", model.networkSiblingSetStateId()); - Assertions.assertEquals(NetworkFeatures.BASIC, model.networkFeatures()); + Assertions.assertEquals("uconuqszfkbey", model.networkSiblingSetId()); + Assertions.assertEquals("ewrmjmwvvjektc", model.subnetId()); + Assertions.assertEquals("senhwlrs", model.networkSiblingSetStateId()); + Assertions.assertEquals(NetworkFeatures.BASIC_STANDARD, model.networkFeatures()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - UpdateNetworkSiblingSetRequest model - = new UpdateNetworkSiblingSetRequest().withNetworkSiblingSetId("ctyqik").withSubnetId("bbovplwzbhvgyugu") - .withNetworkSiblingSetStateId("svmkfssxquk").withNetworkFeatures(NetworkFeatures.BASIC); + UpdateNetworkSiblingSetRequest model = new UpdateNetworkSiblingSetRequest() + .withNetworkSiblingSetId("uconuqszfkbey").withSubnetId("ewrmjmwvvjektc") + .withNetworkSiblingSetStateId("senhwlrs").withNetworkFeatures(NetworkFeatures.BASIC_STANDARD); model = BinaryData.fromObject(model).toObject(UpdateNetworkSiblingSetRequest.class); - Assertions.assertEquals("ctyqik", model.networkSiblingSetId()); - Assertions.assertEquals("bbovplwzbhvgyugu", model.subnetId()); - Assertions.assertEquals("svmkfssxquk", model.networkSiblingSetStateId()); - Assertions.assertEquals(NetworkFeatures.BASIC, model.networkFeatures()); + Assertions.assertEquals("uconuqszfkbey", model.networkSiblingSetId()); + Assertions.assertEquals("ewrmjmwvvjektc", model.subnetId()); + Assertions.assertEquals("senhwlrs", model.networkSiblingSetStateId()); + Assertions.assertEquals(NetworkFeatures.BASIC_STANDARD, model.networkFeatures()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UserAssignedIdentityTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UserAssignedIdentityTests.java index 3711611aa4580..c0b93beb57d10 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UserAssignedIdentityTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/UserAssignedIdentityTests.java @@ -11,7 +11,7 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { UserAssignedIdentity model = BinaryData.fromString( - "{\"principalId\":\"d1734cd8-b673-4e87-b4ee-6f00cca9fcf0\",\"clientId\":\"5845f0d8-3244-4e1d-9dab-23b1e56ee2c8\"}") + "{\"principalId\":\"e8e27ede-b960-492b-9ad3-2a1eb06a6518\",\"clientId\":\"b50f57c7-03f1-482c-b559-6715bceae11e\"}") .toObject(UserAssignedIdentity.class); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupPropertiesTests.java deleted file mode 100644 index 119a4d41de278..0000000000000 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupPropertiesTests.java +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.netapp.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; -import org.junit.jupiter.api.Assertions; - -public final class VolumeBackupPropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - VolumeBackupProperties model = BinaryData.fromString( - "{\"backupPolicyId\":\"zbn\",\"policyEnforced\":false,\"backupEnabled\":true,\"backupVaultId\":\"stdbhhxsrzdzu\"}") - .toObject(VolumeBackupProperties.class); - Assertions.assertEquals("zbn", model.backupPolicyId()); - Assertions.assertEquals(false, model.policyEnforced()); - Assertions.assertEquals(true, model.backupEnabled()); - Assertions.assertEquals("stdbhhxsrzdzu", model.backupVaultId()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - VolumeBackupProperties model = new VolumeBackupProperties().withBackupPolicyId("zbn").withPolicyEnforced(false) - .withBackupEnabled(true).withBackupVaultId("stdbhhxsrzdzu"); - model = BinaryData.fromObject(model).toObject(VolumeBackupProperties.class); - Assertions.assertEquals("zbn", model.backupPolicyId()); - Assertions.assertEquals(false, model.policyEnforced()); - Assertions.assertEquals(true, model.backupEnabled()); - Assertions.assertEquals("stdbhhxsrzdzu", model.backupVaultId()); - } -} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupsTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupsTests.java index 5f71912d88622..f4a64874a6889 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupsTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeBackupsTests.java @@ -11,21 +11,21 @@ public final class VolumeBackupsTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VolumeBackups model = BinaryData - .fromString("{\"volumeName\":\"ttaugzxnfaazp\",\"backupsCount\":2134025822,\"policyEnabled\":true}") - .toObject(VolumeBackups.class); - Assertions.assertEquals("ttaugzxnfaazp", model.volumeName()); - Assertions.assertEquals(2134025822, model.backupsCount()); + VolumeBackups model + = BinaryData.fromString("{\"volumeName\":\"wae\",\"backupsCount\":809360205,\"policyEnabled\":true}") + .toObject(VolumeBackups.class); + Assertions.assertEquals("wae", model.volumeName()); + Assertions.assertEquals(809360205, model.backupsCount()); Assertions.assertEquals(true, model.policyEnabled()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { VolumeBackups model - = new VolumeBackups().withVolumeName("ttaugzxnfaazp").withBackupsCount(2134025822).withPolicyEnabled(true); + = new VolumeBackups().withVolumeName("wae").withBackupsCount(809360205).withPolicyEnabled(true); model = BinaryData.fromObject(model).toObject(VolumeBackups.class); - Assertions.assertEquals("ttaugzxnfaazp", model.volumeName()); - Assertions.assertEquals(2134025822, model.backupsCount()); + Assertions.assertEquals("wae", model.volumeName()); + Assertions.assertEquals(809360205, model.backupsCount()); Assertions.assertEquals(true, model.policyEnabled()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteMockTests.java index 69f234197f320..d2c9a01dd756c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeGroupsDeleteMockTests.java @@ -45,7 +45,8 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumeGroups().delete("rag", "gdphtvdula", "vl", com.azure.core.util.Context.NONE); + manager.volumeGroups().delete("ptfujgicgaaoept", "aqutdewemxswvruu", "zzjgehkfki", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesDataProtectionTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesDataProtectionTests.java index 975fb129f6903..20f3e77ccc3e9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesDataProtectionTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesDataProtectionTests.java @@ -5,7 +5,6 @@ package com.azure.resourcemanager.netapp.generated; import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesDataProtection; import com.azure.resourcemanager.netapp.models.VolumeSnapshotProperties; import org.junit.jupiter.api.Assertions; @@ -13,27 +12,16 @@ public final class VolumePatchPropertiesDataProtectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VolumePatchPropertiesDataProtection model = BinaryData.fromString( - "{\"backup\":{\"backupPolicyId\":\"rkwofyyvoqa\",\"policyEnforced\":true,\"backupEnabled\":true,\"backupVaultId\":\"btgiwbwoenwas\"},\"snapshot\":{\"snapshotPolicyId\":\"dtkcnqxwbpokulp\"}}") + VolumePatchPropertiesDataProtection model = BinaryData.fromString("{\"snapshot\":{\"snapshotPolicyId\":\"d\"}}") .toObject(VolumePatchPropertiesDataProtection.class); - Assertions.assertEquals("rkwofyyvoqa", model.backup().backupPolicyId()); - Assertions.assertEquals(true, model.backup().policyEnforced()); - Assertions.assertEquals(true, model.backup().backupEnabled()); - Assertions.assertEquals("btgiwbwoenwas", model.backup().backupVaultId()); - Assertions.assertEquals("dtkcnqxwbpokulp", model.snapshot().snapshotPolicyId()); + Assertions.assertEquals("d", model.snapshot().snapshotPolicyId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { VolumePatchPropertiesDataProtection model = new VolumePatchPropertiesDataProtection() - .withBackup(new VolumeBackupProperties().withBackupPolicyId("rkwofyyvoqa").withPolicyEnforced(true) - .withBackupEnabled(true).withBackupVaultId("btgiwbwoenwas")) - .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("dtkcnqxwbpokulp")); + .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("d")); model = BinaryData.fromObject(model).toObject(VolumePatchPropertiesDataProtection.class); - Assertions.assertEquals("rkwofyyvoqa", model.backup().backupPolicyId()); - Assertions.assertEquals(true, model.backup().policyEnforced()); - Assertions.assertEquals(true, model.backup().backupEnabled()); - Assertions.assertEquals("btgiwbwoenwas", model.backup().backupVaultId()); - Assertions.assertEquals("dtkcnqxwbpokulp", model.snapshot().snapshotPolicyId()); + Assertions.assertEquals("d", model.snapshot().snapshotPolicyId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesExportPolicyTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesExportPolicyTests.java index 2373ec7129ab6..7aec6206598e9 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesExportPolicyTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesExportPolicyTests.java @@ -15,58 +15,58 @@ public final class VolumePatchPropertiesExportPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumePatchPropertiesExportPolicy model = BinaryData.fromString( - "{\"rules\":[{\"ruleIndex\":369562825,\"unixReadOnly\":true,\"unixReadWrite\":true,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":false,\"cifs\":false,\"nfsv3\":false,\"nfsv41\":false,\"allowedClients\":\"uxvypomgkopkwh\",\"hasRootAccess\":false,\"chownMode\":\"Restricted\"},{\"ruleIndex\":996289807,\"unixReadOnly\":false,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"elxprglyatddck\",\"hasRootAccess\":true,\"chownMode\":\"Unrestricted\"},{\"ruleIndex\":41792863,\"unixReadOnly\":false,\"unixReadWrite\":true,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":false,\"nfsv41\":false,\"allowedClients\":\"bmrqyibahwfl\",\"hasRootAccess\":true,\"chownMode\":\"Unrestricted\"}]}") + "{\"rules\":[{\"ruleIndex\":1757000388,\"unixReadOnly\":true,\"unixReadWrite\":true,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"naenqpehindo\",\"hasRootAccess\":true,\"chownMode\":\"Restricted\"},{\"ruleIndex\":1448466618,\"unixReadOnly\":false,\"unixReadWrite\":true,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":false,\"nfsv41\":true,\"allowedClients\":\"lkthu\",\"hasRootAccess\":false,\"chownMode\":\"Restricted\"},{\"ruleIndex\":1315398087,\"unixReadOnly\":false,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"fmmdnbbg\",\"hasRootAccess\":true,\"chownMode\":\"Unrestricted\"}]}") .toObject(VolumePatchPropertiesExportPolicy.class); - Assertions.assertEquals(369562825, model.rules().get(0).ruleIndex()); + Assertions.assertEquals(1757000388, model.rules().get(0).ruleIndex()); Assertions.assertEquals(true, model.rules().get(0).unixReadOnly()); Assertions.assertEquals(true, model.rules().get(0).unixReadWrite()); Assertions.assertEquals(true, model.rules().get(0).kerberos5ReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5ReadWrite()); - Assertions.assertEquals(true, model.rules().get(0).kerberos5IReadOnly()); - Assertions.assertEquals(true, model.rules().get(0).kerberos5IReadWrite()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadWrite()); - Assertions.assertEquals(false, model.rules().get(0).cifs()); - Assertions.assertEquals(false, model.rules().get(0).nfsv3()); - Assertions.assertEquals(false, model.rules().get(0).nfsv41()); - Assertions.assertEquals("uxvypomgkopkwh", model.rules().get(0).allowedClients()); - Assertions.assertEquals(false, model.rules().get(0).hasRootAccess()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5ReadWrite()); + Assertions.assertEquals(false, model.rules().get(0).kerberos5IReadOnly()); + Assertions.assertEquals(false, model.rules().get(0).kerberos5IReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadOnly()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).cifs()); + Assertions.assertEquals(true, model.rules().get(0).nfsv3()); + Assertions.assertEquals(true, model.rules().get(0).nfsv41()); + Assertions.assertEquals("naenqpehindo", model.rules().get(0).allowedClients()); + Assertions.assertEquals(true, model.rules().get(0).hasRootAccess()); Assertions.assertEquals(ChownMode.RESTRICTED, model.rules().get(0).chownMode()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { VolumePatchPropertiesExportPolicy model = new VolumePatchPropertiesExportPolicy().withRules(Arrays.asList( - new ExportPolicyRule().withRuleIndex(369562825).withUnixReadOnly(true).withUnixReadWrite(true) - .withKerberos5ReadOnly(true).withKerberos5ReadWrite(false).withKerberos5IReadOnly(true) - .withKerberos5IReadWrite(true).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false) - .withCifs(false).withNfsv3(false).withNfsv41(false).withAllowedClients("uxvypomgkopkwh") - .withHasRootAccess(false).withChownMode(ChownMode.RESTRICTED), - new ExportPolicyRule().withRuleIndex(996289807).withUnixReadOnly(false).withUnixReadWrite(false) - .withKerberos5ReadOnly(true).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false) + new ExportPolicyRule().withRuleIndex(1757000388).withUnixReadOnly(true).withUnixReadWrite(true) + .withKerberos5ReadOnly(true).withKerberos5ReadWrite(true).withKerberos5IReadOnly(false) .withKerberos5IReadWrite(false).withKerberos5PReadOnly(true).withKerberos5PReadWrite(true) - .withCifs(false).withNfsv3(true).withNfsv41(true).withAllowedClients("elxprglyatddck") - .withHasRootAccess(true).withChownMode(ChownMode.UNRESTRICTED), - new ExportPolicyRule().withRuleIndex(41792863).withUnixReadOnly(false).withUnixReadWrite(true) - .withKerberos5ReadOnly(true).withKerberos5ReadWrite(false).withKerberos5IReadOnly(false) - .withKerberos5IReadWrite(true).withKerberos5PReadOnly(true).withKerberos5PReadWrite(true).withCifs(true) - .withNfsv3(false).withNfsv41(false).withAllowedClients("bmrqyibahwfl").withHasRootAccess(true) + .withCifs(true).withNfsv3(true).withNfsv41(true).withAllowedClients("naenqpehindo") + .withHasRootAccess(true).withChownMode(ChownMode.RESTRICTED), + new ExportPolicyRule().withRuleIndex(1448466618).withUnixReadOnly(false).withUnixReadWrite(true) + .withKerberos5ReadOnly(false).withKerberos5ReadWrite(true).withKerberos5IReadOnly(true) + .withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(true) + .withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("lkthu").withHasRootAccess(false) + .withChownMode(ChownMode.RESTRICTED), + new ExportPolicyRule().withRuleIndex(1315398087).withUnixReadOnly(false).withUnixReadWrite(false) + .withKerberos5ReadOnly(true).withKerberos5ReadWrite(true).withKerberos5IReadOnly(true) + .withKerberos5IReadWrite(false).withKerberos5PReadOnly(true).withKerberos5PReadWrite(true) + .withCifs(true).withNfsv3(true).withNfsv41(true).withAllowedClients("fmmdnbbg").withHasRootAccess(true) .withChownMode(ChownMode.UNRESTRICTED))); model = BinaryData.fromObject(model).toObject(VolumePatchPropertiesExportPolicy.class); - Assertions.assertEquals(369562825, model.rules().get(0).ruleIndex()); + Assertions.assertEquals(1757000388, model.rules().get(0).ruleIndex()); Assertions.assertEquals(true, model.rules().get(0).unixReadOnly()); Assertions.assertEquals(true, model.rules().get(0).unixReadWrite()); Assertions.assertEquals(true, model.rules().get(0).kerberos5ReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5ReadWrite()); - Assertions.assertEquals(true, model.rules().get(0).kerberos5IReadOnly()); - Assertions.assertEquals(true, model.rules().get(0).kerberos5IReadWrite()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadWrite()); - Assertions.assertEquals(false, model.rules().get(0).cifs()); - Assertions.assertEquals(false, model.rules().get(0).nfsv3()); - Assertions.assertEquals(false, model.rules().get(0).nfsv41()); - Assertions.assertEquals("uxvypomgkopkwh", model.rules().get(0).allowedClients()); - Assertions.assertEquals(false, model.rules().get(0).hasRootAccess()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5ReadWrite()); + Assertions.assertEquals(false, model.rules().get(0).kerberos5IReadOnly()); + Assertions.assertEquals(false, model.rules().get(0).kerberos5IReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadOnly()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).cifs()); + Assertions.assertEquals(true, model.rules().get(0).nfsv3()); + Assertions.assertEquals(true, model.rules().get(0).nfsv41()); + Assertions.assertEquals("naenqpehindo", model.rules().get(0).allowedClients()); + Assertions.assertEquals(true, model.rules().get(0).hasRootAccess()); Assertions.assertEquals(ChownMode.RESTRICTED, model.rules().get(0).chownMode()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesTests.java index f98bdb8574800..0f977f7ba02d1 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchPropertiesTests.java @@ -12,7 +12,6 @@ import com.azure.resourcemanager.netapp.models.ServiceLevel; import com.azure.resourcemanager.netapp.models.SmbAccessBasedEnumeration; import com.azure.resourcemanager.netapp.models.SmbNonBrowsable; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesDataProtection; import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesExportPolicy; import com.azure.resourcemanager.netapp.models.VolumeSnapshotProperties; @@ -23,96 +22,86 @@ public final class VolumePatchPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumePatchProperties model = BinaryData.fromString( - "{\"serviceLevel\":\"Standard\",\"usageThreshold\":5262686588919430594,\"exportPolicy\":{\"rules\":[{\"ruleIndex\":1635087277,\"unixReadOnly\":false,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"dldwmgxc\",\"hasRootAccess\":true,\"chownMode\":\"Restricted\"}]},\"throughputMibps\":24.985813,\"dataProtection\":{\"backup\":{\"backupPolicyId\":\"oegrpkhjwn\",\"policyEnforced\":false,\"backupEnabled\":true,\"backupVaultId\":\"i\"},\"snapshot\":{\"snapshotPolicyId\":\"ggkzzlvmbmpa\"}},\"isDefaultQuotaEnabled\":true,\"defaultUserQuotaInKiBs\":2096694396602012626,\"defaultGroupQuotaInKiBs\":3624195401715068445,\"unixPermissions\":\"wsbpfvm\",\"coolAccess\":false,\"coolnessPeriod\":1337725410,\"coolAccessRetrievalPolicy\":\"OnRead\",\"snapshotDirectoryVisible\":false,\"smbAccessBasedEnumeration\":\"Disabled\",\"smbNonBrowsable\":\"Disabled\"}") + "{\"serviceLevel\":\"Ultra\",\"usageThreshold\":6774114395187736753,\"exportPolicy\":{\"rules\":[{\"ruleIndex\":309197108,\"unixReadOnly\":true,\"unixReadWrite\":false,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":false,\"nfsv41\":true,\"allowedClients\":\"sqfsubcgjbirxb\",\"hasRootAccess\":false,\"chownMode\":\"Restricted\"}]},\"throughputMibps\":16.671253,\"dataProtection\":{\"snapshot\":{\"snapshotPolicyId\":\"twss\"}},\"isDefaultQuotaEnabled\":true,\"defaultUserQuotaInKiBs\":6673655197593346070,\"defaultGroupQuotaInKiBs\":6942246112545525154,\"unixPermissions\":\"xilzznf\",\"coolAccess\":true,\"coolnessPeriod\":1331345486,\"coolAccessRetrievalPolicy\":\"OnRead\",\"snapshotDirectoryVisible\":true,\"smbAccessBasedEnumeration\":\"Disabled\",\"smbNonBrowsable\":\"Enabled\"}") .toObject(VolumePatchProperties.class); - Assertions.assertEquals(ServiceLevel.STANDARD, model.serviceLevel()); - Assertions.assertEquals(5262686588919430594L, model.usageThreshold()); - Assertions.assertEquals(1635087277, model.exportPolicy().rules().get(0).ruleIndex()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadOnly()); + Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); + Assertions.assertEquals(6774114395187736753L, model.usageThreshold()); + Assertions.assertEquals(309197108, model.exportPolicy().rules().get(0).ruleIndex()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).unixReadOnly()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadWrite()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5ReadWrite()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadWrite()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadOnly()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5IReadWrite()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadWrite()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadOnly()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadWrite()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).cifs()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv3()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).nfsv3()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv41()); - Assertions.assertEquals("dldwmgxc", model.exportPolicy().rules().get(0).allowedClients()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).hasRootAccess()); + Assertions.assertEquals("sqfsubcgjbirxb", model.exportPolicy().rules().get(0).allowedClients()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).hasRootAccess()); Assertions.assertEquals(ChownMode.RESTRICTED, model.exportPolicy().rules().get(0).chownMode()); - Assertions.assertEquals(24.985813F, model.throughputMibps()); - Assertions.assertEquals("oegrpkhjwn", model.dataProtection().backup().backupPolicyId()); - Assertions.assertEquals(false, model.dataProtection().backup().policyEnforced()); - Assertions.assertEquals(true, model.dataProtection().backup().backupEnabled()); - Assertions.assertEquals("i", model.dataProtection().backup().backupVaultId()); - Assertions.assertEquals("ggkzzlvmbmpa", model.dataProtection().snapshot().snapshotPolicyId()); + Assertions.assertEquals(16.671253F, model.throughputMibps()); + Assertions.assertEquals("twss", model.dataProtection().snapshot().snapshotPolicyId()); Assertions.assertEquals(true, model.isDefaultQuotaEnabled()); - Assertions.assertEquals(2096694396602012626L, model.defaultUserQuotaInKiBs()); - Assertions.assertEquals(3624195401715068445L, model.defaultGroupQuotaInKiBs()); - Assertions.assertEquals("wsbpfvm", model.unixPermissions()); - Assertions.assertEquals(false, model.coolAccess()); - Assertions.assertEquals(1337725410, model.coolnessPeriod()); + Assertions.assertEquals(6673655197593346070L, model.defaultUserQuotaInKiBs()); + Assertions.assertEquals(6942246112545525154L, model.defaultGroupQuotaInKiBs()); + Assertions.assertEquals("xilzznf", model.unixPermissions()); + Assertions.assertEquals(true, model.coolAccess()); + Assertions.assertEquals(1331345486, model.coolnessPeriod()); Assertions.assertEquals(CoolAccessRetrievalPolicy.ON_READ, model.coolAccessRetrievalPolicy()); - Assertions.assertEquals(false, model.snapshotDirectoryVisible()); + Assertions.assertEquals(true, model.snapshotDirectoryVisible()); Assertions.assertEquals(SmbAccessBasedEnumeration.DISABLED, model.smbAccessBasedEnumeration()); - Assertions.assertEquals(SmbNonBrowsable.DISABLED, model.smbNonBrowsable()); + Assertions.assertEquals(SmbNonBrowsable.ENABLED, model.smbNonBrowsable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumePatchProperties model = new VolumePatchProperties().withServiceLevel(ServiceLevel.STANDARD) - .withUsageThreshold(5262686588919430594L) - .withExportPolicy(new VolumePatchPropertiesExportPolicy() - .withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(1635087277).withUnixReadOnly(false) - .withUnixReadWrite(false).withKerberos5ReadOnly(true).withKerberos5ReadWrite(false) - .withKerberos5IReadOnly(false).withKerberos5IReadWrite(true).withKerberos5PReadOnly(true) - .withKerberos5PReadWrite(true).withCifs(false).withNfsv3(true).withNfsv41(true) - .withAllowedClients("dldwmgxc").withHasRootAccess(true).withChownMode(ChownMode.RESTRICTED)))) - .withThroughputMibps(24.985813F) + VolumePatchProperties model = new VolumePatchProperties().withServiceLevel(ServiceLevel.ULTRA) + .withUsageThreshold(6774114395187736753L) + .withExportPolicy(new VolumePatchPropertiesExportPolicy().withRules(Arrays + .asList(new ExportPolicyRule().withRuleIndex(309197108).withUnixReadOnly(true).withUnixReadWrite(false) + .withKerberos5ReadOnly(false).withKerberos5ReadWrite(true).withKerberos5IReadOnly(false) + .withKerberos5IReadWrite(false).withKerberos5PReadOnly(true).withKerberos5PReadWrite(true) + .withCifs(false).withNfsv3(false).withNfsv41(true).withAllowedClients("sqfsubcgjbirxb") + .withHasRootAccess(false).withChownMode(ChownMode.RESTRICTED)))) + .withThroughputMibps(16.671253F) .withDataProtection(new VolumePatchPropertiesDataProtection() - .withBackup(new VolumeBackupProperties().withBackupPolicyId("oegrpkhjwn").withPolicyEnforced(false) - .withBackupEnabled(true).withBackupVaultId("i")) - .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("ggkzzlvmbmpa"))) - .withIsDefaultQuotaEnabled(true).withDefaultUserQuotaInKiBs(2096694396602012626L) - .withDefaultGroupQuotaInKiBs(3624195401715068445L).withUnixPermissions("wsbpfvm").withCoolAccess(false) - .withCoolnessPeriod(1337725410).withCoolAccessRetrievalPolicy(CoolAccessRetrievalPolicy.ON_READ) - .withSnapshotDirectoryVisible(false).withSmbAccessBasedEnumeration(SmbAccessBasedEnumeration.DISABLED) - .withSmbNonBrowsable(SmbNonBrowsable.DISABLED); + .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("twss"))) + .withIsDefaultQuotaEnabled(true).withDefaultUserQuotaInKiBs(6673655197593346070L) + .withDefaultGroupQuotaInKiBs(6942246112545525154L).withUnixPermissions("xilzznf").withCoolAccess(true) + .withCoolnessPeriod(1331345486).withCoolAccessRetrievalPolicy(CoolAccessRetrievalPolicy.ON_READ) + .withSnapshotDirectoryVisible(true).withSmbAccessBasedEnumeration(SmbAccessBasedEnumeration.DISABLED) + .withSmbNonBrowsable(SmbNonBrowsable.ENABLED); model = BinaryData.fromObject(model).toObject(VolumePatchProperties.class); - Assertions.assertEquals(ServiceLevel.STANDARD, model.serviceLevel()); - Assertions.assertEquals(5262686588919430594L, model.usageThreshold()); - Assertions.assertEquals(1635087277, model.exportPolicy().rules().get(0).ruleIndex()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadOnly()); + Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); + Assertions.assertEquals(6774114395187736753L, model.usageThreshold()); + Assertions.assertEquals(309197108, model.exportPolicy().rules().get(0).ruleIndex()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).unixReadOnly()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadWrite()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5ReadWrite()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadWrite()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadOnly()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5IReadWrite()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadWrite()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadOnly()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadWrite()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).cifs()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv3()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).nfsv3()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv41()); - Assertions.assertEquals("dldwmgxc", model.exportPolicy().rules().get(0).allowedClients()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).hasRootAccess()); + Assertions.assertEquals("sqfsubcgjbirxb", model.exportPolicy().rules().get(0).allowedClients()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).hasRootAccess()); Assertions.assertEquals(ChownMode.RESTRICTED, model.exportPolicy().rules().get(0).chownMode()); - Assertions.assertEquals(24.985813F, model.throughputMibps()); - Assertions.assertEquals("oegrpkhjwn", model.dataProtection().backup().backupPolicyId()); - Assertions.assertEquals(false, model.dataProtection().backup().policyEnforced()); - Assertions.assertEquals(true, model.dataProtection().backup().backupEnabled()); - Assertions.assertEquals("i", model.dataProtection().backup().backupVaultId()); - Assertions.assertEquals("ggkzzlvmbmpa", model.dataProtection().snapshot().snapshotPolicyId()); + Assertions.assertEquals(16.671253F, model.throughputMibps()); + Assertions.assertEquals("twss", model.dataProtection().snapshot().snapshotPolicyId()); Assertions.assertEquals(true, model.isDefaultQuotaEnabled()); - Assertions.assertEquals(2096694396602012626L, model.defaultUserQuotaInKiBs()); - Assertions.assertEquals(3624195401715068445L, model.defaultGroupQuotaInKiBs()); - Assertions.assertEquals("wsbpfvm", model.unixPermissions()); - Assertions.assertEquals(false, model.coolAccess()); - Assertions.assertEquals(1337725410, model.coolnessPeriod()); + Assertions.assertEquals(6673655197593346070L, model.defaultUserQuotaInKiBs()); + Assertions.assertEquals(6942246112545525154L, model.defaultGroupQuotaInKiBs()); + Assertions.assertEquals("xilzznf", model.unixPermissions()); + Assertions.assertEquals(true, model.coolAccess()); + Assertions.assertEquals(1331345486, model.coolnessPeriod()); Assertions.assertEquals(CoolAccessRetrievalPolicy.ON_READ, model.coolAccessRetrievalPolicy()); - Assertions.assertEquals(false, model.snapshotDirectoryVisible()); + Assertions.assertEquals(true, model.snapshotDirectoryVisible()); Assertions.assertEquals(SmbAccessBasedEnumeration.DISABLED, model.smbAccessBasedEnumeration()); - Assertions.assertEquals(SmbNonBrowsable.DISABLED, model.smbNonBrowsable()); + Assertions.assertEquals(SmbNonBrowsable.ENABLED, model.smbNonBrowsable()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchTests.java index dc8fe9d23e15b..ff08c9ea417b0 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePatchTests.java @@ -11,7 +11,6 @@ import com.azure.resourcemanager.netapp.models.ServiceLevel; import com.azure.resourcemanager.netapp.models.SmbAccessBasedEnumeration; import com.azure.resourcemanager.netapp.models.SmbNonBrowsable; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; import com.azure.resourcemanager.netapp.models.VolumePatch; import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesDataProtection; import com.azure.resourcemanager.netapp.models.VolumePatchPropertiesExportPolicy; @@ -25,110 +24,106 @@ public final class VolumePatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumePatch model = BinaryData.fromString( - "{\"properties\":{\"serviceLevel\":\"Premium\",\"usageThreshold\":1361117454482820932,\"exportPolicy\":{\"rules\":[{\"ruleIndex\":1245356368,\"unixReadOnly\":false,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"dlkzgxhuri\",\"hasRootAccess\":true,\"chownMode\":\"Restricted\"},{\"ruleIndex\":1647151619,\"unixReadOnly\":true,\"unixReadWrite\":true,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":false,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"evseotgqrlltmuwl\",\"hasRootAccess\":true,\"chownMode\":\"Unrestricted\"},{\"ruleIndex\":1035303462,\"unixReadOnly\":true,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":true,\"nfsv41\":false,\"allowedClients\":\"pxebmnzbt\",\"hasRootAccess\":true,\"chownMode\":\"Restricted\"}]},\"throughputMibps\":61.801933,\"dataProtection\":{\"backup\":{\"backupPolicyId\":\"hdneuelfph\",\"policyEnforced\":true,\"backupEnabled\":false,\"backupVaultId\":\"zfikd\"},\"snapshot\":{\"snapshotPolicyId\":\"quuvxzxcl\"}},\"isDefaultQuotaEnabled\":true,\"defaultUserQuotaInKiBs\":6834800268886491051,\"defaultGroupQuotaInKiBs\":758398116524411728,\"unixPermissions\":\"sg\",\"coolAccess\":true,\"coolnessPeriod\":1373872192,\"coolAccessRetrievalPolicy\":\"OnRead\",\"snapshotDirectoryVisible\":false,\"smbAccessBasedEnumeration\":\"Enabled\",\"smbNonBrowsable\":\"Disabled\"},\"location\":\"aljutiiswac\",\"tags\":{\"pnppfuf\":\"dkzzewkfvhqcrail\"},\"id\":\"rwdmhdlxyjrxsa\",\"name\":\"afcnih\",\"type\":\"wqapnedgfbcvk\"}") + "{\"properties\":{\"serviceLevel\":\"Ultra\",\"usageThreshold\":2245300278286998069,\"exportPolicy\":{\"rules\":[{\"ruleIndex\":1252528512,\"unixReadOnly\":true,\"unixReadWrite\":false,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":false,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":false,\"nfsv41\":false,\"allowedClients\":\"hcaalnjix\",\"hasRootAccess\":false,\"chownMode\":\"Unrestricted\"},{\"ruleIndex\":939698337,\"unixReadOnly\":true,\"unixReadWrite\":true,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":false,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"xhnrztfolhb\",\"hasRootAccess\":true,\"chownMode\":\"Unrestricted\"},{\"ruleIndex\":1904300707,\"unixReadOnly\":false,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":true,\"cifs\":false,\"nfsv3\":false,\"nfsv41\":false,\"allowedClients\":\"vpgylgqgitxmed\",\"hasRootAccess\":false,\"chownMode\":\"Unrestricted\"},{\"ruleIndex\":73392328,\"unixReadOnly\":true,\"unixReadWrite\":false,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":false,\"allowedClients\":\"eoellwptfdygp\",\"hasRootAccess\":false,\"chownMode\":\"Restricted\"}]},\"throughputMibps\":82.87768,\"dataProtection\":{\"snapshot\":{\"snapshotPolicyId\":\"fqrhhuaopppc\"}},\"isDefaultQuotaEnabled\":true,\"defaultUserQuotaInKiBs\":2031358735864920909,\"defaultGroupQuotaInKiBs\":1205860539886844150,\"unixPermissions\":\"zxctobgb\",\"coolAccess\":false,\"coolnessPeriod\":1356840761,\"coolAccessRetrievalPolicy\":\"Default\",\"snapshotDirectoryVisible\":true,\"smbAccessBasedEnumeration\":\"Disabled\",\"smbNonBrowsable\":\"Disabled\"},\"location\":\"cfbu\",\"tags\":{\"xxjyngudivkrtsw\":\"fqjhhkxbpvjymj\",\"vjfdx\":\"xqzvszjfa\",\"qtdo\":\"ivetvtcq\"},\"id\":\"mcbxvwvxysl\",\"name\":\"bhsfxob\",\"type\":\"ytkblmpew\"}") .toObject(VolumePatch.class); - Assertions.assertEquals("aljutiiswac", model.location()); - Assertions.assertEquals("dkzzewkfvhqcrail", model.tags().get("pnppfuf")); - Assertions.assertEquals(ServiceLevel.PREMIUM, model.serviceLevel()); - Assertions.assertEquals(1361117454482820932L, model.usageThreshold()); - Assertions.assertEquals(1245356368, model.exportPolicy().rules().get(0).ruleIndex()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadOnly()); + Assertions.assertEquals("cfbu", model.location()); + Assertions.assertEquals("fqjhhkxbpvjymj", model.tags().get("xxjyngudivkrtsw")); + Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); + Assertions.assertEquals(2245300278286998069L, model.usageThreshold()); + Assertions.assertEquals(1252528512, model.exportPolicy().rules().get(0).ruleIndex()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).unixReadOnly()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadWrite()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadWrite()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadOnly()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadWrite()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5PReadOnly()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadOnly()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadWrite()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).cifs()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv3()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv41()); - Assertions.assertEquals("dlkzgxhuri", model.exportPolicy().rules().get(0).allowedClients()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).hasRootAccess()); - Assertions.assertEquals(ChownMode.RESTRICTED, model.exportPolicy().rules().get(0).chownMode()); - Assertions.assertEquals(61.801933F, model.throughputMibps()); - Assertions.assertEquals("hdneuelfph", model.dataProtection().backup().backupPolicyId()); - Assertions.assertEquals(true, model.dataProtection().backup().policyEnforced()); - Assertions.assertEquals(false, model.dataProtection().backup().backupEnabled()); - Assertions.assertEquals("zfikd", model.dataProtection().backup().backupVaultId()); - Assertions.assertEquals("quuvxzxcl", model.dataProtection().snapshot().snapshotPolicyId()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).nfsv3()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).nfsv41()); + Assertions.assertEquals("hcaalnjix", model.exportPolicy().rules().get(0).allowedClients()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).hasRootAccess()); + Assertions.assertEquals(ChownMode.UNRESTRICTED, model.exportPolicy().rules().get(0).chownMode()); + Assertions.assertEquals(82.87768F, model.throughputMibps()); + Assertions.assertEquals("fqrhhuaopppc", model.dataProtection().snapshot().snapshotPolicyId()); Assertions.assertEquals(true, model.isDefaultQuotaEnabled()); - Assertions.assertEquals(6834800268886491051L, model.defaultUserQuotaInKiBs()); - Assertions.assertEquals(758398116524411728L, model.defaultGroupQuotaInKiBs()); - Assertions.assertEquals("sg", model.unixPermissions()); - Assertions.assertEquals(true, model.coolAccess()); - Assertions.assertEquals(1373872192, model.coolnessPeriod()); - Assertions.assertEquals(CoolAccessRetrievalPolicy.ON_READ, model.coolAccessRetrievalPolicy()); - Assertions.assertEquals(false, model.snapshotDirectoryVisible()); - Assertions.assertEquals(SmbAccessBasedEnumeration.ENABLED, model.smbAccessBasedEnumeration()); + Assertions.assertEquals(2031358735864920909L, model.defaultUserQuotaInKiBs()); + Assertions.assertEquals(1205860539886844150L, model.defaultGroupQuotaInKiBs()); + Assertions.assertEquals("zxctobgb", model.unixPermissions()); + Assertions.assertEquals(false, model.coolAccess()); + Assertions.assertEquals(1356840761, model.coolnessPeriod()); + Assertions.assertEquals(CoolAccessRetrievalPolicy.DEFAULT, model.coolAccessRetrievalPolicy()); + Assertions.assertEquals(true, model.snapshotDirectoryVisible()); + Assertions.assertEquals(SmbAccessBasedEnumeration.DISABLED, model.smbAccessBasedEnumeration()); Assertions.assertEquals(SmbNonBrowsable.DISABLED, model.smbNonBrowsable()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumePatch model = new VolumePatch().withLocation("aljutiiswac").withTags(mapOf("pnppfuf", "dkzzewkfvhqcrail")) - .withServiceLevel(ServiceLevel.PREMIUM).withUsageThreshold(1361117454482820932L) + VolumePatch model = new VolumePatch().withLocation("cfbu") + .withTags(mapOf("xxjyngudivkrtsw", "fqjhhkxbpvjymj", "vjfdx", "xqzvszjfa", "qtdo", "ivetvtcq")) + .withServiceLevel(ServiceLevel.ULTRA).withUsageThreshold(2245300278286998069L) .withExportPolicy(new VolumePatchPropertiesExportPolicy().withRules(Arrays.asList( - new ExportPolicyRule().withRuleIndex(1245356368).withUnixReadOnly(false).withUnixReadWrite(false) + new ExportPolicyRule().withRuleIndex(1252528512).withUnixReadOnly(true).withUnixReadWrite(false) + .withKerberos5ReadOnly(false).withKerberos5ReadWrite(true).withKerberos5IReadOnly(false) + .withKerberos5IReadWrite(false).withKerberos5PReadOnly(true).withKerberos5PReadWrite(true) + .withCifs(true).withNfsv3(false).withNfsv41(false).withAllowedClients("hcaalnjix") + .withHasRootAccess(false).withChownMode(ChownMode.UNRESTRICTED), + new ExportPolicyRule().withRuleIndex(939698337).withUnixReadOnly(true).withUnixReadWrite(true) .withKerberos5ReadOnly(true).withKerberos5ReadWrite(true).withKerberos5IReadOnly(false) - .withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(true) - .withCifs(true).withNfsv3(true).withNfsv41(true).withAllowedClients("dlkzgxhuri") - .withHasRootAccess(true).withChownMode(ChownMode.RESTRICTED), - new ExportPolicyRule().withRuleIndex(1647151619).withUnixReadOnly(true).withUnixReadWrite(true) - .withKerberos5ReadOnly(true).withKerberos5ReadWrite(false).withKerberos5IReadOnly(true) - .withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false) - .withCifs(true).withNfsv3(true).withNfsv41(true).withAllowedClients("evseotgqrlltmuwl") + .withKerberos5IReadWrite(true).withKerberos5PReadOnly(true).withKerberos5PReadWrite(false) + .withCifs(true).withNfsv3(true).withNfsv41(true).withAllowedClients("xhnrztfolhb") .withHasRootAccess(true).withChownMode(ChownMode.UNRESTRICTED), - new ExportPolicyRule().withRuleIndex(1035303462).withUnixReadOnly(true).withUnixReadWrite(false) + new ExportPolicyRule().withRuleIndex(1904300707).withUnixReadOnly(false).withUnixReadWrite(false) .withKerberos5ReadOnly(true).withKerberos5ReadWrite(true).withKerberos5IReadOnly(true) - .withKerberos5IReadWrite(false).withKerberos5PReadOnly(false).withKerberos5PReadWrite(true) - .withCifs(false).withNfsv3(true).withNfsv41(false).withAllowedClients("pxebmnzbt") - .withHasRootAccess(true).withChownMode(ChownMode.RESTRICTED)))) - .withThroughputMibps(61.801933F) + .withKerberos5IReadWrite(true).withKerberos5PReadOnly(false).withKerberos5PReadWrite(true) + .withCifs(false).withNfsv3(false).withNfsv41(false).withAllowedClients("vpgylgqgitxmed") + .withHasRootAccess(false).withChownMode(ChownMode.UNRESTRICTED), + new ExportPolicyRule().withRuleIndex(73392328).withUnixReadOnly(true).withUnixReadWrite(false) + .withKerberos5ReadOnly(false).withKerberos5ReadWrite(true).withKerberos5IReadOnly(false) + .withKerberos5IReadWrite(true).withKerberos5PReadOnly(false).withKerberos5PReadWrite(true) + .withCifs(true).withNfsv3(true).withNfsv41(false).withAllowedClients("eoellwptfdygp") + .withHasRootAccess(false).withChownMode(ChownMode.RESTRICTED)))) + .withThroughputMibps(82.87768F) .withDataProtection(new VolumePatchPropertiesDataProtection() - .withBackup(new VolumeBackupProperties().withBackupPolicyId("hdneuelfph").withPolicyEnforced(true) - .withBackupEnabled(false).withBackupVaultId("zfikd")) - .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("quuvxzxcl"))) - .withIsDefaultQuotaEnabled(true).withDefaultUserQuotaInKiBs(6834800268886491051L) - .withDefaultGroupQuotaInKiBs(758398116524411728L).withUnixPermissions("sg").withCoolAccess(true) - .withCoolnessPeriod(1373872192).withCoolAccessRetrievalPolicy(CoolAccessRetrievalPolicy.ON_READ) - .withSnapshotDirectoryVisible(false).withSmbAccessBasedEnumeration(SmbAccessBasedEnumeration.ENABLED) + .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("fqrhhuaopppc"))) + .withIsDefaultQuotaEnabled(true).withDefaultUserQuotaInKiBs(2031358735864920909L) + .withDefaultGroupQuotaInKiBs(1205860539886844150L).withUnixPermissions("zxctobgb").withCoolAccess(false) + .withCoolnessPeriod(1356840761).withCoolAccessRetrievalPolicy(CoolAccessRetrievalPolicy.DEFAULT) + .withSnapshotDirectoryVisible(true).withSmbAccessBasedEnumeration(SmbAccessBasedEnumeration.DISABLED) .withSmbNonBrowsable(SmbNonBrowsable.DISABLED); model = BinaryData.fromObject(model).toObject(VolumePatch.class); - Assertions.assertEquals("aljutiiswac", model.location()); - Assertions.assertEquals("dkzzewkfvhqcrail", model.tags().get("pnppfuf")); - Assertions.assertEquals(ServiceLevel.PREMIUM, model.serviceLevel()); - Assertions.assertEquals(1361117454482820932L, model.usageThreshold()); - Assertions.assertEquals(1245356368, model.exportPolicy().rules().get(0).ruleIndex()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadOnly()); + Assertions.assertEquals("cfbu", model.location()); + Assertions.assertEquals("fqjhhkxbpvjymj", model.tags().get("xxjyngudivkrtsw")); + Assertions.assertEquals(ServiceLevel.ULTRA, model.serviceLevel()); + Assertions.assertEquals(2245300278286998069L, model.usageThreshold()); + Assertions.assertEquals(1252528512, model.exportPolicy().rules().get(0).ruleIndex()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).unixReadOnly()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).unixReadWrite()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5ReadOnly()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5ReadWrite()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadOnly()); Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5IReadWrite()); - Assertions.assertEquals(false, model.exportPolicy().rules().get(0).kerberos5PReadOnly()); + Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadOnly()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).kerberos5PReadWrite()); Assertions.assertEquals(true, model.exportPolicy().rules().get(0).cifs()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv3()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).nfsv41()); - Assertions.assertEquals("dlkzgxhuri", model.exportPolicy().rules().get(0).allowedClients()); - Assertions.assertEquals(true, model.exportPolicy().rules().get(0).hasRootAccess()); - Assertions.assertEquals(ChownMode.RESTRICTED, model.exportPolicy().rules().get(0).chownMode()); - Assertions.assertEquals(61.801933F, model.throughputMibps()); - Assertions.assertEquals("hdneuelfph", model.dataProtection().backup().backupPolicyId()); - Assertions.assertEquals(true, model.dataProtection().backup().policyEnforced()); - Assertions.assertEquals(false, model.dataProtection().backup().backupEnabled()); - Assertions.assertEquals("zfikd", model.dataProtection().backup().backupVaultId()); - Assertions.assertEquals("quuvxzxcl", model.dataProtection().snapshot().snapshotPolicyId()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).nfsv3()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).nfsv41()); + Assertions.assertEquals("hcaalnjix", model.exportPolicy().rules().get(0).allowedClients()); + Assertions.assertEquals(false, model.exportPolicy().rules().get(0).hasRootAccess()); + Assertions.assertEquals(ChownMode.UNRESTRICTED, model.exportPolicy().rules().get(0).chownMode()); + Assertions.assertEquals(82.87768F, model.throughputMibps()); + Assertions.assertEquals("fqrhhuaopppc", model.dataProtection().snapshot().snapshotPolicyId()); Assertions.assertEquals(true, model.isDefaultQuotaEnabled()); - Assertions.assertEquals(6834800268886491051L, model.defaultUserQuotaInKiBs()); - Assertions.assertEquals(758398116524411728L, model.defaultGroupQuotaInKiBs()); - Assertions.assertEquals("sg", model.unixPermissions()); - Assertions.assertEquals(true, model.coolAccess()); - Assertions.assertEquals(1373872192, model.coolnessPeriod()); - Assertions.assertEquals(CoolAccessRetrievalPolicy.ON_READ, model.coolAccessRetrievalPolicy()); - Assertions.assertEquals(false, model.snapshotDirectoryVisible()); - Assertions.assertEquals(SmbAccessBasedEnumeration.ENABLED, model.smbAccessBasedEnumeration()); + Assertions.assertEquals(2031358735864920909L, model.defaultUserQuotaInKiBs()); + Assertions.assertEquals(1205860539886844150L, model.defaultGroupQuotaInKiBs()); + Assertions.assertEquals("zxctobgb", model.unixPermissions()); + Assertions.assertEquals(false, model.coolAccess()); + Assertions.assertEquals(1356840761, model.coolnessPeriod()); + Assertions.assertEquals(CoolAccessRetrievalPolicy.DEFAULT, model.coolAccessRetrievalPolicy()); + Assertions.assertEquals(true, model.snapshotDirectoryVisible()); + Assertions.assertEquals(SmbAccessBasedEnumeration.DISABLED, model.smbAccessBasedEnumeration()); Assertions.assertEquals(SmbNonBrowsable.DISABLED, model.smbNonBrowsable()); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesDataProtectionTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesDataProtectionTests.java index 1aff05f33e2fd..350be53a19c1a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesDataProtectionTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesDataProtectionTests.java @@ -6,10 +6,8 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.netapp.models.EndpointType; -import com.azure.resourcemanager.netapp.models.RemotePath; import com.azure.resourcemanager.netapp.models.ReplicationObject; import com.azure.resourcemanager.netapp.models.ReplicationSchedule; -import com.azure.resourcemanager.netapp.models.VolumeBackupProperties; import com.azure.resourcemanager.netapp.models.VolumePropertiesDataProtection; import com.azure.resourcemanager.netapp.models.VolumeRelocationProperties; import com.azure.resourcemanager.netapp.models.VolumeSnapshotProperties; @@ -19,49 +17,30 @@ public final class VolumePropertiesDataProtectionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumePropertiesDataProtection model = BinaryData.fromString( - "{\"backup\":{\"backupPolicyId\":\"duhavhqlkt\",\"policyEnforced\":true,\"backupEnabled\":false,\"backupVaultId\":\"lbg\"},\"replication\":{\"replicationId\":\"uie\",\"endpointType\":\"dst\",\"replicationSchedule\":\"_10minutely\",\"remoteVolumeResourceId\":\"y\",\"remotePath\":{\"externalHostName\":\"aolps\",\"serverName\":\"lqlfm\",\"volumeName\":\"dnbbglzps\"},\"remoteVolumeRegion\":\"ydmcwyhzdxssa\"},\"snapshot\":{\"snapshotPolicyId\":\"mnvdfzn\"},\"volumeRelocation\":{\"relocationRequested\":false,\"readyToBeFinalized\":false}}") + "{\"replication\":{\"replicationId\":\"vgqzcjrvxd\",\"endpointType\":\"dst\",\"replicationSchedule\":\"_10minutely\",\"remoteVolumeResourceId\":\"lxkvu\",\"remoteVolumeRegion\":\"hzovawjvzunlut\"},\"snapshot\":{\"snapshotPolicyId\":\"prnxipeil\"},\"volumeRelocation\":{\"relocationRequested\":true,\"readyToBeFinalized\":false}}") .toObject(VolumePropertiesDataProtection.class); - Assertions.assertEquals("duhavhqlkt", model.backup().backupPolicyId()); - Assertions.assertEquals(true, model.backup().policyEnforced()); - Assertions.assertEquals(false, model.backup().backupEnabled()); - Assertions.assertEquals("lbg", model.backup().backupVaultId()); Assertions.assertEquals(EndpointType.DST, model.replication().endpointType()); Assertions.assertEquals(ReplicationSchedule.ONE_ZEROMINUTELY, model.replication().replicationSchedule()); - Assertions.assertEquals("y", model.replication().remoteVolumeResourceId()); - Assertions.assertEquals("aolps", model.replication().remotePath().externalHostname()); - Assertions.assertEquals("lqlfm", model.replication().remotePath().serverName()); - Assertions.assertEquals("dnbbglzps", model.replication().remotePath().volumeName()); - Assertions.assertEquals("ydmcwyhzdxssa", model.replication().remoteVolumeRegion()); - Assertions.assertEquals("mnvdfzn", model.snapshot().snapshotPolicyId()); - Assertions.assertEquals(false, model.volumeRelocation().relocationRequested()); + Assertions.assertEquals("lxkvu", model.replication().remoteVolumeResourceId()); + Assertions.assertEquals("hzovawjvzunlut", model.replication().remoteVolumeRegion()); + Assertions.assertEquals("prnxipeil", model.snapshot().snapshotPolicyId()); + Assertions.assertEquals(true, model.volumeRelocation().relocationRequested()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumePropertiesDataProtection model - = new VolumePropertiesDataProtection() - .withBackup(new VolumeBackupProperties().withBackupPolicyId("duhavhqlkt").withPolicyEnforced(true) - .withBackupEnabled(false).withBackupVaultId("lbg")) - .withReplication(new ReplicationObject().withEndpointType(EndpointType.DST) - .withReplicationSchedule(ReplicationSchedule.ONE_ZEROMINUTELY).withRemoteVolumeResourceId("y") - .withRemotePath(new RemotePath().withExternalHostname("aolps").withServerName("lqlfm") - .withVolumeName("dnbbglzps")) - .withRemoteVolumeRegion("ydmcwyhzdxssa")) - .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("mnvdfzn")) - .withVolumeRelocation(new VolumeRelocationProperties().withRelocationRequested(false)); + VolumePropertiesDataProtection model = new VolumePropertiesDataProtection() + .withReplication(new ReplicationObject().withEndpointType(EndpointType.DST) + .withReplicationSchedule(ReplicationSchedule.ONE_ZEROMINUTELY).withRemoteVolumeResourceId("lxkvu") + .withRemoteVolumeRegion("hzovawjvzunlut")) + .withSnapshot(new VolumeSnapshotProperties().withSnapshotPolicyId("prnxipeil")) + .withVolumeRelocation(new VolumeRelocationProperties().withRelocationRequested(true)); model = BinaryData.fromObject(model).toObject(VolumePropertiesDataProtection.class); - Assertions.assertEquals("duhavhqlkt", model.backup().backupPolicyId()); - Assertions.assertEquals(true, model.backup().policyEnforced()); - Assertions.assertEquals(false, model.backup().backupEnabled()); - Assertions.assertEquals("lbg", model.backup().backupVaultId()); Assertions.assertEquals(EndpointType.DST, model.replication().endpointType()); Assertions.assertEquals(ReplicationSchedule.ONE_ZEROMINUTELY, model.replication().replicationSchedule()); - Assertions.assertEquals("y", model.replication().remoteVolumeResourceId()); - Assertions.assertEquals("aolps", model.replication().remotePath().externalHostname()); - Assertions.assertEquals("lqlfm", model.replication().remotePath().serverName()); - Assertions.assertEquals("dnbbglzps", model.replication().remotePath().volumeName()); - Assertions.assertEquals("ydmcwyhzdxssa", model.replication().remoteVolumeRegion()); - Assertions.assertEquals("mnvdfzn", model.snapshot().snapshotPolicyId()); - Assertions.assertEquals(false, model.volumeRelocation().relocationRequested()); + Assertions.assertEquals("lxkvu", model.replication().remoteVolumeResourceId()); + Assertions.assertEquals("hzovawjvzunlut", model.replication().remoteVolumeRegion()); + Assertions.assertEquals("prnxipeil", model.snapshot().snapshotPolicyId()); + Assertions.assertEquals(true, model.volumeRelocation().relocationRequested()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesExportPolicyTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesExportPolicyTests.java index 3aef08b73eeca..0fabbbe24966a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesExportPolicyTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumePropertiesExportPolicyTests.java @@ -15,48 +15,53 @@ public final class VolumePropertiesExportPolicyTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumePropertiesExportPolicy model = BinaryData.fromString( - "{\"rules\":[{\"ruleIndex\":561759175,\"unixReadOnly\":false,\"unixReadWrite\":true,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":false,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":false,\"allowedClients\":\"jzbexilzznfq\",\"hasRootAccess\":false,\"chownMode\":\"Restricted\"}]}") + "{\"rules\":[{\"ruleIndex\":1149825898,\"unixReadOnly\":false,\"unixReadWrite\":true,\"kerberos5ReadOnly\":false,\"kerberos5ReadWrite\":true,\"kerberos5iReadOnly\":false,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":true,\"kerberos5pReadWrite\":true,\"cifs\":true,\"nfsv3\":true,\"nfsv41\":true,\"allowedClients\":\"jriplrbpbewtghf\",\"hasRootAccess\":true,\"chownMode\":\"Restricted\"},{\"ruleIndex\":1553365340,\"unixReadOnly\":true,\"unixReadWrite\":false,\"kerberos5ReadOnly\":true,\"kerberos5ReadWrite\":false,\"kerberos5iReadOnly\":true,\"kerberos5iReadWrite\":true,\"kerberos5pReadOnly\":false,\"kerberos5pReadWrite\":false,\"cifs\":true,\"nfsv3\":false,\"nfsv41\":true,\"allowedClients\":\"loayqcgw\",\"hasRootAccess\":false,\"chownMode\":\"Unrestricted\"}]}") .toObject(VolumePropertiesExportPolicy.class); - Assertions.assertEquals(561759175, model.rules().get(0).ruleIndex()); + Assertions.assertEquals(1149825898, model.rules().get(0).ruleIndex()); Assertions.assertEquals(false, model.rules().get(0).unixReadOnly()); Assertions.assertEquals(true, model.rules().get(0).unixReadWrite()); Assertions.assertEquals(false, model.rules().get(0).kerberos5ReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5ReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5ReadWrite()); Assertions.assertEquals(false, model.rules().get(0).kerberos5IReadOnly()); Assertions.assertEquals(true, model.rules().get(0).kerberos5IReadWrite()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadOnly()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadWrite()); Assertions.assertEquals(true, model.rules().get(0).cifs()); Assertions.assertEquals(true, model.rules().get(0).nfsv3()); - Assertions.assertEquals(false, model.rules().get(0).nfsv41()); - Assertions.assertEquals("jzbexilzznfq", model.rules().get(0).allowedClients()); - Assertions.assertEquals(false, model.rules().get(0).hasRootAccess()); + Assertions.assertEquals(true, model.rules().get(0).nfsv41()); + Assertions.assertEquals("jriplrbpbewtghf", model.rules().get(0).allowedClients()); + Assertions.assertEquals(true, model.rules().get(0).hasRootAccess()); Assertions.assertEquals(ChownMode.RESTRICTED, model.rules().get(0).chownMode()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumePropertiesExportPolicy model = new VolumePropertiesExportPolicy() - .withRules(Arrays.asList(new ExportPolicyRule().withRuleIndex(561759175).withUnixReadOnly(false) - .withUnixReadWrite(true).withKerberos5ReadOnly(false).withKerberos5ReadWrite(false) - .withKerberos5IReadOnly(false).withKerberos5IReadWrite(true).withKerberos5PReadOnly(false) - .withKerberos5PReadWrite(false).withCifs(true).withNfsv3(true).withNfsv41(false) - .withAllowedClients("jzbexilzznfq").withHasRootAccess(false).withChownMode(ChownMode.RESTRICTED))); + VolumePropertiesExportPolicy model = new VolumePropertiesExportPolicy().withRules(Arrays.asList( + new ExportPolicyRule().withRuleIndex(1149825898).withUnixReadOnly(false).withUnixReadWrite(true) + .withKerberos5ReadOnly(false).withKerberos5ReadWrite(true).withKerberos5IReadOnly(false) + .withKerberos5IReadWrite(true).withKerberos5PReadOnly(true).withKerberos5PReadWrite(true).withCifs(true) + .withNfsv3(true).withNfsv41(true).withAllowedClients("jriplrbpbewtghf").withHasRootAccess(true) + .withChownMode(ChownMode.RESTRICTED), + new ExportPolicyRule().withRuleIndex(1553365340).withUnixReadOnly(true).withUnixReadWrite(false) + .withKerberos5ReadOnly(true).withKerberos5ReadWrite(false).withKerberos5IReadOnly(true) + .withKerberos5IReadWrite(true).withKerberos5PReadOnly(false).withKerberos5PReadWrite(false) + .withCifs(true).withNfsv3(false).withNfsv41(true).withAllowedClients("loayqcgw") + .withHasRootAccess(false).withChownMode(ChownMode.UNRESTRICTED))); model = BinaryData.fromObject(model).toObject(VolumePropertiesExportPolicy.class); - Assertions.assertEquals(561759175, model.rules().get(0).ruleIndex()); + Assertions.assertEquals(1149825898, model.rules().get(0).ruleIndex()); Assertions.assertEquals(false, model.rules().get(0).unixReadOnly()); Assertions.assertEquals(true, model.rules().get(0).unixReadWrite()); Assertions.assertEquals(false, model.rules().get(0).kerberos5ReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5ReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5ReadWrite()); Assertions.assertEquals(false, model.rules().get(0).kerberos5IReadOnly()); Assertions.assertEquals(true, model.rules().get(0).kerberos5IReadWrite()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadOnly()); - Assertions.assertEquals(false, model.rules().get(0).kerberos5PReadWrite()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadOnly()); + Assertions.assertEquals(true, model.rules().get(0).kerberos5PReadWrite()); Assertions.assertEquals(true, model.rules().get(0).cifs()); Assertions.assertEquals(true, model.rules().get(0).nfsv3()); - Assertions.assertEquals(false, model.rules().get(0).nfsv41()); - Assertions.assertEquals("jzbexilzznfq", model.rules().get(0).allowedClients()); - Assertions.assertEquals(false, model.rules().get(0).hasRootAccess()); + Assertions.assertEquals(true, model.rules().get(0).nfsv41()); + Assertions.assertEquals("jriplrbpbewtghf", model.rules().get(0).allowedClients()); + Assertions.assertEquals(true, model.rules().get(0).hasRootAccess()); Assertions.assertEquals(ChownMode.RESTRICTED, model.rules().get(0).chownMode()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRuleInnerTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRuleInnerTests.java index f988270c02f6b..8197628d5438f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRuleInnerTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRuleInnerTests.java @@ -15,26 +15,26 @@ public final class VolumeQuotaRuleInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumeQuotaRuleInner model = BinaryData.fromString( - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"quotaSizeInKiBs\":1929545324703716474,\"quotaType\":\"DefaultUserQuota\",\"quotaTarget\":\"srtkfa\"},\"location\":\"opqgikyzirtxdyux\",\"tags\":{\"wgioilqukry\":\"ntps\",\"gufhyaomtbg\":\"xtqmieoxor\",\"fo\":\"havgrvk\",\"mv\":\"jzhpjbibgjmfx\"},\"id\":\"cluyovwxnbkf\",\"name\":\"zzxscyhwzdgiruj\",\"type\":\"zbomvzzbtdcqvpni\"}") + "{\"properties\":{\"provisioningState\":\"Patching\",\"quotaSizeInKiBs\":6449918274697799558,\"quotaType\":\"IndividualUserQuota\",\"quotaTarget\":\"xqi\"},\"location\":\"qunyowxwlmdjr\",\"tags\":{\"vfvpdbodaciz\":\"g\",\"lhkrribdeibqipqk\":\"j\",\"rwkq\":\"hvxndzwmkrefajpj\"},\"id\":\"yhgbijtjivfx\",\"name\":\"sjabibs\",\"type\":\"stawfsdjpvkv\"}") .toObject(VolumeQuotaRuleInner.class); - Assertions.assertEquals("opqgikyzirtxdyux", model.location()); - Assertions.assertEquals("ntps", model.tags().get("wgioilqukry")); - Assertions.assertEquals(1929545324703716474L, model.quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, model.quotaType()); - Assertions.assertEquals("srtkfa", model.quotaTarget()); + Assertions.assertEquals("qunyowxwlmdjr", model.location()); + Assertions.assertEquals("g", model.tags().get("vfvpdbodaciz")); + Assertions.assertEquals(6449918274697799558L, model.quotaSizeInKiBs()); + Assertions.assertEquals(Type.INDIVIDUAL_USER_QUOTA, model.quotaType()); + Assertions.assertEquals("xqi", model.quotaTarget()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeQuotaRuleInner model = new VolumeQuotaRuleInner().withLocation("opqgikyzirtxdyux") - .withTags(mapOf("wgioilqukry", "ntps", "gufhyaomtbg", "xtqmieoxor", "fo", "havgrvk", "mv", "jzhpjbibgjmfx")) - .withQuotaSizeInKiBs(1929545324703716474L).withQuotaType(Type.DEFAULT_USER_QUOTA).withQuotaTarget("srtkfa"); + VolumeQuotaRuleInner model = new VolumeQuotaRuleInner().withLocation("qunyowxwlmdjr") + .withTags(mapOf("vfvpdbodaciz", "g", "lhkrribdeibqipqk", "j", "rwkq", "hvxndzwmkrefajpj")) + .withQuotaSizeInKiBs(6449918274697799558L).withQuotaType(Type.INDIVIDUAL_USER_QUOTA).withQuotaTarget("xqi"); model = BinaryData.fromObject(model).toObject(VolumeQuotaRuleInner.class); - Assertions.assertEquals("opqgikyzirtxdyux", model.location()); - Assertions.assertEquals("ntps", model.tags().get("wgioilqukry")); - Assertions.assertEquals(1929545324703716474L, model.quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, model.quotaType()); - Assertions.assertEquals("srtkfa", model.quotaTarget()); + Assertions.assertEquals("qunyowxwlmdjr", model.location()); + Assertions.assertEquals("g", model.tags().get("vfvpdbodaciz")); + Assertions.assertEquals(6449918274697799558L, model.quotaSizeInKiBs()); + Assertions.assertEquals(Type.INDIVIDUAL_USER_QUOTA, model.quotaType()); + Assertions.assertEquals("xqi", model.quotaTarget()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulePatchTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulePatchTests.java index 637483c5f8955..699f471d90115 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulePatchTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulePatchTests.java @@ -15,24 +15,26 @@ public final class VolumeQuotaRulePatchTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumeQuotaRulePatch model = BinaryData.fromString( - "{\"tags\":{\"rfh\":\"sgaojfmwncot\",\"xoftpipiwyczu\":\"rctym\"},\"properties\":{\"provisioningState\":\"Patching\",\"quotaSizeInKiBs\":5299678736839094769,\"quotaType\":\"IndividualUserQuota\",\"quotaTarget\":\"hyus\"}}") + "{\"tags\":{\"zhjjklffhmouwq\":\"jabudurgkakmo\",\"eeyebi\":\"gzrf\",\"wrv\":\"ikayuhqlbjbsybb\",\"slthaq\":\"ldgmfpgvmpip\"},\"properties\":{\"provisioningState\":\"Accepted\",\"quotaSizeInKiBs\":152886456423473805,\"quotaType\":\"DefaultGroupQuota\",\"quotaTarget\":\"dsrezpdrhneuyow\"}}") .toObject(VolumeQuotaRulePatch.class); - Assertions.assertEquals("sgaojfmwncot", model.tags().get("rfh")); - Assertions.assertEquals(5299678736839094769L, model.quotaSizeInKiBs()); - Assertions.assertEquals(Type.INDIVIDUAL_USER_QUOTA, model.quotaType()); - Assertions.assertEquals("hyus", model.quotaTarget()); + Assertions.assertEquals("jabudurgkakmo", model.tags().get("zhjjklffhmouwq")); + Assertions.assertEquals(152886456423473805L, model.quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, model.quotaType()); + Assertions.assertEquals("dsrezpdrhneuyow", model.quotaTarget()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { VolumeQuotaRulePatch model = new VolumeQuotaRulePatch() - .withTags(mapOf("rfh", "sgaojfmwncot", "xoftpipiwyczu", "rctym")).withQuotaSizeInKiBs(5299678736839094769L) - .withQuotaType(Type.INDIVIDUAL_USER_QUOTA).withQuotaTarget("hyus"); + .withTags(mapOf("zhjjklffhmouwq", "jabudurgkakmo", "eeyebi", "gzrf", "wrv", "ikayuhqlbjbsybb", "slthaq", + "ldgmfpgvmpip")) + .withQuotaSizeInKiBs(152886456423473805L).withQuotaType(Type.DEFAULT_GROUP_QUOTA) + .withQuotaTarget("dsrezpdrhneuyow"); model = BinaryData.fromObject(model).toObject(VolumeQuotaRulePatch.class); - Assertions.assertEquals("sgaojfmwncot", model.tags().get("rfh")); - Assertions.assertEquals(5299678736839094769L, model.quotaSizeInKiBs()); - Assertions.assertEquals(Type.INDIVIDUAL_USER_QUOTA, model.quotaType()); - Assertions.assertEquals("hyus", model.quotaTarget()); + Assertions.assertEquals("jabudurgkakmo", model.tags().get("zhjjklffhmouwq")); + Assertions.assertEquals(152886456423473805L, model.quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, model.quotaType()); + Assertions.assertEquals("dsrezpdrhneuyow", model.quotaTarget()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateMockTests.java index 92b87ceb96ba7..18802062903eb 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesCreateMockTests.java @@ -34,7 +34,7 @@ public void testCreate() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"quotaSizeInKiBs\":8901688908822636445,\"quotaType\":\"DefaultGroupQuota\",\"quotaTarget\":\"tnpqmemczjk\"},\"location\":\"ykyujxsg\",\"tags\":{\"yejylmbkzudnigrf\":\"rr\",\"wlpxuzzjg\":\"hotj\",\"qotoihiqakydiwfb\":\"refqy\",\"spodaqax\":\"kwpzdqtvh\"},\"id\":\"ipietgbe\",\"name\":\"jfulbmoic\",\"type\":\"dlpnfpubn\"}"; + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"quotaSizeInKiBs\":1851749746038265021,\"quotaType\":\"DefaultUserQuota\",\"quotaTarget\":\"hfqpofv\"},\"location\":\"cblembnkbwv\",\"tags\":{\"qihebw\":\"kdi\"},\"id\":\"swbzuwfmdurage\",\"name\":\"izvcjfe\",\"type\":\"isdju\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,16 +51,18 @@ public void testCreate() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - VolumeQuotaRule response = manager.volumeQuotaRules().define("cbvopwndyqleallk").withRegion("yzfhotlhi") - .withExistingVolume("ycjimryvwgcwwpbm", "gwe", "ydsx", "efoh").withTags(mapOf("vuicp", "ychunsjlpjrtwszh")) - .withQuotaSizeInKiBs(571481130330646230L).withQuotaType(Type.DEFAULT_GROUP_QUOTA) - .withQuotaTarget("xpvbrdfjm").create(); + VolumeQuotaRule response = manager.volumeQuotaRules().define("kqscazuawxtzx").withRegion("sflvgsgzwywakoih") + .withExistingVolume("cpopmxel", "wcltyjede", "xm", "f") + .withTags(mapOf("zotqyryuzcb", "mjblmljhlny", "peujlzqn", "qqvxmvwfgtayxons", "nzoibgsxgnx", "cvsql", + "mpqoxw", "yqo")) + .withQuotaSizeInKiBs(5532317119730945052L).withQuotaType(Type.INDIVIDUAL_GROUP_QUOTA) + .withQuotaTarget("vxcushsphaivmx").create(); - Assertions.assertEquals("ykyujxsg", response.location()); - Assertions.assertEquals("rr", response.tags().get("yejylmbkzudnigrf")); - Assertions.assertEquals(8901688908822636445L, response.quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, response.quotaType()); - Assertions.assertEquals("tnpqmemczjk", response.quotaTarget()); + Assertions.assertEquals("cblembnkbwv", response.location()); + Assertions.assertEquals("kdi", response.tags().get("qihebw")); + Assertions.assertEquals(1851749746038265021L, response.quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, response.quotaType()); + Assertions.assertEquals("hfqpofv", response.quotaTarget()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteMockTests.java index af9a09e04f386..b2a093f970f7c 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesDeleteMockTests.java @@ -45,7 +45,7 @@ public void testDelete() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumeQuotaRules().delete("owlkjxnqpv", "gf", "tmhqykiz", "d", "saoafcluqvox", + manager.volumeQuotaRules().delete("awjutifd", "fmvigorqjbttzh", "aglkafhon", "juj", "ickpz", com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetWithResponseMockTests.java index 8c32944df20ec..0dd2a5918fd06 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesGetWithResponseMockTests.java @@ -32,7 +32,7 @@ public void testGetWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"properties\":{\"provisioningState\":\"Accepted\",\"quotaSizeInKiBs\":6402661217613808008,\"quotaType\":\"DefaultGroupQuota\",\"quotaTarget\":\"oqboshbragapyyr\"},\"location\":\"svbpavbopfppdbwn\",\"tags\":{\"asjcaacfdmmcpu\":\"ahxku\",\"zeh\":\"mehqepvufh\",\"clgsc\":\"whoqhnlbqnbldxe\",\"rsrrmoucsofldp\":\"orim\"},\"id\":\"viyfcaabeolhbhlv\",\"name\":\"mxuq\",\"type\":\"bsxtkcudfbsfarfs\"}"; + = "{\"properties\":{\"provisioningState\":\"Creating\",\"quotaSizeInKiBs\":3265340554220962405,\"quotaType\":\"DefaultUserQuota\",\"quotaTarget\":\"ajqglcf\"},\"location\":\"l\",\"tags\":{\"zrdpsovwxznptgoe\":\"yxynq\",\"vntjlrigjk\":\"ybbabpfhvfsl\",\"xwaabzmifrygznmm\":\"kyrioovzid\"},\"id\":\"xrizkzobgop\",\"name\":\"lhslnelxieixyn\",\"type\":\"lxecwcrojphslh\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -49,13 +49,13 @@ public void testGetWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - VolumeQuotaRule response = manager.volumeQuotaRules().getWithResponse("zb", "ybww", "bdvibidmhmwffpl", "muvapc", - "ccrrvweyoxoyyu", com.azure.core.util.Context.NONE).getValue(); + VolumeQuotaRule response = manager.volumeQuotaRules().getWithResponse("ids", "ault", "ijjumfq", + "azlnqnmcjngzqdqx", "bjwgnyfus", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals("svbpavbopfppdbwn", response.location()); - Assertions.assertEquals("ahxku", response.tags().get("asjcaacfdmmcpu")); - Assertions.assertEquals(6402661217613808008L, response.quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, response.quotaType()); - Assertions.assertEquals("oqboshbragapyyr", response.quotaTarget()); + Assertions.assertEquals("l", response.location()); + Assertions.assertEquals("yxynq", response.tags().get("zrdpsovwxznptgoe")); + Assertions.assertEquals(3265340554220962405L, response.quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, response.quotaType()); + Assertions.assertEquals("ajqglcf", response.quotaTarget()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeMockTests.java index 8668bc2662c4c..3f73d4c3866a8 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListByVolumeMockTests.java @@ -33,7 +33,7 @@ public void testListByVolume() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"provisioningState\":\"Patching\",\"quotaSizeInKiBs\":6035951377180684883,\"quotaType\":\"IndividualGroupQuota\",\"quotaTarget\":\"oxczytp\"},\"location\":\"nwvroevytlyokrr\",\"tags\":{\"rxklobdxnazpmk\":\"uxvnsasbcrymodi\",\"vfxzopjh\":\"lmv\",\"d\":\"zxlioh\"},\"id\":\"dtfgxqbawpcbb\",\"name\":\"zqcyknap\",\"type\":\"ofyuicd\"}]}"; + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\",\"quotaSizeInKiBs\":8351814854754932643,\"quotaType\":\"DefaultUserQuota\",\"quotaTarget\":\"bwefqsfapaqtfer\"},\"location\":\"wexjkmfxapjwogq\",\"tags\":{\"qwpwyawbzasqbuc\":\"bpudcdab\"},\"id\":\"jg\",\"name\":\"yexaoguy\",\"type\":\"i\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,13 +50,13 @@ public void testListByVolume() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.volumeQuotaRules().listByVolume("ninvudbchaqdt", "qecrqctmxx", - "tddmf", "huytxzvtzn", com.azure.core.util.Context.NONE); + PagedIterable response = manager.volumeQuotaRules().listByVolume("cpilj", "a", "zv", + "chndbnwie", com.azure.core.util.Context.NONE); - Assertions.assertEquals("nwvroevytlyokrr", response.iterator().next().location()); - Assertions.assertEquals("uxvnsasbcrymodi", response.iterator().next().tags().get("rxklobdxnazpmk")); - Assertions.assertEquals(6035951377180684883L, response.iterator().next().quotaSizeInKiBs()); - Assertions.assertEquals(Type.INDIVIDUAL_GROUP_QUOTA, response.iterator().next().quotaType()); - Assertions.assertEquals("oxczytp", response.iterator().next().quotaTarget()); + Assertions.assertEquals("wexjkmfxapjwogq", response.iterator().next().location()); + Assertions.assertEquals("bpudcdab", response.iterator().next().tags().get("qwpwyawbzasqbuc")); + Assertions.assertEquals(8351814854754932643L, response.iterator().next().quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, response.iterator().next().quotaType()); + Assertions.assertEquals("bwefqsfapaqtfer", response.iterator().next().quotaTarget()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListTests.java index 82f378c7a4edd..b343860aea051 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesListTests.java @@ -17,31 +17,29 @@ public final class VolumeQuotaRulesListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumeQuotaRulesList model = BinaryData.fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Patching\",\"quotaSizeInKiBs\":5649279960979439308,\"quotaType\":\"DefaultGroupQuota\",\"quotaTarget\":\"bihwqknfdnt\"},\"location\":\"chrdgoihxumwcto\",\"tags\":{\"udfdlwgg\":\"jl\"},\"id\":\"tsbwtovvtgse\",\"name\":\"nqfiufxqknpi\",\"type\":\"gnepttwqmsni\"},{\"properties\":{\"provisioningState\":\"Moving\",\"quotaSizeInKiBs\":8728192731821949448,\"quotaType\":\"IndividualUserQuota\",\"quotaTarget\":\"lpijnkrxfrd\"},\"location\":\"c\",\"tags\":{\"hftwesgog\":\"izzronasxiftozqy\",\"sxkkg\":\"zhonnxkrlgnyhmo\",\"rghxjb\":\"h\"},\"id\":\"hqxvcxgfrpdsofbs\",\"name\":\"rnsvbuswd\",\"type\":\"z\"}]}") + "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"quotaSizeInKiBs\":4528392326800353907,\"quotaType\":\"IndividualGroupQuota\",\"quotaTarget\":\"wem\"},\"location\":\"rncsdtclu\",\"tags\":{\"lfeadcygq\":\"pbsfgytgu\",\"elolppvksrpqvuj\":\"kyhejhzisxgf\",\"wib\":\"raehtwdwrft\",\"cstwity\":\"rcdlbhshfwpr\"},\"id\":\"hevxcced\",\"name\":\"pnmdyodnwzxltjcv\",\"type\":\"hlt\"}]}") .toObject(VolumeQuotaRulesList.class); - Assertions.assertEquals("chrdgoihxumwcto", model.value().get(0).location()); - Assertions.assertEquals("jl", model.value().get(0).tags().get("udfdlwgg")); - Assertions.assertEquals(5649279960979439308L, model.value().get(0).quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, model.value().get(0).quotaType()); - Assertions.assertEquals("bihwqknfdnt", model.value().get(0).quotaTarget()); + Assertions.assertEquals("rncsdtclu", model.value().get(0).location()); + Assertions.assertEquals("pbsfgytgu", model.value().get(0).tags().get("lfeadcygq")); + Assertions.assertEquals(4528392326800353907L, model.value().get(0).quotaSizeInKiBs()); + Assertions.assertEquals(Type.INDIVIDUAL_GROUP_QUOTA, model.value().get(0).quotaType()); + Assertions.assertEquals("wem", model.value().get(0).quotaTarget()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeQuotaRulesList model = new VolumeQuotaRulesList().withValue(Arrays.asList( - new VolumeQuotaRuleInner().withLocation("chrdgoihxumwcto").withTags(mapOf("udfdlwgg", "jl")) - .withQuotaSizeInKiBs(5649279960979439308L).withQuotaType(Type.DEFAULT_GROUP_QUOTA) - .withQuotaTarget("bihwqknfdnt"), - new VolumeQuotaRuleInner().withLocation("c") - .withTags(mapOf("hftwesgog", "izzronasxiftozqy", "sxkkg", "zhonnxkrlgnyhmo", "rghxjb", "h")) - .withQuotaSizeInKiBs(8728192731821949448L).withQuotaType(Type.INDIVIDUAL_USER_QUOTA) - .withQuotaTarget("lpijnkrxfrd"))); + VolumeQuotaRulesList model + = new VolumeQuotaRulesList().withValue(Arrays.asList(new VolumeQuotaRuleInner().withLocation("rncsdtclu") + .withTags(mapOf("lfeadcygq", "pbsfgytgu", "elolppvksrpqvuj", "kyhejhzisxgf", "wib", "raehtwdwrft", + "cstwity", "rcdlbhshfwpr")) + .withQuotaSizeInKiBs(4528392326800353907L).withQuotaType(Type.INDIVIDUAL_GROUP_QUOTA) + .withQuotaTarget("wem"))); model = BinaryData.fromObject(model).toObject(VolumeQuotaRulesList.class); - Assertions.assertEquals("chrdgoihxumwcto", model.value().get(0).location()); - Assertions.assertEquals("jl", model.value().get(0).tags().get("udfdlwgg")); - Assertions.assertEquals(5649279960979439308L, model.value().get(0).quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, model.value().get(0).quotaType()); - Assertions.assertEquals("bihwqknfdnt", model.value().get(0).quotaTarget()); + Assertions.assertEquals("rncsdtclu", model.value().get(0).location()); + Assertions.assertEquals("pbsfgytgu", model.value().get(0).tags().get("lfeadcygq")); + Assertions.assertEquals(4528392326800353907L, model.value().get(0).quotaSizeInKiBs()); + Assertions.assertEquals(Type.INDIVIDUAL_GROUP_QUOTA, model.value().get(0).quotaType()); + Assertions.assertEquals("wem", model.value().get(0).quotaTarget()); } // Use "Map.of" if available diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesPropertiesTests.java index 440789922f84a..4085a07e1e741 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeQuotaRulesPropertiesTests.java @@ -13,20 +13,20 @@ public final class VolumeQuotaRulesPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumeQuotaRulesProperties model = BinaryData.fromString( - "{\"provisioningState\":\"Creating\",\"quotaSizeInKiBs\":4156012930167419412,\"quotaType\":\"DefaultUserQuota\",\"quotaTarget\":\"shfssnrbgyef\"}") + "{\"provisioningState\":\"Creating\",\"quotaSizeInKiBs\":7435265632351903503,\"quotaType\":\"DefaultGroupQuota\",\"quotaTarget\":\"kd\"}") .toObject(VolumeQuotaRulesProperties.class); - Assertions.assertEquals(4156012930167419412L, model.quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, model.quotaType()); - Assertions.assertEquals("shfssnrbgyef", model.quotaTarget()); + Assertions.assertEquals(7435265632351903503L, model.quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, model.quotaType()); + Assertions.assertEquals("kd", model.quotaTarget()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeQuotaRulesProperties model = new VolumeQuotaRulesProperties().withQuotaSizeInKiBs(4156012930167419412L) - .withQuotaType(Type.DEFAULT_USER_QUOTA).withQuotaTarget("shfssnrbgyef"); + VolumeQuotaRulesProperties model = new VolumeQuotaRulesProperties().withQuotaSizeInKiBs(7435265632351903503L) + .withQuotaType(Type.DEFAULT_GROUP_QUOTA).withQuotaTarget("kd"); model = BinaryData.fromObject(model).toObject(VolumeQuotaRulesProperties.class); - Assertions.assertEquals(4156012930167419412L, model.quotaSizeInKiBs()); - Assertions.assertEquals(Type.DEFAULT_USER_QUOTA, model.quotaType()); - Assertions.assertEquals("shfssnrbgyef", model.quotaTarget()); + Assertions.assertEquals(7435265632351903503L, model.quotaSizeInKiBs()); + Assertions.assertEquals(Type.DEFAULT_GROUP_QUOTA, model.quotaType()); + Assertions.assertEquals("kd", model.quotaTarget()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRelocationPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRelocationPropertiesTests.java index 7ebdaf532ab9f..0640f8fafd174 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRelocationPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRelocationPropertiesTests.java @@ -12,15 +12,15 @@ public final class VolumeRelocationPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumeRelocationProperties model - = BinaryData.fromString("{\"relocationRequested\":false,\"readyToBeFinalized\":false}") + = BinaryData.fromString("{\"relocationRequested\":true,\"readyToBeFinalized\":false}") .toObject(VolumeRelocationProperties.class); - Assertions.assertEquals(false, model.relocationRequested()); + Assertions.assertEquals(true, model.relocationRequested()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeRelocationProperties model = new VolumeRelocationProperties().withRelocationRequested(false); + VolumeRelocationProperties model = new VolumeRelocationProperties().withRelocationRequested(true); model = BinaryData.fromObject(model).toObject(VolumeRelocationProperties.class); - Assertions.assertEquals(false, model.relocationRequested()); + Assertions.assertEquals(true, model.relocationRequested()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRevertTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRevertTests.java index cef11d42f9f30..5cf8be1577f4e 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRevertTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeRevertTests.java @@ -11,14 +11,14 @@ public final class VolumeRevertTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VolumeRevert model = BinaryData.fromString("{\"snapshotId\":\"jwaa\"}").toObject(VolumeRevert.class); - Assertions.assertEquals("jwaa", model.snapshotId()); + VolumeRevert model = BinaryData.fromString("{\"snapshotId\":\"wyhzdx\"}").toObject(VolumeRevert.class); + Assertions.assertEquals("wyhzdx", model.snapshotId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeRevert model = new VolumeRevert().withSnapshotId("jwaa"); + VolumeRevert model = new VolumeRevert().withSnapshotId("wyhzdx"); model = BinaryData.fromObject(model).toObject(VolumeRevert.class); - Assertions.assertEquals("jwaa", model.snapshotId()); + Assertions.assertEquals("wyhzdx", model.snapshotId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeSnapshotPropertiesTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeSnapshotPropertiesTests.java index 811316d08ab9e..1a44683415cc4 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeSnapshotPropertiesTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumeSnapshotPropertiesTests.java @@ -12,14 +12,14 @@ public final class VolumeSnapshotPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { VolumeSnapshotProperties model - = BinaryData.fromString("{\"snapshotPolicyId\":\"konzmnsik\"}").toObject(VolumeSnapshotProperties.class); - Assertions.assertEquals("konzmnsik", model.snapshotPolicyId()); + = BinaryData.fromString("{\"snapshotPolicyId\":\"p\"}").toObject(VolumeSnapshotProperties.class); + Assertions.assertEquals("p", model.snapshotPolicyId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VolumeSnapshotProperties model = new VolumeSnapshotProperties().withSnapshotPolicyId("konzmnsik"); + VolumeSnapshotProperties model = new VolumeSnapshotProperties().withSnapshotPolicyId("p"); model = BinaryData.fromObject(model).toObject(VolumeSnapshotProperties.class); - Assertions.assertEquals("konzmnsik", model.snapshotPolicyId()); + Assertions.assertEquals("p", model.snapshotPolicyId()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationMockTests.java index 0b9a7419d693e..241e2629f0083 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesAuthorizeReplicationMockTests.java @@ -46,8 +46,8 @@ public void testAuthorizeReplication() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().authorizeReplication("aoqltfaey", "inmfgvxirp", "hriypoqeyhlqhy", "prlpy", - new AuthorizeRequest().withRemoteVolumeResourceId("uciqdsme"), com.azure.core.util.Context.NONE); + manager.volumes().authorizeReplication("lqxihhrmooiz", "seypxiutcxapz", "y", "petogebjox", + new AuthorizeRequest().withRemoteVolumeResourceId("hvnh"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksMockTests.java index d9185fba738fc..3d4e77089b28a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakFileLocksMockTests.java @@ -46,8 +46,8 @@ public void testBreakFileLocks() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().breakFileLocks("wjlvizbfhfov", "acqpbtuodxesza", "belawumuaslzkwr", "woycqucwyha", - new BreakFileLocksRequest().withClientIp("omd").withConfirmRunningDisruptiveOperation(true), + manager.volumes().breakFileLocks("aspavehhr", "kbunzoz", "dhcxgkmoy", "cdyuibhmfdnbzyd", + new BreakFileLocksRequest().withClientIp("vfcjnaeoisrvhmg").withConfirmRunningDisruptiveOperation(true), com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationMockTests.java index 306244e6f5d32..8c6225ada7855 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesBreakReplicationMockTests.java @@ -46,8 +46,8 @@ public void testBreakReplication() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().breakReplication("qspkcdqzhlctd", "unqndyfpchrqb", "jjrcgegydc", "boxjumvq", - new BreakReplicationRequest().withForceBreakReplication(true), com.azure.core.util.Context.NONE); + manager.volumes().breakReplication("iuagydwqfbylyrfg", "agt", "ojocqwogf", "zjvusfzldmo", + new BreakReplicationRequest().withForceBreakReplication(false), com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationMockTests.java index 88161039960ac..1391fbc7d7179 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesDeleteReplicationMockTests.java @@ -45,7 +45,7 @@ public void testDeleteReplication() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().deleteReplication("acvbmqz", "qqxlajr", "wxacevehj", "uyxoaf", + manager.volumes().deleteReplication("kvyklxubyjaffmm", "bl", "qcuubgqibrta", "metttwgd", com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationMockTests.java index eeed8366877ec..083f9aab228fc 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesFinalizeRelocationMockTests.java @@ -45,7 +45,8 @@ public void testFinalizeRelocation() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().finalizeRelocation("mkyi", "cysihs", "qcwdhoh", "dtmcd", com.azure.core.util.Context.NONE); + manager.volumes().finalizeRelocation("jhlfzswpchwahf", "ousnfepgfewe", "wlyxgncxyk", "hdjhlimmbcx", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserMockTests.java index 3ff0d95017c9c..15cce6e31a623 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListGetGroupIdListForLdapUserMockTests.java @@ -31,8 +31,7 @@ public void testListGetGroupIdListForLdapUser() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr - = "{\"groupIdsForLdapUser\":[\"jibnxmysu\",\"swqrntvlwijp\",\"ttexoqqpwcyyufmh\",\"uncuw\"]}"; + String responseStr = "{\"groupIdsForLdapUser\":[\"eknlra\",\"iaa\"]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,9 +49,9 @@ public void testListGetGroupIdListForLdapUser() throws Exception { new AzureProfile("", "", AzureEnvironment.AZURE)); GetGroupIdListForLdapUserResponse response - = manager.volumes().listGetGroupIdListForLdapUser("wuhpsvfuur", "tlwexxwlalniexz", "rzpgep", "tybbwwpgda", - new GetGroupIdListForLdapUserRequest().withUsername("chzyvlixqnrk"), com.azure.core.util.Context.NONE); + = manager.volumes().listGetGroupIdListForLdapUser("fukiscvwmzhw", "lefaxvxilcbtgn", "nzeyqxtjj", "zqlqhyc", + new GetGroupIdListForLdapUserRequest().withUsername("vodggxdbee"), com.azure.core.util.Context.NONE); - Assertions.assertEquals("jibnxmysu", response.groupIdsForLdapUser().get(0)); + Assertions.assertEquals("eknlra", response.groupIdsForLdapUser().get(0)); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsMockTests.java index e0b4d73c0e6f8..951af7d00131f 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesListReplicationsMockTests.java @@ -34,7 +34,7 @@ public void testListReplications() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"endpointType\":\"src\",\"replicationSchedule\":\"_10minutely\",\"remoteVolumeResourceId\":\"mfp\",\"remoteVolumeRegion\":\"ojeevy\"}]}"; + = "{\"value\":[{\"endpointType\":\"src\",\"replicationSchedule\":\"daily\",\"remoteVolumeResourceId\":\"yowc\",\"remoteVolumeRegion\":\"qovekqvgqou\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -51,12 +51,12 @@ public void testListReplications() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.volumes().listReplications("pctf", "mdxotngfdgu", "eyzihgrky", - "i", com.azure.core.util.Context.NONE); + PagedIterable response = manager.volumes().listReplications("plcwkhi", "ihlhzdsqtzb", + "rgnowcjhfgm", "ecactx", com.azure.core.util.Context.NONE); Assertions.assertEquals(EndpointType.SRC, response.iterator().next().endpointType()); - Assertions.assertEquals(ReplicationSchedule.ONE_ZEROMINUTELY, response.iterator().next().replicationSchedule()); - Assertions.assertEquals("mfp", response.iterator().next().remoteVolumeResourceId()); - Assertions.assertEquals("ojeevy", response.iterator().next().remoteVolumeRegion()); + Assertions.assertEquals(ReplicationSchedule.DAILY, response.iterator().next().replicationSchedule()); + Assertions.assertEquals("yowc", response.iterator().next().remoteVolumeResourceId()); + Assertions.assertEquals("qovekqvgqou", response.iterator().next().remoteVolumeRegion()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeMockTests.java index 96dec8b43d6e5..3f323463a812a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesPoolChangeMockTests.java @@ -46,8 +46,8 @@ public void testPoolChange() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().poolChange("mavmq", "oudorhcgyyp", "otwypundmb", "hu", - new PoolChangeRequest().withNewPoolResourceId("cmjkavlgorbmftpm"), com.azure.core.util.Context.NONE); + manager.volumes().poolChange("lftidgfcwqmpim", "qxzhem", "yhohujswtwkozzwc", "lkb", + new PoolChangeRequest().withNewPoolResourceId("wpfaj"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationMockTests.java index b2ebb6268fa32..d3a017e92be05 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReInitializeReplicationMockTests.java @@ -45,7 +45,7 @@ public void testReInitializeReplication() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().reInitializeReplication("iitdfuxt", "asiibmiybnnust", "nlj", "nmgixh", + manager.volumes().reInitializeReplication("abrqnkkzj", "jb", "rgaehvvibrxjj", "toqbeitpkxztmoob", com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusWithResponseMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusWithResponseMockTests.java index 6441e7bbe7ba6..70655ef4c536b 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusWithResponseMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesReplicationStatusWithResponseMockTests.java @@ -33,7 +33,7 @@ public void testReplicationStatusWithResponse() throws Exception { ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"healthy\":true,\"relationshipStatus\":\"Idle\",\"mirrorState\":\"Mirrored\",\"totalProgress\":\"fqwmkyoquf\",\"errorMessage\":\"ruzslzoj\"}"; + = "{\"healthy\":true,\"relationshipStatus\":\"Transferring\",\"mirrorState\":\"Mirrored\",\"totalProgress\":\"jqctojcmisofie\",\"errorMessage\":\"efojyqdhcupl\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -50,13 +50,13 @@ public void testReplicationStatusWithResponse() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - ReplicationStatus response = manager.volumes().replicationStatusWithResponse("ihrraiouaub", "jtlo", "xfuojrn", - "iflrzpasccbiu", com.azure.core.util.Context.NONE).getValue(); + ReplicationStatus response = manager.volumes() + .replicationStatusWithResponse("ylfsbtk", "dp", "s", "wn", com.azure.core.util.Context.NONE).getValue(); Assertions.assertEquals(true, response.healthy()); - Assertions.assertEquals(RelationshipStatus.IDLE, response.relationshipStatus()); + Assertions.assertEquals(RelationshipStatus.TRANSFERRING, response.relationshipStatus()); Assertions.assertEquals(MirrorState.MIRRORED, response.mirrorState()); - Assertions.assertEquals("fqwmkyoquf", response.totalProgress()); - Assertions.assertEquals("ruzslzoj", response.errorMessage()); + Assertions.assertEquals("jqctojcmisofie", response.totalProgress()); + Assertions.assertEquals("efojyqdhcupl", response.errorMessage()); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationMockTests.java index b5746902dfc9a..1babd90ebb03a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesResyncReplicationMockTests.java @@ -45,7 +45,7 @@ public void testResyncReplication() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().resyncReplication("yhsgz", "czbgomfgbeg", "qgleohibetnluank", "rfxeeebtij", + manager.volumes().resyncReplication("ifzmpjwyivqi", "f", "cvhrfsp", "uagrttikteusqc", com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertMockTests.java index 287d91240d33f..0eff423a53770 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertMockTests.java @@ -46,8 +46,8 @@ public void testRevert() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().revert("n", "ypuuwwltvuqjctze", "keifzzhmkdasv", "lyhb", - new VolumeRevert().withSnapshotId("udchxgsrboldforo"), com.azure.core.util.Context.NONE); + manager.volumes().revert("hlfkqojpy", "vgtrdcnifmzzs", "ymbrnysuxmpraf", "g", + new VolumeRevert().withSnapshotId("hocxvdfffwafqrou"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationMockTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationMockTests.java index d2e7d938409c9..b93bce96466e5 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationMockTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/VolumesRevertRelocationMockTests.java @@ -45,7 +45,7 @@ public void testRevertRelocation() throws Exception { tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.volumes().revertRelocation("sufco", "dxbzlmcmuap", "vhdbevwqqxey", "ko", + manager.volumes().revertRelocation("h", "cporxvxcjz", "qizxfpxtgqscjavf", "juhdqazkmtgguwpi", com.azure.core.util.Context.NONE); } diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/WeeklyScheduleTests.java b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/WeeklyScheduleTests.java index f6ecfa603a74f..7e644d5a71613 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/WeeklyScheduleTests.java +++ b/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/WeeklyScheduleTests.java @@ -12,24 +12,24 @@ public final class WeeklyScheduleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { WeeklySchedule model = BinaryData.fromString( - "{\"snapshotsToKeep\":214137082,\"day\":\"msbvdkcrodtjinf\",\"hour\":1492452539,\"minute\":274733430,\"usedBytes\":3850222588525127321}") + "{\"snapshotsToKeep\":672785070,\"day\":\"gjhkycubeddg\",\"hour\":947899145,\"minute\":583924547,\"usedBytes\":3778047541183075264}") .toObject(WeeklySchedule.class); - Assertions.assertEquals(214137082, model.snapshotsToKeep()); - Assertions.assertEquals("msbvdkcrodtjinf", model.day()); - Assertions.assertEquals(1492452539, model.hour()); - Assertions.assertEquals(274733430, model.minute()); - Assertions.assertEquals(3850222588525127321L, model.usedBytes()); + Assertions.assertEquals(672785070, model.snapshotsToKeep()); + Assertions.assertEquals("gjhkycubeddg", model.day()); + Assertions.assertEquals(947899145, model.hour()); + Assertions.assertEquals(583924547, model.minute()); + Assertions.assertEquals(3778047541183075264L, model.usedBytes()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - WeeklySchedule model = new WeeklySchedule().withSnapshotsToKeep(214137082).withDay("msbvdkcrodtjinf") - .withHour(1492452539).withMinute(274733430).withUsedBytes(3850222588525127321L); + WeeklySchedule model = new WeeklySchedule().withSnapshotsToKeep(672785070).withDay("gjhkycubeddg") + .withHour(947899145).withMinute(583924547).withUsedBytes(3778047541183075264L); model = BinaryData.fromObject(model).toObject(WeeklySchedule.class); - Assertions.assertEquals(214137082, model.snapshotsToKeep()); - Assertions.assertEquals("msbvdkcrodtjinf", model.day()); - Assertions.assertEquals(1492452539, model.hour()); - Assertions.assertEquals(274733430, model.minute()); - Assertions.assertEquals(3850222588525127321L, model.usedBytes()); + Assertions.assertEquals(672785070, model.snapshotsToKeep()); + Assertions.assertEquals("gjhkycubeddg", model.day()); + Assertions.assertEquals(947899145, model.hour()); + Assertions.assertEquals(583924547, model.minute()); + Assertions.assertEquals(3778047541183075264L, model.usedBytes()); } } diff --git a/sdk/netapp/tests.mgmt.yml b/sdk/netapp/tests.mgmt.yml index 032ebec5bc6b4..3f51774fe9325 100644 --- a/sdk/netapp/tests.mgmt.yml +++ b/sdk/netapp/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: netapp - Artifacts: - - name: azure-resourcemanager-netapp - groupId: com.azure.resourcemanager - safeName: azureresourcemanagernetapp - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: netapp + Artifacts: + - name: azure-resourcemanager-netapp + groupId: com.azure.resourcemanager + safeName: azureresourcemanagernetapp + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/networkanalytics/azure-resourcemanager-networkanalytics/pom.xml b/sdk/networkanalytics/azure-resourcemanager-networkanalytics/pom.xml index 6b15ca1b22dfa..2f3fb5c5dd175 100644 --- a/sdk/networkanalytics/azure-resourcemanager-networkanalytics/pom.xml +++ b/sdk/networkanalytics/azure-resourcemanager-networkanalytics/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -90,13 +90,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/networkcloud/azure-resourcemanager-networkcloud/pom.xml b/sdk/networkcloud/azure-resourcemanager-networkcloud/pom.xml index b383e56517277..e37444d8422f6 100644 --- a/sdk/networkcloud/azure-resourcemanager-networkcloud/pom.xml +++ b/sdk/networkcloud/azure-resourcemanager-networkcloud/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/networkfunction/azure-resourcemanager-networkfunction/pom.xml b/sdk/networkfunction/azure-resourcemanager-networkfunction/pom.xml index 83d4d71010470..c9a368f62e21f 100644 --- a/sdk/networkfunction/azure-resourcemanager-networkfunction/pom.xml +++ b/sdk/networkfunction/azure-resourcemanager-networkfunction/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/CHANGELOG.md b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/CHANGELOG.md index d38db317ede72..80306d4bc6512 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/CHANGELOG.md +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.1 (Unreleased) +## 1.2.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,79 @@ ### Other Changes +## 1.1.0 (2024-03-15) + +- Azure Resource Manager NewRelicObservability client library for Java. This package contains Microsoft Azure SDK for NewRelicObservability Management SDK. Package tag package-2024-01-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +* `models.ConfigurationName` was added + +* `models.MarketplaceSaaSInfo` was added + +* `models.MonitoredSubscriptionProperties$UpdateStages` was added + +* `models.MonitoredSubscription` was added + +* `models.MonitoredSubscriptionProperties$Definition` was added + +* `models.BillingInfoResponse` was added + +* `models.MonitoredSubscriptionPropertiesList` was added + +* `models.ConnectedPartnerResources` was added + +* `models.SubscriptionList` was added + +* `models.ConnectedPartnerResourcesListResponse` was added + +* `models.ConnectedPartnerResourcesListFormat` was added + +* `models.BillingInfoes` was added + +* `models.PatchOperation` was added + +* `models.MonitoredSubscriptionProperties$Update` was added + +* `models.Status` was added + +* `models.MonitoredSubscriptionProperties$DefinitionStages` was added + +* `models.ConnectedPartnerResourceProperties` was added + +* `models.MonitoredSubscriptionProperties` was added + +* `models.LinkedResource` was added + +* `models.PartnerBillingEntity` was added + +* `models.MonitoredSubscriptions` was added + +* `models.LinkedResourceListResponse` was added + +#### `models.NewRelicMonitorResource` was modified + +* `saaSAzureSubscriptionStatus()` was added +* `listLinkedResources(com.azure.core.util.Context)` was added +* `listLinkedResources()` was added +* `subscriptionState()` was added + +#### `models.NewRelicMonitorResource$Definition` was modified + +* `withSaaSAzureSubscriptionStatus(java.lang.String)` was added +* `withSubscriptionState(java.lang.String)` was added + +#### `models.Monitors` was modified + +* `listLinkedResources(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listLinkedResources(java.lang.String,java.lang.String)` was added + +#### `NewRelicObservabilityManager` was modified + +* `connectedPartnerResources()` was added +* `monitoredSubscriptions()` was added +* `billingInfoes()` was added + ## 1.0.0 (2023-05-18) - Azure Resource Manager NewRelicObservability client library for Java. This package contains Microsoft Azure SDK for NewRelicObservability Management SDK. Package tag package-2022-07-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/README.md b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/README.md index ee2a7521149c3..83d0779a85648 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/README.md +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/README.md @@ -2,7 +2,7 @@ Azure Resource Manager NewRelicObservability client library for Java. -This package contains Microsoft Azure SDK for NewRelicObservability Management SDK. Package tag package-2022-07-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for NewRelicObservability Management SDK. Package tag package-2024-01-01. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-newrelicobservability - 1.0.0 + 1.1.0 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -94,7 +94,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty @@ -103,3 +103,5 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [cg]: https://github.com/Azure/azure-sdk-for-java/blob/main/CONTRIBUTING.md [coc]: https://opensource.microsoft.com/codeofconduct/ [coc_faq]: https://opensource.microsoft.com/codeofconduct/faq/ + +![Impressions](https://azure-sdk-impressions.azurewebsites.net/api/impressions/azure-sdk-for-java%2Fsdk%2Fnewrelicobservability%2Fazure-resourcemanager-newrelicobservability%2FREADME.png) diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/SAMPLE.md b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/SAMPLE.md index d4d683f27b3b1..e482148640f75 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/SAMPLE.md +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/SAMPLE.md @@ -5,6 +5,22 @@ - [List](#accounts_list) +## BillingInfo + +- [Get](#billinginfo_get) + +## ConnectedPartnerResources + +- [List](#connectedpartnerresources_list) + +## MonitoredSubscriptions + +- [CreateorUpdate](#monitoredsubscriptions_createorupdate) +- [Delete](#monitoredsubscriptions_delete) +- [Get](#monitoredsubscriptions_get) +- [List](#monitoredsubscriptions_list) +- [Update](#monitoredsubscriptions_update) + ## Monitors - [CreateOrUpdate](#monitors_createorupdate) @@ -16,6 +32,7 @@ - [ListAppServices](#monitors_listappservices) - [ListByResourceGroup](#monitors_listbyresourcegroup) - [ListHosts](#monitors_listhosts) +- [ListLinkedResources](#monitors_listlinkedresources) - [ListMonitoredResources](#monitors_listmonitoredresources) - [SwitchBilling](#monitors_switchbilling) - [Update](#monitors_update) @@ -43,14 +60,17 @@ ### Accounts_List ```java -/** Samples for Accounts List. */ +/** + * Samples for Accounts List. + */ public final class AccountsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Accounts_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Accounts_List_MinimumSet_Gen.json */ /** * Sample code: Accounts_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void accountsListMinimumSetGen( @@ -59,11 +79,12 @@ public final class AccountsListSamples { } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Accounts_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Accounts_List_MaximumSet_Gen.json */ /** * Sample code: Accounts_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void accountsListMaximumSetGen( @@ -73,12 +94,190 @@ public final class AccountsListSamples { } ``` +### BillingInfo_Get + +```java +/** + * Samples for BillingInfo Get. + */ +public final class BillingInfoGetSamples { + /* + * x-ms-original-file: + * specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/BillingInfo_Get.json + */ + /** + * Sample code: BillingInfo_Get. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void + billingInfoGet(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.billingInfoes().getWithResponse("myResourceGroup", "myMonitor", com.azure.core.util.Context.NONE); + } +} +``` + +### ConnectedPartnerResources_List + +```java +/** + * Samples for ConnectedPartnerResources List. + */ +public final class ConnectedPartnerResourcesListSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * ConnectedPartnerResources_List.json + */ + /** + * Sample code: ConnectedPartnerResources_List. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void connectedPartnerResourcesList( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.connectedPartnerResources().list("myResourceGroup", "myMonitor", null, + com.azure.core.util.Context.NONE); + } +} +``` + +### MonitoredSubscriptions_CreateorUpdate + +```java +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * Samples for MonitoredSubscriptions CreateorUpdate. + */ +public final class MonitoredSubscriptionsCreateorUpdateSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_CreateorUpdate.json + */ + /** + * Sample code: Monitors_AddMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsAddMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().define(ConfigurationName.DEFAULT) + .withExistingMonitor("myResourceGroup", "myMonitor").create(); + } +} +``` + +### MonitoredSubscriptions_Delete + +```java +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * Samples for MonitoredSubscriptions Delete. + */ +public final class MonitoredSubscriptionsDeleteSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_Delete.json + */ + /** + * Sample code: Monitors_DeleteMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsDeleteMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().delete("myResourceGroup", "myMonitor", ConfigurationName.DEFAULT, + com.azure.core.util.Context.NONE); + } +} +``` + +### MonitoredSubscriptions_Get + +```java +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * Samples for MonitoredSubscriptions Get. + */ +public final class MonitoredSubscriptionsGetSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_Get.json + */ + /** + * Sample code: Monitors_GetMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsGetMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().getWithResponse("myResourceGroup", "myMonitor", ConfigurationName.DEFAULT, + com.azure.core.util.Context.NONE); + } +} +``` + +### MonitoredSubscriptions_List + +```java +/** + * Samples for MonitoredSubscriptions List. + */ +public final class MonitoredSubscriptionsListSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_List.json + */ + /** + * Sample code: Monitors_GetMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsGetMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().list("myResourceGroup", "myMonitor", com.azure.core.util.Context.NONE); + } +} +``` + +### MonitoredSubscriptions_Update + +```java +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; + +/** + * Samples for MonitoredSubscriptions Update. + */ +public final class MonitoredSubscriptionsUpdateSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_Update.json + */ + /** + * Sample code: Monitors_UpdateMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsUpdateMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + MonitoredSubscriptionProperties resource = manager.monitoredSubscriptions().getWithResponse("myResourceGroup", + "myMonitor", ConfigurationName.DEFAULT, com.azure.core.util.Context.NONE).getValue(); + resource.update().apply(); + } +} +``` + ### Monitors_CreateOrUpdate ```java import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; import com.azure.resourcemanager.newrelicobservability.models.AccountInfo; import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; +import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; +import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; import com.azure.resourcemanager.newrelicobservability.models.NewRelicAccountProperties; import com.azure.resourcemanager.newrelicobservability.models.NewRelicSingleSignOnProperties; import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; @@ -87,62 +286,48 @@ import com.azure.resourcemanager.newrelicobservability.models.PlanData; import com.azure.resourcemanager.newrelicobservability.models.ProvisioningState; import com.azure.resourcemanager.newrelicobservability.models.SingleSignOnStates; import com.azure.resourcemanager.newrelicobservability.models.UsageType; +import com.azure.resourcemanager.newrelicobservability.models.UserAssignedIdentity; import com.azure.resourcemanager.newrelicobservability.models.UserInfo; import java.time.OffsetDateTime; import java.util.HashMap; import java.util.Map; -/** Samples for Monitors CreateOrUpdate. */ +/** + * Samples for Monitors CreateOrUpdate. + */ public final class MonitorsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_CreateOrUpdate_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_CreateOrUpdate_MaximumSet_Gen.json */ /** * Sample code: Monitors_CreateOrUpdate_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsCreateOrUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .define("cdlymktqw") - .withRegion("k") - .withExistingResourceGroup("rgNewRelic") - .withTags(mapOf("key6976", "oaxfhf")) - .withNewRelicAccountProperties( - new NewRelicAccountProperties() - .withUserId("vcscxlncofcuduadesd") - .withAccountInfo( - new AccountInfo() - .withAccountId("xhqmg") - .withIngestionKey("fakeTokenPlaceholder") - .withRegion("ljcf")) - .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) - .withSingleSignOnProperties( - new NewRelicSingleSignOnProperties() - .withSingleSignOnState(SingleSignOnStates.INITIAL) - .withEnterpriseAppId("kwiwfz") - .withSingleSignOnUrl("kvseueuljsxmfwpqctz") - .withProvisioningState(ProvisioningState.ACCEPTED))) - .withUserInfo( - new UserInfo() - .withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") - .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc") - .withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") - .withPhoneNumber("krf") - .withCountry("hslqnwdanrconqyekwbnttaetv")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("tbbiaga") - .withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.LIFTR) - .create(); + manager.monitors().define("cdlymktqw").withRegion("k").withExistingResourceGroup("rgNewRelic") + .withTags(mapOf("key6976", "fakeTokenPlaceholder")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("key8903", new UserAssignedIdentity()))) + .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("vcscxlncofcuduadesd") + .withAccountInfo(new AccountInfo() + .withAccountId("xhqmg").withIngestionKey("fakeTokenPlaceholder").withRegion("ljcf")) + .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) + .withSingleSignOnProperties(new NewRelicSingleSignOnProperties() + .withSingleSignOnState(SingleSignOnStates.INITIAL).withEnterpriseAppId("kwiwfz") + .withSingleSignOnUrl("kvseueuljsxmfwpqctz").withProvisioningState(ProvisioningState.ACCEPTED))) + .withUserInfo(new UserInfo().withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") + .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc").withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") + .withPhoneNumber("krf").withCountry("hslqnwdanrconqyekwbnttaetv")) + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.YEARLY) + .withPlanDetails("tbbiaga").withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) + .withOrgCreationSource(OrgCreationSource.LIFTR).withAccountCreationSource(AccountCreationSource.LIFTR) + .withSubscriptionState("Suspended").withSaaSAzureSubscriptionStatus("Subscribed").create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -159,44 +344,38 @@ public final class MonitorsCreateOrUpdateSamples { ### Monitors_Delete ```java -/** Samples for Monitors Delete. */ +/** + * Samples for Monitors Delete. + */ public final class MonitorsDeleteSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Delete_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Delete_MinimumSet_Gen.json */ /** * Sample code: Monitors_Delete_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsDeleteMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .delete( - "rgopenapi", - "ruxvg@xqkmdhrnoo.hlmbpm", - "ipxmlcbonyxtolzejcjshkmlron", - com.azure.core.util.Context.NONE); + manager.monitors().delete("rgopenapi", "ruxvg@xqkmdhrnoo.hlmbpm", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Delete_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Delete_MaximumSet_Gen.json */ /** * Sample code: Monitors_Delete_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsDeleteMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .delete( - "rgopenapi", - "ruxvg@xqkmdhrnoo.hlmbpm", - "ipxmlcbonyxtolzejcjshkmlron", - com.azure.core.util.Context.NONE); + manager.monitors().delete("rgopenapi", "ruxvg@xqkmdhrnoo.hlmbpm", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } ``` @@ -204,18 +383,21 @@ public final class MonitorsDeleteSamples { ### Monitors_GetByResourceGroup ```java -/** Samples for Monitors GetByResourceGroup. */ +/** + * Samples for Monitors GetByResourceGroup. + */ public final class MonitorsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Get_MaximumSet_Gen.json */ /** * Sample code: Monitors_Get_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void monitorsGetMaximumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + public static void + monitorsGetMaximumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { manager.monitors().getByResourceGroupWithResponse("rgNewRelic", "cdlymktqw", com.azure.core.util.Context.NONE); } } @@ -226,44 +408,38 @@ public final class MonitorsGetByResourceGroupSamples { ```java import com.azure.resourcemanager.newrelicobservability.models.MetricsRequest; -/** Samples for Monitors GetMetricRules. */ +/** + * Samples for Monitors GetMetricRules. + */ public final class MonitorsGetMetricRulesSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricRules_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricRules_MinimumSet_Gen.json */ /** * Sample code: Monitors_GetMetricRules_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricRulesMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricRulesWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricRulesWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricRules_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricRules_MaximumSet_Gen.json */ /** * Sample code: Monitors_GetMetricRules_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricRulesMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricRulesWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricRulesWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), com.azure.core.util.Context.NONE); } } ``` @@ -274,46 +450,44 @@ public final class MonitorsGetMetricRulesSamples { import com.azure.resourcemanager.newrelicobservability.models.MetricsStatusRequest; import java.util.Arrays; -/** Samples for Monitors GetMetricStatus. */ +/** + * Samples for Monitors GetMetricStatus. + */ public final class MonitorsGetMetricStatusSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricStatus_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricStatus_MinimumSet_Gen.json */ /** * Sample code: Monitors_GetMetricStatus_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricStatusMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricStatusWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsStatusRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricStatusWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsStatusRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricStatus_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricStatus_MaximumSet_Gen.json */ /** * Sample code: Monitors_GetMetricStatus_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricStatusMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricStatusWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsStatusRequest() - .withAzureResourceIds(Arrays.asList("enfghpfw")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricStatusWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsStatusRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } ``` @@ -321,14 +495,17 @@ public final class MonitorsGetMetricStatusSamples { ### Monitors_List ```java -/** Samples for Monitors List. */ +/** + * Samples for Monitors List. + */ public final class MonitorsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListBySubscription_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListBySubscription_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListBySubscription_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListBySubscriptionMaximumSetGen( @@ -344,46 +521,44 @@ public final class MonitorsListSamples { import com.azure.resourcemanager.newrelicobservability.models.AppServicesGetRequest; import java.util.Arrays; -/** Samples for Monitors ListAppServices. */ +/** + * Samples for Monitors ListAppServices. + */ public final class MonitorsListAppServicesSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListAppServices_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListAppServices_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListAppServices_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListAppServicesMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listAppServices( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new AppServicesGetRequest() - .withAzureResourceIds(Arrays.asList("pvzrksrmzowobuhxpwiotnpcvjbu")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listAppServices("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new AppServicesGetRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListAppServices_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListAppServices_MinimumSet_Gen.json */ /** * Sample code: Monitors_ListAppServices_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListAppServicesMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listAppServices( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new AppServicesGetRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listAppServices("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new AppServicesGetRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } ``` @@ -391,14 +566,17 @@ public final class MonitorsListAppServicesSamples { ### Monitors_ListByResourceGroup ```java -/** Samples for Monitors ListByResourceGroup. */ +/** + * Samples for Monitors ListByResourceGroup. + */ public final class MonitorsListByResourceGroupSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListByResourceGroup_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListByResourceGroup_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListByResourceGroup_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListByResourceGroupMaximumSetGen( @@ -414,48 +592,66 @@ public final class MonitorsListByResourceGroupSamples { import com.azure.resourcemanager.newrelicobservability.models.HostsGetRequest; import java.util.Arrays; -/** Samples for Monitors ListHosts. */ +/** + * Samples for Monitors ListHosts. + */ public final class MonitorsListHostsSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListHosts_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListHosts_MinimumSet_Gen.json */ /** * Sample code: Monitors_ListHosts_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListHostsMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listHosts( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - new HostsGetRequest() - .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listHosts( + "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", new HostsGetRequest() + .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")).withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListHosts_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListHosts_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListHosts_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListHostsMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listHosts( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - new HostsGetRequest() - .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listHosts( + "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", new HostsGetRequest() + .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")).withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); + } +} +``` + +### Monitors_ListLinkedResources + +```java +/** + * Samples for Monitors ListLinkedResources. + */ +public final class MonitorsListLinkedResourcesSamples { + /* + * x-ms-original-file: + * specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/LinkedResources_List. + * json + */ + /** + * Sample code: Monitors_ListLinkedResources. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsListLinkedResources( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitors().listLinkedResources("myResourceGroup", "myMonitor", com.azure.core.util.Context.NONE); } } ``` @@ -463,36 +659,38 @@ public final class MonitorsListHostsSamples { ### Monitors_ListMonitoredResources ```java -/** Samples for Monitors ListMonitoredResources. */ +/** + * Samples for Monitors ListMonitoredResources. + */ public final class MonitorsListMonitoredResourcesSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListMonitoredResources_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListMonitoredResources_MinimumSet_Gen.json */ /** * Sample code: Monitors_ListMonitoredResources_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListMonitoredResourcesMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListMonitoredResources_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListMonitoredResources_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListMonitoredResources_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListMonitoredResourcesMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } ``` @@ -506,53 +704,44 @@ import com.azure.resourcemanager.newrelicobservability.models.SwitchBillingReque import com.azure.resourcemanager.newrelicobservability.models.UsageType; import java.time.OffsetDateTime; -/** Samples for Monitors SwitchBilling. */ +/** + * Samples for Monitors SwitchBilling. + */ public final class MonitorsSwitchBillingSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_SwitchBilling_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_SwitchBilling_MinimumSet_Gen.json */ /** * Sample code: Monitors_SwitchBilling_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsSwitchBillingMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .switchBillingWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new SwitchBillingRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().switchBillingWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new SwitchBillingRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_SwitchBilling_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_SwitchBilling_MaximumSet_Gen.json */ /** * Sample code: Monitors_SwitchBilling_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsSwitchBillingMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .switchBillingWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new SwitchBillingRequest() - .withAzureResourceId("enfghpfw") - .withOrganizationId("k") - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("tbbiaga") - .withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().switchBillingWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new SwitchBillingRequest().withAzureResourceId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz") + .withOrganizationId("k") + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.YEARLY) + .withPlanDetails("tbbiaga").withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } ``` @@ -580,63 +769,43 @@ import java.time.OffsetDateTime; import java.util.HashMap; import java.util.Map; -/** Samples for Monitors Update. */ +/** + * Samples for Monitors Update. + */ public final class MonitorsUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Update_MaximumSet_Gen.json */ /** * Sample code: Monitors_Update_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - NewRelicMonitorResource resource = - manager - .monitors() - .getByResourceGroupWithResponse("rgNewRelic", "cdlymktqw", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("key164", "jqakdrrmmyzytqu")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf("key8903", new UserAssignedIdentity()))) - .withNewRelicAccountProperties( - new NewRelicAccountProperties() - .withUserId("vcscxlncofcuduadesd") - .withAccountInfo( - new AccountInfo() - .withAccountId("xhqmg") - .withIngestionKey("fakeTokenPlaceholder") - .withRegion("ljcf")) - .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) - .withSingleSignOnProperties( - new NewRelicSingleSignOnProperties() - .withSingleSignOnState(SingleSignOnStates.INITIAL) - .withEnterpriseAppId("kwiwfz") - .withSingleSignOnUrl("kvseueuljsxmfwpqctz") - .withProvisioningState(ProvisioningState.ACCEPTED))) - .withUserInfo( - new UserInfo() - .withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") - .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc") - .withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") - .withPhoneNumber("krf") - .withCountry("hslqnwdanrconqyekwbnttaetv")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("tbbiaga") - .withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.LIFTR) + NewRelicMonitorResource resource = manager.monitors() + .getByResourceGroupWithResponse("rgNewRelic", "cdlymktqw", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("key164", "fakeTokenPlaceholder")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("key8903", new UserAssignedIdentity()))) + .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("vcscxlncofcuduadesd") + .withAccountInfo(new AccountInfo() + .withAccountId("xhqmg").withIngestionKey("fakeTokenPlaceholder").withRegion("ljcf")) + .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) + .withSingleSignOnProperties(new NewRelicSingleSignOnProperties() + .withSingleSignOnState(SingleSignOnStates.INITIAL).withEnterpriseAppId("kwiwfz") + .withSingleSignOnUrl("kvseueuljsxmfwpqctz").withProvisioningState(ProvisioningState.ACCEPTED))) + .withUserInfo(new UserInfo().withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") + .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc").withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") + .withPhoneNumber("krf").withCountry("hslqnwdanrconqyekwbnttaetv")) + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.YEARLY) + .withPlanDetails("tbbiaga").withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) + .withOrgCreationSource(OrgCreationSource.LIFTR).withAccountCreationSource(AccountCreationSource.LIFTR) .apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -653,36 +822,38 @@ public final class MonitorsUpdateSamples { ### Monitors_VmHostPayload ```java -/** Samples for Monitors VmHostPayload. */ +/** + * Samples for Monitors VmHostPayload. + */ public final class MonitorsVmHostPayloadSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_VmHostPayload_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_VmHostPayload_MinimumSet_Gen.json */ /** * Sample code: Monitors_VmHostPayload_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsVmHostPayloadMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_VmHostPayload_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_VmHostPayload_MaximumSet_Gen.json */ /** * Sample code: Monitors_VmHostPayload_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsVmHostPayloadMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } ``` @@ -690,14 +861,17 @@ public final class MonitorsVmHostPayloadSamples { ### Operations_List ```java -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Operations_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Operations_List_MinimumSet_Gen.json */ /** * Sample code: Operations_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void operationsListMinimumSetGen( @@ -706,11 +880,12 @@ public final class OperationsListSamples { } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Operations_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Operations_List_MaximumSet_Gen.json */ /** * Sample code: Operations_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void operationsListMaximumSetGen( @@ -723,14 +898,17 @@ public final class OperationsListSamples { ### Organizations_List ```java -/** Samples for Organizations List. */ +/** + * Samples for Organizations List. + */ public final class OrganizationsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Organizations_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Organizations_List_MinimumSet_Gen.json */ /** * Sample code: Organizations_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void organizationsListMinimumSetGen( @@ -739,11 +917,12 @@ public final class OrganizationsListSamples { } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Organizations_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Organizations_List_MaximumSet_Gen.json */ /** * Sample code: Organizations_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void organizationsListMaximumSetGen( @@ -756,32 +935,36 @@ public final class OrganizationsListSamples { ### Plans_List ```java -/** Samples for Plans List. */ +/** + * Samples for Plans List. + */ public final class PlansListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Plans_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Plans_List_MaximumSet_Gen.json */ /** * Sample code: Plans_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void plansListMaximumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + public static void + plansListMaximumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { manager.plans().list("pwuxgvrmkk", "hilawwjz", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Plans_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Plans_List_MinimumSet_Gen.json */ /** * Sample code: Plans_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void plansListMinimumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager.plans().list(null, null, com.azure.core.util.Context.NONE); + public static void + plansListMinimumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.plans().list(null, "hilawwjz", com.azure.core.util.Context.NONE); } } ``` @@ -798,62 +981,48 @@ import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLo import com.azure.resourcemanager.newrelicobservability.models.TagAction; import java.util.Arrays; -/** Samples for TagRules CreateOrUpdate. */ +/** + * Samples for TagRules CreateOrUpdate. + */ public final class TagRulesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_CreateOrUpdate_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_CreateOrUpdate_MaximumSet_Gen.json */ /** * Sample code: TagRules_CreateOrUpdate_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesCreateOrUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .define("bxcantgzggsepbhqmedjqyrqeezmfb") + manager.tagRules().define("bxcantgzggsepbhqmedjqyrqeezmfb") .withExistingMonitor("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron") - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.ENABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE)))) - .withMetricRules( - new MetricRulesInner() - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE))) - .withUserEmail("test@testing.com")) + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner() + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE))) + .withUserEmail("test@testing.com")) .create(); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_CreateOrUpdate_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_CreateOrUpdate_MinimumSet_Gen.json */ /** * Sample code: TagRules_CreateOrUpdate_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesCreateOrUpdateMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .define("bxcantgzggsepbhqmedjqyrqeezmfb") - .withExistingMonitor("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron") - .create(); + manager.tagRules().define("bxcantgzggsepbhqmedjqyrqeezmfb") + .withExistingMonitor("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron").create(); } } ``` @@ -861,44 +1030,38 @@ public final class TagRulesCreateOrUpdateSamples { ### TagRules_Delete ```java -/** Samples for TagRules Delete. */ +/** + * Samples for TagRules Delete. + */ public final class TagRulesDeleteSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Delete_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Delete_MinimumSet_Gen.json */ /** * Sample code: TagRules_Delete_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesDeleteMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .delete( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + manager.tagRules().delete("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Delete_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Delete_MaximumSet_Gen.json */ /** * Sample code: TagRules_Delete_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesDeleteMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .delete( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + manager.tagRules().delete("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } } ``` @@ -906,44 +1069,38 @@ public final class TagRulesDeleteSamples { ### TagRules_Get ```java -/** Samples for TagRules Get. */ +/** + * Samples for TagRules Get. + */ public final class TagRulesGetSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Get_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Get_MinimumSet_Gen.json */ /** * Sample code: TagRules_Get_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void tagRulesGetMinimumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + public static void + tagRulesGetMinimumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Get_MaximumSet_Gen.json */ /** * Sample code: TagRules_Get_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void tagRulesGetMaximumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + public static void + tagRulesGetMaximumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } } ``` @@ -951,38 +1108,38 @@ public final class TagRulesGetSamples { ### TagRules_ListByNewRelicMonitorResource ```java -/** Samples for TagRules ListByNewRelicMonitorResource. */ +/** + * Samples for TagRules ListByNewRelicMonitorResource. + */ public final class TagRulesListByNewRelicMonitorResourceSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_ListByNewRelicMonitorResource_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_ListByNewRelicMonitorResource_MinimumSet_Gen.json */ /** * Sample code: TagRules_ListByNewRelicMonitorResource_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesListByNewRelicMonitorResourceMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .listByNewRelicMonitorResource( - "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.tagRules().listByNewRelicMonitorResource("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_ListByNewRelicMonitorResource_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_ListByNewRelicMonitorResource_MaximumSet_Gen.json */ /** * Sample code: TagRules_ListByNewRelicMonitorResource_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesListByNewRelicMonitorResourceMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .listByNewRelicMonitorResource( - "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.tagRules().listByNewRelicMonitorResource("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } ``` @@ -1000,73 +1157,49 @@ import com.azure.resourcemanager.newrelicobservability.models.TagAction; import com.azure.resourcemanager.newrelicobservability.models.TagRule; import java.util.Arrays; -/** Samples for TagRules Update. */ +/** + * Samples for TagRules Update. + */ public final class TagRulesUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Update_MaximumSet_Gen.json */ /** * Sample code: TagRules_Update_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - TagRule resource = - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.ENABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE)))) - .withMetricRules( - new MetricRulesInner() - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE))) - .withUserEmail("test@testing.com")) + TagRule resource = manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + "bxcantgzggsepbhqmedjqyrqeezmfb", com.azure.core.util.Context.NONE).getValue(); + resource.update() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner() + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE))) + .withUserEmail("test@testing.com")) .apply(); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Update_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Update_MinimumSet_Gen.json */ /** * Sample code: TagRules_Update_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesUpdateMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - TagRule resource = - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE) - .getValue(); + TagRule resource = manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + "bxcantgzggsepbhqmedjqyrqeezmfb", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/pom.xml b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/pom.xml index 8f4a444af767e..2a631631be400 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/pom.xml +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-newrelicobservability - 1.1.0-beta.1 + 1.2.0-beta.1 jar Microsoft Azure SDK for NewRelicObservability Management - This package contains Microsoft Azure SDK for NewRelicObservability Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Package tag package-2022-07-01. + This package contains Microsoft Azure SDK for NewRelicObservability Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Package tag package-2024-01-01. https://github.com/Azure/azure-sdk-for-java @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/NewRelicObservabilityManager.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/NewRelicObservabilityManager.java index f4814b4ee32f6..39c86b6333721 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/NewRelicObservabilityManager.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/NewRelicObservabilityManager.java @@ -25,6 +25,9 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.newrelicobservability.fluent.NewRelicObservability; import com.azure.resourcemanager.newrelicobservability.implementation.AccountsImpl; +import com.azure.resourcemanager.newrelicobservability.implementation.BillingInfoesImpl; +import com.azure.resourcemanager.newrelicobservability.implementation.ConnectedPartnerResourcesImpl; +import com.azure.resourcemanager.newrelicobservability.implementation.MonitoredSubscriptionsImpl; import com.azure.resourcemanager.newrelicobservability.implementation.MonitorsImpl; import com.azure.resourcemanager.newrelicobservability.implementation.NewRelicObservabilityBuilder; import com.azure.resourcemanager.newrelicobservability.implementation.OperationsImpl; @@ -32,6 +35,9 @@ import com.azure.resourcemanager.newrelicobservability.implementation.PlansImpl; import com.azure.resourcemanager.newrelicobservability.implementation.TagRulesImpl; import com.azure.resourcemanager.newrelicobservability.models.Accounts; +import com.azure.resourcemanager.newrelicobservability.models.BillingInfoes; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResources; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptions; import com.azure.resourcemanager.newrelicobservability.models.Monitors; import com.azure.resourcemanager.newrelicobservability.models.Operations; import com.azure.resourcemanager.newrelicobservability.models.Organizations; @@ -44,7 +50,9 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to NewRelicObservabilityManager. */ +/** + * Entry point to NewRelicObservabilityManager. + */ public final class NewRelicObservabilityManager { private Operations operations; @@ -56,26 +64,28 @@ public final class NewRelicObservabilityManager { private Plans plans; + private BillingInfoes billingInfoes; + + private ConnectedPartnerResources connectedPartnerResources; + private TagRules tagRules; + private MonitoredSubscriptions monitoredSubscriptions; + private final NewRelicObservability clientObject; - private NewRelicObservabilityManager( - HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { + private NewRelicObservabilityManager(HttpPipeline httpPipeline, AzureProfile profile, + Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new NewRelicObservabilityBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new NewRelicObservabilityBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of NewRelicObservability service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the NewRelicObservability service API instance. @@ -88,7 +98,7 @@ public static NewRelicObservabilityManager authenticate(TokenCredential credenti /** * Creates an instance of NewRelicObservability service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the NewRelicObservability service API instance. @@ -101,14 +111,16 @@ public static NewRelicObservabilityManager authenticate(HttpPipeline httpPipelin /** * Gets a Configurable instance that can be used to create NewRelicObservabilityManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new NewRelicObservabilityManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -180,8 +192,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -198,8 +210,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -219,21 +231,12 @@ public NewRelicObservabilityManager authenticate(TokenCredential credential, Azu Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.newrelicobservability") - .append("/") - .append("1.0.0"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.newrelicobservability") + .append("/").append("1.1.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -252,38 +255,25 @@ public NewRelicObservabilityManager authenticate(TokenCredential credential, Azu policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new NewRelicObservabilityManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -295,7 +285,7 @@ public Operations operations() { /** * Gets the resource collection API of Accounts. - * + * * @return Resource collection API of Accounts. */ public Accounts accounts() { @@ -307,7 +297,7 @@ public Accounts accounts() { /** * Gets the resource collection API of Monitors. It manages NewRelicMonitorResource. - * + * * @return Resource collection API of Monitors. */ public Monitors monitors() { @@ -319,7 +309,7 @@ public Monitors monitors() { /** * Gets the resource collection API of Organizations. - * + * * @return Resource collection API of Organizations. */ public Organizations organizations() { @@ -331,7 +321,7 @@ public Organizations organizations() { /** * Gets the resource collection API of Plans. - * + * * @return Resource collection API of Plans. */ public Plans plans() { @@ -341,9 +331,34 @@ public Plans plans() { return plans; } + /** + * Gets the resource collection API of BillingInfoes. + * + * @return Resource collection API of BillingInfoes. + */ + public BillingInfoes billingInfoes() { + if (this.billingInfoes == null) { + this.billingInfoes = new BillingInfoesImpl(clientObject.getBillingInfoes(), this); + } + return billingInfoes; + } + + /** + * Gets the resource collection API of ConnectedPartnerResources. + * + * @return Resource collection API of ConnectedPartnerResources. + */ + public ConnectedPartnerResources connectedPartnerResources() { + if (this.connectedPartnerResources == null) { + this.connectedPartnerResources + = new ConnectedPartnerResourcesImpl(clientObject.getConnectedPartnerResources(), this); + } + return connectedPartnerResources; + } + /** * Gets the resource collection API of TagRules. It manages TagRule. - * + * * @return Resource collection API of TagRules. */ public TagRules tagRules() { @@ -354,8 +369,23 @@ public TagRules tagRules() { } /** - * @return Wrapped service client NewRelicObservability providing direct access to the underlying auto-generated API - * implementation, based on Azure REST API. + * Gets the resource collection API of MonitoredSubscriptions. It manages MonitoredSubscriptionProperties. + * + * @return Resource collection API of MonitoredSubscriptions. + */ + public MonitoredSubscriptions monitoredSubscriptions() { + if (this.monitoredSubscriptions == null) { + this.monitoredSubscriptions + = new MonitoredSubscriptionsImpl(clientObject.getMonitoredSubscriptions(), this); + } + return monitoredSubscriptions; + } + + /** + * Gets wrapped service client NewRelicObservability providing direct access to the underlying auto-generated API + * implementation, based on Azure REST API. + * + * @return Wrapped service client NewRelicObservability. */ public NewRelicObservability serviceClient() { return this.clientObject; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/AccountsClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/AccountsClient.java index bebbcf51a9f01..c8897603e468a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/AccountsClient.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/AccountsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.newrelicobservability.fluent.models.AccountResourceInner; -/** An instance of this class provides access to all the operations defined in AccountsClient. */ +/** + * An instance of this class provides access to all the operations defined in AccountsClient. + */ public interface AccountsClient { /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -27,7 +29,7 @@ public interface AccountsClient { /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/BillingInfoesClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/BillingInfoesClient.java new file mode 100644 index 0000000000000..24fbecdcd7ca2 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/BillingInfoesClient.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner; + +/** + * An instance of this class provides access to all the operations defined in BillingInfoesClient. + */ +public interface BillingInfoesClient { + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String monitorName, Context context); + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + BillingInfoResponseInner get(String resourceGroupName, String monitorName); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/ConnectedPartnerResourcesClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/ConnectedPartnerResourcesClient.java new file mode 100644 index 0000000000000..fdb42682cc57d --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/ConnectedPartnerResourcesClient.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; + +/** + * An instance of this class provides access to all the operations defined in ConnectedPartnerResourcesClient. + */ +public interface ConnectedPartnerResourcesClient { + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String monitorName); + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String monitorName, + String body, Context context); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitoredSubscriptionsClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitoredSubscriptionsClient.java new file mode 100644 index 0000000000000..3c6d45adb7000 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitoredSubscriptionsClient.java @@ -0,0 +1,272 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * An instance of this class provides access to all the operations defined in MonitoredSubscriptionsClient. + */ +public interface MonitoredSubscriptionsClient { + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String monitorName); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String monitorName, + Context context); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + MonitoredSubscriptionPropertiesInner get(String resourceGroupName, String monitorName, + ConfigurationName configurationName); + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName); + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body, Context context); + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + MonitoredSubscriptionPropertiesInner createorUpdate(String resourceGroupName, String monitorName, + ConfigurationName configurationName); + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + MonitoredSubscriptionPropertiesInner createorUpdate(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, MonitoredSubscriptionPropertiesInner> + beginUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, MonitoredSubscriptionPropertiesInner> beginUpdate( + String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body, Context context); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + MonitoredSubscriptionPropertiesInner update(String resourceGroupName, String monitorName, + ConfigurationName configurationName); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + MonitoredSubscriptionPropertiesInner update(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, + ConfigurationName configurationName); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName, Context context); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitorsClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitorsClient.java index 974a640345eb2..549902266fa9a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitorsClient.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/MonitorsClient.java @@ -12,6 +12,7 @@ import com.azure.core.util.Context; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.newrelicobservability.fluent.models.AppServiceInfoInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricsStatusResponseInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredResourceInner; @@ -26,62 +27,64 @@ import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResourceUpdate; import com.azure.resourcemanager.newrelicobservability.models.SwitchBillingRequest; -/** An instance of this class provides access to all the operations defined in MonitorsClient. */ +/** + * An instance of this class provides access to all the operations defined in MonitorsClient. + */ public interface MonitorsClient { /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByResourceGroup(String resourceGroupName); /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByResourceGroup(String resourceGroupName, Context context); /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -91,12 +94,12 @@ public interface MonitorsClient { * @return a NewRelicMonitorResource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String monitorName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String monitorName, + Context context); /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -109,7 +112,7 @@ Response getByResourceGroupWithResponse( /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -119,12 +122,12 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, NewRelicMonitorResourceInner> beginCreateOrUpdate( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource); + SyncPoller, NewRelicMonitorResourceInner> + beginCreateOrUpdate(String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource); /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -140,7 +143,7 @@ SyncPoller, NewRelicMonitorResourceInne /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -150,12 +153,12 @@ SyncPoller, NewRelicMonitorResourceInne * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - NewRelicMonitorResourceInner createOrUpdate( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource); + NewRelicMonitorResourceInner createOrUpdate(String resourceGroupName, String monitorName, + NewRelicMonitorResourceInner resource); /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -166,12 +169,12 @@ NewRelicMonitorResourceInner createOrUpdate( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - NewRelicMonitorResourceInner createOrUpdate( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource, Context context); + NewRelicMonitorResourceInner createOrUpdate(String resourceGroupName, String monitorName, + NewRelicMonitorResourceInner resource, Context context); /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -182,12 +185,12 @@ NewRelicMonitorResourceInner createOrUpdate( * @return a Monitor Resource by NewRelic along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties, Context context); + Response updateWithResponse(String resourceGroupName, String monitorName, + NewRelicMonitorResourceUpdate properties, Context context); /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -197,12 +200,12 @@ Response updateWithResponse( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - NewRelicMonitorResourceInner update( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties); + NewRelicMonitorResourceInner update(String resourceGroupName, String monitorName, + NewRelicMonitorResourceUpdate properties); /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -216,7 +219,7 @@ NewRelicMonitorResourceInner update( /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -227,12 +230,12 @@ NewRelicMonitorResourceInner update( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String userEmail, String monitorName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String userEmail, String monitorName, + Context context); /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -245,7 +248,7 @@ SyncPoller, Void> beginDelete( /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -259,7 +262,7 @@ SyncPoller, Void> beginDelete( /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -270,12 +273,12 @@ SyncPoller, Void> beginDelete( * @return metric rules along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getMetricRulesWithResponse( - String resourceGroupName, String monitorName, MetricsRequest request, Context context); + Response getMetricRulesWithResponse(String resourceGroupName, String monitorName, + MetricsRequest request, Context context); /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -289,7 +292,7 @@ Response getMetricRulesWithResponse( /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -300,12 +303,12 @@ Response getMetricRulesWithResponse( * @return metric status along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getMetricStatusWithResponse( - String resourceGroupName, String monitorName, MetricsStatusRequest request, Context context); + Response getMetricStatusWithResponse(String resourceGroupName, String monitorName, + MetricsStatusRequest request, Context context); /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -315,12 +318,12 @@ Response getMetricStatusWithResponse( * @return metric status. */ @ServiceMethod(returns = ReturnType.SINGLE) - MetricsStatusResponseInner getMetricStatus( - String resourceGroupName, String monitorName, MetricsStatusRequest request); + MetricsStatusResponseInner getMetricStatus(String resourceGroupName, String monitorName, + MetricsStatusRequest request); /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -330,12 +333,12 @@ MetricsStatusResponseInner getMetricStatus( * @return response of a list app services Operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request); + PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request); /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -346,12 +349,12 @@ PagedIterable listAppServices( * @return response of a list app services Operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request, Context context); + PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request, Context context); /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -362,12 +365,12 @@ PagedIterable listAppServices( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - MonitorsSwitchBillingResponse switchBillingWithResponse( - String resourceGroupName, String monitorName, SwitchBillingRequest request, Context context); + MonitorsSwitchBillingResponse switchBillingWithResponse(String resourceGroupName, String monitorName, + SwitchBillingRequest request, Context context); /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -377,12 +380,12 @@ MonitorsSwitchBillingResponse switchBillingWithResponse( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - NewRelicMonitorResourceInner switchBilling( - String resourceGroupName, String monitorName, SwitchBillingRequest request); + NewRelicMonitorResourceInner switchBilling(String resourceGroupName, String monitorName, + SwitchBillingRequest request); /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -396,7 +399,7 @@ NewRelicMonitorResourceInner switchBilling( /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -407,42 +410,70 @@ NewRelicMonitorResourceInner switchBilling( * @return response of a list VM Host Operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listHosts( - String resourceGroupName, String monitorName, HostsGetRequest request, Context context); + PagedIterable listHosts(String resourceGroupName, String monitorName, HostsGetRequest request, + Context context); /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listMonitoredResources(String resourceGroupName, String monitorName); /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listMonitoredResources(String resourceGroupName, String monitorName, + Context context); + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listLinkedResources(String resourceGroupName, String monitorName); + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return response of a list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listMonitoredResources( - String resourceGroupName, String monitorName, Context context); + PagedIterable listLinkedResources(String resourceGroupName, String monitorName, + Context context); /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -452,12 +483,12 @@ PagedIterable listMonitoredResources( * @return response of payload to be passed while installing VM agent along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response vmHostPayloadWithResponse( - String resourceGroupName, String monitorName, Context context); + Response vmHostPayloadWithResponse(String resourceGroupName, String monitorName, + Context context); /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/NewRelicObservability.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/NewRelicObservability.java index 5057a6922bb54..7e0ff48dea9da 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/NewRelicObservability.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/NewRelicObservability.java @@ -7,82 +7,105 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for NewRelicObservability class. */ +/** + * The interface for NewRelicObservability class. + */ public interface NewRelicObservability { /** * Gets The ID of the target subscription. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the AccountsClient object to access its operations. - * + * * @return the AccountsClient object. */ AccountsClient getAccounts(); /** * Gets the MonitorsClient object to access its operations. - * + * * @return the MonitorsClient object. */ MonitorsClient getMonitors(); /** * Gets the OrganizationsClient object to access its operations. - * + * * @return the OrganizationsClient object. */ OrganizationsClient getOrganizations(); /** * Gets the PlansClient object to access its operations. - * + * * @return the PlansClient object. */ PlansClient getPlans(); + /** + * Gets the BillingInfoesClient object to access its operations. + * + * @return the BillingInfoesClient object. + */ + BillingInfoesClient getBillingInfoes(); + + /** + * Gets the ConnectedPartnerResourcesClient object to access its operations. + * + * @return the ConnectedPartnerResourcesClient object. + */ + ConnectedPartnerResourcesClient getConnectedPartnerResources(); + /** * Gets the TagRulesClient object to access its operations. - * + * * @return the TagRulesClient object. */ TagRulesClient getTagRules(); + + /** + * Gets the MonitoredSubscriptionsClient object to access its operations. + * + * @return the MonitoredSubscriptionsClient object. + */ + MonitoredSubscriptionsClient getMonitoredSubscriptions(); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OperationsClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OperationsClient.java index 030cebabc15ae..a307b7f3ffeff 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OperationsClient.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OperationsClient.java @@ -10,28 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.newrelicobservability.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * List the operations for the provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OrganizationsClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OrganizationsClient.java index 022c25a18cfe6..491a4703922aa 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OrganizationsClient.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/OrganizationsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.newrelicobservability.fluent.models.OrganizationResourceInner; -/** An instance of this class provides access to all the operations defined in OrganizationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OrganizationsClient. + */ public interface OrganizationsClient { /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -27,7 +29,7 @@ public interface OrganizationsClient { /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/PlansClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/PlansClient.java index 5fdc8edde21f2..0b27f3e793e2a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/PlansClient.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/PlansClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.newrelicobservability.fluent.models.PlanDataResourceInner; -/** An instance of this class provides access to all the operations defined in PlansClient. */ +/** + * An instance of this class provides access to all the operations defined in PlansClient. + */ public interface PlansClient { /** * List plans data. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface PlansClient { /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @param context The context to associate with this operation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/TagRulesClient.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/TagRulesClient.java index b7d971519c4e7..e3d1e34d15e1c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/TagRulesClient.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/TagRulesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdateInner; -/** An instance of this class provides access to all the operations defined in TagRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in TagRulesClient. + */ public interface TagRulesClient { /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -31,7 +33,7 @@ public interface TagRulesClient { /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -41,12 +43,12 @@ public interface TagRulesClient { * @return the response of a TagRule list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByNewRelicMonitorResource( - String resourceGroupName, String monitorName, Context context); + PagedIterable listByNewRelicMonitorResource(String resourceGroupName, String monitorName, + Context context); /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -57,12 +59,12 @@ PagedIterable listByNewRelicMonitorResource( * @return a TagRule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String monitorName, String ruleSetName, Context context); + Response getWithResponse(String resourceGroupName, String monitorName, String ruleSetName, + Context context); /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -76,7 +78,7 @@ Response getWithResponse( /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -87,12 +89,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, TagRuleInner> beginCreateOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource); + SyncPoller, TagRuleInner> beginCreateOrUpdate(String resourceGroupName, String monitorName, + String ruleSetName, TagRuleInner resource); /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -104,12 +106,12 @@ SyncPoller, TagRuleInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, TagRuleInner> beginCreateOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context); + SyncPoller, TagRuleInner> beginCreateOrUpdate(String resourceGroupName, String monitorName, + String ruleSetName, TagRuleInner resource, Context context); /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -120,12 +122,12 @@ SyncPoller, TagRuleInner> beginCreateOrUpdate( * @return a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.SINGLE) - TagRuleInner createOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource); + TagRuleInner createOrUpdate(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleInner resource); /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -137,12 +139,12 @@ TagRuleInner createOrUpdate( * @return a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.SINGLE) - TagRuleInner createOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context); + TagRuleInner createOrUpdate(String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, + Context context); /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -156,7 +158,7 @@ TagRuleInner createOrUpdate( /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -167,12 +169,12 @@ TagRuleInner createOrUpdate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String monitorName, String ruleSetName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, String ruleSetName, + Context context); /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -185,7 +187,7 @@ SyncPoller, Void> beginDelete( /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -199,7 +201,7 @@ SyncPoller, Void> beginDelete( /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -211,16 +213,12 @@ SyncPoller, Void> beginDelete( * @return a tag rule belonging to NewRelic account along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String monitorName, - String ruleSetName, - TagRuleUpdateInner properties, - Context context); + Response updateWithResponse(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleUpdateInner properties, Context context); /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -231,6 +229,6 @@ Response updateWithResponse( * @return a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.SINGLE) - TagRuleInner update( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleUpdateInner properties); + TagRuleInner update(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleUpdateInner properties); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountProperties.java index fe7862b7a39e4..9c2d1eba9b28d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** List of all the New relic accounts for the given user. */ +/** + * List of all the New relic accounts for the given user. + */ @Fluent public final class AccountProperties { /* @@ -29,18 +31,20 @@ public final class AccountProperties { private String accountName; /* - * region + * Region where New Relic account is present */ @JsonProperty(value = "region") private String region; - /** Creates an instance of AccountProperties class. */ + /** + * Creates an instance of AccountProperties class. + */ public AccountProperties() { } /** * Get the organizationId property: organization id. - * + * * @return the organizationId value. */ public String organizationId() { @@ -49,7 +53,7 @@ public String organizationId() { /** * Set the organizationId property: organization id. - * + * * @param organizationId the organizationId value to set. * @return the AccountProperties object itself. */ @@ -60,7 +64,7 @@ public AccountProperties withOrganizationId(String organizationId) { /** * Get the accountId property: account id. - * + * * @return the accountId value. */ public String accountId() { @@ -69,7 +73,7 @@ public String accountId() { /** * Set the accountId property: account id. - * + * * @param accountId the accountId value to set. * @return the AccountProperties object itself. */ @@ -80,7 +84,7 @@ public AccountProperties withAccountId(String accountId) { /** * Get the accountName property: account name. - * + * * @return the accountName value. */ public String accountName() { @@ -89,7 +93,7 @@ public String accountName() { /** * Set the accountName property: account name. - * + * * @param accountName the accountName value to set. * @return the AccountProperties object itself. */ @@ -99,8 +103,8 @@ public AccountProperties withAccountName(String accountName) { } /** - * Get the region property: region. - * + * Get the region property: Region where New Relic account is present. + * * @return the region value. */ public String region() { @@ -108,8 +112,8 @@ public String region() { } /** - * Set the region property: region. - * + * Set the region property: Region where New Relic account is present. + * * @param region the region value to set. * @return the AccountProperties object itself. */ @@ -120,7 +124,7 @@ public AccountProperties withRegion(String region) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountResourceInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountResourceInner.java index 1ed1f744a4001..b9fea867ecf55 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountResourceInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AccountResourceInner.java @@ -9,7 +9,9 @@ import com.azure.core.management.SystemData; import com.fasterxml.jackson.annotation.JsonProperty; -/** The details of a account resource. */ +/** + * The details of a account resource. + */ @Fluent public final class AccountResourceInner extends ProxyResource { /* @@ -24,13 +26,15 @@ public final class AccountResourceInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of AccountResourceInner class. */ + /** + * Creates an instance of AccountResourceInner class. + */ public AccountResourceInner() { } /** * Get the innerProperties property: The resource-specific properties for this resource. - * + * * @return the innerProperties value. */ private AccountProperties innerProperties() { @@ -39,7 +43,7 @@ private AccountProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -48,7 +52,7 @@ public SystemData systemData() { /** * Get the organizationId property: organization id. - * + * * @return the organizationId value. */ public String organizationId() { @@ -57,7 +61,7 @@ public String organizationId() { /** * Set the organizationId property: organization id. - * + * * @param organizationId the organizationId value to set. * @return the AccountResourceInner object itself. */ @@ -71,7 +75,7 @@ public AccountResourceInner withOrganizationId(String organizationId) { /** * Get the accountId property: account id. - * + * * @return the accountId value. */ public String accountId() { @@ -80,7 +84,7 @@ public String accountId() { /** * Set the accountId property: account id. - * + * * @param accountId the accountId value to set. * @return the AccountResourceInner object itself. */ @@ -94,7 +98,7 @@ public AccountResourceInner withAccountId(String accountId) { /** * Get the accountName property: account name. - * + * * @return the accountName value. */ public String accountName() { @@ -103,7 +107,7 @@ public String accountName() { /** * Set the accountName property: account name. - * + * * @param accountName the accountName value to set. * @return the AccountResourceInner object itself. */ @@ -116,8 +120,8 @@ public AccountResourceInner withAccountName(String accountName) { } /** - * Get the region property: region. - * + * Get the region property: Region where New Relic account is present. + * * @return the region value. */ public String region() { @@ -125,8 +129,8 @@ public String region() { } /** - * Set the region property: region. - * + * Set the region property: Region where New Relic account is present. + * * @param region the region value to set. * @return the AccountResourceInner object itself. */ @@ -140,7 +144,7 @@ public AccountResourceInner withRegion(String region) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AppServiceInfoInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AppServiceInfoInner.java index a06da081e1979..6ebc8c67419cb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AppServiceInfoInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/AppServiceInfoInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of VM Resource having NewRelic OneAgent installed. */ +/** + * Details of VM Resource having NewRelic OneAgent installed. + */ @Fluent public final class AppServiceInfoInner { /* @@ -28,13 +30,15 @@ public final class AppServiceInfoInner { @JsonProperty(value = "agentStatus") private String agentStatus; - /** Creates an instance of AppServiceInfoInner class. */ + /** + * Creates an instance of AppServiceInfoInner class. + */ public AppServiceInfoInner() { } /** * Get the azureResourceId property: Azure App service resource ID. - * + * * @return the azureResourceId value. */ public String azureResourceId() { @@ -43,7 +47,7 @@ public String azureResourceId() { /** * Set the azureResourceId property: Azure App service resource ID. - * + * * @param azureResourceId the azureResourceId value to set. * @return the AppServiceInfoInner object itself. */ @@ -54,7 +58,7 @@ public AppServiceInfoInner withAzureResourceId(String azureResourceId) { /** * Get the agentVersion property: Version of the NewRelic agent installed on the App service. - * + * * @return the agentVersion value. */ public String agentVersion() { @@ -63,7 +67,7 @@ public String agentVersion() { /** * Set the agentVersion property: Version of the NewRelic agent installed on the App service. - * + * * @param agentVersion the agentVersion value to set. * @return the AppServiceInfoInner object itself. */ @@ -74,7 +78,7 @@ public AppServiceInfoInner withAgentVersion(String agentVersion) { /** * Get the agentStatus property: Status of the NewRelic agent installed on the App service. - * + * * @return the agentStatus value. */ public String agentStatus() { @@ -83,7 +87,7 @@ public String agentStatus() { /** * Set the agentStatus property: Status of the NewRelic agent installed on the App service. - * + * * @param agentStatus the agentStatus value to set. * @return the AppServiceInfoInner object itself. */ @@ -94,7 +98,7 @@ public AppServiceInfoInner withAgentStatus(String agentStatus) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/BillingInfoResponseInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/BillingInfoResponseInner.java new file mode 100644 index 0000000000000..835449f04a4f7 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/BillingInfoResponseInner.java @@ -0,0 +1,88 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.models.MarketplaceSaaSInfo; +import com.azure.resourcemanager.newrelicobservability.models.PartnerBillingEntity; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Marketplace Subscription and Organization details to which resource gets billed into. + */ +@Fluent +public final class BillingInfoResponseInner { + /* + * Marketplace Subscription details + */ + @JsonProperty(value = "marketplaceSaasInfo") + private MarketplaceSaaSInfo marketplaceSaasInfo; + + /* + * Partner Billing Entity details: Organization Info + */ + @JsonProperty(value = "partnerBillingEntity") + private PartnerBillingEntity partnerBillingEntity; + + /** + * Creates an instance of BillingInfoResponseInner class. + */ + public BillingInfoResponseInner() { + } + + /** + * Get the marketplaceSaasInfo property: Marketplace Subscription details. + * + * @return the marketplaceSaasInfo value. + */ + public MarketplaceSaaSInfo marketplaceSaasInfo() { + return this.marketplaceSaasInfo; + } + + /** + * Set the marketplaceSaasInfo property: Marketplace Subscription details. + * + * @param marketplaceSaasInfo the marketplaceSaasInfo value to set. + * @return the BillingInfoResponseInner object itself. + */ + public BillingInfoResponseInner withMarketplaceSaasInfo(MarketplaceSaaSInfo marketplaceSaasInfo) { + this.marketplaceSaasInfo = marketplaceSaasInfo; + return this; + } + + /** + * Get the partnerBillingEntity property: Partner Billing Entity details: Organization Info. + * + * @return the partnerBillingEntity value. + */ + public PartnerBillingEntity partnerBillingEntity() { + return this.partnerBillingEntity; + } + + /** + * Set the partnerBillingEntity property: Partner Billing Entity details: Organization Info. + * + * @param partnerBillingEntity the partnerBillingEntity value to set. + * @return the BillingInfoResponseInner object itself. + */ + public BillingInfoResponseInner withPartnerBillingEntity(PartnerBillingEntity partnerBillingEntity) { + this.partnerBillingEntity = partnerBillingEntity; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (marketplaceSaasInfo() != null) { + marketplaceSaasInfo().validate(); + } + if (partnerBillingEntity() != null) { + partnerBillingEntity().validate(); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/ConnectedPartnerResourcesListFormatInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/ConnectedPartnerResourcesListFormatInner.java new file mode 100644 index 0000000000000..19a6b659015e8 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/ConnectedPartnerResourcesListFormatInner.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourceProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Connected Partner Resources List Format. + */ +@Fluent +public final class ConnectedPartnerResourcesListFormatInner { + /* + * Connected Partner Resource Properties + */ + @JsonProperty(value = "properties") + private ConnectedPartnerResourceProperties properties; + + /** + * Creates an instance of ConnectedPartnerResourcesListFormatInner class. + */ + public ConnectedPartnerResourcesListFormatInner() { + } + + /** + * Get the properties property: Connected Partner Resource Properties. + * + * @return the properties value. + */ + public ConnectedPartnerResourceProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Connected Partner Resource Properties. + * + * @param properties the properties value to set. + * @return the ConnectedPartnerResourcesListFormatInner object itself. + */ + public ConnectedPartnerResourcesListFormatInner withProperties(ConnectedPartnerResourceProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/LinkedResourceInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/LinkedResourceInner.java new file mode 100644 index 0000000000000..d3f7375f257f8 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/LinkedResourceInner.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The definition of a linked resource. + */ +@Fluent +public final class LinkedResourceInner { + /* + * The ARM id of the linked resource. + */ + @JsonProperty(value = "id") + private String id; + + /** + * Creates an instance of LinkedResourceInner class. + */ + public LinkedResourceInner() { + } + + /** + * Get the id property: The ARM id of the linked resource. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: The ARM id of the linked resource. + * + * @param id the id value to set. + * @return the LinkedResourceInner object itself. + */ + public LinkedResourceInner withId(String id) { + this.id = id; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricRulesInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricRulesInner.java index 48c67894582cf..eadf1eb2a7474 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricRulesInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricRulesInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Set of rules for sending metrics for the Monitor resource. */ +/** + * Set of rules for sending metrics for the Monitor resource. + */ @Fluent public final class MetricRulesInner { /* @@ -31,13 +33,15 @@ public final class MetricRulesInner { @JsonProperty(value = "userEmail") private String userEmail; - /** Creates an instance of MetricRulesInner class. */ + /** + * Creates an instance of MetricRulesInner class. + */ public MetricRulesInner() { } /** * Get the sendMetrics property: Flag specifying if metrics should be sent for the Monitor resource. - * + * * @return the sendMetrics value. */ public SendMetricsStatus sendMetrics() { @@ -46,7 +50,7 @@ public SendMetricsStatus sendMetrics() { /** * Set the sendMetrics property: Flag specifying if metrics should be sent for the Monitor resource. - * + * * @param sendMetrics the sendMetrics value to set. * @return the MetricRulesInner object itself. */ @@ -57,7 +61,7 @@ public MetricRulesInner withSendMetrics(SendMetricsStatus sendMetrics) { /** * Get the filteringTags property: List of filtering tags to be used for capturing metrics. - * + * * @return the filteringTags value. */ public List filteringTags() { @@ -66,7 +70,7 @@ public List filteringTags() { /** * Set the filteringTags property: List of filtering tags to be used for capturing metrics. - * + * * @param filteringTags the filteringTags value to set. * @return the MetricRulesInner object itself. */ @@ -77,7 +81,7 @@ public MetricRulesInner withFilteringTags(List filteringTags) { /** * Get the userEmail property: User Email. - * + * * @return the userEmail value. */ public String userEmail() { @@ -86,7 +90,7 @@ public String userEmail() { /** * Set the userEmail property: User Email. - * + * * @param userEmail the userEmail value to set. * @return the MetricRulesInner object itself. */ @@ -97,7 +101,7 @@ public MetricRulesInner withUserEmail(String userEmail) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricsStatusResponseInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricsStatusResponseInner.java index 56ae936dc2f12..e3e0cb38499ec 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricsStatusResponseInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MetricsStatusResponseInner.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response of get metrics status Operation. */ +/** + * Response of get metrics status Operation. + */ @Fluent public final class MetricsStatusResponseInner { /* @@ -17,13 +19,15 @@ public final class MetricsStatusResponseInner { @JsonProperty(value = "azureResourceIds") private List azureResourceIds; - /** Creates an instance of MetricsStatusResponseInner class. */ + /** + * Creates an instance of MetricsStatusResponseInner class. + */ public MetricsStatusResponseInner() { } /** * Get the azureResourceIds property: Azure resource IDs. - * + * * @return the azureResourceIds value. */ public List azureResourceIds() { @@ -32,7 +36,7 @@ public List azureResourceIds() { /** * Set the azureResourceIds property: Azure resource IDs. - * + * * @param azureResourceIds the azureResourceIds value to set. * @return the MetricsStatusResponseInner object itself. */ @@ -43,7 +47,7 @@ public MetricsStatusResponseInner withAzureResourceIds(List azureResourc /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitorProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitorProperties.java index cfad9ad1adfe3..9684d76c19645 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitorProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitorProperties.java @@ -16,7 +16,9 @@ import com.azure.resourcemanager.newrelicobservability.models.UserInfo; import com.fasterxml.jackson.annotation.JsonProperty; -/** Properties specific to the NewRelic Monitor resource. */ +/** + * Properties specific to the NewRelic Monitor resource. + */ @Fluent public final class MonitorProperties { /* @@ -85,13 +87,27 @@ public final class MonitorProperties { @JsonProperty(value = "accountCreationSource") private AccountCreationSource accountCreationSource; - /** Creates an instance of MonitorProperties class. */ + /* + * State of the Azure Subscription containing the monitor resource + */ + @JsonProperty(value = "subscriptionState") + private String subscriptionState; + + /* + * Status of Azure Subscription where Marketplace SaaS is located. + */ + @JsonProperty(value = "saaSAzureSubscriptionStatus") + private String saaSAzureSubscriptionStatus; + + /** + * Creates an instance of MonitorProperties class. + */ public MonitorProperties() { } /** * Get the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -100,7 +116,7 @@ public ProvisioningState provisioningState() { /** * Get the monitoringStatus property: MonitoringStatus of the resource. - * + * * @return the monitoringStatus value. */ public MonitoringStatus monitoringStatus() { @@ -109,7 +125,7 @@ public MonitoringStatus monitoringStatus() { /** * Get the marketplaceSubscriptionStatus property: NewRelic Organization properties of the resource. - * + * * @return the marketplaceSubscriptionStatus value. */ public MarketplaceSubscriptionStatus marketplaceSubscriptionStatus() { @@ -118,7 +134,7 @@ public MarketplaceSubscriptionStatus marketplaceSubscriptionStatus() { /** * Get the marketplaceSubscriptionId property: Marketplace Subscription Id. - * + * * @return the marketplaceSubscriptionId value. */ public String marketplaceSubscriptionId() { @@ -127,7 +143,7 @@ public String marketplaceSubscriptionId() { /** * Get the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @return the newRelicAccountProperties value. */ public NewRelicAccountProperties newRelicAccountProperties() { @@ -136,7 +152,7 @@ public NewRelicAccountProperties newRelicAccountProperties() { /** * Set the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @param newRelicAccountProperties the newRelicAccountProperties value to set. * @return the MonitorProperties object itself. */ @@ -147,7 +163,7 @@ public MonitorProperties withNewRelicAccountProperties(NewRelicAccountProperties /** * Get the userInfo property: User Info. - * + * * @return the userInfo value. */ public UserInfo userInfo() { @@ -156,7 +172,7 @@ public UserInfo userInfo() { /** * Set the userInfo property: User Info. - * + * * @param userInfo the userInfo value to set. * @return the MonitorProperties object itself. */ @@ -167,7 +183,7 @@ public MonitorProperties withUserInfo(UserInfo userInfo) { /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -176,7 +192,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the MonitorProperties object itself. */ @@ -187,7 +203,7 @@ public MonitorProperties withPlanData(PlanData planData) { /** * Get the liftrResourceCategory property: Liftr resource category. - * + * * @return the liftrResourceCategory value. */ public LiftrResourceCategories liftrResourceCategory() { @@ -196,7 +212,7 @@ public LiftrResourceCategories liftrResourceCategory() { /** * Get the liftrResourcePreference property: Liftr resource preference. The priority of the resource. - * + * * @return the liftrResourcePreference value. */ public Integer liftrResourcePreference() { @@ -205,7 +221,7 @@ public Integer liftrResourcePreference() { /** * Get the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ public OrgCreationSource orgCreationSource() { @@ -214,7 +230,7 @@ public OrgCreationSource orgCreationSource() { /** * Set the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource the orgCreationSource value to set. * @return the MonitorProperties object itself. */ @@ -225,7 +241,7 @@ public MonitorProperties withOrgCreationSource(OrgCreationSource orgCreationSour /** * Get the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ public AccountCreationSource accountCreationSource() { @@ -234,7 +250,7 @@ public AccountCreationSource accountCreationSource() { /** * Set the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource the accountCreationSource value to set. * @return the MonitorProperties object itself. */ @@ -243,9 +259,49 @@ public MonitorProperties withAccountCreationSource(AccountCreationSource account return this; } + /** + * Get the subscriptionState property: State of the Azure Subscription containing the monitor resource. + * + * @return the subscriptionState value. + */ + public String subscriptionState() { + return this.subscriptionState; + } + + /** + * Set the subscriptionState property: State of the Azure Subscription containing the monitor resource. + * + * @param subscriptionState the subscriptionState value to set. + * @return the MonitorProperties object itself. + */ + public MonitorProperties withSubscriptionState(String subscriptionState) { + this.subscriptionState = subscriptionState; + return this; + } + + /** + * Get the saaSAzureSubscriptionStatus property: Status of Azure Subscription where Marketplace SaaS is located. + * + * @return the saaSAzureSubscriptionStatus value. + */ + public String saaSAzureSubscriptionStatus() { + return this.saaSAzureSubscriptionStatus; + } + + /** + * Set the saaSAzureSubscriptionStatus property: Status of Azure Subscription where Marketplace SaaS is located. + * + * @param saaSAzureSubscriptionStatus the saaSAzureSubscriptionStatus value to set. + * @return the MonitorProperties object itself. + */ + public MonitorProperties withSaaSAzureSubscriptionStatus(String saaSAzureSubscriptionStatus) { + this.saaSAzureSubscriptionStatus = saaSAzureSubscriptionStatus; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredResourceInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredResourceInner.java index 294cfd43a25ed..2067b0609c848 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredResourceInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredResourceInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.newrelicobservability.models.SendingMetricsStatus; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of resource being monitored by NewRelic monitor resource. */ +/** + * Details of resource being monitored by NewRelic monitor resource. + */ @Fluent public final class MonitoredResourceInner { /* @@ -42,13 +44,15 @@ public final class MonitoredResourceInner { @JsonProperty(value = "reasonForLogsStatus") private String reasonForLogsStatus; - /** Creates an instance of MonitoredResourceInner class. */ + /** + * Creates an instance of MonitoredResourceInner class. + */ public MonitoredResourceInner() { } /** * Get the id property: The ARM id of the resource. - * + * * @return the id value. */ public String id() { @@ -57,7 +61,7 @@ public String id() { /** * Set the id property: The ARM id of the resource. - * + * * @param id the id value to set. * @return the MonitoredResourceInner object itself. */ @@ -68,7 +72,7 @@ public MonitoredResourceInner withId(String id) { /** * Get the sendingMetrics property: Flag indicating if resource is sending metrics to NewRelic. - * + * * @return the sendingMetrics value. */ public SendingMetricsStatus sendingMetrics() { @@ -77,7 +81,7 @@ public SendingMetricsStatus sendingMetrics() { /** * Set the sendingMetrics property: Flag indicating if resource is sending metrics to NewRelic. - * + * * @param sendingMetrics the sendingMetrics value to set. * @return the MonitoredResourceInner object itself. */ @@ -89,7 +93,7 @@ public MonitoredResourceInner withSendingMetrics(SendingMetricsStatus sendingMet /** * Get the reasonForMetricsStatus property: Reason for why the resource is sending metrics (or why it is not * sending). - * + * * @return the reasonForMetricsStatus value. */ public String reasonForMetricsStatus() { @@ -99,7 +103,7 @@ public String reasonForMetricsStatus() { /** * Set the reasonForMetricsStatus property: Reason for why the resource is sending metrics (or why it is not * sending). - * + * * @param reasonForMetricsStatus the reasonForMetricsStatus value to set. * @return the MonitoredResourceInner object itself. */ @@ -110,7 +114,7 @@ public MonitoredResourceInner withReasonForMetricsStatus(String reasonForMetrics /** * Get the sendingLogs property: Flag indicating if resource is sending logs to NewRelic. - * + * * @return the sendingLogs value. */ public SendingLogsStatus sendingLogs() { @@ -119,7 +123,7 @@ public SendingLogsStatus sendingLogs() { /** * Set the sendingLogs property: Flag indicating if resource is sending logs to NewRelic. - * + * * @param sendingLogs the sendingLogs value to set. * @return the MonitoredResourceInner object itself. */ @@ -130,7 +134,7 @@ public MonitoredResourceInner withSendingLogs(SendingLogsStatus sendingLogs) { /** * Get the reasonForLogsStatus property: Reason for why the resource is sending logs (or why it is not sending). - * + * * @return the reasonForLogsStatus value. */ public String reasonForLogsStatus() { @@ -139,7 +143,7 @@ public String reasonForLogsStatus() { /** * Set the reasonForLogsStatus property: Reason for why the resource is sending logs (or why it is not sending). - * + * * @param reasonForLogsStatus the reasonForLogsStatus value to set. * @return the MonitoredResourceInner object itself. */ @@ -150,7 +154,7 @@ public MonitoredResourceInner withReasonForLogsStatus(String reasonForLogsStatus /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionInner.java new file mode 100644 index 0000000000000..46e8a7c4dbee2 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionInner.java @@ -0,0 +1,136 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The list of subscriptions and it's monitoring status by current NewRelic monitor. + */ +@Fluent +public final class MonitoredSubscriptionInner { + /* + * The subscriptionId to be monitored. + */ + @JsonProperty(value = "subscriptionId") + private String subscriptionId; + + /* + * The state of monitoring. + */ + @JsonProperty(value = "status") + private Status status; + + /* + * The reason of not monitoring the subscription. + */ + @JsonProperty(value = "error") + private String error; + + /* + * The resource-specific properties for this resource. + */ + @JsonProperty(value = "tagRules") + private MonitoringTagRulesPropertiesInner tagRules; + + /** + * Creates an instance of MonitoredSubscriptionInner class. + */ + public MonitoredSubscriptionInner() { + } + + /** + * Get the subscriptionId property: The subscriptionId to be monitored. + * + * @return the subscriptionId value. + */ + public String subscriptionId() { + return this.subscriptionId; + } + + /** + * Set the subscriptionId property: The subscriptionId to be monitored. + * + * @param subscriptionId the subscriptionId value to set. + * @return the MonitoredSubscriptionInner object itself. + */ + public MonitoredSubscriptionInner withSubscriptionId(String subscriptionId) { + this.subscriptionId = subscriptionId; + return this; + } + + /** + * Get the status property: The state of monitoring. + * + * @return the status value. + */ + public Status status() { + return this.status; + } + + /** + * Set the status property: The state of monitoring. + * + * @param status the status value to set. + * @return the MonitoredSubscriptionInner object itself. + */ + public MonitoredSubscriptionInner withStatus(Status status) { + this.status = status; + return this; + } + + /** + * Get the error property: The reason of not monitoring the subscription. + * + * @return the error value. + */ + public String error() { + return this.error; + } + + /** + * Set the error property: The reason of not monitoring the subscription. + * + * @param error the error value to set. + * @return the MonitoredSubscriptionInner object itself. + */ + public MonitoredSubscriptionInner withError(String error) { + this.error = error; + return this; + } + + /** + * Get the tagRules property: The resource-specific properties for this resource. + * + * @return the tagRules value. + */ + public MonitoringTagRulesPropertiesInner tagRules() { + return this.tagRules; + } + + /** + * Set the tagRules property: The resource-specific properties for this resource. + * + * @param tagRules the tagRules value to set. + * @return the MonitoredSubscriptionInner object itself. + */ + public MonitoredSubscriptionInner withTagRules(MonitoringTagRulesPropertiesInner tagRules) { + this.tagRules = tagRules; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (tagRules() != null) { + tagRules().validate(); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionPropertiesInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionPropertiesInner.java new file mode 100644 index 0000000000000..aa16c325c746f --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoredSubscriptionPropertiesInner.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.resourcemanager.newrelicobservability.models.SubscriptionList; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ +@Fluent +public final class MonitoredSubscriptionPropertiesInner extends ProxyResource { + /* + * The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @JsonProperty(value = "properties") + private SubscriptionList properties; + + /** + * Creates an instance of MonitoredSubscriptionPropertiesInner class. + */ + public MonitoredSubscriptionPropertiesInner() { + } + + /** + * Get the properties property: The request to update subscriptions needed to be monitored by the NewRelic monitor + * resource. + * + * @return the properties value. + */ + public SubscriptionList properties() { + return this.properties; + } + + /** + * Set the properties property: The request to update subscriptions needed to be monitored by the NewRelic monitor + * resource. + * + * @param properties the properties value to set. + * @return the MonitoredSubscriptionPropertiesInner object itself. + */ + public MonitoredSubscriptionPropertiesInner withProperties(SubscriptionList properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoringTagRulesPropertiesInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoringTagRulesPropertiesInner.java index d52542789b3f9..69fba227d8739 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoringTagRulesPropertiesInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/MonitoringTagRulesPropertiesInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.newrelicobservability.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** The resource-specific properties for this resource. */ +/** + * The resource-specific properties for this resource. + */ @Fluent public final class MonitoringTagRulesPropertiesInner { /* @@ -30,13 +32,15 @@ public final class MonitoringTagRulesPropertiesInner { @JsonProperty(value = "metricRules") private MetricRulesInner metricRules; - /** Creates an instance of MonitoringTagRulesPropertiesInner class. */ + /** + * Creates an instance of MonitoringTagRulesPropertiesInner class. + */ public MonitoringTagRulesPropertiesInner() { } /** * Get the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -45,7 +49,7 @@ public ProvisioningState provisioningState() { /** * Get the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ public LogRules logRules() { @@ -54,7 +58,7 @@ public LogRules logRules() { /** * Set the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @param logRules the logRules value to set. * @return the MonitoringTagRulesPropertiesInner object itself. */ @@ -65,7 +69,7 @@ public MonitoringTagRulesPropertiesInner withLogRules(LogRules logRules) { /** * Get the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ public MetricRulesInner metricRules() { @@ -74,7 +78,7 @@ public MetricRulesInner metricRules() { /** * Set the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @param metricRules the metricRules value to set. * @return the MonitoringTagRulesPropertiesInner object itself. */ @@ -85,7 +89,7 @@ public MonitoringTagRulesPropertiesInner withMetricRules(MetricRulesInner metric /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceInner.java index 54fbf990fbf46..2d24d2e99e2bc 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceInner.java @@ -21,7 +21,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** A Monitor Resource by NewRelic. */ +/** + * A Monitor Resource by NewRelic. + */ @Fluent public final class NewRelicMonitorResourceInner extends Resource { /* @@ -42,13 +44,15 @@ public final class NewRelicMonitorResourceInner extends Resource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of NewRelicMonitorResourceInner class. */ + /** + * Creates an instance of NewRelicMonitorResourceInner class. + */ public NewRelicMonitorResourceInner() { } /** * Get the innerProperties property: The resource-specific properties for this resource. - * + * * @return the innerProperties value. */ private MonitorProperties innerProperties() { @@ -57,7 +61,7 @@ private MonitorProperties innerProperties() { /** * Get the identity property: The managed service identities assigned to this resource. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -66,7 +70,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The managed service identities assigned to this resource. - * + * * @param identity the identity value to set. * @return the NewRelicMonitorResourceInner object itself. */ @@ -77,21 +81,25 @@ public NewRelicMonitorResourceInner withIdentity(ManagedServiceIdentity identity /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public NewRelicMonitorResourceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public NewRelicMonitorResourceInner withTags(Map tags) { super.withTags(tags); @@ -100,7 +108,7 @@ public NewRelicMonitorResourceInner withTags(Map tags) { /** * Get the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -109,7 +117,7 @@ public ProvisioningState provisioningState() { /** * Get the monitoringStatus property: MonitoringStatus of the resource. - * + * * @return the monitoringStatus value. */ public MonitoringStatus monitoringStatus() { @@ -118,7 +126,7 @@ public MonitoringStatus monitoringStatus() { /** * Get the marketplaceSubscriptionStatus property: NewRelic Organization properties of the resource. - * + * * @return the marketplaceSubscriptionStatus value. */ public MarketplaceSubscriptionStatus marketplaceSubscriptionStatus() { @@ -127,7 +135,7 @@ public MarketplaceSubscriptionStatus marketplaceSubscriptionStatus() { /** * Get the marketplaceSubscriptionId property: Marketplace Subscription Id. - * + * * @return the marketplaceSubscriptionId value. */ public String marketplaceSubscriptionId() { @@ -136,7 +144,7 @@ public String marketplaceSubscriptionId() { /** * Get the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @return the newRelicAccountProperties value. */ public NewRelicAccountProperties newRelicAccountProperties() { @@ -145,12 +153,12 @@ public NewRelicAccountProperties newRelicAccountProperties() { /** * Set the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @param newRelicAccountProperties the newRelicAccountProperties value to set. * @return the NewRelicMonitorResourceInner object itself. */ - public NewRelicMonitorResourceInner withNewRelicAccountProperties( - NewRelicAccountProperties newRelicAccountProperties) { + public NewRelicMonitorResourceInner + withNewRelicAccountProperties(NewRelicAccountProperties newRelicAccountProperties) { if (this.innerProperties() == null) { this.innerProperties = new MonitorProperties(); } @@ -160,7 +168,7 @@ public NewRelicMonitorResourceInner withNewRelicAccountProperties( /** * Get the userInfo property: User Info. - * + * * @return the userInfo value. */ public UserInfo userInfo() { @@ -169,7 +177,7 @@ public UserInfo userInfo() { /** * Set the userInfo property: User Info. - * + * * @param userInfo the userInfo value to set. * @return the NewRelicMonitorResourceInner object itself. */ @@ -183,7 +191,7 @@ public NewRelicMonitorResourceInner withUserInfo(UserInfo userInfo) { /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -192,7 +200,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the NewRelicMonitorResourceInner object itself. */ @@ -206,7 +214,7 @@ public NewRelicMonitorResourceInner withPlanData(PlanData planData) { /** * Get the liftrResourceCategory property: Liftr resource category. - * + * * @return the liftrResourceCategory value. */ public LiftrResourceCategories liftrResourceCategory() { @@ -215,7 +223,7 @@ public LiftrResourceCategories liftrResourceCategory() { /** * Get the liftrResourcePreference property: Liftr resource preference. The priority of the resource. - * + * * @return the liftrResourcePreference value. */ public Integer liftrResourcePreference() { @@ -224,7 +232,7 @@ public Integer liftrResourcePreference() { /** * Get the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ public OrgCreationSource orgCreationSource() { @@ -233,7 +241,7 @@ public OrgCreationSource orgCreationSource() { /** * Set the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource the orgCreationSource value to set. * @return the NewRelicMonitorResourceInner object itself. */ @@ -247,7 +255,7 @@ public NewRelicMonitorResourceInner withOrgCreationSource(OrgCreationSource orgC /** * Get the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ public AccountCreationSource accountCreationSource() { @@ -256,7 +264,7 @@ public AccountCreationSource accountCreationSource() { /** * Set the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource the accountCreationSource value to set. * @return the NewRelicMonitorResourceInner object itself. */ @@ -268,17 +276,61 @@ public NewRelicMonitorResourceInner withAccountCreationSource(AccountCreationSou return this; } + /** + * Get the subscriptionState property: State of the Azure Subscription containing the monitor resource. + * + * @return the subscriptionState value. + */ + public String subscriptionState() { + return this.innerProperties() == null ? null : this.innerProperties().subscriptionState(); + } + + /** + * Set the subscriptionState property: State of the Azure Subscription containing the monitor resource. + * + * @param subscriptionState the subscriptionState value to set. + * @return the NewRelicMonitorResourceInner object itself. + */ + public NewRelicMonitorResourceInner withSubscriptionState(String subscriptionState) { + if (this.innerProperties() == null) { + this.innerProperties = new MonitorProperties(); + } + this.innerProperties().withSubscriptionState(subscriptionState); + return this; + } + + /** + * Get the saaSAzureSubscriptionStatus property: Status of Azure Subscription where Marketplace SaaS is located. + * + * @return the saaSAzureSubscriptionStatus value. + */ + public String saaSAzureSubscriptionStatus() { + return this.innerProperties() == null ? null : this.innerProperties().saaSAzureSubscriptionStatus(); + } + + /** + * Set the saaSAzureSubscriptionStatus property: Status of Azure Subscription where Marketplace SaaS is located. + * + * @param saaSAzureSubscriptionStatus the saaSAzureSubscriptionStatus value to set. + * @return the NewRelicMonitorResourceInner object itself. + */ + public NewRelicMonitorResourceInner withSaaSAzureSubscriptionStatus(String saaSAzureSubscriptionStatus) { + if (this.innerProperties() == null) { + this.innerProperties = new MonitorProperties(); + } + this.innerProperties().withSaaSAzureSubscriptionStatus(saaSAzureSubscriptionStatus); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model NewRelicMonitorResourceInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property innerProperties in model NewRelicMonitorResourceInner")); } else { innerProperties().validate(); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceUpdateProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceUpdateProperties.java index 0d8d715951c77..f4ef675e1293d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceUpdateProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/NewRelicMonitorResourceUpdateProperties.java @@ -12,7 +12,9 @@ import com.azure.resourcemanager.newrelicobservability.models.UserInfo; import com.fasterxml.jackson.annotation.JsonProperty; -/** The updatable properties of the NewRelicMonitorResource. */ +/** + * The updatable properties of the NewRelicMonitorResource. + */ @Fluent public final class NewRelicMonitorResourceUpdateProperties { /* @@ -45,13 +47,15 @@ public final class NewRelicMonitorResourceUpdateProperties { @JsonProperty(value = "accountCreationSource") private AccountCreationSource accountCreationSource; - /** Creates an instance of NewRelicMonitorResourceUpdateProperties class. */ + /** + * Creates an instance of NewRelicMonitorResourceUpdateProperties class. + */ public NewRelicMonitorResourceUpdateProperties() { } /** * Get the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @return the newRelicAccountProperties value. */ public NewRelicAccountProperties newRelicAccountProperties() { @@ -60,19 +64,19 @@ public NewRelicAccountProperties newRelicAccountProperties() { /** * Set the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @param newRelicAccountProperties the newRelicAccountProperties value to set. * @return the NewRelicMonitorResourceUpdateProperties object itself. */ - public NewRelicMonitorResourceUpdateProperties withNewRelicAccountProperties( - NewRelicAccountProperties newRelicAccountProperties) { + public NewRelicMonitorResourceUpdateProperties + withNewRelicAccountProperties(NewRelicAccountProperties newRelicAccountProperties) { this.newRelicAccountProperties = newRelicAccountProperties; return this; } /** * Get the userInfo property: User Info. - * + * * @return the userInfo value. */ public UserInfo userInfo() { @@ -81,7 +85,7 @@ public UserInfo userInfo() { /** * Set the userInfo property: User Info. - * + * * @param userInfo the userInfo value to set. * @return the NewRelicMonitorResourceUpdateProperties object itself. */ @@ -92,7 +96,7 @@ public NewRelicMonitorResourceUpdateProperties withUserInfo(UserInfo userInfo) { /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -101,7 +105,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the NewRelicMonitorResourceUpdateProperties object itself. */ @@ -112,7 +116,7 @@ public NewRelicMonitorResourceUpdateProperties withPlanData(PlanData planData) { /** * Get the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ public OrgCreationSource orgCreationSource() { @@ -121,7 +125,7 @@ public OrgCreationSource orgCreationSource() { /** * Set the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource the orgCreationSource value to set. * @return the NewRelicMonitorResourceUpdateProperties object itself. */ @@ -132,7 +136,7 @@ public NewRelicMonitorResourceUpdateProperties withOrgCreationSource(OrgCreation /** * Get the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ public AccountCreationSource accountCreationSource() { @@ -141,19 +145,19 @@ public AccountCreationSource accountCreationSource() { /** * Set the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource the accountCreationSource value to set. * @return the NewRelicMonitorResourceUpdateProperties object itself. */ - public NewRelicMonitorResourceUpdateProperties withAccountCreationSource( - AccountCreationSource accountCreationSource) { + public NewRelicMonitorResourceUpdateProperties + withAccountCreationSource(AccountCreationSource accountCreationSource) { this.accountCreationSource = accountCreationSource; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OperationInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OperationInner.java index ea0884c17c93c..fcb6e352326f2 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OperationInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OperationInner.java @@ -12,8 +12,8 @@ /** * REST API Operation - * - *

Details of a REST API operation, returned from the Resource Provider Operations API. + * + * Details of a REST API operation, returned from the Resource Provider Operations API. */ @Fluent public final class OperationInner { @@ -50,14 +50,16 @@ public final class OperationInner { @JsonProperty(value = "actionType", access = JsonProperty.Access.WRITE_ONLY) private ActionType actionType; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ public String name() { @@ -67,7 +69,7 @@ public String name() { /** * Get the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -76,7 +78,7 @@ public Boolean isDataAction() { /** * Get the display property: Localized display information for this particular operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -85,7 +87,7 @@ public OperationDisplay display() { /** * Set the display property: Localized display information for this particular operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -97,7 +99,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Get the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ public Origin origin() { @@ -107,7 +109,7 @@ public Origin origin() { /** * Get the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ public ActionType actionType() { @@ -116,7 +118,7 @@ public ActionType actionType() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationProperties.java index 3aac3269f2783..687ebeed98901 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.newrelicobservability.models.BillingSource; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of Organizations. */ +/** + * Details of Organizations. + */ @Fluent public final class OrganizationProperties { /* @@ -29,13 +31,15 @@ public final class OrganizationProperties { @JsonProperty(value = "billingSource") private BillingSource billingSource; - /** Creates an instance of OrganizationProperties class. */ + /** + * Creates an instance of OrganizationProperties class. + */ public OrganizationProperties() { } /** * Get the organizationId property: organization id. - * + * * @return the organizationId value. */ public String organizationId() { @@ -44,7 +48,7 @@ public String organizationId() { /** * Set the organizationId property: organization id. - * + * * @param organizationId the organizationId value to set. * @return the OrganizationProperties object itself. */ @@ -55,7 +59,7 @@ public OrganizationProperties withOrganizationId(String organizationId) { /** * Get the organizationName property: organization name. - * + * * @return the organizationName value. */ public String organizationName() { @@ -64,7 +68,7 @@ public String organizationName() { /** * Set the organizationName property: organization name. - * + * * @param organizationName the organizationName value to set. * @return the OrganizationProperties object itself. */ @@ -75,7 +79,7 @@ public OrganizationProperties withOrganizationName(String organizationName) { /** * Get the billingSource property: Billing source. - * + * * @return the billingSource value. */ public BillingSource billingSource() { @@ -84,7 +88,7 @@ public BillingSource billingSource() { /** * Set the billingSource property: Billing source. - * + * * @param billingSource the billingSource value to set. * @return the OrganizationProperties object itself. */ @@ -95,7 +99,7 @@ public OrganizationProperties withBillingSource(BillingSource billingSource) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationResourceInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationResourceInner.java index e4c77069e6c21..49728479de56a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationResourceInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/OrganizationResourceInner.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.newrelicobservability.models.BillingSource; import com.fasterxml.jackson.annotation.JsonProperty; -/** The details of a Organization resource. */ +/** + * The details of a Organization resource. + */ @Fluent public final class OrganizationResourceInner extends ProxyResource { /* @@ -25,13 +27,15 @@ public final class OrganizationResourceInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of OrganizationResourceInner class. */ + /** + * Creates an instance of OrganizationResourceInner class. + */ public OrganizationResourceInner() { } /** * Get the innerProperties property: The resource-specific properties for this resource. - * + * * @return the innerProperties value. */ private OrganizationProperties innerProperties() { @@ -40,7 +44,7 @@ private OrganizationProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -49,7 +53,7 @@ public SystemData systemData() { /** * Get the organizationId property: organization id. - * + * * @return the organizationId value. */ public String organizationId() { @@ -58,7 +62,7 @@ public String organizationId() { /** * Set the organizationId property: organization id. - * + * * @param organizationId the organizationId value to set. * @return the OrganizationResourceInner object itself. */ @@ -72,7 +76,7 @@ public OrganizationResourceInner withOrganizationId(String organizationId) { /** * Get the organizationName property: organization name. - * + * * @return the organizationName value. */ public String organizationName() { @@ -81,7 +85,7 @@ public String organizationName() { /** * Set the organizationName property: organization name. - * + * * @param organizationName the organizationName value to set. * @return the OrganizationResourceInner object itself. */ @@ -95,7 +99,7 @@ public OrganizationResourceInner withOrganizationName(String organizationName) { /** * Get the billingSource property: Billing source. - * + * * @return the billingSource value. */ public BillingSource billingSource() { @@ -104,7 +108,7 @@ public BillingSource billingSource() { /** * Set the billingSource property: Billing source. - * + * * @param billingSource the billingSource value to set. * @return the OrganizationResourceInner object itself. */ @@ -118,7 +122,7 @@ public OrganizationResourceInner withBillingSource(BillingSource billingSource) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataProperties.java index b4ab5beb338a0..2773e5d2527c1 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataProperties.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.newrelicobservability.models.PlanData; import com.fasterxml.jackson.annotation.JsonProperty; -/** Plan details. */ +/** + * Plan details. + */ @Fluent public final class PlanDataProperties { /* @@ -31,13 +33,15 @@ public final class PlanDataProperties { @JsonProperty(value = "accountCreationSource") private AccountCreationSource accountCreationSource; - /** Creates an instance of PlanDataProperties class. */ + /** + * Creates an instance of PlanDataProperties class. + */ public PlanDataProperties() { } /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -46,7 +50,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the PlanDataProperties object itself. */ @@ -57,7 +61,7 @@ public PlanDataProperties withPlanData(PlanData planData) { /** * Get the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ public OrgCreationSource orgCreationSource() { @@ -66,7 +70,7 @@ public OrgCreationSource orgCreationSource() { /** * Set the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource the orgCreationSource value to set. * @return the PlanDataProperties object itself. */ @@ -77,7 +81,7 @@ public PlanDataProperties withOrgCreationSource(OrgCreationSource orgCreationSou /** * Get the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ public AccountCreationSource accountCreationSource() { @@ -86,7 +90,7 @@ public AccountCreationSource accountCreationSource() { /** * Set the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource the accountCreationSource value to set. * @return the PlanDataProperties object itself. */ @@ -97,7 +101,7 @@ public PlanDataProperties withAccountCreationSource(AccountCreationSource accoun /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataResourceInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataResourceInner.java index d541fd5140838..000633eb66ce7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataResourceInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/PlanDataResourceInner.java @@ -12,7 +12,9 @@ import com.azure.resourcemanager.newrelicobservability.models.PlanData; import com.fasterxml.jackson.annotation.JsonProperty; -/** The details of a PlanData resource. */ +/** + * The details of a PlanData resource. + */ @Fluent public final class PlanDataResourceInner extends ProxyResource { /* @@ -27,13 +29,15 @@ public final class PlanDataResourceInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of PlanDataResourceInner class. */ + /** + * Creates an instance of PlanDataResourceInner class. + */ public PlanDataResourceInner() { } /** * Get the innerProperties property: The resource-specific properties for this resource. - * + * * @return the innerProperties value. */ private PlanDataProperties innerProperties() { @@ -42,7 +46,7 @@ private PlanDataProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -51,7 +55,7 @@ public SystemData systemData() { /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -60,7 +64,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the PlanDataResourceInner object itself. */ @@ -74,7 +78,7 @@ public PlanDataResourceInner withPlanData(PlanData planData) { /** * Get the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ public OrgCreationSource orgCreationSource() { @@ -83,7 +87,7 @@ public OrgCreationSource orgCreationSource() { /** * Set the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource the orgCreationSource value to set. * @return the PlanDataResourceInner object itself. */ @@ -97,7 +101,7 @@ public PlanDataResourceInner withOrgCreationSource(OrgCreationSource orgCreation /** * Get the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ public AccountCreationSource accountCreationSource() { @@ -106,7 +110,7 @@ public AccountCreationSource accountCreationSource() { /** * Set the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource the accountCreationSource value to set. * @return the PlanDataResourceInner object itself. */ @@ -120,7 +124,7 @@ public PlanDataResourceInner withAccountCreationSource(AccountCreationSource acc /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleInner.java index 8a237df6055a4..9990f515edcbb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleInner.java @@ -12,7 +12,9 @@ import com.azure.resourcemanager.newrelicobservability.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -/** A tag rule belonging to NewRelic account. */ +/** + * A tag rule belonging to NewRelic account. + */ @Fluent public final class TagRuleInner extends ProxyResource { /* @@ -27,13 +29,15 @@ public final class TagRuleInner extends ProxyResource { @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of TagRuleInner class. */ + /** + * Creates an instance of TagRuleInner class. + */ public TagRuleInner() { } /** * Get the innerProperties property: The resource-specific properties for this resource. - * + * * @return the innerProperties value. */ private MonitoringTagRulesPropertiesInner innerProperties() { @@ -42,7 +46,7 @@ private MonitoringTagRulesPropertiesInner innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -51,7 +55,7 @@ public SystemData systemData() { /** * Get the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -60,7 +64,7 @@ public ProvisioningState provisioningState() { /** * Get the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ public LogRules logRules() { @@ -69,7 +73,7 @@ public LogRules logRules() { /** * Set the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @param logRules the logRules value to set. * @return the TagRuleInner object itself. */ @@ -83,7 +87,7 @@ public TagRuleInner withLogRules(LogRules logRules) { /** * Get the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ public MetricRulesInner metricRules() { @@ -92,7 +96,7 @@ public MetricRulesInner metricRules() { /** * Set the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @param metricRules the metricRules value to set. * @return the TagRuleInner object itself. */ @@ -106,14 +110,13 @@ public TagRuleInner withMetricRules(MetricRulesInner metricRules) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property innerProperties in model TagRuleInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model TagRuleInner")); } else { innerProperties().validate(); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdateInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdateInner.java index dfef473f2a644..20f78dee1a815 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdateInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdateInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.newrelicobservability.models.LogRules; import com.fasterxml.jackson.annotation.JsonProperty; -/** The type used for update operations of the TagRule. */ +/** + * The type used for update operations of the TagRule. + */ @Fluent public final class TagRuleUpdateInner { /* @@ -17,13 +19,15 @@ public final class TagRuleUpdateInner { @JsonProperty(value = "properties") private TagRuleUpdatePropertiesInner innerProperties; - /** Creates an instance of TagRuleUpdateInner class. */ + /** + * Creates an instance of TagRuleUpdateInner class. + */ public TagRuleUpdateInner() { } /** * Get the innerProperties property: The updatable properties of the TagRule. - * + * * @return the innerProperties value. */ private TagRuleUpdatePropertiesInner innerProperties() { @@ -32,7 +36,7 @@ private TagRuleUpdatePropertiesInner innerProperties() { /** * Get the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ public LogRules logRules() { @@ -41,7 +45,7 @@ public LogRules logRules() { /** * Set the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @param logRules the logRules value to set. * @return the TagRuleUpdateInner object itself. */ @@ -55,7 +59,7 @@ public TagRuleUpdateInner withLogRules(LogRules logRules) { /** * Get the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ public MetricRulesInner metricRules() { @@ -64,7 +68,7 @@ public MetricRulesInner metricRules() { /** * Set the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @param metricRules the metricRules value to set. * @return the TagRuleUpdateInner object itself. */ @@ -78,7 +82,7 @@ public TagRuleUpdateInner withMetricRules(MetricRulesInner metricRules) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdatePropertiesInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdatePropertiesInner.java index 687bdad877088..1dc98a2c7f9d6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdatePropertiesInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/TagRuleUpdatePropertiesInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.newrelicobservability.models.LogRules; import com.fasterxml.jackson.annotation.JsonProperty; -/** The updatable properties of the TagRule. */ +/** + * The updatable properties of the TagRule. + */ @Fluent public final class TagRuleUpdatePropertiesInner { /* @@ -23,13 +25,15 @@ public final class TagRuleUpdatePropertiesInner { @JsonProperty(value = "metricRules") private MetricRulesInner metricRules; - /** Creates an instance of TagRuleUpdatePropertiesInner class. */ + /** + * Creates an instance of TagRuleUpdatePropertiesInner class. + */ public TagRuleUpdatePropertiesInner() { } /** * Get the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ public LogRules logRules() { @@ -38,7 +42,7 @@ public LogRules logRules() { /** * Set the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @param logRules the logRules value to set. * @return the TagRuleUpdatePropertiesInner object itself. */ @@ -49,7 +53,7 @@ public TagRuleUpdatePropertiesInner withLogRules(LogRules logRules) { /** * Get the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ public MetricRulesInner metricRules() { @@ -58,7 +62,7 @@ public MetricRulesInner metricRules() { /** * Set the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @param metricRules the metricRules value to set. * @return the TagRuleUpdatePropertiesInner object itself. */ @@ -69,7 +73,7 @@ public TagRuleUpdatePropertiesInner withMetricRules(MetricRulesInner metricRules /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMExtensionPayloadInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMExtensionPayloadInner.java index c8bd75268003c..6923a21104e71 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMExtensionPayloadInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMExtensionPayloadInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Response of payload to be passed while installing VM agent. */ +/** + * Response of payload to be passed while installing VM agent. + */ @Fluent public final class VMExtensionPayloadInner { /* @@ -16,13 +18,15 @@ public final class VMExtensionPayloadInner { @JsonProperty(value = "ingestionKey") private String ingestionKey; - /** Creates an instance of VMExtensionPayloadInner class. */ + /** + * Creates an instance of VMExtensionPayloadInner class. + */ public VMExtensionPayloadInner() { } /** * Get the ingestionKey property: Ingestion key of the account. - * + * * @return the ingestionKey value. */ public String ingestionKey() { @@ -31,7 +35,7 @@ public String ingestionKey() { /** * Set the ingestionKey property: Ingestion key of the account. - * + * * @param ingestionKey the ingestionKey value to set. * @return the VMExtensionPayloadInner object itself. */ @@ -42,7 +46,7 @@ public VMExtensionPayloadInner withIngestionKey(String ingestionKey) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMInfoInner.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMInfoInner.java index 34029b0c1850d..337f45f108224 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMInfoInner.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/VMInfoInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of VM Resource having NewRelic OneAgent installed. */ +/** + * Details of VM Resource having NewRelic OneAgent installed. + */ @Fluent public final class VMInfoInner { /* @@ -28,13 +30,15 @@ public final class VMInfoInner { @JsonProperty(value = "agentStatus") private String agentStatus; - /** Creates an instance of VMInfoInner class. */ + /** + * Creates an instance of VMInfoInner class. + */ public VMInfoInner() { } /** * Get the vmId property: Azure VM resource ID. - * + * * @return the vmId value. */ public String vmId() { @@ -43,7 +47,7 @@ public String vmId() { /** * Set the vmId property: Azure VM resource ID. - * + * * @param vmId the vmId value to set. * @return the VMInfoInner object itself. */ @@ -54,7 +58,7 @@ public VMInfoInner withVmId(String vmId) { /** * Get the agentVersion property: Version of the NewRelic agent installed on the VM. - * + * * @return the agentVersion value. */ public String agentVersion() { @@ -63,7 +67,7 @@ public String agentVersion() { /** * Set the agentVersion property: Version of the NewRelic agent installed on the VM. - * + * * @param agentVersion the agentVersion value to set. * @return the VMInfoInner object itself. */ @@ -74,7 +78,7 @@ public VMInfoInner withAgentVersion(String agentVersion) { /** * Get the agentStatus property: Status of the NewRelic agent installed on the VM. - * + * * @return the agentStatus value. */ public String agentStatus() { @@ -83,7 +87,7 @@ public String agentStatus() { /** * Set the agentStatus property: Status of the NewRelic agent installed on the VM. - * + * * @param agentStatus the agentStatus value to set. * @return the VMInfoInner object itself. */ @@ -94,7 +98,7 @@ public VMInfoInner withAgentStatus(String agentStatus) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/package-info.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/package-info.java index 92ec93e13d33e..57f83d2067eff 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/package-info.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the inner data models for NewRelicObservability. null. */ +/** + * Package containing the inner data models for NewRelicObservability. + * null. + */ package com.azure.resourcemanager.newrelicobservability.fluent.models; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/package-info.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/package-info.java index 9639651ea926e..ac6fed96890a7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/package-info.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/fluent/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the service clients for NewRelicObservability. null. */ +/** + * Package containing the service clients for NewRelicObservability. + * null. + */ package com.azure.resourcemanager.newrelicobservability.fluent; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountResourceImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountResourceImpl.java index 02212054a6241..ba5eb8ffef8f6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountResourceImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountResourceImpl.java @@ -13,8 +13,7 @@ public final class AccountResourceImpl implements AccountResource { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - AccountResourceImpl( - AccountResourceInner innerObject, + AccountResourceImpl(AccountResourceInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsClientImpl.java index a04acdb1b9fb5..486fe2a102817 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsClientImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsClientImpl.java @@ -30,17 +30,23 @@ import com.azure.resourcemanager.newrelicobservability.models.AccountsListResponse; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AccountsClient. */ +/** + * An instance of this class provides access to all the operations defined in AccountsClient. + */ public final class AccountsClientImpl implements AccountsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AccountsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final NewRelicObservabilityImpl client; /** * Initializes an instance of AccountsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AccountsClientImpl(NewRelicObservabilityImpl client) { @@ -55,54 +61,43 @@ public final class AccountsClientImpl implements AccountsClient { @Host("{$host}") @ServiceInterface(name = "NewRelicObservabilit") public interface AccountsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/NewRelic.Observability/accounts") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("userEmail") String userEmail, - @QueryParam("location") String location, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("userEmail") String userEmail, @QueryParam("location") String location, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all accounts Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String userEmail, String location) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (userEmail == null) { return Mono.error(new IllegalArgumentException("Parameter userEmail is required and cannot be null.")); @@ -112,32 +107,16 @@ private Mono> listSinglePageAsync(String use } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - userEmail, - location, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), userEmail, location, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. @@ -145,22 +124,18 @@ private Mono> listSinglePageAsync(String use * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all accounts Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String userEmail, String location, Context context) { + private Mono> listSinglePageAsync(String userEmail, String location, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (userEmail == null) { return Mono.error(new IllegalArgumentException("Parameter userEmail is required and cannot be null.")); @@ -171,28 +146,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - userEmail, - location, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), userEmail, + location, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -202,13 +164,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String userEmail, String location) { - return new PagedFlux<>( - () -> listSinglePageAsync(userEmail, location), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(userEmail, location), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. @@ -219,14 +181,13 @@ private PagedFlux listAsync(String userEmail, String locat */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String userEmail, String location, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(userEmail, location, context), + return new PagedFlux<>(() -> listSinglePageAsync(userEmail, location, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -241,7 +202,7 @@ public PagedIterable list(String userEmail, String locatio /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. @@ -257,14 +218,15 @@ public PagedIterable list(String userEmail, String locatio /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all accounts Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -272,37 +234,28 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all accounts Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -310,23 +263,13 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsImpl.java index c886051fb1ba0..989d43f5cd39c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AccountsImpl.java @@ -19,8 +19,7 @@ public final class AccountsImpl implements Accounts { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - public AccountsImpl( - AccountsClient innerClient, + public AccountsImpl(AccountsClient innerClient, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public AccountsImpl( public PagedIterable list(String userEmail, String location) { PagedIterable inner = this.serviceClient().list(userEmail, location); - return Utils.mapPage(inner, inner1 -> new AccountResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AccountResourceImpl(inner1, this.manager())); } public PagedIterable list(String userEmail, String location, Context context) { PagedIterable inner = this.serviceClient().list(userEmail, location, context); - return Utils.mapPage(inner, inner1 -> new AccountResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AccountResourceImpl(inner1, this.manager())); } private AccountsClient serviceClient() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AppServiceInfoImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AppServiceInfoImpl.java index d23cb2edb8530..8839036a43f06 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AppServiceInfoImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/AppServiceInfoImpl.java @@ -12,8 +12,7 @@ public final class AppServiceInfoImpl implements AppServiceInfo { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - AppServiceInfoImpl( - AppServiceInfoInner innerObject, + AppServiceInfoImpl(AppServiceInfoInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoResponseImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoResponseImpl.java new file mode 100644 index 0000000000000..d0c650d1d86b2 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoResponseImpl.java @@ -0,0 +1,38 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner; +import com.azure.resourcemanager.newrelicobservability.models.BillingInfoResponse; +import com.azure.resourcemanager.newrelicobservability.models.MarketplaceSaaSInfo; +import com.azure.resourcemanager.newrelicobservability.models.PartnerBillingEntity; + +public final class BillingInfoResponseImpl implements BillingInfoResponse { + private BillingInfoResponseInner innerObject; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + BillingInfoResponseImpl(BillingInfoResponseInner innerObject, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public MarketplaceSaaSInfo marketplaceSaasInfo() { + return this.innerModel().marketplaceSaasInfo(); + } + + public PartnerBillingEntity partnerBillingEntity() { + return this.innerModel().partnerBillingEntity(); + } + + public BillingInfoResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesClientImpl.java new file mode 100644 index 0000000000000..421d8cd8c863b --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesClientImpl.java @@ -0,0 +1,188 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.newrelicobservability.fluent.BillingInfoesClient; +import com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in BillingInfoesClient. + */ +public final class BillingInfoesClientImpl implements BillingInfoesClient { + /** + * The proxy service used to perform REST calls. + */ + private final BillingInfoesService service; + + /** + * The service client containing this operation class. + */ + private final NewRelicObservabilityImpl client; + + /** + * Initializes an instance of BillingInfoesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + BillingInfoesClientImpl(NewRelicObservabilityImpl client) { + this.service + = RestProxy.create(BillingInfoesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for NewRelicObservabilityBillingInfoes to be used by the proxy service + * to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "NewRelicObservabilit") + public interface BillingInfoesService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/getBillingInfo") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String monitorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, String monitorName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, accept, context); + } + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String monitorName) { + return getWithResponseAsync(resourceGroupName, monitorName).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String monitorName, + Context context) { + return getWithResponseAsync(resourceGroupName, monitorName, context).block(); + } + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public BillingInfoResponseInner get(String resourceGroupName, String monitorName) { + return getWithResponse(resourceGroupName, monitorName, Context.NONE).getValue(); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesImpl.java new file mode 100644 index 0000000000000..6e3272bc8b5ed --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/BillingInfoesImpl.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.newrelicobservability.fluent.BillingInfoesClient; +import com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner; +import com.azure.resourcemanager.newrelicobservability.models.BillingInfoResponse; +import com.azure.resourcemanager.newrelicobservability.models.BillingInfoes; + +public final class BillingInfoesImpl implements BillingInfoes { + private static final ClientLogger LOGGER = new ClientLogger(BillingInfoesImpl.class); + + private final BillingInfoesClient innerClient; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + public BillingInfoesImpl(BillingInfoesClient innerClient, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response getWithResponse(String resourceGroupName, String monitorName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, monitorName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new BillingInfoResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public BillingInfoResponse get(String resourceGroupName, String monitorName) { + BillingInfoResponseInner inner = this.serviceClient().get(resourceGroupName, monitorName); + if (inner != null) { + return new BillingInfoResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + private BillingInfoesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesClientImpl.java new file mode 100644 index 0000000000000..122d706635971 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesClientImpl.java @@ -0,0 +1,312 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.newrelicobservability.fluent.ConnectedPartnerResourcesClient; +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourcesListResponse; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in ConnectedPartnerResourcesClient. + */ +public final class ConnectedPartnerResourcesClientImpl implements ConnectedPartnerResourcesClient { + /** + * The proxy service used to perform REST calls. + */ + private final ConnectedPartnerResourcesService service; + + /** + * The service client containing this operation class. + */ + private final NewRelicObservabilityImpl client; + + /** + * Initializes an instance of ConnectedPartnerResourcesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + ConnectedPartnerResourcesClientImpl(NewRelicObservabilityImpl client) { + this.service = RestProxy.create(ConnectedPartnerResourcesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for NewRelicObservabilityConnectedPartnerResources to be used by the + * proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "NewRelicObservabilit") + public interface ConnectedPartnerResourcesService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/listConnectedPartnerResources") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") String body, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String monitorName, String body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, body, accept, context)) + .>map( + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String monitorName, String body, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, body, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String monitorName, + String body) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, monitorName, body), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, + String monitorName) { + final String body = null; + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, monitorName, body), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String monitorName, + String body, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, monitorName, body, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String monitorName) { + final String body = null; + return new PagedIterable<>(listAsync(resourceGroupName, monitorName, body)); + } + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String monitorName, + String body, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, monitorName, body, context)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map( + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments along with {@link PagedResponse} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesImpl.java new file mode 100644 index 0000000000000..cf5e56d9ca933 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.newrelicobservability.fluent.ConnectedPartnerResourcesClient; +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResources; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourcesListFormat; + +public final class ConnectedPartnerResourcesImpl implements ConnectedPartnerResources { + private static final ClientLogger LOGGER = new ClientLogger(ConnectedPartnerResourcesImpl.class); + + private final ConnectedPartnerResourcesClient innerClient; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + public ConnectedPartnerResourcesImpl(ConnectedPartnerResourcesClient innerClient, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String monitorName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, monitorName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new ConnectedPartnerResourcesListFormatImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String monitorName, + String body, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, monitorName, body, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new ConnectedPartnerResourcesListFormatImpl(inner1, this.manager())); + } + + private ConnectedPartnerResourcesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesListFormatImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesListFormatImpl.java new file mode 100644 index 0000000000000..ead42119d6e2f --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ConnectedPartnerResourcesListFormatImpl.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourceProperties; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourcesListFormat; + +public final class ConnectedPartnerResourcesListFormatImpl implements ConnectedPartnerResourcesListFormat { + private ConnectedPartnerResourcesListFormatInner innerObject; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + ConnectedPartnerResourcesListFormatImpl(ConnectedPartnerResourcesListFormatInner innerObject, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public ConnectedPartnerResourceProperties properties() { + return this.innerModel().properties(); + } + + public ConnectedPartnerResourcesListFormatInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/LinkedResourceImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/LinkedResourceImpl.java new file mode 100644 index 0000000000000..81804fb1e6098 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/LinkedResourceImpl.java @@ -0,0 +1,32 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; +import com.azure.resourcemanager.newrelicobservability.models.LinkedResource; + +public final class LinkedResourceImpl implements LinkedResource { + private LinkedResourceInner innerObject; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + LinkedResourceImpl(LinkedResourceInner innerObject, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public LinkedResourceInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricRulesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricRulesImpl.java index 5f217ce0d08c3..29a6752b64185 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricRulesImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricRulesImpl.java @@ -16,8 +16,7 @@ public final class MetricRulesImpl implements MetricRules { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - MetricRulesImpl( - MetricRulesInner innerObject, + MetricRulesImpl(MetricRulesInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricsStatusResponseImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricsStatusResponseImpl.java index a3550a8900319..35926b05312a9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricsStatusResponseImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MetricsStatusResponseImpl.java @@ -14,8 +14,7 @@ public final class MetricsStatusResponseImpl implements MetricsStatusResponse { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - MetricsStatusResponseImpl( - MetricsStatusResponseInner innerObject, + MetricsStatusResponseImpl(MetricsStatusResponseInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredResourceImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredResourceImpl.java index dbf820dee7c53..cad9c336284a8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredResourceImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredResourceImpl.java @@ -14,8 +14,7 @@ public final class MonitoredResourceImpl implements MonitoredResource { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - MonitoredResourceImpl( - MonitoredResourceInner innerObject, + MonitoredResourceImpl(MonitoredResourceInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionImpl.java new file mode 100644 index 0000000000000..5c1e56a7a5ab8 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionImpl.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscription; +import com.azure.resourcemanager.newrelicobservability.models.MonitoringTagRulesProperties; +import com.azure.resourcemanager.newrelicobservability.models.Status; + +public final class MonitoredSubscriptionImpl implements MonitoredSubscription { + private MonitoredSubscriptionInner innerObject; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + MonitoredSubscriptionImpl(MonitoredSubscriptionInner innerObject, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String subscriptionId() { + return this.innerModel().subscriptionId(); + } + + public Status status() { + return this.innerModel().status(); + } + + public String error() { + return this.innerModel().error(); + } + + public MonitoringTagRulesProperties tagRules() { + MonitoringTagRulesPropertiesInner inner = this.innerModel().tagRules(); + if (inner != null) { + return new MonitoringTagRulesPropertiesImpl(inner, this.manager()); + } else { + return null; + } + } + + public MonitoredSubscriptionInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionPropertiesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionPropertiesImpl.java new file mode 100644 index 0000000000000..8876220a39cb5 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionPropertiesImpl.java @@ -0,0 +1,120 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.util.Context; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; +import com.azure.resourcemanager.newrelicobservability.models.SubscriptionList; + +public final class MonitoredSubscriptionPropertiesImpl implements MonitoredSubscriptionProperties, + MonitoredSubscriptionProperties.Definition, MonitoredSubscriptionProperties.Update { + private MonitoredSubscriptionPropertiesInner innerObject; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SubscriptionList properties() { + return this.innerModel().properties(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public MonitoredSubscriptionPropertiesInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String monitorName; + + private ConfigurationName configurationName; + + public MonitoredSubscriptionPropertiesImpl withExistingMonitor(String resourceGroupName, String monitorName) { + this.resourceGroupName = resourceGroupName; + this.monitorName = monitorName; + return this; + } + + public MonitoredSubscriptionProperties create() { + this.innerObject = serviceManager.serviceClient().getMonitoredSubscriptions().createorUpdate(resourceGroupName, + monitorName, configurationName, this.innerModel(), Context.NONE); + return this; + } + + public MonitoredSubscriptionProperties create(Context context) { + this.innerObject = serviceManager.serviceClient().getMonitoredSubscriptions().createorUpdate(resourceGroupName, + monitorName, configurationName, this.innerModel(), context); + return this; + } + + MonitoredSubscriptionPropertiesImpl(ConfigurationName name, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerObject = new MonitoredSubscriptionPropertiesInner(); + this.serviceManager = serviceManager; + this.configurationName = name; + } + + public MonitoredSubscriptionPropertiesImpl update() { + return this; + } + + public MonitoredSubscriptionProperties apply() { + this.innerObject = serviceManager.serviceClient().getMonitoredSubscriptions().update(resourceGroupName, + monitorName, configurationName, this.innerModel(), Context.NONE); + return this; + } + + public MonitoredSubscriptionProperties apply(Context context) { + this.innerObject = serviceManager.serviceClient().getMonitoredSubscriptions().update(resourceGroupName, + monitorName, configurationName, this.innerModel(), context); + return this; + } + + MonitoredSubscriptionPropertiesImpl(MonitoredSubscriptionPropertiesInner innerObject, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.monitorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "monitors"); + this.configurationName = ConfigurationName + .fromString(ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "monitoredSubscriptions")); + } + + public MonitoredSubscriptionProperties refresh() { + this.innerObject = serviceManager.serviceClient().getMonitoredSubscriptions() + .getWithResponse(resourceGroupName, monitorName, configurationName, Context.NONE).getValue(); + return this; + } + + public MonitoredSubscriptionProperties refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getMonitoredSubscriptions() + .getWithResponse(resourceGroupName, monitorName, configurationName, context).getValue(); + return this; + } + + public MonitoredSubscriptionPropertiesImpl withProperties(SubscriptionList properties) { + this.innerModel().withProperties(properties); + return this; + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsClientImpl.java new file mode 100644 index 0000000000000..ff12c8a207b3a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsClientImpl.java @@ -0,0 +1,1299 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.newrelicobservability.fluent.MonitoredSubscriptionsClient; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionPropertiesList; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in MonitoredSubscriptionsClient. + */ +public final class MonitoredSubscriptionsClientImpl implements MonitoredSubscriptionsClient { + /** + * The proxy service used to perform REST calls. + */ + private final MonitoredSubscriptionsService service; + + /** + * The service client containing this operation class. + */ + private final NewRelicObservabilityImpl client; + + /** + * Initializes an instance of MonitoredSubscriptionsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + MonitoredSubscriptionsClientImpl(NewRelicObservabilityImpl client) { + this.service = RestProxy.create(MonitoredSubscriptionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for NewRelicObservabilityMonitoredSubscriptions to be used by the proxy + * service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "NewRelicObservabilit") + public interface MonitoredSubscriptionsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredSubscriptions") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredSubscriptions/{configurationName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("configurationName") ConfigurationName configurationName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredSubscriptions/{configurationName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createorUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("configurationName") ConfigurationName configurationName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") MonitoredSubscriptionPropertiesInner body, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredSubscriptions/{configurationName}") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("configurationName") ConfigurationName configurationName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") MonitoredSubscriptionPropertiesInner body, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredSubscriptions/{configurationName}") + @ExpectedResponses({ 202, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("configurationName") ConfigurationName configurationName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String monitorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String monitorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, monitorName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String monitorName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, monitorName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String monitorName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, monitorName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String monitorName) { + return new PagedIterable<>(listAsync(resourceGroupName, monitorName)); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String monitorName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, monitorName, context)); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String monitorName, ConfigurationName configurationName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, configurationName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String monitorName, ConfigurationName configurationName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, monitorName, + configurationName, this.client.getApiVersion(), accept, context); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + return getWithResponseAsync(resourceGroupName, monitorName, configurationName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context) { + return getWithResponseAsync(resourceGroupName, monitorName, configurationName, context).block(); + } + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public MonitoredSubscriptionPropertiesInner get(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + return getWithResponse(resourceGroupName, monitorName, configurationName, Context.NONE).getValue(); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createorUpdateWithResponseAsync(String resourceGroupName, + String monitorName, ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + if (body != null) { + body.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createorUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, configurationName, this.client.getApiVersion(), body, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createorUpdateWithResponseAsync(String resourceGroupName, + String monitorName, ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + if (body != null) { + body.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createorUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + monitorName, configurationName, this.client.getApiVersion(), body, accept, context); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdateAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body) { + Mono>> mono + = createorUpdateWithResponseAsync(resourceGroupName, monitorName, configurationName, body); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), MonitoredSubscriptionPropertiesInner.class, + MonitoredSubscriptionPropertiesInner.class, this.client.getContext()); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdateAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + Mono>> mono + = createorUpdateWithResponseAsync(resourceGroupName, monitorName, configurationName, body); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), MonitoredSubscriptionPropertiesInner.class, + MonitoredSubscriptionPropertiesInner.class, this.client.getContext()); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdateAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = createorUpdateWithResponseAsync(resourceGroupName, monitorName, configurationName, body, context); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), MonitoredSubscriptionPropertiesInner.class, + MonitoredSubscriptionPropertiesInner.class, context); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + return this.beginCreateorUpdateAsync(resourceGroupName, monitorName, configurationName, body).getSyncPoller(); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, MonitoredSubscriptionPropertiesInner> + beginCreateorUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body, Context context) { + return this.beginCreateorUpdateAsync(resourceGroupName, monitorName, configurationName, body, context) + .getSyncPoller(); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createorUpdateAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body) { + return beginCreateorUpdateAsync(resourceGroupName, monitorName, configurationName, body).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createorUpdateAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + return beginCreateorUpdateAsync(resourceGroupName, monitorName, configurationName, body).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createorUpdateAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context) { + return beginCreateorUpdateAsync(resourceGroupName, monitorName, configurationName, body, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public MonitoredSubscriptionPropertiesInner createorUpdate(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + return createorUpdateAsync(resourceGroupName, monitorName, configurationName, body).block(); + } + + /** + * Add the subscriptions that should be monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public MonitoredSubscriptionPropertiesInner createorUpdate(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context) { + return createorUpdateAsync(resourceGroupName, monitorName, configurationName, body, context).block(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + if (body != null) { + body.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, configurationName, this.client.getApiVersion(), body, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + if (body != null) { + body.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + monitorName, configurationName, this.client.getApiVersion(), body, accept, context); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, MonitoredSubscriptionPropertiesInner> + beginUpdateAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, monitorName, configurationName, body); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), MonitoredSubscriptionPropertiesInner.class, + MonitoredSubscriptionPropertiesInner.class, this.client.getContext()); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, MonitoredSubscriptionPropertiesInner> + beginUpdateAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + Mono>> mono + = updateWithResponseAsync(resourceGroupName, monitorName, configurationName, body); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), MonitoredSubscriptionPropertiesInner.class, + MonitoredSubscriptionPropertiesInner.class, this.client.getContext()); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, MonitoredSubscriptionPropertiesInner> + beginUpdateAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, monitorName, configurationName, body, context); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), MonitoredSubscriptionPropertiesInner.class, + MonitoredSubscriptionPropertiesInner.class, context); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, MonitoredSubscriptionPropertiesInner> + beginUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + return this.beginUpdateAsync(resourceGroupName, monitorName, configurationName, body).getSyncPoller(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of the request to update subscriptions needed to be monitored by the + * NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, MonitoredSubscriptionPropertiesInner> + beginUpdate(String resourceGroupName, String monitorName, ConfigurationName configurationName, + MonitoredSubscriptionPropertiesInner body, Context context) { + return this.beginUpdateAsync(resourceGroupName, monitorName, configurationName, body, context).getSyncPoller(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body) { + return beginUpdateAsync(resourceGroupName, monitorName, configurationName, body).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + return beginUpdateAsync(resourceGroupName, monitorName, configurationName, body).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context) { + return beginUpdateAsync(resourceGroupName, monitorName, configurationName, body, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public MonitoredSubscriptionPropertiesInner update(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + final MonitoredSubscriptionPropertiesInner body = null; + return updateAsync(resourceGroupName, monitorName, configurationName, body).block(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param body The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public MonitoredSubscriptionPropertiesInner update(String resourceGroupName, String monitorName, + ConfigurationName configurationName, MonitoredSubscriptionPropertiesInner body, Context context) { + return updateAsync(resourceGroupName, monitorName, configurationName, body, context).block(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, configurationName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + if (configurationName == null) { + return Mono + .error(new IllegalArgumentException("Parameter configurationName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + monitorName, configurationName, this.client.getApiVersion(), accept, context); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, monitorName, configurationName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, monitorName, configurationName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + return this.beginDeleteAsync(resourceGroupName, monitorName, configurationName).getSyncPoller(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context) { + return this.beginDeleteAsync(resourceGroupName, monitorName, configurationName, context).getSyncPoller(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + return beginDeleteAsync(resourceGroupName, monitorName, configurationName).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String monitorName, ConfigurationName configurationName, + Context context) { + return beginDeleteAsync(resourceGroupName, monitorName, configurationName, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + deleteAsync(resourceGroupName, monitorName, configurationName).block(); + } + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName, + Context context) { + deleteAsync(resourceGroupName, monitorName, configurationName, context).block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsImpl.java new file mode 100644 index 0000000000000..02c95f088a1dd --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoredSubscriptionsImpl.java @@ -0,0 +1,169 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.newrelicobservability.fluent.MonitoredSubscriptionsClient; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptions; + +public final class MonitoredSubscriptionsImpl implements MonitoredSubscriptions { + private static final ClientLogger LOGGER = new ClientLogger(MonitoredSubscriptionsImpl.class); + + private final MonitoredSubscriptionsClient innerClient; + + private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; + + public MonitoredSubscriptionsImpl(MonitoredSubscriptionsClient innerClient, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String monitorName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, monitorName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new MonitoredSubscriptionPropertiesImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String monitorName, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, monitorName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new MonitoredSubscriptionPropertiesImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, monitorName, configurationName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new MonitoredSubscriptionPropertiesImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public MonitoredSubscriptionProperties get(String resourceGroupName, String monitorName, + ConfigurationName configurationName) { + MonitoredSubscriptionPropertiesInner inner + = this.serviceClient().get(resourceGroupName, monitorName, configurationName); + if (inner != null) { + return new MonitoredSubscriptionPropertiesImpl(inner, this.manager()); + } else { + return null; + } + } + + public void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName) { + this.serviceClient().delete(resourceGroupName, monitorName, configurationName); + } + + public void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName, + Context context) { + this.serviceClient().delete(resourceGroupName, monitorName, configurationName, context); + } + + public MonitoredSubscriptionProperties getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); + if (monitorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + } + String configurationNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "monitoredSubscriptions"); + if (configurationNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'monitoredSubscriptions'.", id))); + } + ConfigurationName configurationName = ConfigurationName.fromString(configurationNameLocal); + return this.getWithResponse(resourceGroupName, monitorName, configurationName, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); + if (monitorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + } + String configurationNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "monitoredSubscriptions"); + if (configurationNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'monitoredSubscriptions'.", id))); + } + ConfigurationName configurationName = ConfigurationName.fromString(configurationNameLocal); + return this.getWithResponse(resourceGroupName, monitorName, configurationName, context); + } + + public void deleteById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); + if (monitorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + } + String configurationNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "monitoredSubscriptions"); + if (configurationNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'monitoredSubscriptions'.", id))); + } + ConfigurationName configurationName = ConfigurationName.fromString(configurationNameLocal); + this.delete(resourceGroupName, monitorName, configurationName, Context.NONE); + } + + public void deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); + if (monitorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + } + String configurationNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "monitoredSubscriptions"); + if (configurationNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'monitoredSubscriptions'.", id))); + } + ConfigurationName configurationName = ConfigurationName.fromString(configurationNameLocal); + this.delete(resourceGroupName, monitorName, configurationName, context); + } + + private MonitoredSubscriptionsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager() { + return this.serviceManager; + } + + public MonitoredSubscriptionPropertiesImpl define(ConfigurationName name) { + return new MonitoredSubscriptionPropertiesImpl(name, this.manager()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoringTagRulesPropertiesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoringTagRulesPropertiesImpl.java index 316db538be5a3..49f3b352aadd7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoringTagRulesPropertiesImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitoringTagRulesPropertiesImpl.java @@ -16,8 +16,7 @@ public final class MonitoringTagRulesPropertiesImpl implements MonitoringTagRule private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - MonitoringTagRulesPropertiesImpl( - MonitoringTagRulesPropertiesInner innerObject, + MonitoringTagRulesPropertiesImpl(MonitoringTagRulesPropertiesInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsClientImpl.java index 5bd0c27357ca0..bc95b7a057d3e 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsClientImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsClientImpl.java @@ -35,6 +35,7 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.newrelicobservability.fluent.MonitorsClient; import com.azure.resourcemanager.newrelicobservability.fluent.models.AppServiceInfoInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricsStatusResponseInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredResourceInner; @@ -44,6 +45,7 @@ import com.azure.resourcemanager.newrelicobservability.models.AppServicesGetRequest; import com.azure.resourcemanager.newrelicobservability.models.AppServicesListResponse; import com.azure.resourcemanager.newrelicobservability.models.HostsGetRequest; +import com.azure.resourcemanager.newrelicobservability.models.LinkedResourceListResponse; import com.azure.resourcemanager.newrelicobservability.models.MetricsRequest; import com.azure.resourcemanager.newrelicobservability.models.MetricsStatusRequest; import com.azure.resourcemanager.newrelicobservability.models.MonitoredResourceListResponse; @@ -56,17 +58,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in MonitorsClient. */ +/** + * An instance of this class provides access to all the operations defined in MonitorsClient. + */ public final class MonitorsClientImpl implements MonitorsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final MonitorsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final NewRelicObservabilityImpl client; /** * Initializes an instance of MonitorsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ MonitorsClientImpl(NewRelicObservabilityImpl client) { @@ -81,348 +89,259 @@ public final class MonitorsClientImpl implements MonitorsClient { @Host("{$host}") @ServiceInterface(name = "NewRelicObservabilit") public interface MonitorsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/NewRelic.Observability/monitors") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") - @ExpectedResponses({200, 201}) + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @BodyParam("application/json") NewRelicMonitorResourceInner resource, - @HeaderParam("Accept") String accept, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") NewRelicMonitorResourceInner resource, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, @BodyParam("application/json") NewRelicMonitorResourceUpdate properties, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("userEmail") String userEmail, - @PathParam("monitorName") String monitorName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/getMetricRules") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("userEmail") String userEmail, + @PathParam("monitorName") String monitorName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/getMetricRules") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getMetricRules( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @BodyParam("application/json") MetricsRequest request, - @HeaderParam("Accept") String accept, + Mono> getMetricRules(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") MetricsRequest request, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/getMetricStatus") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/getMetricStatus") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getMetricStatus( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @BodyParam("application/json") MetricsStatusRequest request, - @HeaderParam("Accept") String accept, + Mono> getMetricStatus(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") MetricsStatusRequest request, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/listAppServices") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/listAppServices") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAppServices( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @BodyParam("application/json") AppServicesGetRequest request, - @HeaderParam("Accept") String accept, + Mono> listAppServices(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") AppServicesGetRequest request, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/switchBilling") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/switchBilling") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono switchBilling( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @BodyParam("application/json") SwitchBillingRequest request, - @HeaderParam("Accept") String accept, + Mono switchBilling(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") SwitchBillingRequest request, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/listHosts") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/listHosts") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listHosts( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @BodyParam("application/json") HostsGetRequest request, - @HeaderParam("Accept") String accept, + Mono> listHosts(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @BodyParam("application/json") HostsGetRequest request, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredResources") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/monitoredResources") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listMonitoredResources( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> listMonitoredResources(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/listLinkedResources") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listLinkedResources(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/vmHostPayloads") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/vmHostPayloads") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> vmHostPayload( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @HeaderParam("Accept") String accept, - Context context); + Mono> vmHostPayload(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listAppServicesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listHostsNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listMonitoredResourcesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listLinkedResourcesNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -431,17 +350,17 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -450,13 +369,13 @@ public PagedIterable list() { /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -465,28 +384,24 @@ public PagedIterable list(Context context) { /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -494,53 +409,34 @@ private Mono> listByResourceGroupSin } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -549,27 +445,15 @@ private Mono> listByResourceGroupSin final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -578,14 +462,13 @@ private Mono> listByResourceGroupSin */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -594,22 +477,21 @@ private PagedFlux listByResourceGroupAsync(String * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + private PagedFlux listByResourceGroupAsync(String resourceGroupName, + Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByResourceGroup(String resourceGroupName) { @@ -618,14 +500,14 @@ public PagedIterable listByResourceGroup(String re /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { @@ -634,7 +516,7 @@ public PagedIterable listByResourceGroup(String re /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -643,19 +525,15 @@ public PagedIterable listByResourceGroup(String re * @return a NewRelicMonitorResource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String monitorName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String monitorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -666,23 +544,14 @@ private Mono> getByResourceGroupWithRespo } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -692,19 +561,15 @@ private Mono> getByResourceGroupWithRespo * @return a NewRelicMonitorResource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String monitorName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String monitorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -715,20 +580,13 @@ private Mono> getByResourceGroupWithRespo } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context); } /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -744,7 +602,7 @@ private Mono getByResourceGroupAsync(String resour /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -754,14 +612,14 @@ private Mono getByResourceGroupAsync(String resour * @return a NewRelicMonitorResource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String monitorName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String monitorName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, monitorName, context).block(); } /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -776,7 +634,7 @@ public NewRelicMonitorResourceInner getByResourceGroup(String resourceGroupName, /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -786,19 +644,15 @@ public NewRelicMonitorResourceInner getByResourceGroup(String resourceGroupName, * @return a Monitor Resource by NewRelic along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String monitorName, NewRelicMonitorResourceInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -814,24 +668,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, resource, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -842,19 +686,15 @@ private Mono>> createOrUpdateWithResponseAsync( * @return a Monitor Resource by NewRelic along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String monitorName, NewRelicMonitorResourceInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -870,21 +710,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, resource, accept, context); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -894,23 +726,18 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, NewRelicMonitorResourceInner> beginCreateOrUpdateAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, monitorName, resource); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - NewRelicMonitorResourceInner.class, - NewRelicMonitorResourceInner.class, - this.client.getContext()); + private PollerFlux, NewRelicMonitorResourceInner> + beginCreateOrUpdateAsync(String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, monitorName, resource); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), NewRelicMonitorResourceInner.class, NewRelicMonitorResourceInner.class, + this.client.getContext()); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -924,21 +751,16 @@ private PollerFlux, NewRelicMonitorReso private PollerFlux, NewRelicMonitorResourceInner> beginCreateOrUpdateAsync( String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, monitorName, resource, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - NewRelicMonitorResourceInner.class, - NewRelicMonitorResourceInner.class, - context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, monitorName, resource, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), NewRelicMonitorResourceInner.class, NewRelicMonitorResourceInner.class, + context); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -948,14 +770,14 @@ private PollerFlux, NewRelicMonitorReso * @return the {@link SyncPoller} for polling of a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, NewRelicMonitorResourceInner> beginCreateOrUpdate( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { + public SyncPoller, NewRelicMonitorResourceInner> + beginCreateOrUpdate(String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { return this.beginCreateOrUpdateAsync(resourceGroupName, monitorName, resource).getSyncPoller(); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -973,7 +795,7 @@ public SyncPoller, NewRelicMonitorResou /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -983,16 +805,15 @@ public SyncPoller, NewRelicMonitorResou * @return a Monitor Resource by NewRelic on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { - return beginCreateOrUpdateAsync(resourceGroupName, monitorName, resource) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String monitorName, + NewRelicMonitorResourceInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, monitorName, resource).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -1003,16 +824,15 @@ private Mono createOrUpdateAsync( * @return a Monitor Resource by NewRelic on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, monitorName, resource, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String monitorName, + NewRelicMonitorResourceInner resource, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, monitorName, resource, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -1022,14 +842,14 @@ private Mono createOrUpdateAsync( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NewRelicMonitorResourceInner createOrUpdate( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource) { + public NewRelicMonitorResourceInner createOrUpdate(String resourceGroupName, String monitorName, + NewRelicMonitorResourceInner resource) { return createOrUpdateAsync(resourceGroupName, monitorName, resource).block(); } /** * Create a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param resource Resource create parameters. @@ -1040,14 +860,14 @@ public NewRelicMonitorResourceInner createOrUpdate( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NewRelicMonitorResourceInner createOrUpdate( - String resourceGroupName, String monitorName, NewRelicMonitorResourceInner resource, Context context) { + public NewRelicMonitorResourceInner createOrUpdate(String resourceGroupName, String monitorName, + NewRelicMonitorResourceInner resource, Context context) { return createOrUpdateAsync(resourceGroupName, monitorName, resource, context).block(); } /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -1057,19 +877,15 @@ public NewRelicMonitorResourceInner createOrUpdate( * @return a Monitor Resource by NewRelic along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String monitorName, NewRelicMonitorResourceUpdate properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1085,24 +901,14 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - properties, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, properties, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -1113,19 +919,15 @@ private Mono> updateWithResponseAsync( * @return a Monitor Resource by NewRelic along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties, Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String monitorName, NewRelicMonitorResourceUpdate properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1141,21 +943,13 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - properties, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, properties, accept, context); } /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -1165,15 +959,15 @@ private Mono> updateWithResponseAsync( * @return a Monitor Resource by NewRelic on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties) { + private Mono updateAsync(String resourceGroupName, String monitorName, + NewRelicMonitorResourceUpdate properties) { return updateWithResponseAsync(resourceGroupName, monitorName, properties) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -1184,14 +978,14 @@ private Mono updateAsync( * @return a Monitor Resource by NewRelic along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties, Context context) { + public Response updateWithResponse(String resourceGroupName, String monitorName, + NewRelicMonitorResourceUpdate properties, Context context) { return updateWithResponseAsync(resourceGroupName, monitorName, properties, context).block(); } /** * Update a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param properties The resource properties to be updated. @@ -1201,14 +995,14 @@ public Response updateWithResponse( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NewRelicMonitorResourceInner update( - String resourceGroupName, String monitorName, NewRelicMonitorResourceUpdate properties) { + public NewRelicMonitorResourceInner update(String resourceGroupName, String monitorName, + NewRelicMonitorResourceUpdate properties) { return updateWithResponse(resourceGroupName, monitorName, properties, Context.NONE).getValue(); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1218,19 +1012,15 @@ public NewRelicMonitorResourceInner update( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String userEmail, String monitorName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String userEmail, + String monitorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1244,24 +1034,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - userEmail, - monitorName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, userEmail, monitorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1272,19 +1052,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String userEmail, String monitorName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String userEmail, + String monitorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1298,21 +1074,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - userEmail, - monitorName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, userEmail, monitorName, accept, context); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1322,18 +1090,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String userEmail, String monitorName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String userEmail, + String monitorName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, userEmail, monitorName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1344,19 +1110,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String userEmail, String monitorName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String userEmail, + String monitorName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, userEmail, monitorName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, userEmail, monitorName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1366,14 +1131,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String userEmail, String monitorName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String userEmail, + String monitorName) { return this.beginDeleteAsync(resourceGroupName, userEmail, monitorName).getSyncPoller(); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1384,14 +1149,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String userEmail, String monitorName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String userEmail, + String monitorName, Context context) { return this.beginDeleteAsync(resourceGroupName, userEmail, monitorName, context).getSyncPoller(); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1402,14 +1167,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String userEmail, String monitorName) { - return beginDeleteAsync(resourceGroupName, userEmail, monitorName) - .last() + return beginDeleteAsync(resourceGroupName, userEmail, monitorName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1421,14 +1185,13 @@ private Mono deleteAsync(String resourceGroupName, String userEmail, Strin */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String userEmail, String monitorName, Context context) { - return beginDeleteAsync(resourceGroupName, userEmail, monitorName, context) - .last() + return beginDeleteAsync(resourceGroupName, userEmail, monitorName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1443,7 +1206,7 @@ public void delete(String resourceGroupName, String userEmail, String monitorNam /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -1459,7 +1222,7 @@ public void delete(String resourceGroupName, String userEmail, String monitorNam /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1469,19 +1232,15 @@ public void delete(String resourceGroupName, String userEmail, String monitorNam * @return metric rules along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getMetricRulesWithResponseAsync( - String resourceGroupName, String monitorName, MetricsRequest request) { + private Mono> getMetricRulesWithResponseAsync(String resourceGroupName, + String monitorName, MetricsRequest request) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1497,24 +1256,14 @@ private Mono> getMetricRulesWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getMetricRules( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context)) + .withContext(context -> service.getMetricRules(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1525,19 +1274,15 @@ private Mono> getMetricRulesWithResponseAsync( * @return metric rules along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getMetricRulesWithResponseAsync( - String resourceGroupName, String monitorName, MetricsRequest request, Context context) { + private Mono> getMetricRulesWithResponseAsync(String resourceGroupName, + String monitorName, MetricsRequest request, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1553,21 +1298,13 @@ private Mono> getMetricRulesWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getMetricRules( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context); + return service.getMetricRules(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context); } /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1577,15 +1314,15 @@ private Mono> getMetricRulesWithResponseAsync( * @return metric rules on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getMetricRulesAsync( - String resourceGroupName, String monitorName, MetricsRequest request) { + private Mono getMetricRulesAsync(String resourceGroupName, String monitorName, + MetricsRequest request) { return getMetricRulesWithResponseAsync(resourceGroupName, monitorName, request) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1596,14 +1333,14 @@ private Mono getMetricRulesAsync( * @return metric rules along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getMetricRulesWithResponse( - String resourceGroupName, String monitorName, MetricsRequest request, Context context) { + public Response getMetricRulesWithResponse(String resourceGroupName, String monitorName, + MetricsRequest request, Context context) { return getMetricRulesWithResponseAsync(resourceGroupName, monitorName, request, context).block(); } /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1619,7 +1356,7 @@ public MetricRulesInner getMetricRules(String resourceGroupName, String monitorN /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1629,19 +1366,15 @@ public MetricRulesInner getMetricRules(String resourceGroupName, String monitorN * @return metric status along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getMetricStatusWithResponseAsync( - String resourceGroupName, String monitorName, MetricsStatusRequest request) { + private Mono> getMetricStatusWithResponseAsync(String resourceGroupName, + String monitorName, MetricsStatusRequest request) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1657,24 +1390,14 @@ private Mono> getMetricStatusWithResponseAs } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getMetricStatus( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context)) + .withContext(context -> service.getMetricStatus(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1685,19 +1408,15 @@ private Mono> getMetricStatusWithResponseAs * @return metric status along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getMetricStatusWithResponseAsync( - String resourceGroupName, String monitorName, MetricsStatusRequest request, Context context) { + private Mono> getMetricStatusWithResponseAsync(String resourceGroupName, + String monitorName, MetricsStatusRequest request, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1713,21 +1432,13 @@ private Mono> getMetricStatusWithResponseAs } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getMetricStatus( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context); + return service.getMetricStatus(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context); } /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1737,15 +1448,15 @@ private Mono> getMetricStatusWithResponseAs * @return metric status on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getMetricStatusAsync( - String resourceGroupName, String monitorName, MetricsStatusRequest request) { + private Mono getMetricStatusAsync(String resourceGroupName, String monitorName, + MetricsStatusRequest request) { return getMetricStatusWithResponseAsync(resourceGroupName, monitorName, request) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1756,14 +1467,14 @@ private Mono getMetricStatusAsync( * @return metric status along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getMetricStatusWithResponse( - String resourceGroupName, String monitorName, MetricsStatusRequest request, Context context) { + public Response getMetricStatusWithResponse(String resourceGroupName, + String monitorName, MetricsStatusRequest request, Context context) { return getMetricStatusWithResponseAsync(resourceGroupName, monitorName, request, context).block(); } /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -1773,14 +1484,14 @@ public Response getMetricStatusWithResponse( * @return metric status. */ @ServiceMethod(returns = ReturnType.SINGLE) - public MetricsStatusResponseInner getMetricStatus( - String resourceGroupName, String monitorName, MetricsStatusRequest request) { + public MetricsStatusResponseInner getMetricStatus(String resourceGroupName, String monitorName, + MetricsStatusRequest request) { return getMetricStatusWithResponse(resourceGroupName, monitorName, request, Context.NONE).getValue(); } /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -1788,22 +1499,18 @@ public MetricsStatusResponseInner getMetricStatus( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list app services Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAppServicesSinglePageAsync( - String resourceGroupName, String monitorName, AppServicesGetRequest request) { + private Mono> listAppServicesSinglePageAsync(String resourceGroupName, + String monitorName, AppServicesGetRequest request) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1819,33 +1526,16 @@ private Mono> listAppServicesSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listAppServices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listAppServices(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -1854,22 +1544,18 @@ private Mono> listAppServicesSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list app services Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAppServicesSinglePageAsync( - String resourceGroupName, String monitorName, AppServicesGetRequest request, Context context) { + private Mono> listAppServicesSinglePageAsync(String resourceGroupName, + String monitorName, AppServicesGetRequest request, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1886,29 +1572,15 @@ private Mono> listAppServicesSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listAppServices( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listAppServices(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, request, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -1918,16 +1590,15 @@ private Mono> listAppServicesSinglePageAsync( * @return response of a list app services Operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAppServicesAsync( - String resourceGroupName, String monitorName, AppServicesGetRequest request) { - return new PagedFlux<>( - () -> listAppServicesSinglePageAsync(resourceGroupName, monitorName, request), + private PagedFlux listAppServicesAsync(String resourceGroupName, String monitorName, + AppServicesGetRequest request) { + return new PagedFlux<>(() -> listAppServicesSinglePageAsync(resourceGroupName, monitorName, request), nextLink -> listAppServicesNextSinglePageAsync(nextLink)); } /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -1938,16 +1609,15 @@ private PagedFlux listAppServicesAsync( * @return response of a list app services Operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAppServicesAsync( - String resourceGroupName, String monitorName, AppServicesGetRequest request, Context context) { - return new PagedFlux<>( - () -> listAppServicesSinglePageAsync(resourceGroupName, monitorName, request, context), + private PagedFlux listAppServicesAsync(String resourceGroupName, String monitorName, + AppServicesGetRequest request, Context context) { + return new PagedFlux<>(() -> listAppServicesSinglePageAsync(resourceGroupName, monitorName, request, context), nextLink -> listAppServicesNextSinglePageAsync(nextLink, context)); } /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -1957,14 +1627,14 @@ private PagedFlux listAppServicesAsync( * @return response of a list app services Operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request) { + public PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request) { return new PagedIterable<>(listAppServicesAsync(resourceGroupName, monitorName, request)); } /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -1975,14 +1645,14 @@ public PagedIterable listAppServices( * @return response of a list app services Operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request, Context context) { + public PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request, Context context) { return new PagedIterable<>(listAppServicesAsync(resourceGroupName, monitorName, request, context)); } /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -1992,19 +1662,15 @@ public PagedIterable listAppServices( * @return a Monitor Resource by NewRelic on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono switchBillingWithResponseAsync( - String resourceGroupName, String monitorName, SwitchBillingRequest request) { + private Mono switchBillingWithResponseAsync(String resourceGroupName, + String monitorName, SwitchBillingRequest request) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2020,24 +1686,14 @@ private Mono switchBillingWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .switchBilling( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context)) + .withContext(context -> service.switchBilling(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -2048,19 +1704,15 @@ private Mono switchBillingWithResponseAsync( * @return a Monitor Resource by NewRelic on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono switchBillingWithResponseAsync( - String resourceGroupName, String monitorName, SwitchBillingRequest request, Context context) { + private Mono switchBillingWithResponseAsync(String resourceGroupName, + String monitorName, SwitchBillingRequest request, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2076,21 +1728,13 @@ private Mono switchBillingWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .switchBilling( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context); + return service.switchBilling(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context); } /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -2100,15 +1744,15 @@ private Mono switchBillingWithResponseAsync( * @return a Monitor Resource by NewRelic on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono switchBillingAsync( - String resourceGroupName, String monitorName, SwitchBillingRequest request) { + private Mono switchBillingAsync(String resourceGroupName, String monitorName, + SwitchBillingRequest request) { return switchBillingWithResponseAsync(resourceGroupName, monitorName, request) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -2119,14 +1763,14 @@ private Mono switchBillingAsync( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - public MonitorsSwitchBillingResponse switchBillingWithResponse( - String resourceGroupName, String monitorName, SwitchBillingRequest request, Context context) { + public MonitorsSwitchBillingResponse switchBillingWithResponse(String resourceGroupName, String monitorName, + SwitchBillingRequest request, Context context) { return switchBillingWithResponseAsync(resourceGroupName, monitorName, request, context).block(); } /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -2136,37 +1780,33 @@ public MonitorsSwitchBillingResponse switchBillingWithResponse( * @return a Monitor Resource by NewRelic. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NewRelicMonitorResourceInner switchBilling( - String resourceGroupName, String monitorName, SwitchBillingRequest request) { + public NewRelicMonitorResourceInner switchBilling(String resourceGroupName, String monitorName, + SwitchBillingRequest request) { return switchBillingWithResponse(resourceGroupName, monitorName, request, Context.NONE).getValue(); } /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listHostsSinglePageAsync( - String resourceGroupName, String monitorName, HostsGetRequest request) { + private Mono> listHostsSinglePageAsync(String resourceGroupName, String monitorName, + HostsGetRequest request) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2182,33 +1822,16 @@ private Mono> listHostsSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listHosts( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listHosts(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, request, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -2216,23 +1839,19 @@ private Mono> listHostsSinglePageAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listHostsSinglePageAsync( - String resourceGroupName, String monitorName, HostsGetRequest request, Context context) { + private Mono> listHostsSinglePageAsync(String resourceGroupName, String monitorName, + HostsGetRequest request, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2249,29 +1868,15 @@ private Mono> listHostsSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listHosts( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - request, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listHosts(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, request, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -2281,16 +1886,15 @@ private Mono> listHostsSinglePageAsync( * @return response of a list VM Host Operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listHostsAsync( - String resourceGroupName, String monitorName, HostsGetRequest request) { - return new PagedFlux<>( - () -> listHostsSinglePageAsync(resourceGroupName, monitorName, request), + private PagedFlux listHostsAsync(String resourceGroupName, String monitorName, + HostsGetRequest request) { + return new PagedFlux<>(() -> listHostsSinglePageAsync(resourceGroupName, monitorName, request), nextLink -> listHostsNextSinglePageAsync(nextLink)); } /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -2301,16 +1905,15 @@ private PagedFlux listHostsAsync( * @return response of a list VM Host Operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listHostsAsync( - String resourceGroupName, String monitorName, HostsGetRequest request, Context context) { - return new PagedFlux<>( - () -> listHostsSinglePageAsync(resourceGroupName, monitorName, request, context), + private PagedFlux listHostsAsync(String resourceGroupName, String monitorName, HostsGetRequest request, + Context context) { + return new PagedFlux<>(() -> listHostsSinglePageAsync(resourceGroupName, monitorName, request, context), nextLink -> listHostsNextSinglePageAsync(nextLink, context)); } /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -2326,7 +1929,7 @@ public PagedIterable listHosts(String resourceGroupName, String mon /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -2337,36 +1940,32 @@ public PagedIterable listHosts(String resourceGroupName, String mon * @return response of a list VM Host Operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listHosts( - String resourceGroupName, String monitorName, HostsGetRequest request, Context context) { + public PagedIterable listHosts(String resourceGroupName, String monitorName, HostsGetRequest request, + Context context) { return new PagedIterable<>(listHostsAsync(resourceGroupName, monitorName, request, context)); } /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the resources being monitored by NewRelic monitor resource along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listMonitoredResourcesSinglePageAsync( - String resourceGroupName, String monitorName) { + private Mono> listMonitoredResourcesSinglePageAsync(String resourceGroupName, + String monitorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2378,31 +1977,16 @@ private Mono> listMonitoredResourcesSingle final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listMonitoredResources( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listMonitoredResources(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -2410,22 +1994,18 @@ private Mono> listMonitoredResourcesSingle * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the resources being monitored by NewRelic monitor resource along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listMonitoredResourcesSinglePageAsync( - String resourceGroupName, String monitorName, Context context) { + private Mono> listMonitoredResourcesSinglePageAsync(String resourceGroupName, + String monitorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2437,74 +2017,59 @@ private Mono> listMonitoredResourcesSingle final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listMonitoredResources( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listMonitoredResources(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedFlux}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listMonitoredResourcesAsync( - String resourceGroupName, String monitorName) { - return new PagedFlux<>( - () -> listMonitoredResourcesSinglePageAsync(resourceGroupName, monitorName), + private PagedFlux listMonitoredResourcesAsync(String resourceGroupName, + String monitorName) { + return new PagedFlux<>(() -> listMonitoredResourcesSinglePageAsync(resourceGroupName, monitorName), nextLink -> listMonitoredResourcesNextSinglePageAsync(nextLink)); } /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedFlux}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listMonitoredResourcesAsync( - String resourceGroupName, String monitorName, Context context) { - return new PagedFlux<>( - () -> listMonitoredResourcesSinglePageAsync(resourceGroupName, monitorName, context), + private PagedFlux listMonitoredResourcesAsync(String resourceGroupName, String monitorName, + Context context) { + return new PagedFlux<>(() -> listMonitoredResourcesSinglePageAsync(resourceGroupName, monitorName, context), nextLink -> listMonitoredResourcesNextSinglePageAsync(nextLink, context)); } /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listMonitoredResources(String resourceGroupName, String monitorName) { @@ -2513,47 +2078,185 @@ public PagedIterable listMonitoredResources(String resou /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listMonitoredResources( - String resourceGroupName, String monitorName, Context context) { + public PagedIterable listMonitoredResources(String resourceGroupName, String monitorName, + Context context) { return new PagedIterable<>(listMonitoredResourcesAsync(resourceGroupName, monitorName, context)); } /** - * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response of payload to be passed while installing VM agent along with {@link Response} on successful - * completion of {@link Mono}. + * @return response of a list operation along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> vmHostPayloadWithResponseAsync( - String resourceGroupName, String monitorName) { + private Mono> listLinkedResourcesSinglePageAsync(String resourceGroupName, + String monitorName) { if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listLinkedResources(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listLinkedResourcesSinglePageAsync(String resourceGroupName, + String monitorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (monitorName == null) { + return Mono.error(new IllegalArgumentException("Parameter monitorName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listLinkedResources(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + monitorName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listLinkedResourcesAsync(String resourceGroupName, String monitorName) { + return new PagedFlux<>(() -> listLinkedResourcesSinglePageAsync(resourceGroupName, monitorName), + nextLink -> listLinkedResourcesNextSinglePageAsync(nextLink)); + } + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listLinkedResourcesAsync(String resourceGroupName, String monitorName, + Context context) { + return new PagedFlux<>(() -> listLinkedResourcesSinglePageAsync(resourceGroupName, monitorName, context), + nextLink -> listLinkedResourcesNextSinglePageAsync(nextLink, context)); + } + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listLinkedResources(String resourceGroupName, String monitorName) { + return new PagedIterable<>(listLinkedResourcesAsync(resourceGroupName, monitorName)); + } + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listLinkedResources(String resourceGroupName, String monitorName, + Context context) { + return new PagedIterable<>(listLinkedResourcesAsync(resourceGroupName, monitorName, context)); + } + + /** + * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of payload to be passed while installing VM agent along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> vmHostPayloadWithResponseAsync(String resourceGroupName, + String monitorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2564,23 +2267,14 @@ private Mono> vmHostPayloadWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .vmHostPayload( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context)) + .withContext(context -> service.vmHostPayload(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -2588,22 +2282,18 @@ private Mono> vmHostPayloadWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of payload to be passed while installing VM agent along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> vmHostPayloadWithResponseAsync( - String resourceGroupName, String monitorName, Context context) { + private Mono> vmHostPayloadWithResponseAsync(String resourceGroupName, + String monitorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2614,20 +2304,13 @@ private Mono> vmHostPayloadWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .vmHostPayload( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context); + return service.vmHostPayload(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context); } /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2643,7 +2326,7 @@ private Mono vmHostPayloadAsync(String resourceGroupNam /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -2653,14 +2336,14 @@ private Mono vmHostPayloadAsync(String resourceGroupNam * @return response of payload to be passed while installing VM agent along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response vmHostPayloadWithResponse( - String resourceGroupName, String monitorName, Context context) { + public Response vmHostPayloadWithResponse(String resourceGroupName, String monitorName, + Context context) { return vmHostPayloadWithResponseAsync(resourceGroupName, monitorName, context).block(); } /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2675,14 +2358,15 @@ public VMExtensionPayloadInner vmHostPayload(String resourceGroupName, String mo /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -2690,76 +2374,59 @@ private Mono> listBySubscriptionNext return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -2767,76 +2434,59 @@ private Mono> listByResourceGroupNex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a NewRelicMonitorResource list operation along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list app services Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAppServicesNextSinglePageAsync(String nextLink) { @@ -2844,75 +2494,58 @@ private Mono> listAppServicesNextSinglePageAs return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listAppServicesNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list app services Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAppServicesNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listAppServicesNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAppServicesNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAppServicesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listHostsNextSinglePageAsync(String nextLink) { @@ -2920,37 +2553,29 @@ private Mono> listHostsNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listHostsNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return response of a list VM Host Operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listHostsNextSinglePageAsync(String nextLink, Context context) { @@ -2958,36 +2583,27 @@ private Mono> listHostsNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listHostsNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listHostsNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the resources being monitored by NewRelic monitor resource along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listMonitoredResourcesNextSinglePageAsync(String nextLink) { @@ -2995,63 +2611,103 @@ private Mono> listMonitoredResourcesNextSi return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listMonitoredResourcesNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the resources being monitored by NewRelic monitor resource along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listMonitoredResourcesNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listMonitoredResourcesNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listMonitoredResourcesNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listMonitoredResourcesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listLinkedResourcesNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listLinkedResourcesNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listLinkedResourcesNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listLinkedResourcesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsImpl.java index 6d8880b1276ba..6b5c40a49849f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/MonitorsImpl.java @@ -11,6 +11,7 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.newrelicobservability.fluent.MonitorsClient; import com.azure.resourcemanager.newrelicobservability.fluent.models.AppServiceInfoInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricsStatusResponseInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredResourceInner; @@ -20,6 +21,7 @@ import com.azure.resourcemanager.newrelicobservability.models.AppServiceInfo; import com.azure.resourcemanager.newrelicobservability.models.AppServicesGetRequest; import com.azure.resourcemanager.newrelicobservability.models.HostsGetRequest; +import com.azure.resourcemanager.newrelicobservability.models.LinkedResource; import com.azure.resourcemanager.newrelicobservability.models.MetricRules; import com.azure.resourcemanager.newrelicobservability.models.MetricsRequest; import com.azure.resourcemanager.newrelicobservability.models.MetricsStatusRequest; @@ -39,8 +41,7 @@ public final class MonitorsImpl implements Monitors { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - public MonitorsImpl( - MonitorsClient innerClient, + public MonitorsImpl(MonitorsClient innerClient, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -48,34 +49,31 @@ public MonitorsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NewRelicMonitorResourceImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String monitorName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, monitorName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String monitorName, Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, monitorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new NewRelicMonitorResourceImpl(inner.getValue(), this.manager())); } else { return null; @@ -99,15 +97,12 @@ public void delete(String resourceGroupName, String userEmail, String monitorNam this.serviceClient().delete(resourceGroupName, userEmail, monitorName, context); } - public Response getMetricRulesWithResponse( - String resourceGroupName, String monitorName, MetricsRequest request, Context context) { - Response inner = - this.serviceClient().getMetricRulesWithResponse(resourceGroupName, monitorName, request, context); + public Response getMetricRulesWithResponse(String resourceGroupName, String monitorName, + MetricsRequest request, Context context) { + Response inner + = this.serviceClient().getMetricRulesWithResponse(resourceGroupName, monitorName, request, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new MetricRulesImpl(inner.getValue(), this.manager())); } else { return null; @@ -123,25 +118,22 @@ public MetricRules getMetricRules(String resourceGroupName, String monitorName, } } - public Response getMetricStatusWithResponse( - String resourceGroupName, String monitorName, MetricsStatusRequest request, Context context) { - Response inner = - this.serviceClient().getMetricStatusWithResponse(resourceGroupName, monitorName, request, context); + public Response getMetricStatusWithResponse(String resourceGroupName, String monitorName, + MetricsStatusRequest request, Context context) { + Response inner + = this.serviceClient().getMetricStatusWithResponse(resourceGroupName, monitorName, request, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new MetricsStatusResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public MetricsStatusResponse getMetricStatus( - String resourceGroupName, String monitorName, MetricsStatusRequest request) { - MetricsStatusResponseInner inner = - this.serviceClient().getMetricStatus(resourceGroupName, monitorName, request); + public MetricsStatusResponse getMetricStatus(String resourceGroupName, String monitorName, + MetricsStatusRequest request) { + MetricsStatusResponseInner inner + = this.serviceClient().getMetricStatus(resourceGroupName, monitorName, request); if (inner != null) { return new MetricsStatusResponseImpl(inner, this.manager()); } else { @@ -149,39 +141,36 @@ public MetricsStatusResponse getMetricStatus( } } - public PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request) { - PagedIterable inner = - this.serviceClient().listAppServices(resourceGroupName, monitorName, request); - return Utils.mapPage(inner, inner1 -> new AppServiceInfoImpl(inner1, this.manager())); + public PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request) { + PagedIterable inner + = this.serviceClient().listAppServices(resourceGroupName, monitorName, request); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AppServiceInfoImpl(inner1, this.manager())); } - public PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request, Context context) { - PagedIterable inner = - this.serviceClient().listAppServices(resourceGroupName, monitorName, request, context); - return Utils.mapPage(inner, inner1 -> new AppServiceInfoImpl(inner1, this.manager())); + public PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request, Context context) { + PagedIterable inner + = this.serviceClient().listAppServices(resourceGroupName, monitorName, request, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AppServiceInfoImpl(inner1, this.manager())); } - public Response switchBillingWithResponse( - String resourceGroupName, String monitorName, SwitchBillingRequest request, Context context) { - MonitorsSwitchBillingResponse inner = - this.serviceClient().switchBillingWithResponse(resourceGroupName, monitorName, request, context); + public Response switchBillingWithResponse(String resourceGroupName, String monitorName, + SwitchBillingRequest request, Context context) { + MonitorsSwitchBillingResponse inner + = this.serviceClient().switchBillingWithResponse(resourceGroupName, monitorName, request, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new NewRelicMonitorResourceImpl(inner.getValue(), this.manager())); } else { return null; } } - public NewRelicMonitorResource switchBilling( - String resourceGroupName, String monitorName, SwitchBillingRequest request) { - NewRelicMonitorResourceInner inner = - this.serviceClient().switchBilling(resourceGroupName, monitorName, request); + public NewRelicMonitorResource switchBilling(String resourceGroupName, String monitorName, + SwitchBillingRequest request) { + NewRelicMonitorResourceInner inner + = this.serviceClient().switchBilling(resourceGroupName, monitorName, request); if (inner != null) { return new NewRelicMonitorResourceImpl(inner, this.manager()); } else { @@ -191,38 +180,48 @@ public NewRelicMonitorResource switchBilling( public PagedIterable listHosts(String resourceGroupName, String monitorName, HostsGetRequest request) { PagedIterable inner = this.serviceClient().listHosts(resourceGroupName, monitorName, request); - return Utils.mapPage(inner, inner1 -> new VMInfoImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VMInfoImpl(inner1, this.manager())); } - public PagedIterable listHosts( - String resourceGroupName, String monitorName, HostsGetRequest request, Context context) { - PagedIterable inner = - this.serviceClient().listHosts(resourceGroupName, monitorName, request, context); - return Utils.mapPage(inner, inner1 -> new VMInfoImpl(inner1, this.manager())); + public PagedIterable listHosts(String resourceGroupName, String monitorName, HostsGetRequest request, + Context context) { + PagedIterable inner + = this.serviceClient().listHosts(resourceGroupName, monitorName, request, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new VMInfoImpl(inner1, this.manager())); } public PagedIterable listMonitoredResources(String resourceGroupName, String monitorName) { - PagedIterable inner = - this.serviceClient().listMonitoredResources(resourceGroupName, monitorName); - return Utils.mapPage(inner, inner1 -> new MonitoredResourceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listMonitoredResources(resourceGroupName, monitorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new MonitoredResourceImpl(inner1, this.manager())); } - public PagedIterable listMonitoredResources( - String resourceGroupName, String monitorName, Context context) { - PagedIterable inner = - this.serviceClient().listMonitoredResources(resourceGroupName, monitorName, context); - return Utils.mapPage(inner, inner1 -> new MonitoredResourceImpl(inner1, this.manager())); + public PagedIterable listMonitoredResources(String resourceGroupName, String monitorName, + Context context) { + PagedIterable inner + = this.serviceClient().listMonitoredResources(resourceGroupName, monitorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new MonitoredResourceImpl(inner1, this.manager())); } - public Response vmHostPayloadWithResponse( - String resourceGroupName, String monitorName, Context context) { - Response inner = - this.serviceClient().vmHostPayloadWithResponse(resourceGroupName, monitorName, context); + public PagedIterable listLinkedResources(String resourceGroupName, String monitorName) { + PagedIterable inner + = this.serviceClient().listLinkedResources(resourceGroupName, monitorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new LinkedResourceImpl(inner1, this.manager())); + } + + public PagedIterable listLinkedResources(String resourceGroupName, String monitorName, + Context context) { + PagedIterable inner + = this.serviceClient().listLinkedResources(resourceGroupName, monitorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new LinkedResourceImpl(inner1, this.manager())); + } + + public Response vmHostPayloadWithResponse(String resourceGroupName, String monitorName, + Context context) { + Response inner + = this.serviceClient().vmHostPayloadWithResponse(resourceGroupName, monitorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new VMExtensionPayloadImpl(inner.getValue(), this.manager())); } else { return null; @@ -239,39 +238,29 @@ public VMExtensionPayload vmHostPayload(String resourceGroupName, String monitor } public NewRelicMonitorResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String monitorName = Utils.getValueFromIdByName(id, "monitors"); + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); if (monitorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, monitorName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String monitorName = Utils.getValueFromIdByName(id, "monitors"); + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); if (monitorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, monitorName, context); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicMonitorResourceImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicMonitorResourceImpl.java index fbf595fb3c19c..8ca7c2d051021 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicMonitorResourceImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicMonitorResourceImpl.java @@ -15,6 +15,7 @@ import com.azure.resourcemanager.newrelicobservability.models.AppServicesGetRequest; import com.azure.resourcemanager.newrelicobservability.models.HostsGetRequest; import com.azure.resourcemanager.newrelicobservability.models.LiftrResourceCategories; +import com.azure.resourcemanager.newrelicobservability.models.LinkedResource; import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; import com.azure.resourcemanager.newrelicobservability.models.MarketplaceSubscriptionStatus; import com.azure.resourcemanager.newrelicobservability.models.MetricRules; @@ -119,6 +120,14 @@ public AccountCreationSource accountCreationSource() { return this.innerModel().accountCreationSource(); } + public String subscriptionState() { + return this.innerModel().subscriptionState(); + } + + public String saaSAzureSubscriptionStatus() { + return this.innerModel().saaSAzureSubscriptionStatus(); + } + public Region region() { return Region.fromName(this.regionName()); } @@ -151,25 +160,19 @@ public NewRelicMonitorResourceImpl withExistingResourceGroup(String resourceGrou } public NewRelicMonitorResource create() { - this.innerObject = - serviceManager - .serviceClient() - .getMonitors() - .createOrUpdate(resourceGroupName, monitorName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getMonitors().createOrUpdate(resourceGroupName, monitorName, + this.innerModel(), Context.NONE); return this; } public NewRelicMonitorResource create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getMonitors() - .createOrUpdate(resourceGroupName, monitorName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getMonitors().createOrUpdate(resourceGroupName, monitorName, + this.innerModel(), context); return this; } - NewRelicMonitorResourceImpl( - String name, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + NewRelicMonitorResourceImpl(String name, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = new NewRelicMonitorResourceInner(); this.serviceManager = serviceManager; this.monitorName = name; @@ -181,51 +184,34 @@ public NewRelicMonitorResourceImpl update() { } public NewRelicMonitorResource apply() { - this.innerObject = - serviceManager - .serviceClient() - .getMonitors() - .updateWithResponse(resourceGroupName, monitorName, updateProperties, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMonitors() + .updateWithResponse(resourceGroupName, monitorName, updateProperties, Context.NONE).getValue(); return this; } public NewRelicMonitorResource apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getMonitors() - .updateWithResponse(resourceGroupName, monitorName, updateProperties, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMonitors() + .updateWithResponse(resourceGroupName, monitorName, updateProperties, context).getValue(); return this; } - NewRelicMonitorResourceImpl( - NewRelicMonitorResourceInner innerObject, + NewRelicMonitorResourceImpl(NewRelicMonitorResourceInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.monitorName = Utils.getValueFromIdByName(innerObject.id(), "monitors"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.monitorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "monitors"); } public NewRelicMonitorResource refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getMonitors() - .getByResourceGroupWithResponse(resourceGroupName, monitorName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMonitors() + .getByResourceGroupWithResponse(resourceGroupName, monitorName, Context.NONE).getValue(); return this; } public NewRelicMonitorResource refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getMonitors() - .getByResourceGroupWithResponse(resourceGroupName, monitorName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getMonitors() + .getByResourceGroupWithResponse(resourceGroupName, monitorName, context).getValue(); return this; } @@ -277,6 +263,14 @@ public PagedIterable listMonitoredResources(Context context) return serviceManager.monitors().listMonitoredResources(resourceGroupName, monitorName, context); } + public PagedIterable listLinkedResources() { + return serviceManager.monitors().listLinkedResources(resourceGroupName, monitorName); + } + + public PagedIterable listLinkedResources(Context context) { + return serviceManager.monitors().listLinkedResources(resourceGroupName, monitorName, context); + } + public Response vmHostPayloadWithResponse(Context context) { return serviceManager.monitors().vmHostPayloadWithResponse(resourceGroupName, monitorName, context); } @@ -315,8 +309,8 @@ public NewRelicMonitorResourceImpl withIdentity(ManagedServiceIdentity identity) } } - public NewRelicMonitorResourceImpl withNewRelicAccountProperties( - NewRelicAccountProperties newRelicAccountProperties) { + public NewRelicMonitorResourceImpl + withNewRelicAccountProperties(NewRelicAccountProperties newRelicAccountProperties) { if (isInCreateMode()) { this.innerModel().withNewRelicAccountProperties(newRelicAccountProperties); return this; @@ -366,6 +360,16 @@ public NewRelicMonitorResourceImpl withAccountCreationSource(AccountCreationSour } } + public NewRelicMonitorResourceImpl withSubscriptionState(String subscriptionState) { + this.innerModel().withSubscriptionState(subscriptionState); + return this; + } + + public NewRelicMonitorResourceImpl withSaaSAzureSubscriptionStatus(String saaSAzureSubscriptionStatus) { + this.innerModel().withSaaSAzureSubscriptionStatus(saaSAzureSubscriptionStatus); + return this; + } + private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityBuilder.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityBuilder.java index 67751d8138dc1..948fbec97902c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityBuilder.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityBuilder.java @@ -14,8 +14,10 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the NewRelicObservabilityImpl type. */ -@ServiceClientBuilder(serviceClients = {NewRelicObservabilityImpl.class}) +/** + * A builder for creating a new instance of the NewRelicObservabilityImpl type. + */ +@ServiceClientBuilder(serviceClients = { NewRelicObservabilityImpl.class }) public final class NewRelicObservabilityBuilder { /* * The ID of the target subscription. @@ -24,7 +26,7 @@ public final class NewRelicObservabilityBuilder { /** * Sets The ID of the target subscription. - * + * * @param subscriptionId the subscriptionId value. * @return the NewRelicObservabilityBuilder. */ @@ -40,7 +42,7 @@ public NewRelicObservabilityBuilder subscriptionId(String subscriptionId) { /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the NewRelicObservabilityBuilder. */ @@ -56,7 +58,7 @@ public NewRelicObservabilityBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the NewRelicObservabilityBuilder. */ @@ -72,7 +74,7 @@ public NewRelicObservabilityBuilder environment(AzureEnvironment environment) { /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the NewRelicObservabilityBuilder. */ @@ -88,7 +90,7 @@ public NewRelicObservabilityBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the NewRelicObservabilityBuilder. */ @@ -104,7 +106,7 @@ public NewRelicObservabilityBuilder defaultPollInterval(Duration defaultPollInte /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the NewRelicObservabilityBuilder. */ @@ -115,30 +117,20 @@ public NewRelicObservabilityBuilder serializerAdapter(SerializerAdapter serializ /** * Builds an instance of NewRelicObservabilityImpl with the provided parameters. - * + * * @return an instance of NewRelicObservabilityImpl. */ public NewRelicObservabilityImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - NewRelicObservabilityImpl client = - new NewRelicObservabilityImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + NewRelicObservabilityImpl client = new NewRelicObservabilityImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityImpl.java index e771fe627b1ac..860f9503958ab 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/NewRelicObservabilityImpl.java @@ -23,6 +23,9 @@ import com.azure.core.util.serializer.SerializerAdapter; import com.azure.core.util.serializer.SerializerEncoding; import com.azure.resourcemanager.newrelicobservability.fluent.AccountsClient; +import com.azure.resourcemanager.newrelicobservability.fluent.BillingInfoesClient; +import com.azure.resourcemanager.newrelicobservability.fluent.ConnectedPartnerResourcesClient; +import com.azure.resourcemanager.newrelicobservability.fluent.MonitoredSubscriptionsClient; import com.azure.resourcemanager.newrelicobservability.fluent.MonitorsClient; import com.azure.resourcemanager.newrelicobservability.fluent.NewRelicObservability; import com.azure.resourcemanager.newrelicobservability.fluent.OperationsClient; @@ -38,156 +41,224 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the NewRelicObservabilityImpl type. */ +/** + * Initializes a new instance of the NewRelicObservabilityImpl type. + */ @ServiceClient(builder = NewRelicObservabilityBuilder.class) public final class NewRelicObservabilityImpl implements NewRelicObservability { - /** The ID of the target subscription. */ + /** + * The ID of the target subscription. + */ private final String subscriptionId; /** * Gets The ID of the target subscription. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The AccountsClient object to access its operations. */ + /** + * The AccountsClient object to access its operations. + */ private final AccountsClient accounts; /** * Gets the AccountsClient object to access its operations. - * + * * @return the AccountsClient object. */ public AccountsClient getAccounts() { return this.accounts; } - /** The MonitorsClient object to access its operations. */ + /** + * The MonitorsClient object to access its operations. + */ private final MonitorsClient monitors; /** * Gets the MonitorsClient object to access its operations. - * + * * @return the MonitorsClient object. */ public MonitorsClient getMonitors() { return this.monitors; } - /** The OrganizationsClient object to access its operations. */ + /** + * The OrganizationsClient object to access its operations. + */ private final OrganizationsClient organizations; /** * Gets the OrganizationsClient object to access its operations. - * + * * @return the OrganizationsClient object. */ public OrganizationsClient getOrganizations() { return this.organizations; } - /** The PlansClient object to access its operations. */ + /** + * The PlansClient object to access its operations. + */ private final PlansClient plans; /** * Gets the PlansClient object to access its operations. - * + * * @return the PlansClient object. */ public PlansClient getPlans() { return this.plans; } - /** The TagRulesClient object to access its operations. */ + /** + * The BillingInfoesClient object to access its operations. + */ + private final BillingInfoesClient billingInfoes; + + /** + * Gets the BillingInfoesClient object to access its operations. + * + * @return the BillingInfoesClient object. + */ + public BillingInfoesClient getBillingInfoes() { + return this.billingInfoes; + } + + /** + * The ConnectedPartnerResourcesClient object to access its operations. + */ + private final ConnectedPartnerResourcesClient connectedPartnerResources; + + /** + * Gets the ConnectedPartnerResourcesClient object to access its operations. + * + * @return the ConnectedPartnerResourcesClient object. + */ + public ConnectedPartnerResourcesClient getConnectedPartnerResources() { + return this.connectedPartnerResources; + } + + /** + * The TagRulesClient object to access its operations. + */ private final TagRulesClient tagRules; /** * Gets the TagRulesClient object to access its operations. - * + * * @return the TagRulesClient object. */ public TagRulesClient getTagRules() { return this.tagRules; } + /** + * The MonitoredSubscriptionsClient object to access its operations. + */ + private final MonitoredSubscriptionsClient monitoredSubscriptions; + + /** + * Gets the MonitoredSubscriptionsClient object to access its operations. + * + * @return the MonitoredSubscriptionsClient object. + */ + public MonitoredSubscriptionsClient getMonitoredSubscriptions() { + return this.monitoredSubscriptions; + } + /** * Initializes an instance of NewRelicObservability client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -195,30 +266,28 @@ public TagRulesClient getTagRules() { * @param subscriptionId The ID of the target subscription. * @param endpoint server parameter. */ - NewRelicObservabilityImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + NewRelicObservabilityImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2022-07-01"; + this.apiVersion = "2024-01-01"; this.operations = new OperationsClientImpl(this); this.accounts = new AccountsClientImpl(this); this.monitors = new MonitorsClientImpl(this); this.organizations = new OrganizationsClientImpl(this); this.plans = new PlansClientImpl(this); + this.billingInfoes = new BillingInfoesClientImpl(this); + this.connectedPartnerResources = new ConnectedPartnerResourcesClientImpl(this); this.tagRules = new TagRulesClientImpl(this); + this.monitoredSubscriptions = new MonitoredSubscriptionsClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -227,7 +296,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -237,7 +306,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -247,26 +316,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -279,19 +337,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationImpl.java index 6339a7b4a6b17..3dc462e32b66d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationImpl.java @@ -15,8 +15,7 @@ public final class OperationImpl implements Operation { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - OperationImpl( - OperationInner innerObject, + OperationImpl(OperationInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsClientImpl.java index aee3a4abf759b..1494f38ad4276 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsClientImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.newrelicobservability.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final NewRelicObservabilityImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(NewRelicObservabilityImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,99 +62,74 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "NewRelicObservabilit") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/NewRelic.Observability/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List the operations for the provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the operations for the provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -157,27 +138,27 @@ private PagedFlux listAsync() { /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List the operations for the provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -186,13 +167,13 @@ public PagedIterable list() { /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -201,14 +182,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -216,37 +198,28 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -254,23 +227,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsImpl.java index 222e5521841f9..b38c24bb37bd9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OperationsImpl.java @@ -19,8 +19,7 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, + public OperationsImpl(OperationsClient innerClient, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public OperationsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationResourceImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationResourceImpl.java index 13feaea00a58b..d419d080a09a7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationResourceImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationResourceImpl.java @@ -14,8 +14,7 @@ public final class OrganizationResourceImpl implements OrganizationResource { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - OrganizationResourceImpl( - OrganizationResourceInner innerObject, + OrganizationResourceImpl(OrganizationResourceInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsClientImpl.java index ce6b533efc2de..2ab8c0aea7e41 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsClientImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsClientImpl.java @@ -30,80 +30,76 @@ import com.azure.resourcemanager.newrelicobservability.models.OrganizationsListResponse; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OrganizationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OrganizationsClient. + */ public final class OrganizationsClientImpl implements OrganizationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OrganizationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final NewRelicObservabilityImpl client; /** * Initializes an instance of OrganizationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OrganizationsClientImpl(NewRelicObservabilityImpl client) { - this.service = - RestProxy.create(OrganizationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OrganizationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for NewRelicObservabilityOrganizations to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for NewRelicObservabilityOrganizations to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "NewRelicObservabilit") public interface OrganizationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/NewRelic.Observability/organizations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("userEmail") String userEmail, - @QueryParam("location") String location, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("userEmail") String userEmail, @QueryParam("location") String location, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all organizations Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String userEmail, String location) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (userEmail == null) { return Mono.error(new IllegalArgumentException("Parameter userEmail is required and cannot be null.")); @@ -113,32 +109,16 @@ private Mono> listSinglePageAsync(Strin } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - userEmail, - location, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), userEmail, location, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. @@ -146,22 +126,18 @@ private Mono> listSinglePageAsync(Strin * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all organizations Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String userEmail, String location, Context context) { + private Mono> listSinglePageAsync(String userEmail, String location, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (userEmail == null) { return Mono.error(new IllegalArgumentException("Parameter userEmail is required and cannot be null.")); @@ -172,28 +148,15 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - userEmail, - location, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), userEmail, + location, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -203,13 +166,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String userEmail, String location) { - return new PagedFlux<>( - () -> listSinglePageAsync(userEmail, location), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(userEmail, location), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. @@ -220,14 +183,13 @@ private PagedFlux listAsync(String userEmail, String */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String userEmail, String location, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(userEmail, location, context), + return new PagedFlux<>(() -> listSinglePageAsync(userEmail, location, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -242,7 +204,7 @@ public PagedIterable list(String userEmail, String lo /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. @@ -258,14 +220,15 @@ public PagedIterable list(String userEmail, String lo /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all organizations Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -273,37 +236,28 @@ private Mono> listNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all organizations Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -311,23 +265,13 @@ private Mono> listNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsImpl.java index 29964833116f3..426898ff2c529 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/OrganizationsImpl.java @@ -19,8 +19,7 @@ public final class OrganizationsImpl implements Organizations { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - public OrganizationsImpl( - OrganizationsClient innerClient, + public OrganizationsImpl(OrganizationsClient innerClient, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public OrganizationsImpl( public PagedIterable list(String userEmail, String location) { PagedIterable inner = this.serviceClient().list(userEmail, location); - return Utils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); } public PagedIterable list(String userEmail, String location, Context context) { PagedIterable inner = this.serviceClient().list(userEmail, location, context); - return Utils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OrganizationResourceImpl(inner1, this.manager())); } private OrganizationsClient serviceClient() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlanDataResourceImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlanDataResourceImpl.java index 8417a3ff9c564..8c37bf733a208 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlanDataResourceImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlanDataResourceImpl.java @@ -16,8 +16,7 @@ public final class PlanDataResourceImpl implements PlanDataResource { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - PlanDataResourceImpl( - PlanDataResourceInner innerObject, + PlanDataResourceImpl(PlanDataResourceInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansClientImpl.java index eb4549b896813..1f99e5b5c0675 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansClientImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansClientImpl.java @@ -30,17 +30,23 @@ import com.azure.resourcemanager.newrelicobservability.models.PlanDataListResponse; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PlansClient. */ +/** + * An instance of this class provides access to all the operations defined in PlansClient. + */ public final class PlansClientImpl implements PlansClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PlansService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final NewRelicObservabilityImpl client; /** * Initializes an instance of PlansClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PlansClientImpl(NewRelicObservabilityImpl client) { @@ -49,89 +55,62 @@ public final class PlansClientImpl implements PlansClient { } /** - * The interface defining all the services for NewRelicObservabilityPlans to be used by the proxy service to perform - * REST calls. + * The interface defining all the services for NewRelicObservabilityPlans to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "NewRelicObservabilit") public interface PlansService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/NewRelic.Observability/plans") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("accountId") String accountId, - @QueryParam("organizationId") String organizationId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("accountId") String accountId, @QueryParam("organizationId") String organizationId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String accountId, String organizationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accountId, - organizationId, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accountId, organizationId, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @param context The context to associate with this operation. @@ -139,48 +118,31 @@ private Mono> listSinglePageAsync(String ac * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String accountId, String organizationId, Context context) { + private Mono> listSinglePageAsync(String accountId, String organizationId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accountId, - organizationId, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accountId, + organizationId, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -190,13 +152,13 @@ private Mono> listSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String accountId, String organizationId) { - return new PagedFlux<>( - () -> listSinglePageAsync(accountId, organizationId), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(accountId, organizationId), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * List plans data. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation as paginated response with {@link PagedFlux}. @@ -205,13 +167,13 @@ private PagedFlux listAsync(String accountId, String orga private PagedFlux listAsync() { final String accountId = null; final String organizationId = null; - return new PagedFlux<>( - () -> listSinglePageAsync(accountId, organizationId), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(accountId, organizationId), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @param context The context to associate with this operation. @@ -222,14 +184,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String accountId, String organizationId, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(accountId, organizationId, context), + return new PagedFlux<>(() -> listSinglePageAsync(accountId, organizationId, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List plans data. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation as paginated response with {@link PagedIterable}. @@ -243,7 +204,7 @@ public PagedIterable list() { /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @param context The context to associate with this operation. @@ -259,14 +220,15 @@ public PagedIterable list(String accountId, String organi /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -274,37 +236,28 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -312,23 +265,13 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansImpl.java index 90f4e3a0f3ca5..b48a97b5aad87 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/PlansImpl.java @@ -19,8 +19,7 @@ public final class PlansImpl implements Plans { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - public PlansImpl( - PlansClient innerClient, + public PlansImpl(PlansClient innerClient, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,12 +27,12 @@ public PlansImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new PlanDataResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PlanDataResourceImpl(inner1, this.manager())); } public PagedIterable list(String accountId, String organizationId, Context context) { PagedIterable inner = this.serviceClient().list(accountId, organizationId, context); - return Utils.mapPage(inner, inner1 -> new PlanDataResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PlanDataResourceImpl(inner1, this.manager())); } private PlansClient serviceClient() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ResourceManagerUtils.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..3590ef2e387a9 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleImpl.java index a83030e577bef..ddf28afca2a4c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleImpl.java @@ -79,25 +79,19 @@ public TagRuleImpl withExistingMonitor(String resourceGroupName, String monitorN } public TagRule create() { - this.innerObject = - serviceManager - .serviceClient() - .getTagRules() - .createOrUpdate(resourceGroupName, monitorName, ruleSetName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getTagRules().createOrUpdate(resourceGroupName, monitorName, + ruleSetName, this.innerModel(), Context.NONE); return this; } public TagRule create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getTagRules() - .createOrUpdate(resourceGroupName, monitorName, ruleSetName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getTagRules().createOrUpdate(resourceGroupName, monitorName, + ruleSetName, this.innerModel(), context); return this; } - TagRuleImpl( - String name, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { + TagRuleImpl(String name, + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = new TagRuleInner(); this.serviceManager = serviceManager; this.ruleSetName = name; @@ -109,52 +103,35 @@ public TagRuleImpl update() { } public TagRule apply() { - this.innerObject = - serviceManager - .serviceClient() - .getTagRules() - .updateWithResponse(resourceGroupName, monitorName, ruleSetName, updateProperties, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getTagRules() + .updateWithResponse(resourceGroupName, monitorName, ruleSetName, updateProperties, Context.NONE).getValue(); return this; } public TagRule apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getTagRules() - .updateWithResponse(resourceGroupName, monitorName, ruleSetName, updateProperties, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getTagRules() + .updateWithResponse(resourceGroupName, monitorName, ruleSetName, updateProperties, context).getValue(); return this; } - TagRuleImpl( - TagRuleInner innerObject, + TagRuleImpl(TagRuleInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.monitorName = Utils.getValueFromIdByName(innerObject.id(), "monitors"); - this.ruleSetName = Utils.getValueFromIdByName(innerObject.id(), "tagRules"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.monitorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "monitors"); + this.ruleSetName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "tagRules"); } public TagRule refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getTagRules() - .getWithResponse(resourceGroupName, monitorName, ruleSetName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getTagRules() + .getWithResponse(resourceGroupName, monitorName, ruleSetName, Context.NONE).getValue(); return this; } public TagRule refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getTagRules() - .getWithResponse(resourceGroupName, monitorName, ruleSetName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getTagRules() + .getWithResponse(resourceGroupName, monitorName, ruleSetName, context).getValue(); return this; } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdateImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdateImpl.java index 0cf157890469b..394d9d937626a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdateImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdateImpl.java @@ -15,8 +15,7 @@ public final class TagRuleUpdateImpl implements TagRuleUpdate { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - TagRuleUpdateImpl( - TagRuleUpdateInner innerObject, + TagRuleUpdateImpl(TagRuleUpdateInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdatePropertiesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdatePropertiesImpl.java index 28601f6977eac..906864e3b94a5 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdatePropertiesImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRuleUpdatePropertiesImpl.java @@ -15,8 +15,7 @@ public final class TagRuleUpdatePropertiesImpl implements TagRuleUpdatePropertie private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - TagRuleUpdatePropertiesImpl( - TagRuleUpdatePropertiesInner innerObject, + TagRuleUpdatePropertiesImpl(TagRuleUpdatePropertiesInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesClientImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesClientImpl.java index d08c7fda411ad..c6a36f438f50c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesClientImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesClientImpl.java @@ -40,17 +40,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in TagRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in TagRulesClient. + */ public final class TagRulesClientImpl implements TagRulesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final TagRulesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final NewRelicObservabilityImpl client; /** * Initializes an instance of TagRulesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ TagRulesClientImpl(NewRelicObservabilityImpl client) { @@ -65,118 +71,83 @@ public final class TagRulesClientImpl implements TagRulesClient { @Host("{$host}") @ServiceInterface(name = "NewRelicObservabilit") public interface TagRulesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByNewRelicMonitorResource( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByNewRelicMonitorResource(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @PathParam("ruleSetName") String ruleSetName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("ruleSetName") String ruleSetName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @PathParam("ruleSetName") String ruleSetName, - @BodyParam("application/json") TagRuleInner resource, - @HeaderParam("Accept") String accept, - Context context); + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("ruleSetName") String ruleSetName, @BodyParam("application/json") TagRuleInner resource, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @PathParam("ruleSetName") String ruleSetName, - @HeaderParam("Accept") String accept, - Context context); + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("ruleSetName") String ruleSetName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/NewRelic.Observability/monitors/{monitorName}/tagRules/{ruleSetName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("monitorName") String monitorName, - @PathParam("ruleSetName") String ruleSetName, - @BodyParam("application/json") TagRuleUpdateInner properties, - @HeaderParam("Accept") String accept, - Context context); + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("monitorName") String monitorName, + @PathParam("ruleSetName") String ruleSetName, @BodyParam("application/json") TagRuleUpdateInner properties, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByNewRelicMonitorResourceNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a TagRule list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNewRelicMonitorResourceSinglePageAsync( - String resourceGroupName, String monitorName) { + private Mono> listByNewRelicMonitorResourceSinglePageAsync(String resourceGroupName, + String monitorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -188,31 +159,16 @@ private Mono> listByNewRelicMonitorResourceSinglePag final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listByNewRelicMonitorResource( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listByNewRelicMonitorResource(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -220,22 +176,18 @@ private Mono> listByNewRelicMonitorResourceSinglePag * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a TagRule list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNewRelicMonitorResourceSinglePageAsync( - String resourceGroupName, String monitorName, Context context) { + private Mono> listByNewRelicMonitorResourceSinglePageAsync(String resourceGroupName, + String monitorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -247,28 +199,15 @@ private Mono> listByNewRelicMonitorResourceSinglePag final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByNewRelicMonitorResource( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByNewRelicMonitorResource(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -278,14 +217,13 @@ private Mono> listByNewRelicMonitorResourceSinglePag */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByNewRelicMonitorResourceAsync(String resourceGroupName, String monitorName) { - return new PagedFlux<>( - () -> listByNewRelicMonitorResourceSinglePageAsync(resourceGroupName, monitorName), + return new PagedFlux<>(() -> listByNewRelicMonitorResourceSinglePageAsync(resourceGroupName, monitorName), nextLink -> listByNewRelicMonitorResourceNextSinglePageAsync(nextLink)); } /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -295,8 +233,8 @@ private PagedFlux listByNewRelicMonitorResourceAsync(String resour * @return the response of a TagRule list operation as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByNewRelicMonitorResourceAsync( - String resourceGroupName, String monitorName, Context context) { + private PagedFlux listByNewRelicMonitorResourceAsync(String resourceGroupName, String monitorName, + Context context) { return new PagedFlux<>( () -> listByNewRelicMonitorResourceSinglePageAsync(resourceGroupName, monitorName, context), nextLink -> listByNewRelicMonitorResourceNextSinglePageAsync(nextLink, context)); @@ -304,7 +242,7 @@ private PagedFlux listByNewRelicMonitorResourceAsync( /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -319,7 +257,7 @@ public PagedIterable listByNewRelicMonitorResource(String resource /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -329,14 +267,14 @@ public PagedIterable listByNewRelicMonitorResource(String resource * @return the response of a TagRule list operation as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByNewRelicMonitorResource( - String resourceGroupName, String monitorName, Context context) { + public PagedIterable listByNewRelicMonitorResource(String resourceGroupName, String monitorName, + Context context) { return new PagedIterable<>(listByNewRelicMonitorResourceAsync(resourceGroupName, monitorName, context)); } /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -346,19 +284,15 @@ public PagedIterable listByNewRelicMonitorResource( * @return a TagRule along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName) { + private Mono> getWithResponseAsync(String resourceGroupName, String monitorName, + String ruleSetName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -372,24 +306,14 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, ruleSetName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -400,19 +324,15 @@ private Mono> getWithResponseAsync( * @return a TagRule along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String monitorName, + String ruleSetName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -426,21 +346,13 @@ private Mono> getWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, ruleSetName, accept, context); } /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -457,7 +369,7 @@ private Mono getAsync(String resourceGroupName, String monitorName /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -468,14 +380,14 @@ private Mono getAsync(String resourceGroupName, String monitorName * @return a TagRule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String monitorName, String ruleSetName, Context context) { + public Response getWithResponse(String resourceGroupName, String monitorName, String ruleSetName, + Context context) { return getWithResponseAsync(resourceGroupName, monitorName, ruleSetName, context).block(); } /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -491,7 +403,7 @@ public TagRuleInner get(String resourceGroupName, String monitorName, String rul /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -499,23 +411,19 @@ public TagRuleInner get(String resourceGroupName, String monitorName, String rul * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of {@link - * Mono}. + * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String monitorName, String ruleSetName, TagRuleInner resource) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -534,25 +442,15 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - resource, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, ruleSetName, resource, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -561,23 +459,19 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of {@link - * Mono}. + * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String monitorName, String ruleSetName, TagRuleInner resource, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -596,22 +490,13 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - resource, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, ruleSetName, resource, accept, context); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -622,19 +507,17 @@ private Mono>> createOrUpdateWithResponseAsync( * @return the {@link PollerFlux} for polling of a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, TagRuleInner> beginCreateOrUpdateAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, monitorName, ruleSetName, resource); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), TagRuleInner.class, TagRuleInner.class, this.client.getContext()); + private PollerFlux, TagRuleInner> beginCreateOrUpdateAsync(String resourceGroupName, + String monitorName, String ruleSetName, TagRuleInner resource) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, monitorName, ruleSetName, resource); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + TagRuleInner.class, TagRuleInner.class, this.client.getContext()); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -646,20 +529,18 @@ private PollerFlux, TagRuleInner> beginCreateOrUpdateAs * @return the {@link PollerFlux} for polling of a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, TagRuleInner> beginCreateOrUpdateAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context) { + private PollerFlux, TagRuleInner> beginCreateOrUpdateAsync(String resourceGroupName, + String monitorName, String ruleSetName, TagRuleInner resource, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, monitorName, ruleSetName, resource, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), TagRuleInner.class, TagRuleInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, monitorName, ruleSetName, resource, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + TagRuleInner.class, TagRuleInner.class, context); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -670,14 +551,14 @@ private PollerFlux, TagRuleInner> beginCreateOrUpdateAs * @return the {@link SyncPoller} for polling of a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, TagRuleInner> beginCreateOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource) { + public SyncPoller, TagRuleInner> beginCreateOrUpdate(String resourceGroupName, + String monitorName, String ruleSetName, TagRuleInner resource) { return this.beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource).getSyncPoller(); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -689,16 +570,15 @@ public SyncPoller, TagRuleInner> beginCreateOrUpdate( * @return the {@link SyncPoller} for polling of a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, TagRuleInner> beginCreateOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource, context) + public SyncPoller, TagRuleInner> beginCreateOrUpdate(String resourceGroupName, + String monitorName, String ruleSetName, TagRuleInner resource, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource, context) .getSyncPoller(); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -709,16 +589,15 @@ public SyncPoller, TagRuleInner> beginCreateOrUpdate( * @return a tag rule belonging to NewRelic account on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource) { - return beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleInner resource) { + return beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -730,16 +609,15 @@ private Mono createOrUpdateAsync( * @return a tag rule belonging to NewRelic account on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleInner resource, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -750,14 +628,14 @@ private Mono createOrUpdateAsync( * @return a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.SINGLE) - public TagRuleInner createOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource) { + public TagRuleInner createOrUpdate(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleInner resource) { return createOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource).block(); } /** * Create a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -769,14 +647,14 @@ public TagRuleInner createOrUpdate( * @return a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.SINGLE) - public TagRuleInner createOrUpdate( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleInner resource, Context context) { + public TagRuleInner createOrUpdate(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleInner resource, Context context) { return createOrUpdateAsync(resourceGroupName, monitorName, ruleSetName, resource, context).block(); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -786,19 +664,15 @@ public TagRuleInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String monitorName, + String ruleSetName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -812,24 +686,14 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, ruleSetName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -840,19 +704,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String monitorName, + String ruleSetName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -866,21 +726,13 @@ private Mono>> deleteWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, ruleSetName, accept, context); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -890,18 +742,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String monitorName, String ruleSetName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String monitorName, + String ruleSetName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, monitorName, ruleSetName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -912,19 +762,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String monitorName, String ruleSetName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String monitorName, + String ruleSetName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, monitorName, ruleSetName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, monitorName, ruleSetName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -934,14 +783,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String monitorName, String ruleSetName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, + String ruleSetName) { return this.beginDeleteAsync(resourceGroupName, monitorName, ruleSetName).getSyncPoller(); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -952,14 +801,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String monitorName, String ruleSetName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String monitorName, + String ruleSetName, Context context) { return this.beginDeleteAsync(resourceGroupName, monitorName, ruleSetName, context).getSyncPoller(); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -970,14 +819,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String monitorName, String ruleSetName) { - return beginDeleteAsync(resourceGroupName, monitorName, ruleSetName) - .last() + return beginDeleteAsync(resourceGroupName, monitorName, ruleSetName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -989,14 +837,13 @@ private Mono deleteAsync(String resourceGroupName, String monitorName, Str */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String monitorName, String ruleSetName, Context context) { - return beginDeleteAsync(resourceGroupName, monitorName, ruleSetName, context) - .last() + return beginDeleteAsync(resourceGroupName, monitorName, ruleSetName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1011,7 +858,7 @@ public void delete(String resourceGroupName, String monitorName, String ruleSetN /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1027,7 +874,7 @@ public void delete(String resourceGroupName, String monitorName, String ruleSetN /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1035,23 +882,19 @@ public void delete(String resourceGroupName, String monitorName, String ruleSetN * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of {@link - * Mono}. + * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleUpdateInner properties) { + private Mono> updateWithResponseAsync(String resourceGroupName, String monitorName, + String ruleSetName, TagRuleUpdateInner properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1069,26 +912,14 @@ private Mono> updateWithResponseAsync( properties.validate(); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - properties, - accept, - context)) + return FluxUtil.withContext(context -> service.update(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, monitorName, ruleSetName, properties, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1097,27 +928,19 @@ private Mono> updateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of {@link - * Mono}. + * @return a tag rule belonging to NewRelic account along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String monitorName, - String ruleSetName, - TagRuleUpdateInner properties, - Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, String monitorName, + String ruleSetName, TagRuleUpdateInner properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1136,22 +959,13 @@ private Mono> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - monitorName, - ruleSetName, - properties, - accept, - context); + return service.update(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, monitorName, ruleSetName, properties, accept, context); } /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1162,15 +976,15 @@ private Mono> updateWithResponseAsync( * @return a tag rule belonging to NewRelic account on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleUpdateInner properties) { + private Mono updateAsync(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleUpdateInner properties) { return updateWithResponseAsync(resourceGroupName, monitorName, ruleSetName, properties) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1182,18 +996,14 @@ private Mono updateAsync( * @return a tag rule belonging to NewRelic account along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String monitorName, - String ruleSetName, - TagRuleUpdateInner properties, - Context context) { + public Response updateWithResponse(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleUpdateInner properties, Context context) { return updateWithResponseAsync(resourceGroupName, monitorName, ruleSetName, properties, context).block(); } /** * Update a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -1204,21 +1014,22 @@ public Response updateWithResponse( * @return a tag rule belonging to NewRelic account. */ @ServiceMethod(returns = ReturnType.SINGLE) - public TagRuleInner update( - String resourceGroupName, String monitorName, String ruleSetName, TagRuleUpdateInner properties) { + public TagRuleInner update(String resourceGroupName, String monitorName, String ruleSetName, + TagRuleUpdateInner properties) { return updateWithResponse(resourceGroupName, monitorName, ruleSetName, properties, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a TagRule list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByNewRelicMonitorResourceNextSinglePageAsync(String nextLink) { @@ -1226,64 +1037,44 @@ private Mono> listByNewRelicMonitorResourceNextSingl return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listByNewRelicMonitorResourceNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listByNewRelicMonitorResourceNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a TagRule list operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByNewRelicMonitorResourceNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByNewRelicMonitorResourceNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByNewRelicMonitorResourceNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByNewRelicMonitorResourceNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesImpl.java index 72b45190028d9..9ddf1405da7cb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/TagRulesImpl.java @@ -21,35 +21,31 @@ public final class TagRulesImpl implements TagRules { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - public TagRulesImpl( - TagRulesClient innerClient, + public TagRulesImpl(TagRulesClient innerClient, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByNewRelicMonitorResource(String resourceGroupName, String monitorName) { - PagedIterable inner = - this.serviceClient().listByNewRelicMonitorResource(resourceGroupName, monitorName); - return Utils.mapPage(inner, inner1 -> new TagRuleImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByNewRelicMonitorResource(resourceGroupName, monitorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TagRuleImpl(inner1, this.manager())); } - public PagedIterable listByNewRelicMonitorResource( - String resourceGroupName, String monitorName, Context context) { - PagedIterable inner = - this.serviceClient().listByNewRelicMonitorResource(resourceGroupName, monitorName, context); - return Utils.mapPage(inner, inner1 -> new TagRuleImpl(inner1, this.manager())); + public PagedIterable listByNewRelicMonitorResource(String resourceGroupName, String monitorName, + Context context) { + PagedIterable inner + = this.serviceClient().listByNewRelicMonitorResource(resourceGroupName, monitorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TagRuleImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String monitorName, String ruleSetName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, monitorName, ruleSetName, context); + public Response getWithResponse(String resourceGroupName, String monitorName, String ruleSetName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, monitorName, ruleSetName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new TagRuleImpl(inner.getValue(), this.manager())); } else { return null; @@ -74,105 +70,77 @@ public void delete(String resourceGroupName, String monitorName, String ruleSetN } public TagRule getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String monitorName = Utils.getValueFromIdByName(id, "monitors"); + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); if (monitorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); } - String ruleSetName = Utils.getValueFromIdByName(id, "tagRules"); + String ruleSetName = ResourceManagerUtils.getValueFromIdByName(id, "tagRules"); if (ruleSetName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); } return this.getWithResponse(resourceGroupName, monitorName, ruleSetName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String monitorName = Utils.getValueFromIdByName(id, "monitors"); + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); if (monitorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); } - String ruleSetName = Utils.getValueFromIdByName(id, "tagRules"); + String ruleSetName = ResourceManagerUtils.getValueFromIdByName(id, "tagRules"); if (ruleSetName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); } return this.getWithResponse(resourceGroupName, monitorName, ruleSetName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String monitorName = Utils.getValueFromIdByName(id, "monitors"); + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); if (monitorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); } - String ruleSetName = Utils.getValueFromIdByName(id, "tagRules"); + String ruleSetName = ResourceManagerUtils.getValueFromIdByName(id, "tagRules"); if (ruleSetName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); } this.delete(resourceGroupName, monitorName, ruleSetName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String monitorName = Utils.getValueFromIdByName(id, "monitors"); + String monitorName = ResourceManagerUtils.getValueFromIdByName(id, "monitors"); if (monitorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'monitors'.", id))); } - String ruleSetName = Utils.getValueFromIdByName(id, "tagRules"); + String ruleSetName = ResourceManagerUtils.getValueFromIdByName(id, "tagRules"); if (ruleSetName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'tagRules'.", id))); } this.delete(resourceGroupName, monitorName, ruleSetName, context); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/Utils.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/Utils.java deleted file mode 100644 index ebc02c51a68de..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMExtensionPayloadImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMExtensionPayloadImpl.java index b39677fa0d3bd..9e75c32537b14 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMExtensionPayloadImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMExtensionPayloadImpl.java @@ -12,8 +12,7 @@ public final class VMExtensionPayloadImpl implements VMExtensionPayload { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - VMExtensionPayloadImpl( - VMExtensionPayloadInner innerObject, + VMExtensionPayloadImpl(VMExtensionPayloadInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMInfoImpl.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMInfoImpl.java index 894cef3b2b8c2..cc20960b980b4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMInfoImpl.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/VMInfoImpl.java @@ -12,8 +12,7 @@ public final class VMInfoImpl implements VMInfo { private final com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager; - VMInfoImpl( - VMInfoInner innerObject, + VMInfoImpl(VMInfoInner innerObject, com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/package-info.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/package-info.java index 2b3418cf1786a..6e9e5fb4037de 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/package-info.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the implementations for NewRelicObservability. null. */ +/** + * Package containing the implementations for NewRelicObservability. + * null. + */ package com.azure.resourcemanager.newrelicobservability.implementation; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountCreationSource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountCreationSource.java index 95f7f6dbfca44..4c2909c34574d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountCreationSource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountCreationSource.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Source of Account creation. */ +/** + * Source of Account creation. + */ public final class AccountCreationSource extends ExpandableStringEnum { - /** Static value LIFTR for AccountCreationSource. */ + /** + * Static value LIFTR for AccountCreationSource. + */ public static final AccountCreationSource LIFTR = fromString("LIFTR"); - /** Static value NEWRELIC for AccountCreationSource. */ + /** + * Static value NEWRELIC for AccountCreationSource. + */ public static final AccountCreationSource NEWRELIC = fromString("NEWRELIC"); /** * Creates a new instance of AccountCreationSource value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AccountCreationSource() { /** * Creates or finds a AccountCreationSource from its string representation. - * + * * @param name a name to look for. * @return the corresponding AccountCreationSource. */ @@ -38,7 +44,7 @@ public static AccountCreationSource fromString(String name) { /** * Gets known AccountCreationSource values. - * + * * @return known AccountCreationSource values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountInfo.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountInfo.java index a8b864c2ad146..bcf9544d539ba 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountInfo.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountInfo.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Account Info of the NewRelic account. */ +/** + * Account Info of the NewRelic account. + */ @Fluent public final class AccountInfo { /* @@ -23,18 +25,20 @@ public final class AccountInfo { private String ingestionKey; /* - * NewRelic account region + * Region where New Relic account is present */ @JsonProperty(value = "region") private String region; - /** Creates an instance of AccountInfo class. */ + /** + * Creates an instance of AccountInfo class. + */ public AccountInfo() { } /** * Get the accountId property: Account id. - * + * * @return the accountId value. */ public String accountId() { @@ -43,7 +47,7 @@ public String accountId() { /** * Set the accountId property: Account id. - * + * * @param accountId the accountId value to set. * @return the AccountInfo object itself. */ @@ -54,7 +58,7 @@ public AccountInfo withAccountId(String accountId) { /** * Get the ingestionKey property: ingestion key of account. - * + * * @return the ingestionKey value. */ public String ingestionKey() { @@ -63,7 +67,7 @@ public String ingestionKey() { /** * Set the ingestionKey property: ingestion key of account. - * + * * @param ingestionKey the ingestionKey value to set. * @return the AccountInfo object itself. */ @@ -73,8 +77,8 @@ public AccountInfo withIngestionKey(String ingestionKey) { } /** - * Get the region property: NewRelic account region. - * + * Get the region property: Region where New Relic account is present. + * * @return the region value. */ public String region() { @@ -82,8 +86,8 @@ public String region() { } /** - * Set the region property: NewRelic account region. - * + * Set the region property: Region where New Relic account is present. + * * @param region the region value to set. * @return the AccountInfo object itself. */ @@ -94,7 +98,7 @@ public AccountInfo withRegion(String region) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountResource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountResource.java index 14d8cad0caa08..3d7bab29e2369 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountResource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountResource.java @@ -7,67 +7,69 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.newrelicobservability.fluent.models.AccountResourceInner; -/** An immutable client-side representation of AccountResource. */ +/** + * An immutable client-side representation of AccountResource. + */ public interface AccountResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the organizationId property: organization id. - * + * * @return the organizationId value. */ String organizationId(); /** * Gets the accountId property: account id. - * + * * @return the accountId value. */ String accountId(); /** * Gets the accountName property: account name. - * + * * @return the accountName value. */ String accountName(); /** - * Gets the region property: region. - * + * Gets the region property: Region where New Relic account is present. + * * @return the region value. */ String region(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.AccountResourceInner object. - * + * * @return the inner object. */ AccountResourceInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Accounts.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Accounts.java index d12ca442f7dcd..ded2222f68a83 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Accounts.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Accounts.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Accounts. */ +/** + * Resource collection API of Accounts. + */ public interface Accounts { /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface Accounts { /** * List all the existing accounts. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountsListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountsListResponse.java index ee3097140c64f..3320e801be2e9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountsListResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AccountsListResponse.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response of get all accounts Operation. */ +/** + * Response of get all accounts Operation. + */ @Fluent public final class AccountsListResponse { /* @@ -25,13 +27,15 @@ public final class AccountsListResponse { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of AccountsListResponse class. */ + /** + * Creates an instance of AccountsListResponse class. + */ public AccountsListResponse() { } /** * Get the value property: The AccountResource items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The AccountResource items on this page. - * + * * @param value the value value to set. * @return the AccountsListResponse object itself. */ @@ -51,7 +55,7 @@ public AccountsListResponse withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the AccountsListResponse object itself. */ @@ -71,14 +75,13 @@ public AccountsListResponse withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model AccountsListResponse")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model AccountsListResponse")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ActionType.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ActionType.java index 4886ca885a69d..865448613306d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ActionType.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ActionType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. */ +/** + * Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. + */ public final class ActionType extends ExpandableStringEnum { - /** Static value Internal for ActionType. */ + /** + * Static value Internal for ActionType. + */ public static final ActionType INTERNAL = fromString("Internal"); /** * Creates a new instance of ActionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ActionType() { /** * Creates or finds a ActionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActionType. */ @@ -35,7 +39,7 @@ public static ActionType fromString(String name) { /** * Gets known ActionType values. - * + * * @return known ActionType values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServiceInfo.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServiceInfo.java index 0469ff65c3a32..7e7e5475267a6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServiceInfo.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServiceInfo.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.AppServiceInfoInner; -/** An immutable client-side representation of AppServiceInfo. */ +/** + * An immutable client-side representation of AppServiceInfo. + */ public interface AppServiceInfo { /** * Gets the azureResourceId property: Azure App service resource ID. - * + * * @return the azureResourceId value. */ String azureResourceId(); /** * Gets the agentVersion property: Version of the NewRelic agent installed on the App service. - * + * * @return the agentVersion value. */ String agentVersion(); /** * Gets the agentStatus property: Status of the NewRelic agent installed on the App service. - * + * * @return the agentStatus value. */ String agentStatus(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.AppServiceInfoInner object. - * + * * @return the inner object. */ AppServiceInfoInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesGetRequest.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesGetRequest.java index 9b7135546362d..2a6ce76478765 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesGetRequest.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesGetRequest.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Request of a app services get Operation. */ +/** + * Request of a app services get Operation. + */ @Fluent public final class AppServicesGetRequest { /* @@ -24,13 +26,15 @@ public final class AppServicesGetRequest { @JsonProperty(value = "userEmail", required = true) private String userEmail; - /** Creates an instance of AppServicesGetRequest class. */ + /** + * Creates an instance of AppServicesGetRequest class. + */ public AppServicesGetRequest() { } /** * Get the azureResourceIds property: Azure resource IDs. - * + * * @return the azureResourceIds value. */ public List azureResourceIds() { @@ -39,7 +43,7 @@ public List azureResourceIds() { /** * Set the azureResourceIds property: Azure resource IDs. - * + * * @param azureResourceIds the azureResourceIds value to set. * @return the AppServicesGetRequest object itself. */ @@ -50,7 +54,7 @@ public AppServicesGetRequest withAzureResourceIds(List azureResourceIds) /** * Get the userEmail property: User Email. - * + * * @return the userEmail value. */ public String userEmail() { @@ -59,7 +63,7 @@ public String userEmail() { /** * Set the userEmail property: User Email. - * + * * @param userEmail the userEmail value to set. * @return the AppServicesGetRequest object itself. */ @@ -70,14 +74,13 @@ public AppServicesGetRequest withUserEmail(String userEmail) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (userEmail() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property userEmail in model AppServicesGetRequest")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property userEmail in model AppServicesGetRequest")); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesListResponse.java index aeba394ecf674..da1f6ef3913c5 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesListResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/AppServicesListResponse.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response of a list app services Operation. */ +/** + * Response of a list app services Operation. + */ @Fluent public final class AppServicesListResponse { /* @@ -25,13 +27,15 @@ public final class AppServicesListResponse { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of AppServicesListResponse class. */ + /** + * Creates an instance of AppServicesListResponse class. + */ public AppServicesListResponse() { } /** * Get the value property: The AppServiceInfo items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The AppServiceInfo items on this page. - * + * * @param value the value value to set. * @return the AppServicesListResponse object itself. */ @@ -51,7 +55,7 @@ public AppServicesListResponse withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the AppServicesListResponse object itself. */ @@ -71,14 +75,13 @@ public AppServicesListResponse withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model AppServicesListResponse")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model AppServicesListResponse")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingCycle.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingCycle.java index 4f150e76b4ee7..6846b132b23c7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingCycle.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingCycle.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Different usage type like YEARLY/MONTHLY. */ +/** + * Different usage type like YEARLY/MONTHLY. + */ public final class BillingCycle extends ExpandableStringEnum { - /** Static value YEARLY for BillingCycle. */ + /** + * Static value YEARLY for BillingCycle. + */ public static final BillingCycle YEARLY = fromString("YEARLY"); - /** Static value MONTHLY for BillingCycle. */ + /** + * Static value MONTHLY for BillingCycle. + */ public static final BillingCycle MONTHLY = fromString("MONTHLY"); - /** Static value WEEKLY for BillingCycle. */ + /** + * Static value WEEKLY for BillingCycle. + */ public static final BillingCycle WEEKLY = fromString("WEEKLY"); /** * Creates a new instance of BillingCycle value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public BillingCycle() { /** * Creates or finds a BillingCycle from its string representation. - * + * * @param name a name to look for. * @return the corresponding BillingCycle. */ @@ -41,7 +49,7 @@ public static BillingCycle fromString(String name) { /** * Gets known BillingCycle values. - * + * * @return known BillingCycle values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoResponse.java new file mode 100644 index 0000000000000..6db10e3cf32af --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoResponse.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner; + +/** + * An immutable client-side representation of BillingInfoResponse. + */ +public interface BillingInfoResponse { + /** + * Gets the marketplaceSaasInfo property: Marketplace Subscription details. + * + * @return the marketplaceSaasInfo value. + */ + MarketplaceSaaSInfo marketplaceSaasInfo(); + + /** + * Gets the partnerBillingEntity property: Partner Billing Entity details: Organization Info. + * + * @return the partnerBillingEntity value. + */ + PartnerBillingEntity partnerBillingEntity(); + + /** + * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner object. + * + * @return the inner object. + */ + BillingInfoResponseInner innerModel(); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoes.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoes.java new file mode 100644 index 0000000000000..16ae0506b26ff --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingInfoes.java @@ -0,0 +1,38 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of BillingInfoes. + */ +public interface BillingInfoes { + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String monitorName, Context context); + + /** + * Get marketplace info mapped to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return marketplace info mapped to the given monitor. + */ + BillingInfoResponse get(String resourceGroupName, String monitorName); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingSource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingSource.java index fb8c2d44ed3cb..1ce36932b649d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingSource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/BillingSource.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Billing source. */ +/** + * Billing source. + */ public final class BillingSource extends ExpandableStringEnum { - /** Static value AZURE for BillingSource. */ + /** + * Static value AZURE for BillingSource. + */ public static final BillingSource AZURE = fromString("AZURE"); - /** Static value NEWRELIC for BillingSource. */ + /** + * Static value NEWRELIC for BillingSource. + */ public static final BillingSource NEWRELIC = fromString("NEWRELIC"); /** * Creates a new instance of BillingSource value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public BillingSource() { /** * Creates or finds a BillingSource from its string representation. - * + * * @param name a name to look for. * @return the corresponding BillingSource. */ @@ -38,7 +44,7 @@ public static BillingSource fromString(String name) { /** * Gets known BillingSource values. - * + * * @return known BillingSource values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConfigurationName.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConfigurationName.java new file mode 100644 index 0000000000000..e3f0b963decaa --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConfigurationName.java @@ -0,0 +1,48 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for ConfigurationName. + */ +public final class ConfigurationName extends ExpandableStringEnum { + /** + * Static value default for ConfigurationName. + */ + public static final ConfigurationName DEFAULT = fromString("default"); + + /** + * Creates a new instance of ConfigurationName value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ConfigurationName() { + } + + /** + * Creates or finds a ConfigurationName from its string representation. + * + * @param name a name to look for. + * @return the corresponding ConfigurationName. + */ + @JsonCreator + public static ConfigurationName fromString(String name) { + return fromString(name, ConfigurationName.class); + } + + /** + * Gets known ConfigurationName values. + * + * @return known ConfigurationName values. + */ + public static Collection values() { + return values(ConfigurationName.class); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourceProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourceProperties.java new file mode 100644 index 0000000000000..9ca3d700c1e5f --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourceProperties.java @@ -0,0 +1,132 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Connected Partner Resource Properties. + */ +@Fluent +public final class ConnectedPartnerResourceProperties { + /* + * NewRelic account name + */ + @JsonProperty(value = "accountName") + private String accountName; + + /* + * NewRelic Account Id + */ + @JsonProperty(value = "accountId") + private String accountId; + + /* + * The azure resource Id of the deployment. + */ + @JsonProperty(value = "azureResourceId") + private String azureResourceId; + + /* + * The location of the deployment. + */ + @JsonProperty(value = "location") + private String location; + + /** + * Creates an instance of ConnectedPartnerResourceProperties class. + */ + public ConnectedPartnerResourceProperties() { + } + + /** + * Get the accountName property: NewRelic account name. + * + * @return the accountName value. + */ + public String accountName() { + return this.accountName; + } + + /** + * Set the accountName property: NewRelic account name. + * + * @param accountName the accountName value to set. + * @return the ConnectedPartnerResourceProperties object itself. + */ + public ConnectedPartnerResourceProperties withAccountName(String accountName) { + this.accountName = accountName; + return this; + } + + /** + * Get the accountId property: NewRelic Account Id. + * + * @return the accountId value. + */ + public String accountId() { + return this.accountId; + } + + /** + * Set the accountId property: NewRelic Account Id. + * + * @param accountId the accountId value to set. + * @return the ConnectedPartnerResourceProperties object itself. + */ + public ConnectedPartnerResourceProperties withAccountId(String accountId) { + this.accountId = accountId; + return this; + } + + /** + * Get the azureResourceId property: The azure resource Id of the deployment. + * + * @return the azureResourceId value. + */ + public String azureResourceId() { + return this.azureResourceId; + } + + /** + * Set the azureResourceId property: The azure resource Id of the deployment. + * + * @param azureResourceId the azureResourceId value to set. + * @return the ConnectedPartnerResourceProperties object itself. + */ + public ConnectedPartnerResourceProperties withAzureResourceId(String azureResourceId) { + this.azureResourceId = azureResourceId; + return this; + } + + /** + * Get the location property: The location of the deployment. + * + * @return the location value. + */ + public String location() { + return this.location; + } + + /** + * Set the location property: The location of the deployment. + * + * @param location the location value to set. + * @return the ConnectedPartnerResourceProperties object itself. + */ + public ConnectedPartnerResourceProperties withLocation(String location) { + this.location = location; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResources.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResources.java new file mode 100644 index 0000000000000..43e14ff91e511 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResources.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.util.Context; + +/** + * Resource collection API of ConnectedPartnerResources. + */ +public interface ConnectedPartnerResources { + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String monitorName); + + /** + * List of all active deployments that are associated with the marketplace subscription linked to the given monitor. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param body Email Id of the user. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of all active newrelic deployments as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String monitorName, String body, + Context context); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListFormat.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListFormat.java new file mode 100644 index 0000000000000..4ffc29ae0f15d --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListFormat.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; + +/** + * An immutable client-side representation of ConnectedPartnerResourcesListFormat. + */ +public interface ConnectedPartnerResourcesListFormat { + /** + * Gets the properties property: Connected Partner Resource Properties. + * + * @return the properties value. + */ + ConnectedPartnerResourceProperties properties(); + + /** + * Gets the inner + * com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner object. + * + * @return the inner object. + */ + ConnectedPartnerResourcesListFormatInner innerModel(); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListResponse.java new file mode 100644 index 0000000000000..f154a03ca3c7a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ConnectedPartnerResourcesListResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of all active newrelic deployments. + */ +@Fluent +public final class ConnectedPartnerResourcesListResponse { + /* + * Results of a list operation. + */ + @JsonProperty(value = "value") + private List value; + + /* + * Link to the next set of results, if any. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of ConnectedPartnerResourcesListResponse class. + */ + public ConnectedPartnerResourcesListResponse() { + } + + /** + * Get the value property: Results of a list operation. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Results of a list operation. + * + * @param value the value value to set. + * @return the ConnectedPartnerResourcesListResponse object itself. + */ + public ConnectedPartnerResourcesListResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to the next set of results, if any. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: Link to the next set of results, if any. + * + * @param nextLink the nextLink value to set. + * @return the ConnectedPartnerResourcesListResponse object itself. + */ + public ConnectedPartnerResourcesListResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/FilteringTag.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/FilteringTag.java index 9c14652281aa7..8866e5b561cd3 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/FilteringTag.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/FilteringTag.java @@ -31,13 +31,15 @@ public final class FilteringTag { @JsonProperty(value = "action") private TagAction action; - /** Creates an instance of FilteringTag class. */ + /** + * Creates an instance of FilteringTag class. + */ public FilteringTag() { } /** * Get the name property: The name (also known as the key) of the tag. - * + * * @return the name value. */ public String name() { @@ -46,7 +48,7 @@ public String name() { /** * Set the name property: The name (also known as the key) of the tag. - * + * * @param name the name value to set. * @return the FilteringTag object itself. */ @@ -57,7 +59,7 @@ public FilteringTag withName(String name) { /** * Get the value property: The value of the tag. - * + * * @return the value value. */ public String value() { @@ -66,7 +68,7 @@ public String value() { /** * Set the value property: The value of the tag. - * + * * @param value the value value to set. * @return the FilteringTag object itself. */ @@ -77,7 +79,7 @@ public FilteringTag withValue(String value) { /** * Get the action property: Valid actions for a filtering tag. Exclusion takes priority over inclusion. - * + * * @return the action value. */ public TagAction action() { @@ -86,7 +88,7 @@ public TagAction action() { /** * Set the action property: Valid actions for a filtering tag. Exclusion takes priority over inclusion. - * + * * @param action the action value to set. * @return the FilteringTag object itself. */ @@ -97,7 +99,7 @@ public FilteringTag withAction(TagAction action) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/HostsGetRequest.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/HostsGetRequest.java index 65832553ea045..5399bc21a498d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/HostsGetRequest.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/HostsGetRequest.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Request of a Hosts get Operation. */ +/** + * Request of a Hosts get Operation. + */ @Fluent public final class HostsGetRequest { /* @@ -24,13 +26,15 @@ public final class HostsGetRequest { @JsonProperty(value = "userEmail", required = true) private String userEmail; - /** Creates an instance of HostsGetRequest class. */ + /** + * Creates an instance of HostsGetRequest class. + */ public HostsGetRequest() { } /** * Get the vmIds property: VM resource IDs. - * + * * @return the vmIds value. */ public List vmIds() { @@ -39,7 +43,7 @@ public List vmIds() { /** * Set the vmIds property: VM resource IDs. - * + * * @param vmIds the vmIds value to set. * @return the HostsGetRequest object itself. */ @@ -50,7 +54,7 @@ public HostsGetRequest withVmIds(List vmIds) { /** * Get the userEmail property: User Email. - * + * * @return the userEmail value. */ public String userEmail() { @@ -59,7 +63,7 @@ public String userEmail() { /** * Set the userEmail property: User Email. - * + * * @param userEmail the userEmail value to set. * @return the HostsGetRequest object itself. */ @@ -70,14 +74,13 @@ public HostsGetRequest withUserEmail(String userEmail) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (userEmail() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property userEmail in model HostsGetRequest")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property userEmail in model HostsGetRequest")); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LiftrResourceCategories.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LiftrResourceCategories.java index c54ff40cd482f..5489faf533647 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LiftrResourceCategories.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LiftrResourceCategories.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Liftr Resource category. */ +/** + * Liftr Resource category. + */ public final class LiftrResourceCategories extends ExpandableStringEnum { - /** Static value Unknown for LiftrResourceCategories. */ + /** + * Static value Unknown for LiftrResourceCategories. + */ public static final LiftrResourceCategories UNKNOWN = fromString("Unknown"); - /** Static value MonitorLogs for LiftrResourceCategories. */ + /** + * Static value MonitorLogs for LiftrResourceCategories. + */ public static final LiftrResourceCategories MONITOR_LOGS = fromString("MonitorLogs"); /** * Creates a new instance of LiftrResourceCategories value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public LiftrResourceCategories() { /** * Creates or finds a LiftrResourceCategories from its string representation. - * + * * @param name a name to look for. * @return the corresponding LiftrResourceCategories. */ @@ -38,7 +44,7 @@ public static LiftrResourceCategories fromString(String name) { /** * Gets known LiftrResourceCategories values. - * + * * @return known LiftrResourceCategories values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResource.java new file mode 100644 index 0000000000000..6f39db758dc60 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResource.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; + +/** + * An immutable client-side representation of LinkedResource. + */ +public interface LinkedResource { + /** + * Gets the id property: The ARM id of the linked resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner object. + * + * @return the inner object. + */ + LinkedResourceInner innerModel(); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResourceListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResourceListResponse.java new file mode 100644 index 0000000000000..77d6d8b301efe --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LinkedResourceListResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Response of a list operation. + */ +@Fluent +public final class LinkedResourceListResponse { + /* + * Results of a list operation. + */ + @JsonProperty(value = "value") + private List value; + + /* + * Link to the next set of results, if any. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of LinkedResourceListResponse class. + */ + public LinkedResourceListResponse() { + } + + /** + * Get the value property: Results of a list operation. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Results of a list operation. + * + * @param value the value value to set. + * @return the LinkedResourceListResponse object itself. + */ + public LinkedResourceListResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Link to the next set of results, if any. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: Link to the next set of results, if any. + * + * @param nextLink the nextLink value to set. + * @return the LinkedResourceListResponse object itself. + */ + public LinkedResourceListResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LogRules.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LogRules.java index 0a0528d2ea1e1..aac1473d13a25 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LogRules.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/LogRules.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Set of rules for sending logs for the Monitor resource. */ +/** + * Set of rules for sending logs for the Monitor resource. + */ @Fluent public final class LogRules { /* @@ -38,13 +40,15 @@ public final class LogRules { @JsonProperty(value = "filteringTags") private List filteringTags; - /** Creates an instance of LogRules class. */ + /** + * Creates an instance of LogRules class. + */ public LogRules() { } /** * Get the sendAadLogs property: Flag specifying if AAD logs should be sent for the Monitor resource. - * + * * @return the sendAadLogs value. */ public SendAadLogsStatus sendAadLogs() { @@ -53,7 +57,7 @@ public SendAadLogsStatus sendAadLogs() { /** * Set the sendAadLogs property: Flag specifying if AAD logs should be sent for the Monitor resource. - * + * * @param sendAadLogs the sendAadLogs value to set. * @return the LogRules object itself. */ @@ -65,7 +69,7 @@ public LogRules withSendAadLogs(SendAadLogsStatus sendAadLogs) { /** * Get the sendSubscriptionLogs property: Flag specifying if subscription logs should be sent for the Monitor * resource. - * + * * @return the sendSubscriptionLogs value. */ public SendSubscriptionLogsStatus sendSubscriptionLogs() { @@ -75,7 +79,7 @@ public SendSubscriptionLogsStatus sendSubscriptionLogs() { /** * Set the sendSubscriptionLogs property: Flag specifying if subscription logs should be sent for the Monitor * resource. - * + * * @param sendSubscriptionLogs the sendSubscriptionLogs value to set. * @return the LogRules object itself. */ @@ -87,7 +91,7 @@ public LogRules withSendSubscriptionLogs(SendSubscriptionLogsStatus sendSubscrip /** * Get the sendActivityLogs property: Flag specifying if activity logs from Azure resources should be sent for the * Monitor resource. - * + * * @return the sendActivityLogs value. */ public SendActivityLogsStatus sendActivityLogs() { @@ -97,7 +101,7 @@ public SendActivityLogsStatus sendActivityLogs() { /** * Set the sendActivityLogs property: Flag specifying if activity logs from Azure resources should be sent for the * Monitor resource. - * + * * @param sendActivityLogs the sendActivityLogs value to set. * @return the LogRules object itself. */ @@ -108,10 +112,10 @@ public LogRules withSendActivityLogs(SendActivityLogsStatus sendActivityLogs) { /** * Get the filteringTags property: List of filtering tags to be used for capturing logs. This only takes effect if - * SendActivityLogs flag is enabled. If empty, all resources will be captured. If only Exclude action is specified, - * the rules will apply to the list of all available resources. If Include actions are specified, the rules will - * only include resources with the associated tags. - * + * SendActivityLogs flag is enabled. If empty, all resources will be captured. + * If only Exclude action is specified, the rules will apply to the list of all available resources. If Include + * actions are specified, the rules will only include resources with the associated tags. + * * @return the filteringTags value. */ public List filteringTags() { @@ -120,10 +124,10 @@ public List filteringTags() { /** * Set the filteringTags property: List of filtering tags to be used for capturing logs. This only takes effect if - * SendActivityLogs flag is enabled. If empty, all resources will be captured. If only Exclude action is specified, - * the rules will apply to the list of all available resources. If Include actions are specified, the rules will - * only include resources with the associated tags. - * + * SendActivityLogs flag is enabled. If empty, all resources will be captured. + * If only Exclude action is specified, the rules will apply to the list of all available resources. If Include + * actions are specified, the rules will only include resources with the associated tags. + * * @param filteringTags the filteringTags value to set. * @return the LogRules object itself. */ @@ -134,7 +138,7 @@ public LogRules withFilteringTags(List filteringTags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentity.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentity.java index b3167c58a3b23..12609b227f0fa 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentity.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentity.java @@ -11,7 +11,9 @@ import java.util.Map; import java.util.UUID; -/** Managed service identity (system assigned and/or user assigned identities). */ +/** + * Managed service identity (system assigned and/or user assigned identities). + */ @Fluent public final class ManagedServiceIdentity { /* @@ -37,21 +39,24 @@ public final class ManagedServiceIdentity { /* * The set of user assigned identities associated with the resource. The userAssignedIdentities dictionary keys * will be ARM resource ids in the form: - * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. + * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/ + * userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. */ @JsonProperty(value = "userAssignedIdentities") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map userAssignedIdentities; - /** Creates an instance of ManagedServiceIdentity class. */ + /** + * Creates an instance of ManagedServiceIdentity class. + */ public ManagedServiceIdentity() { } /** * Get the principalId property: The service principal ID of the system assigned identity. This property will only * be provided for a system assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -59,9 +64,9 @@ public UUID principalId() { } /** - * Get the tenantId property: The tenant ID of the system assigned identity. This property will only be provided for - * a system assigned identity. - * + * Get the tenantId property: The tenant ID of the system assigned identity. This property will only be provided + * for a system assigned identity. + * * @return the tenantId value. */ public UUID tenantId() { @@ -71,7 +76,7 @@ public UUID tenantId() { /** * Get the type property: Type of managed service identity (where both SystemAssigned and UserAssigned types are * allowed). - * + * * @return the type value. */ public ManagedServiceIdentityType type() { @@ -81,7 +86,7 @@ public ManagedServiceIdentityType type() { /** * Set the type property: Type of managed service identity (where both SystemAssigned and UserAssigned types are * allowed). - * + * * @param type the type value to set. * @return the ManagedServiceIdentity object itself. */ @@ -95,7 +100,7 @@ public ManagedServiceIdentity withType(ManagedServiceIdentityType type) { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @return the userAssignedIdentities value. */ public Map userAssignedIdentities() { @@ -107,7 +112,7 @@ public Map userAssignedIdentities() { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @param userAssignedIdentities the userAssignedIdentities value to set. * @return the ManagedServiceIdentity object itself. */ @@ -118,24 +123,20 @@ public ManagedServiceIdentity withUserAssignedIdentities(Map { - if (e != null) { - e.validate(); - } - }); + userAssignedIdentities().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentityType.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentityType.java index 02760709a0322..c0964b0a0531a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentityType.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ManagedServiceIdentityType.java @@ -8,24 +8,34 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Type of managed service identity (where both SystemAssigned and UserAssigned types are allowed). */ +/** + * Type of managed service identity (where both SystemAssigned and UserAssigned types are allowed). + */ public final class ManagedServiceIdentityType extends ExpandableStringEnum { - /** Static value None for ManagedServiceIdentityType. */ + /** + * Static value None for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType NONE = fromString("None"); - /** Static value SystemAssigned for ManagedServiceIdentityType. */ + /** + * Static value SystemAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType SYSTEM_ASSIGNED = fromString("SystemAssigned"); - /** Static value UserAssigned for ManagedServiceIdentityType. */ + /** + * Static value UserAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType USER_ASSIGNED = fromString("UserAssigned"); - /** Static value SystemAssigned, UserAssigned for ManagedServiceIdentityType. */ - public static final ManagedServiceIdentityType SYSTEM_ASSIGNED_USER_ASSIGNED = - fromString("SystemAssigned, UserAssigned"); + /** + * Static value SystemAssigned, UserAssigned for ManagedServiceIdentityType. + */ + public static final ManagedServiceIdentityType SYSTEM_ASSIGNED_USER_ASSIGNED + = fromString("SystemAssigned, UserAssigned"); /** * Creates a new instance of ManagedServiceIdentityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -34,7 +44,7 @@ public ManagedServiceIdentityType() { /** * Creates or finds a ManagedServiceIdentityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ManagedServiceIdentityType. */ @@ -45,7 +55,7 @@ public static ManagedServiceIdentityType fromString(String name) { /** * Gets known ManagedServiceIdentityType values. - * + * * @return known ManagedServiceIdentityType values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSaaSInfo.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSaaSInfo.java new file mode 100644 index 0000000000000..b391698ecb792 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSaaSInfo.java @@ -0,0 +1,160 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Marketplace SAAS Info of the resource. + */ +@Fluent +public final class MarketplaceSaaSInfo { + /* + * Marketplace Subscription Id. This is a GUID-formatted string. + */ + @JsonProperty(value = "marketplaceSubscriptionId") + private String marketplaceSubscriptionId; + + /* + * Marketplace Subscription Details: SAAS Name + */ + @JsonProperty(value = "marketplaceSubscriptionName") + private String marketplaceSubscriptionName; + + /* + * Marketplace Subscription Details: Resource URI + */ + @JsonProperty(value = "marketplaceResourceId") + private String marketplaceResourceId; + + /* + * Marketplace Subscription Details: SaaS Subscription Status + */ + @JsonProperty(value = "marketplaceStatus") + private String marketplaceStatus; + + /* + * The Azure Subscription ID to which the Marketplace Subscription belongs and gets billed into. + */ + @JsonProperty(value = "billedAzureSubscriptionId") + private String billedAzureSubscriptionId; + + /** + * Creates an instance of MarketplaceSaaSInfo class. + */ + public MarketplaceSaaSInfo() { + } + + /** + * Get the marketplaceSubscriptionId property: Marketplace Subscription Id. This is a GUID-formatted string. + * + * @return the marketplaceSubscriptionId value. + */ + public String marketplaceSubscriptionId() { + return this.marketplaceSubscriptionId; + } + + /** + * Set the marketplaceSubscriptionId property: Marketplace Subscription Id. This is a GUID-formatted string. + * + * @param marketplaceSubscriptionId the marketplaceSubscriptionId value to set. + * @return the MarketplaceSaaSInfo object itself. + */ + public MarketplaceSaaSInfo withMarketplaceSubscriptionId(String marketplaceSubscriptionId) { + this.marketplaceSubscriptionId = marketplaceSubscriptionId; + return this; + } + + /** + * Get the marketplaceSubscriptionName property: Marketplace Subscription Details: SAAS Name. + * + * @return the marketplaceSubscriptionName value. + */ + public String marketplaceSubscriptionName() { + return this.marketplaceSubscriptionName; + } + + /** + * Set the marketplaceSubscriptionName property: Marketplace Subscription Details: SAAS Name. + * + * @param marketplaceSubscriptionName the marketplaceSubscriptionName value to set. + * @return the MarketplaceSaaSInfo object itself. + */ + public MarketplaceSaaSInfo withMarketplaceSubscriptionName(String marketplaceSubscriptionName) { + this.marketplaceSubscriptionName = marketplaceSubscriptionName; + return this; + } + + /** + * Get the marketplaceResourceId property: Marketplace Subscription Details: Resource URI. + * + * @return the marketplaceResourceId value. + */ + public String marketplaceResourceId() { + return this.marketplaceResourceId; + } + + /** + * Set the marketplaceResourceId property: Marketplace Subscription Details: Resource URI. + * + * @param marketplaceResourceId the marketplaceResourceId value to set. + * @return the MarketplaceSaaSInfo object itself. + */ + public MarketplaceSaaSInfo withMarketplaceResourceId(String marketplaceResourceId) { + this.marketplaceResourceId = marketplaceResourceId; + return this; + } + + /** + * Get the marketplaceStatus property: Marketplace Subscription Details: SaaS Subscription Status. + * + * @return the marketplaceStatus value. + */ + public String marketplaceStatus() { + return this.marketplaceStatus; + } + + /** + * Set the marketplaceStatus property: Marketplace Subscription Details: SaaS Subscription Status. + * + * @param marketplaceStatus the marketplaceStatus value to set. + * @return the MarketplaceSaaSInfo object itself. + */ + public MarketplaceSaaSInfo withMarketplaceStatus(String marketplaceStatus) { + this.marketplaceStatus = marketplaceStatus; + return this; + } + + /** + * Get the billedAzureSubscriptionId property: The Azure Subscription ID to which the Marketplace Subscription + * belongs and gets billed into. + * + * @return the billedAzureSubscriptionId value. + */ + public String billedAzureSubscriptionId() { + return this.billedAzureSubscriptionId; + } + + /** + * Set the billedAzureSubscriptionId property: The Azure Subscription ID to which the Marketplace Subscription + * belongs and gets billed into. + * + * @param billedAzureSubscriptionId the billedAzureSubscriptionId value to set. + * @return the MarketplaceSaaSInfo object itself. + */ + public MarketplaceSaaSInfo withBilledAzureSubscriptionId(String billedAzureSubscriptionId) { + this.billedAzureSubscriptionId = billedAzureSubscriptionId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSubscriptionStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSubscriptionStatus.java index 4582223ce86b0..6c0036fd3d574 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSubscriptionStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MarketplaceSubscriptionStatus.java @@ -13,15 +13,19 @@ * will go in Suspended state. */ public final class MarketplaceSubscriptionStatus extends ExpandableStringEnum { - /** Static value Active for MarketplaceSubscriptionStatus. */ + /** + * Static value Active for MarketplaceSubscriptionStatus. + */ public static final MarketplaceSubscriptionStatus ACTIVE = fromString("Active"); - /** Static value Suspended for MarketplaceSubscriptionStatus. */ + /** + * Static value Suspended for MarketplaceSubscriptionStatus. + */ public static final MarketplaceSubscriptionStatus SUSPENDED = fromString("Suspended"); /** * Creates a new instance of MarketplaceSubscriptionStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +34,7 @@ public MarketplaceSubscriptionStatus() { /** * Creates or finds a MarketplaceSubscriptionStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding MarketplaceSubscriptionStatus. */ @@ -41,7 +45,7 @@ public static MarketplaceSubscriptionStatus fromString(String name) { /** * Gets known MarketplaceSubscriptionStatus values. - * + * * @return known MarketplaceSubscriptionStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricRules.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricRules.java index 0c465db5300af..78b2d63945578 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricRules.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricRules.java @@ -7,32 +7,34 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import java.util.List; -/** An immutable client-side representation of MetricRules. */ +/** + * An immutable client-side representation of MetricRules. + */ public interface MetricRules { /** * Gets the sendMetrics property: Flag specifying if metrics should be sent for the Monitor resource. - * + * * @return the sendMetrics value. */ SendMetricsStatus sendMetrics(); /** * Gets the filteringTags property: List of filtering tags to be used for capturing metrics. - * + * * @return the filteringTags value. */ List filteringTags(); /** * Gets the userEmail property: User Email. - * + * * @return the userEmail value. */ String userEmail(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner object. - * + * * @return the inner object. */ MetricRulesInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsRequest.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsRequest.java index 81635202807bf..4b9c056fb6579 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsRequest.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsRequest.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Request of get metrics Operation. */ +/** + * Request of get metrics Operation. + */ @Fluent public final class MetricsRequest { /* @@ -17,13 +19,15 @@ public final class MetricsRequest { @JsonProperty(value = "userEmail", required = true) private String userEmail; - /** Creates an instance of MetricsRequest class. */ + /** + * Creates an instance of MetricsRequest class. + */ public MetricsRequest() { } /** * Get the userEmail property: User Email. - * + * * @return the userEmail value. */ public String userEmail() { @@ -32,7 +36,7 @@ public String userEmail() { /** * Set the userEmail property: User Email. - * + * * @param userEmail the userEmail value to set. * @return the MetricsRequest object itself. */ @@ -43,14 +47,13 @@ public MetricsRequest withUserEmail(String userEmail) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (userEmail() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property userEmail in model MetricsRequest")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property userEmail in model MetricsRequest")); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusRequest.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusRequest.java index 241f3542c93bc..e0855e61c0789 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusRequest.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusRequest.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Request of get metrics status Operation. */ +/** + * Request of get metrics status Operation. + */ @Fluent public final class MetricsStatusRequest { /* @@ -24,13 +26,15 @@ public final class MetricsStatusRequest { @JsonProperty(value = "userEmail", required = true) private String userEmail; - /** Creates an instance of MetricsStatusRequest class. */ + /** + * Creates an instance of MetricsStatusRequest class. + */ public MetricsStatusRequest() { } /** * Get the azureResourceIds property: Azure resource IDs. - * + * * @return the azureResourceIds value. */ public List azureResourceIds() { @@ -39,7 +43,7 @@ public List azureResourceIds() { /** * Set the azureResourceIds property: Azure resource IDs. - * + * * @param azureResourceIds the azureResourceIds value to set. * @return the MetricsStatusRequest object itself. */ @@ -50,7 +54,7 @@ public MetricsStatusRequest withAzureResourceIds(List azureResourceIds) /** * Get the userEmail property: User Email. - * + * * @return the userEmail value. */ public String userEmail() { @@ -59,7 +63,7 @@ public String userEmail() { /** * Set the userEmail property: User Email. - * + * * @param userEmail the userEmail value to set. * @return the MetricsStatusRequest object itself. */ @@ -70,14 +74,13 @@ public MetricsStatusRequest withUserEmail(String userEmail) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (userEmail() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property userEmail in model MetricsStatusRequest")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property userEmail in model MetricsStatusRequest")); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusResponse.java index d23ba424f7b83..e0f1fe8a18d8c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MetricsStatusResponse.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricsStatusResponseInner; import java.util.List; -/** An immutable client-side representation of MetricsStatusResponse. */ +/** + * An immutable client-side representation of MetricsStatusResponse. + */ public interface MetricsStatusResponse { /** * Gets the azureResourceIds property: Azure resource IDs. - * + * * @return the azureResourceIds value. */ List azureResourceIds(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.MetricsStatusResponseInner object. - * + * * @return the inner object. */ MetricsStatusResponseInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResource.java index c442d07991839..eb22e9411aac2 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResource.java @@ -6,18 +6,20 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredResourceInner; -/** An immutable client-side representation of MonitoredResource. */ +/** + * An immutable client-side representation of MonitoredResource. + */ public interface MonitoredResource { /** * Gets the id property: The ARM id of the resource. - * + * * @return the id value. */ String id(); /** * Gets the sendingMetrics property: Flag indicating if resource is sending metrics to NewRelic. - * + * * @return the sendingMetrics value. */ SendingMetricsStatus sendingMetrics(); @@ -25,28 +27,28 @@ public interface MonitoredResource { /** * Gets the reasonForMetricsStatus property: Reason for why the resource is sending metrics (or why it is not * sending). - * + * * @return the reasonForMetricsStatus value. */ String reasonForMetricsStatus(); /** * Gets the sendingLogs property: Flag indicating if resource is sending logs to NewRelic. - * + * * @return the sendingLogs value. */ SendingLogsStatus sendingLogs(); /** * Gets the reasonForLogsStatus property: Reason for why the resource is sending logs (or why it is not sending). - * + * * @return the reasonForLogsStatus value. */ String reasonForLogsStatus(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredResourceInner object. - * + * * @return the inner object. */ MonitoredResourceInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResourceListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResourceListResponse.java index 2f44447c2b0b7..f11afb5054c51 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResourceListResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredResourceListResponse.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of all the resources being monitored by NewRelic monitor resource. */ +/** + * List of all the resources being monitored by NewRelic monitor resource. + */ @Fluent public final class MonitoredResourceListResponse { /* @@ -25,13 +27,15 @@ public final class MonitoredResourceListResponse { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of MonitoredResourceListResponse class. */ + /** + * Creates an instance of MonitoredResourceListResponse class. + */ public MonitoredResourceListResponse() { } /** * Get the value property: The MonitoredResource items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The MonitoredResource items on this page. - * + * * @param value the value value to set. * @return the MonitoredResourceListResponse object itself. */ @@ -51,7 +55,7 @@ public MonitoredResourceListResponse withValue(List valu /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the MonitoredResourceListResponse object itself. */ @@ -71,15 +75,13 @@ public MonitoredResourceListResponse withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model MonitoredResourceListResponse")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model MonitoredResourceListResponse")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscription.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscription.java new file mode 100644 index 0000000000000..3d96a8f57e09a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscription.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; + +/** + * An immutable client-side representation of MonitoredSubscription. + */ +public interface MonitoredSubscription { + /** + * Gets the subscriptionId property: The subscriptionId to be monitored. + * + * @return the subscriptionId value. + */ + String subscriptionId(); + + /** + * Gets the status property: The state of monitoring. + * + * @return the status value. + */ + Status status(); + + /** + * Gets the error property: The reason of not monitoring the subscription. + * + * @return the error value. + */ + String error(); + + /** + * Gets the tagRules property: The resource-specific properties for this resource. + * + * @return the tagRules value. + */ + MonitoringTagRulesProperties tagRules(); + + /** + * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner object. + * + * @return the inner object. + */ + MonitoredSubscriptionInner innerModel(); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionProperties.java new file mode 100644 index 0000000000000..45a7565b4328d --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionProperties.java @@ -0,0 +1,187 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.util.Context; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; + +/** + * An immutable client-side representation of MonitoredSubscriptionProperties. + */ +public interface MonitoredSubscriptionProperties { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: The request to update subscriptions needed to be monitored by the NewRelic monitor + * resource. + * + * @return the properties value. + */ + SubscriptionList properties(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner + * object. + * + * @return the inner object. + */ + MonitoredSubscriptionPropertiesInner innerModel(); + + /** + * The entirety of the MonitoredSubscriptionProperties definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The MonitoredSubscriptionProperties definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the MonitoredSubscriptionProperties definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the MonitoredSubscriptionProperties definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, monitorName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @return the next definition stage. + */ + WithCreate withExistingMonitor(String resourceGroupName, String monitorName); + } + + /** + * The stage of the MonitoredSubscriptionProperties definition which contains all the minimum required + * properties for the resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithProperties { + /** + * Executes the create request. + * + * @return the created resource. + */ + MonitoredSubscriptionProperties create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + MonitoredSubscriptionProperties create(Context context); + } + + /** + * The stage of the MonitoredSubscriptionProperties definition allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: The request to update subscriptions needed to be monitored by the + * NewRelic monitor resource.. + * + * @param properties The request to update subscriptions needed to be monitored by the NewRelic monitor + * resource. + * @return the next definition stage. + */ + WithCreate withProperties(SubscriptionList properties); + } + } + + /** + * Begins update for the MonitoredSubscriptionProperties resource. + * + * @return the stage of resource update. + */ + MonitoredSubscriptionProperties.Update update(); + + /** + * The template for MonitoredSubscriptionProperties update. + */ + interface Update extends UpdateStages.WithProperties { + /** + * Executes the update request. + * + * @return the updated resource. + */ + MonitoredSubscriptionProperties apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + MonitoredSubscriptionProperties apply(Context context); + } + + /** + * The MonitoredSubscriptionProperties update stages. + */ + interface UpdateStages { + /** + * The stage of the MonitoredSubscriptionProperties update allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: The request to update subscriptions needed to be monitored by the + * NewRelic monitor resource.. + * + * @param properties The request to update subscriptions needed to be monitored by the NewRelic monitor + * resource. + * @return the next definition stage. + */ + Update withProperties(SubscriptionList properties); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + MonitoredSubscriptionProperties refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + MonitoredSubscriptionProperties refresh(Context context); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionPropertiesList.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionPropertiesList.java new file mode 100644 index 0000000000000..b6aba0334b58f --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptionPropertiesList.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The MonitoredSubscriptionPropertiesList model. + */ +@Fluent +public final class MonitoredSubscriptionPropertiesList { + /* + * The value property. + */ + @JsonProperty(value = "value") + private List value; + + /* + * The link to the next page of items + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of MonitoredSubscriptionPropertiesList class. + */ + public MonitoredSubscriptionPropertiesList() { + } + + /** + * Get the value property: The value property. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: The value property. + * + * @param value the value value to set. + * @return the MonitoredSubscriptionPropertiesList object itself. + */ + public MonitoredSubscriptionPropertiesList withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: The link to the next page of items. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: The link to the next page of items. + * + * @param nextLink the nextLink value to set. + * @return the MonitoredSubscriptionPropertiesList object itself. + */ + public MonitoredSubscriptionPropertiesList withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptions.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptions.java new file mode 100644 index 0000000000000..ca59ee5b8b01a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoredSubscriptions.java @@ -0,0 +1,148 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of MonitoredSubscriptions. + */ +public interface MonitoredSubscriptions { + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String monitorName); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String monitorName, Context context); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String monitorName, + ConfigurationName configurationName, Context context); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ + MonitoredSubscriptionProperties get(String resourceGroupName, String monitorName, + ConfigurationName configurationName); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param configurationName The configuration name. Only 'default' value is supported. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String monitorName, ConfigurationName configurationName, Context context); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response}. + */ + MonitoredSubscriptionProperties getById(String id); + + /** + * List the subscriptions currently being monitored by the NewRelic monitor resource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the request to update subscriptions needed to be monitored by the NewRelic monitor resource along with + * {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteById(String id); + + /** + * Updates the subscriptions that are being monitored by the NewRelic monitor resource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new MonitoredSubscriptionProperties resource. + * + * @param name resource name. + * @return the first stage of the new MonitoredSubscriptionProperties definition. + */ + MonitoredSubscriptionProperties.DefinitionStages.Blank define(ConfigurationName name); +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringStatus.java index 84043394b5be7..fa8ae17da2ddb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Flag specifying if the resource monitoring is enabled or disabled. */ +/** + * Flag specifying if the resource monitoring is enabled or disabled. + */ public final class MonitoringStatus extends ExpandableStringEnum { - /** Static value Enabled for MonitoringStatus. */ + /** + * Static value Enabled for MonitoringStatus. + */ public static final MonitoringStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for MonitoringStatus. */ + /** + * Static value Disabled for MonitoringStatus. + */ public static final MonitoringStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of MonitoringStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public MonitoringStatus() { /** * Creates or finds a MonitoringStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding MonitoringStatus. */ @@ -38,7 +44,7 @@ public static MonitoringStatus fromString(String name) { /** * Gets known MonitoringStatus values. - * + * * @return known MonitoringStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringTagRulesProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringTagRulesProperties.java index 7ce41d44e85c7..594d1455f396e 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringTagRulesProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitoringTagRulesProperties.java @@ -6,25 +6,27 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; -/** An immutable client-side representation of MonitoringTagRulesProperties. */ +/** + * An immutable client-side representation of MonitoringTagRulesProperties. + */ public interface MonitoringTagRulesProperties { /** * Gets the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ LogRules logRules(); /** * Gets the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ MetricRules metricRules(); @@ -32,7 +34,7 @@ public interface MonitoringTagRulesProperties { /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner * object. - * + * * @return the inner object. */ MonitoringTagRulesPropertiesInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Monitors.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Monitors.java index 003c100c04312..726947bd0e70b 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Monitors.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Monitors.java @@ -8,58 +8,60 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Monitors. */ +/** + * Resource collection API of Monitors. + */ public interface Monitors { /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * List NewRelicMonitorResource resources by subscription ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ PagedIterable listByResourceGroup(String resourceGroupName); /** * List NewRelicMonitorResource resources by resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a NewRelicMonitorResource list operation as paginated response with {@link - * PagedIterable}. + * @return the response of a NewRelicMonitorResource list operation as paginated response with + * {@link PagedIterable}. */ PagedIterable listByResourceGroup(String resourceGroupName, Context context); /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -68,12 +70,12 @@ public interface Monitors { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a NewRelicMonitorResource along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String monitorName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String monitorName, + Context context); /** * Get a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -85,7 +87,7 @@ Response getByResourceGroupWithResponse( /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -97,7 +99,7 @@ Response getByResourceGroupWithResponse( /** * Delete a NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param userEmail User Email. * @param monitorName Name of the Monitors resource. @@ -110,7 +112,7 @@ Response getByResourceGroupWithResponse( /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -120,12 +122,12 @@ Response getByResourceGroupWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metric rules along with {@link Response}. */ - Response getMetricRulesWithResponse( - String resourceGroupName, String monitorName, MetricsRequest request, Context context); + Response getMetricRulesWithResponse(String resourceGroupName, String monitorName, + MetricsRequest request, Context context); /** * Get metric rules. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -138,7 +140,7 @@ Response getMetricRulesWithResponse( /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -148,12 +150,12 @@ Response getMetricRulesWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metric status along with {@link Response}. */ - Response getMetricStatusWithResponse( - String resourceGroupName, String monitorName, MetricsStatusRequest request, Context context); + Response getMetricStatusWithResponse(String resourceGroupName, String monitorName, + MetricsStatusRequest request, Context context); /** * Get metric status. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the get metrics status request. @@ -166,7 +168,7 @@ Response getMetricStatusWithResponse( /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -175,12 +177,12 @@ Response getMetricStatusWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list app services Operation as paginated response with {@link PagedIterable}. */ - PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request); + PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request); /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the app services get request. @@ -190,12 +192,12 @@ PagedIterable listAppServices( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list app services Operation as paginated response with {@link PagedIterable}. */ - PagedIterable listAppServices( - String resourceGroupName, String monitorName, AppServicesGetRequest request, Context context); + PagedIterable listAppServices(String resourceGroupName, String monitorName, + AppServicesGetRequest request, Context context); /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -205,12 +207,12 @@ PagedIterable listAppServices( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a Monitor Resource by NewRelic. */ - Response switchBillingWithResponse( - String resourceGroupName, String monitorName, SwitchBillingRequest request, Context context); + Response switchBillingWithResponse(String resourceGroupName, String monitorName, + SwitchBillingRequest request, Context context); /** * Switches the billing for NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the switch billing request. @@ -223,7 +225,7 @@ Response switchBillingWithResponse( /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -236,7 +238,7 @@ Response switchBillingWithResponse( /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param request The details of the Hosts get request. @@ -246,40 +248,65 @@ Response switchBillingWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of a list VM Host Operation as paginated response with {@link PagedIterable}. */ - PagedIterable listHosts( - String resourceGroupName, String monitorName, HostsGetRequest request, Context context); + PagedIterable listHosts(String resourceGroupName, String monitorName, HostsGetRequest request, + Context context); /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ PagedIterable listMonitoredResources(String resourceGroupName, String monitorName); /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ - PagedIterable listMonitoredResources( - String resourceGroupName, String monitorName, Context context); + PagedIterable listMonitoredResources(String resourceGroupName, String monitorName, + Context context); + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + PagedIterable listLinkedResources(String resourceGroupName, String monitorName); + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param monitorName Name of the Monitors resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + PagedIterable listLinkedResources(String resourceGroupName, String monitorName, Context context); /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -288,12 +315,12 @@ PagedIterable listMonitoredResources( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of payload to be passed while installing VM agent along with {@link Response}. */ - Response vmHostPayloadWithResponse( - String resourceGroupName, String monitorName, Context context); + Response vmHostPayloadWithResponse(String resourceGroupName, String monitorName, + Context context); /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -305,7 +332,7 @@ Response vmHostPayloadWithResponse( /** * Get a NewRelicMonitorResource. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -316,7 +343,7 @@ Response vmHostPayloadWithResponse( /** * Get a NewRelicMonitorResource. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -328,7 +355,7 @@ Response vmHostPayloadWithResponse( /** * Begins definition for a new NewRelicMonitorResource resource. - * + * * @param name resource name. * @return the first stage of the new NewRelicMonitorResource definition. */ diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingHeaders.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingHeaders.java index 10dcd94faf923..7f8019b708048 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingHeaders.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingHeaders.java @@ -9,7 +9,9 @@ import com.azure.core.http.HttpHeaders; import com.fasterxml.jackson.annotation.JsonProperty; -/** The MonitorsSwitchBillingHeaders model. */ +/** + * The MonitorsSwitchBillingHeaders model. + */ @Fluent public final class MonitorsSwitchBillingHeaders { /* @@ -21,7 +23,7 @@ public final class MonitorsSwitchBillingHeaders { // HttpHeaders containing the raw property values. /** * Creates an instance of MonitorsSwitchBillingHeaders class. - * + * * @param rawHeaders The raw HttpHeaders that will be used to create the property values. */ public MonitorsSwitchBillingHeaders(HttpHeaders rawHeaders) { @@ -33,7 +35,7 @@ public MonitorsSwitchBillingHeaders(HttpHeaders rawHeaders) { /** * Get the retryAfter property: The Retry-After property. - * + * * @return the retryAfter value. */ public Integer retryAfter() { @@ -42,7 +44,7 @@ public Integer retryAfter() { /** * Set the retryAfter property: The Retry-After property. - * + * * @param retryAfter the retryAfter value to set. * @return the MonitorsSwitchBillingHeaders object itself. */ @@ -53,7 +55,7 @@ public MonitorsSwitchBillingHeaders withRetryAfter(Integer retryAfter) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingResponse.java index cf0db79e8f5f7..90308e08d62ff 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/MonitorsSwitchBillingResponse.java @@ -9,30 +9,28 @@ import com.azure.core.http.rest.ResponseBase; import com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceInner; -/** Contains all response data for the switchBilling operation. */ +/** + * Contains all response data for the switchBilling operation. + */ public final class MonitorsSwitchBillingResponse extends ResponseBase { /** * Creates an instance of MonitorsSwitchBillingResponse. - * + * * @param request the request which resulted in this MonitorsSwitchBillingResponse. * @param statusCode the status code of the HTTP response. * @param rawHeaders the raw headers of the HTTP response. * @param value the deserialized value of the HTTP response. * @param headers the deserialized headers of the HTTP response. */ - public MonitorsSwitchBillingResponse( - HttpRequest request, - int statusCode, - HttpHeaders rawHeaders, - NewRelicMonitorResourceInner value, - MonitorsSwitchBillingHeaders headers) { + public MonitorsSwitchBillingResponse(HttpRequest request, int statusCode, HttpHeaders rawHeaders, + NewRelicMonitorResourceInner value, MonitorsSwitchBillingHeaders headers) { super(request, statusCode, rawHeaders, value, headers); } /** * Gets the deserialized response body. - * + * * @return the deserialized response body. */ @Override diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicAccountProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicAccountProperties.java index 9957e06e3b0d8..d8fcad01865bf 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicAccountProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicAccountProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Properties of the NewRelic account. */ +/** + * Properties of the NewRelic account. + */ @Fluent public final class NewRelicAccountProperties { /* @@ -34,13 +36,15 @@ public final class NewRelicAccountProperties { @JsonProperty(value = "singleSignOnProperties") private NewRelicSingleSignOnProperties singleSignOnProperties; - /** Creates an instance of NewRelicAccountProperties class. */ + /** + * Creates an instance of NewRelicAccountProperties class. + */ public NewRelicAccountProperties() { } /** * Get the userId property: User id. - * + * * @return the userId value. */ public String userId() { @@ -49,7 +53,7 @@ public String userId() { /** * Set the userId property: User id. - * + * * @param userId the userId value to set. * @return the NewRelicAccountProperties object itself. */ @@ -60,7 +64,7 @@ public NewRelicAccountProperties withUserId(String userId) { /** * Get the accountInfo property: NewRelic Account Information. - * + * * @return the accountInfo value. */ public AccountInfo accountInfo() { @@ -69,7 +73,7 @@ public AccountInfo accountInfo() { /** * Set the accountInfo property: NewRelic Account Information. - * + * * @param accountInfo the accountInfo value to set. * @return the NewRelicAccountProperties object itself. */ @@ -80,7 +84,7 @@ public NewRelicAccountProperties withAccountInfo(AccountInfo accountInfo) { /** * Get the organizationInfo property: NewRelic Organization Information. - * + * * @return the organizationInfo value. */ public OrganizationInfo organizationInfo() { @@ -89,7 +93,7 @@ public OrganizationInfo organizationInfo() { /** * Set the organizationInfo property: NewRelic Organization Information. - * + * * @param organizationInfo the organizationInfo value to set. * @return the NewRelicAccountProperties object itself. */ @@ -100,7 +104,7 @@ public NewRelicAccountProperties withOrganizationInfo(OrganizationInfo organizat /** * Get the singleSignOnProperties property: date when plan was applied. - * + * * @return the singleSignOnProperties value. */ public NewRelicSingleSignOnProperties singleSignOnProperties() { @@ -109,7 +113,7 @@ public NewRelicSingleSignOnProperties singleSignOnProperties() { /** * Set the singleSignOnProperties property: date when plan was applied. - * + * * @param singleSignOnProperties the singleSignOnProperties value to set. * @return the NewRelicAccountProperties object itself. */ @@ -120,7 +124,7 @@ public NewRelicAccountProperties withSingleSignOnProperties(NewRelicSingleSignOn /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResource.java index 3359f02510db4..2ee32c6ed8c67 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResource.java @@ -12,179 +12,202 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceInner; import java.util.Map; -/** An immutable client-side representation of NewRelicMonitorResource. */ +/** + * An immutable client-side representation of NewRelicMonitorResource. + */ public interface NewRelicMonitorResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the identity property: The managed service identities assigned to this resource. - * + * * @return the identity value. */ ManagedServiceIdentity identity(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the monitoringStatus property: MonitoringStatus of the resource. - * + * * @return the monitoringStatus value. */ MonitoringStatus monitoringStatus(); /** * Gets the marketplaceSubscriptionStatus property: NewRelic Organization properties of the resource. - * + * * @return the marketplaceSubscriptionStatus value. */ MarketplaceSubscriptionStatus marketplaceSubscriptionStatus(); /** * Gets the marketplaceSubscriptionId property: Marketplace Subscription Id. - * + * * @return the marketplaceSubscriptionId value. */ String marketplaceSubscriptionId(); /** * Gets the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @return the newRelicAccountProperties value. */ NewRelicAccountProperties newRelicAccountProperties(); /** * Gets the userInfo property: User Info. - * + * * @return the userInfo value. */ UserInfo userInfo(); /** * Gets the planData property: Plan details. - * + * * @return the planData value. */ PlanData planData(); /** * Gets the liftrResourceCategory property: Liftr resource category. - * + * * @return the liftrResourceCategory value. */ LiftrResourceCategories liftrResourceCategory(); /** * Gets the liftrResourcePreference property: Liftr resource preference. The priority of the resource. - * + * * @return the liftrResourcePreference value. */ Integer liftrResourcePreference(); /** * Gets the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ OrgCreationSource orgCreationSource(); /** * Gets the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ AccountCreationSource accountCreationSource(); + /** + * Gets the subscriptionState property: State of the Azure Subscription containing the monitor resource. + * + * @return the subscriptionState value. + */ + String subscriptionState(); + + /** + * Gets the saaSAzureSubscriptionStatus property: Status of Azure Subscription where Marketplace SaaS is located. + * + * @return the saaSAzureSubscriptionStatus value. + */ + String saaSAzureSubscriptionStatus(); + /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceInner object. - * + * * @return the inner object. */ NewRelicMonitorResourceInner innerModel(); - /** The entirety of the NewRelicMonitorResource definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the NewRelicMonitorResource definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The NewRelicMonitorResource definition stages. */ + + /** + * The NewRelicMonitorResource definition stages. + */ interface DefinitionStages { - /** The first stage of the NewRelicMonitorResource definition. */ + /** + * The first stage of the NewRelicMonitorResource definition. + */ interface Blank extends WithLocation { } - /** The stage of the NewRelicMonitorResource definition allowing to specify location. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -192,234 +215,306 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the NewRelicMonitorResource definition allowing to specify parent resource. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ WithCreate withExistingResourceGroup(String resourceGroupName); } + /** * The stage of the NewRelicMonitorResource definition which contains all the minimum required properties for * the resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithIdentity, - DefinitionStages.WithNewRelicAccountProperties, - DefinitionStages.WithUserInfo, - DefinitionStages.WithPlanData, - DefinitionStages.WithOrgCreationSource, - DefinitionStages.WithAccountCreationSource { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithNewRelicAccountProperties, DefinitionStages.WithUserInfo, + DefinitionStages.WithPlanData, DefinitionStages.WithOrgCreationSource, + DefinitionStages.WithAccountCreationSource, DefinitionStages.WithSubscriptionState, + DefinitionStages.WithSaaSAzureSubscriptionStatus { /** * Executes the create request. - * + * * @return the created resource. */ NewRelicMonitorResource create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ NewRelicMonitorResource create(Context context); } - /** The stage of the NewRelicMonitorResource definition allowing to specify tags. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the NewRelicMonitorResource definition allowing to specify identity. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The managed service identities assigned to this resource.. - * + * * @param identity The managed service identities assigned to this resource. * @return the next definition stage. */ WithCreate withIdentity(ManagedServiceIdentity identity); } - /** The stage of the NewRelicMonitorResource definition allowing to specify newRelicAccountProperties. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify newRelicAccountProperties. + */ interface WithNewRelicAccountProperties { /** * Specifies the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @param newRelicAccountProperties MarketplaceSubscriptionStatus of the resource. * @return the next definition stage. */ WithCreate withNewRelicAccountProperties(NewRelicAccountProperties newRelicAccountProperties); } - /** The stage of the NewRelicMonitorResource definition allowing to specify userInfo. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify userInfo. + */ interface WithUserInfo { /** * Specifies the userInfo property: User Info. - * + * * @param userInfo User Info. * @return the next definition stage. */ WithCreate withUserInfo(UserInfo userInfo); } - /** The stage of the NewRelicMonitorResource definition allowing to specify planData. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify planData. + */ interface WithPlanData { /** * Specifies the planData property: Plan details. - * + * * @param planData Plan details. * @return the next definition stage. */ WithCreate withPlanData(PlanData planData); } - /** The stage of the NewRelicMonitorResource definition allowing to specify orgCreationSource. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify orgCreationSource. + */ interface WithOrgCreationSource { /** * Specifies the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource Source of org creation. * @return the next definition stage. */ WithCreate withOrgCreationSource(OrgCreationSource orgCreationSource); } - /** The stage of the NewRelicMonitorResource definition allowing to specify accountCreationSource. */ + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify accountCreationSource. + */ interface WithAccountCreationSource { /** * Specifies the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource Source of account creation. * @return the next definition stage. */ WithCreate withAccountCreationSource(AccountCreationSource accountCreationSource); } + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify subscriptionState. + */ + interface WithSubscriptionState { + /** + * Specifies the subscriptionState property: State of the Azure Subscription containing the monitor + * resource. + * + * @param subscriptionState State of the Azure Subscription containing the monitor resource. + * @return the next definition stage. + */ + WithCreate withSubscriptionState(String subscriptionState); + } + + /** + * The stage of the NewRelicMonitorResource definition allowing to specify saaSAzureSubscriptionStatus. + */ + interface WithSaaSAzureSubscriptionStatus { + /** + * Specifies the saaSAzureSubscriptionStatus property: Status of Azure Subscription where Marketplace SaaS + * is located.. + * + * @param saaSAzureSubscriptionStatus Status of Azure Subscription where Marketplace SaaS is located. + * @return the next definition stage. + */ + WithCreate withSaaSAzureSubscriptionStatus(String saaSAzureSubscriptionStatus); + } } + /** * Begins update for the NewRelicMonitorResource resource. - * + * * @return the stage of resource update. */ NewRelicMonitorResource.Update update(); - /** The template for NewRelicMonitorResource update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithIdentity, - UpdateStages.WithNewRelicAccountProperties, - UpdateStages.WithUserInfo, - UpdateStages.WithPlanData, - UpdateStages.WithOrgCreationSource, - UpdateStages.WithAccountCreationSource { + /** + * The template for NewRelicMonitorResource update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithIdentity, + UpdateStages.WithNewRelicAccountProperties, UpdateStages.WithUserInfo, UpdateStages.WithPlanData, + UpdateStages.WithOrgCreationSource, UpdateStages.WithAccountCreationSource { /** * Executes the update request. - * + * * @return the updated resource. */ NewRelicMonitorResource apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ NewRelicMonitorResource apply(Context context); } - /** The NewRelicMonitorResource update stages. */ + + /** + * The NewRelicMonitorResource update stages. + */ interface UpdateStages { - /** The stage of the NewRelicMonitorResource update allowing to specify tags. */ + /** + * The stage of the NewRelicMonitorResource update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the NewRelicMonitorResource update allowing to specify identity. */ + + /** + * The stage of the NewRelicMonitorResource update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The managed service identities assigned to this resource.. - * + * * @param identity The managed service identities assigned to this resource. * @return the next definition stage. */ Update withIdentity(ManagedServiceIdentity identity); } - /** The stage of the NewRelicMonitorResource update allowing to specify newRelicAccountProperties. */ + + /** + * The stage of the NewRelicMonitorResource update allowing to specify newRelicAccountProperties. + */ interface WithNewRelicAccountProperties { /** * Specifies the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @param newRelicAccountProperties MarketplaceSubscriptionStatus of the resource. * @return the next definition stage. */ Update withNewRelicAccountProperties(NewRelicAccountProperties newRelicAccountProperties); } - /** The stage of the NewRelicMonitorResource update allowing to specify userInfo. */ + + /** + * The stage of the NewRelicMonitorResource update allowing to specify userInfo. + */ interface WithUserInfo { /** * Specifies the userInfo property: User Info. - * + * * @param userInfo User Info. * @return the next definition stage. */ Update withUserInfo(UserInfo userInfo); } - /** The stage of the NewRelicMonitorResource update allowing to specify planData. */ + + /** + * The stage of the NewRelicMonitorResource update allowing to specify planData. + */ interface WithPlanData { /** * Specifies the planData property: Plan details. - * + * * @param planData Plan details. * @return the next definition stage. */ Update withPlanData(PlanData planData); } - /** The stage of the NewRelicMonitorResource update allowing to specify orgCreationSource. */ + + /** + * The stage of the NewRelicMonitorResource update allowing to specify orgCreationSource. + */ interface WithOrgCreationSource { /** * Specifies the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource Source of org creation. * @return the next definition stage. */ Update withOrgCreationSource(OrgCreationSource orgCreationSource); } - /** The stage of the NewRelicMonitorResource update allowing to specify accountCreationSource. */ + + /** + * The stage of the NewRelicMonitorResource update allowing to specify accountCreationSource. + */ interface WithAccountCreationSource { /** * Specifies the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource Source of account creation. * @return the next definition stage. */ Update withAccountCreationSource(AccountCreationSource accountCreationSource); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ NewRelicMonitorResource refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -427,7 +522,7 @@ interface WithAccountCreationSource { /** * Get metric rules. - * + * * @param request The details of the get metrics status request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -439,7 +534,7 @@ interface WithAccountCreationSource { /** * Get metric rules. - * + * * @param request The details of the get metrics status request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -450,7 +545,7 @@ interface WithAccountCreationSource { /** * Get metric status. - * + * * @param request The details of the get metrics status request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -462,7 +557,7 @@ interface WithAccountCreationSource { /** * Get metric status. - * + * * @param request The details of the get metrics status request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -473,7 +568,7 @@ interface WithAccountCreationSource { /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param request The details of the app services get request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -484,7 +579,7 @@ interface WithAccountCreationSource { /** * List the app service resources currently being monitored by the NewRelic resource. - * + * * @param request The details of the app services get request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -496,7 +591,7 @@ interface WithAccountCreationSource { /** * Switches the billing for NewRelic monitor resource. - * + * * @param request The details of the switch billing request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -508,7 +603,7 @@ interface WithAccountCreationSource { /** * Switches the billing for NewRelic monitor resource. - * + * * @param request The details of the switch billing request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -519,7 +614,7 @@ interface WithAccountCreationSource { /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param request The details of the Hosts get request. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -530,7 +625,7 @@ interface WithAccountCreationSource { /** * List the compute vm resources currently being monitored by the NewRelic resource. - * + * * @param request The details of the Hosts get request. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -542,29 +637,49 @@ interface WithAccountCreationSource { /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ PagedIterable listMonitoredResources(); /** * List the resources currently being monitored by the NewRelic monitor resource. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with {@link - * PagedIterable}. + * @return list of all the resources being monitored by NewRelic monitor resource as paginated response with + * {@link PagedIterable}. */ PagedIterable listMonitoredResources(Context context); + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + PagedIterable listLinkedResources(); + + /** + * List all Azure resources associated to the same NewRelic organization and account as the target resource. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response of a list operation as paginated response with {@link PagedIterable}. + */ + PagedIterable listLinkedResources(Context context); + /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -575,7 +690,7 @@ interface WithAccountCreationSource { /** * Returns the payload that needs to be passed in the request body for installing NewRelic agent on a VM. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of payload to be passed while installing VM agent. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceListResult.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceListResult.java index 8456cc9e70990..c98892305a9eb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceListResult.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a NewRelicMonitorResource list operation. */ +/** + * The response of a NewRelicMonitorResource list operation. + */ @Fluent public final class NewRelicMonitorResourceListResult { /* @@ -25,13 +27,15 @@ public final class NewRelicMonitorResourceListResult { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of NewRelicMonitorResourceListResult class. */ + /** + * Creates an instance of NewRelicMonitorResourceListResult class. + */ public NewRelicMonitorResourceListResult() { } /** * Get the value property: The NewRelicMonitorResource items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The NewRelicMonitorResource items on this page. - * + * * @param value the value value to set. * @return the NewRelicMonitorResourceListResult object itself. */ @@ -51,7 +55,7 @@ public NewRelicMonitorResourceListResult withValue(List e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceUpdate.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceUpdate.java index 7d1a305813433..116d814f188e9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceUpdate.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicMonitorResourceUpdate.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** The type used for update operations of the NewRelicMonitorResource. */ +/** + * The type used for update operations of the NewRelicMonitorResource. + */ @Fluent public final class NewRelicMonitorResourceUpdate { /* @@ -32,13 +34,15 @@ public final class NewRelicMonitorResourceUpdate { @JsonProperty(value = "properties") private NewRelicMonitorResourceUpdateProperties innerProperties; - /** Creates an instance of NewRelicMonitorResourceUpdate class. */ + /** + * Creates an instance of NewRelicMonitorResourceUpdate class. + */ public NewRelicMonitorResourceUpdate() { } /** * Get the identity property: The managed service identities assigned to this resource. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -47,7 +51,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The managed service identities assigned to this resource. - * + * * @param identity the identity value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ @@ -58,7 +62,7 @@ public NewRelicMonitorResourceUpdate withIdentity(ManagedServiceIdentity identit /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -67,7 +71,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ @@ -78,7 +82,7 @@ public NewRelicMonitorResourceUpdate withTags(Map tags) { /** * Get the innerProperties property: The updatable properties of the NewRelicMonitorResource. - * + * * @return the innerProperties value. */ private NewRelicMonitorResourceUpdateProperties innerProperties() { @@ -87,7 +91,7 @@ private NewRelicMonitorResourceUpdateProperties innerProperties() { /** * Get the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @return the newRelicAccountProperties value. */ public NewRelicAccountProperties newRelicAccountProperties() { @@ -96,12 +100,12 @@ public NewRelicAccountProperties newRelicAccountProperties() { /** * Set the newRelicAccountProperties property: MarketplaceSubscriptionStatus of the resource. - * + * * @param newRelicAccountProperties the newRelicAccountProperties value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ - public NewRelicMonitorResourceUpdate withNewRelicAccountProperties( - NewRelicAccountProperties newRelicAccountProperties) { + public NewRelicMonitorResourceUpdate + withNewRelicAccountProperties(NewRelicAccountProperties newRelicAccountProperties) { if (this.innerProperties() == null) { this.innerProperties = new NewRelicMonitorResourceUpdateProperties(); } @@ -111,7 +115,7 @@ public NewRelicMonitorResourceUpdate withNewRelicAccountProperties( /** * Get the userInfo property: User Info. - * + * * @return the userInfo value. */ public UserInfo userInfo() { @@ -120,7 +124,7 @@ public UserInfo userInfo() { /** * Set the userInfo property: User Info. - * + * * @param userInfo the userInfo value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ @@ -134,7 +138,7 @@ public NewRelicMonitorResourceUpdate withUserInfo(UserInfo userInfo) { /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -143,7 +147,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ @@ -157,7 +161,7 @@ public NewRelicMonitorResourceUpdate withPlanData(PlanData planData) { /** * Get the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ public OrgCreationSource orgCreationSource() { @@ -166,7 +170,7 @@ public OrgCreationSource orgCreationSource() { /** * Set the orgCreationSource property: Source of org creation. - * + * * @param orgCreationSource the orgCreationSource value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ @@ -180,7 +184,7 @@ public NewRelicMonitorResourceUpdate withOrgCreationSource(OrgCreationSource org /** * Get the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ public AccountCreationSource accountCreationSource() { @@ -189,7 +193,7 @@ public AccountCreationSource accountCreationSource() { /** * Set the accountCreationSource property: Source of account creation. - * + * * @param accountCreationSource the accountCreationSource value to set. * @return the NewRelicMonitorResourceUpdate object itself. */ @@ -203,7 +207,7 @@ public NewRelicMonitorResourceUpdate withAccountCreationSource(AccountCreationSo /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicSingleSignOnProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicSingleSignOnProperties.java index fb76f04353517..9ec3bb2746efe 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicSingleSignOnProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/NewRelicSingleSignOnProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Single sign on Info of the NewRelic account. */ +/** + * Single sign on Info of the NewRelic account. + */ @Fluent public final class NewRelicSingleSignOnProperties { /* @@ -34,13 +36,15 @@ public final class NewRelicSingleSignOnProperties { @JsonProperty(value = "provisioningState") private ProvisioningState provisioningState; - /** Creates an instance of NewRelicSingleSignOnProperties class. */ + /** + * Creates an instance of NewRelicSingleSignOnProperties class. + */ public NewRelicSingleSignOnProperties() { } /** * Get the singleSignOnState property: Single sign-on state. - * + * * @return the singleSignOnState value. */ public SingleSignOnStates singleSignOnState() { @@ -49,7 +53,7 @@ public SingleSignOnStates singleSignOnState() { /** * Set the singleSignOnState property: Single sign-on state. - * + * * @param singleSignOnState the singleSignOnState value to set. * @return the NewRelicSingleSignOnProperties object itself. */ @@ -60,7 +64,7 @@ public NewRelicSingleSignOnProperties withSingleSignOnState(SingleSignOnStates s /** * Get the enterpriseAppId property: The Id of the Enterprise App used for Single sign-on. - * + * * @return the enterpriseAppId value. */ public String enterpriseAppId() { @@ -69,7 +73,7 @@ public String enterpriseAppId() { /** * Set the enterpriseAppId property: The Id of the Enterprise App used for Single sign-on. - * + * * @param enterpriseAppId the enterpriseAppId value to set. * @return the NewRelicSingleSignOnProperties object itself. */ @@ -80,7 +84,7 @@ public NewRelicSingleSignOnProperties withEnterpriseAppId(String enterpriseAppId /** * Get the singleSignOnUrl property: The login URL specific to this NewRelic Organization. - * + * * @return the singleSignOnUrl value. */ public String singleSignOnUrl() { @@ -89,7 +93,7 @@ public String singleSignOnUrl() { /** * Set the singleSignOnUrl property: The login URL specific to this NewRelic Organization. - * + * * @param singleSignOnUrl the singleSignOnUrl value to set. * @return the NewRelicSingleSignOnProperties object itself. */ @@ -100,7 +104,7 @@ public NewRelicSingleSignOnProperties withSingleSignOnUrl(String singleSignOnUrl /** * Get the provisioningState property: Provisioning state. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -109,7 +113,7 @@ public ProvisioningState provisioningState() { /** * Set the provisioningState property: Provisioning state. - * + * * @param provisioningState the provisioningState value to set. * @return the NewRelicSingleSignOnProperties object itself. */ @@ -120,7 +124,7 @@ public NewRelicSingleSignOnProperties withProvisioningState(ProvisioningState pr /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operation.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operation.java index 4e0bc20729565..cadeca2fca19a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operation.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operation.java @@ -6,12 +6,14 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ String name(); @@ -19,14 +21,14 @@ public interface Operation { /** * Gets the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Localized display information for this particular operation. - * + * * @return the display value. */ OperationDisplay display(); @@ -34,7 +36,7 @@ public interface Operation { /** * Gets the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ Origin origin(); @@ -42,14 +44,14 @@ public interface Operation { /** * Gets the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ ActionType actionType(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationDisplay.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationDisplay.java index 1a883f178bdfe..72fab579594c3 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationDisplay.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Localized display information for this particular operation. */ +/** + * Localized display information for this particular operation. + */ @Immutable public final class OperationDisplay { /* @@ -37,14 +39,16 @@ public final class OperationDisplay { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: The localized friendly form of the resource provider name, e.g. "Microsoft Monitoring * Insights" or "Microsoft Compute". - * + * * @return the provider value. */ public String provider() { @@ -54,7 +58,7 @@ public String provider() { /** * Get the resource property: The localized friendly name of the resource type related to this operation. E.g. * "Virtual Machines" or "Job Schedule Collections". - * + * * @return the resource value. */ public String resource() { @@ -64,7 +68,7 @@ public String resource() { /** * Get the operation property: The concise, localized friendly name for the operation; suitable for dropdowns. E.g. * "Create or Update Virtual Machine", "Restart Virtual Machine". - * + * * @return the operation value. */ public String operation() { @@ -74,7 +78,7 @@ public String operation() { /** * Get the description property: The short, localized friendly description of the operation; suitable for tool tips * and detailed views. - * + * * @return the description value. */ public String description() { @@ -83,7 +87,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationListResult.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationListResult.java index 22e5c76da7911..13c7b8937074c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationListResult.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OperationListResult.java @@ -10,8 +10,8 @@ import java.util.List; /** - * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set of - * results. + * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set + * of results. */ @Immutable public final class OperationListResult { @@ -27,13 +27,15 @@ public final class OperationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationListResult class. */ + /** + * Creates an instance of OperationListResult class. + */ public OperationListResult() { } /** * Get the value property: List of operations supported by the resource provider. - * + * * @return the value value. */ public List value() { @@ -42,7 +44,7 @@ public List value() { /** * Get the nextLink property: URL to get the next set of operation list results (if there are any). - * + * * @return the nextLink value. */ public String nextLink() { @@ -51,7 +53,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operations.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operations.java index d6f787c73e1a0..684a32ff85cb9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operations.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Operations.java @@ -7,27 +7,29 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * List the operations for the provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * List the operations for the provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrgCreationSource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrgCreationSource.java index 7c4a4fa0fc60a..074728fd70d8d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrgCreationSource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrgCreationSource.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Source of Org creation. */ +/** + * Source of Org creation. + */ public final class OrgCreationSource extends ExpandableStringEnum { - /** Static value LIFTR for OrgCreationSource. */ + /** + * Static value LIFTR for OrgCreationSource. + */ public static final OrgCreationSource LIFTR = fromString("LIFTR"); - /** Static value NEWRELIC for OrgCreationSource. */ + /** + * Static value NEWRELIC for OrgCreationSource. + */ public static final OrgCreationSource NEWRELIC = fromString("NEWRELIC"); /** * Creates a new instance of OrgCreationSource value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public OrgCreationSource() { /** * Creates or finds a OrgCreationSource from its string representation. - * + * * @param name a name to look for. * @return the corresponding OrgCreationSource. */ @@ -38,7 +44,7 @@ public static OrgCreationSource fromString(String name) { /** * Gets known OrgCreationSource values. - * + * * @return known OrgCreationSource values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationInfo.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationInfo.java index 4e55b4ebcf9ff..cd90e5b41d964 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationInfo.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationInfo.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Organization Info of the NewRelic account. */ +/** + * Organization Info of the NewRelic account. + */ @Fluent public final class OrganizationInfo { /* @@ -16,13 +18,15 @@ public final class OrganizationInfo { @JsonProperty(value = "organizationId") private String organizationId; - /** Creates an instance of OrganizationInfo class. */ + /** + * Creates an instance of OrganizationInfo class. + */ public OrganizationInfo() { } /** * Get the organizationId property: Organization id. - * + * * @return the organizationId value. */ public String organizationId() { @@ -31,7 +35,7 @@ public String organizationId() { /** * Set the organizationId property: Organization id. - * + * * @param organizationId the organizationId value to set. * @return the OrganizationInfo object itself. */ @@ -42,7 +46,7 @@ public OrganizationInfo withOrganizationId(String organizationId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationResource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationResource.java index 05e6bb377c641..404d3bf67a8d9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationResource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationResource.java @@ -7,60 +7,62 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.newrelicobservability.fluent.models.OrganizationResourceInner; -/** An immutable client-side representation of OrganizationResource. */ +/** + * An immutable client-side representation of OrganizationResource. + */ public interface OrganizationResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the organizationId property: organization id. - * + * * @return the organizationId value. */ String organizationId(); /** * Gets the organizationName property: organization name. - * + * * @return the organizationName value. */ String organizationName(); /** * Gets the billingSource property: Billing source. - * + * * @return the billingSource value. */ BillingSource billingSource(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.OrganizationResourceInner object. - * + * * @return the inner object. */ OrganizationResourceInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Organizations.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Organizations.java index bd2e6d2483f5f..691a8c4b7900d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Organizations.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Organizations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Organizations. */ +/** + * Resource collection API of Organizations. + */ public interface Organizations { /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface Organizations { /** * List all the existing organizations. - * + * * @param userEmail User Email. * @param location Location for NewRelic. * @param context The context to associate with this operation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationsListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationsListResponse.java index 489abe9dec3a5..e46061fe6a60d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationsListResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/OrganizationsListResponse.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response of get all organizations Operation. */ +/** + * Response of get all organizations Operation. + */ @Fluent public final class OrganizationsListResponse { /* @@ -25,13 +27,15 @@ public final class OrganizationsListResponse { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of OrganizationsListResponse class. */ + /** + * Creates an instance of OrganizationsListResponse class. + */ public OrganizationsListResponse() { } /** * Get the value property: The OrganizationResource items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The OrganizationResource items on this page. - * + * * @param value the value value to set. * @return the OrganizationsListResponse object itself. */ @@ -51,7 +55,7 @@ public OrganizationsListResponse withValue(List value /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the OrganizationsListResponse object itself. */ @@ -71,14 +75,13 @@ public OrganizationsListResponse withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model OrganizationsListResponse")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model OrganizationsListResponse")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Origin.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Origin.java index fa768b1775524..2d78da40d07b9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Origin.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Origin.java @@ -13,18 +13,24 @@ * is "user,system". */ public final class Origin extends ExpandableStringEnum { - /** Static value user for Origin. */ + /** + * Static value user for Origin. + */ public static final Origin USER = fromString("user"); - /** Static value system for Origin. */ + /** + * Static value system for Origin. + */ public static final Origin SYSTEM = fromString("system"); - /** Static value user,system for Origin. */ + /** + * Static value user,system for Origin. + */ public static final Origin USER_SYSTEM = fromString("user,system"); /** * Creates a new instance of Origin value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +39,7 @@ public Origin() { /** * Creates or finds a Origin from its string representation. - * + * * @param name a name to look for. * @return the corresponding Origin. */ @@ -44,7 +50,7 @@ public static Origin fromString(String name) { /** * Gets known Origin values. - * + * * @return known Origin values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PartnerBillingEntity.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PartnerBillingEntity.java new file mode 100644 index 0000000000000..2c1296363c5a8 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PartnerBillingEntity.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Partner Billing details associated with the resource. + */ +@Fluent +public final class PartnerBillingEntity { + /* + * The New Relic Organization Id. + */ + @JsonProperty(value = "organizationId") + private String organizationId; + + /* + * The New Relic Organization Name. + */ + @JsonProperty(value = "organizationName") + private String organizationName; + + /** + * Creates an instance of PartnerBillingEntity class. + */ + public PartnerBillingEntity() { + } + + /** + * Get the organizationId property: The New Relic Organization Id. + * + * @return the organizationId value. + */ + public String organizationId() { + return this.organizationId; + } + + /** + * Set the organizationId property: The New Relic Organization Id. + * + * @param organizationId the organizationId value to set. + * @return the PartnerBillingEntity object itself. + */ + public PartnerBillingEntity withOrganizationId(String organizationId) { + this.organizationId = organizationId; + return this; + } + + /** + * Get the organizationName property: The New Relic Organization Name. + * + * @return the organizationName value. + */ + public String organizationName() { + return this.organizationName; + } + + /** + * Set the organizationName property: The New Relic Organization Name. + * + * @param organizationName the organizationName value to set. + * @return the PartnerBillingEntity object itself. + */ + public PartnerBillingEntity withOrganizationName(String organizationName) { + this.organizationName = organizationName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PatchOperation.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PatchOperation.java new file mode 100644 index 0000000000000..cae070e9070a1 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PatchOperation.java @@ -0,0 +1,68 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The operation for the patch on the resource. + */ +public final class PatchOperation extends ExpandableStringEnum { + /** + * Static value AddBegin for PatchOperation. + */ + public static final PatchOperation ADD_BEGIN = fromString("AddBegin"); + + /** + * Static value AddComplete for PatchOperation. + */ + public static final PatchOperation ADD_COMPLETE = fromString("AddComplete"); + + /** + * Static value DeleteBegin for PatchOperation. + */ + public static final PatchOperation DELETE_BEGIN = fromString("DeleteBegin"); + + /** + * Static value DeleteComplete for PatchOperation. + */ + public static final PatchOperation DELETE_COMPLETE = fromString("DeleteComplete"); + + /** + * Static value Active for PatchOperation. + */ + public static final PatchOperation ACTIVE = fromString("Active"); + + /** + * Creates a new instance of PatchOperation value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PatchOperation() { + } + + /** + * Creates or finds a PatchOperation from its string representation. + * + * @param name a name to look for. + * @return the corresponding PatchOperation. + */ + @JsonCreator + public static PatchOperation fromString(String name) { + return fromString(name, PatchOperation.class); + } + + /** + * Gets known PatchOperation values. + * + * @return known PatchOperation values. + */ + public static Collection values() { + return values(PatchOperation.class); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanData.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanData.java index 1317958829e1a..4d79b71569478 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanData.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanData.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Plan data of NewRelic Monitor resource. */ +/** + * Plan data of NewRelic Monitor resource. + */ @Fluent public final class PlanData { /* @@ -35,13 +37,15 @@ public final class PlanData { @JsonProperty(value = "effectiveDate") private OffsetDateTime effectiveDate; - /** Creates an instance of PlanData class. */ + /** + * Creates an instance of PlanData class. + */ public PlanData() { } /** * Get the usageType property: Different usage type like PAYG/COMMITTED. this could be enum. - * + * * @return the usageType value. */ public UsageType usageType() { @@ -50,7 +54,7 @@ public UsageType usageType() { /** * Set the usageType property: Different usage type like PAYG/COMMITTED. this could be enum. - * + * * @param usageType the usageType value to set. * @return the PlanData object itself. */ @@ -61,7 +65,7 @@ public PlanData withUsageType(UsageType usageType) { /** * Get the billingCycle property: Different billing cycles like MONTHLY/WEEKLY. this could be enum. - * + * * @return the billingCycle value. */ public BillingCycle billingCycle() { @@ -70,7 +74,7 @@ public BillingCycle billingCycle() { /** * Set the billingCycle property: Different billing cycles like MONTHLY/WEEKLY. this could be enum. - * + * * @param billingCycle the billingCycle value to set. * @return the PlanData object itself. */ @@ -81,7 +85,7 @@ public PlanData withBillingCycle(BillingCycle billingCycle) { /** * Get the planDetails property: plan id as published by NewRelic. - * + * * @return the planDetails value. */ public String planDetails() { @@ -90,7 +94,7 @@ public String planDetails() { /** * Set the planDetails property: plan id as published by NewRelic. - * + * * @param planDetails the planDetails value to set. * @return the PlanData object itself. */ @@ -101,7 +105,7 @@ public PlanData withPlanDetails(String planDetails) { /** * Get the effectiveDate property: date when plan was applied. - * + * * @return the effectiveDate value. */ public OffsetDateTime effectiveDate() { @@ -110,7 +114,7 @@ public OffsetDateTime effectiveDate() { /** * Set the effectiveDate property: date when plan was applied. - * + * * @param effectiveDate the effectiveDate value to set. * @return the PlanData object itself. */ @@ -121,7 +125,7 @@ public PlanData withEffectiveDate(OffsetDateTime effectiveDate) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataListResponse.java index 250d852597fe0..d2dda05c88b09 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataListResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataListResponse.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response of get all plan data Operation. */ +/** + * Response of get all plan data Operation. + */ @Fluent public final class PlanDataListResponse { /* @@ -25,13 +27,15 @@ public final class PlanDataListResponse { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of PlanDataListResponse class. */ + /** + * Creates an instance of PlanDataListResponse class. + */ public PlanDataListResponse() { } /** * Get the value property: The PlanDataResource items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The PlanDataResource items on this page. - * + * * @param value the value value to set. * @return the PlanDataListResponse object itself. */ @@ -51,7 +55,7 @@ public PlanDataListResponse withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the PlanDataListResponse object itself. */ @@ -71,14 +75,13 @@ public PlanDataListResponse withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model PlanDataListResponse")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model PlanDataListResponse")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataResource.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataResource.java index 9082254c41831..6eeb165eb3a9c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataResource.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/PlanDataResource.java @@ -7,60 +7,62 @@ import com.azure.core.management.SystemData; import com.azure.resourcemanager.newrelicobservability.fluent.models.PlanDataResourceInner; -/** An immutable client-side representation of PlanDataResource. */ +/** + * An immutable client-side representation of PlanDataResource. + */ public interface PlanDataResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the planData property: Plan details. - * + * * @return the planData value. */ PlanData planData(); /** * Gets the orgCreationSource property: Source of org creation. - * + * * @return the orgCreationSource value. */ OrgCreationSource orgCreationSource(); /** * Gets the accountCreationSource property: Source of account creation. - * + * * @return the accountCreationSource value. */ AccountCreationSource accountCreationSource(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.PlanDataResourceInner object. - * + * * @return the inner object. */ PlanDataResourceInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Plans.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Plans.java index 37ec8e453fc3f..bc1920e6e8c88 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Plans.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Plans.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Plans. */ +/** + * Resource collection API of Plans. + */ public interface Plans { /** * List plans data. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response of get all plan data Operation as paginated response with {@link PagedIterable}. @@ -20,7 +22,7 @@ public interface Plans { /** * List plans data. - * + * * @param accountId Account Id. * @param organizationId Organization Id. * @param context The context to associate with this operation. diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ProvisioningState.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ProvisioningState.java index 067bc104a45ca..de7d9b599c2b7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ProvisioningState.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/ProvisioningState.java @@ -8,38 +8,58 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning State of the Monitor resource. */ +/** + * Provisioning State of the Monitor resource. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Accepted for ProvisioningState. */ + /** + * Static value Accepted for ProvisioningState. + */ public static final ProvisioningState ACCEPTED = fromString("Accepted"); - /** Static value Creating for ProvisioningState. */ + /** + * Static value Creating for ProvisioningState. + */ public static final ProvisioningState CREATING = fromString("Creating"); - /** Static value Updating for ProvisioningState. */ + /** + * Static value Updating for ProvisioningState. + */ public static final ProvisioningState UPDATING = fromString("Updating"); - /** Static value Deleting for ProvisioningState. */ + /** + * Static value Deleting for ProvisioningState. + */ public static final ProvisioningState DELETING = fromString("Deleting"); - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value Deleted for ProvisioningState. */ + /** + * Static value Deleted for ProvisioningState. + */ public static final ProvisioningState DELETED = fromString("Deleted"); - /** Static value NotSpecified for ProvisioningState. */ + /** + * Static value NotSpecified for ProvisioningState. + */ public static final ProvisioningState NOT_SPECIFIED = fromString("NotSpecified"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -48,7 +68,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -59,7 +79,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendAadLogsStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendAadLogsStatus.java index d5597533b8603..ee4878c00331e 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendAadLogsStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendAadLogsStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether AAD logs are being sent. */ +/** + * Indicates whether AAD logs are being sent. + */ public final class SendAadLogsStatus extends ExpandableStringEnum { - /** Static value Enabled for SendAadLogsStatus. */ + /** + * Static value Enabled for SendAadLogsStatus. + */ public static final SendAadLogsStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for SendAadLogsStatus. */ + /** + * Static value Disabled for SendAadLogsStatus. + */ public static final SendAadLogsStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of SendAadLogsStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SendAadLogsStatus() { /** * Creates or finds a SendAadLogsStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding SendAadLogsStatus. */ @@ -38,7 +44,7 @@ public static SendAadLogsStatus fromString(String name) { /** * Gets known SendAadLogsStatus values. - * + * * @return known SendAadLogsStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendActivityLogsStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendActivityLogsStatus.java index ad2ccd83738b2..3afab4836c366 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendActivityLogsStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendActivityLogsStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether activity logs are being sent. */ +/** + * Indicates whether activity logs are being sent. + */ public final class SendActivityLogsStatus extends ExpandableStringEnum { - /** Static value Enabled for SendActivityLogsStatus. */ + /** + * Static value Enabled for SendActivityLogsStatus. + */ public static final SendActivityLogsStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for SendActivityLogsStatus. */ + /** + * Static value Disabled for SendActivityLogsStatus. + */ public static final SendActivityLogsStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of SendActivityLogsStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SendActivityLogsStatus() { /** * Creates or finds a SendActivityLogsStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding SendActivityLogsStatus. */ @@ -38,7 +44,7 @@ public static SendActivityLogsStatus fromString(String name) { /** * Gets known SendActivityLogsStatus values. - * + * * @return known SendActivityLogsStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendMetricsStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendMetricsStatus.java index 86316bd5643ac..93995df8d6bb7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendMetricsStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendMetricsStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether metrics are being sent. */ +/** + * Indicates whether metrics are being sent. + */ public final class SendMetricsStatus extends ExpandableStringEnum { - /** Static value Enabled for SendMetricsStatus. */ + /** + * Static value Enabled for SendMetricsStatus. + */ public static final SendMetricsStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for SendMetricsStatus. */ + /** + * Static value Disabled for SendMetricsStatus. + */ public static final SendMetricsStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of SendMetricsStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SendMetricsStatus() { /** * Creates or finds a SendMetricsStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding SendMetricsStatus. */ @@ -38,7 +44,7 @@ public static SendMetricsStatus fromString(String name) { /** * Gets known SendMetricsStatus values. - * + * * @return known SendMetricsStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendSubscriptionLogsStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendSubscriptionLogsStatus.java index 23b5400c6ce58..be80411e148b4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendSubscriptionLogsStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendSubscriptionLogsStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether subscription logs are being sent. */ +/** + * Indicates whether subscription logs are being sent. + */ public final class SendSubscriptionLogsStatus extends ExpandableStringEnum { - /** Static value Enabled for SendSubscriptionLogsStatus. */ + /** + * Static value Enabled for SendSubscriptionLogsStatus. + */ public static final SendSubscriptionLogsStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for SendSubscriptionLogsStatus. */ + /** + * Static value Disabled for SendSubscriptionLogsStatus. + */ public static final SendSubscriptionLogsStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of SendSubscriptionLogsStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SendSubscriptionLogsStatus() { /** * Creates or finds a SendSubscriptionLogsStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding SendSubscriptionLogsStatus. */ @@ -38,7 +44,7 @@ public static SendSubscriptionLogsStatus fromString(String name) { /** * Gets known SendSubscriptionLogsStatus values. - * + * * @return known SendSubscriptionLogsStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingLogsStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingLogsStatus.java index da558d59ae8de..9be59c17731cd 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingLogsStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingLogsStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether logs are being sent. */ +/** + * Indicates whether logs are being sent. + */ public final class SendingLogsStatus extends ExpandableStringEnum { - /** Static value Enabled for SendingLogsStatus. */ + /** + * Static value Enabled for SendingLogsStatus. + */ public static final SendingLogsStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for SendingLogsStatus. */ + /** + * Static value Disabled for SendingLogsStatus. + */ public static final SendingLogsStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of SendingLogsStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SendingLogsStatus() { /** * Creates or finds a SendingLogsStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding SendingLogsStatus. */ @@ -38,7 +44,7 @@ public static SendingLogsStatus fromString(String name) { /** * Gets known SendingLogsStatus values. - * + * * @return known SendingLogsStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingMetricsStatus.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingMetricsStatus.java index 6cc855315bc75..f1eef9e27b0cb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingMetricsStatus.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SendingMetricsStatus.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Indicates whether metrics are being sent. */ +/** + * Indicates whether metrics are being sent. + */ public final class SendingMetricsStatus extends ExpandableStringEnum { - /** Static value Enabled for SendingMetricsStatus. */ + /** + * Static value Enabled for SendingMetricsStatus. + */ public static final SendingMetricsStatus ENABLED = fromString("Enabled"); - /** Static value Disabled for SendingMetricsStatus. */ + /** + * Static value Disabled for SendingMetricsStatus. + */ public static final SendingMetricsStatus DISABLED = fromString("Disabled"); /** * Creates a new instance of SendingMetricsStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public SendingMetricsStatus() { /** * Creates or finds a SendingMetricsStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding SendingMetricsStatus. */ @@ -38,7 +44,7 @@ public static SendingMetricsStatus fromString(String name) { /** * Gets known SendingMetricsStatus values. - * + * * @return known SendingMetricsStatus values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SingleSignOnStates.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SingleSignOnStates.java index 6f6faa101ed36..c92e95f9d205a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SingleSignOnStates.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SingleSignOnStates.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Various states of the SSO resource. */ +/** + * Various states of the SSO resource. + */ public final class SingleSignOnStates extends ExpandableStringEnum { - /** Static value Initial for SingleSignOnStates. */ + /** + * Static value Initial for SingleSignOnStates. + */ public static final SingleSignOnStates INITIAL = fromString("Initial"); - /** Static value Enable for SingleSignOnStates. */ + /** + * Static value Enable for SingleSignOnStates. + */ public static final SingleSignOnStates ENABLE = fromString("Enable"); - /** Static value Disable for SingleSignOnStates. */ + /** + * Static value Disable for SingleSignOnStates. + */ public static final SingleSignOnStates DISABLE = fromString("Disable"); - /** Static value Existing for SingleSignOnStates. */ + /** + * Static value Existing for SingleSignOnStates. + */ public static final SingleSignOnStates EXISTING = fromString("Existing"); /** * Creates a new instance of SingleSignOnStates value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public SingleSignOnStates() { /** * Creates or finds a SingleSignOnStates from its string representation. - * + * * @param name a name to look for. * @return the corresponding SingleSignOnStates. */ @@ -44,7 +54,7 @@ public static SingleSignOnStates fromString(String name) { /** * Gets known SingleSignOnStates values. - * + * * @return known SingleSignOnStates values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Status.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Status.java new file mode 100644 index 0000000000000..ec90cf8326883 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/Status.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The state of monitoring. + */ +public final class Status extends ExpandableStringEnum { + /** + * Static value InProgress for Status. + */ + public static final Status IN_PROGRESS = fromString("InProgress"); + + /** + * Static value Active for Status. + */ + public static final Status ACTIVE = fromString("Active"); + + /** + * Static value Failed for Status. + */ + public static final Status FAILED = fromString("Failed"); + + /** + * Static value Deleting for Status. + */ + public static final Status DELETING = fromString("Deleting"); + + /** + * Creates a new instance of Status value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public Status() { + } + + /** + * Creates or finds a Status from its string representation. + * + * @param name a name to look for. + * @return the corresponding Status. + */ + @JsonCreator + public static Status fromString(String name) { + return fromString(name, Status.class); + } + + /** + * Gets known Status values. + * + * @return known Status values. + */ + public static Collection values() { + return values(Status.class); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SubscriptionList.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SubscriptionList.java new file mode 100644 index 0000000000000..7a5bf54b8ded3 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SubscriptionList.java @@ -0,0 +1,100 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The request to update subscriptions needed to be monitored by the NewRelic monitor resource. + */ +@Fluent +public final class SubscriptionList { + /* + * The operation for the patch on the resource. + */ + @JsonProperty(value = "patchOperation") + private PatchOperation patchOperation; + + /* + * List of subscriptions and the state of the monitoring. + */ + @JsonProperty(value = "monitoredSubscriptionList") + private List monitoredSubscriptionList; + + /* + * Provisioning State of the resource + */ + @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) + private ProvisioningState provisioningState; + + /** + * Creates an instance of SubscriptionList class. + */ + public SubscriptionList() { + } + + /** + * Get the patchOperation property: The operation for the patch on the resource. + * + * @return the patchOperation value. + */ + public PatchOperation patchOperation() { + return this.patchOperation; + } + + /** + * Set the patchOperation property: The operation for the patch on the resource. + * + * @param patchOperation the patchOperation value to set. + * @return the SubscriptionList object itself. + */ + public SubscriptionList withPatchOperation(PatchOperation patchOperation) { + this.patchOperation = patchOperation; + return this; + } + + /** + * Get the monitoredSubscriptionList property: List of subscriptions and the state of the monitoring. + * + * @return the monitoredSubscriptionList value. + */ + public List monitoredSubscriptionList() { + return this.monitoredSubscriptionList; + } + + /** + * Set the monitoredSubscriptionList property: List of subscriptions and the state of the monitoring. + * + * @param monitoredSubscriptionList the monitoredSubscriptionList value to set. + * @return the SubscriptionList object itself. + */ + public SubscriptionList withMonitoredSubscriptionList(List monitoredSubscriptionList) { + this.monitoredSubscriptionList = monitoredSubscriptionList; + return this; + } + + /** + * Get the provisioningState property: Provisioning State of the resource. + * + * @return the provisioningState value. + */ + public ProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (monitoredSubscriptionList() != null) { + monitoredSubscriptionList().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SwitchBillingRequest.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SwitchBillingRequest.java index ed3e611b96b1a..6171213e240fd 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SwitchBillingRequest.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/SwitchBillingRequest.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Request of a switch billing Operation. */ +/** + * Request of a switch billing Operation. + */ @Fluent public final class SwitchBillingRequest { /* @@ -35,13 +37,15 @@ public final class SwitchBillingRequest { @JsonProperty(value = "userEmail", required = true) private String userEmail; - /** Creates an instance of SwitchBillingRequest class. */ + /** + * Creates an instance of SwitchBillingRequest class. + */ public SwitchBillingRequest() { } /** * Get the azureResourceId property: Azure resource Id. - * + * * @return the azureResourceId value. */ public String azureResourceId() { @@ -50,7 +54,7 @@ public String azureResourceId() { /** * Set the azureResourceId property: Azure resource Id. - * + * * @param azureResourceId the azureResourceId value to set. * @return the SwitchBillingRequest object itself. */ @@ -61,7 +65,7 @@ public SwitchBillingRequest withAzureResourceId(String azureResourceId) { /** * Get the organizationId property: Organization id. - * + * * @return the organizationId value. */ public String organizationId() { @@ -70,7 +74,7 @@ public String organizationId() { /** * Set the organizationId property: Organization id. - * + * * @param organizationId the organizationId value to set. * @return the SwitchBillingRequest object itself. */ @@ -81,7 +85,7 @@ public SwitchBillingRequest withOrganizationId(String organizationId) { /** * Get the planData property: Plan details. - * + * * @return the planData value. */ public PlanData planData() { @@ -90,7 +94,7 @@ public PlanData planData() { /** * Set the planData property: Plan details. - * + * * @param planData the planData value to set. * @return the SwitchBillingRequest object itself. */ @@ -101,7 +105,7 @@ public SwitchBillingRequest withPlanData(PlanData planData) { /** * Get the userEmail property: User Email. - * + * * @return the userEmail value. */ public String userEmail() { @@ -110,7 +114,7 @@ public String userEmail() { /** * Set the userEmail property: User Email. - * + * * @param userEmail the userEmail value to set. * @return the SwitchBillingRequest object itself. */ @@ -121,7 +125,7 @@ public SwitchBillingRequest withUserEmail(String userEmail) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -129,9 +133,8 @@ public void validate() { planData().validate(); } if (userEmail() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property userEmail in model SwitchBillingRequest")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property userEmail in model SwitchBillingRequest")); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagAction.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagAction.java index bc98f7da77a94..e91a643f400dc 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagAction.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagAction.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Valid actions for a filtering tag. Exclusion takes priority over inclusion. */ +/** + * Valid actions for a filtering tag. Exclusion takes priority over inclusion. + */ public final class TagAction extends ExpandableStringEnum { - /** Static value Include for TagAction. */ + /** + * Static value Include for TagAction. + */ public static final TagAction INCLUDE = fromString("Include"); - /** Static value Exclude for TagAction. */ + /** + * Static value Exclude for TagAction. + */ public static final TagAction EXCLUDE = fromString("Exclude"); /** * Creates a new instance of TagAction value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public TagAction() { /** * Creates or finds a TagAction from its string representation. - * + * * @param name a name to look for. * @return the corresponding TagAction. */ @@ -38,7 +44,7 @@ public static TagAction fromString(String name) { /** * Gets known TagAction values. - * + * * @return known TagAction values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRule.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRule.java index 9323754452f34..2862613bc1438 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRule.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRule.java @@ -9,91 +9,104 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleInner; -/** An immutable client-side representation of TagRule. */ +/** + * An immutable client-side representation of TagRule. + */ public interface TagRule { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); /** * Gets the provisioningState property: Provisioning State of the resource. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ LogRules logRules(); /** * Gets the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ MetricRules metricRules(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleInner object. - * + * * @return the inner object. */ TagRuleInner innerModel(); - /** The entirety of the TagRule definition. */ + /** + * The entirety of the TagRule definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The TagRule definition stages. */ + + /** + * The TagRule definition stages. + */ interface DefinitionStages { - /** The first stage of the TagRule definition. */ + /** + * The first stage of the TagRule definition. + */ interface Blank extends WithParentResource { } - /** The stage of the TagRule definition allowing to specify parent resource. */ + + /** + * The stage of the TagRule definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, monitorName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @return the next definition stage. */ WithCreate withExistingMonitor(String resourceGroupName, String monitorName); } + /** * The stage of the TagRule definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. @@ -101,97 +114,115 @@ interface WithParentResource { interface WithCreate extends DefinitionStages.WithLogRules, DefinitionStages.WithMetricRules { /** * Executes the create request. - * + * * @return the created resource. */ TagRule create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ TagRule create(Context context); } - /** The stage of the TagRule definition allowing to specify logRules. */ + + /** + * The stage of the TagRule definition allowing to specify logRules. + */ interface WithLogRules { /** * Specifies the logRules property: Set of rules for sending logs for the Monitor resource.. - * + * * @param logRules Set of rules for sending logs for the Monitor resource. * @return the next definition stage. */ WithCreate withLogRules(LogRules logRules); } - /** The stage of the TagRule definition allowing to specify metricRules. */ + + /** + * The stage of the TagRule definition allowing to specify metricRules. + */ interface WithMetricRules { /** * Specifies the metricRules property: Set of rules for sending metrics for the Monitor resource.. - * + * * @param metricRules Set of rules for sending metrics for the Monitor resource. * @return the next definition stage. */ WithCreate withMetricRules(MetricRulesInner metricRules); } } + /** * Begins update for the TagRule resource. - * + * * @return the stage of resource update. */ TagRule.Update update(); - /** The template for TagRule update. */ + /** + * The template for TagRule update. + */ interface Update extends UpdateStages.WithLogRules, UpdateStages.WithMetricRules { /** * Executes the update request. - * + * * @return the updated resource. */ TagRule apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ TagRule apply(Context context); } - /** The TagRule update stages. */ + + /** + * The TagRule update stages. + */ interface UpdateStages { - /** The stage of the TagRule update allowing to specify logRules. */ + /** + * The stage of the TagRule update allowing to specify logRules. + */ interface WithLogRules { /** * Specifies the logRules property: Set of rules for sending logs for the Monitor resource.. - * + * * @param logRules Set of rules for sending logs for the Monitor resource. * @return the next definition stage. */ Update withLogRules(LogRules logRules); } - /** The stage of the TagRule update allowing to specify metricRules. */ + + /** + * The stage of the TagRule update allowing to specify metricRules. + */ interface WithMetricRules { /** * Specifies the metricRules property: Set of rules for sending metrics for the Monitor resource.. - * + * * @param metricRules Set of rules for sending metrics for the Monitor resource. * @return the next definition stage. */ Update withMetricRules(MetricRulesInner metricRules); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ TagRule refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleListResult.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleListResult.java index 38d3bd256b780..488aafdcca92f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleListResult.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleListResult.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a TagRule list operation. */ +/** + * The response of a TagRule list operation. + */ @Fluent public final class TagRuleListResult { /* @@ -25,13 +27,15 @@ public final class TagRuleListResult { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of TagRuleListResult class. */ + /** + * Creates an instance of TagRuleListResult class. + */ public TagRuleListResult() { } /** * Get the value property: The TagRule items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The TagRule items on this page. - * + * * @param value the value value to set. * @return the TagRuleListResult object itself. */ @@ -51,7 +55,7 @@ public TagRuleListResult withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the TagRuleListResult object itself. */ @@ -71,14 +75,13 @@ public TagRuleListResult withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model TagRuleListResult")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model TagRuleListResult")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdate.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdate.java index 5c8e2deb4fdce..82b241c2ee764 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdate.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdate.java @@ -6,25 +6,27 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdateInner; -/** An immutable client-side representation of TagRuleUpdate. */ +/** + * An immutable client-side representation of TagRuleUpdate. + */ public interface TagRuleUpdate { /** * Gets the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ LogRules logRules(); /** * Gets the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ MetricRules metricRules(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdateInner object. - * + * * @return the inner object. */ TagRuleUpdateInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdateProperties.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdateProperties.java index 831a26093d566..6716f2b5cf645 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdateProperties.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRuleUpdateProperties.java @@ -6,25 +6,27 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdatePropertiesInner; -/** An immutable client-side representation of TagRuleUpdateProperties. */ +/** + * An immutable client-side representation of TagRuleUpdateProperties. + */ public interface TagRuleUpdateProperties { /** * Gets the logRules property: Set of rules for sending logs for the Monitor resource. - * + * * @return the logRules value. */ LogRules logRules(); /** * Gets the metricRules property: Set of rules for sending metrics for the Monitor resource. - * + * * @return the metricRules value. */ MetricRules metricRules(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdatePropertiesInner object. - * + * * @return the inner object. */ TagRuleUpdatePropertiesInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRules.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRules.java index e3fd28c3cc9b1..feb9c13f429e4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRules.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/TagRules.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of TagRules. */ +/** + * Resource collection API of TagRules. + */ public interface TagRules { /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface TagRules { /** * List TagRule resources by NewRelicMonitorResource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param context The context to associate with this operation. @@ -37,7 +39,7 @@ public interface TagRules { /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -47,12 +49,12 @@ public interface TagRules { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a TagRule along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String monitorName, String ruleSetName, Context context); + Response getWithResponse(String resourceGroupName, String monitorName, String ruleSetName, + Context context); /** * Get a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Delete a TagRule. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param monitorName Name of the Monitors resource. * @param ruleSetName Name of the TagRule. @@ -90,7 +92,7 @@ Response getWithResponse( /** * Get a TagRule. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -101,7 +103,7 @@ Response getWithResponse( /** * Get a TagRule. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -113,7 +115,7 @@ Response getWithResponse( /** * Delete a TagRule. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -123,7 +125,7 @@ Response getWithResponse( /** * Delete a TagRule. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -134,7 +136,7 @@ Response getWithResponse( /** * Begins definition for a new TagRule resource. - * + * * @param name resource name. * @return the first stage of the new TagRule definition. */ diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UsageType.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UsageType.java index 48a73b727f4d3..97bd6091e1f49 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UsageType.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UsageType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Different usage type like PAYG/COMMITTED. */ +/** + * Different usage type like PAYG/COMMITTED. + */ public final class UsageType extends ExpandableStringEnum { - /** Static value PAYG for UsageType. */ + /** + * Static value PAYG for UsageType. + */ public static final UsageType PAYG = fromString("PAYG"); - /** Static value COMMITTED for UsageType. */ + /** + * Static value COMMITTED for UsageType. + */ public static final UsageType COMMITTED = fromString("COMMITTED"); /** * Creates a new instance of UsageType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public UsageType() { /** * Creates or finds a UsageType from its string representation. - * + * * @param name a name to look for. * @return the corresponding UsageType. */ @@ -38,7 +44,7 @@ public static UsageType fromString(String name) { /** * Gets known UsageType values. - * + * * @return known UsageType values. */ public static Collection values() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserAssignedIdentity.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserAssignedIdentity.java index fd62890f013f2..48a6239592d13 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserAssignedIdentity.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserAssignedIdentity.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.UUID; -/** User assigned identity properties. */ +/** + * User assigned identity properties. + */ @Immutable public final class UserAssignedIdentity { /* @@ -23,13 +25,15 @@ public final class UserAssignedIdentity { @JsonProperty(value = "clientId", access = JsonProperty.Access.WRITE_ONLY) private UUID clientId; - /** Creates an instance of UserAssignedIdentity class. */ + /** + * Creates an instance of UserAssignedIdentity class. + */ public UserAssignedIdentity() { } /** * Get the principalId property: The principal ID of the assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -38,7 +42,7 @@ public UUID principalId() { /** * Get the clientId property: The client ID of the assigned identity. - * + * * @return the clientId value. */ public UUID clientId() { @@ -47,7 +51,7 @@ public UUID clientId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserInfo.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserInfo.java index 9bd6581205776..a51372366e3f8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserInfo.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/UserInfo.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** User Info of NewRelic Monitor resource. */ +/** + * User Info of NewRelic Monitor resource. + */ @Fluent public final class UserInfo { /* @@ -40,13 +42,15 @@ public final class UserInfo { @JsonProperty(value = "country") private String country; - /** Creates an instance of UserInfo class. */ + /** + * Creates an instance of UserInfo class. + */ public UserInfo() { } /** * Get the firstName property: First name. - * + * * @return the firstName value. */ public String firstName() { @@ -55,7 +59,7 @@ public String firstName() { /** * Set the firstName property: First name. - * + * * @param firstName the firstName value to set. * @return the UserInfo object itself. */ @@ -66,7 +70,7 @@ public UserInfo withFirstName(String firstName) { /** * Get the lastName property: Last name. - * + * * @return the lastName value. */ public String lastName() { @@ -75,7 +79,7 @@ public String lastName() { /** * Set the lastName property: Last name. - * + * * @param lastName the lastName value to set. * @return the UserInfo object itself. */ @@ -86,7 +90,7 @@ public UserInfo withLastName(String lastName) { /** * Get the emailAddress property: User Email. - * + * * @return the emailAddress value. */ public String emailAddress() { @@ -95,7 +99,7 @@ public String emailAddress() { /** * Set the emailAddress property: User Email. - * + * * @param emailAddress the emailAddress value to set. * @return the UserInfo object itself. */ @@ -106,7 +110,7 @@ public UserInfo withEmailAddress(String emailAddress) { /** * Get the phoneNumber property: Contact phone number. - * + * * @return the phoneNumber value. */ public String phoneNumber() { @@ -115,7 +119,7 @@ public String phoneNumber() { /** * Set the phoneNumber property: Contact phone number. - * + * * @param phoneNumber the phoneNumber value to set. * @return the UserInfo object itself. */ @@ -126,7 +130,7 @@ public UserInfo withPhoneNumber(String phoneNumber) { /** * Get the country property: country if user. - * + * * @return the country value. */ public String country() { @@ -135,7 +139,7 @@ public String country() { /** * Set the country property: country if user. - * + * * @param country the country value to set. * @return the UserInfo object itself. */ @@ -146,7 +150,7 @@ public UserInfo withCountry(String country) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMExtensionPayload.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMExtensionPayload.java index 51494f6b65c02..6b52140376eb6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMExtensionPayload.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMExtensionPayload.java @@ -6,18 +6,20 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.VMExtensionPayloadInner; -/** An immutable client-side representation of VMExtensionPayload. */ +/** + * An immutable client-side representation of VMExtensionPayload. + */ public interface VMExtensionPayload { /** * Gets the ingestionKey property: Ingestion key of the account. - * + * * @return the ingestionKey value. */ String ingestionKey(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.VMExtensionPayloadInner object. - * + * * @return the inner object. */ VMExtensionPayloadInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMHostsListResponse.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMHostsListResponse.java index 2c88b3ed93db9..e3a24e32e1652 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMHostsListResponse.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMHostsListResponse.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response of a list VM Host Operation. */ +/** + * Response of a list VM Host Operation. + */ @Fluent public final class VMHostsListResponse { /* @@ -25,13 +27,15 @@ public final class VMHostsListResponse { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of VMHostsListResponse class. */ + /** + * Creates an instance of VMHostsListResponse class. + */ public VMHostsListResponse() { } /** * Get the value property: The VMInfo items on this page. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The VMInfo items on this page. - * + * * @param value the value value to set. * @return the VMHostsListResponse object itself. */ @@ -51,7 +55,7 @@ public VMHostsListResponse withValue(List value) { /** * Get the nextLink property: The link to the next page of items. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,7 +64,7 @@ public String nextLink() { /** * Set the nextLink property: The link to the next page of items. - * + * * @param nextLink the nextLink value to set. * @return the VMHostsListResponse object itself. */ @@ -71,14 +75,13 @@ public VMHostsListResponse withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model VMHostsListResponse")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model VMHostsListResponse")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMInfo.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMInfo.java index a0a41cd6838de..c0e1722fb34f8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMInfo.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/VMInfo.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.newrelicobservability.fluent.models.VMInfoInner; -/** An immutable client-side representation of VMInfo. */ +/** + * An immutable client-side representation of VMInfo. + */ public interface VMInfo { /** * Gets the vmId property: Azure VM resource ID. - * + * * @return the vmId value. */ String vmId(); /** * Gets the agentVersion property: Version of the NewRelic agent installed on the VM. - * + * * @return the agentVersion value. */ String agentVersion(); /** * Gets the agentStatus property: Status of the NewRelic agent installed on the VM. - * + * * @return the agentStatus value. */ String agentStatus(); /** * Gets the inner com.azure.resourcemanager.newrelicobservability.fluent.models.VMInfoInner object. - * + * * @return the inner object. */ VMInfoInner innerModel(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/package-info.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/package-info.java index bbb69d3e3ea9b..340af4fed7366 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/package-info.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the data models for NewRelicObservability. null. */ +/** + * Package containing the data models for NewRelicObservability. + * null. + */ package com.azure.resourcemanager.newrelicobservability.models; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/package-info.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/package-info.java index b9e83c443771e..d4611309be041 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/package-info.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/com/azure/resourcemanager/newrelicobservability/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for NewRelicObservability. null. */ +/** + * Package containing the classes for NewRelicObservability. + * null. + */ package com.azure.resourcemanager.newrelicobservability; diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/module-info.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/module-info.java index 6705bb4a65a92..ea7a0148c61fd 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/module-info.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.newrelicobservability { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.newrelicobservability; exports com.azure.resourcemanager.newrelicobservability.fluent; exports com.azure.resourcemanager.newrelicobservability.fluent.models; exports com.azure.resourcemanager.newrelicobservability.models; - - opens com.azure.resourcemanager.newrelicobservability.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.newrelicobservability.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.newrelicobservability.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.newrelicobservability.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/proxy-config.json b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/proxy-config.json new file mode 100644 index 0000000000000..a8095368d2081 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.newrelicobservability.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.AccountsClientImpl$AccountsService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.MonitorsClientImpl$MonitorsService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.OrganizationsClientImpl$OrganizationsService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.PlansClientImpl$PlansService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.BillingInfoesClientImpl$BillingInfoesService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.ConnectedPartnerResourcesClientImpl$ConnectedPartnerResourcesService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.TagRulesClientImpl$TagRulesService" ], [ "com.azure.resourcemanager.newrelicobservability.implementation.MonitoredSubscriptionsClientImpl$MonitoredSubscriptionsService" ] ] \ No newline at end of file diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/reflect-config.json b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/reflect-config.json new file mode 100644 index 0000000000000..a8e7fce62ba7a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-newrelicobservability/reflect-config.json @@ -0,0 +1,411 @@ +[ { + "name" : "com.azure.resourcemanager.newrelicobservability.models.OperationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.AccountsListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.AccountResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.AccountProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResourceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MonitorProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.NewRelicAccountProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.AccountInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.OrganizationInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.NewRelicSingleSignOnProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.UserInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.PlanData", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.UserAssignedIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.OrganizationsListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.OrganizationResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.OrganizationProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.PlanDataListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.PlanDataResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.PlanDataProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResourceUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceUpdateProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MetricsRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.FilteringTag", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MetricsStatusRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MetricsStatusResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.AppServicesGetRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.AppServicesListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.AppServiceInfoInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SwitchBillingRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.HostsGetRequest", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.VMHostsListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.VMInfoInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MonitoredResourceListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MarketplaceSaaSInfo", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.PartnerBillingEntity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourcesListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.TagRuleListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.LogRules", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdateInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdatePropertiesInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionPropertiesList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SubscriptionList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.LinkedResourceListResponse", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.fluent.models.VMExtensionPayloadInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MonitorsSwitchBillingHeaders", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.Origin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ActionType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MonitoringStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.MarketplaceSubscriptionStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SingleSignOnStates", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.UsageType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.BillingCycle", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.LiftrResourceCategories", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.BillingSource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.TagAction", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SendingMetricsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SendingLogsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.PatchOperation", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.Status", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.newrelicobservability.models.ConfigurationName", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListSamples.java index 3d96f7c3ac164..0fa7cfb503cf8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Accounts List. */ +/** + * Samples for Accounts List. + */ public final class AccountsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Accounts_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Accounts_List_MinimumSet_Gen.json */ /** * Sample code: Accounts_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void accountsListMinimumSetGen( @@ -20,11 +23,12 @@ public static void accountsListMinimumSetGen( } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Accounts_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Accounts_List_MaximumSet_Gen.json */ /** * Sample code: Accounts_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void accountsListMaximumSetGen( diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoGetSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoGetSamples.java new file mode 100644 index 0000000000000..e9162f8cee186 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoGetSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +/** + * Samples for BillingInfo Get. + */ +public final class BillingInfoGetSamples { + /* + * x-ms-original-file: + * specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/BillingInfo_Get.json + */ + /** + * Sample code: BillingInfo_Get. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void + billingInfoGet(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.billingInfoes().getWithResponse("myResourceGroup", "myMonitor", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListSamples.java new file mode 100644 index 0000000000000..3d97174a6caa6 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +/** + * Samples for ConnectedPartnerResources List. + */ +public final class ConnectedPartnerResourcesListSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * ConnectedPartnerResources_List.json + */ + /** + * Sample code: ConnectedPartnerResources_List. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void connectedPartnerResourcesList( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.connectedPartnerResources().list("myResourceGroup", "myMonitor", null, + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateSamples.java new file mode 100644 index 0000000000000..81a98072e16cd --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateSamples.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * Samples for MonitoredSubscriptions CreateorUpdate. + */ +public final class MonitoredSubscriptionsCreateorUpdateSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_CreateorUpdate.json + */ + /** + * Sample code: Monitors_AddMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsAddMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().define(ConfigurationName.DEFAULT) + .withExistingMonitor("myResourceGroup", "myMonitor").create(); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsDeleteSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsDeleteSamples.java new file mode 100644 index 0000000000000..5103c32c3b5f0 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsDeleteSamples.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * Samples for MonitoredSubscriptions Delete. + */ +public final class MonitoredSubscriptionsDeleteSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_Delete.json + */ + /** + * Sample code: Monitors_DeleteMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsDeleteMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().delete("myResourceGroup", "myMonitor", ConfigurationName.DEFAULT, + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetSamples.java new file mode 100644 index 0000000000000..423fffd5efeb3 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetSamples.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; + +/** + * Samples for MonitoredSubscriptions Get. + */ +public final class MonitoredSubscriptionsGetSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_Get.json + */ + /** + * Sample code: Monitors_GetMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsGetMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().getWithResponse("myResourceGroup", "myMonitor", ConfigurationName.DEFAULT, + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListSamples.java new file mode 100644 index 0000000000000..b791d83fb4e85 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +/** + * Samples for MonitoredSubscriptions List. + */ +public final class MonitoredSubscriptionsListSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_List.json + */ + /** + * Sample code: Monitors_GetMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsGetMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitoredSubscriptions().list("myResourceGroup", "myMonitor", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsUpdateSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsUpdateSamples.java new file mode 100644 index 0000000000000..a1f305680bbc5 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsUpdateSamples.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; + +/** + * Samples for MonitoredSubscriptions Update. + */ +public final class MonitoredSubscriptionsUpdateSamples { + /* + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * MonitoredSubscriptions_Update.json + */ + /** + * Sample code: Monitors_UpdateMonitoredSubscriptions. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsUpdateMonitoredSubscriptions( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + MonitoredSubscriptionProperties resource = manager.monitoredSubscriptions().getWithResponse("myResourceGroup", + "myMonitor", ConfigurationName.DEFAULT, com.azure.core.util.Context.NONE).getValue(); + resource.update().apply(); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateSamples.java index ebcfdab1e8113..4b3326d538692 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateSamples.java @@ -7,6 +7,8 @@ import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; import com.azure.resourcemanager.newrelicobservability.models.AccountInfo; import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; +import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; +import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; import com.azure.resourcemanager.newrelicobservability.models.NewRelicAccountProperties; import com.azure.resourcemanager.newrelicobservability.models.NewRelicSingleSignOnProperties; import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; @@ -15,62 +17,48 @@ import com.azure.resourcemanager.newrelicobservability.models.ProvisioningState; import com.azure.resourcemanager.newrelicobservability.models.SingleSignOnStates; import com.azure.resourcemanager.newrelicobservability.models.UsageType; +import com.azure.resourcemanager.newrelicobservability.models.UserAssignedIdentity; import com.azure.resourcemanager.newrelicobservability.models.UserInfo; import java.time.OffsetDateTime; import java.util.HashMap; import java.util.Map; -/** Samples for Monitors CreateOrUpdate. */ +/** + * Samples for Monitors CreateOrUpdate. + */ public final class MonitorsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_CreateOrUpdate_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_CreateOrUpdate_MaximumSet_Gen.json */ /** * Sample code: Monitors_CreateOrUpdate_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsCreateOrUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .define("cdlymktqw") - .withRegion("k") - .withExistingResourceGroup("rgNewRelic") - .withTags(mapOf("key6976", "oaxfhf")) - .withNewRelicAccountProperties( - new NewRelicAccountProperties() - .withUserId("vcscxlncofcuduadesd") - .withAccountInfo( - new AccountInfo() - .withAccountId("xhqmg") - .withIngestionKey("fakeTokenPlaceholder") - .withRegion("ljcf")) - .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) - .withSingleSignOnProperties( - new NewRelicSingleSignOnProperties() - .withSingleSignOnState(SingleSignOnStates.INITIAL) - .withEnterpriseAppId("kwiwfz") - .withSingleSignOnUrl("kvseueuljsxmfwpqctz") - .withProvisioningState(ProvisioningState.ACCEPTED))) - .withUserInfo( - new UserInfo() - .withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") - .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc") - .withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") - .withPhoneNumber("krf") - .withCountry("hslqnwdanrconqyekwbnttaetv")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("tbbiaga") - .withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.LIFTR) - .create(); + manager.monitors().define("cdlymktqw").withRegion("k").withExistingResourceGroup("rgNewRelic") + .withTags(mapOf("key6976", "fakeTokenPlaceholder")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("key8903", new UserAssignedIdentity()))) + .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("vcscxlncofcuduadesd") + .withAccountInfo(new AccountInfo() + .withAccountId("xhqmg").withIngestionKey("fakeTokenPlaceholder").withRegion("ljcf")) + .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) + .withSingleSignOnProperties(new NewRelicSingleSignOnProperties() + .withSingleSignOnState(SingleSignOnStates.INITIAL).withEnterpriseAppId("kwiwfz") + .withSingleSignOnUrl("kvseueuljsxmfwpqctz").withProvisioningState(ProvisioningState.ACCEPTED))) + .withUserInfo(new UserInfo().withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") + .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc").withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") + .withPhoneNumber("krf").withCountry("hslqnwdanrconqyekwbnttaetv")) + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.YEARLY) + .withPlanDetails("tbbiaga").withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) + .withOrgCreationSource(OrgCreationSource.LIFTR).withAccountCreationSource(AccountCreationSource.LIFTR) + .withSubscriptionState("Suspended").withSaaSAzureSubscriptionStatus("Subscribed").create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteSamples.java index df1bcce4f1524..c33964d683a6f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteSamples.java @@ -4,43 +4,37 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Monitors Delete. */ +/** + * Samples for Monitors Delete. + */ public final class MonitorsDeleteSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Delete_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Delete_MinimumSet_Gen.json */ /** * Sample code: Monitors_Delete_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsDeleteMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .delete( - "rgopenapi", - "ruxvg@xqkmdhrnoo.hlmbpm", - "ipxmlcbonyxtolzejcjshkmlron", - com.azure.core.util.Context.NONE); + manager.monitors().delete("rgopenapi", "ruxvg@xqkmdhrnoo.hlmbpm", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Delete_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Delete_MaximumSet_Gen.json */ /** * Sample code: Monitors_Delete_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsDeleteMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .delete( - "rgopenapi", - "ruxvg@xqkmdhrnoo.hlmbpm", - "ipxmlcbonyxtolzejcjshkmlron", - com.azure.core.util.Context.NONE); + manager.monitors().delete("rgopenapi", "ruxvg@xqkmdhrnoo.hlmbpm", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupSamples.java index f3c289c7d18b1..caf1a6615ccbf 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Monitors GetByResourceGroup. */ +/** + * Samples for Monitors GetByResourceGroup. + */ public final class MonitorsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Get_MaximumSet_Gen.json */ /** * Sample code: Monitors_Get_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void monitorsGetMaximumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + public static void + monitorsGetMaximumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { manager.monitors().getByResourceGroupWithResponse("rgNewRelic", "cdlymktqw", com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesSamples.java index 4c135009373f7..a7925763fca43 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesSamples.java @@ -6,43 +6,37 @@ import com.azure.resourcemanager.newrelicobservability.models.MetricsRequest; -/** Samples for Monitors GetMetricRules. */ +/** + * Samples for Monitors GetMetricRules. + */ public final class MonitorsGetMetricRulesSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricRules_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricRules_MinimumSet_Gen.json */ /** * Sample code: Monitors_GetMetricRules_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricRulesMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricRulesWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricRulesWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricRules_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricRules_MaximumSet_Gen.json */ /** * Sample code: Monitors_GetMetricRules_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricRulesMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricRulesWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricRulesWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusSamples.java index 729dd907b15ba..1e76d589c2e1c 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusSamples.java @@ -7,45 +7,43 @@ import com.azure.resourcemanager.newrelicobservability.models.MetricsStatusRequest; import java.util.Arrays; -/** Samples for Monitors GetMetricStatus. */ +/** + * Samples for Monitors GetMetricStatus. + */ public final class MonitorsGetMetricStatusSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricStatus_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricStatus_MinimumSet_Gen.json */ /** * Sample code: Monitors_GetMetricStatus_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricStatusMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricStatusWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsStatusRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricStatusWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsStatusRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_GetMetricStatus_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_GetMetricStatus_MaximumSet_Gen.json */ /** * Sample code: Monitors_GetMetricStatus_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsGetMetricStatusMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .getMetricStatusWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new MetricsStatusRequest() - .withAzureResourceIds(Arrays.asList("enfghpfw")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().getMetricStatusWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new MetricsStatusRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesSamples.java index 70fd06e09d501..4a2902ed55258 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesSamples.java @@ -7,45 +7,43 @@ import com.azure.resourcemanager.newrelicobservability.models.AppServicesGetRequest; import java.util.Arrays; -/** Samples for Monitors ListAppServices. */ +/** + * Samples for Monitors ListAppServices. + */ public final class MonitorsListAppServicesSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListAppServices_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListAppServices_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListAppServices_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListAppServicesMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listAppServices( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new AppServicesGetRequest() - .withAzureResourceIds(Arrays.asList("pvzrksrmzowobuhxpwiotnpcvjbu")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listAppServices("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new AppServicesGetRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListAppServices_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListAppServices_MinimumSet_Gen.json */ /** * Sample code: Monitors_ListAppServices_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListAppServicesMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listAppServices( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new AppServicesGetRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listAppServices("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new AppServicesGetRequest().withAzureResourceIds(Arrays.asList( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz")) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupSamples.java index 69c7e4adf0dd5..f32f6675d0473 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Monitors ListByResourceGroup. */ +/** + * Samples for Monitors ListByResourceGroup. + */ public final class MonitorsListByResourceGroupSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListByResourceGroup_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListByResourceGroup_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListByResourceGroup_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListByResourceGroupMaximumSetGen( diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsSamples.java index d5875b2680b4a..6dc202b91fe24 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsSamples.java @@ -7,47 +7,41 @@ import com.azure.resourcemanager.newrelicobservability.models.HostsGetRequest; import java.util.Arrays; -/** Samples for Monitors ListHosts. */ +/** + * Samples for Monitors ListHosts. + */ public final class MonitorsListHostsSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListHosts_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListHosts_MinimumSet_Gen.json */ /** * Sample code: Monitors_ListHosts_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListHostsMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listHosts( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - new HostsGetRequest() - .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listHosts( + "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", new HostsGetRequest() + .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")).withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListHosts_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListHosts_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListHosts_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListHostsMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listHosts( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - new HostsGetRequest() - .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().listHosts( + "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", new HostsGetRequest() + .withVmIds(Arrays.asList("xzphvxvfmvjrnsgyns")).withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesSamples.java new file mode 100644 index 0000000000000..7010b26e26bdd --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +/** + * Samples for Monitors ListLinkedResources. + */ +public final class MonitorsListLinkedResourcesSamples { + /* + * x-ms-original-file: + * specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/LinkedResources_List. + * json + */ + /** + * Sample code: Monitors_ListLinkedResources. + * + * @param manager Entry point to NewRelicObservabilityManager. + */ + public static void monitorsListLinkedResources( + com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.monitors().listLinkedResources("myResourceGroup", "myMonitor", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesSamples.java index 7e2390ebe2b13..812b01ef5e06f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesSamples.java @@ -4,35 +4,37 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Monitors ListMonitoredResources. */ +/** + * Samples for Monitors ListMonitoredResources. + */ public final class MonitorsListMonitoredResourcesSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListMonitoredResources_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListMonitoredResources_MinimumSet_Gen.json */ /** * Sample code: Monitors_ListMonitoredResources_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListMonitoredResourcesMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListMonitoredResources_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListMonitoredResources_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListMonitoredResources_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListMonitoredResourcesMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().listMonitoredResources("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListSamples.java index ebbf77092329d..d4462aa2861c6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Monitors List. */ +/** + * Samples for Monitors List. + */ public final class MonitorsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_ListBySubscription_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_ListBySubscription_MaximumSet_Gen.json */ /** * Sample code: Monitors_ListBySubscription_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsListBySubscriptionMaximumSetGen( diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingSamples.java index 8ce20bd6d228e..a5102e651ba22 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingSamples.java @@ -10,52 +10,43 @@ import com.azure.resourcemanager.newrelicobservability.models.UsageType; import java.time.OffsetDateTime; -/** Samples for Monitors SwitchBilling. */ +/** + * Samples for Monitors SwitchBilling. + */ public final class MonitorsSwitchBillingSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_SwitchBilling_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_SwitchBilling_MinimumSet_Gen.json */ /** * Sample code: Monitors_SwitchBilling_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsSwitchBillingMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .switchBillingWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new SwitchBillingRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().switchBillingWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new SwitchBillingRequest().withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_SwitchBilling_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_SwitchBilling_MaximumSet_Gen.json */ /** * Sample code: Monitors_SwitchBilling_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsSwitchBillingMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .switchBillingWithResponse( - "rgNewRelic", - "fhcjxnxumkdlgpwanewtkdnyuz", - new SwitchBillingRequest() - .withAzureResourceId("enfghpfw") - .withOrganizationId("k") - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("tbbiaga") - .withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) - .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), - com.azure.core.util.Context.NONE); + manager.monitors().switchBillingWithResponse("rgNewRelic", "fhcjxnxumkdlgpwanewtkdnyuz", + new SwitchBillingRequest().withAzureResourceId( + "/subscriptions/00000000-0000-0000-0000-000000000000/resourceGroups/rgNewRelic/providers/NewRelic.Observability/monitors/fhcjxnxumkdlgpwanewtkdnyuz") + .withOrganizationId("k") + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.YEARLY) + .withPlanDetails("tbbiaga").withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) + .withUserEmail("ruxvg@xqkmdhrnoo.hlmbpm"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsUpdateSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsUpdateSamples.java index 59cdb1788e217..a39b88575f5b1 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsUpdateSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsUpdateSamples.java @@ -24,63 +24,43 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Monitors Update. */ +/** + * Samples for Monitors Update. + */ public final class MonitorsUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_Update_MaximumSet_Gen.json */ /** * Sample code: Monitors_Update_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - NewRelicMonitorResource resource = - manager - .monitors() - .getByResourceGroupWithResponse("rgNewRelic", "cdlymktqw", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("key164", "jqakdrrmmyzytqu")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf("key8903", new UserAssignedIdentity()))) - .withNewRelicAccountProperties( - new NewRelicAccountProperties() - .withUserId("vcscxlncofcuduadesd") - .withAccountInfo( - new AccountInfo() - .withAccountId("xhqmg") - .withIngestionKey("fakeTokenPlaceholder") - .withRegion("ljcf")) - .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) - .withSingleSignOnProperties( - new NewRelicSingleSignOnProperties() - .withSingleSignOnState(SingleSignOnStates.INITIAL) - .withEnterpriseAppId("kwiwfz") - .withSingleSignOnUrl("kvseueuljsxmfwpqctz") - .withProvisioningState(ProvisioningState.ACCEPTED))) - .withUserInfo( - new UserInfo() - .withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") - .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc") - .withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") - .withPhoneNumber("krf") - .withCountry("hslqnwdanrconqyekwbnttaetv")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("tbbiaga") - .withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.LIFTR) + NewRelicMonitorResource resource = manager.monitors() + .getByResourceGroupWithResponse("rgNewRelic", "cdlymktqw", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("key164", "fakeTokenPlaceholder")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("key8903", new UserAssignedIdentity()))) + .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("vcscxlncofcuduadesd") + .withAccountInfo(new AccountInfo() + .withAccountId("xhqmg").withIngestionKey("fakeTokenPlaceholder").withRegion("ljcf")) + .withOrganizationInfo(new OrganizationInfo().withOrganizationId("k")) + .withSingleSignOnProperties(new NewRelicSingleSignOnProperties() + .withSingleSignOnState(SingleSignOnStates.INITIAL).withEnterpriseAppId("kwiwfz") + .withSingleSignOnUrl("kvseueuljsxmfwpqctz").withProvisioningState(ProvisioningState.ACCEPTED))) + .withUserInfo(new UserInfo().withFirstName("vdftzcggirefejajwahhwhyibutramdaotvnuf") + .withLastName("bcsztgqovdlmzfkjdrngidwzqsevagexzzilnlc").withEmailAddress("%6%@4-g.N1.3F-kI1.Ue-.lJso") + .withPhoneNumber("krf").withCountry("hslqnwdanrconqyekwbnttaetv")) + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.YEARLY) + .withPlanDetails("tbbiaga").withEffectiveDate(OffsetDateTime.parse("2022-12-05T14:11:37.786Z"))) + .withOrgCreationSource(OrgCreationSource.LIFTR).withAccountCreationSource(AccountCreationSource.LIFTR) .apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsVmHostPayloadSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsVmHostPayloadSamples.java index 36c4a80ae49f9..2c3ffa0e82754 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsVmHostPayloadSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsVmHostPayloadSamples.java @@ -4,35 +4,37 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Monitors VmHostPayload. */ +/** + * Samples for Monitors VmHostPayload. + */ public final class MonitorsVmHostPayloadSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_VmHostPayload_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_VmHostPayload_MinimumSet_Gen.json */ /** * Sample code: Monitors_VmHostPayload_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsVmHostPayloadMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Monitors_VmHostPayload_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Monitors_VmHostPayload_MaximumSet_Gen.json */ /** * Sample code: Monitors_VmHostPayload_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void monitorsVmHostPayloadMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .monitors() - .vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.monitors().vmHostPayloadWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListSamples.java index 40771394c2e95..1f94f848d7c10 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Operations_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Operations_List_MinimumSet_Gen.json */ /** * Sample code: Operations_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void operationsListMinimumSetGen( @@ -20,11 +23,12 @@ public static void operationsListMinimumSetGen( } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Operations_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Operations_List_MaximumSet_Gen.json */ /** * Sample code: Operations_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void operationsListMaximumSetGen( diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListSamples.java index bd4d2591aa4be..99ba2b9bf7f6f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Organizations List. */ +/** + * Samples for Organizations List. + */ public final class OrganizationsListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Organizations_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Organizations_List_MinimumSet_Gen.json */ /** * Sample code: Organizations_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void organizationsListMinimumSetGen( @@ -20,11 +23,12 @@ public static void organizationsListMinimumSetGen( } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Organizations_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Organizations_List_MaximumSet_Gen.json */ /** * Sample code: Organizations_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void organizationsListMaximumSetGen( diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListSamples.java index 798b3ae411d56..4d834629fc246 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListSamples.java @@ -4,31 +4,35 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for Plans List. */ +/** + * Samples for Plans List. + */ public final class PlansListSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Plans_List_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Plans_List_MaximumSet_Gen.json */ /** * Sample code: Plans_List_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void plansListMaximumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + public static void + plansListMaximumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { manager.plans().list("pwuxgvrmkk", "hilawwjz", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/Plans_List_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * Plans_List_MinimumSet_Gen.json */ /** * Sample code: Plans_List_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void plansListMinimumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager.plans().list(null, null, com.azure.core.util.Context.NONE); + public static void + plansListMinimumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.plans().list(null, "hilawwjz", com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateSamples.java index 779d3a5818e47..855790e35c59a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateSamples.java @@ -13,61 +13,47 @@ import com.azure.resourcemanager.newrelicobservability.models.TagAction; import java.util.Arrays; -/** Samples for TagRules CreateOrUpdate. */ +/** + * Samples for TagRules CreateOrUpdate. + */ public final class TagRulesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_CreateOrUpdate_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_CreateOrUpdate_MaximumSet_Gen.json */ /** * Sample code: TagRules_CreateOrUpdate_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesCreateOrUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .define("bxcantgzggsepbhqmedjqyrqeezmfb") + manager.tagRules().define("bxcantgzggsepbhqmedjqyrqeezmfb") .withExistingMonitor("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron") - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.ENABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE)))) - .withMetricRules( - new MetricRulesInner() - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE))) - .withUserEmail("test@testing.com")) + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner() + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE))) + .withUserEmail("test@testing.com")) .create(); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_CreateOrUpdate_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_CreateOrUpdate_MinimumSet_Gen.json */ /** * Sample code: TagRules_CreateOrUpdate_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesCreateOrUpdateMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .define("bxcantgzggsepbhqmedjqyrqeezmfb") - .withExistingMonitor("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron") - .create(); + manager.tagRules().define("bxcantgzggsepbhqmedjqyrqeezmfb") + .withExistingMonitor("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron").create(); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteSamples.java index bf2ee37a5c51f..0de870c4ee262 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteSamples.java @@ -4,43 +4,37 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for TagRules Delete. */ +/** + * Samples for TagRules Delete. + */ public final class TagRulesDeleteSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Delete_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Delete_MinimumSet_Gen.json */ /** * Sample code: TagRules_Delete_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesDeleteMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .delete( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + manager.tagRules().delete("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Delete_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Delete_MaximumSet_Gen.json */ /** * Sample code: TagRules_Delete_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesDeleteMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .delete( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + manager.tagRules().delete("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetSamples.java index 4dde5c5ce9f91..1fc8dbcdce8e3 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetSamples.java @@ -4,43 +4,37 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for TagRules Get. */ +/** + * Samples for TagRules Get. + */ public final class TagRulesGetSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Get_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Get_MinimumSet_Gen.json */ /** * Sample code: TagRules_Get_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void tagRulesGetMinimumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + public static void + tagRulesGetMinimumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Get_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Get_MaximumSet_Gen.json */ /** * Sample code: TagRules_Get_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ - public static void tagRulesGetMaximumSetGen( - com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE); + public static void + tagRulesGetMaximumSetGen(com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { + manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", "bxcantgzggsepbhqmedjqyrqeezmfb", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceSamples.java index b28c3e3c9fec0..db66fbf3d26a6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceSamples.java @@ -4,37 +4,37 @@ package com.azure.resourcemanager.newrelicobservability.generated; -/** Samples for TagRules ListByNewRelicMonitorResource. */ +/** + * Samples for TagRules ListByNewRelicMonitorResource. + */ public final class TagRulesListByNewRelicMonitorResourceSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_ListByNewRelicMonitorResource_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_ListByNewRelicMonitorResource_MinimumSet_Gen.json */ /** * Sample code: TagRules_ListByNewRelicMonitorResource_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesListByNewRelicMonitorResourceMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .listByNewRelicMonitorResource( - "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.tagRules().listByNewRelicMonitorResource("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_ListByNewRelicMonitorResource_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_ListByNewRelicMonitorResource_MaximumSet_Gen.json */ /** * Sample code: TagRules_ListByNewRelicMonitorResource_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesListByNewRelicMonitorResourceMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - manager - .tagRules() - .listByNewRelicMonitorResource( - "rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", com.azure.core.util.Context.NONE); + manager.tagRules().listByNewRelicMonitorResource("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesUpdateSamples.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesUpdateSamples.java index 1aad4f817a39f..13b4756ffe6d8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesUpdateSamples.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/samples/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesUpdateSamples.java @@ -14,73 +14,49 @@ import com.azure.resourcemanager.newrelicobservability.models.TagRule; import java.util.Arrays; -/** Samples for TagRules Update. */ +/** + * Samples for TagRules Update. + */ public final class TagRulesUpdateSamples { /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Update_MaximumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Update_MaximumSet_Gen.json */ /** * Sample code: TagRules_Update_MaximumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesUpdateMaximumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - TagRule resource = - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.ENABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE)))) - .withMetricRules( - new MetricRulesInner() - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("saokgpjvdlorciqbjmjxazpee") - .withValue("sarxrqsxouhdjwsrqqicbeirdb") - .withAction(TagAction.INCLUDE))) - .withUserEmail("test@testing.com")) + TagRule resource = manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + "bxcantgzggsepbhqmedjqyrqeezmfb", com.azure.core.util.Context.NONE).getValue(); + resource.update() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner() + .withFilteringTags(Arrays.asList(new FilteringTag().withName("saokgpjvdlorciqbjmjxazpee") + .withValue("sarxrqsxouhdjwsrqqicbeirdb").withAction(TagAction.INCLUDE))) + .withUserEmail("test@testing.com")) .apply(); } /* - * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2022-07-01/examples/TagRules_Update_MinimumSet_Gen.json + * x-ms-original-file: specification/newrelic/resource-manager/NewRelic.Observability/stable/2024-01-01/examples/ + * TagRules_Update_MinimumSet_Gen.json */ /** * Sample code: TagRules_Update_MinimumSet_Gen. - * + * * @param manager Entry point to NewRelicObservabilityManager. */ public static void tagRulesUpdateMinimumSetGen( com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager manager) { - TagRule resource = - manager - .tagRules() - .getWithResponse( - "rgopenapi", - "ipxmlcbonyxtolzejcjshkmlron", - "bxcantgzggsepbhqmedjqyrqeezmfb", - com.azure.core.util.Context.NONE) - .getValue(); + TagRule resource = manager.tagRules().getWithResponse("rgopenapi", "ipxmlcbonyxtolzejcjshkmlron", + "bxcantgzggsepbhqmedjqyrqeezmfb", com.azure.core.util.Context.NONE).getValue(); resource.update().apply(); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountPropertiesTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountPropertiesTests.java index 72eef6f3fb19c..132d9876c4ee8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountPropertiesTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountPropertiesTests.java @@ -11,11 +11,9 @@ public final class AccountPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AccountProperties model = - BinaryData - .fromString( - "{\"organizationId\":\"wixjsprozvcp\",\"accountId\":\"eg\",\"accountName\":\"wmfdatscmdvpjhul\",\"region\":\"uvm\"}") - .toObject(AccountProperties.class); + AccountProperties model = BinaryData.fromString( + "{\"organizationId\":\"wixjsprozvcp\",\"accountId\":\"eg\",\"accountName\":\"wmfdatscmdvpjhul\",\"region\":\"uvm\"}") + .toObject(AccountProperties.class); Assertions.assertEquals("wixjsprozvcp", model.organizationId()); Assertions.assertEquals("eg", model.accountId()); Assertions.assertEquals("wmfdatscmdvpjhul", model.accountName()); @@ -24,12 +22,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AccountProperties model = - new AccountProperties() - .withOrganizationId("wixjsprozvcp") - .withAccountId("eg") - .withAccountName("wmfdatscmdvpjhul") - .withRegion("uvm"); + AccountProperties model = new AccountProperties().withOrganizationId("wixjsprozvcp").withAccountId("eg") + .withAccountName("wmfdatscmdvpjhul").withRegion("uvm"); model = BinaryData.fromObject(model).toObject(AccountProperties.class); Assertions.assertEquals("wixjsprozvcp", model.organizationId()); Assertions.assertEquals("eg", model.accountId()); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountResourceInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountResourceInnerTests.java index d48031a8f8193..796c12aba8dc5 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountResourceInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountResourceInnerTests.java @@ -11,11 +11,9 @@ public final class AccountResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AccountResourceInner model = - BinaryData - .fromString( - "{\"properties\":{\"organizationId\":\"s\",\"accountId\":\"nyejhkryhtnap\",\"accountName\":\"wlokjyem\",\"region\":\"vnipjox\"},\"id\":\"nchgej\",\"name\":\"podmailzydehojwy\",\"type\":\"huxinpmqnj\"}") - .toObject(AccountResourceInner.class); + AccountResourceInner model = BinaryData.fromString( + "{\"properties\":{\"organizationId\":\"s\",\"accountId\":\"nyejhkryhtnap\",\"accountName\":\"wlokjyem\",\"region\":\"vnipjox\"},\"id\":\"nchgej\",\"name\":\"podmailzydehojwy\",\"type\":\"huxinpmqnj\"}") + .toObject(AccountResourceInner.class); Assertions.assertEquals("s", model.organizationId()); Assertions.assertEquals("nyejhkryhtnap", model.accountId()); Assertions.assertEquals("wlokjyem", model.accountName()); @@ -24,12 +22,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AccountResourceInner model = - new AccountResourceInner() - .withOrganizationId("s") - .withAccountId("nyejhkryhtnap") - .withAccountName("wlokjyem") - .withRegion("vnipjox"); + AccountResourceInner model = new AccountResourceInner().withOrganizationId("s").withAccountId("nyejhkryhtnap") + .withAccountName("wlokjyem").withRegion("vnipjox"); model = BinaryData.fromObject(model).toObject(AccountResourceInner.class); Assertions.assertEquals("s", model.organizationId()); Assertions.assertEquals("nyejhkryhtnap", model.accountId()); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListMockTests.java index 8e0eb3771d644..00a921ae38e9b 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListMockTests.java @@ -31,41 +31,30 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"organizationId\":\"fclhaaxdbabphlwr\",\"accountId\":\"fkts\",\"accountName\":\"sucocmnyyazttbtw\",\"region\":\"qpuedckzywbiex\"},\"id\":\"eyueaxibxujwb\",\"name\":\"qwalmuzyoxaepd\",\"type\":\"zjancuxr\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"organizationId\":\"bnxknalaulppg\",\"accountId\":\"tpnapnyiropuhpig\",\"accountName\":\"gylgqgitxmedjvcs\",\"region\":\"n\"},\"id\":\"wncwzzhxgktrmg\",\"name\":\"cnapkteoell\",\"type\":\"pt\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.accounts().list("tkcxywnytnrsy", "lqidyby", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("fclhaaxdbabphlwr", response.iterator().next().organizationId()); - Assertions.assertEquals("fkts", response.iterator().next().accountId()); - Assertions.assertEquals("sucocmnyyazttbtw", response.iterator().next().accountName()); - Assertions.assertEquals("qpuedckzywbiex", response.iterator().next().region()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.accounts().list("iidzyexzne", "ixhnrztf", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("bnxknalaulppg", response.iterator().next().organizationId()); + Assertions.assertEquals("tpnapnyiropuhpig", response.iterator().next().accountId()); + Assertions.assertEquals("gylgqgitxmedjvcs", response.iterator().next().accountName()); + Assertions.assertEquals("n", response.iterator().next().region()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListResponseTests.java index d1749734275b7..122541a2e4b6b 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListResponseTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AccountsListResponseTests.java @@ -13,11 +13,9 @@ public final class AccountsListResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AccountsListResponse model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"organizationId\":\"ithxqhabifpi\",\"accountId\":\"wczbys\",\"accountName\":\"pqxu\",\"region\":\"vyq\"},\"id\":\"wby\",\"name\":\"rkxvdum\",\"type\":\"grtfwvu\"}],\"nextLink\":\"gaudcc\"}") - .toObject(AccountsListResponse.class); + AccountsListResponse model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"organizationId\":\"ithxqhabifpi\",\"accountId\":\"wczbys\",\"accountName\":\"pqxu\",\"region\":\"vyq\"},\"id\":\"wby\",\"name\":\"rkxvdum\",\"type\":\"grtfwvu\"}],\"nextLink\":\"gaudcc\"}") + .toObject(AccountsListResponse.class); Assertions.assertEquals("ithxqhabifpi", model.value().get(0).organizationId()); Assertions.assertEquals("wczbys", model.value().get(0).accountId()); Assertions.assertEquals("pqxu", model.value().get(0).accountName()); @@ -27,17 +25,9 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AccountsListResponse model = - new AccountsListResponse() - .withValue( - Arrays - .asList( - new AccountResourceInner() - .withOrganizationId("ithxqhabifpi") - .withAccountId("wczbys") - .withAccountName("pqxu") - .withRegion("vyq"))) - .withNextLink("gaudcc"); + AccountsListResponse model = new AccountsListResponse().withValue(Arrays.asList(new AccountResourceInner() + .withOrganizationId("ithxqhabifpi").withAccountId("wczbys").withAccountName("pqxu").withRegion("vyq"))) + .withNextLink("gaudcc"); model = BinaryData.fromObject(model).toObject(AccountsListResponse.class); Assertions.assertEquals("ithxqhabifpi", model.value().get(0).organizationId()); Assertions.assertEquals("wczbys", model.value().get(0).accountId()); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServiceInfoInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServiceInfoInnerTests.java index 8b390223fdc4f..27583fe61d9ff 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServiceInfoInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServiceInfoInnerTests.java @@ -11,26 +11,21 @@ public final class AppServiceInfoInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AppServiceInfoInner model = - BinaryData - .fromString( - "{\"azureResourceId\":\"azjdyggd\",\"agentVersion\":\"ixhbkuofqweykhm\",\"agentStatus\":\"evfyexfwhybcib\"}") - .toObject(AppServiceInfoInner.class); - Assertions.assertEquals("azjdyggd", model.azureResourceId()); - Assertions.assertEquals("ixhbkuofqweykhm", model.agentVersion()); - Assertions.assertEquals("evfyexfwhybcib", model.agentStatus()); + AppServiceInfoInner model = BinaryData.fromString( + "{\"azureResourceId\":\"zvddntwndeicbtwn\",\"agentVersion\":\"aoqvuh\",\"agentStatus\":\"cffcyddglmj\"}") + .toObject(AppServiceInfoInner.class); + Assertions.assertEquals("zvddntwndeicbtwn", model.azureResourceId()); + Assertions.assertEquals("aoqvuh", model.agentVersion()); + Assertions.assertEquals("cffcyddglmj", model.agentStatus()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AppServiceInfoInner model = - new AppServiceInfoInner() - .withAzureResourceId("azjdyggd") - .withAgentVersion("ixhbkuofqweykhm") - .withAgentStatus("evfyexfwhybcib"); + AppServiceInfoInner model = new AppServiceInfoInner().withAzureResourceId("zvddntwndeicbtwn") + .withAgentVersion("aoqvuh").withAgentStatus("cffcyddglmj"); model = BinaryData.fromObject(model).toObject(AppServiceInfoInner.class); - Assertions.assertEquals("azjdyggd", model.azureResourceId()); - Assertions.assertEquals("ixhbkuofqweykhm", model.agentVersion()); - Assertions.assertEquals("evfyexfwhybcib", model.agentStatus()); + Assertions.assertEquals("zvddntwndeicbtwn", model.azureResourceId()); + Assertions.assertEquals("aoqvuh", model.agentVersion()); + Assertions.assertEquals("cffcyddglmj", model.agentStatus()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesGetRequestTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesGetRequestTests.java index 01035bd93e677..7a9edc9df892a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesGetRequestTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesGetRequestTests.java @@ -12,20 +12,19 @@ public final class AppServicesGetRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AppServicesGetRequest model = - BinaryData - .fromString("{\"azureResourceIds\":[\"dudgwdslfhot\"],\"userEmail\":\"mcy\"}") + AppServicesGetRequest model + = BinaryData.fromString("{\"azureResourceIds\":[\"sglumma\",\"tjaodxobnb\"],\"userEmail\":\"xkqpxo\"}") .toObject(AppServicesGetRequest.class); - Assertions.assertEquals("dudgwdslfhot", model.azureResourceIds().get(0)); - Assertions.assertEquals("mcy", model.userEmail()); + Assertions.assertEquals("sglumma", model.azureResourceIds().get(0)); + Assertions.assertEquals("xkqpxo", model.userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AppServicesGetRequest model = - new AppServicesGetRequest().withAzureResourceIds(Arrays.asList("dudgwdslfhot")).withUserEmail("mcy"); + AppServicesGetRequest model = new AppServicesGetRequest() + .withAzureResourceIds(Arrays.asList("sglumma", "tjaodxobnb")).withUserEmail("xkqpxo"); model = BinaryData.fromObject(model).toObject(AppServicesGetRequest.class); - Assertions.assertEquals("dudgwdslfhot", model.azureResourceIds().get(0)); - Assertions.assertEquals("mcy", model.userEmail()); + Assertions.assertEquals("sglumma", model.azureResourceIds().get(0)); + Assertions.assertEquals("xkqpxo", model.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesListResponseTests.java index 82be020a7c68a..5c9b265cd9863 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesListResponseTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/AppServicesListResponseTests.java @@ -13,37 +13,30 @@ public final class AppServicesListResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - AppServicesListResponse model = - BinaryData - .fromString( - "{\"value\":[{\"azureResourceId\":\"lbjnpgacftadehx\",\"agentVersion\":\"tyfsoppusuesn\",\"agentStatus\":\"dejbavo\"},{\"azureResourceId\":\"zdmohctbqvu\",\"agentVersion\":\"xdn\",\"agentStatus\":\"vo\"}],\"nextLink\":\"ujjugwdkcglh\"}") - .toObject(AppServicesListResponse.class); - Assertions.assertEquals("lbjnpgacftadehx", model.value().get(0).azureResourceId()); - Assertions.assertEquals("tyfsoppusuesn", model.value().get(0).agentVersion()); - Assertions.assertEquals("dejbavo", model.value().get(0).agentStatus()); - Assertions.assertEquals("ujjugwdkcglh", model.nextLink()); + AppServicesListResponse model = BinaryData.fromString( + "{\"value\":[{\"azureResourceId\":\"ionpimexg\",\"agentVersion\":\"xgcp\",\"agentStatus\":\"gmaajrm\"},{\"azureResourceId\":\"jwzrl\",\"agentVersion\":\"mcl\",\"agentStatus\":\"ijcoejctb\"},{\"azureResourceId\":\"qsqsy\",\"agentVersion\":\"kbfkg\",\"agentStatus\":\"dkexxppofm\"},{\"azureResourceId\":\"x\",\"agentVersion\":\"jpgd\",\"agentStatus\":\"ocjjxhvpmouexh\"}],\"nextLink\":\"xibqeojnx\"}") + .toObject(AppServicesListResponse.class); + Assertions.assertEquals("ionpimexg", model.value().get(0).azureResourceId()); + Assertions.assertEquals("xgcp", model.value().get(0).agentVersion()); + Assertions.assertEquals("gmaajrm", model.value().get(0).agentStatus()); + Assertions.assertEquals("xibqeojnx", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - AppServicesListResponse model = - new AppServicesListResponse() - .withValue( - Arrays - .asList( - new AppServiceInfoInner() - .withAzureResourceId("lbjnpgacftadehx") - .withAgentVersion("tyfsoppusuesn") - .withAgentStatus("dejbavo"), - new AppServiceInfoInner() - .withAzureResourceId("zdmohctbqvu") - .withAgentVersion("xdn") - .withAgentStatus("vo"))) - .withNextLink("ujjugwdkcglh"); + AppServicesListResponse model = new AppServicesListResponse().withValue(Arrays.asList( + new AppServiceInfoInner().withAzureResourceId("ionpimexg").withAgentVersion("xgcp") + .withAgentStatus("gmaajrm"), + new AppServiceInfoInner().withAzureResourceId("jwzrl").withAgentVersion("mcl").withAgentStatus("ijcoejctb"), + new AppServiceInfoInner().withAzureResourceId("qsqsy").withAgentVersion("kbfkg") + .withAgentStatus("dkexxppofm"), + new AppServiceInfoInner().withAzureResourceId("x").withAgentVersion("jpgd") + .withAgentStatus("ocjjxhvpmouexh"))) + .withNextLink("xibqeojnx"); model = BinaryData.fromObject(model).toObject(AppServicesListResponse.class); - Assertions.assertEquals("lbjnpgacftadehx", model.value().get(0).azureResourceId()); - Assertions.assertEquals("tyfsoppusuesn", model.value().get(0).agentVersion()); - Assertions.assertEquals("dejbavo", model.value().get(0).agentStatus()); - Assertions.assertEquals("ujjugwdkcglh", model.nextLink()); + Assertions.assertEquals("ionpimexg", model.value().get(0).azureResourceId()); + Assertions.assertEquals("xgcp", model.value().get(0).agentVersion()); + Assertions.assertEquals("gmaajrm", model.value().get(0).agentStatus()); + Assertions.assertEquals("xibqeojnx", model.nextLink()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoResponseInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoResponseInnerTests.java new file mode 100644 index 0000000000000..ac913034b5fbe --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoResponseInnerTests.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.BillingInfoResponseInner; +import com.azure.resourcemanager.newrelicobservability.models.MarketplaceSaaSInfo; +import com.azure.resourcemanager.newrelicobservability.models.PartnerBillingEntity; +import org.junit.jupiter.api.Assertions; + +public final class BillingInfoResponseInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + BillingInfoResponseInner model = BinaryData.fromString( + "{\"marketplaceSaasInfo\":{\"marketplaceSubscriptionId\":\"eo\",\"marketplaceSubscriptionName\":\"okeyyienj\",\"marketplaceResourceId\":\"lwtgrhpdj\",\"marketplaceStatus\":\"umasxazjpq\",\"billedAzureSubscriptionId\":\"gual\"},\"partnerBillingEntity\":{\"organizationId\":\"xhejjzzvdud\",\"organizationName\":\"dslfhotwmcy\"}}") + .toObject(BillingInfoResponseInner.class); + Assertions.assertEquals("eo", model.marketplaceSaasInfo().marketplaceSubscriptionId()); + Assertions.assertEquals("okeyyienj", model.marketplaceSaasInfo().marketplaceSubscriptionName()); + Assertions.assertEquals("lwtgrhpdj", model.marketplaceSaasInfo().marketplaceResourceId()); + Assertions.assertEquals("umasxazjpq", model.marketplaceSaasInfo().marketplaceStatus()); + Assertions.assertEquals("gual", model.marketplaceSaasInfo().billedAzureSubscriptionId()); + Assertions.assertEquals("xhejjzzvdud", model.partnerBillingEntity().organizationId()); + Assertions.assertEquals("dslfhotwmcy", model.partnerBillingEntity().organizationName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + BillingInfoResponseInner model + = new BillingInfoResponseInner() + .withMarketplaceSaasInfo(new MarketplaceSaaSInfo().withMarketplaceSubscriptionId("eo") + .withMarketplaceSubscriptionName("okeyyienj").withMarketplaceResourceId("lwtgrhpdj") + .withMarketplaceStatus("umasxazjpq").withBilledAzureSubscriptionId("gual")) + .withPartnerBillingEntity( + new PartnerBillingEntity().withOrganizationId("xhejjzzvdud").withOrganizationName("dslfhotwmcy")); + model = BinaryData.fromObject(model).toObject(BillingInfoResponseInner.class); + Assertions.assertEquals("eo", model.marketplaceSaasInfo().marketplaceSubscriptionId()); + Assertions.assertEquals("okeyyienj", model.marketplaceSaasInfo().marketplaceSubscriptionName()); + Assertions.assertEquals("lwtgrhpdj", model.marketplaceSaasInfo().marketplaceResourceId()); + Assertions.assertEquals("umasxazjpq", model.marketplaceSaasInfo().marketplaceStatus()); + Assertions.assertEquals("gual", model.marketplaceSaasInfo().billedAzureSubscriptionId()); + Assertions.assertEquals("xhejjzzvdud", model.partnerBillingEntity().organizationId()); + Assertions.assertEquals("dslfhotwmcy", model.partnerBillingEntity().organizationName()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoesGetWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoesGetWithResponseMockTests.java new file mode 100644 index 0000000000000..7af2eaa89e6bd --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/BillingInfoesGetWithResponseMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; +import com.azure.resourcemanager.newrelicobservability.models.BillingInfoResponse; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class BillingInfoesGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"marketplaceSaasInfo\":{\"marketplaceSubscriptionId\":\"mkcxozapvh\",\"marketplaceSubscriptionName\":\"xprglyatddc\",\"marketplaceResourceId\":\"bcuejrjxgci\",\"marketplaceStatus\":\"brh\",\"billedAzureSubscriptionId\":\"xsdqrhzoymibmrqy\"},\"partnerBillingEntity\":{\"organizationId\":\"hwflu\",\"organizationName\":\"dtmhrkwofyyvoqa\"}}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + BillingInfoResponse response = manager.billingInfoes() + .getWithResponse("gkopkwhojvpajqgx", "smocmbq", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("mkcxozapvh", response.marketplaceSaasInfo().marketplaceSubscriptionId()); + Assertions.assertEquals("xprglyatddc", response.marketplaceSaasInfo().marketplaceSubscriptionName()); + Assertions.assertEquals("bcuejrjxgci", response.marketplaceSaasInfo().marketplaceResourceId()); + Assertions.assertEquals("brh", response.marketplaceSaasInfo().marketplaceStatus()); + Assertions.assertEquals("xsdqrhzoymibmrqy", response.marketplaceSaasInfo().billedAzureSubscriptionId()); + Assertions.assertEquals("hwflu", response.partnerBillingEntity().organizationId()); + Assertions.assertEquals("dtmhrkwofyyvoqa", response.partnerBillingEntity().organizationName()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcePropertiesTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcePropertiesTests.java new file mode 100644 index 0000000000000..0ff45eea2894a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcePropertiesTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourceProperties; +import org.junit.jupiter.api.Assertions; + +public final class ConnectedPartnerResourcePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ConnectedPartnerResourceProperties model = BinaryData.fromString( + "{\"accountName\":\"agfuaxbezyiu\",\"accountId\":\"ktwh\",\"azureResourceId\":\"xw\",\"location\":\"wqsmbsur\"}") + .toObject(ConnectedPartnerResourceProperties.class); + Assertions.assertEquals("agfuaxbezyiu", model.accountName()); + Assertions.assertEquals("ktwh", model.accountId()); + Assertions.assertEquals("xw", model.azureResourceId()); + Assertions.assertEquals("wqsmbsur", model.location()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ConnectedPartnerResourceProperties model = new ConnectedPartnerResourceProperties() + .withAccountName("agfuaxbezyiu").withAccountId("ktwh").withAzureResourceId("xw").withLocation("wqsmbsur"); + model = BinaryData.fromObject(model).toObject(ConnectedPartnerResourceProperties.class); + Assertions.assertEquals("agfuaxbezyiu", model.accountName()); + Assertions.assertEquals("ktwh", model.accountId()); + Assertions.assertEquals("xw", model.azureResourceId()); + Assertions.assertEquals("wqsmbsur", model.location()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListFormatInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListFormatInnerTests.java new file mode 100644 index 0000000000000..3041d13cc693d --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListFormatInnerTests.java @@ -0,0 +1,35 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourceProperties; +import org.junit.jupiter.api.Assertions; + +public final class ConnectedPartnerResourcesListFormatInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ConnectedPartnerResourcesListFormatInner model = BinaryData.fromString( + "{\"properties\":{\"accountName\":\"eupfhyhltrpm\",\"accountId\":\"jmcmatuokthfu\",\"azureResourceId\":\"aodsfcpkv\",\"location\":\"dpuozmyz\"}}") + .toObject(ConnectedPartnerResourcesListFormatInner.class); + Assertions.assertEquals("eupfhyhltrpm", model.properties().accountName()); + Assertions.assertEquals("jmcmatuokthfu", model.properties().accountId()); + Assertions.assertEquals("aodsfcpkv", model.properties().azureResourceId()); + Assertions.assertEquals("dpuozmyz", model.properties().location()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ConnectedPartnerResourcesListFormatInner model = new ConnectedPartnerResourcesListFormatInner() + .withProperties(new ConnectedPartnerResourceProperties().withAccountName("eupfhyhltrpm") + .withAccountId("jmcmatuokthfu").withAzureResourceId("aodsfcpkv").withLocation("dpuozmyz")); + model = BinaryData.fromObject(model).toObject(ConnectedPartnerResourcesListFormatInner.class); + Assertions.assertEquals("eupfhyhltrpm", model.properties().accountName()); + Assertions.assertEquals("jmcmatuokthfu", model.properties().accountId()); + Assertions.assertEquals("aodsfcpkv", model.properties().azureResourceId()); + Assertions.assertEquals("dpuozmyz", model.properties().location()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListMockTests.java new file mode 100644 index 0000000000000..95921a1bd289b --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListMockTests.java @@ -0,0 +1,60 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourcesListFormat; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class ConnectedPartnerResourcesListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"accountName\":\"nqxwbp\",\"accountId\":\"ulpiuj\",\"azureResourceId\":\"asipqiio\",\"location\":\"uqerpqlpqwc\"}}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.connectedPartnerResources() + .list("piexpbtgiw", "wo", "nwashrtd", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("nqxwbp", response.iterator().next().properties().accountName()); + Assertions.assertEquals("ulpiuj", response.iterator().next().properties().accountId()); + Assertions.assertEquals("asipqiio", response.iterator().next().properties().azureResourceId()); + Assertions.assertEquals("uqerpqlpqwc", response.iterator().next().properties().location()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListResponseTests.java new file mode 100644 index 0000000000000..807c3cfa111ee --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ConnectedPartnerResourcesListResponseTests.java @@ -0,0 +1,48 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.ConnectedPartnerResourcesListFormatInner; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourceProperties; +import com.azure.resourcemanager.newrelicobservability.models.ConnectedPartnerResourcesListResponse; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class ConnectedPartnerResourcesListResponseTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ConnectedPartnerResourcesListResponse model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"accountName\":\"ggd\",\"accountId\":\"ixhbkuofqweykhm\",\"azureResourceId\":\"evfyexfwhybcib\",\"location\":\"vdcsitynn\"}},{\"properties\":{\"accountName\":\"dectehfiqsc\",\"accountId\":\"ypvhezrkg\",\"azureResourceId\":\"c\",\"location\":\"efovgmk\"}},{\"properties\":{\"accountName\":\"eyyvxyqjpkcat\",\"accountId\":\"ngj\",\"azureResourceId\":\"cczsq\",\"location\":\"hvmdajvnysounq\"}}],\"nextLink\":\"a\"}") + .toObject(ConnectedPartnerResourcesListResponse.class); + Assertions.assertEquals("ggd", model.value().get(0).properties().accountName()); + Assertions.assertEquals("ixhbkuofqweykhm", model.value().get(0).properties().accountId()); + Assertions.assertEquals("evfyexfwhybcib", model.value().get(0).properties().azureResourceId()); + Assertions.assertEquals("vdcsitynn", model.value().get(0).properties().location()); + Assertions.assertEquals("a", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ConnectedPartnerResourcesListResponse model = new ConnectedPartnerResourcesListResponse() + .withValue(Arrays.asList( + new ConnectedPartnerResourcesListFormatInner().withProperties( + new ConnectedPartnerResourceProperties().withAccountName("ggd").withAccountId("ixhbkuofqweykhm") + .withAzureResourceId("evfyexfwhybcib").withLocation("vdcsitynn")), + new ConnectedPartnerResourcesListFormatInner() + .withProperties(new ConnectedPartnerResourceProperties().withAccountName("dectehfiqsc") + .withAccountId("ypvhezrkg").withAzureResourceId("c").withLocation("efovgmk")), + new ConnectedPartnerResourcesListFormatInner() + .withProperties(new ConnectedPartnerResourceProperties().withAccountName("eyyvxyqjpkcat") + .withAccountId("ngj").withAzureResourceId("cczsq").withLocation("hvmdajvnysounq")))) + .withNextLink("a"); + model = BinaryData.fromObject(model).toObject(ConnectedPartnerResourcesListResponse.class); + Assertions.assertEquals("ggd", model.value().get(0).properties().accountName()); + Assertions.assertEquals("ixhbkuofqweykhm", model.value().get(0).properties().accountId()); + Assertions.assertEquals("evfyexfwhybcib", model.value().get(0).properties().azureResourceId()); + Assertions.assertEquals("vdcsitynn", model.value().get(0).properties().location()); + Assertions.assertEquals("a", model.nextLink()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/FilteringTagTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/FilteringTagTests.java index cdb298e9303bd..fb4a30587bf47 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/FilteringTagTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/FilteringTagTests.java @@ -12,22 +12,19 @@ public final class FilteringTagTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - FilteringTag model = - BinaryData - .fromString("{\"name\":\"nqvpkvlrxnje\",\"value\":\"eipheoflokeyy\",\"action\":\"Include\"}") - .toObject(FilteringTag.class); - Assertions.assertEquals("nqvpkvlrxnje", model.name()); - Assertions.assertEquals("eipheoflokeyy", model.value()); - Assertions.assertEquals(TagAction.INCLUDE, model.action()); + FilteringTag model = BinaryData.fromString("{\"name\":\"ajzyul\",\"value\":\"u\",\"action\":\"Exclude\"}") + .toObject(FilteringTag.class); + Assertions.assertEquals("ajzyul", model.name()); + Assertions.assertEquals("u", model.value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.action()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FilteringTag model = - new FilteringTag().withName("nqvpkvlrxnje").withValue("eipheoflokeyy").withAction(TagAction.INCLUDE); + FilteringTag model = new FilteringTag().withName("ajzyul").withValue("u").withAction(TagAction.EXCLUDE); model = BinaryData.fromObject(model).toObject(FilteringTag.class); - Assertions.assertEquals("nqvpkvlrxnje", model.name()); - Assertions.assertEquals("eipheoflokeyy", model.value()); - Assertions.assertEquals(TagAction.INCLUDE, model.action()); + Assertions.assertEquals("ajzyul", model.name()); + Assertions.assertEquals("u", model.value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.action()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/HostsGetRequestTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/HostsGetRequestTests.java index a16732f0db83b..2dc98d1789da7 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/HostsGetRequestTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/HostsGetRequestTests.java @@ -12,20 +12,18 @@ public final class HostsGetRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - HostsGetRequest model = - BinaryData - .fromString("{\"vmIds\":[\"xyqj\",\"k\",\"attpngjcrcczsq\",\"jh\"],\"userEmail\":\"mdajv\"}") - .toObject(HostsGetRequest.class); - Assertions.assertEquals("xyqj", model.vmIds().get(0)); - Assertions.assertEquals("mdajv", model.userEmail()); + HostsGetRequest model = BinaryData.fromString("{\"vmIds\":[\"uhrzayvvt\"],\"userEmail\":\"gvdfgiotkftutq\"}") + .toObject(HostsGetRequest.class); + Assertions.assertEquals("uhrzayvvt", model.vmIds().get(0)); + Assertions.assertEquals("gvdfgiotkftutq", model.userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - HostsGetRequest model = - new HostsGetRequest().withVmIds(Arrays.asList("xyqj", "k", "attpngjcrcczsq", "jh")).withUserEmail("mdajv"); + HostsGetRequest model + = new HostsGetRequest().withVmIds(Arrays.asList("uhrzayvvt")).withUserEmail("gvdfgiotkftutq"); model = BinaryData.fromObject(model).toObject(HostsGetRequest.class); - Assertions.assertEquals("xyqj", model.vmIds().get(0)); - Assertions.assertEquals("mdajv", model.userEmail()); + Assertions.assertEquals("uhrzayvvt", model.vmIds().get(0)); + Assertions.assertEquals("gvdfgiotkftutq", model.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceInnerTests.java new file mode 100644 index 0000000000000..b66a3d9a7dfc1 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceInnerTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; +import org.junit.jupiter.api.Assertions; + +public final class LinkedResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + LinkedResourceInner model = BinaryData.fromString("{\"id\":\"hkfpbs\"}").toObject(LinkedResourceInner.class); + Assertions.assertEquals("hkfpbs", model.id()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + LinkedResourceInner model = new LinkedResourceInner().withId("hkfpbs"); + model = BinaryData.fromObject(model).toObject(LinkedResourceInner.class); + Assertions.assertEquals("hkfpbs", model.id()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceListResponseTests.java new file mode 100644 index 0000000000000..9f573a57da3f0 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LinkedResourceListResponseTests.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.LinkedResourceInner; +import com.azure.resourcemanager.newrelicobservability.models.LinkedResourceListResponse; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class LinkedResourceListResponseTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + LinkedResourceListResponse model = BinaryData.fromString( + "{\"value\":[{\"id\":\"ovawjvzunlu\"},{\"id\":\"nnprn\"},{\"id\":\"peilpjzuaejxdu\"},{\"id\":\"skzbb\"}],\"nextLink\":\"zumveekgpwo\"}") + .toObject(LinkedResourceListResponse.class); + Assertions.assertEquals("ovawjvzunlu", model.value().get(0).id()); + Assertions.assertEquals("zumveekgpwo", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + LinkedResourceListResponse model = new LinkedResourceListResponse().withValue( + Arrays.asList(new LinkedResourceInner().withId("ovawjvzunlu"), new LinkedResourceInner().withId("nnprn"), + new LinkedResourceInner().withId("peilpjzuaejxdu"), new LinkedResourceInner().withId("skzbb"))) + .withNextLink("zumveekgpwo"); + model = BinaryData.fromObject(model).toObject(LinkedResourceListResponse.class); + Assertions.assertEquals("ovawjvzunlu", model.value().get(0).id()); + Assertions.assertEquals("zumveekgpwo", model.nextLink()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LogRulesTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LogRulesTests.java index ea9ee3973b43b..0ef82e45e76e6 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LogRulesTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/LogRulesTests.java @@ -17,45 +17,32 @@ public final class LogRulesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - LogRules model = - BinaryData - .fromString( - "{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"gyxzk\",\"value\":\"ocukoklyax\",\"action\":\"Include\"},{\"name\":\"uqszfk\",\"value\":\"ypewrmjmwvvjekt\",\"action\":\"Include\"},{\"name\":\"nhwlrsffrzpwvl\",\"value\":\"q\",\"action\":\"Exclude\"},{\"name\":\"ylihkaetckt\",\"value\":\"civfsnkymuctq\",\"action\":\"Exclude\"}]}") - .toObject(LogRules.class); - Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.sendAadLogs()); - Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.sendSubscriptionLogs()); - Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.sendActivityLogs()); - Assertions.assertEquals("gyxzk", model.filteringTags().get(0).name()); - Assertions.assertEquals("ocukoklyax", model.filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.INCLUDE, model.filteringTags().get(0).action()); + LogRules model = BinaryData.fromString( + "{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{\"name\":\"cohoq\",\"value\":\"wvl\",\"action\":\"Exclude\"},{\"name\":\"w\",\"value\":\"eun\",\"action\":\"Exclude\"},{\"name\":\"gyxzk\",\"value\":\"ocukoklyax\",\"action\":\"Include\"},{\"name\":\"uqszfk\",\"value\":\"ypewrmjmwvvjekt\",\"action\":\"Include\"}]}") + .toObject(LogRules.class); + Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, model.sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, model.sendActivityLogs()); + Assertions.assertEquals("cohoq", model.filteringTags().get(0).name()); + Assertions.assertEquals("wvl", model.filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.filteringTags().get(0).action()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LogRules model = - new LogRules() - .withSendAadLogs(SendAadLogsStatus.ENABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.DISABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag().withName("gyxzk").withValue("ocukoklyax").withAction(TagAction.INCLUDE), - new FilteringTag() - .withName("uqszfk") - .withValue("ypewrmjmwvvjekt") - .withAction(TagAction.INCLUDE), - new FilteringTag().withName("nhwlrsffrzpwvl").withValue("q").withAction(TagAction.EXCLUDE), - new FilteringTag() - .withName("ylihkaetckt") - .withValue("civfsnkymuctq") - .withAction(TagAction.EXCLUDE))); + LogRules model = new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED).withFilteringTags( + Arrays.asList(new FilteringTag().withName("cohoq").withValue("wvl").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("w").withValue("eun").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("gyxzk").withValue("ocukoklyax").withAction(TagAction.INCLUDE), + new FilteringTag().withName("uqszfk").withValue("ypewrmjmwvvjekt").withAction(TagAction.INCLUDE))); model = BinaryData.fromObject(model).toObject(LogRules.class); - Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.sendAadLogs()); - Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.sendSubscriptionLogs()); - Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.sendActivityLogs()); - Assertions.assertEquals("gyxzk", model.filteringTags().get(0).name()); - Assertions.assertEquals("ocukoklyax", model.filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.INCLUDE, model.filteringTags().get(0).action()); + Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, model.sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, model.sendActivityLogs()); + Assertions.assertEquals("cohoq", model.filteringTags().get(0).name()); + Assertions.assertEquals("wvl", model.filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.filteringTags().get(0).action()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ManagedServiceIdentityTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ManagedServiceIdentityTests.java index f37b181498ebb..0bc0dc5b97a3f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ManagedServiceIdentityTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/ManagedServiceIdentityTests.java @@ -15,25 +15,22 @@ public final class ManagedServiceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ManagedServiceIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"ed20690c-0d99-4300-bd3f-e89837be9fd9\",\"tenantId\":\"07795c14-51aa-4339-8acf-2795ce44c27e\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"t\":{\"principalId\":\"043c8a2a-aab2-4c2d-8beb-1803d690866b\",\"clientId\":\"307e5b23-1b85-4b79-bd0a-0d69d7d7813a\"},\"zzvypyqrimzinp\":{\"principalId\":\"ad3f8f30-f7c2-427b-bca6-ad05c8cbe214\",\"clientId\":\"780adc1c-eea6-4973-bb5a-d44abed6a269\"}}}") - .toObject(ManagedServiceIdentity.class); + ManagedServiceIdentity model = BinaryData.fromString( + "{\"principalId\":\"0d515111-0dc0-4c79-ae35-bca8b0ff4154\",\"tenantId\":\"b65466f3-b3f7-4db6-b20d-92b39e36d15a\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"ouskcqvkocrc\":{\"principalId\":\"edbc0ee6-6ed1-423a-914c-efc493826b54\",\"clientId\":\"38be2d04-ad2e-40ed-97c8-a81b5638be65\"},\"wtnhxbnjbiksqr\":{\"principalId\":\"1f47b747-1ebc-4183-996a-5fb0848fa847\",\"clientId\":\"58511d65-2fdb-49a0-aed6-0629aae71d97\"}}}") + .toObject(ManagedServiceIdentity.class); Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedServiceIdentity model = - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf("t", new UserAssignedIdentity(), "zzvypyqrimzinp", new UserAssignedIdentity())); + ManagedServiceIdentity model = new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities( + mapOf("ouskcqvkocrc", new UserAssignedIdentity(), "wtnhxbnjbiksqr", new UserAssignedIdentity())); model = BinaryData.fromObject(model).toObject(ManagedServiceIdentity.class); Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.type()); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MarketplaceSaaSInfoTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MarketplaceSaaSInfoTests.java new file mode 100644 index 0000000000000..13869be1149ad --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MarketplaceSaaSInfoTests.java @@ -0,0 +1,36 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.models.MarketplaceSaaSInfo; +import org.junit.jupiter.api.Assertions; + +public final class MarketplaceSaaSInfoTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + MarketplaceSaaSInfo model = BinaryData.fromString( + "{\"marketplaceSubscriptionId\":\"wlbjnpgacftade\",\"marketplaceSubscriptionName\":\"nltyfsoppusuesnz\",\"marketplaceResourceId\":\"ej\",\"marketplaceStatus\":\"vorxzdmohct\",\"billedAzureSubscriptionId\":\"vudwx\"}") + .toObject(MarketplaceSaaSInfo.class); + Assertions.assertEquals("wlbjnpgacftade", model.marketplaceSubscriptionId()); + Assertions.assertEquals("nltyfsoppusuesnz", model.marketplaceSubscriptionName()); + Assertions.assertEquals("ej", model.marketplaceResourceId()); + Assertions.assertEquals("vorxzdmohct", model.marketplaceStatus()); + Assertions.assertEquals("vudwx", model.billedAzureSubscriptionId()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + MarketplaceSaaSInfo model = new MarketplaceSaaSInfo().withMarketplaceSubscriptionId("wlbjnpgacftade") + .withMarketplaceSubscriptionName("nltyfsoppusuesnz").withMarketplaceResourceId("ej") + .withMarketplaceStatus("vorxzdmohct").withBilledAzureSubscriptionId("vudwx"); + model = BinaryData.fromObject(model).toObject(MarketplaceSaaSInfo.class); + Assertions.assertEquals("wlbjnpgacftade", model.marketplaceSubscriptionId()); + Assertions.assertEquals("nltyfsoppusuesnz", model.marketplaceSubscriptionName()); + Assertions.assertEquals("ej", model.marketplaceResourceId()); + Assertions.assertEquals("vorxzdmohct", model.marketplaceStatus()); + Assertions.assertEquals("vudwx", model.billedAzureSubscriptionId()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricRulesInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricRulesInnerTests.java index f55f60fa3e462..afcb03cd0f1e4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricRulesInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricRulesInnerTests.java @@ -15,39 +15,29 @@ public final class MetricRulesInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MetricRulesInner model = - BinaryData - .fromString( - "{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{\"name\":\"m\",\"value\":\"qabcypm\",\"action\":\"Exclude\"},{\"name\":\"lzu\",\"value\":\"c\",\"action\":\"Exclude\"},{\"name\":\"nbacfi\",\"value\":\"l\",\"action\":\"Include\"},{\"name\":\"tqgtzxdpnqbqq\",\"value\":\"rjfeallnwsubisnj\",\"action\":\"Include\"}],\"userEmail\":\"ngnzscxaqwoochc\"}") - .toObject(MetricRulesInner.class); + MetricRulesInner model = BinaryData.fromString( + "{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{\"name\":\"dsoifiyipj\",\"value\":\"qwpgrjbzn\",\"action\":\"Exclude\"},{\"name\":\"xv\",\"value\":\"byxqabn\",\"action\":\"Include\"},{\"name\":\"cyshurzafbljjgp\",\"value\":\"oq\",\"action\":\"Include\"}],\"userEmail\":\"ljavbqid\"}") + .toObject(MetricRulesInner.class); Assertions.assertEquals(SendMetricsStatus.ENABLED, model.sendMetrics()); - Assertions.assertEquals("m", model.filteringTags().get(0).name()); - Assertions.assertEquals("qabcypm", model.filteringTags().get(0).value()); + Assertions.assertEquals("dsoifiyipj", model.filteringTags().get(0).name()); + Assertions.assertEquals("qwpgrjbzn", model.filteringTags().get(0).value()); Assertions.assertEquals(TagAction.EXCLUDE, model.filteringTags().get(0).action()); - Assertions.assertEquals("ngnzscxaqwoochc", model.userEmail()); + Assertions.assertEquals("ljavbqid", model.userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MetricRulesInner model = - new MetricRulesInner() - .withSendMetrics(SendMetricsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag().withName("m").withValue("qabcypm").withAction(TagAction.EXCLUDE), - new FilteringTag().withName("lzu").withValue("c").withAction(TagAction.EXCLUDE), - new FilteringTag().withName("nbacfi").withValue("l").withAction(TagAction.INCLUDE), - new FilteringTag() - .withName("tqgtzxdpnqbqq") - .withValue("rjfeallnwsubisnj") - .withAction(TagAction.INCLUDE))) - .withUserEmail("ngnzscxaqwoochc"); + MetricRulesInner model = new MetricRulesInner().withSendMetrics(SendMetricsStatus.ENABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("dsoifiyipj").withValue("qwpgrjbzn").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("xv").withValue("byxqabn").withAction(TagAction.INCLUDE), + new FilteringTag().withName("cyshurzafbljjgp").withValue("oq").withAction(TagAction.INCLUDE))) + .withUserEmail("ljavbqid"); model = BinaryData.fromObject(model).toObject(MetricRulesInner.class); Assertions.assertEquals(SendMetricsStatus.ENABLED, model.sendMetrics()); - Assertions.assertEquals("m", model.filteringTags().get(0).name()); - Assertions.assertEquals("qabcypm", model.filteringTags().get(0).value()); + Assertions.assertEquals("dsoifiyipj", model.filteringTags().get(0).name()); + Assertions.assertEquals("qwpgrjbzn", model.filteringTags().get(0).value()); Assertions.assertEquals(TagAction.EXCLUDE, model.filteringTags().get(0).action()); - Assertions.assertEquals("ngnzscxaqwoochc", model.userEmail()); + Assertions.assertEquals("ljavbqid", model.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsRequestTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsRequestTests.java index f75bed659942b..01a90ad8d6ed4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsRequestTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsRequestTests.java @@ -11,14 +11,15 @@ public final class MetricsRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MetricsRequest model = BinaryData.fromString("{\"userEmail\":\"ofsx\"}").toObject(MetricsRequest.class); - Assertions.assertEquals("ofsx", model.userEmail()); + MetricsRequest model + = BinaryData.fromString("{\"userEmail\":\"rsoodqxhcrmnoh\"}").toObject(MetricsRequest.class); + Assertions.assertEquals("rsoodqxhcrmnoh", model.userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MetricsRequest model = new MetricsRequest().withUserEmail("ofsx"); + MetricsRequest model = new MetricsRequest().withUserEmail("rsoodqxhcrmnoh"); model = BinaryData.fromObject(model).toObject(MetricsRequest.class); - Assertions.assertEquals("ofsx", model.userEmail()); + Assertions.assertEquals("rsoodqxhcrmnoh", model.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusRequestTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusRequestTests.java index 3de4e83529a50..736ace07e176f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusRequestTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusRequestTests.java @@ -12,23 +12,19 @@ public final class MetricsStatusRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MetricsStatusRequest model = - BinaryData - .fromString( - "{\"azureResourceIds\":[\"dlwtgrhpdj\",\"jumasx\",\"zj\",\"qyeg\"],\"userEmail\":\"alhbx\"}") + MetricsStatusRequest model + = BinaryData.fromString("{\"azureResourceIds\":[\"khbzhfepgzg\",\"e\"],\"userEmail\":\"zloc\"}") .toObject(MetricsStatusRequest.class); - Assertions.assertEquals("dlwtgrhpdj", model.azureResourceIds().get(0)); - Assertions.assertEquals("alhbx", model.userEmail()); + Assertions.assertEquals("khbzhfepgzg", model.azureResourceIds().get(0)); + Assertions.assertEquals("zloc", model.userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MetricsStatusRequest model = - new MetricsStatusRequest() - .withAzureResourceIds(Arrays.asList("dlwtgrhpdj", "jumasx", "zj", "qyeg")) - .withUserEmail("alhbx"); + MetricsStatusRequest model + = new MetricsStatusRequest().withAzureResourceIds(Arrays.asList("khbzhfepgzg", "e")).withUserEmail("zloc"); model = BinaryData.fromObject(model).toObject(MetricsStatusRequest.class); - Assertions.assertEquals("dlwtgrhpdj", model.azureResourceIds().get(0)); - Assertions.assertEquals("alhbx", model.userEmail()); + Assertions.assertEquals("khbzhfepgzg", model.azureResourceIds().get(0)); + Assertions.assertEquals("zloc", model.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusResponseInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusResponseInnerTests.java index b1a22fb7525fb..a70fc21e72ec4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusResponseInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MetricsStatusResponseInnerTests.java @@ -12,15 +12,16 @@ public final class MetricsStatusResponseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MetricsStatusResponseInner model = - BinaryData.fromString("{\"azureResourceIds\":[\"jj\"]}").toObject(MetricsStatusResponseInner.class); - Assertions.assertEquals("jj", model.azureResourceIds().get(0)); + MetricsStatusResponseInner model + = BinaryData.fromString("{\"azureResourceIds\":[\"paierh\"]}").toObject(MetricsStatusResponseInner.class); + Assertions.assertEquals("paierh", model.azureResourceIds().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MetricsStatusResponseInner model = new MetricsStatusResponseInner().withAzureResourceIds(Arrays.asList("jj")); + MetricsStatusResponseInner model + = new MetricsStatusResponseInner().withAzureResourceIds(Arrays.asList("paierh")); model = BinaryData.fromObject(model).toObject(MetricsStatusResponseInner.class); - Assertions.assertEquals("jj", model.azureResourceIds().get(0)); + Assertions.assertEquals("paierh", model.azureResourceIds().get(0)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceInnerTests.java index b73a117dc48a6..244fa6cf1ca0b 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceInnerTests.java @@ -13,32 +13,26 @@ public final class MonitoredResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MonitoredResourceInner model = - BinaryData - .fromString( - "{\"id\":\"uqkhrsajiwku\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"kg\",\"sendingLogs\":\"Enabled\",\"reasonForLogsStatus\":\"uimjmvx\"}") - .toObject(MonitoredResourceInner.class); - Assertions.assertEquals("uqkhrsajiwku", model.id()); - Assertions.assertEquals(SendingMetricsStatus.ENABLED, model.sendingMetrics()); - Assertions.assertEquals("kg", model.reasonForMetricsStatus()); + MonitoredResourceInner model = BinaryData.fromString( + "{\"id\":\"scxaq\",\"sendingMetrics\":\"Disabled\",\"reasonForMetricsStatus\":\"hcbonqvpkvlr\",\"sendingLogs\":\"Enabled\",\"reasonForLogsStatus\":\"ase\"}") + .toObject(MonitoredResourceInner.class); + Assertions.assertEquals("scxaq", model.id()); + Assertions.assertEquals(SendingMetricsStatus.DISABLED, model.sendingMetrics()); + Assertions.assertEquals("hcbonqvpkvlr", model.reasonForMetricsStatus()); Assertions.assertEquals(SendingLogsStatus.ENABLED, model.sendingLogs()); - Assertions.assertEquals("uimjmvx", model.reasonForLogsStatus()); + Assertions.assertEquals("ase", model.reasonForLogsStatus()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MonitoredResourceInner model = - new MonitoredResourceInner() - .withId("uqkhrsajiwku") - .withSendingMetrics(SendingMetricsStatus.ENABLED) - .withReasonForMetricsStatus("kg") - .withSendingLogs(SendingLogsStatus.ENABLED) - .withReasonForLogsStatus("uimjmvx"); + MonitoredResourceInner model = new MonitoredResourceInner().withId("scxaq") + .withSendingMetrics(SendingMetricsStatus.DISABLED).withReasonForMetricsStatus("hcbonqvpkvlr") + .withSendingLogs(SendingLogsStatus.ENABLED).withReasonForLogsStatus("ase"); model = BinaryData.fromObject(model).toObject(MonitoredResourceInner.class); - Assertions.assertEquals("uqkhrsajiwku", model.id()); - Assertions.assertEquals(SendingMetricsStatus.ENABLED, model.sendingMetrics()); - Assertions.assertEquals("kg", model.reasonForMetricsStatus()); + Assertions.assertEquals("scxaq", model.id()); + Assertions.assertEquals(SendingMetricsStatus.DISABLED, model.sendingMetrics()); + Assertions.assertEquals("hcbonqvpkvlr", model.reasonForMetricsStatus()); Assertions.assertEquals(SendingLogsStatus.ENABLED, model.sendingLogs()); - Assertions.assertEquals("uimjmvx", model.reasonForLogsStatus()); + Assertions.assertEquals("ase", model.reasonForLogsStatus()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceListResponseTests.java index 25d9a05d8acb6..7bd478ec30c8a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceListResponseTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredResourceListResponseTests.java @@ -15,57 +15,33 @@ public final class MonitoredResourceListResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MonitoredResourceListResponse model = - BinaryData - .fromString( - "{\"value\":[{\"id\":\"gf\",\"sendingMetrics\":\"Disabled\",\"reasonForMetricsStatus\":\"ezyiuokktwhrdxw\",\"sendingLogs\":\"Enabled\",\"reasonForLogsStatus\":\"sm\"},{\"id\":\"ureximoryocfs\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"mddystkiiux\",\"sendingLogs\":\"Disabled\",\"reasonForLogsStatus\":\"dxorrqnbpoczv\"},{\"id\":\"fqrvkdvjsllrmvvd\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"kpnpulexxbczwtr\",\"sendingLogs\":\"Enabled\",\"reasonForLogsStatus\":\"zb\"},{\"id\":\"vsovmyokac\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"lhzdobp\",\"sendingLogs\":\"Enabled\",\"reasonForLogsStatus\":\"lb\"}],\"nextLink\":\"nchrkcciww\"}") - .toObject(MonitoredResourceListResponse.class); - Assertions.assertEquals("gf", model.value().get(0).id()); + MonitoredResourceListResponse model = BinaryData.fromString( + "{\"value\":[{\"id\":\"gyncocpecfvmmc\",\"sendingMetrics\":\"Disabled\",\"reasonForMetricsStatus\":\"xlzevgbmqjqabcy\",\"sendingLogs\":\"Disabled\",\"reasonForLogsStatus\":\"kwlzuvccfwnfn\"},{\"id\":\"cfionl\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"tqgtzxdpnqbqq\",\"sendingLogs\":\"Disabled\",\"reasonForLogsStatus\":\"feallnwsu\"}],\"nextLink\":\"snjampmng\"}") + .toObject(MonitoredResourceListResponse.class); + Assertions.assertEquals("gyncocpecfvmmc", model.value().get(0).id()); Assertions.assertEquals(SendingMetricsStatus.DISABLED, model.value().get(0).sendingMetrics()); - Assertions.assertEquals("ezyiuokktwhrdxw", model.value().get(0).reasonForMetricsStatus()); - Assertions.assertEquals(SendingLogsStatus.ENABLED, model.value().get(0).sendingLogs()); - Assertions.assertEquals("sm", model.value().get(0).reasonForLogsStatus()); - Assertions.assertEquals("nchrkcciww", model.nextLink()); + Assertions.assertEquals("xlzevgbmqjqabcy", model.value().get(0).reasonForMetricsStatus()); + Assertions.assertEquals(SendingLogsStatus.DISABLED, model.value().get(0).sendingLogs()); + Assertions.assertEquals("kwlzuvccfwnfn", model.value().get(0).reasonForLogsStatus()); + Assertions.assertEquals("snjampmng", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MonitoredResourceListResponse model = - new MonitoredResourceListResponse() - .withValue( - Arrays - .asList( - new MonitoredResourceInner() - .withId("gf") - .withSendingMetrics(SendingMetricsStatus.DISABLED) - .withReasonForMetricsStatus("ezyiuokktwhrdxw") - .withSendingLogs(SendingLogsStatus.ENABLED) - .withReasonForLogsStatus("sm"), - new MonitoredResourceInner() - .withId("ureximoryocfs") - .withSendingMetrics(SendingMetricsStatus.ENABLED) - .withReasonForMetricsStatus("mddystkiiux") - .withSendingLogs(SendingLogsStatus.DISABLED) - .withReasonForLogsStatus("dxorrqnbpoczv"), - new MonitoredResourceInner() - .withId("fqrvkdvjsllrmvvd") - .withSendingMetrics(SendingMetricsStatus.ENABLED) - .withReasonForMetricsStatus("kpnpulexxbczwtr") - .withSendingLogs(SendingLogsStatus.ENABLED) - .withReasonForLogsStatus("zb"), - new MonitoredResourceInner() - .withId("vsovmyokac") - .withSendingMetrics(SendingMetricsStatus.ENABLED) - .withReasonForMetricsStatus("lhzdobp") - .withSendingLogs(SendingLogsStatus.ENABLED) - .withReasonForLogsStatus("lb"))) - .withNextLink("nchrkcciww"); + MonitoredResourceListResponse model = new MonitoredResourceListResponse().withValue(Arrays.asList( + new MonitoredResourceInner().withId("gyncocpecfvmmc").withSendingMetrics(SendingMetricsStatus.DISABLED) + .withReasonForMetricsStatus("xlzevgbmqjqabcy").withSendingLogs(SendingLogsStatus.DISABLED) + .withReasonForLogsStatus("kwlzuvccfwnfn"), + new MonitoredResourceInner().withId("cfionl").withSendingMetrics(SendingMetricsStatus.ENABLED) + .withReasonForMetricsStatus("tqgtzxdpnqbqq").withSendingLogs(SendingLogsStatus.DISABLED) + .withReasonForLogsStatus("feallnwsu"))) + .withNextLink("snjampmng"); model = BinaryData.fromObject(model).toObject(MonitoredResourceListResponse.class); - Assertions.assertEquals("gf", model.value().get(0).id()); + Assertions.assertEquals("gyncocpecfvmmc", model.value().get(0).id()); Assertions.assertEquals(SendingMetricsStatus.DISABLED, model.value().get(0).sendingMetrics()); - Assertions.assertEquals("ezyiuokktwhrdxw", model.value().get(0).reasonForMetricsStatus()); - Assertions.assertEquals(SendingLogsStatus.ENABLED, model.value().get(0).sendingLogs()); - Assertions.assertEquals("sm", model.value().get(0).reasonForLogsStatus()); - Assertions.assertEquals("nchrkcciww", model.nextLink()); + Assertions.assertEquals("xlzevgbmqjqabcy", model.value().get(0).reasonForMetricsStatus()); + Assertions.assertEquals(SendingLogsStatus.DISABLED, model.value().get(0).sendingLogs()); + Assertions.assertEquals("kwlzuvccfwnfn", model.value().get(0).reasonForLogsStatus()); + Assertions.assertEquals("snjampmng", model.nextLink()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionInnerTests.java new file mode 100644 index 0000000000000..3407737ec3807 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionInnerTests.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; +import com.azure.resourcemanager.newrelicobservability.models.LogRules; +import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import com.azure.resourcemanager.newrelicobservability.models.TagAction; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class MonitoredSubscriptionInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + MonitoredSubscriptionInner model = BinaryData.fromString( + "{\"subscriptionId\":\"jawgqwg\",\"status\":\"InProgress\",\"error\":\"skxfbk\",\"tagRules\":{\"provisioningState\":\"Accepted\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"uwhvylwzbtdhxujz\",\"value\":\"mpowuwpr\",\"action\":\"Include\"},{\"name\":\"eualupjmkhf\",\"value\":\"bbcswsrtjri\",\"action\":\"Exclude\"},{\"name\":\"pbewtghfgblcgwx\",\"value\":\"lvqhjkbegibtnmx\",\"action\":\"Include\"},{\"name\":\"waloayqcgwr\",\"value\":\"j\",\"action\":\"Include\"}]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{\"name\":\"xongmtsavjcbpwxq\",\"value\":\"rknftguvriuhprwm\",\"action\":\"Exclude\"},{\"name\":\"qtayri\",\"value\":\"ro\",\"action\":\"Include\"},{\"name\":\"xrmcqibycnojvk\",\"value\":\"e\",\"action\":\"Exclude\"},{\"name\":\"zvahapjy\",\"value\":\"pvgqzcjrvxdjzlm\",\"action\":\"Exclude\"}],\"userEmail\":\"vu\"}}}") + .toObject(MonitoredSubscriptionInner.class); + Assertions.assertEquals("jawgqwg", model.subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, model.status()); + Assertions.assertEquals("skxfbk", model.error()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.tagRules().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, + model.tagRules().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.tagRules().logRules().sendActivityLogs()); + Assertions.assertEquals("uwhvylwzbtdhxujz", model.tagRules().logRules().filteringTags().get(0).name()); + Assertions.assertEquals("mpowuwpr", model.tagRules().logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.tagRules().logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.ENABLED, model.tagRules().metricRules().sendMetrics()); + Assertions.assertEquals("xongmtsavjcbpwxq", model.tagRules().metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("rknftguvriuhprwm", model.tagRules().metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.tagRules().metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("vu", model.tagRules().metricRules().userEmail()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + MonitoredSubscriptionInner model = new MonitoredSubscriptionInner().withSubscriptionId("jawgqwg") + .withStatus(Status.IN_PROGRESS).withError("skxfbk") + .withTagRules(new MonitoringTagRulesPropertiesInner().withLogRules(new LogRules() + .withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("uwhvylwzbtdhxujz").withValue("mpowuwpr").withAction(TagAction.INCLUDE), + new FilteringTag().withName("eualupjmkhf").withValue("bbcswsrtjri").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("pbewtghfgblcgwx").withValue("lvqhjkbegibtnmx") + .withAction(TagAction.INCLUDE), + new FilteringTag().withName("waloayqcgwr").withValue("j").withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.ENABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("xongmtsavjcbpwxq").withValue("rknftguvriuhprwm") + .withAction(TagAction.EXCLUDE), + new FilteringTag().withName("qtayri").withValue("ro").withAction(TagAction.INCLUDE), + new FilteringTag().withName("xrmcqibycnojvk").withValue("e").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("zvahapjy").withValue("pvgqzcjrvxdjzlm") + .withAction(TagAction.EXCLUDE))) + .withUserEmail("vu"))); + model = BinaryData.fromObject(model).toObject(MonitoredSubscriptionInner.class); + Assertions.assertEquals("jawgqwg", model.subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, model.status()); + Assertions.assertEquals("skxfbk", model.error()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.tagRules().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, + model.tagRules().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.tagRules().logRules().sendActivityLogs()); + Assertions.assertEquals("uwhvylwzbtdhxujz", model.tagRules().logRules().filteringTags().get(0).name()); + Assertions.assertEquals("mpowuwpr", model.tagRules().logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.tagRules().logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.ENABLED, model.tagRules().metricRules().sendMetrics()); + Assertions.assertEquals("xongmtsavjcbpwxq", model.tagRules().metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("rknftguvriuhprwm", model.tagRules().metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.tagRules().metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("vu", model.tagRules().metricRules().userEmail()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesInnerTests.java new file mode 100644 index 0000000000000..a7b4213213150 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesInnerTests.java @@ -0,0 +1,101 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; +import com.azure.resourcemanager.newrelicobservability.models.LogRules; +import com.azure.resourcemanager.newrelicobservability.models.PatchOperation; +import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import com.azure.resourcemanager.newrelicobservability.models.SubscriptionList; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class MonitoredSubscriptionPropertiesInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + MonitoredSubscriptionPropertiesInner model = BinaryData.fromString( + "{\"properties\":{\"patchOperation\":\"AddComplete\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"ywpmueefjzwfqkq\",\"status\":\"InProgress\",\"error\":\"suyonobglaocq\",\"tagRules\":{\"provisioningState\":\"Failed\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{},{}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{}],\"userEmail\":\"wfudwpzntxhdzhl\"}}},{\"subscriptionId\":\"jbhckfrlhr\",\"status\":\"Active\",\"error\":\"yvpycanuzbpzk\",\"tagRules\":{\"provisioningState\":\"Accepted\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{},{},{},{}]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{},{},{}],\"userEmail\":\"jusrtslhspk\"}}}],\"provisioningState\":\"Updating\"},\"id\":\"maofmxagkv\",\"name\":\"melmqkrha\",\"type\":\"vljua\"}") + .toObject(MonitoredSubscriptionPropertiesInner.class); + Assertions.assertEquals(PatchOperation.ADD_COMPLETE, model.properties().patchOperation()); + Assertions.assertEquals("ywpmueefjzwfqkq", + model.properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, model.properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("suyonobglaocq", model.properties().monitoredSubscriptionList().get(0).error()); + Assertions.assertEquals(SendAadLogsStatus.DISABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.DISABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().logRules().sendActivityLogs()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().metricRules().sendMetrics()); + Assertions.assertEquals("wfudwpzntxhdzhl", + model.properties().monitoredSubscriptionList().get(0).tagRules().metricRules().userEmail()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + MonitoredSubscriptionPropertiesInner model + = new MonitoredSubscriptionPropertiesInner() + .withProperties( + new SubscriptionList() + .withPatchOperation( + PatchOperation.ADD_COMPLETE) + .withMonitoredSubscriptionList( + Arrays + .asList( + new MonitoredSubscriptionInner().withSubscriptionId("ywpmueefjzwfqkq") + .withStatus( + Status.IN_PROGRESS) + .withError("suyonobglaocq") + .withTagRules(new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags( + Arrays.asList(new FilteringTag(), new FilteringTag()))) + .withMetricRules(new MetricRulesInner() + .withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList(new FilteringTag())) + .withUserEmail("wfudwpzntxhdzhl"))), + new MonitoredSubscriptionInner().withSubscriptionId("jbhckfrlhr") + .withStatus(Status.ACTIVE).withError("yvpycanuzbpzk") + .withTagRules(new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag(), new FilteringTag(), + new FilteringTag(), new FilteringTag()))) + .withMetricRules(new MetricRulesInner() + .withSendMetrics(SendMetricsStatus.ENABLED).withFilteringTags(Arrays + .asList(new FilteringTag(), new FilteringTag(), new FilteringTag())) + .withUserEmail("jusrtslhspk")))))); + model = BinaryData.fromObject(model).toObject(MonitoredSubscriptionPropertiesInner.class); + Assertions.assertEquals(PatchOperation.ADD_COMPLETE, model.properties().patchOperation()); + Assertions.assertEquals("ywpmueefjzwfqkq", + model.properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, model.properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("suyonobglaocq", model.properties().monitoredSubscriptionList().get(0).error()); + Assertions.assertEquals(SendAadLogsStatus.DISABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.DISABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().logRules().sendActivityLogs()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, + model.properties().monitoredSubscriptionList().get(0).tagRules().metricRules().sendMetrics()); + Assertions.assertEquals("wfudwpzntxhdzhl", + model.properties().monitoredSubscriptionList().get(0).tagRules().metricRules().userEmail()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesListTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesListTests.java new file mode 100644 index 0000000000000..0a0ee0422274b --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionPropertiesListTests.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionPropertiesList; +import com.azure.resourcemanager.newrelicobservability.models.PatchOperation; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import com.azure.resourcemanager.newrelicobservability.models.SubscriptionList; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class MonitoredSubscriptionPropertiesListTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + MonitoredSubscriptionPropertiesList model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"patchOperation\":\"AddComplete\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"kx\",\"status\":\"Deleting\",\"error\":\"pbh\",\"tagRules\":{}},{\"subscriptionId\":\"tkcxywnytnrsy\",\"status\":\"Active\",\"error\":\"dybyxczfclhaa\",\"tagRules\":{}},{\"subscriptionId\":\"abphlw\",\"status\":\"Active\",\"error\":\"ktsthsucocmny\",\"tagRules\":{}},{\"subscriptionId\":\"t\",\"status\":\"Active\",\"error\":\"wrqpue\",\"tagRules\":{}}],\"provisioningState\":\"Succeeded\"},\"id\":\"ywbiexzfeyueax\",\"name\":\"bxu\",\"type\":\"wbhqwal\"},{\"properties\":{\"patchOperation\":\"DeleteComplete\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"epdkzja\",\"status\":\"InProgress\",\"error\":\"rhdwbavxbniw\",\"tagRules\":{}},{\"subscriptionId\":\"wz\",\"status\":\"Failed\",\"error\":\"pgn\",\"tagRules\":{}},{\"subscriptionId\":\"x\",\"status\":\"Failed\",\"error\":\"bzpfzab\",\"tagRules\":{}},{\"subscriptionId\":\"uhxwtctyqiklbbov\",\"status\":\"Failed\",\"error\":\"bhvgy\",\"tagRules\":{}}],\"provisioningState\":\"Succeeded\"},\"id\":\"svmkfssxquk\",\"name\":\"fpl\",\"type\":\"mg\"},{\"properties\":{\"patchOperation\":\"Active\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"deslp\",\"status\":\"InProgress\",\"error\":\"wiyighxpkdw\",\"tagRules\":{}}],\"provisioningState\":\"Updating\"},\"id\":\"uebbaumnyqup\",\"name\":\"deoj\",\"type\":\"a\"},{\"properties\":{\"patchOperation\":\"Active\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"xpsiebtfhvpes\",\"status\":\"Failed\",\"error\":\"rdqmhjjdhtldwkyz\",\"tagRules\":{}},{\"subscriptionId\":\"tkncwsc\",\"status\":\"Deleting\",\"error\":\"xotogtwrupqsxv\",\"tagRules\":{}},{\"subscriptionId\":\"cykvceo\",\"status\":\"Active\",\"error\":\"ovnotyfjfcnjbkcn\",\"tagRules\":{}}],\"provisioningState\":\"Creating\"},\"id\":\"ttkphywpnvjtoqne\",\"name\":\"mclfplphoxuscr\",\"type\":\"abgy\"}],\"nextLink\":\"sbj\"}") + .toObject(MonitoredSubscriptionPropertiesList.class); + Assertions.assertEquals(PatchOperation.ADD_COMPLETE, model.value().get(0).properties().patchOperation()); + Assertions.assertEquals("kx", + model.value().get(0).properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.DELETING, + model.value().get(0).properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("pbh", model.value().get(0).properties().monitoredSubscriptionList().get(0).error()); + Assertions.assertEquals("sbj", model.nextLink()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + MonitoredSubscriptionPropertiesList model = new MonitoredSubscriptionPropertiesList() + .withValue(Arrays.asList( + new MonitoredSubscriptionPropertiesInner().withProperties(new SubscriptionList() + .withPatchOperation(PatchOperation.ADD_COMPLETE) + .withMonitoredSubscriptionList(Arrays.asList( + new MonitoredSubscriptionInner().withSubscriptionId("kx").withStatus(Status.DELETING) + .withError("pbh").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("tkcxywnytnrsy").withStatus(Status.ACTIVE) + .withError("dybyxczfclhaa").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("abphlw").withStatus(Status.ACTIVE) + .withError("ktsthsucocmny").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("t").withStatus(Status.ACTIVE) + .withError("wrqpue").withTagRules(new MonitoringTagRulesPropertiesInner())))), + new MonitoredSubscriptionPropertiesInner().withProperties(new SubscriptionList() + .withPatchOperation(PatchOperation.DELETE_COMPLETE) + .withMonitoredSubscriptionList(Arrays.asList( + new MonitoredSubscriptionInner().withSubscriptionId("epdkzja").withStatus(Status.IN_PROGRESS) + .withError("rhdwbavxbniw").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("wz").withStatus(Status.FAILED) + .withError("pgn").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("x").withStatus(Status.FAILED) + .withError("bzpfzab").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("uhxwtctyqiklbbov") + .withStatus(Status.FAILED).withError("bhvgy") + .withTagRules(new MonitoringTagRulesPropertiesInner())))), + new MonitoredSubscriptionPropertiesInner() + .withProperties( + new SubscriptionList().withPatchOperation(PatchOperation.ACTIVE).withMonitoredSubscriptionList( + Arrays.asList(new MonitoredSubscriptionInner().withSubscriptionId("deslp") + .withStatus(Status.IN_PROGRESS).withError("wiyighxpkdw") + .withTagRules(new MonitoringTagRulesPropertiesInner())))), + new MonitoredSubscriptionPropertiesInner().withProperties(new SubscriptionList() + .withPatchOperation(PatchOperation.ACTIVE) + .withMonitoredSubscriptionList(Arrays.asList( + new MonitoredSubscriptionInner().withSubscriptionId("xpsiebtfhvpes").withStatus(Status.FAILED) + .withError("rdqmhjjdhtldwkyz").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("tkncwsc").withStatus(Status.DELETING) + .withError("xotogtwrupqsxv").withTagRules(new MonitoringTagRulesPropertiesInner()), + new MonitoredSubscriptionInner().withSubscriptionId("cykvceo").withStatus(Status.ACTIVE) + .withError("ovnotyfjfcnjbkcn").withTagRules(new MonitoringTagRulesPropertiesInner())))))) + .withNextLink("sbj"); + model = BinaryData.fromObject(model).toObject(MonitoredSubscriptionPropertiesList.class); + Assertions.assertEquals(PatchOperation.ADD_COMPLETE, model.value().get(0).properties().patchOperation()); + Assertions.assertEquals("kx", + model.value().get(0).properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.DELETING, + model.value().get(0).properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("pbh", model.value().get(0).properties().monitoredSubscriptionList().get(0).error()); + Assertions.assertEquals("sbj", model.nextLink()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateMockTests.java new file mode 100644 index 0000000000000..0dbae985ff14a --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsCreateorUpdateMockTests.java @@ -0,0 +1,90 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.LogRules; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; +import com.azure.resourcemanager.newrelicobservability.models.PatchOperation; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import com.azure.resourcemanager.newrelicobservability.models.SubscriptionList; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class MonitoredSubscriptionsCreateorUpdateMockTests { + @Test + public void testCreateorUpdate() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"patchOperation\":\"Active\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"zkoj\",\"status\":\"InProgress\",\"error\":\"zfoqouicybxar\",\"tagRules\":{\"provisioningState\":\"NotSpecified\",\"logRules\":{},\"metricRules\":{}}}],\"provisioningState\":\"Succeeded\"},\"id\":\"ciqopidoa\",\"name\":\"ciodhkhaz\",\"type\":\"khnzbonlw\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + MonitoredSubscriptionProperties response + = manager + .monitoredSubscriptions().define(ConfigurationName.DEFAULT).withExistingMonitor("pfuvglsbjjca", + "vxb") + .withProperties( + new SubscriptionList().withPatchOperation(PatchOperation.DELETE_COMPLETE) + .withMonitoredSubscriptionList( + Arrays.asList( + new MonitoredSubscriptionInner().withSubscriptionId("cormr").withStatus(Status.DELETING) + .withError("vcofudfl") + .withTagRules(new MonitoringTagRulesPropertiesInner().withLogRules(new LogRules()) + .withMetricRules(new MetricRulesInner())), + new MonitoredSubscriptionInner().withSubscriptionId("dknnqvsazn") + .withStatus(Status.IN_PROGRESS).withError("rudsg") + .withTagRules(new MonitoringTagRulesPropertiesInner().withLogRules(new LogRules()) + .withMetricRules(new MetricRulesInner())), + new MonitoredSubscriptionInner().withSubscriptionId("grauwjuetaebur") + .withStatus(Status.ACTIVE).withError("ovsm") + .withTagRules(new MonitoringTagRulesPropertiesInner().withLogRules(new LogRules()) + .withMetricRules(new MetricRulesInner())), + new MonitoredSubscriptionInner().withSubscriptionId("q").withStatus(Status.ACTIVE) + .withError("ifrvtpu").withTagRules(new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules()).withMetricRules(new MetricRulesInner()))))) + .create(); + + Assertions.assertEquals(PatchOperation.ACTIVE, response.properties().patchOperation()); + Assertions.assertEquals("zkoj", response.properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, response.properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("zfoqouicybxar", response.properties().monitoredSubscriptionList().get(0).error()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetWithResponseMockTests.java new file mode 100644 index 0000000000000..cb79a28687584 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsGetWithResponseMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; +import com.azure.resourcemanager.newrelicobservability.models.ConfigurationName; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; +import com.azure.resourcemanager.newrelicobservability.models.PatchOperation; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class MonitoredSubscriptionsGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"patchOperation\":\"AddComplete\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"kfpagao\",\"status\":\"Deleting\",\"error\":\"pqblylsyxkqjnsj\",\"tagRules\":{\"provisioningState\":\"Updating\",\"logRules\":{},\"metricRules\":{}}},{\"subscriptionId\":\"xsdszuempsb\",\"status\":\"InProgress\",\"error\":\"beyvpnqicvinvkjj\",\"tagRules\":{\"provisioningState\":\"Canceled\",\"logRules\":{},\"metricRules\":{}}},{\"subscriptionId\":\"kzclewyh\",\"status\":\"InProgress\",\"error\":\"aztz\",\"tagRules\":{\"provisioningState\":\"Creating\",\"logRules\":{},\"metricRules\":{}}},{\"subscriptionId\":\"wyfzqwhxxbuyqa\",\"status\":\"Failed\",\"error\":\"qztpp\",\"tagRules\":{\"provisioningState\":\"Updating\",\"logRules\":{},\"metricRules\":{}}}],\"provisioningState\":\"NotSpecified\"},\"id\":\"altol\",\"name\":\"ncwsob\",\"type\":\"wcsdbnwdcfhucq\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + MonitoredSubscriptionProperties response = manager.monitoredSubscriptions().getWithResponse("vdkcrodtj", + "nfwjlfltkacjvefk", ConfigurationName.DEFAULT, com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals(PatchOperation.ADD_COMPLETE, response.properties().patchOperation()); + Assertions.assertEquals("kfpagao", response.properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.DELETING, response.properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("pqblylsyxkqjnsj", response.properties().monitoredSubscriptionList().get(0).error()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListMockTests.java new file mode 100644 index 0000000000000..c106c739ea813 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoredSubscriptionsListMockTests.java @@ -0,0 +1,65 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; +import com.azure.resourcemanager.newrelicobservability.models.MonitoredSubscriptionProperties; +import com.azure.resourcemanager.newrelicobservability.models.PatchOperation; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class MonitoredSubscriptionsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"patchOperation\":\"AddComplete\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"q\",\"status\":\"Failed\",\"error\":\"axoruzfgsquy\",\"tagRules\":{\"provisioningState\":\"Updating\",\"logRules\":{},\"metricRules\":{}}},{\"subscriptionId\":\"ptramxj\",\"status\":\"Failed\",\"error\":\"wnwxuqlcvyd\",\"tagRules\":{\"provisioningState\":\"Deleting\",\"logRules\":{},\"metricRules\":{}}}],\"provisioningState\":\"Deleted\"},\"id\":\"ojknio\",\"name\":\"kooebwnu\",\"type\":\"hemms\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.monitoredSubscriptions().list("ytdw", "qbrqubpaxhexiili", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(PatchOperation.ADD_COMPLETE, response.iterator().next().properties().patchOperation()); + Assertions.assertEquals("q", + response.iterator().next().properties().monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.FAILED, + response.iterator().next().properties().monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("axoruzfgsquy", + response.iterator().next().properties().monitoredSubscriptionList().get(0).error()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoringTagRulesPropertiesInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoringTagRulesPropertiesInnerTests.java index e40fbe92d6b62..0af9c019fde49 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoringTagRulesPropertiesInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitoringTagRulesPropertiesInnerTests.java @@ -20,78 +20,50 @@ public final class MonitoringTagRulesPropertiesInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - MonitoringTagRulesPropertiesInner model = - BinaryData - .fromString( - "{\"provisioningState\":\"Accepted\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"o\",\"value\":\"ukgjnpiucgygevq\",\"action\":\"Exclude\"},{\"name\":\"pmr\",\"value\":\"izcdrqjsd\",\"action\":\"Exclude\"},{\"name\":\"fyhxde\",\"value\":\"jzicwifsjt\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{\"name\":\"bkh\",\"value\":\"deyeamdphagalpbu\",\"action\":\"Exclude\"},{\"name\":\"pwhonowkg\",\"value\":\"wankixzbi\",\"action\":\"Exclude\"},{\"name\":\"uttmrywnuzoqft\",\"value\":\"qzrnkcqvyxlwhz\",\"action\":\"Exclude\"}],\"userEmail\":\"ohoqqnwvlryav\"}}") - .toObject(MonitoringTagRulesPropertiesInner.class); + MonitoringTagRulesPropertiesInner model = BinaryData.fromString( + "{\"provisioningState\":\"Failed\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"kpoc\",\"value\":\"azyxoegukg\",\"action\":\"Include\"},{\"name\":\"ucgygevqz\",\"value\":\"yp\",\"action\":\"Exclude\"},{\"name\":\"izcdrqjsd\",\"value\":\"dnfyhxdeoejzicwi\",\"action\":\"Include\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"ishc\",\"value\":\"hajdeyeamdpha\",\"action\":\"Exclude\"},{\"name\":\"buxwgip\",\"value\":\"onowk\",\"action\":\"Exclude\"},{\"name\":\"ankixzbinjeput\",\"value\":\"rywn\",\"action\":\"Exclude\"}],\"userEmail\":\"ftiyqzrnkcq\"}}") + .toObject(MonitoringTagRulesPropertiesInner.class); Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.logRules().sendActivityLogs()); - Assertions.assertEquals("o", model.logRules().filteringTags().get(0).name()); - Assertions.assertEquals("ukgjnpiucgygevq", model.logRules().filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.EXCLUDE, model.logRules().filteringTags().get(0).action()); - Assertions.assertEquals(SendMetricsStatus.ENABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("bkh", model.metricRules().filteringTags().get(0).name()); - Assertions.assertEquals("deyeamdphagalpbu", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals("kpoc", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("azyxoegukg", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); + Assertions.assertEquals("ishc", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("hajdeyeamdpha", model.metricRules().filteringTags().get(0).value()); Assertions.assertEquals(TagAction.EXCLUDE, model.metricRules().filteringTags().get(0).action()); - Assertions.assertEquals("ohoqqnwvlryav", model.metricRules().userEmail()); + Assertions.assertEquals("ftiyqzrnkcq", model.metricRules().userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - MonitoringTagRulesPropertiesInner model = - new MonitoringTagRulesPropertiesInner() - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.DISABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) - .withSendActivityLogs(SendActivityLogsStatus.DISABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("o") - .withValue("ukgjnpiucgygevq") - .withAction(TagAction.EXCLUDE), - new FilteringTag() - .withName("pmr") - .withValue("izcdrqjsd") - .withAction(TagAction.EXCLUDE), - new FilteringTag() - .withName("fyhxde") - .withValue("jzicwifsjt") - .withAction(TagAction.EXCLUDE)))) - .withMetricRules( - new MetricRulesInner() - .withSendMetrics(SendMetricsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("bkh") - .withValue("deyeamdphagalpbu") - .withAction(TagAction.EXCLUDE), - new FilteringTag() - .withName("pwhonowkg") - .withValue("wankixzbi") - .withAction(TagAction.EXCLUDE), - new FilteringTag() - .withName("uttmrywnuzoqft") - .withValue("qzrnkcqvyxlwhz") - .withAction(TagAction.EXCLUDE))) - .withUserEmail("ohoqqnwvlryav")); + MonitoringTagRulesPropertiesInner model = new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("kpoc").withValue("azyxoegukg").withAction(TagAction.INCLUDE), + new FilteringTag().withName("ucgygevqz").withValue("yp").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("izcdrqjsd").withValue("dnfyhxdeoejzicwi") + .withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("ishc").withValue("hajdeyeamdpha").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("buxwgip").withValue("onowk").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("ankixzbinjeput").withValue("rywn").withAction(TagAction.EXCLUDE))) + .withUserEmail("ftiyqzrnkcq")); model = BinaryData.fromObject(model).toObject(MonitoringTagRulesPropertiesInner.class); Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.logRules().sendActivityLogs()); - Assertions.assertEquals("o", model.logRules().filteringTags().get(0).name()); - Assertions.assertEquals("ukgjnpiucgygevq", model.logRules().filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.EXCLUDE, model.logRules().filteringTags().get(0).action()); - Assertions.assertEquals(SendMetricsStatus.ENABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("bkh", model.metricRules().filteringTags().get(0).name()); - Assertions.assertEquals("deyeamdphagalpbu", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals("kpoc", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("azyxoegukg", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); + Assertions.assertEquals("ishc", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("hajdeyeamdpha", model.metricRules().filteringTags().get(0).value()); Assertions.assertEquals(TagAction.EXCLUDE, model.metricRules().filteringTags().get(0).action()); - Assertions.assertEquals("ohoqqnwvlryav", model.metricRules().userEmail()); + Assertions.assertEquals("ftiyqzrnkcq", model.metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateMockTests.java deleted file mode 100644 index 6ddc8ee79ad5a..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsCreateOrUpdateMockTests.java +++ /dev/null @@ -1,136 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicAccountProperties; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResource; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.PlanData; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import com.azure.resourcemanager.newrelicobservability.models.UserInfo; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class MonitorsCreateOrUpdateMockTests { - @Test - public void testCreateOrUpdate() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"tgqr\",\"newRelicAccountProperties\":{\"userId\":\"muwlauwzizxbm\"},\"userInfo\":{\"firstName\":\"u\",\"lastName\":\"uvpb\",\"emailAddress\":\"d\",\"phoneNumber\":\"orppxebmnzbtb\",\"country\":\"pglkf\"},\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"el\",\"effectiveDate\":\"2021-04-04T13:11:17Z\"},\"liftrResourceCategory\":\"MonitorLogs\",\"liftrResourcePreference\":314213768,\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"1be7716f-5649-47e7-9220-3cf2fedbeb8d\",\"tenantId\":\"9aa28025-d8d0-4611-a2ed-7f90fe46c74c\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{}},\"location\":\"uuvxz\",\"tags\":{\"zonosgg\":\"vithh\",\"ljuti\":\"hcohfwdsjnk\",\"wkfvhqcrailvp\":\"swacffgdkzz\",\"wdmhdlxyjrxs\":\"ppfufl\"},\"id\":\"gafcnihgwqapnedg\",\"name\":\"bcvkcvqvpkeq\",\"type\":\"cvdrhvoodsot\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - NewRelicMonitorResource response = - manager - .monitors() - .define("l") - .withRegion("hszhedplvwiwu") - .withExistingResourceGroup("ertgccymva") - .withTags( - mapOf( - "dnkwwtppjflcxog", - "mbes", - "qqkdltfzxmhhvhgu", - "okonzmnsikvmkqz", - "xtibqdxbxwakbog", - "eodkwobda")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf())) - .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("wyhzdx")) - .withUserInfo( - new UserInfo() - .withFirstName("mnvdfzn") - .withLastName("ao") - .withEmailAddress("xzb") - .withPhoneNumber("blylpstdbh") - .withCountry("srzdzucerscdn")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.COMMITTED) - .withBillingCycle(BillingCycle.WEEKLY) - .withPlanDetails("jmygtdsslswtmwer") - .withEffectiveDate(OffsetDateTime.parse("2021-09-22T20:42:28Z"))) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.LIFTR) - .create(); - - Assertions.assertEquals("uuvxz", response.location()); - Assertions.assertEquals("vithh", response.tags().get("zonosgg")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.identity().type()); - Assertions.assertEquals("muwlauwzizxbm", response.newRelicAccountProperties().userId()); - Assertions.assertEquals("u", response.userInfo().firstName()); - Assertions.assertEquals("uvpb", response.userInfo().lastName()); - Assertions.assertEquals("d", response.userInfo().emailAddress()); - Assertions.assertEquals("orppxebmnzbtb", response.userInfo().phoneNumber()); - Assertions.assertEquals("pglkf", response.userInfo().country()); - Assertions.assertEquals(UsageType.PAYG, response.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, response.planData().billingCycle()); - Assertions.assertEquals("el", response.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-04-04T13:11:17Z"), response.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, response.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, response.accountCreationSource()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteMockTests.java index 58544416f15e3..f6b697c808f75 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.monitors().delete("l", "zbtd", "xujznbmpowu", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.monitors().delete("lmpewwwfbkr", "rn", "vshqjohxcr", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupWithResponseMockTests.java deleted file mode 100644 index 248a15ae66447..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetByResourceGroupWithResponseMockTests.java +++ /dev/null @@ -1,89 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResource; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class MonitorsGetByResourceGroupWithResponseMockTests { - @Test - public void testGetByResourceGroupWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"provisioningState\":\"Creating\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"melmqkrha\",\"newRelicAccountProperties\":{\"userId\":\"juahaquhcdhmdual\"},\"userInfo\":{\"firstName\":\"fadmws\",\"lastName\":\"r\",\"emailAddress\":\"xpvgo\",\"phoneNumber\":\"lf\",\"country\":\"sgwbnbbeld\"},\"planData\":{\"usageType\":\"COMMITTED\",\"billingCycle\":\"WEEKLY\",\"planDetails\":\"io\",\"effectiveDate\":\"2021-05-06T22:54:03Z\"},\"liftrResourceCategory\":\"Unknown\",\"liftrResourcePreference\":391809232,\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"f8829299-fc95-4144-aef6-fe4aa2d30e0b\",\"tenantId\":\"8d57248d-6b81-42fe-abf1-13066e5741ee\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{}},\"location\":\"owzxcu\",\"tags\":{\"ucww\":\"jooxdjebw\",\"bvmeuecivy\":\"vo\",\"ueiotwmcdyt\":\"zceuojgjrw\",\"it\":\"x\"},\"id\":\"nrjawgqwg\",\"name\":\"hniskxfbkpyc\",\"type\":\"klwndnhjdauwhv\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - NewRelicMonitorResource response = - manager - .monitors() - .getByResourceGroupWithResponse("ehhseyvjusrts", "hspkdeemao", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("owzxcu", response.location()); - Assertions.assertEquals("jooxdjebw", response.tags().get("ucww")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.identity().type()); - Assertions.assertEquals("juahaquhcdhmdual", response.newRelicAccountProperties().userId()); - Assertions.assertEquals("fadmws", response.userInfo().firstName()); - Assertions.assertEquals("r", response.userInfo().lastName()); - Assertions.assertEquals("xpvgo", response.userInfo().emailAddress()); - Assertions.assertEquals("lf", response.userInfo().phoneNumber()); - Assertions.assertEquals("sgwbnbbeld", response.userInfo().country()); - Assertions.assertEquals(UsageType.COMMITTED, response.planData().usageType()); - Assertions.assertEquals(BillingCycle.WEEKLY, response.planData().billingCycle()); - Assertions.assertEquals("io", response.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-06T22:54:03Z"), response.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, response.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, response.accountCreationSource()); - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesWithResponseMockTests.java index 9e0c776c5b3b8..0a0f108eef730 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesWithResponseMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricRulesWithResponseMockTests.java @@ -33,49 +33,31 @@ public void testGetMetricRulesWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"bpbewtghfgb\",\"value\":\"gw\",\"action\":\"Exclude\"},{\"name\":\"v\",\"value\":\"jkbegibtnmxiebww\",\"action\":\"Include\"},{\"name\":\"yqcgwrtzjuzgwy\",\"value\":\"htxongmtsavjc\",\"action\":\"Exclude\"},{\"name\":\"qpsrknftguvri\",\"value\":\"prwmdyvxqt\",\"action\":\"Include\"}],\"userEmail\":\"ww\"}"; + String responseStr + = "{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{\"name\":\"bsrfbj\",\"value\":\"twss\",\"action\":\"Exclude\"},{\"name\":\"pvjzbe\",\"value\":\"l\",\"action\":\"Include\"},{\"name\":\"qqnvwpmq\",\"value\":\"ruoujmk\",\"action\":\"Include\"}],\"userEmail\":\"qytjrybnwjewgd\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - MetricRules response = - manager - .monitors() - .getMetricRulesWithResponse( - "przqlveu", - "lupj", - new MetricsRequest().withUserEmail("khfxobbcswsrt"), - com.azure.core.util.Context.NONE) - .getValue(); + MetricRules response = manager.monitors().getMetricRulesWithResponse("bfovasrruvwbhsq", "sub", + new MetricsRequest().withUserEmail("gjb"), com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(SendMetricsStatus.DISABLED, response.sendMetrics()); - Assertions.assertEquals("bpbewtghfgb", response.filteringTags().get(0).name()); - Assertions.assertEquals("gw", response.filteringTags().get(0).value()); + Assertions.assertEquals(SendMetricsStatus.ENABLED, response.sendMetrics()); + Assertions.assertEquals("bsrfbj", response.filteringTags().get(0).name()); + Assertions.assertEquals("twss", response.filteringTags().get(0).value()); Assertions.assertEquals(TagAction.EXCLUDE, response.filteringTags().get(0).action()); - Assertions.assertEquals("ww", response.userEmail()); + Assertions.assertEquals("qytjrybnwjewgd", response.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusWithResponseMockTests.java index 65526cf5d25b2..701e5002817fd 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusWithResponseMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsGetMetricStatusWithResponseMockTests.java @@ -32,46 +32,29 @@ public void testGetMetricStatusWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"azureResourceIds\":[\"vxdjzlmwlxkvugf\",\"zovawjvz\",\"nluthnnp\"]}"; + String responseStr = "{\"azureResourceIds\":[\"duhavhqlkt\",\"umaq\",\"lbg\",\"cdui\"]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - MetricsStatusResponse response = - manager - .monitors() - .getMetricStatusWithResponse( - "oyq", - "exrmcqibycnojvk", - new MetricsStatusRequest() - .withAzureResourceIds(Arrays.asList("fqsgzvahapjy")) - .withUserEmail("hpvgqz"), - com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("vxdjzlmwlxkvugf", response.azureResourceIds().get(0)); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + MetricsStatusResponse response = manager.monitors() + .getMetricStatusWithResponse("jervnaenqpehi", "doy", + new MetricsStatusRequest().withAzureResourceIds(Arrays.asList("fthnzdn", "sl")).withUserEmail("nayqi"), + com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals("duhavhqlkt", response.azureResourceIds().get(0)); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesMockTests.java index 8fd56d8b4eee8..befffe32bf083 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListAppServicesMockTests.java @@ -33,48 +33,31 @@ public void testListAppServices() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"azureResourceId\":\"nsmvbxwyj\",\"agentVersion\":\"lh\",\"agentStatus\":\"aalnjixi\"}]}"; + String responseStr + = "{\"value\":[{\"azureResourceId\":\"dvxzbncblylpst\",\"agentVersion\":\"hh\",\"agentStatus\":\"rzdzucerscdnt\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .monitors() - .listAppServices( - "nxipeil", - "jzuaejxdultskzbb", - new AppServicesGetRequest() - .withAzureResourceIds(Arrays.asList("umveekgpwozuhkf", "bsjyofdx", "uusdttouwa")) - .withUserEmail("oekqvk"), - com.azure.core.util.Context.NONE); - - Assertions.assertEquals("nsmvbxwyj", response.iterator().next().azureResourceId()); - Assertions.assertEquals("lh", response.iterator().next().agentVersion()); - Assertions.assertEquals("aalnjixi", response.iterator().next().agentStatus()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.monitors().listAppServices( + "r", "gccymvaolpssl", new AppServicesGetRequest() + .withAzureResourceIds(Arrays.asList("mmdnbbglzps", "iydmcwyhzdxs")).withUserEmail("adbzmnvdfznud"), + com.azure.core.util.Context.NONE); + + Assertions.assertEquals("dvxzbncblylpst", response.iterator().next().azureResourceId()); + Assertions.assertEquals("hh", response.iterator().next().agentVersion()); + Assertions.assertEquals("rzdzucerscdnt", response.iterator().next().agentStatus()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupMockTests.java deleted file mode 100644 index 95905092b4f9c..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListByResourceGroupMockTests.java +++ /dev/null @@ -1,89 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResource; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class MonitorsListByResourceGroupMockTests { - @Test - public void testListByResourceGroup() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Accepted\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Suspended\",\"marketplaceSubscriptionId\":\"pqsxvnmicy\",\"newRelicAccountProperties\":{\"userId\":\"eoveilovnotyf\"},\"userInfo\":{\"firstName\":\"k\",\"lastName\":\"x\",\"emailAddress\":\"bttk\",\"phoneNumber\":\"ywpnvjt\",\"country\":\"nermcl\"},\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"YEARLY\",\"planDetails\":\"uscrpabgyepsb\",\"effectiveDate\":\"2021-01-26T13:24:27Z\"},\"liftrResourceCategory\":\"MonitorLogs\",\"liftrResourcePreference\":2075352957,\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"43342a83-3a17-432a-9476-d5df04a78549\",\"tenantId\":\"1d876899-4898-434d-bb81-20fd15cc7d98\",\"type\":\"None\",\"userAssignedIdentities\":{}},\"location\":\"wfqkquj\",\"tags\":{\"xtccmg\":\"uyonobglaoc\",\"wfudwpzntxhdzhl\":\"udxytlmoyrx\",\"hckfrlhrx\":\"qj\",\"ca\":\"bkyvp\"},\"id\":\"uzbpzkafku\",\"name\":\"b\",\"type\":\"rnwb\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.monitors().listByResourceGroup("cwscwsvlx", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("wfqkquj", response.iterator().next().location()); - Assertions.assertEquals("uyonobglaoc", response.iterator().next().tags().get("xtccmg")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.iterator().next().identity().type()); - Assertions.assertEquals("eoveilovnotyf", response.iterator().next().newRelicAccountProperties().userId()); - Assertions.assertEquals("k", response.iterator().next().userInfo().firstName()); - Assertions.assertEquals("x", response.iterator().next().userInfo().lastName()); - Assertions.assertEquals("bttk", response.iterator().next().userInfo().emailAddress()); - Assertions.assertEquals("ywpnvjt", response.iterator().next().userInfo().phoneNumber()); - Assertions.assertEquals("nermcl", response.iterator().next().userInfo().country()); - Assertions.assertEquals(UsageType.PAYG, response.iterator().next().planData().usageType()); - Assertions.assertEquals(BillingCycle.YEARLY, response.iterator().next().planData().billingCycle()); - Assertions.assertEquals("uscrpabgyepsb", response.iterator().next().planData().planDetails()); - Assertions - .assertEquals( - OffsetDateTime.parse("2021-01-26T13:24:27Z"), response.iterator().next().planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, response.iterator().next().orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, response.iterator().next().accountCreationSource()); - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsMockTests.java index ec67050e4df4f..99cda390af124 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListHostsMockTests.java @@ -33,46 +33,30 @@ public void testListHosts() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"vmId\":\"otftpvjzbexilz\",\"agentVersion\":\"fqqnvwpmqtaruo\",\"agentStatus\":\"mkcjhwqytjrybn\"}]}"; + String responseStr + = "{\"value\":[{\"vmId\":\"koievseo\",\"agentVersion\":\"q\",\"agentStatus\":\"ltmuwlauwzizx\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager - .monitors() - .listHosts( - "bfovasrruvwbhsq", - "sub", - new HostsGetRequest().withVmIds(Arrays.asList("birx")).withUserEmail("pybsrfbjfdtw"), - com.azure.core.util.Context.NONE); - - Assertions.assertEquals("otftpvjzbexilz", response.iterator().next().vmId()); - Assertions.assertEquals("fqqnvwpmqtaruo", response.iterator().next().agentVersion()); - Assertions.assertEquals("mkcjhwqytjrybn", response.iterator().next().agentStatus()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.monitors().listHosts("ur", "odkwobd", new HostsGetRequest() + .withVmIds(Arrays.asList("tibqdxbxwakb", "gqxndlkzgxhuripl", "podxunkb", "bxmubyynt")).withUserEmail("lrb"), + com.azure.core.util.Context.NONE); + + Assertions.assertEquals("koievseo", response.iterator().next().vmId()); + Assertions.assertEquals("q", response.iterator().next().agentVersion()); + Assertions.assertEquals("ltmuwlauwzizx", response.iterator().next().agentStatus()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesMockTests.java new file mode 100644 index 0000000000000..6c1ade3ee7a82 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListLinkedResourcesMockTests.java @@ -0,0 +1,56 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; +import com.azure.resourcemanager.newrelicobservability.models.LinkedResource; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class MonitorsListLinkedResourcesMockTests { + @Test + public void testListLinkedResources() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{\"value\":[{\"id\":\"xzxcl\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.monitors().listLinkedResources("phsdyhto", "fikdowwqu", com.azure.core.util.Context.NONE); + + Assertions.assertEquals("xzxcl", response.iterator().next().id()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMockTests.java deleted file mode 100644 index a095504d3229a..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMockTests.java +++ /dev/null @@ -1,88 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResource; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class MonitorsListMockTests { - @Test - public void testList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Creating\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"niwdjsw\",\"newRelicAccountProperties\":{\"userId\":\"dbpgnxytxhp\"},\"userInfo\":{\"firstName\":\"zabglcuhxwt\",\"lastName\":\"yqiklbbovplwzb\",\"emailAddress\":\"gy\",\"phoneNumber\":\"uosvmkfssxqukk\",\"country\":\"l\"},\"planData\":{\"usageType\":\"COMMITTED\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"jzkdeslpvlopwi\",\"effectiveDate\":\"2021-09-12T13:52:28Z\"},\"liftrResourceCategory\":\"Unknown\",\"liftrResourcePreference\":1973154307,\"orgCreationSource\":\"LIFTR\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"75dd3c6a-354c-4390-a355-7aa10b3b870c\",\"tenantId\":\"f4f159ab-7620-4931-a69d-a7b80d4e817a\",\"type\":\"None\",\"userAssignedIdentities\":{}},\"location\":\"umnyqu\",\"tags\":{\"hsmtxpsiebtfhvp\":\"eojnabc\"},\"id\":\"sapskr\",\"name\":\"qmhjjdhtld\",\"type\":\"kyzxuutk\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = manager.monitors().list(com.azure.core.util.Context.NONE); - - Assertions.assertEquals("umnyqu", response.iterator().next().location()); - Assertions.assertEquals("eojnabc", response.iterator().next().tags().get("hsmtxpsiebtfhvp")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, response.iterator().next().identity().type()); - Assertions.assertEquals("dbpgnxytxhp", response.iterator().next().newRelicAccountProperties().userId()); - Assertions.assertEquals("zabglcuhxwt", response.iterator().next().userInfo().firstName()); - Assertions.assertEquals("yqiklbbovplwzb", response.iterator().next().userInfo().lastName()); - Assertions.assertEquals("gy", response.iterator().next().userInfo().emailAddress()); - Assertions.assertEquals("uosvmkfssxqukk", response.iterator().next().userInfo().phoneNumber()); - Assertions.assertEquals("l", response.iterator().next().userInfo().country()); - Assertions.assertEquals(UsageType.COMMITTED, response.iterator().next().planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, response.iterator().next().planData().billingCycle()); - Assertions.assertEquals("jzkdeslpvlopwi", response.iterator().next().planData().planDetails()); - Assertions - .assertEquals( - OffsetDateTime.parse("2021-09-12T13:52:28Z"), response.iterator().next().planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.LIFTR, response.iterator().next().orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, response.iterator().next().accountCreationSource()); - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesMockTests.java index c74206d253e7a..86de7a0c106f2 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsListMonitoredResourcesMockTests.java @@ -33,42 +33,31 @@ public void testListMonitoredResources() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"id\":\"eh\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"ygmi\",\"sendingLogs\":\"Disabled\",\"reasonForLogsStatus\":\"zdnds\"}]}"; + String responseStr + = "{\"value\":[{\"id\":\"orppxebmnzbtb\",\"sendingMetrics\":\"Enabled\",\"reasonForMetricsStatus\":\"lkfg\",\"sendingLogs\":\"Enabled\",\"reasonForLogsStatus\":\"euel\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.monitors().listMonitoredResources("jewgdrjerv", "aen", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.monitors().listMonitoredResources("mpgcjefuzmuvpbt", "d", com.azure.core.util.Context.NONE); - Assertions.assertEquals("eh", response.iterator().next().id()); + Assertions.assertEquals("orppxebmnzbtb", response.iterator().next().id()); Assertions.assertEquals(SendingMetricsStatus.ENABLED, response.iterator().next().sendingMetrics()); - Assertions.assertEquals("ygmi", response.iterator().next().reasonForMetricsStatus()); - Assertions.assertEquals(SendingLogsStatus.DISABLED, response.iterator().next().sendingLogs()); - Assertions.assertEquals("zdnds", response.iterator().next().reasonForLogsStatus()); + Assertions.assertEquals("lkfg", response.iterator().next().reasonForMetricsStatus()); + Assertions.assertEquals(SendingLogsStatus.ENABLED, response.iterator().next().sendingLogs()); + Assertions.assertEquals("euel", response.iterator().next().reasonForLogsStatus()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingWithResponseMockTests.java deleted file mode 100644 index b269b94c6abb2..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/MonitorsSwitchBillingWithResponseMockTests.java +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResource; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.PlanData; -import com.azure.resourcemanager.newrelicobservability.models.SwitchBillingRequest; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class MonitorsSwitchBillingWithResponseMockTests { - @Test - public void testSwitchBillingWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"provisioningState\":\"Updating\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"txmedj\",\"newRelicAccountProperties\":{\"userId\":\"lynqwwncwzzh\"},\"userInfo\":{\"firstName\":\"gucnapkte\",\"lastName\":\"llwptfdy\",\"emailAddress\":\"fqbuaceopzf\",\"phoneNumber\":\"hhuao\",\"country\":\"pcqeqx\"},\"planData\":{\"usageType\":\"COMMITTED\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"xcto\",\"effectiveDate\":\"2021-01-13T03:12:52Z\"},\"liftrResourceCategory\":\"MonitorLogs\",\"liftrResourcePreference\":1740681281,\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"LIFTR\"},\"identity\":{\"principalId\":\"1151cee7-9e16-4851-9d31-70346fb26ae6\",\"tenantId\":\"7bd4edf3-d1c7-4626-a15b-a59b946f8cc5\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"location\":\"bunrmfqjhhk\",\"tags\":{\"vkr\":\"vjymjhxxjyngud\",\"szjfauvjfdxxivet\":\"swbxqz\",\"qaqtdoqmcbxvwvxy\":\"t\",\"obl\":\"lqbhsf\"},\"id\":\"tkblmpewww\",\"name\":\"bkrvrnsvshqj\",\"type\":\"hxcr\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - NewRelicMonitorResource response = - manager - .monitors() - .switchBillingWithResponse( - "xyawj", - "yaqcslyjpkiidz", - new SwitchBillingRequest() - .withAzureResourceId("xznelixhnrztf") - .withOrganizationId("hb") - .withPlanData( - new PlanData() - .withUsageType(UsageType.COMMITTED) - .withBillingCycle(BillingCycle.MONTHLY) - .withPlanDetails("ulppggdtpnapnyir") - .withEffectiveDate(OffsetDateTime.parse("2021-02-09T03:01:47Z"))) - .withUserEmail("hpigv"), - com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("bunrmfqjhhk", response.location()); - Assertions.assertEquals("vjymjhxxjyngud", response.tags().get("vkr")); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); - Assertions.assertEquals("lynqwwncwzzh", response.newRelicAccountProperties().userId()); - Assertions.assertEquals("gucnapkte", response.userInfo().firstName()); - Assertions.assertEquals("llwptfdy", response.userInfo().lastName()); - Assertions.assertEquals("fqbuaceopzf", response.userInfo().emailAddress()); - Assertions.assertEquals("hhuao", response.userInfo().phoneNumber()); - Assertions.assertEquals("pcqeqx", response.userInfo().country()); - Assertions.assertEquals(UsageType.COMMITTED, response.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, response.planData().billingCycle()); - Assertions.assertEquals("xcto", response.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-01-13T03:12:52Z"), response.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, response.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.LIFTR, response.accountCreationSource()); - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceInnerTests.java deleted file mode 100644 index 39395af8acdd5..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceInnerTests.java +++ /dev/null @@ -1,112 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceInner; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicAccountProperties; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.PlanData; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import com.azure.resourcemanager.newrelicobservability.models.UserAssignedIdentity; -import com.azure.resourcemanager.newrelicobservability.models.UserInfo; -import java.time.OffsetDateTime; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class NewRelicMonitorResourceInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - NewRelicMonitorResourceInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Accepted\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"ltmrldh\",\"newRelicAccountProperties\":{\"userId\":\"zzd\"},\"userInfo\":{\"firstName\":\"cdgea\",\"lastName\":\"gphuticndvka\",\"emailAddress\":\"wyiftyhxhur\",\"phoneNumber\":\"ftyxolniw\",\"country\":\"cukjf\"},\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"lryplwckbasyy\",\"effectiveDate\":\"2021-05-24T00:31:31Z\"},\"liftrResourceCategory\":\"MonitorLogs\",\"liftrResourcePreference\":1048772360,\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"LIFTR\"},\"identity\":{\"principalId\":\"17917958-feea-475b-924d-d460356abeb7\",\"tenantId\":\"0c6581b4-9dc9-4139-9af6-885e2ea5f76d\",\"type\":\"None\",\"userAssignedIdentities\":{\"nqgoulzndli\":{\"principalId\":\"1441313f-47dc-4ce5-b867-17a8c2992b93\",\"clientId\":\"0c0cf59f-46a6-479e-954b-b32c3748fb4f\"}}},\"location\":\"yqkgfg\",\"tags\":{\"qedqytbciqfoufl\":\"adgakeqsrxybz\",\"glougpbk\":\"mnkzsmod\",\"uqktap\":\"tmut\",\"kdosvqw\":\"pwgcuertu\"},\"id\":\"bmdg\",\"name\":\"bjf\",\"type\":\"dgmb\"}") - .toObject(NewRelicMonitorResourceInner.class); - Assertions.assertEquals("yqkgfg", model.location()); - Assertions.assertEquals("adgakeqsrxybz", model.tags().get("qedqytbciqfoufl")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("zzd", model.newRelicAccountProperties().userId()); - Assertions.assertEquals("cdgea", model.userInfo().firstName()); - Assertions.assertEquals("gphuticndvka", model.userInfo().lastName()); - Assertions.assertEquals("wyiftyhxhur", model.userInfo().emailAddress()); - Assertions.assertEquals("ftyxolniw", model.userInfo().phoneNumber()); - Assertions.assertEquals("cukjf", model.userInfo().country()); - Assertions.assertEquals(UsageType.PAYG, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); - Assertions.assertEquals("lryplwckbasyy", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-24T00:31:31Z"), model.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.LIFTR, model.accountCreationSource()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - NewRelicMonitorResourceInner model = - new NewRelicMonitorResourceInner() - .withLocation("yqkgfg") - .withTags( - mapOf( - "qedqytbciqfoufl", - "adgakeqsrxybz", - "glougpbk", - "mnkzsmod", - "uqktap", - "tmut", - "kdosvqw", - "pwgcuertu")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf("nqgoulzndli", new UserAssignedIdentity()))) - .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("zzd")) - .withUserInfo( - new UserInfo() - .withFirstName("cdgea") - .withLastName("gphuticndvka") - .withEmailAddress("wyiftyhxhur") - .withPhoneNumber("ftyxolniw") - .withCountry("cukjf")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.MONTHLY) - .withPlanDetails("lryplwckbasyy") - .withEffectiveDate(OffsetDateTime.parse("2021-05-24T00:31:31Z"))) - .withOrgCreationSource(OrgCreationSource.NEWRELIC) - .withAccountCreationSource(AccountCreationSource.LIFTR); - model = BinaryData.fromObject(model).toObject(NewRelicMonitorResourceInner.class); - Assertions.assertEquals("yqkgfg", model.location()); - Assertions.assertEquals("adgakeqsrxybz", model.tags().get("qedqytbciqfoufl")); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("zzd", model.newRelicAccountProperties().userId()); - Assertions.assertEquals("cdgea", model.userInfo().firstName()); - Assertions.assertEquals("gphuticndvka", model.userInfo().lastName()); - Assertions.assertEquals("wyiftyhxhur", model.userInfo().emailAddress()); - Assertions.assertEquals("ftyxolniw", model.userInfo().phoneNumber()); - Assertions.assertEquals("cukjf", model.userInfo().country()); - Assertions.assertEquals(UsageType.PAYG, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); - Assertions.assertEquals("lryplwckbasyy", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-24T00:31:31Z"), model.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.LIFTR, model.accountCreationSource()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceListResultTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceListResultTests.java deleted file mode 100644 index a508f9d0af21e..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceListResultTests.java +++ /dev/null @@ -1,105 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.newrelicobservability.fluent.models.NewRelicMonitorResourceInner; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResourceListResult; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class NewRelicMonitorResourceListResultTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - NewRelicMonitorResourceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Creating\",\"monitoringStatus\":\"Enabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"dio\",\"liftrResourceCategory\":\"Unknown\",\"liftrResourcePreference\":2028392349,\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"08951b99-ca13-4bdc-8322-11806828a442\",\"tenantId\":\"79c88631-a94e-42f2-98e2-666dbc928c5f\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{}},\"location\":\"oacctaza\",\"tags\":{\"fdosyg\":\"lahbcryff\"},\"id\":\"xpaojakhmsbz\",\"name\":\"hcrzevd\",\"type\":\"hlxaolthqtr\"},{\"properties\":{\"provisioningState\":\"Accepted\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Suspended\",\"marketplaceSubscriptionId\":\"s\",\"liftrResourceCategory\":\"Unknown\",\"liftrResourcePreference\":1595507269,\"orgCreationSource\":\"LIFTR\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"e5702e62-e947-47d1-91f9-508630ba078a\",\"tenantId\":\"8352b2cc-4a40-45aa-9eab-5bae81f94e26\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"location\":\"lluwfzitonpeq\",\"tags\":{\"xypininmayhuybbk\":\"kjlxofpdvhpf\",\"vamih\":\"odepoogin\",\"vyevcciqi\":\"ognarxzxtheotus\",\"zrnf\":\"nhungbw\"},\"id\":\"gxg\",\"name\":\"spemvtzfk\",\"type\":\"fublj\"},{\"properties\":{\"provisioningState\":\"Deleting\",\"monitoringStatus\":\"Disabled\",\"marketplaceSubscriptionStatus\":\"Active\",\"marketplaceSubscriptionId\":\"aeqjhqjbasvms\",\"liftrResourceCategory\":\"MonitorLogs\",\"liftrResourcePreference\":951964300,\"orgCreationSource\":\"LIFTR\",\"accountCreationSource\":\"NEWRELIC\"},\"identity\":{\"principalId\":\"7ea6bb96-e02c-41b8-bb30-8f33eb3a80a4\",\"tenantId\":\"27310914-912e-4105-abf8-2ac41b52c5d5\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"location\":\"wrwclxxwrljd\",\"tags\":{\"dkwt\":\"kcqvkocrc\",\"ssainqpjwnzll\":\"hxbnjbiksqrg\",\"bvmgxsabkyqduuji\":\"fmppe\"},\"id\":\"c\",\"name\":\"czdzev\",\"type\":\"dhkrwpdappdsbdk\"}],\"nextLink\":\"rwjfe\"}") - .toObject(NewRelicMonitorResourceListResult.class); - Assertions.assertEquals("oacctaza", model.value().get(0).location()); - Assertions.assertEquals("lahbcryff", model.value().get(0).tags().get("fdosyg")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.value().get(0).identity().type()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.value().get(0).orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.value().get(0).accountCreationSource()); - Assertions.assertEquals("rwjfe", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - NewRelicMonitorResourceListResult model = - new NewRelicMonitorResourceListResult() - .withValue( - Arrays - .asList( - new NewRelicMonitorResourceInner() - .withLocation("oacctaza") - .withTags(mapOf("fdosyg", "lahbcryff")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities(mapOf())) - .withOrgCreationSource(OrgCreationSource.NEWRELIC) - .withAccountCreationSource(AccountCreationSource.NEWRELIC), - new NewRelicMonitorResourceInner() - .withLocation("lluwfzitonpeq") - .withTags( - mapOf( - "xypininmayhuybbk", - "kjlxofpdvhpf", - "vamih", - "odepoogin", - "vyevcciqi", - "ognarxzxtheotus", - "zrnf", - "nhungbw")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED) - .withUserAssignedIdentities(mapOf())) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.NEWRELIC), - new NewRelicMonitorResourceInner() - .withLocation("wrwclxxwrljd") - .withTags( - mapOf( - "dkwt", - "kcqvkocrc", - "ssainqpjwnzll", - "hxbnjbiksqrg", - "bvmgxsabkyqduuji", - "fmppe")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED) - .withUserAssignedIdentities(mapOf())) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.NEWRELIC))) - .withNextLink("rwjfe"); - model = BinaryData.fromObject(model).toObject(NewRelicMonitorResourceListResult.class); - Assertions.assertEquals("oacctaza", model.value().get(0).location()); - Assertions.assertEquals("lahbcryff", model.value().get(0).tags().get("fdosyg")); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.value().get(0).identity().type()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.value().get(0).orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.value().get(0).accountCreationSource()); - Assertions.assertEquals("rwjfe", model.nextLink()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceUpdateTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceUpdateTests.java deleted file mode 100644 index e6fa50cb0c277..0000000000000 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicMonitorResourceUpdateTests.java +++ /dev/null @@ -1,101 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.newrelicobservability.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentity; -import com.azure.resourcemanager.newrelicobservability.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicAccountProperties; -import com.azure.resourcemanager.newrelicobservability.models.NewRelicMonitorResourceUpdate; -import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; -import com.azure.resourcemanager.newrelicobservability.models.PlanData; -import com.azure.resourcemanager.newrelicobservability.models.UsageType; -import com.azure.resourcemanager.newrelicobservability.models.UserAssignedIdentity; -import com.azure.resourcemanager.newrelicobservability.models.UserInfo; -import java.time.OffsetDateTime; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class NewRelicMonitorResourceUpdateTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - NewRelicMonitorResourceUpdate model = - BinaryData - .fromString( - "{\"identity\":{\"principalId\":\"8af364d0-2423-4107-bc81-1b490f4f4095\",\"tenantId\":\"c20fd884-bcd8-4a5a-9fa4-24f086f2191f\",\"type\":\"None\",\"userAssignedIdentities\":{\"ciwqvhk\":{\"principalId\":\"ac4a2a96-2e50-41e8-8716-1e44c18db630\",\"clientId\":\"aefefa77-aa8d-4e04-8ad5-d6ec26306953\"},\"uigdtopbobjog\":{\"principalId\":\"011ce17c-e213-4ecb-a551-6bb9fd7292be\",\"clientId\":\"7edf55c0-8c77-4501-ab62-0461a07096ec\"}}},\"tags\":{\"a\":\"w\"},\"properties\":{\"newRelicAccountProperties\":{\"userId\":\"rzayv\"},\"userInfo\":{\"firstName\":\"f\",\"lastName\":\"otkftutqxlngx\",\"emailAddress\":\"fgugnxkrxdqmid\",\"phoneNumber\":\"hzrvqd\",\"country\":\"bhj\"},\"planData\":{\"usageType\":\"COMMITTED\",\"billingCycle\":\"YEARLY\",\"planDetails\":\"qfbow\",\"effectiveDate\":\"2021-05-19T19:15:18Z\"},\"orgCreationSource\":\"LIFTR\",\"accountCreationSource\":\"NEWRELIC\"}}") - .toObject(NewRelicMonitorResourceUpdate.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("w", model.tags().get("a")); - Assertions.assertEquals("rzayv", model.newRelicAccountProperties().userId()); - Assertions.assertEquals("f", model.userInfo().firstName()); - Assertions.assertEquals("otkftutqxlngx", model.userInfo().lastName()); - Assertions.assertEquals("fgugnxkrxdqmid", model.userInfo().emailAddress()); - Assertions.assertEquals("hzrvqd", model.userInfo().phoneNumber()); - Assertions.assertEquals("bhj", model.userInfo().country()); - Assertions.assertEquals(UsageType.COMMITTED, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.YEARLY, model.planData().billingCycle()); - Assertions.assertEquals("qfbow", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-19T19:15:18Z"), model.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.LIFTR, model.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.accountCreationSource()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - NewRelicMonitorResourceUpdate model = - new NewRelicMonitorResourceUpdate() - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities( - mapOf("ciwqvhk", new UserAssignedIdentity(), "uigdtopbobjog", new UserAssignedIdentity()))) - .withTags(mapOf("a", "w")) - .withNewRelicAccountProperties(new NewRelicAccountProperties().withUserId("rzayv")) - .withUserInfo( - new UserInfo() - .withFirstName("f") - .withLastName("otkftutqxlngx") - .withEmailAddress("fgugnxkrxdqmid") - .withPhoneNumber("hzrvqd") - .withCountry("bhj")) - .withPlanData( - new PlanData() - .withUsageType(UsageType.COMMITTED) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("qfbow") - .withEffectiveDate(OffsetDateTime.parse("2021-05-19T19:15:18Z"))) - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.NEWRELIC); - model = BinaryData.fromObject(model).toObject(NewRelicMonitorResourceUpdate.class); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("w", model.tags().get("a")); - Assertions.assertEquals("rzayv", model.newRelicAccountProperties().userId()); - Assertions.assertEquals("f", model.userInfo().firstName()); - Assertions.assertEquals("otkftutqxlngx", model.userInfo().lastName()); - Assertions.assertEquals("fgugnxkrxdqmid", model.userInfo().emailAddress()); - Assertions.assertEquals("hzrvqd", model.userInfo().phoneNumber()); - Assertions.assertEquals("bhj", model.userInfo().country()); - Assertions.assertEquals(UsageType.COMMITTED, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.YEARLY, model.planData().billingCycle()); - Assertions.assertEquals("qfbow", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-05-19T19:15:18Z"), model.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.LIFTR, model.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.accountCreationSource()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicSingleSignOnPropertiesTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicSingleSignOnPropertiesTests.java index 3f64b074eae53..41ec33a390b16 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicSingleSignOnPropertiesTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/NewRelicSingleSignOnPropertiesTests.java @@ -13,29 +13,24 @@ public final class NewRelicSingleSignOnPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - NewRelicSingleSignOnProperties model = - BinaryData - .fromString( - "{\"singleSignOnState\":\"Initial\",\"enterpriseAppId\":\"itnwuizgazxufi\",\"singleSignOnUrl\":\"ckyfih\",\"provisioningState\":\"NotSpecified\"}") - .toObject(NewRelicSingleSignOnProperties.class); - Assertions.assertEquals(SingleSignOnStates.INITIAL, model.singleSignOnState()); - Assertions.assertEquals("itnwuizgazxufi", model.enterpriseAppId()); - Assertions.assertEquals("ckyfih", model.singleSignOnUrl()); - Assertions.assertEquals(ProvisioningState.NOT_SPECIFIED, model.provisioningState()); + NewRelicSingleSignOnProperties model = BinaryData.fromString( + "{\"singleSignOnState\":\"Enable\",\"enterpriseAppId\":\"o\",\"singleSignOnUrl\":\"si\",\"provisioningState\":\"Deleting\"}") + .toObject(NewRelicSingleSignOnProperties.class); + Assertions.assertEquals(SingleSignOnStates.ENABLE, model.singleSignOnState()); + Assertions.assertEquals("o", model.enterpriseAppId()); + Assertions.assertEquals("si", model.singleSignOnUrl()); + Assertions.assertEquals(ProvisioningState.DELETING, model.provisioningState()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - NewRelicSingleSignOnProperties model = - new NewRelicSingleSignOnProperties() - .withSingleSignOnState(SingleSignOnStates.INITIAL) - .withEnterpriseAppId("itnwuizgazxufi") - .withSingleSignOnUrl("ckyfih") - .withProvisioningState(ProvisioningState.NOT_SPECIFIED); + NewRelicSingleSignOnProperties model + = new NewRelicSingleSignOnProperties().withSingleSignOnState(SingleSignOnStates.ENABLE) + .withEnterpriseAppId("o").withSingleSignOnUrl("si").withProvisioningState(ProvisioningState.DELETING); model = BinaryData.fromObject(model).toObject(NewRelicSingleSignOnProperties.class); - Assertions.assertEquals(SingleSignOnStates.INITIAL, model.singleSignOnState()); - Assertions.assertEquals("itnwuizgazxufi", model.enterpriseAppId()); - Assertions.assertEquals("ckyfih", model.singleSignOnUrl()); - Assertions.assertEquals(ProvisioningState.NOT_SPECIFIED, model.provisioningState()); + Assertions.assertEquals(SingleSignOnStates.ENABLE, model.singleSignOnState()); + Assertions.assertEquals("o", model.enterpriseAppId()); + Assertions.assertEquals("si", model.singleSignOnUrl()); + Assertions.assertEquals(ProvisioningState.DELETING, model.provisioningState()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationDisplayTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationDisplayTests.java index eb95705cc2d70..b350856cac806 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationDisplayTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationDisplayTests.java @@ -10,11 +10,9 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"yrtih\",\"resource\":\"tijbpzvgnwzsymgl\",\"operation\":\"fcyzkohdbihanufh\",\"description\":\"bj\"}") - .toObject(OperationDisplay.class); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"yrtih\",\"resource\":\"tijbpzvgnwzsymgl\",\"operation\":\"fcyzkohdbihanufh\",\"description\":\"bj\"}") + .toObject(OperationDisplay.class); } @org.junit.jupiter.api.Test diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationInnerTests.java index 17f43b2e1daaf..222e69c373cca 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationInnerTests.java @@ -11,11 +11,9 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"usarhmofc\",\"isDataAction\":false,\"display\":{\"provider\":\"urkdtmlx\",\"resource\":\"kuksjtxukcdm\",\"operation\":\"rcryuanzwuxzdxta\",\"description\":\"lhmwhfpmrqobm\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") - .toObject(OperationInner.class); + OperationInner model = BinaryData.fromString( + "{\"name\":\"usarhmofc\",\"isDataAction\":false,\"display\":{\"provider\":\"urkdtmlx\",\"resource\":\"kuksjtxukcdm\",\"operation\":\"rcryuanzwuxzdxta\",\"description\":\"lhmwhfpmrqobm\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") + .toObject(OperationInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationListResultTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationListResultTests.java index 2e507159686a0..fcd014fed12d9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationListResultTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationListResultTests.java @@ -10,11 +10,9 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"quvgjxpybczme\",\"isDataAction\":true,\"display\":{\"provider\":\"pbsphrupidgs\",\"resource\":\"bejhphoycmsxa\",\"operation\":\"hdxbmtqio\",\"description\":\"zehtbmu\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"izhwlrxy\",\"isDataAction\":false,\"display\":{\"provider\":\"ijgkdm\",\"resource\":\"azlobcufpdznrbt\",\"operation\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"rifkwm\",\"isDataAction\":true,\"display\":{\"provider\":\"izntocipao\",\"resource\":\"jpsq\",\"operation\":\"mpoyfd\",\"description\":\"ogknygjofjdd\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"upewnwreitjzy\"}") - .toObject(OperationListResult.class); + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"quvgjxpybczme\",\"isDataAction\":true,\"display\":{\"provider\":\"pbsphrupidgs\",\"resource\":\"bejhphoycmsxa\",\"operation\":\"hdxbmtqio\",\"description\":\"zehtbmu\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"izhwlrxy\",\"isDataAction\":false,\"display\":{\"provider\":\"ijgkdm\",\"resource\":\"azlobcufpdznrbt\",\"operation\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"rifkwm\",\"isDataAction\":true,\"display\":{\"provider\":\"izntocipao\",\"resource\":\"jpsq\",\"operation\":\"mpoyfd\",\"description\":\"ogknygjofjdd\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"upewnwreitjzy\"}") + .toObject(OperationListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListMockTests.java index 56422a5cd7319..34b0f5719bfb5 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OperationsListMockTests.java @@ -30,35 +30,25 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"uynhijg\",\"isDataAction\":false,\"display\":{\"provider\":\"siarbutrcvpn\",\"resource\":\"zmhjrunmp\",\"operation\":\"tdbhrbnla\",\"description\":\"xmyskp\"},\"origin\":\"user\",\"actionType\":\"Internal\"}]}"; + String responseStr + = "{\"value\":[{\"name\":\"uusdttouwa\",\"isDataAction\":false,\"display\":{\"provider\":\"v\",\"resource\":\"lns\",\"operation\":\"bxwyjsflhhcaa\",\"description\":\"jixisxyawjoyaqcs\"},\"origin\":\"system\",\"actionType\":\"Internal\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); + } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationInfoTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationInfoTests.java index 63d7fbe57c8a8..15c572cc37d19 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationInfoTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationInfoTests.java @@ -11,14 +11,15 @@ public final class OrganizationInfoTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OrganizationInfo model = BinaryData.fromString("{\"organizationId\":\"zw\"}").toObject(OrganizationInfo.class); - Assertions.assertEquals("zw", model.organizationId()); + OrganizationInfo model + = BinaryData.fromString("{\"organizationId\":\"amiheognarxz\"}").toObject(OrganizationInfo.class); + Assertions.assertEquals("amiheognarxz", model.organizationId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OrganizationInfo model = new OrganizationInfo().withOrganizationId("zw"); + OrganizationInfo model = new OrganizationInfo().withOrganizationId("amiheognarxz"); model = BinaryData.fromObject(model).toObject(OrganizationInfo.class); - Assertions.assertEquals("zw", model.organizationId()); + Assertions.assertEquals("amiheognarxz", model.organizationId()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationPropertiesTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationPropertiesTests.java index fd2c0dc1577ea..5c7511db25359 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationPropertiesTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationPropertiesTests.java @@ -12,26 +12,21 @@ public final class OrganizationPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OrganizationProperties model = - BinaryData - .fromString( - "{\"organizationId\":\"jwzrl\",\"organizationName\":\"mcl\",\"billingSource\":\"NEWRELIC\"}") - .toObject(OrganizationProperties.class); - Assertions.assertEquals("jwzrl", model.organizationId()); - Assertions.assertEquals("mcl", model.organizationName()); + OrganizationProperties model = BinaryData.fromString( + "{\"organizationId\":\"utduqktapspwgcu\",\"organizationName\":\"tumkdosvqwhbm\",\"billingSource\":\"NEWRELIC\"}") + .toObject(OrganizationProperties.class); + Assertions.assertEquals("utduqktapspwgcu", model.organizationId()); + Assertions.assertEquals("tumkdosvqwhbm", model.organizationName()); Assertions.assertEquals(BillingSource.NEWRELIC, model.billingSource()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OrganizationProperties model = - new OrganizationProperties() - .withOrganizationId("jwzrl") - .withOrganizationName("mcl") - .withBillingSource(BillingSource.NEWRELIC); + OrganizationProperties model = new OrganizationProperties().withOrganizationId("utduqktapspwgcu") + .withOrganizationName("tumkdosvqwhbm").withBillingSource(BillingSource.NEWRELIC); model = BinaryData.fromObject(model).toObject(OrganizationProperties.class); - Assertions.assertEquals("jwzrl", model.organizationId()); - Assertions.assertEquals("mcl", model.organizationName()); + Assertions.assertEquals("utduqktapspwgcu", model.organizationId()); + Assertions.assertEquals("tumkdosvqwhbm", model.organizationName()); Assertions.assertEquals(BillingSource.NEWRELIC, model.billingSource()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationResourceInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationResourceInnerTests.java index d76476215ebdd..32d8a31f9f08e 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationResourceInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationResourceInnerTests.java @@ -12,26 +12,21 @@ public final class OrganizationResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OrganizationResourceInner model = - BinaryData - .fromString( - "{\"properties\":{\"organizationId\":\"okaj\",\"organizationName\":\"npime\",\"billingSource\":\"AZURE\"},\"id\":\"xgcp\",\"name\":\"dg\",\"type\":\"aajrm\"}") - .toObject(OrganizationResourceInner.class); - Assertions.assertEquals("okaj", model.organizationId()); - Assertions.assertEquals("npime", model.organizationName()); - Assertions.assertEquals(BillingSource.AZURE, model.billingSource()); + OrganizationResourceInner model = BinaryData.fromString( + "{\"properties\":{\"organizationId\":\"kgfg\",\"organizationName\":\"madgakeqsrxyb\",\"billingSource\":\"NEWRELIC\"},\"id\":\"dqytbciqfouflmm\",\"name\":\"kzsmodm\",\"type\":\"lougpbkw\"}") + .toObject(OrganizationResourceInner.class); + Assertions.assertEquals("kgfg", model.organizationId()); + Assertions.assertEquals("madgakeqsrxyb", model.organizationName()); + Assertions.assertEquals(BillingSource.NEWRELIC, model.billingSource()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OrganizationResourceInner model = - new OrganizationResourceInner() - .withOrganizationId("okaj") - .withOrganizationName("npime") - .withBillingSource(BillingSource.AZURE); + OrganizationResourceInner model = new OrganizationResourceInner().withOrganizationId("kgfg") + .withOrganizationName("madgakeqsrxyb").withBillingSource(BillingSource.NEWRELIC); model = BinaryData.fromObject(model).toObject(OrganizationResourceInner.class); - Assertions.assertEquals("okaj", model.organizationId()); - Assertions.assertEquals("npime", model.organizationName()); - Assertions.assertEquals(BillingSource.AZURE, model.billingSource()); + Assertions.assertEquals("kgfg", model.organizationId()); + Assertions.assertEquals("madgakeqsrxyb", model.organizationName()); + Assertions.assertEquals(BillingSource.NEWRELIC, model.billingSource()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListMockTests.java index 28529ffbb52b4..6ddce9f2c1410 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListMockTests.java @@ -32,40 +32,29 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"organizationId\":\"twuoegrpkhjwni\",\"organizationName\":\"sluicpdggkzz\",\"billingSource\":\"NEWRELIC\"},\"id\":\"mpaxmodfvuefywsb\",\"name\":\"fvmwy\",\"type\":\"rfouyftaakcpw\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"organizationId\":\"cjwvn\",\"organizationName\":\"ld\",\"billingSource\":\"AZURE\"},\"id\":\"cxrslpmutwuoe\",\"name\":\"rpkhjwn\",\"type\":\"yqsluic\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.organizations().list("obzdopcjwvnhdl", "wmgxcxrsl", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.organizations().list("rhvoods", "tbobz", com.azure.core.util.Context.NONE); - Assertions.assertEquals("twuoegrpkhjwni", response.iterator().next().organizationId()); - Assertions.assertEquals("sluicpdggkzz", response.iterator().next().organizationName()); - Assertions.assertEquals(BillingSource.NEWRELIC, response.iterator().next().billingSource()); + Assertions.assertEquals("cjwvn", response.iterator().next().organizationId()); + Assertions.assertEquals("ld", response.iterator().next().organizationName()); + Assertions.assertEquals(BillingSource.AZURE, response.iterator().next().billingSource()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListResponseTests.java index 1da9f7158aa02..9a0ee97ac5ea8 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListResponseTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/OrganizationsListResponseTests.java @@ -14,41 +14,29 @@ public final class OrganizationsListResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OrganizationsListResponse model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"organizationId\":\"irsoodqxhcrmnohj\",\"organizationName\":\"kwh\",\"billingSource\":\"NEWRELIC\"},\"id\":\"fiyipjxsqwpgrj\",\"name\":\"znorcj\",\"type\":\"vsnb\"},{\"properties\":{\"organizationId\":\"abnmocpcyshu\",\"organizationName\":\"afbljjgpbtoqcjmk\",\"billingSource\":\"AZURE\"},\"id\":\"bqidtqaj\",\"name\":\"yulpkudjkr\",\"type\":\"khbzhfepgzg\"},{\"properties\":{\"organizationId\":\"zloc\",\"organizationName\":\"c\",\"billingSource\":\"NEWRELIC\"},\"id\":\"rhhbcs\",\"name\":\"l\",\"type\":\"mmajtjaodx\"}],\"nextLink\":\"nbdxk\"}") - .toObject(OrganizationsListResponse.class); - Assertions.assertEquals("irsoodqxhcrmnohj", model.value().get(0).organizationId()); - Assertions.assertEquals("kwh", model.value().get(0).organizationName()); + OrganizationsListResponse model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"organizationId\":\"nqpjwnzlljfmpp\",\"organizationName\":\"bvmgxsabkyqduuji\",\"billingSource\":\"NEWRELIC\"},\"id\":\"zdzevndh\",\"name\":\"rwpdappdsbdkvwrw\",\"type\":\"feusnhut\"},{\"properties\":{\"organizationId\":\"tmrldhugjzzdatq\",\"organizationName\":\"oc\",\"billingSource\":\"AZURE\"},\"id\":\"blgphuticn\",\"name\":\"vkaozwyiftyhxhur\",\"type\":\"k\"},{\"properties\":{\"organizationId\":\"xolniwpwcukjfk\",\"organizationName\":\"awxklr\",\"billingSource\":\"NEWRELIC\"},\"id\":\"ckbasyypndd\",\"name\":\"sgcbac\",\"type\":\"hejkotynqgou\"}],\"nextLink\":\"ndlik\"}") + .toObject(OrganizationsListResponse.class); + Assertions.assertEquals("nqpjwnzlljfmpp", model.value().get(0).organizationId()); + Assertions.assertEquals("bvmgxsabkyqduuji", model.value().get(0).organizationName()); Assertions.assertEquals(BillingSource.NEWRELIC, model.value().get(0).billingSource()); - Assertions.assertEquals("nbdxk", model.nextLink()); + Assertions.assertEquals("ndlik", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OrganizationsListResponse model = - new OrganizationsListResponse() - .withValue( - Arrays - .asList( - new OrganizationResourceInner() - .withOrganizationId("irsoodqxhcrmnohj") - .withOrganizationName("kwh") - .withBillingSource(BillingSource.NEWRELIC), - new OrganizationResourceInner() - .withOrganizationId("abnmocpcyshu") - .withOrganizationName("afbljjgpbtoqcjmk") - .withBillingSource(BillingSource.AZURE), - new OrganizationResourceInner() - .withOrganizationId("zloc") - .withOrganizationName("c") - .withBillingSource(BillingSource.NEWRELIC))) - .withNextLink("nbdxk"); + OrganizationsListResponse model = new OrganizationsListResponse().withValue(Arrays.asList( + new OrganizationResourceInner().withOrganizationId("nqpjwnzlljfmpp") + .withOrganizationName("bvmgxsabkyqduuji").withBillingSource(BillingSource.NEWRELIC), + new OrganizationResourceInner().withOrganizationId("tmrldhugjzzdatq").withOrganizationName("oc") + .withBillingSource(BillingSource.AZURE), + new OrganizationResourceInner().withOrganizationId("xolniwpwcukjfk").withOrganizationName("awxklr") + .withBillingSource(BillingSource.NEWRELIC))) + .withNextLink("ndlik"); model = BinaryData.fromObject(model).toObject(OrganizationsListResponse.class); - Assertions.assertEquals("irsoodqxhcrmnohj", model.value().get(0).organizationId()); - Assertions.assertEquals("kwh", model.value().get(0).organizationName()); + Assertions.assertEquals("nqpjwnzlljfmpp", model.value().get(0).organizationId()); + Assertions.assertEquals("bvmgxsabkyqduuji", model.value().get(0).organizationName()); Assertions.assertEquals(BillingSource.NEWRELIC, model.value().get(0).billingSource()); - Assertions.assertEquals("nbdxk", model.nextLink()); + Assertions.assertEquals("ndlik", model.nextLink()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PartnerBillingEntityTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PartnerBillingEntityTests.java new file mode 100644 index 0000000000000..ef9070a10cbc0 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PartnerBillingEntityTests.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.models.PartnerBillingEntity; +import org.junit.jupiter.api.Assertions; + +public final class PartnerBillingEntityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PartnerBillingEntity model + = BinaryData.fromString("{\"organizationId\":\"dnvowg\",\"organizationName\":\"jugwdkcglhsl\"}") + .toObject(PartnerBillingEntity.class); + Assertions.assertEquals("dnvowg", model.organizationId()); + Assertions.assertEquals("jugwdkcglhsl", model.organizationName()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PartnerBillingEntity model + = new PartnerBillingEntity().withOrganizationId("dnvowg").withOrganizationName("jugwdkcglhsl"); + model = BinaryData.fromObject(model).toObject(PartnerBillingEntity.class); + Assertions.assertEquals("dnvowg", model.organizationId()); + Assertions.assertEquals("jugwdkcglhsl", model.organizationName()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataListResponseTests.java index 3c146b86aa769..ba6234a60686f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataListResponseTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataListResponseTests.java @@ -7,38 +7,46 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.newrelicobservability.fluent.models.PlanDataResourceInner; import com.azure.resourcemanager.newrelicobservability.models.AccountCreationSource; +import com.azure.resourcemanager.newrelicobservability.models.BillingCycle; import com.azure.resourcemanager.newrelicobservability.models.OrgCreationSource; +import com.azure.resourcemanager.newrelicobservability.models.PlanData; import com.azure.resourcemanager.newrelicobservability.models.PlanDataListResponse; +import com.azure.resourcemanager.newrelicobservability.models.UsageType; +import java.time.OffsetDateTime; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class PlanDataListResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PlanDataListResponse model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"orgCreationSource\":\"LIFTR\",\"accountCreationSource\":\"NEWRELIC\"},\"id\":\"zaqsqsycbkbfk\",\"name\":\"ukdkexxppofmxa\",\"type\":\"c\"}],\"nextLink\":\"pg\"}") - .toObject(PlanDataListResponse.class); - Assertions.assertEquals(OrgCreationSource.LIFTR, model.value().get(0).orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.value().get(0).accountCreationSource()); - Assertions.assertEquals("pg", model.nextLink()); + PlanDataListResponse model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"WEEKLY\",\"planDetails\":\"mbe\",\"effectiveDate\":\"2021-10-01T04:14:33Z\"},\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"LIFTR\"},\"id\":\"rolfpfp\",\"name\":\"algbquxigjyjg\",\"type\":\"jaoyfhrtx\"}],\"nextLink\":\"n\"}") + .toObject(PlanDataListResponse.class); + Assertions.assertEquals(UsageType.PAYG, model.value().get(0).planData().usageType()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.value().get(0).planData().billingCycle()); + Assertions.assertEquals("mbe", model.value().get(0).planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-01T04:14:33Z"), + model.value().get(0).planData().effectiveDate()); + Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.value().get(0).orgCreationSource()); + Assertions.assertEquals(AccountCreationSource.LIFTR, model.value().get(0).accountCreationSource()); + Assertions.assertEquals("n", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PlanDataListResponse model = - new PlanDataListResponse() - .withValue( - Arrays - .asList( - new PlanDataResourceInner() - .withOrgCreationSource(OrgCreationSource.LIFTR) - .withAccountCreationSource(AccountCreationSource.NEWRELIC))) - .withNextLink("pg"); + PlanDataListResponse model = new PlanDataListResponse().withValue(Arrays.asList(new PlanDataResourceInner() + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.WEEKLY) + .withPlanDetails("mbe").withEffectiveDate(OffsetDateTime.parse("2021-10-01T04:14:33Z"))) + .withOrgCreationSource(OrgCreationSource.NEWRELIC).withAccountCreationSource(AccountCreationSource.LIFTR))) + .withNextLink("n"); model = BinaryData.fromObject(model).toObject(PlanDataListResponse.class); - Assertions.assertEquals(OrgCreationSource.LIFTR, model.value().get(0).orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.value().get(0).accountCreationSource()); - Assertions.assertEquals("pg", model.nextLink()); + Assertions.assertEquals(UsageType.PAYG, model.value().get(0).planData().usageType()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.value().get(0).planData().billingCycle()); + Assertions.assertEquals("mbe", model.value().get(0).planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-10-01T04:14:33Z"), + model.value().get(0).planData().effectiveDate()); + Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.value().get(0).orgCreationSource()); + Assertions.assertEquals(AccountCreationSource.LIFTR, model.value().get(0).accountCreationSource()); + Assertions.assertEquals("n", model.nextLink()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataPropertiesTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataPropertiesTests.java index d7d14f72dfc47..42402532c9a0d 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataPropertiesTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataPropertiesTests.java @@ -17,37 +17,29 @@ public final class PlanDataPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PlanDataProperties model = - BinaryData - .fromString( - "{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"ddglm\",\"effectiveDate\":\"2020-12-21T16:15:31Z\"},\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"NEWRELIC\"}") - .toObject(PlanDataProperties.class); + PlanDataProperties model = BinaryData.fromString( + "{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"WEEKLY\",\"planDetails\":\"rwzwbng\",\"effectiveDate\":\"2021-01-28T09:21:30Z\"},\"orgCreationSource\":\"LIFTR\",\"accountCreationSource\":\"LIFTR\"}") + .toObject(PlanDataProperties.class); Assertions.assertEquals(UsageType.PAYG, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); - Assertions.assertEquals("ddglm", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2020-12-21T16:15:31Z"), model.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.accountCreationSource()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.planData().billingCycle()); + Assertions.assertEquals("rwzwbng", model.planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-28T09:21:30Z"), model.planData().effectiveDate()); + Assertions.assertEquals(OrgCreationSource.LIFTR, model.orgCreationSource()); + Assertions.assertEquals(AccountCreationSource.LIFTR, model.accountCreationSource()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PlanDataProperties model = - new PlanDataProperties() - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.MONTHLY) - .withPlanDetails("ddglm") - .withEffectiveDate(OffsetDateTime.parse("2020-12-21T16:15:31Z"))) - .withOrgCreationSource(OrgCreationSource.NEWRELIC) - .withAccountCreationSource(AccountCreationSource.NEWRELIC); + PlanDataProperties model = new PlanDataProperties() + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.WEEKLY) + .withPlanDetails("rwzwbng").withEffectiveDate(OffsetDateTime.parse("2021-01-28T09:21:30Z"))) + .withOrgCreationSource(OrgCreationSource.LIFTR).withAccountCreationSource(AccountCreationSource.LIFTR); model = BinaryData.fromObject(model).toObject(PlanDataProperties.class); Assertions.assertEquals(UsageType.PAYG, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); - Assertions.assertEquals("ddglm", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2020-12-21T16:15:31Z"), model.planData().effectiveDate()); - Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.NEWRELIC, model.accountCreationSource()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.planData().billingCycle()); + Assertions.assertEquals("rwzwbng", model.planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-01-28T09:21:30Z"), model.planData().effectiveDate()); + Assertions.assertEquals(OrgCreationSource.LIFTR, model.orgCreationSource()); + Assertions.assertEquals(AccountCreationSource.LIFTR, model.accountCreationSource()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataResourceInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataResourceInnerTests.java index d3fa0b9682300..4bb41687e1f6a 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataResourceInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataResourceInnerTests.java @@ -17,36 +17,28 @@ public final class PlanDataResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PlanDataResourceInner model = - BinaryData - .fromString( - "{\"properties\":{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"YEARLY\",\"planDetails\":\"hvpmoue\",\"effectiveDate\":\"2021-06-30T20:07:19Z\"},\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"LIFTR\"},\"id\":\"eojnxqbzvddn\",\"name\":\"wndeicbtwnp\",\"type\":\"aoqvuh\"}") - .toObject(PlanDataResourceInner.class); + PlanDataResourceInner model = BinaryData.fromString( + "{\"properties\":{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"l\",\"effectiveDate\":\"2020-12-23T01:20:43Z\"},\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"LIFTR\"},\"id\":\"rlyxwjkcprbnw\",\"name\":\"xgjvtbv\",\"type\":\"ysszdnrujqguh\"}") + .toObject(PlanDataResourceInner.class); Assertions.assertEquals(UsageType.PAYG, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.YEARLY, model.planData().billingCycle()); - Assertions.assertEquals("hvpmoue", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-30T20:07:19Z"), model.planData().effectiveDate()); + Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); + Assertions.assertEquals("l", model.planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2020-12-23T01:20:43Z"), model.planData().effectiveDate()); Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.orgCreationSource()); Assertions.assertEquals(AccountCreationSource.LIFTR, model.accountCreationSource()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PlanDataResourceInner model = - new PlanDataResourceInner() - .withPlanData( - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.YEARLY) - .withPlanDetails("hvpmoue") - .withEffectiveDate(OffsetDateTime.parse("2021-06-30T20:07:19Z"))) - .withOrgCreationSource(OrgCreationSource.NEWRELIC) - .withAccountCreationSource(AccountCreationSource.LIFTR); + PlanDataResourceInner model = new PlanDataResourceInner() + .withPlanData(new PlanData().withUsageType(UsageType.PAYG).withBillingCycle(BillingCycle.MONTHLY) + .withPlanDetails("l").withEffectiveDate(OffsetDateTime.parse("2020-12-23T01:20:43Z"))) + .withOrgCreationSource(OrgCreationSource.NEWRELIC).withAccountCreationSource(AccountCreationSource.LIFTR); model = BinaryData.fromObject(model).toObject(PlanDataResourceInner.class); Assertions.assertEquals(UsageType.PAYG, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.YEARLY, model.planData().billingCycle()); - Assertions.assertEquals("hvpmoue", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-06-30T20:07:19Z"), model.planData().effectiveDate()); + Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); + Assertions.assertEquals("l", model.planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2020-12-23T01:20:43Z"), model.planData().effectiveDate()); Assertions.assertEquals(OrgCreationSource.NEWRELIC, model.orgCreationSource()); Assertions.assertEquals(AccountCreationSource.LIFTR, model.accountCreationSource()); } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataTests.java index 69a3e3041b2ec..960b4e4cbc787 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlanDataTests.java @@ -14,29 +14,23 @@ public final class PlanDataTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PlanData model = - BinaryData - .fromString( - "{\"usageType\":\"PAYG\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"mdwzjeiachboo\",\"effectiveDate\":\"2021-07-31T08:54:57Z\"}") - .toObject(PlanData.class); - Assertions.assertEquals(UsageType.PAYG, model.usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.billingCycle()); - Assertions.assertEquals("mdwzjeiachboo", model.planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-31T08:54:57Z"), model.effectiveDate()); + PlanData model = BinaryData.fromString( + "{\"usageType\":\"COMMITTED\",\"billingCycle\":\"WEEKLY\",\"planDetails\":\"gsntnbybkzgcwr\",\"effectiveDate\":\"2021-09-26T22:03:54Z\"}") + .toObject(PlanData.class); + Assertions.assertEquals(UsageType.COMMITTED, model.usageType()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.billingCycle()); + Assertions.assertEquals("gsntnbybkzgcwr", model.planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-09-26T22:03:54Z"), model.effectiveDate()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PlanData model = - new PlanData() - .withUsageType(UsageType.PAYG) - .withBillingCycle(BillingCycle.MONTHLY) - .withPlanDetails("mdwzjeiachboo") - .withEffectiveDate(OffsetDateTime.parse("2021-07-31T08:54:57Z")); + PlanData model = new PlanData().withUsageType(UsageType.COMMITTED).withBillingCycle(BillingCycle.WEEKLY) + .withPlanDetails("gsntnbybkzgcwr").withEffectiveDate(OffsetDateTime.parse("2021-09-26T22:03:54Z")); model = BinaryData.fromObject(model).toObject(PlanData.class); - Assertions.assertEquals(UsageType.PAYG, model.usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.billingCycle()); - Assertions.assertEquals("mdwzjeiachboo", model.planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-07-31T08:54:57Z"), model.effectiveDate()); + Assertions.assertEquals(UsageType.COMMITTED, model.usageType()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.billingCycle()); + Assertions.assertEquals("gsntnbybkzgcwr", model.planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-09-26T22:03:54Z"), model.effectiveDate()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListMockTests.java index 427c5b45ffb7a..3e999fe148cb0 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/PlansListMockTests.java @@ -35,45 +35,33 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"YEARLY\",\"planDetails\":\"jmquxvypomgk\",\"effectiveDate\":\"2021-08-12T00:08:05Z\"},\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"LIFTR\"},\"id\":\"pajqgxysm\",\"name\":\"cmbqfqvmk\",\"type\":\"xozap\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"planData\":{\"usageType\":\"PAYG\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"ft\",\"effectiveDate\":\"2021-05-16T02:46:56Z\"},\"orgCreationSource\":\"NEWRELIC\",\"accountCreationSource\":\"NEWRELIC\"},\"id\":\"zvqtmnubexkp\",\"name\":\"ksmond\",\"type\":\"mquxvypo\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.plans().list("yzvqt", "nubexk", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.plans().list("dggkzzlvmbmpa", "modfvuefywsbpfvm", com.azure.core.util.Context.NONE); Assertions.assertEquals(UsageType.PAYG, response.iterator().next().planData().usageType()); - Assertions.assertEquals(BillingCycle.YEARLY, response.iterator().next().planData().billingCycle()); - Assertions.assertEquals("jmquxvypomgk", response.iterator().next().planData().planDetails()); - Assertions - .assertEquals( - OffsetDateTime.parse("2021-08-12T00:08:05Z"), response.iterator().next().planData().effectiveDate()); + Assertions.assertEquals(BillingCycle.MONTHLY, response.iterator().next().planData().billingCycle()); + Assertions.assertEquals("ft", response.iterator().next().planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-05-16T02:46:56Z"), + response.iterator().next().planData().effectiveDate()); Assertions.assertEquals(OrgCreationSource.NEWRELIC, response.iterator().next().orgCreationSource()); - Assertions.assertEquals(AccountCreationSource.LIFTR, response.iterator().next().accountCreationSource()); + Assertions.assertEquals(AccountCreationSource.NEWRELIC, response.iterator().next().accountCreationSource()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SubscriptionListTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SubscriptionListTests.java new file mode 100644 index 0000000000000..c9c20a9dac033 --- /dev/null +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SubscriptionListTests.java @@ -0,0 +1,99 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.newrelicobservability.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoredSubscriptionInner; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MonitoringTagRulesPropertiesInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; +import com.azure.resourcemanager.newrelicobservability.models.LogRules; +import com.azure.resourcemanager.newrelicobservability.models.PatchOperation; +import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.Status; +import com.azure.resourcemanager.newrelicobservability.models.SubscriptionList; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class SubscriptionListTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + SubscriptionList model = BinaryData.fromString( + "{\"patchOperation\":\"DeleteComplete\",\"monitoredSubscriptionList\":[{\"subscriptionId\":\"dhmdua\",\"status\":\"InProgress\",\"error\":\"qpv\",\"tagRules\":{\"provisioningState\":\"Creating\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{}]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[{},{},{},{}],\"userEmail\":\"fmisg\"}}},{\"subscriptionId\":\"nbbelda\",\"status\":\"Failed\",\"error\":\"ali\",\"tagRules\":{\"provisioningState\":\"Deleting\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{},{},{},{}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{}],\"userEmail\":\"owzxcu\"}}},{\"subscriptionId\":\"cjooxdjebwpucwwf\",\"status\":\"Active\",\"error\":\"vmeueci\",\"tagRules\":{\"provisioningState\":\"Deleted\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{},{}],\"userEmail\":\"wmcdytdxwi\"}}}],\"provisioningState\":\"Deleting\"}") + .toObject(SubscriptionList.class); + Assertions.assertEquals(PatchOperation.DELETE_COMPLETE, model.patchOperation()); + Assertions.assertEquals("dhmdua", model.monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, model.monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("qpv", model.monitoredSubscriptionList().get(0).error()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().logRules().sendActivityLogs()); + Assertions.assertEquals(SendMetricsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().metricRules().sendMetrics()); + Assertions.assertEquals("fmisg", model.monitoredSubscriptionList().get(0).tagRules().metricRules().userEmail()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + SubscriptionList model + = new SubscriptionList().withPatchOperation(PatchOperation.DELETE_COMPLETE) + .withMonitoredSubscriptionList(Arrays + .asList( + new MonitoredSubscriptionInner().withSubscriptionId("dhmdua").withStatus(Status.IN_PROGRESS) + .withError("qpv").withTagRules( + new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag()))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag(), new FilteringTag(), + new FilteringTag(), new FilteringTag())) + .withUserEmail("fmisg"))), + new MonitoredSubscriptionInner().withSubscriptionId("nbbelda").withStatus(Status.FAILED) + .withError( + "ali") + .withTagRules(new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags(Arrays.asList(new FilteringTag(), new FilteringTag(), + new FilteringTag(), new FilteringTag()))) + .withMetricRules(new MetricRulesInner() + .withSendMetrics(SendMetricsStatus.DISABLED).withFilteringTags( + Arrays.asList(new FilteringTag())) + .withUserEmail("owzxcu"))), + new MonitoredSubscriptionInner().withSubscriptionId("cjooxdjebwpucwwf") + .withStatus(Status.ACTIVE).withError("vmeueci") + .withTagRules(new MonitoringTagRulesPropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags(Arrays.asList(new FilteringTag()))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList(new FilteringTag(), new FilteringTag())) + .withUserEmail("wmcdytdxwi"))))); + model = BinaryData.fromObject(model).toObject(SubscriptionList.class); + Assertions.assertEquals(PatchOperation.DELETE_COMPLETE, model.patchOperation()); + Assertions.assertEquals("dhmdua", model.monitoredSubscriptionList().get(0).subscriptionId()); + Assertions.assertEquals(Status.IN_PROGRESS, model.monitoredSubscriptionList().get(0).status()); + Assertions.assertEquals("qpv", model.monitoredSubscriptionList().get(0).error()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().logRules().sendActivityLogs()); + Assertions.assertEquals(SendMetricsStatus.ENABLED, + model.monitoredSubscriptionList().get(0).tagRules().metricRules().sendMetrics()); + Assertions.assertEquals("fmisg", model.monitoredSubscriptionList().get(0).tagRules().metricRules().userEmail()); + } +} diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SwitchBillingRequestTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SwitchBillingRequestTests.java index 51bfece6cd933..7d46e64b928e5 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SwitchBillingRequestTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/SwitchBillingRequestTests.java @@ -15,40 +15,32 @@ public final class SwitchBillingRequestTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SwitchBillingRequest model = - BinaryData - .fromString( - "{\"azureResourceId\":\"vdcsitynn\",\"organizationId\":\"mdectehfiqscjey\",\"planData\":{\"usageType\":\"COMMITTED\",\"billingCycle\":\"MONTHLY\",\"planDetails\":\"gqhcjrefovg\",\"effectiveDate\":\"2021-10-26T09:37:04Z\"},\"userEmail\":\"sle\"}") - .toObject(SwitchBillingRequest.class); - Assertions.assertEquals("vdcsitynn", model.azureResourceId()); - Assertions.assertEquals("mdectehfiqscjey", model.organizationId()); + SwitchBillingRequest model = BinaryData.fromString( + "{\"azureResourceId\":\"jqkwpyeicx\",\"organizationId\":\"ciwqvhk\",\"planData\":{\"usageType\":\"COMMITTED\",\"billingCycle\":\"WEEKLY\",\"planDetails\":\"topbobjogh\",\"effectiveDate\":\"2021-04-17T11:58:30Z\"},\"userEmail\":\"u\"}") + .toObject(SwitchBillingRequest.class); + Assertions.assertEquals("jqkwpyeicx", model.azureResourceId()); + Assertions.assertEquals("ciwqvhk", model.organizationId()); Assertions.assertEquals(UsageType.COMMITTED, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); - Assertions.assertEquals("gqhcjrefovg", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-26T09:37:04Z"), model.planData().effectiveDate()); - Assertions.assertEquals("sle", model.userEmail()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.planData().billingCycle()); + Assertions.assertEquals("topbobjogh", model.planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-17T11:58:30Z"), model.planData().effectiveDate()); + Assertions.assertEquals("u", model.userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SwitchBillingRequest model = - new SwitchBillingRequest() - .withAzureResourceId("vdcsitynn") - .withOrganizationId("mdectehfiqscjey") - .withPlanData( - new PlanData() - .withUsageType(UsageType.COMMITTED) - .withBillingCycle(BillingCycle.MONTHLY) - .withPlanDetails("gqhcjrefovg") - .withEffectiveDate(OffsetDateTime.parse("2021-10-26T09:37:04Z"))) - .withUserEmail("sle"); + SwitchBillingRequest model + = new SwitchBillingRequest().withAzureResourceId("jqkwpyeicx").withOrganizationId("ciwqvhk") + .withPlanData(new PlanData().withUsageType(UsageType.COMMITTED).withBillingCycle(BillingCycle.WEEKLY) + .withPlanDetails("topbobjogh").withEffectiveDate(OffsetDateTime.parse("2021-04-17T11:58:30Z"))) + .withUserEmail("u"); model = BinaryData.fromObject(model).toObject(SwitchBillingRequest.class); - Assertions.assertEquals("vdcsitynn", model.azureResourceId()); - Assertions.assertEquals("mdectehfiqscjey", model.organizationId()); + Assertions.assertEquals("jqkwpyeicx", model.azureResourceId()); + Assertions.assertEquals("ciwqvhk", model.organizationId()); Assertions.assertEquals(UsageType.COMMITTED, model.planData().usageType()); - Assertions.assertEquals(BillingCycle.MONTHLY, model.planData().billingCycle()); - Assertions.assertEquals("gqhcjrefovg", model.planData().planDetails()); - Assertions.assertEquals(OffsetDateTime.parse("2021-10-26T09:37:04Z"), model.planData().effectiveDate()); - Assertions.assertEquals("sle", model.userEmail()); + Assertions.assertEquals(BillingCycle.WEEKLY, model.planData().billingCycle()); + Assertions.assertEquals("topbobjogh", model.planData().planDetails()); + Assertions.assertEquals(OffsetDateTime.parse("2021-04-17T11:58:30Z"), model.planData().effectiveDate()); + Assertions.assertEquals("u", model.userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleInnerTests.java index f446c92709af6..e6eb2d114f5bc 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleInnerTests.java @@ -7,49 +7,63 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; import com.azure.resourcemanager.newrelicobservability.models.LogRules; import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.TagAction; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class TagRuleInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TagRuleInner model = - BinaryData - .fromString( - "{\"properties\":{\"provisioningState\":\"Canceled\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[],\"userEmail\":\"burvjxxjnspy\"}},\"id\":\"tko\",\"name\":\"nkoukn\",\"type\":\"udwtiukbl\"}") - .toObject(TagRuleInner.class); - Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); + TagRuleInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Deleting\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"yokacspkw\",\"value\":\"zdobpxjmflbvvnch\",\"action\":\"Include\"},{\"name\":\"iwwzjuqk\",\"value\":\"sa\",\"action\":\"Exclude\"},{\"name\":\"uo\",\"value\":\"skghsauuimj\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"idyjrrfbyaosvexc\",\"value\":\"npc\",\"action\":\"Exclude\"},{\"name\":\"ohslkevlegg\",\"value\":\"buhfmvfaxkffeiit\",\"action\":\"Include\"}],\"userEmail\":\"ez\"}},\"id\":\"shxmzsbbzoggigrx\",\"name\":\"burvjxxjnspy\",\"type\":\"ptkoenkoukn\"}") + .toObject(TagRuleInner.class); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.logRules().sendActivityLogs()); - Assertions.assertEquals(SendMetricsStatus.ENABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("burvjxxjnspy", model.metricRules().userEmail()); + Assertions.assertEquals("yokacspkw", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("zdobpxjmflbvvnch", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); + Assertions.assertEquals("idyjrrfbyaosvexc", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("npc", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("ez", model.metricRules().userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TagRuleInner model = - new TagRuleInner() - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.DISABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.DISABLED) - .withFilteringTags(Arrays.asList())) - .withMetricRules( - new MetricRulesInner() - .withSendMetrics(SendMetricsStatus.ENABLED) - .withFilteringTags(Arrays.asList()) - .withUserEmail("burvjxxjnspy")); + TagRuleInner model = new TagRuleInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("yokacspkw").withValue("zdobpxjmflbvvnch") + .withAction(TagAction.INCLUDE), + new FilteringTag().withName("iwwzjuqk").withValue("sa").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("uo").withValue("skghsauuimj").withAction(TagAction.EXCLUDE)))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("idyjrrfbyaosvexc").withValue("npc").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("ohslkevlegg").withValue("buhfmvfaxkffeiit") + .withAction(TagAction.INCLUDE))) + .withUserEmail("ez")); model = BinaryData.fromObject(model).toObject(TagRuleInner.class); - Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.logRules().sendActivityLogs()); - Assertions.assertEquals(SendMetricsStatus.ENABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("burvjxxjnspy", model.metricRules().userEmail()); + Assertions.assertEquals("yokacspkw", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("zdobpxjmflbvvnch", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); + Assertions.assertEquals("idyjrrfbyaosvexc", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("npc", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("ez", model.metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleListResultTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleListResultTests.java index 82b1370b03de2..fe7a746bdcd5b 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleListResultTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleListResultTests.java @@ -5,7 +5,14 @@ package com.azure.resourcemanager.newrelicobservability.generated; import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; +import com.azure.resourcemanager.newrelicobservability.models.LogRules; +import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; +import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.TagRuleListResult; import java.util.Arrays; import org.junit.jupiter.api.Assertions; @@ -13,21 +20,42 @@ public final class TagRuleListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TagRuleListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\"},\"id\":\"dyjrrfbyaosv\",\"name\":\"xc\",\"type\":\"onpc\"},{\"properties\":{\"provisioningState\":\"Accepted\"},\"id\":\"slkevle\",\"name\":\"gz\",\"type\":\"buhfmvfaxkffeiit\"}],\"nextLink\":\"vmezy\"}") - .toObject(TagRuleListResult.class); - Assertions.assertEquals("vmezy", model.nextLink()); + TagRuleListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"NotSpecified\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{},{}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{},{},{},{}],\"userEmail\":\"stkiiuxhqyud\"}},\"id\":\"rrqnbpoczvyifqrv\",\"name\":\"dvjsllrmvvdf\",\"type\":\"atkpnp\"}],\"nextLink\":\"exxbczwtr\"}") + .toObject(TagRuleListResult.class); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.value().get(0).logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, + model.value().get(0).logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, model.value().get(0).logRules().sendActivityLogs()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, model.value().get(0).metricRules().sendMetrics()); + Assertions.assertEquals("stkiiuxhqyud", model.value().get(0).metricRules().userEmail()); + Assertions.assertEquals("exxbczwtr", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TagRuleListResult model = - new TagRuleListResult() - .withValue(Arrays.asList(new TagRuleInner(), new TagRuleInner())) - .withNextLink("vmezy"); + TagRuleListResult model + = new TagRuleListResult() + .withValue( + Arrays + .asList( + new TagRuleInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.ENABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag(), new FilteringTag()))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList(new FilteringTag(), new FilteringTag(), + new FilteringTag(), new FilteringTag())) + .withUserEmail("stkiiuxhqyud")))) + .withNextLink("exxbczwtr"); model = BinaryData.fromObject(model).toObject(TagRuleListResult.class); - Assertions.assertEquals("vmezy", model.nextLink()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, model.value().get(0).logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, + model.value().get(0).logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, model.value().get(0).logRules().sendActivityLogs()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, model.value().get(0).metricRules().sendMetrics()); + Assertions.assertEquals("stkiiuxhqyud", model.value().get(0).metricRules().userEmail()); + Assertions.assertEquals("exxbczwtr", model.nextLink()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdateInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdateInnerTests.java index 9009dbb1fc46c..27c37b9679eb9 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdateInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdateInnerTests.java @@ -7,49 +7,62 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; import com.azure.resourcemanager.newrelicobservability.fluent.models.TagRuleUpdateInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; import com.azure.resourcemanager.newrelicobservability.models.LogRules; import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.TagAction; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class TagRuleUpdateInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TagRuleUpdateInner model = - BinaryData - .fromString( - "{\"properties\":{\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[],\"userEmail\":\"xfvjrbirp\"}}}") - .toObject(TagRuleUpdateInner.class); + TagRuleUpdateInner model = BinaryData.fromString( + "{\"properties\":{\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"pwvlqdq\",\"value\":\"iqylihkaetck\",\"action\":\"Exclude\"},{\"name\":\"ivfsnk\",\"value\":\"uctqhjfbe\",\"action\":\"Exclude\"},{\"name\":\"xerf\",\"value\":\"utttxfvjrbirp\",\"action\":\"Exclude\"},{\"name\":\"c\",\"value\":\"ahfn\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"ujqgidok\",\"value\":\"ljyoxgvcltb\",\"action\":\"Include\"},{\"name\":\"ghkjeszzhbi\",\"value\":\"txfvgx\",\"action\":\"Include\"}],\"userEmail\":\"xnehmpvec\"}}}") + .toObject(TagRuleUpdateInner.class); Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); - Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.logRules().sendActivityLogs()); + Assertions.assertEquals("pwvlqdq", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("iqylihkaetck", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.logRules().filteringTags().get(0).action()); Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("xfvjrbirp", model.metricRules().userEmail()); + Assertions.assertEquals("ujqgidok", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("ljyoxgvcltb", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("xnehmpvec", model.metricRules().userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TagRuleUpdateInner model = - new TagRuleUpdateInner() - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.DISABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.DISABLED) - .withFilteringTags(Arrays.asList())) - .withMetricRules( - new MetricRulesInner() - .withSendMetrics(SendMetricsStatus.DISABLED) - .withFilteringTags(Arrays.asList()) - .withUserEmail("xfvjrbirp")); + TagRuleUpdateInner model = new TagRuleUpdateInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("pwvlqdq").withValue("iqylihkaetck").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("ivfsnk").withValue("uctqhjfbe").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("xerf").withValue("utttxfvjrbirp").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("c").withValue("ahfn").withAction(TagAction.EXCLUDE)))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("ujqgidok").withValue("ljyoxgvcltb").withAction(TagAction.INCLUDE), + new FilteringTag().withName("ghkjeszzhbi").withValue("txfvgx").withAction(TagAction.INCLUDE))) + .withUserEmail("xnehmpvec")); model = BinaryData.fromObject(model).toObject(TagRuleUpdateInner.class); Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); - Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, model.logRules().sendActivityLogs()); + Assertions.assertEquals("pwvlqdq", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("iqylihkaetck", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.logRules().filteringTags().get(0).action()); Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("xfvjrbirp", model.metricRules().userEmail()); + Assertions.assertEquals("ujqgidok", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("ljyoxgvcltb", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("xnehmpvec", model.metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdatePropertiesInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdatePropertiesInnerTests.java index 6b4858324a0bd..d71171c475488 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdatePropertiesInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRuleUpdatePropertiesInnerTests.java @@ -20,74 +20,49 @@ public final class TagRuleUpdatePropertiesInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TagRuleUpdatePropertiesInner model = - BinaryData - .fromString( - "{\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{\"name\":\"ljkyqxjvuuj\",\"value\":\"idokgjlj\",\"action\":\"Include\"},{\"name\":\"vcltbgsncgh\",\"value\":\"esz\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"vgxbfs\",\"value\":\"nehmpvecx\",\"action\":\"Exclude\"},{\"name\":\"bfqkkr\",\"value\":\"pukgriwflzlfb\",\"action\":\"Include\"},{\"name\":\"zycispn\",\"value\":\"ahmgkbrp\",\"action\":\"Include\"}],\"userEmail\":\"ibnuqqkpik\"}}") - .toObject(TagRuleUpdatePropertiesInner.class); + TagRuleUpdatePropertiesInner model = BinaryData.fromString( + "{\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{\"name\":\"bmpukgriwflz\",\"value\":\"bxzpuzycisp\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"rp\",\"value\":\"dhibnuq\",\"action\":\"Include\"},{\"name\":\"kadrgvt\",\"value\":\"gnbuy\",\"action\":\"Exclude\"},{\"name\":\"gg\",\"value\":\"bfs\",\"action\":\"Include\"},{\"name\":\"utrc\",\"value\":\"na\",\"action\":\"Exclude\"}],\"userEmail\":\"jrunmpxtt\"}}") + .toObject(TagRuleUpdatePropertiesInner.class); Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.ENABLED, model.logRules().sendActivityLogs()); - Assertions.assertEquals("ljkyqxjvuuj", model.logRules().filteringTags().get(0).name()); - Assertions.assertEquals("idokgjlj", model.logRules().filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.INCLUDE, model.logRules().filteringTags().get(0).action()); + Assertions.assertEquals("bmpukgriwflz", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("bxzpuzycisp", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.logRules().filteringTags().get(0).action()); Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("vgxbfs", model.metricRules().filteringTags().get(0).name()); - Assertions.assertEquals("nehmpvecx", model.metricRules().filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.EXCLUDE, model.metricRules().filteringTags().get(0).action()); - Assertions.assertEquals("ibnuqqkpik", model.metricRules().userEmail()); + Assertions.assertEquals("rp", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("dhibnuq", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("jrunmpxtt", model.metricRules().userEmail()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TagRuleUpdatePropertiesInner model = - new TagRuleUpdatePropertiesInner() - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.DISABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) - .withSendActivityLogs(SendActivityLogsStatus.ENABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("ljkyqxjvuuj") - .withValue("idokgjlj") - .withAction(TagAction.INCLUDE), - new FilteringTag() - .withName("vcltbgsncgh") - .withValue("esz") - .withAction(TagAction.EXCLUDE)))) - .withMetricRules( - new MetricRulesInner() - .withSendMetrics(SendMetricsStatus.DISABLED) - .withFilteringTags( - Arrays - .asList( - new FilteringTag() - .withName("vgxbfs") - .withValue("nehmpvecx") - .withAction(TagAction.EXCLUDE), - new FilteringTag() - .withName("bfqkkr") - .withValue("pukgriwflzlfb") - .withAction(TagAction.INCLUDE), - new FilteringTag() - .withName("zycispn") - .withValue("ahmgkbrp") - .withAction(TagAction.INCLUDE))) - .withUserEmail("ibnuqqkpik")); + TagRuleUpdatePropertiesInner model + = new TagRuleUpdatePropertiesInner() + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.ENABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList(new FilteringTag().withName("bmpukgriwflz") + .withValue("bxzpuzycisp").withAction(TagAction.EXCLUDE)))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("rp").withValue("dhibnuq").withAction(TagAction.INCLUDE), + new FilteringTag().withName("kadrgvt").withValue("gnbuy").withAction(TagAction.EXCLUDE), + new FilteringTag().withName("gg").withValue("bfs").withAction(TagAction.INCLUDE), + new FilteringTag().withName("utrc").withValue("na").withAction(TagAction.EXCLUDE))) + .withUserEmail("jrunmpxtt")); model = BinaryData.fromObject(model).toObject(TagRuleUpdatePropertiesInner.class); Assertions.assertEquals(SendAadLogsStatus.DISABLED, model.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, model.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.ENABLED, model.logRules().sendActivityLogs()); - Assertions.assertEquals("ljkyqxjvuuj", model.logRules().filteringTags().get(0).name()); - Assertions.assertEquals("idokgjlj", model.logRules().filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.INCLUDE, model.logRules().filteringTags().get(0).action()); + Assertions.assertEquals("bmpukgriwflz", model.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("bxzpuzycisp", model.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, model.logRules().filteringTags().get(0).action()); Assertions.assertEquals(SendMetricsStatus.DISABLED, model.metricRules().sendMetrics()); - Assertions.assertEquals("vgxbfs", model.metricRules().filteringTags().get(0).name()); - Assertions.assertEquals("nehmpvecx", model.metricRules().filteringTags().get(0).value()); - Assertions.assertEquals(TagAction.EXCLUDE, model.metricRules().filteringTags().get(0).action()); - Assertions.assertEquals("ibnuqqkpik", model.metricRules().userEmail()); + Assertions.assertEquals("rp", model.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("dhibnuq", model.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, model.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("jrunmpxtt", model.metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateMockTests.java index b8e2ab00d537b..77faf87ec3a0f 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesCreateOrUpdateMockTests.java @@ -13,11 +13,13 @@ import com.azure.core.management.profile.AzureProfile; import com.azure.resourcemanager.newrelicobservability.NewRelicObservabilityManager; import com.azure.resourcemanager.newrelicobservability.fluent.models.MetricRulesInner; +import com.azure.resourcemanager.newrelicobservability.models.FilteringTag; import com.azure.resourcemanager.newrelicobservability.models.LogRules; import com.azure.resourcemanager.newrelicobservability.models.SendAadLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.TagAction; import com.azure.resourcemanager.newrelicobservability.models.TagRule; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -37,57 +39,49 @@ public void testCreateOrUpdate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Succeeded\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[],\"userEmail\":\"ynl\"}},\"id\":\"huopxodlqiynto\",\"name\":\"zihleosjswsr\",\"type\":\"slyzrpzbchckqq\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"jmoadsuv\",\"value\":\"m\",\"action\":\"Include\"},{\"name\":\"jsjqbjhhyx\",\"value\":\"wlycoduhpkxkg\",\"action\":\"Include\"},{\"name\":\"eqnajxqugjhkycu\",\"value\":\"ddg\",\"action\":\"Include\"},{\"name\":\"wqm\",\"value\":\"alkrmn\",\"action\":\"Include\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"udfnbyxba\",\"value\":\"bjyvay\",\"action\":\"Exclude\"}],\"userEmail\":\"rzrtuzqogsex\"}},\"id\":\"vfdnwnwmewzsyyce\",\"name\":\"zsoibjudpfrxtr\",\"type\":\"hzv\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - TagRule response = - manager - .tagRules() - .define("jxywsuws") - .withExistingMonitor("pkkpw", "reqnovvqfov") - .withLogRules( - new LogRules() - .withSendAadLogs(SendAadLogsStatus.DISABLED) - .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) - .withSendActivityLogs(SendActivityLogsStatus.DISABLED) - .withFilteringTags(Arrays.asList())) - .withMetricRules( - new MetricRulesInner() - .withSendMetrics(SendMetricsStatus.ENABLED) - .withFilteringTags(Arrays.asList()) - .withUserEmail("aeneqnzarrwl")) - .create(); + TagRule response = manager.tagRules().define("hfjx").withExistingMonitor("himdbl", "gwimfn") + .withLogRules(new LogRules().withSendAadLogs(SendAadLogsStatus.DISABLED) + .withSendSubscriptionLogs(SendSubscriptionLogsStatus.DISABLED) + .withSendActivityLogs(SendActivityLogsStatus.ENABLED) + .withFilteringTags(Arrays.asList( + new FilteringTag().withName("kzikfjawneaivxwc").withValue("lpcirelsf") + .withAction(TagAction.EXCLUDE), + new FilteringTag().withName("wabfatkl").withValue("xbjhwuaanozjosph") + .withAction(TagAction.INCLUDE)))) + .withMetricRules(new MetricRulesInner().withSendMetrics(SendMetricsStatus.DISABLED) + .withFilteringTags( + Arrays.asList(new FilteringTag().withName("ag").withValue("vimjwos").withAction(TagAction.INCLUDE))) + .withUserEmail("tcs")) + .create(); - Assertions.assertEquals(SendAadLogsStatus.DISABLED, response.logRules().sendAadLogs()); - Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, response.logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendAadLogsStatus.ENABLED, response.logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, response.logRules().sendSubscriptionLogs()); Assertions.assertEquals(SendActivityLogsStatus.DISABLED, response.logRules().sendActivityLogs()); - Assertions.assertEquals(SendMetricsStatus.ENABLED, response.metricRules().sendMetrics()); - Assertions.assertEquals("ynl", response.metricRules().userEmail()); + Assertions.assertEquals("jmoadsuv", response.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("m", response.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, response.logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, response.metricRules().sendMetrics()); + Assertions.assertEquals("udfnbyxba", response.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("bjyvay", response.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, response.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("rzrtuzqogsex", response.metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteMockTests.java index 2146f36e0ff9e..427d9d8d2c9df 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.tagRules().delete("hykojoxafnndlpic", "koymkcd", "h", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.tagRules().delete("g", "xrtfudxep", "gyqagvrvmnpkuk", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetWithResponseMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetWithResponseMockTests.java index 3690bb768f43e..1a74254a0b155 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetWithResponseMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesGetWithResponseMockTests.java @@ -16,6 +16,7 @@ import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.TagAction; import com.azure.resourcemanager.newrelicobservability.models.TagRule; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -34,45 +35,37 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"provisioningState\":\"Deleting\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[],\"userEmail\":\"bpokulpiujwaasip\"}},\"id\":\"iobyu\",\"name\":\"erpqlpqwcciuqg\",\"type\":\"dbutauvfbtkuwhh\"}"; + String responseStr + = "{\"properties\":{\"provisioningState\":\"Canceled\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Enabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[{\"name\":\"zqioxiysuii\",\"value\":\"nkedyatrwyhqmib\",\"action\":\"Include\"},{\"name\":\"itsmypyyn\",\"value\":\"dpumnzgmw\",\"action\":\"Exclude\"},{\"name\":\"biknsorgjhxbld\",\"value\":\"wwrlkdmtncv\",\"action\":\"Exclude\"},{\"name\":\"llxdyhgs\",\"value\":\"cogjltdtbn\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"vcikhnvpamqgx\",\"value\":\"u\",\"action\":\"Include\"},{\"name\":\"ywggx\",\"value\":\"lla\",\"action\":\"Include\"},{\"name\":\"wuipiccjzkzivg\",\"value\":\"c\",\"action\":\"Include\"}],\"userEmail\":\"hyrnxxmu\"}},\"id\":\"dndrdvstkwqqtche\",\"name\":\"lmfmtdaay\",\"type\":\"dvwvgpio\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - TagRule response = - manager - .tagRules() - .getWithResponse("yvoqa", "piexpbtgiw", "wo", com.azure.core.util.Context.NONE) - .getValue(); + TagRule response = manager.tagRules() + .getWithResponse("synljphuopxodl", "iyntorzihle", "sjswsrms", com.azure.core.util.Context.NONE).getValue(); - Assertions.assertEquals(SendAadLogsStatus.ENABLED, response.logRules().sendAadLogs()); + Assertions.assertEquals(SendAadLogsStatus.DISABLED, response.logRules().sendAadLogs()); Assertions.assertEquals(SendSubscriptionLogsStatus.ENABLED, response.logRules().sendSubscriptionLogs()); - Assertions.assertEquals(SendActivityLogsStatus.ENABLED, response.logRules().sendActivityLogs()); + Assertions.assertEquals(SendActivityLogsStatus.DISABLED, response.logRules().sendActivityLogs()); + Assertions.assertEquals("zqioxiysuii", response.logRules().filteringTags().get(0).name()); + Assertions.assertEquals("nkedyatrwyhqmib", response.logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, response.logRules().filteringTags().get(0).action()); Assertions.assertEquals(SendMetricsStatus.DISABLED, response.metricRules().sendMetrics()); - Assertions.assertEquals("bpokulpiujwaasip", response.metricRules().userEmail()); + Assertions.assertEquals("vcikhnvpamqgx", response.metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("u", response.metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.INCLUDE, response.metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("hyrnxxmu", response.metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceMockTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceMockTests.java index ae48a178d864e..c573380a87ffb 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceMockTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/TagRulesListByNewRelicMonitorResourceMockTests.java @@ -17,6 +17,7 @@ import com.azure.resourcemanager.newrelicobservability.models.SendActivityLogsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendMetricsStatus; import com.azure.resourcemanager.newrelicobservability.models.SendSubscriptionLogsStatus; +import com.azure.resourcemanager.newrelicobservability.models.TagAction; import com.azure.resourcemanager.newrelicobservability.models.TagRule; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -35,45 +36,42 @@ public void testListByNewRelicMonitorResource() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"logRules\":{\"sendAadLogs\":\"Enabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Disabled\",\"filteringTags\":[]},\"metricRules\":{\"sendMetrics\":\"Enabled\",\"filteringTags\":[],\"userEmail\":\"brh\"}},\"id\":\"xsdqrhzoymibmrqy\",\"name\":\"bahwfl\",\"type\":\"szdtmhrkwof\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Failed\",\"logRules\":{\"sendAadLogs\":\"Disabled\",\"sendSubscriptionLogs\":\"Disabled\",\"sendActivityLogs\":\"Enabled\",\"filteringTags\":[{\"name\":\"lpichk\",\"value\":\"mkcdyhbpkkpwdre\",\"action\":\"Exclude\"}]},\"metricRules\":{\"sendMetrics\":\"Disabled\",\"filteringTags\":[{\"name\":\"ljxywsu\",\"value\":\"yrs\",\"action\":\"Exclude\"},{\"name\":\"tgadgvraeaen\",\"value\":\"nzar\",\"action\":\"Exclude\"},{\"name\":\"uu\",\"value\":\"fqka\",\"action\":\"Include\"}],\"userEmail\":\"ipfpubji\"}},\"id\":\"wifto\",\"name\":\"qkvpuvksgplsakn\",\"type\":\"n\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - NewRelicObservabilityManager manager = - NewRelicObservabilityManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + NewRelicObservabilityManager manager = NewRelicObservabilityManager.configure().withHttpClient(httpClient) + .authenticate(tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.tagRules().listByNewRelicMonitorResource("helxprglya", "dd", com.azure.core.util.Context.NONE); + PagedIterable response = manager.tagRules().listByNewRelicMonitorResource("iuqgbdbutauv", "btkuwhh", + com.azure.core.util.Context.NONE); - Assertions.assertEquals(SendAadLogsStatus.ENABLED, response.iterator().next().logRules().sendAadLogs()); - Assertions - .assertEquals( - SendSubscriptionLogsStatus.DISABLED, response.iterator().next().logRules().sendSubscriptionLogs()); - Assertions - .assertEquals(SendActivityLogsStatus.DISABLED, response.iterator().next().logRules().sendActivityLogs()); - Assertions.assertEquals(SendMetricsStatus.ENABLED, response.iterator().next().metricRules().sendMetrics()); - Assertions.assertEquals("brh", response.iterator().next().metricRules().userEmail()); + Assertions.assertEquals(SendAadLogsStatus.DISABLED, response.iterator().next().logRules().sendAadLogs()); + Assertions.assertEquals(SendSubscriptionLogsStatus.DISABLED, + response.iterator().next().logRules().sendSubscriptionLogs()); + Assertions.assertEquals(SendActivityLogsStatus.ENABLED, + response.iterator().next().logRules().sendActivityLogs()); + Assertions.assertEquals("lpichk", response.iterator().next().logRules().filteringTags().get(0).name()); + Assertions.assertEquals("mkcdyhbpkkpwdre", + response.iterator().next().logRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, + response.iterator().next().logRules().filteringTags().get(0).action()); + Assertions.assertEquals(SendMetricsStatus.DISABLED, response.iterator().next().metricRules().sendMetrics()); + Assertions.assertEquals("ljxywsu", response.iterator().next().metricRules().filteringTags().get(0).name()); + Assertions.assertEquals("yrs", response.iterator().next().metricRules().filteringTags().get(0).value()); + Assertions.assertEquals(TagAction.EXCLUDE, + response.iterator().next().metricRules().filteringTags().get(0).action()); + Assertions.assertEquals("ipfpubji", response.iterator().next().metricRules().userEmail()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserAssignedIdentityTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserAssignedIdentityTests.java index 13428ef958d10..8add577f3dac0 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserAssignedIdentityTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserAssignedIdentityTests.java @@ -10,11 +10,9 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UserAssignedIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"b179bc5a-ba24-44a5-98e8-6105794eb3a6\",\"clientId\":\"e9347589-9d4e-419f-a817-e67cd01fd86d\"}") - .toObject(UserAssignedIdentity.class); + UserAssignedIdentity model = BinaryData.fromString( + "{\"principalId\":\"8a6e797b-d54f-4f62-b647-5febff1104b3\",\"clientId\":\"02d7ff7b-5b05-448c-bc76-2918b6a0e6a4\"}") + .toObject(UserAssignedIdentity.class); } @org.junit.jupiter.api.Test diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserInfoTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserInfoTests.java index 616acfe584a52..c47281c464c49 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserInfoTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/UserInfoTests.java @@ -11,32 +11,25 @@ public final class UserInfoTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UserInfo model = - BinaryData - .fromString( - "{\"firstName\":\"fvzwdzuhty\",\"lastName\":\"isdkfthwxmnteiw\",\"emailAddress\":\"pvkmijcmmxdcuf\",\"phoneNumber\":\"srp\",\"country\":\"zidnsezcxtbzsgfy\"}") - .toObject(UserInfo.class); - Assertions.assertEquals("fvzwdzuhty", model.firstName()); - Assertions.assertEquals("isdkfthwxmnteiw", model.lastName()); - Assertions.assertEquals("pvkmijcmmxdcuf", model.emailAddress()); - Assertions.assertEquals("srp", model.phoneNumber()); - Assertions.assertEquals("zidnsezcxtbzsgfy", model.country()); + UserInfo model = BinaryData.fromString( + "{\"firstName\":\"cciqihnhungbwjz\",\"lastName\":\"fygxgispemvtzfk\",\"emailAddress\":\"ubljofxqe\",\"phoneNumber\":\"jaeq\",\"country\":\"qjbasvms\"}") + .toObject(UserInfo.class); + Assertions.assertEquals("cciqihnhungbwjz", model.firstName()); + Assertions.assertEquals("fygxgispemvtzfk", model.lastName()); + Assertions.assertEquals("ubljofxqe", model.emailAddress()); + Assertions.assertEquals("jaeq", model.phoneNumber()); + Assertions.assertEquals("qjbasvms", model.country()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - UserInfo model = - new UserInfo() - .withFirstName("fvzwdzuhty") - .withLastName("isdkfthwxmnteiw") - .withEmailAddress("pvkmijcmmxdcuf") - .withPhoneNumber("srp") - .withCountry("zidnsezcxtbzsgfy"); + UserInfo model = new UserInfo().withFirstName("cciqihnhungbwjz").withLastName("fygxgispemvtzfk") + .withEmailAddress("ubljofxqe").withPhoneNumber("jaeq").withCountry("qjbasvms"); model = BinaryData.fromObject(model).toObject(UserInfo.class); - Assertions.assertEquals("fvzwdzuhty", model.firstName()); - Assertions.assertEquals("isdkfthwxmnteiw", model.lastName()); - Assertions.assertEquals("pvkmijcmmxdcuf", model.emailAddress()); - Assertions.assertEquals("srp", model.phoneNumber()); - Assertions.assertEquals("zidnsezcxtbzsgfy", model.country()); + Assertions.assertEquals("cciqihnhungbwjz", model.firstName()); + Assertions.assertEquals("fygxgispemvtzfk", model.lastName()); + Assertions.assertEquals("ubljofxqe", model.emailAddress()); + Assertions.assertEquals("jaeq", model.phoneNumber()); + Assertions.assertEquals("qjbasvms", model.country()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMHostsListResponseTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMHostsListResponseTests.java index c3077072d3127..03f5b171bd3e4 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMHostsListResponseTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMHostsListResponseTests.java @@ -13,33 +13,25 @@ public final class VMHostsListResponseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VMHostsListResponse model = - BinaryData - .fromString( - "{\"value\":[{\"vmId\":\"ounqecano\",\"agentVersion\":\"upfh\",\"agentStatus\":\"ltrpmopj\"}],\"nextLink\":\"matuok\"}") - .toObject(VMHostsListResponse.class); - Assertions.assertEquals("ounqecano", model.value().get(0).vmId()); - Assertions.assertEquals("upfh", model.value().get(0).agentVersion()); - Assertions.assertEquals("ltrpmopj", model.value().get(0).agentStatus()); - Assertions.assertEquals("matuok", model.nextLink()); + VMHostsListResponse model = BinaryData.fromString( + "{\"value\":[{\"vmId\":\"gxlefgugnxkrxd\",\"agentVersion\":\"i\",\"agentStatus\":\"thz\"}],\"nextLink\":\"qdrabhjybigehoqf\"}") + .toObject(VMHostsListResponse.class); + Assertions.assertEquals("gxlefgugnxkrxd", model.value().get(0).vmId()); + Assertions.assertEquals("i", model.value().get(0).agentVersion()); + Assertions.assertEquals("thz", model.value().get(0).agentStatus()); + Assertions.assertEquals("qdrabhjybigehoqf", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VMHostsListResponse model = - new VMHostsListResponse() - .withValue( - Arrays - .asList( - new VMInfoInner() - .withVmId("ounqecano") - .withAgentVersion("upfh") - .withAgentStatus("ltrpmopj"))) - .withNextLink("matuok"); + VMHostsListResponse model = new VMHostsListResponse() + .withValue(Arrays + .asList(new VMInfoInner().withVmId("gxlefgugnxkrxd").withAgentVersion("i").withAgentStatus("thz"))) + .withNextLink("qdrabhjybigehoqf"); model = BinaryData.fromObject(model).toObject(VMHostsListResponse.class); - Assertions.assertEquals("ounqecano", model.value().get(0).vmId()); - Assertions.assertEquals("upfh", model.value().get(0).agentVersion()); - Assertions.assertEquals("ltrpmopj", model.value().get(0).agentStatus()); - Assertions.assertEquals("matuok", model.nextLink()); + Assertions.assertEquals("gxlefgugnxkrxd", model.value().get(0).vmId()); + Assertions.assertEquals("i", model.value().get(0).agentVersion()); + Assertions.assertEquals("thz", model.value().get(0).agentStatus()); + Assertions.assertEquals("qdrabhjybigehoqf", model.nextLink()); } } diff --git a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMInfoInnerTests.java b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMInfoInnerTests.java index 8679fe6298854..8fc9411a20b82 100644 --- a/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMInfoInnerTests.java +++ b/sdk/newrelicobservability/azure-resourcemanager-newrelicobservability/src/test/java/com/azure/resourcemanager/newrelicobservability/generated/VMInfoInnerTests.java @@ -11,21 +11,21 @@ public final class VMInfoInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - VMInfoInner model = - BinaryData - .fromString("{\"vmId\":\"fu\",\"agentVersion\":\"aodsfcpkv\",\"agentStatus\":\"dpuozmyz\"}") - .toObject(VMInfoInner.class); - Assertions.assertEquals("fu", model.vmId()); - Assertions.assertEquals("aodsfcpkv", model.agentVersion()); - Assertions.assertEquals("dpuozmyz", model.agentStatus()); + VMInfoInner model = BinaryData + .fromString("{\"vmId\":\"wska\",\"agentVersion\":\"ktzlcuiywg\",\"agentStatus\":\"wgndrvynhzgpp\"}") + .toObject(VMInfoInner.class); + Assertions.assertEquals("wska", model.vmId()); + Assertions.assertEquals("ktzlcuiywg", model.agentVersion()); + Assertions.assertEquals("wgndrvynhzgpp", model.agentStatus()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - VMInfoInner model = new VMInfoInner().withVmId("fu").withAgentVersion("aodsfcpkv").withAgentStatus("dpuozmyz"); + VMInfoInner model + = new VMInfoInner().withVmId("wska").withAgentVersion("ktzlcuiywg").withAgentStatus("wgndrvynhzgpp"); model = BinaryData.fromObject(model).toObject(VMInfoInner.class); - Assertions.assertEquals("fu", model.vmId()); - Assertions.assertEquals("aodsfcpkv", model.agentVersion()); - Assertions.assertEquals("dpuozmyz", model.agentStatus()); + Assertions.assertEquals("wska", model.vmId()); + Assertions.assertEquals("ktzlcuiywg", model.agentVersion()); + Assertions.assertEquals("wgndrvynhzgpp", model.agentStatus()); } } diff --git a/sdk/nginx/azure-resourcemanager-nginx/pom.xml b/sdk/nginx/azure-resourcemanager-nginx/pom.xml index 84b3b6343bbb6..b52caea098fa3 100644 --- a/sdk/nginx/azure-resourcemanager-nginx/pom.xml +++ b/sdk/nginx/azure-resourcemanager-nginx/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/CHANGELOG.md b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/CHANGELOG.md index f0d3470b4877c..f11cd2034fc87 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/CHANGELOG.md +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.0.0-beta.4 (Unreleased) +## 1.0.0-beta.5 (Unreleased) ### Features Added @@ -10,6 +10,372 @@ ### Other Changes +## 1.0.0-beta.4 (2024-03-15) + +- Azure Resource Manager NotificationHubs client library for Java. This package contains Microsoft Azure SDK for NotificationHubs Management SDK. Microsoft Notification Hubs Resource Provider REST API. Package tag package-preview-2023-10. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Breaking Changes + +* `models.NamespaceCreateOrUpdateParameters` was removed + +* `models.PolicykeyResource` was removed + +* `models.SharedAccessAuthorizationRuleCreateOrUpdateParameters` was removed + +* `models.NotificationHubCreateOrUpdateParameters` was removed + +#### `models.NamespaceResource$DefinitionStages` was modified + +* `withLocation(java.lang.String)` was removed in stage 1 +* `withLocation(com.azure.core.management.Region)` was removed in stage 1 +* Stage 3 was added + +#### `models.SharedAccessAuthorizationRuleResource$DefinitionStages` was modified + +* `withExistingNamespace(java.lang.String,java.lang.String)` was removed in stage 1 + +#### `models.SharedAccessAuthorizationRuleResource$Definition` was modified + +* `withProperties(fluent.models.SharedAccessAuthorizationRuleProperties)` was removed +* `withExistingNamespace(java.lang.String,java.lang.String)` was removed + +#### `models.NotificationHubResource` was modified + +* `debugSendWithResponse(java.lang.Object,com.azure.core.util.Context)` was removed + +#### `models.SharedAccessAuthorizationRuleListResult` was modified + +* `withValue(java.util.List)` was removed +* `withNextLink(java.lang.String)` was removed + +#### `models.DebugSendResponse` was modified + +* `java.lang.Float failure()` -> `java.lang.Long failure()` +* `java.lang.Object results()` -> `java.util.List results()` +* `sku()` was removed +* `java.lang.Float success()` -> `java.lang.Long success()` + +#### `models.PnsCredentialsResource` was modified + +* `sku()` was removed + +#### `models.NotificationHubResource$Update` was modified + +* `withAuthorizationRules(java.util.List)` was removed +* `withNamePropertiesName(java.lang.String)` was removed + +#### `models.NotificationHubListResult` was modified + +* `withValue(java.util.List)` was removed +* `withNextLink(java.lang.String)` was removed + +#### `models.SharedAccessAuthorizationRuleResource$Update` was modified + +* `withProperties(fluent.models.SharedAccessAuthorizationRuleProperties)` was removed + +#### `models.Namespaces` was modified + +* `regenerateKeysWithResponse(java.lang.String,java.lang.String,java.lang.String,models.PolicykeyResource,com.azure.core.util.Context)` was removed +* `delete(java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `deleteAuthorizationRuleByIdWithResponse(java.lang.String,com.azure.core.util.Context)` was removed +* `listByResourceGroup(java.lang.String,com.azure.core.util.Context)` was removed +* `defineAuthorizationRule(java.lang.String)` was removed +* `deleteAuthorizationRuleById(java.lang.String)` was removed +* `getAuthorizationRuleById(java.lang.String)` was removed +* `void deleteByIdWithResponse(java.lang.String,com.azure.core.util.Context)` -> `com.azure.core.http.rest.Response deleteByIdWithResponse(java.lang.String,com.azure.core.util.Context)` +* `list(com.azure.core.util.Context)` was removed +* `regenerateKeys(java.lang.String,java.lang.String,java.lang.String,models.PolicykeyResource)` was removed +* `getAuthorizationRuleByIdWithResponse(java.lang.String,com.azure.core.util.Context)` was removed + +#### `models.AccessRights` was modified + +* `toString()` was removed +* `models.AccessRights[] values()` -> `java.util.Collection values()` +* `valueOf(java.lang.String)` was removed + +#### `models.NotificationHubPatchParameters` was modified + +* `withLocation(java.lang.String)` was removed +* `withTags(java.util.Map)` was removed +* `withNamePropertiesName(java.lang.String)` was removed +* `withAuthorizationRules(java.util.List)` was removed +* `namePropertiesName()` was removed + +#### `models.NotificationHubResource$Definition` was modified + +* `withAuthorizationRules(java.util.List)` was removed + +#### `NotificationHubsManager` was modified + +* `fluent.NotificationHubsManagementClient serviceClient()` -> `fluent.NotificationHubsRPClient serviceClient()` + +#### `models.NotificationHubs` was modified + +* `createOrUpdateAuthorizationRule(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.SharedAccessAuthorizationRuleCreateOrUpdateParameters)` was removed +* `createOrUpdateAuthorizationRuleWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.SharedAccessAuthorizationRuleCreateOrUpdateParameters,com.azure.core.util.Context)` was removed +* `list(java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `debugSendWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.Object,com.azure.core.util.Context)` was removed +* `regenerateKeys(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.PolicykeyResource)` was removed +* `regenerateKeysWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.PolicykeyResource,com.azure.core.util.Context)` was removed + +#### `models.NamespaceListResult` was modified + +* `withNextLink(java.lang.String)` was removed +* `withValue(java.util.List)` was removed + +#### `models.NamespaceType` was modified + +* `toString()` was removed +* `models.NamespaceType[] values()` -> `java.util.Collection values()` +* `valueOf(java.lang.String)` was removed + +#### `models.SharedAccessAuthorizationRuleResource` was modified + +* `sku()` was removed +* `regenerateKeys(models.PolicykeyResource)` was removed +* `java.lang.String modifiedTime()` -> `java.time.OffsetDateTime modifiedTime()` +* `java.lang.String createdTime()` -> `java.time.OffsetDateTime createdTime()` +* `regenerateKeysWithResponse(models.PolicykeyResource,com.azure.core.util.Context)` was removed + +#### `models.NamespaceResource$Definition` was modified + +* `withServiceBusEndpoint(java.lang.String)` was removed +* `withNamePropertiesName(java.lang.String)` was removed +* `withLocation(java.lang.String)` was removed +* `withCritical(java.lang.Boolean)` was removed +* `withRegion(java.lang.String)` was removed +* `withSubscriptionId(java.lang.String)` was removed +* `withUpdatedAt(java.time.OffsetDateTime)` was removed +* `withProvisioningState(java.lang.String)` was removed +* `withEnabled(java.lang.Boolean)` was removed +* `withStatus(java.lang.String)` was removed +* `withCreatedAt(java.time.OffsetDateTime)` was removed +* `withLocation(com.azure.core.management.Region)` was removed + +#### `models.NamespaceResource` was modified + +* `java.lang.String status()` -> `models.NamespaceStatus status()` +* `java.lang.String provisioningState()` -> `models.OperationProvisioningState provisioningState()` + +### Features Added + +* `models.OperationProperties` was added + +* `models.PrivateLinkConnectionStatus` was added + +* `models.PolicyKeyResource` was added + +* `models.ReplicationRegion` was added + +* `models.Availability` was added + +* `models.RemotePrivateEndpointConnection` was added + +* `models.NamespaceStatus` was added + +* `models.OperationProvisioningState` was added + +* `models.PrivateEndpointConnections` was added + +* `models.NetworkAcls` was added + +* `models.RegistrationResult` was added + +* `models.BrowserCredential` was added + +* `models.PrivateEndpointConnectionProperties` was added + +* `models.FcmV1Credential` was added + +* `models.PrivateLinkResourceListResult` was added + +* `models.PrivateEndpointConnectionResource` was added + +* `models.LogSpecification` was added + +* `models.ZoneRedundancyPreference` was added + +* `models.IpRule` was added + +* `models.PrivateLinkResourceProperties` was added + +* `models.ServiceSpecification` was added + +* `models.RemotePrivateLinkServiceConnectionState` was added + +* `models.XiaomiCredential` was added + +* `models.PublicNetworkAccess` was added + +* `models.PrivateLinkResource` was added + +* `models.PrivateEndpointConnectionResourceListResult` was added + +* `models.MetricSpecification` was added + +* `models.PublicInternetAuthorizationRule` was added + +* `models.PolicyKeyType` was added + +* `models.PrivateEndpointConnectionProvisioningState` was added + +#### `models.SharedAccessAuthorizationRuleResource$Definition` was modified + +* `withRegion(java.lang.String)` was added +* `withExistingNotificationHub(java.lang.String,java.lang.String,java.lang.String)` was added +* `withRights(java.util.List)` was added +* `withPrimaryKey(java.lang.String)` was added +* `withSecondaryKey(java.lang.String)` was added +* `withRegion(com.azure.core.management.Region)` was added +* `withTags(java.util.Map)` was added + +#### `models.NotificationHubResource` was modified + +* `dailyMaxActiveDevices()` was added +* `fcmV1Credential()` was added +* `resourceGroupName()` was added +* `xiaomiCredential()` was added +* `systemData()` was added +* `debugSendWithResponse(com.azure.core.util.Context)` was added +* `browserCredential()` was added + +#### `models.OperationDisplay` was modified + +* `description()` was added + +#### `models.NamespaceResource$Update` was modified + +* `withProperties(fluent.models.NamespaceProperties)` was added + +#### `models.DebugSendResponse` was modified + +* `systemData()` was added + +#### `models.NamespacePatchParameters` was modified + +* `withProperties(fluent.models.NamespaceProperties)` was added +* `properties()` was added + +#### `models.PnsCredentialsResource` was modified + +* `browserCredential()` was added +* `systemData()` was added +* `xiaomiCredential()` was added +* `fcmV1Credential()` was added + +#### `models.NotificationHubResource$Update` was modified + +* `withBrowserCredential(models.BrowserCredential)` was added +* `withXiaomiCredential(models.XiaomiCredential)` was added +* `withFcmV1Credential(models.FcmV1Credential)` was added + +#### `models.SharedAccessAuthorizationRuleResource$Update` was modified + +* `withRights(java.util.List)` was added +* `withSecondaryKey(java.lang.String)` was added +* `withTags(java.util.Map)` was added +* `withPrimaryKey(java.lang.String)` was added + +#### `models.Namespaces` was modified + +* `regenerateKeys(java.lang.String,java.lang.String,java.lang.String,models.PolicyKeyResource)` was added +* `getPnsCredentials(java.lang.String,java.lang.String)` was added +* `createOrUpdateAuthorizationRuleWithResponse(java.lang.String,java.lang.String,java.lang.String,fluent.models.SharedAccessAuthorizationRuleResourceInner,com.azure.core.util.Context)` was added +* `regenerateKeysWithResponse(java.lang.String,java.lang.String,java.lang.String,models.PolicyKeyResource,com.azure.core.util.Context)` was added +* `createOrUpdateAuthorizationRule(java.lang.String,java.lang.String,java.lang.String,fluent.models.SharedAccessAuthorizationRuleResourceInner)` was added +* `getPnsCredentialsWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `deleteByResourceGroupWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listByResourceGroup(java.lang.String,java.lang.String,java.lang.Integer,com.azure.core.util.Context)` was added +* `list(java.lang.String,java.lang.Integer,com.azure.core.util.Context)` was added + +#### `models.NotificationHubPatchParameters` was modified + +* `fcmV1Credential()` was added +* `withFcmV1Credential(models.FcmV1Credential)` was added +* `browserCredential()` was added +* `dailyMaxActiveDevices()` was added +* `name()` was added +* `tags()` was added +* `withXiaomiCredential(models.XiaomiCredential)` was added +* `xiaomiCredential()` was added +* `withBrowserCredential(models.BrowserCredential)` was added +* `withName(java.lang.String)` was added + +#### `models.WnsCredential` was modified + +* `certificateKey()` was added +* `withCertificateKey(java.lang.String)` was added +* `withWnsCertificate(java.lang.String)` was added +* `wnsCertificate()` was added + +#### `models.NotificationHubResource$Definition` was modified + +* `withXiaomiCredential(models.XiaomiCredential)` was added +* `withBrowserCredential(models.BrowserCredential)` was added +* `withFcmV1Credential(models.FcmV1Credential)` was added + +#### `models.Operation` was modified + +* `properties()` was added +* `isDataAction()` was added + +#### `NotificationHubsManager` was modified + +* `authenticate(com.azure.core.http.HttpPipeline,com.azure.core.management.profile.AzureProfile)` was added +* `privateEndpointConnections()` was added + +#### `models.NotificationHubs` was modified + +* `getAuthorizationRuleById(java.lang.String)` was added +* `defineAuthorizationRule(java.lang.String)` was added +* `list(java.lang.String,java.lang.String,java.lang.String,java.lang.Integer,com.azure.core.util.Context)` was added +* `debugSendWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `deleteAuthorizationRuleById(java.lang.String)` was added +* `regenerateKeys(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.PolicyKeyResource)` was added +* `getAuthorizationRuleByIdWithResponse(java.lang.String,com.azure.core.util.Context)` was added +* `regenerateKeysWithResponse(java.lang.String,java.lang.String,java.lang.String,java.lang.String,models.PolicyKeyResource,com.azure.core.util.Context)` was added +* `deleteAuthorizationRuleByIdWithResponse(java.lang.String,com.azure.core.util.Context)` was added + +#### `models.CheckAvailabilityResult` was modified + +* `systemData()` was added + +#### `models.SharedAccessAuthorizationRuleResource` was modified + +* `regenerateKeysWithResponse(models.PolicyKeyResource,com.azure.core.util.Context)` was added +* `resourceGroupName()` was added +* `regenerateKeys(models.PolicyKeyResource)` was added +* `systemData()` was added + +#### `NotificationHubsManager$Configurable` was modified + +* `withRetryOptions(com.azure.core.http.policy.RetryOptions)` was added + +#### `models.NamespaceResource$Definition` was modified + +* `withRegion(java.lang.String)` was added +* `withRegion(com.azure.core.management.Region)` was added +* `withZoneRedundancy(models.ZoneRedundancyPreference)` was added +* `withNetworkAcls(models.NetworkAcls)` was added +* `withPnsCredentials(fluent.models.PnsCredentials)` was added +* `withReplicationRegion(models.ReplicationRegion)` was added +* `withPublicNetworkAccess(models.PublicNetworkAccess)` was added +* `withStatus(models.NamespaceStatus)` was added +* `withProvisioningState(models.OperationProvisioningState)` was added + +#### `models.NamespaceResource` was modified + +* `systemData()` was added +* `publicNetworkAccess()` was added +* `zoneRedundancy()` was added +* `privateEndpointConnections()` was added +* `getPnsCredentials()` was added +* `getPnsCredentialsWithResponse(com.azure.core.util.Context)` was added +* `networkAcls()` was added +* `replicationRegion()` was added +* `resourceGroupName()` was added +* `pnsCredentials()` was added + ## 1.0.0-beta.3 (2022-01-04) - Azure Resource Manager NotificationHubs client library for Java. This package contains Microsoft Azure SDK for NotificationHubs Management SDK. Azure NotificationHub client. Package tag package-2017-04. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/README.md b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/README.md index e2f0444c7ae9d..ea3c281570379 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/README.md +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/README.md @@ -2,7 +2,7 @@ Azure Resource Manager NotificationHubs client library for Java. -This package contains Microsoft Azure SDK for NotificationHubs Management SDK. Azure NotificationHub client. Package tag package-2017-04. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for NotificationHubs Management SDK. Microsoft Notification Hubs Resource Provider REST API. Package tag package-preview-2023-10. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-notificationhubs - 1.0.0-beta.3 + 1.0.0-beta.4 ``` [//]: # ({x-version-update-end}) @@ -41,19 +41,19 @@ Various documentation is available to help you get started Azure Management Libraries require a `TokenCredential` implementation for authentication and an `HttpClient` implementation for HTTP client. -[Azure Identity][azure_identity] package and [Azure Core Netty HTTP][azure_core_http_netty] package provide the default implementation. +[Azure Identity][azure_identity] and [Azure Core Netty HTTP][azure_core_http_netty] packages provide the default implementation. ### Authentication -By default, Azure Active Directory token authentication depends on correct configure of following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. - `AZURE_CLIENT_SECRET` or `AZURE_CLIENT_CERTIFICATE_PATH` for client secret or client certificate. -In addition, Azure subscription ID can be configured via environment variable `AZURE_SUBSCRIPTION_ID`. +In addition, Azure subscription ID can be configured via `AZURE_SUBSCRIPTION_ID` environment variable. -With above configuration, `azure` client can be authenticated by following code: +With above configuration, `azure` client can be authenticated using the following code: ```java AzureProfile profile = new AzureProfile(AzureEnvironment.AZURE); @@ -83,20 +83,25 @@ See [API design][design] for general introduction on design and key concepts on ## Contributing -For details on contributing to this repository, see the [contributing guide](https://github.com/Azure/azure-sdk-for-java/blob/main/CONTRIBUTING.md). +For details on contributing to this repository, see the [contributing guide][cg]. -1. Fork it -1. Create your feature branch (`git checkout -b my-new-feature`) -1. Commit your changes (`git commit -am 'Add some feature'`) -1. Push to the branch (`git push origin my-new-feature`) -1. Create new Pull Request +This project welcomes contributions and suggestions. Most contributions require you to agree to a Contributor License Agreement (CLA) declaring that you have the right to, and actually do, grant us the rights to use your contribution. For details, visit . + +When you submit a pull request, a CLA-bot will automatically determine whether you need to provide a CLA and decorate the PR appropriately (e.g., label, comment). Simply follow the instructions provided by the bot. You will only need to do this once across all repositories using our CLA. + +This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For more information see the [Code of Conduct FAQ][coc_faq] or contact with any additional questions or comments. [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty [authenticate]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/resourcemanager/docs/AUTH.md [design]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/resourcemanager/docs/DESIGN.md +[cg]: https://github.com/Azure/azure-sdk-for-java/blob/main/CONTRIBUTING.md +[coc]: https://opensource.microsoft.com/codeofconduct/ +[coc_faq]: https://opensource.microsoft.com/codeofconduct/faq/ + +![Impressions](https://azure-sdk-impressions.azurewebsites.net/api/impressions/azure-sdk-for-java%2Fsdk%2Fnotificationhubs%2Fazure-resourcemanager-notificationhubs%2FREADME.png) diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/SAMPLE.md b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/SAMPLE.md index 973de5b2e5253..0660ffe24c3ef 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/SAMPLE.md +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/SAMPLE.md @@ -10,12 +10,13 @@ - [DeleteAuthorizationRule](#namespaces_deleteauthorizationrule) - [GetAuthorizationRule](#namespaces_getauthorizationrule) - [GetByResourceGroup](#namespaces_getbyresourcegroup) +- [GetPnsCredentials](#namespaces_getpnscredentials) - [List](#namespaces_list) - [ListAuthorizationRules](#namespaces_listauthorizationrules) - [ListByResourceGroup](#namespaces_listbyresourcegroup) - [ListKeys](#namespaces_listkeys) -- [Patch](#namespaces_patch) - [RegenerateKeys](#namespaces_regeneratekeys) +- [Update](#namespaces_update) ## NotificationHubs @@ -31,34 +32,44 @@ - [List](#notificationhubs_list) - [ListAuthorizationRules](#notificationhubs_listauthorizationrules) - [ListKeys](#notificationhubs_listkeys) -- [Patch](#notificationhubs_patch) - [RegenerateKeys](#notificationhubs_regeneratekeys) +- [Update](#notificationhubs_update) ## Operations - [List](#operations_list) + +## PrivateEndpointConnections + +- [Delete](#privateendpointconnections_delete) +- [Get](#privateendpointconnections_get) +- [GetGroupId](#privateendpointconnections_getgroupid) +- [List](#privateendpointconnections_list) +- [ListGroupIds](#privateendpointconnections_listgroupids) +- [Update](#privateendpointconnections_update) ### Namespaces_CheckAvailability ```java -import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -/** Samples for Namespaces CheckAvailability. */ +/** + * Samples for Namespaces CheckAvailability. + */ public final class NamespacesCheckAvailabilitySamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceCheckNameAvailability.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/CheckAvailability.json */ /** - * Sample code: NameSpaceCheckNameAvailability. - * + * Sample code: Namespaces_CheckAvailability. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceCheckNameAvailability( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .checkAvailabilityWithResponse( - new CheckAvailabilityParameters().withName("sdk-Namespace-2924"), Context.NONE); + public static void + namespacesCheckAvailability(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().checkAvailabilityWithResponse( + new CheckAvailabilityParameters().withName("sdk-Namespace-2924"), com.azure.core.util.Context.NONE); } } ``` @@ -66,32 +77,45 @@ public final class NamespacesCheckAvailabilitySamples { ### Namespaces_CreateOrUpdate ```java +import com.azure.resourcemanager.notificationhubs.models.AccessRights; +import com.azure.resourcemanager.notificationhubs.models.IpRule; +import com.azure.resourcemanager.notificationhubs.models.NetworkAcls; +import com.azure.resourcemanager.notificationhubs.models.PublicInternetAuthorizationRule; import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.SkuName; +import com.azure.resourcemanager.notificationhubs.models.ZoneRedundancyPreference; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for Namespaces CreateOrUpdate. */ +/** + * Samples for Namespaces CreateOrUpdate. + */ public final class NamespacesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/CreateOrUpdate.json */ /** - * Sample code: NameSpaceCreate. - * + * Sample code: Namespaces_CreateOrUpdate. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceCreate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .define("nh-sdk-ns") - .withLocation("South Central US") - .withExistingResourceGroup("5ktrial") - .withTags(mapOf("tag1", "value1", "tag2", "value2")) + public static void + namespacesCreateOrUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().define("nh-sdk-ns").withRegion("South Central US").withExistingResourceGroup("5ktrial") .withSku(new Sku().withName(SkuName.STANDARD).withTier("Standard")) + .withTags(mapOf("tag1", "value1", "tag2", "value2")).withZoneRedundancy(ZoneRedundancyPreference.ENABLED) + .withNetworkAcls(new NetworkAcls() + .withIpRules(Arrays.asList(new IpRule().withIpMask("185.48.100.00/24") + .withRights(Arrays.asList(AccessRights.MANAGE, AccessRights.SEND, AccessRights.LISTEN)))) + .withPublicNetworkRule( + new PublicInternetAuthorizationRule().withRights(Arrays.asList(AccessRights.LISTEN)))) .create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -108,30 +132,31 @@ public final class NamespacesCreateOrUpdateSamples { ### Namespaces_CreateOrUpdateAuthorizationRule ```java -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; +import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.AccessRights; import java.util.Arrays; -/** Samples for Namespaces CreateOrUpdateAuthorizationRule. */ +/** + * Samples for Namespaces CreateOrUpdateAuthorizationRule. + */ public final class NamespacesCreateOrUpdateAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleCreateOrUpdate.json */ /** - * Sample code: NameSpaceAuthorizationRuleCreate. - * + * Sample code: Namespaces_CreateOrUpdateAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleCreate( + public static void namespacesCreateOrUpdateAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .defineAuthorizationRule("sdk-AuthRules-1788") - .withExistingNamespace("5ktrial", "nh-sdk-ns") - .withProperties( - new SharedAccessAuthorizationRuleProperties() - .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND))) - .create(); + manager.namespaces() + .createOrUpdateAuthorizationRuleWithResponse( + "5ktrial", "nh-sdk-ns", "sdk-AuthRules-1788", new SharedAccessAuthorizationRuleResourceInner() + .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND)), + com.azure.core.util.Context.NONE); } } ``` @@ -139,20 +164,23 @@ public final class NamespacesCreateOrUpdateAuthorizationRuleSamples { ### Namespaces_Delete ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces Delete. */ +/** + * Samples for Namespaces Delete. + */ public final class NamespacesDeleteSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/Delete.json */ /** - * Sample code: NameSpaceDelete. - * + * Sample code: Namespaces_Delete. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().delete("5ktrial", "nh-sdk-ns", Context.NONE); + public static void namespacesDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().deleteByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", + com.azure.core.util.Context.NONE); } } ``` @@ -160,23 +188,24 @@ public final class NamespacesDeleteSamples { ### Namespaces_DeleteAuthorizationRule ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces DeleteAuthorizationRule. */ +/** + * Samples for Namespaces DeleteAuthorizationRule. + */ public final class NamespacesDeleteAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleDelete.json */ /** - * Sample code: NameSpaceAuthorizationRuleDelete. - * + * Sample code: Namespaces_DeleteAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleDelete( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .deleteAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", Context.NONE); + public static void + namespacesDeleteAuthorizationRule(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().deleteAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + com.azure.core.util.Context.NONE); } } ``` @@ -184,23 +213,24 @@ public final class NamespacesDeleteAuthorizationRuleSamples { ### Namespaces_GetAuthorizationRule ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces GetAuthorizationRule. */ +/** + * Samples for Namespaces GetAuthorizationRule. + */ public final class NamespacesGetAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleGet.json */ /** - * Sample code: NameSpaceAuthorizationRuleGet. - * + * Sample code: Namespaces_GetAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleGet( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .getAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", Context.NONE); + public static void + namespacesGetAuthorizationRule(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().getAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + com.azure.core.util.Context.NONE); } } ``` @@ -208,41 +238,69 @@ public final class NamespacesGetAuthorizationRuleSamples { ### Namespaces_GetByResourceGroup ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces GetByResourceGroup. */ +/** + * Samples for Namespaces GetByResourceGroup. + */ public final class NamespacesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/Get.json */ /** - * Sample code: NameSpaceGet. - * + * Sample code: Namespaces_Get. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", Context.NONE); + public static void namespacesGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); } } ``` -### Namespaces_List +### Namespaces_GetPnsCredentials ```java -import com.azure.core.util.Context; +/** + * Samples for Namespaces GetPnsCredentials. + */ +public final class NamespacesGetPnsCredentialsSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PnsCredentialsGet.json + */ + /** + * Sample code: Namespaces_GetPnsCredentials. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + namespacesGetPnsCredentials(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().getPnsCredentialsWithResponse("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); + } +} +``` + +### Namespaces_List -/** Samples for Namespaces List. */ +```java +/** + * Samples for Namespaces List. + */ public final class NamespacesListSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceList.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/ListBySubscription.json */ /** - * Sample code: NameSpaceList. - * + * Sample code: Namespaces_ListAll. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().list(Context.NONE); + public static void namespacesListAll(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().list(null, null, com.azure.core.util.Context.NONE); } } ``` @@ -250,21 +308,23 @@ public final class NamespacesListSamples { ### Namespaces_ListAuthorizationRules ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces ListAuthorizationRules. */ +/** + * Samples for Namespaces ListAuthorizationRules. + */ public final class NamespacesListAuthorizationRulesSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleListAll.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleList.json */ /** - * Sample code: NameSpaceAuthorizationRuleListAll. - * + * Sample code: Namespaces_ListAuthorizationRules. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleListAll( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().listAuthorizationRules("5ktrial", "nh-sdk-ns", Context.NONE); + public static void + namespacesListAuthorizationRules(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().listAuthorizationRules("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); } } ``` @@ -272,21 +332,22 @@ public final class NamespacesListAuthorizationRulesSamples { ### Namespaces_ListByResourceGroup ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces ListByResourceGroup. */ +/** + * Samples for Namespaces ListByResourceGroup. + */ public final class NamespacesListByResourceGroupSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceListByResourceGroup.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/ListByResourceGroup.json */ /** - * Sample code: NameSpaceListByResourceGroup. - * + * Sample code: Namespaces_List. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceListByResourceGroup( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().listByResourceGroup("5ktrial", Context.NONE); + public static void namespacesList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().listByResourceGroup("5ktrial", null, null, com.azure.core.util.Context.NONE); } } ``` @@ -294,55 +355,92 @@ public final class NamespacesListByResourceGroupSamples { ### Namespaces_ListKeys ```java -import com.azure.core.util.Context; - -/** Samples for Namespaces ListKeys. */ +/** + * Samples for Namespaces ListKeys. + */ public final class NamespacesListKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleListKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleListKeys.json */ /** - * Sample code: NameSpaceAuthorizationRuleListKey. - * + * Sample code: Namespaces_ListKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleListKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().listKeysWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", Context.NONE); + public static void namespacesListKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().listKeysWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + com.azure.core.util.Context.NONE); + } +} +``` + +### Namespaces_RegenerateKeys + +```java +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyType; + +/** + * Samples for Namespaces RegenerateKeys. + */ +public final class NamespacesRegenerateKeysSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleRegenerateKey.json + */ + /** + * Sample code: Namespaces_RegenerateKeys. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + namespacesRegenerateKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().regenerateKeysWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + new PolicyKeyResource().withPolicyKey(PolicyKeyType.PRIMARY_KEY), com.azure.core.util.Context.NONE); } } ``` -### Namespaces_Patch +### Namespaces_Update ```java -import com.azure.core.util.Context; +import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentials; +import com.azure.resourcemanager.notificationhubs.models.GcmCredential; import com.azure.resourcemanager.notificationhubs.models.NamespaceResource; import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.SkuName; import java.util.HashMap; import java.util.Map; -/** Samples for Namespaces Patch. */ -public final class NamespacesPatchSamples { +/** + * Samples for Namespaces Update. + */ +public final class NamespacesUpdateSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceUpdate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/Update.json */ /** - * Sample code: NameSpaceUpdate. - * + * Sample code: Namespaces_Update. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - NamespaceResource resource = - manager.namespaces().getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", Context.NONE).getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withSku(new Sku().withName(SkuName.STANDARD).withTier("Standard")) + public static void namespacesUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + NamespaceResource resource = manager.namespaces() + .getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value3")).withSku(new Sku().withName(SkuName.FREE)) + .withProperties( + new NamespaceProperties().withPnsCredentials(new PnsCredentials().withGcmCredential(new GcmCredential() + .withGcmEndpoint("https://fcm.googleapis.com/fcm/send").withGoogleApiKey("fakeTokenPlaceholder")))) .apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -356,61 +454,30 @@ public final class NamespacesPatchSamples { } ``` -### Namespaces_RegenerateKeys - -```java -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; - -/** Samples for Namespaces RegenerateKeys. */ -public final class NamespacesRegenerateKeysSamples { - /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleRegenrateKey.json - */ - /** - * Sample code: NameSpaceAuthorizationRuleRegenerateKey. - * - * @param manager Entry point to NotificationHubsManager. - */ - public static void nameSpaceAuthorizationRuleRegenerateKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .regenerateKeysWithResponse( - "5ktrial", - "nh-sdk-ns", - "RootManageSharedAccessKey", - new PolicykeyResource().withPolicyKey("PrimaryKey"), - Context.NONE); - } -} -``` - ### NotificationHubs_CheckNotificationHubAvailability ```java -import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -/** Samples for NotificationHubs CheckNotificationHubAvailability. */ +/** + * Samples for NotificationHubs CheckNotificationHubAvailability. + */ public final class NotificationHubsCheckNotificationHubAvailabilitySamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubCheckNameAvailability.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/CheckAvailability.json */ /** - * Sample code: notificationHubCheckNameAvailability. - * + * Sample code: NotificationHubs_CheckNotificationHubAvailability. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubCheckNameAvailability( + public static void notificationHubsCheckNotificationHubAvailability( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .checkNotificationHubAvailabilityWithResponse( - "5ktrial", - "locp-newns", - new CheckAvailabilityParameters().withName("sdktest").withLocation("West Europe"), - Context.NONE); + manager.notificationHubs().checkNotificationHubAvailabilityWithResponse("5ktrial", "locp-newns", + new CheckAvailabilityParameters().withName("sdktest").withLocation("West Europe"), + com.azure.core.util.Context.NONE); } } ``` @@ -418,24 +485,24 @@ public final class NotificationHubsCheckNotificationHubAvailabilitySamples { ### NotificationHubs_CreateOrUpdate ```java -/** Samples for NotificationHubs CreateOrUpdate. */ +/** + * Samples for NotificationHubs CreateOrUpdate. + */ public final class NotificationHubsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/CreateOrUpdate.json */ /** - * Sample code: NotificationHubCreate. - * + * Sample code: NotificationHubs_CreateOrUpdate. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubCreate( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .define("nh-sdk-hub") - .withRegion("eastus") - .withExistingNamespace("5ktrial", "nh-sdk-ns") - .create(); + public static void + notificationHubsCreateOrUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().define("nh-sdk-hub").withRegion("eastus") + .withExistingNamespace("5ktrial", "nh-sdk-ns").create(); } } ``` @@ -443,36 +510,28 @@ public final class NotificationHubsCreateOrUpdateSamples { ### NotificationHubs_CreateOrUpdateAuthorizationRule ```java -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; import com.azure.resourcemanager.notificationhubs.models.AccessRights; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; import java.util.Arrays; -/** Samples for NotificationHubs CreateOrUpdateAuthorizationRule. */ +/** + * Samples for NotificationHubs CreateOrUpdateAuthorizationRule. + */ public final class NotificationHubsCreateOrUpdateAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleCreateOrUpdate.json */ /** - * Sample code: NotificationHubAuthorizationRuleCreate. - * + * Sample code: NotificationHubs_CreateOrUpdateAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleCreate( + public static void notificationHubsCreateOrUpdateAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .createOrUpdateAuthorizationRuleWithResponse( - "5ktrial", - "nh-sdk-ns", - "nh-sdk-hub", - "DefaultListenSharedAccessSignature", - new SharedAccessAuthorizationRuleCreateOrUpdateParameters() - .withProperties( - new SharedAccessAuthorizationRuleProperties() - .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND))), - Context.NONE); + manager.notificationHubs().defineAuthorizationRule("MyManageSharedAccessKey") + .withExistingNotificationHub("5ktrial", "nh-sdk-ns", "nh-sdk-hub") + .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND)).create(); } } ``` @@ -480,33 +539,24 @@ public final class NotificationHubsCreateOrUpdateAuthorizationRuleSamples { ### NotificationHubs_DebugSend ```java -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.Context; -import com.azure.core.util.serializer.SerializerEncoding; -import java.io.IOException; - -/** Samples for NotificationHubs DebugSend. */ +/** + * Samples for NotificationHubs DebugSend. + */ public final class NotificationHubsDebugSendSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubDebugSend.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/DebugSend.json */ /** - * Sample code: debugsend. - * + * Sample code: NotificationHubs_DebugSend. + * * @param manager Entry point to NotificationHubsManager. */ - public static void debugsend(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) - throws IOException { - manager - .notificationHubs() - .debugSendWithResponse( - "5ktrial", - "nh-sdk-ns", - "nh-sdk-hub", - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize("{\"data\":{\"message\":\"Hello\"}}", Object.class, SerializerEncoding.JSON), - Context.NONE); + public static void + notificationHubsDebugSend(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().debugSendWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } ``` @@ -514,21 +564,24 @@ public final class NotificationHubsDebugSendSamples { ### NotificationHubs_Delete ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs Delete. */ +/** + * Samples for NotificationHubs Delete. + */ public final class NotificationHubsDeleteSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/Delete.json */ /** - * Sample code: NotificationHubDelete. - * + * Sample code: NotificationHubs_Delete. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubDelete( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().deleteWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + public static void + notificationHubsDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().deleteWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } ``` @@ -536,24 +589,24 @@ public final class NotificationHubsDeleteSamples { ### NotificationHubs_DeleteAuthorizationRule ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs DeleteAuthorizationRule. */ +/** + * Samples for NotificationHubs DeleteAuthorizationRule. + */ public final class NotificationHubsDeleteAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleDelete.json */ /** - * Sample code: NotificationHubAuthorizationRuleDelete. - * + * Sample code: NotificationHubs_DeleteAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleDelete( + public static void notificationHubsDeleteAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .deleteAuthorizationRuleWithResponse( - "5ktrial", "nh-sdk-ns", "nh-sdk-hub", "DefaultListenSharedAccessSignature", Context.NONE); + manager.notificationHubs().deleteAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + "DefaultListenSharedAccessSignature", com.azure.core.util.Context.NONE); } } ``` @@ -561,20 +614,23 @@ public final class NotificationHubsDeleteAuthorizationRuleSamples { ### NotificationHubs_Get ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs Get. */ +/** + * Samples for NotificationHubs Get. + */ public final class NotificationHubsGetSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/Get.json */ /** - * Sample code: NotificationHubGet. - * + * Sample code: NotificationHubs_Get. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().getWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + public static void notificationHubsGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().getWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } ``` @@ -582,24 +638,24 @@ public final class NotificationHubsGetSamples { ### NotificationHubs_GetAuthorizationRule ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs GetAuthorizationRule. */ +/** + * Samples for NotificationHubs GetAuthorizationRule. + */ public final class NotificationHubsGetAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleGet.json */ /** - * Sample code: NotificationHubAuthorizationRuleGet. - * + * Sample code: NotificationHubs_GetAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleGet( + public static void notificationHubsGetAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .getAuthorizationRuleWithResponse( - "5ktrial", "nh-sdk-ns", "nh-sdk-hub", "DefaultListenSharedAccessSignature", Context.NONE); + manager.notificationHubs().getAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + "DefaultListenSharedAccessSignature", com.azure.core.util.Context.NONE); } } ``` @@ -607,21 +663,24 @@ public final class NotificationHubsGetAuthorizationRuleSamples { ### NotificationHubs_GetPnsCredentials ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs GetPnsCredentials. */ +/** + * Samples for NotificationHubs GetPnsCredentials. + */ public final class NotificationHubsGetPnsCredentialsSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubPnsCredentials.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/PnsCredentialsGet.json */ /** - * Sample code: notificationHubPnsCredentials. - * + * Sample code: NotificationHubs_GetPnsCredentials. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubPnsCredentials( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().getPnsCredentialsWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + public static void + notificationHubsGetPnsCredentials(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().getPnsCredentialsWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } ``` @@ -629,21 +688,23 @@ public final class NotificationHubsGetPnsCredentialsSamples { ### NotificationHubs_List ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs List. */ +/** + * Samples for NotificationHubs List. + */ public final class NotificationHubsListSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubListByNameSpace.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/List.json */ /** - * Sample code: NotificationHubListByNameSpace. - * + * Sample code: NotificationHubs_List. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubListByNameSpace( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().list("5ktrial", "nh-sdk-ns", Context.NONE); + public static void + notificationHubsList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().list("5ktrial", "nh-sdk-ns", null, null, com.azure.core.util.Context.NONE); } } ``` @@ -651,21 +712,24 @@ public final class NotificationHubsListSamples { ### NotificationHubs_ListAuthorizationRules ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs ListAuthorizationRules. */ +/** + * Samples for NotificationHubs ListAuthorizationRules. + */ public final class NotificationHubsListAuthorizationRulesSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleListAll.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleList.json */ /** - * Sample code: NotificationHubAuthorizationRuleListAll. - * + * Sample code: NotificationHubs_ListAuthorizationRules. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleListAll( + public static void notificationHubsListAuthorizationRules( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().listAuthorizationRules("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + manager.notificationHubs().listAuthorizationRules("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } ``` @@ -673,82 +737,85 @@ public final class NotificationHubsListAuthorizationRulesSamples { ### NotificationHubs_ListKeys ```java -import com.azure.core.util.Context; - -/** Samples for NotificationHubs ListKeys. */ +/** + * Samples for NotificationHubs ListKeys. + */ public final class NotificationHubsListKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleListKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleListKeys.json */ /** - * Sample code: NotificationHubAuthorizationRuleListKey. - * + * Sample code: NotificationHubs_ListKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleListKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .listKeysWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", "sdk-AuthRules-5800", Context.NONE); + public static void + notificationHubsListKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().listKeysWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", "sdk-AuthRules-5800", + com.azure.core.util.Context.NONE); } } ``` -### NotificationHubs_Patch +### NotificationHubs_RegenerateKeys ```java -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.NotificationHubResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyType; -/** Samples for NotificationHubs Patch. */ -public final class NotificationHubsPatchSamples { +/** + * Samples for NotificationHubs RegenerateKeys. + */ +public final class NotificationHubsRegenerateKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubPatch.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleRegenerateKey.json */ /** - * Sample code: NotificationHubPatch. - * + * Sample code: NotificationHubs_RegenerateKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubPatch( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - NotificationHubResource resource = - manager - .notificationHubs() - .getWithResponse("sdkresourceGroup", "nh-sdk-ns", "sdk-notificationHubs-8708", Context.NONE) - .getValue(); - resource.update().apply(); + public static void + notificationHubsRegenerateKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().regenerateKeysWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + "DefaultListenSharedAccessSignature", new PolicyKeyResource().withPolicyKey(PolicyKeyType.PRIMARY_KEY), + com.azure.core.util.Context.NONE); } } ``` -### NotificationHubs_RegenerateKeys +### NotificationHubs_Update ```java -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; +import com.azure.resourcemanager.notificationhubs.models.GcmCredential; +import com.azure.resourcemanager.notificationhubs.models.NotificationHubResource; -/** Samples for NotificationHubs RegenerateKeys. */ -public final class NotificationHubsRegenerateKeysSamples { +/** + * Samples for NotificationHubs Update. + */ +public final class NotificationHubsUpdateSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleRegenrateKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/Update.json */ /** - * Sample code: NotificationHubAuthorizationRuleRegenrateKey. - * + * Sample code: NotificationHubs_Update. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleRegenrateKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .regenerateKeysWithResponse( - "5ktrial", - "nh-sdk-ns", - "nh-sdk-hub", - "DefaultListenSharedAccessSignature", - new PolicykeyResource().withPolicyKey("PrimaryKey"), - Context.NONE); + public static void + notificationHubsUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + NotificationHubResource resource = manager.notificationHubs().getWithResponse("sdkresourceGroup", "nh-sdk-ns", + "sdk-notificationHubs-8708", com.azure.core.util.Context.NONE).getValue(); + resource + .update().withRegistrationTtl("10675199.02:48:05.4775807").withGcmCredential(new GcmCredential() + .withGcmEndpoint("https://fcm.googleapis.com/fcm/send").withGoogleApiKey("fakeTokenPlaceholder")) + .apply(); } } ``` @@ -756,20 +823,180 @@ public final class NotificationHubsRegenerateKeysSamples { ### Operations_List ```java -import com.azure.core.util.Context; - -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NHOperationsList.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NHOperationsList.json */ /** - * Sample code: OperationsList. - * + * Sample code: Operations_List. + * * @param manager Entry point to NotificationHubsManager. */ public static void operationsList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.operations().list(Context.NONE); + manager.operations().list(com.azure.core.util.Context.NONE); + } +} +``` + +### PrivateEndpointConnections_Delete + +```java +/** + * Samples for PrivateEndpointConnections Delete. + */ +public final class PrivateEndpointConnectionsDeleteSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionDelete.json + */ + /** + * Sample code: PrivateEndpointConnections_Delete. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().delete("5ktrial", "nh-sdk-ns", + "nh-sdk-ns.1fa229cd-bf3f-47f0-8c49-afb36723997e", com.azure.core.util.Context.NONE); + } +} +``` + +### PrivateEndpointConnections_Get + +```java +/** + * Samples for PrivateEndpointConnections Get. + */ +public final class PrivateEndpointConnectionsGetSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionGet.json + */ + /** + * Sample code: PrivateEndpointConnections_Get. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().getWithResponse("5ktrial", "nh-sdk-ns", + "nh-sdk-ns.1fa229cd-bf3f-47f0-8c49-afb36723997e", com.azure.core.util.Context.NONE); + } +} +``` + +### PrivateEndpointConnections_GetGroupId + +```java +/** + * Samples for PrivateEndpointConnections GetGroupId. + */ +public final class PrivateEndpointConnectionsGetGroupIdSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateLinkResourceGet.json + */ + /** + * Sample code: PrivateEndpointConnections_GetGroupId. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void privateEndpointConnectionsGetGroupId( + com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().getGroupIdWithResponse("5ktrial", "nh-sdk-ns", "namespace", + com.azure.core.util.Context.NONE); + } +} +``` + +### PrivateEndpointConnections_List + +```java +/** + * Samples for PrivateEndpointConnections List. + */ +public final class PrivateEndpointConnectionsListSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionList.json + */ + /** + * Sample code: PrivateEndpointConnections_List. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().list("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); + } +} +``` + +### PrivateEndpointConnections_ListGroupIds + +```java +/** + * Samples for PrivateEndpointConnections ListGroupIds. + */ +public final class PrivateEndpointConnectionsListGroupIdsSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateLinkResourceList.json + */ + /** + * Sample code: PrivateEndpointConnections_ListGroupIds. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void privateEndpointConnectionsListGroupIds( + com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().listGroupIds("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); + } +} +``` + +### PrivateEndpointConnections_Update + +```java +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState; + +/** + * Samples for PrivateEndpointConnections Update. + */ +public final class PrivateEndpointConnectionsUpdateSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionUpdate.json + */ + /** + * Sample code: PrivateEndpointConnections_Update. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().update("5ktrial", "nh-sdk-ns", + "nh-sdk-ns.1fa229cd-bf3f-47f0-8c49-afb36723997e", + new PrivateEndpointConnectionResourceInner().withProperties(new PrivateEndpointConnectionProperties() + .withPrivateEndpoint(new RemotePrivateEndpointConnection()).withPrivateLinkServiceConnectionState( + new RemotePrivateLinkServiceConnectionState().withStatus(PrivateLinkConnectionStatus.APPROVED))), + com.azure.core.util.Context.NONE); } } ``` diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/pom.xml b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/pom.xml index d870e55d33704..95610a8314a9e 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/pom.xml +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/pom.xml @@ -1,3 +1,8 @@ + 4.0.0 @@ -9,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-notificationhubs - 1.0.0-beta.4 + 1.0.0-beta.5 jar Microsoft Azure SDK for NotificationHubs Management - This package contains Microsoft Azure SDK for NotificationHubs Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Azure NotificationHub client. Package tag package-2017-04. + This package contains Microsoft Azure SDK for NotificationHubs Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Microsoft Notification Hubs Resource Provider REST API. Package tag package-preview-2023-10. https://github.com/Azure/azure-sdk-for-java @@ -38,7 +43,9 @@ UTF-8 - true + 0 + 0 + true @@ -51,5 +58,53 @@ azure-core-management 1.12.0 + + com.azure + azure-core-test + 1.24.0 + test + + + com.azure + azure-identity + 1.11.4 + test + + + org.junit.jupiter + junit-jupiter-api + 5.9.3 + test + + + org.junit.jupiter + junit-jupiter-engine + 5.9.3 + test + + + org.mockito + mockito-core + 4.11.0 + test + + + net.bytebuddy + byte-buddy + 1.14.12 + test + + + net.bytebuddy + byte-buddy-agent + 1.14.12 + test + + + org.slf4j + slf4j-simple + 1.7.36 + test + diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/NotificationHubsManager.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/NotificationHubsManager.java index 165323bfae254..ff3ad7a45de42 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/NotificationHubsManager.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/NotificationHubsManager.java @@ -10,25 +10,29 @@ import com.azure.core.http.HttpPipelineBuilder; import com.azure.core.http.HttpPipelinePosition; import com.azure.core.http.policy.AddDatePolicy; +import com.azure.core.http.policy.AddHeadersFromContextPolicy; import com.azure.core.http.policy.HttpLogOptions; import com.azure.core.http.policy.HttpLoggingPolicy; import com.azure.core.http.policy.HttpPipelinePolicy; import com.azure.core.http.policy.HttpPolicyProviders; import com.azure.core.http.policy.RequestIdPolicy; +import com.azure.core.http.policy.RetryOptions; import com.azure.core.http.policy.RetryPolicy; import com.azure.core.http.policy.UserAgentPolicy; import com.azure.core.management.http.policy.ArmChallengeAuthenticationPolicy; import com.azure.core.management.profile.AzureProfile; import com.azure.core.util.Configuration; import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.notificationhubs.fluent.NotificationHubsManagementClient; +import com.azure.resourcemanager.notificationhubs.fluent.NotificationHubsRPClient; import com.azure.resourcemanager.notificationhubs.implementation.NamespacesImpl; import com.azure.resourcemanager.notificationhubs.implementation.NotificationHubsImpl; -import com.azure.resourcemanager.notificationhubs.implementation.NotificationHubsManagementClientBuilder; +import com.azure.resourcemanager.notificationhubs.implementation.NotificationHubsRPClientBuilder; import com.azure.resourcemanager.notificationhubs.implementation.OperationsImpl; +import com.azure.resourcemanager.notificationhubs.implementation.PrivateEndpointConnectionsImpl; import com.azure.resourcemanager.notificationhubs.models.Namespaces; import com.azure.resourcemanager.notificationhubs.models.NotificationHubs; import com.azure.resourcemanager.notificationhubs.models.Operations; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnections; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -36,31 +40,32 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to NotificationHubsManager. Azure NotificationHub client. */ +/** + * Entry point to NotificationHubsManager. + * Microsoft Notification Hubs Resource Provider REST API. + */ public final class NotificationHubsManager { - private Operations operations; + private NotificationHubs notificationHubs; private Namespaces namespaces; - private NotificationHubs notificationHubs; + private Operations operations; - private final NotificationHubsManagementClient clientObject; + private PrivateEndpointConnections privateEndpointConnections; + + private final NotificationHubsRPClient clientObject; private NotificationHubsManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new NotificationHubsManagementClientBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new NotificationHubsRPClientBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of NotificationHubs service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the NotificationHubs service API instance. @@ -71,24 +76,40 @@ public static NotificationHubsManager authenticate(TokenCredential credential, A return configure().authenticate(credential, profile); } + /** + * Creates an instance of NotificationHubs service API entry point. + * + * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. + * @param profile the Azure profile for client. + * @return the NotificationHubs service API instance. + */ + public static NotificationHubsManager authenticate(HttpPipeline httpPipeline, AzureProfile profile) { + Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); + Objects.requireNonNull(profile, "'profile' cannot be null."); + return new NotificationHubsManager(httpPipeline, profile, null); + } + /** * Gets a Configurable instance that can be used to create NotificationHubsManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new NotificationHubsManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { - private final ClientLogger logger = new ClientLogger(Configurable.class); + private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); private HttpClient httpClient; private HttpLogOptions httpLogOptions; private final List policies = new ArrayList<>(); private final List scopes = new ArrayList<>(); private RetryPolicy retryPolicy; + private RetryOptions retryOptions; private Duration defaultPollInterval; private Configurable() { @@ -149,6 +170,19 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { return this; } + /** + * Sets the retry options for the HTTP pipeline retry policy. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + * + * @param retryOptions the retry options for the HTTP pipeline retry policy. + * @return the configurable object itself. + */ + public Configurable withRetryOptions(RetryOptions retryOptions) { + this.retryOptions = Objects.requireNonNull(retryOptions, "'retryOptions' cannot be null."); + return this; + } + /** * Sets the default poll interval, used when service does not provide "Retry-After" header. * @@ -156,9 +190,11 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = Objects.requireNonNull(defaultPollInterval, "'retryPolicy' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { - throw logger.logExceptionAsError(new IllegalArgumentException("'httpPipeline' cannot be negative")); + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); } return this; } @@ -175,21 +211,12 @@ public NotificationHubsManager authenticate(TokenCredential credential, AzurePro Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.notificationhubs") - .append("/") - .append("1.0.0-beta.3"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.notificationhubs") + .append("/").append("1.0.0-beta.4"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -198,49 +225,50 @@ public NotificationHubsManager authenticate(TokenCredential credential, AzurePro scopes.add(profile.getEnvironment().getManagementEndpoint() + "/.default"); } if (retryPolicy == null) { - retryPolicy = new RetryPolicy("Retry-After", ChronoUnit.SECONDS); + if (retryOptions != null) { + retryPolicy = new RetryPolicy(retryOptions); + } else { + retryPolicy = new RetryPolicy("Retry-After", ChronoUnit.SECONDS); + } } List policies = new ArrayList<>(); policies.add(new UserAgentPolicy(userAgentBuilder.toString())); + policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new NotificationHubsManager(httpPipeline, profile, defaultPollInterval); } } - /** @return Resource collection API of Operations. */ - public Operations operations() { - if (this.operations == null) { - this.operations = new OperationsImpl(clientObject.getOperations(), this); + /** + * Gets the resource collection API of NotificationHubs. It manages NotificationHubResource, + * SharedAccessAuthorizationRuleResource. + * + * @return Resource collection API of NotificationHubs. + */ + public NotificationHubs notificationHubs() { + if (this.notificationHubs == null) { + this.notificationHubs = new NotificationHubsImpl(clientObject.getNotificationHubs(), this); } - return operations; + return notificationHubs; } - /** @return Resource collection API of Namespaces. */ + /** + * Gets the resource collection API of Namespaces. It manages NamespaceResource. + * + * @return Resource collection API of Namespaces. + */ public Namespaces namespaces() { if (this.namespaces == null) { this.namespaces = new NamespacesImpl(clientObject.getNamespaces(), this); @@ -248,19 +276,38 @@ public Namespaces namespaces() { return namespaces; } - /** @return Resource collection API of NotificationHubs. */ - public NotificationHubs notificationHubs() { - if (this.notificationHubs == null) { - this.notificationHubs = new NotificationHubsImpl(clientObject.getNotificationHubs(), this); + /** + * Gets the resource collection API of Operations. + * + * @return Resource collection API of Operations. + */ + public Operations operations() { + if (this.operations == null) { + this.operations = new OperationsImpl(clientObject.getOperations(), this); } - return notificationHubs; + return operations; + } + + /** + * Gets the resource collection API of PrivateEndpointConnections. + * + * @return Resource collection API of PrivateEndpointConnections. + */ + public PrivateEndpointConnections privateEndpointConnections() { + if (this.privateEndpointConnections == null) { + this.privateEndpointConnections + = new PrivateEndpointConnectionsImpl(clientObject.getPrivateEndpointConnections(), this); + } + return privateEndpointConnections; } /** - * @return Wrapped service client NotificationHubsManagementClient providing direct access to the underlying - * auto-generated API implementation, based on Azure REST API. + * Gets wrapped service client NotificationHubsRPClient providing direct access to the underlying auto-generated API + * implementation, based on Azure REST API. + * + * @return Wrapped service client NotificationHubsRPClient. */ - public NotificationHubsManagementClient serviceClient() { + public NotificationHubsRPClient serviceClient() { return this.clientObject; } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NamespacesClient.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NamespacesClient.java index 72ea0544d1c35..2c275f1c4cfdd 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NamespacesClient.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NamespacesClient.java @@ -13,371 +13,388 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner; import com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -import com.azure.resourcemanager.notificationhubs.models.NamespaceCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.NamespacePatchParameters; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; -/** An instance of this class provides access to all the operations defined in NamespacesClient. */ +/** + * An instance of this class provides access to all the operations defined in NamespacesClient. + */ public interface NamespacesClient { /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. + * + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - CheckAvailabilityResultInner checkAvailability(CheckAvailabilityParameters parameters); + Response checkAvailabilityWithResponse(CheckAvailabilityParameters parameters, + Context context); /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. - * @param context The context to associate with this operation. + * + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a CheckAvailability resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response checkAvailabilityWithResponse( - CheckAvailabilityParameters parameters, Context context); + CheckAvailabilityResultInner checkAvailability(CheckAvailabilityParameters parameters); /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - NamespaceResourceInner createOrUpdate( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters); + Response getByResourceGroupWithResponse(String resourceGroupName, String namespaceName, + Context context); /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. - * @param context The context to associate with this operation. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters, Context context); + NamespaceResourceInner getByResourceGroup(String resourceGroupName, String namespaceName); /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link SyncPoller} for polling of notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.SINGLE) - NamespaceResourceInner patch(String resourceGroupName, String namespaceName, NamespacePatchParameters parameters); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, NamespaceResourceInner> beginCreateOrUpdate(String resourceGroupName, + String namespaceName, NamespaceResourceInner parameters); /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link SyncPoller} for polling of notification Hubs Namespace Resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, NamespaceResourceInner> beginCreateOrUpdate(String resourceGroupName, + String namespaceName, NamespaceResourceInner parameters, Context context); + + /** + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response patchWithResponse( - String resourceGroupName, String namespaceName, NamespacePatchParameters parameters, Context context); + NamespaceResourceInner createOrUpdate(String resourceGroupName, String namespaceName, + NamespaceResourceInner parameters); /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName); + @ServiceMethod(returns = ReturnType.SINGLE) + NamespaceResourceInner createOrUpdate(String resourceGroupName, String namespaceName, + NamespaceResourceInner parameters, Context context); /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName, Context context); + @ServiceMethod(returns = ReturnType.SINGLE) + Response updateWithResponse(String resourceGroupName, String namespaceName, + NamespacePatchParameters parameters, Context context); /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String namespaceName); + NamespaceResourceInner update(String resourceGroupName, String namespaceName, NamespacePatchParameters parameters); /** * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String namespaceName, Context context); + Response deleteWithResponse(String resourceGroupName, String namespaceName, Context context); /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - NamespaceResourceInner getByResourceGroup(String resourceGroupName, String namespaceName); + void delete(String resourceGroupName, String namespaceName); /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. + * Lists all the available namespaces within the subscription. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String namespaceName, Context context); + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(); /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters); + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String skipToken, Integer top, Context context); /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. - * @param context The context to associate with this operation. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context); + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByResourceGroup(String resourceGroupName); /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName); + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByResourceGroup(String resourceGroupName, String skipToken, Integer top, + Context context); /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context); + Response createOrUpdateAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context); /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return response for POST requests that return single SharedAccessAuthorizationRule. */ @ServiceMethod(returns = ReturnType.SINGLE) - SharedAccessAuthorizationRuleResourceInner getAuthorizationRule( - String resourceGroupName, String namespaceName, String authorizationRuleName); + SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule(String resourceGroupName, + String namespaceName, String authorizationRuleName, SharedAccessAuthorizationRuleResourceInner parameters); /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context); + Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context); /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroup(String resourceGroupName); + @ServiceMethod(returns = ReturnType.SINGLE) + void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName); /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroup(String resourceGroupName, Context context); + @ServiceMethod(returns = ReturnType.SINGLE) + Response getAuthorizationRuleWithResponse(String resourceGroupName, + String namespaceName, String authorizationRuleName, Context context); /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(); + @ServiceMethod(returns = ReturnType.SINGLE) + SharedAccessAuthorizationRuleResourceInner getAuthorizationRule(String resourceGroupName, String namespaceName, + String authorizationRuleName); /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * - * @param context The context to associate with this operation. + * Gets the authorization rules for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the authorization rules for a namespace as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(Context context); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName); /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, Context context); /** - * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets the Primary and Secondary ConnectionStrings to the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the Primary and Secondary ConnectionStrings to the namespace along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, Context context); + @ServiceMethod(returns = ReturnType.SINGLE) + Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context); /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -387,55 +404,64 @@ PagedIterable listAuthorizationRules ResourceListKeysInner listKeys(String resourceGroupName, String namespaceName, String authorizationRuleName); /** - * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listKeysWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context); + Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, PolicyKeyResource parameters, Context context); /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). */ @ServiceMethod(returns = ReturnType.SINGLE) - ResourceListKeysInner regenerateKeys( - String resourceGroupName, String namespaceName, String authorizationRuleName, PolicykeyResource parameters); + ResourceListKeysInner regenerateKeys(String resourceGroupName, String namespaceName, String authorizationRuleName, + PolicyKeyResource parameters); /** - * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - PolicykeyResource parameters, + Response getPnsCredentialsWithResponse(String resourceGroupName, String namespaceName, Context context); + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + PnsCredentialsResourceInner getPnsCredentials(String resourceGroupName, String namespaceName); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsClient.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsClient.java index b439c9399d94b..f88f06ea4740c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsClient.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsClient.java @@ -16,493 +16,459 @@ import com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -import com.azure.resourcemanager.notificationhubs.models.NotificationHubCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.NotificationHubPatchParameters; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; -/** An instance of this class provides access to all the operations defined in NotificationHubsClient. */ +/** + * An instance of this class provides access to all the operations defined in NotificationHubsClient. + */ public interface NotificationHubsClient { /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - CheckAvailabilityResultInner checkNotificationHubAvailability( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters); + Response checkNotificationHubAvailabilityWithResponse(String resourceGroupName, + String namespaceName, CheckAvailabilityParameters parameters, Context context); /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a CheckAvailability resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response checkNotificationHubAvailabilityWithResponse( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters, Context context); + CheckAvailabilityResultInner checkNotificationHubAvailability(String resourceGroupName, String namespaceName, + CheckAvailabilityParameters parameters); /** - * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the notification hub along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context); + + /** + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the notification hub. */ @ServiceMethod(returns = ReturnType.SINGLE) - NotificationHubResourceInner createOrUpdate( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters); + NotificationHubResourceInner get(String resourceGroupName, String namespaceName, String notificationHubName); /** * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubResourceInner parameters, Context context); /** - * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Update a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - NotificationHubResourceInner patch(String resourceGroupName, String namespaceName, String notificationHubName); + NotificationHubResourceInner createOrUpdate(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubResourceInner parameters); /** * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to patch a NotificationHub Resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response patchWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubPatchParameters parameters, - Context context); + Response updateWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubPatchParameters parameters, Context context); /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Patch a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String namespaceName, String notificationHubName); + NotificationHubResourceInner update(String resourceGroupName, String namespaceName, String notificationHubName, + NotificationHubPatchParameters parameters); /** * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + Response deleteWithResponse(String resourceGroupName, String namespaceName, String notificationHubName, + Context context); /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - NotificationHubResourceInner get(String resourceGroupName, String namespaceName, String notificationHubName); + void delete(String resourceGroupName, String namespaceName, String notificationHubName); /** * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String namespaceName); /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - DebugSendResponseInner debugSend(String resourceGroupName, String namespaceName, String notificationHubName); + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String namespaceName, String skipToken, + Integer top, Context context); /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Debug send parameters. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return description of a NotificationHub Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response debugSendWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Object parameters, Context context); + Response debugSendWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context); /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters); + DebugSendResponseInner debugSend(String resourceGroupName, String namespaceName, String notificationHubName); /** * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response createOrUpdateAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context); + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context); /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Updates an authorization rule for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for POST requests that return single SharedAccessAuthorizationRule. */ @ServiceMethod(returns = ReturnType.SINGLE) - void deleteAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName); + SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters); /** * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context); + Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context); /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. */ @ServiceMethod(returns = ReturnType.SINGLE) - SharedAccessAuthorizationRuleResourceInner getAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName); + void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName); /** * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. + * @return an authorization rule for a NotificationHub by name along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context); - - /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String resourceGroupName, String namespaceName); + Response getAuthorizationRuleWithResponse(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, Context context); /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return an authorization rule for a NotificationHub by name. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String resourceGroupName, String namespaceName, Context context); + @ServiceMethod(returns = ReturnType.SINGLE) + SharedAccessAuthorizationRuleResourceInner getAuthorizationRule(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName); /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName); /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName, Context context); /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - ResourceListKeysInner listKeys( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName); + Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context); /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Primary and Secondary ConnectionStrings to the NotificationHub. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context); + ResourceListKeysInner listKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName); /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - ResourceListKeysInner regenerateKeys( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters); + Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters, Context context); /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). */ @ServiceMethod(returns = ReturnType.SINGLE) - Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context); + ResourceListKeysInner regenerateKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName, PolicyKeyResource parameters); /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - PnsCredentialsResourceInner getPnsCredentials( - String resourceGroupName, String namespaceName, String notificationHubName); + Response getPnsCredentialsWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context); /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param context The context to associate with this operation. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a NotificationHub PNS Credentials. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getPnsCredentialsWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + PnsCredentialsResourceInner getPnsCredentials(String resourceGroupName, String namespaceName, + String notificationHubName); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsManagementClient.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsRPClient.java similarity index 74% rename from sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsManagementClient.java rename to sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsRPClient.java index 29ade8278fa8e..776096c9a446e 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsManagementClient.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/NotificationHubsRPClient.java @@ -7,62 +7,70 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for NotificationHubsManagementClient class. */ -public interface NotificationHubsManagementClient { +/** + * The interface for NotificationHubsRPClient class. + */ +public interface NotificationHubsRPClient { /** - * Gets Gets subscription credentials which uniquely identify Microsoft Azure subscription. The subscription ID - * forms part of the URI for every service call. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** - * Gets the OperationsClient object to access its operations. - * - * @return the OperationsClient object. + * Gets the NotificationHubsClient object to access its operations. + * + * @return the NotificationHubsClient object. */ - OperationsClient getOperations(); + NotificationHubsClient getNotificationHubs(); /** * Gets the NamespacesClient object to access its operations. - * + * * @return the NamespacesClient object. */ NamespacesClient getNamespaces(); /** - * Gets the NotificationHubsClient object to access its operations. - * - * @return the NotificationHubsClient object. + * Gets the OperationsClient object to access its operations. + * + * @return the OperationsClient object. */ - NotificationHubsClient getNotificationHubs(); + OperationsClient getOperations(); + + /** + * Gets the PrivateEndpointConnectionsClient object to access its operations. + * + * @return the PrivateEndpointConnectionsClient object. + */ + PrivateEndpointConnectionsClient getPrivateEndpointConnections(); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/OperationsClient.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/OperationsClient.java index 321a39a96140a..a15807903b075 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/OperationsClient.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/OperationsClient.java @@ -10,26 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/PrivateEndpointConnectionsClient.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/PrivateEndpointConnectionsClient.java new file mode 100644 index 0000000000000..dbc6f4cd30a9f --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/PrivateEndpointConnectionsClient.java @@ -0,0 +1,306 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; + +/** + * An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. + */ +public interface PrivateEndpointConnectionsClient { + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of represents a Private Endpoint Connection ARM resource - a + * sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, PrivateEndpointConnectionResourceInner> beginUpdate( + String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + PrivateEndpointConnectionResourceInner parameters); + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of represents a Private Endpoint Connection ARM resource - a + * sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, PrivateEndpointConnectionResourceInner> beginUpdate( + String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + PrivateEndpointConnectionResourceInner parameters, Context context); + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + PrivateEndpointConnectionResourceInner update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters); + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + PrivateEndpointConnectionResourceInner update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters, Context context); + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName); + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context); + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName); + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, Context context); + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context); + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + PrivateEndpointConnectionResourceInner get(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName); + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String namespaceName); + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String namespaceName, + Context context); + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getGroupIdWithResponse(String resourceGroupName, String namespaceName, + String subResourceName, Context context); + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + PrivateLinkResourceInner getGroupId(String resourceGroupName, String namespaceName, String subResourceName); + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listGroupIds(String resourceGroupName, String namespaceName); + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listGroupIds(String resourceGroupName, String namespaceName, + Context context); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/AdmCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/AdmCredentialProperties.java index 7fe79588aac0b..137c2dcc2e5b7 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/AdmCredentialProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/AdmCredentialProperties.java @@ -6,35 +6,40 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub AdmCredential. */ +/** + * Description of a NotificationHub AdmCredential. + */ @Fluent public final class AdmCredentialProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(AdmCredentialProperties.class); - /* - * The client identifier. + * Gets or sets the client identifier. */ @JsonProperty(value = "clientId") private String clientId; /* - * The credential secret access key. + * Gets or sets the credential secret access key. */ @JsonProperty(value = "clientSecret") private String clientSecret; /* - * The URL of the authorization token. + * Gets or sets the URL of the authorization token. */ @JsonProperty(value = "authTokenUrl") private String authTokenUrl; /** - * Get the clientId property: The client identifier. - * + * Creates an instance of AdmCredentialProperties class. + */ + public AdmCredentialProperties() { + } + + /** + * Get the clientId property: Gets or sets the client identifier. + * * @return the clientId value. */ public String clientId() { @@ -42,8 +47,8 @@ public String clientId() { } /** - * Set the clientId property: The client identifier. - * + * Set the clientId property: Gets or sets the client identifier. + * * @param clientId the clientId value to set. * @return the AdmCredentialProperties object itself. */ @@ -53,8 +58,8 @@ public AdmCredentialProperties withClientId(String clientId) { } /** - * Get the clientSecret property: The credential secret access key. - * + * Get the clientSecret property: Gets or sets the credential secret access key. + * * @return the clientSecret value. */ public String clientSecret() { @@ -62,8 +67,8 @@ public String clientSecret() { } /** - * Set the clientSecret property: The credential secret access key. - * + * Set the clientSecret property: Gets or sets the credential secret access key. + * * @param clientSecret the clientSecret value to set. * @return the AdmCredentialProperties object itself. */ @@ -73,8 +78,8 @@ public AdmCredentialProperties withClientSecret(String clientSecret) { } /** - * Get the authTokenUrl property: The URL of the authorization token. - * + * Get the authTokenUrl property: Gets or sets the URL of the authorization token. + * * @return the authTokenUrl value. */ public String authTokenUrl() { @@ -82,8 +87,8 @@ public String authTokenUrl() { } /** - * Set the authTokenUrl property: The URL of the authorization token. - * + * Set the authTokenUrl property: Gets or sets the URL of the authorization token. + * * @param authTokenUrl the authTokenUrl value to set. * @return the AdmCredentialProperties object itself. */ @@ -94,9 +99,23 @@ public AdmCredentialProperties withAuthTokenUrl(String authTokenUrl) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (clientId() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property clientId in model AdmCredentialProperties")); + } + if (clientSecret() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property clientSecret in model AdmCredentialProperties")); + } + if (authTokenUrl() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property authTokenUrl in model AdmCredentialProperties")); + } } + + private static final ClientLogger LOGGER = new ClientLogger(AdmCredentialProperties.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ApnsCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ApnsCredentialProperties.java index 323255b06eaca..aa0fdd36120f1 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ApnsCredentialProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ApnsCredentialProperties.java @@ -6,81 +6,73 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; /** - * Description of a NotificationHub ApnsCredential. Note that there is no explicit switch between Certificate and Token - * Authentication Modes. The mode is determined based on the properties passed in. + * Description of a NotificationHub ApnsCredential. */ @Fluent public final class ApnsCredentialProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(ApnsCredentialProperties.class); - /* - * The APNS certificate. Specify if using Certificate Authentication Mode. + * Gets or sets the APNS certificate. */ @JsonProperty(value = "apnsCertificate") private String apnsCertificate; /* - * The APNS certificate password if it exists. + * Gets or sets the certificate key. */ @JsonProperty(value = "certificateKey") private String certificateKey; /* - * The APNS endpoint of this credential. If using Certificate - * Authentication Mode and Sandbox specify - * 'gateway.sandbox.push.apple.com'. If using Certificate Authentication - * Mode and Production specify 'gateway.push.apple.com'. If using Token - * Authentication Mode and Sandbox specify - * 'https://api.development.push.apple.com:443/3/device'. If using Token - * Authentication Mode and Production specify - * 'https://api.push.apple.com:443/3/device'. + * Gets or sets the endpoint of this credential. */ @JsonProperty(value = "endpoint") private String endpoint; /* - * The APNS certificate thumbprint. Specify if using Certificate - * Authentication Mode. + * Gets or sets the APNS certificate Thumbprint */ @JsonProperty(value = "thumbprint") private String thumbprint; /* - * A 10-character key identifier (kid) key, obtained from your developer - * account. Specify if using Token Authentication Mode. + * Gets or sets a 10-character key identifier (kid) key, obtained from + * your developer account */ @JsonProperty(value = "keyId") private String keyId; /* - * The name of the application or BundleId. Specify if using Token - * Authentication Mode. + * Gets or sets the name of the application */ @JsonProperty(value = "appName") private String appName; /* - * The issuer (iss) registered claim key. The value is a 10-character - * TeamId, obtained from your developer account. Specify if using Token - * Authentication Mode. + * Gets or sets the issuer (iss) registered claim key, whose value is + * your 10-character Team ID, obtained from your developer account */ @JsonProperty(value = "appId") private String appId; /* - * Provider Authentication Token, obtained through your developer account. - * Specify if using Token Authentication Mode. + * Gets or sets provider Authentication Token, obtained through your + * developer account */ @JsonProperty(value = "token") private String token; /** - * Get the apnsCertificate property: The APNS certificate. Specify if using Certificate Authentication Mode. - * + * Creates an instance of ApnsCredentialProperties class. + */ + public ApnsCredentialProperties() { + } + + /** + * Get the apnsCertificate property: Gets or sets the APNS certificate. + * * @return the apnsCertificate value. */ public String apnsCertificate() { @@ -88,8 +80,8 @@ public String apnsCertificate() { } /** - * Set the apnsCertificate property: The APNS certificate. Specify if using Certificate Authentication Mode. - * + * Set the apnsCertificate property: Gets or sets the APNS certificate. + * * @param apnsCertificate the apnsCertificate value to set. * @return the ApnsCredentialProperties object itself. */ @@ -99,8 +91,8 @@ public ApnsCredentialProperties withApnsCertificate(String apnsCertificate) { } /** - * Get the certificateKey property: The APNS certificate password if it exists. - * + * Get the certificateKey property: Gets or sets the certificate key. + * * @return the certificateKey value. */ public String certificateKey() { @@ -108,8 +100,8 @@ public String certificateKey() { } /** - * Set the certificateKey property: The APNS certificate password if it exists. - * + * Set the certificateKey property: Gets or sets the certificate key. + * * @param certificateKey the certificateKey value to set. * @return the ApnsCredentialProperties object itself. */ @@ -119,12 +111,8 @@ public ApnsCredentialProperties withCertificateKey(String certificateKey) { } /** - * Get the endpoint property: The APNS endpoint of this credential. If using Certificate Authentication Mode and - * Sandbox specify 'gateway.sandbox.push.apple.com'. If using Certificate Authentication Mode and Production specify - * 'gateway.push.apple.com'. If using Token Authentication Mode and Sandbox specify - * 'https://api.development.push.apple.com:443/3/device'. If using Token Authentication Mode and Production specify - * 'https://api.push.apple.com:443/3/device'. - * + * Get the endpoint property: Gets or sets the endpoint of this credential. + * * @return the endpoint value. */ public String endpoint() { @@ -132,12 +120,8 @@ public String endpoint() { } /** - * Set the endpoint property: The APNS endpoint of this credential. If using Certificate Authentication Mode and - * Sandbox specify 'gateway.sandbox.push.apple.com'. If using Certificate Authentication Mode and Production specify - * 'gateway.push.apple.com'. If using Token Authentication Mode and Sandbox specify - * 'https://api.development.push.apple.com:443/3/device'. If using Token Authentication Mode and Production specify - * 'https://api.push.apple.com:443/3/device'. - * + * Set the endpoint property: Gets or sets the endpoint of this credential. + * * @param endpoint the endpoint value to set. * @return the ApnsCredentialProperties object itself. */ @@ -147,8 +131,8 @@ public ApnsCredentialProperties withEndpoint(String endpoint) { } /** - * Get the thumbprint property: The APNS certificate thumbprint. Specify if using Certificate Authentication Mode. - * + * Get the thumbprint property: Gets or sets the APNS certificate Thumbprint. + * * @return the thumbprint value. */ public String thumbprint() { @@ -156,8 +140,8 @@ public String thumbprint() { } /** - * Set the thumbprint property: The APNS certificate thumbprint. Specify if using Certificate Authentication Mode. - * + * Set the thumbprint property: Gets or sets the APNS certificate Thumbprint. + * * @param thumbprint the thumbprint value to set. * @return the ApnsCredentialProperties object itself. */ @@ -167,9 +151,9 @@ public ApnsCredentialProperties withThumbprint(String thumbprint) { } /** - * Get the keyId property: A 10-character key identifier (kid) key, obtained from your developer account. Specify if - * using Token Authentication Mode. - * + * Get the keyId property: Gets or sets a 10-character key identifier (kid) key, obtained from + * your developer account. + * * @return the keyId value. */ public String keyId() { @@ -177,9 +161,9 @@ public String keyId() { } /** - * Set the keyId property: A 10-character key identifier (kid) key, obtained from your developer account. Specify if - * using Token Authentication Mode. - * + * Set the keyId property: Gets or sets a 10-character key identifier (kid) key, obtained from + * your developer account. + * * @param keyId the keyId value to set. * @return the ApnsCredentialProperties object itself. */ @@ -189,8 +173,8 @@ public ApnsCredentialProperties withKeyId(String keyId) { } /** - * Get the appName property: The name of the application or BundleId. Specify if using Token Authentication Mode. - * + * Get the appName property: Gets or sets the name of the application. + * * @return the appName value. */ public String appName() { @@ -198,8 +182,8 @@ public String appName() { } /** - * Set the appName property: The name of the application or BundleId. Specify if using Token Authentication Mode. - * + * Set the appName property: Gets or sets the name of the application. + * * @param appName the appName value to set. * @return the ApnsCredentialProperties object itself. */ @@ -209,9 +193,9 @@ public ApnsCredentialProperties withAppName(String appName) { } /** - * Get the appId property: The issuer (iss) registered claim key. The value is a 10-character TeamId, obtained from - * your developer account. Specify if using Token Authentication Mode. - * + * Get the appId property: Gets or sets the issuer (iss) registered claim key, whose value is + * your 10-character Team ID, obtained from your developer account. + * * @return the appId value. */ public String appId() { @@ -219,9 +203,9 @@ public String appId() { } /** - * Set the appId property: The issuer (iss) registered claim key. The value is a 10-character TeamId, obtained from - * your developer account. Specify if using Token Authentication Mode. - * + * Set the appId property: Gets or sets the issuer (iss) registered claim key, whose value is + * your 10-character Team ID, obtained from your developer account. + * * @param appId the appId value to set. * @return the ApnsCredentialProperties object itself. */ @@ -231,9 +215,9 @@ public ApnsCredentialProperties withAppId(String appId) { } /** - * Get the token property: Provider Authentication Token, obtained through your developer account. Specify if using - * Token Authentication Mode. - * + * Get the token property: Gets or sets provider Authentication Token, obtained through your + * developer account. + * * @return the token value. */ public String token() { @@ -241,9 +225,9 @@ public String token() { } /** - * Set the token property: Provider Authentication Token, obtained through your developer account. Specify if using - * Token Authentication Mode. - * + * Set the token property: Gets or sets provider Authentication Token, obtained through your + * developer account. + * * @param token the token value to set. * @return the ApnsCredentialProperties object itself. */ @@ -254,9 +238,15 @@ public ApnsCredentialProperties withToken(String token) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (endpoint() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property endpoint in model ApnsCredentialProperties")); + } } + + private static final ClientLogger LOGGER = new ClientLogger(ApnsCredentialProperties.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BaiduCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BaiduCredentialProperties.java index 2f24be391677c..85a3173744028 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BaiduCredentialProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BaiduCredentialProperties.java @@ -6,35 +6,40 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub BaiduCredential. */ +/** + * Description of a NotificationHub BaiduCredential. + */ @Fluent public final class BaiduCredentialProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(BaiduCredentialProperties.class); - /* - * Baidu Api Key. + * Gets or sets baidu Api Key. */ @JsonProperty(value = "baiduApiKey") private String baiduApiKey; /* - * Baidu Endpoint. + * Gets or sets baidu Endpoint. */ @JsonProperty(value = "baiduEndPoint") private String baiduEndPoint; /* - * Baidu Secret Key + * Gets or sets baidu Secret Key */ @JsonProperty(value = "baiduSecretKey") private String baiduSecretKey; /** - * Get the baiduApiKey property: Baidu Api Key. - * + * Creates an instance of BaiduCredentialProperties class. + */ + public BaiduCredentialProperties() { + } + + /** + * Get the baiduApiKey property: Gets or sets baidu Api Key. + * * @return the baiduApiKey value. */ public String baiduApiKey() { @@ -42,8 +47,8 @@ public String baiduApiKey() { } /** - * Set the baiduApiKey property: Baidu Api Key. - * + * Set the baiduApiKey property: Gets or sets baidu Api Key. + * * @param baiduApiKey the baiduApiKey value to set. * @return the BaiduCredentialProperties object itself. */ @@ -53,8 +58,8 @@ public BaiduCredentialProperties withBaiduApiKey(String baiduApiKey) { } /** - * Get the baiduEndPoint property: Baidu Endpoint. - * + * Get the baiduEndPoint property: Gets or sets baidu Endpoint. + * * @return the baiduEndPoint value. */ public String baiduEndPoint() { @@ -62,8 +67,8 @@ public String baiduEndPoint() { } /** - * Set the baiduEndPoint property: Baidu Endpoint. - * + * Set the baiduEndPoint property: Gets or sets baidu Endpoint. + * * @param baiduEndPoint the baiduEndPoint value to set. * @return the BaiduCredentialProperties object itself. */ @@ -73,8 +78,8 @@ public BaiduCredentialProperties withBaiduEndPoint(String baiduEndPoint) { } /** - * Get the baiduSecretKey property: Baidu Secret Key. - * + * Get the baiduSecretKey property: Gets or sets baidu Secret Key. + * * @return the baiduSecretKey value. */ public String baiduSecretKey() { @@ -82,8 +87,8 @@ public String baiduSecretKey() { } /** - * Set the baiduSecretKey property: Baidu Secret Key. - * + * Set the baiduSecretKey property: Gets or sets baidu Secret Key. + * * @param baiduSecretKey the baiduSecretKey value to set. * @return the BaiduCredentialProperties object itself. */ @@ -94,9 +99,23 @@ public BaiduCredentialProperties withBaiduSecretKey(String baiduSecretKey) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (baiduApiKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property baiduApiKey in model BaiduCredentialProperties")); + } + if (baiduEndPoint() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property baiduEndPoint in model BaiduCredentialProperties")); + } + if (baiduSecretKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property baiduSecretKey in model BaiduCredentialProperties")); + } } + + private static final ClientLogger LOGGER = new ClientLogger(BaiduCredentialProperties.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BrowserCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BrowserCredentialProperties.java new file mode 100644 index 0000000000000..612f2a8b2ee04 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/BrowserCredentialProperties.java @@ -0,0 +1,121 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Description of a NotificationHub BrowserCredential. + */ +@Fluent +public final class BrowserCredentialProperties { + /* + * Gets or sets web push subject. + */ + @JsonProperty(value = "subject") + private String subject; + + /* + * Gets or sets VAPID private key. + */ + @JsonProperty(value = "vapidPrivateKey") + private String vapidPrivateKey; + + /* + * Gets or sets VAPID public key. + */ + @JsonProperty(value = "vapidPublicKey") + private String vapidPublicKey; + + /** + * Creates an instance of BrowserCredentialProperties class. + */ + public BrowserCredentialProperties() { + } + + /** + * Get the subject property: Gets or sets web push subject. + * + * @return the subject value. + */ + public String subject() { + return this.subject; + } + + /** + * Set the subject property: Gets or sets web push subject. + * + * @param subject the subject value to set. + * @return the BrowserCredentialProperties object itself. + */ + public BrowserCredentialProperties withSubject(String subject) { + this.subject = subject; + return this; + } + + /** + * Get the vapidPrivateKey property: Gets or sets VAPID private key. + * + * @return the vapidPrivateKey value. + */ + public String vapidPrivateKey() { + return this.vapidPrivateKey; + } + + /** + * Set the vapidPrivateKey property: Gets or sets VAPID private key. + * + * @param vapidPrivateKey the vapidPrivateKey value to set. + * @return the BrowserCredentialProperties object itself. + */ + public BrowserCredentialProperties withVapidPrivateKey(String vapidPrivateKey) { + this.vapidPrivateKey = vapidPrivateKey; + return this; + } + + /** + * Get the vapidPublicKey property: Gets or sets VAPID public key. + * + * @return the vapidPublicKey value. + */ + public String vapidPublicKey() { + return this.vapidPublicKey; + } + + /** + * Set the vapidPublicKey property: Gets or sets VAPID public key. + * + * @param vapidPublicKey the vapidPublicKey value to set. + * @return the BrowserCredentialProperties object itself. + */ + public BrowserCredentialProperties withVapidPublicKey(String vapidPublicKey) { + this.vapidPublicKey = vapidPublicKey; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (subject() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property subject in model BrowserCredentialProperties")); + } + if (vapidPrivateKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property vapidPrivateKey in model BrowserCredentialProperties")); + } + if (vapidPublicKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property vapidPublicKey in model BrowserCredentialProperties")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(BrowserCredentialProperties.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/CheckAvailabilityResultInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/CheckAvailabilityResultInner.java index 01a9fc91faaf8..a3839b8f6b9f3 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/CheckAvailabilityResultInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/CheckAvailabilityResultInner.java @@ -5,35 +5,60 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.models.Sku; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Description of a CheckAvailability resource. */ +/** + * Description of a CheckAvailability resource. + */ @Fluent -public final class CheckAvailabilityResultInner extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(CheckAvailabilityResultInner.class); - +public final class CheckAvailabilityResultInner extends ProxyResource { /* - * True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. + * Gets or sets true if the name is available and can be used to + * create new Namespace/NotificationHub. Otherwise false. */ @JsonProperty(value = "isAvailiable") private Boolean isAvailiable; /* - * The sku of the created namespace + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "location") + private String location; + + /* + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; + + /* + * The Sku description for a namespace */ @JsonProperty(value = "sku") private Sku sku; + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of CheckAvailabilityResultInner class. + */ + public CheckAvailabilityResultInner() { + } + /** - * Get the isAvailiable property: True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. - * + * Get the isAvailiable property: Gets or sets true if the name is available and can be used to + * create new Namespace/NotificationHub. Otherwise false. + * * @return the isAvailiable value. */ public Boolean isAvailiable() { @@ -41,9 +66,9 @@ public Boolean isAvailiable() { } /** - * Set the isAvailiable property: True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. - * + * Set the isAvailiable property: Gets or sets true if the name is available and can be used to + * create new Namespace/NotificationHub. Otherwise false. + * * @param isAvailiable the isAvailiable value to set. * @return the CheckAvailabilityResultInner object itself. */ @@ -53,8 +78,48 @@ public CheckAvailabilityResultInner withIsAvailiable(Boolean isAvailiable) { } /** - * Get the sku property: The sku of the created namespace. - * + * Get the location property: Deprecated - only for compatibility. + * + * @return the location value. + */ + public String location() { + return this.location; + } + + /** + * Set the location property: Deprecated - only for compatibility. + * + * @param location the location value to set. + * @return the CheckAvailabilityResultInner object itself. + */ + public CheckAvailabilityResultInner withLocation(String location) { + this.location = location; + return this; + } + + /** + * Get the tags property: Deprecated - only for compatibility. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; + } + + /** + * Set the tags property: Deprecated - only for compatibility. + * + * @param tags the tags value to set. + * @return the CheckAvailabilityResultInner object itself. + */ + public CheckAvailabilityResultInner withTags(Map tags) { + this.tags = tags; + return this; + } + + /** + * Get the sku property: The Sku description for a namespace. + * * @return the sku value. */ public Sku sku() { @@ -62,8 +127,8 @@ public Sku sku() { } /** - * Set the sku property: The sku of the created namespace. - * + * Set the sku property: The Sku description for a namespace. + * * @param sku the sku value to set. * @return the CheckAvailabilityResultInner object itself. */ @@ -72,23 +137,18 @@ public CheckAvailabilityResultInner withSku(Sku sku) { return this; } - /** {@inheritDoc} */ - @Override - public CheckAvailabilityResultInner withLocation(String location) { - super.withLocation(location); - return this; - } - - /** {@inheritDoc} */ - @Override - public CheckAvailabilityResultInner withTags(Map tags) { - super.withTags(tags); - return this; + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResponseInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResponseInner.java index 891d9bb7e8d8c..e6bca3a07a338 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResponseInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResponseInner.java @@ -5,33 +5,53 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.notificationhubs.models.Sku; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.models.RegistrationResult; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; import java.util.Map; -/** Description of a NotificationHub Resource. */ +/** + * Description of a NotificationHub Resource. + */ @Fluent -public final class DebugSendResponseInner extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(DebugSendResponseInner.class); - +public final class DebugSendResponseInner extends ProxyResource { /* - * Properties of the NotificationHub. + * Result of DebugSend operations. */ @JsonProperty(value = "properties") private DebugSendResult innerProperties; /* - * The sku of the created namespace + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "location") + private String location; + + /* + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of DebugSendResponseInner class. */ - @JsonProperty(value = "sku") - private Sku sku; + public DebugSendResponseInner() { + } /** - * Get the innerProperties property: Properties of the NotificationHub. - * + * Get the innerProperties property: Result of DebugSend operations. + * * @return the innerProperties value. */ private DebugSendResult innerProperties() { @@ -39,119 +59,89 @@ private DebugSendResult innerProperties() { } /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. + * Get the location property: Deprecated - only for compatibility. + * + * @return the location value. */ - public Sku sku() { - return this.sku; + public String location() { + return this.location; } /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. + * Set the location property: Deprecated - only for compatibility. + * + * @param location the location value to set. * @return the DebugSendResponseInner object itself. */ - public DebugSendResponseInner withSku(Sku sku) { - this.sku = sku; - return this; - } - - /** {@inheritDoc} */ - @Override public DebugSendResponseInner withLocation(String location) { - super.withLocation(location); - return this; - } - - /** {@inheritDoc} */ - @Override - public DebugSendResponseInner withTags(Map tags) { - super.withTags(tags); + this.location = location; return this; } /** - * Get the success property: successful send. - * - * @return the success value. + * Get the tags property: Deprecated - only for compatibility. + * + * @return the tags value. */ - public Float success() { - return this.innerProperties() == null ? null : this.innerProperties().success(); + public Map tags() { + return this.tags; } /** - * Set the success property: successful send. - * - * @param success the success value to set. + * Set the tags property: Deprecated - only for compatibility. + * + * @param tags the tags value to set. * @return the DebugSendResponseInner object itself. */ - public DebugSendResponseInner withSuccess(Float success) { - if (this.innerProperties() == null) { - this.innerProperties = new DebugSendResult(); - } - this.innerProperties().withSuccess(success); + public DebugSendResponseInner withTags(Map tags) { + this.tags = tags; return this; } /** - * Get the failure property: send failure. - * - * @return the failure value. + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - public Float failure() { - return this.innerProperties() == null ? null : this.innerProperties().failure(); + public SystemData systemData() { + return this.systemData; } /** - * Set the failure property: send failure. - * - * @param failure the failure value to set. - * @return the DebugSendResponseInner object itself. + * Get the success property: Gets or sets successful send. + * + * @return the success value. */ - public DebugSendResponseInner withFailure(Float failure) { - if (this.innerProperties() == null) { - this.innerProperties = new DebugSendResult(); - } - this.innerProperties().withFailure(failure); - return this; + public Long success() { + return this.innerProperties() == null ? null : this.innerProperties().success(); } /** - * Get the results property: actual failure description. - * - * @return the results value. + * Get the failure property: Gets or sets send failure. + * + * @return the failure value. */ - public Object results() { - return this.innerProperties() == null ? null : this.innerProperties().results(); + public Long failure() { + return this.innerProperties() == null ? null : this.innerProperties().failure(); } /** - * Set the results property: actual failure description. - * - * @param results the results value to set. - * @return the DebugSendResponseInner object itself. + * Get the results property: Gets or sets actual failure description. + * + * @return the results value. */ - public DebugSendResponseInner withResults(Object results) { - if (this.innerProperties() == null) { - this.innerProperties = new DebugSendResult(); - } - this.innerProperties().withResults(results); - return this; + public List results() { + return this.innerProperties() == null ? null : this.innerProperties().results(); } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() != null) { innerProperties().validate(); } - if (sku() != null) { - sku().validate(); - } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResult.java index a9c882cb31d9b..0fea74d1c98dd 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResult.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/DebugSendResult.java @@ -4,99 +4,75 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.core.annotation.Immutable; +import com.azure.resourcemanager.notificationhubs.models.RegistrationResult; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; -/** The DebugSendResult model. */ -@Fluent +/** + * Result of DebugSend operations. + */ +@Immutable public final class DebugSendResult { - @JsonIgnore private final ClientLogger logger = new ClientLogger(DebugSendResult.class); - /* - * successful send + * Gets or sets successful send */ - @JsonProperty(value = "success") - private Float success; + @JsonProperty(value = "success", access = JsonProperty.Access.WRITE_ONLY) + private Long success; /* - * send failure + * Gets or sets send failure */ - @JsonProperty(value = "failure") - private Float failure; + @JsonProperty(value = "failure", access = JsonProperty.Access.WRITE_ONLY) + private Long failure; /* - * actual failure description + * Gets or sets actual failure description */ - @JsonProperty(value = "results") - private Object results; + @JsonProperty(value = "results", access = JsonProperty.Access.WRITE_ONLY) + private List results; /** - * Get the success property: successful send. - * - * @return the success value. + * Creates an instance of DebugSendResult class. */ - public Float success() { - return this.success; + public DebugSendResult() { } /** - * Set the success property: successful send. - * - * @param success the success value to set. - * @return the DebugSendResult object itself. + * Get the success property: Gets or sets successful send. + * + * @return the success value. */ - public DebugSendResult withSuccess(Float success) { - this.success = success; - return this; + public Long success() { + return this.success; } /** - * Get the failure property: send failure. - * + * Get the failure property: Gets or sets send failure. + * * @return the failure value. */ - public Float failure() { + public Long failure() { return this.failure; } /** - * Set the failure property: send failure. - * - * @param failure the failure value to set. - * @return the DebugSendResult object itself. - */ - public DebugSendResult withFailure(Float failure) { - this.failure = failure; - return this; - } - - /** - * Get the results property: actual failure description. - * + * Get the results property: Gets or sets actual failure description. + * * @return the results value. */ - public Object results() { + public List results() { return this.results; } - /** - * Set the results property: actual failure description. - * - * @param results the results value to set. - * @return the DebugSendResult object itself. - */ - public DebugSendResult withResults(Object results) { - this.results = results; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (results() != null) { + results().forEach(e -> e.validate()); + } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/FcmV1CredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/FcmV1CredentialProperties.java new file mode 100644 index 0000000000000..1983cf6c83e7e --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/FcmV1CredentialProperties.java @@ -0,0 +1,121 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Description of a NotificationHub FcmV1Credential. + */ +@Fluent +public final class FcmV1CredentialProperties { + /* + * Gets or sets client email. + */ + @JsonProperty(value = "clientEmail") + private String clientEmail; + + /* + * Gets or sets private key. + */ + @JsonProperty(value = "privateKey") + private String privateKey; + + /* + * Gets or sets project id. + */ + @JsonProperty(value = "projectId") + private String projectId; + + /** + * Creates an instance of FcmV1CredentialProperties class. + */ + public FcmV1CredentialProperties() { + } + + /** + * Get the clientEmail property: Gets or sets client email. + * + * @return the clientEmail value. + */ + public String clientEmail() { + return this.clientEmail; + } + + /** + * Set the clientEmail property: Gets or sets client email. + * + * @param clientEmail the clientEmail value to set. + * @return the FcmV1CredentialProperties object itself. + */ + public FcmV1CredentialProperties withClientEmail(String clientEmail) { + this.clientEmail = clientEmail; + return this; + } + + /** + * Get the privateKey property: Gets or sets private key. + * + * @return the privateKey value. + */ + public String privateKey() { + return this.privateKey; + } + + /** + * Set the privateKey property: Gets or sets private key. + * + * @param privateKey the privateKey value to set. + * @return the FcmV1CredentialProperties object itself. + */ + public FcmV1CredentialProperties withPrivateKey(String privateKey) { + this.privateKey = privateKey; + return this; + } + + /** + * Get the projectId property: Gets or sets project id. + * + * @return the projectId value. + */ + public String projectId() { + return this.projectId; + } + + /** + * Set the projectId property: Gets or sets project id. + * + * @param projectId the projectId value to set. + * @return the FcmV1CredentialProperties object itself. + */ + public FcmV1CredentialProperties withProjectId(String projectId) { + this.projectId = projectId; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (clientEmail() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property clientEmail in model FcmV1CredentialProperties")); + } + if (privateKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property privateKey in model FcmV1CredentialProperties")); + } + if (projectId() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property projectId in model FcmV1CredentialProperties")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(FcmV1CredentialProperties.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/GcmCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/GcmCredentialProperties.java index 545a4338761b7..95a30786bc12c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/GcmCredentialProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/GcmCredentialProperties.java @@ -6,30 +6,34 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub GcmCredential. */ +/** + * Description of a NotificationHub GcmCredential. + */ @Fluent public final class GcmCredentialProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(GcmCredentialProperties.class); - /* - * The FCM legacy endpoint. Default value is - * 'https://fcm.googleapis.com/fcm/send' + * Gets or sets the GCM endpoint. */ @JsonProperty(value = "gcmEndpoint") private String gcmEndpoint; /* - * The Google API key. + * Gets or sets the Google API key. */ @JsonProperty(value = "googleApiKey") private String googleApiKey; /** - * Get the gcmEndpoint property: The FCM legacy endpoint. Default value is 'https://fcm.googleapis.com/fcm/send'. - * + * Creates an instance of GcmCredentialProperties class. + */ + public GcmCredentialProperties() { + } + + /** + * Get the gcmEndpoint property: Gets or sets the GCM endpoint. + * * @return the gcmEndpoint value. */ public String gcmEndpoint() { @@ -37,8 +41,8 @@ public String gcmEndpoint() { } /** - * Set the gcmEndpoint property: The FCM legacy endpoint. Default value is 'https://fcm.googleapis.com/fcm/send'. - * + * Set the gcmEndpoint property: Gets or sets the GCM endpoint. + * * @param gcmEndpoint the gcmEndpoint value to set. * @return the GcmCredentialProperties object itself. */ @@ -48,8 +52,8 @@ public GcmCredentialProperties withGcmEndpoint(String gcmEndpoint) { } /** - * Get the googleApiKey property: The Google API key. - * + * Get the googleApiKey property: Gets or sets the Google API key. + * * @return the googleApiKey value. */ public String googleApiKey() { @@ -57,8 +61,8 @@ public String googleApiKey() { } /** - * Set the googleApiKey property: The Google API key. - * + * Set the googleApiKey property: Gets or sets the Google API key. + * * @param googleApiKey the googleApiKey value to set. * @return the GcmCredentialProperties object itself. */ @@ -69,9 +73,15 @@ public GcmCredentialProperties withGoogleApiKey(String googleApiKey) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (googleApiKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property googleApiKey in model GcmCredentialProperties")); + } } + + private static final ClientLogger LOGGER = new ClientLogger(GcmCredentialProperties.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/MpnsCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/MpnsCredentialProperties.java index 7caace3ebf7f8..a89683150154b 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/MpnsCredentialProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/MpnsCredentialProperties.java @@ -6,35 +6,40 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub MpnsCredential. */ +/** + * Description of a NotificationHub MpnsCredential. + */ @Fluent public final class MpnsCredentialProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(MpnsCredentialProperties.class); - /* - * The MPNS certificate. + * Gets or sets the MPNS certificate. */ @JsonProperty(value = "mpnsCertificate") private String mpnsCertificate; /* - * The certificate key for this credential. + * Gets or sets the certificate key for this credential. */ @JsonProperty(value = "certificateKey") private String certificateKey; /* - * The MPNS certificate Thumbprint + * Gets or sets the MPNS certificate Thumbprint */ @JsonProperty(value = "thumbprint") private String thumbprint; /** - * Get the mpnsCertificate property: The MPNS certificate. - * + * Creates an instance of MpnsCredentialProperties class. + */ + public MpnsCredentialProperties() { + } + + /** + * Get the mpnsCertificate property: Gets or sets the MPNS certificate. + * * @return the mpnsCertificate value. */ public String mpnsCertificate() { @@ -42,8 +47,8 @@ public String mpnsCertificate() { } /** - * Set the mpnsCertificate property: The MPNS certificate. - * + * Set the mpnsCertificate property: Gets or sets the MPNS certificate. + * * @param mpnsCertificate the mpnsCertificate value to set. * @return the MpnsCredentialProperties object itself. */ @@ -53,8 +58,8 @@ public MpnsCredentialProperties withMpnsCertificate(String mpnsCertificate) { } /** - * Get the certificateKey property: The certificate key for this credential. - * + * Get the certificateKey property: Gets or sets the certificate key for this credential. + * * @return the certificateKey value. */ public String certificateKey() { @@ -62,8 +67,8 @@ public String certificateKey() { } /** - * Set the certificateKey property: The certificate key for this credential. - * + * Set the certificateKey property: Gets or sets the certificate key for this credential. + * * @param certificateKey the certificateKey value to set. * @return the MpnsCredentialProperties object itself. */ @@ -73,8 +78,8 @@ public MpnsCredentialProperties withCertificateKey(String certificateKey) { } /** - * Get the thumbprint property: The MPNS certificate Thumbprint. - * + * Get the thumbprint property: Gets or sets the MPNS certificate Thumbprint. + * * @return the thumbprint value. */ public String thumbprint() { @@ -82,8 +87,8 @@ public String thumbprint() { } /** - * Set the thumbprint property: The MPNS certificate Thumbprint. - * + * Set the thumbprint property: Gets or sets the MPNS certificate Thumbprint. + * * @param thumbprint the thumbprint value to set. * @return the MpnsCredentialProperties object itself. */ @@ -94,9 +99,23 @@ public MpnsCredentialProperties withThumbprint(String thumbprint) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (mpnsCertificate() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property mpnsCertificate in model MpnsCredentialProperties")); + } + if (certificateKey() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property certificateKey in model MpnsCredentialProperties")); + } + if (thumbprint() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property thumbprint in model MpnsCredentialProperties")); + } } + + private static final ClientLogger LOGGER = new ClientLogger(MpnsCredentialProperties.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceProperties.java index 98bb7a5132168..707e51b2d5f85 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceProperties.java @@ -5,224 +5,260 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.notificationhubs.models.NamespaceStatus; import com.azure.resourcemanager.notificationhubs.models.NamespaceType; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.models.NetworkAcls; +import com.azure.resourcemanager.notificationhubs.models.OperationProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PublicNetworkAccess; +import com.azure.resourcemanager.notificationhubs.models.ReplicationRegion; +import com.azure.resourcemanager.notificationhubs.models.ZoneRedundancyPreference; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; +import java.util.List; -/** Namespace properties. */ +/** + * Represents namespace properties. + */ @Fluent public final class NamespaceProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NamespaceProperties.class); - /* - * The name of the namespace. + * Name of the Notification Hubs namespace. This is immutable property, set automatically + * by the service when the namespace is created. */ - @JsonProperty(value = "name") + @JsonProperty(value = "name", access = JsonProperty.Access.WRITE_ONLY) private String name; /* - * Provisioning state of the Namespace. + * Defines values for OperationProvisioningState. */ @JsonProperty(value = "provisioningState") - private String provisioningState; + private OperationProvisioningState provisioningState; + + /* + * Namespace status. + */ + @JsonProperty(value = "status") + private NamespaceStatus status; + + /* + * Gets or sets whether or not the namespace is currently enabled. + */ + @JsonProperty(value = "enabled", access = JsonProperty.Access.WRITE_ONLY) + private Boolean enabled; /* - * Specifies the targeted region in which the namespace should be created. - * It can be any of the following values: Australia East, Australia - * Southeast, Central US, East US, East US 2, West US, North Central US, - * South Central US, East Asia, Southeast Asia, Brazil South, Japan East, - * Japan West, North Europe, West Europe + * Gets or sets whether or not the namespace is set as Critical. */ - @JsonProperty(value = "region") + @JsonProperty(value = "critical", access = JsonProperty.Access.WRITE_ONLY) + private Boolean critical; + + /* + * Namespace subscription id. + */ + @JsonProperty(value = "subscriptionId", access = JsonProperty.Access.WRITE_ONLY) + private String subscriptionId; + + /* + * Region. The value is always set to the same value as Namespace.Location, so we are deprecating + * this property. + */ + @JsonProperty(value = "region", access = JsonProperty.Access.WRITE_ONLY) private String region; /* - * Identifier for Azure Insights metrics + * Azure Insights Metrics id. */ @JsonProperty(value = "metricId", access = JsonProperty.Access.WRITE_ONLY) private String metricId; /* - * Status of the namespace. It can be any of these values:1 = - * Created/Active2 = Creating3 = Suspended4 = Deleting + * Time when the namespace was created. */ - @JsonProperty(value = "status") - private String status; + @JsonProperty(value = "createdAt", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime createdAt; /* - * The time the namespace was created. + * Time when the namespace was updated. */ - @JsonProperty(value = "createdAt") - private OffsetDateTime createdAt; + @JsonProperty(value = "updatedAt", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime updatedAt; /* - * The time the namespace was updated. + * Defines values for NamespaceType. */ - @JsonProperty(value = "updatedAt") - private OffsetDateTime updatedAt; + @JsonProperty(value = "namespaceType") + private NamespaceType namespaceType; /* - * Endpoint you can use to perform NotificationHub operations. + * Allowed replication region */ - @JsonProperty(value = "serviceBusEndpoint") - private String serviceBusEndpoint; + @JsonProperty(value = "replicationRegion") + private ReplicationRegion replicationRegion; /* - * The Id of the Azure subscription associated with the namespace. + * Namespace SKU name. */ - @JsonProperty(value = "subscriptionId") - private String subscriptionId; + @JsonProperty(value = "zoneRedundancy") + private ZoneRedundancyPreference zoneRedundancy; /* - * ScaleUnit where the namespace gets created + * A collection of network authorization rules. */ - @JsonProperty(value = "scaleUnit") - private String scaleUnit; + @JsonProperty(value = "networkAcls") + private NetworkAcls networkAcls; /* - * Whether or not the namespace is currently enabled. + * Collection of Notification Hub or Notification Hub Namespace PNS credentials. */ - @JsonProperty(value = "enabled") - private Boolean enabled; + @JsonProperty(value = "pnsCredentials") + private PnsCredentials pnsCredentials; /* - * Whether or not the namespace is set as Critical. + * Gets or sets endpoint you can use to perform NotificationHub + * operations. */ - @JsonProperty(value = "critical") - private Boolean critical; + @JsonProperty(value = "serviceBusEndpoint", access = JsonProperty.Access.WRITE_ONLY) + private String serviceBusEndpoint; + + /* + * Private Endpoint Connections for namespace + */ + @JsonProperty(value = "privateEndpointConnections", access = JsonProperty.Access.WRITE_ONLY) + private List privateEndpointConnections; + + /* + * Gets or sets scaleUnit where the namespace gets created + */ + @JsonProperty(value = "scaleUnit") + private String scaleUnit; /* - * Data center for the namespace + * Deprecated. */ @JsonProperty(value = "dataCenter") private String dataCenter; /* - * The namespace type. + * Type of public network access. */ - @JsonProperty(value = "namespaceType") - private NamespaceType namespaceType; + @JsonProperty(value = "publicNetworkAccess") + private PublicNetworkAccess publicNetworkAccess; /** - * Get the name property: The name of the namespace. - * - * @return the name value. + * Creates an instance of NamespaceProperties class. */ - public String name() { - return this.name; + public NamespaceProperties() { } /** - * Set the name property: The name of the namespace. - * - * @param name the name value to set. - * @return the NamespaceProperties object itself. + * Get the name property: Name of the Notification Hubs namespace. This is immutable property, set automatically + * by the service when the namespace is created. + * + * @return the name value. */ - public NamespaceProperties withName(String name) { - this.name = name; - return this; + public String name() { + return this.name; } /** - * Get the provisioningState property: Provisioning state of the Namespace. - * + * Get the provisioningState property: Defines values for OperationProvisioningState. + * * @return the provisioningState value. */ - public String provisioningState() { + public OperationProvisioningState provisioningState() { return this.provisioningState; } /** - * Set the provisioningState property: Provisioning state of the Namespace. - * + * Set the provisioningState property: Defines values for OperationProvisioningState. + * * @param provisioningState the provisioningState value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withProvisioningState(String provisioningState) { + public NamespaceProperties withProvisioningState(OperationProvisioningState provisioningState) { this.provisioningState = provisioningState; return this; } /** - * Get the region property: Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North Central - * US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West Europe. - * - * @return the region value. + * Get the status property: Namespace status. + * + * @return the status value. */ - public String region() { - return this.region; + public NamespaceStatus status() { + return this.status; } /** - * Set the region property: Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North Central - * US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West Europe. - * - * @param region the region value to set. + * Set the status property: Namespace status. + * + * @param status the status value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withRegion(String region) { - this.region = region; + public NamespaceProperties withStatus(NamespaceStatus status) { + this.status = status; return this; } /** - * Get the metricId property: Identifier for Azure Insights metrics. - * - * @return the metricId value. + * Get the enabled property: Gets or sets whether or not the namespace is currently enabled. + * + * @return the enabled value. */ - public String metricId() { - return this.metricId; + public Boolean enabled() { + return this.enabled; } /** - * Get the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. - * - * @return the status value. + * Get the critical property: Gets or sets whether or not the namespace is set as Critical. + * + * @return the critical value. */ - public String status() { - return this.status; + public Boolean critical() { + return this.critical; } /** - * Set the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. - * - * @param status the status value to set. - * @return the NamespaceProperties object itself. + * Get the subscriptionId property: Namespace subscription id. + * + * @return the subscriptionId value. */ - public NamespaceProperties withStatus(String status) { - this.status = status; - return this; + public String subscriptionId() { + return this.subscriptionId; } /** - * Get the createdAt property: The time the namespace was created. - * - * @return the createdAt value. + * Get the region property: Region. The value is always set to the same value as Namespace.Location, so we are + * deprecating + * this property. + * + * @return the region value. */ - public OffsetDateTime createdAt() { - return this.createdAt; + public String region() { + return this.region; } /** - * Set the createdAt property: The time the namespace was created. - * - * @param createdAt the createdAt value to set. - * @return the NamespaceProperties object itself. + * Get the metricId property: Azure Insights Metrics id. + * + * @return the metricId value. */ - public NamespaceProperties withCreatedAt(OffsetDateTime createdAt) { - this.createdAt = createdAt; - return this; + public String metricId() { + return this.metricId; + } + + /** + * Get the createdAt property: Time when the namespace was created. + * + * @return the createdAt value. + */ + public OffsetDateTime createdAt() { + return this.createdAt; } /** - * Get the updatedAt property: The time the namespace was updated. - * + * Get the updatedAt property: Time when the namespace was updated. + * * @return the updatedAt value. */ public OffsetDateTime updatedAt() { @@ -230,119 +266,147 @@ public OffsetDateTime updatedAt() { } /** - * Set the updatedAt property: The time the namespace was updated. - * - * @param updatedAt the updatedAt value to set. + * Get the namespaceType property: Defines values for NamespaceType. + * + * @return the namespaceType value. + */ + public NamespaceType namespaceType() { + return this.namespaceType; + } + + /** + * Set the namespaceType property: Defines values for NamespaceType. + * + * @param namespaceType the namespaceType value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withUpdatedAt(OffsetDateTime updatedAt) { - this.updatedAt = updatedAt; + public NamespaceProperties withNamespaceType(NamespaceType namespaceType) { + this.namespaceType = namespaceType; return this; } /** - * Get the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @return the serviceBusEndpoint value. + * Get the replicationRegion property: Allowed replication region. + * + * @return the replicationRegion value. */ - public String serviceBusEndpoint() { - return this.serviceBusEndpoint; + public ReplicationRegion replicationRegion() { + return this.replicationRegion; } /** - * Set the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @param serviceBusEndpoint the serviceBusEndpoint value to set. + * Set the replicationRegion property: Allowed replication region. + * + * @param replicationRegion the replicationRegion value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withServiceBusEndpoint(String serviceBusEndpoint) { - this.serviceBusEndpoint = serviceBusEndpoint; + public NamespaceProperties withReplicationRegion(ReplicationRegion replicationRegion) { + this.replicationRegion = replicationRegion; return this; } /** - * Get the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @return the subscriptionId value. + * Get the zoneRedundancy property: Namespace SKU name. + * + * @return the zoneRedundancy value. */ - public String subscriptionId() { - return this.subscriptionId; + public ZoneRedundancyPreference zoneRedundancy() { + return this.zoneRedundancy; } /** - * Set the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @param subscriptionId the subscriptionId value to set. + * Set the zoneRedundancy property: Namespace SKU name. + * + * @param zoneRedundancy the zoneRedundancy value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withSubscriptionId(String subscriptionId) { - this.subscriptionId = subscriptionId; + public NamespaceProperties withZoneRedundancy(ZoneRedundancyPreference zoneRedundancy) { + this.zoneRedundancy = zoneRedundancy; return this; } /** - * Get the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @return the scaleUnit value. + * Get the networkAcls property: A collection of network authorization rules. + * + * @return the networkAcls value. */ - public String scaleUnit() { - return this.scaleUnit; + public NetworkAcls networkAcls() { + return this.networkAcls; } /** - * Set the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @param scaleUnit the scaleUnit value to set. + * Set the networkAcls property: A collection of network authorization rules. + * + * @param networkAcls the networkAcls value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withScaleUnit(String scaleUnit) { - this.scaleUnit = scaleUnit; + public NamespaceProperties withNetworkAcls(NetworkAcls networkAcls) { + this.networkAcls = networkAcls; return this; } /** - * Get the enabled property: Whether or not the namespace is currently enabled. - * - * @return the enabled value. + * Get the pnsCredentials property: Collection of Notification Hub or Notification Hub Namespace PNS credentials. + * + * @return the pnsCredentials value. */ - public Boolean enabled() { - return this.enabled; + public PnsCredentials pnsCredentials() { + return this.pnsCredentials; } /** - * Set the enabled property: Whether or not the namespace is currently enabled. - * - * @param enabled the enabled value to set. + * Set the pnsCredentials property: Collection of Notification Hub or Notification Hub Namespace PNS credentials. + * + * @param pnsCredentials the pnsCredentials value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withEnabled(Boolean enabled) { - this.enabled = enabled; + public NamespaceProperties withPnsCredentials(PnsCredentials pnsCredentials) { + this.pnsCredentials = pnsCredentials; return this; } /** - * Get the critical property: Whether or not the namespace is set as Critical. - * - * @return the critical value. + * Get the serviceBusEndpoint property: Gets or sets endpoint you can use to perform NotificationHub + * operations. + * + * @return the serviceBusEndpoint value. */ - public Boolean critical() { - return this.critical; + public String serviceBusEndpoint() { + return this.serviceBusEndpoint; } /** - * Set the critical property: Whether or not the namespace is set as Critical. - * - * @param critical the critical value to set. + * Get the privateEndpointConnections property: Private Endpoint Connections for namespace. + * + * @return the privateEndpointConnections value. + */ + public List privateEndpointConnections() { + return this.privateEndpointConnections; + } + + /** + * Get the scaleUnit property: Gets or sets scaleUnit where the namespace gets created. + * + * @return the scaleUnit value. + */ + public String scaleUnit() { + return this.scaleUnit; + } + + /** + * Set the scaleUnit property: Gets or sets scaleUnit where the namespace gets created. + * + * @param scaleUnit the scaleUnit value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withCritical(Boolean critical) { - this.critical = critical; + public NamespaceProperties withScaleUnit(String scaleUnit) { + this.scaleUnit = scaleUnit; return this; } /** - * Get the dataCenter property: Data center for the namespace. - * + * Get the dataCenter property: Deprecated. + * * @return the dataCenter value. */ public String dataCenter() { @@ -350,8 +414,8 @@ public String dataCenter() { } /** - * Set the dataCenter property: Data center for the namespace. - * + * Set the dataCenter property: Deprecated. + * * @param dataCenter the dataCenter value to set. * @return the NamespaceProperties object itself. */ @@ -361,30 +425,39 @@ public NamespaceProperties withDataCenter(String dataCenter) { } /** - * Get the namespaceType property: The namespace type. - * - * @return the namespaceType value. + * Get the publicNetworkAccess property: Type of public network access. + * + * @return the publicNetworkAccess value. */ - public NamespaceType namespaceType() { - return this.namespaceType; + public PublicNetworkAccess publicNetworkAccess() { + return this.publicNetworkAccess; } /** - * Set the namespaceType property: The namespace type. - * - * @param namespaceType the namespaceType value to set. + * Set the publicNetworkAccess property: Type of public network access. + * + * @param publicNetworkAccess the publicNetworkAccess value to set. * @return the NamespaceProperties object itself. */ - public NamespaceProperties withNamespaceType(NamespaceType namespaceType) { - this.namespaceType = namespaceType; + public NamespaceProperties withPublicNetworkAccess(PublicNetworkAccess publicNetworkAccess) { + this.publicNetworkAccess = publicNetworkAccess; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (networkAcls() != null) { + networkAcls().validate(); + } + if (pnsCredentials() != null) { + pnsCredentials().validate(); + } + if (privateEndpointConnections() != null) { + privateEndpointConnections().forEach(e -> e.validate()); + } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceResourceInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceResourceInner.java index 74cb45a017ad5..c669e503086ea 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceResourceInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NamespaceResourceInner.java @@ -6,43 +6,53 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; +import com.azure.core.management.SystemData; import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.notificationhubs.models.NamespaceStatus; import com.azure.resourcemanager.notificationhubs.models.NamespaceType; +import com.azure.resourcemanager.notificationhubs.models.NetworkAcls; +import com.azure.resourcemanager.notificationhubs.models.OperationProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PublicNetworkAccess; +import com.azure.resourcemanager.notificationhubs.models.ReplicationRegion; import com.azure.resourcemanager.notificationhubs.models.Sku; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.models.ZoneRedundancyPreference; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; +import java.util.List; import java.util.Map; -/** Description of a Namespace resource. */ +/** + * Notification Hubs Namespace Resource. + */ @Fluent public final class NamespaceResourceInner extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NamespaceResourceInner.class); + /* + * The Sku description for a namespace + */ + @JsonProperty(value = "sku", required = true) + private Sku sku; /* - * Properties of the Namespace. + * Represents namespace properties. */ @JsonProperty(value = "properties") private NamespaceProperties innerProperties; /* - * The sku of the created namespace + * Azure Resource Manager metadata containing createdBy and modifiedBy information. */ - @JsonProperty(value = "sku") - private Sku sku; + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; /** - * Get the innerProperties property: Properties of the Namespace. - * - * @return the innerProperties value. + * Creates an instance of NamespaceResourceInner class. */ - private NamespaceProperties innerProperties() { - return this.innerProperties; + public NamespaceResourceInner() { } /** - * Get the sku property: The sku of the created namespace. - * + * Get the sku property: The Sku description for a namespace. + * * @return the sku value. */ public Sku sku() { @@ -50,8 +60,8 @@ public Sku sku() { } /** - * Set the sku property: The sku of the created namespace. - * + * Set the sku property: The Sku description for a namespace. + * * @param sku the sku value to set. * @return the NamespaceResourceInner object itself. */ @@ -60,14 +70,36 @@ public NamespaceResourceInner withSku(Sku sku) { return this; } - /** {@inheritDoc} */ + /** + * Get the innerProperties property: Represents namespace properties. + * + * @return the innerProperties value. + */ + private NamespaceProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * {@inheritDoc} + */ @Override public NamespaceResourceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public NamespaceResourceInner withTags(Map tags) { super.withTags(tags); @@ -75,8 +107,9 @@ public NamespaceResourceInner withTags(Map tags) { } /** - * Get the name property: The name of the namespace. - * + * Get the name property: Name of the Notification Hubs namespace. This is immutable property, set automatically + * by the service when the namespace is created. + * * @return the name value. */ public String namePropertiesName() { @@ -84,35 +117,21 @@ public String namePropertiesName() { } /** - * Set the name property: The name of the namespace. - * - * @param name the name value to set. - * @return the NamespaceResourceInner object itself. - */ - public NamespaceResourceInner withNamePropertiesName(String name) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withName(name); - return this; - } - - /** - * Get the provisioningState property: Provisioning state of the Namespace. - * + * Get the provisioningState property: Defines values for OperationProvisioningState. + * * @return the provisioningState value. */ - public String provisioningState() { + public OperationProvisioningState provisioningState() { return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); } /** - * Set the provisioningState property: Provisioning state of the Namespace. - * + * Set the provisioningState property: Defines values for OperationProvisioningState. + * * @param provisioningState the provisioningState value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withProvisioningState(String provisioningState) { + public NamespaceResourceInner withProvisioningState(OperationProvisioningState provisioningState) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } @@ -121,92 +140,87 @@ public NamespaceResourceInner withProvisioningState(String provisioningState) { } /** - * Get the region property: Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North Central - * US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West Europe. - * - * @return the region value. + * Get the status property: Namespace status. + * + * @return the status value. */ - public String region() { - return this.innerProperties() == null ? null : this.innerProperties().region(); + public NamespaceStatus status() { + return this.innerProperties() == null ? null : this.innerProperties().status(); } /** - * Set the region property: Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North Central - * US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West Europe. - * - * @param region the region value to set. + * Set the status property: Namespace status. + * + * @param status the status value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withRegion(String region) { + public NamespaceResourceInner withStatus(NamespaceStatus status) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withRegion(region); + this.innerProperties().withStatus(status); return this; } /** - * Get the metricId property: Identifier for Azure Insights metrics. - * - * @return the metricId value. + * Get the enabled property: Gets or sets whether or not the namespace is currently enabled. + * + * @return the enabled value. */ - public String metricId() { - return this.innerProperties() == null ? null : this.innerProperties().metricId(); + public Boolean enabled() { + return this.innerProperties() == null ? null : this.innerProperties().enabled(); } /** - * Get the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. - * - * @return the status value. + * Get the critical property: Gets or sets whether or not the namespace is set as Critical. + * + * @return the critical value. */ - public String status() { - return this.innerProperties() == null ? null : this.innerProperties().status(); + public Boolean critical() { + return this.innerProperties() == null ? null : this.innerProperties().critical(); } /** - * Set the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. - * - * @param status the status value to set. - * @return the NamespaceResourceInner object itself. + * Get the subscriptionId property: Namespace subscription id. + * + * @return the subscriptionId value. */ - public NamespaceResourceInner withStatus(String status) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withStatus(status); - return this; + public String subscriptionId() { + return this.innerProperties() == null ? null : this.innerProperties().subscriptionId(); } /** - * Get the createdAt property: The time the namespace was created. - * - * @return the createdAt value. + * Get the region property: Region. The value is always set to the same value as Namespace.Location, so we are + * deprecating + * this property. + * + * @return the region value. */ - public OffsetDateTime createdAt() { - return this.innerProperties() == null ? null : this.innerProperties().createdAt(); + public String region() { + return this.innerProperties() == null ? null : this.innerProperties().region(); } /** - * Set the createdAt property: The time the namespace was created. - * - * @param createdAt the createdAt value to set. - * @return the NamespaceResourceInner object itself. + * Get the metricId property: Azure Insights Metrics id. + * + * @return the metricId value. */ - public NamespaceResourceInner withCreatedAt(OffsetDateTime createdAt) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withCreatedAt(createdAt); - return this; + public String metricId() { + return this.innerProperties() == null ? null : this.innerProperties().metricId(); } /** - * Get the updatedAt property: The time the namespace was updated. - * + * Get the createdAt property: Time when the namespace was created. + * + * @return the createdAt value. + */ + public OffsetDateTime createdAt() { + return this.innerProperties() == null ? null : this.innerProperties().createdAt(); + } + + /** + * Get the updatedAt property: Time when the namespace was updated. + * * @return the updatedAt value. */ public OffsetDateTime updatedAt() { @@ -214,137 +228,165 @@ public OffsetDateTime updatedAt() { } /** - * Set the updatedAt property: The time the namespace was updated. - * - * @param updatedAt the updatedAt value to set. + * Get the namespaceType property: Defines values for NamespaceType. + * + * @return the namespaceType value. + */ + public NamespaceType namespaceType() { + return this.innerProperties() == null ? null : this.innerProperties().namespaceType(); + } + + /** + * Set the namespaceType property: Defines values for NamespaceType. + * + * @param namespaceType the namespaceType value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withUpdatedAt(OffsetDateTime updatedAt) { + public NamespaceResourceInner withNamespaceType(NamespaceType namespaceType) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withUpdatedAt(updatedAt); + this.innerProperties().withNamespaceType(namespaceType); return this; } /** - * Get the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @return the serviceBusEndpoint value. + * Get the replicationRegion property: Allowed replication region. + * + * @return the replicationRegion value. */ - public String serviceBusEndpoint() { - return this.innerProperties() == null ? null : this.innerProperties().serviceBusEndpoint(); + public ReplicationRegion replicationRegion() { + return this.innerProperties() == null ? null : this.innerProperties().replicationRegion(); } /** - * Set the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @param serviceBusEndpoint the serviceBusEndpoint value to set. + * Set the replicationRegion property: Allowed replication region. + * + * @param replicationRegion the replicationRegion value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withServiceBusEndpoint(String serviceBusEndpoint) { + public NamespaceResourceInner withReplicationRegion(ReplicationRegion replicationRegion) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withServiceBusEndpoint(serviceBusEndpoint); + this.innerProperties().withReplicationRegion(replicationRegion); return this; } /** - * Get the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @return the subscriptionId value. + * Get the zoneRedundancy property: Namespace SKU name. + * + * @return the zoneRedundancy value. */ - public String subscriptionId() { - return this.innerProperties() == null ? null : this.innerProperties().subscriptionId(); + public ZoneRedundancyPreference zoneRedundancy() { + return this.innerProperties() == null ? null : this.innerProperties().zoneRedundancy(); } /** - * Set the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @param subscriptionId the subscriptionId value to set. + * Set the zoneRedundancy property: Namespace SKU name. + * + * @param zoneRedundancy the zoneRedundancy value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withSubscriptionId(String subscriptionId) { + public NamespaceResourceInner withZoneRedundancy(ZoneRedundancyPreference zoneRedundancy) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withSubscriptionId(subscriptionId); + this.innerProperties().withZoneRedundancy(zoneRedundancy); return this; } /** - * Get the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @return the scaleUnit value. + * Get the networkAcls property: A collection of network authorization rules. + * + * @return the networkAcls value. */ - public String scaleUnit() { - return this.innerProperties() == null ? null : this.innerProperties().scaleUnit(); + public NetworkAcls networkAcls() { + return this.innerProperties() == null ? null : this.innerProperties().networkAcls(); } /** - * Set the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @param scaleUnit the scaleUnit value to set. + * Set the networkAcls property: A collection of network authorization rules. + * + * @param networkAcls the networkAcls value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withScaleUnit(String scaleUnit) { + public NamespaceResourceInner withNetworkAcls(NetworkAcls networkAcls) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withScaleUnit(scaleUnit); + this.innerProperties().withNetworkAcls(networkAcls); return this; } /** - * Get the enabled property: Whether or not the namespace is currently enabled. - * - * @return the enabled value. + * Get the pnsCredentials property: Collection of Notification Hub or Notification Hub Namespace PNS credentials. + * + * @return the pnsCredentials value. */ - public Boolean enabled() { - return this.innerProperties() == null ? null : this.innerProperties().enabled(); + public PnsCredentials pnsCredentials() { + return this.innerProperties() == null ? null : this.innerProperties().pnsCredentials(); } /** - * Set the enabled property: Whether or not the namespace is currently enabled. - * - * @param enabled the enabled value to set. + * Set the pnsCredentials property: Collection of Notification Hub or Notification Hub Namespace PNS credentials. + * + * @param pnsCredentials the pnsCredentials value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withEnabled(Boolean enabled) { + public NamespaceResourceInner withPnsCredentials(PnsCredentials pnsCredentials) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withEnabled(enabled); + this.innerProperties().withPnsCredentials(pnsCredentials); return this; } /** - * Get the critical property: Whether or not the namespace is set as Critical. - * - * @return the critical value. + * Get the serviceBusEndpoint property: Gets or sets endpoint you can use to perform NotificationHub + * operations. + * + * @return the serviceBusEndpoint value. */ - public Boolean critical() { - return this.innerProperties() == null ? null : this.innerProperties().critical(); + public String serviceBusEndpoint() { + return this.innerProperties() == null ? null : this.innerProperties().serviceBusEndpoint(); + } + + /** + * Get the privateEndpointConnections property: Private Endpoint Connections for namespace. + * + * @return the privateEndpointConnections value. + */ + public List privateEndpointConnections() { + return this.innerProperties() == null ? null : this.innerProperties().privateEndpointConnections(); + } + + /** + * Get the scaleUnit property: Gets or sets scaleUnit where the namespace gets created. + * + * @return the scaleUnit value. + */ + public String scaleUnit() { + return this.innerProperties() == null ? null : this.innerProperties().scaleUnit(); } /** - * Set the critical property: Whether or not the namespace is set as Critical. - * - * @param critical the critical value to set. + * Set the scaleUnit property: Gets or sets scaleUnit where the namespace gets created. + * + * @param scaleUnit the scaleUnit value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withCritical(Boolean critical) { + public NamespaceResourceInner withScaleUnit(String scaleUnit) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withCritical(critical); + this.innerProperties().withScaleUnit(scaleUnit); return this; } /** - * Get the dataCenter property: Data center for the namespace. - * + * Get the dataCenter property: Deprecated. + * * @return the dataCenter value. */ public String dataCenter() { @@ -352,8 +394,8 @@ public String dataCenter() { } /** - * Set the dataCenter property: Data center for the namespace. - * + * Set the dataCenter property: Deprecated. + * * @param dataCenter the dataCenter value to set. * @return the NamespaceResourceInner object itself. */ @@ -366,39 +408,44 @@ public NamespaceResourceInner withDataCenter(String dataCenter) { } /** - * Get the namespaceType property: The namespace type. - * - * @return the namespaceType value. + * Get the publicNetworkAccess property: Type of public network access. + * + * @return the publicNetworkAccess value. */ - public NamespaceType namespaceType() { - return this.innerProperties() == null ? null : this.innerProperties().namespaceType(); + public PublicNetworkAccess publicNetworkAccess() { + return this.innerProperties() == null ? null : this.innerProperties().publicNetworkAccess(); } /** - * Set the namespaceType property: The namespace type. - * - * @param namespaceType the namespaceType value to set. + * Set the publicNetworkAccess property: Type of public network access. + * + * @param publicNetworkAccess the publicNetworkAccess value to set. * @return the NamespaceResourceInner object itself. */ - public NamespaceResourceInner withNamespaceType(NamespaceType namespaceType) { + public NamespaceResourceInner withPublicNetworkAccess(PublicNetworkAccess publicNetworkAccess) { if (this.innerProperties() == null) { this.innerProperties = new NamespaceProperties(); } - this.innerProperties().withNamespaceType(namespaceType); + this.innerProperties().withPublicNetworkAccess(publicNetworkAccess); return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (sku() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sku in model NamespaceResourceInner")); + } else { + sku().validate(); + } if (innerProperties() != null) { innerProperties().validate(); } - if (sku() != null) { - sku().validate(); - } } + + private static final ClientLogger LOGGER = new ClientLogger(NamespaceResourceInner.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubProperties.java index 60f154e59a183..b100ded20327f 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubProperties.java @@ -5,79 +5,110 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.models.AdmCredential; import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; +import com.azure.resourcemanager.notificationhubs.models.BrowserCredential; +import com.azure.resourcemanager.notificationhubs.models.FcmV1Credential; import com.azure.resourcemanager.notificationhubs.models.GcmCredential; import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; import com.azure.resourcemanager.notificationhubs.models.WnsCredential; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.models.XiaomiCredential; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** NotificationHub properties. */ +/** + * NotificationHub properties. + */ @Fluent public final class NotificationHubProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NotificationHubProperties.class); - /* - * The NotificationHub name. + * Gets or sets the NotificationHub name. */ @JsonProperty(value = "name") private String name; /* - * The RegistrationTtl of the created NotificationHub + * Gets or sets the RegistrationTtl of the created NotificationHub */ @JsonProperty(value = "registrationTtl") private String registrationTtl; /* - * The AuthorizationRules of the created NotificationHub + * Gets or sets the AuthorizationRules of the created NotificationHub */ - @JsonProperty(value = "authorizationRules") + @JsonProperty(value = "authorizationRules", access = JsonProperty.Access.WRITE_ONLY) private List authorizationRules; /* - * The ApnsCredential of the created NotificationHub + * Description of a NotificationHub ApnsCredential. */ @JsonProperty(value = "apnsCredential") private ApnsCredential apnsCredential; /* - * The WnsCredential of the created NotificationHub + * Description of a NotificationHub WnsCredential. */ @JsonProperty(value = "wnsCredential") private WnsCredential wnsCredential; /* - * The GcmCredential of the created NotificationHub + * Description of a NotificationHub GcmCredential. */ @JsonProperty(value = "gcmCredential") private GcmCredential gcmCredential; /* - * The MpnsCredential of the created NotificationHub + * Description of a NotificationHub MpnsCredential. */ @JsonProperty(value = "mpnsCredential") private MpnsCredential mpnsCredential; /* - * The AdmCredential of the created NotificationHub + * Description of a NotificationHub AdmCredential. */ @JsonProperty(value = "admCredential") private AdmCredential admCredential; /* - * The BaiduCredential of the created NotificationHub + * Description of a NotificationHub BaiduCredential. */ @JsonProperty(value = "baiduCredential") private BaiduCredential baiduCredential; + /* + * Description of a NotificationHub BrowserCredential. + */ + @JsonProperty(value = "browserCredential") + private BrowserCredential browserCredential; + + /* + * Description of a NotificationHub XiaomiCredential. + */ + @JsonProperty(value = "xiaomiCredential") + private XiaomiCredential xiaomiCredential; + + /* + * Description of a NotificationHub FcmV1Credential. + */ + @JsonProperty(value = "fcmV1Credential") + private FcmV1Credential fcmV1Credential; + + /* + * The dailyMaxActiveDevices property. + */ + @JsonProperty(value = "dailyMaxActiveDevices", access = JsonProperty.Access.WRITE_ONLY) + private Long dailyMaxActiveDevices; + + /** + * Creates an instance of NotificationHubProperties class. + */ + public NotificationHubProperties() { + } + /** - * Get the name property: The NotificationHub name. - * + * Get the name property: Gets or sets the NotificationHub name. + * * @return the name value. */ public String name() { @@ -85,8 +116,8 @@ public String name() { } /** - * Set the name property: The NotificationHub name. - * + * Set the name property: Gets or sets the NotificationHub name. + * * @param name the name value to set. * @return the NotificationHubProperties object itself. */ @@ -96,8 +127,8 @@ public NotificationHubProperties withName(String name) { } /** - * Get the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Get the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @return the registrationTtl value. */ public String registrationTtl() { @@ -105,8 +136,8 @@ public String registrationTtl() { } /** - * Set the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Set the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @param registrationTtl the registrationTtl value to set. * @return the NotificationHubProperties object itself. */ @@ -116,8 +147,8 @@ public NotificationHubProperties withRegistrationTtl(String registrationTtl) { } /** - * Get the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * + * Get the authorizationRules property: Gets or sets the AuthorizationRules of the created NotificationHub. + * * @return the authorizationRules value. */ public List authorizationRules() { @@ -125,20 +156,8 @@ public List authorizationRules() { } /** - * Set the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @param authorizationRules the authorizationRules value to set. - * @return the NotificationHubProperties object itself. - */ - public NotificationHubProperties withAuthorizationRules( - List authorizationRules) { - this.authorizationRules = authorizationRules; - return this; - } - - /** - * Get the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Get the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @return the apnsCredential value. */ public ApnsCredential apnsCredential() { @@ -146,8 +165,8 @@ public ApnsCredential apnsCredential() { } /** - * Set the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Set the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @param apnsCredential the apnsCredential value to set. * @return the NotificationHubProperties object itself. */ @@ -157,8 +176,8 @@ public NotificationHubProperties withApnsCredential(ApnsCredential apnsCredentia } /** - * Get the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Get the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @return the wnsCredential value. */ public WnsCredential wnsCredential() { @@ -166,8 +185,8 @@ public WnsCredential wnsCredential() { } /** - * Set the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Set the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @param wnsCredential the wnsCredential value to set. * @return the NotificationHubProperties object itself. */ @@ -177,8 +196,8 @@ public NotificationHubProperties withWnsCredential(WnsCredential wnsCredential) } /** - * Get the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Get the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @return the gcmCredential value. */ public GcmCredential gcmCredential() { @@ -186,8 +205,8 @@ public GcmCredential gcmCredential() { } /** - * Set the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Set the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @param gcmCredential the gcmCredential value to set. * @return the NotificationHubProperties object itself. */ @@ -197,8 +216,8 @@ public NotificationHubProperties withGcmCredential(GcmCredential gcmCredential) } /** - * Get the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Get the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @return the mpnsCredential value. */ public MpnsCredential mpnsCredential() { @@ -206,8 +225,8 @@ public MpnsCredential mpnsCredential() { } /** - * Set the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Set the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @param mpnsCredential the mpnsCredential value to set. * @return the NotificationHubProperties object itself. */ @@ -217,8 +236,8 @@ public NotificationHubProperties withMpnsCredential(MpnsCredential mpnsCredentia } /** - * Get the admCredential property: The AdmCredential of the created NotificationHub. - * + * Get the admCredential property: Description of a NotificationHub AdmCredential. + * * @return the admCredential value. */ public AdmCredential admCredential() { @@ -226,8 +245,8 @@ public AdmCredential admCredential() { } /** - * Set the admCredential property: The AdmCredential of the created NotificationHub. - * + * Set the admCredential property: Description of a NotificationHub AdmCredential. + * * @param admCredential the admCredential value to set. * @return the NotificationHubProperties object itself. */ @@ -237,8 +256,8 @@ public NotificationHubProperties withAdmCredential(AdmCredential admCredential) } /** - * Get the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Get the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @return the baiduCredential value. */ public BaiduCredential baiduCredential() { @@ -246,8 +265,8 @@ public BaiduCredential baiduCredential() { } /** - * Set the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Set the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @param baiduCredential the baiduCredential value to set. * @return the NotificationHubProperties object itself. */ @@ -256,9 +275,78 @@ public NotificationHubProperties withBaiduCredential(BaiduCredential baiduCreden return this; } + /** + * Get the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + public BrowserCredential browserCredential() { + return this.browserCredential; + } + + /** + * Set the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @param browserCredential the browserCredential value to set. + * @return the NotificationHubProperties object itself. + */ + public NotificationHubProperties withBrowserCredential(BrowserCredential browserCredential) { + this.browserCredential = browserCredential; + return this; + } + + /** + * Get the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. + */ + public XiaomiCredential xiaomiCredential() { + return this.xiaomiCredential; + } + + /** + * Set the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @param xiaomiCredential the xiaomiCredential value to set. + * @return the NotificationHubProperties object itself. + */ + public NotificationHubProperties withXiaomiCredential(XiaomiCredential xiaomiCredential) { + this.xiaomiCredential = xiaomiCredential; + return this; + } + + /** + * Get the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + public FcmV1Credential fcmV1Credential() { + return this.fcmV1Credential; + } + + /** + * Set the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @param fcmV1Credential the fcmV1Credential value to set. + * @return the NotificationHubProperties object itself. + */ + public NotificationHubProperties withFcmV1Credential(FcmV1Credential fcmV1Credential) { + this.fcmV1Credential = fcmV1Credential; + return this; + } + + /** + * Get the dailyMaxActiveDevices property: The dailyMaxActiveDevices property. + * + * @return the dailyMaxActiveDevices value. + */ + public Long dailyMaxActiveDevices() { + return this.dailyMaxActiveDevices; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -283,5 +371,14 @@ public void validate() { if (baiduCredential() != null) { baiduCredential().validate(); } + if (browserCredential() != null) { + browserCredential().validate(); + } + if (xiaomiCredential() != null) { + xiaomiCredential().validate(); + } + if (fcmV1Credential() != null) { + fcmV1Credential().validate(); + } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubResourceInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubResourceInner.java index c449525c9cb6f..09dbde1e7cb70 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubResourceInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/NotificationHubResourceInner.java @@ -6,39 +6,53 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.models.AdmCredential; import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; +import com.azure.resourcemanager.notificationhubs.models.BrowserCredential; +import com.azure.resourcemanager.notificationhubs.models.FcmV1Credential; import com.azure.resourcemanager.notificationhubs.models.GcmCredential; import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.WnsCredential; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.models.XiaomiCredential; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; import java.util.Map; -/** Description of a NotificationHub Resource. */ +/** + * Notification Hub Resource. + */ @Fluent public final class NotificationHubResourceInner extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NotificationHubResourceInner.class); - /* - * Properties of the NotificationHub. + * NotificationHub properties. */ @JsonProperty(value = "properties") private NotificationHubProperties innerProperties; /* - * The sku of the created namespace + * The Sku description for a namespace */ @JsonProperty(value = "sku") private Sku sku; + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + /** - * Get the innerProperties property: Properties of the NotificationHub. - * + * Creates an instance of NotificationHubResourceInner class. + */ + public NotificationHubResourceInner() { + } + + /** + * Get the innerProperties property: NotificationHub properties. + * * @return the innerProperties value. */ private NotificationHubProperties innerProperties() { @@ -46,8 +60,8 @@ private NotificationHubProperties innerProperties() { } /** - * Get the sku property: The sku of the created namespace. - * + * Get the sku property: The Sku description for a namespace. + * * @return the sku value. */ public Sku sku() { @@ -55,8 +69,8 @@ public Sku sku() { } /** - * Set the sku property: The sku of the created namespace. - * + * Set the sku property: The Sku description for a namespace. + * * @param sku the sku value to set. * @return the NotificationHubResourceInner object itself. */ @@ -65,14 +79,27 @@ public NotificationHubResourceInner withSku(Sku sku) { return this; } - /** {@inheritDoc} */ + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * {@inheritDoc} + */ @Override public NotificationHubResourceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public NotificationHubResourceInner withTags(Map tags) { super.withTags(tags); @@ -80,8 +107,8 @@ public NotificationHubResourceInner withTags(Map tags) { } /** - * Get the name property: The NotificationHub name. - * + * Get the name property: Gets or sets the NotificationHub name. + * * @return the name value. */ public String namePropertiesName() { @@ -89,8 +116,8 @@ public String namePropertiesName() { } /** - * Set the name property: The NotificationHub name. - * + * Set the name property: Gets or sets the NotificationHub name. + * * @param name the name value to set. * @return the NotificationHubResourceInner object itself. */ @@ -103,8 +130,8 @@ public NotificationHubResourceInner withNamePropertiesName(String name) { } /** - * Get the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Get the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @return the registrationTtl value. */ public String registrationTtl() { @@ -112,8 +139,8 @@ public String registrationTtl() { } /** - * Set the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Set the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @param registrationTtl the registrationTtl value to set. * @return the NotificationHubResourceInner object itself. */ @@ -126,8 +153,8 @@ public NotificationHubResourceInner withRegistrationTtl(String registrationTtl) } /** - * Get the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * + * Get the authorizationRules property: Gets or sets the AuthorizationRules of the created NotificationHub. + * * @return the authorizationRules value. */ public List authorizationRules() { @@ -135,23 +162,8 @@ public List authorizationRules() { } /** - * Set the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @param authorizationRules the authorizationRules value to set. - * @return the NotificationHubResourceInner object itself. - */ - public NotificationHubResourceInner withAuthorizationRules( - List authorizationRules) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withAuthorizationRules(authorizationRules); - return this; - } - - /** - * Get the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Get the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @return the apnsCredential value. */ public ApnsCredential apnsCredential() { @@ -159,8 +171,8 @@ public ApnsCredential apnsCredential() { } /** - * Set the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Set the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @param apnsCredential the apnsCredential value to set. * @return the NotificationHubResourceInner object itself. */ @@ -173,8 +185,8 @@ public NotificationHubResourceInner withApnsCredential(ApnsCredential apnsCreden } /** - * Get the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Get the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @return the wnsCredential value. */ public WnsCredential wnsCredential() { @@ -182,8 +194,8 @@ public WnsCredential wnsCredential() { } /** - * Set the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Set the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @param wnsCredential the wnsCredential value to set. * @return the NotificationHubResourceInner object itself. */ @@ -196,8 +208,8 @@ public NotificationHubResourceInner withWnsCredential(WnsCredential wnsCredentia } /** - * Get the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Get the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @return the gcmCredential value. */ public GcmCredential gcmCredential() { @@ -205,8 +217,8 @@ public GcmCredential gcmCredential() { } /** - * Set the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Set the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @param gcmCredential the gcmCredential value to set. * @return the NotificationHubResourceInner object itself. */ @@ -219,8 +231,8 @@ public NotificationHubResourceInner withGcmCredential(GcmCredential gcmCredentia } /** - * Get the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Get the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @return the mpnsCredential value. */ public MpnsCredential mpnsCredential() { @@ -228,8 +240,8 @@ public MpnsCredential mpnsCredential() { } /** - * Set the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Set the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @param mpnsCredential the mpnsCredential value to set. * @return the NotificationHubResourceInner object itself. */ @@ -242,8 +254,8 @@ public NotificationHubResourceInner withMpnsCredential(MpnsCredential mpnsCreden } /** - * Get the admCredential property: The AdmCredential of the created NotificationHub. - * + * Get the admCredential property: Description of a NotificationHub AdmCredential. + * * @return the admCredential value. */ public AdmCredential admCredential() { @@ -251,8 +263,8 @@ public AdmCredential admCredential() { } /** - * Set the admCredential property: The AdmCredential of the created NotificationHub. - * + * Set the admCredential property: Description of a NotificationHub AdmCredential. + * * @param admCredential the admCredential value to set. * @return the NotificationHubResourceInner object itself. */ @@ -265,8 +277,8 @@ public NotificationHubResourceInner withAdmCredential(AdmCredential admCredentia } /** - * Get the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Get the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @return the baiduCredential value. */ public BaiduCredential baiduCredential() { @@ -274,8 +286,8 @@ public BaiduCredential baiduCredential() { } /** - * Set the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Set the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @param baiduCredential the baiduCredential value to set. * @return the NotificationHubResourceInner object itself. */ @@ -287,9 +299,87 @@ public NotificationHubResourceInner withBaiduCredential(BaiduCredential baiduCre return this; } + /** + * Get the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + public BrowserCredential browserCredential() { + return this.innerProperties() == null ? null : this.innerProperties().browserCredential(); + } + + /** + * Set the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @param browserCredential the browserCredential value to set. + * @return the NotificationHubResourceInner object itself. + */ + public NotificationHubResourceInner withBrowserCredential(BrowserCredential browserCredential) { + if (this.innerProperties() == null) { + this.innerProperties = new NotificationHubProperties(); + } + this.innerProperties().withBrowserCredential(browserCredential); + return this; + } + + /** + * Get the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. + */ + public XiaomiCredential xiaomiCredential() { + return this.innerProperties() == null ? null : this.innerProperties().xiaomiCredential(); + } + + /** + * Set the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @param xiaomiCredential the xiaomiCredential value to set. + * @return the NotificationHubResourceInner object itself. + */ + public NotificationHubResourceInner withXiaomiCredential(XiaomiCredential xiaomiCredential) { + if (this.innerProperties() == null) { + this.innerProperties = new NotificationHubProperties(); + } + this.innerProperties().withXiaomiCredential(xiaomiCredential); + return this; + } + + /** + * Get the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + public FcmV1Credential fcmV1Credential() { + return this.innerProperties() == null ? null : this.innerProperties().fcmV1Credential(); + } + + /** + * Set the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @param fcmV1Credential the fcmV1Credential value to set. + * @return the NotificationHubResourceInner object itself. + */ + public NotificationHubResourceInner withFcmV1Credential(FcmV1Credential fcmV1Credential) { + if (this.innerProperties() == null) { + this.innerProperties = new NotificationHubProperties(); + } + this.innerProperties().withFcmV1Credential(fcmV1Credential); + return this; + } + + /** + * Get the dailyMaxActiveDevices property: The dailyMaxActiveDevices property. + * + * @return the dailyMaxActiveDevices value. + */ + public Long dailyMaxActiveDevices() { + return this.innerProperties() == null ? null : this.innerProperties().dailyMaxActiveDevices(); + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/OperationInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/OperationInner.java index 9a6e47c1903ac..c86dca6fafca1 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/OperationInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/OperationInner.java @@ -5,18 +5,17 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.models.OperationDisplay; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.models.OperationProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** A NotificationHubs REST API operation. */ +/** + * A NotificationHubs REST API operation. + */ @Fluent public final class OperationInner { - @JsonIgnore private final ClientLogger logger = new ClientLogger(OperationInner.class); - /* - * Operation name: {provider}/{resource}/{operation} + * Gets operation name: {provider}/{resource}/{operation} */ @JsonProperty(value = "name", access = JsonProperty.Access.WRITE_ONLY) private String name; @@ -27,9 +26,27 @@ public final class OperationInner { @JsonProperty(value = "display") private OperationDisplay display; + /* + * Optional operation properties. + */ + @JsonProperty(value = "properties") + private OperationProperties properties; + + /* + * Gets or sets IsDataAction property. It is used to differentiate management and data plane operations. + */ + @JsonProperty(value = "isDataAction", access = JsonProperty.Access.WRITE_ONLY) + private Boolean isDataAction; + /** - * Get the name property: Operation name: {provider}/{resource}/{operation}. - * + * Creates an instance of OperationInner class. + */ + public OperationInner() { + } + + /** + * Get the name property: Gets operation name: {provider}/{resource}/{operation}. + * * @return the name value. */ public String name() { @@ -38,7 +55,7 @@ public String name() { /** * Get the display property: The object that represents the operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -47,7 +64,7 @@ public OperationDisplay display() { /** * Set the display property: The object that represents the operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -56,14 +73,47 @@ public OperationInner withDisplay(OperationDisplay display) { return this; } + /** + * Get the properties property: Optional operation properties. + * + * @return the properties value. + */ + public OperationProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Optional operation properties. + * + * @param properties the properties value to set. + * @return the OperationInner object itself. + */ + public OperationInner withProperties(OperationProperties properties) { + this.properties = properties; + return this; + } + + /** + * Get the isDataAction property: Gets or sets IsDataAction property. It is used to differentiate management and + * data plane operations. + * + * @return the isDataAction value. + */ + public Boolean isDataAction() { + return this.isDataAction; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (display() != null) { display().validate(); } + if (properties() != null) { + properties().validate(); + } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentials.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentials.java new file mode 100644 index 0000000000000..4db3fc44a86a2 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentials.java @@ -0,0 +1,298 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.notificationhubs.models.AdmCredential; +import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; +import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; +import com.azure.resourcemanager.notificationhubs.models.BrowserCredential; +import com.azure.resourcemanager.notificationhubs.models.FcmV1Credential; +import com.azure.resourcemanager.notificationhubs.models.GcmCredential; +import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; +import com.azure.resourcemanager.notificationhubs.models.WnsCredential; +import com.azure.resourcemanager.notificationhubs.models.XiaomiCredential; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Collection of Notification Hub or Notification Hub Namespace PNS credentials. + */ +@Fluent +public final class PnsCredentials { + /* + * Description of a NotificationHub AdmCredential. + */ + @JsonProperty(value = "admCredential") + private AdmCredential admCredential; + + /* + * Description of a NotificationHub ApnsCredential. + */ + @JsonProperty(value = "apnsCredential") + private ApnsCredential apnsCredential; + + /* + * Description of a NotificationHub BaiduCredential. + */ + @JsonProperty(value = "baiduCredential") + private BaiduCredential baiduCredential; + + /* + * Description of a NotificationHub BrowserCredential. + */ + @JsonProperty(value = "browserCredential") + private BrowserCredential browserCredential; + + /* + * Description of a NotificationHub GcmCredential. + */ + @JsonProperty(value = "gcmCredential") + private GcmCredential gcmCredential; + + /* + * Description of a NotificationHub MpnsCredential. + */ + @JsonProperty(value = "mpnsCredential") + private MpnsCredential mpnsCredential; + + /* + * Description of a NotificationHub WnsCredential. + */ + @JsonProperty(value = "wnsCredential") + private WnsCredential wnsCredential; + + /* + * Description of a NotificationHub XiaomiCredential. + */ + @JsonProperty(value = "xiaomiCredential") + private XiaomiCredential xiaomiCredential; + + /* + * Description of a NotificationHub FcmV1Credential. + */ + @JsonProperty(value = "fcmV1Credential") + private FcmV1Credential fcmV1Credential; + + /** + * Creates an instance of PnsCredentials class. + */ + public PnsCredentials() { + } + + /** + * Get the admCredential property: Description of a NotificationHub AdmCredential. + * + * @return the admCredential value. + */ + public AdmCredential admCredential() { + return this.admCredential; + } + + /** + * Set the admCredential property: Description of a NotificationHub AdmCredential. + * + * @param admCredential the admCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withAdmCredential(AdmCredential admCredential) { + this.admCredential = admCredential; + return this; + } + + /** + * Get the apnsCredential property: Description of a NotificationHub ApnsCredential. + * + * @return the apnsCredential value. + */ + public ApnsCredential apnsCredential() { + return this.apnsCredential; + } + + /** + * Set the apnsCredential property: Description of a NotificationHub ApnsCredential. + * + * @param apnsCredential the apnsCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withApnsCredential(ApnsCredential apnsCredential) { + this.apnsCredential = apnsCredential; + return this; + } + + /** + * Get the baiduCredential property: Description of a NotificationHub BaiduCredential. + * + * @return the baiduCredential value. + */ + public BaiduCredential baiduCredential() { + return this.baiduCredential; + } + + /** + * Set the baiduCredential property: Description of a NotificationHub BaiduCredential. + * + * @param baiduCredential the baiduCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withBaiduCredential(BaiduCredential baiduCredential) { + this.baiduCredential = baiduCredential; + return this; + } + + /** + * Get the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + public BrowserCredential browserCredential() { + return this.browserCredential; + } + + /** + * Set the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @param browserCredential the browserCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withBrowserCredential(BrowserCredential browserCredential) { + this.browserCredential = browserCredential; + return this; + } + + /** + * Get the gcmCredential property: Description of a NotificationHub GcmCredential. + * + * @return the gcmCredential value. + */ + public GcmCredential gcmCredential() { + return this.gcmCredential; + } + + /** + * Set the gcmCredential property: Description of a NotificationHub GcmCredential. + * + * @param gcmCredential the gcmCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withGcmCredential(GcmCredential gcmCredential) { + this.gcmCredential = gcmCredential; + return this; + } + + /** + * Get the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * + * @return the mpnsCredential value. + */ + public MpnsCredential mpnsCredential() { + return this.mpnsCredential; + } + + /** + * Set the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * + * @param mpnsCredential the mpnsCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withMpnsCredential(MpnsCredential mpnsCredential) { + this.mpnsCredential = mpnsCredential; + return this; + } + + /** + * Get the wnsCredential property: Description of a NotificationHub WnsCredential. + * + * @return the wnsCredential value. + */ + public WnsCredential wnsCredential() { + return this.wnsCredential; + } + + /** + * Set the wnsCredential property: Description of a NotificationHub WnsCredential. + * + * @param wnsCredential the wnsCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withWnsCredential(WnsCredential wnsCredential) { + this.wnsCredential = wnsCredential; + return this; + } + + /** + * Get the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. + */ + public XiaomiCredential xiaomiCredential() { + return this.xiaomiCredential; + } + + /** + * Set the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @param xiaomiCredential the xiaomiCredential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withXiaomiCredential(XiaomiCredential xiaomiCredential) { + this.xiaomiCredential = xiaomiCredential; + return this; + } + + /** + * Get the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + public FcmV1Credential fcmV1Credential() { + return this.fcmV1Credential; + } + + /** + * Set the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @param fcmV1Credential the fcmV1Credential value to set. + * @return the PnsCredentials object itself. + */ + public PnsCredentials withFcmV1Credential(FcmV1Credential fcmV1Credential) { + this.fcmV1Credential = fcmV1Credential; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (admCredential() != null) { + admCredential().validate(); + } + if (apnsCredential() != null) { + apnsCredential().validate(); + } + if (baiduCredential() != null) { + baiduCredential().validate(); + } + if (browserCredential() != null) { + browserCredential().validate(); + } + if (gcmCredential() != null) { + gcmCredential().validate(); + } + if (mpnsCredential() != null) { + mpnsCredential().validate(); + } + if (wnsCredential() != null) { + wnsCredential().validate(); + } + if (xiaomiCredential() != null) { + xiaomiCredential().validate(); + } + if (fcmV1Credential() != null) { + fcmV1Credential().validate(); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsProperties.java deleted file mode 100644 index a23d279302ccc..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsProperties.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.notificationhubs.models.AdmCredential; -import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; -import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; -import com.azure.resourcemanager.notificationhubs.models.GcmCredential; -import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; -import com.azure.resourcemanager.notificationhubs.models.WnsCredential; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** Description of a NotificationHub PNS Credentials. */ -@Fluent -public final class PnsCredentialsProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(PnsCredentialsProperties.class); - - /* - * The ApnsCredential of the created NotificationHub - */ - @JsonProperty(value = "apnsCredential") - private ApnsCredential apnsCredential; - - /* - * The WnsCredential of the created NotificationHub - */ - @JsonProperty(value = "wnsCredential") - private WnsCredential wnsCredential; - - /* - * The GcmCredential of the created NotificationHub - */ - @JsonProperty(value = "gcmCredential") - private GcmCredential gcmCredential; - - /* - * The MpnsCredential of the created NotificationHub - */ - @JsonProperty(value = "mpnsCredential") - private MpnsCredential mpnsCredential; - - /* - * The AdmCredential of the created NotificationHub - */ - @JsonProperty(value = "admCredential") - private AdmCredential admCredential; - - /* - * The BaiduCredential of the created NotificationHub - */ - @JsonProperty(value = "baiduCredential") - private BaiduCredential baiduCredential; - - /** - * Get the apnsCredential property: The ApnsCredential of the created NotificationHub. - * - * @return the apnsCredential value. - */ - public ApnsCredential apnsCredential() { - return this.apnsCredential; - } - - /** - * Set the apnsCredential property: The ApnsCredential of the created NotificationHub. - * - * @param apnsCredential the apnsCredential value to set. - * @return the PnsCredentialsProperties object itself. - */ - public PnsCredentialsProperties withApnsCredential(ApnsCredential apnsCredential) { - this.apnsCredential = apnsCredential; - return this; - } - - /** - * Get the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @return the wnsCredential value. - */ - public WnsCredential wnsCredential() { - return this.wnsCredential; - } - - /** - * Set the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @param wnsCredential the wnsCredential value to set. - * @return the PnsCredentialsProperties object itself. - */ - public PnsCredentialsProperties withWnsCredential(WnsCredential wnsCredential) { - this.wnsCredential = wnsCredential; - return this; - } - - /** - * Get the gcmCredential property: The GcmCredential of the created NotificationHub. - * - * @return the gcmCredential value. - */ - public GcmCredential gcmCredential() { - return this.gcmCredential; - } - - /** - * Set the gcmCredential property: The GcmCredential of the created NotificationHub. - * - * @param gcmCredential the gcmCredential value to set. - * @return the PnsCredentialsProperties object itself. - */ - public PnsCredentialsProperties withGcmCredential(GcmCredential gcmCredential) { - this.gcmCredential = gcmCredential; - return this; - } - - /** - * Get the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * - * @return the mpnsCredential value. - */ - public MpnsCredential mpnsCredential() { - return this.mpnsCredential; - } - - /** - * Set the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * - * @param mpnsCredential the mpnsCredential value to set. - * @return the PnsCredentialsProperties object itself. - */ - public PnsCredentialsProperties withMpnsCredential(MpnsCredential mpnsCredential) { - this.mpnsCredential = mpnsCredential; - return this; - } - - /** - * Get the admCredential property: The AdmCredential of the created NotificationHub. - * - * @return the admCredential value. - */ - public AdmCredential admCredential() { - return this.admCredential; - } - - /** - * Set the admCredential property: The AdmCredential of the created NotificationHub. - * - * @param admCredential the admCredential value to set. - * @return the PnsCredentialsProperties object itself. - */ - public PnsCredentialsProperties withAdmCredential(AdmCredential admCredential) { - this.admCredential = admCredential; - return this; - } - - /** - * Get the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @return the baiduCredential value. - */ - public BaiduCredential baiduCredential() { - return this.baiduCredential; - } - - /** - * Set the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @param baiduCredential the baiduCredential value to set. - * @return the PnsCredentialsProperties object itself. - */ - public PnsCredentialsProperties withBaiduCredential(BaiduCredential baiduCredential) { - this.baiduCredential = baiduCredential; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (apnsCredential() != null) { - apnsCredential().validate(); - } - if (wnsCredential() != null) { - wnsCredential().validate(); - } - if (gcmCredential() != null) { - gcmCredential().validate(); - } - if (mpnsCredential() != null) { - mpnsCredential().validate(); - } - if (admCredential() != null) { - admCredential().validate(); - } - if (baiduCredential() != null) { - baiduCredential().validate(); - } - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsResourceInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsResourceInner.java index 9aff52334465c..6a7fe0c7634ed 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsResourceInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PnsCredentialsResourceInner.java @@ -5,82 +5,143 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.models.AdmCredential; import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; +import com.azure.resourcemanager.notificationhubs.models.BrowserCredential; +import com.azure.resourcemanager.notificationhubs.models.FcmV1Credential; import com.azure.resourcemanager.notificationhubs.models.GcmCredential; import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; -import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.WnsCredential; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.models.XiaomiCredential; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Description of a NotificationHub PNS Credentials. */ +/** + * Description of a NotificationHub PNS Credentials. This is a response of the POST requests that return namespace or + * hubs + * PNS credentials. + */ @Fluent -public final class PnsCredentialsResourceInner extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(PnsCredentialsResourceInner.class); - +public final class PnsCredentialsResourceInner extends ProxyResource { /* - * NotificationHub PNS Credentials. + * Collection of Notification Hub or Notification Hub Namespace PNS credentials. */ @JsonProperty(value = "properties") - private PnsCredentialsProperties innerProperties; + private PnsCredentials innerProperties; + + /* + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "location") + private String location; + + /* + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; /* - * The sku of the created namespace + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of PnsCredentialsResourceInner class. */ - @JsonProperty(value = "sku") - private Sku sku; + public PnsCredentialsResourceInner() { + } /** - * Get the innerProperties property: NotificationHub PNS Credentials. - * + * Get the innerProperties property: Collection of Notification Hub or Notification Hub Namespace PNS credentials. + * * @return the innerProperties value. */ - private PnsCredentialsProperties innerProperties() { + private PnsCredentials innerProperties() { return this.innerProperties; } /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. + * Get the location property: Deprecated - only for compatibility. + * + * @return the location value. */ - public Sku sku() { - return this.sku; + public String location() { + return this.location; } /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. + * Set the location property: Deprecated - only for compatibility. + * + * @param location the location value to set. * @return the PnsCredentialsResourceInner object itself. */ - public PnsCredentialsResourceInner withSku(Sku sku) { - this.sku = sku; + public PnsCredentialsResourceInner withLocation(String location) { + this.location = location; return this; } - /** {@inheritDoc} */ - @Override - public PnsCredentialsResourceInner withLocation(String location) { - super.withLocation(location); - return this; + /** + * Get the tags property: Deprecated - only for compatibility. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; } - /** {@inheritDoc} */ - @Override + /** + * Set the tags property: Deprecated - only for compatibility. + * + * @param tags the tags value to set. + * @return the PnsCredentialsResourceInner object itself. + */ public PnsCredentialsResourceInner withTags(Map tags) { - super.withTags(tags); + this.tags = tags; + return this; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the admCredential property: Description of a NotificationHub AdmCredential. + * + * @return the admCredential value. + */ + public AdmCredential admCredential() { + return this.innerProperties() == null ? null : this.innerProperties().admCredential(); + } + + /** + * Set the admCredential property: Description of a NotificationHub AdmCredential. + * + * @param admCredential the admCredential value to set. + * @return the PnsCredentialsResourceInner object itself. + */ + public PnsCredentialsResourceInner withAdmCredential(AdmCredential admCredential) { + if (this.innerProperties() == null) { + this.innerProperties = new PnsCredentials(); + } + this.innerProperties().withAdmCredential(admCredential); return this; } /** - * Get the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Get the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @return the apnsCredential value. */ public ApnsCredential apnsCredential() { @@ -88,45 +149,68 @@ public ApnsCredential apnsCredential() { } /** - * Set the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Set the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @param apnsCredential the apnsCredential value to set. * @return the PnsCredentialsResourceInner object itself. */ public PnsCredentialsResourceInner withApnsCredential(ApnsCredential apnsCredential) { if (this.innerProperties() == null) { - this.innerProperties = new PnsCredentialsProperties(); + this.innerProperties = new PnsCredentials(); } this.innerProperties().withApnsCredential(apnsCredential); return this; } /** - * Get the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @return the wnsCredential value. + * Get the baiduCredential property: Description of a NotificationHub BaiduCredential. + * + * @return the baiduCredential value. */ - public WnsCredential wnsCredential() { - return this.innerProperties() == null ? null : this.innerProperties().wnsCredential(); + public BaiduCredential baiduCredential() { + return this.innerProperties() == null ? null : this.innerProperties().baiduCredential(); } /** - * Set the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @param wnsCredential the wnsCredential value to set. + * Set the baiduCredential property: Description of a NotificationHub BaiduCredential. + * + * @param baiduCredential the baiduCredential value to set. * @return the PnsCredentialsResourceInner object itself. */ - public PnsCredentialsResourceInner withWnsCredential(WnsCredential wnsCredential) { + public PnsCredentialsResourceInner withBaiduCredential(BaiduCredential baiduCredential) { if (this.innerProperties() == null) { - this.innerProperties = new PnsCredentialsProperties(); + this.innerProperties = new PnsCredentials(); } - this.innerProperties().withWnsCredential(wnsCredential); + this.innerProperties().withBaiduCredential(baiduCredential); + return this; + } + + /** + * Get the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + public BrowserCredential browserCredential() { + return this.innerProperties() == null ? null : this.innerProperties().browserCredential(); + } + + /** + * Set the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @param browserCredential the browserCredential value to set. + * @return the PnsCredentialsResourceInner object itself. + */ + public PnsCredentialsResourceInner withBrowserCredential(BrowserCredential browserCredential) { + if (this.innerProperties() == null) { + this.innerProperties = new PnsCredentials(); + } + this.innerProperties().withBrowserCredential(browserCredential); return this; } /** - * Get the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Get the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @return the gcmCredential value. */ public GcmCredential gcmCredential() { @@ -134,22 +218,22 @@ public GcmCredential gcmCredential() { } /** - * Set the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Set the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @param gcmCredential the gcmCredential value to set. * @return the PnsCredentialsResourceInner object itself. */ public PnsCredentialsResourceInner withGcmCredential(GcmCredential gcmCredential) { if (this.innerProperties() == null) { - this.innerProperties = new PnsCredentialsProperties(); + this.innerProperties = new PnsCredentials(); } this.innerProperties().withGcmCredential(gcmCredential); return this; } /** - * Get the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Get the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @return the mpnsCredential value. */ public MpnsCredential mpnsCredential() { @@ -157,76 +241,96 @@ public MpnsCredential mpnsCredential() { } /** - * Set the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Set the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @param mpnsCredential the mpnsCredential value to set. * @return the PnsCredentialsResourceInner object itself. */ public PnsCredentialsResourceInner withMpnsCredential(MpnsCredential mpnsCredential) { if (this.innerProperties() == null) { - this.innerProperties = new PnsCredentialsProperties(); + this.innerProperties = new PnsCredentials(); } this.innerProperties().withMpnsCredential(mpnsCredential); return this; } /** - * Get the admCredential property: The AdmCredential of the created NotificationHub. - * - * @return the admCredential value. + * Get the wnsCredential property: Description of a NotificationHub WnsCredential. + * + * @return the wnsCredential value. */ - public AdmCredential admCredential() { - return this.innerProperties() == null ? null : this.innerProperties().admCredential(); + public WnsCredential wnsCredential() { + return this.innerProperties() == null ? null : this.innerProperties().wnsCredential(); } /** - * Set the admCredential property: The AdmCredential of the created NotificationHub. - * - * @param admCredential the admCredential value to set. + * Set the wnsCredential property: Description of a NotificationHub WnsCredential. + * + * @param wnsCredential the wnsCredential value to set. * @return the PnsCredentialsResourceInner object itself. */ - public PnsCredentialsResourceInner withAdmCredential(AdmCredential admCredential) { + public PnsCredentialsResourceInner withWnsCredential(WnsCredential wnsCredential) { if (this.innerProperties() == null) { - this.innerProperties = new PnsCredentialsProperties(); + this.innerProperties = new PnsCredentials(); } - this.innerProperties().withAdmCredential(admCredential); + this.innerProperties().withWnsCredential(wnsCredential); return this; } /** - * Get the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @return the baiduCredential value. + * Get the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. */ - public BaiduCredential baiduCredential() { - return this.innerProperties() == null ? null : this.innerProperties().baiduCredential(); + public XiaomiCredential xiaomiCredential() { + return this.innerProperties() == null ? null : this.innerProperties().xiaomiCredential(); } /** - * Set the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @param baiduCredential the baiduCredential value to set. + * Set the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @param xiaomiCredential the xiaomiCredential value to set. * @return the PnsCredentialsResourceInner object itself. */ - public PnsCredentialsResourceInner withBaiduCredential(BaiduCredential baiduCredential) { + public PnsCredentialsResourceInner withXiaomiCredential(XiaomiCredential xiaomiCredential) { if (this.innerProperties() == null) { - this.innerProperties = new PnsCredentialsProperties(); + this.innerProperties = new PnsCredentials(); } - this.innerProperties().withBaiduCredential(baiduCredential); + this.innerProperties().withXiaomiCredential(xiaomiCredential); + return this; + } + + /** + * Get the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + public FcmV1Credential fcmV1Credential() { + return this.innerProperties() == null ? null : this.innerProperties().fcmV1Credential(); + } + + /** + * Set the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @param fcmV1Credential the fcmV1Credential value to set. + * @return the PnsCredentialsResourceInner object itself. + */ + public PnsCredentialsResourceInner withFcmV1Credential(FcmV1Credential fcmV1Credential) { + if (this.innerProperties() == null) { + this.innerProperties = new PnsCredentials(); + } + this.innerProperties().withFcmV1Credential(fcmV1Credential); return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() != null) { innerProperties().validate(); } - if (sku() != null) { - sku().validate(); - } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateEndpointConnectionResourceInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateEndpointConnectionResourceInner.java new file mode 100644 index 0000000000000..73bd75517ebb5 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateEndpointConnectionResourceInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ +@Fluent +public final class PrivateEndpointConnectionResourceInner extends ProxyResource { + /* + * Private Endpoint Connection properties. + */ + @JsonProperty(value = "properties") + private PrivateEndpointConnectionProperties properties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of PrivateEndpointConnectionResourceInner class. + */ + public PrivateEndpointConnectionResourceInner() { + } + + /** + * Get the properties property: Private Endpoint Connection properties. + * + * @return the properties value. + */ + public PrivateEndpointConnectionProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Private Endpoint Connection properties. + * + * @param properties the properties value to set. + * @return the PrivateEndpointConnectionResourceInner object itself. + */ + public PrivateEndpointConnectionResourceInner withProperties(PrivateEndpointConnectionProperties properties) { + this.properties = properties; + return this; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateLinkResourceInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateLinkResourceInner.java new file mode 100644 index 0000000000000..1284c8d685c23 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/PrivateLinkResourceInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * A Private Link Arm Resource. + */ +@Fluent +public final class PrivateLinkResourceInner extends ProxyResource { + /* + * Represents properties of Private Link Resource. + */ + @JsonProperty(value = "properties") + private PrivateLinkResourceProperties properties; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of PrivateLinkResourceInner class. + */ + public PrivateLinkResourceInner() { + } + + /** + * Get the properties property: Represents properties of Private Link Resource. + * + * @return the properties value. + */ + public PrivateLinkResourceProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Represents properties of Private Link Resource. + * + * @param properties the properties value to set. + * @return the PrivateLinkResourceInner object itself. + */ + public PrivateLinkResourceInner withProperties(PrivateLinkResourceProperties properties) { + this.properties = properties; + return this; + } + + /** + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ResourceListKeysInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ResourceListKeysInner.java index 7ad697fcd00a8..81ecc7292b5eb 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ResourceListKeysInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/ResourceListKeysInner.java @@ -4,69 +4,64 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Namespace/NotificationHub Connection String. */ -@Fluent +/** + * Response for the POST request that returns Namespace or NotificationHub access keys (connection strings). + */ +@Immutable public final class ResourceListKeysInner { - @JsonIgnore private final ClientLogger logger = new ClientLogger(ResourceListKeysInner.class); - /* - * PrimaryConnectionString of the AuthorizationRule. + * Gets or sets primaryConnectionString of the AuthorizationRule. */ - @JsonProperty(value = "primaryConnectionString") + @JsonProperty(value = "primaryConnectionString", access = JsonProperty.Access.WRITE_ONLY) private String primaryConnectionString; /* - * SecondaryConnectionString of the created AuthorizationRule + * Gets or sets secondaryConnectionString of the created + * AuthorizationRule */ - @JsonProperty(value = "secondaryConnectionString") + @JsonProperty(value = "secondaryConnectionString", access = JsonProperty.Access.WRITE_ONLY) private String secondaryConnectionString; /* - * PrimaryKey of the created AuthorizationRule. + * Gets or sets primaryKey of the created AuthorizationRule. */ - @JsonProperty(value = "primaryKey") + @JsonProperty(value = "primaryKey", access = JsonProperty.Access.WRITE_ONLY) private String primaryKey; /* - * SecondaryKey of the created AuthorizationRule + * Gets or sets secondaryKey of the created AuthorizationRule */ - @JsonProperty(value = "secondaryKey") + @JsonProperty(value = "secondaryKey", access = JsonProperty.Access.WRITE_ONLY) private String secondaryKey; /* - * KeyName of the created AuthorizationRule + * Gets or sets keyName of the created AuthorizationRule */ - @JsonProperty(value = "keyName") + @JsonProperty(value = "keyName", access = JsonProperty.Access.WRITE_ONLY) private String keyName; /** - * Get the primaryConnectionString property: PrimaryConnectionString of the AuthorizationRule. - * - * @return the primaryConnectionString value. + * Creates an instance of ResourceListKeysInner class. */ - public String primaryConnectionString() { - return this.primaryConnectionString; + public ResourceListKeysInner() { } /** - * Set the primaryConnectionString property: PrimaryConnectionString of the AuthorizationRule. - * - * @param primaryConnectionString the primaryConnectionString value to set. - * @return the ResourceListKeysInner object itself. + * Get the primaryConnectionString property: Gets or sets primaryConnectionString of the AuthorizationRule. + * + * @return the primaryConnectionString value. */ - public ResourceListKeysInner withPrimaryConnectionString(String primaryConnectionString) { - this.primaryConnectionString = primaryConnectionString; - return this; + public String primaryConnectionString() { + return this.primaryConnectionString; } /** - * Get the secondaryConnectionString property: SecondaryConnectionString of the created AuthorizationRule. - * + * Get the secondaryConnectionString property: Gets or sets secondaryConnectionString of the created + * AuthorizationRule. + * * @return the secondaryConnectionString value. */ public String secondaryConnectionString() { @@ -74,19 +69,8 @@ public String secondaryConnectionString() { } /** - * Set the secondaryConnectionString property: SecondaryConnectionString of the created AuthorizationRule. - * - * @param secondaryConnectionString the secondaryConnectionString value to set. - * @return the ResourceListKeysInner object itself. - */ - public ResourceListKeysInner withSecondaryConnectionString(String secondaryConnectionString) { - this.secondaryConnectionString = secondaryConnectionString; - return this; - } - - /** - * Get the primaryKey property: PrimaryKey of the created AuthorizationRule. - * + * Get the primaryKey property: Gets or sets primaryKey of the created AuthorizationRule. + * * @return the primaryKey value. */ public String primaryKey() { @@ -94,19 +78,8 @@ public String primaryKey() { } /** - * Set the primaryKey property: PrimaryKey of the created AuthorizationRule. - * - * @param primaryKey the primaryKey value to set. - * @return the ResourceListKeysInner object itself. - */ - public ResourceListKeysInner withPrimaryKey(String primaryKey) { - this.primaryKey = primaryKey; - return this; - } - - /** - * Get the secondaryKey property: SecondaryKey of the created AuthorizationRule. - * + * Get the secondaryKey property: Gets or sets secondaryKey of the created AuthorizationRule. + * * @return the secondaryKey value. */ public String secondaryKey() { @@ -114,39 +87,17 @@ public String secondaryKey() { } /** - * Set the secondaryKey property: SecondaryKey of the created AuthorizationRule. - * - * @param secondaryKey the secondaryKey value to set. - * @return the ResourceListKeysInner object itself. - */ - public ResourceListKeysInner withSecondaryKey(String secondaryKey) { - this.secondaryKey = secondaryKey; - return this; - } - - /** - * Get the keyName property: KeyName of the created AuthorizationRule. - * + * Get the keyName property: Gets or sets keyName of the created AuthorizationRule. + * * @return the keyName value. */ public String keyName() { return this.keyName; } - /** - * Set the keyName property: KeyName of the created AuthorizationRule. - * - * @param keyName the keyName value to set. - * @return the ResourceListKeysInner object itself. - */ - public ResourceListKeysInner withKeyName(String keyName) { - this.keyName = keyName; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleProperties.java index dc180123e3763..d1c81aec65424 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleProperties.java @@ -7,74 +7,80 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.models.AccessRights; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; import java.util.List; -/** SharedAccessAuthorizationRule properties. */ +/** + * SharedAccessAuthorizationRule properties. + */ @Fluent public final class SharedAccessAuthorizationRuleProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(SharedAccessAuthorizationRuleProperties.class); - /* - * The rights associated with the rule. + * Gets or sets the rights associated with the rule. */ - @JsonProperty(value = "rights") + @JsonProperty(value = "rights", required = true) private List rights; /* - * A base64-encoded 256-bit primary key for signing and validating the SAS - * token. + * Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. */ - @JsonProperty(value = "primaryKey", access = JsonProperty.Access.WRITE_ONLY) + @JsonProperty(value = "primaryKey") private String primaryKey; /* - * A base64-encoded 256-bit primary key for signing and validating the SAS - * token. + * Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. */ - @JsonProperty(value = "secondaryKey", access = JsonProperty.Access.WRITE_ONLY) + @JsonProperty(value = "secondaryKey") private String secondaryKey; /* - * A string that describes the authorization rule. + * Gets a string that describes the authorization rule. */ @JsonProperty(value = "keyName", access = JsonProperty.Access.WRITE_ONLY) private String keyName; /* - * A string that describes the claim type + * Gets the last modified time for this rule */ - @JsonProperty(value = "claimType", access = JsonProperty.Access.WRITE_ONLY) - private String claimType; + @JsonProperty(value = "modifiedTime", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime modifiedTime; /* - * A string that describes the claim value + * Gets the created time for this rule */ - @JsonProperty(value = "claimValue", access = JsonProperty.Access.WRITE_ONLY) - private String claimValue; + @JsonProperty(value = "createdTime", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime createdTime; /* - * The last modified time for this rule + * Gets a string that describes the claim type */ - @JsonProperty(value = "modifiedTime", access = JsonProperty.Access.WRITE_ONLY) - private String modifiedTime; + @JsonProperty(value = "claimType", access = JsonProperty.Access.WRITE_ONLY) + private String claimType; /* - * The created time for this rule + * Gets a string that describes the claim value */ - @JsonProperty(value = "createdTime", access = JsonProperty.Access.WRITE_ONLY) - private String createdTime; + @JsonProperty(value = "claimValue", access = JsonProperty.Access.WRITE_ONLY) + private String claimValue; /* - * The revision number for the rule + * Gets the revision number for the rule */ @JsonProperty(value = "revision", access = JsonProperty.Access.WRITE_ONLY) private Integer revision; /** - * Get the rights property: The rights associated with the rule. - * + * Creates an instance of SharedAccessAuthorizationRuleProperties class. + */ + public SharedAccessAuthorizationRuleProperties() { + } + + /** + * Get the rights property: Gets or sets the rights associated with the rule. + * * @return the rights value. */ public List rights() { @@ -82,8 +88,8 @@ public List rights() { } /** - * Set the rights property: The rights associated with the rule. - * + * Set the rights property: Gets or sets the rights associated with the rule. + * * @param rights the rights value to set. * @return the SharedAccessAuthorizationRuleProperties object itself. */ @@ -93,8 +99,9 @@ public SharedAccessAuthorizationRuleProperties withRights(List rig } /** - * Get the primaryKey property: A base64-encoded 256-bit primary key for signing and validating the SAS token. - * + * Get the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * * @return the primaryKey value. */ public String primaryKey() { @@ -102,8 +109,21 @@ public String primaryKey() { } /** - * Get the secondaryKey property: A base64-encoded 256-bit primary key for signing and validating the SAS token. - * + * Set the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * + * @param primaryKey the primaryKey value to set. + * @return the SharedAccessAuthorizationRuleProperties object itself. + */ + public SharedAccessAuthorizationRuleProperties withPrimaryKey(String primaryKey) { + this.primaryKey = primaryKey; + return this; + } + + /** + * Get the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * * @return the secondaryKey value. */ public String secondaryKey() { @@ -111,8 +131,20 @@ public String secondaryKey() { } /** - * Get the keyName property: A string that describes the authorization rule. - * + * Set the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * + * @param secondaryKey the secondaryKey value to set. + * @return the SharedAccessAuthorizationRuleProperties object itself. + */ + public SharedAccessAuthorizationRuleProperties withSecondaryKey(String secondaryKey) { + this.secondaryKey = secondaryKey; + return this; + } + + /** + * Get the keyName property: Gets a string that describes the authorization rule. + * * @return the keyName value. */ public String keyName() { @@ -120,44 +152,44 @@ public String keyName() { } /** - * Get the claimType property: A string that describes the claim type. - * - * @return the claimType value. + * Get the modifiedTime property: Gets the last modified time for this rule. + * + * @return the modifiedTime value. */ - public String claimType() { - return this.claimType; + public OffsetDateTime modifiedTime() { + return this.modifiedTime; } /** - * Get the claimValue property: A string that describes the claim value. - * - * @return the claimValue value. + * Get the createdTime property: Gets the created time for this rule. + * + * @return the createdTime value. */ - public String claimValue() { - return this.claimValue; + public OffsetDateTime createdTime() { + return this.createdTime; } /** - * Get the modifiedTime property: The last modified time for this rule. - * - * @return the modifiedTime value. + * Get the claimType property: Gets a string that describes the claim type. + * + * @return the claimType value. */ - public String modifiedTime() { - return this.modifiedTime; + public String claimType() { + return this.claimType; } /** - * Get the createdTime property: The created time for this rule. - * - * @return the createdTime value. + * Get the claimValue property: Gets a string that describes the claim value. + * + * @return the claimValue value. */ - public String createdTime() { - return this.createdTime; + public String claimValue() { + return this.claimValue; } /** - * Get the revision property: The revision number for the rule. - * + * Get the revision property: Gets the revision number for the rule. + * * @return the revision value. */ public Integer revision() { @@ -166,9 +198,15 @@ public Integer revision() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { + if (rights() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property rights in model SharedAccessAuthorizationRuleProperties")); + } } + + private static final ClientLogger LOGGER = new ClientLogger(SharedAccessAuthorizationRuleProperties.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleResourceInner.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleResourceInner.java index c0c3bbbe4f9ee..12eefd02e020c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleResourceInner.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/SharedAccessAuthorizationRuleResourceInner.java @@ -5,35 +5,54 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.models.AccessRights; -import com.azure.resourcemanager.notificationhubs.models.Sku; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; import java.util.List; import java.util.Map; -/** Description of a Namespace AuthorizationRules. */ +/** + * Response for POST requests that return single SharedAccessAuthorizationRule. + */ @Fluent -public final class SharedAccessAuthorizationRuleResourceInner extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(SharedAccessAuthorizationRuleResourceInner.class); - +public final class SharedAccessAuthorizationRuleResourceInner extends ProxyResource { /* - * Properties of the Namespace AuthorizationRule. + * SharedAccessAuthorizationRule properties. */ @JsonProperty(value = "properties") private SharedAccessAuthorizationRuleProperties innerProperties; /* - * The sku of the created namespace + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "location") + private String location; + + /* + * Deprecated - only for compatibility. + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; + + /* + * Azure Resource Manager metadata containing createdBy and modifiedBy information. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /** + * Creates an instance of SharedAccessAuthorizationRuleResourceInner class. */ - @JsonProperty(value = "sku") - private Sku sku; + public SharedAccessAuthorizationRuleResourceInner() { + } /** - * Get the innerProperties property: Properties of the Namespace AuthorizationRule. - * + * Get the innerProperties property: SharedAccessAuthorizationRule properties. + * * @return the innerProperties value. */ private SharedAccessAuthorizationRuleProperties innerProperties() { @@ -41,42 +60,57 @@ private SharedAccessAuthorizationRuleProperties innerProperties() { } /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. + * Get the location property: Deprecated - only for compatibility. + * + * @return the location value. */ - public Sku sku() { - return this.sku; + public String location() { + return this.location; } /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. + * Set the location property: Deprecated - only for compatibility. + * + * @param location the location value to set. * @return the SharedAccessAuthorizationRuleResourceInner object itself. */ - public SharedAccessAuthorizationRuleResourceInner withSku(Sku sku) { - this.sku = sku; + public SharedAccessAuthorizationRuleResourceInner withLocation(String location) { + this.location = location; return this; } - /** {@inheritDoc} */ - @Override - public SharedAccessAuthorizationRuleResourceInner withLocation(String location) { - super.withLocation(location); - return this; + /** + * Get the tags property: Deprecated - only for compatibility. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; } - /** {@inheritDoc} */ - @Override + /** + * Set the tags property: Deprecated - only for compatibility. + * + * @param tags the tags value to set. + * @return the SharedAccessAuthorizationRuleResourceInner object itself. + */ public SharedAccessAuthorizationRuleResourceInner withTags(Map tags) { - super.withTags(tags); + this.tags = tags; return this; } /** - * Get the rights property: The rights associated with the rule. - * + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the rights property: Gets or sets the rights associated with the rule. + * * @return the rights value. */ public List rights() { @@ -84,8 +118,8 @@ public List rights() { } /** - * Set the rights property: The rights associated with the rule. - * + * Set the rights property: Gets or sets the rights associated with the rule. + * * @param rights the rights value to set. * @return the SharedAccessAuthorizationRuleResourceInner object itself. */ @@ -98,8 +132,9 @@ public SharedAccessAuthorizationRuleResourceInner withRights(List } /** - * Get the primaryKey property: A base64-encoded 256-bit primary key for signing and validating the SAS token. - * + * Get the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * * @return the primaryKey value. */ public String primaryKey() { @@ -107,8 +142,24 @@ public String primaryKey() { } /** - * Get the secondaryKey property: A base64-encoded 256-bit primary key for signing and validating the SAS token. - * + * Set the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * + * @param primaryKey the primaryKey value to set. + * @return the SharedAccessAuthorizationRuleResourceInner object itself. + */ + public SharedAccessAuthorizationRuleResourceInner withPrimaryKey(String primaryKey) { + if (this.innerProperties() == null) { + this.innerProperties = new SharedAccessAuthorizationRuleProperties(); + } + this.innerProperties().withPrimaryKey(primaryKey); + return this; + } + + /** + * Get the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * * @return the secondaryKey value. */ public String secondaryKey() { @@ -116,8 +167,23 @@ public String secondaryKey() { } /** - * Get the keyName property: A string that describes the authorization rule. - * + * Set the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * + * @param secondaryKey the secondaryKey value to set. + * @return the SharedAccessAuthorizationRuleResourceInner object itself. + */ + public SharedAccessAuthorizationRuleResourceInner withSecondaryKey(String secondaryKey) { + if (this.innerProperties() == null) { + this.innerProperties = new SharedAccessAuthorizationRuleProperties(); + } + this.innerProperties().withSecondaryKey(secondaryKey); + return this; + } + + /** + * Get the keyName property: Gets a string that describes the authorization rule. + * * @return the keyName value. */ public String keyName() { @@ -125,44 +191,44 @@ public String keyName() { } /** - * Get the claimType property: A string that describes the claim type. - * - * @return the claimType value. + * Get the modifiedTime property: Gets the last modified time for this rule. + * + * @return the modifiedTime value. */ - public String claimType() { - return this.innerProperties() == null ? null : this.innerProperties().claimType(); + public OffsetDateTime modifiedTime() { + return this.innerProperties() == null ? null : this.innerProperties().modifiedTime(); } /** - * Get the claimValue property: A string that describes the claim value. - * - * @return the claimValue value. + * Get the createdTime property: Gets the created time for this rule. + * + * @return the createdTime value. */ - public String claimValue() { - return this.innerProperties() == null ? null : this.innerProperties().claimValue(); + public OffsetDateTime createdTime() { + return this.innerProperties() == null ? null : this.innerProperties().createdTime(); } /** - * Get the modifiedTime property: The last modified time for this rule. - * - * @return the modifiedTime value. + * Get the claimType property: Gets a string that describes the claim type. + * + * @return the claimType value. */ - public String modifiedTime() { - return this.innerProperties() == null ? null : this.innerProperties().modifiedTime(); + public String claimType() { + return this.innerProperties() == null ? null : this.innerProperties().claimType(); } /** - * Get the createdTime property: The created time for this rule. - * - * @return the createdTime value. + * Get the claimValue property: Gets a string that describes the claim value. + * + * @return the claimValue value. */ - public String createdTime() { - return this.innerProperties() == null ? null : this.innerProperties().createdTime(); + public String claimValue() { + return this.innerProperties() == null ? null : this.innerProperties().claimValue(); } /** - * Get the revision property: The revision number for the rule. - * + * Get the revision property: Gets the revision number for the rule. + * * @return the revision value. */ public Integer revision() { @@ -171,15 +237,12 @@ public Integer revision() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (innerProperties() != null) { innerProperties().validate(); } - if (sku() != null) { - sku().validate(); - } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/WnsCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/WnsCredentialProperties.java index 2543b327d4962..afca3d42885ab 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/WnsCredentialProperties.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/WnsCredentialProperties.java @@ -5,36 +5,52 @@ package com.azure.resourcemanager.notificationhubs.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub WnsCredential. */ +/** + * Description of a NotificationHub WnsCredential. + */ @Fluent public final class WnsCredentialProperties { - @JsonIgnore private final ClientLogger logger = new ClientLogger(WnsCredentialProperties.class); - /* - * The package ID for this credential. + * Gets or sets the package ID for this credential. */ @JsonProperty(value = "packageSid") private String packageSid; /* - * The secret key. + * Gets or sets the secret key. */ @JsonProperty(value = "secretKey") private String secretKey; /* - * The Windows Live endpoint. + * Gets or sets the Windows Live endpoint. */ @JsonProperty(value = "windowsLiveEndpoint") private String windowsLiveEndpoint; + /* + * Ges or sets the WNS Certificate Key. + */ + @JsonProperty(value = "certificateKey") + private String certificateKey; + + /* + * Gets or sets the WNS Certificate. + */ + @JsonProperty(value = "wnsCertificate") + private String wnsCertificate; + + /** + * Creates an instance of WnsCredentialProperties class. + */ + public WnsCredentialProperties() { + } + /** - * Get the packageSid property: The package ID for this credential. - * + * Get the packageSid property: Gets or sets the package ID for this credential. + * * @return the packageSid value. */ public String packageSid() { @@ -42,8 +58,8 @@ public String packageSid() { } /** - * Set the packageSid property: The package ID for this credential. - * + * Set the packageSid property: Gets or sets the package ID for this credential. + * * @param packageSid the packageSid value to set. * @return the WnsCredentialProperties object itself. */ @@ -53,8 +69,8 @@ public WnsCredentialProperties withPackageSid(String packageSid) { } /** - * Get the secretKey property: The secret key. - * + * Get the secretKey property: Gets or sets the secret key. + * * @return the secretKey value. */ public String secretKey() { @@ -62,8 +78,8 @@ public String secretKey() { } /** - * Set the secretKey property: The secret key. - * + * Set the secretKey property: Gets or sets the secret key. + * * @param secretKey the secretKey value to set. * @return the WnsCredentialProperties object itself. */ @@ -73,8 +89,8 @@ public WnsCredentialProperties withSecretKey(String secretKey) { } /** - * Get the windowsLiveEndpoint property: The Windows Live endpoint. - * + * Get the windowsLiveEndpoint property: Gets or sets the Windows Live endpoint. + * * @return the windowsLiveEndpoint value. */ public String windowsLiveEndpoint() { @@ -82,8 +98,8 @@ public String windowsLiveEndpoint() { } /** - * Set the windowsLiveEndpoint property: The Windows Live endpoint. - * + * Set the windowsLiveEndpoint property: Gets or sets the Windows Live endpoint. + * * @param windowsLiveEndpoint the windowsLiveEndpoint value to set. * @return the WnsCredentialProperties object itself. */ @@ -92,9 +108,49 @@ public WnsCredentialProperties withWindowsLiveEndpoint(String windowsLiveEndpoin return this; } + /** + * Get the certificateKey property: Ges or sets the WNS Certificate Key. + * + * @return the certificateKey value. + */ + public String certificateKey() { + return this.certificateKey; + } + + /** + * Set the certificateKey property: Ges or sets the WNS Certificate Key. + * + * @param certificateKey the certificateKey value to set. + * @return the WnsCredentialProperties object itself. + */ + public WnsCredentialProperties withCertificateKey(String certificateKey) { + this.certificateKey = certificateKey; + return this; + } + + /** + * Get the wnsCertificate property: Gets or sets the WNS Certificate. + * + * @return the wnsCertificate value. + */ + public String wnsCertificate() { + return this.wnsCertificate; + } + + /** + * Set the wnsCertificate property: Gets or sets the WNS Certificate. + * + * @param wnsCertificate the wnsCertificate value to set. + * @return the WnsCredentialProperties object itself. + */ + public WnsCredentialProperties withWnsCertificate(String wnsCertificate) { + this.wnsCertificate = wnsCertificate; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/XiaomiCredentialProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/XiaomiCredentialProperties.java new file mode 100644 index 0000000000000..fc54fdf2d03db --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/XiaomiCredentialProperties.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Description of a NotificationHub XiaomiCredentialProperties. + */ +@Fluent +public final class XiaomiCredentialProperties { + /* + * Gets or sets app secret. + */ + @JsonProperty(value = "appSecret") + private String appSecret; + + /* + * Gets or sets xiaomi service endpoint. + */ + @JsonProperty(value = "endpoint") + private String endpoint; + + /** + * Creates an instance of XiaomiCredentialProperties class. + */ + public XiaomiCredentialProperties() { + } + + /** + * Get the appSecret property: Gets or sets app secret. + * + * @return the appSecret value. + */ + public String appSecret() { + return this.appSecret; + } + + /** + * Set the appSecret property: Gets or sets app secret. + * + * @param appSecret the appSecret value to set. + * @return the XiaomiCredentialProperties object itself. + */ + public XiaomiCredentialProperties withAppSecret(String appSecret) { + this.appSecret = appSecret; + return this; + } + + /** + * Get the endpoint property: Gets or sets xiaomi service endpoint. + * + * @return the endpoint value. + */ + public String endpoint() { + return this.endpoint; + } + + /** + * Set the endpoint property: Gets or sets xiaomi service endpoint. + * + * @param endpoint the endpoint value to set. + * @return the XiaomiCredentialProperties object itself. + */ + public XiaomiCredentialProperties withEndpoint(String endpoint) { + this.endpoint = endpoint; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/package-info.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/package-info.java index 19cb916a17da6..db55bc8657e5f 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/package-info.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the inner data models for NotificationHubsManagementClient. Azure NotificationHub client. */ +/** + * Package containing the inner data models for NotificationHubsRPClient. + * Microsoft Notification Hubs Resource Provider REST API. + */ package com.azure.resourcemanager.notificationhubs.fluent.models; diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/package-info.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/package-info.java index db5c8de530468..4d9193cbc5a32 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/package-info.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/fluent/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the service clients for NotificationHubsManagementClient. Azure NotificationHub client. */ +/** + * Package containing the service clients for NotificationHubsRPClient. + * Microsoft Notification Hubs Resource Provider REST API. + */ package com.azure.resourcemanager.notificationhubs.fluent; diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/CheckAvailabilityResultImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/CheckAvailabilityResultImpl.java index 2241049176363..8d22da60d7eb0 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/CheckAvailabilityResultImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/CheckAvailabilityResultImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.notificationhubs.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityResult; import com.azure.resourcemanager.notificationhubs.models.Sku; @@ -15,8 +16,7 @@ public final class CheckAvailabilityResultImpl implements CheckAvailabilityResul private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - CheckAvailabilityResultImpl( - CheckAvailabilityResultInner innerObject, + CheckAvailabilityResultImpl(CheckAvailabilityResultInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -34,6 +34,10 @@ public String type() { return this.innerModel().type(); } + public Boolean isAvailiable() { + return this.innerModel().isAvailiable(); + } + public String location() { return this.innerModel().location(); } @@ -47,14 +51,14 @@ public Map tags() { } } - public Boolean isAvailiable() { - return this.innerModel().isAvailiable(); - } - public Sku sku() { return this.innerModel().sku(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public CheckAvailabilityResultInner innerModel() { return this.innerObject; } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/DebugSendResponseImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/DebugSendResponseImpl.java index bcf2fa40e4d2c..f4650efc8e956 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/DebugSendResponseImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/DebugSendResponseImpl.java @@ -4,10 +4,12 @@ package com.azure.resourcemanager.notificationhubs.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResponseInner; import com.azure.resourcemanager.notificationhubs.models.DebugSendResponse; -import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.RegistrationResult; import java.util.Collections; +import java.util.List; import java.util.Map; public final class DebugSendResponseImpl implements DebugSendResponse { @@ -15,8 +17,7 @@ public final class DebugSendResponseImpl implements DebugSendResponse { private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - DebugSendResponseImpl( - DebugSendResponseInner innerObject, + DebugSendResponseImpl(DebugSendResponseInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -47,20 +48,25 @@ public Map tags() { } } - public Sku sku() { - return this.innerModel().sku(); + public SystemData systemData() { + return this.innerModel().systemData(); } - public Float success() { + public Long success() { return this.innerModel().success(); } - public Float failure() { + public Long failure() { return this.innerModel().failure(); } - public Object results() { - return this.innerModel().results(); + public List results() { + List inner = this.innerModel().results(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } } public DebugSendResponseInner innerModel() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespaceResourceImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespaceResourceImpl.java index cf7c11dbcab06..09d71024d767c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespaceResourceImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespaceResourceImpl.java @@ -4,17 +4,31 @@ package com.azure.resourcemanager.notificationhubs.implementation; +import com.azure.core.http.rest.Response; import com.azure.core.management.Region; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; +import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties; import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner; -import com.azure.resourcemanager.notificationhubs.models.NamespaceCreateOrUpdateParameters; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentials; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; import com.azure.resourcemanager.notificationhubs.models.NamespacePatchParameters; import com.azure.resourcemanager.notificationhubs.models.NamespaceResource; +import com.azure.resourcemanager.notificationhubs.models.NamespaceStatus; import com.azure.resourcemanager.notificationhubs.models.NamespaceType; +import com.azure.resourcemanager.notificationhubs.models.NetworkAcls; +import com.azure.resourcemanager.notificationhubs.models.OperationProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PnsCredentialsResource; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResource; +import com.azure.resourcemanager.notificationhubs.models.PublicNetworkAccess; +import com.azure.resourcemanager.notificationhubs.models.ReplicationRegion; import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.ZoneRedundancyPreference; import java.time.OffsetDateTime; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public final class NamespaceResourceImpl implements NamespaceResource, NamespaceResource.Definition, NamespaceResource.Update { @@ -51,14 +65,34 @@ public Sku sku() { return this.innerModel().sku(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String namePropertiesName() { return this.innerModel().namePropertiesName(); } - public String provisioningState() { + public OperationProvisioningState provisioningState() { return this.innerModel().provisioningState(); } + public NamespaceStatus status() { + return this.innerModel().status(); + } + + public Boolean enabled() { + return this.innerModel().enabled(); + } + + public Boolean critical() { + return this.innerModel().critical(); + } + + public String subscriptionId() { + return this.innerModel().subscriptionId(); + } + public String region() { return this.innerModel().region(); } @@ -67,10 +101,6 @@ public String metricId() { return this.innerModel().metricId(); } - public String status() { - return this.innerModel().status(); - } - public OffsetDateTime createdAt() { return this.innerModel().createdAt(); } @@ -79,32 +109,55 @@ public OffsetDateTime updatedAt() { return this.innerModel().updatedAt(); } - public String serviceBusEndpoint() { - return this.innerModel().serviceBusEndpoint(); + public NamespaceType namespaceType() { + return this.innerModel().namespaceType(); } - public String subscriptionId() { - return this.innerModel().subscriptionId(); + public ReplicationRegion replicationRegion() { + return this.innerModel().replicationRegion(); } - public String scaleUnit() { - return this.innerModel().scaleUnit(); + public ZoneRedundancyPreference zoneRedundancy() { + return this.innerModel().zoneRedundancy(); } - public Boolean enabled() { - return this.innerModel().enabled(); + public NetworkAcls networkAcls() { + return this.innerModel().networkAcls(); } - public Boolean critical() { - return this.innerModel().critical(); + public PnsCredentials pnsCredentials() { + return this.innerModel().pnsCredentials(); + } + + public String serviceBusEndpoint() { + return this.innerModel().serviceBusEndpoint(); + } + + public List privateEndpointConnections() { + List inner = this.innerModel().privateEndpointConnections(); + if (inner != null) { + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new PrivateEndpointConnectionResourceImpl(inner1, this.manager())) + .collect(Collectors.toList())); + } else { + return Collections.emptyList(); + } + } + + public String scaleUnit() { + return this.innerModel().scaleUnit(); } public String dataCenter() { return this.innerModel().dataCenter(); } - public NamespaceType namespaceType() { - return this.innerModel().namespaceType(); + public PublicNetworkAccess publicNetworkAccess() { + return this.innerModel().publicNetworkAccess(); + } + + public String resourceGroupName() { + return resourceGroupName; } public NamespaceResourceInner innerModel() { @@ -119,8 +172,6 @@ private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manag private String namespaceName; - private NamespaceCreateOrUpdateParameters createParameters; - private NamespacePatchParameters updateParameters; public NamespaceResourceImpl withExistingResourceGroup(String resourceGroupName) { @@ -129,31 +180,22 @@ public NamespaceResourceImpl withExistingResourceGroup(String resourceGroupName) } public NamespaceResource create() { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .createOrUpdateWithResponse(resourceGroupName, namespaceName, createParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNamespaces().createOrUpdate(resourceGroupName, + namespaceName, this.innerModel(), Context.NONE); return this; } public NamespaceResource create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .createOrUpdateWithResponse(resourceGroupName, namespaceName, createParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNamespaces().createOrUpdate(resourceGroupName, + namespaceName, this.innerModel(), context); return this; } - NamespaceResourceImpl( - String name, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + NamespaceResourceImpl(String name, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = new NamespaceResourceInner(); this.serviceManager = serviceManager; this.namespaceName = name; - this.createParameters = new NamespaceCreateOrUpdateParameters(); } public NamespaceResourceImpl update() { @@ -162,146 +204,127 @@ public NamespaceResourceImpl update() { } public NamespaceResource apply() { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .patchWithResponse(resourceGroupName, namespaceName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNamespaces() + .updateWithResponse(resourceGroupName, namespaceName, updateParameters, Context.NONE).getValue(); return this; } public NamespaceResource apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .patchWithResponse(resourceGroupName, namespaceName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNamespaces() + .updateWithResponse(resourceGroupName, namespaceName, updateParameters, context).getValue(); return this; } - NamespaceResourceImpl( - NamespaceResourceInner innerObject, + NamespaceResourceImpl(NamespaceResourceInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.namespaceName = Utils.getValueFromIdByName(innerObject.id(), "namespaces"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.namespaceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "namespaces"); } public NamespaceResource refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .getByResourceGroupWithResponse(resourceGroupName, namespaceName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNamespaces() + .getByResourceGroupWithResponse(resourceGroupName, namespaceName, Context.NONE).getValue(); return this; } public NamespaceResource refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .getByResourceGroupWithResponse(resourceGroupName, namespaceName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNamespaces() + .getByResourceGroupWithResponse(resourceGroupName, namespaceName, context).getValue(); return this; } - public NamespaceResourceImpl withLocation(Region location) { - this.createParameters.withLocation(location.toString()); + public Response getPnsCredentialsWithResponse(Context context) { + return serviceManager.namespaces().getPnsCredentialsWithResponse(resourceGroupName, namespaceName, context); + } + + public PnsCredentialsResource getPnsCredentials() { + return serviceManager.namespaces().getPnsCredentials(resourceGroupName, namespaceName); + } + + public NamespaceResourceImpl withRegion(Region location) { + this.innerModel().withLocation(location.toString()); return this; } - public NamespaceResourceImpl withLocation(String location) { - this.createParameters.withLocation(location); + public NamespaceResourceImpl withRegion(String location) { + this.innerModel().withLocation(location); return this; } - public NamespaceResourceImpl withTags(Map tags) { + public NamespaceResourceImpl withSku(Sku sku) { if (isInCreateMode()) { - this.createParameters.withTags(tags); + this.innerModel().withSku(sku); return this; } else { - this.updateParameters.withTags(tags); + this.updateParameters.withSku(sku); return this; } } - public NamespaceResourceImpl withSku(Sku sku) { + public NamespaceResourceImpl withTags(Map tags) { if (isInCreateMode()) { - this.createParameters.withSku(sku); + this.innerModel().withTags(tags); return this; } else { - this.updateParameters.withSku(sku); + this.updateParameters.withTags(tags); return this; } } - public NamespaceResourceImpl withNamePropertiesName(String namePropertiesName) { - this.createParameters.withNamePropertiesName(namePropertiesName); - return this; - } - - public NamespaceResourceImpl withProvisioningState(String provisioningState) { - this.createParameters.withProvisioningState(provisioningState); + public NamespaceResourceImpl withProvisioningState(OperationProvisioningState provisioningState) { + this.innerModel().withProvisioningState(provisioningState); return this; } - public NamespaceResourceImpl withRegion(String region) { - this.createParameters.withRegion(region); + public NamespaceResourceImpl withStatus(NamespaceStatus status) { + this.innerModel().withStatus(status); return this; } - public NamespaceResourceImpl withStatus(String status) { - this.createParameters.withStatus(status); + public NamespaceResourceImpl withNamespaceType(NamespaceType namespaceType) { + this.innerModel().withNamespaceType(namespaceType); return this; } - public NamespaceResourceImpl withCreatedAt(OffsetDateTime createdAt) { - this.createParameters.withCreatedAt(createdAt); + public NamespaceResourceImpl withReplicationRegion(ReplicationRegion replicationRegion) { + this.innerModel().withReplicationRegion(replicationRegion); return this; } - public NamespaceResourceImpl withUpdatedAt(OffsetDateTime updatedAt) { - this.createParameters.withUpdatedAt(updatedAt); + public NamespaceResourceImpl withZoneRedundancy(ZoneRedundancyPreference zoneRedundancy) { + this.innerModel().withZoneRedundancy(zoneRedundancy); return this; } - public NamespaceResourceImpl withServiceBusEndpoint(String serviceBusEndpoint) { - this.createParameters.withServiceBusEndpoint(serviceBusEndpoint); + public NamespaceResourceImpl withNetworkAcls(NetworkAcls networkAcls) { + this.innerModel().withNetworkAcls(networkAcls); return this; } - public NamespaceResourceImpl withSubscriptionId(String subscriptionId) { - this.createParameters.withSubscriptionId(subscriptionId); + public NamespaceResourceImpl withPnsCredentials(PnsCredentials pnsCredentials) { + this.innerModel().withPnsCredentials(pnsCredentials); return this; } public NamespaceResourceImpl withScaleUnit(String scaleUnit) { - this.createParameters.withScaleUnit(scaleUnit); + this.innerModel().withScaleUnit(scaleUnit); return this; } - public NamespaceResourceImpl withEnabled(Boolean enabled) { - this.createParameters.withEnabled(enabled); - return this; - } - - public NamespaceResourceImpl withCritical(Boolean critical) { - this.createParameters.withCritical(critical); + public NamespaceResourceImpl withDataCenter(String dataCenter) { + this.innerModel().withDataCenter(dataCenter); return this; } - public NamespaceResourceImpl withDataCenter(String dataCenter) { - this.createParameters.withDataCenter(dataCenter); + public NamespaceResourceImpl withPublicNetworkAccess(PublicNetworkAccess publicNetworkAccess) { + this.innerModel().withPublicNetworkAccess(publicNetworkAccess); return this; } - public NamespaceResourceImpl withNamespaceType(NamespaceType namespaceType) { - this.createParameters.withNamespaceType(namespaceType); + public NamespaceResourceImpl withProperties(NamespaceProperties properties) { + this.updateParameters.withProperties(properties); return this; } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesClientImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesClientImpl.java index 98023acf2c612..f97632e8971c2 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesClientImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesClientImpl.java @@ -31,302 +31,238 @@ import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.FluxUtil; -import com.azure.core.util.logging.ClientLogger; import com.azure.core.util.polling.PollerFlux; import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.notificationhubs.fluent.NamespacesClient; import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner; import com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -import com.azure.resourcemanager.notificationhubs.models.NamespaceCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.NamespaceListResult; import com.azure.resourcemanager.notificationhubs.models.NamespacePatchParameters; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleListResult; import java.nio.ByteBuffer; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in NamespacesClient. */ +/** + * An instance of this class provides access to all the operations defined in NamespacesClient. + */ public final class NamespacesClientImpl implements NamespacesClient { - private final ClientLogger logger = new ClientLogger(NamespacesClientImpl.class); - - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final NamespacesService service; - /** The service client containing this operation class. */ - private final NotificationHubsManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final NotificationHubsRPClientImpl client; /** * Initializes an instance of NamespacesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - NamespacesClientImpl(NotificationHubsManagementClientImpl client) { - this.service = - RestProxy.create(NamespacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + NamespacesClientImpl(NotificationHubsRPClientImpl client) { + this.service + = RestProxy.create(NamespacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for NotificationHubsManagementClientNamespaces to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for NotificationHubsRPClientNamespaces to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "NotificationHubsMana") - private interface NamespacesService { - @Headers({"Content-Type: application/json"}) + @ServiceInterface(name = "NotificationHubsRPCl") + public interface NamespacesService { + @Headers({ "Content-Type: application/json" }) @Post("/subscriptions/{subscriptionId}/providers/Microsoft.NotificationHubs/checkNamespaceAvailability") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> checkAvailability( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") CheckAvailabilityParameters parameters, - @HeaderParam("Accept") String accept, + Mono> checkAvailability(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") CheckAvailabilityParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @QueryParam("api-version") String apiVersion, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") NamespaceCreateOrUpdateParameters parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> patch( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @QueryParam("api-version") String apiVersion, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") NamespacePatchParameters parameters, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") NamespaceResourceInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @QueryParam("api-version") String apiVersion, + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") NamespacePatchParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/AuthorizationRules/{authorizationRuleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.NotificationHubs/namespaces") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdateAuthorizationRule( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("$skipToken") String skipToken, + @QueryParam("$top") Integer top, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$skipToken") String skipToken, + @QueryParam("$top") Integer top, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/AuthorizationRules/{authorizationRuleName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/authorizationRules/{authorizationRuleName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> deleteAuthorizationRule( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + Mono> createOrUpdateAuthorizationRule( + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("authorizationRuleName") String authorizationRuleName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") SharedAccessAuthorizationRuleResourceInner parameters, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/authorizationRules/{authorizationRuleName}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> deleteAuthorizationRule(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("authorizationRuleName") String authorizationRuleName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/AuthorizationRules/{authorizationRuleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/authorizationRules/{authorizationRuleName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> getAuthorizationRule( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/authorizationRules") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listAuthorizationRules( + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.NotificationHubs/namespaces") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/authorizationRules/{authorizationRuleName}/listKeys") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> listKeys(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("authorizationRuleName") String authorizationRuleName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/AuthorizationRules") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/authorizationRules/{authorizationRuleName}/regenerateKeys") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAuthorizationRules( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @QueryParam("api-version") String apiVersion, + Mono> regenerateKeys(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/AuthorizationRules/{authorizationRuleName}/listKeys") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listKeys( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") PolicyKeyResource parameters, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/AuthorizationRules/{authorizationRuleName}/regenerateKeys") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/pnsCredentials") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> regenerateKeys( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, + Mono> getPnsCredentials(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") PolicykeyResource parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listAllNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAllNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listAuthorizationRulesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. + * + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> checkAvailabilityWithResponseAsync( - CheckAvailabilityParameters parameters) { + private Mono> + checkAvailabilityWithResponseAsync(CheckAvailabilityParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -335,44 +271,33 @@ private Mono> checkAvailabilityWithRespon } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .checkAvailability( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.checkAvailability(this.client.getEndpoint(), + this.client.getSubscriptionId(), this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. + * + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> checkAvailabilityWithResponseAsync( - CheckAvailabilityParameters parameters, Context context) { + private Mono> + checkAvailabilityWithResponseAsync(CheckAvailabilityParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -381,91 +306,78 @@ private Mono> checkAvailabilityWithRespon } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .checkAvailability( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.checkAvailability(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), parameters, accept, context); } /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. + * + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono checkAvailabilityAsync(CheckAvailabilityParameters parameters) { - return checkAvailabilityWithResponseAsync(parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + return checkAvailabilityWithResponseAsync(parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. + * + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CheckAvailabilityResultInner checkAvailability(CheckAvailabilityParameters parameters) { - return checkAvailabilityAsync(parameters).block(); + public Response checkAvailabilityWithResponse(CheckAvailabilityParameters parameters, + Context context) { + return checkAvailabilityWithResponseAsync(parameters, context).block(); } /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. - * @param context The context to associate with this operation. + * + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a CheckAvailability resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response checkAvailabilityWithResponse( - CheckAvailabilityParameters parameters, Context context) { - return checkAvailabilityWithResponseAsync(parameters, context).block(); + public CheckAvailabilityResultInner checkAvailability(CheckAvailabilityParameters parameters) { + return checkAvailabilityWithResponse(parameters, Context.NONE).getValue(); } /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String namespaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -474,55 +386,36 @@ private Mono> createOrUpdateWithResponseAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String namespaceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -531,113 +424,82 @@ private Mono> createOrUpdateWithResponseAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, this.client.getApiVersion(), accept, context); } /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters) { - return createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono getByResourceGroupAsync(String resourceGroupName, String namespaceName) { + return getByResourceGroupWithResponseAsync(resourceGroupName, namespaceName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NamespaceResourceInner createOrUpdate( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters) { - return createOrUpdateAsync(resourceGroupName, namespaceName, parameters).block(); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String namespaceName, Context context) { + return getByResourceGroupWithResponseAsync(resourceGroupName, namespaceName, context).block(); } /** - * Creates/Updates a service namespace. Once created, this namespace's resource manifest is immutable. This - * operation is idempotent. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to create a Namespace Resource. - * @param context The context to associate with this operation. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, String namespaceName, NamespaceCreateOrUpdateParameters parameters, Context context) { - return createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, parameters, context).block(); + public NamespaceResourceInner getByResourceGroup(String resourceGroupName, String namespaceName) { + return getByResourceGroupWithResponse(resourceGroupName, namespaceName, Context.NONE).getValue(); } /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> patchWithResponseAsync( - String resourceGroupName, String namespaceName, NamespacePatchParameters parameters) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String namespaceName, NamespaceResourceInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -646,12 +508,6 @@ private Mono> patchWithResponseAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -659,41 +515,34 @@ private Mono> patchWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .patch( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return notification Hubs Namespace Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> patchWithResponseAsync( - String resourceGroupName, String namespaceName, NamespacePatchParameters parameters, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String namespaceName, NamespaceResourceInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -702,12 +551,6 @@ private Mono> patchWithResponseAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -715,319 +558,316 @@ private Mono> patchWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .patch( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, this.client.getApiVersion(), parameters, accept, context); } /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link PollerFlux} for polling of notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono patchAsync( - String resourceGroupName, String namespaceName, NamespacePatchParameters parameters) { - return patchWithResponseAsync(resourceGroupName, namespaceName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, NamespaceResourceInner> + beginCreateOrUpdateAsync(String resourceGroupName, String namespaceName, NamespaceResourceInner parameters) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, parameters); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), NamespaceResourceInner.class, NamespaceResourceInner.class, + this.client.getContext()); } /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link PollerFlux} for polling of notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public NamespaceResourceInner patch( - String resourceGroupName, String namespaceName, NamespacePatchParameters parameters) { - return patchAsync(resourceGroupName, namespaceName, parameters).block(); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, NamespaceResourceInner> beginCreateOrUpdateAsync( + String resourceGroupName, String namespaceName, NamespaceResourceInner parameters, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, parameters, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), NamespaceResourceInner.class, NamespaceResourceInner.class, context); } /** - * Patches the existing namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters Parameters supplied to patch a Namespace Resource. - * @param context The context to associate with this operation. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link SyncPoller} for polling of notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response patchWithResponse( - String resourceGroupName, String namespaceName, NamespacePatchParameters parameters, Context context) { - return patchWithResponseAsync(resourceGroupName, namespaceName, parameters, context).block(); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, NamespaceResourceInner> + beginCreateOrUpdate(String resourceGroupName, String namespaceName, NamespaceResourceInner parameters) { + return this.beginCreateOrUpdateAsync(resourceGroupName, namespaceName, parameters).getSyncPoller(); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return the {@link SyncPoller} for polling of notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync(String resourceGroupName, String namespaceName) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (namespaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, NamespaceResourceInner> beginCreateOrUpdate( + String resourceGroupName, String namespaceName, NamespaceResourceInner parameters, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, namespaceName, parameters, context).getSyncPoller(); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String namespaceName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (namespaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context); + private Mono createOrUpdateAsync(String resourceGroupName, String namespaceName, + NamespaceResourceInner parameters) { + return beginCreateOrUpdateAsync(resourceGroupName, namespaceName, parameters).last() + .flatMap(this.client::getLroFinalResultOrError); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource on successful completion of {@link Mono}. */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String namespaceName) { - Mono>> mono = deleteWithResponseAsync(resourceGroupName, namespaceName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String namespaceName, + NamespaceResourceInner parameters, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, namespaceName, parameters, context).last() + .flatMap(this.client::getLroFinalResultOrError); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String namespaceName, Context context) { - context = this.client.mergeContext(context); - Mono>> mono = deleteWithResponseAsync(resourceGroupName, namespaceName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + @ServiceMethod(returns = ReturnType.SINGLE) + public NamespaceResourceInner createOrUpdate(String resourceGroupName, String namespaceName, + NamespaceResourceInner parameters) { + return createOrUpdateAsync(resourceGroupName, namespaceName, parameters).block(); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Creates / Updates a Notification Hub namespace. This operation is idempotent. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource. */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName) { - return beginDeleteAsync(resourceGroupName, namespaceName).getSyncPoller(); + @ServiceMethod(returns = ReturnType.SINGLE) + public NamespaceResourceInner createOrUpdate(String resourceGroupName, String namespaceName, + NamespaceResourceInner parameters, Context context) { + return createOrUpdateAsync(resourceGroupName, namespaceName, parameters, context).block(); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource along with {@link Response} on successful completion of + * {@link Mono}. */ - @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String namespaceName, Context context) { - return beginDeleteAsync(resourceGroupName, namespaceName, context).getSyncPoller(); + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> updateWithResponseAsync(String resourceGroupName, + String namespaceName, NamespacePatchParameters parameters) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), parameters, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String namespaceName) { - return beginDeleteAsync(resourceGroupName, namespaceName).last().flatMap(this.client::getLroFinalResultOrError); + private Mono> updateWithResponseAsync(String resourceGroupName, + String namespaceName, NamespacePatchParameters parameters, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, this.client.getApiVersion(), parameters, accept, context); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hubs Namespace Resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String namespaceName, Context context) { - return beginDeleteAsync(resourceGroupName, namespaceName, context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono updateAsync(String resourceGroupName, String namespaceName, + NamespacePatchParameters parameters) { + return updateWithResponseAsync(resourceGroupName, namespaceName, parameters) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String namespaceName) { - deleteAsync(resourceGroupName, namespaceName).block(); + public Response updateWithResponse(String resourceGroupName, String namespaceName, + NamespacePatchParameters parameters, Context context) { + return updateWithResponseAsync(resourceGroupName, namespaceName, parameters, context).block(); } /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Patches the existing namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String namespaceName, Context context) { - deleteAsync(resourceGroupName, namespaceName, context).block(); + public NamespaceResourceInner update(String resourceGroupName, String namespaceName, + NamespacePatchParameters parameters) { + return updateWithResponse(resourceGroupName, namespaceName, parameters, Context.NONE).getValue(); } /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String namespaceName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String namespaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1036,47 +876,34 @@ private Mono> getByResourceGroupWithResponseAsy if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String namespaceName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String namespaceName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1085,458 +912,383 @@ private Mono> getByResourceGroupWithResponseAsy if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, this.client.getApiVersion(), accept, context); } /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getByResourceGroupAsync(String resourceGroupName, String namespaceName) { - return getByResourceGroupWithResponseAsync(resourceGroupName, namespaceName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono deleteAsync(String resourceGroupName, String namespaceName) { + return deleteWithResponseAsync(resourceGroupName, namespaceName).flatMap(ignored -> Mono.empty()); } /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NamespaceResourceInner getByResourceGroup(String resourceGroupName, String namespaceName) { - return getByResourceGroupAsync(resourceGroupName, namespaceName).block(); + public Response deleteWithResponse(String resourceGroupName, String namespaceName, Context context) { + return deleteWithResponseAsync(resourceGroupName, namespaceName, context).block(); } /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String namespaceName, Context context) { - return getByResourceGroupWithResponseAsync(resourceGroupName, namespaceName, context).block(); + public void delete(String resourceGroupName, String namespaceName) { + deleteWithResponse(resourceGroupName, namespaceName, Context.NONE); } /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { + private Mono> listSinglePageAsync(String skipToken, Integer top) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (namespaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); - } - if (authorizationRuleName == null) { - return Mono - .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdateAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), skipToken, + top, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, + private Mono> listSinglePageAsync(String skipToken, Integer top, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (namespaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); - } - if (authorizationRuleName == null) { - return Mono - .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .createOrUpdateAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), skipToken, top, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAuthorizationRuleAsync( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { - return createOrUpdateAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, authorizationRuleName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String skipToken, Integer top) { + return new PagedFlux<>(() -> listSinglePageAsync(skipToken, top), + nextLink -> listAllNextSinglePageAsync(nextLink)); } /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Lists all the available namespaces within the subscription. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Namespace operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync() { + final String skipToken = null; + final Integer top = null; + return new PagedFlux<>(() -> listSinglePageAsync(skipToken, top), + nextLink -> listAllNextSinglePageAsync(nextLink)); + } + + /** + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { - return createOrUpdateAuthorizationRuleAsync(resourceGroupName, namespaceName, authorizationRuleName, parameters) - .block(); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String skipToken, Integer top, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(skipToken, top, context), + nextLink -> listAllNextSinglePageAsync(nextLink, context)); } /** - * Creates an authorization rule for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Lists all the available namespaces within the subscription. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list() { + final String skipToken = null; + final Integer top = null; + return new PagedIterable<>(listAsync(skipToken, top)); + } + + /** + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context) { - return createOrUpdateAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, authorizationRuleName, parameters, context) - .block(); + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String skipToken, Integer top, Context context) { + return new PagedIterable<>(listAsync(skipToken, top, context)); } /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteAuthorizationRuleWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + String skipToken, Integer top) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (namespaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); - } - if (authorizationRuleName == null) { - return Mono - .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } + final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .deleteAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context)) + context -> service.listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, skipToken, top, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteAuthorizationRuleWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + String skipToken, Integer top, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (namespaceName == null) { - return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); - } - if (authorizationRuleName == null) { - return Mono - .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } + final String accept = "application/json"; context = this.client.mergeContext(context); return service - .deleteAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + skipToken, top, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return the response of the List Namespace operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAuthorizationRuleAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName) { - return deleteAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName) - .flatMap((Response res) -> Mono.empty()); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName, String skipToken, + Integer top) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, skipToken, top), + nextLink -> listNextSinglePageAsync(nextLink)); } /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Namespace operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName) { - deleteAuthorizationRuleAsync(resourceGroupName, namespaceName, authorizationRuleName).block(); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName) { + final String skipToken = null; + final Integer top = null; + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, skipToken, top), + nextLink -> listNextSinglePageAsync(nextLink)); } /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the response of the List Namespace operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { - return deleteAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, authorizationRuleName, context) - .block(); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName, String skipToken, + Integer top, Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, skipToken, top, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByResourceGroup(String resourceGroupName) { + final String skipToken = null; + final Integer top = null; + return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, skipToken, top)); + } + + /** + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByResourceGroup(String resourceGroupName, String skipToken, + Integer top, Context context) { + return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, skipToken, top, context)); + } + + /** + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response} + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthorizationRuleWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName) { + private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1549,49 +1301,44 @@ private Mono> getAuthorizat return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.createOrUpdateAuthorizationRule(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, authorizationRuleName, + this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response} + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthorizationRuleWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { + private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1604,413 +1351,371 @@ private Mono> getAuthorizat return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.createOrUpdateAuthorizationRule(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, authorizationRuleName, this.client.getApiVersion(), parameters, accept, + context); } /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return response for POST requests that return single SharedAccessAuthorizationRule on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAuthorizationRuleAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName) { - return getAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono createOrUpdateAuthorizationRuleAsync( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters) { + return createOrUpdateAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, + parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SharedAccessAuthorizationRuleResourceInner getAuthorizationRule( - String resourceGroupName, String namespaceName, String authorizationRuleName) { - return getAuthorizationRuleAsync(resourceGroupName, namespaceName, authorizationRuleName).block(); + public Response createOrUpdateAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context) { + return createOrUpdateAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, + parameters, context).block(); } /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. - * @param context The context to associate with this operation. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return response for POST requests that return single SharedAccessAuthorizationRule. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { - return getAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, context) - .block(); + public SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule(String resourceGroupName, + String namespaceName, String authorizationRuleName, SharedAccessAuthorizationRuleResourceInner parameters) { + return createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, + parameters, Context.NONE).getValue(); } /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { + private Mono> deleteAuthorizationRuleWithResponseAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (authorizationRuleName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.deleteAuthorizationRule(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, authorizationRuleName, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> deleteAuthorizationRuleWithResponseAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (authorizationRuleName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.deleteAuthorizationRule(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, authorizationRuleName, this.client.getApiVersion(), accept, context); } /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return A {@link Mono} that completes when a successful response is received. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listNextSinglePageAsync(nextLink)); + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAuthorizationRuleAsync(String resourceGroupName, String namespaceName, + String authorizationRuleName) { + return deleteAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName) + .flatMap(ignored -> Mono.empty()); } /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), - nextLink -> listNextSinglePageAsync(nextLink, context)); + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context) { + return deleteAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, + context).block(); } /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup(String resourceGroupName) { - return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName)); + @ServiceMethod(returns = ReturnType.SINGLE) + public void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName) { + deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, Context.NONE); } /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. - * @param context The context to associate with this operation. + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, context)); - } - - /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync() { + private Mono> getAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String authorizationRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (authorizationRuleName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.getAuthorizationRule(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, authorizationRuleName, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(Context context) { + private Mono> getAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (authorizationRuleName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.getAuthorizationRule(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, authorizationRuleName, this.client.getApiVersion(), accept, context); } /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name on successful completion of {@link Mono}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync() { - return new PagedFlux<>(() -> listSinglePageAsync(), nextLink -> listAllNextSinglePageAsync(nextLink)); + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAuthorizationRuleAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName) { + return getAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listAllNextSinglePageAsync(nextLink, context)); - } - - /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list() { - return new PagedIterable<>(listAsync()); + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { + return getAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, context) + .block(); } /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * - * @param context The context to associate with this operation. + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(Context context) { - return new PagedIterable<>(listAsync(context)); + @ServiceMethod(returns = ReturnType.SINGLE) + public SharedAccessAuthorizationRuleResourceInner getAuthorizationRule(String resourceGroupName, + String namespaceName, String authorizationRuleName) { + return getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, Context.NONE) + .getValue(); } /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAuthorizationRulesSinglePageAsync( - String resourceGroupName, String namespaceName) { + private Mono> + listAuthorizationRulesSinglePageAsync(String resourceGroupName, String namespaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2019,56 +1724,39 @@ private Mono> listAuth if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listAuthorizationRules( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + context -> service.listAuthorizationRules(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAuthorizationRulesSinglePageAsync( - String resourceGroupName, String namespaceName, Context context) { + private Mono> + listAuthorizationRulesSinglePageAsync(String resourceGroupName, String namespaceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2077,123 +1765,105 @@ private Mono> listAuth if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listAuthorizationRules( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listAuthorizationRules(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAuthorizationRulesAsync( - String resourceGroupName, String namespaceName) { - return new PagedFlux<>( - () -> listAuthorizationRulesSinglePageAsync(resourceGroupName, namespaceName), + private PagedFlux listAuthorizationRulesAsync(String resourceGroupName, + String namespaceName) { + return new PagedFlux<>(() -> listAuthorizationRulesSinglePageAsync(resourceGroupName, namespaceName), nextLink -> listAuthorizationRulesNextSinglePageAsync(nextLink)); } /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAuthorizationRulesAsync( - String resourceGroupName, String namespaceName, Context context) { - return new PagedFlux<>( - () -> listAuthorizationRulesSinglePageAsync(resourceGroupName, namespaceName, context), + private PagedFlux listAuthorizationRulesAsync(String resourceGroupName, + String namespaceName, Context context) { + return new PagedFlux<>(() -> listAuthorizationRulesSinglePageAsync(resourceGroupName, namespaceName, context), nextLink -> listAuthorizationRulesNextSinglePageAsync(nextLink, context)); } /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName) { + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName) { return new PagedIterable<>(listAuthorizationRulesAsync(resourceGroupName, namespaceName)); } /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, Context context) { + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, Context context) { return new PagedIterable<>(listAuthorizationRulesAsync(resourceGroupName, namespaceName, context)); } /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the namespace along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listKeysWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName) { + private Mono> listKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2206,49 +1876,36 @@ private Mono> listKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.listKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, authorizationRuleName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the namespace along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listKeysWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { + private Mono> listKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2261,106 +1918,88 @@ private Mono> listKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.listKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, authorizationRuleName, this.client.getApiVersion(), accept, context); } /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the namespace on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listKeysAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName) { + private Mono listKeysAsync(String resourceGroupName, String namespaceName, + String authorizationRuleName) { return listKeysWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the namespace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ResourceListKeysInner listKeys( - String resourceGroupName, String namespaceName, String authorizationRuleName) { - return listKeysAsync(resourceGroupName, namespaceName, authorizationRuleName).block(); + public Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context) { + return listKeysWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, context).block(); } /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Primary and Secondary ConnectionStrings to the namespace. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listKeysWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { - return listKeysWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, context).block(); + public ResourceListKeysInner listKeys(String resourceGroupName, String namespaceName, + String authorizationRuleName) { + return listKeysWithResponse(resourceGroupName, namespaceName, authorizationRuleName, Context.NONE).getValue(); } /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> regenerateKeysWithResponseAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName, PolicykeyResource parameters) { + private Mono> regenerateKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName, PolicyKeyResource parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2373,12 +2012,6 @@ private Mono> regenerateKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -2386,47 +2019,36 @@ private Mono> regenerateKeysWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .regenerateKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.regenerateKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, authorizationRuleName, this.client.getApiVersion(), parameters, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> regenerateKeysWithResponseAsync( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context) { + private Mono> regenerateKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String authorizationRuleName, PolicyKeyResource parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2439,12 +2061,6 @@ private Mono> regenerateKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -2452,301 +2068,365 @@ private Mono> regenerateKeysWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .regenerateKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.regenerateKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, authorizationRuleName, this.client.getApiVersion(), parameters, accept, context); } /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono regenerateKeysAsync( - String resourceGroupName, String namespaceName, String authorizationRuleName, PolicykeyResource parameters) { + private Mono regenerateKeysAsync(String resourceGroupName, String namespaceName, + String authorizationRuleName, PolicyKeyResource parameters) { return regenerateKeysWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ResourceListKeysInner regenerateKeys( - String resourceGroupName, String namespaceName, String authorizationRuleName, PolicykeyResource parameters) { - return regenerateKeysAsync(resourceGroupName, namespaceName, authorizationRuleName, parameters).block(); + public Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, PolicyKeyResource parameters, Context context) { + return regenerateKeysWithResponseAsync(resourceGroupName, namespaceName, authorizationRuleName, parameters, + context).block(); } /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ResourceListKeysInner regenerateKeys(String resourceGroupName, String namespaceName, + String authorizationRuleName, PolicyKeyResource parameters) { + return regenerateKeysWithResponse(resourceGroupName, namespaceName, authorizationRuleName, parameters, + Context.NONE).getValue(); + } + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getPnsCredentialsWithResponseAsync(String resourceGroupName, + String namespaceName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.getPnsCredentials(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return description of a NotificationHub PNS Credentials along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context) { - return regenerateKeysWithResponseAsync( - resourceGroupName, namespaceName, authorizationRuleName, parameters, context) - .block(); + private Mono> getPnsCredentialsWithResponseAsync(String resourceGroupName, + String namespaceName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getPnsCredentials(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, this.client.getApiVersion(), accept, context); + } + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getPnsCredentialsAsync(String resourceGroupName, String namespaceName) { + return getPnsCredentialsWithResponseAsync(resourceGroupName, namespaceName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getPnsCredentialsWithResponse(String resourceGroupName, + String namespaceName, Context context) { + return getPnsCredentialsWithResponseAsync(resourceGroupName, namespaceName, context).block(); + } + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public PnsCredentialsResourceInner getPnsCredentials(String resourceGroupName, String namespaceName) { + return getPnsCredentialsWithResponse(resourceGroupName, namespaceName, Context.NONE).getValue(); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink) { + private Mono> listAllNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listAllNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink, Context context) { + private Mono> listAllNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAllNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAllNextSinglePageAsync(String nextLink) { + private Mono> listNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listAllNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAllNextSinglePageAsync(String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAllNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAuthorizationRulesNextSinglePageAsync( - String nextLink) { + private Mono> + listAuthorizationRulesNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listAuthorizationRulesNext(nextLink, this.client.getEndpoint(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAuthorizationRulesNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listAuthorizationRulesNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAuthorizationRulesNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAuthorizationRulesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesImpl.java index c64ce4ac778a6..246b2e83225ab 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NamespacesImpl.java @@ -12,31 +12,43 @@ import com.azure.resourcemanager.notificationhubs.fluent.NamespacesClient; import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner; import com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityResult; import com.azure.resourcemanager.notificationhubs.models.NamespaceResource; import com.azure.resourcemanager.notificationhubs.models.Namespaces; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; +import com.azure.resourcemanager.notificationhubs.models.PnsCredentialsResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; import com.azure.resourcemanager.notificationhubs.models.ResourceListKeys; import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleResource; -import com.fasterxml.jackson.annotation.JsonIgnore; public final class NamespacesImpl implements Namespaces { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NamespacesImpl.class); + private static final ClientLogger LOGGER = new ClientLogger(NamespacesImpl.class); private final NamespacesClient innerClient; private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - public NamespacesImpl( - NamespacesClient innerClient, + public NamespacesImpl(NamespacesClient innerClient, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } + public Response checkAvailabilityWithResponse(CheckAvailabilityParameters parameters, + Context context) { + Response inner + = this.serviceClient().checkAvailabilityWithResponse(parameters, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new CheckAvailabilityResultImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + public CheckAvailabilityResult checkAvailability(CheckAvailabilityParameters parameters) { CheckAvailabilityResultInner inner = this.serviceClient().checkAvailability(parameters); if (inner != null) { @@ -46,130 +58,147 @@ public CheckAvailabilityResult checkAvailability(CheckAvailabilityParameters par } } - public Response checkAvailabilityWithResponse( - CheckAvailabilityParameters parameters, Context context) { - Response inner = - this.serviceClient().checkAvailabilityWithResponse(parameters, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String namespaceName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, namespaceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new CheckAvailabilityResultImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new NamespaceResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public NamespaceResource getByResourceGroup(String resourceGroupName, String namespaceName) { + NamespaceResourceInner inner = this.serviceClient().getByResourceGroup(resourceGroupName, namespaceName); + if (inner != null) { + return new NamespaceResourceImpl(inner, this.manager()); } else { return null; } } + public Response deleteByResourceGroupWithResponse(String resourceGroupName, String namespaceName, + Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, namespaceName, context); + } + public void deleteByResourceGroup(String resourceGroupName, String namespaceName) { this.serviceClient().delete(resourceGroupName, namespaceName); } - public void delete(String resourceGroupName, String namespaceName, Context context) { - this.serviceClient().delete(resourceGroupName, namespaceName, context); + public PagedIterable list() { + PagedIterable inner = this.serviceClient().list(); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); } - public NamespaceResource getByResourceGroup(String resourceGroupName, String namespaceName) { - NamespaceResourceInner inner = this.serviceClient().getByResourceGroup(resourceGroupName, namespaceName); + public PagedIterable list(String skipToken, Integer top, Context context) { + PagedIterable inner = this.serviceClient().list(skipToken, top, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByResourceGroup(String resourceGroupName) { + PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); + } + + public PagedIterable listByResourceGroup(String resourceGroupName, String skipToken, Integer top, + Context context) { + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, skipToken, top, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); + } + + public Response createOrUpdateAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context) { + Response inner + = this.serviceClient().createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, namespaceName, + authorizationRuleName, parameters, context); if (inner != null) { - return new NamespaceResourceImpl(inner, this.manager()); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SharedAccessAuthorizationRuleResourceImpl(inner.getValue(), this.manager())); } else { return null; } } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String namespaceName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, namespaceName, context); + public SharedAccessAuthorizationRuleResource createOrUpdateAuthorizationRule(String resourceGroupName, + String namespaceName, String authorizationRuleName, SharedAccessAuthorizationRuleResourceInner parameters) { + SharedAccessAuthorizationRuleResourceInner inner = this.serviceClient() + .createOrUpdateAuthorizationRule(resourceGroupName, namespaceName, authorizationRuleName, parameters); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new NamespaceResourceImpl(inner.getValue(), this.manager())); + return new SharedAccessAuthorizationRuleResourceImpl(inner, this.manager()); } else { return null; } } - public void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName) { - this.serviceClient().deleteAuthorizationRule(resourceGroupName, namespaceName, authorizationRuleName); + public Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context) { + return this.serviceClient().deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, + authorizationRuleName, context); } - public Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { - return this - .serviceClient() - .deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); + public void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName) { + this.serviceClient().deleteAuthorizationRule(resourceGroupName, namespaceName, authorizationRuleName); } - public SharedAccessAuthorizationRuleResource getAuthorizationRule( - String resourceGroupName, String namespaceName, String authorizationRuleName) { - SharedAccessAuthorizationRuleResourceInner inner = - this.serviceClient().getAuthorizationRule(resourceGroupName, namespaceName, authorizationRuleName); + public Response getAuthorizationRuleWithResponse(String resourceGroupName, + String namespaceName, String authorizationRuleName, Context context) { + Response inner = this.serviceClient() + .getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); if (inner != null) { - return new SharedAccessAuthorizationRuleResourceImpl(inner, this.manager()); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SharedAccessAuthorizationRuleResourceImpl(inner.getValue(), this.manager())); } else { return null; } } - public Response getAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { - Response inner = - this - .serviceClient() - .getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); + public SharedAccessAuthorizationRuleResource getAuthorizationRule(String resourceGroupName, String namespaceName, + String authorizationRuleName) { + SharedAccessAuthorizationRuleResourceInner inner + = this.serviceClient().getAuthorizationRule(resourceGroupName, namespaceName, authorizationRuleName); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new SharedAccessAuthorizationRuleResourceImpl(inner.getValue(), this.manager())); + return new SharedAccessAuthorizationRuleResourceImpl(inner, this.manager()); } else { return null; } } - public PagedIterable listByResourceGroup(String resourceGroupName) { - PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName) { + PagedIterable inner + = this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); } - public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, Context context) { + PagedIterable inner + = this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); } - public PagedIterable list() { - PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); - } - - public PagedIterable list(Context context) { - PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new NamespaceResourceImpl(inner1, this.manager())); - } - - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName) { - PagedIterable inner = - this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName); - return Utils.mapPage(inner, inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); - } - - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, Context context) { - PagedIterable inner = - this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName, context); - return Utils.mapPage(inner, inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); + public Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context) { + Response inner = this.serviceClient().listKeysWithResponse(resourceGroupName, + namespaceName, authorizationRuleName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ResourceListKeysImpl(inner.getValue(), this.manager())); + } else { + return null; + } } public ResourceListKeys listKeys(String resourceGroupName, String namespaceName, String authorizationRuleName) { - ResourceListKeysInner inner = - this.serviceClient().listKeys(resourceGroupName, namespaceName, authorizationRuleName); + ResourceListKeysInner inner + = this.serviceClient().listKeys(resourceGroupName, namespaceName, authorizationRuleName); if (inner != null) { return new ResourceListKeysImpl(inner, this.manager()); } else { @@ -177,25 +206,22 @@ public ResourceListKeys listKeys(String resourceGroupName, String namespaceName, } } - public Response listKeysWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context) { - Response inner = - this.serviceClient().listKeysWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); + public Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, PolicyKeyResource parameters, Context context) { + Response inner = this.serviceClient().regenerateKeysWithResponse(resourceGroupName, + namespaceName, authorizationRuleName, parameters, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ResourceListKeysImpl(inner.getValue(), this.manager())); } else { return null; } } - public ResourceListKeys regenerateKeys( - String resourceGroupName, String namespaceName, String authorizationRuleName, PolicykeyResource parameters) { - ResourceListKeysInner inner = - this.serviceClient().regenerateKeys(resourceGroupName, namespaceName, authorizationRuleName, parameters); + public ResourceListKeys regenerateKeys(String resourceGroupName, String namespaceName, String authorizationRuleName, + PolicyKeyResource parameters) { + ResourceListKeysInner inner + = this.serviceClient().regenerateKeys(resourceGroupName, namespaceName, authorizationRuleName, parameters); if (inner != null) { return new ResourceListKeysImpl(inner, this.manager()); } else { @@ -203,218 +229,81 @@ public ResourceListKeys regenerateKeys( } } - public Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context) { - Response inner = - this - .serviceClient() - .regenerateKeysWithResponse( - resourceGroupName, namespaceName, authorizationRuleName, parameters, context); + public Response getPnsCredentialsWithResponse(String resourceGroupName, + String namespaceName, Context context) { + Response inner + = this.serviceClient().getPnsCredentialsWithResponse(resourceGroupName, namespaceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ResourceListKeysImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new PnsCredentialsResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public PnsCredentialsResource getPnsCredentials(String resourceGroupName, String namespaceName) { + PnsCredentialsResourceInner inner = this.serviceClient().getPnsCredentials(resourceGroupName, namespaceName); + if (inner != null) { + return new PnsCredentialsResourceImpl(inner, this.manager()); } else { return null; } } public NamespaceResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, namespaceName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, namespaceName, context); } - public SharedAccessAuthorizationRuleResource getAuthorizationRuleById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); - if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); - } - String authorizationRuleName = Utils.getValueFromIdByName(id, "AuthorizationRules"); - if (authorizationRuleName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'AuthorizationRules'.", id))); - } - return this - .getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, Context.NONE) - .getValue(); - } - - public Response getAuthorizationRuleByIdWithResponse( - String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); - if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); - } - String authorizationRuleName = Utils.getValueFromIdByName(id, "AuthorizationRules"); - if (authorizationRuleName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'AuthorizationRules'.", id))); - } - return this.getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); - } - public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } - this.delete(resourceGroupName, namespaceName, Context.NONE); + this.deleteByResourceGroupWithResponse(resourceGroupName, namespaceName, Context.NONE); } - public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + public Response deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } - this.delete(resourceGroupName, namespaceName, context); - } - - public void deleteAuthorizationRuleById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); - if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); - } - String authorizationRuleName = Utils.getValueFromIdByName(id, "AuthorizationRules"); - if (authorizationRuleName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'AuthorizationRules'.", id))); - } - this.deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, Context.NONE); - } - - public Response deleteAuthorizationRuleByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); - if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); - } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); - if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); - } - String authorizationRuleName = Utils.getValueFromIdByName(id, "AuthorizationRules"); - if (authorizationRuleName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'AuthorizationRules'.", id))); - } - return this - .deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); + return this.deleteByResourceGroupWithResponse(resourceGroupName, namespaceName, context); } private NamespacesClient serviceClient() { @@ -428,8 +317,4 @@ private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manag public NamespaceResourceImpl define(String name) { return new NamespaceResourceImpl(name, this.manager()); } - - public SharedAccessAuthorizationRuleResourceImpl defineAuthorizationRule(String name) { - return new SharedAccessAuthorizationRuleResourceImpl(name, this.manager()); - } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubResourceImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubResourceImpl.java index fcb4c75f93365..db0a8b75fecfb 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubResourceImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubResourceImpl.java @@ -6,21 +6,24 @@ import com.azure.core.http.rest.Response; import com.azure.core.management.Region; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubResourceInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; import com.azure.resourcemanager.notificationhubs.models.AdmCredential; import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; +import com.azure.resourcemanager.notificationhubs.models.BrowserCredential; import com.azure.resourcemanager.notificationhubs.models.DebugSendResponse; +import com.azure.resourcemanager.notificationhubs.models.FcmV1Credential; import com.azure.resourcemanager.notificationhubs.models.GcmCredential; import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; -import com.azure.resourcemanager.notificationhubs.models.NotificationHubCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.NotificationHubPatchParameters; import com.azure.resourcemanager.notificationhubs.models.NotificationHubResource; import com.azure.resourcemanager.notificationhubs.models.PnsCredentialsResource; import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.WnsCredential; +import com.azure.resourcemanager.notificationhubs.models.XiaomiCredential; import java.util.Collections; import java.util.List; import java.util.Map; @@ -60,6 +63,10 @@ public Sku sku() { return this.innerModel().sku(); } + public SystemData systemData() { + return this.innerModel().systemData(); + } + public String namePropertiesName() { return this.innerModel().namePropertiesName(); } @@ -101,6 +108,22 @@ public BaiduCredential baiduCredential() { return this.innerModel().baiduCredential(); } + public BrowserCredential browserCredential() { + return this.innerModel().browserCredential(); + } + + public XiaomiCredential xiaomiCredential() { + return this.innerModel().xiaomiCredential(); + } + + public FcmV1Credential fcmV1Credential() { + return this.innerModel().fcmV1Credential(); + } + + public Long dailyMaxActiveDevices() { + return this.innerModel().dailyMaxActiveDevices(); + } + public Region region() { return Region.fromName(this.regionName()); } @@ -109,6 +132,10 @@ public String regionName() { return this.location(); } + public String resourceGroupName() { + return resourceGroupName; + } + public NotificationHubResourceInner innerModel() { return this.innerObject; } @@ -123,8 +150,6 @@ private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manag private String notificationHubName; - private NotificationHubCreateOrUpdateParameters createParameters; - private NotificationHubPatchParameters updateParameters; public NotificationHubResourceImpl withExistingNamespace(String resourceGroupName, String namespaceName) { @@ -134,33 +159,24 @@ public NotificationHubResourceImpl withExistingNamespace(String resourceGroupNam } public NotificationHubResource create() { - this.innerObject = - serviceManager - .serviceClient() - .getNotificationHubs() - .createOrUpdateWithResponse( - resourceGroupName, namespaceName, notificationHubName, createParameters, Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getNotificationHubs().createOrUpdateWithResponse(resourceGroupName, + namespaceName, notificationHubName, this.innerModel(), Context.NONE).getValue(); return this; } public NotificationHubResource create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNotificationHubs() - .createOrUpdateWithResponse( - resourceGroupName, namespaceName, notificationHubName, createParameters, context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getNotificationHubs().createOrUpdateWithResponse(resourceGroupName, + namespaceName, notificationHubName, this.innerModel(), context).getValue(); return this; } - NotificationHubResourceImpl( - String name, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + NotificationHubResourceImpl(String name, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = new NotificationHubResourceInner(); this.serviceManager = serviceManager; this.notificationHubName = name; - this.createParameters = new NotificationHubCreateOrUpdateParameters(); } public NotificationHubResourceImpl update() { @@ -169,91 +185,72 @@ public NotificationHubResourceImpl update() { } public NotificationHubResource apply() { - this.innerObject = - serviceManager - .serviceClient() - .getNotificationHubs() - .patchWithResponse( - resourceGroupName, namespaceName, notificationHubName, updateParameters, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNotificationHubs() + .updateWithResponse(resourceGroupName, namespaceName, notificationHubName, updateParameters, Context.NONE) + .getValue(); return this; } public NotificationHubResource apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNotificationHubs() - .patchWithResponse(resourceGroupName, namespaceName, notificationHubName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNotificationHubs() + .updateWithResponse(resourceGroupName, namespaceName, notificationHubName, updateParameters, context) + .getValue(); return this; } - NotificationHubResourceImpl( - NotificationHubResourceInner innerObject, + NotificationHubResourceImpl(NotificationHubResourceInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.namespaceName = Utils.getValueFromIdByName(innerObject.id(), "namespaces"); - this.notificationHubName = Utils.getValueFromIdByName(innerObject.id(), "notificationHubs"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.namespaceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "namespaces"); + this.notificationHubName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "notificationHubs"); } public NotificationHubResource refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getNotificationHubs() - .getWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNotificationHubs() + .getWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE).getValue(); return this; } public NotificationHubResource refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNotificationHubs() - .getWithResponse(resourceGroupName, namespaceName, notificationHubName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNotificationHubs() + .getWithResponse(resourceGroupName, namespaceName, notificationHubName, context).getValue(); return this; } + public Response debugSendWithResponse(Context context) { + return serviceManager.notificationHubs().debugSendWithResponse(resourceGroupName, namespaceName, + notificationHubName, context); + } + public DebugSendResponse debugSend() { return serviceManager.notificationHubs().debugSend(resourceGroupName, namespaceName, notificationHubName); } - public Response debugSendWithResponse(Object parameters, Context context) { - return serviceManager - .notificationHubs() - .debugSendWithResponse(resourceGroupName, namespaceName, notificationHubName, parameters, context); + public Response getPnsCredentialsWithResponse(Context context) { + return serviceManager.notificationHubs().getPnsCredentialsWithResponse(resourceGroupName, namespaceName, + notificationHubName, context); } public PnsCredentialsResource getPnsCredentials() { - return serviceManager - .notificationHubs() - .getPnsCredentials(resourceGroupName, namespaceName, notificationHubName); - } - - public Response getPnsCredentialsWithResponse(Context context) { - return serviceManager - .notificationHubs() - .getPnsCredentialsWithResponse(resourceGroupName, namespaceName, notificationHubName, context); + return serviceManager.notificationHubs().getPnsCredentials(resourceGroupName, namespaceName, + notificationHubName); } public NotificationHubResourceImpl withRegion(Region location) { - this.createParameters.withLocation(location.toString()); + this.innerModel().withLocation(location.toString()); return this; } public NotificationHubResourceImpl withRegion(String location) { - this.createParameters.withLocation(location); + this.innerModel().withLocation(location); return this; } public NotificationHubResourceImpl withTags(Map tags) { if (isInCreateMode()) { - this.createParameters.withTags(tags); + this.innerModel().withTags(tags); return this; } else { this.updateParameters.withTags(tags); @@ -263,7 +260,7 @@ public NotificationHubResourceImpl withTags(Map tags) { public NotificationHubResourceImpl withSku(Sku sku) { if (isInCreateMode()) { - this.createParameters.withSku(sku); + this.innerModel().withSku(sku); return this; } else { this.updateParameters.withSku(sku); @@ -272,18 +269,13 @@ public NotificationHubResourceImpl withSku(Sku sku) { } public NotificationHubResourceImpl withNamePropertiesName(String namePropertiesName) { - if (isInCreateMode()) { - this.createParameters.withNamePropertiesName(namePropertiesName); - return this; - } else { - this.updateParameters.withNamePropertiesName(namePropertiesName); - return this; - } + this.innerModel().withNamePropertiesName(namePropertiesName); + return this; } public NotificationHubResourceImpl withRegistrationTtl(String registrationTtl) { if (isInCreateMode()) { - this.createParameters.withRegistrationTtl(registrationTtl); + this.innerModel().withRegistrationTtl(registrationTtl); return this; } else { this.updateParameters.withRegistrationTtl(registrationTtl); @@ -291,20 +283,9 @@ public NotificationHubResourceImpl withRegistrationTtl(String registrationTtl) { } } - public NotificationHubResourceImpl withAuthorizationRules( - List authorizationRules) { - if (isInCreateMode()) { - this.createParameters.withAuthorizationRules(authorizationRules); - return this; - } else { - this.updateParameters.withAuthorizationRules(authorizationRules); - return this; - } - } - public NotificationHubResourceImpl withApnsCredential(ApnsCredential apnsCredential) { if (isInCreateMode()) { - this.createParameters.withApnsCredential(apnsCredential); + this.innerModel().withApnsCredential(apnsCredential); return this; } else { this.updateParameters.withApnsCredential(apnsCredential); @@ -314,7 +295,7 @@ public NotificationHubResourceImpl withApnsCredential(ApnsCredential apnsCredent public NotificationHubResourceImpl withWnsCredential(WnsCredential wnsCredential) { if (isInCreateMode()) { - this.createParameters.withWnsCredential(wnsCredential); + this.innerModel().withWnsCredential(wnsCredential); return this; } else { this.updateParameters.withWnsCredential(wnsCredential); @@ -324,7 +305,7 @@ public NotificationHubResourceImpl withWnsCredential(WnsCredential wnsCredential public NotificationHubResourceImpl withGcmCredential(GcmCredential gcmCredential) { if (isInCreateMode()) { - this.createParameters.withGcmCredential(gcmCredential); + this.innerModel().withGcmCredential(gcmCredential); return this; } else { this.updateParameters.withGcmCredential(gcmCredential); @@ -334,7 +315,7 @@ public NotificationHubResourceImpl withGcmCredential(GcmCredential gcmCredential public NotificationHubResourceImpl withMpnsCredential(MpnsCredential mpnsCredential) { if (isInCreateMode()) { - this.createParameters.withMpnsCredential(mpnsCredential); + this.innerModel().withMpnsCredential(mpnsCredential); return this; } else { this.updateParameters.withMpnsCredential(mpnsCredential); @@ -344,7 +325,7 @@ public NotificationHubResourceImpl withMpnsCredential(MpnsCredential mpnsCredent public NotificationHubResourceImpl withAdmCredential(AdmCredential admCredential) { if (isInCreateMode()) { - this.createParameters.withAdmCredential(admCredential); + this.innerModel().withAdmCredential(admCredential); return this; } else { this.updateParameters.withAdmCredential(admCredential); @@ -354,7 +335,7 @@ public NotificationHubResourceImpl withAdmCredential(AdmCredential admCredential public NotificationHubResourceImpl withBaiduCredential(BaiduCredential baiduCredential) { if (isInCreateMode()) { - this.createParameters.withBaiduCredential(baiduCredential); + this.innerModel().withBaiduCredential(baiduCredential); return this; } else { this.updateParameters.withBaiduCredential(baiduCredential); @@ -362,6 +343,36 @@ public NotificationHubResourceImpl withBaiduCredential(BaiduCredential baiduCred } } + public NotificationHubResourceImpl withBrowserCredential(BrowserCredential browserCredential) { + if (isInCreateMode()) { + this.innerModel().withBrowserCredential(browserCredential); + return this; + } else { + this.updateParameters.withBrowserCredential(browserCredential); + return this; + } + } + + public NotificationHubResourceImpl withXiaomiCredential(XiaomiCredential xiaomiCredential) { + if (isInCreateMode()) { + this.innerModel().withXiaomiCredential(xiaomiCredential); + return this; + } else { + this.updateParameters.withXiaomiCredential(xiaomiCredential); + return this; + } + } + + public NotificationHubResourceImpl withFcmV1Credential(FcmV1Credential fcmV1Credential) { + if (isInCreateMode()) { + this.innerModel().withFcmV1Credential(fcmV1Credential); + return this; + } else { + this.updateParameters.withFcmV1Credential(fcmV1Credential); + return this; + } + } + private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsClientImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsClientImpl.java index 049041cadf3df..1675515e49a90 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsClientImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsClientImpl.java @@ -30,7 +30,6 @@ import com.azure.core.management.exception.ManagementException; import com.azure.core.util.Context; import com.azure.core.util.FluxUtil; -import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.NotificationHubsClient; import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; import com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResponseInner; @@ -39,318 +38,234 @@ import com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -import com.azure.resourcemanager.notificationhubs.models.NotificationHubCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.NotificationHubListResult; import com.azure.resourcemanager.notificationhubs.models.NotificationHubPatchParameters; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in NotificationHubsClient. */ +/** + * An instance of this class provides access to all the operations defined in NotificationHubsClient. + */ public final class NotificationHubsClientImpl implements NotificationHubsClient { - private final ClientLogger logger = new ClientLogger(NotificationHubsClientImpl.class); - - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final NotificationHubsService service; - /** The service client containing this operation class. */ - private final NotificationHubsManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final NotificationHubsRPClientImpl client; /** * Initializes an instance of NotificationHubsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - NotificationHubsClientImpl(NotificationHubsManagementClientImpl client) { - this.service = - RestProxy.create(NotificationHubsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + NotificationHubsClientImpl(NotificationHubsRPClientImpl client) { + this.service + = RestProxy.create(NotificationHubsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for NotificationHubsManagementClientNotificationHubs to be used by the - * proxy service to perform REST calls. + * The interface defining all the services for NotificationHubsRPClientNotificationHubs to be used by the proxy + * service to perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "NotificationHubsMana") - private interface NotificationHubsService { - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/checkNotificationHubAvailability") - @ExpectedResponses({200}) + @ServiceInterface(name = "NotificationHubsRPCl") + public interface NotificationHubsService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/checkNotificationHubAvailability") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> checkNotificationHubAvailability( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") CheckAvailabilityParameters parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") CheckAvailabilityParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") NotificationHubCreateOrUpdateParameters parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> patch( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") NotificationHubResourceInner parameters, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> update(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") NotificationHubPatchParameters parameters, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, + Mono> list(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("$skipToken") String skipToken, @QueryParam("$top") Integer top, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/debugsend") - @ExpectedResponses({201}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/debugsend") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> debugSend( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, + Mono> debugSend(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") Object parameters, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/AuthorizationRules" - + "/{authorizationRuleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/authorizationRules/{authorizationRuleName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> createOrUpdateAuthorizationRule( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("notificationHubName") String notificationHubName, @PathParam("authorizationRuleName") String authorizationRuleName, @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - @HeaderParam("Accept") String accept, - Context context); + @BodyParam("application/json") SharedAccessAuthorizationRuleResourceInner parameters, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/AuthorizationRules" - + "/{authorizationRuleName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/authorizationRules/{authorizationRuleName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> deleteAuthorizationRule( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + Mono> deleteAuthorizationRule(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("notificationHubName") String notificationHubName, @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/AuthorizationRules" - + "/{authorizationRuleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/authorizationRules/{authorizationRuleName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> getAuthorizationRule( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("notificationHubName") String notificationHubName, @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/AuthorizationRules") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/authorizationRules") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listAuthorizationRules( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/AuthorizationRules" - + "/{authorizationRuleName}/listKeys") - @ExpectedResponses({200}) + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/authorizationRules/{authorizationRuleName}/listKeys") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listKeys( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + Mono> listKeys(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("notificationHubName") String notificationHubName, @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/AuthorizationRules" - + "/{authorizationRuleName}/regenerateKeys") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/authorizationRules/{authorizationRuleName}/regenerateKeys") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> regenerateKeys( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, + Mono> regenerateKeys(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, @PathParam("notificationHubName") String notificationHubName, @PathParam("authorizationRuleName") String authorizationRuleName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") PolicykeyResource parameters, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @BodyParam("application/json") PolicyKeyResource parameters, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs" - + "/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/pnsCredentials") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/notificationHubs/{notificationHubName}/pnsCredentials") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getPnsCredentials( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("namespaceName") String namespaceName, - @PathParam("notificationHubName") String notificationHubName, - @QueryParam("api-version") String apiVersion, + Mono> getPnsCredentials(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("notificationHubName") String notificationHubName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listAuthorizationRulesNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> checkNotificationHubAvailabilityWithResponseAsync( String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -359,12 +274,6 @@ private Mono> checkNotificationHubAvailab if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -372,41 +281,35 @@ private Mono> checkNotificationHubAvailab } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .checkNotificationHubAvailability( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.checkNotificationHubAvailability(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, this.client.getApiVersion(), + parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> checkNotificationHubAvailabilityWithResponseAsync( String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -415,12 +318,6 @@ private Mono> checkNotificationHubAvailab if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -428,102 +325,86 @@ private Mono> checkNotificationHubAvailab } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .checkNotificationHubAvailability( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.checkNotificationHubAvailability(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), parameters, accept, context); } /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono checkNotificationHubAvailabilityAsync( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters) { + private Mono checkNotificationHubAvailabilityAsync(String resourceGroupName, + String namespaceName, CheckAvailabilityParameters parameters) { return checkNotificationHubAvailabilityWithResponseAsync(resourceGroupName, namespaceName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CheckAvailabilityResultInner checkNotificationHubAvailability( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters) { - return checkNotificationHubAvailabilityAsync(resourceGroupName, namespaceName, parameters).block(); + public Response checkNotificationHubAvailabilityWithResponse(String resourceGroupName, + String namespaceName, CheckAvailabilityParameters parameters, Context context) { + return checkNotificationHubAvailabilityWithResponseAsync(resourceGroupName, namespaceName, parameters, context) + .block(); } /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a CheckAvailability resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response checkNotificationHubAvailabilityWithResponse( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters, Context context) { - return checkNotificationHubAvailabilityWithResponseAsync(resourceGroupName, namespaceName, parameters, context) - .block(); + public CheckAvailabilityResultInner checkNotificationHubAvailability(String resourceGroupName, String namespaceName, + CheckAvailabilityParameters parameters) { + return checkNotificationHubAvailabilityWithResponse(resourceGroupName, namespaceName, parameters, Context.NONE) + .getValue(); } /** - * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the notification hub along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters) { + private Mono> getWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -536,60 +417,35 @@ private Mono> createOrUpdateWithResponseA return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the notification hub along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters, - Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -602,130 +458,87 @@ private Mono> createOrUpdateWithResponseA return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, namespaceName, + notificationHubName, this.client.getApiVersion(), accept, context); } /** - * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the notification hub on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters) { - return createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono getAsync(String resourceGroupName, String namespaceName, + String notificationHubName) { + return getWithResponseAsync(resourceGroupName, namespaceName, notificationHubName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the notification hub along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NotificationHubResourceInner createOrUpdate( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters) { - return createOrUpdateAsync(resourceGroupName, namespaceName, notificationHubName, parameters).block(); + public Response getWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context) { + return getWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context).block(); } /** - * Creates/Update a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to the create/update a NotificationHub Resource. - * @param context The context to associate with this operation. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the notification hub. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubCreateOrUpdateParameters parameters, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, parameters, context) - .block(); + public NotificationHubResourceInner get(String resourceGroupName, String namespaceName, + String notificationHubName) { + return getWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE).getValue(); } /** - * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to patch a NotificationHub Resource. + * Creates/Update a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> patchWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubPatchParameters parameters) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, NotificationHubResourceInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -738,58 +551,42 @@ private Mono> patchWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters != null) { + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { parameters.validate(); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .patch( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, this.client.getApiVersion(), parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to patch a NotificationHub Resource. + * Creates/Update a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> patchWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubPatchParameters parameters, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, NotificationHubResourceInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -802,146 +599,97 @@ private Mono> patchWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters != null) { + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { parameters.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .patch( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), parameters, accept, context); } /** - * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to patch a NotificationHub Resource. + * Creates/Update a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono patchAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubPatchParameters parameters) { - return patchWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono createOrUpdateAsync(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubResourceInner parameters) { + return createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Update a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono patchAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { - final NotificationHubPatchParameters parameters = null; - return patchWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + public Response createOrUpdateWithResponse(String resourceGroupName, + String namespaceName, String notificationHubName, NotificationHubResourceInner parameters, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters, + context).block(); } /** - * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Update a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NotificationHubResourceInner patch( - String resourceGroupName, String namespaceName, String notificationHubName) { - final NotificationHubPatchParameters parameters = null; - return patchAsync(resourceGroupName, namespaceName, notificationHubName, parameters).block(); + public NotificationHubResourceInner createOrUpdate(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubResourceInner parameters) { + return createOrUpdateWithResponse(resourceGroupName, namespaceName, notificationHubName, parameters, + Context.NONE).getValue(); } /** * Patch a NotificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Parameters supplied to patch a NotificationHub Resource. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return notification Hub Resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response patchWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - NotificationHubPatchParameters parameters, - Context context) { - return patchWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters, context) - .block(); - } - - /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, NotificationHubPatchParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -954,47 +702,42 @@ private Mono> deleteWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); } + final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context)) + context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Patch a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hub Resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, NotificationHubPatchParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1007,93 +750,96 @@ private Mono> deleteWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), parameters, accept, context); } /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Patch a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return notification Hub Resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String namespaceName, String notificationHubName) { - return deleteWithResponseAsync(resourceGroupName, namespaceName, notificationHubName) - .flatMap((Response res) -> Mono.empty()); + private Mono updateAsync(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubPatchParameters parameters) { + return updateWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Patch a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hub Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String namespaceName, String notificationHubName) { - deleteAsync(resourceGroupName, namespaceName, notificationHubName).block(); + public Response updateWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubPatchParameters parameters, Context context) { + return updateWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters, context) + .block(); } /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param context The context to associate with this operation. + * Patch a NotificationHub in a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return notification Hub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - return deleteWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context).block(); + public NotificationHubResourceInner update(String resourceGroupName, String namespaceName, + String notificationHubName, NotificationHubPatchParameters parameters) { + return updateWithResponse(resourceGroupName, namespaceName, notificationHubName, parameters, Context.NONE) + .getValue(); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String namespaceName, + String notificationHubName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1106,49 +852,35 @@ private Mono> getWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String namespaceName, + String notificationHubName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1161,106 +893,85 @@ private Mono> getWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), accept, context); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { - return getWithResponseAsync(resourceGroupName, namespaceName, notificationHubName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono deleteAsync(String resourceGroupName, String namespaceName, String notificationHubName) { + return deleteWithResponseAsync(resourceGroupName, namespaceName, notificationHubName) + .flatMap(ignored -> Mono.empty()); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public NotificationHubResourceInner get( - String resourceGroupName, String namespaceName, String notificationHubName) { - return getAsync(resourceGroupName, namespaceName, notificationHubName).block(); + public Response deleteWithResponse(String resourceGroupName, String namespaceName, String notificationHubName, + Context context) { + return deleteWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context).block(); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param context The context to associate with this operation. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - return getWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context).block(); + public void delete(String resourceGroupName, String namespaceName, String notificationHubName) { + deleteWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE); } /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Debug send parameters. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> debugSendWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, Object parameters) { + private Mono> listSinglePageAsync(String resourceGroupName, + String namespaceName, String skipToken, Integer top) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1269,59 +980,39 @@ private Mono> debugSendWithResponseAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (notificationHubName == null) { - return Mono - .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .debugSend( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, skipToken, top, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Debug send parameters. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> debugSendWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - Object parameters, - Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String namespaceName, String skipToken, Integer top, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1330,150 +1021,130 @@ private Mono> debugSendWithResponseAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (notificationHubName == null) { - return Mono - .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .debugSend( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, namespaceName, + skipToken, top, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Debug send parameters. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono debugSendAsync( - String resourceGroupName, String namespaceName, String notificationHubName, Object parameters) { - return debugSendWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String namespaceName, + String skipToken, Integer top) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, namespaceName, skipToken, top), + nextLink -> listNextSinglePageAsync(nextLink)); } /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono debugSendAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { - final Object parameters = null; - return debugSendWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String namespaceName) { + final String skipToken = null; + final Integer top = null; + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, namespaceName, skipToken, top), + nextLink -> listNextSinglePageAsync(nextLink)); } /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedFlux}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public DebugSendResponseInner debugSend( - String resourceGroupName, String namespaceName, String notificationHubName) { - final Object parameters = null; - return debugSendAsync(resourceGroupName, namespaceName, notificationHubName, parameters).block(); + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String namespaceName, + String skipToken, Integer top, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, namespaceName, skipToken, top, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String namespaceName) { + final String skipToken = null; + final Integer top = null; + return new PagedIterable<>(listAsync(resourceGroupName, namespaceName, skipToken, top)); } /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Debug send parameters. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedIterable}. */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response debugSendWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - Object parameters, - Context context) { - return debugSendWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, parameters, context) - .block(); + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String namespaceName, + String skipToken, Integer top, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, namespaceName, skipToken, top, context)); } /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { + private Mono> debugSendWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1486,67 +1157,36 @@ private Mono> createOrUpdat return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (authorizationRuleName == null) { - return Mono - .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdateAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.debugSend(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context) { + private Mono> debugSendWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1559,141 +1199,90 @@ private Mono> createOrUpdat return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (authorizationRuleName == null) { - return Mono - .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (parameters == null) { - return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); - } else { - parameters.validate(); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdateAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.debugSend(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), accept, context); } /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAuthorizationRuleAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { - return createOrUpdateAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono debugSendAsync(String resourceGroupName, String namespaceName, + String notificationHubName) { + return debugSendWithResponseAsync(resourceGroupName, namespaceName, notificationHubName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { - return createOrUpdateAuthorizationRuleAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters) - .block(); + public Response debugSendWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context) { + return debugSendWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context).block(); } /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. - * @param context The context to associate with this operation. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context) { - return createOrUpdateAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters, context) - .block(); + public DebugSendResponseInner debugSend(String resourceGroupName, String namespaceName, + String notificationHubName) { + return debugSendWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE).getValue(); } /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Updates an authorization rule for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response} + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteAuthorizationRuleWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { + private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1710,53 +1299,45 @@ private Mono> deleteAuthorizationRuleWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); } + final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .deleteAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context)) + .withContext(context -> service.createOrUpdateAuthorizationRule(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Updates an authorization rule for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response} + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteAuthorizationRuleWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { + private Mono> createOrUpdateAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1773,108 +1354,105 @@ private Mono> deleteAuthorizationRuleWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); } + final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .deleteAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - context); + return service.createOrUpdateAuthorizationRule(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, this.client.getApiVersion(), + parameters, accept, context); } /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Updates an authorization rule for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the completion. + * @return response for POST requests that return single SharedAccessAuthorizationRule on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAuthorizationRuleAsync( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - return deleteAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName) - .flatMap((Response res) -> Mono.empty()); + private Mono createOrUpdateAuthorizationRuleAsync( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters) { + return createOrUpdateAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Updates an authorization rule for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void deleteAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - deleteAuthorizationRuleAsync(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName) - .block(); + public Response createOrUpdateAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context) { + return createOrUpdateAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, parameters, context).block(); } /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Creates/Updates an authorization rule for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. - * @param context The context to associate with this operation. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return response for POST requests that return single SharedAccessAuthorizationRule. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { - return deleteAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, context) - .block(); + public SharedAccessAuthorizationRuleResourceInner createOrUpdateAuthorizationRule(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters) { + return createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, parameters, Context.NONE).getValue(); } /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthorizationRuleWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { + private Mono> deleteAuthorizationRuleWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1891,55 +1469,37 @@ private Mono> getAuthorizat return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.deleteAuthorizationRule(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getAuthorizationRuleWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { + private Mono> deleteAuthorizationRuleWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1956,116 +1516,93 @@ private Mono> getAuthorizat return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getAuthorizationRule( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.deleteAuthorizationRule(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, this.client.getApiVersion(), + accept, context); } /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAuthorizationRuleAsync( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - return getAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono deleteAuthorizationRuleAsync(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName) { + return deleteAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName).flatMap(ignored -> Mono.empty()); } /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SharedAccessAuthorizationRuleResourceInner getAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - return getAuthorizationRuleAsync(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName) - .block(); + public Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context) { + return deleteAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, context).block(); } /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. - * @param context The context to associate with this operation. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { - return getAuthorizationRuleWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, context) - .block(); + public void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName) { + deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, Context.NONE); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return an authorization rule for a NotificationHub by name along with {@link Response} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String namespaceName) { + private Mono> getAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2074,56 +1611,47 @@ private Mono> listSinglePageAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { + if (notificationHubName == null) { + return Mono + .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); + } + if (authorizationRuleName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.getAuthorizationRule(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return an authorization rule for a NotificationHub by name along with {@link Response} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String namespaceName, Context context) { + private Mono> getAuthorizationRuleWithResponseAsync( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2132,120 +1660,102 @@ private Mono> listSinglePageAsync( if (namespaceName == null) { return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { + if (notificationHubName == null) { + return Mono + .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); + } + if (authorizationRuleName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.getAuthorizationRule(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, this.client.getApiVersion(), + accept, context); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return an authorization rule for a NotificationHub by name on successful completion of {@link Mono}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String resourceGroupName, String namespaceName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, namespaceName), nextLink -> listNextSinglePageAsync(nextLink)); + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAuthorizationRuleAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName) { + return getAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String namespaceName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, namespaceName, context), - nextLink -> listNextSinglePageAsync(nextLink, context)); - } - - /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return an authorization rule for a NotificationHub by name along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String resourceGroupName, String namespaceName) { - return new PagedIterable<>(listAsync(resourceGroupName, namespaceName)); + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName, + Context context) { + return getAuthorizationRuleWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, context).block(); } /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return an authorization rule for a NotificationHub by name. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String namespaceName, Context context) { - return new PagedIterable<>(listAsync(resourceGroupName, namespaceName, context)); + @ServiceMethod(returns = ReturnType.SINGLE) + public SharedAccessAuthorizationRuleResourceInner getAuthorizationRule(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName) { + return getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, Context.NONE).getValue(); } /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAuthorizationRulesSinglePageAsync( String resourceGroupName, String namespaceName, String notificationHubName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2258,58 +1768,40 @@ private Mono> listAuth return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listAuthorizationRules( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.listAuthorizationRules(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, notificationHubName, + this.client.getApiVersion(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub along with {@link PagedResponse} on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAuthorizationRulesSinglePageAsync( String resourceGroupName, String namespaceName, String notificationHubName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2322,49 +1814,29 @@ private Mono> listAuth return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listAuthorizationRules( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listAuthorizationRules(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAuthorizationRulesAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { + private PagedFlux listAuthorizationRulesAsync(String resourceGroupName, + String namespaceName, String notificationHubName) { return new PagedFlux<>( () -> listAuthorizationRulesSinglePageAsync(resourceGroupName, namespaceName, notificationHubName), nextLink -> listAuthorizationRulesNextSinglePageAsync(nextLink)); @@ -2372,19 +1844,19 @@ private PagedFlux listAuthorizationR /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAuthorizationRulesAsync( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { + private PagedFlux listAuthorizationRulesAsync(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { return new PagedFlux<>( () -> listAuthorizationRulesSinglePageAsync(resourceGroupName, namespaceName, notificationHubName, context), nextLink -> listAuthorizationRulesNextSinglePageAsync(nextLink, context)); @@ -2392,60 +1864,63 @@ private PagedFlux listAuthorizationR /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName) { + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName) { return new PagedIterable<>(listAuthorizationRulesAsync(resourceGroupName, namespaceName, notificationHubName)); } /** * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { return new PagedIterable<>( listAuthorizationRulesAsync(resourceGroupName, namespaceName, notificationHubName, context)); } /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub along with {@link Response} on + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listKeysWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { + private Mono> listKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2462,55 +1937,38 @@ private Mono> listKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.listKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub along with {@link Response} on + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listKeysWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { + private Mono> listKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2527,121 +1985,95 @@ private Mono> listKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.listKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, this.client.getApiVersion(), accept, context); } /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listKeysAsync( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { + private Mono listKeysAsync(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName) { return listKeysWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ResourceListKeysInner listKeys( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - return listKeysAsync(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName).block(); + public Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context) { + return listKeysWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, + context).block(); } /** * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Primary and Secondary ConnectionStrings to the NotificationHub. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { - return listKeysWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, context) - .block(); + public ResourceListKeysInner listKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName) { + return listKeysWithResponse(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, + Context.NONE).getValue(); } /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> regenerateKeysWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters) { + private Mono> regenerateKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2658,12 +2090,6 @@ private Mono> regenerateKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -2671,50 +2097,38 @@ private Mono> regenerateKeysWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .regenerateKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.regenerateKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, + this.client.getApiVersion(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> regenerateKeysWithResponseAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters, + private Mono> regenerateKeysWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2731,12 +2145,6 @@ private Mono> regenerateKeysWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter authorizationRuleName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); } else { @@ -2744,123 +2152,96 @@ private Mono> regenerateKeysWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .regenerateKeys( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - authorizationRuleName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.regenerateKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, this.client.getApiVersion(), parameters, accept, + context); } /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono regenerateKeysAsync( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters) { - return regenerateKeysWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + private Mono regenerateKeysAsync(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters) { + return regenerateKeysWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, parameters).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ResourceListKeysInner regenerateKeys( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters) { - return regenerateKeysAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters) - .block(); + public Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters, Context context) { + return regenerateKeysWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, parameters, context).block(); } /** * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context) { - return regenerateKeysWithResponseAsync( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters, context) - .block(); + public ResourceListKeysInner regenerateKeys(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters) { + return regenerateKeysWithResponse(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, + parameters, Context.NONE).getValue(); } /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return description of a NotificationHub PNS Credentials along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getPnsCredentialsWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { + private Mono> getPnsCredentialsWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2873,49 +2254,37 @@ private Mono> getPnsCredentialsWithRespons return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getPnsCredentials( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.getPnsCredentials(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, namespaceName, notificationHubName, + this.client.getApiVersion(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return description of a NotificationHub PNS Credentials along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getPnsCredentialsWithResponseAsync( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { + private Mono> getPnsCredentialsWithResponseAsync(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2928,95 +2297,78 @@ private Mono> getPnsCredentialsWithRespons return Mono .error(new IllegalArgumentException("Parameter notificationHubName is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getPnsCredentials( - this.client.getEndpoint(), - resourceGroupName, - namespaceName, - notificationHubName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.getPnsCredentials(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, notificationHubName, this.client.getApiVersion(), accept, context); } /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return description of a NotificationHub PNS Credentials on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getPnsCredentialsAsync( - String resourceGroupName, String namespaceName, String notificationHubName) { + private Mono getPnsCredentialsAsync(String resourceGroupName, String namespaceName, + String notificationHubName) { return getPnsCredentialsWithResponseAsync(resourceGroupName, namespaceName, notificationHubName) - .flatMap( - (Response res) -> { - if (res.getValue() != null) { - return Mono.just(res.getValue()); - } else { - return Mono.empty(); - } - }); + .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PnsCredentialsResourceInner getPnsCredentials( - String resourceGroupName, String namespaceName, String notificationHubName) { - return getPnsCredentialsAsync(resourceGroupName, namespaceName, notificationHubName).block(); + public Response getPnsCredentialsWithResponse(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { + return getPnsCredentialsWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context) + .block(); } /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param context The context to associate with this operation. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a NotificationHub PNS Credentials. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getPnsCredentialsWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - return getPnsCredentialsWithResponseAsync(resourceGroupName, namespaceName, notificationHubName, context) - .block(); + public PnsCredentialsResourceInner getPnsCredentials(String resourceGroupName, String namespaceName, + String notificationHubName) { + return getPnsCredentialsWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE) + .getValue(); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return the response of the List NotificationHub operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -3024,134 +2376,105 @@ private Mono> listNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return the response of the List NotificationHub operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAuthorizationRulesNextSinglePageAsync( - String nextLink) { + private Mono> + listAuthorizationRulesNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listAuthorizationRulesNext(nextLink, this.client.getEndpoint(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the response of the List Namespace operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAuthorizationRulesNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listAuthorizationRulesNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAuthorizationRulesNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAuthorizationRulesNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsImpl.java index c666479008319..a3ca895aa5ee4 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsImpl.java @@ -22,454 +22,409 @@ import com.azure.resourcemanager.notificationhubs.models.NotificationHubResource; import com.azure.resourcemanager.notificationhubs.models.NotificationHubs; import com.azure.resourcemanager.notificationhubs.models.PnsCredentialsResource; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; import com.azure.resourcemanager.notificationhubs.models.ResourceListKeys; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleResource; -import com.fasterxml.jackson.annotation.JsonIgnore; public final class NotificationHubsImpl implements NotificationHubs { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NotificationHubsImpl.class); + private static final ClientLogger LOGGER = new ClientLogger(NotificationHubsImpl.class); private final NotificationHubsClient innerClient; private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - public NotificationHubsImpl( - NotificationHubsClient innerClient, + public NotificationHubsImpl(NotificationHubsClient innerClient, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public CheckAvailabilityResult checkNotificationHubAvailability( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters) { - CheckAvailabilityResultInner inner = - this.serviceClient().checkNotificationHubAvailability(resourceGroupName, namespaceName, parameters); + public Response checkNotificationHubAvailabilityWithResponse(String resourceGroupName, + String namespaceName, CheckAvailabilityParameters parameters, Context context) { + Response inner = this.serviceClient() + .checkNotificationHubAvailabilityWithResponse(resourceGroupName, namespaceName, parameters, context); if (inner != null) { - return new CheckAvailabilityResultImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response checkNotificationHubAvailabilityWithResponse( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters, Context context) { - Response inner = - this - .serviceClient() - .checkNotificationHubAvailabilityWithResponse(resourceGroupName, namespaceName, parameters, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CheckAvailabilityResultImpl(inner.getValue(), this.manager())); } else { return null; } } - public void delete(String resourceGroupName, String namespaceName, String notificationHubName) { - this.serviceClient().delete(resourceGroupName, namespaceName, notificationHubName); - } - - public Response deleteWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - return this.serviceClient().deleteWithResponse(resourceGroupName, namespaceName, notificationHubName, context); - } - - public NotificationHubResource get(String resourceGroupName, String namespaceName, String notificationHubName) { - NotificationHubResourceInner inner = - this.serviceClient().get(resourceGroupName, namespaceName, notificationHubName); + public CheckAvailabilityResult checkNotificationHubAvailability(String resourceGroupName, String namespaceName, + CheckAvailabilityParameters parameters) { + CheckAvailabilityResultInner inner + = this.serviceClient().checkNotificationHubAvailability(resourceGroupName, namespaceName, parameters); if (inner != null) { - return new NotificationHubResourceImpl(inner, this.manager()); + return new CheckAvailabilityResultImpl(inner, this.manager()); } else { return null; } } - public Response getWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, namespaceName, notificationHubName, context); + public Response getWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, namespaceName, notificationHubName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new NotificationHubResourceImpl(inner.getValue(), this.manager())); } else { return null; } } - public DebugSendResponse debugSend(String resourceGroupName, String namespaceName, String notificationHubName) { - DebugSendResponseInner inner = - this.serviceClient().debugSend(resourceGroupName, namespaceName, notificationHubName); + public NotificationHubResource get(String resourceGroupName, String namespaceName, String notificationHubName) { + NotificationHubResourceInner inner + = this.serviceClient().get(resourceGroupName, namespaceName, notificationHubName); if (inner != null) { - return new DebugSendResponseImpl(inner, this.manager()); + return new NotificationHubResourceImpl(inner, this.manager()); } else { return null; } } - public Response debugSendWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - Object parameters, + public Response deleteWithResponse(String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - Response inner = - this - .serviceClient() - .debugSendWithResponse(resourceGroupName, namespaceName, notificationHubName, parameters, context); + return this.serviceClient().deleteWithResponse(resourceGroupName, namespaceName, notificationHubName, context); + } + + public void delete(String resourceGroupName, String namespaceName, String notificationHubName) { + this.serviceClient().delete(resourceGroupName, namespaceName, notificationHubName); + } + + public PagedIterable list(String resourceGroupName, String namespaceName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, namespaceName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NotificationHubResourceImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String namespaceName, String skipToken, + Integer top, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, namespaceName, skipToken, top, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new NotificationHubResourceImpl(inner1, this.manager())); + } + + public Response debugSendWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context) { + Response inner = this.serviceClient().debugSendWithResponse(resourceGroupName, + namespaceName, notificationHubName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DebugSendResponseImpl(inner.getValue(), this.manager())); } else { return null; } } - public SharedAccessAuthorizationRuleResource createOrUpdateAuthorizationRule( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters) { - SharedAccessAuthorizationRuleResourceInner inner = - this - .serviceClient() - .createOrUpdateAuthorizationRule( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters); + public DebugSendResponse debugSend(String resourceGroupName, String namespaceName, String notificationHubName) { + DebugSendResponseInner inner + = this.serviceClient().debugSend(resourceGroupName, namespaceName, notificationHubName); if (inner != null) { - return new SharedAccessAuthorizationRuleResourceImpl(inner, this.manager()); + return new DebugSendResponseImpl(inner, this.manager()); } else { return null; } } - public Response createOrUpdateAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new SharedAccessAuthorizationRuleResourceImpl(inner.getValue(), this.manager())); - } else { - return null; - } + public Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context) { + return this.serviceClient().deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, + notificationHubName, authorizationRuleName, context); } - public void deleteAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - this - .serviceClient() - .deleteAuthorizationRule(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName); + public void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName) { + this.serviceClient().deleteAuthorizationRule(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName); } - public Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { - return this - .serviceClient() - .deleteAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, context); - } - - public SharedAccessAuthorizationRuleResource getAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - SharedAccessAuthorizationRuleResourceInner inner = - this - .serviceClient() - .getAuthorizationRule(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName); + public Response getAuthorizationRuleWithResponse(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, Context context) { + Response inner + = this.serviceClient().getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, + notificationHubName, authorizationRuleName, context); if (inner != null) { - return new SharedAccessAuthorizationRuleResourceImpl(inner, this.manager()); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new SharedAccessAuthorizationRuleResourceImpl(inner.getValue(), this.manager())); } else { return null; } } - public Response getAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { - Response inner = - this - .serviceClient() - .getAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, context); + public SharedAccessAuthorizationRuleResource getAuthorizationRule(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName) { + SharedAccessAuthorizationRuleResourceInner inner = this.serviceClient().getAuthorizationRule(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new SharedAccessAuthorizationRuleResourceImpl(inner.getValue(), this.manager())); + return new SharedAccessAuthorizationRuleResourceImpl(inner, this.manager()); } else { return null; } } - public PagedIterable list(String resourceGroupName, String namespaceName) { - PagedIterable inner = this.serviceClient().list(resourceGroupName, namespaceName); - return Utils.mapPage(inner, inner1 -> new NotificationHubResourceImpl(inner1, this.manager())); - } - - public PagedIterable list( - String resourceGroupName, String namespaceName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, namespaceName, context); - return Utils.mapPage(inner, inner1 -> new NotificationHubResourceImpl(inner1, this.manager())); - } - - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName) { - PagedIterable inner = - this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName, notificationHubName); - return Utils.mapPage(inner, inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName) { + PagedIterable inner + = this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName, notificationHubName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); } - public PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - PagedIterable inner = - this.serviceClient().listAuthorizationRules(resourceGroupName, namespaceName, notificationHubName, context); - return Utils.mapPage(inner, inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); + public PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { + PagedIterable inner = this.serviceClient() + .listAuthorizationRules(resourceGroupName, namespaceName, notificationHubName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SharedAccessAuthorizationRuleResourceImpl(inner1, this.manager())); } - public ResourceListKeys listKeys( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName) { - ResourceListKeysInner inner = - this.serviceClient().listKeys(resourceGroupName, namespaceName, notificationHubName, authorizationRuleName); + public Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context) { + Response inner = this.serviceClient().listKeysWithResponse(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, context); if (inner != null) { - return new ResourceListKeysImpl(inner, this.manager()); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ResourceListKeysImpl(inner.getValue(), this.manager())); } else { return null; } } - public Response listKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context) { - Response inner = - this - .serviceClient() - .listKeysWithResponse( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, context); + public ResourceListKeys listKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName) { + ResourceListKeysInner inner = this.serviceClient().listKeys(resourceGroupName, namespaceName, + notificationHubName, authorizationRuleName); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ResourceListKeysImpl(inner.getValue(), this.manager())); + return new ResourceListKeysImpl(inner, this.manager()); } else { return null; } } - public ResourceListKeys regenerateKeys( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters) { - ResourceListKeysInner inner = - this - .serviceClient() - .regenerateKeys( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters); + public Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters, Context context) { + Response inner = this.serviceClient().regenerateKeysWithResponse(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, parameters, context); if (inner != null) { - return new ResourceListKeysImpl(inner, this.manager()); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ResourceListKeysImpl(inner.getValue(), this.manager())); } else { return null; } } - public Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context) { - Response inner = - this - .serviceClient() - .regenerateKeysWithResponse( - resourceGroupName, namespaceName, notificationHubName, authorizationRuleName, parameters, context); + public ResourceListKeys regenerateKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName, PolicyKeyResource parameters) { + ResourceListKeysInner inner = this.serviceClient().regenerateKeys(resourceGroupName, namespaceName, + notificationHubName, authorizationRuleName, parameters); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ResourceListKeysImpl(inner.getValue(), this.manager())); + return new ResourceListKeysImpl(inner, this.manager()); } else { return null; } } - public PnsCredentialsResource getPnsCredentials( - String resourceGroupName, String namespaceName, String notificationHubName) { - PnsCredentialsResourceInner inner = - this.serviceClient().getPnsCredentials(resourceGroupName, namespaceName, notificationHubName); + public Response getPnsCredentialsWithResponse(String resourceGroupName, + String namespaceName, String notificationHubName, Context context) { + Response inner = this.serviceClient() + .getPnsCredentialsWithResponse(resourceGroupName, namespaceName, notificationHubName, context); if (inner != null) { - return new PnsCredentialsResourceImpl(inner, this.manager()); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new PnsCredentialsResourceImpl(inner.getValue(), this.manager())); } else { return null; } } - public Response getPnsCredentialsWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context) { - Response inner = - this - .serviceClient() - .getPnsCredentialsWithResponse(resourceGroupName, namespaceName, notificationHubName, context); + public PnsCredentialsResource getPnsCredentials(String resourceGroupName, String namespaceName, + String notificationHubName) { + PnsCredentialsResourceInner inner + = this.serviceClient().getPnsCredentials(resourceGroupName, namespaceName, notificationHubName); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new PnsCredentialsResourceImpl(inner.getValue(), this.manager())); + return new PnsCredentialsResourceImpl(inner, this.manager()); } else { return null; } } public NotificationHubResource getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } - String notificationHubName = Utils.getValueFromIdByName(id, "notificationHubs"); + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); if (notificationHubName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); } return this.getWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } - String notificationHubName = Utils.getValueFromIdByName(id, "notificationHubs"); + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); if (notificationHubName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); } return this.getWithResponse(resourceGroupName, namespaceName, notificationHubName, context); } + public SharedAccessAuthorizationRuleResource getAuthorizationRuleById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); + if (namespaceName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + } + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); + if (notificationHubName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + } + String authorizationRuleName = ResourceManagerUtils.getValueFromIdByName(id, "authorizationRules"); + if (authorizationRuleName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authorizationRules'.", id))); + } + return this.getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, Context.NONE).getValue(); + } + + public Response getAuthorizationRuleByIdWithResponse(String id, + Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); + if (namespaceName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + } + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); + if (notificationHubName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + } + String authorizationRuleName = ResourceManagerUtils.getValueFromIdByName(id, "authorizationRules"); + if (authorizationRuleName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authorizationRules'.", id))); + } + return this.getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, context); + } + public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } - String notificationHubName = Utils.getValueFromIdByName(id, "notificationHubs"); + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); if (notificationHubName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); } this.deleteWithResponse(resourceGroupName, namespaceName, notificationHubName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String namespaceName = Utils.getValueFromIdByName(id, "namespaces"); + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); if (namespaceName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); } - String notificationHubName = Utils.getValueFromIdByName(id, "notificationHubs"); + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); if (notificationHubName == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); } return this.deleteWithResponse(resourceGroupName, namespaceName, notificationHubName, context); } + public void deleteAuthorizationRuleById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); + if (namespaceName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + } + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); + if (notificationHubName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + } + String authorizationRuleName = ResourceManagerUtils.getValueFromIdByName(id, "authorizationRules"); + if (authorizationRuleName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authorizationRules'.", id))); + } + this.deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, Context.NONE); + } + + public Response deleteAuthorizationRuleByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String namespaceName = ResourceManagerUtils.getValueFromIdByName(id, "namespaces"); + if (namespaceName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'namespaces'.", id))); + } + String notificationHubName = ResourceManagerUtils.getValueFromIdByName(id, "notificationHubs"); + if (notificationHubName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'notificationHubs'.", id))); + } + String authorizationRuleName = ResourceManagerUtils.getValueFromIdByName(id, "authorizationRules"); + if (authorizationRuleName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'authorizationRules'.", id))); + } + return this.deleteAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, context); + } + private NotificationHubsClient serviceClient() { return this.innerClient; } @@ -481,4 +436,8 @@ private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manag public NotificationHubResourceImpl define(String name) { return new NotificationHubResourceImpl(name, this.manager()); } + + public SharedAccessAuthorizationRuleResourceImpl defineAuthorizationRule(String name) { + return new SharedAccessAuthorizationRuleResourceImpl(name, this.manager()); + } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientBuilder.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientBuilder.java deleted file mode 100644 index 20e35f695c917..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientBuilder.java +++ /dev/null @@ -1,149 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.implementation; - -import com.azure.core.annotation.ServiceClientBuilder; -import com.azure.core.http.HttpPipeline; -import com.azure.core.http.HttpPipelineBuilder; -import com.azure.core.http.policy.CookiePolicy; -import com.azure.core.http.policy.RetryPolicy; -import com.azure.core.http.policy.UserAgentPolicy; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.serializer.SerializerAdapter; -import java.time.Duration; - -/** A builder for creating a new instance of the NotificationHubsManagementClientImpl type. */ -@ServiceClientBuilder(serviceClients = {NotificationHubsManagementClientImpl.class}) -public final class NotificationHubsManagementClientBuilder { - /* - * Gets subscription credentials which uniquely identify Microsoft Azure - * subscription. The subscription ID forms part of the URI for every - * service call. - */ - private String subscriptionId; - - /** - * Sets Gets subscription credentials which uniquely identify Microsoft Azure subscription. The subscription ID - * forms part of the URI for every service call. - * - * @param subscriptionId the subscriptionId value. - * @return the NotificationHubsManagementClientBuilder. - */ - public NotificationHubsManagementClientBuilder subscriptionId(String subscriptionId) { - this.subscriptionId = subscriptionId; - return this; - } - - /* - * server parameter - */ - private String endpoint; - - /** - * Sets server parameter. - * - * @param endpoint the endpoint value. - * @return the NotificationHubsManagementClientBuilder. - */ - public NotificationHubsManagementClientBuilder endpoint(String endpoint) { - this.endpoint = endpoint; - return this; - } - - /* - * The environment to connect to - */ - private AzureEnvironment environment; - - /** - * Sets The environment to connect to. - * - * @param environment the environment value. - * @return the NotificationHubsManagementClientBuilder. - */ - public NotificationHubsManagementClientBuilder environment(AzureEnvironment environment) { - this.environment = environment; - return this; - } - - /* - * The default poll interval for long-running operation - */ - private Duration defaultPollInterval; - - /** - * Sets The default poll interval for long-running operation. - * - * @param defaultPollInterval the defaultPollInterval value. - * @return the NotificationHubsManagementClientBuilder. - */ - public NotificationHubsManagementClientBuilder defaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = defaultPollInterval; - return this; - } - - /* - * The HTTP pipeline to send requests through - */ - private HttpPipeline pipeline; - - /** - * Sets The HTTP pipeline to send requests through. - * - * @param pipeline the pipeline value. - * @return the NotificationHubsManagementClientBuilder. - */ - public NotificationHubsManagementClientBuilder pipeline(HttpPipeline pipeline) { - this.pipeline = pipeline; - return this; - } - - /* - * The serializer to serialize an object into a string - */ - private SerializerAdapter serializerAdapter; - - /** - * Sets The serializer to serialize an object into a string. - * - * @param serializerAdapter the serializerAdapter value. - * @return the NotificationHubsManagementClientBuilder. - */ - public NotificationHubsManagementClientBuilder serializerAdapter(SerializerAdapter serializerAdapter) { - this.serializerAdapter = serializerAdapter; - return this; - } - - /** - * Builds an instance of NotificationHubsManagementClientImpl with the provided parameters. - * - * @return an instance of NotificationHubsManagementClientImpl. - */ - public NotificationHubsManagementClientImpl buildClient() { - if (endpoint == null) { - this.endpoint = "https://management.azure.com"; - } - if (environment == null) { - this.environment = AzureEnvironment.AZURE; - } - if (defaultPollInterval == null) { - this.defaultPollInterval = Duration.ofSeconds(30); - } - if (pipeline == null) { - this.pipeline = - new HttpPipelineBuilder() - .policies(new UserAgentPolicy(), new RetryPolicy(), new CookiePolicy()) - .build(); - } - if (serializerAdapter == null) { - this.serializerAdapter = SerializerFactory.createDefaultManagementSerializerAdapter(); - } - NotificationHubsManagementClientImpl client = - new NotificationHubsManagementClientImpl( - pipeline, serializerAdapter, defaultPollInterval, environment, subscriptionId, endpoint); - return client; - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientBuilder.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientBuilder.java new file mode 100644 index 0000000000000..bb8c45151f274 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientBuilder.java @@ -0,0 +1,136 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.implementation; + +import com.azure.core.annotation.ServiceClientBuilder; +import com.azure.core.http.HttpPipeline; +import com.azure.core.http.HttpPipelineBuilder; +import com.azure.core.http.policy.RetryPolicy; +import com.azure.core.http.policy.UserAgentPolicy; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.serializer.SerializerFactory; +import com.azure.core.util.serializer.SerializerAdapter; +import java.time.Duration; + +/** + * A builder for creating a new instance of the NotificationHubsRPClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { NotificationHubsRPClientImpl.class }) +public final class NotificationHubsRPClientBuilder { + /* + * The ID of the target subscription. The value must be an UUID. + */ + private String subscriptionId; + + /** + * Sets The ID of the target subscription. The value must be an UUID. + * + * @param subscriptionId the subscriptionId value. + * @return the NotificationHubsRPClientBuilder. + */ + public NotificationHubsRPClientBuilder subscriptionId(String subscriptionId) { + this.subscriptionId = subscriptionId; + return this; + } + + /* + * server parameter + */ + private String endpoint; + + /** + * Sets server parameter. + * + * @param endpoint the endpoint value. + * @return the NotificationHubsRPClientBuilder. + */ + public NotificationHubsRPClientBuilder endpoint(String endpoint) { + this.endpoint = endpoint; + return this; + } + + /* + * The environment to connect to + */ + private AzureEnvironment environment; + + /** + * Sets The environment to connect to. + * + * @param environment the environment value. + * @return the NotificationHubsRPClientBuilder. + */ + public NotificationHubsRPClientBuilder environment(AzureEnvironment environment) { + this.environment = environment; + return this; + } + + /* + * The HTTP pipeline to send requests through + */ + private HttpPipeline pipeline; + + /** + * Sets The HTTP pipeline to send requests through. + * + * @param pipeline the pipeline value. + * @return the NotificationHubsRPClientBuilder. + */ + public NotificationHubsRPClientBuilder pipeline(HttpPipeline pipeline) { + this.pipeline = pipeline; + return this; + } + + /* + * The default poll interval for long-running operation + */ + private Duration defaultPollInterval; + + /** + * Sets The default poll interval for long-running operation. + * + * @param defaultPollInterval the defaultPollInterval value. + * @return the NotificationHubsRPClientBuilder. + */ + public NotificationHubsRPClientBuilder defaultPollInterval(Duration defaultPollInterval) { + this.defaultPollInterval = defaultPollInterval; + return this; + } + + /* + * The serializer to serialize an object into a string + */ + private SerializerAdapter serializerAdapter; + + /** + * Sets The serializer to serialize an object into a string. + * + * @param serializerAdapter the serializerAdapter value. + * @return the NotificationHubsRPClientBuilder. + */ + public NotificationHubsRPClientBuilder serializerAdapter(SerializerAdapter serializerAdapter) { + this.serializerAdapter = serializerAdapter; + return this; + } + + /** + * Builds an instance of NotificationHubsRPClientImpl with the provided parameters. + * + * @return an instance of NotificationHubsRPClientImpl. + */ + public NotificationHubsRPClientImpl buildClient() { + String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; + AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + NotificationHubsRPClientImpl client = new NotificationHubsRPClientImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); + return client; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientImpl.java similarity index 73% rename from sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientImpl.java rename to sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientImpl.java index fbc07739322e9..e34ee028c80f4 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsManagementClientImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/NotificationHubsRPClientImpl.java @@ -15,6 +15,7 @@ import com.azure.core.management.polling.PollResult; import com.azure.core.management.polling.PollerFactory; import com.azure.core.util.Context; +import com.azure.core.util.CoreUtils; import com.azure.core.util.logging.ClientLogger; import com.azure.core.util.polling.AsyncPollResponse; import com.azure.core.util.polling.LongRunningOperationStatus; @@ -23,167 +24,190 @@ import com.azure.core.util.serializer.SerializerEncoding; import com.azure.resourcemanager.notificationhubs.fluent.NamespacesClient; import com.azure.resourcemanager.notificationhubs.fluent.NotificationHubsClient; -import com.azure.resourcemanager.notificationhubs.fluent.NotificationHubsManagementClient; +import com.azure.resourcemanager.notificationhubs.fluent.NotificationHubsRPClient; import com.azure.resourcemanager.notificationhubs.fluent.OperationsClient; +import com.azure.resourcemanager.notificationhubs.fluent.PrivateEndpointConnectionsClient; import java.io.IOException; import java.lang.reflect.Type; import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.time.Duration; -import java.util.Map; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the NotificationHubsManagementClientImpl type. */ -@ServiceClient(builder = NotificationHubsManagementClientBuilder.class) -public final class NotificationHubsManagementClientImpl implements NotificationHubsManagementClient { - private final ClientLogger logger = new ClientLogger(NotificationHubsManagementClientImpl.class); - +/** + * Initializes a new instance of the NotificationHubsRPClientImpl type. + */ +@ServiceClient(builder = NotificationHubsRPClientBuilder.class) +public final class NotificationHubsRPClientImpl implements NotificationHubsRPClient { /** - * Gets subscription credentials which uniquely identify Microsoft Azure subscription. The subscription ID forms - * part of the URI for every service call. + * The ID of the target subscription. The value must be an UUID. */ private final String subscriptionId; /** - * Gets Gets subscription credentials which uniquely identify Microsoft Azure subscription. The subscription ID - * forms part of the URI for every service call. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The OperationsClient object to access its operations. */ - private final OperationsClient operations; + /** + * The NotificationHubsClient object to access its operations. + */ + private final NotificationHubsClient notificationHubs; /** - * Gets the OperationsClient object to access its operations. - * - * @return the OperationsClient object. + * Gets the NotificationHubsClient object to access its operations. + * + * @return the NotificationHubsClient object. */ - public OperationsClient getOperations() { - return this.operations; + public NotificationHubsClient getNotificationHubs() { + return this.notificationHubs; } - /** The NamespacesClient object to access its operations. */ + /** + * The NamespacesClient object to access its operations. + */ private final NamespacesClient namespaces; /** * Gets the NamespacesClient object to access its operations. - * + * * @return the NamespacesClient object. */ public NamespacesClient getNamespaces() { return this.namespaces; } - /** The NotificationHubsClient object to access its operations. */ - private final NotificationHubsClient notificationHubs; + /** + * The OperationsClient object to access its operations. + */ + private final OperationsClient operations; /** - * Gets the NotificationHubsClient object to access its operations. - * - * @return the NotificationHubsClient object. + * Gets the OperationsClient object to access its operations. + * + * @return the OperationsClient object. */ - public NotificationHubsClient getNotificationHubs() { - return this.notificationHubs; + public OperationsClient getOperations() { + return this.operations; } /** - * Initializes an instance of NotificationHubsManagementClient client. - * + * The PrivateEndpointConnectionsClient object to access its operations. + */ + private final PrivateEndpointConnectionsClient privateEndpointConnections; + + /** + * Gets the PrivateEndpointConnectionsClient object to access its operations. + * + * @return the PrivateEndpointConnectionsClient object. + */ + public PrivateEndpointConnectionsClient getPrivateEndpointConnections() { + return this.privateEndpointConnections; + } + + /** + * Initializes an instance of NotificationHubsRPClient client. + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. * @param environment The Azure environment. - * @param subscriptionId Gets subscription credentials which uniquely identify Microsoft Azure subscription. The - * subscription ID forms part of the URI for every service call. + * @param subscriptionId The ID of the target subscription. The value must be an UUID. * @param endpoint server parameter. */ - NotificationHubsManagementClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + NotificationHubsRPClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2017-04-01"; - this.operations = new OperationsClientImpl(this); - this.namespaces = new NamespacesClientImpl(this); + this.apiVersion = "2023-10-01-preview"; this.notificationHubs = new NotificationHubsClientImpl(this); + this.namespaces = new NamespacesClientImpl(this); + this.operations = new OperationsClientImpl(this); + this.privateEndpointConnections = new PrivateEndpointConnectionsClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -192,20 +216,17 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ public Context mergeContext(Context context) { - for (Map.Entry entry : this.getContext().getValues().entrySet()) { - context = context.addData(entry.getKey(), entry.getValue()); - } - return context; + return CoreUtils.mergeContexts(this.getContext(), context); } /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -215,26 +236,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -247,24 +257,21 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } } catch (IOException | RuntimeException ioe) { - logger.logThrowableAsWarning(ioe); + LOGGER.logThrowableAsWarning(ioe); } } } else { @@ -323,4 +330,6 @@ public Mono getBodyAsString(Charset charset) { return Mono.just(new String(responseBody, charset)); } } + + private static final ClientLogger LOGGER = new ClientLogger(NotificationHubsRPClientImpl.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationImpl.java index 8f57a1356d253..986c34f1417cf 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationImpl.java @@ -7,14 +7,15 @@ import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; import com.azure.resourcemanager.notificationhubs.models.Operation; import com.azure.resourcemanager.notificationhubs.models.OperationDisplay; +import com.azure.resourcemanager.notificationhubs.models.OperationProperties; public final class OperationImpl implements Operation { private OperationInner innerObject; private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - OperationImpl( - OperationInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + OperationImpl(OperationInner innerObject, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -27,6 +28,14 @@ public OperationDisplay display() { return this.innerModel().display(); } + public OperationProperties properties() { + return this.innerModel().properties(); + } + + public Boolean isDataAction() { + return this.innerModel().isDataAction(); + } + public OperationInner innerModel() { return this.innerObject; } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsClientImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsClientImpl.java index d29d2d2431058..3da66d8f7dc64 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsClientImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsClientImpl.java @@ -25,130 +25,110 @@ import com.azure.core.management.exception.ManagementException; import com.azure.core.util.Context; import com.azure.core.util.FluxUtil; -import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.OperationsClient; import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; import com.azure.resourcemanager.notificationhubs.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - private final ClientLogger logger = new ClientLogger(OperationsClientImpl.class); - - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ - private final NotificationHubsManagementClientImpl client; + /** + * The service client containing this operation class. + */ + private final NotificationHubsRPClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ - OperationsClientImpl(NotificationHubsManagementClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + OperationsClientImpl(NotificationHubsRPClientImpl client) { + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for NotificationHubsManagementClientOperations to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for NotificationHubsRPClientOperations to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") - @ServiceInterface(name = "NotificationHubsMana") - private interface OperationsService { - @Headers({"Content-Type: application/json"}) + @ServiceInterface(name = "NotificationHubsRPCl") + public interface OperationsService { + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.NotificationHubs/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -156,26 +136,27 @@ private PagedFlux listAsync() { } /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -183,13 +164,14 @@ public PagedIterable list() { } /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -198,12 +180,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -211,35 +196,28 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * - * @param nextLink The nextLink parameter. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations along with {@link PagedResponse} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -247,23 +225,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsImpl.java index 57b639649b8b1..9f63228665c38 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/OperationsImpl.java @@ -11,17 +11,15 @@ import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; import com.azure.resourcemanager.notificationhubs.models.Operation; import com.azure.resourcemanager.notificationhubs.models.Operations; -import com.fasterxml.jackson.annotation.JsonIgnore; public final class OperationsImpl implements Operations { - @JsonIgnore private final ClientLogger logger = new ClientLogger(OperationsImpl.class); + private static final ClientLogger LOGGER = new ClientLogger(OperationsImpl.class); private final OperationsClient innerClient; private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, + public OperationsImpl(OperationsClient innerClient, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -29,12 +27,12 @@ public OperationsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PnsCredentialsResourceImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PnsCredentialsResourceImpl.java index 3cc65e4cf300c..b09a7c02f0ddc 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PnsCredentialsResourceImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PnsCredentialsResourceImpl.java @@ -4,15 +4,18 @@ package com.azure.resourcemanager.notificationhubs.implementation; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner; import com.azure.resourcemanager.notificationhubs.models.AdmCredential; import com.azure.resourcemanager.notificationhubs.models.ApnsCredential; import com.azure.resourcemanager.notificationhubs.models.BaiduCredential; +import com.azure.resourcemanager.notificationhubs.models.BrowserCredential; +import com.azure.resourcemanager.notificationhubs.models.FcmV1Credential; import com.azure.resourcemanager.notificationhubs.models.GcmCredential; import com.azure.resourcemanager.notificationhubs.models.MpnsCredential; import com.azure.resourcemanager.notificationhubs.models.PnsCredentialsResource; -import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.WnsCredential; +import com.azure.resourcemanager.notificationhubs.models.XiaomiCredential; import java.util.Collections; import java.util.Map; @@ -21,8 +24,7 @@ public final class PnsCredentialsResourceImpl implements PnsCredentialsResource private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - PnsCredentialsResourceImpl( - PnsCredentialsResourceInner innerObject, + PnsCredentialsResourceImpl(PnsCredentialsResourceInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -53,16 +55,24 @@ public Map tags() { } } - public Sku sku() { - return this.innerModel().sku(); + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public AdmCredential admCredential() { + return this.innerModel().admCredential(); } public ApnsCredential apnsCredential() { return this.innerModel().apnsCredential(); } - public WnsCredential wnsCredential() { - return this.innerModel().wnsCredential(); + public BaiduCredential baiduCredential() { + return this.innerModel().baiduCredential(); + } + + public BrowserCredential browserCredential() { + return this.innerModel().browserCredential(); } public GcmCredential gcmCredential() { @@ -73,12 +83,16 @@ public MpnsCredential mpnsCredential() { return this.innerModel().mpnsCredential(); } - public AdmCredential admCredential() { - return this.innerModel().admCredential(); + public WnsCredential wnsCredential() { + return this.innerModel().wnsCredential(); } - public BaiduCredential baiduCredential() { - return this.innerModel().baiduCredential(); + public XiaomiCredential xiaomiCredential() { + return this.innerModel().xiaomiCredential(); + } + + public FcmV1Credential fcmV1Credential() { + return this.innerModel().fcmV1Credential(); } public PnsCredentialsResourceInner innerModel() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionResourceImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionResourceImpl.java new file mode 100644 index 0000000000000..c00cad98763fa --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionResourceImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResource; + +public final class PrivateEndpointConnectionResourceImpl implements PrivateEndpointConnectionResource { + private PrivateEndpointConnectionResourceInner innerObject; + + private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; + + PrivateEndpointConnectionResourceImpl(PrivateEndpointConnectionResourceInner innerObject, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public PrivateEndpointConnectionProperties properties() { + return this.innerModel().properties(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public PrivateEndpointConnectionResourceInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsClientImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsClientImpl.java new file mode 100644 index 0000000000000..d17b04e2e4fd7 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsClientImpl.java @@ -0,0 +1,1288 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.notificationhubs.fluent.PrivateEndpointConnectionsClient; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResourceListResult; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceListResult; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. + */ +public final class PrivateEndpointConnectionsClientImpl implements PrivateEndpointConnectionsClient { + /** + * The proxy service used to perform REST calls. + */ + private final PrivateEndpointConnectionsService service; + + /** + * The service client containing this operation class. + */ + private final NotificationHubsRPClientImpl client; + + /** + * Initializes an instance of PrivateEndpointConnectionsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + PrivateEndpointConnectionsClientImpl(NotificationHubsRPClientImpl client) { + this.service = RestProxy.create(PrivateEndpointConnectionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for NotificationHubsRPClientPrivateEndpointConnections to be used by the + * proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "NotificationHubsRPCl") + public interface PrivateEndpointConnectionsService { + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") PrivateEndpointConnectionResourceInner parameters, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 202, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/privateEndpointConnections") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/privateLinkResources/{subResourceName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getGroupId(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @PathParam("subResourceName") String subResourceName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.NotificationHubs/namespaces/{namespaceName}/privateLinkResources") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listGroupIds(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("namespaceName") String namespaceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (privateEndpointConnectionName == null) { + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); + } + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, privateEndpointConnectionName, this.client.getApiVersion(), + parameters, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (privateEndpointConnectionName == null) { + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); + } + if (parameters == null) { + return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); + } else { + parameters.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, privateEndpointConnectionName, this.client.getApiVersion(), parameters, accept, context); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of represents a Private Endpoint Connection ARM resource - a + * sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, PrivateEndpointConnectionResourceInner> + beginUpdateAsync(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + PrivateEndpointConnectionResourceInner parameters) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), PrivateEndpointConnectionResourceInner.class, + PrivateEndpointConnectionResourceInner.class, this.client.getContext()); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of represents a Private Endpoint Connection ARM resource - a + * sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, PrivateEndpointConnectionResourceInner> + beginUpdateAsync(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + PrivateEndpointConnectionResourceInner parameters, Context context) { + context = this.client.mergeContext(context); + Mono>> mono = updateWithResponseAsync(resourceGroupName, namespaceName, + privateEndpointConnectionName, parameters, context); + return this.client.getLroResult( + mono, this.client.getHttpPipeline(), PrivateEndpointConnectionResourceInner.class, + PrivateEndpointConnectionResourceInner.class, context); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of represents a Private Endpoint Connection ARM resource - a + * sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, PrivateEndpointConnectionResourceInner> + beginUpdate(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + PrivateEndpointConnectionResourceInner parameters) { + return this.beginUpdateAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters) + .getSyncPoller(); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of represents a Private Endpoint Connection ARM resource - a + * sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, PrivateEndpointConnectionResourceInner> + beginUpdate(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + PrivateEndpointConnectionResourceInner parameters, Context context) { + return this + .beginUpdateAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters, context) + .getSyncPoller(); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters) { + return beginUpdateAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters, Context context) { + return beginUpdateAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters, context) + .last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public PrivateEndpointConnectionResourceInner update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters) { + return updateAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters).block(); + } + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public PrivateEndpointConnectionResourceInner update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters, Context context) { + return updateAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters, context) + .block(); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (privateEndpointConnectionName == null) { + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, privateEndpointConnectionName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (privateEndpointConnectionName == null) { + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, privateEndpointConnectionName, this.client.getApiVersion(), accept, context); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, namespaceName, privateEndpointConnectionName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + return this.beginDeleteAsync(resourceGroupName, namespaceName, privateEndpointConnectionName).getSyncPoller(); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context) { + return this.beginDeleteAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, context) + .getSyncPoller(); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + return beginDeleteAsync(resourceGroupName, namespaceName, privateEndpointConnectionName).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + Context context) { + return beginDeleteAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName) { + deleteAsync(resourceGroupName, namespaceName, privateEndpointConnectionName).block(); + } + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + Context context) { + deleteAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, context).block(); + } + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String namespaceName, String privateEndpointConnectionName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (privateEndpointConnectionName == null) { + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, privateEndpointConnectionName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String namespaceName, String privateEndpointConnectionName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (privateEndpointConnectionName == null) { + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, namespaceName, + privateEndpointConnectionName, this.client.getApiVersion(), accept, context); + } + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + return getWithResponseAsync(resourceGroupName, namespaceName, privateEndpointConnectionName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, + String namespaceName, String privateEndpointConnectionName, Context context) { + return getWithResponseAsync(resourceGroupName, namespaceName, privateEndpointConnectionName, context).block(); + } + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public PrivateEndpointConnectionResourceInner get(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + return getWithResponse(resourceGroupName, namespaceName, privateEndpointConnectionName, Context.NONE) + .getValue(); + } + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation along with {@link PagedResponse} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String namespaceName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation along with {@link PagedResponse} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String namespaceName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, namespaceName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); + } + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, + String namespaceName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, namespaceName)); + } + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String namespaceName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, namespaceName, context)); + } + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String namespaceName) { + return new PagedIterable<>(listAsync(resourceGroupName, namespaceName)); + } + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String namespaceName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, namespaceName, context)); + } + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getGroupIdWithResponseAsync(String resourceGroupName, + String namespaceName, String subResourceName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (subResourceName == null) { + return Mono + .error(new IllegalArgumentException("Parameter subResourceName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getGroupId(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, subResourceName, this.client.getApiVersion(), accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getGroupIdWithResponseAsync(String resourceGroupName, + String namespaceName, String subResourceName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + if (subResourceName == null) { + return Mono + .error(new IllegalArgumentException("Parameter subResourceName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getGroupId(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + namespaceName, subResourceName, this.client.getApiVersion(), accept, context); + } + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getGroupIdAsync(String resourceGroupName, String namespaceName, + String subResourceName) { + return getGroupIdWithResponseAsync(resourceGroupName, namespaceName, subResourceName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getGroupIdWithResponse(String resourceGroupName, String namespaceName, + String subResourceName, Context context) { + return getGroupIdWithResponseAsync(resourceGroupName, namespaceName, subResourceName, context).block(); + } + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public PrivateLinkResourceInner getGroupId(String resourceGroupName, String namespaceName, String subResourceName) { + return getGroupIdWithResponse(resourceGroupName, namespaceName, subResourceName, Context.NONE).getValue(); + } + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listGroupIdsSinglePageAsync(String resourceGroupName, + String namespaceName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listGroupIds(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, namespaceName, this.client.getApiVersion(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listGroupIdsSinglePageAsync(String resourceGroupName, + String namespaceName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (namespaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter namespaceName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listGroupIds(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, namespaceName, + this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); + } + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listGroupIdsAsync(String resourceGroupName, String namespaceName) { + return new PagedFlux<>(() -> listGroupIdsSinglePageAsync(resourceGroupName, namespaceName)); + } + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listGroupIdsAsync(String resourceGroupName, String namespaceName, + Context context) { + return new PagedFlux<>(() -> listGroupIdsSinglePageAsync(resourceGroupName, namespaceName, context)); + } + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listGroupIds(String resourceGroupName, String namespaceName) { + return new PagedIterable<>(listGroupIdsAsync(resourceGroupName, namespaceName)); + } + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with + * {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listGroupIds(String resourceGroupName, String namespaceName, + Context context) { + return new PagedIterable<>(listGroupIdsAsync(resourceGroupName, namespaceName, context)); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsImpl.java new file mode 100644 index 0000000000000..aea9501a468a4 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateEndpointConnectionsImpl.java @@ -0,0 +1,143 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.notificationhubs.fluent.PrivateEndpointConnectionsClient; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResource; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnections; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResource; + +public final class PrivateEndpointConnectionsImpl implements PrivateEndpointConnections { + private static final ClientLogger LOGGER = new ClientLogger(PrivateEndpointConnectionsImpl.class); + + private final PrivateEndpointConnectionsClient innerClient; + + private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; + + public PrivateEndpointConnectionsImpl(PrivateEndpointConnectionsClient innerClient, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PrivateEndpointConnectionResource update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters) { + PrivateEndpointConnectionResourceInner inner + = this.serviceClient().update(resourceGroupName, namespaceName, privateEndpointConnectionName, parameters); + if (inner != null) { + return new PrivateEndpointConnectionResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + public PrivateEndpointConnectionResource update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters, Context context) { + PrivateEndpointConnectionResourceInner inner = this.serviceClient().update(resourceGroupName, namespaceName, + privateEndpointConnectionName, parameters, context); + if (inner != null) { + return new PrivateEndpointConnectionResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + public void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName) { + this.serviceClient().delete(resourceGroupName, namespaceName, privateEndpointConnectionName); + } + + public void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, + Context context) { + this.serviceClient().delete(resourceGroupName, namespaceName, privateEndpointConnectionName, context); + } + + public Response getWithResponse(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + namespaceName, privateEndpointConnectionName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new PrivateEndpointConnectionResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public PrivateEndpointConnectionResource get(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName) { + PrivateEndpointConnectionResourceInner inner + = this.serviceClient().get(resourceGroupName, namespaceName, privateEndpointConnectionName); + if (inner != null) { + return new PrivateEndpointConnectionResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable list(String resourceGroupName, String namespaceName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, namespaceName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new PrivateEndpointConnectionResourceImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String namespaceName, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, namespaceName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new PrivateEndpointConnectionResourceImpl(inner1, this.manager())); + } + + public Response getGroupIdWithResponse(String resourceGroupName, String namespaceName, + String subResourceName, Context context) { + Response inner + = this.serviceClient().getGroupIdWithResponse(resourceGroupName, namespaceName, subResourceName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new PrivateLinkResourceImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public PrivateLinkResource getGroupId(String resourceGroupName, String namespaceName, String subResourceName) { + PrivateLinkResourceInner inner + = this.serviceClient().getGroupId(resourceGroupName, namespaceName, subResourceName); + if (inner != null) { + return new PrivateLinkResourceImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable listGroupIds(String resourceGroupName, String namespaceName) { + PagedIterable inner + = this.serviceClient().listGroupIds(resourceGroupName, namespaceName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + } + + public PagedIterable listGroupIds(String resourceGroupName, String namespaceName, + Context context) { + PagedIterable inner + = this.serviceClient().listGroupIds(resourceGroupName, namespaceName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + } + + private PrivateEndpointConnectionsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateLinkResourceImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateLinkResourceImpl.java new file mode 100644 index 0000000000000..79155aceeb06c --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/PrivateLinkResourceImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResource; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceProperties; + +public final class PrivateLinkResourceImpl implements PrivateLinkResource { + private PrivateLinkResourceInner innerObject; + + private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; + + PrivateLinkResourceImpl(PrivateLinkResourceInner innerObject, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public PrivateLinkResourceProperties properties() { + return this.innerModel().properties(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public PrivateLinkResourceInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceListKeysImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceListKeysImpl.java index 015fb23a02683..da8404016e1f7 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceListKeysImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceListKeysImpl.java @@ -12,8 +12,7 @@ public final class ResourceListKeysImpl implements ResourceListKeys { private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; - ResourceListKeysImpl( - ResourceListKeysInner innerObject, + ResourceListKeysImpl(ResourceListKeysInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceManagerUtils.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..7dfe9983b1e43 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/SharedAccessAuthorizationRuleResourceImpl.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/SharedAccessAuthorizationRuleResourceImpl.java index f791ef4a75140..0b74a733b4333 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/SharedAccessAuthorizationRuleResourceImpl.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/SharedAccessAuthorizationRuleResourceImpl.java @@ -6,23 +6,20 @@ import com.azure.core.http.rest.Response; import com.azure.core.management.Region; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.AccessRights; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; import com.azure.resourcemanager.notificationhubs.models.ResourceListKeys; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleResource; -import com.azure.resourcemanager.notificationhubs.models.Sku; +import java.time.OffsetDateTime; import java.util.Collections; import java.util.List; import java.util.Map; -public final class SharedAccessAuthorizationRuleResourceImpl - implements SharedAccessAuthorizationRuleResource, - SharedAccessAuthorizationRuleResource.Definition, - SharedAccessAuthorizationRuleResource.Update { +public final class SharedAccessAuthorizationRuleResourceImpl implements SharedAccessAuthorizationRuleResource, + SharedAccessAuthorizationRuleResource.Definition, SharedAccessAuthorizationRuleResource.Update { private SharedAccessAuthorizationRuleResourceInner innerObject; private final com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager; @@ -52,8 +49,8 @@ public Map tags() { } } - public Sku sku() { - return this.innerModel().sku(); + public SystemData systemData() { + return this.innerModel().systemData(); } public List rights() { @@ -77,20 +74,20 @@ public String keyName() { return this.innerModel().keyName(); } - public String claimType() { - return this.innerModel().claimType(); + public OffsetDateTime modifiedTime() { + return this.innerModel().modifiedTime(); } - public String claimValue() { - return this.innerModel().claimValue(); + public OffsetDateTime createdTime() { + return this.innerModel().createdTime(); } - public String modifiedTime() { - return this.innerModel().modifiedTime(); + public String claimType() { + return this.innerModel().claimType(); } - public String createdTime() { - return this.innerModel().createdTime(); + public String claimValue() { + return this.innerModel().claimValue(); } public Integer revision() { @@ -105,6 +102,10 @@ public String regionName() { return this.location(); } + public String resourceGroupName() { + return resourceGroupName; + } + public SharedAccessAuthorizationRuleResourceInner innerModel() { return this.innerObject; } @@ -117,140 +118,134 @@ private com.azure.resourcemanager.notificationhubs.NotificationHubsManager manag private String namespaceName; - private String authorizationRuleName; - - private SharedAccessAuthorizationRuleCreateOrUpdateParameters createParameters; + private String notificationHubName; - private SharedAccessAuthorizationRuleCreateOrUpdateParameters updateParameters; + private String authorizationRuleName; - public SharedAccessAuthorizationRuleResourceImpl withExistingNamespace( - String resourceGroupName, String namespaceName) { + public SharedAccessAuthorizationRuleResourceImpl withExistingNotificationHub(String resourceGroupName, + String namespaceName, String notificationHubName) { this.resourceGroupName = resourceGroupName; this.namespaceName = namespaceName; + this.notificationHubName = notificationHubName; return this; } public SharedAccessAuthorizationRuleResource create() { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .createOrUpdateAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, authorizationRuleName, createParameters, Context.NONE) + this.innerObject + = serviceManager + .serviceClient().getNotificationHubs().createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, this.innerModel(), Context.NONE) .getValue(); return this; } public SharedAccessAuthorizationRuleResource create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .createOrUpdateAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, authorizationRuleName, createParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNotificationHubs() + .createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, this.innerModel(), context) + .getValue(); return this; } - SharedAccessAuthorizationRuleResourceImpl( - String name, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { + SharedAccessAuthorizationRuleResourceImpl(String name, + com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = new SharedAccessAuthorizationRuleResourceInner(); this.serviceManager = serviceManager; this.authorizationRuleName = name; - this.createParameters = new SharedAccessAuthorizationRuleCreateOrUpdateParameters(); } public SharedAccessAuthorizationRuleResourceImpl update() { - this.updateParameters = new SharedAccessAuthorizationRuleCreateOrUpdateParameters(); return this; } public SharedAccessAuthorizationRuleResource apply() { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .createOrUpdateAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, authorizationRuleName, updateParameters, Context.NONE) + this.innerObject + = serviceManager + .serviceClient().getNotificationHubs().createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, this.innerModel(), Context.NONE) .getValue(); return this; } public SharedAccessAuthorizationRuleResource apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .createOrUpdateAuthorizationRuleWithResponse( - resourceGroupName, namespaceName, authorizationRuleName, updateParameters, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getNotificationHubs() + .createOrUpdateAuthorizationRuleWithResponse(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, this.innerModel(), context) + .getValue(); return this; } - SharedAccessAuthorizationRuleResourceImpl( - SharedAccessAuthorizationRuleResourceInner innerObject, + SharedAccessAuthorizationRuleResourceImpl(SharedAccessAuthorizationRuleResourceInner innerObject, com.azure.resourcemanager.notificationhubs.NotificationHubsManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.namespaceName = Utils.getValueFromIdByName(innerObject.id(), "namespaces"); - this.authorizationRuleName = Utils.getValueFromIdByName(innerObject.id(), "AuthorizationRules"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.namespaceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "namespaces"); + this.notificationHubName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "notificationHubs"); + this.authorizationRuleName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "authorizationRules"); } public SharedAccessAuthorizationRuleResource refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getNotificationHubs().getAuthorizationRuleWithResponse(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, Context.NONE).getValue(); return this; } public SharedAccessAuthorizationRuleResource refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getNamespaces() - .getAuthorizationRuleWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getNotificationHubs().getAuthorizationRuleWithResponse(resourceGroupName, + namespaceName, notificationHubName, authorizationRuleName, context).getValue(); return this; } + public Response listKeysWithResponse(Context context) { + return serviceManager.notificationHubs().listKeysWithResponse(resourceGroupName, namespaceName, + notificationHubName, authorizationRuleName, context); + } + public ResourceListKeys listKeys() { - return serviceManager.namespaces().listKeys(resourceGroupName, namespaceName, authorizationRuleName); + return serviceManager.notificationHubs().listKeys(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName); } - public Response listKeysWithResponse(Context context) { - return serviceManager - .namespaces() - .listKeysWithResponse(resourceGroupName, namespaceName, authorizationRuleName, context); + public Response regenerateKeysWithResponse(PolicyKeyResource parameters, Context context) { + return serviceManager.notificationHubs().regenerateKeysWithResponse(resourceGroupName, namespaceName, + notificationHubName, authorizationRuleName, parameters, context); } - public ResourceListKeys regenerateKeys(PolicykeyResource parameters) { - return serviceManager - .namespaces() - .regenerateKeys(resourceGroupName, namespaceName, authorizationRuleName, parameters); + public ResourceListKeys regenerateKeys(PolicyKeyResource parameters) { + return serviceManager.notificationHubs().regenerateKeys(resourceGroupName, namespaceName, notificationHubName, + authorizationRuleName, parameters); } - public Response regenerateKeysWithResponse(PolicykeyResource parameters, Context context) { - return serviceManager - .namespaces() - .regenerateKeysWithResponse(resourceGroupName, namespaceName, authorizationRuleName, parameters, context); + public SharedAccessAuthorizationRuleResourceImpl withRegion(Region location) { + this.innerModel().withLocation(location.toString()); + return this; } - public SharedAccessAuthorizationRuleResourceImpl withProperties( - SharedAccessAuthorizationRuleProperties properties) { - if (isInCreateMode()) { - this.createParameters.withProperties(properties); - return this; - } else { - this.updateParameters.withProperties(properties); - return this; - } + public SharedAccessAuthorizationRuleResourceImpl withRegion(String location) { + this.innerModel().withLocation(location); + return this; } - private boolean isInCreateMode() { - return this.innerModel().id() == null; + public SharedAccessAuthorizationRuleResourceImpl withTags(Map tags) { + this.innerModel().withTags(tags); + return this; + } + + public SharedAccessAuthorizationRuleResourceImpl withRights(List rights) { + this.innerModel().withRights(rights); + return this; + } + + public SharedAccessAuthorizationRuleResourceImpl withPrimaryKey(String primaryKey) { + this.innerModel().withPrimaryKey(primaryKey); + return this; + } + + public SharedAccessAuthorizationRuleResourceImpl withSecondaryKey(String secondaryKey) { + this.innerModel().withSecondaryKey(secondaryKey); + return this; } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/Utils.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/Utils.java deleted file mode 100644 index 007c16f2410b8..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/package-info.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/package-info.java index ca1c8efbd029e..f9124becc1e51 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/package-info.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the implementations for NotificationHubsManagementClient. Azure NotificationHub client. */ +/** + * Package containing the implementations for NotificationHubsRPClient. + * Microsoft Notification Hubs Resource Provider REST API. + */ package com.azure.resourcemanager.notificationhubs.implementation; diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AccessRights.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AccessRights.java index abe4c32bbd324..c8cdc81deb865 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AccessRights.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AccessRights.java @@ -4,47 +4,55 @@ package com.azure.resourcemanager.notificationhubs.models; +import com.azure.core.util.ExpandableStringEnum; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonValue; +import java.util.Collection; -/** Defines values for AccessRights. */ -public enum AccessRights { - /** Enum value Manage. */ - MANAGE("Manage"), - - /** Enum value Send. */ - SEND("Send"), +/** + * Defines values for AccessRights. + */ +public final class AccessRights extends ExpandableStringEnum { + /** + * Static value Manage for AccessRights. + */ + public static final AccessRights MANAGE = fromString("Manage"); - /** Enum value Listen. */ - LISTEN("Listen"); + /** + * Static value Send for AccessRights. + */ + public static final AccessRights SEND = fromString("Send"); - /** The actual serialized value for a AccessRights instance. */ - private final String value; + /** + * Static value Listen for AccessRights. + */ + public static final AccessRights LISTEN = fromString("Listen"); - AccessRights(String value) { - this.value = value; + /** + * Creates a new instance of AccessRights value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public AccessRights() { } /** - * Parses a serialized value to a AccessRights instance. - * - * @param value the serialized value to parse. - * @return the parsed AccessRights object, or null if unable to parse. + * Creates or finds a AccessRights from its string representation. + * + * @param name a name to look for. + * @return the corresponding AccessRights. */ @JsonCreator - public static AccessRights fromString(String value) { - AccessRights[] items = AccessRights.values(); - for (AccessRights item : items) { - if (item.toString().equalsIgnoreCase(value)) { - return item; - } - } - return null; + public static AccessRights fromString(String name) { + return fromString(name, AccessRights.class); } - @JsonValue - @Override - public String toString() { - return this.value; + /** + * Gets known AccessRights values. + * + * @return known AccessRights values. + */ + public static Collection values() { + return values(AccessRights.class); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AdmCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AdmCredential.java index 05e8771693af4..f59f405f66368 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AdmCredential.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/AdmCredential.java @@ -7,23 +7,28 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.AdmCredentialProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub AdmCredential. */ +/** + * Description of a NotificationHub AdmCredential. + */ @Fluent public final class AdmCredential { - @JsonIgnore private final ClientLogger logger = new ClientLogger(AdmCredential.class); - /* - * Properties of NotificationHub AdmCredential. + * Description of a NotificationHub AdmCredential. + */ + @JsonProperty(value = "properties", required = true) + private AdmCredentialProperties innerProperties = new AdmCredentialProperties(); + + /** + * Creates an instance of AdmCredential class. */ - @JsonProperty(value = "properties") - private AdmCredentialProperties innerProperties; + public AdmCredential() { + } /** - * Get the innerProperties property: Properties of NotificationHub AdmCredential. - * + * Get the innerProperties property: Description of a NotificationHub AdmCredential. + * * @return the innerProperties value. */ private AdmCredentialProperties innerProperties() { @@ -31,8 +36,8 @@ private AdmCredentialProperties innerProperties() { } /** - * Get the clientId property: The client identifier. - * + * Get the clientId property: Gets or sets the client identifier. + * * @return the clientId value. */ public String clientId() { @@ -40,8 +45,8 @@ public String clientId() { } /** - * Set the clientId property: The client identifier. - * + * Set the clientId property: Gets or sets the client identifier. + * * @param clientId the clientId value to set. * @return the AdmCredential object itself. */ @@ -54,8 +59,8 @@ public AdmCredential withClientId(String clientId) { } /** - * Get the clientSecret property: The credential secret access key. - * + * Get the clientSecret property: Gets or sets the credential secret access key. + * * @return the clientSecret value. */ public String clientSecret() { @@ -63,8 +68,8 @@ public String clientSecret() { } /** - * Set the clientSecret property: The credential secret access key. - * + * Set the clientSecret property: Gets or sets the credential secret access key. + * * @param clientSecret the clientSecret value to set. * @return the AdmCredential object itself. */ @@ -77,8 +82,8 @@ public AdmCredential withClientSecret(String clientSecret) { } /** - * Get the authTokenUrl property: The URL of the authorization token. - * + * Get the authTokenUrl property: Gets or sets the URL of the authorization token. + * * @return the authTokenUrl value. */ public String authTokenUrl() { @@ -86,8 +91,8 @@ public String authTokenUrl() { } /** - * Set the authTokenUrl property: The URL of the authorization token. - * + * Set the authTokenUrl property: Gets or sets the URL of the authorization token. + * * @param authTokenUrl the authTokenUrl value to set. * @return the AdmCredential object itself. */ @@ -101,12 +106,17 @@ public AdmCredential withAuthTokenUrl(String authTokenUrl) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model AdmCredential")); + } else { innerProperties().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(AdmCredential.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ApnsCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ApnsCredential.java index bfbfac3238d75..9a1ba9f3c4f31 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ApnsCredential.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ApnsCredential.java @@ -7,23 +7,28 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.ApnsCredentialProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub ApnsCredential. */ +/** + * Description of a NotificationHub ApnsCredential. + */ @Fluent public final class ApnsCredential { - @JsonIgnore private final ClientLogger logger = new ClientLogger(ApnsCredential.class); - /* - * Properties of NotificationHub ApnsCredential. + * Description of a NotificationHub ApnsCredential. + */ + @JsonProperty(value = "properties", required = true) + private ApnsCredentialProperties innerProperties = new ApnsCredentialProperties(); + + /** + * Creates an instance of ApnsCredential class. */ - @JsonProperty(value = "properties") - private ApnsCredentialProperties innerProperties; + public ApnsCredential() { + } /** - * Get the innerProperties property: Properties of NotificationHub ApnsCredential. - * + * Get the innerProperties property: Description of a NotificationHub ApnsCredential. + * * @return the innerProperties value. */ private ApnsCredentialProperties innerProperties() { @@ -31,8 +36,8 @@ private ApnsCredentialProperties innerProperties() { } /** - * Get the apnsCertificate property: The APNS certificate. Specify if using Certificate Authentication Mode. - * + * Get the apnsCertificate property: Gets or sets the APNS certificate. + * * @return the apnsCertificate value. */ public String apnsCertificate() { @@ -40,8 +45,8 @@ public String apnsCertificate() { } /** - * Set the apnsCertificate property: The APNS certificate. Specify if using Certificate Authentication Mode. - * + * Set the apnsCertificate property: Gets or sets the APNS certificate. + * * @param apnsCertificate the apnsCertificate value to set. * @return the ApnsCredential object itself. */ @@ -54,8 +59,8 @@ public ApnsCredential withApnsCertificate(String apnsCertificate) { } /** - * Get the certificateKey property: The APNS certificate password if it exists. - * + * Get the certificateKey property: Gets or sets the certificate key. + * * @return the certificateKey value. */ public String certificateKey() { @@ -63,8 +68,8 @@ public String certificateKey() { } /** - * Set the certificateKey property: The APNS certificate password if it exists. - * + * Set the certificateKey property: Gets or sets the certificate key. + * * @param certificateKey the certificateKey value to set. * @return the ApnsCredential object itself. */ @@ -77,12 +82,8 @@ public ApnsCredential withCertificateKey(String certificateKey) { } /** - * Get the endpoint property: The APNS endpoint of this credential. If using Certificate Authentication Mode and - * Sandbox specify 'gateway.sandbox.push.apple.com'. If using Certificate Authentication Mode and Production specify - * 'gateway.push.apple.com'. If using Token Authentication Mode and Sandbox specify - * 'https://api.development.push.apple.com:443/3/device'. If using Token Authentication Mode and Production specify - * 'https://api.push.apple.com:443/3/device'. - * + * Get the endpoint property: Gets or sets the endpoint of this credential. + * * @return the endpoint value. */ public String endpoint() { @@ -90,12 +91,8 @@ public String endpoint() { } /** - * Set the endpoint property: The APNS endpoint of this credential. If using Certificate Authentication Mode and - * Sandbox specify 'gateway.sandbox.push.apple.com'. If using Certificate Authentication Mode and Production specify - * 'gateway.push.apple.com'. If using Token Authentication Mode and Sandbox specify - * 'https://api.development.push.apple.com:443/3/device'. If using Token Authentication Mode and Production specify - * 'https://api.push.apple.com:443/3/device'. - * + * Set the endpoint property: Gets or sets the endpoint of this credential. + * * @param endpoint the endpoint value to set. * @return the ApnsCredential object itself. */ @@ -108,8 +105,8 @@ public ApnsCredential withEndpoint(String endpoint) { } /** - * Get the thumbprint property: The APNS certificate thumbprint. Specify if using Certificate Authentication Mode. - * + * Get the thumbprint property: Gets or sets the APNS certificate Thumbprint. + * * @return the thumbprint value. */ public String thumbprint() { @@ -117,8 +114,8 @@ public String thumbprint() { } /** - * Set the thumbprint property: The APNS certificate thumbprint. Specify if using Certificate Authentication Mode. - * + * Set the thumbprint property: Gets or sets the APNS certificate Thumbprint. + * * @param thumbprint the thumbprint value to set. * @return the ApnsCredential object itself. */ @@ -131,9 +128,9 @@ public ApnsCredential withThumbprint(String thumbprint) { } /** - * Get the keyId property: A 10-character key identifier (kid) key, obtained from your developer account. Specify if - * using Token Authentication Mode. - * + * Get the keyId property: Gets or sets a 10-character key identifier (kid) key, obtained from + * your developer account. + * * @return the keyId value. */ public String keyId() { @@ -141,9 +138,9 @@ public String keyId() { } /** - * Set the keyId property: A 10-character key identifier (kid) key, obtained from your developer account. Specify if - * using Token Authentication Mode. - * + * Set the keyId property: Gets or sets a 10-character key identifier (kid) key, obtained from + * your developer account. + * * @param keyId the keyId value to set. * @return the ApnsCredential object itself. */ @@ -156,8 +153,8 @@ public ApnsCredential withKeyId(String keyId) { } /** - * Get the appName property: The name of the application or BundleId. Specify if using Token Authentication Mode. - * + * Get the appName property: Gets or sets the name of the application. + * * @return the appName value. */ public String appName() { @@ -165,8 +162,8 @@ public String appName() { } /** - * Set the appName property: The name of the application or BundleId. Specify if using Token Authentication Mode. - * + * Set the appName property: Gets or sets the name of the application. + * * @param appName the appName value to set. * @return the ApnsCredential object itself. */ @@ -179,9 +176,9 @@ public ApnsCredential withAppName(String appName) { } /** - * Get the appId property: The issuer (iss) registered claim key. The value is a 10-character TeamId, obtained from - * your developer account. Specify if using Token Authentication Mode. - * + * Get the appId property: Gets or sets the issuer (iss) registered claim key, whose value is + * your 10-character Team ID, obtained from your developer account. + * * @return the appId value. */ public String appId() { @@ -189,9 +186,9 @@ public String appId() { } /** - * Set the appId property: The issuer (iss) registered claim key. The value is a 10-character TeamId, obtained from - * your developer account. Specify if using Token Authentication Mode. - * + * Set the appId property: Gets or sets the issuer (iss) registered claim key, whose value is + * your 10-character Team ID, obtained from your developer account. + * * @param appId the appId value to set. * @return the ApnsCredential object itself. */ @@ -204,9 +201,9 @@ public ApnsCredential withAppId(String appId) { } /** - * Get the token property: Provider Authentication Token, obtained through your developer account. Specify if using - * Token Authentication Mode. - * + * Get the token property: Gets or sets provider Authentication Token, obtained through your + * developer account. + * * @return the token value. */ public String token() { @@ -214,9 +211,9 @@ public String token() { } /** - * Set the token property: Provider Authentication Token, obtained through your developer account. Specify if using - * Token Authentication Mode. - * + * Set the token property: Gets or sets provider Authentication Token, obtained through your + * developer account. + * * @param token the token value to set. * @return the ApnsCredential object itself. */ @@ -230,12 +227,17 @@ public ApnsCredential withToken(String token) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model ApnsCredential")); + } else { innerProperties().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(ApnsCredential.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Availability.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Availability.java new file mode 100644 index 0000000000000..d1ed1f601ce16 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Availability.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Represents metric availability (part of RP operation descriptions). + */ +@Immutable +public final class Availability { + /* + * Time grain of the availability. + */ + @JsonProperty(value = "timeGrain", access = JsonProperty.Access.WRITE_ONLY) + private String timeGrain; + + /* + * Duration of the availability blob. + */ + @JsonProperty(value = "blobDuration", access = JsonProperty.Access.WRITE_ONLY) + private String blobDuration; + + /** + * Creates an instance of Availability class. + */ + public Availability() { + } + + /** + * Get the timeGrain property: Time grain of the availability. + * + * @return the timeGrain value. + */ + public String timeGrain() { + return this.timeGrain; + } + + /** + * Get the blobDuration property: Duration of the availability blob. + * + * @return the blobDuration value. + */ + public String blobDuration() { + return this.blobDuration; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BaiduCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BaiduCredential.java index 7c2d552d066ce..2fe3ad7a4f755 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BaiduCredential.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BaiduCredential.java @@ -7,23 +7,28 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.BaiduCredentialProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub BaiduCredential. */ +/** + * Description of a NotificationHub BaiduCredential. + */ @Fluent public final class BaiduCredential { - @JsonIgnore private final ClientLogger logger = new ClientLogger(BaiduCredential.class); - /* - * Properties of NotificationHub BaiduCredential. + * Description of a NotificationHub BaiduCredential. + */ + @JsonProperty(value = "properties", required = true) + private BaiduCredentialProperties innerProperties = new BaiduCredentialProperties(); + + /** + * Creates an instance of BaiduCredential class. */ - @JsonProperty(value = "properties") - private BaiduCredentialProperties innerProperties; + public BaiduCredential() { + } /** - * Get the innerProperties property: Properties of NotificationHub BaiduCredential. - * + * Get the innerProperties property: Description of a NotificationHub BaiduCredential. + * * @return the innerProperties value. */ private BaiduCredentialProperties innerProperties() { @@ -31,8 +36,8 @@ private BaiduCredentialProperties innerProperties() { } /** - * Get the baiduApiKey property: Baidu Api Key. - * + * Get the baiduApiKey property: Gets or sets baidu Api Key. + * * @return the baiduApiKey value. */ public String baiduApiKey() { @@ -40,8 +45,8 @@ public String baiduApiKey() { } /** - * Set the baiduApiKey property: Baidu Api Key. - * + * Set the baiduApiKey property: Gets or sets baidu Api Key. + * * @param baiduApiKey the baiduApiKey value to set. * @return the BaiduCredential object itself. */ @@ -54,8 +59,8 @@ public BaiduCredential withBaiduApiKey(String baiduApiKey) { } /** - * Get the baiduEndPoint property: Baidu Endpoint. - * + * Get the baiduEndPoint property: Gets or sets baidu Endpoint. + * * @return the baiduEndPoint value. */ public String baiduEndPoint() { @@ -63,8 +68,8 @@ public String baiduEndPoint() { } /** - * Set the baiduEndPoint property: Baidu Endpoint. - * + * Set the baiduEndPoint property: Gets or sets baidu Endpoint. + * * @param baiduEndPoint the baiduEndPoint value to set. * @return the BaiduCredential object itself. */ @@ -77,8 +82,8 @@ public BaiduCredential withBaiduEndPoint(String baiduEndPoint) { } /** - * Get the baiduSecretKey property: Baidu Secret Key. - * + * Get the baiduSecretKey property: Gets or sets baidu Secret Key. + * * @return the baiduSecretKey value. */ public String baiduSecretKey() { @@ -86,8 +91,8 @@ public String baiduSecretKey() { } /** - * Set the baiduSecretKey property: Baidu Secret Key. - * + * Set the baiduSecretKey property: Gets or sets baidu Secret Key. + * * @param baiduSecretKey the baiduSecretKey value to set. * @return the BaiduCredential object itself. */ @@ -101,12 +106,17 @@ public BaiduCredential withBaiduSecretKey(String baiduSecretKey) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model BaiduCredential")); + } else { innerProperties().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(BaiduCredential.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BrowserCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BrowserCredential.java new file mode 100644 index 0000000000000..127a697ef6c0a --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/BrowserCredential.java @@ -0,0 +1,122 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.notificationhubs.fluent.models.BrowserCredentialProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Description of a NotificationHub BrowserCredential. + */ +@Fluent +public final class BrowserCredential { + /* + * Description of a NotificationHub BrowserCredential. + */ + @JsonProperty(value = "properties", required = true) + private BrowserCredentialProperties innerProperties = new BrowserCredentialProperties(); + + /** + * Creates an instance of BrowserCredential class. + */ + public BrowserCredential() { + } + + /** + * Get the innerProperties property: Description of a NotificationHub BrowserCredential. + * + * @return the innerProperties value. + */ + private BrowserCredentialProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the subject property: Gets or sets web push subject. + * + * @return the subject value. + */ + public String subject() { + return this.innerProperties() == null ? null : this.innerProperties().subject(); + } + + /** + * Set the subject property: Gets or sets web push subject. + * + * @param subject the subject value to set. + * @return the BrowserCredential object itself. + */ + public BrowserCredential withSubject(String subject) { + if (this.innerProperties() == null) { + this.innerProperties = new BrowserCredentialProperties(); + } + this.innerProperties().withSubject(subject); + return this; + } + + /** + * Get the vapidPrivateKey property: Gets or sets VAPID private key. + * + * @return the vapidPrivateKey value. + */ + public String vapidPrivateKey() { + return this.innerProperties() == null ? null : this.innerProperties().vapidPrivateKey(); + } + + /** + * Set the vapidPrivateKey property: Gets or sets VAPID private key. + * + * @param vapidPrivateKey the vapidPrivateKey value to set. + * @return the BrowserCredential object itself. + */ + public BrowserCredential withVapidPrivateKey(String vapidPrivateKey) { + if (this.innerProperties() == null) { + this.innerProperties = new BrowserCredentialProperties(); + } + this.innerProperties().withVapidPrivateKey(vapidPrivateKey); + return this; + } + + /** + * Get the vapidPublicKey property: Gets or sets VAPID public key. + * + * @return the vapidPublicKey value. + */ + public String vapidPublicKey() { + return this.innerProperties() == null ? null : this.innerProperties().vapidPublicKey(); + } + + /** + * Set the vapidPublicKey property: Gets or sets VAPID public key. + * + * @param vapidPublicKey the vapidPublicKey value to set. + * @return the BrowserCredential object itself. + */ + public BrowserCredential withVapidPublicKey(String vapidPublicKey) { + if (this.innerProperties() == null) { + this.innerProperties = new BrowserCredentialProperties(); + } + this.innerProperties().withVapidPublicKey(vapidPublicKey); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model BrowserCredential")); + } else { + innerProperties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(BrowserCredential.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityParameters.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityParameters.java index 4e52f753c5604..cac2fe6c2837a 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityParameters.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityParameters.java @@ -6,63 +6,68 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Parameters supplied to the Check Name Availability for Namespace and NotificationHubs. */ +/** + * Parameters supplied to the Check Name Availability for Namespace and + * NotificationHubs. + */ @Fluent public final class CheckAvailabilityParameters { - @JsonIgnore private final ClientLogger logger = new ClientLogger(CheckAvailabilityParameters.class); - /* - * Resource Id + * Gets resource Id */ @JsonProperty(value = "id", access = JsonProperty.Access.WRITE_ONLY) private String id; /* - * Resource name + * Gets or sets resource name */ @JsonProperty(value = "name", required = true) private String name; /* - * Resource type + * Gets resource type */ @JsonProperty(value = "type", access = JsonProperty.Access.WRITE_ONLY) private String type; /* - * Resource location + * Gets or sets resource location */ @JsonProperty(value = "location") private String location; /* - * Resource tags + * Gets or sets resource tags */ @JsonProperty(value = "tags") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; /* - * The sku of the created namespace + * Not used and deprecated since API version 2023-01-01-preview + */ + @JsonProperty(value = "isAvailiable") + private Boolean isAvailiable; + + /* + * The Sku description for a namespace */ @JsonProperty(value = "sku") private Sku sku; - /* - * True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. + /** + * Creates an instance of CheckAvailabilityParameters class. */ - @JsonProperty(value = "isAvailiable") - private Boolean isAvailiable; + public CheckAvailabilityParameters() { + } /** - * Get the id property: Resource Id. - * + * Get the id property: Gets resource Id. + * * @return the id value. */ public String id() { @@ -70,8 +75,8 @@ public String id() { } /** - * Get the name property: Resource name. - * + * Get the name property: Gets or sets resource name. + * * @return the name value. */ public String name() { @@ -79,8 +84,8 @@ public String name() { } /** - * Set the name property: Resource name. - * + * Set the name property: Gets or sets resource name. + * * @param name the name value to set. * @return the CheckAvailabilityParameters object itself. */ @@ -90,8 +95,8 @@ public CheckAvailabilityParameters withName(String name) { } /** - * Get the type property: Resource type. - * + * Get the type property: Gets resource type. + * * @return the type value. */ public String type() { @@ -99,8 +104,8 @@ public String type() { } /** - * Get the location property: Resource location. - * + * Get the location property: Gets or sets resource location. + * * @return the location value. */ public String location() { @@ -108,8 +113,8 @@ public String location() { } /** - * Set the location property: Resource location. - * + * Set the location property: Gets or sets resource location. + * * @param location the location value to set. * @return the CheckAvailabilityParameters object itself. */ @@ -119,8 +124,8 @@ public CheckAvailabilityParameters withLocation(String location) { } /** - * Get the tags property: Resource tags. - * + * Get the tags property: Gets or sets resource tags. + * * @return the tags value. */ public Map tags() { @@ -128,8 +133,8 @@ public Map tags() { } /** - * Set the tags property: Resource tags. - * + * Set the tags property: Gets or sets resource tags. + * * @param tags the tags value to set. * @return the CheckAvailabilityParameters object itself. */ @@ -139,61 +144,59 @@ public CheckAvailabilityParameters withTags(Map tags) { } /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. + * Get the isAvailiable property: Not used and deprecated since API version 2023-01-01-preview. + * + * @return the isAvailiable value. */ - public Sku sku() { - return this.sku; + public Boolean isAvailiable() { + return this.isAvailiable; } /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. + * Set the isAvailiable property: Not used and deprecated since API version 2023-01-01-preview. + * + * @param isAvailiable the isAvailiable value to set. * @return the CheckAvailabilityParameters object itself. */ - public CheckAvailabilityParameters withSku(Sku sku) { - this.sku = sku; + public CheckAvailabilityParameters withIsAvailiable(Boolean isAvailiable) { + this.isAvailiable = isAvailiable; return this; } /** - * Get the isAvailiable property: True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. - * - * @return the isAvailiable value. + * Get the sku property: The Sku description for a namespace. + * + * @return the sku value. */ - public Boolean isAvailiable() { - return this.isAvailiable; + public Sku sku() { + return this.sku; } /** - * Set the isAvailiable property: True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. - * - * @param isAvailiable the isAvailiable value to set. + * Set the sku property: The Sku description for a namespace. + * + * @param sku the sku value to set. * @return the CheckAvailabilityParameters object itself. */ - public CheckAvailabilityParameters withIsAvailiable(Boolean isAvailiable) { - this.isAvailiable = isAvailiable; + public CheckAvailabilityParameters withSku(Sku sku) { + this.sku = sku; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property name in model CheckAvailabilityParameters")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property name in model CheckAvailabilityParameters")); } if (sku() != null) { sku().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(CheckAvailabilityParameters.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityResult.java index 4b5a1065dce74..de2080caa999f 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityResult.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/CheckAvailabilityResult.java @@ -4,64 +4,74 @@ package com.azure.resourcemanager.notificationhubs.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; import java.util.Map; -/** An immutable client-side representation of CheckAvailabilityResult. */ +/** + * An immutable client-side representation of CheckAvailabilityResult. + */ public interface CheckAvailabilityResult { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the location property: The geo-location where the resource lives. - * + * Gets the isAvailiable property: Gets or sets true if the name is available and can be used to + * create new Namespace/NotificationHub. Otherwise false. + * + * @return the isAvailiable value. + */ + Boolean isAvailiable(); + + /** + * Gets the location property: Deprecated - only for compatibility. + * * @return the location value. */ String location(); /** - * Gets the tags property: Resource tags. - * + * Gets the tags property: Deprecated - only for compatibility. + * * @return the tags value. */ Map tags(); /** - * Gets the isAvailiable property: True if the name is available and can be used to create new - * Namespace/NotificationHub. Otherwise false. - * - * @return the isAvailiable value. + * Gets the sku property: The Sku description for a namespace. + * + * @return the sku value. */ - Boolean isAvailiable(); + Sku sku(); /** - * Gets the sku property: The sku of the created namespace. - * - * @return the sku value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - Sku sku(); + SystemData systemData(); /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner object. - * + * * @return the inner object. */ CheckAvailabilityResultInner innerModel(); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/DebugSendResponse.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/DebugSendResponse.java index 781df439509d8..51863393ab90d 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/DebugSendResponse.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/DebugSendResponse.java @@ -4,77 +4,81 @@ package com.azure.resourcemanager.notificationhubs.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResponseInner; +import java.util.List; import java.util.Map; -/** An immutable client-side representation of DebugSendResponse. */ +/** + * An immutable client-side representation of DebugSendResponse. + */ public interface DebugSendResponse { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the location property: The geo-location where the resource lives. - * + * Gets the location property: Deprecated - only for compatibility. + * * @return the location value. */ String location(); /** - * Gets the tags property: Resource tags. - * + * Gets the tags property: Deprecated - only for compatibility. + * * @return the tags value. */ Map tags(); /** - * Gets the sku property: The sku of the created namespace. - * - * @return the sku value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - Sku sku(); + SystemData systemData(); /** - * Gets the success property: successful send. - * + * Gets the success property: Gets or sets successful send. + * * @return the success value. */ - Float success(); + Long success(); /** - * Gets the failure property: send failure. - * + * Gets the failure property: Gets or sets send failure. + * * @return the failure value. */ - Float failure(); + Long failure(); /** - * Gets the results property: actual failure description. - * + * Gets the results property: Gets or sets actual failure description. + * * @return the results value. */ - Object results(); + List results(); /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResponseInner object. - * + * * @return the inner object. */ DebugSendResponseInner innerModel(); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/FcmV1Credential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/FcmV1Credential.java new file mode 100644 index 0000000000000..0c97113303e94 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/FcmV1Credential.java @@ -0,0 +1,122 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.notificationhubs.fluent.models.FcmV1CredentialProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Description of a NotificationHub FcmV1Credential. + */ +@Fluent +public final class FcmV1Credential { + /* + * Description of a NotificationHub FcmV1Credential. + */ + @JsonProperty(value = "properties", required = true) + private FcmV1CredentialProperties innerProperties = new FcmV1CredentialProperties(); + + /** + * Creates an instance of FcmV1Credential class. + */ + public FcmV1Credential() { + } + + /** + * Get the innerProperties property: Description of a NotificationHub FcmV1Credential. + * + * @return the innerProperties value. + */ + private FcmV1CredentialProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the clientEmail property: Gets or sets client email. + * + * @return the clientEmail value. + */ + public String clientEmail() { + return this.innerProperties() == null ? null : this.innerProperties().clientEmail(); + } + + /** + * Set the clientEmail property: Gets or sets client email. + * + * @param clientEmail the clientEmail value to set. + * @return the FcmV1Credential object itself. + */ + public FcmV1Credential withClientEmail(String clientEmail) { + if (this.innerProperties() == null) { + this.innerProperties = new FcmV1CredentialProperties(); + } + this.innerProperties().withClientEmail(clientEmail); + return this; + } + + /** + * Get the privateKey property: Gets or sets private key. + * + * @return the privateKey value. + */ + public String privateKey() { + return this.innerProperties() == null ? null : this.innerProperties().privateKey(); + } + + /** + * Set the privateKey property: Gets or sets private key. + * + * @param privateKey the privateKey value to set. + * @return the FcmV1Credential object itself. + */ + public FcmV1Credential withPrivateKey(String privateKey) { + if (this.innerProperties() == null) { + this.innerProperties = new FcmV1CredentialProperties(); + } + this.innerProperties().withPrivateKey(privateKey); + return this; + } + + /** + * Get the projectId property: Gets or sets project id. + * + * @return the projectId value. + */ + public String projectId() { + return this.innerProperties() == null ? null : this.innerProperties().projectId(); + } + + /** + * Set the projectId property: Gets or sets project id. + * + * @param projectId the projectId value to set. + * @return the FcmV1Credential object itself. + */ + public FcmV1Credential withProjectId(String projectId) { + if (this.innerProperties() == null) { + this.innerProperties = new FcmV1CredentialProperties(); + } + this.innerProperties().withProjectId(projectId); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model FcmV1Credential")); + } else { + innerProperties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(FcmV1Credential.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/GcmCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/GcmCredential.java index 7974d3ea2431e..cb9d69db10a74 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/GcmCredential.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/GcmCredential.java @@ -7,23 +7,28 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.GcmCredentialProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub GcmCredential. */ +/** + * Description of a NotificationHub GcmCredential. + */ @Fluent public final class GcmCredential { - @JsonIgnore private final ClientLogger logger = new ClientLogger(GcmCredential.class); - /* - * Properties of NotificationHub GcmCredential. + * Description of a NotificationHub GcmCredential. + */ + @JsonProperty(value = "properties", required = true) + private GcmCredentialProperties innerProperties = new GcmCredentialProperties(); + + /** + * Creates an instance of GcmCredential class. */ - @JsonProperty(value = "properties") - private GcmCredentialProperties innerProperties; + public GcmCredential() { + } /** - * Get the innerProperties property: Properties of NotificationHub GcmCredential. - * + * Get the innerProperties property: Description of a NotificationHub GcmCredential. + * * @return the innerProperties value. */ private GcmCredentialProperties innerProperties() { @@ -31,8 +36,8 @@ private GcmCredentialProperties innerProperties() { } /** - * Get the gcmEndpoint property: The FCM legacy endpoint. Default value is 'https://fcm.googleapis.com/fcm/send'. - * + * Get the gcmEndpoint property: Gets or sets the GCM endpoint. + * * @return the gcmEndpoint value. */ public String gcmEndpoint() { @@ -40,8 +45,8 @@ public String gcmEndpoint() { } /** - * Set the gcmEndpoint property: The FCM legacy endpoint. Default value is 'https://fcm.googleapis.com/fcm/send'. - * + * Set the gcmEndpoint property: Gets or sets the GCM endpoint. + * * @param gcmEndpoint the gcmEndpoint value to set. * @return the GcmCredential object itself. */ @@ -54,8 +59,8 @@ public GcmCredential withGcmEndpoint(String gcmEndpoint) { } /** - * Get the googleApiKey property: The Google API key. - * + * Get the googleApiKey property: Gets or sets the Google API key. + * * @return the googleApiKey value. */ public String googleApiKey() { @@ -63,8 +68,8 @@ public String googleApiKey() { } /** - * Set the googleApiKey property: The Google API key. - * + * Set the googleApiKey property: Gets or sets the Google API key. + * * @param googleApiKey the googleApiKey value to set. * @return the GcmCredential object itself. */ @@ -78,12 +83,17 @@ public GcmCredential withGoogleApiKey(String googleApiKey) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model GcmCredential")); + } else { innerProperties().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(GcmCredential.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/IpRule.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/IpRule.java new file mode 100644 index 0000000000000..213032752de74 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/IpRule.java @@ -0,0 +1,92 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * A network authorization rule that filters traffic based on IP address. + */ +@Fluent +public final class IpRule { + /* + * IP mask. + */ + @JsonProperty(value = "ipMask", required = true) + private String ipMask; + + /* + * List of access rights. + */ + @JsonProperty(value = "rights", required = true) + private List rights; + + /** + * Creates an instance of IpRule class. + */ + public IpRule() { + } + + /** + * Get the ipMask property: IP mask. + * + * @return the ipMask value. + */ + public String ipMask() { + return this.ipMask; + } + + /** + * Set the ipMask property: IP mask. + * + * @param ipMask the ipMask value to set. + * @return the IpRule object itself. + */ + public IpRule withIpMask(String ipMask) { + this.ipMask = ipMask; + return this; + } + + /** + * Get the rights property: List of access rights. + * + * @return the rights value. + */ + public List rights() { + return this.rights; + } + + /** + * Set the rights property: List of access rights. + * + * @param rights the rights value to set. + * @return the IpRule object itself. + */ + public IpRule withRights(List rights) { + this.rights = rights; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (ipMask() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property ipMask in model IpRule")); + } + if (rights() == null) { + throw LOGGER + .logExceptionAsError(new IllegalArgumentException("Missing required property rights in model IpRule")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(IpRule.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/LogSpecification.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/LogSpecification.java new file mode 100644 index 0000000000000..7671c04a9e64a --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/LogSpecification.java @@ -0,0 +1,100 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * A single log category specification. + */ +@Fluent +public final class LogSpecification { + /* + * Name of the log category. + */ + @JsonProperty(value = "name", access = JsonProperty.Access.WRITE_ONLY) + private String name; + + /* + * Display name of the log category. + */ + @JsonProperty(value = "displayName", access = JsonProperty.Access.WRITE_ONLY) + private String displayName; + + /* + * Duration of data written to a single blob. + */ + @JsonProperty(value = "blobDuration", access = JsonProperty.Access.WRITE_ONLY) + private String blobDuration; + + /* + * Category group for the log specification. + */ + @JsonProperty(value = "categoryGroups") + private List categoryGroups; + + /** + * Creates an instance of LogSpecification class. + */ + public LogSpecification() { + } + + /** + * Get the name property: Name of the log category. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Get the displayName property: Display name of the log category. + * + * @return the displayName value. + */ + public String displayName() { + return this.displayName; + } + + /** + * Get the blobDuration property: Duration of data written to a single blob. + * + * @return the blobDuration value. + */ + public String blobDuration() { + return this.blobDuration; + } + + /** + * Get the categoryGroups property: Category group for the log specification. + * + * @return the categoryGroups value. + */ + public List categoryGroups() { + return this.categoryGroups; + } + + /** + * Set the categoryGroups property: Category group for the log specification. + * + * @param categoryGroups the categoryGroups value to set. + * @return the LogSpecification object itself. + */ + public LogSpecification withCategoryGroups(List categoryGroups) { + this.categoryGroups = categoryGroups; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MetricSpecification.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MetricSpecification.java new file mode 100644 index 0000000000000..561adaaee7293 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MetricSpecification.java @@ -0,0 +1,171 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * A metric specification. + */ +@Immutable +public final class MetricSpecification { + /* + * Metric name / id. + */ + @JsonProperty(value = "name", access = JsonProperty.Access.WRITE_ONLY) + private String name; + + /* + * User-visible metric name. + */ + @JsonProperty(value = "displayName", access = JsonProperty.Access.WRITE_ONLY) + private String displayName; + + /* + * Description of the metric. + */ + @JsonProperty(value = "displayDescription", access = JsonProperty.Access.WRITE_ONLY) + private String displayDescription; + + /* + * Metric unit. + */ + @JsonProperty(value = "unit", access = JsonProperty.Access.WRITE_ONLY) + private String unit; + + /* + * Type of the aggregation (Average, Minimum, Maximum, Total or Count). + */ + @JsonProperty(value = "aggregationType", access = JsonProperty.Access.WRITE_ONLY) + private String aggregationType; + + /* + * List of availabilities. + */ + @JsonProperty(value = "availabilities", access = JsonProperty.Access.WRITE_ONLY) + private List availabilities; + + /* + * List of supported time grain types. + */ + @JsonProperty(value = "supportedTimeGrainTypes", access = JsonProperty.Access.WRITE_ONLY) + private List supportedTimeGrainTypes; + + /* + * The matching regex pattern to be applied to the field pointed by the "metricsFilterPathSelector" flag in the ARM + * manifest. + */ + @JsonProperty(value = "metricFilterPattern", access = JsonProperty.Access.WRITE_ONLY) + private String metricFilterPattern; + + /* + * Optional property. If set to true, then zero will be returned for time duration where no metric is emitted / + * published. + */ + @JsonProperty(value = "fillGapWithZero", access = JsonProperty.Access.WRITE_ONLY) + private Boolean fillGapWithZero; + + /** + * Creates an instance of MetricSpecification class. + */ + public MetricSpecification() { + } + + /** + * Get the name property: Metric name / id. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Get the displayName property: User-visible metric name. + * + * @return the displayName value. + */ + public String displayName() { + return this.displayName; + } + + /** + * Get the displayDescription property: Description of the metric. + * + * @return the displayDescription value. + */ + public String displayDescription() { + return this.displayDescription; + } + + /** + * Get the unit property: Metric unit. + * + * @return the unit value. + */ + public String unit() { + return this.unit; + } + + /** + * Get the aggregationType property: Type of the aggregation (Average, Minimum, Maximum, Total or Count). + * + * @return the aggregationType value. + */ + public String aggregationType() { + return this.aggregationType; + } + + /** + * Get the availabilities property: List of availabilities. + * + * @return the availabilities value. + */ + public List availabilities() { + return this.availabilities; + } + + /** + * Get the supportedTimeGrainTypes property: List of supported time grain types. + * + * @return the supportedTimeGrainTypes value. + */ + public List supportedTimeGrainTypes() { + return this.supportedTimeGrainTypes; + } + + /** + * Get the metricFilterPattern property: The matching regex pattern to be applied to the field pointed by the + * "metricsFilterPathSelector" flag in the ARM manifest. + * + * @return the metricFilterPattern value. + */ + public String metricFilterPattern() { + return this.metricFilterPattern; + } + + /** + * Get the fillGapWithZero property: Optional property. If set to true, then zero will be returned for time + * duration where no metric is emitted / published. + * + * @return the fillGapWithZero value. + */ + public Boolean fillGapWithZero() { + return this.fillGapWithZero; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (availabilities() != null) { + availabilities().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MpnsCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MpnsCredential.java index 368884a11e7ea..8d78c65557dcf 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MpnsCredential.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/MpnsCredential.java @@ -7,23 +7,28 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.MpnsCredentialProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub MpnsCredential. */ +/** + * Description of a NotificationHub MpnsCredential. + */ @Fluent public final class MpnsCredential { - @JsonIgnore private final ClientLogger logger = new ClientLogger(MpnsCredential.class); - /* - * Properties of NotificationHub MpnsCredential. + * Description of a NotificationHub MpnsCredential. + */ + @JsonProperty(value = "properties", required = true) + private MpnsCredentialProperties innerProperties = new MpnsCredentialProperties(); + + /** + * Creates an instance of MpnsCredential class. */ - @JsonProperty(value = "properties") - private MpnsCredentialProperties innerProperties; + public MpnsCredential() { + } /** - * Get the innerProperties property: Properties of NotificationHub MpnsCredential. - * + * Get the innerProperties property: Description of a NotificationHub MpnsCredential. + * * @return the innerProperties value. */ private MpnsCredentialProperties innerProperties() { @@ -31,8 +36,8 @@ private MpnsCredentialProperties innerProperties() { } /** - * Get the mpnsCertificate property: The MPNS certificate. - * + * Get the mpnsCertificate property: Gets or sets the MPNS certificate. + * * @return the mpnsCertificate value. */ public String mpnsCertificate() { @@ -40,8 +45,8 @@ public String mpnsCertificate() { } /** - * Set the mpnsCertificate property: The MPNS certificate. - * + * Set the mpnsCertificate property: Gets or sets the MPNS certificate. + * * @param mpnsCertificate the mpnsCertificate value to set. * @return the MpnsCredential object itself. */ @@ -54,8 +59,8 @@ public MpnsCredential withMpnsCertificate(String mpnsCertificate) { } /** - * Get the certificateKey property: The certificate key for this credential. - * + * Get the certificateKey property: Gets or sets the certificate key for this credential. + * * @return the certificateKey value. */ public String certificateKey() { @@ -63,8 +68,8 @@ public String certificateKey() { } /** - * Set the certificateKey property: The certificate key for this credential. - * + * Set the certificateKey property: Gets or sets the certificate key for this credential. + * * @param certificateKey the certificateKey value to set. * @return the MpnsCredential object itself. */ @@ -77,8 +82,8 @@ public MpnsCredential withCertificateKey(String certificateKey) { } /** - * Get the thumbprint property: The MPNS certificate Thumbprint. - * + * Get the thumbprint property: Gets or sets the MPNS certificate Thumbprint. + * * @return the thumbprint value. */ public String thumbprint() { @@ -86,8 +91,8 @@ public String thumbprint() { } /** - * Set the thumbprint property: The MPNS certificate Thumbprint. - * + * Set the thumbprint property: Gets or sets the MPNS certificate Thumbprint. + * * @param thumbprint the thumbprint value to set. * @return the MpnsCredential object itself. */ @@ -101,12 +106,17 @@ public MpnsCredential withThumbprint(String thumbprint) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model MpnsCredential")); + } else { innerProperties().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(MpnsCredential.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceCreateOrUpdateParameters.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceCreateOrUpdateParameters.java deleted file mode 100644 index f35bedb0b4bfc..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceCreateOrUpdateParameters.java +++ /dev/null @@ -1,403 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.time.OffsetDateTime; -import java.util.Map; - -/** Parameters supplied to the CreateOrUpdate Namespace operation. */ -@Fluent -public final class NamespaceCreateOrUpdateParameters extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NamespaceCreateOrUpdateParameters.class); - - /* - * Properties of the Namespace. - */ - @JsonProperty(value = "properties") - private NamespaceProperties innerProperties; - - /* - * The sku of the created namespace - */ - @JsonProperty(value = "sku") - private Sku sku; - - /** - * Get the innerProperties property: Properties of the Namespace. - * - * @return the innerProperties value. - */ - private NamespaceProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. - */ - public Sku sku() { - return this.sku; - } - - /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withSku(Sku sku) { - this.sku = sku; - return this; - } - - /** {@inheritDoc} */ - @Override - public NamespaceCreateOrUpdateParameters withLocation(String location) { - super.withLocation(location); - return this; - } - - /** {@inheritDoc} */ - @Override - public NamespaceCreateOrUpdateParameters withTags(Map tags) { - super.withTags(tags); - return this; - } - - /** - * Get the name property: The name of the namespace. - * - * @return the name value. - */ - public String namePropertiesName() { - return this.innerProperties() == null ? null : this.innerProperties().name(); - } - - /** - * Set the name property: The name of the namespace. - * - * @param name the name value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withNamePropertiesName(String name) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withName(name); - return this; - } - - /** - * Get the provisioningState property: Provisioning state of the Namespace. - * - * @return the provisioningState value. - */ - public String provisioningState() { - return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); - } - - /** - * Set the provisioningState property: Provisioning state of the Namespace. - * - * @param provisioningState the provisioningState value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withProvisioningState(String provisioningState) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withProvisioningState(provisioningState); - return this; - } - - /** - * Get the region property: Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North Central - * US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West Europe. - * - * @return the region value. - */ - public String region() { - return this.innerProperties() == null ? null : this.innerProperties().region(); - } - - /** - * Set the region property: Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North Central - * US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West Europe. - * - * @param region the region value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withRegion(String region) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withRegion(region); - return this; - } - - /** - * Get the metricId property: Identifier for Azure Insights metrics. - * - * @return the metricId value. - */ - public String metricId() { - return this.innerProperties() == null ? null : this.innerProperties().metricId(); - } - - /** - * Get the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. - * - * @return the status value. - */ - public String status() { - return this.innerProperties() == null ? null : this.innerProperties().status(); - } - - /** - * Set the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. - * - * @param status the status value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withStatus(String status) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withStatus(status); - return this; - } - - /** - * Get the createdAt property: The time the namespace was created. - * - * @return the createdAt value. - */ - public OffsetDateTime createdAt() { - return this.innerProperties() == null ? null : this.innerProperties().createdAt(); - } - - /** - * Set the createdAt property: The time the namespace was created. - * - * @param createdAt the createdAt value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withCreatedAt(OffsetDateTime createdAt) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withCreatedAt(createdAt); - return this; - } - - /** - * Get the updatedAt property: The time the namespace was updated. - * - * @return the updatedAt value. - */ - public OffsetDateTime updatedAt() { - return this.innerProperties() == null ? null : this.innerProperties().updatedAt(); - } - - /** - * Set the updatedAt property: The time the namespace was updated. - * - * @param updatedAt the updatedAt value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withUpdatedAt(OffsetDateTime updatedAt) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withUpdatedAt(updatedAt); - return this; - } - - /** - * Get the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @return the serviceBusEndpoint value. - */ - public String serviceBusEndpoint() { - return this.innerProperties() == null ? null : this.innerProperties().serviceBusEndpoint(); - } - - /** - * Set the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @param serviceBusEndpoint the serviceBusEndpoint value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withServiceBusEndpoint(String serviceBusEndpoint) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withServiceBusEndpoint(serviceBusEndpoint); - return this; - } - - /** - * Get the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @return the subscriptionId value. - */ - public String subscriptionId() { - return this.innerProperties() == null ? null : this.innerProperties().subscriptionId(); - } - - /** - * Set the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @param subscriptionId the subscriptionId value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withSubscriptionId(String subscriptionId) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withSubscriptionId(subscriptionId); - return this; - } - - /** - * Get the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @return the scaleUnit value. - */ - public String scaleUnit() { - return this.innerProperties() == null ? null : this.innerProperties().scaleUnit(); - } - - /** - * Set the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @param scaleUnit the scaleUnit value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withScaleUnit(String scaleUnit) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withScaleUnit(scaleUnit); - return this; - } - - /** - * Get the enabled property: Whether or not the namespace is currently enabled. - * - * @return the enabled value. - */ - public Boolean enabled() { - return this.innerProperties() == null ? null : this.innerProperties().enabled(); - } - - /** - * Set the enabled property: Whether or not the namespace is currently enabled. - * - * @param enabled the enabled value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withEnabled(Boolean enabled) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withEnabled(enabled); - return this; - } - - /** - * Get the critical property: Whether or not the namespace is set as Critical. - * - * @return the critical value. - */ - public Boolean critical() { - return this.innerProperties() == null ? null : this.innerProperties().critical(); - } - - /** - * Set the critical property: Whether or not the namespace is set as Critical. - * - * @param critical the critical value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withCritical(Boolean critical) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withCritical(critical); - return this; - } - - /** - * Get the dataCenter property: Data center for the namespace. - * - * @return the dataCenter value. - */ - public String dataCenter() { - return this.innerProperties() == null ? null : this.innerProperties().dataCenter(); - } - - /** - * Set the dataCenter property: Data center for the namespace. - * - * @param dataCenter the dataCenter value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withDataCenter(String dataCenter) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withDataCenter(dataCenter); - return this; - } - - /** - * Get the namespaceType property: The namespace type. - * - * @return the namespaceType value. - */ - public NamespaceType namespaceType() { - return this.innerProperties() == null ? null : this.innerProperties().namespaceType(); - } - - /** - * Set the namespaceType property: The namespace type. - * - * @param namespaceType the namespaceType value to set. - * @return the NamespaceCreateOrUpdateParameters object itself. - */ - public NamespaceCreateOrUpdateParameters withNamespaceType(NamespaceType namespaceType) { - if (this.innerProperties() == null) { - this.innerProperties = new NamespaceProperties(); - } - this.innerProperties().withNamespaceType(namespaceType); - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - if (sku() != null) { - sku().validate(); - } - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceListResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceListResult.java index c32c94d14e523..cd7f5ec58d7c7 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceListResult.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceListResult.java @@ -4,76 +4,55 @@ package com.azure.resourcemanager.notificationhubs.models; -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.annotation.Immutable; import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of the List Namespace operation. */ -@Fluent +/** + * The response of the List Namespace operation. + */ +@Immutable public final class NamespaceListResult { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NamespaceListResult.class); - /* - * Result of the List Namespace operation. + * Gets or sets result of the List AuthorizationRules operation. */ - @JsonProperty(value = "value") + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) private List value; /* - * Link to the next set of results. Not empty if Value contains incomplete - * list of Namespaces + * Gets or sets link to the next set of results. */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; /** - * Get the value property: Result of the List Namespace operation. - * - * @return the value value. + * Creates an instance of NamespaceListResult class. */ - public List value() { - return this.value; + public NamespaceListResult() { } /** - * Set the value property: Result of the List Namespace operation. - * - * @param value the value value to set. - * @return the NamespaceListResult object itself. + * Get the value property: Gets or sets result of the List AuthorizationRules operation. + * + * @return the value value. */ - public NamespaceListResult withValue(List value) { - this.value = value; - return this; + public List value() { + return this.value; } /** - * Get the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of - * Namespaces. - * + * Get the nextLink property: Gets or sets link to the next set of results. + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of - * Namespaces. - * - * @param nextLink the nextLink value to set. - * @return the NamespaceListResult object itself. - */ - public NamespaceListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespacePatchParameters.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespacePatchParameters.java index 63420a9686716..117d9d4710b17 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespacePatchParameters.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespacePatchParameters.java @@ -5,78 +5,112 @@ package com.azure.resourcemanager.notificationhubs.models; import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties; import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Parameters supplied to the Patch Namespace operation. */ +/** + * Patch parameter for NamespaceResource. + */ @Fluent public final class NamespacePatchParameters { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NamespacePatchParameters.class); + /* + * The Sku description for a namespace + */ + @JsonProperty(value = "sku") + private Sku sku; + + /* + * Represents namespace properties. + */ + @JsonProperty(value = "properties") + private NamespaceProperties properties; /* - * Resource tags + * Dictionary of */ @JsonProperty(value = "tags") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /* - * The sku of the created namespace + /** + * Creates an instance of NamespacePatchParameters class. */ - @JsonProperty(value = "sku") - private Sku sku; + public NamespacePatchParameters() { + } /** - * Get the tags property: Resource tags. - * - * @return the tags value. + * Get the sku property: The Sku description for a namespace. + * + * @return the sku value. */ - public Map tags() { - return this.tags; + public Sku sku() { + return this.sku; } /** - * Set the tags property: Resource tags. - * - * @param tags the tags value to set. + * Set the sku property: The Sku description for a namespace. + * + * @param sku the sku value to set. * @return the NamespacePatchParameters object itself. */ - public NamespacePatchParameters withTags(Map tags) { - this.tags = tags; + public NamespacePatchParameters withSku(Sku sku) { + this.sku = sku; return this; } /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. + * Get the properties property: Represents namespace properties. + * + * @return the properties value. */ - public Sku sku() { - return this.sku; + public NamespaceProperties properties() { + return this.properties; } /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. + * Set the properties property: Represents namespace properties. + * + * @param properties the properties value to set. * @return the NamespacePatchParameters object itself. */ - public NamespacePatchParameters withSku(Sku sku) { - this.sku = sku; + public NamespacePatchParameters withProperties(NamespaceProperties properties) { + this.properties = properties; + return this; + } + + /** + * Get the tags property: Dictionary of <string>. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; + } + + /** + * Set the tags property: Dictionary of <string>. + * + * @param tags the tags value to set. + * @return the NamespacePatchParameters object itself. + */ + public NamespacePatchParameters withTags(Map tags) { + this.tags = tags; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sku() != null) { sku().validate(); } + if (properties() != null) { + properties().validate(); + } } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceResource.java index efe1c33c9e3ab..468cac691bd54 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceResource.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceResource.java @@ -4,458 +4,565 @@ package com.azure.resourcemanager.notificationhubs.models; +import com.azure.core.http.rest.Response; import com.azure.core.management.Region; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; +import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties; import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentials; import java.time.OffsetDateTime; +import java.util.List; import java.util.Map; -/** An immutable client-side representation of NamespaceResource. */ +/** + * An immutable client-side representation of NamespaceResource. + */ public interface NamespaceResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** - * Gets the sku property: The sku of the created namespace. - * + * Gets the sku property: The Sku description for a namespace. + * * @return the sku value. */ Sku sku(); /** - * Gets the namePropertiesName property: The name of the namespace. - * + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the namePropertiesName property: Name of the Notification Hubs namespace. This is immutable property, set + * automatically + * by the service when the namespace is created. + * * @return the namePropertiesName value. */ String namePropertiesName(); /** - * Gets the provisioningState property: Provisioning state of the Namespace. - * + * Gets the provisioningState property: Defines values for OperationProvisioningState. + * * @return the provisioningState value. */ - String provisioningState(); + OperationProvisioningState provisioningState(); /** - * Gets the region property: Specifies the targeted region in which the namespace should be created. It can be any - * of the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, North - * Central US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, North Europe, West - * Europe. - * + * Gets the status property: Namespace status. + * + * @return the status value. + */ + NamespaceStatus status(); + + /** + * Gets the enabled property: Gets or sets whether or not the namespace is currently enabled. + * + * @return the enabled value. + */ + Boolean enabled(); + + /** + * Gets the critical property: Gets or sets whether or not the namespace is set as Critical. + * + * @return the critical value. + */ + Boolean critical(); + + /** + * Gets the subscriptionId property: Namespace subscription id. + * + * @return the subscriptionId value. + */ + String subscriptionId(); + + /** + * Gets the region property: Region. The value is always set to the same value as Namespace.Location, so we are + * deprecating + * this property. + * * @return the region value. */ String region(); /** - * Gets the metricId property: Identifier for Azure Insights metrics. - * + * Gets the metricId property: Azure Insights Metrics id. + * * @return the metricId value. */ String metricId(); /** - * Gets the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 - * = Suspended4 = Deleting. - * - * @return the status value. - */ - String status(); - - /** - * Gets the createdAt property: The time the namespace was created. - * + * Gets the createdAt property: Time when the namespace was created. + * * @return the createdAt value. */ OffsetDateTime createdAt(); /** - * Gets the updatedAt property: The time the namespace was updated. - * + * Gets the updatedAt property: Time when the namespace was updated. + * * @return the updatedAt value. */ OffsetDateTime updatedAt(); /** - * Gets the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations. - * - * @return the serviceBusEndpoint value. + * Gets the namespaceType property: Defines values for NamespaceType. + * + * @return the namespaceType value. */ - String serviceBusEndpoint(); + NamespaceType namespaceType(); /** - * Gets the subscriptionId property: The Id of the Azure subscription associated with the namespace. - * - * @return the subscriptionId value. + * Gets the replicationRegion property: Allowed replication region. + * + * @return the replicationRegion value. */ - String subscriptionId(); + ReplicationRegion replicationRegion(); /** - * Gets the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @return the scaleUnit value. + * Gets the zoneRedundancy property: Namespace SKU name. + * + * @return the zoneRedundancy value. */ - String scaleUnit(); + ZoneRedundancyPreference zoneRedundancy(); /** - * Gets the enabled property: Whether or not the namespace is currently enabled. - * - * @return the enabled value. + * Gets the networkAcls property: A collection of network authorization rules. + * + * @return the networkAcls value. */ - Boolean enabled(); + NetworkAcls networkAcls(); /** - * Gets the critical property: Whether or not the namespace is set as Critical. - * - * @return the critical value. + * Gets the pnsCredentials property: Collection of Notification Hub or Notification Hub Namespace PNS credentials. + * + * @return the pnsCredentials value. */ - Boolean critical(); + PnsCredentials pnsCredentials(); + + /** + * Gets the serviceBusEndpoint property: Gets or sets endpoint you can use to perform NotificationHub + * operations. + * + * @return the serviceBusEndpoint value. + */ + String serviceBusEndpoint(); + + /** + * Gets the privateEndpointConnections property: Private Endpoint Connections for namespace. + * + * @return the privateEndpointConnections value. + */ + List privateEndpointConnections(); /** - * Gets the dataCenter property: Data center for the namespace. - * + * Gets the scaleUnit property: Gets or sets scaleUnit where the namespace gets created. + * + * @return the scaleUnit value. + */ + String scaleUnit(); + + /** + * Gets the dataCenter property: Deprecated. + * * @return the dataCenter value. */ String dataCenter(); /** - * Gets the namespaceType property: The namespace type. - * - * @return the namespaceType value. + * Gets the publicNetworkAccess property: Type of public network access. + * + * @return the publicNetworkAccess value. */ - NamespaceType namespaceType(); + PublicNetworkAccess publicNetworkAccess(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner object. - * + * * @return the inner object. */ NamespaceResourceInner innerModel(); - /** The entirety of the NamespaceResource definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the NamespaceResource definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithSku, DefinitionStages.WithCreate { } - /** The NamespaceResource definition stages. */ + + /** + * The NamespaceResource definition stages. + */ interface DefinitionStages { - /** The first stage of the NamespaceResource definition. */ + /** + * The first stage of the NamespaceResource definition. + */ interface Blank extends WithLocation { } - /** The stage of the NamespaceResource definition allowing to specify location. */ + + /** + * The stage of the NamespaceResource definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ - WithResourceGroup withLocation(Region location); + WithResourceGroup withRegion(Region location); /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ - WithResourceGroup withLocation(String location); + WithResourceGroup withRegion(String location); } - /** The stage of the NamespaceResource definition allowing to specify parent resource. */ + + /** + * The stage of the NamespaceResource definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @return the next definition stage. + */ + WithSku withExistingResourceGroup(String resourceGroupName); + } + + /** + * The stage of the NamespaceResource definition allowing to specify sku. + */ + interface WithSku { + /** + * Specifies the sku property: The Sku description for a namespace. + * + * @param sku The Sku description for a namespace. * @return the next definition stage. */ - WithCreate withExistingResourceGroup(String resourceGroupName); + WithCreate withSku(Sku sku); } + /** * The stage of the NamespaceResource definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithSku, - DefinitionStages.WithNamePropertiesName, - DefinitionStages.WithProvisioningState, - DefinitionStages.WithRegion, - DefinitionStages.WithStatus, - DefinitionStages.WithCreatedAt, - DefinitionStages.WithUpdatedAt, - DefinitionStages.WithServiceBusEndpoint, - DefinitionStages.WithSubscriptionId, - DefinitionStages.WithScaleUnit, - DefinitionStages.WithEnabled, - DefinitionStages.WithCritical, - DefinitionStages.WithDataCenter, - DefinitionStages.WithNamespaceType { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithProvisioningState, + DefinitionStages.WithStatus, DefinitionStages.WithNamespaceType, DefinitionStages.WithReplicationRegion, + DefinitionStages.WithZoneRedundancy, DefinitionStages.WithNetworkAcls, DefinitionStages.WithPnsCredentials, + DefinitionStages.WithScaleUnit, DefinitionStages.WithDataCenter, DefinitionStages.WithPublicNetworkAccess { /** * Executes the create request. - * + * * @return the created resource. */ NamespaceResource create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ NamespaceResource create(Context context); } - /** The stage of the NamespaceResource definition allowing to specify tags. */ + + /** + * The stage of the NamespaceResource definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the NamespaceResource definition allowing to specify sku. */ - interface WithSku { - /** - * Specifies the sku property: The sku of the created namespace. - * - * @param sku The sku of the created namespace. - * @return the next definition stage. - */ - WithCreate withSku(Sku sku); - } - /** The stage of the NamespaceResource definition allowing to specify namePropertiesName. */ - interface WithNamePropertiesName { - /** - * Specifies the namePropertiesName property: The name of the namespace.. - * - * @param namePropertiesName The name of the namespace. - * @return the next definition stage. - */ - WithCreate withNamePropertiesName(String namePropertiesName); - } - /** The stage of the NamespaceResource definition allowing to specify provisioningState. */ + + /** + * The stage of the NamespaceResource definition allowing to specify provisioningState. + */ interface WithProvisioningState { /** - * Specifies the provisioningState property: Provisioning state of the Namespace.. - * - * @param provisioningState Provisioning state of the Namespace. + * Specifies the provisioningState property: Defines values for OperationProvisioningState.. + * + * @param provisioningState Defines values for OperationProvisioningState. * @return the next definition stage. */ - WithCreate withProvisioningState(String provisioningState); + WithCreate withProvisioningState(OperationProvisioningState provisioningState); } - /** The stage of the NamespaceResource definition allowing to specify region. */ - interface WithRegion { + + /** + * The stage of the NamespaceResource definition allowing to specify status. + */ + interface WithStatus { /** - * Specifies the region property: Specifies the targeted region in which the namespace should be created. It - * can be any of the following values: Australia East, Australia Southeast, Central US, East US, East US 2, - * West US, North Central US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan - * West, North Europe, West Europe. - * - * @param region Specifies the targeted region in which the namespace should be created. It can be any of - * the following values: Australia East, Australia Southeast, Central US, East US, East US 2, West US, - * North Central US, South Central US, East Asia, Southeast Asia, Brazil South, Japan East, Japan West, - * North Europe, West Europe. + * Specifies the status property: Namespace status.. + * + * @param status Namespace status. * @return the next definition stage. */ - WithCreate withRegion(String region); + WithCreate withStatus(NamespaceStatus status); } - /** The stage of the NamespaceResource definition allowing to specify status. */ - interface WithStatus { + + /** + * The stage of the NamespaceResource definition allowing to specify namespaceType. + */ + interface WithNamespaceType { /** - * Specifies the status property: Status of the namespace. It can be any of these values:1 = Created/Active2 - * = Creating3 = Suspended4 = Deleting. - * - * @param status Status of the namespace. It can be any of these values:1 = Created/Active2 = Creating3 = - * Suspended4 = Deleting. + * Specifies the namespaceType property: Defines values for NamespaceType.. + * + * @param namespaceType Defines values for NamespaceType. * @return the next definition stage. */ - WithCreate withStatus(String status); + WithCreate withNamespaceType(NamespaceType namespaceType); } - /** The stage of the NamespaceResource definition allowing to specify createdAt. */ - interface WithCreatedAt { + + /** + * The stage of the NamespaceResource definition allowing to specify replicationRegion. + */ + interface WithReplicationRegion { /** - * Specifies the createdAt property: The time the namespace was created.. - * - * @param createdAt The time the namespace was created. + * Specifies the replicationRegion property: Allowed replication region. + * + * @param replicationRegion Allowed replication region. * @return the next definition stage. */ - WithCreate withCreatedAt(OffsetDateTime createdAt); + WithCreate withReplicationRegion(ReplicationRegion replicationRegion); } - /** The stage of the NamespaceResource definition allowing to specify updatedAt. */ - interface WithUpdatedAt { + + /** + * The stage of the NamespaceResource definition allowing to specify zoneRedundancy. + */ + interface WithZoneRedundancy { /** - * Specifies the updatedAt property: The time the namespace was updated.. - * - * @param updatedAt The time the namespace was updated. + * Specifies the zoneRedundancy property: Namespace SKU name.. + * + * @param zoneRedundancy Namespace SKU name. * @return the next definition stage. */ - WithCreate withUpdatedAt(OffsetDateTime updatedAt); + WithCreate withZoneRedundancy(ZoneRedundancyPreference zoneRedundancy); } - /** The stage of the NamespaceResource definition allowing to specify serviceBusEndpoint. */ - interface WithServiceBusEndpoint { + + /** + * The stage of the NamespaceResource definition allowing to specify networkAcls. + */ + interface WithNetworkAcls { /** - * Specifies the serviceBusEndpoint property: Endpoint you can use to perform NotificationHub operations.. - * - * @param serviceBusEndpoint Endpoint you can use to perform NotificationHub operations. + * Specifies the networkAcls property: A collection of network authorization rules.. + * + * @param networkAcls A collection of network authorization rules. * @return the next definition stage. */ - WithCreate withServiceBusEndpoint(String serviceBusEndpoint); + WithCreate withNetworkAcls(NetworkAcls networkAcls); } - /** The stage of the NamespaceResource definition allowing to specify subscriptionId. */ - interface WithSubscriptionId { + + /** + * The stage of the NamespaceResource definition allowing to specify pnsCredentials. + */ + interface WithPnsCredentials { /** - * Specifies the subscriptionId property: The Id of the Azure subscription associated with the namespace.. - * - * @param subscriptionId The Id of the Azure subscription associated with the namespace. + * Specifies the pnsCredentials property: Collection of Notification Hub or Notification Hub Namespace PNS + * credentials.. + * + * @param pnsCredentials Collection of Notification Hub or Notification Hub Namespace PNS credentials. * @return the next definition stage. */ - WithCreate withSubscriptionId(String subscriptionId); + WithCreate withPnsCredentials(PnsCredentials pnsCredentials); } - /** The stage of the NamespaceResource definition allowing to specify scaleUnit. */ + + /** + * The stage of the NamespaceResource definition allowing to specify scaleUnit. + */ interface WithScaleUnit { /** - * Specifies the scaleUnit property: ScaleUnit where the namespace gets created. - * - * @param scaleUnit ScaleUnit where the namespace gets created. + * Specifies the scaleUnit property: Gets or sets scaleUnit where the namespace gets created. + * + * @param scaleUnit Gets or sets scaleUnit where the namespace gets created. * @return the next definition stage. */ WithCreate withScaleUnit(String scaleUnit); } - /** The stage of the NamespaceResource definition allowing to specify enabled. */ - interface WithEnabled { - /** - * Specifies the enabled property: Whether or not the namespace is currently enabled.. - * - * @param enabled Whether or not the namespace is currently enabled. - * @return the next definition stage. - */ - WithCreate withEnabled(Boolean enabled); - } - /** The stage of the NamespaceResource definition allowing to specify critical. */ - interface WithCritical { - /** - * Specifies the critical property: Whether or not the namespace is set as Critical.. - * - * @param critical Whether or not the namespace is set as Critical. - * @return the next definition stage. - */ - WithCreate withCritical(Boolean critical); - } - /** The stage of the NamespaceResource definition allowing to specify dataCenter. */ + + /** + * The stage of the NamespaceResource definition allowing to specify dataCenter. + */ interface WithDataCenter { /** - * Specifies the dataCenter property: Data center for the namespace. - * - * @param dataCenter Data center for the namespace. + * Specifies the dataCenter property: Deprecated.. + * + * @param dataCenter Deprecated. * @return the next definition stage. */ WithCreate withDataCenter(String dataCenter); } - /** The stage of the NamespaceResource definition allowing to specify namespaceType. */ - interface WithNamespaceType { + + /** + * The stage of the NamespaceResource definition allowing to specify publicNetworkAccess. + */ + interface WithPublicNetworkAccess { /** - * Specifies the namespaceType property: The namespace type.. - * - * @param namespaceType The namespace type. + * Specifies the publicNetworkAccess property: Type of public network access.. + * + * @param publicNetworkAccess Type of public network access. * @return the next definition stage. */ - WithCreate withNamespaceType(NamespaceType namespaceType); + WithCreate withPublicNetworkAccess(PublicNetworkAccess publicNetworkAccess); } } + /** * Begins update for the NamespaceResource resource. - * + * * @return the stage of resource update. */ NamespaceResource.Update update(); - /** The template for NamespaceResource update. */ - interface Update extends UpdateStages.WithTags, UpdateStages.WithSku { + /** + * The template for NamespaceResource update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithSku, UpdateStages.WithProperties { /** * Executes the update request. - * + * * @return the updated resource. */ NamespaceResource apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ NamespaceResource apply(Context context); } - /** The NamespaceResource update stages. */ + + /** + * The NamespaceResource update stages. + */ interface UpdateStages { - /** The stage of the NamespaceResource update allowing to specify tags. */ + /** + * The stage of the NamespaceResource update allowing to specify tags. + */ interface WithTags { /** - * Specifies the tags property: Resource tags. - * - * @param tags Resource tags. + * Specifies the tags property: Dictionary of <string>. + * + * @param tags Dictionary of <string>. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the NamespaceResource update allowing to specify sku. */ + + /** + * The stage of the NamespaceResource update allowing to specify sku. + */ interface WithSku { /** - * Specifies the sku property: The sku of the created namespace. - * - * @param sku The sku of the created namespace. + * Specifies the sku property: The Sku description for a namespace. + * + * @param sku The Sku description for a namespace. * @return the next definition stage. */ Update withSku(Sku sku); } + + /** + * The stage of the NamespaceResource update allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Represents namespace properties.. + * + * @param properties Represents namespace properties. + * @return the next definition stage. + */ + Update withProperties(NamespaceProperties properties); + } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ NamespaceResource refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ NamespaceResource refresh(Context context); + + /** + * Lists the PNS credentials associated with a namespace. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. + */ + Response getPnsCredentialsWithResponse(Context context); + + /** + * Lists the PNS credentials associated with a namespace. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials. + */ + PnsCredentialsResource getPnsCredentials(); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceStatus.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceStatus.java new file mode 100644 index 0000000000000..58acaf2aa9178 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceStatus.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Namespace status. + */ +public final class NamespaceStatus extends ExpandableStringEnum { + /** + * Static value Created for NamespaceStatus. + */ + public static final NamespaceStatus CREATED = fromString("Created"); + + /** + * Static value Creating for NamespaceStatus. + */ + public static final NamespaceStatus CREATING = fromString("Creating"); + + /** + * Static value Suspended for NamespaceStatus. + */ + public static final NamespaceStatus SUSPENDED = fromString("Suspended"); + + /** + * Static value Deleting for NamespaceStatus. + */ + public static final NamespaceStatus DELETING = fromString("Deleting"); + + /** + * Creates a new instance of NamespaceStatus value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public NamespaceStatus() { + } + + /** + * Creates or finds a NamespaceStatus from its string representation. + * + * @param name a name to look for. + * @return the corresponding NamespaceStatus. + */ + @JsonCreator + public static NamespaceStatus fromString(String name) { + return fromString(name, NamespaceStatus.class); + } + + /** + * Gets known NamespaceStatus values. + * + * @return known NamespaceStatus values. + */ + public static Collection values() { + return values(NamespaceStatus.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceType.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceType.java index 608989fde35f5..8255c95a00eec 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceType.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NamespaceType.java @@ -4,44 +4,50 @@ package com.azure.resourcemanager.notificationhubs.models; +import com.azure.core.util.ExpandableStringEnum; import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonValue; +import java.util.Collection; -/** Defines values for NamespaceType. */ -public enum NamespaceType { - /** Enum value Messaging. */ - MESSAGING("Messaging"), - - /** Enum value NotificationHub. */ - NOTIFICATION_HUB("NotificationHub"); +/** + * Defines values for NamespaceType. + */ +public final class NamespaceType extends ExpandableStringEnum { + /** + * Static value Messaging for NamespaceType. + */ + public static final NamespaceType MESSAGING = fromString("Messaging"); - /** The actual serialized value for a NamespaceType instance. */ - private final String value; + /** + * Static value NotificationHub for NamespaceType. + */ + public static final NamespaceType NOTIFICATION_HUB = fromString("NotificationHub"); - NamespaceType(String value) { - this.value = value; + /** + * Creates a new instance of NamespaceType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public NamespaceType() { } /** - * Parses a serialized value to a NamespaceType instance. - * - * @param value the serialized value to parse. - * @return the parsed NamespaceType object, or null if unable to parse. + * Creates or finds a NamespaceType from its string representation. + * + * @param name a name to look for. + * @return the corresponding NamespaceType. */ @JsonCreator - public static NamespaceType fromString(String value) { - NamespaceType[] items = NamespaceType.values(); - for (NamespaceType item : items) { - if (item.toString().equalsIgnoreCase(value)) { - return item; - } - } - return null; + public static NamespaceType fromString(String name) { + return fromString(name, NamespaceType.class); } - @JsonValue - @Override - public String toString() { - return this.value; + /** + * Gets known NamespaceType values. + * + * @return known NamespaceType values. + */ + public static Collection values() { + return values(NamespaceType.class); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Namespaces.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Namespaces.java index 5a8e18fba0b9d..b791e1d655150 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Namespaces.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Namespaces.java @@ -7,377 +7,387 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; +import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; -/** Resource collection API of Namespaces. */ +/** + * Resource collection API of Namespaces. + */ public interface Namespaces { /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. + * + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response}. */ - CheckAvailabilityResult checkAvailability(CheckAvailabilityParameters parameters); + Response checkAvailabilityWithResponse(CheckAvailabilityParameters parameters, + Context context); /** * Checks the availability of the given service namespace across all Azure subscriptions. This is useful because the * domain name is created based on the service namespace name. - * - * @param parameters The namespace name. - * @param context The context to associate with this operation. + * + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a CheckAvailability resource. */ - Response checkAvailabilityWithResponse( - CheckAvailabilityParameters parameters, Context context); + CheckAvailabilityResult checkAvailability(CheckAvailabilityParameters parameters); /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource along with {@link Response}. */ - void deleteByResourceGroup(String resourceGroupName, String namespaceName); + Response getByResourceGroupWithResponse(String resourceGroupName, String namespaceName, + Context context); + + /** + * Returns the given namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource. + */ + NamespaceResource getByResourceGroup(String resourceGroupName, String namespaceName); /** * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. */ - void delete(String resourceGroupName, String namespaceName, Context context); + Response deleteByResourceGroupWithResponse(String resourceGroupName, String namespaceName, Context context); /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. */ - NamespaceResource getByResourceGroup(String resourceGroupName, String namespaceName); + void deleteByResourceGroup(String resourceGroupName, String namespaceName); /** - * Returns the description for the specified namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. + * Lists all the available namespaces within the subscription. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String namespaceName, Context context); + PagedIterable list(); /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. + * Lists all the available namespaces within the subscription. + * + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName); + PagedIterable list(String skipToken, Integer top, Context context); /** - * Deletes a namespace authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization Rule Name. - * @param context The context to associate with this operation. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context); + PagedIterable listByResourceGroup(String resourceGroupName); /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Lists the available namespaces within a resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param skipToken Skip token for subsequent requests. + * @param top Maximum number of results to return. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return the response of the List Namespace operation as paginated response with {@link PagedIterable}. */ - SharedAccessAuthorizationRuleResource getAuthorizationRule( - String resourceGroupName, String namespaceName, String authorizationRuleName); + PagedIterable listByResourceGroup(String resourceGroupName, String skipToken, Integer top, + Context context); /** - * Gets an authorization rule for a namespace by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName Authorization rule name. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return response for POST requests that return single SharedAccessAuthorizationRule along with {@link Response}. */ - Response getAuthorizationRuleWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context); + Response createOrUpdateAuthorizationRuleWithResponse( + String resourceGroupName, String namespaceName, String authorizationRuleName, + SharedAccessAuthorizationRuleResourceInner parameters, Context context); /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Creates an authorization rule for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return response for POST requests that return single SharedAccessAuthorizationRule. */ - PagedIterable listByResourceGroup(String resourceGroupName); + SharedAccessAuthorizationRuleResource createOrUpdateAuthorizationRule(String resourceGroupName, + String namespaceName, String authorizationRuleName, SharedAccessAuthorizationRuleResourceInner parameters); /** - * Lists the available namespaces within a resourceGroup. - * - * @param resourceGroupName The name of the resource group. If resourceGroupName value is null the method lists all - * the namespaces within subscription. + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return the {@link Response}. */ - PagedIterable listByResourceGroup(String resourceGroupName, Context context); + Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context); /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * + * Deletes a namespace authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. */ - PagedIterable list(); + void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String authorizationRuleName); /** - * Lists all the available namespaces within the subscription irrespective of the resourceGroups. - * + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List Namespace operation. + * @return an authorization rule for a namespace by name along with {@link Response}. */ - PagedIterable list(Context context); + Response getAuthorizationRuleWithResponse(String resourceGroupName, + String namespaceName, String authorizationRuleName, Context context); /** - * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets an authorization rule for a namespace by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return an authorization rule for a namespace by name. */ - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName); + SharedAccessAuthorizationRuleResource getAuthorizationRule(String resourceGroupName, String namespaceName, + String authorizationRuleName); /** * Gets the authorization rules for a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedIterable}. */ - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, Context context); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName); /** - * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * Gets the authorization rules for a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the authorization rules for a namespace as paginated response with {@link PagedIterable}. */ - ResourceListKeys listKeys(String resourceGroupName, String namespaceName, String authorizationRuleName); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, Context context); /** * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the namespace along with {@link Response}. */ - Response listKeysWithResponse( - String resourceGroupName, String namespaceName, String authorizationRuleName, Context context); + Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, Context context); /** - * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * Gets the Primary and Secondary ConnectionStrings to the namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return the Primary and Secondary ConnectionStrings to the namespace. */ - ResourceListKeys regenerateKeys( - String resourceGroupName, String namespaceName, String authorizationRuleName, PolicykeyResource parameters); + ResourceListKeys listKeys(String resourceGroupName, String namespaceName, String authorizationRuleName); /** * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param authorizationRuleName The connection string of the namespace for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ - Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context); + Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String authorizationRuleName, PolicyKeyResource parameters, Context context); /** - * Returns the description for the specified namespace. - * - * @param id the resource ID. + * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). */ - NamespaceResource getById(String id); + ResourceListKeys regenerateKeys(String resourceGroupName, String namespaceName, String authorizationRuleName, + PolicyKeyResource parameters); /** - * Returns the description for the specified namespace. - * - * @param id the resource ID. + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace resource. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. */ - Response getByIdWithResponse(String id, Context context); - - /** - * Gets an authorization rule for a namespace by name. - * - * @param id the resource ID. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. - */ - SharedAccessAuthorizationRuleResource getAuthorizationRuleById(String id); + Response getPnsCredentialsWithResponse(String resourceGroupName, String namespaceName, + Context context); /** - * Gets an authorization rule for a namespace by name. - * - * @param id the resource ID. - * @param context The context to associate with this operation. + * Lists the PNS credentials associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a namespace by name. + * @return description of a NotificationHub PNS Credentials. */ - Response getAuthorizationRuleByIdWithResponse(String id, Context context); + PnsCredentialsResource getPnsCredentials(String resourceGroupName, String namespaceName); /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * + * Returns the given namespace. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource along with {@link Response}. */ - void deleteById(String id); + NamespaceResource getById(String id); /** - * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. - * + * Returns the given namespace. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return notification Hubs Namespace Resource along with {@link Response}. */ - void deleteByIdWithResponse(String id, Context context); + Response getByIdWithResponse(String id, Context context); /** - * Deletes a namespace authorization rule. - * + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void deleteAuthorizationRuleById(String id); + void deleteById(String id); /** - * Deletes a namespace authorization rule. - * + * Deletes an existing namespace. This operation also removes all associated notificationHubs under the namespace. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the {@link Response}. */ - Response deleteAuthorizationRuleByIdWithResponse(String id, Context context); + Response deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new NamespaceResource resource. - * + * * @param name resource name. * @return the first stage of the new NamespaceResource definition. */ NamespaceResource.DefinitionStages.Blank define(String name); - - /** - * Begins definition for a new SharedAccessAuthorizationRuleResource resource. - * - * @param name resource name. - * @return the first stage of the new SharedAccessAuthorizationRuleResource definition. - */ - SharedAccessAuthorizationRuleResource.DefinitionStages.Blank defineAuthorizationRule(String name); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NetworkAcls.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NetworkAcls.java new file mode 100644 index 0000000000000..d4ba373f3df9e --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NetworkAcls.java @@ -0,0 +1,89 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * A collection of network authorization rules. + */ +@Fluent +public final class NetworkAcls { + /* + * List of IP rules. + */ + @JsonProperty(value = "ipRules") + private List ipRules; + + /* + * A default (public Internet) network authorization rule, which contains rights if no other network rule matches. + */ + @JsonProperty(value = "publicNetworkRule") + private PublicInternetAuthorizationRule publicNetworkRule; + + /** + * Creates an instance of NetworkAcls class. + */ + public NetworkAcls() { + } + + /** + * Get the ipRules property: List of IP rules. + * + * @return the ipRules value. + */ + public List ipRules() { + return this.ipRules; + } + + /** + * Set the ipRules property: List of IP rules. + * + * @param ipRules the ipRules value to set. + * @return the NetworkAcls object itself. + */ + public NetworkAcls withIpRules(List ipRules) { + this.ipRules = ipRules; + return this; + } + + /** + * Get the publicNetworkRule property: A default (public Internet) network authorization rule, which contains + * rights if no other network rule matches. + * + * @return the publicNetworkRule value. + */ + public PublicInternetAuthorizationRule publicNetworkRule() { + return this.publicNetworkRule; + } + + /** + * Set the publicNetworkRule property: A default (public Internet) network authorization rule, which contains + * rights if no other network rule matches. + * + * @param publicNetworkRule the publicNetworkRule value to set. + * @return the NetworkAcls object itself. + */ + public NetworkAcls withPublicNetworkRule(PublicInternetAuthorizationRule publicNetworkRule) { + this.publicNetworkRule = publicNetworkRule; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (ipRules() != null) { + ipRules().forEach(e -> e.validate()); + } + if (publicNetworkRule() != null) { + publicNetworkRule().validate(); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubCreateOrUpdateParameters.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubCreateOrUpdateParameters.java deleted file mode 100644 index b09303dbac6f5..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubCreateOrUpdateParameters.java +++ /dev/null @@ -1,303 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubProperties; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; -import java.util.Map; - -/** Parameters supplied to the CreateOrUpdate NotificationHub operation. */ -@Fluent -public final class NotificationHubCreateOrUpdateParameters extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NotificationHubCreateOrUpdateParameters.class); - - /* - * Properties of the NotificationHub. - */ - @JsonProperty(value = "properties", required = true) - private NotificationHubProperties innerProperties = new NotificationHubProperties(); - - /* - * The sku of the created namespace - */ - @JsonProperty(value = "sku") - private Sku sku; - - /** - * Get the innerProperties property: Properties of the NotificationHub. - * - * @return the innerProperties value. - */ - private NotificationHubProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the sku property: The sku of the created namespace. - * - * @return the sku value. - */ - public Sku sku() { - return this.sku; - } - - /** - * Set the sku property: The sku of the created namespace. - * - * @param sku the sku value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withSku(Sku sku) { - this.sku = sku; - return this; - } - - /** {@inheritDoc} */ - @Override - public NotificationHubCreateOrUpdateParameters withLocation(String location) { - super.withLocation(location); - return this; - } - - /** {@inheritDoc} */ - @Override - public NotificationHubCreateOrUpdateParameters withTags(Map tags) { - super.withTags(tags); - return this; - } - - /** - * Get the name property: The NotificationHub name. - * - * @return the name value. - */ - public String namePropertiesName() { - return this.innerProperties() == null ? null : this.innerProperties().name(); - } - - /** - * Set the name property: The NotificationHub name. - * - * @param name the name value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withNamePropertiesName(String name) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withName(name); - return this; - } - - /** - * Get the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * - * @return the registrationTtl value. - */ - public String registrationTtl() { - return this.innerProperties() == null ? null : this.innerProperties().registrationTtl(); - } - - /** - * Set the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * - * @param registrationTtl the registrationTtl value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withRegistrationTtl(String registrationTtl) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withRegistrationTtl(registrationTtl); - return this; - } - - /** - * Get the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @return the authorizationRules value. - */ - public List authorizationRules() { - return this.innerProperties() == null ? null : this.innerProperties().authorizationRules(); - } - - /** - * Set the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @param authorizationRules the authorizationRules value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withAuthorizationRules( - List authorizationRules) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withAuthorizationRules(authorizationRules); - return this; - } - - /** - * Get the apnsCredential property: The ApnsCredential of the created NotificationHub. - * - * @return the apnsCredential value. - */ - public ApnsCredential apnsCredential() { - return this.innerProperties() == null ? null : this.innerProperties().apnsCredential(); - } - - /** - * Set the apnsCredential property: The ApnsCredential of the created NotificationHub. - * - * @param apnsCredential the apnsCredential value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withApnsCredential(ApnsCredential apnsCredential) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withApnsCredential(apnsCredential); - return this; - } - - /** - * Get the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @return the wnsCredential value. - */ - public WnsCredential wnsCredential() { - return this.innerProperties() == null ? null : this.innerProperties().wnsCredential(); - } - - /** - * Set the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @param wnsCredential the wnsCredential value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withWnsCredential(WnsCredential wnsCredential) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withWnsCredential(wnsCredential); - return this; - } - - /** - * Get the gcmCredential property: The GcmCredential of the created NotificationHub. - * - * @return the gcmCredential value. - */ - public GcmCredential gcmCredential() { - return this.innerProperties() == null ? null : this.innerProperties().gcmCredential(); - } - - /** - * Set the gcmCredential property: The GcmCredential of the created NotificationHub. - * - * @param gcmCredential the gcmCredential value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withGcmCredential(GcmCredential gcmCredential) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withGcmCredential(gcmCredential); - return this; - } - - /** - * Get the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * - * @return the mpnsCredential value. - */ - public MpnsCredential mpnsCredential() { - return this.innerProperties() == null ? null : this.innerProperties().mpnsCredential(); - } - - /** - * Set the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * - * @param mpnsCredential the mpnsCredential value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withMpnsCredential(MpnsCredential mpnsCredential) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withMpnsCredential(mpnsCredential); - return this; - } - - /** - * Get the admCredential property: The AdmCredential of the created NotificationHub. - * - * @return the admCredential value. - */ - public AdmCredential admCredential() { - return this.innerProperties() == null ? null : this.innerProperties().admCredential(); - } - - /** - * Set the admCredential property: The AdmCredential of the created NotificationHub. - * - * @param admCredential the admCredential value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withAdmCredential(AdmCredential admCredential) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withAdmCredential(admCredential); - return this; - } - - /** - * Get the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @return the baiduCredential value. - */ - public BaiduCredential baiduCredential() { - return this.innerProperties() == null ? null : this.innerProperties().baiduCredential(); - } - - /** - * Set the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @param baiduCredential the baiduCredential value to set. - * @return the NotificationHubCreateOrUpdateParameters object itself. - */ - public NotificationHubCreateOrUpdateParameters withBaiduCredential(BaiduCredential baiduCredential) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withBaiduCredential(baiduCredential); - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property innerProperties in model NotificationHubCreateOrUpdateParameters")); - } else { - innerProperties().validate(); - } - if (sku() != null) { - sku().validate(); - } - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubListResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubListResult.java index 98f73934118c1..c7111b47d99f5 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubListResult.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubListResult.java @@ -4,76 +4,55 @@ package com.azure.resourcemanager.notificationhubs.models; -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.annotation.Immutable; import com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubResourceInner; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of the List NotificationHub operation. */ -@Fluent +/** + * The response of the List NotificationHub operation. + */ +@Immutable public final class NotificationHubListResult { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NotificationHubListResult.class); - /* - * Result of the List NotificationHub operation. + * Gets or sets result of the List AuthorizationRules operation. */ - @JsonProperty(value = "value") + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) private List value; /* - * Link to the next set of results. Not empty if Value contains incomplete - * list of NotificationHub + * Gets or sets link to the next set of results. */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; /** - * Get the value property: Result of the List NotificationHub operation. - * - * @return the value value. + * Creates an instance of NotificationHubListResult class. */ - public List value() { - return this.value; + public NotificationHubListResult() { } /** - * Set the value property: Result of the List NotificationHub operation. - * - * @param value the value value to set. - * @return the NotificationHubListResult object itself. + * Get the value property: Gets or sets result of the List AuthorizationRules operation. + * + * @return the value value. */ - public NotificationHubListResult withValue(List value) { - this.value = value; - return this; + public List value() { + return this.value; } /** - * Get the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of - * NotificationHub. - * + * Get the nextLink property: Gets or sets link to the next set of results. + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of - * NotificationHub. - * - * @param nextLink the nextLink value to set. - * @return the NotificationHubListResult object itself. - */ - public NotificationHubListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubPatchParameters.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubPatchParameters.java index 23d02c3cfca96..0f4c8953f4c82 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubPatchParameters.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubPatchParameters.java @@ -5,35 +5,46 @@ package com.azure.resourcemanager.notificationhubs.models; import com.azure.core.annotation.Fluent; -import com.azure.core.management.Resource; -import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubProperties; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; import java.util.Map; -/** Parameters supplied to the patch NotificationHub operation. */ +/** + * Patch parameter for NamespaceResource. + */ @Fluent -public final class NotificationHubPatchParameters extends Resource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(NotificationHubPatchParameters.class); - +public final class NotificationHubPatchParameters { /* - * Properties of the NotificationHub. + * NotificationHub properties. */ @JsonProperty(value = "properties") private NotificationHubProperties innerProperties; /* - * The sku of the created namespace + * The Sku description for a namespace */ @JsonProperty(value = "sku") private Sku sku; + /* + * Dictionary of + */ + @JsonProperty(value = "tags") + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map tags; + /** - * Get the innerProperties property: Properties of the NotificationHub. - * + * Creates an instance of NotificationHubPatchParameters class. + */ + public NotificationHubPatchParameters() { + } + + /** + * Get the innerProperties property: NotificationHub properties. + * * @return the innerProperties value. */ private NotificationHubProperties innerProperties() { @@ -41,8 +52,8 @@ private NotificationHubProperties innerProperties() { } /** - * Get the sku property: The sku of the created namespace. - * + * Get the sku property: The Sku description for a namespace. + * * @return the sku value. */ public Sku sku() { @@ -50,8 +61,8 @@ public Sku sku() { } /** - * Set the sku property: The sku of the created namespace. - * + * Set the sku property: The Sku description for a namespace. + * * @param sku the sku value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -60,36 +71,42 @@ public NotificationHubPatchParameters withSku(Sku sku) { return this; } - /** {@inheritDoc} */ - @Override - public NotificationHubPatchParameters withLocation(String location) { - super.withLocation(location); - return this; + /** + * Get the tags property: Dictionary of <string>. + * + * @return the tags value. + */ + public Map tags() { + return this.tags; } - /** {@inheritDoc} */ - @Override + /** + * Set the tags property: Dictionary of <string>. + * + * @param tags the tags value to set. + * @return the NotificationHubPatchParameters object itself. + */ public NotificationHubPatchParameters withTags(Map tags) { - super.withTags(tags); + this.tags = tags; return this; } /** - * Get the name property: The NotificationHub name. - * + * Get the name property: Gets or sets the NotificationHub name. + * * @return the name value. */ - public String namePropertiesName() { + public String name() { return this.innerProperties() == null ? null : this.innerProperties().name(); } /** - * Set the name property: The NotificationHub name. - * + * Set the name property: Gets or sets the NotificationHub name. + * * @param name the name value to set. * @return the NotificationHubPatchParameters object itself. */ - public NotificationHubPatchParameters withNamePropertiesName(String name) { + public NotificationHubPatchParameters withName(String name) { if (this.innerProperties() == null) { this.innerProperties = new NotificationHubProperties(); } @@ -98,8 +115,8 @@ public NotificationHubPatchParameters withNamePropertiesName(String name) { } /** - * Get the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Get the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @return the registrationTtl value. */ public String registrationTtl() { @@ -107,8 +124,8 @@ public String registrationTtl() { } /** - * Set the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Set the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @param registrationTtl the registrationTtl value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -121,8 +138,8 @@ public NotificationHubPatchParameters withRegistrationTtl(String registrationTtl } /** - * Get the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * + * Get the authorizationRules property: Gets or sets the AuthorizationRules of the created NotificationHub. + * * @return the authorizationRules value. */ public List authorizationRules() { @@ -130,23 +147,8 @@ public List authorizationRules() { } /** - * Set the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @param authorizationRules the authorizationRules value to set. - * @return the NotificationHubPatchParameters object itself. - */ - public NotificationHubPatchParameters withAuthorizationRules( - List authorizationRules) { - if (this.innerProperties() == null) { - this.innerProperties = new NotificationHubProperties(); - } - this.innerProperties().withAuthorizationRules(authorizationRules); - return this; - } - - /** - * Get the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Get the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @return the apnsCredential value. */ public ApnsCredential apnsCredential() { @@ -154,8 +156,8 @@ public ApnsCredential apnsCredential() { } /** - * Set the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Set the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @param apnsCredential the apnsCredential value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -168,8 +170,8 @@ public NotificationHubPatchParameters withApnsCredential(ApnsCredential apnsCred } /** - * Get the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Get the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @return the wnsCredential value. */ public WnsCredential wnsCredential() { @@ -177,8 +179,8 @@ public WnsCredential wnsCredential() { } /** - * Set the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Set the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @param wnsCredential the wnsCredential value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -191,8 +193,8 @@ public NotificationHubPatchParameters withWnsCredential(WnsCredential wnsCredent } /** - * Get the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Get the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @return the gcmCredential value. */ public GcmCredential gcmCredential() { @@ -200,8 +202,8 @@ public GcmCredential gcmCredential() { } /** - * Set the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Set the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @param gcmCredential the gcmCredential value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -214,8 +216,8 @@ public NotificationHubPatchParameters withGcmCredential(GcmCredential gcmCredent } /** - * Get the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Get the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @return the mpnsCredential value. */ public MpnsCredential mpnsCredential() { @@ -223,8 +225,8 @@ public MpnsCredential mpnsCredential() { } /** - * Set the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Set the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @param mpnsCredential the mpnsCredential value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -237,8 +239,8 @@ public NotificationHubPatchParameters withMpnsCredential(MpnsCredential mpnsCred } /** - * Get the admCredential property: The AdmCredential of the created NotificationHub. - * + * Get the admCredential property: Description of a NotificationHub AdmCredential. + * * @return the admCredential value. */ public AdmCredential admCredential() { @@ -246,8 +248,8 @@ public AdmCredential admCredential() { } /** - * Set the admCredential property: The AdmCredential of the created NotificationHub. - * + * Set the admCredential property: Description of a NotificationHub AdmCredential. + * * @param admCredential the admCredential value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -260,8 +262,8 @@ public NotificationHubPatchParameters withAdmCredential(AdmCredential admCredent } /** - * Get the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Get the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @return the baiduCredential value. */ public BaiduCredential baiduCredential() { @@ -269,8 +271,8 @@ public BaiduCredential baiduCredential() { } /** - * Set the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Set the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @param baiduCredential the baiduCredential value to set. * @return the NotificationHubPatchParameters object itself. */ @@ -282,9 +284,87 @@ public NotificationHubPatchParameters withBaiduCredential(BaiduCredential baiduC return this; } + /** + * Get the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + public BrowserCredential browserCredential() { + return this.innerProperties() == null ? null : this.innerProperties().browserCredential(); + } + + /** + * Set the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @param browserCredential the browserCredential value to set. + * @return the NotificationHubPatchParameters object itself. + */ + public NotificationHubPatchParameters withBrowserCredential(BrowserCredential browserCredential) { + if (this.innerProperties() == null) { + this.innerProperties = new NotificationHubProperties(); + } + this.innerProperties().withBrowserCredential(browserCredential); + return this; + } + + /** + * Get the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. + */ + public XiaomiCredential xiaomiCredential() { + return this.innerProperties() == null ? null : this.innerProperties().xiaomiCredential(); + } + + /** + * Set the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @param xiaomiCredential the xiaomiCredential value to set. + * @return the NotificationHubPatchParameters object itself. + */ + public NotificationHubPatchParameters withXiaomiCredential(XiaomiCredential xiaomiCredential) { + if (this.innerProperties() == null) { + this.innerProperties = new NotificationHubProperties(); + } + this.innerProperties().withXiaomiCredential(xiaomiCredential); + return this; + } + + /** + * Get the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + public FcmV1Credential fcmV1Credential() { + return this.innerProperties() == null ? null : this.innerProperties().fcmV1Credential(); + } + + /** + * Set the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @param fcmV1Credential the fcmV1Credential value to set. + * @return the NotificationHubPatchParameters object itself. + */ + public NotificationHubPatchParameters withFcmV1Credential(FcmV1Credential fcmV1Credential) { + if (this.innerProperties() == null) { + this.innerProperties = new NotificationHubProperties(); + } + this.innerProperties().withFcmV1Credential(fcmV1Credential); + return this; + } + + /** + * Get the dailyMaxActiveDevices property: The dailyMaxActiveDevices property. + * + * @return the dailyMaxActiveDevices value. + */ + public Long dailyMaxActiveDevices() { + return this.innerProperties() == null ? null : this.innerProperties().dailyMaxActiveDevices(); + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubResource.java index 4b0f0bfb36730..2fea4b6cab16d 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubResource.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubResource.java @@ -6,157 +6,209 @@ import com.azure.core.http.rest.Response; import com.azure.core.management.Region; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubResourceInner; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; import java.util.List; import java.util.Map; -/** An immutable client-side representation of NotificationHubResource. */ +/** + * An immutable client-side representation of NotificationHubResource. + */ public interface NotificationHubResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** - * Gets the sku property: The sku of the created namespace. - * + * Gets the sku property: The Sku description for a namespace. + * * @return the sku value. */ Sku sku(); /** - * Gets the namePropertiesName property: The NotificationHub name. - * + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the namePropertiesName property: Gets or sets the NotificationHub name. + * * @return the namePropertiesName value. */ String namePropertiesName(); /** - * Gets the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * + * Gets the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * * @return the registrationTtl value. */ String registrationTtl(); /** - * Gets the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * + * Gets the authorizationRules property: Gets or sets the AuthorizationRules of the created NotificationHub. + * * @return the authorizationRules value. */ List authorizationRules(); /** - * Gets the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Gets the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @return the apnsCredential value. */ ApnsCredential apnsCredential(); /** - * Gets the wnsCredential property: The WnsCredential of the created NotificationHub. - * + * Gets the wnsCredential property: Description of a NotificationHub WnsCredential. + * * @return the wnsCredential value. */ WnsCredential wnsCredential(); /** - * Gets the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Gets the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @return the gcmCredential value. */ GcmCredential gcmCredential(); /** - * Gets the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Gets the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @return the mpnsCredential value. */ MpnsCredential mpnsCredential(); /** - * Gets the admCredential property: The AdmCredential of the created NotificationHub. - * + * Gets the admCredential property: Description of a NotificationHub AdmCredential. + * * @return the admCredential value. */ AdmCredential admCredential(); /** - * Gets the baiduCredential property: The BaiduCredential of the created NotificationHub. - * + * Gets the baiduCredential property: Description of a NotificationHub BaiduCredential. + * * @return the baiduCredential value. */ BaiduCredential baiduCredential(); + /** + * Gets the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + BrowserCredential browserCredential(); + + /** + * Gets the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. + */ + XiaomiCredential xiaomiCredential(); + + /** + * Gets the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + FcmV1Credential fcmV1Credential(); + + /** + * Gets the dailyMaxActiveDevices property: The dailyMaxActiveDevices property. + * + * @return the dailyMaxActiveDevices value. + */ + Long dailyMaxActiveDevices(); + /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubResourceInner object. - * + * * @return the inner object. */ NotificationHubResourceInner innerModel(); - /** The entirety of the NotificationHubResource definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithParentResource, - DefinitionStages.WithCreate { + /** + * The entirety of the NotificationHubResource definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The NotificationHubResource definition stages. */ + + /** + * The NotificationHubResource definition stages. + */ interface DefinitionStages { - /** The first stage of the NotificationHubResource definition. */ + /** + * The first stage of the NotificationHubResource definition. + */ interface Blank extends WithLocation { } - /** The stage of the NotificationHubResource definition allowing to specify location. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -164,366 +216,466 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithParentResource withRegion(String location); } - /** The stage of the NotificationHubResource definition allowing to specify parent resource. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, namespaceName. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @return the next definition stage. */ WithCreate withExistingNamespace(String resourceGroupName, String namespaceName); } + /** * The stage of the NotificationHubResource definition which contains all the minimum required properties for * the resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithSku, - DefinitionStages.WithNamePropertiesName, - DefinitionStages.WithRegistrationTtl, - DefinitionStages.WithAuthorizationRules, - DefinitionStages.WithApnsCredential, - DefinitionStages.WithWnsCredential, - DefinitionStages.WithGcmCredential, - DefinitionStages.WithMpnsCredential, - DefinitionStages.WithAdmCredential, - DefinitionStages.WithBaiduCredential { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithSku, + DefinitionStages.WithNamePropertiesName, DefinitionStages.WithRegistrationTtl, + DefinitionStages.WithApnsCredential, DefinitionStages.WithWnsCredential, DefinitionStages.WithGcmCredential, + DefinitionStages.WithMpnsCredential, DefinitionStages.WithAdmCredential, + DefinitionStages.WithBaiduCredential, DefinitionStages.WithBrowserCredential, + DefinitionStages.WithXiaomiCredential, DefinitionStages.WithFcmV1Credential { /** * Executes the create request. - * + * * @return the created resource. */ NotificationHubResource create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ NotificationHubResource create(Context context); } - /** The stage of the NotificationHubResource definition allowing to specify tags. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the NotificationHubResource definition allowing to specify sku. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify sku. + */ interface WithSku { /** - * Specifies the sku property: The sku of the created namespace. - * - * @param sku The sku of the created namespace. + * Specifies the sku property: The Sku description for a namespace. + * + * @param sku The Sku description for a namespace. * @return the next definition stage. */ WithCreate withSku(Sku sku); } - /** The stage of the NotificationHubResource definition allowing to specify namePropertiesName. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify namePropertiesName. + */ interface WithNamePropertiesName { /** - * Specifies the namePropertiesName property: The NotificationHub name.. - * - * @param namePropertiesName The NotificationHub name. + * Specifies the namePropertiesName property: Gets or sets the NotificationHub name.. + * + * @param namePropertiesName Gets or sets the NotificationHub name. * @return the next definition stage. */ WithCreate withNamePropertiesName(String namePropertiesName); } - /** The stage of the NotificationHubResource definition allowing to specify registrationTtl. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify registrationTtl. + */ interface WithRegistrationTtl { /** - * Specifies the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * - * @param registrationTtl The RegistrationTtl of the created NotificationHub. + * Specifies the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * + * @param registrationTtl Gets or sets the RegistrationTtl of the created NotificationHub. * @return the next definition stage. */ WithCreate withRegistrationTtl(String registrationTtl); } - /** The stage of the NotificationHubResource definition allowing to specify authorizationRules. */ - interface WithAuthorizationRules { - /** - * Specifies the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @param authorizationRules The AuthorizationRules of the created NotificationHub. - * @return the next definition stage. - */ - WithCreate withAuthorizationRules(List authorizationRules); - } - /** The stage of the NotificationHubResource definition allowing to specify apnsCredential. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify apnsCredential. + */ interface WithApnsCredential { /** - * Specifies the apnsCredential property: The ApnsCredential of the created NotificationHub. - * - * @param apnsCredential The ApnsCredential of the created NotificationHub. + * Specifies the apnsCredential property: Description of a NotificationHub ApnsCredential.. + * + * @param apnsCredential Description of a NotificationHub ApnsCredential. * @return the next definition stage. */ WithCreate withApnsCredential(ApnsCredential apnsCredential); } - /** The stage of the NotificationHubResource definition allowing to specify wnsCredential. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify wnsCredential. + */ interface WithWnsCredential { /** - * Specifies the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @param wnsCredential The WnsCredential of the created NotificationHub. + * Specifies the wnsCredential property: Description of a NotificationHub WnsCredential.. + * + * @param wnsCredential Description of a NotificationHub WnsCredential. * @return the next definition stage. */ WithCreate withWnsCredential(WnsCredential wnsCredential); } - /** The stage of the NotificationHubResource definition allowing to specify gcmCredential. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify gcmCredential. + */ interface WithGcmCredential { /** - * Specifies the gcmCredential property: The GcmCredential of the created NotificationHub. - * - * @param gcmCredential The GcmCredential of the created NotificationHub. + * Specifies the gcmCredential property: Description of a NotificationHub GcmCredential.. + * + * @param gcmCredential Description of a NotificationHub GcmCredential. * @return the next definition stage. */ WithCreate withGcmCredential(GcmCredential gcmCredential); } - /** The stage of the NotificationHubResource definition allowing to specify mpnsCredential. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify mpnsCredential. + */ interface WithMpnsCredential { /** - * Specifies the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * - * @param mpnsCredential The MpnsCredential of the created NotificationHub. + * Specifies the mpnsCredential property: Description of a NotificationHub MpnsCredential.. + * + * @param mpnsCredential Description of a NotificationHub MpnsCredential. * @return the next definition stage. */ WithCreate withMpnsCredential(MpnsCredential mpnsCredential); } - /** The stage of the NotificationHubResource definition allowing to specify admCredential. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify admCredential. + */ interface WithAdmCredential { /** - * Specifies the admCredential property: The AdmCredential of the created NotificationHub. - * - * @param admCredential The AdmCredential of the created NotificationHub. + * Specifies the admCredential property: Description of a NotificationHub AdmCredential.. + * + * @param admCredential Description of a NotificationHub AdmCredential. * @return the next definition stage. */ WithCreate withAdmCredential(AdmCredential admCredential); } - /** The stage of the NotificationHubResource definition allowing to specify baiduCredential. */ + + /** + * The stage of the NotificationHubResource definition allowing to specify baiduCredential. + */ interface WithBaiduCredential { /** - * Specifies the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @param baiduCredential The BaiduCredential of the created NotificationHub. + * Specifies the baiduCredential property: Description of a NotificationHub BaiduCredential.. + * + * @param baiduCredential Description of a NotificationHub BaiduCredential. * @return the next definition stage. */ WithCreate withBaiduCredential(BaiduCredential baiduCredential); } + + /** + * The stage of the NotificationHubResource definition allowing to specify browserCredential. + */ + interface WithBrowserCredential { + /** + * Specifies the browserCredential property: Description of a NotificationHub BrowserCredential.. + * + * @param browserCredential Description of a NotificationHub BrowserCredential. + * @return the next definition stage. + */ + WithCreate withBrowserCredential(BrowserCredential browserCredential); + } + + /** + * The stage of the NotificationHubResource definition allowing to specify xiaomiCredential. + */ + interface WithXiaomiCredential { + /** + * Specifies the xiaomiCredential property: Description of a NotificationHub XiaomiCredential.. + * + * @param xiaomiCredential Description of a NotificationHub XiaomiCredential. + * @return the next definition stage. + */ + WithCreate withXiaomiCredential(XiaomiCredential xiaomiCredential); + } + + /** + * The stage of the NotificationHubResource definition allowing to specify fcmV1Credential. + */ + interface WithFcmV1Credential { + /** + * Specifies the fcmV1Credential property: Description of a NotificationHub FcmV1Credential.. + * + * @param fcmV1Credential Description of a NotificationHub FcmV1Credential. + * @return the next definition stage. + */ + WithCreate withFcmV1Credential(FcmV1Credential fcmV1Credential); + } } + /** * Begins update for the NotificationHubResource resource. - * + * * @return the stage of resource update. */ NotificationHubResource.Update update(); - /** The template for NotificationHubResource update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithSku, - UpdateStages.WithNamePropertiesName, - UpdateStages.WithRegistrationTtl, - UpdateStages.WithAuthorizationRules, - UpdateStages.WithApnsCredential, - UpdateStages.WithWnsCredential, - UpdateStages.WithGcmCredential, - UpdateStages.WithMpnsCredential, - UpdateStages.WithAdmCredential, - UpdateStages.WithBaiduCredential { + /** + * The template for NotificationHubResource update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithSku, UpdateStages.WithRegistrationTtl, + UpdateStages.WithApnsCredential, UpdateStages.WithWnsCredential, UpdateStages.WithGcmCredential, + UpdateStages.WithMpnsCredential, UpdateStages.WithAdmCredential, UpdateStages.WithBaiduCredential, + UpdateStages.WithBrowserCredential, UpdateStages.WithXiaomiCredential, UpdateStages.WithFcmV1Credential { /** * Executes the update request. - * + * * @return the updated resource. */ NotificationHubResource apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ NotificationHubResource apply(Context context); } - /** The NotificationHubResource update stages. */ + + /** + * The NotificationHubResource update stages. + */ interface UpdateStages { - /** The stage of the NotificationHubResource update allowing to specify tags. */ + /** + * The stage of the NotificationHubResource update allowing to specify tags. + */ interface WithTags { /** - * Specifies the tags property: Resource tags.. - * - * @param tags Resource tags. + * Specifies the tags property: Dictionary of <string>. + * + * @param tags Dictionary of <string>. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the NotificationHubResource update allowing to specify sku. */ + + /** + * The stage of the NotificationHubResource update allowing to specify sku. + */ interface WithSku { /** - * Specifies the sku property: The sku of the created namespace. - * - * @param sku The sku of the created namespace. + * Specifies the sku property: The Sku description for a namespace. + * + * @param sku The Sku description for a namespace. * @return the next definition stage. */ Update withSku(Sku sku); } - /** The stage of the NotificationHubResource update allowing to specify namePropertiesName. */ - interface WithNamePropertiesName { - /** - * Specifies the namePropertiesName property: The NotificationHub name.. - * - * @param namePropertiesName The NotificationHub name. - * @return the next definition stage. - */ - Update withNamePropertiesName(String namePropertiesName); - } - /** The stage of the NotificationHubResource update allowing to specify registrationTtl. */ + + /** + * The stage of the NotificationHubResource update allowing to specify registrationTtl. + */ interface WithRegistrationTtl { /** - * Specifies the registrationTtl property: The RegistrationTtl of the created NotificationHub. - * - * @param registrationTtl The RegistrationTtl of the created NotificationHub. + * Specifies the registrationTtl property: Gets or sets the RegistrationTtl of the created NotificationHub. + * + * @param registrationTtl Gets or sets the RegistrationTtl of the created NotificationHub. * @return the next definition stage. */ Update withRegistrationTtl(String registrationTtl); } - /** The stage of the NotificationHubResource update allowing to specify authorizationRules. */ - interface WithAuthorizationRules { - /** - * Specifies the authorizationRules property: The AuthorizationRules of the created NotificationHub. - * - * @param authorizationRules The AuthorizationRules of the created NotificationHub. - * @return the next definition stage. - */ - Update withAuthorizationRules(List authorizationRules); - } - /** The stage of the NotificationHubResource update allowing to specify apnsCredential. */ + + /** + * The stage of the NotificationHubResource update allowing to specify apnsCredential. + */ interface WithApnsCredential { /** - * Specifies the apnsCredential property: The ApnsCredential of the created NotificationHub. - * - * @param apnsCredential The ApnsCredential of the created NotificationHub. + * Specifies the apnsCredential property: Description of a NotificationHub ApnsCredential.. + * + * @param apnsCredential Description of a NotificationHub ApnsCredential. * @return the next definition stage. */ Update withApnsCredential(ApnsCredential apnsCredential); } - /** The stage of the NotificationHubResource update allowing to specify wnsCredential. */ + + /** + * The stage of the NotificationHubResource update allowing to specify wnsCredential. + */ interface WithWnsCredential { /** - * Specifies the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @param wnsCredential The WnsCredential of the created NotificationHub. + * Specifies the wnsCredential property: Description of a NotificationHub WnsCredential.. + * + * @param wnsCredential Description of a NotificationHub WnsCredential. * @return the next definition stage. */ Update withWnsCredential(WnsCredential wnsCredential); } - /** The stage of the NotificationHubResource update allowing to specify gcmCredential. */ + + /** + * The stage of the NotificationHubResource update allowing to specify gcmCredential. + */ interface WithGcmCredential { /** - * Specifies the gcmCredential property: The GcmCredential of the created NotificationHub. - * - * @param gcmCredential The GcmCredential of the created NotificationHub. + * Specifies the gcmCredential property: Description of a NotificationHub GcmCredential.. + * + * @param gcmCredential Description of a NotificationHub GcmCredential. * @return the next definition stage. */ Update withGcmCredential(GcmCredential gcmCredential); } - /** The stage of the NotificationHubResource update allowing to specify mpnsCredential. */ + + /** + * The stage of the NotificationHubResource update allowing to specify mpnsCredential. + */ interface WithMpnsCredential { /** - * Specifies the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * - * @param mpnsCredential The MpnsCredential of the created NotificationHub. + * Specifies the mpnsCredential property: Description of a NotificationHub MpnsCredential.. + * + * @param mpnsCredential Description of a NotificationHub MpnsCredential. * @return the next definition stage. */ Update withMpnsCredential(MpnsCredential mpnsCredential); } - /** The stage of the NotificationHubResource update allowing to specify admCredential. */ + + /** + * The stage of the NotificationHubResource update allowing to specify admCredential. + */ interface WithAdmCredential { /** - * Specifies the admCredential property: The AdmCredential of the created NotificationHub. - * - * @param admCredential The AdmCredential of the created NotificationHub. + * Specifies the admCredential property: Description of a NotificationHub AdmCredential.. + * + * @param admCredential Description of a NotificationHub AdmCredential. * @return the next definition stage. */ Update withAdmCredential(AdmCredential admCredential); } - /** The stage of the NotificationHubResource update allowing to specify baiduCredential. */ + + /** + * The stage of the NotificationHubResource update allowing to specify baiduCredential. + */ interface WithBaiduCredential { /** - * Specifies the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @param baiduCredential The BaiduCredential of the created NotificationHub. + * Specifies the baiduCredential property: Description of a NotificationHub BaiduCredential.. + * + * @param baiduCredential Description of a NotificationHub BaiduCredential. * @return the next definition stage. */ Update withBaiduCredential(BaiduCredential baiduCredential); } + + /** + * The stage of the NotificationHubResource update allowing to specify browserCredential. + */ + interface WithBrowserCredential { + /** + * Specifies the browserCredential property: Description of a NotificationHub BrowserCredential.. + * + * @param browserCredential Description of a NotificationHub BrowserCredential. + * @return the next definition stage. + */ + Update withBrowserCredential(BrowserCredential browserCredential); + } + + /** + * The stage of the NotificationHubResource update allowing to specify xiaomiCredential. + */ + interface WithXiaomiCredential { + /** + * Specifies the xiaomiCredential property: Description of a NotificationHub XiaomiCredential.. + * + * @param xiaomiCredential Description of a NotificationHub XiaomiCredential. + * @return the next definition stage. + */ + Update withXiaomiCredential(XiaomiCredential xiaomiCredential); + } + + /** + * The stage of the NotificationHubResource update allowing to specify fcmV1Credential. + */ + interface WithFcmV1Credential { + /** + * Specifies the fcmV1Credential property: Description of a NotificationHub FcmV1Credential.. + * + * @param fcmV1Credential Description of a NotificationHub FcmV1Credential. + * @return the next definition stage. + */ + Update withFcmV1Credential(FcmV1Credential fcmV1Credential); + } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ NotificationHubResource refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ NotificationHubResource refresh(Context context); /** - * test send a push notification. - * + * Test send a push notification. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return description of a NotificationHub Resource along with {@link Response}. */ - DebugSendResponse debugSend(); + Response debugSendWithResponse(Context context); /** - * test send a push notification. - * - * @param parameters Debug send parameters. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. + * Test send a push notification. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a NotificationHub Resource. */ - Response debugSendWithResponse(Object parameters, Context context); + DebugSendResponse debugSend(); /** - * Lists the PNS Credentials associated with a notification hub . - * + * Lists the PNS Credentials associated with a notification hub. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. */ - PnsCredentialsResource getPnsCredentials(); + Response getPnsCredentialsWithResponse(Context context); /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. + * Lists the PNS Credentials associated with a notification hub. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a NotificationHub PNS Credentials. */ - Response getPnsCredentialsWithResponse(Context context); + PnsCredentialsResource getPnsCredentials(); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubs.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubs.java index d110d611d45b0..398b087db2141 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubs.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/NotificationHubs.java @@ -8,442 +8,437 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of NotificationHubs. */ +/** + * Resource collection API of NotificationHubs. + */ public interface NotificationHubs { /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a CheckAvailability resource. + * @return description of a CheckAvailability resource along with {@link Response}. */ - CheckAvailabilityResult checkNotificationHubAvailability( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters); + Response checkNotificationHubAvailabilityWithResponse(String resourceGroupName, + String namespaceName, CheckAvailabilityParameters parameters, Context context); /** * Checks the availability of the given notificationHub in a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param parameters The notificationHub name. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return description of a CheckAvailability resource. */ - Response checkNotificationHubAvailabilityWithResponse( - String resourceGroupName, String namespaceName, CheckAvailabilityParameters parameters, Context context); + CheckAvailabilityResult checkNotificationHubAvailability(String resourceGroupName, String namespaceName, + CheckAvailabilityParameters parameters); /** - * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the notification hub along with {@link Response}. */ - void delete(String resourceGroupName, String namespaceName, String notificationHubName); + Response getWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context); + + /** + * Gets the notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the notification hub. + */ + NotificationHubResource get(String resourceGroupName, String namespaceName, String notificationHubName); /** * Deletes a notification hub associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + Response deleteWithResponse(String resourceGroupName, String namespaceName, String notificationHubName, + Context context); /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notification hub associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. */ - NotificationHubResource get(String resourceGroupName, String namespaceName, String notificationHubName); + void delete(String resourceGroupName, String namespaceName, String notificationHubName); /** * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param context The context to associate with this operation. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedIterable}. */ - Response getWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + PagedIterable list(String resourceGroupName, String namespaceName); /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Lists the notification hubs associated with a namespace. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param skipToken Continuation token. + * @param top Page size. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the response of the List NotificationHub operation as paginated response with {@link PagedIterable}. */ - DebugSendResponse debugSend(String resourceGroupName, String namespaceName, String notificationHubName); + PagedIterable list(String resourceGroupName, String namespaceName, String skipToken, + Integer top, Context context); /** - * test send a push notification. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param parameters Debug send parameters. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return description of a NotificationHub Resource along with {@link Response}. */ - Response debugSendWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Object parameters, Context context); + Response debugSendWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context); /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. + * Test send a push notification. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. + * @return description of a NotificationHub Resource. */ - SharedAccessAuthorizationRuleResource createOrUpdateAuthorizationRule( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters); + DebugSendResponse debugSend(String resourceGroupName, String namespaceName, String notificationHubName); /** - * Creates/Updates an authorization rule for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Deletes a notificationHub authorization rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. - * @param parameters The shared access authorization rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a Namespace AuthorizationRules. - */ - Response createOrUpdateAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - SharedAccessAuthorizationRuleCreateOrUpdateParameters parameters, - Context context); + * @return the {@link Response}. + */ + Response deleteAuthorizationRuleWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context); /** * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void deleteAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName); + void deleteAuthorizationRule(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName); /** - * Deletes a notificationHub authorization rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return an authorization rule for a NotificationHub by name along with {@link Response}. */ - Response deleteAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context); + Response getAuthorizationRuleWithResponse(String resourceGroupName, + String namespaceName, String notificationHubName, String authorizationRuleName, Context context); /** * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an authorization rule for a NotificationHub by name. */ - SharedAccessAuthorizationRuleResource getAuthorizationRule( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName); + SharedAccessAuthorizationRuleResource getAuthorizationRule(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName); /** - * Gets an authorization rule for a NotificationHub by name. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName authorization rule name. - * @param context The context to associate with this operation. + * Gets the authorization rules for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an authorization rule for a NotificationHub by name. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedIterable}. */ - Response getAuthorizationRuleWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName); /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets the authorization rules for a NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return the authorization rules for a NotificationHub as paginated response with {@link PagedIterable}. */ - PagedIterable list(String resourceGroupName, String namespaceName); + PagedIterable listAuthorizationRules(String resourceGroupName, + String namespaceName, String notificationHubName, Context context); /** - * Lists the notification hubs associated with a namespace. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of the List NotificationHub operation. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub along with {@link Response}. */ - PagedIterable list(String resourceGroupName, String namespaceName, Context context); + Response listKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, Context context); /** - * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub. */ - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName); + ResourceListKeys listKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName); /** - * Gets the authorization rules for a NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the authorization rules for a NotificationHub. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ - PagedIterable listAuthorizationRules( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + Response regenerateKeysWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, String authorizationRuleName, PolicyKeyResource parameters, Context context); /** - * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @param authorizationRuleName Authorization Rule Name. + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). */ - ResourceListKeys listKeys( - String resourceGroupName, String namespaceName, String notificationHubName, String authorizationRuleName); + ResourceListKeys regenerateKeys(String resourceGroupName, String namespaceName, String notificationHubName, + String authorizationRuleName, PolicyKeyResource parameters); /** - * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the NotificationHub. + * @return description of a NotificationHub PNS Credentials along with {@link Response}. */ - Response listKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - Context context); + Response getPnsCredentialsWithResponse(String resourceGroupName, String namespaceName, + String notificationHubName, Context context); /** - * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * Lists the PNS Credentials associated with a notification hub. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return description of a NotificationHub PNS Credentials. + */ + PnsCredentialsResource getPnsCredentials(String resourceGroupName, String namespaceName, + String notificationHubName); + + /** + * Gets the notification hub. + * + * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return the notification hub along with {@link Response}. */ - ResourceListKeys regenerateKeys( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters); + NotificationHubResource getById(String id); /** - * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. - * @param authorizationRuleName The connection string of the NotificationHub for the specified authorizationRule. - * @param parameters Parameters supplied to regenerate the NotificationHub Authorization Rule Key. + * Gets the notification hub. + * + * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. - */ - Response regenerateKeysWithResponse( - String resourceGroupName, - String namespaceName, - String notificationHubName, - String authorizationRuleName, - PolicykeyResource parameters, - Context context); + * @return the notification hub along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return an authorization rule for a NotificationHub by name along with {@link Response}. */ - PnsCredentialsResource getPnsCredentials( - String resourceGroupName, String namespaceName, String notificationHubName); + SharedAccessAuthorizationRuleResource getAuthorizationRuleById(String id); /** - * Lists the PNS Credentials associated with a notification hub . - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. - * @param notificationHubName The notification hub name. + * Gets an authorization rule for a NotificationHub by name. + * + * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub PNS Credentials. + * @return an authorization rule for a NotificationHub by name along with {@link Response}. */ - Response getPnsCredentialsWithResponse( - String resourceGroupName, String namespaceName, String notificationHubName, Context context); + Response getAuthorizationRuleByIdWithResponse(String id, Context context); /** - * Lists the notification hubs associated with a namespace. - * + * Deletes a notification hub associated with a namespace. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. */ - NotificationHubResource getById(String id); + void deleteById(String id); /** - * Lists the notification hubs associated with a namespace. - * + * Deletes a notification hub associated with a namespace. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return description of a NotificationHub Resource. + * @return the {@link Response}. */ - Response getByIdWithResponse(String id, Context context); + Response deleteByIdWithResponse(String id, Context context); /** - * Deletes a notification hub associated with a namespace. - * + * Deletes a notificationHub authorization rule. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void deleteById(String id); + void deleteAuthorizationRuleById(String id); /** - * Deletes a notification hub associated with a namespace. - * + * Deletes a notificationHub authorization rule. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response. + * @return the {@link Response}. */ - Response deleteByIdWithResponse(String id, Context context); + Response deleteAuthorizationRuleByIdWithResponse(String id, Context context); /** * Begins definition for a new NotificationHubResource resource. - * + * * @param name resource name. * @return the first stage of the new NotificationHubResource definition. */ NotificationHubResource.DefinitionStages.Blank define(String name); + + /** + * Begins definition for a new SharedAccessAuthorizationRuleResource resource. + * + * @param name resource name. + * @return the first stage of the new SharedAccessAuthorizationRuleResource definition. + */ + SharedAccessAuthorizationRuleResource.DefinitionStages.Blank defineAuthorizationRule(String name); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operation.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operation.java index 95fbe6ab9f345..aeddcc1c21e5e 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operation.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operation.java @@ -6,25 +6,42 @@ import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** - * Gets the name property: Operation name: {provider}/{resource}/{operation}. - * + * Gets the name property: Gets operation name: {provider}/{resource}/{operation}. + * * @return the name value. */ String name(); /** * Gets the display property: The object that represents the operation. - * + * * @return the display value. */ OperationDisplay display(); + /** + * Gets the properties property: Optional operation properties. + * + * @return the properties value. + */ + OperationProperties properties(); + + /** + * Gets the isDataAction property: Gets or sets IsDataAction property. It is used to differentiate management and + * data plane operations. + * + * @return the isDataAction value. + */ + Boolean isDataAction(); + /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationDisplay.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationDisplay.java index 751ada6bd3cf1..de6423de6b899 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationDisplay.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationDisplay.java @@ -5,36 +5,46 @@ package com.azure.resourcemanager.notificationhubs.models; import com.azure.core.annotation.Immutable; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** The object that represents the operation. */ +/** + * The object that represents the operation. + */ @Immutable public final class OperationDisplay { - @JsonIgnore private final ClientLogger logger = new ClientLogger(OperationDisplay.class); - /* - * Service provider: Microsoft.NotificationHubs + * Gets service provider: Microsoft.NotificationHubs */ @JsonProperty(value = "provider", access = JsonProperty.Access.WRITE_ONLY) private String provider; /* - * Resource on which the operation is performed: Invoice, etc. + * Gets resource on which the operation is performed: Invoice, etc. */ @JsonProperty(value = "resource", access = JsonProperty.Access.WRITE_ONLY) private String resource; /* - * Operation type: Read, write, delete, etc. + * Gets operation type: Read, write, delete, etc. */ @JsonProperty(value = "operation", access = JsonProperty.Access.WRITE_ONLY) private String operation; + /* + * Human-friendly operation description. + */ + @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) + private String description; + + /** + * Creates an instance of OperationDisplay class. + */ + public OperationDisplay() { + } + /** - * Get the provider property: Service provider: Microsoft.NotificationHubs. - * + * Get the provider property: Gets service provider: Microsoft.NotificationHubs. + * * @return the provider value. */ public String provider() { @@ -42,8 +52,8 @@ public String provider() { } /** - * Get the resource property: Resource on which the operation is performed: Invoice, etc. - * + * Get the resource property: Gets resource on which the operation is performed: Invoice, etc. + * * @return the resource value. */ public String resource() { @@ -51,17 +61,26 @@ public String resource() { } /** - * Get the operation property: Operation type: Read, write, delete, etc. - * + * Get the operation property: Gets operation type: Read, write, delete, etc. + * * @return the operation value. */ public String operation() { return this.operation; } + /** + * Get the description property: Human-friendly operation description. + * + * @return the description value. + */ + public String description() { + return this.description; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationListResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationListResult.java index 5db3a18fb8427..0a8d5601052c8 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationListResult.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationListResult.java @@ -5,37 +5,40 @@ package com.azure.resourcemanager.notificationhubs.models; import com.azure.core.annotation.Immutable; -import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; /** - * Result of the request to list NotificationHubs operations. It contains a list of operations and a URL link to get the - * next set of results. + * Result of the request to list NotificationHubs operations. It contains + * a list of operations and a URL link to get the next set of results. */ @Immutable public final class OperationListResult { - @JsonIgnore private final ClientLogger logger = new ClientLogger(OperationListResult.class); - /* - * List of NotificationHubs operations supported by the + * Gets list of NotificationHubs operations supported by the * Microsoft.NotificationHubs resource provider. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) private List value; /* - * URL to get the next set of operation list results if there are any. + * Gets URL to get the next set of operation list results if there are + * any. */ @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; /** - * Get the value property: List of NotificationHubs operations supported by the Microsoft.NotificationHubs resource - * provider. - * + * Creates an instance of OperationListResult class. + */ + public OperationListResult() { + } + + /** + * Get the value property: Gets list of NotificationHubs operations supported by the + * Microsoft.NotificationHubs resource provider. + * * @return the value value. */ public List value() { @@ -43,8 +46,9 @@ public List value() { } /** - * Get the nextLink property: URL to get the next set of operation list results if there are any. - * + * Get the nextLink property: Gets URL to get the next set of operation list results if there are + * any. + * * @return the nextLink value. */ public String nextLink() { @@ -53,7 +57,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProperties.java new file mode 100644 index 0000000000000..7176d08c6ad78 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProperties.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Optional operation properties. + */ +@Fluent +public final class OperationProperties { + /* + * Optional service specification used in Operations API. + */ + @JsonProperty(value = "serviceSpecification") + private ServiceSpecification serviceSpecification; + + /** + * Creates an instance of OperationProperties class. + */ + public OperationProperties() { + } + + /** + * Get the serviceSpecification property: Optional service specification used in Operations API. + * + * @return the serviceSpecification value. + */ + public ServiceSpecification serviceSpecification() { + return this.serviceSpecification; + } + + /** + * Set the serviceSpecification property: Optional service specification used in Operations API. + * + * @param serviceSpecification the serviceSpecification value to set. + * @return the OperationProperties object itself. + */ + public OperationProperties withServiceSpecification(ServiceSpecification serviceSpecification) { + this.serviceSpecification = serviceSpecification; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (serviceSpecification() != null) { + serviceSpecification().validate(); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProvisioningState.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProvisioningState.java new file mode 100644 index 0000000000000..a8f564ef67ffa --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/OperationProvisioningState.java @@ -0,0 +1,78 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for OperationProvisioningState. + */ +public final class OperationProvisioningState extends ExpandableStringEnum { + /** + * Static value Unknown for OperationProvisioningState. + */ + public static final OperationProvisioningState UNKNOWN = fromString("Unknown"); + + /** + * Static value InProgress for OperationProvisioningState. + */ + public static final OperationProvisioningState IN_PROGRESS = fromString("InProgress"); + + /** + * Static value Succeeded for OperationProvisioningState. + */ + public static final OperationProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Failed for OperationProvisioningState. + */ + public static final OperationProvisioningState FAILED = fromString("Failed"); + + /** + * Static value Canceled for OperationProvisioningState. + */ + public static final OperationProvisioningState CANCELED = fromString("Canceled"); + + /** + * Static value Pending for OperationProvisioningState. + */ + public static final OperationProvisioningState PENDING = fromString("Pending"); + + /** + * Static value Disabled for OperationProvisioningState. + */ + public static final OperationProvisioningState DISABLED = fromString("Disabled"); + + /** + * Creates a new instance of OperationProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public OperationProvisioningState() { + } + + /** + * Creates or finds a OperationProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding OperationProvisioningState. + */ + @JsonCreator + public static OperationProvisioningState fromString(String name) { + return fromString(name, OperationProvisioningState.class); + } + + /** + * Gets known OperationProvisioningState values. + * + * @return known OperationProvisioningState values. + */ + public static Collection values() { + return values(OperationProvisioningState.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operations.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operations.java index 1b3dd11ab30ab..91c34040c7d82 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operations.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Operations.java @@ -7,25 +7,29 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** - * Lists all of the available NotificationHubs REST API operations. - * + * Lists all available Notification Hubs operations. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return result of the request to list NotificationHubs operations. + * @return result of the request to list NotificationHubs operations as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PnsCredentialsResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PnsCredentialsResource.java index 46ae0fe56a1d0..1ee5f612fab16 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PnsCredentialsResource.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PnsCredentialsResource.java @@ -4,98 +4,122 @@ package com.azure.resourcemanager.notificationhubs.models; +import com.azure.core.management.SystemData; import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner; import java.util.Map; -/** An immutable client-side representation of PnsCredentialsResource. */ +/** + * An immutable client-side representation of PnsCredentialsResource. + */ public interface PnsCredentialsResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the location property: The geo-location where the resource lives. - * + * Gets the location property: Deprecated - only for compatibility. + * * @return the location value. */ String location(); /** - * Gets the tags property: Resource tags. - * + * Gets the tags property: Deprecated - only for compatibility. + * * @return the tags value. */ Map tags(); /** - * Gets the sku property: The sku of the created namespace. - * - * @return the sku value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - Sku sku(); + SystemData systemData(); /** - * Gets the apnsCredential property: The ApnsCredential of the created NotificationHub. - * + * Gets the admCredential property: Description of a NotificationHub AdmCredential. + * + * @return the admCredential value. + */ + AdmCredential admCredential(); + + /** + * Gets the apnsCredential property: Description of a NotificationHub ApnsCredential. + * * @return the apnsCredential value. */ ApnsCredential apnsCredential(); /** - * Gets the wnsCredential property: The WnsCredential of the created NotificationHub. - * - * @return the wnsCredential value. + * Gets the baiduCredential property: Description of a NotificationHub BaiduCredential. + * + * @return the baiduCredential value. */ - WnsCredential wnsCredential(); + BaiduCredential baiduCredential(); /** - * Gets the gcmCredential property: The GcmCredential of the created NotificationHub. - * + * Gets the browserCredential property: Description of a NotificationHub BrowserCredential. + * + * @return the browserCredential value. + */ + BrowserCredential browserCredential(); + + /** + * Gets the gcmCredential property: Description of a NotificationHub GcmCredential. + * * @return the gcmCredential value. */ GcmCredential gcmCredential(); /** - * Gets the mpnsCredential property: The MpnsCredential of the created NotificationHub. - * + * Gets the mpnsCredential property: Description of a NotificationHub MpnsCredential. + * * @return the mpnsCredential value. */ MpnsCredential mpnsCredential(); /** - * Gets the admCredential property: The AdmCredential of the created NotificationHub. - * - * @return the admCredential value. + * Gets the wnsCredential property: Description of a NotificationHub WnsCredential. + * + * @return the wnsCredential value. */ - AdmCredential admCredential(); + WnsCredential wnsCredential(); /** - * Gets the baiduCredential property: The BaiduCredential of the created NotificationHub. - * - * @return the baiduCredential value. + * Gets the xiaomiCredential property: Description of a NotificationHub XiaomiCredential. + * + * @return the xiaomiCredential value. */ - BaiduCredential baiduCredential(); + XiaomiCredential xiaomiCredential(); + + /** + * Gets the fcmV1Credential property: Description of a NotificationHub FcmV1Credential. + * + * @return the fcmV1Credential value. + */ + FcmV1Credential fcmV1Credential(); /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner object. - * + * * @return the inner object. */ PnsCredentialsResourceInner innerModel(); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyResource.java new file mode 100644 index 0000000000000..20e01cc0edd19 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyResource.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Namespace / NotificationHub Regenerate Keys request. + */ +@Fluent +public final class PolicyKeyResource { + /* + * Type of Shared Access Policy Key (primary or secondary). + */ + @JsonProperty(value = "policyKey", required = true) + private PolicyKeyType policyKey; + + /** + * Creates an instance of PolicyKeyResource class. + */ + public PolicyKeyResource() { + } + + /** + * Get the policyKey property: Type of Shared Access Policy Key (primary or secondary). + * + * @return the policyKey value. + */ + public PolicyKeyType policyKey() { + return this.policyKey; + } + + /** + * Set the policyKey property: Type of Shared Access Policy Key (primary or secondary). + * + * @param policyKey the policyKey value to set. + * @return the PolicyKeyResource object itself. + */ + public PolicyKeyResource withPolicyKey(PolicyKeyType policyKey) { + this.policyKey = policyKey; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (policyKey() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property policyKey in model PolicyKeyResource")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(PolicyKeyResource.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyType.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyType.java new file mode 100644 index 0000000000000..d1716ebc75203 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicyKeyType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Type of Shared Access Policy Key (primary or secondary). + */ +public final class PolicyKeyType extends ExpandableStringEnum { + /** + * Static value PrimaryKey for PolicyKeyType. + */ + public static final PolicyKeyType PRIMARY_KEY = fromString("PrimaryKey"); + + /** + * Static value SecondaryKey for PolicyKeyType. + */ + public static final PolicyKeyType SECONDARY_KEY = fromString("SecondaryKey"); + + /** + * Creates a new instance of PolicyKeyType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PolicyKeyType() { + } + + /** + * Creates or finds a PolicyKeyType from its string representation. + * + * @param name a name to look for. + * @return the corresponding PolicyKeyType. + */ + @JsonCreator + public static PolicyKeyType fromString(String name) { + return fromString(name, PolicyKeyType.class); + } + + /** + * Gets known PolicyKeyType values. + * + * @return known PolicyKeyType values. + */ + public static Collection values() { + return values(PolicyKeyType.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicykeyResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicykeyResource.java deleted file mode 100644 index c6c67fbdffb00..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PolicykeyResource.java +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** Namespace/NotificationHub Regenerate Keys. */ -@Fluent -public final class PolicykeyResource { - @JsonIgnore private final ClientLogger logger = new ClientLogger(PolicykeyResource.class); - - /* - * Name of the key that has to be regenerated for the - * Namespace/Notification Hub Authorization Rule. The value can be Primary - * Key/Secondary Key. - */ - @JsonProperty(value = "policyKey") - private String policyKey; - - /** - * Get the policyKey property: Name of the key that has to be regenerated for the Namespace/Notification Hub - * Authorization Rule. The value can be Primary Key/Secondary Key. - * - * @return the policyKey value. - */ - public String policyKey() { - return this.policyKey; - } - - /** - * Set the policyKey property: Name of the key that has to be regenerated for the Namespace/Notification Hub - * Authorization Rule. The value can be Primary Key/Secondary Key. - * - * @param policyKey the policyKey value to set. - * @return the PolicykeyResource object itself. - */ - public PolicykeyResource withPolicyKey(String policyKey) { - this.policyKey = policyKey; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProperties.java new file mode 100644 index 0000000000000..3dde1f3525c82 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProperties.java @@ -0,0 +1,134 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Private Endpoint Connection properties. + */ +@Fluent +public final class PrivateEndpointConnectionProperties { + /* + * State of Private Endpoint Connection. + */ + @JsonProperty(value = "provisioningState") + private PrivateEndpointConnectionProvisioningState provisioningState; + + /* + * Represents a Private Endpoint that is connected to Notification Hubs namespace using Private Endpoint + * Connection. + */ + @JsonProperty(value = "privateEndpoint") + private RemotePrivateEndpointConnection privateEndpoint; + + /* + * List of group ids. For Notification Hubs, it always contains a single "namespace" element. + */ + @JsonProperty(value = "groupIds", access = JsonProperty.Access.WRITE_ONLY) + private List groupIds; + + /* + * State of the Private Link Service connection. + */ + @JsonProperty(value = "privateLinkServiceConnectionState") + private RemotePrivateLinkServiceConnectionState privateLinkServiceConnectionState; + + /** + * Creates an instance of PrivateEndpointConnectionProperties class. + */ + public PrivateEndpointConnectionProperties() { + } + + /** + * Get the provisioningState property: State of Private Endpoint Connection. + * + * @return the provisioningState value. + */ + public PrivateEndpointConnectionProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Set the provisioningState property: State of Private Endpoint Connection. + * + * @param provisioningState the provisioningState value to set. + * @return the PrivateEndpointConnectionProperties object itself. + */ + public PrivateEndpointConnectionProperties + withProvisioningState(PrivateEndpointConnectionProvisioningState provisioningState) { + this.provisioningState = provisioningState; + return this; + } + + /** + * Get the privateEndpoint property: Represents a Private Endpoint that is connected to Notification Hubs namespace + * using Private Endpoint Connection. + * + * @return the privateEndpoint value. + */ + public RemotePrivateEndpointConnection privateEndpoint() { + return this.privateEndpoint; + } + + /** + * Set the privateEndpoint property: Represents a Private Endpoint that is connected to Notification Hubs namespace + * using Private Endpoint Connection. + * + * @param privateEndpoint the privateEndpoint value to set. + * @return the PrivateEndpointConnectionProperties object itself. + */ + public PrivateEndpointConnectionProperties withPrivateEndpoint(RemotePrivateEndpointConnection privateEndpoint) { + this.privateEndpoint = privateEndpoint; + return this; + } + + /** + * Get the groupIds property: List of group ids. For Notification Hubs, it always contains a single "namespace" + * element. + * + * @return the groupIds value. + */ + public List groupIds() { + return this.groupIds; + } + + /** + * Get the privateLinkServiceConnectionState property: State of the Private Link Service connection. + * + * @return the privateLinkServiceConnectionState value. + */ + public RemotePrivateLinkServiceConnectionState privateLinkServiceConnectionState() { + return this.privateLinkServiceConnectionState; + } + + /** + * Set the privateLinkServiceConnectionState property: State of the Private Link Service connection. + * + * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. + * @return the PrivateEndpointConnectionProperties object itself. + */ + public PrivateEndpointConnectionProperties withPrivateLinkServiceConnectionState( + RemotePrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + this.privateLinkServiceConnectionState = privateLinkServiceConnectionState; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (privateEndpoint() != null) { + privateEndpoint().validate(); + } + if (privateLinkServiceConnectionState() != null) { + privateLinkServiceConnectionState().validate(); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProvisioningState.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProvisioningState.java new file mode 100644 index 0000000000000..7ad99e23aa7f5 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionProvisioningState.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * State of Private Endpoint Connection. + */ +public final class PrivateEndpointConnectionProvisioningState + extends ExpandableStringEnum { + /** + * Static value Unknown for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState UNKNOWN = fromString("Unknown"); + + /** + * Static value Succeeded for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Creating for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState CREATING = fromString("Creating"); + + /** + * Static value Updating for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState UPDATING = fromString("Updating"); + + /** + * Static value UpdatingByProxy for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState UPDATING_BY_PROXY = fromString("UpdatingByProxy"); + + /** + * Static value Deleting for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState DELETING = fromString("Deleting"); + + /** + * Static value DeletingByProxy for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState DELETING_BY_PROXY = fromString("DeletingByProxy"); + + /** + * Static value Deleted for PrivateEndpointConnectionProvisioningState. + */ + public static final PrivateEndpointConnectionProvisioningState DELETED = fromString("Deleted"); + + /** + * Creates a new instance of PrivateEndpointConnectionProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PrivateEndpointConnectionProvisioningState() { + } + + /** + * Creates or finds a PrivateEndpointConnectionProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding PrivateEndpointConnectionProvisioningState. + */ + @JsonCreator + public static PrivateEndpointConnectionProvisioningState fromString(String name) { + return fromString(name, PrivateEndpointConnectionProvisioningState.class); + } + + /** + * Gets known PrivateEndpointConnectionProvisioningState values. + * + * @return known PrivateEndpointConnectionProvisioningState values. + */ + public static Collection values() { + return values(PrivateEndpointConnectionProvisioningState.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResource.java new file mode 100644 index 0000000000000..534d225a1b44e --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResource.java @@ -0,0 +1,56 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; + +/** + * An immutable client-side representation of PrivateEndpointConnectionResource. + */ +public interface PrivateEndpointConnectionResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: Private Endpoint Connection properties. + * + * @return the properties value. + */ + PrivateEndpointConnectionProperties properties(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner + * object. + * + * @return the inner object. + */ + PrivateEndpointConnectionResourceInner innerModel(); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResourceListResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResourceListResult.java new file mode 100644 index 0000000000000..0307cdd01be3c --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnectionResourceListResult.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The response of the List Private Endpoint Connections operation. + */ +@Immutable +public final class PrivateEndpointConnectionResourceListResult { + /* + * Gets or sets result of the List AuthorizationRules operation. + */ + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) + private List value; + + /* + * Gets or sets link to the next set of results. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of PrivateEndpointConnectionResourceListResult class. + */ + public PrivateEndpointConnectionResourceListResult() { + } + + /** + * Get the value property: Gets or sets result of the List AuthorizationRules operation. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Get the nextLink property: Gets or sets link to the next set of results. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnections.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnections.java new file mode 100644 index 0000000000000..55da61f609d43 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateEndpointConnections.java @@ -0,0 +1,216 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; + +/** + * Resource collection API of PrivateEndpointConnections. + */ +public interface PrivateEndpointConnections { + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + PrivateEndpointConnectionResource update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters); + + /** + * Approves or rejects Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param parameters Description of the Private Endpoint Connection resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + PrivateEndpointConnectionResource update(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, PrivateEndpointConnectionResourceInner parameters, Context context); + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName); + + /** + * Deletes the Private Endpoint Connection. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String namespaceName, String privateEndpointConnectionName, Context context); + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace + * along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName, Context context); + + /** + * Returns a Private Endpoint Connection with a given name. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param privateEndpointConnectionName Private Endpoint Connection Name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return represents a Private Endpoint Connection ARM resource - a sub-resource of Notification Hubs namespace. + */ + PrivateEndpointConnectionResource get(String resourceGroupName, String namespaceName, + String privateEndpointConnectionName); + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String namespaceName); + + /** + * Returns all Private Endpoint Connections that belong to the given Notification Hubs namespace. + * This is a public API that can be called directly by Notification Hubs users. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Endpoint Connections operation as paginated response with + * {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String namespaceName, + Context context); + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource along with {@link Response}. + */ + Response getGroupIdWithResponse(String resourceGroupName, String namespaceName, + String subResourceName, Context context); + + /** + * Returns Group Id response. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param subResourceName Name of the Private Link sub-resource. The only supported sub-resource is "namespace". + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a Private Link Arm Resource. + */ + PrivateLinkResource getGroupId(String resourceGroupName, String namespaceName, String subResourceName); + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with + * {@link PagedIterable}. + */ + PagedIterable listGroupIds(String resourceGroupName, String namespaceName); + + /** + * Returns all Group Ids supported by the Notification Hubs RP. + * This is a public API required by the Networking RP contract. It can be used directly by Notification Hubs users. + * + * Even though this namespace requires subscription id, resource group and namespace name, it returns a constant + * payload (for a given namespacE) every time it's called. + * That's why we don't send it to the sibling RP, but process it directly in the scale unit that received the + * request. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the response of the List Private Link Resources operation as paginated response with + * {@link PagedIterable}. + */ + PagedIterable listGroupIds(String resourceGroupName, String namespaceName, Context context); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkConnectionStatus.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkConnectionStatus.java new file mode 100644 index 0000000000000..bcd7e1f80008a --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkConnectionStatus.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * State of Private Link Connection. + */ +public final class PrivateLinkConnectionStatus extends ExpandableStringEnum { + /** + * Static value Disconnected for PrivateLinkConnectionStatus. + */ + public static final PrivateLinkConnectionStatus DISCONNECTED = fromString("Disconnected"); + + /** + * Static value Pending for PrivateLinkConnectionStatus. + */ + public static final PrivateLinkConnectionStatus PENDING = fromString("Pending"); + + /** + * Static value Approved for PrivateLinkConnectionStatus. + */ + public static final PrivateLinkConnectionStatus APPROVED = fromString("Approved"); + + /** + * Static value Rejected for PrivateLinkConnectionStatus. + */ + public static final PrivateLinkConnectionStatus REJECTED = fromString("Rejected"); + + /** + * Creates a new instance of PrivateLinkConnectionStatus value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PrivateLinkConnectionStatus() { + } + + /** + * Creates or finds a PrivateLinkConnectionStatus from its string representation. + * + * @param name a name to look for. + * @return the corresponding PrivateLinkConnectionStatus. + */ + @JsonCreator + public static PrivateLinkConnectionStatus fromString(String name) { + return fromString(name, PrivateLinkConnectionStatus.class); + } + + /** + * Gets known PrivateLinkConnectionStatus values. + * + * @return known PrivateLinkConnectionStatus values. + */ + public static Collection values() { + return values(PrivateLinkConnectionStatus.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResource.java new file mode 100644 index 0000000000000..0790a91cf1333 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResource.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; + +/** + * An immutable client-side representation of PrivateLinkResource. + */ +public interface PrivateLinkResource { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: Represents properties of Private Link Resource. + * + * @return the properties value. + */ + PrivateLinkResourceProperties properties(); + + /** + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner object. + * + * @return the inner object. + */ + PrivateLinkResourceInner innerModel(); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceListResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceListResult.java new file mode 100644 index 0000000000000..e0767ca63e0b5 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceListResult.java @@ -0,0 +1,63 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * The response of the List Private Link Resources operation. + */ +@Immutable +public final class PrivateLinkResourceListResult { + /* + * Gets or sets result of the List AuthorizationRules operation. + */ + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) + private List value; + + /* + * Gets or sets link to the next set of results. + */ + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) + private String nextLink; + + /** + * Creates an instance of PrivateLinkResourceListResult class. + */ + public PrivateLinkResourceListResult() { + } + + /** + * Get the value property: Gets or sets result of the List AuthorizationRules operation. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Get the nextLink property: Gets or sets link to the next set of results. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceProperties.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceProperties.java new file mode 100644 index 0000000000000..2ff4c2420bace --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PrivateLinkResourceProperties.java @@ -0,0 +1,77 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Represents properties of Private Link Resource. + */ +@Immutable +public final class PrivateLinkResourceProperties { + /* + * A Group Id for Private Link. For Notification Hubs, it is always set to "namespace". + */ + @JsonProperty(value = "groupId", access = JsonProperty.Access.WRITE_ONLY) + private String groupId; + + /* + * Required members. For Notification Hubs, it's always a collection with a single "namespace" item. + */ + @JsonProperty(value = "requiredMembers", access = JsonProperty.Access.WRITE_ONLY) + private List requiredMembers; + + /* + * Required DNS zone names. For Notification Hubs, it contains two CNames for Service Bus and Notification Hubs + * zones. + */ + @JsonProperty(value = "requiredZoneNames", access = JsonProperty.Access.WRITE_ONLY) + private List requiredZoneNames; + + /** + * Creates an instance of PrivateLinkResourceProperties class. + */ + public PrivateLinkResourceProperties() { + } + + /** + * Get the groupId property: A Group Id for Private Link. For Notification Hubs, it is always set to "namespace". + * + * @return the groupId value. + */ + public String groupId() { + return this.groupId; + } + + /** + * Get the requiredMembers property: Required members. For Notification Hubs, it's always a collection with a + * single "namespace" item. + * + * @return the requiredMembers value. + */ + public List requiredMembers() { + return this.requiredMembers; + } + + /** + * Get the requiredZoneNames property: Required DNS zone names. For Notification Hubs, it contains two CNames for + * Service Bus and Notification Hubs zones. + * + * @return the requiredZoneNames value. + */ + public List requiredZoneNames() { + return this.requiredZoneNames; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicInternetAuthorizationRule.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicInternetAuthorizationRule.java new file mode 100644 index 0000000000000..bb40711d145d4 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicInternetAuthorizationRule.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * A default (public Internet) network authorization rule, which contains rights if no other network rule matches. + */ +@Fluent +public final class PublicInternetAuthorizationRule { + /* + * List of access rights. + */ + @JsonProperty(value = "rights", required = true) + private List rights; + + /** + * Creates an instance of PublicInternetAuthorizationRule class. + */ + public PublicInternetAuthorizationRule() { + } + + /** + * Get the rights property: List of access rights. + * + * @return the rights value. + */ + public List rights() { + return this.rights; + } + + /** + * Set the rights property: List of access rights. + * + * @param rights the rights value to set. + * @return the PublicInternetAuthorizationRule object itself. + */ + public PublicInternetAuthorizationRule withRights(List rights) { + this.rights = rights; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (rights() == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property rights in model PublicInternetAuthorizationRule")); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(PublicInternetAuthorizationRule.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicNetworkAccess.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicNetworkAccess.java new file mode 100644 index 0000000000000..1f9415b93f9ca --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/PublicNetworkAccess.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Type of public network access. + */ +public final class PublicNetworkAccess extends ExpandableStringEnum { + /** + * Static value Enabled for PublicNetworkAccess. + */ + public static final PublicNetworkAccess ENABLED = fromString("Enabled"); + + /** + * Static value Disabled for PublicNetworkAccess. + */ + public static final PublicNetworkAccess DISABLED = fromString("Disabled"); + + /** + * Creates a new instance of PublicNetworkAccess value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public PublicNetworkAccess() { + } + + /** + * Creates or finds a PublicNetworkAccess from its string representation. + * + * @param name a name to look for. + * @return the corresponding PublicNetworkAccess. + */ + @JsonCreator + public static PublicNetworkAccess fromString(String name) { + return fromString(name, PublicNetworkAccess.class); + } + + /** + * Gets known PublicNetworkAccess values. + * + * @return known PublicNetworkAccess values. + */ + public static Collection values() { + return values(PublicNetworkAccess.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RegistrationResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RegistrationResult.java new file mode 100644 index 0000000000000..83ede8414b096 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RegistrationResult.java @@ -0,0 +1,88 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Notification result for a single registration. + */ +@Immutable +public final class RegistrationResult { + /* + * PNS type. + */ + @JsonProperty(value = "applicationPlatform", access = JsonProperty.Access.WRITE_ONLY) + private String applicationPlatform; + + /* + * PNS handle. + */ + @JsonProperty(value = "pnsHandle", access = JsonProperty.Access.WRITE_ONLY) + private String pnsHandle; + + /* + * Registration id. + */ + @JsonProperty(value = "registrationId", access = JsonProperty.Access.WRITE_ONLY) + private String registrationId; + + /* + * Notification outcome. + */ + @JsonProperty(value = "outcome", access = JsonProperty.Access.WRITE_ONLY) + private String outcome; + + /** + * Creates an instance of RegistrationResult class. + */ + public RegistrationResult() { + } + + /** + * Get the applicationPlatform property: PNS type. + * + * @return the applicationPlatform value. + */ + public String applicationPlatform() { + return this.applicationPlatform; + } + + /** + * Get the pnsHandle property: PNS handle. + * + * @return the pnsHandle value. + */ + public String pnsHandle() { + return this.pnsHandle; + } + + /** + * Get the registrationId property: Registration id. + * + * @return the registrationId value. + */ + public String registrationId() { + return this.registrationId; + } + + /** + * Get the outcome property: Notification outcome. + * + * @return the outcome value. + */ + public String outcome() { + return this.outcome; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateEndpointConnection.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateEndpointConnection.java new file mode 100644 index 0000000000000..019caed130fbf --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateEndpointConnection.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Represents a Private Endpoint that is connected to Notification Hubs namespace using Private Endpoint Connection. + */ +@Immutable +public final class RemotePrivateEndpointConnection { + /* + * ARM resource ID of the Private Endpoint. This may belong to different subscription and resource group than a + * Notification Hubs namespace. + */ + @JsonProperty(value = "id", access = JsonProperty.Access.WRITE_ONLY) + private String id; + + /** + * Creates an instance of RemotePrivateEndpointConnection class. + */ + public RemotePrivateEndpointConnection() { + } + + /** + * Get the id property: ARM resource ID of the Private Endpoint. This may belong to different subscription and + * resource group than a Notification Hubs namespace. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateLinkServiceConnectionState.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateLinkServiceConnectionState.java new file mode 100644 index 0000000000000..80672d299a401 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/RemotePrivateLinkServiceConnectionState.java @@ -0,0 +1,84 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * State of the Private Link Service connection. + */ +@Fluent +public final class RemotePrivateLinkServiceConnectionState { + /* + * State of Private Link Connection. + */ + @JsonProperty(value = "status") + private PrivateLinkConnectionStatus status; + + /* + * Human-friendly description. + */ + @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) + private String description; + + /* + * Human-friendly description of required actions. + */ + @JsonProperty(value = "actionsRequired", access = JsonProperty.Access.WRITE_ONLY) + private String actionsRequired; + + /** + * Creates an instance of RemotePrivateLinkServiceConnectionState class. + */ + public RemotePrivateLinkServiceConnectionState() { + } + + /** + * Get the status property: State of Private Link Connection. + * + * @return the status value. + */ + public PrivateLinkConnectionStatus status() { + return this.status; + } + + /** + * Set the status property: State of Private Link Connection. + * + * @param status the status value to set. + * @return the RemotePrivateLinkServiceConnectionState object itself. + */ + public RemotePrivateLinkServiceConnectionState withStatus(PrivateLinkConnectionStatus status) { + this.status = status; + return this; + } + + /** + * Get the description property: Human-friendly description. + * + * @return the description value. + */ + public String description() { + return this.description; + } + + /** + * Get the actionsRequired property: Human-friendly description of required actions. + * + * @return the actionsRequired value. + */ + public String actionsRequired() { + return this.actionsRequired; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ReplicationRegion.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ReplicationRegion.java new file mode 100644 index 0000000000000..9a3a0a1611988 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ReplicationRegion.java @@ -0,0 +1,83 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Allowed replication region. + */ +public final class ReplicationRegion extends ExpandableStringEnum { + /** + * Static value Default for ReplicationRegion. + */ + public static final ReplicationRegion DEFAULT = fromString("Default"); + + /** + * Static value WestUs2 for ReplicationRegion. + */ + public static final ReplicationRegion WEST_US2 = fromString("WestUs2"); + + /** + * Static value NorthEurope for ReplicationRegion. + */ + public static final ReplicationRegion NORTH_EUROPE = fromString("NorthEurope"); + + /** + * Static value AustraliaEast for ReplicationRegion. + */ + public static final ReplicationRegion AUSTRALIA_EAST = fromString("AustraliaEast"); + + /** + * Static value BrazilSouth for ReplicationRegion. + */ + public static final ReplicationRegion BRAZIL_SOUTH = fromString("BrazilSouth"); + + /** + * Static value SouthEastAsia for ReplicationRegion. + */ + public static final ReplicationRegion SOUTH_EAST_ASIA = fromString("SouthEastAsia"); + + /** + * Static value SouthAfricaNorth for ReplicationRegion. + */ + public static final ReplicationRegion SOUTH_AFRICA_NORTH = fromString("SouthAfricaNorth"); + + /** + * Static value None for ReplicationRegion. + */ + public static final ReplicationRegion NONE = fromString("None"); + + /** + * Creates a new instance of ReplicationRegion value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ReplicationRegion() { + } + + /** + * Creates or finds a ReplicationRegion from its string representation. + * + * @param name a name to look for. + * @return the corresponding ReplicationRegion. + */ + @JsonCreator + public static ReplicationRegion fromString(String name) { + return fromString(name, ReplicationRegion.class); + } + + /** + * Gets known ReplicationRegion values. + * + * @return known ReplicationRegion values. + */ + public static Collection values() { + return values(ReplicationRegion.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ResourceListKeys.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ResourceListKeys.java index dbabd59400fbc..40299e86115d6 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ResourceListKeys.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ResourceListKeys.java @@ -6,46 +6,49 @@ import com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner; -/** An immutable client-side representation of ResourceListKeys. */ +/** + * An immutable client-side representation of ResourceListKeys. + */ public interface ResourceListKeys { /** - * Gets the primaryConnectionString property: PrimaryConnectionString of the AuthorizationRule. - * + * Gets the primaryConnectionString property: Gets or sets primaryConnectionString of the AuthorizationRule. + * * @return the primaryConnectionString value. */ String primaryConnectionString(); /** - * Gets the secondaryConnectionString property: SecondaryConnectionString of the created AuthorizationRule. - * + * Gets the secondaryConnectionString property: Gets or sets secondaryConnectionString of the created + * AuthorizationRule. + * * @return the secondaryConnectionString value. */ String secondaryConnectionString(); /** - * Gets the primaryKey property: PrimaryKey of the created AuthorizationRule. - * + * Gets the primaryKey property: Gets or sets primaryKey of the created AuthorizationRule. + * * @return the primaryKey value. */ String primaryKey(); /** - * Gets the secondaryKey property: SecondaryKey of the created AuthorizationRule. - * + * Gets the secondaryKey property: Gets or sets secondaryKey of the created AuthorizationRule. + * * @return the secondaryKey value. */ String secondaryKey(); /** - * Gets the keyName property: KeyName of the created AuthorizationRule. - * + * Gets the keyName property: Gets or sets keyName of the created AuthorizationRule. + * * @return the keyName value. */ String keyName(); /** * Gets the inner com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner object. - * + * * @return the inner object. */ ResourceListKeysInner innerModel(); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ServiceSpecification.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ServiceSpecification.java new file mode 100644 index 0000000000000..d4b1204deb821 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ServiceSpecification.java @@ -0,0 +1,65 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Immutable; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Optional service specification used in Operations API. + */ +@Immutable +public final class ServiceSpecification { + /* + * Log specifications. + */ + @JsonProperty(value = "logSpecifications", access = JsonProperty.Access.WRITE_ONLY) + private List logSpecifications; + + /* + * Metric specification. + */ + @JsonProperty(value = "metricSpecifications", access = JsonProperty.Access.WRITE_ONLY) + private List metricSpecifications; + + /** + * Creates an instance of ServiceSpecification class. + */ + public ServiceSpecification() { + } + + /** + * Get the logSpecifications property: Log specifications. + * + * @return the logSpecifications value. + */ + public List logSpecifications() { + return this.logSpecifications; + } + + /** + * Get the metricSpecifications property: Metric specification. + * + * @return the metricSpecifications value. + */ + public List metricSpecifications() { + return this.metricSpecifications; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (logSpecifications() != null) { + logSpecifications().forEach(e -> e.validate()); + } + if (metricSpecifications() != null) { + metricSpecifications().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleCreateOrUpdateParameters.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleCreateOrUpdateParameters.java deleted file mode 100644 index 9d9b09ab72c01..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleCreateOrUpdateParameters.java +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** Parameters supplied to the CreateOrUpdate Namespace AuthorizationRules. */ -@Fluent -public final class SharedAccessAuthorizationRuleCreateOrUpdateParameters { - @JsonIgnore - private final ClientLogger logger = new ClientLogger(SharedAccessAuthorizationRuleCreateOrUpdateParameters.class); - - /* - * Properties of the Namespace AuthorizationRules. - */ - @JsonProperty(value = "properties", required = true) - private SharedAccessAuthorizationRuleProperties properties; - - /** - * Get the properties property: Properties of the Namespace AuthorizationRules. - * - * @return the properties value. - */ - public SharedAccessAuthorizationRuleProperties properties() { - return this.properties; - } - - /** - * Set the properties property: Properties of the Namespace AuthorizationRules. - * - * @param properties the properties value to set. - * @return the SharedAccessAuthorizationRuleCreateOrUpdateParameters object itself. - */ - public SharedAccessAuthorizationRuleCreateOrUpdateParameters withProperties( - SharedAccessAuthorizationRuleProperties properties) { - this.properties = properties; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (properties() == null) { - throw logger - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property properties in model" - + " SharedAccessAuthorizationRuleCreateOrUpdateParameters")); - } else { - properties().validate(); - } - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleListResult.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleListResult.java index 7e3d4eb561dfe..e8c96ae9ea2c8 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleListResult.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleListResult.java @@ -4,76 +4,55 @@ package com.azure.resourcemanager.notificationhubs.models; -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; +import com.azure.core.annotation.Immutable; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of the List Namespace operation. */ -@Fluent +/** + * The response of the List Namespace operation. + */ +@Immutable public final class SharedAccessAuthorizationRuleListResult { - @JsonIgnore private final ClientLogger logger = new ClientLogger(SharedAccessAuthorizationRuleListResult.class); - /* - * Result of the List AuthorizationRules operation. + * Gets or sets result of the List AuthorizationRules operation. */ - @JsonProperty(value = "value") + @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) private List value; /* - * Link to the next set of results. Not empty if Value contains incomplete - * list of AuthorizationRules + * Gets or sets link to the next set of results. */ - @JsonProperty(value = "nextLink") + @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; /** - * Get the value property: Result of the List AuthorizationRules operation. - * - * @return the value value. + * Creates an instance of SharedAccessAuthorizationRuleListResult class. */ - public List value() { - return this.value; + public SharedAccessAuthorizationRuleListResult() { } /** - * Set the value property: Result of the List AuthorizationRules operation. - * - * @param value the value value to set. - * @return the SharedAccessAuthorizationRuleListResult object itself. + * Get the value property: Gets or sets result of the List AuthorizationRules operation. + * + * @return the value value. */ - public SharedAccessAuthorizationRuleListResult withValue(List value) { - this.value = value; - return this; + public List value() { + return this.value; } /** - * Get the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of - * AuthorizationRules. - * + * Get the nextLink property: Gets or sets link to the next set of results. + * * @return the nextLink value. */ public String nextLink() { return this.nextLink; } - /** - * Set the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of - * AuthorizationRules. - * - * @param nextLink the nextLink value to set. - * @return the SharedAccessAuthorizationRuleListResult object itself. - */ - public SharedAccessAuthorizationRuleListResult withNextLink(String nextLink) { - this.nextLink = nextLink; - return this; - } - /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleResource.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleResource.java index 8653e9598efda..955291155e36c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleResource.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SharedAccessAuthorizationRuleResource.java @@ -6,287 +6,431 @@ import com.azure.core.http.rest.Response; import com.azure.core.management.Region; +import com.azure.core.management.SystemData; import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; +import java.time.OffsetDateTime; import java.util.List; import java.util.Map; -/** An immutable client-side representation of SharedAccessAuthorizationRuleResource. */ +/** + * An immutable client-side representation of SharedAccessAuthorizationRuleResource. + */ public interface SharedAccessAuthorizationRuleResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** - * Gets the location property: The geo-location where the resource lives. - * + * Gets the location property: Deprecated - only for compatibility. + * * @return the location value. */ String location(); /** - * Gets the tags property: Resource tags. - * + * Gets the tags property: Deprecated - only for compatibility. + * * @return the tags value. */ Map tags(); /** - * Gets the sku property: The sku of the created namespace. - * - * @return the sku value. + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * + * @return the systemData value. */ - Sku sku(); + SystemData systemData(); /** - * Gets the rights property: The rights associated with the rule. - * + * Gets the rights property: Gets or sets the rights associated with the rule. + * * @return the rights value. */ List rights(); /** - * Gets the primaryKey property: A base64-encoded 256-bit primary key for signing and validating the SAS token. - * + * Gets the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * * @return the primaryKey value. */ String primaryKey(); /** - * Gets the secondaryKey property: A base64-encoded 256-bit primary key for signing and validating the SAS token. - * + * Gets the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * * @return the secondaryKey value. */ String secondaryKey(); /** - * Gets the keyName property: A string that describes the authorization rule. - * + * Gets the keyName property: Gets a string that describes the authorization rule. + * * @return the keyName value. */ String keyName(); /** - * Gets the claimType property: A string that describes the claim type. - * - * @return the claimType value. + * Gets the modifiedTime property: Gets the last modified time for this rule. + * + * @return the modifiedTime value. */ - String claimType(); + OffsetDateTime modifiedTime(); /** - * Gets the claimValue property: A string that describes the claim value. - * - * @return the claimValue value. + * Gets the createdTime property: Gets the created time for this rule. + * + * @return the createdTime value. */ - String claimValue(); + OffsetDateTime createdTime(); /** - * Gets the modifiedTime property: The last modified time for this rule. - * - * @return the modifiedTime value. + * Gets the claimType property: Gets a string that describes the claim type. + * + * @return the claimType value. */ - String modifiedTime(); + String claimType(); /** - * Gets the createdTime property: The created time for this rule. - * - * @return the createdTime value. + * Gets the claimValue property: Gets a string that describes the claim value. + * + * @return the claimValue value. */ - String createdTime(); + String claimValue(); /** - * Gets the revision property: The revision number for the rule. - * + * Gets the revision property: Gets the revision number for the rule. + * * @return the revision value. */ Integer revision(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + /** * Gets the inner * com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner object. - * + * * @return the inner object. */ SharedAccessAuthorizationRuleResourceInner innerModel(); - /** The entirety of the SharedAccessAuthorizationRuleResource definition. */ + /** + * The entirety of the SharedAccessAuthorizationRuleResource definition. + */ interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithParentResource, - DefinitionStages.WithProperties, - DefinitionStages.WithCreate { + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The SharedAccessAuthorizationRuleResource definition stages. */ + + /** + * The SharedAccessAuthorizationRuleResource definition stages. + */ interface DefinitionStages { - /** The first stage of the SharedAccessAuthorizationRuleResource definition. */ + /** + * The first stage of the SharedAccessAuthorizationRuleResource definition. + */ interface Blank extends WithParentResource { } - /** The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify parent resource. */ + + /** + * The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify parent resource. + */ interface WithParentResource { /** - * Specifies resourceGroupName, namespaceName. - * - * @param resourceGroupName The name of the resource group. - * @param namespaceName The namespace name. + * Specifies resourceGroupName, namespaceName, notificationHubName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param namespaceName Namespace name. + * @param notificationHubName Notification Hub name. * @return the next definition stage. */ - WithProperties withExistingNamespace(String resourceGroupName, String namespaceName); - } - /** The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify properties. */ - interface WithProperties { - /** - * Specifies the properties property: Properties of the Namespace AuthorizationRules.. - * - * @param properties Properties of the Namespace AuthorizationRules. - * @return the next definition stage. - */ - WithCreate withProperties(SharedAccessAuthorizationRuleProperties properties); + WithCreate withExistingNotificationHub(String resourceGroupName, String namespaceName, + String notificationHubName); } + /** * The stage of the SharedAccessAuthorizationRuleResource definition which contains all the minimum required * properties for the resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate { + interface WithCreate extends DefinitionStages.WithLocation, DefinitionStages.WithTags, + DefinitionStages.WithRights, DefinitionStages.WithPrimaryKey, DefinitionStages.WithSecondaryKey { /** * Executes the create request. - * + * * @return the created resource. */ SharedAccessAuthorizationRuleResource create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ SharedAccessAuthorizationRuleResource create(Context context); } + + /** + * The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify location. + */ + interface WithLocation { + /** + * Specifies the region for the resource. + * + * @param location Deprecated - only for compatibility. + * @return the next definition stage. + */ + WithCreate withRegion(Region location); + + /** + * Specifies the region for the resource. + * + * @param location Deprecated - only for compatibility. + * @return the next definition stage. + */ + WithCreate withRegion(String location); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify tags. + */ + interface WithTags { + /** + * Specifies the tags property: Deprecated - only for compatibility.. + * + * @param tags Deprecated - only for compatibility. + * @return the next definition stage. + */ + WithCreate withTags(Map tags); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify rights. + */ + interface WithRights { + /** + * Specifies the rights property: Gets or sets the rights associated with the rule.. + * + * @param rights Gets or sets the rights associated with the rule. + * @return the next definition stage. + */ + WithCreate withRights(List rights); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify primaryKey. + */ + interface WithPrimaryKey { + /** + * Specifies the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token.. + * + * @param primaryKey Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * @return the next definition stage. + */ + WithCreate withPrimaryKey(String primaryKey); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource definition allowing to specify secondaryKey. + */ + interface WithSecondaryKey { + /** + * Specifies the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token.. + * + * @param secondaryKey Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * @return the next definition stage. + */ + WithCreate withSecondaryKey(String secondaryKey); + } } + /** * Begins update for the SharedAccessAuthorizationRuleResource resource. - * + * * @return the stage of resource update. */ SharedAccessAuthorizationRuleResource.Update update(); - /** The template for SharedAccessAuthorizationRuleResource update. */ - interface Update extends UpdateStages.WithProperties { + /** + * The template for SharedAccessAuthorizationRuleResource update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithRights, UpdateStages.WithPrimaryKey, + UpdateStages.WithSecondaryKey { /** * Executes the update request. - * + * * @return the updated resource. */ SharedAccessAuthorizationRuleResource apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ SharedAccessAuthorizationRuleResource apply(Context context); } - /** The SharedAccessAuthorizationRuleResource update stages. */ + + /** + * The SharedAccessAuthorizationRuleResource update stages. + */ interface UpdateStages { - /** The stage of the SharedAccessAuthorizationRuleResource update allowing to specify properties. */ - interface WithProperties { + /** + * The stage of the SharedAccessAuthorizationRuleResource update allowing to specify tags. + */ + interface WithTags { + /** + * Specifies the tags property: Deprecated - only for compatibility.. + * + * @param tags Deprecated - only for compatibility. + * @return the next definition stage. + */ + Update withTags(Map tags); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource update allowing to specify rights. + */ + interface WithRights { + /** + * Specifies the rights property: Gets or sets the rights associated with the rule.. + * + * @param rights Gets or sets the rights associated with the rule. + * @return the next definition stage. + */ + Update withRights(List rights); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource update allowing to specify primaryKey. + */ + interface WithPrimaryKey { /** - * Specifies the properties property: Properties of the Namespace AuthorizationRules.. - * - * @param properties Properties of the Namespace AuthorizationRules. + * Specifies the primaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token.. + * + * @param primaryKey Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. * @return the next definition stage. */ - Update withProperties(SharedAccessAuthorizationRuleProperties properties); + Update withPrimaryKey(String primaryKey); + } + + /** + * The stage of the SharedAccessAuthorizationRuleResource update allowing to specify secondaryKey. + */ + interface WithSecondaryKey { + /** + * Specifies the secondaryKey property: Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token.. + * + * @param secondaryKey Gets a base64-encoded 256-bit primary key for signing and + * validating the SAS token. + * @return the next definition stage. + */ + Update withSecondaryKey(String secondaryKey); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ SharedAccessAuthorizationRuleResource refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ SharedAccessAuthorizationRuleResource refresh(Context context); /** - * Gets the Primary and Secondary ConnectionStrings to the namespace. - * + * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub along with {@link Response}. */ - ResourceListKeys listKeys(); + Response listKeysWithResponse(Context context); /** - * Gets the Primary and Secondary ConnectionStrings to the namespace. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. + * Gets the Primary and Secondary ConnectionStrings to the NotificationHub. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Primary and Secondary ConnectionStrings to the namespace. + * @return the Primary and Secondary ConnectionStrings to the NotificationHub. */ - Response listKeysWithResponse(Context context); + ResourceListKeys listKeys(); /** - * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. + * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. + * + * @param parameters Request content. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings) + * along with {@link Response}. */ - ResourceListKeys regenerateKeys(PolicykeyResource parameters); + Response regenerateKeysWithResponse(PolicyKeyResource parameters, Context context); /** - * Regenerates the Primary/Secondary Keys to the Namespace Authorization Rule. - * - * @param parameters Parameters supplied to regenerate the Namespace Authorization Rule Key. - * @param context The context to associate with this operation. + * Regenerates the Primary/Secondary Keys to the NotificationHub Authorization Rule. + * + * @param parameters Request content. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return namespace/NotificationHub Connection String. + * @return response for the POST request that returns Namespace or NotificationHub access keys (connection strings). */ - Response regenerateKeysWithResponse(PolicykeyResource parameters, Context context); + ResourceListKeys regenerateKeys(PolicyKeyResource parameters); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Sku.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Sku.java index edbd09c6138c6..eb9fe8a85e097 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Sku.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/Sku.java @@ -6,47 +6,52 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Sku description for a namespace. */ +/** + * The Sku description for a namespace. + */ @Fluent public final class Sku { - @JsonIgnore private final ClientLogger logger = new ClientLogger(Sku.class); - /* - * Name of the notification hub sku + * Namespace SKU name. */ @JsonProperty(value = "name", required = true) private SkuName name; /* - * The tier of particular sku + * Gets or sets the tier of particular sku */ @JsonProperty(value = "tier") private String tier; /* - * The Sku size + * Gets or sets the Sku size */ @JsonProperty(value = "size") private String size; /* - * The Sku Family + * Gets or sets the Sku Family */ @JsonProperty(value = "family") private String family; /* - * The capacity of the resource + * Gets or sets the capacity of the resource */ @JsonProperty(value = "capacity") private Integer capacity; /** - * Get the name property: Name of the notification hub sku. - * + * Creates an instance of Sku class. + */ + public Sku() { + } + + /** + * Get the name property: Namespace SKU name. + * * @return the name value. */ public SkuName name() { @@ -54,8 +59,8 @@ public SkuName name() { } /** - * Set the name property: Name of the notification hub sku. - * + * Set the name property: Namespace SKU name. + * * @param name the name value to set. * @return the Sku object itself. */ @@ -65,8 +70,8 @@ public Sku withName(SkuName name) { } /** - * Get the tier property: The tier of particular sku. - * + * Get the tier property: Gets or sets the tier of particular sku. + * * @return the tier value. */ public String tier() { @@ -74,8 +79,8 @@ public String tier() { } /** - * Set the tier property: The tier of particular sku. - * + * Set the tier property: Gets or sets the tier of particular sku. + * * @param tier the tier value to set. * @return the Sku object itself. */ @@ -85,8 +90,8 @@ public Sku withTier(String tier) { } /** - * Get the size property: The Sku size. - * + * Get the size property: Gets or sets the Sku size. + * * @return the size value. */ public String size() { @@ -94,8 +99,8 @@ public String size() { } /** - * Set the size property: The Sku size. - * + * Set the size property: Gets or sets the Sku size. + * * @param size the size value to set. * @return the Sku object itself. */ @@ -105,8 +110,8 @@ public Sku withSize(String size) { } /** - * Get the family property: The Sku Family. - * + * Get the family property: Gets or sets the Sku Family. + * * @return the family value. */ public String family() { @@ -114,8 +119,8 @@ public String family() { } /** - * Set the family property: The Sku Family. - * + * Set the family property: Gets or sets the Sku Family. + * * @param family the family value to set. * @return the Sku object itself. */ @@ -125,8 +130,8 @@ public Sku withFamily(String family) { } /** - * Get the capacity property: The capacity of the resource. - * + * Get the capacity property: Gets or sets the capacity of the resource. + * * @return the capacity value. */ public Integer capacity() { @@ -134,8 +139,8 @@ public Integer capacity() { } /** - * Set the capacity property: The capacity of the resource. - * + * Set the capacity property: Gets or sets the capacity of the resource. + * * @param capacity the capacity value to set. * @return the Sku object itself. */ @@ -146,13 +151,15 @@ public Sku withCapacity(Integer capacity) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (name() == null) { - throw logger + throw LOGGER .logExceptionAsError(new IllegalArgumentException("Missing required property name in model Sku")); } } + + private static final ClientLogger LOGGER = new ClientLogger(Sku.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SkuName.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SkuName.java index 34611409d6d43..1110459cfa3c2 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SkuName.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/SkuName.java @@ -8,20 +8,37 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for SkuName. */ +/** + * Namespace SKU name. + */ public final class SkuName extends ExpandableStringEnum { - /** Static value Free for SkuName. */ + /** + * Static value Free for SkuName. + */ public static final SkuName FREE = fromString("Free"); - /** Static value Basic for SkuName. */ + /** + * Static value Basic for SkuName. + */ public static final SkuName BASIC = fromString("Basic"); - /** Static value Standard for SkuName. */ + /** + * Static value Standard for SkuName. + */ public static final SkuName STANDARD = fromString("Standard"); + /** + * Creates a new instance of SkuName value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public SkuName() { + } + /** * Creates or finds a SkuName from its string representation. - * + * * @param name a name to look for. * @return the corresponding SkuName. */ @@ -30,7 +47,11 @@ public static SkuName fromString(String name) { return fromString(name, SkuName.class); } - /** @return known SkuName values. */ + /** + * Gets known SkuName values. + * + * @return known SkuName values. + */ public static Collection values() { return values(SkuName.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/WnsCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/WnsCredential.java index 2c7f1b9f1dcde..407d915cd1aad 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/WnsCredential.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/WnsCredential.java @@ -7,23 +7,28 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.notificationhubs.fluent.models.WnsCredentialProperties; -import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -/** Description of a NotificationHub WnsCredential. */ +/** + * Description of a NotificationHub WnsCredential. + */ @Fluent public final class WnsCredential { - @JsonIgnore private final ClientLogger logger = new ClientLogger(WnsCredential.class); - /* - * Properties of NotificationHub WnsCredential. + * Description of a NotificationHub WnsCredential. + */ + @JsonProperty(value = "properties", required = true) + private WnsCredentialProperties innerProperties = new WnsCredentialProperties(); + + /** + * Creates an instance of WnsCredential class. */ - @JsonProperty(value = "properties") - private WnsCredentialProperties innerProperties; + public WnsCredential() { + } /** - * Get the innerProperties property: Properties of NotificationHub WnsCredential. - * + * Get the innerProperties property: Description of a NotificationHub WnsCredential. + * * @return the innerProperties value. */ private WnsCredentialProperties innerProperties() { @@ -31,8 +36,8 @@ private WnsCredentialProperties innerProperties() { } /** - * Get the packageSid property: The package ID for this credential. - * + * Get the packageSid property: Gets or sets the package ID for this credential. + * * @return the packageSid value. */ public String packageSid() { @@ -40,8 +45,8 @@ public String packageSid() { } /** - * Set the packageSid property: The package ID for this credential. - * + * Set the packageSid property: Gets or sets the package ID for this credential. + * * @param packageSid the packageSid value to set. * @return the WnsCredential object itself. */ @@ -54,8 +59,8 @@ public WnsCredential withPackageSid(String packageSid) { } /** - * Get the secretKey property: The secret key. - * + * Get the secretKey property: Gets or sets the secret key. + * * @return the secretKey value. */ public String secretKey() { @@ -63,8 +68,8 @@ public String secretKey() { } /** - * Set the secretKey property: The secret key. - * + * Set the secretKey property: Gets or sets the secret key. + * * @param secretKey the secretKey value to set. * @return the WnsCredential object itself. */ @@ -77,8 +82,8 @@ public WnsCredential withSecretKey(String secretKey) { } /** - * Get the windowsLiveEndpoint property: The Windows Live endpoint. - * + * Get the windowsLiveEndpoint property: Gets or sets the Windows Live endpoint. + * * @return the windowsLiveEndpoint value. */ public String windowsLiveEndpoint() { @@ -86,8 +91,8 @@ public String windowsLiveEndpoint() { } /** - * Set the windowsLiveEndpoint property: The Windows Live endpoint. - * + * Set the windowsLiveEndpoint property: Gets or sets the Windows Live endpoint. + * * @param windowsLiveEndpoint the windowsLiveEndpoint value to set. * @return the WnsCredential object itself. */ @@ -99,14 +104,65 @@ public WnsCredential withWindowsLiveEndpoint(String windowsLiveEndpoint) { return this; } + /** + * Get the certificateKey property: Ges or sets the WNS Certificate Key. + * + * @return the certificateKey value. + */ + public String certificateKey() { + return this.innerProperties() == null ? null : this.innerProperties().certificateKey(); + } + + /** + * Set the certificateKey property: Ges or sets the WNS Certificate Key. + * + * @param certificateKey the certificateKey value to set. + * @return the WnsCredential object itself. + */ + public WnsCredential withCertificateKey(String certificateKey) { + if (this.innerProperties() == null) { + this.innerProperties = new WnsCredentialProperties(); + } + this.innerProperties().withCertificateKey(certificateKey); + return this; + } + + /** + * Get the wnsCertificate property: Gets or sets the WNS Certificate. + * + * @return the wnsCertificate value. + */ + public String wnsCertificate() { + return this.innerProperties() == null ? null : this.innerProperties().wnsCertificate(); + } + + /** + * Set the wnsCertificate property: Gets or sets the WNS Certificate. + * + * @param wnsCertificate the wnsCertificate value to set. + * @return the WnsCredential object itself. + */ + public WnsCredential withWnsCertificate(String wnsCertificate) { + if (this.innerProperties() == null) { + this.innerProperties = new WnsCredentialProperties(); + } + this.innerProperties().withWnsCertificate(wnsCertificate); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (innerProperties() != null) { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model WnsCredential")); + } else { innerProperties().validate(); } } + + private static final ClientLogger LOGGER = new ClientLogger(WnsCredential.class); } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/XiaomiCredential.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/XiaomiCredential.java new file mode 100644 index 0000000000000..c95297f91906e --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/XiaomiCredential.java @@ -0,0 +1,99 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.notificationhubs.fluent.models.XiaomiCredentialProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Description of a NotificationHub XiaomiCredential. + */ +@Fluent +public final class XiaomiCredential { + /* + * Description of a NotificationHub XiaomiCredentialProperties. + */ + @JsonProperty(value = "properties", required = true) + private XiaomiCredentialProperties innerProperties = new XiaomiCredentialProperties(); + + /** + * Creates an instance of XiaomiCredential class. + */ + public XiaomiCredential() { + } + + /** + * Get the innerProperties property: Description of a NotificationHub XiaomiCredentialProperties. + * + * @return the innerProperties value. + */ + private XiaomiCredentialProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the appSecret property: Gets or sets app secret. + * + * @return the appSecret value. + */ + public String appSecret() { + return this.innerProperties() == null ? null : this.innerProperties().appSecret(); + } + + /** + * Set the appSecret property: Gets or sets app secret. + * + * @param appSecret the appSecret value to set. + * @return the XiaomiCredential object itself. + */ + public XiaomiCredential withAppSecret(String appSecret) { + if (this.innerProperties() == null) { + this.innerProperties = new XiaomiCredentialProperties(); + } + this.innerProperties().withAppSecret(appSecret); + return this; + } + + /** + * Get the endpoint property: Gets or sets xiaomi service endpoint. + * + * @return the endpoint value. + */ + public String endpoint() { + return this.innerProperties() == null ? null : this.innerProperties().endpoint(); + } + + /** + * Set the endpoint property: Gets or sets xiaomi service endpoint. + * + * @param endpoint the endpoint value to set. + * @return the XiaomiCredential object itself. + */ + public XiaomiCredential withEndpoint(String endpoint) { + if (this.innerProperties() == null) { + this.innerProperties = new XiaomiCredentialProperties(); + } + this.innerProperties().withEndpoint(endpoint); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property innerProperties in model XiaomiCredential")); + } else { + innerProperties().validate(); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(XiaomiCredential.class); +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ZoneRedundancyPreference.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ZoneRedundancyPreference.java new file mode 100644 index 0000000000000..c3a15a4c68459 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/ZoneRedundancyPreference.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Namespace SKU name. + */ +public final class ZoneRedundancyPreference extends ExpandableStringEnum { + /** + * Static value Disabled for ZoneRedundancyPreference. + */ + public static final ZoneRedundancyPreference DISABLED = fromString("Disabled"); + + /** + * Static value Enabled for ZoneRedundancyPreference. + */ + public static final ZoneRedundancyPreference ENABLED = fromString("Enabled"); + + /** + * Creates a new instance of ZoneRedundancyPreference value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ZoneRedundancyPreference() { + } + + /** + * Creates or finds a ZoneRedundancyPreference from its string representation. + * + * @param name a name to look for. + * @return the corresponding ZoneRedundancyPreference. + */ + @JsonCreator + public static ZoneRedundancyPreference fromString(String name) { + return fromString(name, ZoneRedundancyPreference.class); + } + + /** + * Gets known ZoneRedundancyPreference values. + * + * @return known ZoneRedundancyPreference values. + */ + public static Collection values() { + return values(ZoneRedundancyPreference.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/package-info.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/package-info.java index c61811367b57e..fed4433154092 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/package-info.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the data models for NotificationHubsManagementClient. Azure NotificationHub client. */ +/** + * Package containing the data models for NotificationHubsRPClient. + * Microsoft Notification Hubs Resource Provider REST API. + */ package com.azure.resourcemanager.notificationhubs.models; diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/package-info.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/package-info.java index d6c16347032e4..b6ff59b7fa0e4 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/package-info.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/com/azure/resourcemanager/notificationhubs/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for NotificationHubsManagementClient. Azure NotificationHub client. */ +/** + * Package containing the classes for NotificationHubsRPClient. + * Microsoft Notification Hubs Resource Provider REST API. + */ package com.azure.resourcemanager.notificationhubs; diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/module-info.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/module-info.java index 3e0c96d01782f..156c99d08fbc6 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/module-info.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.notificationhubs { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.notificationhubs; exports com.azure.resourcemanager.notificationhubs.fluent; exports com.azure.resourcemanager.notificationhubs.fluent.models; exports com.azure.resourcemanager.notificationhubs.models; - - opens com.azure.resourcemanager.notificationhubs.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.notificationhubs.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.notificationhubs.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.notificationhubs.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/proxy-config.json b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/proxy-config.json new file mode 100644 index 0000000000000..410d81d5d2391 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.notificationhubs.implementation.NotificationHubsClientImpl$NotificationHubsService" ], [ "com.azure.resourcemanager.notificationhubs.implementation.NamespacesClientImpl$NamespacesService" ], [ "com.azure.resourcemanager.notificationhubs.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.notificationhubs.implementation.PrivateEndpointConnectionsClientImpl$PrivateEndpointConnectionsService" ] ] \ No newline at end of file diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/reflect-config.json b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/reflect-config.json new file mode 100644 index 0000000000000..e359d9a88b1a6 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-notificationhubs/reflect-config.json @@ -0,0 +1,346 @@ +[ { + "name" : "com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.Sku", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.NotificationHubProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.ApnsCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.ApnsCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.WnsCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.WnsCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.GcmCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.GcmCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.MpnsCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.MpnsCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.AdmCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.AdmCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.BaiduCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.BaiduCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.BrowserCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.BrowserCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.XiaomiCredential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.XiaomiCredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.FcmV1Credential", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.FcmV1CredentialProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NotificationHubPatchParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NotificationHubListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResponseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.RegistrationResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.ResourceListKeysInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentialsResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentials", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NetworkAcls", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.IpRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PublicInternetAuthorizationRule", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NamespacePatchParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NamespaceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.OperationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.OperationProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.ServiceSpecification", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.LogSpecification", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.MetricSpecification", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.Availability", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResourceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.SkuName", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.AccessRights", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PolicyKeyType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.OperationProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NamespaceStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.NamespaceType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.ReplicationRegion", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.ZoneRedundancyPreference", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.notificationhubs.models.PublicNetworkAccess", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilitySamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilitySamples.java index 4d7053e87bcdc..6ebb0400a4e8e 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilitySamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilitySamples.java @@ -4,24 +4,25 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -/** Samples for Namespaces CheckAvailability. */ +/** + * Samples for Namespaces CheckAvailability. + */ public final class NamespacesCheckAvailabilitySamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceCheckNameAvailability.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/CheckAvailability.json */ /** - * Sample code: NameSpaceCheckNameAvailability. - * + * Sample code: Namespaces_CheckAvailability. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceCheckNameAvailability( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .checkAvailabilityWithResponse( - new CheckAvailabilityParameters().withName("sdk-Namespace-2924"), Context.NONE); + public static void + namespacesCheckAvailability(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().checkAvailabilityWithResponse( + new CheckAvailabilityParameters().withName("sdk-Namespace-2924"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateAuthorizationRuleSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateAuthorizationRuleSamples.java index 41ba40f3f1e92..191c87cc43c11 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateAuthorizationRuleSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateAuthorizationRuleSamples.java @@ -4,29 +4,30 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; +import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleResourceInner; import com.azure.resourcemanager.notificationhubs.models.AccessRights; import java.util.Arrays; -/** Samples for Namespaces CreateOrUpdateAuthorizationRule. */ +/** + * Samples for Namespaces CreateOrUpdateAuthorizationRule. + */ public final class NamespacesCreateOrUpdateAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleCreateOrUpdate.json */ /** - * Sample code: NameSpaceAuthorizationRuleCreate. - * + * Sample code: Namespaces_CreateOrUpdateAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleCreate( + public static void namespacesCreateOrUpdateAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .defineAuthorizationRule("sdk-AuthRules-1788") - .withExistingNamespace("5ktrial", "nh-sdk-ns") - .withProperties( - new SharedAccessAuthorizationRuleProperties() - .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND))) - .create(); + manager.namespaces() + .createOrUpdateAuthorizationRuleWithResponse( + "5ktrial", "nh-sdk-ns", "sdk-AuthRules-1788", new SharedAccessAuthorizationRuleResourceInner() + .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND)), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateSamples.java index b22ad02306f3e..27c70fe301c8d 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCreateOrUpdateSamples.java @@ -4,32 +4,45 @@ package com.azure.resourcemanager.notificationhubs.generated; +import com.azure.resourcemanager.notificationhubs.models.AccessRights; +import com.azure.resourcemanager.notificationhubs.models.IpRule; +import com.azure.resourcemanager.notificationhubs.models.NetworkAcls; +import com.azure.resourcemanager.notificationhubs.models.PublicInternetAuthorizationRule; import com.azure.resourcemanager.notificationhubs.models.Sku; import com.azure.resourcemanager.notificationhubs.models.SkuName; +import com.azure.resourcemanager.notificationhubs.models.ZoneRedundancyPreference; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for Namespaces CreateOrUpdate. */ +/** + * Samples for Namespaces CreateOrUpdate. + */ public final class NamespacesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/CreateOrUpdate.json */ /** - * Sample code: NameSpaceCreate. - * + * Sample code: Namespaces_CreateOrUpdate. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceCreate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .define("nh-sdk-ns") - .withLocation("South Central US") - .withExistingResourceGroup("5ktrial") - .withTags(mapOf("tag1", "value1", "tag2", "value2")) + public static void + namespacesCreateOrUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().define("nh-sdk-ns").withRegion("South Central US").withExistingResourceGroup("5ktrial") .withSku(new Sku().withName(SkuName.STANDARD).withTier("Standard")) + .withTags(mapOf("tag1", "value1", "tag2", "value2")).withZoneRedundancy(ZoneRedundancyPreference.ENABLED) + .withNetworkAcls(new NetworkAcls() + .withIpRules(Arrays.asList(new IpRule().withIpMask("185.48.100.00/24") + .withRights(Arrays.asList(AccessRights.MANAGE, AccessRights.SEND, AccessRights.LISTEN)))) + .withPublicNetworkRule( + new PublicInternetAuthorizationRule().withRights(Arrays.asList(AccessRights.LISTEN)))) .create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleSamples.java index 13779f27f8ee3..89d2a55cb921e 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleSamples.java @@ -4,22 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces DeleteAuthorizationRule. */ +/** + * Samples for Namespaces DeleteAuthorizationRule. + */ public final class NamespacesDeleteAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleDelete.json */ /** - * Sample code: NameSpaceAuthorizationRuleDelete. - * + * Sample code: Namespaces_DeleteAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleDelete( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .deleteAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", Context.NONE); + public static void + namespacesDeleteAuthorizationRule(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().deleteAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteSamples.java index 818b8668a887a..5d49ea066f70a 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces Delete. */ +/** + * Samples for Namespaces Delete. + */ public final class NamespacesDeleteSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/Delete.json */ /** - * Sample code: NameSpaceDelete. - * + * Sample code: Namespaces_Delete. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().delete("5ktrial", "nh-sdk-ns", Context.NONE); + public static void namespacesDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().deleteByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetAuthorizationRuleSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetAuthorizationRuleSamples.java index 2826567ef50d2..0f2095d97d87b 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetAuthorizationRuleSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetAuthorizationRuleSamples.java @@ -4,22 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces GetAuthorizationRule. */ +/** + * Samples for Namespaces GetAuthorizationRule. + */ public final class NamespacesGetAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleGet.json */ /** - * Sample code: NameSpaceAuthorizationRuleGet. - * + * Sample code: Namespaces_GetAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleGet( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .getAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", Context.NONE); + public static void + namespacesGetAuthorizationRule(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().getAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetByResourceGroupSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetByResourceGroupSamples.java index 94cfca12c92b4..2bc963e74879a 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetByResourceGroupSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetByResourceGroupSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces GetByResourceGroup. */ +/** + * Samples for Namespaces GetByResourceGroup. + */ public final class NamespacesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/Get.json */ /** - * Sample code: NameSpaceGet. - * + * Sample code: Namespaces_Get. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", Context.NONE); + public static void namespacesGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetPnsCredentialsSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetPnsCredentialsSamples.java new file mode 100644 index 0000000000000..e27e3eda7ce32 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesGetPnsCredentialsSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +/** + * Samples for Namespaces GetPnsCredentials. + */ +public final class NamespacesGetPnsCredentialsSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PnsCredentialsGet.json + */ + /** + * Sample code: Namespaces_GetPnsCredentials. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + namespacesGetPnsCredentials(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().getPnsCredentialsWithResponse("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListAuthorizationRulesSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListAuthorizationRulesSamples.java index b6283a71e6ced..ca9837073b48e 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListAuthorizationRulesSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListAuthorizationRulesSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces ListAuthorizationRules. */ +/** + * Samples for Namespaces ListAuthorizationRules. + */ public final class NamespacesListAuthorizationRulesSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleListAll.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleList.json */ /** - * Sample code: NameSpaceAuthorizationRuleListAll. - * + * Sample code: Namespaces_ListAuthorizationRules. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleListAll( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().listAuthorizationRules("5ktrial", "nh-sdk-ns", Context.NONE); + public static void + namespacesListAuthorizationRules(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().listAuthorizationRules("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListByResourceGroupSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListByResourceGroupSamples.java index 8ebf2c8ae78c6..b7e8f30d87ee2 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListByResourceGroupSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListByResourceGroupSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces ListByResourceGroup. */ +/** + * Samples for Namespaces ListByResourceGroup. + */ public final class NamespacesListByResourceGroupSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceListByResourceGroup.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/ListByResourceGroup.json */ /** - * Sample code: NameSpaceListByResourceGroup. - * + * Sample code: Namespaces_List. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceListByResourceGroup( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().listByResourceGroup("5ktrial", Context.NONE); + public static void namespacesList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().listByResourceGroup("5ktrial", null, null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListKeysSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListKeysSamples.java index 93341e9eef6e2..53e838e73a984 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListKeysSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListKeysSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces ListKeys. */ +/** + * Samples for Namespaces ListKeys. + */ public final class NamespacesListKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleListKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleListKeys.json */ /** - * Sample code: NameSpaceAuthorizationRuleListKey. - * + * Sample code: Namespaces_ListKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleListKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().listKeysWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", Context.NONE); + public static void namespacesListKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().listKeysWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListSamples.java index 71829aab87c2e..2dd84cfa60eec 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesListSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Namespaces List. */ +/** + * Samples for Namespaces List. + */ public final class NamespacesListSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceList.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/ListBySubscription.json */ /** - * Sample code: NameSpaceList. - * + * Sample code: Namespaces_ListAll. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.namespaces().list(Context.NONE); + public static void namespacesListAll(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().list(null, null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesPatchSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesPatchSamples.java deleted file mode 100644 index 4f1599d98142f..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesPatchSamples.java +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.generated; - -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.NamespaceResource; -import com.azure.resourcemanager.notificationhubs.models.Sku; -import com.azure.resourcemanager.notificationhubs.models.SkuName; -import java.util.HashMap; -import java.util.Map; - -/** Samples for Namespaces Patch. */ -public final class NamespacesPatchSamples { - /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceUpdate.json - */ - /** - * Sample code: NameSpaceUpdate. - * - * @param manager Entry point to NotificationHubsManager. - */ - public static void nameSpaceUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - NamespaceResource resource = - manager.namespaces().getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", Context.NONE).getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1", "tag2", "value2")) - .withSku(new Sku().withName(SkuName.STANDARD).withTier("Standard")) - .apply(); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesRegenerateKeysSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesRegenerateKeysSamples.java index db77f78fa7378..ad0db8a73e0f6 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesRegenerateKeysSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesRegenerateKeysSamples.java @@ -4,28 +4,26 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyType; -/** Samples for Namespaces RegenerateKeys. */ +/** + * Samples for Namespaces RegenerateKeys. + */ public final class NamespacesRegenerateKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/Namespaces/NHNameSpaceAuthorizationRuleRegenrateKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/AuthorizationRuleRegenerateKey.json */ /** - * Sample code: NameSpaceAuthorizationRuleRegenerateKey. - * + * Sample code: Namespaces_RegenerateKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void nameSpaceAuthorizationRuleRegenerateKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .namespaces() - .regenerateKeysWithResponse( - "5ktrial", - "nh-sdk-ns", - "RootManageSharedAccessKey", - new PolicykeyResource().withPolicyKey("PrimaryKey"), - Context.NONE); + public static void + namespacesRegenerateKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.namespaces().regenerateKeysWithResponse("5ktrial", "nh-sdk-ns", "RootManageSharedAccessKey", + new PolicyKeyResource().withPolicyKey(PolicyKeyType.PRIMARY_KEY), com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesUpdateSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesUpdateSamples.java new file mode 100644 index 0000000000000..335d80b055e3b --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesUpdateSamples.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.resourcemanager.notificationhubs.fluent.models.NamespaceProperties; +import com.azure.resourcemanager.notificationhubs.fluent.models.PnsCredentials; +import com.azure.resourcemanager.notificationhubs.models.GcmCredential; +import com.azure.resourcemanager.notificationhubs.models.NamespaceResource; +import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.SkuName; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Namespaces Update. + */ +public final class NamespacesUpdateSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/Update.json + */ + /** + * Sample code: Namespaces_Update. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void namespacesUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + NamespaceResource resource = manager.namespaces() + .getByResourceGroupWithResponse("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value3")).withSku(new Sku().withName(SkuName.FREE)) + .withProperties( + new NamespaceProperties().withPnsCredentials(new PnsCredentials().withGcmCredential(new GcmCredential() + .withGcmEndpoint("https://fcm.googleapis.com/fcm/send").withGoogleApiKey("fakeTokenPlaceholder")))) + .apply(); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilitySamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilitySamples.java index 2d9513c4895b9..e04cd067e7b0d 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilitySamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilitySamples.java @@ -4,27 +4,26 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; -/** Samples for NotificationHubs CheckNotificationHubAvailability. */ +/** + * Samples for NotificationHubs CheckNotificationHubAvailability. + */ public final class NotificationHubsCheckNotificationHubAvailabilitySamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubCheckNameAvailability.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/CheckAvailability.json */ /** - * Sample code: notificationHubCheckNameAvailability. - * + * Sample code: NotificationHubs_CheckNotificationHubAvailability. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubCheckNameAvailability( + public static void notificationHubsCheckNotificationHubAvailability( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .checkNotificationHubAvailabilityWithResponse( - "5ktrial", - "locp-newns", - new CheckAvailabilityParameters().withName("sdktest").withLocation("West Europe"), - Context.NONE); + manager.notificationHubs().checkNotificationHubAvailabilityWithResponse("5ktrial", "locp-newns", + new CheckAvailabilityParameters().withName("sdktest").withLocation("West Europe"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateAuthorizationRuleSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateAuthorizationRuleSamples.java index 0cfdfee7dee32..d1b07a1b943b8 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateAuthorizationRuleSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateAuthorizationRuleSamples.java @@ -4,35 +4,27 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.fluent.models.SharedAccessAuthorizationRuleProperties; import com.azure.resourcemanager.notificationhubs.models.AccessRights; -import com.azure.resourcemanager.notificationhubs.models.SharedAccessAuthorizationRuleCreateOrUpdateParameters; import java.util.Arrays; -/** Samples for NotificationHubs CreateOrUpdateAuthorizationRule. */ +/** + * Samples for NotificationHubs CreateOrUpdateAuthorizationRule. + */ public final class NotificationHubsCreateOrUpdateAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleCreateOrUpdate.json */ /** - * Sample code: NotificationHubAuthorizationRuleCreate. - * + * Sample code: NotificationHubs_CreateOrUpdateAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleCreate( + public static void notificationHubsCreateOrUpdateAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .createOrUpdateAuthorizationRuleWithResponse( - "5ktrial", - "nh-sdk-ns", - "nh-sdk-hub", - "DefaultListenSharedAccessSignature", - new SharedAccessAuthorizationRuleCreateOrUpdateParameters() - .withProperties( - new SharedAccessAuthorizationRuleProperties() - .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND))), - Context.NONE); + manager.notificationHubs().defineAuthorizationRule("MyManageSharedAccessKey") + .withExistingNotificationHub("5ktrial", "nh-sdk-ns", "nh-sdk-hub") + .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.SEND)).create(); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateSamples.java index c3886cf7fe42f..9a6b1f0a5721c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCreateOrUpdateSamples.java @@ -4,23 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -/** Samples for NotificationHubs CreateOrUpdate. */ +/** + * Samples for NotificationHubs CreateOrUpdate. + */ public final class NotificationHubsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubCreate.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/CreateOrUpdate.json */ /** - * Sample code: NotificationHubCreate. - * + * Sample code: NotificationHubs_CreateOrUpdate. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubCreate( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .define("nh-sdk-hub") - .withRegion("eastus") - .withExistingNamespace("5ktrial", "nh-sdk-ns") - .create(); + public static void + notificationHubsCreateOrUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().define("nh-sdk-hub").withRegion("eastus") + .withExistingNamespace("5ktrial", "nh-sdk-ns").create(); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendSamples.java index 1f8cfae5c86f7..0849ac11fc485 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendSamples.java @@ -4,32 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.management.serializer.SerializerFactory; -import com.azure.core.util.Context; -import com.azure.core.util.serializer.SerializerEncoding; -import java.io.IOException; - -/** Samples for NotificationHubs DebugSend. */ +/** + * Samples for NotificationHubs DebugSend. + */ public final class NotificationHubsDebugSendSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubDebugSend.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/DebugSend.json */ /** - * Sample code: debugsend. - * + * Sample code: NotificationHubs_DebugSend. + * * @param manager Entry point to NotificationHubsManager. */ - public static void debugsend(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) - throws IOException { - manager - .notificationHubs() - .debugSendWithResponse( - "5ktrial", - "nh-sdk-ns", - "nh-sdk-hub", - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize("{\"data\":{\"message\":\"Hello\"}}", Object.class, SerializerEncoding.JSON), - Context.NONE); + public static void + notificationHubsDebugSend(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().debugSendWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleSamples.java index e46dd28a621f8..fcc60452444c2 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleSamples.java @@ -4,23 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs DeleteAuthorizationRule. */ +/** + * Samples for NotificationHubs DeleteAuthorizationRule. + */ public final class NotificationHubsDeleteAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleDelete.json */ /** - * Sample code: NotificationHubAuthorizationRuleDelete. - * + * Sample code: NotificationHubs_DeleteAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleDelete( + public static void notificationHubsDeleteAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .deleteAuthorizationRuleWithResponse( - "5ktrial", "nh-sdk-ns", "nh-sdk-hub", "DefaultListenSharedAccessSignature", Context.NONE); + manager.notificationHubs().deleteAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + "DefaultListenSharedAccessSignature", com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteSamples.java index 51e801436cb99..e5a97e4343a99 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs Delete. */ +/** + * Samples for NotificationHubs Delete. + */ public final class NotificationHubsDeleteSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubDelete.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/Delete.json */ /** - * Sample code: NotificationHubDelete. - * + * Sample code: NotificationHubs_Delete. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubDelete( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().deleteWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + public static void + notificationHubsDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().deleteWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetAuthorizationRuleSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetAuthorizationRuleSamples.java index f853812e86499..677690253365c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetAuthorizationRuleSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetAuthorizationRuleSamples.java @@ -4,23 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs GetAuthorizationRule. */ +/** + * Samples for NotificationHubs GetAuthorizationRule. + */ public final class NotificationHubsGetAuthorizationRuleSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleGet.json */ /** - * Sample code: NotificationHubAuthorizationRuleGet. - * + * Sample code: NotificationHubs_GetAuthorizationRule. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleGet( + public static void notificationHubsGetAuthorizationRule( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .getAuthorizationRuleWithResponse( - "5ktrial", "nh-sdk-ns", "nh-sdk-hub", "DefaultListenSharedAccessSignature", Context.NONE); + manager.notificationHubs().getAuthorizationRuleWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + "DefaultListenSharedAccessSignature", com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetPnsCredentialsSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetPnsCredentialsSamples.java index 225519c3939e2..30d594b4fdb51 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetPnsCredentialsSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetPnsCredentialsSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs GetPnsCredentials. */ +/** + * Samples for NotificationHubs GetPnsCredentials. + */ public final class NotificationHubsGetPnsCredentialsSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubPnsCredentials.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/PnsCredentialsGet.json */ /** - * Sample code: notificationHubPnsCredentials. - * + * Sample code: NotificationHubs_GetPnsCredentials. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubPnsCredentials( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().getPnsCredentialsWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + public static void + notificationHubsGetPnsCredentials(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().getPnsCredentialsWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetSamples.java index bea059fbe1cc8..5fb37394c3e00 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsGetSamples.java @@ -4,19 +4,22 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs Get. */ +/** + * Samples for NotificationHubs Get. + */ public final class NotificationHubsGetSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubGet.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/Get.json */ /** - * Sample code: NotificationHubGet. - * + * Sample code: NotificationHubs_Get. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().getWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + public static void notificationHubsGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().getWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListAuthorizationRulesSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListAuthorizationRulesSamples.java index f45602fb94b8e..74d88c41ce488 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListAuthorizationRulesSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListAuthorizationRulesSamples.java @@ -4,20 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs ListAuthorizationRules. */ +/** + * Samples for NotificationHubs ListAuthorizationRules. + */ public final class NotificationHubsListAuthorizationRulesSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleListAll.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleList.json */ /** - * Sample code: NotificationHubAuthorizationRuleListAll. - * + * Sample code: NotificationHubs_ListAuthorizationRules. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleListAll( + public static void notificationHubsListAuthorizationRules( com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().listAuthorizationRules("5ktrial", "nh-sdk-ns", "nh-sdk-hub", Context.NONE); + manager.notificationHubs().listAuthorizationRules("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListKeysSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListKeysSamples.java index 0eed821801061..46d69b78fe369 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListKeysSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListKeysSamples.java @@ -4,22 +4,23 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs ListKeys. */ +/** + * Samples for NotificationHubs ListKeys. + */ public final class NotificationHubsListKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleListKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleListKeys.json */ /** - * Sample code: NotificationHubAuthorizationRuleListKey. - * + * Sample code: NotificationHubs_ListKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleListKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .listKeysWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", "sdk-AuthRules-5800", Context.NONE); + public static void + notificationHubsListKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().listKeysWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", "sdk-AuthRules-5800", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListSamples.java index 3423e56b436cd..9ce4e0d8b3fde 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsListSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for NotificationHubs List. */ +/** + * Samples for NotificationHubs List. + */ public final class NotificationHubsListSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubListByNameSpace.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/List.json */ /** - * Sample code: NotificationHubListByNameSpace. - * + * Sample code: NotificationHubs_List. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubListByNameSpace( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.notificationHubs().list("5ktrial", "nh-sdk-ns", Context.NONE); + public static void + notificationHubsList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().list("5ktrial", "nh-sdk-ns", null, null, com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsPatchSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsPatchSamples.java deleted file mode 100644 index fe78c5320db2e..0000000000000 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsPatchSamples.java +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.notificationhubs.generated; - -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.NotificationHubResource; - -/** Samples for NotificationHubs Patch. */ -public final class NotificationHubsPatchSamples { - /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubPatch.json - */ - /** - * Sample code: NotificationHubPatch. - * - * @param manager Entry point to NotificationHubsManager. - */ - public static void notificationHubPatch( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - NotificationHubResource resource = - manager - .notificationHubs() - .getWithResponse("sdkresourceGroup", "nh-sdk-ns", "sdk-notificationHubs-8708", Context.NONE) - .getValue(); - resource.update().apply(); - } -} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsRegenerateKeysSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsRegenerateKeysSamples.java index 7480cb0a38040..83fc353905501 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsRegenerateKeysSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsRegenerateKeysSamples.java @@ -4,29 +4,27 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; -import com.azure.resourcemanager.notificationhubs.models.PolicykeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyResource; +import com.azure.resourcemanager.notificationhubs.models.PolicyKeyType; -/** Samples for NotificationHubs RegenerateKeys. */ +/** + * Samples for NotificationHubs RegenerateKeys. + */ public final class NotificationHubsRegenerateKeysSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NotificationHubs/NotificationHubAuthorizationRuleRegenrateKey.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/AuthorizationRuleRegenerateKey.json */ /** - * Sample code: NotificationHubAuthorizationRuleRegenrateKey. - * + * Sample code: NotificationHubs_RegenerateKeys. + * * @param manager Entry point to NotificationHubsManager. */ - public static void notificationHubAuthorizationRuleRegenrateKey( - com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager - .notificationHubs() - .regenerateKeysWithResponse( - "5ktrial", - "nh-sdk-ns", - "nh-sdk-hub", - "DefaultListenSharedAccessSignature", - new PolicykeyResource().withPolicyKey("PrimaryKey"), - Context.NONE); + public static void + notificationHubsRegenerateKeys(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.notificationHubs().regenerateKeysWithResponse("5ktrial", "nh-sdk-ns", "nh-sdk-hub", + "DefaultListenSharedAccessSignature", new PolicyKeyResource().withPolicyKey(PolicyKeyType.PRIMARY_KEY), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsUpdateSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsUpdateSamples.java new file mode 100644 index 0000000000000..6f0ed0ff64f03 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsUpdateSamples.java @@ -0,0 +1,33 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.resourcemanager.notificationhubs.models.GcmCredential; +import com.azure.resourcemanager.notificationhubs.models.NotificationHubResource; + +/** + * Samples for NotificationHubs Update. + */ +public final class NotificationHubsUpdateSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NotificationHubs/Update.json + */ + /** + * Sample code: NotificationHubs_Update. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + notificationHubsUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + NotificationHubResource resource = manager.notificationHubs().getWithResponse("sdkresourceGroup", "nh-sdk-ns", + "sdk-notificationHubs-8708", com.azure.core.util.Context.NONE).getValue(); + resource + .update().withRegistrationTtl("10675199.02:48:05.4775807").withGcmCredential(new GcmCredential() + .withGcmEndpoint("https://fcm.googleapis.com/fcm/send").withGoogleApiKey("fakeTokenPlaceholder")) + .apply(); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListSamples.java index ee887f656794e..d4cca114a1c8c 100644 --- a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListSamples.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListSamples.java @@ -4,19 +4,21 @@ package com.azure.resourcemanager.notificationhubs.generated; -import com.azure.core.util.Context; - -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/stable/2017-04-01/examples/NHOperationsList.json + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * NHOperationsList.json */ /** - * Sample code: OperationsList. - * + * Sample code: Operations_List. + * * @param manager Entry point to NotificationHubsManager. */ public static void operationsList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { - manager.operations().list(Context.NONE); + manager.operations().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsDeleteSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsDeleteSamples.java new file mode 100644 index 0000000000000..d6622a2e55430 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsDeleteSamples.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +/** + * Samples for PrivateEndpointConnections Delete. + */ +public final class PrivateEndpointConnectionsDeleteSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionDelete.json + */ + /** + * Sample code: PrivateEndpointConnections_Delete. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsDelete(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().delete("5ktrial", "nh-sdk-ns", + "nh-sdk-ns.1fa229cd-bf3f-47f0-8c49-afb36723997e", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdSamples.java new file mode 100644 index 0000000000000..7f9b7e601be63 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdSamples.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +/** + * Samples for PrivateEndpointConnections GetGroupId. + */ +public final class PrivateEndpointConnectionsGetGroupIdSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateLinkResourceGet.json + */ + /** + * Sample code: PrivateEndpointConnections_GetGroupId. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void privateEndpointConnectionsGetGroupId( + com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().getGroupIdWithResponse("5ktrial", "nh-sdk-ns", "namespace", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetSamples.java new file mode 100644 index 0000000000000..a261cf5cd6867 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetSamples.java @@ -0,0 +1,26 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +/** + * Samples for PrivateEndpointConnections Get. + */ +public final class PrivateEndpointConnectionsGetSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionGet.json + */ + /** + * Sample code: PrivateEndpointConnections_Get. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsGet(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().getWithResponse("5ktrial", "nh-sdk-ns", + "nh-sdk-ns.1fa229cd-bf3f-47f0-8c49-afb36723997e", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsSamples.java new file mode 100644 index 0000000000000..8094961c62ecd --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +/** + * Samples for PrivateEndpointConnections ListGroupIds. + */ +public final class PrivateEndpointConnectionsListGroupIdsSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateLinkResourceList.json + */ + /** + * Sample code: PrivateEndpointConnections_ListGroupIds. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void privateEndpointConnectionsListGroupIds( + com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().listGroupIds("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListSamples.java new file mode 100644 index 0000000000000..ca931b65cadc1 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListSamples.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +/** + * Samples for PrivateEndpointConnections List. + */ +public final class PrivateEndpointConnectionsListSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionList.json + */ + /** + * Sample code: PrivateEndpointConnections_List. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsList(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().list("5ktrial", "nh-sdk-ns", com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateSamples.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateSamples.java new file mode 100644 index 0000000000000..46db57fbd1c01 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/samples/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateSamples.java @@ -0,0 +1,36 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState; + +/** + * Samples for PrivateEndpointConnections Update. + */ +public final class PrivateEndpointConnectionsUpdateSamples { + /* + * x-ms-original-file: + * specification/notificationhubs/resource-manager/Microsoft.NotificationHubs/preview/2023-10-01-preview/examples/ + * Namespaces/PrivateEndpointConnectionUpdate.json + */ + /** + * Sample code: PrivateEndpointConnections_Update. + * + * @param manager Entry point to NotificationHubsManager. + */ + public static void + privateEndpointConnectionsUpdate(com.azure.resourcemanager.notificationhubs.NotificationHubsManager manager) { + manager.privateEndpointConnections().update("5ktrial", "nh-sdk-ns", + "nh-sdk-ns.1fa229cd-bf3f-47f0-8c49-afb36723997e", + new PrivateEndpointConnectionResourceInner().withProperties(new PrivateEndpointConnectionProperties() + .withPrivateEndpoint(new RemotePrivateEndpointConnection()).withPrivateLinkServiceConnectionState( + new RemotePrivateLinkServiceConnectionState().withStatus(PrivateLinkConnectionStatus.APPROVED))), + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/AvailabilityTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/AvailabilityTests.java new file mode 100644 index 0000000000000..b4094b2c7274a --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/AvailabilityTests.java @@ -0,0 +1,22 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.Availability; + +public final class AvailabilityTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + Availability model = BinaryData.fromString("{\"timeGrain\":\"xhniskxfbkpycgk\",\"blobDuration\":\"ndnhj\"}") + .toObject(Availability.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + Availability model = new Availability(); + model = BinaryData.fromObject(model).toObject(Availability.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityParametersTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityParametersTests.java new file mode 100644 index 0000000000000..9b9016956382e --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityParametersTests.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; +import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.SkuName; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class CheckAvailabilityParametersTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CheckAvailabilityParameters model = BinaryData.fromString( + "{\"id\":\"mhquvgjxp\",\"name\":\"bczmeh\",\"type\":\"zopbsphrupidgs\",\"location\":\"bejhphoycmsxa\",\"tags\":{\"zehtbmu\":\"dxbmtqioq\",\"wnoi\":\"p\",\"bqsoqijg\":\"hwlrx\"},\"isAvailiable\":true,\"sku\":{\"name\":\"Standard\",\"tier\":\"zlobcufpd\",\"size\":\"rbt\",\"family\":\"qjnqglhqgnufoooj\",\"capacity\":1080301009}}") + .toObject(CheckAvailabilityParameters.class); + Assertions.assertEquals("bczmeh", model.name()); + Assertions.assertEquals("bejhphoycmsxa", model.location()); + Assertions.assertEquals("dxbmtqioq", model.tags().get("zehtbmu")); + Assertions.assertEquals(true, model.isAvailiable()); + Assertions.assertEquals(SkuName.STANDARD, model.sku().name()); + Assertions.assertEquals("zlobcufpd", model.sku().tier()); + Assertions.assertEquals("rbt", model.sku().size()); + Assertions.assertEquals("qjnqglhqgnufoooj", model.sku().family()); + Assertions.assertEquals(1080301009, model.sku().capacity()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CheckAvailabilityParameters model = new CheckAvailabilityParameters().withName("bczmeh") + .withLocation("bejhphoycmsxa").withTags(mapOf("zehtbmu", "dxbmtqioq", "wnoi", "p", "bqsoqijg", "hwlrx")) + .withIsAvailiable(true).withSku(new Sku().withName(SkuName.STANDARD).withTier("zlobcufpd").withSize("rbt") + .withFamily("qjnqglhqgnufoooj").withCapacity(1080301009)); + model = BinaryData.fromObject(model).toObject(CheckAvailabilityParameters.class); + Assertions.assertEquals("bczmeh", model.name()); + Assertions.assertEquals("bejhphoycmsxa", model.location()); + Assertions.assertEquals("dxbmtqioq", model.tags().get("zehtbmu")); + Assertions.assertEquals(true, model.isAvailiable()); + Assertions.assertEquals(SkuName.STANDARD, model.sku().name()); + Assertions.assertEquals("zlobcufpd", model.sku().tier()); + Assertions.assertEquals("rbt", model.sku().size()); + Assertions.assertEquals("qjnqglhqgnufoooj", model.sku().family()); + Assertions.assertEquals(1080301009, model.sku().capacity()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityResultInnerTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityResultInnerTests.java new file mode 100644 index 0000000000000..041cea251be0c --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/CheckAvailabilityResultInnerTests.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.fluent.models.CheckAvailabilityResultInner; +import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.SkuName; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class CheckAvailabilityResultInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + CheckAvailabilityResultInner model = BinaryData.fromString( + "{\"isAvailiable\":true,\"location\":\"kwm\",\"tags\":{\"psqucmpoyf\":\"tsizntocipaoua\",\"knygjofjddeq\":\"kfo\",\"upewnwreitjzy\":\"rd\",\"ofcqhsm\":\"lusarh\"},\"sku\":{\"name\":\"Standard\",\"tier\":\"dtmlxhekuksjt\",\"size\":\"kc\",\"family\":\"parcry\",\"capacity\":1654422017},\"id\":\"w\",\"name\":\"xzdxtayrlhmwh\",\"type\":\"pmrqobm\"}") + .toObject(CheckAvailabilityResultInner.class); + Assertions.assertEquals(true, model.isAvailiable()); + Assertions.assertEquals("kwm", model.location()); + Assertions.assertEquals("tsizntocipaoua", model.tags().get("psqucmpoyf")); + Assertions.assertEquals(SkuName.STANDARD, model.sku().name()); + Assertions.assertEquals("dtmlxhekuksjt", model.sku().tier()); + Assertions.assertEquals("kc", model.sku().size()); + Assertions.assertEquals("parcry", model.sku().family()); + Assertions.assertEquals(1654422017, model.sku().capacity()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + CheckAvailabilityResultInner model + = new CheckAvailabilityResultInner().withIsAvailiable(true).withLocation("kwm") + .withTags(mapOf("psqucmpoyf", "tsizntocipaoua", "knygjofjddeq", "kfo", "upewnwreitjzy", "rd", "ofcqhsm", + "lusarh")) + .withSku(new Sku().withName(SkuName.STANDARD).withTier("dtmlxhekuksjt").withSize("kc") + .withFamily("parcry").withCapacity(1654422017)); + model = BinaryData.fromObject(model).toObject(CheckAvailabilityResultInner.class); + Assertions.assertEquals(true, model.isAvailiable()); + Assertions.assertEquals("kwm", model.location()); + Assertions.assertEquals("tsizntocipaoua", model.tags().get("psqucmpoyf")); + Assertions.assertEquals(SkuName.STANDARD, model.sku().name()); + Assertions.assertEquals("dtmlxhekuksjt", model.sku().tier()); + Assertions.assertEquals("kc", model.sku().size()); + Assertions.assertEquals("parcry", model.sku().family()); + Assertions.assertEquals(1654422017, model.sku().capacity()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResponseInnerTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResponseInnerTests.java new file mode 100644 index 0000000000000..0e17dc973c9d4 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResponseInnerTests.java @@ -0,0 +1,43 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResponseInner; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; + +public final class DebugSendResponseInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DebugSendResponseInner model = BinaryData.fromString( + "{\"properties\":{\"success\":736999913427384233,\"failure\":8894415977580296297,\"results\":[{\"applicationPlatform\":\"ntnbybkzg\",\"pnsHandle\":\"rwclxxwrljdo\",\"registrationId\":\"kcqvkocrc\",\"outcome\":\"kwt\"},{\"applicationPlatform\":\"xbnjbiksq\",\"pnsHandle\":\"lssai\",\"registrationId\":\"p\",\"outcome\":\"nzl\"},{\"applicationPlatform\":\"fmppe\",\"pnsHandle\":\"vmgxsab\",\"registrationId\":\"qduujitcjczdz\",\"outcome\":\"ndhkrw\"},{\"applicationPlatform\":\"appd\",\"pnsHandle\":\"dkvwrwjfe\",\"registrationId\":\"nhutjeltmrldhugj\",\"outcome\":\"datqxhocdgeabl\"}]},\"location\":\"huticndvkao\",\"tags\":{\"tyhxhurokft\":\"i\",\"iawxklry\":\"xolniwpwcukjfk\",\"cbacphejkotynqg\":\"lwckbasyypnddhs\"},\"id\":\"l\",\"name\":\"ndlik\",\"type\":\"yqkgfg\"}") + .toObject(DebugSendResponseInner.class); + Assertions.assertEquals("huticndvkao", model.location()); + Assertions.assertEquals("i", model.tags().get("tyhxhurokft")); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DebugSendResponseInner model = new DebugSendResponseInner().withLocation("huticndvkao") + .withTags(mapOf("tyhxhurokft", "i", "iawxklry", "xolniwpwcukjfk", "cbacphejkotynqg", "lwckbasyypnddhs")); + model = BinaryData.fromObject(model).toObject(DebugSendResponseInner.class); + Assertions.assertEquals("huticndvkao", model.location()); + Assertions.assertEquals("i", model.tags().get("tyhxhurokft")); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResultTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResultTests.java new file mode 100644 index 0000000000000..44ba818c5ed69 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/DebugSendResultTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.fluent.models.DebugSendResult; + +public final class DebugSendResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + DebugSendResult model = BinaryData.fromString( + "{\"success\":5040157276386873949,\"failure\":7701952608806501172,\"results\":[{\"applicationPlatform\":\"s\",\"pnsHandle\":\"yb\",\"registrationId\":\"qedqytbciqfoufl\",\"outcome\":\"nkzsmodmglou\"},{\"applicationPlatform\":\"b\",\"pnsHandle\":\"tmut\",\"registrationId\":\"qktapspwgcuert\",\"outcome\":\"kdosvqw\"},{\"applicationPlatform\":\"mdgbbjfdd\",\"pnsHandle\":\"bmbexppbhtqqro\",\"registrationId\":\"p\",\"outcome\":\"s\"},{\"applicationPlatform\":\"gbquxigj\",\"pnsHandle\":\"gzjaoyfhrtxilne\",\"registrationId\":\"ujysvle\",\"outcome\":\"vfqawrlyxwjkcpr\"}]}") + .toObject(DebugSendResult.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + DebugSendResult model = new DebugSendResult(); + model = BinaryData.fromObject(model).toObject(DebugSendResult.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/IpRuleTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/IpRuleTests.java new file mode 100644 index 0000000000000..c86c08f4c67ef --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/IpRuleTests.java @@ -0,0 +1,30 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.AccessRights; +import com.azure.resourcemanager.notificationhubs.models.IpRule; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class IpRuleTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + IpRule model = BinaryData.fromString("{\"ipMask\":\"k\",\"rights\":[\"Send\",\"Manage\",\"Manage\",\"Send\"]}") + .toObject(IpRule.class); + Assertions.assertEquals("k", model.ipMask()); + Assertions.assertEquals(AccessRights.SEND, model.rights().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + IpRule model = new IpRule().withIpMask("k") + .withRights(Arrays.asList(AccessRights.SEND, AccessRights.MANAGE, AccessRights.MANAGE, AccessRights.SEND)); + model = BinaryData.fromObject(model).toObject(IpRule.class); + Assertions.assertEquals("k", model.ipMask()); + Assertions.assertEquals(AccessRights.SEND, model.rights().get(0)); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/LogSpecificationTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/LogSpecificationTests.java new file mode 100644 index 0000000000000..1f5b98b3d7675 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/LogSpecificationTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.LogSpecification; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class LogSpecificationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + LogSpecification model = BinaryData.fromString( + "{\"name\":\"fm\",\"displayName\":\"gkvtmelmqkrhah\",\"blobDuration\":\"juahaquhcdhmdual\",\"categoryGroups\":[\"qpv\",\"adm\",\"sr\"]}") + .toObject(LogSpecification.class); + Assertions.assertEquals("qpv", model.categoryGroups().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + LogSpecification model = new LogSpecification().withCategoryGroups(Arrays.asList("qpv", "adm", "sr")); + model = BinaryData.fromObject(model).toObject(LogSpecification.class); + Assertions.assertEquals("qpv", model.categoryGroups().get(0)); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/MetricSpecificationTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/MetricSpecificationTests.java new file mode 100644 index 0000000000000..838810564d4bd --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/MetricSpecificationTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.MetricSpecification; + +public final class MetricSpecificationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + MetricSpecification model = BinaryData.fromString( + "{\"name\":\"gvxp\",\"displayName\":\"omzlfmi\",\"displayDescription\":\"wbnb\",\"unit\":\"ldawkzbaliourqha\",\"aggregationType\":\"uhashsfwx\",\"availabilities\":[{\"timeGrain\":\"z\",\"blobDuration\":\"ugicjooxdjebw\"},{\"timeGrain\":\"cwwfvovbvme\",\"blobDuration\":\"civyhzceuo\"}],\"supportedTimeGrainTypes\":[\"rw\",\"ueiotwmcdyt\",\"x\"],\"metricFilterPattern\":\"txnrjaw\",\"fillGapWithZero\":false}") + .toObject(MetricSpecification.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + MetricSpecification model = new MetricSpecification(); + model = BinaryData.fromObject(model).toObject(MetricSpecification.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilityWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilityWithResponseMockTests.java new file mode 100644 index 0000000000000..aaf089e9e0ee9 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesCheckAvailabilityWithResponseMockTests.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; +import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityResult; +import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.SkuName; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class NamespacesCheckAvailabilityWithResponseMockTests { + @Test + public void testCheckAvailabilityWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"isAvailiable\":false,\"location\":\"mueedndrdvstk\",\"tags\":{\"tdaaygdvwvg\":\"tchealmf\"},\"sku\":{\"name\":\"Basic\",\"tier\":\"g\",\"size\":\"rtfudxepxg\",\"family\":\"agvrvmnpkuk\",\"capacity\":101100234},\"id\":\"dblx\",\"name\":\"wi\",\"type\":\"fnjhfjxwmszkkfo\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + CheckAvailabilityResult response = manager.namespaces() + .checkAvailabilityWithResponse( + new CheckAvailabilityParameters() + .withName("tllxdyhgsyocogj").withLocation("ocrkvcikh").withTags(mapOf("gxk", "amqgxqquezikyw")) + .withIsAvailiable(false).withSku(new Sku().withName(SkuName.BASIC).withTier("elwuipi") + .withSize("jzkzi").withFamily("vvcnayr").withCapacity(108895877)), + com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals(false, response.isAvailiable()); + Assertions.assertEquals("mueedndrdvstk", response.location()); + Assertions.assertEquals("tchealmf", response.tags().get("tdaaygdvwvg")); + Assertions.assertEquals(SkuName.BASIC, response.sku().name()); + Assertions.assertEquals("g", response.sku().tier()); + Assertions.assertEquals("rtfudxepxg", response.sku().size()); + Assertions.assertEquals("agvrvmnpkuk", response.sku().family()); + Assertions.assertEquals(101100234, response.sku().capacity()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleWithResponseMockTests.java new file mode 100644 index 0000000000000..47768ea164bbd --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteAuthorizationRuleWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class NamespacesDeleteAuthorizationRuleWithResponseMockTests { + @Test + public void testDeleteAuthorizationRuleWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.namespaces().deleteAuthorizationRuleWithResponse("iolxor", "altol", "ncwsob", + com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteByResourceGroupWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteByResourceGroupWithResponseMockTests.java new file mode 100644 index 0000000000000..69baef7644385 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NamespacesDeleteByResourceGroupWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class NamespacesDeleteByResourceGroupWithResponseMockTests { + @Test + public void testDeleteWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.namespaces().deleteByResourceGroupWithResponse("sofwqmzqalkrmnji", "pxacqqudfn", + com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NetworkAclsTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NetworkAclsTests.java new file mode 100644 index 0000000000000..8e91c35e2a185 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NetworkAclsTests.java @@ -0,0 +1,40 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.AccessRights; +import com.azure.resourcemanager.notificationhubs.models.IpRule; +import com.azure.resourcemanager.notificationhubs.models.NetworkAcls; +import com.azure.resourcemanager.notificationhubs.models.PublicInternetAuthorizationRule; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class NetworkAclsTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + NetworkAcls model = BinaryData.fromString( + "{\"ipRules\":[{\"ipMask\":\"jwzrl\",\"rights\":[\"Manage\",\"Listen\"]},{\"ipMask\":\"lwhijcoejctbzaq\",\"rights\":[\"Send\",\"Manage\",\"Send\"]}],\"publicNetworkRule\":{\"rights\":[\"Listen\",\"Listen\"]}}") + .toObject(NetworkAcls.class); + Assertions.assertEquals("jwzrl", model.ipRules().get(0).ipMask()); + Assertions.assertEquals(AccessRights.MANAGE, model.ipRules().get(0).rights().get(0)); + Assertions.assertEquals(AccessRights.LISTEN, model.publicNetworkRule().rights().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + NetworkAcls model = new NetworkAcls() + .withIpRules(Arrays.asList( + new IpRule().withIpMask("jwzrl").withRights(Arrays.asList(AccessRights.MANAGE, AccessRights.LISTEN)), + new IpRule().withIpMask("lwhijcoejctbzaq") + .withRights(Arrays.asList(AccessRights.SEND, AccessRights.MANAGE, AccessRights.SEND)))) + .withPublicNetworkRule(new PublicInternetAuthorizationRule() + .withRights(Arrays.asList(AccessRights.LISTEN, AccessRights.LISTEN))); + model = BinaryData.fromObject(model).toObject(NetworkAcls.class); + Assertions.assertEquals("jwzrl", model.ipRules().get(0).ipMask()); + Assertions.assertEquals(AccessRights.MANAGE, model.ipRules().get(0).rights().get(0)); + Assertions.assertEquals(AccessRights.LISTEN, model.publicNetworkRule().rights().get(0)); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilityWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilityWithResponseMockTests.java new file mode 100644 index 0000000000000..ce1c2d0803aa0 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsCheckNotificationHubAvailabilityWithResponseMockTests.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityParameters; +import com.azure.resourcemanager.notificationhubs.models.CheckAvailabilityResult; +import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.SkuName; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class NotificationHubsCheckNotificationHubAvailabilityWithResponseMockTests { + @Test + public void testCheckNotificationHubAvailabilityWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"isAvailiable\":false,\"location\":\"qqkdltfzxmhhvhgu\",\"tags\":{\"ibqdxbxwakbogqx\":\"dkwobdagx\",\"lbpodxunk\":\"dlkzgxhuri\"},\"sku\":{\"name\":\"Basic\",\"tier\":\"mubyynt\",\"size\":\"rbqtkoie\",\"family\":\"eotg\",\"capacity\":209864806},\"id\":\"tmuwlauwzi\",\"name\":\"xbmp\",\"type\":\"cjefuzmu\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + CheckAvailabilityResult response = manager.notificationHubs() + .checkNotificationHubAvailabilityWithResponse("mdnbbglzpswiy", "mcwyhzdxssadb", + new CheckAvailabilityParameters().withName("dvxzbncblylpst").withLocation("rzdzucerscdnt") + .withTags(mapOf("tmweriofzpyq", "fiwjmygtdssls", "hhszh", "emwabnet", "lvwiwubmwmbesl", "d")) + .withIsAvailiable(false).withSku(new Sku().withName(SkuName.FREE).withTier("pp").withSize("lcxog") + .withFamily("konzmnsik").withCapacity(1013346228)), + com.azure.core.util.Context.NONE) + .getValue(); + + Assertions.assertEquals(false, response.isAvailiable()); + Assertions.assertEquals("qqkdltfzxmhhvhgu", response.location()); + Assertions.assertEquals("dkwobdagx", response.tags().get("ibqdxbxwakbogqx")); + Assertions.assertEquals(SkuName.BASIC, response.sku().name()); + Assertions.assertEquals("mubyynt", response.sku().tier()); + Assertions.assertEquals("rbqtkoie", response.sku().size()); + Assertions.assertEquals("eotg", response.sku().family()); + Assertions.assertEquals(209864806, response.sku().capacity()); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendWithResponseMockTests.java new file mode 100644 index 0000000000000..d231078c02b93 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDebugSendWithResponseMockTests.java @@ -0,0 +1,57 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.DebugSendResponse; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class NotificationHubsDebugSendWithResponseMockTests { + @Test + public void testDebugSendWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"success\":5181138864066743428,\"failure\":3926019895987631021,\"results\":[{\"applicationPlatform\":\"vo\",\"pnsHandle\":\"sotbob\",\"registrationId\":\"opcjwvnhd\",\"outcome\":\"wmgxcxrsl\"},{\"applicationPlatform\":\"utwu\",\"pnsHandle\":\"grpkhjwniyqs\",\"registrationId\":\"i\",\"outcome\":\"dggkzzlvmbmpa\"}]},\"location\":\"odfvuefywsbp\",\"tags\":{\"wiyzvqtmnubexkp\":\"wyhrfouyftaakc\",\"mquxvypo\":\"ksmond\"},\"id\":\"kopkwhojvpajqgx\",\"name\":\"smocmbq\",\"type\":\"qvmkcxo\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + DebugSendResponse response = manager.notificationHubs() + .debugSendWithResponse("hgw", "apnedgfbcvkc", "q", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("odfvuefywsbp", response.location()); + Assertions.assertEquals("wyhrfouyftaakc", response.tags().get("wiyzvqtmnubexkp")); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleWithResponseMockTests.java new file mode 100644 index 0000000000000..6c24a2a07dc53 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteAuthorizationRuleWithResponseMockTests.java @@ -0,0 +1,52 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class NotificationHubsDeleteAuthorizationRuleWithResponseMockTests { + @Test + public void testDeleteAuthorizationRuleWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.notificationHubs().deleteAuthorizationRuleWithResponse("apvhelxprgly", "tddckcb", "uejrjxgc", + "qibrhosxsdqrhzoy", com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteWithResponseMockTests.java similarity index 80% rename from sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteWithResponseMockTests.java rename to sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteWithResponseMockTests.java index e1a9fe98c373f..10b3daf4e4993 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListAddressesDeleteWithResponseMockTests.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/NotificationHubsDeleteWithResponseMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.generated; +package com.azure.resourcemanager.notificationhubs.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -11,7 +11,7 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.communication.CommunicationManager; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -21,7 +21,7 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SuppressionListAddressesDeleteWithResponseMockTests { +public final class NotificationHubsDeleteWithResponseMockTests { @Test public void testDeleteWithResponse() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); @@ -41,12 +41,11 @@ public void testDeleteWithResponse() throws Exception { return Mono.just(httpResponse); })); - CommunicationManager manager = CommunicationManager.configure().withHttpClient(httpClient).authenticate( + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - manager.suppressionListAddresses().deleteWithResponse("kif", "vtpuqujmqlgk", "btndo", "ongbjcnt", - "jitcjedftwwaez", com.azure.core.util.Context.NONE); + manager.notificationHubs().deleteWithResponse("fikdowwqu", "v", "zx", com.azure.core.util.Context.NONE); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationDisplayTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationDisplayTests.java new file mode 100644 index 0000000000000..a39db2b5e33aa --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationDisplayTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.OperationDisplay; + +public final class OperationDisplayTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"yxoegukgjnp\",\"resource\":\"cgygev\",\"operation\":\"ntypmrbpizcdrqj\",\"description\":\"pyd\"}") + .toObject(OperationDisplay.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + OperationDisplay model = new OperationDisplay(); + model = BinaryData.fromObject(model).toObject(OperationDisplay.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationInnerTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationInnerTests.java new file mode 100644 index 0000000000000..6e9b317abbcbb --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationInnerTests.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.fluent.models.OperationInner; +import com.azure.resourcemanager.notificationhubs.models.OperationDisplay; +import com.azure.resourcemanager.notificationhubs.models.OperationProperties; +import com.azure.resourcemanager.notificationhubs.models.ServiceSpecification; + +public final class OperationInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + OperationInner model = BinaryData.fromString( + "{\"name\":\"ae\",\"display\":{\"provider\":\"hy\",\"resource\":\"trpmo\",\"operation\":\"mcmatuokthfuiu\",\"description\":\"dsfcpkvxodpuoz\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{\"name\":\"gf\",\"displayName\":\"xbezyiuokktwh\",\"blobDuration\":\"xw\",\"categoryGroups\":[\"q\",\"mbsureximo\"]},{\"name\":\"ocfs\",\"displayName\":\"s\",\"blobDuration\":\"ddystkiiuxhqy\",\"categoryGroups\":[\"o\",\"rq\"]},{\"name\":\"poczvyifqrvkdvjs\",\"displayName\":\"rm\",\"blobDuration\":\"d\",\"categoryGroups\":[\"t\",\"pnpulexxbczwtru\"]},{\"name\":\"qzbqjvsov\",\"displayName\":\"okacspk\",\"blobDuration\":\"hzdobpxjmflbvvnc\",\"categoryGroups\":[\"cciw\",\"zjuqkhrsaj\",\"wkuofoskghsauu\",\"mjmvxieduugidyjr\"]}],\"metricSpecifications\":[{\"name\":\"aos\",\"displayName\":\"xc\",\"displayDescription\":\"npc\",\"unit\":\"ocohslkevleg\",\"aggregationType\":\"fbuhfmvfaxkffe\",\"availabilities\":[{}],\"supportedTimeGrainTypes\":[\"vmezy\",\"shxmzsbbzoggigrx\",\"burvjxxjnspy\",\"ptkoenkoukn\"],\"metricFilterPattern\":\"dwtiukbldngkp\",\"fillGapWithZero\":true}]}},\"isDataAction\":true}") + .toObject(OperationInner.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + OperationInner model = new OperationInner().withDisplay(new OperationDisplay()) + .withProperties(new OperationProperties().withServiceSpecification(new ServiceSpecification())); + model = BinaryData.fromObject(model).toObject(OperationInner.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationListResultTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationListResultTests.java new file mode 100644 index 0000000000000..b6b3c6e030ccb --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationListResultTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.OperationListResult; + +public final class OperationListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"mohctb\",\"display\":{\"provider\":\"dwxdndnv\",\"resource\":\"gujjugwdkcglh\",\"operation\":\"azjdyggd\",\"description\":\"ixhbkuofqweykhm\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{},{}],\"metricSpecifications\":[{},{}]}},\"isDataAction\":true},{\"name\":\"hybcibv\",\"display\":{\"provider\":\"c\",\"resource\":\"tynnaamdectehfi\",\"operation\":\"cj\",\"description\":\"pvhez\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{},{},{}],\"metricSpecifications\":[{},{},{},{}]}},\"isDataAction\":false},{\"name\":\"o\",\"display\":{\"provider\":\"kqsleyyvxy\",\"resource\":\"pkc\",\"operation\":\"tpngjcrcczsqpjh\",\"description\":\"daj\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{},{},{}],\"metricSpecifications\":[{},{},{}]}},\"isDataAction\":false}],\"nextLink\":\"a\"}") + .toObject(OperationListResult.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + OperationListResult model = new OperationListResult(); + model = BinaryData.fromObject(model).toObject(OperationListResult.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationPropertiesTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationPropertiesTests.java new file mode 100644 index 0000000000000..00c6bc5c41e04 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationPropertiesTests.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.OperationProperties; +import com.azure.resourcemanager.notificationhubs.models.ServiceSpecification; + +public final class OperationPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + OperationProperties model = BinaryData.fromString( + "{\"serviceSpecification\":{\"logSpecifications\":[{\"name\":\"de\",\"displayName\":\"jzicwifsjt\",\"blobDuration\":\"zfbishcbkhaj\",\"categoryGroups\":[\"eamdp\",\"agalpbuxwgipwhon\",\"wkgshwa\",\"kix\"]},{\"name\":\"injep\",\"displayName\":\"tmryw\",\"blobDuration\":\"zoqftiyqzrnkcqvy\",\"categoryGroups\":[\"hzls\",\"cohoq\",\"nwvlryavwhheunmm\"]}],\"metricSpecifications\":[{\"name\":\"xzko\",\"displayName\":\"cukoklyaxuconu\",\"displayDescription\":\"zf\",\"unit\":\"eyp\",\"aggregationType\":\"rmjmwvvjektc\",\"availabilities\":[{\"timeGrain\":\"hwlrsf\",\"blobDuration\":\"zpwv\"},{\"timeGrain\":\"dqgbiqylihkaetc\",\"blobDuration\":\"vfcivfsnkymuc\"}],\"supportedTimeGrainTypes\":[\"jf\",\"ebrjcxe\",\"fuwutttxf\"],\"metricFilterPattern\":\"rbirphxe\",\"fillGapWithZero\":false},{\"name\":\"ahfn\",\"displayName\":\"kyqxjvuujqgidokg\",\"displayDescription\":\"jyoxgvclt\",\"unit\":\"sncghkjeszz\",\"aggregationType\":\"ijhtxf\",\"availabilities\":[{\"timeGrain\":\"fsm\",\"blobDuration\":\"eh\"}],\"supportedTimeGrainTypes\":[\"ec\",\"godebfqkkrbmpu\",\"gr\",\"wflzlfbxzpuzy\"],\"metricFilterPattern\":\"spnqzahmgkb\",\"fillGapWithZero\":true}]}}") + .toObject(OperationProperties.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + OperationProperties model = new OperationProperties().withServiceSpecification(new ServiceSpecification()); + model = BinaryData.fromObject(model).toObject(OperationProperties.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListMockTests.java new file mode 100644 index 0000000000000..a7b7f9f26b520 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/OperationsListMockTests.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.Operation; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class OperationsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"name\":\"wr\",\"display\":{\"provider\":\"twaenuuzko\",\"resource\":\"minrfdw\",\"operation\":\"uhhziuiefozbhdm\",\"description\":\"l\"},\"properties\":{\"serviceSpecification\":{\"logSpecifications\":[{\"name\":\"trmaequiahxicslf\",\"displayName\":\"qzpiyyl\",\"blobDuration\":\"lnswhcc\",\"categoryGroups\":[\"kaivwit\"]},{\"name\":\"cywuggwol\",\"displayName\":\"czbwemhairsbr\",\"blobDuration\":\"dwmsweypqwd\",\"categoryGroups\":[\"icccn\",\"qhuexm\",\"ttlstvlzywemhz\"]},{\"name\":\"csdtclusiypbs\",\"displayName\":\"ytguslf\",\"blobDuration\":\"dcygqukyhejhz\",\"categoryGroups\":[\"gfpelolppvksrpqv\",\"jzraehtwdwrf\",\"swibyr\"]}],\"metricSpecifications\":[{\"name\":\"h\",\"displayName\":\"fwpracstwi\",\"displayDescription\":\"khevxccedc\",\"unit\":\"md\",\"aggregationType\":\"dnwzxltjcvnhltiu\",\"availabilities\":[{},{}],\"supportedTimeGrainTypes\":[\"vvwxqi\",\"y\"],\"metricFilterPattern\":\"nyowxwlmdjrkvfg\",\"fillGapWithZero\":false}]}},\"isDataAction\":false}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionPropertiesTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionPropertiesTests.java new file mode 100644 index 0000000000000..4d7550af01b2c --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionPropertiesTests.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState; +import org.junit.jupiter.api.Assertions; + +public final class PrivateEndpointConnectionPropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PrivateEndpointConnectionProperties model = BinaryData.fromString( + "{\"provisioningState\":\"Deleting\",\"privateEndpoint\":{\"id\":\"opbobj\"},\"groupIds\":[\"m\"],\"privateLinkServiceConnectionState\":{\"status\":\"Disconnected\",\"description\":\"a\",\"actionsRequired\":\"rzayv\"}}") + .toObject(PrivateEndpointConnectionProperties.class); + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.DELETING, model.provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.DISCONNECTED, + model.privateLinkServiceConnectionState().status()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PrivateEndpointConnectionProperties model = new PrivateEndpointConnectionProperties() + .withProvisioningState(PrivateEndpointConnectionProvisioningState.DELETING) + .withPrivateEndpoint(new RemotePrivateEndpointConnection()).withPrivateLinkServiceConnectionState( + new RemotePrivateLinkServiceConnectionState().withStatus(PrivateLinkConnectionStatus.DISCONNECTED)); + model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionProperties.class); + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.DELETING, model.provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.DISCONNECTED, + model.privateLinkServiceConnectionState().status()); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceInnerTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceInnerTests.java new file mode 100644 index 0000000000000..00943ed1b7a59 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceInnerTests.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState; +import org.junit.jupiter.api.Assertions; + +public final class PrivateEndpointConnectionResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PrivateEndpointConnectionResourceInner model = BinaryData.fromString( + "{\"properties\":{\"provisioningState\":\"Succeeded\",\"privateEndpoint\":{\"id\":\"axcfjpgddtocjjx\"},\"groupIds\":[\"mouexhdzx\",\"bqe\",\"jnxqbzvddntwn\"],\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"twnpzaoqvuhrhcf\",\"actionsRequired\":\"yd\"}},\"id\":\"lmjthjq\",\"name\":\"wpyeicxmqciwqvh\",\"type\":\"hix\"}") + .toObject(PrivateEndpointConnectionResourceInner.class); + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.SUCCEEDED, + model.properties().provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.REJECTED, + model.properties().privateLinkServiceConnectionState().status()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PrivateEndpointConnectionResourceInner model + = new PrivateEndpointConnectionResourceInner().withProperties(new PrivateEndpointConnectionProperties() + .withProvisioningState(PrivateEndpointConnectionProvisioningState.SUCCEEDED) + .withPrivateEndpoint(new RemotePrivateEndpointConnection()).withPrivateLinkServiceConnectionState( + new RemotePrivateLinkServiceConnectionState().withStatus(PrivateLinkConnectionStatus.REJECTED))); + model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionResourceInner.class); + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.SUCCEEDED, + model.properties().provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.REJECTED, + model.properties().privateLinkServiceConnectionState().status()); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceListResultTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceListResultTests.java new file mode 100644 index 0000000000000..51855c090f16c --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionResourceListResultTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResourceListResult; + +public final class PrivateEndpointConnectionResourceListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PrivateEndpointConnectionResourceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"provisioningState\":\"Deleting\",\"privateEndpoint\":{\"id\":\"wzbtdhxu\"},\"groupIds\":[\"bm\",\"ow\",\"wpr\"],\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"ualupjmkh\",\"actionsRequired\":\"obbc\"}},\"id\":\"s\",\"name\":\"tjrip\",\"type\":\"rbpbewtghfgblcg\"},{\"properties\":{\"provisioningState\":\"Deleting\",\"privateEndpoint\":{\"id\":\"qhjk\"},\"groupIds\":[\"ibtnmxiebwwaloay\",\"cgwrtzjuz\"],\"privateLinkServiceConnectionState\":{\"status\":\"Disconnected\",\"description\":\"htxongmtsavjc\",\"actionsRequired\":\"wxqpsrknftguvri\"}},\"id\":\"prwmdyvxqt\",\"name\":\"yriwwroyqb\",\"type\":\"xrmcqibycnojvk\"},{\"properties\":{\"provisioningState\":\"Deleting\",\"privateEndpoint\":{\"id\":\"gzva\"},\"groupIds\":[\"jyzhpvgq\",\"cjrvxdjzlmwlxkv\"],\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"ovawjvzunlu\",\"actionsRequired\":\"nnprn\"}},\"id\":\"peilpjzuaejxdu\",\"name\":\"tskzbbtdzumveek\",\"type\":\"pwo\"},{\"properties\":{\"provisioningState\":\"Creating\",\"privateEndpoint\":{\"id\":\"bsjyofdx\"},\"groupIds\":[\"sd\"],\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"aboekqv\",\"actionsRequired\":\"lns\"}},\"id\":\"bxwyjsflhhcaa\",\"name\":\"n\",\"type\":\"ixisxyawjoy\"}],\"nextLink\":\"cslyjpk\"}") + .toObject(PrivateEndpointConnectionResourceListResult.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PrivateEndpointConnectionResourceListResult model = new PrivateEndpointConnectionResourceListResult(); + model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionResourceListResult.class); + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdWithResponseMockTests.java similarity index 66% rename from sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusWithResponseMockTests.java rename to sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdWithResponseMockTests.java index 318c605c27ba3..ea33b802550e4 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/test/java/com/azure/resourcemanager/netapp/generated/BackupsGetLatestStatusWithResponseMockTests.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetGroupIdWithResponseMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.generated; +package com.azure.resourcemanager.notificationhubs.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -11,8 +11,8 @@ import com.azure.core.http.HttpResponse; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.netapp.NetAppFilesManager; -import com.azure.resourcemanager.netapp.models.BackupStatus; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResource; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; @@ -22,15 +22,15 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class BackupsGetLatestStatusWithResponseMockTests { +public final class PrivateEndpointConnectionsGetGroupIdWithResponseMockTests { @Test - public void testGetLatestStatusWithResponse() throws Exception { + public void testGetGroupIdWithResponse() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"healthy\":false,\"relationshipStatus\":\"Idle\",\"mirrorState\":\"Broken\",\"unhealthyReason\":\"jnnawtqa\",\"errorMessage\":\"xuckpggqoweyir\",\"lastTransferSize\":4448034441017849158,\"lastTransferType\":\"ngwflqqmpizruwn\",\"totalTransferBytes\":2484776416149492622,\"transferProgressBytes\":363622537990763049}"; + = "{\"properties\":{\"groupId\":\"fcqdpsq\",\"requiredMembers\":[\"psvuoymgc\",\"elvezrypq\",\"mfe\"],\"requiredZoneNames\":[\"rqwky\",\"kobopgxed\",\"owepbqpcrfkb\",\"ccsnjvcdwxlpq\"]},\"id\":\"ftnkhtj\",\"name\":\"y\",\"type\":\"ngwfqatm\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -43,13 +43,12 @@ public void testGetLatestStatusWithResponse() throws Exception { return Mono.just(httpResponse); })); - NetAppFilesManager manager = NetAppFilesManager.configure().withHttpClient(httpClient).authenticate( + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - BackupStatus response = manager.backups() - .getLatestStatusWithResponse("nlb", "jkwrusnkq", "hsyrqunj", "hdenxaulk", com.azure.core.util.Context.NONE) - .getValue(); + PrivateLinkResource response = manager.privateEndpointConnections() + .getGroupIdWithResponse("hwyg", "lvdnkfx", "semdwzrmu", com.azure.core.util.Context.NONE).getValue(); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java new file mode 100644 index 0000000000000..0d8f13aef55a9 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResource; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class PrivateEndpointConnectionsGetWithResponseMockTests { + @Test + public void testGetWithResponse() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"provisioningState\":\"Deleted\",\"privateEndpoint\":{\"id\":\"wp\"},\"groupIds\":[\"wbtlhflsjcdh\",\"zfjvfbgofe\"],\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"qmqhldvriii\",\"actionsRequired\":\"nalghfkvtvsexso\"}},\"id\":\"el\",\"name\":\"qhhahhxvrhmzkwpj\",\"type\":\"wws\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PrivateEndpointConnectionResource response = manager.privateEndpointConnections() + .getWithResponse("okixrjqcir", "zpfrla", "szrnwo", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.DELETED, + response.properties().provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.APPROVED, + response.properties().privateLinkServiceConnectionState().status()); + } +} diff --git a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsMockTests.java similarity index 66% rename from sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainMockTests.java rename to sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsMockTests.java index df770f114f50c..71261772b5673 100644 --- a/sdk/communication/azure-resourcemanager-communication/src/test/java/com/azure/resourcemanager/communication/generated/SuppressionListsListByDomainMockTests.java +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListGroupIdsMockTests.java @@ -2,7 +2,7 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.communication.generated; +package com.azure.resourcemanager.notificationhubs.generated; import com.azure.core.credential.AccessToken; import com.azure.core.http.HttpClient; @@ -12,27 +12,26 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.management.AzureEnvironment; import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.communication.CommunicationManager; -import com.azure.resourcemanager.communication.models.SuppressionListResource; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResource; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -public final class SuppressionListsListByDomainMockTests { +public final class PrivateEndpointConnectionsListGroupIdsMockTests { @Test - public void testListByDomain() throws Exception { + public void testListGroupIds() throws Exception { HttpClient httpClient = Mockito.mock(HttpClient.class); HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); String responseStr - = "{\"value\":[{\"properties\":{\"listName\":\"zel\",\"lastUpdatedTimeStamp\":\"irels\",\"createdTimeStamp\":\"aenwabf\",\"dataLocation\":\"kl\"},\"id\":\"xbjhwuaanozjosph\",\"name\":\"oulpjrv\",\"type\":\"ag\"}]}"; + = "{\"value\":[{\"properties\":{\"groupId\":\"hkjoqr\",\"requiredMembers\":[\"aatjinrvgoupmfi\",\"bfggjioolvr\",\"x\"],\"requiredZoneNames\":[\"kkgll\",\"wjygvjayvblmhvk\",\"uhbxvvy\"]},\"id\":\"s\",\"name\":\"pbyrqufegxu\",\"type\":\"wz\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); @@ -45,13 +44,12 @@ public void testListByDomain() throws Exception { return Mono.just(httpResponse); })); - CommunicationManager manager = CommunicationManager.configure().withHttpClient(httpClient).authenticate( + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = manager.suppressionLists().listByDomain("szkkfoqre", - "fkzikfj", "wneaiv", com.azure.core.util.Context.NONE); + PagedIterable response = manager.privateEndpointConnections().listGroupIds("dhtmdvypgikd", + "szywkbirryu", com.azure.core.util.Context.NONE); - Assertions.assertEquals("zel", response.iterator().next().listName()); } } diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListMockTests.java new file mode 100644 index 0000000000000..20731fa29fa94 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsListMockTests.java @@ -0,0 +1,62 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResource; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class PrivateEndpointConnectionsListMockTests { + @Test + public void testList() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"value\":[{\"properties\":{\"provisioningState\":\"Updating\",\"privateEndpoint\":{\"id\":\"ufzdmsyq\"},\"groupIds\":[\"hwhbotzingamv\",\"phoszqz\",\"dphqamv\",\"kfwynw\"],\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"kayh\",\"actionsRequired\":\"nvyq\"}},\"id\":\"tkzwpcnpwzc\",\"name\":\"a\",\"type\":\"sgvvsccyajguq\"}]}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.privateEndpointConnections().list("ughftqsx", "qxujxukndxd", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.UPDATING, + response.iterator().next().properties().provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.PENDING, + response.iterator().next().properties().privateLinkServiceConnectionState().status()); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateMockTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateMockTests.java new file mode 100644 index 0000000000000..a690c61895de2 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateEndpointConnectionsUpdateMockTests.java @@ -0,0 +1,70 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.notificationhubs.NotificationHubsManager; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateEndpointConnectionResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProperties; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionProvisioningState; +import com.azure.resourcemanager.notificationhubs.models.PrivateEndpointConnectionResource; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class PrivateEndpointConnectionsUpdateMockTests { + @Test + public void testUpdate() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr + = "{\"properties\":{\"provisioningState\":\"Succeeded\",\"privateEndpoint\":{\"id\":\"hqlbjbsybbq\"},\"groupIds\":[\"t\",\"dgmfpgvmpipasl\",\"haq\",\"x\"],\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"twbdsrezpdrhn\",\"actionsRequired\":\"yowqkdwytisibir\"}},\"id\":\"pikpz\",\"name\":\"mejzanlfzxia\",\"type\":\"rmbzo\"}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + NotificationHubsManager manager = NotificationHubsManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PrivateEndpointConnectionResource response + = manager.privateEndpointConnections().update("dbodacizsjqlhkrr", "bdeibqipqk", "hvxndzwmkrefajpj", + new PrivateEndpointConnectionResourceInner().withProperties(new PrivateEndpointConnectionProperties() + .withProvisioningState(PrivateEndpointConnectionProvisioningState.CREATING) + .withPrivateEndpoint(new RemotePrivateEndpointConnection()).withPrivateLinkServiceConnectionState( + new RemotePrivateLinkServiceConnectionState().withStatus(PrivateLinkConnectionStatus.PENDING))), + com.azure.core.util.Context.NONE); + + Assertions.assertEquals(PrivateEndpointConnectionProvisioningState.SUCCEEDED, + response.properties().provisioningState()); + Assertions.assertEquals(PrivateLinkConnectionStatus.APPROVED, + response.properties().privateLinkServiceConnectionState().status()); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceInnerTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceInnerTests.java new file mode 100644 index 0000000000000..52b365aa7ca52 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceInnerTests.java @@ -0,0 +1,25 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.fluent.models.PrivateLinkResourceInner; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceProperties; + +public final class PrivateLinkResourceInnerTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PrivateLinkResourceInner model = BinaryData.fromString( + "{\"properties\":{\"groupId\":\"zyexzn\",\"requiredMembers\":[\"xhnrztfolhb\",\"xknalaulppg\"],\"requiredZoneNames\":[\"pnapnyiropuh\",\"igvpgylg\",\"git\",\"medjvcslynqwwncw\"]},\"id\":\"hxg\",\"name\":\"trmgucnapkte\",\"type\":\"ellwptfdy\"}") + .toObject(PrivateLinkResourceInner.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PrivateLinkResourceInner model + = new PrivateLinkResourceInner().withProperties(new PrivateLinkResourceProperties()); + model = BinaryData.fromObject(model).toObject(PrivateLinkResourceInner.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceListResultTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceListResultTests.java new file mode 100644 index 0000000000000..d061ed16c5128 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourceListResultTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceListResult; + +public final class PrivateLinkResourceListResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PrivateLinkResourceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"groupId\":\"jhhkxbp\",\"requiredMembers\":[\"mjh\",\"xjyngudivk\",\"tswb\",\"qzvszjf\"],\"requiredZoneNames\":[\"j\",\"dxxiv\"]},\"id\":\"vtcqaqtdo\",\"name\":\"mcbxvwvxysl\",\"type\":\"bhsfxob\"},{\"properties\":{\"groupId\":\"k\",\"requiredMembers\":[\"pe\"],\"requiredZoneNames\":[\"fbkrvrnsvs\",\"q\",\"ohxcrsbfova\"]},\"id\":\"ruvw\",\"name\":\"hsqfsubcgjbirxbp\",\"type\":\"bsrfbj\"},{\"properties\":{\"groupId\":\"w\",\"requiredMembers\":[\"t\",\"tpvjzbexilzznfqq\",\"vwpm\",\"taruoujmkcj\"],\"requiredZoneNames\":[\"ytjrybnwjewgdr\",\"ervnaenqpehi\",\"doy\",\"mifthnzdnd\"]},\"id\":\"gnayqigynduh\",\"name\":\"vhqlkthumaqo\",\"type\":\"bgycduiertgccym\"}],\"nextLink\":\"olpsslqlf\"}") + .toObject(PrivateLinkResourceListResult.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PrivateLinkResourceListResult model = new PrivateLinkResourceListResult(); + model = BinaryData.fromObject(model).toObject(PrivateLinkResourceListResult.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourcePropertiesTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourcePropertiesTests.java new file mode 100644 index 0000000000000..b2cf4431ef970 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PrivateLinkResourcePropertiesTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkResourceProperties; + +public final class PrivateLinkResourcePropertiesTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PrivateLinkResourceProperties model = BinaryData.fromString( + "{\"groupId\":\"fqbuaceopzf\",\"requiredMembers\":[\"huaoppp\",\"qeqxo\"],\"requiredZoneNames\":[\"ahzxctobgbk\",\"moizpos\",\"mgrcfbu\"]}") + .toObject(PrivateLinkResourceProperties.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PrivateLinkResourceProperties model = new PrivateLinkResourceProperties(); + model = BinaryData.fromObject(model).toObject(PrivateLinkResourceProperties.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PublicInternetAuthorizationRuleTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PublicInternetAuthorizationRuleTests.java new file mode 100644 index 0000000000000..f068e46a37a9d --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/PublicInternetAuthorizationRuleTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.AccessRights; +import com.azure.resourcemanager.notificationhubs.models.PublicInternetAuthorizationRule; +import java.util.Arrays; +import org.junit.jupiter.api.Assertions; + +public final class PublicInternetAuthorizationRuleTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + PublicInternetAuthorizationRule model = BinaryData.fromString("{\"rights\":[\"Manage\",\"Manage\"]}") + .toObject(PublicInternetAuthorizationRule.class); + Assertions.assertEquals(AccessRights.MANAGE, model.rights().get(0)); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + PublicInternetAuthorizationRule model + = new PublicInternetAuthorizationRule().withRights(Arrays.asList(AccessRights.MANAGE, AccessRights.MANAGE)); + model = BinaryData.fromObject(model).toObject(PublicInternetAuthorizationRule.class); + Assertions.assertEquals(AccessRights.MANAGE, model.rights().get(0)); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RegistrationResultTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RegistrationResultTests.java new file mode 100644 index 0000000000000..a5cfc318af68d --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RegistrationResultTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.RegistrationResult; + +public final class RegistrationResultTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RegistrationResult model = BinaryData.fromString( + "{\"applicationPlatform\":\"wbxgjvt\",\"pnsHandle\":\"p\",\"registrationId\":\"szdnr\",\"outcome\":\"qguhmuo\"}") + .toObject(RegistrationResult.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RegistrationResult model = new RegistrationResult(); + model = BinaryData.fromObject(model).toObject(RegistrationResult.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateEndpointConnectionTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateEndpointConnectionTests.java new file mode 100644 index 0000000000000..b51be4bbbc417 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateEndpointConnectionTests.java @@ -0,0 +1,22 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateEndpointConnection; + +public final class RemotePrivateEndpointConnectionTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RemotePrivateEndpointConnection model + = BinaryData.fromString("{\"id\":\"pgvdf\"}").toObject(RemotePrivateEndpointConnection.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RemotePrivateEndpointConnection model = new RemotePrivateEndpointConnection(); + model = BinaryData.fromObject(model).toObject(RemotePrivateEndpointConnection.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateLinkServiceConnectionStateTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateLinkServiceConnectionStateTests.java new file mode 100644 index 0000000000000..1610a103d41b4 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/RemotePrivateLinkServiceConnectionStateTests.java @@ -0,0 +1,28 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.PrivateLinkConnectionStatus; +import com.azure.resourcemanager.notificationhubs.models.RemotePrivateLinkServiceConnectionState; +import org.junit.jupiter.api.Assertions; + +public final class RemotePrivateLinkServiceConnectionStateTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + RemotePrivateLinkServiceConnectionState model = BinaryData + .fromString("{\"status\":\"Rejected\",\"description\":\"kftutqxlngxlefg\",\"actionsRequired\":\"nxkrx\"}") + .toObject(RemotePrivateLinkServiceConnectionState.class); + Assertions.assertEquals(PrivateLinkConnectionStatus.REJECTED, model.status()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + RemotePrivateLinkServiceConnectionState model + = new RemotePrivateLinkServiceConnectionState().withStatus(PrivateLinkConnectionStatus.REJECTED); + model = BinaryData.fromObject(model).toObject(RemotePrivateLinkServiceConnectionState.class); + Assertions.assertEquals(PrivateLinkConnectionStatus.REJECTED, model.status()); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/ServiceSpecificationTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/ServiceSpecificationTests.java new file mode 100644 index 0000000000000..a64d0a9445499 --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/ServiceSpecificationTests.java @@ -0,0 +1,23 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.ServiceSpecification; + +public final class ServiceSpecificationTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + ServiceSpecification model = BinaryData.fromString( + "{\"logSpecifications\":[{\"name\":\"ibnuqqkpik\",\"displayName\":\"rgvtqag\",\"blobDuration\":\"uynhijg\",\"categoryGroups\":[\"bfs\"]},{\"name\":\"rbu\",\"displayName\":\"cvpnazzmhjrunmpx\",\"blobDuration\":\"dbhrbnlankxm\",\"categoryGroups\":[\"pbh\"]}],\"metricSpecifications\":[{\"name\":\"kcxywnyt\",\"displayName\":\"synlqidybyxczfc\",\"displayDescription\":\"aaxdbabphlwrq\",\"unit\":\"ktsthsucocmny\",\"aggregationType\":\"zt\",\"availabilities\":[{\"timeGrain\":\"wrqpue\",\"blobDuration\":\"kzywbiex\"},{\"timeGrain\":\"eyueaxibxujwb\",\"blobDuration\":\"walm\"}],\"supportedTimeGrainTypes\":[\"oxaepd\"],\"metricFilterPattern\":\"jancu\",\"fillGapWithZero\":true},{\"name\":\"wbavxbniwdj\",\"displayName\":\"zt\",\"displayDescription\":\"bpg\",\"unit\":\"ytxhp\",\"aggregationType\":\"bzpfzab\",\"availabilities\":[{\"timeGrain\":\"hxw\",\"blobDuration\":\"tyq\"}],\"supportedTimeGrainTypes\":[\"bbovplwzbhvgyugu\",\"svmkfssxquk\"],\"metricFilterPattern\":\"plgmgsxnk\",\"fillGapWithZero\":false},{\"name\":\"es\",\"displayName\":\"vlopwiyighx\",\"displayDescription\":\"dwzbaiue\",\"unit\":\"a\",\"aggregationType\":\"nyqupedeojnabck\",\"availabilities\":[{\"timeGrain\":\"xpsiebtfhvpes\",\"blobDuration\":\"skrdqmhjj\"},{\"timeGrain\":\"tldwkyzxuutk\",\"blobDuration\":\"ws\"},{\"timeGrain\":\"svlxotogtwrup\",\"blobDuration\":\"xvnmicykvceov\"}],\"supportedTimeGrainTypes\":[\"ovnotyfjfcnjbkcn\"],\"metricFilterPattern\":\"hbttkphyw\",\"fillGapWithZero\":true},{\"name\":\"t\",\"displayName\":\"nermcl\",\"displayDescription\":\"lphox\",\"unit\":\"crpab\",\"aggregationType\":\"epsbjtazqu\",\"availabilities\":[{\"timeGrain\":\"pmueefjzwfqk\",\"blobDuration\":\"jidsuyonobglaoc\"},{\"timeGrain\":\"tcc\",\"blobDuration\":\"yudxytlmoy\"},{\"timeGrain\":\"vwfudwpzntxhd\",\"blobDuration\":\"lrqjbhckfr\"},{\"timeGrain\":\"rxsbkyvp\",\"blobDuration\":\"anuzbpzkafkuw\"}],\"supportedTimeGrainTypes\":[\"nwbmeh\"],\"metricFilterPattern\":\"eyvjusrtslhspkde\",\"fillGapWithZero\":true}]}") + .toObject(ServiceSpecification.class); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + ServiceSpecification model = new ServiceSpecification(); + model = BinaryData.fromObject(model).toObject(ServiceSpecification.class); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/SkuTests.java b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/SkuTests.java new file mode 100644 index 0000000000000..5643a925da0fb --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/java/com/azure/resourcemanager/notificationhubs/generated/SkuTests.java @@ -0,0 +1,37 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.notificationhubs.generated; + +import com.azure.core.util.BinaryData; +import com.azure.resourcemanager.notificationhubs.models.Sku; +import com.azure.resourcemanager.notificationhubs.models.SkuName; +import org.junit.jupiter.api.Assertions; + +public final class SkuTests { + @org.junit.jupiter.api.Test + public void testDeserialize() throws Exception { + Sku model = BinaryData + .fromString( + "{\"name\":\"Basic\",\"tier\":\"qes\",\"size\":\"gdf\",\"family\":\"lzl\",\"capacity\":1040747911}") + .toObject(Sku.class); + Assertions.assertEquals(SkuName.BASIC, model.name()); + Assertions.assertEquals("qes", model.tier()); + Assertions.assertEquals("gdf", model.size()); + Assertions.assertEquals("lzl", model.family()); + Assertions.assertEquals(1040747911, model.capacity()); + } + + @org.junit.jupiter.api.Test + public void testSerialize() throws Exception { + Sku model = new Sku().withName(SkuName.BASIC).withTier("qes").withSize("gdf").withFamily("lzl") + .withCapacity(1040747911); + model = BinaryData.fromObject(model).toObject(Sku.class); + Assertions.assertEquals(SkuName.BASIC, model.name()); + Assertions.assertEquals("qes", model.tier()); + Assertions.assertEquals("gdf", model.size()); + Assertions.assertEquals("lzl", model.family()); + Assertions.assertEquals(1040747911, model.capacity()); + } +} diff --git a/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000000..1f0955d450f0d --- /dev/null +++ b/sdk/notificationhubs/azure-resourcemanager-notificationhubs/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1 @@ +mock-maker-inline diff --git a/sdk/openai/azure-ai-openai-assistants/pom.xml b/sdk/openai/azure-ai-openai-assistants/pom.xml index 016a390a0bfa1..323f817c260f8 100644 --- a/sdk/openai/azure-ai-openai-assistants/pom.xml +++ b/sdk/openai/azure-ai-openai-assistants/pom.xml @@ -83,7 +83,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/openai/azure-ai-openai/CHANGELOG.md b/sdk/openai/azure-ai-openai/CHANGELOG.md index 5a327d509d9c4..c114743ceec45 100644 --- a/sdk/openai/azure-ai-openai/CHANGELOG.md +++ b/sdk/openai/azure-ai-openai/CHANGELOG.md @@ -4,6 +4,9 @@ ### Features Added +- Added a new overload `getChatCompletionsStreamWithResponse` that takes `RequestOptions` to provide the flexibility to + modify the HTTP request. + ### Breaking Changes ### Bugs Fixed diff --git a/sdk/openai/azure-ai-openai/README.md b/sdk/openai/azure-ai-openai/README.md index 65db7a74cb05b..7a9f97fbb0f6e 100644 --- a/sdk/openai/azure-ai-openai/README.md +++ b/sdk/openai/azure-ai-openai/README.md @@ -234,24 +234,20 @@ chatMessages.add(new ChatRequestUserMessage("Can you help me?")); chatMessages.add(new ChatRequestAssistantMessage("Of course, me hearty! What can I do for ye?")); chatMessages.add(new ChatRequestUserMessage("What's the best way to train a parrot?")); -IterableStream chatCompletionsStream = client.getChatCompletionsStream("{deploymentOrModelName}", - new ChatCompletionsOptions(chatMessages)); - -chatCompletionsStream - .stream() - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) when service fix the issue. - .skip(1) - .forEach(chatCompletions -> { - ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); - if (delta.getRole() != null) { - System.out.println("Role = " + delta.getRole()); - } - if (delta.getContent() != null) { - System.out.print(delta.getContent()); - } - }); +client.getChatCompletionsStream("{deploymentOrModelName}", new ChatCompletionsOptions(chatMessages)) + .forEach(chatCompletions -> { + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return; + } + ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); + if (delta.getRole() != null) { + System.out.println("Role = " + delta.getRole()); + } + if (delta.getContent() != null) { + String content = delta.getContent(); + System.out.print(content); + } + }); ``` To compute tokens in streaming chat completions, see sample [Streaming Chat Completions][sample_get_chat_completions_streaming]. diff --git a/sdk/openai/azure-ai-openai/assets.json b/sdk/openai/azure-ai-openai/assets.json index 32b4adc79a596..c497cb6ddb36f 100644 --- a/sdk/openai/azure-ai-openai/assets.json +++ b/sdk/openai/azure-ai-openai/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/openai/azure-ai-openai", - "Tag": "java/openai/azure-ai-openai_915389e465" + "Tag": "java/openai/azure-ai-openai_76031b0cb0" } diff --git a/sdk/openai/azure-ai-openai/pom.xml b/sdk/openai/azure-ai-openai/pom.xml index fbe634fdbd1d2..64b678f710147 100644 --- a/sdk/openai/azure-ai-openai/pom.xml +++ b/sdk/openai/azure-ai-openai/pom.xml @@ -86,7 +86,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIAsyncClient.java b/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIAsyncClient.java index e64a406382b43..07d214cb56c83 100644 --- a/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIAsyncClient.java +++ b/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIAsyncClient.java @@ -46,6 +46,7 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.core.JsonProcessingException; import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicReference; import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; @@ -556,6 +557,57 @@ public Mono> getChatCompletionsWithResponse(String dep .map(response -> new SimpleResponse<>(response, response.getValue().toObject(ChatCompletions.class))); } + /** + * Gets chat completions for the provided chat messages. Chat completions support a wide variety of tasks and + * generate text that continues from or "completes" provided prompt data. + * + *

+ * Code Samples + *

+ * + *
+     * openAIAsyncClient.getChatCompletionsStreamWithResponse(deploymentOrModelId, new ChatCompletionsOptions(chatMessages),
+     *                 new RequestOptions().setHeader("my-header", "my-header-value"))
+     *         .subscribe(
+     *                 response -> System.out.print(response.getValue().getId()),
+     *                 error -> System.err.println("There was an error getting chat completions." + error),
+     *                 () -> System.out.println("Completed called getChatCompletionsStreamWithResponse."));
+     * 
+ * + * + * @param deploymentOrModelName Specifies either the model deployment name (when using Azure OpenAI) or model name + * (when using non-Azure OpenAI) to use for this request. + * @param chatCompletionsOptions The configuration information for a chat completions request. Completions support a + * wide variety of tasks and generate text that continues from or "completes" provided prompt data. + * @param requestOptions The options to configure the HTTP request before HTTP client sends it. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws HttpResponseException thrown if the request is rejected by server. + * @throws ClientAuthenticationException thrown if the request is rejected by server on status code 401. + * @throws ResourceNotFoundException thrown if the request is rejected by server on status code 404. + * @throws ResourceModifiedException thrown if the request is rejected by server on status code 409. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return chat completions stream for the provided chat messages. Completions support a wide variety of tasks and + * generate text that continues from or "completes" provided prompt data. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public Flux> getChatCompletionsStreamWithResponse(String deploymentOrModelName, + ChatCompletionsOptions chatCompletionsOptions, RequestOptions requestOptions) { + chatCompletionsOptions.setStream(true); + Mono> chatCompletionsWithResponse = getChatCompletionsWithResponse(deploymentOrModelName, + BinaryData.fromObject(chatCompletionsOptions), requestOptions); + AtomicReference> responseCopy = new AtomicReference<>(); + Flux responseStream = chatCompletionsWithResponse.flatMapMany(response -> { + responseCopy.set(response); + return response.getValue().toFluxByteBuffer(); + }); + OpenAIServerSentEvents chatCompletionsStream + = new OpenAIServerSentEvents<>(responseStream, ChatCompletions.class); + return chatCompletionsStream.getEvents() + .map(chatCompletions -> new SimpleResponse<>(responseCopy.get(), chatCompletions)); + } + /** * Return the embeddings for a given prompt. * @@ -646,21 +698,10 @@ public Mono getCompletions(String deploymentOrModelName, String pro *
      * openAIAsyncClient
      *         .getChatCompletionsStream(deploymentOrModelId, new ChatCompletionsOptions(chatMessages))
-     *         .toStream()
-     *         // Remove .skip(1) when using Non-Azure OpenAI API
-     *         // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug.
-     *         // TODO: remove .skip(1) after service fixes the issue.
-     *         .skip(1)
-     *         .forEach(chatCompletions -> {
-     *             ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta();
-     *             if (delta.getRole() != null) {
-     *                 System.out.println("Role = " + delta.getRole());
-     *             }
-     *             if (delta.getContent() != null) {
-     *                 String content = delta.getContent();
-     *                 System.out.print(content);
-     *             }
-     *         });
+     *         .subscribe(
+     *                 chatCompletions -> System.out.print(chatCompletions.getId()),
+     *                 error -> System.err.println("There was an error getting chat completions." + error),
+     *                 () -> System.out.println("Completed called getChatCompletionsStream."));
      * 
* * diff --git a/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIClient.java b/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIClient.java index 4505e47d41368..4d4f8c01907dc 100644 --- a/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIClient.java +++ b/sdk/openai/azure-ai-openai/src/main/java/com/azure/ai/openai/OpenAIClient.java @@ -689,12 +689,10 @@ public ChatCompletions getChatCompletions(String deploymentOrModelName, * *
      * openAIClient.getChatCompletionsStream(deploymentOrModelId, new ChatCompletionsOptions(chatMessages))
-     *         .stream()
-     *         // Remove .skip(1) when using Non-Azure OpenAI API
-     *         // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug.
-     *         // TODO: remove .skip(1) after service fixes the issue.
-     *         .skip(1)
      *         .forEach(chatCompletions -> {
+     *             if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) {
+     *                 return;
+     *             }
      *             ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta();
      *             if (delta.getRole() != null) {
      *                 System.out.println("Role = " + delta.getRole());
@@ -732,6 +730,60 @@ public IterableStream getChatCompletionsStream(String deploymen
         return new IterableStream<>(chatCompletionsStream.getEvents());
     }
 
+    /**
+     * Gets chat completions for the provided chat messages in streaming mode. Chat completions support a wide variety
+     * of tasks and generate text that continues from or "completes" provided prompt data.
+     * 

+ * Code Samples + *

+ * + *
+     * openAIClient.getChatCompletionsStreamWithResponse(deploymentOrModelId, new ChatCompletionsOptions(chatMessages),
+     *                 new RequestOptions().setHeader("my-header", "my-header-value"))
+     *         .getValue()
+     *         .forEach(chatCompletions -> {
+     *             if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) {
+     *                 return;
+     *             }
+     *             ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta();
+     *             if (delta.getRole() != null) {
+     *                 System.out.println("Role = " + delta.getRole());
+     *             }
+     *             if (delta.getContent() != null) {
+     *                 String content = delta.getContent();
+     *                 System.out.print(content);
+     *             }
+     *         });
+     * 
+ * + * + * @param deploymentOrModelName Specifies either the model deployment name (when using Azure OpenAI) or model name + * (when using non-Azure OpenAI) to use for this request. + * @param chatCompletionsOptions The configuration information for a chat completions request. Completions support a + * wide variety of tasks and generate text that continues from or "completes" provided prompt data. + * @param requestOptions The options to configure the HTTP request before HTTP client sends it. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws HttpResponseException thrown if the request is rejected by server. + * @throws ClientAuthenticationException thrown if the request is rejected by server on status code 401. + * @throws ResourceNotFoundException thrown if the request is rejected by server on status code 404. + * @throws ResourceModifiedException thrown if the request is rejected by server on status code 409. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return chat completions stream for the provided chat messages. Completions support a wide variety of tasks and + * generate text that continues from or "completes" provided prompt data. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public Response> getChatCompletionsStreamWithResponse(String deploymentOrModelName, + ChatCompletionsOptions chatCompletionsOptions, RequestOptions requestOptions) { + chatCompletionsOptions.setStream(true); + Response response = getChatCompletionsWithResponse(deploymentOrModelName, + BinaryData.fromObject(chatCompletionsOptions), requestOptions); + Flux responseStream = response.getValue().toFluxByteBuffer(); + OpenAIServerSentEvents chatCompletionsStream + = new OpenAIServerSentEvents<>(responseStream, ChatCompletions.class); + return new SimpleResponse<>(response, new IterableStream<>(chatCompletionsStream.getEvents())); + } + /** * Gets transcribed text and associated metadata from provided spoken audio file data. Audio will be transcribed in * the written language corresponding to the language it was spoken in. diff --git a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/StreamingChatSample.java b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/StreamingChatSample.java index b9f1ea4bef4cb..96bd4fddd781a 100644 --- a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/StreamingChatSample.java +++ b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/StreamingChatSample.java @@ -12,6 +12,7 @@ import com.azure.ai.openai.models.ChatResponseMessage; import com.azure.core.credential.AzureKeyCredential; import com.azure.core.util.Configuration; +import com.azure.core.util.CoreUtils; import com.azure.core.util.IterableStream; import com.knuddels.jtokkit.Encodings; import com.knuddels.jtokkit.api.Encoding; @@ -73,11 +74,11 @@ public static void main(String[] args) { // } chatCompletionsStream .stream() - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) after service fixes the issue. - .skip(1) .forEach(chatCompletions -> { + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return; + } + ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); if (delta.getRole() != null) { diff --git a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIAsyncClientJavaDocCodeSnippets.java b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIAsyncClientJavaDocCodeSnippets.java index 3a93b4d470b48..edfee9abca5cb 100644 --- a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIAsyncClientJavaDocCodeSnippets.java +++ b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIAsyncClientJavaDocCodeSnippets.java @@ -11,13 +11,14 @@ import com.azure.ai.openai.models.ChatRequestMessage; import com.azure.ai.openai.models.ChatRequestSystemMessage; import com.azure.ai.openai.models.ChatRequestUserMessage; -import com.azure.ai.openai.models.ChatResponseMessage; import com.azure.core.credential.AzureKeyCredential; +import com.azure.core.http.rest.RequestOptions; import com.azure.core.util.Configuration; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.TimeUnit; /** * Code snippets for {@link OpenAIAsyncClient} @@ -29,32 +30,35 @@ public class OpenAIAsyncClientJavaDocCodeSnippets { * Code snippets for {@link OpenAIClient#getChatCompletionsStream(String, ChatCompletionsOptions)} */ @Test - public void getChatCompletionsStream() { - String deploymentOrModelId = "gpt-4-1106-preview"; + public void getChatCompletionsStream() throws InterruptedException { + String deploymentOrModelId = Configuration.getGlobalConfiguration().get("OPENAI_DEPLOYMENT_OR_MODEL_ID"); List chatMessages = new ArrayList<>(); chatMessages.add(new ChatRequestSystemMessage("You are a helpful assistant. You will talk like a pirate.")); chatMessages.add(new ChatRequestUserMessage("Can you help me?")); chatMessages.add(new ChatRequestAssistantMessage("Of course, me hearty! What can I do for ye?")); chatMessages.add(new ChatRequestUserMessage("What's the best way to train a parrot?")); + // BEGIN: com.azure.ai.openai.OpenAIAsyncClient.getChatCompletionsStream#String-ChatCompletionsOptions openAIAsyncClient .getChatCompletionsStream(deploymentOrModelId, new ChatCompletionsOptions(chatMessages)) - .toStream() - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) after service fixes the issue. - .skip(1) - .forEach(chatCompletions -> { - ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); - if (delta.getRole() != null) { - System.out.println("Role = " + delta.getRole()); - } - if (delta.getContent() != null) { - String content = delta.getContent(); - System.out.print(content); - } - }); + .subscribe( + chatCompletions -> System.out.print(chatCompletions.getId()), + error -> System.err.println("There was an error getting chat completions." + error), + () -> System.out.println("Completed called getChatCompletionsStream.")); // END: com.azure.ai.openai.OpenAIAsyncClient.getChatCompletionsStream#String-ChatCompletionsOptions + + // With Response Code Snippet + + // BEGIN: com.azure.ai.openai.OpenAIAsyncClient.getChatCompletionsStream#String-ChatCompletionsOptionsMaxOverload + openAIAsyncClient.getChatCompletionsStreamWithResponse(deploymentOrModelId, new ChatCompletionsOptions(chatMessages), + new RequestOptions().setHeader("my-header", "my-header-value")) + .subscribe( + response -> System.out.print(response.getValue().getId()), + error -> System.err.println("There was an error getting chat completions." + error), + () -> System.out.println("Completed called getChatCompletionsStreamWithResponse.")); + // END: com.azure.ai.openai.OpenAIAsyncClient.getChatCompletionsStream#String-ChatCompletionsOptionsMaxOverload + + TimeUnit.SECONDS.sleep(10); } private OpenAIAsyncClient getOpenAIAsyncClient() { diff --git a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIClientJavaDocCodeSnippets.java b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIClientJavaDocCodeSnippets.java index 4c0edc428a0a9..ad4a2a65aae98 100644 --- a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIClientJavaDocCodeSnippets.java +++ b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/OpenAIClientJavaDocCodeSnippets.java @@ -12,7 +12,9 @@ import com.azure.ai.openai.models.ChatRequestUserMessage; import com.azure.ai.openai.models.ChatResponseMessage; import com.azure.core.credential.AzureKeyCredential; +import com.azure.core.http.rest.RequestOptions; import com.azure.core.util.Configuration; +import com.azure.core.util.CoreUtils; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -30,7 +32,7 @@ public class OpenAIClientJavaDocCodeSnippets { */ @Test public void getChatCompletionsStream() { - String deploymentOrModelId = "gpt-4-1106-preview"; + String deploymentOrModelId = Configuration.getGlobalConfiguration().get("OPENAI_DEPLOYMENT_OR_MODEL_ID"); List chatMessages = new ArrayList<>(); chatMessages.add(new ChatRequestSystemMessage("You are a helpful assistant. You will talk like a pirate.")); chatMessages.add(new ChatRequestUserMessage("Can you help me?")); @@ -39,12 +41,10 @@ public void getChatCompletionsStream() { // BEGIN: com.azure.ai.openai.OpenAIClient.getChatCompletionsStream#String-ChatCompletionsOptions openAIClient.getChatCompletionsStream(deploymentOrModelId, new ChatCompletionsOptions(chatMessages)) - .stream() - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) after service fixes the issue. - .skip(1) .forEach(chatCompletions -> { + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return; + } ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); if (delta.getRole() != null) { System.out.println("Role = " + delta.getRole()); @@ -55,6 +55,27 @@ public void getChatCompletionsStream() { } }); // END: com.azure.ai.openai.OpenAIClient.getChatCompletionsStream#String-ChatCompletionsOptions + + // With Response Code Snippet + + // BEGIN: com.azure.ai.openai.OpenAIClient.getChatCompletionsStream#String-ChatCompletionsOptionsMaxOverload + openAIClient.getChatCompletionsStreamWithResponse(deploymentOrModelId, new ChatCompletionsOptions(chatMessages), + new RequestOptions().setHeader("my-header", "my-header-value")) + .getValue() + .forEach(chatCompletions -> { + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return; + } + ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); + if (delta.getRole() != null) { + System.out.println("Role = " + delta.getRole()); + } + if (delta.getContent() != null) { + String content = delta.getContent(); + System.out.print(content); + } + }); + // END: com.azure.ai.openai.OpenAIClient.getChatCompletionsStream#String-ChatCompletionsOptionsMaxOverload } private OpenAIClient getOpenAIClient() { diff --git a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/ReadmeSamples.java b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/ReadmeSamples.java index f4d3775166f8d..7e98aea002c19 100644 --- a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/ReadmeSamples.java +++ b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/impl/ReadmeSamples.java @@ -46,6 +46,7 @@ import com.azure.core.credential.TokenCredential; import com.azure.core.http.ProxyOptions; import com.azure.core.util.BinaryData; +import com.azure.core.util.CoreUtils; import com.azure.core.util.HttpClientOptions; import com.azure.core.util.IterableStream; import com.azure.identity.DefaultAzureCredentialBuilder; @@ -190,24 +191,20 @@ public void getChatCompletionsStream() { chatMessages.add(new ChatRequestAssistantMessage("Of course, me hearty! What can I do for ye?")); chatMessages.add(new ChatRequestUserMessage("What's the best way to train a parrot?")); - IterableStream chatCompletionsStream = client.getChatCompletionsStream("{deploymentOrModelName}", - new ChatCompletionsOptions(chatMessages)); - - chatCompletionsStream - .stream() - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) when service fix the issue. - .skip(1) - .forEach(chatCompletions -> { - ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); - if (delta.getRole() != null) { - System.out.println("Role = " + delta.getRole()); - } - if (delta.getContent() != null) { - System.out.print(delta.getContent()); - } - }); + client.getChatCompletionsStream("{deploymentOrModelName}", new ChatCompletionsOptions(chatMessages)) + .forEach(chatCompletions -> { + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return; + } + ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); + if (delta.getRole() != null) { + System.out.println("Role = " + delta.getRole()); + } + if (delta.getContent() != null) { + String content = delta.getContent(); + System.out.print(content); + } + }); // END: readme-sample-getChatCompletionsStream } diff --git a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamAsyncSample.java b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamAsyncSample.java index 8b02b9194e819..80afe09f7596f 100644 --- a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamAsyncSample.java +++ b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamAsyncSample.java @@ -12,6 +12,8 @@ import com.azure.ai.openai.models.ChatRequestUserMessage; import com.azure.ai.openai.models.ChatResponseMessage; import com.azure.core.credential.AzureKeyCredential; +import com.azure.core.util.Configuration; +import com.azure.core.util.CoreUtils; import java.util.ArrayList; import java.util.List; @@ -32,9 +34,9 @@ public class GetChatCompletionsStreamAsyncSample { * @param args Unused. Arguments to the program. */ public static void main(String[] args) throws InterruptedException { - String azureOpenaiKey = "{azure-open-ai-key}"; - String endpoint = "{azure-open-ai-endpoint}"; - String deploymentOrModelId = "{azure-open-ai-deployment-model-id}"; + String azureOpenaiKey = Configuration.getGlobalConfiguration().get("AZURE_OPENAI_KEY"); + String endpoint = Configuration.getGlobalConfiguration().get("AZURE_OPENAI_ENDPOINT"); + String deploymentOrModelId = Configuration.getGlobalConfiguration().get("OPENAI_DEPLOYMENT_OR_MODEL_ID"); OpenAIAsyncClient client = new OpenAIClientBuilder() .endpoint(endpoint) @@ -48,10 +50,6 @@ public static void main(String[] args) throws InterruptedException { chatMessages.add(new ChatRequestUserMessage("What's the best way to train a parrot?")); client.getChatCompletionsStream(deploymentOrModelId, new ChatCompletionsOptions(chatMessages)) - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) when service fix the issue. - .skip(1) .map(chatCompletions -> { /* The delta is the message content for a streaming response. * Subsequence of streaming delta will be like: @@ -68,6 +66,11 @@ public static void main(String[] args) throws InterruptedException { * "content": "'t" * } */ + + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return null; + } + ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); if (delta.getRole() != null) { @@ -78,7 +81,7 @@ public static void main(String[] args) throws InterruptedException { .subscribe( System.out::print, error -> System.err.println("There was an error getting chat completions." + error), - () -> System.out.println("Completed called getChatCompletions.")); + () -> System.out.println("Completed called getChatCompletionsStream.")); // The .subscribe() creation and assignment is not a blocking call. For the purpose of this example, we sleep diff --git a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamSample.java b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamSample.java index 1fffbd1708c33..21931a6c18d5b 100644 --- a/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamSample.java +++ b/sdk/openai/azure-ai-openai/src/samples/java/com/azure/ai/openai/usage/GetChatCompletionsStreamSample.java @@ -13,6 +13,8 @@ import com.azure.ai.openai.models.ChatRequestUserMessage; import com.azure.ai.openai.models.ChatResponseMessage; import com.azure.core.credential.AzureKeyCredential; +import com.azure.core.util.Configuration; +import com.azure.core.util.CoreUtils; import com.azure.core.util.IterableStream; import java.util.ArrayList; @@ -33,9 +35,9 @@ public class GetChatCompletionsStreamSample { * @param args Unused. Arguments to the program. */ public static void main(String[] args) { - String azureOpenaiKey = "{azure-open-ai-key}"; - String endpoint = "{azure-open-ai-endpoint}"; - String deploymentOrModelId = "{azure-open-ai-deployment-model-id}"; + String azureOpenaiKey = Configuration.getGlobalConfiguration().get("AZURE_OPENAI_KEY"); + String endpoint = Configuration.getGlobalConfiguration().get("AZURE_OPENAI_ENDPOINT"); + String deploymentOrModelId = Configuration.getGlobalConfiguration().get("OPENAI_DEPLOYMENT_OR_MODEL_ID"); OpenAIClient client = new OpenAIClientBuilder() .endpoint(endpoint) @@ -67,11 +69,10 @@ public static void main(String[] args) { // } chatCompletionsStream .stream() - // Remove .skip(1) when using Non-Azure OpenAI API - // Note: the first chat completions can be ignored when using Azure OpenAI service which is a known service bug. - // TODO: remove .skip(1) when service fix the issue. - .skip(1) .forEach(chatCompletions -> { + if (CoreUtils.isNullOrEmpty(chatCompletions.getChoices())) { + return; + } ChatResponseMessage delta = chatCompletions.getChoices().get(0).getDelta(); if (delta.getRole() != null) { System.out.println("Role = " + delta.getRole()); diff --git a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAIAsyncClientTest.java b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAIAsyncClientTest.java index d128b24d5995d..083cdf7b588d7 100644 --- a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAIAsyncClientTest.java +++ b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAIAsyncClientTest.java @@ -201,6 +201,24 @@ public void testGetChatCompletionsStream(HttpClient httpClient, OpenAIServiceVer }); } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) + @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") + public void testGetChatCompletionsStreamWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { + client = getNonAzureOpenAIAsyncClient(httpClient); + getChatCompletionsWithResponseRunnerForNonAzure(deploymentId -> chatMessages -> requestOptions -> { + StepVerifier.create(client.getChatCompletionsStreamWithResponse(deploymentId, + new ChatCompletionsOptions(chatMessages), requestOptions)) + .recordWith(ArrayList::new) + .thenConsumeWhile(response -> { + assertResponseRequestHeader(response.getRequest()); + assertChatCompletionsStream(response.getValue()); + return true; + }) + .consumeRecordedWith(messageList -> assertTrue(messageList.size() > 1)) + .verifyComplete(); + }); + } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") public void testGetChatCompletionsWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { diff --git a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAISyncClientTest.java b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAISyncClientTest.java index 3765ff790dbb9..3d97a5661c62d 100644 --- a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAISyncClientTest.java +++ b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/NonAzureOpenAISyncClientTest.java @@ -175,6 +175,20 @@ public void testGetChatCompletionsStream(HttpClient httpClient, OpenAIServiceVer }); } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) + @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") + public void testGetChatCompletionsStreamWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { + client = getNonAzureOpenAISyncClient(httpClient); + getChatCompletionsWithResponseRunnerForNonAzure(deploymentId -> chatMessages -> requestOptions -> { + Response> response = client.getChatCompletionsStreamWithResponse( + deploymentId, new ChatCompletionsOptions(chatMessages), requestOptions); + assertResponseRequestHeader(response.getRequest()); + IterableStream value = response.getValue(); + assertTrue(value.stream().toArray().length > 1); + value.forEach(OpenAIClientTestBase::assertChatCompletionsStream); + }); + } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") public void testGetChatCompletionsWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { diff --git a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIAsyncClientTest.java b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIAsyncClientTest.java index cca54c793b26e..df2e27096bc34 100644 --- a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIAsyncClientTest.java +++ b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIAsyncClientTest.java @@ -197,6 +197,24 @@ public void testGetChatCompletionsStream(HttpClient httpClient, OpenAIServiceVer }); } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) + @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") + public void testGetChatCompletionsStreamWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { + client = getOpenAIAsyncClient(httpClient, serviceVersion); + getChatCompletionsWithResponseRunner(deploymentId -> chatMessages -> requestOptions -> { + StepVerifier.create(client.getChatCompletionsStreamWithResponse(deploymentId, + new ChatCompletionsOptions(chatMessages), requestOptions)) + .recordWith(ArrayList::new) + .thenConsumeWhile(response -> { + assertResponseRequestHeader(response.getRequest()); + assertChatCompletionsStream(response.getValue()); + return true; + }) + .consumeRecordedWith(messageList -> assertTrue(messageList.size() > 1)) + .verifyComplete(); + }); + } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") public void testGetChatCompletionsWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { @@ -415,7 +433,6 @@ public void testCompletionStreamContentFiltering(HttpClient httpClient, OpenAISe for (Iterator it = messageList.iterator(); it.hasNext();) { Completions completions = it.next(); if (i == 0) { - System.out.println("First stream message"); assertEquals(1, completions.getPromptFilterResults().size()); assertSafePromptContentFilterResults(completions.getPromptFilterResults().get(0)); } else if (i == messageList.size() - 1) { diff --git a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIClientTestBase.java b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIClientTestBase.java index ccd0fc58c341d..776dbff6d74c9 100644 --- a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIClientTestBase.java +++ b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAIClientTestBase.java @@ -51,6 +51,10 @@ import com.azure.core.credential.KeyCredential; import com.azure.core.exception.HttpResponseException; import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.policy.AddHeadersFromContextPolicy; +import com.azure.core.http.rest.RequestOptions; import com.azure.core.http.rest.Response; import com.azure.core.test.TestMode; import com.azure.core.test.TestProxyTestBase; @@ -59,6 +63,7 @@ import com.azure.core.test.models.TestProxySanitizerType; import com.azure.core.util.BinaryData; import com.azure.core.util.Configuration; +import com.azure.core.util.Context; import org.junit.jupiter.api.Test; import java.nio.file.Path; @@ -68,6 +73,8 @@ import java.util.List; import java.util.Map; import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -205,10 +212,20 @@ void getChatCompletionsRunner(BiConsumer> testR testRunner.accept("gpt-35-turbo-1106", getChatMessages()); } + void getChatCompletionsWithResponseRunner(Function, Consumer>> testRunner) { + testRunner.apply("gpt-35-turbo-1106").apply(getChatMessages()).accept(getRequestOption()); + } + void getChatCompletionsRunnerForNonAzure(BiConsumer> testRunner) { testRunner.accept("gpt-3.5-turbo", getChatMessages()); } + void getChatCompletionsWithResponseRunnerForNonAzure(Function, Consumer>> testRunner) { + testRunner.apply("gpt-3.5-turbo").apply(getChatMessages()).accept(getRequestOption()); + } + void getChatCompletionsAzureChatSearchRunner(BiConsumer testRunner) { ChatCompletionsOptions chatCompletionsOptions = new ChatCompletionsOptions( Arrays.asList(new ChatRequestUserMessage("What does PR complete mean?"))); @@ -369,6 +386,14 @@ private List getChatMessages() { return chatMessages; } + private RequestOptions getRequestOption() { + HttpHeaders headers = new HttpHeaders(); + headers.set("my-header1", "my-header1-value"); + headers.set("my-header2", "my-header2-value"); + Context context = new Context(AddHeadersFromContextPolicy.AZURE_REQUEST_HTTP_HEADERS_KEY, headers); + return new RequestOptions().setContext(context).setHeader("my-header3", "my-header3-value"); + } + private List getChatRequestMessagesWithVision() { List chatMessages = new ArrayList<>(); chatMessages.add(new ChatRequestSystemMessage("You are a helpful assistant that describes images")); @@ -675,6 +700,23 @@ static void validateImageGenerationContentFilteringException(HttpResponseExcepti assertNotNull(contentFilterResults); } + static void assertResponseRequestHeader(HttpRequest request) { + request.getHeaders().stream().filter(header -> { + String name = header.getName(); + return "my-header1".equals(name) || "my-header2".equals(name) || "my-header3".equals(name); + }).forEach(header -> { + if (header.getName().equals("my-header1")) { + assertEquals("my-header1-value", header.getValue()); + } else if (header.getName().equals("my-header2")) { + assertEquals("my-header2-value", header.getValue()); + } else if (header.getName().equals("my-header3")) { + assertEquals("my-header3-value", header.getValue()); + } else { + assertFalse(true); + } + }); + } + protected static final String BATMAN_TRANSCRIPTION = "Skills and Abilities. Batman has no inherent superpowers. He relies on his own " + "scientific knowledge, detective skills, and athletic prowess. In the stories, Batman is " diff --git a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAISyncClientTest.java b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAISyncClientTest.java index 42c14d80638cd..6a29bf7e81a80 100644 --- a/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAISyncClientTest.java +++ b/sdk/openai/azure-ai-openai/src/test/java/com/azure/ai/openai/OpenAISyncClientTest.java @@ -169,6 +169,20 @@ public void testGetChatCompletionsStream(HttpClient httpClient, OpenAIServiceVer }); } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) + @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") + public void testGetChatCompletionsStreamWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { + client = getOpenAIClient(httpClient, serviceVersion); + getChatCompletionsWithResponseRunner(deploymentId -> chatMessages -> requestOptions -> { + Response> response = client.getChatCompletionsStreamWithResponse( + deploymentId, new ChatCompletionsOptions(chatMessages), requestOptions); + assertResponseRequestHeader(response.getRequest()); + IterableStream value = response.getValue(); + assertTrue(value.stream().toArray().length > 1); + value.forEach(OpenAIClientTestBase::assertChatCompletionsStream); + }); + } + @ParameterizedTest(name = DISPLAY_NAME_WITH_ARGUMENTS) @MethodSource("com.azure.ai.openai.TestUtils#getTestParameters") public void testGetChatCompletionsWithResponse(HttpClient httpClient, OpenAIServiceVersion serviceVersion) { @@ -355,7 +369,6 @@ public void testCompletionStreamContentFiltering(HttpClient httpClient, OpenAISe Completions completions = it.next(); assertCompletionsStream(completions); if (i == 0) { - System.out.println("First stream message"); // The first stream message has the prompt filter result assertEquals(1, completions.getPromptFilterResults().size()); assertSafePromptContentFilterResults(completions.getPromptFilterResults().get(0)); diff --git a/sdk/operationsmanagement/azure-resourcemanager-operationsmanagement/pom.xml b/sdk/operationsmanagement/azure-resourcemanager-operationsmanagement/pom.xml index 6aa7aaecf5470..e5cdf0d565104 100644 --- a/sdk/operationsmanagement/azure-resourcemanager-operationsmanagement/pom.xml +++ b/sdk/operationsmanagement/azure-resourcemanager-operationsmanagement/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/orbital/azure-resourcemanager-orbital/pom.xml b/sdk/orbital/azure-resourcemanager-orbital/pom.xml index 0a24f1b62c139..a6126d246f369 100644 --- a/sdk/orbital/azure-resourcemanager-orbital/pom.xml +++ b/sdk/orbital/azure-resourcemanager-orbital/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/paloaltonetworks/azure-resourcemanager-paloaltonetworks-ngfw/pom.xml b/sdk/paloaltonetworks/azure-resourcemanager-paloaltonetworks-ngfw/pom.xml index e1d17df292071..fe78c6a988f54 100644 --- a/sdk/paloaltonetworks/azure-resourcemanager-paloaltonetworks-ngfw/pom.xml +++ b/sdk/paloaltonetworks/azure-resourcemanager-paloaltonetworks-ngfw/pom.xml @@ -66,13 +66,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -98,13 +98,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/paloaltonetworks/tests.mgmt.yml b/sdk/paloaltonetworks/tests.mgmt.yml index 2cd1b3cfc63af..e108589086e85 100644 --- a/sdk/paloaltonetworks/tests.mgmt.yml +++ b/sdk/paloaltonetworks/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: paloaltonetworks - Artifacts: - - name: azure-resourcemanager-paloaltonetworks-ngfw - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerpaloaltonetworksngfw - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: paloaltonetworks + Artifacts: + - name: azure-resourcemanager-paloaltonetworks-ngfw + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerpaloaltonetworksngfw + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/peering/azure-resourcemanager-peering/pom.xml b/sdk/peering/azure-resourcemanager-peering/pom.xml index 457289c4f8673..22322b6762008 100644 --- a/sdk/peering/azure-resourcemanager-peering/pom.xml +++ b/sdk/peering/azure-resourcemanager-peering/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/personalizer/azure-ai-personalizer/pom.xml b/sdk/personalizer/azure-ai-personalizer/pom.xml index 5d82a4b5d920d..3c85f55a30986 100644 --- a/sdk/personalizer/azure-ai-personalizer/pom.xml +++ b/sdk/personalizer/azure-ai-personalizer/pom.xml @@ -91,7 +91,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/personalizer/tests.yml b/sdk/personalizer/tests.yml index 53f474dc7c90c..d1ec7203a35a1 100644 --- a/sdk/personalizer/tests.yml +++ b/sdk/personalizer/tests.yml @@ -1,17 +1,17 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: personalizer - timeoutInMinutes: 240 # how long to run the job before automatically cancelling - Artifacts: - - name: azure-ai-personalizer - groupId: com.azure - safeName: azureaipersonalizer - EnvVars: - AZURE_TENANT_ID: $(PERSONALIZER_TENANT_ID) - AZURE_CLIENT_ID: $(PERSONALIZER_CLIENT_ID) - AZURE_CLIENT_SECRET: $(PERSONALIZER_CLIENT_SECRET) - PERSONALIZER_ENDPOINT_STATIC: $(java-personalizer-test-singleslot-endpoint) - PERSONALIZER_API_KEY_STATIC: $(java-personalizer-test-singleslot-api-key) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: personalizer + timeoutInMinutes: 240 # how long to run the job before automatically cancelling + Artifacts: + - name: azure-ai-personalizer + groupId: com.azure + safeName: azureaipersonalizer + EnvVars: + AZURE_TENANT_ID: $(PERSONALIZER_TENANT_ID) + AZURE_CLIENT_ID: $(PERSONALIZER_CLIENT_ID) + AZURE_CLIENT_SECRET: $(PERSONALIZER_CLIENT_SECRET) + PERSONALIZER_ENDPOINT_STATIC: $(java-personalizer-test-singleslot-endpoint) + PERSONALIZER_API_KEY_STATIC: $(java-personalizer-test-singleslot-api-key) diff --git a/sdk/playwrighttesting/azure-resourcemanager-playwrighttesting/pom.xml b/sdk/playwrighttesting/azure-resourcemanager-playwrighttesting/pom.xml index 328f07eab9182..e8e4d242d8d22 100644 --- a/sdk/playwrighttesting/azure-resourcemanager-playwrighttesting/pom.xml +++ b/sdk/playwrighttesting/azure-resourcemanager-playwrighttesting/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/policyinsights/azure-resourcemanager-policyinsights/pom.xml b/sdk/policyinsights/azure-resourcemanager-policyinsights/pom.xml index 2e1369905d79f..9e48b9477c2dd 100644 --- a/sdk/policyinsights/azure-resourcemanager-policyinsights/pom.xml +++ b/sdk/policyinsights/azure-resourcemanager-policyinsights/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -80,13 +80,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/postgresql/azure-resourcemanager-postgresql/pom.xml b/sdk/postgresql/azure-resourcemanager-postgresql/pom.xml index b7c0063565df9..d72de90ece771 100644 --- a/sdk/postgresql/azure-resourcemanager-postgresql/pom.xml +++ b/sdk/postgresql/azure-resourcemanager-postgresql/pom.xml @@ -61,7 +61,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -79,7 +79,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/postgresql/tests.mgmt.yml b/sdk/postgresql/tests.mgmt.yml index dd221a669142a..28e91ffc11b1a 100644 --- a/sdk/postgresql/tests.mgmt.yml +++ b/sdk/postgresql/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: postgresql - Artifacts: - - name: azure-resourcemanager-postgresql - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerpostgresql - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: postgresql + Artifacts: + - name: azure-resourcemanager-postgresql + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerpostgresql + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/postgresqlflexibleserver/azure-resourcemanager-postgresqlflexibleserver/pom.xml b/sdk/postgresqlflexibleserver/azure-resourcemanager-postgresqlflexibleserver/pom.xml index b277d66f26617..cbc37f364d59e 100644 --- a/sdk/postgresqlflexibleserver/azure-resourcemanager-postgresqlflexibleserver/pom.xml +++ b/sdk/postgresqlflexibleserver/azure-resourcemanager-postgresqlflexibleserver/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -109,7 +109,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/postgresqlflexibleserver/tests.mgmt.yml b/sdk/postgresqlflexibleserver/tests.mgmt.yml index ec062f978a78c..306720e46cbec 100644 --- a/sdk/postgresqlflexibleserver/tests.mgmt.yml +++ b/sdk/postgresqlflexibleserver/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: postgresqlflexibleserver - Artifacts: - - name: azure-resourcemanager-postgresqlflexibleserver - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerpostgresqlflexibleserver - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: postgresqlflexibleserver + Artifacts: + - name: azure-resourcemanager-postgresqlflexibleserver + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerpostgresqlflexibleserver + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/powerbidedicated/azure-resourcemanager-powerbidedicated/pom.xml b/sdk/powerbidedicated/azure-resourcemanager-powerbidedicated/pom.xml index b933d93e0966b..e48f58a7aefe8 100644 --- a/sdk/powerbidedicated/azure-resourcemanager-powerbidedicated/pom.xml +++ b/sdk/powerbidedicated/azure-resourcemanager-powerbidedicated/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -86,13 +86,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/providerhub/azure-resourcemanager-providerhub/pom.xml b/sdk/providerhub/azure-resourcemanager-providerhub/pom.xml index cc2b9aec3a0e4..e02c8581a1f4a 100644 --- a/sdk/providerhub/azure-resourcemanager-providerhub/pom.xml +++ b/sdk/providerhub/azure-resourcemanager-providerhub/pom.xml @@ -60,7 +60,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test @@ -72,7 +72,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/providerhub/tests.mgmt.yml b/sdk/providerhub/tests.mgmt.yml index 8f097f7636cd2..fbcee5621ccb0 100644 --- a/sdk/providerhub/tests.mgmt.yml +++ b/sdk/providerhub/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: providerhub - Artifacts: - - name: azure-resourcemanager-providerhub - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerproviderhub - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* \ No newline at end of file +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: providerhub + Artifacts: + - name: azure-resourcemanager-providerhub + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerproviderhub + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/purview/azure-analytics-purview-administration/pom.xml b/sdk/purview/azure-analytics-purview-administration/pom.xml index e25a5fb5c009d..0ee7655716f12 100644 --- a/sdk/purview/azure-analytics-purview-administration/pom.xml +++ b/sdk/purview/azure-analytics-purview-administration/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/purview/azure-analytics-purview-catalog/pom.xml b/sdk/purview/azure-analytics-purview-catalog/pom.xml index 0ad572278b013..8c29fe0609c93 100644 --- a/sdk/purview/azure-analytics-purview-catalog/pom.xml +++ b/sdk/purview/azure-analytics-purview-catalog/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/purview/azure-analytics-purview-datamap/pom.xml b/sdk/purview/azure-analytics-purview-datamap/pom.xml index 7643772122747..c6b69f9fe53bd 100644 --- a/sdk/purview/azure-analytics-purview-datamap/pom.xml +++ b/sdk/purview/azure-analytics-purview-datamap/pom.xml @@ -84,7 +84,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/purview/azure-analytics-purview-scanning/pom.xml b/sdk/purview/azure-analytics-purview-scanning/pom.xml index 8886786b8c401..e799bb628e546 100644 --- a/sdk/purview/azure-analytics-purview-scanning/pom.xml +++ b/sdk/purview/azure-analytics-purview-scanning/pom.xml @@ -72,7 +72,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/purview/azure-analytics-purview-sharing/pom.xml b/sdk/purview/azure-analytics-purview-sharing/pom.xml index 43baeccf235a1..c56d43faa059d 100644 --- a/sdk/purview/azure-analytics-purview-sharing/pom.xml +++ b/sdk/purview/azure-analytics-purview-sharing/pom.xml @@ -78,7 +78,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/purview/azure-analytics-purview-workflow/pom.xml b/sdk/purview/azure-analytics-purview-workflow/pom.xml index e266b1eeb19e8..f8a545e206707 100644 --- a/sdk/purview/azure-analytics-purview-workflow/pom.xml +++ b/sdk/purview/azure-analytics-purview-workflow/pom.xml @@ -79,7 +79,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/purview/azure-resourcemanager-purview/pom.xml b/sdk/purview/azure-resourcemanager-purview/pom.xml index fa3db9e838912..54f0cbd0c6074 100644 --- a/sdk/purview/azure-resourcemanager-purview/pom.xml +++ b/sdk/purview/azure-resourcemanager-purview/pom.xml @@ -61,7 +61,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,13 +87,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/quantum/azure-quantum-jobs/pom.xml b/sdk/quantum/azure-quantum-jobs/pom.xml index 3e70acbe11fac..3c609fbf1b207 100644 --- a/sdk/quantum/azure-quantum-jobs/pom.xml +++ b/sdk/quantum/azure-quantum-jobs/pom.xml @@ -76,13 +76,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure azure-storage-blob - 12.25.2 + 12.25.3 test diff --git a/sdk/quantum/azure-resourcemanager-quantum/CHANGELOG.md b/sdk/quantum/azure-resourcemanager-quantum/CHANGELOG.md index 1199ab0a10326..e994384abd27f 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/CHANGELOG.md +++ b/sdk/quantum/azure-resourcemanager-quantum/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.0.0-beta.2 (Unreleased) +## 1.0.0-beta.3 (Unreleased) ### Features Added @@ -10,6 +10,35 @@ ### Other Changes +## 1.0.0-beta.2 (2024-03-15) + +- Azure Resource Manager AzureQuantum client library for Java. This package contains Microsoft Azure SDK for AzureQuantum Management SDK. Package tag package-2023-11-13-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Features Added + +* `models.ApiKey` was added + +* `models.ApiKeys` was added + +* `models.ListKeysResult` was added + +* `models.KeyType` was added + +#### `models.QuantumWorkspace$Definition` was modified + +* `withApiKeyEnabled(java.lang.Boolean)` was added + +#### `models.WorkspaceOperations` was modified + +* `regenerateKeysWithResponse(java.lang.String,java.lang.String,models.ApiKeys,com.azure.core.util.Context)` was added +* `listKeys(java.lang.String,java.lang.String)` was added +* `listKeysWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `regenerateKeys(java.lang.String,java.lang.String,models.ApiKeys)` was added + +#### `models.QuantumWorkspace` was modified + +* `apiKeyEnabled()` was added + ## 1.0.0-beta.1 (2023-07-21) - Azure Resource Manager AzureQuantum client library for Java. This package contains Microsoft Azure SDK for AzureQuantum Management SDK. Package tag package-2022-01-10-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/quantum/azure-resourcemanager-quantum/README.md b/sdk/quantum/azure-resourcemanager-quantum/README.md index 559192cd6e770..d09472dc23940 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/README.md +++ b/sdk/quantum/azure-resourcemanager-quantum/README.md @@ -2,7 +2,7 @@ Azure Resource Manager AzureQuantum client library for Java. -This package contains Microsoft Azure SDK for AzureQuantum Management SDK. Package tag package-2022-01-10-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for AzureQuantum Management SDK. Package tag package-2023-11-13-preview. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-quantum - 1.0.0-beta.1 + 1.0.0-beta.2 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -94,7 +94,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty diff --git a/sdk/quantum/azure-resourcemanager-quantum/SAMPLE.md b/sdk/quantum/azure-resourcemanager-quantum/SAMPLE.md index 24dabb4e88d15..22738b15d8c83 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/SAMPLE.md +++ b/sdk/quantum/azure-resourcemanager-quantum/SAMPLE.md @@ -12,6 +12,8 @@ ## WorkspaceOperation - [CheckNameAvailability](#workspaceoperation_checknameavailability) +- [ListKeys](#workspaceoperation_listkeys) +- [RegenerateKeys](#workspaceoperation_regeneratekeys) ## Workspaces @@ -24,14 +26,17 @@ ### Offerings_List ```java -/** Samples for Offerings List. */ +/** + * Samples for Offerings List. + */ public final class OfferingsListSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/offeringsList.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/offeringsList.json */ /** * Sample code: OfferingsList. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void offeringsList(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { @@ -43,14 +48,17 @@ public final class OfferingsListSamples { ### Operations_List ```java -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/operations.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/operations.json */ /** * Sample code: Operations. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void operations(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { @@ -64,58 +72,106 @@ public final class OperationsListSamples { ```java import com.azure.resourcemanager.quantum.models.CheckNameAvailabilityParameters; -/** Samples for WorkspaceOperation CheckNameAvailability. */ +/** + * Samples for WorkspaceOperation CheckNameAvailability. + */ public final class WorkspaceOperationCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesCheckNameAvailability.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesCheckNameAvailability.json */ /** * Sample code: QuantumWorkspacesCheckNameAvailability. - * + * * @param manager Entry point to AzureQuantumManager. */ - public static void quantumWorkspacesCheckNameAvailability( - com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - manager - .workspaceOperations() - .checkNameAvailabilityWithResponse( - "westus2", - new CheckNameAvailabilityParameters() - .withName("sample-workspace-name") - .withType("Microsoft.Quantum/Workspaces"), + public static void + quantumWorkspacesCheckNameAvailability(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + manager.workspaceOperations() + .checkNameAvailabilityWithResponse("westus2", new CheckNameAvailabilityParameters() + .withName("sample-workspace-name").withType("Microsoft.Quantum/Workspaces"), com.azure.core.util.Context.NONE); } } ``` +### WorkspaceOperation_ListKeys + +```java +/** + * Samples for WorkspaceOperation ListKeys. + */ +public final class WorkspaceOperationListKeysSamples { + /* + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/listKeys.json + */ + /** + * Sample code: ListKeys. + * + * @param manager Entry point to AzureQuantumManager. + */ + public static void listKeys(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + manager.workspaceOperations().listKeysWithResponse("quantumResourcegroup", "quantumworkspace1", + com.azure.core.util.Context.NONE); + } +} +``` + +### WorkspaceOperation_RegenerateKeys + +```java +import com.azure.resourcemanager.quantum.models.ApiKeys; +import com.azure.resourcemanager.quantum.models.KeyType; +import java.util.Arrays; + +/** + * Samples for WorkspaceOperation RegenerateKeys. + */ +public final class WorkspaceOperationRegenerateKeysSamples { + /* + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/regenerateKey.json + */ + /** + * Sample code: RegenerateKey. + * + * @param manager Entry point to AzureQuantumManager. + */ + public static void regenerateKey(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + manager.workspaceOperations().regenerateKeysWithResponse("quantumResourcegroup", "quantumworkspace1", + new ApiKeys().withKeys(Arrays.asList(KeyType.PRIMARY, KeyType.SECONDARY)), + com.azure.core.util.Context.NONE); + } +} +``` + ### Workspaces_CreateOrUpdate ```java import com.azure.resourcemanager.quantum.models.Provider; import java.util.Arrays; -/** Samples for Workspaces CreateOrUpdate. */ +/** + * Samples for Workspaces CreateOrUpdate. + */ public final class WorkspacesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesPut.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/quantumWorkspacesPut + * .json */ /** * Sample code: QuantumWorkspacesPut. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesPut(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - manager - .workspaces() - .define("quantumworkspace1") - .withRegion("West US") + manager.workspaces().define("quantumworkspace1").withRegion("West US") .withExistingResourceGroup("quantumResourcegroup") - .withProviders( - Arrays - .asList( - new Provider().withProviderId("Honeywell").withProviderSku("Basic"), - new Provider().withProviderId("IonQ").withProviderSku("Basic"), - new Provider().withProviderId("OneQBit").withProviderSku("Basic"))) + .withProviders(Arrays.asList(new Provider().withProviderId("Honeywell").withProviderSku("Basic"), + new Provider().withProviderId("IonQ").withProviderSku("Basic"), + new Provider().withProviderId("OneQBit").withProviderSku("Basic"))) .withStorageAccount( "/subscriptions/1C4B2828-7D49-494F-933D-061373BE28C2/resourceGroups/quantumResourcegroup/providers/Microsoft.Storage/storageAccounts/testStorageAccount") .create(); @@ -126,14 +182,17 @@ public final class WorkspacesCreateOrUpdateSamples { ### Workspaces_Delete ```java -/** Samples for Workspaces Delete. */ +/** + * Samples for Workspaces Delete. + */ public final class WorkspacesDeleteSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesDelete.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesDelete.json */ /** * Sample code: QuantumWorkspacesDelete. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesDelete(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { @@ -145,21 +204,23 @@ public final class WorkspacesDeleteSamples { ### Workspaces_GetByResourceGroup ```java -/** Samples for Workspaces GetByResourceGroup. */ +/** + * Samples for Workspaces GetByResourceGroup. + */ public final class WorkspacesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesGet.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/quantumWorkspacesGet + * .json */ /** * Sample code: QuantumWorkspacesGet. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesGet(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - manager - .workspaces() - .getByResourceGroupWithResponse( - "quantumResourcegroup", "quantumworkspace1", com.azure.core.util.Context.NONE); + manager.workspaces().getByResourceGroupWithResponse("quantumResourcegroup", "quantumworkspace1", + com.azure.core.util.Context.NONE); } } ``` @@ -167,18 +228,21 @@ public final class WorkspacesGetByResourceGroupSamples { ### Workspaces_List ```java -/** Samples for Workspaces List. */ +/** + * Samples for Workspaces List. + */ public final class WorkspacesListSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesListSubscription.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesListSubscription.json */ /** * Sample code: QuantumWorkspacesListBySubscription. - * + * * @param manager Entry point to AzureQuantumManager. */ - public static void quantumWorkspacesListBySubscription( - com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + public static void + quantumWorkspacesListBySubscription(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { manager.workspaces().list(com.azure.core.util.Context.NONE); } } @@ -187,18 +251,21 @@ public final class WorkspacesListSamples { ### Workspaces_ListByResourceGroup ```java -/** Samples for Workspaces ListByResourceGroup. */ +/** + * Samples for Workspaces ListByResourceGroup. + */ public final class WorkspacesListByResourceGroupSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesListResourceGroup.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesListResourceGroup.json */ /** * Sample code: QuantumWorkspacesListByResourceGroup. - * + * * @param manager Entry point to AzureQuantumManager. */ - public static void quantumWorkspacesListByResourceGroup( - com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + public static void + quantumWorkspacesListByResourceGroup(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { manager.workspaces().listByResourceGroup("quantumResourcegroup", com.azure.core.util.Context.NONE); } } @@ -211,26 +278,26 @@ import com.azure.resourcemanager.quantum.models.QuantumWorkspace; import java.util.HashMap; import java.util.Map; -/** Samples for Workspaces UpdateTags. */ +/** + * Samples for Workspaces UpdateTags. + */ public final class WorkspacesUpdateTagsSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesPatch.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesPatch.json */ /** * Sample code: QuantumWorkspacesPatchTags. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesPatchTags(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - QuantumWorkspace resource = - manager - .workspaces() - .getByResourceGroupWithResponse( - "quantumResourcegroup", "quantumworkspace1", com.azure.core.util.Context.NONE) - .getValue(); + QuantumWorkspace resource = manager.workspaces().getByResourceGroupWithResponse("quantumResourcegroup", + "quantumworkspace1", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/pom.xml b/sdk/quantum/azure-resourcemanager-quantum/pom.xml index 958a629bd3198..62fe9ebd58d09 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/pom.xml +++ b/sdk/quantum/azure-resourcemanager-quantum/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-quantum - 1.0.0-beta.2 + 1.0.0-beta.3 jar Microsoft Azure SDK for AzureQuantum Management - This package contains Microsoft Azure SDK for AzureQuantum Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Package tag package-2022-01-10-preview. + This package contains Microsoft Azure SDK for AzureQuantum Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. Package tag package-2023-11-13-preview. https://github.com/Azure/azure-sdk-for-java @@ -45,6 +45,7 @@ UTF-8 0 0 + true @@ -66,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,18 +88,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/AzureQuantumManager.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/AzureQuantumManager.java index ca94aca262c9b..ac4e2d2605aa3 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/AzureQuantumManager.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/AzureQuantumManager.java @@ -40,7 +40,9 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to AzureQuantumManager. */ +/** + * Entry point to AzureQuantumManager. + */ public final class AzureQuantumManager { private Workspaces workspaces; @@ -55,18 +57,14 @@ public final class AzureQuantumManager { private AzureQuantumManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new AzureQuantumManagementClientBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new AzureQuantumManagementClientBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of AzureQuantum service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the AzureQuantum service API instance. @@ -79,7 +77,7 @@ public static AzureQuantumManager authenticate(TokenCredential credential, Azure /** * Creates an instance of AzureQuantum service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the AzureQuantum service API instance. @@ -92,14 +90,16 @@ public static AzureQuantumManager authenticate(HttpPipeline httpPipeline, AzureP /** * Gets a Configurable instance that can be used to create AzureQuantumManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new AzureQuantumManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -171,8 +171,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -189,8 +189,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -210,21 +210,12 @@ public AzureQuantumManager authenticate(TokenCredential credential, AzureProfile Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.quantum") - .append("/") - .append("1.0.0-beta.1"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.quantum").append("/") + .append("1.0.0-beta.2"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -243,38 +234,25 @@ public AzureQuantumManager authenticate(TokenCredential credential, AzureProfile policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new AzureQuantumManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of Workspaces. It manages QuantumWorkspace. - * + * * @return Resource collection API of Workspaces. */ public Workspaces workspaces() { @@ -286,7 +264,7 @@ public Workspaces workspaces() { /** * Gets the resource collection API of Offerings. - * + * * @return Resource collection API of Offerings. */ public Offerings offerings() { @@ -298,7 +276,7 @@ public Offerings offerings() { /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -310,7 +288,7 @@ public Operations operations() { /** * Gets the resource collection API of WorkspaceOperations. - * + * * @return Resource collection API of WorkspaceOperations. */ public WorkspaceOperations workspaceOperations() { @@ -321,8 +299,10 @@ public WorkspaceOperations workspaceOperations() { } /** - * @return Wrapped service client AzureQuantumManagementClient providing direct access to the underlying - * auto-generated API implementation, based on Azure REST API. + * Gets wrapped service client AzureQuantumManagementClient providing direct access to the underlying auto-generated + * API implementation, based on Azure REST API. + * + * @return Wrapped service client AzureQuantumManagementClient. */ public AzureQuantumManagementClient serviceClient() { return this.clientObject; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/AzureQuantumManagementClient.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/AzureQuantumManagementClient.java index 249f89e353400..5abc8e88d148d 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/AzureQuantumManagementClient.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/AzureQuantumManagementClient.java @@ -7,67 +7,69 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for AzureQuantumManagementClient class. */ +/** + * The interface for AzureQuantumManagementClient class. + */ public interface AzureQuantumManagementClient { /** - * Gets The Azure subscription ID. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the WorkspacesClient object to access its operations. - * + * * @return the WorkspacesClient object. */ WorkspacesClient getWorkspaces(); /** * Gets the OfferingsClient object to access its operations. - * + * * @return the OfferingsClient object. */ OfferingsClient getOfferings(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the WorkspaceOperationsClient object to access its operations. - * + * * @return the WorkspaceOperationsClient object. */ WorkspaceOperationsClient getWorkspaceOperations(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OfferingsClient.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OfferingsClient.java index 33ca6ac458d9a..8fa8eebb1bba1 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OfferingsClient.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OfferingsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.quantum.fluent.models.ProviderDescriptionInner; -/** An instance of this class provides access to all the operations defined in OfferingsClient. */ +/** + * An instance of this class provides access to all the operations defined in OfferingsClient. + */ public interface OfferingsClient { /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -26,7 +28,7 @@ public interface OfferingsClient { /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OperationsClient.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OperationsClient.java index 853bab8ec4207..8cf0abb9ba601 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OperationsClient.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/OperationsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.quantum.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Returns list of operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return lists the operations available as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface OperationsClient { /** * Returns list of operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspaceOperationsClient.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspaceOperationsClient.java index 00ff2607c5da3..de541b77db5a1 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspaceOperationsClient.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspaceOperationsClient.java @@ -9,13 +9,17 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; import com.azure.resourcemanager.quantum.fluent.models.CheckNameAvailabilityResultInner; +import com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner; +import com.azure.resourcemanager.quantum.models.ApiKeys; import com.azure.resourcemanager.quantum.models.CheckNameAvailabilityParameters; -/** An instance of this class provides access to all the operations defined in WorkspaceOperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspaceOperationsClient. + */ public interface WorkspaceOperationsClient { /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @param context The context to associate with this operation. @@ -25,12 +29,12 @@ public interface WorkspaceOperationsClient { * @return result of check name availability along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response checkNameAvailabilityWithResponse( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context); + Response checkNameAvailabilityWithResponse(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context); /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -39,6 +43,66 @@ Response checkNameAvailabilityWithResponse( * @return result of check name availability. */ @ServiceMethod(returns = ReturnType.SINGLE) - CheckNameAvailabilityResultInner checkNameAvailability( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters); + CheckNameAvailabilityResultInner checkNameAvailability(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters); + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listKeysWithResponse(String resourceGroupName, String workspaceName, Context context); + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ListKeysResultInner listKeys(String resourceGroupName, String workspaceName); + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response regenerateKeysWithResponse(String resourceGroupName, String workspaceName, ApiKeys keySpecification, + Context context); + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void regenerateKeys(String resourceGroupName, String workspaceName, ApiKeys keySpecification); } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspacesClient.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspacesClient.java index befc7e2f04690..0327e407ff0a1 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspacesClient.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/WorkspacesClient.java @@ -14,12 +14,14 @@ import com.azure.resourcemanager.quantum.fluent.models.QuantumWorkspaceInner; import com.azure.resourcemanager.quantum.models.TagsObject; -/** An instance of this class provides access to all the operations defined in WorkspacesClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspacesClient. + */ public interface WorkspacesClient { /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -28,13 +30,13 @@ public interface WorkspacesClient { * @return the resource proxy definition object for quantum workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context); /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -46,8 +48,8 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -56,13 +58,13 @@ Response getByResourceGroupWithResponse( * @return the {@link SyncPoller} for polling of the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, QuantumWorkspaceInner> beginCreateOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace); + SyncPoller, QuantumWorkspaceInner> beginCreateOrUpdate(String resourceGroupName, + String workspaceName, QuantumWorkspaceInner quantumWorkspace); /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -72,13 +74,13 @@ SyncPoller, QuantumWorkspaceInner> beginCreate * @return the {@link SyncPoller} for polling of the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, QuantumWorkspaceInner> beginCreateOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context); + SyncPoller, QuantumWorkspaceInner> beginCreateOrUpdate(String resourceGroupName, + String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context); /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -87,13 +89,13 @@ SyncPoller, QuantumWorkspaceInner> beginCreate * @return the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.SINGLE) - QuantumWorkspaceInner createOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace); + QuantumWorkspaceInner createOrUpdate(String resourceGroupName, String workspaceName, + QuantumWorkspaceInner quantumWorkspace); /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -103,13 +105,13 @@ QuantumWorkspaceInner createOrUpdate( * @return the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.SINGLE) - QuantumWorkspaceInner createOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context); + QuantumWorkspaceInner createOrUpdate(String resourceGroupName, String workspaceName, + QuantumWorkspaceInner quantumWorkspace, Context context); /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @param context The context to associate with this operation. @@ -119,13 +121,13 @@ QuantumWorkspaceInner createOrUpdate( * @return the resource proxy definition object for quantum workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateTagsWithResponse( - String resourceGroupName, String workspaceName, TagsObject workspaceTags, Context context); + Response updateTagsWithResponse(String resourceGroupName, String workspaceName, + TagsObject workspaceTags, Context context); /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -138,8 +140,8 @@ Response updateTagsWithResponse( /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -151,8 +153,8 @@ Response updateTagsWithResponse( /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -165,8 +167,8 @@ Response updateTagsWithResponse( /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -177,8 +179,8 @@ Response updateTagsWithResponse( /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -190,7 +192,7 @@ Response updateTagsWithResponse( /** * Gets the list of Workspaces within a Subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a Subscription as paginated response with {@link PagedIterable}. @@ -200,7 +202,7 @@ Response updateTagsWithResponse( /** * Gets the list of Workspaces within a Subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -212,8 +214,8 @@ Response updateTagsWithResponse( /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -224,8 +226,8 @@ Response updateTagsWithResponse( /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/CheckNameAvailabilityResultInner.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/CheckNameAvailabilityResultInner.java index a084a45608909..00ae28badfe13 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/CheckNameAvailabilityResultInner.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/CheckNameAvailabilityResultInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Result of check name availability. */ +/** + * Result of check name availability. + */ @Fluent public final class CheckNameAvailabilityResultInner { /* @@ -28,13 +30,15 @@ public final class CheckNameAvailabilityResultInner { @JsonProperty(value = "message", access = JsonProperty.Access.WRITE_ONLY) private String message; - /** Creates an instance of CheckNameAvailabilityResultInner class. */ + /** + * Creates an instance of CheckNameAvailabilityResultInner class. + */ public CheckNameAvailabilityResultInner() { } /** * Get the nameAvailable property: Indicator of availability of the Quantum Workspace resource name. - * + * * @return the nameAvailable value. */ public Boolean nameAvailable() { @@ -43,7 +47,7 @@ public Boolean nameAvailable() { /** * Set the nameAvailable property: Indicator of availability of the Quantum Workspace resource name. - * + * * @param nameAvailable the nameAvailable value to set. * @return the CheckNameAvailabilityResultInner object itself. */ @@ -54,7 +58,7 @@ public CheckNameAvailabilityResultInner withNameAvailable(Boolean nameAvailable) /** * Get the reason property: The reason of unavailability. - * + * * @return the reason value. */ public String reason() { @@ -63,7 +67,7 @@ public String reason() { /** * Set the reason property: The reason of unavailability. - * + * * @param reason the reason value to set. * @return the CheckNameAvailabilityResultInner object itself. */ @@ -74,7 +78,7 @@ public CheckNameAvailabilityResultInner withReason(String reason) { /** * Get the message property: The detailed info regarding the reason associated with the Namespace. - * + * * @return the message value. */ public String message() { @@ -83,7 +87,7 @@ public String message() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ListKeysResultInner.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ListKeysResultInner.java new file mode 100644 index 0000000000000..14e99b9fc3392 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ListKeysResultInner.java @@ -0,0 +1,143 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.quantum.models.ApiKey; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Result of list Api keys and connection strings. + */ +@Fluent +public final class ListKeysResultInner { + /* + * Indicator of enablement of the Quantum workspace Api keys. + */ + @JsonProperty(value = "apiKeyEnabled") + private Boolean apiKeyEnabled; + + /* + * The quantum workspace primary api key. + */ + @JsonProperty(value = "primaryKey") + private ApiKey primaryKey; + + /* + * The quantum workspace secondary api key. + */ + @JsonProperty(value = "secondaryKey") + private ApiKey secondaryKey; + + /* + * The connection string of the primary api key. + */ + @JsonProperty(value = "primaryConnectionString", access = JsonProperty.Access.WRITE_ONLY) + private String primaryConnectionString; + + /* + * The connection string of the secondary api key. + */ + @JsonProperty(value = "secondaryConnectionString", access = JsonProperty.Access.WRITE_ONLY) + private String secondaryConnectionString; + + /** + * Creates an instance of ListKeysResultInner class. + */ + public ListKeysResultInner() { + } + + /** + * Get the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @return the apiKeyEnabled value. + */ + public Boolean apiKeyEnabled() { + return this.apiKeyEnabled; + } + + /** + * Set the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @param apiKeyEnabled the apiKeyEnabled value to set. + * @return the ListKeysResultInner object itself. + */ + public ListKeysResultInner withApiKeyEnabled(Boolean apiKeyEnabled) { + this.apiKeyEnabled = apiKeyEnabled; + return this; + } + + /** + * Get the primaryKey property: The quantum workspace primary api key. + * + * @return the primaryKey value. + */ + public ApiKey primaryKey() { + return this.primaryKey; + } + + /** + * Set the primaryKey property: The quantum workspace primary api key. + * + * @param primaryKey the primaryKey value to set. + * @return the ListKeysResultInner object itself. + */ + public ListKeysResultInner withPrimaryKey(ApiKey primaryKey) { + this.primaryKey = primaryKey; + return this; + } + + /** + * Get the secondaryKey property: The quantum workspace secondary api key. + * + * @return the secondaryKey value. + */ + public ApiKey secondaryKey() { + return this.secondaryKey; + } + + /** + * Set the secondaryKey property: The quantum workspace secondary api key. + * + * @param secondaryKey the secondaryKey value to set. + * @return the ListKeysResultInner object itself. + */ + public ListKeysResultInner withSecondaryKey(ApiKey secondaryKey) { + this.secondaryKey = secondaryKey; + return this; + } + + /** + * Get the primaryConnectionString property: The connection string of the primary api key. + * + * @return the primaryConnectionString value. + */ + public String primaryConnectionString() { + return this.primaryConnectionString; + } + + /** + * Get the secondaryConnectionString property: The connection string of the secondary api key. + * + * @return the secondaryConnectionString value. + */ + public String secondaryConnectionString() { + return this.secondaryConnectionString; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (primaryKey() != null) { + primaryKey().validate(); + } + if (secondaryKey() != null) { + secondaryKey().validate(); + } + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/OperationInner.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/OperationInner.java index 578a81d3907e5..7999f3ec19941 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/OperationInner.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/OperationInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.quantum.models.OperationDisplay; import com.fasterxml.jackson.annotation.JsonProperty; -/** Operation provided by provider. */ +/** + * Operation provided by provider. + */ @Fluent public final class OperationInner { /* @@ -29,13 +31,15 @@ public final class OperationInner { @JsonProperty(value = "display") private OperationDisplay display; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: Name of the operation. - * + * * @return the name value. */ public String name() { @@ -44,7 +48,7 @@ public String name() { /** * Set the name property: Name of the operation. - * + * * @param name the name value to set. * @return the OperationInner object itself. */ @@ -55,7 +59,7 @@ public OperationInner withName(String name) { /** * Get the isDataAction property: Indicates whether the operation is a data action. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -64,7 +68,7 @@ public Boolean isDataAction() { /** * Set the isDataAction property: Indicates whether the operation is a data action. - * + * * @param isDataAction the isDataAction value to set. * @return the OperationInner object itself. */ @@ -75,7 +79,7 @@ public OperationInner withIsDataAction(Boolean isDataAction) { /** * Get the display property: Properties of the operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -84,7 +88,7 @@ public OperationDisplay display() { /** * Set the display property: Properties of the operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -95,7 +99,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ProviderDescriptionInner.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ProviderDescriptionInner.java index c31bc6feb023e..d69072b964de1 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ProviderDescriptionInner.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/ProviderDescriptionInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.quantum.models.ProviderProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Information about an offering. A provider offering is an entity that offers Targets to run Azure Quantum Jobs. */ +/** + * Information about an offering. A provider offering is an entity that offers Targets to run Azure Quantum Jobs. + */ @Fluent public final class ProviderDescriptionInner { /* @@ -24,18 +26,20 @@ public final class ProviderDescriptionInner { private String name; /* - * A list of provider-specific properties. + * Provider properties. */ @JsonProperty(value = "properties") private ProviderProperties properties; - /** Creates an instance of ProviderDescriptionInner class. */ + /** + * Creates an instance of ProviderDescriptionInner class. + */ public ProviderDescriptionInner() { } /** * Get the id property: Unique provider's id. - * + * * @return the id value. */ public String id() { @@ -44,7 +48,7 @@ public String id() { /** * Set the id property: Unique provider's id. - * + * * @param id the id value to set. * @return the ProviderDescriptionInner object itself. */ @@ -55,7 +59,7 @@ public ProviderDescriptionInner withId(String id) { /** * Get the name property: Provider's display name. - * + * * @return the name value. */ public String name() { @@ -63,8 +67,8 @@ public String name() { } /** - * Get the properties property: A list of provider-specific properties. - * + * Get the properties property: Provider properties. + * * @return the properties value. */ public ProviderProperties properties() { @@ -72,8 +76,8 @@ public ProviderProperties properties() { } /** - * Set the properties property: A list of provider-specific properties. - * + * Set the properties property: Provider properties. + * * @param properties the properties value to set. * @return the ProviderDescriptionInner object itself. */ @@ -84,7 +88,7 @@ public ProviderDescriptionInner withProperties(ProviderProperties properties) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/QuantumWorkspaceInner.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/QuantumWorkspaceInner.java index feb2d8c96e7c2..21d0ef1297e82 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/QuantumWorkspaceInner.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/QuantumWorkspaceInner.java @@ -15,7 +15,9 @@ import java.util.List; import java.util.Map; -/** The resource proxy definition object for quantum workspace. */ +/** + * The resource proxy definition object for quantum workspace. + */ @Fluent public final class QuantumWorkspaceInner extends Resource { /* @@ -31,18 +33,20 @@ public final class QuantumWorkspaceInner extends Resource { private QuantumWorkspaceIdentity identity; /* - * System metadata + * Azure Resource Manager metadata containing createdBy and modifiedBy information. */ @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) private SystemData systemData; - /** Creates an instance of QuantumWorkspaceInner class. */ + /** + * Creates an instance of QuantumWorkspaceInner class. + */ public QuantumWorkspaceInner() { } /** * Get the innerProperties property: Gets or sets the properties. Define quantum workspace's specific properties. - * + * * @return the innerProperties value. */ private WorkspaceResourceProperties innerProperties() { @@ -51,7 +55,7 @@ private WorkspaceResourceProperties innerProperties() { /** * Get the identity property: Managed Identity information. - * + * * @return the identity value. */ public QuantumWorkspaceIdentity identity() { @@ -60,7 +64,7 @@ public QuantumWorkspaceIdentity identity() { /** * Set the identity property: Managed Identity information. - * + * * @param identity the identity value to set. * @return the QuantumWorkspaceInner object itself. */ @@ -70,22 +74,26 @@ public QuantumWorkspaceInner withIdentity(QuantumWorkspaceIdentity identity) { } /** - * Get the systemData property: System metadata. - * + * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * * @return the systemData value. */ public SystemData systemData() { return this.systemData; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public QuantumWorkspaceInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public QuantumWorkspaceInner withTags(Map tags) { super.withTags(tags); @@ -94,7 +102,7 @@ public QuantumWorkspaceInner withTags(Map tags) { /** * Get the providers property: List of Providers selected for this Workspace. - * + * * @return the providers value. */ public List providers() { @@ -103,7 +111,7 @@ public List providers() { /** * Set the providers property: List of Providers selected for this Workspace. - * + * * @param providers the providers value to set. * @return the QuantumWorkspaceInner object itself. */ @@ -117,7 +125,7 @@ public QuantumWorkspaceInner withProviders(List providers) { /** * Get the usable property: Whether the current workspace is ready to accept Jobs. - * + * * @return the usable value. */ public UsableStatus usable() { @@ -126,7 +134,7 @@ public UsableStatus usable() { /** * Get the provisioningState property: Provisioning status field. - * + * * @return the provisioningState value. */ public ProvisioningStatus provisioningState() { @@ -135,7 +143,7 @@ public ProvisioningStatus provisioningState() { /** * Get the storageAccount property: ARM Resource Id of the storage account associated with this workspace. - * + * * @return the storageAccount value. */ public String storageAccount() { @@ -144,7 +152,7 @@ public String storageAccount() { /** * Set the storageAccount property: ARM Resource Id of the storage account associated with this workspace. - * + * * @param storageAccount the storageAccount value to set. * @return the QuantumWorkspaceInner object itself. */ @@ -158,16 +166,39 @@ public QuantumWorkspaceInner withStorageAccount(String storageAccount) { /** * Get the endpointUri property: The URI of the workspace endpoint. - * + * * @return the endpointUri value. */ public String endpointUri() { return this.innerProperties() == null ? null : this.innerProperties().endpointUri(); } + /** + * Get the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @return the apiKeyEnabled value. + */ + public Boolean apiKeyEnabled() { + return this.innerProperties() == null ? null : this.innerProperties().apiKeyEnabled(); + } + + /** + * Set the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @param apiKeyEnabled the apiKeyEnabled value to set. + * @return the QuantumWorkspaceInner object itself. + */ + public QuantumWorkspaceInner withApiKeyEnabled(Boolean apiKeyEnabled) { + if (this.innerProperties() == null) { + this.innerProperties = new WorkspaceResourceProperties(); + } + this.innerProperties().withApiKeyEnabled(apiKeyEnabled); + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/WorkspaceResourceProperties.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/WorkspaceResourceProperties.java index 5500f424b8827..c2575b4554b7f 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/WorkspaceResourceProperties.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/WorkspaceResourceProperties.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Properties of a Workspace. */ +/** + * Properties of a Workspace. + */ @Fluent public final class WorkspaceResourceProperties { /* @@ -44,13 +46,21 @@ public final class WorkspaceResourceProperties { @JsonProperty(value = "endpointUri", access = JsonProperty.Access.WRITE_ONLY) private String endpointUri; - /** Creates an instance of WorkspaceResourceProperties class. */ + /* + * Indicator of enablement of the Quantum workspace Api keys. + */ + @JsonProperty(value = "apiKeyEnabled") + private Boolean apiKeyEnabled; + + /** + * Creates an instance of WorkspaceResourceProperties class. + */ public WorkspaceResourceProperties() { } /** * Get the providers property: List of Providers selected for this Workspace. - * + * * @return the providers value. */ public List providers() { @@ -59,7 +69,7 @@ public List providers() { /** * Set the providers property: List of Providers selected for this Workspace. - * + * * @param providers the providers value to set. * @return the WorkspaceResourceProperties object itself. */ @@ -70,7 +80,7 @@ public WorkspaceResourceProperties withProviders(List providers) { /** * Get the usable property: Whether the current workspace is ready to accept Jobs. - * + * * @return the usable value. */ public UsableStatus usable() { @@ -79,7 +89,7 @@ public UsableStatus usable() { /** * Get the provisioningState property: Provisioning status field. - * + * * @return the provisioningState value. */ public ProvisioningStatus provisioningState() { @@ -88,7 +98,7 @@ public ProvisioningStatus provisioningState() { /** * Get the storageAccount property: ARM Resource Id of the storage account associated with this workspace. - * + * * @return the storageAccount value. */ public String storageAccount() { @@ -97,7 +107,7 @@ public String storageAccount() { /** * Set the storageAccount property: ARM Resource Id of the storage account associated with this workspace. - * + * * @param storageAccount the storageAccount value to set. * @return the WorkspaceResourceProperties object itself. */ @@ -108,16 +118,36 @@ public WorkspaceResourceProperties withStorageAccount(String storageAccount) { /** * Get the endpointUri property: The URI of the workspace endpoint. - * + * * @return the endpointUri value. */ public String endpointUri() { return this.endpointUri; } + /** + * Get the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @return the apiKeyEnabled value. + */ + public Boolean apiKeyEnabled() { + return this.apiKeyEnabled; + } + + /** + * Set the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @param apiKeyEnabled the apiKeyEnabled value to set. + * @return the WorkspaceResourceProperties object itself. + */ + public WorkspaceResourceProperties withApiKeyEnabled(Boolean apiKeyEnabled) { + this.apiKeyEnabled = apiKeyEnabled; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/package-info.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/package-info.java index 4938a46816b86..97e1040ffdbe7 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/package-info.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the inner data models for AzureQuantumManagementClient. null. */ +/** + * Package containing the inner data models for AzureQuantumManagementClient. + * null. + */ package com.azure.resourcemanager.quantum.fluent.models; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/package-info.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/package-info.java index 041a6f802ab8d..3ce074a39d8d7 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/package-info.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/fluent/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the service clients for AzureQuantumManagementClient. null. */ +/** + * Package containing the service clients for AzureQuantumManagementClient. + * null. + */ package com.azure.resourcemanager.quantum.fluent; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientBuilder.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientBuilder.java index ba73cc1c76ec2..ac063248595fa 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientBuilder.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientBuilder.java @@ -14,17 +14,19 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the AzureQuantumManagementClientImpl type. */ -@ServiceClientBuilder(serviceClients = {AzureQuantumManagementClientImpl.class}) +/** + * A builder for creating a new instance of the AzureQuantumManagementClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { AzureQuantumManagementClientImpl.class }) public final class AzureQuantumManagementClientBuilder { /* - * The Azure subscription ID. + * The ID of the target subscription. The value must be an UUID. */ private String subscriptionId; /** - * Sets The Azure subscription ID. - * + * Sets The ID of the target subscription. The value must be an UUID. + * * @param subscriptionId the subscriptionId value. * @return the AzureQuantumManagementClientBuilder. */ @@ -40,7 +42,7 @@ public AzureQuantumManagementClientBuilder subscriptionId(String subscriptionId) /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the AzureQuantumManagementClientBuilder. */ @@ -56,7 +58,7 @@ public AzureQuantumManagementClientBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the AzureQuantumManagementClientBuilder. */ @@ -72,7 +74,7 @@ public AzureQuantumManagementClientBuilder environment(AzureEnvironment environm /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the AzureQuantumManagementClientBuilder. */ @@ -88,7 +90,7 @@ public AzureQuantumManagementClientBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the AzureQuantumManagementClientBuilder. */ @@ -104,7 +106,7 @@ public AzureQuantumManagementClientBuilder defaultPollInterval(Duration defaultP /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the AzureQuantumManagementClientBuilder. */ @@ -115,30 +117,20 @@ public AzureQuantumManagementClientBuilder serializerAdapter(SerializerAdapter s /** * Builds an instance of AzureQuantumManagementClientImpl with the provided parameters. - * + * * @return an instance of AzureQuantumManagementClientImpl. */ public AzureQuantumManagementClientImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - AzureQuantumManagementClientImpl client = - new AzureQuantumManagementClientImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + AzureQuantumManagementClientImpl client = new AzureQuantumManagementClientImpl(localPipeline, + localSerializerAdapter, localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientImpl.java index 2229df61e391f..25a8da29e3990 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/AzureQuantumManagementClientImpl.java @@ -36,123 +36,145 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the AzureQuantumManagementClientImpl type. */ +/** + * Initializes a new instance of the AzureQuantumManagementClientImpl type. + */ @ServiceClient(builder = AzureQuantumManagementClientBuilder.class) public final class AzureQuantumManagementClientImpl implements AzureQuantumManagementClient { - /** The Azure subscription ID. */ + /** + * The ID of the target subscription. The value must be an UUID. + */ private final String subscriptionId; /** - * Gets The Azure subscription ID. - * + * Gets The ID of the target subscription. The value must be an UUID. + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The WorkspacesClient object to access its operations. */ + /** + * The WorkspacesClient object to access its operations. + */ private final WorkspacesClient workspaces; /** * Gets the WorkspacesClient object to access its operations. - * + * * @return the WorkspacesClient object. */ public WorkspacesClient getWorkspaces() { return this.workspaces; } - /** The OfferingsClient object to access its operations. */ + /** + * The OfferingsClient object to access its operations. + */ private final OfferingsClient offerings; /** * Gets the OfferingsClient object to access its operations. - * + * * @return the OfferingsClient object. */ public OfferingsClient getOfferings() { return this.offerings; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The WorkspaceOperationsClient object to access its operations. */ + /** + * The WorkspaceOperationsClient object to access its operations. + */ private final WorkspaceOperationsClient workspaceOperations; /** * Gets the WorkspaceOperationsClient object to access its operations. - * + * * @return the WorkspaceOperationsClient object. */ public WorkspaceOperationsClient getWorkspaceOperations() { @@ -161,27 +183,22 @@ public WorkspaceOperationsClient getWorkspaceOperations() { /** * Initializes an instance of AzureQuantumManagementClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. * @param environment The Azure environment. - * @param subscriptionId The Azure subscription ID. + * @param subscriptionId The ID of the target subscription. The value must be an UUID. * @param endpoint server parameter. */ - AzureQuantumManagementClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + AzureQuantumManagementClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2022-01-10-preview"; + this.apiVersion = "2023-11-13-preview"; this.workspaces = new WorkspacesClientImpl(this); this.offerings = new OfferingsClientImpl(this); this.operations = new OperationsClientImpl(this); @@ -190,7 +207,7 @@ public WorkspaceOperationsClient getWorkspaceOperations() { /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -199,7 +216,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -209,7 +226,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -219,26 +236,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -251,19 +257,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/CheckNameAvailabilityResultImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/CheckNameAvailabilityResultImpl.java index 99933db4068fd..be6ba9505006e 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/CheckNameAvailabilityResultImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/CheckNameAvailabilityResultImpl.java @@ -12,8 +12,7 @@ public final class CheckNameAvailabilityResultImpl implements CheckNameAvailabil private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; - CheckNameAvailabilityResultImpl( - CheckNameAvailabilityResultInner innerObject, + CheckNameAvailabilityResultImpl(CheckNameAvailabilityResultInner innerObject, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ListKeysResultImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ListKeysResultImpl.java new file mode 100644 index 0000000000000..f93c3102ad128 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ListKeysResultImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.implementation; + +import com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner; +import com.azure.resourcemanager.quantum.models.ApiKey; +import com.azure.resourcemanager.quantum.models.ListKeysResult; + +public final class ListKeysResultImpl implements ListKeysResult { + private ListKeysResultInner innerObject; + + private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; + + ListKeysResultImpl(ListKeysResultInner innerObject, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public Boolean apiKeyEnabled() { + return this.innerModel().apiKeyEnabled(); + } + + public ApiKey primaryKey() { + return this.innerModel().primaryKey(); + } + + public ApiKey secondaryKey() { + return this.innerModel().secondaryKey(); + } + + public String primaryConnectionString() { + return this.innerModel().primaryConnectionString(); + } + + public String secondaryConnectionString() { + return this.innerModel().secondaryConnectionString(); + } + + public ListKeysResultInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.quantum.AzureQuantumManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsClientImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsClientImpl.java index d650e0038f1d2..fa26c6ccceb54 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsClientImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsClientImpl.java @@ -30,130 +30,106 @@ import com.azure.resourcemanager.quantum.models.OfferingsListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OfferingsClient. */ +/** + * An instance of this class provides access to all the operations defined in OfferingsClient. + */ public final class OfferingsClientImpl implements OfferingsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OfferingsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final AzureQuantumManagementClientImpl client; /** * Initializes an instance of OfferingsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OfferingsClientImpl(AzureQuantumManagementClientImpl client) { - this.service = - RestProxy.create(OfferingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OfferingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for AzureQuantumManagementClientOfferings to be used by the proxy service - * to perform REST calls. + * The interface defining all the services for AzureQuantumManagementClientOfferings to be used by the proxy + * service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "AzureQuantumManageme") public interface OfferingsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Quantum/locations/{locationName}/offerings") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @PathParam("locationName") String locationName, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @PathParam("locationName") String locationName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Providers operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String locationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - locationName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), locationName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Providers operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String locationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); @@ -161,27 +137,15 @@ private Mono> listSinglePageAsync(String final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - locationName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), locationName, + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -195,7 +159,7 @@ private PagedFlux listAsync(String locationName) { /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -205,13 +169,13 @@ private PagedFlux listAsync(String locationName) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String locationName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(locationName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(locationName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -225,7 +189,7 @@ public PagedIterable list(String locationName) { /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -240,14 +204,15 @@ public PagedIterable list(String locationName, Context /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Providers operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -255,37 +220,28 @@ private Mono> listNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Providers operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -293,23 +249,13 @@ private Mono> listNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsImpl.java index d399e6b37c017..5e84df36d44c4 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OfferingsImpl.java @@ -19,20 +19,20 @@ public final class OfferingsImpl implements Offerings { private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; - public OfferingsImpl( - OfferingsClient innerClient, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + public OfferingsImpl(OfferingsClient innerClient, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String locationName) { PagedIterable inner = this.serviceClient().list(locationName); - return Utils.mapPage(inner, inner1 -> new ProviderDescriptionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ProviderDescriptionImpl(inner1, this.manager())); } public PagedIterable list(String locationName, Context context) { PagedIterable inner = this.serviceClient().list(locationName, context); - return Utils.mapPage(inner, inner1 -> new ProviderDescriptionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ProviderDescriptionImpl(inner1, this.manager())); } private OfferingsClient serviceClient() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsClientImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsClientImpl.java index 838168c503332..41c36ef53a0eb 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsClientImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.quantum.models.OperationsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final AzureQuantumManagementClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(AzureQuantumManagementClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,30 +62,24 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "AzureQuantumManageme") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Quantum/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Returns list of operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return lists the operations available along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -87,30 +87,21 @@ Mono> listNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Returns list of operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -120,29 +111,19 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Returns list of operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return lists the operations available as paginated response with {@link PagedFlux}. @@ -154,7 +135,7 @@ private PagedFlux listAsync() { /** * Returns list of operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -163,13 +144,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Returns list of operations. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return lists the operations available as paginated response with {@link PagedIterable}. @@ -181,7 +162,7 @@ public PagedIterable list() { /** * Returns list of operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -195,9 +176,10 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -209,31 +191,22 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -246,23 +219,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsImpl.java index eef200082da48..ec0cc3e556175 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/OperationsImpl.java @@ -19,20 +19,20 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + public OperationsImpl(OperationsClient innerClient, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ProviderDescriptionImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ProviderDescriptionImpl.java index b3068ee6a14f5..51527e0b96c6f 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ProviderDescriptionImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ProviderDescriptionImpl.java @@ -13,8 +13,8 @@ public final class ProviderDescriptionImpl implements ProviderDescription { private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; - ProviderDescriptionImpl( - ProviderDescriptionInner innerObject, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + ProviderDescriptionImpl(ProviderDescriptionInner innerObject, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/QuantumWorkspaceImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/QuantumWorkspaceImpl.java index e704154bc2c50..ac7d2eb64feab 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/QuantumWorkspaceImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/QuantumWorkspaceImpl.java @@ -82,6 +82,10 @@ public String endpointUri() { return this.innerModel().endpointUri(); } + public Boolean apiKeyEnabled() { + return this.innerModel().apiKeyEnabled(); + } + public Region region() { return Region.fromName(this.regionName()); } @@ -114,20 +118,14 @@ public QuantumWorkspaceImpl withExistingResourceGroup(String resourceGroupName) } public QuantumWorkspace create() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .createOrUpdate(resourceGroupName, workspaceName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getWorkspaces().createOrUpdate(resourceGroupName, + workspaceName, this.innerModel(), Context.NONE); return this; } public QuantumWorkspace create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .createOrUpdate(resourceGroupName, workspaceName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getWorkspaces().createOrUpdate(resourceGroupName, + workspaceName, this.innerModel(), context); return this; } @@ -143,50 +141,34 @@ public QuantumWorkspaceImpl update() { } public QuantumWorkspace apply() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .updateTagsWithResponse(resourceGroupName, workspaceName, updateWorkspaceTags, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .updateTagsWithResponse(resourceGroupName, workspaceName, updateWorkspaceTags, Context.NONE).getValue(); return this; } public QuantumWorkspace apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .updateTagsWithResponse(resourceGroupName, workspaceName, updateWorkspaceTags, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .updateTagsWithResponse(resourceGroupName, workspaceName, updateWorkspaceTags, context).getValue(); return this; } - QuantumWorkspaceImpl( - QuantumWorkspaceInner innerObject, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + QuantumWorkspaceImpl(QuantumWorkspaceInner innerObject, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.workspaceName = Utils.getValueFromIdByName(innerObject.id(), "workspaces"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.workspaceName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "workspaces"); } public QuantumWorkspace refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .getByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .getByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE).getValue(); return this; } public QuantumWorkspace refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaces() - .getByResourceGroupWithResponse(resourceGroupName, workspaceName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaces() + .getByResourceGroupWithResponse(resourceGroupName, workspaceName, context).getValue(); return this; } @@ -225,6 +207,11 @@ public QuantumWorkspaceImpl withStorageAccount(String storageAccount) { return this; } + public QuantumWorkspaceImpl withApiKeyEnabled(Boolean apiKeyEnabled) { + this.innerModel().withApiKeyEnabled(apiKeyEnabled); + return this; + } + private boolean isInCreateMode() { return this.innerModel().id() == null; } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ResourceManagerUtils.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..80193f25918bc --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/Utils.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/Utils.java deleted file mode 100644 index 49c15a11c078d..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsClientImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsClientImpl.java index 717fc5552d86b..f3844e12ba27a 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsClientImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsClientImpl.java @@ -24,25 +24,33 @@ import com.azure.core.util.FluxUtil; import com.azure.resourcemanager.quantum.fluent.WorkspaceOperationsClient; import com.azure.resourcemanager.quantum.fluent.models.CheckNameAvailabilityResultInner; +import com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner; +import com.azure.resourcemanager.quantum.models.ApiKeys; import com.azure.resourcemanager.quantum.models.CheckNameAvailabilityParameters; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in WorkspaceOperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspaceOperationsClient. + */ public final class WorkspaceOperationsClientImpl implements WorkspaceOperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final WorkspaceOperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final AzureQuantumManagementClientImpl client; /** * Initializes an instance of WorkspaceOperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ WorkspaceOperationsClientImpl(AzureQuantumManagementClientImpl client) { - this.service = - RestProxy.create(WorkspaceOperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(WorkspaceOperationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -53,24 +61,39 @@ public final class WorkspaceOperationsClientImpl implements WorkspaceOperationsC @Host("{$host}") @ServiceInterface(name = "AzureQuantumManageme") public interface WorkspaceOperationsService { - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Quantum/locations/{locationName}/checkNameAvailability") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Quantum/locations/{locationName}/checkNameAvailability") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> checkNameAvailability( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, + Mono> checkNameAvailability(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, @PathParam("locationName") String locationName, @BodyParam("application/json") CheckNameAvailabilityParameters checkNameAvailabilityParameters, - @HeaderParam("Accept") String accept, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}/listKeys") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listKeys(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}/regenerateKey") + @ExpectedResponses({ 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> regenerateKeys(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("workspaceName") String workspaceName, + @BodyParam("application/json") ApiKeys keySpecification, @HeaderParam("Accept") String accept, Context context); } /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,50 +102,36 @@ Mono> checkNameAvailability( * @return result of check name availability along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> checkNameAvailabilityWithResponseAsync( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters) { + private Mono> checkNameAvailabilityWithResponseAsync(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); } if (checkNameAvailabilityParameters == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter checkNameAvailabilityParameters is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter checkNameAvailabilityParameters is required and cannot be null.")); } else { checkNameAvailabilityParameters.validate(); } final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - locationName, - checkNameAvailabilityParameters, - accept, - context)) + context -> service.checkNameAvailability(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), locationName, checkNameAvailabilityParameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @param context The context to associate with this operation. @@ -132,47 +141,34 @@ private Mono> checkNameAvailabilityWi * @return result of check name availability along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> checkNameAvailabilityWithResponseAsync( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context) { + private Mono> checkNameAvailabilityWithResponseAsync(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (locationName == null) { return Mono.error(new IllegalArgumentException("Parameter locationName is required and cannot be null.")); } if (checkNameAvailabilityParameters == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter checkNameAvailabilityParameters is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter checkNameAvailabilityParameters is required and cannot be null.")); } else { checkNameAvailabilityParameters.validate(); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .checkNameAvailability( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - this.client.getApiVersion(), - locationName, - checkNameAvailabilityParameters, - accept, - context); + return service.checkNameAvailability(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), locationName, checkNameAvailabilityParameters, accept, context); } /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -181,15 +177,15 @@ private Mono> checkNameAvailabilityWi * @return result of check name availability on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono checkNameAvailabilityAsync( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters) { + private Mono checkNameAvailabilityAsync(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters) { return checkNameAvailabilityWithResponseAsync(locationName, checkNameAvailabilityParameters) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @param context The context to associate with this operation. @@ -199,14 +195,14 @@ private Mono checkNameAvailabilityAsync( * @return result of check name availability along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response checkNameAvailabilityWithResponse( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context) { + public Response checkNameAvailabilityWithResponse(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context) { return checkNameAvailabilityWithResponseAsync(locationName, checkNameAvailabilityParameters, context).block(); } /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -215,9 +211,273 @@ public Response checkNameAvailabilityWithRespo * @return result of check name availability. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CheckNameAvailabilityResultInner checkNameAvailability( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters) { + public CheckNameAvailabilityResultInner checkNameAvailability(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters) { return checkNameAvailabilityWithResponse(locationName, checkNameAvailabilityParameters, Context.NONE) .getValue(); } + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listKeysWithResponseAsync(String resourceGroupName, + String workspaceName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), resourceGroupName, workspaceName, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listKeysWithResponseAsync(String resourceGroupName, + String workspaceName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), this.client.getApiVersion(), + resourceGroupName, workspaceName, accept, context); + } + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listKeysAsync(String resourceGroupName, String workspaceName) { + return listKeysWithResponseAsync(resourceGroupName, workspaceName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listKeysWithResponse(String resourceGroupName, String workspaceName, + Context context) { + return listKeysWithResponseAsync(resourceGroupName, workspaceName, context).block(); + } + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ListKeysResultInner listKeys(String resourceGroupName, String workspaceName) { + return listKeysWithResponse(resourceGroupName, workspaceName, Context.NONE).getValue(); + } + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> regenerateKeysWithResponseAsync(String resourceGroupName, String workspaceName, + ApiKeys keySpecification) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (keySpecification == null) { + return Mono + .error(new IllegalArgumentException("Parameter keySpecification is required and cannot be null.")); + } else { + keySpecification.validate(); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.regenerateKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), resourceGroupName, workspaceName, keySpecification, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> regenerateKeysWithResponseAsync(String resourceGroupName, String workspaceName, + ApiKeys keySpecification, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (workspaceName == null) { + return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); + } + if (keySpecification == null) { + return Mono + .error(new IllegalArgumentException("Parameter keySpecification is required and cannot be null.")); + } else { + keySpecification.validate(); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.regenerateKeys(this.client.getEndpoint(), this.client.getSubscriptionId(), + this.client.getApiVersion(), resourceGroupName, workspaceName, keySpecification, accept, context); + } + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono regenerateKeysAsync(String resourceGroupName, String workspaceName, ApiKeys keySpecification) { + return regenerateKeysWithResponseAsync(resourceGroupName, workspaceName, keySpecification) + .flatMap(ignored -> Mono.empty()); + } + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response regenerateKeysWithResponse(String resourceGroupName, String workspaceName, + ApiKeys keySpecification, Context context) { + return regenerateKeysWithResponseAsync(resourceGroupName, workspaceName, keySpecification, context).block(); + } + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void regenerateKeys(String resourceGroupName, String workspaceName, ApiKeys keySpecification) { + regenerateKeysWithResponse(resourceGroupName, workspaceName, keySpecification, Context.NONE); + } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsImpl.java index 95ccbab12dd08..6c6f4245adf0b 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspaceOperationsImpl.java @@ -10,8 +10,11 @@ import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.quantum.fluent.WorkspaceOperationsClient; import com.azure.resourcemanager.quantum.fluent.models.CheckNameAvailabilityResultInner; +import com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner; +import com.azure.resourcemanager.quantum.models.ApiKeys; import com.azure.resourcemanager.quantum.models.CheckNameAvailabilityParameters; import com.azure.resourcemanager.quantum.models.CheckNameAvailabilityResult; +import com.azure.resourcemanager.quantum.models.ListKeysResult; import com.azure.resourcemanager.quantum.models.WorkspaceOperations; public final class WorkspaceOperationsImpl implements WorkspaceOperations { @@ -21,33 +24,28 @@ public final class WorkspaceOperationsImpl implements WorkspaceOperations { private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; - public WorkspaceOperationsImpl( - WorkspaceOperationsClient innerClient, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + public WorkspaceOperationsImpl(WorkspaceOperationsClient innerClient, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response checkNameAvailabilityWithResponse( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context) { - Response inner = - this - .serviceClient() - .checkNameAvailabilityWithResponse(locationName, checkNameAvailabilityParameters, context); + public Response checkNameAvailabilityWithResponse(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context) { + Response inner = this.serviceClient() + .checkNameAvailabilityWithResponse(locationName, checkNameAvailabilityParameters, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CheckNameAvailabilityResultImpl(inner.getValue(), this.manager())); } else { return null; } } - public CheckNameAvailabilityResult checkNameAvailability( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters) { - CheckNameAvailabilityResultInner inner = - this.serviceClient().checkNameAvailability(locationName, checkNameAvailabilityParameters); + public CheckNameAvailabilityResult checkNameAvailability(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters) { + CheckNameAvailabilityResultInner inner + = this.serviceClient().checkNameAvailability(locationName, checkNameAvailabilityParameters); if (inner != null) { return new CheckNameAvailabilityResultImpl(inner, this.manager()); } else { @@ -55,6 +53,37 @@ public CheckNameAvailabilityResult checkNameAvailability( } } + public Response listKeysWithResponse(String resourceGroupName, String workspaceName, + Context context) { + Response inner + = this.serviceClient().listKeysWithResponse(resourceGroupName, workspaceName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ListKeysResultImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ListKeysResult listKeys(String resourceGroupName, String workspaceName) { + ListKeysResultInner inner = this.serviceClient().listKeys(resourceGroupName, workspaceName); + if (inner != null) { + return new ListKeysResultImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response regenerateKeysWithResponse(String resourceGroupName, String workspaceName, + ApiKeys keySpecification, Context context) { + return this.serviceClient().regenerateKeysWithResponse(resourceGroupName, workspaceName, keySpecification, + context); + } + + public void regenerateKeys(String resourceGroupName, String workspaceName, ApiKeys keySpecification) { + this.serviceClient().regenerateKeys(resourceGroupName, workspaceName, keySpecification); + } + private WorkspaceOperationsClient serviceClient() { return this.innerClient; } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesClientImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesClientImpl.java index 23014563643fb..e206cc1df543a 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesClientImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesClientImpl.java @@ -40,22 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in WorkspacesClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspacesClient. + */ public final class WorkspacesClientImpl implements WorkspacesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final WorkspacesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final AzureQuantumManagementClientImpl client; /** * Initializes an instance of WorkspacesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ WorkspacesClientImpl(AzureQuantumManagementClientImpl client) { - this.service = - RestProxy.create(WorkspacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(WorkspacesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,209 +72,153 @@ public final class WorkspacesClientImpl implements WorkspacesClient { @Host("{$host}") @ServiceInterface(name = "AzureQuantumManageme") public interface WorkspacesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("workspaceName") String workspaceName, - @HeaderParam("Accept") String accept, - Context context); + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("workspaceName") String workspaceName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> createOrUpdate( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("workspaceName") String workspaceName, - @BodyParam("application/json") QuantumWorkspaceInner quantumWorkspace, - @HeaderParam("Accept") String accept, + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("workspaceName") String workspaceName, + @BodyParam("application/json") QuantumWorkspaceInner quantumWorkspace, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateTags( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("workspaceName") String workspaceName, - @BodyParam("application/json") TagsObject workspaceTags, - @HeaderParam("Accept") String accept, + Mono> updateTags(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("workspaceName") String workspaceName, + @BodyParam("application/json") TagsObject workspaceTags, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces/{workspaceName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("workspaceName") String workspaceName, - @HeaderParam("Accept") String accept, - Context context); + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("workspaceName") String workspaceName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Quantum/workspaces") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Quantum/workspaces") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String workspaceName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String workspaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), workspaceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String workspaceName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String workspaceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), workspaceName, accept, context); } /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -283,8 +233,8 @@ private Mono getByResourceGroupAsync(String resourceGroup /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -293,15 +243,15 @@ private Mono getByResourceGroupAsync(String resourceGroup * @return the resource proxy definition object for quantum workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String workspaceName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, workspaceName, context).block(); } /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -315,34 +265,30 @@ public QuantumWorkspaceInner getByResourceGroup(String resourceGroupName, String /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String workspaceName, QuantumWorkspaceInner quantumWorkspace) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); @@ -355,25 +301,16 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - quantumWorkspace, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), workspaceName, quantumWorkspace, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -381,26 +318,22 @@ private Mono>> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createOrUpdateWithResponseAsync( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context) { + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); @@ -413,22 +346,14 @@ private Mono>> createOrUpdateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - quantumWorkspace, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), workspaceName, quantumWorkspace, accept, context); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -439,22 +364,17 @@ private Mono>> createOrUpdateWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, QuantumWorkspaceInner> beginCreateOrUpdateAsync( String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace) { - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, workspaceName, quantumWorkspace); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - QuantumWorkspaceInner.class, - QuantumWorkspaceInner.class, - this.client.getContext()); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, workspaceName, quantumWorkspace); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), QuantumWorkspaceInner.class, QuantumWorkspaceInner.class, + this.client.getContext()); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -467,18 +387,16 @@ private PollerFlux, QuantumWorkspaceInner> beg private PollerFlux, QuantumWorkspaceInner> beginCreateOrUpdateAsync( String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createOrUpdateWithResponseAsync(resourceGroupName, workspaceName, quantumWorkspace, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), QuantumWorkspaceInner.class, QuantumWorkspaceInner.class, context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, workspaceName, quantumWorkspace, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), QuantumWorkspaceInner.class, QuantumWorkspaceInner.class, context); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -487,15 +405,15 @@ private PollerFlux, QuantumWorkspaceInner> beg * @return the {@link SyncPoller} for polling of the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, QuantumWorkspaceInner> beginCreateOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace) { + public SyncPoller, QuantumWorkspaceInner> + beginCreateOrUpdate(String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace) { return this.beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace).getSyncPoller(); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -507,15 +425,14 @@ public SyncPoller, QuantumWorkspaceInner> begi @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, QuantumWorkspaceInner> beginCreateOrUpdate( String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context) { - return this - .beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace, context) + return this.beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace, context) .getSyncPoller(); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -524,17 +441,16 @@ public SyncPoller, QuantumWorkspaceInner> begi * @return the resource proxy definition object for quantum workspace on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace) { - return beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String workspaceName, + QuantumWorkspaceInner quantumWorkspace) { + return beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -544,17 +460,16 @@ private Mono createOrUpdateAsync( * @return the resource proxy definition object for quantum workspace on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context) { - return beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace, context) - .last() + private Mono createOrUpdateAsync(String resourceGroupName, String workspaceName, + QuantumWorkspaceInner quantumWorkspace, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -563,15 +478,15 @@ private Mono createOrUpdateAsync( * @return the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.SINGLE) - public QuantumWorkspaceInner createOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace) { + public QuantumWorkspaceInner createOrUpdate(String resourceGroupName, String workspaceName, + QuantumWorkspaceInner quantumWorkspace) { return createOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace).block(); } /** * Creates or updates a workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param quantumWorkspace Workspace details. * @param context The context to associate with this operation. @@ -581,41 +496,37 @@ public QuantumWorkspaceInner createOrUpdate( * @return the resource proxy definition object for quantum workspace. */ @ServiceMethod(returns = ReturnType.SINGLE) - public QuantumWorkspaceInner createOrUpdate( - String resourceGroupName, String workspaceName, QuantumWorkspaceInner quantumWorkspace, Context context) { + public QuantumWorkspaceInner createOrUpdate(String resourceGroupName, String workspaceName, + QuantumWorkspaceInner quantumWorkspace, Context context) { return createOrUpdateAsync(resourceGroupName, workspaceName, quantumWorkspace, context).block(); } /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String workspaceName, TagsObject workspaceTags) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String workspaceName, TagsObject workspaceTags) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); @@ -628,24 +539,15 @@ private Mono> updateTagsWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .updateTags( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - workspaceTags, - accept, - context)) + context -> service.updateTags(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), workspaceName, workspaceTags, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @param context The context to associate with this operation. @@ -653,26 +555,22 @@ private Mono> updateTagsWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateTagsWithResponseAsync( - String resourceGroupName, String workspaceName, TagsObject workspaceTags, Context context) { + private Mono> updateTagsWithResponseAsync(String resourceGroupName, + String workspaceName, TagsObject workspaceTags, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); @@ -684,22 +582,14 @@ private Mono> updateTagsWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateTags( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - workspaceTags, - accept, - context); + return service.updateTags(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), workspaceName, workspaceTags, accept, context); } /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -708,16 +598,16 @@ private Mono> updateTagsWithResponseAsync( * @return the resource proxy definition object for quantum workspace on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateTagsAsync( - String resourceGroupName, String workspaceName, TagsObject workspaceTags) { + private Mono updateTagsAsync(String resourceGroupName, String workspaceName, + TagsObject workspaceTags) { return updateTagsWithResponseAsync(resourceGroupName, workspaceName, workspaceTags) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @param context The context to associate with this operation. @@ -727,15 +617,15 @@ private Mono updateTagsAsync( * @return the resource proxy definition object for quantum workspace along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateTagsWithResponse( - String resourceGroupName, String workspaceName, TagsObject workspaceTags, Context context) { + public Response updateTagsWithResponse(String resourceGroupName, String workspaceName, + TagsObject workspaceTags, Context context) { return updateTagsWithResponseAsync(resourceGroupName, workspaceName, workspaceTags, context).block(); } /** * Updates an existing workspace's tags. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param workspaceTags Parameters supplied to update tags. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -750,8 +640,8 @@ public QuantumWorkspaceInner updateTags(String resourceGroupName, String workspa /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -761,44 +651,31 @@ public QuantumWorkspaceInner updateTags(String resourceGroupName, String workspa @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String workspaceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), workspaceName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -807,44 +684,33 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String workspaceName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String workspaceName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceName == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - workspaceName, - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), workspaceName, accept, context); } /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -854,16 +720,14 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String workspaceName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, workspaceName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -872,19 +736,18 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String workspaceName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String workspaceName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, workspaceName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -898,8 +761,8 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -908,15 +771,15 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String workspaceName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String workspaceName, + Context context) { return this.beginDeleteAsync(resourceGroupName, workspaceName, context).getSyncPoller(); } /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -930,8 +793,8 @@ private Mono deleteAsync(String resourceGroupName, String workspaceName) { /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -941,15 +804,14 @@ private Mono deleteAsync(String resourceGroupName, String workspaceName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String workspaceName, Context context) { - return beginDeleteAsync(resourceGroupName, workspaceName, context) - .last() + return beginDeleteAsync(resourceGroupName, workspaceName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -962,8 +824,8 @@ public void delete(String resourceGroupName, String workspaceName) { /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -977,109 +839,76 @@ public void delete(String resourceGroupName, String workspaceName, Context conte /** * Gets the list of Workspaces within a Subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a Subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the list of Workspaces within a Subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a Subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the list of Workspaces within a Subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a Subscription as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Gets the list of Workspaces within a Subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1088,13 +917,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Gets the list of Workspaces within a Subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a Subscription as paginated response with {@link PagedIterable}. @@ -1106,7 +935,7 @@ public PagedIterable list() { /** * Gets the list of Workspaces within a Subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1120,111 +949,76 @@ public PagedIterable list(Context context) { /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a resource group along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a resource group along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1232,15 +1026,14 @@ private Mono> listByResourceGroupSinglePage */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1249,15 +1042,14 @@ private PagedFlux listByResourceGroupAsync(String resourc */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1270,8 +1062,8 @@ public PagedIterable listByResourceGroup(String resourceG /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1285,14 +1077,15 @@ public PagedIterable listByResourceGroup(String resourceG /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Workspaces operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -1300,76 +1093,59 @@ private Mono> listBySubscriptionNextSingleP return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Workspaces operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Workspaces operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1377,63 +1153,45 @@ private Mono> listByResourceGroupNextSingle return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response of a list Workspaces operation along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesImpl.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesImpl.java index b21c1882a3faa..dbc1e50275437 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesImpl.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/WorkspacesImpl.java @@ -21,21 +21,18 @@ public final class WorkspacesImpl implements Workspaces { private final com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager; - public WorkspacesImpl( - WorkspacesClient innerClient, com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { + public WorkspacesImpl(WorkspacesClient innerClient, + com.azure.resourcemanager.quantum.AzureQuantumManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, workspaceName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, workspaceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new QuantumWorkspaceImpl(inner.getValue(), this.manager())); } else { return null; @@ -61,97 +58,77 @@ public void delete(String resourceGroupName, String workspaceName, Context conte public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new QuantumWorkspaceImpl(inner1, this.manager())); } public QuantumWorkspace getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, workspaceName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, workspaceName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } this.delete(resourceGroupName, workspaceName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String workspaceName = Utils.getValueFromIdByName(id, "workspaces"); + String workspaceName = ResourceManagerUtils.getValueFromIdByName(id, "workspaces"); if (workspaceName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaces'.", id))); } this.delete(resourceGroupName, workspaceName, context); } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/package-info.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/package-info.java index b633b5f9e9efa..da03b6a59a562 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/package-info.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/implementation/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the implementations for AzureQuantumManagementClient. null. */ +/** + * Package containing the implementations for AzureQuantumManagementClient. + * null. + */ package com.azure.resourcemanager.quantum.implementation; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKey.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKey.java new file mode 100644 index 0000000000000..2ccd978845535 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKey.java @@ -0,0 +1,70 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * Azure quantum workspace Api key details. + */ +@Fluent +public final class ApiKey { + /* + * The creation time of the api key. + */ + @JsonProperty(value = "createdAt") + private OffsetDateTime createdAt; + + /* + * The Api key. + */ + @JsonProperty(value = "key", access = JsonProperty.Access.WRITE_ONLY) + private String key; + + /** + * Creates an instance of ApiKey class. + */ + public ApiKey() { + } + + /** + * Get the createdAt property: The creation time of the api key. + * + * @return the createdAt value. + */ + public OffsetDateTime createdAt() { + return this.createdAt; + } + + /** + * Set the createdAt property: The creation time of the api key. + * + * @param createdAt the createdAt value to set. + * @return the ApiKey object itself. + */ + public ApiKey withCreatedAt(OffsetDateTime createdAt) { + this.createdAt = createdAt; + return this; + } + + /** + * Get the key property: The Api key. + * + * @return the key value. + */ + public String key() { + return this.key; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKeys.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKeys.java new file mode 100644 index 0000000000000..3997ee70d873c --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ApiKeys.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of api keys to be generated. + */ +@Fluent +public final class ApiKeys { + /* + * A list of api key names. + */ + @JsonProperty(value = "keys") + private List keys; + + /** + * Creates an instance of ApiKeys class. + */ + public ApiKeys() { + } + + /** + * Get the keys property: A list of api key names. + * + * @return the keys value. + */ + public List keys() { + return this.keys; + } + + /** + * Set the keys property: A list of api key names. + * + * @param keys the keys value to set. + * @return the ApiKeys object itself. + */ + public ApiKeys withKeys(List keys) { + this.keys = keys; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityParameters.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityParameters.java index f8954e4717fbd..3a9d19a3aa255 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityParameters.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityParameters.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of check name availability request body. */ +/** + * Details of check name availability request body. + */ @Fluent public final class CheckNameAvailabilityParameters { /* @@ -22,13 +24,15 @@ public final class CheckNameAvailabilityParameters { @JsonProperty(value = "type") private String type; - /** Creates an instance of CheckNameAvailabilityParameters class. */ + /** + * Creates an instance of CheckNameAvailabilityParameters class. + */ public CheckNameAvailabilityParameters() { } /** * Get the name property: Name for checking availability. - * + * * @return the name value. */ public String name() { @@ -37,7 +41,7 @@ public String name() { /** * Set the name property: Name for checking availability. - * + * * @param name the name value to set. * @return the CheckNameAvailabilityParameters object itself. */ @@ -48,7 +52,7 @@ public CheckNameAvailabilityParameters withName(String name) { /** * Get the type property: The resource type of Quantum Workspace. - * + * * @return the type value. */ public String type() { @@ -57,7 +61,7 @@ public String type() { /** * Set the type property: The resource type of Quantum Workspace. - * + * * @param type the type value to set. * @return the CheckNameAvailabilityParameters object itself. */ @@ -68,7 +72,7 @@ public CheckNameAvailabilityParameters withType(String type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityResult.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityResult.java index b641bfec86dc7..c75f5fd2fa568 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityResult.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/CheckNameAvailabilityResult.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.quantum.fluent.models.CheckNameAvailabilityResultInner; -/** An immutable client-side representation of CheckNameAvailabilityResult. */ +/** + * An immutable client-side representation of CheckNameAvailabilityResult. + */ public interface CheckNameAvailabilityResult { /** * Gets the nameAvailable property: Indicator of availability of the Quantum Workspace resource name. - * + * * @return the nameAvailable value. */ Boolean nameAvailable(); /** * Gets the reason property: The reason of unavailability. - * + * * @return the reason value. */ String reason(); /** * Gets the message property: The detailed info regarding the reason associated with the Namespace. - * + * * @return the message value. */ String message(); /** * Gets the inner com.azure.resourcemanager.quantum.fluent.models.CheckNameAvailabilityResultInner object. - * + * * @return the inner object. */ CheckNameAvailabilityResultInner innerModel(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/KeyType.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/KeyType.java new file mode 100644 index 0000000000000..e941f698f0f91 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/KeyType.java @@ -0,0 +1,53 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Defines values for KeyType. + */ +public final class KeyType extends ExpandableStringEnum { + /** + * Static value Primary for KeyType. + */ + public static final KeyType PRIMARY = fromString("Primary"); + + /** + * Static value Secondary for KeyType. + */ + public static final KeyType SECONDARY = fromString("Secondary"); + + /** + * Creates a new instance of KeyType value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public KeyType() { + } + + /** + * Creates or finds a KeyType from its string representation. + * + * @param name a name to look for. + * @return the corresponding KeyType. + */ + @JsonCreator + public static KeyType fromString(String name) { + return fromString(name, KeyType.class); + } + + /** + * Gets known KeyType values. + * + * @return known KeyType values. + */ + public static Collection values() { + return values(KeyType.class); + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ListKeysResult.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ListKeysResult.java new file mode 100644 index 0000000000000..a1cf8cc247dd3 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ListKeysResult.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.models; + +import com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner; + +/** + * An immutable client-side representation of ListKeysResult. + */ +public interface ListKeysResult { + /** + * Gets the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @return the apiKeyEnabled value. + */ + Boolean apiKeyEnabled(); + + /** + * Gets the primaryKey property: The quantum workspace primary api key. + * + * @return the primaryKey value. + */ + ApiKey primaryKey(); + + /** + * Gets the secondaryKey property: The quantum workspace secondary api key. + * + * @return the secondaryKey value. + */ + ApiKey secondaryKey(); + + /** + * Gets the primaryConnectionString property: The connection string of the primary api key. + * + * @return the primaryConnectionString value. + */ + String primaryConnectionString(); + + /** + * Gets the secondaryConnectionString property: The connection string of the secondary api key. + * + * @return the secondaryConnectionString value. + */ + String secondaryConnectionString(); + + /** + * Gets the inner com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner object. + * + * @return the inner object. + */ + ListKeysResultInner innerModel(); +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Offerings.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Offerings.java index 70cdfc72c1cbf..218051dd38983 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Offerings.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Offerings.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Offerings. */ +/** + * Resource collection API of Offerings. + */ public interface Offerings { /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -22,7 +24,7 @@ public interface Offerings { /** * Returns the list of all provider offerings available for the given location. - * + * * @param locationName Location. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OfferingsListResult.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OfferingsListResult.java index 4922cd3232189..d0758647272b5 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OfferingsListResult.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OfferingsListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a list Providers operation. */ +/** + * The response of a list Providers operation. + */ @Fluent public final class OfferingsListResult { /* @@ -24,13 +26,15 @@ public final class OfferingsListResult { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of OfferingsListResult class. */ + /** + * Creates an instance of OfferingsListResult class. + */ public OfferingsListResult() { } /** * Get the value property: Result of a list Providers operation. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: Result of a list Providers operation. - * + * * @param value the value value to set. * @return the OfferingsListResult object itself. */ @@ -51,7 +55,7 @@ public OfferingsListResult withValue(List value) { /** * Get the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of * Providers. - * + * * @return the nextLink value. */ public String nextLink() { @@ -61,7 +65,7 @@ public String nextLink() { /** * Set the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of * Providers. - * + * * @param nextLink the nextLink value to set. * @return the OfferingsListResult object itself. */ @@ -72,7 +76,7 @@ public OfferingsListResult withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operation.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operation.java index 263e3b656555a..71c941240c5ed 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operation.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operation.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.quantum.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: Name of the operation. - * + * * @return the name value. */ String name(); /** * Gets the isDataAction property: Indicates whether the operation is a data action. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Properties of the operation. - * + * * @return the display value. */ OperationDisplay display(); /** * Gets the inner com.azure.resourcemanager.quantum.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationDisplay.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationDisplay.java index d5eb519e4a4aa..c5f949c542432 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationDisplay.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Properties of the operation. */ +/** + * Properties of the operation. + */ @Fluent public final class OperationDisplay { /* @@ -34,13 +36,15 @@ public final class OperationDisplay { @JsonProperty(value = "description") private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: Provider name. - * + * * @return the provider value. */ public String provider() { @@ -49,7 +53,7 @@ public String provider() { /** * Set the provider property: Provider name. - * + * * @param provider the provider value to set. * @return the OperationDisplay object itself. */ @@ -60,7 +64,7 @@ public OperationDisplay withProvider(String provider) { /** * Get the resource property: Resource name. - * + * * @return the resource value. */ public String resource() { @@ -69,7 +73,7 @@ public String resource() { /** * Set the resource property: Resource name. - * + * * @param resource the resource value to set. * @return the OperationDisplay object itself. */ @@ -80,7 +84,7 @@ public OperationDisplay withResource(String resource) { /** * Get the operation property: Operation name. - * + * * @return the operation value. */ public String operation() { @@ -89,7 +93,7 @@ public String operation() { /** * Set the operation property: Operation name. - * + * * @param operation the operation value to set. * @return the OperationDisplay object itself. */ @@ -100,7 +104,7 @@ public OperationDisplay withOperation(String operation) { /** * Get the description property: Description of the operation. - * + * * @return the description value. */ public String description() { @@ -109,7 +113,7 @@ public String description() { /** * Set the description property: Description of the operation. - * + * * @param description the description value to set. * @return the OperationDisplay object itself. */ @@ -120,7 +124,7 @@ public OperationDisplay withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operations.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operations.java index 4fb4be8b5ec78..5996d174c8606 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operations.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Operations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Returns list of operations. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return lists the operations available as paginated response with {@link PagedIterable}. @@ -20,7 +22,7 @@ public interface Operations { /** * Returns list of operations. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationsList.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationsList.java index 3bdc44ca65e44..74b2427d2689e 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationsList.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/OperationsList.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Lists the operations available. */ +/** + * Lists the operations available. + */ @Fluent public final class OperationsList { /* @@ -25,13 +27,15 @@ public final class OperationsList { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of OperationsList class. */ + /** + * Creates an instance of OperationsList class. + */ public OperationsList() { } /** * Get the nextLink property: Url to follow for getting next page of operations. - * + * * @return the nextLink value. */ public String nextLink() { @@ -40,7 +44,7 @@ public String nextLink() { /** * Set the nextLink property: Url to follow for getting next page of operations. - * + * * @param nextLink the nextLink value to set. * @return the OperationsList object itself. */ @@ -51,7 +55,7 @@ public OperationsList withNextLink(String nextLink) { /** * Get the value property: Array of operations. - * + * * @return the value value. */ public List value() { @@ -60,7 +64,7 @@ public List value() { /** * Set the value property: Array of operations. - * + * * @param value the value value to set. * @return the OperationsList object itself. */ @@ -71,14 +75,13 @@ public OperationsList withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model OperationsList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model OperationsList")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDetail.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDetail.java index b2aa8e2ef67db..92437bc7cdca1 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDetail.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDetail.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Detailed pricing information for an sku. */ +/** + * Detailed pricing information for an sku. + */ @Fluent public final class PricingDetail { /* @@ -22,13 +24,15 @@ public final class PricingDetail { @JsonProperty(value = "value") private String value; - /** Creates an instance of PricingDetail class. */ + /** + * Creates an instance of PricingDetail class. + */ public PricingDetail() { } /** * Get the id property: Unique id for this pricing information. - * + * * @return the id value. */ public String id() { @@ -37,7 +41,7 @@ public String id() { /** * Set the id property: Unique id for this pricing information. - * + * * @param id the id value to set. * @return the PricingDetail object itself. */ @@ -48,7 +52,7 @@ public PricingDetail withId(String id) { /** * Get the value property: The unit cost of this sku. - * + * * @return the value value. */ public String value() { @@ -57,7 +61,7 @@ public String value() { /** * Set the value property: The unit cost of this sku. - * + * * @param value the value value to set. * @return the PricingDetail object itself. */ @@ -68,7 +72,7 @@ public PricingDetail withValue(String value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDimension.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDimension.java index 360a17fe523b5..9229f49e9a081 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDimension.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/PricingDimension.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Information about pricing dimension. */ +/** + * Information about pricing dimension. + */ @Fluent public final class PricingDimension { /* @@ -22,13 +24,15 @@ public final class PricingDimension { @JsonProperty(value = "name") private String name; - /** Creates an instance of PricingDimension class. */ + /** + * Creates an instance of PricingDimension class. + */ public PricingDimension() { } /** * Get the id property: Unique id of this pricing dimension. - * + * * @return the id value. */ public String id() { @@ -37,7 +41,7 @@ public String id() { /** * Set the id property: Unique id of this pricing dimension. - * + * * @param id the id value to set. * @return the PricingDimension object itself. */ @@ -48,7 +52,7 @@ public PricingDimension withId(String id) { /** * Get the name property: The display name of this pricing dimension. - * + * * @return the name value. */ public String name() { @@ -57,7 +61,7 @@ public String name() { /** * Set the name property: The display name of this pricing dimension. - * + * * @param name the name value to set. * @return the PricingDimension object itself. */ @@ -68,7 +72,7 @@ public PricingDimension withName(String name) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Provider.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Provider.java index e411c7daf4012..f43e9f3717ba0 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Provider.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Provider.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Information about a Provider. A Provider is an entity that offers Targets to run Azure Quantum Jobs. */ +/** + * Information about a Provider. A Provider is an entity that offers Targets to run Azure Quantum Jobs. + */ @Fluent public final class Provider { /* @@ -46,13 +48,15 @@ public final class Provider { @JsonProperty(value = "resourceUsageId") private String resourceUsageId; - /** Creates an instance of Provider class. */ + /** + * Creates an instance of Provider class. + */ public Provider() { } /** * Get the providerId property: Unique id of this provider. - * + * * @return the providerId value. */ public String providerId() { @@ -61,7 +65,7 @@ public String providerId() { /** * Set the providerId property: Unique id of this provider. - * + * * @param providerId the providerId value to set. * @return the Provider object itself. */ @@ -72,7 +76,7 @@ public Provider withProviderId(String providerId) { /** * Get the providerSku property: The sku associated with pricing information for this provider. - * + * * @return the providerSku value. */ public String providerSku() { @@ -81,7 +85,7 @@ public String providerSku() { /** * Set the providerSku property: The sku associated with pricing information for this provider. - * + * * @param providerSku the providerSku value to set. * @return the Provider object itself. */ @@ -92,7 +96,7 @@ public Provider withProviderSku(String providerSku) { /** * Get the instanceUri property: A Uri identifying the specific instance of this provider. - * + * * @return the instanceUri value. */ public String instanceUri() { @@ -101,7 +105,7 @@ public String instanceUri() { /** * Set the instanceUri property: A Uri identifying the specific instance of this provider. - * + * * @param instanceUri the instanceUri value to set. * @return the Provider object itself. */ @@ -112,7 +116,7 @@ public Provider withInstanceUri(String instanceUri) { /** * Get the applicationName property: The provider's marketplace application display name. - * + * * @return the applicationName value. */ public String applicationName() { @@ -121,7 +125,7 @@ public String applicationName() { /** * Set the applicationName property: The provider's marketplace application display name. - * + * * @param applicationName the applicationName value to set. * @return the Provider object itself. */ @@ -132,7 +136,7 @@ public Provider withApplicationName(String applicationName) { /** * Get the provisioningState property: Provisioning status field. - * + * * @return the provisioningState value. */ public Status provisioningState() { @@ -141,7 +145,7 @@ public Status provisioningState() { /** * Set the provisioningState property: Provisioning status field. - * + * * @param provisioningState the provisioningState value to set. * @return the Provider object itself. */ @@ -152,7 +156,7 @@ public Provider withProvisioningState(Status provisioningState) { /** * Get the resourceUsageId property: Id to track resource usage for the provider. - * + * * @return the resourceUsageId value. */ public String resourceUsageId() { @@ -161,7 +165,7 @@ public String resourceUsageId() { /** * Set the resourceUsageId property: Id to track resource usage for the provider. - * + * * @param resourceUsageId the resourceUsageId value to set. * @return the Provider object itself. */ @@ -172,7 +176,7 @@ public Provider withResourceUsageId(String resourceUsageId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderDescription.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderDescription.java index c24c29097dcd0..a4e4415595b02 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderDescription.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderDescription.java @@ -6,32 +6,34 @@ import com.azure.resourcemanager.quantum.fluent.models.ProviderDescriptionInner; -/** An immutable client-side representation of ProviderDescription. */ +/** + * An immutable client-side representation of ProviderDescription. + */ public interface ProviderDescription { /** * Gets the id property: Unique provider's id. - * + * * @return the id value. */ String id(); /** * Gets the name property: Provider's display name. - * + * * @return the name value. */ String name(); /** - * Gets the properties property: A list of provider-specific properties. - * + * Gets the properties property: Provider properties. + * * @return the properties value. */ ProviderProperties properties(); /** * Gets the inner com.azure.resourcemanager.quantum.fluent.models.ProviderDescriptionInner object. - * + * * @return the inner object. */ ProviderDescriptionInner innerModel(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderProperties.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderProperties.java index f9bee19edc18d..da872729f7068 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderProperties.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Provider properties. */ +/** + * Provider properties. + */ @Fluent public final class ProviderProperties { /* @@ -71,13 +73,15 @@ public final class ProviderProperties { @JsonProperty(value = "pricingDimensions") private List pricingDimensions; - /** Creates an instance of ProviderProperties class. */ + /** + * Creates an instance of ProviderProperties class. + */ public ProviderProperties() { } /** * Get the description property: A description about this provider. - * + * * @return the description value. */ public String description() { @@ -86,7 +90,7 @@ public String description() { /** * Get the providerType property: Provider type. - * + * * @return the providerType value. */ public String providerType() { @@ -95,7 +99,7 @@ public String providerType() { /** * Get the company property: Company name. - * + * * @return the company value. */ public String company() { @@ -104,7 +108,7 @@ public String company() { /** * Get the defaultEndpoint property: Provider's default endpoint. - * + * * @return the defaultEndpoint value. */ public String defaultEndpoint() { @@ -113,7 +117,7 @@ public String defaultEndpoint() { /** * Get the aad property: Azure Active Directory info. - * + * * @return the aad value. */ public ProviderPropertiesAad aad() { @@ -122,7 +126,7 @@ public ProviderPropertiesAad aad() { /** * Set the aad property: Azure Active Directory info. - * + * * @param aad the aad value to set. * @return the ProviderProperties object itself. */ @@ -133,7 +137,7 @@ public ProviderProperties withAad(ProviderPropertiesAad aad) { /** * Get the managedApplication property: Provider's Managed-Application info. - * + * * @return the managedApplication value. */ public ProviderPropertiesManagedApplication managedApplication() { @@ -142,7 +146,7 @@ public ProviderPropertiesManagedApplication managedApplication() { /** * Set the managedApplication property: Provider's Managed-Application info. - * + * * @param managedApplication the managedApplication value to set. * @return the ProviderProperties object itself. */ @@ -153,7 +157,7 @@ public ProviderProperties withManagedApplication(ProviderPropertiesManagedApplic /** * Get the targets property: The list of targets available from this provider. - * + * * @return the targets value. */ public List targets() { @@ -162,7 +166,7 @@ public List targets() { /** * Set the targets property: The list of targets available from this provider. - * + * * @param targets the targets value to set. * @return the ProviderProperties object itself. */ @@ -173,7 +177,7 @@ public ProviderProperties withTargets(List targets) { /** * Get the skus property: The list of skus available from this provider. - * + * * @return the skus value. */ public List skus() { @@ -182,7 +186,7 @@ public List skus() { /** * Set the skus property: The list of skus available from this provider. - * + * * @param skus the skus value to set. * @return the ProviderProperties object itself. */ @@ -193,7 +197,7 @@ public ProviderProperties withSkus(List skus) { /** * Get the quotaDimensions property: The list of quota dimensions from the provider. - * + * * @return the quotaDimensions value. */ public List quotaDimensions() { @@ -202,7 +206,7 @@ public List quotaDimensions() { /** * Set the quotaDimensions property: The list of quota dimensions from the provider. - * + * * @param quotaDimensions the quotaDimensions value to set. * @return the ProviderProperties object itself. */ @@ -213,7 +217,7 @@ public ProviderProperties withQuotaDimensions(List quotaDimensio /** * Get the pricingDimensions property: The list of pricing dimensions from the provider. - * + * * @return the pricingDimensions value. */ public List pricingDimensions() { @@ -222,7 +226,7 @@ public List pricingDimensions() { /** * Set the pricingDimensions property: The list of pricing dimensions from the provider. - * + * * @param pricingDimensions the pricingDimensions value to set. * @return the ProviderProperties object itself. */ @@ -233,7 +237,7 @@ public ProviderProperties withPricingDimensions(List pricingDi /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesAad.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesAad.java index dd8984c72283a..63555eee9bdfc 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesAad.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesAad.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Azure Active Directory info. */ +/** + * Azure Active Directory info. + */ @Immutable public final class ProviderPropertiesAad { /* @@ -22,13 +24,15 @@ public final class ProviderPropertiesAad { @JsonProperty(value = "tenantId", access = JsonProperty.Access.WRITE_ONLY) private String tenantId; - /** Creates an instance of ProviderPropertiesAad class. */ + /** + * Creates an instance of ProviderPropertiesAad class. + */ public ProviderPropertiesAad() { } /** * Get the applicationId property: Provider's application id. - * + * * @return the applicationId value. */ public String applicationId() { @@ -37,7 +41,7 @@ public String applicationId() { /** * Get the tenantId property: Provider's tenant id. - * + * * @return the tenantId value. */ public String tenantId() { @@ -46,7 +50,7 @@ public String tenantId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesManagedApplication.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesManagedApplication.java index 7173344b70594..4da9b4fec7c47 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesManagedApplication.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProviderPropertiesManagedApplication.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Provider's Managed-Application info. */ +/** + * Provider's Managed-Application info. + */ @Immutable public final class ProviderPropertiesManagedApplication { /* @@ -22,13 +24,15 @@ public final class ProviderPropertiesManagedApplication { @JsonProperty(value = "offerId", access = JsonProperty.Access.WRITE_ONLY) private String offerId; - /** Creates an instance of ProviderPropertiesManagedApplication class. */ + /** + * Creates an instance of ProviderPropertiesManagedApplication class. + */ public ProviderPropertiesManagedApplication() { } /** * Get the publisherId property: Provider's publisher id. - * + * * @return the publisherId value. */ public String publisherId() { @@ -37,7 +41,7 @@ public String publisherId() { /** * Get the offerId property: Provider's offer id. - * + * * @return the offerId value. */ public String offerId() { @@ -46,7 +50,7 @@ public String offerId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProvisioningStatus.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProvisioningStatus.java index 3dbe6454ac3f9..2292d9e6703ea 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProvisioningStatus.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ProvisioningStatus.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning status field. */ +/** + * Provisioning status field. + */ public final class ProvisioningStatus extends ExpandableStringEnum { - /** Static value Succeeded for ProvisioningStatus. */ + /** + * Static value Succeeded for ProvisioningStatus. + */ public static final ProvisioningStatus SUCCEEDED = fromString("Succeeded"); - /** Static value ProviderLaunching for ProvisioningStatus. */ + /** + * Static value ProviderLaunching for ProvisioningStatus. + */ public static final ProvisioningStatus PROVIDER_LAUNCHING = fromString("ProviderLaunching"); - /** Static value ProviderUpdating for ProvisioningStatus. */ + /** + * Static value ProviderUpdating for ProvisioningStatus. + */ public static final ProvisioningStatus PROVIDER_UPDATING = fromString("ProviderUpdating"); - /** Static value ProviderDeleting for ProvisioningStatus. */ + /** + * Static value ProviderDeleting for ProvisioningStatus. + */ public static final ProvisioningStatus PROVIDER_DELETING = fromString("ProviderDeleting"); - /** Static value ProviderProvisioning for ProvisioningStatus. */ + /** + * Static value ProviderProvisioning for ProvisioningStatus. + */ public static final ProvisioningStatus PROVIDER_PROVISIONING = fromString("ProviderProvisioning"); - /** Static value Failed for ProvisioningStatus. */ + /** + * Static value Failed for ProvisioningStatus. + */ public static final ProvisioningStatus FAILED = fromString("Failed"); /** * Creates a new instance of ProvisioningStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public ProvisioningStatus() { /** * Creates or finds a ProvisioningStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningStatus. */ @@ -50,7 +64,7 @@ public static ProvisioningStatus fromString(String name) { /** * Gets known ProvisioningStatus values. - * + * * @return known ProvisioningStatus values. */ public static Collection values() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspace.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspace.java index fe0be1d172523..c89ca27e3e86b 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspace.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspace.java @@ -11,139 +11,153 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of QuantumWorkspace. */ +/** + * An immutable client-side representation of QuantumWorkspace. + */ public interface QuantumWorkspace { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the identity property: Managed Identity information. - * + * * @return the identity value. */ QuantumWorkspaceIdentity identity(); /** - * Gets the systemData property: System metadata. - * + * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. + * * @return the systemData value. */ SystemData systemData(); /** * Gets the providers property: List of Providers selected for this Workspace. - * + * * @return the providers value. */ List providers(); /** * Gets the usable property: Whether the current workspace is ready to accept Jobs. - * + * * @return the usable value. */ UsableStatus usable(); /** * Gets the provisioningState property: Provisioning status field. - * + * * @return the provisioningState value. */ ProvisioningStatus provisioningState(); /** * Gets the storageAccount property: ARM Resource Id of the storage account associated with this workspace. - * + * * @return the storageAccount value. */ String storageAccount(); /** * Gets the endpointUri property: The URI of the workspace endpoint. - * + * * @return the endpointUri value. */ String endpointUri(); + /** + * Gets the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys. + * + * @return the apiKeyEnabled value. + */ + Boolean apiKeyEnabled(); + /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.quantum.fluent.models.QuantumWorkspaceInner object. - * + * * @return the inner object. */ QuantumWorkspaceInner innerModel(); - /** The entirety of the QuantumWorkspace definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the QuantumWorkspace definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The QuantumWorkspace definition stages. */ + /** + * The QuantumWorkspace definition stages. + */ interface DefinitionStages { - /** The first stage of the QuantumWorkspace definition. */ + /** + * The first stage of the QuantumWorkspace definition. + */ interface Blank extends WithLocation { } - /** The stage of the QuantumWorkspace definition allowing to specify location. */ + /** + * The stage of the QuantumWorkspace definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -151,19 +165,21 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the QuantumWorkspace definition allowing to specify parent resource. */ + /** + * The stage of the QuantumWorkspace definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ WithCreate withExistingResourceGroup(String resourceGroupName); @@ -173,105 +189,129 @@ interface WithResourceGroup { * The stage of the QuantumWorkspace definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithIdentity, - DefinitionStages.WithProviders, - DefinitionStages.WithStorageAccount { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithIdentity, + DefinitionStages.WithProviders, DefinitionStages.WithStorageAccount, DefinitionStages.WithApiKeyEnabled { /** * Executes the create request. - * + * * @return the created resource. */ QuantumWorkspace create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ QuantumWorkspace create(Context context); } - /** The stage of the QuantumWorkspace definition allowing to specify tags. */ + /** + * The stage of the QuantumWorkspace definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the QuantumWorkspace definition allowing to specify identity. */ + /** + * The stage of the QuantumWorkspace definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: Managed Identity information.. - * + * * @param identity Managed Identity information. * @return the next definition stage. */ WithCreate withIdentity(QuantumWorkspaceIdentity identity); } - /** The stage of the QuantumWorkspace definition allowing to specify providers. */ + /** + * The stage of the QuantumWorkspace definition allowing to specify providers. + */ interface WithProviders { /** * Specifies the providers property: List of Providers selected for this Workspace. - * + * * @param providers List of Providers selected for this Workspace. * @return the next definition stage. */ WithCreate withProviders(List providers); } - /** The stage of the QuantumWorkspace definition allowing to specify storageAccount. */ + /** + * The stage of the QuantumWorkspace definition allowing to specify storageAccount. + */ interface WithStorageAccount { /** * Specifies the storageAccount property: ARM Resource Id of the storage account associated with this * workspace.. - * + * * @param storageAccount ARM Resource Id of the storage account associated with this workspace. * @return the next definition stage. */ WithCreate withStorageAccount(String storageAccount); } + + /** + * The stage of the QuantumWorkspace definition allowing to specify apiKeyEnabled. + */ + interface WithApiKeyEnabled { + /** + * Specifies the apiKeyEnabled property: Indicator of enablement of the Quantum workspace Api keys.. + * + * @param apiKeyEnabled Indicator of enablement of the Quantum workspace Api keys. + * @return the next definition stage. + */ + WithCreate withApiKeyEnabled(Boolean apiKeyEnabled); + } } /** * Begins update for the QuantumWorkspace resource. - * + * * @return the stage of resource update. */ QuantumWorkspace.Update update(); - /** The template for QuantumWorkspace update. */ + /** + * The template for QuantumWorkspace update. + */ interface Update extends UpdateStages.WithTags { /** * Executes the update request. - * + * * @return the updated resource. */ QuantumWorkspace apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ QuantumWorkspace apply(Context context); } - /** The QuantumWorkspace update stages. */ + /** + * The QuantumWorkspace update stages. + */ interface UpdateStages { - /** The stage of the QuantumWorkspace update allowing to specify tags. */ + /** + * The stage of the QuantumWorkspace update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ @@ -281,14 +321,14 @@ interface WithTags { /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ QuantumWorkspace refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspaceIdentity.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspaceIdentity.java index f78a76a60ecd0..a3eb31bdbd0dc 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspaceIdentity.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuantumWorkspaceIdentity.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Managed Identity information. */ +/** + * Managed Identity information. + */ @Fluent public final class QuantumWorkspaceIdentity { /* @@ -28,13 +30,15 @@ public final class QuantumWorkspaceIdentity { @JsonProperty(value = "type") private ResourceIdentityType type; - /** Creates an instance of QuantumWorkspaceIdentity class. */ + /** + * Creates an instance of QuantumWorkspaceIdentity class. + */ public QuantumWorkspaceIdentity() { } /** * Get the principalId property: The principal ID of resource identity. - * + * * @return the principalId value. */ public String principalId() { @@ -43,7 +47,7 @@ public String principalId() { /** * Get the tenantId property: The tenant ID of resource. - * + * * @return the tenantId value. */ public String tenantId() { @@ -52,7 +56,7 @@ public String tenantId() { /** * Get the type property: The identity type. - * + * * @return the type value. */ public ResourceIdentityType type() { @@ -61,7 +65,7 @@ public ResourceIdentityType type() { /** * Set the type property: The identity type. - * + * * @param type the type value to set. * @return the QuantumWorkspaceIdentity object itself. */ @@ -72,7 +76,7 @@ public QuantumWorkspaceIdentity withType(ResourceIdentityType type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuotaDimension.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuotaDimension.java index 7caeee2c8a9e4..e33541f7e7b51 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuotaDimension.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/QuotaDimension.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Information about a specific quota dimension. */ +/** + * Information about a specific quota dimension. + */ @Fluent public final class QuotaDimension { /* @@ -58,13 +60,15 @@ public final class QuotaDimension { @JsonProperty(value = "unitPlural") private String unitPlural; - /** Creates an instance of QuotaDimension class. */ + /** + * Creates an instance of QuotaDimension class. + */ public QuotaDimension() { } /** * Get the id property: Unique id of this dimension. - * + * * @return the id value. */ public String id() { @@ -73,7 +77,7 @@ public String id() { /** * Set the id property: Unique id of this dimension. - * + * * @param id the id value to set. * @return the QuotaDimension object itself. */ @@ -84,7 +88,7 @@ public QuotaDimension withId(String id) { /** * Get the scope property: The scope of this quota dimension. - * + * * @return the scope value. */ public String scope() { @@ -93,7 +97,7 @@ public String scope() { /** * Set the scope property: The scope of this quota dimension. - * + * * @param scope the scope value to set. * @return the QuotaDimension object itself. */ @@ -104,7 +108,7 @@ public QuotaDimension withScope(String scope) { /** * Get the period property: The reset period of this quota dimension. - * + * * @return the period value. */ public String period() { @@ -113,7 +117,7 @@ public String period() { /** * Set the period property: The reset period of this quota dimension. - * + * * @param period the period value to set. * @return the QuotaDimension object itself. */ @@ -124,7 +128,7 @@ public QuotaDimension withPeriod(String period) { /** * Get the quota property: The max limit of this dimension. - * + * * @return the quota value. */ public Float quota() { @@ -133,7 +137,7 @@ public Float quota() { /** * Set the quota property: The max limit of this dimension. - * + * * @param quota the quota value to set. * @return the QuotaDimension object itself. */ @@ -144,7 +148,7 @@ public QuotaDimension withQuota(Float quota) { /** * Get the name property: The display name of this quota dimension. - * + * * @return the name value. */ public String name() { @@ -153,7 +157,7 @@ public String name() { /** * Set the name property: The display name of this quota dimension. - * + * * @param name the name value to set. * @return the QuotaDimension object itself. */ @@ -164,7 +168,7 @@ public QuotaDimension withName(String name) { /** * Get the description property: A description about this quota dimension. - * + * * @return the description value. */ public String description() { @@ -173,7 +177,7 @@ public String description() { /** * Set the description property: A description about this quota dimension. - * + * * @param description the description value to set. * @return the QuotaDimension object itself. */ @@ -184,7 +188,7 @@ public QuotaDimension withDescription(String description) { /** * Get the unit property: The standard unit of measurement used for this quota dimension. - * + * * @return the unit value. */ public String unit() { @@ -193,7 +197,7 @@ public String unit() { /** * Set the unit property: The standard unit of measurement used for this quota dimension. - * + * * @param unit the unit value to set. * @return the QuotaDimension object itself. */ @@ -204,7 +208,7 @@ public QuotaDimension withUnit(String unit) { /** * Get the unitPlural property: The standard unit of measurement used for this quota dimension in plural form. - * + * * @return the unitPlural value. */ public String unitPlural() { @@ -213,7 +217,7 @@ public String unitPlural() { /** * Set the unitPlural property: The standard unit of measurement used for this quota dimension in plural form. - * + * * @param unitPlural the unitPlural value to set. * @return the QuotaDimension object itself. */ @@ -224,7 +228,7 @@ public QuotaDimension withUnitPlural(String unitPlural) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ResourceIdentityType.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ResourceIdentityType.java index 3c3e1302bf5a9..2a37ec8cc9f11 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ResourceIdentityType.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/ResourceIdentityType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The identity type. */ +/** + * The identity type. + */ public final class ResourceIdentityType extends ExpandableStringEnum { - /** Static value SystemAssigned for ResourceIdentityType. */ + /** + * Static value SystemAssigned for ResourceIdentityType. + */ public static final ResourceIdentityType SYSTEM_ASSIGNED = fromString("SystemAssigned"); - /** Static value None for ResourceIdentityType. */ + /** + * Static value None for ResourceIdentityType. + */ public static final ResourceIdentityType NONE = fromString("None"); /** * Creates a new instance of ResourceIdentityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ResourceIdentityType() { /** * Creates or finds a ResourceIdentityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ResourceIdentityType. */ @@ -38,7 +44,7 @@ public static ResourceIdentityType fromString(String name) { /** * Gets known ResourceIdentityType values. - * + * * @return known ResourceIdentityType values. */ public static Collection values() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/SkuDescription.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/SkuDescription.java index b6df9e4a859b6..bd2c996874a96 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/SkuDescription.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/SkuDescription.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Information about a specific sku. */ +/** + * Information about a specific sku. + */ @Fluent public final class SkuDescription { /* @@ -65,13 +67,15 @@ public final class SkuDescription { @JsonProperty(value = "pricingDetails") private List pricingDetails; - /** Creates an instance of SkuDescription class. */ + /** + * Creates an instance of SkuDescription class. + */ public SkuDescription() { } /** * Get the id property: Unique sku id. - * + * * @return the id value. */ public String id() { @@ -80,7 +84,7 @@ public String id() { /** * Set the id property: Unique sku id. - * + * * @param id the id value to set. * @return the SkuDescription object itself. */ @@ -91,7 +95,7 @@ public SkuDescription withId(String id) { /** * Get the name property: Display name of this sku. - * + * * @return the name value. */ public String name() { @@ -100,7 +104,7 @@ public String name() { /** * Set the name property: Display name of this sku. - * + * * @param name the name value to set. * @return the SkuDescription object itself. */ @@ -111,7 +115,7 @@ public SkuDescription withName(String name) { /** * Get the version property: Display name of this sku. - * + * * @return the version value. */ public String version() { @@ -120,7 +124,7 @@ public String version() { /** * Set the version property: Display name of this sku. - * + * * @param version the version value to set. * @return the SkuDescription object itself. */ @@ -131,7 +135,7 @@ public SkuDescription withVersion(String version) { /** * Get the description property: Description about this sku. - * + * * @return the description value. */ public String description() { @@ -140,7 +144,7 @@ public String description() { /** * Set the description property: Description about this sku. - * + * * @param description the description value to set. * @return the SkuDescription object itself. */ @@ -151,7 +155,7 @@ public SkuDescription withDescription(String description) { /** * Get the restrictedAccessUri property: Uri to subscribe to the restricted access sku. - * + * * @return the restrictedAccessUri value. */ public String restrictedAccessUri() { @@ -160,7 +164,7 @@ public String restrictedAccessUri() { /** * Set the restrictedAccessUri property: Uri to subscribe to the restricted access sku. - * + * * @param restrictedAccessUri the restrictedAccessUri value to set. * @return the SkuDescription object itself. */ @@ -172,7 +176,7 @@ public SkuDescription withRestrictedAccessUri(String restrictedAccessUri) { /** * Get the autoAdd property: Flag to indicate whether the sku should be automatically added during workspace * creation. - * + * * @return the autoAdd value. */ public Boolean autoAdd() { @@ -182,7 +186,7 @@ public Boolean autoAdd() { /** * Set the autoAdd property: Flag to indicate whether the sku should be automatically added during workspace * creation. - * + * * @param autoAdd the autoAdd value to set. * @return the SkuDescription object itself. */ @@ -193,7 +197,7 @@ public SkuDescription withAutoAdd(Boolean autoAdd) { /** * Get the targets property: The list of targets available for this sku. - * + * * @return the targets value. */ public List targets() { @@ -202,7 +206,7 @@ public List targets() { /** * Set the targets property: The list of targets available for this sku. - * + * * @param targets the targets value to set. * @return the SkuDescription object itself. */ @@ -213,7 +217,7 @@ public SkuDescription withTargets(List targets) { /** * Get the quotaDimensions property: The list of quota dimensions for this sku. - * + * * @return the quotaDimensions value. */ public List quotaDimensions() { @@ -222,7 +226,7 @@ public List quotaDimensions() { /** * Set the quotaDimensions property: The list of quota dimensions for this sku. - * + * * @param quotaDimensions the quotaDimensions value to set. * @return the SkuDescription object itself. */ @@ -233,7 +237,7 @@ public SkuDescription withQuotaDimensions(List quotaDimensions) /** * Get the pricingDetails property: The list of pricing details for the sku. - * + * * @return the pricingDetails value. */ public List pricingDetails() { @@ -242,7 +246,7 @@ public List pricingDetails() { /** * Set the pricingDetails property: The list of pricing details for the sku. - * + * * @param pricingDetails the pricingDetails value to set. * @return the SkuDescription object itself. */ @@ -253,7 +257,7 @@ public SkuDescription withPricingDetails(List pricingDetails) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Status.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Status.java index 11a266cbc1897..8aa0b62039541 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Status.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Status.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Provisioning status field. */ +/** + * Provisioning status field. + */ public final class Status extends ExpandableStringEnum { - /** Static value Succeeded for Status. */ + /** + * Static value Succeeded for Status. + */ public static final Status SUCCEEDED = fromString("Succeeded"); - /** Static value Launching for Status. */ + /** + * Static value Launching for Status. + */ public static final Status LAUNCHING = fromString("Launching"); - /** Static value Updating for Status. */ + /** + * Static value Updating for Status. + */ public static final Status UPDATING = fromString("Updating"); - /** Static value Deleting for Status. */ + /** + * Static value Deleting for Status. + */ public static final Status DELETING = fromString("Deleting"); - /** Static value Deleted for Status. */ + /** + * Static value Deleted for Status. + */ public static final Status DELETED = fromString("Deleted"); - /** Static value Failed for Status. */ + /** + * Static value Failed for Status. + */ public static final Status FAILED = fromString("Failed"); /** * Creates a new instance of Status value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public Status() { /** * Creates or finds a Status from its string representation. - * + * * @param name a name to look for. * @return the corresponding Status. */ @@ -50,7 +64,7 @@ public static Status fromString(String name) { /** * Gets known Status values. - * + * * @return known Status values. */ public static Collection values() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TagsObject.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TagsObject.java index 0783cd7ec7fa6..5f801c75d9a11 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TagsObject.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TagsObject.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Tags object for patch operations. */ +/** + * Tags object for patch operations. + */ @Fluent public final class TagsObject { /* @@ -19,13 +21,15 @@ public final class TagsObject { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of TagsObject class. */ + /** + * Creates an instance of TagsObject class. + */ public TagsObject() { } /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -34,7 +38,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the TagsObject object itself. */ @@ -45,7 +49,7 @@ public TagsObject withTags(Map tags) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TargetDescription.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TargetDescription.java index c8b4526f54d65..e27fa8acc7008 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TargetDescription.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/TargetDescription.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Information about a Target. A target is the component that can process a specific type of Job. */ +/** + * Information about a Target. A target is the component that can process a specific type of Job. + */ @Fluent public final class TargetDescription { /* @@ -41,13 +43,15 @@ public final class TargetDescription { @JsonProperty(value = "acceptedContentEncodings") private List acceptedContentEncodings; - /** Creates an instance of TargetDescription class. */ + /** + * Creates an instance of TargetDescription class. + */ public TargetDescription() { } /** * Get the id property: Unique target id. - * + * * @return the id value. */ public String id() { @@ -56,7 +60,7 @@ public String id() { /** * Set the id property: Unique target id. - * + * * @param id the id value to set. * @return the TargetDescription object itself. */ @@ -67,7 +71,7 @@ public TargetDescription withId(String id) { /** * Get the name property: Display name of this target. - * + * * @return the name value. */ public String name() { @@ -76,7 +80,7 @@ public String name() { /** * Set the name property: Display name of this target. - * + * * @param name the name value to set. * @return the TargetDescription object itself. */ @@ -87,7 +91,7 @@ public TargetDescription withName(String name) { /** * Get the description property: A description about this target. - * + * * @return the description value. */ public String description() { @@ -96,7 +100,7 @@ public String description() { /** * Set the description property: A description about this target. - * + * * @param description the description value to set. * @return the TargetDescription object itself. */ @@ -107,7 +111,7 @@ public TargetDescription withDescription(String description) { /** * Get the acceptedDataFormats property: List of data formats accepted by this target. - * + * * @return the acceptedDataFormats value. */ public List acceptedDataFormats() { @@ -116,7 +120,7 @@ public List acceptedDataFormats() { /** * Set the acceptedDataFormats property: List of data formats accepted by this target. - * + * * @param acceptedDataFormats the acceptedDataFormats value to set. * @return the TargetDescription object itself. */ @@ -127,7 +131,7 @@ public TargetDescription withAcceptedDataFormats(List acceptedDataFormat /** * Get the acceptedContentEncodings property: List of content encodings accepted by this target. - * + * * @return the acceptedContentEncodings value. */ public List acceptedContentEncodings() { @@ -136,7 +140,7 @@ public List acceptedContentEncodings() { /** * Set the acceptedContentEncodings property: List of content encodings accepted by this target. - * + * * @param acceptedContentEncodings the acceptedContentEncodings value to set. * @return the TargetDescription object itself. */ @@ -147,7 +151,7 @@ public TargetDescription withAcceptedContentEncodings(List acceptedConte /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/UsableStatus.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/UsableStatus.java index 9504e82bdcd4e..84b24afc02909 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/UsableStatus.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/UsableStatus.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Whether the current workspace is ready to accept Jobs. */ +/** + * Whether the current workspace is ready to accept Jobs. + */ public final class UsableStatus extends ExpandableStringEnum { - /** Static value Yes for UsableStatus. */ + /** + * Static value Yes for UsableStatus. + */ public static final UsableStatus YES = fromString("Yes"); - /** Static value No for UsableStatus. */ + /** + * Static value No for UsableStatus. + */ public static final UsableStatus NO = fromString("No"); - /** Static value Partial for UsableStatus. */ + /** + * Static value Partial for UsableStatus. + */ public static final UsableStatus PARTIAL = fromString("Partial"); /** * Creates a new instance of UsableStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public UsableStatus() { /** * Creates or finds a UsableStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding UsableStatus. */ @@ -41,7 +49,7 @@ public static UsableStatus fromString(String name) { /** * Gets known UsableStatus values. - * + * * @return known UsableStatus values. */ public static Collection values() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceListResult.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceListResult.java index 2a28c6caff5b7..334d372f217c3 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceListResult.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a list Workspaces operation. */ +/** + * The response of a list Workspaces operation. + */ @Fluent public final class WorkspaceListResult { /* @@ -24,13 +26,15 @@ public final class WorkspaceListResult { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of WorkspaceListResult class. */ + /** + * Creates an instance of WorkspaceListResult class. + */ public WorkspaceListResult() { } /** * Get the value property: Result of a list Workspaces operation. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: Result of a list Workspaces operation. - * + * * @param value the value value to set. * @return the WorkspaceListResult object itself. */ @@ -51,7 +55,7 @@ public WorkspaceListResult withValue(List value) { /** * Get the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of * Workspaces. - * + * * @return the nextLink value. */ public String nextLink() { @@ -61,7 +65,7 @@ public String nextLink() { /** * Set the nextLink property: Link to the next set of results. Not empty if Value contains incomplete list of * Workspaces. - * + * * @param nextLink the nextLink value to set. * @return the WorkspaceListResult object itself. */ @@ -72,7 +76,7 @@ public WorkspaceListResult withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceOperations.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceOperations.java index b212e665ec3f0..3acd9829b53cb 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceOperations.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/WorkspaceOperations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of WorkspaceOperations. */ +/** + * Resource collection API of WorkspaceOperations. + */ public interface WorkspaceOperations { /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @param context The context to associate with this operation. @@ -20,12 +22,12 @@ public interface WorkspaceOperations { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return result of check name availability along with {@link Response}. */ - Response checkNameAvailabilityWithResponse( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context); + Response checkNameAvailabilityWithResponse(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters, Context context); /** * Check the availability of the resource name. - * + * * @param locationName Location. * @param checkNameAvailabilityParameters The name and type of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -33,6 +35,62 @@ Response checkNameAvailabilityWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return result of check name availability. */ - CheckNameAvailabilityResult checkNameAvailability( - String locationName, CheckNameAvailabilityParameters checkNameAvailabilityParameters); + CheckNameAvailabilityResult checkNameAvailability(String locationName, + CheckNameAvailabilityParameters checkNameAvailabilityParameters); + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs along with {@link Response}. + */ + Response listKeysWithResponse(String resourceGroupName, String workspaceName, Context context); + + /** + * Get the keys to use with the Quantum APIs. A key is used to authenticate and authorize access to the Quantum REST + * APIs. Only one key is needed at a time; two are given to provide seamless key regeneration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the keys to use with the Quantum APIs. + */ + ListKeysResult listKeys(String resourceGroupName, String workspaceName); + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + Response regenerateKeysWithResponse(String resourceGroupName, String workspaceName, ApiKeys keySpecification, + Context context); + + /** + * Regenerate either the primary or secondary key for use with the Quantum APIs. The old key will stop working + * immediately. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param workspaceName The name of the quantum workspace resource. + * @param keySpecification Which key to regenerate: primary or secondary. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void regenerateKeys(String resourceGroupName, String workspaceName, ApiKeys keySpecification); } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Workspaces.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Workspaces.java index 9a3ce0baffbad..d36eded906481 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Workspaces.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/Workspaces.java @@ -8,12 +8,14 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Workspaces. */ +/** + * Resource collection API of Workspaces. + */ public interface Workspaces { /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -21,13 +23,13 @@ public interface Workspaces { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource proxy definition object for quantum workspace along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String workspaceName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String workspaceName, + Context context); /** * Returns the Workspace resource associated with the given name. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -38,8 +40,8 @@ Response getByResourceGroupWithResponse( /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -49,8 +51,8 @@ Response getByResourceGroupWithResponse( /** * Deletes a Workspace resource. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param workspaceName The name of the quantum workspace resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -61,7 +63,7 @@ Response getByResourceGroupWithResponse( /** * Gets the list of Workspaces within a Subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of Workspaces within a Subscription as paginated response with {@link PagedIterable}. @@ -70,7 +72,7 @@ Response getByResourceGroupWithResponse( /** * Gets the list of Workspaces within a Subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -81,8 +83,8 @@ Response getByResourceGroupWithResponse( /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -92,8 +94,8 @@ Response getByResourceGroupWithResponse( /** * Gets the list of Workspaces within a resource group. - * - * @param resourceGroupName The name of the resource group. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -104,7 +106,7 @@ Response getByResourceGroupWithResponse( /** * Returns the Workspace resource associated with the given name. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -115,7 +117,7 @@ Response getByResourceGroupWithResponse( /** * Returns the Workspace resource associated with the given name. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -127,7 +129,7 @@ Response getByResourceGroupWithResponse( /** * Deletes a Workspace resource. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -137,7 +139,7 @@ Response getByResourceGroupWithResponse( /** * Deletes a Workspace resource. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -148,7 +150,7 @@ Response getByResourceGroupWithResponse( /** * Begins definition for a new QuantumWorkspace resource. - * + * * @param name resource name. * @return the first stage of the new QuantumWorkspace definition. */ diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/package-info.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/package-info.java index 186530e5e0eb3..bb29f3eb81af8 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/package-info.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/models/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the data models for AzureQuantumManagementClient. null. */ +/** + * Package containing the data models for AzureQuantumManagementClient. + * null. + */ package com.azure.resourcemanager.quantum.models; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/package-info.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/package-info.java index 71e8e34a4378c..55c6aa3a1ae82 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/package-info.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/com/azure/resourcemanager/quantum/package-info.java @@ -2,5 +2,8 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -/** Package containing the classes for AzureQuantumManagementClient. null. */ +/** + * Package containing the classes for AzureQuantumManagementClient. + * null. + */ package com.azure.resourcemanager.quantum; diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/module-info.java b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/module-info.java index 7b0c53d5d30b1..d1b747a338013 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/main/java/module-info.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.quantum { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.quantum; exports com.azure.resourcemanager.quantum.fluent; exports com.azure.resourcemanager.quantum.fluent.models; exports com.azure.resourcemanager.quantum.models; - - opens com.azure.resourcemanager.quantum.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.quantum.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.quantum.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.quantum.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/proxy-config.json b/sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/proxy-config.json new file mode 100644 index 0000000000000..176ab623574db --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.quantum.implementation.WorkspacesClientImpl$WorkspacesService" ], [ "com.azure.resourcemanager.quantum.implementation.OfferingsClientImpl$OfferingsService" ], [ "com.azure.resourcemanager.quantum.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.quantum.implementation.WorkspaceOperationsClientImpl$WorkspaceOperationsService" ] ] \ No newline at end of file diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/reflect-config.json b/sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/reflect-config.json new file mode 100644 index 0000000000000..e36c12ea51714 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-quantum/reflect-config.json @@ -0,0 +1,146 @@ +[ { + "name" : "com.azure.resourcemanager.quantum.fluent.models.QuantumWorkspaceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.fluent.models.WorkspaceResourceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.Provider", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.QuantumWorkspaceIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.TagsObject", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.WorkspaceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.OfferingsListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.fluent.models.ProviderDescriptionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ProviderProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ProviderPropertiesAad", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ProviderPropertiesManagedApplication", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.TargetDescription", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.SkuDescription", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.QuotaDimension", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.PricingDetail", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.PricingDimension", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.OperationsList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.CheckNameAvailabilityParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.fluent.models.CheckNameAvailabilityResultInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.fluent.models.ListKeysResultInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ApiKey", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ApiKeys", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.Status", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.UsableStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ProvisioningStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.ResourceIdentityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.quantum.models.KeyType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OfferingsListSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OfferingsListSamples.java index 3c653dc70138a..6e8f743dfbda1 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OfferingsListSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OfferingsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.quantum.generated; -/** Samples for Offerings List. */ +/** + * Samples for Offerings List. + */ public final class OfferingsListSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/offeringsList.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/offeringsList.json */ /** * Sample code: OfferingsList. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void offeringsList(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OperationsListSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OperationsListSamples.java index b77c89f29d6e0..d25909608e0a6 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OperationsListSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/OperationsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.quantum.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/operations.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/operations.json */ /** * Sample code: Operations. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void operations(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationCheckNameAvailabilitySamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationCheckNameAvailabilitySamples.java index ab3867ba8c5d6..5c3a27642ef85 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationCheckNameAvailabilitySamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationCheckNameAvailabilitySamples.java @@ -6,25 +6,24 @@ import com.azure.resourcemanager.quantum.models.CheckNameAvailabilityParameters; -/** Samples for WorkspaceOperation CheckNameAvailability. */ +/** + * Samples for WorkspaceOperation CheckNameAvailability. + */ public final class WorkspaceOperationCheckNameAvailabilitySamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesCheckNameAvailability.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesCheckNameAvailability.json */ /** * Sample code: QuantumWorkspacesCheckNameAvailability. - * + * * @param manager Entry point to AzureQuantumManager. */ - public static void quantumWorkspacesCheckNameAvailability( - com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - manager - .workspaceOperations() - .checkNameAvailabilityWithResponse( - "westus2", - new CheckNameAvailabilityParameters() - .withName("sample-workspace-name") - .withType("Microsoft.Quantum/Workspaces"), + public static void + quantumWorkspacesCheckNameAvailability(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + manager.workspaceOperations() + .checkNameAvailabilityWithResponse("westus2", new CheckNameAvailabilityParameters() + .withName("sample-workspace-name").withType("Microsoft.Quantum/Workspaces"), com.azure.core.util.Context.NONE); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationListKeysSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationListKeysSamples.java new file mode 100644 index 0000000000000..95ce88726bf8c --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationListKeysSamples.java @@ -0,0 +1,24 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.generated; + +/** + * Samples for WorkspaceOperation ListKeys. + */ +public final class WorkspaceOperationListKeysSamples { + /* + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/listKeys.json + */ + /** + * Sample code: ListKeys. + * + * @param manager Entry point to AzureQuantumManager. + */ + public static void listKeys(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + manager.workspaceOperations().listKeysWithResponse("quantumResourcegroup", "quantumworkspace1", + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationRegenerateKeysSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationRegenerateKeysSamples.java new file mode 100644 index 0000000000000..9a876bceb1858 --- /dev/null +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationRegenerateKeysSamples.java @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.quantum.generated; + +import com.azure.resourcemanager.quantum.models.ApiKeys; +import com.azure.resourcemanager.quantum.models.KeyType; +import java.util.Arrays; + +/** + * Samples for WorkspaceOperation RegenerateKeys. + */ +public final class WorkspaceOperationRegenerateKeysSamples { + /* + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/regenerateKey.json + */ + /** + * Sample code: RegenerateKey. + * + * @param manager Entry point to AzureQuantumManager. + */ + public static void regenerateKey(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + manager.workspaceOperations().regenerateKeysWithResponse("quantumResourcegroup", "quantumworkspace1", + new ApiKeys().withKeys(Arrays.asList(KeyType.PRIMARY, KeyType.SECONDARY)), + com.azure.core.util.Context.NONE); + } +} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateSamples.java index 3960560260eb3..b4e61fee32c24 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateSamples.java @@ -7,28 +7,26 @@ import com.azure.resourcemanager.quantum.models.Provider; import java.util.Arrays; -/** Samples for Workspaces CreateOrUpdate. */ +/** + * Samples for Workspaces CreateOrUpdate. + */ public final class WorkspacesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesPut.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/quantumWorkspacesPut + * .json */ /** * Sample code: QuantumWorkspacesPut. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesPut(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - manager - .workspaces() - .define("quantumworkspace1") - .withRegion("West US") + manager.workspaces().define("quantumworkspace1").withRegion("West US") .withExistingResourceGroup("quantumResourcegroup") - .withProviders( - Arrays - .asList( - new Provider().withProviderId("Honeywell").withProviderSku("Basic"), - new Provider().withProviderId("IonQ").withProviderSku("Basic"), - new Provider().withProviderId("OneQBit").withProviderSku("Basic"))) + .withProviders(Arrays.asList(new Provider().withProviderId("Honeywell").withProviderSku("Basic"), + new Provider().withProviderId("IonQ").withProviderSku("Basic"), + new Provider().withProviderId("OneQBit").withProviderSku("Basic"))) .withStorageAccount( "/subscriptions/1C4B2828-7D49-494F-933D-061373BE28C2/resourceGroups/quantumResourcegroup/providers/Microsoft.Storage/storageAccounts/testStorageAccount") .create(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteSamples.java index 5c72896d3b1dc..269930594804e 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.quantum.generated; -/** Samples for Workspaces Delete. */ +/** + * Samples for Workspaces Delete. + */ public final class WorkspacesDeleteSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesDelete.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesDelete.json */ /** * Sample code: QuantumWorkspacesDelete. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesDelete(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupSamples.java index 083173be395f6..80d38be113167 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.quantum.generated; -/** Samples for Workspaces GetByResourceGroup. */ +/** + * Samples for Workspaces GetByResourceGroup. + */ public final class WorkspacesGetByResourceGroupSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesGet.json + * x-ms-original-file: + * specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/quantumWorkspacesGet + * .json */ /** * Sample code: QuantumWorkspacesGet. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesGet(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - manager - .workspaces() - .getByResourceGroupWithResponse( - "quantumResourcegroup", "quantumworkspace1", com.azure.core.util.Context.NONE); + manager.workspaces().getByResourceGroupWithResponse("quantumResourcegroup", "quantumworkspace1", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupSamples.java index a67c363c849c1..dac33d4a44fe4 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.quantum.generated; -/** Samples for Workspaces ListByResourceGroup. */ +/** + * Samples for Workspaces ListByResourceGroup. + */ public final class WorkspacesListByResourceGroupSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesListResourceGroup.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesListResourceGroup.json */ /** * Sample code: QuantumWorkspacesListByResourceGroup. - * + * * @param manager Entry point to AzureQuantumManager. */ - public static void quantumWorkspacesListByResourceGroup( - com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + public static void + quantumWorkspacesListByResourceGroup(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { manager.workspaces().listByResourceGroup("quantumResourcegroup", com.azure.core.util.Context.NONE); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListSamples.java index 736b4f0cfa25b..7edde7c5bd46f 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesListSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.quantum.generated; -/** Samples for Workspaces List. */ +/** + * Samples for Workspaces List. + */ public final class WorkspacesListSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesListSubscription.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesListSubscription.json */ /** * Sample code: QuantumWorkspacesListBySubscription. - * + * * @param manager Entry point to AzureQuantumManager. */ - public static void quantumWorkspacesListBySubscription( - com.azure.resourcemanager.quantum.AzureQuantumManager manager) { + public static void + quantumWorkspacesListBySubscription(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { manager.workspaces().list(com.azure.core.util.Context.NONE); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesUpdateTagsSamples.java b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesUpdateTagsSamples.java index d64a307ef3122..950bc0ffd840b 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesUpdateTagsSamples.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/samples/java/com/azure/resourcemanager/quantum/generated/WorkspacesUpdateTagsSamples.java @@ -8,26 +8,26 @@ import java.util.HashMap; import java.util.Map; -/** Samples for Workspaces UpdateTags. */ +/** + * Samples for Workspaces UpdateTags. + */ public final class WorkspacesUpdateTagsSamples { /* - * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2022-01-10-preview/examples/quantumWorkspacesPatch.json + * x-ms-original-file: specification/quantum/resource-manager/Microsoft.Quantum/preview/2023-11-13-preview/examples/ + * quantumWorkspacesPatch.json */ /** * Sample code: QuantumWorkspacesPatchTags. - * + * * @param manager Entry point to AzureQuantumManager. */ public static void quantumWorkspacesPatchTags(com.azure.resourcemanager.quantum.AzureQuantumManager manager) { - QuantumWorkspace resource = - manager - .workspaces() - .getByResourceGroupWithResponse( - "quantumResourcegroup", "quantumworkspace1", com.azure.core.util.Context.NONE) - .getValue(); + QuantumWorkspace resource = manager.workspaces().getByResourceGroupWithResponse("quantumResourcegroup", + "quantumworkspace1", com.azure.core.util.Context.NONE).getValue(); resource.update().withTags(mapOf("tag1", "value1", "tag2", "value2")).apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityParametersTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityParametersTests.java index eb7de5dbaa867..bc892bb03b5ab 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityParametersTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityParametersTests.java @@ -11,20 +11,19 @@ public final class CheckNameAvailabilityParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CheckNameAvailabilityParameters model = - BinaryData - .fromString("{\"name\":\"vhelxprglyatdd\",\"type\":\"cbcuejrjxgciqi\"}") + CheckNameAvailabilityParameters model + = BinaryData.fromString("{\"name\":\"bttdumorppxe\",\"type\":\"nzbtbhj\"}") .toObject(CheckNameAvailabilityParameters.class); - Assertions.assertEquals("vhelxprglyatdd", model.name()); - Assertions.assertEquals("cbcuejrjxgciqi", model.type()); + Assertions.assertEquals("bttdumorppxe", model.name()); + Assertions.assertEquals("nzbtbhj", model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CheckNameAvailabilityParameters model = - new CheckNameAvailabilityParameters().withName("vhelxprglyatdd").withType("cbcuejrjxgciqi"); + CheckNameAvailabilityParameters model + = new CheckNameAvailabilityParameters().withName("bttdumorppxe").withType("nzbtbhj"); model = BinaryData.fromObject(model).toObject(CheckNameAvailabilityParameters.class); - Assertions.assertEquals("vhelxprglyatdd", model.name()); - Assertions.assertEquals("cbcuejrjxgciqi", model.type()); + Assertions.assertEquals("bttdumorppxe", model.name()); + Assertions.assertEquals("nzbtbhj", model.type()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityResultInnerTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityResultInnerTests.java index 09da4ffceebbc..57978289100be 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityResultInnerTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/CheckNameAvailabilityResultInnerTests.java @@ -11,20 +11,19 @@ public final class CheckNameAvailabilityResultInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - CheckNameAvailabilityResultInner model = - BinaryData - .fromString("{\"nameAvailable\":false,\"reason\":\"sxsdqrhzoymibm\",\"message\":\"yiba\"}") - .toObject(CheckNameAvailabilityResultInner.class); + CheckNameAvailabilityResultInner model = BinaryData + .fromString("{\"nameAvailable\":false,\"reason\":\"fgohdneuelfphs\",\"message\":\"htozfikdow\"}") + .toObject(CheckNameAvailabilityResultInner.class); Assertions.assertEquals(false, model.nameAvailable()); - Assertions.assertEquals("sxsdqrhzoymibm", model.reason()); + Assertions.assertEquals("fgohdneuelfphs", model.reason()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - CheckNameAvailabilityResultInner model = - new CheckNameAvailabilityResultInner().withNameAvailable(false).withReason("sxsdqrhzoymibm"); + CheckNameAvailabilityResultInner model + = new CheckNameAvailabilityResultInner().withNameAvailable(false).withReason("fgohdneuelfphs"); model = BinaryData.fromObject(model).toObject(CheckNameAvailabilityResultInner.class); Assertions.assertEquals(false, model.nameAvailable()); - Assertions.assertEquals("sxsdqrhzoymibm", model.reason()); + Assertions.assertEquals("fgohdneuelfphs", model.reason()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListMockTests.java index afeba7164dc0f..ffe58f1226a7d 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListMockTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListMockTests.java @@ -31,69 +31,53 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"id\":\"oegokdwbwh\",\"name\":\"z\",\"properties\":{\"description\":\"rvexztvb\",\"providerType\":\"gsfraoyzkoow\",\"company\":\"mnguxawqaldsyu\",\"defaultEndpoint\":\"imerqfobwyznk\",\"aad\":{\"applicationId\":\"utwpfhp\",\"tenantId\":\"m\"},\"managedApplication\":{\"publisherId\":\"kdsnfdsdoakgtdl\",\"offerId\":\"kzevdlhewpusds\"},\"targets\":[{\"id\":\"ogvbbejdcngq\",\"name\":\"oakufgm\",\"description\":\"rwr\",\"acceptedDataFormats\":[\"twaenuuzko\",\"bminrfdwoyuhhzi\",\"iefozbhdmsml\"],\"acceptedContentEncodings\":[\"hoftr\",\"ae\",\"u\",\"ah\"]},{\"id\":\"cslfaoqzpiyylha\",\"name\":\"swhccsphk\",\"description\":\"vwitqscyw\",\"acceptedDataFormats\":[\"woluhczbwemhair\"],\"acceptedContentEncodings\":[\"gzd\",\"msweypqwdxggicc\",\"n\"]},{\"id\":\"huexmk\",\"name\":\"lstvlzywe\",\"description\":\"zrncsdt\",\"acceptedDataFormats\":[\"siypbs\"],\"acceptedContentEncodings\":[\"tg\",\"sl\"]}],\"skus\":[{\"id\":\"cy\",\"name\":\"ukyhejhzis\",\"version\":\"fpel\",\"description\":\"p\",\"restrictedAccessUri\":\"ksrpqv\",\"autoAdd\":false,\"targets\":[\"ehtwdwrft\",\"wib\"],\"quotaDimensions\":[{}],\"pricingDetails\":[{}]}],\"quotaDimensions\":[{\"id\":\"hfwpracstwit\",\"scope\":\"hevxcced\",\"period\":\"nmdyodnwzxl\",\"quota\":58.360874,\"name\":\"nhltiugcxn\",\"description\":\"vwxqibyqunyo\",\"unit\":\"wlmdjrkv\",\"unitPlural\":\"bvfvpdbod\"}],\"pricingDimensions\":[{\"id\":\"sjq\",\"name\":\"krribdeibqi\"},{\"id\":\"kghv\",\"name\":\"dzwmkrefajpj\"},{\"id\":\"wkqnyhg\",\"name\":\"j\"}]}}]}"; + String responseStr + = "{\"value\":[{\"id\":\"bnnhadoocrkvcik\",\"name\":\"vpa\",\"properties\":{\"description\":\"x\",\"providerType\":\"u\",\"company\":\"ik\",\"defaultEndpoint\":\"ggxkallatmelwuip\",\"aad\":{\"applicationId\":\"jzkzi\",\"tenantId\":\"vvcnayr\"},\"managedApplication\":{\"publisherId\":\"nxxmueedndrdv\",\"offerId\":\"kwqqtchealmf\"},\"targets\":[{\"id\":\"aygdvwvgpioh\",\"name\":\"xrtfudxep\",\"description\":\"yqagvrvm\",\"acceptedDataFormats\":[\"ukghimdblxgw\"],\"acceptedContentEncodings\":[\"njhf\",\"xw\",\"szkkfoqre\",\"fkzikfj\"]}],\"skus\":[{\"id\":\"a\",\"name\":\"xwczelpcire\",\"version\":\"feaenwab\",\"description\":\"tkl\",\"restrictedAccessUri\":\"xbjhwuaanozjosph\",\"autoAdd\":false,\"targets\":[\"jrvxaglrv\",\"mjwosytx\"],\"quotaDimensions\":[{},{},{},{}],\"pricingDetails\":[{},{}]}],\"quotaDimensions\":[{\"id\":\"tq\",\"scope\":\"iekkezz\",\"period\":\"hlyfjhdgqgg\",\"quota\":29.760027,\"name\":\"nyga\",\"description\":\"idb\",\"unit\":\"atpxl\",\"unitPlural\":\"xcyjmoadsuvarmy\"},{\"id\":\"mjsjqb\",\"scope\":\"hyxxrwlycoduhpk\",\"period\":\"gymare\",\"quota\":23.266273,\"name\":\"xqugjhkycubedd\",\"description\":\"sofwqmzqalkrmnji\",\"unit\":\"xacqqudfnbyx\",\"unitPlural\":\"aabjyvayffimrz\"},{\"id\":\"uzqogsexnevf\",\"scope\":\"wnwmewzs\",\"period\":\"ceuzsoibjudpfr\",\"quota\":98.30812,\"name\":\"hzv\",\"description\":\"tdwkqbrq\",\"unit\":\"paxh\",\"unitPlural\":\"iilivpdtiirqtd\"},{\"id\":\"axoruzfgsquy\",\"scope\":\"rxxle\",\"period\":\"ramxjezwlwnw\",\"quota\":41.191147,\"name\":\"cvydypatdoo\",\"description\":\"jkniodko\",\"unit\":\"bw\",\"unitPlural\":\"jhemms\"}],\"pricingDimensions\":[{\"id\":\"c\",\"name\":\"dtjinfw\"},{\"id\":\"fltkacjv\",\"name\":\"kdlfoa\"}]}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureQuantumManager manager = AzureQuantumManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.offerings().list("khnzbonlw", com.azure.core.util.Context.NONE); + PagedIterable response = manager.offerings().list("jlt", com.azure.core.util.Context.NONE); - Assertions.assertEquals("oegokdwbwh", response.iterator().next().id()); - Assertions.assertEquals("ogvbbejdcngq", response.iterator().next().properties().targets().get(0).id()); - Assertions.assertEquals("oakufgm", response.iterator().next().properties().targets().get(0).name()); - Assertions.assertEquals("rwr", response.iterator().next().properties().targets().get(0).description()); - Assertions - .assertEquals( - "twaenuuzko", response.iterator().next().properties().targets().get(0).acceptedDataFormats().get(0)); - Assertions - .assertEquals( - "hoftr", response.iterator().next().properties().targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("cy", response.iterator().next().properties().skus().get(0).id()); - Assertions.assertEquals("ukyhejhzis", response.iterator().next().properties().skus().get(0).name()); - Assertions.assertEquals("fpel", response.iterator().next().properties().skus().get(0).version()); - Assertions.assertEquals("p", response.iterator().next().properties().skus().get(0).description()); - Assertions.assertEquals("ksrpqv", response.iterator().next().properties().skus().get(0).restrictedAccessUri()); + Assertions.assertEquals("bnnhadoocrkvcik", response.iterator().next().id()); + Assertions.assertEquals("aygdvwvgpioh", response.iterator().next().properties().targets().get(0).id()); + Assertions.assertEquals("xrtfudxep", response.iterator().next().properties().targets().get(0).name()); + Assertions.assertEquals("yqagvrvm", response.iterator().next().properties().targets().get(0).description()); + Assertions.assertEquals("ukghimdblxgw", + response.iterator().next().properties().targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("njhf", + response.iterator().next().properties().targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("a", response.iterator().next().properties().skus().get(0).id()); + Assertions.assertEquals("xwczelpcire", response.iterator().next().properties().skus().get(0).name()); + Assertions.assertEquals("feaenwab", response.iterator().next().properties().skus().get(0).version()); + Assertions.assertEquals("tkl", response.iterator().next().properties().skus().get(0).description()); + Assertions.assertEquals("xbjhwuaanozjosph", + response.iterator().next().properties().skus().get(0).restrictedAccessUri()); Assertions.assertEquals(false, response.iterator().next().properties().skus().get(0).autoAdd()); - Assertions.assertEquals("ehtwdwrft", response.iterator().next().properties().skus().get(0).targets().get(0)); - Assertions.assertEquals("hfwpracstwit", response.iterator().next().properties().quotaDimensions().get(0).id()); - Assertions.assertEquals("hevxcced", response.iterator().next().properties().quotaDimensions().get(0).scope()); - Assertions - .assertEquals("nmdyodnwzxl", response.iterator().next().properties().quotaDimensions().get(0).period()); - Assertions.assertEquals(58.360874F, response.iterator().next().properties().quotaDimensions().get(0).quota()); - Assertions.assertEquals("nhltiugcxn", response.iterator().next().properties().quotaDimensions().get(0).name()); - Assertions - .assertEquals( - "vwxqibyqunyo", response.iterator().next().properties().quotaDimensions().get(0).description()); - Assertions.assertEquals("wlmdjrkv", response.iterator().next().properties().quotaDimensions().get(0).unit()); - Assertions - .assertEquals("bvfvpdbod", response.iterator().next().properties().quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("sjq", response.iterator().next().properties().pricingDimensions().get(0).id()); - Assertions - .assertEquals("krribdeibqi", response.iterator().next().properties().pricingDimensions().get(0).name()); + Assertions.assertEquals("jrvxaglrv", response.iterator().next().properties().skus().get(0).targets().get(0)); + Assertions.assertEquals("tq", response.iterator().next().properties().quotaDimensions().get(0).id()); + Assertions.assertEquals("iekkezz", response.iterator().next().properties().quotaDimensions().get(0).scope()); + Assertions.assertEquals("hlyfjhdgqgg", + response.iterator().next().properties().quotaDimensions().get(0).period()); + Assertions.assertEquals(29.760027F, response.iterator().next().properties().quotaDimensions().get(0).quota()); + Assertions.assertEquals("nyga", response.iterator().next().properties().quotaDimensions().get(0).name()); + Assertions.assertEquals("idb", response.iterator().next().properties().quotaDimensions().get(0).description()); + Assertions.assertEquals("atpxl", response.iterator().next().properties().quotaDimensions().get(0).unit()); + Assertions.assertEquals("xcyjmoadsuvarmy", + response.iterator().next().properties().quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("c", response.iterator().next().properties().pricingDimensions().get(0).id()); + Assertions.assertEquals("dtjinfw", response.iterator().next().properties().pricingDimensions().get(0).name()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListResultTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListResultTests.java index 264dbedc1a963..061c05fe5d9ea 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListResultTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OfferingsListResultTests.java @@ -21,256 +21,162 @@ public final class OfferingsListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OfferingsListResult model = - BinaryData - .fromString( - "{\"value\":[{\"id\":\"zdn\",\"name\":\"jq\",\"properties\":{\"description\":\"muouqfp\",\"providerType\":\"zw\",\"company\":\"g\",\"defaultEndpoint\":\"tnwu\",\"aad\":{\"applicationId\":\"a\",\"tenantId\":\"ufizuckyf\"},\"managedApplication\":{\"publisherId\":\"fidfvzw\",\"offerId\":\"uht\"},\"targets\":[{\"id\":\"sdkf\",\"name\":\"wxmnteiwao\",\"description\":\"km\",\"acceptedDataFormats\":[\"mmxdcu\"],\"acceptedContentEncodings\":[\"srp\",\"mzidnsezcxtb\",\"sgfyccsnew\"]},{\"id\":\"wzjeiach\",\"name\":\"osfln\",\"description\":\"sfqpteehz\",\"acceptedDataFormats\":[\"pyqr\",\"mzinpvswjdk\",\"rsoodqxhcrmnoh\"],\"acceptedContentEncodings\":[\"kwh\",\"soifiyipjxsqw\"]}],\"skus\":[{\"id\":\"bznorcjxvsnby\",\"name\":\"abnmocpcyshu\",\"version\":\"afbljjgpbtoqcjmk\",\"description\":\"a\",\"restrictedAccessUri\":\"qidtqajzyu\",\"autoAdd\":true,\"targets\":[\"jkrlkhbzhfepg\"],\"quotaDimensions\":[{}],\"pricingDetails\":[{},{},{}]}],\"quotaDimensions\":[{\"id\":\"c\",\"scope\":\"c\",\"period\":\"ierhhbcsglummaj\",\"quota\":29.64415,\"name\":\"dxob\",\"description\":\"dxkqpx\",\"unit\":\"ajionpimexgstxg\",\"unitPlural\":\"odgmaajrmvdjwz\"}],\"pricingDimensions\":[{\"id\":\"mcl\",\"name\":\"ijcoejctb\"},{\"id\":\"qsqsy\",\"name\":\"kbfkg\"},{\"id\":\"dkexxppofm\",\"name\":\"x\"},{\"id\":\"jpgd\",\"name\":\"ocjjxhvpmouexh\"}]}},{\"id\":\"xibqeojnx\",\"name\":\"zvddntwndeicbtwn\",\"properties\":{\"description\":\"oqvuhr\",\"providerType\":\"f\",\"company\":\"yd\",\"defaultEndpoint\":\"lmjthjq\",\"aad\":{\"applicationId\":\"yeicxmqciwqvhk\",\"tenantId\":\"xuigdtopbobj\"},\"managedApplication\":{\"publisherId\":\"m\",\"offerId\":\"u\"},\"targets\":[{\"id\":\"hrzayvvtpgvdf\",\"name\":\"otkftutqxlngx\",\"description\":\"fgugnxkrxdqmid\",\"acceptedDataFormats\":[\"zrvqdr\",\"bhj\"],\"acceptedContentEncodings\":[\"gehoqfbowskany\",\"tzlcuiywgqywgn\"]}],\"skus\":[{\"id\":\"nhzgpphrcgyn\",\"name\":\"cpecfvmmcoofs\",\"version\":\"zevgb\",\"description\":\"jqabcypmivkwlzuv\",\"restrictedAccessUri\":\"fwnfnb\",\"autoAdd\":false,\"targets\":[\"nlebxetqgtzxd\",\"nqbqqwxr\",\"feallnwsu\",\"isnjampmngnz\"],\"quotaDimensions\":[{},{},{},{}],\"pricingDetails\":[{},{},{}]},{\"id\":\"ooch\",\"name\":\"onq\",\"version\":\"kvlrxnj\",\"description\":\"seiphe\",\"restrictedAccessUri\":\"lokeyy\",\"autoAdd\":false,\"targets\":[\"dlwtgrhpdj\",\"jumasx\",\"zj\",\"qyeg\"],\"quotaDimensions\":[{},{},{},{}],\"pricingDetails\":[{}]},{\"id\":\"xhejjzzvdud\",\"name\":\"dslfhotwmcy\",\"version\":\"wlbjnpgacftade\",\"description\":\"nltyfsoppusuesnz\",\"restrictedAccessUri\":\"ej\",\"autoAdd\":true,\"targets\":[\"xzdmohctb\",\"vudwx\",\"ndnvo\",\"gujjugwdkcglh\"],\"quotaDimensions\":[{},{}],\"pricingDetails\":[{},{},{}]}],\"quotaDimensions\":[{\"id\":\"gdtjixhbkuofqwey\",\"scope\":\"menevfyexfwh\",\"period\":\"cibvyvdcsitynn\",\"quota\":87.51548,\"name\":\"ectehf\",\"description\":\"scjeypv\",\"unit\":\"zrkgqhcjrefovg\",\"unitPlural\":\"qsl\"},{\"id\":\"yvxyqjp\",\"scope\":\"attpngjcrcczsq\",\"period\":\"hvmdajvnysounq\",\"quota\":2.2711873,\"name\":\"oaeupfhyhltrpmo\",\"description\":\"mcmatuokthfuiu\",\"unit\":\"dsfcpkvxodpuoz\",\"unitPlural\":\"zydagfuaxbezyiuo\"},{\"id\":\"twhrdxwzywqsm\",\"scope\":\"ureximoryocfs\",\"period\":\"s\",\"quota\":76.61667,\"name\":\"ys\",\"description\":\"i\",\"unit\":\"xhqyudxorrqnb\",\"unitPlural\":\"czvyifq\"}],\"pricingDimensions\":[{\"id\":\"vjsllrmvvdfw\",\"name\":\"kpnpulexxbczwtr\"},{\"id\":\"iqzbq\",\"name\":\"sovmyokacspkwl\"}]}}],\"nextLink\":\"dobpxjmflbvvn\"}") - .toObject(OfferingsListResult.class); - Assertions.assertEquals("zdn", model.value().get(0).id()); - Assertions.assertEquals("sdkf", model.value().get(0).properties().targets().get(0).id()); - Assertions.assertEquals("wxmnteiwao", model.value().get(0).properties().targets().get(0).name()); - Assertions.assertEquals("km", model.value().get(0).properties().targets().get(0).description()); - Assertions - .assertEquals("mmxdcu", model.value().get(0).properties().targets().get(0).acceptedDataFormats().get(0)); - Assertions - .assertEquals("srp", model.value().get(0).properties().targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("bznorcjxvsnby", model.value().get(0).properties().skus().get(0).id()); - Assertions.assertEquals("abnmocpcyshu", model.value().get(0).properties().skus().get(0).name()); - Assertions.assertEquals("afbljjgpbtoqcjmk", model.value().get(0).properties().skus().get(0).version()); - Assertions.assertEquals("a", model.value().get(0).properties().skus().get(0).description()); - Assertions.assertEquals("qidtqajzyu", model.value().get(0).properties().skus().get(0).restrictedAccessUri()); - Assertions.assertEquals(true, model.value().get(0).properties().skus().get(0).autoAdd()); - Assertions.assertEquals("jkrlkhbzhfepg", model.value().get(0).properties().skus().get(0).targets().get(0)); - Assertions.assertEquals("c", model.value().get(0).properties().quotaDimensions().get(0).id()); - Assertions.assertEquals("c", model.value().get(0).properties().quotaDimensions().get(0).scope()); - Assertions.assertEquals("ierhhbcsglummaj", model.value().get(0).properties().quotaDimensions().get(0).period()); - Assertions.assertEquals(29.64415F, model.value().get(0).properties().quotaDimensions().get(0).quota()); - Assertions.assertEquals("dxob", model.value().get(0).properties().quotaDimensions().get(0).name()); - Assertions.assertEquals("dxkqpx", model.value().get(0).properties().quotaDimensions().get(0).description()); - Assertions.assertEquals("ajionpimexgstxg", model.value().get(0).properties().quotaDimensions().get(0).unit()); - Assertions - .assertEquals("odgmaajrmvdjwz", model.value().get(0).properties().quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("mcl", model.value().get(0).properties().pricingDimensions().get(0).id()); - Assertions.assertEquals("ijcoejctb", model.value().get(0).properties().pricingDimensions().get(0).name()); - Assertions.assertEquals("dobpxjmflbvvn", model.nextLink()); + OfferingsListResult model = BinaryData.fromString( + "{\"value\":[{\"id\":\"huxinpmqnj\",\"name\":\"wixjsprozvcp\",\"properties\":{\"description\":\"gjvw\",\"providerType\":\"datscmd\",\"company\":\"jhulsuuvmkjo\",\"defaultEndpoint\":\"rwfndiod\",\"aad\":{\"applicationId\":\"lwejdpv\",\"tenantId\":\"yoqpsoaccta\"},\"managedApplication\":{\"publisherId\":\"ljlahbcryf\",\"offerId\":\"fdosyg\"},\"targets\":[{\"id\":\"ojakhmsbzjhcrze\",\"name\":\"phlxa\",\"description\":\"thqt\",\"acceptedDataFormats\":[\"jbp\",\"zfsinzgvf\",\"jrwzox\"],\"acceptedContentEncodings\":[\"felluwfzitonpe\",\"fpjkjlxofp\"]},{\"id\":\"hpf\",\"name\":\"ypininm\",\"description\":\"huyb\",\"acceptedDataFormats\":[\"odepoogin\",\"vamih\"],\"acceptedContentEncodings\":[\"narxzxtheotus\",\"vyevcciqi\",\"nhungbw\"]},{\"id\":\"rnfygxgispem\",\"name\":\"zfkufubljofx\",\"description\":\"ofjaeqjhqjb\",\"acceptedDataFormats\":[\"msmjqulngsntn\"],\"acceptedContentEncodings\":[\"kzgcwrwcl\",\"xwrljdouskcqvkoc\",\"cjdkwtnhxbnjbi\"]},{\"id\":\"qrglssainqpjwn\",\"name\":\"ljfmppee\",\"description\":\"mgxsab\",\"acceptedDataFormats\":[\"duuji\",\"c\",\"czdzev\",\"dhkrwpdappdsbdk\"],\"acceptedContentEncodings\":[\"wjfeusnhutjel\",\"mrldhu\"]}],\"skus\":[{\"id\":\"datqxhocdgeabl\",\"name\":\"huticndvkao\",\"version\":\"yiftyhxhuro\",\"description\":\"tyxolniwpwc\",\"restrictedAccessUri\":\"jfkgiawxk\",\"autoAdd\":false,\"targets\":[\"wckbasyypnd\",\"hsgcbacphejkot\",\"nqgoulzndli\",\"wyqkgfgibm\"],\"quotaDimensions\":[{},{}],\"pricingDetails\":[{},{},{},{}]}],\"quotaDimensions\":[{\"id\":\"rxybz\",\"scope\":\"e\",\"period\":\"ytb\",\"quota\":19.418764,\"name\":\"ouf\",\"description\":\"mnkzsmod\",\"unit\":\"lougpbkw\",\"unitPlural\":\"utduqktapspwgcu\"},{\"id\":\"tumkdosvqwhbm\",\"scope\":\"bbjfddgmbmbexp\",\"period\":\"htqqrolfp\",\"quota\":1.6021609,\"name\":\"lgbquxig\",\"description\":\"jgzjaoyfhrtx\",\"unit\":\"n\",\"unitPlural\":\"kujysvlejuvfq\"},{\"id\":\"rlyxwjkcprbnw\",\"scope\":\"gjvtbv\",\"period\":\"sszdnru\",\"quota\":66.73355,\"name\":\"hmuouqfprwzwbn\",\"description\":\"itnwuizgazxufi\",\"unit\":\"ckyfih\",\"unitPlural\":\"idf\"}],\"pricingDimensions\":[{\"id\":\"zuhtymwisdkfthwx\",\"name\":\"t\"},{\"id\":\"waopvkmijcmmxd\",\"name\":\"fufsrpymzi\"},{\"id\":\"sezcxtb\",\"name\":\"gfycc\"}]}},{\"id\":\"ewmdw\",\"name\":\"eiachboosflnr\",\"properties\":{\"description\":\"qpteeh\",\"providerType\":\"vypyqrimzinpv\",\"company\":\"jdkirsoodqx\",\"defaultEndpoint\":\"rmnohj\",\"aad\":{\"applicationId\":\"whdsoifiyip\",\"tenantId\":\"sqwpgrjb\"},\"managedApplication\":{\"publisherId\":\"rcjxvsnbyxqabn\",\"offerId\":\"cpc\"},\"targets\":[{\"id\":\"rzafbljjgpbtoqcj\",\"name\":\"ljavbqid\",\"description\":\"ajzyul\",\"acceptedDataFormats\":[\"dj\"],\"acceptedContentEncodings\":[\"khbzhfepgzg\",\"e\"]},{\"id\":\"locx\",\"name\":\"paierh\",\"description\":\"csglum\",\"acceptedDataFormats\":[\"tjaodxobnb\",\"xkqpxo\",\"ajionpimexgstxg\",\"po\"],\"acceptedContentEncodings\":[\"aajrm\",\"djwzrlov\",\"clwhijcoejctbz\",\"qsqsy\"]},{\"id\":\"kbfkg\",\"name\":\"dkexxppofm\",\"description\":\"x\",\"acceptedDataFormats\":[\"pg\"],\"acceptedContentEncodings\":[\"ocjjxhvpmouexh\"]}],\"skus\":[{\"id\":\"bqe\",\"name\":\"nxqbzvddn\",\"version\":\"ndei\",\"description\":\"twnpzaoqvuhrhcf\",\"restrictedAccessUri\":\"yd\",\"autoAdd\":false,\"targets\":[\"t\",\"jqkwpyeicx\",\"qciwqvhkhixuigdt\"],\"quotaDimensions\":[{}],\"pricingDetails\":[{},{},{},{}]},{\"id\":\"og\",\"name\":\"e\",\"version\":\"a\",\"description\":\"uhrzayvvt\",\"restrictedAccessUri\":\"vdfgiotk\",\"autoAdd\":false,\"targets\":[\"xlngx\",\"efgugnxk\"],\"quotaDimensions\":[{},{}],\"pricingDetails\":[{}]},{\"id\":\"dt\",\"name\":\"zrvqdr\",\"version\":\"hjybigehoqfbo\",\"description\":\"kanyktzlcuiywg\",\"restrictedAccessUri\":\"wgndrvynhzgpp\",\"autoAdd\":true,\"targets\":[\"nc\",\"cpecfvmmcoofs\",\"lzevgbmqjqab\",\"y\"],\"quotaDimensions\":[{},{},{},{}],\"pricingDetails\":[{},{},{},{}]}],\"quotaDimensions\":[{\"id\":\"uvcc\",\"scope\":\"nfnbacfionlebxe\",\"period\":\"gtzxdpn\",\"quota\":19.734276,\"name\":\"wxrjfeallnwsub\",\"description\":\"njampm\",\"unit\":\"nzscxa\",\"unitPlural\":\"ooch\"}],\"pricingDimensions\":[{\"id\":\"qvpkvlrxnjeaseip\",\"name\":\"oflokey\"}]}}],\"nextLink\":\"enjbdlwtgrhp\"}") + .toObject(OfferingsListResult.class); + Assertions.assertEquals("huxinpmqnj", model.value().get(0).id()); + Assertions.assertEquals("ojakhmsbzjhcrze", model.value().get(0).properties().targets().get(0).id()); + Assertions.assertEquals("phlxa", model.value().get(0).properties().targets().get(0).name()); + Assertions.assertEquals("thqt", model.value().get(0).properties().targets().get(0).description()); + Assertions.assertEquals("jbp", model.value().get(0).properties().targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("felluwfzitonpe", + model.value().get(0).properties().targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("datqxhocdgeabl", model.value().get(0).properties().skus().get(0).id()); + Assertions.assertEquals("huticndvkao", model.value().get(0).properties().skus().get(0).name()); + Assertions.assertEquals("yiftyhxhuro", model.value().get(0).properties().skus().get(0).version()); + Assertions.assertEquals("tyxolniwpwc", model.value().get(0).properties().skus().get(0).description()); + Assertions.assertEquals("jfkgiawxk", model.value().get(0).properties().skus().get(0).restrictedAccessUri()); + Assertions.assertEquals(false, model.value().get(0).properties().skus().get(0).autoAdd()); + Assertions.assertEquals("wckbasyypnd", model.value().get(0).properties().skus().get(0).targets().get(0)); + Assertions.assertEquals("rxybz", model.value().get(0).properties().quotaDimensions().get(0).id()); + Assertions.assertEquals("e", model.value().get(0).properties().quotaDimensions().get(0).scope()); + Assertions.assertEquals("ytb", model.value().get(0).properties().quotaDimensions().get(0).period()); + Assertions.assertEquals(19.418764F, model.value().get(0).properties().quotaDimensions().get(0).quota()); + Assertions.assertEquals("ouf", model.value().get(0).properties().quotaDimensions().get(0).name()); + Assertions.assertEquals("mnkzsmod", model.value().get(0).properties().quotaDimensions().get(0).description()); + Assertions.assertEquals("lougpbkw", model.value().get(0).properties().quotaDimensions().get(0).unit()); + Assertions.assertEquals("utduqktapspwgcu", + model.value().get(0).properties().quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("zuhtymwisdkfthwx", model.value().get(0).properties().pricingDimensions().get(0).id()); + Assertions.assertEquals("t", model.value().get(0).properties().pricingDimensions().get(0).name()); + Assertions.assertEquals("enjbdlwtgrhp", model.nextLink()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OfferingsListResult model = - new OfferingsListResult() + OfferingsListResult model + = new OfferingsListResult() .withValue( Arrays .asList( - new ProviderDescriptionInner() - .withId("zdn") - .withProperties( - new ProviderProperties() - .withAad(new ProviderPropertiesAad()) - .withManagedApplication(new ProviderPropertiesManagedApplication()) + new ProviderDescriptionInner().withId("huxinpmqnj") + .withProperties(new ProviderProperties().withAad(new ProviderPropertiesAad()) + .withManagedApplication(new ProviderPropertiesManagedApplication()) + .withTargets(Arrays.asList(new TargetDescription().withId("ojakhmsbzjhcrze") + .withName("phlxa").withDescription("thqt") + .withAcceptedDataFormats(Arrays.asList("jbp", "zfsinzgvf", "jrwzox")) + .withAcceptedContentEncodings(Arrays.asList("felluwfzitonpe", "fpjkjlxofp")), + new TargetDescription().withId("hpf").withName("ypininm") + .withDescription("huyb") + .withAcceptedDataFormats(Arrays.asList("odepoogin", "vamih")) + .withAcceptedContentEncodings( + Arrays.asList("narxzxtheotus", "vyevcciqi", "nhungbw")), + new TargetDescription().withId("rnfygxgispem").withName("zfkufubljofx") + .withDescription("ofjaeqjhqjb") + .withAcceptedDataFormats(Arrays.asList("msmjqulngsntn")) + .withAcceptedContentEncodings( + Arrays.asList("kzgcwrwcl", "xwrljdouskcqvkoc", "cjdkwtnhxbnjbi")), + new TargetDescription().withId("qrglssainqpjwn").withName("ljfmppee") + .withDescription("mgxsab") + .withAcceptedDataFormats( + Arrays.asList("duuji", "c", "czdzev", "dhkrwpdappdsbdk")) + .withAcceptedContentEncodings(Arrays.asList("wjfeusnhutjel", "mrldhu")))) + .withSkus(Arrays.asList(new SkuDescription().withId("datqxhocdgeabl") + .withName("huticndvkao").withVersion("yiftyhxhuro") + .withDescription("tyxolniwpwc").withRestrictedAccessUri("jfkgiawxk") + .withAutoAdd(false) .withTargets( - Arrays - .asList( - new TargetDescription() - .withId("sdkf") - .withName("wxmnteiwao") - .withDescription("km") - .withAcceptedDataFormats(Arrays.asList("mmxdcu")) - .withAcceptedContentEncodings( - Arrays.asList("srp", "mzidnsezcxtb", "sgfyccsnew")), - new TargetDescription() - .withId("wzjeiach") - .withName("osfln") - .withDescription("sfqpteehz") - .withAcceptedDataFormats( - Arrays.asList("pyqr", "mzinpvswjdk", "rsoodqxhcrmnoh")) - .withAcceptedContentEncodings( - Arrays.asList("kwh", "soifiyipjxsqw")))) - .withSkus( - Arrays - .asList( - new SkuDescription() - .withId("bznorcjxvsnby") - .withName("abnmocpcyshu") - .withVersion("afbljjgpbtoqcjmk") - .withDescription("a") - .withRestrictedAccessUri("qidtqajzyu") - .withAutoAdd(true) - .withTargets(Arrays.asList("jkrlkhbzhfepg")) - .withQuotaDimensions(Arrays.asList(new QuotaDimension())) - .withPricingDetails( - Arrays - .asList( - new PricingDetail(), - new PricingDetail(), - new PricingDetail())))) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("c") - .withScope("c") - .withPeriod("ierhhbcsglummaj") - .withQuota(29.64415F) - .withName("dxob") - .withDescription("dxkqpx") - .withUnit("ajionpimexgstxg") - .withUnitPlural("odgmaajrmvdjwz"))) - .withPricingDimensions( - Arrays - .asList( - new PricingDimension().withId("mcl").withName("ijcoejctb"), - new PricingDimension().withId("qsqsy").withName("kbfkg"), - new PricingDimension().withId("dkexxppofm").withName("x"), - new PricingDimension().withId("jpgd").withName("ocjjxhvpmouexh")))), - new ProviderDescriptionInner() - .withId("xibqeojnx") - .withProperties( - new ProviderProperties() - .withAad(new ProviderPropertiesAad()) - .withManagedApplication(new ProviderPropertiesManagedApplication()) - .withTargets( - Arrays - .asList( - new TargetDescription() - .withId("hrzayvvtpgvdf") - .withName("otkftutqxlngx") - .withDescription("fgugnxkrxdqmid") - .withAcceptedDataFormats(Arrays.asList("zrvqdr", "bhj")) - .withAcceptedContentEncodings( - Arrays.asList("gehoqfbowskany", "tzlcuiywgqywgn")))) - .withSkus( - Arrays - .asList( - new SkuDescription() - .withId("nhzgpphrcgyn") - .withName("cpecfvmmcoofs") - .withVersion("zevgb") - .withDescription("jqabcypmivkwlzuv") - .withRestrictedAccessUri("fwnfnb") - .withAutoAdd(false) - .withTargets( - Arrays - .asList( - "nlebxetqgtzxd", - "nqbqqwxr", - "feallnwsu", - "isnjampmngnz")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension(), - new QuotaDimension(), - new QuotaDimension(), - new QuotaDimension())) - .withPricingDetails( - Arrays - .asList( - new PricingDetail(), - new PricingDetail(), - new PricingDetail())), - new SkuDescription() - .withId("ooch") - .withName("onq") - .withVersion("kvlrxnj") - .withDescription("seiphe") - .withRestrictedAccessUri("lokeyy") - .withAutoAdd(false) - .withTargets( - Arrays.asList("dlwtgrhpdj", "jumasx", "zj", "qyeg")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension(), - new QuotaDimension(), - new QuotaDimension(), - new QuotaDimension())) - .withPricingDetails(Arrays.asList(new PricingDetail())), - new SkuDescription() - .withId("xhejjzzvdud") - .withName("dslfhotwmcy") - .withVersion("wlbjnpgacftade") - .withDescription("nltyfsoppusuesnz") - .withRestrictedAccessUri("ej") - .withAutoAdd(true) - .withTargets( - Arrays - .asList("xzdmohctb", "vudwx", "ndnvo", "gujjugwdkcglh")) - .withQuotaDimensions( - Arrays.asList(new QuotaDimension(), new QuotaDimension())) - .withPricingDetails( - Arrays - .asList( - new PricingDetail(), - new PricingDetail(), - new PricingDetail())))) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("gdtjixhbkuofqwey") - .withScope("menevfyexfwh") - .withPeriod("cibvyvdcsitynn") - .withQuota(87.51548F) - .withName("ectehf") - .withDescription("scjeypv") - .withUnit("zrkgqhcjrefovg") - .withUnitPlural("qsl"), - new QuotaDimension() - .withId("yvxyqjp") - .withScope("attpngjcrcczsq") - .withPeriod("hvmdajvnysounq") - .withQuota(2.2711873F) - .withName("oaeupfhyhltrpmo") - .withDescription("mcmatuokthfuiu") - .withUnit("dsfcpkvxodpuoz") - .withUnitPlural("zydagfuaxbezyiuo"), - new QuotaDimension() - .withId("twhrdxwzywqsm") - .withScope("ureximoryocfs") - .withPeriod("s") - .withQuota(76.61667F) - .withName("ys") - .withDescription("i") - .withUnit("xhqyudxorrqnb") - .withUnitPlural("czvyifq"))) - .withPricingDimensions( - Arrays - .asList( - new PricingDimension() - .withId("vjsllrmvvdfw") - .withName("kpnpulexxbczwtr"), - new PricingDimension() - .withId("iqzbq") - .withName("sovmyokacspkwl")))))) - .withNextLink("dobpxjmflbvvn"); + Arrays.asList("wckbasyypnd", "hsgcbacphejkot", "nqgoulzndli", "wyqkgfgibm")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension(), new QuotaDimension())) + .withPricingDetails(Arrays.asList(new PricingDetail(), new PricingDetail(), + new PricingDetail(), new PricingDetail())))) + .withQuotaDimensions(Arrays.asList( + new QuotaDimension().withId("rxybz").withScope("e").withPeriod("ytb") + .withQuota(19.418764F).withName("ouf").withDescription("mnkzsmod") + .withUnit("lougpbkw").withUnitPlural("utduqktapspwgcu"), + new QuotaDimension() + .withId("tumkdosvqwhbm").withScope("bbjfddgmbmbexp").withPeriod("htqqrolfp") + .withQuota(1.6021609F).withName("lgbquxig").withDescription("jgzjaoyfhrtx") + .withUnit("n").withUnitPlural("kujysvlejuvfq"), + new QuotaDimension().withId("rlyxwjkcprbnw").withScope("gjvtbv") + .withPeriod("sszdnru").withQuota(66.73355F).withName("hmuouqfprwzwbn") + .withDescription("itnwuizgazxufi").withUnit("ckyfih") + .withUnitPlural("idf"))) + .withPricingDimensions( + Arrays.asList(new PricingDimension().withId("zuhtymwisdkfthwx").withName("t"), + new PricingDimension().withId("waopvkmijcmmxd") + .withName("fufsrpymzi"), + new PricingDimension().withId("sezcxtb").withName("gfycc")))), + new ProviderDescriptionInner().withId("ewmdw").withProperties(new ProviderProperties() + .withAad(new ProviderPropertiesAad()) + .withManagedApplication(new ProviderPropertiesManagedApplication()) + .withTargets(Arrays.asList( + new TargetDescription().withId("rzafbljjgpbtoqcj").withName("ljavbqid") + .withDescription("ajzyul").withAcceptedDataFormats(Arrays.asList("dj")) + .withAcceptedContentEncodings(Arrays.asList("khbzhfepgzg", "e")), + new TargetDescription().withId("locx").withName("paierh").withDescription("csglum") + .withAcceptedDataFormats( + Arrays.asList("tjaodxobnb", "xkqpxo", "ajionpimexgstxg", "po")) + .withAcceptedContentEncodings( + Arrays.asList("aajrm", "djwzrlov", "clwhijcoejctbz", "qsqsy")), + new TargetDescription().withId("kbfkg").withName("dkexxppofm").withDescription("x") + .withAcceptedDataFormats(Arrays.asList("pg")) + .withAcceptedContentEncodings(Arrays.asList("ocjjxhvpmouexh")))) + .withSkus(Arrays.asList( + new SkuDescription().withId("bqe").withName("nxqbzvddn").withVersion("ndei") + .withDescription("twnpzaoqvuhrhcf").withRestrictedAccessUri("yd") + .withAutoAdd(false) + .withTargets(Arrays.asList("t", "jqkwpyeicx", "qciwqvhkhixuigdt")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension())) + .withPricingDetails( + Arrays.asList(new PricingDetail(), new PricingDetail(), new PricingDetail(), + new PricingDetail())), + new SkuDescription().withId("og").withName("e").withVersion("a") + .withDescription("uhrzayvvt").withRestrictedAccessUri("vdfgiotk") + .withAutoAdd(false).withTargets(Arrays.asList("xlngx", "efgugnxk")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension(), new QuotaDimension())) + .withPricingDetails(Arrays.asList(new PricingDetail())), + new SkuDescription().withId("dt").withName("zrvqdr").withVersion("hjybigehoqfbo") + .withDescription("kanyktzlcuiywg").withRestrictedAccessUri("wgndrvynhzgpp") + .withAutoAdd(true) + .withTargets(Arrays.asList("nc", "cpecfvmmcoofs", "lzevgbmqjqab", "y")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension(), new QuotaDimension(), + new QuotaDimension(), new QuotaDimension())) + .withPricingDetails(Arrays.asList(new PricingDetail(), new PricingDetail(), + new PricingDetail(), new PricingDetail())))) + .withQuotaDimensions( + Arrays.asList(new QuotaDimension().withId("uvcc").withScope("nfnbacfionlebxe") + .withPeriod("gtzxdpn").withQuota(19.734276F).withName("wxrjfeallnwsub") + .withDescription("njampm").withUnit("nzscxa").withUnitPlural("ooch"))) + .withPricingDimensions(Arrays + .asList(new PricingDimension().withId("qvpkvlrxnjeaseip").withName("oflokey")))))) + .withNextLink("enjbdlwtgrhp"); model = BinaryData.fromObject(model).toObject(OfferingsListResult.class); - Assertions.assertEquals("zdn", model.value().get(0).id()); - Assertions.assertEquals("sdkf", model.value().get(0).properties().targets().get(0).id()); - Assertions.assertEquals("wxmnteiwao", model.value().get(0).properties().targets().get(0).name()); - Assertions.assertEquals("km", model.value().get(0).properties().targets().get(0).description()); - Assertions - .assertEquals("mmxdcu", model.value().get(0).properties().targets().get(0).acceptedDataFormats().get(0)); - Assertions - .assertEquals("srp", model.value().get(0).properties().targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("bznorcjxvsnby", model.value().get(0).properties().skus().get(0).id()); - Assertions.assertEquals("abnmocpcyshu", model.value().get(0).properties().skus().get(0).name()); - Assertions.assertEquals("afbljjgpbtoqcjmk", model.value().get(0).properties().skus().get(0).version()); - Assertions.assertEquals("a", model.value().get(0).properties().skus().get(0).description()); - Assertions.assertEquals("qidtqajzyu", model.value().get(0).properties().skus().get(0).restrictedAccessUri()); - Assertions.assertEquals(true, model.value().get(0).properties().skus().get(0).autoAdd()); - Assertions.assertEquals("jkrlkhbzhfepg", model.value().get(0).properties().skus().get(0).targets().get(0)); - Assertions.assertEquals("c", model.value().get(0).properties().quotaDimensions().get(0).id()); - Assertions.assertEquals("c", model.value().get(0).properties().quotaDimensions().get(0).scope()); - Assertions.assertEquals("ierhhbcsglummaj", model.value().get(0).properties().quotaDimensions().get(0).period()); - Assertions.assertEquals(29.64415F, model.value().get(0).properties().quotaDimensions().get(0).quota()); - Assertions.assertEquals("dxob", model.value().get(0).properties().quotaDimensions().get(0).name()); - Assertions.assertEquals("dxkqpx", model.value().get(0).properties().quotaDimensions().get(0).description()); - Assertions.assertEquals("ajionpimexgstxg", model.value().get(0).properties().quotaDimensions().get(0).unit()); - Assertions - .assertEquals("odgmaajrmvdjwz", model.value().get(0).properties().quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("mcl", model.value().get(0).properties().pricingDimensions().get(0).id()); - Assertions.assertEquals("ijcoejctb", model.value().get(0).properties().pricingDimensions().get(0).name()); - Assertions.assertEquals("dobpxjmflbvvn", model.nextLink()); + Assertions.assertEquals("huxinpmqnj", model.value().get(0).id()); + Assertions.assertEquals("ojakhmsbzjhcrze", model.value().get(0).properties().targets().get(0).id()); + Assertions.assertEquals("phlxa", model.value().get(0).properties().targets().get(0).name()); + Assertions.assertEquals("thqt", model.value().get(0).properties().targets().get(0).description()); + Assertions.assertEquals("jbp", model.value().get(0).properties().targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("felluwfzitonpe", + model.value().get(0).properties().targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("datqxhocdgeabl", model.value().get(0).properties().skus().get(0).id()); + Assertions.assertEquals("huticndvkao", model.value().get(0).properties().skus().get(0).name()); + Assertions.assertEquals("yiftyhxhuro", model.value().get(0).properties().skus().get(0).version()); + Assertions.assertEquals("tyxolniwpwc", model.value().get(0).properties().skus().get(0).description()); + Assertions.assertEquals("jfkgiawxk", model.value().get(0).properties().skus().get(0).restrictedAccessUri()); + Assertions.assertEquals(false, model.value().get(0).properties().skus().get(0).autoAdd()); + Assertions.assertEquals("wckbasyypnd", model.value().get(0).properties().skus().get(0).targets().get(0)); + Assertions.assertEquals("rxybz", model.value().get(0).properties().quotaDimensions().get(0).id()); + Assertions.assertEquals("e", model.value().get(0).properties().quotaDimensions().get(0).scope()); + Assertions.assertEquals("ytb", model.value().get(0).properties().quotaDimensions().get(0).period()); + Assertions.assertEquals(19.418764F, model.value().get(0).properties().quotaDimensions().get(0).quota()); + Assertions.assertEquals("ouf", model.value().get(0).properties().quotaDimensions().get(0).name()); + Assertions.assertEquals("mnkzsmod", model.value().get(0).properties().quotaDimensions().get(0).description()); + Assertions.assertEquals("lougpbkw", model.value().get(0).properties().quotaDimensions().get(0).unit()); + Assertions.assertEquals("utduqktapspwgcu", + model.value().get(0).properties().quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("zuhtymwisdkfthwx", model.value().get(0).properties().pricingDimensions().get(0).id()); + Assertions.assertEquals("t", model.value().get(0).properties().pricingDimensions().get(0).name()); + Assertions.assertEquals("enjbdlwtgrhp", model.nextLink()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationDisplayTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationDisplayTests.java index 227dc4ba15f72..e04aa4a6afbe2 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationDisplayTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationDisplayTests.java @@ -11,29 +11,23 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"vypomgkopkwho\",\"resource\":\"pajqgxysm\",\"operation\":\"mbqfqvmk\",\"description\":\"oz\"}") - .toObject(OperationDisplay.class); - Assertions.assertEquals("vypomgkopkwho", model.provider()); - Assertions.assertEquals("pajqgxysm", model.resource()); - Assertions.assertEquals("mbqfqvmk", model.operation()); - Assertions.assertEquals("oz", model.description()); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"eotg\",\"resource\":\"l\",\"operation\":\"muwlauwzizxbm\",\"description\":\"cjefuzmu\"}") + .toObject(OperationDisplay.class); + Assertions.assertEquals("eotg", model.provider()); + Assertions.assertEquals("l", model.resource()); + Assertions.assertEquals("muwlauwzizxbm", model.operation()); + Assertions.assertEquals("cjefuzmu", model.description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationDisplay model = - new OperationDisplay() - .withProvider("vypomgkopkwho") - .withResource("pajqgxysm") - .withOperation("mbqfqvmk") - .withDescription("oz"); + OperationDisplay model = new OperationDisplay().withProvider("eotg").withResource("l") + .withOperation("muwlauwzizxbm").withDescription("cjefuzmu"); model = BinaryData.fromObject(model).toObject(OperationDisplay.class); - Assertions.assertEquals("vypomgkopkwho", model.provider()); - Assertions.assertEquals("pajqgxysm", model.resource()); - Assertions.assertEquals("mbqfqvmk", model.operation()); - Assertions.assertEquals("oz", model.description()); + Assertions.assertEquals("eotg", model.provider()); + Assertions.assertEquals("l", model.resource()); + Assertions.assertEquals("muwlauwzizxbm", model.operation()); + Assertions.assertEquals("cjefuzmu", model.description()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationInnerTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationInnerTests.java index af5a47524cb35..ace4359356999 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationInnerTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationInnerTests.java @@ -12,37 +12,28 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"odfvuefywsbp\",\"isDataAction\":false,\"display\":{\"provider\":\"hrfou\",\"resource\":\"taakc\",\"operation\":\"iyzvqtmnub\",\"description\":\"kpzksmondjmq\"}}") - .toObject(OperationInner.class); - Assertions.assertEquals("odfvuefywsbp", model.name()); - Assertions.assertEquals(false, model.isDataAction()); - Assertions.assertEquals("hrfou", model.display().provider()); - Assertions.assertEquals("taakc", model.display().resource()); - Assertions.assertEquals("iyzvqtmnub", model.display().operation()); - Assertions.assertEquals("kpzksmondjmq", model.display().description()); + OperationInner model = BinaryData.fromString( + "{\"name\":\"bqdxbx\",\"isDataAction\":true,\"display\":{\"provider\":\"gqxndlkzgxhuripl\",\"resource\":\"odxun\",\"operation\":\"ebxmubyynt\",\"description\":\"rbqtkoie\"}}") + .toObject(OperationInner.class); + Assertions.assertEquals("bqdxbx", model.name()); + Assertions.assertEquals(true, model.isDataAction()); + Assertions.assertEquals("gqxndlkzgxhuripl", model.display().provider()); + Assertions.assertEquals("odxun", model.display().resource()); + Assertions.assertEquals("ebxmubyynt", model.display().operation()); + Assertions.assertEquals("rbqtkoie", model.display().description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationInner model = - new OperationInner() - .withName("odfvuefywsbp") - .withIsDataAction(false) - .withDisplay( - new OperationDisplay() - .withProvider("hrfou") - .withResource("taakc") - .withOperation("iyzvqtmnub") - .withDescription("kpzksmondjmq")); + OperationInner model = new OperationInner().withName("bqdxbx").withIsDataAction(true) + .withDisplay(new OperationDisplay().withProvider("gqxndlkzgxhuripl").withResource("odxun") + .withOperation("ebxmubyynt").withDescription("rbqtkoie")); model = BinaryData.fromObject(model).toObject(OperationInner.class); - Assertions.assertEquals("odfvuefywsbp", model.name()); - Assertions.assertEquals(false, model.isDataAction()); - Assertions.assertEquals("hrfou", model.display().provider()); - Assertions.assertEquals("taakc", model.display().resource()); - Assertions.assertEquals("iyzvqtmnub", model.display().operation()); - Assertions.assertEquals("kpzksmondjmq", model.display().description()); + Assertions.assertEquals("bqdxbx", model.name()); + Assertions.assertEquals(true, model.isDataAction()); + Assertions.assertEquals("gqxndlkzgxhuripl", model.display().provider()); + Assertions.assertEquals("odxun", model.display().resource()); + Assertions.assertEquals("ebxmubyynt", model.display().operation()); + Assertions.assertEquals("rbqtkoie", model.display().description()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListMockTests.java index e6b5cabaea11c..a54615584d27a 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListMockTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListMockTests.java @@ -31,42 +31,31 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"ivfxzsjabibsyst\",\"isDataAction\":true,\"display\":{\"provider\":\"jpvkvpbjxbkzbzkd\",\"resource\":\"cjabudurgkakmo\",\"operation\":\"hjjklff\",\"description\":\"ouw\"}}]}"; + String responseStr + = "{\"value\":[{\"name\":\"gkfpaga\",\"isDataAction\":true,\"display\":{\"provider\":\"pqblylsyxkqjnsj\",\"resource\":\"vti\",\"operation\":\"xsdszuempsb\",\"description\":\"f\"}}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureQuantumManager manager = AzureQuantumManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); - Assertions.assertEquals("ivfxzsjabibsyst", response.iterator().next().name()); + Assertions.assertEquals("gkfpaga", response.iterator().next().name()); Assertions.assertEquals(true, response.iterator().next().isDataAction()); - Assertions.assertEquals("jpvkvpbjxbkzbzkd", response.iterator().next().display().provider()); - Assertions.assertEquals("cjabudurgkakmo", response.iterator().next().display().resource()); - Assertions.assertEquals("hjjklff", response.iterator().next().display().operation()); - Assertions.assertEquals("ouw", response.iterator().next().display().description()); + Assertions.assertEquals("pqblylsyxkqjnsj", response.iterator().next().display().provider()); + Assertions.assertEquals("vti", response.iterator().next().display().resource()); + Assertions.assertEquals("xsdszuempsb", response.iterator().next().display().operation()); + Assertions.assertEquals("f", response.iterator().next().display().description()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListTests.java index fa22cb677cc07..f18faa9a682ac 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/OperationsListTests.java @@ -14,53 +14,30 @@ public final class OperationsListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationsList model = - BinaryData - .fromString( - "{\"nextLink\":\"wkfvhqcrailvp\",\"value\":[{\"name\":\"fuflrwdmhdlx\",\"isDataAction\":true,\"display\":{\"provider\":\"agafcnihgwqap\",\"resource\":\"dgfbcvkcvqvpke\",\"operation\":\"cvdrhvoodsot\",\"description\":\"bzdopcj\"}},{\"name\":\"nhdldwmgxcx\",\"isDataAction\":true,\"display\":{\"provider\":\"utwu\",\"resource\":\"grpkhjwniyqs\",\"operation\":\"i\",\"description\":\"dggkzzlvmbmpa\"}}]}") - .toObject(OperationsList.class); - Assertions.assertEquals("wkfvhqcrailvp", model.nextLink()); - Assertions.assertEquals("fuflrwdmhdlx", model.value().get(0).name()); + OperationsList model = BinaryData.fromString( + "{\"nextLink\":\"gaokonzmnsikv\",\"value\":[{\"name\":\"ze\",\"isDataAction\":true,\"display\":{\"provider\":\"tfz\",\"resource\":\"hhvh\",\"operation\":\"r\",\"description\":\"dkwobdagx\"}}]}") + .toObject(OperationsList.class); + Assertions.assertEquals("gaokonzmnsikv", model.nextLink()); + Assertions.assertEquals("ze", model.value().get(0).name()); Assertions.assertEquals(true, model.value().get(0).isDataAction()); - Assertions.assertEquals("agafcnihgwqap", model.value().get(0).display().provider()); - Assertions.assertEquals("dgfbcvkcvqvpke", model.value().get(0).display().resource()); - Assertions.assertEquals("cvdrhvoodsot", model.value().get(0).display().operation()); - Assertions.assertEquals("bzdopcj", model.value().get(0).display().description()); + Assertions.assertEquals("tfz", model.value().get(0).display().provider()); + Assertions.assertEquals("hhvh", model.value().get(0).display().resource()); + Assertions.assertEquals("r", model.value().get(0).display().operation()); + Assertions.assertEquals("dkwobdagx", model.value().get(0).display().description()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationsList model = - new OperationsList() - .withNextLink("wkfvhqcrailvp") - .withValue( - Arrays - .asList( - new OperationInner() - .withName("fuflrwdmhdlx") - .withIsDataAction(true) - .withDisplay( - new OperationDisplay() - .withProvider("agafcnihgwqap") - .withResource("dgfbcvkcvqvpke") - .withOperation("cvdrhvoodsot") - .withDescription("bzdopcj")), - new OperationInner() - .withName("nhdldwmgxcx") - .withIsDataAction(true) - .withDisplay( - new OperationDisplay() - .withProvider("utwu") - .withResource("grpkhjwniyqs") - .withOperation("i") - .withDescription("dggkzzlvmbmpa")))); + OperationsList model = new OperationsList().withNextLink("gaokonzmnsikv").withValue( + Arrays.asList(new OperationInner().withName("ze").withIsDataAction(true).withDisplay(new OperationDisplay() + .withProvider("tfz").withResource("hhvh").withOperation("r").withDescription("dkwobdagx")))); model = BinaryData.fromObject(model).toObject(OperationsList.class); - Assertions.assertEquals("wkfvhqcrailvp", model.nextLink()); - Assertions.assertEquals("fuflrwdmhdlx", model.value().get(0).name()); + Assertions.assertEquals("gaokonzmnsikv", model.nextLink()); + Assertions.assertEquals("ze", model.value().get(0).name()); Assertions.assertEquals(true, model.value().get(0).isDataAction()); - Assertions.assertEquals("agafcnihgwqap", model.value().get(0).display().provider()); - Assertions.assertEquals("dgfbcvkcvqvpke", model.value().get(0).display().resource()); - Assertions.assertEquals("cvdrhvoodsot", model.value().get(0).display().operation()); - Assertions.assertEquals("bzdopcj", model.value().get(0).display().description()); + Assertions.assertEquals("tfz", model.value().get(0).display().provider()); + Assertions.assertEquals("hhvh", model.value().get(0).display().resource()); + Assertions.assertEquals("r", model.value().get(0).display().operation()); + Assertions.assertEquals("dkwobdagx", model.value().get(0).display().description()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDetailTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDetailTests.java index 068c64b88cb1d..f661174e0f5f9 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDetailTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDetailTests.java @@ -11,16 +11,17 @@ public final class PricingDetailTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PricingDetail model = BinaryData.fromString("{\"id\":\"c\",\"value\":\"fwdsj\"}").toObject(PricingDetail.class); - Assertions.assertEquals("c", model.id()); - Assertions.assertEquals("fwdsj", model.value()); + PricingDetail model + = BinaryData.fromString("{\"id\":\"wab\",\"value\":\"tshhszhedp\"}").toObject(PricingDetail.class); + Assertions.assertEquals("wab", model.id()); + Assertions.assertEquals("tshhszhedp", model.value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PricingDetail model = new PricingDetail().withId("c").withValue("fwdsj"); + PricingDetail model = new PricingDetail().withId("wab").withValue("tshhszhedp"); model = BinaryData.fromObject(model).toObject(PricingDetail.class); - Assertions.assertEquals("c", model.id()); - Assertions.assertEquals("fwdsj", model.value()); + Assertions.assertEquals("wab", model.id()); + Assertions.assertEquals("tshhszhedp", model.value()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDimensionTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDimensionTests.java index 6a3ebd4981777..f52f4fd6c08f4 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDimensionTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/PricingDimensionTests.java @@ -11,17 +11,17 @@ public final class PricingDimensionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PricingDimension model = - BinaryData.fromString("{\"id\":\"aljutiiswac\",\"name\":\"gdkz\"}").toObject(PricingDimension.class); - Assertions.assertEquals("aljutiiswac", model.id()); - Assertions.assertEquals("gdkz", model.name()); + PricingDimension model = BinaryData.fromString("{\"id\":\"wiwubm\",\"name\":\"besldnkwwtppjflc\"}") + .toObject(PricingDimension.class); + Assertions.assertEquals("wiwubm", model.id()); + Assertions.assertEquals("besldnkwwtppjflc", model.name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PricingDimension model = new PricingDimension().withId("aljutiiswac").withName("gdkz"); + PricingDimension model = new PricingDimension().withId("wiwubm").withName("besldnkwwtppjflc"); model = BinaryData.fromObject(model).toObject(PricingDimension.class); - Assertions.assertEquals("aljutiiswac", model.id()); - Assertions.assertEquals("gdkz", model.name()); + Assertions.assertEquals("wiwubm", model.id()); + Assertions.assertEquals("besldnkwwtppjflc", model.name()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderDescriptionInnerTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderDescriptionInnerTests.java index 758e2bae0aac2..d625dc658d9ad 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderDescriptionInnerTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderDescriptionInnerTests.java @@ -20,187 +20,175 @@ public final class ProviderDescriptionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProviderDescriptionInner model = - BinaryData - .fromString( - "{\"id\":\"rkcciwwzjuqk\",\"name\":\"sa\",\"properties\":{\"description\":\"ku\",\"providerType\":\"oskg\",\"company\":\"auu\",\"defaultEndpoint\":\"jmvxie\",\"aad\":{\"applicationId\":\"gidyjrrf\",\"tenantId\":\"aos\"},\"managedApplication\":{\"publisherId\":\"csonpclhoco\",\"offerId\":\"lkevle\"},\"targets\":[{\"id\":\"buhfmvfaxkffeiit\",\"name\":\"vmezy\",\"description\":\"hxmzsbbzoggig\",\"acceptedDataFormats\":[\"burvjxxjnspy\",\"ptkoenkoukn\",\"udwtiukbl\",\"ngkpocipazy\"],\"acceptedContentEncodings\":[\"gukgjnpiucgygevq\",\"ntypmrbpizcdrqj\",\"dpydn\"]},{\"id\":\"hxdeoejz\",\"name\":\"w\",\"description\":\"sjttgzfbish\",\"acceptedDataFormats\":[\"hajdeyeamdpha\"],\"acceptedContentEncodings\":[\"pbuxwgipwhon\",\"wkgshwa\",\"kix\",\"bin\"]}],\"skus\":[{\"id\":\"ttmrywnuzoqf\",\"name\":\"yqzrnkcqvyxlw\",\"version\":\"lsicohoqqnwv\",\"description\":\"yav\",\"restrictedAccessUri\":\"heun\",\"autoAdd\":true,\"targets\":[\"yxzk\",\"noc\"],\"quotaDimensions\":[{\"id\":\"lyaxuc\",\"scope\":\"uqszfk\",\"period\":\"ypewrmjmwvvjekt\",\"quota\":49.062435,\"name\":\"nhwlrsffrzpwvl\",\"description\":\"q\",\"unit\":\"iqylihkaetck\",\"unitPlural\":\"fcivfsnkym\"},{\"id\":\"tqhjfbebrjcx\",\"scope\":\"fuwutttxf\",\"period\":\"rbirphxe\",\"quota\":12.502766,\"name\":\"ahfn\",\"description\":\"kyqxjvuujqgidokg\",\"unit\":\"jyoxgvclt\",\"unitPlural\":\"sncghkjeszz\"},{\"id\":\"ijhtxf\",\"scope\":\"xbf\",\"period\":\"xnehmpvec\",\"quota\":79.49234,\"name\":\"ebfqkkrbm\",\"description\":\"kgriwflzlfbx\",\"unit\":\"uzycispnqza\",\"unitPlural\":\"gkbrpyyd\"}],\"pricingDetails\":[{\"id\":\"uqqkpik\",\"value\":\"rgvtqag\"},{\"id\":\"uynhijg\",\"value\":\"ebf\"},{\"id\":\"arbu\",\"value\":\"cvpnazzmhjrunmpx\"}]}],\"quotaDimensions\":[{\"id\":\"hrbnlankxmyskpbh\",\"scope\":\"btkcxywnytnrsyn\",\"period\":\"idybyxczf\",\"quota\":67.49647,\"name\":\"axdbabph\",\"description\":\"rqlfktsthsucocmn\",\"unit\":\"azt\",\"unitPlural\":\"twwrqp\"},{\"id\":\"dckzywbiexz\",\"scope\":\"yueaxibxujwb\",\"period\":\"walm\",\"quota\":22.792763,\"name\":\"xaepdkzjancuxr\",\"description\":\"wbavxbniwdj\",\"unit\":\"zt\",\"unitPlural\":\"bpg\"},{\"id\":\"ytxhp\",\"scope\":\"bzpfzab\",\"period\":\"cuh\",\"quota\":24.924797,\"name\":\"tyq\",\"description\":\"lbbovplw\",\"unit\":\"hvgyuguosvmk\",\"unitPlural\":\"sxqu\"}],\"pricingDimensions\":[{\"id\":\"l\",\"name\":\"gsxnkjzkdeslpv\"}]}}") - .toObject(ProviderDescriptionInner.class); - Assertions.assertEquals("rkcciwwzjuqk", model.id()); - Assertions.assertEquals("buhfmvfaxkffeiit", model.properties().targets().get(0).id()); - Assertions.assertEquals("vmezy", model.properties().targets().get(0).name()); - Assertions.assertEquals("hxmzsbbzoggig", model.properties().targets().get(0).description()); - Assertions.assertEquals("burvjxxjnspy", model.properties().targets().get(0).acceptedDataFormats().get(0)); - Assertions - .assertEquals("gukgjnpiucgygevq", model.properties().targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("ttmrywnuzoqf", model.properties().skus().get(0).id()); - Assertions.assertEquals("yqzrnkcqvyxlw", model.properties().skus().get(0).name()); - Assertions.assertEquals("lsicohoqqnwv", model.properties().skus().get(0).version()); - Assertions.assertEquals("yav", model.properties().skus().get(0).description()); - Assertions.assertEquals("heun", model.properties().skus().get(0).restrictedAccessUri()); - Assertions.assertEquals(true, model.properties().skus().get(0).autoAdd()); - Assertions.assertEquals("yxzk", model.properties().skus().get(0).targets().get(0)); - Assertions.assertEquals("lyaxuc", model.properties().skus().get(0).quotaDimensions().get(0).id()); - Assertions.assertEquals("uqszfk", model.properties().skus().get(0).quotaDimensions().get(0).scope()); - Assertions.assertEquals("ypewrmjmwvvjekt", model.properties().skus().get(0).quotaDimensions().get(0).period()); - Assertions.assertEquals(49.062435F, model.properties().skus().get(0).quotaDimensions().get(0).quota()); - Assertions.assertEquals("nhwlrsffrzpwvl", model.properties().skus().get(0).quotaDimensions().get(0).name()); - Assertions.assertEquals("q", model.properties().skus().get(0).quotaDimensions().get(0).description()); - Assertions.assertEquals("iqylihkaetck", model.properties().skus().get(0).quotaDimensions().get(0).unit()); - Assertions.assertEquals("fcivfsnkym", model.properties().skus().get(0).quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("uqqkpik", model.properties().skus().get(0).pricingDetails().get(0).id()); - Assertions.assertEquals("rgvtqag", model.properties().skus().get(0).pricingDetails().get(0).value()); - Assertions.assertEquals("hrbnlankxmyskpbh", model.properties().quotaDimensions().get(0).id()); - Assertions.assertEquals("btkcxywnytnrsyn", model.properties().quotaDimensions().get(0).scope()); - Assertions.assertEquals("idybyxczf", model.properties().quotaDimensions().get(0).period()); - Assertions.assertEquals(67.49647F, model.properties().quotaDimensions().get(0).quota()); - Assertions.assertEquals("axdbabph", model.properties().quotaDimensions().get(0).name()); - Assertions.assertEquals("rqlfktsthsucocmn", model.properties().quotaDimensions().get(0).description()); - Assertions.assertEquals("azt", model.properties().quotaDimensions().get(0).unit()); - Assertions.assertEquals("twwrqp", model.properties().quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("l", model.properties().pricingDimensions().get(0).id()); - Assertions.assertEquals("gsxnkjzkdeslpv", model.properties().pricingDimensions().get(0).name()); + ProviderDescriptionInner model = BinaryData.fromString( + "{\"id\":\"pj\",\"name\":\"asxazjpqyegualhb\",\"properties\":{\"description\":\"e\",\"providerType\":\"zzvdudgwds\",\"company\":\"hotwmcynpwlbjnp\",\"defaultEndpoint\":\"cftadeh\",\"aad\":{\"applicationId\":\"tyfsoppusuesn\",\"tenantId\":\"dejbavo\"},\"managedApplication\":{\"publisherId\":\"dmoh\",\"offerId\":\"bqvudwxdndn\"},\"targets\":[{\"id\":\"ujjugwdkcglh\",\"name\":\"azjdyggd\",\"description\":\"ixhbkuofqweykhm\",\"acceptedDataFormats\":[\"vfyexfw\",\"ybcibvyvdcsit\",\"nnaamdectehfiqsc\",\"eypvhezrkg\"],\"acceptedContentEncodings\":[\"jrefovgmkqsle\"]}],\"skus\":[{\"id\":\"yqjpkcatt\",\"name\":\"gjcrcczsqpjhvm\",\"version\":\"jvnysounqe\",\"description\":\"noae\",\"restrictedAccessUri\":\"fhyhltrpmopjmcma\",\"autoAdd\":false,\"targets\":[\"hfuiuaodsfc\",\"kvxod\",\"uozmyzydagfua\",\"bezy\"],\"quotaDimensions\":[{\"id\":\"ktwh\",\"scope\":\"xw\",\"period\":\"wqsmbsur\",\"quota\":85.49606,\"name\":\"o\",\"description\":\"ocfs\",\"unit\":\"s\",\"unitPlural\":\"ddystkiiuxhqy\"},{\"id\":\"xorrqnb\",\"scope\":\"czvyifq\",\"period\":\"kdvjsll\",\"quota\":80.76506,\"name\":\"d\",\"description\":\"atkpnp\",\"unit\":\"exxbczwtr\",\"unitPlural\":\"iqzbq\"},{\"id\":\"sovmyokacspkwl\",\"scope\":\"dobpxjmflbvvn\",\"period\":\"rkcciwwzjuqk\",\"quota\":7.702547,\"name\":\"jiwkuofoskghsau\",\"description\":\"mjmvxieduugidyjr\",\"unit\":\"byao\",\"unitPlural\":\"e\"}],\"pricingDetails\":[{\"id\":\"npc\",\"value\":\"ocohslkevleg\"},{\"id\":\"fbuhfmvfaxkffe\",\"value\":\"th\"},{\"id\":\"m\",\"value\":\"yvshxmz\"}]},{\"id\":\"bzoggigrx\",\"name\":\"ur\",\"version\":\"xxjnspydptk\",\"description\":\"nkoukn\",\"restrictedAccessUri\":\"dwtiukbldngkp\",\"autoAdd\":true,\"targets\":[\"z\",\"xoegukgjnpiucgy\",\"evqzntypmrbp\"],\"quotaDimensions\":[{\"id\":\"r\",\"scope\":\"sdpydnfyhxdeoejz\",\"period\":\"w\",\"quota\":67.792435,\"name\":\"ttgzfbis\",\"description\":\"bkh\",\"unit\":\"deyeamdphagalpbu\",\"unitPlural\":\"gipwhonowkg\"}],\"pricingDetails\":[{\"id\":\"nkixzbinj\",\"value\":\"uttmrywnuzoqft\"},{\"id\":\"qzrnkcqvyxlwhz\",\"value\":\"icohoqqnwvl\"},{\"id\":\"avwhheunm\",\"value\":\"hgyxzkonoc\"}]},{\"id\":\"oklyaxuconuq\",\"name\":\"fkbey\",\"version\":\"wrmjmwvvjektc\",\"description\":\"enhwlrs\",\"restrictedAccessUri\":\"rzpwvlqdqgbiq\",\"autoAdd\":true,\"targets\":[\"aetcktvfcivfs\"],\"quotaDimensions\":[{\"id\":\"uctqhjfbe\",\"scope\":\"jcxerfuwu\",\"period\":\"txfvjrbirph\",\"quota\":84.526054,\"name\":\"yva\",\"description\":\"nljky\",\"unit\":\"j\",\"unitPlural\":\"ujqgidok\"},{\"id\":\"ljyoxgvcltb\",\"scope\":\"ncghkje\",\"period\":\"zhbijhtxfv\",\"quota\":15.551442,\"name\":\"smx\",\"description\":\"hmpvecx\",\"unit\":\"debfqkkrbmpukgri\",\"unitPlural\":\"lzlfbxzpuz\"},{\"id\":\"ispnqzahmgkbrp\",\"scope\":\"dhibnuq\",\"period\":\"pikad\",\"quota\":14.613795,\"name\":\"qagnbuyn\",\"description\":\"jggmebfsiarbu\",\"unit\":\"cvpnazzmhjrunmpx\",\"unitPlural\":\"dbhrbnlankxm\"},{\"id\":\"k\",\"scope\":\"henbtkcxywnytn\",\"period\":\"yn\",\"quota\":52.88253,\"name\":\"ybyxc\",\"description\":\"clha\",\"unit\":\"dbabp\",\"unitPlural\":\"wrqlfktsthsuco\"}],\"pricingDetails\":[{\"id\":\"yazttbtwwrqpue\",\"value\":\"kzywbiex\"},{\"id\":\"eyueaxibxujwb\",\"value\":\"walm\"},{\"id\":\"yoxa\",\"value\":\"dkzjancuxrh\"},{\"id\":\"bavxbniwdjswzt\",\"value\":\"bpg\"}]},{\"id\":\"ytxhp\",\"name\":\"bzpfzab\",\"version\":\"cuh\",\"description\":\"tcty\",\"restrictedAccessUri\":\"klbb\",\"autoAdd\":true,\"targets\":[\"zbhvgyuguosv\",\"kfssxqukkf\",\"l\"],\"quotaDimensions\":[{\"id\":\"xnkjzkdesl\",\"scope\":\"lopwiyig\",\"period\":\"pkdwzbai\",\"quota\":69.34189,\"name\":\"a\",\"description\":\"nyqupedeojnabck\",\"unit\":\"mtxpsiebtfh\",\"unitPlural\":\"esap\"},{\"id\":\"rdqmhjjdhtldwkyz\",\"scope\":\"utknc\",\"period\":\"cwsvlxotog\",\"quota\":71.378235,\"name\":\"pqsxvnmicy\",\"description\":\"ceoveilovno\",\"unit\":\"fj\",\"unitPlural\":\"njbkcnxdhbttkph\"},{\"id\":\"pnvjtoqnermclf\",\"scope\":\"phoxus\",\"period\":\"pabgyeps\",\"quota\":95.976326,\"name\":\"zq\",\"description\":\"xywpmueefjzwfqkq\",\"unit\":\"ids\",\"unitPlural\":\"onobglaocqx\"},{\"id\":\"cmgyud\",\"scope\":\"tlmoyrx\",\"period\":\"fudwpznt\",\"quota\":32.9984,\"name\":\"hl\",\"description\":\"jbhckfrlhr\",\"unit\":\"bkyvp\",\"unitPlural\":\"anuzbpzkafkuw\"}],\"pricingDetails\":[{\"id\":\"wbme\",\"value\":\"seyvj\"}]}],\"quotaDimensions\":[{\"id\":\"slhs\",\"scope\":\"deemao\",\"period\":\"xagkvtmelmqkrh\",\"quota\":25.794971,\"name\":\"juahaquhcdhmdual\",\"description\":\"xqpvfadmw\",\"unit\":\"crgvxpvgom\",\"unitPlural\":\"fmisg\"}],\"pricingDimensions\":[{\"id\":\"beldawkzbaliourq\",\"name\":\"kauhashsfwxo\"},{\"id\":\"w\",\"name\":\"cugicjoox\"}]}}") + .toObject(ProviderDescriptionInner.class); + Assertions.assertEquals("pj", model.id()); + Assertions.assertEquals("ujjugwdkcglh", model.properties().targets().get(0).id()); + Assertions.assertEquals("azjdyggd", model.properties().targets().get(0).name()); + Assertions.assertEquals("ixhbkuofqweykhm", model.properties().targets().get(0).description()); + Assertions.assertEquals("vfyexfw", model.properties().targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("jrefovgmkqsle", model.properties().targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("yqjpkcatt", model.properties().skus().get(0).id()); + Assertions.assertEquals("gjcrcczsqpjhvm", model.properties().skus().get(0).name()); + Assertions.assertEquals("jvnysounqe", model.properties().skus().get(0).version()); + Assertions.assertEquals("noae", model.properties().skus().get(0).description()); + Assertions.assertEquals("fhyhltrpmopjmcma", model.properties().skus().get(0).restrictedAccessUri()); + Assertions.assertEquals(false, model.properties().skus().get(0).autoAdd()); + Assertions.assertEquals("hfuiuaodsfc", model.properties().skus().get(0).targets().get(0)); + Assertions.assertEquals("ktwh", model.properties().skus().get(0).quotaDimensions().get(0).id()); + Assertions.assertEquals("xw", model.properties().skus().get(0).quotaDimensions().get(0).scope()); + Assertions.assertEquals("wqsmbsur", model.properties().skus().get(0).quotaDimensions().get(0).period()); + Assertions.assertEquals(85.49606F, model.properties().skus().get(0).quotaDimensions().get(0).quota()); + Assertions.assertEquals("o", model.properties().skus().get(0).quotaDimensions().get(0).name()); + Assertions.assertEquals("ocfs", model.properties().skus().get(0).quotaDimensions().get(0).description()); + Assertions.assertEquals("s", model.properties().skus().get(0).quotaDimensions().get(0).unit()); + Assertions.assertEquals("ddystkiiuxhqy", + model.properties().skus().get(0).quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("npc", model.properties().skus().get(0).pricingDetails().get(0).id()); + Assertions.assertEquals("ocohslkevleg", model.properties().skus().get(0).pricingDetails().get(0).value()); + Assertions.assertEquals("slhs", model.properties().quotaDimensions().get(0).id()); + Assertions.assertEquals("deemao", model.properties().quotaDimensions().get(0).scope()); + Assertions.assertEquals("xagkvtmelmqkrh", model.properties().quotaDimensions().get(0).period()); + Assertions.assertEquals(25.794971F, model.properties().quotaDimensions().get(0).quota()); + Assertions.assertEquals("juahaquhcdhmdual", model.properties().quotaDimensions().get(0).name()); + Assertions.assertEquals("xqpvfadmw", model.properties().quotaDimensions().get(0).description()); + Assertions.assertEquals("crgvxpvgom", model.properties().quotaDimensions().get(0).unit()); + Assertions.assertEquals("fmisg", model.properties().quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("beldawkzbaliourq", model.properties().pricingDimensions().get(0).id()); + Assertions.assertEquals("kauhashsfwxo", model.properties().pricingDimensions().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProviderDescriptionInner model = - new ProviderDescriptionInner() - .withId("rkcciwwzjuqk") + ProviderDescriptionInner model + = new ProviderDescriptionInner().withId("pj") .withProperties( new ProviderProperties() - .withAad(new ProviderPropertiesAad()) + .withAad( + new ProviderPropertiesAad()) .withManagedApplication(new ProviderPropertiesManagedApplication()) - .withTargets( - Arrays - .asList( - new TargetDescription() - .withId("buhfmvfaxkffeiit") - .withName("vmezy") - .withDescription("hxmzsbbzoggig") - .withAcceptedDataFormats( - Arrays.asList("burvjxxjnspy", "ptkoenkoukn", "udwtiukbl", "ngkpocipazy")) - .withAcceptedContentEncodings( - Arrays.asList("gukgjnpiucgygevq", "ntypmrbpizcdrqj", "dpydn")), - new TargetDescription() - .withId("hxdeoejz") - .withName("w") - .withDescription("sjttgzfbish") - .withAcceptedDataFormats(Arrays.asList("hajdeyeamdpha")) - .withAcceptedContentEncodings( - Arrays.asList("pbuxwgipwhon", "wkgshwa", "kix", "bin")))) - .withSkus( - Arrays - .asList( - new SkuDescription() - .withId("ttmrywnuzoqf") - .withName("yqzrnkcqvyxlw") - .withVersion("lsicohoqqnwv") - .withDescription("yav") - .withRestrictedAccessUri("heun") - .withAutoAdd(true) - .withTargets(Arrays.asList("yxzk", "noc")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("lyaxuc") - .withScope("uqszfk") - .withPeriod("ypewrmjmwvvjekt") - .withQuota(49.062435F) - .withName("nhwlrsffrzpwvl") - .withDescription("q") - .withUnit("iqylihkaetck") - .withUnitPlural("fcivfsnkym"), - new QuotaDimension() - .withId("tqhjfbebrjcx") - .withScope("fuwutttxf") - .withPeriod("rbirphxe") - .withQuota(12.502766F) - .withName("ahfn") - .withDescription("kyqxjvuujqgidokg") - .withUnit("jyoxgvclt") - .withUnitPlural("sncghkjeszz"), - new QuotaDimension() - .withId("ijhtxf") - .withScope("xbf") - .withPeriod("xnehmpvec") - .withQuota(79.49234F) - .withName("ebfqkkrbm") - .withDescription("kgriwflzlfbx") - .withUnit("uzycispnqza") - .withUnitPlural("gkbrpyyd"))) - .withPricingDetails( - Arrays - .asList( - new PricingDetail().withId("uqqkpik").withValue("rgvtqag"), - new PricingDetail().withId("uynhijg").withValue("ebf"), - new PricingDetail().withId("arbu").withValue("cvpnazzmhjrunmpx"))))) - .withQuotaDimensions( - Arrays - .asList( + .withTargets(Arrays.asList(new TargetDescription().withId("ujjugwdkcglh").withName("azjdyggd") + .withDescription("ixhbkuofqweykhm") + .withAcceptedDataFormats( + Arrays.asList("vfyexfw", "ybcibvyvdcsit", "nnaamdectehfiqsc", "eypvhezrkg")) + .withAcceptedContentEncodings(Arrays.asList("jrefovgmkqsle")))) + .withSkus(Arrays.asList( + new SkuDescription() + .withId("yqjpkcatt").withName("gjcrcczsqpjhvm").withVersion("jvnysounqe") + .withDescription("noae").withRestrictedAccessUri("fhyhltrpmopjmcma").withAutoAdd(false) + .withTargets(Arrays.asList("hfuiuaodsfc", "kvxod", "uozmyzydagfua", "bezy")) + .withQuotaDimensions(Arrays + .asList( + new QuotaDimension().withId("ktwh").withScope("xw").withPeriod("wqsmbsur") + .withQuota(85.49606F).withName("o").withDescription("ocfs").withUnit("s") + .withUnitPlural("ddystkiiuxhqy"), + new QuotaDimension().withId("xorrqnb").withScope("czvyifq") + .withPeriod("kdvjsll").withQuota(80.76506F).withName("d") + .withDescription("atkpnp").withUnit("exxbczwtr").withUnitPlural("iqzbq"), + new QuotaDimension().withId("sovmyokacspkwl").withScope("dobpxjmflbvvn") + .withPeriod("rkcciwwzjuqk").withQuota(7.702547F).withName("jiwkuofoskghsau") + .withDescription("mjmvxieduugidyjr").withUnit("byao").withUnitPlural("e"))) + .withPricingDetails(Arrays.asList(new PricingDetail() + .withId("npc").withValue("ocohslkevleg"), + new PricingDetail().withId("fbuhfmvfaxkffe").withValue("th"), + new PricingDetail().withId("m").withValue("yvshxmz"))), + new SkuDescription().withId("bzoggigrx").withName("ur").withVersion("xxjnspydptk") + .withDescription("nkoukn").withRestrictedAccessUri("dwtiukbldngkp").withAutoAdd(true) + .withTargets(Arrays.asList("z", "xoegukgjnpiucgy", "evqzntypmrbp")) + .withQuotaDimensions(Arrays + .asList( + new QuotaDimension().withId("r").withScope("sdpydnfyhxdeoejz").withPeriod("w") + .withQuota(67.792435F).withName("ttgzfbis").withDescription("bkh") + .withUnit("deyeamdphagalpbu").withUnitPlural("gipwhonowkg"))) + .withPricingDetails(Arrays + .asList( + new PricingDetail().withId("nkixzbinj").withValue("uttmrywnuzoqft"), + new PricingDetail().withId("qzrnkcqvyxlwhz").withValue("icohoqqnwvl"), + new PricingDetail().withId("avwhheunm").withValue("hgyxzkonoc"))), + new SkuDescription().withId("oklyaxuconuq").withName("fkbey").withVersion("wrmjmwvvjektc") + .withDescription("enhwlrs").withRestrictedAccessUri("rzpwvlqdqgbiq").withAutoAdd(true) + .withTargets(Arrays.asList("aetcktvfcivfs")) + .withQuotaDimensions(Arrays + .asList(new QuotaDimension().withId("uctqhjfbe").withScope("jcxerfuwu") + .withPeriod("txfvjrbirph").withQuota(84.526054F).withName("yva") + .withDescription("nljky").withUnit("j").withUnitPlural("ujqgidok"), + new QuotaDimension().withId("ljyoxgvcltb").withScope("ncghkje") + .withPeriod("zhbijhtxfv").withQuota(15.551442F).withName("smx") + .withDescription("hmpvecx").withUnit("debfqkkrbmpukgri") + .withUnitPlural("lzlfbxzpuz"), + new QuotaDimension() + .withId("ispnqzahmgkbrp").withScope("dhibnuq").withPeriod("pikad") + .withQuota(14.613795F).withName("qagnbuyn").withDescription("jggmebfsiarbu") + .withUnit("cvpnazzmhjrunmpx").withUnitPlural("dbhrbnlankxm"), + new QuotaDimension().withId("k").withScope("henbtkcxywnytn").withPeriod("yn") + .withQuota(52.88253F).withName("ybyxc").withDescription("clha") + .withUnit("dbabp").withUnitPlural("wrqlfktsthsuco"))) + .withPricingDetails(Arrays + .asList( + new PricingDetail().withId("yazttbtwwrqpue").withValue("kzywbiex"), + new PricingDetail().withId("eyueaxibxujwb").withValue("walm"), + new PricingDetail().withId("yoxa").withValue("dkzjancuxrh"), + new PricingDetail().withId("bavxbniwdjswzt").withValue("bpg"))), + new SkuDescription().withId("ytxhp").withName("bzpfzab").withVersion("cuh") + .withDescription("tcty").withRestrictedAccessUri("klbb").withAutoAdd(true) + .withTargets(Arrays.asList("zbhvgyuguosv", "kfssxqukkf", "l")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension().withId("xnkjzkdesl") + .withScope("lopwiyig").withPeriod("pkdwzbai").withQuota(69.34189F).withName("a") + .withDescription("nyqupedeojnabck").withUnit("mtxpsiebtfh").withUnitPlural("esap"), new QuotaDimension() - .withId("hrbnlankxmyskpbh") - .withScope("btkcxywnytnrsyn") - .withPeriod("idybyxczf") - .withQuota(67.49647F) - .withName("axdbabph") - .withDescription("rqlfktsthsucocmn") - .withUnit("azt") - .withUnitPlural("twwrqp"), - new QuotaDimension() - .withId("dckzywbiexz") - .withScope("yueaxibxujwb") - .withPeriod("walm") - .withQuota(22.792763F) - .withName("xaepdkzjancuxr") - .withDescription("wbavxbniwdj") - .withUnit("zt") - .withUnitPlural("bpg"), - new QuotaDimension() - .withId("ytxhp") - .withScope("bzpfzab") - .withPeriod("cuh") - .withQuota(24.924797F) - .withName("tyq") - .withDescription("lbbovplw") - .withUnit("hvgyuguosvmk") - .withUnitPlural("sxqu"))) + .withId("rdqmhjjdhtldwkyz").withScope("utknc").withPeriod("cwsvlxotog") + .withQuota(71.378235F).withName("pqsxvnmicy").withDescription("ceoveilovno") + .withUnit("fj").withUnitPlural("njbkcnxdhbttkph"), + new QuotaDimension().withId("pnvjtoqnermclf").withScope("phoxus") + .withPeriod("pabgyeps").withQuota(95.976326F).withName("zq") + .withDescription("xywpmueefjzwfqkq").withUnit("ids") + .withUnitPlural("onobglaocqx"), + new QuotaDimension().withId("cmgyud").withScope("tlmoyrx").withPeriod("fudwpznt") + .withQuota(32.9984F).withName("hl").withDescription("jbhckfrlhr") + .withUnit("bkyvp").withUnitPlural("anuzbpzkafkuw"))) + .withPricingDetails( + Arrays.asList(new PricingDetail().withId("wbme").withValue("seyvj"))))) + .withQuotaDimensions(Arrays.asList(new QuotaDimension().withId("slhs").withScope("deemao") + .withPeriod("xagkvtmelmqkrh").withQuota(25.794971F).withName("juahaquhcdhmdual") + .withDescription("xqpvfadmw").withUnit("crgvxpvgom").withUnitPlural("fmisg"))) .withPricingDimensions( - Arrays.asList(new PricingDimension().withId("l").withName("gsxnkjzkdeslpv")))); + Arrays.asList(new PricingDimension().withId("beldawkzbaliourq").withName("kauhashsfwxo"), + new PricingDimension().withId("w").withName("cugicjoox")))); model = BinaryData.fromObject(model).toObject(ProviderDescriptionInner.class); - Assertions.assertEquals("rkcciwwzjuqk", model.id()); - Assertions.assertEquals("buhfmvfaxkffeiit", model.properties().targets().get(0).id()); - Assertions.assertEquals("vmezy", model.properties().targets().get(0).name()); - Assertions.assertEquals("hxmzsbbzoggig", model.properties().targets().get(0).description()); - Assertions.assertEquals("burvjxxjnspy", model.properties().targets().get(0).acceptedDataFormats().get(0)); - Assertions - .assertEquals("gukgjnpiucgygevq", model.properties().targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("ttmrywnuzoqf", model.properties().skus().get(0).id()); - Assertions.assertEquals("yqzrnkcqvyxlw", model.properties().skus().get(0).name()); - Assertions.assertEquals("lsicohoqqnwv", model.properties().skus().get(0).version()); - Assertions.assertEquals("yav", model.properties().skus().get(0).description()); - Assertions.assertEquals("heun", model.properties().skus().get(0).restrictedAccessUri()); - Assertions.assertEquals(true, model.properties().skus().get(0).autoAdd()); - Assertions.assertEquals("yxzk", model.properties().skus().get(0).targets().get(0)); - Assertions.assertEquals("lyaxuc", model.properties().skus().get(0).quotaDimensions().get(0).id()); - Assertions.assertEquals("uqszfk", model.properties().skus().get(0).quotaDimensions().get(0).scope()); - Assertions.assertEquals("ypewrmjmwvvjekt", model.properties().skus().get(0).quotaDimensions().get(0).period()); - Assertions.assertEquals(49.062435F, model.properties().skus().get(0).quotaDimensions().get(0).quota()); - Assertions.assertEquals("nhwlrsffrzpwvl", model.properties().skus().get(0).quotaDimensions().get(0).name()); - Assertions.assertEquals("q", model.properties().skus().get(0).quotaDimensions().get(0).description()); - Assertions.assertEquals("iqylihkaetck", model.properties().skus().get(0).quotaDimensions().get(0).unit()); - Assertions.assertEquals("fcivfsnkym", model.properties().skus().get(0).quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("uqqkpik", model.properties().skus().get(0).pricingDetails().get(0).id()); - Assertions.assertEquals("rgvtqag", model.properties().skus().get(0).pricingDetails().get(0).value()); - Assertions.assertEquals("hrbnlankxmyskpbh", model.properties().quotaDimensions().get(0).id()); - Assertions.assertEquals("btkcxywnytnrsyn", model.properties().quotaDimensions().get(0).scope()); - Assertions.assertEquals("idybyxczf", model.properties().quotaDimensions().get(0).period()); - Assertions.assertEquals(67.49647F, model.properties().quotaDimensions().get(0).quota()); - Assertions.assertEquals("axdbabph", model.properties().quotaDimensions().get(0).name()); - Assertions.assertEquals("rqlfktsthsucocmn", model.properties().quotaDimensions().get(0).description()); - Assertions.assertEquals("azt", model.properties().quotaDimensions().get(0).unit()); - Assertions.assertEquals("twwrqp", model.properties().quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("l", model.properties().pricingDimensions().get(0).id()); - Assertions.assertEquals("gsxnkjzkdeslpv", model.properties().pricingDimensions().get(0).name()); + Assertions.assertEquals("pj", model.id()); + Assertions.assertEquals("ujjugwdkcglh", model.properties().targets().get(0).id()); + Assertions.assertEquals("azjdyggd", model.properties().targets().get(0).name()); + Assertions.assertEquals("ixhbkuofqweykhm", model.properties().targets().get(0).description()); + Assertions.assertEquals("vfyexfw", model.properties().targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("jrefovgmkqsle", model.properties().targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("yqjpkcatt", model.properties().skus().get(0).id()); + Assertions.assertEquals("gjcrcczsqpjhvm", model.properties().skus().get(0).name()); + Assertions.assertEquals("jvnysounqe", model.properties().skus().get(0).version()); + Assertions.assertEquals("noae", model.properties().skus().get(0).description()); + Assertions.assertEquals("fhyhltrpmopjmcma", model.properties().skus().get(0).restrictedAccessUri()); + Assertions.assertEquals(false, model.properties().skus().get(0).autoAdd()); + Assertions.assertEquals("hfuiuaodsfc", model.properties().skus().get(0).targets().get(0)); + Assertions.assertEquals("ktwh", model.properties().skus().get(0).quotaDimensions().get(0).id()); + Assertions.assertEquals("xw", model.properties().skus().get(0).quotaDimensions().get(0).scope()); + Assertions.assertEquals("wqsmbsur", model.properties().skus().get(0).quotaDimensions().get(0).period()); + Assertions.assertEquals(85.49606F, model.properties().skus().get(0).quotaDimensions().get(0).quota()); + Assertions.assertEquals("o", model.properties().skus().get(0).quotaDimensions().get(0).name()); + Assertions.assertEquals("ocfs", model.properties().skus().get(0).quotaDimensions().get(0).description()); + Assertions.assertEquals("s", model.properties().skus().get(0).quotaDimensions().get(0).unit()); + Assertions.assertEquals("ddystkiiuxhqy", + model.properties().skus().get(0).quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("npc", model.properties().skus().get(0).pricingDetails().get(0).id()); + Assertions.assertEquals("ocohslkevleg", model.properties().skus().get(0).pricingDetails().get(0).value()); + Assertions.assertEquals("slhs", model.properties().quotaDimensions().get(0).id()); + Assertions.assertEquals("deemao", model.properties().quotaDimensions().get(0).scope()); + Assertions.assertEquals("xagkvtmelmqkrh", model.properties().quotaDimensions().get(0).period()); + Assertions.assertEquals(25.794971F, model.properties().quotaDimensions().get(0).quota()); + Assertions.assertEquals("juahaquhcdhmdual", model.properties().quotaDimensions().get(0).name()); + Assertions.assertEquals("xqpvfadmw", model.properties().quotaDimensions().get(0).description()); + Assertions.assertEquals("crgvxpvgom", model.properties().quotaDimensions().get(0).unit()); + Assertions.assertEquals("fmisg", model.properties().quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("beldawkzbaliourq", model.properties().pricingDimensions().get(0).id()); + Assertions.assertEquals("kauhashsfwxo", model.properties().pricingDimensions().get(0).name()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesAadTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesAadTests.java index 9d630a75dd85e..e4c662264c05d 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesAadTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesAadTests.java @@ -10,10 +10,8 @@ public final class ProviderPropertiesAadTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProviderPropertiesAad model = - BinaryData - .fromString("{\"applicationId\":\"qolbgyc\",\"tenantId\":\"iertgccymvaolp\"}") - .toObject(ProviderPropertiesAad.class); + ProviderPropertiesAad model = BinaryData.fromString("{\"applicationId\":\"z\",\"tenantId\":\"hzxct\"}") + .toObject(ProviderPropertiesAad.class); } @org.junit.jupiter.api.Test diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesManagedApplicationTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesManagedApplicationTests.java index 1568d26fdd1aa..1c3626843b445 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesManagedApplicationTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesManagedApplicationTests.java @@ -10,9 +10,8 @@ public final class ProviderPropertiesManagedApplicationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProviderPropertiesManagedApplication model = - BinaryData - .fromString("{\"publisherId\":\"lqlfm\",\"offerId\":\"n\"}") + ProviderPropertiesManagedApplication model + = BinaryData.fromString("{\"publisherId\":\"gbkdmoizpos\",\"offerId\":\"grcfb\"}") .toObject(ProviderPropertiesManagedApplication.class); } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesTests.java index 49c54f117c67f..603f64b453f2d 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderPropertiesTests.java @@ -19,223 +19,101 @@ public final class ProviderPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ProviderProperties model = - BinaryData - .fromString( - "{\"description\":\"pwiyig\",\"providerType\":\"pkdwzbai\",\"company\":\"bbaumnyquped\",\"defaultEndpoint\":\"jn\",\"aad\":{\"applicationId\":\"khsmtxpsiebt\",\"tenantId\":\"vpesapskrdqmhjjd\"},\"managedApplication\":{\"publisherId\":\"dwkyzxuu\",\"offerId\":\"ncwscwsvlxoto\"},\"targets\":[{\"id\":\"upqsx\",\"name\":\"micykvceoveilo\",\"description\":\"oty\",\"acceptedDataFormats\":[\"cnjbkcnxdhbt\"],\"acceptedContentEncodings\":[\"h\",\"wpn\"]},{\"id\":\"t\",\"name\":\"nermcl\",\"description\":\"lphox\",\"acceptedDataFormats\":[\"rpabg\",\"epsbjtazqu\"],\"acceptedContentEncodings\":[\"wpmueefj\",\"wfqkquj\",\"dsuyonobgla\",\"cq\"]},{\"id\":\"ccm\",\"name\":\"udxytlmoyrx\",\"description\":\"fudwpznt\",\"acceptedDataFormats\":[\"zhlrqjb\",\"ck\"],\"acceptedContentEncodings\":[\"hrxsbk\",\"vpycanuzbp\",\"kafkuwbcrnwbm\"]},{\"id\":\"hseyvju\",\"name\":\"tslhspkdeem\",\"description\":\"fm\",\"acceptedDataFormats\":[\"kv\",\"melmqkrha\",\"vljua\",\"aquhcdhm\"],\"acceptedContentEncodings\":[\"laexqp\",\"fadmws\",\"crgvxpvgom\",\"lf\"]}],\"skus\":[{\"id\":\"wbnb\",\"name\":\"ldawkzbaliourqha\",\"version\":\"uhashsfwx\",\"description\":\"owzxcu\",\"restrictedAccessUri\":\"cjooxdjebwpucwwf\",\"autoAdd\":false,\"targets\":[\"meue\",\"ivyhzceuojgjrwju\"],\"quotaDimensions\":[{\"id\":\"wmcdytdxwi\",\"scope\":\"nrjawgqwg\",\"period\":\"ni\",\"quota\":5.4993095,\"name\":\"bkpyc\",\"description\":\"lwn\",\"unit\":\"hjdauwhvylwz\",\"unitPlural\":\"dhxujznbmpo\"},{\"id\":\"wpr\",\"scope\":\"lve\",\"period\":\"lupj\",\"quota\":80.70111,\"name\":\"xobbcswsrt\",\"description\":\"iplrbpbewtghfgb\",\"unit\":\"gw\",\"unitPlural\":\"vlvqhjkbegi\"}],\"pricingDetails\":[{\"id\":\"xiebwwaloayqcg\",\"value\":\"tzjuzgwyzmhtxo\"},{\"id\":\"mtsavjcbpwxqp\",\"value\":\"knftguvriuh\"},{\"id\":\"wmdyvxqtay\",\"value\":\"ww\"},{\"id\":\"yqbexrmcqibycno\",\"value\":\"knme\"}]},{\"id\":\"sgzvahapjyzhpv\",\"name\":\"zcjrvxdjzlmwlx\",\"version\":\"ug\",\"description\":\"zovawjvz\",\"restrictedAccessUri\":\"luthn\",\"autoAdd\":false,\"targets\":[\"i\",\"eilpjzuaejxdu\",\"tskzbbtdzumveek\",\"pwo\"],\"quotaDimensions\":[{\"id\":\"fpbsjyofdxl\",\"scope\":\"sd\",\"period\":\"ouwaboekqvkeln\",\"quota\":63.19878,\"name\":\"xwyjsflhhc\",\"description\":\"lnjixisxya\",\"unit\":\"oyaqcslyjpkiid\",\"unitPlural\":\"exznelixhnr\"},{\"id\":\"folhbnxknal\",\"scope\":\"lp\",\"period\":\"gdtpnapnyiro\",\"quota\":27.694834,\"name\":\"igvpgylg\",\"description\":\"itxmedjvcslynqww\",\"unit\":\"wzz\",\"unitPlural\":\"gktrmgucnapkte\"}],\"pricingDetails\":[{\"id\":\"wptfdy\",\"value\":\"fqbuaceopzf\"},{\"id\":\"hhuao\",\"value\":\"pcqeqx\"}]},{\"id\":\"z\",\"name\":\"hzxct\",\"version\":\"gbkdmoizpos\",\"description\":\"grcfb\",\"restrictedAccessUri\":\"rmfqjhhkxbpvj\",\"autoAdd\":false,\"targets\":[\"xjyngudivk\",\"tswb\",\"qzvszjf\",\"uvjfdxxive\"],\"quotaDimensions\":[{\"id\":\"qaqtdoqmcbxvwvxy\",\"scope\":\"qbhsfxobl\",\"period\":\"k\",\"quota\":19.958847,\"name\":\"ewwwfbkrvrnsv\",\"description\":\"q\",\"unit\":\"hxcr\",\"unitPlural\":\"fovasr\"}],\"pricingDetails\":[{\"id\":\"bhsqfsubcgjbirxb\",\"value\":\"bsrfbj\"}]}],\"quotaDimensions\":[{\"id\":\"ssotftpv\",\"scope\":\"bexilzznfqqnv\",\"period\":\"mqtaruoujmkcjh\",\"quota\":83.08001,\"name\":\"jrybnwjewgdrjer\",\"description\":\"aen\",\"unit\":\"eh\",\"unitPlural\":\"doy\"}],\"pricingDimensions\":[{\"id\":\"thnzd\",\"name\":\"sl\"},{\"id\":\"ayqigynduhav\",\"name\":\"lkthu\"}]}") - .toObject(ProviderProperties.class); - Assertions.assertEquals("upqsx", model.targets().get(0).id()); - Assertions.assertEquals("micykvceoveilo", model.targets().get(0).name()); - Assertions.assertEquals("oty", model.targets().get(0).description()); - Assertions.assertEquals("cnjbkcnxdhbt", model.targets().get(0).acceptedDataFormats().get(0)); - Assertions.assertEquals("h", model.targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("wbnb", model.skus().get(0).id()); - Assertions.assertEquals("ldawkzbaliourqha", model.skus().get(0).name()); - Assertions.assertEquals("uhashsfwx", model.skus().get(0).version()); - Assertions.assertEquals("owzxcu", model.skus().get(0).description()); - Assertions.assertEquals("cjooxdjebwpucwwf", model.skus().get(0).restrictedAccessUri()); + ProviderProperties model = BinaryData.fromString( + "{\"description\":\"ebwpucwwfvo\",\"providerType\":\"vmeueci\",\"company\":\"hzceuojgjrwjue\",\"defaultEndpoint\":\"twm\",\"aad\":{\"applicationId\":\"tdx\",\"tenantId\":\"txnrjaw\"},\"managedApplication\":{\"publisherId\":\"gxhnisk\",\"offerId\":\"bkpyc\"},\"targets\":[{\"id\":\"ndnhj\",\"name\":\"uwhvylwzbtdhxujz\",\"description\":\"mpowuwpr\",\"acceptedDataFormats\":[\"veual\"],\"acceptedContentEncodings\":[\"mkh\"]}],\"skus\":[{\"id\":\"bcswsrt\",\"name\":\"iplrbpbewtghfgb\",\"version\":\"gw\",\"description\":\"vlvqhjkbegi\",\"restrictedAccessUri\":\"nmxiebwwaloayqc\",\"autoAdd\":false,\"targets\":[\"j\",\"zg\"],\"quotaDimensions\":[{\"id\":\"htxongmtsavjc\",\"scope\":\"wxqpsrknftguvri\",\"period\":\"prwmdyvxqt\",\"quota\":42.644672,\"name\":\"ww\",\"description\":\"yqbexrmcqibycno\",\"unit\":\"knme\",\"unitPlural\":\"sgzvahapjyzhpv\"}],\"pricingDetails\":[{\"id\":\"j\",\"value\":\"xdjzlmwlxk\"},{\"id\":\"gfhzovawjvzunlut\",\"value\":\"n\"},{\"id\":\"nxipeil\",\"value\":\"zuaejxd\"},{\"id\":\"tskzbbtdzumveek\",\"value\":\"wozuhkf\"}]}],\"quotaDimensions\":[{\"id\":\"yofd\",\"scope\":\"uusdttouwa\",\"period\":\"ekqvkeln\",\"quota\":63.19878,\"name\":\"xwyjsflhhc\",\"description\":\"lnjixisxya\",\"unit\":\"oyaqcslyjpkiid\",\"unitPlural\":\"exznelixhnr\"},{\"id\":\"folhbnxknal\",\"scope\":\"lp\",\"period\":\"gdtpnapnyiro\",\"quota\":27.694834,\"name\":\"igvpgylg\",\"description\":\"itxmedjvcslynqww\",\"unit\":\"wzz\",\"unitPlural\":\"gktrmgucnapkte\"}],\"pricingDimensions\":[{\"id\":\"wptfdy\",\"name\":\"fqbuaceopzf\"},{\"id\":\"hhuao\",\"name\":\"pcqeqx\"}]}") + .toObject(ProviderProperties.class); + Assertions.assertEquals("ndnhj", model.targets().get(0).id()); + Assertions.assertEquals("uwhvylwzbtdhxujz", model.targets().get(0).name()); + Assertions.assertEquals("mpowuwpr", model.targets().get(0).description()); + Assertions.assertEquals("veual", model.targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("mkh", model.targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("bcswsrt", model.skus().get(0).id()); + Assertions.assertEquals("iplrbpbewtghfgb", model.skus().get(0).name()); + Assertions.assertEquals("gw", model.skus().get(0).version()); + Assertions.assertEquals("vlvqhjkbegi", model.skus().get(0).description()); + Assertions.assertEquals("nmxiebwwaloayqc", model.skus().get(0).restrictedAccessUri()); Assertions.assertEquals(false, model.skus().get(0).autoAdd()); - Assertions.assertEquals("meue", model.skus().get(0).targets().get(0)); - Assertions.assertEquals("wmcdytdxwi", model.skus().get(0).quotaDimensions().get(0).id()); - Assertions.assertEquals("nrjawgqwg", model.skus().get(0).quotaDimensions().get(0).scope()); - Assertions.assertEquals("ni", model.skus().get(0).quotaDimensions().get(0).period()); - Assertions.assertEquals(5.4993095F, model.skus().get(0).quotaDimensions().get(0).quota()); - Assertions.assertEquals("bkpyc", model.skus().get(0).quotaDimensions().get(0).name()); - Assertions.assertEquals("lwn", model.skus().get(0).quotaDimensions().get(0).description()); - Assertions.assertEquals("hjdauwhvylwz", model.skus().get(0).quotaDimensions().get(0).unit()); - Assertions.assertEquals("dhxujznbmpo", model.skus().get(0).quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("xiebwwaloayqcg", model.skus().get(0).pricingDetails().get(0).id()); - Assertions.assertEquals("tzjuzgwyzmhtxo", model.skus().get(0).pricingDetails().get(0).value()); - Assertions.assertEquals("ssotftpv", model.quotaDimensions().get(0).id()); - Assertions.assertEquals("bexilzznfqqnv", model.quotaDimensions().get(0).scope()); - Assertions.assertEquals("mqtaruoujmkcjh", model.quotaDimensions().get(0).period()); - Assertions.assertEquals(83.08001F, model.quotaDimensions().get(0).quota()); - Assertions.assertEquals("jrybnwjewgdrjer", model.quotaDimensions().get(0).name()); - Assertions.assertEquals("aen", model.quotaDimensions().get(0).description()); - Assertions.assertEquals("eh", model.quotaDimensions().get(0).unit()); - Assertions.assertEquals("doy", model.quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("thnzd", model.pricingDimensions().get(0).id()); - Assertions.assertEquals("sl", model.pricingDimensions().get(0).name()); + Assertions.assertEquals("j", model.skus().get(0).targets().get(0)); + Assertions.assertEquals("htxongmtsavjc", model.skus().get(0).quotaDimensions().get(0).id()); + Assertions.assertEquals("wxqpsrknftguvri", model.skus().get(0).quotaDimensions().get(0).scope()); + Assertions.assertEquals("prwmdyvxqt", model.skus().get(0).quotaDimensions().get(0).period()); + Assertions.assertEquals(42.644672F, model.skus().get(0).quotaDimensions().get(0).quota()); + Assertions.assertEquals("ww", model.skus().get(0).quotaDimensions().get(0).name()); + Assertions.assertEquals("yqbexrmcqibycno", model.skus().get(0).quotaDimensions().get(0).description()); + Assertions.assertEquals("knme", model.skus().get(0).quotaDimensions().get(0).unit()); + Assertions.assertEquals("sgzvahapjyzhpv", model.skus().get(0).quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("j", model.skus().get(0).pricingDetails().get(0).id()); + Assertions.assertEquals("xdjzlmwlxk", model.skus().get(0).pricingDetails().get(0).value()); + Assertions.assertEquals("yofd", model.quotaDimensions().get(0).id()); + Assertions.assertEquals("uusdttouwa", model.quotaDimensions().get(0).scope()); + Assertions.assertEquals("ekqvkeln", model.quotaDimensions().get(0).period()); + Assertions.assertEquals(63.19878F, model.quotaDimensions().get(0).quota()); + Assertions.assertEquals("xwyjsflhhc", model.quotaDimensions().get(0).name()); + Assertions.assertEquals("lnjixisxya", model.quotaDimensions().get(0).description()); + Assertions.assertEquals("oyaqcslyjpkiid", model.quotaDimensions().get(0).unit()); + Assertions.assertEquals("exznelixhnr", model.quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("wptfdy", model.pricingDimensions().get(0).id()); + Assertions.assertEquals("fqbuaceopzf", model.pricingDimensions().get(0).name()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ProviderProperties model = - new ProviderProperties() - .withAad(new ProviderPropertiesAad()) - .withManagedApplication(new ProviderPropertiesManagedApplication()) - .withTargets( - Arrays - .asList( - new TargetDescription() - .withId("upqsx") - .withName("micykvceoveilo") - .withDescription("oty") - .withAcceptedDataFormats(Arrays.asList("cnjbkcnxdhbt")) - .withAcceptedContentEncodings(Arrays.asList("h", "wpn")), - new TargetDescription() - .withId("t") - .withName("nermcl") - .withDescription("lphox") - .withAcceptedDataFormats(Arrays.asList("rpabg", "epsbjtazqu")) - .withAcceptedContentEncodings( - Arrays.asList("wpmueefj", "wfqkquj", "dsuyonobgla", "cq")), - new TargetDescription() - .withId("ccm") - .withName("udxytlmoyrx") - .withDescription("fudwpznt") - .withAcceptedDataFormats(Arrays.asList("zhlrqjb", "ck")) - .withAcceptedContentEncodings(Arrays.asList("hrxsbk", "vpycanuzbp", "kafkuwbcrnwbm")), - new TargetDescription() - .withId("hseyvju") - .withName("tslhspkdeem") - .withDescription("fm") - .withAcceptedDataFormats(Arrays.asList("kv", "melmqkrha", "vljua", "aquhcdhm")) - .withAcceptedContentEncodings(Arrays.asList("laexqp", "fadmws", "crgvxpvgom", "lf")))) - .withSkus( - Arrays - .asList( - new SkuDescription() - .withId("wbnb") - .withName("ldawkzbaliourqha") - .withVersion("uhashsfwx") - .withDescription("owzxcu") - .withRestrictedAccessUri("cjooxdjebwpucwwf") - .withAutoAdd(false) - .withTargets(Arrays.asList("meue", "ivyhzceuojgjrwju")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("wmcdytdxwi") - .withScope("nrjawgqwg") - .withPeriod("ni") - .withQuota(5.4993095F) - .withName("bkpyc") - .withDescription("lwn") - .withUnit("hjdauwhvylwz") - .withUnitPlural("dhxujznbmpo"), - new QuotaDimension() - .withId("wpr") - .withScope("lve") - .withPeriod("lupj") - .withQuota(80.70111F) - .withName("xobbcswsrt") - .withDescription("iplrbpbewtghfgb") - .withUnit("gw") - .withUnitPlural("vlvqhjkbegi"))) - .withPricingDetails( - Arrays - .asList( - new PricingDetail().withId("xiebwwaloayqcg").withValue("tzjuzgwyzmhtxo"), - new PricingDetail().withId("mtsavjcbpwxqp").withValue("knftguvriuh"), - new PricingDetail().withId("wmdyvxqtay").withValue("ww"), - new PricingDetail().withId("yqbexrmcqibycno").withValue("knme"))), - new SkuDescription() - .withId("sgzvahapjyzhpv") - .withName("zcjrvxdjzlmwlx") - .withVersion("ug") - .withDescription("zovawjvz") - .withRestrictedAccessUri("luthn") - .withAutoAdd(false) - .withTargets(Arrays.asList("i", "eilpjzuaejxdu", "tskzbbtdzumveek", "pwo")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("fpbsjyofdxl") - .withScope("sd") - .withPeriod("ouwaboekqvkeln") - .withQuota(63.19878F) - .withName("xwyjsflhhc") - .withDescription("lnjixisxya") - .withUnit("oyaqcslyjpkiid") - .withUnitPlural("exznelixhnr"), - new QuotaDimension() - .withId("folhbnxknal") - .withScope("lp") - .withPeriod("gdtpnapnyiro") - .withQuota(27.694834F) - .withName("igvpgylg") - .withDescription("itxmedjvcslynqww") - .withUnit("wzz") - .withUnitPlural("gktrmgucnapkte"))) - .withPricingDetails( - Arrays - .asList( - new PricingDetail().withId("wptfdy").withValue("fqbuaceopzf"), - new PricingDetail().withId("hhuao").withValue("pcqeqx"))), - new SkuDescription() - .withId("z") - .withName("hzxct") - .withVersion("gbkdmoizpos") - .withDescription("grcfb") - .withRestrictedAccessUri("rmfqjhhkxbpvj") - .withAutoAdd(false) - .withTargets(Arrays.asList("xjyngudivk", "tswb", "qzvszjf", "uvjfdxxive")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("qaqtdoqmcbxvwvxy") - .withScope("qbhsfxobl") - .withPeriod("k") - .withQuota(19.958847F) - .withName("ewwwfbkrvrnsv") - .withDescription("q") - .withUnit("hxcr") - .withUnitPlural("fovasr"))) - .withPricingDetails( - Arrays.asList(new PricingDetail().withId("bhsqfsubcgjbirxb").withValue("bsrfbj"))))) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("ssotftpv") - .withScope("bexilzznfqqnv") - .withPeriod("mqtaruoujmkcjh") - .withQuota(83.08001F) - .withName("jrybnwjewgdrjer") - .withDescription("aen") - .withUnit("eh") - .withUnitPlural("doy"))) - .withPricingDimensions( - Arrays - .asList( - new PricingDimension().withId("thnzd").withName("sl"), - new PricingDimension().withId("ayqigynduhav").withName("lkthu"))); + ProviderProperties model = new ProviderProperties().withAad(new ProviderPropertiesAad()) + .withManagedApplication(new ProviderPropertiesManagedApplication()) + .withTargets(Arrays.asList(new TargetDescription().withId("ndnhj").withName("uwhvylwzbtdhxujz") + .withDescription("mpowuwpr").withAcceptedDataFormats(Arrays.asList("veual")) + .withAcceptedContentEncodings(Arrays.asList("mkh")))) + .withSkus(Arrays.asList(new SkuDescription().withId("bcswsrt").withName("iplrbpbewtghfgb").withVersion("gw") + .withDescription("vlvqhjkbegi").withRestrictedAccessUri("nmxiebwwaloayqc").withAutoAdd(false) + .withTargets(Arrays.asList("j", "zg")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension().withId("htxongmtsavjc") + .withScope("wxqpsrknftguvri").withPeriod("prwmdyvxqt").withQuota(42.644672F).withName("ww") + .withDescription("yqbexrmcqibycno").withUnit("knme").withUnitPlural("sgzvahapjyzhpv"))) + .withPricingDetails(Arrays.asList(new PricingDetail().withId("j").withValue("xdjzlmwlxk"), + new PricingDetail().withId("gfhzovawjvzunlut").withValue("n"), + new PricingDetail().withId("nxipeil").withValue("zuaejxd"), + new PricingDetail().withId("tskzbbtdzumveek").withValue("wozuhkf"))))) + .withQuotaDimensions(Arrays.asList( + new QuotaDimension().withId("yofd").withScope("uusdttouwa").withPeriod("ekqvkeln").withQuota(63.19878F) + .withName("xwyjsflhhc").withDescription("lnjixisxya").withUnit("oyaqcslyjpkiid") + .withUnitPlural("exznelixhnr"), + new QuotaDimension().withId("folhbnxknal").withScope("lp").withPeriod("gdtpnapnyiro") + .withQuota(27.694834F).withName("igvpgylg").withDescription("itxmedjvcslynqww").withUnit("wzz") + .withUnitPlural("gktrmgucnapkte"))) + .withPricingDimensions(Arrays.asList(new PricingDimension().withId("wptfdy").withName("fqbuaceopzf"), + new PricingDimension().withId("hhuao").withName("pcqeqx"))); model = BinaryData.fromObject(model).toObject(ProviderProperties.class); - Assertions.assertEquals("upqsx", model.targets().get(0).id()); - Assertions.assertEquals("micykvceoveilo", model.targets().get(0).name()); - Assertions.assertEquals("oty", model.targets().get(0).description()); - Assertions.assertEquals("cnjbkcnxdhbt", model.targets().get(0).acceptedDataFormats().get(0)); - Assertions.assertEquals("h", model.targets().get(0).acceptedContentEncodings().get(0)); - Assertions.assertEquals("wbnb", model.skus().get(0).id()); - Assertions.assertEquals("ldawkzbaliourqha", model.skus().get(0).name()); - Assertions.assertEquals("uhashsfwx", model.skus().get(0).version()); - Assertions.assertEquals("owzxcu", model.skus().get(0).description()); - Assertions.assertEquals("cjooxdjebwpucwwf", model.skus().get(0).restrictedAccessUri()); + Assertions.assertEquals("ndnhj", model.targets().get(0).id()); + Assertions.assertEquals("uwhvylwzbtdhxujz", model.targets().get(0).name()); + Assertions.assertEquals("mpowuwpr", model.targets().get(0).description()); + Assertions.assertEquals("veual", model.targets().get(0).acceptedDataFormats().get(0)); + Assertions.assertEquals("mkh", model.targets().get(0).acceptedContentEncodings().get(0)); + Assertions.assertEquals("bcswsrt", model.skus().get(0).id()); + Assertions.assertEquals("iplrbpbewtghfgb", model.skus().get(0).name()); + Assertions.assertEquals("gw", model.skus().get(0).version()); + Assertions.assertEquals("vlvqhjkbegi", model.skus().get(0).description()); + Assertions.assertEquals("nmxiebwwaloayqc", model.skus().get(0).restrictedAccessUri()); Assertions.assertEquals(false, model.skus().get(0).autoAdd()); - Assertions.assertEquals("meue", model.skus().get(0).targets().get(0)); - Assertions.assertEquals("wmcdytdxwi", model.skus().get(0).quotaDimensions().get(0).id()); - Assertions.assertEquals("nrjawgqwg", model.skus().get(0).quotaDimensions().get(0).scope()); - Assertions.assertEquals("ni", model.skus().get(0).quotaDimensions().get(0).period()); - Assertions.assertEquals(5.4993095F, model.skus().get(0).quotaDimensions().get(0).quota()); - Assertions.assertEquals("bkpyc", model.skus().get(0).quotaDimensions().get(0).name()); - Assertions.assertEquals("lwn", model.skus().get(0).quotaDimensions().get(0).description()); - Assertions.assertEquals("hjdauwhvylwz", model.skus().get(0).quotaDimensions().get(0).unit()); - Assertions.assertEquals("dhxujznbmpo", model.skus().get(0).quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("xiebwwaloayqcg", model.skus().get(0).pricingDetails().get(0).id()); - Assertions.assertEquals("tzjuzgwyzmhtxo", model.skus().get(0).pricingDetails().get(0).value()); - Assertions.assertEquals("ssotftpv", model.quotaDimensions().get(0).id()); - Assertions.assertEquals("bexilzznfqqnv", model.quotaDimensions().get(0).scope()); - Assertions.assertEquals("mqtaruoujmkcjh", model.quotaDimensions().get(0).period()); - Assertions.assertEquals(83.08001F, model.quotaDimensions().get(0).quota()); - Assertions.assertEquals("jrybnwjewgdrjer", model.quotaDimensions().get(0).name()); - Assertions.assertEquals("aen", model.quotaDimensions().get(0).description()); - Assertions.assertEquals("eh", model.quotaDimensions().get(0).unit()); - Assertions.assertEquals("doy", model.quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("thnzd", model.pricingDimensions().get(0).id()); - Assertions.assertEquals("sl", model.pricingDimensions().get(0).name()); + Assertions.assertEquals("j", model.skus().get(0).targets().get(0)); + Assertions.assertEquals("htxongmtsavjc", model.skus().get(0).quotaDimensions().get(0).id()); + Assertions.assertEquals("wxqpsrknftguvri", model.skus().get(0).quotaDimensions().get(0).scope()); + Assertions.assertEquals("prwmdyvxqt", model.skus().get(0).quotaDimensions().get(0).period()); + Assertions.assertEquals(42.644672F, model.skus().get(0).quotaDimensions().get(0).quota()); + Assertions.assertEquals("ww", model.skus().get(0).quotaDimensions().get(0).name()); + Assertions.assertEquals("yqbexrmcqibycno", model.skus().get(0).quotaDimensions().get(0).description()); + Assertions.assertEquals("knme", model.skus().get(0).quotaDimensions().get(0).unit()); + Assertions.assertEquals("sgzvahapjyzhpv", model.skus().get(0).quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("j", model.skus().get(0).pricingDetails().get(0).id()); + Assertions.assertEquals("xdjzlmwlxk", model.skus().get(0).pricingDetails().get(0).value()); + Assertions.assertEquals("yofd", model.quotaDimensions().get(0).id()); + Assertions.assertEquals("uusdttouwa", model.quotaDimensions().get(0).scope()); + Assertions.assertEquals("ekqvkeln", model.quotaDimensions().get(0).period()); + Assertions.assertEquals(63.19878F, model.quotaDimensions().get(0).quota()); + Assertions.assertEquals("xwyjsflhhc", model.quotaDimensions().get(0).name()); + Assertions.assertEquals("lnjixisxya", model.quotaDimensions().get(0).description()); + Assertions.assertEquals("oyaqcslyjpkiid", model.quotaDimensions().get(0).unit()); + Assertions.assertEquals("exznelixhnr", model.quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("wptfdy", model.pricingDimensions().get(0).id()); + Assertions.assertEquals("fqbuaceopzf", model.pricingDimensions().get(0).name()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderTests.java index a5ff8076d0c53..3f43a0a184c11 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/ProviderTests.java @@ -12,35 +12,27 @@ public final class ProviderTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Provider model = - BinaryData - .fromString( - "{\"providerId\":\"lhmwhfpmrqobm\",\"providerSku\":\"kknryrtihf\",\"instanceUri\":\"ijbpzvgnwzsymgl\",\"applicationName\":\"fcyzkohdbihanufh\",\"provisioningState\":\"Updating\",\"resourceUsageId\":\"y\"}") - .toObject(Provider.class); - Assertions.assertEquals("lhmwhfpmrqobm", model.providerId()); - Assertions.assertEquals("kknryrtihf", model.providerSku()); - Assertions.assertEquals("ijbpzvgnwzsymgl", model.instanceUri()); - Assertions.assertEquals("fcyzkohdbihanufh", model.applicationName()); - Assertions.assertEquals(Status.UPDATING, model.provisioningState()); + Provider model = BinaryData.fromString( + "{\"providerId\":\"mofcq\",\"providerSku\":\"my\",\"instanceUri\":\"kdtmlxhekuk\",\"applicationName\":\"txukcdmp\",\"provisioningState\":\"Failed\",\"resourceUsageId\":\"y\"}") + .toObject(Provider.class); + Assertions.assertEquals("mofcq", model.providerId()); + Assertions.assertEquals("my", model.providerSku()); + Assertions.assertEquals("kdtmlxhekuk", model.instanceUri()); + Assertions.assertEquals("txukcdmp", model.applicationName()); + Assertions.assertEquals(Status.FAILED, model.provisioningState()); Assertions.assertEquals("y", model.resourceUsageId()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Provider model = - new Provider() - .withProviderId("lhmwhfpmrqobm") - .withProviderSku("kknryrtihf") - .withInstanceUri("ijbpzvgnwzsymgl") - .withApplicationName("fcyzkohdbihanufh") - .withProvisioningState(Status.UPDATING) - .withResourceUsageId("y"); + Provider model = new Provider().withProviderId("mofcq").withProviderSku("my").withInstanceUri("kdtmlxhekuk") + .withApplicationName("txukcdmp").withProvisioningState(Status.FAILED).withResourceUsageId("y"); model = BinaryData.fromObject(model).toObject(Provider.class); - Assertions.assertEquals("lhmwhfpmrqobm", model.providerId()); - Assertions.assertEquals("kknryrtihf", model.providerSku()); - Assertions.assertEquals("ijbpzvgnwzsymgl", model.instanceUri()); - Assertions.assertEquals("fcyzkohdbihanufh", model.applicationName()); - Assertions.assertEquals(Status.UPDATING, model.provisioningState()); + Assertions.assertEquals("mofcq", model.providerId()); + Assertions.assertEquals("my", model.providerSku()); + Assertions.assertEquals("kdtmlxhekuk", model.instanceUri()); + Assertions.assertEquals("txukcdmp", model.applicationName()); + Assertions.assertEquals(Status.FAILED, model.provisioningState()); Assertions.assertEquals("y", model.resourceUsageId()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceIdentityTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceIdentityTests.java index 61f01ad36955a..8bb2736853bdd 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceIdentityTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceIdentityTests.java @@ -12,9 +12,8 @@ public final class QuantumWorkspaceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - QuantumWorkspaceIdentity model = - BinaryData - .fromString("{\"principalId\":\"git\",\"tenantId\":\"qhabifpikxwcz\",\"type\":\"None\"}") + QuantumWorkspaceIdentity model + = BinaryData.fromString("{\"principalId\":\"nzwuxzd\",\"tenantId\":\"a\",\"type\":\"None\"}") .toObject(QuantumWorkspaceIdentity.class); Assertions.assertEquals(ResourceIdentityType.NONE, model.type()); } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceInnerTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceInnerTests.java deleted file mode 100644 index e36406f76ced0..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuantumWorkspaceInnerTests.java +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.quantum.fluent.models.QuantumWorkspaceInner; -import com.azure.resourcemanager.quantum.models.Provider; -import com.azure.resourcemanager.quantum.models.QuantumWorkspaceIdentity; -import com.azure.resourcemanager.quantum.models.ResourceIdentityType; -import com.azure.resourcemanager.quantum.models.Status; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class QuantumWorkspaceInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - QuantumWorkspaceInner model = - BinaryData - .fromString( - "{\"properties\":{\"providers\":[{\"providerId\":\"uv\",\"providerSku\":\"xpyb\",\"instanceUri\":\"m\",\"applicationName\":\"mtz\",\"provisioningState\":\"Launching\",\"resourceUsageId\":\"phrupidgsybbejhp\"}],\"usable\":\"Partial\",\"provisioningState\":\"ProviderUpdating\",\"storageAccount\":\"xaobhdxbmtqioqjz\",\"endpointUri\":\"tbmufpo\"},\"identity\":{\"principalId\":\"izhwlrxy\",\"tenantId\":\"soqijg\",\"type\":\"None\"},\"location\":\"pazlobcufpdz\",\"tags\":{\"qqjnqgl\":\"t\",\"foooj\":\"qgn\"},\"id\":\"wifsq\",\"name\":\"saagdf\",\"type\":\"glzlhjxrifkwmrv\"}") - .toObject(QuantumWorkspaceInner.class); - Assertions.assertEquals("pazlobcufpdz", model.location()); - Assertions.assertEquals("t", model.tags().get("qqjnqgl")); - Assertions.assertEquals(ResourceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("uv", model.providers().get(0).providerId()); - Assertions.assertEquals("xpyb", model.providers().get(0).providerSku()); - Assertions.assertEquals("m", model.providers().get(0).instanceUri()); - Assertions.assertEquals("mtz", model.providers().get(0).applicationName()); - Assertions.assertEquals(Status.LAUNCHING, model.providers().get(0).provisioningState()); - Assertions.assertEquals("phrupidgsybbejhp", model.providers().get(0).resourceUsageId()); - Assertions.assertEquals("xaobhdxbmtqioqjz", model.storageAccount()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - QuantumWorkspaceInner model = - new QuantumWorkspaceInner() - .withLocation("pazlobcufpdz") - .withTags(mapOf("qqjnqgl", "t", "foooj", "qgn")) - .withIdentity(new QuantumWorkspaceIdentity().withType(ResourceIdentityType.NONE)) - .withProviders( - Arrays - .asList( - new Provider() - .withProviderId("uv") - .withProviderSku("xpyb") - .withInstanceUri("m") - .withApplicationName("mtz") - .withProvisioningState(Status.LAUNCHING) - .withResourceUsageId("phrupidgsybbejhp"))) - .withStorageAccount("xaobhdxbmtqioqjz"); - model = BinaryData.fromObject(model).toObject(QuantumWorkspaceInner.class); - Assertions.assertEquals("pazlobcufpdz", model.location()); - Assertions.assertEquals("t", model.tags().get("qqjnqgl")); - Assertions.assertEquals(ResourceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals("uv", model.providers().get(0).providerId()); - Assertions.assertEquals("xpyb", model.providers().get(0).providerSku()); - Assertions.assertEquals("m", model.providers().get(0).instanceUri()); - Assertions.assertEquals("mtz", model.providers().get(0).applicationName()); - Assertions.assertEquals(Status.LAUNCHING, model.providers().get(0).provisioningState()); - Assertions.assertEquals("phrupidgsybbejhp", model.providers().get(0).resourceUsageId()); - Assertions.assertEquals("xaobhdxbmtqioqjz", model.storageAccount()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuotaDimensionTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuotaDimensionTests.java index 03c9f791e0982..d2c3a6d0277cf 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuotaDimensionTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/QuotaDimensionTests.java @@ -11,41 +11,32 @@ public final class QuotaDimensionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - QuotaDimension model = - BinaryData - .fromString( - "{\"id\":\"cjefuzmu\",\"scope\":\"bttdumorppxe\",\"period\":\"nzbtbhj\",\"quota\":21.287733,\"name\":\"fgohdneuelfphs\",\"description\":\"htozfikdow\",\"unit\":\"uuvxz\",\"unitPlural\":\"lvithhqzonosgg\"}") - .toObject(QuotaDimension.class); - Assertions.assertEquals("cjefuzmu", model.id()); - Assertions.assertEquals("bttdumorppxe", model.scope()); - Assertions.assertEquals("nzbtbhj", model.period()); - Assertions.assertEquals(21.287733F, model.quota()); - Assertions.assertEquals("fgohdneuelfphs", model.name()); - Assertions.assertEquals("htozfikdow", model.description()); - Assertions.assertEquals("uuvxz", model.unit()); - Assertions.assertEquals("lvithhqzonosgg", model.unitPlural()); + QuotaDimension model = BinaryData.fromString( + "{\"id\":\"hzdxssadbzm\",\"scope\":\"dfznudaodv\",\"period\":\"bncblylpstdbhhx\",\"quota\":95.43601,\"name\":\"zucerscdntnev\",\"description\":\"wjmy\",\"unit\":\"dsslswt\",\"unitPlural\":\"eriofzpyqs\"}") + .toObject(QuotaDimension.class); + Assertions.assertEquals("hzdxssadbzm", model.id()); + Assertions.assertEquals("dfznudaodv", model.scope()); + Assertions.assertEquals("bncblylpstdbhhx", model.period()); + Assertions.assertEquals(95.43601F, model.quota()); + Assertions.assertEquals("zucerscdntnev", model.name()); + Assertions.assertEquals("wjmy", model.description()); + Assertions.assertEquals("dsslswt", model.unit()); + Assertions.assertEquals("eriofzpyqs", model.unitPlural()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - QuotaDimension model = - new QuotaDimension() - .withId("cjefuzmu") - .withScope("bttdumorppxe") - .withPeriod("nzbtbhj") - .withQuota(21.287733F) - .withName("fgohdneuelfphs") - .withDescription("htozfikdow") - .withUnit("uuvxz") - .withUnitPlural("lvithhqzonosgg"); + QuotaDimension model = new QuotaDimension().withId("hzdxssadbzm").withScope("dfznudaodv") + .withPeriod("bncblylpstdbhhx").withQuota(95.43601F).withName("zucerscdntnev").withDescription("wjmy") + .withUnit("dsslswt").withUnitPlural("eriofzpyqs"); model = BinaryData.fromObject(model).toObject(QuotaDimension.class); - Assertions.assertEquals("cjefuzmu", model.id()); - Assertions.assertEquals("bttdumorppxe", model.scope()); - Assertions.assertEquals("nzbtbhj", model.period()); - Assertions.assertEquals(21.287733F, model.quota()); - Assertions.assertEquals("fgohdneuelfphs", model.name()); - Assertions.assertEquals("htozfikdow", model.description()); - Assertions.assertEquals("uuvxz", model.unit()); - Assertions.assertEquals("lvithhqzonosgg", model.unitPlural()); + Assertions.assertEquals("hzdxssadbzm", model.id()); + Assertions.assertEquals("dfznudaodv", model.scope()); + Assertions.assertEquals("bncblylpstdbhhx", model.period()); + Assertions.assertEquals(95.43601F, model.quota()); + Assertions.assertEquals("zucerscdntnev", model.name()); + Assertions.assertEquals("wjmy", model.description()); + Assertions.assertEquals("dsslswt", model.unit()); + Assertions.assertEquals("eriofzpyqs", model.unitPlural()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/SkuDescriptionTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/SkuDescriptionTests.java index 28d1960d3ec3e..881d7cb830fe7 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/SkuDescriptionTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/SkuDescriptionTests.java @@ -14,71 +14,57 @@ public final class SkuDescriptionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - SkuDescription model = - BinaryData - .fromString( - "{\"id\":\"ets\",\"name\":\"szhedplvw\",\"version\":\"ubmwmbesld\",\"description\":\"wwtppj\",\"restrictedAccessUri\":\"cxogaokonzm\",\"autoAdd\":true,\"targets\":[\"mkqzeqqkdltfzxmh\",\"v\"],\"quotaDimensions\":[{\"id\":\"eodkwobda\",\"scope\":\"tibqdxbxwakb\",\"period\":\"qxn\",\"quota\":52.112804,\"name\":\"gxhuriplbp\",\"description\":\"xunkbebxmubyynt\",\"unit\":\"rbqtkoie\",\"unitPlural\":\"eotg\"}],\"pricingDetails\":[{\"id\":\"tmuwlauwzi\",\"value\":\"bm\"}]}") - .toObject(SkuDescription.class); - Assertions.assertEquals("ets", model.id()); - Assertions.assertEquals("szhedplvw", model.name()); - Assertions.assertEquals("ubmwmbesld", model.version()); - Assertions.assertEquals("wwtppj", model.description()); - Assertions.assertEquals("cxogaokonzm", model.restrictedAccessUri()); - Assertions.assertEquals(true, model.autoAdd()); - Assertions.assertEquals("mkqzeqqkdltfzxmh", model.targets().get(0)); - Assertions.assertEquals("eodkwobda", model.quotaDimensions().get(0).id()); - Assertions.assertEquals("tibqdxbxwakb", model.quotaDimensions().get(0).scope()); - Assertions.assertEquals("qxn", model.quotaDimensions().get(0).period()); - Assertions.assertEquals(52.112804F, model.quotaDimensions().get(0).quota()); - Assertions.assertEquals("gxhuriplbp", model.quotaDimensions().get(0).name()); - Assertions.assertEquals("xunkbebxmubyynt", model.quotaDimensions().get(0).description()); - Assertions.assertEquals("rbqtkoie", model.quotaDimensions().get(0).unit()); - Assertions.assertEquals("eotg", model.quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("tmuwlauwzi", model.pricingDetails().get(0).id()); - Assertions.assertEquals("bm", model.pricingDetails().get(0).value()); + SkuDescription model = BinaryData.fromString( + "{\"id\":\"mcbxvwvxysl\",\"name\":\"hsfxoblytkb\",\"version\":\"pe\",\"description\":\"wfbkrvrns\",\"restrictedAccessUri\":\"hqjohxcrsbfova\",\"autoAdd\":false,\"targets\":[\"wbhsqfsub\"],\"quotaDimensions\":[{\"id\":\"irx\",\"scope\":\"ybsrfbjfdtwss\",\"period\":\"ftpvjzbexil\",\"quota\":45.3637,\"name\":\"qqnvwpmq\",\"description\":\"ruoujmk\",\"unit\":\"hwqytj\",\"unitPlural\":\"bnw\"}],\"pricingDetails\":[{\"id\":\"drjervnaenqpehin\",\"value\":\"ygmi\"},{\"id\":\"hnzdndslgnayqi\",\"value\":\"nduhavhqlkthum\"},{\"id\":\"olbgycduiertgccy\",\"value\":\"aolps\"},{\"id\":\"qlfmmdnbb\",\"value\":\"zpswiydmc\"}]}") + .toObject(SkuDescription.class); + Assertions.assertEquals("mcbxvwvxysl", model.id()); + Assertions.assertEquals("hsfxoblytkb", model.name()); + Assertions.assertEquals("pe", model.version()); + Assertions.assertEquals("wfbkrvrns", model.description()); + Assertions.assertEquals("hqjohxcrsbfova", model.restrictedAccessUri()); + Assertions.assertEquals(false, model.autoAdd()); + Assertions.assertEquals("wbhsqfsub", model.targets().get(0)); + Assertions.assertEquals("irx", model.quotaDimensions().get(0).id()); + Assertions.assertEquals("ybsrfbjfdtwss", model.quotaDimensions().get(0).scope()); + Assertions.assertEquals("ftpvjzbexil", model.quotaDimensions().get(0).period()); + Assertions.assertEquals(45.3637F, model.quotaDimensions().get(0).quota()); + Assertions.assertEquals("qqnvwpmq", model.quotaDimensions().get(0).name()); + Assertions.assertEquals("ruoujmk", model.quotaDimensions().get(0).description()); + Assertions.assertEquals("hwqytj", model.quotaDimensions().get(0).unit()); + Assertions.assertEquals("bnw", model.quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("drjervnaenqpehin", model.pricingDetails().get(0).id()); + Assertions.assertEquals("ygmi", model.pricingDetails().get(0).value()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - SkuDescription model = - new SkuDescription() - .withId("ets") - .withName("szhedplvw") - .withVersion("ubmwmbesld") - .withDescription("wwtppj") - .withRestrictedAccessUri("cxogaokonzm") - .withAutoAdd(true) - .withTargets(Arrays.asList("mkqzeqqkdltfzxmh", "v")) - .withQuotaDimensions( - Arrays - .asList( - new QuotaDimension() - .withId("eodkwobda") - .withScope("tibqdxbxwakb") - .withPeriod("qxn") - .withQuota(52.112804F) - .withName("gxhuriplbp") - .withDescription("xunkbebxmubyynt") - .withUnit("rbqtkoie") - .withUnitPlural("eotg"))) - .withPricingDetails(Arrays.asList(new PricingDetail().withId("tmuwlauwzi").withValue("bm"))); + SkuDescription model = new SkuDescription().withId("mcbxvwvxysl").withName("hsfxoblytkb").withVersion("pe") + .withDescription("wfbkrvrns").withRestrictedAccessUri("hqjohxcrsbfova").withAutoAdd(false) + .withTargets(Arrays.asList("wbhsqfsub")) + .withQuotaDimensions(Arrays.asList(new QuotaDimension().withId("irx").withScope("ybsrfbjfdtwss") + .withPeriod("ftpvjzbexil").withQuota(45.3637F).withName("qqnvwpmq").withDescription("ruoujmk") + .withUnit("hwqytj").withUnitPlural("bnw"))) + .withPricingDetails(Arrays.asList(new PricingDetail().withId("drjervnaenqpehin").withValue("ygmi"), + new PricingDetail().withId("hnzdndslgnayqi").withValue("nduhavhqlkthum"), + new PricingDetail().withId("olbgycduiertgccy").withValue("aolps"), + new PricingDetail().withId("qlfmmdnbb").withValue("zpswiydmc"))); model = BinaryData.fromObject(model).toObject(SkuDescription.class); - Assertions.assertEquals("ets", model.id()); - Assertions.assertEquals("szhedplvw", model.name()); - Assertions.assertEquals("ubmwmbesld", model.version()); - Assertions.assertEquals("wwtppj", model.description()); - Assertions.assertEquals("cxogaokonzm", model.restrictedAccessUri()); - Assertions.assertEquals(true, model.autoAdd()); - Assertions.assertEquals("mkqzeqqkdltfzxmh", model.targets().get(0)); - Assertions.assertEquals("eodkwobda", model.quotaDimensions().get(0).id()); - Assertions.assertEquals("tibqdxbxwakb", model.quotaDimensions().get(0).scope()); - Assertions.assertEquals("qxn", model.quotaDimensions().get(0).period()); - Assertions.assertEquals(52.112804F, model.quotaDimensions().get(0).quota()); - Assertions.assertEquals("gxhuriplbp", model.quotaDimensions().get(0).name()); - Assertions.assertEquals("xunkbebxmubyynt", model.quotaDimensions().get(0).description()); - Assertions.assertEquals("rbqtkoie", model.quotaDimensions().get(0).unit()); - Assertions.assertEquals("eotg", model.quotaDimensions().get(0).unitPlural()); - Assertions.assertEquals("tmuwlauwzi", model.pricingDetails().get(0).id()); - Assertions.assertEquals("bm", model.pricingDetails().get(0).value()); + Assertions.assertEquals("mcbxvwvxysl", model.id()); + Assertions.assertEquals("hsfxoblytkb", model.name()); + Assertions.assertEquals("pe", model.version()); + Assertions.assertEquals("wfbkrvrns", model.description()); + Assertions.assertEquals("hqjohxcrsbfova", model.restrictedAccessUri()); + Assertions.assertEquals(false, model.autoAdd()); + Assertions.assertEquals("wbhsqfsub", model.targets().get(0)); + Assertions.assertEquals("irx", model.quotaDimensions().get(0).id()); + Assertions.assertEquals("ybsrfbjfdtwss", model.quotaDimensions().get(0).scope()); + Assertions.assertEquals("ftpvjzbexil", model.quotaDimensions().get(0).period()); + Assertions.assertEquals(45.3637F, model.quotaDimensions().get(0).quota()); + Assertions.assertEquals("qqnvwpmq", model.quotaDimensions().get(0).name()); + Assertions.assertEquals("ruoujmk", model.quotaDimensions().get(0).description()); + Assertions.assertEquals("hwqytj", model.quotaDimensions().get(0).unit()); + Assertions.assertEquals("bnw", model.quotaDimensions().get(0).unitPlural()); + Assertions.assertEquals("drjervnaenqpehin", model.pricingDetails().get(0).id()); + Assertions.assertEquals("ygmi", model.pricingDetails().get(0).value()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TagsObjectTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TagsObjectTests.java index 6cfd4c088436c..9576ff2211642 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TagsObjectTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TagsObjectTests.java @@ -13,20 +13,19 @@ public final class TagsObjectTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TagsObject model = - BinaryData - .fromString("{\"tags\":{\"ivyqniwbybrkxvd\":\"pqxu\",\"fwvuk\":\"mjgr\"}}") - .toObject(TagsObject.class); - Assertions.assertEquals("pqxu", model.tags().get("ivyqniwbybrkxvd")); + TagsObject model + = BinaryData.fromString("{\"tags\":{\"knryrtihfxtij\":\"whfpmrqobmtu\"}}").toObject(TagsObject.class); + Assertions.assertEquals("whfpmrqobmtu", model.tags().get("knryrtihfxtij")); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TagsObject model = new TagsObject().withTags(mapOf("ivyqniwbybrkxvd", "pqxu", "fwvuk", "mjgr")); + TagsObject model = new TagsObject().withTags(mapOf("knryrtihfxtij", "whfpmrqobmtu")); model = BinaryData.fromObject(model).toObject(TagsObject.class); - Assertions.assertEquals("pqxu", model.tags().get("ivyqniwbybrkxvd")); + Assertions.assertEquals("whfpmrqobmtu", model.tags().get("knryrtihfxtij")); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TargetDescriptionTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TargetDescriptionTests.java index e7930997040f5..c8515fbc389d8 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TargetDescriptionTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/TargetDescriptionTests.java @@ -12,32 +12,26 @@ public final class TargetDescriptionTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - TargetDescription model = - BinaryData - .fromString( - "{\"id\":\"glzpswiydm\",\"name\":\"yhz\",\"description\":\"ss\",\"acceptedDataFormats\":[\"zmnvdfznudaod\",\"xzb\",\"cblylpstdbhhxsr\"],\"acceptedContentEncodings\":[\"ucerscdntnevfi\",\"jmygtdsslswtmwer\",\"ofz\",\"yqsemwa\"]}") - .toObject(TargetDescription.class); - Assertions.assertEquals("glzpswiydm", model.id()); - Assertions.assertEquals("yhz", model.name()); - Assertions.assertEquals("ss", model.description()); - Assertions.assertEquals("zmnvdfznudaod", model.acceptedDataFormats().get(0)); - Assertions.assertEquals("ucerscdntnevfi", model.acceptedContentEncodings().get(0)); + TargetDescription model = BinaryData.fromString( + "{\"id\":\"rmfqjhhkxbpvj\",\"name\":\"jhxxjyn\",\"description\":\"divkrt\",\"acceptedDataFormats\":[\"xqzvszjfa\"],\"acceptedContentEncodings\":[\"fdxxivetvtcqaqtd\"]}") + .toObject(TargetDescription.class); + Assertions.assertEquals("rmfqjhhkxbpvj", model.id()); + Assertions.assertEquals("jhxxjyn", model.name()); + Assertions.assertEquals("divkrt", model.description()); + Assertions.assertEquals("xqzvszjfa", model.acceptedDataFormats().get(0)); + Assertions.assertEquals("fdxxivetvtcqaqtd", model.acceptedContentEncodings().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - TargetDescription model = - new TargetDescription() - .withId("glzpswiydm") - .withName("yhz") - .withDescription("ss") - .withAcceptedDataFormats(Arrays.asList("zmnvdfznudaod", "xzb", "cblylpstdbhhxsr")) - .withAcceptedContentEncodings(Arrays.asList("ucerscdntnevfi", "jmygtdsslswtmwer", "ofz", "yqsemwa")); + TargetDescription model = new TargetDescription().withId("rmfqjhhkxbpvj").withName("jhxxjyn") + .withDescription("divkrt").withAcceptedDataFormats(Arrays.asList("xqzvszjfa")) + .withAcceptedContentEncodings(Arrays.asList("fdxxivetvtcqaqtd")); model = BinaryData.fromObject(model).toObject(TargetDescription.class); - Assertions.assertEquals("glzpswiydm", model.id()); - Assertions.assertEquals("yhz", model.name()); - Assertions.assertEquals("ss", model.description()); - Assertions.assertEquals("zmnvdfznudaod", model.acceptedDataFormats().get(0)); - Assertions.assertEquals("ucerscdntnevfi", model.acceptedContentEncodings().get(0)); + Assertions.assertEquals("rmfqjhhkxbpvj", model.id()); + Assertions.assertEquals("jhxxjyn", model.name()); + Assertions.assertEquals("divkrt", model.description()); + Assertions.assertEquals("xqzvszjfa", model.acceptedDataFormats().get(0)); + Assertions.assertEquals("fdxxivetvtcqaqtd", model.acceptedContentEncodings().get(0)); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceListResultTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceListResultTests.java deleted file mode 100644 index 005d2678a135d..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceListResultTests.java +++ /dev/null @@ -1,154 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.quantum.fluent.models.QuantumWorkspaceInner; -import com.azure.resourcemanager.quantum.models.Provider; -import com.azure.resourcemanager.quantum.models.QuantumWorkspaceIdentity; -import com.azure.resourcemanager.quantum.models.ResourceIdentityType; -import com.azure.resourcemanager.quantum.models.Status; -import com.azure.resourcemanager.quantum.models.WorkspaceListResult; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class WorkspaceListResultTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - WorkspaceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"providers\":[{\"providerId\":\"snhsjcnyejhkryh\",\"providerSku\":\"apcz\",\"instanceUri\":\"okjye\",\"applicationName\":\"kvnipjoxz\",\"provisioningState\":\"Launching\",\"resourceUsageId\":\"gejspodmailzyde\"},{\"providerId\":\"jwyahuxinpmqnja\",\"providerSku\":\"ixjsprozvcputeg\",\"instanceUri\":\"wmfdatscmdvpjhul\",\"applicationName\":\"uvm\",\"provisioningState\":\"Deleting\",\"resourceUsageId\":\"k\"}],\"usable\":\"Partial\",\"provisioningState\":\"ProviderLaunching\",\"storageAccount\":\"odjpslwejd\",\"endpointUri\":\"wryoqpsoacc\"},\"identity\":{\"principalId\":\"akl\",\"tenantId\":\"ahbc\",\"type\":\"None\"},\"location\":\"dfdosygexp\",\"tags\":{\"vdphlxaolthqtr\":\"akhmsbzjhcrz\",\"gvfcj\":\"qjbpfzfsin\",\"xjtfelluwfzit\":\"wzo\"},\"id\":\"np\",\"name\":\"qfpjk\",\"type\":\"lxofpdvhpfxxypin\"},{\"properties\":{\"providers\":[{\"providerId\":\"huyb\",\"providerSku\":\"podepoo\",\"instanceUri\":\"nuvamiheogna\",\"applicationName\":\"zxtheotusivyevcc\",\"provisioningState\":\"Succeeded\",\"resourceUsageId\":\"nhungbw\"},{\"providerId\":\"rnfygxgispem\",\"providerSku\":\"zfkufubljofx\",\"instanceUri\":\"ofjaeqjhqjb\",\"applicationName\":\"v\",\"provisioningState\":\"Succeeded\",\"resourceUsageId\":\"qulngsntnbybkzgc\"}],\"usable\":\"Yes\",\"provisioningState\":\"Succeeded\",\"storageAccount\":\"xwrljdouskcqvkoc\",\"endpointUri\":\"jdkwtnhxbnjb\"},\"identity\":{\"principalId\":\"qrglssainqpjwn\",\"tenantId\":\"ljfmppee\",\"type\":\"SystemAssigned\"},\"location\":\"xsabkyqdu\",\"tags\":{\"zdzevndh\":\"tcj\",\"feusnhut\":\"rwpdappdsbdkvwrw\"},\"id\":\"eltmrldhugjzzdat\",\"name\":\"xhocdgeablgphuti\",\"type\":\"ndv\"},{\"properties\":{\"providers\":[{\"providerId\":\"yiftyhxhuro\",\"providerSku\":\"tyxolniwpwc\",\"instanceUri\":\"jfkgiawxk\",\"applicationName\":\"ypl\",\"provisioningState\":\"Updating\",\"resourceUsageId\":\"asy\"},{\"providerId\":\"nddhsgcbacph\",\"providerSku\":\"koty\",\"instanceUri\":\"gou\",\"applicationName\":\"ndlik\",\"provisioningState\":\"Updating\",\"resourceUsageId\":\"gfgibm\"},{\"providerId\":\"gakeqsr\",\"providerSku\":\"bzqqedqytbciq\",\"instanceUri\":\"uflmm\",\"applicationName\":\"zsm\",\"provisioningState\":\"Failed\",\"resourceUsageId\":\"lougpbkw\"}],\"usable\":\"No\",\"provisioningState\":\"Failed\",\"storageAccount\":\"qktapspwgcuert\",\"endpointUri\":\"kdosvqw\"},\"identity\":{\"principalId\":\"dgbb\",\"tenantId\":\"ddgmb\",\"type\":\"SystemAssigned\"},\"location\":\"ppbhtqqrolfp\",\"tags\":{\"jaoyfhrtx\":\"algbquxigjyjg\"},\"id\":\"lnerkujysvleju\",\"name\":\"fqawrlyxw\",\"type\":\"kcprbnw\"}],\"nextLink\":\"gjvtbv\"}") - .toObject(WorkspaceListResult.class); - Assertions.assertEquals("dfdosygexp", model.value().get(0).location()); - Assertions.assertEquals("akhmsbzjhcrz", model.value().get(0).tags().get("vdphlxaolthqtr")); - Assertions.assertEquals(ResourceIdentityType.NONE, model.value().get(0).identity().type()); - Assertions.assertEquals("snhsjcnyejhkryh", model.value().get(0).providers().get(0).providerId()); - Assertions.assertEquals("apcz", model.value().get(0).providers().get(0).providerSku()); - Assertions.assertEquals("okjye", model.value().get(0).providers().get(0).instanceUri()); - Assertions.assertEquals("kvnipjoxz", model.value().get(0).providers().get(0).applicationName()); - Assertions.assertEquals(Status.LAUNCHING, model.value().get(0).providers().get(0).provisioningState()); - Assertions.assertEquals("gejspodmailzyde", model.value().get(0).providers().get(0).resourceUsageId()); - Assertions.assertEquals("odjpslwejd", model.value().get(0).storageAccount()); - Assertions.assertEquals("gjvtbv", model.nextLink()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - WorkspaceListResult model = - new WorkspaceListResult() - .withValue( - Arrays - .asList( - new QuantumWorkspaceInner() - .withLocation("dfdosygexp") - .withTags( - mapOf( - "vdphlxaolthqtr", - "akhmsbzjhcrz", - "gvfcj", - "qjbpfzfsin", - "xjtfelluwfzit", - "wzo")) - .withIdentity(new QuantumWorkspaceIdentity().withType(ResourceIdentityType.NONE)) - .withProviders( - Arrays - .asList( - new Provider() - .withProviderId("snhsjcnyejhkryh") - .withProviderSku("apcz") - .withInstanceUri("okjye") - .withApplicationName("kvnipjoxz") - .withProvisioningState(Status.LAUNCHING) - .withResourceUsageId("gejspodmailzyde"), - new Provider() - .withProviderId("jwyahuxinpmqnja") - .withProviderSku("ixjsprozvcputeg") - .withInstanceUri("wmfdatscmdvpjhul") - .withApplicationName("uvm") - .withProvisioningState(Status.DELETING) - .withResourceUsageId("k"))) - .withStorageAccount("odjpslwejd"), - new QuantumWorkspaceInner() - .withLocation("xsabkyqdu") - .withTags(mapOf("zdzevndh", "tcj", "feusnhut", "rwpdappdsbdkvwrw")) - .withIdentity( - new QuantumWorkspaceIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)) - .withProviders( - Arrays - .asList( - new Provider() - .withProviderId("huyb") - .withProviderSku("podepoo") - .withInstanceUri("nuvamiheogna") - .withApplicationName("zxtheotusivyevcc") - .withProvisioningState(Status.SUCCEEDED) - .withResourceUsageId("nhungbw"), - new Provider() - .withProviderId("rnfygxgispem") - .withProviderSku("zfkufubljofx") - .withInstanceUri("ofjaeqjhqjb") - .withApplicationName("v") - .withProvisioningState(Status.SUCCEEDED) - .withResourceUsageId("qulngsntnbybkzgc"))) - .withStorageAccount("xwrljdouskcqvkoc"), - new QuantumWorkspaceInner() - .withLocation("ppbhtqqrolfp") - .withTags(mapOf("jaoyfhrtx", "algbquxigjyjg")) - .withIdentity( - new QuantumWorkspaceIdentity().withType(ResourceIdentityType.SYSTEM_ASSIGNED)) - .withProviders( - Arrays - .asList( - new Provider() - .withProviderId("yiftyhxhuro") - .withProviderSku("tyxolniwpwc") - .withInstanceUri("jfkgiawxk") - .withApplicationName("ypl") - .withProvisioningState(Status.UPDATING) - .withResourceUsageId("asy"), - new Provider() - .withProviderId("nddhsgcbacph") - .withProviderSku("koty") - .withInstanceUri("gou") - .withApplicationName("ndlik") - .withProvisioningState(Status.UPDATING) - .withResourceUsageId("gfgibm"), - new Provider() - .withProviderId("gakeqsr") - .withProviderSku("bzqqedqytbciq") - .withInstanceUri("uflmm") - .withApplicationName("zsm") - .withProvisioningState(Status.FAILED) - .withResourceUsageId("lougpbkw"))) - .withStorageAccount("qktapspwgcuert"))) - .withNextLink("gjvtbv"); - model = BinaryData.fromObject(model).toObject(WorkspaceListResult.class); - Assertions.assertEquals("dfdosygexp", model.value().get(0).location()); - Assertions.assertEquals("akhmsbzjhcrz", model.value().get(0).tags().get("vdphlxaolthqtr")); - Assertions.assertEquals(ResourceIdentityType.NONE, model.value().get(0).identity().type()); - Assertions.assertEquals("snhsjcnyejhkryh", model.value().get(0).providers().get(0).providerId()); - Assertions.assertEquals("apcz", model.value().get(0).providers().get(0).providerSku()); - Assertions.assertEquals("okjye", model.value().get(0).providers().get(0).instanceUri()); - Assertions.assertEquals("kvnipjoxz", model.value().get(0).providers().get(0).applicationName()); - Assertions.assertEquals(Status.LAUNCHING, model.value().get(0).providers().get(0).provisioningState()); - Assertions.assertEquals("gejspodmailzyde", model.value().get(0).providers().get(0).resourceUsageId()); - Assertions.assertEquals("odjpslwejd", model.value().get(0).storageAccount()); - Assertions.assertEquals("gjvtbv", model.nextLink()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationsCheckNameAvailabilityWithResponseMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationsCheckNameAvailabilityWithResponseMockTests.java index b2ba4d2ee15f7..2c4e0e513af5b 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationsCheckNameAvailabilityWithResponseMockTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceOperationsCheckNameAvailabilityWithResponseMockTests.java @@ -31,44 +31,30 @@ public void testCheckNameAvailabilityWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = "{\"nameAvailable\":true,\"reason\":\"gmfpgvmp\",\"message\":\"as\"}"; + String responseStr = "{\"nameAvailable\":true,\"reason\":\"ncckw\",\"message\":\"zqwhxxbuyqaxzfeq\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + AzureQuantumManager manager = AzureQuantumManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - CheckNameAvailabilityResult response = - manager - .workspaceOperations() - .checkNameAvailabilityWithResponse( - "lgzrfzeeyeb", - new CheckNameAvailabilityParameters().withName("ikayuhqlbjbsybb").withType("r"), - com.azure.core.util.Context.NONE) - .getValue(); + CheckNameAvailabilityResult response = manager.workspaceOperations() + .checkNameAvailabilityWithResponse("beyvpnqicvinvkjj", + new CheckNameAvailabilityParameters().withName("xrbuukzclew").withType("mlwpazt"), + com.azure.core.util.Context.NONE) + .getValue(); Assertions.assertEquals(true, response.nameAvailable()); - Assertions.assertEquals("gmfpgvmp", response.reason()); + Assertions.assertEquals("ncckw", response.reason()); } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceResourcePropertiesTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceResourcePropertiesTests.java deleted file mode 100644 index 6000ff0deb9a4..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspaceResourcePropertiesTests.java +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.quantum.fluent.models.WorkspaceResourceProperties; -import com.azure.resourcemanager.quantum.models.Provider; -import com.azure.resourcemanager.quantum.models.Status; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class WorkspaceResourcePropertiesTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - WorkspaceResourceProperties model = - BinaryData - .fromString( - "{\"providers\":[{\"providerId\":\"zntocipaouajps\",\"providerSku\":\"cmpoyfdkfogkny\",\"instanceUri\":\"ofjdde\",\"applicationName\":\"rd\",\"provisioningState\":\"Deleted\",\"resourceUsageId\":\"wnw\"},{\"providerId\":\"itjz\",\"providerSku\":\"lusarh\",\"instanceUri\":\"fcqhsmyurkd\",\"applicationName\":\"lx\",\"provisioningState\":\"Succeeded\",\"resourceUsageId\":\"k\"}],\"usable\":\"Partial\",\"provisioningState\":\"ProviderDeleting\",\"storageAccount\":\"cdm\",\"endpointUri\":\"rcryuanzwuxzdxta\"}") - .toObject(WorkspaceResourceProperties.class); - Assertions.assertEquals("zntocipaouajps", model.providers().get(0).providerId()); - Assertions.assertEquals("cmpoyfdkfogkny", model.providers().get(0).providerSku()); - Assertions.assertEquals("ofjdde", model.providers().get(0).instanceUri()); - Assertions.assertEquals("rd", model.providers().get(0).applicationName()); - Assertions.assertEquals(Status.DELETED, model.providers().get(0).provisioningState()); - Assertions.assertEquals("wnw", model.providers().get(0).resourceUsageId()); - Assertions.assertEquals("cdm", model.storageAccount()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - WorkspaceResourceProperties model = - new WorkspaceResourceProperties() - .withProviders( - Arrays - .asList( - new Provider() - .withProviderId("zntocipaouajps") - .withProviderSku("cmpoyfdkfogkny") - .withInstanceUri("ofjdde") - .withApplicationName("rd") - .withProvisioningState(Status.DELETED) - .withResourceUsageId("wnw"), - new Provider() - .withProviderId("itjz") - .withProviderSku("lusarh") - .withInstanceUri("fcqhsmyurkd") - .withApplicationName("lx") - .withProvisioningState(Status.SUCCEEDED) - .withResourceUsageId("k"))) - .withStorageAccount("cdm"); - model = BinaryData.fromObject(model).toObject(WorkspaceResourceProperties.class); - Assertions.assertEquals("zntocipaouajps", model.providers().get(0).providerId()); - Assertions.assertEquals("cmpoyfdkfogkny", model.providers().get(0).providerSku()); - Assertions.assertEquals("ofjdde", model.providers().get(0).instanceUri()); - Assertions.assertEquals("rd", model.providers().get(0).applicationName()); - Assertions.assertEquals(Status.DELETED, model.providers().get(0).provisioningState()); - Assertions.assertEquals("wnw", model.providers().get(0).resourceUsageId()); - Assertions.assertEquals("cdm", model.storageAccount()); - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateMockTests.java deleted file mode 100644 index c78dd8e622c66..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesCreateOrUpdateMockTests.java +++ /dev/null @@ -1,133 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.quantum.AzureQuantumManager; -import com.azure.resourcemanager.quantum.models.Provider; -import com.azure.resourcemanager.quantum.models.QuantumWorkspace; -import com.azure.resourcemanager.quantum.models.QuantumWorkspaceIdentity; -import com.azure.resourcemanager.quantum.models.ResourceIdentityType; -import com.azure.resourcemanager.quantum.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class WorkspacesCreateOrUpdateMockTests { - @Test - public void testCreateOrUpdate() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"providers\":[{\"providerId\":\"tncormrlxqtvco\",\"providerSku\":\"dflvkg\",\"instanceUri\":\"bgdknnqv\",\"applicationName\":\"znqntoru\",\"provisioningState\":\"Updating\",\"resourceUsageId\":\"a\"}],\"usable\":\"Partial\",\"provisioningState\":\"Succeeded\",\"storageAccount\":\"rauwjuetaebu\",\"endpointUri\":\"vdmovsmzlxwabm\"},\"identity\":{\"principalId\":\"fkifr\",\"tenantId\":\"puqujmqlgkfbtn\",\"type\":\"SystemAssigned\"},\"location\":\"n\",\"tags\":{\"ujitcjedftww\":\"cn\",\"foqouicybx\":\"ezkojvdcp\"},\"id\":\"rzgszufoxci\",\"name\":\"opidoamciodh\",\"type\":\"haz\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - QuantumWorkspace response = - manager - .workspaces() - .define("vfdnwnwmewzsyyce") - .withRegion("yfzqwhxxbu") - .withExistingResourceGroup("exn") - .withTags(mapOf("ztppriolxorjalto", "xzfe")) - .withIdentity(new QuantumWorkspaceIdentity().withType(ResourceIdentityType.NONE)) - .withProviders( - Arrays - .asList( - new Provider() - .withProviderId("bjudpfrxtrthzv") - .withProviderSku("tdwkqbrq") - .withInstanceUri("paxh") - .withApplicationName("iilivpdtiirqtd") - .withProvisioningState(Status.SUCCEEDED) - .withResourceUsageId("oruzfgsquyfxrxx"), - new Provider() - .withProviderId("ptramxj") - .withProviderSku("wlwnwxuqlcv") - .withInstanceUri("ypatdooaojkniod") - .withApplicationName("oebwnujhemms") - .withProvisioningState(Status.DELETING) - .withResourceUsageId("c"), - new Provider() - .withProviderId("dtjinfw") - .withProviderSku("fltkacjv") - .withInstanceUri("kdlfoa") - .withApplicationName("gkfpaga") - .withProvisioningState(Status.UPDATING) - .withResourceUsageId("lpqblylsyxk"), - new Provider() - .withProviderId("nsj") - .withProviderSku("vti") - .withInstanceUri("xsdszuempsb") - .withApplicationName("f") - .withProvisioningState(Status.FAILED) - .withResourceUsageId("v"))) - .withStorageAccount("invkjjxdxrbuu") - .create(); - - Assertions.assertEquals("n", response.location()); - Assertions.assertEquals("cn", response.tags().get("ujitcjedftww")); - Assertions.assertEquals(ResourceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); - Assertions.assertEquals("tncormrlxqtvco", response.providers().get(0).providerId()); - Assertions.assertEquals("dflvkg", response.providers().get(0).providerSku()); - Assertions.assertEquals("bgdknnqv", response.providers().get(0).instanceUri()); - Assertions.assertEquals("znqntoru", response.providers().get(0).applicationName()); - Assertions.assertEquals(Status.UPDATING, response.providers().get(0).provisioningState()); - Assertions.assertEquals("a", response.providers().get(0).resourceUsageId()); - Assertions.assertEquals("rauwjuetaebu", response.storageAccount()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteMockTests.java index 1fd691eae2d46..9112c03b3f645 100644 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteMockTests.java +++ b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.workspaces().delete("wznm", "biknsorgjhxbld", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + AzureQuantumManager manager = AzureQuantumManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.workspaces().delete("uxvypomgkopkwh", "jvp", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java deleted file mode 100644 index 0b8a93e712a3d..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesGetByResourceGroupWithResponseMockTests.java +++ /dev/null @@ -1,81 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.quantum.AzureQuantumManager; -import com.azure.resourcemanager.quantum.models.QuantumWorkspace; -import com.azure.resourcemanager.quantum.models.ResourceIdentityType; -import com.azure.resourcemanager.quantum.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class WorkspacesGetByResourceGroupWithResponseMockTests { - @Test - public void testGetByResourceGroupWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"properties\":{\"providers\":[{\"providerId\":\"qacpiex\",\"providerSku\":\"tg\",\"instanceUri\":\"bwoenwashrt\",\"applicationName\":\"kcnqxwbpo\",\"provisioningState\":\"Succeeded\",\"resourceUsageId\":\"iujwa\"},{\"providerId\":\"ipqiiobyuqerpq\",\"providerSku\":\"qwcciuqg\",\"instanceUri\":\"butauvfb\",\"applicationName\":\"uwhhmhykojoxafn\",\"provisioningState\":\"Launching\",\"resourceUsageId\":\"ichkoymkcdyhb\"},{\"providerId\":\"kpw\",\"providerSku\":\"eqnovvqfovl\",\"instanceUri\":\"ywsuwsy\",\"applicationName\":\"ndsytgadg\",\"provisioningState\":\"Launching\",\"resourceUsageId\":\"aeneqnzarrwl\"}],\"usable\":\"No\",\"provisioningState\":\"ProviderProvisioning\",\"storageAccount\":\"qkacewii\",\"endpointUri\":\"pubjibw\"},\"identity\":{\"principalId\":\"tohqkvpuvksgp\",\"tenantId\":\"aknynfsynljphuop\",\"type\":\"None\"},\"location\":\"qiy\",\"tags\":{\"hleosjsw\":\"rz\",\"qioxi\":\"rmslyzrpzbchckqq\"},\"id\":\"suiizynkedyat\",\"name\":\"wyhqmibzyhwits\",\"type\":\"ypyynpcdpumnzg\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - QuantumWorkspace response = - manager - .workspaces() - .getByResourceGroupWithResponse("wfluszdt", "hrkwo", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("qiy", response.location()); - Assertions.assertEquals("rz", response.tags().get("hleosjsw")); - Assertions.assertEquals(ResourceIdentityType.NONE, response.identity().type()); - Assertions.assertEquals("qacpiex", response.providers().get(0).providerId()); - Assertions.assertEquals("tg", response.providers().get(0).providerSku()); - Assertions.assertEquals("bwoenwashrt", response.providers().get(0).instanceUri()); - Assertions.assertEquals("kcnqxwbpo", response.providers().get(0).applicationName()); - Assertions.assertEquals(Status.SUCCEEDED, response.providers().get(0).provisioningState()); - Assertions.assertEquals("iujwa", response.providers().get(0).resourceUsageId()); - Assertions.assertEquals("qkacewii", response.storageAccount()); - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupMockTests.java deleted file mode 100644 index 2fd7d9da4e3e8..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListByResourceGroupMockTests.java +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.quantum.AzureQuantumManager; -import com.azure.resourcemanager.quantum.models.QuantumWorkspace; -import com.azure.resourcemanager.quantum.models.ResourceIdentityType; -import com.azure.resourcemanager.quantum.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class WorkspacesListByResourceGroupMockTests { - @Test - public void testListByResourceGroup() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"providers\":[{\"providerId\":\"kzikfjawneaivxwc\",\"providerSku\":\"lpcirelsf\",\"instanceUri\":\"enwabfatk\",\"applicationName\":\"dxbjhwuaanozj\",\"provisioningState\":\"Updating\",\"resourceUsageId\":\"youlp\"},{\"providerId\":\"v\",\"providerSku\":\"glrvimjwosytxi\",\"instanceUri\":\"skfc\",\"applicationName\":\"qumiek\",\"provisioningState\":\"Deleted\",\"resourceUsageId\":\"ikh\"}],\"usable\":\"No\",\"provisioningState\":\"Failed\",\"storageAccount\":\"gqggebdunygae\",\"endpointUri\":\"db\"},\"identity\":{\"principalId\":\"t\",\"tenantId\":\"llrxcyjmoad\",\"type\":\"None\"},\"location\":\"r\",\"tags\":{\"hhyxxrw\":\"dmjsjqb\",\"duhpk\":\"yc\",\"hky\":\"kgymareqnajxqug\",\"sofwqmzqalkrmnji\":\"ubeddg\"},\"id\":\"pxacqqudfn\",\"name\":\"yxbaaabjyvayf\",\"type\":\"imrzrtuzqog\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.workspaces().listByResourceGroup("fnjhfjxwmszkkfo", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("r", response.iterator().next().location()); - Assertions.assertEquals("dmjsjqb", response.iterator().next().tags().get("hhyxxrw")); - Assertions.assertEquals(ResourceIdentityType.NONE, response.iterator().next().identity().type()); - Assertions.assertEquals("kzikfjawneaivxwc", response.iterator().next().providers().get(0).providerId()); - Assertions.assertEquals("lpcirelsf", response.iterator().next().providers().get(0).providerSku()); - Assertions.assertEquals("enwabfatk", response.iterator().next().providers().get(0).instanceUri()); - Assertions.assertEquals("dxbjhwuaanozj", response.iterator().next().providers().get(0).applicationName()); - Assertions.assertEquals(Status.UPDATING, response.iterator().next().providers().get(0).provisioningState()); - Assertions.assertEquals("youlp", response.iterator().next().providers().get(0).resourceUsageId()); - Assertions.assertEquals("gqggebdunygae", response.iterator().next().storageAccount()); - } -} diff --git a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListMockTests.java b/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListMockTests.java deleted file mode 100644 index 86c21d798c05a..0000000000000 --- a/sdk/quantum/azure-resourcemanager-quantum/src/test/java/com/azure/resourcemanager/quantum/generated/WorkspacesListMockTests.java +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.quantum.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.quantum.AzureQuantumManager; -import com.azure.resourcemanager.quantum.models.QuantumWorkspace; -import com.azure.resourcemanager.quantum.models.ResourceIdentityType; -import com.azure.resourcemanager.quantum.models.Status; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class WorkspacesListMockTests { - @Test - public void testList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"properties\":{\"providers\":[{\"providerId\":\"lkdmtncvokotllxd\",\"providerSku\":\"gsyocogj\",\"instanceUri\":\"dtbnnha\",\"applicationName\":\"ocrkvcikh\",\"provisioningState\":\"Failed\",\"resourceUsageId\":\"mqg\"},{\"providerId\":\"queziky\",\"providerSku\":\"gxk\",\"instanceUri\":\"la\",\"applicationName\":\"elwuipi\",\"provisioningState\":\"Deleted\",\"resourceUsageId\":\"kzivgvvcnayrh\"}],\"usable\":\"Partial\",\"provisioningState\":\"ProviderDeleting\",\"storageAccount\":\"ueedndrdvs\",\"endpointUri\":\"wq\"},\"identity\":{\"principalId\":\"healmfmtda\",\"tenantId\":\"gdv\",\"type\":\"None\"},\"location\":\"iohgwxrtfud\",\"tags\":{\"qagvrvm\":\"xg\"},\"id\":\"pkukghi\",\"name\":\"dblx\",\"type\":\"wi\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - AzureQuantumManager manager = - AzureQuantumManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = manager.workspaces().list(com.azure.core.util.Context.NONE); - - Assertions.assertEquals("iohgwxrtfud", response.iterator().next().location()); - Assertions.assertEquals("xg", response.iterator().next().tags().get("qagvrvm")); - Assertions.assertEquals(ResourceIdentityType.NONE, response.iterator().next().identity().type()); - Assertions.assertEquals("lkdmtncvokotllxd", response.iterator().next().providers().get(0).providerId()); - Assertions.assertEquals("gsyocogj", response.iterator().next().providers().get(0).providerSku()); - Assertions.assertEquals("dtbnnha", response.iterator().next().providers().get(0).instanceUri()); - Assertions.assertEquals("ocrkvcikh", response.iterator().next().providers().get(0).applicationName()); - Assertions.assertEquals(Status.FAILED, response.iterator().next().providers().get(0).provisioningState()); - Assertions.assertEquals("mqg", response.iterator().next().providers().get(0).resourceUsageId()); - Assertions.assertEquals("ueedndrdvs", response.iterator().next().storageAccount()); - } -} diff --git a/sdk/qumulo/azure-resourcemanager-qumulo/pom.xml b/sdk/qumulo/azure-resourcemanager-qumulo/pom.xml index 3d63b37c1a505..7772366710197 100644 --- a/sdk/qumulo/azure-resourcemanager-qumulo/pom.xml +++ b/sdk/qumulo/azure-resourcemanager-qumulo/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/quota/azure-resourcemanager-quota/pom.xml b/sdk/quota/azure-resourcemanager-quota/pom.xml index a0a57dadad420..7d69303a0fe39 100644 --- a/sdk/quota/azure-resourcemanager-quota/pom.xml +++ b/sdk/quota/azure-resourcemanager-quota/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/recoveryservices/azure-resourcemanager-recoveryservices/pom.xml b/sdk/recoveryservices/azure-resourcemanager-recoveryservices/pom.xml index 2c44ed89e04b5..088189b238f43 100644 --- a/sdk/recoveryservices/azure-resourcemanager-recoveryservices/pom.xml +++ b/sdk/recoveryservices/azure-resourcemanager-recoveryservices/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -110,7 +110,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/recoveryservices/tests.mgmt.yml b/sdk/recoveryservices/tests.mgmt.yml index 1184a0c920875..4edcc54948303 100644 --- a/sdk/recoveryservices/tests.mgmt.yml +++ b/sdk/recoveryservices/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: recoveryservices - Artifacts: - - name: azure-resourcemanager-recoveryservices - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerrecoveryservices - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: recoveryservices + Artifacts: + - name: azure-resourcemanager-recoveryservices + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerrecoveryservices + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/recoveryservicesbackup/azure-resourcemanager-recoveryservicesbackup/pom.xml b/sdk/recoveryservicesbackup/azure-resourcemanager-recoveryservicesbackup/pom.xml index 8b4e7ee5de491..fa39dc465ee61 100644 --- a/sdk/recoveryservicesbackup/azure-resourcemanager-recoveryservicesbackup/pom.xml +++ b/sdk/recoveryservicesbackup/azure-resourcemanager-recoveryservicesbackup/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -91,13 +91,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -115,7 +115,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/recoveryservicesbackup/tests.mgmt.yml b/sdk/recoveryservicesbackup/tests.mgmt.yml index e868aeab61977..25e4d334cd847 100644 --- a/sdk/recoveryservicesbackup/tests.mgmt.yml +++ b/sdk/recoveryservicesbackup/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: recoveryservicesbackup - Artifacts: - - name: azure-resourcemanager-recoveryservicesbackup - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerrecoveryservicesbackup - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: recoveryservicesbackup + Artifacts: + - name: azure-resourcemanager-recoveryservicesbackup + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerrecoveryservicesbackup + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/recoveryservicesdatareplication/azure-resourcemanager-recoveryservicesdatareplication/pom.xml b/sdk/recoveryservicesdatareplication/azure-resourcemanager-recoveryservicesdatareplication/pom.xml index 2b242bd109fb1..3406e65a9ab9e 100644 --- a/sdk/recoveryservicesdatareplication/azure-resourcemanager-recoveryservicesdatareplication/pom.xml +++ b/sdk/recoveryservicesdatareplication/azure-resourcemanager-recoveryservicesdatareplication/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/recoveryservicessiterecovery/azure-resourcemanager-recoveryservicessiterecovery/pom.xml b/sdk/recoveryservicessiterecovery/azure-resourcemanager-recoveryservicessiterecovery/pom.xml index 76875c4bc1ba9..2cd4e461429a8 100644 --- a/sdk/recoveryservicessiterecovery/azure-resourcemanager-recoveryservicessiterecovery/pom.xml +++ b/sdk/recoveryservicessiterecovery/azure-resourcemanager-recoveryservicessiterecovery/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -90,13 +90,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/CHANGELOG.md b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/CHANGELOG.md index 583af28ae9fbe..48659a6d3eb95 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/CHANGELOG.md +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.1.0-beta.4 (Unreleased) +## 2.1.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,123 @@ ### Other Changes +## 2.0.0 (2024-03-14) + +- Azure Resource Manager RedisEnterprise client library for Java. This package contains Microsoft Azure SDK for RedisEnterprise Management SDK. REST API for managing Redis Enterprise resources in Azure. Package tag package-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Breaking Changes + +#### `models.ImportClusterParameters` was modified + +* `sasUri()` was removed +* `withSasUri(java.lang.String)` was removed + +#### `models.Database$Update` was modified + +* `withModules(java.util.List)` was removed +* `withClusteringPolicy(models.ClusteringPolicy)` was removed + +#### `models.PrivateEndpointConnections` was modified + +* `deleteWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `com.azure.core.http.rest.Response deleteByIdWithResponse(java.lang.String,com.azure.core.util.Context)` -> `void deleteByIdWithResponse(java.lang.String,com.azure.core.util.Context)` + +### Features Added + +* `models.ManagedServiceIdentity` was added + +* `models.ForceUnlinkParameters` was added + +* `models.UserAssignedIdentity` was added + +* `models.ClusterPropertiesEncryption` was added + +* `models.DatabasePropertiesGeoReplication` was added + +* `models.LinkedDatabase` was added + +* `models.ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity` was added + +* `models.LinkState` was added + +* `models.ManagedServiceIdentityType` was added + +* `models.FlushParameters` was added + +* `models.CmkIdentityType` was added + +* `models.ClusterPropertiesEncryptionCustomerManagedKeyEncryption` was added + +#### `models.ClusterUpdate` was modified + +* `withIdentity(models.ManagedServiceIdentity)` was added +* `identity()` was added +* `encryption()` was added +* `withEncryption(models.ClusterPropertiesEncryption)` was added + +#### `models.DatabaseUpdate` was modified + +* `withGeoReplication(models.DatabasePropertiesGeoReplication)` was added +* `geoReplication()` was added + +#### `models.Cluster$Update` was modified + +* `withEncryption(models.ClusterPropertiesEncryption)` was added +* `withIdentity(models.ManagedServiceIdentity)` was added + +#### `RedisEnterpriseManager` was modified + +* `authenticate(com.azure.core.http.HttpPipeline,com.azure.core.management.profile.AzureProfile)` was added + +#### `models.Database$Definition` was modified + +* `withGeoReplication(models.DatabasePropertiesGeoReplication)` was added + +#### `models.ImportClusterParameters` was modified + +* `sasUris()` was added +* `withSasUris(java.util.List)` was added + +#### `models.Cluster$Definition` was modified + +* `withEncryption(models.ClusterPropertiesEncryption)` was added +* `withIdentity(models.ManagedServiceIdentity)` was added + +#### `RedisEnterpriseManager$Configurable` was modified + +* `withRetryOptions(com.azure.core.http.policy.RetryOptions)` was added +* `withScope(java.lang.String)` was added + +#### `models.PrivateEndpointConnections` was modified + +* `delete(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added + +#### `models.Database` was modified + +* `flush(models.FlushParameters,com.azure.core.util.Context)` was added +* `geoReplication()` was added +* `forceUnlink(models.ForceUnlinkParameters,com.azure.core.util.Context)` was added +* `forceUnlink(models.ForceUnlinkParameters)` was added +* `flush(models.FlushParameters)` was added +* `resourceGroupName()` was added + +#### `models.PrivateEndpointConnection` was modified + +* `resourceGroupName()` was added + +#### `models.Cluster` was modified + +* `encryption()` was added +* `resourceGroupName()` was added +* `identity()` was added + +#### `models.Databases` was modified + +* `flush(java.lang.String,java.lang.String,java.lang.String,models.FlushParameters,com.azure.core.util.Context)` was added +* `forceUnlink(java.lang.String,java.lang.String,java.lang.String,models.ForceUnlinkParameters)` was added +* `flush(java.lang.String,java.lang.String,java.lang.String,models.FlushParameters)` was added +* `forceUnlink(java.lang.String,java.lang.String,java.lang.String,models.ForceUnlinkParameters,com.azure.core.util.Context)` was added + ## 1.1.0-beta.3 (2023-03-21) - Azure Resource Manager RedisEnterprise client library for Java. This package contains Microsoft Azure SDK for RedisEnterprise Management SDK. REST API for managing Redis Enterprise resources in Azure. Package tag package-preview-2023-03. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/README.md b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/README.md index f0c7862a34854..49b1dd7eec847 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/README.md +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/README.md @@ -2,7 +2,7 @@ Azure Resource Manager RedisEnterprise client library for Java. -This package contains Microsoft Azure SDK for RedisEnterprise Management SDK. REST API for managing Redis Enterprise resources in Azure. Package tag package-preview-2023-03. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). +This package contains Microsoft Azure SDK for RedisEnterprise Management SDK. REST API for managing Redis Enterprise resources in Azure. Package tag package-2023-11. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). ## We'd love to hear your feedback @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-redisenterprise - 1.1.0-beta.3 + 2.0.0 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -74,6 +74,16 @@ See [API design][design] for general introduction on design and key concepts on ## Examples +```java +cluster = redisEnterpriseManager.redisEnterprises() + .define(clusterName) + .withRegion(REGION) + .withExistingResourceGroup(resourceGroupName) + .withSku(new Sku().withName(SkuName.ENTERPRISE_E10).withCapacity(2)) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE)) + .withMinimumTlsVersion(TlsVersion.ONE_TWO) + .create(); +``` [Code snippets and samples](https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/redisenterprise/azure-resourcemanager-redisenterprise/SAMPLE.md) @@ -94,7 +104,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty @@ -103,3 +113,5 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [cg]: https://github.com/Azure/azure-sdk-for-java/blob/main/CONTRIBUTING.md [coc]: https://opensource.microsoft.com/codeofconduct/ [coc_faq]: https://opensource.microsoft.com/codeofconduct/faq/ + +![Impressions](https://azure-sdk-impressions.azurewebsites.net/api/impressions/azure-sdk-for-java%2Fsdk%2Fredisenterprise%2Fazure-resourcemanager-redisenterprise%2FREADME.png) diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/SAMPLE.md b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/SAMPLE.md index c1913a573123d..f25d452b2a4d0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/SAMPLE.md +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/SAMPLE.md @@ -42,10 +42,6 @@ - [List](#redisenterprise_list) - [ListByResourceGroup](#redisenterprise_listbyresourcegroup) - [Update](#redisenterprise_update) - -## Skus - -- [List](#skus_list) ### Databases_Create ```java @@ -59,66 +55,50 @@ import com.azure.resourcemanager.redisenterprise.models.Persistence; import com.azure.resourcemanager.redisenterprise.models.Protocol; import java.util.Arrays; -/** Samples for Databases Create. */ +/** + * Samples for Databases Create. + */ public final class DatabasesCreateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesCreate.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesCreate.json */ /** * Sample code: RedisEnterpriseDatabasesCreate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesCreate( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .define("default") - .withExistingRedisEnterprise("rg1", "cache1") - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(10000) - .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) - .withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) + public static void + redisEnterpriseDatabasesCreate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().define("default").withExistingRedisEnterprise("rg1", "cache1") + .withClientProtocol(Protocol.ENCRYPTED).withPort(10000) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER).withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) .withPersistence(new Persistence().withAofEnabled(true).withAofFrequency(AofFrequency.ONES)) - .withModules( - Arrays - .asList( - new Module().withName("RedisBloom").withArgs("ERROR_RATE 0.00 INITIAL_SIZE 400"), - new Module().withName("RedisTimeSeries").withArgs("RETENTION_POLICY 20"), - new Module().withName("RediSearch"))) + .withModules(Arrays.asList(new Module().withName("RedisBloom").withArgs("ERROR_RATE 0.00 INITIAL_SIZE 400"), + new Module().withName("RedisTimeSeries").withArgs("RETENTION_POLICY 20"), + new Module().withName("RediSearch"))) .create(); } /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesCreateWithGeoReplication.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesCreateWithGeoReplication.json */ /** * Sample code: RedisEnterpriseDatabasesCreate With Active Geo Replication. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDatabasesCreateWithActiveGeoReplication( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .define("default") - .withExistingRedisEnterprise("rg1", "cache1") - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(10000) - .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) - .withEvictionPolicy(EvictionPolicy.NO_EVICTION) - .withGeoReplication( - new DatabasePropertiesGeoReplication() - .withGroupNickname("groupName") - .withLinkedDatabases( - Arrays - .asList( - new LinkedDatabase() - .withId( - "/subscriptions/subid1/resourceGroups/rg1/providers/Microsoft.Cache/redisEnterprise/cache1/databases/default"), - new LinkedDatabase() - .withId( - "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")))) + manager.databases().define("default").withExistingRedisEnterprise("rg1", "cache1") + .withClientProtocol(Protocol.ENCRYPTED).withPort(10000) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER).withEvictionPolicy(EvictionPolicy.NO_EVICTION) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("groupName") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase().withId( + "/subscriptions/subid1/resourceGroups/rg1/providers/Microsoft.Cache/redisEnterprise/cache1/databases/default"), + new LinkedDatabase().withId( + "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")))) .create(); } } @@ -127,18 +107,21 @@ public final class DatabasesCreateSamples { ### Databases_Delete ```java -/** Samples for Databases Delete. */ +/** + * Samples for Databases Delete. + */ public final class DatabasesDeleteSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesDelete.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesDelete.json */ /** * Sample code: RedisEnterpriseDatabasesDelete. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesDelete( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseDatabasesDelete(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.databases().delete("rg1", "cache1", "db1", com.azure.core.util.Context.NONE); } } @@ -149,27 +132,25 @@ public final class DatabasesDeleteSamples { ```java import com.azure.resourcemanager.redisenterprise.models.ExportClusterParameters; -/** Samples for Databases Export. */ +/** + * Samples for Databases Export. + */ public final class DatabasesExportSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesExport.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesExport.json */ /** * Sample code: RedisEnterpriseDatabasesExport. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesExport( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .export( - "rg1", - "cache1", - "default", - new ExportClusterParameters() - .withSasUri("https://contosostorage.blob.core.window.net/urlToBlobContainer?sasKeyParameters"), - com.azure.core.util.Context.NONE); + public static void + redisEnterpriseDatabasesExport(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().export("rg1", "cache1", "default", + new ExportClusterParameters() + .withSasUri("https://contosostorage.blob.core.window.net/urlToBlobContainer?sasKeyParameters"), + com.azure.core.util.Context.NONE); } } ``` @@ -180,30 +161,24 @@ public final class DatabasesExportSamples { import com.azure.resourcemanager.redisenterprise.models.FlushParameters; import java.util.Arrays; -/** Samples for Databases Flush. */ +/** + * Samples for Databases Flush. + */ public final class DatabasesFlushSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesFlush.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesFlush.json */ /** * Sample code: How to flush all the keys in the database. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void howToFlushAllTheKeysInTheDatabase( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .flush( - "rg1", - "cache1", - "default", - new FlushParameters() - .withIds( - Arrays - .asList( - "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), - com.azure.core.util.Context.NONE); + public static void + howToFlushAllTheKeysInTheDatabase(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().flush("rg1", "cache1", "default", new FlushParameters().withIds(Arrays.asList( + "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), + com.azure.core.util.Context.NONE); } } ``` @@ -214,30 +189,24 @@ public final class DatabasesFlushSamples { import com.azure.resourcemanager.redisenterprise.models.ForceUnlinkParameters; import java.util.Arrays; -/** Samples for Databases ForceUnlink. */ +/** + * Samples for Databases ForceUnlink. + */ public final class DatabasesForceUnlinkSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesForceUnlink.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesForceUnlink.json */ /** * Sample code: How to unlink a database during a regional outage. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void howToUnlinkADatabaseDuringARegionalOutage( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .forceUnlink( - "rg1", - "cache1", - "default", - new ForceUnlinkParameters() - .withIds( - Arrays - .asList( - "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), - com.azure.core.util.Context.NONE); + manager.databases().forceUnlink("rg1", "cache1", "default", new ForceUnlinkParameters().withIds(Arrays.asList( + "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), + com.azure.core.util.Context.NONE); } } ``` @@ -245,18 +214,21 @@ public final class DatabasesForceUnlinkSamples { ### Databases_Get ```java -/** Samples for Databases Get. */ +/** + * Samples for Databases Get. + */ public final class DatabasesGetSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesGet.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesGet.json */ /** * Sample code: RedisEnterpriseDatabasesGet. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesGet( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseDatabasesGet(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.databases().getWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE); } } @@ -268,31 +240,26 @@ public final class DatabasesGetSamples { import com.azure.resourcemanager.redisenterprise.models.ImportClusterParameters; import java.util.Arrays; -/** Samples for Databases ImportMethod. */ +/** + * Samples for Databases ImportMethod. + */ public final class DatabasesImportMethodSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesImport.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesImport.json */ /** * Sample code: RedisEnterpriseDatabasesImport. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesImport( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .importMethod( - "rg1", - "cache1", - "default", - new ImportClusterParameters() - .withSasUris( - Arrays - .asList( - "https://contosostorage.blob.core.window.net/urltoBlobFile1?sasKeyParameters", - "https://contosostorage.blob.core.window.net/urltoBlobFile2?sasKeyParameters")), - com.azure.core.util.Context.NONE); + public static void + redisEnterpriseDatabasesImport(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().importMethod("rg1", "cache1", "default", + new ImportClusterParameters().withSasUris( + Arrays.asList("https://contosostorage.blob.core.window.net/urltoBlobFile1?sasKeyParameters", + "https://contosostorage.blob.core.window.net/urltoBlobFile2?sasKeyParameters")), + com.azure.core.util.Context.NONE); } } ``` @@ -300,14 +267,17 @@ public final class DatabasesImportMethodSamples { ### Databases_ListByCluster ```java -/** Samples for Databases ListByCluster. */ +/** + * Samples for Databases ListByCluster. + */ public final class DatabasesListByClusterSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesListByCluster.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesListByCluster.json */ /** * Sample code: RedisEnterpriseDatabasesListByCluster. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDatabasesListByCluster( @@ -320,18 +290,21 @@ public final class DatabasesListByClusterSamples { ### Databases_ListKeys ```java -/** Samples for Databases ListKeys. */ +/** + * Samples for Databases ListKeys. + */ public final class DatabasesListKeysSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesListKeys.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesListKeys.json */ /** * Sample code: RedisEnterpriseDatabasesListKeys. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesListKeys( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseDatabasesListKeys(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.databases().listKeysWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE); } } @@ -343,26 +316,23 @@ public final class DatabasesListKeysSamples { import com.azure.resourcemanager.redisenterprise.models.AccessKeyType; import com.azure.resourcemanager.redisenterprise.models.RegenerateKeyParameters; -/** Samples for Databases RegenerateKey. */ +/** + * Samples for Databases RegenerateKey. + */ public final class DatabasesRegenerateKeySamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesRegenerateKey.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesRegenerateKey.json */ /** * Sample code: RedisEnterpriseDatabasesRegenerateKey. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDatabasesRegenerateKey( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .regenerateKey( - "rg1", - "cache1", - "default", - new RegenerateKeyParameters().withKeyType(AccessKeyType.PRIMARY), - com.azure.core.util.Context.NONE); + manager.databases().regenerateKey("rg1", "cache1", "default", + new RegenerateKeyParameters().withKeyType(AccessKeyType.PRIMARY), com.azure.core.util.Context.NONE); } } ``` @@ -376,29 +346,25 @@ import com.azure.resourcemanager.redisenterprise.models.Persistence; import com.azure.resourcemanager.redisenterprise.models.Protocol; import com.azure.resourcemanager.redisenterprise.models.RdbFrequency; -/** Samples for Databases Update. */ +/** + * Samples for Databases Update. + */ public final class DatabasesUpdateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesUpdate.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesUpdate.json */ /** * Sample code: RedisEnterpriseDatabasesUpdate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesUpdate( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - Database resource = - manager - .databases() - .getWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withClientProtocol(Protocol.ENCRYPTED) - .withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) - .withPersistence(new Persistence().withRdbEnabled(true).withRdbFrequency(RdbFrequency.ONE_TWOH)) - .apply(); + public static void + redisEnterpriseDatabasesUpdate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + Database resource = manager.databases() + .getWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE).getValue(); + resource.update().withClientProtocol(Protocol.ENCRYPTED).withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) + .withPersistence(new Persistence().withRdbEnabled(true).withRdbFrequency(RdbFrequency.ONE_TWOH)).apply(); } } ``` @@ -406,14 +372,17 @@ public final class DatabasesUpdateSamples { ### Operations_List ```java -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/OperationsList.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/OperationsList.json */ /** * Sample code: OperationsList. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void operationsList(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { @@ -425,14 +394,18 @@ public final class OperationsListSamples { ### OperationsStatus_Get ```java -/** Samples for OperationsStatus Get. */ +/** + * Samples for OperationsStatus Get. + */ public final class OperationsStatusGetSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/OperationsStatusGet.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/OperationsStatusGet. + * json */ /** * Sample code: OperationsStatusGet. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void operationsStatusGet(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { @@ -444,21 +417,22 @@ public final class OperationsStatusGetSamples { ### PrivateEndpointConnections_Delete ```java -/** Samples for PrivateEndpointConnections Delete. */ +/** + * Samples for PrivateEndpointConnections Delete. + */ public final class PrivateEndpointConnectionsDeleteSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDeletePrivateEndpointConnection.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDeletePrivateEndpointConnection.json */ /** * Sample code: RedisEnterpriseDeletePrivateEndpointConnection. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDeletePrivateEndpointConnection( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .privateEndpointConnections() - .deleteWithResponse("rg1", "cache1", "pectest01", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().delete("rg1", "cache1", "pectest01", com.azure.core.util.Context.NONE); } } ``` @@ -466,21 +440,23 @@ public final class PrivateEndpointConnectionsDeleteSamples { ### PrivateEndpointConnections_Get ```java -/** Samples for PrivateEndpointConnections Get. */ +/** + * Samples for PrivateEndpointConnections Get. + */ public final class PrivateEndpointConnectionsGetSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseGetPrivateEndpointConnection.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseGetPrivateEndpointConnection.json */ /** * Sample code: RedisEnterpriseGetPrivateEndpointConnection. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseGetPrivateEndpointConnection( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .privateEndpointConnections() - .getWithResponse("rg1", "cache1", "pectest01", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().getWithResponse("rg1", "cache1", "pectest01", + com.azure.core.util.Context.NONE); } } ``` @@ -488,14 +464,17 @@ public final class PrivateEndpointConnectionsGetSamples { ### PrivateEndpointConnections_List ```java -/** Samples for PrivateEndpointConnections List. */ +/** + * Samples for PrivateEndpointConnections List. + */ public final class PrivateEndpointConnectionsListSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseListPrivateEndpointConnections.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseListPrivateEndpointConnections.json */ /** * Sample code: RedisEnterpriseListPrivateEndpointConnections. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseListPrivateEndpointConnections( @@ -511,26 +490,24 @@ public final class PrivateEndpointConnectionsListSamples { import com.azure.resourcemanager.redisenterprise.models.PrivateEndpointServiceConnectionStatus; import com.azure.resourcemanager.redisenterprise.models.PrivateLinkServiceConnectionState; -/** Samples for PrivateEndpointConnections Put. */ +/** + * Samples for PrivateEndpointConnections Put. + */ public final class PrivateEndpointConnectionsPutSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterprisePutPrivateEndpointConnection.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterprisePutPrivateEndpointConnection.json */ /** * Sample code: RedisEnterprisePutPrivateEndpointConnection. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterprisePutPrivateEndpointConnection( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .privateEndpointConnections() - .define("pectest01") - .withExistingRedisEnterprise("rg1", "cache1") - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) - .withDescription("Auto-Approved")) + manager.privateEndpointConnections().define("pectest01").withExistingRedisEnterprise("rg1", "cache1") + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED).withDescription("Auto-Approved")) .create(); } } @@ -539,14 +516,17 @@ public final class PrivateEndpointConnectionsPutSamples { ### PrivateLinkResources_ListByCluster ```java -/** Samples for PrivateLinkResources ListByCluster. */ +/** + * Samples for PrivateLinkResources ListByCluster. + */ public final class PrivateLinkResourcesListByClusterSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseListPrivateLinkResources.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseListPrivateLinkResources.json */ /** * Sample code: RedisEnterpriseListPrivateLinkResources. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseListPrivateLinkResources( @@ -573,46 +553,40 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for RedisEnterprise Create. */ +/** + * Samples for RedisEnterprise Create. + */ public final class RedisEnterpriseCreateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseCreate.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseCreate. + * json */ /** * Sample code: RedisEnterpriseCreate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseCreate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .redisEnterprises() - .define("cache1") - .withRegion("West US") - .withExistingResourceGroup("rg1") + manager.redisEnterprises().define("cache1").withRegion("West US").withExistingResourceGroup("rg1") .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F300).withCapacity(3)) - .withTags(mapOf("tag1", "value1")) - .withZones(Arrays.asList("1", "2", "3")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity", - new UserAssignedIdentity()))) + .withTags(mapOf("tag1", "value1")).withZones(Arrays.asList("1", "2", "3")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity", + new UserAssignedIdentity()))) .withMinimumTlsVersion(TlsVersion.ONE_TWO) - .withEncryption( - new ClusterPropertiesEncryption() - .withCustomerManagedKeyEncryption( - new ClusterPropertiesEncryptionCustomerManagedKeyEncryption() - .withKeyEncryptionKeyIdentity( - new ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() - .withUserAssignedIdentityResourceId( - "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity") - .withIdentityType(CmkIdentityType.USER_ASSIGNED_IDENTITY)) - .withKeyEncryptionKeyUrl("fakeTokenPlaceholder"))) + .withEncryption(new ClusterPropertiesEncryption().withCustomerManagedKeyEncryption( + new ClusterPropertiesEncryptionCustomerManagedKeyEncryption().withKeyEncryptionKeyIdentity( + new ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() + .withUserAssignedIdentityResourceId( + "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity") + .withIdentityType(CmkIdentityType.USER_ASSIGNED_IDENTITY)) + .withKeyEncryptionKeyUrl("fakeTokenPlaceholder"))) .create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -629,14 +603,18 @@ public final class RedisEnterpriseCreateSamples { ### RedisEnterprise_Delete ```java -/** Samples for RedisEnterprise Delete. */ +/** + * Samples for RedisEnterprise Delete. + */ public final class RedisEnterpriseDeleteSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDelete.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseDelete. + * json */ /** * Sample code: RedisEnterpriseDelete. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDelete(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { @@ -648,14 +626,17 @@ public final class RedisEnterpriseDeleteSamples { ### RedisEnterprise_GetByResourceGroup ```java -/** Samples for RedisEnterprise GetByResourceGroup. */ +/** + * Samples for RedisEnterprise GetByResourceGroup. + */ public final class RedisEnterpriseGetByResourceGroupSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseGet.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseGet.json */ /** * Sample code: RedisEnterpriseGet. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseGet(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { @@ -667,14 +648,18 @@ public final class RedisEnterpriseGetByResourceGroupSamples { ### RedisEnterprise_List ```java -/** Samples for RedisEnterprise List. */ +/** + * Samples for RedisEnterprise List. + */ public final class RedisEnterpriseListSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseList.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseList. + * json */ /** * Sample code: RedisEnterpriseList. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseList(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { @@ -686,18 +671,21 @@ public final class RedisEnterpriseListSamples { ### RedisEnterprise_ListByResourceGroup ```java -/** Samples for RedisEnterprise ListByResourceGroup. */ +/** + * Samples for RedisEnterprise ListByResourceGroup. + */ public final class RedisEnterpriseListByResourceGroupSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseListByResourceGroup.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseListByResourceGroup.json */ /** * Sample code: RedisEnterpriseListByResourceGroup. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseListByResourceGroup( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseListByResourceGroup(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.redisEnterprises().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); } } @@ -713,30 +701,29 @@ import com.azure.resourcemanager.redisenterprise.models.TlsVersion; import java.util.HashMap; import java.util.Map; -/** Samples for RedisEnterprise Update. */ +/** + * Samples for RedisEnterprise Update. + */ public final class RedisEnterpriseUpdateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseUpdate.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseUpdate. + * json */ /** * Sample code: RedisEnterpriseUpdate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseUpdate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - Cluster resource = - manager - .redisEnterprises() - .getByResourceGroupWithResponse("rg1", "cache1", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1")) + Cluster resource = manager.redisEnterprises() + .getByResourceGroupWithResponse("rg1", "cache1", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value1")) .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F300).withCapacity(9)) - .withMinimumTlsVersion(TlsVersion.ONE_TWO) - .apply(); + .withMinimumTlsVersion(TlsVersion.ONE_TWO).apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -750,22 +737,3 @@ public final class RedisEnterpriseUpdateSamples { } ``` -### Skus_List - -```java -/** Samples for Skus List. */ -public final class SkusListSamples { - /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseSkusList.json - */ - /** - * Sample code: SkusList. - * - * @param manager Entry point to RedisEnterpriseManager. - */ - public static void skusList(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager.skus().list("westus2", com.azure.core.util.Context.NONE); - } -} -``` - diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/pom.xml b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/pom.xml index 26b2a8f7ae0c3..996ac40dccd3c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/pom.xml +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/pom.xml @@ -14,11 +14,11 @@ com.azure.resourcemanager azure-resourcemanager-redisenterprise - 1.1.0-beta.4 + 2.1.0-beta.1 jar Microsoft Azure SDK for RedisEnterprise Management - This package contains Microsoft Azure SDK for RedisEnterprise Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. REST API for managing Redis Enterprise resources in Azure. Package tag package-preview-2023-03. + This package contains Microsoft Azure SDK for RedisEnterprise Management SDK. For documentation on how to use this package, please see https://aka.ms/azsdk/java/mgmt. REST API for managing Redis Enterprise resources in Azure. Package tag package-2023-11. https://github.com/Azure/azure-sdk-for-java @@ -45,7 +45,6 @@ UTF-8 0 0 - true @@ -58,12 +57,6 @@ azure-core-management 1.12.0 - - com.azure.resourcemanager - azure-resourcemanager-resources - 2.36.0 - test - com.azure azure-core-test @@ -73,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -94,18 +87,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -114,5 +105,11 @@ 1.7.36 test + + com.azure.resourcemanager + azure-resourcemanager-resources + 2.37.0 + test + diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/RedisEnterpriseManager.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/RedisEnterpriseManager.java index ed9b33f6b7bcf..c1d8ecb24b86d 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/RedisEnterpriseManager.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/RedisEnterpriseManager.java @@ -31,14 +31,12 @@ import com.azure.resourcemanager.redisenterprise.implementation.PrivateLinkResourcesImpl; import com.azure.resourcemanager.redisenterprise.implementation.RedisEnterpriseManagementClientBuilder; import com.azure.resourcemanager.redisenterprise.implementation.RedisEnterprisesImpl; -import com.azure.resourcemanager.redisenterprise.implementation.SkusImpl; import com.azure.resourcemanager.redisenterprise.models.Databases; import com.azure.resourcemanager.redisenterprise.models.Operations; import com.azure.resourcemanager.redisenterprise.models.OperationsStatus; import com.azure.resourcemanager.redisenterprise.models.PrivateEndpointConnections; import com.azure.resourcemanager.redisenterprise.models.PrivateLinkResources; import com.azure.resourcemanager.redisenterprise.models.RedisEnterprises; -import com.azure.resourcemanager.redisenterprise.models.Skus; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.ArrayList; @@ -46,7 +44,10 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to RedisEnterpriseManager. REST API for managing Redis Enterprise resources in Azure. */ +/** + * Entry point to RedisEnterpriseManager. + * REST API for managing Redis Enterprise resources in Azure. + */ public final class RedisEnterpriseManager { private Operations operations; @@ -60,25 +61,19 @@ public final class RedisEnterpriseManager { private PrivateLinkResources privateLinkResources; - private Skus skus; - private final RedisEnterpriseManagementClient clientObject; private RedisEnterpriseManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new RedisEnterpriseManagementClientBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new RedisEnterpriseManagementClientBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of RedisEnterprise service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the RedisEnterprise service API instance. @@ -91,7 +86,7 @@ public static RedisEnterpriseManager authenticate(TokenCredential credential, Az /** * Creates an instance of RedisEnterprise service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the RedisEnterprise service API instance. @@ -104,14 +99,16 @@ public static RedisEnterpriseManager authenticate(HttpPipeline httpPipeline, Azu /** * Gets a Configurable instance that can be used to create RedisEnterpriseManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new RedisEnterpriseManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -183,8 +180,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -201,8 +198,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -222,21 +219,12 @@ public RedisEnterpriseManager authenticate(TokenCredential credential, AzureProf Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.redisenterprise") - .append("/") - .append("1.1.0-beta.3"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.redisenterprise") + .append("/").append("2.0.0"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -255,38 +243,25 @@ public RedisEnterpriseManager authenticate(TokenCredential credential, AzureProf policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new RedisEnterpriseManager(httpPipeline, profile, defaultPollInterval); } } /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -298,7 +273,7 @@ public Operations operations() { /** * Gets the resource collection API of OperationsStatus. - * + * * @return Resource collection API of OperationsStatus. */ public OperationsStatus operationsStatus() { @@ -310,7 +285,7 @@ public OperationsStatus operationsStatus() { /** * Gets the resource collection API of RedisEnterprises. It manages Cluster. - * + * * @return Resource collection API of RedisEnterprises. */ public RedisEnterprises redisEnterprises() { @@ -322,7 +297,7 @@ public RedisEnterprises redisEnterprises() { /** * Gets the resource collection API of Databases. It manages Database. - * + * * @return Resource collection API of Databases. */ public Databases databases() { @@ -334,20 +309,20 @@ public Databases databases() { /** * Gets the resource collection API of PrivateEndpointConnections. It manages PrivateEndpointConnection. - * + * * @return Resource collection API of PrivateEndpointConnections. */ public PrivateEndpointConnections privateEndpointConnections() { if (this.privateEndpointConnections == null) { - this.privateEndpointConnections = - new PrivateEndpointConnectionsImpl(clientObject.getPrivateEndpointConnections(), this); + this.privateEndpointConnections + = new PrivateEndpointConnectionsImpl(clientObject.getPrivateEndpointConnections(), this); } return privateEndpointConnections; } /** * Gets the resource collection API of PrivateLinkResources. - * + * * @return Resource collection API of PrivateLinkResources. */ public PrivateLinkResources privateLinkResources() { @@ -358,20 +333,10 @@ public PrivateLinkResources privateLinkResources() { } /** - * Gets the resource collection API of Skus. - * - * @return Resource collection API of Skus. - */ - public Skus skus() { - if (this.skus == null) { - this.skus = new SkusImpl(clientObject.getSkus(), this); - } - return skus; - } - - /** - * @return Wrapped service client RedisEnterpriseManagementClient providing direct access to the underlying - * auto-generated API implementation, based on Azure REST API. + * Gets wrapped service client RedisEnterpriseManagementClient providing direct access to the underlying + * auto-generated API implementation, based on Azure REST API. + * + * @return Wrapped service client RedisEnterpriseManagementClient. */ public RedisEnterpriseManagementClient serviceClient() { return this.clientObject; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/DatabasesClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/DatabasesClient.java index d1aae1395eaab..d0bebe41fe497 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/DatabasesClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/DatabasesClient.java @@ -20,11 +20,13 @@ import com.azure.resourcemanager.redisenterprise.models.ImportClusterParameters; import com.azure.resourcemanager.redisenterprise.models.RegenerateKeyParameters; -/** An instance of this class provides access to all the operations defined in DatabasesClient. */ +/** + * An instance of this class provides access to all the operations defined in DatabasesClient. + */ public interface DatabasesClient { /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -37,7 +39,7 @@ public interface DatabasesClient { /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -51,7 +53,7 @@ public interface DatabasesClient { /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -62,12 +64,12 @@ public interface DatabasesClient { * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DatabaseInner> beginCreate( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters); + SyncPoller, DatabaseInner> beginCreate(String resourceGroupName, String clusterName, + String databaseName, DatabaseInner parameters); /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -79,12 +81,12 @@ SyncPoller, DatabaseInner> beginCreate( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DatabaseInner> beginCreate( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context); + SyncPoller, DatabaseInner> beginCreate(String resourceGroupName, String clusterName, + String databaseName, DatabaseInner parameters, Context context); /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -99,7 +101,7 @@ SyncPoller, DatabaseInner> beginCreate( /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -111,12 +113,12 @@ SyncPoller, DatabaseInner> beginCreate( * @return describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - DatabaseInner create( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context); + DatabaseInner create(String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, + Context context); /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -127,12 +129,12 @@ DatabaseInner create( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DatabaseInner> beginUpdate( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters); + SyncPoller, DatabaseInner> beginUpdate(String resourceGroupName, String clusterName, + String databaseName, DatabaseUpdate parameters); /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -144,12 +146,12 @@ SyncPoller, DatabaseInner> beginUpdate( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, DatabaseInner> beginUpdate( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context); + SyncPoller, DatabaseInner> beginUpdate(String resourceGroupName, String clusterName, + String databaseName, DatabaseUpdate parameters, Context context); /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -164,7 +166,7 @@ SyncPoller, DatabaseInner> beginUpdate( /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -176,12 +178,12 @@ SyncPoller, DatabaseInner> beginUpdate( * @return describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - DatabaseInner update( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context); + DatabaseInner update(String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, + Context context); /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -192,12 +194,12 @@ DatabaseInner update( * @return information about a database in a RedisEnterprise cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context); /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -211,7 +213,7 @@ Response getWithResponse( /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -225,7 +227,7 @@ Response getWithResponse( /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -236,12 +238,12 @@ Response getWithResponse( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String databaseName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, String databaseName, + Context context); /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -254,7 +256,7 @@ SyncPoller, Void> beginDelete( /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -268,7 +270,7 @@ SyncPoller, Void> beginDelete( /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -279,12 +281,12 @@ SyncPoller, Void> beginDelete( * @return access keys along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listKeysWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context); + Response listKeysWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context); /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -298,7 +300,7 @@ Response listKeysWithResponse( /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -309,12 +311,12 @@ Response listKeysWithResponse( * @return the {@link SyncPoller} for polling of access keys. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AccessKeysInner> beginRegenerateKey( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters); + SyncPoller, AccessKeysInner> beginRegenerateKey(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters); /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -326,16 +328,12 @@ SyncPoller, AccessKeysInner> beginRegenerateKey( * @return the {@link SyncPoller} for polling of access keys. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, AccessKeysInner> beginRegenerateKey( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context); + SyncPoller, AccessKeysInner> beginRegenerateKey(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters, Context context); /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -346,12 +344,12 @@ SyncPoller, AccessKeysInner> beginRegenerateKey( * @return access keys. */ @ServiceMethod(returns = ReturnType.SINGLE) - AccessKeysInner regenerateKey( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters); + AccessKeysInner regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters); /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -363,16 +361,12 @@ AccessKeysInner regenerateKey( * @return access keys. */ @ServiceMethod(returns = ReturnType.SINGLE) - AccessKeysInner regenerateKey( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context); + AccessKeysInner regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters, Context context); /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -383,12 +377,12 @@ AccessKeysInner regenerateKey( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginImportMethod( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters); + SyncPoller, Void> beginImportMethod(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters); /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -400,16 +394,12 @@ SyncPoller, Void> beginImportMethod( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginImportMethod( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context); + SyncPoller, Void> beginImportMethod(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters, Context context); /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -419,12 +409,12 @@ SyncPoller, Void> beginImportMethod( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void importMethod( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters); + void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters); /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -435,16 +425,12 @@ void importMethod( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void importMethod( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context); + void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters, Context context); /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -455,12 +441,12 @@ void importMethod( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginExport( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters); + SyncPoller, Void> beginExport(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters); /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -472,16 +458,12 @@ SyncPoller, Void> beginExport( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginExport( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context); + SyncPoller, Void> beginExport(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters, Context context); /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -495,7 +477,7 @@ SyncPoller, Void> beginExport( /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -506,16 +488,12 @@ SyncPoller, Void> beginExport( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void export( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, + void export(String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters, Context context); /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -526,12 +504,12 @@ void export( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginForceUnlink( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters); + SyncPoller, Void> beginForceUnlink(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters); /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -543,16 +521,12 @@ SyncPoller, Void> beginForceUnlink( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginForceUnlink( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context); + SyncPoller, Void> beginForceUnlink(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters, Context context); /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -562,12 +536,12 @@ SyncPoller, Void> beginForceUnlink( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void forceUnlink( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters); + void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters); /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -578,16 +552,12 @@ void forceUnlink( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void forceUnlink( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context); + void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters, Context context); /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -598,12 +568,12 @@ void forceUnlink( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginFlush( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters); + SyncPoller, Void> beginFlush(String resourceGroupName, String clusterName, String databaseName, + FlushParameters parameters); /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -615,12 +585,12 @@ SyncPoller, Void> beginFlush( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginFlush( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, Context context); + SyncPoller, Void> beginFlush(String resourceGroupName, String clusterName, String databaseName, + FlushParameters parameters, Context context); /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -634,7 +604,7 @@ SyncPoller, Void> beginFlush( /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -645,6 +615,6 @@ SyncPoller, Void> beginFlush( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void flush( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, Context context); + void flush(String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, + Context context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsClient.java index 9e09dae539eb5..1e0faa8ee6b09 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsClient.java @@ -10,28 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsStatusClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsStatusClient.java index e45462b06ebb0..ff0d88d8857dc 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsStatusClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/OperationsStatusClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.OperationStatusInner; -/** An instance of this class provides access to all the operations defined in OperationsStatusClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsStatusClient. + */ public interface OperationsStatusClient { /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @param context The context to associate with this operation. @@ -28,7 +30,7 @@ public interface OperationsStatusClient { /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateEndpointConnectionsClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateEndpointConnectionsClient.java index b54a00a48fc67..9a21027ce0ead 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateEndpointConnectionsClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateEndpointConnectionsClient.java @@ -13,25 +13,27 @@ import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner; -/** An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. + */ public interface PrivateEndpointConnectionsClient { /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String resourceGroupName, String clusterName); /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -39,52 +41,52 @@ public interface PrivateEndpointConnectionsClient { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String resourceGroupName, String clusterName, Context context); /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context); /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified private endpoint connection associated with the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - PrivateEndpointConnectionInner get( - String resourceGroupName, String clusterName, String privateEndpointConnectionName); + PrivateEndpointConnectionInner get(String resourceGroupName, String clusterName, + String privateEndpointConnectionName); /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -93,18 +95,16 @@ PrivateEndpointConnectionInner get( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PrivateEndpointConnectionInner> beginPut( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, + String resourceGroupName, String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner properties); /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -114,19 +114,16 @@ SyncPoller, PrivateEndpointConnection */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) SyncPoller, PrivateEndpointConnectionInner> beginPut( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context); + String resourceGroupName, String clusterName, String privateEndpointConnectionName, + PrivateEndpointConnectionInner properties, Context context); /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -134,19 +131,16 @@ SyncPoller, PrivateEndpointConnection * @return the Private Endpoint Connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PrivateEndpointConnectionInner put( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties); + PrivateEndpointConnectionInner put(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties); /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -155,41 +149,68 @@ PrivateEndpointConnectionInner put( * @return the Private Endpoint Connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - PrivateEndpointConnectionInner put( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context); + PrivateEndpointConnectionInner put(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties, Context context); /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName); + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. + * @return the {@link SyncPoller} for polling of long-running operation. */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context); /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName); + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateLinkResourcesClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateLinkResourcesClient.java index 6b488e78a6421..e256337f4c4e5 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateLinkResourcesClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/PrivateLinkResourcesClient.java @@ -10,25 +10,27 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceInner; -/** An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. + */ public interface PrivateLinkResourcesClient { /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByCluster(String resourceGroupName, String clusterName); /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -36,9 +38,9 @@ public interface PrivateLinkResourcesClient { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context); + PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterpriseManagementClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterpriseManagementClient.java index e1e773db6a070..16a20e4557d41 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterpriseManagementClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterpriseManagementClient.java @@ -7,89 +7,84 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for RedisEnterpriseManagementClient class. */ +/** + * The interface for RedisEnterpriseManagementClient class. + */ public interface RedisEnterpriseManagementClient { /** * Gets The ID of the target subscription. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets Api Version. - * + * * @return the apiVersion value. */ String getApiVersion(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ OperationsClient getOperations(); /** * Gets the OperationsStatusClient object to access its operations. - * + * * @return the OperationsStatusClient object. */ OperationsStatusClient getOperationsStatus(); /** * Gets the RedisEnterprisesClient object to access its operations. - * + * * @return the RedisEnterprisesClient object. */ RedisEnterprisesClient getRedisEnterprises(); /** * Gets the DatabasesClient object to access its operations. - * + * * @return the DatabasesClient object. */ DatabasesClient getDatabases(); /** * Gets the PrivateEndpointConnectionsClient object to access its operations. - * + * * @return the PrivateEndpointConnectionsClient object. */ PrivateEndpointConnectionsClient getPrivateEndpointConnections(); /** * Gets the PrivateLinkResourcesClient object to access its operations. - * + * * @return the PrivateLinkResourcesClient object. */ PrivateLinkResourcesClient getPrivateLinkResources(); - - /** - * Gets the SkusClient object to access its operations. - * - * @return the SkusClient object. - */ - SkusClient getSkus(); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterprisesClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterprisesClient.java index 3f8e4c243f489..21f9da532407c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterprisesClient.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/RedisEnterprisesClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner; import com.azure.resourcemanager.redisenterprise.models.ClusterUpdate; -/** An instance of this class provides access to all the operations defined in RedisEnterprisesClient. */ +/** + * An instance of this class provides access to all the operations defined in RedisEnterprisesClient. + */ public interface RedisEnterprisesClient { /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -28,12 +30,12 @@ public interface RedisEnterprisesClient { * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters); + SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters); /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -44,12 +46,12 @@ SyncPoller, ClusterInner> beginCreate( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context); + SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters, Context context); /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -63,7 +65,7 @@ SyncPoller, ClusterInner> beginCreate( /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -78,7 +80,7 @@ SyncPoller, ClusterInner> beginCreate( /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -88,12 +90,12 @@ SyncPoller, ClusterInner> beginCreate( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterUpdate parameters); + SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterUpdate parameters); /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -104,12 +106,12 @@ SyncPoller, ClusterInner> beginUpdate( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterUpdate parameters, Context context); + SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterUpdate parameters, Context context); /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -123,7 +125,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -138,7 +140,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -151,7 +153,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -165,7 +167,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -177,7 +179,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -190,7 +192,7 @@ SyncPoller, ClusterInner> beginUpdate( /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -200,12 +202,12 @@ SyncPoller, ClusterInner> beginUpdate( * @return information about a RedisEnterprise cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String clusterName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, + Context context); /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -218,7 +220,7 @@ Response getByResourceGroupWithResponse( /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -230,7 +232,7 @@ Response getByResourceGroupWithResponse( /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -243,24 +245,24 @@ Response getByResourceGroupWithResponse( /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link - * PagedIterable}. + * @return all RedisEnterprise clusters in the specified subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link - * PagedIterable}. + * @return all RedisEnterprise clusters in the specified subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/SkusClient.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/SkusClient.java deleted file mode 100644 index 98d77370a8749..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/SkusClient.java +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.util.Context; -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; - -/** An instance of this class provides access to all the operations defined in SkusClient. */ -public interface SkusClient { - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String location); - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String location, Context context); -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/AccessKeysInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/AccessKeysInner.java index 35d134e1b1243..53a8268b2bd08 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/AccessKeysInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/AccessKeysInner.java @@ -9,8 +9,8 @@ /** * Access keys - * - *

The secret access keys used for authenticating connections to redis. + * + * The secret access keys used for authenticating connections to redis. */ @Immutable public final class AccessKeysInner { @@ -26,13 +26,15 @@ public final class AccessKeysInner { @JsonProperty(value = "secondaryKey", access = JsonProperty.Access.WRITE_ONLY) private String secondaryKey; - /** Creates an instance of AccessKeysInner class. */ + /** + * Creates an instance of AccessKeysInner class. + */ public AccessKeysInner() { } /** * Get the primaryKey property: The current primary key that clients can use to authenticate. - * + * * @return the primaryKey value. */ public String primaryKey() { @@ -41,7 +43,7 @@ public String primaryKey() { /** * Get the secondaryKey property: The current secondary key that clients can use to authenticate. - * + * * @return the secondaryKey value. */ public String secondaryKey() { @@ -50,7 +52,7 @@ public String secondaryKey() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterInner.java index 5124741e5fae8..9c5eb4efebc40 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterInner.java @@ -6,7 +6,6 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.Resource; -import com.azure.core.management.SystemData; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryption; import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentity; @@ -18,7 +17,9 @@ import java.util.List; import java.util.Map; -/** Describes the RedisEnterprise cluster. */ +/** + * Describes the RedisEnterprise cluster. + */ @Fluent public final class ClusterInner extends Resource { /* @@ -41,25 +42,21 @@ public final class ClusterInner extends Resource { /* * RedisEnterprise cluster properties - * + * * Other properties of the cluster. */ @JsonProperty(value = "properties") private ClusterProperties innerProperties; - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. + /** + * Creates an instance of ClusterInner class. */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** Creates an instance of ClusterInner class. */ public ClusterInner() { } /** * Get the sku property: The SKU to create, which affects price, performance, and features. - * + * * @return the sku value. */ public Sku sku() { @@ -68,7 +65,7 @@ public Sku sku() { /** * Set the sku property: The SKU to create, which affects price, performance, and features. - * + * * @param sku the sku value to set. * @return the ClusterInner object itself. */ @@ -79,7 +76,7 @@ public ClusterInner withSku(Sku sku) { /** * Get the zones property: The Availability Zones where this cluster will be deployed. - * + * * @return the zones value. */ public List zones() { @@ -88,7 +85,7 @@ public List zones() { /** * Set the zones property: The Availability Zones where this cluster will be deployed. - * + * * @param zones the zones value to set. * @return the ClusterInner object itself. */ @@ -99,7 +96,7 @@ public ClusterInner withZones(List zones) { /** * Get the identity property: The identity of the resource. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -108,7 +105,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The identity of the resource. - * + * * @param identity the identity value to set. * @return the ClusterInner object itself. */ @@ -119,9 +116,9 @@ public ClusterInner withIdentity(ManagedServiceIdentity identity) { /** * Get the innerProperties property: RedisEnterprise cluster properties - * - *

Other properties of the cluster. - * + * + * Other properties of the cluster. + * * @return the innerProperties value. */ private ClusterProperties innerProperties() { @@ -129,22 +126,17 @@ private ClusterProperties innerProperties() { } /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. + * {@inheritDoc} */ - public SystemData systemData() { - return this.systemData; - } - - /** {@inheritDoc} */ @Override public ClusterInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ClusterInner withTags(Map tags) { super.withTags(tags); @@ -153,7 +145,7 @@ public ClusterInner withTags(Map tags) { /** * Get the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @return the minimumTlsVersion value. */ public TlsVersion minimumTlsVersion() { @@ -162,7 +154,7 @@ public TlsVersion minimumTlsVersion() { /** * Set the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @param minimumTlsVersion the minimumTlsVersion value to set. * @return the ClusterInner object itself. */ @@ -176,7 +168,7 @@ public ClusterInner withMinimumTlsVersion(TlsVersion minimumTlsVersion) { /** * Get the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @return the encryption value. */ public ClusterPropertiesEncryption encryption() { @@ -185,7 +177,7 @@ public ClusterPropertiesEncryption encryption() { /** * Set the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @param encryption the encryption value to set. * @return the ClusterInner object itself. */ @@ -199,7 +191,7 @@ public ClusterInner withEncryption(ClusterPropertiesEncryption encryption) { /** * Get the hostname property: DNS name of the cluster endpoint. - * + * * @return the hostname value. */ public String hostname() { @@ -208,7 +200,7 @@ public String hostname() { /** * Get the provisioningState property: Current provisioning status of the cluster. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -217,7 +209,7 @@ public ProvisioningState provisioningState() { /** * Get the resourceState property: Current resource status of the cluster. - * + * * @return the resourceState value. */ public ResourceState resourceState() { @@ -226,7 +218,7 @@ public ResourceState resourceState() { /** * Get the redisVersion property: Version of redis the cluster supports, e.g. '6'. - * + * * @return the redisVersion value. */ public String redisVersion() { @@ -236,7 +228,7 @@ public String redisVersion() { /** * Get the privateEndpointConnections property: List of private endpoint connections associated with the specified * RedisEnterprise cluster. - * + * * @return the privateEndpointConnections value. */ public List privateEndpointConnections() { @@ -245,14 +237,13 @@ public List privateEndpointConnections() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sku() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property sku in model ClusterInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sku in model ClusterInner")); } else { sku().validate(); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterProperties.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterProperties.java index 4a5059d7521bb..6981f20eecd18 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterProperties.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/ClusterProperties.java @@ -14,8 +14,8 @@ /** * RedisEnterprise cluster properties - * - *

Properties of RedisEnterprise clusters, as opposed to general resource properties like location, tags. + * + * Properties of RedisEnterprise clusters, as opposed to general resource properties like location, tags. */ @Fluent public final class ClusterProperties { @@ -61,13 +61,15 @@ public final class ClusterProperties { @JsonProperty(value = "privateEndpointConnections", access = JsonProperty.Access.WRITE_ONLY) private List privateEndpointConnections; - /** Creates an instance of ClusterProperties class. */ + /** + * Creates an instance of ClusterProperties class. + */ public ClusterProperties() { } /** * Get the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @return the minimumTlsVersion value. */ public TlsVersion minimumTlsVersion() { @@ -76,7 +78,7 @@ public TlsVersion minimumTlsVersion() { /** * Set the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @param minimumTlsVersion the minimumTlsVersion value to set. * @return the ClusterProperties object itself. */ @@ -87,7 +89,7 @@ public ClusterProperties withMinimumTlsVersion(TlsVersion minimumTlsVersion) { /** * Get the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @return the encryption value. */ public ClusterPropertiesEncryption encryption() { @@ -96,7 +98,7 @@ public ClusterPropertiesEncryption encryption() { /** * Set the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @param encryption the encryption value to set. * @return the ClusterProperties object itself. */ @@ -107,7 +109,7 @@ public ClusterProperties withEncryption(ClusterPropertiesEncryption encryption) /** * Get the hostname property: DNS name of the cluster endpoint. - * + * * @return the hostname value. */ public String hostname() { @@ -116,7 +118,7 @@ public String hostname() { /** * Get the provisioningState property: Current provisioning status of the cluster. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -125,7 +127,7 @@ public ProvisioningState provisioningState() { /** * Get the resourceState property: Current resource status of the cluster. - * + * * @return the resourceState value. */ public ResourceState resourceState() { @@ -134,7 +136,7 @@ public ResourceState resourceState() { /** * Get the redisVersion property: Version of redis the cluster supports, e.g. '6'. - * + * * @return the redisVersion value. */ public String redisVersion() { @@ -144,7 +146,7 @@ public String redisVersion() { /** * Get the privateEndpointConnections property: List of private endpoint connections associated with the specified * RedisEnterprise cluster. - * + * * @return the privateEndpointConnections value. */ public List privateEndpointConnections() { @@ -153,7 +155,7 @@ public List privateEndpointConnections() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseInner.java index e3682443bb984..40e0f1cf0cef0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseInner.java @@ -6,7 +6,6 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; import com.azure.resourcemanager.redisenterprise.models.ClusteringPolicy; import com.azure.resourcemanager.redisenterprise.models.DatabasePropertiesGeoReplication; import com.azure.resourcemanager.redisenterprise.models.EvictionPolicy; @@ -18,51 +17,40 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes a database on the RedisEnterprise cluster. */ +/** + * Describes a database on the RedisEnterprise cluster. + */ @Fluent public final class DatabaseInner extends ProxyResource { /* * RedisEnterprise database properties - * + * * Other properties of the database. */ @JsonProperty(value = "properties") private DatabaseProperties innerProperties; - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. + /** + * Creates an instance of DatabaseInner class. */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** Creates an instance of DatabaseInner class. */ public DatabaseInner() { } /** * Get the innerProperties property: RedisEnterprise database properties - * - *

Other properties of the database. - * + * + * Other properties of the database. + * * @return the innerProperties value. */ private DatabaseProperties innerProperties() { return this.innerProperties; } - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - /** * Get the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @return the clientProtocol value. */ public Protocol clientProtocol() { @@ -72,7 +60,7 @@ public Protocol clientProtocol() { /** * Set the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @param clientProtocol the clientProtocol value to set. * @return the DatabaseInner object itself. */ @@ -87,7 +75,7 @@ public DatabaseInner withClientProtocol(Protocol clientProtocol) { /** * Get the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @return the port value. */ public Integer port() { @@ -97,7 +85,7 @@ public Integer port() { /** * Set the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @param port the port value to set. * @return the DatabaseInner object itself. */ @@ -111,7 +99,7 @@ public DatabaseInner withPort(Integer port) { /** * Get the provisioningState property: Current provisioning status of the database. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -120,7 +108,7 @@ public ProvisioningState provisioningState() { /** * Get the resourceState property: Current resource status of the database. - * + * * @return the resourceState value. */ public ResourceState resourceState() { @@ -129,7 +117,7 @@ public ResourceState resourceState() { /** * Get the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @return the clusteringPolicy value. */ public ClusteringPolicy clusteringPolicy() { @@ -138,7 +126,7 @@ public ClusteringPolicy clusteringPolicy() { /** * Set the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @param clusteringPolicy the clusteringPolicy value to set. * @return the DatabaseInner object itself. */ @@ -152,7 +140,7 @@ public DatabaseInner withClusteringPolicy(ClusteringPolicy clusteringPolicy) { /** * Get the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @return the evictionPolicy value. */ public EvictionPolicy evictionPolicy() { @@ -161,7 +149,7 @@ public EvictionPolicy evictionPolicy() { /** * Set the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @param evictionPolicy the evictionPolicy value to set. * @return the DatabaseInner object itself. */ @@ -175,7 +163,7 @@ public DatabaseInner withEvictionPolicy(EvictionPolicy evictionPolicy) { /** * Get the persistence property: Persistence settings. - * + * * @return the persistence value. */ public Persistence persistence() { @@ -184,7 +172,7 @@ public Persistence persistence() { /** * Set the persistence property: Persistence settings. - * + * * @param persistence the persistence value to set. * @return the DatabaseInner object itself. */ @@ -197,9 +185,9 @@ public DatabaseInner withPersistence(Persistence persistence) { } /** - * Get the modules property: Optional set of redis modules to enable in this database - modules can only be added at - * creation time. - * + * Get the modules property: Optional set of redis modules to enable in this database - modules can only be added + * at creation time. + * * @return the modules value. */ public List modules() { @@ -207,9 +195,9 @@ public List modules() { } /** - * Set the modules property: Optional set of redis modules to enable in this database - modules can only be added at - * creation time. - * + * Set the modules property: Optional set of redis modules to enable in this database - modules can only be added + * at creation time. + * * @param modules the modules value to set. * @return the DatabaseInner object itself. */ @@ -223,7 +211,7 @@ public DatabaseInner withModules(List modules) { /** * Get the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @return the geoReplication value. */ public DatabasePropertiesGeoReplication geoReplication() { @@ -232,7 +220,7 @@ public DatabasePropertiesGeoReplication geoReplication() { /** * Set the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @param geoReplication the geoReplication value to set. * @return the DatabaseInner object itself. */ @@ -246,7 +234,7 @@ public DatabaseInner withGeoReplication(DatabasePropertiesGeoReplication geoRepl /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseProperties.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseProperties.java index 41e9bed19b120..f4dae4e18578b 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseProperties.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/DatabaseProperties.java @@ -18,8 +18,8 @@ /** * RedisEnterprise database properties - * - *

Properties of RedisEnterprise databases, as opposed to general resource properties like location, tags. + * + * Properties of RedisEnterprise databases, as opposed to general resource properties like location, tags. */ @Fluent public final class DatabaseProperties { @@ -78,14 +78,16 @@ public final class DatabaseProperties { @JsonProperty(value = "geoReplication") private DatabasePropertiesGeoReplication geoReplication; - /** Creates an instance of DatabaseProperties class. */ + /** + * Creates an instance of DatabaseProperties class. + */ public DatabaseProperties() { } /** * Get the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @return the clientProtocol value. */ public Protocol clientProtocol() { @@ -95,7 +97,7 @@ public Protocol clientProtocol() { /** * Set the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @param clientProtocol the clientProtocol value to set. * @return the DatabaseProperties object itself. */ @@ -107,7 +109,7 @@ public DatabaseProperties withClientProtocol(Protocol clientProtocol) { /** * Get the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @return the port value. */ public Integer port() { @@ -117,7 +119,7 @@ public Integer port() { /** * Set the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @param port the port value to set. * @return the DatabaseProperties object itself. */ @@ -128,7 +130,7 @@ public DatabaseProperties withPort(Integer port) { /** * Get the provisioningState property: Current provisioning status of the database. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -137,7 +139,7 @@ public ProvisioningState provisioningState() { /** * Get the resourceState property: Current resource status of the database. - * + * * @return the resourceState value. */ public ResourceState resourceState() { @@ -146,7 +148,7 @@ public ResourceState resourceState() { /** * Get the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @return the clusteringPolicy value. */ public ClusteringPolicy clusteringPolicy() { @@ -155,7 +157,7 @@ public ClusteringPolicy clusteringPolicy() { /** * Set the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @param clusteringPolicy the clusteringPolicy value to set. * @return the DatabaseProperties object itself. */ @@ -166,7 +168,7 @@ public DatabaseProperties withClusteringPolicy(ClusteringPolicy clusteringPolicy /** * Get the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @return the evictionPolicy value. */ public EvictionPolicy evictionPolicy() { @@ -175,7 +177,7 @@ public EvictionPolicy evictionPolicy() { /** * Set the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @param evictionPolicy the evictionPolicy value to set. * @return the DatabaseProperties object itself. */ @@ -186,7 +188,7 @@ public DatabaseProperties withEvictionPolicy(EvictionPolicy evictionPolicy) { /** * Get the persistence property: Persistence settings. - * + * * @return the persistence value. */ public Persistence persistence() { @@ -195,7 +197,7 @@ public Persistence persistence() { /** * Set the persistence property: Persistence settings. - * + * * @param persistence the persistence value to set. * @return the DatabaseProperties object itself. */ @@ -205,9 +207,9 @@ public DatabaseProperties withPersistence(Persistence persistence) { } /** - * Get the modules property: Optional set of redis modules to enable in this database - modules can only be added at - * creation time. - * + * Get the modules property: Optional set of redis modules to enable in this database - modules can only be added + * at creation time. + * * @return the modules value. */ public List modules() { @@ -215,9 +217,9 @@ public List modules() { } /** - * Set the modules property: Optional set of redis modules to enable in this database - modules can only be added at - * creation time. - * + * Set the modules property: Optional set of redis modules to enable in this database - modules can only be added + * at creation time. + * * @param modules the modules value to set. * @return the DatabaseProperties object itself. */ @@ -228,7 +230,7 @@ public DatabaseProperties withModules(List modules) { /** * Get the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @return the geoReplication value. */ public DatabasePropertiesGeoReplication geoReplication() { @@ -237,7 +239,7 @@ public DatabasePropertiesGeoReplication geoReplication() { /** * Set the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @param geoReplication the geoReplication value to set. * @return the DatabaseProperties object itself. */ @@ -248,7 +250,7 @@ public DatabaseProperties withGeoReplication(DatabasePropertiesGeoReplication ge /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationInner.java index 679f57b6c954e..f10d075c614f6 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationInner.java @@ -12,8 +12,8 @@ /** * REST API Operation - * - *

Details of a REST API operation, returned from the Resource Provider Operations API. + * + * Details of a REST API operation, returned from the Resource Provider Operations API. */ @Fluent public final class OperationInner { @@ -50,14 +50,16 @@ public final class OperationInner { @JsonProperty(value = "actionType", access = JsonProperty.Access.WRITE_ONLY) private ActionType actionType; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ public String name() { @@ -67,7 +69,7 @@ public String name() { /** * Get the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ public Boolean isDataAction() { @@ -76,7 +78,7 @@ public Boolean isDataAction() { /** * Get the display property: Localized display information for this particular operation. - * + * * @return the display value. */ public OperationDisplay display() { @@ -85,7 +87,7 @@ public OperationDisplay display() { /** * Set the display property: Localized display information for this particular operation. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -97,7 +99,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Get the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ public Origin origin() { @@ -107,7 +109,7 @@ public Origin origin() { /** * Get the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ public ActionType actionType() { @@ -116,7 +118,7 @@ public ActionType actionType() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationStatusInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationStatusInner.java index 0f2c8fa71b65d..1c4173fe170cf 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationStatusInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/OperationStatusInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.exception.ManagementError; import com.fasterxml.jackson.annotation.JsonProperty; -/** The status of a long-running operation. */ +/** + * The status of a long-running operation. + */ @Fluent public final class OperationStatusInner { /* @@ -43,19 +45,21 @@ public final class OperationStatusInner { /* * Error response - * + * * Error response describing why the operation failed. */ @JsonProperty(value = "error") private ManagementError error; - /** Creates an instance of OperationStatusInner class. */ + /** + * Creates an instance of OperationStatusInner class. + */ public OperationStatusInner() { } /** * Get the id property: The operation's unique id. - * + * * @return the id value. */ public String id() { @@ -64,7 +68,7 @@ public String id() { /** * Set the id property: The operation's unique id. - * + * * @param id the id value to set. * @return the OperationStatusInner object itself. */ @@ -75,7 +79,7 @@ public OperationStatusInner withId(String id) { /** * Get the name property: The operation's name. - * + * * @return the name value. */ public String name() { @@ -84,7 +88,7 @@ public String name() { /** * Set the name property: The operation's name. - * + * * @param name the name value to set. * @return the OperationStatusInner object itself. */ @@ -95,7 +99,7 @@ public OperationStatusInner withName(String name) { /** * Get the startTime property: The start time of the operation. - * + * * @return the startTime value. */ public String startTime() { @@ -104,7 +108,7 @@ public String startTime() { /** * Set the startTime property: The start time of the operation. - * + * * @param startTime the startTime value to set. * @return the OperationStatusInner object itself. */ @@ -115,7 +119,7 @@ public OperationStatusInner withStartTime(String startTime) { /** * Get the endTime property: The end time of the operation. - * + * * @return the endTime value. */ public String endTime() { @@ -124,7 +128,7 @@ public String endTime() { /** * Set the endTime property: The end time of the operation. - * + * * @param endTime the endTime value to set. * @return the OperationStatusInner object itself. */ @@ -135,7 +139,7 @@ public OperationStatusInner withEndTime(String endTime) { /** * Get the status property: The current status of the operation. - * + * * @return the status value. */ public String status() { @@ -144,7 +148,7 @@ public String status() { /** * Set the status property: The current status of the operation. - * + * * @param status the status value to set. * @return the OperationStatusInner object itself. */ @@ -155,9 +159,9 @@ public OperationStatusInner withStatus(String status) { /** * Get the error property: Error response - * - *

Error response describing why the operation failed. - * + * + * Error response describing why the operation failed. + * * @return the error value. */ public ManagementError error() { @@ -166,9 +170,9 @@ public ManagementError error() { /** * Set the error property: Error response - * - *

Error response describing why the operation failed. - * + * + * Error response describing why the operation failed. + * * @param error the error value to set. * @return the OperationStatusInner object itself. */ @@ -179,7 +183,7 @@ public OperationStatusInner withError(ManagementError error) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionInner.java index af62758cf6aec..5b58b67e6aa88 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionInner.java @@ -6,13 +6,14 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; import com.azure.resourcemanager.redisenterprise.models.PrivateEndpoint; import com.azure.resourcemanager.redisenterprise.models.PrivateEndpointConnectionProvisioningState; import com.azure.resourcemanager.redisenterprise.models.PrivateLinkServiceConnectionState; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Private Endpoint Connection resource. */ +/** + * The Private Endpoint Connection resource. + */ @Fluent public final class PrivateEndpointConnectionInner extends ProxyResource { /* @@ -21,37 +22,24 @@ public final class PrivateEndpointConnectionInner extends ProxyResource { @JsonProperty(value = "properties") private PrivateEndpointConnectionProperties innerProperties; - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. + /** + * Creates an instance of PrivateEndpointConnectionInner class. */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** Creates an instance of PrivateEndpointConnectionInner class. */ public PrivateEndpointConnectionInner() { } /** * Get the innerProperties property: Resource properties. - * + * * @return the innerProperties value. */ private PrivateEndpointConnectionProperties innerProperties() { return this.innerProperties; } - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - /** * Get the privateEndpoint property: The resource of private end point. - * + * * @return the privateEndpoint value. */ public PrivateEndpoint privateEndpoint() { @@ -60,7 +48,7 @@ public PrivateEndpoint privateEndpoint() { /** * Set the privateEndpoint property: The resource of private end point. - * + * * @param privateEndpoint the privateEndpoint value to set. * @return the PrivateEndpointConnectionInner object itself. */ @@ -73,9 +61,9 @@ public PrivateEndpointConnectionInner withPrivateEndpoint(PrivateEndpoint privat } /** - * Get the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Get the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @return the privateLinkServiceConnectionState value. */ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { @@ -83,14 +71,14 @@ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { } /** - * Set the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Set the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. * @return the PrivateEndpointConnectionInner object itself. */ - public PrivateEndpointConnectionInner withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionInner + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { if (this.innerProperties() == null) { this.innerProperties = new PrivateEndpointConnectionProperties(); } @@ -100,7 +88,7 @@ public PrivateEndpointConnectionInner withPrivateLinkServiceConnectionState( /** * Get the provisioningState property: The provisioning state of the private endpoint connection resource. - * + * * @return the provisioningState value. */ public PrivateEndpointConnectionProvisioningState provisioningState() { @@ -109,7 +97,7 @@ public PrivateEndpointConnectionProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionProperties.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionProperties.java index 4a0b49721c26b..570f45f5ec532 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionProperties.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateEndpointConnectionProperties.java @@ -11,7 +11,9 @@ import com.azure.resourcemanager.redisenterprise.models.PrivateLinkServiceConnectionState; import com.fasterxml.jackson.annotation.JsonProperty; -/** Properties of the PrivateEndpointConnectProperties. */ +/** + * Properties of the PrivateEndpointConnectProperties. + */ @Fluent public final class PrivateEndpointConnectionProperties { /* @@ -32,13 +34,15 @@ public final class PrivateEndpointConnectionProperties { @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) private PrivateEndpointConnectionProvisioningState provisioningState; - /** Creates an instance of PrivateEndpointConnectionProperties class. */ + /** + * Creates an instance of PrivateEndpointConnectionProperties class. + */ public PrivateEndpointConnectionProperties() { } /** * Get the privateEndpoint property: The resource of private end point. - * + * * @return the privateEndpoint value. */ public PrivateEndpoint privateEndpoint() { @@ -47,7 +51,7 @@ public PrivateEndpoint privateEndpoint() { /** * Set the privateEndpoint property: The resource of private end point. - * + * * @param privateEndpoint the privateEndpoint value to set. * @return the PrivateEndpointConnectionProperties object itself. */ @@ -57,9 +61,9 @@ public PrivateEndpointConnectionProperties withPrivateEndpoint(PrivateEndpoint p } /** - * Get the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Get the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @return the privateLinkServiceConnectionState value. */ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { @@ -67,21 +71,21 @@ public PrivateLinkServiceConnectionState privateLinkServiceConnectionState() { } /** - * Set the privateLinkServiceConnectionState property: A collection of information about the state of the connection - * between service consumer and provider. - * + * Set the privateLinkServiceConnectionState property: A collection of information about the state of the + * connection between service consumer and provider. + * * @param privateLinkServiceConnectionState the privateLinkServiceConnectionState value to set. * @return the PrivateEndpointConnectionProperties object itself. */ - public PrivateEndpointConnectionProperties withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionProperties + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { this.privateLinkServiceConnectionState = privateLinkServiceConnectionState; return this; } /** * Get the provisioningState property: The provisioning state of the private endpoint connection resource. - * + * * @return the provisioningState value. */ public PrivateEndpointConnectionProvisioningState provisioningState() { @@ -90,7 +94,7 @@ public PrivateEndpointConnectionProvisioningState provisioningState() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -98,11 +102,8 @@ public void validate() { privateEndpoint().validate(); } if (privateLinkServiceConnectionState() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property privateLinkServiceConnectionState in model" - + " PrivateEndpointConnectionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property privateLinkServiceConnectionState in model PrivateEndpointConnectionProperties")); } else { privateLinkServiceConnectionState().validate(); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceInner.java index e4db579f0bb82..9d098b7658970 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceInner.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceInner.java @@ -6,11 +6,12 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.core.management.SystemData; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A private link resource. */ +/** + * A private link resource. + */ @Fluent public final class PrivateLinkResourceInner extends ProxyResource { /* @@ -19,37 +20,24 @@ public final class PrivateLinkResourceInner extends ProxyResource { @JsonProperty(value = "properties") private PrivateLinkResourceProperties innerProperties; - /* - * Azure Resource Manager metadata containing createdBy and modifiedBy information. + /** + * Creates an instance of PrivateLinkResourceInner class. */ - @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) - private SystemData systemData; - - /** Creates an instance of PrivateLinkResourceInner class. */ public PrivateLinkResourceInner() { } /** * Get the innerProperties property: Resource properties. - * + * * @return the innerProperties value. */ private PrivateLinkResourceProperties innerProperties() { return this.innerProperties; } - /** - * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - public SystemData systemData() { - return this.systemData; - } - /** * Get the groupId property: The private link resource group id. - * + * * @return the groupId value. */ public String groupId() { @@ -58,7 +46,7 @@ public String groupId() { /** * Get the requiredMembers property: The private link resource required member names. - * + * * @return the requiredMembers value. */ public List requiredMembers() { @@ -67,7 +55,7 @@ public List requiredMembers() { /** * Get the requiredZoneNames property: The private link resource Private link DNS zone name. - * + * * @return the requiredZoneNames value. */ public List requiredZoneNames() { @@ -76,7 +64,7 @@ public List requiredZoneNames() { /** * Set the requiredZoneNames property: The private link resource Private link DNS zone name. - * + * * @param requiredZoneNames the requiredZoneNames value to set. * @return the PrivateLinkResourceInner object itself. */ @@ -90,7 +78,7 @@ public PrivateLinkResourceInner withRequiredZoneNames(List requiredZoneN /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceProperties.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceProperties.java index 5b134967b42d8..13545d6bd7ec8 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceProperties.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/PrivateLinkResourceProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Properties of a private link resource. */ +/** + * Properties of a private link resource. + */ @Fluent public final class PrivateLinkResourceProperties { /* @@ -29,13 +31,15 @@ public final class PrivateLinkResourceProperties { @JsonProperty(value = "requiredZoneNames") private List requiredZoneNames; - /** Creates an instance of PrivateLinkResourceProperties class. */ + /** + * Creates an instance of PrivateLinkResourceProperties class. + */ public PrivateLinkResourceProperties() { } /** * Get the groupId property: The private link resource group id. - * + * * @return the groupId value. */ public String groupId() { @@ -44,7 +48,7 @@ public String groupId() { /** * Get the requiredMembers property: The private link resource required member names. - * + * * @return the requiredMembers value. */ public List requiredMembers() { @@ -53,7 +57,7 @@ public List requiredMembers() { /** * Get the requiredZoneNames property: The private link resource Private link DNS zone name. - * + * * @return the requiredZoneNames value. */ public List requiredZoneNames() { @@ -62,7 +66,7 @@ public List requiredZoneNames() { /** * Set the requiredZoneNames property: The private link resource Private link DNS zone name. - * + * * @param requiredZoneNames the requiredZoneNames value to set. * @return the PrivateLinkResourceProperties object itself. */ @@ -73,7 +77,7 @@ public PrivateLinkResourceProperties withRequiredZoneNames(List required /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/RegionSkuDetailInner.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/RegionSkuDetailInner.java deleted file mode 100644 index 4b42bfc2cf4f1..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/RegionSkuDetailInner.java +++ /dev/null @@ -1,120 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.redisenterprise.models.LocationInfo; -import com.azure.resourcemanager.redisenterprise.models.SkuDetail; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Region Sku Detail - * - *

Details about the location requested and the available skus in the location. - */ -@Fluent -public final class RegionSkuDetailInner { - /* - * Resource type which has the SKU, such as Microsoft.Cache/redisEnterprise - */ - @JsonProperty(value = "resourceType") - private String resourceType; - - /* - * Details about location and its capabilities - */ - @JsonProperty(value = "locationInfo") - private LocationInfo locationInfo; - - /* - * Sku Detail - * - * Details about available skus - */ - @JsonProperty(value = "skuDetails") - private SkuDetail skuDetails; - - /** Creates an instance of RegionSkuDetailInner class. */ - public RegionSkuDetailInner() { - } - - /** - * Get the resourceType property: Resource type which has the SKU, such as Microsoft.Cache/redisEnterprise. - * - * @return the resourceType value. - */ - public String resourceType() { - return this.resourceType; - } - - /** - * Set the resourceType property: Resource type which has the SKU, such as Microsoft.Cache/redisEnterprise. - * - * @param resourceType the resourceType value to set. - * @return the RegionSkuDetailInner object itself. - */ - public RegionSkuDetailInner withResourceType(String resourceType) { - this.resourceType = resourceType; - return this; - } - - /** - * Get the locationInfo property: Details about location and its capabilities. - * - * @return the locationInfo value. - */ - public LocationInfo locationInfo() { - return this.locationInfo; - } - - /** - * Set the locationInfo property: Details about location and its capabilities. - * - * @param locationInfo the locationInfo value to set. - * @return the RegionSkuDetailInner object itself. - */ - public RegionSkuDetailInner withLocationInfo(LocationInfo locationInfo) { - this.locationInfo = locationInfo; - return this; - } - - /** - * Get the skuDetails property: Sku Detail - * - *

Details about available skus. - * - * @return the skuDetails value. - */ - public SkuDetail skuDetails() { - return this.skuDetails; - } - - /** - * Set the skuDetails property: Sku Detail - * - *

Details about available skus. - * - * @param skuDetails the skuDetails value to set. - * @return the RegionSkuDetailInner object itself. - */ - public RegionSkuDetailInner withSkuDetails(SkuDetail skuDetails) { - this.skuDetails = skuDetails; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (locationInfo() != null) { - locationInfo().validate(); - } - if (skuDetails() != null) { - skuDetails().validate(); - } - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/package-info.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/package-info.java index 80926ee76c2c4..c35ee78e57510 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/package-info.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the inner data models for RedisEnterpriseManagementClient. REST API for managing Redis Enterprise - * resources in Azure. + * Package containing the inner data models for RedisEnterpriseManagementClient. + * REST API for managing Redis Enterprise resources in Azure. */ package com.azure.resourcemanager.redisenterprise.fluent.models; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/package-info.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/package-info.java index 5d8e436893642..b7f1c60d0a1ed 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/package-info.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/fluent/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the service clients for RedisEnterpriseManagementClient. REST API for managing Redis Enterprise - * resources in Azure. + * Package containing the service clients for RedisEnterpriseManagementClient. + * REST API for managing Redis Enterprise resources in Azure. */ package com.azure.resourcemanager.redisenterprise.fluent; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/AccessKeysImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/AccessKeysImpl.java index 35d13f564ed3e..461b023af5554 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/AccessKeysImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/AccessKeysImpl.java @@ -12,8 +12,8 @@ public final class AccessKeysImpl implements AccessKeys { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - AccessKeysImpl( - AccessKeysInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + AccessKeysImpl(AccessKeysInner innerObject, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ClusterImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ClusterImpl.java index 39e8896e1b344..a33f5e7c0cdb4 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ClusterImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ClusterImpl.java @@ -5,7 +5,6 @@ package com.azure.resourcemanager.redisenterprise.implementation; import com.azure.core.management.Region; -import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner; @@ -70,10 +69,6 @@ public ManagedServiceIdentity identity() { return this.innerModel().identity(); } - public SystemData systemData() { - return this.innerModel().systemData(); - } - public TlsVersion minimumTlsVersion() { return this.innerModel().minimumTlsVersion(); } @@ -101,12 +96,8 @@ public String redisVersion() { public List privateEndpointConnections() { List inner = this.innerModel().privateEndpointConnections(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } @@ -144,20 +135,14 @@ public ClusterImpl withExistingResourceGroup(String resourceGroupName) { } public Cluster create() { - this.innerObject = - serviceManager - .serviceClient() - .getRedisEnterprises() - .create(resourceGroupName, clusterName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getRedisEnterprises().create(resourceGroupName, clusterName, + this.innerModel(), Context.NONE); return this; } public Cluster create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getRedisEnterprises() - .create(resourceGroupName, clusterName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getRedisEnterprises().create(resourceGroupName, clusterName, + this.innerModel(), context); return this; } @@ -173,48 +158,34 @@ public ClusterImpl update() { } public Cluster apply() { - this.innerObject = - serviceManager - .serviceClient() - .getRedisEnterprises() - .update(resourceGroupName, clusterName, updateParameters, Context.NONE); + this.innerObject = serviceManager.serviceClient().getRedisEnterprises().update(resourceGroupName, clusterName, + updateParameters, Context.NONE); return this; } public Cluster apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getRedisEnterprises() - .update(resourceGroupName, clusterName, updateParameters, context); + this.innerObject = serviceManager.serviceClient().getRedisEnterprises().update(resourceGroupName, clusterName, + updateParameters, context); return this; } - ClusterImpl( - ClusterInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + ClusterImpl(ClusterInner innerObject, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.clusterName = Utils.getValueFromIdByName(innerObject.id(), "redisEnterprise"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.clusterName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "redisEnterprise"); } public Cluster refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getRedisEnterprises() - .getByResourceGroupWithResponse(resourceGroupName, clusterName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getRedisEnterprises() + .getByResourceGroupWithResponse(resourceGroupName, clusterName, Context.NONE).getValue(); return this; } public Cluster refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getRedisEnterprises() - .getByResourceGroupWithResponse(resourceGroupName, clusterName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getRedisEnterprises() + .getByResourceGroupWithResponse(resourceGroupName, clusterName, context).getValue(); return this; } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabaseImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabaseImpl.java index 851a39d700dfd..1164c588d3244 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabaseImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabaseImpl.java @@ -5,7 +5,6 @@ package com.azure.resourcemanager.redisenterprise.implementation; import com.azure.core.http.rest.Response; -import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.DatabaseInner; import com.azure.resourcemanager.redisenterprise.models.AccessKeys; @@ -44,10 +43,6 @@ public String type() { return this.innerModel().type(); } - public SystemData systemData() { - return this.innerModel().systemData(); - } - public Protocol clientProtocol() { return this.innerModel().clientProtocol(); } @@ -116,20 +111,14 @@ public DatabaseImpl withExistingRedisEnterprise(String resourceGroupName, String } public Database create() { - this.innerObject = - serviceManager - .serviceClient() - .getDatabases() - .create(resourceGroupName, clusterName, databaseName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getDatabases().create(resourceGroupName, clusterName, + databaseName, this.innerModel(), Context.NONE); return this; } public Database create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDatabases() - .create(resourceGroupName, clusterName, databaseName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getDatabases().create(resourceGroupName, clusterName, + databaseName, this.innerModel(), context); return this; } @@ -145,49 +134,35 @@ public DatabaseImpl update() { } public Database apply() { - this.innerObject = - serviceManager - .serviceClient() - .getDatabases() - .update(resourceGroupName, clusterName, databaseName, updateParameters, Context.NONE); + this.innerObject = serviceManager.serviceClient().getDatabases().update(resourceGroupName, clusterName, + databaseName, updateParameters, Context.NONE); return this; } public Database apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDatabases() - .update(resourceGroupName, clusterName, databaseName, updateParameters, context); + this.innerObject = serviceManager.serviceClient().getDatabases().update(resourceGroupName, clusterName, + databaseName, updateParameters, context); return this; } - DatabaseImpl( - DatabaseInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + DatabaseImpl(DatabaseInner innerObject, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.clusterName = Utils.getValueFromIdByName(innerObject.id(), "redisEnterprise"); - this.databaseName = Utils.getValueFromIdByName(innerObject.id(), "databases"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.clusterName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "redisEnterprise"); + this.databaseName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "databases"); } public Database refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getDatabases() - .getWithResponse(resourceGroupName, clusterName, databaseName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDatabases() + .getWithResponse(resourceGroupName, clusterName, databaseName, Context.NONE).getValue(); return this; } public Database refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDatabases() - .getWithResponse(resourceGroupName, clusterName, databaseName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDatabases() + .getWithResponse(resourceGroupName, clusterName, databaseName, context).getValue(); return this; } @@ -204,9 +179,8 @@ public AccessKeys regenerateKey(RegenerateKeyParameters parameters) { } public AccessKeys regenerateKey(RegenerateKeyParameters parameters, Context context) { - return serviceManager - .databases() - .regenerateKey(resourceGroupName, clusterName, databaseName, parameters, context); + return serviceManager.databases().regenerateKey(resourceGroupName, clusterName, databaseName, parameters, + context); } public void importMethod(ImportClusterParameters parameters) { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesClientImpl.java index c66195ef1909b..9b7f6186e09a0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesClientImpl.java @@ -47,22 +47,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DatabasesClient. */ +/** + * An instance of this class provides access to all the operations defined in DatabasesClient. + */ public final class DatabasesClientImpl implements DatabasesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DatabasesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final RedisEnterpriseManagementClientImpl client; /** * Initializes an instance of DatabasesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ DatabasesClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = - RestProxy.create(DatabasesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(DatabasesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -73,213 +79,147 @@ public final class DatabasesClientImpl implements DatabasesClient { @Host("{$host}") @ServiceInterface(name = "RedisEnterpriseManag") public interface DatabasesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") - @ExpectedResponses({200, 201}) + Mono> listByCluster(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> create( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") DatabaseInner parameters, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") - @ExpectedResponses({200, 202}) + Mono>> create(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @BodyParam("application/json") DatabaseInner parameters, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") DatabaseUpdate parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") DatabaseUpdate parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") - @ExpectedResponses({200, 202, 204}) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/listKeys") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/listKeys") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listKeys( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/regenerateKey") - @ExpectedResponses({200, 202}) + Mono> listKeys(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/regenerateKey") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> regenerateKey( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, + Mono>> regenerateKey(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") RegenerateKeyParameters parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") RegenerateKeyParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/import") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/import") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> importMethod( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, + Mono>> importMethod(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") ImportClusterParameters parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ImportClusterParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/export") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/export") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> export( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, + Mono>> export(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") ExportClusterParameters parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ExportClusterParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/forceUnlink") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/forceUnlink") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> forceUnlink( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, + Mono>> forceUnlink(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") ForceUnlinkParameters parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ForceUnlinkParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/flush") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/databases/{databaseName}/flush") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> flush( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @PathParam("databaseName") String databaseName, - @QueryParam("api-version") String apiVersion, + Mono>> flush(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @PathParam("databaseName") String databaseName, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") FlushParameters parameters, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") FlushParameters parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByClusterNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByClusterNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all databases in the specified RedisEnterprise cluster along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -290,32 +230,16 @@ private Mono> listByClusterSinglePageAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), resourceGroupName, clusterName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -323,22 +247,18 @@ private Mono> listByClusterSinglePageAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all databases in the specified RedisEnterprise cluster along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -350,28 +270,15 @@ private Mono> listByClusterSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - resourceGroupName, - clusterName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByCluster(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), + resourceGroupName, clusterName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -381,14 +288,13 @@ private Mono> listByClusterSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName) { - return new PagedFlux<>( - () -> listByClusterSinglePageAsync(resourceGroupName, clusterName), + return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName), nextLink -> listByClusterNextSinglePageAsync(nextLink)); } /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -399,14 +305,13 @@ private PagedFlux listByClusterAsync(String resourceGroupName, St */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, Context context) { - return new PagedFlux<>( - () -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context), + return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context), nextLink -> listByClusterNextSinglePageAsync(nextLink, context)); } /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -421,7 +326,7 @@ public PagedIterable listByCluster(String resourceGroupName, Stri /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -437,7 +342,7 @@ public PagedIterable listByCluster(String resourceGroupName, Stri /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -446,16 +351,14 @@ public PagedIterable listByCluster(String resourceGroupName, Stri * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes a database on the RedisEnterprise cluster along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, DatabaseInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -468,10 +371,8 @@ private Mono>> createWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -481,24 +382,14 @@ private Mono>> createWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .create( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.create(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -508,16 +399,14 @@ private Mono>> createWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes a database on the RedisEnterprise cluster along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, DatabaseInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -530,10 +419,8 @@ private Mono>> createWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -542,22 +429,13 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.create(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -568,23 +446,17 @@ private Mono>> createWithResponseAsync( * @return the {@link PollerFlux} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DatabaseInner> beginCreateAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters) { - Mono>> mono = - createWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DatabaseInner.class, - DatabaseInner.class, - this.client.getContext()); + private PollerFlux, DatabaseInner> beginCreateAsync(String resourceGroupName, + String clusterName, String databaseName, DatabaseInner parameters) { + Mono>> mono + = createWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DatabaseInner.class, DatabaseInner.class, this.client.getContext()); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -596,20 +468,18 @@ private PollerFlux, DatabaseInner> beginCreateAsync( * @return the {@link PollerFlux} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DatabaseInner> beginCreateAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context) { + private PollerFlux, DatabaseInner> beginCreateAsync(String resourceGroupName, + String clusterName, String databaseName, DatabaseInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DatabaseInner.class, DatabaseInner.class, context); + Mono>> mono + = createWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DatabaseInner.class, DatabaseInner.class, context); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -620,14 +490,14 @@ private PollerFlux, DatabaseInner> beginCreateAsync( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DatabaseInner> beginCreate( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters) { + public SyncPoller, DatabaseInner> beginCreate(String resourceGroupName, + String clusterName, String databaseName, DatabaseInner parameters) { return this.beginCreateAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -639,14 +509,14 @@ public SyncPoller, DatabaseInner> beginCreate( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DatabaseInner> beginCreate( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context) { + public SyncPoller, DatabaseInner> beginCreate(String resourceGroupName, + String clusterName, String databaseName, DatabaseInner parameters, Context context) { return this.beginCreateAsync(resourceGroupName, clusterName, databaseName, parameters, context).getSyncPoller(); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -657,16 +527,15 @@ public SyncPoller, DatabaseInner> beginCreate( * @return describes a database on the RedisEnterprise cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters) { - return beginCreateAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono createAsync(String resourceGroupName, String clusterName, String databaseName, + DatabaseInner parameters) { + return beginCreateAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -678,16 +547,15 @@ private Mono createAsync( * @return describes a database on the RedisEnterprise cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context) { - return beginCreateAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono createAsync(String resourceGroupName, String clusterName, String databaseName, + DatabaseInner parameters, Context context) { + return beginCreateAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -698,14 +566,14 @@ private Mono createAsync( * @return describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DatabaseInner create( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters) { + public DatabaseInner create(String resourceGroupName, String clusterName, String databaseName, + DatabaseInner parameters) { return createAsync(resourceGroupName, clusterName, databaseName, parameters).block(); } /** * Creates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -717,14 +585,14 @@ public DatabaseInner create( * @return describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DatabaseInner create( - String resourceGroupName, String clusterName, String databaseName, DatabaseInner parameters, Context context) { + public DatabaseInner create(String resourceGroupName, String clusterName, String databaseName, + DatabaseInner parameters, Context context) { return createAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -733,16 +601,14 @@ public DatabaseInner create( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes a database on the RedisEnterprise cluster along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, DatabaseUpdate parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -755,10 +621,8 @@ private Mono>> updateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -768,24 +632,14 @@ private Mono>> updateWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .update( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.update(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -795,16 +649,14 @@ private Mono>> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes a database on the RedisEnterprise cluster along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, DatabaseUpdate parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -817,10 +669,8 @@ private Mono>> updateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -829,22 +679,13 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.update(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -855,23 +696,17 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DatabaseInner> beginUpdateAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters) { - Mono>> mono = - updateWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - DatabaseInner.class, - DatabaseInner.class, - this.client.getContext()); + private PollerFlux, DatabaseInner> beginUpdateAsync(String resourceGroupName, + String clusterName, String databaseName, DatabaseUpdate parameters) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DatabaseInner.class, DatabaseInner.class, this.client.getContext()); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -883,20 +718,18 @@ private PollerFlux, DatabaseInner> beginUpdateAsync( * @return the {@link PollerFlux} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, DatabaseInner> beginUpdateAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { + private PollerFlux, DatabaseInner> beginUpdateAsync(String resourceGroupName, + String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), DatabaseInner.class, DatabaseInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + DatabaseInner.class, DatabaseInner.class, context); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -907,14 +740,14 @@ private PollerFlux, DatabaseInner> beginUpdateAsync( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DatabaseInner> beginUpdate( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters) { + public SyncPoller, DatabaseInner> beginUpdate(String resourceGroupName, + String clusterName, String databaseName, DatabaseUpdate parameters) { return this.beginUpdateAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -926,14 +759,14 @@ public SyncPoller, DatabaseInner> beginUpdate( * @return the {@link SyncPoller} for polling of describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, DatabaseInner> beginUpdate( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { + public SyncPoller, DatabaseInner> beginUpdate(String resourceGroupName, + String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { return this.beginUpdateAsync(resourceGroupName, clusterName, databaseName, parameters, context).getSyncPoller(); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -944,16 +777,15 @@ public SyncPoller, DatabaseInner> beginUpdate( * @return describes a database on the RedisEnterprise cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters) { - return beginUpdateAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono updateAsync(String resourceGroupName, String clusterName, String databaseName, + DatabaseUpdate parameters) { + return beginUpdateAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -965,16 +797,15 @@ private Mono updateAsync( * @return describes a database on the RedisEnterprise cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { - return beginUpdateAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono updateAsync(String resourceGroupName, String clusterName, String databaseName, + DatabaseUpdate parameters, Context context) { + return beginUpdateAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -985,14 +816,14 @@ private Mono updateAsync( * @return describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DatabaseInner update( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters) { + public DatabaseInner update(String resourceGroupName, String clusterName, String databaseName, + DatabaseUpdate parameters) { return updateAsync(resourceGroupName, clusterName, databaseName, parameters).block(); } /** * Updates a database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1004,14 +835,14 @@ public DatabaseInner update( * @return describes a database on the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DatabaseInner update( - String resourceGroupName, String clusterName, String databaseName, DatabaseUpdate parameters, Context context) { + public DatabaseInner update(String resourceGroupName, String clusterName, String databaseName, + DatabaseUpdate parameters, Context context) { return updateAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1019,16 +850,14 @@ public DatabaseInner update( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a database in a RedisEnterprise cluster along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1041,31 +870,19 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1074,16 +891,14 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a database in a RedisEnterprise cluster along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1096,28 +911,18 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1134,7 +939,7 @@ private Mono getAsync(String resourceGroupName, String clusterNam /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1145,14 +950,14 @@ private Mono getAsync(String resourceGroupName, String clusterNam * @return information about a database in a RedisEnterprise cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context) { return getWithResponseAsync(resourceGroupName, clusterName, databaseName, context).block(); } /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1168,7 +973,7 @@ public DatabaseInner get(String resourceGroupName, String clusterName, String da /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1178,13 +983,11 @@ public DatabaseInner get(String resourceGroupName, String clusterName, String da * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1197,31 +1000,19 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, clusterName, + databaseName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1232,13 +1023,11 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1251,28 +1040,18 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1282,18 +1061,16 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String databaseName) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String databaseName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, clusterName, databaseName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1304,19 +1081,18 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, String databaseName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String databaseName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, clusterName, databaseName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, databaseName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1326,14 +1102,14 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String databaseName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String databaseName) { return this.beginDeleteAsync(resourceGroupName, clusterName, databaseName).getSyncPoller(); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1344,14 +1120,14 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, String databaseName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String databaseName, Context context) { return this.beginDeleteAsync(resourceGroupName, clusterName, databaseName, context).getSyncPoller(); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1362,14 +1138,13 @@ public SyncPoller, Void> beginDelete( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String databaseName) { - return beginDeleteAsync(resourceGroupName, clusterName, databaseName) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, databaseName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1381,14 +1156,13 @@ private Mono deleteAsync(String resourceGroupName, String clusterName, Str */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String databaseName, Context context) { - return beginDeleteAsync(resourceGroupName, clusterName, databaseName, context) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, databaseName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1403,7 +1177,7 @@ public void delete(String resourceGroupName, String clusterName, String database /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1419,7 +1193,7 @@ public void delete(String resourceGroupName, String clusterName, String database /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1429,13 +1203,11 @@ public void delete(String resourceGroupName, String clusterName, String database * @return access keys along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listKeysWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName) { + private Mono> listKeysWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1448,31 +1220,19 @@ private Mono> listKeysWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listKeys( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.listKeys(this.client.getEndpoint(), resourceGroupName, clusterName, + databaseName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1483,13 +1243,11 @@ private Mono> listKeysWithResponseAsync( * @return access keys along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listKeysWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, Context context) { + private Mono> listKeysWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1502,28 +1260,18 @@ private Mono> listKeysWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listKeys( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.listKeys(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1540,7 +1288,7 @@ private Mono listKeysAsync(String resourceGroupName, String clu /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1551,14 +1299,14 @@ private Mono listKeysAsync(String resourceGroupName, String clu * @return access keys along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listKeysWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context) { + public Response listKeysWithResponse(String resourceGroupName, String clusterName, + String databaseName, Context context) { return listKeysWithResponseAsync(resourceGroupName, clusterName, databaseName, context).block(); } /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1574,7 +1322,7 @@ public AccessKeysInner listKeys(String resourceGroupName, String clusterName, St /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1585,13 +1333,11 @@ public AccessKeysInner listKeys(String resourceGroupName, String clusterName, St * @return access keys along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> regenerateKeyWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters) { + private Mono>> regenerateKeyWithResponseAsync(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1604,10 +1350,8 @@ private Mono>> regenerateKeyWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1616,25 +1360,15 @@ private Mono>> regenerateKeyWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .regenerateKey( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.regenerateKey(this.client.getEndpoint(), resourceGroupName, clusterName, + databaseName, this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1646,17 +1380,11 @@ private Mono>> regenerateKeyWithResponseAsync( * @return access keys along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> regenerateKeyWithResponseAsync( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context) { + private Mono>> regenerateKeyWithResponseAsync(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1669,10 +1397,8 @@ private Mono>> regenerateKeyWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1681,22 +1407,13 @@ private Mono>> regenerateKeyWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .regenerateKey( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.regenerateKey(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1707,23 +1424,17 @@ private Mono>> regenerateKeyWithResponseAsync( * @return the {@link PollerFlux} for polling of access keys. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AccessKeysInner> beginRegenerateKeyAsync( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters) { - Mono>> mono = - regenerateKeyWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - AccessKeysInner.class, - AccessKeysInner.class, - this.client.getContext()); + private PollerFlux, AccessKeysInner> beginRegenerateKeyAsync(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters) { + Mono>> mono + = regenerateKeyWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + AccessKeysInner.class, AccessKeysInner.class, this.client.getContext()); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1735,24 +1446,18 @@ private PollerFlux, AccessKeysInner> beginRegenerate * @return the {@link PollerFlux} for polling of access keys. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, AccessKeysInner> beginRegenerateKeyAsync( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context) { + private PollerFlux, AccessKeysInner> beginRegenerateKeyAsync(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - regenerateKeyWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), AccessKeysInner.class, AccessKeysInner.class, context); + Mono>> mono + = regenerateKeyWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + AccessKeysInner.class, AccessKeysInner.class, context); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1763,14 +1468,14 @@ private PollerFlux, AccessKeysInner> beginRegenerate * @return the {@link SyncPoller} for polling of access keys. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AccessKeysInner> beginRegenerateKey( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters) { + public SyncPoller, AccessKeysInner> beginRegenerateKey(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters) { return this.beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1782,20 +1487,15 @@ public SyncPoller, AccessKeysInner> beginRegenerateK * @return the {@link SyncPoller} for polling of access keys. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, AccessKeysInner> beginRegenerateKey( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context) { - return this - .beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters, context) + public SyncPoller, AccessKeysInner> beginRegenerateKey(String resourceGroupName, + String clusterName, String databaseName, RegenerateKeyParameters parameters, Context context) { + return this.beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters, context) .getSyncPoller(); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1806,16 +1506,15 @@ public SyncPoller, AccessKeysInner> beginRegenerateK * @return access keys on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono regenerateKeyAsync( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters) { - return beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono regenerateKeyAsync(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters) { + return beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1827,20 +1526,15 @@ private Mono regenerateKeyAsync( * @return access keys on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono regenerateKeyAsync( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context) { - return beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono regenerateKeyAsync(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters, Context context) { + return beginRegenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1851,14 +1545,14 @@ private Mono regenerateKeyAsync( * @return access keys. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AccessKeysInner regenerateKey( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters) { + public AccessKeysInner regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters) { return regenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters).block(); } /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1870,18 +1564,14 @@ public AccessKeysInner regenerateKey( * @return access keys. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AccessKeysInner regenerateKey( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context) { + public AccessKeysInner regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters, Context context) { return regenerateKeyAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1892,13 +1582,11 @@ public AccessKeysInner regenerateKey( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> importMethodWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters) { + private Mono>> importMethodWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1911,10 +1599,8 @@ private Mono>> importMethodWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1924,24 +1610,14 @@ private Mono>> importMethodWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .importMethod( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.importMethod(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -1953,17 +1629,11 @@ private Mono>> importMethodWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> importMethodWithResponseAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context) { + private Mono>> importMethodWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1976,10 +1646,8 @@ private Mono>> importMethodWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -1988,22 +1656,13 @@ private Mono>> importMethodWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .importMethod( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.importMethod(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2014,19 +1673,17 @@ private Mono>> importMethodWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginImportMethodAsync( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters) { - Mono>> mono = - importMethodWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginImportMethodAsync(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters) { + Mono>> mono + = importMethodWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2038,23 +1695,18 @@ private PollerFlux, Void> beginImportMethodAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginImportMethodAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context) { + private PollerFlux, Void> beginImportMethodAsync(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - importMethodWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = importMethodWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2065,14 +1717,14 @@ private PollerFlux, Void> beginImportMethodAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginImportMethod( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters) { + public SyncPoller, Void> beginImportMethod(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters) { return this.beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2084,20 +1736,15 @@ public SyncPoller, Void> beginImportMethod( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginImportMethod( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context) { - return this - .beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters, context) + public SyncPoller, Void> beginImportMethod(String resourceGroupName, String clusterName, + String databaseName, ImportClusterParameters parameters, Context context) { + return this.beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters, context) .getSyncPoller(); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2108,16 +1755,15 @@ public SyncPoller, Void> beginImportMethod( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono importMethodAsync( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters) { - return beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono importMethodAsync(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters) { + return beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2129,20 +1775,15 @@ private Mono importMethodAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono importMethodAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context) { - return beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono importMethodAsync(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters, Context context) { + return beginImportMethodAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2152,14 +1793,14 @@ private Mono importMethodAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void importMethod( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters) { + public void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters) { importMethodAsync(resourceGroupName, clusterName, databaseName, parameters).block(); } /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2170,18 +1811,14 @@ public void importMethod( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void importMethod( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context) { + public void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters, Context context) { importMethodAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2192,13 +1829,11 @@ public void importMethod( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> exportWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters) { + private Mono>> exportWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, ExportClusterParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2211,10 +1846,8 @@ private Mono>> exportWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2224,24 +1857,14 @@ private Mono>> exportWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .export( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.export(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2253,17 +1876,11 @@ private Mono>> exportWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> exportWithResponseAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context) { + private Mono>> exportWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, ExportClusterParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2276,10 +1893,8 @@ private Mono>> exportWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2288,22 +1903,13 @@ private Mono>> exportWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .export( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.export(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2314,19 +1920,17 @@ private Mono>> exportWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginExportAsync( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters) { - Mono>> mono = - exportWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginExportAsync(String resourceGroupName, String clusterName, + String databaseName, ExportClusterParameters parameters) { + Mono>> mono + = exportWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2338,23 +1942,18 @@ private PollerFlux, Void> beginExportAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginExportAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context) { + private PollerFlux, Void> beginExportAsync(String resourceGroupName, String clusterName, + String databaseName, ExportClusterParameters parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - exportWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = exportWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2365,14 +1964,14 @@ private PollerFlux, Void> beginExportAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginExport( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters) { + public SyncPoller, Void> beginExport(String resourceGroupName, String clusterName, + String databaseName, ExportClusterParameters parameters) { return this.beginExportAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2384,18 +1983,14 @@ public SyncPoller, Void> beginExport( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginExport( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context) { + public SyncPoller, Void> beginExport(String resourceGroupName, String clusterName, + String databaseName, ExportClusterParameters parameters, Context context) { return this.beginExportAsync(resourceGroupName, clusterName, databaseName, parameters, context).getSyncPoller(); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2406,16 +2001,15 @@ public SyncPoller, Void> beginExport( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono exportAsync( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters) { - return beginExportAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono exportAsync(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters) { + return beginExportAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2427,20 +2021,15 @@ private Mono exportAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono exportAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context) { - return beginExportAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono exportAsync(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters, Context context) { + return beginExportAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2450,14 +2039,14 @@ private Mono exportAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void export( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters) { + public void export(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters) { exportAsync(resourceGroupName, clusterName, databaseName, parameters).block(); } /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2468,18 +2057,14 @@ public void export( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void export( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context) { + public void export(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters, Context context) { exportAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2490,13 +2075,11 @@ public void export( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> forceUnlinkWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters) { + private Mono>> forceUnlinkWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2509,10 +2092,8 @@ private Mono>> forceUnlinkWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2522,24 +2103,14 @@ private Mono>> forceUnlinkWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .forceUnlink( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.forceUnlink(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2551,17 +2122,11 @@ private Mono>> forceUnlinkWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> forceUnlinkWithResponseAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context) { + private Mono>> forceUnlinkWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2574,10 +2139,8 @@ private Mono>> forceUnlinkWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2586,22 +2149,13 @@ private Mono>> forceUnlinkWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .forceUnlink( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.forceUnlink(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2612,19 +2166,17 @@ private Mono>> forceUnlinkWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginForceUnlinkAsync( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters) { - Mono>> mono = - forceUnlinkWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginForceUnlinkAsync(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters) { + Mono>> mono + = forceUnlinkWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2636,23 +2188,18 @@ private PollerFlux, Void> beginForceUnlinkAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginForceUnlinkAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context) { + private PollerFlux, Void> beginForceUnlinkAsync(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - forceUnlinkWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = forceUnlinkWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2663,14 +2210,14 @@ private PollerFlux, Void> beginForceUnlinkAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginForceUnlink( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters) { + public SyncPoller, Void> beginForceUnlink(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters) { return this.beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2682,20 +2229,15 @@ public SyncPoller, Void> beginForceUnlink( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginForceUnlink( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context) { - return this - .beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters, context) + public SyncPoller, Void> beginForceUnlink(String resourceGroupName, String clusterName, + String databaseName, ForceUnlinkParameters parameters, Context context) { + return this.beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters, context) .getSyncPoller(); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2706,16 +2248,15 @@ public SyncPoller, Void> beginForceUnlink( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono forceUnlinkAsync( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters) { - return beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono forceUnlinkAsync(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters) { + return beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2727,20 +2268,15 @@ private Mono forceUnlinkAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono forceUnlinkAsync( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context) { - return beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono forceUnlinkAsync(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters, Context context) { + return beginForceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2750,14 +2286,14 @@ private Mono forceUnlinkAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void forceUnlink( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters) { + public void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters) { forceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters).block(); } /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2768,18 +2304,14 @@ public void forceUnlink( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void forceUnlink( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context) { + public void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters, Context context) { forceUnlinkAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2790,13 +2322,11 @@ public void forceUnlink( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> flushWithResponseAsync( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters) { + private Mono>> flushWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, FlushParameters parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2809,10 +2339,8 @@ private Mono>> flushWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2822,24 +2350,14 @@ private Mono>> flushWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .flush( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + context -> service.flush(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2851,17 +2369,11 @@ private Mono>> flushWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> flushWithResponseAsync( - String resourceGroupName, - String clusterName, - String databaseName, - FlushParameters parameters, - Context context) { + private Mono>> flushWithResponseAsync(String resourceGroupName, String clusterName, + String databaseName, FlushParameters parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2874,10 +2386,8 @@ private Mono>> flushWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter databaseName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -2886,22 +2396,13 @@ private Mono>> flushWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .flush( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - databaseName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.flush(this.client.getEndpoint(), resourceGroupName, clusterName, databaseName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2912,19 +2413,17 @@ private Mono>> flushWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginFlushAsync( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters) { - Mono>> mono = - flushWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginFlushAsync(String resourceGroupName, String clusterName, + String databaseName, FlushParameters parameters) { + Mono>> mono + = flushWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2936,23 +2435,18 @@ private PollerFlux, Void> beginFlushAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginFlushAsync( - String resourceGroupName, - String clusterName, - String databaseName, - FlushParameters parameters, - Context context) { + private PollerFlux, Void> beginFlushAsync(String resourceGroupName, String clusterName, + String databaseName, FlushParameters parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - flushWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = flushWithResponseAsync(resourceGroupName, clusterName, databaseName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2963,14 +2457,14 @@ private PollerFlux, Void> beginFlushAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginFlush( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters) { + public SyncPoller, Void> beginFlush(String resourceGroupName, String clusterName, + String databaseName, FlushParameters parameters) { return this.beginFlushAsync(resourceGroupName, clusterName, databaseName, parameters).getSyncPoller(); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -2982,18 +2476,14 @@ public SyncPoller, Void> beginFlush( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginFlush( - String resourceGroupName, - String clusterName, - String databaseName, - FlushParameters parameters, - Context context) { + public SyncPoller, Void> beginFlush(String resourceGroupName, String clusterName, + String databaseName, FlushParameters parameters, Context context) { return this.beginFlushAsync(resourceGroupName, clusterName, databaseName, parameters, context).getSyncPoller(); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -3004,16 +2494,15 @@ public SyncPoller, Void> beginFlush( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono flushAsync( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters) { - return beginFlushAsync(resourceGroupName, clusterName, databaseName, parameters) - .last() + private Mono flushAsync(String resourceGroupName, String clusterName, String databaseName, + FlushParameters parameters) { + return beginFlushAsync(resourceGroupName, clusterName, databaseName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -3025,20 +2514,15 @@ private Mono flushAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono flushAsync( - String resourceGroupName, - String clusterName, - String databaseName, - FlushParameters parameters, - Context context) { - return beginFlushAsync(resourceGroupName, clusterName, databaseName, parameters, context) - .last() + private Mono flushAsync(String resourceGroupName, String clusterName, String databaseName, + FlushParameters parameters, Context context) { + return beginFlushAsync(resourceGroupName, clusterName, databaseName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -3054,7 +2538,7 @@ public void flush(String resourceGroupName, String clusterName, String databaseN /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -3065,25 +2549,22 @@ public void flush(String resourceGroupName, String clusterName, String databaseN * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void flush( - String resourceGroupName, - String clusterName, - String databaseName, - FlushParameters parameters, + public void flush(String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, Context context) { flushAsync(resourceGroupName, clusterName, databaseName, parameters, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByClusterNextSinglePageAsync(String nextLink) { @@ -3091,37 +2572,29 @@ private Mono> listByClusterNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByClusterNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByClusterNextSinglePageAsync(String nextLink, Context context) { @@ -3129,23 +2602,13 @@ private Mono> listByClusterNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByClusterNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByClusterNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesImpl.java index a582b7f13129c..d6292d7ce14d2 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/DatabasesImpl.java @@ -28,32 +28,29 @@ public final class DatabasesImpl implements Databases { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - public DatabasesImpl( - DatabasesClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + public DatabasesImpl(DatabasesClient innerClient, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByCluster(String resourceGroupName, String clusterName) { PagedIterable inner = this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new DatabaseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DatabaseImpl(inner1, this.manager())); } public PagedIterable listByCluster(String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new DatabaseImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DatabaseImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, clusterName, databaseName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, clusterName, databaseName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DatabaseImpl(inner.getValue(), this.manager())); } else { return null; @@ -77,15 +74,12 @@ public void delete(String resourceGroupName, String clusterName, String database this.serviceClient().delete(resourceGroupName, clusterName, databaseName, context); } - public Response listKeysWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context) { - Response inner = - this.serviceClient().listKeysWithResponse(resourceGroupName, clusterName, databaseName, context); + public Response listKeysWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context) { + Response inner + = this.serviceClient().listKeysWithResponse(resourceGroupName, clusterName, databaseName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AccessKeysImpl(inner.getValue(), this.manager())); } else { return null; @@ -101,10 +95,10 @@ public AccessKeys listKeys(String resourceGroupName, String clusterName, String } } - public AccessKeys regenerateKey( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters) { - AccessKeysInner inner = - this.serviceClient().regenerateKey(resourceGroupName, clusterName, databaseName, parameters); + public AccessKeys regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters) { + AccessKeysInner inner + = this.serviceClient().regenerateKey(resourceGroupName, clusterName, databaseName, parameters); if (inner != null) { return new AccessKeysImpl(inner, this.manager()); } else { @@ -112,14 +106,10 @@ public AccessKeys regenerateKey( } } - public AccessKeys regenerateKey( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context) { - AccessKeysInner inner = - this.serviceClient().regenerateKey(resourceGroupName, clusterName, databaseName, parameters, context); + public AccessKeys regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters, Context context) { + AccessKeysInner inner + = this.serviceClient().regenerateKey(resourceGroupName, clusterName, databaseName, parameters, context); if (inner != null) { return new AccessKeysImpl(inner, this.manager()); } else { @@ -127,45 +117,33 @@ public AccessKeys regenerateKey( } } - public void importMethod( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters) { + public void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters) { this.serviceClient().importMethod(resourceGroupName, clusterName, databaseName, parameters); } - public void importMethod( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context) { + public void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters, Context context) { this.serviceClient().importMethod(resourceGroupName, clusterName, databaseName, parameters, context); } - public void export( - String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters) { + public void export(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters) { this.serviceClient().export(resourceGroupName, clusterName, databaseName, parameters); } - public void export( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, - Context context) { + public void export(String resourceGroupName, String clusterName, String databaseName, + ExportClusterParameters parameters, Context context) { this.serviceClient().export(resourceGroupName, clusterName, databaseName, parameters, context); } - public void forceUnlink( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters) { + public void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters) { this.serviceClient().forceUnlink(resourceGroupName, clusterName, databaseName, parameters); } - public void forceUnlink( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context) { + public void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters, Context context) { this.serviceClient().forceUnlink(resourceGroupName, clusterName, databaseName, parameters, context); } @@ -173,119 +151,83 @@ public void flush(String resourceGroupName, String clusterName, String databaseN this.serviceClient().flush(resourceGroupName, clusterName, databaseName, parameters); } - public void flush( - String resourceGroupName, - String clusterName, - String databaseName, - FlushParameters parameters, + public void flush(String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, Context context) { this.serviceClient().flush(resourceGroupName, clusterName, databaseName, parameters, context); } public Database getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String databaseName = Utils.getValueFromIdByName(id, "databases"); + String databaseName = ResourceManagerUtils.getValueFromIdByName(id, "databases"); if (databaseName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, databaseName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String databaseName = Utils.getValueFromIdByName(id, "databases"); + String databaseName = ResourceManagerUtils.getValueFromIdByName(id, "databases"); if (databaseName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, databaseName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String databaseName = Utils.getValueFromIdByName(id, "databases"); + String databaseName = ResourceManagerUtils.getValueFromIdByName(id, "databases"); if (databaseName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); } this.delete(resourceGroupName, clusterName, databaseName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String databaseName = Utils.getValueFromIdByName(id, "databases"); + String databaseName = ResourceManagerUtils.getValueFromIdByName(id, "databases"); if (databaseName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'databases'.", id))); } this.delete(resourceGroupName, clusterName, databaseName, context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationImpl.java index 0d21741190863..4215b23540a4d 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationImpl.java @@ -15,8 +15,8 @@ public final class OperationImpl implements Operation { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - OperationImpl( - OperationInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + OperationImpl(OperationInner innerObject, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationStatusImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationStatusImpl.java index e1054230d9aee..d837f905a090f 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationStatusImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationStatusImpl.java @@ -13,8 +13,7 @@ public final class OperationStatusImpl implements OperationStatus { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - OperationStatusImpl( - OperationStatusInner innerObject, + OperationStatusImpl(OperationStatusInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsClientImpl.java index 6db72f33a7c19..cf492b719c74f 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.redisenterprise.models.OperationListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final RedisEnterpriseManagementClientImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,99 +62,74 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "RedisEnterpriseManag") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Cache/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getApiVersion(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -157,27 +138,27 @@ private PagedFlux listAsync() { /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedFlux}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -186,13 +167,13 @@ public PagedIterable list() { /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -201,14 +182,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -216,37 +198,28 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of REST API operations supported by an Azure Resource Provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -254,23 +227,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsImpl.java index 30cb741c56e16..9cb9e05d9e006 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsImpl.java @@ -19,20 +19,20 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + public OperationsImpl(OperationsClient innerClient, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusClientImpl.java index 82259ee469a7f..5709f8c7f6996 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusClientImpl.java @@ -25,22 +25,28 @@ import com.azure.resourcemanager.redisenterprise.fluent.models.OperationStatusInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsStatusClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsStatusClient. + */ public final class OperationsStatusClientImpl implements OperationsStatusClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsStatusService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final RedisEnterpriseManagementClientImpl client; /** * Initializes an instance of OperationsStatusClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsStatusClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = - RestProxy.create(OperationsStatusService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsStatusService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -51,24 +57,19 @@ public final class OperationsStatusClientImpl implements OperationsStatusClient @Host("{$host}") @ServiceInterface(name = "RedisEnterpriseManag") public interface OperationsStatusService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Cache/locations/{location}/operationsStatus/{operationId}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Cache/locations/{location}/operationsStatus/{operationId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("location") String location, - @PathParam("operationId") String operationId, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @PathParam("location") String location, @PathParam("operationId") String operationId, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); } /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,10 +80,8 @@ Mono> get( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String location, String operationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); @@ -91,30 +90,19 @@ private Mono> getWithResponseAsync(String locatio return Mono.error(new IllegalArgumentException("Parameter operationId is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - location, - operationId, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), location, operationId, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @param context The context to associate with this operation. @@ -124,13 +112,11 @@ private Mono> getWithResponseAsync(String locatio * @return the status of operation along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String location, String operationId, Context context) { + private Mono> getWithResponseAsync(String location, String operationId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (location == null) { return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); @@ -139,27 +125,18 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter operationId is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - location, - operationId, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), location, operationId, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context); } /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -174,7 +151,7 @@ private Mono getAsync(String location, String operationId) /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @param context The context to associate with this operation. @@ -190,7 +167,7 @@ public Response getWithResponse(String location, String op /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusImpl.java index 6d44d30954f33..dfb047f0e3f04 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/OperationsStatusImpl.java @@ -20,8 +20,7 @@ public final class OperationsStatusImpl implements OperationsStatus { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - public OperationsStatusImpl( - OperationsStatusClient innerClient, + public OperationsStatusImpl(OperationsStatusClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,10 +29,7 @@ public OperationsStatusImpl( public Response getWithResponse(String location, String operationId, Context context) { Response inner = this.serviceClient().getWithResponse(location, operationId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new OperationStatusImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionImpl.java index 120bf562f2ba2..fa79e7e79e4d2 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionImpl.java @@ -4,7 +4,6 @@ package com.azure.resourcemanager.redisenterprise.implementation; -import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner; import com.azure.resourcemanager.redisenterprise.models.PrivateEndpoint; @@ -30,10 +29,6 @@ public String type() { return this.innerModel().type(); } - public SystemData systemData() { - return this.innerModel().systemData(); - } - public PrivateEndpoint privateEndpoint() { return this.innerModel().privateEndpoint(); } @@ -71,25 +66,19 @@ public PrivateEndpointConnectionImpl withExistingRedisEnterprise(String resource } public PrivateEndpointConnection create() { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .put(resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections().put(resourceGroupName, + clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); return this; } public PrivateEndpointConnection create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .put(resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections().put(resourceGroupName, + clusterName, privateEndpointConnectionName, this.innerModel(), context); return this; } - PrivateEndpointConnectionImpl( - String name, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { + PrivateEndpointConnectionImpl(String name, + com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = new PrivateEndpointConnectionInner(); this.serviceManager = serviceManager; this.privateEndpointConnectionName = name; @@ -100,50 +89,36 @@ public PrivateEndpointConnectionImpl update() { } public PrivateEndpointConnection apply() { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .put(resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections().put(resourceGroupName, + clusterName, privateEndpointConnectionName, this.innerModel(), Context.NONE); return this; } public PrivateEndpointConnection apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .put(resourceGroupName, clusterName, privateEndpointConnectionName, this.innerModel(), context); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections().put(resourceGroupName, + clusterName, privateEndpointConnectionName, this.innerModel(), context); return this; } - PrivateEndpointConnectionImpl( - PrivateEndpointConnectionInner innerObject, + PrivateEndpointConnectionImpl(PrivateEndpointConnectionInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.clusterName = Utils.getValueFromIdByName(innerObject.id(), "redisEnterprise"); - this.privateEndpointConnectionName = Utils.getValueFromIdByName(innerObject.id(), "privateEndpointConnections"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.clusterName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "redisEnterprise"); + this.privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "privateEndpointConnections"); } public PrivateEndpointConnection refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections() + .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE).getValue(); return this; } public PrivateEndpointConnection refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getPrivateEndpointConnections() - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getPrivateEndpointConnections() + .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context).getValue(); return this; } @@ -152,8 +127,8 @@ public PrivateEndpointConnectionImpl withPrivateEndpoint(PrivateEndpoint private return this; } - public PrivateEndpointConnectionImpl withPrivateLinkServiceConnectionState( - PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { + public PrivateEndpointConnectionImpl + withPrivateLinkServiceConnectionState(PrivateLinkServiceConnectionState privateLinkServiceConnectionState) { this.innerModel().withPrivateLinkServiceConnectionState(privateLinkServiceConnectionState); return this; } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsClientImpl.java index 5acef90ea55b4..dc52812e9f6f1 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsClientImpl.java @@ -38,24 +38,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateEndpointConnectionsClient. + */ public final class PrivateEndpointConnectionsClientImpl implements PrivateEndpointConnectionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PrivateEndpointConnectionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final RedisEnterpriseManagementClientImpl client; /** * Initializes an instance of PrivateEndpointConnectionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PrivateEndpointConnectionsClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = - RestProxy - .create( - PrivateEndpointConnectionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(PrivateEndpointConnectionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -66,86 +70,64 @@ public final class PrivateEndpointConnectionsClientImpl implements PrivateEndpoi @Host("{$host}") @ServiceInterface(name = "RedisEnterpriseManag") public interface PrivateEndpointConnectionsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") - @ExpectedResponses({200}) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, + Mono> get(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") - @ExpectedResponses({201}) + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> put( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> put(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, @BodyParam("application/json") PrivateEndpointConnectionInner properties, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateEndpointConnections/{privateEndpointConnectionName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("privateEndpointConnectionName") String privateEndpointConnectionName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of private endpoint connection associated with the specified storage account along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return list of private endpoint connection associated with the specified storage account along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -155,51 +137,36 @@ private Mono> listSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.list(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of private endpoint connection associated with the specified storage account along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return list of private endpoint connection associated with the specified storage account along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -209,38 +176,28 @@ private Mono> listSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .list(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String clusterName) { @@ -249,7 +206,7 @@ private PagedFlux listAsync(String resourceGroup /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -257,24 +214,24 @@ private PagedFlux listAsync(String resourceGroup * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String clusterName, Context context) { + private PagedFlux listAsync(String resourceGroupName, String clusterName, + Context context) { return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, clusterName, context)); } /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String resourceGroupName, String clusterName) { @@ -283,7 +240,7 @@ public PagedIterable list(String resourceGroupNa /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -291,35 +248,33 @@ public PagedIterable list(String resourceGroupNa * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable list(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listAsync(resourceGroupName, clusterName, context)); } /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response} on successful completion of {@link Mono}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String clusterName, String privateEndpointConnectionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -329,56 +284,41 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), resourceGroupName, clusterName, + privateEndpointConnectionName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response} on successful completion of {@link Mono}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String clusterName, String privateEndpointConnectionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -388,114 +328,97 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - privateEndpointConnectionName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.get(this.client.getEndpoint(), resourceGroupName, clusterName, privateEndpointConnectionName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified private endpoint connection associated with the RedisEnterprise cluster on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + private Mono getAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { return getWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + public Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { return getWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).block(); } /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the specified private endpoint connection associated with the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PrivateEndpointConnectionInner get( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + public PrivateEndpointConnectionInner get(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { return getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE).getValue(); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Private Endpoint Connection resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return the Private Endpoint Connection resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> putWithResponseAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties) { + private Mono>> putWithResponseAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -505,16 +428,12 @@ private Mono>> putWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } if (properties == null) { return Mono.error(new IllegalArgumentException("Parameter properties is required and cannot be null.")); @@ -523,49 +442,33 @@ private Mono>> putWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .put( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - privateEndpointConnectionName, - properties, - accept, - context)) + .withContext(context -> service.put(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), privateEndpointConnectionName, properties, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Private Endpoint Connection resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return the Private Endpoint Connection resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> putWithResponseAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context) { + private Mono>> putWithResponseAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -575,16 +478,12 @@ private Mono>> putWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } if (properties == null) { return Mono.error(new IllegalArgumentException("Parameter properties is required and cannot be null.")); @@ -593,26 +492,17 @@ private Mono>> putWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .put( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - privateEndpointConnectionName, - properties, - accept, - context); + return service.put(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), privateEndpointConnectionName, properties, accept, context); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -621,29 +511,22 @@ private Mono>> putWithResponseAsync( */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PrivateEndpointConnectionInner> beginPutAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, + String resourceGroupName, String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner properties) { - Mono>> mono = - putWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PrivateEndpointConnectionInner.class, - PrivateEndpointConnectionInner.class, - this.client.getContext()); + Mono>> mono + = putWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PrivateEndpointConnectionInner.class, PrivateEndpointConnectionInner.class, + this.client.getContext()); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -653,31 +536,23 @@ private PollerFlux, PrivateEndpointCo */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, PrivateEndpointConnectionInner> beginPutAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context) { + String resourceGroupName, String clusterName, String privateEndpointConnectionName, + PrivateEndpointConnectionInner properties, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - putWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context); - return this - .client - .getLroResult( - mono, - this.client.getHttpPipeline(), - PrivateEndpointConnectionInner.class, - PrivateEndpointConnectionInner.class, - context); + Mono>> mono + = putWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), PrivateEndpointConnectionInner.class, PrivateEndpointConnectionInner.class, + context); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -686,22 +561,19 @@ private PollerFlux, PrivateEndpointCo */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PrivateEndpointConnectionInner> beginPut( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, + String resourceGroupName, String clusterName, String privateEndpointConnectionName, PrivateEndpointConnectionInner properties) { - return this - .beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties) + return this.beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties) .getSyncPoller(); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -711,23 +583,19 @@ public SyncPoller, PrivateEndpointCon */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) public SyncPoller, PrivateEndpointConnectionInner> beginPut( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context) { - return this - .beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context) + String resourceGroupName, String clusterName, String privateEndpointConnectionName, + PrivateEndpointConnectionInner properties, Context context) { + return this.beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context) .getSyncPoller(); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -735,23 +603,19 @@ public SyncPoller, PrivateEndpointCon * @return the Private Endpoint Connection resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono putAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties) { - return beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties) - .last() + private Mono putAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties) { + return beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -760,24 +624,19 @@ private Mono putAsync( * @return the Private Endpoint Connection resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono putAsync( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context) { - return beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context) - .last() + private Mono putAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties, Context context) { + return beginPutAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -785,21 +644,18 @@ private Mono putAsync( * @return the Private Endpoint Connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PrivateEndpointConnectionInner put( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties) { + public PrivateEndpointConnectionInner put(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties) { return putAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties).block(); } /** * Updates the state of the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param properties The private endpoint connection properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -808,35 +664,29 @@ public PrivateEndpointConnectionInner put( * @return the Private Endpoint Connection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PrivateEndpointConnectionInner put( - String resourceGroupName, - String clusterName, - String privateEndpointConnectionName, - PrivateEndpointConnectionInner properties, - Context context) { + public PrivateEndpointConnectionInner put(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, PrivateEndpointConnectionInner properties, Context context) { return putAsync(resourceGroupName, clusterName, privateEndpointConnectionName, properties, context).block(); } /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -846,41 +696,28 @@ private Mono> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - privateEndpointConnectionName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), privateEndpointConnectionName, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -888,13 +725,11 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -904,38 +739,108 @@ private Mono> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (privateEndpointConnectionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter privateEndpointConnectionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter privateEndpointConnectionName is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - privateEndpointConnectionName, - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), privateEndpointConnectionName, accept, context); + } + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { + return this.beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName).getSyncPoller(); } /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { + return this.beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context) + .getSyncPoller(); + } + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -943,42 +848,61 @@ private Mono> deleteWithResponseAsync( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, String privateEndpointConnectionName) { - return deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName) - .flatMap(ignored -> Mono.empty()); + return beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName).last() + .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { - return deleteWithResponseAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).block(); + private Mono deleteAsync(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + Context context) { + return beginDeleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).last() + .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) public void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName) { - deleteWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE); + deleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName).block(); + } + + /** + * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param clusterName The name of the RedisEnterprise cluster. + * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure + * resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + Context context) { + deleteAsync(resourceGroupName, clusterName, privateEndpointConnectionName, context).block(); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsImpl.java index a34e9a42241d7..a2b4aeb5e2202 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateEndpointConnectionsImpl.java @@ -21,8 +21,7 @@ public final class PrivateEndpointConnectionsImpl implements PrivateEndpointConn private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - public PrivateEndpointConnectionsImpl( - PrivateEndpointConnectionsClient innerClient, + public PrivateEndpointConnectionsImpl(PrivateEndpointConnectionsClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,37 +29,32 @@ public PrivateEndpointConnectionsImpl( public PagedIterable list(String resourceGroupName, String clusterName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); } - public PagedIterable list( - String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String clusterName, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateEndpointConnectionImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context); + public Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + clusterName, privateEndpointConnectionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PrivateEndpointConnectionImpl(inner.getValue(), this.manager())); } else { return null; } } - public PrivateEndpointConnection get( - String resourceGroupName, String clusterName, String privateEndpointConnectionName) { - PrivateEndpointConnectionInner inner = - this.serviceClient().get(resourceGroupName, clusterName, privateEndpointConnectionName); + public PrivateEndpointConnection get(String resourceGroupName, String clusterName, + String privateEndpointConnectionName) { + PrivateEndpointConnectionInner inner + = this.serviceClient().get(resourceGroupName, clusterName, privateEndpointConnectionName); if (inner != null) { return new PrivateEndpointConnectionImpl(inner, this.manager()); } else { @@ -68,137 +62,94 @@ public PrivateEndpointConnection get( } } - public Response deleteWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context); - } - public void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName) { this.serviceClient().delete(resourceGroupName, clusterName, privateEndpointConnectionName); } + public void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName, + Context context) { + this.serviceClient().delete(resourceGroupName, clusterName, privateEndpointConnectionName, context); + } + public PrivateEndpointConnection getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } - return this - .getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE) + return this.getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } return this.getWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } - this.deleteWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE); + this.delete(resourceGroupName, clusterName, privateEndpointConnectionName, Context.NONE); } - public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + public void deleteByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } - String privateEndpointConnectionName = Utils.getValueFromIdByName(id, "privateEndpointConnections"); + String privateEndpointConnectionName + = ResourceManagerUtils.getValueFromIdByName(id, "privateEndpointConnections"); if (privateEndpointConnectionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'privateEndpointConnections'.", id))); } - return this.deleteWithResponse(resourceGroupName, clusterName, privateEndpointConnectionName, context); + this.delete(resourceGroupName, clusterName, privateEndpointConnectionName, context); } private PrivateEndpointConnectionsClient serviceClient() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourceImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourceImpl.java index 9b943035858fa..50c7942b345af 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourceImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourceImpl.java @@ -4,7 +4,6 @@ package com.azure.resourcemanager.redisenterprise.implementation; -import com.azure.core.management.SystemData; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceInner; import com.azure.resourcemanager.redisenterprise.models.PrivateLinkResource; import java.util.Collections; @@ -15,8 +14,7 @@ public final class PrivateLinkResourceImpl implements PrivateLinkResource { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - PrivateLinkResourceImpl( - PrivateLinkResourceInner innerObject, + PrivateLinkResourceImpl(PrivateLinkResourceInner innerObject, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -34,10 +32,6 @@ public String type() { return this.innerModel().type(); } - public SystemData systemData() { - return this.innerModel().systemData(); - } - public String groupId() { return this.innerModel().groupId(); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesClientImpl.java index 2a6bd8381d857..baadcc2e502dd 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesClientImpl.java @@ -30,67 +30,65 @@ import com.azure.resourcemanager.redisenterprise.models.PrivateLinkResourceListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. */ +/** + * An instance of this class provides access to all the operations defined in PrivateLinkResourcesClient. + */ public final class PrivateLinkResourcesClientImpl implements PrivateLinkResourcesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PrivateLinkResourcesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final RedisEnterpriseManagementClientImpl client; /** * Initializes an instance of PrivateLinkResourcesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PrivateLinkResourcesClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = - RestProxy - .create(PrivateLinkResourcesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(PrivateLinkResourcesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for RedisEnterpriseManagementClientPrivateLinkResources to be used by the - * proxy service to perform REST calls. + * The interface defining all the services for RedisEnterpriseManagementClientPrivateLinkResources to be used by + * the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "RedisEnterpriseManag") public interface PrivateLinkResourcesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateLinkResources") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}/privateLinkResources") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByCluster( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByCluster(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); } /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the private link resources that need to be created for a RedisEnterprise cluster along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return the private link resources that need to be created for a RedisEnterprise cluster along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -100,51 +98,36 @@ private Mono> listByClusterSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByCluster( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) + .withContext(context -> service.listByCluster(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the private link resources that need to be created for a RedisEnterprise cluster along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return the private link resources that need to be created for a RedisEnterprise cluster along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByClusterSinglePageAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> listByClusterSinglePageAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -154,38 +137,28 @@ private Mono> listByClusterSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByCluster( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); + .listByCluster(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), null, null)); } /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName) { @@ -194,7 +167,7 @@ private PagedFlux listByClusterAsync(String resourceGr /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -202,24 +175,24 @@ private PagedFlux listByClusterAsync(String resourceGr * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByClusterAsync( - String resourceGroupName, String clusterName, Context context) { + private PagedFlux listByClusterAsync(String resourceGroupName, String clusterName, + Context context) { return new PagedFlux<>(() -> listByClusterSinglePageAsync(resourceGroupName, clusterName, context)); } /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByCluster(String resourceGroupName, String clusterName) { @@ -228,7 +201,7 @@ public PagedIterable listByCluster(String resourceGrou /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -236,11 +209,11 @@ public PagedIterable listByCluster(String resourceGrou * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { return new PagedIterable<>(listByClusterAsync(resourceGroupName, clusterName, context)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesImpl.java index 930dc94d7db45..9e219d1a9f8fd 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/PrivateLinkResourcesImpl.java @@ -19,24 +19,23 @@ public final class PrivateLinkResourcesImpl implements PrivateLinkResources { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - public PrivateLinkResourcesImpl( - PrivateLinkResourcesClient innerClient, + public PrivateLinkResourcesImpl(PrivateLinkResourcesClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listByCluster(String resourceGroupName, String clusterName) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName); - return Utils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); } - public PagedIterable listByCluster( - String resourceGroupName, String clusterName, Context context) { - PagedIterable inner = - this.serviceClient().listByCluster(resourceGroupName, clusterName, context); - return Utils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); + public PagedIterable listByCluster(String resourceGroupName, String clusterName, + Context context) { + PagedIterable inner + = this.serviceClient().listByCluster(resourceGroupName, clusterName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new PrivateLinkResourceImpl(inner1, this.manager())); } private PrivateLinkResourcesClient serviceClient() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientBuilder.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientBuilder.java index 998d6c966e8ca..d98a796994457 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientBuilder.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientBuilder.java @@ -14,8 +14,10 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the RedisEnterpriseManagementClientImpl type. */ -@ServiceClientBuilder(serviceClients = {RedisEnterpriseManagementClientImpl.class}) +/** + * A builder for creating a new instance of the RedisEnterpriseManagementClientImpl type. + */ +@ServiceClientBuilder(serviceClients = { RedisEnterpriseManagementClientImpl.class }) public final class RedisEnterpriseManagementClientBuilder { /* * The ID of the target subscription. @@ -24,7 +26,7 @@ public final class RedisEnterpriseManagementClientBuilder { /** * Sets The ID of the target subscription. - * + * * @param subscriptionId the subscriptionId value. * @return the RedisEnterpriseManagementClientBuilder. */ @@ -40,7 +42,7 @@ public RedisEnterpriseManagementClientBuilder subscriptionId(String subscription /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the RedisEnterpriseManagementClientBuilder. */ @@ -56,7 +58,7 @@ public RedisEnterpriseManagementClientBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the RedisEnterpriseManagementClientBuilder. */ @@ -72,7 +74,7 @@ public RedisEnterpriseManagementClientBuilder environment(AzureEnvironment envir /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the RedisEnterpriseManagementClientBuilder. */ @@ -88,7 +90,7 @@ public RedisEnterpriseManagementClientBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the RedisEnterpriseManagementClientBuilder. */ @@ -104,7 +106,7 @@ public RedisEnterpriseManagementClientBuilder defaultPollInterval(Duration defau /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the RedisEnterpriseManagementClientBuilder. */ @@ -115,30 +117,20 @@ public RedisEnterpriseManagementClientBuilder serializerAdapter(SerializerAdapte /** * Builds an instance of RedisEnterpriseManagementClientImpl with the provided parameters. - * + * * @return an instance of RedisEnterpriseManagementClientImpl. */ public RedisEnterpriseManagementClientImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - RedisEnterpriseManagementClientImpl client = - new RedisEnterpriseManagementClientImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + RedisEnterpriseManagementClientImpl client = new RedisEnterpriseManagementClientImpl(localPipeline, + localSerializerAdapter, localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientImpl.java index bd4aadb26a2e8..582b33738a214 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterpriseManagementClientImpl.java @@ -29,7 +29,6 @@ import com.azure.resourcemanager.redisenterprise.fluent.PrivateLinkResourcesClient; import com.azure.resourcemanager.redisenterprise.fluent.RedisEnterpriseManagementClient; import com.azure.resourcemanager.redisenterprise.fluent.RedisEnterprisesClient; -import com.azure.resourcemanager.redisenterprise.fluent.SkusClient; import java.io.IOException; import java.lang.reflect.Type; import java.nio.ByteBuffer; @@ -39,168 +38,182 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the RedisEnterpriseManagementClientImpl type. */ +/** + * Initializes a new instance of the RedisEnterpriseManagementClientImpl type. + */ @ServiceClient(builder = RedisEnterpriseManagementClientBuilder.class) public final class RedisEnterpriseManagementClientImpl implements RedisEnterpriseManagementClient { - /** The ID of the target subscription. */ + /** + * The ID of the target subscription. + */ private final String subscriptionId; /** * Gets The ID of the target subscription. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** Api Version. */ + /** + * Api Version. + */ private final String apiVersion; /** * Gets Api Version. - * + * * @return the apiVersion value. */ public String getApiVersion() { return this.apiVersion; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The OperationsStatusClient object to access its operations. */ + /** + * The OperationsStatusClient object to access its operations. + */ private final OperationsStatusClient operationsStatus; /** * Gets the OperationsStatusClient object to access its operations. - * + * * @return the OperationsStatusClient object. */ public OperationsStatusClient getOperationsStatus() { return this.operationsStatus; } - /** The RedisEnterprisesClient object to access its operations. */ + /** + * The RedisEnterprisesClient object to access its operations. + */ private final RedisEnterprisesClient redisEnterprises; /** * Gets the RedisEnterprisesClient object to access its operations. - * + * * @return the RedisEnterprisesClient object. */ public RedisEnterprisesClient getRedisEnterprises() { return this.redisEnterprises; } - /** The DatabasesClient object to access its operations. */ + /** + * The DatabasesClient object to access its operations. + */ private final DatabasesClient databases; /** * Gets the DatabasesClient object to access its operations. - * + * * @return the DatabasesClient object. */ public DatabasesClient getDatabases() { return this.databases; } - /** The PrivateEndpointConnectionsClient object to access its operations. */ + /** + * The PrivateEndpointConnectionsClient object to access its operations. + */ private final PrivateEndpointConnectionsClient privateEndpointConnections; /** * Gets the PrivateEndpointConnectionsClient object to access its operations. - * + * * @return the PrivateEndpointConnectionsClient object. */ public PrivateEndpointConnectionsClient getPrivateEndpointConnections() { return this.privateEndpointConnections; } - /** The PrivateLinkResourcesClient object to access its operations. */ + /** + * The PrivateLinkResourcesClient object to access its operations. + */ private final PrivateLinkResourcesClient privateLinkResources; /** * Gets the PrivateLinkResourcesClient object to access its operations. - * + * * @return the PrivateLinkResourcesClient object. */ public PrivateLinkResourcesClient getPrivateLinkResources() { return this.privateLinkResources; } - /** The SkusClient object to access its operations. */ - private final SkusClient skus; - - /** - * Gets the SkusClient object to access its operations. - * - * @return the SkusClient object. - */ - public SkusClient getSkus() { - return this.skus; - } - /** * Initializes an instance of RedisEnterpriseManagementClient client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -208,31 +221,25 @@ public SkusClient getSkus() { * @param subscriptionId The ID of the target subscription. * @param endpoint server parameter. */ - RedisEnterpriseManagementClientImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + RedisEnterpriseManagementClientImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, + Duration defaultPollInterval, AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.apiVersion = "2023-03-01-preview"; + this.apiVersion = "2023-11-01"; this.operations = new OperationsClientImpl(this); this.operationsStatus = new OperationsStatusClientImpl(this); this.redisEnterprises = new RedisEnterprisesClientImpl(this); this.databases = new DatabasesClientImpl(this); this.privateEndpointConnections = new PrivateEndpointConnectionsClientImpl(this); this.privateLinkResources = new PrivateLinkResourcesClientImpl(this); - this.skus = new SkusClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -241,7 +248,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -251,7 +258,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -261,26 +268,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -293,19 +289,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesClientImpl.java index e97c22f9e28f9..f57ae6e431cbf 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesClientImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesClientImpl.java @@ -40,22 +40,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in RedisEnterprisesClient. */ +/** + * An instance of this class provides access to all the operations defined in RedisEnterprisesClient. + */ public final class RedisEnterprisesClientImpl implements RedisEnterprisesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final RedisEnterprisesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final RedisEnterpriseManagementClientImpl client; /** * Initializes an instance of RedisEnterprisesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ RedisEnterprisesClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = - RestProxy.create(RedisEnterprisesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(RedisEnterprisesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -66,129 +72,94 @@ public final class RedisEnterprisesClientImpl implements RedisEnterprisesClient @Host("{$host}") @ServiceInterface(name = "RedisEnterpriseManag") public interface RedisEnterprisesService { - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> create( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") ClusterInner parameters, - @HeaderParam("Accept") String accept, + Mono>> create(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") ClusterInner parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> update( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @BodyParam("application/json") ClusterUpdate parameters, - @HeaderParam("Accept") String accept, + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @BodyParam("application/json") ClusterUpdate parameters, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") - @ExpectedResponses({200}) + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise/{clusterName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("clusterName") String clusterName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise") - @ExpectedResponses({200}) + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("clusterName") String clusterName, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Cache/redisEnterprise") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Cache/redisEnterprise") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of {@link - * Mono}. + * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, ClusterInner parameters) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + ClusterInner parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -198,10 +169,8 @@ private Mono>> createWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -210,24 +179,14 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -235,17 +194,15 @@ private Mono>> createWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of {@link - * Mono}. + * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> createWithResponseAsync( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { + private Mono>> createWithResponseAsync(String resourceGroupName, String clusterName, + ClusterInner parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -255,10 +212,8 @@ private Mono>> createWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -267,21 +222,13 @@ private Mono>> createWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.create(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), parameters, accept, context); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -291,18 +238,16 @@ private Mono>> createWithResponseAsync( * @return the {@link PollerFlux} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginCreateAsync( - String resourceGroupName, String clusterName, ClusterInner parameters) { + private PollerFlux, ClusterInner> beginCreateAsync(String resourceGroupName, + String clusterName, ClusterInner parameters) { Mono>> mono = createWithResponseAsync(resourceGroupName, clusterName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, this.client.getContext()); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -313,20 +258,18 @@ private PollerFlux, ClusterInner> beginCreateAsync( * @return the {@link PollerFlux} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginCreateAsync( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { + private PollerFlux, ClusterInner> beginCreateAsync(String resourceGroupName, + String clusterName, ClusterInner parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - createWithResponseAsync(resourceGroupName, clusterName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, context); + Mono>> mono + = createWithResponseAsync(resourceGroupName, clusterName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, context); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -336,14 +279,14 @@ private PollerFlux, ClusterInner> beginCreateAsync( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters) { + public SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters) { return this.beginCreateAsync(resourceGroupName, clusterName, parameters).getSyncPoller(); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -354,14 +297,14 @@ public SyncPoller, ClusterInner> beginCreate( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginCreate( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { + public SyncPoller, ClusterInner> beginCreate(String resourceGroupName, String clusterName, + ClusterInner parameters, Context context) { return this.beginCreateAsync(resourceGroupName, clusterName, parameters, context).getSyncPoller(); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -372,14 +315,13 @@ public SyncPoller, ClusterInner> beginCreate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono createAsync(String resourceGroupName, String clusterName, ClusterInner parameters) { - return beginCreateAsync(resourceGroupName, clusterName, parameters) - .last() + return beginCreateAsync(resourceGroupName, clusterName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -390,16 +332,15 @@ private Mono createAsync(String resourceGroupName, String clusterN * @return describes the RedisEnterprise cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String clusterName, ClusterInner parameters, Context context) { - return beginCreateAsync(resourceGroupName, clusterName, parameters, context) - .last() + private Mono createAsync(String resourceGroupName, String clusterName, ClusterInner parameters, + Context context) { + return beginCreateAsync(resourceGroupName, clusterName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -415,7 +356,7 @@ public ClusterInner create(String resourceGroupName, String clusterName, Cluster /** * Creates or updates an existing (overwrite/recreate, with potential downtime) cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Create RedisEnterprise operation. @@ -432,24 +373,22 @@ public ClusterInner create(String resourceGroupName, String clusterName, Cluster /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of {@link - * Mono}. + * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String clusterName, ClusterUpdate parameters) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String clusterName, + ClusterUpdate parameters) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -459,10 +398,8 @@ private Mono>> updateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -471,24 +408,14 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), parameters, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -496,17 +423,15 @@ private Mono>> updateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of {@link - * Mono}. + * @return describes the RedisEnterprise cluster along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> updateWithResponseAsync( - String resourceGroupName, String clusterName, ClusterUpdate parameters, Context context) { + private Mono>> updateWithResponseAsync(String resourceGroupName, String clusterName, + ClusterUpdate parameters, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -516,10 +441,8 @@ private Mono>> updateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (parameters == null) { return Mono.error(new IllegalArgumentException("Parameter parameters is required and cannot be null.")); @@ -528,21 +451,13 @@ private Mono>> updateWithResponseAsync( } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - parameters, - accept, - context); + return service.update(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), parameters, accept, context); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -552,18 +467,16 @@ private Mono>> updateWithResponseAsync( * @return the {@link PollerFlux} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginUpdateAsync( - String resourceGroupName, String clusterName, ClusterUpdate parameters) { + private PollerFlux, ClusterInner> beginUpdateAsync(String resourceGroupName, + String clusterName, ClusterUpdate parameters) { Mono>> mono = updateWithResponseAsync(resourceGroupName, clusterName, parameters); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, this.client.getContext()); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -574,20 +487,18 @@ private PollerFlux, ClusterInner> beginUpdateAsync( * @return the {@link PollerFlux} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, ClusterInner> beginUpdateAsync( - String resourceGroupName, String clusterName, ClusterUpdate parameters, Context context) { + private PollerFlux, ClusterInner> beginUpdateAsync(String resourceGroupName, + String clusterName, ClusterUpdate parameters, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - updateWithResponseAsync(resourceGroupName, clusterName, parameters, context); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), ClusterInner.class, ClusterInner.class, context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, clusterName, parameters, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ClusterInner.class, ClusterInner.class, context); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -597,14 +508,14 @@ private PollerFlux, ClusterInner> beginUpdateAsync( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterUpdate parameters) { + public SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterUpdate parameters) { return this.beginUpdateAsync(resourceGroupName, clusterName, parameters).getSyncPoller(); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -615,14 +526,14 @@ public SyncPoller, ClusterInner> beginUpdate( * @return the {@link SyncPoller} for polling of describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, ClusterInner> beginUpdate( - String resourceGroupName, String clusterName, ClusterUpdate parameters, Context context) { + public SyncPoller, ClusterInner> beginUpdate(String resourceGroupName, String clusterName, + ClusterUpdate parameters, Context context) { return this.beginUpdateAsync(resourceGroupName, clusterName, parameters, context).getSyncPoller(); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -633,14 +544,13 @@ public SyncPoller, ClusterInner> beginUpdate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono updateAsync(String resourceGroupName, String clusterName, ClusterUpdate parameters) { - return beginUpdateAsync(resourceGroupName, clusterName, parameters) - .last() + return beginUpdateAsync(resourceGroupName, clusterName, parameters).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -651,16 +561,15 @@ private Mono updateAsync(String resourceGroupName, String clusterN * @return describes the RedisEnterprise cluster on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String clusterName, ClusterUpdate parameters, Context context) { - return beginUpdateAsync(resourceGroupName, clusterName, parameters, context) - .last() + private Mono updateAsync(String resourceGroupName, String clusterName, ClusterUpdate parameters, + Context context) { + return beginUpdateAsync(resourceGroupName, clusterName, parameters, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -676,7 +585,7 @@ public ClusterInner update(String resourceGroupName, String clusterName, Cluster /** * Updates an existing RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param parameters Parameters supplied to the Update RedisEnterprise operation. @@ -687,14 +596,14 @@ public ClusterInner update(String resourceGroupName, String clusterName, Cluster * @return describes the RedisEnterprise cluster. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ClusterInner update( - String resourceGroupName, String clusterName, ClusterUpdate parameters, Context context) { + public ClusterInner update(String resourceGroupName, String clusterName, ClusterUpdate parameters, + Context context) { return updateAsync(resourceGroupName, clusterName, parameters, context).block(); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -705,10 +614,8 @@ public ClusterInner update( @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -718,30 +625,19 @@ private Mono>> deleteWithResponseAsync(String resource return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -751,13 +647,11 @@ private Mono>> deleteWithResponseAsync(String resource * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String clusterName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -767,27 +661,18 @@ private Mono>> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.delete(this.client.getEndpoint(), resourceGroupName, clusterName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -798,15 +683,13 @@ private Mono>> deleteWithResponseAsync( @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName) { Mono>> mono = deleteWithResponseAsync(resourceGroupName, clusterName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -816,18 +699,17 @@ private PollerFlux, Void> beginDeleteAsync(String resourceGroup * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String clusterName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String clusterName, + Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(resourceGroupName, clusterName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -842,7 +724,7 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -852,14 +734,14 @@ public SyncPoller, Void> beginDelete(String resourceGroupName, * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String clusterName, Context context) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String clusterName, + Context context) { return this.beginDeleteAsync(resourceGroupName, clusterName, context).getSyncPoller(); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -874,7 +756,7 @@ private Mono deleteAsync(String resourceGroupName, String clusterName) { /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -885,14 +767,13 @@ private Mono deleteAsync(String resourceGroupName, String clusterName) { */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono deleteAsync(String resourceGroupName, String clusterName, Context context) { - return beginDeleteAsync(resourceGroupName, clusterName, context) - .last() + return beginDeleteAsync(resourceGroupName, clusterName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -906,7 +787,7 @@ public void delete(String resourceGroupName, String clusterName) { /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -921,23 +802,21 @@ public void delete(String resourceGroupName, String clusterName, Context context /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a RedisEnterprise cluster along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String clusterName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String clusterName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -947,30 +826,19 @@ private Mono> getByResourceGroupWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), resourceGroupName, + clusterName, this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -978,16 +846,14 @@ private Mono> getByResourceGroupWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a RedisEnterprise cluster along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String clusterName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String clusterName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -997,27 +863,18 @@ private Mono> getByResourceGroupWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter clusterName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - clusterName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), resourceGroupName, clusterName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context); } /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1033,7 +890,7 @@ private Mono getByResourceGroupAsync(String resourceGroupName, Str /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -1043,14 +900,14 @@ private Mono getByResourceGroupAsync(String resourceGroupName, Str * @return information about a RedisEnterprise cluster along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String clusterName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, clusterName, context).block(); } /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1065,110 +922,75 @@ public ClusterInner getByResourceGroup(String resourceGroupName, String clusterN /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), resourceGroupName, + this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - resourceGroupName, - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), resourceGroupName, this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1177,14 +999,13 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1194,14 +1015,13 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupNam */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1215,7 +1035,7 @@ public PagedIterable listByResourceGroup(String resourceGroupName) /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1230,96 +1050,63 @@ public PagedIterable listByResourceGroup(String resourceGroupName, /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all RedisEnterprise clusters in the specified subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getApiVersion(), + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all RedisEnterprise clusters in the specified subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getApiVersion(), - this.client.getSubscriptionId(), - accept, + .list(this.client.getEndpoint(), this.client.getApiVersion(), this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link PagedFlux}. @@ -1331,7 +1118,7 @@ private PagedFlux listAsync() { /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1340,17 +1127,17 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link - * PagedIterable}. + * @return all RedisEnterprise clusters in the specified subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -1359,13 +1146,13 @@ public PagedIterable list() { /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link - * PagedIterable}. + * @return all RedisEnterprise clusters in the specified subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -1374,14 +1161,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1389,38 +1177,30 @@ private Mono> listByResourceGroupNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, Context context) { @@ -1428,36 +1208,27 @@ private Mono> listByResourceGroupNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1465,37 +1236,28 @@ private Mono> listNextSinglePageAsync(String nextLin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return the response of a list-all operation along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1503,23 +1265,13 @@ private Mono> listNextSinglePageAsync(String nextLin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesImpl.java index fc2ebe8fcbcfb..2406c89841766 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesImpl.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RedisEnterprisesImpl.java @@ -21,8 +21,7 @@ public final class RedisEnterprisesImpl implements RedisEnterprises { private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - public RedisEnterprisesImpl( - RedisEnterprisesClient innerClient, + public RedisEnterprisesImpl(RedisEnterprisesClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -36,15 +35,12 @@ public void delete(String resourceGroupName, String clusterName, Context context this.serviceClient().delete(resourceGroupName, clusterName, context); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String clusterName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, clusterName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String clusterName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, clusterName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ClusterImpl(inner.getValue(), this.manager())); } else { return null; @@ -62,100 +58,76 @@ public Cluster getByResourceGroup(String resourceGroupName, String clusterName) public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ClusterImpl(inner1, this.manager())); } public Cluster getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, clusterName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, clusterName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } this.delete(resourceGroupName, clusterName, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String clusterName = Utils.getValueFromIdByName(id, "redisEnterprise"); + String clusterName = ResourceManagerUtils.getValueFromIdByName(id, "redisEnterprise"); if (clusterName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'redisEnterprise'.", id))); } this.delete(resourceGroupName, clusterName, context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RegionSkuDetailImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RegionSkuDetailImpl.java deleted file mode 100644 index 047d9d2f74e84..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/RegionSkuDetailImpl.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.implementation; - -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; -import com.azure.resourcemanager.redisenterprise.models.LocationInfo; -import com.azure.resourcemanager.redisenterprise.models.RegionSkuDetail; -import com.azure.resourcemanager.redisenterprise.models.SkuDetail; - -public final class RegionSkuDetailImpl implements RegionSkuDetail { - private RegionSkuDetailInner innerObject; - - private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - - RegionSkuDetailImpl( - RegionSkuDetailInner innerObject, - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public String resourceType() { - return this.innerModel().resourceType(); - } - - public LocationInfo locationInfo() { - return this.innerModel().locationInfo(); - } - - public SkuDetail skuDetails() { - return this.innerModel().skuDetails(); - } - - public RegionSkuDetailInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ResourceManagerUtils.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..fc227fb68588e --- /dev/null +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.redisenterprise.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusClientImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusClientImpl.java deleted file mode 100644 index 86f6a63c3672c..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusClientImpl.java +++ /dev/null @@ -1,222 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.implementation; - -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.Get; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.redisenterprise.fluent.SkusClient; -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; -import com.azure.resourcemanager.redisenterprise.models.RegionSkuDetails; -import reactor.core.publisher.Mono; - -/** An instance of this class provides access to all the operations defined in SkusClient. */ -public final class SkusClientImpl implements SkusClient { - /** The proxy service used to perform REST calls. */ - private final SkusService service; - - /** The service client containing this operation class. */ - private final RedisEnterpriseManagementClientImpl client; - - /** - * Initializes an instance of SkusClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - SkusClientImpl(RedisEnterpriseManagementClientImpl client) { - this.service = RestProxy.create(SkusService.class, client.getHttpPipeline(), client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for RedisEnterpriseManagementClientSkus to be used by the proxy service - * to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "RedisEnterpriseManag") - public interface SkusService { - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Cache/locations/{location}/skus") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("location") String location, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - } - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id along with {@link - * PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(String location) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (location == null) { - return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - location, - this.client.getApiVersion(), - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id along with {@link - * PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(String location, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (location == null) { - return Mono.error(new IllegalArgumentException("Parameter location is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - location, - this.client.getApiVersion(), - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), res.getStatusCode(), res.getHeaders(), res.getValue().value(), null, null)); - } - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id as paginated response with - * {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String location) { - return new PagedFlux<>(() -> listSinglePageAsync(location)); - } - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id as paginated response with - * {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String location, Context context) { - return new PagedFlux<>(() -> listSinglePageAsync(location, context)); - } - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String location) { - return new PagedIterable<>(listAsync(location)); - } - - /** - * Gets information about skus in specified location for the given subscription id. - * - * @param location The name of Azure region. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information about skus in specified location for the given subscription id as paginated response with - * {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String location, Context context) { - return new PagedIterable<>(listAsync(location, context)); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusImpl.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusImpl.java deleted file mode 100644 index ef64fe0143f8d..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/SkusImpl.java +++ /dev/null @@ -1,45 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.redisenterprise.fluent.SkusClient; -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; -import com.azure.resourcemanager.redisenterprise.models.RegionSkuDetail; -import com.azure.resourcemanager.redisenterprise.models.Skus; - -public final class SkusImpl implements Skus { - private static final ClientLogger LOGGER = new ClientLogger(SkusImpl.class); - - private final SkusClient innerClient; - - private final com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager; - - public SkusImpl( - SkusClient innerClient, com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable list(String location) { - PagedIterable inner = this.serviceClient().list(location); - return Utils.mapPage(inner, inner1 -> new RegionSkuDetailImpl(inner1, this.manager())); - } - - public PagedIterable list(String location, Context context) { - PagedIterable inner = this.serviceClient().list(location, context); - return Utils.mapPage(inner, inner1 -> new RegionSkuDetailImpl(inner1, this.manager())); - } - - private SkusClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/Utils.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/Utils.java deleted file mode 100644 index 6d2c2ff2db617..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/package-info.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/package-info.java index 6d6314b6e4417..2f6dee987cc9e 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/package-info.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/implementation/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the implementations for RedisEnterpriseManagementClient. REST API for managing Redis Enterprise - * resources in Azure. + * Package containing the implementations for RedisEnterpriseManagementClient. + * REST API for managing Redis Enterprise resources in Azure. */ package com.azure.resourcemanager.redisenterprise.implementation; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeyType.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeyType.java index a475e6db70f98..a4675dbf3ddaa 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeyType.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeyType.java @@ -7,15 +7,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; -/** Which access key to regenerate. */ +/** + * Which access key to regenerate. + */ public enum AccessKeyType { - /** Enum value Primary. */ + /** + * Enum value Primary. + */ PRIMARY("Primary"), - /** Enum value Secondary. */ + /** + * Enum value Secondary. + */ SECONDARY("Secondary"); - /** The actual serialized value for a AccessKeyType instance. */ + /** + * The actual serialized value for a AccessKeyType instance. + */ private final String value; AccessKeyType(String value) { @@ -24,7 +32,7 @@ public enum AccessKeyType { /** * Parses a serialized value to a AccessKeyType instance. - * + * * @param value the serialized value to parse. * @return the parsed AccessKeyType object, or null if unable to parse. */ @@ -42,7 +50,9 @@ public static AccessKeyType fromString(String value) { return null; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @JsonValue @Override public String toString() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeys.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeys.java index ce721d5180da4..4e45fcd3476ae 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeys.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AccessKeys.java @@ -6,25 +6,27 @@ import com.azure.resourcemanager.redisenterprise.fluent.models.AccessKeysInner; -/** An immutable client-side representation of AccessKeys. */ +/** + * An immutable client-side representation of AccessKeys. + */ public interface AccessKeys { /** * Gets the primaryKey property: The current primary key that clients can use to authenticate. - * + * * @return the primaryKey value. */ String primaryKey(); /** * Gets the secondaryKey property: The current secondary key that clients can use to authenticate. - * + * * @return the secondaryKey value. */ String secondaryKey(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.AccessKeysInner object. - * + * * @return the inner object. */ AccessKeysInner innerModel(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ActionType.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ActionType.java index e3af6536802d4..467c40c033a52 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ActionType.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ActionType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. */ +/** + * Enum. Indicates the action type. "Internal" refers to actions that are for internal only APIs. + */ public final class ActionType extends ExpandableStringEnum { - /** Static value Internal for ActionType. */ + /** + * Static value Internal for ActionType. + */ public static final ActionType INTERNAL = fromString("Internal"); /** * Creates a new instance of ActionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ActionType() { /** * Creates or finds a ActionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActionType. */ @@ -35,7 +39,7 @@ public static ActionType fromString(String name) { /** * Gets known ActionType values. - * + * * @return known ActionType values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AofFrequency.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AofFrequency.java index 42056137ab5e8..883da29b82218 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AofFrequency.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/AofFrequency.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Sets the frequency at which data is written to disk. */ +/** + * Sets the frequency at which data is written to disk. + */ public final class AofFrequency extends ExpandableStringEnum { - /** Static value 1s for AofFrequency. */ + /** + * Static value 1s for AofFrequency. + */ public static final AofFrequency ONES = fromString("1s"); - /** Static value always for AofFrequency. */ + /** + * Static value always for AofFrequency. + */ public static final AofFrequency ALWAYS = fromString("always"); /** * Creates a new instance of AofFrequency value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AofFrequency() { /** * Creates or finds a AofFrequency from its string representation. - * + * * @param name a name to look for. * @return the corresponding AofFrequency. */ @@ -38,7 +44,7 @@ public static AofFrequency fromString(String name) { /** * Gets known AofFrequency values. - * + * * @return known AofFrequency values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Cluster.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Cluster.java index 39ac8576d9391..ea4c27c80fbf8 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Cluster.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Cluster.java @@ -5,115 +5,109 @@ package com.azure.resourcemanager.redisenterprise.models; import com.azure.core.management.Region; -import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner; import java.util.List; import java.util.Map; -/** An immutable client-side representation of Cluster. */ +/** + * An immutable client-side representation of Cluster. + */ public interface Cluster { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the sku property: The SKU to create, which affects price, performance, and features. - * + * * @return the sku value. */ Sku sku(); /** * Gets the zones property: The Availability Zones where this cluster will be deployed. - * + * * @return the zones value. */ List zones(); /** * Gets the identity property: The identity of the resource. - * + * * @return the identity value. */ ManagedServiceIdentity identity(); - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - /** * Gets the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @return the minimumTlsVersion value. */ TlsVersion minimumTlsVersion(); /** * Gets the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @return the encryption value. */ ClusterPropertiesEncryption encryption(); /** * Gets the hostname property: DNS name of the cluster endpoint. - * + * * @return the hostname value. */ String hostname(); /** * Gets the provisioningState property: Current provisioning status of the cluster. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the resourceState property: Current resource status of the cluster. - * + * * @return the resourceState value. */ ResourceState resourceState(); /** * Gets the redisVersion property: Version of redis the cluster supports, e.g. '6'. - * + * * @return the redisVersion value. */ String redisVersion(); @@ -121,57 +115,63 @@ public interface Cluster { /** * Gets the privateEndpointConnections property: List of private endpoint connections associated with the specified * RedisEnterprise cluster. - * + * * @return the privateEndpointConnections value. */ List privateEndpointConnections(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner object. - * + * * @return the inner object. */ ClusterInner innerModel(); - /** The entirety of the Cluster definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithSku, - DefinitionStages.WithCreate { + /** + * The entirety of the Cluster definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithSku, DefinitionStages.WithCreate { } - /** The Cluster definition stages. */ + + /** + * The Cluster definition stages. + */ interface DefinitionStages { - /** The first stage of the Cluster definition. */ + /** + * The first stage of the Cluster definition. + */ interface Blank extends WithLocation { } - /** The stage of the Cluster definition allowing to specify location. */ + + /** + * The stage of the Cluster definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -179,200 +179,235 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Cluster definition allowing to specify parent resource. */ + + /** + * The stage of the Cluster definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @return the next definition stage. */ WithSku withExistingResourceGroup(String resourceGroupName); } - /** The stage of the Cluster definition allowing to specify sku. */ + + /** + * The stage of the Cluster definition allowing to specify sku. + */ interface WithSku { /** * Specifies the sku property: The SKU to create, which affects price, performance, and features.. - * + * * @param sku The SKU to create, which affects price, performance, and features. * @return the next definition stage. */ WithCreate withSku(Sku sku); } + /** * The stage of the Cluster definition which contains all the minimum required properties for the resource to be * created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithZones, - DefinitionStages.WithIdentity, - DefinitionStages.WithMinimumTlsVersion, - DefinitionStages.WithEncryption { + interface WithCreate extends DefinitionStages.WithTags, DefinitionStages.WithZones, + DefinitionStages.WithIdentity, DefinitionStages.WithMinimumTlsVersion, DefinitionStages.WithEncryption { /** * Executes the create request. - * + * * @return the created resource. */ Cluster create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Cluster create(Context context); } - /** The stage of the Cluster definition allowing to specify tags. */ + + /** + * The stage of the Cluster definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Cluster definition allowing to specify zones. */ + + /** + * The stage of the Cluster definition allowing to specify zones. + */ interface WithZones { /** * Specifies the zones property: The Availability Zones where this cluster will be deployed.. - * + * * @param zones The Availability Zones where this cluster will be deployed. * @return the next definition stage. */ WithCreate withZones(List zones); } - /** The stage of the Cluster definition allowing to specify identity. */ + + /** + * The stage of the Cluster definition allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The identity of the resource.. - * + * * @param identity The identity of the resource. * @return the next definition stage. */ WithCreate withIdentity(ManagedServiceIdentity identity); } - /** The stage of the Cluster definition allowing to specify minimumTlsVersion. */ + + /** + * The stage of the Cluster definition allowing to specify minimumTlsVersion. + */ interface WithMinimumTlsVersion { /** * Specifies the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @param minimumTlsVersion The minimum TLS version for the cluster to support, e.g. '1.2'. * @return the next definition stage. */ WithCreate withMinimumTlsVersion(TlsVersion minimumTlsVersion); } - /** The stage of the Cluster definition allowing to specify encryption. */ + + /** + * The stage of the Cluster definition allowing to specify encryption. + */ interface WithEncryption { /** * Specifies the encryption property: Encryption-at-rest configuration for the cluster.. - * + * * @param encryption Encryption-at-rest configuration for the cluster. * @return the next definition stage. */ WithCreate withEncryption(ClusterPropertiesEncryption encryption); } } + /** * Begins update for the Cluster resource. - * + * * @return the stage of resource update. */ Cluster.Update update(); - /** The template for Cluster update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithSku, - UpdateStages.WithIdentity, - UpdateStages.WithMinimumTlsVersion, - UpdateStages.WithEncryption { + /** + * The template for Cluster update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithSku, UpdateStages.WithIdentity, + UpdateStages.WithMinimumTlsVersion, UpdateStages.WithEncryption { /** * Executes the update request. - * + * * @return the updated resource. */ Cluster apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Cluster apply(Context context); } - /** The Cluster update stages. */ + + /** + * The Cluster update stages. + */ interface UpdateStages { - /** The stage of the Cluster update allowing to specify tags. */ + /** + * The stage of the Cluster update allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the Cluster update allowing to specify sku. */ + + /** + * The stage of the Cluster update allowing to specify sku. + */ interface WithSku { /** * Specifies the sku property: The SKU to create, which affects price, performance, and features.. - * + * * @param sku The SKU to create, which affects price, performance, and features. * @return the next definition stage. */ Update withSku(Sku sku); } - /** The stage of the Cluster update allowing to specify identity. */ + + /** + * The stage of the Cluster update allowing to specify identity. + */ interface WithIdentity { /** * Specifies the identity property: The identity of the resource.. - * + * * @param identity The identity of the resource. * @return the next definition stage. */ Update withIdentity(ManagedServiceIdentity identity); } - /** The stage of the Cluster update allowing to specify minimumTlsVersion. */ + + /** + * The stage of the Cluster update allowing to specify minimumTlsVersion. + */ interface WithMinimumTlsVersion { /** * Specifies the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @param minimumTlsVersion The minimum TLS version for the cluster to support, e.g. '1.2'. * @return the next definition stage. */ Update withMinimumTlsVersion(TlsVersion minimumTlsVersion); } - /** The stage of the Cluster update allowing to specify encryption. */ + + /** + * The stage of the Cluster update allowing to specify encryption. + */ interface WithEncryption { /** * Specifies the encryption property: Encryption-at-rest configuration for the cluster.. - * + * * @param encryption Encryption-at-rest configuration for the cluster. * @return the next definition stage. */ Update withEncryption(ClusterPropertiesEncryption encryption); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Cluster refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterList.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterList.java index b121135d06dd6..021737ed0d4fa 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterList.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a list-all operation. */ +/** + * The response of a list-all operation. + */ @Fluent public final class ClusterList { /* @@ -24,13 +26,15 @@ public final class ClusterList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ClusterList class. */ + /** + * Creates an instance of ClusterList class. + */ public ClusterList() { } /** * Get the value property: List of clusters. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of clusters. - * + * * @param value the value value to set. * @return the ClusterList object itself. */ @@ -50,7 +54,7 @@ public ClusterList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryption.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryption.java index f2567b4f38028..0d5b76fc49162 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryption.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryption.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Encryption-at-rest configuration for the cluster. */ +/** + * Encryption-at-rest configuration for the cluster. + */ @Fluent public final class ClusterPropertiesEncryption { /* @@ -17,14 +19,16 @@ public final class ClusterPropertiesEncryption { @JsonProperty(value = "customerManagedKeyEncryption") private ClusterPropertiesEncryptionCustomerManagedKeyEncryption customerManagedKeyEncryption; - /** Creates an instance of ClusterPropertiesEncryption class. */ + /** + * Creates an instance of ClusterPropertiesEncryption class. + */ public ClusterPropertiesEncryption() { } /** * Get the customerManagedKeyEncryption property: All Customer-managed key encryption properties for the resource. * Set this to an empty object to use Microsoft-managed key encryption. - * + * * @return the customerManagedKeyEncryption value. */ public ClusterPropertiesEncryptionCustomerManagedKeyEncryption customerManagedKeyEncryption() { @@ -34,7 +38,7 @@ public ClusterPropertiesEncryptionCustomerManagedKeyEncryption customerManagedKe /** * Set the customerManagedKeyEncryption property: All Customer-managed key encryption properties for the resource. * Set this to an empty object to use Microsoft-managed key encryption. - * + * * @param customerManagedKeyEncryption the customerManagedKeyEncryption value to set. * @return the ClusterPropertiesEncryption object itself. */ @@ -46,7 +50,7 @@ public ClusterPropertiesEncryption withCustomerManagedKeyEncryption( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryption.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryption.java index 422d57f0fd556..61781faef996b 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryption.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryption.java @@ -8,8 +8,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** - * All Customer-managed key encryption properties for the resource. Set this to an empty object to use Microsoft-managed - * key encryption. + * All Customer-managed key encryption properties for the resource. Set this to an empty object to use + * Microsoft-managed key encryption. */ @Fluent public final class ClusterPropertiesEncryptionCustomerManagedKeyEncryption { @@ -27,14 +27,16 @@ public final class ClusterPropertiesEncryptionCustomerManagedKeyEncryption { @JsonProperty(value = "keyEncryptionKeyUrl") private String keyEncryptionKeyUrl; - /** Creates an instance of ClusterPropertiesEncryptionCustomerManagedKeyEncryption class. */ + /** + * Creates an instance of ClusterPropertiesEncryptionCustomerManagedKeyEncryption class. + */ public ClusterPropertiesEncryptionCustomerManagedKeyEncryption() { } /** * Get the keyEncryptionKeyIdentity property: All identity configuration for Customer-managed key settings defining * which identity should be used to auth to Key Vault. - * + * * @return the keyEncryptionKeyIdentity value. */ public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity keyEncryptionKeyIdentity() { @@ -44,7 +46,7 @@ public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity keyEnc /** * Set the keyEncryptionKeyIdentity property: All identity configuration for Customer-managed key settings defining * which identity should be used to auth to Key Vault. - * + * * @param keyEncryptionKeyIdentity the keyEncryptionKeyIdentity value to set. * @return the ClusterPropertiesEncryptionCustomerManagedKeyEncryption object itself. */ @@ -57,7 +59,7 @@ public ClusterPropertiesEncryptionCustomerManagedKeyEncryption withKeyEncryption /** * Get the keyEncryptionKeyUrl property: Key encryption key Url, versioned only. Ex: * https://contosovault.vault.azure.net/keys/contosokek/562a4bb76b524a1493a6afe8e536ee78. - * + * * @return the keyEncryptionKeyUrl value. */ public String keyEncryptionKeyUrl() { @@ -67,7 +69,7 @@ public String keyEncryptionKeyUrl() { /** * Set the keyEncryptionKeyUrl property: Key encryption key Url, versioned only. Ex: * https://contosovault.vault.azure.net/keys/contosokek/562a4bb76b524a1493a6afe8e536ee78. - * + * * @param keyEncryptionKeyUrl the keyEncryptionKeyUrl value to set. * @return the ClusterPropertiesEncryptionCustomerManagedKeyEncryption object itself. */ @@ -78,7 +80,7 @@ public ClusterPropertiesEncryptionCustomerManagedKeyEncryption withKeyEncryption /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.java index 61305f2c8c000..b8dd4d8c73a56 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.java @@ -26,7 +26,9 @@ public final class ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIde @JsonProperty(value = "identityType") private CmkIdentityType identityType; - /** Creates an instance of ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity class. */ + /** + * Creates an instance of ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity class. + */ public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() { } @@ -34,7 +36,7 @@ public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() { * Get the userAssignedIdentityResourceId property: User assigned identity to use for accessing key encryption key * Url. Ex: /subscriptions/<sub uuid>/resourceGroups/<resource * group>/providers/Microsoft.ManagedIdentity/userAssignedIdentities/myId. - * + * * @return the userAssignedIdentityResourceId value. */ public String userAssignedIdentityResourceId() { @@ -45,12 +47,12 @@ public String userAssignedIdentityResourceId() { * Set the userAssignedIdentityResourceId property: User assigned identity to use for accessing key encryption key * Url. Ex: /subscriptions/<sub uuid>/resourceGroups/<resource * group>/providers/Microsoft.ManagedIdentity/userAssignedIdentities/myId. - * + * * @param userAssignedIdentityResourceId the userAssignedIdentityResourceId value to set. * @return the ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity object itself. */ - public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity withUserAssignedIdentityResourceId( - String userAssignedIdentityResourceId) { + public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity + withUserAssignedIdentityResourceId(String userAssignedIdentityResourceId) { this.userAssignedIdentityResourceId = userAssignedIdentityResourceId; return this; } @@ -58,7 +60,7 @@ public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity withUs /** * Get the identityType property: Only userAssignedIdentity is supported in this API version; other types may be * supported in the future. - * + * * @return the identityType value. */ public CmkIdentityType identityType() { @@ -68,19 +70,19 @@ public CmkIdentityType identityType() { /** * Set the identityType property: Only userAssignedIdentity is supported in this API version; other types may be * supported in the future. - * + * * @param identityType the identityType value to set. * @return the ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity object itself. */ - public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity withIdentityType( - CmkIdentityType identityType) { + public ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity + withIdentityType(CmkIdentityType identityType) { this.identityType = identityType; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterUpdate.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterUpdate.java index e36423281d05f..bd16a5ebd2d97 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterUpdate.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusterUpdate.java @@ -12,7 +12,9 @@ import java.util.List; import java.util.Map; -/** A partial update to the RedisEnterprise cluster. */ +/** + * A partial update to the RedisEnterprise cluster. + */ @Fluent public final class ClusterUpdate { /* @@ -23,7 +25,7 @@ public final class ClusterUpdate { /* * RedisEnterprise cluster properties - * + * * Other properties of the cluster. */ @JsonProperty(value = "properties") @@ -42,13 +44,15 @@ public final class ClusterUpdate { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of ClusterUpdate class. */ + /** + * Creates an instance of ClusterUpdate class. + */ public ClusterUpdate() { } /** * Get the sku property: The SKU to create, which affects price, performance, and features. - * + * * @return the sku value. */ public Sku sku() { @@ -57,7 +61,7 @@ public Sku sku() { /** * Set the sku property: The SKU to create, which affects price, performance, and features. - * + * * @param sku the sku value to set. * @return the ClusterUpdate object itself. */ @@ -68,9 +72,9 @@ public ClusterUpdate withSku(Sku sku) { /** * Get the innerProperties property: RedisEnterprise cluster properties - * - *

Other properties of the cluster. - * + * + * Other properties of the cluster. + * * @return the innerProperties value. */ private ClusterProperties innerProperties() { @@ -79,7 +83,7 @@ private ClusterProperties innerProperties() { /** * Get the identity property: The identity of the resource. - * + * * @return the identity value. */ public ManagedServiceIdentity identity() { @@ -88,7 +92,7 @@ public ManagedServiceIdentity identity() { /** * Set the identity property: The identity of the resource. - * + * * @param identity the identity value to set. * @return the ClusterUpdate object itself. */ @@ -99,7 +103,7 @@ public ClusterUpdate withIdentity(ManagedServiceIdentity identity) { /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -108,7 +112,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the ClusterUpdate object itself. */ @@ -119,7 +123,7 @@ public ClusterUpdate withTags(Map tags) { /** * Get the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @return the minimumTlsVersion value. */ public TlsVersion minimumTlsVersion() { @@ -128,7 +132,7 @@ public TlsVersion minimumTlsVersion() { /** * Set the minimumTlsVersion property: The minimum TLS version for the cluster to support, e.g. '1.2'. - * + * * @param minimumTlsVersion the minimumTlsVersion value to set. * @return the ClusterUpdate object itself. */ @@ -142,7 +146,7 @@ public ClusterUpdate withMinimumTlsVersion(TlsVersion minimumTlsVersion) { /** * Get the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @return the encryption value. */ public ClusterPropertiesEncryption encryption() { @@ -151,7 +155,7 @@ public ClusterPropertiesEncryption encryption() { /** * Set the encryption property: Encryption-at-rest configuration for the cluster. - * + * * @param encryption the encryption value to set. * @return the ClusterUpdate object itself. */ @@ -165,7 +169,7 @@ public ClusterUpdate withEncryption(ClusterPropertiesEncryption encryption) { /** * Get the hostname property: DNS name of the cluster endpoint. - * + * * @return the hostname value. */ public String hostname() { @@ -174,7 +178,7 @@ public String hostname() { /** * Get the provisioningState property: Current provisioning status of the cluster. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -183,7 +187,7 @@ public ProvisioningState provisioningState() { /** * Get the resourceState property: Current resource status of the cluster. - * + * * @return the resourceState value. */ public ResourceState resourceState() { @@ -192,7 +196,7 @@ public ResourceState resourceState() { /** * Get the redisVersion property: Version of redis the cluster supports, e.g. '6'. - * + * * @return the redisVersion value. */ public String redisVersion() { @@ -202,7 +206,7 @@ public String redisVersion() { /** * Get the privateEndpointConnections property: List of private endpoint connections associated with the specified * RedisEnterprise cluster. - * + * * @return the privateEndpointConnections value. */ public List privateEndpointConnections() { @@ -211,7 +215,7 @@ public List privateEndpointConnections() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusteringPolicy.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusteringPolicy.java index 13f694d8a5f06..93a160d494be7 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusteringPolicy.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ClusteringPolicy.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Clustering policy - default is OSSCluster. Specified at create time. */ +/** + * Clustering policy - default is OSSCluster. Specified at create time. + */ public final class ClusteringPolicy extends ExpandableStringEnum { - /** Static value EnterpriseCluster for ClusteringPolicy. */ + /** + * Static value EnterpriseCluster for ClusteringPolicy. + */ public static final ClusteringPolicy ENTERPRISE_CLUSTER = fromString("EnterpriseCluster"); - /** Static value OSSCluster for ClusteringPolicy. */ + /** + * Static value OSSCluster for ClusteringPolicy. + */ public static final ClusteringPolicy OSSCLUSTER = fromString("OSSCluster"); /** * Creates a new instance of ClusteringPolicy value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ClusteringPolicy() { /** * Creates or finds a ClusteringPolicy from its string representation. - * + * * @param name a name to look for. * @return the corresponding ClusteringPolicy. */ @@ -38,7 +44,7 @@ public static ClusteringPolicy fromString(String name) { /** * Gets known ClusteringPolicy values. - * + * * @return known ClusteringPolicy values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/CmkIdentityType.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/CmkIdentityType.java index a5a43a2f9b143..282a382624fc4 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/CmkIdentityType.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/CmkIdentityType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Only userAssignedIdentity is supported in this API version; other types may be supported in the future. */ +/** + * Only userAssignedIdentity is supported in this API version; other types may be supported in the future. + */ public final class CmkIdentityType extends ExpandableStringEnum { - /** Static value systemAssignedIdentity for CmkIdentityType. */ + /** + * Static value systemAssignedIdentity for CmkIdentityType. + */ public static final CmkIdentityType SYSTEM_ASSIGNED_IDENTITY = fromString("systemAssignedIdentity"); - /** Static value userAssignedIdentity for CmkIdentityType. */ + /** + * Static value userAssignedIdentity for CmkIdentityType. + */ public static final CmkIdentityType USER_ASSIGNED_IDENTITY = fromString("userAssignedIdentity"); /** * Creates a new instance of CmkIdentityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public CmkIdentityType() { /** * Creates or finds a CmkIdentityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding CmkIdentityType. */ @@ -38,7 +44,7 @@ public static CmkIdentityType fromString(String name) { /** * Gets known CmkIdentityType values. - * + * * @return known CmkIdentityType values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Database.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Database.java index 93d3dfb8c1c42..c9675b6940c36 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Database.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Database.java @@ -5,45 +5,39 @@ package com.azure.resourcemanager.redisenterprise.models; import com.azure.core.http.rest.Response; -import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.DatabaseInner; import java.util.List; -/** An immutable client-side representation of Database. */ +/** + * An immutable client-side representation of Database. + */ public interface Database { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - /** * Gets the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @return the clientProtocol value. */ Protocol clientProtocol(); @@ -51,42 +45,42 @@ public interface Database { /** * Gets the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @return the port value. */ Integer port(); /** * Gets the provisioningState property: Current provisioning status of the database. - * + * * @return the provisioningState value. */ ProvisioningState provisioningState(); /** * Gets the resourceState property: Current resource status of the database. - * + * * @return the resourceState value. */ ResourceState resourceState(); /** * Gets the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @return the clusteringPolicy value. */ ClusteringPolicy clusteringPolicy(); /** * Gets the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @return the evictionPolicy value. */ EvictionPolicy evictionPolicy(); /** * Gets the persistence property: Persistence settings. - * + * * @return the persistence value. */ Persistence persistence(); @@ -94,227 +88,269 @@ public interface Database { /** * Gets the modules property: Optional set of redis modules to enable in this database - modules can only be added * at creation time. - * + * * @return the modules value. */ List modules(); /** * Gets the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @return the geoReplication value. */ DatabasePropertiesGeoReplication geoReplication(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.DatabaseInner object. - * + * * @return the inner object. */ DatabaseInner innerModel(); - /** The entirety of the Database definition. */ + /** + * The entirety of the Database definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The Database definition stages. */ + + /** + * The Database definition stages. + */ interface DefinitionStages { - /** The first stage of the Database definition. */ + /** + * The first stage of the Database definition. + */ interface Blank extends WithParentResource { } - /** The stage of the Database definition allowing to specify parent resource. */ + + /** + * The stage of the Database definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, clusterName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @return the next definition stage. */ WithCreate withExistingRedisEnterprise(String resourceGroupName, String clusterName); } + /** * The stage of the Database definition which contains all the minimum required properties for the resource to * be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithClientProtocol, - DefinitionStages.WithPort, - DefinitionStages.WithClusteringPolicy, - DefinitionStages.WithEvictionPolicy, - DefinitionStages.WithPersistence, - DefinitionStages.WithModules, - DefinitionStages.WithGeoReplication { + interface WithCreate extends DefinitionStages.WithClientProtocol, DefinitionStages.WithPort, + DefinitionStages.WithClusteringPolicy, DefinitionStages.WithEvictionPolicy, + DefinitionStages.WithPersistence, DefinitionStages.WithModules, DefinitionStages.WithGeoReplication { /** * Executes the create request. - * + * * @return the created resource. */ Database create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Database create(Context context); } - /** The stage of the Database definition allowing to specify clientProtocol. */ + + /** + * The stage of the Database definition allowing to specify clientProtocol. + */ interface WithClientProtocol { /** * Specifies the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or * plaintext redis protocols. Default is TLS-encrypted.. - * + * * @param clientProtocol Specifies whether redis clients can connect using TLS-encrypted or plaintext redis - * protocols. Default is TLS-encrypted. + * protocols. Default is TLS-encrypted. * @return the next definition stage. */ WithCreate withClientProtocol(Protocol clientProtocol); } - /** The stage of the Database definition allowing to specify port. */ + + /** + * The stage of the Database definition allowing to specify port. + */ interface WithPort { /** * Specifies the port property: TCP port of the database endpoint. Specified at create time. Defaults to an * available port.. - * + * * @param port TCP port of the database endpoint. Specified at create time. Defaults to an available port. * @return the next definition stage. */ WithCreate withPort(Integer port); } - /** The stage of the Database definition allowing to specify clusteringPolicy. */ + + /** + * The stage of the Database definition allowing to specify clusteringPolicy. + */ interface WithClusteringPolicy { /** * Specifies the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create * time.. - * + * * @param clusteringPolicy Clustering policy - default is OSSCluster. Specified at create time. * @return the next definition stage. */ WithCreate withClusteringPolicy(ClusteringPolicy clusteringPolicy); } - /** The stage of the Database definition allowing to specify evictionPolicy. */ + + /** + * The stage of the Database definition allowing to specify evictionPolicy. + */ interface WithEvictionPolicy { /** * Specifies the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @param evictionPolicy Redis eviction policy - default is VolatileLRU. * @return the next definition stage. */ WithCreate withEvictionPolicy(EvictionPolicy evictionPolicy); } - /** The stage of the Database definition allowing to specify persistence. */ + + /** + * The stage of the Database definition allowing to specify persistence. + */ interface WithPersistence { /** * Specifies the persistence property: Persistence settings. - * + * * @param persistence Persistence settings. * @return the next definition stage. */ WithCreate withPersistence(Persistence persistence); } - /** The stage of the Database definition allowing to specify modules. */ + + /** + * The stage of the Database definition allowing to specify modules. + */ interface WithModules { /** * Specifies the modules property: Optional set of redis modules to enable in this database - modules can * only be added at creation time.. - * + * * @param modules Optional set of redis modules to enable in this database - modules can only be added at - * creation time. + * creation time. * @return the next definition stage. */ WithCreate withModules(List modules); } - /** The stage of the Database definition allowing to specify geoReplication. */ + + /** + * The stage of the Database definition allowing to specify geoReplication. + */ interface WithGeoReplication { /** * Specifies the geoReplication property: Optional set of properties to configure geo replication for this * database.. - * + * * @param geoReplication Optional set of properties to configure geo replication for this database. * @return the next definition stage. */ WithCreate withGeoReplication(DatabasePropertiesGeoReplication geoReplication); } } + /** * Begins update for the Database resource. - * + * * @return the stage of resource update. */ Database.Update update(); - /** The template for Database update. */ + /** + * The template for Database update. + */ interface Update extends UpdateStages.WithClientProtocol, UpdateStages.WithEvictionPolicy, UpdateStages.WithPersistence { /** * Executes the update request. - * + * * @return the updated resource. */ Database apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Database apply(Context context); } - /** The Database update stages. */ + + /** + * The Database update stages. + */ interface UpdateStages { - /** The stage of the Database update allowing to specify clientProtocol. */ + /** + * The stage of the Database update allowing to specify clientProtocol. + */ interface WithClientProtocol { /** * Specifies the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or * plaintext redis protocols. Default is TLS-encrypted.. - * + * * @param clientProtocol Specifies whether redis clients can connect using TLS-encrypted or plaintext redis - * protocols. Default is TLS-encrypted. + * protocols. Default is TLS-encrypted. * @return the next definition stage. */ Update withClientProtocol(Protocol clientProtocol); } - /** The stage of the Database update allowing to specify evictionPolicy. */ + + /** + * The stage of the Database update allowing to specify evictionPolicy. + */ interface WithEvictionPolicy { /** * Specifies the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @param evictionPolicy Redis eviction policy - default is VolatileLRU. * @return the next definition stage. */ Update withEvictionPolicy(EvictionPolicy evictionPolicy); } - /** The stage of the Database update allowing to specify persistence. */ + + /** + * The stage of the Database update allowing to specify persistence. + */ interface WithPersistence { /** * Specifies the persistence property: Persistence settings. - * + * * @param persistence Persistence settings. * @return the next definition stage. */ Update withPersistence(Persistence persistence); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Database refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -322,7 +358,7 @@ interface WithPersistence { /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -333,7 +369,7 @@ interface WithPersistence { /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return access keys. @@ -342,7 +378,7 @@ interface WithPersistence { /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param parameters Specifies which key to regenerate. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -353,7 +389,7 @@ interface WithPersistence { /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param parameters Specifies which key to regenerate. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -365,7 +401,7 @@ interface WithPersistence { /** * Imports database files to target database. - * + * * @param parameters Storage information for importing into the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -375,7 +411,7 @@ interface WithPersistence { /** * Imports database files to target database. - * + * * @param parameters Storage information for importing into the cluster. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -386,7 +422,7 @@ interface WithPersistence { /** * Exports a database file from target database. - * + * * @param parameters Storage information for exporting into the cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -396,7 +432,7 @@ interface WithPersistence { /** * Exports a database file from target database. - * + * * @param parameters Storage information for exporting into the cluster. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -407,7 +443,7 @@ interface WithPersistence { /** * Forcibly removes the link to the specified database resource. - * + * * @param parameters Information identifying the database to be unlinked. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -417,7 +453,7 @@ interface WithPersistence { /** * Forcibly removes the link to the specified database resource. - * + * * @param parameters Information identifying the database to be unlinked. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -428,7 +464,7 @@ interface WithPersistence { /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param parameters Information identifying the databases to be flushed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -438,7 +474,7 @@ interface WithPersistence { /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param parameters Information identifying the databases to be flushed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseList.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseList.java index 685e20d87c3c3..1ca2c81761ff9 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseList.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The response of a list-all operation. */ +/** + * The response of a list-all operation. + */ @Fluent public final class DatabaseList { /* @@ -24,13 +26,15 @@ public final class DatabaseList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DatabaseList class. */ + /** + * Creates an instance of DatabaseList class. + */ public DatabaseList() { } /** * Get the value property: List of databases. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of databases. - * + * * @param value the value value to set. * @return the DatabaseList object itself. */ @@ -50,7 +54,7 @@ public DatabaseList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page of results. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabasePropertiesGeoReplication.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabasePropertiesGeoReplication.java index 8279a01bed5ce..6cc0a10c95e72 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabasePropertiesGeoReplication.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabasePropertiesGeoReplication.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Optional set of properties to configure geo replication for this database. */ +/** + * Optional set of properties to configure geo replication for this database. + */ @Fluent public final class DatabasePropertiesGeoReplication { /* @@ -23,13 +25,15 @@ public final class DatabasePropertiesGeoReplication { @JsonProperty(value = "linkedDatabases") private List linkedDatabases; - /** Creates an instance of DatabasePropertiesGeoReplication class. */ + /** + * Creates an instance of DatabasePropertiesGeoReplication class. + */ public DatabasePropertiesGeoReplication() { } /** * Get the groupNickname property: Name for the group of linked database resources. - * + * * @return the groupNickname value. */ public String groupNickname() { @@ -38,7 +42,7 @@ public String groupNickname() { /** * Set the groupNickname property: Name for the group of linked database resources. - * + * * @param groupNickname the groupNickname value to set. * @return the DatabasePropertiesGeoReplication object itself. */ @@ -49,7 +53,7 @@ public DatabasePropertiesGeoReplication withGroupNickname(String groupNickname) /** * Get the linkedDatabases property: List of database resources to link with this database. - * + * * @return the linkedDatabases value. */ public List linkedDatabases() { @@ -58,7 +62,7 @@ public List linkedDatabases() { /** * Set the linkedDatabases property: List of database resources to link with this database. - * + * * @param linkedDatabases the linkedDatabases value to set. * @return the DatabasePropertiesGeoReplication object itself. */ @@ -69,7 +73,7 @@ public DatabasePropertiesGeoReplication withLinkedDatabases(List /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseUpdate.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseUpdate.java index 39fcb08b896a7..f91cbdf6d8b5b 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseUpdate.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/DatabaseUpdate.java @@ -9,26 +9,30 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A partial update to the RedisEnterprise database. */ +/** + * A partial update to the RedisEnterprise database. + */ @Fluent public final class DatabaseUpdate { /* * RedisEnterprise database properties - * + * * Properties of the database. */ @JsonProperty(value = "properties") private DatabaseProperties innerProperties; - /** Creates an instance of DatabaseUpdate class. */ + /** + * Creates an instance of DatabaseUpdate class. + */ public DatabaseUpdate() { } /** * Get the innerProperties property: RedisEnterprise database properties - * - *

Properties of the database. - * + * + * Properties of the database. + * * @return the innerProperties value. */ private DatabaseProperties innerProperties() { @@ -38,7 +42,7 @@ private DatabaseProperties innerProperties() { /** * Get the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @return the clientProtocol value. */ public Protocol clientProtocol() { @@ -48,7 +52,7 @@ public Protocol clientProtocol() { /** * Set the clientProtocol property: Specifies whether redis clients can connect using TLS-encrypted or plaintext * redis protocols. Default is TLS-encrypted. - * + * * @param clientProtocol the clientProtocol value to set. * @return the DatabaseUpdate object itself. */ @@ -63,7 +67,7 @@ public DatabaseUpdate withClientProtocol(Protocol clientProtocol) { /** * Get the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @return the port value. */ public Integer port() { @@ -73,7 +77,7 @@ public Integer port() { /** * Set the port property: TCP port of the database endpoint. Specified at create time. Defaults to an available * port. - * + * * @param port the port value to set. * @return the DatabaseUpdate object itself. */ @@ -87,7 +91,7 @@ public DatabaseUpdate withPort(Integer port) { /** * Get the provisioningState property: Current provisioning status of the database. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -96,7 +100,7 @@ public ProvisioningState provisioningState() { /** * Get the resourceState property: Current resource status of the database. - * + * * @return the resourceState value. */ public ResourceState resourceState() { @@ -105,7 +109,7 @@ public ResourceState resourceState() { /** * Get the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @return the clusteringPolicy value. */ public ClusteringPolicy clusteringPolicy() { @@ -114,7 +118,7 @@ public ClusteringPolicy clusteringPolicy() { /** * Set the clusteringPolicy property: Clustering policy - default is OSSCluster. Specified at create time. - * + * * @param clusteringPolicy the clusteringPolicy value to set. * @return the DatabaseUpdate object itself. */ @@ -128,7 +132,7 @@ public DatabaseUpdate withClusteringPolicy(ClusteringPolicy clusteringPolicy) { /** * Get the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @return the evictionPolicy value. */ public EvictionPolicy evictionPolicy() { @@ -137,7 +141,7 @@ public EvictionPolicy evictionPolicy() { /** * Set the evictionPolicy property: Redis eviction policy - default is VolatileLRU. - * + * * @param evictionPolicy the evictionPolicy value to set. * @return the DatabaseUpdate object itself. */ @@ -151,7 +155,7 @@ public DatabaseUpdate withEvictionPolicy(EvictionPolicy evictionPolicy) { /** * Get the persistence property: Persistence settings. - * + * * @return the persistence value. */ public Persistence persistence() { @@ -160,7 +164,7 @@ public Persistence persistence() { /** * Set the persistence property: Persistence settings. - * + * * @param persistence the persistence value to set. * @return the DatabaseUpdate object itself. */ @@ -173,9 +177,9 @@ public DatabaseUpdate withPersistence(Persistence persistence) { } /** - * Get the modules property: Optional set of redis modules to enable in this database - modules can only be added at - * creation time. - * + * Get the modules property: Optional set of redis modules to enable in this database - modules can only be added + * at creation time. + * * @return the modules value. */ public List modules() { @@ -183,9 +187,9 @@ public List modules() { } /** - * Set the modules property: Optional set of redis modules to enable in this database - modules can only be added at - * creation time. - * + * Set the modules property: Optional set of redis modules to enable in this database - modules can only be added + * at creation time. + * * @param modules the modules value to set. * @return the DatabaseUpdate object itself. */ @@ -199,7 +203,7 @@ public DatabaseUpdate withModules(List modules) { /** * Get the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @return the geoReplication value. */ public DatabasePropertiesGeoReplication geoReplication() { @@ -208,7 +212,7 @@ public DatabasePropertiesGeoReplication geoReplication() { /** * Set the geoReplication property: Optional set of properties to configure geo replication for this database. - * + * * @param geoReplication the geoReplication value to set. * @return the DatabaseUpdate object itself. */ @@ -222,7 +226,7 @@ public DatabaseUpdate withGeoReplication(DatabasePropertiesGeoReplication geoRep /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Databases.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Databases.java index f0e707857436e..6b281950fc9ff 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Databases.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Databases.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Databases. */ +/** + * Resource collection API of Databases. + */ public interface Databases { /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,7 +26,7 @@ public interface Databases { /** * Gets all databases in the specified RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -37,7 +39,7 @@ public interface Databases { /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -47,12 +49,12 @@ public interface Databases { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information about a database in a RedisEnterprise cluster along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context); /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -65,7 +67,7 @@ Response getWithResponse( /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -77,7 +79,7 @@ Response getWithResponse( /** * Deletes a single database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -90,7 +92,7 @@ Response getWithResponse( /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -100,12 +102,12 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return access keys along with {@link Response}. */ - Response listKeysWithResponse( - String resourceGroupName, String clusterName, String databaseName, Context context); + Response listKeysWithResponse(String resourceGroupName, String clusterName, String databaseName, + Context context); /** * Retrieves the access keys for the RedisEnterprise database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -118,7 +120,7 @@ Response listKeysWithResponse( /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -128,12 +130,12 @@ Response listKeysWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return access keys. */ - AccessKeys regenerateKey( - String resourceGroupName, String clusterName, String databaseName, RegenerateKeyParameters parameters); + AccessKeys regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters); /** * Regenerates the RedisEnterprise database's access keys. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -144,16 +146,12 @@ AccessKeys regenerateKey( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return access keys. */ - AccessKeys regenerateKey( - String resourceGroupName, - String clusterName, - String databaseName, - RegenerateKeyParameters parameters, - Context context); + AccessKeys regenerateKey(String resourceGroupName, String clusterName, String databaseName, + RegenerateKeyParameters parameters, Context context); /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -162,12 +160,12 @@ AccessKeys regenerateKey( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void importMethod( - String resourceGroupName, String clusterName, String databaseName, ImportClusterParameters parameters); + void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters); /** * Imports database files to target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -177,16 +175,12 @@ void importMethod( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void importMethod( - String resourceGroupName, - String clusterName, - String databaseName, - ImportClusterParameters parameters, - Context context); + void importMethod(String resourceGroupName, String clusterName, String databaseName, + ImportClusterParameters parameters, Context context); /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -199,7 +193,7 @@ void importMethod( /** * Exports a database file from target database. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -209,16 +203,12 @@ void importMethod( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void export( - String resourceGroupName, - String clusterName, - String databaseName, - ExportClusterParameters parameters, + void export(String resourceGroupName, String clusterName, String databaseName, ExportClusterParameters parameters, Context context); /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -227,12 +217,12 @@ void export( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void forceUnlink( - String resourceGroupName, String clusterName, String databaseName, ForceUnlinkParameters parameters); + void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters); /** * Forcibly removes the link to the specified database resource. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -242,16 +232,12 @@ void forceUnlink( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void forceUnlink( - String resourceGroupName, - String clusterName, - String databaseName, - ForceUnlinkParameters parameters, - Context context); + void forceUnlink(String resourceGroupName, String clusterName, String databaseName, + ForceUnlinkParameters parameters, Context context); /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -264,7 +250,7 @@ void forceUnlink( /** * Flushes all the keys in this database and also from its linked databases. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param databaseName The name of the database. @@ -274,12 +260,12 @@ void forceUnlink( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void flush( - String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, Context context); + void flush(String resourceGroupName, String clusterName, String databaseName, FlushParameters parameters, + Context context); /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -290,7 +276,7 @@ void flush( /** * Gets information about a database in a RedisEnterprise cluster. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -302,7 +288,7 @@ void flush( /** * Deletes a single database. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -312,7 +298,7 @@ void flush( /** * Deletes a single database. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -323,7 +309,7 @@ void flush( /** * Begins definition for a new Database resource. - * + * * @param name resource name. * @return the first stage of the new Database definition. */ diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/EvictionPolicy.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/EvictionPolicy.java index 6ccd183fd91c8..409368742a5de 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/EvictionPolicy.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/EvictionPolicy.java @@ -8,35 +8,53 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Redis eviction policy - default is VolatileLRU. */ +/** + * Redis eviction policy - default is VolatileLRU. + */ public final class EvictionPolicy extends ExpandableStringEnum { - /** Static value AllKeysLFU for EvictionPolicy. */ + /** + * Static value AllKeysLFU for EvictionPolicy. + */ public static final EvictionPolicy ALL_KEYS_LFU = fromString("AllKeysLFU"); - /** Static value AllKeysLRU for EvictionPolicy. */ + /** + * Static value AllKeysLRU for EvictionPolicy. + */ public static final EvictionPolicy ALL_KEYS_LRU = fromString("AllKeysLRU"); - /** Static value AllKeysRandom for EvictionPolicy. */ + /** + * Static value AllKeysRandom for EvictionPolicy. + */ public static final EvictionPolicy ALL_KEYS_RANDOM = fromString("AllKeysRandom"); - /** Static value VolatileLRU for EvictionPolicy. */ + /** + * Static value VolatileLRU for EvictionPolicy. + */ public static final EvictionPolicy VOLATILE_LRU = fromString("VolatileLRU"); - /** Static value VolatileLFU for EvictionPolicy. */ + /** + * Static value VolatileLFU for EvictionPolicy. + */ public static final EvictionPolicy VOLATILE_LFU = fromString("VolatileLFU"); - /** Static value VolatileTTL for EvictionPolicy. */ + /** + * Static value VolatileTTL for EvictionPolicy. + */ public static final EvictionPolicy VOLATILE_TTL = fromString("VolatileTTL"); - /** Static value VolatileRandom for EvictionPolicy. */ + /** + * Static value VolatileRandom for EvictionPolicy. + */ public static final EvictionPolicy VOLATILE_RANDOM = fromString("VolatileRandom"); - /** Static value NoEviction for EvictionPolicy. */ + /** + * Static value NoEviction for EvictionPolicy. + */ public static final EvictionPolicy NO_EVICTION = fromString("NoEviction"); /** * Creates a new instance of EvictionPolicy value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -45,7 +63,7 @@ public EvictionPolicy() { /** * Creates or finds a EvictionPolicy from its string representation. - * + * * @param name a name to look for. * @return the corresponding EvictionPolicy. */ @@ -56,7 +74,7 @@ public static EvictionPolicy fromString(String name) { /** * Gets known EvictionPolicy values. - * + * * @return known EvictionPolicy values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ExportClusterParameters.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ExportClusterParameters.java index db429aa7d8e98..7ef65e4963494 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ExportClusterParameters.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ExportClusterParameters.java @@ -10,24 +10,26 @@ /** * Export an RDB file into a target database - * - *

Parameters for a Redis Enterprise export operation. + * + * Parameters for a Redis Enterprise export operation. */ @Fluent public final class ExportClusterParameters { /* * SAS URI for the target directory to export to */ - @JsonProperty(value = "sasUri", required = true) + @JsonProperty(value = "sasUri") private String sasUri; - /** Creates an instance of ExportClusterParameters class. */ + /** + * Creates an instance of ExportClusterParameters class. + */ public ExportClusterParameters() { } /** * Get the sasUri property: SAS URI for the target directory to export to. - * + * * @return the sasUri value. */ public String sasUri() { @@ -36,7 +38,7 @@ public String sasUri() { /** * Set the sasUri property: SAS URI for the target directory to export to. - * + * * @param sasUri the sasUri value to set. * @return the ExportClusterParameters object itself. */ @@ -47,14 +49,13 @@ public ExportClusterParameters withSasUri(String sasUri) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sasUri() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property sasUri in model ExportClusterParameters")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sasUri in model ExportClusterParameters")); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/FlushParameters.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/FlushParameters.java index 6961dd359be76..234309ec17d22 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/FlushParameters.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/FlushParameters.java @@ -9,26 +9,28 @@ import java.util.List; /** - * Flush all the keys from the current and all its linked databases - * - *

Parameters for a Redis Enterprise active geo-replication flush operation. + * Flush all the keys from the current and all its linked databases. + * + * Parameters for a Redis Enterprise active geo-replication flush operation. */ @Fluent public final class FlushParameters { /* - * The resource identifiers of all the other database resources in the georeplication group to be flushed + * The identifiers of all the other database resources in the georeplication group to be flushed. */ @JsonProperty(value = "ids") private List ids; - /** Creates an instance of FlushParameters class. */ + /** + * Creates an instance of FlushParameters class. + */ public FlushParameters() { } /** - * Get the ids property: The resource identifiers of all the other database resources in the georeplication group to - * be flushed. - * + * Get the ids property: The identifiers of all the other database resources in the georeplication group to be + * flushed. + * * @return the ids value. */ public List ids() { @@ -36,9 +38,9 @@ public List ids() { } /** - * Set the ids property: The resource identifiers of all the other database resources in the georeplication group to - * be flushed. - * + * Set the ids property: The identifiers of all the other database resources in the georeplication group to be + * flushed. + * * @param ids the ids value to set. * @return the FlushParameters object itself. */ @@ -49,7 +51,7 @@ public FlushParameters withIds(List ids) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ForceUnlinkParameters.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ForceUnlinkParameters.java index 281a5e554c0f9..6b5a8155d7ea3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ForceUnlinkParameters.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ForceUnlinkParameters.java @@ -11,8 +11,8 @@ /** * Forcibly unlink another database from this database. - * - *

Parameters for a Redis Enterprise Active Geo Replication Force Unlink operation. + * + * Parameters for a Redis Enterprise Active Geo Replication Force Unlink operation. */ @Fluent public final class ForceUnlinkParameters { @@ -22,13 +22,15 @@ public final class ForceUnlinkParameters { @JsonProperty(value = "ids", required = true) private List ids; - /** Creates an instance of ForceUnlinkParameters class. */ + /** + * Creates an instance of ForceUnlinkParameters class. + */ public ForceUnlinkParameters() { } /** * Get the ids property: The resource IDs of the database resources to be unlinked. - * + * * @return the ids value. */ public List ids() { @@ -37,7 +39,7 @@ public List ids() { /** * Set the ids property: The resource IDs of the database resources to be unlinked. - * + * * @param ids the ids value to set. * @return the ForceUnlinkParameters object itself. */ @@ -48,14 +50,13 @@ public ForceUnlinkParameters withIds(List ids) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (ids() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property ids in model ForceUnlinkParameters")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ids in model ForceUnlinkParameters")); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ImportClusterParameters.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ImportClusterParameters.java index 2c1fdd2517c02..1d07aaefd2d20 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ImportClusterParameters.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ImportClusterParameters.java @@ -11,24 +11,26 @@ /** * Import RDB files into a target database - * - *

Parameters for a Redis Enterprise import operation. + * + * Parameters for a Redis Enterprise import operation. */ @Fluent public final class ImportClusterParameters { /* * SAS URIs for the target blobs to import from */ - @JsonProperty(value = "sasUris", required = true) + @JsonProperty(value = "sasUris") private List sasUris; - /** Creates an instance of ImportClusterParameters class. */ + /** + * Creates an instance of ImportClusterParameters class. + */ public ImportClusterParameters() { } /** * Get the sasUris property: SAS URIs for the target blobs to import from. - * + * * @return the sasUris value. */ public List sasUris() { @@ -37,7 +39,7 @@ public List sasUris() { /** * Set the sasUris property: SAS URIs for the target blobs to import from. - * + * * @param sasUris the sasUris value to set. * @return the ImportClusterParameters object itself. */ @@ -48,14 +50,13 @@ public ImportClusterParameters withSasUris(List sasUris) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sasUris() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property sasUris in model ImportClusterParameters")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property sasUris in model ImportClusterParameters")); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkState.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkState.java index f817042457407..f366e6360e5c0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkState.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkState.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** State of the link between the database resources. */ +/** + * State of the link between the database resources. + */ public final class LinkState extends ExpandableStringEnum { - /** Static value Linked for LinkState. */ + /** + * Static value Linked for LinkState. + */ public static final LinkState LINKED = fromString("Linked"); - /** Static value Linking for LinkState. */ + /** + * Static value Linking for LinkState. + */ public static final LinkState LINKING = fromString("Linking"); - /** Static value Unlinking for LinkState. */ + /** + * Static value Unlinking for LinkState. + */ public static final LinkState UNLINKING = fromString("Unlinking"); - /** Static value LinkFailed for LinkState. */ + /** + * Static value LinkFailed for LinkState. + */ public static final LinkState LINK_FAILED = fromString("LinkFailed"); - /** Static value UnlinkFailed for LinkState. */ + /** + * Static value UnlinkFailed for LinkState. + */ public static final LinkState UNLINK_FAILED = fromString("UnlinkFailed"); /** * Creates a new instance of LinkState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public LinkState() { /** * Creates or finds a LinkState from its string representation. - * + * * @param name a name to look for. * @return the corresponding LinkState. */ @@ -47,7 +59,7 @@ public static LinkState fromString(String name) { /** * Gets known LinkState values. - * + * * @return known LinkState values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkedDatabase.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkedDatabase.java index 43a74c90a4d02..a62c569dd62b0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkedDatabase.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LinkedDatabase.java @@ -9,8 +9,8 @@ /** * Linked Database - * - *

Specifies details of a linked database resource. + * + * Specifies details of a linked database resource. */ @Fluent public final class LinkedDatabase { @@ -26,13 +26,15 @@ public final class LinkedDatabase { @JsonProperty(value = "state", access = JsonProperty.Access.WRITE_ONLY) private LinkState state; - /** Creates an instance of LinkedDatabase class. */ + /** + * Creates an instance of LinkedDatabase class. + */ public LinkedDatabase() { } /** * Get the id property: Resource ID of a database resource to link with this database. - * + * * @return the id value. */ public String id() { @@ -41,7 +43,7 @@ public String id() { /** * Set the id property: Resource ID of a database resource to link with this database. - * + * * @param id the id value to set. * @return the LinkedDatabase object itself. */ @@ -52,7 +54,7 @@ public LinkedDatabase withId(String id) { /** * Get the state property: State of the link between the database resources. - * + * * @return the state value. */ public LinkState state() { @@ -61,7 +63,7 @@ public LinkState state() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LocationInfo.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LocationInfo.java deleted file mode 100644 index c07423fea2d1f..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/LocationInfo.java +++ /dev/null @@ -1,80 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** Information about location (for example: features that it supports). */ -@Fluent -public final class LocationInfo { - /* - * Location name - */ - @JsonProperty(value = "location") - private String location; - - /* - * List of capabilities - */ - @JsonProperty(value = "capabilities") - private List capabilities; - - /** Creates an instance of LocationInfo class. */ - public LocationInfo() { - } - - /** - * Get the location property: Location name. - * - * @return the location value. - */ - public String location() { - return this.location; - } - - /** - * Set the location property: Location name. - * - * @param location the location value to set. - * @return the LocationInfo object itself. - */ - public LocationInfo withLocation(String location) { - this.location = location; - return this; - } - - /** - * Get the capabilities property: List of capabilities. - * - * @return the capabilities value. - */ - public List capabilities() { - return this.capabilities; - } - - /** - * Set the capabilities property: List of capabilities. - * - * @param capabilities the capabilities value to set. - * @return the LocationInfo object itself. - */ - public LocationInfo withCapabilities(List capabilities) { - this.capabilities = capabilities; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (capabilities() != null) { - capabilities().forEach(e -> e.validate()); - } - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentity.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentity.java index cf6427594f7f5..060f6a8036bc9 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentity.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentity.java @@ -11,7 +11,9 @@ import java.util.Map; import java.util.UUID; -/** Managed service identity (system assigned and/or user assigned identities). */ +/** + * Managed service identity (system assigned and/or user assigned identities). + */ @Fluent public final class ManagedServiceIdentity { /* @@ -37,21 +39,24 @@ public final class ManagedServiceIdentity { /* * The set of user assigned identities associated with the resource. The userAssignedIdentities dictionary keys * will be ARM resource ids in the form: - * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. + * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/ + * userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. */ @JsonProperty(value = "userAssignedIdentities") @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map userAssignedIdentities; - /** Creates an instance of ManagedServiceIdentity class. */ + /** + * Creates an instance of ManagedServiceIdentity class. + */ public ManagedServiceIdentity() { } /** * Get the principalId property: The service principal ID of the system assigned identity. This property will only * be provided for a system assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -59,9 +64,9 @@ public UUID principalId() { } /** - * Get the tenantId property: The tenant ID of the system assigned identity. This property will only be provided for - * a system assigned identity. - * + * Get the tenantId property: The tenant ID of the system assigned identity. This property will only be provided + * for a system assigned identity. + * * @return the tenantId value. */ public UUID tenantId() { @@ -71,7 +76,7 @@ public UUID tenantId() { /** * Get the type property: Type of managed service identity (where both SystemAssigned and UserAssigned types are * allowed). - * + * * @return the type value. */ public ManagedServiceIdentityType type() { @@ -81,7 +86,7 @@ public ManagedServiceIdentityType type() { /** * Set the type property: Type of managed service identity (where both SystemAssigned and UserAssigned types are * allowed). - * + * * @param type the type value to set. * @return the ManagedServiceIdentity object itself. */ @@ -95,7 +100,7 @@ public ManagedServiceIdentity withType(ManagedServiceIdentityType type) { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @return the userAssignedIdentities value. */ public Map userAssignedIdentities() { @@ -107,7 +112,7 @@ public Map userAssignedIdentities() { * userAssignedIdentities dictionary keys will be ARM resource ids in the form: * '/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ManagedIdentity/userAssignedIdentities/{identityName}. * The dictionary values can be empty objects ({}) in requests. - * + * * @param userAssignedIdentities the userAssignedIdentities value to set. * @return the ManagedServiceIdentity object itself. */ @@ -118,24 +123,20 @@ public ManagedServiceIdentity withUserAssignedIdentities(Map { - if (e != null) { - e.validate(); - } - }); + userAssignedIdentities().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentityType.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentityType.java index b82525ecbff7d..1989490672c03 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentityType.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ManagedServiceIdentityType.java @@ -8,24 +8,34 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Type of managed service identity (where both SystemAssigned and UserAssigned types are allowed). */ +/** + * Type of managed service identity (where both SystemAssigned and UserAssigned types are allowed). + */ public final class ManagedServiceIdentityType extends ExpandableStringEnum { - /** Static value None for ManagedServiceIdentityType. */ + /** + * Static value None for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType NONE = fromString("None"); - /** Static value SystemAssigned for ManagedServiceIdentityType. */ + /** + * Static value SystemAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType SYSTEM_ASSIGNED = fromString("SystemAssigned"); - /** Static value UserAssigned for ManagedServiceIdentityType. */ + /** + * Static value UserAssigned for ManagedServiceIdentityType. + */ public static final ManagedServiceIdentityType USER_ASSIGNED = fromString("UserAssigned"); - /** Static value SystemAssigned, UserAssigned for ManagedServiceIdentityType. */ - public static final ManagedServiceIdentityType SYSTEM_ASSIGNED_USER_ASSIGNED = - fromString("SystemAssigned, UserAssigned"); + /** + * Static value SystemAssigned, UserAssigned for ManagedServiceIdentityType. + */ + public static final ManagedServiceIdentityType SYSTEM_ASSIGNED_USER_ASSIGNED + = fromString("SystemAssigned, UserAssigned"); /** * Creates a new instance of ManagedServiceIdentityType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -34,7 +44,7 @@ public ManagedServiceIdentityType() { /** * Creates or finds a ManagedServiceIdentityType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ManagedServiceIdentityType. */ @@ -45,7 +55,7 @@ public static ManagedServiceIdentityType fromString(String name) { /** * Gets known ManagedServiceIdentityType values. - * + * * @return known ManagedServiceIdentityType values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Module.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Module.java index 69320e7ecbf91..b4988d137a297 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Module.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Module.java @@ -10,8 +10,8 @@ /** * Module settings - * - *

Specifies configuration of a redis module. + * + * Specifies configuration of a redis module. */ @Fluent public final class Module { @@ -33,13 +33,15 @@ public final class Module { @JsonProperty(value = "version", access = JsonProperty.Access.WRITE_ONLY) private String version; - /** Creates an instance of Module class. */ + /** + * Creates an instance of Module class. + */ public Module() { } /** * Get the name property: The name of the module, e.g. 'RedisBloom', 'RediSearch', 'RedisTimeSeries'. - * + * * @return the name value. */ public String name() { @@ -48,7 +50,7 @@ public String name() { /** * Set the name property: The name of the module, e.g. 'RedisBloom', 'RediSearch', 'RedisTimeSeries'. - * + * * @param name the name value to set. * @return the Module object itself. */ @@ -59,7 +61,7 @@ public Module withName(String name) { /** * Get the args property: Configuration options for the module, e.g. 'ERROR_RATE 0.01 INITIAL_SIZE 400'. - * + * * @return the args value. */ public String args() { @@ -68,7 +70,7 @@ public String args() { /** * Set the args property: Configuration options for the module, e.g. 'ERROR_RATE 0.01 INITIAL_SIZE 400'. - * + * * @param args the args value to set. * @return the Module object itself. */ @@ -79,7 +81,7 @@ public Module withArgs(String args) { /** * Get the version property: The version of the module, e.g. '1.0'. - * + * * @return the version value. */ public String version() { @@ -88,7 +90,7 @@ public String version() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operation.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operation.java index 597cc8a4f9db1..1064946ab9f5a 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operation.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operation.java @@ -6,12 +6,14 @@ import com.azure.resourcemanager.redisenterprise.fluent.models.OperationInner; -/** An immutable client-side representation of Operation. */ +/** + * An immutable client-side representation of Operation. + */ public interface Operation { /** * Gets the name property: The name of the operation, as per Resource-Based Access Control (RBAC). Examples: * "Microsoft.Compute/virtualMachines/write", "Microsoft.Compute/virtualMachines/capture/action". - * + * * @return the name value. */ String name(); @@ -19,14 +21,14 @@ public interface Operation { /** * Gets the isDataAction property: Whether the operation applies to data-plane. This is "true" for data-plane * operations and "false" for ARM/control-plane operations. - * + * * @return the isDataAction value. */ Boolean isDataAction(); /** * Gets the display property: Localized display information for this particular operation. - * + * * @return the display value. */ OperationDisplay display(); @@ -34,7 +36,7 @@ public interface Operation { /** * Gets the origin property: The intended executor of the operation; as in Resource Based Access Control (RBAC) and * audit logs UX. Default value is "user,system". - * + * * @return the origin value. */ Origin origin(); @@ -42,14 +44,14 @@ public interface Operation { /** * Gets the actionType property: Enum. Indicates the action type. "Internal" refers to actions that are for internal * only APIs. - * + * * @return the actionType value. */ ActionType actionType(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.OperationInner object. - * + * * @return the inner object. */ OperationInner innerModel(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationDisplay.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationDisplay.java index 75df33a0c397a..be3c0ea2bf625 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationDisplay.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationDisplay.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Localized display information for this particular operation. */ +/** + * Localized display information for this particular operation. + */ @Immutable public final class OperationDisplay { /* @@ -37,14 +39,16 @@ public final class OperationDisplay { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of OperationDisplay class. */ + /** + * Creates an instance of OperationDisplay class. + */ public OperationDisplay() { } /** * Get the provider property: The localized friendly form of the resource provider name, e.g. "Microsoft Monitoring * Insights" or "Microsoft Compute". - * + * * @return the provider value. */ public String provider() { @@ -54,7 +58,7 @@ public String provider() { /** * Get the resource property: The localized friendly name of the resource type related to this operation. E.g. * "Virtual Machines" or "Job Schedule Collections". - * + * * @return the resource value. */ public String resource() { @@ -64,7 +68,7 @@ public String resource() { /** * Get the operation property: The concise, localized friendly name for the operation; suitable for dropdowns. E.g. * "Create or Update Virtual Machine", "Restart Virtual Machine". - * + * * @return the operation value. */ public String operation() { @@ -74,7 +78,7 @@ public String operation() { /** * Get the description property: The short, localized friendly description of the operation; suitable for tool tips * and detailed views. - * + * * @return the description value. */ public String description() { @@ -83,7 +87,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationListResult.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationListResult.java index 7cc56e480a448..9fbf3132ce519 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationListResult.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationListResult.java @@ -10,8 +10,8 @@ import java.util.List; /** - * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set of - * results. + * A list of REST API operations supported by an Azure Resource Provider. It contains an URL link to get the next set + * of results. */ @Immutable public final class OperationListResult { @@ -27,13 +27,15 @@ public final class OperationListResult { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of OperationListResult class. */ + /** + * Creates an instance of OperationListResult class. + */ public OperationListResult() { } /** * Get the value property: List of operations supported by the resource provider. - * + * * @return the value value. */ public List value() { @@ -42,7 +44,7 @@ public List value() { /** * Get the nextLink property: URL to get the next set of operation list results (if there are any). - * + * * @return the nextLink value. */ public String nextLink() { @@ -51,7 +53,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationStatus.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationStatus.java index c51babac7c554..db3410c9fbc8a 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationStatus.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationStatus.java @@ -7,55 +7,57 @@ import com.azure.core.management.exception.ManagementError; import com.azure.resourcemanager.redisenterprise.fluent.models.OperationStatusInner; -/** An immutable client-side representation of OperationStatus. */ +/** + * An immutable client-side representation of OperationStatus. + */ public interface OperationStatus { /** * Gets the id property: The operation's unique id. - * + * * @return the id value. */ String id(); /** * Gets the name property: The operation's name. - * + * * @return the name value. */ String name(); /** * Gets the startTime property: The start time of the operation. - * + * * @return the startTime value. */ String startTime(); /** * Gets the endTime property: The end time of the operation. - * + * * @return the endTime value. */ String endTime(); /** * Gets the status property: The current status of the operation. - * + * * @return the status value. */ String status(); /** * Gets the error property: Error response - * - *

Error response describing why the operation failed. - * + * + * Error response describing why the operation failed. + * * @return the error value. */ ManagementError error(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.OperationStatusInner object. - * + * * @return the inner object. */ OperationStatusInner innerModel(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operations.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operations.java index 95ad8762dacd0..a4d1030d23a5d 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operations.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Operations.java @@ -7,27 +7,29 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Operations. */ +/** + * Resource collection API of Operations. + */ public interface Operations { /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Lists all of the available REST API operations of the Microsoft.Cache provider. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with {@link - * PagedIterable}. + * @return a list of REST API operations supported by an Azure Resource Provider as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationsStatus.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationsStatus.java index bc89ca67af357..efec7d8777fc6 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationsStatus.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/OperationsStatus.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of OperationsStatus. */ +/** + * Resource collection API of OperationsStatus. + */ public interface OperationsStatus { /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @param context The context to associate with this operation. @@ -24,7 +26,7 @@ public interface OperationsStatus { /** * Gets the status of operation. - * + * * @param location The name of Azure region. * @param operationId The ID of an ongoing async operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Origin.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Origin.java index 8ffed4685bd62..3cb9480e94cfd 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Origin.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Origin.java @@ -13,18 +13,24 @@ * is "user,system". */ public final class Origin extends ExpandableStringEnum { - /** Static value user for Origin. */ + /** + * Static value user for Origin. + */ public static final Origin USER = fromString("user"); - /** Static value system for Origin. */ + /** + * Static value system for Origin. + */ public static final Origin SYSTEM = fromString("system"); - /** Static value user,system for Origin. */ + /** + * Static value user,system for Origin. + */ public static final Origin USER_SYSTEM = fromString("user,system"); /** * Creates a new instance of Origin value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +39,7 @@ public Origin() { /** * Creates or finds a Origin from its string representation. - * + * * @param name a name to look for. * @return the corresponding Origin. */ @@ -44,7 +50,7 @@ public static Origin fromString(String name) { /** * Gets known Origin values. - * + * * @return known Origin values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Persistence.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Persistence.java index 03eb7fbf2f419..8d952a40b163c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Persistence.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Persistence.java @@ -9,8 +9,8 @@ /** * Persistence settings - * - *

Persistence-related configuration for the RedisEnterprise database. + * + * Persistence-related configuration for the RedisEnterprise database. */ @Fluent public final class Persistence { @@ -38,13 +38,15 @@ public final class Persistence { @JsonProperty(value = "rdbFrequency") private RdbFrequency rdbFrequency; - /** Creates an instance of Persistence class. */ + /** + * Creates an instance of Persistence class. + */ public Persistence() { } /** * Get the aofEnabled property: Sets whether AOF is enabled. - * + * * @return the aofEnabled value. */ public Boolean aofEnabled() { @@ -53,7 +55,7 @@ public Boolean aofEnabled() { /** * Set the aofEnabled property: Sets whether AOF is enabled. - * + * * @param aofEnabled the aofEnabled value to set. * @return the Persistence object itself. */ @@ -64,7 +66,7 @@ public Persistence withAofEnabled(Boolean aofEnabled) { /** * Get the rdbEnabled property: Sets whether RDB is enabled. - * + * * @return the rdbEnabled value. */ public Boolean rdbEnabled() { @@ -73,7 +75,7 @@ public Boolean rdbEnabled() { /** * Set the rdbEnabled property: Sets whether RDB is enabled. - * + * * @param rdbEnabled the rdbEnabled value to set. * @return the Persistence object itself. */ @@ -84,7 +86,7 @@ public Persistence withRdbEnabled(Boolean rdbEnabled) { /** * Get the aofFrequency property: Sets the frequency at which data is written to disk. - * + * * @return the aofFrequency value. */ public AofFrequency aofFrequency() { @@ -93,7 +95,7 @@ public AofFrequency aofFrequency() { /** * Set the aofFrequency property: Sets the frequency at which data is written to disk. - * + * * @param aofFrequency the aofFrequency value to set. * @return the Persistence object itself. */ @@ -104,7 +106,7 @@ public Persistence withAofFrequency(AofFrequency aofFrequency) { /** * Get the rdbFrequency property: Sets the frequency at which a snapshot of the database is created. - * + * * @return the rdbFrequency value. */ public RdbFrequency rdbFrequency() { @@ -113,7 +115,7 @@ public RdbFrequency rdbFrequency() { /** * Set the rdbFrequency property: Sets the frequency at which a snapshot of the database is created. - * + * * @param rdbFrequency the rdbFrequency value to set. * @return the Persistence object itself. */ @@ -124,7 +126,7 @@ public Persistence withRdbFrequency(RdbFrequency rdbFrequency) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpoint.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpoint.java index d02e68409dc21..0afd32e9496c0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpoint.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpoint.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Private Endpoint resource. */ +/** + * The Private Endpoint resource. + */ @Immutable public final class PrivateEndpoint { /* @@ -16,13 +18,15 @@ public final class PrivateEndpoint { @JsonProperty(value = "id", access = JsonProperty.Access.WRITE_ONLY) private String id; - /** Creates an instance of PrivateEndpoint class. */ + /** + * Creates an instance of PrivateEndpoint class. + */ public PrivateEndpoint() { } /** * Get the id property: The ARM identifier for Private Endpoint. - * + * * @return the id value. */ public String id() { @@ -31,7 +35,7 @@ public String id() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnection.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnection.java index d9c1082fad26c..c5e995fae6b15 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnection.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnection.java @@ -4,43 +4,37 @@ package com.azure.resourcemanager.redisenterprise.models; -import com.azure.core.management.SystemData; import com.azure.core.util.Context; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner; -/** An immutable client-side representation of PrivateEndpointConnection. */ +/** + * An immutable client-side representation of PrivateEndpointConnection. + */ public interface PrivateEndpointConnection { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - /** * Gets the privateEndpoint property: The resource of private end point. - * + * * @return the privateEndpoint value. */ PrivateEndpoint privateEndpoint(); @@ -48,52 +42,63 @@ public interface PrivateEndpointConnection { /** * Gets the privateLinkServiceConnectionState property: A collection of information about the state of the * connection between service consumer and provider. - * + * * @return the privateLinkServiceConnectionState value. */ PrivateLinkServiceConnectionState privateLinkServiceConnectionState(); /** * Gets the provisioningState property: The provisioning state of the private endpoint connection resource. - * + * * @return the provisioningState value. */ PrivateEndpointConnectionProvisioningState provisioningState(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner object. - * + * * @return the inner object. */ PrivateEndpointConnectionInner innerModel(); - /** The entirety of the PrivateEndpointConnection definition. */ + /** + * The entirety of the PrivateEndpointConnection definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The PrivateEndpointConnection definition stages. */ + + /** + * The PrivateEndpointConnection definition stages. + */ interface DefinitionStages { - /** The first stage of the PrivateEndpointConnection definition. */ + /** + * The first stage of the PrivateEndpointConnection definition. + */ interface Blank extends WithParentResource { } - /** The stage of the PrivateEndpointConnection definition allowing to specify parent resource. */ + + /** + * The stage of the PrivateEndpointConnection definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies resourceGroupName, clusterName. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @return the next definition stage. */ WithCreate withExistingRedisEnterprise(String resourceGroupName, String clusterName); } + /** * The stage of the PrivateEndpointConnection definition which contains all the minimum required properties for * the resource to be created, but also allows for any other optional properties to be specified. @@ -102,29 +107,33 @@ interface WithCreate extends DefinitionStages.WithPrivateEndpoint, DefinitionStages.WithPrivateLinkServiceConnectionState { /** * Executes the create request. - * + * * @return the created resource. */ PrivateEndpointConnection create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ PrivateEndpointConnection create(Context context); } - /** The stage of the PrivateEndpointConnection definition allowing to specify privateEndpoint. */ + + /** + * The stage of the PrivateEndpointConnection definition allowing to specify privateEndpoint. + */ interface WithPrivateEndpoint { /** * Specifies the privateEndpoint property: The resource of private end point.. - * + * * @param privateEndpoint The resource of private end point. * @return the next definition stage. */ WithCreate withPrivateEndpoint(PrivateEndpoint privateEndpoint); } + /** * The stage of the PrivateEndpointConnection definition allowing to specify privateLinkServiceConnectionState. */ @@ -132,75 +141,87 @@ interface WithPrivateLinkServiceConnectionState { /** * Specifies the privateLinkServiceConnectionState property: A collection of information about the state of * the connection between service consumer and provider.. - * + * * @param privateLinkServiceConnectionState A collection of information about the state of the connection - * between service consumer and provider. + * between service consumer and provider. * @return the next definition stage. */ WithCreate withPrivateLinkServiceConnectionState( PrivateLinkServiceConnectionState privateLinkServiceConnectionState); } } + /** * Begins update for the PrivateEndpointConnection resource. - * + * * @return the stage of resource update. */ PrivateEndpointConnection.Update update(); - /** The template for PrivateEndpointConnection update. */ + /** + * The template for PrivateEndpointConnection update. + */ interface Update extends UpdateStages.WithPrivateEndpoint, UpdateStages.WithPrivateLinkServiceConnectionState { /** * Executes the update request. - * + * * @return the updated resource. */ PrivateEndpointConnection apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ PrivateEndpointConnection apply(Context context); } - /** The PrivateEndpointConnection update stages. */ + + /** + * The PrivateEndpointConnection update stages. + */ interface UpdateStages { - /** The stage of the PrivateEndpointConnection update allowing to specify privateEndpoint. */ + /** + * The stage of the PrivateEndpointConnection update allowing to specify privateEndpoint. + */ interface WithPrivateEndpoint { /** * Specifies the privateEndpoint property: The resource of private end point.. - * + * * @param privateEndpoint The resource of private end point. * @return the next definition stage. */ Update withPrivateEndpoint(PrivateEndpoint privateEndpoint); } - /** The stage of the PrivateEndpointConnection update allowing to specify privateLinkServiceConnectionState. */ + + /** + * The stage of the PrivateEndpointConnection update allowing to specify privateLinkServiceConnectionState. + */ interface WithPrivateLinkServiceConnectionState { /** * Specifies the privateLinkServiceConnectionState property: A collection of information about the state of * the connection between service consumer and provider.. - * + * * @param privateLinkServiceConnectionState A collection of information about the state of the connection - * between service consumer and provider. + * between service consumer and provider. * @return the next definition stage. */ Update withPrivateLinkServiceConnectionState( PrivateLinkServiceConnectionState privateLinkServiceConnectionState); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ PrivateEndpointConnection refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnectionListResult.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnectionListResult.java index 345835b415ba9..54bf2339054a4 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnectionListResult.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnectionListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of private endpoint connection associated with the specified storage account. */ +/** + * List of private endpoint connection associated with the specified storage account. + */ @Fluent public final class PrivateEndpointConnectionListResult { /* @@ -18,13 +20,15 @@ public final class PrivateEndpointConnectionListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of PrivateEndpointConnectionListResult class. */ + /** + * Creates an instance of PrivateEndpointConnectionListResult class. + */ public PrivateEndpointConnectionListResult() { } /** * Get the value property: Array of private endpoint connections. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Array of private endpoint connections. - * + * * @param value the value value to set. * @return the PrivateEndpointConnectionListResult object itself. */ @@ -44,7 +48,7 @@ public PrivateEndpointConnectionListResult withValue(List { - /** Static value Succeeded for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Succeeded for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Creating for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Creating for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState CREATING = fromString("Creating"); - /** Static value Deleting for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Deleting for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState DELETING = fromString("Deleting"); - /** Static value Failed for PrivateEndpointConnectionProvisioningState. */ + /** + * Static value Failed for PrivateEndpointConnectionProvisioningState. + */ public static final PrivateEndpointConnectionProvisioningState FAILED = fromString("Failed"); /** * Creates a new instance of PrivateEndpointConnectionProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -34,7 +44,7 @@ public PrivateEndpointConnectionProvisioningState() { /** * Creates or finds a PrivateEndpointConnectionProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding PrivateEndpointConnectionProvisioningState. */ @@ -45,7 +55,7 @@ public static PrivateEndpointConnectionProvisioningState fromString(String name) /** * Gets known PrivateEndpointConnectionProvisioningState values. - * + * * @return known PrivateEndpointConnectionProvisioningState values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnections.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnections.java index 04d8334ce2964..cb17e60dc6d2e 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnections.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointConnections.java @@ -8,24 +8,26 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of PrivateEndpointConnections. */ +/** + * Resource collection API of PrivateEndpointConnections. + */ public interface PrivateEndpointConnections { /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ PagedIterable list(String resourceGroupName, String clusterName); /** * Lists all the private endpoint connections associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -33,34 +35,34 @@ public interface PrivateEndpointConnections { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of private endpoint connection associated with the specified storage account as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ PagedIterable list(String resourceGroupName, String clusterName, Context context); /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + Response getWithResponse(String resourceGroupName, String clusterName, + String privateEndpointConnectionName, Context context); /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -70,61 +72,59 @@ Response getWithResponse( /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. - * @param context The context to associate with this operation. + * resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. */ - Response deleteWithResponse( - String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); + void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName); /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param privateEndpointConnectionName The name of the private endpoint connection associated with the Azure - * resource. + * resource. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName); + void delete(String resourceGroupName, String clusterName, String privateEndpointConnectionName, Context context); /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response}. */ PrivateEndpointConnection getById(String id); /** * Gets the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with {@link - * Response}. + * @return the specified private endpoint connection associated with the RedisEnterprise cluster along with + * {@link Response}. */ Response getByIdWithResponse(String id, Context context); /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -134,19 +134,18 @@ Response deleteWithResponse( /** * Deletes the specified private endpoint connection associated with the RedisEnterprise cluster. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. */ - Response deleteByIdWithResponse(String id, Context context); + void deleteByIdWithResponse(String id, Context context); /** * Begins definition for a new PrivateEndpointConnection resource. - * + * * @param name resource name. * @return the first stage of the new PrivateEndpointConnection definition. */ diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointServiceConnectionStatus.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointServiceConnectionStatus.java index 87cffd02b839d..62834ab94a2a1 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointServiceConnectionStatus.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateEndpointServiceConnectionStatus.java @@ -8,21 +8,29 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The private endpoint connection status. */ +/** + * The private endpoint connection status. + */ public final class PrivateEndpointServiceConnectionStatus extends ExpandableStringEnum { - /** Static value Pending for PrivateEndpointServiceConnectionStatus. */ + /** + * Static value Pending for PrivateEndpointServiceConnectionStatus. + */ public static final PrivateEndpointServiceConnectionStatus PENDING = fromString("Pending"); - /** Static value Approved for PrivateEndpointServiceConnectionStatus. */ + /** + * Static value Approved for PrivateEndpointServiceConnectionStatus. + */ public static final PrivateEndpointServiceConnectionStatus APPROVED = fromString("Approved"); - /** Static value Rejected for PrivateEndpointServiceConnectionStatus. */ + /** + * Static value Rejected for PrivateEndpointServiceConnectionStatus. + */ public static final PrivateEndpointServiceConnectionStatus REJECTED = fromString("Rejected"); /** * Creates a new instance of PrivateEndpointServiceConnectionStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -31,7 +39,7 @@ public PrivateEndpointServiceConnectionStatus() { /** * Creates or finds a PrivateEndpointServiceConnectionStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding PrivateEndpointServiceConnectionStatus. */ @@ -42,7 +50,7 @@ public static PrivateEndpointServiceConnectionStatus fromString(String name) { /** * Gets known PrivateEndpointServiceConnectionStatus values. - * + * * @return known PrivateEndpointServiceConnectionStatus values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResource.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResource.java index dfbee46525677..969720bb90a0a 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResource.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResource.java @@ -4,64 +4,58 @@ package com.azure.resourcemanager.redisenterprise.models; -import com.azure.core.management.SystemData; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceInner; import java.util.List; -/** An immutable client-side representation of PrivateLinkResource. */ +/** + * An immutable client-side representation of PrivateLinkResource. + */ public interface PrivateLinkResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); - /** - * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * - * @return the systemData value. - */ - SystemData systemData(); - /** * Gets the groupId property: The private link resource group id. - * + * * @return the groupId value. */ String groupId(); /** * Gets the requiredMembers property: The private link resource required member names. - * + * * @return the requiredMembers value. */ List requiredMembers(); /** * Gets the requiredZoneNames property: The private link resource Private link DNS zone name. - * + * * @return the requiredZoneNames value. */ List requiredZoneNames(); /** * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceInner object. - * + * * @return the inner object. */ PrivateLinkResourceInner innerModel(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResourceListResult.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResourceListResult.java index 51465789943c0..892a0f1390faf 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResourceListResult.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResourceListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of private link resources. */ +/** + * A list of private link resources. + */ @Fluent public final class PrivateLinkResourceListResult { /* @@ -18,13 +20,15 @@ public final class PrivateLinkResourceListResult { @JsonProperty(value = "value") private List value; - /** Creates an instance of PrivateLinkResourceListResult class. */ + /** + * Creates an instance of PrivateLinkResourceListResult class. + */ public PrivateLinkResourceListResult() { } /** * Get the value property: Array of private link resources. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: Array of private link resources. - * + * * @param value the value value to set. * @return the PrivateLinkResourceListResult object itself. */ @@ -44,7 +48,7 @@ public PrivateLinkResourceListResult withValue(List va /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResources.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResources.java index ed9d89094ddb8..9587a575981dd 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResources.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkResources.java @@ -7,24 +7,26 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of PrivateLinkResources. */ +/** + * Resource collection API of PrivateLinkResources. + */ public interface PrivateLinkResources { /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ PagedIterable listByCluster(String resourceGroupName, String clusterName); /** * Gets the private link resources that need to be created for a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -32,7 +34,7 @@ public interface PrivateLinkResources { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the private link resources that need to be created for a RedisEnterprise cluster as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ PagedIterable listByCluster(String resourceGroupName, String clusterName, Context context); } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkServiceConnectionState.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkServiceConnectionState.java index 1d20e5e9160ad..cdfb90ce20b3e 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkServiceConnectionState.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/PrivateLinkServiceConnectionState.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** A collection of information about the state of the connection between service consumer and provider. */ +/** + * A collection of information about the state of the connection between service consumer and provider. + */ @Fluent public final class PrivateLinkServiceConnectionState { /* @@ -28,14 +30,16 @@ public final class PrivateLinkServiceConnectionState { @JsonProperty(value = "actionsRequired") private String actionsRequired; - /** Creates an instance of PrivateLinkServiceConnectionState class. */ + /** + * Creates an instance of PrivateLinkServiceConnectionState class. + */ public PrivateLinkServiceConnectionState() { } /** * Get the status property: Indicates whether the connection has been Approved/Rejected/Removed by the owner of the * service. - * + * * @return the status value. */ public PrivateEndpointServiceConnectionStatus status() { @@ -45,7 +49,7 @@ public PrivateEndpointServiceConnectionStatus status() { /** * Set the status property: Indicates whether the connection has been Approved/Rejected/Removed by the owner of the * service. - * + * * @param status the status value to set. * @return the PrivateLinkServiceConnectionState object itself. */ @@ -56,7 +60,7 @@ public PrivateLinkServiceConnectionState withStatus(PrivateEndpointServiceConnec /** * Get the description property: The reason for approval/rejection of the connection. - * + * * @return the description value. */ public String description() { @@ -65,7 +69,7 @@ public String description() { /** * Set the description property: The reason for approval/rejection of the connection. - * + * * @param description the description value to set. * @return the PrivateLinkServiceConnectionState object itself. */ @@ -77,7 +81,7 @@ public PrivateLinkServiceConnectionState withDescription(String description) { /** * Get the actionsRequired property: A message indicating if changes on the service provider require any updates on * the consumer. - * + * * @return the actionsRequired value. */ public String actionsRequired() { @@ -87,7 +91,7 @@ public String actionsRequired() { /** * Set the actionsRequired property: A message indicating if changes on the service provider require any updates on * the consumer. - * + * * @param actionsRequired the actionsRequired value to set. * @return the PrivateLinkServiceConnectionState object itself. */ @@ -98,7 +102,7 @@ public PrivateLinkServiceConnectionState withActionsRequired(String actionsRequi /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Protocol.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Protocol.java index a57dda4ba7e9f..b47a4d728a398 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Protocol.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Protocol.java @@ -13,15 +13,19 @@ * TLS-encrypted. */ public final class Protocol extends ExpandableStringEnum { - /** Static value Encrypted for Protocol. */ + /** + * Static value Encrypted for Protocol. + */ public static final Protocol ENCRYPTED = fromString("Encrypted"); - /** Static value Plaintext for Protocol. */ + /** + * Static value Plaintext for Protocol. + */ public static final Protocol PLAINTEXT = fromString("Plaintext"); /** * Creates a new instance of Protocol value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +34,7 @@ public Protocol() { /** * Creates or finds a Protocol from its string representation. - * + * * @param name a name to look for. * @return the corresponding Protocol. */ @@ -41,7 +45,7 @@ public static Protocol fromString(String name) { /** * Gets known Protocol values. - * + * * @return known Protocol values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ProvisioningState.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ProvisioningState.java index d02bbe974cd59..b1604d4705d6c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ProvisioningState.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ProvisioningState.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current provisioning status. */ +/** + * Current provisioning status. + */ public final class ProvisioningState extends ExpandableStringEnum { - /** Static value Succeeded for ProvisioningState. */ + /** + * Static value Succeeded for ProvisioningState. + */ public static final ProvisioningState SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for ProvisioningState. */ + /** + * Static value Failed for ProvisioningState. + */ public static final ProvisioningState FAILED = fromString("Failed"); - /** Static value Canceled for ProvisioningState. */ + /** + * Static value Canceled for ProvisioningState. + */ public static final ProvisioningState CANCELED = fromString("Canceled"); - /** Static value Creating for ProvisioningState. */ + /** + * Static value Creating for ProvisioningState. + */ public static final ProvisioningState CREATING = fromString("Creating"); - /** Static value Updating for ProvisioningState. */ + /** + * Static value Updating for ProvisioningState. + */ public static final ProvisioningState UPDATING = fromString("Updating"); - /** Static value Deleting for ProvisioningState. */ + /** + * Static value Deleting for ProvisioningState. + */ public static final ProvisioningState DELETING = fromString("Deleting"); /** * Creates a new instance of ProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public ProvisioningState() { /** * Creates or finds a ProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ProvisioningState. */ @@ -50,7 +64,7 @@ public static ProvisioningState fromString(String name) { /** * Gets known ProvisioningState values. - * + * * @return known ProvisioningState values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RdbFrequency.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RdbFrequency.java index 3a4efbdfa429d..b18ab89609caa 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RdbFrequency.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RdbFrequency.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Sets the frequency at which a snapshot of the database is created. */ +/** + * Sets the frequency at which a snapshot of the database is created. + */ public final class RdbFrequency extends ExpandableStringEnum { - /** Static value 1h for RdbFrequency. */ + /** + * Static value 1h for RdbFrequency. + */ public static final RdbFrequency ONEH = fromString("1h"); - /** Static value 6h for RdbFrequency. */ + /** + * Static value 6h for RdbFrequency. + */ public static final RdbFrequency SIXH = fromString("6h"); - /** Static value 12h for RdbFrequency. */ + /** + * Static value 12h for RdbFrequency. + */ public static final RdbFrequency ONE_TWOH = fromString("12h"); /** * Creates a new instance of RdbFrequency value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public RdbFrequency() { /** * Creates or finds a RdbFrequency from its string representation. - * + * * @param name a name to look for. * @return the corresponding RdbFrequency. */ @@ -41,7 +49,7 @@ public static RdbFrequency fromString(String name) { /** * Gets known RdbFrequency values. - * + * * @return known RdbFrequency values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RedisEnterprises.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RedisEnterprises.java index 2a66480a02f4d..81585c1f18241 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RedisEnterprises.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RedisEnterprises.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of RedisEnterprises. */ +/** + * Resource collection API of RedisEnterprises. + */ public interface RedisEnterprises { /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface RedisEnterprises { /** * Deletes a RedisEnterprise cache cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -35,7 +37,7 @@ public interface RedisEnterprises { /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @param context The context to associate with this operation. @@ -48,7 +50,7 @@ public interface RedisEnterprises { /** * Gets information about a RedisEnterprise cluster. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param clusterName The name of the RedisEnterprise cluster. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -60,7 +62,7 @@ public interface RedisEnterprises { /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -71,7 +73,7 @@ public interface RedisEnterprises { /** * Lists all RedisEnterprise clusters in a resource group. - * + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -83,29 +85,29 @@ public interface RedisEnterprises { /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link - * PagedIterable}. + * @return all RedisEnterprise clusters in the specified subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Gets all RedisEnterprise clusters in the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all RedisEnterprise clusters in the specified subscription as paginated response with {@link - * PagedIterable}. + * @return all RedisEnterprise clusters in the specified subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Gets information about a RedisEnterprise cluster. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -116,7 +118,7 @@ public interface RedisEnterprises { /** * Gets information about a RedisEnterprise cluster. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -128,7 +130,7 @@ public interface RedisEnterprises { /** * Deletes a RedisEnterprise cache cluster. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -138,7 +140,7 @@ public interface RedisEnterprises { /** * Deletes a RedisEnterprise cache cluster. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -149,7 +151,7 @@ public interface RedisEnterprises { /** * Begins definition for a new Cluster resource. - * + * * @param name resource name. * @return the first stage of the new Cluster definition. */ diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegenerateKeyParameters.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegenerateKeyParameters.java index 017bc502c99ca..02c94f1e6f789 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegenerateKeyParameters.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegenerateKeyParameters.java @@ -10,8 +10,8 @@ /** * Regenerate access keys request - * - *

Specifies which access keys to reset to a new random value. + * + * Specifies which access keys to reset to a new random value. */ @Fluent public final class RegenerateKeyParameters { @@ -21,13 +21,15 @@ public final class RegenerateKeyParameters { @JsonProperty(value = "keyType", required = true) private AccessKeyType keyType; - /** Creates an instance of RegenerateKeyParameters class. */ + /** + * Creates an instance of RegenerateKeyParameters class. + */ public RegenerateKeyParameters() { } /** * Get the keyType property: Which access key to regenerate. - * + * * @return the keyType value. */ public AccessKeyType keyType() { @@ -36,7 +38,7 @@ public AccessKeyType keyType() { /** * Set the keyType property: Which access key to regenerate. - * + * * @param keyType the keyType value to set. * @return the RegenerateKeyParameters object itself. */ @@ -47,14 +49,13 @@ public RegenerateKeyParameters withKeyType(AccessKeyType keyType) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (keyType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property keyType in model RegenerateKeyParameters")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property keyType in model RegenerateKeyParameters")); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegionSkuDetail.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegionSkuDetail.java deleted file mode 100644 index 520df8717f2fa..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/RegionSkuDetail.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.models; - -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; - -/** An immutable client-side representation of RegionSkuDetail. */ -public interface RegionSkuDetail { - /** - * Gets the resourceType property: Resource type which has the SKU, such as Microsoft.Cache/redisEnterprise. - * - * @return the resourceType value. - */ - String resourceType(); - - /** - * Gets the locationInfo property: Details about location and its capabilities. - * - * @return the locationInfo value. - */ - LocationInfo locationInfo(); - - /** - * Gets the skuDetails property: Sku Detail - * - *

Details about available skus. - * - * @return the skuDetails value. - */ - SkuDetail skuDetails(); - - /** - * Gets the inner com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner object. - * - * @return the inner object. - */ - RegionSkuDetailInner innerModel(); -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ResourceState.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ResourceState.java index 05fd95a4b78b6..2688378e7ae71 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ResourceState.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/ResourceState.java @@ -8,47 +8,83 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Current resource status. */ +/** + * Current resource status. + */ public final class ResourceState extends ExpandableStringEnum { - /** Static value Running for ResourceState. */ + /** + * Static value Running for ResourceState. + */ public static final ResourceState RUNNING = fromString("Running"); - /** Static value Creating for ResourceState. */ + /** + * Static value Creating for ResourceState. + */ public static final ResourceState CREATING = fromString("Creating"); - /** Static value CreateFailed for ResourceState. */ + /** + * Static value CreateFailed for ResourceState. + */ public static final ResourceState CREATE_FAILED = fromString("CreateFailed"); - /** Static value Updating for ResourceState. */ + /** + * Static value Updating for ResourceState. + */ public static final ResourceState UPDATING = fromString("Updating"); - /** Static value UpdateFailed for ResourceState. */ + /** + * Static value UpdateFailed for ResourceState. + */ public static final ResourceState UPDATE_FAILED = fromString("UpdateFailed"); - /** Static value Deleting for ResourceState. */ + /** + * Static value Deleting for ResourceState. + */ public static final ResourceState DELETING = fromString("Deleting"); - /** Static value DeleteFailed for ResourceState. */ + /** + * Static value DeleteFailed for ResourceState. + */ public static final ResourceState DELETE_FAILED = fromString("DeleteFailed"); - /** Static value Enabling for ResourceState. */ + /** + * Static value Enabling for ResourceState. + */ public static final ResourceState ENABLING = fromString("Enabling"); - /** Static value EnableFailed for ResourceState. */ + /** + * Static value EnableFailed for ResourceState. + */ public static final ResourceState ENABLE_FAILED = fromString("EnableFailed"); - /** Static value Disabling for ResourceState. */ + /** + * Static value Disabling for ResourceState. + */ public static final ResourceState DISABLING = fromString("Disabling"); - /** Static value DisableFailed for ResourceState. */ + /** + * Static value DisableFailed for ResourceState. + */ public static final ResourceState DISABLE_FAILED = fromString("DisableFailed"); - /** Static value Disabled for ResourceState. */ + /** + * Static value Disabled for ResourceState. + */ public static final ResourceState DISABLED = fromString("Disabled"); + /** + * Static value Scaling for ResourceState. + */ + public static final ResourceState SCALING = fromString("Scaling"); + + /** + * Static value ScalingFailed for ResourceState. + */ + public static final ResourceState SCALING_FAILED = fromString("ScalingFailed"); + /** * Creates a new instance of ResourceState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -57,7 +93,7 @@ public ResourceState() { /** * Creates or finds a ResourceState from its string representation. - * + * * @param name a name to look for. * @return the corresponding ResourceState. */ @@ -68,7 +104,7 @@ public static ResourceState fromString(String name) { /** * Gets known ResourceState values. - * + * * @return known ResourceState values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Sku.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Sku.java index 9b088f3d24aab..0684e7aa0f3b4 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Sku.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/Sku.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** SKU parameters supplied to the create RedisEnterprise operation. */ +/** + * SKU parameters supplied to the create RedisEnterprise operation. + */ @Fluent public final class Sku { /* @@ -24,14 +26,16 @@ public final class Sku { @JsonProperty(value = "capacity") private Integer capacity; - /** Creates an instance of Sku class. */ + /** + * Creates an instance of Sku class. + */ public Sku() { } /** * Get the name property: The type of RedisEnterprise cluster to deploy. Possible values: (Enterprise_E10, * EnterpriseFlash_F300 etc.). - * + * * @return the name value. */ public SkuName name() { @@ -41,7 +45,7 @@ public SkuName name() { /** * Set the name property: The type of RedisEnterprise cluster to deploy. Possible values: (Enterprise_E10, * EnterpriseFlash_F300 etc.). - * + * * @param name the name value to set. * @return the Sku object itself. */ @@ -53,7 +57,7 @@ public Sku withName(SkuName name) { /** * Get the capacity property: The size of the RedisEnterprise cluster. Defaults to 2 or 3 depending on SKU. Valid * values are (2, 4, 6, ...) for Enterprise SKUs and (3, 9, 15, ...) for Flash SKUs. - * + * * @return the capacity value. */ public Integer capacity() { @@ -63,7 +67,7 @@ public Integer capacity() { /** * Set the capacity property: The size of the RedisEnterprise cluster. Defaults to 2 or 3 depending on SKU. Valid * values are (2, 4, 6, ...) for Enterprise SKUs and (3, 9, 15, ...) for Flash SKUs. - * + * * @param capacity the capacity value to set. * @return the Sku object itself. */ @@ -74,7 +78,7 @@ public Sku withCapacity(Integer capacity) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuDetail.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuDetail.java deleted file mode 100644 index bb8bf16368d17..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuDetail.java +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.models; - -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** - * Sku Detail - * - *

Information about Sku. - */ -@Fluent -public final class SkuDetail { - /* - * The type of RedisEnterprise cluster to deploy. Possible values: (Enterprise_E10, EnterpriseFlash_F300 etc.) - */ - @JsonProperty(value = "name") - private SkuName name; - - /** Creates an instance of SkuDetail class. */ - public SkuDetail() { - } - - /** - * Get the name property: The type of RedisEnterprise cluster to deploy. Possible values: (Enterprise_E10, - * EnterpriseFlash_F300 etc.). - * - * @return the name value. - */ - public SkuName name() { - return this.name; - } - - /** - * Set the name property: The type of RedisEnterprise cluster to deploy. Possible values: (Enterprise_E10, - * EnterpriseFlash_F300 etc.). - * - * @param name the name value to set. - * @return the SkuDetail object itself. - */ - public SkuDetail withName(SkuName name) { - this.name = name; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuName.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuName.java index cfc69f2906aa7..bf59b6225bae3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuName.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/SkuName.java @@ -8,32 +8,48 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for SkuName. */ +/** + * The type of RedisEnterprise cluster to deploy. Possible values: (Enterprise_E10, EnterpriseFlash_F300 etc.). + */ public final class SkuName extends ExpandableStringEnum { - /** Static value Enterprise_E10 for SkuName. */ + /** + * Static value Enterprise_E10 for SkuName. + */ public static final SkuName ENTERPRISE_E10 = fromString("Enterprise_E10"); - /** Static value Enterprise_E20 for SkuName. */ + /** + * Static value Enterprise_E20 for SkuName. + */ public static final SkuName ENTERPRISE_E20 = fromString("Enterprise_E20"); - /** Static value Enterprise_E50 for SkuName. */ + /** + * Static value Enterprise_E50 for SkuName. + */ public static final SkuName ENTERPRISE_E50 = fromString("Enterprise_E50"); - /** Static value Enterprise_E100 for SkuName. */ + /** + * Static value Enterprise_E100 for SkuName. + */ public static final SkuName ENTERPRISE_E100 = fromString("Enterprise_E100"); - /** Static value EnterpriseFlash_F300 for SkuName. */ + /** + * Static value EnterpriseFlash_F300 for SkuName. + */ public static final SkuName ENTERPRISE_FLASH_F300 = fromString("EnterpriseFlash_F300"); - /** Static value EnterpriseFlash_F700 for SkuName. */ + /** + * Static value EnterpriseFlash_F700 for SkuName. + */ public static final SkuName ENTERPRISE_FLASH_F700 = fromString("EnterpriseFlash_F700"); - /** Static value EnterpriseFlash_F1500 for SkuName. */ + /** + * Static value EnterpriseFlash_F1500 for SkuName. + */ public static final SkuName ENTERPRISE_FLASH_F1500 = fromString("EnterpriseFlash_F1500"); /** * Creates a new instance of SkuName value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -42,7 +58,7 @@ public SkuName() { /** * Creates or finds a SkuName from its string representation. - * + * * @param name a name to look for. * @return the corresponding SkuName. */ @@ -53,7 +69,7 @@ public static SkuName fromString(String name) { /** * Gets known SkuName values. - * + * * @return known SkuName values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/TlsVersion.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/TlsVersion.java index 608d71f2c4fa7..50ca42039d537 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/TlsVersion.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/TlsVersion.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The minimum TLS version for the cluster to support, e.g. '1.2'. */ +/** + * The minimum TLS version for the cluster to support, e.g. '1.2'. + */ public final class TlsVersion extends ExpandableStringEnum { - /** Static value 1.0 for TlsVersion. */ + /** + * Static value 1.0 for TlsVersion. + */ public static final TlsVersion ONE_ZERO = fromString("1.0"); - /** Static value 1.1 for TlsVersion. */ + /** + * Static value 1.1 for TlsVersion. + */ public static final TlsVersion ONE_ONE = fromString("1.1"); - /** Static value 1.2 for TlsVersion. */ + /** + * Static value 1.2 for TlsVersion. + */ public static final TlsVersion ONE_TWO = fromString("1.2"); /** * Creates a new instance of TlsVersion value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public TlsVersion() { /** * Creates or finds a TlsVersion from its string representation. - * + * * @param name a name to look for. * @return the corresponding TlsVersion. */ @@ -41,7 +49,7 @@ public static TlsVersion fromString(String name) { /** * Gets known TlsVersion values. - * + * * @return known TlsVersion values. */ public static Collection values() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/UserAssignedIdentity.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/UserAssignedIdentity.java index d76987e96ef90..aca35fbfc7d8c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/UserAssignedIdentity.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/UserAssignedIdentity.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.UUID; -/** User assigned identity properties. */ +/** + * User assigned identity properties. + */ @Immutable public final class UserAssignedIdentity { /* @@ -23,13 +25,15 @@ public final class UserAssignedIdentity { @JsonProperty(value = "clientId", access = JsonProperty.Access.WRITE_ONLY) private UUID clientId; - /** Creates an instance of UserAssignedIdentity class. */ + /** + * Creates an instance of UserAssignedIdentity class. + */ public UserAssignedIdentity() { } /** * Get the principalId property: The principal ID of the assigned identity. - * + * * @return the principalId value. */ public UUID principalId() { @@ -38,7 +42,7 @@ public UUID principalId() { /** * Get the clientId property: The client ID of the assigned identity. - * + * * @return the clientId value. */ public UUID clientId() { @@ -47,7 +51,7 @@ public UUID clientId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/package-info.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/package-info.java index 703339625cbca..041660f977ea0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/package-info.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the data models for RedisEnterpriseManagementClient. REST API for managing Redis Enterprise - * resources in Azure. + * Package containing the data models for RedisEnterpriseManagementClient. + * REST API for managing Redis Enterprise resources in Azure. */ package com.azure.resourcemanager.redisenterprise.models; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/package-info.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/package-info.java index 6d8d8eca96118..3ebf99a5ba10e 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/package-info.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/com/azure/resourcemanager/redisenterprise/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the classes for RedisEnterpriseManagementClient. REST API for managing Redis Enterprise resources - * in Azure. + * Package containing the classes for RedisEnterpriseManagementClient. + * REST API for managing Redis Enterprise resources in Azure. */ package com.azure.resourcemanager.redisenterprise; diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/module-info.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/module-info.java index b44e9511748d8..5a13ca9787876 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/module-info.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/java/module-info.java @@ -4,16 +4,10 @@ module com.azure.resourcemanager.redisenterprise { requires transitive com.azure.core.management; - exports com.azure.resourcemanager.redisenterprise; exports com.azure.resourcemanager.redisenterprise.fluent; exports com.azure.resourcemanager.redisenterprise.fluent.models; exports com.azure.resourcemanager.redisenterprise.models; - - opens com.azure.resourcemanager.redisenterprise.fluent.models to - com.azure.core, - com.fasterxml.jackson.databind; - opens com.azure.resourcemanager.redisenterprise.models to - com.azure.core, - com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.redisenterprise.fluent.models to com.azure.core, com.fasterxml.jackson.databind; + opens com.azure.resourcemanager.redisenterprise.models to com.azure.core, com.fasterxml.jackson.databind; } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/proxy-config.json b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/proxy-config.json new file mode 100644 index 0000000000000..071af40d70b0e --- /dev/null +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/proxy-config.json @@ -0,0 +1 @@ +[ [ "com.azure.resourcemanager.redisenterprise.implementation.OperationsClientImpl$OperationsService" ], [ "com.azure.resourcemanager.redisenterprise.implementation.OperationsStatusClientImpl$OperationsStatusService" ], [ "com.azure.resourcemanager.redisenterprise.implementation.RedisEnterprisesClientImpl$RedisEnterprisesService" ], [ "com.azure.resourcemanager.redisenterprise.implementation.DatabasesClientImpl$DatabasesService" ], [ "com.azure.resourcemanager.redisenterprise.implementation.PrivateEndpointConnectionsClientImpl$PrivateEndpointConnectionsService" ], [ "com.azure.resourcemanager.redisenterprise.implementation.PrivateLinkResourcesClientImpl$PrivateLinkResourcesService" ] ] \ No newline at end of file diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/reflect-config.json b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/reflect-config.json new file mode 100644 index 0000000000000..b1424710c74bc --- /dev/null +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/main/resources/META-INF/native-image/com.azure.resourcemanager/azure-resourcemanager-redisenterprise/reflect-config.json @@ -0,0 +1,266 @@ +[ { + "name" : "com.azure.resourcemanager.redisenterprise.models.OperationListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.OperationInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.OperationDisplay", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.OperationStatusInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.Sku", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.UserAssignedIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.ClusterProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryption", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryptionCustomerManagedKeyEncryption", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.PrivateEndpoint", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.PrivateLinkServiceConnectionState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ClusterUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ClusterList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.DatabaseList", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.DatabaseInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.DatabaseProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.Persistence", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.Module", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.DatabasePropertiesGeoReplication", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.LinkedDatabase", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.DatabaseUpdate", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.AccessKeysInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.RegenerateKeyParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ImportClusterParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ExportClusterParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.PrivateEndpointConnectionListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.PrivateLinkResourceListResult", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceInner", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceProperties", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ForceUnlinkParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.FlushParameters", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.Origin", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ActionType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.SkuName", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.TlsVersion", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.CmkIdentityType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ResourceState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.PrivateEndpointServiceConnectionStatus", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.PrivateEndpointConnectionProvisioningState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.Protocol", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.ClusteringPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.EvictionPolicy", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.AofFrequency", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.RdbFrequency", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.LinkState", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +}, { + "name" : "com.azure.resourcemanager.redisenterprise.models.AccessKeyType", + "allDeclaredConstructors" : true, + "allDeclaredFields" : true, + "allDeclaredMethods" : true +} ] \ No newline at end of file diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateSamples.java index 38cd9fcae483d..11de094d111e3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateSamples.java @@ -14,66 +14,50 @@ import com.azure.resourcemanager.redisenterprise.models.Protocol; import java.util.Arrays; -/** Samples for Databases Create. */ +/** + * Samples for Databases Create. + */ public final class DatabasesCreateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesCreate.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesCreate.json */ /** * Sample code: RedisEnterpriseDatabasesCreate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesCreate( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .define("default") - .withExistingRedisEnterprise("rg1", "cache1") - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(10000) - .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) - .withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) + public static void + redisEnterpriseDatabasesCreate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().define("default").withExistingRedisEnterprise("rg1", "cache1") + .withClientProtocol(Protocol.ENCRYPTED).withPort(10000) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER).withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) .withPersistence(new Persistence().withAofEnabled(true).withAofFrequency(AofFrequency.ONES)) - .withModules( - Arrays - .asList( - new Module().withName("RedisBloom").withArgs("ERROR_RATE 0.00 INITIAL_SIZE 400"), - new Module().withName("RedisTimeSeries").withArgs("RETENTION_POLICY 20"), - new Module().withName("RediSearch"))) + .withModules(Arrays.asList(new Module().withName("RedisBloom").withArgs("ERROR_RATE 0.00 INITIAL_SIZE 400"), + new Module().withName("RedisTimeSeries").withArgs("RETENTION_POLICY 20"), + new Module().withName("RediSearch"))) .create(); } /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesCreateWithGeoReplication.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesCreateWithGeoReplication.json */ /** * Sample code: RedisEnterpriseDatabasesCreate With Active Geo Replication. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDatabasesCreateWithActiveGeoReplication( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .define("default") - .withExistingRedisEnterprise("rg1", "cache1") - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(10000) - .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) - .withEvictionPolicy(EvictionPolicy.NO_EVICTION) - .withGeoReplication( - new DatabasePropertiesGeoReplication() - .withGroupNickname("groupName") - .withLinkedDatabases( - Arrays - .asList( - new LinkedDatabase() - .withId( - "/subscriptions/subid1/resourceGroups/rg1/providers/Microsoft.Cache/redisEnterprise/cache1/databases/default"), - new LinkedDatabase() - .withId( - "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")))) + manager.databases().define("default").withExistingRedisEnterprise("rg1", "cache1") + .withClientProtocol(Protocol.ENCRYPTED).withPort(10000) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER).withEvictionPolicy(EvictionPolicy.NO_EVICTION) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("groupName") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase().withId( + "/subscriptions/subid1/resourceGroups/rg1/providers/Microsoft.Cache/redisEnterprise/cache1/databases/default"), + new LinkedDatabase().withId( + "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")))) .create(); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteSamples.java index 62f5a06b8e6f3..a2605bb875a63 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for Databases Delete. */ +/** + * Samples for Databases Delete. + */ public final class DatabasesDeleteSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesDelete.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesDelete.json */ /** * Sample code: RedisEnterpriseDatabasesDelete. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesDelete( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseDatabasesDelete(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.databases().delete("rg1", "cache1", "db1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportSamples.java index 3f88f12b2f0f0..0b434eec4a832 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportSamples.java @@ -6,26 +6,24 @@ import com.azure.resourcemanager.redisenterprise.models.ExportClusterParameters; -/** Samples for Databases Export. */ +/** + * Samples for Databases Export. + */ public final class DatabasesExportSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesExport.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesExport.json */ /** * Sample code: RedisEnterpriseDatabasesExport. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesExport( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .export( - "rg1", - "cache1", - "default", - new ExportClusterParameters() - .withSasUri("https://contosostorage.blob.core.window.net/urlToBlobContainer?sasKeyParameters"), - com.azure.core.util.Context.NONE); + public static void + redisEnterpriseDatabasesExport(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().export("rg1", "cache1", "default", + new ExportClusterParameters() + .withSasUri("https://contosostorage.blob.core.window.net/urlToBlobContainer?sasKeyParameters"), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushSamples.java index 326ea36a78eca..b53f83619a0bf 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushSamples.java @@ -7,29 +7,23 @@ import com.azure.resourcemanager.redisenterprise.models.FlushParameters; import java.util.Arrays; -/** Samples for Databases Flush. */ +/** + * Samples for Databases Flush. + */ public final class DatabasesFlushSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesFlush.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesFlush.json */ /** * Sample code: How to flush all the keys in the database. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void howToFlushAllTheKeysInTheDatabase( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .flush( - "rg1", - "cache1", - "default", - new FlushParameters() - .withIds( - Arrays - .asList( - "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), - com.azure.core.util.Context.NONE); + public static void + howToFlushAllTheKeysInTheDatabase(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().flush("rg1", "cache1", "default", new FlushParameters().withIds(Arrays.asList( + "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkSamples.java index 937ba66aa8c83..c76904eb40076 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkSamples.java @@ -7,29 +7,23 @@ import com.azure.resourcemanager.redisenterprise.models.ForceUnlinkParameters; import java.util.Arrays; -/** Samples for Databases ForceUnlink. */ +/** + * Samples for Databases ForceUnlink. + */ public final class DatabasesForceUnlinkSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesForceUnlink.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesForceUnlink.json */ /** * Sample code: How to unlink a database during a regional outage. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void howToUnlinkADatabaseDuringARegionalOutage( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .forceUnlink( - "rg1", - "cache1", - "default", - new ForceUnlinkParameters() - .withIds( - Arrays - .asList( - "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), - com.azure.core.util.Context.NONE); + manager.databases().forceUnlink("rg1", "cache1", "default", new ForceUnlinkParameters().withIds(Arrays.asList( + "/subscriptions/subid2/resourceGroups/rg2/providers/Microsoft.Cache/redisEnterprise/cache2/databases/default")), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetSamples.java index 4afaa584562cd..736659c9b6b68 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for Databases Get. */ +/** + * Samples for Databases Get. + */ public final class DatabasesGetSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesGet.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesGet.json */ /** * Sample code: RedisEnterpriseDatabasesGet. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesGet( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseDatabasesGet(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.databases().getWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodSamples.java index 51e054badbecf..06f0f2d503b8c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodSamples.java @@ -7,30 +7,25 @@ import com.azure.resourcemanager.redisenterprise.models.ImportClusterParameters; import java.util.Arrays; -/** Samples for Databases ImportMethod. */ +/** + * Samples for Databases ImportMethod. + */ public final class DatabasesImportMethodSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesImport.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesImport.json */ /** * Sample code: RedisEnterpriseDatabasesImport. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesImport( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .importMethod( - "rg1", - "cache1", - "default", - new ImportClusterParameters() - .withSasUris( - Arrays - .asList( - "https://contosostorage.blob.core.window.net/urltoBlobFile1?sasKeyParameters", - "https://contosostorage.blob.core.window.net/urltoBlobFile2?sasKeyParameters")), - com.azure.core.util.Context.NONE); + public static void + redisEnterpriseDatabasesImport(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + manager.databases().importMethod("rg1", "cache1", "default", + new ImportClusterParameters().withSasUris( + Arrays.asList("https://contosostorage.blob.core.window.net/urltoBlobFile1?sasKeyParameters", + "https://contosostorage.blob.core.window.net/urltoBlobFile2?sasKeyParameters")), + com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterSamples.java index 3d2345f7f550e..5a65273d92acb 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for Databases ListByCluster. */ +/** + * Samples for Databases ListByCluster. + */ public final class DatabasesListByClusterSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesListByCluster.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesListByCluster.json */ /** * Sample code: RedisEnterpriseDatabasesListByCluster. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDatabasesListByCluster( diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListKeysSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListKeysSamples.java index 810aebae03b76..3dd99317428f7 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListKeysSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListKeysSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for Databases ListKeys. */ +/** + * Samples for Databases ListKeys. + */ public final class DatabasesListKeysSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesListKeys.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesListKeys.json */ /** * Sample code: RedisEnterpriseDatabasesListKeys. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesListKeys( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseDatabasesListKeys(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.databases().listKeysWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesRegenerateKeySamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesRegenerateKeySamples.java index a35052532820a..c7c64a87fe0ae 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesRegenerateKeySamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesRegenerateKeySamples.java @@ -7,25 +7,22 @@ import com.azure.resourcemanager.redisenterprise.models.AccessKeyType; import com.azure.resourcemanager.redisenterprise.models.RegenerateKeyParameters; -/** Samples for Databases RegenerateKey. */ +/** + * Samples for Databases RegenerateKey. + */ public final class DatabasesRegenerateKeySamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesRegenerateKey.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesRegenerateKey.json */ /** * Sample code: RedisEnterpriseDatabasesRegenerateKey. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDatabasesRegenerateKey( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .databases() - .regenerateKey( - "rg1", - "cache1", - "default", - new RegenerateKeyParameters().withKeyType(AccessKeyType.PRIMARY), - com.azure.core.util.Context.NONE); + manager.databases().regenerateKey("rg1", "cache1", "default", + new RegenerateKeyParameters().withKeyType(AccessKeyType.PRIMARY), com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesUpdateSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesUpdateSamples.java index 56565da1f6655..d8bcdc4a07d48 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesUpdateSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesUpdateSamples.java @@ -10,28 +10,24 @@ import com.azure.resourcemanager.redisenterprise.models.Protocol; import com.azure.resourcemanager.redisenterprise.models.RdbFrequency; -/** Samples for Databases Update. */ +/** + * Samples for Databases Update. + */ public final class DatabasesUpdateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDatabasesUpdate.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDatabasesUpdate.json */ /** * Sample code: RedisEnterpriseDatabasesUpdate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseDatabasesUpdate( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - Database resource = - manager - .databases() - .getWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withClientProtocol(Protocol.ENCRYPTED) - .withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) - .withPersistence(new Persistence().withRdbEnabled(true).withRdbFrequency(RdbFrequency.ONE_TWOH)) - .apply(); + public static void + redisEnterpriseDatabasesUpdate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + Database resource = manager.databases() + .getWithResponse("rg1", "cache1", "default", com.azure.core.util.Context.NONE).getValue(); + resource.update().withClientProtocol(Protocol.ENCRYPTED).withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) + .withPersistence(new Persistence().withRdbEnabled(true).withRdbFrequency(RdbFrequency.ONE_TWOH)).apply(); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListSamples.java index 665c43997f3f8..a4119bf73d2a0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for Operations List. */ +/** + * Samples for Operations List. + */ public final class OperationsListSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/OperationsList.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/OperationsList.json */ /** * Sample code: OperationsList. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void operationsList(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetSamples.java index 271cbd96ea8b2..22be22a3d617e 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for OperationsStatus Get. */ +/** + * Samples for OperationsStatus Get. + */ public final class OperationsStatusGetSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/OperationsStatusGet.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/OperationsStatusGet. + * json */ /** * Sample code: OperationsStatusGet. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void operationsStatusGet(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteSamples.java index 67c0b482b470e..310c58376b34b 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteSamples.java @@ -4,20 +4,21 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for PrivateEndpointConnections Delete. */ +/** + * Samples for PrivateEndpointConnections Delete. + */ public final class PrivateEndpointConnectionsDeleteSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDeletePrivateEndpointConnection.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseDeletePrivateEndpointConnection.json */ /** * Sample code: RedisEnterpriseDeletePrivateEndpointConnection. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDeletePrivateEndpointConnection( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .privateEndpointConnections() - .deleteWithResponse("rg1", "cache1", "pectest01", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().delete("rg1", "cache1", "pectest01", com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetSamples.java index 926418d4529e7..76bae04a0e43b 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetSamples.java @@ -4,20 +4,22 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for PrivateEndpointConnections Get. */ +/** + * Samples for PrivateEndpointConnections Get. + */ public final class PrivateEndpointConnectionsGetSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseGetPrivateEndpointConnection.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseGetPrivateEndpointConnection.json */ /** * Sample code: RedisEnterpriseGetPrivateEndpointConnection. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseGetPrivateEndpointConnection( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .privateEndpointConnections() - .getWithResponse("rg1", "cache1", "pectest01", com.azure.core.util.Context.NONE); + manager.privateEndpointConnections().getWithResponse("rg1", "cache1", "pectest01", + com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListSamples.java index 137cdd9ad5709..73b09cc53b5ff 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for PrivateEndpointConnections List. */ +/** + * Samples for PrivateEndpointConnections List. + */ public final class PrivateEndpointConnectionsListSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseListPrivateEndpointConnections.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseListPrivateEndpointConnections.json */ /** * Sample code: RedisEnterpriseListPrivateEndpointConnections. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseListPrivateEndpointConnections( diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsPutSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsPutSamples.java index afa3503d855fb..60344e1402c0c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsPutSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsPutSamples.java @@ -7,26 +7,24 @@ import com.azure.resourcemanager.redisenterprise.models.PrivateEndpointServiceConnectionStatus; import com.azure.resourcemanager.redisenterprise.models.PrivateLinkServiceConnectionState; -/** Samples for PrivateEndpointConnections Put. */ +/** + * Samples for PrivateEndpointConnections Put. + */ public final class PrivateEndpointConnectionsPutSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterprisePutPrivateEndpointConnection.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterprisePutPrivateEndpointConnection.json */ /** * Sample code: RedisEnterprisePutPrivateEndpointConnection. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterprisePutPrivateEndpointConnection( com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .privateEndpointConnections() - .define("pectest01") - .withExistingRedisEnterprise("rg1", "cache1") - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) - .withDescription("Auto-Approved")) + manager.privateEndpointConnections().define("pectest01").withExistingRedisEnterprise("rg1", "cache1") + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED).withDescription("Auto-Approved")) .create(); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterSamples.java index 95b20f5b140b6..124a55a1e97aa 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for PrivateLinkResources ListByCluster. */ +/** + * Samples for PrivateLinkResources ListByCluster. + */ public final class PrivateLinkResourcesListByClusterSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseListPrivateLinkResources.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseListPrivateLinkResources.json */ /** * Sample code: RedisEnterpriseListPrivateLinkResources. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseListPrivateLinkResources( diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseCreateSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseCreateSamples.java index 0e215713432d1..44656fb0b8a0b 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseCreateSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseCreateSamples.java @@ -18,46 +18,40 @@ import java.util.HashMap; import java.util.Map; -/** Samples for RedisEnterprise Create. */ +/** + * Samples for RedisEnterprise Create. + */ public final class RedisEnterpriseCreateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseCreate.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseCreate. + * json */ /** * Sample code: RedisEnterpriseCreate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseCreate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager - .redisEnterprises() - .define("cache1") - .withRegion("West US") - .withExistingResourceGroup("rg1") + manager.redisEnterprises().define("cache1").withRegion("West US").withExistingResourceGroup("rg1") .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F300).withCapacity(3)) - .withTags(mapOf("tag1", "value1")) - .withZones(Arrays.asList("1", "2", "3")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity", - new UserAssignedIdentity()))) + .withTags(mapOf("tag1", "value1")).withZones(Arrays.asList("1", "2", "3")) + .withIdentity(new ManagedServiceIdentity().withType(ManagedServiceIdentityType.USER_ASSIGNED) + .withUserAssignedIdentities(mapOf( + "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity", + new UserAssignedIdentity()))) .withMinimumTlsVersion(TlsVersion.ONE_TWO) - .withEncryption( - new ClusterPropertiesEncryption() - .withCustomerManagedKeyEncryption( - new ClusterPropertiesEncryptionCustomerManagedKeyEncryption() - .withKeyEncryptionKeyIdentity( - new ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() - .withUserAssignedIdentityResourceId( - "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity") - .withIdentityType(CmkIdentityType.USER_ASSIGNED_IDENTITY)) - .withKeyEncryptionKeyUrl("fakeTokenPlaceholder"))) + .withEncryption(new ClusterPropertiesEncryption().withCustomerManagedKeyEncryption( + new ClusterPropertiesEncryptionCustomerManagedKeyEncryption().withKeyEncryptionKeyIdentity( + new ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() + .withUserAssignedIdentityResourceId( + "/subscriptions/your-subscription/resourceGroups/your-rg/providers/Microsoft.ManagedIdentity/userAssignedIdentities/your-identity") + .withIdentityType(CmkIdentityType.USER_ASSIGNED_IDENTITY)) + .withKeyEncryptionKeyUrl("fakeTokenPlaceholder"))) .create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseDeleteSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseDeleteSamples.java index a84c3525976dc..a43b25be2ee7c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseDeleteSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseDeleteSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for RedisEnterprise Delete. */ +/** + * Samples for RedisEnterprise Delete. + */ public final class RedisEnterpriseDeleteSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseDelete.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseDelete. + * json */ /** * Sample code: RedisEnterpriseDelete. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseDelete(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseGetByResourceGroupSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseGetByResourceGroupSamples.java index 65dfb9caa35f5..42bd6cce96997 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseGetByResourceGroupSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseGetByResourceGroupSamples.java @@ -4,14 +4,17 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for RedisEnterprise GetByResourceGroup. */ +/** + * Samples for RedisEnterprise GetByResourceGroup. + */ public final class RedisEnterpriseGetByResourceGroupSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseGet.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseGet.json */ /** * Sample code: RedisEnterpriseGet. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseGet(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListByResourceGroupSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListByResourceGroupSamples.java index 6bf487cdaecfc..188ec01e84ea1 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListByResourceGroupSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListByResourceGroupSamples.java @@ -4,18 +4,21 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for RedisEnterprise ListByResourceGroup. */ +/** + * Samples for RedisEnterprise ListByResourceGroup. + */ public final class RedisEnterpriseListByResourceGroupSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseListByResourceGroup.json + * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/ + * RedisEnterpriseListByResourceGroup.json */ /** * Sample code: RedisEnterpriseListByResourceGroup. - * + * * @param manager Entry point to RedisEnterpriseManager. */ - public static void redisEnterpriseListByResourceGroup( - com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { + public static void + redisEnterpriseListByResourceGroup(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { manager.redisEnterprises().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListSamples.java index caee449cc761d..9a39b2cb9632c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseListSamples.java @@ -4,14 +4,18 @@ package com.azure.resourcemanager.redisenterprise.generated; -/** Samples for RedisEnterprise List. */ +/** + * Samples for RedisEnterprise List. + */ public final class RedisEnterpriseListSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseList.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseList. + * json */ /** * Sample code: RedisEnterpriseList. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseList(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseUpdateSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseUpdateSamples.java index 05b9adf7bc81e..d441fb3a4ad31 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseUpdateSamples.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterpriseUpdateSamples.java @@ -11,30 +11,29 @@ import java.util.HashMap; import java.util.Map; -/** Samples for RedisEnterprise Update. */ +/** + * Samples for RedisEnterprise Update. + */ public final class RedisEnterpriseUpdateSamples { /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseUpdate.json + * x-ms-original-file: + * specification/redisenterprise/resource-manager/Microsoft.Cache/stable/2023-11-01/examples/RedisEnterpriseUpdate. + * json */ /** * Sample code: RedisEnterpriseUpdate. - * + * * @param manager Entry point to RedisEnterpriseManager. */ public static void redisEnterpriseUpdate(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - Cluster resource = - manager - .redisEnterprises() - .getByResourceGroupWithResponse("rg1", "cache1", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("tag1", "value1")) + Cluster resource = manager.redisEnterprises() + .getByResourceGroupWithResponse("rg1", "cache1", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("tag1", "value1")) .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F300).withCapacity(9)) - .withMinimumTlsVersion(TlsVersion.ONE_TWO) - .apply(); + .withMinimumTlsVersion(TlsVersion.ONE_TWO).apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/SkusListSamples.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/SkusListSamples.java deleted file mode 100644 index a7560f8f95a0c..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/samples/java/com/azure/resourcemanager/redisenterprise/generated/SkusListSamples.java +++ /dev/null @@ -1,20 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -/** Samples for Skus List. */ -public final class SkusListSamples { - /* - * x-ms-original-file: specification/redisenterprise/resource-manager/Microsoft.Cache/preview/2023-03-01-preview/examples/RedisEnterpriseSkusList.json - */ - /** - * Sample code: SkusList. - * - * @param manager Entry point to RedisEnterpriseManager. - */ - public static void skusList(com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager manager) { - manager.skus().list("westus2", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/CapabilityTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/CapabilityTests.java deleted file mode 100644 index 610043a8d46dd..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/CapabilityTests.java +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.models.Capability; -import org.junit.jupiter.api.Assertions; - -public final class CapabilityTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - Capability model = - BinaryData.fromString("{\"name\":\"oklyaxuconuq\",\"value\":false}").toObject(Capability.class); - Assertions.assertEquals("oklyaxuconuq", model.name()); - Assertions.assertEquals(false, model.value()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - Capability model = new Capability().withName("oklyaxuconuq").withValue(false); - model = BinaryData.fromObject(model).toObject(Capability.class); - Assertions.assertEquals("oklyaxuconuq", model.name()); - Assertions.assertEquals(false, model.value()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterInnerTests.java deleted file mode 100644 index 15f7bbc8cbedb..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterInnerTests.java +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner; -import com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryption; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentity; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.Sku; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import com.azure.resourcemanager.redisenterprise.models.UserAssignedIdentity; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class ClusterInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - ClusterInner model = - BinaryData - .fromString( - "{\"sku\":{\"name\":\"EnterpriseFlash_F700\",\"capacity\":992034682},\"zones\":[\"k\",\"vd\"],\"identity\":{\"principalId\":\"f5d47d63-4d8c-4425-9f08-befe321ff670\",\"tenantId\":\"aa43d95d-3905-4c9d-96c4-efd75eb5dbc2\",\"type\":\"None\",\"userAssignedIdentities\":{\"ukxgaud\":{\"principalId\":\"7a20e069-609c-4680-b9aa-e9ce7d87d024\",\"clientId\":\"1efc16fa-c5ff-48a1-9c85-4910c07da9c6\"},\"nhsjcnyej\":{\"principalId\":\"e40317fa-41f4-40d8-baea-a1e193f6138a\",\"clientId\":\"61cfc4f6-8a7d-4ccf-bc66-c8b1864a2f63\"}}},\"properties\":{\"minimumTlsVersion\":\"1.0\",\"encryption\":{},\"hostName\":\"apcz\",\"provisioningState\":\"Creating\",\"resourceState\":\"DeleteFailed\",\"redisVersion\":\"emkkvnipjox\",\"privateEndpointConnections\":[{\"id\":\"gejspodmailzyde\",\"name\":\"o\",\"type\":\"wyahuxinpmqnja\"},{\"id\":\"xj\",\"name\":\"prozvcputegjvwmf\",\"type\":\"atscmd\"}]},\"location\":\"jhulsuuvmkjo\",\"tags\":{\"ej\":\"wfndiodjpsl\",\"azakl\":\"pvwryoqpsoacc\"},\"id\":\"lahbcryff\",\"name\":\"fdosyg\",\"type\":\"xpaojakhmsbz\"}") - .toObject(ClusterInner.class); - Assertions.assertEquals("jhulsuuvmkjo", model.location()); - Assertions.assertEquals("wfndiodjpsl", model.tags().get("ej")); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F700, model.sku().name()); - Assertions.assertEquals(992034682, model.sku().capacity()); - Assertions.assertEquals("k", model.zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals(TlsVersion.ONE_ZERO, model.minimumTlsVersion()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - ClusterInner model = - new ClusterInner() - .withLocation("jhulsuuvmkjo") - .withTags(mapOf("ej", "wfndiodjpsl", "azakl", "pvwryoqpsoacc")) - .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F700).withCapacity(992034682)) - .withZones(Arrays.asList("k", "vd")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities( - mapOf("ukxgaud", new UserAssignedIdentity(), "nhsjcnyej", new UserAssignedIdentity()))) - .withMinimumTlsVersion(TlsVersion.ONE_ZERO) - .withEncryption(new ClusterPropertiesEncryption()); - model = BinaryData.fromObject(model).toObject(ClusterInner.class); - Assertions.assertEquals("jhulsuuvmkjo", model.location()); - Assertions.assertEquals("wfndiodjpsl", model.tags().get("ej")); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F700, model.sku().name()); - Assertions.assertEquals(992034682, model.sku().capacity()); - Assertions.assertEquals("k", model.zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.identity().type()); - Assertions.assertEquals(TlsVersion.ONE_ZERO, model.minimumTlsVersion()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterListTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterListTests.java deleted file mode 100644 index 35b760d40b2d0..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterListTests.java +++ /dev/null @@ -1,114 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.fluent.models.ClusterInner; -import com.azure.resourcemanager.redisenterprise.models.ClusterList; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentity; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.Sku; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class ClusterListTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - ClusterList model = - BinaryData - .fromString( - "{\"value\":[{\"sku\":{\"name\":\"Enterprise_E100\",\"capacity\":757698897},\"zones\":[\"l\",\"ndlik\",\"yqkgfg\"],\"identity\":{\"principalId\":\"90eff83e-825f-48f3-965e-3d795be6a629\",\"tenantId\":\"75977f0f-8699-47eb-a4d1-da4ff4ec7b73\",\"type\":\"None\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.1\",\"hostName\":\"rxybz\",\"provisioningState\":\"Canceled\",\"resourceState\":\"Disabled\",\"redisVersion\":\"tbciqfouflmm\",\"privateEndpointConnections\":[]},\"location\":\"smodmgloug\",\"tags\":{\"ta\":\"wtmutduq\",\"whbmd\":\"spwgcuertumkdosv\",\"bhtqqrolfpfpsa\":\"bbjfddgmbmbexp\",\"jgzjaoyfhrtx\":\"gbquxigj\"},\"id\":\"lnerkujysvleju\",\"name\":\"fqawrlyxw\",\"type\":\"kcprbnw\"},{\"sku\":{\"name\":\"EnterpriseFlash_F1500\",\"capacity\":19976049},\"zones\":[\"bvpyss\",\"dnrujqguhmuouqfp\",\"wzwbnguitn\"],\"identity\":{\"principalId\":\"c00a688f-e999-484c-a8a0-66e1185caef2\",\"tenantId\":\"2cbddb9d-555d-4e1d-bad2-29624caf2ff6\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.1\",\"hostName\":\"zuckyfi\",\"provisioningState\":\"Creating\",\"resourceState\":\"EnableFailed\",\"redisVersion\":\"vzwdzuhtymwis\",\"privateEndpointConnections\":[]},\"location\":\"thwxmnteiwaopvkm\",\"tags\":{\"uf\":\"mmxdcu\"},\"id\":\"rpymzidnsez\",\"name\":\"xtbzsgfyccsne\",\"type\":\"mdwzjeiachboo\"},{\"sku\":{\"name\":\"Enterprise_E20\",\"capacity\":219333087},\"zones\":[\"sfqpteehz\",\"vypyqrimzinpv\",\"wjdk\"],\"identity\":{\"principalId\":\"3bd9ea38-b56c-4244-85ed-751adf8ece99\",\"tenantId\":\"e55d238e-b999-422a-bc19-990b3d105f10\",\"type\":\"None\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.0\",\"hostName\":\"mnoh\",\"provisioningState\":\"Failed\",\"resourceState\":\"Running\",\"redisVersion\":\"dsoifiyipj\",\"privateEndpointConnections\":[]},\"location\":\"wpgrjbzno\",\"tags\":{\"nb\":\"xv\",\"rzafbljjgpbtoqcj\":\"xqabnmocpcysh\",\"vbqid\":\"klj\"},\"id\":\"qajzyulpkudjkr\",\"name\":\"khbzhfepgzg\",\"type\":\"e\"},{\"sku\":{\"name\":\"Enterprise_E20\",\"capacity\":1678385538},\"zones\":[\"scpai\",\"rhhbcs\",\"l\"],\"identity\":{\"principalId\":\"7b158bd3-97df-4c2a-ac7e-1b2217741991\",\"tenantId\":\"e484b802-9883-4733-a687-d29514741b98\",\"type\":\"SystemAssigned," - + " UserAssigned\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.2\",\"hostName\":\"obnbdxkqpxokaj\",\"provisioningState\":\"Succeeded\",\"resourceState\":\"Disabling\",\"redisVersion\":\"ex\",\"privateEndpointConnections\":[]},\"location\":\"xgcp\",\"tags\":{\"mcl\":\"maajrmvdjwzrlo\",\"jctbza\":\"hijco\"},\"id\":\"s\",\"name\":\"sycbkbfk\",\"type\":\"ukdkexxppofmxa\"}],\"nextLink\":\"fjpgddtocjjxhvp\"}") - .toObject(ClusterList.class); - Assertions.assertEquals("smodmgloug", model.value().get(0).location()); - Assertions.assertEquals("wtmutduq", model.value().get(0).tags().get("ta")); - Assertions.assertEquals(SkuName.ENTERPRISE_E100, model.value().get(0).sku().name()); - Assertions.assertEquals(757698897, model.value().get(0).sku().capacity()); - Assertions.assertEquals("l", model.value().get(0).zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.value().get(0).identity().type()); - Assertions.assertEquals(TlsVersion.ONE_ONE, model.value().get(0).minimumTlsVersion()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - ClusterList model = - new ClusterList() - .withValue( - Arrays - .asList( - new ClusterInner() - .withLocation("smodmgloug") - .withTags( - mapOf( - "ta", - "wtmutduq", - "whbmd", - "spwgcuertumkdosv", - "bhtqqrolfpfpsa", - "bbjfddgmbmbexp", - "jgzjaoyfhrtx", - "gbquxigj")) - .withSku(new Sku().withName(SkuName.ENTERPRISE_E100).withCapacity(757698897)) - .withZones(Arrays.asList("l", "ndlik", "yqkgfg")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf())) - .withMinimumTlsVersion(TlsVersion.ONE_ONE), - new ClusterInner() - .withLocation("thwxmnteiwaopvkm") - .withTags(mapOf("uf", "mmxdcu")) - .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F1500).withCapacity(19976049)) - .withZones(Arrays.asList("bvpyss", "dnrujqguhmuouqfp", "wzwbnguitn")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED) - .withUserAssignedIdentities(mapOf())) - .withMinimumTlsVersion(TlsVersion.ONE_ONE), - new ClusterInner() - .withLocation("wpgrjbzno") - .withTags(mapOf("nb", "xv", "rzafbljjgpbtoqcj", "xqabnmocpcysh", "vbqid", "klj")) - .withSku(new Sku().withName(SkuName.ENTERPRISE_E20).withCapacity(219333087)) - .withZones(Arrays.asList("sfqpteehz", "vypyqrimzinpv", "wjdk")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.NONE) - .withUserAssignedIdentities(mapOf())) - .withMinimumTlsVersion(TlsVersion.ONE_ZERO), - new ClusterInner() - .withLocation("xgcp") - .withTags(mapOf("mcl", "maajrmvdjwzrlo", "jctbza", "hijco")) - .withSku(new Sku().withName(SkuName.ENTERPRISE_E20).withCapacity(1678385538)) - .withZones(Arrays.asList("scpai", "rhhbcs", "l")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) - .withUserAssignedIdentities(mapOf())) - .withMinimumTlsVersion(TlsVersion.ONE_TWO))); - model = BinaryData.fromObject(model).toObject(ClusterList.class); - Assertions.assertEquals("smodmgloug", model.value().get(0).location()); - Assertions.assertEquals("wtmutduq", model.value().get(0).tags().get("ta")); - Assertions.assertEquals(SkuName.ENTERPRISE_E100, model.value().get(0).sku().name()); - Assertions.assertEquals(757698897, model.value().get(0).sku().capacity()); - Assertions.assertEquals("l", model.value().get(0).zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.value().get(0).identity().type()); - Assertions.assertEquals(TlsVersion.ONE_ONE, model.value().get(0).minimumTlsVersion()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentityTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentityTests.java index f4ebd3e0e5c6c..49f3b0a186c86 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentityTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentityTests.java @@ -12,26 +12,23 @@ public final class ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity model = - BinaryData - .fromString( - "{\"userAssignedIdentityResourceId\":\"ininmay\",\"identityType\":\"systemAssignedIdentity\"}") - .toObject(ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.class); - Assertions.assertEquals("ininmay", model.userAssignedIdentityResourceId()); + ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity model = BinaryData + .fromString( + "{\"userAssignedIdentityResourceId\":\"wfndiodjpsl\",\"identityType\":\"systemAssignedIdentity\"}") + .toObject(ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.class); + Assertions.assertEquals("wfndiodjpsl", model.userAssignedIdentityResourceId()); Assertions.assertEquals(CmkIdentityType.SYSTEM_ASSIGNED_IDENTITY, model.identityType()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity model = - new ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() - .withUserAssignedIdentityResourceId("ininmay") + ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity model + = new ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity() + .withUserAssignedIdentityResourceId("wfndiodjpsl") .withIdentityType(CmkIdentityType.SYSTEM_ASSIGNED_IDENTITY); - model = - BinaryData - .fromObject(model) - .toObject(ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.class); - Assertions.assertEquals("ininmay", model.userAssignedIdentityResourceId()); + model = BinaryData.fromObject(model) + .toObject(ClusterPropertiesEncryptionCustomerManagedKeyEncryptionKeyIdentity.class); + Assertions.assertEquals("wfndiodjpsl", model.userAssignedIdentityResourceId()); Assertions.assertEquals(CmkIdentityType.SYSTEM_ASSIGNED_IDENTITY, model.identityType()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterUpdateTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterUpdateTests.java deleted file mode 100644 index 5465c3945be3e..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ClusterUpdateTests.java +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryption; -import com.azure.resourcemanager.redisenterprise.models.ClusterUpdate; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentity; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.Sku; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import com.azure.resourcemanager.redisenterprise.models.UserAssignedIdentity; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; - -public final class ClusterUpdateTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - ClusterUpdate model = - BinaryData - .fromString( - "{\"sku\":{\"name\":\"EnterpriseFlash_F300\",\"capacity\":422670277},\"properties\":{\"minimumTlsVersion\":\"1.0\",\"encryption\":{},\"hostName\":\"tnhxbn\",\"provisioningState\":\"Creating\",\"resourceState\":\"EnableFailed\",\"redisVersion\":\"rglssainqpj\",\"privateEndpointConnections\":[{\"id\":\"jfm\",\"name\":\"pee\",\"type\":\"vmgxsab\"},{\"id\":\"duuji\",\"name\":\"c\",\"type\":\"czdzev\"},{\"id\":\"krwpdap\",\"name\":\"dsbdkvwrwjf\",\"type\":\"usnhutje\"},{\"id\":\"rl\",\"name\":\"hugjzzdatqxhoc\",\"type\":\"geablgphuticndvk\"}]},\"identity\":{\"principalId\":\"e84f62b1-2b03-467b-b87a-d96e0b7bea22\",\"tenantId\":\"658aedbe-c6c3-465e-887d-c6bc7115ba00\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{\"hxh\":{\"principalId\":\"ef6b2490-4e98-4cfb-b2e2-351940dfc312\",\"clientId\":\"05a96c3d-d9e2-42d5-aa75-35d9cd89e45c\"},\"k\":{\"principalId\":\"4c71974d-b491-42db-8c3a-2b0ae199951c\",\"clientId\":\"6d3c5a9d-76cb-4700-b927-e153ac1b3d29\"},\"xolniwpwcukjfk\":{\"principalId\":\"3c01bd84-5c98-4fff-acf3-94fdcf59eff5\",\"clientId\":\"dd3813d9-95e6-4ddf-86ce-64f6207edea3\"}}},\"tags\":{\"klryplwck\":\"w\",\"pnddhsgcbacphejk\":\"asy\"}}") - .toObject(ClusterUpdate.class); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F300, model.sku().name()); - Assertions.assertEquals(422670277, model.sku().capacity()); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.identity().type()); - Assertions.assertEquals("w", model.tags().get("klryplwck")); - Assertions.assertEquals(TlsVersion.ONE_ZERO, model.minimumTlsVersion()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - ClusterUpdate model = - new ClusterUpdate() - .withSku(new Sku().withName(SkuName.ENTERPRISE_FLASH_F300).withCapacity(422670277)) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf( - "hxh", - new UserAssignedIdentity(), - "k", - new UserAssignedIdentity(), - "xolniwpwcukjfk", - new UserAssignedIdentity()))) - .withTags(mapOf("klryplwck", "w", "pnddhsgcbacphejk", "asy")) - .withMinimumTlsVersion(TlsVersion.ONE_ZERO) - .withEncryption(new ClusterPropertiesEncryption()); - model = BinaryData.fromObject(model).toObject(ClusterUpdate.class); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F300, model.sku().name()); - Assertions.assertEquals(422670277, model.sku().capacity()); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, model.identity().type()); - Assertions.assertEquals("w", model.tags().get("klryplwck")); - Assertions.assertEquals(TlsVersion.ONE_ZERO, model.minimumTlsVersion()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseInnerTests.java index 8bc50f5acc18a..ab94e225b219f 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseInnerTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseInnerTests.java @@ -10,6 +10,7 @@ import com.azure.resourcemanager.redisenterprise.models.ClusteringPolicy; import com.azure.resourcemanager.redisenterprise.models.DatabasePropertiesGeoReplication; import com.azure.resourcemanager.redisenterprise.models.EvictionPolicy; +import com.azure.resourcemanager.redisenterprise.models.LinkedDatabase; import com.azure.resourcemanager.redisenterprise.models.Module; import com.azure.resourcemanager.redisenterprise.models.Persistence; import com.azure.resourcemanager.redisenterprise.models.Protocol; @@ -20,60 +21,47 @@ public final class DatabaseInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DatabaseInner model = - BinaryData - .fromString( - "{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":248377661,\"provisioningState\":\"Succeeded\",\"resourceState\":\"Creating\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"VolatileRandom\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"always\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"ftutqxlngxlefgu\",\"args\":\"xkrxdqmi\",\"version\":\"thz\"},{\"name\":\"vqdra\",\"args\":\"jybige\",\"version\":\"qfbow\"},{\"name\":\"kanyktzlcuiywg\",\"args\":\"wgndrvynhzgpp\",\"version\":\"cgyncocpecf\"},{\"name\":\"m\",\"args\":\"oo\",\"version\":\"xlzevgbmqjqabcy\"}],\"geoReplication\":{\"groupNickname\":\"vkwlzuvccfwnf\",\"linkedDatabases\":[]}},\"id\":\"fionl\",\"name\":\"bxetqgtzxdpn\",\"type\":\"bqqwxrj\"}") - .toObject(DatabaseInner.class); + DatabaseInner model = BinaryData.fromString( + "{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1181471279,\"provisioningState\":\"Deleting\",\"resourceState\":\"Disabled\",\"clusteringPolicy\":\"EnterpriseCluster\",\"evictionPolicy\":\"VolatileTTL\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"rsoodqxhcrmnoh\",\"args\":\"ckwhds\",\"version\":\"fiyipjxsqwpgrj\"},{\"name\":\"znorcj\",\"args\":\"snb\",\"version\":\"qabnmoc\"},{\"name\":\"cyshurzafbljjgp\",\"args\":\"oq\",\"version\":\"mkljavb\"},{\"name\":\"idtqajzyu\",\"args\":\"kudjkrlkhb\",\"version\":\"fepgzgq\"}],\"geoReplication\":{\"groupNickname\":\"locx\",\"linkedDatabases\":[{\"id\":\"ierhhbcsglummaj\",\"state\":\"UnlinkFailed\"},{\"id\":\"dxob\",\"state\":\"UnlinkFailed\"}]}},\"id\":\"xkqpxo\",\"name\":\"ajionpimexgstxg\",\"type\":\"po\"}") + .toObject(DatabaseInner.class); Assertions.assertEquals(Protocol.ENCRYPTED, model.clientProtocol()); - Assertions.assertEquals(248377661, model.port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.VOLATILE_RANDOM, model.evictionPolicy()); + Assertions.assertEquals(1181471279, model.port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.VOLATILE_TTL, model.evictionPolicy()); Assertions.assertEquals(true, model.persistence().aofEnabled()); Assertions.assertEquals(false, model.persistence().rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, model.persistence().aofFrequency()); + Assertions.assertEquals(AofFrequency.ONES, model.persistence().aofFrequency()); Assertions.assertEquals(RdbFrequency.SIXH, model.persistence().rdbFrequency()); - Assertions.assertEquals("ftutqxlngxlefgu", model.modules().get(0).name()); - Assertions.assertEquals("xkrxdqmi", model.modules().get(0).args()); - Assertions.assertEquals("vkwlzuvccfwnf", model.geoReplication().groupNickname()); + Assertions.assertEquals("rsoodqxhcrmnoh", model.modules().get(0).name()); + Assertions.assertEquals("ckwhds", model.modules().get(0).args()); + Assertions.assertEquals("locx", model.geoReplication().groupNickname()); + Assertions.assertEquals("ierhhbcsglummaj", model.geoReplication().linkedDatabases().get(0).id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabaseInner model = - new DatabaseInner() - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(248377661) - .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER) - .withEvictionPolicy(EvictionPolicy.VOLATILE_RANDOM) - .withPersistence( - new Persistence() - .withAofEnabled(true) - .withRdbEnabled(false) - .withAofFrequency(AofFrequency.ALWAYS) - .withRdbFrequency(RdbFrequency.SIXH)) - .withModules( - Arrays - .asList( - new Module().withName("ftutqxlngxlefgu").withArgs("xkrxdqmi"), - new Module().withName("vqdra").withArgs("jybige"), - new Module().withName("kanyktzlcuiywg").withArgs("wgndrvynhzgpp"), - new Module().withName("m").withArgs("oo"))) - .withGeoReplication( - new DatabasePropertiesGeoReplication() - .withGroupNickname("vkwlzuvccfwnf") - .withLinkedDatabases(Arrays.asList())); + DatabaseInner model = new DatabaseInner().withClientProtocol(Protocol.ENCRYPTED).withPort(1181471279) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER).withEvictionPolicy(EvictionPolicy.VOLATILE_TTL) + .withPersistence(new Persistence().withAofEnabled(true).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.SIXH)) + .withModules(Arrays.asList(new Module().withName("rsoodqxhcrmnoh").withArgs("ckwhds"), + new Module().withName("znorcj").withArgs("snb"), + new Module().withName("cyshurzafbljjgp").withArgs("oq"), + new Module().withName("idtqajzyu").withArgs("kudjkrlkhb"))) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("locx").withLinkedDatabases( + Arrays.asList(new LinkedDatabase().withId("ierhhbcsglummaj"), new LinkedDatabase().withId("dxob")))); model = BinaryData.fromObject(model).toObject(DatabaseInner.class); Assertions.assertEquals(Protocol.ENCRYPTED, model.clientProtocol()); - Assertions.assertEquals(248377661, model.port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.VOLATILE_RANDOM, model.evictionPolicy()); + Assertions.assertEquals(1181471279, model.port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.VOLATILE_TTL, model.evictionPolicy()); Assertions.assertEquals(true, model.persistence().aofEnabled()); Assertions.assertEquals(false, model.persistence().rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, model.persistence().aofFrequency()); + Assertions.assertEquals(AofFrequency.ONES, model.persistence().aofFrequency()); Assertions.assertEquals(RdbFrequency.SIXH, model.persistence().rdbFrequency()); - Assertions.assertEquals("ftutqxlngxlefgu", model.modules().get(0).name()); - Assertions.assertEquals("xkrxdqmi", model.modules().get(0).args()); - Assertions.assertEquals("vkwlzuvccfwnf", model.geoReplication().groupNickname()); + Assertions.assertEquals("rsoodqxhcrmnoh", model.modules().get(0).name()); + Assertions.assertEquals("ckwhds", model.modules().get(0).args()); + Assertions.assertEquals("locx", model.geoReplication().groupNickname()); + Assertions.assertEquals("ierhhbcsglummaj", model.geoReplication().linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseListTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseListTests.java index 49347126f3ae9..adce54331cdac 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseListTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseListTests.java @@ -6,50 +6,93 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.redisenterprise.fluent.models.DatabaseInner; +import com.azure.resourcemanager.redisenterprise.models.AofFrequency; import com.azure.resourcemanager.redisenterprise.models.ClusteringPolicy; import com.azure.resourcemanager.redisenterprise.models.DatabaseList; +import com.azure.resourcemanager.redisenterprise.models.DatabasePropertiesGeoReplication; import com.azure.resourcemanager.redisenterprise.models.EvictionPolicy; +import com.azure.resourcemanager.redisenterprise.models.LinkedDatabase; +import com.azure.resourcemanager.redisenterprise.models.Module; +import com.azure.resourcemanager.redisenterprise.models.Persistence; import com.azure.resourcemanager.redisenterprise.models.Protocol; +import com.azure.resourcemanager.redisenterprise.models.RdbFrequency; import java.util.Arrays; import org.junit.jupiter.api.Assertions; public final class DatabaseListTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DatabaseList model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":2146697807,\"provisioningState\":\"Deleting\",\"resourceState\":\"Disabling\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"VolatileLFU\",\"modules\":[]},\"id\":\"vddntwn\",\"name\":\"eic\",\"type\":\"twnpzaoqvuhrhcf\"},{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1519639894,\"provisioningState\":\"Failed\",\"resourceState\":\"Creating\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"AllKeysLFU\",\"modules\":[]},\"id\":\"icxm\",\"name\":\"ciwqvhk\",\"type\":\"ixuigdtopbobj\"}],\"nextLink\":\"hm\"}") - .toObject(DatabaseList.class); + DatabaseList model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1018660183,\"provisioningState\":\"Canceled\",\"resourceState\":\"Disabling\",\"clusteringPolicy\":\"EnterpriseCluster\",\"evictionPolicy\":\"NoEviction\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"1h\"},\"modules\":[{\"name\":\"qulngsntnbybkzgc\",\"args\":\"wclxxwrl\",\"version\":\"ouskcqvkocrc\"}],\"geoReplication\":{\"groupNickname\":\"wtnhxbnjbiksqr\",\"linkedDatabases\":[{},{},{},{}]}},\"id\":\"sainqpjwnzl\",\"name\":\"jfm\",\"type\":\"pee\"},{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1969125414,\"provisioningState\":\"Updating\",\"resourceState\":\"Running\",\"clusteringPolicy\":\"EnterpriseCluster\",\"evictionPolicy\":\"VolatileLRU\",\"persistence\":{\"aofEnabled\":false,\"rdbEnabled\":true,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"1h\"},\"modules\":[{\"name\":\"ndhkrw\",\"args\":\"appd\",\"version\":\"dkvwrwjfe\"},{\"name\":\"snhu\",\"args\":\"eltmrldhugjzzdat\",\"version\":\"hocdgeab\"},{\"name\":\"gphuticndvka\",\"args\":\"wyiftyhxhur\",\"version\":\"ftyxolniw\"}],\"geoReplication\":{\"groupNickname\":\"ukjfkgiawxklr\",\"linkedDatabases\":[{},{},{},{}]}},\"id\":\"wckbasyypnd\",\"name\":\"hsgcbacphejkot\",\"type\":\"nqgoulzndli\"},{\"properties\":{\"clientProtocol\":\"Plaintext\",\"port\":1581994668,\"provisioningState\":\"Creating\",\"resourceState\":\"Scaling\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"VolatileLRU\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"qedqytbciqfoufl\",\"args\":\"nkzsmodmglou\",\"version\":\"b\"},{\"name\":\"wtmutduq\",\"args\":\"ap\",\"version\":\"wgcu\"},{\"name\":\"rtumkdosvq\",\"args\":\"bmdg\",\"version\":\"jfddgmbmbe\"},{\"name\":\"ppbhtqqrolfp\",\"args\":\"s\",\"version\":\"gbquxigj\"}],\"geoReplication\":{\"groupNickname\":\"zjaoyfhrtxil\",\"linkedDatabases\":[{},{}]}},\"id\":\"kujysvlejuvfq\",\"name\":\"wrlyxwjkcprb\",\"type\":\"wbxgjvt\"},{\"properties\":{\"clientProtocol\":\"Plaintext\",\"port\":1314012997,\"provisioningState\":\"Failed\",\"resourceState\":\"Disabling\",\"clusteringPolicy\":\"EnterpriseCluster\",\"evictionPolicy\":\"VolatileTTL\",\"persistence\":{\"aofEnabled\":false,\"rdbEnabled\":false,\"aofFrequency\":\"always\",\"rdbFrequency\":\"12h\"},\"modules\":[{\"name\":\"zw\",\"args\":\"g\",\"version\":\"tnwu\"},{\"name\":\"zga\",\"args\":\"ufizuckyf\",\"version\":\"rfidfvzwdz\"},{\"name\":\"htymw\",\"args\":\"dkfthwxmnt\",\"version\":\"waopvkmijcmmxd\"}],\"geoReplication\":{\"groupNickname\":\"uf\",\"linkedDatabases\":[{},{},{},{}]}},\"id\":\"ymzidn\",\"name\":\"ezcxtbzsgfyccsne\",\"type\":\"mdwzjeiachboo\"}],\"nextLink\":\"lnrosfqp\"}") + .toObject(DatabaseList.class); Assertions.assertEquals(Protocol.ENCRYPTED, model.value().get(0).clientProtocol()); - Assertions.assertEquals(2146697807, model.value().get(0).port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.value().get(0).clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.VOLATILE_LFU, model.value().get(0).evictionPolicy()); + Assertions.assertEquals(1018660183, model.value().get(0).port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.value().get(0).clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.NO_EVICTION, model.value().get(0).evictionPolicy()); + Assertions.assertEquals(true, model.value().get(0).persistence().aofEnabled()); + Assertions.assertEquals(false, model.value().get(0).persistence().rdbEnabled()); + Assertions.assertEquals(AofFrequency.ONES, model.value().get(0).persistence().aofFrequency()); + Assertions.assertEquals(RdbFrequency.ONEH, model.value().get(0).persistence().rdbFrequency()); + Assertions.assertEquals("qulngsntnbybkzgc", model.value().get(0).modules().get(0).name()); + Assertions.assertEquals("wclxxwrl", model.value().get(0).modules().get(0).args()); + Assertions.assertEquals("wtnhxbnjbiksqr", model.value().get(0).geoReplication().groupNickname()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabaseList model = - new DatabaseList() - .withValue( - Arrays - .asList( - new DatabaseInner() - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(2146697807) - .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER) - .withEvictionPolicy(EvictionPolicy.VOLATILE_LFU) - .withModules(Arrays.asList()), - new DatabaseInner() - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(1519639894) - .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER) - .withEvictionPolicy(EvictionPolicy.ALL_KEYS_LFU) - .withModules(Arrays.asList()))); + DatabaseList model = new DatabaseList().withValue(Arrays.asList( + new DatabaseInner().withClientProtocol(Protocol.ENCRYPTED).withPort(1018660183) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) + .withEvictionPolicy(EvictionPolicy.NO_EVICTION) + .withPersistence(new Persistence().withAofEnabled(true).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.ONEH)) + .withModules(Arrays.asList(new Module().withName("qulngsntnbybkzgc").withArgs("wclxxwrl"))) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("wtnhxbnjbiksqr") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase(), new LinkedDatabase(), new LinkedDatabase(), + new LinkedDatabase()))), + new DatabaseInner().withClientProtocol(Protocol.ENCRYPTED).withPort(1969125414) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) + .withEvictionPolicy(EvictionPolicy.VOLATILE_LRU) + .withPersistence(new Persistence().withAofEnabled(false).withRdbEnabled(true) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.ONEH)) + .withModules(Arrays.asList(new Module().withName("ndhkrw").withArgs("appd"), + new Module().withName("snhu").withArgs("eltmrldhugjzzdat"), + new Module().withName("gphuticndvka").withArgs("wyiftyhxhur"))) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("ukjfkgiawxklr") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase(), new LinkedDatabase(), new LinkedDatabase(), + new LinkedDatabase()))), + new DatabaseInner().withClientProtocol(Protocol.PLAINTEXT).withPort(1581994668) + .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER).withEvictionPolicy(EvictionPolicy.VOLATILE_LRU) + .withPersistence(new Persistence().withAofEnabled(true).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.SIXH)) + .withModules(Arrays.asList(new Module().withName("qedqytbciqfoufl").withArgs("nkzsmodmglou"), + new Module().withName("wtmutduq").withArgs("ap"), + new Module().withName("rtumkdosvq").withArgs("bmdg"), + new Module().withName("ppbhtqqrolfp").withArgs("s"))) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("zjaoyfhrtxil") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase(), new LinkedDatabase()))), + new DatabaseInner().withClientProtocol(Protocol.PLAINTEXT).withPort(1314012997) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) + .withEvictionPolicy(EvictionPolicy.VOLATILE_TTL) + .withPersistence(new Persistence().withAofEnabled(false).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ALWAYS).withRdbFrequency(RdbFrequency.ONE_TWOH)) + .withModules(Arrays.asList(new Module().withName("zw").withArgs("g"), + new Module().withName("zga").withArgs("ufizuckyf"), + new Module().withName("htymw").withArgs("dkfthwxmnt"))) + .withGeoReplication( + new DatabasePropertiesGeoReplication().withGroupNickname("uf").withLinkedDatabases(Arrays.asList( + new LinkedDatabase(), new LinkedDatabase(), new LinkedDatabase(), new LinkedDatabase()))))); model = BinaryData.fromObject(model).toObject(DatabaseList.class); Assertions.assertEquals(Protocol.ENCRYPTED, model.value().get(0).clientProtocol()); - Assertions.assertEquals(2146697807, model.value().get(0).port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.value().get(0).clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.VOLATILE_LFU, model.value().get(0).evictionPolicy()); + Assertions.assertEquals(1018660183, model.value().get(0).port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.value().get(0).clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.NO_EVICTION, model.value().get(0).evictionPolicy()); + Assertions.assertEquals(true, model.value().get(0).persistence().aofEnabled()); + Assertions.assertEquals(false, model.value().get(0).persistence().rdbEnabled()); + Assertions.assertEquals(AofFrequency.ONES, model.value().get(0).persistence().aofFrequency()); + Assertions.assertEquals(RdbFrequency.ONEH, model.value().get(0).persistence().rdbFrequency()); + Assertions.assertEquals("qulngsntnbybkzgc", model.value().get(0).modules().get(0).name()); + Assertions.assertEquals("wclxxwrl", model.value().get(0).modules().get(0).args()); + Assertions.assertEquals("wtnhxbnjbiksqr", model.value().get(0).geoReplication().groupNickname()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesGeoReplicationTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesGeoReplicationTests.java index 6cf75af31d898..bbfda1488ce93 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesGeoReplicationTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesGeoReplicationTests.java @@ -13,29 +13,21 @@ public final class DatabasePropertiesGeoReplicationTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DatabasePropertiesGeoReplication model = - BinaryData - .fromString( - "{\"groupNickname\":\"zzvdudgwds\",\"linkedDatabases\":[{\"id\":\"twmcynpwlb\",\"state\":\"LinkFailed\"},{\"id\":\"acfta\",\"state\":\"UnlinkFailed\"},{\"id\":\"nltyfsoppusuesnz\",\"state\":\"Linking\"},{\"id\":\"bavo\",\"state\":\"Unlinking\"}]}") - .toObject(DatabasePropertiesGeoReplication.class); - Assertions.assertEquals("zzvdudgwds", model.groupNickname()); - Assertions.assertEquals("twmcynpwlb", model.linkedDatabases().get(0).id()); + DatabasePropertiesGeoReplication model = BinaryData.fromString( + "{\"groupNickname\":\"q\",\"linkedDatabases\":[{\"id\":\"ixuigdtopbobj\",\"state\":\"UnlinkFailed\"},{\"id\":\"e\",\"state\":\"Linked\"},{\"id\":\"a\",\"state\":\"LinkFailed\"},{\"id\":\"ayvvtpgvdf\",\"state\":\"Unlinking\"}]}") + .toObject(DatabasePropertiesGeoReplication.class); + Assertions.assertEquals("q", model.groupNickname()); + Assertions.assertEquals("ixuigdtopbobj", model.linkedDatabases().get(0).id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabasePropertiesGeoReplication model = - new DatabasePropertiesGeoReplication() - .withGroupNickname("zzvdudgwds") - .withLinkedDatabases( - Arrays - .asList( - new LinkedDatabase().withId("twmcynpwlb"), - new LinkedDatabase().withId("acfta"), - new LinkedDatabase().withId("nltyfsoppusuesnz"), - new LinkedDatabase().withId("bavo"))); + DatabasePropertiesGeoReplication model + = new DatabasePropertiesGeoReplication().withGroupNickname("q").withLinkedDatabases( + Arrays.asList(new LinkedDatabase().withId("ixuigdtopbobj"), new LinkedDatabase().withId("e"), + new LinkedDatabase().withId("a"), new LinkedDatabase().withId("ayvvtpgvdf"))); model = BinaryData.fromObject(model).toObject(DatabasePropertiesGeoReplication.class); - Assertions.assertEquals("zzvdudgwds", model.groupNickname()); - Assertions.assertEquals("twmcynpwlb", model.linkedDatabases().get(0).id()); + Assertions.assertEquals("q", model.groupNickname()); + Assertions.assertEquals("ixuigdtopbobj", model.linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesTests.java index 1717ca2791d1b..8d219ecd3afd3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasePropertiesTests.java @@ -21,56 +21,45 @@ public final class DatabasePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DatabaseProperties model = - BinaryData - .fromString( - "{\"clientProtocol\":\"Encrypted\",\"port\":61111335,\"provisioningState\":\"Failed\",\"resourceState\":\"Running\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"AllKeysRandom\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":true,\"aofFrequency\":\"always\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"xaqwoochcbonqv\",\"args\":\"vlrxnjeaseiph\",\"version\":\"f\"}],\"geoReplication\":{\"groupNickname\":\"eyy\",\"linkedDatabases\":[{\"id\":\"bdlwtgrhpdjpj\",\"state\":\"Unlinking\"}]}}") - .toObject(DatabaseProperties.class); - Assertions.assertEquals(Protocol.ENCRYPTED, model.clientProtocol()); - Assertions.assertEquals(61111335, model.port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.ALL_KEYS_RANDOM, model.evictionPolicy()); + DatabaseProperties model = BinaryData.fromString( + "{\"clientProtocol\":\"Plaintext\",\"port\":1860899173,\"provisioningState\":\"Deleting\",\"resourceState\":\"CreateFailed\",\"clusteringPolicy\":\"EnterpriseCluster\",\"evictionPolicy\":\"NoEviction\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"always\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"coejctbzaqs\",\"args\":\"y\",\"version\":\"kbfkg\"}],\"geoReplication\":{\"groupNickname\":\"kexxppof\",\"linkedDatabases\":[{\"id\":\"c\",\"state\":\"Unlinking\"},{\"id\":\"ddtocjjxhvp\",\"state\":\"Unlinking\"},{\"id\":\"xhdzxibqeojnx\",\"state\":\"Unlinking\"},{\"id\":\"ddntwndei\",\"state\":\"Unlinking\"}]}}") + .toObject(DatabaseProperties.class); + Assertions.assertEquals(Protocol.PLAINTEXT, model.clientProtocol()); + Assertions.assertEquals(1860899173, model.port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.NO_EVICTION, model.evictionPolicy()); Assertions.assertEquals(true, model.persistence().aofEnabled()); - Assertions.assertEquals(true, model.persistence().rdbEnabled()); + Assertions.assertEquals(false, model.persistence().rdbEnabled()); Assertions.assertEquals(AofFrequency.ALWAYS, model.persistence().aofFrequency()); Assertions.assertEquals(RdbFrequency.SIXH, model.persistence().rdbFrequency()); - Assertions.assertEquals("xaqwoochcbonqv", model.modules().get(0).name()); - Assertions.assertEquals("vlrxnjeaseiph", model.modules().get(0).args()); - Assertions.assertEquals("eyy", model.geoReplication().groupNickname()); - Assertions.assertEquals("bdlwtgrhpdjpj", model.geoReplication().linkedDatabases().get(0).id()); + Assertions.assertEquals("coejctbzaqs", model.modules().get(0).name()); + Assertions.assertEquals("y", model.modules().get(0).args()); + Assertions.assertEquals("kexxppof", model.geoReplication().groupNickname()); + Assertions.assertEquals("c", model.geoReplication().linkedDatabases().get(0).id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabaseProperties model = - new DatabaseProperties() - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(61111335) - .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER) - .withEvictionPolicy(EvictionPolicy.ALL_KEYS_RANDOM) - .withPersistence( - new Persistence() - .withAofEnabled(true) - .withRdbEnabled(true) - .withAofFrequency(AofFrequency.ALWAYS) - .withRdbFrequency(RdbFrequency.SIXH)) - .withModules(Arrays.asList(new Module().withName("xaqwoochcbonqv").withArgs("vlrxnjeaseiph"))) - .withGeoReplication( - new DatabasePropertiesGeoReplication() - .withGroupNickname("eyy") - .withLinkedDatabases(Arrays.asList(new LinkedDatabase().withId("bdlwtgrhpdjpj")))); + DatabaseProperties model = new DatabaseProperties().withClientProtocol(Protocol.PLAINTEXT).withPort(1860899173) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER).withEvictionPolicy(EvictionPolicy.NO_EVICTION) + .withPersistence(new Persistence().withAofEnabled(true).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ALWAYS).withRdbFrequency(RdbFrequency.SIXH)) + .withModules(Arrays.asList(new Module().withName("coejctbzaqs").withArgs("y"))).withGeoReplication( + new DatabasePropertiesGeoReplication().withGroupNickname("kexxppof").withLinkedDatabases( + Arrays.asList(new LinkedDatabase().withId("c"), new LinkedDatabase().withId("ddtocjjxhvp"), + new LinkedDatabase().withId("xhdzxibqeojnx"), new LinkedDatabase().withId("ddntwndei")))); model = BinaryData.fromObject(model).toObject(DatabaseProperties.class); - Assertions.assertEquals(Protocol.ENCRYPTED, model.clientProtocol()); - Assertions.assertEquals(61111335, model.port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.ALL_KEYS_RANDOM, model.evictionPolicy()); + Assertions.assertEquals(Protocol.PLAINTEXT, model.clientProtocol()); + Assertions.assertEquals(1860899173, model.port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.NO_EVICTION, model.evictionPolicy()); Assertions.assertEquals(true, model.persistence().aofEnabled()); - Assertions.assertEquals(true, model.persistence().rdbEnabled()); + Assertions.assertEquals(false, model.persistence().rdbEnabled()); Assertions.assertEquals(AofFrequency.ALWAYS, model.persistence().aofFrequency()); Assertions.assertEquals(RdbFrequency.SIXH, model.persistence().rdbFrequency()); - Assertions.assertEquals("xaqwoochcbonqv", model.modules().get(0).name()); - Assertions.assertEquals("vlrxnjeaseiph", model.modules().get(0).args()); - Assertions.assertEquals("eyy", model.geoReplication().groupNickname()); - Assertions.assertEquals("bdlwtgrhpdjpj", model.geoReplication().linkedDatabases().get(0).id()); + Assertions.assertEquals("coejctbzaqs", model.modules().get(0).name()); + Assertions.assertEquals("y", model.modules().get(0).args()); + Assertions.assertEquals("kexxppof", model.geoReplication().groupNickname()); + Assertions.assertEquals("c", model.geoReplication().linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseUpdateTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseUpdateTests.java index b3064decc0129..28d452bdb93e3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseUpdateTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabaseUpdateTests.java @@ -10,6 +10,7 @@ import com.azure.resourcemanager.redisenterprise.models.DatabasePropertiesGeoReplication; import com.azure.resourcemanager.redisenterprise.models.DatabaseUpdate; import com.azure.resourcemanager.redisenterprise.models.EvictionPolicy; +import com.azure.resourcemanager.redisenterprise.models.LinkedDatabase; import com.azure.resourcemanager.redisenterprise.models.Module; import com.azure.resourcemanager.redisenterprise.models.Persistence; import com.azure.resourcemanager.redisenterprise.models.Protocol; @@ -20,58 +21,46 @@ public final class DatabaseUpdateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - DatabaseUpdate model = - BinaryData - .fromString( - "{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1222672575,\"provisioningState\":\"Creating\",\"resourceState\":\"Disabled\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"NoEviction\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"always\",\"rdbFrequency\":\"1h\"},\"modules\":[{\"name\":\"lazjdyggdtjixhbk\",\"args\":\"fqweykhmene\",\"version\":\"yexfwh\"},{\"name\":\"bcibvyvdcsitynn\",\"args\":\"mdectehfiqscjey\",\"version\":\"hezrkgq\"}],\"geoReplication\":{\"groupNickname\":\"refovgmkqsleyyvx\",\"linkedDatabases\":[]}}}") - .toObject(DatabaseUpdate.class); + DatabaseUpdate model = BinaryData.fromString( + "{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1788312026,\"provisioningState\":\"Succeeded\",\"resourceState\":\"Running\",\"clusteringPolicy\":\"EnterpriseCluster\",\"evictionPolicy\":\"AllKeysRandom\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":true,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"igeho\",\"args\":\"bowsk\",\"version\":\"yktz\"}],\"geoReplication\":{\"groupNickname\":\"iywgqywgndrvynh\",\"linkedDatabases\":[{\"id\":\"hrc\",\"state\":\"Linked\"},{\"id\":\"ocpecfvmmco\",\"state\":\"LinkFailed\"},{\"id\":\"lzevgbmqjqab\",\"state\":\"UnlinkFailed\"}]}}}") + .toObject(DatabaseUpdate.class); Assertions.assertEquals(Protocol.ENCRYPTED, model.clientProtocol()); - Assertions.assertEquals(1222672575, model.port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.NO_EVICTION, model.evictionPolicy()); + Assertions.assertEquals(1788312026, model.port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.ALL_KEYS_RANDOM, model.evictionPolicy()); Assertions.assertEquals(true, model.persistence().aofEnabled()); - Assertions.assertEquals(false, model.persistence().rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, model.persistence().aofFrequency()); - Assertions.assertEquals(RdbFrequency.ONEH, model.persistence().rdbFrequency()); - Assertions.assertEquals("lazjdyggdtjixhbk", model.modules().get(0).name()); - Assertions.assertEquals("fqweykhmene", model.modules().get(0).args()); - Assertions.assertEquals("refovgmkqsleyyvx", model.geoReplication().groupNickname()); + Assertions.assertEquals(true, model.persistence().rdbEnabled()); + Assertions.assertEquals(AofFrequency.ONES, model.persistence().aofFrequency()); + Assertions.assertEquals(RdbFrequency.SIXH, model.persistence().rdbFrequency()); + Assertions.assertEquals("igeho", model.modules().get(0).name()); + Assertions.assertEquals("bowsk", model.modules().get(0).args()); + Assertions.assertEquals("iywgqywgndrvynh", model.geoReplication().groupNickname()); + Assertions.assertEquals("hrc", model.geoReplication().linkedDatabases().get(0).id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - DatabaseUpdate model = - new DatabaseUpdate() - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(1222672575) - .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER) - .withEvictionPolicy(EvictionPolicy.NO_EVICTION) - .withPersistence( - new Persistence() - .withAofEnabled(true) - .withRdbEnabled(false) - .withAofFrequency(AofFrequency.ALWAYS) - .withRdbFrequency(RdbFrequency.ONEH)) - .withModules( - Arrays - .asList( - new Module().withName("lazjdyggdtjixhbk").withArgs("fqweykhmene"), - new Module().withName("bcibvyvdcsitynn").withArgs("mdectehfiqscjey"))) - .withGeoReplication( - new DatabasePropertiesGeoReplication() - .withGroupNickname("refovgmkqsleyyvx") - .withLinkedDatabases(Arrays.asList())); + DatabaseUpdate model = new DatabaseUpdate().withClientProtocol(Protocol.ENCRYPTED).withPort(1788312026) + .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) + .withEvictionPolicy(EvictionPolicy.ALL_KEYS_RANDOM) + .withPersistence(new Persistence().withAofEnabled(true).withRdbEnabled(true) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.SIXH)) + .withModules(Arrays.asList(new Module().withName("igeho").withArgs("bowsk"))) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("iywgqywgndrvynh") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase().withId("hrc"), + new LinkedDatabase().withId("ocpecfvmmco"), new LinkedDatabase().withId("lzevgbmqjqab")))); model = BinaryData.fromObject(model).toObject(DatabaseUpdate.class); Assertions.assertEquals(Protocol.ENCRYPTED, model.clientProtocol()); - Assertions.assertEquals(1222672575, model.port()); - Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, model.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.NO_EVICTION, model.evictionPolicy()); + Assertions.assertEquals(1788312026, model.port()); + Assertions.assertEquals(ClusteringPolicy.ENTERPRISE_CLUSTER, model.clusteringPolicy()); + Assertions.assertEquals(EvictionPolicy.ALL_KEYS_RANDOM, model.evictionPolicy()); Assertions.assertEquals(true, model.persistence().aofEnabled()); - Assertions.assertEquals(false, model.persistence().rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, model.persistence().aofFrequency()); - Assertions.assertEquals(RdbFrequency.ONEH, model.persistence().rdbFrequency()); - Assertions.assertEquals("lazjdyggdtjixhbk", model.modules().get(0).name()); - Assertions.assertEquals("fqweykhmene", model.modules().get(0).args()); - Assertions.assertEquals("refovgmkqsleyyvx", model.geoReplication().groupNickname()); + Assertions.assertEquals(true, model.persistence().rdbEnabled()); + Assertions.assertEquals(AofFrequency.ONES, model.persistence().aofFrequency()); + Assertions.assertEquals(RdbFrequency.SIXH, model.persistence().rdbFrequency()); + Assertions.assertEquals("igeho", model.modules().get(0).name()); + Assertions.assertEquals("bowsk", model.modules().get(0).args()); + Assertions.assertEquals("iywgqywgndrvynh", model.geoReplication().groupNickname()); + Assertions.assertEquals("hrc", model.geoReplication().linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateMockTests.java index 1d026883e806c..2500fcfb67429 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesCreateMockTests.java @@ -17,6 +17,8 @@ import com.azure.resourcemanager.redisenterprise.models.Database; import com.azure.resourcemanager.redisenterprise.models.DatabasePropertiesGeoReplication; import com.azure.resourcemanager.redisenterprise.models.EvictionPolicy; +import com.azure.resourcemanager.redisenterprise.models.LinkedDatabase; +import com.azure.resourcemanager.redisenterprise.models.Module; import com.azure.resourcemanager.redisenterprise.models.Persistence; import com.azure.resourcemanager.redisenterprise.models.Protocol; import com.azure.resourcemanager.redisenterprise.models.RdbFrequency; @@ -38,65 +40,51 @@ public void testCreate() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":1688847935,\"provisioningState\":\"Succeeded\",\"resourceState\":\"CreateFailed\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"VolatileLFU\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"12h\"},\"modules\":[],\"geoReplication\":{\"groupNickname\":\"qaqtdoqmcbxvwvxy\",\"linkedDatabases\":[]}},\"id\":\"bhsfxob\",\"name\":\"ytkblmpew\",\"type\":\"wfbkrvrns\"}"; + String responseStr + = "{\"properties\":{\"clientProtocol\":\"Plaintext\",\"port\":2046724372,\"provisioningState\":\"Succeeded\",\"resourceState\":\"UpdateFailed\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"VolatileTTL\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":true,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"6h\"},\"modules\":[{\"name\":\"pvjymjhxxjyng\",\"args\":\"ivkrtsw\",\"version\":\"qzvszjf\"},{\"name\":\"uvjfdxxive\",\"args\":\"t\",\"version\":\"aqtdoqmcbx\"},{\"name\":\"wvxysl\",\"args\":\"hsfxoblytkb\",\"version\":\"pe\"}],\"geoReplication\":{\"groupNickname\":\"fbkrvrnsvs\",\"linkedDatabases\":[{\"id\":\"hxcr\",\"state\":\"UnlinkFailed\"},{\"id\":\"vasrruvwb\",\"state\":\"Linking\"}]}},\"id\":\"fsubcgjbirxbpy\",\"name\":\"srfbjfdtwss\",\"type\":\"t\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Database response = - manager - .databases() - .define("pfqbuaceopzf") - .withExistingRedisEnterprise("pkteo", "llwptfdy") - .withClientProtocol(Protocol.ENCRYPTED) - .withPort(350693240) - .withClusteringPolicy(ClusteringPolicy.ENTERPRISE_CLUSTER) - .withEvictionPolicy(EvictionPolicy.VOLATILE_LFU) - .withPersistence( - new Persistence() - .withAofEnabled(true) - .withRdbEnabled(true) - .withAofFrequency(AofFrequency.ONES) - .withRdbFrequency(RdbFrequency.ONEH)) - .withModules(Arrays.asList()) - .withGeoReplication( - new DatabasePropertiesGeoReplication() - .withGroupNickname("gbkdmoizpos") - .withLinkedDatabases(Arrays.asList())) + Database response + = manager.databases().define("vawjvzunlu").withExistingRedisEnterprise("vgqzcjrvxd", "zlmwlxkvugfhz") + .withClientProtocol(Protocol.ENCRYPTED).withPort(1845273018) + .withClusteringPolicy(ClusteringPolicy.OSSCLUSTER).withEvictionPolicy(EvictionPolicy.ALL_KEYS_LRU) + .withPersistence(new Persistence().withAofEnabled(true).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.SIXH)) + .withModules(Arrays.asList(new Module().withName("skzbb").withArgs("zumveekgpwo"), + new Module().withName("yofd").withArgs("uusdttouwa"), + new Module().withName("mvb").withArgs("yjsflhhcaalnji"), + new Module().withName("yjpkiidzyexz").withArgs("lixhnrztfol"))) + .withGeoReplication(new DatabasePropertiesGeoReplication().withGroupNickname("dtpnapnyiropuhp") + .withLinkedDatabases(Arrays.asList(new LinkedDatabase().withId("gylgqgitxmedjvcs"), + new LinkedDatabase().withId("wwncwzzhxgk"), new LinkedDatabase().withId("ucnapkteoellwp"), + new LinkedDatabase().withId("gpfqbuace")))) .create(); - Assertions.assertEquals(Protocol.ENCRYPTED, response.clientProtocol()); - Assertions.assertEquals(1688847935, response.port()); + Assertions.assertEquals(Protocol.PLAINTEXT, response.clientProtocol()); + Assertions.assertEquals(2046724372, response.port()); Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, response.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.VOLATILE_LFU, response.evictionPolicy()); + Assertions.assertEquals(EvictionPolicy.VOLATILE_TTL, response.evictionPolicy()); Assertions.assertEquals(true, response.persistence().aofEnabled()); - Assertions.assertEquals(false, response.persistence().rdbEnabled()); + Assertions.assertEquals(true, response.persistence().rdbEnabled()); Assertions.assertEquals(AofFrequency.ONES, response.persistence().aofFrequency()); - Assertions.assertEquals(RdbFrequency.ONE_TWOH, response.persistence().rdbFrequency()); - Assertions.assertEquals("qaqtdoqmcbxvwvxy", response.geoReplication().groupNickname()); + Assertions.assertEquals(RdbFrequency.SIXH, response.persistence().rdbFrequency()); + Assertions.assertEquals("pvjymjhxxjyng", response.modules().get(0).name()); + Assertions.assertEquals("ivkrtsw", response.modules().get(0).args()); + Assertions.assertEquals("fbkrvrnsvs", response.geoReplication().groupNickname()); + Assertions.assertEquals("hxcr", response.geoReplication().linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteMockTests.java index 35e3751ef932f..2d2f0842a9ecc 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.databases().delete("aloayqcgwrtzju", "gwyzm", "txon", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.databases().delete("hashsfwxosow", "xcug", "cjooxdjebwpucwwf", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportMockTests.java index 7571027a5ed35..378d3b732eda3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesExportMockTests.java @@ -33,37 +33,21 @@ public void testExport() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.databases().export("fgb", "c", "wxzvlvqhjkb", new ExportClusterParameters().withSasUri("gibtnm"), + com.azure.core.util.Context.NONE); - manager - .databases() - .export( - "us", - "ttouwaboekqvkel", - "smv", - new ExportClusterParameters().withSasUri("xwyjsflhhc"), - com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushMockTests.java index 3f68e7a662de9..396e64753bc92 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesFlushMockTests.java @@ -34,37 +34,22 @@ public void testFlush() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.databases().flush("xongmtsavjcbpwxq", "srknftguv", "iuhprwmdyvxqta", + new FlushParameters().withIds(Arrays.asList("ww", "oyq", "exrmcqibycnojvk", "mefqsgzvahapjyzh")), + com.azure.core.util.Context.NONE); - manager - .databases() - .flush( - "tfolhbnx", - "nalaulppg", - "dtpnapnyiropuhp", - new FlushParameters().withIds(Arrays.asList("pgylg", "git", "medjvcslynqwwncw", "zhxgktrmgucn")), - com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkMockTests.java index e588496574d9a..ee3f4457f8ad3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesForceUnlinkMockTests.java @@ -34,37 +34,21 @@ public void testForceUnlink() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.databases().forceUnlink("iebwwaloayqcgwrt", "j", "zg", + new ForceUnlinkParameters().withIds(Arrays.asList("zmh")), com.azure.core.util.Context.NONE); - manager - .databases() - .forceUnlink( - "aln", - "ixisxyawjoy", - "qcslyjpkiid", - new ForceUnlinkParameters().withIds(Arrays.asList("exznelixhnr")), - com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetWithResponseMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetWithResponseMockTests.java index e44c59e23dae2..cbd3f47f3da20 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetWithResponseMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesGetWithResponseMockTests.java @@ -35,49 +35,38 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"clientProtocol\":\"Plaintext\",\"port\":96984034,\"provisioningState\":\"Succeeded\",\"resourceState\":\"DeleteFailed\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"NoEviction\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":true,\"aofFrequency\":\"always\",\"rdbFrequency\":\"12h\"},\"modules\":[],\"geoReplication\":{\"groupNickname\":\"blcg\",\"linkedDatabases\":[]}},\"id\":\"vlvqhjkbegi\",\"name\":\"t\",\"type\":\"mxiebw\"}"; + String responseStr + = "{\"properties\":{\"clientProtocol\":\"Plaintext\",\"port\":435564330,\"provisioningState\":\"Canceled\",\"resourceState\":\"Deleting\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"AllKeysLFU\",\"persistence\":{\"aofEnabled\":false,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"1h\"},\"modules\":[{\"name\":\"hdzhlrqj\",\"args\":\"ck\",\"version\":\"lhrxsbkyvpyc\"},{\"name\":\"n\",\"args\":\"bpzkafkuwbc\",\"version\":\"wbme\"},{\"name\":\"hseyvju\",\"args\":\"tslhspkdeem\",\"version\":\"fm\"},{\"name\":\"ag\",\"args\":\"t\",\"version\":\"lmqkrhahvlj\"}],\"geoReplication\":{\"groupNickname\":\"aquhcdhm\",\"linkedDatabases\":[{\"id\":\"aex\",\"state\":\"Unlinking\"},{\"id\":\"adm\",\"state\":\"Linking\"},{\"id\":\"r\",\"state\":\"Unlinking\"},{\"id\":\"vgomz\",\"state\":\"Linking\"}]}},\"id\":\"isgwbnbbeldawkz\",\"name\":\"ali\",\"type\":\"urqhaka\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - Database response = - manager - .databases() - .getWithResponse("pow", "wpr", "qlveualupjmkh", com.azure.core.util.Context.NONE) - .getValue(); + Database response = manager.databases() + .getWithResponse("kqujidsuyono", "glaocq", "tcc", com.azure.core.util.Context.NONE).getValue(); Assertions.assertEquals(Protocol.PLAINTEXT, response.clientProtocol()); - Assertions.assertEquals(96984034, response.port()); + Assertions.assertEquals(435564330, response.port()); Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, response.clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.NO_EVICTION, response.evictionPolicy()); - Assertions.assertEquals(true, response.persistence().aofEnabled()); - Assertions.assertEquals(true, response.persistence().rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, response.persistence().aofFrequency()); - Assertions.assertEquals(RdbFrequency.ONE_TWOH, response.persistence().rdbFrequency()); - Assertions.assertEquals("blcg", response.geoReplication().groupNickname()); + Assertions.assertEquals(EvictionPolicy.ALL_KEYS_LFU, response.evictionPolicy()); + Assertions.assertEquals(false, response.persistence().aofEnabled()); + Assertions.assertEquals(false, response.persistence().rdbEnabled()); + Assertions.assertEquals(AofFrequency.ONES, response.persistence().aofFrequency()); + Assertions.assertEquals(RdbFrequency.ONEH, response.persistence().rdbFrequency()); + Assertions.assertEquals("hdzhlrqj", response.modules().get(0).name()); + Assertions.assertEquals("ck", response.modules().get(0).args()); + Assertions.assertEquals("aquhcdhm", response.geoReplication().groupNickname()); + Assertions.assertEquals("aex", response.geoReplication().linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodMockTests.java index e1dece8e9b95e..359e7a28196d3 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesImportMethodMockTests.java @@ -34,37 +34,22 @@ public void testImportMethod() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.databases().importMethod("wzbtdhxu", "znbmpowuwprzq", "veual", + new ImportClusterParameters().withSasUris(Arrays.asList("j", "khfxobbcswsrt", "riplrbpbewtg")), + com.azure.core.util.Context.NONE); - manager - .databases() - .importMethod( - "awjvzunluthnnp", - "nxipeil", - "jzuaejxdultskzbb", - new ImportClusterParameters().withSasUris(Arrays.asList("zumveekgpwo", "uh", "fpbsjyofdxl")), - com.azure.core.util.Context.NONE); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterMockTests.java index e4ae57805bfc3..dd01d47a6598a 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/DatabasesListByClusterMockTests.java @@ -36,46 +36,38 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"clientProtocol\":\"Encrypted\",\"port\":838785311,\"provisioningState\":\"Creating\",\"resourceState\":\"Creating\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"AllKeysRandom\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"always\",\"rdbFrequency\":\"6h\"},\"modules\":[],\"geoReplication\":{\"groupNickname\":\"cg\",\"linkedDatabases\":[]}},\"id\":\"ndnhj\",\"name\":\"auwhvylwzbtdhx\",\"type\":\"jznb\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"clientProtocol\":\"Plaintext\",\"port\":718463085,\"provisioningState\":\"Creating\",\"resourceState\":\"Scaling\",\"clusteringPolicy\":\"OSSCluster\",\"evictionPolicy\":\"VolatileTTL\",\"persistence\":{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"12h\"},\"modules\":[{\"name\":\"khsmtxpsiebt\",\"args\":\"vpesapskrdqmhjjd\",\"version\":\"ldwkyzxuutkn\"},{\"name\":\"ws\",\"args\":\"svlxotogtwrup\",\"version\":\"xvnmicykvceov\"},{\"name\":\"ilovnot\",\"args\":\"jfcn\",\"version\":\"k\"}],\"geoReplication\":{\"groupNickname\":\"dhbt\",\"linkedDatabases\":[{\"id\":\"ywpnvjt\",\"state\":\"LinkFailed\"},{\"id\":\"rmclfplphoxu\",\"state\":\"LinkFailed\"}]}},\"id\":\"pabgyeps\",\"name\":\"jta\",\"type\":\"qugxywpmueefjzwf\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.databases().listByCluster("hzceuojgjrwjue", "otwmcdyt", com.azure.core.util.Context.NONE); + PagedIterable response + = manager.databases().listByCluster("sxnkjzkdeslpvlo", "wiyighxpkdw", com.azure.core.util.Context.NONE); - Assertions.assertEquals(Protocol.ENCRYPTED, response.iterator().next().clientProtocol()); - Assertions.assertEquals(838785311, response.iterator().next().port()); + Assertions.assertEquals(Protocol.PLAINTEXT, response.iterator().next().clientProtocol()); + Assertions.assertEquals(718463085, response.iterator().next().port()); Assertions.assertEquals(ClusteringPolicy.OSSCLUSTER, response.iterator().next().clusteringPolicy()); - Assertions.assertEquals(EvictionPolicy.ALL_KEYS_RANDOM, response.iterator().next().evictionPolicy()); + Assertions.assertEquals(EvictionPolicy.VOLATILE_TTL, response.iterator().next().evictionPolicy()); Assertions.assertEquals(true, response.iterator().next().persistence().aofEnabled()); Assertions.assertEquals(false, response.iterator().next().persistence().rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, response.iterator().next().persistence().aofFrequency()); - Assertions.assertEquals(RdbFrequency.SIXH, response.iterator().next().persistence().rdbFrequency()); - Assertions.assertEquals("cg", response.iterator().next().geoReplication().groupNickname()); + Assertions.assertEquals(AofFrequency.ONES, response.iterator().next().persistence().aofFrequency()); + Assertions.assertEquals(RdbFrequency.ONE_TWOH, response.iterator().next().persistence().rdbFrequency()); + Assertions.assertEquals("khsmtxpsiebt", response.iterator().next().modules().get(0).name()); + Assertions.assertEquals("vpesapskrdqmhjjd", response.iterator().next().modules().get(0).args()); + Assertions.assertEquals("dhbt", response.iterator().next().geoReplication().groupNickname()); + Assertions.assertEquals("ywpnvjt", response.iterator().next().geoReplication().linkedDatabases().get(0).id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ExportClusterParametersTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ExportClusterParametersTests.java index 9590cb888fc5f..9aec1d90eeacb 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ExportClusterParametersTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ExportClusterParametersTests.java @@ -11,15 +11,15 @@ public final class ExportClusterParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ExportClusterParameters model = - BinaryData.fromString("{\"sasUri\":\"mdajv\"}").toObject(ExportClusterParameters.class); - Assertions.assertEquals("mdajv", model.sasUri()); + ExportClusterParameters model + = BinaryData.fromString("{\"sasUri\":\"tqgtzxdpnqbqq\"}").toObject(ExportClusterParameters.class); + Assertions.assertEquals("tqgtzxdpnqbqq", model.sasUri()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ExportClusterParameters model = new ExportClusterParameters().withSasUri("mdajv"); + ExportClusterParameters model = new ExportClusterParameters().withSasUri("tqgtzxdpnqbqq"); model = BinaryData.fromObject(model).toObject(ExportClusterParameters.class); - Assertions.assertEquals("mdajv", model.sasUri()); + Assertions.assertEquals("tqgtzxdpnqbqq", model.sasUri()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/FlushParametersTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/FlushParametersTests.java index 5b48bf4dfa7a7..c13beaf29a980 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/FlushParametersTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/FlushParametersTests.java @@ -12,15 +12,17 @@ public final class FlushParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - FlushParameters model = - BinaryData.fromString("{\"ids\":[\"yp\",\"rbpizc\",\"r\",\"j\"]}").toObject(FlushParameters.class); - Assertions.assertEquals("yp", model.ids().get(0)); + FlushParameters model + = BinaryData.fromString("{\"ids\":[\"hcbkhajde\",\"eamdp\",\"agalpbuxwgipwhon\",\"wkgshwa\"]}") + .toObject(FlushParameters.class); + Assertions.assertEquals("hcbkhajde", model.ids().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - FlushParameters model = new FlushParameters().withIds(Arrays.asList("yp", "rbpizc", "r", "j")); + FlushParameters model + = new FlushParameters().withIds(Arrays.asList("hcbkhajde", "eamdp", "agalpbuxwgipwhon", "wkgshwa")); model = BinaryData.fromObject(model).toObject(FlushParameters.class); - Assertions.assertEquals("yp", model.ids().get(0)); + Assertions.assertEquals("hcbkhajde", model.ids().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ForceUnlinkParametersTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ForceUnlinkParametersTests.java index eabac60a4c479..ab5a35a3604b2 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ForceUnlinkParametersTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ForceUnlinkParametersTests.java @@ -12,18 +12,17 @@ public final class ForceUnlinkParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ForceUnlinkParameters model = - BinaryData - .fromString("{\"ids\":[\"udwtiukbl\",\"ngkpocipazy\",\"o\",\"gukgjnpiucgygevq\"]}") + ForceUnlinkParameters model + = BinaryData.fromString("{\"ids\":[\"typmrbpizcdrqjsd\",\"ydnfyhxdeoejz\",\"cwif\",\"jttgzf\"]}") .toObject(ForceUnlinkParameters.class); - Assertions.assertEquals("udwtiukbl", model.ids().get(0)); + Assertions.assertEquals("typmrbpizcdrqjsd", model.ids().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ForceUnlinkParameters model = - new ForceUnlinkParameters().withIds(Arrays.asList("udwtiukbl", "ngkpocipazy", "o", "gukgjnpiucgygevq")); + ForceUnlinkParameters model + = new ForceUnlinkParameters().withIds(Arrays.asList("typmrbpizcdrqjsd", "ydnfyhxdeoejz", "cwif", "jttgzf")); model = BinaryData.fromObject(model).toObject(ForceUnlinkParameters.class); - Assertions.assertEquals("udwtiukbl", model.ids().get(0)); + Assertions.assertEquals("typmrbpizcdrqjsd", model.ids().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ImportClusterParametersTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ImportClusterParametersTests.java index 288858100d8f4..cdd25f2c9dc54 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ImportClusterParametersTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ImportClusterParametersTests.java @@ -12,15 +12,15 @@ public final class ImportClusterParametersTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ImportClusterParameters model = - BinaryData.fromString("{\"sasUris\":[\"tpngjcrcczsqpjh\"]}").toObject(ImportClusterParameters.class); - Assertions.assertEquals("tpngjcrcczsqpjh", model.sasUris().get(0)); + ImportClusterParameters model + = BinaryData.fromString("{\"sasUris\":[\"ionle\",\"x\"]}").toObject(ImportClusterParameters.class); + Assertions.assertEquals("ionle", model.sasUris().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ImportClusterParameters model = new ImportClusterParameters().withSasUris(Arrays.asList("tpngjcrcczsqpjh")); + ImportClusterParameters model = new ImportClusterParameters().withSasUris(Arrays.asList("ionle", "x")); model = BinaryData.fromObject(model).toObject(ImportClusterParameters.class); - Assertions.assertEquals("tpngjcrcczsqpjh", model.sasUris().get(0)); + Assertions.assertEquals("ionle", model.sasUris().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LinkedDatabaseTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LinkedDatabaseTests.java index 5c53a83717ef8..5a74d936f13fe 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LinkedDatabaseTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LinkedDatabaseTests.java @@ -11,15 +11,15 @@ public final class LinkedDatabaseTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - LinkedDatabase model = - BinaryData.fromString("{\"id\":\"mohctb\",\"state\":\"Unlinking\"}").toObject(LinkedDatabase.class); - Assertions.assertEquals("mohctb", model.id()); + LinkedDatabase model = BinaryData.fromString("{\"id\":\"kftutqxlngxlefg\",\"state\":\"LinkFailed\"}") + .toObject(LinkedDatabase.class); + Assertions.assertEquals("kftutqxlngxlefg", model.id()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - LinkedDatabase model = new LinkedDatabase().withId("mohctb"); + LinkedDatabase model = new LinkedDatabase().withId("kftutqxlngxlefg"); model = BinaryData.fromObject(model).toObject(LinkedDatabase.class); - Assertions.assertEquals("mohctb", model.id()); + Assertions.assertEquals("kftutqxlngxlefg", model.id()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LocationInfoTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LocationInfoTests.java deleted file mode 100644 index 8d035589a9a92..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/LocationInfoTests.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.models.Capability; -import com.azure.resourcemanager.redisenterprise.models.LocationInfo; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class LocationInfoTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - LocationInfo model = - BinaryData - .fromString( - "{\"location\":\"iyqzrnk\",\"capabilities\":[{\"name\":\"xlwhzlsicoh\",\"value\":false},{\"name\":\"wvl\",\"value\":true},{\"name\":\"w\",\"value\":false},{\"name\":\"nmmqhgyxzkon\",\"value\":false}]}") - .toObject(LocationInfo.class); - Assertions.assertEquals("iyqzrnk", model.location()); - Assertions.assertEquals("xlwhzlsicoh", model.capabilities().get(0).name()); - Assertions.assertEquals(false, model.capabilities().get(0).value()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - LocationInfo model = - new LocationInfo() - .withLocation("iyqzrnk") - .withCapabilities( - Arrays - .asList( - new Capability().withName("xlwhzlsicoh").withValue(false), - new Capability().withName("wvl").withValue(true), - new Capability().withName("w").withValue(false), - new Capability().withName("nmmqhgyxzkon").withValue(false))); - model = BinaryData.fromObject(model).toObject(LocationInfo.class); - Assertions.assertEquals("iyqzrnk", model.location()); - Assertions.assertEquals("xlwhzlsicoh", model.capabilities().get(0).name()); - Assertions.assertEquals(false, model.capabilities().get(0).value()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ManagedServiceIdentityTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ManagedServiceIdentityTests.java index 1c914772d06a8..147070528b838 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ManagedServiceIdentityTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ManagedServiceIdentityTests.java @@ -15,26 +15,22 @@ public final class ManagedServiceIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - ManagedServiceIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"dabd5595-3707-45a0-8967-0fd31b5a53c6\",\"tenantId\":\"ec59c503-6073-4511-9117-7263b52fba34\",\"type\":\"SystemAssigned," - + " UserAssigned\",\"userAssignedIdentities\":{\"xaolth\":{\"principalId\":\"90144684-e22a-4def-8298-6504ea6b234f\",\"clientId\":\"a3ae5ac2-093c-43b6-bc9a-18fbabe04950\"},\"gqjbpfzfsin\":{\"principalId\":\"793822a9-fb3b-4447-ad4b-344968f46931\",\"clientId\":\"8058c9f0-23ec-4873-8165-f8a4b89577e3\"}}}") - .toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED, model.type()); + ManagedServiceIdentity model = BinaryData.fromString( + "{\"principalId\":\"ee2fc666-c55f-431b-a6bc-61881f1b8fc3\",\"tenantId\":\"e1312760-042d-45e1-9b81-99ee0f46631b\",\"type\":\"None\",\"userAssignedIdentities\":{\"xzjnchgejs\":{\"principalId\":\"1b5e20d5-2dfe-4ce5-be5a-4bd176d998f5\",\"clientId\":\"d99c1b42-ac26-4ba1-8a5b-46038209ba5a\"},\"mailzydehojw\":{\"principalId\":\"6b0d2ae4-1a6c-4b89-badc-953fd6e5f275\",\"clientId\":\"223d7217-6517-4e41-b917-3194ba217fa6\"},\"uxinpmqnjaq\":{\"principalId\":\"e76b7bc5-cd22-4b1a-8f84-452a28712b31\",\"clientId\":\"c0816e2b-247f-4d3e-ad21-8ed0076537f0\"}}}") + .toObject(ManagedServiceIdentity.class); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - ManagedServiceIdentity model = - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) - .withUserAssignedIdentities( - mapOf("xaolth", new UserAssignedIdentity(), "gqjbpfzfsin", new UserAssignedIdentity())); + ManagedServiceIdentity model = new ManagedServiceIdentity().withType(ManagedServiceIdentityType.NONE) + .withUserAssignedIdentities(mapOf("xzjnchgejs", new UserAssignedIdentity(), "mailzydehojw", + new UserAssignedIdentity(), "uxinpmqnjaq", new UserAssignedIdentity())); model = BinaryData.fromObject(model).toObject(ManagedServiceIdentity.class); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED, model.type()); + Assertions.assertEquals(ManagedServiceIdentityType.NONE, model.type()); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ModuleTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ModuleTests.java index af73ca64f3555..450aa27b4f067 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ModuleTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/ModuleTests.java @@ -11,17 +11,17 @@ public final class ModuleTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Module model = - BinaryData.fromString("{\"name\":\"e\",\"args\":\"alhbx\",\"version\":\"e\"}").toObject(Module.class); - Assertions.assertEquals("e", model.name()); - Assertions.assertEquals("alhbx", model.args()); + Module model = BinaryData.fromString("{\"name\":\"uhrhcffcyddgl\",\"args\":\"t\",\"version\":\"qkwpyeicxmqc\"}") + .toObject(Module.class); + Assertions.assertEquals("uhrhcffcyddgl", model.name()); + Assertions.assertEquals("t", model.args()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Module model = new Module().withName("e").withArgs("alhbx"); + Module model = new Module().withName("uhrhcffcyddgl").withArgs("t"); model = BinaryData.fromObject(model).toObject(Module.class); - Assertions.assertEquals("e", model.name()); - Assertions.assertEquals("alhbx", model.args()); + Assertions.assertEquals("uhrhcffcyddgl", model.name()); + Assertions.assertEquals("t", model.args()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationDisplayTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationDisplayTests.java index 30619a60b740a..7b0e49090cbb0 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationDisplayTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationDisplayTests.java @@ -10,11 +10,9 @@ public final class OperationDisplayTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationDisplay model = - BinaryData - .fromString( - "{\"provider\":\"yrtih\",\"resource\":\"tijbpzvgnwzsymgl\",\"operation\":\"fcyzkohdbihanufh\",\"description\":\"bj\"}") - .toObject(OperationDisplay.class); + OperationDisplay model = BinaryData.fromString( + "{\"provider\":\"yrtih\",\"resource\":\"tijbpzvgnwzsymgl\",\"operation\":\"fcyzkohdbihanufh\",\"description\":\"bj\"}") + .toObject(OperationDisplay.class); } @org.junit.jupiter.api.Test diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationInnerTests.java index 4310fe891b131..f3c6bea0d56ec 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationInnerTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationInnerTests.java @@ -11,11 +11,9 @@ public final class OperationInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationInner model = - BinaryData - .fromString( - "{\"name\":\"usarhmofc\",\"isDataAction\":false,\"display\":{\"provider\":\"urkdtmlx\",\"resource\":\"kuksjtxukcdm\",\"operation\":\"rcryuanzwuxzdxta\",\"description\":\"lhmwhfpmrqobm\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") - .toObject(OperationInner.class); + OperationInner model = BinaryData.fromString( + "{\"name\":\"usarhmofc\",\"isDataAction\":false,\"display\":{\"provider\":\"urkdtmlx\",\"resource\":\"kuksjtxukcdm\",\"operation\":\"rcryuanzwuxzdxta\",\"description\":\"lhmwhfpmrqobm\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}") + .toObject(OperationInner.class); } @org.junit.jupiter.api.Test diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationListResultTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationListResultTests.java index 1b0b23c234564..3fd7015bf9ab6 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationListResultTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationListResultTests.java @@ -10,11 +10,9 @@ public final class OperationListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationListResult model = - BinaryData - .fromString( - "{\"value\":[{\"name\":\"quvgjxpybczme\",\"isDataAction\":true,\"display\":{\"provider\":\"pbsphrupidgs\",\"resource\":\"bejhphoycmsxa\",\"operation\":\"hdxbmtqio\",\"description\":\"zehtbmu\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"izhwlrxy\",\"isDataAction\":false,\"display\":{\"provider\":\"ijgkdm\",\"resource\":\"azlobcufpdznrbt\",\"operation\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"rifkwm\",\"isDataAction\":true,\"display\":{\"provider\":\"izntocipao\",\"resource\":\"jpsq\",\"operation\":\"mpoyfd\",\"description\":\"ogknygjofjdd\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"upewnwreitjzy\"}") - .toObject(OperationListResult.class); + OperationListResult model = BinaryData.fromString( + "{\"value\":[{\"name\":\"quvgjxpybczme\",\"isDataAction\":true,\"display\":{\"provider\":\"pbsphrupidgs\",\"resource\":\"bejhphoycmsxa\",\"operation\":\"hdxbmtqio\",\"description\":\"zehtbmu\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"izhwlrxy\",\"isDataAction\":false,\"display\":{\"provider\":\"ijgkdm\",\"resource\":\"azlobcufpdznrbt\",\"operation\":\"qjnqglhqgnufoooj\",\"description\":\"ifsqesaagdfmg\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"},{\"name\":\"rifkwm\",\"isDataAction\":true,\"display\":{\"provider\":\"izntocipao\",\"resource\":\"jpsq\",\"operation\":\"mpoyfd\",\"description\":\"ogknygjofjdd\"},\"origin\":\"user,system\",\"actionType\":\"Internal\"}],\"nextLink\":\"upewnwreitjzy\"}") + .toObject(OperationListResult.class); } @org.junit.jupiter.api.Test diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationStatusInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationStatusInnerTests.java index 3368f76057d52..cca0226b24a90 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationStatusInnerTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationStatusInnerTests.java @@ -11,11 +11,9 @@ public final class OperationStatusInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - OperationStatusInner model = - BinaryData - .fromString( - "{\"id\":\"a\",\"name\":\"th\",\"startTime\":\"hab\",\"endTime\":\"pikxwczbyscnpqxu\",\"status\":\"vyq\"}") - .toObject(OperationStatusInner.class); + OperationStatusInner model = BinaryData.fromString( + "{\"id\":\"a\",\"name\":\"th\",\"startTime\":\"hab\",\"endTime\":\"pikxwczbyscnpqxu\",\"status\":\"vyq\"}") + .toObject(OperationStatusInner.class); Assertions.assertEquals("a", model.id()); Assertions.assertEquals("th", model.name()); Assertions.assertEquals("hab", model.startTime()); @@ -25,13 +23,8 @@ public void testDeserialize() throws Exception { @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - OperationStatusInner model = - new OperationStatusInner() - .withId("a") - .withName("th") - .withStartTime("hab") - .withEndTime("pikxwczbyscnpqxu") - .withStatus("vyq"); + OperationStatusInner model = new OperationStatusInner().withId("a").withName("th").withStartTime("hab") + .withEndTime("pikxwczbyscnpqxu").withStatus("vyq"); model = BinaryData.fromObject(model).toObject(OperationStatusInner.class); Assertions.assertEquals("a", model.id()); Assertions.assertEquals("th", model.name()); diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListMockTests.java index ba33e9507fea4..ee95a28a849f1 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsListMockTests.java @@ -30,35 +30,25 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"name\":\"ypewrmjmwvvjekt\",\"isDataAction\":false,\"display\":{\"provider\":\"hwlrsf\",\"resource\":\"zpwv\",\"operation\":\"dqgbiqylihkaetc\",\"description\":\"vfcivfsnkymuc\"},\"origin\":\"user\",\"actionType\":\"Internal\"}]}"; + String responseStr + = "{\"value\":[{\"name\":\"ixzbinjeputtmryw\",\"isDataAction\":true,\"display\":{\"provider\":\"ftiyqzrnkcq\",\"resource\":\"xlwhzlsicoh\",\"operation\":\"qnwvlrya\",\"description\":\"hheunmmqhgyx\"},\"origin\":\"user\",\"actionType\":\"Internal\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); PagedIterable response = manager.operations().list(com.azure.core.util.Context.NONE); + } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetWithResponseMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetWithResponseMockTests.java index 06dcd7a9cb7c7..2ff122a77c722 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetWithResponseMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/OperationsStatusGetWithResponseMockTests.java @@ -30,45 +30,31 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"id\":\"hfnljkyq\",\"name\":\"vuujq\",\"startTime\":\"dokgjl\",\"endTime\":\"oxgvclt\",\"status\":\"sncghkjeszz\"}"; + String responseStr + = "{\"id\":\"ewrmjmwvvjektc\",\"name\":\"enhwlrs\",\"startTime\":\"rzpwvlqdqgbiq\",\"endTime\":\"ihkaetcktvfc\",\"status\":\"fsnkymuctq\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - OperationStatus response = - manager - .operationsStatus() - .getWithResponse("bebrjcxerfuwuttt", "fvjrbirphxepcy", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("hfnljkyq", response.id()); - Assertions.assertEquals("vuujq", response.name()); - Assertions.assertEquals("dokgjl", response.startTime()); - Assertions.assertEquals("oxgvclt", response.endTime()); - Assertions.assertEquals("sncghkjeszz", response.status()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + OperationStatus response = manager.operationsStatus() + .getWithResponse("cukoklyaxuconu", "szfkbe", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals("ewrmjmwvvjektc", response.id()); + Assertions.assertEquals("enhwlrs", response.name()); + Assertions.assertEquals("rzpwvlqdqgbiq", response.startTime()); + Assertions.assertEquals("ihkaetcktvfc", response.endTime()); + Assertions.assertEquals("fsnkymuctq", response.status()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PersistenceTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PersistenceTests.java index d55a1c70ed64e..173bc5e86af6a 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PersistenceTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PersistenceTests.java @@ -13,29 +13,23 @@ public final class PersistenceTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Persistence model = - BinaryData - .fromString( - "{\"aofEnabled\":true,\"rdbEnabled\":false,\"aofFrequency\":\"always\",\"rdbFrequency\":\"12h\"}") - .toObject(Persistence.class); - Assertions.assertEquals(true, model.aofEnabled()); + Persistence model = BinaryData + .fromString("{\"aofEnabled\":false,\"rdbEnabled\":false,\"aofFrequency\":\"1s\",\"rdbFrequency\":\"12h\"}") + .toObject(Persistence.class); + Assertions.assertEquals(false, model.aofEnabled()); Assertions.assertEquals(false, model.rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, model.aofFrequency()); + Assertions.assertEquals(AofFrequency.ONES, model.aofFrequency()); Assertions.assertEquals(RdbFrequency.ONE_TWOH, model.rdbFrequency()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Persistence model = - new Persistence() - .withAofEnabled(true) - .withRdbEnabled(false) - .withAofFrequency(AofFrequency.ALWAYS) - .withRdbFrequency(RdbFrequency.ONE_TWOH); + Persistence model = new Persistence().withAofEnabled(false).withRdbEnabled(false) + .withAofFrequency(AofFrequency.ONES).withRdbFrequency(RdbFrequency.ONE_TWOH); model = BinaryData.fromObject(model).toObject(Persistence.class); - Assertions.assertEquals(true, model.aofEnabled()); + Assertions.assertEquals(false, model.aofEnabled()); Assertions.assertEquals(false, model.rdbEnabled()); - Assertions.assertEquals(AofFrequency.ALWAYS, model.aofFrequency()); + Assertions.assertEquals(AofFrequency.ONES, model.aofFrequency()); Assertions.assertEquals(RdbFrequency.ONE_TWOH, model.rdbFrequency()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionInnerTests.java index 6cd32544b5f12..07f4790bfa22a 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionInnerTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionInnerTests.java @@ -14,33 +14,25 @@ public final class PrivateEndpointConnectionInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionInner model = - BinaryData - .fromString( - "{\"properties\":{\"privateEndpoint\":{\"id\":\"podepoo\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"vamih\",\"actionsRequired\":\"gnarxzxtheo\"},\"provisioningState\":\"Succeeded\"},\"id\":\"vyevcciqi\",\"name\":\"nhungbw\",\"type\":\"zrnf\"}") - .toObject(PrivateEndpointConnectionInner.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("vamih", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("gnarxzxtheo", model.privateLinkServiceConnectionState().actionsRequired()); + PrivateEndpointConnectionInner model = BinaryData.fromString( + "{\"properties\":{\"privateEndpoint\":{\"id\":\"wryoqpsoacc\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"kljla\",\"actionsRequired\":\"cr\"},\"provisioningState\":\"Deleting\"},\"id\":\"dfdosygexp\",\"name\":\"ojakhmsbzjhcrze\",\"type\":\"dphlxaolt\"}") + .toObject(PrivateEndpointConnectionInner.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("kljla", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("cr", model.privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionInner model = - new PrivateEndpointConnectionInner() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) - .withDescription("vamih") - .withActionsRequired("gnarxzxtheo")); + PrivateEndpointConnectionInner model = new PrivateEndpointConnectionInner() + .withPrivateEndpoint(new PrivateEndpoint()).withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("kljla").withActionsRequired("cr")); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionInner.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("vamih", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("gnarxzxtheo", model.privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("kljla", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("cr", model.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionListResultTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionListResultTests.java index 4ab89f86798fa..d4273b1f37831 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionListResultTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionListResultTests.java @@ -6,30 +6,52 @@ import com.azure.core.util.BinaryData; import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateEndpointConnectionInner; +import com.azure.resourcemanager.redisenterprise.models.PrivateEndpoint; import com.azure.resourcemanager.redisenterprise.models.PrivateEndpointConnectionListResult; +import com.azure.resourcemanager.redisenterprise.models.PrivateEndpointServiceConnectionStatus; +import com.azure.resourcemanager.redisenterprise.models.PrivateLinkServiceConnectionState; import java.util.Arrays; +import org.junit.jupiter.api.Assertions; public final class PrivateEndpointConnectionListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"provisioningState\":\"Succeeded\"},\"id\":\"canoaeupf\",\"name\":\"yhltrpmopjmcm\",\"type\":\"tuo\"},{\"properties\":{\"provisioningState\":\"Failed\"},\"id\":\"uaodsfcpk\",\"name\":\"xodpuozmyzydagfu\",\"type\":\"xbezyiuokktwh\"},{\"properties\":{\"provisioningState\":\"Deleting\"},\"id\":\"wqsmbsur\",\"name\":\"xim\",\"type\":\"ryocfsfksymdd\"},{\"properties\":{\"provisioningState\":\"Succeeded\"},\"id\":\"uxh\",\"name\":\"yudxorrqnbp\",\"type\":\"czvyifq\"}]}") - .toObject(PrivateEndpointConnectionListResult.class); + PrivateEndpointConnectionListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"privateEndpoint\":{\"id\":\"al\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"ubisnjampmng\",\"actionsRequired\":\"scxaq\"},\"provisioningState\":\"Failed\"},\"id\":\"chcbonqvpkvlrxnj\",\"name\":\"ase\",\"type\":\"pheoflokeyy\"},{\"properties\":{\"privateEndpoint\":{\"id\":\"bdlwtgrhpdjpj\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"xazjpqyegual\",\"actionsRequired\":\"xxhejjzzvd\"},\"provisioningState\":\"Deleting\"},\"id\":\"wdslfhotwmcy\",\"name\":\"pwlbjnpg\",\"type\":\"cftadeh\"},{\"properties\":{\"privateEndpoint\":{\"id\":\"yfsoppu\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"nzwdejba\",\"actionsRequired\":\"rxzdmohctbqvudwx\"},\"provisioningState\":\"Creating\"},\"id\":\"nvowgujju\",\"name\":\"wdkcglhsl\",\"type\":\"zj\"},{\"properties\":{\"privateEndpoint\":{\"id\":\"dtjixhb\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"qweykhmenev\",\"actionsRequired\":\"exfwhy\"},\"provisioningState\":\"Succeeded\"},\"id\":\"bvyvdcsity\",\"name\":\"naamde\",\"type\":\"tehfiqscjeypvh\"}]}") + .toObject(PrivateEndpointConnectionListResult.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.value().get(0).privateLinkServiceConnectionState().status()); + Assertions.assertEquals("ubisnjampmng", model.value().get(0).privateLinkServiceConnectionState().description()); + Assertions.assertEquals("scxaq", model.value().get(0).privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionListResult model = - new PrivateEndpointConnectionListResult() + PrivateEndpointConnectionListResult model + = new PrivateEndpointConnectionListResult() .withValue( Arrays .asList( - new PrivateEndpointConnectionInner(), - new PrivateEndpointConnectionInner(), - new PrivateEndpointConnectionInner(), - new PrivateEndpointConnectionInner())); + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.APPROVED) + .withDescription("ubisnjampmng").withActionsRequired("scxaq")), + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) + .withDescription("xazjpqyegual").withActionsRequired("xxhejjzzvd")), + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.PENDING) + .withDescription("nzwdejba").withActionsRequired("rxzdmohctbqvudwx")), + new PrivateEndpointConnectionInner().withPrivateEndpoint(new PrivateEndpoint()) + .withPrivateLinkServiceConnectionState(new PrivateLinkServiceConnectionState() + .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) + .withDescription("qweykhmenev").withActionsRequired("exfwhy")))); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionListResult.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.APPROVED, + model.value().get(0).privateLinkServiceConnectionState().status()); + Assertions.assertEquals("ubisnjampmng", model.value().get(0).privateLinkServiceConnectionState().description()); + Assertions.assertEquals("scxaq", model.value().get(0).privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionPropertiesTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionPropertiesTests.java index df218cb4d4456..a1e0ca6c9b219 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionPropertiesTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionPropertiesTests.java @@ -14,33 +14,25 @@ public final class PrivateEndpointConnectionPropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpointConnectionProperties model = - BinaryData - .fromString( - "{\"privateEndpoint\":{\"id\":\"gispemvtzfkufubl\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"qeof\",\"actionsRequired\":\"e\"},\"provisioningState\":\"Deleting\"}") - .toObject(PrivateEndpointConnectionProperties.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("qeof", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("e", model.privateLinkServiceConnectionState().actionsRequired()); + PrivateEndpointConnectionProperties model = BinaryData.fromString( + "{\"privateEndpoint\":{\"id\":\"rgqjbpfzfsinzg\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"rwzoxxjtfelluwf\",\"actionsRequired\":\"tonpe\"},\"provisioningState\":\"Deleting\"}") + .toObject(PrivateEndpointConnectionProperties.class); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.REJECTED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("rwzoxxjtfelluwf", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("tonpe", model.privateLinkServiceConnectionState().actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateEndpointConnectionProperties model = - new PrivateEndpointConnectionProperties() - .withPrivateEndpoint(new PrivateEndpoint()) - .withPrivateLinkServiceConnectionState( - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) - .withDescription("qeof") - .withActionsRequired("e")); + PrivateEndpointConnectionProperties model = new PrivateEndpointConnectionProperties() + .withPrivateEndpoint(new PrivateEndpoint()).withPrivateLinkServiceConnectionState( + new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.REJECTED) + .withDescription("rwzoxxjtfelluwf").withActionsRequired("tonpe")); model = BinaryData.fromObject(model).toObject(PrivateEndpointConnectionProperties.class); - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.REJECTED, model.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("qeof", model.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("e", model.privateLinkServiceConnectionState().actionsRequired()); + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.REJECTED, + model.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("rwzoxxjtfelluwf", model.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("tonpe", model.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteMockTests.java new file mode 100644 index 0000000000000..e308fdf6ff003 --- /dev/null +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteMockTests.java @@ -0,0 +1,51 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.redisenterprise.generated; + +import com.azure.core.credential.AccessToken; +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpRequest; +import com.azure.core.http.HttpResponse; +import com.azure.core.management.AzureEnvironment; +import com.azure.core.management.profile.AzureProfile; +import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +public final class PrivateEndpointConnectionsDeleteMockTests { + @Test + public void testDelete() throws Exception { + HttpClient httpClient = Mockito.mock(HttpClient.class); + HttpResponse httpResponse = Mockito.mock(HttpResponse.class); + ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); + + String responseStr = "{}"; + + Mockito.when(httpResponse.getStatusCode()).thenReturn(200); + Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); + Mockito.when(httpResponse.getBody()) + .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); + Mockito.when(httpResponse.getBodyAsByteArray()) + .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.privateEndpointConnections().delete("bhhxsrzdzuc", "rsc", "ntnev", com.azure.core.util.Context.NONE); + + } +} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteWithResponseMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteWithResponseMockTests.java deleted file mode 100644 index 5697992989324..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsDeleteWithResponseMockTests.java +++ /dev/null @@ -1,63 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class PrivateEndpointConnectionsDeleteWithResponseMockTests { - @Test - public void testDeleteWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = "{}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager - .privateEndpointConnections() - .deleteWithResponse("lfmmdnbbglzpswi", "d", "cwyhzdxssa", com.azure.core.util.Context.NONE); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java index 4558d341901dc..50f8fd3d6b363 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsGetWithResponseMockTests.java @@ -31,45 +31,30 @@ public void testGetWithResponse() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"properties\":{\"privateEndpoint\":{\"id\":\"thnzd\"},\"privateLinkServiceConnectionState\":{\"status\":\"Approved\",\"description\":\"gnayqigynduh\",\"actionsRequired\":\"hqlkthumaqo\"},\"provisioningState\":\"Creating\"},\"id\":\"cdui\",\"name\":\"r\",\"type\":\"gccymvaolpssl\"}"; + String responseStr + = "{\"properties\":{\"privateEndpoint\":{\"id\":\"psslqlfmm\"},\"privateLinkServiceConnectionState\":{\"status\":\"Rejected\",\"description\":\"glzpswiydm\",\"actionsRequired\":\"yhz\"},\"provisioningState\":\"Succeeded\"},\"id\":\"sadbz\",\"name\":\"nvdfznuda\",\"type\":\"dvxzbncblylpst\"}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PrivateEndpointConnection response = - manager - .privateEndpointConnections() - .getWithResponse("tjrybnwjewgdr", "ervnaenqpehi", "doy", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.APPROVED, response.privateLinkServiceConnectionState().status()); - Assertions.assertEquals("gnayqigynduh", response.privateLinkServiceConnectionState().description()); - Assertions.assertEquals("hqlkthumaqo", response.privateLinkServiceConnectionState().actionsRequired()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PrivateEndpointConnection response = manager.privateEndpointConnections() + .getWithResponse("ynduha", "hqlkthumaqo", "bgycduiertgccym", com.azure.core.util.Context.NONE).getValue(); + + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.REJECTED, + response.privateLinkServiceConnectionState().status()); + Assertions.assertEquals("glzpswiydm", response.privateLinkServiceConnectionState().description()); + Assertions.assertEquals("yhz", response.privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListMockTests.java index 9aff5ed5cd0ae..d6c454569e1ee 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointConnectionsListMockTests.java @@ -32,45 +32,31 @@ public void testList() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"privateEndpoint\":{\"id\":\"sub\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"irx\",\"actionsRequired\":\"ybsrfbjfdtwss\"},\"provisioningState\":\"Deleting\"},\"id\":\"pvjzbe\",\"name\":\"ilzznfqqnvwp\",\"type\":\"qtaruoujmkcjhwq\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"privateEndpoint\":{\"id\":\"uoujmkcjhwqy\"},\"privateLinkServiceConnectionState\":{\"status\":\"Pending\",\"description\":\"bnw\",\"actionsRequired\":\"wgdrjervnaenqp\"},\"provisioningState\":\"Failed\"},\"id\":\"ndoygmifthnzdnd\",\"name\":\"l\",\"type\":\"nayqi\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.privateEndpointConnections().list("shqjohxcrsbf", "vasrruvwb", com.azure.core.util.Context.NONE); - - Assertions - .assertEquals( - PrivateEndpointServiceConnectionStatus.PENDING, - response.iterator().next().privateLinkServiceConnectionState().status()); - Assertions.assertEquals("irx", response.iterator().next().privateLinkServiceConnectionState().description()); - Assertions - .assertEquals( - "ybsrfbjfdtwss", response.iterator().next().privateLinkServiceConnectionState().actionsRequired()); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + PagedIterable response + = manager.privateEndpointConnections().list("tpvjzbexilzznfqq", "vwpm", com.azure.core.util.Context.NONE); + + Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, + response.iterator().next().privateLinkServiceConnectionState().status()); + Assertions.assertEquals("bnw", response.iterator().next().privateLinkServiceConnectionState().description()); + Assertions.assertEquals("wgdrjervnaenqp", + response.iterator().next().privateLinkServiceConnectionState().actionsRequired()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointTests.java index e3c58a78aa859..d06e632f85bf2 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateEndpointTests.java @@ -10,7 +10,7 @@ public final class PrivateEndpointTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateEndpoint model = BinaryData.fromString("{\"id\":\"jbasvmsmjqulngs\"}").toObject(PrivateEndpoint.class); + PrivateEndpoint model = BinaryData.fromString("{\"id\":\"kjlxofpdvhpf\"}").toObject(PrivateEndpoint.class); } @org.junit.jupiter.api.Test diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceInnerTests.java index d785a22925e72..11ea59eba9cdb 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceInnerTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceInnerTests.java @@ -12,18 +12,17 @@ public final class PrivateLinkResourceInnerTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkResourceInner model = - BinaryData - .fromString( - "{\"properties\":{\"groupId\":\"uo\",\"requiredMembers\":[\"kg\",\"sauuimj\",\"vxieduugidyj\"],\"requiredZoneNames\":[\"byao\"]},\"id\":\"e\",\"name\":\"csonpclhoco\",\"type\":\"slkevle\"}") - .toObject(PrivateLinkResourceInner.class); - Assertions.assertEquals("byao", model.requiredZoneNames().get(0)); + PrivateLinkResourceInner model = BinaryData.fromString( + "{\"properties\":{\"groupId\":\"byao\",\"requiredMembers\":[\"xc\"],\"requiredZoneNames\":[\"pclhocohslk\"]},\"id\":\"vleggzfbuhfmvfax\",\"name\":\"ffeii\",\"type\":\"hl\"}") + .toObject(PrivateLinkResourceInner.class); + Assertions.assertEquals("pclhocohslk", model.requiredZoneNames().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkResourceInner model = new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("byao")); + PrivateLinkResourceInner model + = new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("pclhocohslk")); model = BinaryData.fromObject(model).toObject(PrivateLinkResourceInner.class); - Assertions.assertEquals("byao", model.requiredZoneNames().get(0)); + Assertions.assertEquals("pclhocohslk", model.requiredZoneNames().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceListResultTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceListResultTests.java index 1fb137970057b..b0a5e2a138c3d 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceListResultTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourceListResultTests.java @@ -8,26 +8,25 @@ import com.azure.resourcemanager.redisenterprise.fluent.models.PrivateLinkResourceInner; import com.azure.resourcemanager.redisenterprise.models.PrivateLinkResourceListResult; import java.util.Arrays; +import org.junit.jupiter.api.Assertions; public final class PrivateLinkResourceListResultTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkResourceListResult model = - BinaryData - .fromString( - "{\"value\":[{\"properties\":{\"groupId\":\"jsllrmv\",\"requiredMembers\":[],\"requiredZoneNames\":[]},\"id\":\"atkpnp\",\"name\":\"lexxbczwtru\",\"type\":\"iqzbq\"},{\"properties\":{\"groupId\":\"ovm\",\"requiredMembers\":[],\"requiredZoneNames\":[]},\"id\":\"cspkwlhzdobpxjmf\",\"name\":\"bvvnchrkcciw\",\"type\":\"zjuqkhrsaj\"}]}") - .toObject(PrivateLinkResourceListResult.class); + PrivateLinkResourceListResult model = BinaryData.fromString( + "{\"value\":[{\"properties\":{\"groupId\":\"q\",\"requiredMembers\":[\"refovgmkqsleyyvx\",\"qjpkcattpngjcrc\",\"zsqpjhvmdajvny\",\"ounqecano\"],\"requiredZoneNames\":[\"pfhyhl\"]},\"id\":\"rpmopjmc\",\"name\":\"atuokthfuiu\",\"type\":\"odsfcpkvxodpuozm\"},{\"properties\":{\"groupId\":\"dagfuaxbezyiuok\",\"requiredMembers\":[\"hrdxwzywqsmbs\",\"reximoryocfs\",\"ksymd\",\"ys\"],\"requiredZoneNames\":[\"iuxhqyudxorr\"]},\"id\":\"nbpoczvyifqrvkdv\",\"name\":\"sllr\",\"type\":\"vvdfwatkpnpul\"},{\"properties\":{\"groupId\":\"bczw\",\"requiredMembers\":[\"wiqzbqjvsovmyo\",\"acspkwl\"],\"requiredZoneNames\":[\"obpxjmflbvvn\",\"hrk\",\"ciwwzjuqkhr\",\"ajiwkuo\"]},\"id\":\"oskg\",\"name\":\"sauuimj\",\"type\":\"vxieduugidyj\"}]}") + .toObject(PrivateLinkResourceListResult.class); + Assertions.assertEquals("pfhyhl", model.value().get(0).requiredZoneNames().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkResourceListResult model = - new PrivateLinkResourceListResult() - .withValue( - Arrays - .asList( - new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList()), - new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList()))); + PrivateLinkResourceListResult model = new PrivateLinkResourceListResult() + .withValue(Arrays.asList(new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("pfhyhl")), + new PrivateLinkResourceInner().withRequiredZoneNames(Arrays.asList("iuxhqyudxorr")), + new PrivateLinkResourceInner() + .withRequiredZoneNames(Arrays.asList("obpxjmflbvvn", "hrk", "ciwwzjuqkhr", "ajiwkuo")))); model = BinaryData.fromObject(model).toObject(PrivateLinkResourceListResult.class); + Assertions.assertEquals("pfhyhl", model.value().get(0).requiredZoneNames().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcePropertiesTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcePropertiesTests.java index 9d8eaf83c242e..af748ba8dfe24 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcePropertiesTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcePropertiesTests.java @@ -12,20 +12,17 @@ public final class PrivateLinkResourcePropertiesTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkResourceProperties model = - BinaryData - .fromString( - "{\"groupId\":\"zfbuhf\",\"requiredMembers\":[\"axkffei\",\"th\",\"vmezy\"],\"requiredZoneNames\":[\"xmzsbbzogg\",\"grxwbu\",\"vjxxjnsp\",\"dptkoenkouk\"]}") - .toObject(PrivateLinkResourceProperties.class); - Assertions.assertEquals("xmzsbbzogg", model.requiredZoneNames().get(0)); + PrivateLinkResourceProperties model = BinaryData.fromString( + "{\"groupId\":\"ez\",\"requiredMembers\":[\"hxmzsbbzoggig\",\"xwburvjxxjns\",\"ydptkoen\",\"ou\"],\"requiredZoneNames\":[\"udwtiukbl\",\"ngkpocipazy\",\"o\",\"gukgjnpiucgygevq\"]}") + .toObject(PrivateLinkResourceProperties.class); + Assertions.assertEquals("udwtiukbl", model.requiredZoneNames().get(0)); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkResourceProperties model = - new PrivateLinkResourceProperties() - .withRequiredZoneNames(Arrays.asList("xmzsbbzogg", "grxwbu", "vjxxjnsp", "dptkoenkouk")); + PrivateLinkResourceProperties model = new PrivateLinkResourceProperties() + .withRequiredZoneNames(Arrays.asList("udwtiukbl", "ngkpocipazy", "o", "gukgjnpiucgygevq")); model = BinaryData.fromObject(model).toObject(PrivateLinkResourceProperties.class); - Assertions.assertEquals("xmzsbbzogg", model.requiredZoneNames().get(0)); + Assertions.assertEquals("udwtiukbl", model.requiredZoneNames().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterMockTests.java index bd162feb4a6d2..7f8aa7f434b6c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkResourcesListByClusterMockTests.java @@ -31,38 +31,27 @@ public void testListByCluster() throws Exception { HttpResponse httpResponse = Mockito.mock(HttpResponse.class); ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - String responseStr = - "{\"value\":[{\"properties\":{\"groupId\":\"bncblylpstdbhhx\",\"requiredMembers\":[\"dzu\",\"erscdntne\",\"fiwjmygtdssls\",\"tmweriofzpyq\"],\"requiredZoneNames\":[\"wab\",\"ets\",\"hszhedplvwiwu\",\"mwmbes\"]},\"id\":\"nkww\",\"name\":\"pp\",\"type\":\"flcxoga\"}]}"; + String responseStr + = "{\"value\":[{\"properties\":{\"groupId\":\"wab\",\"requiredMembers\":[\"shhszhedplvwiw\",\"bmwmbesldnkw\",\"tppjflcx\"],\"requiredZoneNames\":[\"okonzmnsikvmkqz\",\"qqkdltfzxmhhvhgu\",\"eodkwobda\"]},\"id\":\"xtibqdxbxwakbog\",\"name\":\"xndlkzgxhu\",\"type\":\"iplbpodxunkbebxm\"}]}"; Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); - PagedIterable response = - manager.privateLinkResources().listByCluster("bzmnvdfznud", "od", com.azure.core.util.Context.NONE); + PagedIterable response = manager.privateLinkResources().listByCluster("iwjmygtdssls", + "tmweriofzpyq", com.azure.core.util.Context.NONE); - Assertions.assertEquals("wab", response.iterator().next().requiredZoneNames().get(0)); + Assertions.assertEquals("okonzmnsikvmkqz", response.iterator().next().requiredZoneNames().get(0)); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkServiceConnectionStateTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkServiceConnectionStateTests.java index 87ed5d97b7b19..5b3095a4461ac 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkServiceConnectionStateTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/PrivateLinkServiceConnectionStateTests.java @@ -12,26 +12,22 @@ public final class PrivateLinkServiceConnectionStateTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - PrivateLinkServiceConnectionState model = - BinaryData - .fromString( - "{\"status\":\"Pending\",\"description\":\"ybkzgcwr\",\"actionsRequired\":\"lxxwrljdouskc\"}") + PrivateLinkServiceConnectionState model + = BinaryData.fromString("{\"status\":\"Pending\",\"description\":\"ininmay\",\"actionsRequired\":\"ybb\"}") .toObject(PrivateLinkServiceConnectionState.class); Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, model.status()); - Assertions.assertEquals("ybkzgcwr", model.description()); - Assertions.assertEquals("lxxwrljdouskc", model.actionsRequired()); + Assertions.assertEquals("ininmay", model.description()); + Assertions.assertEquals("ybb", model.actionsRequired()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - PrivateLinkServiceConnectionState model = - new PrivateLinkServiceConnectionState() - .withStatus(PrivateEndpointServiceConnectionStatus.PENDING) - .withDescription("ybkzgcwr") - .withActionsRequired("lxxwrljdouskc"); + PrivateLinkServiceConnectionState model + = new PrivateLinkServiceConnectionState().withStatus(PrivateEndpointServiceConnectionStatus.PENDING) + .withDescription("ininmay").withActionsRequired("ybb"); model = BinaryData.fromObject(model).toObject(PrivateLinkServiceConnectionState.class); Assertions.assertEquals(PrivateEndpointServiceConnectionStatus.PENDING, model.status()); - Assertions.assertEquals("ybkzgcwr", model.description()); - Assertions.assertEquals("lxxwrljdouskc", model.actionsRequired()); + Assertions.assertEquals("ininmay", model.description()); + Assertions.assertEquals("ybb", model.actionsRequired()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesCreateMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesCreateMockTests.java deleted file mode 100644 index b86dcba68f90f..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesCreateMockTests.java +++ /dev/null @@ -1,107 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; -import com.azure.resourcemanager.redisenterprise.models.Cluster; -import com.azure.resourcemanager.redisenterprise.models.ClusterPropertiesEncryption; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentity; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.Sku; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class RedisEnterprisesCreateMockTests { - @Test - public void testCreate() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"sku\":{\"name\":\"EnterpriseFlash_F1500\",\"capacity\":946722582},\"zones\":[\"ehhseyvjusrts\",\"hspkdeemao\",\"mx\",\"gkvtmelmqkrhah\"],\"identity\":{\"principalId\":\"50dd1237-7f30-4f28-bca3-7c9de2f1af8d\",\"tenantId\":\"2f6d603e-e2e2-4f45-84e6-bbe28e373dad\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.2\",\"encryption\":{},\"hostName\":\"mdua\",\"provisioningState\":\"Succeeded\",\"resourceState\":\"Updating\",\"redisVersion\":\"vfadmws\",\"privateEndpointConnections\":[]},\"location\":\"vxpvgomz\",\"tags\":{\"ali\":\"isgwbnbbeldawkz\",\"hashsfwxosow\":\"urqhaka\"},\"id\":\"xcug\",\"name\":\"cjooxdjebwpucwwf\",\"type\":\"ovbvmeueciv\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Cluster response = - manager - .redisEnterprises() - .define("kphywpnvjto") - .withRegion("kqujidsuyono") - .withExistingResourceGroup("nxdhbt") - .withSku(new Sku().withName(SkuName.ENTERPRISE_E20).withCapacity(963875156)) - .withTags(mapOf("dxyt", "aocqxtccmgy", "txhdzh", "moyrxvwfudwpz", "rxsbkyvp", "rqjbhckfrl")) - .withZones(Arrays.asList("lfplp")) - .withIdentity( - new ManagedServiceIdentity() - .withType(ManagedServiceIdentityType.SYSTEM_ASSIGNED_USER_ASSIGNED) - .withUserAssignedIdentities(mapOf())) - .withMinimumTlsVersion(TlsVersion.ONE_ONE) - .withEncryption(new ClusterPropertiesEncryption()) - .create(); - - Assertions.assertEquals("vxpvgomz", response.location()); - Assertions.assertEquals("isgwbnbbeldawkz", response.tags().get("ali")); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F1500, response.sku().name()); - Assertions.assertEquals(946722582, response.sku().capacity()); - Assertions.assertEquals("ehhseyvjusrts", response.zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); - Assertions.assertEquals(TlsVersion.ONE_TWO, response.minimumTlsVersion()); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesDeleteMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesDeleteMockTests.java index 6778b7b1679b0..371f37bd5403c 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesDeleteMockTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesDeleteMockTests.java @@ -32,30 +32,20 @@ public void testDelete() throws Exception { Mockito.when(httpResponse.getStatusCode()).thenReturn(200); Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) + Mockito.when(httpResponse.getBody()) .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) + Mockito.when(httpResponse.getBodyAsByteArray()) .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - manager.redisEnterprises().delete("ijhtxf", "gx", com.azure.core.util.Context.NONE); + Mockito.when(httpClient.send(httpRequest.capture(), Mockito.any())).thenReturn(Mono.defer(() -> { + Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); + return Mono.just(httpResponse); + })); + + RedisEnterpriseManager manager = RedisEnterpriseManager.configure().withHttpClient(httpClient).authenticate( + tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), + new AzureProfile("", "", AzureEnvironment.AZURE)); + + manager.redisEnterprises().delete("fbebrjcxer", "uwutttxfvjrbi", com.azure.core.util.Context.NONE); + } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesGetByResourceGroupWithResponseMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesGetByResourceGroupWithResponseMockTests.java deleted file mode 100644 index e8c177eca1dad..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesGetByResourceGroupWithResponseMockTests.java +++ /dev/null @@ -1,79 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; -import com.azure.resourcemanager.redisenterprise.models.Cluster; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class RedisEnterprisesGetByResourceGroupWithResponseMockTests { - @Test - public void testGetByResourceGroupWithResponse() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"sku\":{\"name\":\"Enterprise_E10\",\"capacity\":68854326},\"zones\":[\"qkkrb\",\"pukgriwflzlfb\"],\"identity\":{\"principalId\":\"55e58f56-9e33-48ba-885c-97ad97c5f51b\",\"tenantId\":\"36313e29-f41c-4156-b818-bc7c5007c8de\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.2\",\"encryption\":{},\"hostName\":\"ahmgkbrp\",\"provisioningState\":\"Canceled\",\"resourceState\":\"Enabling\",\"redisVersion\":\"nuqqkpikadrgvt\",\"privateEndpointConnections\":[]},\"location\":\"buynhijggm\",\"tags\":{\"zzmhjrunmpxttd\":\"siarbutrcvpn\",\"nbtkcxywnytnr\":\"hrbnlankxmyskpbh\",\"qidybyx\":\"yn\"},\"id\":\"zfcl\",\"name\":\"aaxdbabphlwrq\",\"type\":\"fkts\"}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - Cluster response = - manager - .redisEnterprises() - .getByResourceGroupWithResponse("fsm", "nehmpvecx", com.azure.core.util.Context.NONE) - .getValue(); - - Assertions.assertEquals("buynhijggm", response.location()); - Assertions.assertEquals("siarbutrcvpn", response.tags().get("zzmhjrunmpxttd")); - Assertions.assertEquals(SkuName.ENTERPRISE_E10, response.sku().name()); - Assertions.assertEquals(68854326, response.sku().capacity()); - Assertions.assertEquals("qkkrb", response.zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, response.identity().type()); - Assertions.assertEquals(TlsVersion.ONE_TWO, response.minimumTlsVersion()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListByResourceGroupMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListByResourceGroupMockTests.java deleted file mode 100644 index db5fe315e648c..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListByResourceGroupMockTests.java +++ /dev/null @@ -1,78 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; -import com.azure.resourcemanager.redisenterprise.models.Cluster; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class RedisEnterprisesListByResourceGroupMockTests { - @Test - public void testListByResourceGroup() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"sku\":{\"name\":\"EnterpriseFlash_F700\",\"capacity\":1572561560},\"zones\":[\"t\"],\"identity\":{\"principalId\":\"a8db838c-5ddd-40c4-beff-8564c1e49c3f\",\"tenantId\":\"5b6879d8-e7f2-4945-b9c6-8fc033e50d5c\",\"type\":\"SystemAssigned\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.1\",\"encryption\":{},\"hostName\":\"zywbiex\",\"provisioningState\":\"Failed\",\"resourceState\":\"UpdateFailed\",\"redisVersion\":\"axibxujw\",\"privateEndpointConnections\":[]},\"location\":\"almuzyoxaepdkzja\",\"tags\":{\"zt\":\"xrhdwbavxbniwdjs\"},\"id\":\"dbpgnxytxhp\",\"name\":\"xbzpfzab\",\"type\":\"lcuhxwtctyqiklb\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = - manager.redisEnterprises().listByResourceGroup("hsucoc", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("almuzyoxaepdkzja", response.iterator().next().location()); - Assertions.assertEquals("xrhdwbavxbniwdjs", response.iterator().next().tags().get("zt")); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F700, response.iterator().next().sku().name()); - Assertions.assertEquals(1572561560, response.iterator().next().sku().capacity()); - Assertions.assertEquals("t", response.iterator().next().zones().get(0)); - Assertions - .assertEquals(ManagedServiceIdentityType.SYSTEM_ASSIGNED, response.iterator().next().identity().type()); - Assertions.assertEquals(TlsVersion.ONE_ONE, response.iterator().next().minimumTlsVersion()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListMockTests.java deleted file mode 100644 index 4f0cb9d9b1bb4..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RedisEnterprisesListMockTests.java +++ /dev/null @@ -1,76 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; -import com.azure.resourcemanager.redisenterprise.models.Cluster; -import com.azure.resourcemanager.redisenterprise.models.ManagedServiceIdentityType; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import com.azure.resourcemanager.redisenterprise.models.TlsVersion; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class RedisEnterprisesListMockTests { - @Test - public void testList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"sku\":{\"name\":\"Enterprise_E100\",\"capacity\":946920071},\"zones\":[\"zbhvgyuguosv\",\"kfssxqukkf\",\"l\"],\"identity\":{\"principalId\":\"d05e3a18-1e79-4f8c-a32e-611e8bf0eff7\",\"tenantId\":\"1a47553c-b560-458e-ae9d-b13b7525c4cd\",\"type\":\"UserAssigned\",\"userAssignedIdentities\":{}},\"properties\":{\"minimumTlsVersion\":\"1.2\",\"encryption\":{},\"hostName\":\"lpvlopw\",\"provisioningState\":\"Canceled\",\"resourceState\":\"CreateFailed\",\"redisVersion\":\"pkdwzbai\",\"privateEndpointConnections\":[]},\"location\":\"a\",\"tags\":{\"mtxpsiebtfh\":\"yqupedeojnabckh\",\"htldwk\":\"pesapskrdqmhjj\",\"otogtwrupqs\":\"zxuutkncwscwsvl\",\"cykvceo\":\"vnm\"},\"id\":\"eil\",\"name\":\"vnotyfjfcnj\",\"type\":\"k\"}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = manager.redisEnterprises().list(com.azure.core.util.Context.NONE); - - Assertions.assertEquals("a", response.iterator().next().location()); - Assertions.assertEquals("yqupedeojnabckh", response.iterator().next().tags().get("mtxpsiebtfh")); - Assertions.assertEquals(SkuName.ENTERPRISE_E100, response.iterator().next().sku().name()); - Assertions.assertEquals(946920071, response.iterator().next().sku().capacity()); - Assertions.assertEquals("zbhvgyuguosv", response.iterator().next().zones().get(0)); - Assertions.assertEquals(ManagedServiceIdentityType.USER_ASSIGNED, response.iterator().next().identity().type()); - Assertions.assertEquals(TlsVersion.ONE_TWO, response.iterator().next().minimumTlsVersion()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailInnerTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailInnerTests.java deleted file mode 100644 index 5c0b74e0470f8..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailInnerTests.java +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; -import com.azure.resourcemanager.redisenterprise.models.Capability; -import com.azure.resourcemanager.redisenterprise.models.LocationInfo; -import com.azure.resourcemanager.redisenterprise.models.SkuDetail; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class RegionSkuDetailInnerTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - RegionSkuDetailInner model = - BinaryData - .fromString( - "{\"resourceType\":\"jdeyeamdpha\",\"locationInfo\":{\"location\":\"pbuxwgipwhon\",\"capabilities\":[{\"name\":\"shwankixzbinje\",\"value\":true},{\"name\":\"mryw\",\"value\":true}]},\"skuDetails\":{\"name\":\"Enterprise_E10\"}}") - .toObject(RegionSkuDetailInner.class); - Assertions.assertEquals("jdeyeamdpha", model.resourceType()); - Assertions.assertEquals("pbuxwgipwhon", model.locationInfo().location()); - Assertions.assertEquals("shwankixzbinje", model.locationInfo().capabilities().get(0).name()); - Assertions.assertEquals(true, model.locationInfo().capabilities().get(0).value()); - Assertions.assertEquals(SkuName.ENTERPRISE_E10, model.skuDetails().name()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - RegionSkuDetailInner model = - new RegionSkuDetailInner() - .withResourceType("jdeyeamdpha") - .withLocationInfo( - new LocationInfo() - .withLocation("pbuxwgipwhon") - .withCapabilities( - Arrays - .asList( - new Capability().withName("shwankixzbinje").withValue(true), - new Capability().withName("mryw").withValue(true)))) - .withSkuDetails(new SkuDetail().withName(SkuName.ENTERPRISE_E10)); - model = BinaryData.fromObject(model).toObject(RegionSkuDetailInner.class); - Assertions.assertEquals("jdeyeamdpha", model.resourceType()); - Assertions.assertEquals("pbuxwgipwhon", model.locationInfo().location()); - Assertions.assertEquals("shwankixzbinje", model.locationInfo().capabilities().get(0).name()); - Assertions.assertEquals(true, model.locationInfo().capabilities().get(0).value()); - Assertions.assertEquals(SkuName.ENTERPRISE_E10, model.skuDetails().name()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailsTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailsTests.java deleted file mode 100644 index b9d124b482176..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/RegionSkuDetailsTests.java +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.fluent.models.RegionSkuDetailInner; -import com.azure.resourcemanager.redisenterprise.models.LocationInfo; -import com.azure.resourcemanager.redisenterprise.models.RegionSkuDetails; -import com.azure.resourcemanager.redisenterprise.models.SkuDetail; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import java.util.Arrays; -import org.junit.jupiter.api.Assertions; - -public final class RegionSkuDetailsTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - RegionSkuDetails model = - BinaryData - .fromString( - "{\"value\":[{\"resourceType\":\"dnfyhxdeoejzicwi\",\"locationInfo\":{\"location\":\"ttgzfbis\",\"capabilities\":[]},\"skuDetails\":{\"name\":\"Enterprise_E20\"}}]}") - .toObject(RegionSkuDetails.class); - Assertions.assertEquals("dnfyhxdeoejzicwi", model.value().get(0).resourceType()); - Assertions.assertEquals("ttgzfbis", model.value().get(0).locationInfo().location()); - Assertions.assertEquals(SkuName.ENTERPRISE_E20, model.value().get(0).skuDetails().name()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - RegionSkuDetails model = - new RegionSkuDetails() - .withValue( - Arrays - .asList( - new RegionSkuDetailInner() - .withResourceType("dnfyhxdeoejzicwi") - .withLocationInfo( - new LocationInfo().withLocation("ttgzfbis").withCapabilities(Arrays.asList())) - .withSkuDetails(new SkuDetail().withName(SkuName.ENTERPRISE_E20)))); - model = BinaryData.fromObject(model).toObject(RegionSkuDetails.class); - Assertions.assertEquals("dnfyhxdeoejzicwi", model.value().get(0).resourceType()); - Assertions.assertEquals("ttgzfbis", model.value().get(0).locationInfo().location()); - Assertions.assertEquals(SkuName.ENTERPRISE_E20, model.value().get(0).skuDetails().name()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuDetailTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuDetailTests.java deleted file mode 100644 index b34533b1da795..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuDetailTests.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.util.BinaryData; -import com.azure.resourcemanager.redisenterprise.models.SkuDetail; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import org.junit.jupiter.api.Assertions; - -public final class SkuDetailTests { - @org.junit.jupiter.api.Test - public void testDeserialize() throws Exception { - SkuDetail model = BinaryData.fromString("{\"name\":\"Enterprise_E100\"}").toObject(SkuDetail.class); - Assertions.assertEquals(SkuName.ENTERPRISE_E100, model.name()); - } - - @org.junit.jupiter.api.Test - public void testSerialize() throws Exception { - SkuDetail model = new SkuDetail().withName(SkuName.ENTERPRISE_E100); - model = BinaryData.fromObject(model).toObject(SkuDetail.class); - Assertions.assertEquals(SkuName.ENTERPRISE_E100, model.name()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuTests.java index 445d576836c0e..ae7371360da05 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkuTests.java @@ -12,16 +12,16 @@ public final class SkuTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - Sku model = BinaryData.fromString("{\"name\":\"Enterprise_E20\",\"capacity\":1041872032}").toObject(Sku.class); - Assertions.assertEquals(SkuName.ENTERPRISE_E20, model.name()); - Assertions.assertEquals(1041872032, model.capacity()); + Sku model = BinaryData.fromString("{\"name\":\"Enterprise_E100\",\"capacity\":186394728}").toObject(Sku.class); + Assertions.assertEquals(SkuName.ENTERPRISE_E100, model.name()); + Assertions.assertEquals(186394728, model.capacity()); } @org.junit.jupiter.api.Test public void testSerialize() throws Exception { - Sku model = new Sku().withName(SkuName.ENTERPRISE_E20).withCapacity(1041872032); + Sku model = new Sku().withName(SkuName.ENTERPRISE_E100).withCapacity(186394728); model = BinaryData.fromObject(model).toObject(Sku.class); - Assertions.assertEquals(SkuName.ENTERPRISE_E20, model.name()); - Assertions.assertEquals(1041872032, model.capacity()); + Assertions.assertEquals(SkuName.ENTERPRISE_E100, model.name()); + Assertions.assertEquals(186394728, model.capacity()); } } diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkusListMockTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkusListMockTests.java deleted file mode 100644 index 12ba74366352e..0000000000000 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/SkusListMockTests.java +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.redisenterprise.generated; - -import com.azure.core.credential.AccessToken; -import com.azure.core.http.HttpClient; -import com.azure.core.http.HttpHeaders; -import com.azure.core.http.HttpRequest; -import com.azure.core.http.HttpResponse; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.management.AzureEnvironment; -import com.azure.core.management.profile.AzureProfile; -import com.azure.resourcemanager.redisenterprise.RedisEnterpriseManager; -import com.azure.resourcemanager.redisenterprise.models.RegionSkuDetail; -import com.azure.resourcemanager.redisenterprise.models.SkuName; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.OffsetDateTime; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.mockito.ArgumentCaptor; -import org.mockito.Mockito; -import reactor.core.publisher.Flux; -import reactor.core.publisher.Mono; - -public final class SkusListMockTests { - @Test - public void testList() throws Exception { - HttpClient httpClient = Mockito.mock(HttpClient.class); - HttpResponse httpResponse = Mockito.mock(HttpResponse.class); - ArgumentCaptor httpRequest = ArgumentCaptor.forClass(HttpRequest.class); - - String responseStr = - "{\"value\":[{\"resourceType\":\"kqze\",\"locationInfo\":{\"location\":\"dltfz\",\"capabilities\":[]},\"skuDetails\":{\"name\":\"EnterpriseFlash_F1500\"}}]}"; - - Mockito.when(httpResponse.getStatusCode()).thenReturn(200); - Mockito.when(httpResponse.getHeaders()).thenReturn(new HttpHeaders()); - Mockito - .when(httpResponse.getBody()) - .thenReturn(Flux.just(ByteBuffer.wrap(responseStr.getBytes(StandardCharsets.UTF_8)))); - Mockito - .when(httpResponse.getBodyAsByteArray()) - .thenReturn(Mono.just(responseStr.getBytes(StandardCharsets.UTF_8))); - Mockito - .when(httpClient.send(httpRequest.capture(), Mockito.any())) - .thenReturn( - Mono - .defer( - () -> { - Mockito.when(httpResponse.getRequest()).thenReturn(httpRequest.getValue()); - return Mono.just(httpResponse); - })); - - RedisEnterpriseManager manager = - RedisEnterpriseManager - .configure() - .withHttpClient(httpClient) - .authenticate( - tokenRequestContext -> Mono.just(new AccessToken("this_is_a_token", OffsetDateTime.MAX)), - new AzureProfile("", "", AzureEnvironment.AZURE)); - - PagedIterable response = manager.skus().list("konzmnsik", com.azure.core.util.Context.NONE); - - Assertions.assertEquals("kqze", response.iterator().next().resourceType()); - Assertions.assertEquals("dltfz", response.iterator().next().locationInfo().location()); - Assertions.assertEquals(SkuName.ENTERPRISE_FLASH_F1500, response.iterator().next().skuDetails().name()); - } -} diff --git a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/UserAssignedIdentityTests.java b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/UserAssignedIdentityTests.java index 083300105568e..86c2da2c40e07 100644 --- a/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/UserAssignedIdentityTests.java +++ b/sdk/redisenterprise/azure-resourcemanager-redisenterprise/src/test/java/com/azure/resourcemanager/redisenterprise/generated/UserAssignedIdentityTests.java @@ -10,11 +10,9 @@ public final class UserAssignedIdentityTests { @org.junit.jupiter.api.Test public void testDeserialize() throws Exception { - UserAssignedIdentity model = - BinaryData - .fromString( - "{\"principalId\":\"5c1631dd-070c-4338-9225-6fff8f2c7868\",\"clientId\":\"a805f8c8-5f6c-431e-b4b3-0a93d1604117\"}") - .toObject(UserAssignedIdentity.class); + UserAssignedIdentity model = BinaryData.fromString( + "{\"principalId\":\"cf493044-d698-4524-826d-c8839a172bc7\",\"clientId\":\"2384cfe8-8300-4e4d-ae78-ef8c08e59994\"}") + .toObject(UserAssignedIdentity.class); } @org.junit.jupiter.api.Test diff --git a/sdk/redisenterprise/tests.mgmt.yml b/sdk/redisenterprise/tests.mgmt.yml index 55d3ca6deaf28..18688e8fa69ea 100644 --- a/sdk/redisenterprise/tests.mgmt.yml +++ b/sdk/redisenterprise/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: redisenterprise - Artifacts: - - name: azure-resourcemanager-redisenterprise - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerredisenterprise - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: redisenterprise + Artifacts: + - name: azure-resourcemanager-redisenterprise + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerredisenterprise + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/CHANGELOG.md b/sdk/remoterendering/azure-mixedreality-remoterendering/CHANGELOG.md index 202b6bbade2b0..8d4ca45c70ada 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/CHANGELOG.md +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/CHANGELOG.md @@ -10,6 +10,17 @@ ### Other Changes + +## 1.1.27 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-mixedreality-authentication` from `1.2.20` to version `1.2.21`. + + ## 1.1.26 (2024-02-20) ### Other Changes diff --git a/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml b/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml index 94643ef02be3b..730dcb99022fd 100644 --- a/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml +++ b/sdk/remoterendering/azure-mixedreality-remoterendering/pom.xml @@ -36,7 +36,7 @@ com.azure azure-mixedreality-authentication - 1.2.21 + 1.2.22 @@ -79,13 +79,13 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/remoterendering/tests.yml b/sdk/remoterendering/tests.yml index 0d5d046b26324..c670833e7a30d 100644 --- a/sdk/remoterendering/tests.yml +++ b/sdk/remoterendering/tests.yml @@ -1,11 +1,11 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: remoterendering - Location: eastus2 - Artifacts: - - name: azure-mixedreality-remoterendering - groupId: com.azure - safeName: azuremixedrealityremoterendering \ No newline at end of file +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: remoterendering + Location: eastus2 + Artifacts: + - name: azure-mixedreality-remoterendering + groupId: com.azure + safeName: azuremixedrealityremoterendering diff --git a/sdk/reservations/azure-resourcemanager-reservations/pom.xml b/sdk/reservations/azure-resourcemanager-reservations/pom.xml index 25174b8ed25a4..4eed6151efa8e 100644 --- a/sdk/reservations/azure-resourcemanager-reservations/pom.xml +++ b/sdk/reservations/azure-resourcemanager-reservations/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/resourceconnector/azure-resourcemanager-resourceconnector/pom.xml b/sdk/resourceconnector/azure-resourcemanager-resourceconnector/pom.xml index f52d8766587ab..346083a561ee4 100644 --- a/sdk/resourceconnector/azure-resourcemanager-resourceconnector/pom.xml +++ b/sdk/resourceconnector/azure-resourcemanager-resourceconnector/pom.xml @@ -66,7 +66,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -92,13 +92,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/resourcegraph/azure-resourcemanager-resourcegraph/pom.xml b/sdk/resourcegraph/azure-resourcemanager-resourcegraph/pom.xml index d6fbd4d042004..9d664bd5474e5 100644 --- a/sdk/resourcegraph/azure-resourcemanager-resourcegraph/pom.xml +++ b/sdk/resourcegraph/azure-resourcemanager-resourcegraph/pom.xml @@ -60,7 +60,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -80,13 +80,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/resourcehealth/azure-resourcemanager-resourcehealth/pom.xml b/sdk/resourcehealth/azure-resourcemanager-resourcehealth/pom.xml index f2f83720f8f67..45423f2511f05 100644 --- a/sdk/resourcehealth/azure-resourcemanager-resourcehealth/pom.xml +++ b/sdk/resourcehealth/azure-resourcemanager-resourcehealth/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-compute - 2.36.0 + 2.37.0 test diff --git a/sdk/resourcehealth/tests.mgmt.yml b/sdk/resourcehealth/tests.mgmt.yml index d413d6376c8fc..2e8bb50673345 100644 --- a/sdk/resourcehealth/tests.mgmt.yml +++ b/sdk/resourcehealth/tests.mgmt.yml @@ -2,12 +2,12 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: resourcehealth - Artifacts: - - name: azure-resourcemanager-resourcehealth - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerresourcehealth - Clouds: 'Public' +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: resourcehealth + Artifacts: + - name: azure-resourcemanager-resourcehealth + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerresourcehealth + Clouds: 'Public' diff --git a/sdk/resourcemanager/README.md b/sdk/resourcemanager/README.md index 06c5578664da1..dfba3099c1d8a 100644 --- a/sdk/resourcemanager/README.md +++ b/sdk/resourcemanager/README.md @@ -38,7 +38,7 @@ For your convenience, we have provided a multi-service package that includes som com.azure.resourcemanager azure-resourcemanager - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) @@ -100,7 +100,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen com.azure azure-identity - 1.11.2 + 1.11.4 ``` [//]: # ({x-version-update-end}) @@ -112,7 +112,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen com.azure azure-core-http-netty - 1.14.0 + 1.14.1 ``` [//]: # ({x-version-update-end}) @@ -303,7 +303,7 @@ For example, here is sample maven dependency for Compute package. com.azure.resourcemanager azure-resourcemanager-compute - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-appplatform/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-appplatform/CHANGELOG.md index b233d2bf106ff..91de01dc5386a 100644 --- a/sdk/resourcemanager/azure-resourcemanager-appplatform/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-appplatform/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-appplatform/README.md b/sdk/resourcemanager/azure-resourcemanager-appplatform/README.md index 558493b9b374c..c7ae1043efb58 100644 --- a/sdk/resourcemanager/azure-resourcemanager-appplatform/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-appplatform/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-appplatform - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-appplatform/pom.xml b/sdk/resourcemanager/azure-resourcemanager-appplatform/pom.xml index 5eced62977549..e5325ab4aea2e 100644 --- a/sdk/resourcemanager/azure-resourcemanager-appplatform/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-appplatform/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-appplatform - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for App Platform Management @@ -62,12 +62,12 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure azure-storage-file-share - 12.21.2 + 12.21.3 com.azure @@ -102,25 +102,25 @@ com.azure azure-security-keyvault-certificates - 4.6.0 + 4.6.1 test com.azure.resourcemanager azure-resourcemanager-keyvault - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-dns - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-appservice - 2.37.0-beta.1 + 2.38.0-beta.1 test diff --git a/sdk/resourcemanager/azure-resourcemanager-appservice/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-appservice/CHANGELOG.md index 655c65125f996..e4998ea24fbaf 100644 --- a/sdk/resourcemanager/azure-resourcemanager-appservice/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-appservice/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.1 (2024-03-04) ### Bugs Fixed diff --git a/sdk/resourcemanager/azure-resourcemanager-appservice/README.md b/sdk/resourcemanager/azure-resourcemanager-appservice/README.md index 09beaa586f8e8..cda117e8291da 100644 --- a/sdk/resourcemanager/azure-resourcemanager-appservice/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-appservice/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-appservice - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-appservice/pom.xml b/sdk/resourcemanager/azure-resourcemanager-appservice/pom.xml index 8d7497740d5a9..04ee254548b99 100644 --- a/sdk/resourcemanager/azure-resourcemanager-appservice/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-appservice/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-appservice - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for App Service Management @@ -64,27 +64,27 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-keyvault - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-dns - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter @@ -125,7 +125,7 @@ com.azure.resourcemanager azure-resourcemanager-appcontainers - 1.0.0-beta.6 + 1.0.0-beta.7 test diff --git a/sdk/resourcemanager/azure-resourcemanager-authorization/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-authorization/CHANGELOG.md index 2ebf75c7ed35d..7c188c69d0dd9 100644 --- a/sdk/resourcemanager/azure-resourcemanager-authorization/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-authorization/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-authorization/README.md b/sdk/resourcemanager/azure-resourcemanager-authorization/README.md index 9bb9c71c81e03..88c0d4ea9f11c 100644 --- a/sdk/resourcemanager/azure-resourcemanager-authorization/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-authorization/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-authorization - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-authorization/pom.xml b/sdk/resourcemanager/azure-resourcemanager-authorization/pom.xml index 08d5bd37a9343..0786e010b90ce 100644 --- a/sdk/resourcemanager/azure-resourcemanager-authorization/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-authorization/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Authorization Management @@ -60,7 +60,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.slf4j @@ -97,13 +97,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/resourcemanager/azure-resourcemanager-cdn/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-cdn/CHANGELOG.md index fc4f8d597b5e5..9083997230bf3 100644 --- a/sdk/resourcemanager/azure-resourcemanager-cdn/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-cdn/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-cdn/README.md b/sdk/resourcemanager/azure-resourcemanager-cdn/README.md index cb887ad5ce025..b1bd3bfceb108 100644 --- a/sdk/resourcemanager/azure-resourcemanager-cdn/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-cdn/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-cdn - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-cdn/pom.xml b/sdk/resourcemanager/azure-resourcemanager-cdn/pom.xml index 812f9084e7968..a517191e20766 100644 --- a/sdk/resourcemanager/azure-resourcemanager-cdn/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-cdn/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-cdn - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for CDN Management @@ -58,7 +58,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-compute/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-compute/CHANGELOG.md index 10d0471c72e07..5b536292b1c01 100644 --- a/sdk/resourcemanager/azure-resourcemanager-compute/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-compute/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Features Added diff --git a/sdk/resourcemanager/azure-resourcemanager-compute/README.md b/sdk/resourcemanager/azure-resourcemanager-compute/README.md index 71f51fa067055..848ccf95124ef 100644 --- a/sdk/resourcemanager/azure-resourcemanager-compute/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-compute/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-compute - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-compute/pom.xml b/sdk/resourcemanager/azure-resourcemanager-compute/pom.xml index cc318f52acbba..71cbb9795d0f5 100644 --- a/sdk/resourcemanager/azure-resourcemanager-compute/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-compute/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-compute - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Compute Management @@ -69,27 +69,27 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-network - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 org.slf4j @@ -118,13 +118,13 @@ com.azure.resourcemanager azure-resourcemanager-keyvault - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure azure-storage-blob - 12.25.2 + 12.25.3 test diff --git a/sdk/resourcemanager/azure-resourcemanager-containerinstance/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-containerinstance/CHANGELOG.md index 0b96a57eee154..1c1572454623f 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerinstance/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-containerinstance/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-containerinstance/README.md b/sdk/resourcemanager/azure-resourcemanager-containerinstance/README.md index 6c44f4a33594c..91571fff18afe 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerinstance/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-containerinstance/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-containerinstance - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-containerinstance/pom.xml b/sdk/resourcemanager/azure-resourcemanager-containerinstance/pom.xml index 26d01a9b4ceab..b50451b73fe3f 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerinstance/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-containerinstance/pom.xml @@ -10,7 +10,7 @@ com.azure.resourcemanager azure-resourcemanager-containerinstance - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Container Instance Management @@ -60,32 +60,32 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-network - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure azure-storage-file-share - 12.21.2 + 12.21.3 com.azure diff --git a/sdk/resourcemanager/azure-resourcemanager-containerregistry/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-containerregistry/CHANGELOG.md index 4967c72d7f227..90abce1f5a0cd 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerregistry/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-containerregistry/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-containerregistry/README.md b/sdk/resourcemanager/azure-resourcemanager-containerregistry/README.md index 31095ce340a82..5e488226ef7fc 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerregistry/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-containerregistry/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-containerregistry - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-containerregistry/pom.xml b/sdk/resourcemanager/azure-resourcemanager-containerregistry/pom.xml index a1679fe7ff905..8f1401cf421b4 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerregistry/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-containerregistry/pom.xml @@ -10,7 +10,7 @@ com.azure.resourcemanager azure-resourcemanager-containerregistry - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Container Registry Management @@ -54,7 +54,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.slf4j diff --git a/sdk/resourcemanager/azure-resourcemanager-containerservice/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-containerservice/CHANGELOG.md index 9c7c235d4fe9c..c0c3ae05a6493 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerservice/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-containerservice/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-containerservice/README.md b/sdk/resourcemanager/azure-resourcemanager-containerservice/README.md index 6dee1d18ddf43..67f17798a7758 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerservice/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-containerservice/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-containerservice - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-containerservice/pom.xml b/sdk/resourcemanager/azure-resourcemanager-containerservice/pom.xml index 0f3179646528c..11157850c3003 100644 --- a/sdk/resourcemanager/azure-resourcemanager-containerservice/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-containerservice/pom.xml @@ -10,7 +10,7 @@ com.azure.resourcemanager azure-resourcemanager-containerservice - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Container Service Management @@ -55,7 +55,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-cosmos/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-cosmos/CHANGELOG.md index 3ce402056201b..753325b59de28 100644 --- a/sdk/resourcemanager/azure-resourcemanager-cosmos/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-cosmos/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,22 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + +## 2.37.0-beta.1 (2024-03-07) + +### Other Changes + +#### Dependency Updates + +- Preview release for `api-version` `2024-02-15-preview`. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-cosmos/README.md b/sdk/resourcemanager/azure-resourcemanager-cosmos/README.md index cb8f7be4ecf75..e19ce261874cc 100644 --- a/sdk/resourcemanager/azure-resourcemanager-cosmos/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-cosmos/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-cosmos - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-cosmos/pom.xml b/sdk/resourcemanager/azure-resourcemanager-cosmos/pom.xml index a92e5aa62331d..1299f15cd4b0a 100644 --- a/sdk/resourcemanager/azure-resourcemanager-cosmos/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-cosmos/pom.xml @@ -10,7 +10,7 @@ com.azure.resourcemanager azure-resourcemanager-cosmos - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for CosmosDB Management @@ -56,7 +56,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter @@ -73,13 +73,13 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-network - 2.37.0-beta.1 + 2.38.0-beta.1 test diff --git a/sdk/resourcemanager/azure-resourcemanager-dns/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-dns/CHANGELOG.md index d8269f4b917b9..12f18d8730fd2 100644 --- a/sdk/resourcemanager/azure-resourcemanager-dns/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-dns/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-dns/README.md b/sdk/resourcemanager/azure-resourcemanager-dns/README.md index 2d9ebe7d64dff..76dacd72636c6 100644 --- a/sdk/resourcemanager/azure-resourcemanager-dns/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-dns/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-dns - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-dns/pom.xml b/sdk/resourcemanager/azure-resourcemanager-dns/pom.xml index e5aa3b0d0dfda..5b8e392a1929c 100644 --- a/sdk/resourcemanager/azure-resourcemanager-dns/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-dns/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-dns - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for DNS Management @@ -58,7 +58,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 commons-io diff --git a/sdk/resourcemanager/azure-resourcemanager-eventhubs/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-eventhubs/CHANGELOG.md index 127c0ed6ce152..0a14ea1b9d5a2 100644 --- a/sdk/resourcemanager/azure-resourcemanager-eventhubs/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-eventhubs/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-eventhubs/README.md b/sdk/resourcemanager/azure-resourcemanager-eventhubs/README.md index 3d08eff90913a..3167fcb7db564 100644 --- a/sdk/resourcemanager/azure-resourcemanager-eventhubs/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-eventhubs/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-eventhubs - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-eventhubs/pom.xml b/sdk/resourcemanager/azure-resourcemanager-eventhubs/pom.xml index 108da0019aa88..1a238d038428d 100644 --- a/sdk/resourcemanager/azure-resourcemanager-eventhubs/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-eventhubs/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-eventhubs - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for EventHubs Management @@ -59,12 +59,12 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-keyvault/CHANGELOG.md index cac43f2e3fd11..40086ec423a15 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,12 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Features Added + +- Supported disabling public network access in `Vault` via `disablePublicNetworkAccess()`, for private link feature. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/README.md b/sdk/resourcemanager/azure-resourcemanager-keyvault/README.md index 613c71a025608..559d9a386f4e9 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-keyvault - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/assets.json b/sdk/resourcemanager/azure-resourcemanager-keyvault/assets.json index 27a205709fc41..2bafaf0cd04da 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/assets.json +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/resourcemanager/azure-resourcemanager-keyvault", - "Tag": "java/resourcemanager/azure-resourcemanager-keyvault_a24b8fea25" + "Tag": "java/resourcemanager/azure-resourcemanager-keyvault_1a7324e6c1" } diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/pom.xml b/sdk/resourcemanager/azure-resourcemanager-keyvault/pom.xml index dfbe634b43176..87e9309b25149 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-keyvault - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Key Vault Management @@ -59,17 +59,17 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure azure-security-keyvault-keys - 4.8.0 + 4.8.1 com.azure @@ -80,7 +80,7 @@ com.azure azure-security-keyvault-secrets - 4.8.0 + 4.8.1 com.azure diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/implementation/VaultImpl.java b/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/implementation/VaultImpl.java index 7cf9763a18208..70bc5eaff5c57 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/implementation/VaultImpl.java +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/implementation/VaultImpl.java @@ -25,6 +25,7 @@ import com.azure.resourcemanager.keyvault.models.NetworkRuleSet; import com.azure.resourcemanager.keyvault.models.PrivateEndpointServiceConnectionStatus; import com.azure.resourcemanager.keyvault.models.PrivateLinkServiceConnectionState; +import com.azure.resourcemanager.keyvault.models.PublicNetworkAccess; import com.azure.resourcemanager.keyvault.models.Secrets; import com.azure.resourcemanager.keyvault.models.Sku; import com.azure.resourcemanager.keyvault.models.SkuFamily; @@ -173,6 +174,13 @@ public boolean roleBasedAccessControlEnabled() { return ResourceManagerUtils.toPrimitiveBoolean(innerModel().properties().enableRbacAuthorization()); } + @Override + public PublicNetworkAccess publicNetworkAccess() { + return (innerModel().properties() == null || innerModel().properties().publicNetworkAccess() == null) + ? null + : PublicNetworkAccess.fromString(innerModel().properties().publicNetworkAccess()); + } + @Override public boolean enabledForDeployment() { if (innerModel().properties() == null) { @@ -415,6 +423,24 @@ public NetworkRuleSet networkRuleSet() { return innerModel().properties().networkAcls(); } + @Override + public VaultImpl enablePublicNetworkAccess() { + if (innerModel().properties() == null) { + innerModel().withProperties(new VaultProperties()); + } + this.innerModel().properties().withPublicNetworkAccess(PublicNetworkAccess.ENABLED.toString()); + return this; + } + + @Override + public VaultImpl disablePublicNetworkAccess() { + if (innerModel().properties() == null) { + innerModel().withProperties(new VaultProperties()); + } + this.innerModel().properties().withPublicNetworkAccess(PublicNetworkAccess.DISABLED.toString()); + return this; + } + @Override public VaultImpl withAccessFromAllNetworks() { if (innerModel().properties().networkAcls() == null) { diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/models/Vault.java b/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/models/Vault.java index 3a088900edfcf..824b6b15ef8bd 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/models/Vault.java +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/src/main/java/com/azure/resourcemanager/keyvault/models/Vault.java @@ -62,6 +62,13 @@ public interface Vault */ boolean roleBasedAccessControlEnabled(); + /** + * Whether the vault can be accessed from public network. + * + * @return whether the vault can be accessed from public network. + */ + PublicNetworkAccess publicNetworkAccess(); + /** * @return whether Azure Virtual Machines are permitted to retrieve certificates stored as secrets from the key * vault. @@ -166,6 +173,12 @@ interface WithAccessPolicy { /** A key vault definition allowing the networkAcl to be set. */ interface WithNetworkRuleSet { + /** + * Disables public network access for the vault, for private link feature. + * + * @return the next stage of the definition + */ + WithCreate disablePublicNetworkAccess(); /** * Specifies that by default access to key vault should be allowed from all networks. @@ -362,6 +375,19 @@ interface WithAccessPolicy { /** A key vault update allowing the NetworkRuleSet to be set. */ interface WithNetworkRuleSet { + /** + * Enables public network access for the vault. + * + * @return the next stage of the update + */ + Update enablePublicNetworkAccess(); + + /** + * Disables public network access for the vault, for private link feature. + * + * @return the next stage of the update + */ + Update disablePublicNetworkAccess(); /** * Specifies that by default access to key vault should be allowed from all networks. diff --git a/sdk/resourcemanager/azure-resourcemanager-keyvault/src/test/java/com/azure/resourcemanager/keyvault/VaultTests.java b/sdk/resourcemanager/azure-resourcemanager-keyvault/src/test/java/com/azure/resourcemanager/keyvault/VaultTests.java index 76788aa71a11f..b644e8d12ee0c 100644 --- a/sdk/resourcemanager/azure-resourcemanager-keyvault/src/test/java/com/azure/resourcemanager/keyvault/VaultTests.java +++ b/sdk/resourcemanager/azure-resourcemanager-keyvault/src/test/java/com/azure/resourcemanager/keyvault/VaultTests.java @@ -11,6 +11,7 @@ import com.azure.resourcemanager.keyvault.models.CertificatePermissions; import com.azure.resourcemanager.keyvault.models.KeyPermissions; import com.azure.resourcemanager.keyvault.models.NetworkRuleBypassOptions; +import com.azure.resourcemanager.keyvault.models.PublicNetworkAccess; import com.azure.resourcemanager.keyvault.models.SecretPermissions; import com.azure.resourcemanager.keyvault.models.Vault; import com.azure.core.management.Region; @@ -282,6 +283,26 @@ public void canEnableSoftDeleteAndPurge() throws InterruptedException { } } + @Test + public void canDisablePublicNetworkAccess() { + Vault vault = keyVaultManager.vaults().define(vaultName) + .withRegion(Region.US_WEST) + .withNewResourceGroup(rgName) + .withEmptyAccessPolicy() + .disablePublicNetworkAccess() + .create(); + + Assertions.assertEquals(PublicNetworkAccess.DISABLED, vault.publicNetworkAccess()); + Assertions.assertEquals(PublicNetworkAccess.DISABLED, keyVaultManager.vaults().getById(vault.id()).publicNetworkAccess()); + + vault.update() + .enablePublicNetworkAccess() + .apply(); + + Assertions.assertEquals(PublicNetworkAccess.ENABLED, vault.publicNetworkAccess()); + Assertions.assertEquals(PublicNetworkAccess.ENABLED, keyVaultManager.vaults().getById(vault.id()).publicNetworkAccess()); + } + private void assertVaultDeleted(String name, String location) { boolean deleted = false; try { diff --git a/sdk/resourcemanager/azure-resourcemanager-monitor/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-monitor/CHANGELOG.md index 8e343eb861eb0..86ed35378a917 100644 --- a/sdk/resourcemanager/azure-resourcemanager-monitor/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-monitor/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-monitor/README.md b/sdk/resourcemanager/azure-resourcemanager-monitor/README.md index 5bb77c49b1039..eaf82c0e5acda 100644 --- a/sdk/resourcemanager/azure-resourcemanager-monitor/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-monitor/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-monitor - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-monitor/pom.xml b/sdk/resourcemanager/azure-resourcemanager-monitor/pom.xml index 822ac03be9af1..0ae14305469c9 100644 --- a/sdk/resourcemanager/azure-resourcemanager-monitor/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-monitor/pom.xml @@ -10,7 +10,7 @@ com.azure.resourcemanager azure-resourcemanager-monitor - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Monitor Management @@ -61,7 +61,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter @@ -84,31 +84,31 @@ com.azure.resourcemanager azure-resourcemanager-compute - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-appservice - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-eventhubs - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-sql - 2.37.0-beta.1 + 2.38.0-beta.1 test diff --git a/sdk/resourcemanager/azure-resourcemanager-msi/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-msi/CHANGELOG.md index 6bf4bb85aa7dd..f7787ff01af4b 100644 --- a/sdk/resourcemanager/azure-resourcemanager-msi/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-msi/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-msi/README.md b/sdk/resourcemanager/azure-resourcemanager-msi/README.md index db672ca359172..b18391ec3f551 100644 --- a/sdk/resourcemanager/azure-resourcemanager-msi/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-msi/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-msi - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-msi/pom.xml b/sdk/resourcemanager/azure-resourcemanager-msi/pom.xml index b71a75bde9d89..b3e26c1b58245 100644 --- a/sdk/resourcemanager/azure-resourcemanager-msi/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-msi/pom.xml @@ -10,7 +10,7 @@ com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Managed Service Identity (MSI) Management @@ -57,12 +57,12 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md index dc512db20c3c9..350f7c5721d1e 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-network/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-network/README.md b/sdk/resourcemanager/azure-resourcemanager-network/README.md index 662de2c6d66ab..abbe5735d9bbe 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-network/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-network - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-network/pom.xml b/sdk/resourcemanager/azure-resourcemanager-network/pom.xml index c90fceadd7b9c..acd453e9cf76c 100644 --- a/sdk/resourcemanager/azure-resourcemanager-network/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-network/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-network - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Network Management @@ -64,7 +64,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter @@ -87,19 +87,19 @@ com.azure azure-security-keyvault-certificates - 4.6.0 + 4.6.1 test com.azure.resourcemanager azure-resourcemanager-keyvault - 2.37.0-beta.1 + 2.38.0-beta.1 test com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 test diff --git a/sdk/resourcemanager/azure-resourcemanager-perf/pom.xml b/sdk/resourcemanager/azure-resourcemanager-perf/pom.xml index aacc62bcaa65c..0a1951aadae1d 100644 --- a/sdk/resourcemanager/azure-resourcemanager-perf/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-perf/pom.xml @@ -25,7 +25,7 @@ com.azure.resourcemanager azure-resourcemanager - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager diff --git a/sdk/resourcemanager/azure-resourcemanager-privatedns/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-privatedns/CHANGELOG.md index 054b24b7f3e90..fc93d0366f7de 100644 --- a/sdk/resourcemanager/azure-resourcemanager-privatedns/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-privatedns/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-privatedns/README.md b/sdk/resourcemanager/azure-resourcemanager-privatedns/README.md index 9160688ce8f6d..e5c8ae5941e36 100644 --- a/sdk/resourcemanager/azure-resourcemanager-privatedns/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-privatedns/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-privatedns - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-privatedns/pom.xml b/sdk/resourcemanager/azure-resourcemanager-privatedns/pom.xml index 040d8ac7aa962..ac0f9fcfd1595 100644 --- a/sdk/resourcemanager/azure-resourcemanager-privatedns/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-privatedns/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-privatedns - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Private DNS Management @@ -61,12 +61,12 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-network - 2.37.0-beta.1 + 2.38.0-beta.1 test diff --git a/sdk/resourcemanager/azure-resourcemanager-redis/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-redis/CHANGELOG.md index 72bf157d8a1b9..095cc473a508f 100644 --- a/sdk/resourcemanager/azure-resourcemanager-redis/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-redis/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-redis/README.md b/sdk/resourcemanager/azure-resourcemanager-redis/README.md index 55457d3318fb7..211cc66392c34 100644 --- a/sdk/resourcemanager/azure-resourcemanager-redis/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-redis/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-redis - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-redis/pom.xml b/sdk/resourcemanager/azure-resourcemanager-redis/pom.xml index 373325a6b58d0..a16f817c51e51 100644 --- a/sdk/resourcemanager/azure-resourcemanager-redis/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-redis/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-redis - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Redis Cache Management @@ -61,7 +61,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter @@ -84,7 +84,7 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 test diff --git a/sdk/resourcemanager/azure-resourcemanager-resources/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-resources/CHANGELOG.md index b39070820845b..6e385ab6f4f1e 100644 --- a/sdk/resourcemanager/azure-resourcemanager-resources/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-resources/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-resources/README.md b/sdk/resourcemanager/azure-resourcemanager-resources/README.md index 30acfd2b9e0c2..b1204b5682069 100644 --- a/sdk/resourcemanager/azure-resourcemanager-resources/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-resources/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-resources/pom.xml b/sdk/resourcemanager/azure-resourcemanager-resources/pom.xml index 64f7956a94e9d..4f0b30c34df76 100644 --- a/sdk/resourcemanager/azure-resourcemanager-resources/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-resources/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Resource Management @@ -94,7 +94,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -108,13 +108,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/resourcemanager/azure-resourcemanager-samples/README.md b/sdk/resourcemanager/azure-resourcemanager-samples/README.md index f133dd0f68b52..7778310e67a5d 100644 --- a/sdk/resourcemanager/azure-resourcemanager-samples/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-samples/README.md @@ -17,7 +17,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-samples/pom.xml b/sdk/resourcemanager/azure-resourcemanager-samples/pom.xml index fd5224f48da8f..4d953630dc6b6 100644 --- a/sdk/resourcemanager/azure-resourcemanager-samples/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-samples/pom.xml @@ -54,7 +54,7 @@ com.azure.resourcemanager azure-resourcemanager - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager @@ -74,7 +74,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 com.jcraft @@ -104,27 +104,27 @@ com.google.guava guava - 33.0.0-jre + 33.1.0-jre com.azure azure-storage-blob - 12.25.2 + 12.25.3 com.azure azure-cosmos - 4.56.0 + 4.57.0 com.azure azure-security-keyvault-certificates - 4.6.0 + 4.6.1 com.azure azure-messaging-servicebus - 7.15.1 + 7.15.2 io.fabric8 @@ -156,7 +156,7 @@ com.azure azure-security-keyvault-administration - 4.5.0 + 4.5.1 test @@ -198,7 +198,7 @@ io.fabric8:kubernetes-client:[5.12.3] com.jcraft:jsch:[0.1.55] org.slf4j:slf4j-simple:[1.7.36] - com.google.guava:guava:[33.0.0-jre] + com.google.guava:guava:[33.1.0-jre] com.github.docker-java:docker-java:[3.2.1] com.microsoft.sqlserver:mssql-jdbc:[10.2.3.jre8] commons-net:commons-net:[3.9.0] diff --git a/sdk/resourcemanager/azure-resourcemanager-search/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-search/CHANGELOG.md index d2430b3a97a43..db8dcec672228 100644 --- a/sdk/resourcemanager/azure-resourcemanager-search/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-search/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.2 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.37.0-beta.1 (2024-03-07) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-search/README.md b/sdk/resourcemanager/azure-resourcemanager-search/README.md index afa0fc4243c54..793361161f0f8 100644 --- a/sdk/resourcemanager/azure-resourcemanager-search/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-search/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-search - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-search/pom.xml b/sdk/resourcemanager/azure-resourcemanager-search/pom.xml index 2ff6315cce102..ab93ee04eb376 100644 --- a/sdk/resourcemanager/azure-resourcemanager-search/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-search/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-search - 2.37.0-beta.2 + 2.38.0-beta.1 jar Microsoft Azure SDK for Cognitive Search Management @@ -58,7 +58,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-servicebus/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-servicebus/CHANGELOG.md index e6d63a665c73b..e1932b3b092ac 100644 --- a/sdk/resourcemanager/azure-resourcemanager-servicebus/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-servicebus/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-servicebus/README.md b/sdk/resourcemanager/azure-resourcemanager-servicebus/README.md index dbdd8eaa6fe44..ea69e81222935 100644 --- a/sdk/resourcemanager/azure-resourcemanager-servicebus/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-servicebus/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-servicebus - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-servicebus/pom.xml b/sdk/resourcemanager/azure-resourcemanager-servicebus/pom.xml index 726b889c328b6..06a6e046dcec0 100644 --- a/sdk/resourcemanager/azure-resourcemanager-servicebus/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-servicebus/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-servicebus - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for ServiceBus Management @@ -58,7 +58,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-sql/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-sql/CHANGELOG.md index 01c7cc29cefcd..84d43a05f03d2 100644 --- a/sdk/resourcemanager/azure-resourcemanager-sql/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-sql/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-sql/README.md b/sdk/resourcemanager/azure-resourcemanager-sql/README.md index 4d23875cae8d7..a7330e499d429 100644 --- a/sdk/resourcemanager/azure-resourcemanager-sql/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-sql/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-sql - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-sql/pom.xml b/sdk/resourcemanager/azure-resourcemanager-sql/pom.xml index 9393b2d149fbe..5495fb6e2d347 100644 --- a/sdk/resourcemanager/azure-resourcemanager-sql/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-sql/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-sql - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Sql Management @@ -63,12 +63,12 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 org.junit.jupiter diff --git a/sdk/resourcemanager/azure-resourcemanager-storage/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-storage/CHANGELOG.md index 1fbef57bd71df..9fc9422edfca1 100644 --- a/sdk/resourcemanager/azure-resourcemanager-storage/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-storage/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Features Added diff --git a/sdk/resourcemanager/azure-resourcemanager-storage/README.md b/sdk/resourcemanager/azure-resourcemanager-storage/README.md index d7a881123df51..ab8183be83dd4 100644 --- a/sdk/resourcemanager/azure-resourcemanager-storage/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-storage/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-storage - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-storage/pom.xml b/sdk/resourcemanager/azure-resourcemanager-storage/pom.xml index ed67bcb1b49f3..faecd87488235 100644 --- a/sdk/resourcemanager/azure-resourcemanager-storage/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-storage/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Storage Management @@ -62,17 +62,17 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 org.slf4j diff --git a/sdk/resourcemanager/azure-resourcemanager-test/pom.xml b/sdk/resourcemanager/azure-resourcemanager-test/pom.xml index 784861c8c01d8..161bbec5e0e44 100644 --- a/sdk/resourcemanager/azure-resourcemanager-test/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-test/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 com.azure diff --git a/sdk/resourcemanager/azure-resourcemanager-trafficmanager/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager-trafficmanager/CHANGELOG.md index f89c412461386..86cafd44b2a99 100644 --- a/sdk/resourcemanager/azure-resourcemanager-trafficmanager/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager-trafficmanager/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager-trafficmanager/README.md b/sdk/resourcemanager/azure-resourcemanager-trafficmanager/README.md index 1f64643c8441a..0207e3aa930b3 100644 --- a/sdk/resourcemanager/azure-resourcemanager-trafficmanager/README.md +++ b/sdk/resourcemanager/azure-resourcemanager-trafficmanager/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager-trafficmanager - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager-trafficmanager/pom.xml b/sdk/resourcemanager/azure-resourcemanager-trafficmanager/pom.xml index 6e33e55859a62..5070556159bf4 100644 --- a/sdk/resourcemanager/azure-resourcemanager-trafficmanager/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager-trafficmanager/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-trafficmanager - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Traffic Manager Management @@ -58,7 +58,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 org.slf4j diff --git a/sdk/resourcemanager/azure-resourcemanager/CHANGELOG.md b/sdk/resourcemanager/azure-resourcemanager/CHANGELOG.md index b5f9338e809df..c2f859e003b15 100644 --- a/sdk/resourcemanager/azure-resourcemanager/CHANGELOG.md +++ b/sdk/resourcemanager/azure-resourcemanager/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 2.37.0-beta.1 (Unreleased) +## 2.38.0-beta.1 (Unreleased) ### Features Added @@ -10,6 +10,14 @@ ### Other Changes +## 2.37.0 (2024-03-22) + +### Other Changes + +#### Dependency Updates + +- Updated core dependency from resources. + ## 2.36.0 (2024-02-29) ### Other Changes diff --git a/sdk/resourcemanager/azure-resourcemanager/README.md b/sdk/resourcemanager/azure-resourcemanager/README.md index e7127eac1449a..3519646971366 100644 --- a/sdk/resourcemanager/azure-resourcemanager/README.md +++ b/sdk/resourcemanager/azure-resourcemanager/README.md @@ -18,7 +18,7 @@ For documentation on how to use this package, please see [Azure Management Libra com.azure.resourcemanager azure-resourcemanager - 2.36.0 + 2.37.0 ``` [//]: # ({x-version-update-end}) diff --git a/sdk/resourcemanager/azure-resourcemanager/assets.json b/sdk/resourcemanager/azure-resourcemanager/assets.json index b20fdf0f448be..6e1850be1d664 100644 --- a/sdk/resourcemanager/azure-resourcemanager/assets.json +++ b/sdk/resourcemanager/azure-resourcemanager/assets.json @@ -2,5 +2,5 @@ "AssetsRepo": "Azure/azure-sdk-assets", "AssetsRepoPrefixPath": "java", "TagPrefix": "java/resourcemanager/azure-resourcemanager", - "Tag": "java/resourcemanager/azure-resourcemanager_e04fb35c26" + "Tag": "java/resourcemanager/azure-resourcemanager_b75c83930c" } diff --git a/sdk/resourcemanager/azure-resourcemanager/pom.xml b/sdk/resourcemanager/azure-resourcemanager/pom.xml index 291f75d2d644f..6d6a1824231f3 100644 --- a/sdk/resourcemanager/azure-resourcemanager/pom.xml +++ b/sdk/resourcemanager/azure-resourcemanager/pom.xml @@ -15,7 +15,7 @@ com.azure.resourcemanager azure-resourcemanager - 2.37.0-beta.1 + 2.38.0-beta.1 jar Microsoft Azure SDK for Management @@ -81,117 +81,117 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-storage - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-compute - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-network - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-keyvault - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-msi - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-sql - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-authorization - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-appservice - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-cosmos - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-containerservice - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-monitor - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-containerregistry - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-dns - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-appplatform - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-containerinstance - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-privatedns - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-redis - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-eventhubs - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-trafficmanager - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-servicebus - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-cdn - 2.37.0-beta.1 + 2.38.0-beta.1 com.azure.resourcemanager azure-resourcemanager-search - 2.37.0-beta.2 + 2.38.0-beta.1 org.junit.jupiter @@ -234,13 +234,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/resourcemanager/azure-resourcemanager/src/test/java/com/azure/resourcemanager/PrivateLinkTests.java b/sdk/resourcemanager/azure-resourcemanager/src/test/java/com/azure/resourcemanager/PrivateLinkTests.java index 19f8634b617e2..882ce2ab1b5fc 100644 --- a/sdk/resourcemanager/azure-resourcemanager/src/test/java/com/azure/resourcemanager/PrivateLinkTests.java +++ b/sdk/resourcemanager/azure-resourcemanager/src/test/java/com/azure/resourcemanager/PrivateLinkTests.java @@ -377,6 +377,7 @@ public void testPrivateEndpointVault() { .withRegion(region) .withNewResourceGroup(rgName) .withEmptyAccessPolicy() + .disablePublicNetworkAccess() .create(); validatePrivateLinkResource(vault, subResourceName.toString()); diff --git a/sdk/resourcemanager/ci.yml b/sdk/resourcemanager/ci.yml index 8d791f83c1a35..576efa2b7294d 100644 --- a/sdk/resourcemanager/ci.yml +++ b/sdk/resourcemanager/ci.yml @@ -285,6 +285,8 @@ extends: safeName: azureresourcemanagersamples - name: azure-resourcemanager-perf groupId: com.azure.resourcemanager + - name: azure-resourcemanager-test + groupId: com.azure.resourcemanager # required by the above perf library - name: perf-test-core groupId: com.azure diff --git a/sdk/resourcemanager/docs/SINGLE_SERVICE_PACKAGES.md b/sdk/resourcemanager/docs/SINGLE_SERVICE_PACKAGES.md index de0dc26252a76..93e2b64206b0d 100644 --- a/sdk/resourcemanager/docs/SINGLE_SERVICE_PACKAGES.md +++ b/sdk/resourcemanager/docs/SINGLE_SERVICE_PACKAGES.md @@ -93,6 +93,8 @@ service

appcontainers +* [package-preview-2023-11](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/app/resource-manager#tag-package-preview-2023-11) + * [1.0.0-beta.7](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-appcontainers/1.0.0-beta.7) * [package-2023-05](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/app/resource-manager#tag-package-2023-05) * [1.0.0-beta.6](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-appcontainers/1.0.0-beta.6) * [package-preview-2022-11](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/app/resource-manager#tag-package-preview-2022-11) @@ -284,6 +286,8 @@ service
chaos +* [package-2024-01](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/chaos/resource-manager#tag-package-2024-01) + * [1.1.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-chaos/1.1.0) * [package-2023-11](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/chaos/resource-manager#tag-package-2023-11) * [1.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-chaos/1.0.0) * [package-2023-04-15-preview](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/chaos/resource-manager#tag-package-2023-04-15-preview) @@ -325,6 +329,8 @@ service
communication +* [package-2023-04](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/communication/resource-manager#tag-package-2023-04) + * [2.1.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-communication/2.1.0) * [package-preview-2023-06](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/communication/resource-manager#tag-package-preview-2023-06) * [2.1.0-beta.2](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-communication/2.1.0-beta.2) * [package-preview-2023-04](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/communication/resource-manager#tag-package-preview-2023-04) @@ -358,6 +364,8 @@ service
confluent +* [package-2024-02](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/confluent/resource-manager#tag-package-2024-02) + * [1.1.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-confluent/1.1.0) * [package-2023-08-22](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/confluent/resource-manager#tag-package-2023-08-22) * [1.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-confluent/1.0.0) * [package-preview-2021-09](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/confluent/resource-manager#tag-package-preview-2021-09) @@ -407,6 +415,8 @@ service
cosmosdbforpostgresql +* [package-preview-2023-03](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/postgresqlhsc/resource-manager#tag-package-preview-2023-03) + * [1.1.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/1.1.0-beta.1) * [package-2022-11-08](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/postgresqlhsc/resource-manager#tag-package-2022-11-08) * [1.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/1.0.0) * [1.0.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-cosmosdbforpostgresql/1.0.0-beta.1) @@ -500,6 +510,7 @@ service datafactory * [package-2018-06](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/datafactory/resource-manager#tag-package-2018-06) + * [1.0.0-beta.27](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-datafactory/1.0.0-beta.27) * [1.0.0-beta.26](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-datafactory/1.0.0-beta.26) * [1.0.0-beta.25](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-datafactory/1.0.0-beta.25) * [1.0.0-beta.24](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-datafactory/1.0.0-beta.24) @@ -990,6 +1001,8 @@ service
iotfirmwaredefense +* [package-2024-01-10](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/fist/resource-manager#tag-package-2024-01-10) + * [1.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-iotfirmwaredefense/1.0.0) * [package-2023-02-08-preview](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/fist/resource-manager#tag-package-2023-02-08-preview) * [1.0.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-iotfirmwaredefense/1.0.0-beta.1)
@@ -1238,6 +1251,8 @@ service
mobilenetwork +* [package-2024-02](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/mobilenetwork/resource-manager#tag-package-2024-02) + * [1.1.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-mobilenetwork/1.1.0) * [package-2023-09](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/mobilenetwork/resource-manager#tag-package-2023-09) * [1.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-mobilenetwork/1.0.0) * [package-2023-06](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/mobilenetwork/resource-manager#tag-package-2023-06) @@ -1281,6 +1296,8 @@ service
netapp +* [package-netapp-2023-07-01](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/netapp/resource-manager#tag-package-netapp-2023-07-01) + * [1.1.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-netapp/1.1.0) * [package-preview-2023-05](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/netapp/resource-manager#tag-package-preview-2023-05) * [1.1.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-netapp/1.1.0-beta.1) * [package-netapp-2023-05-01](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/netapp/resource-manager#tag-package-netapp-2023-05-01) @@ -1347,6 +1364,8 @@ service
newrelicobservability +* [package-2024-01-01](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/newrelic/resource-manager#tag-package-2024-01-01) + * [1.1.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-newrelicobservability/1.1.0) * [package-2022-07-01](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/newrelic/resource-manager#tag-package-2022-07-01) * [1.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-newrelicobservability/1.0.0) * [1.0.0-beta.2](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-newrelicobservability/1.0.0-beta.2) @@ -1369,6 +1388,8 @@ service
notificationhubs +* [package-preview-2023-10](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/notificationhubs/resource-manager#tag-package-preview-2023-10) + * [1.0.0-beta.4](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-notificationhubs/1.0.0-beta.4) * [package-2017-04](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/notificationhubs/resource-manager#tag-package-2017-04) * [1.0.0-beta.3](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-notificationhubs/1.0.0-beta.3) * [1.0.0-beta.2](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-notificationhubs/1.0.0-beta.2) @@ -1506,6 +1527,8 @@ service
quantum +* [package-2023-11-13-preview](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/quantum/resource-manager#tag-package-2023-11-13-preview) + * [1.0.0-beta.2](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-quantum/1.0.0-beta.2) * [package-2022-01-10-preview](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/quantum/resource-manager#tag-package-2022-01-10-preview) * [1.0.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-quantum/1.0.0-beta.1)
@@ -1604,6 +1627,8 @@ service
redisenterprise +* [package-2023-11](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/redisenterprise/resource-manager#tag-package-2023-11) + * [2.0.0](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-redisenterprise/2.0.0) * [package-preview-2023-03](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/redisenterprise/resource-manager#tag-package-preview-2023-03) * [1.1.0-beta.3](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-redisenterprise/1.1.0-beta.3) * [1.1.0-beta.2](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-redisenterprise/1.1.0-beta.2) @@ -1697,6 +1722,7 @@ service security * [package-composite-v3](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/security/resource-manager#tag-package-composite-v3) + * [1.0.0-beta.6](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-security/1.0.0-beta.6) * [1.0.0-beta.5](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-security/1.0.0-beta.5) * [1.0.0-beta.4](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-security/1.0.0-beta.4) * [1.0.0-beta.3](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-security/1.0.0-beta.3) @@ -1823,6 +1849,14 @@ service * [1.0.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-sqlvirtualmachine/1.0.0-beta.1)
+
+
+ storageactions + +* [package-2023-01-01](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/storageactions/resource-manager#tag-package-2023-01-01) + * [1.0.0-beta.1](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-storageactions/1.0.0-beta.1) +
+
storagecache @@ -1906,6 +1940,8 @@ service
support +* [package-preview-2023-06](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/support/resource-manager#tag-package-preview-2023-06) + * [1.0.0-beta.4](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-support/1.0.0-beta.4) * [package-preview-2022-09](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/support/resource-manager#tag-package-preview-2022-09) * [1.0.0-beta.3](https://repo1.maven.org/maven2/com/azure/resourcemanager/azure-resourcemanager-support/1.0.0-beta.3) * [package-2020-04](https://github.com/Azure/azure-rest-api-specs/tree/main/specification/support/resource-manager#tag-package-2020-04) diff --git a/sdk/resourcemanagerhybrid/azure-resourcemanager-compute/pom.xml b/sdk/resourcemanagerhybrid/azure-resourcemanager-compute/pom.xml index fd49ec03d9ff1..f48e1278564de 100644 --- a/sdk/resourcemanagerhybrid/azure-resourcemanager-compute/pom.xml +++ b/sdk/resourcemanagerhybrid/azure-resourcemanager-compute/pom.xml @@ -83,7 +83,7 @@ com.azure.resourcemanager azure-resourcemanager-msi - 2.36.0 + 2.37.0 com.azure.resourcemanager @@ -123,7 +123,7 @@ com.azure azure-storage-blob - 12.25.2 + 12.25.3 test diff --git a/sdk/resourcemanagerhybrid/azure-resourcemanager-keyvault/pom.xml b/sdk/resourcemanagerhybrid/azure-resourcemanager-keyvault/pom.xml index c5851a3c4d068..9a71b1b977800 100644 --- a/sdk/resourcemanagerhybrid/azure-resourcemanager-keyvault/pom.xml +++ b/sdk/resourcemanagerhybrid/azure-resourcemanager-keyvault/pom.xml @@ -68,7 +68,7 @@ com.azure azure-security-keyvault-keys - 4.8.0 + 4.8.1 com.azure @@ -79,7 +79,7 @@ com.azure azure-security-keyvault-secrets - 4.8.0 + 4.8.1 com.azure diff --git a/sdk/resourcemanagerhybrid/azure-resourcemanager-network/pom.xml b/sdk/resourcemanagerhybrid/azure-resourcemanager-network/pom.xml index 44528cfa704f1..585dcf77dae90 100644 --- a/sdk/resourcemanagerhybrid/azure-resourcemanager-network/pom.xml +++ b/sdk/resourcemanagerhybrid/azure-resourcemanager-network/pom.xml @@ -87,7 +87,7 @@ com.azure azure-security-keyvault-certificates - 4.6.0 + 4.6.1 test @@ -99,7 +99,7 @@ com.azure.resourcemanager azure-resourcemanager-msi - 2.36.0 + 2.37.0 test diff --git a/sdk/resourcemanagerhybrid/azure-resourcemanager-resources/pom.xml b/sdk/resourcemanagerhybrid/azure-resourcemanager-resources/pom.xml index e17615ce9a157..9e685f84288b6 100644 --- a/sdk/resourcemanagerhybrid/azure-resourcemanager-resources/pom.xml +++ b/sdk/resourcemanagerhybrid/azure-resourcemanager-resources/pom.xml @@ -93,7 +93,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/resourcemanagerhybrid/azure-resourcemanager-test/pom.xml b/sdk/resourcemanagerhybrid/azure-resourcemanager-test/pom.xml index e816cc7920849..6e39c66604f5c 100644 --- a/sdk/resourcemanagerhybrid/azure-resourcemanager-test/pom.xml +++ b/sdk/resourcemanagerhybrid/azure-resourcemanager-test/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 com.azure diff --git a/sdk/resourcemover/azure-resourcemanager-resourcemover/pom.xml b/sdk/resourcemover/azure-resourcemanager-resourcemover/pom.xml index 2c3ba90e26ea2..89198c28ae674 100644 --- a/sdk/resourcemover/azure-resourcemanager-resourcemover/pom.xml +++ b/sdk/resourcemover/azure-resourcemanager-resourcemover/pom.xml @@ -67,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -93,13 +93,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test @@ -111,7 +111,7 @@ com.azure.resourcemanager azure-resourcemanager-resources - 2.36.0 + 2.37.0 test diff --git a/sdk/resourcemover/tests.mgmt.yml b/sdk/resourcemover/tests.mgmt.yml index 157f58ef2d5c8..4f3da09d85d77 100644 --- a/sdk/resourcemover/tests.mgmt.yml +++ b/sdk/resourcemover/tests.mgmt.yml @@ -2,15 +2,15 @@ trigger: none pr: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: resourcemover - Artifacts: - - name: azure-resourcemanager-resourcemover - groupId: com.azure.resourcemanager - safeName: azureresourcemanagerresourcemover - Clouds: 'Public' - # Only run tests on Windows to save cost. - MatrixFilters: - - pool=.*(win).* +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: resourcemover + Artifacts: + - name: azure-resourcemanager-resourcemover + groupId: com.azure.resourcemanager + safeName: azureresourcemanagerresourcemover + Clouds: 'Public' + # Only run tests on Windows to save cost. + MatrixFilters: + - pool=.*(win).* diff --git a/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/CHANGELOG.md b/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/CHANGELOG.md index d20a1e022eed6..0c229c29eaf4e 100644 --- a/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/CHANGELOG.md +++ b/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/CHANGELOG.md @@ -10,6 +10,37 @@ ### Other Changes + +## 1.1.15 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-data-schemaregistry` from `1.4.3` to version `1.4.4`. + + +## 1.1.14 (2024-02-22) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.45.1` to version `1.46.0`. +- Upgraded `azure-data-schemaregistry` from `1.4.2` to version `1.4.3`. + + +## 1.1.13 (2023-12-04) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-data-schemaregistry` from `1.4.1` to version `1.4.2`. +- Upgraded `azure-core` from `1.45.0` to version `1.45.1`. + + ## 1.1.12 (2023-11-20) ### Other Changes diff --git a/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/pom.xml b/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/pom.xml index d15946dc2435a..081973dcfe653 100644 --- a/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/pom.xml +++ b/sdk/schemaregistry/azure-data-schemaregistry-apacheavro/pom.xml @@ -80,7 +80,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -110,13 +110,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure azure-messaging-eventhubs - 5.18.1 + 5.18.2 test @@ -130,13 +130,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/schemaregistry/azure-data-schemaregistry-jsonschema/pom.xml b/sdk/schemaregistry/azure-data-schemaregistry-jsonschema/pom.xml index 3a7d6c0b7e408..780c9b19822e3 100644 --- a/sdk/schemaregistry/azure-data-schemaregistry-jsonschema/pom.xml +++ b/sdk/schemaregistry/azure-data-schemaregistry-jsonschema/pom.xml @@ -61,7 +61,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -91,13 +91,13 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test com.azure azure-messaging-eventhubs - 5.18.1 + 5.18.2 test @@ -111,13 +111,13 @@ net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/schemaregistry/azure-data-schemaregistry/CHANGELOG.md b/sdk/schemaregistry/azure-data-schemaregistry/CHANGELOG.md index 02576fb451e64..ca16750be9add 100644 --- a/sdk/schemaregistry/azure-data-schemaregistry/CHANGELOG.md +++ b/sdk/schemaregistry/azure-data-schemaregistry/CHANGELOG.md @@ -10,6 +10,27 @@ ### Other Changes + +## 1.4.4 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + +## 1.4.3 (2024-02-22) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.45.1` to version `1.46.0`. +- Upgraded `azure-core-http-netty` from `1.13.11` to version `1.14.0`. + + ## 1.4.2 (2023-12-04) ### Other Changes diff --git a/sdk/schemaregistry/azure-data-schemaregistry/pom.xml b/sdk/schemaregistry/azure-data-schemaregistry/pom.xml index 0449f386b73dc..344ffb9ca2886 100644 --- a/sdk/schemaregistry/azure-data-schemaregistry/pom.xml +++ b/sdk/schemaregistry/azure-data-schemaregistry/pom.xml @@ -64,7 +64,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test @@ -100,7 +100,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test diff --git a/sdk/schemaregistry/tests.yml b/sdk/schemaregistry/tests.yml index 626809e10862e..cb5df2446cac5 100644 --- a/sdk/schemaregistry/tests.yml +++ b/sdk/schemaregistry/tests.yml @@ -1,20 +1,20 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: schemaregistry - Artifacts: - - name: azure-data-schemaregistry - groupId: com.azure - safeName: azuredataschemaregistry - - name: azure-data-schemaregistry-apacheavro - groupId: com.azure - safeName: azuredataschemaregistryapacheavro - TimeoutInMinutes: 60 - EnvVars: - AZURE_CLIENT_ID: $(SCHEMAREGISTRY_CLIENT_ID) - AZURE_CLIENT_SECRET: $(SCHEMAREGISTRY_CLIENT_SECRET) - AZURE_TENANT_ID: $(SCHEMAREGISTRY_TENANT_ID) - SCHEMA_REGISTRY_GROUP: $(SCHEMA_REGISTRY_GROUP) - SCHEMA_REGISTRY_ENDPOINT: $(SCHEMA_REGISTRY_ENDPOINT) +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: schemaregistry + Artifacts: + - name: azure-data-schemaregistry + groupId: com.azure + safeName: azuredataschemaregistry + - name: azure-data-schemaregistry-apacheavro + groupId: com.azure + safeName: azuredataschemaregistryapacheavro + TimeoutInMinutes: 60 + EnvVars: + AZURE_CLIENT_ID: $(SCHEMAREGISTRY_CLIENT_ID) + AZURE_CLIENT_SECRET: $(SCHEMAREGISTRY_CLIENT_SECRET) + AZURE_TENANT_ID: $(SCHEMAREGISTRY_TENANT_ID) + SCHEMA_REGISTRY_GROUP: $(SCHEMA_REGISTRY_GROUP) + SCHEMA_REGISTRY_ENDPOINT: $(SCHEMA_REGISTRY_ENDPOINT) diff --git a/sdk/search/azure-search-documents/CHANGELOG.md b/sdk/search/azure-search-documents/CHANGELOG.md index cb266dd6656c0..1a35220e4c2a6 100644 --- a/sdk/search/azure-search-documents/CHANGELOG.md +++ b/sdk/search/azure-search-documents/CHANGELOG.md @@ -1,5 +1,27 @@ # Release History +## 11.7.0-beta.3 (Unreleased) + +### Features Added + +### Breaking Changes + +### Bugs Fixed + +### Other Changes + + +## 11.6.3 (2024-03-20) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core-serializer-json-jackson` from `1.4.9` to version `1.4.10`. +- Upgraded `azure-core` from `1.46.0` to version `1.47.0`. +- Upgraded `azure-core-http-netty` from `1.14.0` to version `1.14.1`. + + ## 11.7.0-beta.2 (2024-03-11) ### Features Added @@ -20,6 +42,17 @@ - Upgraded `azure-core-http-netty` to version `1.14.1`. - Upgraded `azure-core` to version `1.47.0`. + +## 11.6.2 (2024-02-22) + +### Other Changes + +#### Dependency Updates + +- Upgraded `azure-core` from `1.45.1` to version `1.46.0`. +- Upgraded `azure-core-http-netty` from `1.13.11` to version `1.14.0`. + + ## 11.7.0-beta.1 (2024-02-09) ### Features Added diff --git a/sdk/search/azure-search-documents/pom.xml b/sdk/search/azure-search-documents/pom.xml index 2e6c2fa5460d1..5f3c86e0a8d2f 100644 --- a/sdk/search/azure-search-documents/pom.xml +++ b/sdk/search/azure-search-documents/pom.xml @@ -16,7 +16,7 @@ com.azure azure-search-documents - 11.7.0-beta.2 + 11.7.0-beta.3 jar @@ -100,7 +100,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -124,7 +124,7 @@ io.projectreactor reactor-test - 3.4.34 + 3.4.36 test diff --git a/sdk/search/azure-search-perf/pom.xml b/sdk/search/azure-search-perf/pom.xml index e1036ef48ed60..23784663b388e 100644 --- a/sdk/search/azure-search-perf/pom.xml +++ b/sdk/search/azure-search-perf/pom.xml @@ -29,7 +29,7 @@ com.azure azure-search-documents - 11.7.0-beta.2 + 11.7.0-beta.3 diff --git a/sdk/search/tests.yml b/sdk/search/tests.yml index bd476f8ee810b..65fa17418f63f 100644 --- a/sdk/search/tests.yml +++ b/sdk/search/tests.yml @@ -1,24 +1,24 @@ trigger: none -stages: - - template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml - parameters: - ServiceDirectory: search - # TODO: change 'Preview' cloud back to public after search RP fixes deletion metadata issue - Clouds: 'Preview' - SupportedClouds: 'Preview,UsGov,China' - TimeoutInMinutes: 60 - CloudConfig: - Preview: - SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources-preview) - Location: 'eastus2euap' - UsGov: - SubscriptionConfiguration: $(sub-config-gov-test-resources) - China: - SubscriptionConfiguration: $(sub-config-cn-test-resources) - EnvVars: - AZURE_SEARCH_TEST_MODE: Live - Artifacts: - - name: azure-search-documents - groupId: com.azure - safeName: azuresearch +extends: + template: /eng/pipelines/templates/stages/archetype-sdk-tests.yml + parameters: + ServiceDirectory: search + # TODO: change 'Preview' cloud back to public after search RP fixes deletion metadata issue + Clouds: 'Preview' + SupportedClouds: 'Preview,UsGov,China' + TimeoutInMinutes: 60 + CloudConfig: + Preview: + SubscriptionConfiguration: $(sub-config-azure-cloud-test-resources-preview) + Location: 'eastus2euap' + UsGov: + SubscriptionConfiguration: $(sub-config-gov-test-resources) + China: + SubscriptionConfiguration: $(sub-config-cn-test-resources) + EnvVars: + AZURE_SEARCH_TEST_MODE: Live + Artifacts: + - name: azure-search-documents + groupId: com.azure + safeName: azuresearch diff --git a/sdk/security/azure-resourcemanager-security/CHANGELOG.md b/sdk/security/azure-resourcemanager-security/CHANGELOG.md index bd1644944ed74..95aa74b1ef094 100644 --- a/sdk/security/azure-resourcemanager-security/CHANGELOG.md +++ b/sdk/security/azure-resourcemanager-security/CHANGELOG.md @@ -1,6 +1,6 @@ # Release History -## 1.0.0-beta.6 (Unreleased) +## 1.0.0-beta.7 (Unreleased) ### Features Added @@ -10,6 +10,479 @@ ### Other Changes +## 1.0.0-beta.6 (2024-03-14) + +- Azure Resource Manager Security client library for Java. This package contains Microsoft Azure SDK for Security Management SDK. API spec for Microsoft.Security (Azure Security Center) resource provider. Package tag package-composite-v3. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). + +### Breaking Changes + +* `models.ApiCollectionOffboardings` was removed + +* `models.IngestionSettingList` was removed + +* `models.IngestionConnectionString` was removed + +* `models.Roles` was removed + +* `models.ScopeName` was removed + +* `models.IngestionSettings` was removed + +* `models.IngestionSetting` was removed + +* `models.IngestionSetting$Definition` was removed + +* `models.ApiCollectionResponse` was removed + +* `models.SecurityContactPropertiesAlertNotifications` was removed + +* `models.IngestionSettingToken` was removed + +* `models.ApiCollectionOnboardings` was removed + +* `models.HealthReportOperations` was removed + +* `models.ApiCollectionResponseList` was removed + +* `models.IngestionSetting$DefinitionStages` was removed + +* `models.ConnectionStrings` was removed + +#### `models.Extension` was modified + +* `models.OperationStatus operationStatus()` -> `models.OperationStatusAutoGenerated operationStatus()` + +#### `models.SecurityContacts` was modified + +* `delete(java.lang.String)` was removed +* `get(java.lang.String)` was removed +* `define(java.lang.String)` was removed +* `deleteWithResponse(java.lang.String,com.azure.core.util.Context)` was removed +* `getWithResponse(java.lang.String,com.azure.core.util.Context)` was removed + +#### `SecurityManager` was modified + +* `apiCollectionOffboardings()` was removed +* `apiCollectionOnboardings()` was removed +* `ingestionSettings()` was removed +* `healthReportOperations()` was removed + +#### `models.ApiCollections` was modified + +* `list(java.lang.String,java.lang.String)` was removed +* `list(java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed +* `get(java.lang.String,java.lang.String,java.lang.String)` was removed +* `getWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was removed + +#### `models.SecurityContact$Definition` was modified + +* `withAlertNotifications(models.SecurityContactPropertiesAlertNotifications)` was removed + +#### `models.Pricings` was modified + +* `getWithResponse(java.lang.String,com.azure.core.util.Context)` was removed +* `get(java.lang.String)` was removed +* `updateWithResponse(java.lang.String,fluent.models.PricingInner,com.azure.core.util.Context)` was removed +* `listWithResponse(com.azure.core.util.Context)` was removed +* `update(java.lang.String,fluent.models.PricingInner)` was removed +* `list()` was removed + +#### `models.Settings` was modified + +* `getWithResponse(models.SettingName,com.azure.core.util.Context)` was removed +* `updateWithResponse(models.SettingName,fluent.models.SettingInner,com.azure.core.util.Context)` was removed +* `get(models.SettingName)` was removed +* `update(models.SettingName,fluent.models.SettingInner)` was removed + +#### `models.HealthDataClassification` was modified + +* `models.ScopeName scope()` -> `java.lang.String scope()` +* `withScope(models.ScopeName)` was removed + +#### `models.OperationStatus` was modified + +* `withCode(models.Code)` was removed +* `models.Code code()` -> `java.lang.String code()` + +#### `models.SecurityContact` was modified + +* `alertNotifications()` was removed + +### Features Added + +* `models.OnboardingState` was added + +* `models.NotificationsSourceAlert` was added + +* `models.ServerVulnerabilityAssessmentsAzureSettingSelectedProvider` was added + +* `models.DevOpsProvisioningState` was added + +* `models.AzureDevOpsRepository$UpdateStages` was added + +* `models.AzureDevOpsProject$UpdateStages` was added + +* `models.DefenderForStorageSetting$Definition` was added + +* `models.AzureDevOpsRepository$Definition` was added + +* `models.DefenderForContainersAwsOfferingMdcContainersImageAssessment` was added + +* `models.DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S` was added + +* `models.ActionableRemediationState` was added + +* `models.GitHubRepositoryProperties` was added + +* `models.SensitivitySettings` was added + +* `models.SecurityContactRole` was added + +* `models.AzureDevOpsProject$Update` was added + +* `models.DefenderCspmGcpOfferingMdcContainersImageAssessment` was added + +* `models.AzureDevOpsProjects` was added + +* `models.DevOpsConfigurationProperties` was added + +* `models.AzureDevOpsOrgProperties` was added + +* `models.GitLabProject` was added + +* `models.NotificationsSourceAttackPath` was added + +* `models.GitLabProjects` was added + +* `models.GitHubOwnerProperties` was added + +* `models.Label` was added + +* `models.DefenderForStorageSetting$DefinitionStages` was added + +* `models.DefenderCspmGcpOfferingVmScanners` was added + +* `models.ServerVulnerabilityAssessmentsSettingKindName` was added + +* `models.AzureDevOpsRepository$DefinitionStages` was added + +* `models.AzureDevOpsOrgListResponse` was added + +* `models.AnnotateDefaultBranchState` was added + +* `models.ActionableRemediation` was added + +* `models.OperationStatusResult` was added + +* `models.GitLabSubgroups` was added + +* `models.GetSensitivitySettingsListResponse` was added + +* `models.AzureDevOpsRepository$Update` was added + +* `models.Enforce` was added + +* `models.AzureDevOpsRepos` was added + +* `models.GitLabGroup` was added + +* `models.Inherited` was added + +* `models.AzureDevOpsRepositoryListResponse` was added + +* `models.GitHubRepository` was added + +* `models.AzureServersSetting` was added + +* `models.ServerVulnerabilityAssessmentsSettings` was added + +* `models.SecurityContactName` was added + +* `models.ApiCollectionList` was added + +* `models.DefenderCspmGcpOfferingMdcContainersAgentlessDiscoveryK8S` was added + +* `models.GitHubOwners` was added + +* `models.SourceType` was added + +* `models.DevOpsOperationResults` was added + +* `models.DevOpsConfigurationListResponse` was added + +* `models.ServerVulnerabilityAssessmentsSettingsList` was added + +* `models.GetSensitivitySettingsResponseProperties` was added + +* `models.GitLabProjectListResponse` was added + +* `models.GitLabGroupProperties` was added + +* `models.ResourcesCoverageStatus` was added + +* `models.ServerVulnerabilityAssessmentsSettingKind` was added + +* `models.DefenderForStorages` was added + +* `models.GitHubOwner` was added + +* `models.MipIntegrationStatus` was added + +* `models.AzureDevOpsProject` was added + +* `models.AzureDevOpsOrg$Definition` was added + +* `models.DefenderForContainersGcpOfferingMdcContainersImageAssessment` was added + +* `models.ResourceProviders` was added + +* `models.AzureDevOpsProject$Definition` was added + +* `models.GitLabGroupListResponse` was added + +* `models.AutomationUpdateModel` was added + +* `models.DefenderCspmAwsOfferingCiemOidc` was added + +* `models.UpdateSensitivitySettingsRequest` was added + +* `models.DefenderCspmGcpOfferingVmScannersConfiguration` was added + +* `models.DevOpsConfigurations` was added + +* `models.DefenderCspmAwsOfferingCiem` was added + +* `models.AzureDevOpsOrg$UpdateStages` was added + +* `models.AzureDevOpsOrg` was added + +* `models.DefenderCspmGcpOfferingDataSensitivityDiscovery` was added + +* `models.AzureDevOpsRepositoryProperties` was added + +* `models.Authorization` was added + +* `models.AzureDevOpsProjectListResponse` was added + +* `models.GitHubRepositoryListResponse` was added + +* `models.AzureDevOpsRepository` was added + +* `models.GitHubRepos` was added + +* `models.CategoryConfiguration` was added + +* `models.DefenderCspmAwsOfferingCiemDiscovery` was added + +* `models.SettingNameAutoGenerated` was added + +* `models.RuleCategory` was added + +* `models.BuiltInInfoType` was added + +* `models.GitLabProjectProperties` was added + +* `models.ServerVulnerabilityAssessmentsSetting` was added + +* `models.DefenderForStorageSetting` was added + +* `models.GitLabGroups` was added + +* `models.OperationStatusAutoGenerated` was added + +* `models.DefenderCspmGcpOfferingCiemDiscovery` was added + +* `models.ApiCollection` was added + +* `models.InfoType` was added + +* `models.AutoDiscovery` was added + +* `models.GitHubOwnerListResponse` was added + +* `models.InheritFromParentState` was added + +* `models.GetSensitivitySettingsResponse` was added + +* `models.MinimalRiskLevel` was added + +* `models.AzureDevOpsOrg$Update` was added + +* `models.GetSensitivitySettingsResponsePropertiesMipInformation` was added + +* `models.TargetBranchConfiguration` was added + +* `models.AzureDevOpsProjectProperties` was added + +* `models.DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S` was added + +* `models.DefenderCspmAwsOfferingMdcContainersImageAssessment` was added + +* `models.AzureDevOpsOrg$DefinitionStages` was added + +* `models.DevOpsConfiguration` was added + +* `models.AzureDevOpsProject$DefinitionStages` was added + +* `models.AzureDevOpsOrgs` was added + +* `models.DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S` was added + +* `models.NotificationsSource` was added + +#### `models.GcpProjectEnvironmentData` was modified + +* `scanInterval()` was added +* `withScanInterval(java.lang.Long)` was added + +#### `models.SecurityContactList` was modified + +* `withValue(java.util.List)` was added + +#### `models.Pricing` was modified + +* `resourcesCoverageStatus()` was added +* `inherited()` was added +* `enforce()` was added +* `inheritedFrom()` was added + +#### `models.HealthReport` was modified + +* `affectedDefendersSubPlans()` was added +* `reportAdditionalData()` was added + +#### `models.HealthReports` was modified + +* `get(java.lang.String,java.lang.String)` was added +* `getWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added + +#### `models.SecurityContacts` was modified + +* `define(models.SecurityContactName)` was added +* `getWithResponse(models.SecurityContactName,com.azure.core.util.Context)` was added +* `get(models.SecurityContactName)` was added +* `deleteWithResponse(models.SecurityContactName,com.azure.core.util.Context)` was added +* `delete(models.SecurityContactName)` was added + +#### `models.AutomationActionEventHub` was modified + +* `isTrustedServiceEnabled()` was added +* `withIsTrustedServiceEnabled(java.lang.Boolean)` was added + +#### `models.AwsEnvironmentData` was modified + +* `scanInterval()` was added +* `withScanInterval(java.lang.Long)` was added + +#### `SecurityManager` was modified + +* `serverVulnerabilityAssessmentsSettings()` was added +* `azureDevOpsOrgs()` was added +* `devOpsOperationResults()` was added +* `azureDevOpsProjects()` was added +* `devOpsConfigurations()` was added +* `gitLabProjects()` was added +* `gitHubRepos()` was added +* `azureDevOpsRepos()` was added +* `gitLabSubgroups()` was added +* `resourceProviders()` was added +* `sensitivitySettings()` was added +* `defenderForStorages()` was added +* `gitHubOwners()` was added +* `gitLabGroups()` was added + +#### `models.ApiCollections` was modified + +* `onboardAzureApiManagementApi(java.lang.String,java.lang.String,java.lang.String)` was added +* `listByResourceGroup(java.lang.String)` was added +* `list(com.azure.core.util.Context)` was added +* `getByAzureApiManagementService(java.lang.String,java.lang.String,java.lang.String)` was added +* `offboardAzureApiManagementApi(java.lang.String,java.lang.String,java.lang.String)` was added +* `offboardAzureApiManagementApiWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `onboardAzureApiManagementApi(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listByAzureApiManagementService(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listByAzureApiManagementService(java.lang.String,java.lang.String)` was added +* `list()` was added +* `getByAzureApiManagementServiceWithResponse(java.lang.String,java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `listByResourceGroup(java.lang.String,com.azure.core.util.Context)` was added + +#### `models.StatusAutoGenerated` was modified + +* `lastScannedDate()` was added +* `reason()` was added + +#### `models.SecurityContact$Definition` was modified + +* `withNotificationsSources(java.util.List)` was added +* `withIsEnabled(java.lang.Boolean)` was added + +#### `models.DefenderForContainersAwsOffering` was modified + +* `mdcContainersImageAssessment()` was added +* `withMdcContainersAgentlessDiscoveryK8S(models.DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S)` was added +* `withMdcContainersImageAssessment(models.DefenderForContainersAwsOfferingMdcContainersImageAssessment)` was added +* `mdcContainersAgentlessDiscoveryK8S()` was added + +#### `models.Pricings` was modified + +* `listWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `deleteByResourceGroup(java.lang.String,java.lang.String)` was added +* `deleteByResourceGroupWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `list(java.lang.String)` was added +* `update(java.lang.String,java.lang.String,fluent.models.PricingInner)` was added +* `get(java.lang.String,java.lang.String)` was added +* `getWithResponse(java.lang.String,java.lang.String,com.azure.core.util.Context)` was added +* `updateWithResponse(java.lang.String,java.lang.String,fluent.models.PricingInner,com.azure.core.util.Context)` was added + +#### `models.Settings` was modified + +* `get(models.SettingNameAutoGenerated)` was added +* `getWithResponse(models.SettingNameAutoGenerated,com.azure.core.util.Context)` was added +* `update(models.SettingNameAutoGenerated,fluent.models.SettingInner)` was added +* `updateWithResponse(models.SettingNameAutoGenerated,fluent.models.SettingInner,com.azure.core.util.Context)` was added + +#### `models.DefenderForContainersGcpOffering` was modified + +* `withMdcContainersImageAssessment(models.DefenderForContainersGcpOfferingMdcContainersImageAssessment)` was added +* `mdcContainersAgentlessDiscoveryK8S()` was added +* `withMdcContainersAgentlessDiscoveryK8S(models.DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S)` was added +* `mdcContainersImageAssessment()` was added + +#### `models.DefenderCspmAwsOffering` was modified + +* `mdcContainersAgentlessDiscoveryK8S()` was added +* `mdcContainersImageAssessment()` was added +* `withCiem(models.DefenderCspmAwsOfferingCiem)` was added +* `ciem()` was added +* `withMdcContainersImageAssessment(models.DefenderCspmAwsOfferingMdcContainersImageAssessment)` was added +* `withMdcContainersAgentlessDiscoveryK8S(models.DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S)` was added + +#### `models.HealthDataClassification` was modified + +* `withScope(java.lang.String)` was added + +#### `models.DefenderCspmGcpOffering` was modified + +* `ciemDiscovery()` was added +* `mdcContainersAgentlessDiscoveryK8S()` was added +* `withVmScanners(models.DefenderCspmGcpOfferingVmScanners)` was added +* `withCiemDiscovery(models.DefenderCspmGcpOfferingCiemDiscovery)` was added +* `withMdcContainersAgentlessDiscoveryK8S(models.DefenderCspmGcpOfferingMdcContainersAgentlessDiscoveryK8S)` was added +* `withDataSensitivityDiscovery(models.DefenderCspmGcpOfferingDataSensitivityDiscovery)` was added +* `vmScanners()` was added +* `withMdcContainersImageAssessment(models.DefenderCspmGcpOfferingMdcContainersImageAssessment)` was added +* `mdcContainersImageAssessment()` was added +* `dataSensitivityDiscovery()` was added + +#### `models.OperationStatus` was modified + +* `withCode(java.lang.String)` was added + +#### `models.SecurityContact` was modified + +* `notificationsSources()` was added +* `isEnabled()` was added + ## 1.0.0-beta.5 (2023-04-18) - Azure Resource Manager Security client library for Java. This package contains Microsoft Azure SDK for Security Management SDK. API spec for Microsoft.Security (Azure Security Center) resource provider. Package tag package-composite-v3. For documentation on how to use this package, please see [Azure Management Libraries for Java](https://aka.ms/azsdk/java/mgmt). diff --git a/sdk/security/azure-resourcemanager-security/README.md b/sdk/security/azure-resourcemanager-security/README.md index 960b33cd33f3f..53cd81088b649 100644 --- a/sdk/security/azure-resourcemanager-security/README.md +++ b/sdk/security/azure-resourcemanager-security/README.md @@ -32,7 +32,7 @@ Various documentation is available to help you get started com.azure.resourcemanager azure-resourcemanager-security - 1.0.0-beta.5 + 1.0.0-beta.6 ``` [//]: # ({x-version-update-end}) @@ -45,7 +45,7 @@ Azure Management Libraries require a `TokenCredential` implementation for authen ### Authentication -By default, Azure Active Directory token authentication depends on correct configuration of the following environment variables. +By default, Microsoft Entra ID token authentication depends on correct configuration of the following environment variables. - `AZURE_CLIENT_ID` for Azure client ID. - `AZURE_TENANT_ID` for Azure tenant ID. @@ -94,7 +94,7 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [survey]: https://microsoft.qualtrics.com/jfe/form/SV_ehN0lIk2FKEBkwd?Q_CHL=DOCS [docs]: https://azure.github.io/azure-sdk-for-java/ -[jdk]: https://docs.microsoft.com/java/azure/jdk/ +[jdk]: https://learn.microsoft.com/azure/developer/java/fundamentals/ [azure_subscription]: https://azure.microsoft.com/free/ [azure_identity]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/identity/azure-identity [azure_core_http_netty]: https://github.com/Azure/azure-sdk-for-java/blob/main/sdk/core/azure-core-http-netty @@ -103,3 +103,5 @@ This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For m [cg]: https://github.com/Azure/azure-sdk-for-java/blob/main/CONTRIBUTING.md [coc]: https://opensource.microsoft.com/codeofconduct/ [coc_faq]: https://opensource.microsoft.com/codeofconduct/faq/ + +![Impressions](https://azure-sdk-impressions.azurewebsites.net/api/impressions/azure-sdk-for-java%2Fsdk%2Fsecurity%2Fazure-resourcemanager-security%2FREADME.png) diff --git a/sdk/security/azure-resourcemanager-security/SAMPLE.md b/sdk/security/azure-resourcemanager-security/SAMPLE.md index 3972495626d94..ecf7fe4c229c5 100644 --- a/sdk/security/azure-resourcemanager-security/SAMPLE.md +++ b/sdk/security/azure-resourcemanager-security/SAMPLE.md @@ -50,18 +50,14 @@ - [List](#allowedconnections_list) - [ListByHomeRegion](#allowedconnections_listbyhomeregion) -## ApiCollection +## ApiCollections -- [Get](#apicollection_get) -- [List](#apicollection_list) - -## ApiCollectionOffboarding - -- [Delete](#apicollectionoffboarding_delete) - -## ApiCollectionOnboarding - -- [Create](#apicollectiononboarding_create) +- [GetByAzureApiManagementService](#apicollections_getbyazureapimanagementservice) +- [List](#apicollections_list) +- [ListByAzureApiManagementService](#apicollections_listbyazureapimanagementservice) +- [ListByResourceGroup](#apicollections_listbyresourcegroup) +- [OffboardAzureApiManagementApi](#apicollections_offboardazureapimanagementapi) +- [OnboardAzureApiManagementApi](#apicollections_onboardazureapimanagementapi) ## ApplicationOperation @@ -102,8 +98,31 @@ - [GetByResourceGroup](#automations_getbyresourcegroup) - [List](#automations_list) - [ListByResourceGroup](#automations_listbyresourcegroup) +- [Update](#automations_update) - [Validate](#automations_validate) +## AzureDevOpsOrgs + +- [CreateOrUpdate](#azuredevopsorgs_createorupdate) +- [Get](#azuredevopsorgs_get) +- [List](#azuredevopsorgs_list) +- [ListAvailable](#azuredevopsorgs_listavailable) +- [Update](#azuredevopsorgs_update) + +## AzureDevOpsProjects + +- [CreateOrUpdate](#azuredevopsprojects_createorupdate) +- [Get](#azuredevopsprojects_get) +- [List](#azuredevopsprojects_list) +- [Update](#azuredevopsprojects_update) + +## AzureDevOpsRepos + +- [CreateOrUpdate](#azuredevopsrepos_createorupdate) +- [Get](#azuredevopsrepos_get) +- [List](#azuredevopsrepos_list) +- [Update](#azuredevopsrepos_update) + ## ComplianceResults - [Get](#complianceresults_get) @@ -137,6 +156,23 @@ - [List](#customentitystoreassignments_list) - [ListByResourceGroup](#customentitystoreassignments_listbyresourcegroup) +## DefenderForStorage + +- [Create](#defenderforstorage_create) +- [Get](#defenderforstorage_get) + +## DevOpsConfigurations + +- [CreateOrUpdate](#devopsconfigurations_createorupdate) +- [Delete](#devopsconfigurations_delete) +- [Get](#devopsconfigurations_get) +- [List](#devopsconfigurations_list) +- [Update](#devopsconfigurations_update) + +## DevOpsOperationResults + +- [Get](#devopsoperationresults_get) + ## DeviceSecurityGroups - [CreateOrUpdate](#devicesecuritygroups_createorupdate) @@ -156,6 +192,32 @@ - [List](#externalsecuritysolutions_list) - [ListByHomeRegion](#externalsecuritysolutions_listbyhomeregion) +## GitHubOwners + +- [Get](#githubowners_get) +- [List](#githubowners_list) +- [ListAvailable](#githubowners_listavailable) + +## GitHubRepos + +- [Get](#githubrepos_get) +- [List](#githubrepos_list) + +## GitLabGroups + +- [Get](#gitlabgroups_get) +- [List](#gitlabgroups_list) +- [ListAvailable](#gitlabgroups_listavailable) + +## GitLabProjects + +- [Get](#gitlabprojects_get) +- [List](#gitlabprojects_list) + +## GitLabSubgroups + +- [List](#gitlabsubgroups_list) + ## GovernanceAssignments - [CreateOrUpdate](#governanceassignments_createorupdate) @@ -172,12 +234,9 @@ - [List](#governancerules_list) - [OperationResults](#governancerules_operationresults) -## HealthReportOperation - -- [Get](#healthreportoperation_get) - ## HealthReports +- [Get](#healthreports_get) - [List](#healthreports_list) ## InformationProtectionPolicies @@ -186,15 +245,6 @@ - [Get](#informationprotectionpolicies_get) - [List](#informationprotectionpolicies_list) -## IngestionSettings - -- [Create](#ingestionsettings_create) -- [Delete](#ingestionsettings_delete) -- [Get](#ingestionsettings_get) -- [List](#ingestionsettings_list) -- [ListConnectionStrings](#ingestionsettings_listconnectionstrings) -- [ListTokens](#ingestionsettings_listtokens) - ## IotSecuritySolution - [CreateOrUpdate](#iotsecuritysolution_createorupdate) @@ -243,6 +293,7 @@ ## Pricings +- [Delete](#pricings_delete) - [Get](#pricings_get) - [List](#pricings_list) - [Update](#pricings_update) @@ -262,6 +313,11 @@ - [Get](#regulatorycompliancestandards_get) - [List](#regulatorycompliancestandards_list) +## ResourceProvider + +- [GetSensitivitySettings](#resourceprovider_getsensitivitysettings) +- [UpdateSensitivitySettings](#resourceprovider_updatesensitivitysettings) + ## SecureScoreControlDefinitions - [List](#securescorecontroldefinitions_list) @@ -320,6 +376,10 @@ - [List](#securitysolutionsreferencedata_list) - [ListByHomeRegion](#securitysolutionsreferencedata_listbyhomeregion) +## SensitivitySettings + +- [List](#sensitivitysettings_list) + ## ServerVulnerabilityAssessment - [CreateOrUpdate](#servervulnerabilityassessment_createorupdate) @@ -327,6 +387,13 @@ - [Get](#servervulnerabilityassessment_get) - [ListByExtendedResource](#servervulnerabilityassessment_listbyextendedresource) +## ServerVulnerabilityAssessmentsSettings + +- [CreateOrUpdate](#servervulnerabilityassessmentssettings_createorupdate) +- [Delete](#servervulnerabilityassessmentssettings_delete) +- [Get](#servervulnerabilityassessmentssettings_get) +- [List](#servervulnerabilityassessmentssettings_list) + ## Settings - [Get](#settings_get) @@ -389,21 +456,24 @@ ### AdaptiveApplicationControls_Delete ```java -/** Samples for AdaptiveApplicationControls Delete. */ +/** + * Samples for AdaptiveApplicationControls Delete. + */ public final class AdaptiveApplicationControlsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/DeleteAdaptiveApplicationControls_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/ + * DeleteAdaptiveApplicationControls_example.json */ /** * Sample code: Delete an application control machine group. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteAnApplicationControlMachineGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .adaptiveApplicationControls() - .deleteByResourceGroupWithResponse("centralus", "GROUP1", com.azure.core.util.Context.NONE); + public static void + deleteAnApplicationControlMachineGroup(com.azure.resourcemanager.security.SecurityManager manager) { + manager.adaptiveApplicationControls().deleteByResourceGroupWithResponse("centralus", "GROUP1", + com.azure.core.util.Context.NONE); } } ``` @@ -411,21 +481,24 @@ public final class AdaptiveApplicationControlsDeleteSamples { ### AdaptiveApplicationControls_Get ```java -/** Samples for AdaptiveApplicationControls Get. */ +/** + * Samples for AdaptiveApplicationControls Get. + */ public final class AdaptiveApplicationControlsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/GetAdaptiveApplicationControlsGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/ + * GetAdaptiveApplicationControlsGroup_example.json */ /** * Sample code: Gets a configured application control VM/server group. - * + * * @param manager Entry point to SecurityManager. */ - public static void getsAConfiguredApplicationControlVMServerGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .adaptiveApplicationControls() - .getWithResponse("centralus", "ERELGROUP1", com.azure.core.util.Context.NONE); + public static void + getsAConfiguredApplicationControlVMServerGroup(com.azure.resourcemanager.security.SecurityManager manager) { + manager.adaptiveApplicationControls().getWithResponse("centralus", "ERELGROUP1", + com.azure.core.util.Context.NONE); } } ``` @@ -433,14 +506,18 @@ public final class AdaptiveApplicationControlsGetSamples { ### AdaptiveApplicationControls_List ```java -/** Samples for AdaptiveApplicationControls List. */ +/** + * Samples for AdaptiveApplicationControls List. + */ public final class AdaptiveApplicationControlsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/GetAdaptiveApplicationControlsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/ + * GetAdaptiveApplicationControlsSubscription_example.json */ /** * Sample code: Gets a list of application control groups of machines for the subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getsAListOfApplicationControlGroupsOfMachinesForTheSubscription( @@ -467,114 +544,70 @@ import com.azure.resourcemanager.security.models.UserRecommendation; import com.azure.resourcemanager.security.models.VmRecommendation; import java.util.Arrays; -/** Samples for AdaptiveApplicationControls Put. */ +/** + * Samples for AdaptiveApplicationControls Put. + */ public final class AdaptiveApplicationControlsPutSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/PutAdaptiveApplicationControls_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ApplicationWhitelistings/ + * PutAdaptiveApplicationControls_example.json */ /** * Sample code: Update an application control machine group by adding a new application. - * + * * @param manager Entry point to SecurityManager. */ public static void updateAnApplicationControlMachineGroupByAddingANewApplication( com.azure.resourcemanager.security.SecurityManager manager) { - AdaptiveApplicationControlGroup resource = - manager - .adaptiveApplicationControls() - .getWithResponse("centralus", "ERELGROUP1", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withEnforcementMode(EnforcementMode.AUDIT) - .withProtectionMode( - new ProtectionMode() - .withExe(EnforcementMode.AUDIT) - .withMsi(EnforcementMode.NONE) - .withScript(EnforcementMode.NONE)) - .withVmRecommendations( - Arrays - .asList( - new VmRecommendation() - .withConfigurationStatus(ConfigurationStatus.CONFIGURED) - .withRecommendationAction(RecommendationAction.RECOMMENDED) - .withResourceId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourcegroups/erelh-stable/providers/microsoft.compute/virtualmachines/erelh-16090") - .withEnforcementSupport(EnforcementSupport.SUPPORTED), - new VmRecommendation() - .withConfigurationStatus(ConfigurationStatus.CONFIGURED) - .withRecommendationAction(RecommendationAction.RECOMMENDED) - .withResourceId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourcegroups/matanvs/providers/microsoft.compute/virtualmachines/matanvs19") - .withEnforcementSupport(EnforcementSupport.SUPPORTED))) - .withPathRecommendations( - Arrays - .asList( - new PathRecommendation() - .withPath("[Exe] O=MICROSOFT CORPORATION, L=REDMOND, S=WASHINGTON, C=US\\*\\*\\0.0.0.0") - .withAction(RecommendationAction.RECOMMENDED) - .withType(RecommendationType.fromString("PublisherSignature")) - .withPublisherInfo( - new PublisherInfo() - .withPublisherName("O=MICROSOFT CORPORATION, L=REDMOND, S=WASHINGTON, C=US") - .withProductName("*") - .withBinaryName("*") - .withVersion("0.0.0.0")) - .withCommon(true) - .withUserSids(Arrays.asList("S-1-1-0")) - .withUsernames( - Arrays - .asList( - new UserRecommendation() - .withUsername("Everyone") - .withRecommendationAction(RecommendationAction.RECOMMENDED))) - .withFileType(FileType.EXE) - .withConfigurationStatus(ConfigurationStatus.CONFIGURED), - new PathRecommendation() - .withPath("%OSDRIVE%\\WINDOWSAZURE\\SECAGENT\\WASECAGENTPROV.EXE") - .withAction(RecommendationAction.RECOMMENDED) - .withType(RecommendationType.fromString("ProductSignature")) - .withPublisherInfo( - new PublisherInfo() - .withPublisherName("CN=MICROSOFT AZURE DEPENDENCY CODE SIGN") - .withProductName("MICROSOFT® COREXT") - .withBinaryName("*") - .withVersion("0.0.0.0")) - .withCommon(true) - .withUserSids(Arrays.asList("S-1-1-0")) - .withUsernames( - Arrays - .asList( - new UserRecommendation() - .withUsername("NT AUTHORITY\\SYSTEM") - .withRecommendationAction(RecommendationAction.RECOMMENDED))) - .withFileType(FileType.EXE) - .withConfigurationStatus(ConfigurationStatus.CONFIGURED), - new PathRecommendation() - .withPath("%OSDRIVE%\\WINDOWSAZURE\\PACKAGES_201973_7415\\COLLECTGUESTLOGS.EXE") - .withAction(RecommendationAction.RECOMMENDED) - .withType(RecommendationType.fromString("PublisherSignature")) - .withPublisherInfo( - new PublisherInfo() - .withPublisherName("CN=MICROSOFT AZURE DEPENDENCY CODE SIGN") - .withProductName("*") - .withBinaryName("*") - .withVersion("0.0.0.0")) - .withCommon(true) - .withUserSids(Arrays.asList("S-1-1-0")) - .withUsernames( - Arrays - .asList( - new UserRecommendation() - .withUsername("NT AUTHORITY\\SYSTEM") - .withRecommendationAction(RecommendationAction.RECOMMENDED))) - .withFileType(FileType.EXE) - .withConfigurationStatus(ConfigurationStatus.CONFIGURED), - new PathRecommendation() - .withPath("C:\\directory\\file.exe") - .withAction(RecommendationAction.ADD) - .withType(RecommendationType.fromString("File")) - .withCommon(true))) + AdaptiveApplicationControlGroup resource = manager.adaptiveApplicationControls() + .getWithResponse("centralus", "ERELGROUP1", com.azure.core.util.Context.NONE).getValue(); + resource.update().withEnforcementMode(EnforcementMode.AUDIT) + .withProtectionMode(new ProtectionMode().withExe(EnforcementMode.AUDIT).withMsi(EnforcementMode.NONE) + .withScript(EnforcementMode.NONE)) + .withVmRecommendations(Arrays.asList(new VmRecommendation() + .withConfigurationStatus(ConfigurationStatus.CONFIGURED) + .withRecommendationAction(RecommendationAction.RECOMMENDED) + .withResourceId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourcegroups/erelh-stable/providers/microsoft.compute/virtualmachines/erelh-16090") + .withEnforcementSupport(EnforcementSupport.SUPPORTED), + new VmRecommendation().withConfigurationStatus(ConfigurationStatus.CONFIGURED) + .withRecommendationAction(RecommendationAction.RECOMMENDED) + .withResourceId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourcegroups/matanvs/providers/microsoft.compute/virtualmachines/matanvs19") + .withEnforcementSupport(EnforcementSupport.SUPPORTED))) + .withPathRecommendations(Arrays.asList( + new PathRecommendation() + .withPath("[Exe] O=MICROSOFT CORPORATION, L=REDMOND, S=WASHINGTON, C=US\\*\\*\\0.0.0.0") + .withAction(RecommendationAction.RECOMMENDED) + .withType(RecommendationType.fromString("PublisherSignature")) + .withPublisherInfo( + new PublisherInfo().withPublisherName("O=MICROSOFT CORPORATION, L=REDMOND, S=WASHINGTON, C=US") + .withProductName("*").withBinaryName("*").withVersion("0.0.0.0")) + .withCommon(true).withUserSids(Arrays.asList("S-1-1-0")) + .withUsernames(Arrays.asList(new UserRecommendation().withUsername("Everyone") + .withRecommendationAction(RecommendationAction.RECOMMENDED))) + .withFileType(FileType.EXE).withConfigurationStatus(ConfigurationStatus.CONFIGURED), + new PathRecommendation().withPath("%OSDRIVE%\\WINDOWSAZURE\\SECAGENT\\WASECAGENTPROV.EXE") + .withAction(RecommendationAction.RECOMMENDED) + .withType(RecommendationType.fromString("ProductSignature")) + .withPublisherInfo(new PublisherInfo().withPublisherName("CN=MICROSOFT AZURE DEPENDENCY CODE SIGN") + .withProductName("MICROSOFT® COREXT").withBinaryName("*").withVersion("0.0.0.0")) + .withCommon(true).withUserSids(Arrays.asList("S-1-1-0")) + .withUsernames(Arrays.asList(new UserRecommendation().withUsername("NT AUTHORITY\\SYSTEM") + .withRecommendationAction(RecommendationAction.RECOMMENDED))) + .withFileType(FileType.EXE).withConfigurationStatus(ConfigurationStatus.CONFIGURED), + new PathRecommendation().withPath("%OSDRIVE%\\WINDOWSAZURE\\PACKAGES_201973_7415\\COLLECTGUESTLOGS.EXE") + .withAction(RecommendationAction.RECOMMENDED) + .withType(RecommendationType.fromString("PublisherSignature")) + .withPublisherInfo(new PublisherInfo().withPublisherName("CN=MICROSOFT AZURE DEPENDENCY CODE SIGN") + .withProductName("*").withBinaryName("*").withVersion("0.0.0.0")) + .withCommon(true).withUserSids(Arrays.asList("S-1-1-0")) + .withUsernames(Arrays.asList(new UserRecommendation().withUsername("NT AUTHORITY\\SYSTEM") + .withRecommendationAction(RecommendationAction.RECOMMENDED))) + .withFileType(FileType.EXE).withConfigurationStatus(ConfigurationStatus.CONFIGURED), + new PathRecommendation().withPath("C:\\directory\\file.exe").withAction(RecommendationAction.ADD) + .withType(RecommendationType.fromString("File")).withCommon(true))) .apply(); } } @@ -589,48 +622,34 @@ import com.azure.resourcemanager.security.models.Rule; import com.azure.resourcemanager.security.models.TransportProtocol; import java.util.Arrays; -/** Samples for AdaptiveNetworkHardenings Enforce. */ +/** + * Samples for AdaptiveNetworkHardenings Enforce. + */ public final class AdaptiveNetworkHardeningsEnforceSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AdaptiveNetworkHardenings/EnforceAdaptiveNetworkHardeningRules_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AdaptiveNetworkHardenings/ + * EnforceAdaptiveNetworkHardeningRules_example.json */ /** * Sample code: Enforces the given rules on the NSG(s) listed in the request. - * + * * @param manager Entry point to SecurityManager. */ - public static void enforcesTheGivenRulesOnTheNSGSListedInTheRequest( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .adaptiveNetworkHardenings() - .enforce( - "rg1", - "Microsoft.Compute", - "virtualMachines", - "vm1", - "default", - new AdaptiveNetworkHardeningEnforceRequest() - .withRules( - Arrays - .asList( - new Rule() - .withName("rule1") - .withDirection(Direction.INBOUND) - .withDestinationPort(3389) - .withProtocols(Arrays.asList(TransportProtocol.TCP)) - .withIpAddresses(Arrays.asList("100.10.1.1", "200.20.2.2", "81.199.3.0/24")), - new Rule() - .withName("rule2") - .withDirection(Direction.INBOUND) - .withDestinationPort(22) - .withProtocols(Arrays.asList(TransportProtocol.TCP)) - .withIpAddresses(Arrays.asList()))) - .withNetworkSecurityGroups( - Arrays - .asList( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/rg1/providers/Microsoft.Network/networkSecurityGroups/nsg1", - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/rg2/providers/Microsoft.Network/networkSecurityGroups/nsg2")), - com.azure.core.util.Context.NONE); + public static void + enforcesTheGivenRulesOnTheNSGSListedInTheRequest(com.azure.resourcemanager.security.SecurityManager manager) { + manager.adaptiveNetworkHardenings().enforce("rg1", "Microsoft.Compute", "virtualMachines", "vm1", "default", + new AdaptiveNetworkHardeningEnforceRequest() + .withRules(Arrays.asList( + new Rule().withName("rule1").withDirection(Direction.INBOUND).withDestinationPort(3389) + .withProtocols(Arrays.asList(TransportProtocol.TCP)) + .withIpAddresses(Arrays.asList("100.10.1.1", "200.20.2.2", "81.199.3.0/24")), + new Rule().withName("rule2").withDirection(Direction.INBOUND).withDestinationPort(22) + .withProtocols(Arrays.asList(TransportProtocol.TCP)).withIpAddresses(Arrays.asList()))) + .withNetworkSecurityGroups(Arrays.asList( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/rg1/providers/Microsoft.Network/networkSecurityGroups/nsg1", + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/rg2/providers/Microsoft.Network/networkSecurityGroups/nsg2")), + com.azure.core.util.Context.NONE); } } ``` @@ -638,22 +657,24 @@ public final class AdaptiveNetworkHardeningsEnforceSamples { ### AdaptiveNetworkHardenings_Get ```java -/** Samples for AdaptiveNetworkHardenings Get. */ +/** + * Samples for AdaptiveNetworkHardenings Get. + */ public final class AdaptiveNetworkHardeningsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AdaptiveNetworkHardenings/GetAdaptiveNetworkHardening_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AdaptiveNetworkHardenings/ + * GetAdaptiveNetworkHardening_example.json */ /** * Sample code: Get a single Adaptive Network Hardening resource. - * + * * @param manager Entry point to SecurityManager. */ - public static void getASingleAdaptiveNetworkHardeningResource( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .adaptiveNetworkHardenings() - .getWithResponse( - "rg1", "Microsoft.Compute", "virtualMachines", "vm1", "default", com.azure.core.util.Context.NONE); + public static void + getASingleAdaptiveNetworkHardeningResource(com.azure.resourcemanager.security.SecurityManager manager) { + manager.adaptiveNetworkHardenings().getWithResponse("rg1", "Microsoft.Compute", "virtualMachines", "vm1", + "default", com.azure.core.util.Context.NONE); } } ``` @@ -661,22 +682,24 @@ public final class AdaptiveNetworkHardeningsGetSamples { ### AdaptiveNetworkHardenings_ListByExtendedResource ```java -/** Samples for AdaptiveNetworkHardenings ListByExtendedResource. */ +/** + * Samples for AdaptiveNetworkHardenings ListByExtendedResource. + */ public final class AdaptiveNetworkHardeningsListByExtendedResourceSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AdaptiveNetworkHardenings/ListByExtendedResourceAdaptiveNetworkHardenings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AdaptiveNetworkHardenings/ + * ListByExtendedResourceAdaptiveNetworkHardenings_example.json */ /** * Sample code: List Adaptive Network Hardenings resources of an extended resource. - * + * * @param manager Entry point to SecurityManager. */ public static void listAdaptiveNetworkHardeningsResourcesOfAnExtendedResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .adaptiveNetworkHardenings() - .listByExtendedResource( - "rg1", "Microsoft.Compute", "virtualMachines", "vm1", com.azure.core.util.Context.NONE); + manager.adaptiveNetworkHardenings().listByExtendedResource("rg1", "Microsoft.Compute", "virtualMachines", "vm1", + com.azure.core.util.Context.NONE); } } ``` @@ -684,25 +707,25 @@ public final class AdaptiveNetworkHardeningsListByExtendedResourceSamples { ### AdvancedThreatProtection_Create ```java -/** Samples for AdvancedThreatProtection Create. */ +/** + * Samples for AdvancedThreatProtection Create. + */ public final class AdvancedThreatProtectionCreateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-01-01/examples/AdvancedThreatProtection/PutAdvancedThreatProtectionSettings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-01-01/examples/AdvancedThreatProtection/ + * PutAdvancedThreatProtectionSettings_example.json */ /** * Sample code: Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param manager Entry point to SecurityManager. */ public static void createsOrUpdatesTheAdvancedThreatProtectionSettingsOnASpecifiedResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .advancedThreatProtections() - .define() - .withExistingResourceId( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Storage/storageAccounts/samplestorageaccount") - .withIsEnabled(true) - .create(); + manager.advancedThreatProtections().define().withExistingResourceId( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Storage/storageAccounts/samplestorageaccount") + .withIsEnabled(true).create(); } } ``` @@ -710,23 +733,25 @@ public final class AdvancedThreatProtectionCreateSamples { ### AdvancedThreatProtection_Get ```java -/** Samples for AdvancedThreatProtection Get. */ +/** + * Samples for AdvancedThreatProtection Get. + */ public final class AdvancedThreatProtectionGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-01-01/examples/AdvancedThreatProtection/GetAdvancedThreatProtectionSettings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-01-01/examples/AdvancedThreatProtection/ + * GetAdvancedThreatProtectionSettings_example.json */ /** * Sample code: Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param manager Entry point to SecurityManager. */ public static void getsTheAdvancedThreatProtectionSettingsForTheSpecifiedResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .advancedThreatProtections() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Storage/storageAccounts/samplestorageaccount", - com.azure.core.util.Context.NONE); + manager.advancedThreatProtections().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Storage/storageAccounts/samplestorageaccount", + com.azure.core.util.Context.NONE); } } ``` @@ -734,25 +759,23 @@ public final class AdvancedThreatProtectionGetSamples { ### Alerts_GetResourceGroupLevel ```java -/** Samples for Alerts GetResourceGroupLevel. */ +/** + * Samples for Alerts GetResourceGroupLevel. + */ public final class AlertsGetResourceGroupLevelSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/GetAlertResourceGroupLocation_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * GetAlertResourceGroupLocation_example.json */ /** * Sample code: Get security alert on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAlertOnAResourceGroupFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .getResourceGroupLevelWithResponse( - "myRg1", - "westeurope", - "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", - com.azure.core.util.Context.NONE); + manager.alerts().getResourceGroupLevelWithResponse("myRg1", "westeurope", + "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", com.azure.core.util.Context.NONE); } } ``` @@ -760,24 +783,23 @@ public final class AlertsGetResourceGroupLevelSamples { ### Alerts_GetSubscriptionLevel ```java -/** Samples for Alerts GetSubscriptionLevel. */ +/** + * Samples for Alerts GetSubscriptionLevel. + */ public final class AlertsGetSubscriptionLevelSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/GetAlertSubscriptionLocation_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * GetAlertSubscriptionLocation_example.json */ /** * Sample code: Get security alert on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAlertOnASubscriptionFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .getSubscriptionLevelWithResponse( - "westeurope", - "2518770965529163669_F144EE95-A3E5-42DA-A279-967D115809AA", - com.azure.core.util.Context.NONE); + manager.alerts().getSubscriptionLevelWithResponse("westeurope", + "2518770965529163669_F144EE95-A3E5-42DA-A279-967D115809AA", com.azure.core.util.Context.NONE); } } ``` @@ -785,14 +807,17 @@ public final class AlertsGetSubscriptionLevelSamples { ### Alerts_List ```java -/** Samples for Alerts List. */ +/** + * Samples for Alerts List. + */ public final class AlertsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/GetAlertsSubscription_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * GetAlertsSubscription_example.json */ /** * Sample code: Get security alerts on a subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAlertsOnASubscription(com.azure.resourcemanager.security.SecurityManager manager) { @@ -804,14 +829,17 @@ public final class AlertsListSamples { ### Alerts_ListByResourceGroup ```java -/** Samples for Alerts ListByResourceGroup. */ +/** + * Samples for Alerts ListByResourceGroup. + */ public final class AlertsListByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/GetAlertsResourceGroup_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * GetAlertsResourceGroup_example.json */ /** * Sample code: Get security alerts on a resource group. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAlertsOnAResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { @@ -823,14 +851,17 @@ public final class AlertsListByResourceGroupSamples { ### Alerts_ListResourceGroupLevelByRegion ```java -/** Samples for Alerts ListResourceGroupLevelByRegion. */ +/** + * Samples for Alerts ListResourceGroupLevelByRegion. + */ public final class AlertsListResourceGroupLevelByRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/GetAlertsResourceGroupLocation_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * GetAlertsResourceGroupLocation_example.json */ /** * Sample code: Get security alerts on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAlertsOnAResourceGroupFromASecurityDataLocation( @@ -843,14 +874,17 @@ public final class AlertsListResourceGroupLevelByRegionSamples { ### Alerts_ListSubscriptionLevelByRegion ```java -/** Samples for Alerts ListSubscriptionLevelByRegion. */ +/** + * Samples for Alerts ListSubscriptionLevelByRegion. + */ public final class AlertsListSubscriptionLevelByRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/GetAlertsSubscriptionsLocation_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * GetAlertsSubscriptionsLocation_example.json */ /** * Sample code: Get security alerts on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAlertsOnASubscriptionFromASecurityDataLocation( @@ -868,38 +902,27 @@ import com.azure.resourcemanager.security.models.AlertSimulatorRequestBody; import com.azure.resourcemanager.security.models.BundleType; import java.util.Arrays; -/** Samples for Alerts Simulate. */ +/** + * Samples for Alerts Simulate. + */ public final class AlertsSimulateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/SimulateAlerts_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * SimulateAlerts_example.json */ /** * Sample code: Simulate security alerts on a subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void simulateSecurityAlertsOnASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .simulate( - "centralus", - new AlertSimulatorRequestBody() - .withProperties( - new AlertSimulatorBundlesRequestProperties() - .withBundles( - Arrays - .asList( - BundleType.APP_SERVICES, - BundleType.DNS, - BundleType.KEY_VAULTS, - BundleType.KUBERNETES_SERVICE, - BundleType.RESOURCE_MANAGER, - BundleType.SQL_SERVERS, - BundleType.STORAGE_ACCOUNTS, - BundleType.VIRTUAL_MACHINES, - BundleType.COSMOS_DBS))), - com.azure.core.util.Context.NONE); + public static void + simulateSecurityAlertsOnASubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.alerts().simulate("centralus", + new AlertSimulatorRequestBody().withProperties(new AlertSimulatorBundlesRequestProperties() + .withBundles(Arrays.asList(BundleType.APP_SERVICES, BundleType.DNS, BundleType.KEY_VAULTS, + BundleType.KUBERNETES_SERVICE, BundleType.RESOURCE_MANAGER, BundleType.SQL_SERVERS, + BundleType.STORAGE_ACCOUNTS, BundleType.VIRTUAL_MACHINES, BundleType.COSMOS_DBS))), + com.azure.core.util.Context.NONE); } } ``` @@ -907,25 +930,23 @@ public final class AlertsSimulateSamples { ### Alerts_UpdateResourceGroupLevelStateToActivate ```java -/** Samples for Alerts UpdateResourceGroupLevelStateToActivate. */ +/** + * Samples for Alerts UpdateResourceGroupLevelStateToActivate. + */ public final class AlertsUpdateResourceGroupLevelStateToActivateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertResourceGroupLocation_activate_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertResourceGroupLocation_activate_example.json */ /** * Sample code: Update security alert state on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnAResourceGroupFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateResourceGroupLevelStateToActivateWithResponse( - "myRg2", - "westeurope", - "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", - com.azure.core.util.Context.NONE); + manager.alerts().updateResourceGroupLevelStateToActivateWithResponse("myRg2", "westeurope", + "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", com.azure.core.util.Context.NONE); } } ``` @@ -933,25 +954,23 @@ public final class AlertsUpdateResourceGroupLevelStateToActivateSamples { ### Alerts_UpdateResourceGroupLevelStateToDismiss ```java -/** Samples for Alerts UpdateResourceGroupLevelStateToDismiss. */ +/** + * Samples for Alerts UpdateResourceGroupLevelStateToDismiss. + */ public final class AlertsUpdateResourceGroupLevelStateToDismissSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertResourceGroupLocation_dismiss_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertResourceGroupLocation_dismiss_example.json */ /** * Sample code: Update security alert state on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnAResourceGroupFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateResourceGroupLevelStateToDismissWithResponse( - "myRg2", - "westeurope", - "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", - com.azure.core.util.Context.NONE); + manager.alerts().updateResourceGroupLevelStateToDismissWithResponse("myRg2", "westeurope", + "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", com.azure.core.util.Context.NONE); } } ``` @@ -959,25 +978,23 @@ public final class AlertsUpdateResourceGroupLevelStateToDismissSamples { ### Alerts_UpdateResourceGroupLevelStateToInProgress ```java -/** Samples for Alerts UpdateResourceGroupLevelStateToInProgress. */ +/** + * Samples for Alerts UpdateResourceGroupLevelStateToInProgress. + */ public final class AlertsUpdateResourceGroupLevelStateToInProgressSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertResourceGroupLocation_inProgress_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertResourceGroupLocation_inProgress_example.json */ /** * Sample code: Update security alert state on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnAResourceGroupFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateResourceGroupLevelStateToInProgressWithResponse( - "myRg2", - "westeurope", - "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", - com.azure.core.util.Context.NONE); + manager.alerts().updateResourceGroupLevelStateToInProgressWithResponse("myRg2", "westeurope", + "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", com.azure.core.util.Context.NONE); } } ``` @@ -985,25 +1002,23 @@ public final class AlertsUpdateResourceGroupLevelStateToInProgressSamples { ### Alerts_UpdateResourceGroupLevelStateToResolve ```java -/** Samples for Alerts UpdateResourceGroupLevelStateToResolve. */ +/** + * Samples for Alerts UpdateResourceGroupLevelStateToResolve. + */ public final class AlertsUpdateResourceGroupLevelStateToResolveSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertResourceGroupLocation_resolve_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertResourceGroupLocation_resolve_example.json */ /** * Sample code: Update security alert state on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnAResourceGroupFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateResourceGroupLevelStateToResolveWithResponse( - "myRg2", - "westeurope", - "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", - com.azure.core.util.Context.NONE); + manager.alerts().updateResourceGroupLevelStateToResolveWithResponse("myRg2", "westeurope", + "2518765996949954086_2325cf9e-42a2-4f72-ae7f-9b863cba2d22", com.azure.core.util.Context.NONE); } } ``` @@ -1011,24 +1026,23 @@ public final class AlertsUpdateResourceGroupLevelStateToResolveSamples { ### Alerts_UpdateSubscriptionLevelStateToActivate ```java -/** Samples for Alerts UpdateSubscriptionLevelStateToActivate. */ +/** + * Samples for Alerts UpdateSubscriptionLevelStateToActivate. + */ public final class AlertsUpdateSubscriptionLevelStateToActivateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertSubscriptionLocation_activate_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertSubscriptionLocation_activate_example.json */ /** * Sample code: Update security alert state on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnASubscriptionFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateSubscriptionLevelStateToActivateWithResponse( - "westeurope", - "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", - com.azure.core.util.Context.NONE); + manager.alerts().updateSubscriptionLevelStateToActivateWithResponse("westeurope", + "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", com.azure.core.util.Context.NONE); } } ``` @@ -1036,24 +1050,23 @@ public final class AlertsUpdateSubscriptionLevelStateToActivateSamples { ### Alerts_UpdateSubscriptionLevelStateToDismiss ```java -/** Samples for Alerts UpdateSubscriptionLevelStateToDismiss. */ +/** + * Samples for Alerts UpdateSubscriptionLevelStateToDismiss. + */ public final class AlertsUpdateSubscriptionLevelStateToDismissSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertSubscriptionLocation_dismiss_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertSubscriptionLocation_dismiss_example.json */ /** * Sample code: Update security alert state on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnASubscriptionFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateSubscriptionLevelStateToDismissWithResponse( - "westeurope", - "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", - com.azure.core.util.Context.NONE); + manager.alerts().updateSubscriptionLevelStateToDismissWithResponse("westeurope", + "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", com.azure.core.util.Context.NONE); } } ``` @@ -1061,24 +1074,23 @@ public final class AlertsUpdateSubscriptionLevelStateToDismissSamples { ### Alerts_UpdateSubscriptionLevelStateToInProgress ```java -/** Samples for Alerts UpdateSubscriptionLevelStateToInProgress. */ +/** + * Samples for Alerts UpdateSubscriptionLevelStateToInProgress. + */ public final class AlertsUpdateSubscriptionLevelStateToInProgressSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertSubscriptionLocation_inProgress_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertSubscriptionLocation_inProgress_example.json */ /** * Sample code: Update security alert state on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnASubscriptionFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateSubscriptionLevelStateToInProgressWithResponse( - "westeurope", - "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", - com.azure.core.util.Context.NONE); + manager.alerts().updateSubscriptionLevelStateToInProgressWithResponse("westeurope", + "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", com.azure.core.util.Context.NONE); } } ``` @@ -1086,24 +1098,23 @@ public final class AlertsUpdateSubscriptionLevelStateToInProgressSamples { ### Alerts_UpdateSubscriptionLevelStateToResolve ```java -/** Samples for Alerts UpdateSubscriptionLevelStateToResolve. */ +/** + * Samples for Alerts UpdateSubscriptionLevelStateToResolve. + */ public final class AlertsUpdateSubscriptionLevelStateToResolveSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/UpdateAlertSubscriptionLocation_resolve_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-01-01/examples/Alerts/ + * UpdateAlertSubscriptionLocation_resolve_example.json */ /** * Sample code: Update security alert state on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void updateSecurityAlertStateOnASubscriptionFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .alerts() - .updateSubscriptionLevelStateToResolveWithResponse( - "westeurope", - "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", - com.azure.core.util.Context.NONE); + manager.alerts().updateSubscriptionLevelStateToResolveWithResponse("westeurope", + "2518298467986649999_4d25bfef-2d77-4a08-adc0-3e35715cc92a", com.azure.core.util.Context.NONE); } } ``` @@ -1111,18 +1122,22 @@ public final class AlertsUpdateSubscriptionLevelStateToResolveSamples { ### AlertsSuppressionRules_Delete ```java -/** Samples for AlertsSuppressionRules Delete. */ +/** + * Samples for AlertsSuppressionRules Delete. + */ public final class AlertsSuppressionRulesDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/AlertsSuppressionRules/DeleteAlertsSuppressionRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * AlertsSuppressionRules/DeleteAlertsSuppressionRule_example.json */ /** * Sample code: Delete suppression rule data for a subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteSuppressionRuleDataForASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + deleteSuppressionRuleDataForASubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.alertsSuppressionRules().deleteWithResponse("dismissIpAnomalyAlerts", com.azure.core.util.Context.NONE); } } @@ -1131,18 +1146,22 @@ public final class AlertsSuppressionRulesDeleteSamples { ### AlertsSuppressionRules_Get ```java -/** Samples for AlertsSuppressionRules Get. */ +/** + * Samples for AlertsSuppressionRules Get. + */ public final class AlertsSuppressionRulesGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/AlertsSuppressionRules/GetAlertsSuppressionRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * AlertsSuppressionRules/GetAlertsSuppressionRule_example.json */ /** * Sample code: Get suppression alert rule for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSuppressionAlertRuleForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getSuppressionAlertRuleForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.alertsSuppressionRules().getWithResponse("dismissIpAnomalyAlerts", com.azure.core.util.Context.NONE); } } @@ -1151,14 +1170,18 @@ public final class AlertsSuppressionRulesGetSamples { ### AlertsSuppressionRules_List ```java -/** Samples for AlertsSuppressionRules List. */ +/** + * Samples for AlertsSuppressionRules List. + */ public final class AlertsSuppressionRulesListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/AlertsSuppressionRules/GetAlertsSuppressionRules_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * AlertsSuppressionRules/GetAlertsSuppressionRules_example.json */ /** * Sample code: Get suppression rules for subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getSuppressionRulesForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { @@ -1166,11 +1189,13 @@ public final class AlertsSuppressionRulesListSamples { } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/AlertsSuppressionRules/GetAlertsSuppressionRulesWithAlertType_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * AlertsSuppressionRules/GetAlertsSuppressionRulesWithAlertType_example.json */ /** * Sample code: Get suppression alert rule for subscription, filtered by AlertType. - * + * * @param manager Entry point to SecurityManager. */ public static void getSuppressionAlertRuleForSubscriptionFilteredByAlertType( @@ -1195,50 +1220,37 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for AlertsSuppressionRules Update. */ +/** + * Samples for AlertsSuppressionRules Update. + */ public final class AlertsSuppressionRulesUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/AlertsSuppressionRules/PutAlertsSuppressionRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * AlertsSuppressionRules/PutAlertsSuppressionRule_example.json */ /** * Sample code: Update or create suppression rule for subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void updateOrCreateSuppressionRuleForSubscription( com.azure.resourcemanager.security.SecurityManager manager) throws IOException { - manager - .alertsSuppressionRules() - .updateWithResponse( - "dismissIpAnomalyAlerts", - new AlertsSuppressionRuleInner() - .withAlertType("IpAnomaly") - .withExpirationDateUtc(OffsetDateTime.parse("2019-12-01T19:50:47.083633Z")) - .withReason("FalsePositive") - .withState(RuleState.ENABLED) - .withComment("Test VM") - .withSuppressionAlertsScope( - new SuppressionAlertsScope() - .withAllOf( - Arrays - .asList( - new ScopeElement() - .withField("entities.ip.address") - .withAdditionalProperties( - mapOf( - "in", - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "[\"104.215.95.187\",\"52.164.206.56\"]", - Object.class, - SerializerEncoding.JSON))), - new ScopeElement() - .withField("entities.process.commandline") - .withAdditionalProperties(mapOf("contains", "POWERSHELL.EXE"))))), - com.azure.core.util.Context.NONE); - } - + manager.alertsSuppressionRules().updateWithResponse("dismissIpAnomalyAlerts", + new AlertsSuppressionRuleInner().withAlertType("IpAnomaly") + .withExpirationDateUtc(OffsetDateTime.parse("2019-12-01T19:50:47.083633Z")).withReason("FalsePositive") + .withState(RuleState.ENABLED).withComment("Test VM") + .withSuppressionAlertsScope(new SuppressionAlertsScope().withAllOf(Arrays.asList( + new ScopeElement().withField("entities.ip.address") + .withAdditionalProperties(mapOf("in", + SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "[\"104.215.95.187\",\"52.164.206.56\"]", Object.class, SerializerEncoding.JSON))), + new ScopeElement().withField("entities.process.commandline") + .withAdditionalProperties(mapOf("contains", "POWERSHELL.EXE"))))), + com.azure.core.util.Context.NONE); + } + + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -1257,20 +1269,23 @@ public final class AlertsSuppressionRulesUpdateSamples { ```java import com.azure.resourcemanager.security.models.ConnectionType; -/** Samples for AllowedConnections Get. */ +/** + * Samples for AllowedConnections Get. + */ public final class AllowedConnectionsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AllowedConnections/GetAllowedConnections_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AllowedConnections/ + * GetAllowedConnections_example.json */ /** * Sample code: Get allowed connections. - * + * * @param manager Entry point to SecurityManager. */ public static void getAllowedConnections(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .allowedConnections() - .getWithResponse("myResourceGroup", "centralus", ConnectionType.INTERNAL, com.azure.core.util.Context.NONE); + manager.allowedConnections().getWithResponse("myResourceGroup", "centralus", ConnectionType.INTERNAL, + com.azure.core.util.Context.NONE); } } ``` @@ -1278,18 +1293,22 @@ public final class AllowedConnectionsGetSamples { ### AllowedConnections_List ```java -/** Samples for AllowedConnections List. */ +/** + * Samples for AllowedConnections List. + */ public final class AllowedConnectionsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AllowedConnections/GetAllowedConnectionsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AllowedConnections/ + * GetAllowedConnectionsSubscription_example.json */ /** * Sample code: Get allowed connections on a subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getAllowedConnectionsOnASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getAllowedConnectionsOnASubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.allowedConnections().list(com.azure.core.util.Context.NONE); } } @@ -1298,14 +1317,18 @@ public final class AllowedConnectionsListSamples { ### AllowedConnections_ListByHomeRegion ```java -/** Samples for AllowedConnections ListByHomeRegion. */ +/** + * Samples for AllowedConnections ListByHomeRegion. + */ public final class AllowedConnectionsListByHomeRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AllowedConnections/GetAllowedConnectionsSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/AllowedConnections/ + * GetAllowedConnectionsSubscriptionLocation_example.json */ /** * Sample code: Get allowed connections on a subscription from security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getAllowedConnectionsOnASubscriptionFromSecurityDataLocation( @@ -1315,86 +1338,152 @@ public final class AllowedConnectionsListByHomeRegionSamples { } ``` -### ApiCollection_Get +### ApiCollections_GetByAzureApiManagementService ```java -/** Samples for ApiCollection Get. */ -public final class ApiCollectionGetSamples { +/** + * Samples for ApiCollections GetByAzureApiManagementService. + */ +public final class ApiCollectionsGetByAzureApiManagementServiceSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-11-20-preview/examples/ApiCollections/APICollection_Get_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2023-11-15/examples/ApiCollections/ + * APICollections_GetByAzureApiManagementService_example.json */ /** - * Sample code: Gets an Azure API Management API if it has been onboarded to Defender for APIs. - * + * Sample code: Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. + * * @param manager Entry point to SecurityManager. */ - public static void getsAnAzureAPIManagementAPIIfItHasBeenOnboardedToDefenderForAPIs( + public static void getsAnAzureAPIManagementAPIIfItHasBeenOnboardedToMicrosoftDefenderForAPIs( com.azure.resourcemanager.security.SecurityManager manager) { - manager.apiCollections().getWithResponse("rg1", "apimService1", "echo-api", com.azure.core.util.Context.NONE); + manager.apiCollections().getByAzureApiManagementServiceWithResponse("rg1", "apimService1", "echo-api", + com.azure.core.util.Context.NONE); } } ``` -### ApiCollection_List +### ApiCollections_List ```java -/** Samples for ApiCollection List. */ -public final class ApiCollectionListSamples { +/** + * Samples for ApiCollections List. + */ +public final class ApiCollectionsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-11-20-preview/examples/ApiCollections/APICollection_List_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2023-11-15/examples/ApiCollections/ + * APICollections_ListBySubscription_example.json */ /** - * Sample code: Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. - * + * Sample code: Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender + * for APIs. + * * @param manager Entry point to SecurityManager. */ - public static void getsAListOfAzureAPIManagementAPIsThatHaveBeenOnboardedToDefenderForAPIs( + public static void getsAListOfAPICollectionsWithinASubscriptionThatHaveBeenOnboardedToMicrosoftDefenderForAPIs( com.azure.resourcemanager.security.SecurityManager manager) { - manager.apiCollections().list("rg1", "apimService1", com.azure.core.util.Context.NONE); + manager.apiCollections().list(com.azure.core.util.Context.NONE); } } ``` -### ApiCollectionOffboarding_Delete +### ApiCollections_ListByAzureApiManagementService ```java -/** Samples for ApiCollectionOffboarding Delete. */ -public final class ApiCollectionOffboardingDeleteSamples { +/** + * Samples for ApiCollections ListByAzureApiManagementService. + */ +public final class ApiCollectionsListByAzureApiManagementServiceSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-11-20-preview/examples/ApiCollections/APICollectionOffboarding_Delete_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2023-11-15/examples/ApiCollections/ + * APICollections_ListByAzureApiManagementService_example.json */ /** - * Sample code: Delete a security recommendation task on a resource. - * + * Sample code: Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. + * + * @param manager Entry point to SecurityManager. + */ + public static void getsAListOfAzureAPIManagementAPIsThatHaveBeenOnboardedToMicrosoftDefenderForAPIs( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.apiCollections().listByAzureApiManagementService("rg1", "apimService1", + com.azure.core.util.Context.NONE); + } +} +``` + +### ApiCollections_ListByResourceGroup + +```java +/** + * Samples for ApiCollections ListByResourceGroup. + */ +public final class ApiCollectionsListByResourceGroupSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2023-11-15/examples/ApiCollections/ + * APICollections_ListByResourceGroup_example.json + */ + /** + * Sample code: Gets a list of API collections within a resource group that have been onboarded to Microsoft + * Defender for APIs. + * + * @param manager Entry point to SecurityManager. + */ + public static void getsAListOfAPICollectionsWithinAResourceGroupThatHaveBeenOnboardedToMicrosoftDefenderForAPIs( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.apiCollections().listByResourceGroup("rg1", com.azure.core.util.Context.NONE); + } +} +``` + +### ApiCollections_OffboardAzureApiManagementApi + +```java +/** + * Samples for ApiCollections OffboardAzureApiManagementApi. + */ +public final class ApiCollectionsOffboardAzureApiManagementApiSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2023-11-15/examples/ApiCollections/ + * APICollections_OffboardAzureApiManagementApi_example.json + */ + /** + * Sample code: Offboard an Azure API Management API from Microsoft Defender for APIs. + * * @param manager Entry point to SecurityManager. */ - public static void deleteASecurityRecommendationTaskOnAResource( + public static void offboardAnAzureAPIManagementAPIFromMicrosoftDefenderForAPIs( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .apiCollectionOffboardings() - .deleteWithResponse("rg1", "apimService1", "echo-api", com.azure.core.util.Context.NONE); + manager.apiCollections().offboardAzureApiManagementApiWithResponse("rg1", "apimService1", "echo-api", + com.azure.core.util.Context.NONE); } } ``` -### ApiCollectionOnboarding_Create +### ApiCollections_OnboardAzureApiManagementApi ```java -/** Samples for ApiCollectionOnboarding Create. */ -public final class ApiCollectionOnboardingCreateSamples { +/** + * Samples for ApiCollections OnboardAzureApiManagementApi. + */ +public final class ApiCollectionsOnboardAzureApiManagementApiSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-11-20-preview/examples/ApiCollections/APICollectionOnboarding_Create_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2023-11-15/examples/ApiCollections/ + * APICollections_OnboardAzureApiManagementApi_example.json */ /** - * Sample code: Onboard an Azure API Management API to Defender for APIs. - * + * Sample code: Onboard an Azure API Management API to Microsoft Defender for APIs. + * * @param manager Entry point to SecurityManager. */ - public static void onboardAnAzureAPIManagementAPIToDefenderForAPIs( + public static void onboardAnAzureAPIManagementAPIToMicrosoftDefenderForAPIs( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .apiCollectionOnboardings() - .createWithResponse("rg1", "apimService1", "echo-api", com.azure.core.util.Context.NONE); + manager.apiCollections().onboardAzureApiManagementApi("rg1", "apimService1", "echo-api", + com.azure.core.util.Context.NONE); } } ``` @@ -1408,33 +1497,28 @@ import com.azure.resourcemanager.security.models.ApplicationSourceResourceType; import java.io.IOException; import java.util.Arrays; -/** Samples for ApplicationOperation CreateOrUpdate. */ +/** + * Samples for ApplicationOperation CreateOrUpdate. + */ public final class ApplicationOperationCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/PutApplication_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * PutApplication_example.json */ /** * Sample code: Create application. - * + * * @param manager Entry point to SecurityManager. */ public static void createApplication(com.azure.resourcemanager.security.SecurityManager manager) throws IOException { - manager - .applicationOperations() - .define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") - .withDisplayName("Admin's application") - .withDescription("An application on critical recommendations") + manager.applicationOperations().define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") + .withDisplayName("Admin's application").withDescription("An application on critical recommendations") .withSourceResourceType(ApplicationSourceResourceType.ASSESSMENTS) - .withConditionSets( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"conditions\":[{\"operator\":\"contains\",\"property\":\"$.Id\",\"value\":\"-bil-\"}]}", - Object.class, - SerializerEncoding.JSON))) + .withConditionSets(Arrays.asList(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"conditions\":[{\"operator\":\"contains\",\"property\":\"$.Id\",\"value\":\"-bil-\"}]}", + Object.class, SerializerEncoding.JSON))) .create(); } } @@ -1443,20 +1527,23 @@ public final class ApplicationOperationCreateOrUpdateSamples { ### ApplicationOperation_Delete ```java -/** Samples for ApplicationOperation Delete. */ +/** + * Samples for ApplicationOperation Delete. + */ public final class ApplicationOperationDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/DeleteApplication_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * DeleteApplication_example.json */ /** * Sample code: Delete security Application. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteSecurityApplication(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .applicationOperations() - .deleteWithResponse("ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); + manager.applicationOperations().deleteWithResponse("ad9a8e26-29d9-4829-bb30-e597a58cdbb8", + com.azure.core.util.Context.NONE); } } ``` @@ -1464,21 +1551,24 @@ public final class ApplicationOperationDeleteSamples { ### ApplicationOperation_Get ```java -/** Samples for ApplicationOperation Get. */ +/** + * Samples for ApplicationOperation Get. + */ public final class ApplicationOperationGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/GetApplication_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * GetApplication_example.json */ /** * Sample code: Get security application by specific applicationId. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityApplicationBySpecificApplicationId( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .applicationOperations() - .getWithResponse("ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); + public static void + getSecurityApplicationBySpecificApplicationId(com.azure.resourcemanager.security.SecurityManager manager) { + manager.applicationOperations().getWithResponse("ad9a8e26-29d9-4829-bb30-e597a58cdbb8", + com.azure.core.util.Context.NONE); } } ``` @@ -1486,18 +1576,22 @@ public final class ApplicationOperationGetSamples { ### Applications_List ```java -/** Samples for Applications List. */ +/** + * Samples for Applications List. + */ public final class ApplicationsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ListBySubscriptionApplications_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * ListBySubscriptionApplications_example.json */ /** * Sample code: List applications security by subscription level scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void listApplicationsSecurityBySubscriptionLevelScope( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listApplicationsSecurityBySubscriptionLevelScope(com.azure.resourcemanager.security.SecurityManager manager) { manager.applications().list(com.azure.core.util.Context.NONE); } } @@ -1510,26 +1604,26 @@ import com.azure.resourcemanager.security.models.AssessmentStatus; import com.azure.resourcemanager.security.models.AssessmentStatusCode; import com.azure.resourcemanager.security.models.AzureResourceDetails; -/** Samples for Assessments CreateOrUpdate. */ +/** + * Samples for Assessments CreateOrUpdate. + */ public final class AssessmentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/PutAssessment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/ + * PutAssessment_example.json */ /** * Sample code: Create security recommendation task on a resource. - * + * * @param manager Entry point to SecurityManager. */ - public static void createSecurityRecommendationTaskOnAResource( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessments() - .define("8bb8be0a-6010-4789-812f-e4d661c4ed0e") - .withExistingResourceId( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2") + public static void + createSecurityRecommendationTaskOnAResource(com.azure.resourcemanager.security.SecurityManager manager) { + manager.assessments().define("8bb8be0a-6010-4789-812f-e4d661c4ed0e").withExistingResourceId( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2") .withStatus(new AssessmentStatus().withCode(AssessmentStatusCode.HEALTHY)) - .withResourceDetails(new AzureResourceDetails()) - .create(); + .withResourceDetails(new AzureResourceDetails()).create(); } } ``` @@ -1537,24 +1631,25 @@ public final class AssessmentsCreateOrUpdateSamples { ### Assessments_Delete ```java -/** Samples for Assessments Delete. */ +/** + * Samples for Assessments Delete. + */ public final class AssessmentsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/DeleteAssessment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/ + * DeleteAssessment_example.json */ /** * Sample code: Delete a security recommendation task on a resource. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteASecurityRecommendationTaskOnAResource( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessments() - .deleteByResourceGroupWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2", - "8bb8be0a-6010-4789-812f-e4d661c4ed0e", - com.azure.core.util.Context.NONE); + public static void + deleteASecurityRecommendationTaskOnAResource(com.azure.resourcemanager.security.SecurityManager manager) { + manager.assessments().deleteByResourceGroupWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2", + "8bb8be0a-6010-4789-812f-e4d661c4ed0e", com.azure.core.util.Context.NONE); } } ``` @@ -1564,44 +1659,42 @@ public final class AssessmentsDeleteSamples { ```java import com.azure.resourcemanager.security.models.ExpandEnum; -/** Samples for Assessments Get. */ +/** + * Samples for Assessments Get. + */ public final class AssessmentsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/GetAssessmentWithExpand_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/ + * GetAssessmentWithExpand_example.json */ /** * Sample code: Get security recommendation task from security data location with expand parameter. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityRecommendationTaskFromSecurityDataLocationWithExpandParameter( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessments() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2", - "21300918-b2e3-0346-785f-c77ff57d243b", - ExpandEnum.LINKS, - com.azure.core.util.Context.NONE); + manager.assessments().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2", + "21300918-b2e3-0346-785f-c77ff57d243b", ExpandEnum.LINKS, com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/GetAssessment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/ + * GetAssessment_example.json */ /** * Sample code: Get security recommendation task from security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityRecommendationTaskFromSecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessments() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2", - "21300918-b2e3-0346-785f-c77ff57d243b", - null, - com.azure.core.util.Context.NONE); + manager.assessments().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.Compute/virtualMachineScaleSets/vmss2", + "21300918-b2e3-0346-785f-c77ff57d243b", null, com.azure.core.util.Context.NONE); } } ``` @@ -1609,20 +1702,23 @@ public final class AssessmentsGetSamples { ### Assessments_List ```java -/** Samples for Assessments List. */ +/** + * Samples for Assessments List. + */ public final class AssessmentsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/ListAssessments_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/Assessments/ + * ListAssessments_example.json */ /** * Sample code: List security assessments. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecurityAssessments(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessments() - .list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", com.azure.core.util.Context.NONE); + manager.assessments().list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + com.azure.core.util.Context.NONE); } } ``` @@ -1638,36 +1734,32 @@ import com.azure.resourcemanager.security.models.Threats; import com.azure.resourcemanager.security.models.UserImpact; import java.util.Arrays; -/** Samples for AssessmentsMetadata CreateInSubscription. */ +/** + * Samples for AssessmentsMetadata CreateInSubscription. + */ public final class AssessmentsMetadataCreateInSubscriptionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/CreateAssessmentsMetadata_subscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ + * CreateAssessmentsMetadata_subscription_example.json */ /** * Sample code: Create security assessment metadata for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void createSecurityAssessmentMetadataForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessmentsMetadatas() - .define("ca039e75-a276-4175-aebc-bcd41e4b14b7") + public static void + createSecurityAssessmentMetadataForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.assessmentsMetadatas().define("ca039e75-a276-4175-aebc-bcd41e4b14b7") .withDisplayName("Install endpoint protection solution on virtual machine scale sets") .withDescription( - "Install an endpoint protection solution on your virtual machines scale sets, to protect them from" - + " threats and vulnerabilities.") + "Install an endpoint protection solution on your virtual machines scale sets, to protect them from threats and vulnerabilities.") .withRemediationDescription( - "To install an endpoint protection solution: 1. Follow" - + " the instructions in How do I turn on antimalware in my virtual machine scale set") - .withCategories(Arrays.asList(Categories.COMPUTE)) - .withSeverity(Severity.MEDIUM) - .withUserImpact(UserImpact.LOW) - .withImplementationEffort(ImplementationEffort.LOW) + "To install an endpoint protection solution: 1. Follow the instructions in How do I turn on antimalware in my virtual machine scale set") + .withCategories(Arrays.asList(Categories.COMPUTE)).withSeverity(Severity.MEDIUM) + .withUserImpact(UserImpact.LOW).withImplementationEffort(ImplementationEffort.LOW) .withThreats(Arrays.asList(Threats.DATA_EXFILTRATION, Threats.DATA_SPILLAGE, Threats.MALICIOUS_INSIDER)) - .withAssessmentType(AssessmentType.CUSTOMER_MANAGED) - .create(); + .withAssessmentType(AssessmentType.CUSTOMER_MANAGED).create(); } } ``` @@ -1675,21 +1767,24 @@ public final class AssessmentsMetadataCreateInSubscriptionSamples { ### AssessmentsMetadata_DeleteInSubscription ```java -/** Samples for AssessmentsMetadata DeleteInSubscription. */ +/** + * Samples for AssessmentsMetadata DeleteInSubscription. + */ public final class AssessmentsMetadataDeleteInSubscriptionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/DeleteAssessmentsMetadata_subscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ + * DeleteAssessmentsMetadata_subscription_example.json */ /** * Sample code: Delete a security assessment metadata for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteASecurityAssessmentMetadataForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessmentsMetadatas() - .deleteInSubscriptionWithResponse("ca039e75-a276-4175-aebc-bcd41e4b14b7", com.azure.core.util.Context.NONE); + public static void + deleteASecurityAssessmentMetadataForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.assessmentsMetadatas().deleteInSubscriptionWithResponse("ca039e75-a276-4175-aebc-bcd41e4b14b7", + com.azure.core.util.Context.NONE); } } ``` @@ -1697,20 +1792,23 @@ public final class AssessmentsMetadataDeleteInSubscriptionSamples { ### AssessmentsMetadata_Get ```java -/** Samples for AssessmentsMetadata Get. */ +/** + * Samples for AssessmentsMetadata Get. + */ public final class AssessmentsMetadataGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/GetAssessmentsMetadata_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ + * GetAssessmentsMetadata_example.json */ /** * Sample code: Get security assessment metadata. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityAssessmentMetadata(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessmentsMetadatas() - .getWithResponse("21300918-b2e3-0346-785f-c77ff57d243b", com.azure.core.util.Context.NONE); + manager.assessmentsMetadatas().getWithResponse("21300918-b2e3-0346-785f-c77ff57d243b", + com.azure.core.util.Context.NONE); } } ``` @@ -1718,21 +1816,24 @@ public final class AssessmentsMetadataGetSamples { ### AssessmentsMetadata_GetInSubscription ```java -/** Samples for AssessmentsMetadata GetInSubscription. */ +/** + * Samples for AssessmentsMetadata GetInSubscription. + */ public final class AssessmentsMetadataGetInSubscriptionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/GetAssessmentsMetadata_subscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ + * GetAssessmentsMetadata_subscription_example.json */ /** * Sample code: Get security assessment metadata for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityAssessmentMetadataForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .assessmentsMetadatas() - .getInSubscriptionWithResponse("21300918-b2e3-0346-785f-c77ff57d243b", com.azure.core.util.Context.NONE); + public static void + getSecurityAssessmentMetadataForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.assessmentsMetadatas().getInSubscriptionWithResponse("21300918-b2e3-0346-785f-c77ff57d243b", + com.azure.core.util.Context.NONE); } } ``` @@ -1740,14 +1841,18 @@ public final class AssessmentsMetadataGetInSubscriptionSamples { ### AssessmentsMetadata_List ```java -/** Samples for AssessmentsMetadata List. */ +/** + * Samples for AssessmentsMetadata List. + */ public final class AssessmentsMetadataListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ListAssessmentsMetadata_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ + * ListAssessmentsMetadata_example.json */ /** * Sample code: List security assessment metadata. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecurityAssessmentMetadata(com.azure.resourcemanager.security.SecurityManager manager) { @@ -1759,18 +1864,22 @@ public final class AssessmentsMetadataListSamples { ### AssessmentsMetadata_ListBySubscription ```java -/** Samples for AssessmentsMetadata ListBySubscription. */ +/** + * Samples for AssessmentsMetadata ListBySubscription. + */ public final class AssessmentsMetadataListBySubscriptionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ListAssessmentsMetadata_subscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2021-06-01/examples/AssessmentsMetadata/ + * ListAssessmentsMetadata_subscription_example.json */ /** * Sample code: List security assessment metadata for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void listSecurityAssessmentMetadataForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listSecurityAssessmentMetadataForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.assessmentsMetadatas().listBySubscription(com.azure.core.util.Context.NONE); } } @@ -1781,18 +1890,22 @@ public final class AssessmentsMetadataListBySubscriptionSamples { ```java import com.azure.resourcemanager.security.models.AutoProvision; -/** Samples for AutoProvisioningSettings Create. */ +/** + * Samples for AutoProvisioningSettings Create. + */ public final class AutoProvisioningSettingsCreateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/AutoProvisioningSettings/CreateAutoProvisioningSettingsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * AutoProvisioningSettings/CreateAutoProvisioningSettingsSubscription_example.json */ /** * Sample code: Create auto provisioning settings for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void createAutoProvisioningSettingsForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + createAutoProvisioningSettingsForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.autoProvisioningSettings().define("default").withAutoProvision(AutoProvision.ON).create(); } } @@ -1801,18 +1914,22 @@ public final class AutoProvisioningSettingsCreateSamples { ### AutoProvisioningSettings_Get ```java -/** Samples for AutoProvisioningSettings Get. */ +/** + * Samples for AutoProvisioningSettings Get. + */ public final class AutoProvisioningSettingsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/AutoProvisioningSettings/GetAutoProvisioningSettingSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * AutoProvisioningSettings/GetAutoProvisioningSettingSubscription_example.json */ /** * Sample code: Get an auto provisioning setting for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getAnAutoProvisioningSettingForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getAnAutoProvisioningSettingForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.autoProvisioningSettings().getWithResponse("default", com.azure.core.util.Context.NONE); } } @@ -1821,18 +1938,22 @@ public final class AutoProvisioningSettingsGetSamples { ### AutoProvisioningSettings_List ```java -/** Samples for AutoProvisioningSettings List. */ +/** + * Samples for AutoProvisioningSettings List. + */ public final class AutoProvisioningSettingsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/AutoProvisioningSettings/GetAutoProvisioningSettingsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * AutoProvisioningSettings/GetAutoProvisioningSettingsSubscription_example.json */ /** * Sample code: Get auto provisioning settings for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getAutoProvisioningSettingsForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getAutoProvisioningSettingsForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.autoProvisioningSettings().list(com.azure.core.util.Context.NONE); } } @@ -1853,162 +1974,97 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for Automations CreateOrUpdate. */ +/** + * Samples for Automations CreateOrUpdate. + */ public final class AutomationsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/PutAutomationAllAssessments_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * PutAutomationAllAssessments_example.json */ /** * Sample code: Create or update a security automation for all assessments (including all severities). - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateASecurityAutomationForAllAssessmentsIncludingAllSeverities( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .automations() - .define("exampleAutomation") - .withRegion("Central US") - .withExistingResourceGroup("exampleResourceGroup") - .withTags(mapOf()) + manager.automations().define("exampleAutomation").withRegion("Central US") + .withExistingResourceGroup("exampleResourceGroup").withTags(mapOf()) .withDescription( - "An example of a security automation that triggers one LogicApp resource (myTest1) on any security" - + " assessment") + "An example of a security automation that triggers one LogicApp resource (myTest1) on any security assessment") .withIsEnabled(true) - .withScopes( - Arrays - .asList( - new AutomationScope() - .withDescription( - "A description that helps to identify this scope - for example: security assessments" - + " that relate to the resource group myResourceGroup within the subscription" - + " a5caac9c-5c04-49af-b3d0-e204f40345d5") - .withScopePath( - "/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) + .withScopes(Arrays.asList(new AutomationScope().withDescription( + "A description that helps to identify this scope - for example: security assessments that relate to the resource group myResourceGroup within the subscription a5caac9c-5c04-49af-b3d0-e204f40345d5") + .withScopePath("/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) .withSources(Arrays.asList(new AutomationSource().withEventSource(EventSource.ASSESSMENTS))) - .withActions( - Arrays - .asList( - new AutomationActionLogicApp() - .withLogicAppResourceId( - "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") - .withUri("https://exampleTriggerUri1.com"))) + .withActions(Arrays.asList(new AutomationActionLogicApp().withLogicAppResourceId( + "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") + .withUri("https://exampleTriggerUri1.com"))) .create(); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/PutDisableAutomation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * PutDisableAutomation_example.json */ /** * Sample code: Disable or enable a security automation. - * + * * @param manager Entry point to SecurityManager. */ public static void disableOrEnableASecurityAutomation(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .automations() - .define("exampleAutomation") - .withRegion("Central US") - .withExistingResourceGroup("exampleResourceGroup") - .withTags(mapOf()) + manager.automations().define("exampleAutomation").withRegion("Central US") + .withExistingResourceGroup("exampleResourceGroup").withTags(mapOf()) .withDescription( - "An example of a security automation that triggers one LogicApp resource (myTest1) on any security" - + " assessment of type customAssessment") + "An example of a security automation that triggers one LogicApp resource (myTest1) on any security assessment of type customAssessment") .withIsEnabled(false) - .withScopes( - Arrays - .asList( - new AutomationScope() - .withDescription( - "A description that helps to identify this scope - for example: security assessments" - + " that relate to the resource group myResourceGroup within the subscription" - + " a5caac9c-5c04-49af-b3d0-e204f40345d5") - .withScopePath( - "/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) - .withSources( - Arrays - .asList( - new AutomationSource() - .withEventSource(EventSource.ASSESSMENTS) - .withRuleSets( - Arrays - .asList( - new AutomationRuleSet() - .withRules( - Arrays - .asList( - new AutomationTriggeringRule() - .withPropertyJPath("$.Entity.AssessmentType") - .withPropertyType(PropertyType.STRING) - .withExpectedValue("customAssessment") - .withOperator(Operator.EQUALS))))))) - .withActions( - Arrays - .asList( - new AutomationActionLogicApp() - .withLogicAppResourceId( - "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") - .withUri("https://exampleTriggerUri1.com"))) + .withScopes(Arrays.asList(new AutomationScope().withDescription( + "A description that helps to identify this scope - for example: security assessments that relate to the resource group myResourceGroup within the subscription a5caac9c-5c04-49af-b3d0-e204f40345d5") + .withScopePath("/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) + .withSources(Arrays.asList(new AutomationSource().withEventSource(EventSource.ASSESSMENTS).withRuleSets( + Arrays.asList(new AutomationRuleSet().withRules(Arrays.asList(new AutomationTriggeringRule() + .withPropertyJPath("$.Entity.AssessmentType").withPropertyType(PropertyType.STRING) + .withExpectedValue("customAssessment").withOperator(Operator.EQUALS))))))) + .withActions(Arrays.asList(new AutomationActionLogicApp().withLogicAppResourceId( + "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") + .withUri("https://exampleTriggerUri1.com"))) .create(); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/PutAutomationHighSeverityAssessments_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * PutAutomationHighSeverityAssessments_example.json */ /** * Sample code: Create or update a security automation for all high severity assessments. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateASecurityAutomationForAllHighSeverityAssessments( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .automations() - .define("exampleAutomation") - .withRegion("Central US") - .withExistingResourceGroup("exampleResourceGroup") - .withTags(mapOf()) + manager.automations().define("exampleAutomation").withRegion("Central US") + .withExistingResourceGroup("exampleResourceGroup").withTags(mapOf()) .withDescription( - "An example of a security automation that triggers one LogicApp resource (myTest1) on any high severity" - + " security assessment") + "An example of a security automation that triggers one LogicApp resource (myTest1) on any high severity security assessment") .withIsEnabled(true) - .withScopes( - Arrays - .asList( - new AutomationScope() - .withDescription( - "A description that helps to identify this scope - for example: security assessments" - + " that relate to the resource group myResourceGroup within the subscription" - + " a5caac9c-5c04-49af-b3d0-e204f40345d5") - .withScopePath( - "/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) - .withSources( - Arrays - .asList( - new AutomationSource() - .withEventSource(EventSource.ASSESSMENTS) - .withRuleSets( - Arrays - .asList( - new AutomationRuleSet() - .withRules( - Arrays - .asList( - new AutomationTriggeringRule() - .withPropertyJPath("properties.metadata.severity") - .withPropertyType(PropertyType.STRING) - .withExpectedValue("High") - .withOperator(Operator.EQUALS))))))) - .withActions( - Arrays - .asList( - new AutomationActionLogicApp() - .withLogicAppResourceId( - "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") - .withUri("https://exampleTriggerUri1.com"))) + .withScopes(Arrays.asList(new AutomationScope().withDescription( + "A description that helps to identify this scope - for example: security assessments that relate to the resource group myResourceGroup within the subscription a5caac9c-5c04-49af-b3d0-e204f40345d5") + .withScopePath("/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) + .withSources(Arrays.asList(new AutomationSource().withEventSource(EventSource.ASSESSMENTS).withRuleSets( + Arrays.asList(new AutomationRuleSet().withRules(Arrays.asList(new AutomationTriggeringRule() + .withPropertyJPath("properties.metadata.severity").withPropertyType(PropertyType.STRING) + .withExpectedValue("High").withOperator(Operator.EQUALS))))))) + .withActions(Arrays.asList(new AutomationActionLogicApp().withLogicAppResourceId( + "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") + .withUri("https://exampleTriggerUri1.com"))) .create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -2025,20 +2081,23 @@ public final class AutomationsCreateOrUpdateSamples { ### Automations_Delete ```java -/** Samples for Automations Delete. */ +/** + * Samples for Automations Delete. + */ public final class AutomationsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/DeleteAutomation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * DeleteAutomation_example.json */ /** * Sample code: Delete a security automation. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteASecurityAutomation(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .automations() - .deleteByResourceGroupWithResponse("myRg", "myAutomationName", com.azure.core.util.Context.NONE); + manager.automations().deleteByResourceGroupWithResponse("myRg", "myAutomationName", + com.azure.core.util.Context.NONE); } } ``` @@ -2046,21 +2105,23 @@ public final class AutomationsDeleteSamples { ### Automations_GetByResourceGroup ```java -/** Samples for Automations GetByResourceGroup. */ +/** + * Samples for Automations GetByResourceGroup. + */ public final class AutomationsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/GetAutomationResourceGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * GetAutomationResourceGroup_example.json */ /** * Sample code: Retrieve a security automation. - * + * * @param manager Entry point to SecurityManager. */ public static void retrieveASecurityAutomation(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .automations() - .getByResourceGroupWithResponse( - "exampleResourceGroup", "exampleAutomation", com.azure.core.util.Context.NONE); + manager.automations().getByResourceGroupWithResponse("exampleResourceGroup", "exampleAutomation", + com.azure.core.util.Context.NONE); } } ``` @@ -2068,18 +2129,22 @@ public final class AutomationsGetByResourceGroupSamples { ### Automations_List ```java -/** Samples for Automations List. */ +/** + * Samples for Automations List. + */ public final class AutomationsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/GetAutomationsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * GetAutomationsSubscription_example.json */ /** * Sample code: List all security automations of a specified subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void listAllSecurityAutomationsOfASpecifiedSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listAllSecurityAutomationsOfASpecifiedSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.automations().list(com.azure.core.util.Context.NONE); } } @@ -2088,14 +2153,18 @@ public final class AutomationsListSamples { ### Automations_ListByResourceGroup ```java -/** Samples for Automations ListByResourceGroup. */ +/** + * Samples for Automations ListByResourceGroup. + */ public final class AutomationsListByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/GetAutomationsResourceGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * GetAutomationsResourceGroup_example.json */ /** * Sample code: List all security automations of a specified resource group. - * + * * @param manager Entry point to SecurityManager. */ public static void listAllSecurityAutomationsOfASpecifiedResourceGroup( @@ -2105,6 +2174,61 @@ public final class AutomationsListByResourceGroupSamples { } ``` +### Automations_Update + +```java +import com.azure.resourcemanager.security.models.Automation; +import com.azure.resourcemanager.security.models.AutomationActionLogicApp; +import com.azure.resourcemanager.security.models.AutomationScope; +import com.azure.resourcemanager.security.models.AutomationSource; +import com.azure.resourcemanager.security.models.EventSource; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +/** + * Samples for Automations Update. + */ +public final class AutomationsUpdateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * PatchAutomation_example.json + */ + /** + * Sample code: Update a security automation. + * + * @param manager Entry point to SecurityManager. + */ + public static void updateASecurityAutomation(com.azure.resourcemanager.security.SecurityManager manager) { + Automation resource = manager.automations().getByResourceGroupWithResponse("exampleResourceGroup", + "exampleAutomation", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("Example", "exampleTag")).withDescription( + "An example of a security automation that triggers one LogicApp resource (myTest1) on any security assessment") + .withScopes(Arrays.asList(new AutomationScope().withDescription( + "A description that helps to identify this scope - for example: security assessments that relate to the resource group myResourceGroup within the subscription a5caac9c-5c04-49af-b3d0-e204f40345d5") + .withScopePath("/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) + .withSources(Arrays.asList(new AutomationSource().withEventSource(EventSource.ASSESSMENTS))) + .withActions(Arrays.asList(new AutomationActionLogicApp().withLogicAppResourceId( + "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") + .withUri("https://exampleTriggerUri1.com"))) + .apply(); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; + } +} +``` + ### Automations_Validate ```java @@ -2121,67 +2245,41 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for Automations Validate. */ +/** + * Samples for Automations Validate. + */ public final class AutomationsValidateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/Automations/ValidateAutomation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/Automations/ + * ValidateAutomation_example.json */ /** * Sample code: Validate the security automation model before create or update. - * + * * @param manager Entry point to SecurityManager. */ public static void validateTheSecurityAutomationModelBeforeCreateOrUpdate( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .automations() - .validateWithResponse( - "exampleResourceGroup", - "exampleAutomation", - new AutomationInner() - .withLocation("Central US") - .withTags(mapOf()) - .withDescription( - "An example of a security automation that triggers one LogicApp resource (myTest1) on any" - + " security assessment of type customAssessment") - .withIsEnabled(true) - .withScopes( - Arrays - .asList( - new AutomationScope() - .withDescription( - "A description that helps to identify this scope - for example: security" - + " assessments that relate to the resource group myResourceGroup within" - + " the subscription a5caac9c-5c04-49af-b3d0-e204f40345d5") - .withScopePath( - "/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) - .withSources( - Arrays - .asList( - new AutomationSource() - .withEventSource(EventSource.ASSESSMENTS) - .withRuleSets( - Arrays - .asList( - new AutomationRuleSet() - .withRules( - Arrays - .asList( - new AutomationTriggeringRule() - .withPropertyJPath("$.Entity.AssessmentType") - .withPropertyType(PropertyType.STRING) - .withExpectedValue("customAssessment") - .withOperator(Operator.EQUALS))))))) - .withActions( - Arrays - .asList( - new AutomationActionLogicApp() - .withLogicAppResourceId( - "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") - .withUri("https://exampleTriggerUri1.com"))), - com.azure.core.util.Context.NONE); - } - + manager.automations().validateWithResponse("exampleResourceGroup", "exampleAutomation", new AutomationInner() + .withLocation("Central US").withTags(mapOf()) + .withDescription( + "An example of a security automation that triggers one LogicApp resource (myTest1) on any security assessment of type customAssessment") + .withIsEnabled(true) + .withScopes(Arrays.asList(new AutomationScope().withDescription( + "A description that helps to identify this scope - for example: security assessments that relate to the resource group myResourceGroup within the subscription a5caac9c-5c04-49af-b3d0-e204f40345d5") + .withScopePath("/subscriptions/a5caac9c-5c04-49af-b3d0-e204f40345d5/resourceGroups/myResourceGroup"))) + .withSources(Arrays.asList(new AutomationSource().withEventSource(EventSource.ASSESSMENTS).withRuleSets( + Arrays.asList(new AutomationRuleSet().withRules(Arrays.asList(new AutomationTriggeringRule() + .withPropertyJPath("$.Entity.AssessmentType").withPropertyType(PropertyType.STRING) + .withExpectedValue("customAssessment").withOperator(Operator.EQUALS))))))) + .withActions(Arrays.asList(new AutomationActionLogicApp().withLogicAppResourceId( + "/subscriptions/e54a4a18-5b94-4f90-9471-bd3decad8a2e/resourceGroups/sample/providers/Microsoft.Logic/workflows/MyTest1") + .withUri("https://exampleTriggerUri1.com"))), + com.azure.core.util.Context.NONE); + } + + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -2195,399 +2293,780 @@ public final class AutomationsValidateSamples { } ``` -### ComplianceResults_Get +### AzureDevOpsOrgs_CreateOrUpdate ```java -/** Samples for ComplianceResults Get. */ -public final class ComplianceResultsGetSamples { +import com.azure.resourcemanager.security.models.ActionableRemediation; +import com.azure.resourcemanager.security.models.ActionableRemediationState; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgProperties; +import com.azure.resourcemanager.security.models.OnboardingState; + +/** + * Samples for AzureDevOpsOrgs CreateOrUpdate. + */ +public final class AzureDevOpsOrgsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2017-08-01/examples/ComplianceResults/GetComplianceResults_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/CreateOrUpdateAzureDevOpsOrgs_example.json */ /** - * Sample code: Get compliance results on subscription. - * + * Sample code: CreateOrUpdate_AzureDevOpsOrgs. + * * @param manager Entry point to SecurityManager. */ - public static void getComplianceResultsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .complianceResults() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", - "DesignateMoreThanOneOwner", - com.azure.core.util.Context.NONE); + public static void createOrUpdateAzureDevOpsOrgs(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsOrgs().define("myAzDevOpsOrg") + .withExistingSecurityConnector("myRg", "mySecurityConnectorName") + .withProperties(new AzureDevOpsOrgProperties().withOnboardingState(OnboardingState.NOT_APPLICABLE) + .withActionableRemediation(new ActionableRemediation().withState(ActionableRemediationState.ENABLED))) + .create(); } } ``` -### ComplianceResults_List +### AzureDevOpsOrgs_Get ```java -/** Samples for ComplianceResults List. */ -public final class ComplianceResultsListSamples { +/** + * Samples for AzureDevOpsOrgs Get. + */ +public final class AzureDevOpsOrgsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2017-08-01/examples/ComplianceResults/ListComplianceResults_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetAzureDevOpsOrgs_example.json */ /** - * Sample code: Get compliance results on subscription. - * + * Sample code: Get_AzureDevOpsOrgs. + * * @param manager Entry point to SecurityManager. */ - public static void getComplianceResultsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .complianceResults() - .list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", com.azure.core.util.Context.NONE); + public static void getAzureDevOpsOrgs(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsOrgs().getWithResponse("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", + com.azure.core.util.Context.NONE); } } ``` -### Compliances_Get +### AzureDevOpsOrgs_List ```java -/** Samples for Compliances Get. */ -public final class CompliancesGetSamples { +/** + * Samples for AzureDevOpsOrgs List. + */ +public final class AzureDevOpsOrgsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/Compliances/GetCompliance_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListAzureDevOpsOrgs_example.json */ /** - * Sample code: Get security compliance data for a day. - * + * Sample code: List_AzureDevOpsOrgs. + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityComplianceDataForADay(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .compliances() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "2018-01-01Z", com.azure.core.util.Context.NONE); + public static void listAzureDevOpsOrgs(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsOrgs().list("myRg", "mySecurityConnectorName", com.azure.core.util.Context.NONE); } } ``` -### Compliances_List +### AzureDevOpsOrgs_ListAvailable ```java -/** Samples for Compliances List. */ -public final class CompliancesListSamples { +/** + * Samples for AzureDevOpsOrgs ListAvailable. + */ +public final class AzureDevOpsOrgsListAvailableSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/Compliances/GetCompliances_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListAvailableAzureDevOpsOrgs_example.json */ /** - * Sample code: Get security compliance data over time. - * + * Sample code: ListAvailable_AzureDevOpsOrgs. + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityComplianceDataOverTime(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .compliances() - .list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", com.azure.core.util.Context.NONE); + public static void listAvailableAzureDevOpsOrgs(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsOrgs().listAvailableWithResponse("myRg", "mySecurityConnectorName", + com.azure.core.util.Context.NONE); } } ``` -### Connectors_CreateOrUpdate +### AzureDevOpsOrgs_Update ```java -import com.azure.resourcemanager.security.models.AutoProvision; -import com.azure.resourcemanager.security.models.AwAssumeRoleAuthenticationDetailsProperties; -import com.azure.resourcemanager.security.models.AwsCredsAuthenticationDetailsProperties; -import com.azure.resourcemanager.security.models.GcpCredentialsDetailsProperties; -import com.azure.resourcemanager.security.models.HybridComputeSettingsProperties; -import com.azure.resourcemanager.security.models.ProxyServerProperties; -import com.azure.resourcemanager.security.models.ServicePrincipalProperties; +import com.azure.resourcemanager.security.models.ActionableRemediation; +import com.azure.resourcemanager.security.models.ActionableRemediationState; +import com.azure.resourcemanager.security.models.AzureDevOpsOrg; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgProperties; +import com.azure.resourcemanager.security.models.OnboardingState; -/** Samples for Connectors CreateOrUpdate. */ -public final class ConnectorsCreateOrUpdateSamples { +/** + * Samples for AzureDevOpsOrgs Update. + */ +public final class AzureDevOpsOrgsUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/CreateUpdateGcpCredentialsConnectorSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/UpdateAzureDevOpsOrgs_example.json */ /** - * Sample code: gcpCredentials - Create a cloud account connector for a subscription. - * + * Sample code: Update_AzureDevOpsOrgs. + * * @param manager Entry point to SecurityManager. */ - public static void gcpCredentialsCreateACloudAccountConnectorForASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .connectors() - .define("gcp_dev") - .withHybridComputeSettings(new HybridComputeSettingsProperties().withAutoProvision(AutoProvision.OFF)) - .withAuthenticationDetails( - new GcpCredentialsDetailsProperties() - .withOrganizationId("AscDemoOrg") - .withType("service_account") - .withProjectId("asc-project-1234") - .withPrivateKeyId("fakeTokenPlaceholder") - .withPrivateKey("fakeTokenPlaceholder") - .withClientEmail("asc-135@asc-project-1234.iam.gserviceaccount.com") - .withClientId("105889053725632919854") - .withAuthUri("https://accounts.google.com/o/oauth2/auth") - .withTokenUri("fakeTokenPlaceholder") - .withAuthProviderX509CertUrl("https://www.googleapis.com/oauth2/v1/certs") - .withClientX509CertUrl( - "https://www.googleapis.com/robot/v1/metadata/x509/asc-135%40asc-project-1234.iam.gserviceaccount.com")) - .create(); + public static void updateAzureDevOpsOrgs(com.azure.resourcemanager.security.SecurityManager manager) { + AzureDevOpsOrg resource = manager.azureDevOpsOrgs() + .getWithResponse("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", com.azure.core.util.Context.NONE) + .getValue(); + resource.update() + .withProperties(new AzureDevOpsOrgProperties().withOnboardingState(OnboardingState.NOT_APPLICABLE) + .withActionableRemediation(new ActionableRemediation().withState(ActionableRemediationState.ENABLED))) + .apply(); } +} +``` - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/CreateUpdateAwsCredConnectorSubscription_example.json - */ - /** - * Sample code: AwsCred - Create a cloud account connector for a subscription. - * - * @param manager Entry point to SecurityManager. - */ - public static void awsCredCreateACloudAccountConnectorForASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .connectors() - .define("aws_dev1") - .withHybridComputeSettings( - new HybridComputeSettingsProperties() - .withAutoProvision(AutoProvision.ON) - .withResourceGroupName("AwsConnectorRG") - .withRegion("West US 2") - .withProxyServer(new ProxyServerProperties().withIp("167.220.197.140").withPort("34")) - .withServicePrincipal( - new ServicePrincipalProperties() - .withApplicationId("ad9bcd79-be9c-45ab-abd8-80ca1654a7d1") - .withSecret("fakeTokenPlaceholder"))) - .withAuthenticationDetails( - new AwsCredsAuthenticationDetailsProperties() - .withAwsAccessKeyId("fakeTokenPlaceholder") - .withAwsSecretAccessKey("fakeTokenPlaceholder")) - .create(); - } +### AzureDevOpsProjects_CreateOrUpdate + +```java +import com.azure.resourcemanager.security.models.ActionableRemediation; +import com.azure.resourcemanager.security.models.ActionableRemediationState; +import com.azure.resourcemanager.security.models.AzureDevOpsProjectProperties; +import com.azure.resourcemanager.security.models.OnboardingState; +/** + * Samples for AzureDevOpsProjects CreateOrUpdate. + */ +public final class AzureDevOpsProjectsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/CreateUpdateAwsAssumeRoleConnectorSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/CreateOrUpdateAzureDevOpsProjects_example.json */ /** - * Sample code: AwsAssumeRole - Create a cloud account connector for a subscription. - * + * Sample code: CreateOrUpdate_AzureDevOpsProjects. + * * @param manager Entry point to SecurityManager. */ - public static void awsAssumeRoleCreateACloudAccountConnectorForASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .connectors() - .define("aws_dev2") - .withHybridComputeSettings( - new HybridComputeSettingsProperties() - .withAutoProvision(AutoProvision.ON) - .withResourceGroupName("AwsConnectorRG") - .withRegion("West US 2") - .withProxyServer(new ProxyServerProperties().withIp("167.220.197.140").withPort("34")) - .withServicePrincipal( - new ServicePrincipalProperties() - .withApplicationId("ad9bcd79-be9c-45ab-abd8-80ca1654a7d1") - .withSecret("fakeTokenPlaceholder"))) - .withAuthenticationDetails( - new AwAssumeRoleAuthenticationDetailsProperties() - .withAwsAssumeRoleArn("arn:aws:iam::81231569658:role/AscConnector") - .withAwsExternalId("20ff7fc3-e762-44dd-bd96-b71116dcdc23")) + public static void createOrUpdateAzureDevOpsProjects(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsProjects().define("myAzDevOpsProject") + .withExistingAzureDevOpsOrg("myRg", "mySecurityConnectorName", "myAzDevOpsOrg") + .withProperties(new AzureDevOpsProjectProperties().withOnboardingState(OnboardingState.NOT_APPLICABLE) + .withActionableRemediation(new ActionableRemediation().withState(ActionableRemediationState.ENABLED))) .create(); } } ``` -### Connectors_Delete +### AzureDevOpsProjects_Get ```java -/** Samples for Connectors Delete. */ -public final class ConnectorsDeleteSamples { +/** + * Samples for AzureDevOpsProjects Get. + */ +public final class AzureDevOpsProjectsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/DeleteConnectorSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetAzureDevOpsProjects_example.json */ /** - * Sample code: Delete a cloud account connector from a subscription. - * + * Sample code: Get_AzureDevOpsProjects. + * * @param manager Entry point to SecurityManager. */ - public static void deleteACloudAccountConnectorFromASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.connectors().deleteWithResponse("aws_dev1", com.azure.core.util.Context.NONE); + public static void getAzureDevOpsProjects(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsProjects().getWithResponse("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", + "myAzDevOpsProject", com.azure.core.util.Context.NONE); } } ``` -### Connectors_Get +### AzureDevOpsProjects_List ```java -/** Samples for Connectors Get. */ -public final class ConnectorsGetSamples { +/** + * Samples for AzureDevOpsProjects List. + */ +public final class AzureDevOpsProjectsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/GetConnectorSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListAzureDevOpsProjects_example.json */ /** - * Sample code: Details of a specific cloud account connector. - * + * Sample code: List_AzureDevOpsProjects. + * * @param manager Entry point to SecurityManager. */ - public static void detailsOfASpecificCloudAccountConnector( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.connectors().getWithResponse("aws_dev1", com.azure.core.util.Context.NONE); + public static void listAzureDevOpsProjects(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsProjects().list("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", + com.azure.core.util.Context.NONE); } } ``` -### Connectors_List +### AzureDevOpsProjects_Update ```java -/** Samples for Connectors List. */ -public final class ConnectorsListSamples { +import com.azure.resourcemanager.security.models.ActionableRemediation; +import com.azure.resourcemanager.security.models.ActionableRemediationState; +import com.azure.resourcemanager.security.models.AzureDevOpsProject; +import com.azure.resourcemanager.security.models.AzureDevOpsProjectProperties; +import com.azure.resourcemanager.security.models.OnboardingState; + +/** + * Samples for AzureDevOpsProjects Update. + */ +public final class AzureDevOpsProjectsUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/GetListConnectorSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/UpdateAzureDevOpsProjects_example.json */ /** - * Sample code: Get all cloud accounts connectors of a subscription. - * + * Sample code: Update_AzureDevOpsProjects. + * * @param manager Entry point to SecurityManager. */ - public static void getAllCloudAccountsConnectorsOfASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.connectors().list(com.azure.core.util.Context.NONE); + public static void updateAzureDevOpsProjects(com.azure.resourcemanager.security.SecurityManager manager) { + AzureDevOpsProject resource = manager.azureDevOpsProjects().getWithResponse("myRg", "mySecurityConnectorName", + "myAzDevOpsOrg", "myAzDevOpsProject", com.azure.core.util.Context.NONE).getValue(); + resource.update() + .withProperties(new AzureDevOpsProjectProperties().withOnboardingState(OnboardingState.NOT_APPLICABLE) + .withActionableRemediation(new ActionableRemediation().withState(ActionableRemediationState.ENABLED))) + .apply(); } } ``` -### CustomAssessmentAutomations_Create +### AzureDevOpsRepos_CreateOrUpdate ```java -import com.azure.resourcemanager.security.models.SeverityEnum; -import com.azure.resourcemanager.security.models.SupportedCloudEnum; +import com.azure.resourcemanager.security.models.ActionableRemediation; +import com.azure.resourcemanager.security.models.ActionableRemediationState; +import com.azure.resourcemanager.security.models.AzureDevOpsRepositoryProperties; +import com.azure.resourcemanager.security.models.OnboardingState; -/** Samples for CustomAssessmentAutomations Create. */ -public final class CustomAssessmentAutomationsCreateSamples { +/** + * Samples for AzureDevOpsRepos CreateOrUpdate. + */ +public final class AzureDevOpsReposCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomAssessmentAutomations/customAssessmentAutomationCreate_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/CreateOrUpdateAzureDevOpsRepos_example.json */ /** - * Sample code: Create a Custom Assessment Automation. - * + * Sample code: CreateOrUpdate_AzureDevOpsRepos. + * * @param manager Entry point to SecurityManager. */ - public static void createACustomAssessmentAutomation(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customAssessmentAutomations() - .define("MyCustomAssessmentAutomation") - .withExistingResourceGroup("TestResourceGroup") - .withCompressedQuery( - "DQAKAEkAYQBtAF8ARwByAG8AdQBwAA0ACgB8ACAAZQB4AHQAZQBuAGQAIABIAGUAYQBsAHQAaABTAHQAYQB0AHUAcwAgAD0AIABpAGYAZgAoAHQAbwBzAHQAcgBpAG4AZwAoAFIAZQBjAG8AcgBkAC4AVQBzAGUAcgBOAGEAbQBlACkAIABjAG8AbgB0AGEAaQBuAHMAIAAnAHUAcwBlAHIAJwAsACAAJwBVAE4ASABFAEEATABUAEgAWQAnACwAIAAnAEgARQBBAEwAVABIAFkAJwApAA0ACgA=") - .withSupportedCloud(SupportedCloudEnum.AWS) - .withSeverity(SeverityEnum.MEDIUM) - .withDisplayName("Password Policy") - .withDescription("Data should be encrypted") - .withRemediationDescription("Encrypt store by...") + public static void createOrUpdateAzureDevOpsRepos(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsRepos().define("myAzDevOpsRepo") + .withExistingProject("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", "myAzDevOpsProject") + .withProperties(new AzureDevOpsRepositoryProperties().withOnboardingState(OnboardingState.NOT_APPLICABLE) + .withActionableRemediation(new ActionableRemediation().withState(ActionableRemediationState.ENABLED))) .create(); } } ``` -### CustomAssessmentAutomations_Delete +### AzureDevOpsRepos_Get ```java -/** Samples for CustomAssessmentAutomations Delete. */ -public final class CustomAssessmentAutomationsDeleteSamples { +/** + * Samples for AzureDevOpsRepos Get. + */ +public final class AzureDevOpsReposGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomAssessmentAutomations/customAssessmentAutomationDelete_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetAzureDevOpsRepos_example.json */ /** - * Sample code: Delete a Custom Assessment Automation. - * + * Sample code: Get_AzureDevOpsRepos. + * * @param manager Entry point to SecurityManager. */ - public static void deleteACustomAssessmentAutomation(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customAssessmentAutomations() - .deleteByResourceGroupWithResponse( - "TestResourceGroup", "MyCustomAssessmentAutomation", com.azure.core.util.Context.NONE); + public static void getAzureDevOpsRepos(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsRepos().getWithResponse("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", + "myAzDevOpsProject", "myAzDevOpsRepo", com.azure.core.util.Context.NONE); } } ``` -### CustomAssessmentAutomations_GetByResourceGroup +### AzureDevOpsRepos_List ```java -/** Samples for CustomAssessmentAutomations GetByResourceGroup. */ -public final class CustomAssessmentAutomationsGetByResourceGroupSamples { +/** + * Samples for AzureDevOpsRepos List. + */ +public final class AzureDevOpsReposListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomAssessmentAutomations/customAssessmentAutomationGet_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListAzureDevOpsRepos_example.json */ /** - * Sample code: Get a Custom Assessment Automation. - * + * Sample code: List_AzureDevOpsRepos. + * * @param manager Entry point to SecurityManager. */ - public static void getACustomAssessmentAutomation(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customAssessmentAutomations() - .getByResourceGroupWithResponse( - "TestResourceGroup", "MyCustomAssessmentAutomation", com.azure.core.util.Context.NONE); + public static void listAzureDevOpsRepos(com.azure.resourcemanager.security.SecurityManager manager) { + manager.azureDevOpsRepos().list("myRg", "mySecurityConnectorName", "myAzDevOpsOrg", "myAzDevOpsProject", + com.azure.core.util.Context.NONE); } } ``` -### CustomAssessmentAutomations_List +### AzureDevOpsRepos_Update ```java -/** Samples for CustomAssessmentAutomations List. */ -public final class CustomAssessmentAutomationsListSamples { +import com.azure.resourcemanager.security.models.ActionableRemediation; +import com.azure.resourcemanager.security.models.ActionableRemediationState; +import com.azure.resourcemanager.security.models.AzureDevOpsRepository; +import com.azure.resourcemanager.security.models.AzureDevOpsRepositoryProperties; +import com.azure.resourcemanager.security.models.OnboardingState; + +/** + * Samples for AzureDevOpsRepos Update. + */ +public final class AzureDevOpsReposUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomAssessmentAutomations/customAssessmentAutomationListBySubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/UpdateAzureDevOpsRepos_example.json */ /** - * Sample code: List Custom Assessment Automations in a subscription. - * + * Sample code: Update_AzureDevOpsRepos. + * * @param manager Entry point to SecurityManager. */ - public static void listCustomAssessmentAutomationsInASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.customAssessmentAutomations().list(com.azure.core.util.Context.NONE); + public static void updateAzureDevOpsRepos(com.azure.resourcemanager.security.SecurityManager manager) { + AzureDevOpsRepository resource = manager.azureDevOpsRepos().getWithResponse("myRg", "mySecurityConnectorName", + "myAzDevOpsOrg", "myAzDevOpsProject", "myAzDevOpsRepo", com.azure.core.util.Context.NONE).getValue(); + resource.update() + .withProperties(new AzureDevOpsRepositoryProperties().withOnboardingState(OnboardingState.NOT_APPLICABLE) + .withActionableRemediation(new ActionableRemediation().withState(ActionableRemediationState.ENABLED))) + .apply(); } } ``` -### CustomAssessmentAutomations_ListByResourceGroup +### ComplianceResults_Get ```java -/** Samples for CustomAssessmentAutomations ListByResourceGroup. */ -public final class CustomAssessmentAutomationsListByResourceGroupSamples { +/** + * Samples for ComplianceResults Get. + */ +public final class ComplianceResultsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomAssessmentAutomations/customAssessmentAutomationListByResourceGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2017-08-01/examples/ComplianceResults/ + * GetComplianceResults_example.json */ /** - * Sample code: List Custom Assessment Automations in a subscription and a resource group. - * + * Sample code: Get compliance results on subscription. + * * @param manager Entry point to SecurityManager. */ - public static void listCustomAssessmentAutomationsInASubscriptionAndAResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customAssessmentAutomations() - .listByResourceGroup("TestResourceGroup", com.azure.core.util.Context.NONE); + public static void getComplianceResultsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.complianceResults().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "DesignateMoreThanOneOwner", com.azure.core.util.Context.NONE); } } ``` -### CustomEntityStoreAssignments_Create +### ComplianceResults_List ```java -/** Samples for CustomEntityStoreAssignments Create. */ -public final class CustomEntityStoreAssignmentsCreateSamples { +/** + * Samples for ComplianceResults List. + */ +public final class ComplianceResultsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomEntityStoreAssignments/customEntityStoreAssignmentCreate_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2017-08-01/examples/ComplianceResults/ + * ListComplianceResults_example.json */ /** - * Sample code: Create a custom entity store assignment. - * + * Sample code: Get compliance results on subscription. + * * @param manager Entry point to SecurityManager. */ - public static void createACustomEntityStoreAssignment(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customEntityStoreAssignments() - .define("33e7cc6e-a139-4723-a0e5-76993aee0771") - .withExistingResourceGroup("TestResourceGroup") - .withPrincipal("aaduser=f3923a3e-ad57-4752-b1a9-fbf3c8e5e082;72f988bf-86f1-41af-91ab-2d7cd011db47") - .create(); + public static void getComplianceResultsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.complianceResults().list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + com.azure.core.util.Context.NONE); + } +} +``` + +### Compliances_Get + +```java +/** + * Samples for Compliances Get. + */ +public final class CompliancesGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/Compliances/ + * GetCompliance_example.json + */ + /** + * Sample code: Get security compliance data for a day. + * + * @param manager Entry point to SecurityManager. + */ + public static void getSecurityComplianceDataForADay(com.azure.resourcemanager.security.SecurityManager manager) { + manager.compliances().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "2018-01-01Z", + com.azure.core.util.Context.NONE); + } +} +``` + +### Compliances_List + +```java +/** + * Samples for Compliances List. + */ +public final class CompliancesListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/Compliances/ + * GetCompliances_example.json + */ + /** + * Sample code: Get security compliance data over time. + * + * @param manager Entry point to SecurityManager. + */ + public static void getSecurityComplianceDataOverTime(com.azure.resourcemanager.security.SecurityManager manager) { + manager.compliances().list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + com.azure.core.util.Context.NONE); + } +} +``` + +### Connectors_CreateOrUpdate + +```java +import com.azure.resourcemanager.security.models.AutoProvision; +import com.azure.resourcemanager.security.models.AwAssumeRoleAuthenticationDetailsProperties; +import com.azure.resourcemanager.security.models.AwsCredsAuthenticationDetailsProperties; +import com.azure.resourcemanager.security.models.GcpCredentialsDetailsProperties; +import com.azure.resourcemanager.security.models.HybridComputeSettingsProperties; +import com.azure.resourcemanager.security.models.ProxyServerProperties; +import com.azure.resourcemanager.security.models.ServicePrincipalProperties; + +/** + * Samples for Connectors CreateOrUpdate. + */ +public final class ConnectorsCreateOrUpdateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/ + * CreateUpdateGcpCredentialsConnectorSubscription_example.json + */ + /** + * Sample code: gcpCredentials - Create a cloud account connector for a subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void gcpCredentialsCreateACloudAccountConnectorForASubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.connectors().define("gcp_dev") + .withHybridComputeSettings(new HybridComputeSettingsProperties().withAutoProvision(AutoProvision.OFF)) + .withAuthenticationDetails(new GcpCredentialsDetailsProperties().withOrganizationId("AscDemoOrg") + .withType("service_account").withProjectId("asc-project-1234").withPrivateKeyId("fakeTokenPlaceholder") + .withPrivateKey("fakeTokenPlaceholder") + .withClientEmail("asc-135@asc-project-1234.iam.gserviceaccount.com") + .withClientId("105889053725632919854").withAuthUri("https://accounts.google.com/o/oauth2/auth") + .withTokenUri("fakeTokenPlaceholder") + .withAuthProviderX509CertUrl("https://www.googleapis.com/oauth2/v1/certs").withClientX509CertUrl( + "https://www.googleapis.com/robot/v1/metadata/x509/asc-135%40asc-project-1234.iam.gserviceaccount.com")) + .create(); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/ + * CreateUpdateAwsCredConnectorSubscription_example.json + */ + /** + * Sample code: AwsCred - Create a cloud account connector for a subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void awsCredCreateACloudAccountConnectorForASubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.connectors().define("aws_dev1") + .withHybridComputeSettings(new HybridComputeSettingsProperties().withAutoProvision(AutoProvision.ON) + .withResourceGroupName("AwsConnectorRG").withRegion("West US 2") + .withProxyServer(new ProxyServerProperties().withIp("167.220.197.140").withPort("34")) + .withServicePrincipal(new ServicePrincipalProperties() + .withApplicationId("ad9bcd79-be9c-45ab-abd8-80ca1654a7d1").withSecret("fakeTokenPlaceholder"))) + .withAuthenticationDetails(new AwsCredsAuthenticationDetailsProperties() + .withAwsAccessKeyId("fakeTokenPlaceholder").withAwsSecretAccessKey("fakeTokenPlaceholder")) + .create(); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/ + * CreateUpdateAwsAssumeRoleConnectorSubscription_example.json + */ + /** + * Sample code: AwsAssumeRole - Create a cloud account connector for a subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void awsAssumeRoleCreateACloudAccountConnectorForASubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.connectors().define("aws_dev2") + .withHybridComputeSettings(new HybridComputeSettingsProperties().withAutoProvision(AutoProvision.ON) + .withResourceGroupName("AwsConnectorRG").withRegion("West US 2") + .withProxyServer(new ProxyServerProperties().withIp("167.220.197.140").withPort("34")) + .withServicePrincipal(new ServicePrincipalProperties() + .withApplicationId("ad9bcd79-be9c-45ab-abd8-80ca1654a7d1").withSecret("fakeTokenPlaceholder"))) + .withAuthenticationDetails(new AwAssumeRoleAuthenticationDetailsProperties() + .withAwsAssumeRoleArn("arn:aws:iam::81231569658:role/AscConnector") + .withAwsExternalId("20ff7fc3-e762-44dd-bd96-b71116dcdc23")) + .create(); + } +} +``` + +### Connectors_Delete + +```java +/** + * Samples for Connectors Delete. + */ +public final class ConnectorsDeleteSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/ + * DeleteConnectorSubscription_example.json + */ + /** + * Sample code: Delete a cloud account connector from a subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void + deleteACloudAccountConnectorFromASubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.connectors().deleteWithResponse("aws_dev1", com.azure.core.util.Context.NONE); + } +} +``` + +### Connectors_Get + +```java +/** + * Samples for Connectors Get. + */ +public final class ConnectorsGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/ + * GetConnectorSubscription_example.json + */ + /** + * Sample code: Details of a specific cloud account connector. + * + * @param manager Entry point to SecurityManager. + */ + public static void + detailsOfASpecificCloudAccountConnector(com.azure.resourcemanager.security.SecurityManager manager) { + manager.connectors().getWithResponse("aws_dev1", com.azure.core.util.Context.NONE); + } +} +``` + +### Connectors_List + +```java +/** + * Samples for Connectors List. + */ +public final class ConnectorsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/Connectors/ + * GetListConnectorSubscription_example.json + */ + /** + * Sample code: Get all cloud accounts connectors of a subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void + getAllCloudAccountsConnectorsOfASubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.connectors().list(com.azure.core.util.Context.NONE); + } +} +``` + +### CustomAssessmentAutomations_Create + +```java +import com.azure.resourcemanager.security.models.SeverityEnum; +import com.azure.resourcemanager.security.models.SupportedCloudEnum; + +/** + * Samples for CustomAssessmentAutomations Create. + */ +public final class CustomAssessmentAutomationsCreateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomAssessmentAutomations/customAssessmentAutomationCreate_example.json + */ + /** + * Sample code: Create a Custom Assessment Automation. + * + * @param manager Entry point to SecurityManager. + */ + public static void createACustomAssessmentAutomation(com.azure.resourcemanager.security.SecurityManager manager) { + manager.customAssessmentAutomations().define("MyCustomAssessmentAutomation") + .withExistingResourceGroup("TestResourceGroup") + .withCompressedQuery( + "DQAKAEkAYQBtAF8ARwByAG8AdQBwAA0ACgB8ACAAZQB4AHQAZQBuAGQAIABIAGUAYQBsAHQAaABTAHQAYQB0AHUAcwAgAD0AIABpAGYAZgAoAHQAbwBzAHQAcgBpAG4AZwAoAFIAZQBjAG8AcgBkAC4AVQBzAGUAcgBOAGEAbQBlACkAIABjAG8AbgB0AGEAaQBuAHMAIAAnAHUAcwBlAHIAJwAsACAAJwBVAE4ASABFAEEATABUAEgAWQAnACwAIAAnAEgARQBBAEwAVABIAFkAJwApAA0ACgA=") + .withSupportedCloud(SupportedCloudEnum.AWS).withSeverity(SeverityEnum.MEDIUM) + .withDisplayName("Password Policy").withDescription("Data should be encrypted") + .withRemediationDescription("Encrypt store by...").create(); + } +} +``` + +### CustomAssessmentAutomations_Delete + +```java +/** + * Samples for CustomAssessmentAutomations Delete. + */ +public final class CustomAssessmentAutomationsDeleteSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomAssessmentAutomations/customAssessmentAutomationDelete_example.json + */ + /** + * Sample code: Delete a Custom Assessment Automation. + * + * @param manager Entry point to SecurityManager. + */ + public static void deleteACustomAssessmentAutomation(com.azure.resourcemanager.security.SecurityManager manager) { + manager.customAssessmentAutomations().deleteByResourceGroupWithResponse("TestResourceGroup", + "MyCustomAssessmentAutomation", com.azure.core.util.Context.NONE); + } +} +``` + +### CustomAssessmentAutomations_GetByResourceGroup + +```java +/** + * Samples for CustomAssessmentAutomations GetByResourceGroup. + */ +public final class CustomAssessmentAutomationsGetByResourceGroupSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomAssessmentAutomations/customAssessmentAutomationGet_example.json + */ + /** + * Sample code: Get a Custom Assessment Automation. + * + * @param manager Entry point to SecurityManager. + */ + public static void getACustomAssessmentAutomation(com.azure.resourcemanager.security.SecurityManager manager) { + manager.customAssessmentAutomations().getByResourceGroupWithResponse("TestResourceGroup", + "MyCustomAssessmentAutomation", com.azure.core.util.Context.NONE); + } +} +``` + +### CustomAssessmentAutomations_List + +```java +/** + * Samples for CustomAssessmentAutomations List. + */ +public final class CustomAssessmentAutomationsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomAssessmentAutomations/customAssessmentAutomationListBySubscription_example.json + */ + /** + * Sample code: List Custom Assessment Automations in a subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void + listCustomAssessmentAutomationsInASubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.customAssessmentAutomations().list(com.azure.core.util.Context.NONE); + } +} +``` + +### CustomAssessmentAutomations_ListByResourceGroup + +```java +/** + * Samples for CustomAssessmentAutomations ListByResourceGroup. + */ +public final class CustomAssessmentAutomationsListByResourceGroupSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomAssessmentAutomations/customAssessmentAutomationListByResourceGroup_example.json + */ + /** + * Sample code: List Custom Assessment Automations in a subscription and a resource group. + * + * @param manager Entry point to SecurityManager. + */ + public static void listCustomAssessmentAutomationsInASubscriptionAndAResourceGroup( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.customAssessmentAutomations().listByResourceGroup("TestResourceGroup", + com.azure.core.util.Context.NONE); + } +} +``` + +### CustomEntityStoreAssignments_Create + +```java +/** + * Samples for CustomEntityStoreAssignments Create. + */ +public final class CustomEntityStoreAssignmentsCreateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomEntityStoreAssignments/customEntityStoreAssignmentCreate_example.json + */ + /** + * Sample code: Create a custom entity store assignment. + * + * @param manager Entry point to SecurityManager. + */ + public static void createACustomEntityStoreAssignment(com.azure.resourcemanager.security.SecurityManager manager) { + manager.customEntityStoreAssignments().define("33e7cc6e-a139-4723-a0e5-76993aee0771") + .withExistingResourceGroup("TestResourceGroup") + .withPrincipal("aaduser=f3923a3e-ad57-4752-b1a9-fbf3c8e5e082;72f988bf-86f1-41af-91ab-2d7cd011db47") + .create(); } } ``` @@ -2595,21 +3074,23 @@ public final class CustomEntityStoreAssignmentsCreateSamples { ### CustomEntityStoreAssignments_Delete ```java -/** Samples for CustomEntityStoreAssignments Delete. */ +/** + * Samples for CustomEntityStoreAssignments Delete. + */ public final class CustomEntityStoreAssignmentsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomEntityStoreAssignments/customEntityStoreAssignmentDelete_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomEntityStoreAssignments/customEntityStoreAssignmentDelete_example.json */ /** * Sample code: Delete a custom entity store assignment. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteACustomEntityStoreAssignment(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customEntityStoreAssignments() - .deleteByResourceGroupWithResponse( - "TestResourceGroup", "33e7cc6e-a139-4723-a0e5-76993aee0771", com.azure.core.util.Context.NONE); + manager.customEntityStoreAssignments().deleteByResourceGroupWithResponse("TestResourceGroup", + "33e7cc6e-a139-4723-a0e5-76993aee0771", com.azure.core.util.Context.NONE); } } ``` @@ -2617,21 +3098,23 @@ public final class CustomEntityStoreAssignmentsDeleteSamples { ### CustomEntityStoreAssignments_GetByResourceGroup ```java -/** Samples for CustomEntityStoreAssignments GetByResourceGroup. */ +/** + * Samples for CustomEntityStoreAssignments GetByResourceGroup. + */ public final class CustomEntityStoreAssignmentsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomEntityStoreAssignments/customEntityStoreAssignmentGet_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomEntityStoreAssignments/customEntityStoreAssignmentGet_example.json */ /** * Sample code: Get a custom entity store assignment. - * + * * @param manager Entry point to SecurityManager. */ public static void getACustomEntityStoreAssignment(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customEntityStoreAssignments() - .getByResourceGroupWithResponse( - "TestResourceGroup", "33e7cc6e-a139-4723-a0e5-76993aee0771", com.azure.core.util.Context.NONE); + manager.customEntityStoreAssignments().getByResourceGroupWithResponse("TestResourceGroup", + "33e7cc6e-a139-4723-a0e5-76993aee0771", com.azure.core.util.Context.NONE); } } ``` @@ -2639,18 +3122,22 @@ public final class CustomEntityStoreAssignmentsGetByResourceGroupSamples { ### CustomEntityStoreAssignments_List ```java -/** Samples for CustomEntityStoreAssignments List. */ +/** + * Samples for CustomEntityStoreAssignments List. + */ public final class CustomEntityStoreAssignmentsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomEntityStoreAssignments/customEntityStoreAssignmentListBySubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomEntityStoreAssignments/customEntityStoreAssignmentListBySubscription_example.json */ /** * Sample code: List custom entity store assignments in a subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void listCustomEntityStoreAssignmentsInASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listCustomEntityStoreAssignmentsInASubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.customEntityStoreAssignments().list(com.azure.core.util.Context.NONE); } } @@ -2658,22 +3145,296 @@ public final class CustomEntityStoreAssignmentsListSamples { ### CustomEntityStoreAssignments_ListByResourceGroup -```java -/** Samples for CustomEntityStoreAssignments ListByResourceGroup. */ -public final class CustomEntityStoreAssignmentsListByResourceGroupSamples { +```java +/** + * Samples for CustomEntityStoreAssignments ListByResourceGroup. + */ +public final class CustomEntityStoreAssignmentsListByResourceGroupSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/ + * CustomEntityStoreAssignments/customEntityStoreAssignmentListByResourceGroup_example.json + */ + /** + * Sample code: List custom entity store assignments in a subscription and a resource group. + * + * @param manager Entry point to SecurityManager. + */ + public static void listCustomEntityStoreAssignmentsInASubscriptionAndAResourceGroup( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.customEntityStoreAssignments().listByResourceGroup("TestResourceGroup", + com.azure.core.util.Context.NONE); + } +} +``` + +### DefenderForStorage_Create + +```java +import com.azure.resourcemanager.security.models.SettingName; + +/** + * Samples for DefenderForStorage Create. + */ +public final class DefenderForStorageCreateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-12-01-preview/examples/DefenderForStorage + * /PutDefenderForStorageSettings_example.json + */ + /** + * Sample code: Creates or updates the Defender for Storage settings on a specified resource. + * + * @param manager Entry point to SecurityManager. + */ + public static void createsOrUpdatesTheDefenderForStorageSettingsOnASpecifiedResource( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.defenderForStorages().define(SettingName.CURRENT).withExistingResourceId( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Storage/storageAccounts/samplestorageaccount") + .withIsEnabled(true).withOverrideSubscriptionLevelSettings(true) + .withScanResultsEventGridTopicResourceId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.EventGrid/topics/sampletopic") + .withIsEnabledMalwareScanningIsEnabled(true).withCapGBPerMonth(-1) + .withIsEnabledSensitiveDataDiscoveryIsEnabled(true).create(); + } +} +``` + +### DefenderForStorage_Get + +```java +import com.azure.resourcemanager.security.models.SettingName; + +/** + * Samples for DefenderForStorage Get. + */ +public final class DefenderForStorageGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-12-01-preview/examples/DefenderForStorage + * /GetDefenderForStorageSettings_example.json + */ + /** + * Sample code: Gets the Defender for Storage settings for the specified resource. + * + * @param manager Entry point to SecurityManager. + */ + public static void getsTheDefenderForStorageSettingsForTheSpecifiedResource( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.defenderForStorages().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Storage/storageAccounts/samplestorageaccount", + SettingName.CURRENT, com.azure.core.util.Context.NONE); + } +} +``` + +### DevOpsConfigurations_CreateOrUpdate + +```java +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; +import com.azure.resourcemanager.security.models.Authorization; +import com.azure.resourcemanager.security.models.AutoDiscovery; +import com.azure.resourcemanager.security.models.DevOpsConfigurationProperties; +import java.util.Arrays; + +/** + * Samples for DevOpsConfigurations CreateOrUpdate. + */ +public final class DevOpsConfigurationsCreateOrUpdateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/CreateOrUpdateDevOpsConfigurationsOnboardCurrentAndFuture_example.json + */ + /** + * Sample code: CreateOrUpdate_DevOpsConfigurations_OnboardCurrentAndFuture. + * + * @param manager Entry point to SecurityManager. + */ + public static void createOrUpdateDevOpsConfigurationsOnboardCurrentAndFuture( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().createOrUpdate("myRg", "mySecurityConnectorName", + new DevOpsConfigurationInner().withProperties(new DevOpsConfigurationProperties() + .withAuthorization(new Authorization().withCode("fakeTokenPlaceholder")) + .withAutoDiscovery(AutoDiscovery.ENABLED)), + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/CreateOrUpdateDevOpsConfigurationsOnboardCurrentOnly_example.json + */ + /** + * Sample code: CreateOrUpdate_DevOpsConfigurations_OnboardCurrentOnly. + * + * @param manager Entry point to SecurityManager. + */ + public static void createOrUpdateDevOpsConfigurationsOnboardCurrentOnly( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().createOrUpdate("myRg", "mySecurityConnectorName", + new DevOpsConfigurationInner().withProperties(new DevOpsConfigurationProperties() + .withAuthorization(new Authorization().withCode("fakeTokenPlaceholder")) + .withAutoDiscovery(AutoDiscovery.DISABLED)), + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/CreateOrUpdateDevOpsConfigurationsOnboardSelected_example.json + */ + /** + * Sample code: CreateOrUpdate_DevOpsConfigurations_OnboardSelected. + * + * @param manager Entry point to SecurityManager. + */ + public static void + createOrUpdateDevOpsConfigurationsOnboardSelected(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().createOrUpdate("myRg", "mySecurityConnectorName", + new DevOpsConfigurationInner().withProperties(new DevOpsConfigurationProperties() + .withAuthorization(new Authorization().withCode("fakeTokenPlaceholder")) + .withAutoDiscovery(AutoDiscovery.DISABLED).withTopLevelInventoryList(Arrays.asList("org1", "org2"))), + com.azure.core.util.Context.NONE); + } +} +``` + +### DevOpsConfigurations_Delete + +```java +/** + * Samples for DevOpsConfigurations Delete. + */ +public final class DevOpsConfigurationsDeleteSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/DeleteDevOpsConfigurations_example.json + */ + /** + * Sample code: Delete_DevOpsConfigurations. + * + * @param manager Entry point to SecurityManager. + */ + public static void deleteDevOpsConfigurations(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().delete("myRg", "mySecurityConnectorName", com.azure.core.util.Context.NONE); + } +} +``` + +### DevOpsConfigurations_Get + +```java +/** + * Samples for DevOpsConfigurations Get. + */ +public final class DevOpsConfigurationsGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetDevOpsConfigurations_example.json + */ + /** + * Sample code: Get_DevOpsConfigurations. + * + * @param manager Entry point to SecurityManager. + */ + public static void getDevOpsConfigurations(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().getWithResponse("myRg", "mySecurityConnectorName", + com.azure.core.util.Context.NONE); + } +} +``` + +### DevOpsConfigurations_List + +```java +/** + * Samples for DevOpsConfigurations List. + */ +public final class DevOpsConfigurationsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListDevOpsConfigurations_example.json + */ + /** + * Sample code: List_DevOpsConfigurations. + * + * @param manager Entry point to SecurityManager. + */ + public static void listDevOpsConfigurations(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().list("myRg", "mySecurityConnectorName", com.azure.core.util.Context.NONE); + } +} +``` + +### DevOpsConfigurations_Update + +```java +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; +import com.azure.resourcemanager.security.models.AutoDiscovery; +import com.azure.resourcemanager.security.models.DevOpsConfigurationProperties; + +/** + * Samples for DevOpsConfigurations Update. + */ +public final class DevOpsConfigurationsUpdateSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/UpdateDevOpsConfigurations_example.json + */ + /** + * Sample code: Update_DevOpsConfigurations. + * + * @param manager Entry point to SecurityManager. + */ + public static void updateDevOpsConfigurations(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsConfigurations().update("myRg", "mySecurityConnectorName", + new DevOpsConfigurationInner() + .withProperties(new DevOpsConfigurationProperties().withAutoDiscovery(AutoDiscovery.ENABLED)), + com.azure.core.util.Context.NONE); + } +} +``` + +### DevOpsOperationResults_Get + +```java +/** + * Samples for DevOpsOperationResults Get. + */ +public final class DevOpsOperationResultsGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetDevOpsOperationResultsFailed_example.json + */ + /** + * Sample code: Get_DevOpsOperationResults_Failed. + * + * @param manager Entry point to SecurityManager. + */ + public static void getDevOpsOperationResultsFailed(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsOperationResults().getWithResponse("myRg", "mySecurityConnectorName", + "8d4caace-e7b3-4b3e-af99-73f76829ebcf", com.azure.core.util.Context.NONE); + } + /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-07-01-preview/examples/CustomEntityStoreAssignments/customEntityStoreAssignmentListByResourceGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetDevOpsOperationResultsSucceeded_example.json */ /** - * Sample code: List custom entity store assignments in a subscription and a resource group. - * + * Sample code: Get_DevOpsOperationResults_Succeeded. + * * @param manager Entry point to SecurityManager. */ - public static void listCustomEntityStoreAssignmentsInASubscriptionAndAResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .customEntityStoreAssignments() - .listByResourceGroup("TestResourceGroup", com.azure.core.util.Context.NONE); + public static void getDevOpsOperationResultsSucceeded(com.azure.resourcemanager.security.SecurityManager manager) { + manager.devOpsOperationResults().getWithResponse("myRg", "mySecurityConnectorName", + "4e826cf1-5c36-4808-a7d2-fb4f5170978b", com.azure.core.util.Context.NONE); } } ``` @@ -2685,31 +3446,26 @@ import com.azure.resourcemanager.security.models.ActiveConnectionsNotInAllowedRa import java.time.Duration; import java.util.Arrays; -/** Samples for DeviceSecurityGroups CreateOrUpdate. */ +/** + * Samples for DeviceSecurityGroups CreateOrUpdate. + */ public final class DeviceSecurityGroupsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/PutDeviceSecurityGroups_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/ + * PutDeviceSecurityGroups_example.json */ /** * Sample code: Create or update a device security group for the specified IoT hub resource. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateADeviceSecurityGroupForTheSpecifiedIoTHubResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .deviceSecurityGroups() - .define("samplesecuritygroup") - .withExistingResourceId( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub") - .withTimeWindowRules( - Arrays - .asList( - new ActiveConnectionsNotInAllowedRange() - .withIsEnabled(true) - .withMinThreshold(0) - .withMaxThreshold(30) - .withTimeWindowSize(Duration.parse("PT05M")))) + manager.deviceSecurityGroups().define("samplesecuritygroup").withExistingResourceId( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub") + .withTimeWindowRules(Arrays.asList(new ActiveConnectionsNotInAllowedRange().withIsEnabled(true) + .withMinThreshold(0).withMaxThreshold(30).withTimeWindowSize(Duration.parse("PT05M")))) .create(); } } @@ -2718,24 +3474,25 @@ public final class DeviceSecurityGroupsCreateOrUpdateSamples { ### DeviceSecurityGroups_Delete ```java -/** Samples for DeviceSecurityGroups Delete. */ +/** + * Samples for DeviceSecurityGroups Delete. + */ public final class DeviceSecurityGroupsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/DeleteDeviceSecurityGroups_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/ + * DeleteDeviceSecurityGroups_example.json */ /** * Sample code: Delete a device security group for the specified IoT Hub resource. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteADeviceSecurityGroupForTheSpecifiedIoTHubResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .deviceSecurityGroups() - .deleteByResourceGroupWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub", - "samplesecuritygroup", - com.azure.core.util.Context.NONE); + manager.deviceSecurityGroups().deleteByResourceGroupWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub", + "samplesecuritygroup", com.azure.core.util.Context.NONE); } } ``` @@ -2743,24 +3500,25 @@ public final class DeviceSecurityGroupsDeleteSamples { ### DeviceSecurityGroups_Get ```java -/** Samples for DeviceSecurityGroups Get. */ +/** + * Samples for DeviceSecurityGroups Get. + */ public final class DeviceSecurityGroupsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/GetDeviceSecurityGroups_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/ + * GetDeviceSecurityGroups_example.json */ /** * Sample code: Get a device security group for the specified IoT Hub resource. - * + * * @param manager Entry point to SecurityManager. */ public static void getADeviceSecurityGroupForTheSpecifiedIoTHubResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .deviceSecurityGroups() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub", - "samplesecuritygroup", - com.azure.core.util.Context.NONE); + manager.deviceSecurityGroups().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub", + "samplesecuritygroup", com.azure.core.util.Context.NONE); } } ``` @@ -2768,23 +3526,25 @@ public final class DeviceSecurityGroupsGetSamples { ### DeviceSecurityGroups_List ```java -/** Samples for DeviceSecurityGroups List. */ +/** + * Samples for DeviceSecurityGroups List. + */ public final class DeviceSecurityGroupsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/ListDeviceSecurityGroups_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/DeviceSecurityGroups/ + * ListDeviceSecurityGroups_example.json */ /** * Sample code: List all device security groups for the specified IoT Hub resource. - * + * * @param manager Entry point to SecurityManager. */ public static void listAllDeviceSecurityGroupsForTheSpecifiedIoTHubResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .deviceSecurityGroups() - .list( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub", - com.azure.core.util.Context.NONE); + manager.deviceSecurityGroups().list( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/SampleRG/providers/Microsoft.Devices/iotHubs/sampleiothub", + com.azure.core.util.Context.NONE); } } ``` @@ -2792,21 +3552,24 @@ public final class DeviceSecurityGroupsListSamples { ### DiscoveredSecuritySolutions_Get ```java -/** Samples for DiscoveredSecuritySolutions Get. */ +/** + * Samples for DiscoveredSecuritySolutions Get. + */ public final class DiscoveredSecuritySolutionsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/DiscoveredSecuritySolutions/GetDiscoveredSecuritySolutionResourceGroupLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/DiscoveredSecuritySolutions + * /GetDiscoveredSecuritySolutionResourceGroupLocation_example.json */ /** * Sample code: Get discovered security solution from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getDiscoveredSecuritySolutionFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .discoveredSecuritySolutions() - .getWithResponse("myRg2", "centralus", "paloalto7", com.azure.core.util.Context.NONE); + manager.discoveredSecuritySolutions().getWithResponse("myRg2", "centralus", "paloalto7", + com.azure.core.util.Context.NONE); } } ``` @@ -2814,14 +3577,18 @@ public final class DiscoveredSecuritySolutionsGetSamples { ### DiscoveredSecuritySolutions_List ```java -/** Samples for DiscoveredSecuritySolutions List. */ +/** + * Samples for DiscoveredSecuritySolutions List. + */ public final class DiscoveredSecuritySolutionsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/DiscoveredSecuritySolutions/GetDiscoveredSecuritySolutionsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/DiscoveredSecuritySolutions + * /GetDiscoveredSecuritySolutionsSubscription_example.json */ /** * Sample code: Get discovered security solutions. - * + * * @param manager Entry point to SecurityManager. */ public static void getDiscoveredSecuritySolutions(com.azure.resourcemanager.security.SecurityManager manager) { @@ -2833,14 +3600,18 @@ public final class DiscoveredSecuritySolutionsListSamples { ### DiscoveredSecuritySolutions_ListByHomeRegion ```java -/** Samples for DiscoveredSecuritySolutions ListByHomeRegion. */ +/** + * Samples for DiscoveredSecuritySolutions ListByHomeRegion. + */ public final class DiscoveredSecuritySolutionsListByHomeRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/DiscoveredSecuritySolutions/GetDiscoveredSecuritySolutionsSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/DiscoveredSecuritySolutions + * /GetDiscoveredSecuritySolutionsSubscriptionLocation_example.json */ /** * Sample code: Get discovered security solutions from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getDiscoveredSecuritySolutionsFromASecurityDataLocation( @@ -2853,24 +3624,23 @@ public final class DiscoveredSecuritySolutionsListByHomeRegionSamples { ### ExternalSecuritySolutions_Get ```java -/** Samples for ExternalSecuritySolutions Get. */ +/** + * Samples for ExternalSecuritySolutions Get. + */ public final class ExternalSecuritySolutionsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ExternalSecuritySolutions/GetExternalSecuritySolution_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ExternalSecuritySolutions/ + * GetExternalSecuritySolution_example.json */ /** * Sample code: Get external security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void getExternalSecuritySolution(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .externalSecuritySolutions() - .getWithResponse( - "defaultresourcegroup-eus", - "centralus", - "aad_defaultworkspace-20ff7fc3-e762-44dd-bd96-b71116dcdc23-eus", - com.azure.core.util.Context.NONE); + manager.externalSecuritySolutions().getWithResponse("defaultresourcegroup-eus", "centralus", + "aad_defaultworkspace-20ff7fc3-e762-44dd-bd96-b71116dcdc23-eus", com.azure.core.util.Context.NONE); } } ``` @@ -2878,18 +3648,22 @@ public final class ExternalSecuritySolutionsGetSamples { ### ExternalSecuritySolutions_List ```java -/** Samples for ExternalSecuritySolutions List. */ +/** + * Samples for ExternalSecuritySolutions List. + */ public final class ExternalSecuritySolutionsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ExternalSecuritySolutions/GetExternalSecuritySolutionsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ExternalSecuritySolutions/ + * GetExternalSecuritySolutionsSubscription_example.json */ /** * Sample code: Get external security solutions on a subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getExternalSecuritySolutionsOnASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getExternalSecuritySolutionsOnASubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.externalSecuritySolutions().list(com.azure.core.util.Context.NONE); } } @@ -2898,14 +3672,18 @@ public final class ExternalSecuritySolutionsListSamples { ### ExternalSecuritySolutions_ListByHomeRegion ```java -/** Samples for ExternalSecuritySolutions ListByHomeRegion. */ +/** + * Samples for ExternalSecuritySolutions ListByHomeRegion. + */ public final class ExternalSecuritySolutionsListByHomeRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ExternalSecuritySolutions/GetExternalSecuritySolutionsSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ExternalSecuritySolutions/ + * GetExternalSecuritySolutionsSubscriptionLocation_example.json */ /** * Sample code: Get external security solutions on a subscription from security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getExternalSecuritySolutionsOnASubscriptionFromSecurityDataLocation( @@ -2915,6 +3693,268 @@ public final class ExternalSecuritySolutionsListByHomeRegionSamples { } ``` +### GitHubOwners_Get + +```java +/** + * Samples for GitHubOwners Get. + */ +public final class GitHubOwnersGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetGitHubOwners_example.json + */ + /** + * Sample code: Get_GitHubOwners. + * + * @param manager Entry point to SecurityManager. + */ + public static void getGitHubOwners(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitHubOwners().getWithResponse("myRg", "mySecurityConnectorName", "myGitHubOwner", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitHubOwners_List + +```java +/** + * Samples for GitHubOwners List. + */ +public final class GitHubOwnersListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListGitHubOwners_example.json + */ + /** + * Sample code: List_GitHubOwners. + * + * @param manager Entry point to SecurityManager. + */ + public static void listGitHubOwners(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitHubOwners().list("myRg", "mySecurityConnectorName", com.azure.core.util.Context.NONE); + } +} +``` + +### GitHubOwners_ListAvailable + +```java +/** + * Samples for GitHubOwners ListAvailable. + */ +public final class GitHubOwnersListAvailableSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListAvailableGitHubOwners_example.json + */ + /** + * Sample code: ListAvailable_GitHubOwners. + * + * @param manager Entry point to SecurityManager. + */ + public static void listAvailableGitHubOwners(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitHubOwners().listAvailableWithResponse("myRg", "mySecurityConnectorName", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitHubRepos_Get + +```java +/** + * Samples for GitHubRepos Get. + */ +public final class GitHubReposGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetGitHubRepos_example.json + */ + /** + * Sample code: Get_GitHubRepos. + * + * @param manager Entry point to SecurityManager. + */ + public static void getGitHubRepos(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitHubRepos().getWithResponse("myRg", "mySecurityConnectorName", "myGitHubOwner", "myGitHubRepo", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitHubRepos_List + +```java +/** + * Samples for GitHubRepos List. + */ +public final class GitHubReposListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListGitHubRepos_example.json + */ + /** + * Sample code: List_GitHubRepos. + * + * @param manager Entry point to SecurityManager. + */ + public static void listGitHubRepos(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitHubRepos().list("myRg", "mySecurityConnectorName", "myGitHubOwner", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitLabGroups_Get + +```java +/** + * Samples for GitLabGroups Get. + */ +public final class GitLabGroupsGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetGitLabGroups_example.json + */ + /** + * Sample code: Get_GitLabGroups. + * + * @param manager Entry point to SecurityManager. + */ + public static void getGitLabGroups(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitLabGroups().getWithResponse("myRg", "mySecurityConnectorName", "myGitLabGroup$mySubGroup", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitLabGroups_List + +```java +/** + * Samples for GitLabGroups List. + */ +public final class GitLabGroupsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListGitLabGroups_example.json + */ + /** + * Sample code: List_GitLabGroups. + * + * @param manager Entry point to SecurityManager. + */ + public static void listGitLabGroups(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitLabGroups().list("myRg", "mySecurityConnectorName", com.azure.core.util.Context.NONE); + } +} +``` + +### GitLabGroups_ListAvailable + +```java +/** + * Samples for GitLabGroups ListAvailable. + */ +public final class GitLabGroupsListAvailableSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListAvailableGitLabGroups_example.json + */ + /** + * Sample code: ListAvailable_GitLabGroups. + * + * @param manager Entry point to SecurityManager. + */ + public static void listAvailableGitLabGroups(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitLabGroups().listAvailableWithResponse("myRg", "mySecurityConnectorName", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitLabProjects_Get + +```java +/** + * Samples for GitLabProjects Get. + */ +public final class GitLabProjectsGetSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/GetGitLabProjects_example.json + */ + /** + * Sample code: Get_GitLabProjects. + * + * @param manager Entry point to SecurityManager. + */ + public static void getGitLabProjects(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitLabProjects().getWithResponse("myRg", "mySecurityConnectorName", "myGitLabGroup$mySubGroup", + "myGitLabProject", com.azure.core.util.Context.NONE); + } +} +``` + +### GitLabProjects_List + +```java +/** + * Samples for GitLabProjects List. + */ +public final class GitLabProjectsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListGitLabProjects_example.json + */ + /** + * Sample code: List_GitLabProjects. + * + * @param manager Entry point to SecurityManager. + */ + public static void listGitLabProjects(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitLabProjects().list("myRg", "mySecurityConnectorName", "myGitLabGroup$mySubGroup", + com.azure.core.util.Context.NONE); + } +} +``` + +### GitLabSubgroups_List + +```java +/** + * Samples for GitLabSubgroups List. + */ +public final class GitLabSubgroupsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-09-01-preview/examples/ + * SecurityConnectorsDevOps/ListGitLabSubgroups_example.json + */ + /** + * Sample code: List_GitLabSubgroups. + * + * @param manager Entry point to SecurityManager. + */ + public static void listGitLabSubgroups(com.azure.resourcemanager.security.SecurityManager manager) { + manager.gitLabSubgroups().listWithResponse("myRg", "mySecurityConnectorName", "myGitLabGroup", + com.azure.core.util.Context.NONE); + } +} +``` + ### GovernanceAssignments_CreateOrUpdate ```java @@ -2923,39 +3963,32 @@ import com.azure.resourcemanager.security.models.GovernanceEmailNotification; import com.azure.resourcemanager.security.models.RemediationEta; import java.time.OffsetDateTime; -/** Samples for GovernanceAssignments CreateOrUpdate. */ +/** + * Samples for GovernanceAssignments CreateOrUpdate. + */ public final class GovernanceAssignmentsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceAssignments/PutGovernanceAssignment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/ + * GovernanceAssignments/PutGovernanceAssignment_example.json */ /** * Sample code: Create Governance assignment. - * + * * @param manager Entry point to SecurityManager. */ public static void createGovernanceAssignment(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceAssignments() - .define("6634ff9f-127b-4bf2-8e6e-b1737f5e789c") - .withExistingAssessment( - "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd/resourceGroups/compute_servers/providers/Microsoft.Compute/virtualMachines/win2012", - "6b9421dd-5555-2251-9b3d-2be58e2f82cd") - .withOwner("user@contoso.com") + manager.governanceAssignments().define("6634ff9f-127b-4bf2-8e6e-b1737f5e789c").withExistingAssessment( + "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd/resourceGroups/compute_servers/providers/Microsoft.Compute/virtualMachines/win2012", + "6b9421dd-5555-2251-9b3d-2be58e2f82cd").withOwner("user@contoso.com") .withRemediationDueDate(OffsetDateTime.parse("2022-01-07T13:00:00.0000000Z")) - .withRemediationEta( - new RemediationEta() - .withEta(OffsetDateTime.parse("2022-01-08T13:00:00.0000000Z")) - .withJustification("Justification of ETA")) + .withRemediationEta(new RemediationEta().withEta(OffsetDateTime.parse("2022-01-08T13:00:00.0000000Z")) + .withJustification("Justification of ETA")) .withIsGracePeriod(true) - .withGovernanceEmailNotification( - new GovernanceEmailNotification() - .withDisableManagerEmailNotification(false) - .withDisableOwnerEmailNotification(false)) - .withAdditionalData( - new GovernanceAssignmentAdditionalData() - .withTicketNumber(123123) - .withTicketLink("https://snow.com") - .withTicketStatus("Active")) + .withGovernanceEmailNotification(new GovernanceEmailNotification() + .withDisableManagerEmailNotification(false).withDisableOwnerEmailNotification(false)) + .withAdditionalData(new GovernanceAssignmentAdditionalData().withTicketNumber(123123) + .withTicketLink("https://snow.com").withTicketStatus("Active")) .create(); } } @@ -2964,24 +3997,25 @@ public final class GovernanceAssignmentsCreateOrUpdateSamples { ### GovernanceAssignments_Delete ```java -/** Samples for GovernanceAssignments Delete. */ +/** + * Samples for GovernanceAssignments Delete. + */ public final class GovernanceAssignmentsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceAssignments/DeleteGovernanceAssignment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/ + * GovernanceAssignments/DeleteGovernanceAssignment_example.json */ /** * Sample code: Delete security assignment. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteSecurityAssignment(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceAssignments() - .deleteWithResponse( - "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd/resourceGroups/compute_servers/providers/Microsoft.Compute/virtualMachines/win2012", - "6b9421dd-5555-2251-9b3d-2be58e2f82cd", - "6634ff9f-127b-4bf2-8e6e-b1737f5e789c", - com.azure.core.util.Context.NONE); + manager.governanceAssignments().deleteWithResponse( + "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd/resourceGroups/compute_servers/providers/Microsoft.Compute/virtualMachines/win2012", + "6b9421dd-5555-2251-9b3d-2be58e2f82cd", "6634ff9f-127b-4bf2-8e6e-b1737f5e789c", + com.azure.core.util.Context.NONE); } } ``` @@ -2989,25 +4023,26 @@ public final class GovernanceAssignmentsDeleteSamples { ### GovernanceAssignments_Get ```java -/** Samples for GovernanceAssignments Get. */ +/** + * Samples for GovernanceAssignments Get. + */ public final class GovernanceAssignmentsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceAssignments/GetGovernanceAssignment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/ + * GovernanceAssignments/GetGovernanceAssignment_example.json */ /** * Sample code: Get governanceAssignment by specific governanceAssignmentKey. - * + * * @param manager Entry point to SecurityManager. */ public static void getGovernanceAssignmentBySpecificGovernanceAssignmentKey( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceAssignments() - .getWithResponse( - "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd/resourceGroups/compute_servers/providers/Microsoft.Compute/virtualMachines/win2012", - "6b9421dd-5555-2251-9b3d-2be58e2f82cd", - "6634ff9f-127b-4bf2-8e6e-b1737f5e789c", - com.azure.core.util.Context.NONE); + manager.governanceAssignments().getWithResponse( + "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd/resourceGroups/compute_servers/providers/Microsoft.Compute/virtualMachines/win2012", + "6b9421dd-5555-2251-9b3d-2be58e2f82cd", "6634ff9f-127b-4bf2-8e6e-b1737f5e789c", + com.azure.core.util.Context.NONE); } } ``` @@ -3015,23 +4050,23 @@ public final class GovernanceAssignmentsGetSamples { ### GovernanceAssignments_List ```java -/** Samples for GovernanceAssignments List. */ +/** + * Samples for GovernanceAssignments List. + */ public final class GovernanceAssignmentsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceAssignments/ListGovernanceAssignments_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/ + * GovernanceAssignments/ListGovernanceAssignments_example.json */ /** * Sample code: List governance assignments. - * + * * @param manager Entry point to SecurityManager. */ public static void listGovernanceAssignments(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceAssignments() - .list( - "subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd", - "6b9421dd-5555-2251-9b3d-2be58e2f82cd", - com.azure.core.util.Context.NONE); + manager.governanceAssignments().list("subscriptions/c32e05d9-7207-4e22-bdf4-4f7d9c72e5fd", + "6b9421dd-5555-2251-9b3d-2be58e2f82cd", com.azure.core.util.Context.NONE); } } ``` @@ -3049,136 +4084,92 @@ import com.azure.resourcemanager.security.models.GovernanceRuleType; import java.io.IOException; import java.util.Arrays; -/** Samples for GovernanceRules CreateOrUpdate. */ +/** + * Samples for GovernanceRules CreateOrUpdate. + */ public final class GovernanceRulesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/PutManagementGroupGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * PutManagementGroupGovernanceRule_example.json */ /** * Sample code: Create or update governance rule over management group scope. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateGovernanceRuleOverManagementGroupScope( com.azure.resourcemanager.security.SecurityManager manager) throws IOException { - manager - .governanceRules() - .define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") + manager.governanceRules().define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") .withExistingScope("providers/Microsoft.Management/managementGroups/contoso") - .withDisplayName("Management group rule") - .withDescription("A rule for a management group") - .withRemediationTimeframe("7.00:00:00") - .withIsGracePeriod(true) - .withRulePriority(200) - .withIsDisabled(false) + .withDisplayName("Management group rule").withDescription("A rule for a management group") + .withRemediationTimeframe("7.00:00:00").withIsGracePeriod(true).withRulePriority(200).withIsDisabled(false) .withRuleType(GovernanceRuleType.INTEGRATED) .withSourceResourceType(GovernanceRuleSourceResourceType.ASSESSMENTS) .withExcludedScopes(Arrays.asList("/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23")) - .withConditionSets( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"conditions\":[{\"operator\":\"In\",\"property\":\"$.AssessmentKey\",\"value\":\"[\\\"b1cd27e0-4ecc-4246-939f-49c426d9d72f\\\"," - + " \\\"fe83f80b-073d-4ccf-93d9-6797eb870201\\\"]\"}]}", - Object.class, - SerializerEncoding.JSON))) - .withOwnerSource( - new GovernanceRuleOwnerSource() - .withType(GovernanceRuleOwnerSourceType.MANUALLY) - .withValue("user@contoso.com")) - .withGovernanceEmailNotification( - new GovernanceRuleEmailNotification() - .withDisableManagerEmailNotification(true) - .withDisableOwnerEmailNotification(false)) + .withConditionSets(Arrays.asList(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"conditions\":[{\"operator\":\"In\",\"property\":\"$.AssessmentKey\",\"value\":\"[\\\"b1cd27e0-4ecc-4246-939f-49c426d9d72f\\\", \\\"fe83f80b-073d-4ccf-93d9-6797eb870201\\\"]\"}]}", + Object.class, SerializerEncoding.JSON))) + .withOwnerSource(new GovernanceRuleOwnerSource().withType(GovernanceRuleOwnerSourceType.MANUALLY) + .withValue("user@contoso.com")) + .withGovernanceEmailNotification(new GovernanceRuleEmailNotification() + .withDisableManagerEmailNotification(true).withDisableOwnerEmailNotification(false)) .create(); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/PutSecurityConnectorGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * PutSecurityConnectorGovernanceRule_example.json */ /** * Sample code: Create or update governance rule over security connector scope. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateGovernanceRuleOverSecurityConnectorScope( com.azure.resourcemanager.security.SecurityManager manager) throws IOException { - manager - .governanceRules() - .define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") - .withExistingScope( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector") - .withDisplayName("GCP Admin's rule") - .withDescription("A rule on critical GCP recommendations") - .withRemediationTimeframe("7.00:00:00") - .withIsGracePeriod(true) - .withRulePriority(200) - .withIsDisabled(false) + manager.governanceRules().define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8").withExistingScope( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector") + .withDisplayName("GCP Admin's rule").withDescription("A rule on critical GCP recommendations") + .withRemediationTimeframe("7.00:00:00").withIsGracePeriod(true).withRulePriority(200).withIsDisabled(false) .withRuleType(GovernanceRuleType.INTEGRATED) .withSourceResourceType(GovernanceRuleSourceResourceType.ASSESSMENTS) - .withConditionSets( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"conditions\":[{\"operator\":\"In\",\"property\":\"$.AssessmentKey\",\"value\":\"[\\\"b1cd27e0-4ecc-4246-939f-49c426d9d72f\\\"," - + " \\\"fe83f80b-073d-4ccf-93d9-6797eb870201\\\"]\"}]}", - Object.class, - SerializerEncoding.JSON))) - .withOwnerSource( - new GovernanceRuleOwnerSource() - .withType(GovernanceRuleOwnerSourceType.MANUALLY) - .withValue("user@contoso.com")) - .withGovernanceEmailNotification( - new GovernanceRuleEmailNotification() - .withDisableManagerEmailNotification(true) - .withDisableOwnerEmailNotification(false)) + .withConditionSets(Arrays.asList(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"conditions\":[{\"operator\":\"In\",\"property\":\"$.AssessmentKey\",\"value\":\"[\\\"b1cd27e0-4ecc-4246-939f-49c426d9d72f\\\", \\\"fe83f80b-073d-4ccf-93d9-6797eb870201\\\"]\"}]}", + Object.class, SerializerEncoding.JSON))) + .withOwnerSource(new GovernanceRuleOwnerSource().withType(GovernanceRuleOwnerSourceType.MANUALLY) + .withValue("user@contoso.com")) + .withGovernanceEmailNotification(new GovernanceRuleEmailNotification() + .withDisableManagerEmailNotification(true).withDisableOwnerEmailNotification(false)) .create(); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/PutGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * PutGovernanceRule_example.json */ /** * Sample code: Create or update governance rule over subscription scope. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateGovernanceRuleOverSubscriptionScope( com.azure.resourcemanager.security.SecurityManager manager) throws IOException { - manager - .governanceRules() - .define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") - .withExistingScope("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23") - .withDisplayName("Admin's rule") - .withDescription("A rule for critical recommendations") - .withRemediationTimeframe("7.00:00:00") - .withIsGracePeriod(true) - .withRulePriority(200) - .withIsDisabled(false) + manager.governanceRules().define("ad9a8e26-29d9-4829-bb30-e597a58cdbb8") + .withExistingScope("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23").withDisplayName("Admin's rule") + .withDescription("A rule for critical recommendations").withRemediationTimeframe("7.00:00:00") + .withIsGracePeriod(true).withRulePriority(200).withIsDisabled(false) .withRuleType(GovernanceRuleType.INTEGRATED) .withSourceResourceType(GovernanceRuleSourceResourceType.ASSESSMENTS) - .withConditionSets( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"conditions\":[{\"operator\":\"In\",\"property\":\"$.AssessmentKey\",\"value\":\"[\\\"b1cd27e0-4ecc-4246-939f-49c426d9d72f\\\"," - + " \\\"fe83f80b-073d-4ccf-93d9-6797eb870201\\\"]\"}]}", - Object.class, - SerializerEncoding.JSON))) - .withOwnerSource( - new GovernanceRuleOwnerSource() - .withType(GovernanceRuleOwnerSourceType.MANUALLY) - .withValue("user@contoso.com")) - .withGovernanceEmailNotification( - new GovernanceRuleEmailNotification() - .withDisableManagerEmailNotification(false) - .withDisableOwnerEmailNotification(false)) + .withConditionSets(Arrays.asList(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"conditions\":[{\"operator\":\"In\",\"property\":\"$.AssessmentKey\",\"value\":\"[\\\"b1cd27e0-4ecc-4246-939f-49c426d9d72f\\\", \\\"fe83f80b-073d-4ccf-93d9-6797eb870201\\\"]\"}]}", + Object.class, SerializerEncoding.JSON))) + .withOwnerSource(new GovernanceRuleOwnerSource().withType(GovernanceRuleOwnerSourceType.MANUALLY) + .withValue("user@contoso.com")) + .withGovernanceEmailNotification(new GovernanceRuleEmailNotification() + .withDisableManagerEmailNotification(false).withDisableOwnerEmailNotification(false)) .create(); } } @@ -3187,60 +4178,57 @@ public final class GovernanceRulesCreateOrUpdateSamples { ### GovernanceRules_Delete ```java -/** Samples for GovernanceRules Delete. */ +/** + * Samples for GovernanceRules Delete. + */ public final class GovernanceRulesDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/DeleteManagementGroupGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * DeleteManagementGroupGovernanceRule_example.json */ /** * Sample code: Delete a Governance rule over management group scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteAGovernanceRuleOverManagementGroupScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .delete( - "providers/Microsoft.Management/managementGroups/contoso", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + deleteAGovernanceRuleOverManagementGroupScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().delete("providers/Microsoft.Management/managementGroups/contoso", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/DeleteGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * DeleteGovernanceRule_example.json */ /** * Sample code: Delete a Governance rule over subscription scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteAGovernanceRuleOverSubscriptionScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .delete( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + deleteAGovernanceRuleOverSubscriptionScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().delete("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/DeleteSecurityConnectorGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * DeleteSecurityConnectorGovernanceRule_example.json */ /** * Sample code: Delete a Governance rule over security connector scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteAGovernanceRuleOverSecurityConnectorScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .delete( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + deleteAGovernanceRuleOverSecurityConnectorScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().delete( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } } ``` @@ -3248,63 +4236,58 @@ public final class GovernanceRulesDeleteSamples { ### GovernanceRules_Execute ```java -/** Samples for GovernanceRules Execute. */ + +/** + * Samples for GovernanceRules Execute. + */ public final class GovernanceRulesExecuteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/PostSecurityConnectorGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * PostSecurityConnectorGovernanceRule_example.json */ /** * Sample code: Execute governance rule over security connector scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void executeGovernanceRuleOverSecurityConnectorScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .execute( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - null, - com.azure.core.util.Context.NONE); + public static void + executeGovernanceRuleOverSecurityConnectorScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().execute( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", null, com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/PostGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * PostGovernanceRule_example.json */ /** * Sample code: Execute Governance rule over subscription scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void executeGovernanceRuleOverSubscriptionScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .execute( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - null, - com.azure.core.util.Context.NONE); + public static void + executeGovernanceRuleOverSubscriptionScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().execute("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", null, com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/PostManagementGroupGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * PostManagementGroupGovernanceRule_example.json */ /** * Sample code: Execute governance rule over management group scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void executeGovernanceRuleOverManagementGroupScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .execute( - "providers/Microsoft.Management/managementGroups/contoso", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - null, - com.azure.core.util.Context.NONE); + public static void + executeGovernanceRuleOverManagementGroupScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().execute("providers/Microsoft.Management/managementGroups/contoso", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", null, com.azure.core.util.Context.NONE); } } ``` @@ -3312,60 +4295,57 @@ public final class GovernanceRulesExecuteSamples { ### GovernanceRules_Get ```java -/** Samples for GovernanceRules Get. */ +/** + * Samples for GovernanceRules Get. + */ public final class GovernanceRulesGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/GetManagementGroupGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * GetManagementGroupGovernanceRule_example.json */ /** * Sample code: Get a governance rule over management group scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void getAGovernanceRuleOverManagementGroupScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .getWithResponse( - "providers/Microsoft.Management/managementGroups/contoso", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + getAGovernanceRuleOverManagementGroupScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().getWithResponse("providers/Microsoft.Management/managementGroups/contoso", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/GetGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * GetGovernanceRule_example.json */ /** * Sample code: Get a governance rule over subscription scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void getAGovernanceRuleOverSubscriptionScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + getAGovernanceRuleOverSubscriptionScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/GetSecurityConnectorGovernanceRule_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * GetSecurityConnectorGovernanceRule_example.json */ /** * Sample code: Get a governance rule over security connector scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void getAGovernanceRuleOverSecurityConnectorScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .getWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + getAGovernanceRuleOverSecurityConnectorScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } } ``` @@ -3373,53 +4353,57 @@ public final class GovernanceRulesGetSamples { ### GovernanceRules_List ```java -/** Samples for GovernanceRules List. */ +/** + * Samples for GovernanceRules List. + */ public final class GovernanceRulesListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ListBySubscriptionGovernanceRules_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * ListBySubscriptionGovernanceRules_example.json */ /** * Sample code: List governance rules by subscription scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void listGovernanceRulesBySubscriptionScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", com.azure.core.util.Context.NONE); + public static void + listGovernanceRulesBySubscriptionScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ListBySecurityConnectorGovernanceRules_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * ListBySecurityConnectorGovernanceRules_example.json */ /** * Sample code: List governance rules by security connector scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void listGovernanceRulesBySecurityConnectorScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .list( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", - com.azure.core.util.Context.NONE); + public static void + listGovernanceRulesBySecurityConnectorScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().list( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ListByManagementGroupGovernanceRules_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * ListByManagementGroupGovernanceRules_example.json */ /** * Sample code: List governance rules by management group scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void listGovernanceRulesByManagementGroupScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .list("providers/Microsoft.Management/managementGroups/contoso", com.azure.core.util.Context.NONE); + public static void + listGovernanceRulesByManagementGroupScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.governanceRules().list("providers/Microsoft.Management/managementGroups/contoso", + com.azure.core.util.Context.NONE); } } ``` @@ -3427,87 +4411,85 @@ public final class GovernanceRulesListSamples { ### GovernanceRules_OperationResults ```java -/** Samples for GovernanceRules OperationResults. */ +/** + * Samples for GovernanceRules OperationResults. + */ public final class GovernanceRulesOperationResultsSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/GetManagementGroupGovernanceRuleExecuteStatus_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * GetManagementGroupGovernanceRuleExecuteStatus_example.json */ /** * Sample code: Get governance rules long run operation result over management group. - * + * * @param manager Entry point to SecurityManager. */ public static void getGovernanceRulesLongRunOperationResultOverManagementGroup( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .operationResultsWithResponse( - "providers/Microsoft.Management/managementGroups/contoso", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - "58b33f4f-c8c7-4b01-99cc-d437db4d40dd", - com.azure.core.util.Context.NONE); + manager.governanceRules().operationResultsWithResponse( + "providers/Microsoft.Management/managementGroups/contoso", "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", + "58b33f4f-c8c7-4b01-99cc-d437db4d40dd", com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/GetSecurityConnectorGovernanceRuleExecuteStatus_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * GetSecurityConnectorGovernanceRuleExecuteStatus_example.json */ /** * Sample code: Get governance rules long run operation result over security connector. - * + * * @param manager Entry point to SecurityManager. */ public static void getGovernanceRulesLongRunOperationResultOverSecurityConnector( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .operationResultsWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - "58b33f4f-c8c7-4b01-99cc-d437db4d40dd", - com.azure.core.util.Context.NONE); + manager.governanceRules().operationResultsWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/gcpResourceGroup/providers/Microsoft.Security/securityConnectors/gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", "58b33f4f-c8c7-4b01-99cc-d437db4d40dd", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/GetGovernanceRuleExecuteStatus_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-01-01-preview/examples/GovernanceRules/ + * GetGovernanceRuleExecuteStatus_example.json */ /** * Sample code: Get governance rules long run operation result over subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getGovernanceRulesLongRunOperationResultOverSubscription( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .governanceRules() - .operationResultsWithResponse( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - "58b33f4f-c8c7-4b01-99cc-d437db4d40dd", - com.azure.core.util.Context.NONE); + manager.governanceRules().operationResultsWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", "58b33f4f-c8c7-4b01-99cc-d437db4d40dd", + com.azure.core.util.Context.NONE); } } ``` -### HealthReportOperation_Get +### HealthReports_Get ```java -/** Samples for HealthReportOperation Get. */ -public final class HealthReportOperationGetSamples { +/** + * Samples for HealthReports Get. + */ +public final class HealthReportsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/HealthReports/GetHealthReport_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-05-01-preview/examples/HealthReports/ + * GetHealthReports_example.json */ /** * Sample code: Get health report of resource. - * + * * @param manager Entry point to SecurityManager. */ public static void getHealthReportOfResource(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .healthReportOperations() - .getWithResponse( - "subscriptions/a1efb6ca-fbc5-4782-9aaa-5c7daded1ce2/resourcegroups/E2E-IBB0WX/providers/Microsoft.Security/securityconnectors/AwsConnectorAllOfferings", - "909c629a-bf39-4521-8e4f-10b443a0bc02", - com.azure.core.util.Context.NONE); + manager.healthReports().getWithResponse( + "subscriptions/a1efb6ca-fbc5-4782-9aaa-5c7daded1ce2/resourcegroups/E2E-IBB0WX/providers/Microsoft.Security/securityconnectors/AwsConnectorAllOfferings", + "909c629a-bf39-4521-8e4f-10b443a0bc02", com.azure.core.util.Context.NONE); } } ``` @@ -3515,20 +4497,23 @@ public final class HealthReportOperationGetSamples { ### HealthReports_List ```java -/** Samples for HealthReports List. */ +/** + * Samples for HealthReports List. + */ public final class HealthReportsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/HealthReports/ListHealthReports_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-05-01-preview/examples/HealthReports/ + * ListHealthReports_example.json */ /** * Sample code: List health reports. - * + * * @param manager Entry point to SecurityManager. */ public static void listHealthReports(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .healthReports() - .list("subscriptions/a1efb6ca-fbc5-4782-9aaa-5c7daded1ce2", com.azure.core.util.Context.NONE); + manager.healthReports().list("subscriptions/a1efb6ca-fbc5-4782-9aaa-5c7daded1ce2", + com.azure.core.util.Context.NONE); } } ``` @@ -3542,260 +4527,128 @@ import com.azure.resourcemanager.security.models.InformationType; import com.azure.resourcemanager.security.models.SensitivityLabel; import java.util.Arrays; import java.util.HashMap; -import java.util.Map; -import java.util.UUID; - -/** Samples for InformationProtectionPolicies CreateOrUpdate. */ -public final class InformationProtectionPoliciesCreateOrUpdateSamples { - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/InformationProtectionPolicies/CreateOrUpdateInformationProtectionPolicy_example.json - */ - /** - * Sample code: Create or update an information protection policy for a management group. - * - * @param manager Entry point to SecurityManager. - */ - public static void createOrUpdateAnInformationProtectionPolicyForAManagementGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .informationProtectionPolicies() - .define(InformationProtectionPolicyName.CUSTOM) - .withExistingScope("providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e") - .withLabels( - mapOf( - "1345da73-bc5a-4a8f-b7dd-3820eb713da8", - new SensitivityLabel().withDisplayName("Public").withOrder(100).withEnabled(true), - "575739d2-3d53-4df0-9042-4c7772d5c7b1", - new SensitivityLabel().withDisplayName("Confidential").withOrder(300).withEnabled(true), - "7aa516c7-5a53-4857-bc6e-6808c6acd542", - new SensitivityLabel().withDisplayName("General").withOrder(200).withEnabled(true))) - .withInformationTypes( - mapOf( - "3bf35491-99b8-41f2-86d5-c1200a7df658", - new InformationType() - .withDisplayName("Custom") - .withOrder(1400) - .withRecommendedLabelId(UUID.fromString("7aa516c7-5a53-4857-bc6e-6808c6acd542")) - .withEnabled(true) - .withCustom(true) - .withKeywords( - Arrays - .asList( - new InformationProtectionKeyword() - .withPattern("%custom%") - .withCustom(true) - .withCanBeNumeric(true))), - "7fb9419d-2473-4ad8-8e11-b25cc8cf6a07", - new InformationType() - .withDisplayName("Networking") - .withOrder(100) - .withRecommendedLabelId(UUID.fromString("575739d2-3d53-4df0-9042-4c7772d5c7b1")) - .withEnabled(true) - .withCustom(false) - .withKeywords( - Arrays - .asList( - new InformationProtectionKeyword() - .withPattern("%networking%") - .withCustom(true) - .withCanBeNumeric(false))))) - .create(); - } - - @SuppressWarnings("unchecked") - private static Map mapOf(Object... inputs) { - Map map = new HashMap<>(); - for (int i = 0; i < inputs.length; i += 2) { - String key = (String) inputs[i]; - T value = (T) inputs[i + 1]; - map.put(key, value); - } - return map; - } -} -``` - -### InformationProtectionPolicies_Get - -```java -import com.azure.resourcemanager.security.models.InformationProtectionPolicyName; - -/** Samples for InformationProtectionPolicies Get. */ -public final class InformationProtectionPoliciesGetSamples { - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/InformationProtectionPolicies/GetCustomInformationProtectionPolicy_example.json - */ - /** - * Sample code: Get the customized information protection policy for a management group. - * - * @param manager Entry point to SecurityManager. - */ - public static void getTheCustomizedInformationProtectionPolicyForAManagementGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .informationProtectionPolicies() - .getWithResponse( - "providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e", - InformationProtectionPolicyName.CUSTOM, - com.azure.core.util.Context.NONE); - } - - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/InformationProtectionPolicies/GetEffectiveInformationProtectionPolicy_example.json - */ - /** - * Sample code: Get the effective information protection policy for a management group. - * - * @param manager Entry point to SecurityManager. - */ - public static void getTheEffectiveInformationProtectionPolicyForAManagementGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .informationProtectionPolicies() - .getWithResponse( - "providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e", - InformationProtectionPolicyName.EFFECTIVE, - com.azure.core.util.Context.NONE); - } -} -``` - -### InformationProtectionPolicies_List - -```java -/** Samples for InformationProtectionPolicies List. */ -public final class InformationProtectionPoliciesListSamples { - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/InformationProtectionPolicies/ListInformationProtectionPolicies_example.json - */ - /** - * Sample code: Get information protection policies. - * - * @param manager Entry point to SecurityManager. - */ - public static void getInformationProtectionPolicies(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .informationProtectionPolicies() - .list( - "providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e", - com.azure.core.util.Context.NONE); - } -} -``` - -### IngestionSettings_Create - -```java -/** Samples for IngestionSettings Create. */ -public final class IngestionSettingsCreateSamples { - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-01-15-preview/examples/IngestionSettings/CreateIngestionSetting_example.json - */ - /** - * Sample code: Create an ingestion setting for subscription. - * - * @param manager Entry point to SecurityManager. - */ - public static void createAnIngestionSettingForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.ingestionSettings().define("default").create(); - } -} -``` - -### IngestionSettings_Delete - -```java -/** Samples for IngestionSettings Delete. */ -public final class IngestionSettingsDeleteSamples { - /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-01-15-preview/examples/IngestionSettings/DeleteIngestionSetting_example.json - */ - /** - * Sample code: Delete an ingestion setting for the subscription. - * - * @param manager Entry point to SecurityManager. - */ - public static void deleteAnIngestionSettingForTheSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.ingestionSettings().deleteWithResponse("default", com.azure.core.util.Context.NONE); - } -} -``` - -### IngestionSettings_Get +import java.util.Map; +import java.util.UUID; -```java -/** Samples for IngestionSettings Get. */ -public final class IngestionSettingsGetSamples { +/** + * Samples for InformationProtectionPolicies CreateOrUpdate. + */ +public final class InformationProtectionPoliciesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-01-15-preview/examples/IngestionSettings/GetIngestionSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * InformationProtectionPolicies/CreateOrUpdateInformationProtectionPolicy_example.json */ /** - * Sample code: Get a ingestion setting on subscription. - * + * Sample code: Create or update an information protection policy for a management group. + * * @param manager Entry point to SecurityManager. */ - public static void getAIngestionSettingOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager.ingestionSettings().getWithResponse("default", com.azure.core.util.Context.NONE); + public static void createOrUpdateAnInformationProtectionPolicyForAManagementGroup( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.informationProtectionPolicies().define(InformationProtectionPolicyName.CUSTOM) + .withExistingScope("providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e") + .withLabels( + mapOf("1345da73-bc5a-4a8f-b7dd-3820eb713da8", + new SensitivityLabel().withDisplayName("Public").withOrder(100).withEnabled(true), + "575739d2-3d53-4df0-9042-4c7772d5c7b1", + new SensitivityLabel().withDisplayName("Confidential").withOrder(300).withEnabled(true), + "7aa516c7-5a53-4857-bc6e-6808c6acd542", new SensitivityLabel().withDisplayName("General") + .withOrder(200).withEnabled(true))) + .withInformationTypes(mapOf("3bf35491-99b8-41f2-86d5-c1200a7df658", + new InformationType().withDisplayName("Custom").withOrder(1400) + .withRecommendedLabelId(UUID.fromString("7aa516c7-5a53-4857-bc6e-6808c6acd542")).withEnabled(true) + .withCustom(true) + .withKeywords(Arrays.asList(new InformationProtectionKeyword() + .withPattern("%custom%").withCustom(true).withCanBeNumeric(true))), + "7fb9419d-2473-4ad8-8e11-b25cc8cf6a07", + new InformationType().withDisplayName("Networking").withOrder(100) + .withRecommendedLabelId(UUID.fromString("575739d2-3d53-4df0-9042-4c7772d5c7b1")).withEnabled(true) + .withCustom(false).withKeywords(Arrays.asList(new InformationProtectionKeyword() + .withPattern("%networking%").withCustom(true).withCanBeNumeric(false))))) + .create(); + } + + // Use "Map.of" if available + @SuppressWarnings("unchecked") + private static Map mapOf(Object... inputs) { + Map map = new HashMap<>(); + for (int i = 0; i < inputs.length; i += 2) { + String key = (String) inputs[i]; + T value = (T) inputs[i + 1]; + map.put(key, value); + } + return map; } } ``` -### IngestionSettings_List +### InformationProtectionPolicies_Get ```java -/** Samples for IngestionSettings List. */ -public final class IngestionSettingsListSamples { +import com.azure.resourcemanager.security.models.InformationProtectionPolicyName; + +/** + * Samples for InformationProtectionPolicies Get. + */ +public final class InformationProtectionPoliciesGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-01-15-preview/examples/IngestionSettings/GetIngestionSettings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * InformationProtectionPolicies/GetCustomInformationProtectionPolicy_example.json */ /** - * Sample code: Get ingestion settings on subscription. - * + * Sample code: Get the customized information protection policy for a management group. + * * @param manager Entry point to SecurityManager. */ - public static void getIngestionSettingsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager.ingestionSettings().list(com.azure.core.util.Context.NONE); + public static void getTheCustomizedInformationProtectionPolicyForAManagementGroup( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.informationProtectionPolicies().getWithResponse( + "providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e", + InformationProtectionPolicyName.CUSTOM, com.azure.core.util.Context.NONE); } -} -``` - -### IngestionSettings_ListConnectionStrings -```java -/** Samples for IngestionSettings ListConnectionStrings. */ -public final class IngestionSettingsListConnectionStringsSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-01-15-preview/examples/IngestionSettings/ListConnectionStrings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * InformationProtectionPolicies/GetEffectiveInformationProtectionPolicy_example.json */ /** - * Sample code: List connection strings for ingesting security data and logs. - * + * Sample code: Get the effective information protection policy for a management group. + * * @param manager Entry point to SecurityManager. */ - public static void listConnectionStringsForIngestingSecurityDataAndLogs( + public static void getTheEffectiveInformationProtectionPolicyForAManagementGroup( com.azure.resourcemanager.security.SecurityManager manager) { - manager.ingestionSettings().listConnectionStringsWithResponse("default", com.azure.core.util.Context.NONE); + manager.informationProtectionPolicies().getWithResponse( + "providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e", + InformationProtectionPolicyName.EFFECTIVE, com.azure.core.util.Context.NONE); } } ``` -### IngestionSettings_ListTokens +### InformationProtectionPolicies_List ```java -/** Samples for IngestionSettings ListTokens. */ -public final class IngestionSettingsListTokensSamples { +/** + * Samples for InformationProtectionPolicies List. + */ +public final class InformationProtectionPoliciesListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-01-15-preview/examples/IngestionSettings/ListTokensIngestionSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/ + * InformationProtectionPolicies/ListInformationProtectionPolicies_example.json */ /** - * Sample code: List ingestion setting tokens. - * + * Sample code: Get information protection policies. + * * @param manager Entry point to SecurityManager. */ - public static void listIngestionSettingTokens(com.azure.resourcemanager.security.SecurityManager manager) { - manager.ingestionSettings().listTokensWithResponse("default", com.azure.core.util.Context.NONE); + public static void getInformationProtectionPolicies(com.azure.resourcemanager.security.SecurityManager manager) { + manager.informationProtectionPolicies().list( + "providers/Microsoft.Management/managementGroups/148059f7-faf3-49a6-ba35-85122112291e", + com.azure.core.util.Context.NONE); } } ``` @@ -3813,50 +4666,42 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for IotSecuritySolution CreateOrUpdate. */ +/** + * Samples for IotSecuritySolution CreateOrUpdate. + */ public final class IotSecuritySolutionCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/CreateIoTSecuritySolution.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * CreateIoTSecuritySolution.json */ /** * Sample code: Create or update a IoT security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateAIoTSecuritySolution(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutions() - .define("default") - .withExistingResourceGroup("MyGroup") - .withRegion("East Us") + manager.iotSecuritySolutions().define("default").withExistingResourceGroup("MyGroup").withRegion("East Us") .withTags(mapOf()) .withWorkspace( "/subscriptions/c4930e90-cd72-4aa5-93e9-2d081d129569/resourceGroups/myRg/providers/Microsoft.OperationalInsights/workspaces/myWorkspace1") - .withDisplayName("Solution Default") - .withStatus(SecuritySolutionStatus.ENABLED) - .withExport(Arrays.asList()) + .withDisplayName("Solution Default").withStatus(SecuritySolutionStatus.ENABLED).withExport(Arrays.asList()) .withDisabledDataSources(Arrays.asList()) - .withIotHubs( - Arrays - .asList( - "/subscriptions/075423e9-7d33-4166-8bdf-3920b04e3735/resourceGroups/myRg/providers/Microsoft.Devices/IotHubs/FirstIotHub")) - .withUserDefinedResources( - new UserDefinedResourcesProperties() - .withQuery("where type != \"microsoft.devices/iothubs\" | where name contains \"iot\"") - .withQuerySubscriptions(Arrays.asList("075423e9-7d33-4166-8bdf-3920b04e3735"))) - .withRecommendationsConfiguration( - Arrays - .asList( - new RecommendationConfigurationProperties() - .withRecommendationType(RecommendationType.IO_T_OPEN_PORTS) - .withStatus(RecommendationConfigStatus.DISABLED), - new RecommendationConfigurationProperties() - .withRecommendationType(RecommendationType.IO_T_SHARED_CREDENTIALS) - .withStatus(RecommendationConfigStatus.DISABLED))) - .withUnmaskedIpLoggingStatus(UnmaskedIpLoggingStatus.ENABLED) - .create(); - } - + .withIotHubs(Arrays.asList( + "/subscriptions/075423e9-7d33-4166-8bdf-3920b04e3735/resourceGroups/myRg/providers/Microsoft.Devices/IotHubs/FirstIotHub")) + .withUserDefinedResources(new UserDefinedResourcesProperties() + .withQuery("where type != \"microsoft.devices/iothubs\" | where name contains \"iot\"") + .withQuerySubscriptions(Arrays.asList("075423e9-7d33-4166-8bdf-3920b04e3735"))) + .withRecommendationsConfiguration(Arrays.asList( + new RecommendationConfigurationProperties().withRecommendationType(RecommendationType.IO_T_OPEN_PORTS) + .withStatus(RecommendationConfigStatus.DISABLED), + new RecommendationConfigurationProperties() + .withRecommendationType(RecommendationType.IO_T_SHARED_CREDENTIALS) + .withStatus(RecommendationConfigStatus.DISABLED))) + .withUnmaskedIpLoggingStatus(UnmaskedIpLoggingStatus.ENABLED).create(); + } + + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -3873,20 +4718,23 @@ public final class IotSecuritySolutionCreateOrUpdateSamples { ### IotSecuritySolution_Delete ```java -/** Samples for IotSecuritySolution Delete. */ +/** + * Samples for IotSecuritySolution Delete. + */ public final class IotSecuritySolutionDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/DeleteIoTSecuritySolution.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * DeleteIoTSecuritySolution.json */ /** * Sample code: Delete an IoT security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteAnIoTSecuritySolution(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutions() - .deleteByResourceGroupWithResponse("MyGroup", "default", com.azure.core.util.Context.NONE); + manager.iotSecuritySolutions().deleteByResourceGroupWithResponse("MyGroup", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -3894,20 +4742,23 @@ public final class IotSecuritySolutionDeleteSamples { ### IotSecuritySolution_GetByResourceGroup ```java -/** Samples for IotSecuritySolution GetByResourceGroup. */ +/** + * Samples for IotSecuritySolution GetByResourceGroup. + */ public final class IotSecuritySolutionGetByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/GetIoTSecuritySolution.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * GetIoTSecuritySolution.json */ /** * Sample code: Get a IoT security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void getAIoTSecuritySolution(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutions() - .getByResourceGroupWithResponse("MyGroup", "default", com.azure.core.util.Context.NONE); + manager.iotSecuritySolutions().getByResourceGroupWithResponse("MyGroup", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -3915,36 +4766,39 @@ public final class IotSecuritySolutionGetByResourceGroupSamples { ### IotSecuritySolution_List ```java -/** Samples for IotSecuritySolution List. */ +/** + * Samples for IotSecuritySolution List. + */ public final class IotSecuritySolutionListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/GetIoTSecuritySolutionsList.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * GetIoTSecuritySolutionsList.json */ /** * Sample code: List IoT Security solutions by subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void listIoTSecuritySolutionsBySubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listIoTSecuritySolutionsBySubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.iotSecuritySolutions().list(null, com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/GetIoTSecuritySolutionsListByIotHub.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * GetIoTSecuritySolutionsListByIotHub.json */ /** * Sample code: List IoT Security solutions by IoT Hub. - * + * * @param manager Entry point to SecurityManager. */ public static void listIoTSecuritySolutionsByIoTHub(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutions() - .list( - "properties.iotHubs/any(i eq" - + " \"/subscriptions/075423e9-7d33-4166-8bdf-3920b04e3735/resourceGroups/myRg/providers/Microsoft.Devices/IotHubs/FirstIotHub\")", - com.azure.core.util.Context.NONE); + manager.iotSecuritySolutions().list( + "properties.iotHubs/any(i eq \"/subscriptions/075423e9-7d33-4166-8bdf-3920b04e3735/resourceGroups/myRg/providers/Microsoft.Devices/IotHubs/FirstIotHub\")", + com.azure.core.util.Context.NONE); } } ``` @@ -3952,37 +4806,39 @@ public final class IotSecuritySolutionListSamples { ### IotSecuritySolution_ListByResourceGroup ```java -/** Samples for IotSecuritySolution ListByResourceGroup. */ +/** + * Samples for IotSecuritySolution ListByResourceGroup. + */ public final class IotSecuritySolutionListByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/GetIoTSecuritySolutionsListByIotHubAndRg.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * GetIoTSecuritySolutionsListByIotHubAndRg.json */ /** * Sample code: List IoT Security solutions by resource group and IoT Hub. - * + * * @param manager Entry point to SecurityManager. */ - public static void listIoTSecuritySolutionsByResourceGroupAndIoTHub( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutions() - .listByResourceGroup( - "MyRg", - "properties.iotHubs/any(i eq" - + " \"/subscriptions/075423e9-7d33-4166-8bdf-3920b04e3735/resourceGroups/myRg/providers/Microsoft.Devices/IotHubs/FirstIotHub\")", - com.azure.core.util.Context.NONE); + public static void + listIoTSecuritySolutionsByResourceGroupAndIoTHub(com.azure.resourcemanager.security.SecurityManager manager) { + manager.iotSecuritySolutions().listByResourceGroup("MyRg", + "properties.iotHubs/any(i eq \"/subscriptions/075423e9-7d33-4166-8bdf-3920b04e3735/resourceGroups/myRg/providers/Microsoft.Devices/IotHubs/FirstIotHub\")", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/GetIoTSecuritySolutionsListByRg.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * GetIoTSecuritySolutionsListByRg.json */ /** * Sample code: List IoT Security solutions by resource group. - * + * * @param manager Entry point to SecurityManager. */ - public static void listIoTSecuritySolutionsByResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listIoTSecuritySolutionsByResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { manager.iotSecuritySolutions().listByResourceGroup("MyGroup", null, com.azure.core.util.Context.NONE); } } @@ -4000,42 +4856,38 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for IotSecuritySolution Update. */ +/** + * Samples for IotSecuritySolution Update. + */ public final class IotSecuritySolutionUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/UpdateIoTSecuritySolution.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutions/ + * UpdateIoTSecuritySolution.json */ /** * Sample code: Use this method to update existing IoT Security solution. - * + * * @param manager Entry point to SecurityManager. */ - public static void useThisMethodToUpdateExistingIoTSecuritySolution( - com.azure.resourcemanager.security.SecurityManager manager) { - IoTSecuritySolutionModel resource = - manager - .iotSecuritySolutions() - .getByResourceGroupWithResponse("myRg", "default", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf("foo", "bar")) - .withUserDefinedResources( - new UserDefinedResourcesProperties() - .withQuery("where type != \"microsoft.devices/iothubs\" | where name contains \"v2\"") - .withQuerySubscriptions(Arrays.asList("075423e9-7d33-4166-8bdf-3920b04e3735"))) - .withRecommendationsConfiguration( - Arrays - .asList( - new RecommendationConfigurationProperties() - .withRecommendationType(RecommendationType.IO_T_OPEN_PORTS) - .withStatus(RecommendationConfigStatus.DISABLED), - new RecommendationConfigurationProperties() - .withRecommendationType(RecommendationType.IO_T_SHARED_CREDENTIALS) - .withStatus(RecommendationConfigStatus.DISABLED))) + public static void + useThisMethodToUpdateExistingIoTSecuritySolution(com.azure.resourcemanager.security.SecurityManager manager) { + IoTSecuritySolutionModel resource = manager.iotSecuritySolutions() + .getByResourceGroupWithResponse("myRg", "default", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf("foo", "bar")) + .withUserDefinedResources(new UserDefinedResourcesProperties() + .withQuery("where type != \"microsoft.devices/iothubs\" | where name contains \"v2\"") + .withQuerySubscriptions(Arrays.asList("075423e9-7d33-4166-8bdf-3920b04e3735"))) + .withRecommendationsConfiguration(Arrays.asList( + new RecommendationConfigurationProperties().withRecommendationType(RecommendationType.IO_T_OPEN_PORTS) + .withStatus(RecommendationConfigStatus.DISABLED), + new RecommendationConfigurationProperties() + .withRecommendationType(RecommendationType.IO_T_SHARED_CREDENTIALS) + .withStatus(RecommendationConfigStatus.DISABLED))) .apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -4052,14 +4904,17 @@ public final class IotSecuritySolutionUpdateSamples { ### IotSecuritySolutionAnalytics_Get ```java -/** Samples for IotSecuritySolutionAnalytics Get. */ +/** + * Samples for IotSecuritySolutionAnalytics Get. + */ public final class IotSecuritySolutionAnalyticsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAnalytics.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAnalytics.json */ /** * Sample code: Get Security Solution Analytics. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecuritySolutionAnalytics(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4071,14 +4926,17 @@ public final class IotSecuritySolutionAnalyticsGetSamples { ### IotSecuritySolutionAnalytics_List ```java -/** Samples for IotSecuritySolutionAnalytics List. */ +/** + * Samples for IotSecuritySolutionAnalytics List. + */ public final class IotSecuritySolutionAnalyticsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAnalyticsList.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAnalyticsList.json */ /** * Sample code: Get Security Solution Analytics. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecuritySolutionAnalytics(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4090,25 +4948,23 @@ public final class IotSecuritySolutionAnalyticsListSamples { ### IotSecuritySolutionsAnalyticsAggregatedAlert_Dismiss ```java -/** Samples for IotSecuritySolutionsAnalyticsAggregatedAlert Dismiss. */ +/** + * Samples for IotSecuritySolutionsAnalyticsAggregatedAlert Dismiss. + */ public final class IotSecuritySolutionsAnalyticsAggregatedAlertDismissSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/PostIoTSecuritySolutionsSecurityAggregatedAlertDismiss.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/PostIoTSecuritySolutionsSecurityAggregatedAlertDismiss.json */ /** * Sample code: Dismiss an aggregated IoT Security Solution Alert. - * + * * @param manager Entry point to SecurityManager. */ - public static void dismissAnAggregatedIoTSecuritySolutionAlert( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutionsAnalyticsAggregatedAlerts() - .dismissWithResponse( - "IoTEdgeResources", - "default", - "IoT_Bruteforce_Fail/2019-02-02/dismiss", - com.azure.core.util.Context.NONE); + public static void + dismissAnAggregatedIoTSecuritySolutionAlert(com.azure.resourcemanager.security.SecurityManager manager) { + manager.iotSecuritySolutionsAnalyticsAggregatedAlerts().dismissWithResponse("IoTEdgeResources", "default", + "IoT_Bruteforce_Fail/2019-02-02/dismiss", com.azure.core.util.Context.NONE); } } ``` @@ -4116,23 +4972,25 @@ public final class IotSecuritySolutionsAnalyticsAggregatedAlertDismissSamples { ### IotSecuritySolutionsAnalyticsAggregatedAlert_Get ```java -/** Samples for IotSecuritySolutionsAnalyticsAggregatedAlert Get. */ +/** + * Samples for IotSecuritySolutionsAnalyticsAggregatedAlert Get. + */ public final class IotSecuritySolutionsAnalyticsAggregatedAlertGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAggregatedAlert.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAggregatedAlert.json */ /** * Sample code: Get the aggregated security analytics alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param manager Entry point to SecurityManager. */ public static void getTheAggregatedSecurityAnalyticsAlertOfYoursIoTSecuritySolutionThisAggregationIsPerformedByAlertName( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutionsAnalyticsAggregatedAlerts() - .getWithResponse("MyGroup", "default", "IoT_Bruteforce_Fail/2019-02-02", com.azure.core.util.Context.NONE); + manager.iotSecuritySolutionsAnalyticsAggregatedAlerts().getWithResponse("MyGroup", "default", + "IoT_Bruteforce_Fail/2019-02-02", com.azure.core.util.Context.NONE); } } ``` @@ -4140,21 +4998,23 @@ public final class IotSecuritySolutionsAnalyticsAggregatedAlertGetSamples { ### IotSecuritySolutionsAnalyticsAggregatedAlert_List ```java -/** Samples for IotSecuritySolutionsAnalyticsAggregatedAlert List. */ +/** + * Samples for IotSecuritySolutionsAnalyticsAggregatedAlert List. + */ public final class IotSecuritySolutionsAnalyticsAggregatedAlertListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAggregatedAlertList.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityAggregatedAlertList.json */ /** * Sample code: Get the aggregated alert list of yours IoT Security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void getTheAggregatedAlertListOfYoursIoTSecuritySolution( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutionsAnalyticsAggregatedAlerts() - .list("MyGroup", "default", null, com.azure.core.util.Context.NONE); + manager.iotSecuritySolutionsAnalyticsAggregatedAlerts().list("MyGroup", "default", null, + com.azure.core.util.Context.NONE); } } ``` @@ -4162,21 +5022,23 @@ public final class IotSecuritySolutionsAnalyticsAggregatedAlertListSamples { ### IotSecuritySolutionsAnalyticsRecommendation_Get ```java -/** Samples for IotSecuritySolutionsAnalyticsRecommendation Get. */ +/** + * Samples for IotSecuritySolutionsAnalyticsRecommendation Get. + */ public final class IotSecuritySolutionsAnalyticsRecommendationGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityRecommendation.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityRecommendation.json */ /** * Sample code: Get the aggregated security analytics recommendation of yours IoT Security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void getTheAggregatedSecurityAnalyticsRecommendationOfYoursIoTSecuritySolution( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutionsAnalyticsRecommendations() - .getWithResponse("IoTEdgeResources", "default", "OpenPortsOnDevice", com.azure.core.util.Context.NONE); + manager.iotSecuritySolutionsAnalyticsRecommendations().getWithResponse("IoTEdgeResources", "default", + "OpenPortsOnDevice", com.azure.core.util.Context.NONE); } } ``` @@ -4184,21 +5046,23 @@ public final class IotSecuritySolutionsAnalyticsRecommendationGetSamples { ### IotSecuritySolutionsAnalyticsRecommendation_List ```java -/** Samples for IotSecuritySolutionsAnalyticsRecommendation List. */ +/** + * Samples for IotSecuritySolutionsAnalyticsRecommendation List. + */ public final class IotSecuritySolutionsAnalyticsRecommendationListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityRecommendationList.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2019-08-01/examples/ + * IoTSecuritySolutionsAnalytics/GetIoTSecuritySolutionsSecurityRecommendationList.json */ /** * Sample code: Get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param manager Entry point to SecurityManager. */ public static void getTheListOfAggregatedSecurityAnalyticsRecommendationsOfYoursIoTSecuritySolution( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .iotSecuritySolutionsAnalyticsRecommendations() - .list("IoTEdgeResources", "default", null, com.azure.core.util.Context.NONE); + manager.iotSecuritySolutionsAnalyticsRecommendations().list("IoTEdgeResources", "default", null, + com.azure.core.util.Context.NONE); } } ``` @@ -4217,63 +5081,39 @@ import com.azure.resourcemanager.security.models.StatusReason; import java.time.OffsetDateTime; import java.util.Arrays; -/** Samples for JitNetworkAccessPolicies CreateOrUpdate. */ +/** + * Samples for JitNetworkAccessPolicies CreateOrUpdate. + */ public final class JitNetworkAccessPoliciesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/CreateJitNetworkAccessPolicy_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * CreateJitNetworkAccessPolicy_example.json */ /** * Sample code: Create JIT network access policy. - * + * * @param manager Entry point to SecurityManager. */ public static void createJITNetworkAccessPolicy(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .jitNetworkAccessPolicies() - .define("default") - .withExistingLocation("myRg1", "westeurope") - .withVirtualMachines( - Arrays - .asList( - new JitNetworkAccessPolicyVirtualMachine() - .withId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg1/providers/Microsoft.Compute/virtualMachines/vm1") - .withPorts( - Arrays - .asList( - new JitNetworkAccessPortRule() - .withNumber(22) - .withProtocol(Protocol.ASTERISK) - .withAllowedSourceAddressPrefix("*") - .withMaxRequestAccessDuration("PT3H"), - new JitNetworkAccessPortRule() - .withNumber(3389) - .withProtocol(Protocol.ASTERISK) - .withAllowedSourceAddressPrefix("*") - .withMaxRequestAccessDuration("PT3H"))))) + manager.jitNetworkAccessPolicies().define("default").withExistingLocation("myRg1", "westeurope") + .withVirtualMachines(Arrays.asList(new JitNetworkAccessPolicyVirtualMachine().withId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg1/providers/Microsoft.Compute/virtualMachines/vm1") + .withPorts(Arrays.asList( + new JitNetworkAccessPortRule().withNumber(22).withProtocol(Protocol.ASTERISK) + .withAllowedSourceAddressPrefix("*").withMaxRequestAccessDuration("PT3H"), + new JitNetworkAccessPortRule().withNumber(3389).withProtocol(Protocol.ASTERISK) + .withAllowedSourceAddressPrefix("*").withMaxRequestAccessDuration("PT3H"))))) .withKind("Basic") - .withRequests( - Arrays - .asList( - new JitNetworkAccessRequestInner() - .withVirtualMachines( - Arrays - .asList( - new JitNetworkAccessRequestVirtualMachine() - .withId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg1/providers/Microsoft.Compute/virtualMachines/vm1") - .withPorts( - Arrays - .asList( - new JitNetworkAccessRequestPort() - .withNumber(3389) - .withAllowedSourceAddressPrefix("192.127.0.2") - .withEndTimeUtc( - OffsetDateTime.parse("2018-05-17T09:06:45.5691611Z")) - .withStatus(Status.INITIATED) - .withStatusReason(StatusReason.USER_REQUESTED))))) - .withStartTimeUtc(OffsetDateTime.parse("2018-05-17T08:06:45.5691611Z")) - .withRequestor("barbara@contoso.com"))) + .withRequests(Arrays.asList(new JitNetworkAccessRequestInner() + .withVirtualMachines(Arrays.asList(new JitNetworkAccessRequestVirtualMachine().withId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg1/providers/Microsoft.Compute/virtualMachines/vm1") + .withPorts(Arrays.asList( + new JitNetworkAccessRequestPort().withNumber(3389).withAllowedSourceAddressPrefix("192.127.0.2") + .withEndTimeUtc(OffsetDateTime.parse("2018-05-17T09:06:45.5691611Z")) + .withStatus(Status.INITIATED).withStatusReason(StatusReason.USER_REQUESTED))))) + .withStartTimeUtc(OffsetDateTime.parse("2018-05-17T08:06:45.5691611Z")) + .withRequestor("barbara@contoso.com"))) .create(); } } @@ -4282,20 +5122,23 @@ public final class JitNetworkAccessPoliciesCreateOrUpdateSamples { ### JitNetworkAccessPolicies_Delete ```java -/** Samples for JitNetworkAccessPolicies Delete. */ +/** + * Samples for JitNetworkAccessPolicies Delete. + */ public final class JitNetworkAccessPoliciesDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/DeleteJitNetworkAccessPolicy_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * DeleteJitNetworkAccessPolicy_example.json */ /** * Sample code: Delete a JIT network access policy. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteAJITNetworkAccessPolicy(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .jitNetworkAccessPolicies() - .deleteWithResponse("myRg1", "westeurope", "default", com.azure.core.util.Context.NONE); + manager.jitNetworkAccessPolicies().deleteWithResponse("myRg1", "westeurope", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -4303,20 +5146,23 @@ public final class JitNetworkAccessPoliciesDeleteSamples { ### JitNetworkAccessPolicies_Get ```java -/** Samples for JitNetworkAccessPolicies Get. */ +/** + * Samples for JitNetworkAccessPolicies Get. + */ public final class JitNetworkAccessPoliciesGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/GetJitNetworkAccessPolicy_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * GetJitNetworkAccessPolicy_example.json */ /** * Sample code: Get JIT network access policy. - * + * * @param manager Entry point to SecurityManager. */ public static void getJITNetworkAccessPolicy(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .jitNetworkAccessPolicies() - .getWithResponse("myRg1", "westeurope", "default", com.azure.core.util.Context.NONE); + manager.jitNetworkAccessPolicies().getWithResponse("myRg1", "westeurope", "default", + com.azure.core.util.Context.NONE); } } ``` @@ -4329,39 +5175,30 @@ import com.azure.resourcemanager.security.models.JitNetworkAccessPolicyInitiateR import com.azure.resourcemanager.security.models.JitNetworkAccessPolicyInitiateVirtualMachine; import java.util.Arrays; -/** Samples for JitNetworkAccessPolicies Initiate. */ +/** + * Samples for JitNetworkAccessPolicies Initiate. + */ public final class JitNetworkAccessPoliciesInitiateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/InitiateJitNetworkAccessPolicy_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * InitiateJitNetworkAccessPolicy_example.json */ /** * Sample code: Initiate an action on a JIT network access policy. - * + * * @param manager Entry point to SecurityManager. */ - public static void initiateAnActionOnAJITNetworkAccessPolicy( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .jitNetworkAccessPolicies() - .initiateWithResponse( - "myRg1", - "westeurope", - "default", - new JitNetworkAccessPolicyInitiateRequest() - .withVirtualMachines( - Arrays - .asList( - new JitNetworkAccessPolicyInitiateVirtualMachine() - .withId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg1/providers/Microsoft.Compute/virtualMachines/vm1") - .withPorts( - Arrays - .asList( - new JitNetworkAccessPolicyInitiatePort() - .withNumber(3389) - .withAllowedSourceAddressPrefix("192.127.0.2"))))) - .withJustification("testing a new version of the product"), - com.azure.core.util.Context.NONE); + public static void + initiateAnActionOnAJITNetworkAccessPolicy(com.azure.resourcemanager.security.SecurityManager manager) { + manager.jitNetworkAccessPolicies().initiateWithResponse("myRg1", "westeurope", "default", + new JitNetworkAccessPolicyInitiateRequest() + .withVirtualMachines(Arrays.asList(new JitNetworkAccessPolicyInitiateVirtualMachine().withId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg1/providers/Microsoft.Compute/virtualMachines/vm1") + .withPorts(Arrays.asList(new JitNetworkAccessPolicyInitiatePort().withNumber(3389) + .withAllowedSourceAddressPrefix("192.127.0.2"))))) + .withJustification("testing a new version of the product"), + com.azure.core.util.Context.NONE); } } ``` @@ -4369,18 +5206,22 @@ public final class JitNetworkAccessPoliciesInitiateSamples { ### JitNetworkAccessPolicies_List ```java -/** Samples for JitNetworkAccessPolicies List. */ +/** + * Samples for JitNetworkAccessPolicies List. + */ public final class JitNetworkAccessPoliciesListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/GetJitNetworkAccessPoliciesSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * GetJitNetworkAccessPoliciesSubscription_example.json */ /** * Sample code: Get JIT network access policies on a subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void getJITNetworkAccessPoliciesOnASubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getJITNetworkAccessPoliciesOnASubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.jitNetworkAccessPolicies().list(com.azure.core.util.Context.NONE); } } @@ -4389,14 +5230,18 @@ public final class JitNetworkAccessPoliciesListSamples { ### JitNetworkAccessPolicies_ListByRegion ```java -/** Samples for JitNetworkAccessPolicies ListByRegion. */ +/** + * Samples for JitNetworkAccessPolicies ListByRegion. + */ public final class JitNetworkAccessPoliciesListByRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/GetJitNetworkAccessPoliciesSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * GetJitNetworkAccessPoliciesSubscriptionLocation_example.json */ /** * Sample code: Get JIT network access policies on a subscription from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getJITNetworkAccessPoliciesOnASubscriptionFromASecurityDataLocation( @@ -4409,18 +5254,22 @@ public final class JitNetworkAccessPoliciesListByRegionSamples { ### JitNetworkAccessPolicies_ListByResourceGroup ```java -/** Samples for JitNetworkAccessPolicies ListByResourceGroup. */ +/** + * Samples for JitNetworkAccessPolicies ListByResourceGroup. + */ public final class JitNetworkAccessPoliciesListByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/GetJitNetworkAccessPoliciesResourceGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * GetJitNetworkAccessPoliciesResourceGroup_example.json */ /** * Sample code: Get JIT network access policies on a resource group. - * + * * @param manager Entry point to SecurityManager. */ - public static void getJITNetworkAccessPoliciesOnAResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getJITNetworkAccessPoliciesOnAResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { manager.jitNetworkAccessPolicies().listByResourceGroup("myRg1", com.azure.core.util.Context.NONE); } } @@ -4429,21 +5278,24 @@ public final class JitNetworkAccessPoliciesListByResourceGroupSamples { ### JitNetworkAccessPolicies_ListByResourceGroupAndRegion ```java -/** Samples for JitNetworkAccessPolicies ListByResourceGroupAndRegion. */ +/** + * Samples for JitNetworkAccessPolicies ListByResourceGroupAndRegion. + */ public final class JitNetworkAccessPoliciesListByResourceGroupAndRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/GetJitNetworkAccessPoliciesResourceGroupLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/JitNetworkAccessPolicies/ + * GetJitNetworkAccessPoliciesResourceGroupLocation_example.json */ /** * Sample code: Get JIT network access policies on a resource group from a security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getJITNetworkAccessPoliciesOnAResourceGroupFromASecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .jitNetworkAccessPolicies() - .listByResourceGroupAndRegion("myRg1", "westeurope", com.azure.core.util.Context.NONE); + manager.jitNetworkAccessPolicies().listByResourceGroupAndRegion("myRg1", "westeurope", + com.azure.core.util.Context.NONE); } } ``` @@ -4451,14 +5303,18 @@ public final class JitNetworkAccessPoliciesListByResourceGroupAndRegionSamples { ### Locations_Get ```java -/** Samples for Locations Get. */ +/** + * Samples for Locations Get. + */ public final class LocationsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Locations/GetLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Locations/ + * GetLocation_example.json */ /** * Sample code: Get security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityDataLocation(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4470,14 +5326,18 @@ public final class LocationsGetSamples { ### Locations_List ```java -/** Samples for Locations List. */ +/** + * Samples for Locations List. + */ public final class LocationsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Locations/GetLocations_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Locations/ + * GetLocations_example.json */ /** * Sample code: Get security data locations. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityDataLocations(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4489,14 +5349,18 @@ public final class LocationsListSamples { ### MdeOnboardings_Get ```java -/** Samples for MdeOnboardings Get. */ +/** + * Samples for MdeOnboardings Get. + */ public final class MdeOnboardingsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-10-01-preview/examples/MdeOnboardings/GetMdeOnboardings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-10-01-preview/examples/MdeOnboardings/ + * GetMdeOnboardings_example.json */ /** * Sample code: The default configuration or data needed to onboard the machine to MDE. - * + * * @param manager Entry point to SecurityManager. */ public static void theDefaultConfigurationOrDataNeededToOnboardTheMachineToMDE( @@ -4509,14 +5373,18 @@ public final class MdeOnboardingsGetSamples { ### MdeOnboardings_List ```java -/** Samples for MdeOnboardings List. */ +/** + * Samples for MdeOnboardings List. + */ public final class MdeOnboardingsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-10-01-preview/examples/MdeOnboardings/ListMdeOnboardings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-10-01-preview/examples/MdeOnboardings/ + * ListMdeOnboardings_example.json */ /** * Sample code: The configuration or data needed to onboard the machine to MDE. - * + * * @param manager Entry point to SecurityManager. */ public static void theConfigurationOrDataNeededToOnboardTheMachineToMDE( @@ -4526,73 +5394,132 @@ public final class MdeOnboardingsListSamples { } ``` +### Pricings_Delete + +```java +/** + * Samples for Pricings Delete. + */ +public final class PricingsDeleteSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * DeleteResourcePricing_example.json + */ + /** + * Sample code: Delete a pricing on resource. + * + * @param manager Entry point to SecurityManager. + */ + public static void deleteAPricingOnResource(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().deleteByResourceGroupWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/DEMO/providers/Microsoft.Compute/virtualMachines/VM-1", + "VirtualMachines", com.azure.core.util.Context.NONE); + } +} +``` + ### Pricings_Get ```java -/** Samples for Pricings Get. */ +/** + * Samples for Pricings Get. + */ public final class PricingsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/GetPricingByNameCloudPosture_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * GetResourcePricingByNameVirtualMachines_example.json + */ + /** + * Sample code: Get pricings on resource - VirtualMachines plan. + * + * @param manager Entry point to SecurityManager. + */ + public static void + getPricingsOnResourceVirtualMachinesPlan(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().getWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/DEMO/providers/Microsoft.Compute/virtualMachines/VM-1", + "VirtualMachines", com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * GetPricingByNameCloudPosture_example.json */ /** * Sample code: Get pricings on subscription - CloudPosture plan. - * + * * @param manager Entry point to SecurityManager. */ - public static void getPricingsOnSubscriptionCloudPosturePlan( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.pricings().getWithResponse("CloudPosture", com.azure.core.util.Context.NONE); + public static void + getPricingsOnSubscriptionCloudPosturePlan(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "CloudPosture", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/GetPricingByNameDns_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * GetPricingByNameDns_example.json */ /** * Sample code: Get pricings on subscription - Dns plan. - * + * * @param manager Entry point to SecurityManager. */ public static void getPricingsOnSubscriptionDnsPlan(com.azure.resourcemanager.security.SecurityManager manager) { - manager.pricings().getWithResponse("Dns", com.azure.core.util.Context.NONE); + manager.pricings().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "Dns", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/GetPricingByNameContainers_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * GetPricingByNameContainers_example.json */ /** * Sample code: Get pricings on subscription - Containers plan. - * + * * @param manager Entry point to SecurityManager. */ - public static void getPricingsOnSubscriptionContainersPlan( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.pricings().getWithResponse("Containers", com.azure.core.util.Context.NONE); + public static void + getPricingsOnSubscriptionContainersPlan(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "Containers", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/GetPricingByNameStorageAccounts_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * GetPricingByNameStorageAccounts_example.json */ /** * Sample code: Get pricings on subscription - StorageAccounts plan. - * + * * @param manager Entry point to SecurityManager. */ - public static void getPricingsOnSubscriptionStorageAccountsPlan( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.pricings().getWithResponse("StorageAccounts", com.azure.core.util.Context.NONE); + public static void + getPricingsOnSubscriptionStorageAccountsPlan(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "StorageAccounts", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/GetPricingByNameVirtualMachines_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * GetPricingByNameVirtualMachines_example.json */ /** * Sample code: Get pricings on subscription - VirtualMachines plan. - * + * * @param manager Entry point to SecurityManager. */ - public static void getPricingsOnSubscriptionVirtualMachinesPlan( - com.azure.resourcemanager.security.SecurityManager manager) { - manager.pricings().getWithResponse("VirtualMachines", com.azure.core.util.Context.NONE); + public static void + getPricingsOnSubscriptionVirtualMachinesPlan(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().getWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "VirtualMachines", + com.azure.core.util.Context.NONE); } } ``` @@ -4600,18 +5527,55 @@ public final class PricingsGetSamples { ### Pricings_List ```java -/** Samples for Pricings List. */ +/** + * Samples for Pricings List. + */ public final class PricingsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/ListPricings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * ListPricingsWithPlanFilter_example.json + */ + /** + * Sample code: Get pricings on subscription with plans filter. + * + * @param manager Entry point to SecurityManager. + */ + public static void + getPricingsOnSubscriptionWithPlansFilter(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().listWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "name in (VirtualMachines,KeyVaults)", com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * ListPricings_example.json */ /** * Sample code: Get pricings on subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getPricingsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager.pricings().listWithResponse(com.azure.core.util.Context.NONE); + manager.pricings().listWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", null, + com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * ListResourcePricings_example.json + */ + /** + * Sample code: Get pricings on resource. + * + * @param manager Entry point to SecurityManager. + */ + public static void getPricingsOnResource(com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().listWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/DEMO/providers/Microsoft.Compute/virtualMachines/VM-1", + null, com.azure.core.util.Context.NONE); } } ``` @@ -4620,43 +5584,78 @@ public final class PricingsListSamples { ```java import com.azure.resourcemanager.security.fluent.models.PricingInner; +import com.azure.resourcemanager.security.models.Enforce; import com.azure.resourcemanager.security.models.PricingTier; -/** Samples for Pricings Update. */ +/** + * Samples for Pricings Update. + */ public final class PricingsUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/PutPricingByNamePartialSuccess_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * PutPricingByName_example.json + */ + /** + * Sample code: Update pricing on subscription (example for CloudPosture plan). + * + * @param manager Entry point to SecurityManager. + */ + public static void updatePricingOnSubscriptionExampleForCloudPosturePlan( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().updateWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "CloudPosture", + new PricingInner().withPricingTier(PricingTier.STANDARD), com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * PutPricingByNamePartialSuccess_example.json + */ + /** + * Sample code: Update pricing on subscription (example for CloudPosture plan) - partial success. + * + * @param manager Entry point to SecurityManager. + */ + public static void updatePricingOnSubscriptionExampleForCloudPosturePlanPartialSuccess( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().updateWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "CloudPosture", + new PricingInner().withPricingTier(PricingTier.STANDARD), com.azure.core.util.Context.NONE); + } + + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * PutPricingVMsByName_example.json */ /** - * Sample code: Update pricing on subscription - partial success. - * + * Sample code: Update pricing on subscription (example for VirtualMachines plan). + * * @param manager Entry point to SecurityManager. */ - public static void updatePricingOnSubscriptionPartialSuccess( + public static void updatePricingOnSubscriptionExampleForVirtualMachinesPlan( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .pricings() - .updateWithResponse( - "CloudPosture", - new PricingInner().withPricingTier(PricingTier.STANDARD), - com.azure.core.util.Context.NONE); + manager.pricings().updateWithResponse("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", "VirtualMachines", + new PricingInner().withPricingTier(PricingTier.STANDARD).withSubPlan("P2").withEnforce(Enforce.TRUE), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-01-01/examples/Pricings/PutPricingByName_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2024-01-01/examples/Pricings/ + * PutResourcePricingByNameVirtualMachines_example.json */ /** - * Sample code: Update pricing on subscription. - * + * Sample code: Update pricing on resource (example for VirtualMachines plan). + * * @param manager Entry point to SecurityManager. */ - public static void updatePricingOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .pricings() - .updateWithResponse( - "CloudPosture", - new PricingInner().withPricingTier(PricingTier.STANDARD), - com.azure.core.util.Context.NONE); + public static void updatePricingOnResourceExampleForVirtualMachinesPlan( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.pricings().updateWithResponse( + "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/DEMO/providers/Microsoft.Compute/virtualMachines/VM-1", + "virtualMachines", new PricingInner().withPricingTier(PricingTier.STANDARD).withSubPlan("P1"), + com.azure.core.util.Context.NONE); } } ``` @@ -4664,22 +5663,24 @@ public final class PricingsUpdateSamples { ### RegulatoryComplianceAssessments_Get ```java -/** Samples for RegulatoryComplianceAssessments Get. */ +/** + * Samples for RegulatoryComplianceAssessments Get. + */ public final class RegulatoryComplianceAssessmentsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/RegulatoryCompliance/getRegulatoryComplianceAssessment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * RegulatoryCompliance/getRegulatoryComplianceAssessment_example.json */ /** * Sample code: Get selected regulatory compliance assessment details and state. - * + * * @param manager Entry point to SecurityManager. */ public static void getSelectedRegulatoryComplianceAssessmentDetailsAndState( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .regulatoryComplianceAssessments() - .getWithResponse( - "PCI-DSS-3.2", "1.1", "968548cb-02b3-8cd2-11f8-0cf64ab1a347", com.azure.core.util.Context.NONE); + manager.regulatoryComplianceAssessments().getWithResponse("PCI-DSS-3.2", "1.1", + "968548cb-02b3-8cd2-11f8-0cf64ab1a347", com.azure.core.util.Context.NONE); } } ``` @@ -4687,14 +5688,18 @@ public final class RegulatoryComplianceAssessmentsGetSamples { ### RegulatoryComplianceAssessments_List ```java -/** Samples for RegulatoryComplianceAssessments List. */ +/** + * Samples for RegulatoryComplianceAssessments List. + */ public final class RegulatoryComplianceAssessmentsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/RegulatoryCompliance/getRegulatoryComplianceAssessmentList_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * RegulatoryCompliance/getRegulatoryComplianceAssessmentList_example.json */ /** * Sample code: Get all assessments mapped to selected regulatory compliance control. - * + * * @param manager Entry point to SecurityManager. */ public static void getAllAssessmentsMappedToSelectedRegulatoryComplianceControl( @@ -4707,14 +5712,18 @@ public final class RegulatoryComplianceAssessmentsListSamples { ### RegulatoryComplianceControls_Get ```java -/** Samples for RegulatoryComplianceControls Get. */ +/** + * Samples for RegulatoryComplianceControls Get. + */ public final class RegulatoryComplianceControlsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/RegulatoryCompliance/getRegulatoryComplianceControl_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * RegulatoryCompliance/getRegulatoryComplianceControl_example.json */ /** * Sample code: Get selected regulatory compliance control details and state. - * + * * @param manager Entry point to SecurityManager. */ public static void getSelectedRegulatoryComplianceControlDetailsAndState( @@ -4727,14 +5736,18 @@ public final class RegulatoryComplianceControlsGetSamples { ### RegulatoryComplianceControls_List ```java -/** Samples for RegulatoryComplianceControls List. */ +/** + * Samples for RegulatoryComplianceControls List. + */ public final class RegulatoryComplianceControlsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/RegulatoryCompliance/getRegulatoryComplianceControlList_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * RegulatoryCompliance/getRegulatoryComplianceControlList_example.json */ /** * Sample code: Get all regulatory compliance controls details and state for selected standard. - * + * * @param manager Entry point to SecurityManager. */ public static void getAllRegulatoryComplianceControlsDetailsAndStateForSelectedStandard( @@ -4747,14 +5760,18 @@ public final class RegulatoryComplianceControlsListSamples { ### RegulatoryComplianceStandards_Get ```java -/** Samples for RegulatoryComplianceStandards Get. */ +/** + * Samples for RegulatoryComplianceStandards Get. + */ public final class RegulatoryComplianceStandardsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/RegulatoryCompliance/getRegulatoryComplianceStandard_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * RegulatoryCompliance/getRegulatoryComplianceStandard_example.json */ /** * Sample code: Get selected regulatory compliance standard details and state. - * + * * @param manager Entry point to SecurityManager. */ public static void getSelectedRegulatoryComplianceStandardDetailsAndState( @@ -4767,14 +5784,18 @@ public final class RegulatoryComplianceStandardsGetSamples { ### RegulatoryComplianceStandards_List ```java -/** Samples for RegulatoryComplianceStandards List. */ +/** + * Samples for RegulatoryComplianceStandards List. + */ public final class RegulatoryComplianceStandardsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/RegulatoryCompliance/getRegulatoryComplianceStandardList_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/ + * RegulatoryCompliance/getRegulatoryComplianceStandardList_example.json */ /** * Sample code: Get all supported regulatory compliance standards details and state. - * + * * @param manager Entry point to SecurityManager. */ public static void getAllSupportedRegulatoryComplianceStandardsDetailsAndState( @@ -4784,17 +5805,77 @@ public final class RegulatoryComplianceStandardsListSamples { } ``` +### ResourceProvider_GetSensitivitySettings + +```java +/** + * Samples for ResourceProvider GetSensitivitySettings. + */ +public final class ResourceProviderGetSensitivitySettingsSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-15-preview/examples/ + * SensitivitySettings/GetSensitivitySettings_example.json + */ + /** + * Sample code: Get sensitivity settings. + * + * @param manager Entry point to SecurityManager. + */ + public static void getSensitivitySettings(com.azure.resourcemanager.security.SecurityManager manager) { + manager.resourceProviders().getSensitivitySettingsWithResponse(com.azure.core.util.Context.NONE); + } +} +``` + +### ResourceProvider_UpdateSensitivitySettings + +```java +import com.azure.resourcemanager.security.models.UpdateSensitivitySettingsRequest; +import java.util.Arrays; +import java.util.UUID; + +/** + * Samples for ResourceProvider UpdateSensitivitySettings. + */ +public final class ResourceProviderUpdateSensitivitySettingsSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-15-preview/examples/ + * SensitivitySettings/PutSensitivitySettings_example.json + */ + /** + * Sample code: Update sensitivity settings. + * + * @param manager Entry point to SecurityManager. + */ + public static void updateSensitivitySettings(com.azure.resourcemanager.security.SecurityManager manager) { + manager.resourceProviders().updateSensitivitySettingsWithResponse( + new UpdateSensitivitySettingsRequest() + .withSensitiveInfoTypesIds(Arrays.asList(UUID.fromString("f2f8a7a1-28c0-404b-9ab4-30a0a7af18cb"), + UUID.fromString("b452f22b-f87d-4f48-8490-ecf0873325b5"), + UUID.fromString("d59ee8b6-2618-404b-a5e7-aa377cd67543"))) + .withSensitivityThresholdLabelOrder(2.0F).withSensitivityThresholdLabelId( + UUID.fromString("f2f8a7a1-28c0-404b-9ab4-30a0a7af18cb")), + com.azure.core.util.Context.NONE); + } +} +``` + ### SecureScoreControlDefinitions_List ```java -/** Samples for SecureScoreControlDefinitions List. */ +/** + * Samples for SecureScoreControlDefinitions List. + */ public final class SecureScoreControlDefinitionsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScoreControlDefinitions/ListSecureScoreControlDefinitions_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * secureScoreControlDefinitions/ListSecureScoreControlDefinitions_example.json */ /** * Sample code: List security controls definition. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecurityControlsDefinition(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4806,18 +5887,21 @@ public final class SecureScoreControlDefinitionsListSamples { ### SecureScoreControlDefinitions_ListBySubscription ```java -/** Samples for SecureScoreControlDefinitions ListBySubscription. */ +/** + * Samples for SecureScoreControlDefinitions ListBySubscription. + */ public final class SecureScoreControlDefinitionsListBySubscriptionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScoreControlDefinitions/ListSecureScoreControlDefinitions_subscription_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * secureScoreControlDefinitions/ListSecureScoreControlDefinitions_subscription_example.json */ /** * Sample code: List security controls definition by subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void listSecurityControlsDefinitionBySubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listSecurityControlsDefinitionBySubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.secureScoreControlDefinitions().listBySubscription(com.azure.core.util.Context.NONE); } } @@ -4826,14 +5910,19 @@ public final class SecureScoreControlDefinitionsListBySubscriptionSamples { ### SecureScoreControls_List ```java -/** Samples for SecureScoreControls List. */ + +/** + * Samples for SecureScoreControls List. + */ public final class SecureScoreControlsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ListSecureScoreControls_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ + * ListSecureScoreControls_example.json */ /** * Sample code: List all secure scores controls. - * + * * @param manager Entry point to SecurityManager. */ public static void listAllSecureScoresControls(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4847,30 +5936,35 @@ public final class SecureScoreControlsListSamples { ```java import com.azure.resourcemanager.security.models.ExpandControlsEnum; -/** Samples for SecureScoreControls ListBySecureScore. */ +/** + * Samples for SecureScoreControls ListBySecureScore. + */ public final class SecureScoreControlsListBySecureScoreSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ListSecureScoreControlsForNameWithExpand_builtin_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ + * ListSecureScoreControlsForNameWithExpand_builtin_example.json */ /** * Sample code: Get security controls and their current score for the specified initiative with the expand * parameter. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityControlsAndTheirCurrentScoreForTheSpecifiedInitiativeWithTheExpandParameter( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .secureScoreControls() - .listBySecureScore("ascScore", ExpandControlsEnum.DEFINITION, com.azure.core.util.Context.NONE); + manager.secureScoreControls().listBySecureScore("ascScore", ExpandControlsEnum.DEFINITION, + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ListSecureScoreControlsForName_builtin_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ + * ListSecureScoreControlsForName_builtin_example.json */ /** * Sample code: Get security controls and their current score for the specified initiative. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityControlsAndTheirCurrentScoreForTheSpecifiedInitiative( @@ -4883,14 +5977,18 @@ public final class SecureScoreControlsListBySecureScoreSamples { ### SecureScores_Get ```java -/** Samples for SecureScores Get. */ +/** + * Samples for SecureScores Get. + */ public final class SecureScoresGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/GetSecureScoresSingle_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ + * GetSecureScoresSingle_example.json */ /** * Sample code: Get single secure score. - * + * * @param manager Entry point to SecurityManager. */ public static void getSingleSecureScore(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4902,14 +6000,18 @@ public final class SecureScoresGetSamples { ### SecureScores_List ```java -/** Samples for SecureScores List. */ +/** + * Samples for SecureScores List. + */ public final class SecureScoresListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ListSecureScores_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/secureScores/ + * ListSecureScores_example.json */ /** * Sample code: List secure scores. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecureScores(com.azure.resourcemanager.security.SecurityManager manager) { @@ -4928,38 +6030,31 @@ import com.azure.resourcemanager.security.models.ApplicationSourceResourceType; import java.io.IOException; import java.util.Arrays; -/** Samples for SecurityConnectorApplicationOperation CreateOrUpdate. */ +/** + * Samples for SecurityConnectorApplicationOperation CreateOrUpdate. + */ public final class SecurityConnectorApplicationOperationCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/PutSecurityConnectorApplication_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * PutSecurityConnectorApplication_example.json */ /** * Sample code: Create Application. - * + * * @param manager Entry point to SecurityManager. */ public static void createApplication(com.azure.resourcemanager.security.SecurityManager manager) throws IOException { - manager - .securityConnectorApplicationOperations() - .createOrUpdateWithResponse( - "gcpResourceGroup", - "gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - new ApplicationInner() - .withDisplayName("GCP Admin's application") - .withDescription("An application on critical GCP recommendations") - .withSourceResourceType(ApplicationSourceResourceType.ASSESSMENTS) - .withConditionSets( - Arrays - .asList( - SerializerFactory - .createDefaultManagementSerializerAdapter() - .deserialize( - "{\"conditions\":[{\"operator\":\"contains\",\"property\":\"$.Id\",\"value\":\"-prod-\"}]}", - Object.class, - SerializerEncoding.JSON))), - com.azure.core.util.Context.NONE); + manager.securityConnectorApplicationOperations().createOrUpdateWithResponse("gcpResourceGroup", "gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", + new ApplicationInner().withDisplayName("GCP Admin's application") + .withDescription("An application on critical GCP recommendations") + .withSourceResourceType(ApplicationSourceResourceType.ASSESSMENTS).withConditionSets( + Arrays.asList(SerializerFactory.createDefaultManagementSerializerAdapter().deserialize( + "{\"conditions\":[{\"operator\":\"contains\",\"property\":\"$.Id\",\"value\":\"-prod-\"}]}", + Object.class, SerializerEncoding.JSON))), + com.azure.core.util.Context.NONE); } } ``` @@ -4967,24 +6062,23 @@ public final class SecurityConnectorApplicationOperationCreateOrUpdateSamples { ### SecurityConnectorApplicationOperation_Delete ```java -/** Samples for SecurityConnectorApplicationOperation Delete. */ +/** + * Samples for SecurityConnectorApplicationOperation Delete. + */ public final class SecurityConnectorApplicationOperationDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/DeleteSecurityConnectorApplication_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * DeleteSecurityConnectorApplication_example.json */ /** * Sample code: Delete security Application. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteSecurityApplication(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityConnectorApplicationOperations() - .deleteWithResponse( - "gcpResourceGroup", - "gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + manager.securityConnectorApplicationOperations().deleteWithResponse("gcpResourceGroup", "gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } } ``` @@ -4992,25 +6086,24 @@ public final class SecurityConnectorApplicationOperationDeleteSamples { ### SecurityConnectorApplicationOperation_Get ```java -/** Samples for SecurityConnectorApplicationOperation Get. */ +/** + * Samples for SecurityConnectorApplicationOperation Get. + */ public final class SecurityConnectorApplicationOperationGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/GetSecurityConnectorApplication_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * GetSecurityConnectorApplication_example.json */ /** * Sample code: Get security applications by specific applicationId. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityApplicationsBySpecificApplicationId( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityConnectorApplicationOperations() - .getWithResponse( - "gcpResourceGroup", - "gcpconnector", - "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", - com.azure.core.util.Context.NONE); + public static void + getSecurityApplicationsBySpecificApplicationId(com.azure.resourcemanager.security.SecurityManager manager) { + manager.securityConnectorApplicationOperations().getWithResponse("gcpResourceGroup", "gcpconnector", + "ad9a8e26-29d9-4829-bb30-e597a58cdbb8", com.azure.core.util.Context.NONE); } } ``` @@ -5018,21 +6111,24 @@ public final class SecurityConnectorApplicationOperationGetSamples { ### SecurityConnectorApplications_List ```java -/** Samples for SecurityConnectorApplications List. */ +/** + * Samples for SecurityConnectorApplications List. + */ public final class SecurityConnectorApplicationsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ListBySecurityConnectorApplications_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2022-07-01-preview/examples/Applications/ + * ListBySecurityConnectorApplications_example.json */ /** * Sample code: List security applications by security connector level scope. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecurityApplicationsBySecurityConnectorLevelScope( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityConnectorApplications() - .list("gcpResourceGroup", "gcpconnector", com.azure.core.util.Context.NONE); + manager.securityConnectorApplications().list("gcpResourceGroup", "gcpconnector", + com.azure.core.util.Context.NONE); } } ``` @@ -5048,36 +6144,31 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for SecurityConnectors CreateOrUpdate. */ +/** + * Samples for SecurityConnectors CreateOrUpdate. + */ public final class SecurityConnectorsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-03-01-preview/examples/SecurityConnectors/PutSecurityConnector_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-10-01-preview/examples/SecurityConnectors + * /PutSecurityConnector_example.json */ /** * Sample code: Create or update a security connector. - * + * * @param manager Entry point to SecurityManager. */ public static void createOrUpdateASecurityConnector(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityConnectors() - .define("exampleSecurityConnectorName") - .withRegion("Central US") - .withExistingResourceGroup("exampleResourceGroup") - .withTags(mapOf()) - .withHierarchyIdentifier("exampleHierarchyId") - .withEnvironmentName(CloudName.AWS) - .withOfferings( - Arrays - .asList( - new CspmMonitorAwsOffering() - .withNativeCloudConnection( - new CspmMonitorAwsOfferingNativeCloudConnection() - .withCloudRoleArn("arn:aws:iam::00000000:role/ASCMonitor")))) - .withEnvironmentData(new AwsEnvironmentData()) - .create(); + manager.securityConnectors().define("exampleSecurityConnectorName").withRegion("Central US") + .withExistingResourceGroup("exampleResourceGroup").withTags(mapOf()) + .withHierarchyIdentifier("exampleHierarchyId").withEnvironmentName(CloudName.AWS) + .withOfferings(Arrays.asList( + new CspmMonitorAwsOffering().withNativeCloudConnection(new CspmMonitorAwsOfferingNativeCloudConnection() + .withCloudRoleArn("arn:aws:iam::00000000:role/ASCMonitor")))) + .withEnvironmentData(new AwsEnvironmentData().withScanInterval(4L)).create(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -5094,20 +6185,23 @@ public final class SecurityConnectorsCreateOrUpdateSamples { ### SecurityConnectors_Delete ```java -/** Samples for SecurityConnectors Delete. */ +/** + * Samples for SecurityConnectors Delete. + */ public final class SecurityConnectorsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-03-01-preview/examples/SecurityConnectors/DeleteSecurityConnector_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-10-01-preview/examples/SecurityConnectors + * /DeleteSecurityConnector_example.json */ /** * Sample code: Delete a security connector. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteASecurityConnector(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityConnectors() - .deleteByResourceGroupWithResponse("myRg", "mySecurityConnectorName", com.azure.core.util.Context.NONE); + manager.securityConnectors().deleteByResourceGroupWithResponse("myRg", "mySecurityConnectorName", + com.azure.core.util.Context.NONE); } } ``` @@ -5115,21 +6209,23 @@ public final class SecurityConnectorsDeleteSamples { ### SecurityConnectors_GetByResourceGroup ```java -/** Samples for SecurityConnectors GetByResourceGroup. */ +/** + * Samples for SecurityConnectors GetByResourceGroup. + */ public final class SecurityConnectorsGetByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-03-01-preview/examples/SecurityConnectors/GetSecurityConnectorSingleResource_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-10-01-preview/examples/SecurityConnectors + * /GetSecurityConnectorSingleResource_example.json */ /** * Sample code: Retrieve a security connector. - * + * * @param manager Entry point to SecurityManager. */ public static void retrieveASecurityConnector(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityConnectors() - .getByResourceGroupWithResponse( - "exampleResourceGroup", "exampleSecurityConnectorName", com.azure.core.util.Context.NONE); + manager.securityConnectors().getByResourceGroupWithResponse("exampleResourceGroup", + "exampleSecurityConnectorName", com.azure.core.util.Context.NONE); } } ``` @@ -5137,18 +6233,22 @@ public final class SecurityConnectorsGetByResourceGroupSamples { ### SecurityConnectors_List ```java -/** Samples for SecurityConnectors List. */ +/** + * Samples for SecurityConnectors List. + */ public final class SecurityConnectorsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-03-01-preview/examples/SecurityConnectors/GetSecurityConnectorsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-10-01-preview/examples/SecurityConnectors + * /GetSecurityConnectorsSubscription_example.json */ /** * Sample code: List all security connectors of a specified subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void listAllSecurityConnectorsOfASpecifiedSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listAllSecurityConnectorsOfASpecifiedSubscription(com.azure.resourcemanager.security.SecurityManager manager) { manager.securityConnectors().list(com.azure.core.util.Context.NONE); } } @@ -5157,18 +6257,22 @@ public final class SecurityConnectorsListSamples { ### SecurityConnectors_ListByResourceGroup ```java -/** Samples for SecurityConnectors ListByResourceGroup. */ +/** + * Samples for SecurityConnectors ListByResourceGroup. + */ public final class SecurityConnectorsListByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-03-01-preview/examples/SecurityConnectors/GetSecurityConnectorsResourceGroup_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-10-01-preview/examples/SecurityConnectors + * /GetSecurityConnectorsResourceGroup_example.json */ /** * Sample code: List all security connectors of a specified resource group. - * + * * @param manager Entry point to SecurityManager. */ - public static void listAllSecurityConnectorsOfASpecifiedResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + listAllSecurityConnectorsOfASpecifiedResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { manager.securityConnectors().listByResourceGroup("exampleResourceGroup", com.azure.core.util.Context.NONE); } } @@ -5186,39 +6290,32 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for SecurityConnectors Update. */ +/** + * Samples for SecurityConnectors Update. + */ public final class SecurityConnectorsUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-03-01-preview/examples/SecurityConnectors/PatchSecurityConnector_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-10-01-preview/examples/SecurityConnectors + * /PatchSecurityConnector_example.json */ /** * Sample code: Update a security connector. - * + * * @param manager Entry point to SecurityManager. */ public static void updateASecurityConnector(com.azure.resourcemanager.security.SecurityManager manager) { - SecurityConnector resource = - manager - .securityConnectors() - .getByResourceGroupWithResponse( - "exampleResourceGroup", "exampleSecurityConnectorName", com.azure.core.util.Context.NONE) - .getValue(); - resource - .update() - .withTags(mapOf()) - .withHierarchyIdentifier("exampleHierarchyId") + SecurityConnector resource = manager.securityConnectors().getByResourceGroupWithResponse("exampleResourceGroup", + "exampleSecurityConnectorName", com.azure.core.util.Context.NONE).getValue(); + resource.update().withTags(mapOf()).withHierarchyIdentifier("exampleHierarchyId") .withEnvironmentName(CloudName.AWS) - .withOfferings( - Arrays - .asList( - new CspmMonitorAwsOffering() - .withNativeCloudConnection( - new CspmMonitorAwsOfferingNativeCloudConnection() - .withCloudRoleArn("arn:aws:iam::00000000:role/ASCMonitor")))) - .withEnvironmentData(new AwsEnvironmentData()) - .apply(); + .withOfferings(Arrays.asList( + new CspmMonitorAwsOffering().withNativeCloudConnection(new CspmMonitorAwsOfferingNativeCloudConnection() + .withCloudRoleArn("arn:aws:iam::00000000:role/ASCMonitor")))) + .withEnvironmentData(new AwsEnvironmentData()).apply(); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -5235,37 +6332,38 @@ public final class SecurityConnectorsUpdateSamples { ### SecurityContacts_Create ```java +import com.azure.resourcemanager.security.models.MinimalRiskLevel; import com.azure.resourcemanager.security.models.MinimalSeverity; -import com.azure.resourcemanager.security.models.Roles; -import com.azure.resourcemanager.security.models.SecurityContactPropertiesAlertNotifications; +import com.azure.resourcemanager.security.models.NotificationsSourceAlert; +import com.azure.resourcemanager.security.models.NotificationsSourceAttackPath; +import com.azure.resourcemanager.security.models.SecurityContactName; import com.azure.resourcemanager.security.models.SecurityContactPropertiesNotificationsByRole; +import com.azure.resourcemanager.security.models.SecurityContactRole; import com.azure.resourcemanager.security.models.State; import java.util.Arrays; -/** Samples for SecurityContacts Create. */ +/** + * Samples for SecurityContacts Create. + */ public final class SecurityContactsCreateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/SecurityContacts/CreateSecurityContact_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/SecurityContacts/ + * CreateSecurityContact_example.json */ /** * Sample code: Create security contact data. - * + * * @param manager Entry point to SecurityManager. */ public static void createSecurityContactData(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityContacts() - .define("default") - .withEmails("john@contoso.com;jane@contoso.com") - .withPhone("(214)275-4038") - .withAlertNotifications( - new SecurityContactPropertiesAlertNotifications() - .withState(State.fromString("On")) - .withMinimalSeverity(MinimalSeverity.LOW)) - .withNotificationsByRole( - new SecurityContactPropertiesNotificationsByRole() - .withState(State.fromString("On")) - .withRoles(Arrays.asList(Roles.OWNER))) + manager.securityContacts().define(SecurityContactName.DEFAULT).withEmails("john@contoso.com;jane@contoso.com") + .withPhone("(214)275-4038").withIsEnabled(true) + .withNotificationsSources( + Arrays.asList(new NotificationsSourceAttackPath().withMinimalRiskLevel(MinimalRiskLevel.CRITICAL), + new NotificationsSourceAlert().withMinimalSeverity(MinimalSeverity.MEDIUM))) + .withNotificationsByRole(new SecurityContactPropertiesNotificationsByRole() + .withState(State.fromString("On")).withRoles(Arrays.asList(SecurityContactRole.OWNER))) .create(); } } @@ -5274,18 +6372,24 @@ public final class SecurityContactsCreateSamples { ### SecurityContacts_Delete ```java -/** Samples for SecurityContacts Delete. */ +import com.azure.resourcemanager.security.models.SecurityContactName; + +/** + * Samples for SecurityContacts Delete. + */ public final class SecurityContactsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/SecurityContacts/DeleteSecurityContact_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/SecurityContacts/ + * DeleteSecurityContact_example.json */ /** * Sample code: Deletes a security contact data. - * + * * @param manager Entry point to SecurityManager. */ public static void deletesASecurityContactData(com.azure.resourcemanager.security.SecurityManager manager) { - manager.securityContacts().deleteWithResponse("default", com.azure.core.util.Context.NONE); + manager.securityContacts().deleteWithResponse(SecurityContactName.DEFAULT, com.azure.core.util.Context.NONE); } } ``` @@ -5293,18 +6397,24 @@ public final class SecurityContactsDeleteSamples { ### SecurityContacts_Get ```java -/** Samples for SecurityContacts Get. */ +import com.azure.resourcemanager.security.models.SecurityContactName; + +/** + * Samples for SecurityContacts Get. + */ public final class SecurityContactsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/SecurityContacts/GetSecurityContact_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/SecurityContacts/ + * GetSecurityContact_example.json */ /** * Sample code: Get a security contact. - * + * * @param manager Entry point to SecurityManager. */ public static void getASecurityContact(com.azure.resourcemanager.security.SecurityManager manager) { - manager.securityContacts().getWithResponse("default", com.azure.core.util.Context.NONE); + manager.securityContacts().getWithResponse(SecurityContactName.DEFAULT, com.azure.core.util.Context.NONE); } } ``` @@ -5312,14 +6422,18 @@ public final class SecurityContactsGetSamples { ### SecurityContacts_List ```java -/** Samples for SecurityContacts List. */ +/** + * Samples for SecurityContacts List. + */ public final class SecurityContactsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2020-01-01-preview/examples/SecurityContacts/GetSecurityContactsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-12-01-preview/examples/SecurityContacts/ + * GetSecurityContactsSubscription_example.json */ /** * Sample code: List security contact data. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecurityContactData(com.azure.resourcemanager.security.SecurityManager manager) { @@ -5331,22 +6445,24 @@ public final class SecurityContactsListSamples { ### SecurityOperators_CreateOrUpdate ```java -/** Samples for SecurityOperators CreateOrUpdate. */ +/** + * Samples for SecurityOperators CreateOrUpdate. + */ public final class SecurityOperatorsCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/PutSecurityOperatorByName_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/ + * PutSecurityOperatorByName_example.json */ /** * Sample code: Create a security operator on the given scope. - * + * * @param manager Entry point to SecurityManager. */ - public static void createASecurityOperatorOnTheGivenScope( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityOperators() - .createOrUpdateWithResponse( - "CloudPosture", "DefenderCSPMSecurityOperator", com.azure.core.util.Context.NONE); + public static void + createASecurityOperatorOnTheGivenScope(com.azure.resourcemanager.security.SecurityManager manager) { + manager.securityOperators().createOrUpdateWithResponse("CloudPosture", "DefenderCSPMSecurityOperator", + com.azure.core.util.Context.NONE); } } ``` @@ -5354,22 +6470,24 @@ public final class SecurityOperatorsCreateOrUpdateSamples { ### SecurityOperators_Delete ```java -/** Samples for SecurityOperators Delete. */ +/** + * Samples for SecurityOperators Delete. + */ public final class SecurityOperatorsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/DeleteSecurityOperatorByName_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/ + * DeleteSecurityOperatorByName_example.json */ /** * Sample code: Delete SecurityOperator on subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteSecurityOperatorOnSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityOperators() - .deleteByResourceGroupWithResponse( - "CloudPosture", "DefenderCSPMSecurityOperator", com.azure.core.util.Context.NONE); + public static void + deleteSecurityOperatorOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.securityOperators().deleteByResourceGroupWithResponse("CloudPosture", "DefenderCSPMSecurityOperator", + com.azure.core.util.Context.NONE); } } ``` @@ -5377,21 +6495,24 @@ public final class SecurityOperatorsDeleteSamples { ### SecurityOperators_Get ```java -/** Samples for SecurityOperators Get. */ +/** + * Samples for SecurityOperators Get. + */ public final class SecurityOperatorsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/GetSecurityOperatorByName_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/ + * GetSecurityOperatorByName_example.json */ /** * Sample code: Get a specific security operator by scope and securityOperatorName. - * + * * @param manager Entry point to SecurityManager. */ public static void getASpecificSecurityOperatorByScopeAndSecurityOperatorName( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securityOperators() - .getWithResponse("CloudPosture", "DefenderCSPMSecurityOperator", com.azure.core.util.Context.NONE); + manager.securityOperators().getWithResponse("CloudPosture", "DefenderCSPMSecurityOperator", + com.azure.core.util.Context.NONE); } } ``` @@ -5399,14 +6520,18 @@ public final class SecurityOperatorsGetSamples { ### SecurityOperators_List ```java -/** Samples for SecurityOperators List. */ +/** + * Samples for SecurityOperators List. + */ public final class SecurityOperatorsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/ListSecurityOperators_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-01-01-preview/examples/SecurityOperators/ + * ListSecurityOperators_example.json */ /** * Sample code: List SecurityOperators. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecurityOperators(com.azure.resourcemanager.security.SecurityManager manager) { @@ -5418,21 +6543,24 @@ public final class SecurityOperatorsListSamples { ### SecuritySolutions_Get ```java -/** Samples for SecuritySolutions Get. */ +/** + * Samples for SecuritySolutions Get. + */ public final class SecuritySolutionsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/SecuritySolutions/GetSecuritySolutionsResourceGroupLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/SecuritySolutions/ + * GetSecuritySolutionsResourceGroupLocation_example.json */ /** * Sample code: Get a security solution from a security data location. - * + * * @param manager Entry point to SecurityManager. */ - public static void getASecuritySolutionFromASecurityDataLocation( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securitySolutions() - .getWithResponse("myRg2", "centralus", "paloalto7", com.azure.core.util.Context.NONE); + public static void + getASecuritySolutionFromASecurityDataLocation(com.azure.resourcemanager.security.SecurityManager manager) { + manager.securitySolutions().getWithResponse("myRg2", "centralus", "paloalto7", + com.azure.core.util.Context.NONE); } } ``` @@ -5440,14 +6568,18 @@ public final class SecuritySolutionsGetSamples { ### SecuritySolutions_List ```java -/** Samples for SecuritySolutions List. */ +/** + * Samples for SecuritySolutions List. + */ public final class SecuritySolutionsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/SecuritySolutions/GetSecuritySolutionsSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/SecuritySolutions/ + * GetSecuritySolutionsSubscription_example.json */ /** * Sample code: Get security solutions. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecuritySolutions(com.azure.resourcemanager.security.SecurityManager manager) { @@ -5459,14 +6591,17 @@ public final class SecuritySolutionsListSamples { ### SecuritySolutionsReferenceData_List ```java -/** Samples for SecuritySolutionsReferenceData List. */ +/** + * Samples for SecuritySolutionsReferenceData List. + */ public final class SecuritySolutionsReferenceDataListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/SecuritySolutionsReferenceData/GetSecuritySolutionsReferenceDataSubscription_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * SecuritySolutionsReferenceData/GetSecuritySolutionsReferenceDataSubscription_example.json */ /** * Sample code: Get security solutions. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecuritySolutions(com.azure.resourcemanager.security.SecurityManager manager) { @@ -5478,21 +6613,46 @@ public final class SecuritySolutionsReferenceDataListSamples { ### SecuritySolutionsReferenceData_ListByHomeRegion ```java -/** Samples for SecuritySolutionsReferenceData ListByHomeRegion. */ +/** + * Samples for SecuritySolutionsReferenceData ListByHomeRegion. + */ public final class SecuritySolutionsReferenceDataListByHomeRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/SecuritySolutionsReferenceData/GetSecuritySolutionsReferenceDataSubscriptionLocation_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * SecuritySolutionsReferenceData/GetSecuritySolutionsReferenceDataSubscriptionLocation_example.json */ /** * Sample code: Get security solutions from a security data location. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSecuritySolutionsFromASecurityDataLocation( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .securitySolutionsReferenceDatas() - .listByHomeRegionWithResponse("westcentralus", com.azure.core.util.Context.NONE); + public static void + getSecuritySolutionsFromASecurityDataLocation(com.azure.resourcemanager.security.SecurityManager manager) { + manager.securitySolutionsReferenceDatas().listByHomeRegionWithResponse("westcentralus", + com.azure.core.util.Context.NONE); + } +} +``` + +### SensitivitySettings_List + +```java +/** + * Samples for SensitivitySettings List. + */ +public final class SensitivitySettingsListSamples { + /* + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-15-preview/examples/ + * SensitivitySettings/GetSensitivitySettingsList_example.json + */ + /** + * Sample code: Get sensitivity settings list. + * + * @param manager Entry point to SecurityManager. + */ + public static void getSensitivitySettingsList(com.azure.resourcemanager.security.SecurityManager manager) { + manager.sensitivitySettings().listWithResponse(com.azure.core.util.Context.NONE); } } ``` @@ -5500,24 +6660,25 @@ public final class SecuritySolutionsReferenceDataListByHomeRegionSamples { ### ServerVulnerabilityAssessment_CreateOrUpdate ```java -/** Samples for ServerVulnerabilityAssessment CreateOrUpdate. */ +/** + * Samples for ServerVulnerabilityAssessment CreateOrUpdate. + */ public final class ServerVulnerabilityAssessmentCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ServerVulnerabilityAssessments/CreateServerVulnerabilityAssessments_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * ServerVulnerabilityAssessments/CreateServerVulnerabilityAssessments_example.json */ /** * Sample code: Create a server vulnerability assessments on a resource. Only 'default' resource is supported. Once * creating the resource, the server will be onboarded to vulnerability assessment by Microsoft.Security. - * + * * @param manager Entry point to SecurityManager. */ public static void createAServerVulnerabilityAssessmentsOnAResourceOnlyDefaultResourceIsSupportedOnceCreatingTheResourceTheServerWillBeOnboardedToVulnerabilityAssessmentByMicrosoftSecurity( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .serverVulnerabilityAssessments() - .createOrUpdateWithResponse( - "rg1", "Microsoft.Compute", "virtualMachines", "vm1", com.azure.core.util.Context.NONE); + manager.serverVulnerabilityAssessments().createOrUpdateWithResponse("rg1", "Microsoft.Compute", + "virtualMachines", "vm1", com.azure.core.util.Context.NONE); } } ``` @@ -5525,23 +6686,25 @@ public final class ServerVulnerabilityAssessmentCreateOrUpdateSamples { ### ServerVulnerabilityAssessment_Delete ```java -/** Samples for ServerVulnerabilityAssessment Delete. */ +/** + * Samples for ServerVulnerabilityAssessment Delete. + */ public final class ServerVulnerabilityAssessmentDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ServerVulnerabilityAssessments/DeleteServerVulnerabilityAssessments_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * ServerVulnerabilityAssessments/DeleteServerVulnerabilityAssessments_example.json */ /** * Sample code: Delete a server vulnerability assessments on a resource. Only 'default' resource is supported. Once * deleting, Microsoft.Security will not provide vulnerability assessment findings on the resource. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteAServerVulnerabilityAssessmentsOnAResourceOnlyDefaultResourceIsSupportedOnceDeletingMicrosoftSecurityWillNotProvideVulnerabilityAssessmentFindingsOnTheResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .serverVulnerabilityAssessments() - .delete("rg1", "Microsoft.Compute", "virtualMachines", "vm1", com.azure.core.util.Context.NONE); + manager.serverVulnerabilityAssessments().delete("rg1", "Microsoft.Compute", "virtualMachines", "vm1", + com.azure.core.util.Context.NONE); } } ``` @@ -5549,23 +6712,25 @@ public final class ServerVulnerabilityAssessmentDeleteSamples { ### ServerVulnerabilityAssessment_Get ```java -/** Samples for ServerVulnerabilityAssessment Get. */ +/** + * Samples for ServerVulnerabilityAssessment Get. + */ public final class ServerVulnerabilityAssessmentGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ServerVulnerabilityAssessments/GetServerVulnerabilityAssessments_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * ServerVulnerabilityAssessments/GetServerVulnerabilityAssessments_example.json */ /** * Sample code: Get a server vulnerability assessments onboarding status on a resource. Currently Microsoft.Security * only supports the single 'default' resource. - * + * * @param manager Entry point to SecurityManager. */ public static void getAServerVulnerabilityAssessmentsOnboardingStatusOnAResourceCurrentlyMicrosoftSecurityOnlySupportsTheSingleDefaultResource( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .serverVulnerabilityAssessments() - .getWithResponse("rg1", "Microsoft.Compute", "virtualMachines", "vm1", com.azure.core.util.Context.NONE); + manager.serverVulnerabilityAssessments().getWithResponse("rg1", "Microsoft.Compute", "virtualMachines", "vm1", + com.azure.core.util.Context.NONE); } } ``` @@ -5573,24 +6738,131 @@ public final class ServerVulnerabilityAssessmentGetSamples { ### ServerVulnerabilityAssessment_ListByExtendedResource ```java -/** Samples for ServerVulnerabilityAssessment ListByExtendedResource. */ +/** + * Samples for ServerVulnerabilityAssessment ListByExtendedResource. + */ public final class ServerVulnerabilityAssessmentListByExtendedResourceSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ServerVulnerabilityAssessments/ListByExtendedResourceServerVulnerabilityAssessments_example.json + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/ + * ServerVulnerabilityAssessments/ListByExtendedResourceServerVulnerabilityAssessments_example.json */ /** * Sample code: Get a list of server vulnerability assessments on a resource. Though this API returns a list, * Currently Microsoft.Security only supports a single default type of server vulnerability assessment. - * + * * @param manager Entry point to SecurityManager. */ public static void getAListOfServerVulnerabilityAssessmentsOnAResourceThoughThisAPIReturnsAListCurrentlyMicrosoftSecurityOnlySupportsASingleDefaultTypeOfServerVulnerabilityAssessment( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .serverVulnerabilityAssessments() - .listByExtendedResourceWithResponse( - "rg1", "Microsoft.Compute", "virtualMachines", "vm1", com.azure.core.util.Context.NONE); + manager.serverVulnerabilityAssessments().listByExtendedResourceWithResponse("rg1", "Microsoft.Compute", + "virtualMachines", "vm1", com.azure.core.util.Context.NONE); + } +} +``` + +### ServerVulnerabilityAssessmentsSettings_CreateOrUpdate + +```java +import com.azure.resourcemanager.security.models.AzureServersSetting; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsAzureSettingSelectedProvider; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingKindName; + +/** + * Samples for ServerVulnerabilityAssessmentsSettings CreateOrUpdate. + */ +public final class ServerVulnerabilityAssessmentsSettingsCreateOrUpdateSamples { + /* + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-05-01/examples/ + * ServerVulnerabilityAssessmentsSettings/PutServerVulnerabilityAssessmentsSetting_example.json + */ + /** + * Sample code: Set a server vulnerability assessments setting of the kind settingKind on the subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void setAServerVulnerabilityAssessmentsSettingOfTheKindSettingKindOnTheSubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.serverVulnerabilityAssessmentsSettings().createOrUpdateWithResponse( + ServerVulnerabilityAssessmentsSettingKindName.AZURE_SERVERS_SETTING, new AzureServersSetting() + .withSelectedProvider(ServerVulnerabilityAssessmentsAzureSettingSelectedProvider.MDE_TVM), + com.azure.core.util.Context.NONE); + } +} +``` + +### ServerVulnerabilityAssessmentsSettings_Delete + +```java +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingKindName; + +/** + * Samples for ServerVulnerabilityAssessmentsSettings Delete. + */ +public final class ServerVulnerabilityAssessmentsSettingsDeleteSamples { + /* + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-05-01/examples/ + * ServerVulnerabilityAssessmentsSettings/DeleteServerVulnerabilityAssessmentsSetting_example.json + */ + /** + * Sample code: Delete the server vulnerability assessments setting of the kind settingKind from the subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void deleteTheServerVulnerabilityAssessmentsSettingOfTheKindSettingKindFromTheSubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.serverVulnerabilityAssessmentsSettings().deleteWithResponse( + ServerVulnerabilityAssessmentsSettingKindName.AZURE_SERVERS_SETTING, com.azure.core.util.Context.NONE); + } +} +``` + +### ServerVulnerabilityAssessmentsSettings_Get + +```java +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingKindName; + +/** + * Samples for ServerVulnerabilityAssessmentsSettings Get. + */ +public final class ServerVulnerabilityAssessmentsSettingsGetSamples { + /* + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-05-01/examples/ + * ServerVulnerabilityAssessmentsSettings/GetServerVulnerabilityAssessmentsSetting_example.json + */ + /** + * Sample code: Get the server vulnerability assessments setting of the kind settingKind that is set on the + * subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void getTheServerVulnerabilityAssessmentsSettingOfTheKindSettingKindThatIsSetOnTheSubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.serverVulnerabilityAssessmentsSettings().getWithResponse( + ServerVulnerabilityAssessmentsSettingKindName.AZURE_SERVERS_SETTING, com.azure.core.util.Context.NONE); + } +} +``` + +### ServerVulnerabilityAssessmentsSettings_List + +```java +/** + * Samples for ServerVulnerabilityAssessmentsSettings List. + */ +public final class ServerVulnerabilityAssessmentsSettingsListSamples { + /* + * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2023-05-01/examples/ + * ServerVulnerabilityAssessmentsSettings/ListServerVulnerabilityAssessmentsSettings_example.json + */ + /** + * Sample code: List the server vulnerability assessments settings set on the subscription. + * + * @param manager Entry point to SecurityManager. + */ + public static void listTheServerVulnerabilityAssessmentsSettingsSetOnTheSubscription( + com.azure.resourcemanager.security.SecurityManager manager) { + manager.serverVulnerabilityAssessmentsSettings().list(com.azure.core.util.Context.NONE); } } ``` @@ -5598,20 +6870,24 @@ public final class ServerVulnerabilityAssessmentListByExtendedResourceSamples { ### Settings_Get ```java -import com.azure.resourcemanager.security.models.SettingName; +import com.azure.resourcemanager.security.models.SettingNameAutoGenerated; -/** Samples for Settings Get. */ +/** + * Samples for Settings Get. + */ public final class SettingsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-05-01/examples/Settings/GetSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2022-05-01/examples/Settings/GetSetting_example + * .json */ /** * Sample code: Get a setting on subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getASettingOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager.settings().getWithResponse(SettingName.WDATP, com.azure.core.util.Context.NONE); + manager.settings().getWithResponse(SettingNameAutoGenerated.WDATP, com.azure.core.util.Context.NONE); } } ``` @@ -5619,14 +6895,18 @@ public final class SettingsGetSamples { ### Settings_List ```java -/** Samples for Settings List. */ +/** + * Samples for Settings List. + */ public final class SettingsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-05-01/examples/Settings/GetSettings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2022-05-01/examples/Settings/ + * GetSettings_example.json */ /** * Sample code: Get settings of subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getSettingsOfSubscription(com.azure.resourcemanager.security.SecurityManager manager) { @@ -5639,23 +6919,25 @@ public final class SettingsListSamples { ```java import com.azure.resourcemanager.security.models.DataExportSettings; -import com.azure.resourcemanager.security.models.SettingName; +import com.azure.resourcemanager.security.models.SettingNameAutoGenerated; -/** Samples for Settings Update. */ +/** + * Samples for Settings Update. + */ public final class SettingsUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2022-05-01/examples/Settings/UpdateSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2022-05-01/examples/Settings/ + * UpdateSetting_example.json */ /** * Sample code: Update a setting for subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void updateASettingForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .settings() - .updateWithResponse( - SettingName.WDATP, new DataExportSettings().withEnabled(true), com.azure.core.util.Context.NONE); + manager.settings().updateWithResponse(SettingNameAutoGenerated.WDATP, + new DataExportSettings().withEnabled(true), com.azure.core.util.Context.NONE); } } ``` @@ -5663,27 +6945,24 @@ public final class SettingsUpdateSamples { ### SoftwareInventories_Get ```java -/** Samples for SoftwareInventories Get. */ +/** + * Samples for SoftwareInventories Get. + */ public final class SoftwareInventoriesGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-05-01-preview/examples/SoftwareInventories/GetSoftware_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-05-01-preview/examples/ + * SoftwareInventories/GetSoftware_example.json */ /** * Sample code: Gets a single software data of the virtual machine. - * + * * @param manager Entry point to SecurityManager. */ - public static void getsASingleSoftwareDataOfTheVirtualMachine( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .softwareInventories() - .getWithResponse( - "EITAN-TESTS", - "Microsoft.Compute", - "virtualMachines", - "Eitan-Test1", - "outlook_16.0.10371.20060", - com.azure.core.util.Context.NONE); + public static void + getsASingleSoftwareDataOfTheVirtualMachine(com.azure.resourcemanager.security.SecurityManager manager) { + manager.softwareInventories().getWithResponse("EITAN-TESTS", "Microsoft.Compute", "virtualMachines", + "Eitan-Test1", "outlook_16.0.10371.20060", com.azure.core.util.Context.NONE); } } ``` @@ -5691,14 +6970,18 @@ public final class SoftwareInventoriesGetSamples { ### SoftwareInventories_List ```java -/** Samples for SoftwareInventories List. */ +/** + * Samples for SoftwareInventories List. + */ public final class SoftwareInventoriesListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-05-01-preview/examples/SoftwareInventories/ListBySubscriptionSoftwareInventories_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-05-01-preview/examples/ + * SoftwareInventories/ListBySubscriptionSoftwareInventories_example.json */ /** * Sample code: Gets the software inventory of all virtual machines in the subscriptions. - * + * * @param manager Entry point to SecurityManager. */ public static void getsTheSoftwareInventoryOfAllVirtualMachinesInTheSubscriptions( @@ -5711,22 +6994,24 @@ public final class SoftwareInventoriesListSamples { ### SoftwareInventories_ListByExtendedResource ```java -/** Samples for SoftwareInventories ListByExtendedResource. */ +/** + * Samples for SoftwareInventories ListByExtendedResource. + */ public final class SoftwareInventoriesListByExtendedResourceSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2021-05-01-preview/examples/SoftwareInventories/ListByExtendedResourceSoftwareInventories_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2021-05-01-preview/examples/ + * SoftwareInventories/ListByExtendedResourceSoftwareInventories_example.json */ /** * Sample code: Gets the software inventory of the virtual machine. - * + * * @param manager Entry point to SecurityManager. */ - public static void getsTheSoftwareInventoryOfTheVirtualMachine( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .softwareInventories() - .listByExtendedResource( - "EITAN-TESTS", "Microsoft.Compute", "virtualMachines", "Eitan-Test1", com.azure.core.util.Context.NONE); + public static void + getsTheSoftwareInventoryOfTheVirtualMachine(com.azure.resourcemanager.security.SecurityManager manager) { + manager.softwareInventories().listByExtendedResource("EITAN-TESTS", "Microsoft.Compute", "virtualMachines", + "Eitan-Test1", com.azure.core.util.Context.NONE); } } ``` @@ -5739,52 +7024,47 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; -/** Samples for SqlVulnerabilityAssessmentBaselineRules Add. */ +/** + * Samples for SqlVulnerabilityAssessmentBaselineRules Add. + */ public final class SqlVulnerabilityAssessmentBaselineRulesAddSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Add.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Add.json */ /** * Sample code: Create a baseline for all rules. - * + * * @param manager Entry point to SecurityManager. */ public static void createABaselineForAllRules(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .addWithResponse( - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - new RulesResultsInput() - .withLatestScan(false) - .withResults( - mapOf( - "VA1234", - Arrays.asList(Arrays.asList("userA", "SELECT"), Arrays.asList("userB", "SELECT")), - "VA5678", - Arrays.asList(Arrays.asList("Test", "0.0.0.0", "125.125.125.125")))), - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentBaselineRules().addWithResponse("55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + new RulesResultsInput().withLatestScan(false).withResults( + mapOf("VA1234", Arrays.asList(Arrays.asList("userA", "SELECT"), Arrays.asList("userB", "SELECT")), + "VA5678", Arrays.asList(Arrays.asList("Test", "0.0.0.0", "125.125.125.125")))), + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_AddLatest.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_AddLatest.json */ /** * Sample code: Create a baseline for all rules using the latest scan results. - * + * * @param manager Entry point to SecurityManager. */ public static void createABaselineForAllRulesUsingTheLatestScanResults( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .addWithResponse( - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - new RulesResultsInput().withLatestScan(true).withResults(mapOf()), - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentBaselineRules().addWithResponse("55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + new RulesResultsInput().withLatestScan(true).withResults(mapOf()), com.azure.core.util.Context.NONE); } + // Use "Map.of" if available @SuppressWarnings("unchecked") private static Map mapOf(Object... inputs) { Map map = new HashMap<>(); @@ -5803,47 +7083,44 @@ public final class SqlVulnerabilityAssessmentBaselineRulesAddSamples { ```java import java.util.Arrays; -/** Samples for SqlVulnerabilityAssessmentBaselineRules CreateOrUpdate. */ +/** + * Samples for SqlVulnerabilityAssessmentBaselineRules CreateOrUpdate. + */ public final class SqlVulnerabilityAssessmentBaselineRulesCreateOrUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_PutLatest.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_PutLatest.json */ /** * Sample code: Create a baseline using the latest scan results. - * + * * @param manager Entry point to SecurityManager. */ - public static void createABaselineUsingTheLatestScanResults( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .define("VA1234") - .withExistingResourceId( - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master") - .withLatestScan(true) - .withResults(Arrays.asList()) - .withWorkspaceId("55555555-6666-7777-8888-999999999999") + public static void + createABaselineUsingTheLatestScanResults(com.azure.resourcemanager.security.SecurityManager manager) { + manager.sqlVulnerabilityAssessmentBaselineRules().define("VA1234").withExistingResourceId( + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master") + .withLatestScan(true).withResults(Arrays.asList()).withWorkspaceId("55555555-6666-7777-8888-999999999999") .create(); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Put.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Put.json */ /** * Sample code: Create a baseline. - * + * * @param manager Entry point to SecurityManager. */ public static void createABaseline(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .define("VA1234") - .withExistingResourceId( - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master") + manager.sqlVulnerabilityAssessmentBaselineRules().define("VA1234").withExistingResourceId( + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master") .withLatestScan(false) .withResults(Arrays.asList(Arrays.asList("userA", "SELECT"), Arrays.asList("userB", "SELECT"))) - .withWorkspaceId("55555555-6666-7777-8888-999999999999") - .create(); + .withWorkspaceId("55555555-6666-7777-8888-999999999999").create(); } } ``` @@ -5851,24 +7128,25 @@ public final class SqlVulnerabilityAssessmentBaselineRulesCreateOrUpdateSamples ### SqlVulnerabilityAssessmentBaselineRules_Delete ```java -/** Samples for SqlVulnerabilityAssessmentBaselineRules Delete. */ +/** + * Samples for SqlVulnerabilityAssessmentBaselineRules Delete. + */ public final class SqlVulnerabilityAssessmentBaselineRulesDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Delete.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Delete.json */ /** * Sample code: Delete the baseline. - * + * * @param manager Entry point to SecurityManager. */ public static void deleteTheBaseline(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .deleteWithResponse( - "VA1234", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentBaselineRules().deleteWithResponse("VA1234", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -5876,24 +7154,25 @@ public final class SqlVulnerabilityAssessmentBaselineRulesDeleteSamples { ### SqlVulnerabilityAssessmentBaselineRules_Get ```java -/** Samples for SqlVulnerabilityAssessmentBaselineRules Get. */ +/** + * Samples for SqlVulnerabilityAssessmentBaselineRules Get. + */ public final class SqlVulnerabilityAssessmentBaselineRulesGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Get.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_Get.json */ /** * Sample code: Get the baseline. - * + * * @param manager Entry point to SecurityManager. */ public static void getTheBaseline(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .getWithResponse( - "VA1234", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentBaselineRules().getWithResponse("VA1234", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -5901,23 +7180,24 @@ public final class SqlVulnerabilityAssessmentBaselineRulesGetSamples { ### SqlVulnerabilityAssessmentBaselineRules_List ```java -/** Samples for SqlVulnerabilityAssessmentBaselineRules List. */ +/** + * Samples for SqlVulnerabilityAssessmentBaselineRules List. + */ public final class SqlVulnerabilityAssessmentBaselineRulesListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_List.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsBaselineRuleOperations/ArcMachineBaselineRules_List.json */ /** * Sample code: List baseline for all rules. - * + * * @param manager Entry point to SecurityManager. */ public static void listBaselineForAllRules(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentBaselineRules() - .listWithResponse( - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentBaselineRules().listWithResponse("55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -5925,44 +7205,42 @@ public final class SqlVulnerabilityAssessmentBaselineRulesListSamples { ### SqlVulnerabilityAssessmentScanResults_Get ```java -/** Samples for SqlVulnerabilityAssessmentScanResults Get. */ +/** + * Samples for SqlVulnerabilityAssessmentScanResults Get. + */ public final class SqlVulnerabilityAssessmentScanResultsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_Get.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_Get.json */ /** * Sample code: Get scan details of a scan record. - * + * * @param manager Entry point to SecurityManager. */ public static void getScanDetailsOfAScanRecord(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScanResults() - .getWithResponse( - "Scheduled-20200623", - "VA2063", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScanResults().getWithResponse("Scheduled-20200623", "VA2063", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_GetLatest.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_GetLatest.json */ /** * Sample code: Get scan details of the latest scan record. - * + * * @param manager Entry point to SecurityManager. */ public static void getScanDetailsOfTheLatestScanRecord(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScanResults() - .getWithResponse( - "latest", - "VA2063", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScanResults().getWithResponse("latest", "VA2063", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -5970,42 +7248,42 @@ public final class SqlVulnerabilityAssessmentScanResultsGetSamples { ### SqlVulnerabilityAssessmentScanResults_List ```java -/** Samples for SqlVulnerabilityAssessmentScanResults List. */ +/** + * Samples for SqlVulnerabilityAssessmentScanResults List. + */ public final class SqlVulnerabilityAssessmentScanResultsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_ListLatest.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_ListLatest.json */ /** * Sample code: List scan results of the latest scan. - * + * * @param manager Entry point to SecurityManager. */ public static void listScanResultsOfTheLatestScan(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScanResults() - .listWithResponse( - "latest", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScanResults().listWithResponse("latest", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_List.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentsScanResultsOperations/ArcMachineScanResults_List.json */ /** * Sample code: List scan results. - * + * * @param manager Entry point to SecurityManager. */ public static void listScanResults(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScanResults() - .listWithResponse( - "Scheduled-20200623", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScanResults().listWithResponse("Scheduled-20200623", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -6013,42 +7291,41 @@ public final class SqlVulnerabilityAssessmentScanResultsListSamples { ### SqlVulnerabilityAssessmentScans_Get ```java -/** Samples for SqlVulnerabilityAssessmentScans Get. */ +/** + * Samples for SqlVulnerabilityAssessmentScans Get. + */ public final class SqlVulnerabilityAssessmentScansGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentScanOperations/ArcMachineScans_Get.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentScanOperations/ArcMachineScans_Get.json */ /** * Sample code: Get scan details of a scan record. - * + * * @param manager Entry point to SecurityManager. */ public static void getScanDetailsOfAScanRecord(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScans() - .getWithResponse( - "Scheduled-20200623", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScans().getWithResponse("Scheduled-20200623", + "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentScanOperations/ArcMachineScans_GetLatest.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentScanOperations/ArcMachineScans_GetLatest.json */ /** * Sample code: Get scan details of the latest scan record. - * + * * @param manager Entry point to SecurityManager. */ public static void getScanDetailsOfTheLatestScanRecord(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScans() - .getWithResponse( - "latest", - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScans().getWithResponse("latest", "55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -6056,23 +7333,24 @@ public final class SqlVulnerabilityAssessmentScansGetSamples { ### SqlVulnerabilityAssessmentScans_List ```java -/** Samples for SqlVulnerabilityAssessmentScans List. */ +/** + * Samples for SqlVulnerabilityAssessmentScans List. + */ public final class SqlVulnerabilityAssessmentScansListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/sqlVulnerabilityAssessmentScanOperations/ArcMachineScans_List.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2023-02-01-preview/examples/ + * sqlVulnerabilityAssessmentScanOperations/ArcMachineScans_List.json */ /** * Sample code: List scan details. - * + * * @param manager Entry point to SecurityManager. */ public static void listScanDetails(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .sqlVulnerabilityAssessmentScans() - .listWithResponse( - "55555555-6666-7777-8888-999999999999", - "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", - com.azure.core.util.Context.NONE); + manager.sqlVulnerabilityAssessmentScans().listWithResponse("55555555-6666-7777-8888-999999999999", + "subscriptions/00000000-1111-2222-3333-444444444444/resourceGroups/Rg/providers/Microsoft.HybridCompute/machines/MyMachine/sqlServers/server1/databases/master", + com.azure.core.util.Context.NONE); } } ``` @@ -6080,25 +7358,26 @@ public final class SqlVulnerabilityAssessmentScansListSamples { ### SubAssessments_Get ```java -/** Samples for SubAssessments Get. */ +/** + * Samples for SubAssessments Get. + */ public final class SubAssessmentsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/SubAssessments/GetSubAssessment_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/SubAssessments/ + * GetSubAssessment_example.json */ /** * Sample code: Get security recommendation task from security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityRecommendationTaskFromSecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .subAssessments() - .getWithResponse( - "subscriptions/212f9889-769e-45ae-ab43-6da33674bd26/resourceGroups/DEMORG/providers/Microsoft.Compute/virtualMachines/vm2", - "1195afff-c881-495e-9bc5-1486211ae03f", - "95f7da9c-a2a4-1322-0758-fcd24ef09b85", - com.azure.core.util.Context.NONE); + manager.subAssessments().getWithResponse( + "subscriptions/212f9889-769e-45ae-ab43-6da33674bd26/resourceGroups/DEMORG/providers/Microsoft.Compute/virtualMachines/vm2", + "1195afff-c881-495e-9bc5-1486211ae03f", "95f7da9c-a2a4-1322-0758-fcd24ef09b85", + com.azure.core.util.Context.NONE); } } ``` @@ -6106,23 +7385,23 @@ public final class SubAssessmentsGetSamples { ### SubAssessments_List ```java -/** Samples for SubAssessments List. */ +/** + * Samples for SubAssessments List. + */ public final class SubAssessmentsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/SubAssessments/ListSubAssessments_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/SubAssessments/ + * ListSubAssessments_example.json */ /** * Sample code: List security sub-assessments. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecuritySubAssessments(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .subAssessments() - .list( - "subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", - "82e20e14-edc5-4373-bfc4-f13121257c37", - com.azure.core.util.Context.NONE); + manager.subAssessments().list("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + "82e20e14-edc5-4373-bfc4-f13121257c37", com.azure.core.util.Context.NONE); } } ``` @@ -6130,20 +7409,23 @@ public final class SubAssessmentsListSamples { ### SubAssessments_ListAll ```java -/** Samples for SubAssessments ListAll. */ +/** + * Samples for SubAssessments ListAll. + */ public final class SubAssessmentsListAllSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/SubAssessments/ListSubscriptionSubAssessments_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2019-01-01-preview/examples/SubAssessments/ + * ListSubscriptionSubAssessments_example.json */ /** * Sample code: List security sub-assessments. - * + * * @param manager Entry point to SecurityManager. */ public static void listSecuritySubAssessments(com.azure.resourcemanager.security.SecurityManager manager) { - manager - .subAssessments() - .listAll("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", com.azure.core.util.Context.NONE); + manager.subAssessments().listAll("subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23", + com.azure.core.util.Context.NONE); } } ``` @@ -6151,22 +7433,24 @@ public final class SubAssessmentsListAllSamples { ### Tasks_GetResourceGroupLevelTask ```java -/** Samples for Tasks GetResourceGroupLevelTask. */ +/** + * Samples for Tasks GetResourceGroupLevelTask. + */ public final class TasksGetResourceGroupLevelTaskSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/GetTaskResourceGroupLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * GetTaskResourceGroupLocation_example.json */ /** * Sample code: Get security recommendation task in a resource group. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityRecommendationTaskInAResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .tasks() - .getResourceGroupLevelTaskWithResponse( - "myRg", "westeurope", "d55b4dc0-779c-c66c-33e5-d7bce24c4222", com.azure.core.util.Context.NONE); + public static void + getSecurityRecommendationTaskInAResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { + manager.tasks().getResourceGroupLevelTaskWithResponse("myRg", "westeurope", + "d55b4dc0-779c-c66c-33e5-d7bce24c4222", com.azure.core.util.Context.NONE); } } ``` @@ -6174,22 +7458,24 @@ public final class TasksGetResourceGroupLevelTaskSamples { ### Tasks_GetSubscriptionLevelTask ```java -/** Samples for Tasks GetSubscriptionLevelTask. */ +/** + * Samples for Tasks GetSubscriptionLevelTask. + */ public final class TasksGetSubscriptionLevelTaskSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/GetTaskSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * GetTaskSubscriptionLocation_example.json */ /** * Sample code: Get security recommendation task from security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityRecommendationTaskFromSecurityDataLocation( com.azure.resourcemanager.security.SecurityManager manager) { - manager - .tasks() - .getSubscriptionLevelTaskWithResponse( - "westeurope", "62609ee7-d0a5-8616-9fe4-1df5cca7758d", com.azure.core.util.Context.NONE); + manager.tasks().getSubscriptionLevelTaskWithResponse("westeurope", "62609ee7-d0a5-8616-9fe4-1df5cca7758d", + com.azure.core.util.Context.NONE); } } ``` @@ -6197,14 +7483,18 @@ public final class TasksGetSubscriptionLevelTaskSamples { ### Tasks_List ```java -/** Samples for Tasks List. */ +/** + * Samples for Tasks List. + */ public final class TasksListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/GetTasksSubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * GetTasksSubscription_example.json */ /** * Sample code: Get security recommendations tasks. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityRecommendationsTasks(com.azure.resourcemanager.security.SecurityManager manager) { @@ -6216,14 +7506,18 @@ public final class TasksListSamples { ### Tasks_ListByHomeRegion ```java -/** Samples for Tasks ListByHomeRegion. */ +/** + * Samples for Tasks ListByHomeRegion. + */ public final class TasksListByHomeRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/GetTasksSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * GetTasksSubscriptionLocation_example.json */ /** * Sample code: Get security recommendations tasks from security data location. - * + * * @param manager Entry point to SecurityManager. */ public static void getSecurityRecommendationsTasksFromSecurityDataLocation( @@ -6236,18 +7530,22 @@ public final class TasksListByHomeRegionSamples { ### Tasks_ListByResourceGroup ```java -/** Samples for Tasks ListByResourceGroup. */ +/** + * Samples for Tasks ListByResourceGroup. + */ public final class TasksListByResourceGroupSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/GetTasksResourceGroupLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * GetTasksResourceGroupLocation_example.json */ /** * Sample code: Get security recommendation tasks in a resource group. - * + * * @param manager Entry point to SecurityManager. */ - public static void getSecurityRecommendationTasksInAResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getSecurityRecommendationTasksInAResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { manager.tasks().listByResourceGroup("myRg", "westeurope", null, com.azure.core.util.Context.NONE); } } @@ -6258,26 +7556,24 @@ public final class TasksListByResourceGroupSamples { ```java import com.azure.resourcemanager.security.models.TaskUpdateActionType; -/** Samples for Tasks UpdateResourceGroupLevelTaskState. */ +/** + * Samples for Tasks UpdateResourceGroupLevelTaskState. + */ public final class TasksUpdateResourceGroupLevelTaskStateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/UpdateTaskResourceGroupLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * UpdateTaskResourceGroupLocation_example.json */ /** * Sample code: Change security recommendation task state. - * + * * @param manager Entry point to SecurityManager. */ - public static void changeSecurityRecommendationTaskState( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .tasks() - .updateResourceGroupLevelTaskStateWithResponse( - "myRg", - "westeurope", - "d55b4dc0-779c-c66c-33e5-d7bce24c4222", - TaskUpdateActionType.DISMISS, - com.azure.core.util.Context.NONE); + public static void + changeSecurityRecommendationTaskState(com.azure.resourcemanager.security.SecurityManager manager) { + manager.tasks().updateResourceGroupLevelTaskStateWithResponse("myRg", "westeurope", + "d55b4dc0-779c-c66c-33e5-d7bce24c4222", TaskUpdateActionType.DISMISS, com.azure.core.util.Context.NONE); } } ``` @@ -6287,25 +7583,24 @@ public final class TasksUpdateResourceGroupLevelTaskStateSamples { ```java import com.azure.resourcemanager.security.models.TaskUpdateActionType; -/** Samples for Tasks UpdateSubscriptionLevelTaskState. */ +/** + * Samples for Tasks UpdateSubscriptionLevelTaskState. + */ public final class TasksUpdateSubscriptionLevelTaskStateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/UpdateTaskSubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2015-06-01-preview/examples/Tasks/ + * UpdateTaskSubscriptionLocation_example.json */ /** * Sample code: Change security recommendation task state. - * + * * @param manager Entry point to SecurityManager. */ - public static void changeSecurityRecommendationTaskState( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .tasks() - .updateSubscriptionLevelTaskStateWithResponse( - "westeurope", - "62609ee7-d0a5-8616-9fe4-1df5cca7758d", - TaskUpdateActionType.DISMISS, - com.azure.core.util.Context.NONE); + public static void + changeSecurityRecommendationTaskState(com.azure.resourcemanager.security.SecurityManager manager) { + manager.tasks().updateSubscriptionLevelTaskStateWithResponse("westeurope", + "62609ee7-d0a5-8616-9fe4-1df5cca7758d", TaskUpdateActionType.DISMISS, com.azure.core.util.Context.NONE); } } ``` @@ -6313,14 +7608,18 @@ public final class TasksUpdateSubscriptionLevelTaskStateSamples { ### Topology_Get ```java -/** Samples for Topology Get. */ +/** + * Samples for Topology Get. + */ public final class TopologyGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/Topology/GetTopology_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/Topology/ + * GetTopology_example.json */ /** * Sample code: Get topology. - * + * * @param manager Entry point to SecurityManager. */ public static void getTopology(com.azure.resourcemanager.security.SecurityManager manager) { @@ -6332,14 +7631,18 @@ public final class TopologyGetSamples { ### Topology_List ```java -/** Samples for Topology List. */ +/** + * Samples for Topology List. + */ public final class TopologyListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/Topology/GetTopologySubscription_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/Topology/ + * GetTopologySubscription_example.json */ /** * Sample code: Get topology on a subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getTopologyOnASubscription(com.azure.resourcemanager.security.SecurityManager manager) { @@ -6351,18 +7654,22 @@ public final class TopologyListSamples { ### Topology_ListByHomeRegion ```java -/** Samples for Topology ListByHomeRegion. */ +/** + * Samples for Topology ListByHomeRegion. + */ public final class TopologyListByHomeRegionSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/Topology/GetTopologySubscriptionLocation_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/stable/2020-01-01/examples/Topology/ + * GetTopologySubscriptionLocation_example.json */ /** * Sample code: Get topology on a subscription from security data location. - * + * * @param manager Entry point to SecurityManager. */ - public static void getTopologyOnASubscriptionFromSecurityDataLocation( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + getTopologyOnASubscriptionFromSecurityDataLocation(com.azure.resourcemanager.security.SecurityManager manager) { manager.topologies().listByHomeRegion("centralus", com.azure.core.util.Context.NONE); } } @@ -6371,25 +7678,25 @@ public final class TopologyListByHomeRegionSamples { ### WorkspaceSettings_Create ```java -/** Samples for WorkspaceSettings Create. */ +/** + * Samples for WorkspaceSettings Create. + */ public final class WorkspaceSettingsCreateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/CreateWorkspaceSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/ + * CreateWorkspaceSetting_example.json */ /** * Sample code: Create a workspace setting data for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void createAWorkspaceSettingDataForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - manager - .workspaceSettings() - .define("default") - .withWorkspaceId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.OperationalInsights/workspaces/myWorkspace") - .withScope("/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23") - .create(); + public static void + createAWorkspaceSettingDataForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + manager.workspaceSettings().define("default").withWorkspaceId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.OperationalInsights/workspaces/myWorkspace") + .withScope("/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23").create(); } } ``` @@ -6397,18 +7704,22 @@ public final class WorkspaceSettingsCreateSamples { ### WorkspaceSettings_Delete ```java -/** Samples for WorkspaceSettings Delete. */ +/** + * Samples for WorkspaceSettings Delete. + */ public final class WorkspaceSettingsDeleteSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/DeleteWorkspaceSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/ + * DeleteWorkspaceSetting_example.json */ /** * Sample code: Delete a workspace setting data for resource group. - * + * * @param manager Entry point to SecurityManager. */ - public static void deleteAWorkspaceSettingDataForResourceGroup( - com.azure.resourcemanager.security.SecurityManager manager) { + public static void + deleteAWorkspaceSettingDataForResourceGroup(com.azure.resourcemanager.security.SecurityManager manager) { manager.workspaceSettings().deleteWithResponse("default", com.azure.core.util.Context.NONE); } } @@ -6417,14 +7728,18 @@ public final class WorkspaceSettingsDeleteSamples { ### WorkspaceSettings_Get ```java -/** Samples for WorkspaceSettings Get. */ +/** + * Samples for WorkspaceSettings Get. + */ public final class WorkspaceSettingsGetSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/GetWorkspaceSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/ + * GetWorkspaceSetting_example.json */ /** * Sample code: Get a workspace setting on subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getAWorkspaceSettingOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { @@ -6436,14 +7751,18 @@ public final class WorkspaceSettingsGetSamples { ### WorkspaceSettings_List ```java -/** Samples for WorkspaceSettings List. */ +/** + * Samples for WorkspaceSettings List. + */ public final class WorkspaceSettingsListSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/GetWorkspaceSettings_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/ + * GetWorkspaceSettings_example.json */ /** * Sample code: Get workspace settings on subscription. - * + * * @param manager Entry point to SecurityManager. */ public static void getWorkspaceSettingsOnSubscription(com.azure.resourcemanager.security.SecurityManager manager) { @@ -6457,24 +7776,26 @@ public final class WorkspaceSettingsListSamples { ```java import com.azure.resourcemanager.security.models.WorkspaceSetting; -/** Samples for WorkspaceSettings Update. */ +/** + * Samples for WorkspaceSettings Update. + */ public final class WorkspaceSettingsUpdateSamples { /* - * x-ms-original-file: specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/UpdateWorkspaceSetting_example.json + * x-ms-original-file: + * specification/security/resource-manager/Microsoft.Security/preview/2017-08-01-preview/examples/WorkspaceSettings/ + * UpdateWorkspaceSetting_example.json */ /** * Sample code: Update a workspace setting data for subscription. - * + * * @param manager Entry point to SecurityManager. */ - public static void updateAWorkspaceSettingDataForSubscription( - com.azure.resourcemanager.security.SecurityManager manager) { - WorkspaceSetting resource = - manager.workspaceSettings().getWithResponse("default", com.azure.core.util.Context.NONE).getValue(); - resource - .update() - .withWorkspaceId( - "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.OperationalInsights/workspaces/myWorkspace") + public static void + updateAWorkspaceSettingDataForSubscription(com.azure.resourcemanager.security.SecurityManager manager) { + WorkspaceSetting resource + = manager.workspaceSettings().getWithResponse("default", com.azure.core.util.Context.NONE).getValue(); + resource.update().withWorkspaceId( + "/subscriptions/20ff7fc3-e762-44dd-bd96-b71116dcdc23/resourceGroups/myRg/providers/Microsoft.OperationalInsights/workspaces/myWorkspace") .apply(); } } diff --git a/sdk/security/azure-resourcemanager-security/pom.xml b/sdk/security/azure-resourcemanager-security/pom.xml index a17ad75df9f51..1163d5eae759c 100644 --- a/sdk/security/azure-resourcemanager-security/pom.xml +++ b/sdk/security/azure-resourcemanager-security/pom.xml @@ -14,7 +14,7 @@ com.azure.resourcemanager azure-resourcemanager-security - 1.0.0-beta.6 + 1.0.0-beta.7 jar Microsoft Azure SDK for Security Management @@ -45,6 +45,7 @@ UTF-8 0 0 + true @@ -66,7 +67,7 @@ com.azure azure-identity - 1.11.3 + 1.11.4 test @@ -87,18 +88,16 @@ 4.11.0 test - - net.bytebuddy byte-buddy - 1.14.8 + 1.14.12 test net.bytebuddy byte-buddy-agent - 1.14.8 + 1.14.12 test diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/SecurityManager.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/SecurityManager.java index 80fd7da717a70..de2f8a7627be7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/SecurityManager.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/SecurityManager.java @@ -30,8 +30,6 @@ import com.azure.resourcemanager.security.implementation.AlertsImpl; import com.azure.resourcemanager.security.implementation.AlertsSuppressionRulesImpl; import com.azure.resourcemanager.security.implementation.AllowedConnectionsImpl; -import com.azure.resourcemanager.security.implementation.ApiCollectionOffboardingsImpl; -import com.azure.resourcemanager.security.implementation.ApiCollectionOnboardingsImpl; import com.azure.resourcemanager.security.implementation.ApiCollectionsImpl; import com.azure.resourcemanager.security.implementation.ApplicationOperationsImpl; import com.azure.resourcemanager.security.implementation.ApplicationsImpl; @@ -39,20 +37,29 @@ import com.azure.resourcemanager.security.implementation.AssessmentsMetadatasImpl; import com.azure.resourcemanager.security.implementation.AutoProvisioningSettingsImpl; import com.azure.resourcemanager.security.implementation.AutomationsImpl; +import com.azure.resourcemanager.security.implementation.AzureDevOpsOrgsImpl; +import com.azure.resourcemanager.security.implementation.AzureDevOpsProjectsImpl; +import com.azure.resourcemanager.security.implementation.AzureDevOpsReposImpl; import com.azure.resourcemanager.security.implementation.ComplianceResultsImpl; import com.azure.resourcemanager.security.implementation.CompliancesImpl; import com.azure.resourcemanager.security.implementation.ConnectorsImpl; import com.azure.resourcemanager.security.implementation.CustomAssessmentAutomationsImpl; import com.azure.resourcemanager.security.implementation.CustomEntityStoreAssignmentsImpl; +import com.azure.resourcemanager.security.implementation.DefenderForStoragesImpl; +import com.azure.resourcemanager.security.implementation.DevOpsConfigurationsImpl; +import com.azure.resourcemanager.security.implementation.DevOpsOperationResultsImpl; import com.azure.resourcemanager.security.implementation.DeviceSecurityGroupsImpl; import com.azure.resourcemanager.security.implementation.DiscoveredSecuritySolutionsImpl; import com.azure.resourcemanager.security.implementation.ExternalSecuritySolutionsImpl; +import com.azure.resourcemanager.security.implementation.GitHubOwnersImpl; +import com.azure.resourcemanager.security.implementation.GitHubReposImpl; +import com.azure.resourcemanager.security.implementation.GitLabGroupsImpl; +import com.azure.resourcemanager.security.implementation.GitLabProjectsImpl; +import com.azure.resourcemanager.security.implementation.GitLabSubgroupsImpl; import com.azure.resourcemanager.security.implementation.GovernanceAssignmentsImpl; import com.azure.resourcemanager.security.implementation.GovernanceRulesImpl; -import com.azure.resourcemanager.security.implementation.HealthReportOperationsImpl; import com.azure.resourcemanager.security.implementation.HealthReportsImpl; import com.azure.resourcemanager.security.implementation.InformationProtectionPoliciesImpl; -import com.azure.resourcemanager.security.implementation.IngestionSettingsImpl; import com.azure.resourcemanager.security.implementation.IotSecuritySolutionAnalyticsImpl; import com.azure.resourcemanager.security.implementation.IotSecuritySolutionsAnalyticsAggregatedAlertsImpl; import com.azure.resourcemanager.security.implementation.IotSecuritySolutionsAnalyticsRecommendationsImpl; @@ -65,6 +72,7 @@ import com.azure.resourcemanager.security.implementation.RegulatoryComplianceAssessmentsImpl; import com.azure.resourcemanager.security.implementation.RegulatoryComplianceControlsImpl; import com.azure.resourcemanager.security.implementation.RegulatoryComplianceStandardsImpl; +import com.azure.resourcemanager.security.implementation.ResourceProvidersImpl; import com.azure.resourcemanager.security.implementation.SecureScoreControlDefinitionsImpl; import com.azure.resourcemanager.security.implementation.SecureScoreControlsImpl; import com.azure.resourcemanager.security.implementation.SecureScoresImpl; @@ -76,7 +84,9 @@ import com.azure.resourcemanager.security.implementation.SecurityOperatorsImpl; import com.azure.resourcemanager.security.implementation.SecuritySolutionsImpl; import com.azure.resourcemanager.security.implementation.SecuritySolutionsReferenceDatasImpl; +import com.azure.resourcemanager.security.implementation.SensitivitySettingsImpl; import com.azure.resourcemanager.security.implementation.ServerVulnerabilityAssessmentsImpl; +import com.azure.resourcemanager.security.implementation.ServerVulnerabilityAssessmentsSettingsImpl; import com.azure.resourcemanager.security.implementation.SettingsImpl; import com.azure.resourcemanager.security.implementation.SoftwareInventoriesImpl; import com.azure.resourcemanager.security.implementation.SqlVulnerabilityAssessmentBaselineRulesImpl; @@ -92,8 +102,6 @@ import com.azure.resourcemanager.security.models.Alerts; import com.azure.resourcemanager.security.models.AlertsSuppressionRules; import com.azure.resourcemanager.security.models.AllowedConnections; -import com.azure.resourcemanager.security.models.ApiCollectionOffboardings; -import com.azure.resourcemanager.security.models.ApiCollectionOnboardings; import com.azure.resourcemanager.security.models.ApiCollections; import com.azure.resourcemanager.security.models.ApplicationOperations; import com.azure.resourcemanager.security.models.Applications; @@ -101,20 +109,29 @@ import com.azure.resourcemanager.security.models.AssessmentsMetadatas; import com.azure.resourcemanager.security.models.AutoProvisioningSettings; import com.azure.resourcemanager.security.models.Automations; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgs; +import com.azure.resourcemanager.security.models.AzureDevOpsProjects; +import com.azure.resourcemanager.security.models.AzureDevOpsRepos; import com.azure.resourcemanager.security.models.ComplianceResults; import com.azure.resourcemanager.security.models.Compliances; import com.azure.resourcemanager.security.models.Connectors; import com.azure.resourcemanager.security.models.CustomAssessmentAutomations; import com.azure.resourcemanager.security.models.CustomEntityStoreAssignments; +import com.azure.resourcemanager.security.models.DefenderForStorages; +import com.azure.resourcemanager.security.models.DevOpsConfigurations; +import com.azure.resourcemanager.security.models.DevOpsOperationResults; import com.azure.resourcemanager.security.models.DeviceSecurityGroups; import com.azure.resourcemanager.security.models.DiscoveredSecuritySolutions; import com.azure.resourcemanager.security.models.ExternalSecuritySolutions; +import com.azure.resourcemanager.security.models.GitHubOwners; +import com.azure.resourcemanager.security.models.GitHubRepos; +import com.azure.resourcemanager.security.models.GitLabGroups; +import com.azure.resourcemanager.security.models.GitLabProjects; +import com.azure.resourcemanager.security.models.GitLabSubgroups; import com.azure.resourcemanager.security.models.GovernanceAssignments; import com.azure.resourcemanager.security.models.GovernanceRules; -import com.azure.resourcemanager.security.models.HealthReportOperations; import com.azure.resourcemanager.security.models.HealthReports; import com.azure.resourcemanager.security.models.InformationProtectionPolicies; -import com.azure.resourcemanager.security.models.IngestionSettings; import com.azure.resourcemanager.security.models.IotSecuritySolutionAnalytics; import com.azure.resourcemanager.security.models.IotSecuritySolutions; import com.azure.resourcemanager.security.models.IotSecuritySolutionsAnalyticsAggregatedAlerts; @@ -127,6 +144,7 @@ import com.azure.resourcemanager.security.models.RegulatoryComplianceAssessments; import com.azure.resourcemanager.security.models.RegulatoryComplianceControls; import com.azure.resourcemanager.security.models.RegulatoryComplianceStandards; +import com.azure.resourcemanager.security.models.ResourceProviders; import com.azure.resourcemanager.security.models.SecureScoreControlDefinitions; import com.azure.resourcemanager.security.models.SecureScoreControls; import com.azure.resourcemanager.security.models.SecureScores; @@ -137,7 +155,9 @@ import com.azure.resourcemanager.security.models.SecurityOperators; import com.azure.resourcemanager.security.models.SecuritySolutions; import com.azure.resourcemanager.security.models.SecuritySolutionsReferenceDatas; +import com.azure.resourcemanager.security.models.SensitivitySettings; import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessments; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettings; import com.azure.resourcemanager.security.models.Settings; import com.azure.resourcemanager.security.models.SoftwareInventories; import com.azure.resourcemanager.security.models.SqlVulnerabilityAssessmentBaselineRules; @@ -154,147 +174,166 @@ import java.util.Objects; import java.util.stream.Collectors; -/** Entry point to SecurityManager. API spec for Microsoft.Security (Azure Security Center) resource provider. */ +/** + * Entry point to SecurityManager. + * API spec for Microsoft.Security (Azure Security Center) resource provider. + */ public final class SecurityManager { - private MdeOnboardings mdeOnboardings; + private Locations locations; - private CustomAssessmentAutomations customAssessmentAutomations; + private Operations operations; - private CustomEntityStoreAssignments customEntityStoreAssignments; + private Tasks tasks; - private ComplianceResults complianceResults; + private AutoProvisioningSettings autoProvisioningSettings; - private Pricings pricings; + private Compliances compliances; - private AdvancedThreatProtections advancedThreatProtections; + private InformationProtectionPolicies informationProtectionPolicies; - private DeviceSecurityGroups deviceSecurityGroups; + private WorkspaceSettings workspaceSettings; - private IotSecuritySolutions iotSecuritySolutions; + private AlertsSuppressionRules alertsSuppressionRules; - private IotSecuritySolutionAnalytics iotSecuritySolutionAnalytics; + private Automations automations; - private IotSecuritySolutionsAnalyticsAggregatedAlerts iotSecuritySolutionsAnalyticsAggregatedAlerts; + private RegulatoryComplianceStandards regulatoryComplianceStandards; - private IotSecuritySolutionsAnalyticsRecommendations iotSecuritySolutionsAnalyticsRecommendations; + private RegulatoryComplianceControls regulatoryComplianceControls; - private Locations locations; + private RegulatoryComplianceAssessments regulatoryComplianceAssessments; - private Operations operations; + private SubAssessments subAssessments; - private Tasks tasks; + private Connectors connectors; - private AutoProvisioningSettings autoProvisioningSettings; + private SecurityContacts securityContacts; - private Compliances compliances; + private SoftwareInventories softwareInventories; - private InformationProtectionPolicies informationProtectionPolicies; + private CustomAssessmentAutomations customAssessmentAutomations; - private SecurityContacts securityContacts; + private CustomEntityStoreAssignments customEntityStoreAssignments; - private WorkspaceSettings workspaceSettings; + private MdeOnboardings mdeOnboardings; - private RegulatoryComplianceStandards regulatoryComplianceStandards; + private GovernanceAssignments governanceAssignments; - private RegulatoryComplianceControls regulatoryComplianceControls; + private GovernanceRules governanceRules; - private RegulatoryComplianceAssessments regulatoryComplianceAssessments; + private Applications applications; - private SubAssessments subAssessments; + private ApplicationOperations applicationOperations; - private Automations automations; + private SecurityConnectorApplications securityConnectorApplications; - private AlertsSuppressionRules alertsSuppressionRules; + private SecurityConnectorApplicationOperations securityConnectorApplicationOperations; - private ServerVulnerabilityAssessments serverVulnerabilityAssessments; + private DefenderForStorages defenderForStorages; - private AssessmentsMetadatas assessmentsMetadatas; + private SecurityOperators securityOperators; - private Assessments assessments; + private SqlVulnerabilityAssessmentBaselineRules sqlVulnerabilityAssessmentBaselineRules; - private AdaptiveApplicationControls adaptiveApplicationControls; + private SqlVulnerabilityAssessmentScans sqlVulnerabilityAssessmentScans; - private AdaptiveNetworkHardenings adaptiveNetworkHardenings; + private SqlVulnerabilityAssessmentScanResults sqlVulnerabilityAssessmentScanResults; - private AllowedConnections allowedConnections; + private ResourceProviders resourceProviders; - private Topologies topologies; + private SensitivitySettings sensitivitySettings; - private JitNetworkAccessPolicies jitNetworkAccessPolicies; + private HealthReports healthReports; - private DiscoveredSecuritySolutions discoveredSecuritySolutions; + private AzureDevOpsOrgs azureDevOpsOrgs; - private SecuritySolutionsReferenceDatas securitySolutionsReferenceDatas; + private AzureDevOpsProjects azureDevOpsProjects; - private ExternalSecuritySolutions externalSecuritySolutions; + private AzureDevOpsRepos azureDevOpsRepos; - private SecureScores secureScores; + private DevOpsConfigurations devOpsConfigurations; - private SecureScoreControls secureScoreControls; + private GitHubOwners gitHubOwners; - private SecureScoreControlDefinitions secureScoreControlDefinitions; + private GitHubRepos gitHubRepos; - private SecuritySolutions securitySolutions; + private GitLabGroups gitLabGroups; - private Connectors connectors; + private GitLabSubgroups gitLabSubgroups; - private Alerts alerts; + private GitLabProjects gitLabProjects; - private Settings settings; + private DevOpsOperationResults devOpsOperationResults; - private IngestionSettings ingestionSettings; + private SecurityConnectors securityConnectors; - private SoftwareInventories softwareInventories; + private ComplianceResults complianceResults; - private GovernanceRules governanceRules; + private AdvancedThreatProtections advancedThreatProtections; - private GovernanceAssignments governanceAssignments; + private DeviceSecurityGroups deviceSecurityGroups; - private Applications applications; + private IotSecuritySolutionAnalytics iotSecuritySolutionAnalytics; - private ApplicationOperations applicationOperations; + private IotSecuritySolutionsAnalyticsAggregatedAlerts iotSecuritySolutionsAnalyticsAggregatedAlerts; - private SecurityConnectorApplications securityConnectorApplications; + private IotSecuritySolutionsAnalyticsRecommendations iotSecuritySolutionsAnalyticsRecommendations; - private SecurityConnectorApplicationOperations securityConnectorApplicationOperations; + private IotSecuritySolutions iotSecuritySolutions; - private ApiCollections apiCollections; + private AdaptiveNetworkHardenings adaptiveNetworkHardenings; - private ApiCollectionOnboardings apiCollectionOnboardings; + private AllowedConnections allowedConnections; - private ApiCollectionOffboardings apiCollectionOffboardings; + private AdaptiveApplicationControls adaptiveApplicationControls; - private HealthReports healthReports; + private DiscoveredSecuritySolutions discoveredSecuritySolutions; - private HealthReportOperations healthReportOperations; + private ExternalSecuritySolutions externalSecuritySolutions; - private SqlVulnerabilityAssessmentScans sqlVulnerabilityAssessmentScans; + private JitNetworkAccessPolicies jitNetworkAccessPolicies; - private SqlVulnerabilityAssessmentScanResults sqlVulnerabilityAssessmentScanResults; + private SecureScores secureScores; - private SqlVulnerabilityAssessmentBaselineRules sqlVulnerabilityAssessmentBaselineRules; + private SecureScoreControls secureScoreControls; - private SecurityConnectors securityConnectors; + private SecureScoreControlDefinitions secureScoreControlDefinitions; - private SecurityOperators securityOperators; + private SecuritySolutions securitySolutions; + + private SecuritySolutionsReferenceDatas securitySolutionsReferenceDatas; + + private ServerVulnerabilityAssessments serverVulnerabilityAssessments; + + private Topologies topologies; + + private AssessmentsMetadatas assessmentsMetadatas; + + private Assessments assessments; + + private Alerts alerts; + + private Settings settings; + + private ServerVulnerabilityAssessmentsSettings serverVulnerabilityAssessmentsSettings; + + private ApiCollections apiCollections; + + private Pricings pricings; private final SecurityCenter clientObject; private SecurityManager(HttpPipeline httpPipeline, AzureProfile profile, Duration defaultPollInterval) { Objects.requireNonNull(httpPipeline, "'httpPipeline' cannot be null."); Objects.requireNonNull(profile, "'profile' cannot be null."); - this.clientObject = - new SecurityCenterBuilder() - .pipeline(httpPipeline) - .endpoint(profile.getEnvironment().getResourceManagerEndpoint()) - .subscriptionId(profile.getSubscriptionId()) - .defaultPollInterval(defaultPollInterval) - .buildClient(); + this.clientObject = new SecurityCenterBuilder().pipeline(httpPipeline) + .endpoint(profile.getEnvironment().getResourceManagerEndpoint()).subscriptionId(profile.getSubscriptionId()) + .defaultPollInterval(defaultPollInterval).buildClient(); } /** * Creates an instance of Security service API entry point. - * + * * @param credential the credential to use. * @param profile the Azure profile for client. * @return the Security service API instance. @@ -307,7 +346,7 @@ public static SecurityManager authenticate(TokenCredential credential, AzureProf /** * Creates an instance of Security service API entry point. - * + * * @param httpPipeline the {@link HttpPipeline} configured with Azure authentication credential. * @param profile the Azure profile for client. * @return the Security service API instance. @@ -320,14 +359,16 @@ public static SecurityManager authenticate(HttpPipeline httpPipeline, AzureProfi /** * Gets a Configurable instance that can be used to create SecurityManager with optional configuration. - * + * * @return the Configurable instance allowing configurations. */ public static Configurable configure() { return new SecurityManager.Configurable(); } - /** The Configurable allowing configurations to be set. */ + /** + * The Configurable allowing configurations to be set. + */ public static final class Configurable { private static final ClientLogger LOGGER = new ClientLogger(Configurable.class); @@ -399,8 +440,8 @@ public Configurable withRetryPolicy(RetryPolicy retryPolicy) { /** * Sets the retry options for the HTTP pipeline retry policy. - * - *

This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. + *

+ * This setting has no effect, if retry policy is set via {@link #withRetryPolicy(RetryPolicy)}. * * @param retryOptions the retry options for the HTTP pipeline retry policy. * @return the configurable object itself. @@ -417,8 +458,8 @@ public Configurable withRetryOptions(RetryOptions retryOptions) { * @return the configurable object itself. */ public Configurable withDefaultPollInterval(Duration defaultPollInterval) { - this.defaultPollInterval = - Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); + this.defaultPollInterval + = Objects.requireNonNull(defaultPollInterval, "'defaultPollInterval' cannot be null."); if (this.defaultPollInterval.isNegative()) { throw LOGGER .logExceptionAsError(new IllegalArgumentException("'defaultPollInterval' cannot be negative")); @@ -438,21 +479,12 @@ public SecurityManager authenticate(TokenCredential credential, AzureProfile pro Objects.requireNonNull(profile, "'profile' cannot be null."); StringBuilder userAgentBuilder = new StringBuilder(); - userAgentBuilder - .append("azsdk-java") - .append("-") - .append("com.azure.resourcemanager.security") - .append("/") - .append("1.0.0-beta.5"); + userAgentBuilder.append("azsdk-java").append("-").append("com.azure.resourcemanager.security").append("/") + .append("1.0.0-beta.6"); if (!Configuration.getGlobalConfiguration().get("AZURE_TELEMETRY_DISABLED", false)) { - userAgentBuilder - .append(" (") - .append(Configuration.getGlobalConfiguration().get("java.version")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.name")) - .append("; ") - .append(Configuration.getGlobalConfiguration().get("os.version")) - .append("; auto-generated)"); + userAgentBuilder.append(" (").append(Configuration.getGlobalConfiguration().get("java.version")) + .append("; ").append(Configuration.getGlobalConfiguration().get("os.name")).append("; ") + .append(Configuration.getGlobalConfiguration().get("os.version")).append("; auto-generated)"); } else { userAgentBuilder.append(" (auto-generated)"); } @@ -471,178 +503,25 @@ public SecurityManager authenticate(TokenCredential credential, AzureProfile pro policies.add(new UserAgentPolicy(userAgentBuilder.toString())); policies.add(new AddHeadersFromContextPolicy()); policies.add(new RequestIdPolicy()); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream().filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_CALL) + .collect(Collectors.toList())); HttpPolicyProviders.addBeforeRetryPolicies(policies); policies.add(retryPolicy); policies.add(new AddDatePolicy()); policies.add(new ArmChallengeAuthenticationPolicy(credential, scopes.toArray(new String[0]))); - policies - .addAll( - this - .policies - .stream() - .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY) - .collect(Collectors.toList())); + policies.addAll(this.policies.stream() + .filter(p -> p.getPipelinePosition() == HttpPipelinePosition.PER_RETRY).collect(Collectors.toList())); HttpPolicyProviders.addAfterRetryPolicies(policies); policies.add(new HttpLoggingPolicy(httpLogOptions)); - HttpPipeline httpPipeline = - new HttpPipelineBuilder() - .httpClient(httpClient) - .policies(policies.toArray(new HttpPipelinePolicy[0])) - .build(); + HttpPipeline httpPipeline = new HttpPipelineBuilder().httpClient(httpClient) + .policies(policies.toArray(new HttpPipelinePolicy[0])).build(); return new SecurityManager(httpPipeline, profile, defaultPollInterval); } } - /** - * Gets the resource collection API of MdeOnboardings. - * - * @return Resource collection API of MdeOnboardings. - */ - public MdeOnboardings mdeOnboardings() { - if (this.mdeOnboardings == null) { - this.mdeOnboardings = new MdeOnboardingsImpl(clientObject.getMdeOnboardings(), this); - } - return mdeOnboardings; - } - - /** - * Gets the resource collection API of CustomAssessmentAutomations. It manages CustomAssessmentAutomation. - * - * @return Resource collection API of CustomAssessmentAutomations. - */ - public CustomAssessmentAutomations customAssessmentAutomations() { - if (this.customAssessmentAutomations == null) { - this.customAssessmentAutomations = - new CustomAssessmentAutomationsImpl(clientObject.getCustomAssessmentAutomations(), this); - } - return customAssessmentAutomations; - } - - /** - * Gets the resource collection API of CustomEntityStoreAssignments. It manages CustomEntityStoreAssignment. - * - * @return Resource collection API of CustomEntityStoreAssignments. - */ - public CustomEntityStoreAssignments customEntityStoreAssignments() { - if (this.customEntityStoreAssignments == null) { - this.customEntityStoreAssignments = - new CustomEntityStoreAssignmentsImpl(clientObject.getCustomEntityStoreAssignments(), this); - } - return customEntityStoreAssignments; - } - - /** - * Gets the resource collection API of ComplianceResults. - * - * @return Resource collection API of ComplianceResults. - */ - public ComplianceResults complianceResults() { - if (this.complianceResults == null) { - this.complianceResults = new ComplianceResultsImpl(clientObject.getComplianceResults(), this); - } - return complianceResults; - } - - /** - * Gets the resource collection API of Pricings. - * - * @return Resource collection API of Pricings. - */ - public Pricings pricings() { - if (this.pricings == null) { - this.pricings = new PricingsImpl(clientObject.getPricings(), this); - } - return pricings; - } - - /** - * Gets the resource collection API of AdvancedThreatProtections. It manages AdvancedThreatProtectionSetting. - * - * @return Resource collection API of AdvancedThreatProtections. - */ - public AdvancedThreatProtections advancedThreatProtections() { - if (this.advancedThreatProtections == null) { - this.advancedThreatProtections = - new AdvancedThreatProtectionsImpl(clientObject.getAdvancedThreatProtections(), this); - } - return advancedThreatProtections; - } - - /** - * Gets the resource collection API of DeviceSecurityGroups. It manages DeviceSecurityGroup. - * - * @return Resource collection API of DeviceSecurityGroups. - */ - public DeviceSecurityGroups deviceSecurityGroups() { - if (this.deviceSecurityGroups == null) { - this.deviceSecurityGroups = new DeviceSecurityGroupsImpl(clientObject.getDeviceSecurityGroups(), this); - } - return deviceSecurityGroups; - } - - /** - * Gets the resource collection API of IotSecuritySolutions. It manages IoTSecuritySolutionModel. - * - * @return Resource collection API of IotSecuritySolutions. - */ - public IotSecuritySolutions iotSecuritySolutions() { - if (this.iotSecuritySolutions == null) { - this.iotSecuritySolutions = new IotSecuritySolutionsImpl(clientObject.getIotSecuritySolutions(), this); - } - return iotSecuritySolutions; - } - - /** - * Gets the resource collection API of IotSecuritySolutionAnalytics. - * - * @return Resource collection API of IotSecuritySolutionAnalytics. - */ - public IotSecuritySolutionAnalytics iotSecuritySolutionAnalytics() { - if (this.iotSecuritySolutionAnalytics == null) { - this.iotSecuritySolutionAnalytics = - new IotSecuritySolutionAnalyticsImpl(clientObject.getIotSecuritySolutionAnalytics(), this); - } - return iotSecuritySolutionAnalytics; - } - - /** - * Gets the resource collection API of IotSecuritySolutionsAnalyticsAggregatedAlerts. - * - * @return Resource collection API of IotSecuritySolutionsAnalyticsAggregatedAlerts. - */ - public IotSecuritySolutionsAnalyticsAggregatedAlerts iotSecuritySolutionsAnalyticsAggregatedAlerts() { - if (this.iotSecuritySolutionsAnalyticsAggregatedAlerts == null) { - this.iotSecuritySolutionsAnalyticsAggregatedAlerts = - new IotSecuritySolutionsAnalyticsAggregatedAlertsImpl( - clientObject.getIotSecuritySolutionsAnalyticsAggregatedAlerts(), this); - } - return iotSecuritySolutionsAnalyticsAggregatedAlerts; - } - - /** - * Gets the resource collection API of IotSecuritySolutionsAnalyticsRecommendations. - * - * @return Resource collection API of IotSecuritySolutionsAnalyticsRecommendations. - */ - public IotSecuritySolutionsAnalyticsRecommendations iotSecuritySolutionsAnalyticsRecommendations() { - if (this.iotSecuritySolutionsAnalyticsRecommendations == null) { - this.iotSecuritySolutionsAnalyticsRecommendations = - new IotSecuritySolutionsAnalyticsRecommendationsImpl( - clientObject.getIotSecuritySolutionsAnalyticsRecommendations(), this); - } - return iotSecuritySolutionsAnalyticsRecommendations; - } - /** * Gets the resource collection API of Locations. - * + * * @return Resource collection API of Locations. */ public Locations locations() { @@ -654,7 +533,7 @@ public Locations locations() { /** * Gets the resource collection API of Operations. - * + * * @return Resource collection API of Operations. */ public Operations operations() { @@ -666,7 +545,7 @@ public Operations operations() { /** * Gets the resource collection API of Tasks. - * + * * @return Resource collection API of Tasks. */ public Tasks tasks() { @@ -678,20 +557,20 @@ public Tasks tasks() { /** * Gets the resource collection API of AutoProvisioningSettings. It manages AutoProvisioningSetting. - * + * * @return Resource collection API of AutoProvisioningSettings. */ public AutoProvisioningSettings autoProvisioningSettings() { if (this.autoProvisioningSettings == null) { - this.autoProvisioningSettings = - new AutoProvisioningSettingsImpl(clientObject.getAutoProvisioningSettings(), this); + this.autoProvisioningSettings + = new AutoProvisioningSettingsImpl(clientObject.getAutoProvisioningSettings(), this); } return autoProvisioningSettings; } /** * Gets the resource collection API of Compliances. - * + * * @return Resource collection API of Compliances. */ public Compliances compliances() { @@ -703,32 +582,20 @@ public Compliances compliances() { /** * Gets the resource collection API of InformationProtectionPolicies. It manages InformationProtectionPolicy. - * + * * @return Resource collection API of InformationProtectionPolicies. */ public InformationProtectionPolicies informationProtectionPolicies() { if (this.informationProtectionPolicies == null) { - this.informationProtectionPolicies = - new InformationProtectionPoliciesImpl(clientObject.getInformationProtectionPolicies(), this); + this.informationProtectionPolicies + = new InformationProtectionPoliciesImpl(clientObject.getInformationProtectionPolicies(), this); } return informationProtectionPolicies; } - /** - * Gets the resource collection API of SecurityContacts. It manages SecurityContact. - * - * @return Resource collection API of SecurityContacts. - */ - public SecurityContacts securityContacts() { - if (this.securityContacts == null) { - this.securityContacts = new SecurityContactsImpl(clientObject.getSecurityContacts(), this); - } - return securityContacts; - } - /** * Gets the resource collection API of WorkspaceSettings. It manages WorkspaceSetting. - * + * * @return Resource collection API of WorkspaceSettings. */ public WorkspaceSettings workspaceSettings() { @@ -739,47 +606,72 @@ public WorkspaceSettings workspaceSettings() { } /** - * Gets the resource collection API of RegulatoryComplianceStandards. - * - * @return Resource collection API of RegulatoryComplianceStandards. - */ + * Gets the resource collection API of AlertsSuppressionRules. + * + * @return Resource collection API of AlertsSuppressionRules. + */ + public AlertsSuppressionRules alertsSuppressionRules() { + if (this.alertsSuppressionRules == null) { + this.alertsSuppressionRules + = new AlertsSuppressionRulesImpl(clientObject.getAlertsSuppressionRules(), this); + } + return alertsSuppressionRules; + } + + /** + * Gets the resource collection API of Automations. It manages Automation. + * + * @return Resource collection API of Automations. + */ + public Automations automations() { + if (this.automations == null) { + this.automations = new AutomationsImpl(clientObject.getAutomations(), this); + } + return automations; + } + + /** + * Gets the resource collection API of RegulatoryComplianceStandards. + * + * @return Resource collection API of RegulatoryComplianceStandards. + */ public RegulatoryComplianceStandards regulatoryComplianceStandards() { if (this.regulatoryComplianceStandards == null) { - this.regulatoryComplianceStandards = - new RegulatoryComplianceStandardsImpl(clientObject.getRegulatoryComplianceStandards(), this); + this.regulatoryComplianceStandards + = new RegulatoryComplianceStandardsImpl(clientObject.getRegulatoryComplianceStandards(), this); } return regulatoryComplianceStandards; } /** * Gets the resource collection API of RegulatoryComplianceControls. - * + * * @return Resource collection API of RegulatoryComplianceControls. */ public RegulatoryComplianceControls regulatoryComplianceControls() { if (this.regulatoryComplianceControls == null) { - this.regulatoryComplianceControls = - new RegulatoryComplianceControlsImpl(clientObject.getRegulatoryComplianceControls(), this); + this.regulatoryComplianceControls + = new RegulatoryComplianceControlsImpl(clientObject.getRegulatoryComplianceControls(), this); } return regulatoryComplianceControls; } /** * Gets the resource collection API of RegulatoryComplianceAssessments. - * + * * @return Resource collection API of RegulatoryComplianceAssessments. */ public RegulatoryComplianceAssessments regulatoryComplianceAssessments() { if (this.regulatoryComplianceAssessments == null) { - this.regulatoryComplianceAssessments = - new RegulatoryComplianceAssessmentsImpl(clientObject.getRegulatoryComplianceAssessments(), this); + this.regulatoryComplianceAssessments + = new RegulatoryComplianceAssessmentsImpl(clientObject.getRegulatoryComplianceAssessments(), this); } return regulatoryComplianceAssessments; } /** * Gets the resource collection API of SubAssessments. - * + * * @return Resource collection API of SubAssessments. */ public SubAssessments subAssessments() { @@ -790,484 +682,727 @@ public SubAssessments subAssessments() { } /** - * Gets the resource collection API of Automations. It manages Automation. - * - * @return Resource collection API of Automations. + * Gets the resource collection API of Connectors. It manages ConnectorSetting. + * + * @return Resource collection API of Connectors. */ - public Automations automations() { - if (this.automations == null) { - this.automations = new AutomationsImpl(clientObject.getAutomations(), this); + public Connectors connectors() { + if (this.connectors == null) { + this.connectors = new ConnectorsImpl(clientObject.getConnectors(), this); } - return automations; + return connectors; } /** - * Gets the resource collection API of AlertsSuppressionRules. - * - * @return Resource collection API of AlertsSuppressionRules. + * Gets the resource collection API of SecurityContacts. It manages SecurityContact. + * + * @return Resource collection API of SecurityContacts. */ - public AlertsSuppressionRules alertsSuppressionRules() { - if (this.alertsSuppressionRules == null) { - this.alertsSuppressionRules = - new AlertsSuppressionRulesImpl(clientObject.getAlertsSuppressionRules(), this); + public SecurityContacts securityContacts() { + if (this.securityContacts == null) { + this.securityContacts = new SecurityContactsImpl(clientObject.getSecurityContacts(), this); } - return alertsSuppressionRules; + return securityContacts; } /** - * Gets the resource collection API of ServerVulnerabilityAssessments. - * - * @return Resource collection API of ServerVulnerabilityAssessments. + * Gets the resource collection API of SoftwareInventories. + * + * @return Resource collection API of SoftwareInventories. */ - public ServerVulnerabilityAssessments serverVulnerabilityAssessments() { - if (this.serverVulnerabilityAssessments == null) { - this.serverVulnerabilityAssessments = - new ServerVulnerabilityAssessmentsImpl(clientObject.getServerVulnerabilityAssessments(), this); + public SoftwareInventories softwareInventories() { + if (this.softwareInventories == null) { + this.softwareInventories = new SoftwareInventoriesImpl(clientObject.getSoftwareInventories(), this); } - return serverVulnerabilityAssessments; + return softwareInventories; } /** - * Gets the resource collection API of AssessmentsMetadatas. It manages SecurityAssessmentMetadataResponse. - * - * @return Resource collection API of AssessmentsMetadatas. + * Gets the resource collection API of CustomAssessmentAutomations. It manages CustomAssessmentAutomation. + * + * @return Resource collection API of CustomAssessmentAutomations. */ - public AssessmentsMetadatas assessmentsMetadatas() { - if (this.assessmentsMetadatas == null) { - this.assessmentsMetadatas = new AssessmentsMetadatasImpl(clientObject.getAssessmentsMetadatas(), this); + public CustomAssessmentAutomations customAssessmentAutomations() { + if (this.customAssessmentAutomations == null) { + this.customAssessmentAutomations + = new CustomAssessmentAutomationsImpl(clientObject.getCustomAssessmentAutomations(), this); } - return assessmentsMetadatas; + return customAssessmentAutomations; } /** - * Gets the resource collection API of Assessments. It manages SecurityAssessmentResponse. - * - * @return Resource collection API of Assessments. + * Gets the resource collection API of CustomEntityStoreAssignments. It manages CustomEntityStoreAssignment. + * + * @return Resource collection API of CustomEntityStoreAssignments. */ - public Assessments assessments() { - if (this.assessments == null) { - this.assessments = new AssessmentsImpl(clientObject.getAssessments(), this); + public CustomEntityStoreAssignments customEntityStoreAssignments() { + if (this.customEntityStoreAssignments == null) { + this.customEntityStoreAssignments + = new CustomEntityStoreAssignmentsImpl(clientObject.getCustomEntityStoreAssignments(), this); } - return assessments; + return customEntityStoreAssignments; } /** - * Gets the resource collection API of AdaptiveApplicationControls. It manages AdaptiveApplicationControlGroup. - * - * @return Resource collection API of AdaptiveApplicationControls. + * Gets the resource collection API of MdeOnboardings. + * + * @return Resource collection API of MdeOnboardings. */ - public AdaptiveApplicationControls adaptiveApplicationControls() { - if (this.adaptiveApplicationControls == null) { - this.adaptiveApplicationControls = - new AdaptiveApplicationControlsImpl(clientObject.getAdaptiveApplicationControls(), this); + public MdeOnboardings mdeOnboardings() { + if (this.mdeOnboardings == null) { + this.mdeOnboardings = new MdeOnboardingsImpl(clientObject.getMdeOnboardings(), this); } - return adaptiveApplicationControls; + return mdeOnboardings; } /** - * Gets the resource collection API of AdaptiveNetworkHardenings. - * - * @return Resource collection API of AdaptiveNetworkHardenings. + * Gets the resource collection API of GovernanceAssignments. It manages GovernanceAssignment. + * + * @return Resource collection API of GovernanceAssignments. */ - public AdaptiveNetworkHardenings adaptiveNetworkHardenings() { - if (this.adaptiveNetworkHardenings == null) { - this.adaptiveNetworkHardenings = - new AdaptiveNetworkHardeningsImpl(clientObject.getAdaptiveNetworkHardenings(), this); + public GovernanceAssignments governanceAssignments() { + if (this.governanceAssignments == null) { + this.governanceAssignments = new GovernanceAssignmentsImpl(clientObject.getGovernanceAssignments(), this); } - return adaptiveNetworkHardenings; + return governanceAssignments; } /** - * Gets the resource collection API of AllowedConnections. - * - * @return Resource collection API of AllowedConnections. + * Gets the resource collection API of GovernanceRules. It manages GovernanceRule. + * + * @return Resource collection API of GovernanceRules. */ - public AllowedConnections allowedConnections() { - if (this.allowedConnections == null) { - this.allowedConnections = new AllowedConnectionsImpl(clientObject.getAllowedConnections(), this); + public GovernanceRules governanceRules() { + if (this.governanceRules == null) { + this.governanceRules = new GovernanceRulesImpl(clientObject.getGovernanceRules(), this); } - return allowedConnections; + return governanceRules; } /** - * Gets the resource collection API of Topologies. - * - * @return Resource collection API of Topologies. + * Gets the resource collection API of Applications. + * + * @return Resource collection API of Applications. */ - public Topologies topologies() { - if (this.topologies == null) { - this.topologies = new TopologiesImpl(clientObject.getTopologies(), this); + public Applications applications() { + if (this.applications == null) { + this.applications = new ApplicationsImpl(clientObject.getApplications(), this); } - return topologies; + return applications; } /** - * Gets the resource collection API of JitNetworkAccessPolicies. It manages JitNetworkAccessPolicy. - * - * @return Resource collection API of JitNetworkAccessPolicies. + * Gets the resource collection API of ApplicationOperations. It manages Application. + * + * @return Resource collection API of ApplicationOperations. */ - public JitNetworkAccessPolicies jitNetworkAccessPolicies() { - if (this.jitNetworkAccessPolicies == null) { - this.jitNetworkAccessPolicies = - new JitNetworkAccessPoliciesImpl(clientObject.getJitNetworkAccessPolicies(), this); + public ApplicationOperations applicationOperations() { + if (this.applicationOperations == null) { + this.applicationOperations = new ApplicationOperationsImpl(clientObject.getApplicationOperations(), this); } - return jitNetworkAccessPolicies; + return applicationOperations; } /** - * Gets the resource collection API of DiscoveredSecuritySolutions. - * - * @return Resource collection API of DiscoveredSecuritySolutions. + * Gets the resource collection API of SecurityConnectorApplications. + * + * @return Resource collection API of SecurityConnectorApplications. */ - public DiscoveredSecuritySolutions discoveredSecuritySolutions() { - if (this.discoveredSecuritySolutions == null) { - this.discoveredSecuritySolutions = - new DiscoveredSecuritySolutionsImpl(clientObject.getDiscoveredSecuritySolutions(), this); + public SecurityConnectorApplications securityConnectorApplications() { + if (this.securityConnectorApplications == null) { + this.securityConnectorApplications + = new SecurityConnectorApplicationsImpl(clientObject.getSecurityConnectorApplications(), this); } - return discoveredSecuritySolutions; + return securityConnectorApplications; } /** - * Gets the resource collection API of SecuritySolutionsReferenceDatas. - * - * @return Resource collection API of SecuritySolutionsReferenceDatas. + * Gets the resource collection API of SecurityConnectorApplicationOperations. + * + * @return Resource collection API of SecurityConnectorApplicationOperations. */ - public SecuritySolutionsReferenceDatas securitySolutionsReferenceDatas() { - if (this.securitySolutionsReferenceDatas == null) { - this.securitySolutionsReferenceDatas = - new SecuritySolutionsReferenceDatasImpl(clientObject.getSecuritySolutionsReferenceDatas(), this); + public SecurityConnectorApplicationOperations securityConnectorApplicationOperations() { + if (this.securityConnectorApplicationOperations == null) { + this.securityConnectorApplicationOperations = new SecurityConnectorApplicationOperationsImpl( + clientObject.getSecurityConnectorApplicationOperations(), this); } - return securitySolutionsReferenceDatas; + return securityConnectorApplicationOperations; } /** - * Gets the resource collection API of ExternalSecuritySolutions. - * - * @return Resource collection API of ExternalSecuritySolutions. + * Gets the resource collection API of DefenderForStorages. It manages DefenderForStorageSetting. + * + * @return Resource collection API of DefenderForStorages. */ - public ExternalSecuritySolutions externalSecuritySolutions() { - if (this.externalSecuritySolutions == null) { - this.externalSecuritySolutions = - new ExternalSecuritySolutionsImpl(clientObject.getExternalSecuritySolutions(), this); + public DefenderForStorages defenderForStorages() { + if (this.defenderForStorages == null) { + this.defenderForStorages = new DefenderForStoragesImpl(clientObject.getDefenderForStorages(), this); } - return externalSecuritySolutions; + return defenderForStorages; } /** - * Gets the resource collection API of SecureScores. - * - * @return Resource collection API of SecureScores. + * Gets the resource collection API of SecurityOperators. + * + * @return Resource collection API of SecurityOperators. */ - public SecureScores secureScores() { - if (this.secureScores == null) { - this.secureScores = new SecureScoresImpl(clientObject.getSecureScores(), this); + public SecurityOperators securityOperators() { + if (this.securityOperators == null) { + this.securityOperators = new SecurityOperatorsImpl(clientObject.getSecurityOperators(), this); } - return secureScores; + return securityOperators; } /** - * Gets the resource collection API of SecureScoreControls. - * - * @return Resource collection API of SecureScoreControls. + * Gets the resource collection API of SqlVulnerabilityAssessmentBaselineRules. It manages RuleResults. + * + * @return Resource collection API of SqlVulnerabilityAssessmentBaselineRules. */ - public SecureScoreControls secureScoreControls() { - if (this.secureScoreControls == null) { - this.secureScoreControls = new SecureScoreControlsImpl(clientObject.getSecureScoreControls(), this); + public SqlVulnerabilityAssessmentBaselineRules sqlVulnerabilityAssessmentBaselineRules() { + if (this.sqlVulnerabilityAssessmentBaselineRules == null) { + this.sqlVulnerabilityAssessmentBaselineRules = new SqlVulnerabilityAssessmentBaselineRulesImpl( + clientObject.getSqlVulnerabilityAssessmentBaselineRules(), this); } - return secureScoreControls; + return sqlVulnerabilityAssessmentBaselineRules; } /** - * Gets the resource collection API of SecureScoreControlDefinitions. - * - * @return Resource collection API of SecureScoreControlDefinitions. + * Gets the resource collection API of SqlVulnerabilityAssessmentScans. + * + * @return Resource collection API of SqlVulnerabilityAssessmentScans. */ - public SecureScoreControlDefinitions secureScoreControlDefinitions() { - if (this.secureScoreControlDefinitions == null) { - this.secureScoreControlDefinitions = - new SecureScoreControlDefinitionsImpl(clientObject.getSecureScoreControlDefinitions(), this); + public SqlVulnerabilityAssessmentScans sqlVulnerabilityAssessmentScans() { + if (this.sqlVulnerabilityAssessmentScans == null) { + this.sqlVulnerabilityAssessmentScans + = new SqlVulnerabilityAssessmentScansImpl(clientObject.getSqlVulnerabilityAssessmentScans(), this); } - return secureScoreControlDefinitions; + return sqlVulnerabilityAssessmentScans; } /** - * Gets the resource collection API of SecuritySolutions. - * - * @return Resource collection API of SecuritySolutions. + * Gets the resource collection API of SqlVulnerabilityAssessmentScanResults. + * + * @return Resource collection API of SqlVulnerabilityAssessmentScanResults. */ - public SecuritySolutions securitySolutions() { - if (this.securitySolutions == null) { - this.securitySolutions = new SecuritySolutionsImpl(clientObject.getSecuritySolutions(), this); + public SqlVulnerabilityAssessmentScanResults sqlVulnerabilityAssessmentScanResults() { + if (this.sqlVulnerabilityAssessmentScanResults == null) { + this.sqlVulnerabilityAssessmentScanResults = new SqlVulnerabilityAssessmentScanResultsImpl( + clientObject.getSqlVulnerabilityAssessmentScanResults(), this); } - return securitySolutions; + return sqlVulnerabilityAssessmentScanResults; } /** - * Gets the resource collection API of Connectors. It manages ConnectorSetting. - * - * @return Resource collection API of Connectors. + * Gets the resource collection API of ResourceProviders. + * + * @return Resource collection API of ResourceProviders. */ - public Connectors connectors() { - if (this.connectors == null) { - this.connectors = new ConnectorsImpl(clientObject.getConnectors(), this); + public ResourceProviders resourceProviders() { + if (this.resourceProviders == null) { + this.resourceProviders = new ResourceProvidersImpl(clientObject.getResourceProviders(), this); } - return connectors; + return resourceProviders; } /** - * Gets the resource collection API of Alerts. - * - * @return Resource collection API of Alerts. + * Gets the resource collection API of SensitivitySettings. + * + * @return Resource collection API of SensitivitySettings. */ - public Alerts alerts() { - if (this.alerts == null) { - this.alerts = new AlertsImpl(clientObject.getAlerts(), this); + public SensitivitySettings sensitivitySettings() { + if (this.sensitivitySettings == null) { + this.sensitivitySettings = new SensitivitySettingsImpl(clientObject.getSensitivitySettings(), this); } - return alerts; + return sensitivitySettings; } /** - * Gets the resource collection API of Settings. - * - * @return Resource collection API of Settings. + * Gets the resource collection API of HealthReports. + * + * @return Resource collection API of HealthReports. */ - public Settings settings() { - if (this.settings == null) { - this.settings = new SettingsImpl(clientObject.getSettings(), this); + public HealthReports healthReports() { + if (this.healthReports == null) { + this.healthReports = new HealthReportsImpl(clientObject.getHealthReports(), this); } - return settings; + return healthReports; } /** - * Gets the resource collection API of IngestionSettings. It manages IngestionSetting. - * - * @return Resource collection API of IngestionSettings. + * Gets the resource collection API of AzureDevOpsOrgs. It manages AzureDevOpsOrg. + * + * @return Resource collection API of AzureDevOpsOrgs. */ - public IngestionSettings ingestionSettings() { - if (this.ingestionSettings == null) { - this.ingestionSettings = new IngestionSettingsImpl(clientObject.getIngestionSettings(), this); + public AzureDevOpsOrgs azureDevOpsOrgs() { + if (this.azureDevOpsOrgs == null) { + this.azureDevOpsOrgs = new AzureDevOpsOrgsImpl(clientObject.getAzureDevOpsOrgs(), this); } - return ingestionSettings; + return azureDevOpsOrgs; } /** - * Gets the resource collection API of SoftwareInventories. - * - * @return Resource collection API of SoftwareInventories. + * Gets the resource collection API of AzureDevOpsProjects. It manages AzureDevOpsProject. + * + * @return Resource collection API of AzureDevOpsProjects. */ - public SoftwareInventories softwareInventories() { - if (this.softwareInventories == null) { - this.softwareInventories = new SoftwareInventoriesImpl(clientObject.getSoftwareInventories(), this); + public AzureDevOpsProjects azureDevOpsProjects() { + if (this.azureDevOpsProjects == null) { + this.azureDevOpsProjects = new AzureDevOpsProjectsImpl(clientObject.getAzureDevOpsProjects(), this); } - return softwareInventories; + return azureDevOpsProjects; } /** - * Gets the resource collection API of GovernanceRules. It manages GovernanceRule. - * - * @return Resource collection API of GovernanceRules. + * Gets the resource collection API of AzureDevOpsRepos. It manages AzureDevOpsRepository. + * + * @return Resource collection API of AzureDevOpsRepos. */ - public GovernanceRules governanceRules() { - if (this.governanceRules == null) { - this.governanceRules = new GovernanceRulesImpl(clientObject.getGovernanceRules(), this); + public AzureDevOpsRepos azureDevOpsRepos() { + if (this.azureDevOpsRepos == null) { + this.azureDevOpsRepos = new AzureDevOpsReposImpl(clientObject.getAzureDevOpsRepos(), this); } - return governanceRules; + return azureDevOpsRepos; } /** - * Gets the resource collection API of GovernanceAssignments. It manages GovernanceAssignment. - * - * @return Resource collection API of GovernanceAssignments. + * Gets the resource collection API of DevOpsConfigurations. + * + * @return Resource collection API of DevOpsConfigurations. */ - public GovernanceAssignments governanceAssignments() { - if (this.governanceAssignments == null) { - this.governanceAssignments = new GovernanceAssignmentsImpl(clientObject.getGovernanceAssignments(), this); + public DevOpsConfigurations devOpsConfigurations() { + if (this.devOpsConfigurations == null) { + this.devOpsConfigurations = new DevOpsConfigurationsImpl(clientObject.getDevOpsConfigurations(), this); } - return governanceAssignments; + return devOpsConfigurations; } /** - * Gets the resource collection API of Applications. - * - * @return Resource collection API of Applications. + * Gets the resource collection API of GitHubOwners. + * + * @return Resource collection API of GitHubOwners. */ - public Applications applications() { - if (this.applications == null) { - this.applications = new ApplicationsImpl(clientObject.getApplications(), this); + public GitHubOwners gitHubOwners() { + if (this.gitHubOwners == null) { + this.gitHubOwners = new GitHubOwnersImpl(clientObject.getGitHubOwners(), this); } - return applications; + return gitHubOwners; } /** - * Gets the resource collection API of ApplicationOperations. It manages Application. - * - * @return Resource collection API of ApplicationOperations. + * Gets the resource collection API of GitHubRepos. + * + * @return Resource collection API of GitHubRepos. */ - public ApplicationOperations applicationOperations() { - if (this.applicationOperations == null) { - this.applicationOperations = new ApplicationOperationsImpl(clientObject.getApplicationOperations(), this); + public GitHubRepos gitHubRepos() { + if (this.gitHubRepos == null) { + this.gitHubRepos = new GitHubReposImpl(clientObject.getGitHubRepos(), this); } - return applicationOperations; + return gitHubRepos; } /** - * Gets the resource collection API of SecurityConnectorApplications. - * - * @return Resource collection API of SecurityConnectorApplications. + * Gets the resource collection API of GitLabGroups. + * + * @return Resource collection API of GitLabGroups. */ - public SecurityConnectorApplications securityConnectorApplications() { - if (this.securityConnectorApplications == null) { - this.securityConnectorApplications = - new SecurityConnectorApplicationsImpl(clientObject.getSecurityConnectorApplications(), this); + public GitLabGroups gitLabGroups() { + if (this.gitLabGroups == null) { + this.gitLabGroups = new GitLabGroupsImpl(clientObject.getGitLabGroups(), this); } - return securityConnectorApplications; + return gitLabGroups; } /** - * Gets the resource collection API of SecurityConnectorApplicationOperations. - * - * @return Resource collection API of SecurityConnectorApplicationOperations. + * Gets the resource collection API of GitLabSubgroups. + * + * @return Resource collection API of GitLabSubgroups. */ - public SecurityConnectorApplicationOperations securityConnectorApplicationOperations() { - if (this.securityConnectorApplicationOperations == null) { - this.securityConnectorApplicationOperations = - new SecurityConnectorApplicationOperationsImpl( - clientObject.getSecurityConnectorApplicationOperations(), this); + public GitLabSubgroups gitLabSubgroups() { + if (this.gitLabSubgroups == null) { + this.gitLabSubgroups = new GitLabSubgroupsImpl(clientObject.getGitLabSubgroups(), this); } - return securityConnectorApplicationOperations; + return gitLabSubgroups; } /** - * Gets the resource collection API of ApiCollections. - * - * @return Resource collection API of ApiCollections. + * Gets the resource collection API of GitLabProjects. + * + * @return Resource collection API of GitLabProjects. */ - public ApiCollections apiCollections() { - if (this.apiCollections == null) { - this.apiCollections = new ApiCollectionsImpl(clientObject.getApiCollections(), this); + public GitLabProjects gitLabProjects() { + if (this.gitLabProjects == null) { + this.gitLabProjects = new GitLabProjectsImpl(clientObject.getGitLabProjects(), this); } - return apiCollections; + return gitLabProjects; } /** - * Gets the resource collection API of ApiCollectionOnboardings. - * - * @return Resource collection API of ApiCollectionOnboardings. + * Gets the resource collection API of DevOpsOperationResults. + * + * @return Resource collection API of DevOpsOperationResults. */ - public ApiCollectionOnboardings apiCollectionOnboardings() { - if (this.apiCollectionOnboardings == null) { - this.apiCollectionOnboardings = - new ApiCollectionOnboardingsImpl(clientObject.getApiCollectionOnboardings(), this); + public DevOpsOperationResults devOpsOperationResults() { + if (this.devOpsOperationResults == null) { + this.devOpsOperationResults + = new DevOpsOperationResultsImpl(clientObject.getDevOpsOperationResults(), this); } - return apiCollectionOnboardings; + return devOpsOperationResults; } /** - * Gets the resource collection API of ApiCollectionOffboardings. - * - * @return Resource collection API of ApiCollectionOffboardings. + * Gets the resource collection API of SecurityConnectors. It manages SecurityConnector. + * + * @return Resource collection API of SecurityConnectors. */ - public ApiCollectionOffboardings apiCollectionOffboardings() { - if (this.apiCollectionOffboardings == null) { - this.apiCollectionOffboardings = - new ApiCollectionOffboardingsImpl(clientObject.getApiCollectionOffboardings(), this); + public SecurityConnectors securityConnectors() { + if (this.securityConnectors == null) { + this.securityConnectors = new SecurityConnectorsImpl(clientObject.getSecurityConnectors(), this); } - return apiCollectionOffboardings; + return securityConnectors; } /** - * Gets the resource collection API of HealthReports. - * - * @return Resource collection API of HealthReports. + * Gets the resource collection API of ComplianceResults. + * + * @return Resource collection API of ComplianceResults. */ - public HealthReports healthReports() { - if (this.healthReports == null) { - this.healthReports = new HealthReportsImpl(clientObject.getHealthReports(), this); + public ComplianceResults complianceResults() { + if (this.complianceResults == null) { + this.complianceResults = new ComplianceResultsImpl(clientObject.getComplianceResults(), this); } - return healthReports; + return complianceResults; } /** - * Gets the resource collection API of HealthReportOperations. - * - * @return Resource collection API of HealthReportOperations. + * Gets the resource collection API of AdvancedThreatProtections. It manages AdvancedThreatProtectionSetting. + * + * @return Resource collection API of AdvancedThreatProtections. */ - public HealthReportOperations healthReportOperations() { - if (this.healthReportOperations == null) { - this.healthReportOperations = - new HealthReportOperationsImpl(clientObject.getHealthReportOperations(), this); + public AdvancedThreatProtections advancedThreatProtections() { + if (this.advancedThreatProtections == null) { + this.advancedThreatProtections + = new AdvancedThreatProtectionsImpl(clientObject.getAdvancedThreatProtections(), this); } - return healthReportOperations; + return advancedThreatProtections; } /** - * Gets the resource collection API of SqlVulnerabilityAssessmentScans. - * - * @return Resource collection API of SqlVulnerabilityAssessmentScans. + * Gets the resource collection API of DeviceSecurityGroups. It manages DeviceSecurityGroup. + * + * @return Resource collection API of DeviceSecurityGroups. */ - public SqlVulnerabilityAssessmentScans sqlVulnerabilityAssessmentScans() { - if (this.sqlVulnerabilityAssessmentScans == null) { - this.sqlVulnerabilityAssessmentScans = - new SqlVulnerabilityAssessmentScansImpl(clientObject.getSqlVulnerabilityAssessmentScans(), this); + public DeviceSecurityGroups deviceSecurityGroups() { + if (this.deviceSecurityGroups == null) { + this.deviceSecurityGroups = new DeviceSecurityGroupsImpl(clientObject.getDeviceSecurityGroups(), this); } - return sqlVulnerabilityAssessmentScans; + return deviceSecurityGroups; } /** - * Gets the resource collection API of SqlVulnerabilityAssessmentScanResults. - * - * @return Resource collection API of SqlVulnerabilityAssessmentScanResults. + * Gets the resource collection API of IotSecuritySolutionAnalytics. + * + * @return Resource collection API of IotSecuritySolutionAnalytics. */ - public SqlVulnerabilityAssessmentScanResults sqlVulnerabilityAssessmentScanResults() { - if (this.sqlVulnerabilityAssessmentScanResults == null) { - this.sqlVulnerabilityAssessmentScanResults = - new SqlVulnerabilityAssessmentScanResultsImpl( - clientObject.getSqlVulnerabilityAssessmentScanResults(), this); + public IotSecuritySolutionAnalytics iotSecuritySolutionAnalytics() { + if (this.iotSecuritySolutionAnalytics == null) { + this.iotSecuritySolutionAnalytics + = new IotSecuritySolutionAnalyticsImpl(clientObject.getIotSecuritySolutionAnalytics(), this); } - return sqlVulnerabilityAssessmentScanResults; + return iotSecuritySolutionAnalytics; } /** - * Gets the resource collection API of SqlVulnerabilityAssessmentBaselineRules. It manages RuleResults. - * - * @return Resource collection API of SqlVulnerabilityAssessmentBaselineRules. + * Gets the resource collection API of IotSecuritySolutionsAnalyticsAggregatedAlerts. + * + * @return Resource collection API of IotSecuritySolutionsAnalyticsAggregatedAlerts. */ - public SqlVulnerabilityAssessmentBaselineRules sqlVulnerabilityAssessmentBaselineRules() { - if (this.sqlVulnerabilityAssessmentBaselineRules == null) { - this.sqlVulnerabilityAssessmentBaselineRules = - new SqlVulnerabilityAssessmentBaselineRulesImpl( - clientObject.getSqlVulnerabilityAssessmentBaselineRules(), this); + public IotSecuritySolutionsAnalyticsAggregatedAlerts iotSecuritySolutionsAnalyticsAggregatedAlerts() { + if (this.iotSecuritySolutionsAnalyticsAggregatedAlerts == null) { + this.iotSecuritySolutionsAnalyticsAggregatedAlerts = new IotSecuritySolutionsAnalyticsAggregatedAlertsImpl( + clientObject.getIotSecuritySolutionsAnalyticsAggregatedAlerts(), this); } - return sqlVulnerabilityAssessmentBaselineRules; + return iotSecuritySolutionsAnalyticsAggregatedAlerts; } /** - * Gets the resource collection API of SecurityConnectors. It manages SecurityConnector. - * - * @return Resource collection API of SecurityConnectors. + * Gets the resource collection API of IotSecuritySolutionsAnalyticsRecommendations. + * + * @return Resource collection API of IotSecuritySolutionsAnalyticsRecommendations. */ - public SecurityConnectors securityConnectors() { - if (this.securityConnectors == null) { - this.securityConnectors = new SecurityConnectorsImpl(clientObject.getSecurityConnectors(), this); + public IotSecuritySolutionsAnalyticsRecommendations iotSecuritySolutionsAnalyticsRecommendations() { + if (this.iotSecuritySolutionsAnalyticsRecommendations == null) { + this.iotSecuritySolutionsAnalyticsRecommendations = new IotSecuritySolutionsAnalyticsRecommendationsImpl( + clientObject.getIotSecuritySolutionsAnalyticsRecommendations(), this); } - return securityConnectors; + return iotSecuritySolutionsAnalyticsRecommendations; } /** - * Gets the resource collection API of SecurityOperators. - * - * @return Resource collection API of SecurityOperators. + * Gets the resource collection API of IotSecuritySolutions. It manages IoTSecuritySolutionModel. + * + * @return Resource collection API of IotSecuritySolutions. */ - public SecurityOperators securityOperators() { - if (this.securityOperators == null) { - this.securityOperators = new SecurityOperatorsImpl(clientObject.getSecurityOperators(), this); + public IotSecuritySolutions iotSecuritySolutions() { + if (this.iotSecuritySolutions == null) { + this.iotSecuritySolutions = new IotSecuritySolutionsImpl(clientObject.getIotSecuritySolutions(), this); } - return securityOperators; + return iotSecuritySolutions; + } + + /** + * Gets the resource collection API of AdaptiveNetworkHardenings. + * + * @return Resource collection API of AdaptiveNetworkHardenings. + */ + public AdaptiveNetworkHardenings adaptiveNetworkHardenings() { + if (this.adaptiveNetworkHardenings == null) { + this.adaptiveNetworkHardenings + = new AdaptiveNetworkHardeningsImpl(clientObject.getAdaptiveNetworkHardenings(), this); + } + return adaptiveNetworkHardenings; + } + + /** + * Gets the resource collection API of AllowedConnections. + * + * @return Resource collection API of AllowedConnections. + */ + public AllowedConnections allowedConnections() { + if (this.allowedConnections == null) { + this.allowedConnections = new AllowedConnectionsImpl(clientObject.getAllowedConnections(), this); + } + return allowedConnections; + } + + /** + * Gets the resource collection API of AdaptiveApplicationControls. It manages AdaptiveApplicationControlGroup. + * + * @return Resource collection API of AdaptiveApplicationControls. + */ + public AdaptiveApplicationControls adaptiveApplicationControls() { + if (this.adaptiveApplicationControls == null) { + this.adaptiveApplicationControls + = new AdaptiveApplicationControlsImpl(clientObject.getAdaptiveApplicationControls(), this); + } + return adaptiveApplicationControls; + } + + /** + * Gets the resource collection API of DiscoveredSecuritySolutions. + * + * @return Resource collection API of DiscoveredSecuritySolutions. + */ + public DiscoveredSecuritySolutions discoveredSecuritySolutions() { + if (this.discoveredSecuritySolutions == null) { + this.discoveredSecuritySolutions + = new DiscoveredSecuritySolutionsImpl(clientObject.getDiscoveredSecuritySolutions(), this); + } + return discoveredSecuritySolutions; + } + + /** + * Gets the resource collection API of ExternalSecuritySolutions. + * + * @return Resource collection API of ExternalSecuritySolutions. + */ + public ExternalSecuritySolutions externalSecuritySolutions() { + if (this.externalSecuritySolutions == null) { + this.externalSecuritySolutions + = new ExternalSecuritySolutionsImpl(clientObject.getExternalSecuritySolutions(), this); + } + return externalSecuritySolutions; + } + + /** + * Gets the resource collection API of JitNetworkAccessPolicies. It manages JitNetworkAccessPolicy. + * + * @return Resource collection API of JitNetworkAccessPolicies. + */ + public JitNetworkAccessPolicies jitNetworkAccessPolicies() { + if (this.jitNetworkAccessPolicies == null) { + this.jitNetworkAccessPolicies + = new JitNetworkAccessPoliciesImpl(clientObject.getJitNetworkAccessPolicies(), this); + } + return jitNetworkAccessPolicies; + } + + /** + * Gets the resource collection API of SecureScores. + * + * @return Resource collection API of SecureScores. + */ + public SecureScores secureScores() { + if (this.secureScores == null) { + this.secureScores = new SecureScoresImpl(clientObject.getSecureScores(), this); + } + return secureScores; + } + + /** + * Gets the resource collection API of SecureScoreControls. + * + * @return Resource collection API of SecureScoreControls. + */ + public SecureScoreControls secureScoreControls() { + if (this.secureScoreControls == null) { + this.secureScoreControls = new SecureScoreControlsImpl(clientObject.getSecureScoreControls(), this); + } + return secureScoreControls; + } + + /** + * Gets the resource collection API of SecureScoreControlDefinitions. + * + * @return Resource collection API of SecureScoreControlDefinitions. + */ + public SecureScoreControlDefinitions secureScoreControlDefinitions() { + if (this.secureScoreControlDefinitions == null) { + this.secureScoreControlDefinitions + = new SecureScoreControlDefinitionsImpl(clientObject.getSecureScoreControlDefinitions(), this); + } + return secureScoreControlDefinitions; + } + + /** + * Gets the resource collection API of SecuritySolutions. + * + * @return Resource collection API of SecuritySolutions. + */ + public SecuritySolutions securitySolutions() { + if (this.securitySolutions == null) { + this.securitySolutions = new SecuritySolutionsImpl(clientObject.getSecuritySolutions(), this); + } + return securitySolutions; + } + + /** + * Gets the resource collection API of SecuritySolutionsReferenceDatas. + * + * @return Resource collection API of SecuritySolutionsReferenceDatas. + */ + public SecuritySolutionsReferenceDatas securitySolutionsReferenceDatas() { + if (this.securitySolutionsReferenceDatas == null) { + this.securitySolutionsReferenceDatas + = new SecuritySolutionsReferenceDatasImpl(clientObject.getSecuritySolutionsReferenceDatas(), this); + } + return securitySolutionsReferenceDatas; + } + + /** + * Gets the resource collection API of ServerVulnerabilityAssessments. + * + * @return Resource collection API of ServerVulnerabilityAssessments. + */ + public ServerVulnerabilityAssessments serverVulnerabilityAssessments() { + if (this.serverVulnerabilityAssessments == null) { + this.serverVulnerabilityAssessments + = new ServerVulnerabilityAssessmentsImpl(clientObject.getServerVulnerabilityAssessments(), this); + } + return serverVulnerabilityAssessments; + } + + /** + * Gets the resource collection API of Topologies. + * + * @return Resource collection API of Topologies. + */ + public Topologies topologies() { + if (this.topologies == null) { + this.topologies = new TopologiesImpl(clientObject.getTopologies(), this); + } + return topologies; + } + + /** + * Gets the resource collection API of AssessmentsMetadatas. It manages SecurityAssessmentMetadataResponse. + * + * @return Resource collection API of AssessmentsMetadatas. + */ + public AssessmentsMetadatas assessmentsMetadatas() { + if (this.assessmentsMetadatas == null) { + this.assessmentsMetadatas = new AssessmentsMetadatasImpl(clientObject.getAssessmentsMetadatas(), this); + } + return assessmentsMetadatas; + } + + /** + * Gets the resource collection API of Assessments. It manages SecurityAssessmentResponse. + * + * @return Resource collection API of Assessments. + */ + public Assessments assessments() { + if (this.assessments == null) { + this.assessments = new AssessmentsImpl(clientObject.getAssessments(), this); + } + return assessments; + } + + /** + * Gets the resource collection API of Alerts. + * + * @return Resource collection API of Alerts. + */ + public Alerts alerts() { + if (this.alerts == null) { + this.alerts = new AlertsImpl(clientObject.getAlerts(), this); + } + return alerts; + } + + /** + * Gets the resource collection API of Settings. + * + * @return Resource collection API of Settings. + */ + public Settings settings() { + if (this.settings == null) { + this.settings = new SettingsImpl(clientObject.getSettings(), this); + } + return settings; + } + + /** + * Gets the resource collection API of ServerVulnerabilityAssessmentsSettings. + * + * @return Resource collection API of ServerVulnerabilityAssessmentsSettings. + */ + public ServerVulnerabilityAssessmentsSettings serverVulnerabilityAssessmentsSettings() { + if (this.serverVulnerabilityAssessmentsSettings == null) { + this.serverVulnerabilityAssessmentsSettings = new ServerVulnerabilityAssessmentsSettingsImpl( + clientObject.getServerVulnerabilityAssessmentsSettings(), this); + } + return serverVulnerabilityAssessmentsSettings; + } + + /** + * Gets the resource collection API of ApiCollections. + * + * @return Resource collection API of ApiCollections. + */ + public ApiCollections apiCollections() { + if (this.apiCollections == null) { + this.apiCollections = new ApiCollectionsImpl(clientObject.getApiCollections(), this); + } + return apiCollections; + } + + /** + * Gets the resource collection API of Pricings. + * + * @return Resource collection API of Pricings. + */ + public Pricings pricings() { + if (this.pricings == null) { + this.pricings = new PricingsImpl(clientObject.getPricings(), this); + } + return pricings; } /** - * @return Wrapped service client SecurityCenter providing direct access to the underlying auto-generated API - * implementation, based on Azure REST API. + * Gets wrapped service client SecurityCenter providing direct access to the underlying auto-generated API + * implementation, based on Azure REST API. + * + * @return Wrapped service client SecurityCenter. */ public SecurityCenter serviceClient() { return this.clientObject; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveApplicationControlsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveApplicationControlsClient.java index 26e579af6d381..413920bdc12c1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveApplicationControlsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveApplicationControlsClient.java @@ -11,11 +11,13 @@ import com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupInner; import com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupsInner; -/** An instance of this class provides access to all the operations defined in AdaptiveApplicationControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in AdaptiveApplicationControlsClient. + */ public interface AdaptiveApplicationControlsClient { /** * Gets a list of application control machine groups for the subscription. - * + * * @param includePathRecommendations Include the policy rules. * @param summary Return output in a summarized form. * @param context The context to associate with this operation. @@ -25,12 +27,12 @@ public interface AdaptiveApplicationControlsClient { * @return a list of application control machine groups for the subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listWithResponse( - Boolean includePathRecommendations, Boolean summary, Context context); + Response listWithResponse(Boolean includePathRecommendations, + Boolean summary, Context context); /** * Gets a list of application control machine groups for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of application control machine groups for the subscription. @@ -40,9 +42,9 @@ Response listWithResponse( /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -51,14 +53,14 @@ Response listWithResponse( * @return an application control VM/server group along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String ascLocation, String groupName, Context context); + Response getWithResponse(String ascLocation, String groupName, + Context context); /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -70,9 +72,9 @@ Response getWithResponse( /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @param context The context to associate with this operation. @@ -82,14 +84,14 @@ Response getWithResponse( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response putWithResponse( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body, Context context); + Response putWithResponse(String ascLocation, String groupName, + AdaptiveApplicationControlGroupInner body, Context context); /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -98,14 +100,14 @@ Response putWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - AdaptiveApplicationControlGroupInner put( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body); + AdaptiveApplicationControlGroupInner put(String ascLocation, String groupName, + AdaptiveApplicationControlGroupInner body); /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -118,9 +120,9 @@ AdaptiveApplicationControlGroupInner put( /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveNetworkHardeningsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveNetworkHardeningsClient.java index 9a39539b447e7..a9d60aff4c1cc 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveNetworkHardeningsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdaptiveNetworkHardeningsClient.java @@ -14,13 +14,15 @@ import com.azure.resourcemanager.security.fluent.models.AdaptiveNetworkHardeningInner; import com.azure.resourcemanager.security.models.AdaptiveNetworkHardeningEnforceRequest; -/** An instance of this class provides access to all the operations defined in AdaptiveNetworkHardeningsClient. */ +/** + * An instance of this class provides access to all the operations defined in AdaptiveNetworkHardeningsClient. + */ public interface AdaptiveNetworkHardeningsClient { /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -28,17 +30,17 @@ public interface AdaptiveNetworkHardeningsClient { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + PagedIterable listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName); /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -47,17 +49,17 @@ PagedIterable listByExtendedResource( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + PagedIterable listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context); /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -69,19 +71,14 @@ PagedIterable listByExtendedResource( * @return a single Adaptive Network Hardening resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - Context context); + Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, Context context); /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -92,18 +89,14 @@ Response getWithResponse( * @return a single Adaptive Network Hardening resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - AdaptiveNetworkHardeningInner get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName); + AdaptiveNetworkHardeningInner get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String adaptiveNetworkHardeningResourceName); /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -115,19 +108,15 @@ AdaptiveNetworkHardeningInner get( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginEnforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, + SyncPoller, Void> beginEnforce(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body); /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -140,20 +129,15 @@ SyncPoller, Void> beginEnforce( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginEnforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context); + SyncPoller, Void> beginEnforce(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, + AdaptiveNetworkHardeningEnforceRequest body, Context context); /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -164,19 +148,14 @@ SyncPoller, Void> beginEnforce( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body); + void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body); /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -188,12 +167,6 @@ void enforce( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context); + void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body, Context context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdvancedThreatProtectionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdvancedThreatProtectionsClient.java index df0146bcab8f7..923759f73f94d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdvancedThreatProtectionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AdvancedThreatProtectionsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AdvancedThreatProtectionSettingInner; -/** An instance of this class provides access to all the operations defined in AdvancedThreatProtectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in AdvancedThreatProtectionsClient. + */ public interface AdvancedThreatProtectionsClient { /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -27,7 +29,7 @@ public interface AdvancedThreatProtectionsClient { /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -39,7 +41,7 @@ public interface AdvancedThreatProtectionsClient { /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @param context The context to associate with this operation. @@ -49,12 +51,12 @@ public interface AdvancedThreatProtectionsClient { * @return the Advanced Threat Protection resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, Context context); + Response createWithResponse(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, Context context); /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -63,6 +65,6 @@ Response createWithResponse( * @return the Advanced Threat Protection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - AdvancedThreatProtectionSettingInner create( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting); + AdvancedThreatProtectionSettingInner create(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsClient.java index 6a8b01d63a827..198c8dcd1e902 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsClient.java @@ -14,11 +14,13 @@ import com.azure.resourcemanager.security.fluent.models.AlertInner; import com.azure.resourcemanager.security.models.AlertSimulatorRequestBody; -/** An instance of this class provides access to all the operations defined in AlertsClient. */ +/** + * An instance of this class provides access to all the operations defined in AlertsClient. + */ public interface AlertsClient { /** * List all the alerts that are associated with the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security alerts as paginated response with {@link PagedIterable}. @@ -28,7 +30,7 @@ public interface AlertsClient { /** * List all the alerts that are associated with the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -40,9 +42,9 @@ public interface AlertsClient { /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -53,9 +55,9 @@ public interface AlertsClient { /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -67,9 +69,9 @@ public interface AlertsClient { /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -80,9 +82,9 @@ public interface AlertsClient { /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -94,11 +96,11 @@ public interface AlertsClient { /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -109,11 +111,11 @@ public interface AlertsClient { /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -121,14 +123,14 @@ public interface AlertsClient { * @return list of security alerts as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listResourceGroupLevelByRegion( - String ascLocation, String resourceGroupName, Context context); + PagedIterable listResourceGroupLevelByRegion(String ascLocation, String resourceGroupName, + Context context); /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -141,9 +143,9 @@ PagedIterable listResourceGroupLevelByRegion( /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -155,30 +157,30 @@ PagedIterable listResourceGroupLevelByRegion( /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an alert that is associated a resource group or a resource in a resource group along with {@link - * Response}. + * @return an alert that is associated a resource group or a resource in a resource group along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getResourceGroupLevelWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response getResourceGroupLevelWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -190,9 +192,9 @@ Response getResourceGroupLevelWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -201,14 +203,14 @@ Response getResourceGroupLevelWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateSubscriptionLevelStateToDismissWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToDismissWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -219,9 +221,9 @@ Response updateSubscriptionLevelStateToDismissWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -230,14 +232,14 @@ Response updateSubscriptionLevelStateToDismissWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateSubscriptionLevelStateToResolveWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToResolveWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -248,9 +250,9 @@ Response updateSubscriptionLevelStateToResolveWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -259,14 +261,14 @@ Response updateSubscriptionLevelStateToResolveWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateSubscriptionLevelStateToActivateWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToActivateWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -277,9 +279,9 @@ Response updateSubscriptionLevelStateToActivateWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -288,14 +290,14 @@ Response updateSubscriptionLevelStateToActivateWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateSubscriptionLevelStateToInProgressWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToInProgressWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -306,11 +308,11 @@ Response updateSubscriptionLevelStateToInProgressWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -319,16 +321,16 @@ Response updateSubscriptionLevelStateToInProgressWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateResourceGroupLevelStateToResolveWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToResolveWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -339,11 +341,11 @@ Response updateResourceGroupLevelStateToResolveWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -352,16 +354,16 @@ Response updateResourceGroupLevelStateToResolveWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateResourceGroupLevelStateToDismissWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToDismissWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -372,11 +374,11 @@ Response updateResourceGroupLevelStateToDismissWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -385,16 +387,16 @@ Response updateResourceGroupLevelStateToDismissWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateResourceGroupLevelStateToActivateWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToActivateWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -405,11 +407,11 @@ Response updateResourceGroupLevelStateToActivateWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -418,16 +420,16 @@ Response updateResourceGroupLevelStateToActivateWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateResourceGroupLevelStateToInProgressWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToInProgressWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -438,9 +440,9 @@ Response updateResourceGroupLevelStateToInProgressWithResponse( /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -448,14 +450,14 @@ Response updateResourceGroupLevelStateToInProgressWithResponse( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginSimulate( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody); + SyncPoller, Void> beginSimulate(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody); /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -464,14 +466,14 @@ SyncPoller, Void> beginSimulate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginSimulate( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody, Context context); + SyncPoller, Void> beginSimulate(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody, Context context); /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -482,9 +484,9 @@ SyncPoller, Void> beginSimulate( /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsSuppressionRulesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsSuppressionRulesClient.java index 64f7db01f3cee..4536de8753de4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsSuppressionRulesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AlertsSuppressionRulesClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AlertsSuppressionRuleInner; -/** An instance of this class provides access to all the operations defined in AlertsSuppressionRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in AlertsSuppressionRulesClient. + */ public interface AlertsSuppressionRulesClient { /** * List of all the dismiss rules for the given subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription as paginated response with {@link PagedIterable}. @@ -25,7 +27,7 @@ public interface AlertsSuppressionRulesClient { /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -38,21 +40,21 @@ public interface AlertsSuppressionRulesClient { /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with {@link - * Response}. + * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response getWithResponse(String alertsSuppressionRuleName, Context context); /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -64,7 +66,7 @@ public interface AlertsSuppressionRulesClient { /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @param context The context to associate with this operation. @@ -74,12 +76,12 @@ public interface AlertsSuppressionRulesClient { * @return describes the suppression rule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule, Context context); + Response updateWithResponse(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule, Context context); /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -88,12 +90,12 @@ Response updateWithResponse( * @return describes the suppression rule. */ @ServiceMethod(returns = ReturnType.SINGLE) - AlertsSuppressionRuleInner update( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule); + AlertsSuppressionRuleInner update(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule); /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -106,7 +108,7 @@ AlertsSuppressionRuleInner update( /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AllowedConnectionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AllowedConnectionsClient.java index 021776947464f..2f0fa1c92398f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AllowedConnectionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AllowedConnectionsClient.java @@ -12,57 +12,59 @@ import com.azure.resourcemanager.security.fluent.models.AllowedConnectionsResourceInner; import com.azure.resourcemanager.security.models.ConnectionType; -/** An instance of this class provides access to all the operations defined in AllowedConnectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in AllowedConnectionsClient. + */ public interface AllowedConnectionsClient { /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedIterable}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedIterable}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedIterable}. + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation); /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedIterable}. + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation, Context context); @@ -70,37 +72,37 @@ public interface AllowedConnectionsClient { /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type along with {@link Response}. + * type along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String ascLocation, ConnectionType connectionType, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + ConnectionType connectionType, Context context); /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type. + * type. */ @ServiceMethod(returns = ReturnType.SINGLE) AllowedConnectionsResourceInner get(String resourceGroupName, String ascLocation, ConnectionType connectionType); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOffboardingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOffboardingsClient.java deleted file mode 100644 index 01617030ae54e..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOffboardingsClient.java +++ /dev/null @@ -1,50 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; - -/** An instance of this class provides access to all the operations defined in ApiCollectionOffboardingsClient. */ -public interface ApiCollectionOffboardingsClient { - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context); - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String resourceGroupName, String serviceName, String apiCollectionId); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOnboardingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOnboardingsClient.java deleted file mode 100644 index 2dd341d3c7e1f..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionOnboardingsClient.java +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; - -/** An instance of this class provides access to all the operations defined in ApiCollectionOnboardingsClient. */ -public interface ApiCollectionOnboardingsClient { - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context); - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - ApiCollectionResponseInner create(String resourceGroupName, String serviceName, String apiCollectionId); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionsClient.java index 6eded28ea9734..be7da87c723d1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApiCollectionsClient.java @@ -8,85 +8,268 @@ import com.azure.core.annotation.ServiceMethod; import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.models.ApiCollectionInner; -/** An instance of this class provides access to all the operations defined in ApiCollectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in ApiCollectionsClient. + */ public interface ApiCollectionsClient { /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(); + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(Context context); + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByResourceGroup(String resourceGroupName); + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable listByResourceGroup(String resourceGroupName, Context context); + + /** + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedIterable}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String resourceGroupName, String serviceName); + PagedIterable listByAzureApiManagementService(String resourceGroupName, String serviceName); /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedIterable}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String resourceGroupName, String serviceName, Context context); + PagedIterable listByAzureApiManagementService(String resourceGroupName, String serviceName, + Context context); + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs along with + * {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getByAzureApiManagementServiceWithResponse(String resourceGroupName, + String serviceName, String apiId, Context context); + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ApiCollectionInner getByAzureApiManagementService(String resourceGroupName, String serviceName, String apiId); + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, ApiCollectionInner> + beginOnboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId); + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, ApiCollectionInner> + beginOnboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId, Context context); + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ApiCollectionInner onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId); + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ApiCollectionInner onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId, + Context context); /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs along with {@link Response}. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context); + Response offboardAzureApiManagementApiWithResponse(String resourceGroupName, String serviceName, String apiId, + Context context); /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs. */ @ServiceMethod(returns = ReturnType.SINGLE) - ApiCollectionResponseInner get(String resourceGroupName, String serviceName, String apiCollectionId); + void offboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationOperationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationOperationsClient.java index 1454ded262341..19482d743f368 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationOperationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationOperationsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ApplicationInner; -/** An instance of this class provides access to all the operations defined in ApplicationOperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in ApplicationOperationsClient. + */ public interface ApplicationOperationsClient { /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -27,7 +29,7 @@ public interface ApplicationOperationsClient { /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -39,7 +41,7 @@ public interface ApplicationOperationsClient { /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @param context The context to associate with this operation. @@ -49,12 +51,12 @@ public interface ApplicationOperationsClient { * @return security Application over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String applicationId, ApplicationInner application, Context context); + Response createOrUpdateWithResponse(String applicationId, ApplicationInner application, + Context context); /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -67,7 +69,7 @@ Response createOrUpdateWithResponse( /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -80,7 +82,7 @@ Response createOrUpdateWithResponse( /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationsClient.java index 1f8851380eb88..a35f403f9bd84 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ApplicationsClient.java @@ -10,28 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ApplicationInner; -/** An instance of this class provides access to all the operations defined in ApplicationsClient. */ +/** + * An instance of this class provides access to all the operations defined in ApplicationsClient. + */ public interface ApplicationsClient { /** * Get a list of all relevant applications over a subscription level scope. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedIterable}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Get a list of all relevant applications over a subscription level scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedIterable}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsClient.java index ea903c0cc61a4..5c702d11ef8b2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsClient.java @@ -13,40 +13,42 @@ import com.azure.resourcemanager.security.models.ExpandEnum; import com.azure.resourcemanager.security.models.SecurityAssessment; -/** An instance of this class provides access to all the operations defined in AssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in AssessmentsClient. + */ public interface AssessmentsClient { /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope); /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope, Context context); /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param expand OData expand. Optional. @@ -57,12 +59,12 @@ public interface AssessmentsClient { * @return a security assessment on your scanned resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceId, String assessmentName, ExpandEnum expand, Context context); + Response getWithResponse(String resourceId, String assessmentName, + ExpandEnum expand, Context context); /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -76,7 +78,7 @@ Response getWithResponse( /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -87,13 +89,13 @@ Response getWithResponse( * @return security assessment on a resource - response format along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceId, String assessmentName, SecurityAssessment assessment, Context context); + Response createOrUpdateWithResponse(String resourceId, String assessmentName, + SecurityAssessment assessment, Context context); /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -103,13 +105,13 @@ Response createOrUpdateWithResponse( * @return security assessment on a resource - response format. */ @ServiceMethod(returns = ReturnType.SINGLE) - SecurityAssessmentResponseInner createOrUpdate( - String resourceId, String assessmentName, SecurityAssessment assessment); + SecurityAssessmentResponseInner createOrUpdate(String resourceId, String assessmentName, + SecurityAssessment assessment); /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. @@ -124,7 +126,7 @@ SecurityAssessmentResponseInner createOrUpdate( /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsMetadatasClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsMetadatasClient.java index d2d08e0c5684f..112d254fa3fba 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsMetadatasClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AssessmentsMetadatasClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecurityAssessmentMetadataResponseInner; -/** An instance of this class provides access to all the operations defined in AssessmentsMetadatasClient. */ +/** + * An instance of this class provides access to all the operations defined in AssessmentsMetadatasClient. + */ public interface AssessmentsMetadatasClient { /** * Get metadata information on all assessment types. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types as paginated response with {@link PagedIterable}. @@ -25,7 +27,7 @@ public interface AssessmentsMetadatasClient { /** * Get metadata information on all assessment types. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -37,7 +39,7 @@ public interface AssessmentsMetadatasClient { /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -50,7 +52,7 @@ public interface AssessmentsMetadatasClient { /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -62,31 +64,31 @@ public interface AssessmentsMetadatasClient { /** * Get metadata information on all assessment types in a specific subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedIterable}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listBySubscription(); /** * Get metadata information on all assessment types in a specific subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedIterable}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listBySubscription(Context context); /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -95,12 +97,12 @@ public interface AssessmentsMetadatasClient { * @return metadata information on an assessment type in a specific subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getInSubscriptionWithResponse( - String assessmentMetadataName, Context context); + Response getInSubscriptionWithResponse(String assessmentMetadataName, + Context context); /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -112,7 +114,7 @@ Response getInSubscriptionWithResponse( /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @param context The context to associate with this operation. @@ -122,12 +124,12 @@ Response getInSubscriptionWithResponse( * @return security assessment metadata response along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createInSubscriptionWithResponse( - String assessmentMetadataName, SecurityAssessmentMetadataResponseInner assessmentMetadata, Context context); + Response createInSubscriptionWithResponse(String assessmentMetadataName, + SecurityAssessmentMetadataResponseInner assessmentMetadata, Context context); /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -136,13 +138,13 @@ Response createInSubscriptionWithRespon * @return security assessment metadata response. */ @ServiceMethod(returns = ReturnType.SINGLE) - SecurityAssessmentMetadataResponseInner createInSubscription( - String assessmentMetadataName, SecurityAssessmentMetadataResponseInner assessmentMetadata); + SecurityAssessmentMetadataResponseInner createInSubscription(String assessmentMetadataName, + SecurityAssessmentMetadataResponseInner assessmentMetadata); /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -156,7 +158,7 @@ SecurityAssessmentMetadataResponseInner createInSubscription( /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutoProvisioningSettingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutoProvisioningSettingsClient.java index 7d93098d7d099..385c41b9a00c2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutoProvisioningSettingsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutoProvisioningSettingsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AutoProvisioningSettingInner; -/** An instance of this class provides access to all the operations defined in AutoProvisioningSettingsClient. */ +/** + * An instance of this class provides access to all the operations defined in AutoProvisioningSettingsClient. + */ public interface AutoProvisioningSettingsClient { /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response as paginated response with {@link PagedIterable}. @@ -25,7 +27,7 @@ public interface AutoProvisioningSettingsClient { /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -37,7 +39,7 @@ public interface AutoProvisioningSettingsClient { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -50,7 +52,7 @@ public interface AutoProvisioningSettingsClient { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -62,7 +64,7 @@ public interface AutoProvisioningSettingsClient { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @param context The context to associate with this operation. @@ -72,12 +74,12 @@ public interface AutoProvisioningSettingsClient { * @return auto provisioning setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String settingName, AutoProvisioningSettingInner setting, Context context); + Response createWithResponse(String settingName, AutoProvisioningSettingInner setting, + Context context); /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutomationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutomationsClient.java index cfbcb6c5613d9..7d18996b5d265 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutomationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AutomationsClient.java @@ -11,13 +11,16 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AutomationInner; import com.azure.resourcemanager.security.fluent.models.AutomationValidationStatusInner; +import com.azure.resourcemanager.security.models.AutomationUpdateModel; -/** An instance of this class provides access to all the operations defined in AutomationsClient. */ +/** + * An instance of this class provides access to all the operations defined in AutomationsClient. + */ public interface AutomationsClient { /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security automations response as paginated response with {@link PagedIterable}. @@ -28,7 +31,7 @@ public interface AutomationsClient { /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -41,9 +44,9 @@ public interface AutomationsClient { /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -55,9 +58,9 @@ public interface AutomationsClient { /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -69,9 +72,9 @@ public interface AutomationsClient { /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -80,14 +83,14 @@ public interface AutomationsClient { * @return the security automation resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String automationName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String automationName, + Context context); /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -100,9 +103,9 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -112,15 +115,15 @@ Response getByResourceGroupWithResponse( * @return the security automation resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, String automationName, AutomationInner automation, Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String automationName, + AutomationInner automation, Context context); /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -131,11 +134,43 @@ Response createOrUpdateWithResponse( @ServiceMethod(returns = ReturnType.SINGLE) AutomationInner createOrUpdate(String resourceGroupName, String automationName, AutomationInner automation); + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response updateWithResponse(String resourceGroupName, String automationName, + AutomationUpdateModel automation, Context context); + + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AutomationInner update(String resourceGroupName, String automationName, AutomationUpdateModel automation); + /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -148,9 +183,9 @@ Response createOrUpdateWithResponse( /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -162,9 +197,9 @@ Response createOrUpdateWithResponse( /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -174,15 +209,15 @@ Response createOrUpdateWithResponse( * @return the security automation model state property bag along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response validateWithResponse( - String resourceGroupName, String automationName, AutomationInner automation, Context context); + Response validateWithResponse(String resourceGroupName, String automationName, + AutomationInner automation, Context context); /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -191,6 +226,6 @@ Response validateWithResponse( * @return the security automation model state property bag. */ @ServiceMethod(returns = ReturnType.SINGLE) - AutomationValidationStatusInner validate( - String resourceGroupName, String automationName, AutomationInner automation); + AutomationValidationStatusInner validate(String resourceGroupName, String automationName, + AutomationInner automation); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsOrgsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsOrgsClient.java new file mode 100644 index 0000000000000..566832c05a501 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsOrgsClient.java @@ -0,0 +1,237 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgListResponseInner; + +/** + * An instance of this class provides access to all the operations defined in AzureDevOpsOrgsClient. + */ +public interface AzureDevOpsOrgsClient { + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context); + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsOrgListResponseInner listAvailable(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, Context context); + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, Context context); + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsOrgInner get(String resourceGroupName, String securityConnectorName, String orgName); + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsOrgInner> beginCreateOrUpdate(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg); + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsOrgInner> beginCreateOrUpdate(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context); + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsOrgInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg); + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsOrgInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg, Context context); + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsOrgInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg); + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsOrgInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context); + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsOrgInner update(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg); + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsOrgInner update(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg, Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsProjectsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsProjectsClient.java new file mode 100644 index 0000000000000..8f0c62b567356 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsProjectsClient.java @@ -0,0 +1,225 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner; + +/** + * An instance of this class provides access to all the operations defined in AzureDevOpsProjectsClient. + */ +public interface AzureDevOpsProjectsClient { + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, String orgName); + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, String orgName, + Context context); + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context); + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsProjectInner get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName); + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsProjectInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject); + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsProjectInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject, Context context); + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsProjectInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, AzureDevOpsProjectInner azureDevOpsProject); + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsProjectInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, AzureDevOpsProjectInner azureDevOpsProject, Context context); + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsProjectInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject); + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsProjectInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject, + Context context); + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsProjectInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, AzureDevOpsProjectInner azureDevOpsProject); + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsProjectInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, AzureDevOpsProjectInner azureDevOpsProject, Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsReposClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsReposClient.java new file mode 100644 index 0000000000000..0c8ef8061ae1e --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/AzureDevOpsReposClient.java @@ -0,0 +1,239 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner; + +/** + * An instance of this class provides access to all the operations defined in AzureDevOpsReposClient. + */ +public interface AzureDevOpsReposClient { + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, String projectName); + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context); + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, Context context); + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsRepositoryInner get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName); + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsRepositoryInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository); + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsRepositoryInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context); + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsRepositoryInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository); + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsRepositoryInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository, Context context); + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsRepositoryInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository); + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, AzureDevOpsRepositoryInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context); + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsRepositoryInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository); + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + AzureDevOpsRepositoryInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository, Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ComplianceResultsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ComplianceResultsClient.java index 834cf0990b35a..2f3bb8e37c727 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ComplianceResultsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ComplianceResultsClient.java @@ -11,13 +11,15 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ComplianceResultInner; -/** An instance of this class provides access to all the operations defined in ComplianceResultsClient. */ +/** + * An instance of this class provides access to all the operations defined in ComplianceResultsClient. + */ public interface ComplianceResultsClient { /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -28,9 +30,9 @@ public interface ComplianceResultsClient { /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -42,7 +44,7 @@ public interface ComplianceResultsClient { /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @param context The context to associate with this operation. @@ -56,7 +58,7 @@ public interface ComplianceResultsClient { /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CompliancesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CompliancesClient.java index 7914ab6d42930..a592b826901c0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CompliancesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CompliancesClient.java @@ -11,13 +11,15 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ComplianceInner; -/** An instance of this class provides access to all the operations defined in CompliancesClient. */ +/** + * An instance of this class provides access to all the operations defined in CompliancesClient. + */ public interface CompliancesClient { /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -28,9 +30,9 @@ public interface CompliancesClient { /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -42,9 +44,9 @@ public interface CompliancesClient { /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -57,9 +59,9 @@ public interface CompliancesClient { /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ConnectorsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ConnectorsClient.java index fc3073c4ec5a4..7f2e366138284 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ConnectorsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ConnectorsClient.java @@ -11,35 +11,37 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ConnectorSettingInner; -/** An instance of this class provides access to all the operations defined in ConnectorsClient. */ +/** + * An instance of this class provides access to all the operations defined in ConnectorsClient. + */ public interface ConnectorsClient { /** * Cloud accounts connectors of a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Cloud accounts connectors of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -52,7 +54,7 @@ public interface ConnectorsClient { /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -65,7 +67,7 @@ public interface ConnectorsClient { /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @param context The context to associate with this operation. @@ -75,13 +77,13 @@ public interface ConnectorsClient { * @return the connector setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String connectorName, ConnectorSettingInner connectorSetting, Context context); + Response createOrUpdateWithResponse(String connectorName, + ConnectorSettingInner connectorSetting, Context context); /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -94,7 +96,7 @@ Response createOrUpdateWithResponse( /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -107,7 +109,7 @@ Response createOrUpdateWithResponse( /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomAssessmentAutomationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomAssessmentAutomationsClient.java index cdd6a0b7a94f7..d274216083560 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomAssessmentAutomationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomAssessmentAutomationsClient.java @@ -12,34 +12,36 @@ import com.azure.resourcemanager.security.fluent.models.CustomAssessmentAutomationInner; import com.azure.resourcemanager.security.models.CustomAssessmentAutomationRequest; -/** An instance of this class provides access to all the operations defined in CustomAssessmentAutomationsClient. */ +/** + * An instance of this class provides access to all the operations defined in CustomAssessmentAutomationsClient. + */ public interface CustomAssessmentAutomationsClient { /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String customAssessmentAutomationName, Context context); /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -51,12 +53,12 @@ Response getByResourceGroupWithResponse( /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @param context The context to associate with this operation. @@ -66,20 +68,18 @@ Response getByResourceGroupWithResponse( * @return custom Assessment Automation along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String resourceGroupName, - String customAssessmentAutomationName, - CustomAssessmentAutomationRequest customAssessmentAutomationBody, + Response createWithResponse(String resourceGroupName, + String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody, Context context); /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -88,18 +88,16 @@ Response createWithResponse( * @return custom Assessment Automation. */ @ServiceMethod(returns = ReturnType.SINGLE) - CustomAssessmentAutomationInner create( - String resourceGroupName, - String customAssessmentAutomationName, + CustomAssessmentAutomationInner create(String resourceGroupName, String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody); /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -112,11 +110,11 @@ CustomAssessmentAutomationInner create( /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -127,11 +125,11 @@ CustomAssessmentAutomationInner create( /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -142,11 +140,11 @@ CustomAssessmentAutomationInner create( /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -158,9 +156,9 @@ CustomAssessmentAutomationInner create( /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations as paginated response with {@link PagedIterable}. @@ -170,9 +168,9 @@ CustomAssessmentAutomationInner create( /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomEntityStoreAssignmentsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomEntityStoreAssignmentsClient.java index 5f7d062043ade..74a13bec8c99f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomEntityStoreAssignmentsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/CustomEntityStoreAssignmentsClient.java @@ -12,34 +12,36 @@ import com.azure.resourcemanager.security.fluent.models.CustomEntityStoreAssignmentInner; import com.azure.resourcemanager.security.models.CustomEntityStoreAssignmentRequest; -/** An instance of this class provides access to all the operations defined in CustomEntityStoreAssignmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in CustomEntityStoreAssignmentsClient. + */ public interface CustomEntityStoreAssignmentsClient { /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response}. + * with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String customEntityStoreAssignmentName, Context context); /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -47,16 +49,16 @@ Response getByResourceGroupWithResponse( * @return a single custom entity store assignment by name for the provided subscription and resource group. */ @ServiceMethod(returns = ReturnType.SINGLE) - CustomEntityStoreAssignmentInner getByResourceGroup( - String resourceGroupName, String customEntityStoreAssignmentName); + CustomEntityStoreAssignmentInner getByResourceGroup(String resourceGroupName, + String customEntityStoreAssignmentName); /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @param context The context to associate with this operation. @@ -66,19 +68,17 @@ CustomEntityStoreAssignmentInner getByResourceGroup( * @return custom entity store assignment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String resourceGroupName, + Response createWithResponse(String resourceGroupName, String customEntityStoreAssignmentName, - CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, - Context context); + CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, Context context); /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -87,18 +87,16 @@ Response createWithResponse( * @return custom entity store assignment. */ @ServiceMethod(returns = ReturnType.SINGLE) - CustomEntityStoreAssignmentInner create( - String resourceGroupName, - String customEntityStoreAssignmentName, + CustomEntityStoreAssignmentInner create(String resourceGroupName, String customEntityStoreAssignmentName, CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody); /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -107,16 +105,16 @@ CustomEntityStoreAssignmentInner create( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context); + Response deleteWithResponse(String resourceGroupName, String customEntityStoreAssignmentName, + Context context); /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -127,11 +125,11 @@ Response deleteWithResponse( /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -142,11 +140,11 @@ Response deleteWithResponse( /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -158,9 +156,9 @@ Response deleteWithResponse( /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments as paginated response with {@link PagedIterable}. @@ -170,9 +168,9 @@ Response deleteWithResponse( /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DefenderForStoragesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DefenderForStoragesClient.java new file mode 100644 index 0000000000000..a703e8d5114d0 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DefenderForStoragesClient.java @@ -0,0 +1,76 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.DefenderForStorageSettingInner; +import com.azure.resourcemanager.security.models.SettingName; + +/** + * An instance of this class provides access to all the operations defined in DefenderForStoragesClient. + */ +public interface DefenderForStoragesClient { + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceId, SettingName settingName, + Context context); + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DefenderForStorageSettingInner get(String resourceId, SettingName settingName); + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createWithResponse(String resourceId, SettingName settingName, + DefenderForStorageSettingInner defenderForStorageSetting, Context context); + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DefenderForStorageSettingInner create(String resourceId, SettingName settingName, + DefenderForStorageSettingInner defenderForStorageSetting); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsConfigurationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsConfigurationsClient.java new file mode 100644 index 0000000000000..072a55b3132f0 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsConfigurationsClient.java @@ -0,0 +1,253 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; + +/** + * An instance of this class provides access to all the operations defined in DevOpsConfigurationsClient. + */ +public interface DevOpsConfigurationsClient { + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName); + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context); + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + Context context); + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DevOpsConfigurationInner get(String resourceGroupName, String securityConnectorName); + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, DevOpsConfigurationInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration); + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, DevOpsConfigurationInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, + Context context); + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DevOpsConfigurationInner createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration); + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DevOpsConfigurationInner createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context); + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, DevOpsConfigurationInner> beginUpdate(String resourceGroupName, + String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration); + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, DevOpsConfigurationInner> beginUpdate(String resourceGroupName, + String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, Context context); + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DevOpsConfigurationInner update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration); + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + DevOpsConfigurationInner update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context); + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String securityConnectorName); + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + SyncPoller, Void> beginDelete(String resourceGroupName, String securityConnectorName, + Context context); + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String securityConnectorName); + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(String resourceGroupName, String securityConnectorName, Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsOperationResultsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsOperationResultsClient.java new file mode 100644 index 0000000000000..104278955f2d8 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DevOpsOperationResultsClient.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.OperationStatusResultInner; + +/** + * An instance of this class provides access to all the operations defined in DevOpsOperationResultsClient. + */ +public interface DevOpsOperationResultsClient { + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String operationResultId, Context context); + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + OperationStatusResultInner get(String resourceGroupName, String securityConnectorName, String operationResultId); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DeviceSecurityGroupsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DeviceSecurityGroupsClient.java index e03cd1b990c0e..72dab9a527ea8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DeviceSecurityGroupsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DeviceSecurityGroupsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.DeviceSecurityGroupInner; -/** An instance of this class provides access to all the operations defined in DeviceSecurityGroupsClient. */ +/** + * An instance of this class provides access to all the operations defined in DeviceSecurityGroupsClient. + */ public interface DeviceSecurityGroupsClient { /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -27,7 +29,7 @@ public interface DeviceSecurityGroupsClient { /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -40,10 +42,10 @@ public interface DeviceSecurityGroupsClient { /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -51,15 +53,15 @@ public interface DeviceSecurityGroupsClient { * @return the device security group resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceId, String deviceSecurityGroupName, Context context); + Response getWithResponse(String resourceId, String deviceSecurityGroupName, + Context context); /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -70,10 +72,10 @@ Response getWithResponse( /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -82,18 +84,15 @@ Response getWithResponse( * @return the device security group resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceId, - String deviceSecurityGroupName, - DeviceSecurityGroupInner deviceSecurityGroup, - Context context); + Response createOrUpdateWithResponse(String resourceId, String deviceSecurityGroupName, + DeviceSecurityGroupInner deviceSecurityGroup, Context context); /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -101,15 +100,15 @@ Response createOrUpdateWithResponse( * @return the device security group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - DeviceSecurityGroupInner createOrUpdate( - String resourceId, String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup); + DeviceSecurityGroupInner createOrUpdate(String resourceId, String deviceSecurityGroupName, + DeviceSecurityGroupInner deviceSecurityGroup); /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -121,10 +120,10 @@ DeviceSecurityGroupInner createOrUpdate( /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DiscoveredSecuritySolutionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DiscoveredSecuritySolutionsClient.java index 692eeaf708ccb..47c4eb45a523d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DiscoveredSecuritySolutionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/DiscoveredSecuritySolutionsClient.java @@ -11,68 +11,70 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.DiscoveredSecuritySolutionInner; -/** An instance of this class provides access to all the operations defined in DiscoveredSecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in DiscoveredSecuritySolutionsClient. + */ public interface DiscoveredSecuritySolutionsClient { /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation); /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation, Context context); /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -81,16 +83,16 @@ public interface DiscoveredSecuritySolutionsClient { * @return a specific discovered Security Solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName, Context context); /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -98,6 +100,6 @@ Response getWithResponse( * @return a specific discovered Security Solution. */ @ServiceMethod(returns = ReturnType.SINGLE) - DiscoveredSecuritySolutionInner get( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName); + DiscoveredSecuritySolutionInner get(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ExternalSecuritySolutionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ExternalSecuritySolutionsClient.java index ace625003ab69..1acdd16924f0a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ExternalSecuritySolutionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ExternalSecuritySolutionsClient.java @@ -11,68 +11,70 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ExternalSecuritySolutionInner; -/** An instance of this class provides access to all the operations defined in ExternalSecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in ExternalSecuritySolutionsClient. + */ public interface ExternalSecuritySolutionsClient { /** * Gets a list of external security solutions for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external security solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of external security solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Gets a list of external security solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external security solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of external security solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedIterable}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation); /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedIterable}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation, Context context); /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -81,16 +83,16 @@ public interface ExternalSecuritySolutionsClient { * @return a specific external Security Solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName, Context context); /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -98,6 +100,6 @@ Response getWithResponse( * @return a specific external Security Solution. */ @ServiceMethod(returns = ReturnType.SINGLE) - ExternalSecuritySolutionInner get( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName); + ExternalSecuritySolutionInner get(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubOwnersClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubOwnersClient.java new file mode 100644 index 0000000000000..7a4c2cb6a4a40 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubOwnersClient.java @@ -0,0 +1,103 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerInner; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerListResponseInner; + +/** + * An instance of this class provides access to all the operations defined in GitHubOwnersClient. + */ +public interface GitHubOwnersClient { + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context); + + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitHubOwnerListResponseInner listAvailable(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, Context context); + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, String ownerName, + Context context); + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitHubOwnerInner get(String resourceGroupName, String securityConnectorName, String ownerName); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubReposClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubReposClient.java new file mode 100644 index 0000000000000..fc4eb380b972a --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitHubReposClient.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.GitHubRepositoryInner; + +/** + * An instance of this class provides access to all the operations defined in GitHubReposClient. + */ +public interface GitHubReposClient { + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, String ownerName); + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, String ownerName, + Context context); + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String ownerName, String repoName, Context context); + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitHubRepositoryInner get(String resourceGroupName, String securityConnectorName, String ownerName, + String repoName); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabGroupsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabGroupsClient.java new file mode 100644 index 0000000000000..a20ab3e0215d0 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabGroupsClient.java @@ -0,0 +1,103 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupInner; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; + +/** + * An instance of this class provides access to all the operations defined in GitLabGroupsClient. + */ +public interface GitLabGroupsClient { + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context); + + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitLabGroupListResponseInner listAvailable(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, Context context); + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context); + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitLabGroupInner get(String resourceGroupName, String securityConnectorName, String groupFQName); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabProjectsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabProjectsClient.java new file mode 100644 index 0000000000000..9faf6c688bccf --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabProjectsClient.java @@ -0,0 +1,82 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.GitLabProjectInner; + +/** + * An instance of this class provides access to all the operations defined in GitLabProjectsClient. + */ +public interface GitLabProjectsClient { + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector as + * paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, String groupFQName); + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector as + * paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(String resourceGroupName, String securityConnectorName, String groupFQName, + Context context); + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, String projectName, Context context); + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitLabProjectInner get(String resourceGroupName, String securityConnectorName, String groupFQName, + String projectName); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabSubgroupsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabSubgroupsClient.java new file mode 100644 index 0000000000000..ffcc005c753ba --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GitLabSubgroupsClient.java @@ -0,0 +1,46 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; + +/** + * An instance of this class provides access to all the operations defined in GitLabSubgroupsClient. + */ +public interface GitLabSubgroupsClient { + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context); + + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GitLabGroupListResponseInner list(String resourceGroupName, String securityConnectorName, String groupFQName); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceAssignmentsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceAssignmentsClient.java index ab8458ddf09cb..bb00ad61eea32 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceAssignmentsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceAssignmentsClient.java @@ -11,47 +11,49 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.GovernanceAssignmentInner; -/** An instance of this class provides access to all the operations defined in GovernanceAssignmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in GovernanceAssignmentsClient. + */ public interface GovernanceAssignmentsClient { /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignments on all of your resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return governance assignments on all of your resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope, String assessmentName); /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignments on all of your resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return governance assignments on all of your resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope, String assessmentName, Context context); /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param context The context to associate with this operation. @@ -61,15 +63,15 @@ public interface GovernanceAssignmentsClient { * @return a specific governanceAssignment for the requested scope by AssignmentKey along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String scope, String assessmentName, String assignmentKey, Context context); + Response getWithResponse(String scope, String assessmentName, String assignmentKey, + Context context); /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -82,10 +84,10 @@ Response getWithResponse( /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. @@ -96,19 +98,15 @@ Response getWithResponse( * @return governance assignment over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String scope, - String assessmentName, - String assignmentKey, - GovernanceAssignmentInner governanceAssignment, - Context context); + Response createOrUpdateWithResponse(String scope, String assessmentName, + String assignmentKey, GovernanceAssignmentInner governanceAssignment, Context context); /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. @@ -118,15 +116,15 @@ Response createOrUpdateWithResponse( * @return governance assignment over a given scope. */ @ServiceMethod(returns = ReturnType.SINGLE) - GovernanceAssignmentInner createOrUpdate( - String scope, String assessmentName, String assignmentKey, GovernanceAssignmentInner governanceAssignment); + GovernanceAssignmentInner createOrUpdate(String scope, String assessmentName, String assignmentKey, + GovernanceAssignmentInner governanceAssignment); /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param context The context to associate with this operation. @@ -140,10 +138,10 @@ GovernanceAssignmentInner createOrUpdate( /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceRulesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceRulesClient.java index d6f4ace6458cf..a75e3e9290b69 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceRulesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/GovernanceRulesClient.java @@ -16,15 +16,17 @@ import com.azure.resourcemanager.security.models.ExecuteGovernanceRuleParams; import com.azure.resourcemanager.security.models.GovernanceRulesOperationResultsResponse; -/** An instance of this class provides access to all the operations defined in GovernanceRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in GovernanceRulesClient. + */ public interface GovernanceRulesClient { /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -35,11 +37,11 @@ public interface GovernanceRulesClient { /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -51,11 +53,11 @@ public interface GovernanceRulesClient { /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -68,11 +70,11 @@ public interface GovernanceRulesClient { /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -84,11 +86,11 @@ public interface GovernanceRulesClient { /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @param context The context to associate with this operation. @@ -98,16 +100,16 @@ public interface GovernanceRulesClient { * @return governance rule over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String scope, String ruleId, GovernanceRuleInner governanceRule, Context context); + Response createOrUpdateWithResponse(String scope, String ruleId, + GovernanceRuleInner governanceRule, Context context); /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -120,11 +122,11 @@ Response createOrUpdateWithResponse( /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -136,11 +138,11 @@ Response createOrUpdateWithResponse( /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -153,11 +155,11 @@ Response createOrUpdateWithResponse( /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -168,11 +170,11 @@ Response createOrUpdateWithResponse( /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -184,11 +186,11 @@ Response createOrUpdateWithResponse( /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -200,11 +202,11 @@ Response createOrUpdateWithResponse( /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -214,16 +216,16 @@ Response createOrUpdateWithResponse( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginExecute( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context); + SyncPoller, Void> beginExecute(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context); /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -234,11 +236,11 @@ SyncPoller, Void> beginExecute( /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -251,11 +253,11 @@ SyncPoller, Void> beginExecute( /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @param context The context to associate with this operation. @@ -265,16 +267,16 @@ SyncPoller, Void> beginExecute( * @return governance rules long run operation result for the requested scope by ruleId and operationId. */ @ServiceMethod(returns = ReturnType.SINGLE) - GovernanceRulesOperationResultsResponse operationResultsWithResponse( - String scope, String ruleId, String operationId, Context context); + GovernanceRulesOperationResultsResponse operationResultsWithResponse(String scope, String ruleId, + String operationId, Context context); /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/HealthReportsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/HealthReportsClient.java index 57ee9785cfd2f..2db32bbb22aa4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/HealthReportsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/HealthReportsClient.java @@ -7,16 +7,19 @@ import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceMethod; import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.HealthReportInner; -/** An instance of this class provides access to all the operations defined in HealthReportsClient. */ +/** + * An instance of this class provides access to all the operations defined in HealthReportsClient. + */ public interface HealthReportsClient { /** * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -30,7 +33,7 @@ public interface HealthReportsClient { * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -40,4 +43,31 @@ public interface HealthReportsClient { */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope, Context context); + + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response getWithResponse(String resourceId, String healthReportName, Context context); + + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + HealthReportInner get(String resourceId, String healthReportName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/InformationProtectionPoliciesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/InformationProtectionPoliciesClient.java index a43e048fb2faf..4004e84aa1033 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/InformationProtectionPoliciesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/InformationProtectionPoliciesClient.java @@ -12,13 +12,15 @@ import com.azure.resourcemanager.security.fluent.models.InformationProtectionPolicyInner; import com.azure.resourcemanager.security.models.InformationProtectionPolicyName; -/** An instance of this class provides access to all the operations defined in InformationProtectionPoliciesClient. */ +/** + * An instance of this class provides access to all the operations defined in InformationProtectionPoliciesClient. + */ public interface InformationProtectionPoliciesClient { /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -27,14 +29,14 @@ public interface InformationProtectionPoliciesClient { * @return information protection policy along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String scope, InformationProtectionPolicyName informationProtectionPolicyName, Context context); + Response getWithResponse(String scope, + InformationProtectionPolicyName informationProtectionPolicyName, Context context); /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -46,9 +48,9 @@ Response getWithResponse( /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @param context The context to associate with this operation. @@ -58,17 +60,15 @@ Response getWithResponse( * @return information protection policy along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String scope, + Response createOrUpdateWithResponse(String scope, InformationProtectionPolicyName informationProtectionPolicyName, - InformationProtectionPolicyInner informationProtectionPolicy, - Context context); + InformationProtectionPolicyInner informationProtectionPolicy, Context context); /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -77,16 +77,15 @@ Response createOrUpdateWithResponse( * @return information protection policy. */ @ServiceMethod(returns = ReturnType.SINGLE) - InformationProtectionPolicyInner createOrUpdate( - String scope, + InformationProtectionPolicyInner createOrUpdate(String scope, InformationProtectionPolicyName informationProtectionPolicyName, InformationProtectionPolicyInner informationProtectionPolicy); /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -97,9 +96,9 @@ InformationProtectionPolicyInner createOrUpdate( /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IngestionSettingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IngestionSettingsClient.java deleted file mode 100644 index c0765e68586c8..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IngestionSettingsClient.java +++ /dev/null @@ -1,169 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent; - -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; -import com.azure.resourcemanager.security.fluent.models.ConnectionStringsInner; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingInner; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingTokenInner; - -/** An instance of this class provides access to all the operations defined in IngestionSettingsClient. */ -public interface IngestionSettingsClient { - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(); - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(Context context); - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String ingestionSettingName, Context context); - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - IngestionSettingInner get(String ingestionSettingName); - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String ingestionSettingName, IngestionSettingInner ingestionSetting, Context context); - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - IngestionSettingInner create(String ingestionSettingName, IngestionSettingInner ingestionSetting); - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse(String ingestionSettingName, Context context); - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String ingestionSettingName); - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response listTokensWithResponse(String ingestionSettingName, Context context); - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - IngestionSettingTokenInner listTokens(String ingestionSettingName); - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - Response listConnectionStringsWithResponse(String ingestionSettingName, Context context); - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - ConnectionStringsInner listConnectionStrings(String ingestionSettingName); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionAnalyticsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionAnalyticsClient.java index 25a019f2f016c..9805485a48cf1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionAnalyticsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionAnalyticsClient.java @@ -11,13 +11,15 @@ import com.azure.resourcemanager.security.fluent.models.IoTSecuritySolutionAnalyticsModelInner; import com.azure.resourcemanager.security.fluent.models.IoTSecuritySolutionAnalyticsModelListInner; -/** An instance of this class provides access to all the operations defined in IotSecuritySolutionAnalyticsClient. */ +/** + * An instance of this class provides access to all the operations defined in IotSecuritySolutionAnalyticsClient. + */ public interface IotSecuritySolutionAnalyticsClient { /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -26,14 +28,14 @@ public interface IotSecuritySolutionAnalyticsClient { * @return list of Security analytics of your IoT Security solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listWithResponse( - String resourceGroupName, String solutionName, Context context); + Response listWithResponse(String resourceGroupName, String solutionName, + Context context); /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -45,9 +47,9 @@ Response listWithResponse( /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -56,14 +58,14 @@ Response listWithResponse( * @return security analytics of your IoT Security solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String solutionName, Context context); + Response getWithResponse(String resourceGroupName, String solutionName, + Context context); /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsAggregatedAlertsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsAggregatedAlertsClient.java index 1ea68d2ece5c7..6f022394b02a1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsAggregatedAlertsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsAggregatedAlertsClient.java @@ -18,9 +18,9 @@ public interface IotSecuritySolutionsAnalyticsAggregatedAlertsClient { /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,9 +32,9 @@ public interface IotSecuritySolutionsAnalyticsAggregatedAlertsClient { /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. @@ -44,15 +44,15 @@ public interface IotSecuritySolutionsAnalyticsAggregatedAlertsClient { * @return list of IoT Security solution aggregated alert data as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String resourceGroupName, String solutionName, Integer top, Context context); + PagedIterable list(String resourceGroupName, String solutionName, Integer top, + Context context); /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @param context The context to associate with this operation. @@ -62,15 +62,15 @@ PagedIterable list( * @return security Solution Aggregated Alert information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context); + Response getWithResponse(String resourceGroupName, String solutionName, + String aggregatedAlertName, Context context); /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -83,9 +83,9 @@ Response getWithResponse( /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @param context The context to associate with this operation. @@ -95,14 +95,14 @@ Response getWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response dismissWithResponse( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context); + Response dismissWithResponse(String resourceGroupName, String solutionName, String aggregatedAlertName, + Context context); /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsRecommendationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsRecommendationsClient.java index 1e534a2b033ea..19cd3bba06d4b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsRecommendationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsAnalyticsRecommendationsClient.java @@ -19,9 +19,9 @@ public interface IotSecuritySolutionsAnalyticsRecommendationsClient { /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @param context The context to associate with this operation. @@ -31,15 +31,15 @@ public interface IotSecuritySolutionsAnalyticsRecommendationsClient { * @return ioT Security solution recommendation information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String solutionName, String aggregatedRecommendationName, Context context); + Response getWithResponse(String resourceGroupName, String solutionName, + String aggregatedRecommendationName, Context context); /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -48,39 +48,39 @@ Response getWithResponse( * @return ioT Security solution recommendation information. */ @ServiceMethod(returns = ReturnType.SINGLE) - IoTSecurityAggregatedRecommendationInner get( - String resourceGroupName, String solutionName, String aggregatedRecommendationName); + IoTSecurityAggregatedRecommendationInner get(String resourceGroupName, String solutionName, + String aggregatedRecommendationName); /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of IoT Security solution aggregated recommendations as paginated response with {@link - * PagedIterable}. + * @return list of IoT Security solution aggregated recommendations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String resourceGroupName, String solutionName); /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of IoT Security solution aggregated recommendations as paginated response with {@link - * PagedIterable}. + * @return list of IoT Security solution aggregated recommendations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String resourceGroupName, String solutionName, Integer top, Context context); + PagedIterable list(String resourceGroupName, String solutionName, + Integer top, Context context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsClient.java index 140ffd6f2f623..cdeb1394af308 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/IotSecuritySolutionsClient.java @@ -12,11 +12,13 @@ import com.azure.resourcemanager.security.fluent.models.IoTSecuritySolutionModelInner; import com.azure.resourcemanager.security.models.UpdateIotSecuritySolutionData; -/** An instance of this class provides access to all the operations defined in IotSecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in IotSecuritySolutionsClient. + */ public interface IotSecuritySolutionsClient { /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solutions as paginated response with {@link PagedIterable}. @@ -26,7 +28,7 @@ public interface IotSecuritySolutionsClient { /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -39,9 +41,9 @@ public interface IotSecuritySolutionsClient { /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -52,9 +54,9 @@ public interface IotSecuritySolutionsClient { /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -63,14 +65,14 @@ public interface IotSecuritySolutionsClient { * @return list of IoT Security solutions as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroup( - String resourceGroupName, String filter, Context context); + PagedIterable listByResourceGroup(String resourceGroupName, String filter, + Context context); /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,14 +81,14 @@ PagedIterable listByResourceGroup( * @return ioT Security solution configuration and resource information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String solutionName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String solutionName, Context context); /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -98,9 +100,9 @@ Response getByResourceGroupWithResponse( /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @param context The context to associate with this operation. @@ -110,17 +112,14 @@ Response getByResourceGroupWithResponse( * @return ioT Security solution configuration and resource information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String solutionName, - IoTSecuritySolutionModelInner iotSecuritySolutionData, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String solutionName, + IoTSecuritySolutionModelInner iotSecuritySolutionData, Context context); /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -129,15 +128,15 @@ Response createOrUpdateWithResponse( * @return ioT Security solution configuration and resource information. */ @ServiceMethod(returns = ReturnType.SINGLE) - IoTSecuritySolutionModelInner createOrUpdate( - String resourceGroupName, String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData); + IoTSecuritySolutionModelInner createOrUpdate(String resourceGroupName, String solutionName, + IoTSecuritySolutionModelInner iotSecuritySolutionData); /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @param context The context to associate with this operation. @@ -147,18 +146,15 @@ IoTSecuritySolutionModelInner createOrUpdate( * @return ioT Security solution configuration and resource information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String solutionName, - UpdateIotSecuritySolutionData updateIotSecuritySolutionData, - Context context); + Response updateWithResponse(String resourceGroupName, String solutionName, + UpdateIotSecuritySolutionData updateIotSecuritySolutionData, Context context); /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -167,14 +163,14 @@ Response updateWithResponse( * @return ioT Security solution configuration and resource information. */ @ServiceMethod(returns = ReturnType.SINGLE) - IoTSecuritySolutionModelInner update( - String resourceGroupName, String solutionName, UpdateIotSecuritySolutionData updateIotSecuritySolutionData); + IoTSecuritySolutionModelInner update(String resourceGroupName, String solutionName, + UpdateIotSecuritySolutionData updateIotSecuritySolutionData); /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -187,9 +183,9 @@ IoTSecuritySolutionModelInner update( /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/JitNetworkAccessPoliciesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/JitNetworkAccessPoliciesClient.java index 3170b606a0ed5..9dc4892f7ed93 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/JitNetworkAccessPoliciesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/JitNetworkAccessPoliciesClient.java @@ -13,11 +13,13 @@ import com.azure.resourcemanager.security.fluent.models.JitNetworkAccessRequestInner; import com.azure.resourcemanager.security.models.JitNetworkAccessPolicyInitiateRequest; -/** An instance of this class provides access to all the operations defined in JitNetworkAccessPoliciesClient. */ +/** + * An instance of this class provides access to all the operations defined in JitNetworkAccessPoliciesClient. + */ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the paginated response with {@link PagedIterable}. @@ -27,7 +29,7 @@ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -39,9 +41,9 @@ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -52,9 +54,9 @@ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -66,9 +68,9 @@ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -79,9 +81,9 @@ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -93,27 +95,27 @@ public interface JitNetworkAccessPoliciesClient { /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroupAndRegion( - String resourceGroupName, String ascLocation); + PagedIterable listByResourceGroupAndRegion(String resourceGroupName, + String ascLocation); /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -121,16 +123,16 @@ PagedIterable listByResourceGroupAndRegion( * @return the paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroupAndRegion( - String resourceGroupName, String ascLocation, Context context); + PagedIterable listByResourceGroupAndRegion(String resourceGroupName, + String ascLocation, Context context); /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -139,16 +141,16 @@ PagedIterable listByResourceGroupAndRegion( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, Context context); /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -160,11 +162,11 @@ Response getWithResponse( /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @param context The context to associate with this operation. @@ -174,20 +176,16 @@ Response getWithResponse( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body, Context context); /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -196,19 +194,16 @@ Response createOrUpdateWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - JitNetworkAccessPolicyInner createOrUpdate( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body); + JitNetworkAccessPolicyInner createOrUpdate(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body); /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -217,16 +212,16 @@ JitNetworkAccessPolicyInner createOrUpdate( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context); + Response deleteWithResponse(String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, + Context context); /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -237,11 +232,11 @@ Response deleteWithResponse( /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @param context The context to associate with this operation. @@ -251,20 +246,16 @@ Response deleteWithResponse( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response initiateWithResponse( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body, - Context context); + Response initiateWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body, Context context); /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -273,9 +264,6 @@ Response initiateWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - JitNetworkAccessRequestInner initiate( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body); + JitNetworkAccessRequestInner initiate(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/LocationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/LocationsClient.java index 9215df9982263..3addbc8dafd4c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/LocationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/LocationsClient.java @@ -11,13 +11,15 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AscLocationInner; -/** An instance of this class provides access to all the operations defined in LocationsClient. */ +/** + * An instance of this class provides access to all the operations defined in LocationsClient. + */ public interface LocationsClient { /** * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data as paginated response with {@link PagedIterable}. @@ -29,7 +31,7 @@ public interface LocationsClient { * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -41,9 +43,9 @@ public interface LocationsClient { /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -55,9 +57,9 @@ public interface LocationsClient { /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/MdeOnboardingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/MdeOnboardingsClient.java index f1a31c9a245a4..1326b7d829a95 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/MdeOnboardingsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/MdeOnboardingsClient.java @@ -11,11 +11,13 @@ import com.azure.resourcemanager.security.fluent.models.MdeOnboardingDataInner; import com.azure.resourcemanager.security.fluent.models.MdeOnboardingDataListInner; -/** An instance of this class provides access to all the operations defined in MdeOnboardingsClient. */ +/** + * An instance of this class provides access to all the operations defined in MdeOnboardingsClient. + */ public interface MdeOnboardingsClient { /** * The configuration or data needed to onboard the machine to MDE. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -27,7 +29,7 @@ public interface MdeOnboardingsClient { /** * The configuration or data needed to onboard the machine to MDE. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all MDE onboarding data resources. @@ -37,20 +39,20 @@ public interface MdeOnboardingsClient { /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the resource of the configuration or data needed to onboard the machine to MDE along with {@link - * Response}. + * @return the resource of the configuration or data needed to onboard the machine to MDE along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response getWithResponse(Context context); /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource of the configuration or data needed to onboard the machine to MDE. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/OperationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/OperationsClient.java index 06e04115fd4b2..4c7cab5d704e4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/OperationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/OperationsClient.java @@ -10,28 +10,30 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.OperationInner; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public interface OperationsClient { /** * Exposes all available operations for discovery purposes. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of possible operations for Microsoft.Security resource provider as paginated response with {@link - * PagedIterable}. + * @return list of possible operations for Microsoft.Security resource provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Exposes all available operations for discovery purposes. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of possible operations for Microsoft.Security resource provider as paginated response with {@link - * PagedIterable}. + * @return list of possible operations for Microsoft.Security resource provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/PricingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/PricingsClient.java index fefcef79c849f..5c660d85ed983 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/PricingsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/PricingsClient.java @@ -11,81 +11,155 @@ import com.azure.resourcemanager.security.fluent.models.PricingInner; import com.azure.resourcemanager.security.fluent.models.PricingListInner; -/** An instance of this class provides access to all the operations defined in PricingsClient. */ +/** + * An instance of this class provides access to all the operations defined in PricingsClient. + */ public interface PricingsClient { /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response along with {@link Response}. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id) along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listWithResponse(Context context); + Response getWithResponse(String scopeId, String pricingName, Context context); /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). */ @ServiceMethod(returns = ReturnType.SINGLE) - PricingListInner list(); + PricingInner get(String scopeId, String pricingName); /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription along with {@link - * Response}. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String pricingName, Context context); + Response updateWithResponse(String scopeId, String pricingName, PricingInner pricing, + Context context); /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard. */ @ServiceMethod(returns = ReturnType.SINGLE) - PricingInner get(String pricingName); + PricingInner update(String scopeId, String pricingName, PricingInner pricing); /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with {@link - * Response}. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse(String pricingName, PricingInner pricing, Context context); + Response deleteWithResponse(String scopeId, String pricingName, Context context); /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard. */ @ServiceMethod(returns = ReturnType.SINGLE) - PricingInner update(String pricingName, PricingInner pricing); + void delete(String scopeId, String pricingName); + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param filter OData filter. Optional. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listWithResponse(String scopeId, String filter, Context context); + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + PricingListInner list(String scopeId); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceAssessmentsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceAssessmentsClient.java index fb3daa044da5a..36f7fb66f1aee 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceAssessmentsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceAssessmentsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.RegulatoryComplianceAssessmentInner; -/** An instance of this class provides access to all the operations defined in RegulatoryComplianceAssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in RegulatoryComplianceAssessmentsClient. + */ public interface RegulatoryComplianceAssessmentsClient { /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -24,12 +26,12 @@ public interface RegulatoryComplianceAssessmentsClient { * @return list of regulatory compliance assessment response as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName); + PagedIterable list(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName); /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param filter OData filter. Optional. @@ -40,15 +42,12 @@ PagedIterable list( * @return list of regulatory compliance assessment response as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String filter, - Context context); + PagedIterable list(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String filter, Context context); /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -59,15 +58,12 @@ PagedIterable list( * @return regulatory compliance assessment details and state along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName, - Context context); + Response getWithResponse(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName, Context context); /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -77,8 +73,6 @@ Response getWithResponse( * @return regulatory compliance assessment details and state. */ @ServiceMethod(returns = ReturnType.SINGLE) - RegulatoryComplianceAssessmentInner get( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName); + RegulatoryComplianceAssessmentInner get(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceControlsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceControlsClient.java index d7f732d0cdb62..d5da21d079244 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceControlsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceControlsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.RegulatoryComplianceControlInner; -/** An instance of this class provides access to all the operations defined in RegulatoryComplianceControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in RegulatoryComplianceControlsClient. + */ public interface RegulatoryComplianceControlsClient { /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -27,7 +29,7 @@ public interface RegulatoryComplianceControlsClient { /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param filter OData filter. Optional. * @param context The context to associate with this operation. @@ -37,12 +39,12 @@ public interface RegulatoryComplianceControlsClient { * @return list of regulatory compliance controls response as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list( - String regulatoryComplianceStandardName, String filter, Context context); + PagedIterable list(String regulatoryComplianceStandardName, String filter, + Context context); /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param context The context to associate with this operation. @@ -52,12 +54,12 @@ PagedIterable list( * @return regulatory compliance control details and state along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName, Context context); + Response getWithResponse(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, Context context); /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -66,6 +68,6 @@ Response getWithResponse( * @return regulatory compliance control details and state. */ @ServiceMethod(returns = ReturnType.SINGLE) - RegulatoryComplianceControlInner get( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName); + RegulatoryComplianceControlInner get(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceStandardsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceStandardsClient.java index 8bb55a3a903c3..598e1cdedbd42 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceStandardsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/RegulatoryComplianceStandardsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.RegulatoryComplianceStandardInner; -/** An instance of this class provides access to all the operations defined in RegulatoryComplianceStandardsClient. */ +/** + * An instance of this class provides access to all the operations defined in RegulatoryComplianceStandardsClient. + */ public interface RegulatoryComplianceStandardsClient { /** * Supported regulatory compliance standards details and state. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response as paginated response with {@link PagedIterable}. @@ -25,7 +27,7 @@ public interface RegulatoryComplianceStandardsClient { /** * Supported regulatory compliance standards details and state. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -38,7 +40,7 @@ public interface RegulatoryComplianceStandardsClient { /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -47,12 +49,12 @@ public interface RegulatoryComplianceStandardsClient { * @return regulatory compliance standard details and state along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String regulatoryComplianceStandardName, Context context); + Response getWithResponse(String regulatoryComplianceStandardName, + Context context); /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppResourceRegionInfosClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ResourceProvidersClient.java similarity index 51% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppResourceRegionInfosClient.java rename to sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ResourceProvidersClient.java index d1077d2c2671c..ee02cca57a35a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/fluent/NetAppResourceRegionInfosClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ResourceProvidersClient.java @@ -2,74 +2,64 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.fluent; +package com.azure.resourcemanager.security.fluent; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceMethod; -import com.azure.core.http.rest.PagedIterable; import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner; +import com.azure.resourcemanager.security.models.UpdateSensitivitySettingsRequest; /** - * An instance of this class provides access to all the operations defined in NetAppResourceRegionInfosClient. + * An instance of this class provides access to all the operations defined in ResourceProvidersClient. */ -public interface NetAppResourceRegionInfosClient { +public interface ResourceProvidersClient { /** - * Describes region specific information. + * Updates data sensitivity settings for sensitive data discovery. * - * Provides region specific information. - * - * @param location The name of the Azure region. + * @param sensitivitySettings The data sensitivity settings to update. + * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources as paginated response with {@link PagedIterable}. + * @return data sensitivity settings for sensitive data discovery along with {@link Response}. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String location); + @ServiceMethod(returns = ReturnType.SINGLE) + Response + updateSensitivitySettingsWithResponse(UpdateSensitivitySettingsRequest sensitivitySettings, Context context); /** - * Describes region specific information. + * Updates data sensitivity settings for sensitive data discovery. * - * Provides region specific information. - * - * @param location The name of the Azure region. - * @param context The context to associate with this operation. + * @param sensitivitySettings The data sensitivity settings to update. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of regionInfo resources as paginated response with {@link PagedIterable}. + * @return data sensitivity settings for sensitive data discovery. */ - @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable list(String location, Context context); + @ServiceMethod(returns = ReturnType.SINGLE) + GetSensitivitySettingsResponseInner updateSensitivitySettings(UpdateSensitivitySettingsRequest sensitivitySettings); /** - * Describes region specific information. - * - * Provides storage to network proximity and logical zone mapping information. + * Gets data sensitivity settings for sensitive data discovery. * - * @param location The name of the Azure region. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item along with {@link Response}. + * @return data sensitivity settings for sensitive data discovery along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String location, Context context); + Response getSensitivitySettingsWithResponse(Context context); /** - * Describes region specific information. + * Gets data sensitivity settings for sensitive data discovery. * - * Provides storage to network proximity and logical zone mapping information. - * - * @param location The name of the Azure region. - * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return information regarding regionInfo Item. + * @return data sensitivity settings for sensitive data discovery. */ @ServiceMethod(returns = ReturnType.SINGLE) - RegionInfoResourceInner get(String location); + GetSensitivitySettingsResponseInner getSensitivitySettings(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlDefinitionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlDefinitionsClient.java index 8b27a45f14e14..3997565dc2f1b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlDefinitionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlDefinitionsClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecureScoreControlDefinitionItemInner; -/** An instance of this class provides access to all the operations defined in SecureScoreControlDefinitionsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecureScoreControlDefinitionsClient. + */ public interface SecureScoreControlDefinitionsClient { /** * List the available security controls, their assessments, and the max score. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security controls definition as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface SecureScoreControlDefinitionsClient { /** * List the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -36,7 +38,7 @@ public interface SecureScoreControlDefinitionsClient { /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security controls definition as paginated response with {@link PagedIterable}. @@ -46,7 +48,7 @@ public interface SecureScoreControlDefinitionsClient { /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlsClient.java index 43be2302b12d4..feb333230160a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoreControlsClient.java @@ -11,42 +11,44 @@ import com.azure.resourcemanager.security.fluent.models.SecureScoreControlDetailsInner; import com.azure.resourcemanager.security.models.ExpandControlsEnum; -/** An instance of this class provides access to all the operations defined in SecureScoreControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecureScoreControlsClient. + */ public interface SecureScoreControlsClient { /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedIterable}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listBySecureScore(String secureScoreName); /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedIterable}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listBySecureScore( - String secureScoreName, ExpandControlsEnum expand, Context context); + PagedIterable listBySecureScore(String secureScoreName, ExpandControlsEnum expand, + Context context); /** * Get all security controls within a scope. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all security controls within a scope as paginated response with {@link PagedIterable}. @@ -56,7 +58,7 @@ PagedIterable listBySecureScore( /** * Get all security controls within a scope. - * + * * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoresClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoresClient.java index 777ef62405f00..6ae5d40458ad5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoresClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecureScoresClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecureScoreItemInner; -/** An instance of this class provides access to all the operations defined in SecureScoresClient. */ +/** + * An instance of this class provides access to all the operations defined in SecureScoresClient. + */ public interface SecureScoresClient { /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of secure scores as paginated response with {@link PagedIterable}. @@ -25,7 +27,7 @@ public interface SecureScoresClient { /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -38,15 +40,15 @@ public interface SecureScoresClient { /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return secure score for a specific Microsoft Defender for Cloud initiative within your current scope along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) Response getWithResponse(String secureScoreName, Context context); @@ -54,9 +56,9 @@ public interface SecureScoresClient { /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityCenter.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityCenter.java index 3632a9b918151..e559f60d43ec3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityCenter.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityCenter.java @@ -7,460 +7,532 @@ import com.azure.core.http.HttpPipeline; import java.time.Duration; -/** The interface for SecurityCenter class. */ +/** + * The interface for SecurityCenter class. + */ public interface SecurityCenter { /** * Gets Azure subscription ID. - * + * * @return the subscriptionId value. */ String getSubscriptionId(); /** * Gets server parameter. - * + * * @return the endpoint value. */ String getEndpoint(); /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ HttpPipeline getHttpPipeline(); /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ Duration getDefaultPollInterval(); /** - * Gets the MdeOnboardingsClient object to access its operations. - * - * @return the MdeOnboardingsClient object. + * Gets the LocationsClient object to access its operations. + * + * @return the LocationsClient object. */ - MdeOnboardingsClient getMdeOnboardings(); + LocationsClient getLocations(); /** - * Gets the CustomAssessmentAutomationsClient object to access its operations. - * - * @return the CustomAssessmentAutomationsClient object. + * Gets the OperationsClient object to access its operations. + * + * @return the OperationsClient object. */ - CustomAssessmentAutomationsClient getCustomAssessmentAutomations(); + OperationsClient getOperations(); /** - * Gets the CustomEntityStoreAssignmentsClient object to access its operations. - * - * @return the CustomEntityStoreAssignmentsClient object. + * Gets the TasksClient object to access its operations. + * + * @return the TasksClient object. */ - CustomEntityStoreAssignmentsClient getCustomEntityStoreAssignments(); + TasksClient getTasks(); /** - * Gets the ComplianceResultsClient object to access its operations. - * - * @return the ComplianceResultsClient object. + * Gets the AutoProvisioningSettingsClient object to access its operations. + * + * @return the AutoProvisioningSettingsClient object. */ - ComplianceResultsClient getComplianceResults(); + AutoProvisioningSettingsClient getAutoProvisioningSettings(); /** - * Gets the PricingsClient object to access its operations. - * - * @return the PricingsClient object. + * Gets the CompliancesClient object to access its operations. + * + * @return the CompliancesClient object. */ - PricingsClient getPricings(); + CompliancesClient getCompliances(); /** - * Gets the AdvancedThreatProtectionsClient object to access its operations. - * - * @return the AdvancedThreatProtectionsClient object. + * Gets the InformationProtectionPoliciesClient object to access its operations. + * + * @return the InformationProtectionPoliciesClient object. */ - AdvancedThreatProtectionsClient getAdvancedThreatProtections(); + InformationProtectionPoliciesClient getInformationProtectionPolicies(); /** - * Gets the DeviceSecurityGroupsClient object to access its operations. - * - * @return the DeviceSecurityGroupsClient object. + * Gets the WorkspaceSettingsClient object to access its operations. + * + * @return the WorkspaceSettingsClient object. */ - DeviceSecurityGroupsClient getDeviceSecurityGroups(); + WorkspaceSettingsClient getWorkspaceSettings(); /** - * Gets the IotSecuritySolutionsClient object to access its operations. - * - * @return the IotSecuritySolutionsClient object. + * Gets the AlertsSuppressionRulesClient object to access its operations. + * + * @return the AlertsSuppressionRulesClient object. */ - IotSecuritySolutionsClient getIotSecuritySolutions(); + AlertsSuppressionRulesClient getAlertsSuppressionRules(); /** - * Gets the IotSecuritySolutionAnalyticsClient object to access its operations. - * - * @return the IotSecuritySolutionAnalyticsClient object. + * Gets the AutomationsClient object to access its operations. + * + * @return the AutomationsClient object. */ - IotSecuritySolutionAnalyticsClient getIotSecuritySolutionAnalytics(); + AutomationsClient getAutomations(); /** - * Gets the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object to access its operations. - * - * @return the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object. + * Gets the RegulatoryComplianceStandardsClient object to access its operations. + * + * @return the RegulatoryComplianceStandardsClient object. */ - IotSecuritySolutionsAnalyticsAggregatedAlertsClient getIotSecuritySolutionsAnalyticsAggregatedAlerts(); + RegulatoryComplianceStandardsClient getRegulatoryComplianceStandards(); /** - * Gets the IotSecuritySolutionsAnalyticsRecommendationsClient object to access its operations. - * - * @return the IotSecuritySolutionsAnalyticsRecommendationsClient object. + * Gets the RegulatoryComplianceControlsClient object to access its operations. + * + * @return the RegulatoryComplianceControlsClient object. */ - IotSecuritySolutionsAnalyticsRecommendationsClient getIotSecuritySolutionsAnalyticsRecommendations(); + RegulatoryComplianceControlsClient getRegulatoryComplianceControls(); /** - * Gets the LocationsClient object to access its operations. - * - * @return the LocationsClient object. + * Gets the RegulatoryComplianceAssessmentsClient object to access its operations. + * + * @return the RegulatoryComplianceAssessmentsClient object. */ - LocationsClient getLocations(); + RegulatoryComplianceAssessmentsClient getRegulatoryComplianceAssessments(); /** - * Gets the OperationsClient object to access its operations. - * - * @return the OperationsClient object. + * Gets the SubAssessmentsClient object to access its operations. + * + * @return the SubAssessmentsClient object. */ - OperationsClient getOperations(); + SubAssessmentsClient getSubAssessments(); /** - * Gets the TasksClient object to access its operations. - * - * @return the TasksClient object. + * Gets the ConnectorsClient object to access its operations. + * + * @return the ConnectorsClient object. */ - TasksClient getTasks(); + ConnectorsClient getConnectors(); /** - * Gets the AutoProvisioningSettingsClient object to access its operations. - * - * @return the AutoProvisioningSettingsClient object. + * Gets the SecurityContactsClient object to access its operations. + * + * @return the SecurityContactsClient object. */ - AutoProvisioningSettingsClient getAutoProvisioningSettings(); + SecurityContactsClient getSecurityContacts(); /** - * Gets the CompliancesClient object to access its operations. - * - * @return the CompliancesClient object. + * Gets the SoftwareInventoriesClient object to access its operations. + * + * @return the SoftwareInventoriesClient object. */ - CompliancesClient getCompliances(); + SoftwareInventoriesClient getSoftwareInventories(); /** - * Gets the InformationProtectionPoliciesClient object to access its operations. - * - * @return the InformationProtectionPoliciesClient object. + * Gets the CustomAssessmentAutomationsClient object to access its operations. + * + * @return the CustomAssessmentAutomationsClient object. */ - InformationProtectionPoliciesClient getInformationProtectionPolicies(); + CustomAssessmentAutomationsClient getCustomAssessmentAutomations(); /** - * Gets the SecurityContactsClient object to access its operations. - * - * @return the SecurityContactsClient object. + * Gets the CustomEntityStoreAssignmentsClient object to access its operations. + * + * @return the CustomEntityStoreAssignmentsClient object. */ - SecurityContactsClient getSecurityContacts(); + CustomEntityStoreAssignmentsClient getCustomEntityStoreAssignments(); /** - * Gets the WorkspaceSettingsClient object to access its operations. - * - * @return the WorkspaceSettingsClient object. + * Gets the MdeOnboardingsClient object to access its operations. + * + * @return the MdeOnboardingsClient object. */ - WorkspaceSettingsClient getWorkspaceSettings(); + MdeOnboardingsClient getMdeOnboardings(); /** - * Gets the RegulatoryComplianceStandardsClient object to access its operations. - * - * @return the RegulatoryComplianceStandardsClient object. + * Gets the GovernanceAssignmentsClient object to access its operations. + * + * @return the GovernanceAssignmentsClient object. */ - RegulatoryComplianceStandardsClient getRegulatoryComplianceStandards(); + GovernanceAssignmentsClient getGovernanceAssignments(); /** - * Gets the RegulatoryComplianceControlsClient object to access its operations. - * - * @return the RegulatoryComplianceControlsClient object. + * Gets the GovernanceRulesClient object to access its operations. + * + * @return the GovernanceRulesClient object. */ - RegulatoryComplianceControlsClient getRegulatoryComplianceControls(); + GovernanceRulesClient getGovernanceRules(); /** - * Gets the RegulatoryComplianceAssessmentsClient object to access its operations. - * - * @return the RegulatoryComplianceAssessmentsClient object. + * Gets the ApplicationsClient object to access its operations. + * + * @return the ApplicationsClient object. */ - RegulatoryComplianceAssessmentsClient getRegulatoryComplianceAssessments(); + ApplicationsClient getApplications(); /** - * Gets the SubAssessmentsClient object to access its operations. - * - * @return the SubAssessmentsClient object. + * Gets the ApplicationOperationsClient object to access its operations. + * + * @return the ApplicationOperationsClient object. */ - SubAssessmentsClient getSubAssessments(); + ApplicationOperationsClient getApplicationOperations(); /** - * Gets the AutomationsClient object to access its operations. - * - * @return the AutomationsClient object. + * Gets the SecurityConnectorApplicationsClient object to access its operations. + * + * @return the SecurityConnectorApplicationsClient object. */ - AutomationsClient getAutomations(); + SecurityConnectorApplicationsClient getSecurityConnectorApplications(); /** - * Gets the AlertsSuppressionRulesClient object to access its operations. - * - * @return the AlertsSuppressionRulesClient object. + * Gets the SecurityConnectorApplicationOperationsClient object to access its operations. + * + * @return the SecurityConnectorApplicationOperationsClient object. */ - AlertsSuppressionRulesClient getAlertsSuppressionRules(); + SecurityConnectorApplicationOperationsClient getSecurityConnectorApplicationOperations(); /** - * Gets the ServerVulnerabilityAssessmentsClient object to access its operations. - * - * @return the ServerVulnerabilityAssessmentsClient object. + * Gets the DefenderForStoragesClient object to access its operations. + * + * @return the DefenderForStoragesClient object. */ - ServerVulnerabilityAssessmentsClient getServerVulnerabilityAssessments(); + DefenderForStoragesClient getDefenderForStorages(); /** - * Gets the AssessmentsMetadatasClient object to access its operations. - * - * @return the AssessmentsMetadatasClient object. + * Gets the SecurityOperatorsClient object to access its operations. + * + * @return the SecurityOperatorsClient object. */ - AssessmentsMetadatasClient getAssessmentsMetadatas(); + SecurityOperatorsClient getSecurityOperators(); /** - * Gets the AssessmentsClient object to access its operations. - * - * @return the AssessmentsClient object. + * Gets the SqlVulnerabilityAssessmentBaselineRulesClient object to access its operations. + * + * @return the SqlVulnerabilityAssessmentBaselineRulesClient object. */ - AssessmentsClient getAssessments(); + SqlVulnerabilityAssessmentBaselineRulesClient getSqlVulnerabilityAssessmentBaselineRules(); /** - * Gets the AdaptiveApplicationControlsClient object to access its operations. - * - * @return the AdaptiveApplicationControlsClient object. + * Gets the SqlVulnerabilityAssessmentScansClient object to access its operations. + * + * @return the SqlVulnerabilityAssessmentScansClient object. */ - AdaptiveApplicationControlsClient getAdaptiveApplicationControls(); + SqlVulnerabilityAssessmentScansClient getSqlVulnerabilityAssessmentScans(); /** - * Gets the AdaptiveNetworkHardeningsClient object to access its operations. - * - * @return the AdaptiveNetworkHardeningsClient object. + * Gets the SqlVulnerabilityAssessmentScanResultsClient object to access its operations. + * + * @return the SqlVulnerabilityAssessmentScanResultsClient object. */ - AdaptiveNetworkHardeningsClient getAdaptiveNetworkHardenings(); + SqlVulnerabilityAssessmentScanResultsClient getSqlVulnerabilityAssessmentScanResults(); /** - * Gets the AllowedConnectionsClient object to access its operations. - * - * @return the AllowedConnectionsClient object. + * Gets the ResourceProvidersClient object to access its operations. + * + * @return the ResourceProvidersClient object. */ - AllowedConnectionsClient getAllowedConnections(); + ResourceProvidersClient getResourceProviders(); /** - * Gets the TopologiesClient object to access its operations. - * - * @return the TopologiesClient object. + * Gets the SensitivitySettingsClient object to access its operations. + * + * @return the SensitivitySettingsClient object. */ - TopologiesClient getTopologies(); + SensitivitySettingsClient getSensitivitySettings(); /** - * Gets the JitNetworkAccessPoliciesClient object to access its operations. - * - * @return the JitNetworkAccessPoliciesClient object. + * Gets the HealthReportsClient object to access its operations. + * + * @return the HealthReportsClient object. */ - JitNetworkAccessPoliciesClient getJitNetworkAccessPolicies(); + HealthReportsClient getHealthReports(); /** - * Gets the DiscoveredSecuritySolutionsClient object to access its operations. - * - * @return the DiscoveredSecuritySolutionsClient object. + * Gets the AzureDevOpsOrgsClient object to access its operations. + * + * @return the AzureDevOpsOrgsClient object. */ - DiscoveredSecuritySolutionsClient getDiscoveredSecuritySolutions(); + AzureDevOpsOrgsClient getAzureDevOpsOrgs(); /** - * Gets the SecuritySolutionsReferenceDatasClient object to access its operations. - * - * @return the SecuritySolutionsReferenceDatasClient object. + * Gets the AzureDevOpsProjectsClient object to access its operations. + * + * @return the AzureDevOpsProjectsClient object. */ - SecuritySolutionsReferenceDatasClient getSecuritySolutionsReferenceDatas(); + AzureDevOpsProjectsClient getAzureDevOpsProjects(); /** - * Gets the ExternalSecuritySolutionsClient object to access its operations. - * - * @return the ExternalSecuritySolutionsClient object. + * Gets the AzureDevOpsReposClient object to access its operations. + * + * @return the AzureDevOpsReposClient object. */ - ExternalSecuritySolutionsClient getExternalSecuritySolutions(); + AzureDevOpsReposClient getAzureDevOpsRepos(); /** - * Gets the SecureScoresClient object to access its operations. - * - * @return the SecureScoresClient object. + * Gets the DevOpsConfigurationsClient object to access its operations. + * + * @return the DevOpsConfigurationsClient object. */ - SecureScoresClient getSecureScores(); + DevOpsConfigurationsClient getDevOpsConfigurations(); /** - * Gets the SecureScoreControlsClient object to access its operations. - * - * @return the SecureScoreControlsClient object. + * Gets the GitHubOwnersClient object to access its operations. + * + * @return the GitHubOwnersClient object. */ - SecureScoreControlsClient getSecureScoreControls(); + GitHubOwnersClient getGitHubOwners(); /** - * Gets the SecureScoreControlDefinitionsClient object to access its operations. - * - * @return the SecureScoreControlDefinitionsClient object. + * Gets the GitHubReposClient object to access its operations. + * + * @return the GitHubReposClient object. */ - SecureScoreControlDefinitionsClient getSecureScoreControlDefinitions(); + GitHubReposClient getGitHubRepos(); /** - * Gets the SecuritySolutionsClient object to access its operations. - * - * @return the SecuritySolutionsClient object. + * Gets the GitLabGroupsClient object to access its operations. + * + * @return the GitLabGroupsClient object. */ - SecuritySolutionsClient getSecuritySolutions(); + GitLabGroupsClient getGitLabGroups(); /** - * Gets the ConnectorsClient object to access its operations. - * - * @return the ConnectorsClient object. + * Gets the GitLabSubgroupsClient object to access its operations. + * + * @return the GitLabSubgroupsClient object. */ - ConnectorsClient getConnectors(); + GitLabSubgroupsClient getGitLabSubgroups(); /** - * Gets the AlertsClient object to access its operations. - * - * @return the AlertsClient object. + * Gets the GitLabProjectsClient object to access its operations. + * + * @return the GitLabProjectsClient object. */ - AlertsClient getAlerts(); + GitLabProjectsClient getGitLabProjects(); /** - * Gets the SettingsClient object to access its operations. - * - * @return the SettingsClient object. + * Gets the DevOpsOperationResultsClient object to access its operations. + * + * @return the DevOpsOperationResultsClient object. */ - SettingsClient getSettings(); + DevOpsOperationResultsClient getDevOpsOperationResults(); /** - * Gets the IngestionSettingsClient object to access its operations. - * - * @return the IngestionSettingsClient object. + * Gets the SecurityConnectorsClient object to access its operations. + * + * @return the SecurityConnectorsClient object. */ - IngestionSettingsClient getIngestionSettings(); + SecurityConnectorsClient getSecurityConnectors(); /** - * Gets the SoftwareInventoriesClient object to access its operations. - * - * @return the SoftwareInventoriesClient object. + * Gets the ComplianceResultsClient object to access its operations. + * + * @return the ComplianceResultsClient object. */ - SoftwareInventoriesClient getSoftwareInventories(); + ComplianceResultsClient getComplianceResults(); /** - * Gets the GovernanceRulesClient object to access its operations. - * - * @return the GovernanceRulesClient object. + * Gets the AdvancedThreatProtectionsClient object to access its operations. + * + * @return the AdvancedThreatProtectionsClient object. */ - GovernanceRulesClient getGovernanceRules(); + AdvancedThreatProtectionsClient getAdvancedThreatProtections(); /** - * Gets the GovernanceAssignmentsClient object to access its operations. - * - * @return the GovernanceAssignmentsClient object. + * Gets the DeviceSecurityGroupsClient object to access its operations. + * + * @return the DeviceSecurityGroupsClient object. */ - GovernanceAssignmentsClient getGovernanceAssignments(); + DeviceSecurityGroupsClient getDeviceSecurityGroups(); /** - * Gets the ApplicationsClient object to access its operations. - * - * @return the ApplicationsClient object. + * Gets the IotSecuritySolutionAnalyticsClient object to access its operations. + * + * @return the IotSecuritySolutionAnalyticsClient object. */ - ApplicationsClient getApplications(); + IotSecuritySolutionAnalyticsClient getIotSecuritySolutionAnalytics(); /** - * Gets the ApplicationOperationsClient object to access its operations. - * - * @return the ApplicationOperationsClient object. + * Gets the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object to access its operations. + * + * @return the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object. */ - ApplicationOperationsClient getApplicationOperations(); + IotSecuritySolutionsAnalyticsAggregatedAlertsClient getIotSecuritySolutionsAnalyticsAggregatedAlerts(); /** - * Gets the SecurityConnectorApplicationsClient object to access its operations. - * - * @return the SecurityConnectorApplicationsClient object. + * Gets the IotSecuritySolutionsAnalyticsRecommendationsClient object to access its operations. + * + * @return the IotSecuritySolutionsAnalyticsRecommendationsClient object. */ - SecurityConnectorApplicationsClient getSecurityConnectorApplications(); + IotSecuritySolutionsAnalyticsRecommendationsClient getIotSecuritySolutionsAnalyticsRecommendations(); /** - * Gets the SecurityConnectorApplicationOperationsClient object to access its operations. - * - * @return the SecurityConnectorApplicationOperationsClient object. + * Gets the IotSecuritySolutionsClient object to access its operations. + * + * @return the IotSecuritySolutionsClient object. */ - SecurityConnectorApplicationOperationsClient getSecurityConnectorApplicationOperations(); + IotSecuritySolutionsClient getIotSecuritySolutions(); /** - * Gets the ApiCollectionsClient object to access its operations. - * - * @return the ApiCollectionsClient object. + * Gets the AdaptiveNetworkHardeningsClient object to access its operations. + * + * @return the AdaptiveNetworkHardeningsClient object. */ - ApiCollectionsClient getApiCollections(); + AdaptiveNetworkHardeningsClient getAdaptiveNetworkHardenings(); /** - * Gets the ApiCollectionOnboardingsClient object to access its operations. - * - * @return the ApiCollectionOnboardingsClient object. + * Gets the AllowedConnectionsClient object to access its operations. + * + * @return the AllowedConnectionsClient object. */ - ApiCollectionOnboardingsClient getApiCollectionOnboardings(); + AllowedConnectionsClient getAllowedConnections(); /** - * Gets the ApiCollectionOffboardingsClient object to access its operations. - * - * @return the ApiCollectionOffboardingsClient object. + * Gets the AdaptiveApplicationControlsClient object to access its operations. + * + * @return the AdaptiveApplicationControlsClient object. */ - ApiCollectionOffboardingsClient getApiCollectionOffboardings(); + AdaptiveApplicationControlsClient getAdaptiveApplicationControls(); /** - * Gets the HealthReportsClient object to access its operations. - * - * @return the HealthReportsClient object. + * Gets the DiscoveredSecuritySolutionsClient object to access its operations. + * + * @return the DiscoveredSecuritySolutionsClient object. */ - HealthReportsClient getHealthReports(); + DiscoveredSecuritySolutionsClient getDiscoveredSecuritySolutions(); /** - * Gets the HealthReportOperationsClient object to access its operations. - * - * @return the HealthReportOperationsClient object. + * Gets the ExternalSecuritySolutionsClient object to access its operations. + * + * @return the ExternalSecuritySolutionsClient object. */ - HealthReportOperationsClient getHealthReportOperations(); + ExternalSecuritySolutionsClient getExternalSecuritySolutions(); /** - * Gets the SqlVulnerabilityAssessmentScansClient object to access its operations. - * - * @return the SqlVulnerabilityAssessmentScansClient object. + * Gets the JitNetworkAccessPoliciesClient object to access its operations. + * + * @return the JitNetworkAccessPoliciesClient object. */ - SqlVulnerabilityAssessmentScansClient getSqlVulnerabilityAssessmentScans(); + JitNetworkAccessPoliciesClient getJitNetworkAccessPolicies(); /** - * Gets the SqlVulnerabilityAssessmentScanResultsClient object to access its operations. - * - * @return the SqlVulnerabilityAssessmentScanResultsClient object. + * Gets the SecureScoresClient object to access its operations. + * + * @return the SecureScoresClient object. */ - SqlVulnerabilityAssessmentScanResultsClient getSqlVulnerabilityAssessmentScanResults(); + SecureScoresClient getSecureScores(); /** - * Gets the SqlVulnerabilityAssessmentBaselineRulesClient object to access its operations. - * - * @return the SqlVulnerabilityAssessmentBaselineRulesClient object. + * Gets the SecureScoreControlsClient object to access its operations. + * + * @return the SecureScoreControlsClient object. */ - SqlVulnerabilityAssessmentBaselineRulesClient getSqlVulnerabilityAssessmentBaselineRules(); + SecureScoreControlsClient getSecureScoreControls(); /** - * Gets the SecurityConnectorsClient object to access its operations. - * - * @return the SecurityConnectorsClient object. + * Gets the SecureScoreControlDefinitionsClient object to access its operations. + * + * @return the SecureScoreControlDefinitionsClient object. */ - SecurityConnectorsClient getSecurityConnectors(); + SecureScoreControlDefinitionsClient getSecureScoreControlDefinitions(); /** - * Gets the SecurityOperatorsClient object to access its operations. - * - * @return the SecurityOperatorsClient object. + * Gets the SecuritySolutionsClient object to access its operations. + * + * @return the SecuritySolutionsClient object. */ - SecurityOperatorsClient getSecurityOperators(); + SecuritySolutionsClient getSecuritySolutions(); + + /** + * Gets the SecuritySolutionsReferenceDatasClient object to access its operations. + * + * @return the SecuritySolutionsReferenceDatasClient object. + */ + SecuritySolutionsReferenceDatasClient getSecuritySolutionsReferenceDatas(); + + /** + * Gets the ServerVulnerabilityAssessmentsClient object to access its operations. + * + * @return the ServerVulnerabilityAssessmentsClient object. + */ + ServerVulnerabilityAssessmentsClient getServerVulnerabilityAssessments(); + + /** + * Gets the TopologiesClient object to access its operations. + * + * @return the TopologiesClient object. + */ + TopologiesClient getTopologies(); + + /** + * Gets the AssessmentsMetadatasClient object to access its operations. + * + * @return the AssessmentsMetadatasClient object. + */ + AssessmentsMetadatasClient getAssessmentsMetadatas(); + + /** + * Gets the AssessmentsClient object to access its operations. + * + * @return the AssessmentsClient object. + */ + AssessmentsClient getAssessments(); + + /** + * Gets the AlertsClient object to access its operations. + * + * @return the AlertsClient object. + */ + AlertsClient getAlerts(); + + /** + * Gets the SettingsClient object to access its operations. + * + * @return the SettingsClient object. + */ + SettingsClient getSettings(); + + /** + * Gets the ServerVulnerabilityAssessmentsSettingsClient object to access its operations. + * + * @return the ServerVulnerabilityAssessmentsSettingsClient object. + */ + ServerVulnerabilityAssessmentsSettingsClient getServerVulnerabilityAssessmentsSettings(); + + /** + * Gets the ApiCollectionsClient object to access its operations. + * + * @return the ApiCollectionsClient object. + */ + ApiCollectionsClient getApiCollections(); + + /** + * Gets the PricingsClient object to access its operations. + * + * @return the PricingsClient object. + */ + PricingsClient getPricings(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationOperationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationOperationsClient.java index 019e8eefacc01..8a5151b952815 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationOperationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationOperationsClient.java @@ -17,9 +17,9 @@ public interface SecurityConnectorApplicationOperationsClient { /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. @@ -29,14 +29,14 @@ public interface SecurityConnectorApplicationOperationsClient { * @return a specific application for the requested scope by applicationId along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String securityConnectorName, String applicationId, Context context); + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, Context context); /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -49,9 +49,9 @@ Response getWithResponse( /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. @@ -62,18 +62,14 @@ Response getWithResponse( * @return security Application over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String securityConnectorName, - String applicationId, - ApplicationInner application, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, ApplicationInner application, Context context); /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. @@ -83,14 +79,14 @@ Response createOrUpdateWithResponse( * @return security Application over a given scope. */ @ServiceMethod(returns = ReturnType.SINGLE) - ApplicationInner createOrUpdate( - String resourceGroupName, String securityConnectorName, String applicationId, ApplicationInner application); + ApplicationInner createOrUpdate(String resourceGroupName, String securityConnectorName, String applicationId, + ApplicationInner application); /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. @@ -100,14 +96,14 @@ ApplicationInner createOrUpdate( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse( - String resourceGroupName, String securityConnectorName, String applicationId, Context context); + Response deleteWithResponse(String resourceGroupName, String securityConnectorName, String applicationId, + Context context); /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationsClient.java index bafeb6ef94d41..fd9ab063f4d8c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorApplicationsClient.java @@ -10,35 +10,37 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ApplicationInner; -/** An instance of this class provides access to all the operations defined in SecurityConnectorApplicationsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityConnectorApplicationsClient. + */ public interface SecurityConnectorApplicationsClient { /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a security connector level scope as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String resourceGroupName, String securityConnectorName); /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a security connector level scope as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String resourceGroupName, String securityConnectorName, Context context); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorsClient.java index a1a51a3fcf9cb..c213bebdc82b5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityConnectorsClient.java @@ -11,12 +11,14 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecurityConnectorInner; -/** An instance of this class provides access to all the operations defined in SecurityConnectorsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityConnectorsClient. + */ public interface SecurityConnectorsClient { /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security connectors response as paginated response with {@link PagedIterable}. @@ -27,7 +29,7 @@ public interface SecurityConnectorsClient { /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -40,9 +42,9 @@ public interface SecurityConnectorsClient { /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -54,9 +56,9 @@ public interface SecurityConnectorsClient { /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -68,9 +70,9 @@ public interface SecurityConnectorsClient { /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,14 +81,14 @@ public interface SecurityConnectorsClient { * @return the security connector resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getByResourceGroupWithResponse( - String resourceGroupName, String securityConnectorName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String securityConnectorName, Context context); /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -99,9 +101,9 @@ Response getByResourceGroupWithResponse( /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @param context The context to associate with this operation. @@ -111,18 +113,15 @@ Response getByResourceGroupWithResponse( * @return the security connector resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, - String securityConnectorName, - SecurityConnectorInner securityConnector, - Context context); + Response createOrUpdateWithResponse(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector, Context context); /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -131,14 +130,14 @@ Response createOrUpdateWithResponse( * @return the security connector resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SecurityConnectorInner createOrUpdate( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector); + SecurityConnectorInner createOrUpdate(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector); /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @param context The context to associate with this operation. @@ -148,17 +147,14 @@ SecurityConnectorInner createOrUpdate( * @return the security connector resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String resourceGroupName, - String securityConnectorName, - SecurityConnectorInner securityConnector, - Context context); + Response updateWithResponse(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector, Context context); /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -167,14 +163,14 @@ Response updateWithResponse( * @return the security connector resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - SecurityConnectorInner update( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector); + SecurityConnectorInner update(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector); /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -187,9 +183,9 @@ SecurityConnectorInner update( /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityContactsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityContactsClient.java index ab1c6851f948f..ed6a348f7243b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityContactsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityContactsClient.java @@ -10,12 +10,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecurityContactInner; +import com.azure.resourcemanager.security.models.SecurityContactName; -/** An instance of this class provides access to all the operations defined in SecurityContactsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityContactsClient. + */ public interface SecurityContactsClient { /** * List all security contact configurations for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security contacts response as paginated response with {@link PagedIterable}. @@ -25,7 +28,7 @@ public interface SecurityContactsClient { /** * List all security contact configurations for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -37,7 +40,7 @@ public interface SecurityContactsClient { /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -46,11 +49,11 @@ public interface SecurityContactsClient { * @return default Security contact configurations for the subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(String securityContactName, Context context); + Response getWithResponse(SecurityContactName securityContactName, Context context); /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -58,11 +61,11 @@ public interface SecurityContactsClient { * @return default Security contact configurations for the subscription. */ @ServiceMethod(returns = ReturnType.SINGLE) - SecurityContactInner get(String securityContactName); + SecurityContactInner get(SecurityContactName securityContactName); /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @param context The context to associate with this operation. @@ -70,15 +73,15 @@ public interface SecurityContactsClient { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String securityContactName, SecurityContactInner securityContact, Context context); + Response createWithResponse(SecurityContactName securityContactName, + SecurityContactInner securityContact, Context context); /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -87,11 +90,11 @@ Response createWithResponse( * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud. */ @ServiceMethod(returns = ReturnType.SINGLE) - SecurityContactInner create(String securityContactName, SecurityContactInner securityContact); + SecurityContactInner create(SecurityContactName securityContactName, SecurityContactInner securityContact); /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -100,16 +103,16 @@ Response createWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response deleteWithResponse(String securityContactName, Context context); + Response deleteWithResponse(SecurityContactName securityContactName, Context context); /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete(String securityContactName); + void delete(SecurityContactName securityContactName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityOperatorsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityOperatorsClient.java index bd785ec861db9..1971f46dd42ef 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityOperatorsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecurityOperatorsClient.java @@ -11,11 +11,13 @@ import com.azure.resourcemanager.security.fluent.models.SecurityOperatorInner; import com.azure.resourcemanager.security.fluent.models.SecurityOperatorListInner; -/** An instance of this class provides access to all the operations defined in SecurityOperatorsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityOperatorsClient. + */ public interface SecurityOperatorsClient { /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -28,7 +30,7 @@ public interface SecurityOperatorsClient { /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -40,7 +42,7 @@ public interface SecurityOperatorsClient { /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -54,7 +56,7 @@ public interface SecurityOperatorsClient { /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -67,7 +69,7 @@ public interface SecurityOperatorsClient { /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -77,12 +79,12 @@ public interface SecurityOperatorsClient { * @return security operator under a given subscription and pricing along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String pricingName, String securityOperatorName, Context context); + Response createOrUpdateWithResponse(String pricingName, String securityOperatorName, + Context context); /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -95,7 +97,7 @@ Response createOrUpdateWithResponse( /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -109,7 +111,7 @@ Response createOrUpdateWithResponse( /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsClient.java index 172a35c22dd14..f922c9711f74c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsClient.java @@ -11,11 +11,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecuritySolutionInner; -/** An instance of this class provides access to all the operations defined in SecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecuritySolutionsClient. + */ public interface SecuritySolutionsClient { /** * Gets a list of Security Solutions for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Security Solutions for the subscription as paginated response with {@link PagedIterable}. @@ -25,7 +27,7 @@ public interface SecuritySolutionsClient { /** * Gets a list of Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -37,11 +39,11 @@ public interface SecuritySolutionsClient { /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -50,16 +52,16 @@ public interface SecuritySolutionsClient { * @return a specific Security Solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String ascLocation, String securitySolutionName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String securitySolutionName, Context context); /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsReferenceDatasClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsReferenceDatasClient.java index 2e07125883e6c..a9a8c60b23a64 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsReferenceDatasClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SecuritySolutionsReferenceDatasClient.java @@ -10,11 +10,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecuritySolutionsReferenceDataListInner; -/** An instance of this class provides access to all the operations defined in SecuritySolutionsReferenceDatasClient. */ +/** + * An instance of this class provides access to all the operations defined in SecuritySolutionsReferenceDatasClient. + */ public interface SecuritySolutionsReferenceDatasClient { /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -26,7 +28,7 @@ public interface SecuritySolutionsReferenceDatasClient { /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all supported Security Solutions for the subscription. @@ -36,9 +38,9 @@ public interface SecuritySolutionsReferenceDatasClient { /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -50,9 +52,9 @@ public interface SecuritySolutionsReferenceDatasClient { /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SensitivitySettingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SensitivitySettingsClient.java new file mode 100644 index 0000000000000..b8884846a989a --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SensitivitySettingsClient.java @@ -0,0 +1,38 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsListResponseInner; + +/** + * An instance of this class provides access to all the operations defined in SensitivitySettingsClient. + */ +public interface SensitivitySettingsClient { + /** + * Gets a list with a single sensitivity settings resource. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response listWithResponse(Context context); + + /** + * Gets a list with a single sensitivity settings resource. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + GetSensitivitySettingsListResponseInner list(); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsClient.java index 26e9c4152c359..375d8f3879e30 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsClient.java @@ -13,13 +13,15 @@ import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentInner; import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsListInner; -/** An instance of this class provides access to all the operations defined in ServerVulnerabilityAssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ServerVulnerabilityAssessmentsClient. + */ public interface ServerVulnerabilityAssessmentsClient { /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -27,18 +29,18 @@ public interface ServerVulnerabilityAssessmentsClient { * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server vulnerability assessment onboarding statuses on a given resource along with {@link - * Response}. + * @return a list of server vulnerability assessment onboarding statuses on a given resource along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response listByExtendedResourceWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + Response listByExtendedResourceWithResponse(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context); /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -48,14 +50,14 @@ Response listByExtendedResourceWithResp * @return a list of server vulnerability assessment onboarding statuses on a given resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerVulnerabilityAssessmentsListInner listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + ServerVulnerabilityAssessmentsListInner listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName); /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -66,14 +68,14 @@ ServerVulnerabilityAssessmentsListInner listByExtendedResource( * @return a server vulnerability assessment onboarding statuses on a given resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context); /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -83,15 +85,15 @@ Response getWithResponse( * @return a server vulnerability assessment onboarding statuses on a given resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerVulnerabilityAssessmentInner get( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + ServerVulnerabilityAssessmentInner get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName); /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -102,15 +104,15 @@ ServerVulnerabilityAssessmentInner get( * @return describes the server vulnerability assessment details on a resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + Response createOrUpdateWithResponse(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context); /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -120,14 +122,14 @@ Response createOrUpdateWithResponse( * @return describes the server vulnerability assessment details on a resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - ServerVulnerabilityAssessmentInner createOrUpdate( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + ServerVulnerabilityAssessmentInner createOrUpdate(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName); /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -137,14 +139,14 @@ ServerVulnerabilityAssessmentInner createOrUpdate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + SyncPoller, Void> beginDelete(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName); /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -155,14 +157,14 @@ SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - SyncPoller, Void> beginDelete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + SyncPoller, Void> beginDelete(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context); /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -175,9 +177,9 @@ SyncPoller, Void> beginDelete( /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -187,6 +189,6 @@ SyncPoller, Void> beginDelete( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void delete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + void delete(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + Context context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsSettingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsSettingsClient.java new file mode 100644 index 0000000000000..10e3f00b359f5 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/ServerVulnerabilityAssessmentsSettingsClient.java @@ -0,0 +1,124 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent; + +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsSettingInner; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingKindName; + +/** + * An instance of this class provides access to all the operations defined in + * ServerVulnerabilityAssessmentsSettingsClient. + */ +public interface ServerVulnerabilityAssessmentsSettingsClient { + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(); + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + PagedIterable list(Context context); + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription along + * with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response + getWithResponse(ServerVulnerabilityAssessmentsSettingKindName settingKind, Context context); + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ServerVulnerabilityAssessmentsSettingInner get(ServerVulnerabilityAssessmentsSettingKindName settingKind); + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response createOrUpdateWithResponse( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting, Context context); + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + ServerVulnerabilityAssessmentsSettingInner createOrUpdate(ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting); + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + Response deleteWithResponse(ServerVulnerabilityAssessmentsSettingKindName settingKind, Context context); + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + void delete(ServerVulnerabilityAssessmentsSettingKindName settingKind); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SettingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SettingsClient.java index 1d2ca46fd4b9a..18ef6bb2b741e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SettingsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SettingsClient.java @@ -10,13 +10,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SettingInner; -import com.azure.resourcemanager.security.models.SettingName; +import com.azure.resourcemanager.security.models.SettingNameAutoGenerated; -/** An instance of this class provides access to all the operations defined in SettingsClient. */ +/** + * An instance of this class provides access to all the operations defined in SettingsClient. + */ public interface SettingsClient { /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return subscription settings list as paginated response with {@link PagedIterable}. @@ -26,7 +28,7 @@ public interface SettingsClient { /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -38,7 +40,7 @@ public interface SettingsClient { /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -47,11 +49,11 @@ public interface SettingsClient { * @return the kind of the security setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse(SettingName settingName, Context context); + Response getWithResponse(SettingNameAutoGenerated settingName, Context context); /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -59,11 +61,11 @@ public interface SettingsClient { * @return the kind of the security setting. */ @ServiceMethod(returns = ReturnType.SINGLE) - SettingInner get(SettingName settingName); + SettingInner get(SettingNameAutoGenerated settingName); /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @param context The context to associate with this operation. @@ -73,11 +75,12 @@ public interface SettingsClient { * @return the kind of the security setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse(SettingName settingName, SettingInner setting, Context context); + Response updateWithResponse(SettingNameAutoGenerated settingName, SettingInner setting, + Context context); /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -86,5 +89,5 @@ public interface SettingsClient { * @return the kind of the security setting. */ @ServiceMethod(returns = ReturnType.SINGLE) - SettingInner update(SettingName settingName, SettingInner setting); + SettingInner update(SettingNameAutoGenerated settingName, SettingInner setting); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SoftwareInventoriesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SoftwareInventoriesClient.java index 68430027e1942..44c87aa6b3438 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SoftwareInventoriesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SoftwareInventoriesClient.java @@ -11,13 +11,15 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SoftwareInner; -/** An instance of this class provides access to all the operations defined in SoftwareInventoriesClient. */ +/** + * An instance of this class provides access to all the operations defined in SoftwareInventoriesClient. + */ public interface SoftwareInventoriesClient { /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -27,14 +29,14 @@ public interface SoftwareInventoriesClient { * @return the software inventory of the virtual machine as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName); /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -45,38 +47,38 @@ PagedIterable listByExtendedResource( * @return the software inventory of the virtual machine as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context); /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the software inventory of all virtual machines in the subscriptions as paginated response with {@link - * PagedIterable}. + * @return the software inventory of all virtual machines in the subscriptions as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the software inventory of all virtual machines in the subscriptions as paginated response with {@link - * PagedIterable}. + * @return the software inventory of all virtual machines in the subscriptions as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -88,19 +90,14 @@ PagedIterable listByExtendedResource( * @return a single software data of the virtual machine along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName, - Context context); + Response getWithResponse(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String softwareName, Context context); /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -111,10 +108,6 @@ Response getWithResponse( * @return a single software data of the virtual machine. */ @ServiceMethod(returns = ReturnType.SINGLE) - SoftwareInner get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, + SoftwareInner get(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, String softwareName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentBaselineRulesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentBaselineRulesClient.java index 01e57856a5c64..4ddb2758ac869 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentBaselineRulesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentBaselineRulesClient.java @@ -20,7 +20,7 @@ public interface SqlVulnerabilityAssessmentBaselineRulesClient { /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -32,12 +32,12 @@ public interface SqlVulnerabilityAssessmentBaselineRulesClient { * @return rule results along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createOrUpdateWithResponse( - String ruleId, String workspaceId, String resourceId, RuleResultsInput body, Context context); + Response createOrUpdateWithResponse(String ruleId, String workspaceId, String resourceId, + RuleResultsInput body, Context context); /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -51,7 +51,7 @@ Response createOrUpdateWithResponse( /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -66,7 +66,7 @@ Response createOrUpdateWithResponse( /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -80,7 +80,7 @@ Response createOrUpdateWithResponse( /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -95,7 +95,7 @@ Response createOrUpdateWithResponse( /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -108,7 +108,7 @@ Response createOrUpdateWithResponse( /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. @@ -122,7 +122,7 @@ Response createOrUpdateWithResponse( /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -135,7 +135,7 @@ Response createOrUpdateWithResponse( /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param body The baseline rules. @@ -146,12 +146,12 @@ Response createOrUpdateWithResponse( * @return a list of rules results along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response addWithResponse( - String workspaceId, String resourceId, RulesResultsInput body, Context context); + Response addWithResponse(String workspaceId, String resourceId, RulesResultsInput body, + Context context); /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScanResultsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScanResultsClient.java index 9ab7d96a286fc..c669c0462cd7b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScanResultsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScanResultsClient.java @@ -18,7 +18,7 @@ public interface SqlVulnerabilityAssessmentScanResultsClient { /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -30,12 +30,12 @@ public interface SqlVulnerabilityAssessmentScanResultsClient { * @return the scan results of a single rule in a scan record along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String scanId, String scanResultId, String workspaceId, String resourceId, Context context); + Response getWithResponse(String scanId, String scanResultId, String workspaceId, String resourceId, + Context context); /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -50,7 +50,7 @@ Response getWithResponse( /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -65,7 +65,7 @@ Response getWithResponse( /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScansClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScansClient.java index c586949d0f773..d87cdc9a55830 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScansClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SqlVulnerabilityAssessmentScansClient.java @@ -11,11 +11,13 @@ import com.azure.resourcemanager.security.fluent.models.ScanInner; import com.azure.resourcemanager.security.fluent.models.ScansInner; -/** An instance of this class provides access to all the operations defined in SqlVulnerabilityAssessmentScansClient. */ +/** + * An instance of this class provides access to all the operations defined in SqlVulnerabilityAssessmentScansClient. + */ public interface SqlVulnerabilityAssessmentScansClient { /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -30,7 +32,7 @@ public interface SqlVulnerabilityAssessmentScansClient { /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -44,7 +46,7 @@ public interface SqlVulnerabilityAssessmentScansClient { /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. @@ -58,7 +60,7 @@ public interface SqlVulnerabilityAssessmentScansClient { /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SubAssessmentsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SubAssessmentsClient.java index 84dfe1f4ca079..4c3cc7e17d9d7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SubAssessmentsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/SubAssessmentsClient.java @@ -11,73 +11,75 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecuritySubAssessmentInner; -/** An instance of this class provides access to all the operations defined in SubAssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in SubAssessmentsClient. + */ public interface SubAssessmentsClient { /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a subscription scope as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listAll(String scope); /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a subscription scope as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listAll(String scope, Context context); /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security sub-assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope, String assessmentName); /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security sub-assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(String scope, String assessmentName, Context context); /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @param context The context to associate with this operation. @@ -87,14 +89,14 @@ public interface SubAssessmentsClient { * @return a security sub-assessment on your scanned resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String scope, String assessmentName, String subAssessmentName, Context context); + Response getWithResponse(String scope, String assessmentName, String subAssessmentName, + Context context); /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TasksClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TasksClient.java index a2c5a01a49be2..5e45193f7e01e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TasksClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TasksClient.java @@ -12,11 +12,13 @@ import com.azure.resourcemanager.security.fluent.models.SecurityTaskInner; import com.azure.resourcemanager.security.models.TaskUpdateActionType; -/** An instance of this class provides access to all the operations defined in TasksClient. */ +/** + * An instance of this class provides access to all the operations defined in TasksClient. + */ public interface TasksClient { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security task recommendations as paginated response with {@link PagedIterable}. @@ -26,7 +28,7 @@ public interface TasksClient { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -39,9 +41,9 @@ public interface TasksClient { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -52,9 +54,9 @@ public interface TasksClient { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -67,9 +69,9 @@ public interface TasksClient { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -78,14 +80,14 @@ public interface TasksClient { * @return security task that we recommend to do in order to strengthen security along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getSubscriptionLevelTaskWithResponse( - String ascLocation, String taskName, Context context); + Response getSubscriptionLevelTaskWithResponse(String ascLocation, String taskName, + Context context); /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -97,9 +99,9 @@ Response getSubscriptionLevelTaskWithResponse( /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @param context The context to associate with this operation. @@ -109,14 +111,14 @@ Response getSubscriptionLevelTaskWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateSubscriptionLevelTaskStateWithResponse( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType, Context context); + Response updateSubscriptionLevelTaskStateWithResponse(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType, Context context); /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -124,16 +126,16 @@ Response updateSubscriptionLevelTaskStateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void updateSubscriptionLevelTaskState( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType); + void updateSubscriptionLevelTaskState(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType); /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -144,11 +146,11 @@ void updateSubscriptionLevelTaskState( /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -157,16 +159,16 @@ void updateSubscriptionLevelTaskState( * @return list of security task recommendations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - PagedIterable listByResourceGroup( - String resourceGroupName, String ascLocation, String filter, Context context); + PagedIterable listByResourceGroup(String resourceGroupName, String ascLocation, String filter, + Context context); /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -175,16 +177,16 @@ PagedIterable listByResourceGroup( * @return security task that we recommend to do in order to strengthen security along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getResourceGroupLevelTaskWithResponse( - String resourceGroupName, String ascLocation, String taskName, Context context); + Response getResourceGroupLevelTaskWithResponse(String resourceGroupName, String ascLocation, + String taskName, Context context); /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -196,11 +198,11 @@ Response getResourceGroupLevelTaskWithResponse( /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @param context The context to associate with this operation. @@ -210,20 +212,16 @@ Response getResourceGroupLevelTaskWithResponse( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateResourceGroupLevelTaskStateWithResponse( - String resourceGroupName, - String ascLocation, - String taskName, - TaskUpdateActionType taskUpdateActionType, - Context context); + Response updateResourceGroupLevelTaskStateWithResponse(String resourceGroupName, String ascLocation, + String taskName, TaskUpdateActionType taskUpdateActionType, Context context); /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -231,6 +229,6 @@ Response updateResourceGroupLevelTaskStateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - void updateResourceGroupLevelTaskState( - String resourceGroupName, String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType); + void updateResourceGroupLevelTaskState(String resourceGroupName, String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TopologiesClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TopologiesClient.java index 2af153a9fd5d2..b0568bbfcf13b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TopologiesClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/TopologiesClient.java @@ -11,68 +11,70 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.TopologyResourceInner; -/** An instance of this class provides access to all the operations defined in TopologiesClient. */ +/** + * An instance of this class provides access to all the operations defined in TopologiesClient. + */ public interface TopologiesClient { /** * Gets a list that allows to build a topology view of a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription as paginated response with {@link - * PagedIterable}. + * @return a list that allows to build a topology view of a subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(); /** * Gets a list that allows to build a topology view of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription as paginated response with {@link - * PagedIterable}. + * @return a list that allows to build a topology view of a subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable list(Context context); /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation); /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) PagedIterable listByHomeRegion(String ascLocation, Context context); /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -81,16 +83,16 @@ public interface TopologiesClient { * @return a specific topology component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response getWithResponse( - String resourceGroupName, String ascLocation, String topologyResourceName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String topologyResourceName, Context context); /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/WorkspaceSettingsClient.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/WorkspaceSettingsClient.java index 4aeb551884764..246993e8efde6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/WorkspaceSettingsClient.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/WorkspaceSettingsClient.java @@ -11,12 +11,14 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.WorkspaceSettingInner; -/** An instance of this class provides access to all the operations defined in WorkspaceSettingsClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspaceSettingsClient. + */ public interface WorkspaceSettingsClient { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of workspace settings response as paginated response with {@link PagedIterable}. @@ -27,7 +29,7 @@ public interface WorkspaceSettingsClient { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -40,7 +42,7 @@ public interface WorkspaceSettingsClient { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -54,7 +56,7 @@ public interface WorkspaceSettingsClient { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -66,7 +68,7 @@ public interface WorkspaceSettingsClient { /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @param context The context to associate with this operation. @@ -76,12 +78,12 @@ public interface WorkspaceSettingsClient { * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response createWithResponse( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting, Context context); + Response createWithResponse(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting, Context context); /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -94,7 +96,7 @@ Response createWithResponse( /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @param context The context to associate with this operation. @@ -104,12 +106,12 @@ Response createWithResponse( * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - Response updateWithResponse( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting, Context context); + Response updateWithResponse(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting, Context context); /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -122,7 +124,7 @@ Response updateWithResponse( /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -135,7 +137,7 @@ Response updateWithResponse( /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveApplicationControlGroupData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveApplicationControlGroupData.java index de1ccc31bb952..3385d4ff5529e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveApplicationControlGroupData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveApplicationControlGroupData.java @@ -16,7 +16,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Represents a machines group and set of rules to be allowed running on a machine. */ +/** + * Represents a machines group and set of rules to be allowed running on a machine. + */ @Fluent public final class AdaptiveApplicationControlGroupData { /* @@ -68,14 +70,16 @@ public final class AdaptiveApplicationControlGroupData { @JsonProperty(value = "pathRecommendations") private List pathRecommendations; - /** Creates an instance of AdaptiveApplicationControlGroupData class. */ + /** + * Creates an instance of AdaptiveApplicationControlGroupData class. + */ public AdaptiveApplicationControlGroupData() { } /** * Get the enforcementMode property: The application control policy enforcement/protection mode of the machine * group. - * + * * @return the enforcementMode value. */ public EnforcementMode enforcementMode() { @@ -85,7 +89,7 @@ public EnforcementMode enforcementMode() { /** * Set the enforcementMode property: The application control policy enforcement/protection mode of the machine * group. - * + * * @param enforcementMode the enforcementMode value to set. * @return the AdaptiveApplicationControlGroupData object itself. */ @@ -97,7 +101,7 @@ public AdaptiveApplicationControlGroupData withEnforcementMode(EnforcementMode e /** * Get the protectionMode property: The protection mode of the collection/file types. Exe/Msi/Script are used for * Windows, Executable is used for Linux. - * + * * @return the protectionMode value. */ public ProtectionMode protectionMode() { @@ -107,7 +111,7 @@ public ProtectionMode protectionMode() { /** * Set the protectionMode property: The protection mode of the collection/file types. Exe/Msi/Script are used for * Windows, Executable is used for Linux. - * + * * @param protectionMode the protectionMode value to set. * @return the AdaptiveApplicationControlGroupData object itself. */ @@ -118,7 +122,7 @@ public AdaptiveApplicationControlGroupData withProtectionMode(ProtectionMode pro /** * Get the configurationStatus property: The configuration status of the machines group or machine or rule. - * + * * @return the configurationStatus value. */ public ConfigurationStatus configurationStatus() { @@ -127,7 +131,7 @@ public ConfigurationStatus configurationStatus() { /** * Get the recommendationStatus property: The initial recommendation status of the machine group or machine. - * + * * @return the recommendationStatus value. */ public RecommendationStatus recommendationStatus() { @@ -136,7 +140,7 @@ public RecommendationStatus recommendationStatus() { /** * Get the issues property: The issues property. - * + * * @return the issues value. */ public List issues() { @@ -145,7 +149,7 @@ public List issues() { /** * Get the sourceSystem property: The source type of the machine group. - * + * * @return the sourceSystem value. */ public SourceSystem sourceSystem() { @@ -154,7 +158,7 @@ public SourceSystem sourceSystem() { /** * Get the vmRecommendations property: The vmRecommendations property. - * + * * @return the vmRecommendations value. */ public List vmRecommendations() { @@ -163,7 +167,7 @@ public List vmRecommendations() { /** * Set the vmRecommendations property: The vmRecommendations property. - * + * * @param vmRecommendations the vmRecommendations value to set. * @return the AdaptiveApplicationControlGroupData object itself. */ @@ -174,7 +178,7 @@ public AdaptiveApplicationControlGroupData withVmRecommendations(List pathRecommendations() { @@ -183,7 +187,7 @@ public List pathRecommendations() { /** * Set the pathRecommendations property: The pathRecommendations property. - * + * * @param pathRecommendations the pathRecommendations value to set. * @return the AdaptiveApplicationControlGroupData object itself. */ @@ -194,7 +198,7 @@ public AdaptiveApplicationControlGroupData withPathRecommendations(List issues() { @@ -135,7 +139,7 @@ public List issues() { /** * Get the sourceSystem property: The source type of the machine group. - * + * * @return the sourceSystem value. */ public SourceSystem sourceSystem() { @@ -144,7 +148,7 @@ public SourceSystem sourceSystem() { /** * Get the vmRecommendations property: The vmRecommendations property. - * + * * @return the vmRecommendations value. */ public List vmRecommendations() { @@ -153,7 +157,7 @@ public List vmRecommendations() { /** * Set the vmRecommendations property: The vmRecommendations property. - * + * * @param vmRecommendations the vmRecommendations value to set. * @return the AdaptiveApplicationControlGroupInner object itself. */ @@ -167,7 +171,7 @@ public AdaptiveApplicationControlGroupInner withVmRecommendations(List pathRecommendations() { @@ -176,7 +180,7 @@ public List pathRecommendations() { /** * Set the pathRecommendations property: The pathRecommendations property. - * + * * @param pathRecommendations the pathRecommendations value to set. * @return the AdaptiveApplicationControlGroupInner object itself. */ @@ -190,15 +194,13 @@ public AdaptiveApplicationControlGroupInner withPathRecommendations(List value; - /** Creates an instance of AdaptiveApplicationControlGroupsInner class. */ + /** + * Creates an instance of AdaptiveApplicationControlGroupsInner class. + */ public AdaptiveApplicationControlGroupsInner() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -35,7 +37,7 @@ public List value() { /** * Set the value property: The value property. - * + * * @param value the value value to set. * @return the AdaptiveApplicationControlGroupsInner object itself. */ @@ -46,7 +48,7 @@ public AdaptiveApplicationControlGroupsInner withValue(List rules() { @@ -45,7 +49,7 @@ public List rules() { /** * Set the rules property: The security rules which are recommended to be effective on the VM. - * + * * @param rules the rules value to set. * @return the AdaptiveNetworkHardeningInner object itself. */ @@ -59,7 +63,7 @@ public AdaptiveNetworkHardeningInner withRules(List rules) { /** * Get the rulesCalculationTime property: The UTC time on which the rules were calculated. - * + * * @return the rulesCalculationTime value. */ public OffsetDateTime rulesCalculationTime() { @@ -68,7 +72,7 @@ public OffsetDateTime rulesCalculationTime() { /** * Set the rulesCalculationTime property: The UTC time on which the rules were calculated. - * + * * @param rulesCalculationTime the rulesCalculationTime value to set. * @return the AdaptiveNetworkHardeningInner object itself. */ @@ -83,7 +87,7 @@ public AdaptiveNetworkHardeningInner withRulesCalculationTime(OffsetDateTime rul /** * Get the effectiveNetworkSecurityGroups property: The Network Security Groups effective on the network interfaces * of the protected resource. - * + * * @return the effectiveNetworkSecurityGroups value. */ public List effectiveNetworkSecurityGroups() { @@ -93,12 +97,12 @@ public List effectiveNetworkSecurityGroups() { /** * Set the effectiveNetworkSecurityGroups property: The Network Security Groups effective on the network interfaces * of the protected resource. - * + * * @param effectiveNetworkSecurityGroups the effectiveNetworkSecurityGroups value to set. * @return the AdaptiveNetworkHardeningInner object itself. */ - public AdaptiveNetworkHardeningInner withEffectiveNetworkSecurityGroups( - List effectiveNetworkSecurityGroups) { + public AdaptiveNetworkHardeningInner + withEffectiveNetworkSecurityGroups(List effectiveNetworkSecurityGroups) { if (this.innerProperties() == null) { this.innerProperties = new AdaptiveNetworkHardeningProperties(); } @@ -108,7 +112,7 @@ public AdaptiveNetworkHardeningInner withEffectiveNetworkSecurityGroups( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveNetworkHardeningProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveNetworkHardeningProperties.java index d674845c04923..c17319b56de30 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveNetworkHardeningProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdaptiveNetworkHardeningProperties.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Adaptive Network Hardening resource properties. */ +/** + * Adaptive Network Hardening resource properties. + */ @Fluent public final class AdaptiveNetworkHardeningProperties { /* @@ -32,13 +34,15 @@ public final class AdaptiveNetworkHardeningProperties { @JsonProperty(value = "effectiveNetworkSecurityGroups") private List effectiveNetworkSecurityGroups; - /** Creates an instance of AdaptiveNetworkHardeningProperties class. */ + /** + * Creates an instance of AdaptiveNetworkHardeningProperties class. + */ public AdaptiveNetworkHardeningProperties() { } /** * Get the rules property: The security rules which are recommended to be effective on the VM. - * + * * @return the rules value. */ public List rules() { @@ -47,7 +51,7 @@ public List rules() { /** * Set the rules property: The security rules which are recommended to be effective on the VM. - * + * * @param rules the rules value to set. * @return the AdaptiveNetworkHardeningProperties object itself. */ @@ -58,7 +62,7 @@ public AdaptiveNetworkHardeningProperties withRules(List rules) { /** * Get the rulesCalculationTime property: The UTC time on which the rules were calculated. - * + * * @return the rulesCalculationTime value. */ public OffsetDateTime rulesCalculationTime() { @@ -67,7 +71,7 @@ public OffsetDateTime rulesCalculationTime() { /** * Set the rulesCalculationTime property: The UTC time on which the rules were calculated. - * + * * @param rulesCalculationTime the rulesCalculationTime value to set. * @return the AdaptiveNetworkHardeningProperties object itself. */ @@ -79,7 +83,7 @@ public AdaptiveNetworkHardeningProperties withRulesCalculationTime(OffsetDateTim /** * Get the effectiveNetworkSecurityGroups property: The Network Security Groups effective on the network interfaces * of the protected resource. - * + * * @return the effectiveNetworkSecurityGroups value. */ public List effectiveNetworkSecurityGroups() { @@ -89,19 +93,19 @@ public List effectiveNetworkSecurityGroups() { /** * Set the effectiveNetworkSecurityGroups property: The Network Security Groups effective on the network interfaces * of the protected resource. - * + * * @param effectiveNetworkSecurityGroups the effectiveNetworkSecurityGroups value to set. * @return the AdaptiveNetworkHardeningProperties object itself. */ - public AdaptiveNetworkHardeningProperties withEffectiveNetworkSecurityGroups( - List effectiveNetworkSecurityGroups) { + public AdaptiveNetworkHardeningProperties + withEffectiveNetworkSecurityGroups(List effectiveNetworkSecurityGroups) { this.effectiveNetworkSecurityGroups = effectiveNetworkSecurityGroups; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionProperties.java index 3255fbeb3048b..76167c2ac2c9a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Advanced Threat Protection settings. */ +/** + * The Advanced Threat Protection settings. + */ @Fluent public final class AdvancedThreatProtectionProperties { /* @@ -16,13 +18,15 @@ public final class AdvancedThreatProtectionProperties { @JsonProperty(value = "isEnabled") private Boolean isEnabled; - /** Creates an instance of AdvancedThreatProtectionProperties class. */ + /** + * Creates an instance of AdvancedThreatProtectionProperties class. + */ public AdvancedThreatProtectionProperties() { } /** * Get the isEnabled property: Indicates whether Advanced Threat Protection is enabled. - * + * * @return the isEnabled value. */ public Boolean isEnabled() { @@ -31,7 +35,7 @@ public Boolean isEnabled() { /** * Set the isEnabled property: Indicates whether Advanced Threat Protection is enabled. - * + * * @param isEnabled the isEnabled value to set. * @return the AdvancedThreatProtectionProperties object itself. */ @@ -42,7 +46,7 @@ public AdvancedThreatProtectionProperties withIsEnabled(Boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionSettingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionSettingInner.java index fb63249f6d372..54ca2c870ac17 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionSettingInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AdvancedThreatProtectionSettingInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.ProxyResource; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Advanced Threat Protection resource. */ +/** + * The Advanced Threat Protection resource. + */ @Fluent public final class AdvancedThreatProtectionSettingInner extends ProxyResource { /* @@ -17,13 +19,15 @@ public final class AdvancedThreatProtectionSettingInner extends ProxyResource { @JsonProperty(value = "properties") private AdvancedThreatProtectionProperties innerProperties; - /** Creates an instance of AdvancedThreatProtectionSettingInner class. */ + /** + * Creates an instance of AdvancedThreatProtectionSettingInner class. + */ public AdvancedThreatProtectionSettingInner() { } /** * Get the innerProperties property: The Advanced Threat Protection settings. - * + * * @return the innerProperties value. */ private AdvancedThreatProtectionProperties innerProperties() { @@ -32,7 +36,7 @@ private AdvancedThreatProtectionProperties innerProperties() { /** * Get the isEnabled property: Indicates whether Advanced Threat Protection is enabled. - * + * * @return the isEnabled value. */ public Boolean isEnabled() { @@ -41,7 +45,7 @@ public Boolean isEnabled() { /** * Set the isEnabled property: Indicates whether Advanced Threat Protection is enabled. - * + * * @param isEnabled the isEnabled value to set. * @return the AdvancedThreatProtectionSettingInner object itself. */ @@ -55,7 +59,7 @@ public AdvancedThreatProtectionSettingInner withIsEnabled(Boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertInner.java index ac8fd6624bd45..1280f5d3ce195 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertInner.java @@ -17,7 +17,9 @@ import java.util.List; import java.util.Map; -/** Security alert. */ +/** + * Security alert. + */ @Fluent public final class AlertInner extends ProxyResource { /* @@ -26,13 +28,15 @@ public final class AlertInner extends ProxyResource { @JsonProperty(value = "properties") private AlertProperties innerProperties; - /** Creates an instance of AlertInner class. */ + /** + * Creates an instance of AlertInner class. + */ public AlertInner() { } /** * Get the innerProperties property: describes security alert properties. - * + * * @return the innerProperties value. */ private AlertProperties innerProperties() { @@ -41,7 +45,7 @@ private AlertProperties innerProperties() { /** * Get the version property: Schema version. - * + * * @return the version value. */ public String version() { @@ -51,7 +55,7 @@ public String version() { /** * Get the alertType property: Unique identifier for the detection logic (all alert instances from the same * detection logic will have the same alertType). - * + * * @return the alertType value. */ public String alertType() { @@ -60,7 +64,7 @@ public String alertType() { /** * Get the systemAlertId property: Unique identifier for the alert. - * + * * @return the systemAlertId value. */ public String systemAlertId() { @@ -70,7 +74,7 @@ public String systemAlertId() { /** * Get the productComponentName property: The name of Azure Security Center pricing tier which powering this alert. * Learn more: https://docs.microsoft.com/en-us/azure/security-center/security-center-pricing. - * + * * @return the productComponentName value. */ public String productComponentName() { @@ -79,7 +83,7 @@ public String productComponentName() { /** * Get the alertDisplayName property: The display name of the alert. - * + * * @return the alertDisplayName value. */ public String alertDisplayName() { @@ -88,7 +92,7 @@ public String alertDisplayName() { /** * Get the description property: Description of the suspicious activity that was detected. - * + * * @return the description value. */ public String description() { @@ -98,7 +102,7 @@ public String description() { /** * Get the severity property: The risk level of the threat that was detected. Learn more: * https://docs.microsoft.com/en-us/azure/security-center/security-center-alerts-overview#how-are-alerts-classified. - * + * * @return the severity value. */ public AlertSeverity severity() { @@ -108,7 +112,7 @@ public AlertSeverity severity() { /** * Get the intent property: The kill chain related intent behind the alert. For list of supported values, and * explanations of Azure Security Center's supported kill chain intents. - * + * * @return the intent value. */ public Intent intent() { @@ -118,7 +122,7 @@ public Intent intent() { /** * Get the startTimeUtc property: The UTC time of the first event or activity included in the alert in ISO8601 * format. - * + * * @return the startTimeUtc value. */ public OffsetDateTime startTimeUtc() { @@ -127,7 +131,7 @@ public OffsetDateTime startTimeUtc() { /** * Get the endTimeUtc property: The UTC time of the last event or activity included in the alert in ISO8601 format. - * + * * @return the endTimeUtc value. */ public OffsetDateTime endTimeUtc() { @@ -138,7 +142,7 @@ public OffsetDateTime endTimeUtc() { * Get the resourceIdentifiers property: The resource identifiers that can be used to direct the alert to the right * product exposure group (tenant, workspace, subscription etc.). There can be multiple identifiers of different * type per alert. - * + * * @return the resourceIdentifiers value. */ public List resourceIdentifiers() { @@ -147,7 +151,7 @@ public List resourceIdentifiers() { /** * Get the remediationSteps property: Manual action items to take to remediate the alert. - * + * * @return the remediationSteps value. */ public List remediationSteps() { @@ -156,7 +160,7 @@ public List remediationSteps() { /** * Get the vendorName property: The name of the vendor that raises the alert. - * + * * @return the vendorName value. */ public String vendorName() { @@ -165,7 +169,7 @@ public String vendorName() { /** * Get the status property: The life cycle status of the alert. - * + * * @return the status value. */ public AlertStatus status() { @@ -174,7 +178,7 @@ public AlertStatus status() { /** * Get the extendedLinks property: Links related to the alert. - * + * * @return the extendedLinks value. */ public List> extendedLinks() { @@ -183,7 +187,7 @@ public List> extendedLinks() { /** * Get the alertUri property: A direct link to the alert page in Azure Portal. - * + * * @return the alertUri value. */ public String alertUri() { @@ -192,7 +196,7 @@ public String alertUri() { /** * Get the timeGeneratedUtc property: The UTC time the alert was generated in ISO8601 format. - * + * * @return the timeGeneratedUtc value. */ public OffsetDateTime timeGeneratedUtc() { @@ -203,7 +207,7 @@ public OffsetDateTime timeGeneratedUtc() { * Get the productName property: The name of the product which published this alert (Microsoft Sentinel, Microsoft * Defender for Identity, Microsoft Defender for Endpoint, Microsoft Defender for Office, Microsoft Defender for * Cloud Apps, and so on). - * + * * @return the productName value. */ public String productName() { @@ -212,7 +216,7 @@ public String productName() { /** * Get the processingEndTimeUtc property: The UTC processing end time of the alert in ISO8601 format. - * + * * @return the processingEndTimeUtc value. */ public OffsetDateTime processingEndTimeUtc() { @@ -221,7 +225,7 @@ public OffsetDateTime processingEndTimeUtc() { /** * Get the entities property: A list of entities related to the alert. - * + * * @return the entities value. */ public List entities() { @@ -231,7 +235,7 @@ public List entities() { /** * Get the isIncident property: This field determines whether the alert is an incident (a compound grouping of * several alerts) or a single alert. - * + * * @return the isIncident value. */ public Boolean isIncident() { @@ -241,7 +245,7 @@ public Boolean isIncident() { /** * Get the correlationKey property: Key for corelating related alerts. Alerts with the same correlation key * considered to be related. - * + * * @return the correlationKey value. */ public String correlationKey() { @@ -250,7 +254,7 @@ public String correlationKey() { /** * Get the extendedProperties property: Custom properties for the alert. - * + * * @return the extendedProperties value. */ public Map extendedProperties() { @@ -259,7 +263,7 @@ public Map extendedProperties() { /** * Set the extendedProperties property: Custom properties for the alert. - * + * * @param extendedProperties the extendedProperties value to set. * @return the AlertInner object itself. */ @@ -273,7 +277,7 @@ public AlertInner withExtendedProperties(Map extendedProperties) /** * Get the compromisedEntity property: The display name of the resource most related to this alert. - * + * * @return the compromisedEntity value. */ public String compromisedEntity() { @@ -282,7 +286,7 @@ public String compromisedEntity() { /** * Get the techniques property: kill chain related techniques behind the alert. - * + * * @return the techniques value. */ public List techniques() { @@ -291,7 +295,7 @@ public List techniques() { /** * Get the subTechniques property: Kill chain related sub-techniques behind the alert. - * + * * @return the subTechniques value. */ public List subTechniques() { @@ -300,7 +304,7 @@ public List subTechniques() { /** * Get the supportingEvidence property: Changing set of properties depending on the supportingEvidence type. - * + * * @return the supportingEvidence value. */ public AlertPropertiesSupportingEvidence supportingEvidence() { @@ -309,7 +313,7 @@ public AlertPropertiesSupportingEvidence supportingEvidence() { /** * Set the supportingEvidence property: Changing set of properties depending on the supportingEvidence type. - * + * * @param supportingEvidence the supportingEvidence value to set. * @return the AlertInner object itself. */ @@ -323,7 +327,7 @@ public AlertInner withSupportingEvidence(AlertPropertiesSupportingEvidence suppo /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertProperties.java index 5a65af987b501..5c3d6c4e39493 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertProperties.java @@ -17,7 +17,9 @@ import java.util.List; import java.util.Map; -/** describes security alert properties. */ +/** + * describes security alert properties. + */ @Fluent public final class AlertProperties { /* @@ -190,13 +192,15 @@ public final class AlertProperties { @JsonProperty(value = "supportingEvidence") private AlertPropertiesSupportingEvidence supportingEvidence; - /** Creates an instance of AlertProperties class. */ + /** + * Creates an instance of AlertProperties class. + */ public AlertProperties() { } /** * Get the version property: Schema version. - * + * * @return the version value. */ public String version() { @@ -206,7 +210,7 @@ public String version() { /** * Get the alertType property: Unique identifier for the detection logic (all alert instances from the same * detection logic will have the same alertType). - * + * * @return the alertType value. */ public String alertType() { @@ -215,7 +219,7 @@ public String alertType() { /** * Get the systemAlertId property: Unique identifier for the alert. - * + * * @return the systemAlertId value. */ public String systemAlertId() { @@ -225,7 +229,7 @@ public String systemAlertId() { /** * Get the productComponentName property: The name of Azure Security Center pricing tier which powering this alert. * Learn more: https://docs.microsoft.com/en-us/azure/security-center/security-center-pricing. - * + * * @return the productComponentName value. */ public String productComponentName() { @@ -234,7 +238,7 @@ public String productComponentName() { /** * Get the alertDisplayName property: The display name of the alert. - * + * * @return the alertDisplayName value. */ public String alertDisplayName() { @@ -243,7 +247,7 @@ public String alertDisplayName() { /** * Get the description property: Description of the suspicious activity that was detected. - * + * * @return the description value. */ public String description() { @@ -253,7 +257,7 @@ public String description() { /** * Get the severity property: The risk level of the threat that was detected. Learn more: * https://docs.microsoft.com/en-us/azure/security-center/security-center-alerts-overview#how-are-alerts-classified. - * + * * @return the severity value. */ public AlertSeverity severity() { @@ -263,7 +267,7 @@ public AlertSeverity severity() { /** * Get the intent property: The kill chain related intent behind the alert. For list of supported values, and * explanations of Azure Security Center's supported kill chain intents. - * + * * @return the intent value. */ public Intent intent() { @@ -273,7 +277,7 @@ public Intent intent() { /** * Get the startTimeUtc property: The UTC time of the first event or activity included in the alert in ISO8601 * format. - * + * * @return the startTimeUtc value. */ public OffsetDateTime startTimeUtc() { @@ -282,7 +286,7 @@ public OffsetDateTime startTimeUtc() { /** * Get the endTimeUtc property: The UTC time of the last event or activity included in the alert in ISO8601 format. - * + * * @return the endTimeUtc value. */ public OffsetDateTime endTimeUtc() { @@ -293,7 +297,7 @@ public OffsetDateTime endTimeUtc() { * Get the resourceIdentifiers property: The resource identifiers that can be used to direct the alert to the right * product exposure group (tenant, workspace, subscription etc.). There can be multiple identifiers of different * type per alert. - * + * * @return the resourceIdentifiers value. */ public List resourceIdentifiers() { @@ -302,7 +306,7 @@ public List resourceIdentifiers() { /** * Get the remediationSteps property: Manual action items to take to remediate the alert. - * + * * @return the remediationSteps value. */ public List remediationSteps() { @@ -311,7 +315,7 @@ public List remediationSteps() { /** * Get the vendorName property: The name of the vendor that raises the alert. - * + * * @return the vendorName value. */ public String vendorName() { @@ -320,7 +324,7 @@ public String vendorName() { /** * Get the status property: The life cycle status of the alert. - * + * * @return the status value. */ public AlertStatus status() { @@ -329,7 +333,7 @@ public AlertStatus status() { /** * Get the extendedLinks property: Links related to the alert. - * + * * @return the extendedLinks value. */ public List> extendedLinks() { @@ -338,7 +342,7 @@ public List> extendedLinks() { /** * Get the alertUri property: A direct link to the alert page in Azure Portal. - * + * * @return the alertUri value. */ public String alertUri() { @@ -347,7 +351,7 @@ public String alertUri() { /** * Get the timeGeneratedUtc property: The UTC time the alert was generated in ISO8601 format. - * + * * @return the timeGeneratedUtc value. */ public OffsetDateTime timeGeneratedUtc() { @@ -358,7 +362,7 @@ public OffsetDateTime timeGeneratedUtc() { * Get the productName property: The name of the product which published this alert (Microsoft Sentinel, Microsoft * Defender for Identity, Microsoft Defender for Endpoint, Microsoft Defender for Office, Microsoft Defender for * Cloud Apps, and so on). - * + * * @return the productName value. */ public String productName() { @@ -367,7 +371,7 @@ public String productName() { /** * Get the processingEndTimeUtc property: The UTC processing end time of the alert in ISO8601 format. - * + * * @return the processingEndTimeUtc value. */ public OffsetDateTime processingEndTimeUtc() { @@ -376,7 +380,7 @@ public OffsetDateTime processingEndTimeUtc() { /** * Get the entities property: A list of entities related to the alert. - * + * * @return the entities value. */ public List entities() { @@ -386,7 +390,7 @@ public List entities() { /** * Get the isIncident property: This field determines whether the alert is an incident (a compound grouping of * several alerts) or a single alert. - * + * * @return the isIncident value. */ public Boolean isIncident() { @@ -396,7 +400,7 @@ public Boolean isIncident() { /** * Get the correlationKey property: Key for corelating related alerts. Alerts with the same correlation key * considered to be related. - * + * * @return the correlationKey value. */ public String correlationKey() { @@ -405,7 +409,7 @@ public String correlationKey() { /** * Get the extendedProperties property: Custom properties for the alert. - * + * * @return the extendedProperties value. */ public Map extendedProperties() { @@ -414,7 +418,7 @@ public Map extendedProperties() { /** * Set the extendedProperties property: Custom properties for the alert. - * + * * @param extendedProperties the extendedProperties value to set. * @return the AlertProperties object itself. */ @@ -425,7 +429,7 @@ public AlertProperties withExtendedProperties(Map extendedProper /** * Get the compromisedEntity property: The display name of the resource most related to this alert. - * + * * @return the compromisedEntity value. */ public String compromisedEntity() { @@ -434,7 +438,7 @@ public String compromisedEntity() { /** * Get the techniques property: kill chain related techniques behind the alert. - * + * * @return the techniques value. */ public List techniques() { @@ -443,7 +447,7 @@ public List techniques() { /** * Get the subTechniques property: Kill chain related sub-techniques behind the alert. - * + * * @return the subTechniques value. */ public List subTechniques() { @@ -452,7 +456,7 @@ public List subTechniques() { /** * Get the supportingEvidence property: Changing set of properties depending on the supportingEvidence type. - * + * * @return the supportingEvidence value. */ public AlertPropertiesSupportingEvidence supportingEvidence() { @@ -461,7 +465,7 @@ public AlertPropertiesSupportingEvidence supportingEvidence() { /** * Set the supportingEvidence property: Changing set of properties depending on the supportingEvidence type. - * + * * @param supportingEvidence the supportingEvidence value to set. * @return the AlertProperties object itself. */ @@ -472,7 +476,7 @@ public AlertProperties withSupportingEvidence(AlertPropertiesSupportingEvidence /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertSyncSettingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertSyncSettingProperties.java index fe3186458ebb7..95d1855ae1f9f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertSyncSettingProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertSyncSettingProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The alert sync setting properties. */ +/** + * The alert sync setting properties. + */ @Fluent public final class AlertSyncSettingProperties { /* @@ -16,13 +18,15 @@ public final class AlertSyncSettingProperties { @JsonProperty(value = "enabled", required = true) private boolean enabled; - /** Creates an instance of AlertSyncSettingProperties class. */ + /** + * Creates an instance of AlertSyncSettingProperties class. + */ public AlertSyncSettingProperties() { } /** * Get the enabled property: Is the alert sync setting enabled. - * + * * @return the enabled value. */ public boolean enabled() { @@ -31,7 +35,7 @@ public boolean enabled() { /** * Set the enabled property: Is the alert sync setting enabled. - * + * * @param enabled the enabled value to set. * @return the AlertSyncSettingProperties object itself. */ @@ -42,7 +46,7 @@ public AlertSyncSettingProperties withEnabled(boolean enabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleInner.java index 110197b8998b2..236997fedc029 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Describes the suppression rule. */ +/** + * Describes the suppression rule. + */ @Fluent public final class AlertsSuppressionRuleInner extends ProxyResource { /* @@ -20,13 +22,15 @@ public final class AlertsSuppressionRuleInner extends ProxyResource { @JsonProperty(value = "properties") private AlertsSuppressionRuleProperties innerProperties; - /** Creates an instance of AlertsSuppressionRuleInner class. */ + /** + * Creates an instance of AlertsSuppressionRuleInner class. + */ public AlertsSuppressionRuleInner() { } /** * Get the innerProperties property: describes AlertsSuppressionRule properties. - * + * * @return the innerProperties value. */ private AlertsSuppressionRuleProperties innerProperties() { @@ -35,7 +39,7 @@ private AlertsSuppressionRuleProperties innerProperties() { /** * Get the alertType property: Type of the alert to automatically suppress. For all alert types, use '*'. - * + * * @return the alertType value. */ public String alertType() { @@ -44,7 +48,7 @@ public String alertType() { /** * Set the alertType property: Type of the alert to automatically suppress. For all alert types, use '*'. - * + * * @param alertType the alertType value to set. * @return the AlertsSuppressionRuleInner object itself. */ @@ -58,7 +62,7 @@ public AlertsSuppressionRuleInner withAlertType(String alertType) { /** * Get the lastModifiedUtc property: The last time this rule was modified. - * + * * @return the lastModifiedUtc value. */ public OffsetDateTime lastModifiedUtc() { @@ -67,8 +71,8 @@ public OffsetDateTime lastModifiedUtc() { /** * Get the expirationDateUtc property: Expiration date of the rule, if value is not provided or provided as null - * this field will default to the maximum allowed expiration date. - * + * there will no expiration at all. + * * @return the expirationDateUtc value. */ public OffsetDateTime expirationDateUtc() { @@ -77,8 +81,8 @@ public OffsetDateTime expirationDateUtc() { /** * Set the expirationDateUtc property: Expiration date of the rule, if value is not provided or provided as null - * this field will default to the maximum allowed expiration date. - * + * there will no expiration at all. + * * @param expirationDateUtc the expirationDateUtc value to set. * @return the AlertsSuppressionRuleInner object itself. */ @@ -92,7 +96,7 @@ public AlertsSuppressionRuleInner withExpirationDateUtc(OffsetDateTime expiratio /** * Get the reason property: The reason for dismissing the alert. - * + * * @return the reason value. */ public String reason() { @@ -101,7 +105,7 @@ public String reason() { /** * Set the reason property: The reason for dismissing the alert. - * + * * @param reason the reason value to set. * @return the AlertsSuppressionRuleInner object itself. */ @@ -115,7 +119,7 @@ public AlertsSuppressionRuleInner withReason(String reason) { /** * Get the state property: Possible states of the rule. - * + * * @return the state value. */ public RuleState state() { @@ -124,7 +128,7 @@ public RuleState state() { /** * Set the state property: Possible states of the rule. - * + * * @param state the state value to set. * @return the AlertsSuppressionRuleInner object itself. */ @@ -138,7 +142,7 @@ public AlertsSuppressionRuleInner withState(RuleState state) { /** * Get the comment property: Any comment regarding the rule. - * + * * @return the comment value. */ public String comment() { @@ -147,7 +151,7 @@ public String comment() { /** * Set the comment property: Any comment regarding the rule. - * + * * @param comment the comment value to set. * @return the AlertsSuppressionRuleInner object itself. */ @@ -161,7 +165,7 @@ public AlertsSuppressionRuleInner withComment(String comment) { /** * Get the suppressionAlertsScope property: The suppression conditions. - * + * * @return the suppressionAlertsScope value. */ public SuppressionAlertsScope suppressionAlertsScope() { @@ -170,7 +174,7 @@ public SuppressionAlertsScope suppressionAlertsScope() { /** * Set the suppressionAlertsScope property: The suppression conditions. - * + * * @param suppressionAlertsScope the suppressionAlertsScope value to set. * @return the AlertsSuppressionRuleInner object itself. */ @@ -184,7 +188,7 @@ public AlertsSuppressionRuleInner withSuppressionAlertsScope(SuppressionAlertsSc /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleProperties.java index 3ec48688de442..3f778272a1585 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AlertsSuppressionRuleProperties.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** describes AlertsSuppressionRule properties. */ +/** + * describes AlertsSuppressionRule properties. + */ @Fluent public final class AlertsSuppressionRuleProperties { /* @@ -27,8 +29,7 @@ public final class AlertsSuppressionRuleProperties { private OffsetDateTime lastModifiedUtc; /* - * Expiration date of the rule, if value is not provided or provided as null this field will default to the maximum - * allowed expiration date. + * Expiration date of the rule, if value is not provided or provided as null there will no expiration at all */ @JsonProperty(value = "expirationDateUtc") private OffsetDateTime expirationDateUtc; @@ -57,13 +58,15 @@ public final class AlertsSuppressionRuleProperties { @JsonProperty(value = "suppressionAlertsScope") private SuppressionAlertsScope suppressionAlertsScope; - /** Creates an instance of AlertsSuppressionRuleProperties class. */ + /** + * Creates an instance of AlertsSuppressionRuleProperties class. + */ public AlertsSuppressionRuleProperties() { } /** * Get the alertType property: Type of the alert to automatically suppress. For all alert types, use '*'. - * + * * @return the alertType value. */ public String alertType() { @@ -72,7 +75,7 @@ public String alertType() { /** * Set the alertType property: Type of the alert to automatically suppress. For all alert types, use '*'. - * + * * @param alertType the alertType value to set. * @return the AlertsSuppressionRuleProperties object itself. */ @@ -83,7 +86,7 @@ public AlertsSuppressionRuleProperties withAlertType(String alertType) { /** * Get the lastModifiedUtc property: The last time this rule was modified. - * + * * @return the lastModifiedUtc value. */ public OffsetDateTime lastModifiedUtc() { @@ -92,8 +95,8 @@ public OffsetDateTime lastModifiedUtc() { /** * Get the expirationDateUtc property: Expiration date of the rule, if value is not provided or provided as null - * this field will default to the maximum allowed expiration date. - * + * there will no expiration at all. + * * @return the expirationDateUtc value. */ public OffsetDateTime expirationDateUtc() { @@ -102,8 +105,8 @@ public OffsetDateTime expirationDateUtc() { /** * Set the expirationDateUtc property: Expiration date of the rule, if value is not provided or provided as null - * this field will default to the maximum allowed expiration date. - * + * there will no expiration at all. + * * @param expirationDateUtc the expirationDateUtc value to set. * @return the AlertsSuppressionRuleProperties object itself. */ @@ -114,7 +117,7 @@ public AlertsSuppressionRuleProperties withExpirationDateUtc(OffsetDateTime expi /** * Get the reason property: The reason for dismissing the alert. - * + * * @return the reason value. */ public String reason() { @@ -123,7 +126,7 @@ public String reason() { /** * Set the reason property: The reason for dismissing the alert. - * + * * @param reason the reason value to set. * @return the AlertsSuppressionRuleProperties object itself. */ @@ -134,7 +137,7 @@ public AlertsSuppressionRuleProperties withReason(String reason) { /** * Get the state property: Possible states of the rule. - * + * * @return the state value. */ public RuleState state() { @@ -143,7 +146,7 @@ public RuleState state() { /** * Set the state property: Possible states of the rule. - * + * * @param state the state value to set. * @return the AlertsSuppressionRuleProperties object itself. */ @@ -154,7 +157,7 @@ public AlertsSuppressionRuleProperties withState(RuleState state) { /** * Get the comment property: Any comment regarding the rule. - * + * * @return the comment value. */ public String comment() { @@ -163,7 +166,7 @@ public String comment() { /** * Set the comment property: Any comment regarding the rule. - * + * * @param comment the comment value to set. * @return the AlertsSuppressionRuleProperties object itself. */ @@ -174,7 +177,7 @@ public AlertsSuppressionRuleProperties withComment(String comment) { /** * Get the suppressionAlertsScope property: The suppression conditions. - * + * * @return the suppressionAlertsScope value. */ public SuppressionAlertsScope suppressionAlertsScope() { @@ -183,7 +186,7 @@ public SuppressionAlertsScope suppressionAlertsScope() { /** * Set the suppressionAlertsScope property: The suppression conditions. - * + * * @param suppressionAlertsScope the suppressionAlertsScope value to set. * @return the AlertsSuppressionRuleProperties object itself. */ @@ -194,27 +197,21 @@ public AlertsSuppressionRuleProperties withSuppressionAlertsScope(SuppressionAle /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (alertType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property alertType in model AlertsSuppressionRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property alertType in model AlertsSuppressionRuleProperties")); } if (reason() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property reason in model AlertsSuppressionRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property reason in model AlertsSuppressionRuleProperties")); } if (state() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property state in model AlertsSuppressionRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property state in model AlertsSuppressionRuleProperties")); } if (suppressionAlertsScope() != null) { suppressionAlertsScope().validate(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceInner.java index 7c222562dfcb1..8e5129cea2704 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceInner.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** The resource whose properties describes the allowed traffic between Azure resources. */ +/** + * The resource whose properties describes the allowed traffic between Azure resources. + */ @Immutable public final class AllowedConnectionsResourceInner extends ProxyResource { /* @@ -26,13 +28,15 @@ public final class AllowedConnectionsResourceInner extends ProxyResource { @JsonProperty(value = "location", access = JsonProperty.Access.WRITE_ONLY) private String location; - /** Creates an instance of AllowedConnectionsResourceInner class. */ + /** + * Creates an instance of AllowedConnectionsResourceInner class. + */ public AllowedConnectionsResourceInner() { } /** * Get the innerProperties property: Describes the allowed traffic between Azure resources. - * + * * @return the innerProperties value. */ private AllowedConnectionsResourceProperties innerProperties() { @@ -41,7 +45,7 @@ private AllowedConnectionsResourceProperties innerProperties() { /** * Get the location property: Location where the resource is stored. - * + * * @return the location value. */ public String location() { @@ -50,7 +54,7 @@ public String location() { /** * Get the calculatedDateTime property: The UTC time on which the allowed connections resource was calculated. - * + * * @return the calculatedDateTime value. */ public OffsetDateTime calculatedDateTime() { @@ -59,7 +63,7 @@ public OffsetDateTime calculatedDateTime() { /** * Get the connectableResources property: List of connectable resources. - * + * * @return the connectableResources value. */ public List connectableResources() { @@ -68,7 +72,7 @@ public List connectableResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceProperties.java index 26132bd10a555..65cec8ebc7925 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AllowedConnectionsResourceProperties.java @@ -10,7 +10,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Describes the allowed traffic between Azure resources. */ +/** + * Describes the allowed traffic between Azure resources. + */ @Immutable public final class AllowedConnectionsResourceProperties { /* @@ -25,13 +27,15 @@ public final class AllowedConnectionsResourceProperties { @JsonProperty(value = "connectableResources", access = JsonProperty.Access.WRITE_ONLY) private List connectableResources; - /** Creates an instance of AllowedConnectionsResourceProperties class. */ + /** + * Creates an instance of AllowedConnectionsResourceProperties class. + */ public AllowedConnectionsResourceProperties() { } /** * Get the calculatedDateTime property: The UTC time on which the allowed connections resource was calculated. - * + * * @return the calculatedDateTime value. */ public OffsetDateTime calculatedDateTime() { @@ -40,7 +44,7 @@ public OffsetDateTime calculatedDateTime() { /** * Get the connectableResources property: List of connectable resources. - * + * * @return the connectableResources value. */ public List connectableResources() { @@ -49,7 +53,7 @@ public List connectableResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionInner.java new file mode 100644 index 0000000000000..60fdc7257adbd --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionInner.java @@ -0,0 +1,145 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Immutable; +import com.azure.core.management.ProxyResource; +import com.azure.resourcemanager.security.models.ProvisioningState; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * An API collection as represented by Microsoft Defender for APIs. + */ +@Immutable +public final class ApiCollectionInner extends ProxyResource { + /* + * Describes the properties of an API collection. + */ + @JsonProperty(value = "properties") + private ApiCollectionProperties innerProperties; + + /** + * Creates an instance of ApiCollectionInner class. + */ + public ApiCollectionInner() { + } + + /** + * Get the innerProperties property: Describes the properties of an API collection. + * + * @return the innerProperties value. + */ + private ApiCollectionProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the provisioningState property: Gets the provisioning state of the API collection. + * + * @return the provisioningState value. + */ + public ProvisioningState provisioningState() { + return this.innerProperties() == null ? null : this.innerProperties().provisioningState(); + } + + /** + * Get the displayName property: The display name of the API collection. + * + * @return the displayName value. + */ + public String displayName() { + return this.innerProperties() == null ? null : this.innerProperties().displayName(); + } + + /** + * Get the discoveredVia property: The resource Id of the resource from where this API collection was discovered. + * + * @return the discoveredVia value. + */ + public String discoveredVia() { + return this.innerProperties() == null ? null : this.innerProperties().discoveredVia(); + } + + /** + * Get the baseUrl property: The base URI for this API collection. All endpoints of this API collection extend this + * base URI. + * + * @return the baseUrl value. + */ + public String baseUrl() { + return this.innerProperties() == null ? null : this.innerProperties().baseUrl(); + } + + /** + * Get the numberOfApiEndpoints property: The number of API endpoints discovered in this API collection. + * + * @return the numberOfApiEndpoints value. + */ + public Long numberOfApiEndpoints() { + return this.innerProperties() == null ? null : this.innerProperties().numberOfApiEndpoints(); + } + + /** + * Get the numberOfInactiveApiEndpoints property: The number of API endpoints in this API collection that have not + * received any API traffic in the last 30 days. + * + * @return the numberOfInactiveApiEndpoints value. + */ + public Long numberOfInactiveApiEndpoints() { + return this.innerProperties() == null ? null : this.innerProperties().numberOfInactiveApiEndpoints(); + } + + /** + * Get the numberOfUnauthenticatedApiEndpoints property: The number of API endpoints in this API collection that + * are unauthenticated. + * + * @return the numberOfUnauthenticatedApiEndpoints value. + */ + public Long numberOfUnauthenticatedApiEndpoints() { + return this.innerProperties() == null ? null : this.innerProperties().numberOfUnauthenticatedApiEndpoints(); + } + + /** + * Get the numberOfExternalApiEndpoints property: The number of API endpoints in this API collection for which API + * traffic from the internet was observed. + * + * @return the numberOfExternalApiEndpoints value. + */ + public Long numberOfExternalApiEndpoints() { + return this.innerProperties() == null ? null : this.innerProperties().numberOfExternalApiEndpoints(); + } + + /** + * Get the numberOfApiEndpointsWithSensitiveDataExposed property: The number of API endpoints in this API + * collection which are exposing sensitive data in their requests and/or responses. + * + * @return the numberOfApiEndpointsWithSensitiveDataExposed value. + */ + public Long numberOfApiEndpointsWithSensitiveDataExposed() { + return this.innerProperties() == null ? null + : this.innerProperties().numberOfApiEndpointsWithSensitiveDataExposed(); + } + + /** + * Get the sensitivityLabel property: The highest priority sensitivity label from Microsoft Purview in this API + * collection. + * + * @return the sensitivityLabel value. + */ + public String sensitivityLabel() { + return this.innerProperties() == null ? null : this.innerProperties().sensitivityLabel(); + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionProperties.java index 951e62c71a5c6..ee64ab453eeaf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionProperties.java @@ -4,34 +4,94 @@ package com.azure.resourcemanager.security.fluent.models; -import com.azure.core.annotation.Fluent; -import com.fasterxml.jackson.annotation.JsonInclude; +import com.azure.core.annotation.Immutable; +import com.azure.resourcemanager.security.models.ProvisioningState; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Map; -/** Describes the properties of an API collection. */ -@Fluent +/** + * Describes the properties of an API collection. + */ +@Immutable public final class ApiCollectionProperties { /* - * The display name of the Azure API Management API. + * Gets the provisioning state of the API collection. */ - @JsonProperty(value = "displayName") + @JsonProperty(value = "provisioningState", access = JsonProperty.Access.WRITE_ONLY) + private ProvisioningState provisioningState; + + /* + * The display name of the API collection. + */ + @JsonProperty(value = "displayName", access = JsonProperty.Access.WRITE_ONLY) private String displayName; /* - * Additional data regarding the API collection. + * The resource Id of the resource from where this API collection was discovered. + */ + @JsonProperty(value = "discoveredVia", access = JsonProperty.Access.WRITE_ONLY) + private String discoveredVia; + + /* + * The base URI for this API collection. All endpoints of this API collection extend this base URI. + */ + @JsonProperty(value = "baseUrl", access = JsonProperty.Access.WRITE_ONLY) + private String baseUrl; + + /* + * The number of API endpoints discovered in this API collection. + */ + @JsonProperty(value = "numberOfApiEndpoints", access = JsonProperty.Access.WRITE_ONLY) + private Long numberOfApiEndpoints; + + /* + * The number of API endpoints in this API collection that have not received any API traffic in the last 30 days. + */ + @JsonProperty(value = "numberOfInactiveApiEndpoints", access = JsonProperty.Access.WRITE_ONLY) + private Long numberOfInactiveApiEndpoints; + + /* + * The number of API endpoints in this API collection that are unauthenticated. */ - @JsonProperty(value = "additionalData") - @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) - private Map additionalData; + @JsonProperty(value = "numberOfUnauthenticatedApiEndpoints", access = JsonProperty.Access.WRITE_ONLY) + private Long numberOfUnauthenticatedApiEndpoints; - /** Creates an instance of ApiCollectionProperties class. */ + /* + * The number of API endpoints in this API collection for which API traffic from the internet was observed. + */ + @JsonProperty(value = "numberOfExternalApiEndpoints", access = JsonProperty.Access.WRITE_ONLY) + private Long numberOfExternalApiEndpoints; + + /* + * The number of API endpoints in this API collection which are exposing sensitive data in their requests and/or + * responses. + */ + @JsonProperty(value = "numberOfApiEndpointsWithSensitiveDataExposed", access = JsonProperty.Access.WRITE_ONLY) + private Long numberOfApiEndpointsWithSensitiveDataExposed; + + /* + * The highest priority sensitivity label from Microsoft Purview in this API collection. + */ + @JsonProperty(value = "sensitivityLabel", access = JsonProperty.Access.WRITE_ONLY) + private String sensitivityLabel; + + /** + * Creates an instance of ApiCollectionProperties class. + */ public ApiCollectionProperties() { } /** - * Get the displayName property: The display name of the Azure API Management API. - * + * Get the provisioningState property: Gets the provisioning state of the API collection. + * + * @return the provisioningState value. + */ + public ProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Get the displayName property: The display name of the API collection. + * * @return the displayName value. */ public String displayName() { @@ -39,39 +99,86 @@ public String displayName() { } /** - * Set the displayName property: The display name of the Azure API Management API. - * - * @param displayName the displayName value to set. - * @return the ApiCollectionProperties object itself. - */ - public ApiCollectionProperties withDisplayName(String displayName) { - this.displayName = displayName; - return this; + * Get the discoveredVia property: The resource Id of the resource from where this API collection was discovered. + * + * @return the discoveredVia value. + */ + public String discoveredVia() { + return this.discoveredVia; + } + + /** + * Get the baseUrl property: The base URI for this API collection. All endpoints of this API collection extend this + * base URI. + * + * @return the baseUrl value. + */ + public String baseUrl() { + return this.baseUrl; + } + + /** + * Get the numberOfApiEndpoints property: The number of API endpoints discovered in this API collection. + * + * @return the numberOfApiEndpoints value. + */ + public Long numberOfApiEndpoints() { + return this.numberOfApiEndpoints; + } + + /** + * Get the numberOfInactiveApiEndpoints property: The number of API endpoints in this API collection that have not + * received any API traffic in the last 30 days. + * + * @return the numberOfInactiveApiEndpoints value. + */ + public Long numberOfInactiveApiEndpoints() { + return this.numberOfInactiveApiEndpoints; } /** - * Get the additionalData property: Additional data regarding the API collection. - * - * @return the additionalData value. + * Get the numberOfUnauthenticatedApiEndpoints property: The number of API endpoints in this API collection that + * are unauthenticated. + * + * @return the numberOfUnauthenticatedApiEndpoints value. */ - public Map additionalData() { - return this.additionalData; + public Long numberOfUnauthenticatedApiEndpoints() { + return this.numberOfUnauthenticatedApiEndpoints; } /** - * Set the additionalData property: Additional data regarding the API collection. - * - * @param additionalData the additionalData value to set. - * @return the ApiCollectionProperties object itself. - */ - public ApiCollectionProperties withAdditionalData(Map additionalData) { - this.additionalData = additionalData; - return this; + * Get the numberOfExternalApiEndpoints property: The number of API endpoints in this API collection for which API + * traffic from the internet was observed. + * + * @return the numberOfExternalApiEndpoints value. + */ + public Long numberOfExternalApiEndpoints() { + return this.numberOfExternalApiEndpoints; + } + + /** + * Get the numberOfApiEndpointsWithSensitiveDataExposed property: The number of API endpoints in this API + * collection which are exposing sensitive data in their requests and/or responses. + * + * @return the numberOfApiEndpointsWithSensitiveDataExposed value. + */ + public Long numberOfApiEndpointsWithSensitiveDataExposed() { + return this.numberOfApiEndpointsWithSensitiveDataExposed; + } + + /** + * Get the sensitivityLabel property: The highest priority sensitivity label from Microsoft Purview in this API + * collection. + * + * @return the sensitivityLabel value. + */ + public String sensitivityLabel() { + return this.sensitivityLabel; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionResponseInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionResponseInner.java deleted file mode 100644 index 6224a7ea4eb48..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApiCollectionResponseInner.java +++ /dev/null @@ -1,90 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.ProxyResource; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Map; - -/** An API collection as represented by Defender for APIs. */ -@Fluent -public final class ApiCollectionResponseInner extends ProxyResource { - /* - * Describes the properties of an API collection. - */ - @JsonProperty(value = "properties") - private ApiCollectionProperties innerProperties; - - /** Creates an instance of ApiCollectionResponseInner class. */ - public ApiCollectionResponseInner() { - } - - /** - * Get the innerProperties property: Describes the properties of an API collection. - * - * @return the innerProperties value. - */ - private ApiCollectionProperties innerProperties() { - return this.innerProperties; - } - - /** - * Get the displayName property: The display name of the Azure API Management API. - * - * @return the displayName value. - */ - public String displayName() { - return this.innerProperties() == null ? null : this.innerProperties().displayName(); - } - - /** - * Set the displayName property: The display name of the Azure API Management API. - * - * @param displayName the displayName value to set. - * @return the ApiCollectionResponseInner object itself. - */ - public ApiCollectionResponseInner withDisplayName(String displayName) { - if (this.innerProperties() == null) { - this.innerProperties = new ApiCollectionProperties(); - } - this.innerProperties().withDisplayName(displayName); - return this; - } - - /** - * Get the additionalData property: Additional data regarding the API collection. - * - * @return the additionalData value. - */ - public Map additionalData() { - return this.innerProperties() == null ? null : this.innerProperties().additionalData(); - } - - /** - * Set the additionalData property: Additional data regarding the API collection. - * - * @param additionalData the additionalData value to set. - * @return the ApiCollectionResponseInner object itself. - */ - public ApiCollectionResponseInner withAdditionalData(Map additionalData) { - if (this.innerProperties() == null) { - this.innerProperties = new ApiCollectionProperties(); - } - this.innerProperties().withAdditionalData(additionalData); - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (innerProperties() != null) { - innerProperties().validate(); - } - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationInner.java index 52bda95475adf..ec4589ff800dd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationInner.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Security Application over a given scope. */ +/** + * Security Application over a given scope. + */ @Fluent public final class ApplicationInner extends ProxyResource { /* @@ -19,13 +21,15 @@ public final class ApplicationInner extends ProxyResource { @JsonProperty(value = "properties") private ApplicationProperties innerProperties; - /** Creates an instance of ApplicationInner class. */ + /** + * Creates an instance of ApplicationInner class. + */ public ApplicationInner() { } /** * Get the innerProperties property: Properties of a security application. - * + * * @return the innerProperties value. */ private ApplicationProperties innerProperties() { @@ -34,7 +38,7 @@ private ApplicationProperties innerProperties() { /** * Get the displayName property: display name of the application. - * + * * @return the displayName value. */ public String displayName() { @@ -43,7 +47,7 @@ public String displayName() { /** * Set the displayName property: display name of the application. - * + * * @param displayName the displayName value to set. * @return the ApplicationInner object itself. */ @@ -57,7 +61,7 @@ public ApplicationInner withDisplayName(String displayName) { /** * Get the description property: description of the application. - * + * * @return the description value. */ public String description() { @@ -66,7 +70,7 @@ public String description() { /** * Set the description property: description of the application. - * + * * @param description the description value to set. * @return the ApplicationInner object itself. */ @@ -80,7 +84,7 @@ public ApplicationInner withDescription(String description) { /** * Get the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @return the sourceResourceType value. */ public ApplicationSourceResourceType sourceResourceType() { @@ -89,7 +93,7 @@ public ApplicationSourceResourceType sourceResourceType() { /** * Set the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @param sourceResourceType the sourceResourceType value to set. * @return the ApplicationInner object itself. */ @@ -103,7 +107,7 @@ public ApplicationInner withSourceResourceType(ApplicationSourceResourceType sou /** * Get the conditionSets property: The application conditionSets - see examples. - * + * * @return the conditionSets value. */ public List conditionSets() { @@ -112,7 +116,7 @@ public List conditionSets() { /** * Set the conditionSets property: The application conditionSets - see examples. - * + * * @param conditionSets the conditionSets value to set. * @return the ApplicationInner object itself. */ @@ -126,7 +130,7 @@ public ApplicationInner withConditionSets(List conditionSets) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationProperties.java index bace1b11b0a5f..a0d1c03be990f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ApplicationProperties.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes properties of an application. */ +/** + * Describes properties of an application. + */ @Fluent public final class ApplicationProperties { /* @@ -37,13 +39,15 @@ public final class ApplicationProperties { @JsonProperty(value = "conditionSets", required = true) private List conditionSets; - /** Creates an instance of ApplicationProperties class. */ + /** + * Creates an instance of ApplicationProperties class. + */ public ApplicationProperties() { } /** * Get the displayName property: display name of the application. - * + * * @return the displayName value. */ public String displayName() { @@ -52,7 +56,7 @@ public String displayName() { /** * Set the displayName property: display name of the application. - * + * * @param displayName the displayName value to set. * @return the ApplicationProperties object itself. */ @@ -63,7 +67,7 @@ public ApplicationProperties withDisplayName(String displayName) { /** * Get the description property: description of the application. - * + * * @return the description value. */ public String description() { @@ -72,7 +76,7 @@ public String description() { /** * Set the description property: description of the application. - * + * * @param description the description value to set. * @return the ApplicationProperties object itself. */ @@ -83,7 +87,7 @@ public ApplicationProperties withDescription(String description) { /** * Get the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @return the sourceResourceType value. */ public ApplicationSourceResourceType sourceResourceType() { @@ -92,7 +96,7 @@ public ApplicationSourceResourceType sourceResourceType() { /** * Set the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @param sourceResourceType the sourceResourceType value to set. * @return the ApplicationProperties object itself. */ @@ -103,7 +107,7 @@ public ApplicationProperties withSourceResourceType(ApplicationSourceResourceTyp /** * Get the conditionSets property: The application conditionSets - see examples. - * + * * @return the conditionSets value. */ public List conditionSets() { @@ -112,7 +116,7 @@ public List conditionSets() { /** * Set the conditionSets property: The application conditionSets - see examples. - * + * * @param conditionSets the conditionSets value to set. * @return the ApplicationProperties object itself. */ @@ -123,21 +127,17 @@ public ApplicationProperties withConditionSets(List conditionSets) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (sourceResourceType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property sourceResourceType in model ApplicationProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sourceResourceType in model ApplicationProperties")); } if (conditionSets() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property conditionSets in model ApplicationProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property conditionSets in model ApplicationProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AscLocationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AscLocationInner.java index 351e2b24190fc..698386ce96506 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AscLocationInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AscLocationInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.ProxyResource; import com.fasterxml.jackson.annotation.JsonProperty; -/** The ASC location of the subscription is in the "name" field. */ +/** + * The ASC location of the subscription is in the "name" field. + */ @Fluent public final class AscLocationInner extends ProxyResource { /* @@ -17,13 +19,15 @@ public final class AscLocationInner extends ProxyResource { @JsonProperty(value = "properties") private Object properties; - /** Creates an instance of AscLocationInner class. */ + /** + * Creates an instance of AscLocationInner class. + */ public AscLocationInner() { } /** * Get the properties property: An empty set of properties. - * + * * @return the properties value. */ public Object properties() { @@ -32,7 +36,7 @@ public Object properties() { /** * Set the properties property: An empty set of properties. - * + * * @param properties the properties value to set. * @return the AscLocationInner object itself. */ @@ -43,7 +47,7 @@ public AscLocationInner withProperties(Object properties) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingInner.java index 8360f8b33cc30..925be47a8c8d4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.AutoProvision; import com.fasterxml.jackson.annotation.JsonProperty; -/** Auto provisioning setting. */ +/** + * Auto provisioning setting. + */ @Fluent public final class AutoProvisioningSettingInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class AutoProvisioningSettingInner extends ProxyResource { @JsonProperty(value = "properties") private AutoProvisioningSettingProperties innerProperties; - /** Creates an instance of AutoProvisioningSettingInner class. */ + /** + * Creates an instance of AutoProvisioningSettingInner class. + */ public AutoProvisioningSettingInner() { } /** * Get the innerProperties property: Auto provisioning setting data. - * + * * @return the innerProperties value. */ private AutoProvisioningSettingProperties innerProperties() { @@ -33,7 +37,7 @@ private AutoProvisioningSettingProperties innerProperties() { /** * Get the autoProvision property: Describes what kind of security agent provisioning action to take. - * + * * @return the autoProvision value. */ public AutoProvision autoProvision() { @@ -42,7 +46,7 @@ public AutoProvision autoProvision() { /** * Set the autoProvision property: Describes what kind of security agent provisioning action to take. - * + * * @param autoProvision the autoProvision value to set. * @return the AutoProvisioningSettingInner object itself. */ @@ -56,7 +60,7 @@ public AutoProvisioningSettingInner withAutoProvision(AutoProvision autoProvisio /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingProperties.java index 52a8ebeec0129..f17bd03d8bd37 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutoProvisioningSettingProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.AutoProvision; import com.fasterxml.jackson.annotation.JsonProperty; -/** describes properties of an auto provisioning setting. */ +/** + * describes properties of an auto provisioning setting. + */ @Fluent public final class AutoProvisioningSettingProperties { /* @@ -18,13 +20,15 @@ public final class AutoProvisioningSettingProperties { @JsonProperty(value = "autoProvision", required = true) private AutoProvision autoProvision; - /** Creates an instance of AutoProvisioningSettingProperties class. */ + /** + * Creates an instance of AutoProvisioningSettingProperties class. + */ public AutoProvisioningSettingProperties() { } /** * Get the autoProvision property: Describes what kind of security agent provisioning action to take. - * + * * @return the autoProvision value. */ public AutoProvision autoProvision() { @@ -33,7 +37,7 @@ public AutoProvision autoProvision() { /** * Set the autoProvision property: Describes what kind of security agent provisioning action to take. - * + * * @param autoProvision the autoProvision value to set. * @return the AutoProvisioningSettingProperties object itself. */ @@ -44,15 +48,13 @@ public AutoProvisioningSettingProperties withAutoProvision(AutoProvision autoPro /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (autoProvision() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property autoProvision in model AutoProvisioningSettingProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property autoProvision in model AutoProvisioningSettingProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationInner.java index 3881c341098ce..ffbb280ca023b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationInner.java @@ -13,7 +13,9 @@ import java.util.List; import java.util.Map; -/** The security automation resource. */ +/** + * The security automation resource. + */ @Fluent public final class AutomationInner extends Resource { /* @@ -22,27 +24,33 @@ public final class AutomationInner extends Resource { @JsonProperty(value = "properties") private AutomationProperties innerProperties; - /** Creates an instance of AutomationInner class. */ + /** + * Creates an instance of AutomationInner class. + */ public AutomationInner() { } /** * Get the innerProperties property: Security automation data. - * + * * @return the innerProperties value. */ private AutomationProperties innerProperties() { return this.innerProperties; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AutomationInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AutomationInner withTags(Map tags) { super.withTags(tags); @@ -51,7 +59,7 @@ public AutomationInner withTags(Map tags) { /** * Get the description property: The security automation description. - * + * * @return the description value. */ public String description() { @@ -60,7 +68,7 @@ public String description() { /** * Set the description property: The security automation description. - * + * * @param description the description value to set. * @return the AutomationInner object itself. */ @@ -74,7 +82,7 @@ public AutomationInner withDescription(String description) { /** * Get the isEnabled property: Indicates whether the security automation is enabled. - * + * * @return the isEnabled value. */ public Boolean isEnabled() { @@ -83,7 +91,7 @@ public Boolean isEnabled() { /** * Set the isEnabled property: Indicates whether the security automation is enabled. - * + * * @param isEnabled the isEnabled value to set. * @return the AutomationInner object itself. */ @@ -97,9 +105,9 @@ public AutomationInner withIsEnabled(Boolean isEnabled) { /** * Get the scopes property: A collection of scopes on which the security automations logic is applied. Supported - * scopes are the subscription itself or a resource group under that subscription. The automation will only apply on - * defined scopes. - * + * scopes are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. + * * @return the scopes value. */ public List scopes() { @@ -108,9 +116,9 @@ public List scopes() { /** * Set the scopes property: A collection of scopes on which the security automations logic is applied. Supported - * scopes are the subscription itself or a resource group under that subscription. The automation will only apply on - * defined scopes. - * + * scopes are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. + * * @param scopes the scopes value to set. * @return the AutomationInner object itself. */ @@ -125,7 +133,7 @@ public AutomationInner withScopes(List scopes) { /** * Get the sources property: A collection of the source event types which evaluate the security automation set of * rules. - * + * * @return the sources value. */ public List sources() { @@ -135,7 +143,7 @@ public List sources() { /** * Set the sources property: A collection of the source event types which evaluate the security automation set of * rules. - * + * * @param sources the sources value to set. * @return the AutomationInner object itself. */ @@ -150,7 +158,7 @@ public AutomationInner withSources(List sources) { /** * Get the actions property: A collection of the actions which are triggered if all the configured rules * evaluations, within at least one rule set, are true. - * + * * @return the actions value. */ public List actions() { @@ -160,7 +168,7 @@ public List actions() { /** * Set the actions property: A collection of the actions which are triggered if all the configured rules * evaluations, within at least one rule set, are true. - * + * * @param actions the actions value to set. * @return the AutomationInner object itself. */ @@ -174,7 +182,7 @@ public AutomationInner withActions(List actions) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationProperties.java index c84b9645e4287..f0ec00d46ad2e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationProperties.java @@ -49,13 +49,15 @@ public final class AutomationProperties { @JsonProperty(value = "actions") private List actions; - /** Creates an instance of AutomationProperties class. */ + /** + * Creates an instance of AutomationProperties class. + */ public AutomationProperties() { } /** * Get the description property: The security automation description. - * + * * @return the description value. */ public String description() { @@ -64,7 +66,7 @@ public String description() { /** * Set the description property: The security automation description. - * + * * @param description the description value to set. * @return the AutomationProperties object itself. */ @@ -75,7 +77,7 @@ public AutomationProperties withDescription(String description) { /** * Get the isEnabled property: Indicates whether the security automation is enabled. - * + * * @return the isEnabled value. */ public Boolean isEnabled() { @@ -84,7 +86,7 @@ public Boolean isEnabled() { /** * Set the isEnabled property: Indicates whether the security automation is enabled. - * + * * @param isEnabled the isEnabled value to set. * @return the AutomationProperties object itself. */ @@ -95,9 +97,9 @@ public AutomationProperties withIsEnabled(Boolean isEnabled) { /** * Get the scopes property: A collection of scopes on which the security automations logic is applied. Supported - * scopes are the subscription itself or a resource group under that subscription. The automation will only apply on - * defined scopes. - * + * scopes are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. + * * @return the scopes value. */ public List scopes() { @@ -106,9 +108,9 @@ public List scopes() { /** * Set the scopes property: A collection of scopes on which the security automations logic is applied. Supported - * scopes are the subscription itself or a resource group under that subscription. The automation will only apply on - * defined scopes. - * + * scopes are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. + * * @param scopes the scopes value to set. * @return the AutomationProperties object itself. */ @@ -120,7 +122,7 @@ public AutomationProperties withScopes(List scopes) { /** * Get the sources property: A collection of the source event types which evaluate the security automation set of * rules. - * + * * @return the sources value. */ public List sources() { @@ -130,7 +132,7 @@ public List sources() { /** * Set the sources property: A collection of the source event types which evaluate the security automation set of * rules. - * + * * @param sources the sources value to set. * @return the AutomationProperties object itself. */ @@ -142,7 +144,7 @@ public AutomationProperties withSources(List sources) { /** * Get the actions property: A collection of the actions which are triggered if all the configured rules * evaluations, within at least one rule set, are true. - * + * * @return the actions value. */ public List actions() { @@ -152,7 +154,7 @@ public List actions() { /** * Set the actions property: A collection of the actions which are triggered if all the configured rules * evaluations, within at least one rule set, are true. - * + * * @param actions the actions value to set. * @return the AutomationProperties object itself. */ @@ -163,7 +165,7 @@ public AutomationProperties withActions(List actions) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationValidationStatusInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationValidationStatusInner.java index a43555c19211a..7ebb43e68de04 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationValidationStatusInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AutomationValidationStatusInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The security automation model state property bag. */ +/** + * The security automation model state property bag. + */ @Fluent public final class AutomationValidationStatusInner { /* @@ -22,13 +24,15 @@ public final class AutomationValidationStatusInner { @JsonProperty(value = "message") private String message; - /** Creates an instance of AutomationValidationStatusInner class. */ + /** + * Creates an instance of AutomationValidationStatusInner class. + */ public AutomationValidationStatusInner() { } /** * Get the isValid property: Indicates whether the model is valid or not. - * + * * @return the isValid value. */ public Boolean isValid() { @@ -37,7 +41,7 @@ public Boolean isValid() { /** * Set the isValid property: Indicates whether the model is valid or not. - * + * * @param isValid the isValid value to set. * @return the AutomationValidationStatusInner object itself. */ @@ -48,7 +52,7 @@ public AutomationValidationStatusInner withIsValid(Boolean isValid) { /** * Get the message property: The validation message. - * + * * @return the message value. */ public String message() { @@ -57,7 +61,7 @@ public String message() { /** * Set the message property: The validation message. - * + * * @param message the message value to set. * @return the AutomationValidationStatusInner object itself. */ @@ -68,7 +72,7 @@ public AutomationValidationStatusInner withMessage(String message) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgInner.java new file mode 100644 index 0000000000000..65a89d987a176 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Azure DevOps Organization resource. + */ +@Fluent +public final class AzureDevOpsOrgInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * Azure DevOps Organization properties. + */ + @JsonProperty(value = "properties") + private AzureDevOpsOrgProperties properties; + + /** + * Creates an instance of AzureDevOpsOrgInner class. + */ + public AzureDevOpsOrgInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: Azure DevOps Organization properties. + * + * @return the properties value. + */ + public AzureDevOpsOrgProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Azure DevOps Organization properties. + * + * @param properties the properties value to set. + * @return the AzureDevOpsOrgInner object itself. + */ + public AzureDevOpsOrgInner withProperties(AzureDevOpsOrgProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultsList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgListResponseInner.java similarity index 52% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultsList.java rename to sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgListResponseInner.java index 81bc9513ab556..d84170ba3669a 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupVaultsList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsOrgListResponseInner.java @@ -2,58 +2,57 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.models; +package com.azure.resourcemanager.security.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.netapp.fluent.models.BackupVaultInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; /** - * List of Backup Vaults. + * List of RP resources which supports pagination. */ @Fluent -public final class BackupVaultsList { +public final class AzureDevOpsOrgListResponseInner { /* - * A list of Backup Vaults + * Gets or sets list of resources. */ @JsonProperty(value = "value") - private List value; + private List value; /* - * URL to get the next set of results. + * Gets or sets next link to scroll over the results. */ @JsonProperty(value = "nextLink") private String nextLink; /** - * Creates an instance of BackupVaultsList class. + * Creates an instance of AzureDevOpsOrgListResponseInner class. */ - public BackupVaultsList() { + public AzureDevOpsOrgListResponseInner() { } /** - * Get the value property: A list of Backup Vaults. + * Get the value property: Gets or sets list of resources. * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Set the value property: A list of Backup Vaults. + * Set the value property: Gets or sets list of resources. * * @param value the value value to set. - * @return the BackupVaultsList object itself. + * @return the AzureDevOpsOrgListResponseInner object itself. */ - public BackupVaultsList withValue(List value) { + public AzureDevOpsOrgListResponseInner withValue(List value) { this.value = value; return this; } /** - * Get the nextLink property: URL to get the next set of results. + * Get the nextLink property: Gets or sets next link to scroll over the results. * * @return the nextLink value. */ @@ -62,12 +61,12 @@ public String nextLink() { } /** - * Set the nextLink property: URL to get the next set of results. + * Set the nextLink property: Gets or sets next link to scroll over the results. * * @param nextLink the nextLink value to set. - * @return the BackupVaultsList object itself. + * @return the AzureDevOpsOrgListResponseInner object itself. */ - public BackupVaultsList withNextLink(String nextLink) { + public AzureDevOpsOrgListResponseInner withNextLink(String nextLink) { this.nextLink = nextLink; return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsProjectInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsProjectInner.java new file mode 100644 index 0000000000000..0b80eb4533bdf --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsProjectInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.AzureDevOpsProjectProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Azure DevOps Project resource. + */ +@Fluent +public final class AzureDevOpsProjectInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * Azure DevOps Project properties. + */ + @JsonProperty(value = "properties") + private AzureDevOpsProjectProperties properties; + + /** + * Creates an instance of AzureDevOpsProjectInner class. + */ + public AzureDevOpsProjectInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: Azure DevOps Project properties. + * + * @return the properties value. + */ + public AzureDevOpsProjectProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Azure DevOps Project properties. + * + * @param properties the properties value to set. + * @return the AzureDevOpsProjectInner object itself. + */ + public AzureDevOpsProjectInner withProperties(AzureDevOpsProjectProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsRepositoryInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsRepositoryInner.java new file mode 100644 index 0000000000000..ce96e9660324f --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/AzureDevOpsRepositoryInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.AzureDevOpsRepositoryProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Azure DevOps Repository resource. + */ +@Fluent +public final class AzureDevOpsRepositoryInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * Azure DevOps Repository properties. + */ + @JsonProperty(value = "properties") + private AzureDevOpsRepositoryProperties properties; + + /** + * Creates an instance of AzureDevOpsRepositoryInner class. + */ + public AzureDevOpsRepositoryInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: Azure DevOps Repository properties. + * + * @return the properties value. + */ + public AzureDevOpsRepositoryProperties properties() { + return this.properties; + } + + /** + * Set the properties property: Azure DevOps Repository properties. + * + * @param properties the properties value to set. + * @return the AzureDevOpsRepositoryInner object itself. + */ + public AzureDevOpsRepositoryInner withProperties(AzureDevOpsRepositoryProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceInner.java index 4017d627a9d07..209bd4b33efe6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceInner.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Compliance of a scope. */ +/** + * Compliance of a scope. + */ @Immutable public final class ComplianceInner extends ProxyResource { /* @@ -20,13 +22,15 @@ public final class ComplianceInner extends ProxyResource { @JsonProperty(value = "properties") private ComplianceProperties innerProperties; - /** Creates an instance of ComplianceInner class. */ + /** + * Creates an instance of ComplianceInner class. + */ public ComplianceInner() { } /** * Get the innerProperties property: Compliance data. - * + * * @return the innerProperties value. */ private ComplianceProperties innerProperties() { @@ -35,7 +39,7 @@ private ComplianceProperties innerProperties() { /** * Get the assessmentTimestampUtcDate property: The timestamp when the Compliance calculation was conducted. - * + * * @return the assessmentTimestampUtcDate value. */ public OffsetDateTime assessmentTimestampUtcDate() { @@ -43,9 +47,9 @@ public OffsetDateTime assessmentTimestampUtcDate() { } /** - * Get the resourceCount property: The resource count of the given subscription for which the Compliance calculation - * was conducted (needed for Management Group Compliance calculation). - * + * Get the resourceCount property: The resource count of the given subscription for which the Compliance + * calculation was conducted (needed for Management Group Compliance calculation). + * * @return the resourceCount value. */ public Integer resourceCount() { @@ -54,7 +58,7 @@ public Integer resourceCount() { /** * Get the assessmentResult property: An array of segment, which is the actually the compliance assessment. - * + * * @return the assessmentResult value. */ public List assessmentResult() { @@ -63,7 +67,7 @@ public List assessmentResult() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceProperties.java index 8eb6d80114fc8..1ff87cb576075 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceProperties.java @@ -36,13 +36,15 @@ public final class ComplianceProperties { @JsonProperty(value = "assessmentResult", access = JsonProperty.Access.WRITE_ONLY) private List assessmentResult; - /** Creates an instance of ComplianceProperties class. */ + /** + * Creates an instance of ComplianceProperties class. + */ public ComplianceProperties() { } /** * Get the assessmentTimestampUtcDate property: The timestamp when the Compliance calculation was conducted. - * + * * @return the assessmentTimestampUtcDate value. */ public OffsetDateTime assessmentTimestampUtcDate() { @@ -50,9 +52,9 @@ public OffsetDateTime assessmentTimestampUtcDate() { } /** - * Get the resourceCount property: The resource count of the given subscription for which the Compliance calculation - * was conducted (needed for Management Group Compliance calculation). - * + * Get the resourceCount property: The resource count of the given subscription for which the Compliance + * calculation was conducted (needed for Management Group Compliance calculation). + * * @return the resourceCount value. */ public Integer resourceCount() { @@ -61,7 +63,7 @@ public Integer resourceCount() { /** * Get the assessmentResult property: An array of segment, which is the actually the compliance assessment. - * + * * @return the assessmentResult value. */ public List assessmentResult() { @@ -70,7 +72,7 @@ public List assessmentResult() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultInner.java index 4487e681da9ba..7ba1143528a38 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.ResourceStatus; import com.fasterxml.jackson.annotation.JsonProperty; -/** a compliance result. */ +/** + * a compliance result. + */ @Immutable public final class ComplianceResultInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class ComplianceResultInner extends ProxyResource { @JsonProperty(value = "properties") private ComplianceResultProperties innerProperties; - /** Creates an instance of ComplianceResultInner class. */ + /** + * Creates an instance of ComplianceResultInner class. + */ public ComplianceResultInner() { } /** * Get the innerProperties property: Compliance result data. - * + * * @return the innerProperties value. */ private ComplianceResultProperties innerProperties() { @@ -33,7 +37,7 @@ private ComplianceResultProperties innerProperties() { /** * Get the resourceStatus property: The status of the resource regarding a single assessment. - * + * * @return the resourceStatus value. */ public ResourceStatus resourceStatus() { @@ -42,7 +46,7 @@ public ResourceStatus resourceStatus() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultProperties.java index 743f7fe1a8b7b..1b58d3960bcce 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ComplianceResultProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.ResourceStatus; import com.fasterxml.jackson.annotation.JsonProperty; -/** Compliance result data. */ +/** + * Compliance result data. + */ @Immutable public final class ComplianceResultProperties { /* @@ -17,13 +19,15 @@ public final class ComplianceResultProperties { @JsonProperty(value = "resourceStatus", access = JsonProperty.Access.WRITE_ONLY) private ResourceStatus resourceStatus; - /** Creates an instance of ComplianceResultProperties class. */ + /** + * Creates an instance of ComplianceResultProperties class. + */ public ComplianceResultProperties() { } /** * Get the resourceStatus property: The status of the resource regarding a single assessment. - * + * * @return the resourceStatus value. */ public ResourceStatus resourceStatus() { @@ -32,7 +36,7 @@ public ResourceStatus resourceStatus() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectionStringsInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectionStringsInner.java deleted file mode 100644 index 4e21ee921fae2..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectionStringsInner.java +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.security.models.IngestionConnectionString; -import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.List; - -/** Connection string for ingesting security data and logs. */ -@Fluent -public final class ConnectionStringsInner { - /* - * Connection strings - */ - @JsonProperty(value = "value", required = true) - private List value; - - /** Creates an instance of ConnectionStringsInner class. */ - public ConnectionStringsInner() { - } - - /** - * Get the value property: Connection strings. - * - * @return the value value. - */ - public List value() { - return this.value; - } - - /** - * Set the value property: Connection strings. - * - * @param value the value value to set. - * @return the ConnectionStringsInner object itself. - */ - public ConnectionStringsInner withValue(List value) { - this.value = value; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model ConnectionStringsInner")); - } else { - value().forEach(e -> e.validate()); - } - } - - private static final ClientLogger LOGGER = new ClientLogger(ConnectionStringsInner.class); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingInner.java index 423238e44dabf..447751f0bfe10 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingInner.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.security.models.HybridComputeSettingsProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** The connector setting. */ +/** + * The connector setting. + */ @Fluent public final class ConnectorSettingInner extends ProxyResource { /* @@ -19,13 +21,15 @@ public final class ConnectorSettingInner extends ProxyResource { @JsonProperty(value = "properties") private ConnectorSettingProperties innerProperties; - /** Creates an instance of ConnectorSettingInner class. */ + /** + * Creates an instance of ConnectorSettingInner class. + */ public ConnectorSettingInner() { } /** * Get the innerProperties property: Connector setting data. - * + * * @return the innerProperties value. */ private ConnectorSettingProperties innerProperties() { @@ -35,7 +39,7 @@ private ConnectorSettingProperties innerProperties() { /** * Get the hybridComputeSettings property: Settings for hybrid compute management. These settings are relevant only * for Arc autoProvision (Hybrid Compute). - * + * * @return the hybridComputeSettings value. */ public HybridComputeSettingsProperties hybridComputeSettings() { @@ -45,7 +49,7 @@ public HybridComputeSettingsProperties hybridComputeSettings() { /** * Set the hybridComputeSettings property: Settings for hybrid compute management. These settings are relevant only * for Arc autoProvision (Hybrid Compute). - * + * * @param hybridComputeSettings the hybridComputeSettings value to set. * @return the ConnectorSettingInner object itself. */ @@ -60,7 +64,7 @@ public ConnectorSettingInner withHybridComputeSettings(HybridComputeSettingsProp /** * Get the authenticationDetails property: Settings for authentication management, these settings are relevant only * for the cloud connector. - * + * * @return the authenticationDetails value. */ public AuthenticationDetailsProperties authenticationDetails() { @@ -70,7 +74,7 @@ public AuthenticationDetailsProperties authenticationDetails() { /** * Set the authenticationDetails property: Settings for authentication management, these settings are relevant only * for the cloud connector. - * + * * @param authenticationDetails the authenticationDetails value to set. * @return the ConnectorSettingInner object itself. */ @@ -84,7 +88,7 @@ public ConnectorSettingInner withAuthenticationDetails(AuthenticationDetailsProp /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingProperties.java index 5a6a558eed594..800caa677e3bf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ConnectorSettingProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.HybridComputeSettingsProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Describes properties of a connector setting. */ +/** + * Describes properties of a connector setting. + */ @Fluent public final class ConnectorSettingProperties { /* @@ -24,14 +26,16 @@ public final class ConnectorSettingProperties { @JsonProperty(value = "authenticationDetails") private AuthenticationDetailsProperties authenticationDetails; - /** Creates an instance of ConnectorSettingProperties class. */ + /** + * Creates an instance of ConnectorSettingProperties class. + */ public ConnectorSettingProperties() { } /** * Get the hybridComputeSettings property: Settings for hybrid compute management. These settings are relevant only * for Arc autoProvision (Hybrid Compute). - * + * * @return the hybridComputeSettings value. */ public HybridComputeSettingsProperties hybridComputeSettings() { @@ -41,7 +45,7 @@ public HybridComputeSettingsProperties hybridComputeSettings() { /** * Set the hybridComputeSettings property: Settings for hybrid compute management. These settings are relevant only * for Arc autoProvision (Hybrid Compute). - * + * * @param hybridComputeSettings the hybridComputeSettings value to set. * @return the ConnectorSettingProperties object itself. */ @@ -53,7 +57,7 @@ public ConnectorSettingProperties withHybridComputeSettings(HybridComputeSetting /** * Get the authenticationDetails property: Settings for authentication management, these settings are relevant only * for the cloud connector. - * + * * @return the authenticationDetails value. */ public AuthenticationDetailsProperties authenticationDetails() { @@ -63,7 +67,7 @@ public AuthenticationDetailsProperties authenticationDetails() { /** * Set the authenticationDetails property: Settings for authentication management, these settings are relevant only * for the cloud connector. - * + * * @param authenticationDetails the authenticationDetails value to set. * @return the ConnectorSettingProperties object itself. */ @@ -74,7 +78,7 @@ public ConnectorSettingProperties withAuthenticationDetails(AuthenticationDetail /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationInner.java index 77f1347c266e5..405c0fe7d59ea 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationInner.java @@ -11,7 +11,9 @@ import com.azure.resourcemanager.security.models.SupportedCloudEnum; import com.fasterxml.jackson.annotation.JsonProperty; -/** Custom Assessment Automation. */ +/** + * Custom Assessment Automation. + */ @Fluent public final class CustomAssessmentAutomationInner extends ProxyResource { /* @@ -26,13 +28,15 @@ public final class CustomAssessmentAutomationInner extends ProxyResource { @JsonProperty(value = "properties") private CustomAssessmentAutomationProperties innerProperties; - /** Creates an instance of CustomAssessmentAutomationInner class. */ + /** + * Creates an instance of CustomAssessmentAutomationInner class. + */ public CustomAssessmentAutomationInner() { } /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -41,7 +45,7 @@ public SystemData systemData() { /** * Get the innerProperties property: describes Custom Assessment Automation properties. - * + * * @return the innerProperties value. */ private CustomAssessmentAutomationProperties innerProperties() { @@ -49,8 +53,9 @@ private CustomAssessmentAutomationProperties innerProperties() { } /** - * Get the compressedQuery property: GZip encoded KQL query representing the assessment automation results required. - * + * Get the compressedQuery property: GZip encoded KQL query representing the assessment automation results + * required. + * * @return the compressedQuery value. */ public String compressedQuery() { @@ -58,8 +63,9 @@ public String compressedQuery() { } /** - * Set the compressedQuery property: GZip encoded KQL query representing the assessment automation results required. - * + * Set the compressedQuery property: GZip encoded KQL query representing the assessment automation results + * required. + * * @param compressedQuery the compressedQuery value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -73,7 +79,7 @@ public CustomAssessmentAutomationInner withCompressedQuery(String compressedQuer /** * Get the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @return the supportedCloud value. */ public SupportedCloudEnum supportedCloud() { @@ -82,7 +88,7 @@ public SupportedCloudEnum supportedCloud() { /** * Set the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @param supportedCloud the supportedCloud value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -96,7 +102,7 @@ public CustomAssessmentAutomationInner withSupportedCloud(SupportedCloudEnum sup /** * Get the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @return the severity value. */ public SeverityEnum severity() { @@ -105,7 +111,7 @@ public SeverityEnum severity() { /** * Set the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @param severity the severity value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -119,7 +125,7 @@ public CustomAssessmentAutomationInner withSeverity(SeverityEnum severity) { /** * Get the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @return the displayName value. */ public String displayName() { @@ -128,7 +134,7 @@ public String displayName() { /** * Set the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @param displayName the displayName value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -143,7 +149,7 @@ public CustomAssessmentAutomationInner withDisplayName(String displayName) { /** * Get the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @return the description value. */ public String description() { @@ -153,7 +159,7 @@ public String description() { /** * Set the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @param description the description value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -168,7 +174,7 @@ public CustomAssessmentAutomationInner withDescription(String description) { /** * Get the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @return the remediationDescription value. */ public String remediationDescription() { @@ -178,7 +184,7 @@ public String remediationDescription() { /** * Set the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @param remediationDescription the remediationDescription value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -193,7 +199,7 @@ public CustomAssessmentAutomationInner withRemediationDescription(String remedia /** * Get the assessmentKey property: The assessment metadata key used when an assessment is generated for this * assessment automation. - * + * * @return the assessmentKey value. */ public String assessmentKey() { @@ -203,7 +209,7 @@ public String assessmentKey() { /** * Set the assessmentKey property: The assessment metadata key used when an assessment is generated for this * assessment automation. - * + * * @param assessmentKey the assessmentKey value to set. * @return the CustomAssessmentAutomationInner object itself. */ @@ -217,7 +223,7 @@ public CustomAssessmentAutomationInner withAssessmentKey(String assessmentKey) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationProperties.java index 9e8384522a946..7c86d35c0290d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.SupportedCloudEnum; import com.fasterxml.jackson.annotation.JsonProperty; -/** describes the Custom Assessment Automation properties. */ +/** + * describes the Custom Assessment Automation properties. + */ @Fluent public final class CustomAssessmentAutomationProperties { /* @@ -54,13 +56,16 @@ public final class CustomAssessmentAutomationProperties { @JsonProperty(value = "assessmentKey") private String assessmentKey; - /** Creates an instance of CustomAssessmentAutomationProperties class. */ + /** + * Creates an instance of CustomAssessmentAutomationProperties class. + */ public CustomAssessmentAutomationProperties() { } /** - * Get the compressedQuery property: GZip encoded KQL query representing the assessment automation results required. - * + * Get the compressedQuery property: GZip encoded KQL query representing the assessment automation results + * required. + * * @return the compressedQuery value. */ public String compressedQuery() { @@ -68,8 +73,9 @@ public String compressedQuery() { } /** - * Set the compressedQuery property: GZip encoded KQL query representing the assessment automation results required. - * + * Set the compressedQuery property: GZip encoded KQL query representing the assessment automation results + * required. + * * @param compressedQuery the compressedQuery value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -80,7 +86,7 @@ public CustomAssessmentAutomationProperties withCompressedQuery(String compresse /** * Get the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @return the supportedCloud value. */ public SupportedCloudEnum supportedCloud() { @@ -89,7 +95,7 @@ public SupportedCloudEnum supportedCloud() { /** * Set the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @param supportedCloud the supportedCloud value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -100,7 +106,7 @@ public CustomAssessmentAutomationProperties withSupportedCloud(SupportedCloudEnu /** * Get the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @return the severity value. */ public SeverityEnum severity() { @@ -109,7 +115,7 @@ public SeverityEnum severity() { /** * Set the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @param severity the severity value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -120,7 +126,7 @@ public CustomAssessmentAutomationProperties withSeverity(SeverityEnum severity) /** * Get the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @return the displayName value. */ public String displayName() { @@ -129,7 +135,7 @@ public String displayName() { /** * Set the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @param displayName the displayName value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -141,7 +147,7 @@ public CustomAssessmentAutomationProperties withDisplayName(String displayName) /** * Get the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @return the description value. */ public String description() { @@ -151,7 +157,7 @@ public String description() { /** * Set the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @param description the description value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -163,7 +169,7 @@ public CustomAssessmentAutomationProperties withDescription(String description) /** * Get the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @return the remediationDescription value. */ public String remediationDescription() { @@ -173,7 +179,7 @@ public String remediationDescription() { /** * Set the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @param remediationDescription the remediationDescription value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -185,7 +191,7 @@ public CustomAssessmentAutomationProperties withRemediationDescription(String re /** * Get the assessmentKey property: The assessment metadata key used when an assessment is generated for this * assessment automation. - * + * * @return the assessmentKey value. */ public String assessmentKey() { @@ -195,7 +201,7 @@ public String assessmentKey() { /** * Set the assessmentKey property: The assessment metadata key used when an assessment is generated for this * assessment automation. - * + * * @param assessmentKey the assessmentKey value to set. * @return the CustomAssessmentAutomationProperties object itself. */ @@ -206,7 +212,7 @@ public CustomAssessmentAutomationProperties withAssessmentKey(String assessmentK /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationRequestProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationRequestProperties.java index 524a5c5db165c..69a9046298a65 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationRequestProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomAssessmentAutomationRequestProperties.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.SupportedCloudEnum; import com.fasterxml.jackson.annotation.JsonProperty; -/** describes the Custom Assessment Automation properties. */ +/** + * describes the Custom Assessment Automation properties. + */ @Fluent public final class CustomAssessmentAutomationRequestProperties { /* @@ -48,14 +50,16 @@ public final class CustomAssessmentAutomationRequestProperties { @JsonProperty(value = "remediationDescription") private String remediationDescription; - /** Creates an instance of CustomAssessmentAutomationRequestProperties class. */ + /** + * Creates an instance of CustomAssessmentAutomationRequestProperties class. + */ public CustomAssessmentAutomationRequestProperties() { } /** * Get the compressedQuery property: Base 64 encoded KQL query representing the assessment automation results * required. - * + * * @return the compressedQuery value. */ public String compressedQuery() { @@ -65,7 +69,7 @@ public String compressedQuery() { /** * Set the compressedQuery property: Base 64 encoded KQL query representing the assessment automation results * required. - * + * * @param compressedQuery the compressedQuery value to set. * @return the CustomAssessmentAutomationRequestProperties object itself. */ @@ -76,7 +80,7 @@ public CustomAssessmentAutomationRequestProperties withCompressedQuery(String co /** * Get the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @return the supportedCloud value. */ public SupportedCloudEnum supportedCloud() { @@ -85,7 +89,7 @@ public SupportedCloudEnum supportedCloud() { /** * Set the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @param supportedCloud the supportedCloud value to set. * @return the CustomAssessmentAutomationRequestProperties object itself. */ @@ -96,7 +100,7 @@ public CustomAssessmentAutomationRequestProperties withSupportedCloud(SupportedC /** * Get the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @return the severity value. */ public SeverityEnum severity() { @@ -105,7 +109,7 @@ public SeverityEnum severity() { /** * Set the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @param severity the severity value to set. * @return the CustomAssessmentAutomationRequestProperties object itself. */ @@ -116,7 +120,7 @@ public CustomAssessmentAutomationRequestProperties withSeverity(SeverityEnum sev /** * Get the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @return the displayName value. */ public String displayName() { @@ -125,7 +129,7 @@ public String displayName() { /** * Set the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @param displayName the displayName value to set. * @return the CustomAssessmentAutomationRequestProperties object itself. */ @@ -137,7 +141,7 @@ public CustomAssessmentAutomationRequestProperties withDisplayName(String displa /** * Get the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @return the description value. */ public String description() { @@ -147,7 +151,7 @@ public String description() { /** * Set the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @param description the description value to set. * @return the CustomAssessmentAutomationRequestProperties object itself. */ @@ -159,7 +163,7 @@ public CustomAssessmentAutomationRequestProperties withDescription(String descri /** * Get the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @return the remediationDescription value. */ public String remediationDescription() { @@ -169,7 +173,7 @@ public String remediationDescription() { /** * Set the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @param remediationDescription the remediationDescription value to set. * @return the CustomAssessmentAutomationRequestProperties object itself. */ @@ -180,7 +184,7 @@ public CustomAssessmentAutomationRequestProperties withRemediationDescription(St /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentInner.java index 0ac905684d259..3a24a6da384ba 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentInner.java @@ -9,7 +9,9 @@ import com.azure.core.management.SystemData; import com.fasterxml.jackson.annotation.JsonProperty; -/** Custom entity store assignment. */ +/** + * Custom entity store assignment. + */ @Fluent public final class CustomEntityStoreAssignmentInner extends ProxyResource { /* @@ -24,13 +26,15 @@ public final class CustomEntityStoreAssignmentInner extends ProxyResource { @JsonProperty(value = "properties") private CustomEntityStoreAssignmentProperties innerProperties; - /** Creates an instance of CustomEntityStoreAssignmentInner class. */ + /** + * Creates an instance of CustomEntityStoreAssignmentInner class. + */ public CustomEntityStoreAssignmentInner() { } /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -39,7 +43,7 @@ public SystemData systemData() { /** * Get the innerProperties property: describes custom entity store assignment properties. - * + * * @return the innerProperties value. */ private CustomEntityStoreAssignmentProperties innerProperties() { @@ -49,7 +53,7 @@ private CustomEntityStoreAssignmentProperties innerProperties() { /** * Get the principal property: The principal assigned with entity store. Format of principal is: [AAD * type]=[PrincipalObjectId];[TenantId]. - * + * * @return the principal value. */ public String principal() { @@ -59,7 +63,7 @@ public String principal() { /** * Set the principal property: The principal assigned with entity store. Format of principal is: [AAD * type]=[PrincipalObjectId];[TenantId]. - * + * * @param principal the principal value to set. * @return the CustomEntityStoreAssignmentInner object itself. */ @@ -73,7 +77,7 @@ public CustomEntityStoreAssignmentInner withPrincipal(String principal) { /** * Get the entityStoreDatabaseLink property: The link to entity store database. - * + * * @return the entityStoreDatabaseLink value. */ public String entityStoreDatabaseLink() { @@ -82,7 +86,7 @@ public String entityStoreDatabaseLink() { /** * Set the entityStoreDatabaseLink property: The link to entity store database. - * + * * @param entityStoreDatabaseLink the entityStoreDatabaseLink value to set. * @return the CustomEntityStoreAssignmentInner object itself. */ @@ -96,7 +100,7 @@ public CustomEntityStoreAssignmentInner withEntityStoreDatabaseLink(String entit /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentProperties.java index cf2b86ebd137c..b6b881e2da6e4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** describes the custom entity store assignment properties. */ +/** + * describes the custom entity store assignment properties. + */ @Fluent public final class CustomEntityStoreAssignmentProperties { /* @@ -22,14 +24,16 @@ public final class CustomEntityStoreAssignmentProperties { @JsonProperty(value = "entityStoreDatabaseLink") private String entityStoreDatabaseLink; - /** Creates an instance of CustomEntityStoreAssignmentProperties class. */ + /** + * Creates an instance of CustomEntityStoreAssignmentProperties class. + */ public CustomEntityStoreAssignmentProperties() { } /** * Get the principal property: The principal assigned with entity store. Format of principal is: [AAD * type]=[PrincipalObjectId];[TenantId]. - * + * * @return the principal value. */ public String principal() { @@ -39,7 +43,7 @@ public String principal() { /** * Set the principal property: The principal assigned with entity store. Format of principal is: [AAD * type]=[PrincipalObjectId];[TenantId]. - * + * * @param principal the principal value to set. * @return the CustomEntityStoreAssignmentProperties object itself. */ @@ -50,7 +54,7 @@ public CustomEntityStoreAssignmentProperties withPrincipal(String principal) { /** * Get the entityStoreDatabaseLink property: The link to entity store database. - * + * * @return the entityStoreDatabaseLink value. */ public String entityStoreDatabaseLink() { @@ -59,7 +63,7 @@ public String entityStoreDatabaseLink() { /** * Set the entityStoreDatabaseLink property: The link to entity store database. - * + * * @param entityStoreDatabaseLink the entityStoreDatabaseLink value to set. * @return the CustomEntityStoreAssignmentProperties object itself. */ @@ -70,7 +74,7 @@ public CustomEntityStoreAssignmentProperties withEntityStoreDatabaseLink(String /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentRequestProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentRequestProperties.java index fc9ebcd59cabe..07235339fa567 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentRequestProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/CustomEntityStoreAssignmentRequestProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** describes properties of custom entity store assignment request. */ +/** + * describes properties of custom entity store assignment request. + */ @Fluent public final class CustomEntityStoreAssignmentRequestProperties { /* @@ -17,14 +19,16 @@ public final class CustomEntityStoreAssignmentRequestProperties { @JsonProperty(value = "principal") private String principal; - /** Creates an instance of CustomEntityStoreAssignmentRequestProperties class. */ + /** + * Creates an instance of CustomEntityStoreAssignmentRequestProperties class. + */ public CustomEntityStoreAssignmentRequestProperties() { } /** - * Get the principal property: The principal assigned with entity store. If not provided, will use caller principal. - * Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. - * + * Get the principal property: The principal assigned with entity store. If not provided, will use caller + * principal. Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. + * * @return the principal value. */ public String principal() { @@ -32,9 +36,9 @@ public String principal() { } /** - * Set the principal property: The principal assigned with entity store. If not provided, will use caller principal. - * Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. - * + * Set the principal property: The principal assigned with entity store. If not provided, will use caller + * principal. Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. + * * @param principal the principal value to set. * @return the CustomEntityStoreAssignmentRequestProperties object itself. */ @@ -45,7 +49,7 @@ public CustomEntityStoreAssignmentRequestProperties withPrincipal(String princip /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DataExportSettingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DataExportSettingProperties.java index 5f8daf7eabc5f..eca261e08f1f2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DataExportSettingProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DataExportSettingProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The data export setting properties. */ +/** + * The data export setting properties. + */ @Fluent public final class DataExportSettingProperties { /* @@ -16,13 +18,15 @@ public final class DataExportSettingProperties { @JsonProperty(value = "enabled", required = true) private boolean enabled; - /** Creates an instance of DataExportSettingProperties class. */ + /** + * Creates an instance of DataExportSettingProperties class. + */ public DataExportSettingProperties() { } /** * Get the enabled property: Is the data export setting enabled. - * + * * @return the enabled value. */ public boolean enabled() { @@ -31,7 +35,7 @@ public boolean enabled() { /** * Set the enabled property: Is the data export setting enabled. - * + * * @param enabled the enabled value to set. * @return the DataExportSettingProperties object itself. */ @@ -42,7 +46,7 @@ public DataExportSettingProperties withEnabled(boolean enabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingInner.java new file mode 100644 index 0000000000000..0a1562f2ddd55 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingInner.java @@ -0,0 +1,220 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.resourcemanager.security.models.OperationStatus; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Defender for Storage resource. + */ +@Fluent +public final class DefenderForStorageSettingInner extends ProxyResource { + /* + * Defender for Storage resource properties. + */ + @JsonProperty(value = "properties") + private DefenderForStorageSettingProperties innerProperties; + + /** + * Creates an instance of DefenderForStorageSettingInner class. + */ + public DefenderForStorageSettingInner() { + } + + /** + * Get the innerProperties property: Defender for Storage resource properties. + * + * @return the innerProperties value. + */ + private DefenderForStorageSettingProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the isEnabled property: Indicates whether Defender for Storage is enabled on this storage account. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.innerProperties() == null ? null : this.innerProperties().isEnabled(); + } + + /** + * Set the isEnabled property: Indicates whether Defender for Storage is enabled on this storage account. + * + * @param isEnabled the isEnabled value to set. + * @return the DefenderForStorageSettingInner object itself. + */ + public DefenderForStorageSettingInner withIsEnabled(Boolean isEnabled) { + if (this.innerProperties() == null) { + this.innerProperties = new DefenderForStorageSettingProperties(); + } + this.innerProperties().withIsEnabled(isEnabled); + return this; + } + + /** + * Get the overrideSubscriptionLevelSettings property: Indicates whether the settings defined for this storage + * account should override the settings defined for the subscription. + * + * @return the overrideSubscriptionLevelSettings value. + */ + public Boolean overrideSubscriptionLevelSettings() { + return this.innerProperties() == null ? null : this.innerProperties().overrideSubscriptionLevelSettings(); + } + + /** + * Set the overrideSubscriptionLevelSettings property: Indicates whether the settings defined for this storage + * account should override the settings defined for the subscription. + * + * @param overrideSubscriptionLevelSettings the overrideSubscriptionLevelSettings value to set. + * @return the DefenderForStorageSettingInner object itself. + */ + public DefenderForStorageSettingInner + withOverrideSubscriptionLevelSettings(Boolean overrideSubscriptionLevelSettings) { + if (this.innerProperties() == null) { + this.innerProperties = new DefenderForStorageSettingProperties(); + } + this.innerProperties().withOverrideSubscriptionLevelSettings(overrideSubscriptionLevelSettings); + return this; + } + + /** + * Get the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @return the scanResultsEventGridTopicResourceId value. + */ + public String scanResultsEventGridTopicResourceId() { + return this.innerProperties() == null ? null : this.innerProperties().scanResultsEventGridTopicResourceId(); + } + + /** + * Set the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @param scanResultsEventGridTopicResourceId the scanResultsEventGridTopicResourceId value to set. + * @return the DefenderForStorageSettingInner object itself. + */ + public DefenderForStorageSettingInner + withScanResultsEventGridTopicResourceId(String scanResultsEventGridTopicResourceId) { + if (this.innerProperties() == null) { + this.innerProperties = new DefenderForStorageSettingProperties(); + } + this.innerProperties().withScanResultsEventGridTopicResourceId(scanResultsEventGridTopicResourceId); + return this; + } + + /** + * Get the operationStatus property: Upon failure or partial success. Additional data describing Malware Scanning + * enable/disable operation. + * + * @return the operationStatus value. + */ + public OperationStatus operationStatus() { + return this.innerProperties() == null ? null : this.innerProperties().operationStatus(); + } + + /** + * Get the isEnabledMalwareScanningIsEnabled property: Indicates whether On Upload malware scanning should be + * enabled. + * + * @return the isEnabledMalwareScanningIsEnabled value. + */ + public Boolean isEnabledMalwareScanningIsEnabled() { + return this.innerProperties() == null ? null : this.innerProperties().isEnabledMalwareScanningIsEnabled(); + } + + /** + * Set the isEnabledMalwareScanningIsEnabled property: Indicates whether On Upload malware scanning should be + * enabled. + * + * @param isEnabledMalwareScanningIsEnabled the isEnabledMalwareScanningIsEnabled value to set. + * @return the DefenderForStorageSettingInner object itself. + */ + public DefenderForStorageSettingInner + withIsEnabledMalwareScanningIsEnabled(Boolean isEnabledMalwareScanningIsEnabled) { + if (this.innerProperties() == null) { + this.innerProperties = new DefenderForStorageSettingProperties(); + } + this.innerProperties().withIsEnabledMalwareScanningIsEnabled(isEnabledMalwareScanningIsEnabled); + return this; + } + + /** + * Get the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @return the capGBPerMonth value. + */ + public Integer capGBPerMonth() { + return this.innerProperties() == null ? null : this.innerProperties().capGBPerMonth(); + } + + /** + * Set the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @param capGBPerMonth the capGBPerMonth value to set. + * @return the DefenderForStorageSettingInner object itself. + */ + public DefenderForStorageSettingInner withCapGBPerMonth(Integer capGBPerMonth) { + if (this.innerProperties() == null) { + this.innerProperties = new DefenderForStorageSettingProperties(); + } + this.innerProperties().withCapGBPerMonth(capGBPerMonth); + return this; + } + + /** + * Get the isEnabledSensitiveDataDiscoveryIsEnabled property: Indicates whether Sensitive Data Discovery should be + * enabled. + * + * @return the isEnabledSensitiveDataDiscoveryIsEnabled value. + */ + public Boolean isEnabledSensitiveDataDiscoveryIsEnabled() { + return this.innerProperties() == null ? null + : this.innerProperties().isEnabledSensitiveDataDiscoveryIsEnabled(); + } + + /** + * Set the isEnabledSensitiveDataDiscoveryIsEnabled property: Indicates whether Sensitive Data Discovery should be + * enabled. + * + * @param isEnabledSensitiveDataDiscoveryIsEnabled the isEnabledSensitiveDataDiscoveryIsEnabled value to set. + * @return the DefenderForStorageSettingInner object itself. + */ + public DefenderForStorageSettingInner + withIsEnabledSensitiveDataDiscoveryIsEnabled(Boolean isEnabledSensitiveDataDiscoveryIsEnabled) { + if (this.innerProperties() == null) { + this.innerProperties = new DefenderForStorageSettingProperties(); + } + this.innerProperties().withIsEnabledSensitiveDataDiscoveryIsEnabled(isEnabledSensitiveDataDiscoveryIsEnabled); + return this; + } + + /** + * Get the operationStatusSensitiveDataDiscoveryOperationStatus property: Upon failure or partial success. + * Additional data describing Sensitive Data Discovery enable/disable operation. + * + * @return the operationStatusSensitiveDataDiscoveryOperationStatus value. + */ + public OperationStatus operationStatusSensitiveDataDiscoveryOperationStatus() { + return this.innerProperties() == null ? null + : this.innerProperties().operationStatusSensitiveDataDiscoveryOperationStatus(); + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingProperties.java new file mode 100644 index 0000000000000..f462bb803cd01 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DefenderForStorageSettingProperties.java @@ -0,0 +1,237 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.models.OperationStatus; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Defender for Storage resource properties. + */ +@Fluent +public final class DefenderForStorageSettingProperties { + /* + * Indicates whether Defender for Storage is enabled on this storage account. + */ + @JsonProperty(value = "isEnabled") + private Boolean isEnabled; + + /* + * Properties of Malware Scanning. + */ + @JsonProperty(value = "malwareScanning") + private MalwareScanningProperties innerMalwareScanning; + + /* + * Properties of Sensitive Data Discovery. + */ + @JsonProperty(value = "sensitiveDataDiscovery") + private SensitiveDataDiscoveryProperties innerSensitiveDataDiscovery; + + /* + * Indicates whether the settings defined for this storage account should override the settings defined for the + * subscription. + */ + @JsonProperty(value = "overrideSubscriptionLevelSettings") + private Boolean overrideSubscriptionLevelSettings; + + /** + * Creates an instance of DefenderForStorageSettingProperties class. + */ + public DefenderForStorageSettingProperties() { + } + + /** + * Get the isEnabled property: Indicates whether Defender for Storage is enabled on this storage account. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.isEnabled; + } + + /** + * Set the isEnabled property: Indicates whether Defender for Storage is enabled on this storage account. + * + * @param isEnabled the isEnabled value to set. + * @return the DefenderForStorageSettingProperties object itself. + */ + public DefenderForStorageSettingProperties withIsEnabled(Boolean isEnabled) { + this.isEnabled = isEnabled; + return this; + } + + /** + * Get the innerMalwareScanning property: Properties of Malware Scanning. + * + * @return the innerMalwareScanning value. + */ + private MalwareScanningProperties innerMalwareScanning() { + return this.innerMalwareScanning; + } + + /** + * Get the innerSensitiveDataDiscovery property: Properties of Sensitive Data Discovery. + * + * @return the innerSensitiveDataDiscovery value. + */ + private SensitiveDataDiscoveryProperties innerSensitiveDataDiscovery() { + return this.innerSensitiveDataDiscovery; + } + + /** + * Get the overrideSubscriptionLevelSettings property: Indicates whether the settings defined for this storage + * account should override the settings defined for the subscription. + * + * @return the overrideSubscriptionLevelSettings value. + */ + public Boolean overrideSubscriptionLevelSettings() { + return this.overrideSubscriptionLevelSettings; + } + + /** + * Set the overrideSubscriptionLevelSettings property: Indicates whether the settings defined for this storage + * account should override the settings defined for the subscription. + * + * @param overrideSubscriptionLevelSettings the overrideSubscriptionLevelSettings value to set. + * @return the DefenderForStorageSettingProperties object itself. + */ + public DefenderForStorageSettingProperties + withOverrideSubscriptionLevelSettings(Boolean overrideSubscriptionLevelSettings) { + this.overrideSubscriptionLevelSettings = overrideSubscriptionLevelSettings; + return this; + } + + /** + * Get the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @return the scanResultsEventGridTopicResourceId value. + */ + public String scanResultsEventGridTopicResourceId() { + return this.innerMalwareScanning() == null ? null + : this.innerMalwareScanning().scanResultsEventGridTopicResourceId(); + } + + /** + * Set the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @param scanResultsEventGridTopicResourceId the scanResultsEventGridTopicResourceId value to set. + * @return the DefenderForStorageSettingProperties object itself. + */ + public DefenderForStorageSettingProperties + withScanResultsEventGridTopicResourceId(String scanResultsEventGridTopicResourceId) { + if (this.innerMalwareScanning() == null) { + this.innerMalwareScanning = new MalwareScanningProperties(); + } + this.innerMalwareScanning().withScanResultsEventGridTopicResourceId(scanResultsEventGridTopicResourceId); + return this; + } + + /** + * Get the operationStatus property: Upon failure or partial success. Additional data describing Malware Scanning + * enable/disable operation. + * + * @return the operationStatus value. + */ + public OperationStatus operationStatus() { + return this.innerMalwareScanning() == null ? null : this.innerMalwareScanning().operationStatus(); + } + + /** + * Get the isEnabled property: Indicates whether On Upload malware scanning should be enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabledMalwareScanningIsEnabled() { + return this.innerMalwareScanning() == null ? null : this.innerMalwareScanning().isEnabled(); + } + + /** + * Set the isEnabled property: Indicates whether On Upload malware scanning should be enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the DefenderForStorageSettingProperties object itself. + */ + public DefenderForStorageSettingProperties withIsEnabledMalwareScanningIsEnabled(Boolean isEnabled) { + if (this.innerMalwareScanning() == null) { + this.innerMalwareScanning = new MalwareScanningProperties(); + } + this.innerMalwareScanning().withIsEnabled(isEnabled); + return this; + } + + /** + * Get the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @return the capGBPerMonth value. + */ + public Integer capGBPerMonth() { + return this.innerMalwareScanning() == null ? null : this.innerMalwareScanning().capGBPerMonth(); + } + + /** + * Set the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @param capGBPerMonth the capGBPerMonth value to set. + * @return the DefenderForStorageSettingProperties object itself. + */ + public DefenderForStorageSettingProperties withCapGBPerMonth(Integer capGBPerMonth) { + if (this.innerMalwareScanning() == null) { + this.innerMalwareScanning = new MalwareScanningProperties(); + } + this.innerMalwareScanning().withCapGBPerMonth(capGBPerMonth); + return this; + } + + /** + * Get the isEnabled property: Indicates whether Sensitive Data Discovery should be enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabledSensitiveDataDiscoveryIsEnabled() { + return this.innerSensitiveDataDiscovery() == null ? null : this.innerSensitiveDataDiscovery().isEnabled(); + } + + /** + * Set the isEnabled property: Indicates whether Sensitive Data Discovery should be enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the DefenderForStorageSettingProperties object itself. + */ + public DefenderForStorageSettingProperties withIsEnabledSensitiveDataDiscoveryIsEnabled(Boolean isEnabled) { + if (this.innerSensitiveDataDiscovery() == null) { + this.innerSensitiveDataDiscovery = new SensitiveDataDiscoveryProperties(); + } + this.innerSensitiveDataDiscovery().withIsEnabled(isEnabled); + return this; + } + + /** + * Get the operationStatus property: Upon failure or partial success. Additional data describing Sensitive Data + * Discovery enable/disable operation. + * + * @return the operationStatus value. + */ + public OperationStatus operationStatusSensitiveDataDiscoveryOperationStatus() { + return this.innerSensitiveDataDiscovery() == null ? null : this.innerSensitiveDataDiscovery().operationStatus(); + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerMalwareScanning() != null) { + innerMalwareScanning().validate(); + } + if (innerSensitiveDataDiscovery() != null) { + innerSensitiveDataDiscovery().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DevOpsConfigurationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DevOpsConfigurationInner.java new file mode 100644 index 0000000000000..84cd2be215f63 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DevOpsConfigurationInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.DevOpsConfigurationProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * DevOps Configuration resource. + */ +@Fluent +public final class DevOpsConfigurationInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * DevOps Configuration properties. + */ + @JsonProperty(value = "properties") + private DevOpsConfigurationProperties properties; + + /** + * Creates an instance of DevOpsConfigurationInner class. + */ + public DevOpsConfigurationInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: DevOps Configuration properties. + * + * @return the properties value. + */ + public DevOpsConfigurationProperties properties() { + return this.properties; + } + + /** + * Set the properties property: DevOps Configuration properties. + * + * @param properties the properties value to set. + * @return the DevOpsConfigurationInner object itself. + */ + public DevOpsConfigurationInner withProperties(DevOpsConfigurationProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupInner.java index 29aa17350b738..4ac5384380fa6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupInner.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The device security group resource. */ +/** + * The device security group resource. + */ @Fluent public final class DeviceSecurityGroupInner extends ProxyResource { /* @@ -22,13 +24,15 @@ public final class DeviceSecurityGroupInner extends ProxyResource { @JsonProperty(value = "properties") private DeviceSecurityGroupProperties innerProperties; - /** Creates an instance of DeviceSecurityGroupInner class. */ + /** + * Creates an instance of DeviceSecurityGroupInner class. + */ public DeviceSecurityGroupInner() { } /** * Get the innerProperties property: Device Security group data. - * + * * @return the innerProperties value. */ private DeviceSecurityGroupProperties innerProperties() { @@ -37,7 +41,7 @@ private DeviceSecurityGroupProperties innerProperties() { /** * Get the thresholdRules property: The list of custom alert threshold rules. - * + * * @return the thresholdRules value. */ public List thresholdRules() { @@ -46,7 +50,7 @@ public List thresholdRules() { /** * Set the thresholdRules property: The list of custom alert threshold rules. - * + * * @param thresholdRules the thresholdRules value to set. * @return the DeviceSecurityGroupInner object itself. */ @@ -60,7 +64,7 @@ public DeviceSecurityGroupInner withThresholdRules(List timeWindowRules() { @@ -69,7 +73,7 @@ public List timeWindowRules() { /** * Set the timeWindowRules property: The list of custom alert time-window rules. - * + * * @param timeWindowRules the timeWindowRules value to set. * @return the DeviceSecurityGroupInner object itself. */ @@ -83,7 +87,7 @@ public DeviceSecurityGroupInner withTimeWindowRules(List allowlistRules() { @@ -92,7 +96,7 @@ public List allowlistRules() { /** * Set the allowlistRules property: The allow-list custom alert rules. - * + * * @param allowlistRules the allowlistRules value to set. * @return the DeviceSecurityGroupInner object itself. */ @@ -106,7 +110,7 @@ public DeviceSecurityGroupInner withAllowlistRules(List denylistRules() { @@ -115,7 +119,7 @@ public List denylistRules() { /** * Set the denylistRules property: The deny-list custom alert rules. - * + * * @param denylistRules the denylistRules value to set. * @return the DeviceSecurityGroupInner object itself. */ @@ -129,7 +133,7 @@ public DeviceSecurityGroupInner withDenylistRules(List /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupProperties.java index 0ee8520c518c5..89cd036e97e25 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/DeviceSecurityGroupProperties.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** describes properties of a security group. */ +/** + * describes properties of a security group. + */ @Fluent public final class DeviceSecurityGroupProperties { /* @@ -39,13 +41,15 @@ public final class DeviceSecurityGroupProperties { @JsonProperty(value = "denylistRules") private List denylistRules; - /** Creates an instance of DeviceSecurityGroupProperties class. */ + /** + * Creates an instance of DeviceSecurityGroupProperties class. + */ public DeviceSecurityGroupProperties() { } /** * Get the thresholdRules property: The list of custom alert threshold rules. - * + * * @return the thresholdRules value. */ public List thresholdRules() { @@ -54,7 +58,7 @@ public List thresholdRules() { /** * Set the thresholdRules property: The list of custom alert threshold rules. - * + * * @param thresholdRules the thresholdRules value to set. * @return the DeviceSecurityGroupProperties object itself. */ @@ -65,7 +69,7 @@ public DeviceSecurityGroupProperties withThresholdRules(List timeWindowRules() { @@ -74,7 +78,7 @@ public List timeWindowRules() { /** * Set the timeWindowRules property: The list of custom alert time-window rules. - * + * * @param timeWindowRules the timeWindowRules value to set. * @return the DeviceSecurityGroupProperties object itself. */ @@ -85,7 +89,7 @@ public DeviceSecurityGroupProperties withTimeWindowRules(List allowlistRules() { @@ -94,7 +98,7 @@ public List allowlistRules() { /** * Set the allowlistRules property: The allow-list custom alert rules. - * + * * @param allowlistRules the allowlistRules value to set. * @return the DeviceSecurityGroupProperties object itself. */ @@ -105,7 +109,7 @@ public DeviceSecurityGroupProperties withAllowlistRules(List denylistRules() { @@ -114,7 +118,7 @@ public List denylistRules() { /** * Set the denylistRules property: The deny-list custom alert rules. - * + * * @param denylistRules the denylistRules value to set. * @return the DeviceSecurityGroupProperties object itself. */ @@ -125,7 +129,7 @@ public DeviceSecurityGroupProperties withDenylistRules(ListList of details about all the available SKUs. + * A list with a single sensitivity settings resource. */ @Fluent -public final class RegionSkuDetails { +public final class GetSensitivitySettingsListResponseInner { /* - * List of Sku Detail + * The value property. */ @JsonProperty(value = "value") - private List value; + private List value; - /** Creates an instance of RegionSkuDetails class. */ - public RegionSkuDetails() { + /** + * Creates an instance of GetSensitivitySettingsListResponseInner class. + */ + public GetSensitivitySettingsListResponseInner() { } /** - * Get the value property: List of Sku Detail. - * + * Get the value property: The value property. + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Set the value property: List of Sku Detail. - * + * Set the value property: The value property. + * * @param value the value value to set. - * @return the RegionSkuDetails object itself. + * @return the GetSensitivitySettingsListResponseInner object itself. */ - public RegionSkuDetails withValue(List value) { + public GetSensitivitySettingsListResponseInner withValue(List value) { this.value = value; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GetSensitivitySettingsResponseInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GetSensitivitySettingsResponseInner.java new file mode 100644 index 0000000000000..9163e92035027 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GetSensitivitySettingsResponseInner.java @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsResponseProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Data sensitivity settings for sensitive data discovery. + */ +@Fluent +public final class GetSensitivitySettingsResponseInner extends ProxyResource { + /* + * The sensitivity settings properties + */ + @JsonProperty(value = "properties") + private GetSensitivitySettingsResponseProperties properties; + + /** + * Creates an instance of GetSensitivitySettingsResponseInner class. + */ + public GetSensitivitySettingsResponseInner() { + } + + /** + * Get the properties property: The sensitivity settings properties. + * + * @return the properties value. + */ + public GetSensitivitySettingsResponseProperties properties() { + return this.properties; + } + + /** + * Set the properties property: The sensitivity settings properties. + * + * @param properties the properties value to set. + * @return the GetSensitivitySettingsResponseInner object itself. + */ + public GetSensitivitySettingsResponseInner withProperties(GetSensitivitySettingsResponseProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerInner.java new file mode 100644 index 0000000000000..cdc48c7f89f21 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.GitHubOwnerProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * GitHub Owner resource. + */ +@Fluent +public final class GitHubOwnerInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * GitHub Owner properties. + */ + @JsonProperty(value = "properties") + private GitHubOwnerProperties properties; + + /** + * Creates an instance of GitHubOwnerInner class. + */ + public GitHubOwnerInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: GitHub Owner properties. + * + * @return the properties value. + */ + public GitHubOwnerProperties properties() { + return this.properties; + } + + /** + * Set the properties property: GitHub Owner properties. + * + * @param properties the properties value to set. + * @return the GitHubOwnerInner object itself. + */ + public GitHubOwnerInner withProperties(GitHubOwnerProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerListResponseInner.java similarity index 53% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsList.java rename to sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerListResponseInner.java index 3bdae065250b0..37ea6f2cba59d 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/BackupsList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubOwnerListResponseInner.java @@ -2,58 +2,57 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.models; +package com.azure.resourcemanager.security.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.netapp.fluent.models.BackupInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; /** - * List of Backups. + * List of RP resources which supports pagination. */ @Fluent -public final class BackupsList { +public final class GitHubOwnerListResponseInner { /* - * A list of Backups + * Gets or sets list of resources. */ @JsonProperty(value = "value") - private List value; + private List value; /* - * URL to get the next set of results. + * Gets or sets next link to scroll over the results. */ @JsonProperty(value = "nextLink") private String nextLink; /** - * Creates an instance of BackupsList class. + * Creates an instance of GitHubOwnerListResponseInner class. */ - public BackupsList() { + public GitHubOwnerListResponseInner() { } /** - * Get the value property: A list of Backups. + * Get the value property: Gets or sets list of resources. * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Set the value property: A list of Backups. + * Set the value property: Gets or sets list of resources. * * @param value the value value to set. - * @return the BackupsList object itself. + * @return the GitHubOwnerListResponseInner object itself. */ - public BackupsList withValue(List value) { + public GitHubOwnerListResponseInner withValue(List value) { this.value = value; return this; } /** - * Get the nextLink property: URL to get the next set of results. + * Get the nextLink property: Gets or sets next link to scroll over the results. * * @return the nextLink value. */ @@ -62,12 +61,12 @@ public String nextLink() { } /** - * Set the nextLink property: URL to get the next set of results. + * Set the nextLink property: Gets or sets next link to scroll over the results. * * @param nextLink the nextLink value to set. - * @return the BackupsList object itself. + * @return the GitHubOwnerListResponseInner object itself. */ - public BackupsList withNextLink(String nextLink) { + public GitHubOwnerListResponseInner withNextLink(String nextLink) { this.nextLink = nextLink; return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubRepositoryInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubRepositoryInner.java new file mode 100644 index 0000000000000..fdf3cf2876ca4 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitHubRepositoryInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.GitHubRepositoryProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * GitHub Repository resource. + */ +@Fluent +public final class GitHubRepositoryInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * GitHub Repository properties. + */ + @JsonProperty(value = "properties") + private GitHubRepositoryProperties properties; + + /** + * Creates an instance of GitHubRepositoryInner class. + */ + public GitHubRepositoryInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: GitHub Repository properties. + * + * @return the properties value. + */ + public GitHubRepositoryProperties properties() { + return this.properties; + } + + /** + * Set the properties property: GitHub Repository properties. + * + * @param properties the properties value to set. + * @return the GitHubRepositoryInner object itself. + */ + public GitHubRepositoryInner withProperties(GitHubRepositoryProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupInner.java new file mode 100644 index 0000000000000..40c4ee5cb99d1 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.GitLabGroupProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * GitLab Group resource. + */ +@Fluent +public final class GitLabGroupInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * GitLab Group properties. + */ + @JsonProperty(value = "properties") + private GitLabGroupProperties properties; + + /** + * Creates an instance of GitLabGroupInner class. + */ + public GitLabGroupInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: GitLab Group properties. + * + * @return the properties value. + */ + public GitLabGroupProperties properties() { + return this.properties; + } + + /** + * Set the properties property: GitLab Group properties. + * + * @param properties the properties value to set. + * @return the GitLabGroupInner object itself. + */ + public GitLabGroupInner withProperties(GitLabGroupProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfosList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupListResponseInner.java similarity index 53% rename from sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfosList.java rename to sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupListResponseInner.java index 4532e186625bb..12ece067735e2 100644 --- a/sdk/netapp/azure-resourcemanager-netapp/src/main/java/com/azure/resourcemanager/netapp/models/RegionInfosList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabGroupListResponseInner.java @@ -2,58 +2,57 @@ // Licensed under the MIT License. // Code generated by Microsoft (R) AutoRest Code Generator. -package com.azure.resourcemanager.netapp.models; +package com.azure.resourcemanager.security.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.netapp.fluent.models.RegionInfoResourceInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; /** - * List of regionInfo resources. + * List of RP resources which supports pagination. */ @Fluent -public final class RegionInfosList { +public final class GitLabGroupListResponseInner { /* - * A list of regionInfo resources + * Gets or sets list of resources. */ @JsonProperty(value = "value") - private List value; + private List value; /* - * URL to get the next set of results. + * Gets or sets next link to scroll over the results. */ @JsonProperty(value = "nextLink") private String nextLink; /** - * Creates an instance of RegionInfosList class. + * Creates an instance of GitLabGroupListResponseInner class. */ - public RegionInfosList() { + public GitLabGroupListResponseInner() { } /** - * Get the value property: A list of regionInfo resources. + * Get the value property: Gets or sets list of resources. * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** - * Set the value property: A list of regionInfo resources. + * Set the value property: Gets or sets list of resources. * * @param value the value value to set. - * @return the RegionInfosList object itself. + * @return the GitLabGroupListResponseInner object itself. */ - public RegionInfosList withValue(List value) { + public GitLabGroupListResponseInner withValue(List value) { this.value = value; return this; } /** - * Get the nextLink property: URL to get the next set of results. + * Get the nextLink property: Gets or sets next link to scroll over the results. * * @return the nextLink value. */ @@ -62,12 +61,12 @@ public String nextLink() { } /** - * Set the nextLink property: URL to get the next set of results. + * Set the nextLink property: Gets or sets next link to scroll over the results. * * @param nextLink the nextLink value to set. - * @return the RegionInfosList object itself. + * @return the GitLabGroupListResponseInner object itself. */ - public RegionInfosList withNextLink(String nextLink) { + public GitLabGroupListResponseInner withNextLink(String nextLink) { this.nextLink = nextLink; return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabProjectInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabProjectInner.java new file mode 100644 index 0000000000000..646ee832cd081 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GitLabProjectInner.java @@ -0,0 +1,75 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.ProxyResource; +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.models.GitLabProjectProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * GitLab Project resource. + */ +@Fluent +public final class GitLabProjectInner extends ProxyResource { + /* + * Metadata pertaining to creation and last modification of the resource. + */ + @JsonProperty(value = "systemData", access = JsonProperty.Access.WRITE_ONLY) + private SystemData systemData; + + /* + * GitLab Project properties. + */ + @JsonProperty(value = "properties") + private GitLabProjectProperties properties; + + /** + * Creates an instance of GitLabProjectInner class. + */ + public GitLabProjectInner() { + } + + /** + * Get the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + public SystemData systemData() { + return this.systemData; + } + + /** + * Get the properties property: GitLab Project properties. + * + * @return the properties value. + */ + public GitLabProjectProperties properties() { + return this.properties; + } + + /** + * Set the properties property: GitLab Project properties. + * + * @param properties the properties value to set. + * @return the GitLabProjectInner object itself. + */ + public GitLabProjectInner withProperties(GitLabProjectProperties properties) { + this.properties = properties; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (properties() != null) { + properties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentInner.java index f880104179cce..b7ca7624af9a3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentInner.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Governance assignment over a given scope. */ +/** + * Governance assignment over a given scope. + */ @Fluent public final class GovernanceAssignmentInner extends ProxyResource { /* @@ -21,13 +23,15 @@ public final class GovernanceAssignmentInner extends ProxyResource { @JsonProperty(value = "properties") private GovernanceAssignmentProperties innerProperties; - /** Creates an instance of GovernanceAssignmentInner class. */ + /** + * Creates an instance of GovernanceAssignmentInner class. + */ public GovernanceAssignmentInner() { } /** * Get the innerProperties property: The properties of a governance assignment. - * + * * @return the innerProperties value. */ private GovernanceAssignmentProperties innerProperties() { @@ -36,7 +40,7 @@ private GovernanceAssignmentProperties innerProperties() { /** * Get the owner property: The Owner for the governance assignment - e.g. user@contoso.com - see example. - * + * * @return the owner value. */ public String owner() { @@ -45,7 +49,7 @@ public String owner() { /** * Set the owner property: The Owner for the governance assignment - e.g. user@contoso.com - see example. - * + * * @param owner the owner value to set. * @return the GovernanceAssignmentInner object itself. */ @@ -58,9 +62,9 @@ public GovernanceAssignmentInner withOwner(String owner) { } /** - * Get the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected (in - * case of active grace-period). - * + * Get the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected + * (in case of active grace-period). + * * @return the remediationDueDate value. */ public OffsetDateTime remediationDueDate() { @@ -68,9 +72,9 @@ public OffsetDateTime remediationDueDate() { } /** - * Set the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected (in - * case of active grace-period). - * + * Set the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected + * (in case of active grace-period). + * * @param remediationDueDate the remediationDueDate value to set. * @return the GovernanceAssignmentInner object itself. */ @@ -84,7 +88,7 @@ public GovernanceAssignmentInner withRemediationDueDate(OffsetDateTime remediati /** * Get the remediationEta property: The ETA (estimated time of arrival) for remediation (optional), see example. - * + * * @return the remediationEta value. */ public RemediationEta remediationEta() { @@ -93,7 +97,7 @@ public RemediationEta remediationEta() { /** * Set the remediationEta property: The ETA (estimated time of arrival) for remediation (optional), see example. - * + * * @param remediationEta the remediationEta value to set. * @return the GovernanceAssignmentInner object itself. */ @@ -107,7 +111,7 @@ public GovernanceAssignmentInner withRemediationEta(RemediationEta remediationEt /** * Get the isGracePeriod property: Defines whether there is a grace period on the governance assignment. - * + * * @return the isGracePeriod value. */ public Boolean isGracePeriod() { @@ -116,7 +120,7 @@ public Boolean isGracePeriod() { /** * Set the isGracePeriod property: Defines whether there is a grace period on the governance assignment. - * + * * @param isGracePeriod the isGracePeriod value to set. * @return the GovernanceAssignmentInner object itself. */ @@ -131,7 +135,7 @@ public GovernanceAssignmentInner withIsGracePeriod(Boolean isGracePeriod) { /** * Get the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @return the governanceEmailNotification value. */ public GovernanceEmailNotification governanceEmailNotification() { @@ -141,12 +145,12 @@ public GovernanceEmailNotification governanceEmailNotification() { /** * Set the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @param governanceEmailNotification the governanceEmailNotification value to set. * @return the GovernanceAssignmentInner object itself. */ - public GovernanceAssignmentInner withGovernanceEmailNotification( - GovernanceEmailNotification governanceEmailNotification) { + public GovernanceAssignmentInner + withGovernanceEmailNotification(GovernanceEmailNotification governanceEmailNotification) { if (this.innerProperties() == null) { this.innerProperties = new GovernanceAssignmentProperties(); } @@ -157,7 +161,7 @@ public GovernanceAssignmentInner withGovernanceEmailNotification( /** * Get the additionalData property: The additional data for the governance assignment - e.g. links to ticket * (optional), see example. - * + * * @return the additionalData value. */ public GovernanceAssignmentAdditionalData additionalData() { @@ -167,7 +171,7 @@ public GovernanceAssignmentAdditionalData additionalData() { /** * Set the additionalData property: The additional data for the governance assignment - e.g. links to ticket * (optional), see example. - * + * * @param additionalData the additionalData value to set. * @return the GovernanceAssignmentInner object itself. */ @@ -181,7 +185,7 @@ public GovernanceAssignmentInner withAdditionalData(GovernanceAssignmentAddition /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentProperties.java index 4fe799c627982..48d00b1ec88bb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceAssignmentProperties.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** Describes properties of an governance assignment. */ +/** + * Describes properties of an governance assignment. + */ @Fluent public final class GovernanceAssignmentProperties { /* @@ -22,7 +24,7 @@ public final class GovernanceAssignmentProperties { private String owner; /* - * The remediation due-date - after this date Secure Score will be affected (in case of active grace-period) + * The remediation due-date - after this date Secure Score will be affected (in case of active grace-period) */ @JsonProperty(value = "remediationDueDate", required = true) private OffsetDateTime remediationDueDate; @@ -52,13 +54,15 @@ public final class GovernanceAssignmentProperties { @JsonProperty(value = "additionalData") private GovernanceAssignmentAdditionalData additionalData; - /** Creates an instance of GovernanceAssignmentProperties class. */ + /** + * Creates an instance of GovernanceAssignmentProperties class. + */ public GovernanceAssignmentProperties() { } /** * Get the owner property: The Owner for the governance assignment - e.g. user@contoso.com - see example. - * + * * @return the owner value. */ public String owner() { @@ -67,7 +71,7 @@ public String owner() { /** * Set the owner property: The Owner for the governance assignment - e.g. user@contoso.com - see example. - * + * * @param owner the owner value to set. * @return the GovernanceAssignmentProperties object itself. */ @@ -77,9 +81,9 @@ public GovernanceAssignmentProperties withOwner(String owner) { } /** - * Get the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected (in - * case of active grace-period). - * + * Get the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected + * (in case of active grace-period). + * * @return the remediationDueDate value. */ public OffsetDateTime remediationDueDate() { @@ -87,9 +91,9 @@ public OffsetDateTime remediationDueDate() { } /** - * Set the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected (in - * case of active grace-period). - * + * Set the remediationDueDate property: The remediation due-date - after this date Secure Score will be affected + * (in case of active grace-period). + * * @param remediationDueDate the remediationDueDate value to set. * @return the GovernanceAssignmentProperties object itself. */ @@ -100,7 +104,7 @@ public GovernanceAssignmentProperties withRemediationDueDate(OffsetDateTime reme /** * Get the remediationEta property: The ETA (estimated time of arrival) for remediation (optional), see example. - * + * * @return the remediationEta value. */ public RemediationEta remediationEta() { @@ -109,7 +113,7 @@ public RemediationEta remediationEta() { /** * Set the remediationEta property: The ETA (estimated time of arrival) for remediation (optional), see example. - * + * * @param remediationEta the remediationEta value to set. * @return the GovernanceAssignmentProperties object itself. */ @@ -120,7 +124,7 @@ public GovernanceAssignmentProperties withRemediationEta(RemediationEta remediat /** * Get the isGracePeriod property: Defines whether there is a grace period on the governance assignment. - * + * * @return the isGracePeriod value. */ public Boolean isGracePeriod() { @@ -129,7 +133,7 @@ public Boolean isGracePeriod() { /** * Set the isGracePeriod property: Defines whether there is a grace period on the governance assignment. - * + * * @param isGracePeriod the isGracePeriod value to set. * @return the GovernanceAssignmentProperties object itself. */ @@ -141,7 +145,7 @@ public GovernanceAssignmentProperties withIsGracePeriod(Boolean isGracePeriod) { /** * Get the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @return the governanceEmailNotification value. */ public GovernanceEmailNotification governanceEmailNotification() { @@ -151,12 +155,12 @@ public GovernanceEmailNotification governanceEmailNotification() { /** * Set the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @param governanceEmailNotification the governanceEmailNotification value to set. * @return the GovernanceAssignmentProperties object itself. */ - public GovernanceAssignmentProperties withGovernanceEmailNotification( - GovernanceEmailNotification governanceEmailNotification) { + public GovernanceAssignmentProperties + withGovernanceEmailNotification(GovernanceEmailNotification governanceEmailNotification) { this.governanceEmailNotification = governanceEmailNotification; return this; } @@ -164,7 +168,7 @@ public GovernanceAssignmentProperties withGovernanceEmailNotification( /** * Get the additionalData property: The additional data for the governance assignment - e.g. links to ticket * (optional), see example. - * + * * @return the additionalData value. */ public GovernanceAssignmentAdditionalData additionalData() { @@ -174,7 +178,7 @@ public GovernanceAssignmentAdditionalData additionalData() { /** * Set the additionalData property: The additional data for the governance assignment - e.g. links to ticket * (optional), see example. - * + * * @param additionalData the additionalData value to set. * @return the GovernanceAssignmentProperties object itself. */ @@ -185,15 +189,13 @@ public GovernanceAssignmentProperties withAdditionalData(GovernanceAssignmentAdd /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (remediationDueDate() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property remediationDueDate in model GovernanceAssignmentProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property remediationDueDate in model GovernanceAssignmentProperties")); } if (remediationEta() != null) { remediationEta().validate(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleInner.java index 62781d9a3ac87..f8f9c07ca09d4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleInner.java @@ -14,7 +14,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Governance rule over a given scope. */ +/** + * Governance rule over a given scope. + */ @Fluent public final class GovernanceRuleInner extends ProxyResource { /* @@ -23,13 +25,15 @@ public final class GovernanceRuleInner extends ProxyResource { @JsonProperty(value = "properties") private GovernanceRuleProperties innerProperties; - /** Creates an instance of GovernanceRuleInner class. */ + /** + * Creates an instance of GovernanceRuleInner class. + */ public GovernanceRuleInner() { } /** * Get the innerProperties property: Properties of a governance rule. - * + * * @return the innerProperties value. */ private GovernanceRuleProperties innerProperties() { @@ -38,7 +42,7 @@ private GovernanceRuleProperties innerProperties() { /** * Get the tenantId property: The tenantId (GUID). - * + * * @return the tenantId value. */ public String tenantId() { @@ -47,7 +51,7 @@ public String tenantId() { /** * Get the displayName property: Display name of the governance rule. - * + * * @return the displayName value. */ public String displayName() { @@ -56,7 +60,7 @@ public String displayName() { /** * Set the displayName property: Display name of the governance rule. - * + * * @param displayName the displayName value to set. * @return the GovernanceRuleInner object itself. */ @@ -70,7 +74,7 @@ public GovernanceRuleInner withDisplayName(String displayName) { /** * Get the description property: Description of the governance rule. - * + * * @return the description value. */ public String description() { @@ -79,7 +83,7 @@ public String description() { /** * Set the description property: Description of the governance rule. - * + * * @param description the description value to set. * @return the GovernanceRuleInner object itself. */ @@ -94,7 +98,7 @@ public GovernanceRuleInner withDescription(String description) { /** * Get the remediationTimeframe property: Governance rule remediation timeframe - this is the time that will affect * on the grace-period duration e.g. 7.00:00:00 - means 7 days. - * + * * @return the remediationTimeframe value. */ public String remediationTimeframe() { @@ -104,7 +108,7 @@ public String remediationTimeframe() { /** * Set the remediationTimeframe property: Governance rule remediation timeframe - this is the time that will affect * on the grace-period duration e.g. 7.00:00:00 - means 7 days. - * + * * @param remediationTimeframe the remediationTimeframe value to set. * @return the GovernanceRuleInner object itself. */ @@ -118,7 +122,7 @@ public GovernanceRuleInner withRemediationTimeframe(String remediationTimeframe) /** * Get the isGracePeriod property: Defines whether there is a grace period on the governance rule. - * + * * @return the isGracePeriod value. */ public Boolean isGracePeriod() { @@ -127,7 +131,7 @@ public Boolean isGracePeriod() { /** * Set the isGracePeriod property: Defines whether there is a grace period on the governance rule. - * + * * @param isGracePeriod the isGracePeriod value to set. * @return the GovernanceRuleInner object itself. */ @@ -142,7 +146,7 @@ public GovernanceRuleInner withIsGracePeriod(Boolean isGracePeriod) { /** * Get the rulePriority property: The governance rule priority, priority to the lower number. Rules with the same * priority on the same scope will not be allowed. - * + * * @return the rulePriority value. */ public Integer rulePriority() { @@ -152,7 +156,7 @@ public Integer rulePriority() { /** * Set the rulePriority property: The governance rule priority, priority to the lower number. Rules with the same * priority on the same scope will not be allowed. - * + * * @param rulePriority the rulePriority value to set. * @return the GovernanceRuleInner object itself. */ @@ -166,7 +170,7 @@ public GovernanceRuleInner withRulePriority(Integer rulePriority) { /** * Get the isDisabled property: Defines whether the rule is active/inactive. - * + * * @return the isDisabled value. */ public Boolean isDisabled() { @@ -175,7 +179,7 @@ public Boolean isDisabled() { /** * Set the isDisabled property: Defines whether the rule is active/inactive. - * + * * @param isDisabled the isDisabled value to set. * @return the GovernanceRuleInner object itself. */ @@ -189,7 +193,7 @@ public GovernanceRuleInner withIsDisabled(Boolean isDisabled) { /** * Get the ruleType property: The rule type of the governance rule, defines the source of the rule e.g. Integrated. - * + * * @return the ruleType value. */ public GovernanceRuleType ruleType() { @@ -198,7 +202,7 @@ public GovernanceRuleType ruleType() { /** * Set the ruleType property: The rule type of the governance rule, defines the source of the rule e.g. Integrated. - * + * * @param ruleType the ruleType value to set. * @return the GovernanceRuleInner object itself. */ @@ -212,7 +216,7 @@ public GovernanceRuleInner withRuleType(GovernanceRuleType ruleType) { /** * Get the sourceResourceType property: The governance rule source, what the rule affects, e.g. Assessments. - * + * * @return the sourceResourceType value. */ public GovernanceRuleSourceResourceType sourceResourceType() { @@ -221,7 +225,7 @@ public GovernanceRuleSourceResourceType sourceResourceType() { /** * Set the sourceResourceType property: The governance rule source, what the rule affects, e.g. Assessments. - * + * * @param sourceResourceType the sourceResourceType value to set. * @return the GovernanceRuleInner object itself. */ @@ -234,8 +238,9 @@ public GovernanceRuleInner withSourceResourceType(GovernanceRuleSourceResourceTy } /** - * Get the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management scopes). - * + * Get the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management + * scopes). + * * @return the excludedScopes value. */ public List excludedScopes() { @@ -243,8 +248,9 @@ public List excludedScopes() { } /** - * Set the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management scopes). - * + * Set the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management + * scopes). + * * @param excludedScopes the excludedScopes value to set. * @return the GovernanceRuleInner object itself. */ @@ -258,7 +264,7 @@ public GovernanceRuleInner withExcludedScopes(List excludedScopes) { /** * Get the conditionSets property: The governance rule conditionSets - see examples. - * + * * @return the conditionSets value. */ public List conditionSets() { @@ -267,7 +273,7 @@ public List conditionSets() { /** * Set the conditionSets property: The governance rule conditionSets - see examples. - * + * * @param conditionSets the conditionSets value to set. * @return the GovernanceRuleInner object itself. */ @@ -282,7 +288,7 @@ public GovernanceRuleInner withConditionSets(List conditionSets) { /** * Get the includeMemberScopes property: Defines whether the rule is management scope rule (master connector as a * single scope or management scope). - * + * * @return the includeMemberScopes value. */ public Boolean includeMemberScopes() { @@ -292,7 +298,7 @@ public Boolean includeMemberScopes() { /** * Set the includeMemberScopes property: Defines whether the rule is management scope rule (master connector as a * single scope or management scope). - * + * * @param includeMemberScopes the includeMemberScopes value to set. * @return the GovernanceRuleInner object itself. */ @@ -307,7 +313,7 @@ public GovernanceRuleInner withIncludeMemberScopes(Boolean includeMemberScopes) /** * Get the ownerSource property: The owner source for the governance rule - e.g. Manually by user@contoso.com - see * example. - * + * * @return the ownerSource value. */ public GovernanceRuleOwnerSource ownerSource() { @@ -317,7 +323,7 @@ public GovernanceRuleOwnerSource ownerSource() { /** * Set the ownerSource property: The owner source for the governance rule - e.g. Manually by user@contoso.com - see * example. - * + * * @param ownerSource the ownerSource value to set. * @return the GovernanceRuleInner object itself. */ @@ -332,7 +338,7 @@ public GovernanceRuleInner withOwnerSource(GovernanceRuleOwnerSource ownerSource /** * Get the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @return the governanceEmailNotification value. */ public GovernanceRuleEmailNotification governanceEmailNotification() { @@ -342,12 +348,12 @@ public GovernanceRuleEmailNotification governanceEmailNotification() { /** * Set the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @param governanceEmailNotification the governanceEmailNotification value to set. * @return the GovernanceRuleInner object itself. */ - public GovernanceRuleInner withGovernanceEmailNotification( - GovernanceRuleEmailNotification governanceEmailNotification) { + public GovernanceRuleInner + withGovernanceEmailNotification(GovernanceRuleEmailNotification governanceEmailNotification) { if (this.innerProperties() == null) { this.innerProperties = new GovernanceRuleProperties(); } @@ -357,7 +363,7 @@ public GovernanceRuleInner withGovernanceEmailNotification( /** * Get the metadata property: The governance rule metadata. - * + * * @return the metadata value. */ public GovernanceRuleMetadata metadata() { @@ -366,7 +372,7 @@ public GovernanceRuleMetadata metadata() { /** * Set the metadata property: The governance rule metadata. - * + * * @param metadata the metadata value to set. * @return the GovernanceRuleInner object itself. */ @@ -380,7 +386,7 @@ public GovernanceRuleInner withMetadata(GovernanceRuleMetadata metadata) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleProperties.java index d54629e94239b..8840e80b5b41e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/GovernanceRuleProperties.java @@ -14,7 +14,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes properties of an governance rule. */ +/** + * Describes properties of an governance rule. + */ @Fluent public final class GovernanceRuleProperties { /* @@ -110,13 +112,15 @@ public final class GovernanceRuleProperties { @JsonProperty(value = "metadata") private GovernanceRuleMetadata metadata; - /** Creates an instance of GovernanceRuleProperties class. */ + /** + * Creates an instance of GovernanceRuleProperties class. + */ public GovernanceRuleProperties() { } /** * Get the tenantId property: The tenantId (GUID). - * + * * @return the tenantId value. */ public String tenantId() { @@ -125,7 +129,7 @@ public String tenantId() { /** * Get the displayName property: Display name of the governance rule. - * + * * @return the displayName value. */ public String displayName() { @@ -134,7 +138,7 @@ public String displayName() { /** * Set the displayName property: Display name of the governance rule. - * + * * @param displayName the displayName value to set. * @return the GovernanceRuleProperties object itself. */ @@ -145,7 +149,7 @@ public GovernanceRuleProperties withDisplayName(String displayName) { /** * Get the description property: Description of the governance rule. - * + * * @return the description value. */ public String description() { @@ -154,7 +158,7 @@ public String description() { /** * Set the description property: Description of the governance rule. - * + * * @param description the description value to set. * @return the GovernanceRuleProperties object itself. */ @@ -166,7 +170,7 @@ public GovernanceRuleProperties withDescription(String description) { /** * Get the remediationTimeframe property: Governance rule remediation timeframe - this is the time that will affect * on the grace-period duration e.g. 7.00:00:00 - means 7 days. - * + * * @return the remediationTimeframe value. */ public String remediationTimeframe() { @@ -176,7 +180,7 @@ public String remediationTimeframe() { /** * Set the remediationTimeframe property: Governance rule remediation timeframe - this is the time that will affect * on the grace-period duration e.g. 7.00:00:00 - means 7 days. - * + * * @param remediationTimeframe the remediationTimeframe value to set. * @return the GovernanceRuleProperties object itself. */ @@ -187,7 +191,7 @@ public GovernanceRuleProperties withRemediationTimeframe(String remediationTimef /** * Get the isGracePeriod property: Defines whether there is a grace period on the governance rule. - * + * * @return the isGracePeriod value. */ public Boolean isGracePeriod() { @@ -196,7 +200,7 @@ public Boolean isGracePeriod() { /** * Set the isGracePeriod property: Defines whether there is a grace period on the governance rule. - * + * * @param isGracePeriod the isGracePeriod value to set. * @return the GovernanceRuleProperties object itself. */ @@ -208,7 +212,7 @@ public GovernanceRuleProperties withIsGracePeriod(Boolean isGracePeriod) { /** * Get the rulePriority property: The governance rule priority, priority to the lower number. Rules with the same * priority on the same scope will not be allowed. - * + * * @return the rulePriority value. */ public int rulePriority() { @@ -218,7 +222,7 @@ public int rulePriority() { /** * Set the rulePriority property: The governance rule priority, priority to the lower number. Rules with the same * priority on the same scope will not be allowed. - * + * * @param rulePriority the rulePriority value to set. * @return the GovernanceRuleProperties object itself. */ @@ -229,7 +233,7 @@ public GovernanceRuleProperties withRulePriority(int rulePriority) { /** * Get the isDisabled property: Defines whether the rule is active/inactive. - * + * * @return the isDisabled value. */ public Boolean isDisabled() { @@ -238,7 +242,7 @@ public Boolean isDisabled() { /** * Set the isDisabled property: Defines whether the rule is active/inactive. - * + * * @param isDisabled the isDisabled value to set. * @return the GovernanceRuleProperties object itself. */ @@ -249,7 +253,7 @@ public GovernanceRuleProperties withIsDisabled(Boolean isDisabled) { /** * Get the ruleType property: The rule type of the governance rule, defines the source of the rule e.g. Integrated. - * + * * @return the ruleType value. */ public GovernanceRuleType ruleType() { @@ -258,7 +262,7 @@ public GovernanceRuleType ruleType() { /** * Set the ruleType property: The rule type of the governance rule, defines the source of the rule e.g. Integrated. - * + * * @param ruleType the ruleType value to set. * @return the GovernanceRuleProperties object itself. */ @@ -269,7 +273,7 @@ public GovernanceRuleProperties withRuleType(GovernanceRuleType ruleType) { /** * Get the sourceResourceType property: The governance rule source, what the rule affects, e.g. Assessments. - * + * * @return the sourceResourceType value. */ public GovernanceRuleSourceResourceType sourceResourceType() { @@ -278,7 +282,7 @@ public GovernanceRuleSourceResourceType sourceResourceType() { /** * Set the sourceResourceType property: The governance rule source, what the rule affects, e.g. Assessments. - * + * * @param sourceResourceType the sourceResourceType value to set. * @return the GovernanceRuleProperties object itself. */ @@ -288,8 +292,9 @@ public GovernanceRuleProperties withSourceResourceType(GovernanceRuleSourceResou } /** - * Get the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management scopes). - * + * Get the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management + * scopes). + * * @return the excludedScopes value. */ public List excludedScopes() { @@ -297,8 +302,9 @@ public List excludedScopes() { } /** - * Set the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management scopes). - * + * Set the excludedScopes property: Excluded scopes, filter out the descendants of the scope (on management + * scopes). + * * @param excludedScopes the excludedScopes value to set. * @return the GovernanceRuleProperties object itself. */ @@ -309,7 +315,7 @@ public GovernanceRuleProperties withExcludedScopes(List excludedScopes) /** * Get the conditionSets property: The governance rule conditionSets - see examples. - * + * * @return the conditionSets value. */ public List conditionSets() { @@ -318,7 +324,7 @@ public List conditionSets() { /** * Set the conditionSets property: The governance rule conditionSets - see examples. - * + * * @param conditionSets the conditionSets value to set. * @return the GovernanceRuleProperties object itself. */ @@ -330,7 +336,7 @@ public GovernanceRuleProperties withConditionSets(List conditionSets) { /** * Get the includeMemberScopes property: Defines whether the rule is management scope rule (master connector as a * single scope or management scope). - * + * * @return the includeMemberScopes value. */ public Boolean includeMemberScopes() { @@ -340,7 +346,7 @@ public Boolean includeMemberScopes() { /** * Set the includeMemberScopes property: Defines whether the rule is management scope rule (master connector as a * single scope or management scope). - * + * * @param includeMemberScopes the includeMemberScopes value to set. * @return the GovernanceRuleProperties object itself. */ @@ -352,7 +358,7 @@ public GovernanceRuleProperties withIncludeMemberScopes(Boolean includeMemberSco /** * Get the ownerSource property: The owner source for the governance rule - e.g. Manually by user@contoso.com - see * example. - * + * * @return the ownerSource value. */ public GovernanceRuleOwnerSource ownerSource() { @@ -362,7 +368,7 @@ public GovernanceRuleOwnerSource ownerSource() { /** * Set the ownerSource property: The owner source for the governance rule - e.g. Manually by user@contoso.com - see * example. - * + * * @param ownerSource the ownerSource value to set. * @return the GovernanceRuleProperties object itself. */ @@ -374,7 +380,7 @@ public GovernanceRuleProperties withOwnerSource(GovernanceRuleOwnerSource ownerS /** * Get the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @return the governanceEmailNotification value. */ public GovernanceRuleEmailNotification governanceEmailNotification() { @@ -384,19 +390,19 @@ public GovernanceRuleEmailNotification governanceEmailNotification() { /** * Set the governanceEmailNotification property: The email notifications settings for the governance rule, states * whether to disable notifications for mangers and owners. - * + * * @param governanceEmailNotification the governanceEmailNotification value to set. * @return the GovernanceRuleProperties object itself. */ - public GovernanceRuleProperties withGovernanceEmailNotification( - GovernanceRuleEmailNotification governanceEmailNotification) { + public GovernanceRuleProperties + withGovernanceEmailNotification(GovernanceRuleEmailNotification governanceEmailNotification) { this.governanceEmailNotification = governanceEmailNotification; return this; } /** * Get the metadata property: The governance rule metadata. - * + * * @return the metadata value. */ public GovernanceRuleMetadata metadata() { @@ -405,7 +411,7 @@ public GovernanceRuleMetadata metadata() { /** * Set the metadata property: The governance rule metadata. - * + * * @param metadata the metadata value to set. * @return the GovernanceRuleProperties object itself. */ @@ -416,39 +422,29 @@ public GovernanceRuleProperties withMetadata(GovernanceRuleMetadata metadata) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (displayName() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property displayName in model GovernanceRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property displayName in model GovernanceRuleProperties")); } if (ruleType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property ruleType in model GovernanceRuleProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property ruleType in model GovernanceRuleProperties")); } if (sourceResourceType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property sourceResourceType in model GovernanceRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property sourceResourceType in model GovernanceRuleProperties")); } if (conditionSets() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property conditionSets in model GovernanceRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property conditionSets in model GovernanceRuleProperties")); } if (ownerSource() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property ownerSource in model GovernanceRuleProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property ownerSource in model GovernanceRuleProperties")); } else { ownerSource().validate(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportInner.java index 31f24ba0c299b..75b77db58ad4f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportInner.java @@ -13,8 +13,11 @@ import com.azure.resourcemanager.security.models.StatusAutoGenerated; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; +import java.util.Map; -/** The health report resource. */ +/** + * The health report resource. + */ @Fluent public final class HealthReportInner extends ProxyResource { /* @@ -23,13 +26,15 @@ public final class HealthReportInner extends ProxyResource { @JsonProperty(value = "properties") private HealthReportProperties innerProperties; - /** Creates an instance of HealthReportInner class. */ + /** + * Creates an instance of HealthReportInner class. + */ public HealthReportInner() { } /** * Get the innerProperties property: Properties of a health report. - * + * * @return the innerProperties value. */ private HealthReportProperties innerProperties() { @@ -38,7 +43,7 @@ private HealthReportProperties innerProperties() { /** * Get the resourceDetails property: The resource details of the health report. - * + * * @return the resourceDetails value. */ public ResourceDetailsAutoGenerated resourceDetails() { @@ -47,7 +52,7 @@ public ResourceDetailsAutoGenerated resourceDetails() { /** * Set the resourceDetails property: The resource details of the health report. - * + * * @param resourceDetails the resourceDetails value to set. * @return the HealthReportInner object itself. */ @@ -61,7 +66,7 @@ public HealthReportInner withResourceDetails(ResourceDetailsAutoGenerated resour /** * Get the environmentDetails property: The environment details of the resource. - * + * * @return the environmentDetails value. */ public EnvironmentDetails environmentDetails() { @@ -70,7 +75,7 @@ public EnvironmentDetails environmentDetails() { /** * Set the environmentDetails property: The environment details of the resource. - * + * * @param environmentDetails the environmentDetails value to set. * @return the HealthReportInner object itself. */ @@ -84,7 +89,7 @@ public HealthReportInner withEnvironmentDetails(EnvironmentDetails environmentDe /** * Get the healthDataClassification property: The classification of the health report. - * + * * @return the healthDataClassification value. */ public HealthDataClassification healthDataClassification() { @@ -93,7 +98,7 @@ public HealthDataClassification healthDataClassification() { /** * Set the healthDataClassification property: The classification of the health report. - * + * * @param healthDataClassification the healthDataClassification value to set. * @return the HealthReportInner object itself. */ @@ -107,7 +112,7 @@ public HealthReportInner withHealthDataClassification(HealthDataClassification h /** * Get the status property: The status of the health report. - * + * * @return the status value. */ public StatusAutoGenerated status() { @@ -116,7 +121,7 @@ public StatusAutoGenerated status() { /** * Set the status property: The status of the health report. - * + * * @param status the status value to set. * @return the HealthReportInner object itself. */ @@ -130,7 +135,7 @@ public HealthReportInner withStatus(StatusAutoGenerated status) { /** * Get the affectedDefendersPlans property: The affected defenders plans by unhealthy report. - * + * * @return the affectedDefendersPlans value. */ public List affectedDefendersPlans() { @@ -139,7 +144,7 @@ public List affectedDefendersPlans() { /** * Set the affectedDefendersPlans property: The affected defenders plans by unhealthy report. - * + * * @param affectedDefendersPlans the affectedDefendersPlans value to set. * @return the HealthReportInner object itself. */ @@ -151,9 +156,42 @@ public HealthReportInner withAffectedDefendersPlans(List affectedDefende return this; } + /** + * Get the affectedDefendersSubPlans property: The affected defenders sub plans by unhealthy report. + * + * @return the affectedDefendersSubPlans value. + */ + public List affectedDefendersSubPlans() { + return this.innerProperties() == null ? null : this.innerProperties().affectedDefendersSubPlans(); + } + + /** + * Set the affectedDefendersSubPlans property: The affected defenders sub plans by unhealthy report. + * + * @param affectedDefendersSubPlans the affectedDefendersSubPlans value to set. + * @return the HealthReportInner object itself. + */ + public HealthReportInner withAffectedDefendersSubPlans(List affectedDefendersSubPlans) { + if (this.innerProperties() == null) { + this.innerProperties = new HealthReportProperties(); + } + this.innerProperties().withAffectedDefendersSubPlans(affectedDefendersSubPlans); + return this; + } + + /** + * Get the reportAdditionalData property: Additional data for the given health report, this field can include more + * details on the resource and the health scenario. + * + * @return the reportAdditionalData value. + */ + public Map reportAdditionalData() { + return this.innerProperties() == null ? null : this.innerProperties().reportAdditionalData(); + } + /** * Get the issues property: A collection of the issues in the report. - * + * * @return the issues value. */ public List issues() { @@ -162,7 +200,7 @@ public List issues() { /** * Set the issues property: A collection of the issues in the report. - * + * * @param issues the issues value to set. * @return the HealthReportInner object itself. */ @@ -176,7 +214,7 @@ public HealthReportInner withIssues(List issues) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportProperties.java index 98ee1832e128c..e503e82693b24 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/HealthReportProperties.java @@ -10,10 +10,14 @@ import com.azure.resourcemanager.security.models.Issue; import com.azure.resourcemanager.security.models.ResourceDetailsAutoGenerated; import com.azure.resourcemanager.security.models.StatusAutoGenerated; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; +import java.util.Map; -/** Describes properties of the health report. */ +/** + * Describes properties of the health report. + */ @Fluent public final class HealthReportProperties { /* @@ -46,19 +50,35 @@ public final class HealthReportProperties { @JsonProperty(value = "affectedDefendersPlans") private List affectedDefendersPlans; + /* + * The affected defenders sub plans by unhealthy report + */ + @JsonProperty(value = "affectedDefendersSubPlans") + private List affectedDefendersSubPlans; + + /* + * Additional data for the given health report, this field can include more details on the resource and the health + * scenario. + */ + @JsonProperty(value = "reportAdditionalData", access = JsonProperty.Access.WRITE_ONLY) + @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) + private Map reportAdditionalData; + /* * A collection of the issues in the report */ @JsonProperty(value = "issues") private List issues; - /** Creates an instance of HealthReportProperties class. */ + /** + * Creates an instance of HealthReportProperties class. + */ public HealthReportProperties() { } /** * Get the resourceDetails property: The resource details of the health report. - * + * * @return the resourceDetails value. */ public ResourceDetailsAutoGenerated resourceDetails() { @@ -67,7 +87,7 @@ public ResourceDetailsAutoGenerated resourceDetails() { /** * Set the resourceDetails property: The resource details of the health report. - * + * * @param resourceDetails the resourceDetails value to set. * @return the HealthReportProperties object itself. */ @@ -78,7 +98,7 @@ public HealthReportProperties withResourceDetails(ResourceDetailsAutoGenerated r /** * Get the environmentDetails property: The environment details of the resource. - * + * * @return the environmentDetails value. */ public EnvironmentDetails environmentDetails() { @@ -87,7 +107,7 @@ public EnvironmentDetails environmentDetails() { /** * Set the environmentDetails property: The environment details of the resource. - * + * * @param environmentDetails the environmentDetails value to set. * @return the HealthReportProperties object itself. */ @@ -98,7 +118,7 @@ public HealthReportProperties withEnvironmentDetails(EnvironmentDetails environm /** * Get the healthDataClassification property: The classification of the health report. - * + * * @return the healthDataClassification value. */ public HealthDataClassification healthDataClassification() { @@ -107,7 +127,7 @@ public HealthDataClassification healthDataClassification() { /** * Set the healthDataClassification property: The classification of the health report. - * + * * @param healthDataClassification the healthDataClassification value to set. * @return the HealthReportProperties object itself. */ @@ -118,7 +138,7 @@ public HealthReportProperties withHealthDataClassification(HealthDataClassificat /** * Get the status property: The status of the health report. - * + * * @return the status value. */ public StatusAutoGenerated status() { @@ -127,7 +147,7 @@ public StatusAutoGenerated status() { /** * Set the status property: The status of the health report. - * + * * @param status the status value to set. * @return the HealthReportProperties object itself. */ @@ -138,7 +158,7 @@ public HealthReportProperties withStatus(StatusAutoGenerated status) { /** * Get the affectedDefendersPlans property: The affected defenders plans by unhealthy report. - * + * * @return the affectedDefendersPlans value. */ public List affectedDefendersPlans() { @@ -147,7 +167,7 @@ public List affectedDefendersPlans() { /** * Set the affectedDefendersPlans property: The affected defenders plans by unhealthy report. - * + * * @param affectedDefendersPlans the affectedDefendersPlans value to set. * @return the HealthReportProperties object itself. */ @@ -156,9 +176,39 @@ public HealthReportProperties withAffectedDefendersPlans(List affectedDe return this; } + /** + * Get the affectedDefendersSubPlans property: The affected defenders sub plans by unhealthy report. + * + * @return the affectedDefendersSubPlans value. + */ + public List affectedDefendersSubPlans() { + return this.affectedDefendersSubPlans; + } + + /** + * Set the affectedDefendersSubPlans property: The affected defenders sub plans by unhealthy report. + * + * @param affectedDefendersSubPlans the affectedDefendersSubPlans value to set. + * @return the HealthReportProperties object itself. + */ + public HealthReportProperties withAffectedDefendersSubPlans(List affectedDefendersSubPlans) { + this.affectedDefendersSubPlans = affectedDefendersSubPlans; + return this; + } + + /** + * Get the reportAdditionalData property: Additional data for the given health report, this field can include more + * details on the resource and the health scenario. + * + * @return the reportAdditionalData value. + */ + public Map reportAdditionalData() { + return this.reportAdditionalData; + } + /** * Get the issues property: A collection of the issues in the report. - * + * * @return the issues value. */ public List issues() { @@ -167,7 +217,7 @@ public List issues() { /** * Set the issues property: A collection of the issues in the report. - * + * * @param issues the issues value to set. * @return the HealthReportProperties object itself. */ @@ -178,7 +228,7 @@ public HealthReportProperties withIssues(List issues) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/InformationProtectionPolicyInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/InformationProtectionPolicyInner.java index f146e90386f24..675a449f3fd5d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/InformationProtectionPolicyInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/InformationProtectionPolicyInner.java @@ -12,7 +12,9 @@ import java.time.OffsetDateTime; import java.util.Map; -/** Information protection policy. */ +/** + * Information protection policy. + */ @Fluent public final class InformationProtectionPolicyInner extends ProxyResource { /* @@ -21,13 +23,15 @@ public final class InformationProtectionPolicyInner extends ProxyResource { @JsonProperty(value = "properties") private InformationProtectionPolicyProperties innerProperties; - /** Creates an instance of InformationProtectionPolicyInner class. */ + /** + * Creates an instance of InformationProtectionPolicyInner class. + */ public InformationProtectionPolicyInner() { } /** * Get the innerProperties property: Information protection policy data. - * + * * @return the innerProperties value. */ private InformationProtectionPolicyProperties innerProperties() { @@ -36,7 +40,7 @@ private InformationProtectionPolicyProperties innerProperties() { /** * Get the lastModifiedUtc property: Describes the last UTC time the policy was modified. - * + * * @return the lastModifiedUtc value. */ public OffsetDateTime lastModifiedUtc() { @@ -45,7 +49,7 @@ public OffsetDateTime lastModifiedUtc() { /** * Get the version property: Describes the version of the policy. - * + * * @return the version value. */ public String version() { @@ -54,7 +58,7 @@ public String version() { /** * Get the labels property: Dictionary of sensitivity labels. - * + * * @return the labels value. */ public Map labels() { @@ -63,7 +67,7 @@ public Map labels() { /** * Set the labels property: Dictionary of sensitivity labels. - * + * * @param labels the labels value to set. * @return the InformationProtectionPolicyInner object itself. */ @@ -77,7 +81,7 @@ public InformationProtectionPolicyInner withLabels(Map /** * Get the informationTypes property: The sensitivity information types. - * + * * @return the informationTypes value. */ public Map informationTypes() { @@ -86,7 +90,7 @@ public Map informationTypes() { /** * Set the informationTypes property: The sensitivity information types. - * + * * @param informationTypes the informationTypes value to set. * @return the InformationProtectionPolicyInner object itself. */ @@ -100,7 +104,7 @@ public InformationProtectionPolicyInner withInformationTypes(Map informationTypes; - /** Creates an instance of InformationProtectionPolicyProperties class. */ + /** + * Creates an instance of InformationProtectionPolicyProperties class. + */ public InformationProtectionPolicyProperties() { } /** * Get the lastModifiedUtc property: Describes the last UTC time the policy was modified. - * + * * @return the lastModifiedUtc value. */ public OffsetDateTime lastModifiedUtc() { @@ -56,7 +60,7 @@ public OffsetDateTime lastModifiedUtc() { /** * Get the version property: Describes the version of the policy. - * + * * @return the version value. */ public String version() { @@ -65,7 +69,7 @@ public String version() { /** * Get the labels property: Dictionary of sensitivity labels. - * + * * @return the labels value. */ public Map labels() { @@ -74,7 +78,7 @@ public Map labels() { /** * Set the labels property: Dictionary of sensitivity labels. - * + * * @param labels the labels value to set. * @return the InformationProtectionPolicyProperties object itself. */ @@ -85,7 +89,7 @@ public InformationProtectionPolicyProperties withLabels(Map informationTypes() { @@ -94,7 +98,7 @@ public Map informationTypes() { /** * Set the informationTypes property: The sensitivity information types. - * + * * @param informationTypes the informationTypes value to set. * @return the InformationProtectionPolicyProperties object itself. */ @@ -105,29 +109,23 @@ public InformationProtectionPolicyProperties withInformationTypes(Map { - if (e != null) { - e.validate(); - } - }); + labels().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } if (informationTypes() != null) { - informationTypes() - .values() - .forEach( - e -> { - if (e != null) { - e.validate(); - } - }); + informationTypes().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingInner.java deleted file mode 100644 index 6bf368c4d2018..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingInner.java +++ /dev/null @@ -1,51 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent.models; - -import com.azure.core.annotation.Fluent; -import com.azure.core.management.ProxyResource; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** Configures how to correlate scan data and logs with resources associated with the subscription. */ -@Fluent -public final class IngestionSettingInner extends ProxyResource { - /* - * Ingestion setting data - */ - @JsonProperty(value = "properties") - private Object properties; - - /** Creates an instance of IngestionSettingInner class. */ - public IngestionSettingInner() { - } - - /** - * Get the properties property: Ingestion setting data. - * - * @return the properties value. - */ - public Object properties() { - return this.properties; - } - - /** - * Set the properties property: Ingestion setting data. - * - * @param properties the properties value to set. - * @return the IngestionSettingInner object itself. - */ - public IngestionSettingInner withProperties(Object properties) { - this.properties = properties; - return this; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingTokenInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingTokenInner.java deleted file mode 100644 index dd95c8c94ce6d..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IngestionSettingTokenInner.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.fluent.models; - -import com.azure.core.annotation.Immutable; -import com.fasterxml.jackson.annotation.JsonProperty; - -/** Configures how to correlate scan data and logs with resources associated with the subscription. */ -@Immutable -public final class IngestionSettingTokenInner { - /* - * The token is used for correlating security data and logs with the resources in the subscription. - */ - @JsonProperty(value = "token", access = JsonProperty.Access.WRITE_ONLY) - private String token; - - /** Creates an instance of IngestionSettingTokenInner class. */ - public IngestionSettingTokenInner() { - } - - /** - * Get the token property: The token is used for correlating security data and logs with the resources in the - * subscription. - * - * @return the token value. - */ - public String token() { - return this.token; - } - - /** - * Validates the instance. - * - * @throws IllegalArgumentException thrown if the instance is not valid. - */ - public void validate() { - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertInner.java index d6ac39d9089d5..6e0d7744ab16d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertInner.java @@ -14,7 +14,9 @@ import java.util.List; import java.util.Map; -/** Security Solution Aggregated Alert information. */ +/** + * Security Solution Aggregated Alert information. + */ @Fluent public final class IoTSecurityAggregatedAlertInner extends ProxyResource { /* @@ -30,13 +32,15 @@ public final class IoTSecurityAggregatedAlertInner extends ProxyResource { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of IoTSecurityAggregatedAlertInner class. */ + /** + * Creates an instance of IoTSecurityAggregatedAlertInner class. + */ public IoTSecurityAggregatedAlertInner() { } /** * Get the innerProperties property: IoT Security solution aggregated alert details. - * + * * @return the innerProperties value. */ private IoTSecurityAggregatedAlertProperties innerProperties() { @@ -45,7 +49,7 @@ private IoTSecurityAggregatedAlertProperties innerProperties() { /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -54,7 +58,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the IoTSecurityAggregatedAlertInner object itself. */ @@ -65,7 +69,7 @@ public IoTSecurityAggregatedAlertInner withTags(Map tags) { /** * Get the alertType property: Name of the alert type. - * + * * @return the alertType value. */ public String alertType() { @@ -74,7 +78,7 @@ public String alertType() { /** * Get the alertDisplayName property: Display name of the alert type. - * + * * @return the alertDisplayName value. */ public String alertDisplayName() { @@ -83,7 +87,7 @@ public String alertDisplayName() { /** * Get the aggregatedDateUtc property: Date of detection. - * + * * @return the aggregatedDateUtc value. */ public LocalDate aggregatedDateUtc() { @@ -92,7 +96,7 @@ public LocalDate aggregatedDateUtc() { /** * Get the vendorName property: Name of the organization that raised the alert. - * + * * @return the vendorName value. */ public String vendorName() { @@ -101,7 +105,7 @@ public String vendorName() { /** * Get the reportedSeverity property: Assessed alert severity. - * + * * @return the reportedSeverity value. */ public ReportedSeverity reportedSeverity() { @@ -110,7 +114,7 @@ public ReportedSeverity reportedSeverity() { /** * Get the remediationSteps property: Recommended steps for remediation. - * + * * @return the remediationSteps value. */ public String remediationSteps() { @@ -119,7 +123,7 @@ public String remediationSteps() { /** * Get the description property: Description of the suspected vulnerability and meaning. - * + * * @return the description value. */ public String description() { @@ -128,7 +132,7 @@ public String description() { /** * Get the count property: Number of alerts occurrences within the aggregated time window. - * + * * @return the count value. */ public Long count() { @@ -137,7 +141,7 @@ public Long count() { /** * Get the effectedResourceType property: Azure resource ID of the resource that received the alerts. - * + * * @return the effectedResourceType value. */ public String effectedResourceType() { @@ -146,7 +150,7 @@ public String effectedResourceType() { /** * Get the systemSource property: The type of the alerted resource (Azure, Non-Azure). - * + * * @return the systemSource value. */ public String systemSource() { @@ -155,7 +159,7 @@ public String systemSource() { /** * Get the actionTaken property: IoT Security solution alert response. - * + * * @return the actionTaken value. */ public String actionTaken() { @@ -164,7 +168,7 @@ public String actionTaken() { /** * Get the logAnalyticsQuery property: Log analytics query for getting the list of affected devices/alerts. - * + * * @return the logAnalyticsQuery value. */ public String logAnalyticsQuery() { @@ -174,7 +178,7 @@ public String logAnalyticsQuery() { /** * Get the topDevicesList property: 10 devices with the highest number of occurrences of this alert type, on this * day. - * + * * @return the topDevicesList value. */ public List topDevicesList() { @@ -183,7 +187,7 @@ public List topDevicesLi /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertProperties.java index 7413ba8442dbd..98c98aaa7f3b5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedAlertProperties.java @@ -11,7 +11,9 @@ import java.time.LocalDate; import java.util.List; -/** IoT Security solution aggregated alert details. */ +/** + * IoT Security solution aggregated alert details. + */ @Immutable public final class IoTSecurityAggregatedAlertProperties { /* @@ -92,13 +94,15 @@ public final class IoTSecurityAggregatedAlertProperties { @JsonProperty(value = "topDevicesList", access = JsonProperty.Access.WRITE_ONLY) private List topDevicesList; - /** Creates an instance of IoTSecurityAggregatedAlertProperties class. */ + /** + * Creates an instance of IoTSecurityAggregatedAlertProperties class. + */ public IoTSecurityAggregatedAlertProperties() { } /** * Get the alertType property: Name of the alert type. - * + * * @return the alertType value. */ public String alertType() { @@ -107,7 +111,7 @@ public String alertType() { /** * Get the alertDisplayName property: Display name of the alert type. - * + * * @return the alertDisplayName value. */ public String alertDisplayName() { @@ -116,7 +120,7 @@ public String alertDisplayName() { /** * Get the aggregatedDateUtc property: Date of detection. - * + * * @return the aggregatedDateUtc value. */ public LocalDate aggregatedDateUtc() { @@ -125,7 +129,7 @@ public LocalDate aggregatedDateUtc() { /** * Get the vendorName property: Name of the organization that raised the alert. - * + * * @return the vendorName value. */ public String vendorName() { @@ -134,7 +138,7 @@ public String vendorName() { /** * Get the reportedSeverity property: Assessed alert severity. - * + * * @return the reportedSeverity value. */ public ReportedSeverity reportedSeverity() { @@ -143,7 +147,7 @@ public ReportedSeverity reportedSeverity() { /** * Get the remediationSteps property: Recommended steps for remediation. - * + * * @return the remediationSteps value. */ public String remediationSteps() { @@ -152,7 +156,7 @@ public String remediationSteps() { /** * Get the description property: Description of the suspected vulnerability and meaning. - * + * * @return the description value. */ public String description() { @@ -161,7 +165,7 @@ public String description() { /** * Get the count property: Number of alerts occurrences within the aggregated time window. - * + * * @return the count value. */ public Long count() { @@ -170,7 +174,7 @@ public Long count() { /** * Get the effectedResourceType property: Azure resource ID of the resource that received the alerts. - * + * * @return the effectedResourceType value. */ public String effectedResourceType() { @@ -179,7 +183,7 @@ public String effectedResourceType() { /** * Get the systemSource property: The type of the alerted resource (Azure, Non-Azure). - * + * * @return the systemSource value. */ public String systemSource() { @@ -188,7 +192,7 @@ public String systemSource() { /** * Get the actionTaken property: IoT Security solution alert response. - * + * * @return the actionTaken value. */ public String actionTaken() { @@ -197,7 +201,7 @@ public String actionTaken() { /** * Get the logAnalyticsQuery property: Log analytics query for getting the list of affected devices/alerts. - * + * * @return the logAnalyticsQuery value. */ public String logAnalyticsQuery() { @@ -207,7 +211,7 @@ public String logAnalyticsQuery() { /** * Get the topDevicesList property: 10 devices with the highest number of occurrences of this alert type, on this * day. - * + * * @return the topDevicesList value. */ public List topDevicesList() { @@ -216,7 +220,7 @@ public List topDevicesLi /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationInner.java index 9d9a7c3beb311..ae23b23f9aeaf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** IoT Security solution recommendation information. */ +/** + * IoT Security solution recommendation information. + */ @Fluent public final class IoTSecurityAggregatedRecommendationInner extends ProxyResource { /* @@ -27,13 +29,15 @@ public final class IoTSecurityAggregatedRecommendationInner extends ProxyResourc @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of IoTSecurityAggregatedRecommendationInner class. */ + /** + * Creates an instance of IoTSecurityAggregatedRecommendationInner class. + */ public IoTSecurityAggregatedRecommendationInner() { } /** * Get the innerProperties property: Security Solution data. - * + * * @return the innerProperties value. */ private IoTSecurityAggregatedRecommendationProperties innerProperties() { @@ -42,7 +46,7 @@ private IoTSecurityAggregatedRecommendationProperties innerProperties() { /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -51,7 +55,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the IoTSecurityAggregatedRecommendationInner object itself. */ @@ -62,7 +66,7 @@ public IoTSecurityAggregatedRecommendationInner withTags(Map tag /** * Get the recommendationName property: Name of the recommendation. - * + * * @return the recommendationName value. */ public String recommendationName() { @@ -71,7 +75,7 @@ public String recommendationName() { /** * Set the recommendationName property: Name of the recommendation. - * + * * @param recommendationName the recommendationName value to set. * @return the IoTSecurityAggregatedRecommendationInner object itself. */ @@ -85,7 +89,7 @@ public IoTSecurityAggregatedRecommendationInner withRecommendationName(String re /** * Get the recommendationDisplayName property: Display name of the recommendation type. - * + * * @return the recommendationDisplayName value. */ public String recommendationDisplayName() { @@ -94,7 +98,7 @@ public String recommendationDisplayName() { /** * Get the description property: Description of the suspected vulnerability and meaning. - * + * * @return the description value. */ public String description() { @@ -103,7 +107,7 @@ public String description() { /** * Get the recommendationTypeId property: Recommendation-type GUID. - * + * * @return the recommendationTypeId value. */ public String recommendationTypeId() { @@ -112,7 +116,7 @@ public String recommendationTypeId() { /** * Get the detectedBy property: Name of the organization that made the recommendation. - * + * * @return the detectedBy value. */ public String detectedBy() { @@ -121,7 +125,7 @@ public String detectedBy() { /** * Get the remediationSteps property: Recommended steps for remediation. - * + * * @return the remediationSteps value. */ public String remediationSteps() { @@ -130,7 +134,7 @@ public String remediationSteps() { /** * Get the reportedSeverity property: Assessed recommendation severity. - * + * * @return the reportedSeverity value. */ public ReportedSeverity reportedSeverity() { @@ -139,7 +143,7 @@ public ReportedSeverity reportedSeverity() { /** * Get the healthyDevices property: Number of healthy devices within the IoT Security solution. - * + * * @return the healthyDevices value. */ public Long healthyDevices() { @@ -148,7 +152,7 @@ public Long healthyDevices() { /** * Get the unhealthyDeviceCount property: Number of unhealthy devices within the IoT Security solution. - * + * * @return the unhealthyDeviceCount value. */ public Long unhealthyDeviceCount() { @@ -157,7 +161,7 @@ public Long unhealthyDeviceCount() { /** * Get the logAnalyticsQuery property: Log analytics query for getting the list of affected devices/alerts. - * + * * @return the logAnalyticsQuery value. */ public String logAnalyticsQuery() { @@ -166,7 +170,7 @@ public String logAnalyticsQuery() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationProperties.java index 9ee9660781831..0c1305dc7710f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecurityAggregatedRecommendationProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.ReportedSeverity; import com.fasterxml.jackson.annotation.JsonProperty; -/** IoT Security solution aggregated recommendation information. */ +/** + * IoT Security solution aggregated recommendation information. + */ @Fluent public final class IoTSecurityAggregatedRecommendationProperties { /* @@ -71,13 +73,15 @@ public final class IoTSecurityAggregatedRecommendationProperties { @JsonProperty(value = "logAnalyticsQuery", access = JsonProperty.Access.WRITE_ONLY) private String logAnalyticsQuery; - /** Creates an instance of IoTSecurityAggregatedRecommendationProperties class. */ + /** + * Creates an instance of IoTSecurityAggregatedRecommendationProperties class. + */ public IoTSecurityAggregatedRecommendationProperties() { } /** * Get the recommendationName property: Name of the recommendation. - * + * * @return the recommendationName value. */ public String recommendationName() { @@ -86,7 +90,7 @@ public String recommendationName() { /** * Set the recommendationName property: Name of the recommendation. - * + * * @param recommendationName the recommendationName value to set. * @return the IoTSecurityAggregatedRecommendationProperties object itself. */ @@ -97,7 +101,7 @@ public IoTSecurityAggregatedRecommendationProperties withRecommendationName(Stri /** * Get the recommendationDisplayName property: Display name of the recommendation type. - * + * * @return the recommendationDisplayName value. */ public String recommendationDisplayName() { @@ -106,7 +110,7 @@ public String recommendationDisplayName() { /** * Get the description property: Description of the suspected vulnerability and meaning. - * + * * @return the description value. */ public String description() { @@ -115,7 +119,7 @@ public String description() { /** * Get the recommendationTypeId property: Recommendation-type GUID. - * + * * @return the recommendationTypeId value. */ public String recommendationTypeId() { @@ -124,7 +128,7 @@ public String recommendationTypeId() { /** * Get the detectedBy property: Name of the organization that made the recommendation. - * + * * @return the detectedBy value. */ public String detectedBy() { @@ -133,7 +137,7 @@ public String detectedBy() { /** * Get the remediationSteps property: Recommended steps for remediation. - * + * * @return the remediationSteps value. */ public String remediationSteps() { @@ -142,7 +146,7 @@ public String remediationSteps() { /** * Get the reportedSeverity property: Assessed recommendation severity. - * + * * @return the reportedSeverity value. */ public ReportedSeverity reportedSeverity() { @@ -151,7 +155,7 @@ public ReportedSeverity reportedSeverity() { /** * Get the healthyDevices property: Number of healthy devices within the IoT Security solution. - * + * * @return the healthyDevices value. */ public Long healthyDevices() { @@ -160,7 +164,7 @@ public Long healthyDevices() { /** * Get the unhealthyDeviceCount property: Number of unhealthy devices within the IoT Security solution. - * + * * @return the unhealthyDeviceCount value. */ public Long unhealthyDeviceCount() { @@ -169,7 +173,7 @@ public Long unhealthyDeviceCount() { /** * Get the logAnalyticsQuery property: Log analytics query for getting the list of affected devices/alerts. - * + * * @return the logAnalyticsQuery value. */ public String logAnalyticsQuery() { @@ -178,7 +182,7 @@ public String logAnalyticsQuery() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelInner.java index fb70db188a5e4..3ed64aca81d3f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelInner.java @@ -14,22 +14,26 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Security analytics of your IoT Security solution. */ +/** + * Security analytics of your IoT Security solution. + */ @Fluent public final class IoTSecuritySolutionAnalyticsModelInner extends ProxyResource { /* - * Security Solution Aggregated Alert data + * Security Solution Aggregated Alert data */ @JsonProperty(value = "properties") private IoTSecuritySolutionAnalyticsModelProperties innerProperties; - /** Creates an instance of IoTSecuritySolutionAnalyticsModelInner class. */ + /** + * Creates an instance of IoTSecuritySolutionAnalyticsModelInner class. + */ public IoTSecuritySolutionAnalyticsModelInner() { } /** * Get the innerProperties property: Security Solution Aggregated Alert data. - * + * * @return the innerProperties value. */ private IoTSecuritySolutionAnalyticsModelProperties innerProperties() { @@ -38,7 +42,7 @@ private IoTSecuritySolutionAnalyticsModelProperties innerProperties() { /** * Get the metrics property: Security analytics of your IoT Security solution. - * + * * @return the metrics value. */ public IoTSeverityMetrics metrics() { @@ -47,7 +51,7 @@ public IoTSeverityMetrics metrics() { /** * Get the unhealthyDeviceCount property: Number of unhealthy devices within your IoT Security solution. - * + * * @return the unhealthyDeviceCount value. */ public Long unhealthyDeviceCount() { @@ -56,7 +60,7 @@ public Long unhealthyDeviceCount() { /** * Get the devicesMetrics property: List of device metrics by the aggregation date. - * + * * @return the devicesMetrics value. */ public List devicesMetrics() { @@ -65,7 +69,7 @@ public List devic /** * Get the topAlertedDevices property: List of the 3 devices with the most alerts. - * + * * @return the topAlertedDevices value. */ public List topAlertedDevices() { @@ -74,12 +78,12 @@ public List topAlertedDevices() { /** * Set the topAlertedDevices property: List of the 3 devices with the most alerts. - * + * * @param topAlertedDevices the topAlertedDevices value to set. * @return the IoTSecuritySolutionAnalyticsModelInner object itself. */ - public IoTSecuritySolutionAnalyticsModelInner withTopAlertedDevices( - List topAlertedDevices) { + public IoTSecuritySolutionAnalyticsModelInner + withTopAlertedDevices(List topAlertedDevices) { if (this.innerProperties() == null) { this.innerProperties = new IoTSecuritySolutionAnalyticsModelProperties(); } @@ -89,7 +93,7 @@ public IoTSecuritySolutionAnalyticsModelInner withTopAlertedDevices( /** * Get the mostPrevalentDeviceAlerts property: List of the 3 most prevalent device alerts. - * + * * @return the mostPrevalentDeviceAlerts value. */ public List mostPrevalentDeviceAlerts() { @@ -98,12 +102,12 @@ public List mostPrevalentDeviceAlerts() { /** * Set the mostPrevalentDeviceAlerts property: List of the 3 most prevalent device alerts. - * + * * @param mostPrevalentDeviceAlerts the mostPrevalentDeviceAlerts value to set. * @return the IoTSecuritySolutionAnalyticsModelInner object itself. */ - public IoTSecuritySolutionAnalyticsModelInner withMostPrevalentDeviceAlerts( - List mostPrevalentDeviceAlerts) { + public IoTSecuritySolutionAnalyticsModelInner + withMostPrevalentDeviceAlerts(List mostPrevalentDeviceAlerts) { if (this.innerProperties() == null) { this.innerProperties = new IoTSecuritySolutionAnalyticsModelProperties(); } @@ -113,7 +117,7 @@ public IoTSecuritySolutionAnalyticsModelInner withMostPrevalentDeviceAlerts( /** * Get the mostPrevalentDeviceRecommendations property: List of the 3 most prevalent device recommendations. - * + * * @return the mostPrevalentDeviceRecommendations value. */ public List mostPrevalentDeviceRecommendations() { @@ -122,7 +126,7 @@ public List mostPrevalentDeviceRecommendations( /** * Set the mostPrevalentDeviceRecommendations property: List of the 3 most prevalent device recommendations. - * + * * @param mostPrevalentDeviceRecommendations the mostPrevalentDeviceRecommendations value to set. * @return the IoTSecuritySolutionAnalyticsModelInner object itself. */ @@ -137,7 +141,7 @@ public IoTSecuritySolutionAnalyticsModelInner withMostPrevalentDeviceRecommendat /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelListInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelListInner.java index 05ebcd73f079a..5b485603f7539 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelListInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelListInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of Security analytics of your IoT Security solution. */ +/** + * List of Security analytics of your IoT Security solution. + */ @Fluent public final class IoTSecuritySolutionAnalyticsModelListInner { /* @@ -24,13 +26,15 @@ public final class IoTSecuritySolutionAnalyticsModelListInner { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of IoTSecuritySolutionAnalyticsModelListInner class. */ + /** + * Creates an instance of IoTSecuritySolutionAnalyticsModelListInner class. + */ public IoTSecuritySolutionAnalyticsModelListInner() { } /** * Get the value property: List of Security analytics of your IoT Security solution. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of Security analytics of your IoT Security solution. - * + * * @param value the value value to set. * @return the IoTSecuritySolutionAnalyticsModelListInner object itself. */ @@ -50,7 +54,7 @@ public IoTSecuritySolutionAnalyticsModelListInner withValue(List e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelProperties.java index 6ca28aeb9f29a..c43a6fdf5de65 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionAnalyticsModelProperties.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Security analytics properties of your IoT Security solution. */ +/** + * Security analytics properties of your IoT Security solution. + */ @Fluent public final class IoTSecuritySolutionAnalyticsModelProperties { /* @@ -52,13 +54,15 @@ public final class IoTSecuritySolutionAnalyticsModelProperties { @JsonProperty(value = "mostPrevalentDeviceRecommendations") private List mostPrevalentDeviceRecommendations; - /** Creates an instance of IoTSecuritySolutionAnalyticsModelProperties class. */ + /** + * Creates an instance of IoTSecuritySolutionAnalyticsModelProperties class. + */ public IoTSecuritySolutionAnalyticsModelProperties() { } /** * Get the metrics property: Security analytics of your IoT Security solution. - * + * * @return the metrics value. */ public IoTSeverityMetrics metrics() { @@ -67,7 +71,7 @@ public IoTSeverityMetrics metrics() { /** * Get the unhealthyDeviceCount property: Number of unhealthy devices within your IoT Security solution. - * + * * @return the unhealthyDeviceCount value. */ public Long unhealthyDeviceCount() { @@ -76,7 +80,7 @@ public Long unhealthyDeviceCount() { /** * Get the devicesMetrics property: List of device metrics by the aggregation date. - * + * * @return the devicesMetrics value. */ public List devicesMetrics() { @@ -85,7 +89,7 @@ public List devic /** * Get the topAlertedDevices property: List of the 3 devices with the most alerts. - * + * * @return the topAlertedDevices value. */ public List topAlertedDevices() { @@ -94,19 +98,19 @@ public List topAlertedDevices() { /** * Set the topAlertedDevices property: List of the 3 devices with the most alerts. - * + * * @param topAlertedDevices the topAlertedDevices value to set. * @return the IoTSecuritySolutionAnalyticsModelProperties object itself. */ - public IoTSecuritySolutionAnalyticsModelProperties withTopAlertedDevices( - List topAlertedDevices) { + public IoTSecuritySolutionAnalyticsModelProperties + withTopAlertedDevices(List topAlertedDevices) { this.topAlertedDevices = topAlertedDevices; return this; } /** * Get the mostPrevalentDeviceAlerts property: List of the 3 most prevalent device alerts. - * + * * @return the mostPrevalentDeviceAlerts value. */ public List mostPrevalentDeviceAlerts() { @@ -115,19 +119,19 @@ public List mostPrevalentDeviceAlerts() { /** * Set the mostPrevalentDeviceAlerts property: List of the 3 most prevalent device alerts. - * + * * @param mostPrevalentDeviceAlerts the mostPrevalentDeviceAlerts value to set. * @return the IoTSecuritySolutionAnalyticsModelProperties object itself. */ - public IoTSecuritySolutionAnalyticsModelProperties withMostPrevalentDeviceAlerts( - List mostPrevalentDeviceAlerts) { + public IoTSecuritySolutionAnalyticsModelProperties + withMostPrevalentDeviceAlerts(List mostPrevalentDeviceAlerts) { this.mostPrevalentDeviceAlerts = mostPrevalentDeviceAlerts; return this; } /** * Get the mostPrevalentDeviceRecommendations property: List of the 3 most prevalent device recommendations. - * + * * @return the mostPrevalentDeviceRecommendations value. */ public List mostPrevalentDeviceRecommendations() { @@ -136,7 +140,7 @@ public List mostPrevalentDeviceRecommendations( /** * Set the mostPrevalentDeviceRecommendations property: List of the 3 most prevalent device recommendations. - * + * * @param mostPrevalentDeviceRecommendations the mostPrevalentDeviceRecommendations value to set. * @return the IoTSecuritySolutionAnalyticsModelProperties object itself. */ @@ -148,7 +152,7 @@ public IoTSecuritySolutionAnalyticsModelProperties withMostPrevalentDeviceRecomm /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionModelInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionModelInner.java index 6659bf29c42b7..215dba52bc7ed 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionModelInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionModelInner.java @@ -19,7 +19,9 @@ import java.util.List; import java.util.Map; -/** IoT Security solution configuration and resource information. */ +/** + * IoT Security solution configuration and resource information. + */ @Fluent public final class IoTSecuritySolutionModelInner extends ProxyResource { /* @@ -47,13 +49,15 @@ public final class IoTSecuritySolutionModelInner extends ProxyResource { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map tags; - /** Creates an instance of IoTSecuritySolutionModelInner class. */ + /** + * Creates an instance of IoTSecuritySolutionModelInner class. + */ public IoTSecuritySolutionModelInner() { } /** * Get the location property: The resource location. - * + * * @return the location value. */ public String location() { @@ -62,7 +66,7 @@ public String location() { /** * Set the location property: The resource location. - * + * * @param location the location value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -73,7 +77,7 @@ public IoTSecuritySolutionModelInner withLocation(String location) { /** * Get the innerProperties property: Security Solution data. - * + * * @return the innerProperties value. */ private IoTSecuritySolutionProperties innerProperties() { @@ -82,7 +86,7 @@ private IoTSecuritySolutionProperties innerProperties() { /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -91,7 +95,7 @@ public SystemData systemData() { /** * Get the tags property: Resource tags. - * + * * @return the tags value. */ public Map tags() { @@ -100,7 +104,7 @@ public Map tags() { /** * Set the tags property: Resource tags. - * + * * @param tags the tags value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -111,7 +115,7 @@ public IoTSecuritySolutionModelInner withTags(Map tags) { /** * Get the workspace property: Workspace resource ID. - * + * * @return the workspace value. */ public String workspace() { @@ -120,7 +124,7 @@ public String workspace() { /** * Set the workspace property: Workspace resource ID. - * + * * @param workspace the workspace value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -134,7 +138,7 @@ public IoTSecuritySolutionModelInner withWorkspace(String workspace) { /** * Get the displayName property: Resource display name. - * + * * @return the displayName value. */ public String displayName() { @@ -143,7 +147,7 @@ public String displayName() { /** * Set the displayName property: Resource display name. - * + * * @param displayName the displayName value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -157,7 +161,7 @@ public IoTSecuritySolutionModelInner withDisplayName(String displayName) { /** * Get the status property: Status of the IoT Security solution. - * + * * @return the status value. */ public SecuritySolutionStatus status() { @@ -166,7 +170,7 @@ public SecuritySolutionStatus status() { /** * Set the status property: Status of the IoT Security solution. - * + * * @param status the status value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -180,7 +184,7 @@ public IoTSecuritySolutionModelInner withStatus(SecuritySolutionStatus status) { /** * Get the export property: List of additional options for exporting to workspace data. - * + * * @return the export value. */ public List export() { @@ -189,7 +193,7 @@ public List export() { /** * Set the export property: List of additional options for exporting to workspace data. - * + * * @param export the export value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -202,8 +206,9 @@ public IoTSecuritySolutionModelInner withExport(List export) { } /** - * Get the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the system. - * + * Get the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the + * system. + * * @return the disabledDataSources value. */ public List disabledDataSources() { @@ -211,8 +216,9 @@ public List disabledDataSources() { } /** - * Set the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the system. - * + * Set the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the + * system. + * * @param disabledDataSources the disabledDataSources value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -226,7 +232,7 @@ public IoTSecuritySolutionModelInner withDisabledDataSources(List di /** * Get the iotHubs property: IoT Hub resource IDs. - * + * * @return the iotHubs value. */ public List iotHubs() { @@ -235,7 +241,7 @@ public List iotHubs() { /** * Set the iotHubs property: IoT Hub resource IDs. - * + * * @param iotHubs the iotHubs value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -249,7 +255,7 @@ public IoTSecuritySolutionModelInner withIotHubs(List iotHubs) { /** * Get the userDefinedResources property: Properties of the IoT Security solution's user defined resources. - * + * * @return the userDefinedResources value. */ public UserDefinedResourcesProperties userDefinedResources() { @@ -258,7 +264,7 @@ public UserDefinedResourcesProperties userDefinedResources() { /** * Set the userDefinedResources property: Properties of the IoT Security solution's user defined resources. - * + * * @param userDefinedResources the userDefinedResources value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -271,9 +277,9 @@ public IoTSecuritySolutionModelInner withUserDefinedResources(UserDefinedResourc } /** - * Get the autoDiscoveredResources property: List of resources that were automatically discovered as relevant to the - * security solution. - * + * Get the autoDiscoveredResources property: List of resources that were automatically discovered as relevant to + * the security solution. + * * @return the autoDiscoveredResources value. */ public List autoDiscoveredResources() { @@ -282,7 +288,7 @@ public List autoDiscoveredResources() { /** * Get the recommendationsConfiguration property: List of the configuration status for each recommendation type. - * + * * @return the recommendationsConfiguration value. */ public List recommendationsConfiguration() { @@ -291,12 +297,12 @@ public List recommendationsConfiguration( /** * Set the recommendationsConfiguration property: List of the configuration status for each recommendation type. - * + * * @param recommendationsConfiguration the recommendationsConfiguration value to set. * @return the IoTSecuritySolutionModelInner object itself. */ - public IoTSecuritySolutionModelInner withRecommendationsConfiguration( - List recommendationsConfiguration) { + public IoTSecuritySolutionModelInner + withRecommendationsConfiguration(List recommendationsConfiguration) { if (this.innerProperties() == null) { this.innerProperties = new IoTSecuritySolutionProperties(); } @@ -306,7 +312,7 @@ public IoTSecuritySolutionModelInner withRecommendationsConfiguration( /** * Get the unmaskedIpLoggingStatus property: Unmasked IP address logging status. - * + * * @return the unmaskedIpLoggingStatus value. */ public UnmaskedIpLoggingStatus unmaskedIpLoggingStatus() { @@ -315,7 +321,7 @@ public UnmaskedIpLoggingStatus unmaskedIpLoggingStatus() { /** * Set the unmaskedIpLoggingStatus property: Unmasked IP address logging status. - * + * * @param unmaskedIpLoggingStatus the unmaskedIpLoggingStatus value to set. * @return the IoTSecuritySolutionModelInner object itself. */ @@ -329,7 +335,7 @@ public IoTSecuritySolutionModelInner withUnmaskedIpLoggingStatus(UnmaskedIpLoggi /** * Get the additionalWorkspaces property: List of additional workspaces. - * + * * @return the additionalWorkspaces value. */ public List additionalWorkspaces() { @@ -338,12 +344,12 @@ public List additionalWorkspaces() { /** * Set the additionalWorkspaces property: List of additional workspaces. - * + * * @param additionalWorkspaces the additionalWorkspaces value to set. * @return the IoTSecuritySolutionModelInner object itself. */ - public IoTSecuritySolutionModelInner withAdditionalWorkspaces( - List additionalWorkspaces) { + public IoTSecuritySolutionModelInner + withAdditionalWorkspaces(List additionalWorkspaces) { if (this.innerProperties() == null) { this.innerProperties = new IoTSecuritySolutionProperties(); } @@ -353,7 +359,7 @@ public IoTSecuritySolutionModelInner withAdditionalWorkspaces( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionProperties.java index daddad737073a..b50c1d45301dc 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/IoTSecuritySolutionProperties.java @@ -16,7 +16,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Security Solution setting data. */ +/** + * Security Solution setting data. + */ @Fluent public final class IoTSecuritySolutionProperties { /* @@ -85,13 +87,15 @@ public final class IoTSecuritySolutionProperties { @JsonProperty(value = "additionalWorkspaces") private List additionalWorkspaces; - /** Creates an instance of IoTSecuritySolutionProperties class. */ + /** + * Creates an instance of IoTSecuritySolutionProperties class. + */ public IoTSecuritySolutionProperties() { } /** * Get the workspace property: Workspace resource ID. - * + * * @return the workspace value. */ public String workspace() { @@ -100,7 +104,7 @@ public String workspace() { /** * Set the workspace property: Workspace resource ID. - * + * * @param workspace the workspace value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -111,7 +115,7 @@ public IoTSecuritySolutionProperties withWorkspace(String workspace) { /** * Get the displayName property: Resource display name. - * + * * @return the displayName value. */ public String displayName() { @@ -120,7 +124,7 @@ public String displayName() { /** * Set the displayName property: Resource display name. - * + * * @param displayName the displayName value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -131,7 +135,7 @@ public IoTSecuritySolutionProperties withDisplayName(String displayName) { /** * Get the status property: Status of the IoT Security solution. - * + * * @return the status value. */ public SecuritySolutionStatus status() { @@ -140,7 +144,7 @@ public SecuritySolutionStatus status() { /** * Set the status property: Status of the IoT Security solution. - * + * * @param status the status value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -151,7 +155,7 @@ public IoTSecuritySolutionProperties withStatus(SecuritySolutionStatus status) { /** * Get the export property: List of additional options for exporting to workspace data. - * + * * @return the export value. */ public List export() { @@ -160,7 +164,7 @@ public List export() { /** * Set the export property: List of additional options for exporting to workspace data. - * + * * @param export the export value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -170,8 +174,9 @@ public IoTSecuritySolutionProperties withExport(List export) { } /** - * Get the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the system. - * + * Get the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the + * system. + * * @return the disabledDataSources value. */ public List disabledDataSources() { @@ -179,8 +184,9 @@ public List disabledDataSources() { } /** - * Set the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the system. - * + * Set the disabledDataSources property: Disabled data sources. Disabling these data sources compromises the + * system. + * * @param disabledDataSources the disabledDataSources value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -191,7 +197,7 @@ public IoTSecuritySolutionProperties withDisabledDataSources(List di /** * Get the iotHubs property: IoT Hub resource IDs. - * + * * @return the iotHubs value. */ public List iotHubs() { @@ -200,7 +206,7 @@ public List iotHubs() { /** * Set the iotHubs property: IoT Hub resource IDs. - * + * * @param iotHubs the iotHubs value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -211,7 +217,7 @@ public IoTSecuritySolutionProperties withIotHubs(List iotHubs) { /** * Get the userDefinedResources property: Properties of the IoT Security solution's user defined resources. - * + * * @return the userDefinedResources value. */ public UserDefinedResourcesProperties userDefinedResources() { @@ -220,7 +226,7 @@ public UserDefinedResourcesProperties userDefinedResources() { /** * Set the userDefinedResources property: Properties of the IoT Security solution's user defined resources. - * + * * @param userDefinedResources the userDefinedResources value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -230,9 +236,9 @@ public IoTSecuritySolutionProperties withUserDefinedResources(UserDefinedResourc } /** - * Get the autoDiscoveredResources property: List of resources that were automatically discovered as relevant to the - * security solution. - * + * Get the autoDiscoveredResources property: List of resources that were automatically discovered as relevant to + * the security solution. + * * @return the autoDiscoveredResources value. */ public List autoDiscoveredResources() { @@ -241,7 +247,7 @@ public List autoDiscoveredResources() { /** * Get the recommendationsConfiguration property: List of the configuration status for each recommendation type. - * + * * @return the recommendationsConfiguration value. */ public List recommendationsConfiguration() { @@ -250,19 +256,19 @@ public List recommendationsConfiguration( /** * Set the recommendationsConfiguration property: List of the configuration status for each recommendation type. - * + * * @param recommendationsConfiguration the recommendationsConfiguration value to set. * @return the IoTSecuritySolutionProperties object itself. */ - public IoTSecuritySolutionProperties withRecommendationsConfiguration( - List recommendationsConfiguration) { + public IoTSecuritySolutionProperties + withRecommendationsConfiguration(List recommendationsConfiguration) { this.recommendationsConfiguration = recommendationsConfiguration; return this; } /** * Get the unmaskedIpLoggingStatus property: Unmasked IP address logging status. - * + * * @return the unmaskedIpLoggingStatus value. */ public UnmaskedIpLoggingStatus unmaskedIpLoggingStatus() { @@ -271,7 +277,7 @@ public UnmaskedIpLoggingStatus unmaskedIpLoggingStatus() { /** * Set the unmaskedIpLoggingStatus property: Unmasked IP address logging status. - * + * * @param unmaskedIpLoggingStatus the unmaskedIpLoggingStatus value to set. * @return the IoTSecuritySolutionProperties object itself. */ @@ -282,7 +288,7 @@ public IoTSecuritySolutionProperties withUnmaskedIpLoggingStatus(UnmaskedIpLoggi /** * Get the additionalWorkspaces property: List of additional workspaces. - * + * * @return the additionalWorkspaces value. */ public List additionalWorkspaces() { @@ -291,33 +297,29 @@ public List additionalWorkspaces() { /** * Set the additionalWorkspaces property: List of additional workspaces. - * + * * @param additionalWorkspaces the additionalWorkspaces value to set. * @return the IoTSecuritySolutionProperties object itself. */ - public IoTSecuritySolutionProperties withAdditionalWorkspaces( - List additionalWorkspaces) { + public IoTSecuritySolutionProperties + withAdditionalWorkspaces(List additionalWorkspaces) { this.additionalWorkspaces = additionalWorkspaces; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (displayName() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property displayName in model IoTSecuritySolutionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property displayName in model IoTSecuritySolutionProperties")); } if (iotHubs() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property iotHubs in model IoTSecuritySolutionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property iotHubs in model IoTSecuritySolutionProperties")); } if (userDefinedResources() != null) { userDefinedResources().validate(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessPolicyInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessPolicyInner.java index 7b307820330e5..9763957454846 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessPolicyInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessPolicyInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The JitNetworkAccessPolicy model. */ +/** + * The JitNetworkAccessPolicy model. + */ @Fluent public final class JitNetworkAccessPolicyInner extends ProxyResource { /* @@ -32,13 +34,15 @@ public final class JitNetworkAccessPolicyInner extends ProxyResource { @JsonProperty(value = "location", access = JsonProperty.Access.WRITE_ONLY) private String location; - /** Creates an instance of JitNetworkAccessPolicyInner class. */ + /** + * Creates an instance of JitNetworkAccessPolicyInner class. + */ public JitNetworkAccessPolicyInner() { } /** * Get the innerProperties property: The properties property. - * + * * @return the innerProperties value. */ private JitNetworkAccessPolicyProperties innerProperties() { @@ -47,7 +51,7 @@ private JitNetworkAccessPolicyProperties innerProperties() { /** * Get the kind property: Kind of the resource. - * + * * @return the kind value. */ public String kind() { @@ -56,7 +60,7 @@ public String kind() { /** * Set the kind property: Kind of the resource. - * + * * @param kind the kind value to set. * @return the JitNetworkAccessPolicyInner object itself. */ @@ -67,7 +71,7 @@ public JitNetworkAccessPolicyInner withKind(String kind) { /** * Get the location property: Location where the resource is stored. - * + * * @return the location value. */ public String location() { @@ -76,7 +80,7 @@ public String location() { /** * Get the virtualMachines property: Configurations for Microsoft.Compute/virtualMachines resource type. - * + * * @return the virtualMachines value. */ public List virtualMachines() { @@ -85,7 +89,7 @@ public List virtualMachines() { /** * Set the virtualMachines property: Configurations for Microsoft.Compute/virtualMachines resource type. - * + * * @param virtualMachines the virtualMachines value to set. * @return the JitNetworkAccessPolicyInner object itself. */ @@ -99,7 +103,7 @@ public JitNetworkAccessPolicyInner withVirtualMachines(List requests() { @@ -108,7 +112,7 @@ public List requests() { /** * Set the requests property: The requests property. - * + * * @param requests the requests value to set. * @return the JitNetworkAccessPolicyInner object itself. */ @@ -122,7 +126,7 @@ public JitNetworkAccessPolicyInner withRequests(List virtualMachines() { @@ -46,19 +50,19 @@ public List virtualMachines() { /** * Set the virtualMachines property: Configurations for Microsoft.Compute/virtualMachines resource type. - * + * * @param virtualMachines the virtualMachines value to set. * @return the JitNetworkAccessPolicyProperties object itself. */ - public JitNetworkAccessPolicyProperties withVirtualMachines( - List virtualMachines) { + public JitNetworkAccessPolicyProperties + withVirtualMachines(List virtualMachines) { this.virtualMachines = virtualMachines; return this; } /** * Get the requests property: The requests property. - * + * * @return the requests value. */ public List requests() { @@ -67,7 +71,7 @@ public List requests() { /** * Set the requests property: The requests property. - * + * * @param requests the requests value to set. * @return the JitNetworkAccessPolicyProperties object itself. */ @@ -78,7 +82,7 @@ public JitNetworkAccessPolicyProperties withRequests(List e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessRequestInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessRequestInner.java index 6864965bacb34..9043ec0754389 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessRequestInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/JitNetworkAccessRequestInner.java @@ -11,7 +11,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** The JitNetworkAccessRequest model. */ +/** + * The JitNetworkAccessRequest model. + */ @Fluent public final class JitNetworkAccessRequestInner { /* @@ -38,13 +40,15 @@ public final class JitNetworkAccessRequestInner { @JsonProperty(value = "justification") private String justification; - /** Creates an instance of JitNetworkAccessRequestInner class. */ + /** + * Creates an instance of JitNetworkAccessRequestInner class. + */ public JitNetworkAccessRequestInner() { } /** * Get the virtualMachines property: The virtualMachines property. - * + * * @return the virtualMachines value. */ public List virtualMachines() { @@ -53,19 +57,19 @@ public List virtualMachines() { /** * Set the virtualMachines property: The virtualMachines property. - * + * * @param virtualMachines the virtualMachines value to set. * @return the JitNetworkAccessRequestInner object itself. */ - public JitNetworkAccessRequestInner withVirtualMachines( - List virtualMachines) { + public JitNetworkAccessRequestInner + withVirtualMachines(List virtualMachines) { this.virtualMachines = virtualMachines; return this; } /** * Get the startTimeUtc property: The start time of the request in UTC. - * + * * @return the startTimeUtc value. */ public OffsetDateTime startTimeUtc() { @@ -74,7 +78,7 @@ public OffsetDateTime startTimeUtc() { /** * Set the startTimeUtc property: The start time of the request in UTC. - * + * * @param startTimeUtc the startTimeUtc value to set. * @return the JitNetworkAccessRequestInner object itself. */ @@ -85,7 +89,7 @@ public JitNetworkAccessRequestInner withStartTimeUtc(OffsetDateTime startTimeUtc /** * Get the requestor property: The identity of the person who made the request. - * + * * @return the requestor value. */ public String requestor() { @@ -94,7 +98,7 @@ public String requestor() { /** * Set the requestor property: The identity of the person who made the request. - * + * * @param requestor the requestor value to set. * @return the JitNetworkAccessRequestInner object itself. */ @@ -105,7 +109,7 @@ public JitNetworkAccessRequestInner withRequestor(String requestor) { /** * Get the justification property: The justification for making the initiate request. - * + * * @return the justification value. */ public String justification() { @@ -114,7 +118,7 @@ public String justification() { /** * Set the justification property: The justification for making the initiate request. - * + * * @param justification the justification value to set. * @return the JitNetworkAccessRequestInner object itself. */ @@ -125,29 +129,23 @@ public JitNetworkAccessRequestInner withJustification(String justification) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (virtualMachines() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property virtualMachines in model JitNetworkAccessRequestInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property virtualMachines in model JitNetworkAccessRequestInner")); } else { virtualMachines().forEach(e -> e.validate()); } if (startTimeUtc() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property startTimeUtc in model JitNetworkAccessRequestInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property startTimeUtc in model JitNetworkAccessRequestInner")); } if (requestor() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property requestor in model JitNetworkAccessRequestInner")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property requestor in model JitNetworkAccessRequestInner")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MalwareScanningProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MalwareScanningProperties.java new file mode 100644 index 0000000000000..fc1926b41e9ea --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MalwareScanningProperties.java @@ -0,0 +1,141 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.models.OperationStatus; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Properties of Malware Scanning. + */ +@Fluent +public final class MalwareScanningProperties { + /* + * Properties of On Upload malware scanning. + */ + @JsonProperty(value = "onUpload") + private OnUploadProperties innerOnUpload; + + /* + * Optional. Resource id of an Event Grid Topic to send scan results to. + */ + @JsonProperty(value = "scanResultsEventGridTopicResourceId") + private String scanResultsEventGridTopicResourceId; + + /* + * Upon failure or partial success. Additional data describing Malware Scanning enable/disable operation. + */ + @JsonProperty(value = "operationStatus", access = JsonProperty.Access.WRITE_ONLY) + private OperationStatus operationStatus; + + /** + * Creates an instance of MalwareScanningProperties class. + */ + public MalwareScanningProperties() { + } + + /** + * Get the innerOnUpload property: Properties of On Upload malware scanning. + * + * @return the innerOnUpload value. + */ + private OnUploadProperties innerOnUpload() { + return this.innerOnUpload; + } + + /** + * Get the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @return the scanResultsEventGridTopicResourceId value. + */ + public String scanResultsEventGridTopicResourceId() { + return this.scanResultsEventGridTopicResourceId; + } + + /** + * Set the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @param scanResultsEventGridTopicResourceId the scanResultsEventGridTopicResourceId value to set. + * @return the MalwareScanningProperties object itself. + */ + public MalwareScanningProperties + withScanResultsEventGridTopicResourceId(String scanResultsEventGridTopicResourceId) { + this.scanResultsEventGridTopicResourceId = scanResultsEventGridTopicResourceId; + return this; + } + + /** + * Get the operationStatus property: Upon failure or partial success. Additional data describing Malware Scanning + * enable/disable operation. + * + * @return the operationStatus value. + */ + public OperationStatus operationStatus() { + return this.operationStatus; + } + + /** + * Get the isEnabled property: Indicates whether On Upload malware scanning should be enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.innerOnUpload() == null ? null : this.innerOnUpload().isEnabled(); + } + + /** + * Set the isEnabled property: Indicates whether On Upload malware scanning should be enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the MalwareScanningProperties object itself. + */ + public MalwareScanningProperties withIsEnabled(Boolean isEnabled) { + if (this.innerOnUpload() == null) { + this.innerOnUpload = new OnUploadProperties(); + } + this.innerOnUpload().withIsEnabled(isEnabled); + return this; + } + + /** + * Get the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @return the capGBPerMonth value. + */ + public Integer capGBPerMonth() { + return this.innerOnUpload() == null ? null : this.innerOnUpload().capGBPerMonth(); + } + + /** + * Set the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @param capGBPerMonth the capGBPerMonth value to set. + * @return the MalwareScanningProperties object itself. + */ + public MalwareScanningProperties withCapGBPerMonth(Integer capGBPerMonth) { + if (this.innerOnUpload() == null) { + this.innerOnUpload = new OnUploadProperties(); + } + this.innerOnUpload().withCapGBPerMonth(capGBPerMonth); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (innerOnUpload() != null) { + innerOnUpload().validate(); + } + if (operationStatus() != null) { + operationStatus().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataInner.java index bd385f7c835d3..3202212b27991 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataInner.java @@ -8,23 +8,29 @@ import com.azure.core.management.ProxyResource; import com.fasterxml.jackson.annotation.JsonProperty; -/** The resource of the configuration or data needed to onboard the machine to MDE. */ +/** + * The resource of the configuration or data needed to onboard the machine to MDE. + */ @Fluent public final class MdeOnboardingDataInner extends ProxyResource { + private static final byte[] EMPTY_BYTE_ARRAY = new byte[0]; + /* * Properties of the MDE configuration or data parameter needed to onboard the machine to MDE */ @JsonProperty(value = "properties") private MdeOnboardingDataProperties innerProperties; - /** Creates an instance of MdeOnboardingDataInner class. */ + /** + * Creates an instance of MdeOnboardingDataInner class. + */ public MdeOnboardingDataInner() { } /** * Get the innerProperties property: Properties of the MDE configuration or data parameter needed to onboard the * machine to MDE. - * + * * @return the innerProperties value. */ private MdeOnboardingDataProperties innerProperties() { @@ -34,17 +40,17 @@ private MdeOnboardingDataProperties innerProperties() { /** * Get the onboardingPackageWindows property: The onboarding package used to onboard Windows machines to MDE, coded * in base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @return the onboardingPackageWindows value. */ public byte[] onboardingPackageWindows() { - return this.innerProperties() == null ? new byte[0] : this.innerProperties().onboardingPackageWindows(); + return this.innerProperties() == null ? EMPTY_BYTE_ARRAY : this.innerProperties().onboardingPackageWindows(); } /** * Set the onboardingPackageWindows property: The onboarding package used to onboard Windows machines to MDE, coded * in base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @param onboardingPackageWindows the onboardingPackageWindows value to set. * @return the MdeOnboardingDataInner object itself. */ @@ -59,17 +65,17 @@ public MdeOnboardingDataInner withOnboardingPackageWindows(byte[] onboardingPack /** * Get the onboardingPackageLinux property: The onboarding package used to onboard Linux machines to MDE, coded in * base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @return the onboardingPackageLinux value. */ public byte[] onboardingPackageLinux() { - return this.innerProperties() == null ? new byte[0] : this.innerProperties().onboardingPackageLinux(); + return this.innerProperties() == null ? EMPTY_BYTE_ARRAY : this.innerProperties().onboardingPackageLinux(); } /** * Set the onboardingPackageLinux property: The onboarding package used to onboard Linux machines to MDE, coded in * base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @param onboardingPackageLinux the onboardingPackageLinux value to set. * @return the MdeOnboardingDataInner object itself. */ @@ -83,7 +89,7 @@ public MdeOnboardingDataInner withOnboardingPackageLinux(byte[] onboardingPackag /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataListInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataListInner.java index b6650c04d95db..5385060ce6a69 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataListInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataListInner.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of all MDE onboarding data resources. */ +/** + * List of all MDE onboarding data resources. + */ @Fluent public final class MdeOnboardingDataListInner { /* @@ -17,13 +19,15 @@ public final class MdeOnboardingDataListInner { @JsonProperty(value = "value") private List value; - /** Creates an instance of MdeOnboardingDataListInner class. */ + /** + * Creates an instance of MdeOnboardingDataListInner class. + */ public MdeOnboardingDataListInner() { } /** * Get the value property: List of the resources of the configuration or data needed to onboard the machine to MDE. - * + * * @return the value value. */ public List value() { @@ -32,7 +36,7 @@ public List value() { /** * Set the value property: List of the resources of the configuration or data needed to onboard the machine to MDE. - * + * * @param value the value value to set. * @return the MdeOnboardingDataListInner object itself. */ @@ -43,7 +47,7 @@ public MdeOnboardingDataListInner withValue(List value) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataProperties.java index b1bae4df584b5..cd5151bb2f884 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/MdeOnboardingDataProperties.java @@ -8,7 +8,9 @@ import com.azure.core.util.CoreUtils; import com.fasterxml.jackson.annotation.JsonProperty; -/** Properties of the MDE configuration or data parameter needed to onboard the machine to MDE. */ +/** + * Properties of the MDE configuration or data parameter needed to onboard the machine to MDE. + */ @Fluent public final class MdeOnboardingDataProperties { /* @@ -25,14 +27,16 @@ public final class MdeOnboardingDataProperties { @JsonProperty(value = "onboardingPackageLinux") private byte[] onboardingPackageLinux; - /** Creates an instance of MdeOnboardingDataProperties class. */ + /** + * Creates an instance of MdeOnboardingDataProperties class. + */ public MdeOnboardingDataProperties() { } /** * Get the onboardingPackageWindows property: The onboarding package used to onboard Windows machines to MDE, coded * in base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @return the onboardingPackageWindows value. */ public byte[] onboardingPackageWindows() { @@ -42,7 +46,7 @@ public byte[] onboardingPackageWindows() { /** * Set the onboardingPackageWindows property: The onboarding package used to onboard Windows machines to MDE, coded * in base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @param onboardingPackageWindows the onboardingPackageWindows value to set. * @return the MdeOnboardingDataProperties object itself. */ @@ -54,7 +58,7 @@ public MdeOnboardingDataProperties withOnboardingPackageWindows(byte[] onboardin /** * Get the onboardingPackageLinux property: The onboarding package used to onboard Linux machines to MDE, coded in * base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @return the onboardingPackageLinux value. */ public byte[] onboardingPackageLinux() { @@ -64,7 +68,7 @@ public byte[] onboardingPackageLinux() { /** * Set the onboardingPackageLinux property: The onboarding package used to onboard Linux machines to MDE, coded in * base64. This can also be used for onboarding using the dedicated VM Extension. - * + * * @param onboardingPackageLinux the onboardingPackageLinux value to set. * @return the MdeOnboardingDataProperties object itself. */ @@ -75,7 +79,7 @@ public MdeOnboardingDataProperties withOnboardingPackageLinux(byte[] onboardingP /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OnUploadProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OnUploadProperties.java new file mode 100644 index 0000000000000..ec474144bbd50 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OnUploadProperties.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Properties of On Upload malware scanning. + */ +@Fluent +public final class OnUploadProperties { + /* + * Indicates whether On Upload malware scanning should be enabled. + */ + @JsonProperty(value = "isEnabled") + private Boolean isEnabled; + + /* + * Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + */ + @JsonProperty(value = "capGBPerMonth") + private Integer capGBPerMonth; + + /** + * Creates an instance of OnUploadProperties class. + */ + public OnUploadProperties() { + } + + /** + * Get the isEnabled property: Indicates whether On Upload malware scanning should be enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.isEnabled; + } + + /** + * Set the isEnabled property: Indicates whether On Upload malware scanning should be enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the OnUploadProperties object itself. + */ + public OnUploadProperties withIsEnabled(Boolean isEnabled) { + this.isEnabled = isEnabled; + return this; + } + + /** + * Get the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @return the capGBPerMonth value. + */ + public Integer capGBPerMonth() { + return this.capGBPerMonth; + } + + /** + * Set the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @param capGBPerMonth the capGBPerMonth value to set. + * @return the OnUploadProperties object itself. + */ + public OnUploadProperties withCapGBPerMonth(Integer capGBPerMonth) { + this.capGBPerMonth = capGBPerMonth; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationInner.java index e681d61a3dd22..bb95b9088b5a1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.OperationDisplay; import com.fasterxml.jackson.annotation.JsonProperty; -/** Possible operation in the REST API of Microsoft.Security. */ +/** + * Possible operation in the REST API of Microsoft.Security. + */ @Fluent public final class OperationInner { /* @@ -29,13 +31,15 @@ public final class OperationInner { @JsonProperty(value = "display") private OperationDisplay display; - /** Creates an instance of OperationInner class. */ + /** + * Creates an instance of OperationInner class. + */ public OperationInner() { } /** * Get the name property: Name of the operation. - * + * * @return the name value. */ public String name() { @@ -44,7 +48,7 @@ public String name() { /** * Get the origin property: Where the operation is originated. - * + * * @return the origin value. */ public String origin() { @@ -53,7 +57,7 @@ public String origin() { /** * Get the display property: Security operation display. - * + * * @return the display value. */ public OperationDisplay display() { @@ -62,7 +66,7 @@ public OperationDisplay display() { /** * Set the display property: Security operation display. - * + * * @param display the display value to set. * @return the OperationInner object itself. */ @@ -73,7 +77,7 @@ public OperationInner withDisplay(OperationDisplay display) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationResultAutoGeneratedInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationResultAutoGeneratedInner.java index c4ca79253a776..697a3841200f3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationResultAutoGeneratedInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationResultAutoGeneratedInner.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.OperationResult; import com.fasterxml.jackson.annotation.JsonProperty; -/** Long run operation status of governance rule over a given scope. */ +/** + * Long run operation status of governance rule over a given scope. + */ @Immutable public final class OperationResultAutoGeneratedInner { /* @@ -17,13 +19,15 @@ public final class OperationResultAutoGeneratedInner { @JsonProperty(value = "status", access = JsonProperty.Access.WRITE_ONLY) private OperationResult status; - /** Creates an instance of OperationResultAutoGeneratedInner class. */ + /** + * Creates an instance of OperationResultAutoGeneratedInner class. + */ public OperationResultAutoGeneratedInner() { } /** * Get the status property: The status of the long run operation result of governance rule. - * + * * @return the status value. */ public OperationResult status() { @@ -32,7 +36,7 @@ public OperationResult status() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationStatusResultInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationStatusResultInner.java new file mode 100644 index 0000000000000..163c04aa5bff2 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/OperationStatusResultInner.java @@ -0,0 +1,249 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.fluent.models; + +import com.azure.core.annotation.Fluent; +import com.azure.core.management.exception.ManagementError; +import com.azure.core.util.logging.ClientLogger; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; +import java.util.List; + +/** + * The current status of an async operation. + */ +@Fluent +public final class OperationStatusResultInner { + /* + * Fully qualified ID for the async operation. + */ + @JsonProperty(value = "id") + private String id; + + /* + * Name of the async operation. + */ + @JsonProperty(value = "name") + private String name; + + /* + * Operation status. + */ + @JsonProperty(value = "status", required = true) + private String status; + + /* + * Percent of the operation that is complete. + */ + @JsonProperty(value = "percentComplete") + private Float percentComplete; + + /* + * The start time of the operation. + */ + @JsonProperty(value = "startTime") + private OffsetDateTime startTime; + + /* + * The end time of the operation. + */ + @JsonProperty(value = "endTime") + private OffsetDateTime endTime; + + /* + * The operations list. + */ + @JsonProperty(value = "operations") + private List operations; + + /* + * If present, details of the operation error. + */ + @JsonProperty(value = "error") + private ManagementError error; + + /** + * Creates an instance of OperationStatusResultInner class. + */ + public OperationStatusResultInner() { + } + + /** + * Get the id property: Fully qualified ID for the async operation. + * + * @return the id value. + */ + public String id() { + return this.id; + } + + /** + * Set the id property: Fully qualified ID for the async operation. + * + * @param id the id value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withId(String id) { + this.id = id; + return this; + } + + /** + * Get the name property: Name of the async operation. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Name of the async operation. + * + * @param name the name value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withName(String name) { + this.name = name; + return this; + } + + /** + * Get the status property: Operation status. + * + * @return the status value. + */ + public String status() { + return this.status; + } + + /** + * Set the status property: Operation status. + * + * @param status the status value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withStatus(String status) { + this.status = status; + return this; + } + + /** + * Get the percentComplete property: Percent of the operation that is complete. + * + * @return the percentComplete value. + */ + public Float percentComplete() { + return this.percentComplete; + } + + /** + * Set the percentComplete property: Percent of the operation that is complete. + * + * @param percentComplete the percentComplete value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withPercentComplete(Float percentComplete) { + this.percentComplete = percentComplete; + return this; + } + + /** + * Get the startTime property: The start time of the operation. + * + * @return the startTime value. + */ + public OffsetDateTime startTime() { + return this.startTime; + } + + /** + * Set the startTime property: The start time of the operation. + * + * @param startTime the startTime value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withStartTime(OffsetDateTime startTime) { + this.startTime = startTime; + return this; + } + + /** + * Get the endTime property: The end time of the operation. + * + * @return the endTime value. + */ + public OffsetDateTime endTime() { + return this.endTime; + } + + /** + * Set the endTime property: The end time of the operation. + * + * @param endTime the endTime value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withEndTime(OffsetDateTime endTime) { + this.endTime = endTime; + return this; + } + + /** + * Get the operations property: The operations list. + * + * @return the operations value. + */ + public List operations() { + return this.operations; + } + + /** + * Set the operations property: The operations list. + * + * @param operations the operations value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withOperations(List operations) { + this.operations = operations; + return this; + } + + /** + * Get the error property: If present, details of the operation error. + * + * @return the error value. + */ + public ManagementError error() { + return this.error; + } + + /** + * Set the error property: If present, details of the operation error. + * + * @param error the error value to set. + * @return the OperationStatusResultInner object itself. + */ + public OperationStatusResultInner withError(ManagementError error) { + this.error = error; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (status() == null) { + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property status in model OperationStatusResultInner")); + } + if (operations() != null) { + operations().forEach(e -> e.validate()); + } + } + + private static final ClientLogger LOGGER = new ClientLogger(OperationStatusResultInner.class); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingInner.java index 99e8664219982..fcb5bbae1b65d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingInner.java @@ -6,8 +6,11 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; +import com.azure.resourcemanager.security.models.Enforce; import com.azure.resourcemanager.security.models.Extension; +import com.azure.resourcemanager.security.models.Inherited; import com.azure.resourcemanager.security.models.PricingTier; +import com.azure.resourcemanager.security.models.ResourcesCoverageStatus; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.Duration; import java.time.OffsetDateTime; @@ -25,13 +28,15 @@ public final class PricingInner extends ProxyResource { @JsonProperty(value = "properties") private PricingProperties innerProperties; - /** Creates an instance of PricingInner class. */ + /** + * Creates an instance of PricingInner class. + */ public PricingInner() { } /** * Get the innerProperties property: Pricing data. - * + * * @return the innerProperties value. */ private PricingProperties innerProperties() { @@ -39,10 +44,10 @@ private PricingProperties innerProperties() { } /** - * Get the pricingTier property: The pricing tier value. Microsoft Defender for Cloud is provided in two pricing - * tiers: free and standard. The standard tier offers advanced security capabilities, while the free tier offers - * basic security features. - * + * Get the pricingTier property: Indicates whether the Defender plan is enabled on the selected scope. Microsoft + * Defender for Cloud is provided in two pricing tiers: free and standard. The standard tier offers advanced + * security capabilities, while the free tier offers basic security features. + * * @return the pricingTier value. */ public PricingTier pricingTier() { @@ -50,10 +55,10 @@ public PricingTier pricingTier() { } /** - * Set the pricingTier property: The pricing tier value. Microsoft Defender for Cloud is provided in two pricing - * tiers: free and standard. The standard tier offers advanced security capabilities, while the free tier offers - * basic security features. - * + * Set the pricingTier property: Indicates whether the Defender plan is enabled on the selected scope. Microsoft + * Defender for Cloud is provided in two pricing tiers: free and standard. The standard tier offers advanced + * security capabilities, while the free tier offers basic security features. + * * @param pricingTier the pricingTier value to set. * @return the PricingInner object itself. */ @@ -66,9 +71,11 @@ public PricingInner withPricingTier(PricingTier pricingTier) { } /** - * Get the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one sub-plan - * is available. Each sub-plan enables a set of security features. When not specified, full plan is applied. - * + * Get the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one + * sub-plan is available. Each sub-plan enables a set of security features. When not specified, full plan is + * applied. For VirtualMachines plan, available sub plans are 'P1' & 'P2', where for resource level only 'P1' + * sub plan is supported. + * * @return the subPlan value. */ public String subPlan() { @@ -76,9 +83,11 @@ public String subPlan() { } /** - * Set the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one sub-plan - * is available. Each sub-plan enables a set of security features. When not specified, full plan is applied. - * + * Set the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one + * sub-plan is available. Each sub-plan enables a set of security features. When not specified, full plan is + * applied. For VirtualMachines plan, available sub plans are 'P1' & 'P2', where for resource level only 'P1' + * sub plan is supported. + * * @param subPlan the subPlan value to set. * @return the PricingInner object itself. */ @@ -93,7 +102,7 @@ public PricingInner withSubPlan(String subPlan) { /** * Get the freeTrialRemainingTime property: The duration left for the subscriptions free trial period - in ISO 8601 * format (e.g. P3Y6M4DT12H30M5S). - * + * * @return the freeTrialRemainingTime value. */ public Duration freeTrialRemainingTime() { @@ -103,7 +112,7 @@ public Duration freeTrialRemainingTime() { /** * Get the enablementTime property: Optional. If `pricingTier` is `Standard` then this property holds the date of * the last time the `pricingTier` was set to `Standard`, when available (e.g 2023-03-01T12:42:42.1921106Z). - * + * * @return the enablementTime value. */ public OffsetDateTime enablementTime() { @@ -111,28 +120,73 @@ public OffsetDateTime enablementTime() { } /** - * Get the deprecated property: Optional. True if the plan is deprecated. If there are replacing plans they will - * appear in `replacedBy` property. - * - * @return the deprecated value. + * Get the enforce property: If set to "False", it allows the descendants of this scope to override the pricing + * configuration set on this scope (allows setting inherited="False"). If set to "True", it prevents overrides and + * forces this pricing configuration on all the descendants of this scope. This field is only available for + * subscription-level pricing. + * + * @return the enforce value. */ - public Boolean deprecated() { - return this.innerProperties() == null ? null : this.innerProperties().deprecated(); + public Enforce enforce() { + return this.innerProperties() == null ? null : this.innerProperties().enforce(); } /** - * Get the replacedBy property: Optional. List of plans that replace this plan. This property exists only if this - * plan is deprecated. - * - * @return the replacedBy value. + * Set the enforce property: If set to "False", it allows the descendants of this scope to override the pricing + * configuration set on this scope (allows setting inherited="False"). If set to "True", it prevents overrides and + * forces this pricing configuration on all the descendants of this scope. This field is only available for + * subscription-level pricing. + * + * @param enforce the enforce value to set. + * @return the PricingInner object itself. */ - public List replacedBy() { - return this.innerProperties() == null ? null : this.innerProperties().replacedBy(); + public PricingInner withEnforce(Enforce enforce) { + if (this.innerProperties() == null) { + this.innerProperties = new PricingProperties(); + } + this.innerProperties().withEnforce(enforce); + return this; + } + + /** + * Get the inherited property: "inherited" = "True" indicates that the current scope inherits its pricing + * configuration from its parent. The ID of the parent scope that provides the inherited configuration is displayed + * in the "inheritedFrom" field. On the other hand, "inherited" = "False" indicates that the current scope has its + * own pricing configuration explicitly set, and does not inherit from its parent. This field is read only and + * available only for resource-level pricing. + * + * @return the inherited value. + */ + public Inherited inherited() { + return this.innerProperties() == null ? null : this.innerProperties().inherited(); + } + + /** + * Get the inheritedFrom property: The id of the scope inherited from. "Null" if not inherited. This field is only + * available for resource-level pricing. + * + * @return the inheritedFrom value. + */ + public String inheritedFrom() { + return this.innerProperties() == null ? null : this.innerProperties().inheritedFrom(); + } + + /** + * Get the resourcesCoverageStatus property: This field is available for subscription-level only, and reflects the + * coverage status of the resources under the subscription. Please note: The "pricingTier" field reflects the plan + * status of the subscription. However, since the plan status can also be defined at the resource level, there + * might be misalignment between the subscription's plan status and the resource status. This field helps indicate + * the coverage status of the resources. + * + * @return the resourcesCoverageStatus value. + */ + public ResourcesCoverageStatus resourcesCoverageStatus() { + return this.innerProperties() == null ? null : this.innerProperties().resourcesCoverageStatus(); } /** * Get the extensions property: Optional. List of extensions offered under a plan. - * + * * @return the extensions value. */ public List extensions() { @@ -141,7 +195,7 @@ public List extensions() { /** * Set the extensions property: Optional. List of extensions offered under a plan. - * + * * @param extensions the extensions value to set. * @return the PricingInner object itself. */ @@ -153,9 +207,29 @@ public PricingInner withExtensions(List extensions) { return this; } + /** + * Get the deprecated property: Optional. True if the plan is deprecated. If there are replacing plans they will + * appear in `replacedBy` property. + * + * @return the deprecated value. + */ + public Boolean deprecated() { + return this.innerProperties() == null ? null : this.innerProperties().deprecated(); + } + + /** + * Get the replacedBy property: Optional. List of plans that replace this plan. This property exists only if this + * plan is deprecated. + * + * @return the replacedBy value. + */ + public List replacedBy() { + return this.innerProperties() == null ? null : this.innerProperties().replacedBy(); + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingListInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingListInner.java index 51b2670b30a29..89e92c08ec658 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingListInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingListInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of pricing configurations response. */ +/** + * List of pricing configurations response. + */ @Fluent public final class PricingListInner { /* @@ -18,13 +20,15 @@ public final class PricingListInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of PricingListInner class. */ + /** + * Creates an instance of PricingListInner class. + */ public PricingListInner() { } /** * Get the value property: List of pricing configurations. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: List of pricing configurations. - * + * * @param value the value value to set. * @return the PricingListInner object itself. */ @@ -44,14 +48,13 @@ public PricingListInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model PricingListInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model PricingListInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingProperties.java index 681894d02f723..e3cd403183f80 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/PricingProperties.java @@ -6,26 +6,33 @@ import com.azure.core.annotation.Fluent; import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.models.Enforce; import com.azure.resourcemanager.security.models.Extension; +import com.azure.resourcemanager.security.models.Inherited; import com.azure.resourcemanager.security.models.PricingTier; +import com.azure.resourcemanager.security.models.ResourcesCoverageStatus; import com.fasterxml.jackson.annotation.JsonProperty; import java.time.Duration; import java.time.OffsetDateTime; import java.util.List; -/** Pricing properties for the relevant scope. */ +/** + * Pricing properties for the relevant scope. + */ @Fluent public final class PricingProperties { /* - * The pricing tier value. Microsoft Defender for Cloud is provided in two pricing tiers: free and standard. The - * standard tier offers advanced security capabilities, while the free tier offers basic security features. + * Indicates whether the Defender plan is enabled on the selected scope. Microsoft Defender for Cloud is provided + * in two pricing tiers: free and standard. The standard tier offers advanced security capabilities, while the free + * tier offers basic security features. */ @JsonProperty(value = "pricingTier", required = true) private PricingTier pricingTier; /* * The sub-plan selected for a Standard pricing configuration, when more than one sub-plan is available. Each - * sub-plan enables a set of security features. When not specified, full plan is applied. + * sub-plan enables a set of security features. When not specified, full plan is applied. For VirtualMachines plan, + * available sub plans are 'P1' & 'P2', where for resource level only 'P1' sub plan is supported. */ @JsonProperty(value = "subPlan") private String subPlan; @@ -43,6 +50,47 @@ public final class PricingProperties { @JsonProperty(value = "enablementTime", access = JsonProperty.Access.WRITE_ONLY) private OffsetDateTime enablementTime; + /* + * If set to "False", it allows the descendants of this scope to override the pricing configuration set on this + * scope (allows setting inherited="False"). If set to "True", it prevents overrides and forces this pricing + * configuration on all the descendants of this scope. This field is only available for subscription-level pricing. + */ + @JsonProperty(value = "enforce") + private Enforce enforce; + + /* + * "inherited" = "True" indicates that the current scope inherits its pricing configuration from its parent. The ID + * of the parent scope that provides the inherited configuration is displayed in the "inheritedFrom" field. On the + * other hand, "inherited" = "False" indicates that the current scope has its own pricing configuration explicitly + * set, and does not inherit from its parent. This field is read only and available only for resource-level + * pricing. + */ + @JsonProperty(value = "inherited", access = JsonProperty.Access.WRITE_ONLY) + private Inherited inherited; + + /* + * The id of the scope inherited from. "Null" if not inherited. This field is only available for resource-level + * pricing. + */ + @JsonProperty(value = "inheritedFrom", access = JsonProperty.Access.WRITE_ONLY) + private String inheritedFrom; + + /* + * This field is available for subscription-level only, and reflects the coverage status of the resources under the + * subscription. Please note: The "pricingTier" field reflects the plan status of the subscription. However, since + * the plan status can also be defined at the resource level, there might be misalignment between the + * subscription's plan status and the resource status. This field helps indicate the coverage status of the + * resources. + */ + @JsonProperty(value = "resourcesCoverageStatus", access = JsonProperty.Access.WRITE_ONLY) + private ResourcesCoverageStatus resourcesCoverageStatus; + + /* + * Optional. List of extensions offered under a plan. + */ + @JsonProperty(value = "extensions") + private List extensions; + /* * Optional. True if the plan is deprecated. If there are replacing plans they will appear in `replacedBy` property */ @@ -55,21 +103,17 @@ public final class PricingProperties { @JsonProperty(value = "replacedBy", access = JsonProperty.Access.WRITE_ONLY) private List replacedBy; - /* - * Optional. List of extensions offered under a plan. + /** + * Creates an instance of PricingProperties class. */ - @JsonProperty(value = "extensions") - private List extensions; - - /** Creates an instance of PricingProperties class. */ public PricingProperties() { } /** - * Get the pricingTier property: The pricing tier value. Microsoft Defender for Cloud is provided in two pricing - * tiers: free and standard. The standard tier offers advanced security capabilities, while the free tier offers - * basic security features. - * + * Get the pricingTier property: Indicates whether the Defender plan is enabled on the selected scope. Microsoft + * Defender for Cloud is provided in two pricing tiers: free and standard. The standard tier offers advanced + * security capabilities, while the free tier offers basic security features. + * * @return the pricingTier value. */ public PricingTier pricingTier() { @@ -77,10 +121,10 @@ public PricingTier pricingTier() { } /** - * Set the pricingTier property: The pricing tier value. Microsoft Defender for Cloud is provided in two pricing - * tiers: free and standard. The standard tier offers advanced security capabilities, while the free tier offers - * basic security features. - * + * Set the pricingTier property: Indicates whether the Defender plan is enabled on the selected scope. Microsoft + * Defender for Cloud is provided in two pricing tiers: free and standard. The standard tier offers advanced + * security capabilities, while the free tier offers basic security features. + * * @param pricingTier the pricingTier value to set. * @return the PricingProperties object itself. */ @@ -90,9 +134,11 @@ public PricingProperties withPricingTier(PricingTier pricingTier) { } /** - * Get the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one sub-plan - * is available. Each sub-plan enables a set of security features. When not specified, full plan is applied. - * + * Get the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one + * sub-plan is available. Each sub-plan enables a set of security features. When not specified, full plan is + * applied. For VirtualMachines plan, available sub plans are 'P1' & 'P2', where for resource level only 'P1' + * sub plan is supported. + * * @return the subPlan value. */ public String subPlan() { @@ -100,9 +146,11 @@ public String subPlan() { } /** - * Set the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one sub-plan - * is available. Each sub-plan enables a set of security features. When not specified, full plan is applied. - * + * Set the subPlan property: The sub-plan selected for a Standard pricing configuration, when more than one + * sub-plan is available. Each sub-plan enables a set of security features. When not specified, full plan is + * applied. For VirtualMachines plan, available sub plans are 'P1' & 'P2', where for resource level only 'P1' + * sub plan is supported. + * * @param subPlan the subPlan value to set. * @return the PricingProperties object itself. */ @@ -114,7 +162,7 @@ public PricingProperties withSubPlan(String subPlan) { /** * Get the freeTrialRemainingTime property: The duration left for the subscriptions free trial period - in ISO 8601 * format (e.g. P3Y6M4DT12H30M5S). - * + * * @return the freeTrialRemainingTime value. */ public Duration freeTrialRemainingTime() { @@ -124,7 +172,7 @@ public Duration freeTrialRemainingTime() { /** * Get the enablementTime property: Optional. If `pricingTier` is `Standard` then this property holds the date of * the last time the `pricingTier` was set to `Standard`, when available (e.g 2023-03-01T12:42:42.1921106Z). - * + * * @return the enablementTime value. */ public OffsetDateTime enablementTime() { @@ -132,28 +180,70 @@ public OffsetDateTime enablementTime() { } /** - * Get the deprecated property: Optional. True if the plan is deprecated. If there are replacing plans they will - * appear in `replacedBy` property. - * - * @return the deprecated value. + * Get the enforce property: If set to "False", it allows the descendants of this scope to override the pricing + * configuration set on this scope (allows setting inherited="False"). If set to "True", it prevents overrides and + * forces this pricing configuration on all the descendants of this scope. This field is only available for + * subscription-level pricing. + * + * @return the enforce value. */ - public Boolean deprecated() { - return this.deprecated; + public Enforce enforce() { + return this.enforce; } /** - * Get the replacedBy property: Optional. List of plans that replace this plan. This property exists only if this - * plan is deprecated. - * - * @return the replacedBy value. + * Set the enforce property: If set to "False", it allows the descendants of this scope to override the pricing + * configuration set on this scope (allows setting inherited="False"). If set to "True", it prevents overrides and + * forces this pricing configuration on all the descendants of this scope. This field is only available for + * subscription-level pricing. + * + * @param enforce the enforce value to set. + * @return the PricingProperties object itself. */ - public List replacedBy() { - return this.replacedBy; + public PricingProperties withEnforce(Enforce enforce) { + this.enforce = enforce; + return this; + } + + /** + * Get the inherited property: "inherited" = "True" indicates that the current scope inherits its pricing + * configuration from its parent. The ID of the parent scope that provides the inherited configuration is displayed + * in the "inheritedFrom" field. On the other hand, "inherited" = "False" indicates that the current scope has its + * own pricing configuration explicitly set, and does not inherit from its parent. This field is read only and + * available only for resource-level pricing. + * + * @return the inherited value. + */ + public Inherited inherited() { + return this.inherited; + } + + /** + * Get the inheritedFrom property: The id of the scope inherited from. "Null" if not inherited. This field is only + * available for resource-level pricing. + * + * @return the inheritedFrom value. + */ + public String inheritedFrom() { + return this.inheritedFrom; + } + + /** + * Get the resourcesCoverageStatus property: This field is available for subscription-level only, and reflects the + * coverage status of the resources under the subscription. Please note: The "pricingTier" field reflects the plan + * status of the subscription. However, since the plan status can also be defined at the resource level, there + * might be misalignment between the subscription's plan status and the resource status. This field helps indicate + * the coverage status of the resources. + * + * @return the resourcesCoverageStatus value. + */ + public ResourcesCoverageStatus resourcesCoverageStatus() { + return this.resourcesCoverageStatus; } /** * Get the extensions property: Optional. List of extensions offered under a plan. - * + * * @return the extensions value. */ public List extensions() { @@ -162,7 +252,7 @@ public List extensions() { /** * Set the extensions property: Optional. List of extensions offered under a plan. - * + * * @param extensions the extensions value to set. * @return the PricingProperties object itself. */ @@ -171,16 +261,35 @@ public PricingProperties withExtensions(List extensions) { return this; } + /** + * Get the deprecated property: Optional. True if the plan is deprecated. If there are replacing plans they will + * appear in `replacedBy` property. + * + * @return the deprecated value. + */ + public Boolean deprecated() { + return this.deprecated; + } + + /** + * Get the replacedBy property: Optional. List of plans that replace this plan. This property exists only if this + * plan is deprecated. + * + * @return the replacedBy value. + */ + public List replacedBy() { + return this.replacedBy; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (pricingTier() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property pricingTier in model PricingProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property pricingTier in model PricingProperties")); } if (extensions() != null) { extensions().forEach(e -> e.validate()); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentInner.java index 308a6f823cecc..f954272f06623 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.State; import com.fasterxml.jackson.annotation.JsonProperty; -/** Regulatory compliance assessment details and state. */ +/** + * Regulatory compliance assessment details and state. + */ @Fluent public final class RegulatoryComplianceAssessmentInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class RegulatoryComplianceAssessmentInner extends ProxyResource { @JsonProperty(value = "properties") private RegulatoryComplianceAssessmentProperties innerProperties; - /** Creates an instance of RegulatoryComplianceAssessmentInner class. */ + /** + * Creates an instance of RegulatoryComplianceAssessmentInner class. + */ public RegulatoryComplianceAssessmentInner() { } /** * Get the innerProperties property: Regulatory compliance assessment data. - * + * * @return the innerProperties value. */ private RegulatoryComplianceAssessmentProperties innerProperties() { @@ -33,7 +37,7 @@ private RegulatoryComplianceAssessmentProperties innerProperties() { /** * Get the description property: The description of the regulatory compliance assessment. - * + * * @return the description value. */ public String description() { @@ -42,7 +46,7 @@ public String description() { /** * Get the assessmentType property: The expected type of assessment contained in the AssessmentDetailsLink. - * + * * @return the assessmentType value. */ public String assessmentType() { @@ -52,7 +56,7 @@ public String assessmentType() { /** * Get the assessmentDetailsLink property: Link to more detailed assessment results data. The response type will be * according to the assessmentType field. - * + * * @return the assessmentDetailsLink value. */ public String assessmentDetailsLink() { @@ -61,7 +65,7 @@ public String assessmentDetailsLink() { /** * Get the state property: Aggregative state based on the assessment's scanned resources states. - * + * * @return the state value. */ public State state() { @@ -70,7 +74,7 @@ public State state() { /** * Set the state property: Aggregative state based on the assessment's scanned resources states. - * + * * @param state the state value to set. * @return the RegulatoryComplianceAssessmentInner object itself. */ @@ -84,7 +88,7 @@ public RegulatoryComplianceAssessmentInner withState(State state) { /** * Get the passedResources property: The given assessment's related resources count with passed state. - * + * * @return the passedResources value. */ public Integer passedResources() { @@ -93,7 +97,7 @@ public Integer passedResources() { /** * Get the failedResources property: The given assessment's related resources count with failed state. - * + * * @return the failedResources value. */ public Integer failedResources() { @@ -102,7 +106,7 @@ public Integer failedResources() { /** * Get the skippedResources property: The given assessment's related resources count with skipped state. - * + * * @return the skippedResources value. */ public Integer skippedResources() { @@ -111,7 +115,7 @@ public Integer skippedResources() { /** * Get the unsupportedResources property: The given assessment's related resources count with unsupported state. - * + * * @return the unsupportedResources value. */ public Integer unsupportedResources() { @@ -120,7 +124,7 @@ public Integer unsupportedResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentProperties.java index 71a2f73ca1518..693742fff1107 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceAssessmentProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.State; import com.fasterxml.jackson.annotation.JsonProperty; -/** Regulatory compliance assessment data. */ +/** + * Regulatory compliance assessment data. + */ @Fluent public final class RegulatoryComplianceAssessmentProperties { /* @@ -59,13 +61,15 @@ public final class RegulatoryComplianceAssessmentProperties { @JsonProperty(value = "unsupportedResources", access = JsonProperty.Access.WRITE_ONLY) private Integer unsupportedResources; - /** Creates an instance of RegulatoryComplianceAssessmentProperties class. */ + /** + * Creates an instance of RegulatoryComplianceAssessmentProperties class. + */ public RegulatoryComplianceAssessmentProperties() { } /** * Get the description property: The description of the regulatory compliance assessment. - * + * * @return the description value. */ public String description() { @@ -74,7 +78,7 @@ public String description() { /** * Get the assessmentType property: The expected type of assessment contained in the AssessmentDetailsLink. - * + * * @return the assessmentType value. */ public String assessmentType() { @@ -84,7 +88,7 @@ public String assessmentType() { /** * Get the assessmentDetailsLink property: Link to more detailed assessment results data. The response type will be * according to the assessmentType field. - * + * * @return the assessmentDetailsLink value. */ public String assessmentDetailsLink() { @@ -93,7 +97,7 @@ public String assessmentDetailsLink() { /** * Get the state property: Aggregative state based on the assessment's scanned resources states. - * + * * @return the state value. */ public State state() { @@ -102,7 +106,7 @@ public State state() { /** * Set the state property: Aggregative state based on the assessment's scanned resources states. - * + * * @param state the state value to set. * @return the RegulatoryComplianceAssessmentProperties object itself. */ @@ -113,7 +117,7 @@ public RegulatoryComplianceAssessmentProperties withState(State state) { /** * Get the passedResources property: The given assessment's related resources count with passed state. - * + * * @return the passedResources value. */ public Integer passedResources() { @@ -122,7 +126,7 @@ public Integer passedResources() { /** * Get the failedResources property: The given assessment's related resources count with failed state. - * + * * @return the failedResources value. */ public Integer failedResources() { @@ -131,7 +135,7 @@ public Integer failedResources() { /** * Get the skippedResources property: The given assessment's related resources count with skipped state. - * + * * @return the skippedResources value. */ public Integer skippedResources() { @@ -140,7 +144,7 @@ public Integer skippedResources() { /** * Get the unsupportedResources property: The given assessment's related resources count with unsupported state. - * + * * @return the unsupportedResources value. */ public Integer unsupportedResources() { @@ -149,7 +153,7 @@ public Integer unsupportedResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlInner.java index 44b3715c5fc67..b92479007f613 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.State; import com.fasterxml.jackson.annotation.JsonProperty; -/** Regulatory compliance control details and state. */ +/** + * Regulatory compliance control details and state. + */ @Fluent public final class RegulatoryComplianceControlInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class RegulatoryComplianceControlInner extends ProxyResource { @JsonProperty(value = "properties") private RegulatoryComplianceControlProperties innerProperties; - /** Creates an instance of RegulatoryComplianceControlInner class. */ + /** + * Creates an instance of RegulatoryComplianceControlInner class. + */ public RegulatoryComplianceControlInner() { } /** * Get the innerProperties property: Regulatory compliance control data. - * + * * @return the innerProperties value. */ private RegulatoryComplianceControlProperties innerProperties() { @@ -33,7 +37,7 @@ private RegulatoryComplianceControlProperties innerProperties() { /** * Get the description property: The description of the regulatory compliance control. - * + * * @return the description value. */ public String description() { @@ -42,7 +46,7 @@ public String description() { /** * Get the state property: Aggregative state based on the control's supported assessments states. - * + * * @return the state value. */ public State state() { @@ -51,7 +55,7 @@ public State state() { /** * Set the state property: Aggregative state based on the control's supported assessments states. - * + * * @param state the state value to set. * @return the RegulatoryComplianceControlInner object itself. */ @@ -66,7 +70,7 @@ public RegulatoryComplianceControlInner withState(State state) { /** * Get the passedAssessments property: The number of supported regulatory compliance assessments of the given * control with a passed state. - * + * * @return the passedAssessments value. */ public Integer passedAssessments() { @@ -76,7 +80,7 @@ public Integer passedAssessments() { /** * Get the failedAssessments property: The number of supported regulatory compliance assessments of the given * control with a failed state. - * + * * @return the failedAssessments value. */ public Integer failedAssessments() { @@ -86,7 +90,7 @@ public Integer failedAssessments() { /** * Get the skippedAssessments property: The number of supported regulatory compliance assessments of the given * control with a skipped state. - * + * * @return the skippedAssessments value. */ public Integer skippedAssessments() { @@ -95,7 +99,7 @@ public Integer skippedAssessments() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlProperties.java index fc016649b89ab..07fd3539ba945 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceControlProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.State; import com.fasterxml.jackson.annotation.JsonProperty; -/** Regulatory compliance control data. */ +/** + * Regulatory compliance control data. + */ @Fluent public final class RegulatoryComplianceControlProperties { /* @@ -41,13 +43,15 @@ public final class RegulatoryComplianceControlProperties { @JsonProperty(value = "skippedAssessments", access = JsonProperty.Access.WRITE_ONLY) private Integer skippedAssessments; - /** Creates an instance of RegulatoryComplianceControlProperties class. */ + /** + * Creates an instance of RegulatoryComplianceControlProperties class. + */ public RegulatoryComplianceControlProperties() { } /** * Get the description property: The description of the regulatory compliance control. - * + * * @return the description value. */ public String description() { @@ -56,7 +60,7 @@ public String description() { /** * Get the state property: Aggregative state based on the control's supported assessments states. - * + * * @return the state value. */ public State state() { @@ -65,7 +69,7 @@ public State state() { /** * Set the state property: Aggregative state based on the control's supported assessments states. - * + * * @param state the state value to set. * @return the RegulatoryComplianceControlProperties object itself. */ @@ -77,7 +81,7 @@ public RegulatoryComplianceControlProperties withState(State state) { /** * Get the passedAssessments property: The number of supported regulatory compliance assessments of the given * control with a passed state. - * + * * @return the passedAssessments value. */ public Integer passedAssessments() { @@ -87,7 +91,7 @@ public Integer passedAssessments() { /** * Get the failedAssessments property: The number of supported regulatory compliance assessments of the given * control with a failed state. - * + * * @return the failedAssessments value. */ public Integer failedAssessments() { @@ -97,7 +101,7 @@ public Integer failedAssessments() { /** * Get the skippedAssessments property: The number of supported regulatory compliance assessments of the given * control with a skipped state. - * + * * @return the skippedAssessments value. */ public Integer skippedAssessments() { @@ -106,7 +110,7 @@ public Integer skippedAssessments() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardInner.java index 462389336ec01..cb2a3663524d6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.State; import com.fasterxml.jackson.annotation.JsonProperty; -/** Regulatory compliance standard details and state. */ +/** + * Regulatory compliance standard details and state. + */ @Fluent public final class RegulatoryComplianceStandardInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class RegulatoryComplianceStandardInner extends ProxyResource { @JsonProperty(value = "properties") private RegulatoryComplianceStandardProperties innerProperties; - /** Creates an instance of RegulatoryComplianceStandardInner class. */ + /** + * Creates an instance of RegulatoryComplianceStandardInner class. + */ public RegulatoryComplianceStandardInner() { } /** * Get the innerProperties property: Regulatory compliance standard data. - * + * * @return the innerProperties value. */ private RegulatoryComplianceStandardProperties innerProperties() { @@ -33,7 +37,7 @@ private RegulatoryComplianceStandardProperties innerProperties() { /** * Get the state property: Aggregative state based on the standard's supported controls states. - * + * * @return the state value. */ public State state() { @@ -42,7 +46,7 @@ public State state() { /** * Set the state property: Aggregative state based on the standard's supported controls states. - * + * * @param state the state value to set. * @return the RegulatoryComplianceStandardInner object itself. */ @@ -57,7 +61,7 @@ public RegulatoryComplianceStandardInner withState(State state) { /** * Get the passedControls property: The number of supported regulatory compliance controls of the given standard * with a passed state. - * + * * @return the passedControls value. */ public Integer passedControls() { @@ -67,7 +71,7 @@ public Integer passedControls() { /** * Get the failedControls property: The number of supported regulatory compliance controls of the given standard * with a failed state. - * + * * @return the failedControls value. */ public Integer failedControls() { @@ -77,7 +81,7 @@ public Integer failedControls() { /** * Get the skippedControls property: The number of supported regulatory compliance controls of the given standard * with a skipped state. - * + * * @return the skippedControls value. */ public Integer skippedControls() { @@ -87,7 +91,7 @@ public Integer skippedControls() { /** * Get the unsupportedControls property: The number of regulatory compliance controls of the given standard which * are unsupported by automated assessments. - * + * * @return the unsupportedControls value. */ public Integer unsupportedControls() { @@ -96,7 +100,7 @@ public Integer unsupportedControls() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardProperties.java index a70b722ae82ff..399380675aaca 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RegulatoryComplianceStandardProperties.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.models.State; import com.fasterxml.jackson.annotation.JsonProperty; -/** Regulatory compliance standard data. */ +/** + * Regulatory compliance standard data. + */ @Fluent public final class RegulatoryComplianceStandardProperties { /* @@ -42,13 +44,15 @@ public final class RegulatoryComplianceStandardProperties { @JsonProperty(value = "unsupportedControls", access = JsonProperty.Access.WRITE_ONLY) private Integer unsupportedControls; - /** Creates an instance of RegulatoryComplianceStandardProperties class. */ + /** + * Creates an instance of RegulatoryComplianceStandardProperties class. + */ public RegulatoryComplianceStandardProperties() { } /** * Get the state property: Aggregative state based on the standard's supported controls states. - * + * * @return the state value. */ public State state() { @@ -57,7 +61,7 @@ public State state() { /** * Set the state property: Aggregative state based on the standard's supported controls states. - * + * * @param state the state value to set. * @return the RegulatoryComplianceStandardProperties object itself. */ @@ -69,7 +73,7 @@ public RegulatoryComplianceStandardProperties withState(State state) { /** * Get the passedControls property: The number of supported regulatory compliance controls of the given standard * with a passed state. - * + * * @return the passedControls value. */ public Integer passedControls() { @@ -79,7 +83,7 @@ public Integer passedControls() { /** * Get the failedControls property: The number of supported regulatory compliance controls of the given standard * with a failed state. - * + * * @return the failedControls value. */ public Integer failedControls() { @@ -89,7 +93,7 @@ public Integer failedControls() { /** * Get the skippedControls property: The number of supported regulatory compliance controls of the given standard * with a skipped state. - * + * * @return the skippedControls value. */ public Integer skippedControls() { @@ -99,7 +103,7 @@ public Integer skippedControls() { /** * Get the unsupportedControls property: The number of regulatory compliance controls of the given standard which * are unsupported by automated assessments. - * + * * @return the unsupportedControls value. */ public Integer unsupportedControls() { @@ -108,7 +112,7 @@ public Integer unsupportedControls() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RuleResultsInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RuleResultsInner.java index 2737512d54422..7ccd3cff0ea88 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RuleResultsInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RuleResultsInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.RuleResultsProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Rule results. */ +/** + * Rule results. + */ @Fluent public final class RuleResultsInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class RuleResultsInner extends ProxyResource { @JsonProperty(value = "properties") private RuleResultsProperties properties; - /** Creates an instance of RuleResultsInner class. */ + /** + * Creates an instance of RuleResultsInner class. + */ public RuleResultsInner() { } /** * Get the properties property: Rule results properties. - * + * * @return the properties value. */ public RuleResultsProperties properties() { @@ -33,7 +37,7 @@ public RuleResultsProperties properties() { /** * Set the properties property: Rule results properties. - * + * * @param properties the properties value to set. * @return the RuleResultsInner object itself. */ @@ -44,7 +48,7 @@ public RuleResultsInner withProperties(RuleResultsProperties properties) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RulesResultsInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RulesResultsInner.java index 2463dd2b01fa4..ea3f8d28ad6b5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RulesResultsInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/RulesResultsInner.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of rules results. */ +/** + * A list of rules results. + */ @Fluent public final class RulesResultsInner { /* @@ -17,13 +19,15 @@ public final class RulesResultsInner { @JsonProperty(value = "value") private List value; - /** Creates an instance of RulesResultsInner class. */ + /** + * Creates an instance of RulesResultsInner class. + */ public RulesResultsInner() { } /** * Get the value property: List of rule results. - * + * * @return the value value. */ public List value() { @@ -32,7 +36,7 @@ public List value() { /** * Set the value property: List of rule results. - * + * * @param value the value value to set. * @return the RulesResultsInner object itself. */ @@ -43,7 +47,7 @@ public RulesResultsInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanInner.java index 55dba8acba1da..72374ef25f0f5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.ScanProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** A vulnerability assessment scan record. */ +/** + * A vulnerability assessment scan record. + */ @Fluent public final class ScanInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class ScanInner extends ProxyResource { @JsonProperty(value = "properties") private ScanProperties properties; - /** Creates an instance of ScanInner class. */ + /** + * Creates an instance of ScanInner class. + */ public ScanInner() { } /** * Get the properties property: A vulnerability assessment scan record properties. - * + * * @return the properties value. */ public ScanProperties properties() { @@ -33,7 +37,7 @@ public ScanProperties properties() { /** * Set the properties property: A vulnerability assessment scan record properties. - * + * * @param properties the properties value to set. * @return the ScanInner object itself. */ @@ -44,7 +48,7 @@ public ScanInner withProperties(ScanProperties properties) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultInner.java index 2878009933cd9..b43faa716bd60 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.ScanResultProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** A vulnerability assessment scan result for a single rule. */ +/** + * A vulnerability assessment scan result for a single rule. + */ @Fluent public final class ScanResultInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class ScanResultInner extends ProxyResource { @JsonProperty(value = "properties") private ScanResultProperties properties; - /** Creates an instance of ScanResultInner class. */ + /** + * Creates an instance of ScanResultInner class. + */ public ScanResultInner() { } /** * Get the properties property: A vulnerability assessment scan result properties for a single rule. - * + * * @return the properties value. */ public ScanResultProperties properties() { @@ -33,7 +37,7 @@ public ScanResultProperties properties() { /** * Set the properties property: A vulnerability assessment scan result properties for a single rule. - * + * * @param properties the properties value to set. * @return the ScanResultInner object itself. */ @@ -44,7 +48,7 @@ public ScanResultInner withProperties(ScanResultProperties properties) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultsInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultsInner.java index 5a1807b5830cc..90d05e1ce0c1f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultsInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScanResultsInner.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of vulnerability assessment scan results. */ +/** + * A list of vulnerability assessment scan results. + */ @Fluent public final class ScanResultsInner { /* @@ -17,13 +19,15 @@ public final class ScanResultsInner { @JsonProperty(value = "value") private List value; - /** Creates an instance of ScanResultsInner class. */ + /** + * Creates an instance of ScanResultsInner class. + */ public ScanResultsInner() { } /** * Get the value property: List of vulnerability assessment scan results. - * + * * @return the value value. */ public List value() { @@ -32,7 +36,7 @@ public List value() { /** * Set the value property: List of vulnerability assessment scan results. - * + * * @param value the value value to set. * @return the ScanResultsInner object itself. */ @@ -43,7 +47,7 @@ public ScanResultsInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScansInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScansInner.java index 18b1d47cef6ac..250deedf95ac4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScansInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScansInner.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of vulnerability assessment scan records. */ +/** + * A list of vulnerability assessment scan records. + */ @Fluent public final class ScansInner { /* @@ -17,13 +19,15 @@ public final class ScansInner { @JsonProperty(value = "value") private List value; - /** Creates an instance of ScansInner class. */ + /** + * Creates an instance of ScansInner class. + */ public ScansInner() { } /** * Get the value property: List of vulnerability assessment scan records. - * + * * @return the value value. */ public List value() { @@ -32,7 +36,7 @@ public List value() { /** * Set the value property: List of vulnerability assessment scan records. - * + * * @param value the value value to set. * @return the ScansInner object itself. */ @@ -43,7 +47,7 @@ public ScansInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScoreDetails.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScoreDetails.java index 86bc7f839e8f2..3189d73d24038 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScoreDetails.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/ScoreDetails.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Calculation result data. */ +/** + * Calculation result data. + */ @Immutable public final class ScoreDetails { /* @@ -28,13 +30,15 @@ public final class ScoreDetails { @JsonProperty(value = "percentage", access = JsonProperty.Access.WRITE_ONLY) private Double percentage; - /** Creates an instance of ScoreDetails class. */ + /** + * Creates an instance of ScoreDetails class. + */ public ScoreDetails() { } /** * Get the max property: Maximum score available. - * + * * @return the max value. */ public Integer max() { @@ -43,7 +47,7 @@ public Integer max() { /** * Get the current property: Current score. - * + * * @return the current value. */ public Double current() { @@ -53,7 +57,7 @@ public Double current() { /** * Get the percentage property: Ratio of the current score divided by the maximum. Rounded to 4 digits after the * decimal point. - * + * * @return the percentage value. */ public Double percentage() { @@ -62,7 +66,7 @@ public Double percentage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemInner.java index 01753634ece23..f667855532f8d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemInner.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Information about the security control. */ +/** + * Information about the security control. + */ @Immutable public final class SecureScoreControlDefinitionItemInner extends ProxyResource { /* @@ -20,13 +22,15 @@ public final class SecureScoreControlDefinitionItemInner extends ProxyResource { @JsonProperty(value = "properties") private SecureScoreControlDefinitionItemProperties innerProperties; - /** Creates an instance of SecureScoreControlDefinitionItemInner class. */ + /** + * Creates an instance of SecureScoreControlDefinitionItemInner class. + */ public SecureScoreControlDefinitionItemInner() { } /** * Get the innerProperties property: Security Control Definition Properties. - * + * * @return the innerProperties value. */ private SecureScoreControlDefinitionItemProperties innerProperties() { @@ -35,7 +39,7 @@ private SecureScoreControlDefinitionItemProperties innerProperties() { /** * Get the displayName property: User friendly display name of the control. - * + * * @return the displayName value. */ public String displayName() { @@ -44,7 +48,7 @@ public String displayName() { /** * Get the description property: User friendly description of the control. - * + * * @return the description value. */ public String description() { @@ -53,7 +57,7 @@ public String description() { /** * Get the maxScore property: Maximum control score (0..10). - * + * * @return the maxScore value. */ public Integer maxScore() { @@ -62,7 +66,7 @@ public Integer maxScore() { /** * Get the source property: Source object from which the control was created. - * + * * @return the source value. */ public SecureScoreControlDefinitionSource source() { @@ -72,7 +76,7 @@ public SecureScoreControlDefinitionSource source() { /** * Get the assessmentDefinitions property: Array of assessments metadata IDs that are included in this security * control. - * + * * @return the assessmentDefinitions value. */ public List assessmentDefinitions() { @@ -81,7 +85,7 @@ public List assessmentDefinitions() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemProperties.java index 85b8e148b797f..86186c34049b9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDefinitionItemProperties.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Security Control Definition Properties. */ +/** + * Security Control Definition Properties. + */ @Immutable public final class SecureScoreControlDefinitionItemProperties { /* @@ -43,13 +45,15 @@ public final class SecureScoreControlDefinitionItemProperties { @JsonProperty(value = "assessmentDefinitions", access = JsonProperty.Access.WRITE_ONLY) private List assessmentDefinitions; - /** Creates an instance of SecureScoreControlDefinitionItemProperties class. */ + /** + * Creates an instance of SecureScoreControlDefinitionItemProperties class. + */ public SecureScoreControlDefinitionItemProperties() { } /** * Get the displayName property: User friendly display name of the control. - * + * * @return the displayName value. */ public String displayName() { @@ -58,7 +62,7 @@ public String displayName() { /** * Get the description property: User friendly description of the control. - * + * * @return the description value. */ public String description() { @@ -67,7 +71,7 @@ public String description() { /** * Get the maxScore property: Maximum control score (0..10). - * + * * @return the maxScore value. */ public Integer maxScore() { @@ -76,7 +80,7 @@ public Integer maxScore() { /** * Get the source property: Source object from which the control was created. - * + * * @return the source value. */ public SecureScoreControlDefinitionSource source() { @@ -86,7 +90,7 @@ public SecureScoreControlDefinitionSource source() { /** * Get the assessmentDefinitions property: Array of assessments metadata IDs that are included in this security * control. - * + * * @return the assessmentDefinitions value. */ public List assessmentDefinitions() { @@ -95,7 +99,7 @@ public List assessmentDefinitions() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDetailsInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDetailsInner.java index bb6d92135d582..14ddcbb0eab4a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDetailsInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlDetailsInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.ProxyResource; import com.fasterxml.jackson.annotation.JsonProperty; -/** Details of the security control, its score, and the health status of the relevant resources. */ +/** + * Details of the security control, its score, and the health status of the relevant resources. + */ @Fluent public final class SecureScoreControlDetailsInner extends ProxyResource { /* @@ -17,13 +19,15 @@ public final class SecureScoreControlDetailsInner extends ProxyResource { @JsonProperty(value = "properties") private SecureScoreControlScoreDetailsInner innerProperties; - /** Creates an instance of SecureScoreControlDetailsInner class. */ + /** + * Creates an instance of SecureScoreControlDetailsInner class. + */ public SecureScoreControlDetailsInner() { } /** * Get the innerProperties property: Calculation result data in control level. - * + * * @return the innerProperties value. */ private SecureScoreControlScoreDetailsInner innerProperties() { @@ -32,7 +36,7 @@ private SecureScoreControlScoreDetailsInner innerProperties() { /** * Get the displayName property: User friendly display name of the control. - * + * * @return the displayName value. */ public String displayName() { @@ -41,7 +45,7 @@ public String displayName() { /** * Get the healthyResourceCount property: Number of healthy resources in the control. - * + * * @return the healthyResourceCount value. */ public Integer healthyResourceCount() { @@ -50,7 +54,7 @@ public Integer healthyResourceCount() { /** * Get the unhealthyResourceCount property: Number of unhealthy resources in the control. - * + * * @return the unhealthyResourceCount value. */ public Integer unhealthyResourceCount() { @@ -59,7 +63,7 @@ public Integer unhealthyResourceCount() { /** * Get the notApplicableResourceCount property: Number of not applicable resources in the control. - * + * * @return the notApplicableResourceCount value. */ public Integer notApplicableResourceCount() { @@ -69,7 +73,7 @@ public Integer notApplicableResourceCount() { /** * Get the weight property: The relative weight for this specific control in each of your subscriptions. Used when * calculating an aggregated score for this control across all of your subscriptions. - * + * * @return the weight value. */ public Long weight() { @@ -78,7 +82,7 @@ public Long weight() { /** * Get the definition property: Information about the security control. - * + * * @return the definition value. */ public SecureScoreControlDefinitionItemInner definition() { @@ -87,7 +91,7 @@ public SecureScoreControlDefinitionItemInner definition() { /** * Set the definition property: Information about the security control. - * + * * @param definition the definition value to set. * @return the SecureScoreControlDetailsInner object itself. */ @@ -101,7 +105,7 @@ public SecureScoreControlDetailsInner withDefinition(SecureScoreControlDefinitio /** * Get the max property: Maximum score available. - * + * * @return the max value. */ public Integer max() { @@ -110,7 +114,7 @@ public Integer max() { /** * Get the current property: Current score. - * + * * @return the current value. */ public Double current() { @@ -120,7 +124,7 @@ public Double current() { /** * Get the percentage property: Ratio of the current score divided by the maximum. Rounded to 4 digits after the * decimal point. - * + * * @return the percentage value. */ public Double percentage() { @@ -129,7 +133,7 @@ public Double percentage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlScoreDetailsInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlScoreDetailsInner.java index 2b4dc29cf1f47..af4f558bfa4d1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlScoreDetailsInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreControlScoreDetailsInner.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Calculation result data in control level. */ +/** + * Calculation result data in control level. + */ @Fluent public final class SecureScoreControlScoreDetailsInner { /* @@ -53,13 +55,15 @@ public final class SecureScoreControlScoreDetailsInner { @JsonProperty(value = "definition") private SecureScoreControlDefinitionItemInner definition; - /** Creates an instance of SecureScoreControlScoreDetailsInner class. */ + /** + * Creates an instance of SecureScoreControlScoreDetailsInner class. + */ public SecureScoreControlScoreDetailsInner() { } /** * Get the displayName property: User friendly display name of the control. - * + * * @return the displayName value. */ public String displayName() { @@ -68,7 +72,7 @@ public String displayName() { /** * Get the innerScore property: Actual score object for the control. - * + * * @return the innerScore value. */ private ScoreDetails innerScore() { @@ -77,7 +81,7 @@ private ScoreDetails innerScore() { /** * Get the healthyResourceCount property: Number of healthy resources in the control. - * + * * @return the healthyResourceCount value. */ public Integer healthyResourceCount() { @@ -86,7 +90,7 @@ public Integer healthyResourceCount() { /** * Get the unhealthyResourceCount property: Number of unhealthy resources in the control. - * + * * @return the unhealthyResourceCount value. */ public Integer unhealthyResourceCount() { @@ -95,7 +99,7 @@ public Integer unhealthyResourceCount() { /** * Get the notApplicableResourceCount property: Number of not applicable resources in the control. - * + * * @return the notApplicableResourceCount value. */ public Integer notApplicableResourceCount() { @@ -105,7 +109,7 @@ public Integer notApplicableResourceCount() { /** * Get the weight property: The relative weight for this specific control in each of your subscriptions. Used when * calculating an aggregated score for this control across all of your subscriptions. - * + * * @return the weight value. */ public Long weight() { @@ -114,7 +118,7 @@ public Long weight() { /** * Get the definition property: Information about the security control. - * + * * @return the definition value. */ public SecureScoreControlDefinitionItemInner definition() { @@ -123,7 +127,7 @@ public SecureScoreControlDefinitionItemInner definition() { /** * Set the definition property: Information about the security control. - * + * * @param definition the definition value to set. * @return the SecureScoreControlScoreDetailsInner object itself. */ @@ -134,7 +138,7 @@ public SecureScoreControlScoreDetailsInner withDefinition(SecureScoreControlDefi /** * Get the max property: Maximum score available. - * + * * @return the max value. */ public Integer max() { @@ -143,7 +147,7 @@ public Integer max() { /** * Get the current property: Current score. - * + * * @return the current value. */ public Double current() { @@ -153,7 +157,7 @@ public Double current() { /** * Get the percentage property: Ratio of the current score divided by the maximum. Rounded to 4 digits after the * decimal point. - * + * * @return the percentage value. */ public Double percentage() { @@ -162,7 +166,7 @@ public Double percentage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemInner.java index c0a9ae5164559..92e59606bbe85 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.ProxyResource; import com.fasterxml.jackson.annotation.JsonProperty; -/** Secure score item data model. */ +/** + * Secure score item data model. + */ @Immutable public final class SecureScoreItemInner extends ProxyResource { /* @@ -17,13 +19,15 @@ public final class SecureScoreItemInner extends ProxyResource { @JsonProperty(value = "properties", access = JsonProperty.Access.WRITE_ONLY) private SecureScoreItemProperties innerProperties; - /** Creates an instance of SecureScoreItemInner class. */ + /** + * Creates an instance of SecureScoreItemInner class. + */ public SecureScoreItemInner() { } /** * Get the innerProperties property: Secure score item. - * + * * @return the innerProperties value. */ private SecureScoreItemProperties innerProperties() { @@ -32,7 +36,7 @@ private SecureScoreItemProperties innerProperties() { /** * Get the displayName property: The initiative’s name. - * + * * @return the displayName value. */ public String displayName() { @@ -42,7 +46,7 @@ public String displayName() { /** * Get the weight property: The relative weight for each subscription. Used when calculating an aggregated secure * score for multiple subscriptions. - * + * * @return the weight value. */ public Long weight() { @@ -51,7 +55,7 @@ public Long weight() { /** * Get the max property: Maximum score available. - * + * * @return the max value. */ public Integer max() { @@ -60,7 +64,7 @@ public Integer max() { /** * Get the current property: Current score. - * + * * @return the current value. */ public Double current() { @@ -70,7 +74,7 @@ public Double current() { /** * Get the percentage property: Ratio of the current score divided by the maximum. Rounded to 4 digits after the * decimal point. - * + * * @return the percentage value. */ public Double percentage() { @@ -79,7 +83,7 @@ public Double percentage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemProperties.java index 889361f4149e3..ea45787bafa58 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecureScoreItemProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Describes properties of a calculated secure score. */ +/** + * Describes properties of a calculated secure score. + */ @Immutable public final class SecureScoreItemProperties { /* @@ -29,13 +31,15 @@ public final class SecureScoreItemProperties { @JsonProperty(value = "weight", access = JsonProperty.Access.WRITE_ONLY) private Long weight; - /** Creates an instance of SecureScoreItemProperties class. */ + /** + * Creates an instance of SecureScoreItemProperties class. + */ public SecureScoreItemProperties() { } /** * Get the displayName property: The initiative’s name. - * + * * @return the displayName value. */ public String displayName() { @@ -44,7 +48,7 @@ public String displayName() { /** * Get the innerScore property: score object. - * + * * @return the innerScore value. */ private ScoreDetails innerScore() { @@ -54,7 +58,7 @@ private ScoreDetails innerScore() { /** * Get the weight property: The relative weight for each subscription. Used when calculating an aggregated secure * score for multiple subscriptions. - * + * * @return the weight value. */ public Long weight() { @@ -63,7 +67,7 @@ public Long weight() { /** * Get the max property: Maximum score available. - * + * * @return the max value. */ public Integer max() { @@ -72,7 +76,7 @@ public Integer max() { /** * Get the current property: Current score. - * + * * @return the current value. */ public Double current() { @@ -82,7 +86,7 @@ public Double current() { /** * Get the percentage property: Ratio of the current score divided by the maximum. Rounded to 4 digits after the * decimal point. - * + * * @return the percentage value. */ public Double percentage() { @@ -91,7 +95,7 @@ public Double percentage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataProperties.java index d2452d561791e..5904ba31260b7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataProperties.java @@ -16,7 +16,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes properties of an assessment metadata. */ +/** + * Describes properties of an assessment metadata. + */ @Fluent public class SecurityAssessmentMetadataProperties { /* @@ -92,13 +94,15 @@ public class SecurityAssessmentMetadataProperties { @JsonProperty(value = "partnerData") private SecurityAssessmentMetadataPartnerData partnerData; - /** Creates an instance of SecurityAssessmentMetadataProperties class. */ + /** + * Creates an instance of SecurityAssessmentMetadataProperties class. + */ public SecurityAssessmentMetadataProperties() { } /** * Get the displayName property: User friendly display name of the assessment. - * + * * @return the displayName value. */ public String displayName() { @@ -107,7 +111,7 @@ public String displayName() { /** * Set the displayName property: User friendly display name of the assessment. - * + * * @param displayName the displayName value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -119,7 +123,7 @@ public SecurityAssessmentMetadataProperties withDisplayName(String displayName) /** * Get the policyDefinitionId property: Azure resource ID of the policy definition that turns this assessment * calculation on. - * + * * @return the policyDefinitionId value. */ public String policyDefinitionId() { @@ -128,7 +132,7 @@ public String policyDefinitionId() { /** * Get the description property: Human readable description of the assessment. - * + * * @return the description value. */ public String description() { @@ -137,7 +141,7 @@ public String description() { /** * Set the description property: Human readable description of the assessment. - * + * * @param description the description value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -149,7 +153,7 @@ public SecurityAssessmentMetadataProperties withDescription(String description) /** * Get the remediationDescription property: Human readable description of what you should do to mitigate this * security issue. - * + * * @return the remediationDescription value. */ public String remediationDescription() { @@ -159,7 +163,7 @@ public String remediationDescription() { /** * Set the remediationDescription property: Human readable description of what you should do to mitigate this * security issue. - * + * * @param remediationDescription the remediationDescription value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -170,7 +174,7 @@ public SecurityAssessmentMetadataProperties withRemediationDescription(String re /** * Get the categories property: The categories property. - * + * * @return the categories value. */ public List categories() { @@ -179,7 +183,7 @@ public List categories() { /** * Set the categories property: The categories property. - * + * * @param categories the categories value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -190,7 +194,7 @@ public SecurityAssessmentMetadataProperties withCategories(List cate /** * Get the severity property: The severity level of the assessment. - * + * * @return the severity value. */ public Severity severity() { @@ -199,7 +203,7 @@ public Severity severity() { /** * Set the severity property: The severity level of the assessment. - * + * * @param severity the severity value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -210,7 +214,7 @@ public SecurityAssessmentMetadataProperties withSeverity(Severity severity) { /** * Get the userImpact property: The user impact of the assessment. - * + * * @return the userImpact value. */ public UserImpact userImpact() { @@ -219,7 +223,7 @@ public UserImpact userImpact() { /** * Set the userImpact property: The user impact of the assessment. - * + * * @param userImpact the userImpact value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -230,7 +234,7 @@ public SecurityAssessmentMetadataProperties withUserImpact(UserImpact userImpact /** * Get the implementationEffort property: The implementation effort required to remediate this assessment. - * + * * @return the implementationEffort value. */ public ImplementationEffort implementationEffort() { @@ -239,7 +243,7 @@ public ImplementationEffort implementationEffort() { /** * Set the implementationEffort property: The implementation effort required to remediate this assessment. - * + * * @param implementationEffort the implementationEffort value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -250,7 +254,7 @@ public SecurityAssessmentMetadataProperties withImplementationEffort(Implementat /** * Get the threats property: The threats property. - * + * * @return the threats value. */ public List threats() { @@ -259,7 +263,7 @@ public List threats() { /** * Set the threats property: The threats property. - * + * * @param threats the threats value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -270,7 +274,7 @@ public SecurityAssessmentMetadataProperties withThreats(List threats) { /** * Get the preview property: True if this assessment is in preview release status. - * + * * @return the preview value. */ public Boolean preview() { @@ -279,7 +283,7 @@ public Boolean preview() { /** * Set the preview property: True if this assessment is in preview release status. - * + * * @param preview the preview value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -291,7 +295,7 @@ public SecurityAssessmentMetadataProperties withPreview(Boolean preview) { /** * Get the assessmentType property: BuiltIn if the assessment based on built-in Azure Policy definition, Custom if * the assessment based on custom Azure Policy definition. - * + * * @return the assessmentType value. */ public AssessmentType assessmentType() { @@ -301,7 +305,7 @@ public AssessmentType assessmentType() { /** * Set the assessmentType property: BuiltIn if the assessment based on built-in Azure Policy definition, Custom if * the assessment based on custom Azure Policy definition. - * + * * @param assessmentType the assessmentType value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -312,7 +316,7 @@ public SecurityAssessmentMetadataProperties withAssessmentType(AssessmentType as /** * Get the partnerData property: Describes the partner that created the assessment. - * + * * @return the partnerData value. */ public SecurityAssessmentMetadataPartnerData partnerData() { @@ -321,7 +325,7 @@ public SecurityAssessmentMetadataPartnerData partnerData() { /** * Set the partnerData property: Describes the partner that created the assessment. - * + * * @param partnerData the partnerData value to set. * @return the SecurityAssessmentMetadataProperties object itself. */ @@ -332,27 +336,21 @@ public SecurityAssessmentMetadataProperties withPartnerData(SecurityAssessmentMe /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (displayName() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property displayName in model SecurityAssessmentMetadataProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property displayName in model SecurityAssessmentMetadataProperties")); } if (severity() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property severity in model SecurityAssessmentMetadataProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property severity in model SecurityAssessmentMetadataProperties")); } if (assessmentType() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property assessmentType in model SecurityAssessmentMetadataProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property assessmentType in model SecurityAssessmentMetadataProperties")); } if (partnerData() != null) { partnerData().validate(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataPropertiesResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataPropertiesResponse.java index eb080f0c54c3f..3d30e4db62099 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataPropertiesResponse.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataPropertiesResponse.java @@ -18,7 +18,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes properties of an assessment metadata response. */ +/** + * Describes properties of an assessment metadata response. + */ @Fluent public final class SecurityAssessmentMetadataPropertiesResponse extends SecurityAssessmentMetadataProperties { /* @@ -45,13 +47,15 @@ public final class SecurityAssessmentMetadataPropertiesResponse extends Security @JsonProperty(value = "techniques") private List techniques; - /** Creates an instance of SecurityAssessmentMetadataPropertiesResponse class. */ + /** + * Creates an instance of SecurityAssessmentMetadataPropertiesResponse class. + */ public SecurityAssessmentMetadataPropertiesResponse() { } /** * Get the publishDates property: The publishDates property. - * + * * @return the publishDates value. */ public SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates() { @@ -60,19 +64,19 @@ public SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates() { /** * Set the publishDates property: The publishDates property. - * + * * @param publishDates the publishDates value to set. * @return the SecurityAssessmentMetadataPropertiesResponse object itself. */ - public SecurityAssessmentMetadataPropertiesResponse withPublishDates( - SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates) { + public SecurityAssessmentMetadataPropertiesResponse + withPublishDates(SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates) { this.publishDates = publishDates; return this; } /** * Get the plannedDeprecationDate property: The plannedDeprecationDate property. - * + * * @return the plannedDeprecationDate value. */ public String plannedDeprecationDate() { @@ -81,7 +85,7 @@ public String plannedDeprecationDate() { /** * Set the plannedDeprecationDate property: The plannedDeprecationDate property. - * + * * @param plannedDeprecationDate the plannedDeprecationDate value to set. * @return the SecurityAssessmentMetadataPropertiesResponse object itself. */ @@ -92,7 +96,7 @@ public SecurityAssessmentMetadataPropertiesResponse withPlannedDeprecationDate(S /** * Get the tactics property: The tactics property. - * + * * @return the tactics value. */ public List tactics() { @@ -101,7 +105,7 @@ public List tactics() { /** * Set the tactics property: The tactics property. - * + * * @param tactics the tactics value to set. * @return the SecurityAssessmentMetadataPropertiesResponse object itself. */ @@ -112,7 +116,7 @@ public SecurityAssessmentMetadataPropertiesResponse withTactics(List ta /** * Get the techniques property: The techniques property. - * + * * @return the techniques value. */ public List techniques() { @@ -121,7 +125,7 @@ public List techniques() { /** * Set the techniques property: The techniques property. - * + * * @param techniques the techniques value to set. * @return the SecurityAssessmentMetadataPropertiesResponse object itself. */ @@ -130,88 +134,110 @@ public SecurityAssessmentMetadataPropertiesResponse withTechniques(List categories) { super.withCategories(categories); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentMetadataPropertiesResponse withSeverity(Severity severity) { super.withSeverity(severity); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentMetadataPropertiesResponse withUserImpact(UserImpact userImpact) { super.withUserImpact(userImpact); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override - public SecurityAssessmentMetadataPropertiesResponse withImplementationEffort( - ImplementationEffort implementationEffort) { + public SecurityAssessmentMetadataPropertiesResponse + withImplementationEffort(ImplementationEffort implementationEffort) { super.withImplementationEffort(implementationEffort); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentMetadataPropertiesResponse withThreats(List threats) { super.withThreats(threats); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentMetadataPropertiesResponse withPreview(Boolean preview) { super.withPreview(preview); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentMetadataPropertiesResponse withAssessmentType(AssessmentType assessmentType) { super.withAssessmentType(assessmentType); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override - public SecurityAssessmentMetadataPropertiesResponse withPartnerData( - SecurityAssessmentMetadataPartnerData partnerData) { + public SecurityAssessmentMetadataPropertiesResponse + withPartnerData(SecurityAssessmentMetadataPartnerData partnerData) { super.withPartnerData(partnerData); return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataResponseInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataResponseInner.java index 1167a0a33df64..eb0cb48a1e91d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataResponseInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentMetadataResponseInner.java @@ -19,7 +19,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Security assessment metadata response. */ +/** + * Security assessment metadata response. + */ @Fluent public final class SecurityAssessmentMetadataResponseInner extends ProxyResource { /* @@ -28,13 +30,15 @@ public final class SecurityAssessmentMetadataResponseInner extends ProxyResource @JsonProperty(value = "properties") private SecurityAssessmentMetadataPropertiesResponse innerProperties; - /** Creates an instance of SecurityAssessmentMetadataResponseInner class. */ + /** + * Creates an instance of SecurityAssessmentMetadataResponseInner class. + */ public SecurityAssessmentMetadataResponseInner() { } /** * Get the innerProperties property: Describes properties of an assessment metadata response. - * + * * @return the innerProperties value. */ private SecurityAssessmentMetadataPropertiesResponse innerProperties() { @@ -43,7 +47,7 @@ private SecurityAssessmentMetadataPropertiesResponse innerProperties() { /** * Get the publishDates property: The publishDates property. - * + * * @return the publishDates value. */ public SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates() { @@ -52,12 +56,12 @@ public SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates() { /** * Set the publishDates property: The publishDates property. - * + * * @param publishDates the publishDates value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ - public SecurityAssessmentMetadataResponseInner withPublishDates( - SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates) { + public SecurityAssessmentMetadataResponseInner + withPublishDates(SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates) { if (this.innerProperties() == null) { this.innerProperties = new SecurityAssessmentMetadataPropertiesResponse(); } @@ -67,7 +71,7 @@ public SecurityAssessmentMetadataResponseInner withPublishDates( /** * Get the plannedDeprecationDate property: The plannedDeprecationDate property. - * + * * @return the plannedDeprecationDate value. */ public String plannedDeprecationDate() { @@ -76,7 +80,7 @@ public String plannedDeprecationDate() { /** * Set the plannedDeprecationDate property: The plannedDeprecationDate property. - * + * * @param plannedDeprecationDate the plannedDeprecationDate value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -90,7 +94,7 @@ public SecurityAssessmentMetadataResponseInner withPlannedDeprecationDate(String /** * Get the tactics property: The tactics property. - * + * * @return the tactics value. */ public List tactics() { @@ -99,7 +103,7 @@ public List tactics() { /** * Set the tactics property: The tactics property. - * + * * @param tactics the tactics value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -113,7 +117,7 @@ public SecurityAssessmentMetadataResponseInner withTactics(List tactics /** * Get the techniques property: The techniques property. - * + * * @return the techniques value. */ public List techniques() { @@ -122,7 +126,7 @@ public List techniques() { /** * Set the techniques property: The techniques property. - * + * * @param techniques the techniques value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -136,7 +140,7 @@ public SecurityAssessmentMetadataResponseInner withTechniques(List t /** * Get the displayName property: User friendly display name of the assessment. - * + * * @return the displayName value. */ public String displayName() { @@ -145,7 +149,7 @@ public String displayName() { /** * Set the displayName property: User friendly display name of the assessment. - * + * * @param displayName the displayName value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -160,7 +164,7 @@ public SecurityAssessmentMetadataResponseInner withDisplayName(String displayNam /** * Get the policyDefinitionId property: Azure resource ID of the policy definition that turns this assessment * calculation on. - * + * * @return the policyDefinitionId value. */ public String policyDefinitionId() { @@ -169,7 +173,7 @@ public String policyDefinitionId() { /** * Get the description property: Human readable description of the assessment. - * + * * @return the description value. */ public String description() { @@ -178,7 +182,7 @@ public String description() { /** * Set the description property: Human readable description of the assessment. - * + * * @param description the description value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -193,7 +197,7 @@ public SecurityAssessmentMetadataResponseInner withDescription(String descriptio /** * Get the remediationDescription property: Human readable description of what you should do to mitigate this * security issue. - * + * * @return the remediationDescription value. */ public String remediationDescription() { @@ -203,7 +207,7 @@ public String remediationDescription() { /** * Set the remediationDescription property: Human readable description of what you should do to mitigate this * security issue. - * + * * @param remediationDescription the remediationDescription value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -217,7 +221,7 @@ public SecurityAssessmentMetadataResponseInner withRemediationDescription(String /** * Get the categories property: The categories property. - * + * * @return the categories value. */ public List categories() { @@ -226,7 +230,7 @@ public List categories() { /** * Set the categories property: The categories property. - * + * * @param categories the categories value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -240,7 +244,7 @@ public SecurityAssessmentMetadataResponseInner withCategories(List c /** * Get the severity property: The severity level of the assessment. - * + * * @return the severity value. */ public Severity severity() { @@ -249,7 +253,7 @@ public Severity severity() { /** * Set the severity property: The severity level of the assessment. - * + * * @param severity the severity value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -263,7 +267,7 @@ public SecurityAssessmentMetadataResponseInner withSeverity(Severity severity) { /** * Get the userImpact property: The user impact of the assessment. - * + * * @return the userImpact value. */ public UserImpact userImpact() { @@ -272,7 +276,7 @@ public UserImpact userImpact() { /** * Set the userImpact property: The user impact of the assessment. - * + * * @param userImpact the userImpact value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -286,7 +290,7 @@ public SecurityAssessmentMetadataResponseInner withUserImpact(UserImpact userImp /** * Get the implementationEffort property: The implementation effort required to remediate this assessment. - * + * * @return the implementationEffort value. */ public ImplementationEffort implementationEffort() { @@ -295,7 +299,7 @@ public ImplementationEffort implementationEffort() { /** * Set the implementationEffort property: The implementation effort required to remediate this assessment. - * + * * @param implementationEffort the implementationEffort value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -309,7 +313,7 @@ public SecurityAssessmentMetadataResponseInner withImplementationEffort(Implemen /** * Get the threats property: The threats property. - * + * * @return the threats value. */ public List threats() { @@ -318,7 +322,7 @@ public List threats() { /** * Set the threats property: The threats property. - * + * * @param threats the threats value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -332,7 +336,7 @@ public SecurityAssessmentMetadataResponseInner withThreats(List threats /** * Get the preview property: True if this assessment is in preview release status. - * + * * @return the preview value. */ public Boolean preview() { @@ -341,7 +345,7 @@ public Boolean preview() { /** * Set the preview property: True if this assessment is in preview release status. - * + * * @param preview the preview value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -356,7 +360,7 @@ public SecurityAssessmentMetadataResponseInner withPreview(Boolean preview) { /** * Get the assessmentType property: BuiltIn if the assessment based on built-in Azure Policy definition, Custom if * the assessment based on custom Azure Policy definition. - * + * * @return the assessmentType value. */ public AssessmentType assessmentType() { @@ -366,7 +370,7 @@ public AssessmentType assessmentType() { /** * Set the assessmentType property: BuiltIn if the assessment based on built-in Azure Policy definition, Custom if * the assessment based on custom Azure Policy definition. - * + * * @param assessmentType the assessmentType value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -380,7 +384,7 @@ public SecurityAssessmentMetadataResponseInner withAssessmentType(AssessmentType /** * Get the partnerData property: Describes the partner that created the assessment. - * + * * @return the partnerData value. */ public SecurityAssessmentMetadataPartnerData partnerData() { @@ -389,7 +393,7 @@ public SecurityAssessmentMetadataPartnerData partnerData() { /** * Set the partnerData property: Describes the partner that created the assessment. - * + * * @param partnerData the partnerData value to set. * @return the SecurityAssessmentMetadataResponseInner object itself. */ @@ -403,7 +407,7 @@ public SecurityAssessmentMetadataResponseInner withPartnerData(SecurityAssessmen /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentProperties.java index d5d7b67530e43..7c76f5f704f87 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentProperties.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Describes properties of an assessment. */ +/** + * Describes properties of an assessment. + */ @Fluent public final class SecurityAssessmentProperties extends SecurityAssessmentPropertiesBase { /* @@ -22,13 +24,15 @@ public final class SecurityAssessmentProperties extends SecurityAssessmentProper @JsonProperty(value = "status", required = true) private AssessmentStatus status; - /** Creates an instance of SecurityAssessmentProperties class. */ + /** + * Creates an instance of SecurityAssessmentProperties class. + */ public SecurityAssessmentProperties() { } /** * Get the status property: The result of the assessment. - * + * * @return the status value. */ public AssessmentStatus status() { @@ -37,7 +41,7 @@ public AssessmentStatus status() { /** * Set the status property: The result of the assessment. - * + * * @param status the status value to set. * @return the SecurityAssessmentProperties object itself. */ @@ -46,28 +50,36 @@ public SecurityAssessmentProperties withStatus(AssessmentStatus status) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentProperties withResourceDetails(ResourceDetails resourceDetails) { super.withResourceDetails(resourceDetails); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentProperties withAdditionalData(Map additionalData) { super.withAdditionalData(additionalData); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentProperties withMetadata(SecurityAssessmentMetadataProperties metadata) { super.withMetadata(metadata); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentProperties withPartnersData(SecurityAssessmentPartnerData partnersData) { super.withPartnersData(partnersData); @@ -76,17 +88,15 @@ public SecurityAssessmentProperties withPartnersData(SecurityAssessmentPartnerDa /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (status() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property status in model SecurityAssessmentProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property status in model SecurityAssessmentProperties")); } else { status().validate(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentPropertiesResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentPropertiesResponse.java index 9c4c6cdec8b2a..7847eb919414d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentPropertiesResponse.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentPropertiesResponse.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Describes properties of an assessment. */ +/** + * Describes properties of an assessment. + */ @Fluent public final class SecurityAssessmentPropertiesResponse extends SecurityAssessmentPropertiesBase { /* @@ -22,13 +24,15 @@ public final class SecurityAssessmentPropertiesResponse extends SecurityAssessme @JsonProperty(value = "status", required = true) private AssessmentStatusResponse status; - /** Creates an instance of SecurityAssessmentPropertiesResponse class. */ + /** + * Creates an instance of SecurityAssessmentPropertiesResponse class. + */ public SecurityAssessmentPropertiesResponse() { } /** * Get the status property: The result of the assessment. - * + * * @return the status value. */ public AssessmentStatusResponse status() { @@ -37,7 +41,7 @@ public AssessmentStatusResponse status() { /** * Set the status property: The result of the assessment. - * + * * @param status the status value to set. * @return the SecurityAssessmentPropertiesResponse object itself. */ @@ -46,28 +50,36 @@ public SecurityAssessmentPropertiesResponse withStatus(AssessmentStatusResponse return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentPropertiesResponse withResourceDetails(ResourceDetails resourceDetails) { super.withResourceDetails(resourceDetails); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentPropertiesResponse withAdditionalData(Map additionalData) { super.withAdditionalData(additionalData); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentPropertiesResponse withMetadata(SecurityAssessmentMetadataProperties metadata) { super.withMetadata(metadata); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityAssessmentPropertiesResponse withPartnersData(SecurityAssessmentPartnerData partnersData) { super.withPartnersData(partnersData); @@ -76,17 +88,15 @@ public SecurityAssessmentPropertiesResponse withPartnersData(SecurityAssessmentP /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (status() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property status in model SecurityAssessmentPropertiesResponse")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property status in model SecurityAssessmentPropertiesResponse")); } else { status().validate(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentResponseInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentResponseInner.java index f3579b78ef478..4400e64eb587a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentResponseInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityAssessmentResponseInner.java @@ -13,7 +13,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** Security assessment on a resource - response format. */ +/** + * Security assessment on a resource - response format. + */ @Fluent public final class SecurityAssessmentResponseInner extends ProxyResource { /* @@ -22,13 +24,15 @@ public final class SecurityAssessmentResponseInner extends ProxyResource { @JsonProperty(value = "properties") private SecurityAssessmentPropertiesResponse innerProperties; - /** Creates an instance of SecurityAssessmentResponseInner class. */ + /** + * Creates an instance of SecurityAssessmentResponseInner class. + */ public SecurityAssessmentResponseInner() { } /** * Get the innerProperties property: Describes properties of an assessment. - * + * * @return the innerProperties value. */ private SecurityAssessmentPropertiesResponse innerProperties() { @@ -37,7 +41,7 @@ private SecurityAssessmentPropertiesResponse innerProperties() { /** * Get the status property: The result of the assessment. - * + * * @return the status value. */ public AssessmentStatusResponse status() { @@ -46,7 +50,7 @@ public AssessmentStatusResponse status() { /** * Set the status property: The result of the assessment. - * + * * @param status the status value to set. * @return the SecurityAssessmentResponseInner object itself. */ @@ -60,7 +64,7 @@ public SecurityAssessmentResponseInner withStatus(AssessmentStatusResponse statu /** * Get the resourceDetails property: Details of the resource that was assessed. - * + * * @return the resourceDetails value. */ public ResourceDetails resourceDetails() { @@ -69,7 +73,7 @@ public ResourceDetails resourceDetails() { /** * Set the resourceDetails property: Details of the resource that was assessed. - * + * * @param resourceDetails the resourceDetails value to set. * @return the SecurityAssessmentResponseInner object itself. */ @@ -83,7 +87,7 @@ public SecurityAssessmentResponseInner withResourceDetails(ResourceDetails resou /** * Get the displayName property: User friendly display name of the assessment. - * + * * @return the displayName value. */ public String displayName() { @@ -92,7 +96,7 @@ public String displayName() { /** * Get the additionalData property: Additional data regarding the assessment. - * + * * @return the additionalData value. */ public Map additionalData() { @@ -101,7 +105,7 @@ public Map additionalData() { /** * Set the additionalData property: Additional data regarding the assessment. - * + * * @param additionalData the additionalData value to set. * @return the SecurityAssessmentResponseInner object itself. */ @@ -115,7 +119,7 @@ public SecurityAssessmentResponseInner withAdditionalData(Map ad /** * Get the links property: Links relevant to the assessment. - * + * * @return the links value. */ public AssessmentLinks links() { @@ -124,7 +128,7 @@ public AssessmentLinks links() { /** * Get the metadata property: Describes properties of an assessment metadata. - * + * * @return the metadata value. */ public SecurityAssessmentMetadataProperties metadata() { @@ -133,7 +137,7 @@ public SecurityAssessmentMetadataProperties metadata() { /** * Set the metadata property: Describes properties of an assessment metadata. - * + * * @param metadata the metadata value to set. * @return the SecurityAssessmentResponseInner object itself. */ @@ -147,7 +151,7 @@ public SecurityAssessmentResponseInner withMetadata(SecurityAssessmentMetadataPr /** * Get the partnersData property: Data regarding 3rd party partner integration. - * + * * @return the partnersData value. */ public SecurityAssessmentPartnerData partnersData() { @@ -156,7 +160,7 @@ public SecurityAssessmentPartnerData partnersData() { /** * Set the partnersData property: Data regarding 3rd party partner integration. - * + * * @param partnersData the partnersData value to set. * @return the SecurityAssessmentResponseInner object itself. */ @@ -170,7 +174,7 @@ public SecurityAssessmentResponseInner withPartnersData(SecurityAssessmentPartne /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorInner.java index a58a7ab73eb4f..3c61bfdcb3083 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorInner.java @@ -15,7 +15,9 @@ import java.util.List; import java.util.Map; -/** The security connector resource. */ +/** + * The security connector resource. + */ @Fluent public final class SecurityConnectorInner extends Resource { /* @@ -30,13 +32,15 @@ public final class SecurityConnectorInner extends Resource { @JsonProperty(value = "properties") private SecurityConnectorProperties innerProperties; - /** Creates an instance of SecurityConnectorInner class. */ + /** + * Creates an instance of SecurityConnectorInner class. + */ public SecurityConnectorInner() { } /** * Get the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ public SystemData systemData() { @@ -45,21 +49,25 @@ public SystemData systemData() { /** * Get the innerProperties property: Security connector data. - * + * * @return the innerProperties value. */ private SecurityConnectorProperties innerProperties() { return this.innerProperties; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityConnectorInner withLocation(String location) { super.withLocation(location); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public SecurityConnectorInner withTags(Map tags) { super.withTags(tags); @@ -69,7 +77,7 @@ public SecurityConnectorInner withTags(Map tags) { /** * Get the hierarchyIdentifier property: The multi cloud resource identifier (account id in case of AWS connector, * project number in case of GCP connector). - * + * * @return the hierarchyIdentifier value. */ public String hierarchyIdentifier() { @@ -79,7 +87,7 @@ public String hierarchyIdentifier() { /** * Set the hierarchyIdentifier property: The multi cloud resource identifier (account id in case of AWS connector, * project number in case of GCP connector). - * + * * @param hierarchyIdentifier the hierarchyIdentifier value to set. * @return the SecurityConnectorInner object itself. */ @@ -94,7 +102,7 @@ public SecurityConnectorInner withHierarchyIdentifier(String hierarchyIdentifier /** * Get the hierarchyIdentifierTrialEndDate property: The date on which the trial period will end, if applicable. * Trial period exists for 30 days after upgrading to payed offerings. - * + * * @return the hierarchyIdentifierTrialEndDate value. */ public OffsetDateTime hierarchyIdentifierTrialEndDate() { @@ -103,7 +111,7 @@ public OffsetDateTime hierarchyIdentifierTrialEndDate() { /** * Get the environmentName property: The multi cloud resource's cloud name. - * + * * @return the environmentName value. */ public CloudName environmentName() { @@ -112,7 +120,7 @@ public CloudName environmentName() { /** * Set the environmentName property: The multi cloud resource's cloud name. - * + * * @param environmentName the environmentName value to set. * @return the SecurityConnectorInner object itself. */ @@ -126,7 +134,7 @@ public SecurityConnectorInner withEnvironmentName(CloudName environmentName) { /** * Get the offerings property: A collection of offerings for the security connector. - * + * * @return the offerings value. */ public List offerings() { @@ -135,7 +143,7 @@ public List offerings() { /** * Set the offerings property: A collection of offerings for the security connector. - * + * * @param offerings the offerings value to set. * @return the SecurityConnectorInner object itself. */ @@ -149,7 +157,7 @@ public SecurityConnectorInner withOfferings(List offerings) { /** * Get the environmentData property: The security connector environment data. - * + * * @return the environmentData value. */ public EnvironmentData environmentData() { @@ -158,7 +166,7 @@ public EnvironmentData environmentData() { /** * Set the environmentData property: The security connector environment data. - * + * * @param environmentData the environmentData value to set. * @return the SecurityConnectorInner object itself. */ @@ -172,7 +180,7 @@ public SecurityConnectorInner withEnvironmentData(EnvironmentData environmentDat /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorProperties.java index df14a0076cf6b..914b410bbf871 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityConnectorProperties.java @@ -12,7 +12,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** A set of properties that defines the security connector configuration. */ +/** + * A set of properties that defines the security connector configuration. + */ @Fluent public final class SecurityConnectorProperties { /* @@ -47,14 +49,16 @@ public final class SecurityConnectorProperties { @JsonProperty(value = "environmentData") private EnvironmentData environmentData; - /** Creates an instance of SecurityConnectorProperties class. */ + /** + * Creates an instance of SecurityConnectorProperties class. + */ public SecurityConnectorProperties() { } /** * Get the hierarchyIdentifier property: The multi cloud resource identifier (account id in case of AWS connector, * project number in case of GCP connector). - * + * * @return the hierarchyIdentifier value. */ public String hierarchyIdentifier() { @@ -64,7 +68,7 @@ public String hierarchyIdentifier() { /** * Set the hierarchyIdentifier property: The multi cloud resource identifier (account id in case of AWS connector, * project number in case of GCP connector). - * + * * @param hierarchyIdentifier the hierarchyIdentifier value to set. * @return the SecurityConnectorProperties object itself. */ @@ -76,7 +80,7 @@ public SecurityConnectorProperties withHierarchyIdentifier(String hierarchyIdent /** * Get the hierarchyIdentifierTrialEndDate property: The date on which the trial period will end, if applicable. * Trial period exists for 30 days after upgrading to payed offerings. - * + * * @return the hierarchyIdentifierTrialEndDate value. */ public OffsetDateTime hierarchyIdentifierTrialEndDate() { @@ -85,7 +89,7 @@ public OffsetDateTime hierarchyIdentifierTrialEndDate() { /** * Get the environmentName property: The multi cloud resource's cloud name. - * + * * @return the environmentName value. */ public CloudName environmentName() { @@ -94,7 +98,7 @@ public CloudName environmentName() { /** * Set the environmentName property: The multi cloud resource's cloud name. - * + * * @param environmentName the environmentName value to set. * @return the SecurityConnectorProperties object itself. */ @@ -105,7 +109,7 @@ public SecurityConnectorProperties withEnvironmentName(CloudName environmentName /** * Get the offerings property: A collection of offerings for the security connector. - * + * * @return the offerings value. */ public List offerings() { @@ -114,7 +118,7 @@ public List offerings() { /** * Set the offerings property: A collection of offerings for the security connector. - * + * * @param offerings the offerings value to set. * @return the SecurityConnectorProperties object itself. */ @@ -125,7 +129,7 @@ public SecurityConnectorProperties withOfferings(List offerings) /** * Get the environmentData property: The security connector environment data. - * + * * @return the environmentData value. */ public EnvironmentData environmentData() { @@ -134,7 +138,7 @@ public EnvironmentData environmentData() { /** * Set the environmentData property: The security connector environment data. - * + * * @param environmentData the environmentData value to set. * @return the SecurityConnectorProperties object itself. */ @@ -145,7 +149,7 @@ public SecurityConnectorProperties withEnvironmentData(EnvironmentData environme /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactInner.java index 0606139e3bf25..c8e234311656e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactInner.java @@ -6,11 +6,14 @@ import com.azure.core.annotation.Fluent; import com.azure.core.management.ProxyResource; -import com.azure.resourcemanager.security.models.SecurityContactPropertiesAlertNotifications; +import com.azure.resourcemanager.security.models.NotificationsSource; import com.azure.resourcemanager.security.models.SecurityContactPropertiesNotificationsByRole; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; -/** Contact details and configurations for notifications coming from Microsoft Defender for Cloud. */ +/** + * Contact details and configurations for notifications coming from Microsoft Defender for Cloud. + */ @Fluent public final class SecurityContactInner extends ProxyResource { /* @@ -19,13 +22,15 @@ public final class SecurityContactInner extends ProxyResource { @JsonProperty(value = "properties") private SecurityContactProperties innerProperties; - /** Creates an instance of SecurityContactInner class. */ + /** + * Creates an instance of SecurityContactInner class. + */ public SecurityContactInner() { } /** * Get the innerProperties property: Security contact data. - * + * * @return the innerProperties value. */ private SecurityContactProperties innerProperties() { @@ -35,7 +40,7 @@ private SecurityContactProperties innerProperties() { /** * Get the emails property: List of email addresses which will get notifications from Microsoft Defender for Cloud * by the configurations defined in this security contact. - * + * * @return the emails value. */ public String emails() { @@ -45,7 +50,7 @@ public String emails() { /** * Set the emails property: List of email addresses which will get notifications from Microsoft Defender for Cloud * by the configurations defined in this security contact. - * + * * @param emails the emails value to set. * @return the SecurityContactInner object itself. */ @@ -59,7 +64,7 @@ public SecurityContactInner withEmails(String emails) { /** * Get the phone property: The security contact's phone number. - * + * * @return the phone value. */ public String phone() { @@ -68,7 +73,7 @@ public String phone() { /** * Set the phone property: The security contact's phone number. - * + * * @param phone the phone value to set. * @return the SecurityContactInner object itself. */ @@ -81,32 +86,55 @@ public SecurityContactInner withPhone(String phone) { } /** - * Get the alertNotifications property: Defines whether to send email notifications about new security alerts. - * - * @return the alertNotifications value. + * Get the isEnabled property: Indicates whether the security contact is enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.innerProperties() == null ? null : this.innerProperties().isEnabled(); + } + + /** + * Set the isEnabled property: Indicates whether the security contact is enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the SecurityContactInner object itself. + */ + public SecurityContactInner withIsEnabled(Boolean isEnabled) { + if (this.innerProperties() == null) { + this.innerProperties = new SecurityContactProperties(); + } + this.innerProperties().withIsEnabled(isEnabled); + return this; + } + + /** + * Get the notificationsSources property: A collection of sources types which evaluate the email notification. + * + * @return the notificationsSources value. */ - public SecurityContactPropertiesAlertNotifications alertNotifications() { - return this.innerProperties() == null ? null : this.innerProperties().alertNotifications(); + public List notificationsSources() { + return this.innerProperties() == null ? null : this.innerProperties().notificationsSources(); } /** - * Set the alertNotifications property: Defines whether to send email notifications about new security alerts. - * - * @param alertNotifications the alertNotifications value to set. + * Set the notificationsSources property: A collection of sources types which evaluate the email notification. + * + * @param notificationsSources the notificationsSources value to set. * @return the SecurityContactInner object itself. */ - public SecurityContactInner withAlertNotifications(SecurityContactPropertiesAlertNotifications alertNotifications) { + public SecurityContactInner withNotificationsSources(List notificationsSources) { if (this.innerProperties() == null) { this.innerProperties = new SecurityContactProperties(); } - this.innerProperties().withAlertNotifications(alertNotifications); + this.innerProperties().withNotificationsSources(notificationsSources); return this; } /** * Get the notificationsByRole property: Defines whether to send email notifications from Microsoft Defender for * Cloud to persons with specific RBAC roles on the subscription. - * + * * @return the notificationsByRole value. */ public SecurityContactPropertiesNotificationsByRole notificationsByRole() { @@ -116,12 +144,12 @@ public SecurityContactPropertiesNotificationsByRole notificationsByRole() { /** * Set the notificationsByRole property: Defines whether to send email notifications from Microsoft Defender for * Cloud to persons with specific RBAC roles on the subscription. - * + * * @param notificationsByRole the notificationsByRole value to set. * @return the SecurityContactInner object itself. */ - public SecurityContactInner withNotificationsByRole( - SecurityContactPropertiesNotificationsByRole notificationsByRole) { + public SecurityContactInner + withNotificationsByRole(SecurityContactPropertiesNotificationsByRole notificationsByRole) { if (this.innerProperties() == null) { this.innerProperties = new SecurityContactProperties(); } @@ -131,7 +159,7 @@ public SecurityContactInner withNotificationsByRole( /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactProperties.java index 34c16a792122a..068536412e208 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityContactProperties.java @@ -5,11 +5,14 @@ package com.azure.resourcemanager.security.fluent.models; import com.azure.core.annotation.Fluent; -import com.azure.resourcemanager.security.models.SecurityContactPropertiesAlertNotifications; +import com.azure.resourcemanager.security.models.NotificationsSource; import com.azure.resourcemanager.security.models.SecurityContactPropertiesNotificationsByRole; import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; -/** Describes security contact properties. */ +/** + * Describes security contact properties. + */ @Fluent public final class SecurityContactProperties { /* @@ -26,10 +29,16 @@ public final class SecurityContactProperties { private String phone; /* - * Defines whether to send email notifications about new security alerts + * Indicates whether the security contact is enabled. */ - @JsonProperty(value = "alertNotifications") - private SecurityContactPropertiesAlertNotifications alertNotifications; + @JsonProperty(value = "isEnabled") + private Boolean isEnabled; + + /* + * A collection of sources types which evaluate the email notification. + */ + @JsonProperty(value = "notificationsSources") + private List notificationsSources; /* * Defines whether to send email notifications from Microsoft Defender for Cloud to persons with specific RBAC @@ -38,14 +47,16 @@ public final class SecurityContactProperties { @JsonProperty(value = "notificationsByRole") private SecurityContactPropertiesNotificationsByRole notificationsByRole; - /** Creates an instance of SecurityContactProperties class. */ + /** + * Creates an instance of SecurityContactProperties class. + */ public SecurityContactProperties() { } /** * Get the emails property: List of email addresses which will get notifications from Microsoft Defender for Cloud * by the configurations defined in this security contact. - * + * * @return the emails value. */ public String emails() { @@ -55,7 +66,7 @@ public String emails() { /** * Set the emails property: List of email addresses which will get notifications from Microsoft Defender for Cloud * by the configurations defined in this security contact. - * + * * @param emails the emails value to set. * @return the SecurityContactProperties object itself. */ @@ -66,7 +77,7 @@ public SecurityContactProperties withEmails(String emails) { /** * Get the phone property: The security contact's phone number. - * + * * @return the phone value. */ public String phone() { @@ -75,7 +86,7 @@ public String phone() { /** * Set the phone property: The security contact's phone number. - * + * * @param phone the phone value to set. * @return the SecurityContactProperties object itself. */ @@ -85,30 +96,49 @@ public SecurityContactProperties withPhone(String phone) { } /** - * Get the alertNotifications property: Defines whether to send email notifications about new security alerts. - * - * @return the alertNotifications value. + * Get the isEnabled property: Indicates whether the security contact is enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.isEnabled; + } + + /** + * Set the isEnabled property: Indicates whether the security contact is enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the SecurityContactProperties object itself. + */ + public SecurityContactProperties withIsEnabled(Boolean isEnabled) { + this.isEnabled = isEnabled; + return this; + } + + /** + * Get the notificationsSources property: A collection of sources types which evaluate the email notification. + * + * @return the notificationsSources value. */ - public SecurityContactPropertiesAlertNotifications alertNotifications() { - return this.alertNotifications; + public List notificationsSources() { + return this.notificationsSources; } /** - * Set the alertNotifications property: Defines whether to send email notifications about new security alerts. - * - * @param alertNotifications the alertNotifications value to set. + * Set the notificationsSources property: A collection of sources types which evaluate the email notification. + * + * @param notificationsSources the notificationsSources value to set. * @return the SecurityContactProperties object itself. */ - public SecurityContactProperties withAlertNotifications( - SecurityContactPropertiesAlertNotifications alertNotifications) { - this.alertNotifications = alertNotifications; + public SecurityContactProperties withNotificationsSources(List notificationsSources) { + this.notificationsSources = notificationsSources; return this; } /** * Get the notificationsByRole property: Defines whether to send email notifications from Microsoft Defender for * Cloud to persons with specific RBAC roles on the subscription. - * + * * @return the notificationsByRole value. */ public SecurityContactPropertiesNotificationsByRole notificationsByRole() { @@ -118,24 +148,24 @@ public SecurityContactPropertiesNotificationsByRole notificationsByRole() { /** * Set the notificationsByRole property: Defines whether to send email notifications from Microsoft Defender for * Cloud to persons with specific RBAC roles on the subscription. - * + * * @param notificationsByRole the notificationsByRole value to set. * @return the SecurityContactProperties object itself. */ - public SecurityContactProperties withNotificationsByRole( - SecurityContactPropertiesNotificationsByRole notificationsByRole) { + public SecurityContactProperties + withNotificationsByRole(SecurityContactPropertiesNotificationsByRole notificationsByRole) { this.notificationsByRole = notificationsByRole; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { - if (alertNotifications() != null) { - alertNotifications().validate(); + if (notificationsSources() != null) { + notificationsSources().forEach(e -> e.validate()); } if (notificationsByRole() != null) { notificationsByRole().validate(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorInner.java index 74c332b17887d..f879cf9eb6b7e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorInner.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.models.Identity; import com.fasterxml.jackson.annotation.JsonProperty; -/** Security operator under a given subscription and pricing. */ +/** + * Security operator under a given subscription and pricing. + */ @Fluent public final class SecurityOperatorInner extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class SecurityOperatorInner extends ProxyResource { @JsonProperty(value = "identity") private Identity identity; - /** Creates an instance of SecurityOperatorInner class. */ + /** + * Creates an instance of SecurityOperatorInner class. + */ public SecurityOperatorInner() { } /** * Get the identity property: Identity for the resource. - * + * * @return the identity value. */ public Identity identity() { @@ -33,7 +37,7 @@ public Identity identity() { /** * Set the identity property: Identity for the resource. - * + * * @param identity the identity value to set. * @return the SecurityOperatorInner object itself. */ @@ -44,7 +48,7 @@ public SecurityOperatorInner withIdentity(Identity identity) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorListInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorListInner.java index de1c00d24b3b8..196d2f65dfd18 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorListInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecurityOperatorListInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of SecurityOperator response. */ +/** + * List of SecurityOperator response. + */ @Fluent public final class SecurityOperatorListInner { /* @@ -18,13 +20,15 @@ public final class SecurityOperatorListInner { @JsonProperty(value = "value", required = true) private List value; - /** Creates an instance of SecurityOperatorListInner class. */ + /** + * Creates an instance of SecurityOperatorListInner class. + */ public SecurityOperatorListInner() { } /** * Get the value property: List of SecurityOperator configurations. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: List of SecurityOperator configurations. - * + * * @param value the value value to set. * @return the SecurityOperatorListInner object itself. */ @@ -44,14 +48,13 @@ public SecurityOperatorListInner withValue(List value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model SecurityOperatorListInner")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model SecurityOperatorListInner")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionInner.java index 8428e621636df..062b8b0c2c1bd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionInner.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.security.models.SecurityFamily; import com.fasterxml.jackson.annotation.JsonProperty; -/** The SecuritySolution model. */ +/** + * The SecuritySolution model. + */ @Fluent public final class SecuritySolutionInner extends ProxyResource { /* @@ -25,13 +27,15 @@ public final class SecuritySolutionInner extends ProxyResource { @JsonProperty(value = "location", access = JsonProperty.Access.WRITE_ONLY) private String location; - /** Creates an instance of SecuritySolutionInner class. */ + /** + * Creates an instance of SecuritySolutionInner class. + */ public SecuritySolutionInner() { } /** * Get the innerProperties property: The properties property. - * + * * @return the innerProperties value. */ private SecuritySolutionProperties innerProperties() { @@ -40,7 +44,7 @@ private SecuritySolutionProperties innerProperties() { /** * Get the location property: Location where the resource is stored. - * + * * @return the location value. */ public String location() { @@ -49,7 +53,7 @@ public String location() { /** * Get the securityFamily property: The security family of the security solution. - * + * * @return the securityFamily value. */ public SecurityFamily securityFamily() { @@ -58,7 +62,7 @@ public SecurityFamily securityFamily() { /** * Set the securityFamily property: The security family of the security solution. - * + * * @param securityFamily the securityFamily value to set. * @return the SecuritySolutionInner object itself. */ @@ -72,7 +76,7 @@ public SecuritySolutionInner withSecurityFamily(SecurityFamily securityFamily) { /** * Get the provisioningState property: The security family provisioning State. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -81,7 +85,7 @@ public ProvisioningState provisioningState() { /** * Set the provisioningState property: The security family provisioning State. - * + * * @param provisioningState the provisioningState value to set. * @return the SecuritySolutionInner object itself. */ @@ -95,7 +99,7 @@ public SecuritySolutionInner withProvisioningState(ProvisioningState provisionin /** * Get the template property: The security solutions' template. - * + * * @return the template value. */ public String template() { @@ -104,7 +108,7 @@ public String template() { /** * Set the template property: The security solutions' template. - * + * * @param template the template value to set. * @return the SecuritySolutionInner object itself. */ @@ -118,7 +122,7 @@ public SecuritySolutionInner withTemplate(String template) { /** * Get the protectionStatus property: The security solutions' status. - * + * * @return the protectionStatus value. */ public String protectionStatus() { @@ -127,7 +131,7 @@ public String protectionStatus() { /** * Set the protectionStatus property: The security solutions' status. - * + * * @param protectionStatus the protectionStatus value to set. * @return the SecuritySolutionInner object itself. */ @@ -141,7 +145,7 @@ public SecuritySolutionInner withProtectionStatus(String protectionStatus) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionProperties.java index bc67143e9d6f4..6ade9eea2f7d5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionProperties.java @@ -10,7 +10,9 @@ import com.azure.resourcemanager.security.models.SecurityFamily; import com.fasterxml.jackson.annotation.JsonProperty; -/** The SecuritySolutionProperties model. */ +/** + * The SecuritySolutionProperties model. + */ @Fluent public final class SecuritySolutionProperties { /* @@ -37,13 +39,15 @@ public final class SecuritySolutionProperties { @JsonProperty(value = "protectionStatus", required = true) private String protectionStatus; - /** Creates an instance of SecuritySolutionProperties class. */ + /** + * Creates an instance of SecuritySolutionProperties class. + */ public SecuritySolutionProperties() { } /** * Get the securityFamily property: The security family of the security solution. - * + * * @return the securityFamily value. */ public SecurityFamily securityFamily() { @@ -52,7 +56,7 @@ public SecurityFamily securityFamily() { /** * Set the securityFamily property: The security family of the security solution. - * + * * @param securityFamily the securityFamily value to set. * @return the SecuritySolutionProperties object itself. */ @@ -63,7 +67,7 @@ public SecuritySolutionProperties withSecurityFamily(SecurityFamily securityFami /** * Get the provisioningState property: The security family provisioning State. - * + * * @return the provisioningState value. */ public ProvisioningState provisioningState() { @@ -72,7 +76,7 @@ public ProvisioningState provisioningState() { /** * Set the provisioningState property: The security family provisioning State. - * + * * @param provisioningState the provisioningState value to set. * @return the SecuritySolutionProperties object itself. */ @@ -83,7 +87,7 @@ public SecuritySolutionProperties withProvisioningState(ProvisioningState provis /** * Get the template property: The security solutions' template. - * + * * @return the template value. */ public String template() { @@ -92,7 +96,7 @@ public String template() { /** * Set the template property: The security solutions' template. - * + * * @param template the template value to set. * @return the SecuritySolutionProperties object itself. */ @@ -103,7 +107,7 @@ public SecuritySolutionProperties withTemplate(String template) { /** * Get the protectionStatus property: The security solutions' status. - * + * * @return the protectionStatus value. */ public String protectionStatus() { @@ -112,7 +116,7 @@ public String protectionStatus() { /** * Set the protectionStatus property: The security solutions' status. - * + * * @param protectionStatus the protectionStatus value to set. * @return the SecuritySolutionProperties object itself. */ @@ -123,33 +127,25 @@ public SecuritySolutionProperties withProtectionStatus(String protectionStatus) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (securityFamily() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property securityFamily in model SecuritySolutionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property securityFamily in model SecuritySolutionProperties")); } if (provisioningState() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property provisioningState in model SecuritySolutionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property provisioningState in model SecuritySolutionProperties")); } if (template() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property template in model SecuritySolutionProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property template in model SecuritySolutionProperties")); } if (protectionStatus() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property protectionStatus in model SecuritySolutionProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property protectionStatus in model SecuritySolutionProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionsReferenceDataListInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionsReferenceDataListInner.java index be2aebaf7be45..812ad1b2dbce7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionsReferenceDataListInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/SecuritySolutionsReferenceDataListInner.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The SecuritySolutionsReferenceDataList model. */ +/** + * The SecuritySolutionsReferenceDataList model. + */ @Fluent public final class SecuritySolutionsReferenceDataListInner { /* @@ -18,13 +20,15 @@ public final class SecuritySolutionsReferenceDataListInner { @JsonProperty(value = "value") private List value; - /** Creates an instance of SecuritySolutionsReferenceDataListInner class. */ + /** + * Creates an instance of SecuritySolutionsReferenceDataListInner class. + */ public SecuritySolutionsReferenceDataListInner() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -33,7 +37,7 @@ public List value() { /** * Set the value property: The value property. - * + * * @param value the value value to set. * @return the SecuritySolutionsReferenceDataListInner object itself. */ @@ -44,7 +48,7 @@ public SecuritySolutionsReferenceDataListInner withValue(List value; - /** Creates an instance of ServerVulnerabilityAssessmentsListInner class. */ + /** + * Creates an instance of ServerVulnerabilityAssessmentsListInner class. + */ public ServerVulnerabilityAssessmentsListInner() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -32,7 +36,7 @@ public List value() { /** * Set the value property: The value property. - * + * * @param value the value value to set. * @return the ServerVulnerabilityAssessmentsListInner object itself. */ @@ -43,7 +47,7 @@ public ServerVulnerabilityAssessmentsListInner withValue(List topologyResources() { @@ -68,7 +72,7 @@ public List topologyResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/TopologyResourceProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/TopologyResourceProperties.java index daa7e08a00259..f224815c47107 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/TopologyResourceProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/TopologyResourceProperties.java @@ -10,7 +10,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** The TopologyResourceProperties model. */ +/** + * The TopologyResourceProperties model. + */ @Immutable public final class TopologyResourceProperties { /* @@ -25,13 +27,15 @@ public final class TopologyResourceProperties { @JsonProperty(value = "topologyResources", access = JsonProperty.Access.WRITE_ONLY) private List topologyResources; - /** Creates an instance of TopologyResourceProperties class. */ + /** + * Creates an instance of TopologyResourceProperties class. + */ public TopologyResourceProperties() { } /** * Get the calculatedDateTime property: The UTC time on which the topology was calculated. - * + * * @return the calculatedDateTime value. */ public OffsetDateTime calculatedDateTime() { @@ -40,7 +44,7 @@ public OffsetDateTime calculatedDateTime() { /** * Get the topologyResources property: Azure resources which are part of this topology resource. - * + * * @return the topologyResources value. */ public List topologyResources() { @@ -49,7 +53,7 @@ public List topologyResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/UpdateIoTSecuritySolutionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/UpdateIoTSecuritySolutionProperties.java index f2bf349d65c32..f8585aed96b89 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/UpdateIoTSecuritySolutionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/UpdateIoTSecuritySolutionProperties.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Update Security Solution setting data. */ +/** + * Update Security Solution setting data. + */ @Fluent public final class UpdateIoTSecuritySolutionProperties { /* @@ -25,13 +27,15 @@ public final class UpdateIoTSecuritySolutionProperties { @JsonProperty(value = "recommendationsConfiguration") private List recommendationsConfiguration; - /** Creates an instance of UpdateIoTSecuritySolutionProperties class. */ + /** + * Creates an instance of UpdateIoTSecuritySolutionProperties class. + */ public UpdateIoTSecuritySolutionProperties() { } /** * Get the userDefinedResources property: Properties of the IoT Security solution's user defined resources. - * + * * @return the userDefinedResources value. */ public UserDefinedResourcesProperties userDefinedResources() { @@ -40,19 +44,19 @@ public UserDefinedResourcesProperties userDefinedResources() { /** * Set the userDefinedResources property: Properties of the IoT Security solution's user defined resources. - * + * * @param userDefinedResources the userDefinedResources value to set. * @return the UpdateIoTSecuritySolutionProperties object itself. */ - public UpdateIoTSecuritySolutionProperties withUserDefinedResources( - UserDefinedResourcesProperties userDefinedResources) { + public UpdateIoTSecuritySolutionProperties + withUserDefinedResources(UserDefinedResourcesProperties userDefinedResources) { this.userDefinedResources = userDefinedResources; return this; } /** * Get the recommendationsConfiguration property: List of the configuration status for each recommendation type. - * + * * @return the recommendationsConfiguration value. */ public List recommendationsConfiguration() { @@ -61,19 +65,19 @@ public List recommendationsConfiguration( /** * Set the recommendationsConfiguration property: List of the configuration status for each recommendation type. - * + * * @param recommendationsConfiguration the recommendationsConfiguration value to set. * @return the UpdateIoTSecuritySolutionProperties object itself. */ - public UpdateIoTSecuritySolutionProperties withRecommendationsConfiguration( - List recommendationsConfiguration) { + public UpdateIoTSecuritySolutionProperties + withRecommendationsConfiguration(List recommendationsConfiguration) { this.recommendationsConfiguration = recommendationsConfiguration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingInner.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingInner.java index d6be77efcba1a..cc8dd71e1fbd5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingInner.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingInner.java @@ -8,7 +8,9 @@ import com.azure.core.management.ProxyResource; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configures where to store the OMS agent data for workspaces under a scope. */ +/** + * Configures where to store the OMS agent data for workspaces under a scope. + */ @Fluent public final class WorkspaceSettingInner extends ProxyResource { /* @@ -17,13 +19,15 @@ public final class WorkspaceSettingInner extends ProxyResource { @JsonProperty(value = "properties") private WorkspaceSettingProperties innerProperties; - /** Creates an instance of WorkspaceSettingInner class. */ + /** + * Creates an instance of WorkspaceSettingInner class. + */ public WorkspaceSettingInner() { } /** * Get the innerProperties property: Workspace setting data. - * + * * @return the innerProperties value. */ private WorkspaceSettingProperties innerProperties() { @@ -32,7 +36,7 @@ private WorkspaceSettingProperties innerProperties() { /** * Get the workspaceId property: The full Azure ID of the workspace to save the data in. - * + * * @return the workspaceId value. */ public String workspaceId() { @@ -41,7 +45,7 @@ public String workspaceId() { /** * Set the workspaceId property: The full Azure ID of the workspace to save the data in. - * + * * @param workspaceId the workspaceId value to set. * @return the WorkspaceSettingInner object itself. */ @@ -54,9 +58,9 @@ public WorkspaceSettingInner withWorkspaceId(String workspaceId) { } /** - * Get the scope property: All the VMs in this scope will send their security data to the mentioned workspace unless - * overridden by a setting with more specific scope. - * + * Get the scope property: All the VMs in this scope will send their security data to the mentioned workspace + * unless overridden by a setting with more specific scope. + * * @return the scope value. */ public String scope() { @@ -64,9 +68,9 @@ public String scope() { } /** - * Set the scope property: All the VMs in this scope will send their security data to the mentioned workspace unless - * overridden by a setting with more specific scope. - * + * Set the scope property: All the VMs in this scope will send their security data to the mentioned workspace + * unless overridden by a setting with more specific scope. + * * @param scope the scope value to set. * @return the WorkspaceSettingInner object itself. */ @@ -80,7 +84,7 @@ public WorkspaceSettingInner withScope(String scope) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingProperties.java index 4b9e5b362af56..3a142a2ce6fe9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/WorkspaceSettingProperties.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** Workspace setting data. */ +/** + * Workspace setting data. + */ @Fluent public final class WorkspaceSettingProperties { /* @@ -24,13 +26,15 @@ public final class WorkspaceSettingProperties { @JsonProperty(value = "scope", required = true) private String scope; - /** Creates an instance of WorkspaceSettingProperties class. */ + /** + * Creates an instance of WorkspaceSettingProperties class. + */ public WorkspaceSettingProperties() { } /** * Get the workspaceId property: The full Azure ID of the workspace to save the data in. - * + * * @return the workspaceId value. */ public String workspaceId() { @@ -39,7 +43,7 @@ public String workspaceId() { /** * Set the workspaceId property: The full Azure ID of the workspace to save the data in. - * + * * @param workspaceId the workspaceId value to set. * @return the WorkspaceSettingProperties object itself. */ @@ -49,9 +53,9 @@ public WorkspaceSettingProperties withWorkspaceId(String workspaceId) { } /** - * Get the scope property: All the VMs in this scope will send their security data to the mentioned workspace unless - * overridden by a setting with more specific scope. - * + * Get the scope property: All the VMs in this scope will send their security data to the mentioned workspace + * unless overridden by a setting with more specific scope. + * * @return the scope value. */ public String scope() { @@ -59,9 +63,9 @@ public String scope() { } /** - * Set the scope property: All the VMs in this scope will send their security data to the mentioned workspace unless - * overridden by a setting with more specific scope. - * + * Set the scope property: All the VMs in this scope will send their security data to the mentioned workspace + * unless overridden by a setting with more specific scope. + * * @param scope the scope value to set. * @return the WorkspaceSettingProperties object itself. */ @@ -72,21 +76,17 @@ public WorkspaceSettingProperties withScope(String scope) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (workspaceId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property workspaceId in model WorkspaceSettingProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property workspaceId in model WorkspaceSettingProperties")); } if (scope() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property scope in model WorkspaceSettingProperties")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property scope in model WorkspaceSettingProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/package-info.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/package-info.java index c572ad42092cf..f1e5e652b0f2a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/package-info.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/models/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the inner data models for SecurityCenter. API spec for Microsoft.Security (Azure Security Center) - * resource provider. + * Package containing the inner data models for SecurityCenter. + * API spec for Microsoft.Security (Azure Security Center) resource provider. */ package com.azure.resourcemanager.security.fluent.models; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/package-info.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/package-info.java index e13426491bd08..3de5e22988091 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/package-info.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/fluent/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the service clients for SecurityCenter. API spec for Microsoft.Security (Azure Security Center) - * resource provider. + * Package containing the service clients for SecurityCenter. + * API spec for Microsoft.Security (Azure Security Center) resource provider. */ package com.azure.resourcemanager.security.fluent; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupImpl.java index dac890dface45..d3f0626e89a5a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupImpl.java @@ -19,10 +19,8 @@ import java.util.Collections; import java.util.List; -public final class AdaptiveApplicationControlGroupImpl - implements AdaptiveApplicationControlGroup, - AdaptiveApplicationControlGroup.Definition, - AdaptiveApplicationControlGroup.Update { +public final class AdaptiveApplicationControlGroupImpl implements AdaptiveApplicationControlGroup, + AdaptiveApplicationControlGroup.Definition, AdaptiveApplicationControlGroup.Update { private AdaptiveApplicationControlGroupInner innerObject; private final com.azure.resourcemanager.security.SecurityManager serviceManager; @@ -116,27 +114,19 @@ public AdaptiveApplicationControlGroupImpl withExistingLocation(String ascLocati } public AdaptiveApplicationControlGroup create() { - this.innerObject = - serviceManager - .serviceClient() - .getAdaptiveApplicationControls() - .putWithResponse(ascLocation, groupName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdaptiveApplicationControls() + .putWithResponse(ascLocation, groupName, this.innerModel(), Context.NONE).getValue(); return this; } public AdaptiveApplicationControlGroup create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAdaptiveApplicationControls() - .putWithResponse(ascLocation, groupName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdaptiveApplicationControls() + .putWithResponse(ascLocation, groupName, this.innerModel(), context).getValue(); return this; } - AdaptiveApplicationControlGroupImpl( - String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + AdaptiveApplicationControlGroupImpl(String name, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = new AdaptiveApplicationControlGroupInner(); this.serviceManager = serviceManager; this.groupName = name; @@ -147,51 +137,34 @@ public AdaptiveApplicationControlGroupImpl update() { } public AdaptiveApplicationControlGroup apply() { - this.innerObject = - serviceManager - .serviceClient() - .getAdaptiveApplicationControls() - .putWithResponse(ascLocation, groupName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdaptiveApplicationControls() + .putWithResponse(ascLocation, groupName, this.innerModel(), Context.NONE).getValue(); return this; } public AdaptiveApplicationControlGroup apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAdaptiveApplicationControls() - .putWithResponse(ascLocation, groupName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdaptiveApplicationControls() + .putWithResponse(ascLocation, groupName, this.innerModel(), context).getValue(); return this; } - AdaptiveApplicationControlGroupImpl( - AdaptiveApplicationControlGroupInner innerObject, + AdaptiveApplicationControlGroupImpl(AdaptiveApplicationControlGroupInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.ascLocation = Utils.getValueFromIdByName(innerObject.id(), "locations"); - this.groupName = Utils.getValueFromIdByName(innerObject.id(), "applicationWhitelistings"); + this.ascLocation = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "locations"); + this.groupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "applicationWhitelistings"); } public AdaptiveApplicationControlGroup refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getAdaptiveApplicationControls() - .getWithResponse(ascLocation, groupName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdaptiveApplicationControls() + .getWithResponse(ascLocation, groupName, Context.NONE).getValue(); return this; } public AdaptiveApplicationControlGroup refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAdaptiveApplicationControls() - .getWithResponse(ascLocation, groupName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdaptiveApplicationControls() + .getWithResponse(ascLocation, groupName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupsImpl.java index ed7a53c75298f..346368a857814 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlGroupsImpl.java @@ -17,8 +17,7 @@ public final class AdaptiveApplicationControlGroupsImpl implements AdaptiveAppli private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AdaptiveApplicationControlGroupsImpl( - AdaptiveApplicationControlGroupsInner innerObject, + AdaptiveApplicationControlGroupsImpl(AdaptiveApplicationControlGroupsInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,9 @@ public final class AdaptiveApplicationControlGroupsImpl implements AdaptiveAppli public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new AdaptiveApplicationControlGroupImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new AdaptiveApplicationControlGroupImpl(inner1, this.manager())) + .collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsClientImpl.java index f5377cd34583c..fd7da17576c09 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsClientImpl.java @@ -29,24 +29,28 @@ import com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupsInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AdaptiveApplicationControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in AdaptiveApplicationControlsClient. + */ public final class AdaptiveApplicationControlsClientImpl implements AdaptiveApplicationControlsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AdaptiveApplicationControlsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AdaptiveApplicationControlsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AdaptiveApplicationControlsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - AdaptiveApplicationControlsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AdaptiveApplicationControlsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -57,109 +61,77 @@ public final class AdaptiveApplicationControlsClientImpl implements AdaptiveAppl @Host("{$host}") @ServiceInterface(name = "SecurityCenterAdapti") public interface AdaptiveApplicationControlsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/applicationWhitelistings") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, @QueryParam("includePathRecommendations") Boolean includePathRecommendations, - @QueryParam("summary") Boolean summary, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("summary") Boolean summary, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/applicationWhitelistings/{groupName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/applicationWhitelistings/{groupName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("groupName") String groupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @PathParam("groupName") String groupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/applicationWhitelistings/{groupName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/applicationWhitelistings/{groupName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> put( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("groupName") String groupName, - @QueryParam("api-version") String apiVersion, + Mono> put(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @PathParam("groupName") String groupName, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") AdaptiveApplicationControlGroupInner body, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/applicationWhitelistings/{groupName}") - @ExpectedResponses({200, 202, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/applicationWhitelistings/{groupName}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("groupName") String groupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @PathParam("groupName") String groupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of application control machine groups for the subscription. - * + * * @param includePathRecommendations Include the policy rules. * @param summary Return output in a summarized form. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of application control machine groups for the subscription along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - Boolean includePathRecommendations, Boolean summary) { + private Mono> + listWithResponseAsync(Boolean includePathRecommendations, Boolean summary) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - apiVersion, - includePathRecommendations, - summary, - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + includePathRecommendations, summary, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of application control machine groups for the subscription. - * + * * @param includePathRecommendations Include the policy rules. * @param summary Return output in a summarized form. * @param context The context to associate with this operation. @@ -167,44 +139,33 @@ private Mono> listWithResponseAs * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of application control machine groups for the subscription along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - Boolean includePathRecommendations, Boolean summary, Context context) { + private Mono> + listWithResponseAsync(Boolean includePathRecommendations, Boolean summary, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - apiVersion, - includePathRecommendations, - summary, - accept, - context); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + includePathRecommendations, summary, accept, context); } /** * Gets a list of application control machine groups for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of application control machine groups for the subscription on successful completion of {@link - * Mono}. + * @return a list of application control machine groups for the subscription on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono listAsync() { @@ -216,7 +177,7 @@ private Mono listAsync() { /** * Gets a list of application control machine groups for the subscription. - * + * * @param includePathRecommendations Include the policy rules. * @param summary Return output in a summarized form. * @param context The context to associate with this operation. @@ -226,14 +187,14 @@ private Mono listAsync() { * @return a list of application control machine groups for the subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listWithResponse( - Boolean includePathRecommendations, Boolean summary, Context context) { + public Response listWithResponse(Boolean includePathRecommendations, + Boolean summary, Context context) { return listWithResponseAsync(includePathRecommendations, summary, context).block(); } /** * Gets a list of application control machine groups for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of application control machine groups for the subscription. @@ -247,30 +208,26 @@ public AdaptiveApplicationControlGroupsInner list() { /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an application control VM/server group along with {@link Response} on successful completion of {@link - * Mono}. + * @return an application control VM/server group along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String ascLocation, String groupName) { + private Mono> getWithResponseAsync(String ascLocation, + String groupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -281,47 +238,34 @@ private Mono> getWithResponseAsyn final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - groupName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, + groupName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an application control VM/server group along with {@link Response} on successful completion of {@link - * Mono}. + * @return an application control VM/server group along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String ascLocation, String groupName, Context context) { + private Mono> getWithResponseAsync(String ascLocation, + String groupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -332,22 +276,15 @@ private Mono> getWithResponseAsyn final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - groupName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, groupName, + apiVersion, accept, context); } /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -361,9 +298,9 @@ private Mono getAsync(String ascLocation, /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -372,16 +309,16 @@ private Mono getAsync(String ascLocation, * @return an application control VM/server group along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String ascLocation, String groupName, Context context) { + public Response getWithResponse(String ascLocation, String groupName, + Context context) { return getWithResponseAsync(ascLocation, groupName, context).block(); } /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -395,9 +332,9 @@ public AdaptiveApplicationControlGroupInner get(String ascLocation, String group /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -406,19 +343,15 @@ public AdaptiveApplicationControlGroupInner get(String ascLocation, String group * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> putWithResponseAsync( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body) { + private Mono> putWithResponseAsync(String ascLocation, + String groupName, AdaptiveApplicationControlGroupInner body) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -434,26 +367,16 @@ private Mono> putWithResponseAsyn final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .put( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - groupName, - apiVersion, - body, - accept, - context)) + .withContext(context -> service.put(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, + groupName, apiVersion, body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @param context The context to associate with this operation. @@ -463,19 +386,15 @@ private Mono> putWithResponseAsyn * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> putWithResponseAsync( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body, Context context) { + private Mono> putWithResponseAsync(String ascLocation, + String groupName, AdaptiveApplicationControlGroupInner body, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -491,23 +410,15 @@ private Mono> putWithResponseAsyn final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .put( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - groupName, - apiVersion, - body, - accept, - context); + return service.put(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, groupName, + apiVersion, body, accept, context); } /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -516,16 +427,16 @@ private Mono> putWithResponseAsyn * @return the response body on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono putAsync( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body) { + private Mono putAsync(String ascLocation, String groupName, + AdaptiveApplicationControlGroupInner body) { return putWithResponseAsync(ascLocation, groupName, body).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @param context The context to associate with this operation. @@ -535,16 +446,16 @@ private Mono putAsync( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response putWithResponse( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body, Context context) { + public Response putWithResponse(String ascLocation, String groupName, + AdaptiveApplicationControlGroupInner body, Context context) { return putWithResponseAsync(ascLocation, groupName, body, context).block(); } /** * Update an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -553,16 +464,16 @@ public Response putWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AdaptiveApplicationControlGroupInner put( - String ascLocation, String groupName, AdaptiveApplicationControlGroupInner body) { + public AdaptiveApplicationControlGroupInner put(String ascLocation, String groupName, + AdaptiveApplicationControlGroupInner body) { return putWithResponse(ascLocation, groupName, body, Context.NONE).getValue(); } /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -572,16 +483,12 @@ public AdaptiveApplicationControlGroupInner put( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String ascLocation, String groupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -592,25 +499,16 @@ private Mono> deleteWithResponseAsync(String ascLocation, String final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - groupName, - apiVersion, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, groupName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -621,16 +519,12 @@ private Mono> deleteWithResponseAsync(String ascLocation, String @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String ascLocation, String groupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -641,22 +535,15 @@ private Mono> deleteWithResponseAsync(String ascLocation, String final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - groupName, - apiVersion, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, groupName, + apiVersion, accept, context); } /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -670,9 +557,9 @@ private Mono deleteAsync(String ascLocation, String groupName) { /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -687,9 +574,9 @@ public Response deleteWithResponse(String ascLocation, String groupName, C /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsImpl.java index df7f0426fc39a..624149443b067 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveApplicationControlsImpl.java @@ -22,22 +22,18 @@ public final class AdaptiveApplicationControlsImpl implements AdaptiveApplicatio private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AdaptiveApplicationControlsImpl( - AdaptiveApplicationControlsClient innerClient, + public AdaptiveApplicationControlsImpl(AdaptiveApplicationControlsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response listWithResponse( - Boolean includePathRecommendations, Boolean summary, Context context) { - Response inner = - this.serviceClient().listWithResponse(includePathRecommendations, summary, context); + public Response listWithResponse(Boolean includePathRecommendations, + Boolean summary, Context context) { + Response inner + = this.serviceClient().listWithResponse(includePathRecommendations, summary, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AdaptiveApplicationControlGroupsImpl(inner.getValue(), this.manager())); } else { return null; @@ -53,15 +49,12 @@ public AdaptiveApplicationControlGroups list() { } } - public Response getWithResponse( - String ascLocation, String groupName, Context context) { - Response inner = - this.serviceClient().getWithResponse(ascLocation, groupName, context); + public Response getWithResponse(String ascLocation, String groupName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(ascLocation, groupName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AdaptiveApplicationControlGroupImpl(inner.getValue(), this.manager())); } else { return null; @@ -86,85 +79,57 @@ public void deleteByResourceGroup(String ascLocation, String groupName) { } public AdaptiveApplicationControlGroup getById(String id) { - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String groupName = Utils.getValueFromIdByName(id, "applicationWhitelistings"); + String groupName = ResourceManagerUtils.getValueFromIdByName(id, "applicationWhitelistings"); if (groupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", id))); } return this.getWithResponse(ascLocation, groupName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String groupName = Utils.getValueFromIdByName(id, "applicationWhitelistings"); + String groupName = ResourceManagerUtils.getValueFromIdByName(id, "applicationWhitelistings"); if (groupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", id))); } return this.getWithResponse(ascLocation, groupName, context); } public void deleteById(String id) { - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String groupName = Utils.getValueFromIdByName(id, "applicationWhitelistings"); + String groupName = ResourceManagerUtils.getValueFromIdByName(id, "applicationWhitelistings"); if (groupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", id))); } this.deleteByResourceGroupWithResponse(ascLocation, groupName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String groupName = Utils.getValueFromIdByName(id, "applicationWhitelistings"); + String groupName = ResourceManagerUtils.getValueFromIdByName(id, "applicationWhitelistings"); if (groupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'applicationWhitelistings'.", id))); } return this.deleteByResourceGroupWithResponse(ascLocation, groupName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningImpl.java index c07af0ebfddca..b2b9dd401a10d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningImpl.java @@ -17,8 +17,8 @@ public final class AdaptiveNetworkHardeningImpl implements AdaptiveNetworkHarden private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AdaptiveNetworkHardeningImpl( - AdaptiveNetworkHardeningInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + AdaptiveNetworkHardeningImpl(AdaptiveNetworkHardeningInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsClientImpl.java index 923dc53de886f..0379942ede44b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsClientImpl.java @@ -38,24 +38,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AdaptiveNetworkHardeningsClient. */ +/** + * An instance of this class provides access to all the operations defined in AdaptiveNetworkHardeningsClient. + */ public final class AdaptiveNetworkHardeningsClientImpl implements AdaptiveNetworkHardeningsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AdaptiveNetworkHardeningsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AdaptiveNetworkHardeningsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AdaptiveNetworkHardeningsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - AdaptiveNetworkHardeningsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AdaptiveNetworkHardeningsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -66,97 +70,77 @@ public final class AdaptiveNetworkHardeningsClientImpl implements AdaptiveNetwor @Host("{$host}") @ServiceInterface(name = "SecurityCenterAdapti") public interface AdaptiveNetworkHardeningsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/adaptiveNetworkHardenings") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/adaptiveNetworkHardenings") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByExtendedResource( - @HostParam("$host") String endpoint, + Mono> listByExtendedResource(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, - @PathParam("resourceName") String resourceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, + @PathParam("resourceName") String resourceName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/adaptiveNetworkHardenings/{adaptiveNetworkHardeningResourceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/adaptiveNetworkHardenings/{adaptiveNetworkHardeningResourceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, @PathParam("resourceName") String resourceName, @PathParam("adaptiveNetworkHardeningResourceName") String adaptiveNetworkHardeningResourceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/adaptiveNetworkHardenings/{adaptiveNetworkHardeningResourceName}/{adaptiveNetworkHardeningEnforceAction}") - @ExpectedResponses({200, 202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/adaptiveNetworkHardenings/{adaptiveNetworkHardeningResourceName}/{adaptiveNetworkHardeningEnforceAction}") + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> enforce( - @HostParam("$host") String endpoint, + Mono>> enforce(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, @PathParam("resourceName") String resourceName, @PathParam("adaptiveNetworkHardeningResourceName") String adaptiveNetworkHardeningResourceName, @PathParam("adaptiveNetworkHardeningEnforceAction") String adaptiveNetworkHardeningEnforceAction, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") AdaptiveNetworkHardeningEnforceRequest body, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByExtendedResourceNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Adaptive Network Hardenings resources in scope of an extended resource along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of Adaptive Network Hardenings resources in scope of an extended resource along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByExtendedResourceSinglePageAsync( String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -176,35 +160,18 @@ private Mono> listByExtendedResourc final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listByExtendedResource( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listByExtendedResource(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -212,23 +179,19 @@ private Mono> listByExtendedResourc * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Adaptive Network Hardenings resources in scope of an extended resource along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of Adaptive Network Hardenings resources in scope of an extended resource along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByExtendedResourceSinglePageAsync( String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -248,32 +211,17 @@ private Mono> listByExtendedResourc final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByExtendedResource( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - apiVersion, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByExtendedResource(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -281,22 +229,20 @@ private Mono> listByExtendedResourc * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByExtendedResourceAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - return new PagedFlux<>( - () -> - listByExtendedResourceSinglePageAsync(resourceGroupName, resourceNamespace, resourceType, resourceName), - nextLink -> listByExtendedResourceNextSinglePageAsync(nextLink)); + private PagedFlux listByExtendedResourceAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { + return new PagedFlux<>(() -> listByExtendedResourceSinglePageAsync(resourceGroupName, resourceNamespace, + resourceType, resourceName), nextLink -> listByExtendedResourceNextSinglePageAsync(nextLink)); } /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -305,23 +251,21 @@ private PagedFlux listByExtendedResourceAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByExtendedResourceAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - return new PagedFlux<>( - () -> - listByExtendedResourceSinglePageAsync( - resourceGroupName, resourceNamespace, resourceType, resourceName, context), + private PagedFlux listByExtendedResourceAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { + return new PagedFlux<>(() -> listByExtendedResourceSinglePageAsync(resourceGroupName, resourceNamespace, + resourceType, resourceName, context), nextLink -> listByExtendedResourceNextSinglePageAsync(nextLink, context)); } /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -329,20 +273,20 @@ private PagedFlux listByExtendedResourceAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + public PagedIterable listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { return new PagedIterable<>( listByExtendedResourceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName)); } /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -351,20 +295,20 @@ public PagedIterable listByExtendedResource( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + public PagedIterable listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { return new PagedIterable<>( listByExtendedResourceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context)); } /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -373,26 +317,19 @@ public PagedIterable listByExtendedResource( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single Adaptive Network Hardening resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, + private Mono> getWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -409,36 +346,23 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter resourceName is required and cannot be null.")); } if (adaptiveNetworkHardeningResourceName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, adaptiveNetworkHardeningResourceName, + apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -448,27 +372,19 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single Adaptive Network Hardening resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, + private Mono> getWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -485,33 +401,22 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter resourceName is required and cannot be null.")); } if (adaptiveNetworkHardeningResourceName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, adaptiveNetworkHardeningResourceName, apiVersion, accept, + context); } /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -522,22 +427,17 @@ private Mono> getWithResponseAsync( * @return a single Adaptive Network Hardening resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName) { - return getWithResponseAsync( - resourceGroupName, resourceNamespace, resourceType, resourceName, adaptiveNetworkHardeningResourceName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono getAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName) { + return getWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -549,28 +449,17 @@ private Mono getAsync( * @return a single Adaptive Network Hardening resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - Context context) { - return getWithResponseAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - context) - .block(); + public Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, Context context) { + return getWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, context).block(); } /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -581,27 +470,17 @@ public Response getWithResponse( * @return a single Adaptive Network Hardening resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AdaptiveNetworkHardeningInner get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName) { - return getWithResponse( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - Context.NONE) - .getValue(); + public AdaptiveNetworkHardeningInner get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String adaptiveNetworkHardeningResourceName) { + return getWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, Context.NONE).getValue(); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -613,24 +492,16 @@ public AdaptiveNetworkHardeningInner get( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> enforceWithResponseAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, + private Mono>> enforceWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -647,10 +518,8 @@ private Mono>> enforceWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter resourceName is required and cannot be null.")); } if (adaptiveNetworkHardeningResourceName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); } if (body == null) { return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); @@ -661,30 +530,17 @@ private Mono>> enforceWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .enforce( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - adaptiveNetworkHardeningEnforceAction, - apiVersion, - body, - accept, - context)) + .withContext(context -> service.enforce(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, adaptiveNetworkHardeningResourceName, + adaptiveNetworkHardeningEnforceAction, apiVersion, body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -697,25 +553,16 @@ private Mono>> enforceWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> enforceWithResponseAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context) { + private Mono>> enforceWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, + AdaptiveNetworkHardeningEnforceRequest body, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -732,10 +579,8 @@ private Mono>> enforceWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter resourceName is required and cannot be null.")); } if (adaptiveNetworkHardeningResourceName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter adaptiveNetworkHardeningResourceName is required and cannot be null.")); } if (body == null) { return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); @@ -746,27 +591,16 @@ private Mono>> enforceWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .enforce( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - adaptiveNetworkHardeningEnforceAction, - apiVersion, - body, - accept, - context); + return service.enforce(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, adaptiveNetworkHardeningResourceName, + adaptiveNetworkHardeningEnforceAction, apiVersion, body, accept, context); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -778,32 +612,20 @@ private Mono>> enforceWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginEnforceAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, + private PollerFlux, Void> beginEnforceAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body) { - Mono>> mono = - enforceWithResponseAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + Mono>> mono = enforceWithResponseAsync(resourceGroupName, resourceNamespace, + resourceType, resourceName, adaptiveNetworkHardeningResourceName, body); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -816,34 +638,21 @@ private PollerFlux, Void> beginEnforceAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginEnforceAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context) { + private PollerFlux, Void> beginEnforceAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, + AdaptiveNetworkHardeningEnforceRequest body, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - enforceWithResponseAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body, - context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono = enforceWithResponseAsync(resourceGroupName, resourceNamespace, + resourceType, resourceName, adaptiveNetworkHardeningResourceName, body, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -855,29 +664,18 @@ private PollerFlux, Void> beginEnforceAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginEnforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, + public SyncPoller, Void> beginEnforce(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body) { - return this - .beginEnforceAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body) - .getSyncPoller(); + return this.beginEnforceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body).getSyncPoller(); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -890,31 +688,18 @@ public SyncPoller, Void> beginEnforce( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginEnforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context) { - return this - .beginEnforceAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body, - context) - .getSyncPoller(); + public SyncPoller, Void> beginEnforce(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, + AdaptiveNetworkHardeningEnforceRequest body, Context context) { + return this.beginEnforceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body, context).getSyncPoller(); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -926,29 +711,17 @@ public SyncPoller, Void> beginEnforce( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono enforceAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body) { - return beginEnforceAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body) - .last() - .flatMap(this.client::getLroFinalResultOrError); + private Mono enforceAsync(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body) { + return beginEnforceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body).last().flatMap(this.client::getLroFinalResultOrError); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -961,31 +734,18 @@ private Mono enforceAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono enforceAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, + private Mono enforceAsync(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body, Context context) { - return beginEnforceAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body, - context) - .last() - .flatMap(this.client::getLroFinalResultOrError); + return beginEnforceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body, context).last().flatMap(this.client::getLroFinalResultOrError); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -996,28 +756,17 @@ private Mono enforceAsync( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body) { - enforceAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body) - .block(); + public void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body) { + enforceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body).block(); } /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -1029,100 +778,70 @@ public void enforce( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context) { - enforceAsync( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body, - context) - .block(); + public void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body, Context context) { + enforceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body, context).block(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for ListAdaptiveNetworkHardenings API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByExtendedResourceNextSinglePageAsync( - String nextLink) { + private Mono> + listByExtendedResourceNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByExtendedResourceNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return response for ListAdaptiveNetworkHardenings API service call along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByExtendedResourceNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listByExtendedResourceNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByExtendedResourceNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByExtendedResourceNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsImpl.java index afcb58db5f239..3d5efda8cf105 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdaptiveNetworkHardeningsImpl.java @@ -22,74 +22,42 @@ public final class AdaptiveNetworkHardeningsImpl implements AdaptiveNetworkHarde private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AdaptiveNetworkHardeningsImpl( - AdaptiveNetworkHardeningsClient innerClient, + public AdaptiveNetworkHardeningsImpl(AdaptiveNetworkHardeningsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - PagedIterable inner = - this - .serviceClient() - .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName); - return Utils.mapPage(inner, inner1 -> new AdaptiveNetworkHardeningImpl(inner1, this.manager())); + public PagedIterable listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { + PagedIterable inner = this.serviceClient() + .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AdaptiveNetworkHardeningImpl(inner1, this.manager())); } - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - PagedIterable inner = - this - .serviceClient() - .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName, context); - return Utils.mapPage(inner, inner1 -> new AdaptiveNetworkHardeningImpl(inner1, this.manager())); + public PagedIterable listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { + PagedIterable inner = this.serviceClient() + .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AdaptiveNetworkHardeningImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - context); + public Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + resourceNamespace, resourceType, resourceName, adaptiveNetworkHardeningResourceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AdaptiveNetworkHardeningImpl(inner.getValue(), this.manager())); } else { return null; } } - public AdaptiveNetworkHardening get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName) { - AdaptiveNetworkHardeningInner inner = - this - .serviceClient() - .get( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName); + public AdaptiveNetworkHardening get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String adaptiveNetworkHardeningResourceName) { + AdaptiveNetworkHardeningInner inner = this.serviceClient().get(resourceGroupName, resourceNamespace, + resourceType, resourceName, adaptiveNetworkHardeningResourceName); if (inner != null) { return new AdaptiveNetworkHardeningImpl(inner, this.manager()); } else { @@ -97,42 +65,16 @@ public AdaptiveNetworkHardening get( } } - public void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body) { - this - .serviceClient() - .enforce( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body); + public void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body) { + this.serviceClient().enforce(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body); } - public void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context) { - this - .serviceClient() - .enforce( - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - adaptiveNetworkHardeningResourceName, - body, - context); + public void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body, Context context) { + this.serviceClient().enforce(resourceGroupName, resourceNamespace, resourceType, resourceName, + adaptiveNetworkHardeningResourceName, body, context); } private AdaptiveNetworkHardeningsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionSettingImpl.java index 42c6609f6803d..320725dcb85b4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionSettingImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionSettingImpl.java @@ -14,8 +14,7 @@ public final class AdvancedThreatProtectionSettingImpl private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AdvancedThreatProtectionSettingImpl( - AdvancedThreatProtectionSettingInner innerObject, + AdvancedThreatProtectionSettingImpl(AdvancedThreatProtectionSettingInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -53,22 +52,14 @@ public AdvancedThreatProtectionSettingImpl withExistingResourceId(String resourc } public AdvancedThreatProtectionSetting create() { - this.innerObject = - serviceManager - .serviceClient() - .getAdvancedThreatProtections() - .createWithResponse(resourceId, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdvancedThreatProtections() + .createWithResponse(resourceId, this.innerModel(), Context.NONE).getValue(); return this; } public AdvancedThreatProtectionSetting create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAdvancedThreatProtections() - .createWithResponse(resourceId, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdvancedThreatProtections() + .createWithResponse(resourceId, this.innerModel(), context).getValue(); return this; } @@ -78,22 +69,14 @@ public AdvancedThreatProtectionSetting create(Context context) { } public AdvancedThreatProtectionSetting refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getAdvancedThreatProtections() - .getWithResponse(resourceId, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdvancedThreatProtections() + .getWithResponse(resourceId, Context.NONE).getValue(); return this; } public AdvancedThreatProtectionSetting refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAdvancedThreatProtections() - .getWithResponse(resourceId, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAdvancedThreatProtections() + .getWithResponse(resourceId, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsClientImpl.java index 1efa34e343b03..d9251b4a74748 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsClientImpl.java @@ -27,24 +27,28 @@ import com.azure.resourcemanager.security.fluent.models.AdvancedThreatProtectionSettingInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AdvancedThreatProtectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in AdvancedThreatProtectionsClient. + */ public final class AdvancedThreatProtectionsClientImpl implements AdvancedThreatProtectionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AdvancedThreatProtectionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AdvancedThreatProtectionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AdvancedThreatProtectionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - AdvancedThreatProtectionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AdvancedThreatProtectionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -55,49 +59,42 @@ public final class AdvancedThreatProtectionsClientImpl implements AdvancedThreat @Host("{$host}") @ServiceInterface(name = "SecurityCenterAdvanc") public interface AdvancedThreatProtectionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/advancedThreatProtectionSettings/{settingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("settingName") String settingName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("settingName") String settingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/{resourceId}/providers/Microsoft.Security/advancedThreatProtectionSettings/{settingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, + Mono> create(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, @PathParam("settingName") String settingName, @BodyParam("application/json") AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); } /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Advanced Threat Protection settings for the specified resource along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -113,23 +110,21 @@ private Mono> getWithResponseAsyn /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the Advanced Threat Protection settings for the specified resource along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, Context context) { + private Mono> getWithResponseAsync(String resourceId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -143,13 +138,13 @@ private Mono> getWithResponseAsyn /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Advanced Threat Protection settings for the specified resource on successful completion of {@link - * Mono}. + * @return the Advanced Threat Protection settings for the specified resource on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync(String resourceId) { @@ -158,7 +153,7 @@ private Mono getAsync(String resourceId) { /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -173,7 +168,7 @@ public Response getWithResponse(String res /** * Gets the Advanced Threat Protection settings for the specified resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -187,32 +182,28 @@ public AdvancedThreatProtectionSettingInner get(String resourceId) { /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Advanced Threat Protection resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return the Advanced Threat Protection resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting) { + private Mono> createWithResponseAsync(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (advancedThreatProtectionSetting == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter advancedThreatProtectionSetting is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter advancedThreatProtectionSetting is required and cannot be null.")); } else { advancedThreatProtectionSetting.validate(); } @@ -220,49 +211,36 @@ private Mono> createWithResponseA final String settingName = "current"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - resourceId, - settingName, - advancedThreatProtectionSetting, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), apiVersion, resourceId, settingName, + advancedThreatProtectionSetting, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the Advanced Threat Protection resource along with {@link Response} on successful completion of {@link - * Mono}. + * @return the Advanced Threat Protection resource along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, Context context) { + private Mono> createWithResponseAsync(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (advancedThreatProtectionSetting == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter advancedThreatProtectionSetting is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter advancedThreatProtectionSetting is required and cannot be null.")); } else { advancedThreatProtectionSetting.validate(); } @@ -270,20 +248,13 @@ private Mono> createWithResponseA final String settingName = "current"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - resourceId, - settingName, - advancedThreatProtectionSetting, - accept, - context); + return service.create(this.client.getEndpoint(), apiVersion, resourceId, settingName, + advancedThreatProtectionSetting, accept, context); } /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -292,15 +263,15 @@ private Mono> createWithResponseA * @return the Advanced Threat Protection resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting) { + private Mono createAsync(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting) { return createWithResponseAsync(resourceId, advancedThreatProtectionSetting) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @param context The context to associate with this operation. @@ -310,14 +281,14 @@ private Mono createAsync( * @return the Advanced Threat Protection resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, Context context) { + public Response createWithResponse(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting, Context context) { return createWithResponseAsync(resourceId, advancedThreatProtectionSetting, context).block(); } /** * Creates or updates the Advanced Threat Protection settings on a specified resource. - * + * * @param resourceId The identifier of the resource. * @param advancedThreatProtectionSetting Advanced Threat Protection Settings. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -326,8 +297,8 @@ public Response createWithResponse( * @return the Advanced Threat Protection resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AdvancedThreatProtectionSettingInner create( - String resourceId, AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting) { + public AdvancedThreatProtectionSettingInner create(String resourceId, + AdvancedThreatProtectionSettingInner advancedThreatProtectionSetting) { return createWithResponse(resourceId, advancedThreatProtectionSetting, Context.NONE).getValue(); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsImpl.java index 90cde665e7882..f16d95c1fbe21 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AdvancedThreatProtectionsImpl.java @@ -20,21 +20,17 @@ public final class AdvancedThreatProtectionsImpl implements AdvancedThreatProtec private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AdvancedThreatProtectionsImpl( - AdvancedThreatProtectionsClient innerClient, + public AdvancedThreatProtectionsImpl(AdvancedThreatProtectionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public Response getWithResponse(String resourceId, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceId, context); + Response inner + = this.serviceClient().getWithResponse(resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AdvancedThreatProtectionSettingImpl(inner.getValue(), this.manager())); } else { return null; @@ -51,33 +47,21 @@ public AdvancedThreatProtectionSetting get(String resourceId) { } public AdvancedThreatProtectionSetting getById(String id) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/advancedThreatProtectionSettings/{settingName}", - "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/advancedThreatProtectionSettings/{settingName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } return this.getWithResponse(resourceId, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/advancedThreatProtectionSettings/{settingName}", - "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/advancedThreatProtectionSettings/{settingName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } return this.getWithResponse(resourceId, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsClientImpl.java index f24350eb7d52c..3aabd1ad5c278 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsClientImpl.java @@ -38,17 +38,23 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AlertsClient. */ +/** + * An instance of this class provides access to all the operations defined in AlertsClient. + */ public final class AlertsClientImpl implements AlertsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AlertsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AlertsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AlertsClientImpl(SecurityCenterImpl client) { @@ -63,257 +69,175 @@ public final class AlertsClientImpl implements AlertsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterAlerts") public interface AlertsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/alerts") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/alerts") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listSubscriptionLevelByRegion( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @HeaderParam("Accept") String accept, - Context context); + Mono> listSubscriptionLevelByRegion(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listResourceGroupLevelByRegion( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, - Context context); + Mono> listResourceGroupLevelByRegion(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("resourceGroupName") String resourceGroupName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getSubscriptionLevel( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}") - @ExpectedResponses({200}) + Mono> getSubscriptionLevel(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("alertName") String alertName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getResourceGroupLevel( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/dismiss") - @ExpectedResponses({204}) + Mono> getResourceGroupLevel(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("alertName") String alertName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/dismiss") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateSubscriptionLevelStateToDismiss( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/resolve") - @ExpectedResponses({204}) + Mono> updateSubscriptionLevelStateToDismiss(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("alertName") String alertName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/resolve") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateSubscriptionLevelStateToResolve( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/activate") - @ExpectedResponses({204}) + Mono> updateSubscriptionLevelStateToResolve(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("alertName") String alertName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/activate") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateSubscriptionLevelStateToActivate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/inProgress") - @ExpectedResponses({204}) + Mono> updateSubscriptionLevelStateToActivate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("alertName") String alertName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/inProgress") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateSubscriptionLevelStateToInProgress( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/resolve") - @ExpectedResponses({204}) + Mono> updateSubscriptionLevelStateToInProgress(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("alertName") String alertName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/resolve") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateResourceGroupLevelStateToResolve( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/dismiss") - @ExpectedResponses({204}) + Mono> updateResourceGroupLevelStateToResolve(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("alertName") String alertName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/dismiss") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateResourceGroupLevelStateToDismiss( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/activate") - @ExpectedResponses({204}) + Mono> updateResourceGroupLevelStateToDismiss(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("alertName") String alertName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/activate") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateResourceGroupLevelStateToActivate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/inProgress") - @ExpectedResponses({204}) + Mono> updateResourceGroupLevelStateToActivate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("alertName") String alertName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/alerts/{alertName}/inProgress") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateResourceGroupLevelStateToInProgress( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("alertName") String alertName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/default/simulate") - @ExpectedResponses({202}) + Mono> updateResourceGroupLevelStateToInProgress(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("alertName") String alertName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/alerts/default/simulate") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> simulate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono>> simulate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, @BodyParam("application/json") AlertSimulatorRequestBody alertSimulatorRequestBody, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listSubscriptionLevelByRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listResourceGroupLevelByRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List all the alerts that are associated with the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -321,39 +245,26 @@ Mono> listResourceGroupLevelByRegionNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the alerts that are associated with the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -363,36 +274,24 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the alerts that are associated with the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security alerts as paginated response with {@link PagedFlux}. @@ -404,7 +303,7 @@ private PagedFlux listAsync() { /** * List all the alerts that are associated with the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -413,13 +312,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List all the alerts that are associated with the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security alerts as paginated response with {@link PagedIterable}. @@ -431,7 +330,7 @@ public PagedIterable list() { /** * List all the alerts that are associated with the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -445,9 +344,9 @@ public PagedIterable list(Context context) { /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -456,16 +355,12 @@ public PagedIterable list(Context context) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -474,33 +369,18 @@ private Mono> listByResourceGroupSinglePageAsync(Strin final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -508,19 +388,15 @@ private Mono> listByResourceGroupSinglePageAsync(Strin * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -530,29 +406,17 @@ private Mono> listByResourceGroupSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -560,16 +424,15 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -578,16 +441,15 @@ private PagedFlux listByResourceGroupAsync(String resourceGroupName) */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -600,9 +462,9 @@ public PagedIterable listByResourceGroup(String resourceGroupName) { /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -616,9 +478,9 @@ public PagedIterable listByResourceGroup(String resourceGroupName, C /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -627,16 +489,12 @@ public PagedIterable listByResourceGroup(String resourceGroupName, C @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSubscriptionLevelByRegionSinglePageAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -644,33 +502,18 @@ private Mono> listSubscriptionLevelByRegionSinglePageA final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listSubscriptionLevelByRegion( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listSubscriptionLevelByRegion(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -678,19 +521,15 @@ private Mono> listSubscriptionLevelByRegionSinglePageA * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSubscriptionLevelByRegionSinglePageAsync( - String ascLocation, Context context) { + private Mono> listSubscriptionLevelByRegionSinglePageAsync(String ascLocation, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -699,24 +538,17 @@ private Mono> listSubscriptionLevelByRegionSinglePageA final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listSubscriptionLevelByRegion( - this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), ascLocation, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listSubscriptionLevelByRegion(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + ascLocation, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -724,16 +556,15 @@ private Mono> listSubscriptionLevelByRegionSinglePageA */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listSubscriptionLevelByRegionAsync(String ascLocation) { - return new PagedFlux<>( - () -> listSubscriptionLevelByRegionSinglePageAsync(ascLocation), + return new PagedFlux<>(() -> listSubscriptionLevelByRegionSinglePageAsync(ascLocation), nextLink -> listSubscriptionLevelByRegionNextSinglePageAsync(nextLink)); } /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -742,16 +573,15 @@ private PagedFlux listSubscriptionLevelByRegionAsync(String ascLocat */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listSubscriptionLevelByRegionAsync(String ascLocation, Context context) { - return new PagedFlux<>( - () -> listSubscriptionLevelByRegionSinglePageAsync(ascLocation, context), + return new PagedFlux<>(() -> listSubscriptionLevelByRegionSinglePageAsync(ascLocation, context), nextLink -> listSubscriptionLevelByRegionNextSinglePageAsync(nextLink, context)); } /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -764,9 +594,9 @@ public PagedIterable listSubscriptionLevelByRegion(String ascLocatio /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -780,30 +610,26 @@ public PagedIterable listSubscriptionLevelByRegion(String ascLocatio /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listResourceGroupLevelByRegionSinglePageAsync( - String ascLocation, String resourceGroupName) { + private Mono> listResourceGroupLevelByRegionSinglePageAsync(String ascLocation, + String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -815,36 +641,20 @@ private Mono> listResourceGroupLevelByRegionSinglePage final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listResourceGroupLevelByRegion( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listResourceGroupLevelByRegion(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -852,19 +662,15 @@ private Mono> listResourceGroupLevelByRegionSinglePage * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listResourceGroupLevelByRegionSinglePageAsync( - String ascLocation, String resourceGroupName, Context context) { + private Mono> listResourceGroupLevelByRegionSinglePageAsync(String ascLocation, + String resourceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -877,32 +683,19 @@ private Mono> listResourceGroupLevelByRegionSinglePage final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listResourceGroupLevelByRegion( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listResourceGroupLevelByRegion(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + ascLocation, resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -910,18 +703,17 @@ private Mono> listResourceGroupLevelByRegionSinglePage */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listResourceGroupLevelByRegionAsync(String ascLocation, String resourceGroupName) { - return new PagedFlux<>( - () -> listResourceGroupLevelByRegionSinglePageAsync(ascLocation, resourceGroupName), + return new PagedFlux<>(() -> listResourceGroupLevelByRegionSinglePageAsync(ascLocation, resourceGroupName), nextLink -> listResourceGroupLevelByRegionNextSinglePageAsync(nextLink)); } /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -929,8 +721,8 @@ private PagedFlux listResourceGroupLevelByRegionAsync(String ascLoca * @return list of security alerts as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listResourceGroupLevelByRegionAsync( - String ascLocation, String resourceGroupName, Context context) { + private PagedFlux listResourceGroupLevelByRegionAsync(String ascLocation, String resourceGroupName, + Context context) { return new PagedFlux<>( () -> listResourceGroupLevelByRegionSinglePageAsync(ascLocation, resourceGroupName, context), nextLink -> listResourceGroupLevelByRegionNextSinglePageAsync(nextLink, context)); @@ -938,11 +730,11 @@ private PagedFlux listResourceGroupLevelByRegionAsync( /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -955,11 +747,11 @@ public PagedIterable listResourceGroupLevelByRegion(String ascLocati /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -967,36 +759,32 @@ public PagedIterable listResourceGroupLevelByRegion(String ascLocati * @return list of security alerts as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listResourceGroupLevelByRegion( - String ascLocation, String resourceGroupName, Context context) { + public PagedIterable listResourceGroupLevelByRegion(String ascLocation, String resourceGroupName, + Context context) { return new PagedIterable<>(listResourceGroupLevelByRegionAsync(ascLocation, resourceGroupName, context)); } /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an alert that is associated with a subscription along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getSubscriptionLevelWithResponseAsync(String ascLocation, String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1007,47 +795,34 @@ private Mono> getSubscriptionLevelWithResponseAsync(String final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getSubscriptionLevel( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context)) + .withContext(context -> service.getSubscriptionLevel(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an alert that is associated with a subscription along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getSubscriptionLevelWithResponseAsync( - String ascLocation, String alertName, Context context) { + private Mono> getSubscriptionLevelWithResponseAsync(String ascLocation, String alertName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1058,22 +833,15 @@ private Mono> getSubscriptionLevelWithResponseAsync( final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getSubscriptionLevel( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context); + return service.getSubscriptionLevel(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + ascLocation, alertName, accept, context); } /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1088,9 +856,9 @@ private Mono getSubscriptionLevelAsync(String ascLocation, String al /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1099,16 +867,16 @@ private Mono getSubscriptionLevelAsync(String ascLocation, String al * @return an alert that is associated with a subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getSubscriptionLevelWithResponse( - String ascLocation, String alertName, Context context) { + public Response getSubscriptionLevelWithResponse(String ascLocation, String alertName, + Context context) { return getSubscriptionLevelWithResponseAsync(ascLocation, alertName, context).block(); } /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1122,32 +890,28 @@ public AlertInner getSubscriptionLevel(String ascLocation, String alertName) { /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an alert that is associated a resource group or a resource in a resource group along with {@link - * Response} on successful completion of {@link Mono}. + * @return an alert that is associated a resource group or a resource in a resource group along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getResourceGroupLevelWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono> getResourceGroupLevelWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1162,50 +926,36 @@ private Mono> getResourceGroupLevelWithResponseAsync( final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getResourceGroupLevel( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context)) + .withContext(context -> service.getResourceGroupLevel(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an alert that is associated a resource group or a resource in a resource group along with {@link - * Response} on successful completion of {@link Mono}. + * @return an alert that is associated a resource group or a resource in a resource group along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getResourceGroupLevelWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName, Context context) { + private Mono> getResourceGroupLevelWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1220,67 +970,59 @@ private Mono> getResourceGroupLevelWithResponseAsync( final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getResourceGroupLevel( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context); + return service.getResourceGroupLevel(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, ascLocation, alertName, accept, context); } /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return an alert that is associated a resource group or a resource in a resource group on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getResourceGroupLevelAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono getResourceGroupLevelAsync(String resourceGroupName, String ascLocation, + String alertName) { return getResourceGroupLevelWithResponseAsync(resourceGroupName, ascLocation, alertName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an alert that is associated a resource group or a resource in a resource group along with {@link - * Response}. + * @return an alert that is associated a resource group or a resource in a resource group along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getResourceGroupLevelWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { + public Response getResourceGroupLevelWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context) { return getResourceGroupLevelWithResponseAsync(resourceGroupName, ascLocation, alertName, context).block(); } /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1294,9 +1036,9 @@ public AlertInner getResourceGroupLevel(String resourceGroupName, String ascLoca /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1304,19 +1046,15 @@ public AlertInner getResourceGroupLevel(String resourceGroupName, String ascLoca * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToDismissWithResponseAsync( - String ascLocation, String alertName) { + private Mono> updateSubscriptionLevelStateToDismissWithResponseAsync(String ascLocation, + String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1327,25 +1065,16 @@ private Mono> updateSubscriptionLevelStateToDismissWithResponseAs final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateSubscriptionLevelStateToDismiss( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context)) + .withContext(context -> service.updateSubscriptionLevelStateToDismiss(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1354,19 +1083,15 @@ private Mono> updateSubscriptionLevelStateToDismissWithResponseAs * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToDismissWithResponseAsync( - String ascLocation, String alertName, Context context) { + private Mono> updateSubscriptionLevelStateToDismissWithResponseAsync(String ascLocation, + String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1377,22 +1102,15 @@ private Mono> updateSubscriptionLevelStateToDismissWithResponseAs final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateSubscriptionLevelStateToDismiss( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context); + return service.updateSubscriptionLevelStateToDismiss(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1407,9 +1125,9 @@ private Mono updateSubscriptionLevelStateToDismissAsync(String ascLocation /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1418,16 +1136,16 @@ private Mono updateSubscriptionLevelStateToDismissAsync(String ascLocation * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateSubscriptionLevelStateToDismissWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToDismissWithResponse(String ascLocation, String alertName, + Context context) { return updateSubscriptionLevelStateToDismissWithResponseAsync(ascLocation, alertName, context).block(); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1440,9 +1158,9 @@ public void updateSubscriptionLevelStateToDismiss(String ascLocation, String ale /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1450,19 +1168,15 @@ public void updateSubscriptionLevelStateToDismiss(String ascLocation, String ale * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToResolveWithResponseAsync( - String ascLocation, String alertName) { + private Mono> updateSubscriptionLevelStateToResolveWithResponseAsync(String ascLocation, + String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1473,25 +1187,16 @@ private Mono> updateSubscriptionLevelStateToResolveWithResponseAs final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateSubscriptionLevelStateToResolve( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context)) + .withContext(context -> service.updateSubscriptionLevelStateToResolve(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1500,19 +1205,15 @@ private Mono> updateSubscriptionLevelStateToResolveWithResponseAs * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToResolveWithResponseAsync( - String ascLocation, String alertName, Context context) { + private Mono> updateSubscriptionLevelStateToResolveWithResponseAsync(String ascLocation, + String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1523,22 +1224,15 @@ private Mono> updateSubscriptionLevelStateToResolveWithResponseAs final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateSubscriptionLevelStateToResolve( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context); + return service.updateSubscriptionLevelStateToResolve(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1553,9 +1247,9 @@ private Mono updateSubscriptionLevelStateToResolveAsync(String ascLocation /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1564,16 +1258,16 @@ private Mono updateSubscriptionLevelStateToResolveAsync(String ascLocation * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateSubscriptionLevelStateToResolveWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToResolveWithResponse(String ascLocation, String alertName, + Context context) { return updateSubscriptionLevelStateToResolveWithResponseAsync(ascLocation, alertName, context).block(); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1586,9 +1280,9 @@ public void updateSubscriptionLevelStateToResolve(String ascLocation, String ale /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1596,19 +1290,15 @@ public void updateSubscriptionLevelStateToResolve(String ascLocation, String ale * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToActivateWithResponseAsync( - String ascLocation, String alertName) { + private Mono> updateSubscriptionLevelStateToActivateWithResponseAsync(String ascLocation, + String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1619,25 +1309,16 @@ private Mono> updateSubscriptionLevelStateToActivateWithResponseA final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateSubscriptionLevelStateToActivate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context)) + .withContext(context -> service.updateSubscriptionLevelStateToActivate(this.client.getEndpoint(), + apiVersion, this.client.getSubscriptionId(), ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1646,19 +1327,15 @@ private Mono> updateSubscriptionLevelStateToActivateWithResponseA * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToActivateWithResponseAsync( - String ascLocation, String alertName, Context context) { + private Mono> updateSubscriptionLevelStateToActivateWithResponseAsync(String ascLocation, + String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1669,22 +1346,15 @@ private Mono> updateSubscriptionLevelStateToActivateWithResponseA final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateSubscriptionLevelStateToActivate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context); + return service.updateSubscriptionLevelStateToActivate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1699,9 +1369,9 @@ private Mono updateSubscriptionLevelStateToActivateAsync(String ascLocatio /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1710,16 +1380,16 @@ private Mono updateSubscriptionLevelStateToActivateAsync(String ascLocatio * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateSubscriptionLevelStateToActivateWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToActivateWithResponse(String ascLocation, String alertName, + Context context) { return updateSubscriptionLevelStateToActivateWithResponseAsync(ascLocation, alertName, context).block(); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1732,9 +1402,9 @@ public void updateSubscriptionLevelStateToActivate(String ascLocation, String al /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1742,19 +1412,15 @@ public void updateSubscriptionLevelStateToActivate(String ascLocation, String al * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToInProgressWithResponseAsync( - String ascLocation, String alertName) { + private Mono> updateSubscriptionLevelStateToInProgressWithResponseAsync(String ascLocation, + String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1765,25 +1431,16 @@ private Mono> updateSubscriptionLevelStateToInProgressWithRespons final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateSubscriptionLevelStateToInProgress( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context)) + .withContext(context -> service.updateSubscriptionLevelStateToInProgress(this.client.getEndpoint(), + apiVersion, this.client.getSubscriptionId(), ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1792,19 +1449,15 @@ private Mono> updateSubscriptionLevelStateToInProgressWithRespons * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelStateToInProgressWithResponseAsync( - String ascLocation, String alertName, Context context) { + private Mono> updateSubscriptionLevelStateToInProgressWithResponseAsync(String ascLocation, + String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -1815,22 +1468,15 @@ private Mono> updateSubscriptionLevelStateToInProgressWithRespons final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateSubscriptionLevelStateToInProgress( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertName, - accept, - context); + return service.updateSubscriptionLevelStateToInProgress(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1845,9 +1491,9 @@ private Mono updateSubscriptionLevelStateToInProgressAsync(String ascLocat /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1856,16 +1502,16 @@ private Mono updateSubscriptionLevelStateToInProgressAsync(String ascLocat * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateSubscriptionLevelStateToInProgressWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToInProgressWithResponse(String ascLocation, String alertName, + Context context) { return updateSubscriptionLevelStateToInProgressWithResponseAsync(ascLocation, alertName, context).block(); } /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1878,11 +1524,11 @@ public void updateSubscriptionLevelStateToInProgress(String ascLocation, String /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1890,19 +1536,15 @@ public void updateSubscriptionLevelStateToInProgress(String ascLocation, String * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToResolveWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono> updateResourceGroupLevelStateToResolveWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1918,27 +1560,18 @@ private Mono> updateResourceGroupLevelStateToResolveWithResponseA final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .updateResourceGroupLevelStateToResolve( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context)) + context -> service.updateResourceGroupLevelStateToResolve(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1947,19 +1580,15 @@ private Mono> updateResourceGroupLevelStateToResolveWithResponseA * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToResolveWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName, Context context) { + private Mono> updateResourceGroupLevelStateToResolveWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1974,25 +1603,17 @@ private Mono> updateResourceGroupLevelStateToResolveWithResponseA final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateResourceGroupLevelStateToResolve( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context); + return service.updateResourceGroupLevelStateToResolve(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2000,19 +1621,19 @@ private Mono> updateResourceGroupLevelStateToResolveWithResponseA * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateResourceGroupLevelStateToResolveAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono updateResourceGroupLevelStateToResolveAsync(String resourceGroupName, String ascLocation, + String alertName) { return updateResourceGroupLevelStateToResolveWithResponseAsync(resourceGroupName, ascLocation, alertName) .flatMap(ignored -> Mono.empty()); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2021,20 +1642,19 @@ private Mono updateResourceGroupLevelStateToResolveAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateResourceGroupLevelStateToResolveWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return updateResourceGroupLevelStateToResolveWithResponseAsync( - resourceGroupName, ascLocation, alertName, context) - .block(); + public Response updateResourceGroupLevelStateToResolveWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return updateResourceGroupLevelStateToResolveWithResponseAsync(resourceGroupName, ascLocation, alertName, + context).block(); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2047,11 +1667,11 @@ public void updateResourceGroupLevelStateToResolve(String resourceGroupName, Str /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2059,19 +1679,15 @@ public void updateResourceGroupLevelStateToResolve(String resourceGroupName, Str * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToDismissWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono> updateResourceGroupLevelStateToDismissWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2087,27 +1703,18 @@ private Mono> updateResourceGroupLevelStateToDismissWithResponseA final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .updateResourceGroupLevelStateToDismiss( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context)) + context -> service.updateResourceGroupLevelStateToDismiss(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2116,19 +1723,15 @@ private Mono> updateResourceGroupLevelStateToDismissWithResponseA * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToDismissWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName, Context context) { + private Mono> updateResourceGroupLevelStateToDismissWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2143,25 +1746,17 @@ private Mono> updateResourceGroupLevelStateToDismissWithResponseA final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateResourceGroupLevelStateToDismiss( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context); + return service.updateResourceGroupLevelStateToDismiss(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2169,19 +1764,19 @@ private Mono> updateResourceGroupLevelStateToDismissWithResponseA * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateResourceGroupLevelStateToDismissAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono updateResourceGroupLevelStateToDismissAsync(String resourceGroupName, String ascLocation, + String alertName) { return updateResourceGroupLevelStateToDismissWithResponseAsync(resourceGroupName, ascLocation, alertName) .flatMap(ignored -> Mono.empty()); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2190,20 +1785,19 @@ private Mono updateResourceGroupLevelStateToDismissAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateResourceGroupLevelStateToDismissWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return updateResourceGroupLevelStateToDismissWithResponseAsync( - resourceGroupName, ascLocation, alertName, context) - .block(); + public Response updateResourceGroupLevelStateToDismissWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return updateResourceGroupLevelStateToDismissWithResponseAsync(resourceGroupName, ascLocation, alertName, + context).block(); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2216,11 +1810,11 @@ public void updateResourceGroupLevelStateToDismiss(String resourceGroupName, Str /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2228,19 +1822,15 @@ public void updateResourceGroupLevelStateToDismiss(String resourceGroupName, Str * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToActivateWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono> updateResourceGroupLevelStateToActivateWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2256,27 +1846,18 @@ private Mono> updateResourceGroupLevelStateToActivateWithResponse final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .updateResourceGroupLevelStateToActivate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context)) + context -> service.updateResourceGroupLevelStateToActivate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2285,19 +1866,15 @@ private Mono> updateResourceGroupLevelStateToActivateWithResponse * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToActivateWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName, Context context) { + private Mono> updateResourceGroupLevelStateToActivateWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2312,25 +1889,17 @@ private Mono> updateResourceGroupLevelStateToActivateWithResponse final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateResourceGroupLevelStateToActivate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context); + return service.updateResourceGroupLevelStateToActivate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2338,19 +1907,19 @@ private Mono> updateResourceGroupLevelStateToActivateWithResponse * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateResourceGroupLevelStateToActivateAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono updateResourceGroupLevelStateToActivateAsync(String resourceGroupName, String ascLocation, + String alertName) { return updateResourceGroupLevelStateToActivateWithResponseAsync(resourceGroupName, ascLocation, alertName) .flatMap(ignored -> Mono.empty()); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2359,38 +1928,37 @@ private Mono updateResourceGroupLevelStateToActivateAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateResourceGroupLevelStateToActivateWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return updateResourceGroupLevelStateToActivateWithResponseAsync( - resourceGroupName, ascLocation, alertName, context) - .block(); + public Response updateResourceGroupLevelStateToActivateWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return updateResourceGroupLevelStateToActivateWithResponseAsync(resourceGroupName, ascLocation, alertName, + context).block(); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void updateResourceGroupLevelStateToActivate( - String resourceGroupName, String ascLocation, String alertName) { + public void updateResourceGroupLevelStateToActivate(String resourceGroupName, String ascLocation, + String alertName) { updateResourceGroupLevelStateToActivateWithResponse(resourceGroupName, ascLocation, alertName, Context.NONE); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2398,19 +1966,15 @@ public void updateResourceGroupLevelStateToActivate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToInProgressWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono> updateResourceGroupLevelStateToInProgressWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2426,27 +1990,18 @@ private Mono> updateResourceGroupLevelStateToInProgressWithRespon final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .updateResourceGroupLevelStateToInProgress( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context)) + context -> service.updateResourceGroupLevelStateToInProgress(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2455,19 +2010,15 @@ private Mono> updateResourceGroupLevelStateToInProgressWithRespon * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelStateToInProgressWithResponseAsync( - String resourceGroupName, String ascLocation, String alertName, Context context) { + private Mono> updateResourceGroupLevelStateToInProgressWithResponseAsync(String resourceGroupName, + String ascLocation, String alertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -2482,25 +2033,17 @@ private Mono> updateResourceGroupLevelStateToInProgressWithRespon final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateResourceGroupLevelStateToInProgress( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - alertName, - accept, - context); + return service.updateResourceGroupLevelStateToInProgress(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, alertName, accept, context); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2508,19 +2051,19 @@ private Mono> updateResourceGroupLevelStateToInProgressWithRespon * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateResourceGroupLevelStateToInProgressAsync( - String resourceGroupName, String ascLocation, String alertName) { + private Mono updateResourceGroupLevelStateToInProgressAsync(String resourceGroupName, String ascLocation, + String alertName) { return updateResourceGroupLevelStateToInProgressWithResponseAsync(resourceGroupName, ascLocation, alertName) .flatMap(ignored -> Mono.empty()); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2529,36 +2072,35 @@ private Mono updateResourceGroupLevelStateToInProgressAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateResourceGroupLevelStateToInProgressWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return updateResourceGroupLevelStateToInProgressWithResponseAsync( - resourceGroupName, ascLocation, alertName, context) - .block(); + public Response updateResourceGroupLevelStateToInProgressWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return updateResourceGroupLevelStateToInProgressWithResponseAsync(resourceGroupName, ascLocation, alertName, + context).block(); } /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void updateResourceGroupLevelStateToInProgress( - String resourceGroupName, String ascLocation, String alertName) { + public void updateResourceGroupLevelStateToInProgress(String resourceGroupName, String ascLocation, + String alertName) { updateResourceGroupLevelStateToInProgressWithResponse(resourceGroupName, ascLocation, alertName, Context.NONE); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2566,53 +2108,38 @@ public void updateResourceGroupLevelStateToInProgress( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> simulateWithResponseAsync( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody) { + private Mono>> simulateWithResponseAsync(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (alertSimulatorRequestBody == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertSimulatorRequestBody is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertSimulatorRequestBody is required and cannot be null.")); } else { alertSimulatorRequestBody.validate(); } final String apiVersion = "2022-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .simulate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertSimulatorRequestBody, - accept, - context)) + .withContext(context -> service.simulate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, alertSimulatorRequestBody, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2621,50 +2148,37 @@ private Mono>> simulateWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> simulateWithResponseAsync( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { + private Mono>> simulateWithResponseAsync(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (alertSimulatorRequestBody == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertSimulatorRequestBody is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertSimulatorRequestBody is required and cannot be null.")); } else { alertSimulatorRequestBody.validate(); } final String apiVersion = "2022-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .simulate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - alertSimulatorRequestBody, - accept, - context); + return service.simulate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), ascLocation, + alertSimulatorRequestBody, accept, context); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2672,20 +2186,18 @@ private Mono>> simulateWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginSimulateAsync( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody) { + private PollerFlux, Void> beginSimulateAsync(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody) { Mono>> mono = simulateWithResponseAsync(ascLocation, alertSimulatorRequestBody); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2694,21 +2206,20 @@ private PollerFlux, Void> beginSimulateAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginSimulateAsync( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { + private PollerFlux, Void> beginSimulateAsync(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - simulateWithResponseAsync(ascLocation, alertSimulatorRequestBody, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = simulateWithResponseAsync(ascLocation, alertSimulatorRequestBody, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2716,16 +2227,16 @@ private PollerFlux, Void> beginSimulateAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginSimulate( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody) { + public SyncPoller, Void> beginSimulate(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody) { return this.beginSimulateAsync(ascLocation, alertSimulatorRequestBody).getSyncPoller(); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2734,16 +2245,16 @@ public SyncPoller, Void> beginSimulate( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginSimulate( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { + public SyncPoller, Void> beginSimulate(String ascLocation, + AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { return this.beginSimulateAsync(ascLocation, alertSimulatorRequestBody, context).getSyncPoller(); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2752,16 +2263,15 @@ public SyncPoller, Void> beginSimulate( */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono simulateAsync(String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody) { - return beginSimulateAsync(ascLocation, alertSimulatorRequestBody) - .last() + return beginSimulateAsync(ascLocation, alertSimulatorRequestBody).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2770,18 +2280,17 @@ private Mono simulateAsync(String ascLocation, AlertSimulatorRequestBody a * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono simulateAsync( - String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody, Context context) { - return beginSimulateAsync(ascLocation, alertSimulatorRequestBody, context) - .last() + private Mono simulateAsync(String ascLocation, AlertSimulatorRequestBody alertSimulatorRequestBody, + Context context) { + return beginSimulateAsync(ascLocation, alertSimulatorRequestBody, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2794,9 +2303,9 @@ public void simulate(String ascLocation, AlertSimulatorRequestBody alertSimulato /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -2810,9 +2319,10 @@ public void simulate(String ascLocation, AlertSimulatorRequestBody alertSimulato /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -2824,31 +2334,22 @@ private Mono> listNextSinglePageAsync(String nextLink) return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2861,31 +2362,22 @@ private Mono> listNextSinglePageAsync(String nextLink, return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -2897,32 +2389,24 @@ private Mono> listByResourceGroupNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -2935,31 +2419,22 @@ private Mono> listByResourceGroupNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -2971,33 +2446,23 @@ private Mono> listSubscriptionLevelByRegionNextSingleP return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listSubscriptionLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listSubscriptionLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -3005,37 +2470,28 @@ private Mono> listSubscriptionLevelByRegionNextSingleP * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSubscriptionLevelByRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listSubscriptionLevelByRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listSubscriptionLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listSubscriptionLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -3047,33 +2503,23 @@ private Mono> listResourceGroupLevelByRegionNextSingle return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listResourceGroupLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listResourceGroupLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -3081,29 +2527,19 @@ private Mono> listResourceGroupLevelByRegionNextSingle * @return list of security alerts along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listResourceGroupLevelByRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listResourceGroupLevelByRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listResourceGroupLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listResourceGroupLevelByRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsImpl.java index 2256ce9a09cd4..cc9acb48a6191 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsImpl.java @@ -29,55 +29,52 @@ public AlertsImpl(AlertsClient innerClient, com.azure.resourcemanager.security.S public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public PagedIterable listSubscriptionLevelByRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listSubscriptionLevelByRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public PagedIterable listSubscriptionLevelByRegion(String ascLocation, Context context) { PagedIterable inner = this.serviceClient().listSubscriptionLevelByRegion(ascLocation, context); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public PagedIterable listResourceGroupLevelByRegion(String ascLocation, String resourceGroupName) { - PagedIterable inner = - this.serviceClient().listResourceGroupLevelByRegion(ascLocation, resourceGroupName); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listResourceGroupLevelByRegion(ascLocation, resourceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } - public PagedIterable listResourceGroupLevelByRegion( - String ascLocation, String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listResourceGroupLevelByRegion(ascLocation, resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); + public PagedIterable listResourceGroupLevelByRegion(String ascLocation, String resourceGroupName, + Context context) { + PagedIterable inner + = this.serviceClient().listResourceGroupLevelByRegion(ascLocation, resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertImpl(inner1, this.manager())); } public Response getSubscriptionLevelWithResponse(String ascLocation, String alertName, Context context) { - Response inner = - this.serviceClient().getSubscriptionLevelWithResponse(ascLocation, alertName, context); + Response inner + = this.serviceClient().getSubscriptionLevelWithResponse(ascLocation, alertName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AlertImpl(inner.getValue(), this.manager())); } else { return null; @@ -93,15 +90,12 @@ public Alert getSubscriptionLevel(String ascLocation, String alertName) { } } - public Response getResourceGroupLevelWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - Response inner = - this.serviceClient().getResourceGroupLevelWithResponse(resourceGroupName, ascLocation, alertName, context); + public Response getResourceGroupLevelWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context) { + Response inner = this.serviceClient().getResourceGroupLevelWithResponse(resourceGroupName, + ascLocation, alertName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AlertImpl(inner.getValue(), this.manager())); } else { return null; @@ -117,8 +111,8 @@ public Alert getResourceGroupLevel(String resourceGroupName, String ascLocation, } } - public Response updateSubscriptionLevelStateToDismissWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToDismissWithResponse(String ascLocation, String alertName, + Context context) { return this.serviceClient().updateSubscriptionLevelStateToDismissWithResponse(ascLocation, alertName, context); } @@ -126,8 +120,8 @@ public void updateSubscriptionLevelStateToDismiss(String ascLocation, String ale this.serviceClient().updateSubscriptionLevelStateToDismiss(ascLocation, alertName); } - public Response updateSubscriptionLevelStateToResolveWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToResolveWithResponse(String ascLocation, String alertName, + Context context) { return this.serviceClient().updateSubscriptionLevelStateToResolveWithResponse(ascLocation, alertName, context); } @@ -135,8 +129,8 @@ public void updateSubscriptionLevelStateToResolve(String ascLocation, String ale this.serviceClient().updateSubscriptionLevelStateToResolve(ascLocation, alertName); } - public Response updateSubscriptionLevelStateToActivateWithResponse( - String ascLocation, String alertName, Context context) { + public Response updateSubscriptionLevelStateToActivateWithResponse(String ascLocation, String alertName, + Context context) { return this.serviceClient().updateSubscriptionLevelStateToActivateWithResponse(ascLocation, alertName, context); } @@ -144,60 +138,55 @@ public void updateSubscriptionLevelStateToActivate(String ascLocation, String al this.serviceClient().updateSubscriptionLevelStateToActivate(ascLocation, alertName); } - public Response updateSubscriptionLevelStateToInProgressWithResponse( - String ascLocation, String alertName, Context context) { - return this - .serviceClient() - .updateSubscriptionLevelStateToInProgressWithResponse(ascLocation, alertName, context); + public Response updateSubscriptionLevelStateToInProgressWithResponse(String ascLocation, String alertName, + Context context) { + return this.serviceClient().updateSubscriptionLevelStateToInProgressWithResponse(ascLocation, alertName, + context); } public void updateSubscriptionLevelStateToInProgress(String ascLocation, String alertName) { this.serviceClient().updateSubscriptionLevelStateToInProgress(ascLocation, alertName); } - public Response updateResourceGroupLevelStateToResolveWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return this - .serviceClient() - .updateResourceGroupLevelStateToResolveWithResponse(resourceGroupName, ascLocation, alertName, context); + public Response updateResourceGroupLevelStateToResolveWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return this.serviceClient().updateResourceGroupLevelStateToResolveWithResponse(resourceGroupName, ascLocation, + alertName, context); } public void updateResourceGroupLevelStateToResolve(String resourceGroupName, String ascLocation, String alertName) { this.serviceClient().updateResourceGroupLevelStateToResolve(resourceGroupName, ascLocation, alertName); } - public Response updateResourceGroupLevelStateToDismissWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return this - .serviceClient() - .updateResourceGroupLevelStateToDismissWithResponse(resourceGroupName, ascLocation, alertName, context); + public Response updateResourceGroupLevelStateToDismissWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return this.serviceClient().updateResourceGroupLevelStateToDismissWithResponse(resourceGroupName, ascLocation, + alertName, context); } public void updateResourceGroupLevelStateToDismiss(String resourceGroupName, String ascLocation, String alertName) { this.serviceClient().updateResourceGroupLevelStateToDismiss(resourceGroupName, ascLocation, alertName); } - public Response updateResourceGroupLevelStateToActivateWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return this - .serviceClient() - .updateResourceGroupLevelStateToActivateWithResponse(resourceGroupName, ascLocation, alertName, context); + public Response updateResourceGroupLevelStateToActivateWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return this.serviceClient().updateResourceGroupLevelStateToActivateWithResponse(resourceGroupName, ascLocation, + alertName, context); } - public void updateResourceGroupLevelStateToActivate( - String resourceGroupName, String ascLocation, String alertName) { + public void updateResourceGroupLevelStateToActivate(String resourceGroupName, String ascLocation, + String alertName) { this.serviceClient().updateResourceGroupLevelStateToActivate(resourceGroupName, ascLocation, alertName); } - public Response updateResourceGroupLevelStateToInProgressWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context) { - return this - .serviceClient() - .updateResourceGroupLevelStateToInProgressWithResponse(resourceGroupName, ascLocation, alertName, context); + public Response updateResourceGroupLevelStateToInProgressWithResponse(String resourceGroupName, + String ascLocation, String alertName, Context context) { + return this.serviceClient().updateResourceGroupLevelStateToInProgressWithResponse(resourceGroupName, + ascLocation, alertName, context); } - public void updateResourceGroupLevelStateToInProgress( - String resourceGroupName, String ascLocation, String alertName) { + public void updateResourceGroupLevelStateToInProgress(String resourceGroupName, String ascLocation, + String alertName) { this.serviceClient().updateResourceGroupLevelStateToInProgress(resourceGroupName, ascLocation, alertName); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRuleImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRuleImpl.java index dba39316deae7..5fbf105426f57 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRuleImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRuleImpl.java @@ -15,8 +15,8 @@ public final class AlertsSuppressionRuleImpl implements AlertsSuppressionRule { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AlertsSuppressionRuleImpl( - AlertsSuppressionRuleInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + AlertsSuppressionRuleImpl(AlertsSuppressionRuleInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesClientImpl.java index b6e2af3f8fd62..838c7900c7d51 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesClientImpl.java @@ -33,23 +33,28 @@ import com.azure.resourcemanager.security.models.AlertsSuppressionRulesList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AlertsSuppressionRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in AlertsSuppressionRulesClient. + */ public final class AlertsSuppressionRulesClientImpl implements AlertsSuppressionRulesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AlertsSuppressionRulesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AlertsSuppressionRulesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AlertsSuppressionRulesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(AlertsSuppressionRulesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AlertsSuppressionRulesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,162 +65,114 @@ public final class AlertsSuppressionRulesClientImpl implements AlertsSuppression @Host("{$host}") @ServiceInterface(name = "SecurityCenterAlerts") public interface AlertsSuppressionRulesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("AlertType") String alertType, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules/{alertsSuppressionRuleName}") - @ExpectedResponses({200}) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("AlertType") String alertType, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules/{alertsSuppressionRuleName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("alertsSuppressionRuleName") String alertsSuppressionRuleName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules/{alertsSuppressionRuleName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules/{alertsSuppressionRuleName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("alertsSuppressionRuleName") String alertsSuppressionRuleName, @BodyParam("application/json") AlertsSuppressionRuleInner alertsSuppressionRule, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules/{alertsSuppressionRuleName}") - @ExpectedResponses({204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/alertsSuppressionRules/{alertsSuppressionRuleName}") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("alertsSuppressionRuleName") String alertsSuppressionRuleName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String alertType) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertType, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + alertType, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String alertType, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), alertType, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -229,7 +186,7 @@ private PagedFlux listAsync(String alertType) { /** * List of all the dismiss rules for the given subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription as paginated response with {@link PagedFlux}. @@ -242,7 +199,7 @@ private PagedFlux listAsync() { /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -252,13 +209,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String alertType, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(alertType, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(alertType, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List of all the dismiss rules for the given subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription as paginated response with {@link PagedIterable}. @@ -271,7 +228,7 @@ public PagedIterable list() { /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -286,104 +243,78 @@ public PagedIterable list(String alertType, Context /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with {@link - * Response} on successful completion of {@link Mono}. + * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String alertsSuppressionRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (alertsSuppressionRuleName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertsSuppressionRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertsSuppressionRuleName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertsSuppressionRuleName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + alertsSuppressionRuleName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with {@link - * Response} on successful completion of {@link Mono}. + * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String alertsSuppressionRuleName, Context context) { + private Mono> getWithResponseAsync(String alertsSuppressionRuleName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (alertsSuppressionRuleName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertsSuppressionRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertsSuppressionRuleName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertsSuppressionRuleName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + alertsSuppressionRuleName, accept, context); } /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync(String alertsSuppressionRuleName) { @@ -392,14 +323,14 @@ private Mono getAsync(String alertsSuppressionRuleNa /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with {@link - * Response}. + * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response getWithResponse(String alertsSuppressionRuleName, Context context) { @@ -408,7 +339,7 @@ public Response getWithResponse(String alertsSuppres /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -422,7 +353,7 @@ public AlertsSuppressionRuleInner get(String alertsSuppressionRuleName) { /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -431,25 +362,19 @@ public AlertsSuppressionRuleInner get(String alertsSuppressionRuleName) { * @return describes the suppression rule along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule) { + private Mono> updateWithResponseAsync(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (alertsSuppressionRuleName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertsSuppressionRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertsSuppressionRuleName is required and cannot be null.")); } if (alertsSuppressionRule == null) { return Mono @@ -460,23 +385,14 @@ private Mono> updateWithResponseAsync( final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertsSuppressionRuleName, - alertsSuppressionRule, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), alertsSuppressionRuleName, alertsSuppressionRule, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @param context The context to associate with this operation. @@ -486,25 +402,19 @@ private Mono> updateWithResponseAsync( * @return describes the suppression rule along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule, Context context) { + private Mono> updateWithResponseAsync(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (alertsSuppressionRuleName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertsSuppressionRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertsSuppressionRuleName is required and cannot be null.")); } if (alertsSuppressionRule == null) { return Mono @@ -515,20 +425,13 @@ private Mono> updateWithResponseAsync( final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertsSuppressionRuleName, - alertsSuppressionRule, - accept, - context); + return service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + alertsSuppressionRuleName, alertsSuppressionRule, accept, context); } /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -537,15 +440,15 @@ private Mono> updateWithResponseAsync( * @return describes the suppression rule on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule) { + private Mono updateAsync(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule) { return updateWithResponseAsync(alertsSuppressionRuleName, alertsSuppressionRule) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @param context The context to associate with this operation. @@ -555,14 +458,14 @@ private Mono updateAsync( * @return describes the suppression rule along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule, Context context) { + public Response updateWithResponse(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule, Context context) { return updateWithResponseAsync(alertsSuppressionRuleName, alertsSuppressionRule, context).block(); } /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -571,14 +474,14 @@ public Response updateWithResponse( * @return describes the suppression rule. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AlertsSuppressionRuleInner update( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule) { + public AlertsSuppressionRuleInner update(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule) { return updateWithResponse(alertsSuppressionRuleName, alertsSuppressionRule, Context.NONE).getValue(); } /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -588,42 +491,28 @@ public AlertsSuppressionRuleInner update( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String alertsSuppressionRuleName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (alertsSuppressionRuleName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertsSuppressionRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertsSuppressionRuleName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertsSuppressionRuleName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), alertsSuppressionRuleName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -634,39 +523,27 @@ private Mono> deleteWithResponseAsync(String alertsSuppressionRul @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String alertsSuppressionRuleName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (alertsSuppressionRuleName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter alertsSuppressionRuleName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter alertsSuppressionRuleName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - alertsSuppressionRuleName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + alertsSuppressionRuleName, accept, context); } /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -680,7 +557,7 @@ private Mono deleteAsync(String alertsSuppressionRuleName) { /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -695,7 +572,7 @@ public Response deleteWithResponse(String alertsSuppressionRuleName, Conte /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -708,14 +585,15 @@ public void delete(String alertsSuppressionRuleName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -723,37 +601,28 @@ private Mono> listNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -761,23 +630,13 @@ private Mono> listNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesImpl.java index fe83aead0087d..353fe376e1f6a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AlertsSuppressionRulesImpl.java @@ -21,30 +21,27 @@ public final class AlertsSuppressionRulesImpl implements AlertsSuppressionRules private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AlertsSuppressionRulesImpl( - AlertsSuppressionRulesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public AlertsSuppressionRulesImpl(AlertsSuppressionRulesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new AlertsSuppressionRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertsSuppressionRuleImpl(inner1, this.manager())); } public PagedIterable list(String alertType, Context context) { PagedIterable inner = this.serviceClient().list(alertType, context); - return Utils.mapPage(inner, inner1 -> new AlertsSuppressionRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AlertsSuppressionRuleImpl(inner1, this.manager())); } public Response getWithResponse(String alertsSuppressionRuleName, Context context) { - Response inner = - this.serviceClient().getWithResponse(alertsSuppressionRuleName, context); + Response inner + = this.serviceClient().getWithResponse(alertsSuppressionRuleName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AlertsSuppressionRuleImpl(inner.getValue(), this.manager())); } else { return null; @@ -60,25 +57,22 @@ public AlertsSuppressionRule get(String alertsSuppressionRuleName) { } } - public Response updateWithResponse( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule, Context context) { - Response inner = - this.serviceClient().updateWithResponse(alertsSuppressionRuleName, alertsSuppressionRule, context); + public Response updateWithResponse(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule, Context context) { + Response inner + = this.serviceClient().updateWithResponse(alertsSuppressionRuleName, alertsSuppressionRule, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AlertsSuppressionRuleImpl(inner.getValue(), this.manager())); } else { return null; } } - public AlertsSuppressionRule update( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule) { - AlertsSuppressionRuleInner inner = - this.serviceClient().update(alertsSuppressionRuleName, alertsSuppressionRule); + public AlertsSuppressionRule update(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule) { + AlertsSuppressionRuleInner inner + = this.serviceClient().update(alertsSuppressionRuleName, alertsSuppressionRule); if (inner != null) { return new AlertsSuppressionRuleImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsClientImpl.java index 6ec6858c8cae4..b0d4918e82294 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsClientImpl.java @@ -31,22 +31,28 @@ import com.azure.resourcemanager.security.models.ConnectionType; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AllowedConnectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in AllowedConnectionsClient. + */ public final class AllowedConnectionsClientImpl implements AllowedConnectionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AllowedConnectionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AllowedConnectionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AllowedConnectionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(AllowedConnectionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AllowedConnectionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -57,153 +63,111 @@ public final class AllowedConnectionsClientImpl implements AllowedConnectionsCli @Host("{$host}") @ServiceInterface(name = "SecurityCenterAllowe") public interface AllowedConnectionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/allowedConnections") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/allowedConnections") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByHomeRegion( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByHomeRegion(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/allowedConnections/{connectionType}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/allowedConnections/{connectionType}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("connectionType") ConnectionType connectionType, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("connectionType") ConnectionType connectionType, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByHomeRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedFlux}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -212,27 +176,27 @@ private PagedFlux listAsync() { /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedFlux}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedIterable}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -241,13 +205,13 @@ public PagedIterable list() { /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedIterable}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -256,28 +220,24 @@ public PagedIterable list(Context context) { /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription and location along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return the list of all possible traffic between resources for the subscription and location along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionSinglePageAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -285,54 +245,35 @@ private Mono> listByHomeRegionSin final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByHomeRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription and location along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return the list of all possible traffic between resources for the subscription and location along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionSinglePageAsync( - String ascLocation, Context context) { + private Mono> listByHomeRegionSinglePageAsync(String ascLocation, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -341,66 +282,57 @@ private Mono> listByHomeRegionSin final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByHomeRegion( - this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedFlux}. + * response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink)); } /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedFlux}. + * response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation, Context context) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, context), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, context), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink, context)); } /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedIterable}. + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation) { @@ -409,15 +341,15 @@ public PagedIterable listByHomeRegion(String as /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedIterable}. + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation, Context context) { @@ -427,32 +359,28 @@ public PagedIterable listByHomeRegion(String as /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type along with {@link Response} on successful completion of {@link Mono}. + * type along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, ConnectionType connectionType) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, ConnectionType connectionType) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -467,51 +395,37 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - connectionType, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, connectionType, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type along with {@link Response} on successful completion of {@link Mono}. + * type along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, ConnectionType connectionType, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, ConnectionType connectionType, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -526,36 +440,28 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - connectionType, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, ascLocation, + connectionType, apiVersion, accept, context); } /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type on successful completion of {@link Mono}. + * type on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String ascLocation, ConnectionType connectionType) { + private Mono getAsync(String resourceGroupName, String ascLocation, + ConnectionType connectionType) { return getWithResponseAsync(resourceGroupName, ascLocation, connectionType) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -563,56 +469,57 @@ private Mono getAsync( /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type along with {@link Response}. + * type along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String ascLocation, ConnectionType connectionType, Context context) { + public Response getWithResponse(String resourceGroupName, String ascLocation, + ConnectionType connectionType, Context context) { return getWithResponseAsync(resourceGroupName, ascLocation, connectionType, context).block(); } /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type. + * type. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AllowedConnectionsResourceInner get( - String resourceGroupName, String ascLocation, ConnectionType connectionType) { + public AllowedConnectionsResourceInner get(String resourceGroupName, String ascLocation, + ConnectionType connectionType) { return getWithResponse(resourceGroupName, ascLocation, connectionType, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all possible traffic between Azure resources along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -620,75 +527,57 @@ private Mono> listNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all possible traffic between Azure resources along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all possible traffic between Azure resources along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionNextSinglePageAsync(String nextLink) { @@ -696,62 +585,44 @@ private Mono> listByHomeRegionNex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all possible traffic between Azure resources along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByHomeRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsImpl.java index 66d36cb8fb158..783c8fb12d198 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsImpl.java @@ -22,42 +22,43 @@ public final class AllowedConnectionsImpl implements AllowedConnections { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AllowedConnectionsImpl( - AllowedConnectionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public AllowedConnectionsImpl(AllowedConnectionsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation, Context context) { - PagedIterable inner = - this.serviceClient().listByHomeRegion(ascLocation, context); - return Utils.mapPage(inner, inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByHomeRegion(ascLocation, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new AllowedConnectionsResourceImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String ascLocation, ConnectionType connectionType, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, ascLocation, connectionType, context); + public Response getWithResponse(String resourceGroupName, String ascLocation, + ConnectionType connectionType, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, ascLocation, connectionType, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AllowedConnectionsResourceImpl(inner.getValue(), this.manager())); } else { return null; @@ -65,8 +66,8 @@ public Response getWithResponse( } public AllowedConnectionsResource get(String resourceGroupName, String ascLocation, ConnectionType connectionType) { - AllowedConnectionsResourceInner inner = - this.serviceClient().get(resourceGroupName, ascLocation, connectionType); + AllowedConnectionsResourceInner inner + = this.serviceClient().get(resourceGroupName, ascLocation, connectionType); if (inner != null) { return new AllowedConnectionsResourceImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsResourceImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsResourceImpl.java index f53610805f1b2..de4ec6bfedfdb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsResourceImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AllowedConnectionsResourceImpl.java @@ -16,8 +16,7 @@ public final class AllowedConnectionsResourceImpl implements AllowedConnectionsR private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AllowedConnectionsResourceImpl( - AllowedConnectionsResourceInner innerObject, + AllowedConnectionsResourceImpl(AllowedConnectionsResourceInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionImpl.java new file mode 100644 index 0000000000000..658835896f614 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionImpl.java @@ -0,0 +1,81 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.resourcemanager.security.fluent.models.ApiCollectionInner; +import com.azure.resourcemanager.security.models.ApiCollection; +import com.azure.resourcemanager.security.models.ProvisioningState; + +public final class ApiCollectionImpl implements ApiCollection { + private ApiCollectionInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + ApiCollectionImpl(ApiCollectionInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public ProvisioningState provisioningState() { + return this.innerModel().provisioningState(); + } + + public String displayName() { + return this.innerModel().displayName(); + } + + public String discoveredVia() { + return this.innerModel().discoveredVia(); + } + + public String baseUrl() { + return this.innerModel().baseUrl(); + } + + public Long numberOfApiEndpoints() { + return this.innerModel().numberOfApiEndpoints(); + } + + public Long numberOfInactiveApiEndpoints() { + return this.innerModel().numberOfInactiveApiEndpoints(); + } + + public Long numberOfUnauthenticatedApiEndpoints() { + return this.innerModel().numberOfUnauthenticatedApiEndpoints(); + } + + public Long numberOfExternalApiEndpoints() { + return this.innerModel().numberOfExternalApiEndpoints(); + } + + public Long numberOfApiEndpointsWithSensitiveDataExposed() { + return this.innerModel().numberOfApiEndpointsWithSensitiveDataExposed(); + } + + public String sensitivityLabel() { + return this.innerModel().sensitivityLabel(); + } + + public ApiCollectionInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsClientImpl.java deleted file mode 100644 index ca575de7d8a71..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsClientImpl.java +++ /dev/null @@ -1,248 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.annotation.Delete; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.security.fluent.ApiCollectionOffboardingsClient; -import reactor.core.publisher.Mono; - -/** An instance of this class provides access to all the operations defined in ApiCollectionOffboardingsClient. */ -public final class ApiCollectionOffboardingsClientImpl implements ApiCollectionOffboardingsClient { - /** The proxy service used to perform REST calls. */ - private final ApiCollectionOffboardingsService service; - - /** The service client containing this operation class. */ - private final SecurityCenterImpl client; - - /** - * Initializes an instance of ApiCollectionOffboardingsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - ApiCollectionOffboardingsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - ApiCollectionOffboardingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for SecurityCenterApiCollectionOffboardings to be used by the proxy - * service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "SecurityCenterApiCol") - public interface ApiCollectionOffboardingsService { - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections/{apiCollectionId}") - @ExpectedResponses({200, 204}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("serviceName") String serviceName, - @PathParam("apiCollectionId") String apiCollectionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - } - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String serviceName, String apiCollectionId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (serviceName == null) { - return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); - } - if (apiCollectionId == null) { - return Mono - .error(new IllegalArgumentException("Parameter apiCollectionId is required and cannot be null.")); - } - final String apiVersion = "2022-11-20-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiCollectionId, - apiVersion, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (serviceName == null) { - return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); - } - if (apiCollectionId == null) { - return Mono - .error(new IllegalArgumentException("Parameter apiCollectionId is required and cannot be null.")); - } - final String apiVersion = "2022-11-20-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiCollectionId, - apiVersion, - accept, - context); - } - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String resourceGroupName, String serviceName, String apiCollectionId) { - return deleteWithResponseAsync(resourceGroupName, serviceName, apiCollectionId) - .flatMap(ignored -> Mono.empty()); - } - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - return deleteWithResponseAsync(resourceGroupName, serviceName, apiCollectionId, context).block(); - } - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String resourceGroupName, String serviceName, String apiCollectionId) { - deleteWithResponse(resourceGroupName, serviceName, apiCollectionId, Context.NONE); - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsImpl.java deleted file mode 100644 index 8e6f9091324b7..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOffboardingsImpl.java +++ /dev/null @@ -1,43 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.security.fluent.ApiCollectionOffboardingsClient; -import com.azure.resourcemanager.security.models.ApiCollectionOffboardings; - -public final class ApiCollectionOffboardingsImpl implements ApiCollectionOffboardings { - private static final ClientLogger LOGGER = new ClientLogger(ApiCollectionOffboardingsImpl.class); - - private final ApiCollectionOffboardingsClient innerClient; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - public ApiCollectionOffboardingsImpl( - ApiCollectionOffboardingsClient innerClient, - com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public Response deleteWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - return this.serviceClient().deleteWithResponse(resourceGroupName, serviceName, apiCollectionId, context); - } - - public void delete(String resourceGroupName, String serviceName, String apiCollectionId) { - this.serviceClient().delete(resourceGroupName, serviceName, apiCollectionId); - } - - private ApiCollectionOffboardingsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsClientImpl.java deleted file mode 100644 index f2eab89967aba..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsClientImpl.java +++ /dev/null @@ -1,252 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Put; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.security.fluent.ApiCollectionOnboardingsClient; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; -import reactor.core.publisher.Mono; - -/** An instance of this class provides access to all the operations defined in ApiCollectionOnboardingsClient. */ -public final class ApiCollectionOnboardingsClientImpl implements ApiCollectionOnboardingsClient { - /** The proxy service used to perform REST calls. */ - private final ApiCollectionOnboardingsService service; - - /** The service client containing this operation class. */ - private final SecurityCenterImpl client; - - /** - * Initializes an instance of ApiCollectionOnboardingsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - ApiCollectionOnboardingsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(ApiCollectionOnboardingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for SecurityCenterApiCollectionOnboardings to be used by the proxy - * service to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "SecurityCenterApiCol") - public interface ApiCollectionOnboardingsService { - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections/{apiCollectionId}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("serviceName") String serviceName, - @PathParam("apiCollectionId") String apiCollectionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - } - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs along with {@link Response} on successful - * completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, String serviceName, String apiCollectionId) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (serviceName == null) { - return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); - } - if (apiCollectionId == null) { - return Mono - .error(new IllegalArgumentException("Parameter apiCollectionId is required and cannot be null.")); - } - final String apiVersion = "2022-11-20-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiCollectionId, - apiVersion, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs along with {@link Response} on successful - * completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (resourceGroupName == null) { - return Mono - .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); - } - if (serviceName == null) { - return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); - } - if (apiCollectionId == null) { - return Mono - .error(new IllegalArgumentException("Parameter apiCollectionId is required and cannot be null.")); - } - final String apiVersion = "2022-11-20-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiCollectionId, - apiVersion, - accept, - context); - } - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, String serviceName, String apiCollectionId) { - return createWithResponseAsync(resourceGroupName, serviceName, apiCollectionId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - return createWithResponseAsync(resourceGroupName, serviceName, apiCollectionId, context).block(); - } - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public ApiCollectionResponseInner create(String resourceGroupName, String serviceName, String apiCollectionId) { - return createWithResponse(resourceGroupName, serviceName, apiCollectionId, Context.NONE).getValue(); - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsImpl.java deleted file mode 100644 index 0fc34f1654954..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionOnboardingsImpl.java +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.security.fluent.ApiCollectionOnboardingsClient; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; -import com.azure.resourcemanager.security.models.ApiCollectionOnboardings; -import com.azure.resourcemanager.security.models.ApiCollectionResponse; - -public final class ApiCollectionOnboardingsImpl implements ApiCollectionOnboardings { - private static final ClientLogger LOGGER = new ClientLogger(ApiCollectionOnboardingsImpl.class); - - private final ApiCollectionOnboardingsClient innerClient; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - public ApiCollectionOnboardingsImpl( - ApiCollectionOnboardingsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public Response createWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - Response inner = - this.serviceClient().createWithResponse(resourceGroupName, serviceName, apiCollectionId, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ApiCollectionResponseImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public ApiCollectionResponse create(String resourceGroupName, String serviceName, String apiCollectionId) { - ApiCollectionResponseInner inner = this.serviceClient().create(resourceGroupName, serviceName, apiCollectionId); - if (inner != null) { - return new ApiCollectionResponseImpl(inner, this.manager()); - } else { - return null; - } - } - - private ApiCollectionOnboardingsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionResponseImpl.java deleted file mode 100644 index 1427fdd1513f9..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionResponseImpl.java +++ /dev/null @@ -1,55 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; -import com.azure.resourcemanager.security.models.ApiCollectionResponse; -import java.util.Collections; -import java.util.Map; - -public final class ApiCollectionResponseImpl implements ApiCollectionResponse { - private ApiCollectionResponseInner innerObject; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - ApiCollectionResponseImpl( - ApiCollectionResponseInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public String displayName() { - return this.innerModel().displayName(); - } - - public Map additionalData() { - Map inner = this.innerModel().additionalData(); - if (inner != null) { - return Collections.unmodifiableMap(inner); - } else { - return Collections.emptyMap(); - } - } - - public ApiCollectionResponseInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsClientImpl.java index 08310c38ed47a..32347f459c68d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsClientImpl.java @@ -4,6 +4,7 @@ package com.azure.resourcemanager.security.implementation; +import com.azure.core.annotation.Delete; import com.azure.core.annotation.ExpectedResponses; import com.azure.core.annotation.Get; import com.azure.core.annotation.HeaderParam; @@ -11,6 +12,7 @@ import com.azure.core.annotation.Host; import com.azure.core.annotation.HostParam; import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; import com.azure.core.annotation.QueryParam; import com.azure.core.annotation.ReturnType; import com.azure.core.annotation.ServiceInterface; @@ -23,29 +25,40 @@ import com.azure.core.http.rest.Response; import com.azure.core.http.rest.RestProxy; import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; import com.azure.core.util.Context; import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; import com.azure.resourcemanager.security.fluent.ApiCollectionsClient; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; -import com.azure.resourcemanager.security.models.ApiCollectionResponseList; +import com.azure.resourcemanager.security.fluent.models.ApiCollectionInner; +import com.azure.resourcemanager.security.models.ApiCollectionList; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ApiCollectionsClient. */ +/** + * An instance of this class provides access to all the operations defined in ApiCollectionsClient. + */ public final class ApiCollectionsClientImpl implements ApiCollectionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ApiCollectionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ApiCollectionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ApiCollectionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(ApiCollectionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ApiCollectionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,75 +69,384 @@ public final class ApiCollectionsClientImpl implements ApiCollectionsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterApiCol") public interface ApiCollectionsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/apiCollections") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/apiCollections") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("serviceName") String serviceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections/{apiCollectionId}") - @ExpectedResponses({200}) + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> listByAzureApiManagementService(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("serviceName") String serviceName, - @PathParam("apiCollectionId") String apiCollectionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("serviceName") String serviceName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections/{apiId}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getByAzureApiManagementService(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("serviceName") String serviceName, + @PathParam("apiId") String apiId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections/{apiId}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> onboardAzureApiManagementApi(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("serviceName") String serviceName, + @PathParam("apiId") String apiId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.ApiManagement/service/{serviceName}/providers/Microsoft.Security/apiCollections/{apiId}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> offboardAzureApiManagementApi(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("serviceName") String serviceName, + @PathParam("apiId") String apiId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listBySubscriptionNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByResourceGroupNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByAzureApiManagementServiceNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync() { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync() { + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + } + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list() { + return new PagedIterable<>(listAsync()); + } + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(Context context) { + return new PagedIterable<>(listAsync(context)); + } + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String serviceName) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * along with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), + nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); + } + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); + } + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByResourceGroup(String resourceGroupName) { + return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName)); + } + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { + return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, context)); + } + + /** + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + listByAzureApiManagementServiceSinglePageAsync(String resourceGroupName, String serviceName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -133,62 +455,42 @@ private Mono> listSinglePageAsync( if (serviceName == null) { return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } - final String apiVersion = "2022-11-20-preview"; + final String apiVersion = "2023-11-15"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByAzureApiManagementService(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, serviceName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String serviceName, Context context) { + private Mono> + listByAzureApiManagementServiceSinglePageAsync(String resourceGroupName, String serviceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -197,147 +499,179 @@ private Mono> listSinglePageAsync( if (serviceName == null) { return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } - final String apiVersion = "2022-11-20-preview"; + final String apiVersion = "2023-11-15"; final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiVersion, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + .listByAzureApiManagementService(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, serviceName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedFlux}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(String resourceGroupName, String serviceName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, serviceName), nextLink -> listNextSinglePageAsync(nextLink)); + private PagedFlux listByAzureApiManagementServiceAsync(String resourceGroupName, + String serviceName) { + return new PagedFlux<>(() -> listByAzureApiManagementServiceSinglePageAsync(resourceGroupName, serviceName), + nextLink -> listByAzureApiManagementServiceNextSinglePageAsync(nextLink)); } /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedFlux}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String serviceName, Context context) { + private PagedFlux listByAzureApiManagementServiceAsync(String resourceGroupName, + String serviceName, Context context) { return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, serviceName, context), - nextLink -> listNextSinglePageAsync(nextLink, context)); + () -> listByAzureApiManagementServiceSinglePageAsync(resourceGroupName, serviceName, context), + nextLink -> listByAzureApiManagementServiceNextSinglePageAsync(nextLink, context)); } /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedIterable}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(String resourceGroupName, String serviceName) { - return new PagedIterable<>(listAsync(resourceGroupName, serviceName)); + public PagedIterable listByAzureApiManagementService(String resourceGroupName, + String serviceName) { + return new PagedIterable<>(listByAzureApiManagementServiceAsync(resourceGroupName, serviceName)); } /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedIterable}. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String serviceName, Context context) { - return new PagedIterable<>(listAsync(resourceGroupName, serviceName, context)); + public PagedIterable listByAzureApiManagementService(String resourceGroupName, + String serviceName, Context context) { + return new PagedIterable<>(listByAzureApiManagementServiceAsync(resourceGroupName, serviceName, context)); } /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs along with {@link Response} on - * successful completion of {@link Mono}. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String serviceName, String apiCollectionId) { + private Mono> getByAzureApiManagementServiceWithResponseAsync(String resourceGroupName, + String serviceName, String apiId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (serviceName == null) { + return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); + } + if (apiId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiId is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.getByAzureApiManagementService(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, serviceName, apiId, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs along with + * {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getByAzureApiManagementServiceWithResponseAsync(String resourceGroupName, + String serviceName, String apiId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -346,60 +680,393 @@ private Mono> getWithResponseAsync( if (serviceName == null) { return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } - if (apiCollectionId == null) { + if (apiId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiId is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getByAzureApiManagementService(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, serviceName, apiId, apiVersion, accept, context); + } + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getByAzureApiManagementServiceAsync(String resourceGroupName, String serviceName, + String apiId) { + return getByAzureApiManagementServiceWithResponseAsync(resourceGroupName, serviceName, apiId) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs along with + * {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getByAzureApiManagementServiceWithResponse(String resourceGroupName, + String serviceName, String apiId, Context context) { + return getByAzureApiManagementServiceWithResponseAsync(resourceGroupName, serviceName, apiId, context).block(); + } + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ApiCollectionInner getByAzureApiManagementService(String resourceGroupName, String serviceName, + String apiId) { + return getByAzureApiManagementServiceWithResponse(resourceGroupName, serviceName, apiId, Context.NONE) + .getValue(); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> onboardAzureApiManagementApiWithResponseAsync(String resourceGroupName, + String serviceName, String apiId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error(new IllegalArgumentException("Parameter apiCollectionId is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (serviceName == null) { + return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } - final String apiVersion = "2022-11-20-preview"; + if (apiId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiId is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiCollectionId, - apiVersion, - accept, - context)) + .withContext(context -> service.onboardAzureApiManagementApi(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, serviceName, apiId, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs along with {@link Response} on - * successful completion of {@link Mono}. + * @return an API collection as represented by Microsoft Defender for APIs along with {@link Response} on successful + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { + private Mono>> onboardAzureApiManagementApiWithResponseAsync(String resourceGroupName, + String serviceName, String apiId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (serviceName == null) { + return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); + } + if (apiId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiId is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.onboardAzureApiManagementApi(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, serviceName, apiId, apiVersion, accept, context); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, ApiCollectionInner> + beginOnboardAzureApiManagementApiAsync(String resourceGroupName, String serviceName, String apiId) { + Mono>> mono + = onboardAzureApiManagementApiWithResponseAsync(resourceGroupName, serviceName, apiId); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ApiCollectionInner.class, ApiCollectionInner.class, this.client.getContext()); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, ApiCollectionInner> beginOnboardAzureApiManagementApiAsync( + String resourceGroupName, String serviceName, String apiId, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = onboardAzureApiManagementApiWithResponseAsync(resourceGroupName, serviceName, apiId, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + ApiCollectionInner.class, ApiCollectionInner.class, context); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, ApiCollectionInner> + beginOnboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId) { + return this.beginOnboardAzureApiManagementApiAsync(resourceGroupName, serviceName, apiId).getSyncPoller(); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, ApiCollectionInner> + beginOnboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId, Context context) { + return this.beginOnboardAzureApiManagementApiAsync(resourceGroupName, serviceName, apiId, context) + .getSyncPoller(); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono onboardAzureApiManagementApiAsync(String resourceGroupName, String serviceName, + String apiId) { + return beginOnboardAzureApiManagementApiAsync(resourceGroupName, serviceName, apiId).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono onboardAzureApiManagementApiAsync(String resourceGroupName, String serviceName, + String apiId, Context context) { + return beginOnboardAzureApiManagementApiAsync(resourceGroupName, serviceName, apiId, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ApiCollectionInner onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId) { + return onboardAzureApiManagementApiAsync(resourceGroupName, serviceName, apiId).block(); + } + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an API collection as represented by Microsoft Defender for APIs. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ApiCollectionInner onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId, + Context context) { + return onboardAzureApiManagementApiAsync(resourceGroupName, serviceName, apiId, context).block(); + } + + /** + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> offboardAzureApiManagementApiWithResponseAsync(String resourceGroupName, + String serviceName, String apiId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -408,165 +1075,301 @@ private Mono> getWithResponseAsync( if (serviceName == null) { return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } - if (apiCollectionId == null) { + if (apiId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiId is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.offboardAzureApiManagementApi(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, serviceName, apiId, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> offboardAzureApiManagementApiWithResponseAsync(String resourceGroupName, + String serviceName, String apiId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { return Mono - .error(new IllegalArgumentException("Parameter apiCollectionId is required and cannot be null.")); + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (serviceName == null) { + return Mono.error(new IllegalArgumentException("Parameter serviceName is required and cannot be null.")); } - final String apiVersion = "2022-11-20-preview"; + if (apiId == null) { + return Mono.error(new IllegalArgumentException("Parameter apiId is required and cannot be null.")); + } + final String apiVersion = "2023-11-15"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - serviceName, - apiCollectionId, - apiVersion, - accept, - context); + return service.offboardAzureApiManagementApi(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, serviceName, apiId, apiVersion, accept, context); } /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs on successful completion of - * {@link Mono}. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String serviceName, String apiCollectionId) { - return getWithResponseAsync(resourceGroupName, serviceName, apiCollectionId) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono offboardAzureApiManagementApiAsync(String resourceGroupName, String serviceName, String apiId) { + return offboardAzureApiManagementApiWithResponseAsync(resourceGroupName, serviceName, apiId) + .flatMap(ignored -> Mono.empty()); } /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs along with {@link Response}. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - return getWithResponseAsync(resourceGroupName, serviceName, apiCollectionId, context).block(); + public Response offboardAzureApiManagementApiWithResponse(String resourceGroupName, String serviceName, + String apiId, Context context) { + return offboardAzureApiManagementApiWithResponseAsync(resourceGroupName, serviceName, apiId, context).block(); } /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ApiCollectionResponseInner get(String resourceGroupName, String serviceName, String apiCollectionId) { - return getWithResponse(resourceGroupName, serviceName, apiCollectionId, Context.NONE).getValue(); + public void offboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId) { + offboardAzureApiManagementApiWithResponse(resourceGroupName, serviceName, apiId, Context.NONE); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a list of API collections as represented by Defender for APIs along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * @return page of a list of API collections as represented by Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext( + context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a list of API collections as represented by Defender for APIs along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * @return page of a list of API collections as represented by Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return page of a list of API collections as represented by Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return page of a list of API collections as represented by Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return page of a list of API collections as represented by Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + listByAzureApiManagementServiceNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listByAzureApiManagementServiceNext(nextLink, this.client.getEndpoint(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return page of a list of API collections as represented by Microsoft Defender for APIs along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listByAzureApiManagementServiceNextSinglePageAsync(String nextLink, + Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listByAzureApiManagementServiceNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsImpl.java index 336ea9d218013..812edc7418b67 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApiCollectionsImpl.java @@ -10,8 +10,8 @@ import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.security.fluent.ApiCollectionsClient; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; -import com.azure.resourcemanager.security.models.ApiCollectionResponse; +import com.azure.resourcemanager.security.fluent.models.ApiCollectionInner; +import com.azure.resourcemanager.security.models.ApiCollection; import com.azure.resourcemanager.security.models.ApiCollections; public final class ApiCollectionsImpl implements ApiCollections { @@ -21,47 +21,98 @@ public final class ApiCollectionsImpl implements ApiCollections { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ApiCollectionsImpl( - ApiCollectionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public ApiCollectionsImpl(ApiCollectionsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable list(String resourceGroupName, String serviceName) { - PagedIterable inner = this.serviceClient().list(resourceGroupName, serviceName); - return Utils.mapPage(inner, inner1 -> new ApiCollectionResponseImpl(inner1, this.manager())); + public PagedIterable list() { + PagedIterable inner = this.serviceClient().list(); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApiCollectionImpl(inner1, this.manager())); } - public PagedIterable list(String resourceGroupName, String serviceName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, serviceName, context); - return Utils.mapPage(inner, inner1 -> new ApiCollectionResponseImpl(inner1, this.manager())); + public PagedIterable list(Context context) { + PagedIterable inner = this.serviceClient().list(context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApiCollectionImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, serviceName, apiCollectionId, context); + public PagedIterable listByResourceGroup(String resourceGroupName) { + PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApiCollectionImpl(inner1, this.manager())); + } + + public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { + PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApiCollectionImpl(inner1, this.manager())); + } + + public PagedIterable listByAzureApiManagementService(String resourceGroupName, String serviceName) { + PagedIterable inner + = this.serviceClient().listByAzureApiManagementService(resourceGroupName, serviceName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApiCollectionImpl(inner1, this.manager())); + } + + public PagedIterable listByAzureApiManagementService(String resourceGroupName, String serviceName, + Context context) { + PagedIterable inner + = this.serviceClient().listByAzureApiManagementService(resourceGroupName, serviceName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApiCollectionImpl(inner1, this.manager())); + } + + public Response getByAzureApiManagementServiceWithResponse(String resourceGroupName, + String serviceName, String apiId, Context context) { + Response inner = this.serviceClient() + .getByAzureApiManagementServiceWithResponse(resourceGroupName, serviceName, apiId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ApiCollectionImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ApiCollection getByAzureApiManagementService(String resourceGroupName, String serviceName, String apiId) { + ApiCollectionInner inner + = this.serviceClient().getByAzureApiManagementService(resourceGroupName, serviceName, apiId); + if (inner != null) { + return new ApiCollectionImpl(inner, this.manager()); + } else { + return null; + } + } + + public ApiCollection onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId) { + ApiCollectionInner inner + = this.serviceClient().onboardAzureApiManagementApi(resourceGroupName, serviceName, apiId); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ApiCollectionResponseImpl(inner.getValue(), this.manager())); + return new ApiCollectionImpl(inner, this.manager()); } else { return null; } } - public ApiCollectionResponse get(String resourceGroupName, String serviceName, String apiCollectionId) { - ApiCollectionResponseInner inner = this.serviceClient().get(resourceGroupName, serviceName, apiCollectionId); + public ApiCollection onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId, + Context context) { + ApiCollectionInner inner + = this.serviceClient().onboardAzureApiManagementApi(resourceGroupName, serviceName, apiId, context); if (inner != null) { - return new ApiCollectionResponseImpl(inner, this.manager()); + return new ApiCollectionImpl(inner, this.manager()); } else { return null; } } + public Response offboardAzureApiManagementApiWithResponse(String resourceGroupName, String serviceName, + String apiId, Context context) { + return this.serviceClient().offboardAzureApiManagementApiWithResponse(resourceGroupName, serviceName, apiId, + context); + } + + public void offboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId) { + this.serviceClient().offboardAzureApiManagementApi(resourceGroupName, serviceName, apiId); + } + private ApiCollectionsClient serviceClient() { return this.innerClient; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationImpl.java index 6bb5827d3d18b..b7aec34108ce6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationImpl.java @@ -60,22 +60,14 @@ private com.azure.resourcemanager.security.SecurityManager manager() { private String applicationId; public Application create() { - this.innerObject = - serviceManager - .serviceClient() - .getApplicationOperations() - .createOrUpdateWithResponse(applicationId, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getApplicationOperations() + .createOrUpdateWithResponse(applicationId, this.innerModel(), Context.NONE).getValue(); return this; } public Application create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getApplicationOperations() - .createOrUpdateWithResponse(applicationId, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getApplicationOperations() + .createOrUpdateWithResponse(applicationId, this.innerModel(), context).getValue(); return this; } @@ -90,48 +82,32 @@ public ApplicationImpl update() { } public Application apply() { - this.innerObject = - serviceManager - .serviceClient() - .getApplicationOperations() - .createOrUpdateWithResponse(applicationId, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getApplicationOperations() + .createOrUpdateWithResponse(applicationId, this.innerModel(), Context.NONE).getValue(); return this; } public Application apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getApplicationOperations() - .createOrUpdateWithResponse(applicationId, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getApplicationOperations() + .createOrUpdateWithResponse(applicationId, this.innerModel(), context).getValue(); return this; } ApplicationImpl(ApplicationInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.applicationId = Utils.getValueFromIdByName(innerObject.id(), "applications"); + this.applicationId = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "applications"); } public Application refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getApplicationOperations() - .getWithResponse(applicationId, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getApplicationOperations() + .getWithResponse(applicationId, Context.NONE).getValue(); return this; } public Application refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getApplicationOperations() - .getWithResponse(applicationId, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getApplicationOperations() + .getWithResponse(applicationId, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsClientImpl.java index 4a46f329ee115..2dee16d2c2d36 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsClientImpl.java @@ -28,23 +28,28 @@ import com.azure.resourcemanager.security.fluent.models.ApplicationInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ApplicationOperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in ApplicationOperationsClient. + */ public final class ApplicationOperationsClientImpl implements ApplicationOperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ApplicationOperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ApplicationOperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ApplicationOperationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(ApplicationOperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ApplicationOperationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -55,66 +60,52 @@ public final class ApplicationOperationsClientImpl implements ApplicationOperati @Host("{$host}") @ServiceInterface(name = "SecurityCenterApplic") public interface ApplicationOperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/applications/{applicationId}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("applicationId") String applicationId, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("applicationId") String applicationId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/applications/{applicationId}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("applicationId") String applicationId, - @BodyParam("application/json") ApplicationInner application, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ApplicationInner application, @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) + @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/applications/{applicationId}") - @ExpectedResponses({200, 204}) + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("applicationId") String applicationId, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("applicationId") String applicationId, Context context); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific application for the requested scope by applicationId along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String applicationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (applicationId == null) { return Mono.error(new IllegalArgumentException("Parameter applicationId is required and cannot be null.")); @@ -122,43 +113,31 @@ private Mono> getWithResponseAsync(String application final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - applicationId, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + applicationId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific application for the requested scope by applicationId along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String applicationId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (applicationId == null) { return Mono.error(new IllegalArgumentException("Parameter applicationId is required and cannot be null.")); @@ -166,14 +145,13 @@ private Mono> getWithResponseAsync(String application final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), applicationId, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), applicationId, + accept, context); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -187,7 +165,7 @@ private Mono getAsync(String applicationId) { /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -202,7 +180,7 @@ public Response getWithResponse(String applicationId, Context /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -216,29 +194,25 @@ public ApplicationInner get(String applicationId) { /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security Application over a given scope along with {@link Response} on successful completion of {@link - * Mono}. + * @return security Application over a given scope along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String applicationId, ApplicationInner application) { + private Mono> createOrUpdateWithResponseAsync(String applicationId, + ApplicationInner application) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (applicationId == null) { return Mono.error(new IllegalArgumentException("Parameter applicationId is required and cannot be null.")); @@ -251,46 +225,33 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - applicationId, - application, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), applicationId, application, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security Application over a given scope along with {@link Response} on successful completion of {@link - * Mono}. + * @return security Application over a given scope along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String applicationId, ApplicationInner application, Context context) { + private Mono> createOrUpdateWithResponseAsync(String applicationId, + ApplicationInner application, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (applicationId == null) { return Mono.error(new IllegalArgumentException("Parameter applicationId is required and cannot be null.")); @@ -303,20 +264,13 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - applicationId, - application, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + applicationId, application, accept, context); } /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -332,7 +286,7 @@ private Mono createOrUpdateAsync(String applicationId, Applica /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @param context The context to associate with this operation. @@ -342,14 +296,14 @@ private Mono createOrUpdateAsync(String applicationId, Applica * @return security Application over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String applicationId, ApplicationInner application, Context context) { + public Response createOrUpdateWithResponse(String applicationId, ApplicationInner application, + Context context) { return createOrUpdateWithResponseAsync(applicationId, application, context).block(); } /** * Creates or update a security application on the given subscription. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -364,7 +318,7 @@ public ApplicationInner createOrUpdate(String applicationId, ApplicationInner ap /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -374,37 +328,26 @@ public ApplicationInner createOrUpdate(String applicationId, ApplicationInner ap @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String applicationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (applicationId == null) { return Mono.error(new IllegalArgumentException("Parameter applicationId is required and cannot be null.")); } final String apiVersion = "2022-07-01-preview"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - applicationId, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), applicationId, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -415,29 +358,25 @@ private Mono> deleteWithResponseAsync(String applicationId) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String applicationId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (applicationId == null) { return Mono.error(new IllegalArgumentException("Parameter applicationId is required and cannot be null.")); } final String apiVersion = "2022-07-01-preview"; context = this.client.mergeContext(context); - return service - .delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), applicationId, context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), applicationId, + context); } /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -451,7 +390,7 @@ private Mono deleteAsync(String applicationId) { /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -466,7 +405,7 @@ public Response deleteWithResponse(String applicationId, Context context) /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsImpl.java index 7d6d4ab2601b3..fb1cc5d536bb5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationOperationsImpl.java @@ -20,8 +20,8 @@ public final class ApplicationOperationsImpl implements ApplicationOperations { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ApplicationOperationsImpl( - ApplicationOperationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public ApplicationOperationsImpl(ApplicationOperationsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -29,10 +29,7 @@ public ApplicationOperationsImpl( public Response getWithResponse(String applicationId, Context context) { Response inner = this.serviceClient().getWithResponse(applicationId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ApplicationImpl(inner.getValue(), this.manager())); } else { return null; @@ -57,45 +54,37 @@ public void delete(String applicationId) { } public Application getById(String id) { - String applicationId = Utils.getValueFromIdByName(id, "applications"); + String applicationId = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } return this.getWithResponse(applicationId, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String applicationId = Utils.getValueFromIdByName(id, "applications"); + String applicationId = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } return this.getWithResponse(applicationId, context); } public void deleteById(String id) { - String applicationId = Utils.getValueFromIdByName(id, "applications"); + String applicationId = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } this.deleteWithResponse(applicationId, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String applicationId = Utils.getValueFromIdByName(id, "applications"); + String applicationId = ResourceManagerUtils.getValueFromIdByName(id, "applications"); if (applicationId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'applications'.", id))); } return this.deleteWithResponse(applicationId, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsClientImpl.java index b604de04135ab..7462cc2038ff8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsClientImpl.java @@ -30,142 +30,117 @@ import com.azure.resourcemanager.security.models.ApplicationsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ApplicationsClient. */ +/** + * An instance of this class provides access to all the operations defined in ApplicationsClient. + */ public final class ApplicationsClientImpl implements ApplicationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ApplicationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ApplicationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ApplicationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(ApplicationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ApplicationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterApplications to be used by the proxy service to perform - * REST calls. + * The interface defining all the services for SecurityCenterApplications to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterApplic") public interface ApplicationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/applications") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of all relevant applications over a subscription level scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a subscription level scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of all relevant applications over a subscription level scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a subscription level scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of all relevant applications over a subscription level scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedFlux}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -174,27 +149,27 @@ private PagedFlux listAsync() { /** * Get a list of all relevant applications over a subscription level scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedFlux}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of all relevant applications over a subscription level scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedIterable}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -203,13 +178,13 @@ public PagedIterable list() { /** * Get a list of all relevant applications over a subscription level scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedIterable}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -218,14 +193,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a security applications list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a security applications list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -233,37 +209,28 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a security applications list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a security applications list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -271,23 +238,13 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsImpl.java index 190bf422bb857..94842779359ea 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ApplicationsImpl.java @@ -19,20 +19,20 @@ public final class ApplicationsImpl implements Applications { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ApplicationsImpl( - ApplicationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public ApplicationsImpl(ApplicationsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); } private ApplicationsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsClientImpl.java index 2cbcb3ac9e6c5..fe1d978241cf6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsClientImpl.java @@ -35,22 +35,28 @@ import com.azure.resourcemanager.security.models.SecurityAssessmentList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in AssessmentsClient. + */ public final class AssessmentsClientImpl implements AssessmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AssessmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AssessmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AssessmentsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(AssessmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(AssessmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -61,84 +67,67 @@ public final class AssessmentsClientImpl implements AssessmentsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterAssess") public interface AssessmentsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/assessments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("assessmentName") String assessmentName, - @QueryParam("$expand") ExpandEnum expand, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("assessmentName") String assessmentName, @QueryParam("$expand") ExpandEnum expand, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, @PathParam("assessmentName") String assessmentName, - @BodyParam("application/json") SecurityAssessment assessment, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") SecurityAssessment assessment, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Delete("/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}") - @ExpectedResponses({200, 204}) + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("assessmentName") String assessmentName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("assessmentName") String assessmentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security assessments on all your scanned resources inside a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -147,37 +136,28 @@ private Mono> listSinglePageAsync final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security assessments on all your scanned resources inside a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -185,29 +165,21 @@ private Mono> listSinglePageAsync final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedFlux}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope) { @@ -216,32 +188,32 @@ private PagedFlux listAsync(String scope) { /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedFlux}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String scope) { @@ -250,15 +222,15 @@ public PagedIterable list(String scope) { /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String scope, Context context) { @@ -267,7 +239,7 @@ public PagedIterable list(String scope, Context /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param expand OData expand. Optional. @@ -275,16 +247,14 @@ public PagedIterable list(String scope, Context * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a security assessment on your scanned resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, String assessmentName, ExpandEnum expand) { + private Mono> getWithResponseAsync(String resourceId, + String assessmentName, ExpandEnum expand) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -295,17 +265,14 @@ private Mono> getWithResponseAsync( final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), apiVersion, resourceId, assessmentName, expand, accept, context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, resourceId, assessmentName, + expand, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param expand OData expand. Optional. @@ -314,16 +281,14 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a security assessment on your scanned resource along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, String assessmentName, ExpandEnum expand, Context context) { + private Mono> getWithResponseAsync(String resourceId, + String assessmentName, ExpandEnum expand, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -339,7 +304,7 @@ private Mono> getWithResponseAsync( /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -356,7 +321,7 @@ private Mono getAsync(String resourceId, String /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param expand OData expand. Optional. @@ -367,14 +332,14 @@ private Mono getAsync(String resourceId, String * @return a security assessment on your scanned resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceId, String assessmentName, ExpandEnum expand, Context context) { + public Response getWithResponse(String resourceId, String assessmentName, + ExpandEnum expand, Context context) { return getWithResponseAsync(resourceId, assessmentName, expand, context).block(); } /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -391,7 +356,7 @@ public SecurityAssessmentResponseInner get(String resourceId, String assessmentN /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -399,16 +364,14 @@ public SecurityAssessmentResponseInner get(String resourceId, String assessmentN * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security assessment on a resource - response format along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceId, String assessmentName, SecurityAssessment assessment) { + private Mono> createOrUpdateWithResponseAsync(String resourceId, + String assessmentName, SecurityAssessment assessment) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -424,24 +387,15 @@ private Mono> createOrUpdateWithRespon final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - resourceId, - assessmentName, - assessment, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, resourceId, + assessmentName, assessment, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -450,16 +404,14 @@ private Mono> createOrUpdateWithRespon * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security assessment on a resource - response format along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceId, String assessmentName, SecurityAssessment assessment, Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceId, + String assessmentName, SecurityAssessment assessment, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -475,15 +427,14 @@ private Mono> createOrUpdateWithRespon final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), apiVersion, resourceId, assessmentName, assessment, accept, context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, resourceId, assessmentName, assessment, + accept, context); } /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -493,8 +444,8 @@ private Mono> createOrUpdateWithRespon * @return security assessment on a resource - response format on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceId, String assessmentName, SecurityAssessment assessment) { + private Mono createOrUpdateAsync(String resourceId, String assessmentName, + SecurityAssessment assessment) { return createOrUpdateWithResponseAsync(resourceId, assessmentName, assessment) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -502,7 +453,7 @@ private Mono createOrUpdateAsync( /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -513,15 +464,15 @@ private Mono createOrUpdateAsync( * @return security assessment on a resource - response format along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceId, String assessmentName, SecurityAssessment assessment, Context context) { + public Response createOrUpdateWithResponse(String resourceId, + String assessmentName, SecurityAssessment assessment, Context context) { return createOrUpdateWithResponseAsync(resourceId, assessmentName, assessment, context).block(); } /** * Create a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param assessment Calculated assessment on a pre-defined assessment metadata. @@ -531,15 +482,15 @@ public Response createOrUpdateWithResponse( * @return security assessment on a resource - response format. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SecurityAssessmentResponseInner createOrUpdate( - String resourceId, String assessmentName, SecurityAssessment assessment) { + public SecurityAssessmentResponseInner createOrUpdate(String resourceId, String assessmentName, + SecurityAssessment assessment) { return createOrUpdateWithResponse(resourceId, assessmentName, assessment, Context.NONE).getValue(); } /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -550,10 +501,8 @@ public SecurityAssessmentResponseInner createOrUpdate( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceId, String assessmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -564,16 +513,15 @@ private Mono> deleteWithResponseAsync(String resourceId, String a final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service.delete(this.client.getEndpoint(), apiVersion, resourceId, assessmentName, accept, context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, resourceId, assessmentName, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. @@ -585,10 +533,8 @@ private Mono> deleteWithResponseAsync(String resourceId, String a @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceId, String assessmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -605,7 +551,7 @@ private Mono> deleteWithResponseAsync(String resourceId, String a /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -621,7 +567,7 @@ private Mono deleteAsync(String resourceId, String assessmentName) { /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. @@ -638,7 +584,7 @@ public Response deleteWithResponse(String resourceId, String assessmentNam /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -652,14 +598,15 @@ public void delete(String resourceId, String assessmentName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a security assessments list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a security assessments list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -667,62 +614,43 @@ private Mono> listNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a security assessments list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a security assessments list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsImpl.java index 0e5cc1fb108d6..e0843e5ad7500 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsImpl.java @@ -22,31 +22,30 @@ public final class AssessmentsImpl implements Assessments { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AssessmentsImpl( - AssessmentsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public AssessmentsImpl(AssessmentsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String scope) { PagedIterable inner = this.serviceClient().list(scope); - return Utils.mapPage(inner, inner1 -> new SecurityAssessmentResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecurityAssessmentResponseImpl(inner1, this.manager())); } public PagedIterable list(String scope, Context context) { PagedIterable inner = this.serviceClient().list(scope, context); - return Utils.mapPage(inner, inner1 -> new SecurityAssessmentResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecurityAssessmentResponseImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceId, String assessmentName, ExpandEnum expand, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceId, assessmentName, expand, context); + public Response getWithResponse(String resourceId, String assessmentName, + ExpandEnum expand, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceId, assessmentName, expand, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityAssessmentResponseImpl(inner.getValue(), this.manager())); } else { return null; @@ -71,98 +70,66 @@ public void deleteByResourceGroup(String resourceId, String assessmentName) { } public SecurityAssessmentResponse getById(String id) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } ExpandEnum localExpand = null; return this.getWithResponse(resourceId, assessmentName, localExpand, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, ExpandEnum expand, Context context) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } return this.getWithResponse(resourceId, assessmentName, expand, context); } public void deleteById(String id) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } this.deleteByResourceGroupWithResponse(resourceId, assessmentName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } return this.deleteByResourceGroupWithResponse(resourceId, assessmentName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasClientImpl.java index 77bbc5dbcc388..31e296e55c852 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasClientImpl.java @@ -33,193 +33,158 @@ import com.azure.resourcemanager.security.models.SecurityAssessmentMetadataResponseList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AssessmentsMetadatasClient. */ +/** + * An instance of this class provides access to all the operations defined in AssessmentsMetadatasClient. + */ public final class AssessmentsMetadatasClientImpl implements AssessmentsMetadatasClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AssessmentsMetadatasService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AssessmentsMetadatasClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AssessmentsMetadatasClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(AssessmentsMetadatasService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AssessmentsMetadatasService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterAssessmentsMetadatas to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for SecurityCenterAssessmentsMetadatas to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterAssess") public interface AssessmentsMetadatasService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Security/assessmentMetadata") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Security/assessmentMetadata/{assessmentMetadataName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, - @PathParam("assessmentMetadataName") String assessmentMetadataName, - @HeaderParam("Accept") String accept, + @PathParam("assessmentMetadataName") String assessmentMetadataName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/assessmentMetadata") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listBySubscription( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listBySubscription(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/assessmentMetadata/{assessmentMetadataName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getInSubscription( - @HostParam("$host") String endpoint, + Mono> getInSubscription(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("assessmentMetadataName") String assessmentMetadataName, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/assessmentMetadata/{assessmentMetadataName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> createInSubscription( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + @HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("assessmentMetadataName") String assessmentMetadataName, @PathParam("subscriptionId") String subscriptionId, @BodyParam("application/json") SecurityAssessmentMetadataResponseInner assessmentMetadata, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/assessmentMetadata/{assessmentMetadataName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/assessmentMetadata/{assessmentMetadataName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> deleteInSubscription( - @HostParam("$host") String endpoint, + Mono> deleteInSubscription(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("assessmentMetadataName") String assessmentMetadataName, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("subscriptionId") String subscriptionId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get metadata information on all assessment types. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) + return FluxUtil.withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get metadata information on all assessment types. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get metadata information on all assessment types. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types as paginated response with {@link PagedFlux}. @@ -231,7 +196,7 @@ private PagedFlux listAsync() { /** * Get metadata information on all assessment types. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -240,13 +205,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get metadata information on all assessment types. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types as paginated response with {@link PagedIterable}. @@ -258,7 +223,7 @@ public PagedIterable list() { /** * Get metadata information on all assessment types. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -272,27 +237,24 @@ public PagedIterable list(Context conte /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on an assessment type along with {@link Response} on successful completion of {@link - * Mono}. + * @return metadata information on an assessment type along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String assessmentMetadataName) { + private Mono> + getWithResponseAsync(String assessmentMetadataName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; @@ -304,28 +266,25 @@ private Mono> getWithResponseA /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on an assessment type along with {@link Response} on successful completion of {@link - * Mono}. + * @return metadata information on an assessment type along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String assessmentMetadataName, Context context) { + private Mono> getWithResponseAsync(String assessmentMetadataName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; @@ -335,7 +294,7 @@ private Mono> getWithResponseA /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -349,7 +308,7 @@ private Mono getAsync(String assessment /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -358,14 +317,14 @@ private Mono getAsync(String assessment * @return metadata information on an assessment type along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String assessmentMetadataName, Context context) { + public Response getWithResponse(String assessmentMetadataName, + Context context) { return getWithResponseAsync(assessmentMetadataName, context).block(); } /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -379,125 +338,100 @@ public SecurityAssessmentMetadataResponseInner get(String assessmentMetadataName /** * Get metadata information on all assessment types in a specific subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types in a specific subscription along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listBySubscription( - this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) + .withContext(context -> service.listBySubscription(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get metadata information on all assessment types in a specific subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types in a specific subscription along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionSinglePageAsync( - Context context) { + private Mono> + listBySubscriptionSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .listBySubscription(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get metadata information on all assessment types in a specific subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedFlux}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySubscriptionAsync() { - return new PagedFlux<>( - () -> listBySubscriptionSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listBySubscriptionSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Get metadata information on all assessment types in a specific subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedFlux}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySubscriptionAsync(Context context) { - return new PagedFlux<>( - () -> listBySubscriptionSinglePageAsync(context), + return new PagedFlux<>(() -> listBySubscriptionSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Get metadata information on all assessment types in a specific subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedIterable}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listBySubscription() { @@ -506,13 +440,13 @@ public PagedIterable listBySubscription /** * Get metadata information on all assessment types in a specific subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedIterable}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listBySubscription(Context context) { @@ -521,103 +455,79 @@ public PagedIterable listBySubscription /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on an assessment type in a specific subscription along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getInSubscriptionWithResponseAsync( - String assessmentMetadataName) { + private Mono> + getInSubscriptionWithResponseAsync(String assessmentMetadataName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getInSubscription( - this.client.getEndpoint(), - apiVersion, - assessmentMetadataName, - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.getInSubscription(this.client.getEndpoint(), apiVersion, + assessmentMetadataName, this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on an assessment type in a specific subscription along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getInSubscriptionWithResponseAsync( - String assessmentMetadataName, Context context) { + private Mono> + getInSubscriptionWithResponseAsync(String assessmentMetadataName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getInSubscription( - this.client.getEndpoint(), - apiVersion, - assessmentMetadataName, - this.client.getSubscriptionId(), - accept, - context); + return service.getInSubscription(this.client.getEndpoint(), apiVersion, assessmentMetadataName, + this.client.getSubscriptionId(), accept, context); } /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on an assessment type in a specific subscription on successful completion of {@link - * Mono}. + * @return metadata information on an assessment type in a specific subscription on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getInSubscriptionAsync(String assessmentMetadataName) { @@ -627,7 +537,7 @@ private Mono getInSubscriptionAsync(Str /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -636,14 +546,14 @@ private Mono getInSubscriptionAsync(Str * @return metadata information on an assessment type in a specific subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getInSubscriptionWithResponse( - String assessmentMetadataName, Context context) { + public Response + getInSubscriptionWithResponse(String assessmentMetadataName, Context context) { return getInSubscriptionWithResponseAsync(assessmentMetadataName, context).block(); } /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -657,34 +567,29 @@ public SecurityAssessmentMetadataResponseInner getInSubscription(String assessme /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessment metadata response along with {@link Response} on successful completion of {@link - * Mono}. + * @return security assessment metadata response along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> createInSubscriptionWithResponseAsync( String assessmentMetadataName, SecurityAssessmentMetadataResponseInner assessmentMetadata) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (assessmentMetadata == null) { return Mono @@ -695,51 +600,37 @@ private Mono> createInSubscrip final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createInSubscription( - this.client.getEndpoint(), - apiVersion, - assessmentMetadataName, - this.client.getSubscriptionId(), - assessmentMetadata, - accept, - context)) + .withContext(context -> service.createInSubscription(this.client.getEndpoint(), apiVersion, + assessmentMetadataName, this.client.getSubscriptionId(), assessmentMetadata, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessment metadata response along with {@link Response} on successful completion of {@link - * Mono}. + * @return security assessment metadata response along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> createInSubscriptionWithResponseAsync( String assessmentMetadataName, SecurityAssessmentMetadataResponseInner assessmentMetadata, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (assessmentMetadata == null) { return Mono @@ -750,20 +641,13 @@ private Mono> createInSubscrip final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createInSubscription( - this.client.getEndpoint(), - apiVersion, - assessmentMetadataName, - this.client.getSubscriptionId(), - assessmentMetadata, - accept, - context); + return service.createInSubscription(this.client.getEndpoint(), apiVersion, assessmentMetadataName, + this.client.getSubscriptionId(), assessmentMetadata, accept, context); } /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -772,15 +656,15 @@ private Mono> createInSubscrip * @return security assessment metadata response on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createInSubscriptionAsync( - String assessmentMetadataName, SecurityAssessmentMetadataResponseInner assessmentMetadata) { + private Mono createInSubscriptionAsync(String assessmentMetadataName, + SecurityAssessmentMetadataResponseInner assessmentMetadata) { return createInSubscriptionWithResponseAsync(assessmentMetadataName, assessmentMetadata) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @param context The context to associate with this operation. @@ -797,7 +681,7 @@ public Response createInSubscriptionWit /** * Create metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param assessmentMetadata AssessmentMetadata object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -806,15 +690,15 @@ public Response createInSubscriptionWit * @return security assessment metadata response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SecurityAssessmentMetadataResponseInner createInSubscription( - String assessmentMetadataName, SecurityAssessmentMetadataResponseInner assessmentMetadata) { + public SecurityAssessmentMetadataResponseInner createInSubscription(String assessmentMetadataName, + SecurityAssessmentMetadataResponseInner assessmentMetadata) { return createInSubscriptionWithResponse(assessmentMetadataName, assessmentMetadata, Context.NONE).getValue(); } /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -824,42 +708,29 @@ public SecurityAssessmentMetadataResponseInner createInSubscription( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteInSubscriptionWithResponseAsync(String assessmentMetadataName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .deleteInSubscription( - this.client.getEndpoint(), - apiVersion, - assessmentMetadataName, - this.client.getSubscriptionId(), - accept, - context)) + .withContext(context -> service.deleteInSubscription(this.client.getEndpoint(), apiVersion, + assessmentMetadataName, this.client.getSubscriptionId(), accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -870,39 +741,28 @@ private Mono> deleteInSubscriptionWithResponseAsync(String assess @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteInSubscriptionWithResponseAsync(String assessmentMetadataName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (assessmentMetadataName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter assessmentMetadataName is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-06-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .deleteInSubscription( - this.client.getEndpoint(), - apiVersion, - assessmentMetadataName, - this.client.getSubscriptionId(), - accept, - context); + return service.deleteInSubscription(this.client.getEndpoint(), apiVersion, assessmentMetadataName, + this.client.getSubscriptionId(), accept, context); } /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -917,7 +777,7 @@ private Mono deleteInSubscriptionAsync(String assessmentMetadataName) { /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -933,7 +793,7 @@ public Response deleteInSubscriptionWithResponse(String assessmentMetadata /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -946,14 +806,15 @@ public void deleteInSubscription(String assessmentMetadataName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -961,140 +822,106 @@ private Mono> listNextSin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security assessment metadata along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasImpl.java index 5a878349df354..6ddb28434bef5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AssessmentsMetadatasImpl.java @@ -21,31 +21,30 @@ public final class AssessmentsMetadatasImpl implements AssessmentsMetadatas { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AssessmentsMetadatasImpl( - AssessmentsMetadatasClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public AssessmentsMetadatasImpl(AssessmentsMetadatasClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); } - public Response getWithResponse( - String assessmentMetadataName, Context context) { - Response inner = - this.serviceClient().getWithResponse(assessmentMetadataName, context); + public Response getWithResponse(String assessmentMetadataName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(assessmentMetadataName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityAssessmentMetadataResponseImpl(inner.getValue(), this.manager())); } else { return null; @@ -63,23 +62,22 @@ public SecurityAssessmentMetadataResponse get(String assessmentMetadataName) { public PagedIterable listBySubscription() { PagedIterable inner = this.serviceClient().listBySubscription(); - return Utils.mapPage(inner, inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); } public PagedIterable listBySubscription(Context context) { PagedIterable inner = this.serviceClient().listBySubscription(context); - return Utils.mapPage(inner, inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecurityAssessmentMetadataResponseImpl(inner1, this.manager())); } - public Response getInSubscriptionWithResponse( - String assessmentMetadataName, Context context) { - Response inner = - this.serviceClient().getInSubscriptionWithResponse(assessmentMetadataName, context); + public Response getInSubscriptionWithResponse(String assessmentMetadataName, + Context context) { + Response inner + = this.serviceClient().getInSubscriptionWithResponse(assessmentMetadataName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityAssessmentMetadataResponseImpl(inner.getValue(), this.manager())); } else { return null; @@ -104,53 +102,37 @@ public void deleteInSubscription(String assessmentMetadataName) { } public SecurityAssessmentMetadataResponse getInSubscriptionById(String id) { - String assessmentMetadataName = Utils.getValueFromIdByName(id, "assessmentMetadata"); + String assessmentMetadataName = ResourceManagerUtils.getValueFromIdByName(id, "assessmentMetadata"); if (assessmentMetadataName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); } return this.getInSubscriptionWithResponse(assessmentMetadataName, Context.NONE).getValue(); } public Response getInSubscriptionByIdWithResponse(String id, Context context) { - String assessmentMetadataName = Utils.getValueFromIdByName(id, "assessmentMetadata"); + String assessmentMetadataName = ResourceManagerUtils.getValueFromIdByName(id, "assessmentMetadata"); if (assessmentMetadataName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); } return this.getInSubscriptionWithResponse(assessmentMetadataName, context); } public void deleteInSubscriptionById(String id) { - String assessmentMetadataName = Utils.getValueFromIdByName(id, "assessmentMetadata"); + String assessmentMetadataName = ResourceManagerUtils.getValueFromIdByName(id, "assessmentMetadata"); if (assessmentMetadataName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); } this.deleteInSubscriptionWithResponse(assessmentMetadataName, Context.NONE); } public Response deleteInSubscriptionByIdWithResponse(String id, Context context) { - String assessmentMetadataName = Utils.getValueFromIdByName(id, "assessmentMetadata"); + String assessmentMetadataName = ResourceManagerUtils.getValueFromIdByName(id, "assessmentMetadata"); if (assessmentMetadataName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessmentMetadata'.", id))); } return this.deleteInSubscriptionWithResponse(assessmentMetadataName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingImpl.java index 896c6cbf97910..8837be6b785c8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingImpl.java @@ -14,8 +14,8 @@ public final class AutoProvisioningSettingImpl implements AutoProvisioningSettin private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AutoProvisioningSettingImpl( - AutoProvisioningSettingInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + AutoProvisioningSettingImpl(AutoProvisioningSettingInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -47,22 +47,14 @@ private com.azure.resourcemanager.security.SecurityManager manager() { private String settingName; public AutoProvisioningSetting create() { - this.innerObject = - serviceManager - .serviceClient() - .getAutoProvisioningSettings() - .createWithResponse(settingName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutoProvisioningSettings() + .createWithResponse(settingName, this.innerModel(), Context.NONE).getValue(); return this; } public AutoProvisioningSetting create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAutoProvisioningSettings() - .createWithResponse(settingName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutoProvisioningSettings() + .createWithResponse(settingName, this.innerModel(), context).getValue(); return this; } @@ -73,22 +65,14 @@ public AutoProvisioningSetting create(Context context) { } public AutoProvisioningSetting refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getAutoProvisioningSettings() - .getWithResponse(settingName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutoProvisioningSettings() + .getWithResponse(settingName, Context.NONE).getValue(); return this; } public AutoProvisioningSetting refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAutoProvisioningSettings() - .getWithResponse(settingName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutoProvisioningSettings() + .getWithResponse(settingName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsClientImpl.java index 9c600ec6bd5db..d5562f17e13fc 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsClientImpl.java @@ -32,23 +32,28 @@ import com.azure.resourcemanager.security.models.AutoProvisioningSettingList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AutoProvisioningSettingsClient. */ +/** + * An instance of this class provides access to all the operations defined in AutoProvisioningSettingsClient. + */ public final class AutoProvisioningSettingsClientImpl implements AutoProvisioningSettingsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AutoProvisioningSettingsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AutoProvisioningSettingsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AutoProvisioningSettingsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(AutoProvisioningSettingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(AutoProvisioningSettingsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -59,137 +64,100 @@ public final class AutoProvisioningSettingsClientImpl implements AutoProvisionin @Host("{$host}") @ServiceInterface(name = "SecurityCenterAutoPr") public interface AutoProvisioningSettingsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/autoProvisioningSettings") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/autoProvisioningSettings/{settingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("settingName") String settingName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("settingName") String settingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/autoProvisioningSettings/{settingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("settingName") String settingName, - @BodyParam("application/json") AutoProvisioningSettingInner setting, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") AutoProvisioningSettingInner setting, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response as paginated response with {@link PagedFlux}. @@ -201,7 +169,7 @@ private PagedFlux listAsync() { /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -210,13 +178,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response as paginated response with {@link PagedIterable}. @@ -228,7 +196,7 @@ public PagedIterable list() { /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -242,7 +210,7 @@ public PagedIterable list(Context context) { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -252,16 +220,12 @@ public PagedIterable list(Context context) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String settingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -269,22 +233,14 @@ private Mono> getWithResponseAsync(String final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - settingName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + settingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -295,16 +251,12 @@ private Mono> getWithResponseAsync(String @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String settingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -312,13 +264,13 @@ private Mono> getWithResponseAsync(String final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingName, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingName, accept, + context); } /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -332,7 +284,7 @@ private Mono getAsync(String settingName) { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -347,7 +299,7 @@ public Response getWithResponse(String settingName /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -361,7 +313,7 @@ public AutoProvisioningSettingInner get(String settingName) { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -370,19 +322,15 @@ public AutoProvisioningSettingInner get(String settingName) { * @return auto provisioning setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String settingName, AutoProvisioningSettingInner setting) { + private Mono> createWithResponseAsync(String settingName, + AutoProvisioningSettingInner setting) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -395,23 +343,14 @@ private Mono> createWithResponseAsync( final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - settingName, - setting, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), settingName, setting, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @param context The context to associate with this operation. @@ -421,19 +360,15 @@ private Mono> createWithResponseAsync( * @return auto provisioning setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String settingName, AutoProvisioningSettingInner setting, Context context) { + private Mono> createWithResponseAsync(String settingName, + AutoProvisioningSettingInner setting, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -446,20 +381,13 @@ private Mono> createWithResponseAsync( final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - settingName, - setting, - accept, - context); + return service.create(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingName, + setting, accept, context); } /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -474,7 +402,7 @@ private Mono createAsync(String settingName, AutoP /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @param context The context to associate with this operation. @@ -484,14 +412,14 @@ private Mono createAsync(String settingName, AutoP * @return auto provisioning setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String settingName, AutoProvisioningSettingInner setting, Context context) { + public Response createWithResponse(String settingName, + AutoProvisioningSettingInner setting, Context context) { return createWithResponseAsync(settingName, setting, context).block(); } /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param setting Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -506,14 +434,15 @@ public AutoProvisioningSettingInner create(String settingName, AutoProvisioningS /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -521,62 +450,43 @@ private Mono> listNextSinglePageAsyn return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsImpl.java index 7d284eeb293ee..6f422238d33ed 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutoProvisioningSettingsImpl.java @@ -21,29 +21,26 @@ public final class AutoProvisioningSettingsImpl implements AutoProvisioningSetti private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AutoProvisioningSettingsImpl( - AutoProvisioningSettingsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public AutoProvisioningSettingsImpl(AutoProvisioningSettingsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new AutoProvisioningSettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AutoProvisioningSettingImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new AutoProvisioningSettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AutoProvisioningSettingImpl(inner1, this.manager())); } public Response getWithResponse(String settingName, Context context) { Response inner = this.serviceClient().getWithResponse(settingName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AutoProvisioningSettingImpl(inner.getValue(), this.manager())); } else { return null; @@ -60,29 +57,19 @@ public AutoProvisioningSetting get(String settingName) { } public AutoProvisioningSetting getById(String id) { - String settingName = Utils.getValueFromIdByName(id, "autoProvisioningSettings"); + String settingName = ResourceManagerUtils.getValueFromIdByName(id, "autoProvisioningSettings"); if (settingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'autoProvisioningSettings'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'autoProvisioningSettings'.", id))); } return this.getWithResponse(settingName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String settingName = Utils.getValueFromIdByName(id, "autoProvisioningSettings"); + String settingName = ResourceManagerUtils.getValueFromIdByName(id, "autoProvisioningSettings"); if (settingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'autoProvisioningSettings'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'autoProvisioningSettings'.", id))); } return this.getWithResponse(settingName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationImpl.java index 1fe4c63468b08..28fdf882a3d21 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationImpl.java @@ -12,6 +12,7 @@ import com.azure.resourcemanager.security.models.AutomationAction; import com.azure.resourcemanager.security.models.AutomationScope; import com.azure.resourcemanager.security.models.AutomationSource; +import com.azure.resourcemanager.security.models.AutomationUpdateModel; import com.azure.resourcemanager.security.models.AutomationValidationStatus; import java.util.Collections; import java.util.List; @@ -106,28 +107,22 @@ private com.azure.resourcemanager.security.SecurityManager manager() { private String automationName; + private AutomationUpdateModel updateAutomation; + public AutomationImpl withExistingResourceGroup(String resourceGroupName) { this.resourceGroupName = resourceGroupName; return this; } public Automation create() { - this.innerObject = - serviceManager - .serviceClient() - .getAutomations() - .createOrUpdateWithResponse(resourceGroupName, automationName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutomations() + .createOrUpdateWithResponse(resourceGroupName, automationName, this.innerModel(), Context.NONE).getValue(); return this; } public Automation create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAutomations() - .createOrUpdateWithResponse(resourceGroupName, automationName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutomations() + .createOrUpdateWithResponse(resourceGroupName, automationName, this.innerModel(), context).getValue(); return this; } @@ -138,60 +133,44 @@ public Automation create(Context context) { } public AutomationImpl update() { + this.updateAutomation = new AutomationUpdateModel(); return this; } public Automation apply() { - this.innerObject = - serviceManager - .serviceClient() - .getAutomations() - .createOrUpdateWithResponse(resourceGroupName, automationName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutomations() + .updateWithResponse(resourceGroupName, automationName, updateAutomation, Context.NONE).getValue(); return this; } public Automation apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAutomations() - .createOrUpdateWithResponse(resourceGroupName, automationName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutomations() + .updateWithResponse(resourceGroupName, automationName, updateAutomation, context).getValue(); return this; } AutomationImpl(AutomationInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.automationName = Utils.getValueFromIdByName(innerObject.id(), "automations"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.automationName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "automations"); } public Automation refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getAutomations() - .getByResourceGroupWithResponse(resourceGroupName, automationName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutomations() + .getByResourceGroupWithResponse(resourceGroupName, automationName, Context.NONE).getValue(); return this; } public Automation refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAutomations() - .getByResourceGroupWithResponse(resourceGroupName, automationName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAutomations() + .getByResourceGroupWithResponse(resourceGroupName, automationName, context).getValue(); return this; } public Response validateWithResponse(AutomationInner automation, Context context) { - return serviceManager - .automations() - .validateWithResponse(resourceGroupName, automationName, automation, context); + return serviceManager.automations().validateWithResponse(resourceGroupName, automationName, automation, + context); } public AutomationValidationStatus validate(AutomationInner automation) { @@ -209,32 +188,66 @@ public AutomationImpl withRegion(String location) { } public AutomationImpl withTags(Map tags) { - this.innerModel().withTags(tags); - return this; + if (isInCreateMode()) { + this.innerModel().withTags(tags); + return this; + } else { + this.updateAutomation.withTags(tags); + return this; + } } public AutomationImpl withDescription(String description) { - this.innerModel().withDescription(description); - return this; + if (isInCreateMode()) { + this.innerModel().withDescription(description); + return this; + } else { + this.updateAutomation.withDescription(description); + return this; + } } public AutomationImpl withIsEnabled(Boolean isEnabled) { - this.innerModel().withIsEnabled(isEnabled); - return this; + if (isInCreateMode()) { + this.innerModel().withIsEnabled(isEnabled); + return this; + } else { + this.updateAutomation.withIsEnabled(isEnabled); + return this; + } } public AutomationImpl withScopes(List scopes) { - this.innerModel().withScopes(scopes); - return this; + if (isInCreateMode()) { + this.innerModel().withScopes(scopes); + return this; + } else { + this.updateAutomation.withScopes(scopes); + return this; + } } public AutomationImpl withSources(List sources) { - this.innerModel().withSources(sources); - return this; + if (isInCreateMode()) { + this.innerModel().withSources(sources); + return this; + } else { + this.updateAutomation.withSources(sources); + return this; + } } public AutomationImpl withActions(List actions) { - this.innerModel().withActions(actions); - return this; + if (isInCreateMode()) { + this.innerModel().withActions(actions); + return this; + } else { + this.updateAutomation.withActions(actions); + return this; + } + } + + private boolean isInCreateMode() { + return this.innerModel().id() == null; } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationValidationStatusImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationValidationStatusImpl.java index 5e3c49c7e027c..62257fd6317d7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationValidationStatusImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationValidationStatusImpl.java @@ -12,8 +12,7 @@ public final class AutomationValidationStatusImpl implements AutomationValidatio private final com.azure.resourcemanager.security.SecurityManager serviceManager; - AutomationValidationStatusImpl( - AutomationValidationStatusInner innerObject, + AutomationValidationStatusImpl(AutomationValidationStatusInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsClientImpl.java index 2348e654c1b27..b2701ff23fdc8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsClientImpl.java @@ -12,6 +12,7 @@ import com.azure.core.annotation.Headers; import com.azure.core.annotation.Host; import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; import com.azure.core.annotation.PathParam; import com.azure.core.annotation.Post; import com.azure.core.annotation.Put; @@ -33,24 +34,31 @@ import com.azure.resourcemanager.security.fluent.models.AutomationInner; import com.azure.resourcemanager.security.fluent.models.AutomationValidationStatusInner; import com.azure.resourcemanager.security.models.AutomationList; +import com.azure.resourcemanager.security.models.AutomationUpdateModel; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in AutomationsClient. */ +/** + * An instance of this class provides access to all the operations defined in AutomationsClient. + */ public final class AutomationsClientImpl implements AutomationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final AutomationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of AutomationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ AutomationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(AutomationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(AutomationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -61,196 +69,152 @@ public final class AutomationsClientImpl implements AutomationsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterAutoma") public interface AutomationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/automations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("automationName") String automationName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("automationName") String automationName, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") AutomationInner automation, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("automationName") String automationName, - @BodyParam("application/json") AutomationInner automation, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") AutomationUpdateModel automation, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") - @ExpectedResponses({204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("automationName") String automationName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("automationName") String automationName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}/validate") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/automations/{automationName}/validate") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> validate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> validate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("automationName") String automationName, - @BodyParam("application/json") AutomationInner automation, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") AutomationInner automation, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security automations response as paginated response with {@link PagedFlux}. @@ -263,7 +227,7 @@ private PagedFlux listAsync() { /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -272,14 +236,14 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security automations response as paginated response with {@link PagedIterable}. @@ -292,7 +256,7 @@ public PagedIterable list() { /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -307,118 +271,83 @@ public PagedIterable list(Context context) { /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -426,17 +355,16 @@ private Mono> listByResourceGroupSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -445,17 +373,16 @@ private PagedFlux listByResourceGroupAsync(String resourceGroup */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -469,9 +396,9 @@ public PagedIterable listByResourceGroup(String resourceGroupNa /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -485,9 +412,9 @@ public PagedIterable listByResourceGroup(String resourceGroupNa /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -495,19 +422,15 @@ public PagedIterable listByResourceGroup(String resourceGroupNa * @return the security automation resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String automationName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String automationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -516,28 +439,19 @@ private Mono> getByResourceGroupWithResponseAsync( if (automationName == null) { return Mono.error(new IllegalArgumentException("Parameter automationName is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, automationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -546,19 +460,15 @@ private Mono> getByResourceGroupWithResponseAsync( * @return the security automation resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String automationName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String automationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -567,25 +477,18 @@ private Mono> getByResourceGroupWithResponseAsync( if (automationName == null) { return Mono.error(new IllegalArgumentException("Parameter automationName is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, automationName, accept, context); } /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -600,9 +503,9 @@ private Mono getByResourceGroupAsync(String resourceGroupName, /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -611,16 +514,16 @@ private Mono getByResourceGroupAsync(String resourceGroupName, * @return the security automation resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String automationName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, String automationName, + Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, automationName, context).block(); } /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -635,9 +538,9 @@ public AutomationInner getByResourceGroup(String resourceGroupName, String autom /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -646,19 +549,15 @@ public AutomationInner getByResourceGroup(String resourceGroupName, String autom * @return the security automation resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String automationName, AutomationInner automation) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String automationName, AutomationInner automation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -672,30 +571,20 @@ private Mono> createOrUpdateWithResponseAsync( } else { automation.validate(); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - automation, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, automationName, automation, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -705,19 +594,15 @@ private Mono> createOrUpdateWithResponseAsync( * @return the security automation resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String automationName, AutomationInner automation, Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String automationName, AutomationInner automation, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -731,27 +616,19 @@ private Mono> createOrUpdateWithResponseAsync( } else { automation.validate(); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - automation, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, automationName, automation, accept, context); } /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -760,8 +637,8 @@ private Mono> createOrUpdateWithResponseAsync( * @return the security automation resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String automationName, AutomationInner automation) { + private Mono createOrUpdateAsync(String resourceGroupName, String automationName, + AutomationInner automation) { return createOrUpdateWithResponseAsync(resourceGroupName, automationName, automation) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -769,9 +646,9 @@ private Mono createOrUpdateAsync( /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -781,17 +658,17 @@ private Mono createOrUpdateAsync( * @return the security automation resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, String automationName, AutomationInner automation, Context context) { + public Response createOrUpdateWithResponse(String resourceGroupName, String automationName, + AutomationInner automation, Context context) { return createOrUpdateWithResponseAsync(resourceGroupName, automationName, automation, context).block(); } /** * Creates or updates a security automation. If a security automation is already created and a subsequent request is * issued for the same automation id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -804,11 +681,152 @@ public AutomationInner createOrUpdate(String resourceGroupName, String automatio return createOrUpdateWithResponse(resourceGroupName, automationName, automation, Context.NONE).getValue(); } + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> updateWithResponseAsync(String resourceGroupName, String automationName, + AutomationUpdateModel automation) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (automationName == null) { + return Mono.error(new IllegalArgumentException("Parameter automationName is required and cannot be null.")); + } + if (automation == null) { + return Mono.error(new IllegalArgumentException("Parameter automation is required and cannot be null.")); + } else { + automation.validate(); + } + final String apiVersion = "2023-12-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, automationName, automation, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> updateWithResponseAsync(String resourceGroupName, String automationName, + AutomationUpdateModel automation, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (automationName == null) { + return Mono.error(new IllegalArgumentException("Parameter automationName is required and cannot be null.")); + } + if (automation == null) { + return Mono.error(new IllegalArgumentException("Parameter automation is required and cannot be null.")); + } else { + automation.validate(); + } + final String apiVersion = "2023-12-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + automationName, automation, accept, context); + } + + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String automationName, + AutomationUpdateModel automation) { + return updateWithResponseAsync(resourceGroupName, automationName, automation) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response updateWithResponse(String resourceGroupName, String automationName, + AutomationUpdateModel automation, Context context) { + return updateWithResponseAsync(resourceGroupName, automationName, automation, context).block(); + } + + /** + * Updates a security automation. + * + * @param resourceGroupName The name of the resource group within the user's subscription. The name is case + * insensitive. + * @param automationName The security automation name. + * @param automation The update model of security automation resource. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the security automation resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AutomationInner update(String resourceGroupName, String automationName, AutomationUpdateModel automation) { + return updateWithResponse(resourceGroupName, automationName, automation, Context.NONE).getValue(); + } + /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -818,16 +836,12 @@ public AutomationInner createOrUpdate(String resourceGroupName, String automatio @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceGroupName, String automationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -836,28 +850,19 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S if (automationName == null) { return Mono.error(new IllegalArgumentException("Parameter automationName is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, automationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -866,19 +871,15 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String automationName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String automationName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -887,25 +888,18 @@ private Mono> deleteWithResponseAsync( if (automationName == null) { return Mono.error(new IllegalArgumentException("Parameter automationName is required and cannot be null.")); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + automationName, accept, context); } /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -919,9 +913,9 @@ private Mono deleteAsync(String resourceGroupName, String automationName) /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -936,9 +930,9 @@ public Response deleteWithResponse(String resourceGroupName, String automa /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -952,31 +946,27 @@ public void delete(String resourceGroupName, String automationName) { /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the security automation model state property bag along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> validateWithResponseAsync( - String resourceGroupName, String automationName, AutomationInner automation) { + private Mono> validateWithResponseAsync(String resourceGroupName, + String automationName, AutomationInner automation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -990,30 +980,20 @@ private Mono> validateWithResponseAsyn } else { automation.validate(); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .validate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - automation, - accept, - context)) + .withContext(context -> service.validate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, automationName, automation, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -1021,22 +1001,18 @@ private Mono> validateWithResponseAsyn * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the security automation model state property bag along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> validateWithResponseAsync( - String resourceGroupName, String automationName, AutomationInner automation, Context context) { + private Mono> validateWithResponseAsync(String resourceGroupName, + String automationName, AutomationInner automation, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1050,27 +1026,19 @@ private Mono> validateWithResponseAsyn } else { automation.validate(); } - final String apiVersion = "2019-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .validate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - automationName, - automation, - accept, - context); + return service.validate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, automationName, automation, accept, context); } /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1079,8 +1047,8 @@ private Mono> validateWithResponseAsyn * @return the security automation model state property bag on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono validateAsync( - String resourceGroupName, String automationName, AutomationInner automation) { + private Mono validateAsync(String resourceGroupName, String automationName, + AutomationInner automation) { return validateWithResponseAsync(resourceGroupName, automationName, automation) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -1088,9 +1056,9 @@ private Mono validateAsync( /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -1100,17 +1068,17 @@ private Mono validateAsync( * @return the security automation model state property bag along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response validateWithResponse( - String resourceGroupName, String automationName, AutomationInner automation, Context context) { + public Response validateWithResponse(String resourceGroupName, + String automationName, AutomationInner automation, Context context) { return validateWithResponseAsync(resourceGroupName, automationName, automation, context).block(); } /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1119,21 +1087,22 @@ public Response validateWithResponse( * @return the security automation model state property bag. */ @ServiceMethod(returns = ReturnType.SINGLE) - public AutomationValidationStatusInner validate( - String resourceGroupName, String automationName, AutomationInner automation) { + public AutomationValidationStatusInner validate(String resourceGroupName, String automationName, + AutomationInner automation) { return validateWithResponse(resourceGroupName, automationName, automation, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1141,37 +1110,28 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1179,36 +1139,27 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1216,63 +1167,45 @@ private Mono> listByResourceGroupNextSinglePageAs return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security automations response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security automations response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsImpl.java index b0ab5eaeb2e0c..e76dfb3dd1cb0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AutomationsImpl.java @@ -23,41 +23,38 @@ public final class AutomationsImpl implements Automations { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public AutomationsImpl( - AutomationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public AutomationsImpl(AutomationsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AutomationImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String automationName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, automationName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, String automationName, + Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, automationName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AutomationImpl(inner.getValue(), this.manager())); } else { return null; @@ -73,8 +70,8 @@ public Automation getByResourceGroup(String resourceGroupName, String automation } } - public Response deleteByResourceGroupWithResponse( - String resourceGroupName, String automationName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceGroupName, String automationName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, automationName, context); } @@ -82,25 +79,22 @@ public void deleteByResourceGroup(String resourceGroupName, String automationNam this.serviceClient().delete(resourceGroupName, automationName); } - public Response validateWithResponse( - String resourceGroupName, String automationName, AutomationInner automation, Context context) { - Response inner = - this.serviceClient().validateWithResponse(resourceGroupName, automationName, automation, context); + public Response validateWithResponse(String resourceGroupName, String automationName, + AutomationInner automation, Context context) { + Response inner + = this.serviceClient().validateWithResponse(resourceGroupName, automationName, automation, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AutomationValidationStatusImpl(inner.getValue(), this.manager())); } else { return null; } } - public AutomationValidationStatus validate( - String resourceGroupName, String automationName, AutomationInner automation) { - AutomationValidationStatusInner inner = - this.serviceClient().validate(resourceGroupName, automationName, automation); + public AutomationValidationStatus validate(String resourceGroupName, String automationName, + AutomationInner automation) { + AutomationValidationStatusInner inner + = this.serviceClient().validate(resourceGroupName, automationName, automation); if (inner != null) { return new AutomationValidationStatusImpl(inner, this.manager()); } else { @@ -109,77 +103,57 @@ public AutomationValidationStatus validate( } public Automation getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String automationName = Utils.getValueFromIdByName(id, "automations"); + String automationName = ResourceManagerUtils.getValueFromIdByName(id, "automations"); if (automationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, automationName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String automationName = Utils.getValueFromIdByName(id, "automations"); + String automationName = ResourceManagerUtils.getValueFromIdByName(id, "automations"); if (automationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, automationName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String automationName = Utils.getValueFromIdByName(id, "automations"); + String automationName = ResourceManagerUtils.getValueFromIdByName(id, "automations"); if (automationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); } this.deleteByResourceGroupWithResponse(resourceGroupName, automationName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String automationName = Utils.getValueFromIdByName(id, "automations"); + String automationName = ResourceManagerUtils.getValueFromIdByName(id, "automations"); if (automationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'automations'.", id))); } return this.deleteByResourceGroupWithResponse(resourceGroupName, automationName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgImpl.java new file mode 100644 index 0000000000000..5a09e1d2d7884 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgImpl.java @@ -0,0 +1,121 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner; +import com.azure.resourcemanager.security.models.AzureDevOpsOrg; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgProperties; + +public final class AzureDevOpsOrgImpl implements AzureDevOpsOrg, AzureDevOpsOrg.Definition, AzureDevOpsOrg.Update { + private AzureDevOpsOrgInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public AzureDevOpsOrgProperties properties() { + return this.innerModel().properties(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public AzureDevOpsOrgInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String securityConnectorName; + + private String orgName; + + public AzureDevOpsOrgImpl withExistingSecurityConnector(String resourceGroupName, String securityConnectorName) { + this.resourceGroupName = resourceGroupName; + this.securityConnectorName = securityConnectorName; + return this; + } + + public AzureDevOpsOrg create() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsOrgs().createOrUpdate(resourceGroupName, + securityConnectorName, orgName, this.innerModel(), Context.NONE); + return this; + } + + public AzureDevOpsOrg create(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsOrgs().createOrUpdate(resourceGroupName, + securityConnectorName, orgName, this.innerModel(), context); + return this; + } + + AzureDevOpsOrgImpl(String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = new AzureDevOpsOrgInner(); + this.serviceManager = serviceManager; + this.orgName = name; + } + + public AzureDevOpsOrgImpl update() { + return this; + } + + public AzureDevOpsOrg apply() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsOrgs().update(resourceGroupName, + securityConnectorName, orgName, this.innerModel(), Context.NONE); + return this; + } + + public AzureDevOpsOrg apply(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsOrgs().update(resourceGroupName, + securityConnectorName, orgName, this.innerModel(), context); + return this; + } + + AzureDevOpsOrgImpl(AzureDevOpsOrgInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.securityConnectorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "securityConnectors"); + this.orgName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "azureDevOpsOrgs"); + } + + public AzureDevOpsOrg refresh() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsOrgs() + .getWithResponse(resourceGroupName, securityConnectorName, orgName, Context.NONE).getValue(); + return this; + } + + public AzureDevOpsOrg refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsOrgs() + .getWithResponse(resourceGroupName, securityConnectorName, orgName, context).getValue(); + return this; + } + + public AzureDevOpsOrgImpl withProperties(AzureDevOpsOrgProperties properties) { + this.innerModel().withProperties(properties); + return this; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgListResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgListResponseImpl.java new file mode 100644 index 0000000000000..300da645bf6b1 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgListResponseImpl.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgListResponseInner; +import com.azure.resourcemanager.security.models.AzureDevOpsOrg; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgListResponse; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public final class AzureDevOpsOrgListResponseImpl implements AzureDevOpsOrgListResponse { + private AzureDevOpsOrgListResponseInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + AzureDevOpsOrgListResponseImpl(AzureDevOpsOrgListResponseInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public List value() { + List inner = this.innerModel().value(); + if (inner != null) { + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new AzureDevOpsOrgImpl(inner1, this.manager())).collect(Collectors.toList())); + } else { + return Collections.emptyList(); + } + } + + public String nextLink() { + return this.innerModel().nextLink(); + } + + public AzureDevOpsOrgListResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsClientImpl.java new file mode 100644 index 0000000000000..05e97549792a6 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsClientImpl.java @@ -0,0 +1,1105 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.AzureDevOpsOrgsClient; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgListResponseInner; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in AzureDevOpsOrgsClient. + */ +public final class AzureDevOpsOrgsClientImpl implements AzureDevOpsOrgsClient { + /** + * The proxy service used to perform REST calls. + */ + private final AzureDevOpsOrgsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of AzureDevOpsOrgsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + AzureDevOpsOrgsClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(AzureDevOpsOrgsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterAzureDevOpsOrgs to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterAzureD") + public interface AzureDevOpsOrgsService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/listAvailableAzureDevOpsOrgs") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listAvailable(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AzureDevOpsOrgInner azureDevOpsOrg, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AzureDevOpsOrgInner azureDevOpsOrg, @HeaderParam("Accept") String accept, + Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listAvailableWithResponseAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listAvailable(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listAvailableWithResponseAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listAvailable(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, accept, context); + } + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAvailableAsync(String resourceGroupName, + String securityConnectorName) { + return listAvailableWithResponseAsync(resourceGroupName, securityConnectorName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + return listAvailableWithResponseAsync(resourceGroupName, securityConnectorName, context).block(); + } + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsOrgListResponseInner listAvailable(String resourceGroupName, String securityConnectorName) { + return listAvailableWithResponse(resourceGroupName, securityConnectorName, Context.NONE).getValue(); + } + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName)); + } + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, context)); + } + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, apiVersion, accept, context); + } + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, String orgName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, orgName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, orgName, context).block(); + } + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsOrgInner get(String resourceGroupName, String securityConnectorName, String orgName) { + return getWithResponse(resourceGroupName, securityConnectorName, orgName, Context.NONE).getValue(); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (azureDevOpsOrg == null) { + return Mono.error(new IllegalArgumentException("Parameter azureDevOpsOrg is required and cannot be null.")); + } else { + azureDevOpsOrg.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, apiVersion, azureDevOpsOrg, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (azureDevOpsOrg == null) { + return Mono.error(new IllegalArgumentException("Parameter azureDevOpsOrg is required and cannot be null.")); + } else { + azureDevOpsOrg.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, apiVersion, azureDevOpsOrg, accept, context); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsOrgInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + AzureDevOpsOrgInner.class, AzureDevOpsOrgInner.class, this.client.getContext()); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsOrgInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + securityConnectorName, orgName, azureDevOpsOrg, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + AzureDevOpsOrgInner.class, AzureDevOpsOrgInner.class, context); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsOrgInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg) + .getSyncPoller(); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsOrgInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, + Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context) + .getSyncPoller(); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context) + .last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsOrgInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg).block(); + } + + /** + * Creates or updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsOrgInner createOrUpdate(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context).block(); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (azureDevOpsOrg == null) { + return Mono.error(new IllegalArgumentException("Parameter azureDevOpsOrg is required and cannot be null.")); + } else { + azureDevOpsOrg.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, apiVersion, azureDevOpsOrg, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (azureDevOpsOrg == null) { + return Mono.error(new IllegalArgumentException("Parameter azureDevOpsOrg is required and cannot be null.")); + } else { + azureDevOpsOrg.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, apiVersion, azureDevOpsOrg, accept, context); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsOrgInner> beginUpdateAsync(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + AzureDevOpsOrgInner.class, AzureDevOpsOrgInner.class, this.client.getContext()); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsOrgInner> beginUpdateAsync(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), + AzureDevOpsOrgInner.class, AzureDevOpsOrgInner.class, context); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsOrgInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg).getSyncPoller(); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsOrgInner> beginUpdate(String resourceGroupName, + String securityConnectorName, String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context) + .getSyncPoller(); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + String orgName, AzureDevOpsOrgInner azureDevOpsOrg) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + String orgName, AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsOrgInner update(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg) { + return updateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg).block(); + } + + /** + * Updates monitored Azure DevOps organization details. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param azureDevOpsOrg The Azure DevOps organization resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsOrgInner update(String resourceGroupName, String securityConnectorName, String orgName, + AzureDevOpsOrgInner azureDevOpsOrg, Context context) { + return updateAsync(resourceGroupName, securityConnectorName, orgName, azureDevOpsOrg, context).block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsImpl.java new file mode 100644 index 0000000000000..a51d49dabb6eb --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsOrgsImpl.java @@ -0,0 +1,135 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.AzureDevOpsOrgsClient; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgListResponseInner; +import com.azure.resourcemanager.security.models.AzureDevOpsOrg; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgListResponse; +import com.azure.resourcemanager.security.models.AzureDevOpsOrgs; + +public final class AzureDevOpsOrgsImpl implements AzureDevOpsOrgs { + private static final ClientLogger LOGGER = new ClientLogger(AzureDevOpsOrgsImpl.class); + + private final AzureDevOpsOrgsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public AzureDevOpsOrgsImpl(AzureDevOpsOrgsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + Response inner + = this.serviceClient().listAvailableWithResponse(resourceGroupName, securityConnectorName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new AzureDevOpsOrgListResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public AzureDevOpsOrgListResponse listAvailable(String resourceGroupName, String securityConnectorName) { + AzureDevOpsOrgListResponseInner inner + = this.serviceClient().listAvailable(resourceGroupName, securityConnectorName); + if (inner != null) { + return new AzureDevOpsOrgListResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, securityConnectorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AzureDevOpsOrgImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AzureDevOpsOrgImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, securityConnectorName, orgName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new AzureDevOpsOrgImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public AzureDevOpsOrg get(String resourceGroupName, String securityConnectorName, String orgName) { + AzureDevOpsOrgInner inner = this.serviceClient().get(resourceGroupName, securityConnectorName, orgName); + if (inner != null) { + return new AzureDevOpsOrgImpl(inner, this.manager()); + } else { + return null; + } + } + + public AzureDevOpsOrg getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); + if (securityConnectorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + } + String orgName = ResourceManagerUtils.getValueFromIdByName(id, "azureDevOpsOrgs"); + if (orgName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'azureDevOpsOrgs'.", id))); + } + return this.getWithResponse(resourceGroupName, securityConnectorName, orgName, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); + if (securityConnectorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + } + String orgName = ResourceManagerUtils.getValueFromIdByName(id, "azureDevOpsOrgs"); + if (orgName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'azureDevOpsOrgs'.", id))); + } + return this.getWithResponse(resourceGroupName, securityConnectorName, orgName, context); + } + + private AzureDevOpsOrgsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + public AzureDevOpsOrgImpl define(String name) { + return new AzureDevOpsOrgImpl(name, this.manager()); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectImpl.java new file mode 100644 index 0000000000000..140d8d0d8286e --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectImpl.java @@ -0,0 +1,127 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner; +import com.azure.resourcemanager.security.models.AzureDevOpsProject; +import com.azure.resourcemanager.security.models.AzureDevOpsProjectProperties; + +public final class AzureDevOpsProjectImpl + implements AzureDevOpsProject, AzureDevOpsProject.Definition, AzureDevOpsProject.Update { + private AzureDevOpsProjectInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public AzureDevOpsProjectProperties properties() { + return this.innerModel().properties(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public AzureDevOpsProjectInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String securityConnectorName; + + private String orgName; + + private String projectName; + + public AzureDevOpsProjectImpl withExistingAzureDevOpsOrg(String resourceGroupName, String securityConnectorName, + String orgName) { + this.resourceGroupName = resourceGroupName; + this.securityConnectorName = securityConnectorName; + this.orgName = orgName; + return this; + } + + public AzureDevOpsProject create() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsProjects().createOrUpdate(resourceGroupName, + securityConnectorName, orgName, projectName, this.innerModel(), Context.NONE); + return this; + } + + public AzureDevOpsProject create(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsProjects().createOrUpdate(resourceGroupName, + securityConnectorName, orgName, projectName, this.innerModel(), context); + return this; + } + + AzureDevOpsProjectImpl(String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = new AzureDevOpsProjectInner(); + this.serviceManager = serviceManager; + this.projectName = name; + } + + public AzureDevOpsProjectImpl update() { + return this; + } + + public AzureDevOpsProject apply() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsProjects().update(resourceGroupName, + securityConnectorName, orgName, projectName, this.innerModel(), Context.NONE); + return this; + } + + public AzureDevOpsProject apply(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsProjects().update(resourceGroupName, + securityConnectorName, orgName, projectName, this.innerModel(), context); + return this; + } + + AzureDevOpsProjectImpl(AzureDevOpsProjectInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.securityConnectorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "securityConnectors"); + this.orgName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "azureDevOpsOrgs"); + this.projectName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "projects"); + } + + public AzureDevOpsProject refresh() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsProjects() + .getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, Context.NONE).getValue(); + return this; + } + + public AzureDevOpsProject refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsProjects() + .getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, context).getValue(); + return this; + } + + public AzureDevOpsProjectImpl withProperties(AzureDevOpsProjectProperties properties) { + this.innerModel().withProperties(properties); + return this; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsClientImpl.java new file mode 100644 index 0000000000000..0dda863a85d79 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsClientImpl.java @@ -0,0 +1,1049 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.AzureDevOpsProjectsClient; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner; +import com.azure.resourcemanager.security.models.AzureDevOpsProjectListResponse; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in AzureDevOpsProjectsClient. + */ +public final class AzureDevOpsProjectsClientImpl implements AzureDevOpsProjectsClient { + /** + * The proxy service used to perform REST calls. + */ + private final AzureDevOpsProjectsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of AzureDevOpsProjectsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + AzureDevOpsProjectsClientImpl(SecurityCenterImpl client) { + this.service = RestProxy.create(AzureDevOpsProjectsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterAzureDevOpsProjects to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterAzureD") + public interface AzureDevOpsProjectsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AzureDevOpsProjectInner azureDevOpsProject, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AzureDevOpsProjectInner azureDevOpsProject, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String orgName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String orgName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + orgName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String orgName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, orgName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String orgName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, orgName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, orgName)); + } + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, orgName, context)); + } + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, projectName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, apiVersion, accept, context); + } + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, orgName, projectName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, orgName, projectName, context).block(); + } + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsProjectInner get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName) { + return getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, Context.NONE).getValue(); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (azureDevOpsProject == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsProject is required and cannot be null.")); + } else { + azureDevOpsProject.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, projectName, apiVersion, azureDevOpsProject, accept, + context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (azureDevOpsProject == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsProject is required and cannot be null.")); + } else { + azureDevOpsProject.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, apiVersion, azureDevOpsProject, accept, context); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsProjectInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject) { + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + securityConnectorName, orgName, projectName, azureDevOpsProject); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsProjectInner.class, AzureDevOpsProjectInner.class, + this.client.getContext()); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsProjectInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject, Context context) { + context = this.client.mergeContext(context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + securityConnectorName, orgName, projectName, azureDevOpsProject, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsProjectInner.class, AzureDevOpsProjectInner.class, context); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsProjectInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, + azureDevOpsProject).getSyncPoller(); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsProjectInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, + azureDevOpsProject, context).getSyncPoller(); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, + azureDevOpsProject).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, + azureDevOpsProject, context).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsProjectInner createOrUpdate(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject) + .block(); + } + + /** + * Creates or updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsProjectInner createOrUpdate(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject, Context context) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject, + context).block(); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (azureDevOpsProject == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsProject is required and cannot be null.")); + } else { + azureDevOpsProject.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, apiVersion, azureDevOpsProject, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (azureDevOpsProject == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsProject is required and cannot be null.")); + } else { + azureDevOpsProject.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, apiVersion, azureDevOpsProject, accept, context); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsProjectInner> beginUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject) { + Mono>> mono = updateWithResponseAsync(resourceGroupName, securityConnectorName, + orgName, projectName, azureDevOpsProject); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsProjectInner.class, AzureDevOpsProjectInner.class, + this.client.getContext()); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsProjectInner> beginUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject, Context context) { + context = this.client.mergeContext(context); + Mono>> mono = updateWithResponseAsync(resourceGroupName, securityConnectorName, + orgName, projectName, azureDevOpsProject, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsProjectInner.class, AzureDevOpsProjectInner.class, context); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsProjectInner> beginUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject) + .getSyncPoller(); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsProjectInner> beginUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, + AzureDevOpsProjectInner azureDevOpsProject, Context context) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject, + context).getSyncPoller(); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject) + .last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, AzureDevOpsProjectInner azureDevOpsProject, Context context) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject, + context).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsProjectInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, AzureDevOpsProjectInner azureDevOpsProject) { + return updateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject).block(); + } + + /** + * Updates a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param azureDevOpsProject The Azure DevOps project resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsProjectInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, AzureDevOpsProjectInner azureDevOpsProject, Context context) { + return updateAsync(resourceGroupName, securityConnectorName, orgName, projectName, azureDevOpsProject, context) + .block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsImpl.java new file mode 100644 index 0000000000000..444c613d071cc --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsProjectsImpl.java @@ -0,0 +1,127 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.AzureDevOpsProjectsClient; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner; +import com.azure.resourcemanager.security.models.AzureDevOpsProject; +import com.azure.resourcemanager.security.models.AzureDevOpsProjects; + +public final class AzureDevOpsProjectsImpl implements AzureDevOpsProjects { + private static final ClientLogger LOGGER = new ClientLogger(AzureDevOpsProjectsImpl.class); + + private final AzureDevOpsProjectsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public AzureDevOpsProjectsImpl(AzureDevOpsProjectsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, orgName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AzureDevOpsProjectImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, orgName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AzureDevOpsProjectImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + securityConnectorName, orgName, projectName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new AzureDevOpsProjectImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public AzureDevOpsProject get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName) { + AzureDevOpsProjectInner inner + = this.serviceClient().get(resourceGroupName, securityConnectorName, orgName, projectName); + if (inner != null) { + return new AzureDevOpsProjectImpl(inner, this.manager()); + } else { + return null; + } + } + + public AzureDevOpsProject getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); + if (securityConnectorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + } + String orgName = ResourceManagerUtils.getValueFromIdByName(id, "azureDevOpsOrgs"); + if (orgName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'azureDevOpsOrgs'.", id))); + } + String projectName = ResourceManagerUtils.getValueFromIdByName(id, "projects"); + if (projectName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'projects'.", id))); + } + return this.getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, Context.NONE) + .getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); + if (securityConnectorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + } + String orgName = ResourceManagerUtils.getValueFromIdByName(id, "azureDevOpsOrgs"); + if (orgName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'azureDevOpsOrgs'.", id))); + } + String projectName = ResourceManagerUtils.getValueFromIdByName(id, "projects"); + if (projectName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'projects'.", id))); + } + return this.getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, context); + } + + private AzureDevOpsProjectsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + public AzureDevOpsProjectImpl define(String name) { + return new AzureDevOpsProjectImpl(name, this.manager()); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposClientImpl.java new file mode 100644 index 0000000000000..7b363074e298e --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposClientImpl.java @@ -0,0 +1,1117 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.AzureDevOpsReposClient; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner; +import com.azure.resourcemanager.security.models.AzureDevOpsRepositoryListResponse; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in AzureDevOpsReposClient. + */ +public final class AzureDevOpsReposClientImpl implements AzureDevOpsReposClient { + /** + * The proxy service used to perform REST calls. + */ + private final AzureDevOpsReposService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of AzureDevOpsReposClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + AzureDevOpsReposClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(AzureDevOpsReposService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterAzureDevOpsRepos to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterAzureD") + public interface AzureDevOpsReposService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}/repos") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}/repos/{repoName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @PathParam("repoName") String repoName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}/repos/{repoName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @PathParam("repoName") String repoName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AzureDevOpsRepositoryInner azureDevOpsRepository, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/azureDevOpsOrgs/{orgName}/projects/{projectName}/repos/{repoName}") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("orgName") String orgName, + @PathParam("projectName") String projectName, @PathParam("repoName") String repoName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") AzureDevOpsRepositoryInner azureDevOpsRepository, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, projectName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + orgName, projectName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName) { + return new PagedFlux<>( + () -> listSinglePageAsync(resourceGroupName, securityConnectorName, orgName, projectName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context) { + return new PagedFlux<>( + () -> listSinglePageAsync(resourceGroupName, securityConnectorName, orgName, projectName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, String projectName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, orgName, projectName)); + } + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, orgName, projectName, context)); + } + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, projectName, repoName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, repoName, apiVersion, accept, context); + } + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, context) + .block(); + } + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsRepositoryInner get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName) { + return getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, repoName, Context.NONE) + .getValue(); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + if (azureDevOpsRepository == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsRepository is required and cannot be null.")); + } else { + azureDevOpsRepository.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, projectName, repoName, apiVersion, + azureDevOpsRepository, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + if (azureDevOpsRepository == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsRepository is required and cannot be null.")); + } else { + azureDevOpsRepository.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, repoName, apiVersion, azureDevOpsRepository, accept, context); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsRepositoryInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository) { + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, azureDevOpsRepository); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsRepositoryInner.class, AzureDevOpsRepositoryInner.class, + this.client.getContext()); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsRepositoryInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + context = this.client.mergeContext(context); + Mono>> mono = createOrUpdateWithResponseAsync(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, azureDevOpsRepository, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsRepositoryInner.class, AzureDevOpsRepositoryInner.class, context); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsRepositoryInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository).getSyncPoller(); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsRepositoryInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository, context).getSyncPoller(); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository, + Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository, context).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsRepositoryInner createOrUpdate(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository).block(); + } + + /** + * Creates or updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsRepositoryInner createOrUpdate(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository, + Context context) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository, context).block(); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + if (azureDevOpsRepository == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsRepository is required and cannot be null.")); + } else { + azureDevOpsRepository.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, orgName, projectName, repoName, apiVersion, + azureDevOpsRepository, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (orgName == null) { + return Mono.error(new IllegalArgumentException("Parameter orgName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + if (azureDevOpsRepository == null) { + return Mono + .error(new IllegalArgumentException("Parameter azureDevOpsRepository is required and cannot be null.")); + } else { + azureDevOpsRepository.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, orgName, projectName, repoName, apiVersion, azureDevOpsRepository, accept, context); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsRepositoryInner> beginUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository) { + Mono>> mono = updateWithResponseAsync(resourceGroupName, securityConnectorName, + orgName, projectName, repoName, azureDevOpsRepository); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsRepositoryInner.class, AzureDevOpsRepositoryInner.class, + this.client.getContext()); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, AzureDevOpsRepositoryInner> beginUpdateAsync( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + context = this.client.mergeContext(context); + Mono>> mono = updateWithResponseAsync(resourceGroupName, securityConnectorName, + orgName, projectName, repoName, azureDevOpsRepository, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), AzureDevOpsRepositoryInner.class, AzureDevOpsRepositoryInner.class, context); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsRepositoryInner> beginUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository).getSyncPoller(); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, AzureDevOpsRepositoryInner> beginUpdate( + String resourceGroupName, String securityConnectorName, String orgName, String projectName, String repoName, + AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository, context).getSyncPoller(); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository, + Context context) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository, context).last().flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsRepositoryInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository) { + return updateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository).block(); + } + + /** + * Updates a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param azureDevOpsRepository The Azure DevOps repository resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public AzureDevOpsRepositoryInner update(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName, AzureDevOpsRepositoryInner azureDevOpsRepository, Context context) { + return updateAsync(resourceGroupName, securityConnectorName, orgName, projectName, repoName, + azureDevOpsRepository, context).block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposImpl.java new file mode 100644 index 0000000000000..53627f6d38224 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsReposImpl.java @@ -0,0 +1,138 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.AzureDevOpsReposClient; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner; +import com.azure.resourcemanager.security.models.AzureDevOpsRepos; +import com.azure.resourcemanager.security.models.AzureDevOpsRepository; + +public final class AzureDevOpsReposImpl implements AzureDevOpsRepos { + private static final ClientLogger LOGGER = new ClientLogger(AzureDevOpsReposImpl.class); + + private final AzureDevOpsReposClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public AzureDevOpsReposImpl(AzureDevOpsReposClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, String projectName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, orgName, projectName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AzureDevOpsRepositoryImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, orgName, projectName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AzureDevOpsRepositoryImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new AzureDevOpsRepositoryImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public AzureDevOpsRepository get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName) { + AzureDevOpsRepositoryInner inner + = this.serviceClient().get(resourceGroupName, securityConnectorName, orgName, projectName, repoName); + if (inner != null) { + return new AzureDevOpsRepositoryImpl(inner, this.manager()); + } else { + return null; + } + } + + public AzureDevOpsRepository getById(String id) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); + if (securityConnectorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + } + String orgName = ResourceManagerUtils.getValueFromIdByName(id, "azureDevOpsOrgs"); + if (orgName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'azureDevOpsOrgs'.", id))); + } + String projectName = ResourceManagerUtils.getValueFromIdByName(id, "projects"); + if (projectName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'projects'.", id))); + } + String repoName = ResourceManagerUtils.getValueFromIdByName(id, "repos"); + if (repoName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'repos'.", id))); + } + return this + .getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, repoName, Context.NONE) + .getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); + if (resourceGroupName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + } + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); + if (securityConnectorName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + } + String orgName = ResourceManagerUtils.getValueFromIdByName(id, "azureDevOpsOrgs"); + if (orgName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'azureDevOpsOrgs'.", id))); + } + String projectName = ResourceManagerUtils.getValueFromIdByName(id, "projects"); + if (projectName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'projects'.", id))); + } + String repoName = ResourceManagerUtils.getValueFromIdByName(id, "repos"); + if (repoName == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'repos'.", id))); + } + return this.getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, repoName, context); + } + + private AzureDevOpsReposClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + public AzureDevOpsRepositoryImpl define(String name) { + return new AzureDevOpsRepositoryImpl(name, this.manager()); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsRepositoryImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsRepositoryImpl.java new file mode 100644 index 0000000000000..752131e4ded17 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/AzureDevOpsRepositoryImpl.java @@ -0,0 +1,133 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner; +import com.azure.resourcemanager.security.models.AzureDevOpsRepository; +import com.azure.resourcemanager.security.models.AzureDevOpsRepositoryProperties; + +public final class AzureDevOpsRepositoryImpl + implements AzureDevOpsRepository, AzureDevOpsRepository.Definition, AzureDevOpsRepository.Update { + private AzureDevOpsRepositoryInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public AzureDevOpsRepositoryProperties properties() { + return this.innerModel().properties(); + } + + public String resourceGroupName() { + return resourceGroupName; + } + + public AzureDevOpsRepositoryInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + private String resourceGroupName; + + private String securityConnectorName; + + private String orgName; + + private String projectName; + + private String repoName; + + public AzureDevOpsRepositoryImpl withExistingProject(String resourceGroupName, String securityConnectorName, + String orgName, String projectName) { + this.resourceGroupName = resourceGroupName; + this.securityConnectorName = securityConnectorName; + this.orgName = orgName; + this.projectName = projectName; + return this; + } + + public AzureDevOpsRepository create() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsRepos().createOrUpdate(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, this.innerModel(), Context.NONE); + return this; + } + + public AzureDevOpsRepository create(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsRepos().createOrUpdate(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, this.innerModel(), context); + return this; + } + + AzureDevOpsRepositoryImpl(String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = new AzureDevOpsRepositoryInner(); + this.serviceManager = serviceManager; + this.repoName = name; + } + + public AzureDevOpsRepositoryImpl update() { + return this; + } + + public AzureDevOpsRepository apply() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsRepos().update(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, this.innerModel(), Context.NONE); + return this; + } + + public AzureDevOpsRepository apply(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsRepos().update(resourceGroupName, + securityConnectorName, orgName, projectName, repoName, this.innerModel(), context); + return this; + } + + AzureDevOpsRepositoryImpl(AzureDevOpsRepositoryInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.securityConnectorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "securityConnectors"); + this.orgName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "azureDevOpsOrgs"); + this.projectName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "projects"); + this.repoName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "repos"); + } + + public AzureDevOpsRepository refresh() { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsRepos() + .getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, repoName, Context.NONE) + .getValue(); + return this; + } + + public AzureDevOpsRepository refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getAzureDevOpsRepos() + .getWithResponse(resourceGroupName, securityConnectorName, orgName, projectName, repoName, context) + .getValue(); + return this; + } + + public AzureDevOpsRepositoryImpl withProperties(AzureDevOpsRepositoryProperties properties) { + this.innerModel().withProperties(properties); + return this; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultImpl.java index dd6a7518892a3..1d8255952a275 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultImpl.java @@ -13,8 +13,8 @@ public final class ComplianceResultImpl implements ComplianceResult { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - ComplianceResultImpl( - ComplianceResultInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + ComplianceResultImpl(ComplianceResultInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsClientImpl.java index 94eef1b3a1f7c..da654a6b037da 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.ComplianceResultList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ComplianceResultsClient. */ +/** + * An instance of this class provides access to all the operations defined in ComplianceResultsClient. + */ public final class ComplianceResultsClientImpl implements ComplianceResultsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ComplianceResultsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ComplianceResultsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ComplianceResultsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(ComplianceResultsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ComplianceResultsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,58 +62,48 @@ public final class ComplianceResultsClientImpl implements ComplianceResultsClien @Host("{$host}") @ServiceInterface(name = "SecurityCenterCompli") public interface ComplianceResultsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/complianceResults") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/complianceResults/{complianceResultName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("complianceResultName") String complianceResultName, - @HeaderParam("Accept") String accept, + @PathParam("complianceResultName") String complianceResultName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of compliance results response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of compliance results response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -116,37 +112,28 @@ private Mono> listSinglePageAsync(String sc final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of compliance results response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of compliance results response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -154,24 +141,16 @@ private Mono> listSinglePageAsync(String sc final String apiVersion = "2017-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -184,9 +163,9 @@ private PagedFlux listAsync(String scope) { /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -195,15 +174,15 @@ private PagedFlux listAsync(String scope) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -216,9 +195,9 @@ public PagedIterable list(String scope) { /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -232,7 +211,7 @@ public PagedIterable list(String scope, Context context) /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -243,10 +222,8 @@ public PagedIterable list(String scope, Context context) @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String resourceId, String complianceResultName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -258,16 +235,14 @@ private Mono> getWithResponseAsync(String resour final String apiVersion = "2017-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get(this.client.getEndpoint(), apiVersion, resourceId, complianceResultName, accept, context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, resourceId, complianceResultName, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @param context The context to associate with this operation. @@ -277,13 +252,11 @@ private Mono> getWithResponseAsync(String resour * @return a compliance result along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, String complianceResultName, Context context) { + private Mono> getWithResponseAsync(String resourceId, String complianceResultName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -300,7 +273,7 @@ private Mono> getWithResponseAsync( /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -315,7 +288,7 @@ private Mono getAsync(String resourceId, String complianc /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @param context The context to associate with this operation. @@ -325,14 +298,14 @@ private Mono getAsync(String resourceId, String complianc * @return a compliance result along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceId, String complianceResultName, Context context) { + public Response getWithResponse(String resourceId, String complianceResultName, + Context context) { return getWithResponseAsync(resourceId, complianceResultName, context).block(); } /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -347,14 +320,15 @@ public ComplianceResultInner get(String resourceId, String complianceResultName) /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of compliance results response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of compliance results response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -362,37 +336,28 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of compliance results response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of compliance results response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -400,23 +365,13 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsImpl.java index 2aa80fd05d011..ec52ed5b3f21c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ComplianceResultsImpl.java @@ -21,30 +21,27 @@ public final class ComplianceResultsImpl implements ComplianceResults { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ComplianceResultsImpl( - ComplianceResultsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public ComplianceResultsImpl(ComplianceResultsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String scope) { PagedIterable inner = this.serviceClient().list(scope); - return Utils.mapPage(inner, inner1 -> new ComplianceResultImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ComplianceResultImpl(inner1, this.manager())); } public PagedIterable list(String scope, Context context) { PagedIterable inner = this.serviceClient().list(scope, context); - return Utils.mapPage(inner, inner1 -> new ComplianceResultImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ComplianceResultImpl(inner1, this.manager())); } public Response getWithResponse(String resourceId, String complianceResultName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceId, complianceResultName, context); + Response inner + = this.serviceClient().getWithResponse(resourceId, complianceResultName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ComplianceResultImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesClientImpl.java index 9dc3ac51dad76..bf5115ad78b8d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.ComplianceList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CompliancesClient. */ +/** + * An instance of this class provides access to all the operations defined in CompliancesClient. + */ public final class CompliancesClientImpl implements CompliancesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CompliancesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of CompliancesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ CompliancesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(CompliancesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(CompliancesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,58 +62,46 @@ public final class CompliancesClientImpl implements CompliancesClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterCompli") public interface CompliancesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/compliances") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/compliances/{complianceName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @PathParam("complianceName") String complianceName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @PathParam("complianceName") String complianceName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -116,37 +110,28 @@ private Mono> listSinglePageAsync(String scope) { final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -154,24 +139,16 @@ private Mono> listSinglePageAsync(String scope, C final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -184,9 +161,9 @@ private PagedFlux listAsync(String scope) { /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -195,15 +172,15 @@ private PagedFlux listAsync(String scope) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -216,9 +193,9 @@ public PagedIterable list(String scope) { /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -232,9 +209,9 @@ public PagedIterable list(String scope, Context context) { /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -244,10 +221,8 @@ public PagedIterable list(String scope, Context context) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String scope, String complianceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -265,9 +240,9 @@ private Mono> getWithResponseAsync(String scope, Strin /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -278,10 +253,8 @@ private Mono> getWithResponseAsync(String scope, Strin @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String scope, String complianceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -297,9 +270,9 @@ private Mono> getWithResponseAsync(String scope, Strin /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -313,9 +286,9 @@ private Mono getAsync(String scope, String complianceName) { /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -330,9 +303,9 @@ public Response getWithResponse(String scope, String compliance /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -346,14 +319,15 @@ public ComplianceInner get(String scope, String complianceName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -361,37 +335,28 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of Compliance objects response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -399,23 +364,13 @@ private Mono> listNextSinglePageAsync(String next return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesImpl.java index b8e4641782c32..e66593bba785e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CompliancesImpl.java @@ -21,29 +21,26 @@ public final class CompliancesImpl implements Compliances { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public CompliancesImpl( - CompliancesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public CompliancesImpl(CompliancesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String scope) { PagedIterable inner = this.serviceClient().list(scope); - return Utils.mapPage(inner, inner1 -> new ComplianceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ComplianceImpl(inner1, this.manager())); } public PagedIterable list(String scope, Context context) { PagedIterable inner = this.serviceClient().list(scope, context); - return Utils.mapPage(inner, inner1 -> new ComplianceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ComplianceImpl(inner1, this.manager())); } public Response getWithResponse(String scope, String complianceName, Context context) { Response inner = this.serviceClient().getWithResponse(scope, complianceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ComplianceImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectionStringsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectionStringsImpl.java deleted file mode 100644 index 9014f64c6f6b6..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectionStringsImpl.java +++ /dev/null @@ -1,40 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.resourcemanager.security.fluent.models.ConnectionStringsInner; -import com.azure.resourcemanager.security.models.ConnectionStrings; -import com.azure.resourcemanager.security.models.IngestionConnectionString; -import java.util.Collections; -import java.util.List; - -public final class ConnectionStringsImpl implements ConnectionStrings { - private ConnectionStringsInner innerObject; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - ConnectionStringsImpl( - ConnectionStringsInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public List value() { - List inner = this.innerModel().value(); - if (inner != null) { - return Collections.unmodifiableList(inner); - } else { - return Collections.emptyList(); - } - } - - public ConnectionStringsInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorSettingImpl.java index 8fdafd0a7a493..0bcb75c18b2a1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorSettingImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorSettingImpl.java @@ -47,22 +47,14 @@ private com.azure.resourcemanager.security.SecurityManager manager() { private String connectorName; public ConnectorSetting create() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectors() - .createOrUpdateWithResponse(connectorName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectors() + .createOrUpdateWithResponse(connectorName, this.innerModel(), Context.NONE).getValue(); return this; } public ConnectorSetting create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectors() - .createOrUpdateWithResponse(connectorName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectors() + .createOrUpdateWithResponse(connectorName, this.innerModel(), context).getValue(); return this; } @@ -77,41 +69,33 @@ public ConnectorSettingImpl update() { } public ConnectorSetting apply() { - this.innerObject = - serviceManager - .serviceClient() - .getConnectors() - .createOrUpdateWithResponse(connectorName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectors() + .createOrUpdateWithResponse(connectorName, this.innerModel(), Context.NONE).getValue(); return this; } public ConnectorSetting apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getConnectors() - .createOrUpdateWithResponse(connectorName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getConnectors() + .createOrUpdateWithResponse(connectorName, this.innerModel(), context).getValue(); return this; } - ConnectorSettingImpl( - ConnectorSettingInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + ConnectorSettingImpl(ConnectorSettingInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.connectorName = Utils.getValueFromIdByName(innerObject.id(), "connectors"); + this.connectorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "connectors"); } public ConnectorSetting refresh() { - this.innerObject = - serviceManager.serviceClient().getConnectors().getWithResponse(connectorName, Context.NONE).getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectors().getWithResponse(connectorName, Context.NONE).getValue(); return this; } public ConnectorSetting refresh(Context context) { - this.innerObject = - serviceManager.serviceClient().getConnectors().getWithResponse(connectorName, context).getValue(); + this.innerObject + = serviceManager.serviceClient().getConnectors().getWithResponse(connectorName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsClientImpl.java index 21700bf0455ab..ff131b2b5d35b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsClientImpl.java @@ -33,22 +33,28 @@ import com.azure.resourcemanager.security.models.ConnectorSettingList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ConnectorsClient. */ +/** + * An instance of this class provides access to all the operations defined in ConnectorsClient. + */ public final class ConnectorsClientImpl implements ConnectorsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ConnectorsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ConnectorsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ConnectorsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(ConnectorsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(ConnectorsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -59,153 +65,111 @@ public final class ConnectorsClientImpl implements ConnectorsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterConnec") public interface ConnectorsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/connectors") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/connectors/{connectorName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("connectorName") String connectorName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("connectorName") String connectorName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/connectors/{connectorName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("connectorName") String connectorName, - @BodyParam("application/json") ConnectorSettingInner connectorSetting, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ConnectorSettingInner connectorSetting, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/connectors/{connectorName}") - @ExpectedResponses({200, 204}) + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("connectorName") String connectorName, - @HeaderParam("Accept") String accept, - Context context); + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("connectorName") String connectorName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Cloud accounts connectors of a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return for a subscription, list of all cloud account connectors and their settings along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return for a subscription, list of all cloud account connectors and their settings along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Cloud accounts connectors of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return for a subscription, list of all cloud account connectors and their settings along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return for a subscription, list of all cloud account connectors and their settings along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Cloud accounts connectors of a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -214,27 +178,27 @@ private PagedFlux listAsync() { /** * Cloud accounts connectors of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Cloud accounts connectors of a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -243,13 +207,13 @@ public PagedIterable list() { /** * Cloud accounts connectors of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -258,7 +222,7 @@ public PagedIterable list(Context context) { /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -268,16 +232,12 @@ public PagedIterable list(Context context) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String connectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (connectorName == null) { return Mono.error(new IllegalArgumentException("Parameter connectorName is required and cannot be null.")); @@ -285,22 +245,14 @@ private Mono> getWithResponseAsync(String connec final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - connectorName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + connectorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -311,16 +263,12 @@ private Mono> getWithResponseAsync(String connec @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String connectorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (connectorName == null) { return Mono.error(new IllegalArgumentException("Parameter connectorName is required and cannot be null.")); @@ -328,14 +276,13 @@ private Mono> getWithResponseAsync(String connec final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), connectorName, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), connectorName, + accept, context); } /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -349,7 +296,7 @@ private Mono getAsync(String connectorName) { /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -364,7 +311,7 @@ public Response getWithResponse(String connectorName, Con /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -379,7 +326,7 @@ public ConnectorSettingInner get(String connectorName) { /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -388,19 +335,15 @@ public ConnectorSettingInner get(String connectorName) { * @return the connector setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String connectorName, ConnectorSettingInner connectorSetting) { + private Mono> createOrUpdateWithResponseAsync(String connectorName, + ConnectorSettingInner connectorSetting) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (connectorName == null) { return Mono.error(new IllegalArgumentException("Parameter connectorName is required and cannot be null.")); @@ -414,24 +357,15 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - connectorName, - connectorSetting, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), connectorName, connectorSetting, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @param context The context to associate with this operation. @@ -441,19 +375,15 @@ private Mono> createOrUpdateWithResponseAsync( * @return the connector setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String connectorName, ConnectorSettingInner connectorSetting, Context context) { + private Mono> createOrUpdateWithResponseAsync(String connectorName, + ConnectorSettingInner connectorSetting, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (connectorName == null) { return Mono.error(new IllegalArgumentException("Parameter connectorName is required and cannot be null.")); @@ -467,21 +397,14 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - connectorName, - connectorSetting, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + connectorName, connectorSetting, accept, context); } /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -490,8 +413,8 @@ private Mono> createOrUpdateWithResponseAsync( * @return the connector setting on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String connectorName, ConnectorSettingInner connectorSetting) { + private Mono createOrUpdateAsync(String connectorName, + ConnectorSettingInner connectorSetting) { return createOrUpdateWithResponseAsync(connectorName, connectorSetting) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -499,7 +422,7 @@ private Mono createOrUpdateAsync( /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @param context The context to associate with this operation. @@ -509,15 +432,15 @@ private Mono createOrUpdateAsync( * @return the connector setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String connectorName, ConnectorSettingInner connectorSetting, Context context) { + public Response createOrUpdateWithResponse(String connectorName, + ConnectorSettingInner connectorSetting, Context context) { return createOrUpdateWithResponseAsync(connectorName, connectorSetting, context).block(); } /** * Create a cloud account connector or update an existing one. Connect to your cloud account. For AWS, use either * account credentials or role-based authentication. For GCP, use account organization credentials. - * + * * @param connectorName Name of the cloud account connector. * @param connectorSetting Settings for the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -532,7 +455,7 @@ public ConnectorSettingInner createOrUpdate(String connectorName, ConnectorSetti /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -542,16 +465,12 @@ public ConnectorSettingInner createOrUpdate(String connectorName, ConnectorSetti @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String connectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (connectorName == null) { return Mono.error(new IllegalArgumentException("Parameter connectorName is required and cannot be null.")); @@ -559,22 +478,14 @@ private Mono> deleteWithResponseAsync(String connectorName) { final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - connectorName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), connectorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -585,16 +496,12 @@ private Mono> deleteWithResponseAsync(String connectorName) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String connectorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (connectorName == null) { return Mono.error(new IllegalArgumentException("Parameter connectorName is required and cannot be null.")); @@ -602,14 +509,13 @@ private Mono> deleteWithResponseAsync(String connectorName, Conte final String apiVersion = "2020-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), connectorName, accept, context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), connectorName, + accept, context); } /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -623,7 +529,7 @@ private Mono deleteAsync(String connectorName) { /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -638,7 +544,7 @@ public Response deleteWithResponse(String connectorName, Context context) /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -651,14 +557,15 @@ public void delete(String connectorName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return for a subscription, list of all cloud account connectors and their settings along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return for a subscription, list of all cloud account connectors and their settings along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -666,37 +573,28 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return for a subscription, list of all cloud account connectors and their settings along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return for a subscription, list of all cloud account connectors and their settings along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -704,23 +602,13 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsImpl.java index ba0dd85d8a644..b0fc9d20799e1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ConnectorsImpl.java @@ -21,29 +21,26 @@ public final class ConnectorsImpl implements Connectors { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ConnectorsImpl( - ConnectorsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public ConnectorsImpl(ConnectorsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ConnectorSettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConnectorSettingImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ConnectorSettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ConnectorSettingImpl(inner1, this.manager())); } public Response getWithResponse(String connectorName, Context context) { Response inner = this.serviceClient().getWithResponse(connectorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ConnectorSettingImpl(inner.getValue(), this.manager())); } else { return null; @@ -68,45 +65,37 @@ public void delete(String connectorName) { } public ConnectorSetting getById(String id) { - String connectorName = Utils.getValueFromIdByName(id, "connectors"); + String connectorName = ResourceManagerUtils.getValueFromIdByName(id, "connectors"); if (connectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); } return this.getWithResponse(connectorName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String connectorName = Utils.getValueFromIdByName(id, "connectors"); + String connectorName = ResourceManagerUtils.getValueFromIdByName(id, "connectors"); if (connectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); } return this.getWithResponse(connectorName, context); } public void deleteById(String id) { - String connectorName = Utils.getValueFromIdByName(id, "connectors"); + String connectorName = ResourceManagerUtils.getValueFromIdByName(id, "connectors"); if (connectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); } this.deleteWithResponse(connectorName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String connectorName = Utils.getValueFromIdByName(id, "connectors"); + String connectorName = ResourceManagerUtils.getValueFromIdByName(id, "connectors"); if (connectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'connectors'.", id))); } return this.deleteWithResponse(connectorName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationImpl.java index 4b7d59d784b08..3576a911cb8d5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationImpl.java @@ -18,8 +18,7 @@ public final class CustomAssessmentAutomationImpl private final com.azure.resourcemanager.security.SecurityManager serviceManager; - CustomAssessmentAutomationImpl( - CustomAssessmentAutomationInner innerObject, + CustomAssessmentAutomationImpl(CustomAssessmentAutomationInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -89,27 +88,16 @@ public CustomAssessmentAutomationImpl withExistingResourceGroup(String resourceG } public CustomAssessmentAutomation create() { - this.innerObject = - serviceManager - .serviceClient() - .getCustomAssessmentAutomations() - .createWithResponse( - resourceGroupName, - customAssessmentAutomationName, - createCustomAssessmentAutomationBody, - Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getCustomAssessmentAutomations().createWithResponse(resourceGroupName, + customAssessmentAutomationName, createCustomAssessmentAutomationBody, Context.NONE).getValue(); return this; } public CustomAssessmentAutomation create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCustomAssessmentAutomations() - .createWithResponse( - resourceGroupName, customAssessmentAutomationName, createCustomAssessmentAutomationBody, context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getCustomAssessmentAutomations().createWithResponse(resourceGroupName, + customAssessmentAutomationName, createCustomAssessmentAutomationBody, context).getValue(); return this; } @@ -121,22 +109,14 @@ public CustomAssessmentAutomation create(Context context) { } public CustomAssessmentAutomation refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getCustomAssessmentAutomations() - .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCustomAssessmentAutomations() + .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, Context.NONE).getValue(); return this; } public CustomAssessmentAutomation refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCustomAssessmentAutomations() - .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCustomAssessmentAutomations() + .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsClientImpl.java index 21111c2d2a9ee..e5169ac0da7f6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsClientImpl.java @@ -34,24 +34,28 @@ import com.azure.resourcemanager.security.models.CustomAssessmentAutomationsListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CustomAssessmentAutomationsClient. */ +/** + * An instance of this class provides access to all the operations defined in CustomAssessmentAutomationsClient. + */ public final class CustomAssessmentAutomationsClientImpl implements CustomAssessmentAutomationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CustomAssessmentAutomationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of CustomAssessmentAutomationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ CustomAssessmentAutomationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - CustomAssessmentAutomationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(CustomAssessmentAutomationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -62,253 +66,202 @@ public final class CustomAssessmentAutomationsClientImpl implements CustomAssess @Host("{$host}") @ServiceInterface(name = "SecurityCenterCustom") public interface CustomAssessmentAutomationsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations/{customAssessmentAutomationName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations/{customAssessmentAutomationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("customAssessmentAutomationName") String customAssessmentAutomationName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations/{customAssessmentAutomationName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations/{customAssessmentAutomationName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("customAssessmentAutomationName") String customAssessmentAutomationName, @BodyParam("application/json") CustomAssessmentAutomationRequest customAssessmentAutomationBody, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations/{customAssessmentAutomationName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations/{customAssessmentAutomationName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("customAssessmentAutomationName") String customAssessmentAutomationName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customAssessmentAutomations") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/customAssessmentAutomations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String customAssessmentAutomationName) { + private Mono> + getByResourceGroupWithResponseAsync(String resourceGroupName, String customAssessmentAutomationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customAssessmentAutomationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customAssessmentAutomationName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, customAssessmentAutomationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getByResourceGroupWithResponseAsync( String resourceGroupName, String customAssessmentAutomationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customAssessmentAutomationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customAssessmentAutomationName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, customAssessmentAutomationName, accept, context); } /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getByResourceGroupAsync( - String resourceGroupName, String customAssessmentAutomationName) { + private Mono getByResourceGroupAsync(String resourceGroupName, + String customAssessmentAutomationName) { return getByResourceGroupWithResponseAsync(resourceGroupName, customAssessmentAutomationName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String customAssessmentAutomationName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, customAssessmentAutomationName, context).block(); } /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -316,20 +269,20 @@ public Response getByResourceGroupWithResponse( * @return a single custom assessment automation by name for the provided subscription and resource group. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CustomAssessmentAutomationInner getByResourceGroup( - String resourceGroupName, String customAssessmentAutomationName) { + public CustomAssessmentAutomationInner getByResourceGroup(String resourceGroupName, + String customAssessmentAutomationName) { return getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, Context.NONE) .getValue(); } /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -338,37 +291,27 @@ public CustomAssessmentAutomationInner getByResourceGroup( * @return custom Assessment Automation along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, - String customAssessmentAutomationName, - CustomAssessmentAutomationRequest customAssessmentAutomationBody) { + private Mono> createWithResponseAsync(String resourceGroupName, + String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customAssessmentAutomationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationName is required and cannot be null.")); } if (customAssessmentAutomationBody == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationBody is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationBody is required and cannot be null.")); } else { customAssessmentAutomationBody.validate(); } @@ -376,28 +319,19 @@ private Mono> createWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customAssessmentAutomationName, - customAssessmentAutomationBody, - accept, - context)) + context -> service.create(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, customAssessmentAutomationName, customAssessmentAutomationBody, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @param context The context to associate with this operation. @@ -407,64 +341,46 @@ private Mono> createWithResponseAsync( * @return custom Assessment Automation along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, - String customAssessmentAutomationName, - CustomAssessmentAutomationRequest customAssessmentAutomationBody, + private Mono> createWithResponseAsync(String resourceGroupName, + String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customAssessmentAutomationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationName is required and cannot be null.")); } if (customAssessmentAutomationBody == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationBody is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationBody is required and cannot be null.")); } else { customAssessmentAutomationBody.validate(); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customAssessmentAutomationName, - customAssessmentAutomationBody, - accept, - context); + return service.create(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + customAssessmentAutomationName, customAssessmentAutomationBody, accept, context); } /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -473,23 +389,20 @@ private Mono> createWithResponseAsync( * @return custom Assessment Automation on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, - String customAssessmentAutomationName, - CustomAssessmentAutomationRequest customAssessmentAutomationBody) { - return createWithResponseAsync( - resourceGroupName, customAssessmentAutomationName, customAssessmentAutomationBody) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono createAsync(String resourceGroupName, + String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody) { + return createWithResponseAsync(resourceGroupName, customAssessmentAutomationName, + customAssessmentAutomationBody).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @param context The context to associate with this operation. @@ -499,24 +412,21 @@ private Mono createAsync( * @return custom Assessment Automation along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String resourceGroupName, - String customAssessmentAutomationName, - CustomAssessmentAutomationRequest customAssessmentAutomationBody, + public Response createWithResponse(String resourceGroupName, + String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody, Context context) { - return createWithResponseAsync( - resourceGroupName, customAssessmentAutomationName, customAssessmentAutomationBody, context) - .block(); + return createWithResponseAsync(resourceGroupName, customAssessmentAutomationName, + customAssessmentAutomationBody, context).block(); } /** * Creates a custom assessment automation - * - *

Creates or updates a custom assessment automation for the provided subscription. Please note that providing an + * + * Creates or updates a custom assessment automation for the provided subscription. Please note that providing an * existing custom assessment automation will replace the existing record. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param customAssessmentAutomationBody Custom Assessment Automation body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -525,22 +435,19 @@ public Response createWithResponse( * @return custom Assessment Automation. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CustomAssessmentAutomationInner create( - String resourceGroupName, - String customAssessmentAutomationName, + public CustomAssessmentAutomationInner create(String resourceGroupName, String customAssessmentAutomationName, CustomAssessmentAutomationRequest customAssessmentAutomationBody) { - return createWithResponse( - resourceGroupName, customAssessmentAutomationName, customAssessmentAutomationBody, Context.NONE) - .getValue(); + return createWithResponse(resourceGroupName, customAssessmentAutomationName, customAssessmentAutomationBody, + Context.NONE).getValue(); } /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -548,54 +455,39 @@ public CustomAssessmentAutomationInner create( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String customAssessmentAutomationName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, + String customAssessmentAutomationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customAssessmentAutomationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customAssessmentAutomationName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, customAssessmentAutomationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -604,51 +496,38 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String customAssessmentAutomationName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, + String customAssessmentAutomationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customAssessmentAutomationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customAssessmentAutomationName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customAssessmentAutomationName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customAssessmentAutomationName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + customAssessmentAutomationName, accept, context); } /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -663,11 +542,11 @@ private Mono deleteAsync(String resourceGroupName, String customAssessment /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -676,18 +555,18 @@ private Mono deleteAsync(String resourceGroupName, String customAssessment * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String customAssessmentAutomationName, + Context context) { return deleteWithResponseAsync(resourceGroupName, customAssessmentAutomationName, context).block(); } /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -700,31 +579,27 @@ public void delete(String resourceGroupName, String customAssessmentAutomationNa /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -733,56 +608,37 @@ private Mono> listByResourceGroup final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -792,31 +648,19 @@ private Mono> listByResourceGroup final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -824,18 +668,17 @@ private Mono> listByResourceGroup */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -843,20 +686,19 @@ private PagedFlux listByResourceGroupAsync(Stri * @return a list of Custom Assessment Automations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + private PagedFlux listByResourceGroupAsync(String resourceGroupName, + Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -869,11 +711,11 @@ public PagedIterable listByResourceGroup(String /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -881,116 +723,91 @@ public PagedIterable listByResourceGroup(String * @return a list of Custom Assessment Automations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup( - String resourceGroupName, Context context) { + public PagedIterable listByResourceGroup(String resourceGroupName, + Context context) { return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, context)); } /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -999,15 +816,15 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations as paginated response with {@link PagedIterable}. @@ -1019,9 +836,9 @@ public PagedIterable list() { /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1035,157 +852,123 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink) { + private Mono> + listByResourceGroupNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsImpl.java index 460555a46574e..0ccaf3e238622 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomAssessmentAutomationsImpl.java @@ -21,34 +21,28 @@ public final class CustomAssessmentAutomationsImpl implements CustomAssessmentAu private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public CustomAssessmentAutomationsImpl( - CustomAssessmentAutomationsClient innerClient, + public CustomAssessmentAutomationsImpl(CustomAssessmentAutomationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context) { - Response inner = - this - .serviceClient() - .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String customAssessmentAutomationName, Context context) { + Response inner = this.serviceClient() + .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CustomAssessmentAutomationImpl(inner.getValue(), this.manager())); } else { return null; } } - public CustomAssessmentAutomation getByResourceGroup( - String resourceGroupName, String customAssessmentAutomationName) { - CustomAssessmentAutomationInner inner = - this.serviceClient().getByResourceGroup(resourceGroupName, customAssessmentAutomationName); + public CustomAssessmentAutomation getByResourceGroup(String resourceGroupName, + String customAssessmentAutomationName) { + CustomAssessmentAutomationInner inner + = this.serviceClient().getByResourceGroup(resourceGroupName, customAssessmentAutomationName); if (inner != null) { return new CustomAssessmentAutomationImpl(inner, this.manager()); } else { @@ -56,8 +50,8 @@ public CustomAssessmentAutomation getByResourceGroup( } } - public Response deleteByResourceGroupWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceGroupName, + String customAssessmentAutomationName, Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, customAssessmentAutomationName, context); } @@ -66,117 +60,88 @@ public void deleteByResourceGroup(String resourceGroupName, String customAssessm } public PagedIterable listByResourceGroup(String resourceGroupName) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomAssessmentAutomationImpl(inner1, this.manager())); } public CustomAssessmentAutomation getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customAssessmentAutomationName = Utils.getValueFromIdByName(id, "customAssessmentAutomations"); + String customAssessmentAutomationName + = ResourceManagerUtils.getValueFromIdByName(id, "customAssessmentAutomations"); if (customAssessmentAutomationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customAssessmentAutomations'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'customAssessmentAutomations'.", id))); } - return this - .getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, Context.NONE) + return this.getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customAssessmentAutomationName = Utils.getValueFromIdByName(id, "customAssessmentAutomations"); + String customAssessmentAutomationName + = ResourceManagerUtils.getValueFromIdByName(id, "customAssessmentAutomations"); if (customAssessmentAutomationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customAssessmentAutomations'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'customAssessmentAutomations'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customAssessmentAutomationName = Utils.getValueFromIdByName(id, "customAssessmentAutomations"); + String customAssessmentAutomationName + = ResourceManagerUtils.getValueFromIdByName(id, "customAssessmentAutomations"); if (customAssessmentAutomationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customAssessmentAutomations'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'customAssessmentAutomations'.", id))); } this.deleteByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customAssessmentAutomationName = Utils.getValueFromIdByName(id, "customAssessmentAutomations"); + String customAssessmentAutomationName + = ResourceManagerUtils.getValueFromIdByName(id, "customAssessmentAutomations"); if (customAssessmentAutomationName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customAssessmentAutomations'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'customAssessmentAutomations'.", id))); } return this.deleteByResourceGroupWithResponse(resourceGroupName, customAssessmentAutomationName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentImpl.java index fb658a4180552..7b253384c9086 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentImpl.java @@ -16,8 +16,7 @@ public final class CustomEntityStoreAssignmentImpl private final com.azure.resourcemanager.security.SecurityManager serviceManager; - CustomEntityStoreAssignmentImpl( - CustomEntityStoreAssignmentInner innerObject, + CustomEntityStoreAssignmentImpl(CustomEntityStoreAssignmentInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -67,30 +66,18 @@ public CustomEntityStoreAssignmentImpl withExistingResourceGroup(String resource } public CustomEntityStoreAssignment create() { - this.innerObject = - serviceManager - .serviceClient() - .getCustomEntityStoreAssignments() - .createWithResponse( - resourceGroupName, - customEntityStoreAssignmentName, - createCustomEntityStoreAssignmentRequestBody, - Context.NONE) + this.innerObject + = serviceManager + .serviceClient().getCustomEntityStoreAssignments().createWithResponse(resourceGroupName, + customEntityStoreAssignmentName, createCustomEntityStoreAssignmentRequestBody, Context.NONE) .getValue(); return this; } public CustomEntityStoreAssignment create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCustomEntityStoreAssignments() - .createWithResponse( - resourceGroupName, - customEntityStoreAssignmentName, - createCustomEntityStoreAssignmentRequestBody, - context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getCustomEntityStoreAssignments().createWithResponse(resourceGroupName, + customEntityStoreAssignmentName, createCustomEntityStoreAssignmentRequestBody, context).getValue(); return this; } @@ -102,22 +89,15 @@ public CustomEntityStoreAssignment create(Context context) { } public CustomEntityStoreAssignment refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getCustomEntityStoreAssignments() - .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCustomEntityStoreAssignments() + .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, Context.NONE) + .getValue(); return this; } public CustomEntityStoreAssignment refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getCustomEntityStoreAssignments() - .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getCustomEntityStoreAssignments() + .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsClientImpl.java index 338a98a95a378..b41e3b2200f10 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsClientImpl.java @@ -34,24 +34,28 @@ import com.azure.resourcemanager.security.models.CustomEntityStoreAssignmentsListResult; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in CustomEntityStoreAssignmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in CustomEntityStoreAssignmentsClient. + */ public final class CustomEntityStoreAssignmentsClientImpl implements CustomEntityStoreAssignmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final CustomEntityStoreAssignmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of CustomEntityStoreAssignmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ CustomEntityStoreAssignmentsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - CustomEntityStoreAssignmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(CustomEntityStoreAssignmentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -62,253 +66,202 @@ public final class CustomEntityStoreAssignmentsClientImpl implements CustomEntit @Host("{$host}") @ServiceInterface(name = "SecurityCenterCustom") public interface CustomEntityStoreAssignmentsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments/{customEntityStoreAssignmentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments/{customEntityStoreAssignmentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("customEntityStoreAssignmentName") String customEntityStoreAssignmentName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments/{customEntityStoreAssignmentName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments/{customEntityStoreAssignmentName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("customEntityStoreAssignmentName") String customEntityStoreAssignmentName, @BodyParam("application/json") CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments/{customEntityStoreAssignmentName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments/{customEntityStoreAssignmentName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("customEntityStoreAssignmentName") String customEntityStoreAssignmentName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourcegroups/{resourceGroupName}/providers/Microsoft.Security/customEntityStoreAssignments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/customEntityStoreAssignments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String customEntityStoreAssignmentName) { + private Mono> + getByResourceGroupWithResponseAsync(String resourceGroupName, String customEntityStoreAssignmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customEntityStoreAssignmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customEntityStoreAssignmentName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, customEntityStoreAssignmentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response} on successful completion of {@link Mono}. + * with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getByResourceGroupWithResponseAsync( String resourceGroupName, String customEntityStoreAssignmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customEntityStoreAssignmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customEntityStoreAssignmentName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, customEntityStoreAssignmentName, accept, context); } /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getByResourceGroupAsync( - String resourceGroupName, String customEntityStoreAssignmentName) { + private Mono getByResourceGroupAsync(String resourceGroupName, + String customEntityStoreAssignmentName) { return getByResourceGroupWithResponseAsync(resourceGroupName, customEntityStoreAssignmentName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response}. + * with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String customEntityStoreAssignmentName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, customEntityStoreAssignmentName, context).block(); } /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -316,19 +269,19 @@ public Response getByResourceGroupWithResponse * @return a single custom entity store assignment by name for the provided subscription and resource group. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CustomEntityStoreAssignmentInner getByResourceGroup( - String resourceGroupName, String customEntityStoreAssignmentName) { + public CustomEntityStoreAssignmentInner getByResourceGroup(String resourceGroupName, + String customEntityStoreAssignmentName) { return getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, Context.NONE) .getValue(); } /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -337,65 +290,47 @@ public CustomEntityStoreAssignmentInner getByResourceGroup( * @return custom entity store assignment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, + private Mono> createWithResponseAsync(String resourceGroupName, String customEntityStoreAssignmentName, CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customEntityStoreAssignmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentName is required and cannot be null.")); } if (customEntityStoreAssignmentRequestBody == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentRequestBody is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentRequestBody is required and cannot be null.")); } else { customEntityStoreAssignmentRequestBody.validate(); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customEntityStoreAssignmentName, - customEntityStoreAssignmentRequestBody, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, customEntityStoreAssignmentName, + customEntityStoreAssignmentRequestBody, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @param context The context to associate with this operation. @@ -405,63 +340,45 @@ private Mono> createWithResponseAsync * @return custom entity store assignment along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String resourceGroupName, + private Mono> createWithResponseAsync(String resourceGroupName, String customEntityStoreAssignmentName, - CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, - Context context) { + CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customEntityStoreAssignmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentName is required and cannot be null.")); } if (customEntityStoreAssignmentRequestBody == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentRequestBody is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentRequestBody is required and cannot be null.")); } else { customEntityStoreAssignmentRequestBody.validate(); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customEntityStoreAssignmentName, - customEntityStoreAssignmentRequestBody, - accept, - context); + return service.create(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + customEntityStoreAssignmentName, customEntityStoreAssignmentRequestBody, accept, context); } /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -470,22 +387,20 @@ private Mono> createWithResponseAsync * @return custom entity store assignment on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String resourceGroupName, + private Mono createAsync(String resourceGroupName, String customEntityStoreAssignmentName, CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody) { - return createWithResponseAsync( - resourceGroupName, customEntityStoreAssignmentName, customEntityStoreAssignmentRequestBody) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + return createWithResponseAsync(resourceGroupName, customEntityStoreAssignmentName, + customEntityStoreAssignmentRequestBody).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @param context The context to associate with this operation. @@ -495,23 +410,20 @@ private Mono createAsync( * @return custom entity store assignment along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String resourceGroupName, + public Response createWithResponse(String resourceGroupName, String customEntityStoreAssignmentName, - CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, - Context context) { - return createWithResponseAsync( - resourceGroupName, customEntityStoreAssignmentName, customEntityStoreAssignmentRequestBody, context) - .block(); + CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody, Context context) { + return createWithResponseAsync(resourceGroupName, customEntityStoreAssignmentName, + customEntityStoreAssignmentRequestBody, context).block(); } /** * Creates a custom entity store assignment - * - *

Creates a custom entity store assignment for the provided subscription, if not already exists. - * + * + * Creates a custom entity store assignment for the provided subscription, if not already exists. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param customEntityStoreAssignmentRequestBody Custom entity store assignment body. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -520,25 +432,19 @@ public Response createWithResponse( * @return custom entity store assignment. */ @ServiceMethod(returns = ReturnType.SINGLE) - public CustomEntityStoreAssignmentInner create( - String resourceGroupName, - String customEntityStoreAssignmentName, + public CustomEntityStoreAssignmentInner create(String resourceGroupName, String customEntityStoreAssignmentName, CustomEntityStoreAssignmentRequest customEntityStoreAssignmentRequestBody) { - return createWithResponse( - resourceGroupName, - customEntityStoreAssignmentName, - customEntityStoreAssignmentRequestBody, - Context.NONE) - .getValue(); + return createWithResponse(resourceGroupName, customEntityStoreAssignmentName, + customEntityStoreAssignmentRequestBody, Context.NONE).getValue(); } /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -546,54 +452,39 @@ public CustomEntityStoreAssignmentInner create( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String customEntityStoreAssignmentName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, + String customEntityStoreAssignmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customEntityStoreAssignmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customEntityStoreAssignmentName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, customEntityStoreAssignmentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -602,51 +493,38 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String customEntityStoreAssignmentName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, + String customEntityStoreAssignmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } if (customEntityStoreAssignmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter customEntityStoreAssignmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter customEntityStoreAssignmentName is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - customEntityStoreAssignmentName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + customEntityStoreAssignmentName, accept, context); } /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -661,11 +539,11 @@ private Mono deleteAsync(String resourceGroupName, String customEntityStor /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -674,18 +552,18 @@ private Mono deleteAsync(String resourceGroupName, String customEntityStor * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String customEntityStoreAssignmentName, + Context context) { return deleteWithResponseAsync(resourceGroupName, customEntityStoreAssignmentName, context).block(); } /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -698,31 +576,27 @@ public void delete(String resourceGroupName, String customEntityStoreAssignmentN /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -731,56 +605,37 @@ private Mono> listByResourceGrou final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -790,31 +645,19 @@ private Mono> listByResourceGrou final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -822,18 +665,17 @@ private Mono> listByResourceGrou */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -841,20 +683,19 @@ private PagedFlux listByResourceGroupAsync(Str * @return a list of custom entity store assignments as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + private PagedFlux listByResourceGroupAsync(String resourceGroupName, + Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -867,11 +708,11 @@ public PagedIterable listByResourceGroup(Strin /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -879,116 +720,91 @@ public PagedIterable listByResourceGroup(Strin * @return a list of custom entity store assignments as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup( - String resourceGroupName, Context context) { + public PagedIterable listByResourceGroup(String resourceGroupName, + Context context) { return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, context)); } /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -997,15 +813,15 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments as paginated response with {@link PagedIterable}. @@ -1017,9 +833,9 @@ public PagedIterable list() { /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1033,157 +849,123 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink) { + private Mono> + listByResourceGroupNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listByResourceGroupNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsImpl.java index 2b1bdc19cc0ed..5ac206c8eea7b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/CustomEntityStoreAssignmentsImpl.java @@ -21,34 +21,28 @@ public final class CustomEntityStoreAssignmentsImpl implements CustomEntityStore private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public CustomEntityStoreAssignmentsImpl( - CustomEntityStoreAssignmentsClient innerClient, + public CustomEntityStoreAssignmentsImpl(CustomEntityStoreAssignmentsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context) { - Response inner = - this - .serviceClient() - .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String customEntityStoreAssignmentName, Context context) { + Response inner = this.serviceClient() + .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new CustomEntityStoreAssignmentImpl(inner.getValue(), this.manager())); } else { return null; } } - public CustomEntityStoreAssignment getByResourceGroup( - String resourceGroupName, String customEntityStoreAssignmentName) { - CustomEntityStoreAssignmentInner inner = - this.serviceClient().getByResourceGroup(resourceGroupName, customEntityStoreAssignmentName); + public CustomEntityStoreAssignment getByResourceGroup(String resourceGroupName, + String customEntityStoreAssignmentName) { + CustomEntityStoreAssignmentInner inner + = this.serviceClient().getByResourceGroup(resourceGroupName, customEntityStoreAssignmentName); if (inner != null) { return new CustomEntityStoreAssignmentImpl(inner, this.manager()); } else { @@ -56,8 +50,8 @@ public CustomEntityStoreAssignment getByResourceGroup( } } - public Response deleteByResourceGroupWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceGroupName, + String customEntityStoreAssignmentName, Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, customEntityStoreAssignmentName, context); } @@ -66,117 +60,88 @@ public void deleteByResourceGroup(String resourceGroupName, String customEntityS } public PagedIterable listByResourceGroup(String resourceGroupName) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new CustomEntityStoreAssignmentImpl(inner1, this.manager())); } public CustomEntityStoreAssignment getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customEntityStoreAssignmentName = Utils.getValueFromIdByName(id, "customEntityStoreAssignments"); + String customEntityStoreAssignmentName + = ResourceManagerUtils.getValueFromIdByName(id, "customEntityStoreAssignments"); if (customEntityStoreAssignmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customEntityStoreAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String.format( + "The resource ID '%s' is not valid. Missing path segment 'customEntityStoreAssignments'.", id))); } - return this - .getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, Context.NONE) + return this.getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customEntityStoreAssignmentName = Utils.getValueFromIdByName(id, "customEntityStoreAssignments"); + String customEntityStoreAssignmentName + = ResourceManagerUtils.getValueFromIdByName(id, "customEntityStoreAssignments"); if (customEntityStoreAssignmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customEntityStoreAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String.format( + "The resource ID '%s' is not valid. Missing path segment 'customEntityStoreAssignments'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customEntityStoreAssignmentName = Utils.getValueFromIdByName(id, "customEntityStoreAssignments"); + String customEntityStoreAssignmentName + = ResourceManagerUtils.getValueFromIdByName(id, "customEntityStoreAssignments"); if (customEntityStoreAssignmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customEntityStoreAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String.format( + "The resource ID '%s' is not valid. Missing path segment 'customEntityStoreAssignments'.", id))); } this.deleteByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourcegroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourcegroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourcegroups'.", id))); } - String customEntityStoreAssignmentName = Utils.getValueFromIdByName(id, "customEntityStoreAssignments"); + String customEntityStoreAssignmentName + = ResourceManagerUtils.getValueFromIdByName(id, "customEntityStoreAssignments"); if (customEntityStoreAssignmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'customEntityStoreAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String.format( + "The resource ID '%s' is not valid. Missing path segment 'customEntityStoreAssignments'.", id))); } return this.deleteByResourceGroupWithResponse(resourceGroupName, customEntityStoreAssignmentName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStorageSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStorageSettingImpl.java new file mode 100644 index 0000000000000..b5754f553d463 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStorageSettingImpl.java @@ -0,0 +1,149 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.DefenderForStorageSettingInner; +import com.azure.resourcemanager.security.models.DefenderForStorageSetting; +import com.azure.resourcemanager.security.models.OperationStatus; +import com.azure.resourcemanager.security.models.SettingName; + +public final class DefenderForStorageSettingImpl + implements DefenderForStorageSetting, DefenderForStorageSetting.Definition { + private DefenderForStorageSettingInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + DefenderForStorageSettingImpl(DefenderForStorageSettingInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public Boolean isEnabled() { + return this.innerModel().isEnabled(); + } + + public Boolean overrideSubscriptionLevelSettings() { + return this.innerModel().overrideSubscriptionLevelSettings(); + } + + public String scanResultsEventGridTopicResourceId() { + return this.innerModel().scanResultsEventGridTopicResourceId(); + } + + public OperationStatus operationStatus() { + return this.innerModel().operationStatus(); + } + + public Boolean isEnabledMalwareScanningIsEnabled() { + return this.innerModel().isEnabledMalwareScanningIsEnabled(); + } + + public Integer capGBPerMonth() { + return this.innerModel().capGBPerMonth(); + } + + public Boolean isEnabledSensitiveDataDiscoveryIsEnabled() { + return this.innerModel().isEnabledSensitiveDataDiscoveryIsEnabled(); + } + + public OperationStatus operationStatusSensitiveDataDiscoveryOperationStatus() { + return this.innerModel().operationStatusSensitiveDataDiscoveryOperationStatus(); + } + + public DefenderForStorageSettingInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + private String resourceId; + + private SettingName settingName; + + public DefenderForStorageSettingImpl withExistingResourceId(String resourceId) { + this.resourceId = resourceId; + return this; + } + + public DefenderForStorageSetting create() { + this.innerObject = serviceManager.serviceClient().getDefenderForStorages() + .createWithResponse(resourceId, settingName, this.innerModel(), Context.NONE).getValue(); + return this; + } + + public DefenderForStorageSetting create(Context context) { + this.innerObject = serviceManager.serviceClient().getDefenderForStorages() + .createWithResponse(resourceId, settingName, this.innerModel(), context).getValue(); + return this; + } + + DefenderForStorageSettingImpl(SettingName name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = new DefenderForStorageSettingInner(); + this.serviceManager = serviceManager; + this.settingName = name; + } + + public DefenderForStorageSetting refresh() { + this.innerObject = serviceManager.serviceClient().getDefenderForStorages() + .getWithResponse(resourceId, settingName, Context.NONE).getValue(); + return this; + } + + public DefenderForStorageSetting refresh(Context context) { + this.innerObject = serviceManager.serviceClient().getDefenderForStorages() + .getWithResponse(resourceId, settingName, context).getValue(); + return this; + } + + public DefenderForStorageSettingImpl withIsEnabled(Boolean isEnabled) { + this.innerModel().withIsEnabled(isEnabled); + return this; + } + + public DefenderForStorageSettingImpl + withOverrideSubscriptionLevelSettings(Boolean overrideSubscriptionLevelSettings) { + this.innerModel().withOverrideSubscriptionLevelSettings(overrideSubscriptionLevelSettings); + return this; + } + + public DefenderForStorageSettingImpl + withScanResultsEventGridTopicResourceId(String scanResultsEventGridTopicResourceId) { + this.innerModel().withScanResultsEventGridTopicResourceId(scanResultsEventGridTopicResourceId); + return this; + } + + public DefenderForStorageSettingImpl + withIsEnabledMalwareScanningIsEnabled(Boolean isEnabledMalwareScanningIsEnabled) { + this.innerModel().withIsEnabledMalwareScanningIsEnabled(isEnabledMalwareScanningIsEnabled); + return this; + } + + public DefenderForStorageSettingImpl withCapGBPerMonth(Integer capGBPerMonth) { + this.innerModel().withCapGBPerMonth(capGBPerMonth); + return this; + } + + public DefenderForStorageSettingImpl + withIsEnabledSensitiveDataDiscoveryIsEnabled(Boolean isEnabledSensitiveDataDiscoveryIsEnabled) { + this.innerModel().withIsEnabledSensitiveDataDiscoveryIsEnabled(isEnabledSensitiveDataDiscoveryIsEnabled); + return this; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesClientImpl.java new file mode 100644 index 0000000000000..8e825be60930f --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesClientImpl.java @@ -0,0 +1,323 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.DefenderForStoragesClient; +import com.azure.resourcemanager.security.fluent.models.DefenderForStorageSettingInner; +import com.azure.resourcemanager.security.models.SettingName; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in DefenderForStoragesClient. + */ +public final class DefenderForStoragesClientImpl implements DefenderForStoragesClient { + /** + * The proxy service used to perform REST calls. + */ + private final DefenderForStoragesService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of DefenderForStoragesClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + DefenderForStoragesClientImpl(SecurityCenterImpl client) { + this.service = RestProxy.create(DefenderForStoragesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterDefenderForStorages to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterDefend") + public interface DefenderForStoragesService { + @Headers({ "Content-Type: application/json" }) + @Get("/{resourceId}/providers/Microsoft.Security/defenderForStorageSettings/{settingName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, + @PathParam("settingName") SettingName settingName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/{resourceId}/providers/Microsoft.Security/defenderForStorageSettings/{settingName}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, + @PathParam("settingName") SettingName settingName, + @BodyParam("application/json") DefenderForStorageSettingInner defenderForStorageSetting, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceId, + SettingName settingName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (resourceId == null) { + return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); + } + if (settingName == null) { + return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); + } + final String apiVersion = "2022-12-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.get(this.client.getEndpoint(), apiVersion, resourceId, settingName, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceId, + SettingName settingName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (resourceId == null) { + return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); + } + if (settingName == null) { + return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); + } + final String apiVersion = "2022-12-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), apiVersion, resourceId, settingName, accept, context); + } + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceId, SettingName settingName) { + return getWithResponseAsync(resourceId, settingName).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceId, SettingName settingName, + Context context) { + return getWithResponseAsync(resourceId, settingName, context).block(); + } + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DefenderForStorageSettingInner get(String resourceId, SettingName settingName) { + return getWithResponse(resourceId, settingName, Context.NONE).getValue(); + } + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createWithResponseAsync(String resourceId, + SettingName settingName, DefenderForStorageSettingInner defenderForStorageSetting) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (resourceId == null) { + return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); + } + if (settingName == null) { + return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); + } + if (defenderForStorageSetting == null) { + return Mono.error( + new IllegalArgumentException("Parameter defenderForStorageSetting is required and cannot be null.")); + } else { + defenderForStorageSetting.validate(); + } + final String apiVersion = "2022-12-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.create(this.client.getEndpoint(), apiVersion, resourceId, settingName, + defenderForStorageSetting, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createWithResponseAsync(String resourceId, + SettingName settingName, DefenderForStorageSettingInner defenderForStorageSetting, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (resourceId == null) { + return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); + } + if (settingName == null) { + return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); + } + if (defenderForStorageSetting == null) { + return Mono.error( + new IllegalArgumentException("Parameter defenderForStorageSetting is required and cannot be null.")); + } else { + defenderForStorageSetting.validate(); + } + final String apiVersion = "2022-12-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.create(this.client.getEndpoint(), apiVersion, resourceId, settingName, defenderForStorageSetting, + accept, context); + } + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createAsync(String resourceId, SettingName settingName, + DefenderForStorageSettingInner defenderForStorageSetting) { + return createWithResponseAsync(resourceId, settingName, defenderForStorageSetting) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createWithResponse(String resourceId, SettingName settingName, + DefenderForStorageSettingInner defenderForStorageSetting, Context context) { + return createWithResponseAsync(resourceId, settingName, defenderForStorageSetting, context).block(); + } + + /** + * Creates or updates the Defender for Storage settings on a specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param defenderForStorageSetting Defender for Storage Settings. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DefenderForStorageSettingInner create(String resourceId, SettingName settingName, + DefenderForStorageSettingInner defenderForStorageSetting) { + return createWithResponse(resourceId, settingName, defenderForStorageSetting, Context.NONE).getValue(); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesImpl.java new file mode 100644 index 0000000000000..1cc0bf9043b7a --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DefenderForStoragesImpl.java @@ -0,0 +1,96 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.DefenderForStoragesClient; +import com.azure.resourcemanager.security.fluent.models.DefenderForStorageSettingInner; +import com.azure.resourcemanager.security.models.DefenderForStorageSetting; +import com.azure.resourcemanager.security.models.DefenderForStorages; +import com.azure.resourcemanager.security.models.SettingName; + +public final class DefenderForStoragesImpl implements DefenderForStorages { + private static final ClientLogger LOGGER = new ClientLogger(DefenderForStoragesImpl.class); + + private final DefenderForStoragesClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public DefenderForStoragesImpl(DefenderForStoragesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response getWithResponse(String resourceId, SettingName settingName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceId, settingName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DefenderForStorageSettingImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public DefenderForStorageSetting get(String resourceId, SettingName settingName) { + DefenderForStorageSettingInner inner = this.serviceClient().get(resourceId, settingName); + if (inner != null) { + return new DefenderForStorageSettingImpl(inner, this.manager()); + } else { + return null; + } + } + + public DefenderForStorageSetting getById(String id) { + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/defenderForStorageSettings/{settingName}", "resourceId"); + if (resourceId == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + } + String settingNameLocal = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/defenderForStorageSettings/{settingName}", "settingName"); + if (settingNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'defenderForStorageSettings'.", id))); + } + SettingName settingName = SettingName.fromString(settingNameLocal); + return this.getWithResponse(resourceId, settingName, Context.NONE).getValue(); + } + + public Response getByIdWithResponse(String id, Context context) { + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/defenderForStorageSettings/{settingName}", "resourceId"); + if (resourceId == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + } + String settingNameLocal = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/defenderForStorageSettings/{settingName}", "settingName"); + if (settingNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'defenderForStorageSettings'.", id))); + } + SettingName settingName = SettingName.fromString(settingNameLocal); + return this.getWithResponse(resourceId, settingName, context); + } + + private DefenderForStoragesClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } + + public DefenderForStorageSettingImpl define(SettingName name) { + return new DefenderForStorageSettingImpl(name, this.manager()); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationImpl.java new file mode 100644 index 0000000000000..9e56179ffc970 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; +import com.azure.resourcemanager.security.models.DevOpsConfiguration; +import com.azure.resourcemanager.security.models.DevOpsConfigurationProperties; + +public final class DevOpsConfigurationImpl implements DevOpsConfiguration { + private DevOpsConfigurationInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + DevOpsConfigurationImpl(DevOpsConfigurationInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public DevOpsConfigurationProperties properties() { + return this.innerModel().properties(); + } + + public DevOpsConfigurationInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsClientImpl.java new file mode 100644 index 0000000000000..aa2a975542fdc --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsClientImpl.java @@ -0,0 +1,1152 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Patch; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.management.polling.PollResult; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.core.util.polling.PollerFlux; +import com.azure.core.util.polling.SyncPoller; +import com.azure.resourcemanager.security.fluent.DevOpsConfigurationsClient; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; +import com.azure.resourcemanager.security.models.DevOpsConfigurationListResponse; +import java.nio.ByteBuffer; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in DevOpsConfigurationsClient. + */ +public final class DevOpsConfigurationsClientImpl implements DevOpsConfigurationsClient { + /** + * The proxy service used to perform REST calls. + */ + private final DevOpsConfigurationsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of DevOpsConfigurationsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + DevOpsConfigurationsClientImpl(SecurityCenterImpl client) { + this.service = RestProxy.create(DevOpsConfigurationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterDevOpsConfigurations to be used by the proxy service + * to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterDevOps") + public interface DevOpsConfigurationsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default") + @ExpectedResponses({ 200, 201, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") DevOpsConfigurationInner devOpsConfiguration, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default") + @ExpectedResponses({ 200, 202 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> update(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") DevOpsConfigurationInner devOpsConfiguration, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default") + @ExpectedResponses({ 200, 202, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono>> delete(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName)); + } + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, context)); + } + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, accept, context); + } + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, context).block(); + } + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DevOpsConfigurationInner get(String resourceGroupName, String securityConnectorName) { + return getWithResponse(resourceGroupName, securityConnectorName, Context.NONE).getValue(); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (devOpsConfiguration == null) { + return Mono + .error(new IllegalArgumentException("Parameter devOpsConfiguration is required and cannot be null.")); + } else { + devOpsConfiguration.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, devOpsConfiguration, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (devOpsConfiguration == null) { + return Mono + .error(new IllegalArgumentException("Parameter devOpsConfiguration is required and cannot be null.")); + } else { + devOpsConfiguration.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, devOpsConfiguration, accept, context); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DevOpsConfigurationInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration) { + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, devOpsConfiguration); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), DevOpsConfigurationInner.class, DevOpsConfigurationInner.class, + this.client.getContext()); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DevOpsConfigurationInner> beginCreateOrUpdateAsync( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), DevOpsConfigurationInner.class, DevOpsConfigurationInner.class, context); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DevOpsConfigurationInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration) + .getSyncPoller(); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DevOpsConfigurationInner> beginCreateOrUpdate( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, + Context context) { + return this.beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context) + .getSyncPoller(); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context) { + return beginCreateOrUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DevOpsConfigurationInner createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration).block(); + } + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DevOpsConfigurationInner createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context) { + return createOrUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context).block(); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (devOpsConfiguration == null) { + return Mono + .error(new IllegalArgumentException("Parameter devOpsConfiguration is required and cannot be null.")); + } else { + devOpsConfiguration.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, devOpsConfiguration, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (devOpsConfiguration == null) { + return Mono + .error(new IllegalArgumentException("Parameter devOpsConfiguration is required and cannot be null.")); + } else { + devOpsConfiguration.validate(); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.update(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, devOpsConfiguration, accept, context); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DevOpsConfigurationInner> beginUpdateAsync( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration) { + Mono>> mono + = updateWithResponseAsync(resourceGroupName, securityConnectorName, devOpsConfiguration); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), DevOpsConfigurationInner.class, DevOpsConfigurationInner.class, + this.client.getContext()); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, DevOpsConfigurationInner> beginUpdateAsync( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = updateWithResponseAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context); + return this.client.getLroResult(mono, + this.client.getHttpPipeline(), DevOpsConfigurationInner.class, DevOpsConfigurationInner.class, context); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DevOpsConfigurationInner> beginUpdate( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration).getSyncPoller(); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, DevOpsConfigurationInner> beginUpdate( + String resourceGroupName, String securityConnectorName, DevOpsConfigurationInner devOpsConfiguration, + Context context) { + return this.beginUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context) + .getSyncPoller(); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context) { + return beginUpdateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DevOpsConfigurationInner update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration) { + return updateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration).block(); + } + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public DevOpsConfigurationInner update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context) { + return updateAsync(resourceGroupName, securityConnectorName, devOpsConfiguration, context).block(); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono>> deleteWithResponseAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, accept, context); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, + String securityConnectorName) { + Mono>> mono = deleteWithResponseAsync(resourceGroupName, securityConnectorName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link PollerFlux} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String securityConnectorName, + Context context) { + context = this.client.mergeContext(context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, securityConnectorName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String securityConnectorName) { + return this.beginDeleteAsync(resourceGroupName, securityConnectorName).getSyncPoller(); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link SyncPoller} for polling of long-running operation. + */ + @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) + public SyncPoller, Void> beginDelete(String resourceGroupName, String securityConnectorName, + Context context) { + return this.beginDeleteAsync(resourceGroupName, securityConnectorName, context).getSyncPoller(); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String securityConnectorName) { + return beginDeleteAsync(resourceGroupName, securityConnectorName).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(String resourceGroupName, String securityConnectorName, Context context) { + return beginDeleteAsync(resourceGroupName, securityConnectorName, context).last() + .flatMap(this.client::getLroFinalResultOrError); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String securityConnectorName) { + deleteAsync(resourceGroupName, securityConnectorName).block(); + } + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(String resourceGroupName, String securityConnectorName, Context context) { + deleteAsync(resourceGroupName, securityConnectorName, context).block(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsImpl.java new file mode 100644 index 0000000000000..9a67afb8bd386 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsConfigurationsImpl.java @@ -0,0 +1,123 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.DevOpsConfigurationsClient; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; +import com.azure.resourcemanager.security.models.DevOpsConfiguration; +import com.azure.resourcemanager.security.models.DevOpsConfigurations; + +public final class DevOpsConfigurationsImpl implements DevOpsConfigurations { + private static final ClientLogger LOGGER = new ClientLogger(DevOpsConfigurationsImpl.class); + + private final DevOpsConfigurationsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public DevOpsConfigurationsImpl(DevOpsConfigurationsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DevOpsConfigurationImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DevOpsConfigurationImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, securityConnectorName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new DevOpsConfigurationImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public DevOpsConfiguration get(String resourceGroupName, String securityConnectorName) { + DevOpsConfigurationInner inner = this.serviceClient().get(resourceGroupName, securityConnectorName); + if (inner != null) { + return new DevOpsConfigurationImpl(inner, this.manager()); + } else { + return null; + } + } + + public DevOpsConfiguration createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration) { + DevOpsConfigurationInner inner + = this.serviceClient().createOrUpdate(resourceGroupName, securityConnectorName, devOpsConfiguration); + if (inner != null) { + return new DevOpsConfigurationImpl(inner, this.manager()); + } else { + return null; + } + } + + public DevOpsConfiguration createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context) { + DevOpsConfigurationInner inner = this.serviceClient().createOrUpdate(resourceGroupName, securityConnectorName, + devOpsConfiguration, context); + if (inner != null) { + return new DevOpsConfigurationImpl(inner, this.manager()); + } else { + return null; + } + } + + public DevOpsConfiguration update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration) { + DevOpsConfigurationInner inner + = this.serviceClient().update(resourceGroupName, securityConnectorName, devOpsConfiguration); + if (inner != null) { + return new DevOpsConfigurationImpl(inner, this.manager()); + } else { + return null; + } + } + + public DevOpsConfiguration update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context) { + DevOpsConfigurationInner inner + = this.serviceClient().update(resourceGroupName, securityConnectorName, devOpsConfiguration, context); + if (inner != null) { + return new DevOpsConfigurationImpl(inner, this.manager()); + } else { + return null; + } + } + + public void deleteByResourceGroup(String resourceGroupName, String securityConnectorName) { + this.serviceClient().delete(resourceGroupName, securityConnectorName); + } + + public void delete(String resourceGroupName, String securityConnectorName, Context context) { + this.serviceClient().delete(resourceGroupName, securityConnectorName, context); + } + + private DevOpsConfigurationsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsClientImpl.java new file mode 100644 index 0000000000000..2877727b9f51c --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsClientImpl.java @@ -0,0 +1,210 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.DevOpsOperationResultsClient; +import com.azure.resourcemanager.security.fluent.models.OperationStatusResultInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in DevOpsOperationResultsClient. + */ +public final class DevOpsOperationResultsClientImpl implements DevOpsOperationResultsClient { + /** + * The proxy service used to perform REST calls. + */ + private final DevOpsOperationResultsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of DevOpsOperationResultsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + DevOpsOperationResultsClientImpl(SecurityCenterImpl client) { + this.service = RestProxy.create(DevOpsOperationResultsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterDevOpsOperationResults to be used by the proxy service + * to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterDevOps") + public interface DevOpsOperationResultsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/operationResults/{operationResultId}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @PathParam("operationResultId") String operationResultId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String operationResultId) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (operationResultId == null) { + return Mono + .error(new IllegalArgumentException("Parameter operationResultId is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, operationResultId, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String operationResultId, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (operationResultId == null) { + return Mono + .error(new IllegalArgumentException("Parameter operationResultId is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, operationResultId, apiVersion, accept, context); + } + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String operationResultId) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, operationResultId) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String operationResultId, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, operationResultId, context).block(); + } + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public OperationStatusResultInner get(String resourceGroupName, String securityConnectorName, + String operationResultId) { + return getWithResponse(resourceGroupName, securityConnectorName, operationResultId, Context.NONE).getValue(); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsImpl.java new file mode 100644 index 0000000000000..72a9d7822e4ad --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DevOpsOperationResultsImpl.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.DevOpsOperationResultsClient; +import com.azure.resourcemanager.security.fluent.models.OperationStatusResultInner; +import com.azure.resourcemanager.security.models.DevOpsOperationResults; +import com.azure.resourcemanager.security.models.OperationStatusResult; + +public final class DevOpsOperationResultsImpl implements DevOpsOperationResults { + private static final ClientLogger LOGGER = new ClientLogger(DevOpsOperationResultsImpl.class); + + private final DevOpsOperationResultsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public DevOpsOperationResultsImpl(DevOpsOperationResultsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String operationResultId, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + securityConnectorName, operationResultId, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new OperationStatusResultImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public OperationStatusResult get(String resourceGroupName, String securityConnectorName, String operationResultId) { + OperationStatusResultInner inner + = this.serviceClient().get(resourceGroupName, securityConnectorName, operationResultId); + if (inner != null) { + return new OperationStatusResultImpl(inner, this.manager()); + } else { + return null; + } + } + + private DevOpsOperationResultsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupImpl.java index 6edb2fe0dd317..c16f6f43d45b6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupImpl.java @@ -86,22 +86,15 @@ public DeviceSecurityGroupImpl withExistingResourceId(String resourceId) { } public DeviceSecurityGroup create() { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceSecurityGroups() - .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceSecurityGroups() + .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), Context.NONE) + .getValue(); return this; } public DeviceSecurityGroup create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceSecurityGroups() - .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceSecurityGroups() + .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), context).getValue(); return this; } @@ -116,60 +109,38 @@ public DeviceSecurityGroupImpl update() { } public DeviceSecurityGroup apply() { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceSecurityGroups() - .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceSecurityGroups() + .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), Context.NONE) + .getValue(); return this; } public DeviceSecurityGroup apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceSecurityGroups() - .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceSecurityGroups() + .createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, this.innerModel(), context).getValue(); return this; } - DeviceSecurityGroupImpl( - DeviceSecurityGroupInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + DeviceSecurityGroupImpl(DeviceSecurityGroupInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceId = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "resourceId"); - this.deviceSecurityGroupName = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "deviceSecurityGroupName"); + this.resourceId = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", "resourceId"); + this.deviceSecurityGroupName = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", + "deviceSecurityGroupName"); } public DeviceSecurityGroup refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceSecurityGroups() - .getWithResponse(resourceId, deviceSecurityGroupName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceSecurityGroups() + .getWithResponse(resourceId, deviceSecurityGroupName, Context.NONE).getValue(); return this; } public DeviceSecurityGroup refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getDeviceSecurityGroups() - .getWithResponse(resourceId, deviceSecurityGroupName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getDeviceSecurityGroups() + .getWithResponse(resourceId, deviceSecurityGroupName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsClientImpl.java index 0d3f936ce0620..a7a01f5ae19ad 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsClientImpl.java @@ -33,95 +33,88 @@ import com.azure.resourcemanager.security.models.DeviceSecurityGroupList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DeviceSecurityGroupsClient. */ +/** + * An instance of this class provides access to all the operations defined in DeviceSecurityGroupsClient. + */ public final class DeviceSecurityGroupsClientImpl implements DeviceSecurityGroupsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DeviceSecurityGroupsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of DeviceSecurityGroupsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ DeviceSecurityGroupsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(DeviceSecurityGroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(DeviceSecurityGroupsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterDeviceSecurityGroups to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for SecurityCenterDeviceSecurityGroups to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterDevice") public interface DeviceSecurityGroupsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("deviceSecurityGroupName") String deviceSecurityGroupName, - @HeaderParam("Accept") String accept, + @PathParam("deviceSecurityGroupName") String deviceSecurityGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, @PathParam("deviceSecurityGroupName") String deviceSecurityGroupName, @BodyParam("application/json") DeviceSecurityGroupInner deviceSecurityGroup, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Delete("/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}") - @ExpectedResponses({200, 204}) + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("deviceSecurityGroupName") String deviceSecurityGroupName, - @HeaderParam("Accept") String accept, + @PathParam("deviceSecurityGroupName") String deviceSecurityGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -131,10 +124,8 @@ Mono> listNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -143,21 +134,14 @@ private Mono> listSinglePageAsync(String final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, resourceId, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -168,10 +152,8 @@ private Mono> listSinglePageAsync(String @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String resourceId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); @@ -179,22 +161,14 @@ private Mono> listSinglePageAsync(String final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, resourceId, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, resourceId, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -208,7 +182,7 @@ private PagedFlux listAsync(String resourceId) { /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -218,13 +192,13 @@ private PagedFlux listAsync(String resourceId) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceId, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceId, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(resourceId, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -238,7 +212,7 @@ public PagedIterable list(String resourceId) { /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -253,54 +227,43 @@ public PagedIterable list(String resourceId, Context c /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the device security group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, String deviceSecurityGroupName) { + private Mono> getWithResponseAsync(String resourceId, + String deviceSecurityGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (deviceSecurityGroupName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - resourceId, - deviceSecurityGroupName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, resourceId, + deviceSecurityGroupName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -308,21 +271,18 @@ private Mono> getWithResponseAsync( * @return the device security group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, String deviceSecurityGroupName, Context context) { + private Mono> getWithResponseAsync(String resourceId, + String deviceSecurityGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (deviceSecurityGroupName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; @@ -332,10 +292,10 @@ private Mono> getWithResponseAsync( /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -349,10 +309,10 @@ private Mono getAsync(String resourceId, String device /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -360,17 +320,17 @@ private Mono getAsync(String resourceId, String device * @return the device security group resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceId, String deviceSecurityGroupName, Context context) { + public Response getWithResponse(String resourceId, String deviceSecurityGroupName, + Context context) { return getWithResponseAsync(resourceId, deviceSecurityGroupName, context).block(); } /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -383,10 +343,10 @@ public DeviceSecurityGroupInner get(String resourceId, String deviceSecurityGrou /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -394,21 +354,18 @@ public DeviceSecurityGroupInner get(String resourceId, String deviceSecurityGrou * @return the device security group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceId, String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup) { + private Mono> createOrUpdateWithResponseAsync(String resourceId, + String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (deviceSecurityGroupName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); } if (deviceSecurityGroup == null) { return Mono @@ -419,26 +376,17 @@ private Mono> createOrUpdateWithResponseAsync final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - resourceId, - deviceSecurityGroupName, - deviceSecurityGroup, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, resourceId, + deviceSecurityGroupName, deviceSecurityGroup, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -447,24 +395,18 @@ private Mono> createOrUpdateWithResponseAsync * @return the device security group resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceId, - String deviceSecurityGroupName, - DeviceSecurityGroupInner deviceSecurityGroup, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceId, + String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (deviceSecurityGroupName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); } if (deviceSecurityGroup == null) { return Mono @@ -475,23 +417,16 @@ private Mono> createOrUpdateWithResponseAsync final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - resourceId, - deviceSecurityGroupName, - deviceSecurityGroup, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, resourceId, deviceSecurityGroupName, + deviceSecurityGroup, accept, context); } /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -499,18 +434,18 @@ private Mono> createOrUpdateWithResponseAsync * @return the device security group resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceId, String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup) { + private Mono createOrUpdateAsync(String resourceId, String deviceSecurityGroupName, + DeviceSecurityGroupInner deviceSecurityGroup) { return createOrUpdateWithResponseAsync(resourceId, deviceSecurityGroupName, deviceSecurityGroup) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -519,21 +454,18 @@ private Mono createOrUpdateAsync( * @return the device security group resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceId, - String deviceSecurityGroupName, - DeviceSecurityGroupInner deviceSecurityGroup, - Context context) { + public Response createOrUpdateWithResponse(String resourceId, + String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup, Context context) { return createOrUpdateWithResponseAsync(resourceId, deviceSecurityGroupName, deviceSecurityGroup, context) .block(); } /** * Use this method to creates or updates the device security group on a specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param deviceSecurityGroup Security group object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -541,18 +473,18 @@ public Response createOrUpdateWithResponse( * @return the device security group resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DeviceSecurityGroupInner createOrUpdate( - String resourceId, String deviceSecurityGroupName, DeviceSecurityGroupInner deviceSecurityGroup) { + public DeviceSecurityGroupInner createOrUpdate(String resourceId, String deviceSecurityGroupName, + DeviceSecurityGroupInner deviceSecurityGroup) { return createOrUpdateWithResponse(resourceId, deviceSecurityGroupName, deviceSecurityGroup, Context.NONE) .getValue(); } /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -561,41 +493,30 @@ public DeviceSecurityGroupInner createOrUpdate( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceId, String deviceSecurityGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (deviceSecurityGroupName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - resourceId, - deviceSecurityGroupName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, resourceId, + deviceSecurityGroupName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -603,35 +524,32 @@ private Mono> deleteWithResponseAsync(String resourceId, String d * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceId, String deviceSecurityGroupName, Context context) { + private Mono> deleteWithResponseAsync(String resourceId, String deviceSecurityGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (resourceId == null) { return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); } if (deviceSecurityGroupName == null) { - return Mono - .error( - new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter deviceSecurityGroupName is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete(this.client.getEndpoint(), apiVersion, resourceId, deviceSecurityGroupName, accept, context); + return service.delete(this.client.getEndpoint(), apiVersion, resourceId, deviceSecurityGroupName, accept, + context); } /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -644,10 +562,10 @@ private Mono deleteAsync(String resourceId, String deviceSecurityGroupName /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -661,10 +579,10 @@ public Response deleteWithResponse(String resourceId, String deviceSecurit /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -676,9 +594,10 @@ public void delete(String resourceId, String deviceSecurityGroupName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -690,31 +609,22 @@ private Mono> listNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -727,23 +637,13 @@ private Mono> listNextSinglePageAsync(St return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsImpl.java index 84a677f1ee3e8..d2941c4ea197b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DeviceSecurityGroupsImpl.java @@ -21,31 +21,28 @@ public final class DeviceSecurityGroupsImpl implements DeviceSecurityGroups { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public DeviceSecurityGroupsImpl( - DeviceSecurityGroupsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public DeviceSecurityGroupsImpl(DeviceSecurityGroupsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String resourceId) { PagedIterable inner = this.serviceClient().list(resourceId); - return Utils.mapPage(inner, inner1 -> new DeviceSecurityGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceSecurityGroupImpl(inner1, this.manager())); } public PagedIterable list(String resourceId, Context context) { PagedIterable inner = this.serviceClient().list(resourceId, context); - return Utils.mapPage(inner, inner1 -> new DeviceSecurityGroupImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new DeviceSecurityGroupImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceId, String deviceSecurityGroupName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceId, deviceSecurityGroupName, context); + public Response getWithResponse(String resourceId, String deviceSecurityGroupName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceId, deviceSecurityGroupName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DeviceSecurityGroupImpl(inner.getValue(), this.manager())); } else { return null; @@ -61,8 +58,8 @@ public DeviceSecurityGroup get(String resourceId, String deviceSecurityGroupName } } - public Response deleteByResourceGroupWithResponse( - String resourceId, String deviceSecurityGroupName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceId, String deviceSecurityGroupName, + Context context) { return this.serviceClient().deleteWithResponse(resourceId, deviceSecurityGroupName, context); } @@ -71,125 +68,69 @@ public void deleteByResourceGroup(String resourceId, String deviceSecurityGroupN } public DeviceSecurityGroup getById(String id) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String deviceSecurityGroupName = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "deviceSecurityGroupName"); + String deviceSecurityGroupName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", + "deviceSecurityGroupName"); if (deviceSecurityGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", id))); } return this.getWithResponse(resourceId, deviceSecurityGroupName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String deviceSecurityGroupName = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "deviceSecurityGroupName"); + String deviceSecurityGroupName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", + "deviceSecurityGroupName"); if (deviceSecurityGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", id))); } return this.getWithResponse(resourceId, deviceSecurityGroupName, context); } public void deleteById(String id) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String deviceSecurityGroupName = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "deviceSecurityGroupName"); + String deviceSecurityGroupName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", + "deviceSecurityGroupName"); if (deviceSecurityGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", id))); } this.deleteByResourceGroupWithResponse(resourceId, deviceSecurityGroupName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceId = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "resourceId"); + String resourceId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", "resourceId"); if (resourceId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceId'.", id))); } - String deviceSecurityGroupName = - Utils - .getValueFromIdByParameterName( - id, - "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", - "deviceSecurityGroupName"); + String deviceSecurityGroupName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{resourceId}/providers/Microsoft.Security/deviceSecurityGroups/{deviceSecurityGroupName}", + "deviceSecurityGroupName"); if (deviceSecurityGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'deviceSecurityGroups'.", id))); } return this.deleteByResourceGroupWithResponse(resourceId, deviceSecurityGroupName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionImpl.java index 65e6eb8b1f5b9..55df609e646e7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionImpl.java @@ -13,8 +13,7 @@ public final class DiscoveredSecuritySolutionImpl implements DiscoveredSecurityS private final com.azure.resourcemanager.security.SecurityManager serviceManager; - DiscoveredSecuritySolutionImpl( - DiscoveredSecuritySolutionInner innerObject, + DiscoveredSecuritySolutionImpl(DiscoveredSecuritySolutionInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsClientImpl.java index ed28a5ae9ae9c..a770cf5d2571c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsClientImpl.java @@ -30,24 +30,28 @@ import com.azure.resourcemanager.security.models.DiscoveredSecuritySolutionList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in DiscoveredSecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in DiscoveredSecuritySolutionsClient. + */ public final class DiscoveredSecuritySolutionsClientImpl implements DiscoveredSecuritySolutionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final DiscoveredSecuritySolutionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of DiscoveredSecuritySolutionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ DiscoveredSecuritySolutionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - DiscoveredSecuritySolutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(DiscoveredSecuritySolutionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -58,154 +62,112 @@ public final class DiscoveredSecuritySolutionsClientImpl implements DiscoveredSe @Host("{$host}") @ServiceInterface(name = "SecurityCenterDiscov") public interface DiscoveredSecuritySolutionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/discoveredSecuritySolutions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/discoveredSecuritySolutions") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/discoveredSecuritySolutions") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByHomeRegion( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByHomeRegion(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/discoveredSecuritySolutions/{discoveredSecuritySolutionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/discoveredSecuritySolutions/{discoveredSecuritySolutionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("discoveredSecuritySolutionName") String discoveredSecuritySolutionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByHomeRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedFlux}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -214,27 +176,27 @@ private PagedFlux listAsync() { /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedFlux}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -243,13 +205,13 @@ public PagedIterable list() { /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -258,28 +220,24 @@ public PagedIterable list(Context context) { /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription and location along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of discovered Security Solutions for the subscription and location along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionSinglePageAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -287,54 +245,35 @@ private Mono> listByHomeRegionSin final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByHomeRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription and location along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of discovered Security Solutions for the subscription and location along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionSinglePageAsync( - String ascLocation, Context context) { + private Mono> listByHomeRegionSinglePageAsync(String ascLocation, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -343,66 +282,57 @@ private Mono> listByHomeRegionSin final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByHomeRegion( - this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink)); } /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation, Context context) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, context), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, context), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink, context)); } /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation) { @@ -411,15 +341,15 @@ public PagedIterable listByHomeRegion(String as /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation, Context context) { @@ -428,32 +358,28 @@ public PagedIterable listByHomeRegion(String as /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a specific discovered Security Solution along with {@link Response} on successful completion of {@link - * Mono}. + * @return a specific discovered Security Solution along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, String discoveredSecuritySolutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -463,58 +389,42 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (discoveredSecuritySolutionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter discoveredSecuritySolutionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter discoveredSecuritySolutionName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - discoveredSecuritySolutionName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, discoveredSecuritySolutionName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a specific discovered Security Solution along with {@link Response} on successful completion of {@link - * Mono}. + * @return a specific discovered Security Solution along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, String discoveredSecuritySolutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -524,33 +434,23 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (discoveredSecuritySolutionName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter discoveredSecuritySolutionName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter discoveredSecuritySolutionName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - discoveredSecuritySolutionName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, ascLocation, + discoveredSecuritySolutionName, apiVersion, accept, context); } /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -558,19 +458,19 @@ private Mono> getWithResponseAsync( * @return a specific discovered Security Solution on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName) { + private Mono getAsync(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName) { return getWithResponseAsync(resourceGroupName, ascLocation, discoveredSecuritySolutionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -579,18 +479,18 @@ private Mono getAsync( * @return a specific discovered Security Solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName, Context context) { + public Response getWithResponse(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName, Context context) { return getWithResponseAsync(resourceGroupName, ascLocation, discoveredSecuritySolutionName, context).block(); } /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -598,16 +498,17 @@ public Response getWithResponse( * @return a specific discovered Security Solution. */ @ServiceMethod(returns = ReturnType.SINGLE) - public DiscoveredSecuritySolutionInner get( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName) { + public DiscoveredSecuritySolutionInner get(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName) { return getWithResponse(resourceGroupName, ascLocation, discoveredSecuritySolutionName, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -619,31 +520,22 @@ private Mono> listNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -651,37 +543,28 @@ private Mono> listNextSinglePageA * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -693,31 +576,23 @@ private Mono> listByHomeRegionNex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -725,29 +600,19 @@ private Mono> listByHomeRegionNex * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByHomeRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsImpl.java index 3fc2fb690fe9c..83b4894ee9426 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/DiscoveredSecuritySolutionsImpl.java @@ -21,8 +21,7 @@ public final class DiscoveredSecuritySolutionsImpl implements DiscoveredSecurity private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public DiscoveredSecuritySolutionsImpl( - DiscoveredSecuritySolutionsClient innerClient, + public DiscoveredSecuritySolutionsImpl(DiscoveredSecuritySolutionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,46 +29,45 @@ public DiscoveredSecuritySolutionsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation, Context context) { - PagedIterable inner = - this.serviceClient().listByHomeRegion(ascLocation, context); - return Utils.mapPage(inner, inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByHomeRegion(ascLocation, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new DiscoveredSecuritySolutionImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, ascLocation, discoveredSecuritySolutionName, context); + public Response getWithResponse(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + ascLocation, discoveredSecuritySolutionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new DiscoveredSecuritySolutionImpl(inner.getValue(), this.manager())); } else { return null; } } - public DiscoveredSecuritySolution get( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName) { - DiscoveredSecuritySolutionInner inner = - this.serviceClient().get(resourceGroupName, ascLocation, discoveredSecuritySolutionName); + public DiscoveredSecuritySolution get(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName) { + DiscoveredSecuritySolutionInner inner + = this.serviceClient().get(resourceGroupName, ascLocation, discoveredSecuritySolutionName); if (inner != null) { return new DiscoveredSecuritySolutionImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionImpl.java index aca3c0f73e305..cdb69a6db7002 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionImpl.java @@ -12,8 +12,8 @@ public final class ExternalSecuritySolutionImpl implements ExternalSecuritySolut private final com.azure.resourcemanager.security.SecurityManager serviceManager; - ExternalSecuritySolutionImpl( - ExternalSecuritySolutionInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + ExternalSecuritySolutionImpl(ExternalSecuritySolutionInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsClientImpl.java index db970ccd5c33d..0b806c14a20d7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsClientImpl.java @@ -30,24 +30,28 @@ import com.azure.resourcemanager.security.models.ExternalSecuritySolutionList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ExternalSecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in ExternalSecuritySolutionsClient. + */ public final class ExternalSecuritySolutionsClientImpl implements ExternalSecuritySolutionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ExternalSecuritySolutionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ExternalSecuritySolutionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ExternalSecuritySolutionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - ExternalSecuritySolutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(ExternalSecuritySolutionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -58,150 +62,108 @@ public final class ExternalSecuritySolutionsClientImpl implements ExternalSecuri @Host("{$host}") @ServiceInterface(name = "SecurityCenterExtern") public interface ExternalSecuritySolutionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/externalSecuritySolutions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/ExternalSecuritySolutions") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/ExternalSecuritySolutions") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByHomeRegion( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByHomeRegion(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/ExternalSecuritySolutions/{externalSecuritySolutionsName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/ExternalSecuritySolutions/{externalSecuritySolutionsName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("externalSecuritySolutionsName") String externalSecuritySolutionsName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByHomeRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of external security solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of external security solutions for the subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of external security solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of external security solutions for the subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of external security solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of external security solutions for the subscription as paginated response with {@link PagedFlux}. @@ -213,7 +175,7 @@ private PagedFlux listAsync() { /** * Gets a list of external security solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -222,17 +184,17 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets a list of external security solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external security solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of external security solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -241,13 +203,13 @@ public PagedIterable list() { /** * Gets a list of external security solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external security solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of external security solutions for the subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -256,28 +218,24 @@ public PagedIterable list(Context context) { /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of external Security Solutions for the subscription and location along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionSinglePageAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -285,54 +243,35 @@ private Mono> listByHomeRegionSingl final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByHomeRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of external Security Solutions for the subscription and location along with {@link PagedResponse} - * on successful completion of {@link Mono}. + * on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionSinglePageAsync( - String ascLocation, Context context) { + private Mono> listByHomeRegionSinglePageAsync(String ascLocation, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -341,66 +280,57 @@ private Mono> listByHomeRegionSingl final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByHomeRegion( - this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedFlux}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink)); } /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedFlux}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation, Context context) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, context), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, context), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink, context)); } /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedIterable}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation) { @@ -409,15 +339,15 @@ public PagedIterable listByHomeRegion(String ascL /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedIterable}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation, Context context) { @@ -426,32 +356,28 @@ public PagedIterable listByHomeRegion(String ascL /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a specific external Security Solution along with {@link Response} on successful completion of {@link - * Mono}. + * @return a specific external Security Solution along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, String externalSecuritySolutionsName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -461,58 +387,42 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (externalSecuritySolutionsName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter externalSecuritySolutionsName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter externalSecuritySolutionsName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - externalSecuritySolutionsName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, externalSecuritySolutionsName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a specific external Security Solution along with {@link Response} on successful completion of {@link - * Mono}. + * @return a specific external Security Solution along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, String externalSecuritySolutionsName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -522,33 +432,23 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (externalSecuritySolutionsName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter externalSecuritySolutionsName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter externalSecuritySolutionsName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - externalSecuritySolutionsName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, ascLocation, + externalSecuritySolutionsName, apiVersion, accept, context); } /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -556,19 +456,19 @@ private Mono> getWithResponseAsync( * @return a specific external Security Solution on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName) { + private Mono getAsync(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName) { return getWithResponseAsync(resourceGroupName, ascLocation, externalSecuritySolutionsName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -577,18 +477,18 @@ private Mono getAsync( * @return a specific external Security Solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName, Context context) { + public Response getWithResponse(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName, Context context) { return getWithResponseAsync(resourceGroupName, ascLocation, externalSecuritySolutionsName, context).block(); } /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -596,16 +496,17 @@ public Response getWithResponse( * @return a specific external Security Solution. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ExternalSecuritySolutionInner get( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName) { + public ExternalSecuritySolutionInner get(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName) { return getWithResponse(resourceGroupName, ascLocation, externalSecuritySolutionsName, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -617,31 +518,22 @@ private Mono> listNextSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -649,37 +541,28 @@ private Mono> listNextSinglePageAsy * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -691,31 +574,23 @@ private Mono> listByHomeRegionNextS return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -723,29 +598,19 @@ private Mono> listByHomeRegionNextS * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByHomeRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsImpl.java index 27d2cba5cf911..cd21d6455bc05 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ExternalSecuritySolutionsImpl.java @@ -21,8 +21,7 @@ public final class ExternalSecuritySolutionsImpl implements ExternalSecuritySolu private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ExternalSecuritySolutionsImpl( - ExternalSecuritySolutionsClient innerClient, + public ExternalSecuritySolutionsImpl(ExternalSecuritySolutionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,46 +29,41 @@ public ExternalSecuritySolutionsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation, Context context) { - PagedIterable inner = - this.serviceClient().listByHomeRegion(ascLocation, context); - return Utils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByHomeRegion(ascLocation, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ExternalSecuritySolutionImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, ascLocation, externalSecuritySolutionsName, context); + public Response getWithResponse(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + ascLocation, externalSecuritySolutionsName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ExternalSecuritySolutionImpl(inner.getValue(), this.manager())); } else { return null; } } - public ExternalSecuritySolution get( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName) { - ExternalSecuritySolutionInner inner = - this.serviceClient().get(resourceGroupName, ascLocation, externalSecuritySolutionsName); + public ExternalSecuritySolution get(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName) { + ExternalSecuritySolutionInner inner + = this.serviceClient().get(resourceGroupName, ascLocation, externalSecuritySolutionsName); if (inner != null) { return new ExternalSecuritySolutionImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsListResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsListResponseImpl.java new file mode 100644 index 0000000000000..403c137feb0d0 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsListResponseImpl.java @@ -0,0 +1,44 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsListResponseInner; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsListResponse; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsResponse; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public final class GetSensitivitySettingsListResponseImpl implements GetSensitivitySettingsListResponse { + private GetSensitivitySettingsListResponseInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GetSensitivitySettingsListResponseImpl(GetSensitivitySettingsListResponseInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public List value() { + List inner = this.innerModel().value(); + if (inner != null) { + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new GetSensitivitySettingsResponseImpl(inner1, this.manager())) + .collect(Collectors.toList())); + } else { + return Collections.emptyList(); + } + } + + public GetSensitivitySettingsListResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsResponseImpl.java new file mode 100644 index 0000000000000..15134d6c65c67 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GetSensitivitySettingsResponseImpl.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsResponse; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsResponseProperties; + +public final class GetSensitivitySettingsResponseImpl implements GetSensitivitySettingsResponse { + private GetSensitivitySettingsResponseInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GetSensitivitySettingsResponseImpl(GetSensitivitySettingsResponseInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public GetSensitivitySettingsResponseProperties properties() { + return this.innerModel().properties(); + } + + public GetSensitivitySettingsResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerImpl.java new file mode 100644 index 0000000000000..c8983ce0b00d9 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerInner; +import com.azure.resourcemanager.security.models.GitHubOwner; +import com.azure.resourcemanager.security.models.GitHubOwnerProperties; + +public final class GitHubOwnerImpl implements GitHubOwner { + private GitHubOwnerInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GitHubOwnerImpl(GitHubOwnerInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public GitHubOwnerProperties properties() { + return this.innerModel().properties(); + } + + public GitHubOwnerInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerListResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerListResponseImpl.java new file mode 100644 index 0000000000000..6d727c5be2593 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnerListResponseImpl.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerInner; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerListResponseInner; +import com.azure.resourcemanager.security.models.GitHubOwner; +import com.azure.resourcemanager.security.models.GitHubOwnerListResponse; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public final class GitHubOwnerListResponseImpl implements GitHubOwnerListResponse { + private GitHubOwnerListResponseInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GitHubOwnerListResponseImpl(GitHubOwnerListResponseInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public List value() { + List inner = this.innerModel().value(); + if (inner != null) { + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new GitHubOwnerImpl(inner1, this.manager())).collect(Collectors.toList())); + } else { + return Collections.emptyList(); + } + } + + public String nextLink() { + return this.innerModel().nextLink(); + } + + public GitHubOwnerListResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersClientImpl.java new file mode 100644 index 0000000000000..dc2f11141c5da --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersClientImpl.java @@ -0,0 +1,566 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.GitHubOwnersClient; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerInner; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerListResponseInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in GitHubOwnersClient. + */ +public final class GitHubOwnersClientImpl implements GitHubOwnersClient { + /** + * The proxy service used to perform REST calls. + */ + private final GitHubOwnersService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of GitHubOwnersClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + GitHubOwnersClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(GitHubOwnersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterGitHubOwners to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterGitHub") + public interface GitHubOwnersService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/listAvailableGitHubOwners") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listAvailable(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitHubOwners") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitHubOwners/{ownerName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("ownerName") String ownerName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listAvailableWithResponseAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listAvailable(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listAvailableWithResponseAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listAvailable(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, accept, context); + } + + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAvailableAsync(String resourceGroupName, + String securityConnectorName) { + return listAvailableWithResponseAsync(resourceGroupName, securityConnectorName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + return listAvailableWithResponseAsync(resourceGroupName, securityConnectorName, context).block(); + } + + /** + * Returns a list of all GitHub owners accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitHubOwnerListResponseInner listAvailable(String resourceGroupName, String securityConnectorName) { + return listAvailableWithResponse(resourceGroupName, securityConnectorName, Context.NONE).getValue(); + } + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName)); + } + + /** + * Returns a list of GitHub owners onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, context)); + } + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String ownerName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (ownerName == null) { + return Mono.error(new IllegalArgumentException("Parameter ownerName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, ownerName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String ownerName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (ownerName == null) { + return Mono.error(new IllegalArgumentException("Parameter ownerName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, ownerName, apiVersion, accept, context); + } + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, String ownerName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, ownerName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String ownerName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, ownerName, context).block(); + } + + /** + * Returns a monitored GitHub owner. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Owner resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitHubOwnerInner get(String resourceGroupName, String securityConnectorName, String ownerName) { + return getWithResponse(resourceGroupName, securityConnectorName, ownerName, Context.NONE).getValue(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersImpl.java new file mode 100644 index 0000000000000..8408b8f3bfec7 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubOwnersImpl.java @@ -0,0 +1,93 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.GitHubOwnersClient; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerInner; +import com.azure.resourcemanager.security.fluent.models.GitHubOwnerListResponseInner; +import com.azure.resourcemanager.security.models.GitHubOwner; +import com.azure.resourcemanager.security.models.GitHubOwnerListResponse; +import com.azure.resourcemanager.security.models.GitHubOwners; + +public final class GitHubOwnersImpl implements GitHubOwners { + private static final ClientLogger LOGGER = new ClientLogger(GitHubOwnersImpl.class); + + private final GitHubOwnersClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public GitHubOwnersImpl(GitHubOwnersClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + Response inner + = this.serviceClient().listAvailableWithResponse(resourceGroupName, securityConnectorName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitHubOwnerListResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitHubOwnerListResponse listAvailable(String resourceGroupName, String securityConnectorName) { + GitHubOwnerListResponseInner inner + = this.serviceClient().listAvailable(resourceGroupName, securityConnectorName); + if (inner != null) { + return new GitHubOwnerListResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, securityConnectorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitHubOwnerImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitHubOwnerImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String ownerName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, securityConnectorName, ownerName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitHubOwnerImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitHubOwner get(String resourceGroupName, String securityConnectorName, String ownerName) { + GitHubOwnerInner inner = this.serviceClient().get(resourceGroupName, securityConnectorName, ownerName); + if (inner != null) { + return new GitHubOwnerImpl(inner, this.manager()); + } else { + return null; + } + } + + private GitHubOwnersClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposClientImpl.java new file mode 100644 index 0000000000000..1fadc2c71993f --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposClientImpl.java @@ -0,0 +1,458 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.GitHubReposClient; +import com.azure.resourcemanager.security.fluent.models.GitHubRepositoryInner; +import com.azure.resourcemanager.security.models.GitHubRepositoryListResponse; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in GitHubReposClient. + */ +public final class GitHubReposClientImpl implements GitHubReposClient { + /** + * The proxy service used to perform REST calls. + */ + private final GitHubReposService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of GitHubReposClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + GitHubReposClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(GitHubReposService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterGitHubRepos to be used by the proxy service to perform + * REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterGitHub") + public interface GitHubReposService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitHubOwners/{ownerName}/repos") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("ownerName") String ownerName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitHubOwners/{ownerName}/repos/{repoName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, @PathParam("ownerName") String ownerName, + @PathParam("repoName") String repoName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String ownerName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (ownerName == null) { + return Mono.error(new IllegalArgumentException("Parameter ownerName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, ownerName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String ownerName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (ownerName == null) { + return Mono.error(new IllegalArgumentException("Parameter ownerName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + ownerName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String ownerName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, ownerName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String ownerName, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, ownerName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String ownerName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, ownerName)); + } + + /** + * Returns a list of GitHub repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String ownerName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, ownerName, context)); + } + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String ownerName, String repoName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (ownerName == null) { + return Mono.error(new IllegalArgumentException("Parameter ownerName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, ownerName, repoName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String ownerName, String repoName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (ownerName == null) { + return Mono.error(new IllegalArgumentException("Parameter ownerName is required and cannot be null.")); + } + if (repoName == null) { + return Mono.error(new IllegalArgumentException("Parameter repoName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, ownerName, repoName, apiVersion, accept, context); + } + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String ownerName, String repoName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, ownerName, repoName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String ownerName, String repoName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, ownerName, repoName, context).block(); + } + + /** + * Returns a monitored GitHub repository. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param ownerName The GitHub owner name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitHub Repository resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitHubRepositoryInner get(String resourceGroupName, String securityConnectorName, String ownerName, + String repoName) { + return getWithResponse(resourceGroupName, securityConnectorName, ownerName, repoName, Context.NONE).getValue(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposImpl.java new file mode 100644 index 0000000000000..e8ab7aec520d0 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubReposImpl.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.GitHubReposClient; +import com.azure.resourcemanager.security.fluent.models.GitHubRepositoryInner; +import com.azure.resourcemanager.security.models.GitHubRepos; +import com.azure.resourcemanager.security.models.GitHubRepository; + +public final class GitHubReposImpl implements GitHubRepos { + private static final ClientLogger LOGGER = new ClientLogger(GitHubReposImpl.class); + + private final GitHubReposClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public GitHubReposImpl(GitHubReposClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String ownerName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, ownerName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitHubRepositoryImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String ownerName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, ownerName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitHubRepositoryImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String ownerName, String repoName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + securityConnectorName, ownerName, repoName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitHubRepositoryImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitHubRepository get(String resourceGroupName, String securityConnectorName, String ownerName, + String repoName) { + GitHubRepositoryInner inner + = this.serviceClient().get(resourceGroupName, securityConnectorName, ownerName, repoName); + if (inner != null) { + return new GitHubRepositoryImpl(inner, this.manager()); + } else { + return null; + } + } + + private GitHubReposClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubRepositoryImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubRepositoryImpl.java new file mode 100644 index 0000000000000..7a7ea6f4781e8 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitHubRepositoryImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.GitHubRepositoryInner; +import com.azure.resourcemanager.security.models.GitHubRepository; +import com.azure.resourcemanager.security.models.GitHubRepositoryProperties; + +public final class GitHubRepositoryImpl implements GitHubRepository { + private GitHubRepositoryInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GitHubRepositoryImpl(GitHubRepositoryInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public GitHubRepositoryProperties properties() { + return this.innerModel().properties(); + } + + public GitHubRepositoryInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupImpl.java new file mode 100644 index 0000000000000..76325ed4607fa --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupImpl.java @@ -0,0 +1,49 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupInner; +import com.azure.resourcemanager.security.models.GitLabGroup; +import com.azure.resourcemanager.security.models.GitLabGroupProperties; + +public final class GitLabGroupImpl implements GitLabGroup { + private GitLabGroupInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GitLabGroupImpl(GitLabGroupInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public GitLabGroupProperties properties() { + return this.innerModel().properties(); + } + + public GitLabGroupInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupListResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupListResponseImpl.java new file mode 100644 index 0000000000000..a5174724475eb --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupListResponseImpl.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.resourcemanager.security.fluent.models.GitLabGroupInner; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; +import com.azure.resourcemanager.security.models.GitLabGroup; +import com.azure.resourcemanager.security.models.GitLabGroupListResponse; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public final class GitLabGroupListResponseImpl implements GitLabGroupListResponse { + private GitLabGroupListResponseInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GitLabGroupListResponseImpl(GitLabGroupListResponseInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public List value() { + List inner = this.innerModel().value(); + if (inner != null) { + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new GitLabGroupImpl(inner1, this.manager())).collect(Collectors.toList())); + } else { + return Collections.emptyList(); + } + } + + public String nextLink() { + return this.innerModel().nextLink(); + } + + public GitLabGroupListResponseInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsClientImpl.java new file mode 100644 index 0000000000000..ce7601333102b --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsClientImpl.java @@ -0,0 +1,568 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.GitLabGroupsClient; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupInner; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in GitLabGroupsClient. + */ +public final class GitLabGroupsClientImpl implements GitLabGroupsClient { + /** + * The proxy service used to perform REST calls. + */ + private final GitLabGroupsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of GitLabGroupsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + GitLabGroupsClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(GitLabGroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterGitLabGroups to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterGitLab") + public interface GitLabGroupsService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/listAvailableGitLabGroups") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listAvailable(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitLabGroups") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitLabGroups/{groupFQName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @PathParam("groupFQName") String groupFQName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listAvailableWithResponseAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.listAvailable(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listAvailableWithResponseAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listAvailable(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, apiVersion, accept, context); + } + + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAvailableAsync(String resourceGroupName, + String securityConnectorName) { + return listAvailableWithResponseAsync(resourceGroupName, securityConnectorName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + return listAvailableWithResponseAsync(resourceGroupName, securityConnectorName, context).block(); + } + + /** + * Returns a list of all GitLab groups accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitLabGroupListResponseInner listAvailable(String resourceGroupName, String securityConnectorName) { + return listAvailableWithResponse(resourceGroupName, securityConnectorName, Context.NONE).getValue(); + } + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName)); + } + + /** + * Returns a list of GitLab groups onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, context)); + } + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String groupFQName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, groupFQName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String groupFQName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, groupFQName, apiVersion, accept, context); + } + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String groupFQName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, groupFQName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, groupFQName, context).block(); + } + + /** + * Returns a monitored GitLab Group resource for a given fully-qualified name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Group resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitLabGroupInner get(String resourceGroupName, String securityConnectorName, String groupFQName) { + return getWithResponse(resourceGroupName, securityConnectorName, groupFQName, Context.NONE).getValue(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsImpl.java new file mode 100644 index 0000000000000..d1480ba8ad7da --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabGroupsImpl.java @@ -0,0 +1,93 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.GitLabGroupsClient; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupInner; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; +import com.azure.resourcemanager.security.models.GitLabGroup; +import com.azure.resourcemanager.security.models.GitLabGroupListResponse; +import com.azure.resourcemanager.security.models.GitLabGroups; + +public final class GitLabGroupsImpl implements GitLabGroups { + private static final ClientLogger LOGGER = new ClientLogger(GitLabGroupsImpl.class); + + private final GitLabGroupsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public GitLabGroupsImpl(GitLabGroupsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + Response inner + = this.serviceClient().listAvailableWithResponse(resourceGroupName, securityConnectorName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitLabGroupListResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitLabGroupListResponse listAvailable(String resourceGroupName, String securityConnectorName) { + GitLabGroupListResponseInner inner + = this.serviceClient().listAvailable(resourceGroupName, securityConnectorName); + if (inner != null) { + return new GitLabGroupListResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName) { + PagedIterable inner = this.serviceClient().list(resourceGroupName, securityConnectorName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitLabGroupImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitLabGroupImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, securityConnectorName, groupFQName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitLabGroupImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitLabGroup get(String resourceGroupName, String securityConnectorName, String groupFQName) { + GitLabGroupInner inner = this.serviceClient().get(resourceGroupName, securityConnectorName, groupFQName); + if (inner != null) { + return new GitLabGroupImpl(inner, this.manager()); + } else { + return null; + } + } + + private GitLabGroupsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectImpl.java new file mode 100644 index 0000000000000..df5e3c2e83e6c --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectImpl.java @@ -0,0 +1,50 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.GitLabProjectInner; +import com.azure.resourcemanager.security.models.GitLabProject; +import com.azure.resourcemanager.security.models.GitLabProjectProperties; + +public final class GitLabProjectImpl implements GitLabProject { + private GitLabProjectInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + GitLabProjectImpl(GitLabProjectInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public GitLabProjectProperties properties() { + return this.innerModel().properties(); + } + + public GitLabProjectInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsClientImpl.java new file mode 100644 index 0000000000000..6fbdc3e23344f --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsClientImpl.java @@ -0,0 +1,466 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.GitLabProjectsClient; +import com.azure.resourcemanager.security.fluent.models.GitLabProjectInner; +import com.azure.resourcemanager.security.models.GitLabProjectListResponse; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in GitLabProjectsClient. + */ +public final class GitLabProjectsClientImpl implements GitLabProjectsClient { + /** + * The proxy service used to perform REST calls. + */ + private final GitLabProjectsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of GitLabProjectsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + GitLabProjectsClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(GitLabProjectsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterGitLabProjects to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterGitLab") + public interface GitLabProjectsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitLabGroups/{groupFQName}/projects") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @PathParam("groupFQName") String groupFQName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitLabGroups/{groupFQName}/projects/{projectName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @PathParam("groupFQName") String groupFQName, @PathParam("projectName") String projectName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector along + * with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String groupFQName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, groupFQName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector along + * with {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, String groupFQName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service + .list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, + groupFQName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector as + * paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String groupFQName) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, groupFQName), + nextLink -> listNextSinglePageAsync(nextLink)); + } + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector as + * paginated response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context) { + return new PagedFlux<>( + () -> listSinglePageAsync(resourceGroupName, securityConnectorName, groupFQName, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); + } + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector as + * paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String groupFQName) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, groupFQName)); + } + + /** + * Gets a list of GitLab projects that are directly owned by given group and onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of GitLab projects that are directly owned by given group and onboarded to the connector as + * paginated response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context) { + return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, groupFQName, context)); + } + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String groupFQName, String projectName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, groupFQName, projectName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String groupFQName, String projectName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + if (projectName == null) { + return Mono.error(new IllegalArgumentException("Parameter projectName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, groupFQName, projectName, apiVersion, accept, context); + } + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String groupFQName, String projectName) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, groupFQName, projectName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, String projectName, Context context) { + return getWithResponseAsync(resourceGroupName, securityConnectorName, groupFQName, projectName, context) + .block(); + } + + /** + * Returns a monitored GitLab Project resource for a given fully-qualified group name and project name. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return gitLab Project resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitLabProjectInner get(String resourceGroupName, String securityConnectorName, String groupFQName, + String projectName) { + return getWithResponse(resourceGroupName, securityConnectorName, groupFQName, projectName, Context.NONE) + .getValue(); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link PagedResponse} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsImpl.java new file mode 100644 index 0000000000000..320e60e0d33b8 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabProjectsImpl.java @@ -0,0 +1,74 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.GitLabProjectsClient; +import com.azure.resourcemanager.security.fluent.models.GitLabProjectInner; +import com.azure.resourcemanager.security.models.GitLabProject; +import com.azure.resourcemanager.security.models.GitLabProjects; + +public final class GitLabProjectsImpl implements GitLabProjects { + private static final ClientLogger LOGGER = new ClientLogger(GitLabProjectsImpl.class); + + private final GitLabProjectsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public GitLabProjectsImpl(GitLabProjectsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, + String groupFQName) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, groupFQName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitLabProjectImpl(inner1, this.manager())); + } + + public PagedIterable list(String resourceGroupName, String securityConnectorName, String groupFQName, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, groupFQName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GitLabProjectImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, String projectName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + securityConnectorName, groupFQName, projectName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitLabProjectImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitLabProject get(String resourceGroupName, String securityConnectorName, String groupFQName, + String projectName) { + GitLabProjectInner inner + = this.serviceClient().get(resourceGroupName, securityConnectorName, groupFQName, projectName); + if (inner != null) { + return new GitLabProjectImpl(inner, this.manager()); + } else { + return null; + } + } + + private GitLabProjectsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsClientImpl.java new file mode 100644 index 0000000000000..e41bff7ba5738 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsClientImpl.java @@ -0,0 +1,209 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Post; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.GitLabSubgroupsClient; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in GitLabSubgroupsClient. + */ +public final class GitLabSubgroupsClientImpl implements GitLabSubgroupsClient { + /** + * The proxy service used to perform REST calls. + */ + private final GitLabSubgroupsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of GitLabSubgroupsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + GitLabSubgroupsClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(GitLabSubgroupsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterGitLabSubgroups to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterGitLab") + public interface GitLabSubgroupsService { + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/devops/default/gitLabGroups/{groupFQName}/listSubgroups") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, + @PathParam("securityConnectorName") String securityConnectorName, + @PathParam("groupFQName") String groupFQName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector along with {@link Response} + * on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(String resourceGroupName, + String securityConnectorName, String groupFQName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, groupFQName, apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector along with {@link Response} + * on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(String resourceGroupName, + String securityConnectorName, String groupFQName, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (resourceGroupName == null) { + return Mono + .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); + } + if (securityConnectorName == null) { + return Mono + .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); + } + if (groupFQName == null) { + return Mono.error(new IllegalArgumentException("Parameter groupFQName is required and cannot be null.")); + } + final String apiVersion = "2023-09-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, groupFQName, apiVersion, accept, context); + } + + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAsync(String resourceGroupName, String securityConnectorName, + String groupFQName) { + return listWithResponseAsync(resourceGroupName, securityConnectorName, groupFQName) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listWithResponse(String resourceGroupName, + String securityConnectorName, String groupFQName, Context context) { + return listWithResponseAsync(resourceGroupName, securityConnectorName, groupFQName, context).block(); + } + + /** + * Gets nested subgroups of given GitLab Group which are onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param groupFQName The GitLab group fully-qualified name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return nested subgroups of given GitLab Group which are onboarded to the connector. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GitLabGroupListResponseInner list(String resourceGroupName, String securityConnectorName, + String groupFQName) { + return listWithResponse(resourceGroupName, securityConnectorName, groupFQName, Context.NONE).getValue(); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsImpl.java new file mode 100644 index 0000000000000..b7df246c9ce36 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GitLabSubgroupsImpl.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.GitLabSubgroupsClient; +import com.azure.resourcemanager.security.fluent.models.GitLabGroupListResponseInner; +import com.azure.resourcemanager.security.models.GitLabGroupListResponse; +import com.azure.resourcemanager.security.models.GitLabSubgroups; + +public final class GitLabSubgroupsImpl implements GitLabSubgroups { + private static final ClientLogger LOGGER = new ClientLogger(GitLabSubgroupsImpl.class); + + private final GitLabSubgroupsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public GitLabSubgroupsImpl(GitLabSubgroupsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response listWithResponse(String resourceGroupName, String securityConnectorName, + String groupFQName, Context context) { + Response inner + = this.serviceClient().listWithResponse(resourceGroupName, securityConnectorName, groupFQName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GitLabGroupListResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GitLabGroupListResponse list(String resourceGroupName, String securityConnectorName, String groupFQName) { + GitLabGroupListResponseInner inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, groupFQName); + if (inner != null) { + return new GitLabGroupListResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + private GitLabSubgroupsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentImpl.java index ec76254016772..90405669bbebc 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentImpl.java @@ -75,22 +75,15 @@ public GovernanceAssignmentImpl withExistingAssessment(String scope, String asse } public GovernanceAssignment create() { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceAssignments() - .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceAssignments() + .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), Context.NONE) + .getValue(); return this; } public GovernanceAssignment create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceAssignments() - .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceAssignments() + .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), context).getValue(); return this; } @@ -105,66 +98,42 @@ public GovernanceAssignmentImpl update() { } public GovernanceAssignment apply() { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceAssignments() - .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceAssignments() + .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), Context.NONE) + .getValue(); return this; } public GovernanceAssignment apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceAssignments() - .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceAssignments() + .createOrUpdateWithResponse(scope, assessmentName, assignmentKey, this.innerModel(), context).getValue(); return this; } - GovernanceAssignmentImpl( - GovernanceAssignmentInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + GovernanceAssignmentImpl(GovernanceAssignmentInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.scope = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "scope"); - this.assessmentName = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assessmentName"); - this.assignmentKey = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assignmentKey"); + this.scope = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "scope"); + this.assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assessmentName"); + this.assignmentKey = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assignmentKey"); } public GovernanceAssignment refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceAssignments() - .getWithResponse(scope, assessmentName, assignmentKey, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceAssignments() + .getWithResponse(scope, assessmentName, assignmentKey, Context.NONE).getValue(); return this; } public GovernanceAssignment refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceAssignments() - .getWithResponse(scope, assessmentName, assignmentKey, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceAssignments() + .getWithResponse(scope, assessmentName, assignmentKey, context).getValue(); return this; } @@ -188,8 +157,8 @@ public GovernanceAssignmentImpl withIsGracePeriod(Boolean isGracePeriod) { return this; } - public GovernanceAssignmentImpl withGovernanceEmailNotification( - GovernanceEmailNotification governanceEmailNotification) { + public GovernanceAssignmentImpl + withGovernanceEmailNotification(GovernanceEmailNotification governanceEmailNotification) { this.innerModel().withGovernanceEmailNotification(governanceEmailNotification); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsClientImpl.java index 2f964e6ef3ee5..fc42a58cdcdbf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsClientImpl.java @@ -33,23 +33,28 @@ import com.azure.resourcemanager.security.models.GovernanceAssignmentsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in GovernanceAssignmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in GovernanceAssignmentsClient. + */ public final class GovernanceAssignmentsClientImpl implements GovernanceAssignmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final GovernanceAssignmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of GovernanceAssignmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ GovernanceAssignmentsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(GovernanceAssignmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(GovernanceAssignmentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,89 +65,68 @@ public final class GovernanceAssignmentsClientImpl implements GovernanceAssignme @Host("{$host}") @ServiceInterface(name = "SecurityCenterGovern") public interface GovernanceAssignmentsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("assessmentName") String assessmentName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("assessmentName") String assessmentName, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("assessmentName") String assessmentName, - @PathParam("assignmentKey") String assignmentKey, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("assessmentName") String assessmentName, @PathParam("assignmentKey") String assignmentKey, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) @Put("/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("assessmentName") String assessmentName, - @PathParam("assignmentKey") String assignmentKey, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("assessmentName") String assessmentName, @PathParam("assignmentKey") String assignmentKey, @BodyParam("application/json") GovernanceAssignmentInner governanceAssignment, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) - @Delete( - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}") - @ExpectedResponses({200, 204}) + @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Delete("/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("assessmentName") String assessmentName, - @PathParam("assignmentKey") String assignmentKey, - Context context); - - @Headers({"Content-Type: application/json"}) + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("scope") String scope, @PathParam("assessmentName") String assessmentName, + @PathParam("assignmentKey") String assignmentKey, Context context); + + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return governance assignments on all of your resources inside a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, String assessmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -155,40 +139,31 @@ private Mono> listSinglePageAsync(Strin return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), apiVersion, scope, assessmentName, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return governance assignments on all of your resources inside a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String scope, String assessmentName, Context context) { + private Mono> listSinglePageAsync(String scope, String assessmentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -199,71 +174,62 @@ private Mono> listSinglePageAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, assessmentName, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, assessmentName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignments on all of your resources inside a scope as paginated response with {@link - * PagedFlux}. + * @return governance assignments on all of your resources inside a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, String assessmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, assessmentName), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, assessmentName), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignments on all of your resources inside a scope as paginated response with {@link - * PagedFlux}. + * @return governance assignments on all of your resources inside a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, String assessmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, assessmentName, context), + return new PagedFlux<>(() -> listSinglePageAsync(scope, assessmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignments on all of your resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return governance assignments on all of your resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String scope, String assessmentName) { @@ -272,17 +238,17 @@ public PagedIterable list(String scope, String assess /** * Get governance assignments on all of your resources inside a scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignments on all of your resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return governance assignments on all of your resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String scope, String assessmentName, Context context) { @@ -291,26 +257,24 @@ public PagedIterable list(String scope, String assess /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific governanceAssignment for the requested scope by AssignmentKey along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scope, String assessmentName, String assignmentKey) { + private Mono> getWithResponseAsync(String scope, String assessmentName, + String assignmentKey) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -324,26 +288,17 @@ private Mono> getWithResponseAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - scope, - assessmentName, - assignmentKey, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, scope, assessmentName, + assignmentKey, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param context The context to associate with this operation. @@ -351,16 +306,14 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific governanceAssignment for the requested scope by AssignmentKey along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scope, String assessmentName, String assignmentKey, Context context) { + private Mono> getWithResponseAsync(String scope, String assessmentName, + String assignmentKey, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -374,23 +327,23 @@ private Mono> getWithResponseAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, scope, assessmentName, assignmentKey, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, scope, assessmentName, assignmentKey, accept, + context); } /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific governanceAssignment for the requested scope by AssignmentKey on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync(String scope, String assessmentName, String assignmentKey) { @@ -400,10 +353,10 @@ private Mono getAsync(String scope, String assessment /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param context The context to associate with this operation. @@ -413,17 +366,17 @@ private Mono getAsync(String scope, String assessment * @return a specific governanceAssignment for the requested scope by AssignmentKey along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String scope, String assessmentName, String assignmentKey, Context context) { + public Response getWithResponse(String scope, String assessmentName, + String assignmentKey, Context context) { return getWithResponseAsync(scope, assessmentName, assignmentKey, context).block(); } /** * Get a specific governanceAssignment for the requested scope by AssignmentKey. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -438,27 +391,25 @@ public GovernanceAssignmentInner get(String scope, String assessmentName, String /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignment over a given scope along with {@link Response} on successful completion of {@link - * Mono}. + * @return governance assignment over a given scope along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String scope, String assessmentName, String assignmentKey, GovernanceAssignmentInner governanceAssignment) { + private Mono> createOrUpdateWithResponseAsync(String scope, + String assessmentName, String assignmentKey, GovernanceAssignmentInner governanceAssignment) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -478,27 +429,17 @@ private Mono> createOrUpdateWithResponseAsyn final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - scope, - assessmentName, - assignmentKey, - governanceAssignment, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, scope, assessmentName, + assignmentKey, governanceAssignment, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. @@ -506,21 +447,15 @@ private Mono> createOrUpdateWithResponseAsyn * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return governance assignment over a given scope along with {@link Response} on successful completion of {@link - * Mono}. + * @return governance assignment over a given scope along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String scope, - String assessmentName, - String assignmentKey, - GovernanceAssignmentInner governanceAssignment, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String scope, + String assessmentName, String assignmentKey, GovernanceAssignmentInner governanceAssignment, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -540,24 +475,16 @@ private Mono> createOrUpdateWithResponseAsyn final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - scope, - assessmentName, - assignmentKey, - governanceAssignment, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, scope, assessmentName, assignmentKey, + governanceAssignment, accept, context); } /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. @@ -567,18 +494,18 @@ private Mono> createOrUpdateWithResponseAsyn * @return governance assignment over a given scope on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String scope, String assessmentName, String assignmentKey, GovernanceAssignmentInner governanceAssignment) { + private Mono createOrUpdateAsync(String scope, String assessmentName, + String assignmentKey, GovernanceAssignmentInner governanceAssignment) { return createOrUpdateWithResponseAsync(scope, assessmentName, assignmentKey, governanceAssignment) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. @@ -589,22 +516,18 @@ private Mono createOrUpdateAsync( * @return governance assignment over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String scope, - String assessmentName, - String assignmentKey, - GovernanceAssignmentInner governanceAssignment, - Context context) { + public Response createOrUpdateWithResponse(String scope, String assessmentName, + String assignmentKey, GovernanceAssignmentInner governanceAssignment, Context context) { return createOrUpdateWithResponseAsync(scope, assessmentName, assignmentKey, governanceAssignment, context) .block(); } /** * Creates or updates a governance assignment on the given subscription. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param governanceAssignment Governance assignment over a subscription scope. @@ -614,18 +537,18 @@ public Response createOrUpdateWithResponse( * @return governance assignment over a given scope. */ @ServiceMethod(returns = ReturnType.SINGLE) - public GovernanceAssignmentInner createOrUpdate( - String scope, String assessmentName, String assignmentKey, GovernanceAssignmentInner governanceAssignment) { + public GovernanceAssignmentInner createOrUpdate(String scope, String assessmentName, String assignmentKey, + GovernanceAssignmentInner governanceAssignment) { return createOrUpdateWithResponse(scope, assessmentName, assignmentKey, governanceAssignment, Context.NONE) .getValue(); } /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -636,10 +559,8 @@ public GovernanceAssignmentInner createOrUpdate( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String scope, String assessmentName, String assignmentKey) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -652,19 +573,17 @@ private Mono> deleteWithResponseAsync(String scope, String assess } final String apiVersion = "2022-01-01-preview"; return FluxUtil - .withContext( - context -> - service - .delete(this.client.getEndpoint(), apiVersion, scope, assessmentName, assignmentKey, context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, scope, assessmentName, + assignmentKey, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param context The context to associate with this operation. @@ -674,13 +593,11 @@ private Mono> deleteWithResponseAsync(String scope, String assess * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String scope, String assessmentName, String assignmentKey, Context context) { + private Mono> deleteWithResponseAsync(String scope, String assessmentName, String assignmentKey, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -698,10 +615,10 @@ private Mono> deleteWithResponseAsync( /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -716,10 +633,10 @@ private Mono deleteAsync(String scope, String assessmentName, String assig /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @param context The context to associate with this operation. @@ -729,17 +646,17 @@ private Mono deleteAsync(String scope, String assessmentName, String assig * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String scope, String assessmentName, String assignmentKey, Context context) { + public Response deleteWithResponse(String scope, String assessmentName, String assignmentKey, + Context context) { return deleteWithResponseAsync(scope, assessmentName, assignmentKey, context).block(); } /** * Delete a GovernanceAssignment over a given scope. - * + * * @param scope The scope of the Governance assignments. Valid scopes are: subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param assessmentName The Assessment Key - A unique key for the assessment type. * @param assignmentKey The governance assignment key - the assessment key of the required governance assignment. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -753,14 +670,15 @@ public void delete(String scope, String assessmentName, String assignmentKey) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a governance assignments list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a governance assignments list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -768,37 +686,28 @@ private Mono> listNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a governance assignments list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a governance assignments list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -806,23 +715,13 @@ private Mono> listNextSinglePageAsync(S return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsImpl.java index d02b5180c6c05..54e9df9a52e2b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceAssignmentsImpl.java @@ -21,31 +21,28 @@ public final class GovernanceAssignmentsImpl implements GovernanceAssignments { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public GovernanceAssignmentsImpl( - GovernanceAssignmentsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public GovernanceAssignmentsImpl(GovernanceAssignmentsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String scope, String assessmentName) { PagedIterable inner = this.serviceClient().list(scope, assessmentName); - return Utils.mapPage(inner, inner1 -> new GovernanceAssignmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GovernanceAssignmentImpl(inner1, this.manager())); } public PagedIterable list(String scope, String assessmentName, Context context) { PagedIterable inner = this.serviceClient().list(scope, assessmentName, context); - return Utils.mapPage(inner, inner1 -> new GovernanceAssignmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GovernanceAssignmentImpl(inner1, this.manager())); } - public Response getWithResponse( - String scope, String assessmentName, String assignmentKey, Context context) { - Response inner = - this.serviceClient().getWithResponse(scope, assessmentName, assignmentKey, context); + public Response getWithResponse(String scope, String assessmentName, String assignmentKey, + Context context) { + Response inner + = this.serviceClient().getWithResponse(scope, assessmentName, assignmentKey, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new GovernanceAssignmentImpl(inner.getValue(), this.manager())); } else { return null; @@ -61,8 +58,8 @@ public GovernanceAssignment get(String scope, String assessmentName, String assi } } - public Response deleteWithResponse( - String scope, String assessmentName, String assignmentKey, Context context) { + public Response deleteWithResponse(String scope, String assessmentName, String assignmentKey, + Context context) { return this.serviceClient().deleteWithResponse(scope, assessmentName, assignmentKey, context); } @@ -71,173 +68,101 @@ public void delete(String scope, String assessmentName, String assignmentKey) { } public GovernanceAssignment getById(String id) { - String scope = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } - String assignmentKey = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assignmentKey"); + String assignmentKey = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assignmentKey"); if (assignmentKey == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", id))); } return this.getWithResponse(scope, assessmentName, assignmentKey, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String scope = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } - String assignmentKey = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assignmentKey"); + String assignmentKey = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assignmentKey"); if (assignmentKey == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", id))); } return this.getWithResponse(scope, assessmentName, assignmentKey, context); } public void deleteById(String id) { - String scope = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } - String assignmentKey = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assignmentKey"); + String assignmentKey = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assignmentKey"); if (assignmentKey == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", id))); } this.deleteWithResponse(scope, assessmentName, assignmentKey, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String scope = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String assessmentName = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assessmentName"); + String assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assessmentName"); if (assessmentName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'assessments'.", id))); } - String assignmentKey = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", - "assignmentKey"); + String assignmentKey = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/governanceAssignments/{assignmentKey}", + "assignmentKey"); if (assignmentKey == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceAssignments'.", id))); } return this.deleteWithResponse(scope, assessmentName, assignmentKey, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRuleImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRuleImpl.java index da9ab6f2522b3..8d633f3016568 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRuleImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRuleImpl.java @@ -121,22 +121,14 @@ public GovernanceRuleImpl withExistingScope(String scope) { } public GovernanceRule create() { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceRules() - .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceRules() + .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), Context.NONE).getValue(); return this; } public GovernanceRule create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceRules() - .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceRules() + .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), context).getValue(); return this; } @@ -151,48 +143,36 @@ public GovernanceRuleImpl update() { } public GovernanceRule apply() { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceRules() - .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceRules() + .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), Context.NONE).getValue(); return this; } public GovernanceRule apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getGovernanceRules() - .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceRules() + .createOrUpdateWithResponse(scope, ruleId, this.innerModel(), context).getValue(); return this; } - GovernanceRuleImpl( - GovernanceRuleInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + GovernanceRuleImpl(GovernanceRuleInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.scope = - Utils - .getValueFromIdByParameterName( - innerObject.id(), "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); - this.ruleId = - Utils - .getValueFromIdByParameterName( - innerObject.id(), "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); + this.scope = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); + this.ruleId = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); } public GovernanceRule refresh() { - this.innerObject = - serviceManager.serviceClient().getGovernanceRules().getWithResponse(scope, ruleId, Context.NONE).getValue(); + this.innerObject = serviceManager.serviceClient().getGovernanceRules() + .getWithResponse(scope, ruleId, Context.NONE).getValue(); return this; } public GovernanceRule refresh(Context context) { - this.innerObject = - serviceManager.serviceClient().getGovernanceRules().getWithResponse(scope, ruleId, context).getValue(); + this.innerObject + = serviceManager.serviceClient().getGovernanceRules().getWithResponse(scope, ruleId, context).getValue(); return this; } @@ -264,8 +244,8 @@ public GovernanceRuleImpl withOwnerSource(GovernanceRuleOwnerSource ownerSource) return this; } - public GovernanceRuleImpl withGovernanceEmailNotification( - GovernanceRuleEmailNotification governanceEmailNotification) { + public GovernanceRuleImpl + withGovernanceEmailNotification(GovernanceRuleEmailNotification governanceEmailNotification) { this.innerModel().withGovernanceEmailNotification(governanceEmailNotification); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesClientImpl.java index 011360e5c693e..9edfb6e6a036b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesClientImpl.java @@ -42,22 +42,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in GovernanceRulesClient. */ +/** + * An instance of this class provides access to all the operations defined in GovernanceRulesClient. + */ public final class GovernanceRulesClientImpl implements GovernanceRulesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final GovernanceRulesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of GovernanceRulesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ GovernanceRulesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(GovernanceRulesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(GovernanceRulesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -68,110 +74,84 @@ public final class GovernanceRulesClientImpl implements GovernanceRulesClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterGovern") public interface GovernanceRulesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/governanceRules") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("ruleId") String ruleId, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("ruleId") String ruleId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("ruleId") String ruleId, - @BodyParam("application/json") GovernanceRuleInner governanceRule, - @HeaderParam("Accept") String accept, - Context context); + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("ruleId") String ruleId, @BodyParam("application/json") GovernanceRuleInner governanceRule, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) + @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) @Delete("/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}") - @ExpectedResponses({200, 202, 204}) + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("ruleId") String ruleId, - Context context); + Mono>> delete(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("ruleId") String ruleId, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Post("/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}/execute") - @ExpectedResponses({202}) + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> execute( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, + Mono>> execute(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, @PathParam("ruleId") String ruleId, @BodyParam("application/json") ExecuteGovernanceRuleParams executeGovernanceRuleParams, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}/operationResults/{operationId}") - @ExpectedResponses({200, 202}) + @ExpectedResponses({ 200, 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono operationResults( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("scope") String scope, - @PathParam("ruleId") String ruleId, - @PathParam("operationId") String operationId, - @HeaderParam("Accept") String accept, - Context context); + Mono operationResults(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("scope") String scope, + @PathParam("ruleId") String ruleId, @PathParam("operationId") String operationId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant governance rules over a scope along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -180,39 +160,30 @@ private Mono> listSinglePageAsync(String scop final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant governance rules over a scope along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -220,26 +191,18 @@ private Mono> listSinglePageAsync(String scop final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -252,11 +215,11 @@ private PagedFlux listAsync(String scope) { /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -265,17 +228,17 @@ private PagedFlux listAsync(String scope) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -288,11 +251,11 @@ public PagedIterable list(String scope) { /** * Get a list of all relevant governance rules over a scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -306,25 +269,23 @@ public PagedIterable list(String scope, Context context) { /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific governance rule for the requested scope by ruleId along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String scope, String ruleId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -341,26 +302,24 @@ private Mono> getWithResponseAsync(String scope, S /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific governance rule for the requested scope by ruleId along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String scope, String ruleId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -376,11 +335,11 @@ private Mono> getWithResponseAsync(String scope, S /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -394,11 +353,11 @@ private Mono getAsync(String scope, String ruleId) { /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -413,11 +372,11 @@ public Response getWithResponse(String scope, String ruleId /** * Get a specific governance rule for the requested scope by ruleId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -431,11 +390,11 @@ public GovernanceRuleInner get(String scope, String ruleId) { /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -444,13 +403,11 @@ public GovernanceRuleInner get(String scope, String ruleId) { * @return governance rule over a given scope along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String scope, String ruleId, GovernanceRuleInner governanceRule) { + private Mono> createOrUpdateWithResponseAsync(String scope, String ruleId, + GovernanceRuleInner governanceRule) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -466,21 +423,18 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), apiVersion, scope, ruleId, governanceRule, accept, context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, scope, ruleId, + governanceRule, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @param context The context to associate with this operation. @@ -490,13 +444,11 @@ private Mono> createOrUpdateWithResponseAsync( * @return governance rule over a given scope along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String scope, String ruleId, GovernanceRuleInner governanceRule, Context context) { + private Mono> createOrUpdateWithResponseAsync(String scope, String ruleId, + GovernanceRuleInner governanceRule, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -512,17 +464,17 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate(this.client.getEndpoint(), apiVersion, scope, ruleId, governanceRule, accept, context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, scope, ruleId, governanceRule, accept, + context); } /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -531,19 +483,19 @@ private Mono> createOrUpdateWithResponseAsync( * @return governance rule over a given scope on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String scope, String ruleId, GovernanceRuleInner governanceRule) { + private Mono createOrUpdateAsync(String scope, String ruleId, + GovernanceRuleInner governanceRule) { return createOrUpdateWithResponseAsync(scope, ruleId, governanceRule) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @param context The context to associate with this operation. @@ -553,18 +505,18 @@ private Mono createOrUpdateAsync( * @return governance rule over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String scope, String ruleId, GovernanceRuleInner governanceRule, Context context) { + public Response createOrUpdateWithResponse(String scope, String ruleId, + GovernanceRuleInner governanceRule, Context context) { return createOrUpdateWithResponseAsync(scope, ruleId, governanceRule, context).block(); } /** * Creates or updates a governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param governanceRule Governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -579,11 +531,11 @@ public GovernanceRuleInner createOrUpdate(String scope, String ruleId, Governanc /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -593,10 +545,8 @@ public GovernanceRuleInner createOrUpdate(String scope, String ruleId, Governanc @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String scope, String ruleId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -612,11 +562,11 @@ private Mono>> deleteWithResponseAsync(String scope, S /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -627,10 +577,8 @@ private Mono>> deleteWithResponseAsync(String scope, S @ServiceMethod(returns = ReturnType.SINGLE) private Mono>> deleteWithResponseAsync(String scope, String ruleId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -645,11 +593,11 @@ private Mono>> deleteWithResponseAsync(String scope, S /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -659,19 +607,17 @@ private Mono>> deleteWithResponseAsync(String scope, S @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) private PollerFlux, Void> beginDeleteAsync(String scope, String ruleId) { Mono>> mono = deleteWithResponseAsync(scope, ruleId); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -683,18 +629,17 @@ private PollerFlux, Void> beginDeleteAsync(String scope, String private PollerFlux, Void> beginDeleteAsync(String scope, String ruleId, Context context) { context = this.client.mergeContext(context); Mono>> mono = deleteWithResponseAsync(scope, ruleId, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -708,11 +653,11 @@ public SyncPoller, Void> beginDelete(String scope, String ruleI /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -727,11 +672,11 @@ public SyncPoller, Void> beginDelete(String scope, String ruleI /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -745,11 +690,11 @@ private Mono deleteAsync(String scope, String ruleId) { /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -764,11 +709,11 @@ private Mono deleteAsync(String scope, String ruleId, Context context) { /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -781,11 +726,11 @@ public void delete(String scope, String ruleId) { /** * Delete a Governance rule over a given scope. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -799,11 +744,11 @@ public void delete(String scope, String ruleId, Context context) { /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -812,13 +757,11 @@ public void delete(String scope, String ruleId, Context context) { * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> executeWithResponseAsync( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams) { + private Mono>> executeWithResponseAsync(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -832,27 +775,18 @@ private Mono>> executeWithResponseAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .execute( - this.client.getEndpoint(), - apiVersion, - scope, - ruleId, - executeGovernanceRuleParams, - accept, - context)) + .withContext(context -> service.execute(this.client.getEndpoint(), apiVersion, scope, ruleId, + executeGovernanceRuleParams, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -862,13 +796,11 @@ private Mono>> executeWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> executeWithResponseAsync( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { + private Mono>> executeWithResponseAsync(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -882,18 +814,17 @@ private Mono>> executeWithResponseAsync( final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .execute( - this.client.getEndpoint(), apiVersion, scope, ruleId, executeGovernanceRuleParams, accept, context); + return service.execute(this.client.getEndpoint(), apiVersion, scope, ruleId, executeGovernanceRuleParams, + accept, context); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -902,22 +833,20 @@ private Mono>> executeWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginExecuteAsync( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams) { + private PollerFlux, Void> beginExecuteAsync(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams) { Mono>> mono = executeWithResponseAsync(scope, ruleId, executeGovernanceRuleParams); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -928,19 +857,17 @@ private PollerFlux, Void> beginExecuteAsync( private PollerFlux, Void> beginExecuteAsync(String scope, String ruleId) { final ExecuteGovernanceRuleParams executeGovernanceRuleParams = null; Mono>> mono = executeWithResponseAsync(scope, ruleId, executeGovernanceRuleParams); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -950,23 +877,22 @@ private PollerFlux, Void> beginExecuteAsync(String scope, Strin * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginExecuteAsync( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { + private PollerFlux, Void> beginExecuteAsync(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - executeWithResponseAsync(scope, ruleId, executeGovernanceRuleParams, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = executeWithResponseAsync(scope, ruleId, executeGovernanceRuleParams, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -981,11 +907,11 @@ public SyncPoller, Void> beginExecute(String scope, String rule /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -995,18 +921,18 @@ public SyncPoller, Void> beginExecute(String scope, String rule * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginExecute( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { + public SyncPoller, Void> beginExecute(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { return this.beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams, context).getSyncPoller(); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1015,20 +941,19 @@ public SyncPoller, Void> beginExecute( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono executeAsync( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams) { - return beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams) - .last() + private Mono executeAsync(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams) { + return beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1038,18 +963,17 @@ private Mono executeAsync( @ServiceMethod(returns = ReturnType.SINGLE) private Mono executeAsync(String scope, String ruleId) { final ExecuteGovernanceRuleParams executeGovernanceRuleParams = null; - return beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams) - .last() + return beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -1059,20 +983,19 @@ private Mono executeAsync(String scope, String ruleId) { * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono executeAsync( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { - return beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams, context) - .last() + private Mono executeAsync(String scope, String ruleId, + ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { + return beginExecuteAsync(scope, ruleId, executeGovernanceRuleParams, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1086,11 +1009,11 @@ public void execute(String scope, String ruleId) { /** * Execute a governance rule. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param executeGovernanceRuleParams Execute governance rule over a given scope. * @param context The context to associate with this operation. @@ -1099,34 +1022,32 @@ public void execute(String scope, String ruleId) { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void execute( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { + public void execute(String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, + Context context) { executeAsync(scope, ruleId, executeGovernanceRuleParams, context).block(); } /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return governance rules long run operation result for the requested scope by ruleId and operationId on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono operationResultsWithResponseAsync( - String scope, String ruleId, String operationId) { + private Mono operationResultsWithResponseAsync(String scope, String ruleId, + String operationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -1140,21 +1061,18 @@ private Mono operationResultsWithRespon final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .operationResults( - this.client.getEndpoint(), apiVersion, scope, ruleId, operationId, accept, context)) + .withContext(context -> service.operationResults(this.client.getEndpoint(), apiVersion, scope, ruleId, + operationId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @param context The context to associate with this operation. @@ -1162,16 +1080,14 @@ private Mono operationResultsWithRespon * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return governance rules long run operation result for the requested scope by ruleId and operationId on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono operationResultsWithResponseAsync( - String scope, String ruleId, String operationId, Context context) { + private Mono operationResultsWithResponseAsync(String scope, String ruleId, + String operationId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -1185,39 +1101,39 @@ private Mono operationResultsWithRespon final String apiVersion = "2022-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .operationResults(this.client.getEndpoint(), apiVersion, scope, ruleId, operationId, accept, context); + return service.operationResults(this.client.getEndpoint(), apiVersion, scope, ruleId, operationId, accept, + context); } /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return governance rules long run operation result for the requested scope by ruleId and operationId on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono operationResultsAsync( - String scope, String ruleId, String operationId) { + private Mono operationResultsAsync(String scope, String ruleId, + String operationId) { return operationResultsWithResponseAsync(scope, ruleId, operationId) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @param context The context to associate with this operation. @@ -1227,18 +1143,18 @@ private Mono operationResultsAsync( * @return governance rules long run operation result for the requested scope by ruleId and operationId. */ @ServiceMethod(returns = ReturnType.SINGLE) - public GovernanceRulesOperationResultsResponse operationResultsWithResponse( - String scope, String ruleId, String operationId, Context context) { + public GovernanceRulesOperationResultsResponse operationResultsWithResponse(String scope, String ruleId, + String operationId, Context context) { return operationResultsWithResponseAsync(scope, ruleId, operationId, context).block(); } /** * Get governance rules long run operation result for the requested scope by ruleId and operationId. - * + * * @param scope The scope of the Governance rules. Valid scopes are: management group (format: - * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: - * 'subscriptions/{subscriptionId}'), or security connector (format: - * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. + * 'providers/Microsoft.Management/managementGroups/{managementGroup}'), subscription (format: + * 'subscriptions/{subscriptionId}'), or security connector (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. * @param ruleId The governance rule key - unique key for the standard governance rule (GUID). * @param operationId The governance rule long running operation unique key. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1253,14 +1169,15 @@ public OperationResultAutoGeneratedInner operationResults(String scope, String r /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a governance rules list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a governance rules list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1268,37 +1185,28 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a governance rules list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a governance rules list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1306,23 +1214,13 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesImpl.java index 30860d8f1e41e..d225d2c02b165 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/GovernanceRulesImpl.java @@ -25,29 +25,26 @@ public final class GovernanceRulesImpl implements GovernanceRules { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public GovernanceRulesImpl( - GovernanceRulesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public GovernanceRulesImpl(GovernanceRulesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String scope) { PagedIterable inner = this.serviceClient().list(scope); - return Utils.mapPage(inner, inner1 -> new GovernanceRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GovernanceRuleImpl(inner1, this.manager())); } public PagedIterable list(String scope, Context context) { PagedIterable inner = this.serviceClient().list(scope, context); - return Utils.mapPage(inner, inner1 -> new GovernanceRuleImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new GovernanceRuleImpl(inner1, this.manager())); } public Response getWithResponse(String scope, String ruleId, Context context) { Response inner = this.serviceClient().getWithResponse(scope, ruleId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new GovernanceRuleImpl(inner.getValue(), this.manager())); } else { return null; @@ -75,20 +72,17 @@ public void execute(String scope, String ruleId) { this.serviceClient().execute(scope, ruleId); } - public void execute( - String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, Context context) { + public void execute(String scope, String ruleId, ExecuteGovernanceRuleParams executeGovernanceRuleParams, + Context context) { this.serviceClient().execute(scope, ruleId, executeGovernanceRuleParams, context); } - public Response operationResultsWithResponse( - String scope, String ruleId, String operationId, Context context) { - GovernanceRulesOperationResultsResponse inner = - this.serviceClient().operationResultsWithResponse(scope, ruleId, operationId, context); + public Response operationResultsWithResponse(String scope, String ruleId, + String operationId, Context context) { + GovernanceRulesOperationResultsResponse inner + = this.serviceClient().operationResultsWithResponse(scope, ruleId, operationId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new OperationResultAutoGeneratedImpl(inner.getValue(), this.manager())); } else { return null; @@ -105,101 +99,65 @@ public OperationResultAutoGenerated operationResults(String scope, String ruleId } public GovernanceRule getById(String id) { - String scope = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String ruleId = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); + String ruleId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); if (ruleId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); } return this.getWithResponse(scope, ruleId, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String scope = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String ruleId = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); + String ruleId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); if (ruleId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); } return this.getWithResponse(scope, ruleId, context); } public void deleteById(String id) { - String scope = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String ruleId = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); + String ruleId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); if (ruleId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); } this.delete(scope, ruleId, Context.NONE); } public void deleteByIdWithResponse(String id, Context context) { - String scope = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String ruleId = - Utils - .getValueFromIdByParameterName( - id, "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); + String ruleId = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/governanceRules/{ruleId}", "ruleId"); if (ruleId == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'governanceRules'.", id))); } this.delete(scope, ruleId, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportImpl.java index e93fab68ae8e3..f238c398a50cb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportImpl.java @@ -13,6 +13,7 @@ import com.azure.resourcemanager.security.models.StatusAutoGenerated; import java.util.Collections; import java.util.List; +import java.util.Map; public final class HealthReportImpl implements HealthReport { private HealthReportInner innerObject; @@ -61,6 +62,24 @@ public List affectedDefendersPlans() { } } + public List affectedDefendersSubPlans() { + List inner = this.innerModel().affectedDefendersSubPlans(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } + } + + public Map reportAdditionalData() { + Map inner = this.innerModel().reportAdditionalData(); + if (inner != null) { + return Collections.unmodifiableMap(inner); + } else { + return Collections.emptyMap(); + } + } + public List issues() { List inner = this.innerModel().issues(); if (inner != null) { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsClientImpl.java deleted file mode 100644 index d2c16f0a8577c..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsClientImpl.java +++ /dev/null @@ -1,180 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.Get; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.security.fluent.HealthReportOperationsClient; -import com.azure.resourcemanager.security.fluent.models.HealthReportInner; -import reactor.core.publisher.Mono; - -/** An instance of this class provides access to all the operations defined in HealthReportOperationsClient. */ -public final class HealthReportOperationsClientImpl implements HealthReportOperationsClient { - /** The proxy service used to perform REST calls. */ - private final HealthReportOperationsService service; - - /** The service client containing this operation class. */ - private final SecurityCenterImpl client; - - /** - * Initializes an instance of HealthReportOperationsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - HealthReportOperationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(HealthReportOperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for SecurityCenterHealthReportOperations to be used by the proxy service - * to perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "SecurityCenterHealth") - public interface HealthReportOperationsService { - @Headers({"Content-Type: application/json"}) - @Get("/{resourceId}/providers/Microsoft.Security/healthReports/{healthReportName}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @PathParam("healthReportName") String healthReportName, - @HeaderParam("Accept") String accept, - Context context); - } - - /** - * Get health report of resource. - * - * @param resourceId The identifier of the resource. - * @param healthReportName The health report Key - Unique key for the health report type. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return health report of resource along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String resourceId, String healthReportName) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (resourceId == null) { - return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); - } - if (healthReportName == null) { - return Mono - .error(new IllegalArgumentException("Parameter healthReportName is required and cannot be null.")); - } - final String apiVersion = "2023-02-01-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.get(this.client.getEndpoint(), apiVersion, resourceId, healthReportName, accept, context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get health report of resource. - * - * @param resourceId The identifier of the resource. - * @param healthReportName The health report Key - Unique key for the health report type. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return health report of resource along with {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceId, String healthReportName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (resourceId == null) { - return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); - } - if (healthReportName == null) { - return Mono - .error(new IllegalArgumentException("Parameter healthReportName is required and cannot be null.")); - } - final String apiVersion = "2023-02-01-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service.get(this.client.getEndpoint(), apiVersion, resourceId, healthReportName, accept, context); - } - - /** - * Get health report of resource. - * - * @param resourceId The identifier of the resource. - * @param healthReportName The health report Key - Unique key for the health report type. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return health report of resource on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String resourceId, String healthReportName) { - return getWithResponseAsync(resourceId, healthReportName).flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Get health report of resource. - * - * @param resourceId The identifier of the resource. - * @param healthReportName The health report Key - Unique key for the health report type. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return health report of resource along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String resourceId, String healthReportName, Context context) { - return getWithResponseAsync(resourceId, healthReportName, context).block(); - } - - /** - * Get health report of resource. - * - * @param resourceId The identifier of the resource. - * @param healthReportName The health report Key - Unique key for the health report type. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return health report of resource. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public HealthReportInner get(String resourceId, String healthReportName) { - return getWithResponse(resourceId, healthReportName, Context.NONE).getValue(); - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsImpl.java deleted file mode 100644 index 6824b37162f94..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportOperationsImpl.java +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.security.fluent.HealthReportOperationsClient; -import com.azure.resourcemanager.security.fluent.models.HealthReportInner; -import com.azure.resourcemanager.security.models.HealthReport; -import com.azure.resourcemanager.security.models.HealthReportOperations; - -public final class HealthReportOperationsImpl implements HealthReportOperations { - private static final ClientLogger LOGGER = new ClientLogger(HealthReportOperationsImpl.class); - - private final HealthReportOperationsClient innerClient; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - public HealthReportOperationsImpl( - HealthReportOperationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public Response getWithResponse(String resourceId, String healthReportName, Context context) { - Response inner = this.serviceClient().getWithResponse(resourceId, healthReportName, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new HealthReportImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public HealthReport get(String resourceId, String healthReportName) { - HealthReportInner inner = this.serviceClient().get(resourceId, healthReportName); - if (inner != null) { - return new HealthReportImpl(inner, this.manager()); - } else { - return null; - } - } - - private HealthReportOperationsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsClientImpl.java index 06047bcaac586..f1e296fdb5573 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.HealthReportsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in HealthReportsClient. */ +/** + * An instance of this class provides access to all the operations defined in HealthReportsClient. + */ public final class HealthReportsClientImpl implements HealthReportsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final HealthReportsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of HealthReportsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ HealthReportsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(HealthReportsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(HealthReportsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,64 +62,59 @@ public final class HealthReportsClientImpl implements HealthReportsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterHealth") public interface HealthReportsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/healthReports") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/{resourceId}/providers/Microsoft.Security/healthReports/{healthReportName}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @HeaderParam("Accept") String accept, + @PathParam(value = "resourceId", encoded = true) String resourceId, + @PathParam("healthReportName") String healthReportName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all health reports inside a scope along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); } - final String apiVersion = "2023-02-01-preview"; + final String apiVersion = "2023-05-01-preview"; final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -121,47 +122,37 @@ private Mono> listSinglePageAsync(String scope) * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all health reports inside a scope along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); } - final String apiVersion = "2023-02-01-preview"; + final String apiVersion = "2023-05-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -177,7 +168,7 @@ private PagedFlux listAsync(String scope) { * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -187,15 +178,15 @@ private PagedFlux listAsync(String scope) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -211,7 +202,7 @@ public PagedIterable list(String scope) { * Get a list of all health reports inside a scope. Valid scopes are: subscription (format: * 'subscriptions/{subscriptionId}'), or security connector (format: * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName})'. - * + * * @param scope The scope at which the operation is performed. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -224,11 +215,120 @@ public PagedIterable list(String scope, Context context) { return new PagedIterable<>(listAsync(scope, context)); } + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceId, String healthReportName) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (resourceId == null) { + return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); + } + if (healthReportName == null) { + return Mono + .error(new IllegalArgumentException("Parameter healthReportName is required and cannot be null.")); + } + final String apiVersion = "2023-05-01-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, resourceId, healthReportName, + accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource along with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getWithResponseAsync(String resourceId, String healthReportName, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (resourceId == null) { + return Mono.error(new IllegalArgumentException("Parameter resourceId is required and cannot be null.")); + } + if (healthReportName == null) { + return Mono + .error(new IllegalArgumentException("Parameter healthReportName is required and cannot be null.")); + } + final String apiVersion = "2023-05-01-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), apiVersion, resourceId, healthReportName, accept, context); + } + + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getAsync(String resourceId, String healthReportName) { + return getWithResponseAsync(resourceId, healthReportName).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getWithResponse(String resourceId, String healthReportName, Context context) { + return getWithResponseAsync(resourceId, healthReportName, context).block(); + } + + /** + * Get health report of resource. + * + * @param resourceId The identifier of the resource. + * @param healthReportName The health report Key - Unique key for the health report type. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return health report of resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public HealthReportInner get(String resourceId, String healthReportName) { + return getWithResponse(resourceId, healthReportName, Context.NONE).getValue(); + } + /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -240,31 +340,22 @@ private Mono> listNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -277,23 +368,13 @@ private Mono> listNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsImpl.java index cf87726c02728..4fea8a3b2f191 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/HealthReportsImpl.java @@ -5,6 +5,8 @@ package com.azure.resourcemanager.security.implementation; import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; import com.azure.core.util.Context; import com.azure.core.util.logging.ClientLogger; import com.azure.resourcemanager.security.fluent.HealthReportsClient; @@ -19,20 +21,39 @@ public final class HealthReportsImpl implements HealthReports { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public HealthReportsImpl( - HealthReportsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public HealthReportsImpl(HealthReportsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String scope) { PagedIterable inner = this.serviceClient().list(scope); - return Utils.mapPage(inner, inner1 -> new HealthReportImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new HealthReportImpl(inner1, this.manager())); } public PagedIterable list(String scope, Context context) { PagedIterable inner = this.serviceClient().list(scope, context); - return Utils.mapPage(inner, inner1 -> new HealthReportImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new HealthReportImpl(inner1, this.manager())); + } + + public Response getWithResponse(String resourceId, String healthReportName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceId, healthReportName, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new HealthReportImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public HealthReport get(String resourceId, String healthReportName) { + HealthReportInner inner = this.serviceClient().get(resourceId, healthReportName); + if (inner != null) { + return new HealthReportImpl(inner, this.manager()); + } else { + return null; + } } private HealthReportsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesClientImpl.java index c766c7e38b84a..23db3021a2bd3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesClientImpl.java @@ -33,26 +33,28 @@ import com.azure.resourcemanager.security.models.InformationProtectionPolicyName; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in InformationProtectionPoliciesClient. */ +/** + * An instance of this class provides access to all the operations defined in InformationProtectionPoliciesClient. + */ public final class InformationProtectionPoliciesClientImpl implements InformationProtectionPoliciesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final InformationProtectionPoliciesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of InformationProtectionPoliciesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ InformationProtectionPoliciesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - InformationProtectionPoliciesService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(InformationProtectionPoliciesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -63,60 +65,47 @@ public final class InformationProtectionPoliciesClientImpl implements Informatio @Host("{$host}") @ServiceInterface(name = "SecurityCenterInform") public interface InformationProtectionPoliciesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @PathParam("informationProtectionPolicyName") - InformationProtectionPolicyName informationProtectionPolicyName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @PathParam("informationProtectionPolicyName") InformationProtectionPolicyName informationProtectionPolicyName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @PathParam("informationProtectionPolicyName") - InformationProtectionPolicyName informationProtectionPolicyName, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @PathParam("informationProtectionPolicyName") InformationProtectionPolicyName informationProtectionPolicyName, @BodyParam("application/json") InformationProtectionPolicyInner informationProtectionPolicy, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/informationProtectionPolicies") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -124,44 +113,32 @@ Mono> listNext( * @return information protection policy along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scope, InformationProtectionPolicyName informationProtectionPolicyName) { + private Mono> getWithResponseAsync(String scope, + InformationProtectionPolicyName informationProtectionPolicyName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); } if (informationProtectionPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter informationProtectionPolicyName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter informationProtectionPolicyName is required and cannot be null.")); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - scope, - informationProtectionPolicyName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, scope, + informationProtectionPolicyName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -170,35 +147,31 @@ private Mono> getWithResponseAsync( * @return information protection policy along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scope, InformationProtectionPolicyName informationProtectionPolicyName, Context context) { + private Mono> getWithResponseAsync(String scope, + InformationProtectionPolicyName informationProtectionPolicyName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); } if (informationProtectionPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter informationProtectionPolicyName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter informationProtectionPolicyName is required and cannot be null.")); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, scope, informationProtectionPolicyName, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, scope, informationProtectionPolicyName, accept, + context); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -206,17 +179,17 @@ private Mono> getWithResponseAsync( * @return information protection policy on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String scope, InformationProtectionPolicyName informationProtectionPolicyName) { + private Mono getAsync(String scope, + InformationProtectionPolicyName informationProtectionPolicyName) { return getWithResponseAsync(scope, informationProtectionPolicyName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -225,16 +198,16 @@ private Mono getAsync( * @return information protection policy along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String scope, InformationProtectionPolicyName informationProtectionPolicyName, Context context) { + public Response getWithResponse(String scope, + InformationProtectionPolicyName informationProtectionPolicyName, Context context) { return getWithResponseAsync(scope, informationProtectionPolicyName, context).block(); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -242,16 +215,16 @@ public Response getWithResponse( * @return information protection policy. */ @ServiceMethod(returns = ReturnType.SINGLE) - public InformationProtectionPolicyInner get( - String scope, InformationProtectionPolicyName informationProtectionPolicyName) { + public InformationProtectionPolicyInner get(String scope, + InformationProtectionPolicyName informationProtectionPolicyName) { return getWithResponse(scope, informationProtectionPolicyName, Context.NONE).getValue(); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -260,55 +233,39 @@ public InformationProtectionPolicyInner get( * @return information protection policy along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String scope, + private Mono> createOrUpdateWithResponseAsync(String scope, InformationProtectionPolicyName informationProtectionPolicyName, InformationProtectionPolicyInner informationProtectionPolicy) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); } if (informationProtectionPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter informationProtectionPolicyName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter informationProtectionPolicyName is required and cannot be null.")); } if (informationProtectionPolicy == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter informationProtectionPolicy is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter informationProtectionPolicy is required and cannot be null.")); } else { informationProtectionPolicy.validate(); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - scope, - informationProtectionPolicyName, - informationProtectionPolicy, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, scope, + informationProtectionPolicyName, informationProtectionPolicy, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @param context The context to associate with this operation. @@ -318,53 +275,38 @@ private Mono> createOrUpdateWithRespo * @return information protection policy along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String scope, + private Mono> createOrUpdateWithResponseAsync(String scope, InformationProtectionPolicyName informationProtectionPolicyName, - InformationProtectionPolicyInner informationProtectionPolicy, - Context context) { + InformationProtectionPolicyInner informationProtectionPolicy, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); } if (informationProtectionPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter informationProtectionPolicyName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter informationProtectionPolicyName is required and cannot be null.")); } if (informationProtectionPolicy == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter informationProtectionPolicy is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter informationProtectionPolicy is required and cannot be null.")); } else { informationProtectionPolicy.validate(); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - scope, - informationProtectionPolicyName, - informationProtectionPolicy, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, scope, informationProtectionPolicyName, + informationProtectionPolicy, accept, context); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -373,8 +315,7 @@ private Mono> createOrUpdateWithRespo * @return information protection policy on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String scope, + private Mono createOrUpdateAsync(String scope, InformationProtectionPolicyName informationProtectionPolicyName, InformationProtectionPolicyInner informationProtectionPolicy) { return createOrUpdateWithResponseAsync(scope, informationProtectionPolicyName, informationProtectionPolicy) @@ -383,9 +324,9 @@ private Mono createOrUpdateAsync( /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @param context The context to associate with this operation. @@ -395,21 +336,18 @@ private Mono createOrUpdateAsync( * @return information protection policy along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String scope, + public Response createOrUpdateWithResponse(String scope, InformationProtectionPolicyName informationProtectionPolicyName, - InformationProtectionPolicyInner informationProtectionPolicy, - Context context) { - return createOrUpdateWithResponseAsync( - scope, informationProtectionPolicyName, informationProtectionPolicy, context) - .block(); + InformationProtectionPolicyInner informationProtectionPolicy, Context context) { + return createOrUpdateWithResponseAsync(scope, informationProtectionPolicyName, informationProtectionPolicy, + context).block(); } /** * Details of the information protection policy. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param informationProtectionPolicyName Name of the information protection policy. * @param informationProtectionPolicy Information protection policy. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -418,33 +356,29 @@ public Response createOrUpdateWithResponse( * @return information protection policy. */ @ServiceMethod(returns = ReturnType.SINGLE) - public InformationProtectionPolicyInner createOrUpdate( - String scope, + public InformationProtectionPolicyInner createOrUpdate(String scope, InformationProtectionPolicyName informationProtectionPolicyName, InformationProtectionPolicyInner informationProtectionPolicy) { - return createOrUpdateWithResponse( - scope, informationProtectionPolicyName, informationProtectionPolicy, Context.NONE) - .getValue(); + return createOrUpdateWithResponse(scope, informationProtectionPolicyName, informationProtectionPolicy, + Context.NONE).getValue(); } /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information protection policies response along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -453,37 +387,28 @@ private Mono> listSinglePageAsyn final String accept = "application/json"; return FluxUtil .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information protection policies response along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -491,24 +416,16 @@ private Mono> listSinglePageAsyn final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -521,9 +438,9 @@ private PagedFlux listAsync(String scope) { /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -532,15 +449,15 @@ private PagedFlux listAsync(String scope) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -553,9 +470,9 @@ public PagedIterable list(String scope) { /** * Information protection policies of a specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -569,14 +486,15 @@ public PagedIterable list(String scope, Contex /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information protection policies response along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -584,62 +502,43 @@ private Mono> listNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return information protection policies response along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesImpl.java index 708c43674c306..a4e09b7273f7a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPoliciesImpl.java @@ -22,30 +22,26 @@ public final class InformationProtectionPoliciesImpl implements InformationProte private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public InformationProtectionPoliciesImpl( - InformationProtectionPoliciesClient innerClient, + public InformationProtectionPoliciesImpl(InformationProtectionPoliciesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String scope, InformationProtectionPolicyName informationProtectionPolicyName, Context context) { - Response inner = - this.serviceClient().getWithResponse(scope, informationProtectionPolicyName, context); + public Response getWithResponse(String scope, + InformationProtectionPolicyName informationProtectionPolicyName, Context context) { + Response inner + = this.serviceClient().getWithResponse(scope, informationProtectionPolicyName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new InformationProtectionPolicyImpl(inner.getValue(), this.manager())); } else { return null; } } - public InformationProtectionPolicy get( - String scope, InformationProtectionPolicyName informationProtectionPolicyName) { + public InformationProtectionPolicy get(String scope, + InformationProtectionPolicyName informationProtectionPolicyName) { InformationProtectionPolicyInner inner = this.serviceClient().get(scope, informationProtectionPolicyName); if (inner != null) { return new InformationProtectionPolicyImpl(inner, this.manager()); @@ -56,79 +52,53 @@ public InformationProtectionPolicy get( public PagedIterable list(String scope) { PagedIterable inner = this.serviceClient().list(scope); - return Utils.mapPage(inner, inner1 -> new InformationProtectionPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new InformationProtectionPolicyImpl(inner1, this.manager())); } public PagedIterable list(String scope, Context context) { PagedIterable inner = this.serviceClient().list(scope, context); - return Utils.mapPage(inner, inner1 -> new InformationProtectionPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new InformationProtectionPolicyImpl(inner1, this.manager())); } public InformationProtectionPolicy getById(String id) { - String scope = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", - "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", + "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String informationProtectionPolicyNameLocal = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", - "informationProtectionPolicyName"); + String informationProtectionPolicyNameLocal = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", + "informationProtectionPolicyName"); if (informationProtectionPolicyNameLocal == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'informationProtectionPolicies'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String.format( + "The resource ID '%s' is not valid. Missing path segment 'informationProtectionPolicies'.", id))); } - InformationProtectionPolicyName informationProtectionPolicyName = - InformationProtectionPolicyName.fromString(informationProtectionPolicyNameLocal); + InformationProtectionPolicyName informationProtectionPolicyName + = InformationProtectionPolicyName.fromString(informationProtectionPolicyNameLocal); return this.getWithResponse(scope, informationProtectionPolicyName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String scope = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", - "scope"); + String scope = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", + "scope"); if (scope == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'scope'.", id))); } - String informationProtectionPolicyNameLocal = - Utils - .getValueFromIdByParameterName( - id, - "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", - "informationProtectionPolicyName"); + String informationProtectionPolicyNameLocal = ResourceManagerUtils.getValueFromIdByParameterName(id, + "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", + "informationProtectionPolicyName"); if (informationProtectionPolicyNameLocal == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment" - + " 'informationProtectionPolicies'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String.format( + "The resource ID '%s' is not valid. Missing path segment 'informationProtectionPolicies'.", id))); } - InformationProtectionPolicyName informationProtectionPolicyName = - InformationProtectionPolicyName.fromString(informationProtectionPolicyNameLocal); + InformationProtectionPolicyName informationProtectionPolicyName + = InformationProtectionPolicyName.fromString(informationProtectionPolicyNameLocal); return this.getWithResponse(scope, informationProtectionPolicyName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPolicyImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPolicyImpl.java index a470297b2563c..f09b0bbb51144 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPolicyImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/InformationProtectionPolicyImpl.java @@ -76,27 +76,20 @@ public InformationProtectionPolicyImpl withExistingScope(String scope) { } public InformationProtectionPolicy create() { - this.innerObject = - serviceManager - .serviceClient() - .getInformationProtectionPolicies() - .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getInformationProtectionPolicies() + .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), Context.NONE) + .getValue(); return this; } public InformationProtectionPolicy create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getInformationProtectionPolicies() - .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getInformationProtectionPolicies() + .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), context).getValue(); return this; } - InformationProtectionPolicyImpl( - InformationProtectionPolicyName name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + InformationProtectionPolicyImpl(InformationProtectionPolicyName name, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = new InformationProtectionPolicyInner(); this.serviceManager = serviceManager; this.informationProtectionPolicyName = name; @@ -107,63 +100,40 @@ public InformationProtectionPolicyImpl update() { } public InformationProtectionPolicy apply() { - this.innerObject = - serviceManager - .serviceClient() - .getInformationProtectionPolicies() - .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getInformationProtectionPolicies() + .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), Context.NONE) + .getValue(); return this; } public InformationProtectionPolicy apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getInformationProtectionPolicies() - .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getInformationProtectionPolicies() + .createOrUpdateWithResponse(scope, informationProtectionPolicyName, this.innerModel(), context).getValue(); return this; } - InformationProtectionPolicyImpl( - InformationProtectionPolicyInner innerObject, + InformationProtectionPolicyImpl(InformationProtectionPolicyInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.scope = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", - "scope"); - this.informationProtectionPolicyName = - InformationProtectionPolicyName - .fromString( - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", - "informationProtectionPolicyName")); + this.scope = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", + "scope"); + this.informationProtectionPolicyName = InformationProtectionPolicyName + .fromString(ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{scope}/providers/Microsoft.Security/informationProtectionPolicies/{informationProtectionPolicyName}", + "informationProtectionPolicyName")); } public InformationProtectionPolicy refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getInformationProtectionPolicies() - .getWithResponse(scope, informationProtectionPolicyName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getInformationProtectionPolicies() + .getWithResponse(scope, informationProtectionPolicyName, Context.NONE).getValue(); return this; } public InformationProtectionPolicy refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getInformationProtectionPolicies() - .getWithResponse(scope, informationProtectionPolicyName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getInformationProtectionPolicies() + .getWithResponse(scope, informationProtectionPolicyName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingImpl.java deleted file mode 100644 index 0100de28a14d0..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingImpl.java +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingInner; -import com.azure.resourcemanager.security.models.ConnectionStrings; -import com.azure.resourcemanager.security.models.IngestionSetting; -import com.azure.resourcemanager.security.models.IngestionSettingToken; - -public final class IngestionSettingImpl implements IngestionSetting, IngestionSetting.Definition { - private IngestionSettingInner innerObject; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - IngestionSettingImpl( - IngestionSettingInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public String id() { - return this.innerModel().id(); - } - - public String name() { - return this.innerModel().name(); - } - - public String type() { - return this.innerModel().type(); - } - - public Object properties() { - return this.innerModel().properties(); - } - - public IngestionSettingInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } - - private String ingestionSettingName; - - public IngestionSetting create() { - this.innerObject = - serviceManager - .serviceClient() - .getIngestionSettings() - .createWithResponse(ingestionSettingName, this.innerModel(), Context.NONE) - .getValue(); - return this; - } - - public IngestionSetting create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getIngestionSettings() - .createWithResponse(ingestionSettingName, this.innerModel(), context) - .getValue(); - return this; - } - - IngestionSettingImpl(String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerObject = new IngestionSettingInner(); - this.serviceManager = serviceManager; - this.ingestionSettingName = name; - } - - public IngestionSetting refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getIngestionSettings() - .getWithResponse(ingestionSettingName, Context.NONE) - .getValue(); - return this; - } - - public IngestionSetting refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getIngestionSettings() - .getWithResponse(ingestionSettingName, context) - .getValue(); - return this; - } - - public Response listTokensWithResponse(Context context) { - return serviceManager.ingestionSettings().listTokensWithResponse(ingestionSettingName, context); - } - - public IngestionSettingToken listTokens() { - return serviceManager.ingestionSettings().listTokens(ingestionSettingName); - } - - public Response listConnectionStringsWithResponse(Context context) { - return serviceManager.ingestionSettings().listConnectionStringsWithResponse(ingestionSettingName, context); - } - - public ConnectionStrings listConnectionStrings() { - return serviceManager.ingestionSettings().listConnectionStrings(ingestionSettingName); - } - - public IngestionSettingImpl withProperties(Object properties) { - this.innerModel().withProperties(properties); - return this; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingTokenImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingTokenImpl.java deleted file mode 100644 index 260f3548df8fc..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingTokenImpl.java +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.resourcemanager.security.fluent.models.IngestionSettingTokenInner; -import com.azure.resourcemanager.security.models.IngestionSettingToken; - -public final class IngestionSettingTokenImpl implements IngestionSettingToken { - private IngestionSettingTokenInner innerObject; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - IngestionSettingTokenImpl( - IngestionSettingTokenInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerObject = innerObject; - this.serviceManager = serviceManager; - } - - public String token() { - return this.innerModel().token(); - } - - public IngestionSettingTokenInner innerModel() { - return this.innerObject; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsClientImpl.java deleted file mode 100644 index 0ded97111c933..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsClientImpl.java +++ /dev/null @@ -1,1030 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.annotation.BodyParam; -import com.azure.core.annotation.Delete; -import com.azure.core.annotation.ExpectedResponses; -import com.azure.core.annotation.Get; -import com.azure.core.annotation.HeaderParam; -import com.azure.core.annotation.Headers; -import com.azure.core.annotation.Host; -import com.azure.core.annotation.HostParam; -import com.azure.core.annotation.PathParam; -import com.azure.core.annotation.Post; -import com.azure.core.annotation.Put; -import com.azure.core.annotation.QueryParam; -import com.azure.core.annotation.ReturnType; -import com.azure.core.annotation.ServiceInterface; -import com.azure.core.annotation.ServiceMethod; -import com.azure.core.annotation.UnexpectedResponseExceptionType; -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.RestProxy; -import com.azure.core.management.exception.ManagementException; -import com.azure.core.util.Context; -import com.azure.core.util.FluxUtil; -import com.azure.resourcemanager.security.fluent.IngestionSettingsClient; -import com.azure.resourcemanager.security.fluent.models.ConnectionStringsInner; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingInner; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingTokenInner; -import com.azure.resourcemanager.security.models.IngestionSettingList; -import reactor.core.publisher.Mono; - -/** An instance of this class provides access to all the operations defined in IngestionSettingsClient. */ -public final class IngestionSettingsClientImpl implements IngestionSettingsClient { - /** The proxy service used to perform REST calls. */ - private final IngestionSettingsService service; - - /** The service client containing this operation class. */ - private final SecurityCenterImpl client; - - /** - * Initializes an instance of IngestionSettingsClientImpl. - * - * @param client the instance of the service client containing this operation class. - */ - IngestionSettingsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(IngestionSettingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); - this.client = client; - } - - /** - * The interface defining all the services for SecurityCenterIngestionSettings to be used by the proxy service to - * perform REST calls. - */ - @Host("{$host}") - @ServiceInterface(name = "SecurityCenterIngest") - public interface IngestionSettingsService { - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/ingestionSettings") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/ingestionSettings/{ingestionSettingName}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ingestionSettingName") String ingestionSettingName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/ingestionSettings/{ingestionSettingName}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ingestionSettingName") String ingestionSettingName, - @BodyParam("application/json") IngestionSettingInner ingestionSetting, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/ingestionSettings/{ingestionSettingName}") - @ExpectedResponses({200, 204}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ingestionSettingName") String ingestionSettingName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/ingestionSettings/{ingestionSettingName}/listTokens") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listTokens( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ingestionSettingName") String ingestionSettingName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/ingestionSettings/{ingestionSettingName}/listConnectionStrings") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listConnectionStrings( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ingestionSettingName") String ingestionSettingName, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get("{nextLink}") - @ExpectedResponses({200}) - @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync() { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync(Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync() { - return new PagedFlux<>(() -> listSinglePageAsync(), nextLink -> listNextSinglePageAsync(nextLink)); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings as paginated response with {@link PagedFlux}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list() { - return new PagedIterable<>(listAsync()); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings as paginated response with {@link PagedIterable}. - */ - @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list(Context context) { - return new PagedIterable<>(listAsync(context)); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String ingestionSettingName) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String ingestionSettingName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String ingestionSettingName) { - return getWithResponseAsync(ingestionSettingName).flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String ingestionSettingName, Context context) { - return getWithResponseAsync(ingestionSettingName, context).block(); - } - - /** - * Settings for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public IngestionSettingInner get(String ingestionSettingName) { - return getWithResponse(ingestionSettingName, Context.NONE).getValue(); - } - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String ingestionSettingName, IngestionSettingInner ingestionSetting) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - if (ingestionSetting == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSetting is required and cannot be null.")); - } else { - ingestionSetting.validate(); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - ingestionSetting, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String ingestionSettingName, IngestionSettingInner ingestionSetting, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - if (ingestionSetting == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSetting is required and cannot be null.")); - } else { - ingestionSetting.validate(); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - ingestionSetting, - accept, - context); - } - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String ingestionSettingName, IngestionSettingInner ingestionSetting) { - return createWithResponseAsync(ingestionSettingName, ingestionSetting) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String ingestionSettingName, IngestionSettingInner ingestionSetting, Context context) { - return createWithResponseAsync(ingestionSettingName, ingestionSetting, context).block(); - } - - /** - * Create setting for ingesting security data and logs to correlate with resources associated with the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param ingestionSetting Ingestion setting object. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public IngestionSettingInner create(String ingestionSettingName, IngestionSettingInner ingestionSetting) { - return createWithResponse(ingestionSettingName, ingestionSetting, Context.NONE).getValue(); - } - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String ingestionSettingName) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String ingestionSettingName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context); - } - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return A {@link Mono} that completes when a successful response is received. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String ingestionSettingName) { - return deleteWithResponseAsync(ingestionSettingName).flatMap(ignored -> Mono.empty()); - } - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse(String ingestionSettingName, Context context) { - return deleteWithResponseAsync(ingestionSettingName, context).block(); - } - - /** - * Deletes the ingestion settings for this subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String ingestionSettingName) { - deleteWithResponse(ingestionSettingName, Context.NONE); - } - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listTokensWithResponseAsync(String ingestionSettingName) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listTokens( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listTokensWithResponseAsync( - String ingestionSettingName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listTokens( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context); - } - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription on - * successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listTokensAsync(String ingestionSettingName) { - return listTokensWithResponseAsync(ingestionSettingName).flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription along with - * {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response listTokensWithResponse(String ingestionSettingName, Context context) { - return listTokensWithResponseAsync(ingestionSettingName, context).block(); - } - - /** - * Returns the token that is used for correlating ingested telemetry with the resources in the subscription. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return configures how to correlate scan data and logs with resources associated with the subscription. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public IngestionSettingTokenInner listTokens(String ingestionSettingName) { - return listTokensWithResponse(ingestionSettingName, Context.NONE).getValue(); - } - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs along with {@link Response} on successful - * completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listConnectionStringsWithResponseAsync(String ingestionSettingName) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .listConnectionStrings( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs along with {@link Response} on successful - * completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listConnectionStringsWithResponseAsync( - String ingestionSettingName, Context context) { - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); - } - if (ingestionSettingName == null) { - return Mono - .error(new IllegalArgumentException("Parameter ingestionSettingName is required and cannot be null.")); - } - final String apiVersion = "2021-01-15-preview"; - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listConnectionStrings( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ingestionSettingName, - accept, - context); - } - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listConnectionStringsAsync(String ingestionSettingName) { - return listConnectionStringsWithResponseAsync(ingestionSettingName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); - } - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs along with {@link Response}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public Response listConnectionStringsWithResponse( - String ingestionSettingName, Context context) { - return listConnectionStringsWithResponseAsync(ingestionSettingName, context).block(); - } - - /** - * Connection strings for ingesting security scan logs and data. - * - * @param ingestionSettingName Name of the ingestion setting. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return connection string for ingesting security data and logs. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - public ConnectionStringsInner listConnectionStrings(String ingestionSettingName) { - return listConnectionStringsWithResponse(ingestionSettingName, Context.NONE).getValue(); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) - .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); - } - - /** - * Get the next page of items. - * - * @param nextLink The URL to get the next list of items - *

The nextLink parameter. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of ingestion settings along with {@link PagedResponse} on successful completion of {@link Mono}. - */ - @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync(String nextLink, Context context) { - if (nextLink == null) { - return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); - } - if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); - } - final String accept = "application/json"; - context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsImpl.java deleted file mode 100644 index da9c5111770eb..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IngestionSettingsImpl.java +++ /dev/null @@ -1,183 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.Response; -import com.azure.core.http.rest.SimpleResponse; -import com.azure.core.util.Context; -import com.azure.core.util.logging.ClientLogger; -import com.azure.resourcemanager.security.fluent.IngestionSettingsClient; -import com.azure.resourcemanager.security.fluent.models.ConnectionStringsInner; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingInner; -import com.azure.resourcemanager.security.fluent.models.IngestionSettingTokenInner; -import com.azure.resourcemanager.security.models.ConnectionStrings; -import com.azure.resourcemanager.security.models.IngestionSetting; -import com.azure.resourcemanager.security.models.IngestionSettingToken; -import com.azure.resourcemanager.security.models.IngestionSettings; - -public final class IngestionSettingsImpl implements IngestionSettings { - private static final ClientLogger LOGGER = new ClientLogger(IngestionSettingsImpl.class); - - private final IngestionSettingsClient innerClient; - - private final com.azure.resourcemanager.security.SecurityManager serviceManager; - - public IngestionSettingsImpl( - IngestionSettingsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { - this.innerClient = innerClient; - this.serviceManager = serviceManager; - } - - public PagedIterable list() { - PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new IngestionSettingImpl(inner1, this.manager())); - } - - public PagedIterable list(Context context) { - PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new IngestionSettingImpl(inner1, this.manager())); - } - - public Response getWithResponse(String ingestionSettingName, Context context) { - Response inner = this.serviceClient().getWithResponse(ingestionSettingName, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new IngestionSettingImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public IngestionSetting get(String ingestionSettingName) { - IngestionSettingInner inner = this.serviceClient().get(ingestionSettingName); - if (inner != null) { - return new IngestionSettingImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response deleteWithResponse(String ingestionSettingName, Context context) { - return this.serviceClient().deleteWithResponse(ingestionSettingName, context); - } - - public void delete(String ingestionSettingName) { - this.serviceClient().delete(ingestionSettingName); - } - - public Response listTokensWithResponse(String ingestionSettingName, Context context) { - Response inner = - this.serviceClient().listTokensWithResponse(ingestionSettingName, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new IngestionSettingTokenImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public IngestionSettingToken listTokens(String ingestionSettingName) { - IngestionSettingTokenInner inner = this.serviceClient().listTokens(ingestionSettingName); - if (inner != null) { - return new IngestionSettingTokenImpl(inner, this.manager()); - } else { - return null; - } - } - - public Response listConnectionStringsWithResponse(String ingestionSettingName, Context context) { - Response inner = - this.serviceClient().listConnectionStringsWithResponse(ingestionSettingName, context); - if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new ConnectionStringsImpl(inner.getValue(), this.manager())); - } else { - return null; - } - } - - public ConnectionStrings listConnectionStrings(String ingestionSettingName) { - ConnectionStringsInner inner = this.serviceClient().listConnectionStrings(ingestionSettingName); - if (inner != null) { - return new ConnectionStringsImpl(inner, this.manager()); - } else { - return null; - } - } - - public IngestionSetting getById(String id) { - String ingestionSettingName = Utils.getValueFromIdByName(id, "ingestionSettings"); - if (ingestionSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'ingestionSettings'.", id))); - } - return this.getWithResponse(ingestionSettingName, Context.NONE).getValue(); - } - - public Response getByIdWithResponse(String id, Context context) { - String ingestionSettingName = Utils.getValueFromIdByName(id, "ingestionSettings"); - if (ingestionSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'ingestionSettings'.", id))); - } - return this.getWithResponse(ingestionSettingName, context); - } - - public void deleteById(String id) { - String ingestionSettingName = Utils.getValueFromIdByName(id, "ingestionSettings"); - if (ingestionSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'ingestionSettings'.", id))); - } - this.deleteWithResponse(ingestionSettingName, Context.NONE); - } - - public Response deleteByIdWithResponse(String id, Context context) { - String ingestionSettingName = Utils.getValueFromIdByName(id, "ingestionSettings"); - if (ingestionSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'ingestionSettings'.", id))); - } - return this.deleteWithResponse(ingestionSettingName, context); - } - - private IngestionSettingsClient serviceClient() { - return this.innerClient; - } - - private com.azure.resourcemanager.security.SecurityManager manager() { - return this.serviceManager; - } - - public IngestionSettingImpl define(String name) { - return new IngestionSettingImpl(name, this.manager()); - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedAlertImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedAlertImpl.java index b6ba40c4985e0..261157ce0e67e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedAlertImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedAlertImpl.java @@ -18,8 +18,7 @@ public final class IoTSecurityAggregatedAlertImpl implements IoTSecurityAggregat private final com.azure.resourcemanager.security.SecurityManager serviceManager; - IoTSecurityAggregatedAlertImpl( - IoTSecurityAggregatedAlertInner innerObject, + IoTSecurityAggregatedAlertImpl(IoTSecurityAggregatedAlertInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedRecommendationImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedRecommendationImpl.java index 8b33589368a06..9ca207df4dd14 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedRecommendationImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecurityAggregatedRecommendationImpl.java @@ -15,8 +15,7 @@ public final class IoTSecurityAggregatedRecommendationImpl implements IoTSecurit private final com.azure.resourcemanager.security.SecurityManager serviceManager; - IoTSecurityAggregatedRecommendationImpl( - IoTSecurityAggregatedRecommendationInner innerObject, + IoTSecurityAggregatedRecommendationImpl(IoTSecurityAggregatedRecommendationInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelImpl.java index 40b21a2f14043..0672331599ce9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelImpl.java @@ -19,8 +19,7 @@ public final class IoTSecuritySolutionAnalyticsModelImpl implements IoTSecurityS private final com.azure.resourcemanager.security.SecurityManager serviceManager; - IoTSecuritySolutionAnalyticsModelImpl( - IoTSecuritySolutionAnalyticsModelInner innerObject, + IoTSecuritySolutionAnalyticsModelImpl(IoTSecuritySolutionAnalyticsModelInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelListImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelListImpl.java index f8cf0fe29c555..5a154b64a23e4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelListImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionAnalyticsModelListImpl.java @@ -17,8 +17,7 @@ public final class IoTSecuritySolutionAnalyticsModelListImpl implements IoTSecur private final com.azure.resourcemanager.security.SecurityManager serviceManager; - IoTSecuritySolutionAnalyticsModelListImpl( - IoTSecuritySolutionAnalyticsModelListInner innerObject, + IoTSecuritySolutionAnalyticsModelListImpl(IoTSecuritySolutionAnalyticsModelListInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,9 @@ public final class IoTSecuritySolutionAnalyticsModelListImpl implements IoTSecur public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new IoTSecuritySolutionAnalyticsModelImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new IoTSecuritySolutionAnalyticsModelImpl(inner1, this.manager())) + .collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionModelImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionModelImpl.java index 849907cb0bb67..ff865c0027bc0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionModelImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IoTSecuritySolutionModelImpl.java @@ -162,22 +162,14 @@ public IoTSecuritySolutionModelImpl withExistingResourceGroup(String resourceGro } public IoTSecuritySolutionModel create() { - this.innerObject = - serviceManager - .serviceClient() - .getIotSecuritySolutions() - .createOrUpdateWithResponse(resourceGroupName, solutionName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getIotSecuritySolutions() + .createOrUpdateWithResponse(resourceGroupName, solutionName, this.innerModel(), Context.NONE).getValue(); return this; } public IoTSecuritySolutionModel create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getIotSecuritySolutions() - .createOrUpdateWithResponse(resourceGroupName, solutionName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getIotSecuritySolutions() + .createOrUpdateWithResponse(resourceGroupName, solutionName, this.innerModel(), context).getValue(); return this; } @@ -193,50 +185,36 @@ public IoTSecuritySolutionModelImpl update() { } public IoTSecuritySolutionModel apply() { - this.innerObject = - serviceManager - .serviceClient() - .getIotSecuritySolutions() - .updateWithResponse(resourceGroupName, solutionName, updateUpdateIotSecuritySolutionData, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getIotSecuritySolutions() + .updateWithResponse(resourceGroupName, solutionName, updateUpdateIotSecuritySolutionData, Context.NONE) + .getValue(); return this; } public IoTSecuritySolutionModel apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getIotSecuritySolutions() - .updateWithResponse(resourceGroupName, solutionName, updateUpdateIotSecuritySolutionData, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getIotSecuritySolutions() + .updateWithResponse(resourceGroupName, solutionName, updateUpdateIotSecuritySolutionData, context) + .getValue(); return this; } - IoTSecuritySolutionModelImpl( - IoTSecuritySolutionModelInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + IoTSecuritySolutionModelImpl(IoTSecuritySolutionModelInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.solutionName = Utils.getValueFromIdByName(innerObject.id(), "iotSecuritySolutions"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.solutionName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "iotSecuritySolutions"); } public IoTSecuritySolutionModel refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getIotSecuritySolutions() - .getByResourceGroupWithResponse(resourceGroupName, solutionName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getIotSecuritySolutions() + .getByResourceGroupWithResponse(resourceGroupName, solutionName, Context.NONE).getValue(); return this; } public IoTSecuritySolutionModel refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getIotSecuritySolutions() - .getByResourceGroupWithResponse(resourceGroupName, solutionName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getIotSecuritySolutions() + .getByResourceGroupWithResponse(resourceGroupName, solutionName, context).getValue(); return this; } @@ -300,8 +278,8 @@ public IoTSecuritySolutionModelImpl withUserDefinedResources(UserDefinedResource } } - public IoTSecuritySolutionModelImpl withRecommendationsConfiguration( - List recommendationsConfiguration) { + public IoTSecuritySolutionModelImpl + withRecommendationsConfiguration(List recommendationsConfiguration) { if (isInCreateMode()) { this.innerModel().withRecommendationsConfiguration(recommendationsConfiguration); return this; @@ -316,8 +294,8 @@ public IoTSecuritySolutionModelImpl withUnmaskedIpLoggingStatus(UnmaskedIpLoggin return this; } - public IoTSecuritySolutionModelImpl withAdditionalWorkspaces( - List additionalWorkspaces) { + public IoTSecuritySolutionModelImpl + withAdditionalWorkspaces(List additionalWorkspaces) { this.innerModel().withAdditionalWorkspaces(additionalWorkspaces); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsClientImpl.java index d047f7931d33d..d7feb1c6f78a1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsClientImpl.java @@ -26,24 +26,28 @@ import com.azure.resourcemanager.security.fluent.models.IoTSecuritySolutionAnalyticsModelListInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in IotSecuritySolutionAnalyticsClient. */ +/** + * An instance of this class provides access to all the operations defined in IotSecuritySolutionAnalyticsClient. + */ public final class IotSecuritySolutionAnalyticsClientImpl implements IotSecuritySolutionAnalyticsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final IotSecuritySolutionAnalyticsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of IotSecuritySolutionAnalyticsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ IotSecuritySolutionAnalyticsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - IotSecuritySolutionAnalyticsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(IotSecuritySolutionAnalyticsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -54,61 +58,47 @@ public final class IotSecuritySolutionAnalyticsClientImpl implements IotSecurity @Host("{$host}") @ServiceInterface(name = "SecurityCenterIotSec") public interface IotSecuritySolutionAnalyticsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @HeaderParam("Accept") String accept, Context context); } /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of Security analytics of your IoT Security solution along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String resourceGroupName, String solutionName) { + private Mono> listWithResponseAsync(String resourceGroupName, + String solutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -120,47 +110,34 @@ private Mono> listWithRespo final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of Security analytics of your IoT Security solution along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String resourceGroupName, String solutionName, Context context) { + private Mono> listWithResponseAsync(String resourceGroupName, + String solutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -172,22 +149,15 @@ private Mono> listWithRespo final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, accept, context); } /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -201,9 +171,9 @@ private Mono listAsync(String resour /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -212,16 +182,16 @@ private Mono listAsync(String resour * @return list of Security analytics of your IoT Security solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listWithResponse( - String resourceGroupName, String solutionName, Context context) { + public Response listWithResponse(String resourceGroupName, + String solutionName, Context context) { return listWithResponseAsync(resourceGroupName, solutionName, context).block(); } /** * Use this method to get IoT security Analytics metrics in an array. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -235,30 +205,26 @@ public IoTSecuritySolutionAnalyticsModelListInner list(String resourceGroupName, /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security analytics of your IoT Security solution along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String solutionName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String solutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -270,47 +236,34 @@ private Mono> getWithResponseAs final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security analytics of your IoT Security solution along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String solutionName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String solutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -322,22 +275,15 @@ private Mono> getWithResponseAs final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, accept, context); } /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -351,9 +297,9 @@ private Mono getAsync(String resourceGro /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -362,16 +308,16 @@ private Mono getAsync(String resourceGro * @return security analytics of your IoT Security solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String solutionName, Context context) { + public Response getWithResponse(String resourceGroupName, + String solutionName, Context context) { return getWithResponseAsync(resourceGroupName, solutionName, context).block(); } /** * Use this method to get IoT Security Analytics metrics. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsImpl.java index 23c075d86a6d1..269039c13994d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionAnalyticsImpl.java @@ -22,22 +22,18 @@ public final class IotSecuritySolutionAnalyticsImpl implements IotSecuritySoluti private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public IotSecuritySolutionAnalyticsImpl( - IotSecuritySolutionAnalyticsClient innerClient, + public IotSecuritySolutionAnalyticsImpl(IotSecuritySolutionAnalyticsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response listWithResponse( - String resourceGroupName, String solutionName, Context context) { - Response inner = - this.serviceClient().listWithResponse(resourceGroupName, solutionName, context); + public Response listWithResponse(String resourceGroupName, + String solutionName, Context context) { + Response inner + = this.serviceClient().listWithResponse(resourceGroupName, solutionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new IoTSecuritySolutionAnalyticsModelListImpl(inner.getValue(), this.manager())); } else { return null; @@ -53,15 +49,12 @@ public IoTSecuritySolutionAnalyticsModelList list(String resourceGroupName, Stri } } - public Response getWithResponse( - String resourceGroupName, String solutionName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, solutionName, context); + public Response getWithResponse(String resourceGroupName, String solutionName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, solutionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new IoTSecuritySolutionAnalyticsModelImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl.java index 9ec5cec7cd575..6fc478c09d4f1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl.java @@ -37,24 +37,24 @@ */ public final class IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl implements IotSecuritySolutionsAnalyticsAggregatedAlertsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final IotSecuritySolutionsAnalyticsAggregatedAlertsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - IotSecuritySolutionsAnalyticsAggregatedAlertsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(IotSecuritySolutionsAnalyticsAggregatedAlertsService.class, + client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -65,89 +65,67 @@ public final class IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl @Host("{$host}") @ServiceInterface(name = "SecurityCenterIotSec") public interface IotSecuritySolutionsAnalyticsAggregatedAlertsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedAlerts") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedAlerts") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @QueryParam("$top") Integer top, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @QueryParam("$top") Integer top, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedAlerts/{aggregatedAlertName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedAlerts/{aggregatedAlertName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @PathParam("aggregatedAlertName") String aggregatedAlertName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @PathParam("aggregatedAlertName") String aggregatedAlertName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedAlerts/{aggregatedAlertName}/dismiss") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedAlerts/{aggregatedAlertName}/dismiss") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> dismiss( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> dismiss(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @PathParam("aggregatedAlertName") String aggregatedAlertName, - @HeaderParam("Accept") String accept, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @PathParam("aggregatedAlertName") String aggregatedAlertName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated alert data along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String solutionName, Integer top) { + private Mono> listSinglePageAsync(String resourceGroupName, + String solutionName, Integer top) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -159,35 +137,18 @@ private Mono> listSinglePageAsync final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - top, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, top, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. @@ -195,22 +156,18 @@ private Mono> listSinglePageAsync * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated alert data along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String solutionName, Integer top, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String solutionName, Integer top, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -223,31 +180,17 @@ private Mono> listSinglePageAsync final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - top, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, top, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -256,18 +199,17 @@ private Mono> listSinglePageAsync * @return list of IoT Security solution aggregated alert data as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String solutionName, Integer top) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, solutionName, top), + private PagedFlux listAsync(String resourceGroupName, String solutionName, + Integer top) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, solutionName, top), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -277,16 +219,15 @@ private PagedFlux listAsync( @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String solutionName) { final Integer top = null; - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, solutionName, top), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, solutionName, top), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. @@ -296,18 +237,17 @@ private PagedFlux listAsync(String resourceGrou * @return list of IoT Security solution aggregated alert data as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String solutionName, Integer top, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, solutionName, top, context), + private PagedFlux listAsync(String resourceGroupName, String solutionName, + Integer top, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, solutionName, top, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -322,9 +262,9 @@ public PagedIterable list(String resourceGroupN /** * Use this method to get the aggregated alert list of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. @@ -334,39 +274,35 @@ public PagedIterable list(String resourceGroupN * @return list of IoT Security solution aggregated alert data as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String solutionName, Integer top, Context context) { + public PagedIterable list(String resourceGroupName, String solutionName, + Integer top, Context context) { return new PagedIterable<>(listAsync(resourceGroupName, solutionName, top, context)); } /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security Solution Aggregated Alert information along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String solutionName, String aggregatedAlertName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String solutionName, String aggregatedAlertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -382,27 +318,17 @@ private Mono> getWithResponseAsync( final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - aggregatedAlertName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, aggregatedAlertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @param context The context to associate with this operation. @@ -410,22 +336,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security Solution Aggregated Alert information along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String solutionName, String aggregatedAlertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -441,24 +363,16 @@ private Mono> getWithResponseAsync( final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - aggregatedAlertName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, aggregatedAlertName, accept, context); } /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -467,8 +381,8 @@ private Mono> getWithResponseAsync( * @return security Solution Aggregated Alert information on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String solutionName, String aggregatedAlertName) { + private Mono getAsync(String resourceGroupName, String solutionName, + String aggregatedAlertName) { return getWithResponseAsync(resourceGroupName, solutionName, aggregatedAlertName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -476,9 +390,9 @@ private Mono getAsync( /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @param context The context to associate with this operation. @@ -488,17 +402,17 @@ private Mono getAsync( * @return security Solution Aggregated Alert information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context) { + public Response getWithResponse(String resourceGroupName, String solutionName, + String aggregatedAlertName, Context context) { return getWithResponseAsync(resourceGroupName, solutionName, aggregatedAlertName, context).block(); } /** * Use this method to get a single the aggregated alert of yours IoT Security solution. This aggregation is * performed by alert name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -507,16 +421,16 @@ public Response getWithResponse( * @return security Solution Aggregated Alert information. */ @ServiceMethod(returns = ReturnType.SINGLE) - public IoTSecurityAggregatedAlertInner get( - String resourceGroupName, String solutionName, String aggregatedAlertName) { + public IoTSecurityAggregatedAlertInner get(String resourceGroupName, String solutionName, + String aggregatedAlertName) { return getWithResponse(resourceGroupName, solutionName, aggregatedAlertName, Context.NONE).getValue(); } /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -525,19 +439,15 @@ public IoTSecurityAggregatedAlertInner get( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> dismissWithResponseAsync( - String resourceGroupName, String solutionName, String aggregatedAlertName) { + private Mono> dismissWithResponseAsync(String resourceGroupName, String solutionName, + String aggregatedAlertName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -553,26 +463,16 @@ private Mono> dismissWithResponseAsync( final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .dismiss( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - aggregatedAlertName, - accept, - context)) + .withContext(context -> service.dismiss(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, solutionName, aggregatedAlertName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @param context The context to associate with this operation. @@ -582,19 +482,15 @@ private Mono> dismissWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> dismissWithResponseAsync( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context) { + private Mono> dismissWithResponseAsync(String resourceGroupName, String solutionName, + String aggregatedAlertName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -610,23 +506,15 @@ private Mono> dismissWithResponseAsync( final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .dismiss( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - aggregatedAlertName, - accept, - context); + return service.dismiss(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, aggregatedAlertName, accept, context); } /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -642,9 +530,9 @@ private Mono dismissAsync(String resourceGroupName, String solutionName, S /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @param context The context to associate with this operation. @@ -654,16 +542,16 @@ private Mono dismissAsync(String resourceGroupName, String solutionName, S * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response dismissWithResponse( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context) { + public Response dismissWithResponse(String resourceGroupName, String solutionName, String aggregatedAlertName, + Context context) { return dismissWithResponseAsync(resourceGroupName, solutionName, aggregatedAlertName, context).block(); } /** * Use this method to dismiss an aggregated IoT Security Solution Alert. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedAlertName Identifier of the aggregated alert. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -677,14 +565,15 @@ public void dismiss(String resourceGroupName, String solutionName, String aggreg /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated alert data along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -692,62 +581,43 @@ private Mono> listNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated alert data along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsImpl.java index be131cbc54315..5875a3586a2d1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsAggregatedAlertsImpl.java @@ -16,8 +16,8 @@ public final class IotSecuritySolutionsAnalyticsAggregatedAlertsImpl implements IotSecuritySolutionsAnalyticsAggregatedAlerts { - private static final ClientLogger LOGGER = - new ClientLogger(IotSecuritySolutionsAnalyticsAggregatedAlertsImpl.class); + private static final ClientLogger LOGGER + = new ClientLogger(IotSecuritySolutionsAnalyticsAggregatedAlertsImpl.class); private final IotSecuritySolutionsAnalyticsAggregatedAlertsClient innerClient; @@ -31,27 +31,26 @@ public IotSecuritySolutionsAnalyticsAggregatedAlertsImpl( } public PagedIterable list(String resourceGroupName, String solutionName) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, solutionName); - return Utils.mapPage(inner, inner1 -> new IoTSecurityAggregatedAlertImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, solutionName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new IoTSecurityAggregatedAlertImpl(inner1, this.manager())); } - public PagedIterable list( - String resourceGroupName, String solutionName, Integer top, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, solutionName, top, context); - return Utils.mapPage(inner, inner1 -> new IoTSecurityAggregatedAlertImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String solutionName, Integer top, + Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, solutionName, top, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new IoTSecurityAggregatedAlertImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, solutionName, aggregatedAlertName, context); + public Response getWithResponse(String resourceGroupName, String solutionName, + String aggregatedAlertName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, solutionName, aggregatedAlertName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new IoTSecurityAggregatedAlertImpl(inner.getValue(), this.manager())); } else { return null; @@ -59,8 +58,8 @@ public Response getWithResponse( } public IoTSecurityAggregatedAlert get(String resourceGroupName, String solutionName, String aggregatedAlertName) { - IoTSecurityAggregatedAlertInner inner = - this.serviceClient().get(resourceGroupName, solutionName, aggregatedAlertName); + IoTSecurityAggregatedAlertInner inner + = this.serviceClient().get(resourceGroupName, solutionName, aggregatedAlertName); if (inner != null) { return new IoTSecurityAggregatedAlertImpl(inner, this.manager()); } else { @@ -68,8 +67,8 @@ public IoTSecurityAggregatedAlert get(String resourceGroupName, String solutionN } } - public Response dismissWithResponse( - String resourceGroupName, String solutionName, String aggregatedAlertName, Context context) { + public Response dismissWithResponse(String resourceGroupName, String solutionName, String aggregatedAlertName, + Context context) { return this.serviceClient().dismissWithResponse(resourceGroupName, solutionName, aggregatedAlertName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsClientImpl.java index 606ce1248876c..198d7643de705 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsClientImpl.java @@ -36,103 +36,86 @@ */ public final class IotSecuritySolutionsAnalyticsRecommendationsClientImpl implements IotSecuritySolutionsAnalyticsRecommendationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final IotSecuritySolutionsAnalyticsRecommendationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of IotSecuritySolutionsAnalyticsRecommendationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ IotSecuritySolutionsAnalyticsRecommendationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - IotSecuritySolutionsAnalyticsRecommendationsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(IotSecuritySolutionsAnalyticsRecommendationsService.class, + client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterIotSecuritySolutionsAnalyticsRecommendations to be used - * by the proxy service to perform REST calls. + * The interface defining all the services for SecurityCenterIotSecuritySolutionsAnalyticsRecommendations to be + * used by the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterIotSec") public interface IotSecuritySolutionsAnalyticsRecommendationsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedRecommendations/{aggregatedRecommendationName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedRecommendations/{aggregatedRecommendationName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, @PathParam("aggregatedRecommendationName") String aggregatedRecommendationName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedRecommendations") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}/analyticsModels/default/aggregatedRecommendations") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @QueryParam("$top") Integer top, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @QueryParam("$top") Integer top, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution recommendation information along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String solutionName, String aggregatedRecommendationName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String solutionName, String aggregatedRecommendationName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -142,35 +125,23 @@ private Mono> getWithResponse return Mono.error(new IllegalArgumentException("Parameter solutionName is required and cannot be null.")); } if (aggregatedRecommendationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter aggregatedRecommendationName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter aggregatedRecommendationName is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - aggregatedRecommendationName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, aggregatedRecommendationName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @param context The context to associate with this operation. @@ -178,22 +149,18 @@ private Mono> getWithResponse * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution recommendation information along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String solutionName, String aggregatedRecommendationName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String solutionName, String aggregatedRecommendationName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -203,32 +170,22 @@ private Mono> getWithResponse return Mono.error(new IllegalArgumentException("Parameter solutionName is required and cannot be null.")); } if (aggregatedRecommendationName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter aggregatedRecommendationName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter aggregatedRecommendationName is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - aggregatedRecommendationName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, aggregatedRecommendationName, accept, context); } /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -237,8 +194,8 @@ private Mono> getWithResponse * @return ioT Security solution recommendation information on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String solutionName, String aggregatedRecommendationName) { + private Mono getAsync(String resourceGroupName, String solutionName, + String aggregatedRecommendationName) { return getWithResponseAsync(resourceGroupName, solutionName, aggregatedRecommendationName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -246,9 +203,9 @@ private Mono getAsync( /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @param context The context to associate with this operation. @@ -258,17 +215,17 @@ private Mono getAsync( * @return ioT Security solution recommendation information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String solutionName, String aggregatedRecommendationName, Context context) { + public Response getWithResponse(String resourceGroupName, + String solutionName, String aggregatedRecommendationName, Context context) { return getWithResponseAsync(resourceGroupName, solutionName, aggregatedRecommendationName, context).block(); } /** * Use this method to get the aggregated security analytics recommendation of yours IoT Security solution. This * aggregation is performed by recommendation name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param aggregatedRecommendationName Name of the recommendation aggregated for this query. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -277,38 +234,34 @@ public Response getWithResponse( * @return ioT Security solution recommendation information. */ @ServiceMethod(returns = ReturnType.SINGLE) - public IoTSecurityAggregatedRecommendationInner get( - String resourceGroupName, String solutionName, String aggregatedRecommendationName) { + public IoTSecurityAggregatedRecommendationInner get(String resourceGroupName, String solutionName, + String aggregatedRecommendationName) { return getWithResponse(resourceGroupName, solutionName, aggregatedRecommendationName, Context.NONE).getValue(); } /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated recommendations along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String solutionName, Integer top) { + private Mono> listSinglePageAsync(String resourceGroupName, + String solutionName, Integer top) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -320,35 +273,19 @@ private Mono> listSingle final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - top, - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, top, accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. @@ -356,22 +293,18 @@ private Mono> listSingle * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated recommendations along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String solutionName, Integer top, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String solutionName, Integer top, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -384,31 +317,17 @@ private Mono> listSingle final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - top, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, top, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -417,18 +336,17 @@ private Mono> listSingle * @return list of IoT Security solution aggregated recommendations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String solutionName, Integer top) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, solutionName, top), + private PagedFlux listAsync(String resourceGroupName, String solutionName, + Integer top) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, solutionName, top), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -436,19 +354,18 @@ private PagedFlux listAsync( * @return list of IoT Security solution aggregated recommendations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String solutionName) { + private PagedFlux listAsync(String resourceGroupName, + String solutionName) { final Integer top = null; - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, solutionName, top), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, solutionName, top), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. @@ -458,24 +375,23 @@ private PagedFlux listAsync( * @return list of IoT Security solution aggregated recommendations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String solutionName, Integer top, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, solutionName, top, context), + private PagedFlux listAsync(String resourceGroupName, String solutionName, + Integer top, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, solutionName, top, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of IoT Security solution aggregated recommendations as paginated response with {@link - * PagedIterable}. + * @return list of IoT Security solution aggregated recommendations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String resourceGroupName, String solutionName) { @@ -485,34 +401,35 @@ public PagedIterable list(String resou /** * Use this method to get the list of aggregated security analytics recommendations of yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param top Number of results to retrieve. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of IoT Security solution aggregated recommendations as paginated response with {@link - * PagedIterable}. + * @return list of IoT Security solution aggregated recommendations as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String solutionName, Integer top, Context context) { + public PagedIterable list(String resourceGroupName, String solutionName, + Integer top, Context context) { return new PagedIterable<>(listAsync(resourceGroupName, solutionName, top, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated recommendations along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -520,62 +437,44 @@ private Mono> listNextSi return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solution aggregated recommendations along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsImpl.java index 785dc37759abd..bff70b76b0ff7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsAnalyticsRecommendationsImpl.java @@ -29,27 +29,22 @@ public IotSecuritySolutionsAnalyticsRecommendationsImpl( this.serviceManager = serviceManager; } - public Response getWithResponse( - String resourceGroupName, String solutionName, String aggregatedRecommendationName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, solutionName, aggregatedRecommendationName, context); + public Response getWithResponse(String resourceGroupName, String solutionName, + String aggregatedRecommendationName, Context context) { + Response inner = this.serviceClient() + .getWithResponse(resourceGroupName, solutionName, aggregatedRecommendationName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new IoTSecurityAggregatedRecommendationImpl(inner.getValue(), this.manager())); } else { return null; } } - public IoTSecurityAggregatedRecommendation get( - String resourceGroupName, String solutionName, String aggregatedRecommendationName) { - IoTSecurityAggregatedRecommendationInner inner = - this.serviceClient().get(resourceGroupName, solutionName, aggregatedRecommendationName); + public IoTSecurityAggregatedRecommendation get(String resourceGroupName, String solutionName, + String aggregatedRecommendationName) { + IoTSecurityAggregatedRecommendationInner inner + = this.serviceClient().get(resourceGroupName, solutionName, aggregatedRecommendationName); if (inner != null) { return new IoTSecurityAggregatedRecommendationImpl(inner, this.manager()); } else { @@ -58,16 +53,18 @@ public IoTSecurityAggregatedRecommendation get( } public PagedIterable list(String resourceGroupName, String solutionName) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, solutionName); - return Utils.mapPage(inner, inner1 -> new IoTSecurityAggregatedRecommendationImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, solutionName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new IoTSecurityAggregatedRecommendationImpl(inner1, this.manager())); } - public PagedIterable list( - String resourceGroupName, String solutionName, Integer top, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, solutionName, top, context); - return Utils.mapPage(inner, inner1 -> new IoTSecurityAggregatedRecommendationImpl(inner1, this.manager())); + public PagedIterable list(String resourceGroupName, String solutionName, + Integer top, Context context) { + PagedIterable inner + = this.serviceClient().list(resourceGroupName, solutionName, top, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new IoTSecurityAggregatedRecommendationImpl(inner1, this.manager())); } private IotSecuritySolutionsAnalyticsRecommendationsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsClientImpl.java index c91df12da6ce3..21b11d9928785 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsClientImpl.java @@ -35,141 +35,113 @@ import com.azure.resourcemanager.security.models.UpdateIotSecuritySolutionData; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in IotSecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in IotSecuritySolutionsClient. + */ public final class IotSecuritySolutionsClientImpl implements IotSecuritySolutionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final IotSecuritySolutionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of IotSecuritySolutionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ IotSecuritySolutionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(IotSecuritySolutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(IotSecuritySolutionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterIotSecuritySolutions to be used by the proxy service to - * perform REST calls. + * The interface defining all the services for SecurityCenterIotSecuritySolutions to be used by the proxy service + * to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterIotSec") public interface IotSecuritySolutionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/iotSecuritySolutions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("$filter") String filter, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @HeaderParam("Accept") String accept, - Context context); + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, @BodyParam("application/json") IoTSecuritySolutionModelInner iotSecuritySolutionData, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, @BodyParam("application/json") UpdateIotSecuritySolutionData updateIotSecuritySolutionData, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/iotSecuritySolutions/{solutionName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("solutionName") String solutionName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("solutionName") String solutionName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -179,45 +151,26 @@ Mono> listByResourceGroupNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -228,36 +181,25 @@ private Mono> listSinglePageAsync(S @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), filter, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -266,13 +208,13 @@ private Mono> listSinglePageAsync(S */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String filter) { - return new PagedFlux<>( - () -> listSinglePageAsync(filter), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(filter), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solutions as paginated response with {@link PagedFlux}. @@ -280,13 +222,13 @@ private PagedFlux listAsync(String filter) { @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { final String filter = null; - return new PagedFlux<>( - () -> listSinglePageAsync(filter), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(filter), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -296,14 +238,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String filter, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(filter, context), + return new PagedFlux<>(() -> listSinglePageAsync(filter, context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of IoT Security solutions as paginated response with {@link PagedIterable}. @@ -316,7 +257,7 @@ public PagedIterable list() { /** * Use this method to get the list of IoT Security solutions by subscription. - * + * * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -331,9 +272,9 @@ public PagedIterable list(String filter, Context /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -341,19 +282,15 @@ public PagedIterable list(String filter, Context * @return list of IoT Security solutions along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, String filter) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -362,34 +299,18 @@ private Mono> listByResourceGroupSi final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -398,19 +319,15 @@ private Mono> listByResourceGroupSi * @return list of IoT Security solutions along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, String filter, Context context) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -420,30 +337,17 @@ private Mono> listByResourceGroupSi final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -452,16 +356,15 @@ private Mono> listByResourceGroupSi */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, String filter) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, filter), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, filter), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -470,16 +373,15 @@ private PagedFlux listByResourceGroupAsync(String @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { final String filter = null; - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, filter), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, filter), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -488,18 +390,17 @@ private PagedFlux listByResourceGroupAsync(String * @return list of IoT Security solutions as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, String filter, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, filter, context), + private PagedFlux listByResourceGroupAsync(String resourceGroupName, String filter, + Context context) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, filter, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -513,9 +414,9 @@ public PagedIterable listByResourceGroup(String r /** * Use this method to get the list IoT Security solutions organized by resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param filter Filter the IoT Security solution with OData syntax. Supports filtering by iotHubs. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -524,37 +425,33 @@ public PagedIterable listByResourceGroup(String r * @return list of IoT Security solutions as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup( - String resourceGroupName, String filter, Context context) { + public PagedIterable listByResourceGroup(String resourceGroupName, String filter, + Context context) { return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, filter, context)); } /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution configuration and resource information along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String solutionName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String solutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -566,47 +463,34 @@ private Mono> getByResourceGroupWithResp final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, solutionName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution configuration and resource information along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String solutionName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String solutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -618,22 +502,15 @@ private Mono> getByResourceGroupWithResp final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, accept, context); } /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -648,9 +525,9 @@ private Mono getByResourceGroupAsync(String resou /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -659,16 +536,16 @@ private Mono getByResourceGroupAsync(String resou * @return ioT Security solution configuration and resource information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String solutionName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String solutionName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, solutionName, context).block(); } /** * User this method to get details of a specific IoT Security solution based on solution name. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -682,31 +559,27 @@ public IoTSecuritySolutionModelInner getByResourceGroup(String resourceGroupName /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution configuration and resource information along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -716,35 +589,23 @@ private Mono> createOrUpdateWithResponse return Mono.error(new IllegalArgumentException("Parameter solutionName is required and cannot be null.")); } if (iotSecuritySolutionData == null) { - return Mono - .error( - new IllegalArgumentException("Parameter iotSecuritySolutionData is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter iotSecuritySolutionData is required and cannot be null.")); } else { iotSecuritySolutionData.validate(); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - iotSecuritySolutionData, - accept, - context)) + return FluxUtil.withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, solutionName, iotSecuritySolutionData, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @param context The context to associate with this operation. @@ -752,25 +613,18 @@ private Mono> createOrUpdateWithResponse * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution configuration and resource information along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String solutionName, - IoTSecuritySolutionModelInner iotSecuritySolutionData, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -780,32 +634,23 @@ private Mono> createOrUpdateWithResponse return Mono.error(new IllegalArgumentException("Parameter solutionName is required and cannot be null.")); } if (iotSecuritySolutionData == null) { - return Mono - .error( - new IllegalArgumentException("Parameter iotSecuritySolutionData is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter iotSecuritySolutionData is required and cannot be null.")); } else { iotSecuritySolutionData.validate(); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - iotSecuritySolutionData, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, iotSecuritySolutionData, accept, context); } /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -814,17 +659,17 @@ private Mono> createOrUpdateWithResponse * @return ioT Security solution configuration and resource information on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData) { + private Mono createOrUpdateAsync(String resourceGroupName, String solutionName, + IoTSecuritySolutionModelInner iotSecuritySolutionData) { return createOrUpdateWithResponseAsync(resourceGroupName, solutionName, iotSecuritySolutionData) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @param context The context to associate with this operation. @@ -834,20 +679,17 @@ private Mono createOrUpdateAsync( * @return ioT Security solution configuration and resource information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String solutionName, - IoTSecuritySolutionModelInner iotSecuritySolutionData, - Context context) { + public Response createOrUpdateWithResponse(String resourceGroupName, + String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData, Context context) { return createOrUpdateWithResponseAsync(resourceGroupName, solutionName, iotSecuritySolutionData, context) .block(); } /** * Use this method to create or update yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param iotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -856,8 +698,8 @@ public Response createOrUpdateWithResponse( * @return ioT Security solution configuration and resource information. */ @ServiceMethod(returns = ReturnType.SINGLE) - public IoTSecuritySolutionModelInner createOrUpdate( - String resourceGroupName, String solutionName, IoTSecuritySolutionModelInner iotSecuritySolutionData) { + public IoTSecuritySolutionModelInner createOrUpdate(String resourceGroupName, String solutionName, + IoTSecuritySolutionModelInner iotSecuritySolutionData) { return createOrUpdateWithResponse(resourceGroupName, solutionName, iotSecuritySolutionData, Context.NONE) .getValue(); } @@ -865,31 +707,27 @@ public IoTSecuritySolutionModelInner createOrUpdate( /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution configuration and resource information along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String solutionName, UpdateIotSecuritySolutionData updateIotSecuritySolutionData) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String solutionName, UpdateIotSecuritySolutionData updateIotSecuritySolutionData) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -899,10 +737,8 @@ private Mono> updateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter solutionName is required and cannot be null.")); } if (updateIotSecuritySolutionData == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter updateIotSecuritySolutionData is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter updateIotSecuritySolutionData is required and cannot be null.")); } else { updateIotSecuritySolutionData.validate(); } @@ -910,26 +746,17 @@ private Mono> updateWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - updateIotSecuritySolutionData, - accept, - context)) + context -> service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, solutionName, updateIotSecuritySolutionData, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @param context The context to associate with this operation. @@ -937,25 +764,18 @@ private Mono> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return ioT Security solution configuration and resource information along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String solutionName, - UpdateIotSecuritySolutionData updateIotSecuritySolutionData, - Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String solutionName, UpdateIotSecuritySolutionData updateIotSecuritySolutionData, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -965,34 +785,24 @@ private Mono> updateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter solutionName is required and cannot be null.")); } if (updateIotSecuritySolutionData == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter updateIotSecuritySolutionData is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter updateIotSecuritySolutionData is required and cannot be null.")); } else { updateIotSecuritySolutionData.validate(); } final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - updateIotSecuritySolutionData, - accept, - context); + return service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, updateIotSecuritySolutionData, accept, context); } /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1001,8 +811,8 @@ private Mono> updateWithResponseAsync( * @return ioT Security solution configuration and resource information on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String solutionName, UpdateIotSecuritySolutionData updateIotSecuritySolutionData) { + private Mono updateAsync(String resourceGroupName, String solutionName, + UpdateIotSecuritySolutionData updateIotSecuritySolutionData) { return updateWithResponseAsync(resourceGroupName, solutionName, updateIotSecuritySolutionData) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -1010,9 +820,9 @@ private Mono updateAsync( /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @param context The context to associate with this operation. @@ -1022,20 +832,17 @@ private Mono updateAsync( * @return ioT Security solution configuration and resource information along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String solutionName, - UpdateIotSecuritySolutionData updateIotSecuritySolutionData, - Context context) { + public Response updateWithResponse(String resourceGroupName, String solutionName, + UpdateIotSecuritySolutionData updateIotSecuritySolutionData, Context context) { return updateWithResponseAsync(resourceGroupName, solutionName, updateIotSecuritySolutionData, context).block(); } /** * Use this method to update existing IoT Security solution tags or user defined resources. To update other fields * use the CreateOrUpdate method. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param updateIotSecuritySolutionData The security solution data. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1044,17 +851,17 @@ public Response updateWithResponse( * @return ioT Security solution configuration and resource information. */ @ServiceMethod(returns = ReturnType.SINGLE) - public IoTSecuritySolutionModelInner update( - String resourceGroupName, String solutionName, UpdateIotSecuritySolutionData updateIotSecuritySolutionData) { + public IoTSecuritySolutionModelInner update(String resourceGroupName, String solutionName, + UpdateIotSecuritySolutionData updateIotSecuritySolutionData) { return updateWithResponse(resourceGroupName, solutionName, updateIotSecuritySolutionData, Context.NONE) .getValue(); } /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1064,16 +871,12 @@ public IoTSecuritySolutionModelInner update( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceGroupName, String solutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1085,25 +888,16 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S final String apiVersion = "2019-08-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, solutionName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1112,19 +906,15 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String solutionName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String solutionName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1136,22 +926,15 @@ private Mono> deleteWithResponseAsync( final String apiVersion = "2019-08-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - solutionName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + solutionName, accept, context); } /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1165,9 +948,9 @@ private Mono deleteAsync(String resourceGroupName, String solutionName) { /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1182,9 +965,9 @@ public Response deleteWithResponse(String resourceGroupName, String soluti /** * Use this method to delete yours IoT Security solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param solutionName The name of the IoT Security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1197,9 +980,10 @@ public void delete(String resourceGroupName, String solutionName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1211,32 +995,24 @@ private Mono> listBySubscriptionNex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1244,37 +1020,28 @@ private Mono> listBySubscriptionNex * @return list of IoT Security solutions along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1286,32 +1053,24 @@ private Mono> listByResourceGroupNe return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1319,29 +1078,19 @@ private Mono> listByResourceGroupNe * @return list of IoT Security solutions along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsImpl.java index 157e75226a149..d1c50ef8066f5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/IotSecuritySolutionsImpl.java @@ -21,44 +21,41 @@ public final class IotSecuritySolutionsImpl implements IotSecuritySolutions { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public IotSecuritySolutionsImpl( - IotSecuritySolutionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public IotSecuritySolutionsImpl(IotSecuritySolutionsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); } public PagedIterable list(String filter, Context context) { PagedIterable inner = this.serviceClient().list(filter, context); - return Utils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); } - public PagedIterable listByResourceGroup( - String resourceGroupName, String filter, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, filter, context); - return Utils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); + public PagedIterable listByResourceGroup(String resourceGroupName, String filter, + Context context) { + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, filter, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new IoTSecuritySolutionModelImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String solutionName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, solutionName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String solutionName, Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, solutionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new IoTSecuritySolutionModelImpl(inner.getValue(), this.manager())); } else { return null; @@ -74,8 +71,8 @@ public IoTSecuritySolutionModel getByResourceGroup(String resourceGroupName, Str } } - public Response deleteByResourceGroupWithResponse( - String resourceGroupName, String solutionName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceGroupName, String solutionName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, solutionName, context); } @@ -84,89 +81,57 @@ public void deleteByResourceGroup(String resourceGroupName, String solutionName) } public IoTSecuritySolutionModel getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String solutionName = Utils.getValueFromIdByName(id, "iotSecuritySolutions"); + String solutionName = ResourceManagerUtils.getValueFromIdByName(id, "iotSecuritySolutions"); if (solutionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, solutionName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String solutionName = Utils.getValueFromIdByName(id, "iotSecuritySolutions"); + String solutionName = ResourceManagerUtils.getValueFromIdByName(id, "iotSecuritySolutions"); if (solutionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, solutionName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String solutionName = Utils.getValueFromIdByName(id, "iotSecuritySolutions"); + String solutionName = ResourceManagerUtils.getValueFromIdByName(id, "iotSecuritySolutions"); if (solutionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", id))); } this.deleteByResourceGroupWithResponse(resourceGroupName, solutionName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String solutionName = Utils.getValueFromIdByName(id, "iotSecuritySolutions"); + String solutionName = ResourceManagerUtils.getValueFromIdByName(id, "iotSecuritySolutions"); if (solutionName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'iotSecuritySolutions'.", id))); } return this.deleteByResourceGroupWithResponse(resourceGroupName, solutionName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesClientImpl.java index 1a990d30535c8..9d5befbcde10a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesClientImpl.java @@ -36,23 +36,28 @@ import com.azure.resourcemanager.security.models.JitNetworkAccessPolicyInitiateRequest; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in JitNetworkAccessPoliciesClient. */ +/** + * An instance of this class provides access to all the operations defined in JitNetworkAccessPoliciesClient. + */ public final class JitNetworkAccessPoliciesClientImpl implements JitNetworkAccessPoliciesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final JitNetworkAccessPoliciesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of JitNetworkAccessPoliciesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ JitNetworkAccessPoliciesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create(JitNetworkAccessPoliciesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(JitNetworkAccessPoliciesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -63,164 +68,121 @@ public final class JitNetworkAccessPoliciesClientImpl implements JitNetworkAcces @Host("{$host}") @ServiceInterface(name = "SecurityCenterJitNet") public interface JitNetworkAccessPoliciesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/jitNetworkAccessPolicies") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByRegion( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByRegion(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/jitNetworkAccessPolicies") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/jitNetworkAccessPolicies") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroupAndRegion( - @HostParam("$host") String endpoint, + Mono> listByResourceGroupAndRegion(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("jitNetworkAccessPolicyName") String jitNetworkAccessPolicyName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("jitNetworkAccessPolicyName") String jitNetworkAccessPolicyName, @QueryParam("api-version") String apiVersion, - @BodyParam("application/json") JitNetworkAccessPolicyInner body, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") JitNetworkAccessPolicyInner body, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("jitNetworkAccessPolicyName") String jitNetworkAccessPolicyName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}/{jitNetworkAccessPolicyInitiateType}") - @ExpectedResponses({202}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/jitNetworkAccessPolicies/{jitNetworkAccessPolicyName}/{jitNetworkAccessPolicyInitiateType}") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> initiate( - @HostParam("$host") String endpoint, + Mono> initiate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("jitNetworkAccessPolicyName") String jitNetworkAccessPolicyName, @PathParam("jitNetworkAccessPolicyInitiateType") String jitNetworkAccessPolicyInitiateType, @QueryParam("api-version") String apiVersion, @BodyParam("application/json") JitNetworkAccessPolicyInitiateRequest body, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupAndRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Policies for protecting resources using Just-in-Time access control. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -228,39 +190,26 @@ Mono> listByResourceGroupAndRegionNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Policies for protecting resources using Just-in-Time access control. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -270,36 +219,24 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Policies for protecting resources using Just-in-Time access control. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the paginated response with {@link PagedFlux}. @@ -311,7 +248,7 @@ private PagedFlux listAsync() { /** * Policies for protecting resources using Just-in-Time access control. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -320,13 +257,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Policies for protecting resources using Just-in-Time access control. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the paginated response with {@link PagedIterable}. @@ -338,7 +275,7 @@ public PagedIterable list() { /** * Policies for protecting resources using Just-in-Time access control. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -352,9 +289,9 @@ public PagedIterable list(Context context) { /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -363,16 +300,12 @@ public PagedIterable list(Context context) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByRegionSinglePageAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -380,33 +313,18 @@ private Mono> listByRegionSinglePageA final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -414,19 +332,15 @@ private Mono> listByRegionSinglePageA * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByRegionSinglePageAsync( - String ascLocation, Context context) { + private Mono> listByRegionSinglePageAsync(String ascLocation, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -435,24 +349,17 @@ private Mono> listByRegionSinglePageA final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByRegion( - this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, + context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -460,15 +367,15 @@ private Mono> listByRegionSinglePageA */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByRegionAsync(String ascLocation) { - return new PagedFlux<>( - () -> listByRegionSinglePageAsync(ascLocation), nextLink -> listByRegionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listByRegionSinglePageAsync(ascLocation), + nextLink -> listByRegionNextSinglePageAsync(nextLink)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -477,16 +384,15 @@ private PagedFlux listByRegionAsync(String ascLocat */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByRegionAsync(String ascLocation, Context context) { - return new PagedFlux<>( - () -> listByRegionSinglePageAsync(ascLocation, context), + return new PagedFlux<>(() -> listByRegionSinglePageAsync(ascLocation, context), nextLink -> listByRegionNextSinglePageAsync(nextLink, context)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -499,9 +405,9 @@ public PagedIterable listByRegion(String ascLocatio /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -515,28 +421,24 @@ public PagedIterable listByRegion(String ascLocatio /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -545,33 +447,18 @@ private Mono> listByResourceGroupSing final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -579,19 +466,15 @@ private Mono> listByResourceGroupSing * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> + listByResourceGroupSinglePageAsync(String resourceGroupName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -601,29 +484,17 @@ private Mono> listByResourceGroupSing final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - apiVersion, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -631,16 +502,15 @@ private Mono> listByResourceGroupSing */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -649,16 +519,15 @@ private PagedFlux listByResourceGroupAsync(String r */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -671,9 +540,9 @@ public PagedIterable listByResourceGroup(String res /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -687,30 +556,26 @@ public PagedIterable listByResourceGroup(String res /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupAndRegionSinglePageAsync( - String resourceGroupName, String ascLocation) { + private Mono> + listByResourceGroupAndRegionSinglePageAsync(String resourceGroupName, String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -722,36 +587,20 @@ private Mono> listByResourceGroupAndR final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroupAndRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroupAndRegion(this.client.getEndpoint(), + this.client.getSubscriptionId(), resourceGroupName, ascLocation, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -759,19 +608,15 @@ private Mono> listByResourceGroupAndR * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupAndRegionSinglePageAsync( - String resourceGroupName, String ascLocation, Context context) { + private Mono> + listByResourceGroupAndRegionSinglePageAsync(String resourceGroupName, String ascLocation, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -784,52 +629,38 @@ private Mono> listByResourceGroupAndR final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroupAndRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - apiVersion, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroupAndRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + ascLocation, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAndRegionAsync( - String resourceGroupName, String ascLocation) { - return new PagedFlux<>( - () -> listByResourceGroupAndRegionSinglePageAsync(resourceGroupName, ascLocation), + private PagedFlux listByResourceGroupAndRegionAsync(String resourceGroupName, + String ascLocation) { + return new PagedFlux<>(() -> listByResourceGroupAndRegionSinglePageAsync(resourceGroupName, ascLocation), nextLink -> listByResourceGroupAndRegionNextSinglePageAsync(nextLink)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -837,8 +668,8 @@ private PagedFlux listByResourceGroupAndRegionAsync * @return the paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAndRegionAsync( - String resourceGroupName, String ascLocation, Context context) { + private PagedFlux listByResourceGroupAndRegionAsync(String resourceGroupName, + String ascLocation, Context context) { return new PagedFlux<>( () -> listByResourceGroupAndRegionSinglePageAsync(resourceGroupName, ascLocation, context), nextLink -> listByResourceGroupAndRegionNextSinglePageAsync(nextLink, context)); @@ -846,29 +677,29 @@ private PagedFlux listByResourceGroupAndRegionAsync /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroupAndRegion( - String resourceGroupName, String ascLocation) { + public PagedIterable listByResourceGroupAndRegion(String resourceGroupName, + String ascLocation) { return new PagedIterable<>(listByResourceGroupAndRegionAsync(resourceGroupName, ascLocation)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -876,18 +707,18 @@ public PagedIterable listByResourceGroupAndRegion( * @return the paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroupAndRegion( - String resourceGroupName, String ascLocation, Context context) { + public PagedIterable listByResourceGroupAndRegion(String resourceGroupName, + String ascLocation, Context context) { return new PagedIterable<>(listByResourceGroupAndRegionAsync(resourceGroupName, ascLocation, context)); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -895,19 +726,15 @@ public PagedIterable listByResourceGroupAndRegion( * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -917,36 +744,24 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, jitNetworkAccessPolicyName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -955,19 +770,15 @@ private Mono> getWithResponseAsync( * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -977,33 +788,23 @@ private Mono> getWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, ascLocation, + jitNetworkAccessPolicyName, apiVersion, accept, context); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1011,19 +812,19 @@ private Mono> getWithResponseAsync( * @return the response body on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName) { + private Mono getAsync(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName) { return getWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1032,18 +833,18 @@ private Mono getAsync( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context) { + public Response getWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, Context context) { return getWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context).block(); } /** * Policies for protecting resources using Just-in-Time access control for the subscription, location. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1051,18 +852,18 @@ public Response getWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public JitNetworkAccessPolicyInner get( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName) { + public JitNetworkAccessPolicyInner get(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName) { return getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, Context.NONE).getValue(); } /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1071,22 +872,15 @@ public JitNetworkAccessPolicyInner get( * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1096,10 +890,8 @@ private Mono> createOrUpdateWithResponseAs return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } if (body == null) { return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); @@ -1109,29 +901,18 @@ private Mono> createOrUpdateWithResponseAs final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - apiVersion, - body, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, jitNetworkAccessPolicyName, apiVersion, body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @param context The context to associate with this operation. @@ -1141,23 +922,15 @@ private Mono> createOrUpdateWithResponseAs * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1167,10 +940,8 @@ private Mono> createOrUpdateWithResponseAs return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } if (body == null) { return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); @@ -1180,26 +951,17 @@ private Mono> createOrUpdateWithResponseAs final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - apiVersion, - body, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, apiVersion, body, accept, context); } /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1208,22 +970,19 @@ private Mono> createOrUpdateWithResponseAs * @return the response body on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body) { + private Mono createOrUpdateAsync(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body) { return createOrUpdateWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @param context The context to associate with this operation. @@ -1233,24 +992,19 @@ private Mono createOrUpdateAsync( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, + context).block(); } /** * Create a policy for protecting resources using Just-in-Time access control. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1259,23 +1013,19 @@ public Response createOrUpdateWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public JitNetworkAccessPolicyInner createOrUpdate( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInner body) { - return createOrUpdateWithResponse( - resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, Context.NONE) - .getValue(); + public JitNetworkAccessPolicyInner createOrUpdate(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInner body) { + return createOrUpdateWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, + Context.NONE).getValue(); } /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1283,19 +1033,15 @@ public JitNetworkAccessPolicyInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1305,36 +1051,24 @@ private Mono> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - apiVersion, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, jitNetworkAccessPolicyName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1343,19 +1077,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1365,33 +1095,23 @@ private Mono> deleteWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - apiVersion, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, apiVersion, accept, context); } /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1406,11 +1126,11 @@ private Mono deleteAsync(String resourceGroupName, String ascLocation, Str /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1419,18 +1139,18 @@ private Mono deleteAsync(String resourceGroupName, String ascLocation, Str * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context) { + public Response deleteWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, Context context) { return deleteWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context).block(); } /** * Delete a Just-in-Time access control policy. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1443,11 +1163,11 @@ public void delete(String resourceGroupName, String ascLocation, String jitNetwo /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1456,22 +1176,15 @@ public void delete(String resourceGroupName, String ascLocation, String jitNetwo * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> initiateWithResponseAsync( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body) { + private Mono> initiateWithResponseAsync(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1481,10 +1194,8 @@ private Mono> initiateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } if (body == null) { return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); @@ -1495,30 +1206,19 @@ private Mono> initiateWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .initiate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - jitNetworkAccessPolicyInitiateType, - apiVersion, - body, - accept, - context)) + .withContext(context -> service.initiate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, jitNetworkAccessPolicyName, jitNetworkAccessPolicyInitiateType, + apiVersion, body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @param context The context to associate with this operation. @@ -1528,23 +1228,16 @@ private Mono> initiateWithResponseAsync( * @return the response body along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> initiateWithResponseAsync( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body, + private Mono> initiateWithResponseAsync(String resourceGroupName, + String ascLocation, String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1554,10 +1247,8 @@ private Mono> initiateWithResponseAsync( return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); } if (jitNetworkAccessPolicyName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter jitNetworkAccessPolicyName is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter jitNetworkAccessPolicyName is required and cannot be null.")); } if (body == null) { return Mono.error(new IllegalArgumentException("Parameter body is required and cannot be null.")); @@ -1568,27 +1259,18 @@ private Mono> initiateWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .initiate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - jitNetworkAccessPolicyName, - jitNetworkAccessPolicyInitiateType, - apiVersion, - body, - accept, - context); + return service.initiate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, jitNetworkAccessPolicyInitiateType, apiVersion, body, accept, + context); } /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1597,22 +1279,19 @@ private Mono> initiateWithResponseAsync( * @return the response body on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono initiateAsync( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body) { + private Mono initiateAsync(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body) { return initiateWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @param context The context to associate with this operation. @@ -1622,23 +1301,19 @@ private Mono initiateAsync( * @return the response body along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response initiateWithResponse( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body, - Context context) { + public Response initiateWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body, Context context) { return initiateWithResponseAsync(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, context) .block(); } /** * Initiate a JIT access from a specific Just-in-Time policy configuration. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param jitNetworkAccessPolicyName Name of a Just-in-Time access configuration policy. * @param body The body parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1647,20 +1322,18 @@ public Response initiateWithResponse( * @return the response. */ @ServiceMethod(returns = ReturnType.SINGLE) - public JitNetworkAccessRequestInner initiate( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body) { + public JitNetworkAccessRequestInner initiate(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body) { return initiateWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, Context.NONE) .getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1672,31 +1345,22 @@ private Mono> listNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1709,31 +1373,22 @@ private Mono> listNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1745,31 +1400,23 @@ private Mono> listByRegionNextSingleP return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1777,37 +1424,28 @@ private Mono> listByRegionNextSingleP * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1819,32 +1457,24 @@ private Mono> listByResourceGroupNext return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1852,76 +1482,57 @@ private Mono> listByResourceGroupNext * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupAndRegionNextSinglePageAsync( - String nextLink) { + private Mono> + listByResourceGroupAndRegionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.listByResourceGroupAndRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext( + context -> service.listByResourceGroupAndRegionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1929,29 +1540,19 @@ private Mono> listByResourceGroupAndR * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupAndRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listByResourceGroupAndRegionNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupAndRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupAndRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesImpl.java index b9d9dcc313b61..b240153a75b77 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPoliciesImpl.java @@ -24,66 +24,63 @@ public final class JitNetworkAccessPoliciesImpl implements JitNetworkAccessPolic private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public JitNetworkAccessPoliciesImpl( - JitNetworkAccessPoliciesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public JitNetworkAccessPoliciesImpl(JitNetworkAccessPoliciesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } public PagedIterable listByRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listByRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } public PagedIterable listByRegion(String ascLocation, Context context) { PagedIterable inner = this.serviceClient().listByRegion(ascLocation, context); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } - public PagedIterable listByResourceGroupAndRegion( - String resourceGroupName, String ascLocation) { - PagedIterable inner = - this.serviceClient().listByResourceGroupAndRegion(resourceGroupName, ascLocation); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + public PagedIterable listByResourceGroupAndRegion(String resourceGroupName, + String ascLocation) { + PagedIterable inner + = this.serviceClient().listByResourceGroupAndRegion(resourceGroupName, ascLocation); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } - public PagedIterable listByResourceGroupAndRegion( - String resourceGroupName, String ascLocation, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroupAndRegion(resourceGroupName, ascLocation, context); - return Utils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); + public PagedIterable listByResourceGroupAndRegion(String resourceGroupName, + String ascLocation, Context context) { + PagedIterable inner + = this.serviceClient().listByResourceGroupAndRegion(resourceGroupName, ascLocation, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new JitNetworkAccessPolicyImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context); + public Response getWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new JitNetworkAccessPolicyImpl(inner.getValue(), this.manager())); } else { return null; @@ -91,8 +88,8 @@ public Response getWithResponse( } public JitNetworkAccessPolicy get(String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName) { - JitNetworkAccessPolicyInner inner = - this.serviceClient().get(resourceGroupName, ascLocation, jitNetworkAccessPolicyName); + JitNetworkAccessPolicyInner inner + = this.serviceClient().get(resourceGroupName, ascLocation, jitNetworkAccessPolicyName); if (inner != null) { return new JitNetworkAccessPolicyImpl(inner, this.manager()); } else { @@ -100,45 +97,32 @@ public JitNetworkAccessPolicy get(String resourceGroupName, String ascLocation, } } - public Response deleteWithResponse( - String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context); + public Response deleteWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, + context); } public void delete(String resourceGroupName, String ascLocation, String jitNetworkAccessPolicyName) { this.serviceClient().delete(resourceGroupName, ascLocation, jitNetworkAccessPolicyName); } - public Response initiateWithResponse( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body, - Context context) { - Response inner = - this - .serviceClient() - .initiateWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body, context); + public Response initiateWithResponse(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body, Context context) { + Response inner = this.serviceClient().initiateWithResponse(resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, body, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new JitNetworkAccessRequestImpl(inner.getValue(), this.manager())); } else { return null; } } - public JitNetworkAccessRequest initiate( - String resourceGroupName, - String ascLocation, - String jitNetworkAccessPolicyName, - JitNetworkAccessPolicyInitiateRequest body) { - JitNetworkAccessRequestInner inner = - this.serviceClient().initiate(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body); + public JitNetworkAccessRequest initiate(String resourceGroupName, String ascLocation, + String jitNetworkAccessPolicyName, JitNetworkAccessPolicyInitiateRequest body) { + JitNetworkAccessRequestInner inner + = this.serviceClient().initiate(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, body); if (inner != null) { return new JitNetworkAccessRequestImpl(inner, this.manager()); } else { @@ -147,119 +131,78 @@ public JitNetworkAccessRequest initiate( } public JitNetworkAccessPolicy getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String jitNetworkAccessPolicyName = Utils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); + String jitNetworkAccessPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); if (jitNetworkAccessPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", id))); } - return this - .getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, Context.NONE) + return this.getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, Context.NONE) .getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String jitNetworkAccessPolicyName = Utils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); + String jitNetworkAccessPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); if (jitNetworkAccessPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", id))); } return this.getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String jitNetworkAccessPolicyName = Utils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); + String jitNetworkAccessPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); if (jitNetworkAccessPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", id))); } this.deleteWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String ascLocation = Utils.getValueFromIdByName(id, "locations"); + String ascLocation = ResourceManagerUtils.getValueFromIdByName(id, "locations"); if (ascLocation == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'locations'.", id))); } - String jitNetworkAccessPolicyName = Utils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); + String jitNetworkAccessPolicyName = ResourceManagerUtils.getValueFromIdByName(id, "jitNetworkAccessPolicies"); if (jitNetworkAccessPolicyName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", - id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException(String + .format("The resource ID '%s' is not valid. Missing path segment 'jitNetworkAccessPolicies'.", id))); } return this.deleteWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPolicyImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPolicyImpl.java index 570ed838040bf..58955cb395329 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPolicyImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessPolicyImpl.java @@ -53,12 +53,8 @@ public List virtualMachines() { public List requests() { List inner = this.innerModel().requests(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new JitNetworkAccessRequestImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new JitNetworkAccessRequestImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } @@ -101,24 +97,16 @@ public JitNetworkAccessPolicyImpl withExistingLocation(String resourceGroupName, } public JitNetworkAccessPolicy create() { - this.innerObject = - serviceManager - .serviceClient() - .getJitNetworkAccessPolicies() - .createOrUpdateWithResponse( - resourceGroupName, ascLocation, jitNetworkAccessPolicyName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getJitNetworkAccessPolicies().createOrUpdateWithResponse(resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, this.innerModel(), Context.NONE).getValue(); return this; } public JitNetworkAccessPolicy create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getJitNetworkAccessPolicies() - .createOrUpdateWithResponse( - resourceGroupName, ascLocation, jitNetworkAccessPolicyName, this.innerModel(), context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getJitNetworkAccessPolicies().createOrUpdateWithResponse(resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, this.innerModel(), context).getValue(); return this; } @@ -133,53 +121,38 @@ public JitNetworkAccessPolicyImpl update() { } public JitNetworkAccessPolicy apply() { - this.innerObject = - serviceManager - .serviceClient() - .getJitNetworkAccessPolicies() - .createOrUpdateWithResponse( - resourceGroupName, ascLocation, jitNetworkAccessPolicyName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getJitNetworkAccessPolicies().createOrUpdateWithResponse(resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, this.innerModel(), Context.NONE).getValue(); return this; } public JitNetworkAccessPolicy apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getJitNetworkAccessPolicies() - .createOrUpdateWithResponse( - resourceGroupName, ascLocation, jitNetworkAccessPolicyName, this.innerModel(), context) - .getValue(); + this.innerObject + = serviceManager.serviceClient().getJitNetworkAccessPolicies().createOrUpdateWithResponse(resourceGroupName, + ascLocation, jitNetworkAccessPolicyName, this.innerModel(), context).getValue(); return this; } - JitNetworkAccessPolicyImpl( - JitNetworkAccessPolicyInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + JitNetworkAccessPolicyImpl(JitNetworkAccessPolicyInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.ascLocation = Utils.getValueFromIdByName(innerObject.id(), "locations"); - this.jitNetworkAccessPolicyName = Utils.getValueFromIdByName(innerObject.id(), "jitNetworkAccessPolicies"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.ascLocation = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "locations"); + this.jitNetworkAccessPolicyName + = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "jitNetworkAccessPolicies"); } public JitNetworkAccessPolicy refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getJitNetworkAccessPolicies() - .getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getJitNetworkAccessPolicies() + .getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, Context.NONE).getValue(); return this; } public JitNetworkAccessPolicy refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getJitNetworkAccessPolicies() - .getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getJitNetworkAccessPolicies() + .getWithResponse(resourceGroupName, ascLocation, jitNetworkAccessPolicyName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessRequestImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessRequestImpl.java index 0cf244c2e8b95..be0e133fab445 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessRequestImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/JitNetworkAccessRequestImpl.java @@ -16,8 +16,8 @@ public final class JitNetworkAccessRequestImpl implements JitNetworkAccessReques private final com.azure.resourcemanager.security.SecurityManager serviceManager; - JitNetworkAccessRequestImpl( - JitNetworkAccessRequestInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + JitNetworkAccessRequestImpl(JitNetworkAccessRequestInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsClientImpl.java index 7cb672fbf87c5..c342d7b9fcf58 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.AscLocationList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in LocationsClient. */ +/** + * An instance of this class provides access to all the operations defined in LocationsClient. + */ public final class LocationsClientImpl implements LocationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final LocationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of LocationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ LocationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(LocationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(LocationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,80 +62,57 @@ public final class LocationsClientImpl implements LocationsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterLocati") public interface LocationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } @@ -137,49 +120,37 @@ private Mono> listSinglePageAsync() { * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data as paginated response with {@link PagedFlux}. @@ -193,7 +164,7 @@ private PagedFlux listAsync() { * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -202,15 +173,15 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data as paginated response with {@link PagedIterable}. @@ -224,7 +195,7 @@ public PagedIterable list() { * The location of the responsible ASC of the specific subscription (home region). For each subscription there is * only one responsible location. The location in the response should be used to read or write other resources in * ASC according to their ID. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -238,28 +209,24 @@ public PagedIterable list(Context context) { /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the ASC location of the subscription is in the "name" field along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -267,44 +234,32 @@ private Mono> getWithResponseAsync(String ascLocation final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + ascLocation, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the ASC location of the subscription is in the "name" field along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String ascLocation, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -312,15 +267,15 @@ private Mono> getWithResponseAsync(String ascLocation final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), ascLocation, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), ascLocation, accept, + context); } /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -333,9 +288,9 @@ private Mono getAsync(String ascLocation) { /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -349,9 +304,9 @@ public Response getWithResponse(String ascLocation, Context co /** * Details of a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -364,14 +319,15 @@ public AscLocationInner get(String ascLocation) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -379,37 +335,28 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of locations where ASC saves your data along with {@link PagedResponse} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -417,23 +364,13 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsImpl.java index d946db5d2e1ae..cc476cfd3e1fe 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/LocationsImpl.java @@ -21,29 +21,26 @@ public final class LocationsImpl implements Locations { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public LocationsImpl( - LocationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public LocationsImpl(LocationsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new AscLocationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AscLocationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new AscLocationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new AscLocationImpl(inner1, this.manager())); } public Response getWithResponse(String ascLocation, Context context) { Response inner = this.serviceClient().getWithResponse(ascLocation, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new AscLocationImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataImpl.java index fdf2e0b92f461..bc505796e77cb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataImpl.java @@ -12,8 +12,8 @@ public final class MdeOnboardingDataImpl implements MdeOnboardingData { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - MdeOnboardingDataImpl( - MdeOnboardingDataInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + MdeOnboardingDataImpl(MdeOnboardingDataInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataListImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataListImpl.java index a6a826ba37146..10b6744e2215a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataListImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingDataListImpl.java @@ -17,8 +17,8 @@ public final class MdeOnboardingDataListImpl implements MdeOnboardingDataList { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - MdeOnboardingDataListImpl( - MdeOnboardingDataListInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + MdeOnboardingDataListImpl(MdeOnboardingDataListInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -26,12 +26,8 @@ public final class MdeOnboardingDataListImpl implements MdeOnboardingDataList { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new MdeOnboardingDataImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new MdeOnboardingDataImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsClientImpl.java index 1425b43ebbf47..d0367a847d38e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsClientImpl.java @@ -26,22 +26,28 @@ import com.azure.resourcemanager.security.fluent.models.MdeOnboardingDataListInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in MdeOnboardingsClient. */ +/** + * An instance of this class provides access to all the operations defined in MdeOnboardingsClient. + */ public final class MdeOnboardingsClientImpl implements MdeOnboardingsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final MdeOnboardingsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of MdeOnboardingsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ MdeOnboardingsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(MdeOnboardingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(MdeOnboardingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -52,84 +58,68 @@ public final class MdeOnboardingsClientImpl implements MdeOnboardingsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterMdeOnb") public interface MdeOnboardingsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/mdeOnboardings") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/mdeOnboardings/default") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); } /** * The configuration or data needed to onboard the machine to MDE. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all MDE onboarding data resources along with {@link Response} on successful completion of {@link - * Mono}. + * @return list of all MDE onboarding data resources along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The configuration or data needed to onboard the machine to MDE. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all MDE onboarding data resources along with {@link Response} on successful completion of {@link - * Mono}. + * @return list of all MDE onboarding data resources along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-10-01-preview"; final String accept = "application/json"; @@ -139,7 +129,7 @@ private Mono> listWithResponseAsync(Context /** * The configuration or data needed to onboard the machine to MDE. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all MDE onboarding data resources on successful completion of {@link Mono}. @@ -151,7 +141,7 @@ private Mono listAsync() { /** * The configuration or data needed to onboard the machine to MDE. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -165,7 +155,7 @@ public Response listWithResponse(Context context) { /** * The configuration or data needed to onboard the machine to MDE. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all MDE onboarding data resources. @@ -177,59 +167,49 @@ public MdeOnboardingDataListInner list() { /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the resource of the configuration or data needed to onboard the machine to MDE along with {@link - * Response} on successful completion of {@link Mono}. + * @return the resource of the configuration or data needed to onboard the machine to MDE along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the resource of the configuration or data needed to onboard the machine to MDE along with {@link - * Response} on successful completion of {@link Mono}. + * @return the resource of the configuration or data needed to onboard the machine to MDE along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-10-01-preview"; final String accept = "application/json"; @@ -239,11 +219,11 @@ private Mono> getWithResponseAsync(Context cont /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource of the configuration or data needed to onboard the machine to MDE on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync() { @@ -252,13 +232,13 @@ private Mono getAsync() { /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the resource of the configuration or data needed to onboard the machine to MDE along with {@link - * Response}. + * @return the resource of the configuration or data needed to onboard the machine to MDE along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response getWithResponse(Context context) { @@ -267,7 +247,7 @@ public Response getWithResponse(Context context) { /** * The default configuration or data needed to onboard the machine to MDE. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the resource of the configuration or data needed to onboard the machine to MDE. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsImpl.java index 4a594ef5dc787..ad745ba5a6f9e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/MdeOnboardingsImpl.java @@ -22,8 +22,8 @@ public final class MdeOnboardingsImpl implements MdeOnboardings { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public MdeOnboardingsImpl( - MdeOnboardingsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public MdeOnboardingsImpl(MdeOnboardingsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -31,10 +31,7 @@ public MdeOnboardingsImpl( public Response listWithResponse(Context context) { Response inner = this.serviceClient().listWithResponse(context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new MdeOnboardingDataListImpl(inner.getValue(), this.manager())); } else { return null; @@ -53,10 +50,7 @@ public MdeOnboardingDataList list() { public Response getWithResponse(Context context) { Response inner = this.serviceClient().getWithResponse(context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new MdeOnboardingDataImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationResultAutoGeneratedImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationResultAutoGeneratedImpl.java index dc4835d2f4bc1..bbeb3b70d2c4f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationResultAutoGeneratedImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationResultAutoGeneratedImpl.java @@ -13,8 +13,7 @@ public final class OperationResultAutoGeneratedImpl implements OperationResultAu private final com.azure.resourcemanager.security.SecurityManager serviceManager; - OperationResultAutoGeneratedImpl( - OperationResultAutoGeneratedInner innerObject, + OperationResultAutoGeneratedImpl(OperationResultAutoGeneratedInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationStatusResultImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationStatusResultImpl.java new file mode 100644 index 0000000000000..b16f6acad6f2d --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationStatusResultImpl.java @@ -0,0 +1,71 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.exception.ManagementError; +import com.azure.resourcemanager.security.fluent.models.OperationStatusResultInner; +import com.azure.resourcemanager.security.models.OperationStatusResult; +import java.time.OffsetDateTime; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public final class OperationStatusResultImpl implements OperationStatusResult { + private OperationStatusResultInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + OperationStatusResultImpl(OperationStatusResultInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String status() { + return this.innerModel().status(); + } + + public Float percentComplete() { + return this.innerModel().percentComplete(); + } + + public OffsetDateTime startTime() { + return this.innerModel().startTime(); + } + + public OffsetDateTime endTime() { + return this.innerModel().endTime(); + } + + public List operations() { + List inner = this.innerModel().operations(); + if (inner != null) { + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new OperationStatusResultImpl(inner1, this.manager())).collect(Collectors.toList())); + } else { + return Collections.emptyList(); + } + } + + public ManagementError error() { + return this.innerModel().error(); + } + + public OperationStatusResultInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsClientImpl.java index df2c60d733e3c..ce9ead3feba03 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.OperationList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in OperationsClient. */ +/** + * An instance of this class provides access to all the operations defined in OperationsClient. + */ public final class OperationsClientImpl implements OperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final OperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of OperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ OperationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(OperationsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,100 +62,74 @@ public final class OperationsClientImpl implements OperationsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterOperat") public interface OperationsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Security/operations") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Exposes all available operations for discovery purposes. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of possible operations for Microsoft.Security resource provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Exposes all available operations for discovery purposes. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of possible operations for Microsoft.Security resource provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Exposes all available operations for discovery purposes. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of possible operations for Microsoft.Security resource provider as paginated response with {@link - * PagedFlux}. + * @return list of possible operations for Microsoft.Security resource provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -158,27 +138,27 @@ private PagedFlux listAsync() { /** * Exposes all available operations for discovery purposes. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of possible operations for Microsoft.Security resource provider as paginated response with {@link - * PagedFlux}. + * @return list of possible operations for Microsoft.Security resource provider as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Exposes all available operations for discovery purposes. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of possible operations for Microsoft.Security resource provider as paginated response with {@link - * PagedIterable}. + * @return list of possible operations for Microsoft.Security resource provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -187,13 +167,13 @@ public PagedIterable list() { /** * Exposes all available operations for discovery purposes. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of possible operations for Microsoft.Security resource provider as paginated response with {@link - * PagedIterable}. + * @return list of possible operations for Microsoft.Security resource provider as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -202,14 +182,15 @@ public PagedIterable list(Context context) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of possible operations for Microsoft.Security resource provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -217,37 +198,28 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of possible operations for Microsoft.Security resource provider along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -255,23 +227,13 @@ private Mono> listNextSinglePageAsync(String nextL return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsImpl.java index 4d0d7488ce5f4..b40714e8b3e83 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/OperationsImpl.java @@ -19,20 +19,20 @@ public final class OperationsImpl implements Operations { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public OperationsImpl( - OperationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public OperationsImpl(OperationsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new OperationImpl(inner1, this.manager())); } private OperationsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingImpl.java index 46e348a6d463e..ab7e27cea4858 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingImpl.java @@ -5,9 +5,12 @@ package com.azure.resourcemanager.security.implementation; import com.azure.resourcemanager.security.fluent.models.PricingInner; +import com.azure.resourcemanager.security.models.Enforce; import com.azure.resourcemanager.security.models.Extension; +import com.azure.resourcemanager.security.models.Inherited; import com.azure.resourcemanager.security.models.Pricing; import com.azure.resourcemanager.security.models.PricingTier; +import com.azure.resourcemanager.security.models.ResourcesCoverageStatus; import java.time.Duration; import java.time.OffsetDateTime; import java.util.Collections; @@ -51,12 +54,24 @@ public OffsetDateTime enablementTime() { return this.innerModel().enablementTime(); } - public Boolean deprecated() { - return this.innerModel().deprecated(); + public Enforce enforce() { + return this.innerModel().enforce(); } - public List replacedBy() { - List inner = this.innerModel().replacedBy(); + public Inherited inherited() { + return this.innerModel().inherited(); + } + + public String inheritedFrom() { + return this.innerModel().inheritedFrom(); + } + + public ResourcesCoverageStatus resourcesCoverageStatus() { + return this.innerModel().resourcesCoverageStatus(); + } + + public List extensions() { + List inner = this.innerModel().extensions(); if (inner != null) { return Collections.unmodifiableList(inner); } else { @@ -64,8 +79,12 @@ public List replacedBy() { } } - public List extensions() { - List inner = this.innerModel().extensions(); + public Boolean deprecated() { + return this.innerModel().deprecated(); + } + + public List replacedBy() { + List inner = this.innerModel().replacedBy(); if (inner != null) { return Collections.unmodifiableList(inner); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingListImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingListImpl.java index fce69ede7db36..17437a75891a0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingListImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingListImpl.java @@ -25,9 +25,8 @@ public final class PricingListImpl implements PricingList { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner.stream().map(inner1 -> new PricingImpl(inner1, this.manager())).collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new PricingImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsClientImpl.java index d1e7fb2693c0c..bc2876b3ba79b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsClientImpl.java @@ -5,6 +5,7 @@ package com.azure.resourcemanager.security.implementation; import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; import com.azure.core.annotation.ExpectedResponses; import com.azure.core.annotation.Get; import com.azure.core.annotation.HeaderParam; @@ -28,17 +29,23 @@ import com.azure.resourcemanager.security.fluent.models.PricingListInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in PricingsClient. */ +/** + * An instance of this class provides access to all the operations defined in PricingsClient. + */ public final class PricingsClientImpl implements PricingsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final PricingsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of PricingsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ PricingsClientImpl(SecurityCenterImpl client) { @@ -53,410 +60,569 @@ public final class PricingsClientImpl implements PricingsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterPricin") public interface PricingsService { - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/{scopeId}/providers/Microsoft.Security/pricings/{pricingName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scopeId", encoded = true) String scopeId, + @PathParam("pricingName") String pricingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/{scopeId}/providers/Microsoft.Security/pricings/{pricingName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("pricingName") String pricingName, - @HeaderParam("Accept") String accept, - Context context); + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scopeId", encoded = true) String scopeId, + @PathParam("pricingName") String pricingName, @BodyParam("application/json") PricingInner pricing, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Delete("/{scopeId}/providers/Microsoft.Security/pricings/{pricingName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("pricingName") String pricingName, - @BodyParam("application/json") PricingInner pricing, - @HeaderParam("Accept") String accept, - Context context); + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam(value = "scopeId", encoded = true) String scopeId, @PathParam("pricingName") String pricingName, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/{scopeId}/providers/Microsoft.Security/pricings") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scopeId", encoded = true) String scopeId, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); } /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response along with {@link Response} on successful completion of {@link - * Mono}. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id) along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync() { + private Mono> getWithResponseAsync(String scopeId, String pricingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); } - final String apiVersion = "2023-01-01"; + if (pricingName == null) { + return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); + } + final String apiVersion = "2024-01-01"; final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) + context -> service.get(this.client.getEndpoint(), apiVersion, scopeId, pricingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response along with {@link Response} on successful completion of {@link - * Mono}. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id) along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync(Context context) { + private Mono> getWithResponseAsync(String scopeId, String pricingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); + } + if (pricingName == null) { + return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); } - final String apiVersion = "2023-01-01"; + final String apiVersion = "2024-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context); + return service.get(this.client.getEndpoint(), apiVersion, scopeId, pricingName, accept, context); } /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response on successful completion of {@link Mono}. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id) on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listAsync() { - return listWithResponseAsync().flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono getAsync(String scopeId, String pricingName) { + return getWithResponseAsync(scopeId, pricingName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response along with {@link Response}. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id) along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listWithResponse(Context context) { - return listWithResponseAsync(context).block(); + public Response getWithResponse(String scopeId, String pricingName, Context context) { + return getWithResponseAsync(scopeId, pricingName, context).block(); } /** - * Lists Microsoft Defender for Cloud pricing configurations in the subscription. - * + * Get the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). At the resource level, supported resource types are 'VirtualMachines, VMSS and ARC Machines'. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param pricingName name of the pricing configuration. + * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of pricing configurations response. + * @return the Defender plans pricing configurations of the selected scope (valid scopes are resource id or a + * subscription id). */ @ServiceMethod(returns = ReturnType.SINGLE) - public PricingListInner list() { - return listWithResponse(Context.NONE).getValue(); + public PricingInner get(String scopeId, String pricingName) { + return getWithResponse(scopeId, pricingName, Context.NONE).getValue(); } /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription along with {@link - * Response} on successful completion of {@link Mono}. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String pricingName) { + private Mono> updateWithResponseAsync(String scopeId, String pricingName, + PricingInner pricing) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); } - final String apiVersion = "2023-01-01"; + if (pricing == null) { + return Mono.error(new IllegalArgumentException("Parameter pricing is required and cannot be null.")); + } else { + pricing.validate(); + } + final String apiVersion = "2024-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), apiVersion, scopeId, pricingName, pricing, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription along with {@link - * Response} on successful completion of {@link Mono}. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String pricingName, Context context) { + private Mono> updateWithResponseAsync(String scopeId, String pricingName, + PricingInner pricing, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); } - final String apiVersion = "2023-01-01"; + if (pricing == null) { + return Mono.error(new IllegalArgumentException("Parameter pricing is required and cannot be null.")); + } else { + pricing.validate(); + } + final String apiVersion = "2024-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), pricingName, accept, context); + return service.update(this.client.getEndpoint(), apiVersion, scopeId, pricingName, pricing, accept, context); } /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription on successful - * completion of {@link Mono}. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard on successful completion + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String pricingName) { - return getWithResponseAsync(pricingName).flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono updateAsync(String scopeId, String pricingName, PricingInner pricing) { + return updateWithResponseAsync(scopeId, pricingName, pricing).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription along with {@link - * Response}. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String pricingName, Context context) { - return getWithResponseAsync(pricingName, context).block(); + public Response updateWithResponse(String scopeId, String pricingName, PricingInner pricing, + Context context) { + return updateWithResponseAsync(scopeId, pricingName, pricing, context).block(); } /** - * Gets a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Updates a provided Microsoft Defender for Cloud pricing configuration in the scope. Valid scopes are: + * subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and ARC Machines' and + * only for plan='VirtualMachines' and subPlan='P1'). + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). * @param pricingName name of the pricing configuration. + * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a provided Microsoft Defender for Cloud pricing configuration in the subscription. + * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PricingInner get(String pricingName) { - return getWithResponse(pricingName, Context.NONE).getValue(); + public PricingInner update(String scopeId, String pricingName, PricingInner pricing) { + return updateWithResponse(scopeId, pricingName, pricing, Context.NONE).getValue(); } /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with {@link - * Response} on successful completion of {@link Mono}. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync(String pricingName, PricingInner pricing) { + private Mono> deleteWithResponseAsync(String scopeId, String pricingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); } - if (pricing == null) { - return Mono.error(new IllegalArgumentException("Parameter pricing is required and cannot be null.")); - } else { - pricing.validate(); - } - final String apiVersion = "2023-01-01"; + final String apiVersion = "2024-01-01"; final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - pricing, - accept, - context)) + context -> service.delete(this.client.getEndpoint(), apiVersion, scopeId, pricingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with {@link - * Response} on successful completion of {@link Mono}. + * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String pricingName, PricingInner pricing, Context context) { + private Mono> deleteWithResponseAsync(String scopeId, String pricingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } - if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); } - if (pricing == null) { - return Mono.error(new IllegalArgumentException("Parameter pricing is required and cannot be null.")); - } else { - pricing.validate(); - } - final String apiVersion = "2023-01-01"; + final String apiVersion = "2024-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - pricing, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, scopeId, pricingName, accept, context); } /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard on successful completion - * of {@link Mono}. + * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(String pricingName, PricingInner pricing) { - return updateWithResponseAsync(pricingName, pricing).flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono deleteAsync(String scopeId, String pricingName) { + return deleteWithResponseAsync(scopeId, pricingName).flatMap(ignored -> Mono.empty()); } /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard along with {@link - * Response}. + * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse(String pricingName, PricingInner pricing, Context context) { - return updateWithResponseAsync(pricingName, pricing, context).block(); + public Response deleteWithResponse(String scopeId, String pricingName, Context context) { + return deleteWithResponseAsync(scopeId, pricingName, context).block(); } /** - * Updates a provided Microsoft Defender for Cloud pricing configuration in the subscription. - * + * Deletes a provided Microsoft Defender for Cloud pricing configuration in a specific resource. Valid only for + * resource scope (Supported resources are: 'VirtualMachines, VMSS and ARC MachinesS'). + * + * @param scopeId The identifier of the resource, (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}). * @param pricingName name of the pricing configuration. - * @param pricing Pricing object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return microsoft Defender for Cloud is provided in two pricing tiers: free and standard. */ @ServiceMethod(returns = ReturnType.SINGLE) - public PricingInner update(String pricingName, PricingInner pricing) { - return updateWithResponse(pricingName, pricing, Context.NONE).getValue(); + public void delete(String scopeId, String pricingName) { + deleteWithResponse(scopeId, pricingName, Context.NONE); + } + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param filter OData filter. Optional. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(String scopeId, String filter) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); + } + final String apiVersion = "2024-01-01"; + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.list(this.client.getEndpoint(), apiVersion, scopeId, filter, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param filter OData filter. Optional. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response along with {@link Response} on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(String scopeId, String filter, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (scopeId == null) { + return Mono.error(new IllegalArgumentException("Parameter scopeId is required and cannot be null.")); + } + final String apiVersion = "2024-01-01"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.list(this.client.getEndpoint(), apiVersion, scopeId, filter, accept, context); + } + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAsync(String scopeId) { + final String filter = null; + return listWithResponseAsync(scopeId, filter).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @param filter OData filter. Optional. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listWithResponse(String scopeId, String filter, Context context) { + return listWithResponseAsync(scopeId, filter, context).block(); + } + + /** + * Lists Microsoft Defender for Cloud pricing configurations of the scopeId, that match the optional given $filter. + * Valid scopes are: subscription id or a specific resource id (Supported resources are: 'VirtualMachines, VMSS and + * ARC Machines'). Valid $filter is: 'name in ({planName1},{planName2},...)'. If $filter is not provided, the + * unfiltered list will be returned. If '$filter=name in (planName1,planName2)' is provided, the returned list + * includes the pricings set for 'planName1' and 'planName2' only. + * + * @param scopeId The scope id of the pricing. Valid scopes are: subscription (format: + * 'subscriptions/{subscriptionId}'), or a specific resource (format: + * 'subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceProviderNamespace}/{resourceType}/{resourceName}) + * - Supported resources are (VirtualMachines). + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of pricing configurations response. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public PricingListInner list(String scopeId) { + final String filter = null; + return listWithResponse(scopeId, filter, Context.NONE).getValue(); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsImpl.java index 99ea26d487348..d74c9313bcc98 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/PricingsImpl.java @@ -27,43 +27,38 @@ public PricingsImpl(PricingsClient innerClient, com.azure.resourcemanager.securi this.serviceManager = serviceManager; } - public Response listWithResponse(Context context) { - Response inner = this.serviceClient().listWithResponse(context); + public Response getWithResponse(String scopeId, String pricingName, Context context) { + Response inner = this.serviceClient().getWithResponse(scopeId, pricingName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new PricingListImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new PricingImpl(inner.getValue(), this.manager())); } else { return null; } } - public PricingList list() { - PricingListInner inner = this.serviceClient().list(); + public Pricing get(String scopeId, String pricingName) { + PricingInner inner = this.serviceClient().get(scopeId, pricingName); if (inner != null) { - return new PricingListImpl(inner, this.manager()); + return new PricingImpl(inner, this.manager()); } else { return null; } } - public Response getWithResponse(String pricingName, Context context) { - Response inner = this.serviceClient().getWithResponse(pricingName, context); + public Response updateWithResponse(String scopeId, String pricingName, PricingInner pricing, + Context context) { + Response inner = this.serviceClient().updateWithResponse(scopeId, pricingName, pricing, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new PricingImpl(inner.getValue(), this.manager())); } else { return null; } } - public Pricing get(String pricingName) { - PricingInner inner = this.serviceClient().get(pricingName); + public Pricing update(String scopeId, String pricingName, PricingInner pricing) { + PricingInner inner = this.serviceClient().update(scopeId, pricingName, pricing); if (inner != null) { return new PricingImpl(inner, this.manager()); } else { @@ -71,23 +66,28 @@ public Pricing get(String pricingName) { } } - public Response updateWithResponse(String pricingName, PricingInner pricing, Context context) { - Response inner = this.serviceClient().updateWithResponse(pricingName, pricing, context); + public Response deleteByResourceGroupWithResponse(String scopeId, String pricingName, Context context) { + return this.serviceClient().deleteWithResponse(scopeId, pricingName, context); + } + + public void deleteByResourceGroup(String scopeId, String pricingName) { + this.serviceClient().delete(scopeId, pricingName); + } + + public Response listWithResponse(String scopeId, String filter, Context context) { + Response inner = this.serviceClient().listWithResponse(scopeId, filter, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), - new PricingImpl(inner.getValue(), this.manager())); + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new PricingListImpl(inner.getValue(), this.manager())); } else { return null; } } - public Pricing update(String pricingName, PricingInner pricing) { - PricingInner inner = this.serviceClient().update(pricingName, pricing); + public PricingList list(String scopeId) { + PricingListInner inner = this.serviceClient().list(scopeId); if (inner != null) { - return new PricingImpl(inner, this.manager()); + return new PricingListImpl(inner, this.manager()); } else { return null; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentImpl.java index d83de36d26c93..8345f2183601e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentImpl.java @@ -13,8 +13,7 @@ public final class RegulatoryComplianceAssessmentImpl implements RegulatoryCompl private final com.azure.resourcemanager.security.SecurityManager serviceManager; - RegulatoryComplianceAssessmentImpl( - RegulatoryComplianceAssessmentInner innerObject, + RegulatoryComplianceAssessmentImpl(RegulatoryComplianceAssessmentInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsClientImpl.java index 90b9f8738eda7..5da14f069015f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsClientImpl.java @@ -30,80 +30,71 @@ import com.azure.resourcemanager.security.models.RegulatoryComplianceAssessmentList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in RegulatoryComplianceAssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in RegulatoryComplianceAssessmentsClient. + */ public final class RegulatoryComplianceAssessmentsClientImpl implements RegulatoryComplianceAssessmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final RegulatoryComplianceAssessmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of RegulatoryComplianceAssessmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ RegulatoryComplianceAssessmentsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - RegulatoryComplianceAssessmentsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(RegulatoryComplianceAssessmentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterRegulatoryComplianceAssessments to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for SecurityCenterRegulatoryComplianceAssessments to be used by the + * proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterRegula") public interface RegulatoryComplianceAssessmentsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls/{regulatoryComplianceControlName}/regulatoryComplianceAssessments") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls/{regulatoryComplianceControlName}/regulatoryComplianceAssessments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("regulatoryComplianceStandardName") String regulatoryComplianceStandardName, @PathParam("regulatoryComplianceControlName") String regulatoryComplianceControlName, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls/{regulatoryComplianceControlName}/regulatoryComplianceAssessments/{regulatoryComplianceAssessmentName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls/{regulatoryComplianceControlName}/regulatoryComplianceAssessments/{regulatoryComplianceAssessmentName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("regulatoryComplianceStandardName") String regulatoryComplianceStandardName, @PathParam("regulatoryComplianceControlName") String regulatoryComplianceControlName, @PathParam("regulatoryComplianceAssessmentName") String regulatoryComplianceAssessmentName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param filter OData filter. Optional. @@ -111,65 +102,40 @@ Mono> listNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance assessment response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync( String regulatoryComplianceStandardName, String regulatoryComplianceControlName, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } if (regulatoryComplianceControlName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceControlName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceControlName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, regulatoryComplianceControlName, filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param filter OData filter. Optional. @@ -178,65 +144,41 @@ private Mono> listSinglePageA * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance assessment response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String filter, + String regulatoryComplianceStandardName, String regulatoryComplianceControlName, String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } if (regulatoryComplianceControlName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceControlName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceControlName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, regulatoryComplianceControlName, filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param filter OData filter. Optional. @@ -246,8 +188,8 @@ private Mono> listSinglePageA * @return list of regulatory compliance assessment response as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName, String filter) { + private PagedFlux listAsync(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String filter) { return new PagedFlux<>( () -> listSinglePageAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter), nextLink -> listNextSinglePageAsync(nextLink)); @@ -255,7 +197,7 @@ private PagedFlux listAsync( /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -264,8 +206,8 @@ private PagedFlux listAsync( * @return list of regulatory compliance assessment response as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { + private PagedFlux listAsync(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName) { final String filter = null; return new PagedFlux<>( () -> listSinglePageAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter), @@ -274,7 +216,7 @@ private PagedFlux listAsync( /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param filter OData filter. Optional. @@ -285,20 +227,15 @@ private PagedFlux listAsync( * @return list of regulatory compliance assessment response as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String filter, - Context context) { - return new PagedFlux<>( - () -> - listSinglePageAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter, context), - nextLink -> listNextSinglePageAsync(nextLink, context)); + private PagedFlux listAsync(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String filter, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(regulatoryComplianceStandardName, + regulatoryComplianceControlName, filter, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -307,8 +244,8 @@ private PagedFlux listAsync( * @return list of regulatory compliance assessment response as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { + public PagedIterable list(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName) { final String filter = null; return new PagedIterable<>( listAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter)); @@ -316,7 +253,7 @@ public PagedIterable list( /** * Details and state of assessments mapped to selected regulatory compliance control. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param filter OData filter. Optional. @@ -327,18 +264,15 @@ public PagedIterable list( * @return list of regulatory compliance assessment response as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String filter, - Context context) { + public PagedIterable list(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String filter, Context context) { return new PagedIterable<>( listAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter, context)); } /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -346,64 +280,44 @@ public PagedIterable list( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return regulatory compliance assessment details and state along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, + String regulatoryComplianceStandardName, String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } if (regulatoryComplianceControlName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceControlName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceControlName is required and cannot be null.")); } if (regulatoryComplianceAssessmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceAssessmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceAssessmentName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - regulatoryComplianceAssessmentName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, regulatoryComplianceControlName, regulatoryComplianceAssessmentName, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -412,62 +326,43 @@ private Mono> getWithResponseAsync * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return regulatory compliance assessment details and state along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName, - Context context) { + String regulatoryComplianceStandardName, String regulatoryComplianceControlName, + String regulatoryComplianceAssessmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } if (regulatoryComplianceControlName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceControlName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceControlName is required and cannot be null.")); } if (regulatoryComplianceAssessmentName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceAssessmentName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceAssessmentName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - regulatoryComplianceAssessmentName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, regulatoryComplianceControlName, regulatoryComplianceAssessmentName, + accept, context); } /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -477,18 +372,15 @@ private Mono> getWithResponseAsync * @return regulatory compliance assessment details and state on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName) { - return getWithResponseAsync( - regulatoryComplianceStandardName, regulatoryComplianceControlName, regulatoryComplianceAssessmentName) - .flatMap(res -> Mono.justOrEmpty(res.getValue())); + private Mono getAsync(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName) { + return getWithResponseAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, + regulatoryComplianceAssessmentName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -499,22 +391,15 @@ private Mono getAsync( * @return regulatory compliance assessment details and state along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName, - Context context) { - return getWithResponseAsync( - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - regulatoryComplianceAssessmentName, - context) - .block(); + public Response getWithResponse(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName, Context context) { + return getWithResponseAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, + regulatoryComplianceAssessmentName, context).block(); } /** * Supported regulatory compliance details and state for selected assessment. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param regulatoryComplianceAssessmentName Name of the regulatory compliance assessment object. @@ -524,28 +409,23 @@ public Response getWithResponse( * @return regulatory compliance assessment details and state. */ @ServiceMethod(returns = ReturnType.SINGLE) - public RegulatoryComplianceAssessmentInner get( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName) { - return getWithResponse( - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - regulatoryComplianceAssessmentName, - Context.NONE) - .getValue(); + public RegulatoryComplianceAssessmentInner get(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName) { + return getWithResponse(regulatoryComplianceStandardName, regulatoryComplianceControlName, + regulatoryComplianceAssessmentName, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance assessment response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -553,62 +433,43 @@ private Mono> listNextSingleP return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance assessment response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsImpl.java index 222a4ee49b8b9..93ca5a7b9bd51 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceAssessmentsImpl.java @@ -21,67 +21,45 @@ public final class RegulatoryComplianceAssessmentsImpl implements RegulatoryComp private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public RegulatoryComplianceAssessmentsImpl( - RegulatoryComplianceAssessmentsClient innerClient, + public RegulatoryComplianceAssessmentsImpl(RegulatoryComplianceAssessmentsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable list( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { - PagedIterable inner = - this.serviceClient().list(regulatoryComplianceStandardName, regulatoryComplianceControlName); - return Utils.mapPage(inner, inner1 -> new RegulatoryComplianceAssessmentImpl(inner1, this.manager())); + public PagedIterable list(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName) { + PagedIterable inner + = this.serviceClient().list(regulatoryComplianceStandardName, regulatoryComplianceControlName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new RegulatoryComplianceAssessmentImpl(inner1, this.manager())); } - public PagedIterable list( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String filter, - Context context) { - PagedIterable inner = - this - .serviceClient() - .list(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter, context); - return Utils.mapPage(inner, inner1 -> new RegulatoryComplianceAssessmentImpl(inner1, this.manager())); + public PagedIterable list(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String filter, Context context) { + PagedIterable inner = this.serviceClient() + .list(regulatoryComplianceStandardName, regulatoryComplianceControlName, filter, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new RegulatoryComplianceAssessmentImpl(inner1, this.manager())); } - public Response getWithResponse( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse( - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - regulatoryComplianceAssessmentName, - context); + public Response getWithResponse(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName, Context context) { + Response inner + = this.serviceClient().getWithResponse(regulatoryComplianceStandardName, regulatoryComplianceControlName, + regulatoryComplianceAssessmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RegulatoryComplianceAssessmentImpl(inner.getValue(), this.manager())); } else { return null; } } - public RegulatoryComplianceAssessment get( - String regulatoryComplianceStandardName, - String regulatoryComplianceControlName, - String regulatoryComplianceAssessmentName) { - RegulatoryComplianceAssessmentInner inner = - this - .serviceClient() - .get( - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - regulatoryComplianceAssessmentName); + public RegulatoryComplianceAssessment get(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, String regulatoryComplianceAssessmentName) { + RegulatoryComplianceAssessmentInner inner = this.serviceClient().get(regulatoryComplianceStandardName, + regulatoryComplianceControlName, regulatoryComplianceAssessmentName); if (inner != null) { return new RegulatoryComplianceAssessmentImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlImpl.java index b51b1ed37d113..9c60baec0bd40 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlImpl.java @@ -13,8 +13,7 @@ public final class RegulatoryComplianceControlImpl implements RegulatoryComplian private final com.azure.resourcemanager.security.SecurityManager serviceManager; - RegulatoryComplianceControlImpl( - RegulatoryComplianceControlInner innerObject, + RegulatoryComplianceControlImpl(RegulatoryComplianceControlInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsClientImpl.java index a1bf42ba21daf..96e7ae2abab03 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsClientImpl.java @@ -30,24 +30,28 @@ import com.azure.resourcemanager.security.models.RegulatoryComplianceControlList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in RegulatoryComplianceControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in RegulatoryComplianceControlsClient. + */ public final class RegulatoryComplianceControlsClientImpl implements RegulatoryComplianceControlsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final RegulatoryComplianceControlsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of RegulatoryComplianceControlsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ RegulatoryComplianceControlsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - RegulatoryComplianceControlsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(RegulatoryComplianceControlsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -58,106 +62,73 @@ public final class RegulatoryComplianceControlsClientImpl implements RegulatoryC @Host("{$host}") @ServiceInterface(name = "SecurityCenterRegula") public interface RegulatoryComplianceControlsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("regulatoryComplianceStandardName") String regulatoryComplianceStandardName, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls/{regulatoryComplianceControlName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}/regulatoryComplianceControls/{regulatoryComplianceControlName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("regulatoryComplianceStandardName") String regulatoryComplianceStandardName, @PathParam("regulatoryComplianceControlName") String regulatoryComplianceControlName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance controls response along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String regulatoryComplianceStandardName, String filter) { + private Mono> + listSinglePageAsync(String regulatoryComplianceStandardName, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param filter OData filter. Optional. * @param context The context to associate with this operation. @@ -165,55 +136,36 @@ private Mono> listSinglePageAsyn * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance controls response along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String regulatoryComplianceStandardName, String filter, Context context) { + private Mono> + listSinglePageAsync(String regulatoryComplianceStandardName, String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -222,16 +174,15 @@ private Mono> listSinglePageAsyn * @return list of regulatory compliance controls response as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String regulatoryComplianceStandardName, String filter) { - return new PagedFlux<>( - () -> listSinglePageAsync(regulatoryComplianceStandardName, filter), + private PagedFlux listAsync(String regulatoryComplianceStandardName, + String filter) { + return new PagedFlux<>(() -> listSinglePageAsync(regulatoryComplianceStandardName, filter), nextLink -> listNextSinglePageAsync(nextLink)); } /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -241,14 +192,13 @@ private PagedFlux listAsync( @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String regulatoryComplianceStandardName) { final String filter = null; - return new PagedFlux<>( - () -> listSinglePageAsync(regulatoryComplianceStandardName, filter), + return new PagedFlux<>(() -> listSinglePageAsync(regulatoryComplianceStandardName, filter), nextLink -> listNextSinglePageAsync(nextLink)); } /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param filter OData filter. Optional. * @param context The context to associate with this operation. @@ -258,16 +208,15 @@ private PagedFlux listAsync(String regulatoryC * @return list of regulatory compliance controls response as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String regulatoryComplianceStandardName, String filter, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(regulatoryComplianceStandardName, filter, context), + private PagedFlux listAsync(String regulatoryComplianceStandardName, + String filter, Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(regulatoryComplianceStandardName, filter, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -282,7 +231,7 @@ public PagedIterable list(String regulatoryCom /** * All supported regulatory compliance controls details and state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param filter OData filter. Optional. * @param context The context to associate with this operation. @@ -292,69 +241,52 @@ public PagedIterable list(String regulatoryCom * @return list of regulatory compliance controls response as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String regulatoryComplianceStandardName, String filter, Context context) { + public PagedIterable list(String regulatoryComplianceStandardName, String filter, + Context context) { return new PagedIterable<>(listAsync(regulatoryComplianceStandardName, filter, context)); } /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return regulatory compliance control details and state along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { + private Mono> + getWithResponseAsync(String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } if (regulatoryComplianceControlName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceControlName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceControlName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, regulatoryComplianceControlName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param context The context to associate with this operation. @@ -362,52 +294,37 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return regulatory compliance control details and state along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync( String regulatoryComplianceStandardName, String regulatoryComplianceControlName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } if (regulatoryComplianceControlName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceControlName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceControlName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - regulatoryComplianceControlName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, regulatoryComplianceControlName, accept, context); } /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -416,15 +333,15 @@ private Mono> getWithResponseAsync( * @return regulatory compliance control details and state on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { + private Mono getAsync(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName) { return getWithResponseAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @param context The context to associate with this operation. @@ -434,14 +351,14 @@ private Mono getAsync( * @return regulatory compliance control details and state along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName, Context context) { + public Response getWithResponse(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, Context context) { return getWithResponseAsync(regulatoryComplianceStandardName, regulatoryComplianceControlName, context).block(); } /** * Selected regulatory compliance control details and state. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param regulatoryComplianceControlName Name of the regulatory compliance control object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -450,22 +367,23 @@ public Response getWithResponse( * @return regulatory compliance control details and state. */ @ServiceMethod(returns = ReturnType.SINGLE) - public RegulatoryComplianceControlInner get( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { + public RegulatoryComplianceControlInner get(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName) { return getWithResponse(regulatoryComplianceStandardName, regulatoryComplianceControlName, Context.NONE) .getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance controls response along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -473,62 +391,43 @@ private Mono> listNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance controls response along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsImpl.java index 31f2bb5591397..253dcaf1bbf58 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceControlsImpl.java @@ -21,47 +21,43 @@ public final class RegulatoryComplianceControlsImpl implements RegulatoryComplia private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public RegulatoryComplianceControlsImpl( - RegulatoryComplianceControlsClient innerClient, + public RegulatoryComplianceControlsImpl(RegulatoryComplianceControlsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list(String regulatoryComplianceStandardName) { - PagedIterable inner = - this.serviceClient().list(regulatoryComplianceStandardName); - return Utils.mapPage(inner, inner1 -> new RegulatoryComplianceControlImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(regulatoryComplianceStandardName); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new RegulatoryComplianceControlImpl(inner1, this.manager())); } - public PagedIterable list( - String regulatoryComplianceStandardName, String filter, Context context) { - PagedIterable inner = - this.serviceClient().list(regulatoryComplianceStandardName, filter, context); - return Utils.mapPage(inner, inner1 -> new RegulatoryComplianceControlImpl(inner1, this.manager())); + public PagedIterable list(String regulatoryComplianceStandardName, String filter, + Context context) { + PagedIterable inner + = this.serviceClient().list(regulatoryComplianceStandardName, filter, context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new RegulatoryComplianceControlImpl(inner1, this.manager())); } - public Response getWithResponse( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(regulatoryComplianceStandardName, regulatoryComplianceControlName, context); + public Response getWithResponse(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName, Context context) { + Response inner = this.serviceClient() + .getWithResponse(regulatoryComplianceStandardName, regulatoryComplianceControlName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RegulatoryComplianceControlImpl(inner.getValue(), this.manager())); } else { return null; } } - public RegulatoryComplianceControl get( - String regulatoryComplianceStandardName, String regulatoryComplianceControlName) { - RegulatoryComplianceControlInner inner = - this.serviceClient().get(regulatoryComplianceStandardName, regulatoryComplianceControlName); + public RegulatoryComplianceControl get(String regulatoryComplianceStandardName, + String regulatoryComplianceControlName) { + RegulatoryComplianceControlInner inner + = this.serviceClient().get(regulatoryComplianceStandardName, regulatoryComplianceControlName); if (inner != null) { return new RegulatoryComplianceControlImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardImpl.java index 19762eafd2517..ac80baff85296 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardImpl.java @@ -13,8 +13,7 @@ public final class RegulatoryComplianceStandardImpl implements RegulatoryComplia private final com.azure.resourcemanager.security.SecurityManager serviceManager; - RegulatoryComplianceStandardImpl( - RegulatoryComplianceStandardInner innerObject, + RegulatoryComplianceStandardImpl(RegulatoryComplianceStandardInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsClientImpl.java index 9c58140377cb1..5682fe203485d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsClientImpl.java @@ -30,26 +30,28 @@ import com.azure.resourcemanager.security.models.RegulatoryComplianceStandardList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in RegulatoryComplianceStandardsClient. */ +/** + * An instance of this class provides access to all the operations defined in RegulatoryComplianceStandardsClient. + */ public final class RegulatoryComplianceStandardsClientImpl implements RegulatoryComplianceStandardsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final RegulatoryComplianceStandardsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of RegulatoryComplianceStandardsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ RegulatoryComplianceStandardsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - RegulatoryComplianceStandardsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(RegulatoryComplianceStandardsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,135 +62,95 @@ public final class RegulatoryComplianceStandardsClientImpl implements Regulatory @Host("{$host}") @ServiceInterface(name = "SecurityCenterRegula") public interface RegulatoryComplianceStandardsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/regulatoryComplianceStandards/{regulatoryComplianceStandardName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("regulatoryComplianceStandardName") String regulatoryComplianceStandardName, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Supported regulatory compliance standards details and state. - * + * * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Supported regulatory compliance standards details and state. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), filter, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Supported regulatory compliance standards details and state. - * + * * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -202,7 +164,7 @@ private PagedFlux listAsync(String filter) { /** * Supported regulatory compliance standards details and state. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response as paginated response with {@link PagedFlux}. @@ -215,7 +177,7 @@ private PagedFlux listAsync() { /** * Supported regulatory compliance standards details and state. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -225,13 +187,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String filter, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(filter, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(filter, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Supported regulatory compliance standards details and state. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response as paginated response with {@link PagedIterable}. @@ -244,7 +206,7 @@ public PagedIterable list() { /** * Supported regulatory compliance standards details and state. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -259,99 +221,73 @@ public PagedIterable list(String filter, Cont /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return regulatory compliance standard details and state along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String regulatoryComplianceStandardName) { + private Mono> + getWithResponseAsync(String regulatoryComplianceStandardName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return regulatory compliance standard details and state along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String regulatoryComplianceStandardName, Context context) { + private Mono> + getWithResponseAsync(String regulatoryComplianceStandardName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (regulatoryComplianceStandardName == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter regulatoryComplianceStandardName is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter regulatoryComplianceStandardName is required and cannot be null.")); } final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - regulatoryComplianceStandardName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + regulatoryComplianceStandardName, accept, context); } /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -365,7 +301,7 @@ private Mono getAsync(String regulatoryCompli /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -374,14 +310,14 @@ private Mono getAsync(String regulatoryCompli * @return regulatory compliance standard details and state along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String regulatoryComplianceStandardName, Context context) { + public Response getWithResponse(String regulatoryComplianceStandardName, + Context context) { return getWithResponseAsync(regulatoryComplianceStandardName, context).block(); } /** * Supported regulatory compliance details state for selected standard. - * + * * @param regulatoryComplianceStandardName Name of the regulatory compliance standard object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -395,14 +331,15 @@ public RegulatoryComplianceStandardInner get(String regulatoryComplianceStandard /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -410,62 +347,43 @@ private Mono> listNextSinglePag return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of regulatory compliance standards response along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsImpl.java index 8f57bbcd3d41d..b6a36939c29e0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RegulatoryComplianceStandardsImpl.java @@ -21,8 +21,7 @@ public final class RegulatoryComplianceStandardsImpl implements RegulatoryCompli private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public RegulatoryComplianceStandardsImpl( - RegulatoryComplianceStandardsClient innerClient, + public RegulatoryComplianceStandardsImpl(RegulatoryComplianceStandardsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,23 +29,22 @@ public RegulatoryComplianceStandardsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new RegulatoryComplianceStandardImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new RegulatoryComplianceStandardImpl(inner1, this.manager())); } public PagedIterable list(String filter, Context context) { PagedIterable inner = this.serviceClient().list(filter, context); - return Utils.mapPage(inner, inner1 -> new RegulatoryComplianceStandardImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new RegulatoryComplianceStandardImpl(inner1, this.manager())); } - public Response getWithResponse( - String regulatoryComplianceStandardName, Context context) { - Response inner = - this.serviceClient().getWithResponse(regulatoryComplianceStandardName, context); + public Response getWithResponse(String regulatoryComplianceStandardName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(regulatoryComplianceStandardName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RegulatoryComplianceStandardImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceManagerUtils.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceManagerUtils.java new file mode 100644 index 0000000000000..9f956c1dc42e1 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceManagerUtils.java @@ -0,0 +1,197 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.util.CoreUtils; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import reactor.core.publisher.Flux; + +final class ResourceManagerUtils { + private ResourceManagerUtils() { + } + + static String getValueFromIdByName(String id, String name) { + if (id == null) { + return null; + } + Iterator itr = Arrays.stream(id.split("/")).iterator(); + while (itr.hasNext()) { + String part = itr.next(); + if (part != null && !part.trim().isEmpty()) { + if (part.equalsIgnoreCase(name)) { + if (itr.hasNext()) { + return itr.next(); + } else { + return null; + } + } + } + } + return null; + + } + + static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { + if (id == null || pathTemplate == null) { + return null; + } + String parameterNameParentheses = "{" + parameterName + "}"; + List idSegmentsReverted = Arrays.asList(id.split("/")); + List pathSegments = Arrays.asList(pathTemplate.split("/")); + Collections.reverse(idSegmentsReverted); + Iterator idItrReverted = idSegmentsReverted.iterator(); + int pathIndex = pathSegments.size(); + while (idItrReverted.hasNext() && pathIndex > 0) { + String idSegment = idItrReverted.next(); + String pathSegment = pathSegments.get(--pathIndex); + if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { + if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { + if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { + List segments = new ArrayList<>(); + segments.add(idSegment); + idItrReverted.forEachRemaining(segments::add); + Collections.reverse(segments); + if (!segments.isEmpty() && segments.get(0).isEmpty()) { + segments.remove(0); + } + return String.join("/", segments); + } else { + return idSegment; + } + } + } + } + return null; + + } + + static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { + return new PagedIterableImpl<>(pageIterable, mapper); + } + + private static final class PagedIterableImpl extends PagedIterable { + + private final PagedIterable pagedIterable; + private final Function mapper; + private final Function, PagedResponse> pageMapper; + + private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { + super(PagedFlux.create(() -> (continuationToken, pageSize) -> Flux + .fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); + this.pagedIterable = pagedIterable; + this.mapper = mapper; + this.pageMapper = getPageMapper(mapper); + } + + private static Function, PagedResponse> getPageMapper(Function mapper) { + return page -> new PagedResponseBase(page.getRequest(), page.getStatusCode(), page.getHeaders(), + page.getElements().stream().map(mapper).collect(Collectors.toList()), page.getContinuationToken(), + null); + } + + @Override + public Stream stream() { + return pagedIterable.stream().map(mapper); + } + + @Override + public Stream> streamByPage() { + return pagedIterable.streamByPage().map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken) { + return pagedIterable.streamByPage(continuationToken).map(pageMapper); + } + + @Override + public Stream> streamByPage(int preferredPageSize) { + return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); + } + + @Override + public Stream> streamByPage(String continuationToken, int preferredPageSize) { + return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(pagedIterable.iterator(), mapper); + } + + @Override + public Iterable> iterableByPage() { + return new IterableImpl<>(pagedIterable.iterableByPage(), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken), pageMapper); + } + + @Override + public Iterable> iterableByPage(int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(preferredPageSize), pageMapper); + } + + @Override + public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { + return new IterableImpl<>(pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); + } + } + + private static final class IteratorImpl implements Iterator { + + private final Iterator iterator; + private final Function mapper; + + private IteratorImpl(Iterator iterator, Function mapper) { + this.iterator = iterator; + this.mapper = mapper; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public S next() { + return mapper.apply(iterator.next()); + } + + @Override + public void remove() { + iterator.remove(); + } + } + + private static final class IterableImpl implements Iterable { + + private final Iterable iterable; + private final Function mapper; + + private IterableImpl(Iterable iterable, Function mapper) { + this.iterable = iterable; + this.mapper = mapper; + } + + @Override + public Iterator iterator() { + return new IteratorImpl<>(iterable.iterator(), mapper); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersClientImpl.java new file mode 100644 index 0000000000000..cf7bc0ec09e20 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersClientImpl.java @@ -0,0 +1,270 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.ResourceProvidersClient; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner; +import com.azure.resourcemanager.security.models.UpdateSensitivitySettingsRequest; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in ResourceProvidersClient. + */ +public final class ResourceProvidersClientImpl implements ResourceProvidersClient { + /** + * The proxy service used to perform REST calls. + */ + private final ResourceProvidersService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of ResourceProvidersClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + ResourceProvidersClientImpl(SecurityCenterImpl client) { + this.service + = RestProxy.create(ResourceProvidersService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterResourceProviders to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterResour") + public interface ResourceProvidersService { + @Headers({ "Content-Type: application/json" }) + @Put("/providers/Microsoft.Security/sensitivitySettings/current") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> updateSensitivitySettings( + @HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @BodyParam("application/json") UpdateSensitivitySettingsRequest sensitivitySettings, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/providers/Microsoft.Security/sensitivitySettings/current") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> getSensitivitySettings(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Updates data sensitivity settings for sensitive data discovery. + * + * @param sensitivitySettings The data sensitivity settings to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + updateSensitivitySettingsWithResponseAsync(UpdateSensitivitySettingsRequest sensitivitySettings) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (sensitivitySettings == null) { + return Mono + .error(new IllegalArgumentException("Parameter sensitivitySettings is required and cannot be null.")); + } else { + sensitivitySettings.validate(); + } + final String apiVersion = "2023-02-15-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.updateSensitivitySettings(this.client.getEndpoint(), apiVersion, + sensitivitySettings, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Updates data sensitivity settings for sensitive data discovery. + * + * @param sensitivitySettings The data sensitivity settings to update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> updateSensitivitySettingsWithResponseAsync( + UpdateSensitivitySettingsRequest sensitivitySettings, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (sensitivitySettings == null) { + return Mono + .error(new IllegalArgumentException("Parameter sensitivitySettings is required and cannot be null.")); + } else { + sensitivitySettings.validate(); + } + final String apiVersion = "2023-02-15-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.updateSensitivitySettings(this.client.getEndpoint(), apiVersion, sensitivitySettings, accept, + context); + } + + /** + * Updates data sensitivity settings for sensitive data discovery. + * + * @param sensitivitySettings The data sensitivity settings to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono + updateSensitivitySettingsAsync(UpdateSensitivitySettingsRequest sensitivitySettings) { + return updateSensitivitySettingsWithResponseAsync(sensitivitySettings) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Updates data sensitivity settings for sensitive data discovery. + * + * @param sensitivitySettings The data sensitivity settings to update. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response + updateSensitivitySettingsWithResponse(UpdateSensitivitySettingsRequest sensitivitySettings, Context context) { + return updateSensitivitySettingsWithResponseAsync(sensitivitySettings, context).block(); + } + + /** + * Updates data sensitivity settings for sensitive data discovery. + * + * @param sensitivitySettings The data sensitivity settings to update. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GetSensitivitySettingsResponseInner + updateSensitivitySettings(UpdateSensitivitySettingsRequest sensitivitySettings) { + return updateSensitivitySettingsWithResponse(sensitivitySettings, Context.NONE).getValue(); + } + + /** + * Gets data sensitivity settings for sensitive data discovery. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> getSensitivitySettingsWithResponseAsync() { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String apiVersion = "2023-02-15-preview"; + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.getSensitivitySettings(this.client.getEndpoint(), apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets data sensitivity settings for sensitive data discovery. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery along with {@link Response} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + getSensitivitySettingsWithResponseAsync(Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String apiVersion = "2023-02-15-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.getSensitivitySettings(this.client.getEndpoint(), apiVersion, accept, context); + } + + /** + * Gets data sensitivity settings for sensitive data discovery. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono getSensitivitySettingsAsync() { + return getSensitivitySettingsWithResponseAsync().flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets data sensitivity settings for sensitive data discovery. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response getSensitivitySettingsWithResponse(Context context) { + return getSensitivitySettingsWithResponseAsync(context).block(); + } + + /** + * Gets data sensitivity settings for sensitive data discovery. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return data sensitivity settings for sensitive data discovery. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GetSensitivitySettingsResponseInner getSensitivitySettings() { + return getSensitivitySettingsWithResponse(Context.NONE).getValue(); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersImpl.java new file mode 100644 index 0000000000000..ed462c3f41d35 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ResourceProvidersImpl.java @@ -0,0 +1,79 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.ResourceProvidersClient; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsResponse; +import com.azure.resourcemanager.security.models.ResourceProviders; +import com.azure.resourcemanager.security.models.UpdateSensitivitySettingsRequest; + +public final class ResourceProvidersImpl implements ResourceProviders { + private static final ClientLogger LOGGER = new ClientLogger(ResourceProvidersImpl.class); + + private final ResourceProvidersClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public ResourceProvidersImpl(ResourceProvidersClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response + updateSensitivitySettingsWithResponse(UpdateSensitivitySettingsRequest sensitivitySettings, Context context) { + Response inner + = this.serviceClient().updateSensitivitySettingsWithResponse(sensitivitySettings, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GetSensitivitySettingsResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GetSensitivitySettingsResponse + updateSensitivitySettings(UpdateSensitivitySettingsRequest sensitivitySettings) { + GetSensitivitySettingsResponseInner inner = this.serviceClient().updateSensitivitySettings(sensitivitySettings); + if (inner != null) { + return new GetSensitivitySettingsResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response getSensitivitySettingsWithResponse(Context context) { + Response inner + = this.serviceClient().getSensitivitySettingsWithResponse(context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GetSensitivitySettingsResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GetSensitivitySettingsResponse getSensitivitySettings() { + GetSensitivitySettingsResponseInner inner = this.serviceClient().getSensitivitySettings(); + if (inner != null) { + return new GetSensitivitySettingsResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + private ResourceProvidersClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RuleResultsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RuleResultsImpl.java index 82aeae996a3c3..efb8a0eab819f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RuleResultsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RuleResultsImpl.java @@ -58,22 +58,14 @@ public RuleResultsImpl withExistingResourceId(String resourceId) { } public RuleResults create() { - this.innerObject = - serviceManager - .serviceClient() - .getSqlVulnerabilityAssessmentBaselineRules() - .createOrUpdateWithResponse(ruleId, createWorkspaceId, resourceId, createBody, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSqlVulnerabilityAssessmentBaselineRules() + .createOrUpdateWithResponse(ruleId, createWorkspaceId, resourceId, createBody, Context.NONE).getValue(); return this; } public RuleResults create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSqlVulnerabilityAssessmentBaselineRules() - .createOrUpdateWithResponse(ruleId, createWorkspaceId, resourceId, createBody, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSqlVulnerabilityAssessmentBaselineRules() + .createOrUpdateWithResponse(ruleId, createWorkspaceId, resourceId, createBody, context).getValue(); return this; } @@ -92,40 +84,26 @@ public RuleResultsImpl update() { } public RuleResults apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSqlVulnerabilityAssessmentBaselineRules() - .createOrUpdateWithResponse(ruleId, updateWorkspaceId, resourceId, updateBody, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSqlVulnerabilityAssessmentBaselineRules() + .createOrUpdateWithResponse(ruleId, updateWorkspaceId, resourceId, updateBody, Context.NONE).getValue(); return this; } public RuleResults apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSqlVulnerabilityAssessmentBaselineRules() - .createOrUpdateWithResponse(ruleId, updateWorkspaceId, resourceId, updateBody, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSqlVulnerabilityAssessmentBaselineRules() + .createOrUpdateWithResponse(ruleId, updateWorkspaceId, resourceId, updateBody, context).getValue(); return this; } RuleResultsImpl(RuleResultsInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.ruleId = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}", - "ruleId"); - this.resourceId = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}", - "resourceId"); + this.ruleId = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}", + "ruleId"); + this.resourceId = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}", + "resourceId"); } public RuleResultsImpl withLatestScan(Boolean latestScan) { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RulesResultsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RulesResultsImpl.java index d048424ac5cb6..6daf8a684865d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RulesResultsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/RulesResultsImpl.java @@ -25,12 +25,8 @@ public final class RulesResultsImpl implements RulesResults { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new RuleResultsImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new RuleResultsImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScanResultsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScanResultsImpl.java index 0380f7db5b053..b6497100bdfd0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScanResultsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScanResultsImpl.java @@ -25,12 +25,8 @@ public final class ScanResultsImpl implements ScanResults { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new ScanResultImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new ScanResultImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScansImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScansImpl.java index 0b79bb6484c41..8c61fe7739eac 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScansImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ScansImpl.java @@ -25,9 +25,8 @@ public final class ScansImpl implements Scans { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner.stream().map(inner1 -> new ScanImpl(inner1, this.manager())).collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new ScanImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionItemImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionItemImpl.java index 0e04649334453..95cbe7eda8480 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionItemImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionItemImpl.java @@ -16,8 +16,7 @@ public final class SecureScoreControlDefinitionItemImpl implements SecureScoreCo private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecureScoreControlDefinitionItemImpl( - SecureScoreControlDefinitionItemInner innerObject, + SecureScoreControlDefinitionItemImpl(SecureScoreControlDefinitionItemInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsClientImpl.java index 8ed2fc1ff10ff..162e49e2dd069 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsClientImpl.java @@ -30,26 +30,28 @@ import com.azure.resourcemanager.security.models.SecureScoreControlDefinitionList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecureScoreControlDefinitionsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecureScoreControlDefinitionsClient. + */ public final class SecureScoreControlDefinitionsClientImpl implements SecureScoreControlDefinitionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecureScoreControlDefinitionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecureScoreControlDefinitionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecureScoreControlDefinitionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SecureScoreControlDefinitionsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SecureScoreControlDefinitionsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,117 +62,87 @@ public final class SecureScoreControlDefinitionsClientImpl implements SecureScor @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecure") public interface SecureScoreControlDefinitionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/providers/Microsoft.Security/secureScoreControlDefinitions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/secureScoreControlDefinitions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listBySubscription( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> listBySubscription(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * List the available security controls, their assessments, and the max score. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List the available security controls, their assessments, and the max score. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security controls definition as paginated response with {@link PagedFlux}. @@ -182,7 +154,7 @@ private PagedFlux listAsync() { /** * List the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -191,13 +163,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List the available security controls, their assessments, and the max score. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security controls definition as paginated response with {@link PagedIterable}. @@ -209,7 +181,7 @@ public PagedIterable list() { /** * List the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -223,103 +195,78 @@ public PagedIterable list(Context context /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listBySubscription( - this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listBySubscription(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionSinglePageAsync( - Context context) { + private Mono> + listBySubscriptionSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .listBySubscription(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security controls definition as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySubscriptionAsync() { - return new PagedFlux<>( - () -> listBySubscriptionSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listBySubscriptionSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -328,14 +275,13 @@ private PagedFlux listBySubscriptionAsync */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySubscriptionAsync(Context context) { - return new PagedFlux<>( - () -> listBySubscriptionSinglePageAsync(context), + return new PagedFlux<>(() -> listBySubscriptionSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security controls definition as paginated response with {@link PagedIterable}. @@ -347,7 +293,7 @@ public PagedIterable listBySubscription() /** * For a specified subscription, list the available security controls, their assessments, and the max score. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -361,14 +307,15 @@ public PagedIterable listBySubscription(C /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -376,140 +323,104 @@ private Mono> listNextSingl return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security controls definition along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security controls definition along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySubscriptionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsImpl.java index 6e0346a7e5366..5423f04374f9d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDefinitionsImpl.java @@ -19,8 +19,7 @@ public final class SecureScoreControlDefinitionsImpl implements SecureScoreContr private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecureScoreControlDefinitionsImpl( - SecureScoreControlDefinitionsClient innerClient, + public SecureScoreControlDefinitionsImpl(SecureScoreControlDefinitionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,22 +27,26 @@ public SecureScoreControlDefinitionsImpl( public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); } public PagedIterable listBySubscription() { PagedIterable inner = this.serviceClient().listBySubscription(); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); } public PagedIterable listBySubscription(Context context) { PagedIterable inner = this.serviceClient().listBySubscription(context); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new SecureScoreControlDefinitionItemImpl(inner1, this.manager())); } private SecureScoreControlDefinitionsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDetailsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDetailsImpl.java index 98ebfdf2afbaa..c9717c67992a8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDetailsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlDetailsImpl.java @@ -14,8 +14,8 @@ public final class SecureScoreControlDetailsImpl implements SecureScoreControlDe private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecureScoreControlDetailsImpl( - SecureScoreControlDetailsInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecureScoreControlDetailsImpl(SecureScoreControlDetailsInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlScoreDetailsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlScoreDetailsImpl.java index d3a1b368b1f60..5c32b8c688903 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlScoreDetailsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlScoreDetailsImpl.java @@ -14,8 +14,7 @@ public final class SecureScoreControlScoreDetailsImpl implements SecureScoreCont private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecureScoreControlScoreDetailsImpl( - SecureScoreControlScoreDetailsInner innerObject, + SecureScoreControlScoreDetailsImpl(SecureScoreControlScoreDetailsInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsClientImpl.java index 198a22b386af5..d9047aab86251 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsClientImpl.java @@ -31,22 +31,28 @@ import com.azure.resourcemanager.security.models.SecureScoreControlList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecureScoreControlsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecureScoreControlsClient. + */ public final class SecureScoreControlsClientImpl implements SecureScoreControlsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecureScoreControlsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecureScoreControlsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecureScoreControlsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SecureScoreControlsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(SecureScoreControlsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -57,79 +63,61 @@ public final class SecureScoreControlsClientImpl implements SecureScoreControlsC @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecure") public interface SecureScoreControlsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/secureScores/{secureScoreName}/secureScoreControls") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/secureScores/{secureScoreName}/secureScoreControls") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listBySecureScore( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("secureScoreName") String secureScoreName, - @QueryParam("$expand") ExpandControlsEnum expand, - @HeaderParam("Accept") String accept, - Context context); + Mono> listBySecureScore(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("secureScoreName") String secureScoreName, @QueryParam("$expand") ExpandControlsEnum expand, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/secureScoreControls") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("$expand") ExpandControlsEnum expand, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("$expand") ExpandControlsEnum expand, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySecureScoreNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param expand OData expand. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all security controls for a specific initiative within a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySecureScoreSinglePageAsync( - String secureScoreName, ExpandControlsEnum expand) { + private Mono> listBySecureScoreSinglePageAsync(String secureScoreName, + ExpandControlsEnum expand) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (secureScoreName == null) { return Mono @@ -138,56 +126,36 @@ private Mono> listBySecureScoreSin final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listBySecureScore( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - secureScoreName, - expand, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listBySecureScore(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), secureScoreName, expand, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all security controls for a specific initiative within a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySecureScoreSinglePageAsync( - String secureScoreName, ExpandControlsEnum expand, Context context) { + private Mono> listBySecureScoreSinglePageAsync(String secureScoreName, + ExpandControlsEnum expand, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (secureScoreName == null) { return Mono @@ -197,95 +165,79 @@ private Mono> listBySecureScoreSin final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listBySecureScore( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - secureScoreName, - expand, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listBySecureScore(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), secureScoreName, + expand, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param expand OData expand. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedFlux}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listBySecureScoreAsync( - String secureScoreName, ExpandControlsEnum expand) { - return new PagedFlux<>( - () -> listBySecureScoreSinglePageAsync(secureScoreName, expand), + private PagedFlux listBySecureScoreAsync(String secureScoreName, + ExpandControlsEnum expand) { + return new PagedFlux<>(() -> listBySecureScoreSinglePageAsync(secureScoreName, expand), nextLink -> listBySecureScoreNextSinglePageAsync(nextLink)); } /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedFlux}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listBySecureScoreAsync(String secureScoreName) { final ExpandControlsEnum expand = null; - return new PagedFlux<>( - () -> listBySecureScoreSinglePageAsync(secureScoreName, expand), + return new PagedFlux<>(() -> listBySecureScoreSinglePageAsync(secureScoreName, expand), nextLink -> listBySecureScoreNextSinglePageAsync(nextLink)); } /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedFlux}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listBySecureScoreAsync( - String secureScoreName, ExpandControlsEnum expand, Context context) { - return new PagedFlux<>( - () -> listBySecureScoreSinglePageAsync(secureScoreName, expand, context), + private PagedFlux listBySecureScoreAsync(String secureScoreName, + ExpandControlsEnum expand, Context context) { + return new PagedFlux<>(() -> listBySecureScoreSinglePageAsync(secureScoreName, expand, context), nextLink -> listBySecureScoreNextSinglePageAsync(nextLink, context)); } /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedIterable}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listBySecureScore(String secureScoreName) { @@ -295,117 +247,87 @@ public PagedIterable listBySecureScore(String se /** * Get all security controls for a specific initiative within a scope. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls for a specific initiative within a scope as paginated response with {@link - * PagedIterable}. + * @return all security controls for a specific initiative within a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listBySecureScore( - String secureScoreName, ExpandControlsEnum expand, Context context) { + public PagedIterable listBySecureScore(String secureScoreName, + ExpandControlsEnum expand, Context context) { return new PagedIterable<>(listBySecureScoreAsync(secureScoreName, expand, context)); } /** * Get all security controls within a scope. - * + * * @param expand OData expand. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls within a scope along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return all security controls within a scope along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(ExpandControlsEnum expand) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - expand, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + expand, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get all security controls within a scope. - * + * * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return all security controls within a scope along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return all security controls within a scope along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - ExpandControlsEnum expand, Context context) { + private Mono> listSinglePageAsync(ExpandControlsEnum expand, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), expand, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get all security controls within a scope. - * + * * @param expand OData expand. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -419,7 +341,7 @@ private PagedFlux listAsync(ExpandControlsEnum e /** * Get all security controls within a scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all security controls within a scope as paginated response with {@link PagedFlux}. @@ -432,7 +354,7 @@ private PagedFlux listAsync() { /** * Get all security controls within a scope. - * + * * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -442,13 +364,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(ExpandControlsEnum expand, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(expand, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(expand, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get all security controls within a scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return all security controls within a scope as paginated response with {@link PagedIterable}. @@ -461,7 +383,7 @@ public PagedIterable list() { /** * Get all security controls within a scope. - * + * * @param expand OData expand. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -476,9 +398,10 @@ public PagedIterable list(ExpandControlsEnum exp /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -490,31 +413,23 @@ private Mono> listBySecureScoreNex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listBySecureScoreNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -522,37 +437,28 @@ private Mono> listBySecureScoreNex * @return list of security controls along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listBySecureScoreNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listBySecureScoreNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySecureScoreNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySecureScoreNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -564,31 +470,22 @@ private Mono> listNextSinglePageAs return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -596,29 +493,19 @@ private Mono> listNextSinglePageAs * @return list of security controls along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsImpl.java index ddce15948c300..8c10b8f58ec89 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreControlsImpl.java @@ -20,32 +20,32 @@ public final class SecureScoreControlsImpl implements SecureScoreControls { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecureScoreControlsImpl( - SecureScoreControlsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SecureScoreControlsImpl(SecureScoreControlsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listBySecureScore(String secureScoreName) { PagedIterable inner = this.serviceClient().listBySecureScore(secureScoreName); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); } - public PagedIterable listBySecureScore( - String secureScoreName, ExpandControlsEnum expand, Context context) { - PagedIterable inner = - this.serviceClient().listBySecureScore(secureScoreName, expand, context); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); + public PagedIterable listBySecureScore(String secureScoreName, ExpandControlsEnum expand, + Context context) { + PagedIterable inner + = this.serviceClient().listBySecureScore(secureScoreName, expand, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); } public PagedIterable list(ExpandControlsEnum expand, Context context) { PagedIterable inner = this.serviceClient().list(expand, context); - return Utils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecureScoreControlDetailsImpl(inner1, this.manager())); } private SecureScoreControlsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreItemImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreItemImpl.java index b7b5a90f0ae93..fc83a1473b342 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreItemImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoreItemImpl.java @@ -12,8 +12,8 @@ public final class SecureScoreItemImpl implements SecureScoreItem { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecureScoreItemImpl( - SecureScoreItemInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecureScoreItemImpl(SecureScoreItemInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresClientImpl.java index 91d2496b7feb0..668ebc1899f5a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresClientImpl.java @@ -30,69 +30,66 @@ import com.azure.resourcemanager.security.models.SecureScoresList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecureScoresClient. */ +/** + * An instance of this class provides access to all the operations defined in SecureScoresClient. + */ public final class SecureScoresClientImpl implements SecureScoresClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecureScoresService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecureScoresClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecureScoresClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SecureScoresService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SecureScoresService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterSecureScores to be used by the proxy service to perform - * REST calls. + * The interface defining all the services for SecurityCenterSecureScores to be used by the proxy service to + * perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecure") public interface SecureScoresService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/secureScores") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/secureScores/{secureScoreName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("secureScoreName") String secureScoreName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("secureScoreName") String secureScoreName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of secure scores along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -100,39 +97,26 @@ Mono> listNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -142,36 +126,24 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of secure scores as paginated response with {@link PagedFlux}. @@ -183,7 +155,7 @@ private PagedFlux listAsync() { /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -192,13 +164,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of secure scores as paginated response with {@link PagedIterable}. @@ -210,7 +182,7 @@ public PagedIterable list() { /** * List secure scores for all your Microsoft Defender for Cloud initiatives within your current scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -225,28 +197,24 @@ public PagedIterable list(Context context) { /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return secure score for a specific Microsoft Defender for Cloud initiative within your current scope along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String secureScoreName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (secureScoreName == null) { return Mono @@ -255,45 +223,33 @@ private Mono> getWithResponseAsync(String secureS final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - secureScoreName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + secureScoreName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return secure score for a specific Microsoft Defender for Cloud initiative within your current scope along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String secureScoreName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (secureScoreName == null) { return Mono @@ -302,27 +258,21 @@ private Mono> getWithResponseAsync(String secureS final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - secureScoreName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), secureScoreName, + accept, context); } /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return secure score for a specific Microsoft Defender for Cloud initiative within your current scope on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync(String secureScoreName) { @@ -332,15 +282,15 @@ private Mono getAsync(String secureScoreName) { /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return secure score for a specific Microsoft Defender for Cloud initiative within your current scope along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response getWithResponse(String secureScoreName, Context context) { @@ -350,9 +300,9 @@ public Response getWithResponse(String secureScoreName, Co /** * Get secure score for a specific Microsoft Defender for Cloud initiative within your current scope. For the ASC * Default initiative, use 'ascScore'. - * + * * @param secureScoreName The initiative name. For the ASC Default initiative, use 'ascScore' as in the sample - * request below. + * request below. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -365,9 +315,10 @@ public SecureScoreItemInner get(String secureScoreName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -379,31 +330,22 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -416,23 +358,13 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresImpl.java index 2f6dbe759c03c..4f0ab3083a89d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecureScoresImpl.java @@ -21,29 +21,26 @@ public final class SecureScoresImpl implements SecureScores { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecureScoresImpl( - SecureScoresClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SecureScoresImpl(SecureScoresClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecureScoreItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecureScoreItemImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SecureScoreItemImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecureScoreItemImpl(inner1, this.manager())); } public Response getWithResponse(String secureScoreName, Context context) { Response inner = this.serviceClient().getWithResponse(secureScoreName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecureScoreItemImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentMetadataResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentMetadataResponseImpl.java index 7455b39d5a297..7dfffc78186b5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentMetadataResponseImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentMetadataResponseImpl.java @@ -26,8 +26,7 @@ public final class SecurityAssessmentMetadataResponseImpl private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecurityAssessmentMetadataResponseImpl( - SecurityAssessmentMetadataResponseInner innerObject, + SecurityAssessmentMetadataResponseImpl(SecurityAssessmentMetadataResponseInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -140,54 +139,38 @@ private com.azure.resourcemanager.security.SecurityManager manager() { private String assessmentMetadataName; public SecurityAssessmentMetadataResponse create() { - this.innerObject = - serviceManager - .serviceClient() - .getAssessmentsMetadatas() - .createInSubscriptionWithResponse(assessmentMetadataName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessmentsMetadatas() + .createInSubscriptionWithResponse(assessmentMetadataName, this.innerModel(), Context.NONE).getValue(); return this; } public SecurityAssessmentMetadataResponse create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAssessmentsMetadatas() - .createInSubscriptionWithResponse(assessmentMetadataName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessmentsMetadatas() + .createInSubscriptionWithResponse(assessmentMetadataName, this.innerModel(), context).getValue(); return this; } - SecurityAssessmentMetadataResponseImpl( - String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecurityAssessmentMetadataResponseImpl(String name, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = new SecurityAssessmentMetadataResponseInner(); this.serviceManager = serviceManager; this.assessmentMetadataName = name; } public SecurityAssessmentMetadataResponse refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getAssessmentsMetadatas() - .getInSubscriptionWithResponse(assessmentMetadataName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessmentsMetadatas() + .getInSubscriptionWithResponse(assessmentMetadataName, Context.NONE).getValue(); return this; } public SecurityAssessmentMetadataResponse refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAssessmentsMetadatas() - .getInSubscriptionWithResponse(assessmentMetadataName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessmentsMetadatas() + .getInSubscriptionWithResponse(assessmentMetadataName, context).getValue(); return this; } - public SecurityAssessmentMetadataResponseImpl withPublishDates( - SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates) { + public SecurityAssessmentMetadataResponseImpl + withPublishDates(SecurityAssessmentMetadataPropertiesResponsePublishDates publishDates) { this.innerModel().withPublishDates(publishDates); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentResponseImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentResponseImpl.java index b589cb09daae9..4be437fd92c37 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentResponseImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityAssessmentResponseImpl.java @@ -91,22 +91,14 @@ public SecurityAssessmentResponseImpl withExistingResourceId(String resourceId) } public SecurityAssessmentResponse create() { - this.innerObject = - serviceManager - .serviceClient() - .getAssessments() - .createOrUpdateWithResponse(resourceId, assessmentName, createAssessment, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessments() + .createOrUpdateWithResponse(resourceId, assessmentName, createAssessment, Context.NONE).getValue(); return this; } public SecurityAssessmentResponse create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAssessments() - .createOrUpdateWithResponse(resourceId, assessmentName, createAssessment, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessments() + .createOrUpdateWithResponse(resourceId, assessmentName, createAssessment, context).getValue(); return this; } @@ -123,63 +115,38 @@ public SecurityAssessmentResponseImpl update() { } public SecurityAssessmentResponse apply() { - this.innerObject = - serviceManager - .serviceClient() - .getAssessments() - .createOrUpdateWithResponse(resourceId, assessmentName, updateAssessment, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessments() + .createOrUpdateWithResponse(resourceId, assessmentName, updateAssessment, Context.NONE).getValue(); return this; } public SecurityAssessmentResponse apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getAssessments() - .createOrUpdateWithResponse(resourceId, assessmentName, updateAssessment, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessments() + .createOrUpdateWithResponse(resourceId, assessmentName, updateAssessment, context).getValue(); return this; } - SecurityAssessmentResponseImpl( - SecurityAssessmentResponseInner innerObject, + SecurityAssessmentResponseImpl(SecurityAssessmentResponseInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceId = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", - "resourceId"); - this.assessmentName = - Utils - .getValueFromIdByParameterName( - innerObject.id(), - "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", - "assessmentName"); + this.resourceId = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "resourceId"); + this.assessmentName = ResourceManagerUtils.getValueFromIdByParameterName(innerObject.id(), + "/{resourceId}/providers/Microsoft.Security/assessments/{assessmentName}", "assessmentName"); } public SecurityAssessmentResponse refresh() { ExpandEnum localExpand = null; - this.innerObject = - serviceManager - .serviceClient() - .getAssessments() - .getWithResponse(resourceId, assessmentName, localExpand, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessments() + .getWithResponse(resourceId, assessmentName, localExpand, Context.NONE).getValue(); return this; } public SecurityAssessmentResponse refresh(Context context) { ExpandEnum localExpand = null; - this.innerObject = - serviceManager - .serviceClient() - .getAssessments() - .getWithResponse(resourceId, assessmentName, localExpand, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getAssessments() + .getWithResponse(resourceId, assessmentName, localExpand, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterBuilder.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterBuilder.java index ff42af6d88d65..ab8c0a8b8fde8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterBuilder.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterBuilder.java @@ -14,8 +14,10 @@ import com.azure.core.util.serializer.SerializerAdapter; import java.time.Duration; -/** A builder for creating a new instance of the SecurityCenterImpl type. */ -@ServiceClientBuilder(serviceClients = {SecurityCenterImpl.class}) +/** + * A builder for creating a new instance of the SecurityCenterImpl type. + */ +@ServiceClientBuilder(serviceClients = { SecurityCenterImpl.class }) public final class SecurityCenterBuilder { /* * Azure subscription ID @@ -24,7 +26,7 @@ public final class SecurityCenterBuilder { /** * Sets Azure subscription ID. - * + * * @param subscriptionId the subscriptionId value. * @return the SecurityCenterBuilder. */ @@ -40,7 +42,7 @@ public SecurityCenterBuilder subscriptionId(String subscriptionId) { /** * Sets server parameter. - * + * * @param endpoint the endpoint value. * @return the SecurityCenterBuilder. */ @@ -56,7 +58,7 @@ public SecurityCenterBuilder endpoint(String endpoint) { /** * Sets The environment to connect to. - * + * * @param environment the environment value. * @return the SecurityCenterBuilder. */ @@ -72,7 +74,7 @@ public SecurityCenterBuilder environment(AzureEnvironment environment) { /** * Sets The HTTP pipeline to send requests through. - * + * * @param pipeline the pipeline value. * @return the SecurityCenterBuilder. */ @@ -88,7 +90,7 @@ public SecurityCenterBuilder pipeline(HttpPipeline pipeline) { /** * Sets The default poll interval for long-running operation. - * + * * @param defaultPollInterval the defaultPollInterval value. * @return the SecurityCenterBuilder. */ @@ -104,7 +106,7 @@ public SecurityCenterBuilder defaultPollInterval(Duration defaultPollInterval) { /** * Sets The serializer to serialize an object into a string. - * + * * @param serializerAdapter the serializerAdapter value. * @return the SecurityCenterBuilder. */ @@ -115,30 +117,20 @@ public SecurityCenterBuilder serializerAdapter(SerializerAdapter serializerAdapt /** * Builds an instance of SecurityCenterImpl with the provided parameters. - * + * * @return an instance of SecurityCenterImpl. */ public SecurityCenterImpl buildClient() { String localEndpoint = (endpoint != null) ? endpoint : "https://management.azure.com"; AzureEnvironment localEnvironment = (environment != null) ? environment : AzureEnvironment.AZURE; - HttpPipeline localPipeline = - (pipeline != null) - ? pipeline - : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); - Duration localDefaultPollInterval = - (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); - SerializerAdapter localSerializerAdapter = - (serializerAdapter != null) - ? serializerAdapter - : SerializerFactory.createDefaultManagementSerializerAdapter(); - SecurityCenterImpl client = - new SecurityCenterImpl( - localPipeline, - localSerializerAdapter, - localDefaultPollInterval, - localEnvironment, - subscriptionId, - localEndpoint); + HttpPipeline localPipeline = (pipeline != null) ? pipeline + : new HttpPipelineBuilder().policies(new UserAgentPolicy(), new RetryPolicy()).build(); + Duration localDefaultPollInterval + = (defaultPollInterval != null) ? defaultPollInterval : Duration.ofSeconds(30); + SerializerAdapter localSerializerAdapter = (serializerAdapter != null) ? serializerAdapter + : SerializerFactory.createDefaultManagementSerializerAdapter(); + SecurityCenterImpl client = new SecurityCenterImpl(localPipeline, localSerializerAdapter, + localDefaultPollInterval, localEnvironment, this.subscriptionId, localEndpoint); return client; } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterImpl.java index 2f43f2f9a5532..a8fd13d165657 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityCenterImpl.java @@ -28,8 +28,6 @@ import com.azure.resourcemanager.security.fluent.AlertsClient; import com.azure.resourcemanager.security.fluent.AlertsSuppressionRulesClient; import com.azure.resourcemanager.security.fluent.AllowedConnectionsClient; -import com.azure.resourcemanager.security.fluent.ApiCollectionOffboardingsClient; -import com.azure.resourcemanager.security.fluent.ApiCollectionOnboardingsClient; import com.azure.resourcemanager.security.fluent.ApiCollectionsClient; import com.azure.resourcemanager.security.fluent.ApplicationOperationsClient; import com.azure.resourcemanager.security.fluent.ApplicationsClient; @@ -37,20 +35,29 @@ import com.azure.resourcemanager.security.fluent.AssessmentsMetadatasClient; import com.azure.resourcemanager.security.fluent.AutoProvisioningSettingsClient; import com.azure.resourcemanager.security.fluent.AutomationsClient; +import com.azure.resourcemanager.security.fluent.AzureDevOpsOrgsClient; +import com.azure.resourcemanager.security.fluent.AzureDevOpsProjectsClient; +import com.azure.resourcemanager.security.fluent.AzureDevOpsReposClient; import com.azure.resourcemanager.security.fluent.ComplianceResultsClient; import com.azure.resourcemanager.security.fluent.CompliancesClient; import com.azure.resourcemanager.security.fluent.ConnectorsClient; import com.azure.resourcemanager.security.fluent.CustomAssessmentAutomationsClient; import com.azure.resourcemanager.security.fluent.CustomEntityStoreAssignmentsClient; +import com.azure.resourcemanager.security.fluent.DefenderForStoragesClient; +import com.azure.resourcemanager.security.fluent.DevOpsConfigurationsClient; +import com.azure.resourcemanager.security.fluent.DevOpsOperationResultsClient; import com.azure.resourcemanager.security.fluent.DeviceSecurityGroupsClient; import com.azure.resourcemanager.security.fluent.DiscoveredSecuritySolutionsClient; import com.azure.resourcemanager.security.fluent.ExternalSecuritySolutionsClient; +import com.azure.resourcemanager.security.fluent.GitHubOwnersClient; +import com.azure.resourcemanager.security.fluent.GitHubReposClient; +import com.azure.resourcemanager.security.fluent.GitLabGroupsClient; +import com.azure.resourcemanager.security.fluent.GitLabProjectsClient; +import com.azure.resourcemanager.security.fluent.GitLabSubgroupsClient; import com.azure.resourcemanager.security.fluent.GovernanceAssignmentsClient; import com.azure.resourcemanager.security.fluent.GovernanceRulesClient; -import com.azure.resourcemanager.security.fluent.HealthReportOperationsClient; import com.azure.resourcemanager.security.fluent.HealthReportsClient; import com.azure.resourcemanager.security.fluent.InformationProtectionPoliciesClient; -import com.azure.resourcemanager.security.fluent.IngestionSettingsClient; import com.azure.resourcemanager.security.fluent.IotSecuritySolutionAnalyticsClient; import com.azure.resourcemanager.security.fluent.IotSecuritySolutionsAnalyticsAggregatedAlertsClient; import com.azure.resourcemanager.security.fluent.IotSecuritySolutionsAnalyticsRecommendationsClient; @@ -63,6 +70,7 @@ import com.azure.resourcemanager.security.fluent.RegulatoryComplianceAssessmentsClient; import com.azure.resourcemanager.security.fluent.RegulatoryComplianceControlsClient; import com.azure.resourcemanager.security.fluent.RegulatoryComplianceStandardsClient; +import com.azure.resourcemanager.security.fluent.ResourceProvidersClient; import com.azure.resourcemanager.security.fluent.SecureScoreControlDefinitionsClient; import com.azure.resourcemanager.security.fluent.SecureScoreControlsClient; import com.azure.resourcemanager.security.fluent.SecureScoresClient; @@ -74,7 +82,9 @@ import com.azure.resourcemanager.security.fluent.SecurityOperatorsClient; import com.azure.resourcemanager.security.fluent.SecuritySolutionsClient; import com.azure.resourcemanager.security.fluent.SecuritySolutionsReferenceDatasClient; +import com.azure.resourcemanager.security.fluent.SensitivitySettingsClient; import com.azure.resourcemanager.security.fluent.ServerVulnerabilityAssessmentsClient; +import com.azure.resourcemanager.security.fluent.ServerVulnerabilityAssessmentsSettingsClient; import com.azure.resourcemanager.security.fluent.SettingsClient; import com.azure.resourcemanager.security.fluent.SoftwareInventoriesClient; import com.azure.resourcemanager.security.fluent.SqlVulnerabilityAssessmentBaselineRulesClient; @@ -93,804 +103,1078 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** Initializes a new instance of the SecurityCenterImpl type. */ +/** + * Initializes a new instance of the SecurityCenterImpl type. + */ @ServiceClient(builder = SecurityCenterBuilder.class) public final class SecurityCenterImpl implements SecurityCenter { - /** Azure subscription ID. */ + /** + * Azure subscription ID. + */ private final String subscriptionId; /** * Gets Azure subscription ID. - * + * * @return the subscriptionId value. */ public String getSubscriptionId() { return this.subscriptionId; } - /** server parameter. */ + /** + * server parameter. + */ private final String endpoint; /** * Gets server parameter. - * + * * @return the endpoint value. */ public String getEndpoint() { return this.endpoint; } - /** The HTTP pipeline to send requests through. */ + /** + * The HTTP pipeline to send requests through. + */ private final HttpPipeline httpPipeline; /** * Gets The HTTP pipeline to send requests through. - * + * * @return the httpPipeline value. */ public HttpPipeline getHttpPipeline() { return this.httpPipeline; } - /** The serializer to serialize an object into a string. */ + /** + * The serializer to serialize an object into a string. + */ private final SerializerAdapter serializerAdapter; /** * Gets The serializer to serialize an object into a string. - * + * * @return the serializerAdapter value. */ SerializerAdapter getSerializerAdapter() { return this.serializerAdapter; } - /** The default poll interval for long-running operation. */ + /** + * The default poll interval for long-running operation. + */ private final Duration defaultPollInterval; /** * Gets The default poll interval for long-running operation. - * + * * @return the defaultPollInterval value. */ public Duration getDefaultPollInterval() { return this.defaultPollInterval; } - /** The MdeOnboardingsClient object to access its operations. */ - private final MdeOnboardingsClient mdeOnboardings; - - /** - * Gets the MdeOnboardingsClient object to access its operations. - * - * @return the MdeOnboardingsClient object. - */ - public MdeOnboardingsClient getMdeOnboardings() { - return this.mdeOnboardings; - } - - /** The CustomAssessmentAutomationsClient object to access its operations. */ - private final CustomAssessmentAutomationsClient customAssessmentAutomations; - - /** - * Gets the CustomAssessmentAutomationsClient object to access its operations. - * - * @return the CustomAssessmentAutomationsClient object. - */ - public CustomAssessmentAutomationsClient getCustomAssessmentAutomations() { - return this.customAssessmentAutomations; - } - - /** The CustomEntityStoreAssignmentsClient object to access its operations. */ - private final CustomEntityStoreAssignmentsClient customEntityStoreAssignments; - - /** - * Gets the CustomEntityStoreAssignmentsClient object to access its operations. - * - * @return the CustomEntityStoreAssignmentsClient object. - */ - public CustomEntityStoreAssignmentsClient getCustomEntityStoreAssignments() { - return this.customEntityStoreAssignments; - } - - /** The ComplianceResultsClient object to access its operations. */ - private final ComplianceResultsClient complianceResults; - - /** - * Gets the ComplianceResultsClient object to access its operations. - * - * @return the ComplianceResultsClient object. - */ - public ComplianceResultsClient getComplianceResults() { - return this.complianceResults; - } - - /** The PricingsClient object to access its operations. */ - private final PricingsClient pricings; - - /** - * Gets the PricingsClient object to access its operations. - * - * @return the PricingsClient object. - */ - public PricingsClient getPricings() { - return this.pricings; - } - - /** The AdvancedThreatProtectionsClient object to access its operations. */ - private final AdvancedThreatProtectionsClient advancedThreatProtections; - - /** - * Gets the AdvancedThreatProtectionsClient object to access its operations. - * - * @return the AdvancedThreatProtectionsClient object. - */ - public AdvancedThreatProtectionsClient getAdvancedThreatProtections() { - return this.advancedThreatProtections; - } - - /** The DeviceSecurityGroupsClient object to access its operations. */ - private final DeviceSecurityGroupsClient deviceSecurityGroups; - - /** - * Gets the DeviceSecurityGroupsClient object to access its operations. - * - * @return the DeviceSecurityGroupsClient object. - */ - public DeviceSecurityGroupsClient getDeviceSecurityGroups() { - return this.deviceSecurityGroups; - } - - /** The IotSecuritySolutionsClient object to access its operations. */ - private final IotSecuritySolutionsClient iotSecuritySolutions; - - /** - * Gets the IotSecuritySolutionsClient object to access its operations. - * - * @return the IotSecuritySolutionsClient object. - */ - public IotSecuritySolutionsClient getIotSecuritySolutions() { - return this.iotSecuritySolutions; - } - - /** The IotSecuritySolutionAnalyticsClient object to access its operations. */ - private final IotSecuritySolutionAnalyticsClient iotSecuritySolutionAnalytics; - - /** - * Gets the IotSecuritySolutionAnalyticsClient object to access its operations. - * - * @return the IotSecuritySolutionAnalyticsClient object. - */ - public IotSecuritySolutionAnalyticsClient getIotSecuritySolutionAnalytics() { - return this.iotSecuritySolutionAnalytics; - } - - /** The IotSecuritySolutionsAnalyticsAggregatedAlertsClient object to access its operations. */ - private final IotSecuritySolutionsAnalyticsAggregatedAlertsClient iotSecuritySolutionsAnalyticsAggregatedAlerts; - - /** - * Gets the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object to access its operations. - * - * @return the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object. - */ - public IotSecuritySolutionsAnalyticsAggregatedAlertsClient getIotSecuritySolutionsAnalyticsAggregatedAlerts() { - return this.iotSecuritySolutionsAnalyticsAggregatedAlerts; - } - - /** The IotSecuritySolutionsAnalyticsRecommendationsClient object to access its operations. */ - private final IotSecuritySolutionsAnalyticsRecommendationsClient iotSecuritySolutionsAnalyticsRecommendations; - /** - * Gets the IotSecuritySolutionsAnalyticsRecommendationsClient object to access its operations. - * - * @return the IotSecuritySolutionsAnalyticsRecommendationsClient object. + * The LocationsClient object to access its operations. */ - public IotSecuritySolutionsAnalyticsRecommendationsClient getIotSecuritySolutionsAnalyticsRecommendations() { - return this.iotSecuritySolutionsAnalyticsRecommendations; - } - - /** The LocationsClient object to access its operations. */ private final LocationsClient locations; /** * Gets the LocationsClient object to access its operations. - * + * * @return the LocationsClient object. */ public LocationsClient getLocations() { return this.locations; } - /** The OperationsClient object to access its operations. */ + /** + * The OperationsClient object to access its operations. + */ private final OperationsClient operations; /** * Gets the OperationsClient object to access its operations. - * + * * @return the OperationsClient object. */ public OperationsClient getOperations() { return this.operations; } - /** The TasksClient object to access its operations. */ + /** + * The TasksClient object to access its operations. + */ private final TasksClient tasks; /** * Gets the TasksClient object to access its operations. - * + * * @return the TasksClient object. */ public TasksClient getTasks() { return this.tasks; } - /** The AutoProvisioningSettingsClient object to access its operations. */ + /** + * The AutoProvisioningSettingsClient object to access its operations. + */ private final AutoProvisioningSettingsClient autoProvisioningSettings; /** * Gets the AutoProvisioningSettingsClient object to access its operations. - * + * * @return the AutoProvisioningSettingsClient object. */ public AutoProvisioningSettingsClient getAutoProvisioningSettings() { return this.autoProvisioningSettings; } - /** The CompliancesClient object to access its operations. */ + /** + * The CompliancesClient object to access its operations. + */ private final CompliancesClient compliances; /** * Gets the CompliancesClient object to access its operations. - * + * * @return the CompliancesClient object. */ public CompliancesClient getCompliances() { return this.compliances; } - /** The InformationProtectionPoliciesClient object to access its operations. */ + /** + * The InformationProtectionPoliciesClient object to access its operations. + */ private final InformationProtectionPoliciesClient informationProtectionPolicies; /** * Gets the InformationProtectionPoliciesClient object to access its operations. - * + * * @return the InformationProtectionPoliciesClient object. */ public InformationProtectionPoliciesClient getInformationProtectionPolicies() { return this.informationProtectionPolicies; } - /** The SecurityContactsClient object to access its operations. */ - private final SecurityContactsClient securityContacts; - /** - * Gets the SecurityContactsClient object to access its operations. - * - * @return the SecurityContactsClient object. + * The WorkspaceSettingsClient object to access its operations. */ - public SecurityContactsClient getSecurityContacts() { - return this.securityContacts; - } - - /** The WorkspaceSettingsClient object to access its operations. */ private final WorkspaceSettingsClient workspaceSettings; /** * Gets the WorkspaceSettingsClient object to access its operations. - * + * * @return the WorkspaceSettingsClient object. */ public WorkspaceSettingsClient getWorkspaceSettings() { return this.workspaceSettings; } - /** The RegulatoryComplianceStandardsClient object to access its operations. */ + /** + * The AlertsSuppressionRulesClient object to access its operations. + */ + private final AlertsSuppressionRulesClient alertsSuppressionRules; + + /** + * Gets the AlertsSuppressionRulesClient object to access its operations. + * + * @return the AlertsSuppressionRulesClient object. + */ + public AlertsSuppressionRulesClient getAlertsSuppressionRules() { + return this.alertsSuppressionRules; + } + + /** + * The AutomationsClient object to access its operations. + */ + private final AutomationsClient automations; + + /** + * Gets the AutomationsClient object to access its operations. + * + * @return the AutomationsClient object. + */ + public AutomationsClient getAutomations() { + return this.automations; + } + + /** + * The RegulatoryComplianceStandardsClient object to access its operations. + */ private final RegulatoryComplianceStandardsClient regulatoryComplianceStandards; /** * Gets the RegulatoryComplianceStandardsClient object to access its operations. - * + * * @return the RegulatoryComplianceStandardsClient object. */ public RegulatoryComplianceStandardsClient getRegulatoryComplianceStandards() { return this.regulatoryComplianceStandards; } - /** The RegulatoryComplianceControlsClient object to access its operations. */ + /** + * The RegulatoryComplianceControlsClient object to access its operations. + */ private final RegulatoryComplianceControlsClient regulatoryComplianceControls; /** * Gets the RegulatoryComplianceControlsClient object to access its operations. - * + * * @return the RegulatoryComplianceControlsClient object. */ public RegulatoryComplianceControlsClient getRegulatoryComplianceControls() { return this.regulatoryComplianceControls; } - /** The RegulatoryComplianceAssessmentsClient object to access its operations. */ + /** + * The RegulatoryComplianceAssessmentsClient object to access its operations. + */ private final RegulatoryComplianceAssessmentsClient regulatoryComplianceAssessments; /** * Gets the RegulatoryComplianceAssessmentsClient object to access its operations. - * + * * @return the RegulatoryComplianceAssessmentsClient object. */ public RegulatoryComplianceAssessmentsClient getRegulatoryComplianceAssessments() { return this.regulatoryComplianceAssessments; } - /** The SubAssessmentsClient object to access its operations. */ + /** + * The SubAssessmentsClient object to access its operations. + */ private final SubAssessmentsClient subAssessments; /** * Gets the SubAssessmentsClient object to access its operations. - * + * * @return the SubAssessmentsClient object. */ public SubAssessmentsClient getSubAssessments() { return this.subAssessments; } - /** The AutomationsClient object to access its operations. */ - private final AutomationsClient automations; + /** + * The ConnectorsClient object to access its operations. + */ + private final ConnectorsClient connectors; /** - * Gets the AutomationsClient object to access its operations. - * - * @return the AutomationsClient object. + * Gets the ConnectorsClient object to access its operations. + * + * @return the ConnectorsClient object. */ - public AutomationsClient getAutomations() { - return this.automations; + public ConnectorsClient getConnectors() { + return this.connectors; } - /** The AlertsSuppressionRulesClient object to access its operations. */ - private final AlertsSuppressionRulesClient alertsSuppressionRules; + /** + * The SecurityContactsClient object to access its operations. + */ + private final SecurityContactsClient securityContacts; /** - * Gets the AlertsSuppressionRulesClient object to access its operations. - * - * @return the AlertsSuppressionRulesClient object. + * Gets the SecurityContactsClient object to access its operations. + * + * @return the SecurityContactsClient object. */ - public AlertsSuppressionRulesClient getAlertsSuppressionRules() { - return this.alertsSuppressionRules; + public SecurityContactsClient getSecurityContacts() { + return this.securityContacts; } - /** The ServerVulnerabilityAssessmentsClient object to access its operations. */ - private final ServerVulnerabilityAssessmentsClient serverVulnerabilityAssessments; + /** + * The SoftwareInventoriesClient object to access its operations. + */ + private final SoftwareInventoriesClient softwareInventories; /** - * Gets the ServerVulnerabilityAssessmentsClient object to access its operations. - * - * @return the ServerVulnerabilityAssessmentsClient object. + * Gets the SoftwareInventoriesClient object to access its operations. + * + * @return the SoftwareInventoriesClient object. */ - public ServerVulnerabilityAssessmentsClient getServerVulnerabilityAssessments() { - return this.serverVulnerabilityAssessments; + public SoftwareInventoriesClient getSoftwareInventories() { + return this.softwareInventories; } - /** The AssessmentsMetadatasClient object to access its operations. */ - private final AssessmentsMetadatasClient assessmentsMetadatas; + /** + * The CustomAssessmentAutomationsClient object to access its operations. + */ + private final CustomAssessmentAutomationsClient customAssessmentAutomations; /** - * Gets the AssessmentsMetadatasClient object to access its operations. - * - * @return the AssessmentsMetadatasClient object. + * Gets the CustomAssessmentAutomationsClient object to access its operations. + * + * @return the CustomAssessmentAutomationsClient object. */ - public AssessmentsMetadatasClient getAssessmentsMetadatas() { - return this.assessmentsMetadatas; + public CustomAssessmentAutomationsClient getCustomAssessmentAutomations() { + return this.customAssessmentAutomations; } - /** The AssessmentsClient object to access its operations. */ - private final AssessmentsClient assessments; + /** + * The CustomEntityStoreAssignmentsClient object to access its operations. + */ + private final CustomEntityStoreAssignmentsClient customEntityStoreAssignments; /** - * Gets the AssessmentsClient object to access its operations. - * - * @return the AssessmentsClient object. + * Gets the CustomEntityStoreAssignmentsClient object to access its operations. + * + * @return the CustomEntityStoreAssignmentsClient object. */ - public AssessmentsClient getAssessments() { - return this.assessments; + public CustomEntityStoreAssignmentsClient getCustomEntityStoreAssignments() { + return this.customEntityStoreAssignments; } - /** The AdaptiveApplicationControlsClient object to access its operations. */ - private final AdaptiveApplicationControlsClient adaptiveApplicationControls; + /** + * The MdeOnboardingsClient object to access its operations. + */ + private final MdeOnboardingsClient mdeOnboardings; /** - * Gets the AdaptiveApplicationControlsClient object to access its operations. - * - * @return the AdaptiveApplicationControlsClient object. + * Gets the MdeOnboardingsClient object to access its operations. + * + * @return the MdeOnboardingsClient object. */ - public AdaptiveApplicationControlsClient getAdaptiveApplicationControls() { - return this.adaptiveApplicationControls; + public MdeOnboardingsClient getMdeOnboardings() { + return this.mdeOnboardings; } - /** The AdaptiveNetworkHardeningsClient object to access its operations. */ - private final AdaptiveNetworkHardeningsClient adaptiveNetworkHardenings; + /** + * The GovernanceAssignmentsClient object to access its operations. + */ + private final GovernanceAssignmentsClient governanceAssignments; /** - * Gets the AdaptiveNetworkHardeningsClient object to access its operations. - * - * @return the AdaptiveNetworkHardeningsClient object. + * Gets the GovernanceAssignmentsClient object to access its operations. + * + * @return the GovernanceAssignmentsClient object. */ - public AdaptiveNetworkHardeningsClient getAdaptiveNetworkHardenings() { - return this.adaptiveNetworkHardenings; + public GovernanceAssignmentsClient getGovernanceAssignments() { + return this.governanceAssignments; } - /** The AllowedConnectionsClient object to access its operations. */ - private final AllowedConnectionsClient allowedConnections; - /** - * Gets the AllowedConnectionsClient object to access its operations. - * - * @return the AllowedConnectionsClient object. + * The GovernanceRulesClient object to access its operations. */ - public AllowedConnectionsClient getAllowedConnections() { - return this.allowedConnections; + private final GovernanceRulesClient governanceRules; + + /** + * Gets the GovernanceRulesClient object to access its operations. + * + * @return the GovernanceRulesClient object. + */ + public GovernanceRulesClient getGovernanceRules() { + return this.governanceRules; } - /** The TopologiesClient object to access its operations. */ - private final TopologiesClient topologies; + /** + * The ApplicationsClient object to access its operations. + */ + private final ApplicationsClient applications; /** - * Gets the TopologiesClient object to access its operations. - * - * @return the TopologiesClient object. + * Gets the ApplicationsClient object to access its operations. + * + * @return the ApplicationsClient object. */ - public TopologiesClient getTopologies() { - return this.topologies; + public ApplicationsClient getApplications() { + return this.applications; } - /** The JitNetworkAccessPoliciesClient object to access its operations. */ - private final JitNetworkAccessPoliciesClient jitNetworkAccessPolicies; + /** + * The ApplicationOperationsClient object to access its operations. + */ + private final ApplicationOperationsClient applicationOperations; /** - * Gets the JitNetworkAccessPoliciesClient object to access its operations. - * - * @return the JitNetworkAccessPoliciesClient object. + * Gets the ApplicationOperationsClient object to access its operations. + * + * @return the ApplicationOperationsClient object. */ - public JitNetworkAccessPoliciesClient getJitNetworkAccessPolicies() { - return this.jitNetworkAccessPolicies; + public ApplicationOperationsClient getApplicationOperations() { + return this.applicationOperations; } - /** The DiscoveredSecuritySolutionsClient object to access its operations. */ - private final DiscoveredSecuritySolutionsClient discoveredSecuritySolutions; + /** + * The SecurityConnectorApplicationsClient object to access its operations. + */ + private final SecurityConnectorApplicationsClient securityConnectorApplications; /** - * Gets the DiscoveredSecuritySolutionsClient object to access its operations. - * - * @return the DiscoveredSecuritySolutionsClient object. + * Gets the SecurityConnectorApplicationsClient object to access its operations. + * + * @return the SecurityConnectorApplicationsClient object. */ - public DiscoveredSecuritySolutionsClient getDiscoveredSecuritySolutions() { - return this.discoveredSecuritySolutions; + public SecurityConnectorApplicationsClient getSecurityConnectorApplications() { + return this.securityConnectorApplications; } - /** The SecuritySolutionsReferenceDatasClient object to access its operations. */ - private final SecuritySolutionsReferenceDatasClient securitySolutionsReferenceDatas; + /** + * The SecurityConnectorApplicationOperationsClient object to access its operations. + */ + private final SecurityConnectorApplicationOperationsClient securityConnectorApplicationOperations; /** - * Gets the SecuritySolutionsReferenceDatasClient object to access its operations. - * - * @return the SecuritySolutionsReferenceDatasClient object. + * Gets the SecurityConnectorApplicationOperationsClient object to access its operations. + * + * @return the SecurityConnectorApplicationOperationsClient object. */ - public SecuritySolutionsReferenceDatasClient getSecuritySolutionsReferenceDatas() { - return this.securitySolutionsReferenceDatas; + public SecurityConnectorApplicationOperationsClient getSecurityConnectorApplicationOperations() { + return this.securityConnectorApplicationOperations; + } + + /** + * The DefenderForStoragesClient object to access its operations. + */ + private final DefenderForStoragesClient defenderForStorages; + + /** + * Gets the DefenderForStoragesClient object to access its operations. + * + * @return the DefenderForStoragesClient object. + */ + public DefenderForStoragesClient getDefenderForStorages() { + return this.defenderForStorages; + } + + /** + * The SecurityOperatorsClient object to access its operations. + */ + private final SecurityOperatorsClient securityOperators; + + /** + * Gets the SecurityOperatorsClient object to access its operations. + * + * @return the SecurityOperatorsClient object. + */ + public SecurityOperatorsClient getSecurityOperators() { + return this.securityOperators; + } + + /** + * The SqlVulnerabilityAssessmentBaselineRulesClient object to access its operations. + */ + private final SqlVulnerabilityAssessmentBaselineRulesClient sqlVulnerabilityAssessmentBaselineRules; + + /** + * Gets the SqlVulnerabilityAssessmentBaselineRulesClient object to access its operations. + * + * @return the SqlVulnerabilityAssessmentBaselineRulesClient object. + */ + public SqlVulnerabilityAssessmentBaselineRulesClient getSqlVulnerabilityAssessmentBaselineRules() { + return this.sqlVulnerabilityAssessmentBaselineRules; + } + + /** + * The SqlVulnerabilityAssessmentScansClient object to access its operations. + */ + private final SqlVulnerabilityAssessmentScansClient sqlVulnerabilityAssessmentScans; + + /** + * Gets the SqlVulnerabilityAssessmentScansClient object to access its operations. + * + * @return the SqlVulnerabilityAssessmentScansClient object. + */ + public SqlVulnerabilityAssessmentScansClient getSqlVulnerabilityAssessmentScans() { + return this.sqlVulnerabilityAssessmentScans; + } + + /** + * The SqlVulnerabilityAssessmentScanResultsClient object to access its operations. + */ + private final SqlVulnerabilityAssessmentScanResultsClient sqlVulnerabilityAssessmentScanResults; + + /** + * Gets the SqlVulnerabilityAssessmentScanResultsClient object to access its operations. + * + * @return the SqlVulnerabilityAssessmentScanResultsClient object. + */ + public SqlVulnerabilityAssessmentScanResultsClient getSqlVulnerabilityAssessmentScanResults() { + return this.sqlVulnerabilityAssessmentScanResults; + } + + /** + * The ResourceProvidersClient object to access its operations. + */ + private final ResourceProvidersClient resourceProviders; + + /** + * Gets the ResourceProvidersClient object to access its operations. + * + * @return the ResourceProvidersClient object. + */ + public ResourceProvidersClient getResourceProviders() { + return this.resourceProviders; + } + + /** + * The SensitivitySettingsClient object to access its operations. + */ + private final SensitivitySettingsClient sensitivitySettings; + + /** + * Gets the SensitivitySettingsClient object to access its operations. + * + * @return the SensitivitySettingsClient object. + */ + public SensitivitySettingsClient getSensitivitySettings() { + return this.sensitivitySettings; + } + + /** + * The HealthReportsClient object to access its operations. + */ + private final HealthReportsClient healthReports; + + /** + * Gets the HealthReportsClient object to access its operations. + * + * @return the HealthReportsClient object. + */ + public HealthReportsClient getHealthReports() { + return this.healthReports; + } + + /** + * The AzureDevOpsOrgsClient object to access its operations. + */ + private final AzureDevOpsOrgsClient azureDevOpsOrgs; + + /** + * Gets the AzureDevOpsOrgsClient object to access its operations. + * + * @return the AzureDevOpsOrgsClient object. + */ + public AzureDevOpsOrgsClient getAzureDevOpsOrgs() { + return this.azureDevOpsOrgs; + } + + /** + * The AzureDevOpsProjectsClient object to access its operations. + */ + private final AzureDevOpsProjectsClient azureDevOpsProjects; + + /** + * Gets the AzureDevOpsProjectsClient object to access its operations. + * + * @return the AzureDevOpsProjectsClient object. + */ + public AzureDevOpsProjectsClient getAzureDevOpsProjects() { + return this.azureDevOpsProjects; + } + + /** + * The AzureDevOpsReposClient object to access its operations. + */ + private final AzureDevOpsReposClient azureDevOpsRepos; + + /** + * Gets the AzureDevOpsReposClient object to access its operations. + * + * @return the AzureDevOpsReposClient object. + */ + public AzureDevOpsReposClient getAzureDevOpsRepos() { + return this.azureDevOpsRepos; + } + + /** + * The DevOpsConfigurationsClient object to access its operations. + */ + private final DevOpsConfigurationsClient devOpsConfigurations; + + /** + * Gets the DevOpsConfigurationsClient object to access its operations. + * + * @return the DevOpsConfigurationsClient object. + */ + public DevOpsConfigurationsClient getDevOpsConfigurations() { + return this.devOpsConfigurations; + } + + /** + * The GitHubOwnersClient object to access its operations. + */ + private final GitHubOwnersClient gitHubOwners; + + /** + * Gets the GitHubOwnersClient object to access its operations. + * + * @return the GitHubOwnersClient object. + */ + public GitHubOwnersClient getGitHubOwners() { + return this.gitHubOwners; + } + + /** + * The GitHubReposClient object to access its operations. + */ + private final GitHubReposClient gitHubRepos; + + /** + * Gets the GitHubReposClient object to access its operations. + * + * @return the GitHubReposClient object. + */ + public GitHubReposClient getGitHubRepos() { + return this.gitHubRepos; + } + + /** + * The GitLabGroupsClient object to access its operations. + */ + private final GitLabGroupsClient gitLabGroups; + + /** + * Gets the GitLabGroupsClient object to access its operations. + * + * @return the GitLabGroupsClient object. + */ + public GitLabGroupsClient getGitLabGroups() { + return this.gitLabGroups; + } + + /** + * The GitLabSubgroupsClient object to access its operations. + */ + private final GitLabSubgroupsClient gitLabSubgroups; + + /** + * Gets the GitLabSubgroupsClient object to access its operations. + * + * @return the GitLabSubgroupsClient object. + */ + public GitLabSubgroupsClient getGitLabSubgroups() { + return this.gitLabSubgroups; + } + + /** + * The GitLabProjectsClient object to access its operations. + */ + private final GitLabProjectsClient gitLabProjects; + + /** + * Gets the GitLabProjectsClient object to access its operations. + * + * @return the GitLabProjectsClient object. + */ + public GitLabProjectsClient getGitLabProjects() { + return this.gitLabProjects; + } + + /** + * The DevOpsOperationResultsClient object to access its operations. + */ + private final DevOpsOperationResultsClient devOpsOperationResults; + + /** + * Gets the DevOpsOperationResultsClient object to access its operations. + * + * @return the DevOpsOperationResultsClient object. + */ + public DevOpsOperationResultsClient getDevOpsOperationResults() { + return this.devOpsOperationResults; + } + + /** + * The SecurityConnectorsClient object to access its operations. + */ + private final SecurityConnectorsClient securityConnectors; + + /** + * Gets the SecurityConnectorsClient object to access its operations. + * + * @return the SecurityConnectorsClient object. + */ + public SecurityConnectorsClient getSecurityConnectors() { + return this.securityConnectors; + } + + /** + * The ComplianceResultsClient object to access its operations. + */ + private final ComplianceResultsClient complianceResults; + + /** + * Gets the ComplianceResultsClient object to access its operations. + * + * @return the ComplianceResultsClient object. + */ + public ComplianceResultsClient getComplianceResults() { + return this.complianceResults; + } + + /** + * The AdvancedThreatProtectionsClient object to access its operations. + */ + private final AdvancedThreatProtectionsClient advancedThreatProtections; + + /** + * Gets the AdvancedThreatProtectionsClient object to access its operations. + * + * @return the AdvancedThreatProtectionsClient object. + */ + public AdvancedThreatProtectionsClient getAdvancedThreatProtections() { + return this.advancedThreatProtections; + } + + /** + * The DeviceSecurityGroupsClient object to access its operations. + */ + private final DeviceSecurityGroupsClient deviceSecurityGroups; + + /** + * Gets the DeviceSecurityGroupsClient object to access its operations. + * + * @return the DeviceSecurityGroupsClient object. + */ + public DeviceSecurityGroupsClient getDeviceSecurityGroups() { + return this.deviceSecurityGroups; + } + + /** + * The IotSecuritySolutionAnalyticsClient object to access its operations. + */ + private final IotSecuritySolutionAnalyticsClient iotSecuritySolutionAnalytics; + + /** + * Gets the IotSecuritySolutionAnalyticsClient object to access its operations. + * + * @return the IotSecuritySolutionAnalyticsClient object. + */ + public IotSecuritySolutionAnalyticsClient getIotSecuritySolutionAnalytics() { + return this.iotSecuritySolutionAnalytics; + } + + /** + * The IotSecuritySolutionsAnalyticsAggregatedAlertsClient object to access its operations. + */ + private final IotSecuritySolutionsAnalyticsAggregatedAlertsClient iotSecuritySolutionsAnalyticsAggregatedAlerts; + + /** + * Gets the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object to access its operations. + * + * @return the IotSecuritySolutionsAnalyticsAggregatedAlertsClient object. + */ + public IotSecuritySolutionsAnalyticsAggregatedAlertsClient getIotSecuritySolutionsAnalyticsAggregatedAlerts() { + return this.iotSecuritySolutionsAnalyticsAggregatedAlerts; + } + + /** + * The IotSecuritySolutionsAnalyticsRecommendationsClient object to access its operations. + */ + private final IotSecuritySolutionsAnalyticsRecommendationsClient iotSecuritySolutionsAnalyticsRecommendations; + + /** + * Gets the IotSecuritySolutionsAnalyticsRecommendationsClient object to access its operations. + * + * @return the IotSecuritySolutionsAnalyticsRecommendationsClient object. + */ + public IotSecuritySolutionsAnalyticsRecommendationsClient getIotSecuritySolutionsAnalyticsRecommendations() { + return this.iotSecuritySolutionsAnalyticsRecommendations; + } + + /** + * The IotSecuritySolutionsClient object to access its operations. + */ + private final IotSecuritySolutionsClient iotSecuritySolutions; + + /** + * Gets the IotSecuritySolutionsClient object to access its operations. + * + * @return the IotSecuritySolutionsClient object. + */ + public IotSecuritySolutionsClient getIotSecuritySolutions() { + return this.iotSecuritySolutions; + } + + /** + * The AdaptiveNetworkHardeningsClient object to access its operations. + */ + private final AdaptiveNetworkHardeningsClient adaptiveNetworkHardenings; + + /** + * Gets the AdaptiveNetworkHardeningsClient object to access its operations. + * + * @return the AdaptiveNetworkHardeningsClient object. + */ + public AdaptiveNetworkHardeningsClient getAdaptiveNetworkHardenings() { + return this.adaptiveNetworkHardenings; + } + + /** + * The AllowedConnectionsClient object to access its operations. + */ + private final AllowedConnectionsClient allowedConnections; + + /** + * Gets the AllowedConnectionsClient object to access its operations. + * + * @return the AllowedConnectionsClient object. + */ + public AllowedConnectionsClient getAllowedConnections() { + return this.allowedConnections; + } + + /** + * The AdaptiveApplicationControlsClient object to access its operations. + */ + private final AdaptiveApplicationControlsClient adaptiveApplicationControls; + + /** + * Gets the AdaptiveApplicationControlsClient object to access its operations. + * + * @return the AdaptiveApplicationControlsClient object. + */ + public AdaptiveApplicationControlsClient getAdaptiveApplicationControls() { + return this.adaptiveApplicationControls; + } + + /** + * The DiscoveredSecuritySolutionsClient object to access its operations. + */ + private final DiscoveredSecuritySolutionsClient discoveredSecuritySolutions; + + /** + * Gets the DiscoveredSecuritySolutionsClient object to access its operations. + * + * @return the DiscoveredSecuritySolutionsClient object. + */ + public DiscoveredSecuritySolutionsClient getDiscoveredSecuritySolutions() { + return this.discoveredSecuritySolutions; } - /** The ExternalSecuritySolutionsClient object to access its operations. */ + /** + * The ExternalSecuritySolutionsClient object to access its operations. + */ private final ExternalSecuritySolutionsClient externalSecuritySolutions; /** * Gets the ExternalSecuritySolutionsClient object to access its operations. - * + * * @return the ExternalSecuritySolutionsClient object. */ public ExternalSecuritySolutionsClient getExternalSecuritySolutions() { return this.externalSecuritySolutions; } - /** The SecureScoresClient object to access its operations. */ + /** + * The JitNetworkAccessPoliciesClient object to access its operations. + */ + private final JitNetworkAccessPoliciesClient jitNetworkAccessPolicies; + + /** + * Gets the JitNetworkAccessPoliciesClient object to access its operations. + * + * @return the JitNetworkAccessPoliciesClient object. + */ + public JitNetworkAccessPoliciesClient getJitNetworkAccessPolicies() { + return this.jitNetworkAccessPolicies; + } + + /** + * The SecureScoresClient object to access its operations. + */ private final SecureScoresClient secureScores; /** * Gets the SecureScoresClient object to access its operations. - * + * * @return the SecureScoresClient object. */ public SecureScoresClient getSecureScores() { return this.secureScores; } - /** The SecureScoreControlsClient object to access its operations. */ + /** + * The SecureScoreControlsClient object to access its operations. + */ private final SecureScoreControlsClient secureScoreControls; /** * Gets the SecureScoreControlsClient object to access its operations. - * + * * @return the SecureScoreControlsClient object. */ public SecureScoreControlsClient getSecureScoreControls() { return this.secureScoreControls; } - /** The SecureScoreControlDefinitionsClient object to access its operations. */ + /** + * The SecureScoreControlDefinitionsClient object to access its operations. + */ private final SecureScoreControlDefinitionsClient secureScoreControlDefinitions; /** * Gets the SecureScoreControlDefinitionsClient object to access its operations. - * + * * @return the SecureScoreControlDefinitionsClient object. */ public SecureScoreControlDefinitionsClient getSecureScoreControlDefinitions() { return this.secureScoreControlDefinitions; } - /** The SecuritySolutionsClient object to access its operations. */ + /** + * The SecuritySolutionsClient object to access its operations. + */ private final SecuritySolutionsClient securitySolutions; /** * Gets the SecuritySolutionsClient object to access its operations. - * + * * @return the SecuritySolutionsClient object. */ public SecuritySolutionsClient getSecuritySolutions() { return this.securitySolutions; } - /** The ConnectorsClient object to access its operations. */ - private final ConnectorsClient connectors; - - /** - * Gets the ConnectorsClient object to access its operations. - * - * @return the ConnectorsClient object. - */ - public ConnectorsClient getConnectors() { - return this.connectors; - } - - /** The AlertsClient object to access its operations. */ - private final AlertsClient alerts; - /** - * Gets the AlertsClient object to access its operations. - * - * @return the AlertsClient object. + * The SecuritySolutionsReferenceDatasClient object to access its operations. */ - public AlertsClient getAlerts() { - return this.alerts; - } - - /** The SettingsClient object to access its operations. */ - private final SettingsClient settings; + private final SecuritySolutionsReferenceDatasClient securitySolutionsReferenceDatas; /** - * Gets the SettingsClient object to access its operations. - * - * @return the SettingsClient object. + * Gets the SecuritySolutionsReferenceDatasClient object to access its operations. + * + * @return the SecuritySolutionsReferenceDatasClient object. */ - public SettingsClient getSettings() { - return this.settings; + public SecuritySolutionsReferenceDatasClient getSecuritySolutionsReferenceDatas() { + return this.securitySolutionsReferenceDatas; } - /** The IngestionSettingsClient object to access its operations. */ - private final IngestionSettingsClient ingestionSettings; - /** - * Gets the IngestionSettingsClient object to access its operations. - * - * @return the IngestionSettingsClient object. + * The ServerVulnerabilityAssessmentsClient object to access its operations. */ - public IngestionSettingsClient getIngestionSettings() { - return this.ingestionSettings; - } - - /** The SoftwareInventoriesClient object to access its operations. */ - private final SoftwareInventoriesClient softwareInventories; + private final ServerVulnerabilityAssessmentsClient serverVulnerabilityAssessments; /** - * Gets the SoftwareInventoriesClient object to access its operations. - * - * @return the SoftwareInventoriesClient object. + * Gets the ServerVulnerabilityAssessmentsClient object to access its operations. + * + * @return the ServerVulnerabilityAssessmentsClient object. */ - public SoftwareInventoriesClient getSoftwareInventories() { - return this.softwareInventories; + public ServerVulnerabilityAssessmentsClient getServerVulnerabilityAssessments() { + return this.serverVulnerabilityAssessments; } - /** The GovernanceRulesClient object to access its operations. */ - private final GovernanceRulesClient governanceRules; - /** - * Gets the GovernanceRulesClient object to access its operations. - * - * @return the GovernanceRulesClient object. + * The TopologiesClient object to access its operations. */ - public GovernanceRulesClient getGovernanceRules() { - return this.governanceRules; - } - - /** The GovernanceAssignmentsClient object to access its operations. */ - private final GovernanceAssignmentsClient governanceAssignments; + private final TopologiesClient topologies; /** - * Gets the GovernanceAssignmentsClient object to access its operations. - * - * @return the GovernanceAssignmentsClient object. + * Gets the TopologiesClient object to access its operations. + * + * @return the TopologiesClient object. */ - public GovernanceAssignmentsClient getGovernanceAssignments() { - return this.governanceAssignments; + public TopologiesClient getTopologies() { + return this.topologies; } - /** The ApplicationsClient object to access its operations. */ - private final ApplicationsClient applications; - /** - * Gets the ApplicationsClient object to access its operations. - * - * @return the ApplicationsClient object. + * The AssessmentsMetadatasClient object to access its operations. */ - public ApplicationsClient getApplications() { - return this.applications; - } - - /** The ApplicationOperationsClient object to access its operations. */ - private final ApplicationOperationsClient applicationOperations; + private final AssessmentsMetadatasClient assessmentsMetadatas; /** - * Gets the ApplicationOperationsClient object to access its operations. - * - * @return the ApplicationOperationsClient object. + * Gets the AssessmentsMetadatasClient object to access its operations. + * + * @return the AssessmentsMetadatasClient object. */ - public ApplicationOperationsClient getApplicationOperations() { - return this.applicationOperations; + public AssessmentsMetadatasClient getAssessmentsMetadatas() { + return this.assessmentsMetadatas; } - /** The SecurityConnectorApplicationsClient object to access its operations. */ - private final SecurityConnectorApplicationsClient securityConnectorApplications; - /** - * Gets the SecurityConnectorApplicationsClient object to access its operations. - * - * @return the SecurityConnectorApplicationsClient object. + * The AssessmentsClient object to access its operations. */ - public SecurityConnectorApplicationsClient getSecurityConnectorApplications() { - return this.securityConnectorApplications; - } - - /** The SecurityConnectorApplicationOperationsClient object to access its operations. */ - private final SecurityConnectorApplicationOperationsClient securityConnectorApplicationOperations; + private final AssessmentsClient assessments; /** - * Gets the SecurityConnectorApplicationOperationsClient object to access its operations. - * - * @return the SecurityConnectorApplicationOperationsClient object. + * Gets the AssessmentsClient object to access its operations. + * + * @return the AssessmentsClient object. */ - public SecurityConnectorApplicationOperationsClient getSecurityConnectorApplicationOperations() { - return this.securityConnectorApplicationOperations; + public AssessmentsClient getAssessments() { + return this.assessments; } - /** The ApiCollectionsClient object to access its operations. */ - private final ApiCollectionsClient apiCollections; - /** - * Gets the ApiCollectionsClient object to access its operations. - * - * @return the ApiCollectionsClient object. + * The AlertsClient object to access its operations. */ - public ApiCollectionsClient getApiCollections() { - return this.apiCollections; - } - - /** The ApiCollectionOnboardingsClient object to access its operations. */ - private final ApiCollectionOnboardingsClient apiCollectionOnboardings; + private final AlertsClient alerts; /** - * Gets the ApiCollectionOnboardingsClient object to access its operations. - * - * @return the ApiCollectionOnboardingsClient object. + * Gets the AlertsClient object to access its operations. + * + * @return the AlertsClient object. */ - public ApiCollectionOnboardingsClient getApiCollectionOnboardings() { - return this.apiCollectionOnboardings; + public AlertsClient getAlerts() { + return this.alerts; } - /** The ApiCollectionOffboardingsClient object to access its operations. */ - private final ApiCollectionOffboardingsClient apiCollectionOffboardings; - /** - * Gets the ApiCollectionOffboardingsClient object to access its operations. - * - * @return the ApiCollectionOffboardingsClient object. + * The SettingsClient object to access its operations. */ - public ApiCollectionOffboardingsClient getApiCollectionOffboardings() { - return this.apiCollectionOffboardings; - } - - /** The HealthReportsClient object to access its operations. */ - private final HealthReportsClient healthReports; + private final SettingsClient settings; /** - * Gets the HealthReportsClient object to access its operations. - * - * @return the HealthReportsClient object. + * Gets the SettingsClient object to access its operations. + * + * @return the SettingsClient object. */ - public HealthReportsClient getHealthReports() { - return this.healthReports; + public SettingsClient getSettings() { + return this.settings; } - /** The HealthReportOperationsClient object to access its operations. */ - private final HealthReportOperationsClient healthReportOperations; - /** - * Gets the HealthReportOperationsClient object to access its operations. - * - * @return the HealthReportOperationsClient object. + * The ServerVulnerabilityAssessmentsSettingsClient object to access its operations. */ - public HealthReportOperationsClient getHealthReportOperations() { - return this.healthReportOperations; - } - - /** The SqlVulnerabilityAssessmentScansClient object to access its operations. */ - private final SqlVulnerabilityAssessmentScansClient sqlVulnerabilityAssessmentScans; + private final ServerVulnerabilityAssessmentsSettingsClient serverVulnerabilityAssessmentsSettings; /** - * Gets the SqlVulnerabilityAssessmentScansClient object to access its operations. - * - * @return the SqlVulnerabilityAssessmentScansClient object. + * Gets the ServerVulnerabilityAssessmentsSettingsClient object to access its operations. + * + * @return the ServerVulnerabilityAssessmentsSettingsClient object. */ - public SqlVulnerabilityAssessmentScansClient getSqlVulnerabilityAssessmentScans() { - return this.sqlVulnerabilityAssessmentScans; + public ServerVulnerabilityAssessmentsSettingsClient getServerVulnerabilityAssessmentsSettings() { + return this.serverVulnerabilityAssessmentsSettings; } - /** The SqlVulnerabilityAssessmentScanResultsClient object to access its operations. */ - private final SqlVulnerabilityAssessmentScanResultsClient sqlVulnerabilityAssessmentScanResults; - /** - * Gets the SqlVulnerabilityAssessmentScanResultsClient object to access its operations. - * - * @return the SqlVulnerabilityAssessmentScanResultsClient object. + * The ApiCollectionsClient object to access its operations. */ - public SqlVulnerabilityAssessmentScanResultsClient getSqlVulnerabilityAssessmentScanResults() { - return this.sqlVulnerabilityAssessmentScanResults; - } - - /** The SqlVulnerabilityAssessmentBaselineRulesClient object to access its operations. */ - private final SqlVulnerabilityAssessmentBaselineRulesClient sqlVulnerabilityAssessmentBaselineRules; + private final ApiCollectionsClient apiCollections; /** - * Gets the SqlVulnerabilityAssessmentBaselineRulesClient object to access its operations. - * - * @return the SqlVulnerabilityAssessmentBaselineRulesClient object. + * Gets the ApiCollectionsClient object to access its operations. + * + * @return the ApiCollectionsClient object. */ - public SqlVulnerabilityAssessmentBaselineRulesClient getSqlVulnerabilityAssessmentBaselineRules() { - return this.sqlVulnerabilityAssessmentBaselineRules; + public ApiCollectionsClient getApiCollections() { + return this.apiCollections; } - /** The SecurityConnectorsClient object to access its operations. */ - private final SecurityConnectorsClient securityConnectors; - /** - * Gets the SecurityConnectorsClient object to access its operations. - * - * @return the SecurityConnectorsClient object. + * The PricingsClient object to access its operations. */ - public SecurityConnectorsClient getSecurityConnectors() { - return this.securityConnectors; - } - - /** The SecurityOperatorsClient object to access its operations. */ - private final SecurityOperatorsClient securityOperators; + private final PricingsClient pricings; /** - * Gets the SecurityOperatorsClient object to access its operations. - * - * @return the SecurityOperatorsClient object. + * Gets the PricingsClient object to access its operations. + * + * @return the PricingsClient object. */ - public SecurityOperatorsClient getSecurityOperators() { - return this.securityOperators; + public PricingsClient getPricings() { + return this.pricings; } /** * Initializes an instance of SecurityCenter client. - * + * * @param httpPipeline The HTTP pipeline to send requests through. * @param serializerAdapter The serializer to serialize an object into a string. * @param defaultPollInterval The default poll interval for long-running operation. @@ -898,86 +1182,91 @@ public SecurityOperatorsClient getSecurityOperators() { * @param subscriptionId Azure subscription ID. * @param endpoint server parameter. */ - SecurityCenterImpl( - HttpPipeline httpPipeline, - SerializerAdapter serializerAdapter, - Duration defaultPollInterval, - AzureEnvironment environment, - String subscriptionId, - String endpoint) { + SecurityCenterImpl(HttpPipeline httpPipeline, SerializerAdapter serializerAdapter, Duration defaultPollInterval, + AzureEnvironment environment, String subscriptionId, String endpoint) { this.httpPipeline = httpPipeline; this.serializerAdapter = serializerAdapter; this.defaultPollInterval = defaultPollInterval; this.subscriptionId = subscriptionId; this.endpoint = endpoint; - this.mdeOnboardings = new MdeOnboardingsClientImpl(this); - this.customAssessmentAutomations = new CustomAssessmentAutomationsClientImpl(this); - this.customEntityStoreAssignments = new CustomEntityStoreAssignmentsClientImpl(this); - this.complianceResults = new ComplianceResultsClientImpl(this); - this.pricings = new PricingsClientImpl(this); - this.advancedThreatProtections = new AdvancedThreatProtectionsClientImpl(this); - this.deviceSecurityGroups = new DeviceSecurityGroupsClientImpl(this); - this.iotSecuritySolutions = new IotSecuritySolutionsClientImpl(this); - this.iotSecuritySolutionAnalytics = new IotSecuritySolutionAnalyticsClientImpl(this); - this.iotSecuritySolutionsAnalyticsAggregatedAlerts = - new IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl(this); - this.iotSecuritySolutionsAnalyticsRecommendations = - new IotSecuritySolutionsAnalyticsRecommendationsClientImpl(this); this.locations = new LocationsClientImpl(this); this.operations = new OperationsClientImpl(this); this.tasks = new TasksClientImpl(this); this.autoProvisioningSettings = new AutoProvisioningSettingsClientImpl(this); this.compliances = new CompliancesClientImpl(this); this.informationProtectionPolicies = new InformationProtectionPoliciesClientImpl(this); - this.securityContacts = new SecurityContactsClientImpl(this); this.workspaceSettings = new WorkspaceSettingsClientImpl(this); + this.alertsSuppressionRules = new AlertsSuppressionRulesClientImpl(this); + this.automations = new AutomationsClientImpl(this); this.regulatoryComplianceStandards = new RegulatoryComplianceStandardsClientImpl(this); this.regulatoryComplianceControls = new RegulatoryComplianceControlsClientImpl(this); this.regulatoryComplianceAssessments = new RegulatoryComplianceAssessmentsClientImpl(this); this.subAssessments = new SubAssessmentsClientImpl(this); - this.automations = new AutomationsClientImpl(this); - this.alertsSuppressionRules = new AlertsSuppressionRulesClientImpl(this); - this.serverVulnerabilityAssessments = new ServerVulnerabilityAssessmentsClientImpl(this); - this.assessmentsMetadatas = new AssessmentsMetadatasClientImpl(this); - this.assessments = new AssessmentsClientImpl(this); - this.adaptiveApplicationControls = new AdaptiveApplicationControlsClientImpl(this); + this.connectors = new ConnectorsClientImpl(this); + this.securityContacts = new SecurityContactsClientImpl(this); + this.softwareInventories = new SoftwareInventoriesClientImpl(this); + this.customAssessmentAutomations = new CustomAssessmentAutomationsClientImpl(this); + this.customEntityStoreAssignments = new CustomEntityStoreAssignmentsClientImpl(this); + this.mdeOnboardings = new MdeOnboardingsClientImpl(this); + this.governanceAssignments = new GovernanceAssignmentsClientImpl(this); + this.governanceRules = new GovernanceRulesClientImpl(this); + this.applications = new ApplicationsClientImpl(this); + this.applicationOperations = new ApplicationOperationsClientImpl(this); + this.securityConnectorApplications = new SecurityConnectorApplicationsClientImpl(this); + this.securityConnectorApplicationOperations = new SecurityConnectorApplicationOperationsClientImpl(this); + this.defenderForStorages = new DefenderForStoragesClientImpl(this); + this.securityOperators = new SecurityOperatorsClientImpl(this); + this.sqlVulnerabilityAssessmentBaselineRules = new SqlVulnerabilityAssessmentBaselineRulesClientImpl(this); + this.sqlVulnerabilityAssessmentScans = new SqlVulnerabilityAssessmentScansClientImpl(this); + this.sqlVulnerabilityAssessmentScanResults = new SqlVulnerabilityAssessmentScanResultsClientImpl(this); + this.resourceProviders = new ResourceProvidersClientImpl(this); + this.sensitivitySettings = new SensitivitySettingsClientImpl(this); + this.healthReports = new HealthReportsClientImpl(this); + this.azureDevOpsOrgs = new AzureDevOpsOrgsClientImpl(this); + this.azureDevOpsProjects = new AzureDevOpsProjectsClientImpl(this); + this.azureDevOpsRepos = new AzureDevOpsReposClientImpl(this); + this.devOpsConfigurations = new DevOpsConfigurationsClientImpl(this); + this.gitHubOwners = new GitHubOwnersClientImpl(this); + this.gitHubRepos = new GitHubReposClientImpl(this); + this.gitLabGroups = new GitLabGroupsClientImpl(this); + this.gitLabSubgroups = new GitLabSubgroupsClientImpl(this); + this.gitLabProjects = new GitLabProjectsClientImpl(this); + this.devOpsOperationResults = new DevOpsOperationResultsClientImpl(this); + this.securityConnectors = new SecurityConnectorsClientImpl(this); + this.complianceResults = new ComplianceResultsClientImpl(this); + this.advancedThreatProtections = new AdvancedThreatProtectionsClientImpl(this); + this.deviceSecurityGroups = new DeviceSecurityGroupsClientImpl(this); + this.iotSecuritySolutionAnalytics = new IotSecuritySolutionAnalyticsClientImpl(this); + this.iotSecuritySolutionsAnalyticsAggregatedAlerts + = new IotSecuritySolutionsAnalyticsAggregatedAlertsClientImpl(this); + this.iotSecuritySolutionsAnalyticsRecommendations + = new IotSecuritySolutionsAnalyticsRecommendationsClientImpl(this); + this.iotSecuritySolutions = new IotSecuritySolutionsClientImpl(this); this.adaptiveNetworkHardenings = new AdaptiveNetworkHardeningsClientImpl(this); this.allowedConnections = new AllowedConnectionsClientImpl(this); - this.topologies = new TopologiesClientImpl(this); - this.jitNetworkAccessPolicies = new JitNetworkAccessPoliciesClientImpl(this); + this.adaptiveApplicationControls = new AdaptiveApplicationControlsClientImpl(this); this.discoveredSecuritySolutions = new DiscoveredSecuritySolutionsClientImpl(this); - this.securitySolutionsReferenceDatas = new SecuritySolutionsReferenceDatasClientImpl(this); this.externalSecuritySolutions = new ExternalSecuritySolutionsClientImpl(this); + this.jitNetworkAccessPolicies = new JitNetworkAccessPoliciesClientImpl(this); this.secureScores = new SecureScoresClientImpl(this); this.secureScoreControls = new SecureScoreControlsClientImpl(this); this.secureScoreControlDefinitions = new SecureScoreControlDefinitionsClientImpl(this); this.securitySolutions = new SecuritySolutionsClientImpl(this); - this.connectors = new ConnectorsClientImpl(this); + this.securitySolutionsReferenceDatas = new SecuritySolutionsReferenceDatasClientImpl(this); + this.serverVulnerabilityAssessments = new ServerVulnerabilityAssessmentsClientImpl(this); + this.topologies = new TopologiesClientImpl(this); + this.assessmentsMetadatas = new AssessmentsMetadatasClientImpl(this); + this.assessments = new AssessmentsClientImpl(this); this.alerts = new AlertsClientImpl(this); this.settings = new SettingsClientImpl(this); - this.ingestionSettings = new IngestionSettingsClientImpl(this); - this.softwareInventories = new SoftwareInventoriesClientImpl(this); - this.governanceRules = new GovernanceRulesClientImpl(this); - this.governanceAssignments = new GovernanceAssignmentsClientImpl(this); - this.applications = new ApplicationsClientImpl(this); - this.applicationOperations = new ApplicationOperationsClientImpl(this); - this.securityConnectorApplications = new SecurityConnectorApplicationsClientImpl(this); - this.securityConnectorApplicationOperations = new SecurityConnectorApplicationOperationsClientImpl(this); + this.serverVulnerabilityAssessmentsSettings = new ServerVulnerabilityAssessmentsSettingsClientImpl(this); this.apiCollections = new ApiCollectionsClientImpl(this); - this.apiCollectionOnboardings = new ApiCollectionOnboardingsClientImpl(this); - this.apiCollectionOffboardings = new ApiCollectionOffboardingsClientImpl(this); - this.healthReports = new HealthReportsClientImpl(this); - this.healthReportOperations = new HealthReportOperationsClientImpl(this); - this.sqlVulnerabilityAssessmentScans = new SqlVulnerabilityAssessmentScansClientImpl(this); - this.sqlVulnerabilityAssessmentScanResults = new SqlVulnerabilityAssessmentScanResultsClientImpl(this); - this.sqlVulnerabilityAssessmentBaselineRules = new SqlVulnerabilityAssessmentBaselineRulesClientImpl(this); - this.securityConnectors = new SecurityConnectorsClientImpl(this); - this.securityOperators = new SecurityOperatorsClientImpl(this); + this.pricings = new PricingsClientImpl(this); } /** * Gets default client context. - * + * * @return the default client context. */ public Context getContext() { @@ -986,7 +1275,7 @@ public Context getContext() { /** * Merges default client context with provided context. - * + * * @param context the context to be merged with default client context. * @return the merged context. */ @@ -996,7 +1285,7 @@ public Context mergeContext(Context context) { /** * Gets long running operation result. - * + * * @param activationResponse the response of activation operation. * @param httpPipeline the http pipeline. * @param pollResultType type of poll result. @@ -1006,26 +1295,15 @@ public Context mergeContext(Context context) { * @param type of final result. * @return poller flux for poll result and final result. */ - public PollerFlux, U> getLroResult( - Mono>> activationResponse, - HttpPipeline httpPipeline, - Type pollResultType, - Type finalResultType, - Context context) { - return PollerFactory - .create( - serializerAdapter, - httpPipeline, - pollResultType, - finalResultType, - defaultPollInterval, - activationResponse, - context); + public PollerFlux, U> getLroResult(Mono>> activationResponse, + HttpPipeline httpPipeline, Type pollResultType, Type finalResultType, Context context) { + return PollerFactory.create(serializerAdapter, httpPipeline, pollResultType, finalResultType, + defaultPollInterval, activationResponse, context); } /** * Gets the final result, or an error, based on last async poll response. - * + * * @param response the last async poll response. * @param type of poll result. * @param type of final result. @@ -1038,19 +1316,16 @@ public Mono getLroFinalResultOrError(AsyncPollResponse, HttpResponse errorResponse = null; PollResult.Error lroError = response.getValue().getError(); if (lroError != null) { - errorResponse = - new HttpResponseImpl( - lroError.getResponseStatusCode(), lroError.getResponseHeaders(), lroError.getResponseBody()); + errorResponse = new HttpResponseImpl(lroError.getResponseStatusCode(), lroError.getResponseHeaders(), + lroError.getResponseBody()); errorMessage = response.getValue().getError().getMessage(); String errorBody = response.getValue().getError().getResponseBody(); if (errorBody != null) { // try to deserialize error body to ManagementError try { - managementError = - this - .getSerializerAdapter() - .deserialize(errorBody, ManagementError.class, SerializerEncoding.JSON); + managementError = this.getSerializerAdapter().deserialize(errorBody, ManagementError.class, + SerializerEncoding.JSON); if (managementError.getCode() == null || managementError.getMessage() == null) { managementError = null; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsClientImpl.java index 8cf5297de0747..1cb19d0395135 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsClientImpl.java @@ -34,24 +34,24 @@ */ public final class SecurityConnectorApplicationOperationsClientImpl implements SecurityConnectorApplicationOperationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecurityConnectorApplicationOperationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecurityConnectorApplicationOperationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecurityConnectorApplicationOperationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SecurityConnectorApplicationOperationsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SecurityConnectorApplicationOperationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -62,79 +62,62 @@ public final class SecurityConnectorApplicationOperationsClientImpl @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecurityConnectorApplicationOperationsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications/{applicationId}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications/{applicationId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("securityConnectorName") String securityConnectorName, - @PathParam("applicationId") String applicationId, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("applicationId") String applicationId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications/{applicationId}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications/{applicationId}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("securityConnectorName") String securityConnectorName, @PathParam("applicationId") String applicationId, - @BodyParam("application/json") ApplicationInner application, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") ApplicationInner application, @HeaderParam("Accept") String accept, Context context); - @Headers({"Accept: application/json;q=0.9", "Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications/{applicationId}") - @ExpectedResponses({200, 204}) + @Headers({ "Accept: application/json;q=0.9", "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications/{applicationId}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("securityConnectorName") String securityConnectorName, - @PathParam("applicationId") String applicationId, - Context context); + @PathParam("applicationId") String applicationId, Context context); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific application for the requested scope by applicationId along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String securityConnectorName, String applicationId) { + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String applicationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -150,26 +133,16 @@ private Mono> getWithResponseAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - applicationId, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, applicationId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. @@ -177,22 +150,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific application for the requested scope by applicationId along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String securityConnectorName, String applicationId, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String securityConnectorName, String applicationId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -208,23 +177,15 @@ private Mono> getWithResponseAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - applicationId, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, applicationId, accept, context); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -233,17 +194,17 @@ private Mono> getWithResponseAsync( * @return a specific application for the requested scope by applicationId on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String securityConnectorName, String applicationId) { + private Mono getAsync(String resourceGroupName, String securityConnectorName, + String applicationId) { return getWithResponseAsync(resourceGroupName, securityConnectorName, applicationId) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. @@ -253,16 +214,16 @@ private Mono getAsync( * @return a specific application for the requested scope by applicationId along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String securityConnectorName, String applicationId, Context context) { + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, Context context) { return getWithResponseAsync(resourceGroupName, securityConnectorName, applicationId, context).block(); } /** * Get a specific application for the requested scope by applicationId. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -277,32 +238,28 @@ public ApplicationInner get(String resourceGroupName, String securityConnectorNa /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security Application over a given scope along with {@link Response} on successful completion of {@link - * Mono}. + * @return security Application over a given scope along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String securityConnectorName, String applicationId, ApplicationInner application) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String applicationId, ApplicationInner application) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -323,27 +280,17 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - applicationId, - application, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, applicationId, application, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. @@ -351,27 +298,19 @@ private Mono> createOrUpdateWithResponseAsync( * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security Application over a given scope along with {@link Response} on successful completion of {@link - * Mono}. + * @return security Application over a given scope along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String securityConnectorName, - String applicationId, - ApplicationInner application, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, String applicationId, ApplicationInner application, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -392,24 +331,15 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - applicationId, - application, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, applicationId, application, accept, context); } /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. @@ -419,17 +349,17 @@ private Mono> createOrUpdateWithResponseAsync( * @return security Application over a given scope on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String securityConnectorName, String applicationId, ApplicationInner application) { + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + String applicationId, ApplicationInner application) { return createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, applicationId, application) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. @@ -440,22 +370,17 @@ private Mono createOrUpdateAsync( * @return security Application over a given scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String securityConnectorName, - String applicationId, - ApplicationInner application, - Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, securityConnectorName, applicationId, application, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, ApplicationInner application, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, applicationId, application, + context).block(); } /** * Creates or update a security Application on the given security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param application Application over a subscription scope. @@ -465,18 +390,17 @@ public Response createOrUpdateWithResponse( * @return security Application over a given scope. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ApplicationInner createOrUpdate( - String resourceGroupName, String securityConnectorName, String applicationId, ApplicationInner application) { - return createOrUpdateWithResponse( - resourceGroupName, securityConnectorName, applicationId, application, Context.NONE) - .getValue(); + public ApplicationInner createOrUpdate(String resourceGroupName, String securityConnectorName, String applicationId, + ApplicationInner application) { + return createOrUpdateWithResponse(resourceGroupName, securityConnectorName, applicationId, application, + Context.NONE).getValue(); } /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -485,19 +409,15 @@ public ApplicationInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String securityConnectorName, String applicationId) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String securityConnectorName, + String applicationId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -512,25 +432,16 @@ private Mono> deleteWithResponseAsync( } final String apiVersion = "2022-07-01-preview"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - applicationId, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, applicationId, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. @@ -540,19 +451,15 @@ private Mono> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String securityConnectorName, String applicationId, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String securityConnectorName, + String applicationId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -567,22 +474,15 @@ private Mono> deleteWithResponseAsync( } final String apiVersion = "2022-07-01-preview"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - applicationId, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, applicationId, context); } /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -598,9 +498,9 @@ private Mono deleteAsync(String resourceGroupName, String securityConnecto /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. @@ -610,16 +510,16 @@ private Mono deleteAsync(String resourceGroupName, String securityConnecto * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse( - String resourceGroupName, String securityConnectorName, String applicationId, Context context) { + public Response deleteWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, Context context) { return deleteWithResponseAsync(resourceGroupName, securityConnectorName, applicationId, context).block(); } /** * Delete an Application over a given scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsImpl.java index 006bf422b7c22..b5d8a48cbf414 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationOperationsImpl.java @@ -20,22 +20,18 @@ public final class SecurityConnectorApplicationOperationsImpl implements Securit private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecurityConnectorApplicationOperationsImpl( - SecurityConnectorApplicationOperationsClient innerClient, + public SecurityConnectorApplicationOperationsImpl(SecurityConnectorApplicationOperationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String resourceGroupName, String securityConnectorName, String applicationId, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, securityConnectorName, applicationId, context); + public Response getWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, securityConnectorName, applicationId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ApplicationImpl(inner.getValue(), this.manager())); } else { return null; @@ -51,32 +47,22 @@ public Application get(String resourceGroupName, String securityConnectorName, S } } - public Response createOrUpdateWithResponse( - String resourceGroupName, - String securityConnectorName, - String applicationId, - ApplicationInner application, - Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateWithResponse( - resourceGroupName, securityConnectorName, applicationId, application, context); + public Response createOrUpdateWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, ApplicationInner application, Context context) { + Response inner = this.serviceClient().createOrUpdateWithResponse(resourceGroupName, + securityConnectorName, applicationId, application, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ApplicationImpl(inner.getValue(), this.manager())); } else { return null; } } - public Application createOrUpdate( - String resourceGroupName, String securityConnectorName, String applicationId, ApplicationInner application) { - ApplicationInner inner = - this.serviceClient().createOrUpdate(resourceGroupName, securityConnectorName, applicationId, application); + public Application createOrUpdate(String resourceGroupName, String securityConnectorName, String applicationId, + ApplicationInner application) { + ApplicationInner inner + = this.serviceClient().createOrUpdate(resourceGroupName, securityConnectorName, applicationId, application); if (inner != null) { return new ApplicationImpl(inner, this.manager()); } else { @@ -84,11 +70,10 @@ public Application createOrUpdate( } } - public Response deleteWithResponse( - String resourceGroupName, String securityConnectorName, String applicationId, Context context) { - return this - .serviceClient() - .deleteWithResponse(resourceGroupName, securityConnectorName, applicationId, context); + public Response deleteWithResponse(String resourceGroupName, String securityConnectorName, + String applicationId, Context context) { + return this.serviceClient().deleteWithResponse(resourceGroupName, securityConnectorName, applicationId, + context); } public void delete(String resourceGroupName, String securityConnectorName, String applicationId) { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsClientImpl.java index 40cd382330cd0..80b9367cdcecb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsClientImpl.java @@ -30,26 +30,28 @@ import com.azure.resourcemanager.security.models.ApplicationsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecurityConnectorApplicationsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityConnectorApplicationsClient. + */ public final class SecurityConnectorApplicationsClientImpl implements SecurityConnectorApplicationsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecurityConnectorApplicationsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecurityConnectorApplicationsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecurityConnectorApplicationsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SecurityConnectorApplicationsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SecurityConnectorApplicationsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,57 +62,46 @@ public final class SecurityConnectorApplicationsClientImpl implements SecurityCo @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecurityConnectorApplicationsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}/providers/Microsoft.Security/applications") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("securityConnectorName") String securityConnectorName, - @HeaderParam("Accept") String accept, + @PathParam("securityConnectorName") String securityConnectorName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a security connector level scope along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of all relevant applications over a security connector level scope along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String securityConnectorName) { + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -123,56 +114,36 @@ private Mono> listSinglePageAsync( final String apiVersion = "2022-07-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a security connector level scope along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list of all relevant applications over a security connector level scope along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String resourceGroupName, String securityConnectorName, Context context) { + private Mono> listSinglePageAsync(String resourceGroupName, + String securityConnectorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -186,76 +157,61 @@ private Mono> listSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a security connector level scope as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String resourceGroupName, String securityConnectorName) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, securityConnectorName), + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName), nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a security connector level scope as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listAsync( - String resourceGroupName, String securityConnectorName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(resourceGroupName, securityConnectorName, context), + private PagedFlux listAsync(String resourceGroupName, String securityConnectorName, + Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(resourceGroupName, securityConnectorName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a security connector level scope as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String resourceGroupName, String securityConnectorName) { @@ -264,33 +220,34 @@ public PagedIterable list(String resourceGroupName, String sec /** * Get a list of all relevant applications over a security connector level scope. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all relevant applications over a security connector level scope as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable list( - String resourceGroupName, String securityConnectorName, Context context) { + public PagedIterable list(String resourceGroupName, String securityConnectorName, + Context context) { return new PagedIterable<>(listAsync(resourceGroupName, securityConnectorName, context)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a security applications list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a security applications list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -298,37 +255,28 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return page of a security applications list along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return page of a security applications list along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -336,23 +284,13 @@ private Mono> listNextSinglePageAsync(String nex return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsImpl.java index d8ac67980999a..655888a76d63d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorApplicationsImpl.java @@ -19,8 +19,7 @@ public final class SecurityConnectorApplicationsImpl implements SecurityConnecto private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecurityConnectorApplicationsImpl( - SecurityConnectorApplicationsClient innerClient, + public SecurityConnectorApplicationsImpl(SecurityConnectorApplicationsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -28,13 +27,13 @@ public SecurityConnectorApplicationsImpl( public PagedIterable list(String resourceGroupName, String securityConnectorName) { PagedIterable inner = this.serviceClient().list(resourceGroupName, securityConnectorName); - return Utils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); } public PagedIterable list(String resourceGroupName, String securityConnectorName, Context context) { - PagedIterable inner = - this.serviceClient().list(resourceGroupName, securityConnectorName, context); - return Utils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().list(resourceGroupName, securityConnectorName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new ApplicationImpl(inner1, this.manager())); } private SecurityConnectorApplicationsClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorImpl.java index 6eaee85d9cac8..78c6586d2743d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorImpl.java @@ -107,22 +107,16 @@ public SecurityConnectorImpl withExistingResourceGroup(String resourceGroupName) } public SecurityConnector create() { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityConnectors() - .createOrUpdateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityConnectors() + .createOrUpdateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), Context.NONE) + .getValue(); return this; } public SecurityConnector create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityConnectors() - .createOrUpdateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityConnectors() + .createOrUpdateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), context) + .getValue(); return this; } @@ -137,50 +131,34 @@ public SecurityConnectorImpl update() { } public SecurityConnector apply() { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityConnectors() - .updateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityConnectors() + .updateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), Context.NONE).getValue(); return this; } public SecurityConnector apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityConnectors() - .updateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityConnectors() + .updateWithResponse(resourceGroupName, securityConnectorName, this.innerModel(), context).getValue(); return this; } - SecurityConnectorImpl( - SecurityConnectorInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecurityConnectorImpl(SecurityConnectorInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.resourceGroupName = Utils.getValueFromIdByName(innerObject.id(), "resourceGroups"); - this.securityConnectorName = Utils.getValueFromIdByName(innerObject.id(), "securityConnectors"); + this.resourceGroupName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "resourceGroups"); + this.securityConnectorName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "securityConnectors"); } public SecurityConnector refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityConnectors() - .getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityConnectors() + .getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, Context.NONE).getValue(); return this; } public SecurityConnector refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityConnectors() - .getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityConnectors() + .getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsClientImpl.java index 2092e42d6eb4f..14e5f5477b9b8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsClientImpl.java @@ -34,22 +34,28 @@ import com.azure.resourcemanager.security.models.SecurityConnectorsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecurityConnectorsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityConnectorsClient. + */ public final class SecurityConnectorsClientImpl implements SecurityConnectorsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecurityConnectorsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecurityConnectorsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecurityConnectorsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SecurityConnectorsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(SecurityConnectorsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -60,196 +66,143 @@ public final class SecurityConnectorsClientImpl implements SecurityConnectorsCli @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecurityConnectorsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securityConnectors") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @HeaderParam("Accept") String accept, + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> getByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("securityConnectorName") String securityConnectorName, - @HeaderParam("Accept") String accept, + @PathParam("securityConnectorName") String securityConnectorName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") - @ExpectedResponses({200, 201}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("securityConnectorName") String securityConnectorName, @BodyParam("application/json") SecurityConnectorInner securityConnector, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Patch( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Patch("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, @PathParam("securityConnectorName") String securityConnectorName, @BodyParam("application/json") SecurityConnectorInner securityConnector, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/securityConnectors/{securityConnectorName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("securityConnectorName") String securityConnectorName, - @HeaderParam("Accept") String accept, + @PathParam("securityConnectorName") String securityConnectorName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security connectors response as paginated response with {@link PagedFlux}. @@ -262,7 +215,7 @@ private PagedFlux listAsync() { /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -271,14 +224,14 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security connectors response as paginated response with {@link PagedIterable}. @@ -291,7 +244,7 @@ public PagedIterable list() { /** * Lists all the security connectors in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -306,118 +259,83 @@ public PagedIterable list(Context context) { /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono .error(new IllegalArgumentException("Parameter resourceGroupName is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -425,17 +343,16 @@ private Mono> listByResourceGroupSinglePag */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -444,17 +361,16 @@ private PagedFlux listByResourceGroupAsync(String resour */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, Context context) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, context), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); } /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -468,9 +384,9 @@ public PagedIterable listByResourceGroup(String resource /** * Lists all the security connectors in the specified resource group. Use the 'nextLink' property in the response to * get the next page of security connectors for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -484,9 +400,9 @@ public PagedIterable listByResourceGroup(String resource /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -494,19 +410,15 @@ public PagedIterable listByResourceGroup(String resource * @return the security connector resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String securityConnectorName) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String securityConnectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -516,28 +428,19 @@ private Mono> getByResourceGroupWithResponseAsy return Mono .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - accept, - context)) + .withContext(context -> service.getByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -546,19 +449,15 @@ private Mono> getByResourceGroupWithResponseAsy * @return the security connector resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getByResourceGroupWithResponseAsync( - String resourceGroupName, String securityConnectorName, Context context) { + private Mono> getByResourceGroupWithResponseAsync(String resourceGroupName, + String securityConnectorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -568,25 +467,18 @@ private Mono> getByResourceGroupWithResponseAsy return Mono .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - accept, - context); + return service.getByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, accept, context); } /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -594,17 +486,17 @@ private Mono> getByResourceGroupWithResponseAsy * @return the security connector resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getByResourceGroupAsync( - String resourceGroupName, String securityConnectorName) { + private Mono getByResourceGroupAsync(String resourceGroupName, + String securityConnectorName) { return getByResourceGroupWithResponseAsync(resourceGroupName, securityConnectorName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -613,16 +505,16 @@ private Mono getByResourceGroupAsync( * @return the security connector resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getByResourceGroupWithResponse( - String resourceGroupName, String securityConnectorName, Context context) { + public Response getByResourceGroupWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { return getByResourceGroupWithResponseAsync(resourceGroupName, securityConnectorName, context).block(); } /** * Retrieves details of a specific security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -637,9 +529,9 @@ public SecurityConnectorInner getByResourceGroup(String resourceGroupName, Strin /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -648,19 +540,15 @@ public SecurityConnectorInner getByResourceGroup(String resourceGroupName, Strin * @return the security connector resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, SecurityConnectorInner securityConnector) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -676,30 +564,21 @@ private Mono> createOrUpdateWithResponseAsync( } else { securityConnector.validate(); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - securityConnector, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, securityConnector, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @param context The context to associate with this operation. @@ -709,22 +588,15 @@ private Mono> createOrUpdateWithResponseAsync( * @return the security connector resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, - String securityConnectorName, - SecurityConnectorInner securityConnector, - Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String securityConnectorName, SecurityConnectorInner securityConnector, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -740,27 +612,19 @@ private Mono> createOrUpdateWithResponseAsync( } else { securityConnector.validate(); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - securityConnector, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, securityConnector, accept, context); } /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -769,8 +633,8 @@ private Mono> createOrUpdateWithResponseAsync( * @return the security connector resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector) { + private Mono createOrUpdateAsync(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector) { return createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, securityConnector) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -778,9 +642,9 @@ private Mono createOrUpdateAsync( /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @param context The context to associate with this operation. @@ -790,11 +654,8 @@ private Mono createOrUpdateAsync( * @return the security connector resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, - String securityConnectorName, - SecurityConnectorInner securityConnector, - Context context) { + public Response createOrUpdateWithResponse(String resourceGroupName, + String securityConnectorName, SecurityConnectorInner securityConnector, Context context) { return createOrUpdateWithResponseAsync(resourceGroupName, securityConnectorName, securityConnector, context) .block(); } @@ -802,9 +663,9 @@ public Response createOrUpdateWithResponse( /** * Creates or updates a security connector. If a security connector is already created and a subsequent request is * issued for the same security connector id, then it will be updated. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -813,17 +674,17 @@ public Response createOrUpdateWithResponse( * @return the security connector resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SecurityConnectorInner createOrUpdate( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector) { + public SecurityConnectorInner createOrUpdate(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector) { return createOrUpdateWithResponse(resourceGroupName, securityConnectorName, securityConnector, Context.NONE) .getValue(); } /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -832,19 +693,15 @@ public SecurityConnectorInner createOrUpdate( * @return the security connector resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, SecurityConnectorInner securityConnector) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -860,29 +717,20 @@ private Mono> updateWithResponseAsync( } else { securityConnector.validate(); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - securityConnector, - accept, - context)) + context -> service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, securityConnectorName, securityConnector, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @param context The context to associate with this operation. @@ -892,22 +740,15 @@ private Mono> updateWithResponseAsync( * @return the security connector resource along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String resourceGroupName, - String securityConnectorName, - SecurityConnectorInner securityConnector, - Context context) { + private Mono> updateWithResponseAsync(String resourceGroupName, + String securityConnectorName, SecurityConnectorInner securityConnector, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -923,26 +764,18 @@ private Mono> updateWithResponseAsync( } else { securityConnector.validate(); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - securityConnector, - accept, - context); + return service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, securityConnector, accept, context); } /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -951,17 +784,17 @@ private Mono> updateWithResponseAsync( * @return the security connector resource on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector) { + private Mono updateAsync(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector) { return updateWithResponseAsync(resourceGroupName, securityConnectorName, securityConnector) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @param context The context to associate with this operation. @@ -971,19 +804,16 @@ private Mono updateAsync( * @return the security connector resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String resourceGroupName, - String securityConnectorName, - SecurityConnectorInner securityConnector, - Context context) { + public Response updateWithResponse(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector, Context context) { return updateWithResponseAsync(resourceGroupName, securityConnectorName, securityConnector, context).block(); } /** * Updates a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param securityConnector The security connector resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -992,16 +822,16 @@ public Response updateWithResponse( * @return the security connector resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SecurityConnectorInner update( - String resourceGroupName, String securityConnectorName, SecurityConnectorInner securityConnector) { + public SecurityConnectorInner update(String resourceGroupName, String securityConnectorName, + SecurityConnectorInner securityConnector) { return updateWithResponse(resourceGroupName, securityConnectorName, securityConnector, Context.NONE).getValue(); } /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1011,16 +841,12 @@ public SecurityConnectorInner update( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String resourceGroupName, String securityConnectorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1030,28 +856,19 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S return Mono .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, securityConnectorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1060,19 +877,15 @@ private Mono> deleteWithResponseAsync(String resourceGroupName, S * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String resourceGroupName, String securityConnectorName, Context context) { + private Mono> deleteWithResponseAsync(String resourceGroupName, String securityConnectorName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1082,25 +895,18 @@ private Mono> deleteWithResponseAsync( return Mono .error(new IllegalArgumentException("Parameter securityConnectorName is required and cannot be null.")); } - final String apiVersion = "2023-03-01-preview"; + final String apiVersion = "2023-10-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - securityConnectorName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), resourceGroupName, + securityConnectorName, accept, context); } /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1114,9 +920,9 @@ private Mono deleteAsync(String resourceGroupName, String securityConnecto /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1131,9 +937,9 @@ public Response deleteWithResponse(String resourceGroupName, String securi /** * Deletes a security connector. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param securityConnectorName The security connector name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1146,14 +952,15 @@ public void delete(String resourceGroupName, String securityConnectorName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1161,37 +968,28 @@ private Mono> listNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1199,36 +997,27 @@ private Mono> listNextSinglePageAsync(Stri return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1236,63 +1025,45 @@ private Mono> listByResourceGroupNextSingl return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security connectors response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security connectors response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsImpl.java index 7ca42cfa356d3..f2939bcf75f26 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityConnectorsImpl.java @@ -21,42 +21,39 @@ public final class SecurityConnectorsImpl implements SecurityConnectors { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecurityConnectorsImpl( - SecurityConnectorsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SecurityConnectorsImpl(SecurityConnectorsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName) { PagedIterable inner = this.serviceClient().listByResourceGroup(resourceGroupName); - return Utils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); } public PagedIterable listByResourceGroup(String resourceGroupName, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, context); - return Utils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityConnectorImpl(inner1, this.manager())); } - public Response getByResourceGroupWithResponse( - String resourceGroupName, String securityConnectorName, Context context) { - Response inner = - this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, context); + public Response getByResourceGroupWithResponse(String resourceGroupName, + String securityConnectorName, Context context) { + Response inner + = this.serviceClient().getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityConnectorImpl(inner.getValue(), this.manager())); } else { return null; @@ -64,8 +61,8 @@ public Response getByResourceGroupWithResponse( } public SecurityConnector getByResourceGroup(String resourceGroupName, String securityConnectorName) { - SecurityConnectorInner inner = - this.serviceClient().getByResourceGroup(resourceGroupName, securityConnectorName); + SecurityConnectorInner inner + = this.serviceClient().getByResourceGroup(resourceGroupName, securityConnectorName); if (inner != null) { return new SecurityConnectorImpl(inner, this.manager()); } else { @@ -73,8 +70,8 @@ public SecurityConnector getByResourceGroup(String resourceGroupName, String sec } } - public Response deleteByResourceGroupWithResponse( - String resourceGroupName, String securityConnectorName, Context context) { + public Response deleteByResourceGroupWithResponse(String resourceGroupName, String securityConnectorName, + Context context) { return this.serviceClient().deleteWithResponse(resourceGroupName, securityConnectorName, context); } @@ -83,85 +80,57 @@ public void deleteByResourceGroup(String resourceGroupName, String securityConne } public SecurityConnector getById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String securityConnectorName = Utils.getValueFromIdByName(id, "securityConnectors"); + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); if (securityConnectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String securityConnectorName = Utils.getValueFromIdByName(id, "securityConnectors"); + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); if (securityConnectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); } return this.getByResourceGroupWithResponse(resourceGroupName, securityConnectorName, context); } public void deleteById(String id) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String securityConnectorName = Utils.getValueFromIdByName(id, "securityConnectors"); + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); if (securityConnectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); } this.deleteByResourceGroupWithResponse(resourceGroupName, securityConnectorName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String resourceGroupName = Utils.getValueFromIdByName(id, "resourceGroups"); + String resourceGroupName = ResourceManagerUtils.getValueFromIdByName(id, "resourceGroups"); if (resourceGroupName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'resourceGroups'.", id))); } - String securityConnectorName = Utils.getValueFromIdByName(id, "securityConnectors"); + String securityConnectorName = ResourceManagerUtils.getValueFromIdByName(id, "securityConnectors"); if (securityConnectorName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityConnectors'.", id))); } return this.deleteByResourceGroupWithResponse(resourceGroupName, securityConnectorName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactImpl.java index b2543af8c36e6..2e389dbfbe73e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactImpl.java @@ -6,17 +6,20 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.SecurityContactInner; +import com.azure.resourcemanager.security.models.NotificationsSource; import com.azure.resourcemanager.security.models.SecurityContact; -import com.azure.resourcemanager.security.models.SecurityContactPropertiesAlertNotifications; +import com.azure.resourcemanager.security.models.SecurityContactName; import com.azure.resourcemanager.security.models.SecurityContactPropertiesNotificationsByRole; +import java.util.Collections; +import java.util.List; public final class SecurityContactImpl implements SecurityContact, SecurityContact.Definition { private SecurityContactInner innerObject; private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecurityContactImpl( - SecurityContactInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecurityContactImpl(SecurityContactInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -41,8 +44,17 @@ public String phone() { return this.innerModel().phone(); } - public SecurityContactPropertiesAlertNotifications alertNotifications() { - return this.innerModel().alertNotifications(); + public Boolean isEnabled() { + return this.innerModel().isEnabled(); + } + + public List notificationsSources() { + List inner = this.innerModel().notificationsSources(); + if (inner != null) { + return Collections.unmodifiableList(inner); + } else { + return Collections.emptyList(); + } } public SecurityContactPropertiesNotificationsByRole notificationsByRole() { @@ -57,51 +69,35 @@ private com.azure.resourcemanager.security.SecurityManager manager() { return this.serviceManager; } - private String securityContactName; + private SecurityContactName securityContactName; public SecurityContact create() { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityContacts() - .createWithResponse(securityContactName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityContacts() + .createWithResponse(securityContactName, this.innerModel(), Context.NONE).getValue(); return this; } public SecurityContact create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityContacts() - .createWithResponse(securityContactName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityContacts() + .createWithResponse(securityContactName, this.innerModel(), context).getValue(); return this; } - SecurityContactImpl(String name, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecurityContactImpl(SecurityContactName name, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = new SecurityContactInner(); this.serviceManager = serviceManager; this.securityContactName = name; } public SecurityContact refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityContacts() - .getWithResponse(securityContactName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityContacts() + .getWithResponse(securityContactName, Context.NONE).getValue(); return this; } public SecurityContact refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getSecurityContacts() - .getWithResponse(securityContactName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getSecurityContacts() + .getWithResponse(securityContactName, context).getValue(); return this; } @@ -115,13 +111,18 @@ public SecurityContactImpl withPhone(String phone) { return this; } - public SecurityContactImpl withAlertNotifications(SecurityContactPropertiesAlertNotifications alertNotifications) { - this.innerModel().withAlertNotifications(alertNotifications); + public SecurityContactImpl withIsEnabled(Boolean isEnabled) { + this.innerModel().withIsEnabled(isEnabled); + return this; + } + + public SecurityContactImpl withNotificationsSources(List notificationsSources) { + this.innerModel().withNotificationsSources(notificationsSources); return this; } - public SecurityContactImpl withNotificationsByRole( - SecurityContactPropertiesNotificationsByRole notificationsByRole) { + public SecurityContactImpl + withNotificationsByRole(SecurityContactPropertiesNotificationsByRole notificationsByRole) { this.innerModel().withNotificationsByRole(notificationsByRole); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsClientImpl.java index 0e358f62e47a0..ac7a1e1c0e3de 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsClientImpl.java @@ -31,24 +31,31 @@ import com.azure.resourcemanager.security.fluent.SecurityContactsClient; import com.azure.resourcemanager.security.fluent.models.SecurityContactInner; import com.azure.resourcemanager.security.models.SecurityContactList; +import com.azure.resourcemanager.security.models.SecurityContactName; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecurityContactsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityContactsClient. + */ public final class SecurityContactsClientImpl implements SecurityContactsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecurityContactsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecurityContactsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecurityContactsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SecurityContactsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SecurityContactsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -59,149 +66,109 @@ public final class SecurityContactsClientImpl implements SecurityContactsClient @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecurityContactsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securityContacts") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securityContacts/{securityContactName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("securityContactName") String securityContactName, - @HeaderParam("Accept") String accept, - Context context); + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("securityContactName") SecurityContactName securityContactName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securityContacts/{securityContactName}") - @ExpectedResponses({200, 201}) + @ExpectedResponses({ 200, 201 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("securityContactName") String securityContactName, - @BodyParam("application/json") SecurityContactInner securityContact, - @HeaderParam("Accept") String accept, + Mono> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("securityContactName") SecurityContactName securityContactName, + @BodyParam("application/json") SecurityContactInner securityContact, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securityContacts/{securityContactName}") - @ExpectedResponses({200, 204}) + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("securityContactName") String securityContactName, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("securityContactName") SecurityContactName securityContactName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * List all security contact configurations for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security contacts response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security contacts response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * List all security contact configurations for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security contacts response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security contacts response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * List all security contact configurations for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security contacts response as paginated response with {@link PagedFlux}. @@ -213,7 +180,7 @@ private PagedFlux listAsync() { /** * List all security contact configurations for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -222,13 +189,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * List all security contact configurations for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security contacts response as paginated response with {@link PagedIterable}. @@ -240,7 +207,7 @@ public PagedIterable list() { /** * List all security contact configurations for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -254,93 +221,72 @@ public PagedIterable list(Context context) { /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return default Security contact configurations for the subscription along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String securityContactName) { + private Mono> getWithResponseAsync(SecurityContactName securityContactName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (securityContactName == null) { return Mono .error(new IllegalArgumentException("Parameter securityContactName is required and cannot be null.")); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - securityContactName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + securityContactName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return default Security contact configurations for the subscription along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(String securityContactName, Context context) { + private Mono> getWithResponseAsync(SecurityContactName securityContactName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (securityContactName == null) { return Mono .error(new IllegalArgumentException("Parameter securityContactName is required and cannot be null.")); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - securityContactName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), securityContactName, + accept, context); } /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -348,13 +294,13 @@ private Mono> getWithResponseAsync(String securit * @return default Security contact configurations for the subscription on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(String securityContactName) { + private Mono getAsync(SecurityContactName securityContactName) { return getWithResponseAsync(securityContactName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -363,13 +309,13 @@ private Mono getAsync(String securityContactName) { * @return default Security contact configurations for the subscription along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(String securityContactName, Context context) { + public Response getWithResponse(SecurityContactName securityContactName, Context context) { return getWithResponseAsync(securityContactName, context).block(); } /** * Get Default Security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -377,35 +323,31 @@ public Response getWithResponse(String securityContactName * @return default Security contact configurations for the subscription. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SecurityContactInner get(String securityContactName) { + public SecurityContactInner get(SecurityContactName securityContactName) { return getWithResponse(securityContactName, Context.NONE).getValue(); } /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String securityContactName, SecurityContactInner securityContact) { + private Mono> createWithResponseAsync(SecurityContactName securityContactName, + SecurityContactInner securityContact) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (securityContactName == null) { return Mono @@ -417,26 +359,17 @@ private Mono> createWithResponseAsync( } else { securityContact.validate(); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - securityContactName, - securityContact, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), securityContactName, securityContact, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @param context The context to associate with this operation. @@ -444,22 +377,18 @@ private Mono> createWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud along with - * {@link Response} on successful completion of {@link Mono}. + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String securityContactName, SecurityContactInner securityContact, Context context) { + private Mono> createWithResponseAsync(SecurityContactName securityContactName, + SecurityContactInner securityContact, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (securityContactName == null) { return Mono @@ -471,40 +400,34 @@ private Mono> createWithResponseAsync( } else { securityContact.validate(); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - securityContactName, - securityContact, - accept, - context); + return service.create(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + securityContactName, securityContact, accept, context); } /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync(String securityContactName, SecurityContactInner securityContact) { + private Mono createAsync(SecurityContactName securityContactName, + SecurityContactInner securityContact) { return createWithResponseAsync(securityContactName, securityContact) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @param context The context to associate with this operation. @@ -512,17 +435,17 @@ private Mono createAsync(String securityContactName, Secur * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud along with - * {@link Response}. + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String securityContactName, SecurityContactInner securityContact, Context context) { + public Response createWithResponse(SecurityContactName securityContactName, + SecurityContactInner securityContact, Context context) { return createWithResponseAsync(securityContactName, securityContact, context).block(); } /** * Create security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param securityContact Security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -531,13 +454,13 @@ public Response createWithResponse( * @return contact details and configurations for notifications coming from Microsoft Defender for Cloud. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SecurityContactInner create(String securityContactName, SecurityContactInner securityContact) { + public SecurityContactInner create(SecurityContactName securityContactName, SecurityContactInner securityContact) { return createWithResponse(securityContactName, securityContact, Context.NONE).getValue(); } /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -545,42 +468,30 @@ public SecurityContactInner create(String securityContactName, SecurityContactIn * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String securityContactName) { + private Mono> deleteWithResponseAsync(SecurityContactName securityContactName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (securityContactName == null) { return Mono .error(new IllegalArgumentException("Parameter securityContactName is required and cannot be null.")); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - securityContactName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), securityContactName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -589,39 +500,29 @@ private Mono> deleteWithResponseAsync(String securityContactName) * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync(String securityContactName, Context context) { + private Mono> deleteWithResponseAsync(SecurityContactName securityContactName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (securityContactName == null) { return Mono .error(new IllegalArgumentException("Parameter securityContactName is required and cannot be null.")); } - final String apiVersion = "2020-01-01-preview"; + final String apiVersion = "2023-12-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - securityContactName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + securityContactName, accept, context); } /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -629,13 +530,13 @@ private Mono> deleteWithResponseAsync(String securityContactName, * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync(String securityContactName) { + private Mono deleteAsync(SecurityContactName securityContactName) { return deleteWithResponseAsync(securityContactName).flatMap(ignored -> Mono.empty()); } /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -644,33 +545,34 @@ private Mono deleteAsync(String securityContactName) { * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response deleteWithResponse(String securityContactName, Context context) { + public Response deleteWithResponse(SecurityContactName securityContactName, Context context) { return deleteWithResponseAsync(securityContactName, context).block(); } /** * Delete security contact configurations for the subscription. - * + * * @param securityContactName Name of the security contact object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete(String securityContactName) { + public void delete(SecurityContactName securityContactName) { deleteWithResponse(securityContactName, Context.NONE); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security contacts response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security contacts response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -678,37 +580,28 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security contacts response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security contacts response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -716,23 +609,13 @@ private Mono> listNextSinglePageAsync(String return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsImpl.java index 913f99337779a..c0f936d77aada 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityContactsImpl.java @@ -12,6 +12,7 @@ import com.azure.resourcemanager.security.fluent.SecurityContactsClient; import com.azure.resourcemanager.security.fluent.models.SecurityContactInner; import com.azure.resourcemanager.security.models.SecurityContact; +import com.azure.resourcemanager.security.models.SecurityContactName; import com.azure.resourcemanager.security.models.SecurityContacts; public final class SecurityContactsImpl implements SecurityContacts { @@ -21,36 +22,33 @@ public final class SecurityContactsImpl implements SecurityContacts { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecurityContactsImpl( - SecurityContactsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SecurityContactsImpl(SecurityContactsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecurityContactImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityContactImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SecurityContactImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityContactImpl(inner1, this.manager())); } - public Response getWithResponse(String securityContactName, Context context) { + public Response getWithResponse(SecurityContactName securityContactName, Context context) { Response inner = this.serviceClient().getWithResponse(securityContactName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityContactImpl(inner.getValue(), this.manager())); } else { return null; } } - public SecurityContact get(String securityContactName) { + public SecurityContact get(SecurityContactName securityContactName) { SecurityContactInner inner = this.serviceClient().get(securityContactName); if (inner != null) { return new SecurityContactImpl(inner, this.manager()); @@ -59,63 +57,51 @@ public SecurityContact get(String securityContactName) { } } - public Response deleteWithResponse(String securityContactName, Context context) { + public Response deleteWithResponse(SecurityContactName securityContactName, Context context) { return this.serviceClient().deleteWithResponse(securityContactName, context); } - public void delete(String securityContactName) { + public void delete(SecurityContactName securityContactName) { this.serviceClient().delete(securityContactName); } public SecurityContact getById(String id) { - String securityContactName = Utils.getValueFromIdByName(id, "securityContacts"); - if (securityContactName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); + String securityContactNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "securityContacts"); + if (securityContactNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); } + SecurityContactName securityContactName = SecurityContactName.fromString(securityContactNameLocal); return this.getWithResponse(securityContactName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String securityContactName = Utils.getValueFromIdByName(id, "securityContacts"); - if (securityContactName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); + String securityContactNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "securityContacts"); + if (securityContactNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); } + SecurityContactName securityContactName = SecurityContactName.fromString(securityContactNameLocal); return this.getWithResponse(securityContactName, context); } public void deleteById(String id) { - String securityContactName = Utils.getValueFromIdByName(id, "securityContacts"); - if (securityContactName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); + String securityContactNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "securityContacts"); + if (securityContactNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); } + SecurityContactName securityContactName = SecurityContactName.fromString(securityContactNameLocal); this.deleteWithResponse(securityContactName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String securityContactName = Utils.getValueFromIdByName(id, "securityContacts"); - if (securityContactName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); + String securityContactNameLocal = ResourceManagerUtils.getValueFromIdByName(id, "securityContacts"); + if (securityContactNameLocal == null) { + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'securityContacts'.", id))); } + SecurityContactName securityContactName = SecurityContactName.fromString(securityContactNameLocal); return this.deleteWithResponse(securityContactName, context); } @@ -127,7 +113,7 @@ private com.azure.resourcemanager.security.SecurityManager manager() { return this.serviceManager; } - public SecurityContactImpl define(String name) { + public SecurityContactImpl define(SecurityContactName name) { return new SecurityContactImpl(name, this.manager()); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorImpl.java index 3564896ce8d37..8b47bb0038230 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorImpl.java @@ -13,8 +13,8 @@ public final class SecurityOperatorImpl implements SecurityOperator { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecurityOperatorImpl( - SecurityOperatorInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecurityOperatorImpl(SecurityOperatorInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorListImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorListImpl.java index 7298243843b0b..9d6080621094e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorListImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorListImpl.java @@ -17,8 +17,8 @@ public final class SecurityOperatorListImpl implements SecurityOperatorList { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecurityOperatorListImpl( - SecurityOperatorListInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecurityOperatorListImpl(SecurityOperatorListInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } @@ -26,12 +26,8 @@ public final class SecurityOperatorListImpl implements SecurityOperatorList { public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new SecurityOperatorImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList(inner.stream() + .map(inner1 -> new SecurityOperatorImpl(inner1, this.manager())).collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsClientImpl.java index cf7c403898ff0..a21e42d065723 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsClientImpl.java @@ -28,22 +28,28 @@ import com.azure.resourcemanager.security.fluent.models.SecurityOperatorListInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecurityOperatorsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecurityOperatorsClient. + */ public final class SecurityOperatorsClientImpl implements SecurityOperatorsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecurityOperatorsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecurityOperatorsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecurityOperatorsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SecurityOperatorsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SecurityOperatorsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -54,64 +60,47 @@ public final class SecurityOperatorsClientImpl implements SecurityOperatorsClien @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecurityOperatorsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("pricingName") String pricingName, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("pricingName") String pricingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators/{securityOperatorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators/{securityOperatorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("pricingName") String pricingName, - @PathParam("securityOperatorName") String securityOperatorName, - @HeaderParam("Accept") String accept, + @PathParam("securityOperatorName") String securityOperatorName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators/{securityOperatorName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators/{securityOperatorName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("pricingName") String pricingName, - @PathParam("securityOperatorName") String securityOperatorName, - @HeaderParam("Accept") String accept, + @PathParam("securityOperatorName") String securityOperatorName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators/{securityOperatorName}") - @ExpectedResponses({200, 204}) + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/pricings/{pricingName}/securityOperators/{securityOperatorName}") + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("pricingName") String pricingName, - @PathParam("securityOperatorName") String securityOperatorName, - @HeaderParam("Accept") String accept, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, @PathParam("pricingName") String pricingName, + @PathParam("securityOperatorName") String securityOperatorName, @HeaderParam("Accept") String accept, Context context); } /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -121,16 +110,12 @@ Mono> delete( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(String pricingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -138,22 +123,14 @@ private Mono> listWithResponseAsync(String p final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - accept, - context)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + pricingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -164,16 +141,12 @@ private Mono> listWithResponseAsync(String p @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(String pricingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -181,13 +154,13 @@ private Mono> listWithResponseAsync(String p final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), pricingName, accept, context); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), pricingName, accept, + context); } /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -201,7 +174,7 @@ private Mono listAsync(String pricingName) { /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -216,7 +189,7 @@ public Response listWithResponse(String pricingName, /** * Lists Microsoft Defender for Cloud securityOperators in the subscription. - * + * * @param pricingName name of the pricing configuration. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -230,29 +203,25 @@ public SecurityOperatorListInner list(String pricingName) { /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific security operator for the requested scope along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String pricingName, String securityOperatorName) { + private Mono> getWithResponseAsync(String pricingName, + String securityOperatorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -264,23 +233,14 @@ private Mono> getWithResponseAsync( final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - securityOperatorName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + pricingName, securityOperatorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -288,22 +248,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific security operator for the requested scope along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String pricingName, String securityOperatorName, Context context) { + private Mono> getWithResponseAsync(String pricingName, String securityOperatorName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -315,20 +271,13 @@ private Mono> getWithResponseAsync( final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - securityOperatorName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), pricingName, + securityOperatorName, accept, context); } /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -343,7 +292,7 @@ private Mono getAsync(String pricingName, String security /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -353,14 +302,14 @@ private Mono getAsync(String pricingName, String security * @return a specific security operator for the requested scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String pricingName, String securityOperatorName, Context context) { + public Response getWithResponse(String pricingName, String securityOperatorName, + Context context) { return getWithResponseAsync(pricingName, securityOperatorName, context).block(); } /** * Get a specific security operator for the requested scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -375,29 +324,25 @@ public SecurityOperatorInner get(String pricingName, String securityOperatorName /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security operator under a given subscription and pricing along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String pricingName, String securityOperatorName) { + private Mono> createOrUpdateWithResponseAsync(String pricingName, + String securityOperatorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -409,23 +354,14 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - securityOperatorName, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), pricingName, securityOperatorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -433,22 +369,18 @@ private Mono> createOrUpdateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security operator under a given subscription and pricing along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String pricingName, String securityOperatorName, Context context) { + private Mono> createOrUpdateWithResponseAsync(String pricingName, + String securityOperatorName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -460,20 +392,13 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - securityOperatorName, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + pricingName, securityOperatorName, accept, context); } /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -489,7 +414,7 @@ private Mono createOrUpdateAsync(String pricingName, Stri /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -499,14 +424,14 @@ private Mono createOrUpdateAsync(String pricingName, Stri * @return security operator under a given subscription and pricing along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String pricingName, String securityOperatorName, Context context) { + public Response createOrUpdateWithResponse(String pricingName, String securityOperatorName, + Context context) { return createOrUpdateWithResponseAsync(pricingName, securityOperatorName, context).block(); } /** * Creates Microsoft Defender for Cloud security operator on the given scope. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -521,7 +446,7 @@ public SecurityOperatorInner createOrUpdate(String pricingName, String securityO /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -532,16 +457,12 @@ public SecurityOperatorInner createOrUpdate(String pricingName, String securityO @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String pricingName, String securityOperatorName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -553,23 +474,14 @@ private Mono> deleteWithResponseAsync(String pricingName, String final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - securityOperatorName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), pricingName, securityOperatorName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -579,19 +491,15 @@ private Mono> deleteWithResponseAsync(String pricingName, String * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String pricingName, String securityOperatorName, Context context) { + private Mono> deleteWithResponseAsync(String pricingName, String securityOperatorName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (pricingName == null) { return Mono.error(new IllegalArgumentException("Parameter pricingName is required and cannot be null.")); @@ -603,20 +511,13 @@ private Mono> deleteWithResponseAsync( final String apiVersion = "2023-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - pricingName, - securityOperatorName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), pricingName, + securityOperatorName, accept, context); } /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -631,7 +532,7 @@ private Mono deleteAsync(String pricingName, String securityOperatorName) /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @param context The context to associate with this operation. @@ -647,7 +548,7 @@ public Response deleteWithResponse(String pricingName, String securityOper /** * Delete Microsoft Defender for Cloud securityOperator in the subscription. - * + * * @param pricingName name of the pricing configuration. * @param securityOperatorName name of the securityOperator. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsImpl.java index 57bc55c282114..e5d70025bc582 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecurityOperatorsImpl.java @@ -22,8 +22,8 @@ public final class SecurityOperatorsImpl implements SecurityOperators { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecurityOperatorsImpl( - SecurityOperatorsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SecurityOperatorsImpl(SecurityOperatorsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } @@ -31,10 +31,7 @@ public SecurityOperatorsImpl( public Response listWithResponse(String pricingName, Context context) { Response inner = this.serviceClient().listWithResponse(pricingName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityOperatorListImpl(inner.getValue(), this.manager())); } else { return null; @@ -50,15 +47,12 @@ public SecurityOperatorList list(String pricingName) { } } - public Response getWithResponse( - String pricingName, String securityOperatorName, Context context) { - Response inner = - this.serviceClient().getWithResponse(pricingName, securityOperatorName, context); + public Response getWithResponse(String pricingName, String securityOperatorName, + Context context) { + Response inner + = this.serviceClient().getWithResponse(pricingName, securityOperatorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityOperatorImpl(inner.getValue(), this.manager())); } else { return null; @@ -74,15 +68,12 @@ public SecurityOperator get(String pricingName, String securityOperatorName) { } } - public Response createOrUpdateWithResponse( - String pricingName, String securityOperatorName, Context context) { - Response inner = - this.serviceClient().createOrUpdateWithResponse(pricingName, securityOperatorName, context); + public Response createOrUpdateWithResponse(String pricingName, String securityOperatorName, + Context context) { + Response inner + = this.serviceClient().createOrUpdateWithResponse(pricingName, securityOperatorName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityOperatorImpl(inner.getValue(), this.manager())); } else { return null; @@ -98,8 +89,8 @@ public SecurityOperator createOrUpdate(String pricingName, String securityOperat } } - public Response deleteByResourceGroupWithResponse( - String pricingName, String securityOperatorName, Context context) { + public Response deleteByResourceGroupWithResponse(String pricingName, String securityOperatorName, + Context context) { return this.serviceClient().deleteWithResponse(pricingName, securityOperatorName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionImpl.java index daa171b42ba68..a62297af1298c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionImpl.java @@ -14,8 +14,8 @@ public final class SecuritySolutionImpl implements SecuritySolution { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecuritySolutionImpl( - SecuritySolutionInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecuritySolutionImpl(SecuritySolutionInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsClientImpl.java index 844b5caf9c4c0..5f6beef104c4c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.SecuritySolutionList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecuritySolutionsClient. */ +/** + * An instance of this class provides access to all the operations defined in SecuritySolutionsClient. + */ public final class SecuritySolutionsClientImpl implements SecuritySolutionsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecuritySolutionsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecuritySolutionsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecuritySolutionsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SecuritySolutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SecuritySolutionsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,127 +62,91 @@ public final class SecuritySolutionsClientImpl implements SecuritySolutionsClien @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecuritySolutionsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securitySolutions") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/securitySolutions/{securitySolutionName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/securitySolutions/{securitySolutionName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("securitySolutionName") String securitySolutionName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Security Solutions for the subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Security Solutions for the subscription along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list of Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Security Solutions for the subscription as paginated response with {@link PagedFlux}. @@ -188,7 +158,7 @@ private PagedFlux listAsync() { /** * Gets a list of Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -197,13 +167,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets a list of Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Security Solutions for the subscription as paginated response with {@link PagedIterable}. @@ -215,7 +185,7 @@ public PagedIterable list() { /** * Gets a list of Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -229,11 +199,11 @@ public PagedIterable list(Context context) { /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -241,19 +211,15 @@ public PagedIterable list(Context context) { * @return a specific Security Solution along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String securitySolutionName) { + private Mono> getWithResponseAsync(String resourceGroupName, String ascLocation, + String securitySolutionName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -269,28 +235,18 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - securitySolutionName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, securitySolutionName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -299,19 +255,15 @@ private Mono> getWithResponseAsync( * @return a specific Security Solution along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String securitySolutionName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String ascLocation, + String securitySolutionName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -327,25 +279,17 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - securitySolutionName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, ascLocation, + securitySolutionName, apiVersion, accept, context); } /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -353,19 +297,19 @@ private Mono> getWithResponseAsync( * @return a specific Security Solution on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String ascLocation, String securitySolutionName) { + private Mono getAsync(String resourceGroupName, String ascLocation, + String securitySolutionName) { return getWithResponseAsync(resourceGroupName, ascLocation, securitySolutionName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -374,18 +318,18 @@ private Mono getAsync( * @return a specific Security Solution along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String ascLocation, String securitySolutionName, Context context) { + public Response getWithResponse(String resourceGroupName, String ascLocation, + String securitySolutionName, Context context) { return getWithResponseAsync(resourceGroupName, ascLocation, securitySolutionName, context).block(); } /** * Gets a specific Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param securitySolutionName Name of security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -399,9 +343,10 @@ public SecuritySolutionInner get(String resourceGroupName, String ascLocation, S /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -413,31 +358,22 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -450,23 +386,13 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsImpl.java index de0c90a619d34..d62f6a016422c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsImpl.java @@ -21,31 +21,28 @@ public final class SecuritySolutionsImpl implements SecuritySolutions { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecuritySolutionsImpl( - SecuritySolutionsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SecuritySolutionsImpl(SecuritySolutionsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecuritySolutionImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SecuritySolutionImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecuritySolutionImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String ascLocation, String securitySolutionName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, ascLocation, securitySolutionName, context); + public Response getWithResponse(String resourceGroupName, String ascLocation, + String securitySolutionName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, ascLocation, securitySolutionName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecuritySolutionImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDataListImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDataListImpl.java index 0e961ffe1342e..ba50dde128ad9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDataListImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDataListImpl.java @@ -15,8 +15,7 @@ public final class SecuritySolutionsReferenceDataListImpl implements SecuritySol private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecuritySolutionsReferenceDataListImpl( - SecuritySolutionsReferenceDataListInner innerObject, + SecuritySolutionsReferenceDataListImpl(SecuritySolutionsReferenceDataListInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasClientImpl.java index 55a07eae1bacf..ac2034e999fa9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasClientImpl.java @@ -25,116 +25,100 @@ import com.azure.resourcemanager.security.fluent.models.SecuritySolutionsReferenceDataListInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SecuritySolutionsReferenceDatasClient. */ +/** + * An instance of this class provides access to all the operations defined in SecuritySolutionsReferenceDatasClient. + */ public final class SecuritySolutionsReferenceDatasClientImpl implements SecuritySolutionsReferenceDatasClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SecuritySolutionsReferenceDatasService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SecuritySolutionsReferenceDatasClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SecuritySolutionsReferenceDatasClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SecuritySolutionsReferenceDatasService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SecuritySolutionsReferenceDatasService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterSecuritySolutionsReferenceDatas to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for SecurityCenterSecuritySolutionsReferenceDatas to be used by the + * proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterSecuri") public interface SecuritySolutionsReferenceDatasService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/securitySolutionsReferenceData") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/securitySolutionsReferenceData") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/securitySolutionsReferenceData") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByHomeRegion( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByHomeRegion(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all supported Security Solutions for the subscription along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all supported Security Solutions for the subscription along with {@link Response} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; @@ -144,7 +128,7 @@ private Mono> listWithResponse /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all supported Security Solutions for the subscription on successful completion of {@link Mono}. @@ -156,7 +140,7 @@ private Mono listAsync() { /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -170,7 +154,7 @@ public Response listWithResponse(Contex /** * Gets a list of all supported Security Solutions for the subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of all supported Security Solutions for the subscription. @@ -182,29 +166,25 @@ public SecuritySolutionsReferenceDataListInner list() { /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all supported Security Solutions for subscription and location along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionWithResponseAsync( - String ascLocation) { + private Mono> + listByHomeRegionWithResponseAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -212,45 +192,33 @@ private Mono> listByHomeRegion final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByHomeRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - apiVersion, - accept, - context)) + .withContext(context -> service.listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all supported Security Solutions for subscription and location along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionWithResponseAsync( - String ascLocation, Context context) { + private Mono> + listByHomeRegionWithResponseAsync(String ascLocation, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -258,21 +226,20 @@ private Mono> listByHomeRegion final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByHomeRegion( - this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, context); + return service.listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, + apiVersion, accept, context); } /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of all supported Security Solutions for subscription and location on successful completion of {@link - * Mono}. + * @return list of all supported Security Solutions for subscription and location on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono listByHomeRegionAsync(String ascLocation) { @@ -281,9 +248,9 @@ private Mono listByHomeRegionAsync(Stri /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -291,16 +258,16 @@ private Mono listByHomeRegionAsync(Stri * @return list of all supported Security Solutions for subscription and location along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listByHomeRegionWithResponse( - String ascLocation, Context context) { + public Response listByHomeRegionWithResponse(String ascLocation, + Context context) { return listByHomeRegionWithResponseAsync(ascLocation, context).block(); } /** * Gets list of all supported Security Solutions for subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasImpl.java index f3c696de64ae2..34d947eb2a2dd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySolutionsReferenceDatasImpl.java @@ -20,8 +20,7 @@ public final class SecuritySolutionsReferenceDatasImpl implements SecuritySoluti private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SecuritySolutionsReferenceDatasImpl( - SecuritySolutionsReferenceDatasClient innerClient, + public SecuritySolutionsReferenceDatasImpl(SecuritySolutionsReferenceDatasClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -30,10 +29,7 @@ public SecuritySolutionsReferenceDatasImpl( public Response listWithResponse(Context context) { Response inner = this.serviceClient().listWithResponse(context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecuritySolutionsReferenceDataListImpl(inner.getValue(), this.manager())); } else { return null; @@ -49,15 +45,12 @@ public SecuritySolutionsReferenceDataList list() { } } - public Response listByHomeRegionWithResponse( - String ascLocation, Context context) { - Response inner = - this.serviceClient().listByHomeRegionWithResponse(ascLocation, context); + public Response listByHomeRegionWithResponse(String ascLocation, + Context context) { + Response inner + = this.serviceClient().listByHomeRegionWithResponse(ascLocation, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecuritySolutionsReferenceDataListImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySubAssessmentImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySubAssessmentImpl.java index 3fb497b072b76..b3497d8c02b1c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySubAssessmentImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SecuritySubAssessmentImpl.java @@ -16,8 +16,8 @@ public final class SecuritySubAssessmentImpl implements SecuritySubAssessment { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - SecuritySubAssessmentImpl( - SecuritySubAssessmentInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + SecuritySubAssessmentImpl(SecuritySubAssessmentInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsClientImpl.java new file mode 100644 index 0000000000000..0a8579a566210 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsClientImpl.java @@ -0,0 +1,146 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.SensitivitySettingsClient; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsListResponseInner; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in SensitivitySettingsClient. + */ +public final class SensitivitySettingsClientImpl implements SensitivitySettingsClient { + /** + * The proxy service used to perform REST calls. + */ + private final SensitivitySettingsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of SensitivitySettingsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + SensitivitySettingsClientImpl(SecurityCenterImpl client) { + this.service = RestProxy.create(SensitivitySettingsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterSensitivitySettings to be used by the proxy service to + * perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterSensit") + public interface SensitivitySettingsService { + @Headers({ "Content-Type: application/json" }) + @Get("/providers/Microsoft.Security/sensitivitySettings") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + } + + /** + * Gets a list with a single sensitivity settings resource. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource along with {@link Response} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync() { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String apiVersion = "2023-02-15-preview"; + final String accept = "application/json"; + return FluxUtil.withContext(context -> service.list(this.client.getEndpoint(), apiVersion, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Gets a list with a single sensitivity settings resource. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource along with {@link Response} on successful completion + * of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listWithResponseAsync(Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String apiVersion = "2023-02-15-preview"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.list(this.client.getEndpoint(), apiVersion, accept, context); + } + + /** + * Gets a list with a single sensitivity settings resource. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono listAsync() { + return listWithResponseAsync().flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Gets a list with a single sensitivity settings resource. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response listWithResponse(Context context) { + return listWithResponseAsync(context).block(); + } + + /** + * Gets a list with a single sensitivity settings resource. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list with a single sensitivity settings resource. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public GetSensitivitySettingsListResponseInner list() { + return listWithResponse(Context.NONE).getValue(); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsImpl.java new file mode 100644 index 0000000000000..385d882247ae9 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SensitivitySettingsImpl.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.SensitivitySettingsClient; +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsListResponseInner; +import com.azure.resourcemanager.security.models.GetSensitivitySettingsListResponse; +import com.azure.resourcemanager.security.models.SensitivitySettings; + +public final class SensitivitySettingsImpl implements SensitivitySettings { + private static final ClientLogger LOGGER = new ClientLogger(SensitivitySettingsImpl.class); + + private final SensitivitySettingsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public SensitivitySettingsImpl(SensitivitySettingsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public Response listWithResponse(Context context) { + Response inner = this.serviceClient().listWithResponse(context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new GetSensitivitySettingsListResponseImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public GetSensitivitySettingsListResponse list() { + GetSensitivitySettingsListResponseInner inner = this.serviceClient().list(); + if (inner != null) { + return new GetSensitivitySettingsListResponseImpl(inner, this.manager()); + } else { + return null; + } + } + + private SensitivitySettingsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentImpl.java index f12420391d629..f3a42ff3a40a9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentImpl.java @@ -13,8 +13,7 @@ public final class ServerVulnerabilityAssessmentImpl implements ServerVulnerabil private final com.azure.resourcemanager.security.SecurityManager serviceManager; - ServerVulnerabilityAssessmentImpl( - ServerVulnerabilityAssessmentInner innerObject, + ServerVulnerabilityAssessmentImpl(ServerVulnerabilityAssessmentInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsClientImpl.java index b84a55fa6e48c..e27cdff5f0ec9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsClientImpl.java @@ -33,26 +33,28 @@ import reactor.core.publisher.Flux; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in ServerVulnerabilityAssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in ServerVulnerabilityAssessmentsClient. + */ public final class ServerVulnerabilityAssessmentsClientImpl implements ServerVulnerabilityAssessmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final ServerVulnerabilityAssessmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of ServerVulnerabilityAssessmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ ServerVulnerabilityAssessmentsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - ServerVulnerabilityAssessmentsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(ServerVulnerabilityAssessmentsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -63,102 +65,78 @@ public final class ServerVulnerabilityAssessmentsClientImpl implements ServerVul @Host("{$host}") @ServiceInterface(name = "SecurityCenterServer") public interface ServerVulnerabilityAssessmentsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByExtendedResource( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, + @HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, - @PathParam("resourceName") String resourceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments/{serverVulnerabilityAssessment}") - @ExpectedResponses({200}) + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, + @PathParam("resourceName") String resourceName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments/{serverVulnerabilityAssessment}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, @PathParam("resourceName") String resourceName, @PathParam("serverVulnerabilityAssessment") String serverVulnerabilityAssessment, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Put( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments/{serverVulnerabilityAssessment}") - @ExpectedResponses({202}) + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments/{serverVulnerabilityAssessment}") + @ExpectedResponses({ 202 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, + Mono> createOrUpdate(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, @PathParam("resourceName") String resourceName, @PathParam("serverVulnerabilityAssessment") String serverVulnerabilityAssessment, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); - - @Headers({"Content-Type: application/json"}) - @Delete( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments/{serverVulnerabilityAssessment}") - @ExpectedResponses({200, 202, 204}) + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/serverVulnerabilityAssessments/{serverVulnerabilityAssessment}") + @ExpectedResponses({ 200, 202, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono>> delete( - @HostParam("$host") String endpoint, + Mono>> delete(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, @PathParam("resourceName") String resourceName, @PathParam("serverVulnerabilityAssessment") String serverVulnerabilityAssessment, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); } /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server vulnerability assessment onboarding statuses on a given resource along with {@link - * Response} on successful completion of {@link Mono}. + * @return a list of server vulnerability assessment onboarding statuses on a given resource along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByExtendedResourceWithResponseAsync( String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -178,26 +156,16 @@ private Mono> listByExtendedRe final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listByExtendedResource( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - apiVersion, - accept, - context)) + context -> service.listByExtendedResource(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -205,23 +173,19 @@ private Mono> listByExtendedRe * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server vulnerability assessment onboarding statuses on a given resource along with {@link - * Response} on successful completion of {@link Mono}. + * @return a list of server vulnerability assessment onboarding statuses on a given resource along with + * {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByExtendedResourceWithResponseAsync( String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -240,24 +204,15 @@ private Mono> listByExtendedRe final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByExtendedResource( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - apiVersion, - accept, - context); + return service.listByExtendedResource(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, apiVersion, accept, context); } /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -265,20 +220,20 @@ private Mono> listByExtendedRe * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of server vulnerability assessment onboarding statuses on a given resource on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono listByExtendedResourceAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono listByExtendedResourceAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { return listByExtendedResourceWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -286,22 +241,21 @@ private Mono listByExtendedResourceAsyn * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of server vulnerability assessment onboarding statuses on a given resource along with {@link - * Response}. + * @return a list of server vulnerability assessment onboarding statuses on a given resource along with + * {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) public Response listByExtendedResourceWithResponse( String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - return listByExtendedResourceWithResponseAsync( - resourceGroupName, resourceNamespace, resourceType, resourceName, context) - .block(); + return listByExtendedResourceWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + context).block(); } /** * Gets a list of server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -311,18 +265,17 @@ public Response listByExtendedResourceW * @return a list of server vulnerability assessment onboarding statuses on a given resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerVulnerabilityAssessmentsListInner listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - return listByExtendedResourceWithResponse( - resourceGroupName, resourceNamespace, resourceType, resourceName, Context.NONE) - .getValue(); + public ServerVulnerabilityAssessmentsListInner listByExtendedResource(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { + return listByExtendedResourceWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, + Context.NONE).getValue(); } /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -330,22 +283,18 @@ public ServerVulnerabilityAssessmentsListInner listByExtendedResource( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a server vulnerability assessment onboarding statuses on a given resource along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono> getWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -365,28 +314,17 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - serverVulnerabilityAssessment, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, serverVulnerabilityAssessment, + apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -395,22 +333,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a server vulnerability assessment onboarding statuses on a given resource along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -430,25 +364,15 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - serverVulnerabilityAssessment, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, serverVulnerabilityAssessment, apiVersion, accept, context); } /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -456,20 +380,20 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a server vulnerability assessment onboarding statuses on a given resource on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono getAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { return getWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -480,16 +404,16 @@ private Mono getAsync( * @return a server vulnerability assessment onboarding statuses on a given resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + public Response getWithResponse(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { return getWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context).block(); } /** * Gets a server vulnerability assessment onboarding statuses on a given resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -499,8 +423,8 @@ public Response getWithResponse( * @return a server vulnerability assessment onboarding statuses on a given resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerVulnerabilityAssessmentInner get( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + public ServerVulnerabilityAssessmentInner get(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { return getWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, Context.NONE) .getValue(); } @@ -508,9 +432,9 @@ public ServerVulnerabilityAssessmentInner get( /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -518,22 +442,18 @@ public ServerVulnerabilityAssessmentInner get( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes the server vulnerability assessment details on a resource along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -553,29 +473,18 @@ private Mono> createOrUpdateWithRes final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - serverVulnerabilityAssessment, - apiVersion, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, serverVulnerabilityAssessment, + apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -584,22 +493,18 @@ private Mono> createOrUpdateWithRes * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes the server vulnerability assessment details on a resource along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + private Mono> createOrUpdateWithResponseAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -619,38 +524,28 @@ private Mono> createOrUpdateWithRes final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - serverVulnerabilityAssessment, - apiVersion, - accept, - context); + return service.createOrUpdate(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, serverVulnerabilityAssessment, apiVersion, accept, context); } /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return describes the server vulnerability assessment details on a resource on successful completion of {@link - * Mono}. + * @return describes the server vulnerability assessment details on a resource on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createOrUpdateAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono createOrUpdateAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { return createOrUpdateWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } @@ -658,9 +553,9 @@ private Mono createOrUpdateAsync( /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -671,19 +566,18 @@ private Mono createOrUpdateAsync( * @return describes the server vulnerability assessment details on a resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - return createOrUpdateWithResponseAsync( - resourceGroupName, resourceNamespace, resourceType, resourceName, context) - .block(); + public Response createOrUpdateWithResponse(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { + return createOrUpdateWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, + context).block(); } /** * Creating a server vulnerability assessment on a resource, which will onboard a resource for having a * vulnerability assessment on it. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -693,18 +587,17 @@ public Response createOrUpdateWithResponse( * @return describes the server vulnerability assessment details on a resource. */ @ServiceMethod(returns = ReturnType.SINGLE) - public ServerVulnerabilityAssessmentInner createOrUpdate( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - return createOrUpdateWithResponse( - resourceGroupName, resourceNamespace, resourceType, resourceName, Context.NONE) - .getValue(); + public ServerVulnerabilityAssessmentInner createOrUpdate(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { + return createOrUpdateWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, + Context.NONE).getValue(); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -714,19 +607,15 @@ public ServerVulnerabilityAssessmentInner createOrUpdate( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -746,28 +635,17 @@ private Mono>> deleteWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - serverVulnerabilityAssessment, - apiVersion, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, serverVulnerabilityAssessment, + apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -778,19 +656,15 @@ private Mono>> deleteWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono>> deleteWithResponseAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + private Mono>> deleteWithResponseAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -810,25 +684,15 @@ private Mono>> deleteWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - serverVulnerabilityAssessment, - apiVersion, - accept, - context); + return service.delete(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, serverVulnerabilityAssessment, apiVersion, accept, context); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -838,21 +702,19 @@ private Mono>> deleteWithResponseAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName); - return this - .client - .getLroResult( - mono, this.client.getHttpPipeline(), Void.class, Void.class, this.client.getContext()); + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + this.client.getContext()); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -863,21 +725,20 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link PollerFlux} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - private PollerFlux, Void> beginDeleteAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + private PollerFlux, Void> beginDeleteAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { context = this.client.mergeContext(context); - Mono>> mono = - deleteWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context); - return this - .client - .getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, context); + Mono>> mono + = deleteWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context); + return this.client.getLroResult(mono, this.client.getHttpPipeline(), Void.class, Void.class, + context); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -887,16 +748,16 @@ private PollerFlux, Void> beginDeleteAsync( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + public SyncPoller, Void> beginDelete(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { return this.beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName).getSyncPoller(); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -907,18 +768,17 @@ public SyncPoller, Void> beginDelete( * @return the {@link SyncPoller} for polling of long-running operation. */ @ServiceMethod(returns = ReturnType.LONG_RUNNING_OPERATION) - public SyncPoller, Void> beginDelete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - return this - .beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context) + public SyncPoller, Void> beginDelete(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { + return this.beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context) .getSyncPoller(); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -928,18 +788,17 @@ public SyncPoller, Void> beginDelete( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - return beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName) - .last() + private Mono deleteAsync(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName) { + return beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -950,18 +809,17 @@ private Mono deleteAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono deleteAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - return beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context) - .last() + private Mono deleteAsync(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, Context context) { + return beginDeleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context).last() .flatMap(this.client::getLroFinalResultOrError); } /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -976,9 +834,9 @@ public void delete(String resourceGroupName, String resourceNamespace, String re /** * Removing server vulnerability assessment from a resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -988,8 +846,8 @@ public void delete(String resourceGroupName, String resourceNamespace, String re * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void delete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + public void delete(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + Context context) { deleteAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context).block(); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsImpl.java index 2538bf7d5ba37..b2ef7622b94e6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsImpl.java @@ -22,37 +22,29 @@ public final class ServerVulnerabilityAssessmentsImpl implements ServerVulnerabi private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public ServerVulnerabilityAssessmentsImpl( - ServerVulnerabilityAssessmentsClient innerClient, + public ServerVulnerabilityAssessmentsImpl(ServerVulnerabilityAssessmentsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response listByExtendedResourceWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - Response inner = - this - .serviceClient() - .listByExtendedResourceWithResponse( - resourceGroupName, resourceNamespace, resourceType, resourceName, context); + public Response listByExtendedResourceWithResponse(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { + Response inner + = this.serviceClient().listByExtendedResourceWithResponse(resourceGroupName, resourceNamespace, + resourceType, resourceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ServerVulnerabilityAssessmentsListImpl(inner.getValue(), this.manager())); } else { return null; } } - public ServerVulnerabilityAssessmentsList listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - ServerVulnerabilityAssessmentsListInner inner = - this - .serviceClient() - .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName); + public ServerVulnerabilityAssessmentsList listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { + ServerVulnerabilityAssessmentsListInner inner = this.serviceClient().listByExtendedResource(resourceGroupName, + resourceNamespace, resourceType, resourceName); if (inner != null) { return new ServerVulnerabilityAssessmentsListImpl(inner, this.manager()); } else { @@ -60,27 +52,22 @@ public ServerVulnerabilityAssessmentsList listByExtendedResource( } } - public Response getWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - Response inner = - this - .serviceClient() - .getWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, context); + public Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, + resourceNamespace, resourceType, resourceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ServerVulnerabilityAssessmentImpl(inner.getValue(), this.manager())); } else { return null; } } - public ServerVulnerabilityAssessment get( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - ServerVulnerabilityAssessmentInner inner = - this.serviceClient().get(resourceGroupName, resourceNamespace, resourceType, resourceName); + public ServerVulnerabilityAssessment get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName) { + ServerVulnerabilityAssessmentInner inner + = this.serviceClient().get(resourceGroupName, resourceNamespace, resourceType, resourceName); if (inner != null) { return new ServerVulnerabilityAssessmentImpl(inner, this.manager()); } else { @@ -88,27 +75,22 @@ public ServerVulnerabilityAssessment get( } } - public Response createOrUpdateWithResponse( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - Response inner = - this - .serviceClient() - .createOrUpdateWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, context); + public Response createOrUpdateWithResponse(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { + Response inner = this.serviceClient() + .createOrUpdateWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ServerVulnerabilityAssessmentImpl(inner.getValue(), this.manager())); } else { return null; } } - public ServerVulnerabilityAssessment createOrUpdate( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - ServerVulnerabilityAssessmentInner inner = - this.serviceClient().createOrUpdate(resourceGroupName, resourceNamespace, resourceType, resourceName); + public ServerVulnerabilityAssessment createOrUpdate(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { + ServerVulnerabilityAssessmentInner inner + = this.serviceClient().createOrUpdate(resourceGroupName, resourceNamespace, resourceType, resourceName); if (inner != null) { return new ServerVulnerabilityAssessmentImpl(inner, this.manager()); } else { @@ -120,8 +102,8 @@ public void delete(String resourceGroupName, String resourceNamespace, String re this.serviceClient().delete(resourceGroupName, resourceNamespace, resourceType, resourceName); } - public void delete( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + public void delete(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + Context context) { this.serviceClient().delete(resourceGroupName, resourceNamespace, resourceType, resourceName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsListImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsListImpl.java index cc84ed836c77d..df3243ae40350 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsListImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsListImpl.java @@ -17,8 +17,7 @@ public final class ServerVulnerabilityAssessmentsListImpl implements ServerVulne private final com.azure.resourcemanager.security.SecurityManager serviceManager; - ServerVulnerabilityAssessmentsListImpl( - ServerVulnerabilityAssessmentsListInner innerObject, + ServerVulnerabilityAssessmentsListImpl(ServerVulnerabilityAssessmentsListInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; @@ -27,12 +26,9 @@ public final class ServerVulnerabilityAssessmentsListImpl implements ServerVulne public List value() { List inner = this.innerModel().value(); if (inner != null) { - return Collections - .unmodifiableList( - inner - .stream() - .map(inner1 -> new ServerVulnerabilityAssessmentImpl(inner1, this.manager())) - .collect(Collectors.toList())); + return Collections.unmodifiableList( + inner.stream().map(inner1 -> new ServerVulnerabilityAssessmentImpl(inner1, this.manager())) + .collect(Collectors.toList())); } else { return Collections.emptyList(); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingImpl.java new file mode 100644 index 0000000000000..9d1a9e1f831ef --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingImpl.java @@ -0,0 +1,45 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsSettingInner; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSetting; + +public final class ServerVulnerabilityAssessmentsSettingImpl implements ServerVulnerabilityAssessmentsSetting { + private ServerVulnerabilityAssessmentsSettingInner innerObject; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + ServerVulnerabilityAssessmentsSettingImpl(ServerVulnerabilityAssessmentsSettingInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerObject = innerObject; + this.serviceManager = serviceManager; + } + + public String id() { + return this.innerModel().id(); + } + + public String name() { + return this.innerModel().name(); + } + + public String type() { + return this.innerModel().type(); + } + + public SystemData systemData() { + return this.innerModel().systemData(); + } + + public ServerVulnerabilityAssessmentsSettingInner innerModel() { + return this.innerObject; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsClientImpl.java new file mode 100644 index 0000000000000..d8fa6ee01d088 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsClientImpl.java @@ -0,0 +1,641 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.annotation.BodyParam; +import com.azure.core.annotation.Delete; +import com.azure.core.annotation.ExpectedResponses; +import com.azure.core.annotation.Get; +import com.azure.core.annotation.HeaderParam; +import com.azure.core.annotation.Headers; +import com.azure.core.annotation.Host; +import com.azure.core.annotation.HostParam; +import com.azure.core.annotation.PathParam; +import com.azure.core.annotation.Put; +import com.azure.core.annotation.QueryParam; +import com.azure.core.annotation.ReturnType; +import com.azure.core.annotation.ServiceInterface; +import com.azure.core.annotation.ServiceMethod; +import com.azure.core.annotation.UnexpectedResponseExceptionType; +import com.azure.core.http.rest.PagedFlux; +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.PagedResponse; +import com.azure.core.http.rest.PagedResponseBase; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.RestProxy; +import com.azure.core.management.exception.ManagementException; +import com.azure.core.util.Context; +import com.azure.core.util.FluxUtil; +import com.azure.resourcemanager.security.fluent.ServerVulnerabilityAssessmentsSettingsClient; +import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsSettingInner; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingKindName; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingsList; +import reactor.core.publisher.Mono; + +/** + * An instance of this class provides access to all the operations defined in + * ServerVulnerabilityAssessmentsSettingsClient. + */ +public final class ServerVulnerabilityAssessmentsSettingsClientImpl + implements ServerVulnerabilityAssessmentsSettingsClient { + /** + * The proxy service used to perform REST calls. + */ + private final ServerVulnerabilityAssessmentsSettingsService service; + + /** + * The service client containing this operation class. + */ + private final SecurityCenterImpl client; + + /** + * Initializes an instance of ServerVulnerabilityAssessmentsSettingsClientImpl. + * + * @param client the instance of the service client containing this operation class. + */ + ServerVulnerabilityAssessmentsSettingsClientImpl(SecurityCenterImpl client) { + this.service = RestProxy.create(ServerVulnerabilityAssessmentsSettingsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); + this.client = client; + } + + /** + * The interface defining all the services for SecurityCenterServerVulnerabilityAssessmentsSettings to be used by + * the proxy service to perform REST calls. + */ + @Host("{$host}") + @ServiceInterface(name = "SecurityCenterServer") + public interface ServerVulnerabilityAssessmentsSettingsService { + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/serverVulnerabilityAssessmentsSettings") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/serverVulnerabilityAssessmentsSettings/{settingKind}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("settingKind") ServerVulnerabilityAssessmentsSettingKindName settingKind, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/serverVulnerabilityAssessmentsSettings/{settingKind}") + @ExpectedResponses({ 200, 201 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("settingKind") ServerVulnerabilityAssessmentsSettingKindName settingKind, + @BodyParam("application/json") ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/serverVulnerabilityAssessmentsSettings/{settingKind}") + @ExpectedResponses({ 200, 204 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, + @PathParam("subscriptionId") String subscriptionId, + @PathParam("settingKind") ServerVulnerabilityAssessmentsSettingKindName settingKind, + @HeaderParam("Accept") String accept, Context context); + + @Headers({ "Content-Type: application/json" }) + @Get("{nextLink}") + @ExpectedResponses({ 200 }) + @UnexpectedResponseExceptionType(ManagementException.class) + Mono> listBySubscriptionNext( + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); + } + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync() { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map( + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope along with + * {@link PagedResponse} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> listSinglePageAsync(Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope as paginated + * response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync() { + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + } + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope as paginated + * response with {@link PagedFlux}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + private PagedFlux listAsync(Context context) { + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + } + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list() { + return new PagedIterable<>(listAsync()); + } + + /** + * Get a list of all the server vulnerability assessments settings over a subscription level scope. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of all the server vulnerability assessments settings over a subscription level scope as paginated + * response with {@link PagedIterable}. + */ + @ServiceMethod(returns = ReturnType.COLLECTION) + public PagedIterable list(Context context) { + return new PagedIterable<>(listAsync(context)); + } + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription along + * with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + getWithResponseAsync(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (settingKind == null) { + return Mono.error(new IllegalArgumentException("Parameter settingKind is required and cannot be null.")); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + settingKind, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription along + * with {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + getWithResponseAsync(ServerVulnerabilityAssessmentsSettingKindName settingKind, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (settingKind == null) { + return Mono.error(new IllegalArgumentException("Parameter settingKind is required and cannot be null.")); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingKind, accept, + context); + } + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono + getAsync(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + return getWithResponseAsync(settingKind).flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription along + * with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response + getWithResponse(ServerVulnerabilityAssessmentsSettingKindName settingKind, Context context) { + return getWithResponseAsync(settingKind, context).block(); + } + + /** + * Get a server vulnerability assessments setting of the requested kind, that is set on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a server vulnerability assessments setting of the requested kind, that is set on the subscription. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ServerVulnerabilityAssessmentsSettingInner get(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + return getWithResponse(settingKind, Context.NONE).getValue(); + } + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (settingKind == null) { + return Mono.error(new IllegalArgumentException("Parameter settingKind is required and cannot be null.")); + } + if (serverVulnerabilityAssessmentsSetting == null) { + return Mono.error(new IllegalArgumentException( + "Parameter serverVulnerabilityAssessmentsSetting is required and cannot be null.")); + } else { + serverVulnerabilityAssessmentsSetting.validate(); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), settingKind, serverVulnerabilityAssessmentsSetting, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope along with {@link Response} on + * successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> createOrUpdateWithResponseAsync( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting, Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (settingKind == null) { + return Mono.error(new IllegalArgumentException("Parameter settingKind is required and cannot be null.")); + } + if (serverVulnerabilityAssessmentsSetting == null) { + return Mono.error(new IllegalArgumentException( + "Parameter serverVulnerabilityAssessmentsSetting is required and cannot be null.")); + } else { + serverVulnerabilityAssessmentsSetting.validate(); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.createOrUpdate(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + settingKind, serverVulnerabilityAssessmentsSetting, accept, context); + } + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope on successful completion of + * {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono createOrUpdateAsync( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting) { + return createOrUpdateWithResponseAsync(settingKind, serverVulnerabilityAssessmentsSetting) + .flatMap(res -> Mono.justOrEmpty(res.getValue())); + } + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope along with {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response createOrUpdateWithResponse( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting, Context context) { + return createOrUpdateWithResponseAsync(settingKind, serverVulnerabilityAssessmentsSetting, context).block(); + } + + /** + * Create or update a server vulnerability assessments setting of the requested kind on the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param serverVulnerabilityAssessmentsSetting A server vulnerability assessments setting over a predefined scope. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a base vulnerability assessments setting on servers in the defined scope. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public ServerVulnerabilityAssessmentsSettingInner createOrUpdate( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting) { + return createOrUpdateWithResponse(settingKind, serverVulnerabilityAssessmentsSetting, Context.NONE).getValue(); + } + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (settingKind == null) { + return Mono.error(new IllegalArgumentException("Parameter settingKind is required and cannot be null.")); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + return FluxUtil + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), settingKind, accept, context)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response} on successful completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> deleteWithResponseAsync(ServerVulnerabilityAssessmentsSettingKindName settingKind, + Context context) { + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + if (this.client.getSubscriptionId() == null) { + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + } + if (settingKind == null) { + return Mono.error(new IllegalArgumentException("Parameter settingKind is required and cannot be null.")); + } + final String apiVersion = "2023-05-01"; + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingKind, + accept, context); + } + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return A {@link Mono} that completes when a successful response is received. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono deleteAsync(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + return deleteWithResponseAsync(settingKind).flatMap(ignored -> Mono.empty()); + } + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the {@link Response}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public Response deleteWithResponse(ServerVulnerabilityAssessmentsSettingKindName settingKind, + Context context) { + return deleteWithResponseAsync(settingKind, context).block(); + } + + /** + * Delete the server vulnerability assessments setting of the requested kind from the subscription. + * + * @param settingKind The kind of the server vulnerability assessments setting. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + public void delete(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + deleteWithResponse(settingKind, Context.NONE); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a page of a server vulnerability assessments settings list along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + return FluxUtil + .withContext( + context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map( + res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)) + .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); + } + + /** + * Get the next page of items. + * + * @param nextLink The URL to get the next list of items + * + * The nextLink parameter. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a page of a server vulnerability assessments settings list along with {@link PagedResponse} on successful + * completion of {@link Mono}. + */ + @ServiceMethod(returns = ReturnType.SINGLE) + private Mono> + listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { + if (nextLink == null) { + return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); + } + if (this.client.getEndpoint() == null) { + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); + } + final String accept = "application/json"; + context = this.client.mergeContext(context); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsImpl.java new file mode 100644 index 0000000000000..674bdfe3be355 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/ServerVulnerabilityAssessmentsSettingsImpl.java @@ -0,0 +1,105 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.implementation; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.util.Context; +import com.azure.core.util.logging.ClientLogger; +import com.azure.resourcemanager.security.fluent.ServerVulnerabilityAssessmentsSettingsClient; +import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsSettingInner; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSetting; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettingKindName; +import com.azure.resourcemanager.security.models.ServerVulnerabilityAssessmentsSettings; + +public final class ServerVulnerabilityAssessmentsSettingsImpl implements ServerVulnerabilityAssessmentsSettings { + private static final ClientLogger LOGGER = new ClientLogger(ServerVulnerabilityAssessmentsSettingsImpl.class); + + private final ServerVulnerabilityAssessmentsSettingsClient innerClient; + + private final com.azure.resourcemanager.security.SecurityManager serviceManager; + + public ServerVulnerabilityAssessmentsSettingsImpl(ServerVulnerabilityAssessmentsSettingsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { + this.innerClient = innerClient; + this.serviceManager = serviceManager; + } + + public PagedIterable list() { + PagedIterable inner = this.serviceClient().list(); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new ServerVulnerabilityAssessmentsSettingImpl(inner1, this.manager())); + } + + public PagedIterable list(Context context) { + PagedIterable inner = this.serviceClient().list(context); + return ResourceManagerUtils.mapPage(inner, + inner1 -> new ServerVulnerabilityAssessmentsSettingImpl(inner1, this.manager())); + } + + public Response + getWithResponse(ServerVulnerabilityAssessmentsSettingKindName settingKind, Context context) { + Response inner + = this.serviceClient().getWithResponse(settingKind, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ServerVulnerabilityAssessmentsSettingImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ServerVulnerabilityAssessmentsSetting get(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + ServerVulnerabilityAssessmentsSettingInner inner = this.serviceClient().get(settingKind); + if (inner != null) { + return new ServerVulnerabilityAssessmentsSettingImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response createOrUpdateWithResponse( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting, Context context) { + Response inner = this.serviceClient() + .createOrUpdateWithResponse(settingKind, serverVulnerabilityAssessmentsSetting, context); + if (inner != null) { + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), + new ServerVulnerabilityAssessmentsSettingImpl(inner.getValue(), this.manager())); + } else { + return null; + } + } + + public ServerVulnerabilityAssessmentsSetting createOrUpdate( + ServerVulnerabilityAssessmentsSettingKindName settingKind, + ServerVulnerabilityAssessmentsSettingInner serverVulnerabilityAssessmentsSetting) { + ServerVulnerabilityAssessmentsSettingInner inner + = this.serviceClient().createOrUpdate(settingKind, serverVulnerabilityAssessmentsSetting); + if (inner != null) { + return new ServerVulnerabilityAssessmentsSettingImpl(inner, this.manager()); + } else { + return null; + } + } + + public Response deleteWithResponse(ServerVulnerabilityAssessmentsSettingKindName settingKind, + Context context) { + return this.serviceClient().deleteWithResponse(settingKind, context); + } + + public void delete(ServerVulnerabilityAssessmentsSettingKindName settingKind) { + this.serviceClient().delete(settingKind); + } + + private ServerVulnerabilityAssessmentsSettingsClient serviceClient() { + return this.innerClient; + } + + private com.azure.resourcemanager.security.SecurityManager manager() { + return this.serviceManager; + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsClientImpl.java index 80ec076b1e331..30ffbfa19d58d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsClientImpl.java @@ -29,21 +29,27 @@ import com.azure.core.util.FluxUtil; import com.azure.resourcemanager.security.fluent.SettingsClient; import com.azure.resourcemanager.security.fluent.models.SettingInner; -import com.azure.resourcemanager.security.models.SettingName; +import com.azure.resourcemanager.security.models.SettingNameAutoGenerated; import com.azure.resourcemanager.security.models.SettingsList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SettingsClient. */ +/** + * An instance of this class provides access to all the operations defined in SettingsClient. + */ public final class SettingsClientImpl implements SettingsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SettingsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SettingsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SettingsClientImpl(SecurityCenterImpl client) { @@ -58,56 +64,43 @@ public final class SettingsClientImpl implements SettingsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterSettin") public interface SettingsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/settings") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/settings/{settingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("settingName") SettingName settingName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("settingName") SettingNameAutoGenerated settingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/settings/{settingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("settingName") SettingName settingName, - @BodyParam("application/json") SettingInner setting, - @HeaderParam("Accept") String accept, - Context context); + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("settingName") SettingNameAutoGenerated settingName, + @BodyParam("application/json") SettingInner setting, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return subscription settings list along with {@link PagedResponse} on successful completion of {@link Mono}. @@ -115,39 +108,26 @@ Mono> listNext( @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2022-05-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -157,36 +137,24 @@ private Mono> listSinglePageAsync() { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2022-05-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return subscription settings list as paginated response with {@link PagedFlux}. @@ -198,7 +166,7 @@ private PagedFlux listAsync() { /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -207,13 +175,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return subscription settings list as paginated response with {@link PagedIterable}. @@ -225,7 +193,7 @@ public PagedIterable list() { /** * Settings about different configurations in Microsoft Defender for Cloud. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -239,7 +207,7 @@ public PagedIterable list(Context context) { /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -247,18 +215,14 @@ public PagedIterable list(Context context) { * @return the kind of the security setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(SettingName settingName) { + private Mono> getWithResponseAsync(SettingNameAutoGenerated settingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -266,22 +230,14 @@ private Mono> getWithResponseAsync(SettingName settingNam final String apiVersion = "2022-05-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - settingName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + settingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -290,18 +246,14 @@ private Mono> getWithResponseAsync(SettingName settingNam * @return the kind of the security setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync(SettingName settingName, Context context) { + private Mono> getWithResponseAsync(SettingNameAutoGenerated settingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -309,13 +261,13 @@ private Mono> getWithResponseAsync(SettingName settingNam final String apiVersion = "2022-05-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingName, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingName, accept, + context); } /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -323,13 +275,13 @@ private Mono> getWithResponseAsync(SettingName settingNam * @return the kind of the security setting on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync(SettingName settingName) { + private Mono getAsync(SettingNameAutoGenerated settingName) { return getWithResponseAsync(settingName).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -338,13 +290,13 @@ private Mono getAsync(SettingName settingName) { * @return the kind of the security setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse(SettingName settingName, Context context) { + public Response getWithResponse(SettingNameAutoGenerated settingName, Context context) { return getWithResponseAsync(settingName, context).block(); } /** * Settings of different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -352,13 +304,13 @@ public Response getWithResponse(SettingName settingName, Context c * @return the kind of the security setting. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SettingInner get(SettingName settingName) { + public SettingInner get(SettingNameAutoGenerated settingName) { return getWithResponse(settingName, Context.NONE).getValue(); } /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -367,18 +319,15 @@ public SettingInner get(SettingName settingName) { * @return the kind of the security setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync(SettingName settingName, SettingInner setting) { + private Mono> updateWithResponseAsync(SettingNameAutoGenerated settingName, + SettingInner setting) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -391,23 +340,14 @@ private Mono> updateWithResponseAsync(SettingName setting final String apiVersion = "2022-05-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - settingName, - setting, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), settingName, setting, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @param context The context to associate with this operation. @@ -417,19 +357,15 @@ private Mono> updateWithResponseAsync(SettingName setting * @return the kind of the security setting along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - SettingName settingName, SettingInner setting, Context context) { + private Mono> updateWithResponseAsync(SettingNameAutoGenerated settingName, + SettingInner setting, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (settingName == null) { return Mono.error(new IllegalArgumentException("Parameter settingName is required and cannot be null.")); @@ -442,20 +378,13 @@ private Mono> updateWithResponseAsync( final String apiVersion = "2022-05-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - settingName, - setting, - accept, - context); + return service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), settingName, + setting, accept, context); } /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -464,13 +393,13 @@ private Mono> updateWithResponseAsync( * @return the kind of the security setting on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync(SettingName settingName, SettingInner setting) { + private Mono updateAsync(SettingNameAutoGenerated settingName, SettingInner setting) { return updateWithResponseAsync(settingName, setting).flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @param context The context to associate with this operation. @@ -480,13 +409,14 @@ private Mono updateAsync(SettingName settingName, SettingInner set * @return the kind of the security setting along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse(SettingName settingName, SettingInner setting, Context context) { + public Response updateWithResponse(SettingNameAutoGenerated settingName, SettingInner setting, + Context context) { return updateWithResponseAsync(settingName, setting, context).block(); } /** * updating settings about different configurations in Microsoft Defender for Cloud. - * + * * @param settingName The name of the setting. * @param setting Setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -495,15 +425,16 @@ public Response updateWithResponse(SettingName settingName, Settin * @return the kind of the security setting. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SettingInner update(SettingName settingName, SettingInner setting) { + public SettingInner update(SettingNameAutoGenerated settingName, SettingInner setting) { return updateWithResponse(settingName, setting, Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -515,31 +446,22 @@ private Mono> listNextSinglePageAsync(String nextLin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -552,23 +474,13 @@ private Mono> listNextSinglePageAsync(String nextLin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsImpl.java index 4c370425ade59..41288aa70b220 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SettingsImpl.java @@ -12,7 +12,7 @@ import com.azure.resourcemanager.security.fluent.SettingsClient; import com.azure.resourcemanager.security.fluent.models.SettingInner; import com.azure.resourcemanager.security.models.Setting; -import com.azure.resourcemanager.security.models.SettingName; +import com.azure.resourcemanager.security.models.SettingNameAutoGenerated; import com.azure.resourcemanager.security.models.Settings; public final class SettingsImpl implements Settings { @@ -29,28 +29,25 @@ public SettingsImpl(SettingsClient innerClient, com.azure.resourcemanager.securi public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SettingImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SettingImpl(inner1, this.manager())); } - public Response getWithResponse(SettingName settingName, Context context) { + public Response getWithResponse(SettingNameAutoGenerated settingName, Context context) { Response inner = this.serviceClient().getWithResponse(settingName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SettingImpl(inner.getValue(), this.manager())); } else { return null; } } - public Setting get(SettingName settingName) { + public Setting get(SettingNameAutoGenerated settingName) { SettingInner inner = this.serviceClient().get(settingName); if (inner != null) { return new SettingImpl(inner, this.manager()); @@ -59,20 +56,18 @@ public Setting get(SettingName settingName) { } } - public Response updateWithResponse(SettingName settingName, SettingInner setting, Context context) { + public Response updateWithResponse(SettingNameAutoGenerated settingName, SettingInner setting, + Context context) { Response inner = this.serviceClient().updateWithResponse(settingName, setting, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SettingImpl(inner.getValue(), this.manager())); } else { return null; } } - public Setting update(SettingName settingName, SettingInner setting) { + public Setting update(SettingNameAutoGenerated settingName, SettingInner setting) { SettingInner inner = this.serviceClient().update(settingName, setting); if (inner != null) { return new SettingImpl(inner, this.manager()); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesClientImpl.java index 9136222bcbf7f..562260a456154 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.SoftwaresList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SoftwareInventoriesClient. */ +/** + * An instance of this class provides access to all the operations defined in SoftwareInventoriesClient. + */ public final class SoftwareInventoriesClientImpl implements SoftwareInventoriesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SoftwareInventoriesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SoftwareInventoriesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SoftwareInventoriesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SoftwareInventoriesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service = RestProxy.create(SoftwareInventoriesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -56,76 +62,58 @@ public final class SoftwareInventoriesClientImpl implements SoftwareInventoriesC @Host("{$host}") @ServiceInterface(name = "SecurityCenterSoftwa") public interface SoftwareInventoriesService { - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/softwareInventories") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/softwareInventories") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByExtendedResource( - @HostParam("$host") String endpoint, + Mono> listByExtendedResource(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, - @PathParam("resourceName") String resourceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, + @PathParam("resourceName") String resourceName, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/softwareInventories") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/softwareInventories/{softwareName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/{resourceNamespace}/{resourceType}/{resourceName}/providers/Microsoft.Security/softwareInventories/{softwareName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("resourceNamespace") String resourceNamespace, - @PathParam("resourceType") String resourceType, - @PathParam("resourceName") String resourceName, - @PathParam("softwareName") String softwareName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @PathParam("resourceNamespace") String resourceNamespace, @PathParam("resourceType") String resourceType, + @PathParam("resourceName") String resourceName, @PathParam("softwareName") String softwareName, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByExtendedResourceNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listBySubscriptionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -133,22 +121,18 @@ Mono> listBySubscriptionNext( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the software inventory of the virtual machine along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByExtendedResourceSinglePageAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + private Mono> listByExtendedResourceSinglePageAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -168,35 +152,18 @@ private Mono> listByExtendedResourceSinglePageAsync final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .listByExtendedResource( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + context -> service.listByExtendedResource(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, resourceNamespace, resourceType, resourceName, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -205,22 +172,18 @@ private Mono> listByExtendedResourceSinglePageAsync * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the software inventory of the virtual machine along with {@link PagedResponse} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByExtendedResourceSinglePageAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + private Mono> listByExtendedResourceSinglePageAsync(String resourceGroupName, + String resourceNamespace, String resourceType, String resourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -240,32 +203,17 @@ private Mono> listByExtendedResourceSinglePageAsync final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByExtendedResource( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - apiVersion, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByExtendedResource(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -275,19 +223,17 @@ private Mono> listByExtendedResourceSinglePageAsync * @return the software inventory of the virtual machine as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByExtendedResourceAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - return new PagedFlux<>( - () -> - listByExtendedResourceSinglePageAsync(resourceGroupName, resourceNamespace, resourceType, resourceName), - nextLink -> listByExtendedResourceNextSinglePageAsync(nextLink)); + private PagedFlux listByExtendedResourceAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { + return new PagedFlux<>(() -> listByExtendedResourceSinglePageAsync(resourceGroupName, resourceNamespace, + resourceType, resourceName), nextLink -> listByExtendedResourceNextSinglePageAsync(nextLink)); } /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -298,20 +244,18 @@ private PagedFlux listByExtendedResourceAsync( * @return the software inventory of the virtual machine as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByExtendedResourceAsync( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - return new PagedFlux<>( - () -> - listByExtendedResourceSinglePageAsync( - resourceGroupName, resourceNamespace, resourceType, resourceName, context), + private PagedFlux listByExtendedResourceAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { + return new PagedFlux<>(() -> listByExtendedResourceSinglePageAsync(resourceGroupName, resourceNamespace, + resourceType, resourceName, context), nextLink -> listByExtendedResourceNextSinglePageAsync(nextLink, context)); } /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -321,17 +265,17 @@ private PagedFlux listByExtendedResourceAsync( * @return the software inventory of the virtual machine as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { + public PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { return new PagedIterable<>( listByExtendedResourceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName)); } /** * Gets the software inventory of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -342,130 +286,105 @@ public PagedIterable listByExtendedResource( * @return the software inventory of the virtual machine as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { + public PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { return new PagedIterable<>( listByExtendedResourceAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, context)); } /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the software inventory of all virtual machines in the subscriptions along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-05-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the software inventory of all virtual machines in the subscriptions along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2021-05-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the software inventory of all virtual machines in the subscriptions as paginated response with {@link - * PagedFlux}. + * @return the software inventory of all virtual machines in the subscriptions as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { - return new PagedFlux<>( - () -> listSinglePageAsync(), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink)); } /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the software inventory of all virtual machines in the subscriptions as paginated response with {@link - * PagedFlux}. + * @return the software inventory of all virtual machines in the subscriptions as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listBySubscriptionNextSinglePageAsync(nextLink, context)); } /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the software inventory of all virtual machines in the subscriptions as paginated response with {@link - * PagedIterable}. + * @return the software inventory of all virtual machines in the subscriptions as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -474,13 +393,13 @@ public PagedIterable list() { /** * Gets the software inventory of all virtual machines in the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the software inventory of all virtual machines in the subscriptions as paginated response with {@link - * PagedIterable}. + * @return the software inventory of all virtual machines in the subscriptions as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -489,9 +408,9 @@ public PagedIterable list(Context context) { /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -500,26 +419,18 @@ public PagedIterable list(Context context) { * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single software data of the virtual machine along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName) { + private Mono> getWithResponseAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String softwareName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -542,27 +453,16 @@ private Mono> getWithResponseAsync( final String accept = "application/json"; return FluxUtil .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - softwareName, - apiVersion, - accept, - context)) + context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, softwareName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -572,27 +472,18 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single software data of the virtual machine along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName, - Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String softwareName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -614,25 +505,15 @@ private Mono> getWithResponseAsync( final String apiVersion = "2021-05-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - resourceNamespace, - resourceType, - resourceName, - softwareName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, + resourceNamespace, resourceType, resourceName, softwareName, apiVersion, accept, context); } /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -643,21 +524,17 @@ private Mono> getWithResponseAsync( * @return a single software data of the virtual machine on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName) { + private Mono getAsync(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String softwareName) { return getWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -669,23 +546,17 @@ private Mono getAsync( * @return a single software data of the virtual machine along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName, - Context context) { - return getWithResponseAsync( - resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName, context) - .block(); + public Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String softwareName, Context context) { + return getWithResponseAsync(resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName, + context).block(); } /** * Gets a single software data of the virtual machine. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -696,27 +567,23 @@ public Response getWithResponse( * @return a single software data of the virtual machine. */ @ServiceMethod(returns = ReturnType.SINGLE) - public SoftwareInner get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName) { - return getWithResponse( - resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName, Context.NONE) - .getValue(); + public SoftwareInner get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String softwareName) { + return getWithResponse(resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName, + Context.NONE).getValue(); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents the software inventory of the virtual machine along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByExtendedResourceNextSinglePageAsync(String nextLink) { @@ -724,76 +591,59 @@ private Mono> listByExtendedResourceNextSinglePageA return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByExtendedResourceNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents the software inventory of the virtual machine along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByExtendedResourceNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByExtendedResourceNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByExtendedResourceNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByExtendedResourceNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents the software inventory of the virtual machine along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink) { @@ -801,38 +651,30 @@ private Mono> listBySubscriptionNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return represents the software inventory of the virtual machine along with {@link PagedResponse} on successful - * completion of {@link Mono}. + * completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listBySubscriptionNextSinglePageAsync(String nextLink, Context context) { @@ -840,23 +682,13 @@ private Mono> listBySubscriptionNextSinglePageAsync return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listBySubscriptionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesImpl.java index fdbee743a3209..5f249c5c1ed07 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SoftwareInventoriesImpl.java @@ -21,71 +21,52 @@ public final class SoftwareInventoriesImpl implements SoftwareInventories { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SoftwareInventoriesImpl( - SoftwareInventoriesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SoftwareInventoriesImpl(SoftwareInventoriesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName) { - PagedIterable inner = - this - .serviceClient() - .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName); - return Utils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); + public PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName) { + PagedIterable inner = this.serviceClient().listByExtendedResource(resourceGroupName, + resourceNamespace, resourceType, resourceName); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); } - public PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context) { - PagedIterable inner = - this - .serviceClient() - .listByExtendedResource(resourceGroupName, resourceNamespace, resourceType, resourceName, context); - return Utils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); + public PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context) { + PagedIterable inner = this.serviceClient().listByExtendedResource(resourceGroupName, + resourceNamespace, resourceType, resourceName, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SoftwareImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String softwareName, - Context context) { - Response inner = - this - .serviceClient() - .getWithResponse( - resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName, context); + public Response getWithResponse(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String softwareName, Context context) { + Response inner = this.serviceClient().getWithResponse(resourceGroupName, resourceNamespace, + resourceType, resourceName, softwareName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SoftwareImpl(inner.getValue(), this.manager())); } else { return null; } } - public Software get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, + public Software get(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, String softwareName) { - SoftwareInner inner = - this.serviceClient().get(resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName); + SoftwareInner inner + = this.serviceClient().get(resourceGroupName, resourceNamespace, resourceType, resourceName, softwareName); if (inner != null) { return new SoftwareImpl(inner, this.manager()); } else { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesClientImpl.java index 434e0aa1a792e..e6758e2a84512 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesClientImpl.java @@ -38,24 +38,24 @@ */ public final class SqlVulnerabilityAssessmentBaselineRulesClientImpl implements SqlVulnerabilityAssessmentBaselineRulesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SqlVulnerabilityAssessmentBaselineRulesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SqlVulnerabilityAssessmentBaselineRulesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SqlVulnerabilityAssessmentBaselineRulesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SqlVulnerabilityAssessmentBaselineRulesService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SqlVulnerabilityAssessmentBaselineRulesService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } @@ -66,75 +66,58 @@ public final class SqlVulnerabilityAssessmentBaselineRulesClientImpl @Host("{$host}") @ServiceInterface(name = "SecurityCenterSqlVul") public interface SqlVulnerabilityAssessmentBaselineRulesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> createOrUpdate( - @HostParam("$host") String endpoint, - @PathParam("ruleId") String ruleId, - @QueryParam("workspaceId") String workspaceId, + Mono> createOrUpdate(@HostParam("$host") String endpoint, + @PathParam("ruleId") String ruleId, @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @BodyParam("application/json") RuleResultsInput body, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") RuleResultsInput body, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("ruleId") String ruleId, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, @PathParam("ruleId") String ruleId, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Delete("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules/{ruleId}") - @ExpectedResponses({200, 204}) + @ExpectedResponses({ 200, 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @PathParam("ruleId") String ruleId, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + Mono> delete(@HostParam("$host") String endpoint, @PathParam("ruleId") String ruleId, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Post("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/baselineRules") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> add( - @HostParam("$host") String endpoint, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, + Mono> add(@HostParam("$host") String endpoint, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, @PathParam(value = "resourceId", encoded = true) String resourceId, - @BodyParam("application/json") RulesResultsInput body, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") RulesResultsInput body, @HeaderParam("Accept") String accept, Context context); } /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -145,13 +128,11 @@ Mono> add( * @return rule results along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String ruleId, String workspaceId, String resourceId, RuleResultsInput body) { + private Mono> createOrUpdateWithResponseAsync(String ruleId, String workspaceId, + String resourceId, RuleResultsInput body) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (ruleId == null) { return Mono.error(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); @@ -168,24 +149,14 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .createOrUpdate( - this.client.getEndpoint(), - ruleId, - workspaceId, - apiVersion, - resourceId, - body, - accept, - context)) + .withContext(context -> service.createOrUpdate(this.client.getEndpoint(), ruleId, workspaceId, apiVersion, + resourceId, body, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -197,13 +168,11 @@ private Mono> createOrUpdateWithResponseAsync( * @return rule results along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createOrUpdateWithResponseAsync( - String ruleId, String workspaceId, String resourceId, RuleResultsInput body, Context context) { + private Mono> createOrUpdateWithResponseAsync(String ruleId, String workspaceId, + String resourceId, RuleResultsInput body, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (ruleId == null) { return Mono.error(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); @@ -220,14 +189,13 @@ private Mono> createOrUpdateWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .createOrUpdate( - this.client.getEndpoint(), ruleId, workspaceId, apiVersion, resourceId, body, accept, context); + return service.createOrUpdate(this.client.getEndpoint(), ruleId, workspaceId, apiVersion, resourceId, body, + accept, context); } /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -245,7 +213,7 @@ private Mono createOrUpdateAsync(String ruleId, String workspa /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -257,14 +225,14 @@ private Mono createOrUpdateAsync(String ruleId, String workspa * @return rule results along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createOrUpdateWithResponse( - String ruleId, String workspaceId, String resourceId, RuleResultsInput body, Context context) { + public Response createOrUpdateWithResponse(String ruleId, String workspaceId, String resourceId, + RuleResultsInput body, Context context) { return createOrUpdateWithResponseAsync(ruleId, workspaceId, resourceId, body, context).block(); } /** * Creates a Baseline for a rule in a database. Will overwrite any previously existing results. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -281,7 +249,7 @@ public RuleResultsInner createOrUpdate(String ruleId, String workspaceId, String /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -289,16 +257,14 @@ public RuleResultsInner createOrUpdate(String ruleId, String workspaceId, String * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the results for a given rule in the Baseline along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String ruleId, String workspaceId, String resourceId) { + private Mono> getWithResponseAsync(String ruleId, String workspaceId, + String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (ruleId == null) { return Mono.error(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); @@ -312,16 +278,14 @@ private Mono> getWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get(this.client.getEndpoint(), ruleId, workspaceId, apiVersion, resourceId, accept, context)) + .withContext(context -> service.get(this.client.getEndpoint(), ruleId, workspaceId, apiVersion, resourceId, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -330,16 +294,14 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the results for a given rule in the Baseline along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String ruleId, String workspaceId, String resourceId, Context context) { + private Mono> getWithResponseAsync(String ruleId, String workspaceId, String resourceId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (ruleId == null) { return Mono.error(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); @@ -358,7 +320,7 @@ private Mono> getWithResponseAsync( /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -374,7 +336,7 @@ private Mono getAsync(String ruleId, String workspaceId, Strin /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -385,14 +347,14 @@ private Mono getAsync(String ruleId, String workspaceId, Strin * @return the results for a given rule in the Baseline along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String ruleId, String workspaceId, String resourceId, Context context) { + public Response getWithResponse(String ruleId, String workspaceId, String resourceId, + Context context) { return getWithResponseAsync(ruleId, workspaceId, resourceId, context).block(); } /** * Gets the results for a given rule in the Baseline. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -408,7 +370,7 @@ public RuleResultsInner get(String ruleId, String workspaceId, String resourceId /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -420,10 +382,8 @@ public RuleResultsInner get(String ruleId, String workspaceId, String resourceId @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String ruleId, String workspaceId, String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (ruleId == null) { return Mono.error(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); @@ -437,17 +397,14 @@ private Mono> deleteWithResponseAsync(String ruleId, String works final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), ruleId, workspaceId, apiVersion, resourceId, accept, context)) + .withContext(context -> service.delete(this.client.getEndpoint(), ruleId, workspaceId, apiVersion, + resourceId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -458,13 +415,11 @@ private Mono> deleteWithResponseAsync(String ruleId, String works * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> deleteWithResponseAsync( - String ruleId, String workspaceId, String resourceId, Context context) { + private Mono> deleteWithResponseAsync(String ruleId, String workspaceId, String resourceId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (ruleId == null) { return Mono.error(new IllegalArgumentException("Parameter ruleId is required and cannot be null.")); @@ -483,7 +438,7 @@ private Mono> deleteWithResponseAsync( /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -499,7 +454,7 @@ private Mono deleteAsync(String ruleId, String workspaceId, String resourc /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -516,7 +471,7 @@ public Response deleteWithResponse(String ruleId, String workspaceId, Stri /** * Deletes a rule from the Baseline of a given database. - * + * * @param ruleId The rule Id. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -531,22 +486,20 @@ public void delete(String ruleId, String workspaceId, String resourceId) { /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the results for all rules in the Baseline along with {@link Response} on successful completion of {@link - * Mono}. + * @return the results for all rules in the Baseline along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(String workspaceId, String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (workspaceId == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceId is required and cannot be null.")); @@ -556,33 +509,29 @@ private Mono> listWithResponseAsync(String workspace } final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.list(this.client.getEndpoint(), workspaceId, apiVersion, resourceId, accept, context)) + return FluxUtil.withContext( + context -> service.list(this.client.getEndpoint(), workspaceId, apiVersion, resourceId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the results for all rules in the Baseline along with {@link Response} on successful completion of {@link - * Mono}. + * @return the results for all rules in the Baseline along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String workspaceId, String resourceId, Context context) { + private Mono> listWithResponseAsync(String workspaceId, String resourceId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (workspaceId == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceId is required and cannot be null.")); @@ -598,7 +547,7 @@ private Mono> listWithResponseAsync( /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -613,7 +562,7 @@ private Mono listAsync(String workspaceId, String resourceId) /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. @@ -629,7 +578,7 @@ public Response listWithResponse(String workspaceId, String r /** * Gets the results for all rules in the Baseline. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -644,7 +593,7 @@ public RulesResultsInner list(String workspaceId, String resourceId) { /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param body The baseline rules. @@ -654,13 +603,11 @@ public RulesResultsInner list(String workspaceId, String resourceId) { * @return a list of rules results along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> addWithResponseAsync( - String workspaceId, String resourceId, RulesResultsInput body) { + private Mono> addWithResponseAsync(String workspaceId, String resourceId, + RulesResultsInput body) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (workspaceId == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceId is required and cannot be null.")); @@ -674,15 +621,14 @@ private Mono> addWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service.add(this.client.getEndpoint(), workspaceId, apiVersion, resourceId, body, accept, context)) + .withContext(context -> service.add(this.client.getEndpoint(), workspaceId, apiVersion, resourceId, body, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param body The baseline rules. @@ -693,13 +639,11 @@ private Mono> addWithResponseAsync( * @return a list of rules results along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> addWithResponseAsync( - String workspaceId, String resourceId, RulesResultsInput body, Context context) { + private Mono> addWithResponseAsync(String workspaceId, String resourceId, + RulesResultsInput body, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (workspaceId == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceId is required and cannot be null.")); @@ -718,7 +662,7 @@ private Mono> addWithResponseAsync( /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -734,7 +678,7 @@ private Mono addAsync(String workspaceId, String resourceId) /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param body The baseline rules. @@ -745,14 +689,14 @@ private Mono addAsync(String workspaceId, String resourceId) * @return a list of rules results along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response addWithResponse( - String workspaceId, String resourceId, RulesResultsInput body, Context context) { + public Response addWithResponse(String workspaceId, String resourceId, RulesResultsInput body, + Context context) { return addWithResponseAsync(workspaceId, resourceId, body, context).block(); } /** * Add a list of baseline rules. Will overwrite any previously existing results (for all rules). - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesImpl.java index 2f3bbda297510..350e1af599c10 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentBaselineRulesImpl.java @@ -23,22 +23,18 @@ public final class SqlVulnerabilityAssessmentBaselineRulesImpl implements SqlVul private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SqlVulnerabilityAssessmentBaselineRulesImpl( - SqlVulnerabilityAssessmentBaselineRulesClient innerClient, + public SqlVulnerabilityAssessmentBaselineRulesImpl(SqlVulnerabilityAssessmentBaselineRulesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String ruleId, String workspaceId, String resourceId, Context context) { - Response inner = - this.serviceClient().getWithResponse(ruleId, workspaceId, resourceId, context); + public Response getWithResponse(String ruleId, String workspaceId, String resourceId, + Context context) { + Response inner + = this.serviceClient().getWithResponse(ruleId, workspaceId, resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RuleResultsImpl(inner.getValue(), this.manager())); } else { return null; @@ -65,10 +61,7 @@ public void delete(String ruleId, String workspaceId, String resourceId) { public Response listWithResponse(String workspaceId, String resourceId, Context context) { Response inner = this.serviceClient().listWithResponse(workspaceId, resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RulesResultsImpl(inner.getValue(), this.manager())); } else { return null; @@ -84,15 +77,12 @@ public RulesResults list(String workspaceId, String resourceId) { } } - public Response addWithResponse( - String workspaceId, String resourceId, RulesResultsInput body, Context context) { - Response inner = - this.serviceClient().addWithResponse(workspaceId, resourceId, body, context); + public Response addWithResponse(String workspaceId, String resourceId, RulesResultsInput body, + Context context) { + Response inner + = this.serviceClient().addWithResponse(workspaceId, resourceId, body, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new RulesResultsImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsClientImpl.java index ba2ec1708d839..a6fdac184b14d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsClientImpl.java @@ -32,67 +32,57 @@ */ public final class SqlVulnerabilityAssessmentScanResultsClientImpl implements SqlVulnerabilityAssessmentScanResultsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SqlVulnerabilityAssessmentScanResultsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SqlVulnerabilityAssessmentScanResultsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SqlVulnerabilityAssessmentScanResultsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SqlVulnerabilityAssessmentScanResultsService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SqlVulnerabilityAssessmentScanResultsService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterSqlVulnerabilityAssessmentScanResults to be used by the - * proxy service to perform REST calls. + * The interface defining all the services for SecurityCenterSqlVulnerabilityAssessmentScanResults to be used by + * the proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterSqlVul") public interface SqlVulnerabilityAssessmentScanResultsService { - @Headers({"Content-Type: application/json"}) - @Get( - "/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/scans/{scanId}/scanResults/{scanResultId}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/scans/{scanId}/scanResults/{scanResultId}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("scanId") String scanId, - @PathParam("scanResultId") String scanResultId, - @QueryParam("workspaceId") String workspaceId, + Mono> get(@HostParam("$host") String endpoint, @PathParam("scanId") String scanId, + @PathParam("scanResultId") String scanResultId, @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/scans/{scanId}/scanResults") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/scans/{scanId}/scanResults") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("scanId") String scanId, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + Mono> list(@HostParam("$host") String endpoint, @PathParam("scanId") String scanId, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); } /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -101,16 +91,14 @@ Mono> list( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the scan results of a single rule in a scan record along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scanId, String scanResultId, String workspaceId, String resourceId) { + private Mono> getWithResponseAsync(String scanId, String scanResultId, String workspaceId, + String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scanId == null) { return Mono.error(new IllegalArgumentException("Parameter scanId is required and cannot be null.")); @@ -127,24 +115,14 @@ private Mono> getWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - scanId, - scanResultId, - workspaceId, - apiVersion, - resourceId, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), scanId, scanResultId, workspaceId, + apiVersion, resourceId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -154,16 +132,14 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the scan results of a single rule in a scan record along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scanId, String scanResultId, String workspaceId, String resourceId, Context context) { + private Mono> getWithResponseAsync(String scanId, String scanResultId, String workspaceId, + String resourceId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scanId == null) { return Mono.error(new IllegalArgumentException("Parameter scanId is required and cannot be null.")); @@ -180,13 +156,13 @@ private Mono> getWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), scanId, scanResultId, workspaceId, apiVersion, resourceId, accept, context); + return service.get(this.client.getEndpoint(), scanId, scanResultId, workspaceId, apiVersion, resourceId, accept, + context); } /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -204,7 +180,7 @@ private Mono getAsync(String scanId, String scanResultId, Strin /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -216,14 +192,14 @@ private Mono getAsync(String scanId, String scanResultId, Strin * @return the scan results of a single rule in a scan record along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String scanId, String scanResultId, String workspaceId, String resourceId, Context context) { + public Response getWithResponse(String scanId, String scanResultId, String workspaceId, + String resourceId, Context context) { return getWithResponseAsync(scanId, scanResultId, workspaceId, resourceId, context).block(); } /** * Gets the scan results of a single rule in a scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param scanResultId The rule Id of the results. * @param workspaceId The workspace Id. @@ -240,7 +216,7 @@ public ScanResultInner get(String scanId, String scanResultId, String workspaceI /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -248,16 +224,14 @@ public ScanResultInner get(String scanId, String scanResultId, String workspaceI * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of scan results for a single scan record along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String scanId, String workspaceId, String resourceId) { + private Mono> listWithResponseAsync(String scanId, String workspaceId, + String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scanId == null) { return Mono.error(new IllegalArgumentException("Parameter scanId is required and cannot be null.")); @@ -271,16 +245,14 @@ private Mono> listWithResponseAsync( final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), scanId, workspaceId, apiVersion, resourceId, accept, context)) + .withContext(context -> service.list(this.client.getEndpoint(), scanId, workspaceId, apiVersion, resourceId, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -289,16 +261,14 @@ private Mono> listWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of scan results for a single scan record along with {@link Response} on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listWithResponseAsync( - String scanId, String workspaceId, String resourceId, Context context) { + private Mono> listWithResponseAsync(String scanId, String workspaceId, String resourceId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scanId == null) { return Mono.error(new IllegalArgumentException("Parameter scanId is required and cannot be null.")); @@ -317,7 +287,7 @@ private Mono> listWithResponseAsync( /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -333,7 +303,7 @@ private Mono listAsync(String scanId, String workspaceId, Stri /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -344,14 +314,14 @@ private Mono listAsync(String scanId, String workspaceId, Stri * @return a list of scan results for a single scan record along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response listWithResponse( - String scanId, String workspaceId, String resourceId, Context context) { + public Response listWithResponse(String scanId, String workspaceId, String resourceId, + Context context) { return listWithResponseAsync(scanId, workspaceId, resourceId, context).block(); } /** * Gets a list of scan results for a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan results for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsImpl.java index ede3a2500e802..ac44a1c6049b8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScanResultsImpl.java @@ -22,22 +22,18 @@ public final class SqlVulnerabilityAssessmentScanResultsImpl implements SqlVulne private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SqlVulnerabilityAssessmentScanResultsImpl( - SqlVulnerabilityAssessmentScanResultsClient innerClient, + public SqlVulnerabilityAssessmentScanResultsImpl(SqlVulnerabilityAssessmentScanResultsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } - public Response getWithResponse( - String scanId, String scanResultId, String workspaceId, String resourceId, Context context) { - Response inner = - this.serviceClient().getWithResponse(scanId, scanResultId, workspaceId, resourceId, context); + public Response getWithResponse(String scanId, String scanResultId, String workspaceId, + String resourceId, Context context) { + Response inner + = this.serviceClient().getWithResponse(scanId, scanResultId, workspaceId, resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ScanResultImpl(inner.getValue(), this.manager())); } else { return null; @@ -53,15 +49,12 @@ public ScanResult get(String scanId, String scanResultId, String workspaceId, St } } - public Response listWithResponse( - String scanId, String workspaceId, String resourceId, Context context) { - Response inner = - this.serviceClient().listWithResponse(scanId, workspaceId, resourceId, context); + public Response listWithResponse(String scanId, String workspaceId, String resourceId, + Context context) { + Response inner + = this.serviceClient().listWithResponse(scanId, workspaceId, resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ScanResultsImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansClientImpl.java index 3a315100d9df3..3bd2eb0709aab 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansClientImpl.java @@ -26,81 +26,74 @@ import com.azure.resourcemanager.security.fluent.models.ScansInner; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SqlVulnerabilityAssessmentScansClient. */ +/** + * An instance of this class provides access to all the operations defined in SqlVulnerabilityAssessmentScansClient. + */ public final class SqlVulnerabilityAssessmentScansClientImpl implements SqlVulnerabilityAssessmentScansClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SqlVulnerabilityAssessmentScansService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SqlVulnerabilityAssessmentScansClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SqlVulnerabilityAssessmentScansClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy - .create( - SqlVulnerabilityAssessmentScansService.class, - client.getHttpPipeline(), - client.getSerializerAdapter()); + this.service = RestProxy.create(SqlVulnerabilityAssessmentScansService.class, client.getHttpPipeline(), + client.getSerializerAdapter()); this.client = client; } /** - * The interface defining all the services for SecurityCenterSqlVulnerabilityAssessmentScans to be used by the proxy - * service to perform REST calls. + * The interface defining all the services for SecurityCenterSqlVulnerabilityAssessmentScans to be used by the + * proxy service to perform REST calls. */ @Host("{$host}") @ServiceInterface(name = "SecurityCenterSqlVul") public interface SqlVulnerabilityAssessmentScansService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/scans/{scanId}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @PathParam("scanId") String scanId, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, @PathParam("scanId") String scanId, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{resourceId}/providers/Microsoft.Security/sqlVulnerabilityAssessments/default/scans") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("workspaceId") String workspaceId, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "resourceId", encoded = true) String resourceId, - @HeaderParam("Accept") String accept, + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("workspaceId") String workspaceId, @QueryParam("api-version") String apiVersion, + @PathParam(value = "resourceId", encoded = true) String resourceId, @HeaderParam("Accept") String accept, Context context); } /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the scan details of a single scan record along with {@link Response} on successful completion of {@link - * Mono}. + * @return the scan details of a single scan record along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String scanId, String workspaceId, String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scanId == null) { return Mono.error(new IllegalArgumentException("Parameter scanId is required and cannot be null.")); @@ -114,16 +107,14 @@ private Mono> getWithResponseAsync(String scanId, String wor final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get(this.client.getEndpoint(), scanId, workspaceId, apiVersion, resourceId, accept, context)) + .withContext(context -> service.get(this.client.getEndpoint(), scanId, workspaceId, apiVersion, resourceId, + accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -131,17 +122,15 @@ private Mono> getWithResponseAsync(String scanId, String wor * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the scan details of a single scan record along with {@link Response} on successful completion of {@link - * Mono}. + * @return the scan details of a single scan record along with {@link Response} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scanId, String workspaceId, String resourceId, Context context) { + private Mono> getWithResponseAsync(String scanId, String workspaceId, String resourceId, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scanId == null) { return Mono.error(new IllegalArgumentException("Parameter scanId is required and cannot be null.")); @@ -160,7 +149,7 @@ private Mono> getWithResponseAsync( /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -176,7 +165,7 @@ private Mono getAsync(String scanId, String workspaceId, String resou /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -193,7 +182,7 @@ public Response getWithResponse(String scanId, String workspaceId, St /** * Gets the scan details of a single scan record. - * + * * @param scanId The scan Id. Type 'latest' to get the scan record for the latest scan. * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. @@ -209,7 +198,7 @@ public ScanInner get(String scanId, String workspaceId, String resourceId) { /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -220,10 +209,8 @@ public ScanInner get(String scanId, String workspaceId, String resourceId) { @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(String workspaceId, String resourceId) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (workspaceId == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceId is required and cannot be null.")); @@ -233,16 +220,14 @@ private Mono> listWithResponseAsync(String workspaceId, Str } final String apiVersion = "2023-02-01-preview"; final String accept = "application/json"; - return FluxUtil - .withContext( - context -> - service.list(this.client.getEndpoint(), workspaceId, apiVersion, resourceId, accept, context)) + return FluxUtil.withContext( + context -> service.list(this.client.getEndpoint(), workspaceId, apiVersion, resourceId, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. @@ -254,10 +239,8 @@ private Mono> listWithResponseAsync(String workspaceId, Str @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listWithResponseAsync(String workspaceId, String resourceId, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (workspaceId == null) { return Mono.error(new IllegalArgumentException("Parameter workspaceId is required and cannot be null.")); @@ -273,7 +256,7 @@ private Mono> listWithResponseAsync(String workspaceId, Str /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -288,7 +271,7 @@ private Mono listAsync(String workspaceId, String resourceId) { /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. @@ -304,7 +287,7 @@ public Response listWithResponse(String workspaceId, String resource /** * Gets a list of scan records. - * + * * @param workspaceId The workspace Id. * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansImpl.java index 8f16764d6269d..a5c3bb15481a1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SqlVulnerabilityAssessmentScansImpl.java @@ -22,8 +22,7 @@ public final class SqlVulnerabilityAssessmentScansImpl implements SqlVulnerabili private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SqlVulnerabilityAssessmentScansImpl( - SqlVulnerabilityAssessmentScansClient innerClient, + public SqlVulnerabilityAssessmentScansImpl(SqlVulnerabilityAssessmentScansClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; @@ -32,10 +31,7 @@ public SqlVulnerabilityAssessmentScansImpl( public Response getWithResponse(String scanId, String workspaceId, String resourceId, Context context) { Response inner = this.serviceClient().getWithResponse(scanId, workspaceId, resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ScanImpl(inner.getValue(), this.manager())); } else { return null; @@ -54,10 +50,7 @@ public Scan get(String scanId, String workspaceId, String resourceId) { public Response listWithResponse(String workspaceId, String resourceId, Context context) { Response inner = this.serviceClient().listWithResponse(workspaceId, resourceId, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new ScansImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsClientImpl.java index a93f80e557e9d..bac88385900c2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.SecuritySubAssessmentList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in SubAssessmentsClient. */ +/** + * An instance of this class provides access to all the operations defined in SubAssessmentsClient. + */ public final class SubAssessmentsClientImpl implements SubAssessmentsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final SubAssessmentsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of SubAssessmentsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ SubAssessmentsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(SubAssessmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(SubAssessmentsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,81 +62,65 @@ public final class SubAssessmentsClientImpl implements SubAssessmentsClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterSubAss") public interface SubAssessmentsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/subAssessments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listAll( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @HeaderParam("Accept") String accept, - Context context); + Mono> listAll(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/subAssessments") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, - @PathParam("assessmentName") String assessmentName, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, + @PathParam("assessmentName") String assessmentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/{scope}/providers/Microsoft.Security/assessments/{assessmentName}/subAssessments/{subAssessmentName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam(value = "scope", encoded = true) String scope, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam(value = "scope", encoded = true) String scope, @PathParam("assessmentName") String assessmentName, - @PathParam("subAssessmentName") String subAssessmentName, - @HeaderParam("Accept") String accept, + @PathParam("subAssessmentName") String subAssessmentName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listAllNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a subscription scope along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return security sub-assessments on all your scanned resources inside a subscription scope along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllSinglePageAsync(String scope) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -139,37 +129,28 @@ private Mono> listAllSinglePageAsync(S final String accept = "application/json"; return FluxUtil .withContext(context -> service.listAll(this.client.getEndpoint(), apiVersion, scope, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a subscription scope along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return security sub-assessments on all your scanned resources inside a subscription scope along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllSinglePageAsync(String scope, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -177,29 +158,21 @@ private Mono> listAllSinglePageAsync(S final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAll(this.client.getEndpoint(), apiVersion, scope, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAll(this.client.getEndpoint(), apiVersion, scope, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a subscription scope as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAllAsync(String scope) { @@ -208,32 +181,32 @@ private PagedFlux listAllAsync(String scope) { /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a subscription scope as paginated response - * with {@link PagedFlux}. + * with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAllAsync(String scope, Context context) { - return new PagedFlux<>( - () -> listAllSinglePageAsync(scope, context), nextLink -> listAllNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listAllSinglePageAsync(scope, context), + nextLink -> listAllNextSinglePageAsync(nextLink, context)); } /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a subscription scope as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAll(String scope) { @@ -242,15 +215,15 @@ public PagedIterable listAll(String scope) { /** * Get security sub-assessments on all your scanned resources inside a subscription scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a subscription scope as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listAll(String scope, Context context) { @@ -259,23 +232,21 @@ public PagedIterable listAll(String scope, Context c /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String scope, String assessmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -288,39 +259,30 @@ private Mono> listSinglePageAsync(Stri return FluxUtil .withContext( context -> service.list(this.client.getEndpoint(), apiVersion, scope, assessmentName, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security sub-assessments on all your scanned resources inside a scope along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listSinglePageAsync( - String scope, String assessmentName, Context context) { + private Mono> listSinglePageAsync(String scope, String assessmentName, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -331,68 +293,59 @@ private Mono> listSinglePageAsync( final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, scope, assessmentName, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, scope, assessmentName, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedFlux}. + * @return security sub-assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, String assessmentName) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, assessmentName), nextLink -> listNextSinglePageAsync(nextLink)); + return new PagedFlux<>(() -> listSinglePageAsync(scope, assessmentName), + nextLink -> listNextSinglePageAsync(nextLink)); } /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedFlux}. + * @return security sub-assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String scope, String assessmentName, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(scope, assessmentName, context), + return new PagedFlux<>(() -> listSinglePageAsync(scope, assessmentName, context), nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security sub-assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String scope, String assessmentName) { @@ -401,16 +354,16 @@ public PagedIterable list(String scope, String asses /** * Get security sub-assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security sub-assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security sub-assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(String scope, String assessmentName, Context context) { @@ -419,25 +372,23 @@ public PagedIterable list(String scope, String asses /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a security sub-assessment on your scanned resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scope, String assessmentName, String subAssessmentName) { + private Mono> getWithResponseAsync(String scope, String assessmentName, + String subAssessmentName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -452,25 +403,16 @@ private Mono> getWithResponseAsync( final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - scope, - assessmentName, - subAssessmentName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, scope, assessmentName, + subAssessmentName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @param context The context to associate with this operation. @@ -478,16 +420,14 @@ private Mono> getWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a security sub-assessment on your scanned resource along with {@link Response} on successful completion - * of {@link Mono}. + * of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String scope, String assessmentName, String subAssessmentName, Context context) { + private Mono> getWithResponseAsync(String scope, String assessmentName, + String subAssessmentName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (scope == null) { return Mono.error(new IllegalArgumentException("Parameter scope is required and cannot be null.")); @@ -502,15 +442,15 @@ private Mono> getWithResponseAsync( final String apiVersion = "2019-01-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get(this.client.getEndpoint(), apiVersion, scope, assessmentName, subAssessmentName, accept, context); + return service.get(this.client.getEndpoint(), apiVersion, scope, assessmentName, subAssessmentName, accept, + context); } /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -526,9 +466,9 @@ private Mono getAsync(String scope, String assessmen /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @param context The context to associate with this operation. @@ -538,16 +478,16 @@ private Mono getAsync(String scope, String assessmen * @return a security sub-assessment on your scanned resource along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String scope, String assessmentName, String subAssessmentName, Context context) { + public Response getWithResponse(String scope, String assessmentName, + String subAssessmentName, Context context) { return getWithResponseAsync(scope, assessmentName, subAssessmentName, context).block(); } /** * Get a security sub-assessment on your scanned resource. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param subAssessmentName The Sub-Assessment Key - Unique key for the sub-assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -562,14 +502,15 @@ public SecuritySubAssessmentInner get(String scope, String assessmentName, Strin /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listAllNextSinglePageAsync(String nextLink) { @@ -577,75 +518,58 @@ private Mono> listAllNextSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listAllNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listAllNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listAllNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listAllNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listAllNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -653,37 +577,28 @@ private Mono> listNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security sub-assessments along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -691,23 +606,13 @@ private Mono> listNextSinglePageAsync( return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsImpl.java index ed66f565392b7..65469f631ba75 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/SubAssessmentsImpl.java @@ -21,41 +21,38 @@ public final class SubAssessmentsImpl implements SubAssessments { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public SubAssessmentsImpl( - SubAssessmentsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public SubAssessmentsImpl(SubAssessmentsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable listAll(String scope) { PagedIterable inner = this.serviceClient().listAll(scope); - return Utils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); } public PagedIterable listAll(String scope, Context context) { PagedIterable inner = this.serviceClient().listAll(scope, context); - return Utils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); } public PagedIterable list(String scope, String assessmentName) { PagedIterable inner = this.serviceClient().list(scope, assessmentName); - return Utils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); } public PagedIterable list(String scope, String assessmentName, Context context) { PagedIterable inner = this.serviceClient().list(scope, assessmentName, context); - return Utils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecuritySubAssessmentImpl(inner1, this.manager())); } - public Response getWithResponse( - String scope, String assessmentName, String subAssessmentName, Context context) { - Response inner = - this.serviceClient().getWithResponse(scope, assessmentName, subAssessmentName, context); + public Response getWithResponse(String scope, String assessmentName, + String subAssessmentName, Context context) { + Response inner + = this.serviceClient().getWithResponse(scope, assessmentName, subAssessmentName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecuritySubAssessmentImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksClientImpl.java index 846466c0abe1b..a019dcc886b49 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksClientImpl.java @@ -32,17 +32,23 @@ import com.azure.resourcemanager.security.models.TaskUpdateActionType; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in TasksClient. */ +/** + * An instance of this class provides access to all the operations defined in TasksClient. + */ public final class TasksClientImpl implements TasksClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final TasksService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of TasksClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ TasksClientImpl(SecurityCenterImpl client) { @@ -57,229 +63,158 @@ public final class TasksClientImpl implements TasksClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterTasks") public interface TasksService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/tasks") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/tasks") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByHomeRegion( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByHomeRegion(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @QueryParam("$filter") String filter, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getSubscriptionLevelTask( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("taskName") String taskName, - @HeaderParam("Accept") String accept, - Context context); + Mono> getSubscriptionLevelTask(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("taskName") String taskName, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}/{taskUpdateActionType}") - @ExpectedResponses({204}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}/{taskUpdateActionType}") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateSubscriptionLevelTaskState( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @PathParam("taskName") String taskName, + Mono> updateSubscriptionLevelTaskState(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("ascLocation") String ascLocation, @PathParam("taskName") String taskName, @PathParam("taskUpdateActionType") TaskUpdateActionType taskUpdateActionType, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/tasks") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/tasks") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByResourceGroup( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @QueryParam("$filter") String filter, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByResourceGroup(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @QueryParam("$filter") String filter, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> getResourceGroupLevelTask( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, - @PathParam("taskName") String taskName, - @HeaderParam("Accept") String accept, - Context context); + Mono> getResourceGroupLevelTask(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, + @PathParam("taskName") String taskName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Post( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}/{taskUpdateActionType}") - @ExpectedResponses({204}) + @Headers({ "Content-Type: application/json" }) + @Post("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/tasks/{taskName}/{taskUpdateActionType}") + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> updateResourceGroupLevelTaskState( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + Mono> updateResourceGroupLevelTaskState(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("taskName") String taskName, @PathParam("taskUpdateActionType") TaskUpdateActionType taskUpdateActionType, - @HeaderParam("Accept") String accept, - Context context); + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByHomeRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByResourceGroupNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); return service .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), filter, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -293,7 +228,7 @@ private PagedFlux listAsync(String filter) { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security task recommendations as paginated response with {@link PagedFlux}. @@ -306,7 +241,7 @@ private PagedFlux listAsync() { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -316,13 +251,13 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(String filter, Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(filter, context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(filter, context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security task recommendations as paginated response with {@link PagedIterable}. @@ -335,7 +270,7 @@ public PagedIterable list() { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -350,29 +285,25 @@ public PagedIterable list(String filter, Context context) { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionSinglePageAsync(String ascLocation, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -380,56 +311,36 @@ private Mono> listByHomeRegionSinglePageAsync(S final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByHomeRegion( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByHomeRegion(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionSinglePageAsync( - String ascLocation, String filter, Context context) { + private Mono> listByHomeRegionSinglePageAsync(String ascLocation, String filter, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -438,30 +349,17 @@ private Mono> listByHomeRegionSinglePageAsync( final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByHomeRegion( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByHomeRegion(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), ascLocation, + filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -470,16 +368,15 @@ private Mono> listByHomeRegionSinglePageAsync( */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation, String filter) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, filter), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, filter), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -488,16 +385,15 @@ private PagedFlux listByHomeRegionAsync(String ascLocation, S @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation) { final String filter = null; - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, filter), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, filter), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -507,16 +403,15 @@ private PagedFlux listByHomeRegionAsync(String ascLocation) { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation, String filter, Context context) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, filter, context), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, filter, context), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink, context)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -530,9 +425,9 @@ public PagedIterable listByHomeRegion(String ascLocation) { /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -547,30 +442,26 @@ public PagedIterable listByHomeRegion(String ascLocation, Str /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security task that we recommend to do in order to strengthen security along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getSubscriptionLevelTaskWithResponseAsync( - String ascLocation, String taskName) { + private Mono> getSubscriptionLevelTaskWithResponseAsync(String ascLocation, + String taskName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -581,47 +472,34 @@ private Mono> getSubscriptionLevelTaskWithResponseAs final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getSubscriptionLevelTask( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - taskName, - accept, - context)) + .withContext(context -> service.getSubscriptionLevelTask(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, taskName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security task that we recommend to do in order to strengthen security along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getSubscriptionLevelTaskWithResponseAsync( - String ascLocation, String taskName, Context context) { + private Mono> getSubscriptionLevelTaskWithResponseAsync(String ascLocation, + String taskName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -632,28 +510,21 @@ private Mono> getSubscriptionLevelTaskWithResponseAs final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getSubscriptionLevelTask( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - taskName, - accept, - context); + return service.getSubscriptionLevelTask(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + ascLocation, taskName, accept, context); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security task that we recommend to do in order to strengthen security on successful completion of {@link - * Mono}. + * @return security task that we recommend to do in order to strengthen security on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getSubscriptionLevelTaskAsync(String ascLocation, String taskName) { @@ -663,9 +534,9 @@ private Mono getSubscriptionLevelTaskAsync(String ascLocation /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -674,16 +545,16 @@ private Mono getSubscriptionLevelTaskAsync(String ascLocation * @return security task that we recommend to do in order to strengthen security along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getSubscriptionLevelTaskWithResponse( - String ascLocation, String taskName, Context context) { + public Response getSubscriptionLevelTaskWithResponse(String ascLocation, String taskName, + Context context) { return getSubscriptionLevelTaskWithResponseAsync(ascLocation, taskName, context).block(); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -697,9 +568,9 @@ public SecurityTaskInner getSubscriptionLevelTask(String ascLocation, String tas /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -708,19 +579,15 @@ public SecurityTaskInner getSubscriptionLevelTask(String ascLocation, String tas * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelTaskStateWithResponseAsync( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { + private Mono> updateSubscriptionLevelTaskStateWithResponseAsync(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -735,26 +602,16 @@ private Mono> updateSubscriptionLevelTaskStateWithResponseAsync( final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateSubscriptionLevelTaskState( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - taskName, - taskUpdateActionType, - accept, - context)) + .withContext(context -> service.updateSubscriptionLevelTaskState(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, taskName, taskUpdateActionType, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @param context The context to associate with this operation. @@ -764,19 +621,15 @@ private Mono> updateSubscriptionLevelTaskStateWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateSubscriptionLevelTaskStateWithResponseAsync( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType, Context context) { + private Mono> updateSubscriptionLevelTaskStateWithResponseAsync(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -791,23 +644,15 @@ private Mono> updateSubscriptionLevelTaskStateWithResponseAsync( final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateSubscriptionLevelTaskState( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - ascLocation, - taskName, - taskUpdateActionType, - accept, - context); + return service.updateSubscriptionLevelTaskState(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), ascLocation, taskName, taskUpdateActionType, accept, context); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -816,17 +661,17 @@ private Mono> updateSubscriptionLevelTaskStateWithResponseAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateSubscriptionLevelTaskStateAsync( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { + private Mono updateSubscriptionLevelTaskStateAsync(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType) { return updateSubscriptionLevelTaskStateWithResponseAsync(ascLocation, taskName, taskUpdateActionType) .flatMap(ignored -> Mono.empty()); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @param context The context to associate with this operation. @@ -836,17 +681,17 @@ private Mono updateSubscriptionLevelTaskStateAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateSubscriptionLevelTaskStateWithResponse( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType, Context context) { + public Response updateSubscriptionLevelTaskStateWithResponse(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType, Context context) { return updateSubscriptionLevelTaskStateWithResponseAsync(ascLocation, taskName, taskUpdateActionType, context) .block(); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -854,39 +699,35 @@ public Response updateSubscriptionLevelTaskStateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void updateSubscriptionLevelTaskState( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { + public void updateSubscriptionLevelTaskState(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType) { updateSubscriptionLevelTaskStateWithResponse(ascLocation, taskName, taskUpdateActionType, Context.NONE); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, String ascLocation, String filter) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + String ascLocation, String filter) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -898,59 +739,38 @@ private Mono> listByResourceGroupSinglePageAsyn final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - filter, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByResourceGroup(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, filter, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupSinglePageAsync( - String resourceGroupName, String ascLocation, String filter, Context context) { + private Mono> listByResourceGroupSinglePageAsync(String resourceGroupName, + String ascLocation, String filter, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -963,33 +783,19 @@ private Mono> listByResourceGroupSinglePageAsyn final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByResourceGroup( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - filter, - accept, - context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByResourceGroup(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, ascLocation, filter, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -997,20 +803,19 @@ private Mono> listByResourceGroupSinglePageAsyn * @return list of security task recommendations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, String ascLocation, String filter) { - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, ascLocation, filter), + private PagedFlux listByResourceGroupAsync(String resourceGroupName, String ascLocation, + String filter) { + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, ascLocation, filter), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1019,18 +824,17 @@ private PagedFlux listByResourceGroupAsync( @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByResourceGroupAsync(String resourceGroupName, String ascLocation) { final String filter = null; - return new PagedFlux<>( - () -> listByResourceGroupSinglePageAsync(resourceGroupName, ascLocation, filter), + return new PagedFlux<>(() -> listByResourceGroupSinglePageAsync(resourceGroupName, ascLocation, filter), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1039,8 +843,8 @@ private PagedFlux listByResourceGroupAsync(String resourceGro * @return list of security task recommendations as paginated response with {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - private PagedFlux listByResourceGroupAsync( - String resourceGroupName, String ascLocation, String filter, Context context) { + private PagedFlux listByResourceGroupAsync(String resourceGroupName, String ascLocation, + String filter, Context context) { return new PagedFlux<>( () -> listByResourceGroupSinglePageAsync(resourceGroupName, ascLocation, filter, context), nextLink -> listByResourceGroupNextSinglePageAsync(nextLink, context)); @@ -1048,11 +852,11 @@ private PagedFlux listByResourceGroupAsync( /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -1066,11 +870,11 @@ public PagedIterable listByResourceGroup(String resourceGroup /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param filter OData filter. Optional. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1079,39 +883,35 @@ public PagedIterable listByResourceGroup(String resourceGroup * @return list of security task recommendations as paginated response with {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) - public PagedIterable listByResourceGroup( - String resourceGroupName, String ascLocation, String filter, Context context) { + public PagedIterable listByResourceGroup(String resourceGroupName, String ascLocation, + String filter, Context context) { return new PagedIterable<>(listByResourceGroupAsync(resourceGroupName, ascLocation, filter, context)); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security task that we recommend to do in order to strengthen security along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getResourceGroupLevelTaskWithResponseAsync( - String resourceGroupName, String ascLocation, String taskName) { + private Mono> getResourceGroupLevelTaskWithResponseAsync(String resourceGroupName, + String ascLocation, String taskName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1126,50 +926,36 @@ private Mono> getResourceGroupLevelTaskWithResponseA final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .getResourceGroupLevelTask( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - taskName, - accept, - context)) + .withContext(context -> service.getResourceGroupLevelTask(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, taskName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return security task that we recommend to do in order to strengthen security along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getResourceGroupLevelTaskWithResponseAsync( - String resourceGroupName, String ascLocation, String taskName, Context context) { + private Mono> getResourceGroupLevelTaskWithResponseAsync(String resourceGroupName, + String ascLocation, String taskName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1184,46 +970,38 @@ private Mono> getResourceGroupLevelTaskWithResponseA final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .getResourceGroupLevelTask( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - taskName, - accept, - context); + return service.getResourceGroupLevelTask(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + resourceGroupName, ascLocation, taskName, accept, context); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security task that we recommend to do in order to strengthen security on successful completion of {@link - * Mono}. + * @return security task that we recommend to do in order to strengthen security on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getResourceGroupLevelTaskAsync( - String resourceGroupName, String ascLocation, String taskName) { + private Mono getResourceGroupLevelTaskAsync(String resourceGroupName, String ascLocation, + String taskName) { return getResourceGroupLevelTaskWithResponseAsync(resourceGroupName, ascLocation, taskName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1232,18 +1010,18 @@ private Mono getResourceGroupLevelTaskAsync( * @return security task that we recommend to do in order to strengthen security along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getResourceGroupLevelTaskWithResponse( - String resourceGroupName, String ascLocation, String taskName, Context context) { + public Response getResourceGroupLevelTaskWithResponse(String resourceGroupName, + String ascLocation, String taskName, Context context) { return getResourceGroupLevelTaskWithResponseAsync(resourceGroupName, ascLocation, taskName, context).block(); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -1257,11 +1035,11 @@ public SecurityTaskInner getResourceGroupLevelTask(String resourceGroupName, Str /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1270,19 +1048,15 @@ public SecurityTaskInner getResourceGroupLevelTask(String resourceGroupName, Str * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelTaskStateWithResponseAsync( - String resourceGroupName, String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { + private Mono> updateResourceGroupLevelTaskStateWithResponseAsync(String resourceGroupName, + String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1301,29 +1075,19 @@ private Mono> updateResourceGroupLevelTaskStateWithResponseAsync( final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .updateResourceGroupLevelTaskState( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - taskName, - taskUpdateActionType, - accept, - context)) + .withContext(context -> service.updateResourceGroupLevelTaskState(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, taskName, taskUpdateActionType, accept, + context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @param context The context to associate with this operation. @@ -1333,23 +1097,15 @@ private Mono> updateResourceGroupLevelTaskStateWithResponseAsync( * @return the {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateResourceGroupLevelTaskStateWithResponseAsync( - String resourceGroupName, - String ascLocation, - String taskName, - TaskUpdateActionType taskUpdateActionType, - Context context) { + private Mono> updateResourceGroupLevelTaskStateWithResponseAsync(String resourceGroupName, + String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -1368,26 +1124,18 @@ private Mono> updateResourceGroupLevelTaskStateWithResponseAsync( final String apiVersion = "2015-06-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .updateResourceGroupLevelTaskState( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - taskName, - taskUpdateActionType, - accept, - context); + return service.updateResourceGroupLevelTaskState(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), resourceGroupName, ascLocation, taskName, taskUpdateActionType, accept, + context); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1396,20 +1144,19 @@ private Mono> updateResourceGroupLevelTaskStateWithResponseAsync( * @return A {@link Mono} that completes when a successful response is received. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateResourceGroupLevelTaskStateAsync( - String resourceGroupName, String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { - return updateResourceGroupLevelTaskStateWithResponseAsync( - resourceGroupName, ascLocation, taskName, taskUpdateActionType) - .flatMap(ignored -> Mono.empty()); + private Mono updateResourceGroupLevelTaskStateAsync(String resourceGroupName, String ascLocation, + String taskName, TaskUpdateActionType taskUpdateActionType) { + return updateResourceGroupLevelTaskStateWithResponseAsync(resourceGroupName, ascLocation, taskName, + taskUpdateActionType).flatMap(ignored -> Mono.empty()); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @param context The context to associate with this operation. @@ -1419,24 +1166,19 @@ private Mono updateResourceGroupLevelTaskStateAsync( * @return the {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateResourceGroupLevelTaskStateWithResponse( - String resourceGroupName, - String ascLocation, - String taskName, - TaskUpdateActionType taskUpdateActionType, - Context context) { - return updateResourceGroupLevelTaskStateWithResponseAsync( - resourceGroupName, ascLocation, taskName, taskUpdateActionType, context) - .block(); + public Response updateResourceGroupLevelTaskStateWithResponse(String resourceGroupName, String ascLocation, + String taskName, TaskUpdateActionType taskUpdateActionType, Context context) { + return updateResourceGroupLevelTaskStateWithResponseAsync(resourceGroupName, ascLocation, taskName, + taskUpdateActionType, context).block(); } /** * Recommended tasks that will help improve the security of the subscription proactively. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param taskName Name of the task object, will be a GUID. * @param taskUpdateActionType Type of the action to do on the task. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -1444,22 +1186,23 @@ public Response updateResourceGroupLevelTaskStateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ @ServiceMethod(returns = ReturnType.SINGLE) - public void updateResourceGroupLevelTaskState( - String resourceGroupName, String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { - updateResourceGroupLevelTaskStateWithResponse( - resourceGroupName, ascLocation, taskName, taskUpdateActionType, Context.NONE); + public void updateResourceGroupLevelTaskState(String resourceGroupName, String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType) { + updateResourceGroupLevelTaskStateWithResponse(resourceGroupName, ascLocation, taskName, taskUpdateActionType, + Context.NONE); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -1467,37 +1210,28 @@ private Mono> listNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -1505,36 +1239,27 @@ private Mono> listNextSinglePageAsync(String ne return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionNextSinglePageAsync(String nextLink) { @@ -1542,75 +1267,58 @@ private Mono> listByHomeRegionNextSinglePageAsy return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByHomeRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByResourceGroupNextSinglePageAsync(String nextLink) { @@ -1618,63 +1326,45 @@ private Mono> listByResourceGroupNextSinglePage return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext( context -> service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), + res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of security task recommendations along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of security task recommendations along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByResourceGroupNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByResourceGroupNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByResourceGroupNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksImpl.java index ec884a63c55c3..7def20fbf2b04 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TasksImpl.java @@ -29,33 +29,30 @@ public TasksImpl(TasksClient innerClient, com.azure.resourcemanager.security.Sec public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); } public PagedIterable list(String filter, Context context) { PagedIterable inner = this.serviceClient().list(filter, context); - return Utils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation, String filter, Context context) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation, filter, context); - return Utils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); } - public Response getSubscriptionLevelTaskWithResponse( - String ascLocation, String taskName, Context context) { - Response inner = - this.serviceClient().getSubscriptionLevelTaskWithResponse(ascLocation, taskName, context); + public Response getSubscriptionLevelTaskWithResponse(String ascLocation, String taskName, + Context context) { + Response inner + = this.serviceClient().getSubscriptionLevelTaskWithResponse(ascLocation, taskName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityTaskImpl(inner.getValue(), this.manager())); } else { return null; @@ -71,42 +68,36 @@ public SecurityTask getSubscriptionLevelTask(String ascLocation, String taskName } } - public Response updateSubscriptionLevelTaskStateWithResponse( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType, Context context) { - return this - .serviceClient() - .updateSubscriptionLevelTaskStateWithResponse(ascLocation, taskName, taskUpdateActionType, context); + public Response updateSubscriptionLevelTaskStateWithResponse(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType, Context context) { + return this.serviceClient().updateSubscriptionLevelTaskStateWithResponse(ascLocation, taskName, + taskUpdateActionType, context); } - public void updateSubscriptionLevelTaskState( - String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { + public void updateSubscriptionLevelTaskState(String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType) { this.serviceClient().updateSubscriptionLevelTaskState(ascLocation, taskName, taskUpdateActionType); } public PagedIterable listByResourceGroup(String resourceGroupName, String ascLocation) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, ascLocation); - return Utils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, ascLocation); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); } - public PagedIterable listByResourceGroup( - String resourceGroupName, String ascLocation, String filter, Context context) { - PagedIterable inner = - this.serviceClient().listByResourceGroup(resourceGroupName, ascLocation, filter, context); - return Utils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); + public PagedIterable listByResourceGroup(String resourceGroupName, String ascLocation, String filter, + Context context) { + PagedIterable inner + = this.serviceClient().listByResourceGroup(resourceGroupName, ascLocation, filter, context); + return ResourceManagerUtils.mapPage(inner, inner1 -> new SecurityTaskImpl(inner1, this.manager())); } - public Response getResourceGroupLevelTaskWithResponse( - String resourceGroupName, String ascLocation, String taskName, Context context) { - Response inner = - this - .serviceClient() - .getResourceGroupLevelTaskWithResponse(resourceGroupName, ascLocation, taskName, context); + public Response getResourceGroupLevelTaskWithResponse(String resourceGroupName, String ascLocation, + String taskName, Context context) { + Response inner = this.serviceClient() + .getResourceGroupLevelTaskWithResponse(resourceGroupName, ascLocation, taskName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new SecurityTaskImpl(inner.getValue(), this.manager())); } else { return null; @@ -114,8 +105,8 @@ public Response getResourceGroupLevelTaskWithResponse( } public SecurityTask getResourceGroupLevelTask(String resourceGroupName, String ascLocation, String taskName) { - SecurityTaskInner inner = - this.serviceClient().getResourceGroupLevelTask(resourceGroupName, ascLocation, taskName); + SecurityTaskInner inner + = this.serviceClient().getResourceGroupLevelTask(resourceGroupName, ascLocation, taskName); if (inner != null) { return new SecurityTaskImpl(inner, this.manager()); } else { @@ -123,23 +114,16 @@ public SecurityTask getResourceGroupLevelTask(String resourceGroupName, String a } } - public Response updateResourceGroupLevelTaskStateWithResponse( - String resourceGroupName, - String ascLocation, - String taskName, - TaskUpdateActionType taskUpdateActionType, - Context context) { - return this - .serviceClient() - .updateResourceGroupLevelTaskStateWithResponse( - resourceGroupName, ascLocation, taskName, taskUpdateActionType, context); + public Response updateResourceGroupLevelTaskStateWithResponse(String resourceGroupName, String ascLocation, + String taskName, TaskUpdateActionType taskUpdateActionType, Context context) { + return this.serviceClient().updateResourceGroupLevelTaskStateWithResponse(resourceGroupName, ascLocation, + taskName, taskUpdateActionType, context); } - public void updateResourceGroupLevelTaskState( - String resourceGroupName, String ascLocation, String taskName, TaskUpdateActionType taskUpdateActionType) { - this - .serviceClient() - .updateResourceGroupLevelTaskState(resourceGroupName, ascLocation, taskName, taskUpdateActionType); + public void updateResourceGroupLevelTaskState(String resourceGroupName, String ascLocation, String taskName, + TaskUpdateActionType taskUpdateActionType) { + this.serviceClient().updateResourceGroupLevelTaskState(resourceGroupName, ascLocation, taskName, + taskUpdateActionType); } private TasksClient serviceClient() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesClientImpl.java index 28a78eafba17b..25fc532781967 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesClientImpl.java @@ -30,22 +30,28 @@ import com.azure.resourcemanager.security.models.TopologyList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in TopologiesClient. */ +/** + * An instance of this class provides access to all the operations defined in TopologiesClient. + */ public final class TopologiesClientImpl implements TopologiesClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final TopologiesService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of TopologiesClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ TopologiesClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(TopologiesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(TopologiesService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -56,153 +62,111 @@ public final class TopologiesClientImpl implements TopologiesClient { @Host("{$host}") @ServiceInterface(name = "SecurityCenterTopolo") public interface TopologiesService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/topologies") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @QueryParam("api-version") String apiVersion, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/locations/{ascLocation}/topologies") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listByHomeRegion( - @HostParam("$host") String endpoint, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("ascLocation") String ascLocation, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + Mono> listByHomeRegion(@HostParam("$host") String endpoint, + @PathParam("subscriptionId") String subscriptionId, @PathParam("ascLocation") String ascLocation, + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) - @Get( - "/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/topologies/{topologyResourceName}") - @ExpectedResponses({200}) + @Headers({ "Content-Type: application/json" }) + @Get("/subscriptions/{subscriptionId}/resourceGroups/{resourceGroupName}/providers/Microsoft.Security/locations/{ascLocation}/topologies/{topologyResourceName}") + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, + Mono> get(@HostParam("$host") String endpoint, @PathParam("subscriptionId") String subscriptionId, - @PathParam("resourceGroupName") String resourceGroupName, - @PathParam("ascLocation") String ascLocation, + @PathParam("resourceGroupName") String resourceGroupName, @PathParam("ascLocation") String ascLocation, @PathParam("topologyResourceName") String topologyResourceName, - @QueryParam("api-version") String apiVersion, - @HeaderParam("Accept") String accept, - Context context); + @QueryParam("api-version") String apiVersion, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) Mono> listByHomeRegionNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + @PathParam(value = "nextLink", encoded = true) String nextLink, @HostParam("$host") String endpoint, + @HeaderParam("Accept") String accept, Context context); } /** * Gets a list that allows to build a topology view of a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list that allows to build a topology view of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription along with {@link PagedResponse} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), this.client.getSubscriptionId(), apiVersion, accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list that allows to build a topology view of a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription as paginated response with {@link - * PagedFlux}. + * @return a list that allows to build a topology view of a subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync() { @@ -211,27 +175,27 @@ private PagedFlux listAsync() { /** * Gets a list that allows to build a topology view of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription as paginated response with {@link - * PagedFlux}. + * @return a list that allows to build a topology view of a subscription as paginated response with + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Gets a list that allows to build a topology view of a subscription. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription as paginated response with {@link - * PagedIterable}. + * @return a list that allows to build a topology view of a subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list() { @@ -240,13 +204,13 @@ public PagedIterable list() { /** * Gets a list that allows to build a topology view of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription as paginated response with {@link - * PagedIterable}. + * @return a list that allows to build a topology view of a subscription as paginated response with + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable list(Context context) { @@ -255,28 +219,24 @@ public PagedIterable list(Context context) { /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription and location along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list that allows to build a topology view of a subscription and location along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listByHomeRegionSinglePageAsync(String ascLocation) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -284,54 +244,35 @@ private Mono> listByHomeRegionSinglePageAsy final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .listByHomeRegion( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - ascLocation, - apiVersion, - accept, - context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), + ascLocation, apiVersion, accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list that allows to build a topology view of a subscription and location along with {@link - * PagedResponse} on successful completion of {@link Mono}. + * @return a list that allows to build a topology view of a subscription and location along with + * {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionSinglePageAsync( - String ascLocation, Context context) { + private Mono> listByHomeRegionSinglePageAsync(String ascLocation, + Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (ascLocation == null) { return Mono.error(new IllegalArgumentException("Parameter ascLocation is required and cannot be null.")); @@ -340,66 +281,57 @@ private Mono> listByHomeRegionSinglePageAsy final String accept = "application/json"; context = this.client.mergeContext(context); return service - .listByHomeRegion( - this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + .listByHomeRegion(this.client.getEndpoint(), this.client.getSubscriptionId(), ascLocation, apiVersion, + accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription and location as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink)); } /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription and location as paginated response with - * {@link PagedFlux}. + * {@link PagedFlux}. */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listByHomeRegionAsync(String ascLocation, Context context) { - return new PagedFlux<>( - () -> listByHomeRegionSinglePageAsync(ascLocation, context), + return new PagedFlux<>(() -> listByHomeRegionSinglePageAsync(ascLocation, context), nextLink -> listByHomeRegionNextSinglePageAsync(nextLink, context)); } /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation) { @@ -408,15 +340,15 @@ public PagedIterable listByHomeRegion(String ascLocation) /** * Gets a list that allows to build a topology view of a subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list that allows to build a topology view of a subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ @ServiceMethod(returns = ReturnType.COLLECTION) public PagedIterable listByHomeRegion(String ascLocation, Context context) { @@ -425,11 +357,11 @@ public PagedIterable listByHomeRegion(String ascLocation, /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -437,19 +369,15 @@ public PagedIterable listByHomeRegion(String ascLocation, * @return a specific topology component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String topologyResourceName) { + private Mono> getWithResponseAsync(String resourceGroupName, String ascLocation, + String topologyResourceName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -465,28 +393,18 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - topologyResourceName, - apiVersion, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), + resourceGroupName, ascLocation, topologyResourceName, apiVersion, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -495,19 +413,15 @@ private Mono> getWithResponseAsync( * @return a specific topology component along with {@link Response} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> getWithResponseAsync( - String resourceGroupName, String ascLocation, String topologyResourceName, Context context) { + private Mono> getWithResponseAsync(String resourceGroupName, String ascLocation, + String topologyResourceName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (resourceGroupName == null) { return Mono @@ -523,25 +437,17 @@ private Mono> getWithResponseAsync( final String apiVersion = "2020-01-01"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - this.client.getSubscriptionId(), - resourceGroupName, - ascLocation, - topologyResourceName, - apiVersion, - accept, - context); + return service.get(this.client.getEndpoint(), this.client.getSubscriptionId(), resourceGroupName, ascLocation, + topologyResourceName, apiVersion, accept, context); } /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -549,19 +455,19 @@ private Mono> getWithResponseAsync( * @return a specific topology component on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono getAsync( - String resourceGroupName, String ascLocation, String topologyResourceName) { + private Mono getAsync(String resourceGroupName, String ascLocation, + String topologyResourceName) { return getWithResponseAsync(resourceGroupName, ascLocation, topologyResourceName) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -570,18 +476,18 @@ private Mono getAsync( * @return a specific topology component along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response getWithResponse( - String resourceGroupName, String ascLocation, String topologyResourceName, Context context) { + public Response getWithResponse(String resourceGroupName, String ascLocation, + String topologyResourceName, Context context) { return getWithResponseAsync(resourceGroupName, ascLocation, topologyResourceName, context).block(); } /** * Gets a specific topology component. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param topologyResourceName Name of a topology resources collection. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -595,9 +501,10 @@ public TopologyResourceInner get(String resourceGroupName, String ascLocation, S /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -609,31 +516,22 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -646,31 +544,22 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -682,31 +571,23 @@ private Mono> listByHomeRegionNextSinglePag return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; return FluxUtil .withContext(context -> service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -714,29 +595,19 @@ private Mono> listByHomeRegionNextSinglePag * @return the response body along with {@link PagedResponse} on successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> listByHomeRegionNextSinglePageAsync( - String nextLink, Context context) { + private Mono> listByHomeRegionNextSinglePageAsync(String nextLink, + Context context) { if (nextLink == null) { return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listByHomeRegionNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesImpl.java index 7aa969963df63..368bedb9ddc8d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologiesImpl.java @@ -21,41 +21,38 @@ public final class TopologiesImpl implements Topologies { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public TopologiesImpl( - TopologiesClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public TopologiesImpl(TopologiesClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation); - return Utils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); } public PagedIterable listByHomeRegion(String ascLocation, Context context) { PagedIterable inner = this.serviceClient().listByHomeRegion(ascLocation, context); - return Utils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new TopologyResourceImpl(inner1, this.manager())); } - public Response getWithResponse( - String resourceGroupName, String ascLocation, String topologyResourceName, Context context) { - Response inner = - this.serviceClient().getWithResponse(resourceGroupName, ascLocation, topologyResourceName, context); + public Response getWithResponse(String resourceGroupName, String ascLocation, + String topologyResourceName, Context context) { + Response inner + = this.serviceClient().getWithResponse(resourceGroupName, ascLocation, topologyResourceName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new TopologyResourceImpl(inner.getValue(), this.manager())); } else { return null; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologyResourceImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologyResourceImpl.java index 9a4e235014809..33b2c36fb7169 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologyResourceImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/TopologyResourceImpl.java @@ -16,8 +16,8 @@ public final class TopologyResourceImpl implements TopologyResource { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - TopologyResourceImpl( - TopologyResourceInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + TopologyResourceImpl(TopologyResourceInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/Utils.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/Utils.java deleted file mode 100644 index 210bf291fe60f..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/Utils.java +++ /dev/null @@ -1,204 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.implementation; - -import com.azure.core.http.rest.PagedFlux; -import com.azure.core.http.rest.PagedIterable; -import com.azure.core.http.rest.PagedResponse; -import com.azure.core.http.rest.PagedResponseBase; -import com.azure.core.util.CoreUtils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import reactor.core.publisher.Flux; - -final class Utils { - static String getValueFromIdByName(String id, String name) { - if (id == null) { - return null; - } - Iterator itr = Arrays.stream(id.split("/")).iterator(); - while (itr.hasNext()) { - String part = itr.next(); - if (part != null && !part.trim().isEmpty()) { - if (part.equalsIgnoreCase(name)) { - if (itr.hasNext()) { - return itr.next(); - } else { - return null; - } - } - } - } - return null; - } - - static String getValueFromIdByParameterName(String id, String pathTemplate, String parameterName) { - if (id == null || pathTemplate == null) { - return null; - } - String parameterNameParentheses = "{" + parameterName + "}"; - List idSegmentsReverted = Arrays.asList(id.split("/")); - List pathSegments = Arrays.asList(pathTemplate.split("/")); - Collections.reverse(idSegmentsReverted); - Iterator idItrReverted = idSegmentsReverted.iterator(); - int pathIndex = pathSegments.size(); - while (idItrReverted.hasNext() && pathIndex > 0) { - String idSegment = idItrReverted.next(); - String pathSegment = pathSegments.get(--pathIndex); - if (!CoreUtils.isNullOrEmpty(idSegment) && !CoreUtils.isNullOrEmpty(pathSegment)) { - if (pathSegment.equalsIgnoreCase(parameterNameParentheses)) { - if (pathIndex == 0 || (pathIndex == 1 && pathSegments.get(0).isEmpty())) { - List segments = new ArrayList<>(); - segments.add(idSegment); - idItrReverted.forEachRemaining(segments::add); - Collections.reverse(segments); - if (segments.size() > 0 && segments.get(0).isEmpty()) { - segments.remove(0); - } - return String.join("/", segments); - } else { - return idSegment; - } - } - } - } - return null; - } - - static PagedIterable mapPage(PagedIterable pageIterable, Function mapper) { - return new PagedIterableImpl(pageIterable, mapper); - } - - private static final class PagedIterableImpl extends PagedIterable { - - private final PagedIterable pagedIterable; - private final Function mapper; - private final Function, PagedResponse> pageMapper; - - private PagedIterableImpl(PagedIterable pagedIterable, Function mapper) { - super( - PagedFlux - .create( - () -> - (continuationToken, pageSize) -> - Flux.fromStream(pagedIterable.streamByPage().map(getPageMapper(mapper))))); - this.pagedIterable = pagedIterable; - this.mapper = mapper; - this.pageMapper = getPageMapper(mapper); - } - - private static Function, PagedResponse> getPageMapper(Function mapper) { - return page -> - new PagedResponseBase( - page.getRequest(), - page.getStatusCode(), - page.getHeaders(), - page.getElements().stream().map(mapper).collect(Collectors.toList()), - page.getContinuationToken(), - null); - } - - @Override - public Stream stream() { - return pagedIterable.stream().map(mapper); - } - - @Override - public Stream> streamByPage() { - return pagedIterable.streamByPage().map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken) { - return pagedIterable.streamByPage(continuationToken).map(pageMapper); - } - - @Override - public Stream> streamByPage(int preferredPageSize) { - return pagedIterable.streamByPage(preferredPageSize).map(pageMapper); - } - - @Override - public Stream> streamByPage(String continuationToken, int preferredPageSize) { - return pagedIterable.streamByPage(continuationToken, preferredPageSize).map(pageMapper); - } - - @Override - public Iterator iterator() { - return new IteratorImpl(pagedIterable.iterator(), mapper); - } - - @Override - public Iterable> iterableByPage() { - return new IterableImpl, PagedResponse>(pagedIterable.iterableByPage(), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken), pageMapper); - } - - @Override - public Iterable> iterableByPage(int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(preferredPageSize), pageMapper); - } - - @Override - public Iterable> iterableByPage(String continuationToken, int preferredPageSize) { - return new IterableImpl, PagedResponse>( - pagedIterable.iterableByPage(continuationToken, preferredPageSize), pageMapper); - } - } - - private static final class IteratorImpl implements Iterator { - - private final Iterator iterator; - private final Function mapper; - - private IteratorImpl(Iterator iterator, Function mapper) { - this.iterator = iterator; - this.mapper = mapper; - } - - @Override - public boolean hasNext() { - return iterator.hasNext(); - } - - @Override - public S next() { - return mapper.apply(iterator.next()); - } - - @Override - public void remove() { - iterator.remove(); - } - } - - private static final class IterableImpl implements Iterable { - - private final Iterable iterable; - private final Function mapper; - - private IterableImpl(Iterable iterable, Function mapper) { - this.iterable = iterable; - this.mapper = mapper; - } - - @Override - public Iterator iterator() { - return new IteratorImpl(iterable.iterator(), mapper); - } - } -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingImpl.java index 2fdd469eb2601..495178a9739bf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingImpl.java @@ -45,22 +45,14 @@ private com.azure.resourcemanager.security.SecurityManager manager() { private String workspaceSettingName; public WorkspaceSetting create() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaceSettings() - .createWithResponse(workspaceSettingName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaceSettings() + .createWithResponse(workspaceSettingName, this.innerModel(), Context.NONE).getValue(); return this; } public WorkspaceSetting create(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaceSettings() - .createWithResponse(workspaceSettingName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaceSettings() + .createWithResponse(workspaceSettingName, this.innerModel(), context).getValue(); return this; } @@ -75,49 +67,33 @@ public WorkspaceSettingImpl update() { } public WorkspaceSetting apply() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaceSettings() - .updateWithResponse(workspaceSettingName, this.innerModel(), Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaceSettings() + .updateWithResponse(workspaceSettingName, this.innerModel(), Context.NONE).getValue(); return this; } public WorkspaceSetting apply(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaceSettings() - .updateWithResponse(workspaceSettingName, this.innerModel(), context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaceSettings() + .updateWithResponse(workspaceSettingName, this.innerModel(), context).getValue(); return this; } - WorkspaceSettingImpl( - WorkspaceSettingInner innerObject, com.azure.resourcemanager.security.SecurityManager serviceManager) { + WorkspaceSettingImpl(WorkspaceSettingInner innerObject, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerObject = innerObject; this.serviceManager = serviceManager; - this.workspaceSettingName = Utils.getValueFromIdByName(innerObject.id(), "workspaceSettings"); + this.workspaceSettingName = ResourceManagerUtils.getValueFromIdByName(innerObject.id(), "workspaceSettings"); } public WorkspaceSetting refresh() { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaceSettings() - .getWithResponse(workspaceSettingName, Context.NONE) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaceSettings() + .getWithResponse(workspaceSettingName, Context.NONE).getValue(); return this; } public WorkspaceSetting refresh(Context context) { - this.innerObject = - serviceManager - .serviceClient() - .getWorkspaceSettings() - .getWithResponse(workspaceSettingName, context) - .getValue(); + this.innerObject = serviceManager.serviceClient().getWorkspaceSettings() + .getWithResponse(workspaceSettingName, context).getValue(); return this; } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsClientImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsClientImpl.java index 7c0ea16bebaca..0ca76484bc655 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsClientImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsClientImpl.java @@ -34,22 +34,28 @@ import com.azure.resourcemanager.security.models.WorkspaceSettingList; import reactor.core.publisher.Mono; -/** An instance of this class provides access to all the operations defined in WorkspaceSettingsClient. */ +/** + * An instance of this class provides access to all the operations defined in WorkspaceSettingsClient. + */ public final class WorkspaceSettingsClientImpl implements WorkspaceSettingsClient { - /** The proxy service used to perform REST calls. */ + /** + * The proxy service used to perform REST calls. + */ private final WorkspaceSettingsService service; - /** The service client containing this operation class. */ + /** + * The service client containing this operation class. + */ private final SecurityCenterImpl client; /** * Initializes an instance of WorkspaceSettingsClientImpl. - * + * * @param client the instance of the service client containing this operation class. */ WorkspaceSettingsClientImpl(SecurityCenterImpl client) { - this.service = - RestProxy.create(WorkspaceSettingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); + this.service + = RestProxy.create(WorkspaceSettingsService.class, client.getHttpPipeline(), client.getSerializerAdapter()); this.client = client; } @@ -60,165 +66,122 @@ public final class WorkspaceSettingsClientImpl implements WorkspaceSettingsClien @Host("{$host}") @ServiceInterface(name = "SecurityCenterWorksp") public interface WorkspaceSettingsService { - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/workspaceSettings") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> list( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @HeaderParam("Accept") String accept, - Context context); + Mono> list(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("/subscriptions/{subscriptionId}/providers/Microsoft.Security/workspaceSettings/{workspaceSettingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> get( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, - @PathParam("workspaceSettingName") String workspaceSettingName, - @HeaderParam("Accept") String accept, + Mono> get(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, + @PathParam("workspaceSettingName") String workspaceSettingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Put("/subscriptions/{subscriptionId}/providers/Microsoft.Security/workspaceSettings/{workspaceSettingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> create( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> create(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("workspaceSettingName") String workspaceSettingName, - @BodyParam("application/json") WorkspaceSettingInner workspaceSetting, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") WorkspaceSettingInner workspaceSetting, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Patch("/subscriptions/{subscriptionId}/providers/Microsoft.Security/workspaceSettings/{workspaceSettingName}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> update( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, - @PathParam("subscriptionId") String subscriptionId, + Mono> update(@HostParam("$host") String endpoint, + @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, @PathParam("workspaceSettingName") String workspaceSettingName, - @BodyParam("application/json") WorkspaceSettingInner workspaceSetting, - @HeaderParam("Accept") String accept, + @BodyParam("application/json") WorkspaceSettingInner workspaceSetting, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Delete("/subscriptions/{subscriptionId}/providers/Microsoft.Security/workspaceSettings/{workspaceSettingName}") - @ExpectedResponses({204}) + @ExpectedResponses({ 204 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> delete( - @HostParam("$host") String endpoint, - @QueryParam("api-version") String apiVersion, + Mono> delete(@HostParam("$host") String endpoint, @QueryParam("api-version") String apiVersion, @PathParam("subscriptionId") String subscriptionId, - @PathParam("workspaceSettingName") String workspaceSettingName, - @HeaderParam("Accept") String accept, + @PathParam("workspaceSettingName") String workspaceSettingName, @HeaderParam("Accept") String accept, Context context); - @Headers({"Content-Type: application/json"}) + @Headers({ "Content-Type: application/json" }) @Get("{nextLink}") - @ExpectedResponses({200}) + @ExpectedResponses({ 200 }) @UnexpectedResponseExceptionType(ManagementException.class) - Mono> listNext( - @PathParam(value = "nextLink", encoded = true) String nextLink, - @HostParam("$host") String endpoint, - @HeaderParam("Accept") String accept, - Context context); + Mono> listNext(@PathParam(value = "nextLink", encoded = true) String nextLink, + @HostParam("$host") String endpoint, @HeaderParam("Accept") String accept, Context context); } /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of workspace settings response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of workspace settings response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync() { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + .withContext(context -> service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of workspace settings response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of workspace settings response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listSinglePageAsync(Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.list(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of workspace settings response as paginated response with {@link PagedFlux}. @@ -231,7 +194,7 @@ private PagedFlux listAsync() { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -240,14 +203,14 @@ private PagedFlux listAsync() { */ @ServiceMethod(returns = ReturnType.COLLECTION) private PagedFlux listAsync(Context context) { - return new PagedFlux<>( - () -> listSinglePageAsync(context), nextLink -> listNextSinglePageAsync(nextLink, context)); + return new PagedFlux<>(() -> listSinglePageAsync(context), + nextLink -> listNextSinglePageAsync(nextLink, context)); } /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of workspace settings response as paginated response with {@link PagedIterable}. @@ -260,7 +223,7 @@ public PagedIterable list() { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -275,27 +238,23 @@ public PagedIterable list(Context context) { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String workspaceSettingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -304,44 +263,32 @@ private Mono> getWithResponseAsync(String worksp final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - accept, - context)) + .withContext(context -> service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + workspaceSettingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> getWithResponseAsync(String workspaceSettingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -350,26 +297,20 @@ private Mono> getWithResponseAsync(String worksp final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .get( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - accept, - context); + return service.get(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), workspaceSettingName, + accept, context); } /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono getAsync(String workspaceSettingName) { @@ -379,7 +320,7 @@ private Mono getAsync(String workspaceSettingName) { /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -395,7 +336,7 @@ public Response getWithResponse(String workspaceSettingNa /** * Settings about where we should store your security data and logs. If the result is empty, it means that no * custom-workspace configuration was set. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -409,29 +350,25 @@ public WorkspaceSettingInner get(String workspaceSettingName) { /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting) { + private Mono> createWithResponseAsync(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -446,23 +383,14 @@ private Mono> createWithResponseAsync( final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - workspaceSetting, - accept, - context)) + .withContext(context -> service.create(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), workspaceSettingName, workspaceSetting, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @param context The context to associate with this operation. @@ -470,22 +398,18 @@ private Mono> createWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> createWithResponseAsync( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting, Context context) { + private Mono> createWithResponseAsync(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -500,38 +424,31 @@ private Mono> createWithResponseAsync( final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .create( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - workspaceSetting, - accept, - context); + return service.create(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + workspaceSettingName, workspaceSetting, accept, context); } /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono createAsync( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting) { + private Mono createAsync(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting) { return createWithResponseAsync(workspaceSettingName, workspaceSetting) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @param context The context to associate with this operation. @@ -541,14 +458,14 @@ private Mono createAsync( * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response createWithResponse( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting, Context context) { + public Response createWithResponse(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting, Context context) { return createWithResponseAsync(workspaceSettingName, workspaceSetting, context).block(); } /** * creating settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -563,29 +480,25 @@ public WorkspaceSettingInner create(String workspaceSettingName, WorkspaceSettin /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting) { + private Mono> updateWithResponseAsync(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -600,23 +513,14 @@ private Mono> updateWithResponseAsync( final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - workspaceSetting, - accept, - context)) + .withContext(context -> service.update(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), workspaceSettingName, workspaceSetting, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @param context The context to associate with this operation. @@ -624,22 +528,18 @@ private Mono> updateWithResponseAsync( * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response} on - * successful completion of {@link Mono}. + * successful completion of {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono> updateWithResponseAsync( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting, Context context) { + private Mono> updateWithResponseAsync(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -654,38 +554,31 @@ private Mono> updateWithResponseAsync( final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .update( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - workspaceSetting, - accept, - context); + return service.update(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + workspaceSettingName, workspaceSetting, accept, context); } /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return configures where to store the OMS agent data for workspaces under a scope on successful completion of - * {@link Mono}. + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) - private Mono updateAsync( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting) { + private Mono updateAsync(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting) { return updateWithResponseAsync(workspaceSettingName, workspaceSetting) .flatMap(res -> Mono.justOrEmpty(res.getValue())); } /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @param context The context to associate with this operation. @@ -695,14 +588,14 @@ private Mono updateAsync( * @return configures where to store the OMS agent data for workspaces under a scope along with {@link Response}. */ @ServiceMethod(returns = ReturnType.SINGLE) - public Response updateWithResponse( - String workspaceSettingName, WorkspaceSettingInner workspaceSetting, Context context) { + public Response updateWithResponse(String workspaceSettingName, + WorkspaceSettingInner workspaceSetting, Context context) { return updateWithResponseAsync(workspaceSettingName, workspaceSetting, context).block(); } /** * Settings about where we should store your security data and logs. - * + * * @param workspaceSettingName Name of the security setting. * @param workspaceSetting Security data setting object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -717,7 +610,7 @@ public WorkspaceSettingInner update(String workspaceSettingName, WorkspaceSettin /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -727,16 +620,12 @@ public WorkspaceSettingInner update(String workspaceSettingName, WorkspaceSettin @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String workspaceSettingName) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -745,22 +634,14 @@ private Mono> deleteWithResponseAsync(String workspaceSettingName final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; return FluxUtil - .withContext( - context -> - service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - accept, - context)) + .withContext(context -> service.delete(this.client.getEndpoint(), apiVersion, + this.client.getSubscriptionId(), workspaceSettingName, accept, context)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -771,16 +652,12 @@ private Mono> deleteWithResponseAsync(String workspaceSettingName @ServiceMethod(returns = ReturnType.SINGLE) private Mono> deleteWithResponseAsync(String workspaceSettingName, Context context) { if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } if (this.client.getSubscriptionId() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getSubscriptionId() is required and cannot be null.")); + return Mono.error(new IllegalArgumentException( + "Parameter this.client.getSubscriptionId() is required and cannot be null.")); } if (workspaceSettingName == null) { return Mono @@ -789,19 +666,13 @@ private Mono> deleteWithResponseAsync(String workspaceSettingName final String apiVersion = "2017-08-01-preview"; final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .delete( - this.client.getEndpoint(), - apiVersion, - this.client.getSubscriptionId(), - workspaceSettingName, - accept, - context); + return service.delete(this.client.getEndpoint(), apiVersion, this.client.getSubscriptionId(), + workspaceSettingName, accept, context); } /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -815,7 +686,7 @@ private Mono deleteAsync(String workspaceSettingName) { /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -830,7 +701,7 @@ public Response deleteWithResponse(String workspaceSettingName, Context co /** * Deletes the custom workspace settings for this subscription. new VMs will report to the default workspace. - * + * * @param workspaceSettingName Name of the security setting. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. @@ -843,14 +714,15 @@ public void delete(String workspaceSettingName) { /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of workspace settings response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of workspace settings response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink) { @@ -858,37 +730,28 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; - return FluxUtil - .withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) - .>map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)) + return FluxUtil.withContext(context -> service.listNext(nextLink, this.client.getEndpoint(), accept, context)) + .>map(res -> new PagedResponseBase<>(res.getRequest(), + res.getStatusCode(), res.getHeaders(), res.getValue().value(), res.getValue().nextLink(), null)) .contextWrite(context -> context.putAll(FluxUtil.toReactorContext(this.client.getContext()).readOnly())); } /** * Get the next page of items. - * + * * @param nextLink The URL to get the next list of items - *

The nextLink parameter. + * + * The nextLink parameter. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return list of workspace settings response along with {@link PagedResponse} on successful completion of {@link - * Mono}. + * @return list of workspace settings response along with {@link PagedResponse} on successful completion of + * {@link Mono}. */ @ServiceMethod(returns = ReturnType.SINGLE) private Mono> listNextSinglePageAsync(String nextLink, Context context) { @@ -896,23 +759,13 @@ private Mono> listNextSinglePageAsync(Strin return Mono.error(new IllegalArgumentException("Parameter nextLink is required and cannot be null.")); } if (this.client.getEndpoint() == null) { - return Mono - .error( - new IllegalArgumentException( - "Parameter this.client.getEndpoint() is required and cannot be null.")); + return Mono.error( + new IllegalArgumentException("Parameter this.client.getEndpoint() is required and cannot be null.")); } final String accept = "application/json"; context = this.client.mergeContext(context); - return service - .listNext(nextLink, this.client.getEndpoint(), accept, context) - .map( - res -> - new PagedResponseBase<>( - res.getRequest(), - res.getStatusCode(), - res.getHeaders(), - res.getValue().value(), - res.getValue().nextLink(), - null)); + return service.listNext(nextLink, this.client.getEndpoint(), accept, context) + .map(res -> new PagedResponseBase<>(res.getRequest(), res.getStatusCode(), res.getHeaders(), + res.getValue().value(), res.getValue().nextLink(), null)); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsImpl.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsImpl.java index 5917842ee7d16..766cb1dcc73ec 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsImpl.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/WorkspaceSettingsImpl.java @@ -21,29 +21,26 @@ public final class WorkspaceSettingsImpl implements WorkspaceSettings { private final com.azure.resourcemanager.security.SecurityManager serviceManager; - public WorkspaceSettingsImpl( - WorkspaceSettingsClient innerClient, com.azure.resourcemanager.security.SecurityManager serviceManager) { + public WorkspaceSettingsImpl(WorkspaceSettingsClient innerClient, + com.azure.resourcemanager.security.SecurityManager serviceManager) { this.innerClient = innerClient; this.serviceManager = serviceManager; } public PagedIterable list() { PagedIterable inner = this.serviceClient().list(); - return Utils.mapPage(inner, inner1 -> new WorkspaceSettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkspaceSettingImpl(inner1, this.manager())); } public PagedIterable list(Context context) { PagedIterable inner = this.serviceClient().list(context); - return Utils.mapPage(inner, inner1 -> new WorkspaceSettingImpl(inner1, this.manager())); + return ResourceManagerUtils.mapPage(inner, inner1 -> new WorkspaceSettingImpl(inner1, this.manager())); } public Response getWithResponse(String workspaceSettingName, Context context) { Response inner = this.serviceClient().getWithResponse(workspaceSettingName, context); if (inner != null) { - return new SimpleResponse<>( - inner.getRequest(), - inner.getStatusCode(), - inner.getHeaders(), + return new SimpleResponse<>(inner.getRequest(), inner.getStatusCode(), inner.getHeaders(), new WorkspaceSettingImpl(inner.getValue(), this.manager())); } else { return null; @@ -68,53 +65,37 @@ public void delete(String workspaceSettingName) { } public WorkspaceSetting getById(String id) { - String workspaceSettingName = Utils.getValueFromIdByName(id, "workspaceSettings"); + String workspaceSettingName = ResourceManagerUtils.getValueFromIdByName(id, "workspaceSettings"); if (workspaceSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); } return this.getWithResponse(workspaceSettingName, Context.NONE).getValue(); } public Response getByIdWithResponse(String id, Context context) { - String workspaceSettingName = Utils.getValueFromIdByName(id, "workspaceSettings"); + String workspaceSettingName = ResourceManagerUtils.getValueFromIdByName(id, "workspaceSettings"); if (workspaceSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); } return this.getWithResponse(workspaceSettingName, context); } public void deleteById(String id) { - String workspaceSettingName = Utils.getValueFromIdByName(id, "workspaceSettings"); + String workspaceSettingName = ResourceManagerUtils.getValueFromIdByName(id, "workspaceSettings"); if (workspaceSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); } this.deleteWithResponse(workspaceSettingName, Context.NONE); } public Response deleteByIdWithResponse(String id, Context context) { - String workspaceSettingName = Utils.getValueFromIdByName(id, "workspaceSettings"); + String workspaceSettingName = ResourceManagerUtils.getValueFromIdByName(id, "workspaceSettings"); if (workspaceSettingName == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - String - .format( - "The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + String.format("The resource ID '%s' is not valid. Missing path segment 'workspaceSettings'.", id))); } return this.deleteWithResponse(workspaceSettingName, context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/package-info.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/package-info.java index 5a853f731e889..25e2d88d98644 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/package-info.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/implementation/package-info.java @@ -3,7 +3,7 @@ // Code generated by Microsoft (R) AutoRest Code Generator. /** - * Package containing the implementations for SecurityCenter. API spec for Microsoft.Security (Azure Security Center) - * resource provider. + * Package containing the implementations for SecurityCenter. + * API spec for Microsoft.Security (Azure Security Center) resource provider. */ package com.azure.resourcemanager.security.implementation; diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityState.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityState.java index 72e27397e81a3..692f9c353be23 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityState.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityState.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The connectivity state of the external AAD solution. */ +/** + * The connectivity state of the external AAD solution. + */ public final class AadConnectivityState extends ExpandableStringEnum { - /** Static value Discovered for AadConnectivityState. */ + /** + * Static value Discovered for AadConnectivityState. + */ public static final AadConnectivityState DISCOVERED = fromString("Discovered"); - /** Static value NotLicensed for AadConnectivityState. */ + /** + * Static value NotLicensed for AadConnectivityState. + */ public static final AadConnectivityState NOT_LICENSED = fromString("NotLicensed"); - /** Static value Connected for AadConnectivityState. */ + /** + * Static value Connected for AadConnectivityState. + */ public static final AadConnectivityState CONNECTED = fromString("Connected"); /** * Creates a new instance of AadConnectivityState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public AadConnectivityState() { /** * Creates or finds a AadConnectivityState from its string representation. - * + * * @param name a name to look for. * @return the corresponding AadConnectivityState. */ @@ -41,7 +49,7 @@ public static AadConnectivityState fromString(String name) { /** * Gets known AadConnectivityState values. - * + * * @return known AadConnectivityState values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityStateAutoGenerated.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityStateAutoGenerated.java index 6cea9268f7e21..3793ecc192a6f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityStateAutoGenerated.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadConnectivityStateAutoGenerated.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Describes an Azure resource with kind. */ +/** + * Describes an Azure resource with kind. + */ @Fluent public class AadConnectivityStateAutoGenerated { /* @@ -16,13 +18,15 @@ public class AadConnectivityStateAutoGenerated { @JsonProperty(value = "connectivityState") private AadConnectivityState connectivityState; - /** Creates an instance of AadConnectivityStateAutoGenerated class. */ + /** + * Creates an instance of AadConnectivityStateAutoGenerated class. + */ public AadConnectivityStateAutoGenerated() { } /** * Get the connectivityState property: The connectivity state of the external AAD solution. - * + * * @return the connectivityState value. */ public AadConnectivityState connectivityState() { @@ -31,7 +35,7 @@ public AadConnectivityState connectivityState() { /** * Set the connectivityState property: The connectivity state of the external AAD solution. - * + * * @param connectivityState the connectivityState value to set. * @return the AadConnectivityStateAutoGenerated object itself. */ @@ -42,7 +46,7 @@ public AadConnectivityStateAutoGenerated withConnectivityState(AadConnectivitySt /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadExternalSecuritySolution.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadExternalSecuritySolution.java index 400b5421f072c..e681f71836317 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadExternalSecuritySolution.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadExternalSecuritySolution.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Represents an AAD identity protection solution which sends logs to an OMS workspace. */ +/** + * Represents an AAD identity protection solution which sends logs to an OMS workspace. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") @JsonTypeName("AAD") @Fluent @@ -21,13 +23,15 @@ public final class AadExternalSecuritySolution extends ExternalSecuritySolutionI @JsonProperty(value = "properties") private AadSolutionProperties properties; - /** Creates an instance of AadExternalSecuritySolution class. */ + /** + * Creates an instance of AadExternalSecuritySolution class. + */ public AadExternalSecuritySolution() { } /** * Get the properties property: The external security solution properties for AAD solutions. - * + * * @return the properties value. */ public AadSolutionProperties properties() { @@ -36,7 +40,7 @@ public AadSolutionProperties properties() { /** * Set the properties property: The external security solution properties for AAD solutions. - * + * * @param properties the properties value to set. * @return the AadExternalSecuritySolution object itself. */ @@ -47,7 +51,7 @@ public AadExternalSecuritySolution withProperties(AadSolutionProperties properti /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadSolutionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadSolutionProperties.java index 66c9643e1e5e5..19b2c7df3f4e9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadSolutionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AadSolutionProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The external security solution properties for AAD solutions. */ +/** + * The external security solution properties for AAD solutions. + */ @Fluent public final class AadSolutionProperties extends ExternalSecuritySolutionProperties { /* @@ -16,13 +18,15 @@ public final class AadSolutionProperties extends ExternalSecuritySolutionPropert @JsonProperty(value = "connectivityState") private AadConnectivityState connectivityState; - /** Creates an instance of AadSolutionProperties class. */ + /** + * Creates an instance of AadSolutionProperties class. + */ public AadSolutionProperties() { } /** * Get the connectivityState property: The connectivity state of the external AAD solution. - * + * * @return the connectivityState value. */ public AadConnectivityState connectivityState() { @@ -31,7 +35,7 @@ public AadConnectivityState connectivityState() { /** * Set the connectivityState property: The connectivity state of the external AAD solution. - * + * * @param connectivityState the connectivityState value to set. * @return the AadSolutionProperties object itself. */ @@ -40,21 +44,27 @@ public AadSolutionProperties withConnectivityState(AadConnectivityState connecti return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AadSolutionProperties withDeviceVendor(String deviceVendor) { super.withDeviceVendor(deviceVendor); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AadSolutionProperties withDeviceType(String deviceType) { super.withDeviceType(deviceType); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AadSolutionProperties withWorkspace(ConnectedWorkspace workspace) { super.withWorkspace(workspace); @@ -63,7 +73,7 @@ public AadSolutionProperties withWorkspace(ConnectedWorkspace workspace) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionType.java index 9f8d0c26c7956..e10d2e3b73718 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionType.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The type of the action that will be triggered by the Automation. */ +/** + * The type of the action that will be triggered by the Automation. + */ public final class ActionType extends ExpandableStringEnum { - /** Static value LogicApp for ActionType. */ + /** + * Static value LogicApp for ActionType. + */ public static final ActionType LOGIC_APP = fromString("LogicApp"); - /** Static value EventHub for ActionType. */ + /** + * Static value EventHub for ActionType. + */ public static final ActionType EVENT_HUB = fromString("EventHub"); - /** Static value Workspace for ActionType. */ + /** + * Static value Workspace for ActionType. + */ public static final ActionType WORKSPACE = fromString("Workspace"); /** * Creates a new instance of ActionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public ActionType() { /** * Creates or finds a ActionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ActionType. */ @@ -41,7 +49,7 @@ public static ActionType fromString(String name) { /** * Gets known ActionType values. - * + * * @return known ActionType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediation.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediation.java new file mode 100644 index 0000000000000..38c754d0aa607 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediation.java @@ -0,0 +1,157 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Configuration payload for PR Annotations. + */ +@Fluent +public final class ActionableRemediation { + /* + * ActionableRemediation Setting. + * None - the setting was never set. + * Enabled - ActionableRemediation is enabled. + * Disabled - ActionableRemediation is disabled. + */ + @JsonProperty(value = "state") + private ActionableRemediationState state; + + /* + * Gets or sets list of categories and severity levels. + */ + @JsonProperty(value = "categoryConfigurations") + private List categoryConfigurations; + + /* + * Repository branch configuration for PR Annotations. + */ + @JsonProperty(value = "branchConfiguration") + private TargetBranchConfiguration branchConfiguration; + + /* + * Update Settings. + * + * Enabled - Resource should inherit configurations from parent. + * Disabled - Resource should not inherit configurations from parent. + */ + @JsonProperty(value = "inheritFromParentState") + private InheritFromParentState inheritFromParentState; + + /** + * Creates an instance of ActionableRemediation class. + */ + public ActionableRemediation() { + } + + /** + * Get the state property: ActionableRemediation Setting. + * None - the setting was never set. + * Enabled - ActionableRemediation is enabled. + * Disabled - ActionableRemediation is disabled. + * + * @return the state value. + */ + public ActionableRemediationState state() { + return this.state; + } + + /** + * Set the state property: ActionableRemediation Setting. + * None - the setting was never set. + * Enabled - ActionableRemediation is enabled. + * Disabled - ActionableRemediation is disabled. + * + * @param state the state value to set. + * @return the ActionableRemediation object itself. + */ + public ActionableRemediation withState(ActionableRemediationState state) { + this.state = state; + return this; + } + + /** + * Get the categoryConfigurations property: Gets or sets list of categories and severity levels. + * + * @return the categoryConfigurations value. + */ + public List categoryConfigurations() { + return this.categoryConfigurations; + } + + /** + * Set the categoryConfigurations property: Gets or sets list of categories and severity levels. + * + * @param categoryConfigurations the categoryConfigurations value to set. + * @return the ActionableRemediation object itself. + */ + public ActionableRemediation withCategoryConfigurations(List categoryConfigurations) { + this.categoryConfigurations = categoryConfigurations; + return this; + } + + /** + * Get the branchConfiguration property: Repository branch configuration for PR Annotations. + * + * @return the branchConfiguration value. + */ + public TargetBranchConfiguration branchConfiguration() { + return this.branchConfiguration; + } + + /** + * Set the branchConfiguration property: Repository branch configuration for PR Annotations. + * + * @param branchConfiguration the branchConfiguration value to set. + * @return the ActionableRemediation object itself. + */ + public ActionableRemediation withBranchConfiguration(TargetBranchConfiguration branchConfiguration) { + this.branchConfiguration = branchConfiguration; + return this; + } + + /** + * Get the inheritFromParentState property: Update Settings. + * + * Enabled - Resource should inherit configurations from parent. + * Disabled - Resource should not inherit configurations from parent. + * + * @return the inheritFromParentState value. + */ + public InheritFromParentState inheritFromParentState() { + return this.inheritFromParentState; + } + + /** + * Set the inheritFromParentState property: Update Settings. + * + * Enabled - Resource should inherit configurations from parent. + * Disabled - Resource should not inherit configurations from parent. + * + * @param inheritFromParentState the inheritFromParentState value to set. + * @return the ActionableRemediation object itself. + */ + public ActionableRemediation withInheritFromParentState(InheritFromParentState inheritFromParentState) { + this.inheritFromParentState = inheritFromParentState; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (categoryConfigurations() != null) { + categoryConfigurations().forEach(e -> e.validate()); + } + if (branchConfiguration() != null) { + branchConfiguration().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediationState.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediationState.java new file mode 100644 index 0000000000000..3a7ee9c6462b4 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActionableRemediationState.java @@ -0,0 +1,61 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * ActionableRemediation Setting. + * None - the setting was never set. + * Enabled - ActionableRemediation is enabled. + * Disabled - ActionableRemediation is disabled. + */ +public final class ActionableRemediationState extends ExpandableStringEnum { + /** + * Static value None for ActionableRemediationState. + */ + public static final ActionableRemediationState NONE = fromString("None"); + + /** + * Static value Disabled for ActionableRemediationState. + */ + public static final ActionableRemediationState DISABLED = fromString("Disabled"); + + /** + * Static value Enabled for ActionableRemediationState. + */ + public static final ActionableRemediationState ENABLED = fromString("Enabled"); + + /** + * Creates a new instance of ActionableRemediationState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public ActionableRemediationState() { + } + + /** + * Creates or finds a ActionableRemediationState from its string representation. + * + * @param name a name to look for. + * @return the corresponding ActionableRemediationState. + */ + @JsonCreator + public static ActionableRemediationState fromString(String name) { + return fromString(name, ActionableRemediationState.class); + } + + /** + * Gets known ActionableRemediationState values. + * + * @return known ActionableRemediationState values. + */ + public static Collection values() { + return values(ActionableRemediationState.class); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActiveConnectionsNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActiveConnectionsNotInAllowedRange.java index b8f0145eb2882..6b7905f7e4d52 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActiveConnectionsNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ActiveConnectionsNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of active connections is not in allowed range. */ +/** + * Number of active connections is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("ActiveConnectionsNotInAllowedRange") @Fluent public final class ActiveConnectionsNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of ActiveConnectionsNotInAllowedRange class. */ + /** + * Creates an instance of ActiveConnectionsNotInAllowedRange class. + */ public ActiveConnectionsNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ActiveConnectionsNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ActiveConnectionsNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ActiveConnectionsNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ActiveConnectionsNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public ActiveConnectionsNotInAllowedRange withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroup.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroup.java index cb789472d0892..7f2d4319197a1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroup.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroup.java @@ -9,32 +9,34 @@ import com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupInner; import java.util.List; -/** An immutable client-side representation of AdaptiveApplicationControlGroup. */ +/** + * An immutable client-side representation of AdaptiveApplicationControlGroup. + */ public interface AdaptiveApplicationControlGroup { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: Location where the resource is stored. - * + * * @return the location value. */ String location(); @@ -42,7 +44,7 @@ public interface AdaptiveApplicationControlGroup { /** * Gets the enforcementMode property: The application control policy enforcement/protection mode of the machine * group. - * + * * @return the enforcementMode value. */ EnforcementMode enforcementMode(); @@ -50,246 +52,281 @@ public interface AdaptiveApplicationControlGroup { /** * Gets the protectionMode property: The protection mode of the collection/file types. Exe/Msi/Script are used for * Windows, Executable is used for Linux. - * + * * @return the protectionMode value. */ ProtectionMode protectionMode(); /** * Gets the configurationStatus property: The configuration status of the machines group or machine or rule. - * + * * @return the configurationStatus value. */ ConfigurationStatus configurationStatus(); /** * Gets the recommendationStatus property: The initial recommendation status of the machine group or machine. - * + * * @return the recommendationStatus value. */ RecommendationStatus recommendationStatus(); /** * Gets the issues property: The issues property. - * + * * @return the issues value. */ List issues(); /** * Gets the sourceSystem property: The source type of the machine group. - * + * * @return the sourceSystem value. */ SourceSystem sourceSystem(); /** * Gets the vmRecommendations property: The vmRecommendations property. - * + * * @return the vmRecommendations value. */ List vmRecommendations(); /** * Gets the pathRecommendations property: The pathRecommendations property. - * + * * @return the pathRecommendations value. */ List pathRecommendations(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupInner object. - * + * * @return the inner object. */ AdaptiveApplicationControlGroupInner innerModel(); - /** The entirety of the AdaptiveApplicationControlGroup definition. */ + /** + * The entirety of the AdaptiveApplicationControlGroup definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { } - /** The AdaptiveApplicationControlGroup definition stages. */ + + /** + * The AdaptiveApplicationControlGroup definition stages. + */ interface DefinitionStages { - /** The first stage of the AdaptiveApplicationControlGroup definition. */ + /** + * The first stage of the AdaptiveApplicationControlGroup definition. + */ interface Blank extends WithParentResource { } - /** The stage of the AdaptiveApplicationControlGroup definition allowing to specify parent resource. */ + + /** + * The stage of the AdaptiveApplicationControlGroup definition allowing to specify parent resource. + */ interface WithParentResource { /** * Specifies ascLocation. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @return the next definition stage. */ WithCreate withExistingLocation(String ascLocation); } + /** * The stage of the AdaptiveApplicationControlGroup definition which contains all the minimum required * properties for the resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithEnforcementMode, - DefinitionStages.WithProtectionMode, - DefinitionStages.WithVmRecommendations, - DefinitionStages.WithPathRecommendations { + interface WithCreate extends DefinitionStages.WithEnforcementMode, DefinitionStages.WithProtectionMode, + DefinitionStages.WithVmRecommendations, DefinitionStages.WithPathRecommendations { /** * Executes the create request. - * + * * @return the created resource. */ AdaptiveApplicationControlGroup create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ AdaptiveApplicationControlGroup create(Context context); } - /** The stage of the AdaptiveApplicationControlGroup definition allowing to specify enforcementMode. */ + + /** + * The stage of the AdaptiveApplicationControlGroup definition allowing to specify enforcementMode. + */ interface WithEnforcementMode { /** * Specifies the enforcementMode property: The application control policy enforcement/protection mode of the * machine group. - * + * * @param enforcementMode The application control policy enforcement/protection mode of the machine group. * @return the next definition stage. */ WithCreate withEnforcementMode(EnforcementMode enforcementMode); } - /** The stage of the AdaptiveApplicationControlGroup definition allowing to specify protectionMode. */ + + /** + * The stage of the AdaptiveApplicationControlGroup definition allowing to specify protectionMode. + */ interface WithProtectionMode { /** * Specifies the protectionMode property: The protection mode of the collection/file types. Exe/Msi/Script * are used for Windows, Executable is used for Linux.. - * + * * @param protectionMode The protection mode of the collection/file types. Exe/Msi/Script are used for - * Windows, Executable is used for Linux. + * Windows, Executable is used for Linux. * @return the next definition stage. */ WithCreate withProtectionMode(ProtectionMode protectionMode); } - /** The stage of the AdaptiveApplicationControlGroup definition allowing to specify vmRecommendations. */ + + /** + * The stage of the AdaptiveApplicationControlGroup definition allowing to specify vmRecommendations. + */ interface WithVmRecommendations { /** * Specifies the vmRecommendations property: The vmRecommendations property.. - * + * * @param vmRecommendations The vmRecommendations property. * @return the next definition stage. */ WithCreate withVmRecommendations(List vmRecommendations); } - /** The stage of the AdaptiveApplicationControlGroup definition allowing to specify pathRecommendations. */ + + /** + * The stage of the AdaptiveApplicationControlGroup definition allowing to specify pathRecommendations. + */ interface WithPathRecommendations { /** * Specifies the pathRecommendations property: The pathRecommendations property.. - * + * * @param pathRecommendations The pathRecommendations property. * @return the next definition stage. */ WithCreate withPathRecommendations(List pathRecommendations); } } + /** * Begins update for the AdaptiveApplicationControlGroup resource. - * + * * @return the stage of resource update. */ AdaptiveApplicationControlGroup.Update update(); - /** The template for AdaptiveApplicationControlGroup update. */ - interface Update - extends UpdateStages.WithEnforcementMode, - UpdateStages.WithProtectionMode, - UpdateStages.WithVmRecommendations, - UpdateStages.WithPathRecommendations { + /** + * The template for AdaptiveApplicationControlGroup update. + */ + interface Update extends UpdateStages.WithEnforcementMode, UpdateStages.WithProtectionMode, + UpdateStages.WithVmRecommendations, UpdateStages.WithPathRecommendations { /** * Executes the update request. - * + * * @return the updated resource. */ AdaptiveApplicationControlGroup apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ AdaptiveApplicationControlGroup apply(Context context); } - /** The AdaptiveApplicationControlGroup update stages. */ + + /** + * The AdaptiveApplicationControlGroup update stages. + */ interface UpdateStages { - /** The stage of the AdaptiveApplicationControlGroup update allowing to specify enforcementMode. */ + /** + * The stage of the AdaptiveApplicationControlGroup update allowing to specify enforcementMode. + */ interface WithEnforcementMode { /** * Specifies the enforcementMode property: The application control policy enforcement/protection mode of the * machine group. - * + * * @param enforcementMode The application control policy enforcement/protection mode of the machine group. * @return the next definition stage. */ Update withEnforcementMode(EnforcementMode enforcementMode); } - /** The stage of the AdaptiveApplicationControlGroup update allowing to specify protectionMode. */ + + /** + * The stage of the AdaptiveApplicationControlGroup update allowing to specify protectionMode. + */ interface WithProtectionMode { /** * Specifies the protectionMode property: The protection mode of the collection/file types. Exe/Msi/Script * are used for Windows, Executable is used for Linux.. - * + * * @param protectionMode The protection mode of the collection/file types. Exe/Msi/Script are used for - * Windows, Executable is used for Linux. + * Windows, Executable is used for Linux. * @return the next definition stage. */ Update withProtectionMode(ProtectionMode protectionMode); } - /** The stage of the AdaptiveApplicationControlGroup update allowing to specify vmRecommendations. */ + + /** + * The stage of the AdaptiveApplicationControlGroup update allowing to specify vmRecommendations. + */ interface WithVmRecommendations { /** * Specifies the vmRecommendations property: The vmRecommendations property.. - * + * * @param vmRecommendations The vmRecommendations property. * @return the next definition stage. */ Update withVmRecommendations(List vmRecommendations); } - /** The stage of the AdaptiveApplicationControlGroup update allowing to specify pathRecommendations. */ + + /** + * The stage of the AdaptiveApplicationControlGroup update allowing to specify pathRecommendations. + */ interface WithPathRecommendations { /** * Specifies the pathRecommendations property: The pathRecommendations property.. - * + * * @param pathRecommendations The pathRecommendations property. * @return the next definition stage. */ Update withPathRecommendations(List pathRecommendations); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ AdaptiveApplicationControlGroup refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroups.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroups.java index a394948e92051..259c22a7be243 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroups.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlGroups.java @@ -7,18 +7,20 @@ import com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupsInner; import java.util.List; -/** An immutable client-side representation of AdaptiveApplicationControlGroups. */ +/** + * An immutable client-side representation of AdaptiveApplicationControlGroups. + */ public interface AdaptiveApplicationControlGroups { /** * Gets the value property: The value property. - * + * * @return the value value. */ List value(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AdaptiveApplicationControlGroupsInner object. - * + * * @return the inner object. */ AdaptiveApplicationControlGroupsInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssue.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssue.java index a4c59fbd56e6c..660a6b156bc5c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssue.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssue.java @@ -8,32 +8,46 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** An alert that machines within a group can have. */ +/** + * An alert that machines within a group can have. + */ public final class AdaptiveApplicationControlIssue extends ExpandableStringEnum { - /** Static value ViolationsAudited for AdaptiveApplicationControlIssue. */ + /** + * Static value ViolationsAudited for AdaptiveApplicationControlIssue. + */ public static final AdaptiveApplicationControlIssue VIOLATIONS_AUDITED = fromString("ViolationsAudited"); - /** Static value ViolationsBlocked for AdaptiveApplicationControlIssue. */ + /** + * Static value ViolationsBlocked for AdaptiveApplicationControlIssue. + */ public static final AdaptiveApplicationControlIssue VIOLATIONS_BLOCKED = fromString("ViolationsBlocked"); - /** Static value MsiAndScriptViolationsAudited for AdaptiveApplicationControlIssue. */ - public static final AdaptiveApplicationControlIssue MSI_AND_SCRIPT_VIOLATIONS_AUDITED = - fromString("MsiAndScriptViolationsAudited"); + /** + * Static value MsiAndScriptViolationsAudited for AdaptiveApplicationControlIssue. + */ + public static final AdaptiveApplicationControlIssue MSI_AND_SCRIPT_VIOLATIONS_AUDITED + = fromString("MsiAndScriptViolationsAudited"); - /** Static value MsiAndScriptViolationsBlocked for AdaptiveApplicationControlIssue. */ - public static final AdaptiveApplicationControlIssue MSI_AND_SCRIPT_VIOLATIONS_BLOCKED = - fromString("MsiAndScriptViolationsBlocked"); + /** + * Static value MsiAndScriptViolationsBlocked for AdaptiveApplicationControlIssue. + */ + public static final AdaptiveApplicationControlIssue MSI_AND_SCRIPT_VIOLATIONS_BLOCKED + = fromString("MsiAndScriptViolationsBlocked"); - /** Static value ExecutableViolationsAudited for AdaptiveApplicationControlIssue. */ - public static final AdaptiveApplicationControlIssue EXECUTABLE_VIOLATIONS_AUDITED = - fromString("ExecutableViolationsAudited"); + /** + * Static value ExecutableViolationsAudited for AdaptiveApplicationControlIssue. + */ + public static final AdaptiveApplicationControlIssue EXECUTABLE_VIOLATIONS_AUDITED + = fromString("ExecutableViolationsAudited"); - /** Static value RulesViolatedManually for AdaptiveApplicationControlIssue. */ + /** + * Static value RulesViolatedManually for AdaptiveApplicationControlIssue. + */ public static final AdaptiveApplicationControlIssue RULES_VIOLATED_MANUALLY = fromString("RulesViolatedManually"); /** * Creates a new instance of AdaptiveApplicationControlIssue value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -42,7 +56,7 @@ public AdaptiveApplicationControlIssue() { /** * Creates or finds a AdaptiveApplicationControlIssue from its string representation. - * + * * @param name a name to look for. * @return the corresponding AdaptiveApplicationControlIssue. */ @@ -53,7 +67,7 @@ public static AdaptiveApplicationControlIssue fromString(String name) { /** * Gets known AdaptiveApplicationControlIssue values. - * + * * @return known AdaptiveApplicationControlIssue values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssueSummary.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssueSummary.java index 57abcac239b7e..dedc12ab3abfb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssueSummary.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControlIssueSummary.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents a summary of the alerts of the machine group. */ +/** + * Represents a summary of the alerts of the machine group. + */ @Fluent public final class AdaptiveApplicationControlIssueSummary { /* @@ -22,13 +24,15 @@ public final class AdaptiveApplicationControlIssueSummary { @JsonProperty(value = "numberOfVms") private Float numberOfVms; - /** Creates an instance of AdaptiveApplicationControlIssueSummary class. */ + /** + * Creates an instance of AdaptiveApplicationControlIssueSummary class. + */ public AdaptiveApplicationControlIssueSummary() { } /** * Get the issue property: An alert that machines within a group can have. - * + * * @return the issue value. */ public AdaptiveApplicationControlIssue issue() { @@ -37,7 +41,7 @@ public AdaptiveApplicationControlIssue issue() { /** * Set the issue property: An alert that machines within a group can have. - * + * * @param issue the issue value to set. * @return the AdaptiveApplicationControlIssueSummary object itself. */ @@ -48,7 +52,7 @@ public AdaptiveApplicationControlIssueSummary withIssue(AdaptiveApplicationContr /** * Get the numberOfVms property: The number of machines in the group that have this alert. - * + * * @return the numberOfVms value. */ public Float numberOfVms() { @@ -57,7 +61,7 @@ public Float numberOfVms() { /** * Set the numberOfVms property: The number of machines in the group that have this alert. - * + * * @param numberOfVms the numberOfVms value to set. * @return the AdaptiveApplicationControlIssueSummary object itself. */ @@ -68,7 +72,7 @@ public AdaptiveApplicationControlIssueSummary withNumberOfVms(Float numberOfVms) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControls.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControls.java index e2584d847eb74..f6dc614ced7c3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControls.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveApplicationControls.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of AdaptiveApplicationControls. */ +/** + * Resource collection API of AdaptiveApplicationControls. + */ public interface AdaptiveApplicationControls { /** * Gets a list of application control machine groups for the subscription. - * + * * @param includePathRecommendations Include the policy rules. * @param summary Return output in a summarized form. * @param context The context to associate with this operation. @@ -20,12 +22,12 @@ public interface AdaptiveApplicationControls { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of application control machine groups for the subscription along with {@link Response}. */ - Response listWithResponse( - Boolean includePathRecommendations, Boolean summary, Context context); + Response listWithResponse(Boolean includePathRecommendations, Boolean summary, + Context context); /** * Gets a list of application control machine groups for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of application control machine groups for the subscription. @@ -34,9 +36,9 @@ Response listWithResponse( /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -48,9 +50,9 @@ Response listWithResponse( /** * Gets an application control VM/server group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -61,9 +63,9 @@ Response listWithResponse( /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -75,9 +77,9 @@ Response listWithResponse( /** * Delete an application control machine group. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param groupName Name of an application control machine group. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -87,7 +89,7 @@ Response listWithResponse( /** * Gets an application control VM/server group. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -98,7 +100,7 @@ Response listWithResponse( /** * Gets an application control VM/server group. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -110,7 +112,7 @@ Response listWithResponse( /** * Delete an application control machine group. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -120,7 +122,7 @@ Response listWithResponse( /** * Delete an application control machine group. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -132,7 +134,7 @@ Response listWithResponse( /** * Begins definition for a new AdaptiveApplicationControlGroup resource. - * + * * @param name resource name. * @return the first stage of the new AdaptiveApplicationControlGroup definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardening.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardening.java index 8cd1c579fa38c..7909db507d640 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardening.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardening.java @@ -8,39 +8,41 @@ import java.time.OffsetDateTime; import java.util.List; -/** An immutable client-side representation of AdaptiveNetworkHardening. */ +/** + * An immutable client-side representation of AdaptiveNetworkHardening. + */ public interface AdaptiveNetworkHardening { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the rules property: The security rules which are recommended to be effective on the VM. - * + * * @return the rules value. */ List rules(); /** * Gets the rulesCalculationTime property: The UTC time on which the rules were calculated. - * + * * @return the rulesCalculationTime value. */ OffsetDateTime rulesCalculationTime(); @@ -48,14 +50,14 @@ public interface AdaptiveNetworkHardening { /** * Gets the effectiveNetworkSecurityGroups property: The Network Security Groups effective on the network interfaces * of the protected resource. - * + * * @return the effectiveNetworkSecurityGroups value. */ List effectiveNetworkSecurityGroups(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AdaptiveNetworkHardeningInner object. - * + * * @return the inner object. */ AdaptiveNetworkHardeningInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningEnforceRequest.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningEnforceRequest.java index 0c6dfdd280f70..0e95b9a6fe2dc 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningEnforceRequest.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningEnforceRequest.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The AdaptiveNetworkHardeningEnforceRequest model. */ +/** + * The AdaptiveNetworkHardeningEnforceRequest model. + */ @Fluent public final class AdaptiveNetworkHardeningEnforceRequest { /* @@ -25,13 +27,15 @@ public final class AdaptiveNetworkHardeningEnforceRequest { @JsonProperty(value = "networkSecurityGroups", required = true) private List networkSecurityGroups; - /** Creates an instance of AdaptiveNetworkHardeningEnforceRequest class. */ + /** + * Creates an instance of AdaptiveNetworkHardeningEnforceRequest class. + */ public AdaptiveNetworkHardeningEnforceRequest() { } /** * Get the rules property: The rules to enforce. - * + * * @return the rules value. */ public List rules() { @@ -40,7 +44,7 @@ public List rules() { /** * Set the rules property: The rules to enforce. - * + * * @param rules the rules value to set. * @return the AdaptiveNetworkHardeningEnforceRequest object itself. */ @@ -50,9 +54,9 @@ public AdaptiveNetworkHardeningEnforceRequest withRules(List rules) { } /** - * Get the networkSecurityGroups property: The Azure resource IDs of the effective network security groups that will - * be updated with the created security rules from the Adaptive Network Hardening rules. - * + * Get the networkSecurityGroups property: The Azure resource IDs of the effective network security groups that + * will be updated with the created security rules from the Adaptive Network Hardening rules. + * * @return the networkSecurityGroups value. */ public List networkSecurityGroups() { @@ -60,9 +64,9 @@ public List networkSecurityGroups() { } /** - * Set the networkSecurityGroups property: The Azure resource IDs of the effective network security groups that will - * be updated with the created security rules from the Adaptive Network Hardening rules. - * + * Set the networkSecurityGroups property: The Azure resource IDs of the effective network security groups that + * will be updated with the created security rules from the Adaptive Network Hardening rules. + * * @param networkSecurityGroups the networkSecurityGroups value to set. * @return the AdaptiveNetworkHardeningEnforceRequest object itself. */ @@ -73,24 +77,19 @@ public AdaptiveNetworkHardeningEnforceRequest withNetworkSecurityGroups(List e.validate()); } if (networkSecurityGroups() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property networkSecurityGroups in model" - + " AdaptiveNetworkHardeningEnforceRequest")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property networkSecurityGroups in model AdaptiveNetworkHardeningEnforceRequest")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardenings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardenings.java index 735512030947b..2adb4a994600a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardenings.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardenings.java @@ -8,13 +8,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of AdaptiveNetworkHardenings. */ +/** + * Resource collection API of AdaptiveNetworkHardenings. + */ public interface AdaptiveNetworkHardenings { /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -22,16 +24,16 @@ public interface AdaptiveNetworkHardenings { * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ - PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName); + PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName); /** * Gets a list of Adaptive Network Hardenings resources in scope of an extended resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -40,16 +42,16 @@ PagedIterable listByExtendedResource( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Adaptive Network Hardenings resources in scope of an extended resource as paginated response - * with {@link PagedIterable}. + * with {@link PagedIterable}. */ - PagedIterable listByExtendedResource( - String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, Context context); + PagedIterable listByExtendedResource(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, Context context); /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -60,19 +62,14 @@ PagedIterable listByExtendedResource( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single Adaptive Network Hardening resource along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - Context context); + Response getWithResponse(String resourceGroupName, String resourceNamespace, + String resourceType, String resourceName, String adaptiveNetworkHardeningResourceName, Context context); /** * Gets a single Adaptive Network Hardening resource. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -82,18 +79,14 @@ Response getWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single Adaptive Network Hardening resource. */ - AdaptiveNetworkHardening get( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName); + AdaptiveNetworkHardening get(String resourceGroupName, String resourceNamespace, String resourceType, + String resourceName, String adaptiveNetworkHardeningResourceName); /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -103,19 +96,14 @@ AdaptiveNetworkHardening get( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body); + void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body); /** * Enforces the given rules on the NSG(s) listed in the request. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param resourceNamespace The Namespace of the resource. * @param resourceType The type of the resource. * @param resourceName Name of the resource. @@ -126,12 +114,6 @@ void enforce( * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. */ - void enforce( - String resourceGroupName, - String resourceNamespace, - String resourceType, - String resourceName, - String adaptiveNetworkHardeningResourceName, - AdaptiveNetworkHardeningEnforceRequest body, - Context context); + void enforce(String resourceGroupName, String resourceNamespace, String resourceType, String resourceName, + String adaptiveNetworkHardeningResourceName, AdaptiveNetworkHardeningEnforceRequest body, Context context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningsList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningsList.java index f4c80bcc0897b..45483b050a143 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningsList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdaptiveNetworkHardeningsList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Response for ListAdaptiveNetworkHardenings API service call. */ +/** + * Response for ListAdaptiveNetworkHardenings API service call. + */ @Fluent public final class AdaptiveNetworkHardeningsList { /* @@ -24,13 +26,15 @@ public final class AdaptiveNetworkHardeningsList { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of AdaptiveNetworkHardeningsList class. */ + /** + * Creates an instance of AdaptiveNetworkHardeningsList class. + */ public AdaptiveNetworkHardeningsList() { } /** * Get the value property: A list of Adaptive Network Hardenings resources. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: A list of Adaptive Network Hardenings resources. - * + * * @param value the value value to set. * @return the AdaptiveNetworkHardeningsList object itself. */ @@ -50,7 +54,7 @@ public AdaptiveNetworkHardeningsList withValue(List { - /** Static value Alerts for AdditionalWorkspaceDataType. */ + /** + * Static value Alerts for AdditionalWorkspaceDataType. + */ public static final AdditionalWorkspaceDataType ALERTS = fromString("Alerts"); - /** Static value RawEvents for AdditionalWorkspaceDataType. */ + /** + * Static value RawEvents for AdditionalWorkspaceDataType. + */ public static final AdditionalWorkspaceDataType RAW_EVENTS = fromString("RawEvents"); /** * Creates a new instance of AdditionalWorkspaceDataType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AdditionalWorkspaceDataType() { /** * Creates or finds a AdditionalWorkspaceDataType from its string representation. - * + * * @param name a name to look for. * @return the corresponding AdditionalWorkspaceDataType. */ @@ -38,7 +44,7 @@ public static AdditionalWorkspaceDataType fromString(String name) { /** * Gets known AdditionalWorkspaceDataType values. - * + * * @return known AdditionalWorkspaceDataType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspaceType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspaceType.java index e893e00c1bbcf..51f98a039b2ba 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspaceType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspaceType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Workspace type. */ +/** + * Workspace type. + */ public final class AdditionalWorkspaceType extends ExpandableStringEnum { - /** Static value Sentinel for AdditionalWorkspaceType. */ + /** + * Static value Sentinel for AdditionalWorkspaceType. + */ public static final AdditionalWorkspaceType SENTINEL = fromString("Sentinel"); /** * Creates a new instance of AdditionalWorkspaceType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public AdditionalWorkspaceType() { /** * Creates or finds a AdditionalWorkspaceType from its string representation. - * + * * @param name a name to look for. * @return the corresponding AdditionalWorkspaceType. */ @@ -35,7 +39,7 @@ public static AdditionalWorkspaceType fromString(String name) { /** * Gets known AdditionalWorkspaceType values. - * + * * @return known AdditionalWorkspaceType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspacesProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspacesProperties.java index 7cd4c486ec2d6..83f3eef478b31 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspacesProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AdditionalWorkspacesProperties.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Properties of the additional workspaces. */ +/** + * Properties of the additional workspaces. + */ @Fluent public final class AdditionalWorkspacesProperties { /* @@ -29,13 +31,15 @@ public final class AdditionalWorkspacesProperties { @JsonProperty(value = "dataTypes") private List dataTypes; - /** Creates an instance of AdditionalWorkspacesProperties class. */ + /** + * Creates an instance of AdditionalWorkspacesProperties class. + */ public AdditionalWorkspacesProperties() { } /** * Get the workspace property: Workspace resource id. - * + * * @return the workspace value. */ public String workspace() { @@ -44,7 +48,7 @@ public String workspace() { /** * Set the workspace property: Workspace resource id. - * + * * @param workspace the workspace value to set. * @return the AdditionalWorkspacesProperties object itself. */ @@ -55,7 +59,7 @@ public AdditionalWorkspacesProperties withWorkspace(String workspace) { /** * Get the type property: Workspace type. - * + * * @return the type value. */ public AdditionalWorkspaceType type() { @@ -64,7 +68,7 @@ public AdditionalWorkspaceType type() { /** * Set the type property: Workspace type. - * + * * @param type the type value to set. * @return the AdditionalWorkspacesProperties object itself. */ @@ -75,7 +79,7 @@ public AdditionalWorkspacesProperties withType(AdditionalWorkspaceType type) { /** * Get the dataTypes property: List of data types sent to workspace. - * + * * @return the dataTypes value. */ public List dataTypes() { @@ -84,7 +88,7 @@ public List dataTypes() { /** * Set the dataTypes property: List of data types sent to workspace. - * + * * @param dataTypes the dataTypes value to set. * @return the AdditionalWorkspacesProperties object itself. */ @@ -95,7 +99,7 @@ public AdditionalWorkspacesProperties withDataTypes(List resourceIdentifiers(); /** * Gets the remediationSteps property: Manual action items to take to remediate the alert. - * + * * @return the remediationSteps value. */ List remediationSteps(); /** * Gets the vendorName property: The name of the vendor that raises the alert. - * + * * @return the vendorName value. */ String vendorName(); /** * Gets the status property: The life cycle status of the alert. - * + * * @return the status value. */ AlertStatus status(); /** * Gets the extendedLinks property: Links related to the alert. - * + * * @return the extendedLinks value. */ List> extendedLinks(); /** * Gets the alertUri property: A direct link to the alert page in Azure Portal. - * + * * @return the alertUri value. */ String alertUri(); /** * Gets the timeGeneratedUtc property: The UTC time the alert was generated in ISO8601 format. - * + * * @return the timeGeneratedUtc value. */ OffsetDateTime timeGeneratedUtc(); @@ -162,21 +164,21 @@ public interface Alert { * Gets the productName property: The name of the product which published this alert (Microsoft Sentinel, Microsoft * Defender for Identity, Microsoft Defender for Endpoint, Microsoft Defender for Office, Microsoft Defender for * Cloud Apps, and so on). - * + * * @return the productName value. */ String productName(); /** * Gets the processingEndTimeUtc property: The UTC processing end time of the alert in ISO8601 format. - * + * * @return the processingEndTimeUtc value. */ OffsetDateTime processingEndTimeUtc(); /** * Gets the entities property: A list of entities related to the alert. - * + * * @return the entities value. */ List entities(); @@ -184,7 +186,7 @@ public interface Alert { /** * Gets the isIncident property: This field determines whether the alert is an incident (a compound grouping of * several alerts) or a single alert. - * + * * @return the isIncident value. */ Boolean isIncident(); @@ -192,49 +194,49 @@ public interface Alert { /** * Gets the correlationKey property: Key for corelating related alerts. Alerts with the same correlation key * considered to be related. - * + * * @return the correlationKey value. */ String correlationKey(); /** * Gets the extendedProperties property: Custom properties for the alert. - * + * * @return the extendedProperties value. */ Map extendedProperties(); /** * Gets the compromisedEntity property: The display name of the resource most related to this alert. - * + * * @return the compromisedEntity value. */ String compromisedEntity(); /** * Gets the techniques property: kill chain related techniques behind the alert. - * + * * @return the techniques value. */ List techniques(); /** * Gets the subTechniques property: Kill chain related sub-techniques behind the alert. - * + * * @return the subTechniques value. */ List subTechniques(); /** * Gets the supportingEvidence property: Changing set of properties depending on the supportingEvidence type. - * + * * @return the supportingEvidence value. */ AlertPropertiesSupportingEvidence supportingEvidence(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AlertInner object. - * + * * @return the inner object. */ AlertInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertEntity.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertEntity.java index b835d2cab6fbb..1f984f888d8de 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertEntity.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertEntity.java @@ -12,7 +12,9 @@ import java.util.HashMap; import java.util.Map; -/** Changing set of properties depending on the entity type. */ +/** + * Changing set of properties depending on the entity type. + */ @Fluent public final class AlertEntity { /* @@ -24,15 +26,18 @@ public final class AlertEntity { /* * Changing set of properties depending on the entity type. */ - @JsonIgnore private Map additionalProperties; + @JsonIgnore + private Map additionalProperties; - /** Creates an instance of AlertEntity class. */ + /** + * Creates an instance of AlertEntity class. + */ public AlertEntity() { } /** * Get the type property: Type of entity. - * + * * @return the type value. */ public String type() { @@ -41,7 +46,7 @@ public String type() { /** * Get the additionalProperties property: Changing set of properties depending on the entity type. - * + * * @return the additionalProperties value. */ @JsonAnyGetter @@ -51,7 +56,7 @@ public Map additionalProperties() { /** * Set the additionalProperties property: Changing set of properties depending on the entity type. - * + * * @param additionalProperties the additionalProperties value to set. * @return the AlertEntity object itself. */ @@ -70,7 +75,7 @@ void withAdditionalProperties(String key, Object value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertList.java index 3e5b19511e577..3e68de674d551 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of security alerts. */ +/** + * List of security alerts. + */ @Fluent public final class AlertList { /* @@ -24,13 +26,15 @@ public final class AlertList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AlertList class. */ + /** + * Creates an instance of AlertList class. + */ public AlertList() { } /** * Get the value property: describes security alert properties. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: describes security alert properties. - * + * * @param value the value value to set. * @return the AlertList object itself. */ @@ -50,7 +54,7 @@ public AlertList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertPropertiesSupportingEvidence.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertPropertiesSupportingEvidence.java index b0e6a2f61c503..a26abb60d3f03 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertPropertiesSupportingEvidence.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertPropertiesSupportingEvidence.java @@ -12,7 +12,9 @@ import java.util.HashMap; import java.util.Map; -/** Changing set of properties depending on the supportingEvidence type. */ +/** + * Changing set of properties depending on the supportingEvidence type. + */ @Fluent public final class AlertPropertiesSupportingEvidence { /* @@ -24,15 +26,18 @@ public final class AlertPropertiesSupportingEvidence { /* * Changing set of properties depending on the supportingEvidence type. */ - @JsonIgnore private Map additionalProperties; + @JsonIgnore + private Map additionalProperties; - /** Creates an instance of AlertPropertiesSupportingEvidence class. */ + /** + * Creates an instance of AlertPropertiesSupportingEvidence class. + */ public AlertPropertiesSupportingEvidence() { } /** * Get the type property: Type of the supportingEvidence. - * + * * @return the type value. */ public String type() { @@ -41,7 +46,7 @@ public String type() { /** * Get the additionalProperties property: Changing set of properties depending on the supportingEvidence type. - * + * * @return the additionalProperties value. */ @JsonAnyGetter @@ -51,7 +56,7 @@ public Map additionalProperties() { /** * Set the additionalProperties property: Changing set of properties depending on the supportingEvidence type. - * + * * @param additionalProperties the additionalProperties value to set. * @return the AlertPropertiesSupportingEvidence object itself. */ @@ -70,7 +75,7 @@ void withAdditionalProperties(String key, Object value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSeverity.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSeverity.java index 52eb272a6f137..70543515ed02d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSeverity.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSeverity.java @@ -13,21 +13,29 @@ * https://docs.microsoft.com/en-us/azure/security-center/security-center-alerts-overview#how-are-alerts-classified. */ public final class AlertSeverity extends ExpandableStringEnum { - /** Static value Informational for AlertSeverity. */ + /** + * Static value Informational for AlertSeverity. + */ public static final AlertSeverity INFORMATIONAL = fromString("Informational"); - /** Static value Low for AlertSeverity. */ + /** + * Static value Low for AlertSeverity. + */ public static final AlertSeverity LOW = fromString("Low"); - /** Static value Medium for AlertSeverity. */ + /** + * Static value Medium for AlertSeverity. + */ public static final AlertSeverity MEDIUM = fromString("Medium"); - /** Static value High for AlertSeverity. */ + /** + * Static value High for AlertSeverity. + */ public static final AlertSeverity HIGH = fromString("High"); /** * Creates a new instance of AlertSeverity value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +44,7 @@ public AlertSeverity() { /** * Creates or finds a AlertSeverity from its string representation. - * + * * @param name a name to look for. * @return the corresponding AlertSeverity. */ @@ -47,7 +55,7 @@ public static AlertSeverity fromString(String name) { /** * Gets known AlertSeverity values. - * + * * @return known AlertSeverity values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorBundlesRequestProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorBundlesRequestProperties.java index 63f1f1f6633b0..1f2fe7999191f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorBundlesRequestProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorBundlesRequestProperties.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Simulate alerts according to this bundles. */ +/** + * Simulate alerts according to this bundles. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") @JsonTypeName("Bundles") @Fluent @@ -21,13 +23,15 @@ public final class AlertSimulatorBundlesRequestProperties extends AlertSimulator @JsonProperty(value = "bundles") private List bundles; - /** Creates an instance of AlertSimulatorBundlesRequestProperties class. */ + /** + * Creates an instance of AlertSimulatorBundlesRequestProperties class. + */ public AlertSimulatorBundlesRequestProperties() { } /** * Get the bundles property: Bundles list. - * + * * @return the bundles value. */ public List bundles() { @@ -36,7 +40,7 @@ public List bundles() { /** * Set the bundles property: Bundles list. - * + * * @param bundles the bundles value to set. * @return the AlertSimulatorBundlesRequestProperties object itself. */ @@ -47,7 +51,7 @@ public AlertSimulatorBundlesRequestProperties withBundles(List bundl /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestBody.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestBody.java index 0e6c98294c16b..68b3a74dde86c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestBody.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestBody.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Alert Simulator request body. */ +/** + * Alert Simulator request body. + */ @Fluent public final class AlertSimulatorRequestBody { /* @@ -16,13 +18,15 @@ public final class AlertSimulatorRequestBody { @JsonProperty(value = "properties") private AlertSimulatorRequestProperties properties; - /** Creates an instance of AlertSimulatorRequestBody class. */ + /** + * Creates an instance of AlertSimulatorRequestBody class. + */ public AlertSimulatorRequestBody() { } /** * Get the properties property: Alert Simulator request body data. - * + * * @return the properties value. */ public AlertSimulatorRequestProperties properties() { @@ -31,7 +35,7 @@ public AlertSimulatorRequestProperties properties() { /** * Set the properties property: Alert Simulator request body data. - * + * * @param properties the properties value to set. * @return the AlertSimulatorRequestBody object itself. */ @@ -42,7 +46,7 @@ public AlertSimulatorRequestBody withProperties(AlertSimulatorRequestProperties /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestProperties.java index f4bc7b5b76c87..5a7b49c61c35a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSimulatorRequestProperties.java @@ -14,28 +14,33 @@ import java.util.HashMap; import java.util.Map; -/** Describes properties of an alert simulation request. */ +/** + * Describes properties of an alert simulation request. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind", defaultImpl = AlertSimulatorRequestProperties.class) @JsonTypeName("AlertSimulatorRequestProperties") -@JsonSubTypes({@JsonSubTypes.Type(name = "Bundles", value = AlertSimulatorBundlesRequestProperties.class)}) +@JsonSubTypes({ @JsonSubTypes.Type(name = "Bundles", value = AlertSimulatorBundlesRequestProperties.class) }) @Fluent public class AlertSimulatorRequestProperties { /* * Describes properties of an alert simulation request */ - @JsonIgnore private Map additionalProperties; + @JsonIgnore + private Map additionalProperties; - /** Creates an instance of AlertSimulatorRequestProperties class. */ + /** + * Creates an instance of AlertSimulatorRequestProperties class. + */ public AlertSimulatorRequestProperties() { } /** * Get the additionalProperties property: Describes properties of an alert simulation request. - * + * * @return the additionalProperties value. */ @JsonAnyGetter @@ -45,7 +50,7 @@ public Map additionalProperties() { /** * Set the additionalProperties property: Describes properties of an alert simulation request. - * + * * @param additionalProperties the additionalProperties value to set. * @return the AlertSimulatorRequestProperties object itself. */ @@ -64,7 +69,7 @@ void withAdditionalProperties(String key, Object value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertStatus.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertStatus.java index 4a5d9c3eebbcb..833c884ccc655 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertStatus.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertStatus.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The life cycle status of the alert. */ +/** + * The life cycle status of the alert. + */ public final class AlertStatus extends ExpandableStringEnum { - /** Static value Active for AlertStatus. */ + /** + * Static value Active for AlertStatus. + */ public static final AlertStatus ACTIVE = fromString("Active"); - /** Static value InProgress for AlertStatus. */ + /** + * Static value InProgress for AlertStatus. + */ public static final AlertStatus IN_PROGRESS = fromString("InProgress"); - /** Static value Resolved for AlertStatus. */ + /** + * Static value Resolved for AlertStatus. + */ public static final AlertStatus RESOLVED = fromString("Resolved"); - /** Static value Dismissed for AlertStatus. */ + /** + * Static value Dismissed for AlertStatus. + */ public static final AlertStatus DISMISSED = fromString("Dismissed"); /** * Creates a new instance of AlertStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public AlertStatus() { /** * Creates or finds a AlertStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding AlertStatus. */ @@ -44,7 +54,7 @@ public static AlertStatus fromString(String name) { /** * Gets known AlertStatus values. - * + * * @return known AlertStatus values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSyncSettings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSyncSettings.java index 029123d935682..a355252434c4e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSyncSettings.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertSyncSettings.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Represents an alert sync setting. */ +/** + * Represents an alert sync setting. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") @JsonTypeName("AlertSyncSettings") @Fluent @@ -22,13 +24,15 @@ public final class AlertSyncSettings extends SettingInner { @JsonProperty(value = "properties") private AlertSyncSettingProperties innerProperties; - /** Creates an instance of AlertSyncSettings class. */ + /** + * Creates an instance of AlertSyncSettings class. + */ public AlertSyncSettings() { } /** * Get the innerProperties property: Alert sync setting data. - * + * * @return the innerProperties value. */ private AlertSyncSettingProperties innerProperties() { @@ -37,7 +41,7 @@ private AlertSyncSettingProperties innerProperties() { /** * Get the enabled property: Is the alert sync setting enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -46,7 +50,7 @@ public Boolean enabled() { /** * Set the enabled property: Is the alert sync setting enabled. - * + * * @param enabled the enabled value to set. * @return the AlertSyncSettings object itself. */ @@ -60,7 +64,7 @@ public AlertSyncSettings withEnabled(Boolean enabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Alerts.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Alerts.java index 6b849c8c6c062..5091d717a7a54 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Alerts.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Alerts.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Alerts. */ +/** + * Resource collection API of Alerts. + */ public interface Alerts { /** * List all the alerts that are associated with the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security alerts as paginated response with {@link PagedIterable}. @@ -21,7 +23,7 @@ public interface Alerts { /** * List all the alerts that are associated with the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,9 +34,9 @@ public interface Alerts { /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -44,9 +46,9 @@ public interface Alerts { /** * List all the alerts that are associated with the resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -57,9 +59,9 @@ public interface Alerts { /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -69,9 +71,9 @@ public interface Alerts { /** * List all the alerts that are associated with the subscription that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -82,11 +84,11 @@ public interface Alerts { /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -96,11 +98,11 @@ public interface Alerts { /** * List all the alerts that are associated with the resource group that are stored in a specific location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -111,9 +113,9 @@ public interface Alerts { /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -125,9 +127,9 @@ public interface Alerts { /** * Get an alert that is associated with a subscription. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -138,29 +140,29 @@ public interface Alerts { /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an alert that is associated a resource group or a resource in a resource group along with {@link - * Response}. + * @return an alert that is associated a resource group or a resource in a resource group along with + * {@link Response}. */ - Response getResourceGroupLevelWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response getResourceGroupLevelWithResponse(String resourceGroupName, String ascLocation, String alertName, + Context context); /** * Get an alert that is associated a resource group or a resource in a resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -171,9 +173,9 @@ Response getResourceGroupLevelWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -181,14 +183,14 @@ Response getResourceGroupLevelWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateSubscriptionLevelStateToDismissWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToDismissWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -198,9 +200,9 @@ Response updateSubscriptionLevelStateToDismissWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -208,14 +210,14 @@ Response updateSubscriptionLevelStateToDismissWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateSubscriptionLevelStateToResolveWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToResolveWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -225,9 +227,9 @@ Response updateSubscriptionLevelStateToResolveWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -235,14 +237,14 @@ Response updateSubscriptionLevelStateToResolveWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateSubscriptionLevelStateToActivateWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToActivateWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -252,9 +254,9 @@ Response updateSubscriptionLevelStateToActivateWithResponse( /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -262,14 +264,14 @@ Response updateSubscriptionLevelStateToActivateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateSubscriptionLevelStateToInProgressWithResponse( - String ascLocation, String alertName, Context context); + Response updateSubscriptionLevelStateToInProgressWithResponse(String ascLocation, String alertName, + Context context); /** * Update the alert's state. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -279,11 +281,11 @@ Response updateSubscriptionLevelStateToInProgressWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -291,16 +293,16 @@ Response updateSubscriptionLevelStateToInProgressWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateResourceGroupLevelStateToResolveWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToResolveWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -310,11 +312,11 @@ Response updateResourceGroupLevelStateToResolveWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -322,16 +324,16 @@ Response updateResourceGroupLevelStateToResolveWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateResourceGroupLevelStateToDismissWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToDismissWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -341,11 +343,11 @@ Response updateResourceGroupLevelStateToDismissWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -353,16 +355,16 @@ Response updateResourceGroupLevelStateToDismissWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateResourceGroupLevelStateToActivateWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToActivateWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -372,11 +374,11 @@ Response updateResourceGroupLevelStateToActivateWithResponse( /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -384,16 +386,16 @@ Response updateResourceGroupLevelStateToActivateWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response updateResourceGroupLevelStateToInProgressWithResponse( - String resourceGroupName, String ascLocation, String alertName, Context context); + Response updateResourceGroupLevelStateToInProgressWithResponse(String resourceGroupName, String ascLocation, + String alertName, Context context); /** * Update the alert's state. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertName Name of the alert object. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -403,9 +405,9 @@ Response updateResourceGroupLevelStateToInProgressWithResponse( /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -415,9 +417,9 @@ Response updateResourceGroupLevelStateToInProgressWithResponse( /** * Simulate security alerts. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param alertSimulatorRequestBody Alert Simulator Request Properties. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRule.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRule.java index 4e7c2860f5f7e..84cf7a11c1b7c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRule.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRule.java @@ -7,82 +7,84 @@ import com.azure.resourcemanager.security.fluent.models.AlertsSuppressionRuleInner; import java.time.OffsetDateTime; -/** An immutable client-side representation of AlertsSuppressionRule. */ +/** + * An immutable client-side representation of AlertsSuppressionRule. + */ public interface AlertsSuppressionRule { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the alertType property: Type of the alert to automatically suppress. For all alert types, use '*'. - * + * * @return the alertType value. */ String alertType(); /** * Gets the lastModifiedUtc property: The last time this rule was modified. - * + * * @return the lastModifiedUtc value. */ OffsetDateTime lastModifiedUtc(); /** * Gets the expirationDateUtc property: Expiration date of the rule, if value is not provided or provided as null - * this field will default to the maximum allowed expiration date. - * + * there will no expiration at all. + * * @return the expirationDateUtc value. */ OffsetDateTime expirationDateUtc(); /** * Gets the reason property: The reason for dismissing the alert. - * + * * @return the reason value. */ String reason(); /** * Gets the state property: Possible states of the rule. - * + * * @return the state value. */ RuleState state(); /** * Gets the comment property: Any comment regarding the rule. - * + * * @return the comment value. */ String comment(); /** * Gets the suppressionAlertsScope property: The suppression conditions. - * + * * @return the suppressionAlertsScope value. */ SuppressionAlertsScope suppressionAlertsScope(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AlertsSuppressionRuleInner object. - * + * * @return the inner object. */ AlertsSuppressionRuleInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRules.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRules.java index 201e06daeff1b..dba56aff33f47 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRules.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRules.java @@ -9,11 +9,13 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AlertsSuppressionRuleInner; -/** Resource collection API of AlertsSuppressionRules. */ +/** + * Resource collection API of AlertsSuppressionRules. + */ public interface AlertsSuppressionRules { /** * List of all the dismiss rules for the given subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return suppression rules list for subscription as paginated response with {@link PagedIterable}. @@ -22,7 +24,7 @@ public interface AlertsSuppressionRules { /** * List of all the dismiss rules for the given subscription. - * + * * @param alertType Type of the alert to get rules for. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -34,20 +36,20 @@ public interface AlertsSuppressionRules { /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with {@link - * Response}. + * @return dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription along with + * {@link Response}. */ Response getWithResponse(String alertsSuppressionRuleName, Context context); /** * Get dismiss rule, with name: {alertsSuppressionRuleName}, for the given subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -58,7 +60,7 @@ public interface AlertsSuppressionRules { /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @param context The context to associate with this operation. @@ -67,12 +69,12 @@ public interface AlertsSuppressionRules { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return describes the suppression rule along with {@link Response}. */ - Response updateWithResponse( - String alertsSuppressionRuleName, AlertsSuppressionRuleInner alertsSuppressionRule, Context context); + Response updateWithResponse(String alertsSuppressionRuleName, + AlertsSuppressionRuleInner alertsSuppressionRule, Context context); /** * Update existing rule or create new rule if it doesn't exist. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param alertsSuppressionRule Suppression rule object. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -84,7 +86,7 @@ Response updateWithResponse( /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -96,7 +98,7 @@ Response updateWithResponse( /** * Delete dismiss alert rule for this subscription. - * + * * @param alertsSuppressionRuleName The unique name of the suppression alert rule. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRulesList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRulesList.java index 7d0e5106fcd2e..2e37c951cc5a6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRulesList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AlertsSuppressionRulesList.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Suppression rules list for subscription. */ +/** + * Suppression rules list for subscription. + */ @Fluent public final class AlertsSuppressionRulesList { /* @@ -25,13 +27,15 @@ public final class AlertsSuppressionRulesList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AlertsSuppressionRulesList class. */ + /** + * Creates an instance of AlertsSuppressionRulesList class. + */ public AlertsSuppressionRulesList() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The value property. - * + * * @param value the value value to set. * @return the AlertsSuppressionRulesList object itself. */ @@ -51,7 +55,7 @@ public AlertsSuppressionRulesList withValue(List val /** * Get the nextLink property: URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,15 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property value in model AlertsSuppressionRulesList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model AlertsSuppressionRulesList")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnections.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnections.java index 7dc20e8e9240b..cd3d58d7360f8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnections.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnections.java @@ -8,90 +8,92 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of AllowedConnections. */ +/** + * Resource collection API of AllowedConnections. + */ public interface AllowedConnections { /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedIterable}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Gets the list of all possible traffic between resources for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the list of all possible traffic between resources for the subscription as paginated response with {@link - * PagedIterable}. + * @return the list of all possible traffic between resources for the subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedIterable}. + * response with {@link PagedIterable}. */ PagedIterable listByHomeRegion(String ascLocation); /** * Gets the list of all possible traffic between resources for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location as paginated - * response with {@link PagedIterable}. + * response with {@link PagedIterable}. */ PagedIterable listByHomeRegion(String ascLocation, Context context); /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type along with {@link Response}. + * type along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String ascLocation, ConnectionType connectionType, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + ConnectionType connectionType, Context context); /** * Gets the list of all possible traffic between resources for the subscription and location, based on connection * type. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param connectionType The type of allowed connections (Internal, External). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the list of all possible traffic between resources for the subscription and location, based on connection - * type. + * type. */ AllowedConnectionsResource get(String resourceGroupName, String ascLocation, ConnectionType connectionType); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsList.java index aa29c6120ec3f..f6d956c3eff21 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of all possible traffic between Azure resources. */ +/** + * List of all possible traffic between Azure resources. + */ @Immutable public final class AllowedConnectionsList { /* @@ -24,13 +26,15 @@ public final class AllowedConnectionsList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AllowedConnectionsList class. */ + /** + * Creates an instance of AllowedConnectionsList class. + */ public AllowedConnectionsList() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsResource.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsResource.java index f7544bb61cefe..92477f0db9964 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsResource.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowedConnectionsResource.java @@ -8,53 +8,55 @@ import java.time.OffsetDateTime; import java.util.List; -/** An immutable client-side representation of AllowedConnectionsResource. */ +/** + * An immutable client-side representation of AllowedConnectionsResource. + */ public interface AllowedConnectionsResource { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: Location where the resource is stored. - * + * * @return the location value. */ String location(); /** * Gets the calculatedDateTime property: The UTC time on which the allowed connections resource was calculated. - * + * * @return the calculatedDateTime value. */ OffsetDateTime calculatedDateTime(); /** * Gets the connectableResources property: List of connectable resources. - * + * * @return the connectableResources value. */ List connectableResources(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AllowedConnectionsResourceInner object. - * + * * @return the inner object. */ AllowedConnectionsResourceInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowlistCustomAlertRule.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowlistCustomAlertRule.java index 31e157bbd2e82..4cb2d83dc1ec2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowlistCustomAlertRule.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AllowlistCustomAlertRule.java @@ -12,7 +12,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** A custom alert rule that checks if a value (depends on the custom alert type) is allowed. */ +/** + * A custom alert rule that checks if a value (depends on the custom alert type) is allowed. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -23,8 +25,7 @@ @JsonSubTypes.Type(name = "ConnectionToIpNotAllowed", value = ConnectionToIpNotAllowed.class), @JsonSubTypes.Type(name = "ConnectionFromIpNotAllowed", value = ConnectionFromIpNotAllowed.class), @JsonSubTypes.Type(name = "LocalUserNotAllowed", value = LocalUserNotAllowed.class), - @JsonSubTypes.Type(name = "ProcessNotAllowed", value = ProcessNotAllowed.class) -}) + @JsonSubTypes.Type(name = "ProcessNotAllowed", value = ProcessNotAllowed.class) }) @Fluent public class AllowlistCustomAlertRule extends ListCustomAlertRule { /* @@ -33,13 +34,15 @@ public class AllowlistCustomAlertRule extends ListCustomAlertRule { @JsonProperty(value = "allowlistValues", required = true) private List allowlistValues; - /** Creates an instance of AllowlistCustomAlertRule class. */ + /** + * Creates an instance of AllowlistCustomAlertRule class. + */ public AllowlistCustomAlertRule() { } /** * Get the allowlistValues property: The values to allow. The format of the values depends on the rule type. - * + * * @return the allowlistValues value. */ public List allowlistValues() { @@ -48,7 +51,7 @@ public List allowlistValues() { /** * Set the allowlistValues property: The values to allow. The format of the values depends on the rule type. - * + * * @param allowlistValues the allowlistValues value to set. * @return the AllowlistCustomAlertRule object itself. */ @@ -57,7 +60,9 @@ public AllowlistCustomAlertRule withAllowlistValues(List allowlistValues return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AllowlistCustomAlertRule withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -66,17 +71,15 @@ public AllowlistCustomAlertRule withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (allowlistValues() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property allowlistValues in model AllowlistCustomAlertRule")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property allowlistValues in model AllowlistCustomAlertRule")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DMessagesNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DMessagesNotInAllowedRange.java index 14079d7a02f2c..d1aaa099e6179 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DMessagesNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DMessagesNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of cloud to device messages (AMQP protocol) is not in allowed range. */ +/** + * Number of cloud to device messages (AMQP protocol) is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("AmqpC2DMessagesNotInAllowedRange") @Fluent public final class AmqpC2DMessagesNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of AmqpC2DMessagesNotInAllowedRange class. */ + /** + * Creates an instance of AmqpC2DMessagesNotInAllowedRange class. + */ public AmqpC2DMessagesNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DMessagesNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DMessagesNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DMessagesNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DMessagesNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public AmqpC2DMessagesNotInAllowedRange withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DRejectedMessagesNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DRejectedMessagesNotInAllowedRange.java index 833c457e1fcc1..e5794a72d5486 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DRejectedMessagesNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpC2DRejectedMessagesNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of rejected cloud to device messages (AMQP protocol) is not in allowed range. */ +/** + * Number of rejected cloud to device messages (AMQP protocol) is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("AmqpC2DRejectedMessagesNotInAllowedRange") @Fluent public final class AmqpC2DRejectedMessagesNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of AmqpC2DRejectedMessagesNotInAllowedRange class. */ + /** + * Creates an instance of AmqpC2DRejectedMessagesNotInAllowedRange class. + */ public AmqpC2DRejectedMessagesNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DRejectedMessagesNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DRejectedMessagesNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DRejectedMessagesNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpC2DRejectedMessagesNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public AmqpC2DRejectedMessagesNotInAllowedRange withIsEnabled(boolean isEnabled) /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpD2CMessagesNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpD2CMessagesNotInAllowedRange.java index d7cb03368abcc..ffc5b1fc88a09 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpD2CMessagesNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AmqpD2CMessagesNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of device to cloud messages (AMQP protocol) is not in allowed range. */ +/** + * Number of device to cloud messages (AMQP protocol) is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("AmqpD2CMessagesNotInAllowedRange") @Fluent public final class AmqpD2CMessagesNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of AmqpD2CMessagesNotInAllowedRange class. */ + /** + * Creates an instance of AmqpD2CMessagesNotInAllowedRange class. + */ public AmqpD2CMessagesNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpD2CMessagesNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpD2CMessagesNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpD2CMessagesNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AmqpD2CMessagesNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public AmqpD2CMessagesNotInAllowedRange withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AnnotateDefaultBranchState.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AnnotateDefaultBranchState.java new file mode 100644 index 0000000000000..610c40f3c2c61 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AnnotateDefaultBranchState.java @@ -0,0 +1,56 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * Configuration of PR Annotations on default branch. + * + * Enabled - PR Annotations are enabled on the resource's default branch. + * Disabled - PR Annotations are disabled on the resource's default branch. + */ +public final class AnnotateDefaultBranchState extends ExpandableStringEnum { + /** + * Static value Disabled for AnnotateDefaultBranchState. + */ + public static final AnnotateDefaultBranchState DISABLED = fromString("Disabled"); + + /** + * Static value Enabled for AnnotateDefaultBranchState. + */ + public static final AnnotateDefaultBranchState ENABLED = fromString("Enabled"); + + /** + * Creates a new instance of AnnotateDefaultBranchState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public AnnotateDefaultBranchState() { + } + + /** + * Creates or finds a AnnotateDefaultBranchState from its string representation. + * + * @param name a name to look for. + * @return the corresponding AnnotateDefaultBranchState. + */ + @JsonCreator + public static AnnotateDefaultBranchState fromString(String name) { + return fromString(name, AnnotateDefaultBranchState.class); + } + + /** + * Gets known AnnotateDefaultBranchState values. + * + * @return known AnnotateDefaultBranchState values. + */ + public static Collection values() { + return values(AnnotateDefaultBranchState.class); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollection.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollection.java new file mode 100644 index 0000000000000..e20f45dc9be3a --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollection.java @@ -0,0 +1,116 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.resourcemanager.security.fluent.models.ApiCollectionInner; + +/** + * An immutable client-side representation of ApiCollection. + */ +public interface ApiCollection { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the provisioningState property: Gets the provisioning state of the API collection. + * + * @return the provisioningState value. + */ + ProvisioningState provisioningState(); + + /** + * Gets the displayName property: The display name of the API collection. + * + * @return the displayName value. + */ + String displayName(); + + /** + * Gets the discoveredVia property: The resource Id of the resource from where this API collection was discovered. + * + * @return the discoveredVia value. + */ + String discoveredVia(); + + /** + * Gets the baseUrl property: The base URI for this API collection. All endpoints of this API collection extend this + * base URI. + * + * @return the baseUrl value. + */ + String baseUrl(); + + /** + * Gets the numberOfApiEndpoints property: The number of API endpoints discovered in this API collection. + * + * @return the numberOfApiEndpoints value. + */ + Long numberOfApiEndpoints(); + + /** + * Gets the numberOfInactiveApiEndpoints property: The number of API endpoints in this API collection that have not + * received any API traffic in the last 30 days. + * + * @return the numberOfInactiveApiEndpoints value. + */ + Long numberOfInactiveApiEndpoints(); + + /** + * Gets the numberOfUnauthenticatedApiEndpoints property: The number of API endpoints in this API collection that + * are unauthenticated. + * + * @return the numberOfUnauthenticatedApiEndpoints value. + */ + Long numberOfUnauthenticatedApiEndpoints(); + + /** + * Gets the numberOfExternalApiEndpoints property: The number of API endpoints in this API collection for which API + * traffic from the internet was observed. + * + * @return the numberOfExternalApiEndpoints value. + */ + Long numberOfExternalApiEndpoints(); + + /** + * Gets the numberOfApiEndpointsWithSensitiveDataExposed property: The number of API endpoints in this API + * collection which are exposing sensitive data in their requests and/or responses. + * + * @return the numberOfApiEndpointsWithSensitiveDataExposed value. + */ + Long numberOfApiEndpointsWithSensitiveDataExposed(); + + /** + * Gets the sensitivityLabel property: The highest priority sensitivity label from Microsoft Purview in this API + * collection. + * + * @return the sensitivityLabel value. + */ + String sensitivityLabel(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.ApiCollectionInner object. + * + * @return the inner object. + */ + ApiCollectionInner innerModel(); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionResponseList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionList.java similarity index 77% rename from sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionResponseList.java rename to sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionList.java index 2666ef5311234..df603453bbd43 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionResponseList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionList.java @@ -5,18 +5,20 @@ package com.azure.resourcemanager.security.models; import com.azure.core.annotation.Immutable; -import com.azure.resourcemanager.security.fluent.models.ApiCollectionResponseInner; +import com.azure.resourcemanager.security.fluent.models.ApiCollectionInner; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Page of a list of API collections as represented by Defender for APIs. */ +/** + * Page of a list of API collections as represented by Microsoft Defender for APIs. + */ @Immutable -public final class ApiCollectionResponseList { +public final class ApiCollectionList { /* * API collections in this page. */ @JsonProperty(value = "value", access = JsonProperty.Access.WRITE_ONLY) - private List value; + private List value; /* * The URI to fetch the next page. @@ -24,22 +26,24 @@ public final class ApiCollectionResponseList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ApiCollectionResponseList class. */ - public ApiCollectionResponseList() { + /** + * Creates an instance of ApiCollectionList class. + */ + public ApiCollectionList() { } /** * Get the value property: API collections in this page. - * + * * @return the value value. */ - public List value() { + public List value() { return this.value; } /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOffboardings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOffboardings.java deleted file mode 100644 index 8fbc3ffe617e9..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOffboardings.java +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.models; - -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; - -/** Resource collection API of ApiCollectionOffboardings. */ -public interface ApiCollectionOffboardings { - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return the {@link Response}. - */ - Response deleteWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context); - - /** - * Offboard an Azure API Management API from Defender for APIs - * - *

Offboard an Azure API Management API from Defender for APIs. The system will stop monitoring the operations - * within the Azure API Management API for intrusive behaviors. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - */ - void delete(String resourceGroupName, String serviceName, String apiCollectionId); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOnboardings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOnboardings.java deleted file mode 100644 index c4109ca69adb4..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollectionOnboardings.java +++ /dev/null @@ -1,47 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.models; - -import com.azure.core.http.rest.Response; -import com.azure.core.util.Context; - -/** Resource collection API of ApiCollectionOnboardings. */ -public interface ApiCollectionOnboardings { - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @param context The context to associate with this operation. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs along with {@link Response}. - */ - Response createWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context); - - /** - * Onboard an Azure API Management API to Defender for APIs - * - *

Onboard an Azure API Management API to Defender for APIs. The system will start monitoring the operations - * within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * - * @param resourceGroupName The name of the resource group. The name is case insensitive. - * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. - * @throws IllegalArgumentException thrown if parameters fail the validation. - * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. - * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an API collection as represented by Defender for APIs. - */ - ApiCollectionResponse create(String resourceGroupName, String serviceName, String apiCollectionId); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollections.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollections.java index d21a442595cb1..e8883e5365b7a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollections.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApiCollections.java @@ -8,78 +8,210 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ApiCollections. */ +/** + * Resource collection API of ApiCollections. + */ public interface ApiCollections { /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + PagedIterable list(); + + /** + * Gets a list of API collections within a subscription + * + * Gets a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs. + * + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a subscription that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + PagedIterable list(Context context); + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + PagedIterable listByResourceGroup(String resourceGroupName); + + /** + * Gets a list of API collections within a resource group + * + * Gets a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of API collections within a resource group that have been onboarded to Microsoft Defender for APIs + * as paginated response with {@link PagedIterable}. + */ + PagedIterable listByResourceGroup(String resourceGroupName, Context context); + + /** + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedIterable}. + */ + PagedIterable listByAzureApiManagementService(String resourceGroupName, String serviceName); + + /** + * Gets a list of onboarded Azure API Management APIs + * + * Gets a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a list of Azure API Management APIs that have been onboarded to Microsoft Defender for APIs as paginated + * response with {@link PagedIterable}. + */ + PagedIterable listByAzureApiManagementService(String resourceGroupName, String serviceName, + Context context); + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs along with + * {@link Response}. + */ + Response getByAzureApiManagementServiceWithResponse(String resourceGroupName, String serviceName, + String apiId, Context context); + + /** + * Gets an onboarded Azure API Management API + * + * Gets an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. If an Azure API + * Management API is onboarded to Microsoft Defender for APIs, the system will monitor the operations within the + * Azure API Management API for intrusive behaviors and provide alerts for attacks that have been detected. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return an Azure API Management API if it has been onboarded to Microsoft Defender for APIs. + */ + ApiCollection getByAzureApiManagementService(String resourceGroupName, String serviceName, String apiId); + + /** + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedIterable}. + * @return an API collection as represented by Microsoft Defender for APIs. */ - PagedIterable list(String resourceGroupName, String serviceName); + ApiCollection onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId); /** - * Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs - * - *

Gets a list of Azure API Management APIs that have been onboarded to Defender for APIs. If an Azure API - * Management API is onboarded to Defender for APIs, the system will monitor the operations within the Azure API - * Management API for intrusive behaviors and provide alerts for attacks that have been detected. - * + * Onboard an Azure API Management API to Microsoft Defender for APIs + * + * Onboard an Azure API Management API to Microsoft Defender for APIs. The system will start monitoring the + * operations within the Azure Management API for intrusive behaviors and provide alerts for attacks that have been + * detected. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of Azure API Management APIs that have been onboarded to Defender for APIs as paginated response - * with {@link PagedIterable}. + * @return an API collection as represented by Microsoft Defender for APIs. */ - PagedIterable list(String resourceGroupName, String serviceName, Context context); + ApiCollection onboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId, + Context context); /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs along with {@link Response}. + * @return the {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String serviceName, String apiCollectionId, Context context); + Response offboardAzureApiManagementApiWithResponse(String resourceGroupName, String serviceName, String apiId, + Context context); /** - * Gets an Azure API Management API if it has been onboarded to Defender for APIs - * - *

Gets an Azure API Management API if it has been onboarded to Defender for APIs. If an Azure API Management API - * is onboarded to Defender for APIs, the system will monitor the operations within the Azure API Management API for - * intrusive behaviors and provide alerts for attacks that have been detected. - * + * Offboard an Azure API Management API from Microsoft Defender for APIs + * + * Offboard an Azure API Management API from Microsoft Defender for APIs. The system will stop monitoring the + * operations within the Azure API Management API for intrusive behaviors. + * * @param resourceGroupName The name of the resource group. The name is case insensitive. * @param serviceName The name of the API Management service. - * @param apiCollectionId A string representing the apiCollections resource within the Microsoft.Security provider - * namespace. This string matches the Azure API Management API name. + * @param apiId API revision identifier. Must be unique in the API Management service instance. Non-current revision + * has ;rev=n as a suffix where n is the revision number. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return an Azure API Management API if it has been onboarded to Defender for APIs. */ - ApiCollectionResponse get(String resourceGroupName, String serviceName, String apiCollectionId); + void offboardAzureApiManagementApi(String resourceGroupName, String serviceName, String apiId); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Application.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Application.java index 9a2925359582e..51f5fa4954dee 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Application.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Application.java @@ -8,218 +8,252 @@ import com.azure.resourcemanager.security.fluent.models.ApplicationInner; import java.util.List; -/** An immutable client-side representation of Application. */ +/** + * An immutable client-side representation of Application. + */ public interface Application { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the displayName property: display name of the application. - * + * * @return the displayName value. */ String displayName(); /** * Gets the description property: description of the application. - * + * * @return the description value. */ String description(); /** * Gets the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @return the sourceResourceType value. */ ApplicationSourceResourceType sourceResourceType(); /** * Gets the conditionSets property: The application conditionSets - see examples. - * + * * @return the conditionSets value. */ List conditionSets(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.ApplicationInner object. - * + * * @return the inner object. */ ApplicationInner innerModel(); - /** The entirety of the Application definition. */ + /** + * The entirety of the Application definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithCreate { } - /** The Application definition stages. */ + + /** + * The Application definition stages. + */ interface DefinitionStages { - /** The first stage of the Application definition. */ + /** + * The first stage of the Application definition. + */ interface Blank extends WithCreate { } + /** * The stage of the Application definition which contains all the minimum required properties for the resource * to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithDisplayName, - DefinitionStages.WithDescription, - DefinitionStages.WithSourceResourceType, - DefinitionStages.WithConditionSets { + interface WithCreate extends DefinitionStages.WithDisplayName, DefinitionStages.WithDescription, + DefinitionStages.WithSourceResourceType, DefinitionStages.WithConditionSets { /** * Executes the create request. - * + * * @return the created resource. */ Application create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Application create(Context context); } - /** The stage of the Application definition allowing to specify displayName. */ + + /** + * The stage of the Application definition allowing to specify displayName. + */ interface WithDisplayName { /** * Specifies the displayName property: display name of the application. - * + * * @param displayName display name of the application. * @return the next definition stage. */ WithCreate withDisplayName(String displayName); } - /** The stage of the Application definition allowing to specify description. */ + + /** + * The stage of the Application definition allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: description of the application. - * + * * @param description description of the application. * @return the next definition stage. */ WithCreate withDescription(String description); } - /** The stage of the Application definition allowing to specify sourceResourceType. */ + + /** + * The stage of the Application definition allowing to specify sourceResourceType. + */ interface WithSourceResourceType { /** * Specifies the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @param sourceResourceType The application source, what it affects, e.g. Assessments. * @return the next definition stage. */ WithCreate withSourceResourceType(ApplicationSourceResourceType sourceResourceType); } - /** The stage of the Application definition allowing to specify conditionSets. */ + + /** + * The stage of the Application definition allowing to specify conditionSets. + */ interface WithConditionSets { /** * Specifies the conditionSets property: The application conditionSets - see examples. - * + * * @param conditionSets The application conditionSets - see examples. * @return the next definition stage. */ WithCreate withConditionSets(List conditionSets); } } + /** * Begins update for the Application resource. - * + * * @return the stage of resource update. */ Application.Update update(); - /** The template for Application update. */ - interface Update - extends UpdateStages.WithDisplayName, - UpdateStages.WithDescription, - UpdateStages.WithSourceResourceType, - UpdateStages.WithConditionSets { + /** + * The template for Application update. + */ + interface Update extends UpdateStages.WithDisplayName, UpdateStages.WithDescription, + UpdateStages.WithSourceResourceType, UpdateStages.WithConditionSets { /** * Executes the update request. - * + * * @return the updated resource. */ Application apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Application apply(Context context); } - /** The Application update stages. */ + + /** + * The Application update stages. + */ interface UpdateStages { - /** The stage of the Application update allowing to specify displayName. */ + /** + * The stage of the Application update allowing to specify displayName. + */ interface WithDisplayName { /** * Specifies the displayName property: display name of the application. - * + * * @param displayName display name of the application. * @return the next definition stage. */ Update withDisplayName(String displayName); } - /** The stage of the Application update allowing to specify description. */ + + /** + * The stage of the Application update allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: description of the application. - * + * * @param description description of the application. * @return the next definition stage. */ Update withDescription(String description); } - /** The stage of the Application update allowing to specify sourceResourceType. */ + + /** + * The stage of the Application update allowing to specify sourceResourceType. + */ interface WithSourceResourceType { /** * Specifies the sourceResourceType property: The application source, what it affects, e.g. Assessments. - * + * * @param sourceResourceType The application source, what it affects, e.g. Assessments. * @return the next definition stage. */ Update withSourceResourceType(ApplicationSourceResourceType sourceResourceType); } - /** The stage of the Application update allowing to specify conditionSets. */ + + /** + * The stage of the Application update allowing to specify conditionSets. + */ interface WithConditionSets { /** * Specifies the conditionSets property: The application conditionSets - see examples. - * + * * @param conditionSets The application conditionSets - see examples. * @return the next definition stage. */ Update withConditionSets(List conditionSets); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Application refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationOperations.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationOperations.java index 233b962c3115a..e5c4555d335d7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationOperations.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationOperations.java @@ -7,11 +7,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ApplicationOperations. */ +/** + * Resource collection API of ApplicationOperations. + */ public interface ApplicationOperations { /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -23,7 +25,7 @@ public interface ApplicationOperations { /** * Get a specific application for the requested scope by applicationId. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -34,7 +36,7 @@ public interface ApplicationOperations { /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -46,7 +48,7 @@ public interface ApplicationOperations { /** * Delete an Application over a given scope. - * + * * @param applicationId The security Application key - unique key for the standard application. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -56,7 +58,7 @@ public interface ApplicationOperations { /** * Get a specific application for the requested scope by applicationId. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -67,7 +69,7 @@ public interface ApplicationOperations { /** * Get a specific application for the requested scope by applicationId. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -79,7 +81,7 @@ public interface ApplicationOperations { /** * Delete an Application over a given scope. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -89,7 +91,7 @@ public interface ApplicationOperations { /** * Delete an Application over a given scope. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -101,7 +103,7 @@ public interface ApplicationOperations { /** * Begins definition for a new Application resource. - * + * * @param name resource name. * @return the first stage of the new Application definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationSourceResourceType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationSourceResourceType.java index 9d2dd77b7267b..0709908b82c83 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationSourceResourceType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationSourceResourceType.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The application source, what it affects, e.g. Assessments. */ +/** + * The application source, what it affects, e.g. Assessments. + */ public final class ApplicationSourceResourceType extends ExpandableStringEnum { - /** Static value Assessments for ApplicationSourceResourceType. */ + /** + * Static value Assessments for ApplicationSourceResourceType. + */ public static final ApplicationSourceResourceType ASSESSMENTS = fromString("Assessments"); /** * Creates a new instance of ApplicationSourceResourceType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ApplicationSourceResourceType() { /** * Creates or finds a ApplicationSourceResourceType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ApplicationSourceResourceType. */ @@ -35,7 +39,7 @@ public static ApplicationSourceResourceType fromString(String name) { /** * Gets known ApplicationSourceResourceType values. - * + * * @return known ApplicationSourceResourceType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Applications.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Applications.java index 64f396cfe67c1..5efcf23b1b757 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Applications.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Applications.java @@ -7,27 +7,29 @@ import com.azure.core.http.rest.PagedIterable; import com.azure.core.util.Context; -/** Resource collection API of Applications. */ +/** + * Resource collection API of Applications. + */ public interface Applications { /** * Get a list of all relevant applications over a subscription level scope. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedIterable}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Get a list of all relevant applications over a subscription level scope. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of all relevant applications over a subscription level scope as paginated response with {@link - * PagedIterable}. + * @return a list of all relevant applications over a subscription level scope as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationsList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationsList.java index deccd03c7231d..78731b63b2d7c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationsList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ApplicationsList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Page of a security applications list. */ +/** + * Page of a security applications list. + */ @Immutable public final class ApplicationsList { /* @@ -24,13 +26,15 @@ public final class ApplicationsList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ApplicationsList class. */ + /** + * Creates an instance of ApplicationsList class. + */ public ApplicationsList() { } /** * Get the value property: Collection of applications in this page. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocation.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocation.java index f052a42ed9163..2b6a9cd03eba3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocation.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocation.java @@ -6,39 +6,41 @@ import com.azure.resourcemanager.security.fluent.models.AscLocationInner; -/** An immutable client-side representation of AscLocation. */ +/** + * An immutable client-side representation of AscLocation. + */ public interface AscLocation { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the properties property: An empty set of properties. - * + * * @return the properties value. */ Object properties(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AscLocationInner object. - * + * * @return the inner object. */ AscLocationInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocationList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocationList.java index c8d7e61915583..37c1ab8c536ea 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocationList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AscLocationList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of locations where ASC saves your data. */ +/** + * List of locations where ASC saves your data. + */ @Immutable public final class AscLocationList { /* @@ -24,13 +26,15 @@ public final class AscLocationList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AscLocationList class. */ + /** + * Creates an instance of AscLocationList class. + */ public AscLocationList() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessedResourceType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessedResourceType.java index ac3b775039ef3..0491ec20d2dd8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessedResourceType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessedResourceType.java @@ -8,21 +8,29 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Sub-assessment resource type. */ +/** + * Sub-assessment resource type. + */ public final class AssessedResourceType extends ExpandableStringEnum { - /** Static value SqlServerVulnerability for AssessedResourceType. */ + /** + * Static value SqlServerVulnerability for AssessedResourceType. + */ public static final AssessedResourceType SQL_SERVER_VULNERABILITY = fromString("SqlServerVulnerability"); - /** Static value ContainerRegistryVulnerability for AssessedResourceType. */ - public static final AssessedResourceType CONTAINER_REGISTRY_VULNERABILITY = - fromString("ContainerRegistryVulnerability"); + /** + * Static value ContainerRegistryVulnerability for AssessedResourceType. + */ + public static final AssessedResourceType CONTAINER_REGISTRY_VULNERABILITY + = fromString("ContainerRegistryVulnerability"); - /** Static value ServerVulnerability for AssessedResourceType. */ + /** + * Static value ServerVulnerability for AssessedResourceType. + */ public static final AssessedResourceType SERVER_VULNERABILITY = fromString("ServerVulnerability"); /** * Creates a new instance of AssessedResourceType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -31,7 +39,7 @@ public AssessedResourceType() { /** * Creates or finds a AssessedResourceType from its string representation. - * + * * @param name a name to look for. * @return the corresponding AssessedResourceType. */ @@ -42,7 +50,7 @@ public static AssessedResourceType fromString(String name) { /** * Gets known AssessedResourceType values. - * + * * @return known AssessedResourceType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentLinks.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentLinks.java index 7c6620d027bd9..0b4cfd261e351 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentLinks.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentLinks.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Links relevant to the assessment. */ +/** + * Links relevant to the assessment. + */ @Immutable public final class AssessmentLinks { /* @@ -16,13 +18,15 @@ public final class AssessmentLinks { @JsonProperty(value = "azurePortalUri", access = JsonProperty.Access.WRITE_ONLY) private String azurePortalUri; - /** Creates an instance of AssessmentLinks class. */ + /** + * Creates an instance of AssessmentLinks class. + */ public AssessmentLinks() { } /** * Get the azurePortalUri property: Link to assessment in Azure Portal. - * + * * @return the azurePortalUri value. */ public String azurePortalUri() { @@ -31,7 +35,7 @@ public String azurePortalUri() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatus.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatus.java index 3acea6c5664ad..7ff0e4acab510 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatus.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatus.java @@ -8,7 +8,9 @@ import com.azure.core.util.logging.ClientLogger; import com.fasterxml.jackson.annotation.JsonProperty; -/** The result of the assessment. */ +/** + * The result of the assessment. + */ @Fluent public class AssessmentStatus { /* @@ -29,13 +31,15 @@ public class AssessmentStatus { @JsonProperty(value = "description") private String description; - /** Creates an instance of AssessmentStatus class. */ + /** + * Creates an instance of AssessmentStatus class. + */ public AssessmentStatus() { } /** * Get the code property: Programmatic code for the status of the assessment. - * + * * @return the code value. */ public AssessmentStatusCode code() { @@ -44,7 +48,7 @@ public AssessmentStatusCode code() { /** * Set the code property: Programmatic code for the status of the assessment. - * + * * @param code the code value to set. * @return the AssessmentStatus object itself. */ @@ -55,7 +59,7 @@ public AssessmentStatus withCode(AssessmentStatusCode code) { /** * Get the cause property: Programmatic code for the cause of the assessment status. - * + * * @return the cause value. */ public String cause() { @@ -64,7 +68,7 @@ public String cause() { /** * Set the cause property: Programmatic code for the cause of the assessment status. - * + * * @param cause the cause value to set. * @return the AssessmentStatus object itself. */ @@ -75,7 +79,7 @@ public AssessmentStatus withCause(String cause) { /** * Get the description property: Human readable description of the assessment status. - * + * * @return the description value. */ public String description() { @@ -84,7 +88,7 @@ public String description() { /** * Set the description property: Human readable description of the assessment status. - * + * * @param description the description value to set. * @return the AssessmentStatus object itself. */ @@ -95,14 +99,13 @@ public AssessmentStatus withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (code() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property code in model AssessmentStatus")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property code in model AssessmentStatus")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusCode.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusCode.java index bb42fd8a66536..68c19755a065f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusCode.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusCode.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Programmatic code for the status of the assessment. */ +/** + * Programmatic code for the status of the assessment. + */ public final class AssessmentStatusCode extends ExpandableStringEnum { - /** Static value Healthy for AssessmentStatusCode. */ + /** + * Static value Healthy for AssessmentStatusCode. + */ public static final AssessmentStatusCode HEALTHY = fromString("Healthy"); - /** Static value Unhealthy for AssessmentStatusCode. */ + /** + * Static value Unhealthy for AssessmentStatusCode. + */ public static final AssessmentStatusCode UNHEALTHY = fromString("Unhealthy"); - /** Static value NotApplicable for AssessmentStatusCode. */ + /** + * Static value NotApplicable for AssessmentStatusCode. + */ public static final AssessmentStatusCode NOT_APPLICABLE = fromString("NotApplicable"); /** * Creates a new instance of AssessmentStatusCode value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public AssessmentStatusCode() { /** * Creates or finds a AssessmentStatusCode from its string representation. - * + * * @param name a name to look for. * @return the corresponding AssessmentStatusCode. */ @@ -41,7 +49,7 @@ public static AssessmentStatusCode fromString(String name) { /** * Gets known AssessmentStatusCode values. - * + * * @return known AssessmentStatusCode values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusResponse.java index 44837f46729be..d268c5ef77ded 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusResponse.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentStatusResponse.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.time.OffsetDateTime; -/** The result of the assessment. */ +/** + * The result of the assessment. + */ @Fluent public final class AssessmentStatusResponse extends AssessmentStatus { /* @@ -23,14 +25,16 @@ public final class AssessmentStatusResponse extends AssessmentStatus { @JsonProperty(value = "statusChangeDate", access = JsonProperty.Access.WRITE_ONLY) private OffsetDateTime statusChangeDate; - /** Creates an instance of AssessmentStatusResponse class. */ + /** + * Creates an instance of AssessmentStatusResponse class. + */ public AssessmentStatusResponse() { } /** * Get the firstEvaluationDate property: The time that the assessment was created and first evaluated. Returned as * UTC time in ISO 8601 format. - * + * * @return the firstEvaluationDate value. */ public OffsetDateTime firstEvaluationDate() { @@ -40,28 +44,34 @@ public OffsetDateTime firstEvaluationDate() { /** * Get the statusChangeDate property: The time that the status of the assessment last changed. Returned as UTC time * in ISO 8601 format. - * + * * @return the statusChangeDate value. */ public OffsetDateTime statusChangeDate() { return this.statusChangeDate; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AssessmentStatusResponse withCode(AssessmentStatusCode code) { super.withCode(code); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AssessmentStatusResponse withCause(String cause) { super.withCause(cause); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AssessmentStatusResponse withDescription(String description) { super.withDescription(description); @@ -70,7 +80,7 @@ public AssessmentStatusResponse withDescription(String description) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentType.java index 18580a1dd30e2..ecf72873685d3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentType.java @@ -13,21 +13,29 @@ * Policy definition. */ public final class AssessmentType extends ExpandableStringEnum { - /** Static value BuiltIn for AssessmentType. */ + /** + * Static value BuiltIn for AssessmentType. + */ public static final AssessmentType BUILT_IN = fromString("BuiltIn"); - /** Static value CustomPolicy for AssessmentType. */ + /** + * Static value CustomPolicy for AssessmentType. + */ public static final AssessmentType CUSTOM_POLICY = fromString("CustomPolicy"); - /** Static value CustomerManaged for AssessmentType. */ + /** + * Static value CustomerManaged for AssessmentType. + */ public static final AssessmentType CUSTOMER_MANAGED = fromString("CustomerManaged"); - /** Static value VerifiedPartner for AssessmentType. */ + /** + * Static value VerifiedPartner for AssessmentType. + */ public static final AssessmentType VERIFIED_PARTNER = fromString("VerifiedPartner"); /** * Creates a new instance of AssessmentType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +44,7 @@ public AssessmentType() { /** * Creates or finds a AssessmentType from its string representation. - * + * * @param name a name to look for. * @return the corresponding AssessmentType. */ @@ -47,7 +55,7 @@ public static AssessmentType fromString(String name) { /** * Gets known AssessmentType values. - * + * * @return known AssessmentType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Assessments.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Assessments.java index d986d2a6d2542..b5a734d81404d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Assessments.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Assessments.java @@ -8,38 +8,40 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Assessments. */ +/** + * Resource collection API of Assessments. + */ public interface Assessments { /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ PagedIterable list(String scope); /** * Get security assessments on all your scanned resources inside a scope. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return security assessments on all your scanned resources inside a scope as paginated response with {@link - * PagedIterable}. + * @return security assessments on all your scanned resources inside a scope as paginated response with + * {@link PagedIterable}. */ PagedIterable list(String scope, Context context); /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param expand OData expand. Optional. @@ -49,12 +51,12 @@ public interface Assessments { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a security assessment on your scanned resource along with {@link Response}. */ - Response getWithResponse( - String resourceId, String assessmentName, ExpandEnum expand, Context context); + Response getWithResponse(String resourceId, String assessmentName, ExpandEnum expand, + Context context); /** * Get a security assessment on your scanned resource. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -67,7 +69,7 @@ Response getWithResponse( /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. @@ -81,7 +83,7 @@ Response getWithResponse( /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param resourceId The identifier of the resource. * @param assessmentName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -92,7 +94,7 @@ Response getWithResponse( /** * Get a security assessment on your scanned resource. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -103,7 +105,7 @@ Response getWithResponse( /** * Get a security assessment on your scanned resource. - * + * * @param id the resource ID. * @param expand OData expand. Optional. * @param context The context to associate with this operation. @@ -117,7 +119,7 @@ Response getWithResponse( /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -128,7 +130,7 @@ Response getWithResponse( /** * Delete a security assessment on your resource. An assessment metadata that describes this assessment must be * predefined with the same name before inserting the assessment result. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -140,7 +142,7 @@ Response getWithResponse( /** * Begins definition for a new SecurityAssessmentResponse resource. - * + * * @param name resource name. * @return the first stage of the new SecurityAssessmentResponse definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentsMetadatas.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentsMetadatas.java index e5145325396c8..33e7a2d02d99d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentsMetadatas.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AssessmentsMetadatas.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of AssessmentsMetadatas. */ +/** + * Resource collection API of AssessmentsMetadatas. + */ public interface AssessmentsMetadatas { /** * Get metadata information on all assessment types. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on all assessment types as paginated response with {@link PagedIterable}. @@ -21,7 +23,7 @@ public interface AssessmentsMetadatas { /** * Get metadata information on all assessment types. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,7 +34,7 @@ public interface AssessmentsMetadatas { /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -44,7 +46,7 @@ public interface AssessmentsMetadatas { /** * Get metadata information on an assessment type. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -55,29 +57,29 @@ public interface AssessmentsMetadatas { /** * Get metadata information on all assessment types in a specific subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedIterable}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable listBySubscription(); /** * Get metadata information on all assessment types in a specific subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return metadata information on all assessment types in a specific subscription as paginated response with {@link - * PagedIterable}. + * @return metadata information on all assessment types in a specific subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable listBySubscription(Context context); /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -85,12 +87,12 @@ public interface AssessmentsMetadatas { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return metadata information on an assessment type in a specific subscription along with {@link Response}. */ - Response getInSubscriptionWithResponse( - String assessmentMetadataName, Context context); + Response getInSubscriptionWithResponse(String assessmentMetadataName, + Context context); /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -102,7 +104,7 @@ Response getInSubscriptionWithResponse( /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -115,7 +117,7 @@ Response getInSubscriptionWithResponse( /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param assessmentMetadataName The Assessment Key - Unique key for the assessment type. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -125,7 +127,7 @@ Response getInSubscriptionWithResponse( /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -136,7 +138,7 @@ Response getInSubscriptionWithResponse( /** * Get metadata information on an assessment type in a specific subscription. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -149,7 +151,7 @@ Response getInSubscriptionWithResponse( /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -160,7 +162,7 @@ Response getInSubscriptionWithResponse( /** * Delete metadata information on an assessment type in a specific subscription, will cause the deletion of all the * assessments of that type in that subscription. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -172,7 +174,7 @@ Response getInSubscriptionWithResponse( /** * Begins definition for a new SecurityAssessmentMetadataResponse resource. - * + * * @param name resource name. * @return the first stage of the new SecurityAssessmentMetadataResponse definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaExternalSecuritySolution.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaExternalSecuritySolution.java index 218d2ba95a851..2609dfb7d76cb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaExternalSecuritySolution.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaExternalSecuritySolution.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Represents an ATA security solution which sends logs to an OMS workspace. */ +/** + * Represents an ATA security solution which sends logs to an OMS workspace. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") @JsonTypeName("ATA") @Fluent @@ -21,13 +23,15 @@ public final class AtaExternalSecuritySolution extends ExternalSecuritySolutionI @JsonProperty(value = "properties") private AtaSolutionProperties properties; - /** Creates an instance of AtaExternalSecuritySolution class. */ + /** + * Creates an instance of AtaExternalSecuritySolution class. + */ public AtaExternalSecuritySolution() { } /** * Get the properties property: The external security solution properties for ATA solutions. - * + * * @return the properties value. */ public AtaSolutionProperties properties() { @@ -36,7 +40,7 @@ public AtaSolutionProperties properties() { /** * Set the properties property: The external security solution properties for ATA solutions. - * + * * @param properties the properties value to set. * @return the AtaExternalSecuritySolution object itself. */ @@ -47,7 +51,7 @@ public AtaExternalSecuritySolution withProperties(AtaSolutionProperties properti /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaSolutionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaSolutionProperties.java index 5167966f1d5d5..ec7f9ebb9d130 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaSolutionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AtaSolutionProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The external security solution properties for ATA solutions. */ +/** + * The external security solution properties for ATA solutions. + */ @Fluent public final class AtaSolutionProperties extends ExternalSecuritySolutionProperties { /* @@ -16,13 +18,15 @@ public final class AtaSolutionProperties extends ExternalSecuritySolutionPropert @JsonProperty(value = "lastEventReceived") private String lastEventReceived; - /** Creates an instance of AtaSolutionProperties class. */ + /** + * Creates an instance of AtaSolutionProperties class. + */ public AtaSolutionProperties() { } /** * Get the lastEventReceived property: The lastEventReceived property. - * + * * @return the lastEventReceived value. */ public String lastEventReceived() { @@ -31,7 +35,7 @@ public String lastEventReceived() { /** * Set the lastEventReceived property: The lastEventReceived property. - * + * * @param lastEventReceived the lastEventReceived value to set. * @return the AtaSolutionProperties object itself. */ @@ -40,21 +44,27 @@ public AtaSolutionProperties withLastEventReceived(String lastEventReceived) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AtaSolutionProperties withDeviceVendor(String deviceVendor) { super.withDeviceVendor(deviceVendor); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AtaSolutionProperties withDeviceType(String deviceType) { super.withDeviceType(deviceType); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public AtaSolutionProperties withWorkspace(ConnectedWorkspace workspace) { super.withWorkspace(workspace); @@ -63,7 +73,7 @@ public AtaSolutionProperties withWorkspace(ConnectedWorkspace workspace) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationDetailsProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationDetailsProperties.java index 71b7f498e5e93..cfa26ae891dcf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationDetailsProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationDetailsProperties.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Settings for cloud authentication management. */ +/** + * Settings for cloud authentication management. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -21,8 +23,7 @@ @JsonSubTypes({ @JsonSubTypes.Type(name = "awsCreds", value = AwsCredsAuthenticationDetailsProperties.class), @JsonSubTypes.Type(name = "awsAssumeRole", value = AwAssumeRoleAuthenticationDetailsProperties.class), - @JsonSubTypes.Type(name = "gcpCredentials", value = GcpCredentialsDetailsProperties.class) -}) + @JsonSubTypes.Type(name = "gcpCredentials", value = GcpCredentialsDetailsProperties.class) }) @Immutable public class AuthenticationDetailsProperties { /* @@ -37,13 +38,15 @@ public class AuthenticationDetailsProperties { @JsonProperty(value = "grantedPermissions", access = JsonProperty.Access.WRITE_ONLY) private List grantedPermissions; - /** Creates an instance of AuthenticationDetailsProperties class. */ + /** + * Creates an instance of AuthenticationDetailsProperties class. + */ public AuthenticationDetailsProperties() { } /** * Get the authenticationProvisioningState property: State of the multi-cloud connector. - * + * * @return the authenticationProvisioningState value. */ public AuthenticationProvisioningState authenticationProvisioningState() { @@ -52,7 +55,7 @@ public AuthenticationProvisioningState authenticationProvisioningState() { /** * Get the grantedPermissions property: The permissions detected in the cloud account. - * + * * @return the grantedPermissions value. */ public List grantedPermissions() { @@ -61,7 +64,7 @@ public List grantedPermissions() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationProvisioningState.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationProvisioningState.java index e81b2a271af08..d0332051ff533 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationProvisioningState.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationProvisioningState.java @@ -8,23 +8,33 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** State of the multi-cloud connector. */ +/** + * State of the multi-cloud connector. + */ public final class AuthenticationProvisioningState extends ExpandableStringEnum { - /** Static value Valid for AuthenticationProvisioningState. */ + /** + * Static value Valid for AuthenticationProvisioningState. + */ public static final AuthenticationProvisioningState VALID = fromString("Valid"); - /** Static value Invalid for AuthenticationProvisioningState. */ + /** + * Static value Invalid for AuthenticationProvisioningState. + */ public static final AuthenticationProvisioningState INVALID = fromString("Invalid"); - /** Static value Expired for AuthenticationProvisioningState. */ + /** + * Static value Expired for AuthenticationProvisioningState. + */ public static final AuthenticationProvisioningState EXPIRED = fromString("Expired"); - /** Static value IncorrectPolicy for AuthenticationProvisioningState. */ + /** + * Static value IncorrectPolicy for AuthenticationProvisioningState. + */ public static final AuthenticationProvisioningState INCORRECT_POLICY = fromString("IncorrectPolicy"); /** * Creates a new instance of AuthenticationProvisioningState value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +43,7 @@ public AuthenticationProvisioningState() { /** * Creates or finds a AuthenticationProvisioningState from its string representation. - * + * * @param name a name to look for. * @return the corresponding AuthenticationProvisioningState. */ @@ -44,7 +54,7 @@ public static AuthenticationProvisioningState fromString(String name) { /** * Gets known AuthenticationProvisioningState values. - * + * * @return known AuthenticationProvisioningState values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationType.java index 39089c8a4a996..0a61e277f217d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AuthenticationType.java @@ -13,18 +13,24 @@ * account organization credentials. */ public final class AuthenticationType extends ExpandableStringEnum { - /** Static value awsCreds for AuthenticationType. */ + /** + * Static value awsCreds for AuthenticationType. + */ public static final AuthenticationType AWS_CREDS = fromString("awsCreds"); - /** Static value awsAssumeRole for AuthenticationType. */ + /** + * Static value awsAssumeRole for AuthenticationType. + */ public static final AuthenticationType AWS_ASSUME_ROLE = fromString("awsAssumeRole"); - /** Static value gcpCredentials for AuthenticationType. */ + /** + * Static value gcpCredentials for AuthenticationType. + */ public static final AuthenticationType GCP_CREDENTIALS = fromString("gcpCredentials"); /** * Creates a new instance of AuthenticationType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -33,7 +39,7 @@ public AuthenticationType() { /** * Creates or finds a AuthenticationType from its string representation. - * + * * @param name a name to look for. * @return the corresponding AuthenticationType. */ @@ -44,7 +50,7 @@ public static AuthenticationType fromString(String name) { /** * Gets known AuthenticationType values. - * + * * @return known AuthenticationType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Authorization.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Authorization.java new file mode 100644 index 0000000000000..489e98d2f97a3 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Authorization.java @@ -0,0 +1,60 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Authorization payload. + */ +@Fluent +public final class Authorization { + /* + * Gets or sets one-time OAuth code to exchange for refresh and access tokens. + * + * Only used during PUT/PATCH operations. The secret is cleared during GET. + */ + @JsonProperty(value = "code") + private String code; + + /** + * Creates an instance of Authorization class. + */ + public Authorization() { + } + + /** + * Get the code property: Gets or sets one-time OAuth code to exchange for refresh and access tokens. + * + * Only used during PUT/PATCH operations. The secret is cleared during GET. + * + * @return the code value. + */ + public String code() { + return this.code; + } + + /** + * Set the code property: Gets or sets one-time OAuth code to exchange for refresh and access tokens. + * + * Only used during PUT/PATCH operations. The secret is cleared during GET. + * + * @param code the code value to set. + * @return the Authorization object itself. + */ + public Authorization withCode(String code) { + this.code = code; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoDiscovery.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoDiscovery.java new file mode 100644 index 0000000000000..b10fbe6540f0d --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoDiscovery.java @@ -0,0 +1,58 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * AutoDiscovery states. + */ +public final class AutoDiscovery extends ExpandableStringEnum { + /** + * Static value Disabled for AutoDiscovery. + */ + public static final AutoDiscovery DISABLED = fromString("Disabled"); + + /** + * Static value Enabled for AutoDiscovery. + */ + public static final AutoDiscovery ENABLED = fromString("Enabled"); + + /** + * Static value NotApplicable for AutoDiscovery. + */ + public static final AutoDiscovery NOT_APPLICABLE = fromString("NotApplicable"); + + /** + * Creates a new instance of AutoDiscovery value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public AutoDiscovery() { + } + + /** + * Creates or finds a AutoDiscovery from its string representation. + * + * @param name a name to look for. + * @return the corresponding AutoDiscovery. + */ + @JsonCreator + public static AutoDiscovery fromString(String name) { + return fromString(name, AutoDiscovery.class); + } + + /** + * Gets known AutoDiscovery values. + * + * @return known AutoDiscovery values. + */ + public static Collection values() { + return values(AutoDiscovery.class); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvision.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvision.java index 0b365391e9a0b..c4f1323e11b60 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvision.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvision.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Describes what kind of security agent provisioning action to take. */ +/** + * Describes what kind of security agent provisioning action to take. + */ public final class AutoProvision extends ExpandableStringEnum { - /** Static value On for AutoProvision. */ + /** + * Static value On for AutoProvision. + */ public static final AutoProvision ON = fromString("On"); - /** Static value Off for AutoProvision. */ + /** + * Static value Off for AutoProvision. + */ public static final AutoProvision OFF = fromString("Off"); /** * Creates a new instance of AutoProvision value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public AutoProvision() { /** * Creates or finds a AutoProvision from its string representation. - * + * * @param name a name to look for. * @return the corresponding AutoProvision. */ @@ -38,7 +44,7 @@ public static AutoProvision fromString(String name) { /** * Gets known AutoProvision values. - * + * * @return known AutoProvision values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSetting.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSetting.java index df4279031dd7a..9925a34a0a57d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSetting.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSetting.java @@ -7,51 +7,61 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AutoProvisioningSettingInner; -/** An immutable client-side representation of AutoProvisioningSetting. */ +/** + * An immutable client-side representation of AutoProvisioningSetting. + */ public interface AutoProvisioningSetting { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the autoProvision property: Describes what kind of security agent provisioning action to take. - * + * * @return the autoProvision value. */ AutoProvision autoProvision(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AutoProvisioningSettingInner object. - * + * * @return the inner object. */ AutoProvisioningSettingInner innerModel(); - /** The entirety of the AutoProvisioningSetting definition. */ + /** + * The entirety of the AutoProvisioningSetting definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithCreate { } - /** The AutoProvisioningSetting definition stages. */ + + /** + * The AutoProvisioningSetting definition stages. + */ interface DefinitionStages { - /** The first stage of the AutoProvisioningSetting definition. */ + /** + * The first stage of the AutoProvisioningSetting definition. + */ interface Blank extends WithCreate { } + /** * The stage of the AutoProvisioningSetting definition which contains all the minimum required properties for * the resource to be created, but also allows for any other optional properties to be specified. @@ -59,40 +69,44 @@ interface Blank extends WithCreate { interface WithCreate extends DefinitionStages.WithAutoProvision { /** * Executes the create request. - * + * * @return the created resource. */ AutoProvisioningSetting create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ AutoProvisioningSetting create(Context context); } - /** The stage of the AutoProvisioningSetting definition allowing to specify autoProvision. */ + + /** + * The stage of the AutoProvisioningSetting definition allowing to specify autoProvision. + */ interface WithAutoProvision { /** * Specifies the autoProvision property: Describes what kind of security agent provisioning action to take. - * + * * @param autoProvision Describes what kind of security agent provisioning action to take. * @return the next definition stage. */ WithCreate withAutoProvision(AutoProvision autoProvision); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ AutoProvisioningSetting refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettingList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettingList.java index de263eefa9dd6..f1fff0f0d7576 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettingList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettingList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of all the auto provisioning settings response. */ +/** + * List of all the auto provisioning settings response. + */ @Fluent public final class AutoProvisioningSettingList { /* @@ -24,13 +26,15 @@ public final class AutoProvisioningSettingList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AutoProvisioningSettingList class. */ + /** + * Creates an instance of AutoProvisioningSettingList class. + */ public AutoProvisioningSettingList() { } /** * Get the value property: List of all the auto provisioning settings. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of all the auto provisioning settings. - * + * * @param value the value value to set. * @return the AutoProvisioningSettingList object itself. */ @@ -50,7 +54,7 @@ public AutoProvisioningSettingList withValue(List /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettings.java index 2b050e6bc336e..772fb6f8a28b8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettings.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutoProvisioningSettings.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of AutoProvisioningSettings. */ +/** + * Resource collection API of AutoProvisioningSettings. + */ public interface AutoProvisioningSettings { /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of all the auto provisioning settings response as paginated response with {@link PagedIterable}. @@ -21,7 +23,7 @@ public interface AutoProvisioningSettings { /** * Exposes the auto provisioning settings of the subscriptions. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -32,7 +34,7 @@ public interface AutoProvisioningSettings { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -44,7 +46,7 @@ public interface AutoProvisioningSettings { /** * Details of a specific setting. - * + * * @param settingName Auto provisioning setting key. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -55,7 +57,7 @@ public interface AutoProvisioningSettings { /** * Details of a specific setting. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -66,7 +68,7 @@ public interface AutoProvisioningSettings { /** * Details of a specific setting. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -78,7 +80,7 @@ public interface AutoProvisioningSettings { /** * Begins definition for a new AutoProvisioningSetting resource. - * + * * @param name resource name. * @return the first stage of the new AutoProvisioningSetting definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automation.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automation.java index b584f1a114a34..1e248cccbe54b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automation.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automation.java @@ -11,53 +11,55 @@ import java.util.List; import java.util.Map; -/** An immutable client-side representation of Automation. */ +/** + * An immutable client-side representation of Automation. + */ public interface Automation { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: The geo-location where the resource lives. - * + * * @return the location value. */ String location(); /** * Gets the tags property: Resource tags. - * + * * @return the tags value. */ Map tags(); /** * Gets the description property: The security automation description. - * + * * @return the description value. */ String description(); /** * Gets the isEnabled property: Indicates whether the security automation is enabled. - * + * * @return the isEnabled value. */ Boolean isEnabled(); @@ -66,7 +68,7 @@ public interface Automation { * Gets the scopes property: A collection of scopes on which the security automations logic is applied. Supported * scopes are the subscription itself or a resource group under that subscription. The automation will only apply on * defined scopes. - * + * * @return the scopes value. */ List scopes(); @@ -74,7 +76,7 @@ public interface Automation { /** * Gets the sources property: A collection of the source event types which evaluate the security automation set of * rules. - * + * * @return the sources value. */ List sources(); @@ -82,56 +84,63 @@ public interface Automation { /** * Gets the actions property: A collection of the actions which are triggered if all the configured rules * evaluations, within at least one rule set, are true. - * + * * @return the actions value. */ List actions(); /** * Gets the region of the resource. - * + * * @return the region of the resource. */ Region region(); /** * Gets the name of the resource region. - * + * * @return the name of the resource region. */ String regionName(); /** * Gets the name of the resource group. - * + * * @return the name of the resource group. */ String resourceGroupName(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AutomationInner object. - * + * * @return the inner object. */ AutomationInner innerModel(); - /** The entirety of the Automation definition. */ - interface Definition - extends DefinitionStages.Blank, - DefinitionStages.WithLocation, - DefinitionStages.WithResourceGroup, - DefinitionStages.WithCreate { + /** + * The entirety of the Automation definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithLocation, + DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The Automation definition stages. */ + + /** + * The Automation definition stages. + */ interface DefinitionStages { - /** The first stage of the Automation definition. */ + /** + * The first stage of the Automation definition. + */ interface Blank extends WithLocation { } - /** The stage of the Automation definition allowing to specify location. */ + + /** + * The stage of the Automation definition allowing to specify location. + */ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ @@ -139,229 +148,266 @@ interface WithLocation { /** * Specifies the region for the resource. - * + * * @param location The geo-location where the resource lives. * @return the next definition stage. */ WithResourceGroup withRegion(String location); } - /** The stage of the Automation definition allowing to specify parent resource. */ + + /** + * The stage of the Automation definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @return the next definition stage. */ WithCreate withExistingResourceGroup(String resourceGroupName); } + /** * The stage of the Automation definition which contains all the minimum required properties for the resource to * be created, but also allows for any other optional properties to be specified. */ interface WithCreate - extends DefinitionStages.WithTags, - DefinitionStages.WithDescription, - DefinitionStages.WithIsEnabled, - DefinitionStages.WithScopes, - DefinitionStages.WithSources, - DefinitionStages.WithActions { + extends DefinitionStages.WithTags, DefinitionStages.WithDescription, DefinitionStages.WithIsEnabled, + DefinitionStages.WithScopes, DefinitionStages.WithSources, DefinitionStages.WithActions { /** * Executes the create request. - * + * * @return the created resource. */ Automation create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ Automation create(Context context); } - /** The stage of the Automation definition allowing to specify tags. */ + + /** + * The stage of the Automation definition allowing to specify tags. + */ interface WithTags { /** * Specifies the tags property: Resource tags.. - * + * * @param tags Resource tags. * @return the next definition stage. */ WithCreate withTags(Map tags); } - /** The stage of the Automation definition allowing to specify description. */ + + /** + * The stage of the Automation definition allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: The security automation description.. - * + * * @param description The security automation description. * @return the next definition stage. */ WithCreate withDescription(String description); } - /** The stage of the Automation definition allowing to specify isEnabled. */ + + /** + * The stage of the Automation definition allowing to specify isEnabled. + */ interface WithIsEnabled { /** * Specifies the isEnabled property: Indicates whether the security automation is enabled.. - * + * * @param isEnabled Indicates whether the security automation is enabled. * @return the next definition stage. */ WithCreate withIsEnabled(Boolean isEnabled); } - /** The stage of the Automation definition allowing to specify scopes. */ + + /** + * The stage of the Automation definition allowing to specify scopes. + */ interface WithScopes { /** * Specifies the scopes property: A collection of scopes on which the security automations logic is applied. * Supported scopes are the subscription itself or a resource group under that subscription. The automation * will only apply on defined scopes.. - * + * * @param scopes A collection of scopes on which the security automations logic is applied. Supported scopes - * are the subscription itself or a resource group under that subscription. The automation will only - * apply on defined scopes. + * are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. * @return the next definition stage. */ WithCreate withScopes(List scopes); } - /** The stage of the Automation definition allowing to specify sources. */ + + /** + * The stage of the Automation definition allowing to specify sources. + */ interface WithSources { /** * Specifies the sources property: A collection of the source event types which evaluate the security * automation set of rules.. - * + * * @param sources A collection of the source event types which evaluate the security automation set of - * rules. + * rules. * @return the next definition stage. */ WithCreate withSources(List sources); } - /** The stage of the Automation definition allowing to specify actions. */ + + /** + * The stage of the Automation definition allowing to specify actions. + */ interface WithActions { /** * Specifies the actions property: A collection of the actions which are triggered if all the configured * rules evaluations, within at least one rule set, are true.. - * + * * @param actions A collection of the actions which are triggered if all the configured rules evaluations, - * within at least one rule set, are true. + * within at least one rule set, are true. * @return the next definition stage. */ WithCreate withActions(List actions); } } + /** * Begins update for the Automation resource. - * + * * @return the stage of resource update. */ Automation.Update update(); - /** The template for Automation update. */ - interface Update - extends UpdateStages.WithTags, - UpdateStages.WithDescription, - UpdateStages.WithIsEnabled, - UpdateStages.WithScopes, - UpdateStages.WithSources, - UpdateStages.WithActions { + /** + * The template for Automation update. + */ + interface Update extends UpdateStages.WithTags, UpdateStages.WithDescription, UpdateStages.WithIsEnabled, + UpdateStages.WithScopes, UpdateStages.WithSources, UpdateStages.WithActions { /** * Executes the update request. - * + * * @return the updated resource. */ Automation apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ Automation apply(Context context); } - /** The Automation update stages. */ + + /** + * The Automation update stages. + */ interface UpdateStages { - /** The stage of the Automation update allowing to specify tags. */ + /** + * The stage of the Automation update allowing to specify tags. + */ interface WithTags { /** - * Specifies the tags property: Resource tags.. - * - * @param tags Resource tags. + * Specifies the tags property: A list of key value pairs that describe the resource.. + * + * @param tags A list of key value pairs that describe the resource. * @return the next definition stage. */ Update withTags(Map tags); } - /** The stage of the Automation update allowing to specify description. */ + + /** + * The stage of the Automation update allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: The security automation description.. - * + * * @param description The security automation description. * @return the next definition stage. */ Update withDescription(String description); } - /** The stage of the Automation update allowing to specify isEnabled. */ + + /** + * The stage of the Automation update allowing to specify isEnabled. + */ interface WithIsEnabled { /** * Specifies the isEnabled property: Indicates whether the security automation is enabled.. - * + * * @param isEnabled Indicates whether the security automation is enabled. * @return the next definition stage. */ Update withIsEnabled(Boolean isEnabled); } - /** The stage of the Automation update allowing to specify scopes. */ + + /** + * The stage of the Automation update allowing to specify scopes. + */ interface WithScopes { /** * Specifies the scopes property: A collection of scopes on which the security automations logic is applied. * Supported scopes are the subscription itself or a resource group under that subscription. The automation * will only apply on defined scopes.. - * + * * @param scopes A collection of scopes on which the security automations logic is applied. Supported scopes - * are the subscription itself or a resource group under that subscription. The automation will only - * apply on defined scopes. + * are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. * @return the next definition stage. */ Update withScopes(List scopes); } - /** The stage of the Automation update allowing to specify sources. */ + + /** + * The stage of the Automation update allowing to specify sources. + */ interface WithSources { /** * Specifies the sources property: A collection of the source event types which evaluate the security * automation set of rules.. - * + * * @param sources A collection of the source event types which evaluate the security automation set of - * rules. + * rules. * @return the next definition stage. */ Update withSources(List sources); } - /** The stage of the Automation update allowing to specify actions. */ + + /** + * The stage of the Automation update allowing to specify actions. + */ interface WithActions { /** * Specifies the actions property: A collection of the actions which are triggered if all the configured * rules evaluations, within at least one rule set, are true.. - * + * * @param actions A collection of the actions which are triggered if all the configured rules evaluations, - * within at least one rule set, are true. + * within at least one rule set, are true. * @return the next definition stage. */ Update withActions(List actions); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ Automation refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ @@ -370,7 +416,7 @@ interface WithActions { /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param automation The security automation resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -383,7 +429,7 @@ interface WithActions { /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationAction.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationAction.java index 5aab572dca457..2932d91e58953 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationAction.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationAction.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The action that should be triggered. */ +/** + * The action that should be triggered. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -19,17 +21,18 @@ @JsonSubTypes({ @JsonSubTypes.Type(name = "LogicApp", value = AutomationActionLogicApp.class), @JsonSubTypes.Type(name = "EventHub", value = AutomationActionEventHub.class), - @JsonSubTypes.Type(name = "Workspace", value = AutomationActionWorkspace.class) -}) + @JsonSubTypes.Type(name = "Workspace", value = AutomationActionWorkspace.class) }) @Immutable public class AutomationAction { - /** Creates an instance of AutomationAction class. */ + /** + * Creates an instance of AutomationAction class. + */ public AutomationAction() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionEventHub.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionEventHub.java index 8d4a940480da5..f0f5d7def421d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionEventHub.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionEventHub.java @@ -35,13 +35,21 @@ public final class AutomationActionEventHub extends AutomationAction { @JsonProperty(value = "connectionString") private String connectionString; - /** Creates an instance of AutomationActionEventHub class. */ + /* + * Indicates whether the trusted service is enabled or not. + */ + @JsonProperty(value = "isTrustedServiceEnabled") + private Boolean isTrustedServiceEnabled; + + /** + * Creates an instance of AutomationActionEventHub class. + */ public AutomationActionEventHub() { } /** * Get the eventHubResourceId property: The target Event Hub Azure Resource ID. - * + * * @return the eventHubResourceId value. */ public String eventHubResourceId() { @@ -50,7 +58,7 @@ public String eventHubResourceId() { /** * Set the eventHubResourceId property: The target Event Hub Azure Resource ID. - * + * * @param eventHubResourceId the eventHubResourceId value to set. * @return the AutomationActionEventHub object itself. */ @@ -61,7 +69,7 @@ public AutomationActionEventHub withEventHubResourceId(String eventHubResourceId /** * Get the sasPolicyName property: The target Event Hub SAS policy name. - * + * * @return the sasPolicyName value. */ public String sasPolicyName() { @@ -71,7 +79,7 @@ public String sasPolicyName() { /** * Get the connectionString property: The target Event Hub connection string (it will not be included in any * response). - * + * * @return the connectionString value. */ public String connectionString() { @@ -81,7 +89,7 @@ public String connectionString() { /** * Set the connectionString property: The target Event Hub connection string (it will not be included in any * response). - * + * * @param connectionString the connectionString value to set. * @return the AutomationActionEventHub object itself. */ @@ -90,9 +98,29 @@ public AutomationActionEventHub withConnectionString(String connectionString) { return this; } + /** + * Get the isTrustedServiceEnabled property: Indicates whether the trusted service is enabled or not. + * + * @return the isTrustedServiceEnabled value. + */ + public Boolean isTrustedServiceEnabled() { + return this.isTrustedServiceEnabled; + } + + /** + * Set the isTrustedServiceEnabled property: Indicates whether the trusted service is enabled or not. + * + * @param isTrustedServiceEnabled the isTrustedServiceEnabled value to set. + * @return the AutomationActionEventHub object itself. + */ + public AutomationActionEventHub withIsTrustedServiceEnabled(Boolean isTrustedServiceEnabled) { + this.isTrustedServiceEnabled = isTrustedServiceEnabled; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionLogicApp.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionLogicApp.java index 38fe7c5d0598e..18128148fbc71 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionLogicApp.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionLogicApp.java @@ -10,8 +10,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName; /** - * The logic app action that should be triggered. To learn more about Microsoft Defender for Cloud's Workflow Automation - * capabilities, visit https://aka.ms/ASCWorkflowAutomationLearnMore. + * The logic app action that should be triggered. To learn more about Microsoft Defender for Cloud's Workflow + * Automation capabilities, visit https://aka.ms/ASCWorkflowAutomationLearnMore. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "actionType") @JsonTypeName("LogicApp") @@ -30,14 +30,16 @@ public final class AutomationActionLogicApp extends AutomationAction { @JsonProperty(value = "uri") private String uri; - /** Creates an instance of AutomationActionLogicApp class. */ + /** + * Creates an instance of AutomationActionLogicApp class. + */ public AutomationActionLogicApp() { } /** * Get the logicAppResourceId property: The triggered Logic App Azure Resource ID. This can also reside on other * subscriptions, given that you have permissions to trigger the Logic App. - * + * * @return the logicAppResourceId value. */ public String logicAppResourceId() { @@ -47,7 +49,7 @@ public String logicAppResourceId() { /** * Set the logicAppResourceId property: The triggered Logic App Azure Resource ID. This can also reside on other * subscriptions, given that you have permissions to trigger the Logic App. - * + * * @param logicAppResourceId the logicAppResourceId value to set. * @return the AutomationActionLogicApp object itself. */ @@ -58,7 +60,7 @@ public AutomationActionLogicApp withLogicAppResourceId(String logicAppResourceId /** * Get the uri property: The Logic App trigger URI endpoint (it will not be included in any response). - * + * * @return the uri value. */ public String uri() { @@ -67,7 +69,7 @@ public String uri() { /** * Set the uri property: The Logic App trigger URI endpoint (it will not be included in any response). - * + * * @param uri the uri value to set. * @return the AutomationActionLogicApp object itself. */ @@ -78,7 +80,7 @@ public AutomationActionLogicApp withUri(String uri) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionWorkspace.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionWorkspace.java index 9fd967924bdfe..ede8b7b09974a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionWorkspace.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationActionWorkspace.java @@ -10,11 +10,11 @@ import com.fasterxml.jackson.annotation.JsonTypeName; /** - * The Log Analytics Workspace to which event data will be exported. Security alerts data will reside in the + * The�Log�Analytics�Workspace�to�which�event data will be exported. Security alerts data will reside in the * 'SecurityAlert' table and the assessments data will reside in the 'SecurityRecommendation' table (under the - * 'Security'/'SecurityCenterFree' solutions). Note that in order to view the data in the workspace, the Security Center - * Log Analytics free/standard solution needs to be enabled on that workspace. To learn more about Microsoft Defender - * for Cloud continuous export capabilities, visit https://aka.ms/ASCExportLearnMore. + * 'Security'/'SecurityCenterFree' solutions). Note that in order to view the data in the workspace, the Security + * Center Log Analytics free/standard solution needs to be enabled on that workspace. To learn more about Microsoft + * Defender for Cloud continuous export capabilities, visit https://aka.ms/ASCExportLearnMore. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "actionType") @JsonTypeName("Workspace") @@ -26,13 +26,15 @@ public final class AutomationActionWorkspace extends AutomationAction { @JsonProperty(value = "workspaceResourceId") private String workspaceResourceId; - /** Creates an instance of AutomationActionWorkspace class. */ + /** + * Creates an instance of AutomationActionWorkspace class. + */ public AutomationActionWorkspace() { } /** * Get the workspaceResourceId property: The fully qualified Log Analytics Workspace Azure Resource ID. - * + * * @return the workspaceResourceId value. */ public String workspaceResourceId() { @@ -41,7 +43,7 @@ public String workspaceResourceId() { /** * Set the workspaceResourceId property: The fully qualified Log Analytics Workspace Azure Resource ID. - * + * * @param workspaceResourceId the workspaceResourceId value to set. * @return the AutomationActionWorkspace object itself. */ @@ -52,7 +54,7 @@ public AutomationActionWorkspace withWorkspaceResourceId(String workspaceResourc /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationList.java index 4bc9fc3e432c4..d330a5ca764d5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationList.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of security automations response. */ +/** + * List of security automations response. + */ @Fluent public final class AutomationList { /* @@ -25,13 +27,15 @@ public final class AutomationList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of AutomationList class. */ + /** + * Creates an instance of AutomationList class. + */ public AutomationList() { } /** * Get the value property: The list of security automations under the given scope. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: The list of security automations under the given scope. - * + * * @param value the value value to set. * @return the AutomationList object itself. */ @@ -51,7 +55,7 @@ public AutomationList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model AutomationList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model AutomationList")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationRuleSet.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationRuleSet.java index eb1a7a25fcd6d..71584c443469d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationRuleSet.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationRuleSet.java @@ -9,8 +9,8 @@ import java.util.List; /** - * A rule set which evaluates all its rules upon an event interception. Only when all the included rules in the rule set - * will be evaluated as 'true', will the event trigger the defined actions. + * A rule set which evaluates all its rules upon an event interception. Only when all the included rules in the rule + * set will be evaluated as 'true', will the event trigger the defined actions. */ @Fluent public final class AutomationRuleSet { @@ -20,13 +20,15 @@ public final class AutomationRuleSet { @JsonProperty(value = "rules") private List rules; - /** Creates an instance of AutomationRuleSet class. */ + /** + * Creates an instance of AutomationRuleSet class. + */ public AutomationRuleSet() { } /** * Get the rules property: The rules property. - * + * * @return the rules value. */ public List rules() { @@ -35,7 +37,7 @@ public List rules() { /** * Set the rules property: The rules property. - * + * * @param rules the rules value to set. * @return the AutomationRuleSet object itself. */ @@ -46,7 +48,7 @@ public AutomationRuleSet withRules(List rules) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationScope.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationScope.java index b6797a2853f0f..1f804b33d891c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationScope.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationScope.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** A single automation scope. */ +/** + * A single automation scope. + */ @Fluent public final class AutomationScope { /* @@ -23,13 +25,15 @@ public final class AutomationScope { @JsonProperty(value = "scopePath") private String scopePath; - /** Creates an instance of AutomationScope class. */ + /** + * Creates an instance of AutomationScope class. + */ public AutomationScope() { } /** * Get the description property: The resources scope description. - * + * * @return the description value. */ public String description() { @@ -38,7 +42,7 @@ public String description() { /** * Set the description property: The resources scope description. - * + * * @param description the description value to set. * @return the AutomationScope object itself. */ @@ -50,7 +54,7 @@ public AutomationScope withDescription(String description) { /** * Get the scopePath property: The resources scope path. Can be the subscription on which the automation is defined * on or a resource group under that subscription (fully qualified Azure resource IDs). - * + * * @return the scopePath value. */ public String scopePath() { @@ -60,7 +64,7 @@ public String scopePath() { /** * Set the scopePath property: The resources scope path. Can be the subscription on which the automation is defined * on or a resource group under that subscription (fully qualified Azure resource IDs). - * + * * @param scopePath the scopePath value to set. * @return the AutomationScope object itself. */ @@ -71,7 +75,7 @@ public AutomationScope withScopePath(String scopePath) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationSource.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationSource.java index 7488319504ca2..26aba46562a9e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationSource.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationSource.java @@ -28,13 +28,15 @@ public final class AutomationSource { @JsonProperty(value = "ruleSets") private List ruleSets; - /** Creates an instance of AutomationSource class. */ + /** + * Creates an instance of AutomationSource class. + */ public AutomationSource() { } /** * Get the eventSource property: A valid event source type. - * + * * @return the eventSource value. */ public EventSource eventSource() { @@ -43,7 +45,7 @@ public EventSource eventSource() { /** * Set the eventSource property: A valid event source type. - * + * * @param eventSource the eventSource value to set. * @return the AutomationSource object itself. */ @@ -55,7 +57,7 @@ public AutomationSource withEventSource(EventSource eventSource) { /** * Get the ruleSets property: A set of rules which evaluate upon event interception. A logical disjunction is * applied between defined rule sets (logical 'or'). - * + * * @return the ruleSets value. */ public List ruleSets() { @@ -65,7 +67,7 @@ public List ruleSets() { /** * Set the ruleSets property: A set of rules which evaluate upon event interception. A logical disjunction is * applied between defined rule sets (logical 'or'). - * + * * @param ruleSets the ruleSets value to set. * @return the AutomationSource object itself. */ @@ -76,7 +78,7 @@ public AutomationSource withRuleSets(List ruleSets) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationTriggeringRule.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationTriggeringRule.java index 662e8158cfe2f..a9e3792e819ca 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationTriggeringRule.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationTriggeringRule.java @@ -37,13 +37,15 @@ public final class AutomationTriggeringRule { @JsonProperty(value = "operator") private Operator operator; - /** Creates an instance of AutomationTriggeringRule class. */ + /** + * Creates an instance of AutomationTriggeringRule class. + */ public AutomationTriggeringRule() { } /** * Get the propertyJPath property: The JPath of the entity model property that should be checked. - * + * * @return the propertyJPath value. */ public String propertyJPath() { @@ -52,7 +54,7 @@ public String propertyJPath() { /** * Set the propertyJPath property: The JPath of the entity model property that should be checked. - * + * * @param propertyJPath the propertyJPath value to set. * @return the AutomationTriggeringRule object itself. */ @@ -64,7 +66,7 @@ public AutomationTriggeringRule withPropertyJPath(String propertyJPath) { /** * Get the propertyType property: The data type of the compared operands (string, integer, floating point number or * a boolean [true/false]]. - * + * * @return the propertyType value. */ public PropertyType propertyType() { @@ -74,7 +76,7 @@ public PropertyType propertyType() { /** * Set the propertyType property: The data type of the compared operands (string, integer, floating point number or * a boolean [true/false]]. - * + * * @param propertyType the propertyType value to set. * @return the AutomationTriggeringRule object itself. */ @@ -85,7 +87,7 @@ public AutomationTriggeringRule withPropertyType(PropertyType propertyType) { /** * Get the expectedValue property: The expected value. - * + * * @return the expectedValue value. */ public String expectedValue() { @@ -94,7 +96,7 @@ public String expectedValue() { /** * Set the expectedValue property: The expected value. - * + * * @param expectedValue the expectedValue value to set. * @return the AutomationTriggeringRule object itself. */ @@ -106,7 +108,7 @@ public AutomationTriggeringRule withExpectedValue(String expectedValue) { /** * Get the operator property: A valid comparer operator to use. A case-insensitive comparison will be applied for * String PropertyType. - * + * * @return the operator value. */ public Operator operator() { @@ -116,7 +118,7 @@ public Operator operator() { /** * Set the operator property: A valid comparer operator to use. A case-insensitive comparison will be applied for * String PropertyType. - * + * * @param operator the operator value to set. * @return the AutomationTriggeringRule object itself. */ @@ -127,7 +129,7 @@ public AutomationTriggeringRule withOperator(Operator operator) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationUpdateModel.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationUpdateModel.java new file mode 100644 index 0000000000000..7c58483c71e5c --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationUpdateModel.java @@ -0,0 +1,183 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.fluent.models.AutomationProperties; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + +/** + * The update model of security automation resource. + */ +@Fluent +public final class AutomationUpdateModel extends Tags { + /* + * Security automation data + */ + @JsonProperty(value = "properties") + private AutomationProperties innerProperties; + + /** + * Creates an instance of AutomationUpdateModel class. + */ + public AutomationUpdateModel() { + } + + /** + * Get the innerProperties property: Security automation data. + * + * @return the innerProperties value. + */ + private AutomationProperties innerProperties() { + return this.innerProperties; + } + + /** + * {@inheritDoc} + */ + @Override + public AutomationUpdateModel withTags(Map tags) { + super.withTags(tags); + return this; + } + + /** + * Get the description property: The security automation description. + * + * @return the description value. + */ + public String description() { + return this.innerProperties() == null ? null : this.innerProperties().description(); + } + + /** + * Set the description property: The security automation description. + * + * @param description the description value to set. + * @return the AutomationUpdateModel object itself. + */ + public AutomationUpdateModel withDescription(String description) { + if (this.innerProperties() == null) { + this.innerProperties = new AutomationProperties(); + } + this.innerProperties().withDescription(description); + return this; + } + + /** + * Get the isEnabled property: Indicates whether the security automation is enabled. + * + * @return the isEnabled value. + */ + public Boolean isEnabled() { + return this.innerProperties() == null ? null : this.innerProperties().isEnabled(); + } + + /** + * Set the isEnabled property: Indicates whether the security automation is enabled. + * + * @param isEnabled the isEnabled value to set. + * @return the AutomationUpdateModel object itself. + */ + public AutomationUpdateModel withIsEnabled(Boolean isEnabled) { + if (this.innerProperties() == null) { + this.innerProperties = new AutomationProperties(); + } + this.innerProperties().withIsEnabled(isEnabled); + return this; + } + + /** + * Get the scopes property: A collection of scopes on which the security automations logic is applied. Supported + * scopes are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. + * + * @return the scopes value. + */ + public List scopes() { + return this.innerProperties() == null ? null : this.innerProperties().scopes(); + } + + /** + * Set the scopes property: A collection of scopes on which the security automations logic is applied. Supported + * scopes are the subscription itself or a resource group under that subscription. The automation will only apply + * on defined scopes. + * + * @param scopes the scopes value to set. + * @return the AutomationUpdateModel object itself. + */ + public AutomationUpdateModel withScopes(List scopes) { + if (this.innerProperties() == null) { + this.innerProperties = new AutomationProperties(); + } + this.innerProperties().withScopes(scopes); + return this; + } + + /** + * Get the sources property: A collection of the source event types which evaluate the security automation set of + * rules. + * + * @return the sources value. + */ + public List sources() { + return this.innerProperties() == null ? null : this.innerProperties().sources(); + } + + /** + * Set the sources property: A collection of the source event types which evaluate the security automation set of + * rules. + * + * @param sources the sources value to set. + * @return the AutomationUpdateModel object itself. + */ + public AutomationUpdateModel withSources(List sources) { + if (this.innerProperties() == null) { + this.innerProperties = new AutomationProperties(); + } + this.innerProperties().withSources(sources); + return this; + } + + /** + * Get the actions property: A collection of the actions which are triggered if all the configured rules + * evaluations, within at least one rule set, are true. + * + * @return the actions value. + */ + public List actions() { + return this.innerProperties() == null ? null : this.innerProperties().actions(); + } + + /** + * Set the actions property: A collection of the actions which are triggered if all the configured rules + * evaluations, within at least one rule set, are true. + * + * @param actions the actions value to set. + * @return the AutomationUpdateModel object itself. + */ + public AutomationUpdateModel withActions(List actions) { + if (this.innerProperties() == null) { + this.innerProperties = new AutomationProperties(); + } + this.innerProperties().withActions(actions); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationValidationStatus.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationValidationStatus.java index 61079decd3e68..5aa5b623c85ca 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationValidationStatus.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AutomationValidationStatus.java @@ -6,25 +6,27 @@ import com.azure.resourcemanager.security.fluent.models.AutomationValidationStatusInner; -/** An immutable client-side representation of AutomationValidationStatus. */ +/** + * An immutable client-side representation of AutomationValidationStatus. + */ public interface AutomationValidationStatus { /** * Gets the isValid property: Indicates whether the model is valid or not. - * + * * @return the isValid value. */ Boolean isValid(); /** * Gets the message property: The validation message. - * + * * @return the message value. */ String message(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.AutomationValidationStatusInner object. - * + * * @return the inner object. */ AutomationValidationStatusInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automations.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automations.java index 73b46db3ba19f..6b9a1155a8282 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automations.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Automations.java @@ -9,12 +9,14 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.AutomationInner; -/** Resource collection API of Automations. */ +/** + * Resource collection API of Automations. + */ public interface Automations { /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return list of security automations response as paginated response with {@link PagedIterable}. @@ -24,7 +26,7 @@ public interface Automations { /** * Lists all the security automations in the specified subscription. Use the 'nextLink' property in the response to * get the next page of security automations for the specified subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -36,9 +38,9 @@ public interface Automations { /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -49,9 +51,9 @@ public interface Automations { /** * Lists all the security automations in the specified resource group. Use the 'nextLink' property in the response * to get the next page of security automations for the specified resource group. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -62,9 +64,9 @@ public interface Automations { /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -72,14 +74,14 @@ public interface Automations { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the security automation resource along with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String automationName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, String automationName, + Context context); /** * Retrieves information about the model of a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -90,9 +92,9 @@ Response getByResourceGroupWithResponse( /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -104,9 +106,9 @@ Response getByResourceGroupWithResponse( /** * Deletes a security automation. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -117,9 +119,9 @@ Response getByResourceGroupWithResponse( /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @param context The context to associate with this operation. @@ -128,15 +130,15 @@ Response getByResourceGroupWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the security automation model state property bag along with {@link Response}. */ - Response validateWithResponse( - String resourceGroupName, String automationName, AutomationInner automation, Context context); + Response validateWithResponse(String resourceGroupName, String automationName, + AutomationInner automation, Context context); /** * Validates the security automation model before create or update. Any validation errors are returned to the * client. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param automationName The security automation name. * @param automation The security automation resource. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -148,7 +150,7 @@ Response validateWithResponse( /** * Retrieves information about the model of a security automation. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -159,7 +161,7 @@ Response validateWithResponse( /** * Retrieves information about the model of a security automation. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -171,7 +173,7 @@ Response validateWithResponse( /** * Deletes a security automation. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -181,7 +183,7 @@ Response validateWithResponse( /** * Deletes a security automation. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -193,7 +195,7 @@ Response validateWithResponse( /** * Begins definition for a new Automation resource. - * + * * @param name resource name. * @return the first stage of the new Automation definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwAssumeRoleAuthenticationDetailsProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwAssumeRoleAuthenticationDetailsProperties.java index 636cbdb29fbb6..970082f3ded18 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwAssumeRoleAuthenticationDetailsProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwAssumeRoleAuthenticationDetailsProperties.java @@ -38,13 +38,15 @@ public final class AwAssumeRoleAuthenticationDetailsProperties extends Authentic @JsonProperty(value = "awsExternalId", required = true) private String awsExternalId; - /** Creates an instance of AwAssumeRoleAuthenticationDetailsProperties class. */ + /** + * Creates an instance of AwAssumeRoleAuthenticationDetailsProperties class. + */ public AwAssumeRoleAuthenticationDetailsProperties() { } /** * Get the accountId property: The ID of the cloud account. - * + * * @return the accountId value. */ public String accountId() { @@ -52,9 +54,9 @@ public String accountId() { } /** - * Get the awsAssumeRoleArn property: Assumed role ID is an identifier that you can use to create temporary security - * credentials. - * + * Get the awsAssumeRoleArn property: Assumed role ID is an identifier that you can use to create temporary + * security credentials. + * * @return the awsAssumeRoleArn value. */ public String awsAssumeRoleArn() { @@ -62,9 +64,9 @@ public String awsAssumeRoleArn() { } /** - * Set the awsAssumeRoleArn property: Assumed role ID is an identifier that you can use to create temporary security - * credentials. - * + * Set the awsAssumeRoleArn property: Assumed role ID is an identifier that you can use to create temporary + * security credentials. + * * @param awsAssumeRoleArn the awsAssumeRoleArn value to set. * @return the AwAssumeRoleAuthenticationDetailsProperties object itself. */ @@ -75,7 +77,7 @@ public AwAssumeRoleAuthenticationDetailsProperties withAwsAssumeRoleArn(String a /** * Get the awsExternalId property: A unique identifier that is required when you assume a role in another account. - * + * * @return the awsExternalId value. */ public String awsExternalId() { @@ -84,7 +86,7 @@ public String awsExternalId() { /** * Set the awsExternalId property: A unique identifier that is required when you assume a role in another account. - * + * * @param awsExternalId the awsExternalId value to set. * @return the AwAssumeRoleAuthenticationDetailsProperties object itself. */ @@ -95,25 +97,19 @@ public AwAssumeRoleAuthenticationDetailsProperties withAwsExternalId(String awsE /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (awsAssumeRoleArn() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property awsAssumeRoleArn in model" - + " AwAssumeRoleAuthenticationDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property awsAssumeRoleArn in model AwAssumeRoleAuthenticationDetailsProperties")); } if (awsExternalId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property awsExternalId in model" - + " AwAssumeRoleAuthenticationDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property awsExternalId in model AwAssumeRoleAuthenticationDetailsProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsCredsAuthenticationDetailsProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsCredsAuthenticationDetailsProperties.java index 5cff776d8f7a0..d6fed89161aac 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsCredsAuthenticationDetailsProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsCredsAuthenticationDetailsProperties.java @@ -37,13 +37,15 @@ public final class AwsCredsAuthenticationDetailsProperties extends Authenticatio @JsonProperty(value = "awsSecretAccessKey", required = true) private String awsSecretAccessKey; - /** Creates an instance of AwsCredsAuthenticationDetailsProperties class. */ + /** + * Creates an instance of AwsCredsAuthenticationDetailsProperties class. + */ public AwsCredsAuthenticationDetailsProperties() { } /** * Get the accountId property: The ID of the cloud account. - * + * * @return the accountId value. */ public String accountId() { @@ -52,7 +54,7 @@ public String accountId() { /** * Get the awsAccessKeyId property: Public key element of the AWS credential object (write only). - * + * * @return the awsAccessKeyId value. */ public String awsAccessKeyId() { @@ -61,7 +63,7 @@ public String awsAccessKeyId() { /** * Set the awsAccessKeyId property: Public key element of the AWS credential object (write only). - * + * * @param awsAccessKeyId the awsAccessKeyId value to set. * @return the AwsCredsAuthenticationDetailsProperties object itself. */ @@ -72,7 +74,7 @@ public AwsCredsAuthenticationDetailsProperties withAwsAccessKeyId(String awsAcce /** * Get the awsSecretAccessKey property: Secret key element of the AWS credential object (write only). - * + * * @return the awsSecretAccessKey value. */ public String awsSecretAccessKey() { @@ -81,7 +83,7 @@ public String awsSecretAccessKey() { /** * Set the awsSecretAccessKey property: Secret key element of the AWS credential object (write only). - * + * * @param awsSecretAccessKey the awsSecretAccessKey value to set. * @return the AwsCredsAuthenticationDetailsProperties object itself. */ @@ -92,24 +94,19 @@ public AwsCredsAuthenticationDetailsProperties withAwsSecretAccessKey(String aws /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (awsAccessKeyId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property awsAccessKeyId in model AwsCredsAuthenticationDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property awsAccessKeyId in model AwsCredsAuthenticationDetailsProperties")); } if (awsSecretAccessKey() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property awsSecretAccessKey in model" - + " AwsCredsAuthenticationDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property awsSecretAccessKey in model AwsCredsAuthenticationDetailsProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsEnvironmentData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsEnvironmentData.java index bfcd9c358cfd0..4ede05827e7fa 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsEnvironmentData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsEnvironmentData.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** The AWS connector environment data. */ +/** + * The AWS connector environment data. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "environmentType") @JsonTypeName("AwsAccount") @Fluent @@ -33,13 +35,21 @@ public final class AwsEnvironmentData extends EnvironmentData { @JsonProperty(value = "accountName", access = JsonProperty.Access.WRITE_ONLY) private String accountName; - /** Creates an instance of AwsEnvironmentData class. */ + /* + * Scan interval in hours (value should be between 1-hour to 24-hours) + */ + @JsonProperty(value = "scanInterval") + private Long scanInterval; + + /** + * Creates an instance of AwsEnvironmentData class. + */ public AwsEnvironmentData() { } /** * Get the organizationalData property: The AWS account's organizational data. - * + * * @return the organizationalData value. */ public AwsOrganizationalData organizationalData() { @@ -48,7 +58,7 @@ public AwsOrganizationalData organizationalData() { /** * Set the organizationalData property: The AWS account's organizational data. - * + * * @param organizationalData the organizationalData value to set. * @return the AwsEnvironmentData object itself. */ @@ -59,7 +69,7 @@ public AwsEnvironmentData withOrganizationalData(AwsOrganizationalData organizat /** * Get the regions property: list of regions to scan. - * + * * @return the regions value. */ public List regions() { @@ -68,7 +78,7 @@ public List regions() { /** * Set the regions property: list of regions to scan. - * + * * @param regions the regions value to set. * @return the AwsEnvironmentData object itself. */ @@ -79,16 +89,36 @@ public AwsEnvironmentData withRegions(List regions) { /** * Get the accountName property: The AWS account name. - * + * * @return the accountName value. */ public String accountName() { return this.accountName; } + /** + * Get the scanInterval property: Scan interval in hours (value should be between 1-hour to 24-hours). + * + * @return the scanInterval value. + */ + public Long scanInterval() { + return this.scanInterval; + } + + /** + * Set the scanInterval property: Scan interval in hours (value should be between 1-hour to 24-hours). + * + * @param scanInterval the scanInterval value to set. + * @return the AwsEnvironmentData object itself. + */ + public AwsEnvironmentData withScanInterval(Long scanInterval) { + this.scanInterval = scanInterval; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalData.java index 01da3de98d1fd..666b6aaaa67a0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalData.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The AWS organization data. */ +/** + * The AWS organization data. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -18,17 +20,18 @@ @JsonTypeName("AwsOrganizationalData") @JsonSubTypes({ @JsonSubTypes.Type(name = "Organization", value = AwsOrganizationalDataMaster.class), - @JsonSubTypes.Type(name = "Member", value = AwsOrganizationalDataMember.class) -}) + @JsonSubTypes.Type(name = "Member", value = AwsOrganizationalDataMember.class) }) @Immutable public class AwsOrganizationalData { - /** Creates an instance of AwsOrganizationalData class. */ + /** + * Creates an instance of AwsOrganizationalData class. + */ public AwsOrganizationalData() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMaster.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMaster.java index b7b8ffb22a08d..1b9e4b252ec0f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMaster.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMaster.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** The AWS organization data for the master account. */ +/** + * The AWS organization data for the master account. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "organizationMembershipType") @JsonTypeName("Organization") @Fluent @@ -27,14 +29,16 @@ public final class AwsOrganizationalDataMaster extends AwsOrganizationalData { @JsonProperty(value = "excludedAccountIds") private List excludedAccountIds; - /** Creates an instance of AwsOrganizationalDataMaster class. */ + /** + * Creates an instance of AwsOrganizationalDataMaster class. + */ public AwsOrganizationalDataMaster() { } /** * Get the stacksetName property: If the multi cloud account is of membership type organization, this will be the * name of the onboarding stackset. - * + * * @return the stacksetName value. */ public String stacksetName() { @@ -44,7 +48,7 @@ public String stacksetName() { /** * Set the stacksetName property: If the multi cloud account is of membership type organization, this will be the * name of the onboarding stackset. - * + * * @param stacksetName the stacksetName value to set. * @return the AwsOrganizationalDataMaster object itself. */ @@ -56,7 +60,7 @@ public AwsOrganizationalDataMaster withStacksetName(String stacksetName) { /** * Get the excludedAccountIds property: If the multi cloud account is of membership type organization, list of * accounts excluded from offering. - * + * * @return the excludedAccountIds value. */ public List excludedAccountIds() { @@ -66,7 +70,7 @@ public List excludedAccountIds() { /** * Set the excludedAccountIds property: If the multi cloud account is of membership type organization, list of * accounts excluded from offering. - * + * * @param excludedAccountIds the excludedAccountIds value to set. * @return the AwsOrganizationalDataMaster object itself. */ @@ -77,7 +81,7 @@ public AwsOrganizationalDataMaster withExcludedAccountIds(List excludedA /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMember.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMember.java index 6951419d2b761..6ebbc86b97402 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMember.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AwsOrganizationalDataMember.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The AWS organization data for the member account. */ +/** + * The AWS organization data for the member account. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "organizationMembershipType") @JsonTypeName("Member") @Fluent @@ -20,14 +22,16 @@ public final class AwsOrganizationalDataMember extends AwsOrganizationalData { @JsonProperty(value = "parentHierarchyId") private String parentHierarchyId; - /** Creates an instance of AwsOrganizationalDataMember class. */ + /** + * Creates an instance of AwsOrganizationalDataMember class. + */ public AwsOrganizationalDataMember() { } /** * Get the parentHierarchyId property: If the multi cloud account is not of membership type organization, this will * be the ID of the account's parent. - * + * * @return the parentHierarchyId value. */ public String parentHierarchyId() { @@ -37,7 +41,7 @@ public String parentHierarchyId() { /** * Set the parentHierarchyId property: If the multi cloud account is not of membership type organization, this will * be the ID of the account's parent. - * + * * @param parentHierarchyId the parentHierarchyId value to set. * @return the AwsOrganizationalDataMember object itself. */ @@ -48,7 +52,7 @@ public AwsOrganizationalDataMember withParentHierarchyId(String parentHierarchyI /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrg.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrg.java new file mode 100644 index 0000000000000..efd6f400d3caf --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrg.java @@ -0,0 +1,189 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner; + +/** + * An immutable client-side representation of AzureDevOpsOrg. + */ +public interface AzureDevOpsOrg { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the properties property: Azure DevOps Organization properties. + * + * @return the properties value. + */ + AzureDevOpsOrgProperties properties(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgInner object. + * + * @return the inner object. + */ + AzureDevOpsOrgInner innerModel(); + + /** + * The entirety of the AzureDevOpsOrg definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The AzureDevOpsOrg definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the AzureDevOpsOrg definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the AzureDevOpsOrg definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, securityConnectorName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @return the next definition stage. + */ + WithCreate withExistingSecurityConnector(String resourceGroupName, String securityConnectorName); + } + + /** + * The stage of the AzureDevOpsOrg definition which contains all the minimum required properties for the + * resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithProperties { + /** + * Executes the create request. + * + * @return the created resource. + */ + AzureDevOpsOrg create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + AzureDevOpsOrg create(Context context); + } + + /** + * The stage of the AzureDevOpsOrg definition allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Azure DevOps Organization properties.. + * + * @param properties Azure DevOps Organization properties. + * @return the next definition stage. + */ + WithCreate withProperties(AzureDevOpsOrgProperties properties); + } + } + + /** + * Begins update for the AzureDevOpsOrg resource. + * + * @return the stage of resource update. + */ + AzureDevOpsOrg.Update update(); + + /** + * The template for AzureDevOpsOrg update. + */ + interface Update extends UpdateStages.WithProperties { + /** + * Executes the update request. + * + * @return the updated resource. + */ + AzureDevOpsOrg apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + AzureDevOpsOrg apply(Context context); + } + + /** + * The AzureDevOpsOrg update stages. + */ + interface UpdateStages { + /** + * The stage of the AzureDevOpsOrg update allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Azure DevOps Organization properties.. + * + * @param properties Azure DevOps Organization properties. + * @return the next definition stage. + */ + Update withProperties(AzureDevOpsOrgProperties properties); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + AzureDevOpsOrg refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + AzureDevOpsOrg refresh(Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgListResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgListResponse.java new file mode 100644 index 0000000000000..4c7dd6da52dab --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgListResponse.java @@ -0,0 +1,34 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgListResponseInner; +import java.util.List; + +/** + * An immutable client-side representation of AzureDevOpsOrgListResponse. + */ +public interface AzureDevOpsOrgListResponse { + /** + * Gets the value property: Gets or sets list of resources. + * + * @return the value value. + */ + List value(); + + /** + * Gets the nextLink property: Gets or sets next link to scroll over the results. + * + * @return the nextLink value. + */ + String nextLink(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.AzureDevOpsOrgListResponseInner object. + * + * @return the inner object. + */ + AzureDevOpsOrgListResponseInner innerModel(); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgProperties.java new file mode 100644 index 0000000000000..6eb6fdc84e256 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgProperties.java @@ -0,0 +1,182 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * Azure DevOps Organization properties. + */ +@Fluent +public final class AzureDevOpsOrgProperties { + /* + * Gets or sets resource status message. + */ + @JsonProperty(value = "provisioningStatusMessage", access = JsonProperty.Access.WRITE_ONLY) + private String provisioningStatusMessage; + + /* + * Gets or sets time when resource was last checked. + */ + @JsonProperty(value = "provisioningStatusUpdateTimeUtc", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime provisioningStatusUpdateTimeUtc; + + /* + * The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + */ + @JsonProperty(value = "provisioningState") + private DevOpsProvisioningState provisioningState; + + /* + * Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + */ + @JsonProperty(value = "onboardingState") + private OnboardingState onboardingState; + + /* + * Configuration payload for PR Annotations. + */ + @JsonProperty(value = "actionableRemediation") + private ActionableRemediation actionableRemediation; + + /** + * Creates an instance of AzureDevOpsOrgProperties class. + */ + public AzureDevOpsOrgProperties() { + } + + /** + * Get the provisioningStatusMessage property: Gets or sets resource status message. + * + * @return the provisioningStatusMessage value. + */ + public String provisioningStatusMessage() { + return this.provisioningStatusMessage; + } + + /** + * Get the provisioningStatusUpdateTimeUtc property: Gets or sets time when resource was last checked. + * + * @return the provisioningStatusUpdateTimeUtc value. + */ + public OffsetDateTime provisioningStatusUpdateTimeUtc() { + return this.provisioningStatusUpdateTimeUtc; + } + + /** + * Get the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @return the provisioningState value. + */ + public DevOpsProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Set the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @param provisioningState the provisioningState value to set. + * @return the AzureDevOpsOrgProperties object itself. + */ + public AzureDevOpsOrgProperties withProvisioningState(DevOpsProvisioningState provisioningState) { + this.provisioningState = provisioningState; + return this; + } + + /** + * Get the onboardingState property: Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + * + * @return the onboardingState value. + */ + public OnboardingState onboardingState() { + return this.onboardingState; + } + + /** + * Set the onboardingState property: Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + * + * @param onboardingState the onboardingState value to set. + * @return the AzureDevOpsOrgProperties object itself. + */ + public AzureDevOpsOrgProperties withOnboardingState(OnboardingState onboardingState) { + this.onboardingState = onboardingState; + return this; + } + + /** + * Get the actionableRemediation property: Configuration payload for PR Annotations. + * + * @return the actionableRemediation value. + */ + public ActionableRemediation actionableRemediation() { + return this.actionableRemediation; + } + + /** + * Set the actionableRemediation property: Configuration payload for PR Annotations. + * + * @param actionableRemediation the actionableRemediation value to set. + * @return the AzureDevOpsOrgProperties object itself. + */ + public AzureDevOpsOrgProperties withActionableRemediation(ActionableRemediation actionableRemediation) { + this.actionableRemediation = actionableRemediation; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (actionableRemediation() != null) { + actionableRemediation().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgs.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgs.java new file mode 100644 index 0000000000000..8da159b8e50e3 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsOrgs.java @@ -0,0 +1,124 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of AzureDevOpsOrgs. + */ +public interface AzureDevOpsOrgs { + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination along with {@link Response}. + */ + Response listAvailableWithResponse(String resourceGroupName, + String securityConnectorName, Context context); + + /** + * Returns a list of all Azure DevOps organizations accessible by the user token consumed by the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination. + */ + AzureDevOpsOrgListResponse listAvailable(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName); + + /** + * Returns a list of Azure DevOps organizations onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName, Context context); + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String securityConnectorName, String orgName, + Context context); + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource. + */ + AzureDevOpsOrg get(String resourceGroupName, String securityConnectorName, String orgName); + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response}. + */ + AzureDevOpsOrg getById(String id); + + /** + * Returns a monitored Azure DevOps organization resource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Organization resource along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new AzureDevOpsOrg resource. + * + * @param name resource name. + * @return the first stage of the new AzureDevOpsOrg definition. + */ + AzureDevOpsOrg.DefinitionStages.Blank define(String name); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProject.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProject.java new file mode 100644 index 0000000000000..c8b69c7bd2278 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProject.java @@ -0,0 +1,191 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner; + +/** + * An immutable client-side representation of AzureDevOpsProject. + */ +public interface AzureDevOpsProject { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the properties property: Azure DevOps Project properties. + * + * @return the properties value. + */ + AzureDevOpsProjectProperties properties(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner object. + * + * @return the inner object. + */ + AzureDevOpsProjectInner innerModel(); + + /** + * The entirety of the AzureDevOpsProject definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The AzureDevOpsProject definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the AzureDevOpsProject definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the AzureDevOpsProject definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, securityConnectorName, orgName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @return the next definition stage. + */ + WithCreate withExistingAzureDevOpsOrg(String resourceGroupName, String securityConnectorName, + String orgName); + } + + /** + * The stage of the AzureDevOpsProject definition which contains all the minimum required properties for the + * resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithProperties { + /** + * Executes the create request. + * + * @return the created resource. + */ + AzureDevOpsProject create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + AzureDevOpsProject create(Context context); + } + + /** + * The stage of the AzureDevOpsProject definition allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Azure DevOps Project properties.. + * + * @param properties Azure DevOps Project properties. + * @return the next definition stage. + */ + WithCreate withProperties(AzureDevOpsProjectProperties properties); + } + } + + /** + * Begins update for the AzureDevOpsProject resource. + * + * @return the stage of resource update. + */ + AzureDevOpsProject.Update update(); + + /** + * The template for AzureDevOpsProject update. + */ + interface Update extends UpdateStages.WithProperties { + /** + * Executes the update request. + * + * @return the updated resource. + */ + AzureDevOpsProject apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + AzureDevOpsProject apply(Context context); + } + + /** + * The AzureDevOpsProject update stages. + */ + interface UpdateStages { + /** + * The stage of the AzureDevOpsProject update allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Azure DevOps Project properties.. + * + * @param properties Azure DevOps Project properties. + * @return the next definition stage. + */ + Update withProperties(AzureDevOpsProjectProperties properties); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + AzureDevOpsProject refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + AzureDevOpsProject refresh(Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectListResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectListResponse.java new file mode 100644 index 0000000000000..3918608a6f0c2 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectListResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsProjectInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of RP resources which supports pagination. + */ +@Fluent +public final class AzureDevOpsProjectListResponse { + /* + * Gets or sets list of resources. + */ + @JsonProperty(value = "value") + private List value; + + /* + * Gets or sets next link to scroll over the results. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of AzureDevOpsProjectListResponse class. + */ + public AzureDevOpsProjectListResponse() { + } + + /** + * Get the value property: Gets or sets list of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Gets or sets list of resources. + * + * @param value the value value to set. + * @return the AzureDevOpsProjectListResponse object itself. + */ + public AzureDevOpsProjectListResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Gets or sets next link to scroll over the results. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: Gets or sets next link to scroll over the results. + * + * @param nextLink the nextLink value to set. + * @return the AzureDevOpsProjectListResponse object itself. + */ + public AzureDevOpsProjectListResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectProperties.java new file mode 100644 index 0000000000000..0e9cb53d01c21 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjectProperties.java @@ -0,0 +1,223 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * Azure DevOps Project properties. + */ +@Fluent +public final class AzureDevOpsProjectProperties { + /* + * Gets or sets resource status message. + */ + @JsonProperty(value = "provisioningStatusMessage", access = JsonProperty.Access.WRITE_ONLY) + private String provisioningStatusMessage; + + /* + * Gets or sets time when resource was last checked. + */ + @JsonProperty(value = "provisioningStatusUpdateTimeUtc", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime provisioningStatusUpdateTimeUtc; + + /* + * The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + */ + @JsonProperty(value = "provisioningState") + private DevOpsProvisioningState provisioningState; + + /* + * Gets or sets parent Azure DevOps Organization name. + */ + @JsonProperty(value = "parentOrgName") + private String parentOrgName; + + /* + * Gets or sets Azure DevOps Project id. + */ + @JsonProperty(value = "projectId", access = JsonProperty.Access.WRITE_ONLY) + private String projectId; + + /* + * Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + */ + @JsonProperty(value = "onboardingState") + private OnboardingState onboardingState; + + /* + * Configuration payload for PR Annotations. + */ + @JsonProperty(value = "actionableRemediation") + private ActionableRemediation actionableRemediation; + + /** + * Creates an instance of AzureDevOpsProjectProperties class. + */ + public AzureDevOpsProjectProperties() { + } + + /** + * Get the provisioningStatusMessage property: Gets or sets resource status message. + * + * @return the provisioningStatusMessage value. + */ + public String provisioningStatusMessage() { + return this.provisioningStatusMessage; + } + + /** + * Get the provisioningStatusUpdateTimeUtc property: Gets or sets time when resource was last checked. + * + * @return the provisioningStatusUpdateTimeUtc value. + */ + public OffsetDateTime provisioningStatusUpdateTimeUtc() { + return this.provisioningStatusUpdateTimeUtc; + } + + /** + * Get the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @return the provisioningState value. + */ + public DevOpsProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Set the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @param provisioningState the provisioningState value to set. + * @return the AzureDevOpsProjectProperties object itself. + */ + public AzureDevOpsProjectProperties withProvisioningState(DevOpsProvisioningState provisioningState) { + this.provisioningState = provisioningState; + return this; + } + + /** + * Get the parentOrgName property: Gets or sets parent Azure DevOps Organization name. + * + * @return the parentOrgName value. + */ + public String parentOrgName() { + return this.parentOrgName; + } + + /** + * Set the parentOrgName property: Gets or sets parent Azure DevOps Organization name. + * + * @param parentOrgName the parentOrgName value to set. + * @return the AzureDevOpsProjectProperties object itself. + */ + public AzureDevOpsProjectProperties withParentOrgName(String parentOrgName) { + this.parentOrgName = parentOrgName; + return this; + } + + /** + * Get the projectId property: Gets or sets Azure DevOps Project id. + * + * @return the projectId value. + */ + public String projectId() { + return this.projectId; + } + + /** + * Get the onboardingState property: Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + * + * @return the onboardingState value. + */ + public OnboardingState onboardingState() { + return this.onboardingState; + } + + /** + * Set the onboardingState property: Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + * + * @param onboardingState the onboardingState value to set. + * @return the AzureDevOpsProjectProperties object itself. + */ + public AzureDevOpsProjectProperties withOnboardingState(OnboardingState onboardingState) { + this.onboardingState = onboardingState; + return this; + } + + /** + * Get the actionableRemediation property: Configuration payload for PR Annotations. + * + * @return the actionableRemediation value. + */ + public ActionableRemediation actionableRemediation() { + return this.actionableRemediation; + } + + /** + * Set the actionableRemediation property: Configuration payload for PR Annotations. + * + * @param actionableRemediation the actionableRemediation value to set. + * @return the AzureDevOpsProjectProperties object itself. + */ + public AzureDevOpsProjectProperties withActionableRemediation(ActionableRemediation actionableRemediation) { + this.actionableRemediation = actionableRemediation; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (actionableRemediation() != null) { + actionableRemediation().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjects.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjects.java new file mode 100644 index 0000000000000..2b5f0629348a4 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsProjects.java @@ -0,0 +1,103 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of AzureDevOpsProjects. + */ +public interface AzureDevOpsProjects { + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName, String orgName); + + /** + * Returns a list of Azure DevOps projects onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName, String orgName, + Context context); + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, Context context); + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource. + */ + AzureDevOpsProject get(String resourceGroupName, String securityConnectorName, String orgName, String projectName); + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response}. + */ + AzureDevOpsProject getById(String id); + + /** + * Returns a monitored Azure DevOps project resource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Project resource along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new AzureDevOpsProject resource. + * + * @param name resource name. + * @return the first stage of the new AzureDevOpsProject definition. + */ + AzureDevOpsProject.DefinitionStages.Blank define(String name); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepos.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepos.java new file mode 100644 index 0000000000000..65f560c6266d6 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepos.java @@ -0,0 +1,109 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of AzureDevOpsRepos. + */ +public interface AzureDevOpsRepos { + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName, String orgName, + String projectName); + + /** + * Returns a list of Azure DevOps repositories onboarded to the connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, Context context); + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String orgName, String projectName, String repoName, Context context); + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @param repoName The repository name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource. + */ + AzureDevOpsRepository get(String resourceGroupName, String securityConnectorName, String orgName, + String projectName, String repoName); + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response}. + */ + AzureDevOpsRepository getById(String id); + + /** + * Returns a monitored Azure DevOps repository resource. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return azure DevOps Repository resource along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new AzureDevOpsRepository resource. + * + * @param name resource name. + * @return the first stage of the new AzureDevOpsRepository definition. + */ + AzureDevOpsRepository.DefinitionStages.Blank define(String name); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepository.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepository.java new file mode 100644 index 0000000000000..7c3114d7c2724 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepository.java @@ -0,0 +1,192 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.management.SystemData; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner; + +/** + * An immutable client-side representation of AzureDevOpsRepository. + */ +public interface AzureDevOpsRepository { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the properties property: Azure DevOps Repository properties. + * + * @return the properties value. + */ + AzureDevOpsRepositoryProperties properties(); + + /** + * Gets the name of the resource group. + * + * @return the name of the resource group. + */ + String resourceGroupName(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner object. + * + * @return the inner object. + */ + AzureDevOpsRepositoryInner innerModel(); + + /** + * The entirety of the AzureDevOpsRepository definition. + */ + interface Definition + extends DefinitionStages.Blank, DefinitionStages.WithParentResource, DefinitionStages.WithCreate { + } + + /** + * The AzureDevOpsRepository definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the AzureDevOpsRepository definition. + */ + interface Blank extends WithParentResource { + } + + /** + * The stage of the AzureDevOpsRepository definition allowing to specify parent resource. + */ + interface WithParentResource { + /** + * Specifies resourceGroupName, securityConnectorName, orgName, projectName. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param orgName The Azure DevOps organization name. + * @param projectName The project name. + * @return the next definition stage. + */ + WithCreate withExistingProject(String resourceGroupName, String securityConnectorName, String orgName, + String projectName); + } + + /** + * The stage of the AzureDevOpsRepository definition which contains all the minimum required properties for the + * resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate extends DefinitionStages.WithProperties { + /** + * Executes the create request. + * + * @return the created resource. + */ + AzureDevOpsRepository create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + AzureDevOpsRepository create(Context context); + } + + /** + * The stage of the AzureDevOpsRepository definition allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Azure DevOps Repository properties.. + * + * @param properties Azure DevOps Repository properties. + * @return the next definition stage. + */ + WithCreate withProperties(AzureDevOpsRepositoryProperties properties); + } + } + + /** + * Begins update for the AzureDevOpsRepository resource. + * + * @return the stage of resource update. + */ + AzureDevOpsRepository.Update update(); + + /** + * The template for AzureDevOpsRepository update. + */ + interface Update extends UpdateStages.WithProperties { + /** + * Executes the update request. + * + * @return the updated resource. + */ + AzureDevOpsRepository apply(); + + /** + * Executes the update request. + * + * @param context The context to associate with this operation. + * @return the updated resource. + */ + AzureDevOpsRepository apply(Context context); + } + + /** + * The AzureDevOpsRepository update stages. + */ + interface UpdateStages { + /** + * The stage of the AzureDevOpsRepository update allowing to specify properties. + */ + interface WithProperties { + /** + * Specifies the properties property: Azure DevOps Repository properties.. + * + * @param properties Azure DevOps Repository properties. + * @return the next definition stage. + */ + Update withProperties(AzureDevOpsRepositoryProperties properties); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + AzureDevOpsRepository refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + AzureDevOpsRepository refresh(Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryListResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryListResponse.java new file mode 100644 index 0000000000000..96282da28947b --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryListResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.fluent.models.AzureDevOpsRepositoryInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of RP resources which supports pagination. + */ +@Fluent +public final class AzureDevOpsRepositoryListResponse { + /* + * Gets or sets list of resources. + */ + @JsonProperty(value = "value") + private List value; + + /* + * Gets or sets next link to scroll over the results. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of AzureDevOpsRepositoryListResponse class. + */ + public AzureDevOpsRepositoryListResponse() { + } + + /** + * Get the value property: Gets or sets list of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Gets or sets list of resources. + * + * @param value the value value to set. + * @return the AzureDevOpsRepositoryListResponse object itself. + */ + public AzureDevOpsRepositoryListResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Gets or sets next link to scroll over the results. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: Gets or sets next link to scroll over the results. + * + * @param nextLink the nextLink value to set. + * @return the AzureDevOpsRepositoryListResponse object itself. + */ + public AzureDevOpsRepositoryListResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryProperties.java new file mode 100644 index 0000000000000..cdb7166152dfb --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsRepositoryProperties.java @@ -0,0 +1,280 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; + +/** + * Azure DevOps Repository properties. + */ +@Fluent +public final class AzureDevOpsRepositoryProperties { + /* + * Gets or sets resource status message. + */ + @JsonProperty(value = "provisioningStatusMessage", access = JsonProperty.Access.WRITE_ONLY) + private String provisioningStatusMessage; + + /* + * Gets or sets time when resource was last checked. + */ + @JsonProperty(value = "provisioningStatusUpdateTimeUtc", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime provisioningStatusUpdateTimeUtc; + + /* + * The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + */ + @JsonProperty(value = "provisioningState") + private DevOpsProvisioningState provisioningState; + + /* + * Gets or sets parent Azure DevOps Organization name. + */ + @JsonProperty(value = "parentOrgName") + private String parentOrgName; + + /* + * Gets or sets parent Azure DevOps Project name. + */ + @JsonProperty(value = "parentProjectName") + private String parentProjectName; + + /* + * Gets or sets Azure DevOps Repository id. + */ + @JsonProperty(value = "repoId", access = JsonProperty.Access.WRITE_ONLY) + private String repoId; + + /* + * Gets or sets Azure DevOps Repository url. + */ + @JsonProperty(value = "repoUrl", access = JsonProperty.Access.WRITE_ONLY) + private String repoUrl; + + /* + * Gets or sets Azure DevOps repository visibility, whether it is public or private etc. + */ + @JsonProperty(value = "visibility", access = JsonProperty.Access.WRITE_ONLY) + private String visibility; + + /* + * Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + */ + @JsonProperty(value = "onboardingState") + private OnboardingState onboardingState; + + /* + * Configuration payload for PR Annotations. + */ + @JsonProperty(value = "actionableRemediation") + private ActionableRemediation actionableRemediation; + + /** + * Creates an instance of AzureDevOpsRepositoryProperties class. + */ + public AzureDevOpsRepositoryProperties() { + } + + /** + * Get the provisioningStatusMessage property: Gets or sets resource status message. + * + * @return the provisioningStatusMessage value. + */ + public String provisioningStatusMessage() { + return this.provisioningStatusMessage; + } + + /** + * Get the provisioningStatusUpdateTimeUtc property: Gets or sets time when resource was last checked. + * + * @return the provisioningStatusUpdateTimeUtc value. + */ + public OffsetDateTime provisioningStatusUpdateTimeUtc() { + return this.provisioningStatusUpdateTimeUtc; + } + + /** + * Get the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @return the provisioningState value. + */ + public DevOpsProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Set the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @param provisioningState the provisioningState value to set. + * @return the AzureDevOpsRepositoryProperties object itself. + */ + public AzureDevOpsRepositoryProperties withProvisioningState(DevOpsProvisioningState provisioningState) { + this.provisioningState = provisioningState; + return this; + } + + /** + * Get the parentOrgName property: Gets or sets parent Azure DevOps Organization name. + * + * @return the parentOrgName value. + */ + public String parentOrgName() { + return this.parentOrgName; + } + + /** + * Set the parentOrgName property: Gets or sets parent Azure DevOps Organization name. + * + * @param parentOrgName the parentOrgName value to set. + * @return the AzureDevOpsRepositoryProperties object itself. + */ + public AzureDevOpsRepositoryProperties withParentOrgName(String parentOrgName) { + this.parentOrgName = parentOrgName; + return this; + } + + /** + * Get the parentProjectName property: Gets or sets parent Azure DevOps Project name. + * + * @return the parentProjectName value. + */ + public String parentProjectName() { + return this.parentProjectName; + } + + /** + * Set the parentProjectName property: Gets or sets parent Azure DevOps Project name. + * + * @param parentProjectName the parentProjectName value to set. + * @return the AzureDevOpsRepositoryProperties object itself. + */ + public AzureDevOpsRepositoryProperties withParentProjectName(String parentProjectName) { + this.parentProjectName = parentProjectName; + return this; + } + + /** + * Get the repoId property: Gets or sets Azure DevOps Repository id. + * + * @return the repoId value. + */ + public String repoId() { + return this.repoId; + } + + /** + * Get the repoUrl property: Gets or sets Azure DevOps Repository url. + * + * @return the repoUrl value. + */ + public String repoUrl() { + return this.repoUrl; + } + + /** + * Get the visibility property: Gets or sets Azure DevOps repository visibility, whether it is public or private + * etc. + * + * @return the visibility value. + */ + public String visibility() { + return this.visibility; + } + + /** + * Get the onboardingState property: Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + * + * @return the onboardingState value. + */ + public OnboardingState onboardingState() { + return this.onboardingState; + } + + /** + * Set the onboardingState property: Details about resource onboarding status across all connectors. + * + * OnboardedByOtherConnector - this resource has already been onboarded to another connector. This is only + * applicable to top-level resources. + * Onboarded - this resource has already been onboarded by the specified connector. + * NotOnboarded - this resource has not been onboarded to any connector. + * NotApplicable - the onboarding state is not applicable to the current endpoint. + * + * @param onboardingState the onboardingState value to set. + * @return the AzureDevOpsRepositoryProperties object itself. + */ + public AzureDevOpsRepositoryProperties withOnboardingState(OnboardingState onboardingState) { + this.onboardingState = onboardingState; + return this; + } + + /** + * Get the actionableRemediation property: Configuration payload for PR Annotations. + * + * @return the actionableRemediation value. + */ + public ActionableRemediation actionableRemediation() { + return this.actionableRemediation; + } + + /** + * Set the actionableRemediation property: Configuration payload for PR Annotations. + * + * @param actionableRemediation the actionableRemediation value to set. + * @return the AzureDevOpsRepositoryProperties object itself. + */ + public AzureDevOpsRepositoryProperties withActionableRemediation(ActionableRemediation actionableRemediation) { + this.actionableRemediation = actionableRemediation; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (actionableRemediation() != null) { + actionableRemediation().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsScopeEnvironmentData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsScopeEnvironmentData.java index 1f9e0d85d8461..2f07e93a253b9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsScopeEnvironmentData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureDevOpsScopeEnvironmentData.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The AzureDevOps scope connector's environment data. */ +/** + * The AzureDevOps scope connector's environment data. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "environmentType") @JsonTypeName("AzureDevOpsScope") @Immutable public final class AzureDevOpsScopeEnvironmentData extends EnvironmentData { - /** Creates an instance of AzureDevOpsScopeEnvironmentData class. */ + /** + * Creates an instance of AzureDevOpsScopeEnvironmentData class. + */ public AzureDevOpsScopeEnvironmentData() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceDetails.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceDetails.java index 93ae42309e0ef..dde973b91ebdd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceDetails.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceDetails.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Details of the Azure resource that was assessed. */ +/** + * Details of the Azure resource that was assessed. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "source") @JsonTypeName("Azure") @Immutable @@ -20,13 +22,15 @@ public final class AzureResourceDetails extends ResourceDetails { @JsonProperty(value = "id", access = JsonProperty.Access.WRITE_ONLY) private String id; - /** Creates an instance of AzureResourceDetails class. */ + /** + * Creates an instance of AzureResourceDetails class. + */ public AzureResourceDetails() { } /** * Get the id property: Azure resource Id of the assessed resource. - * + * * @return the id value. */ public String id() { @@ -35,7 +39,7 @@ public String id() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceIdentifier.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceIdentifier.java index 71269931e0706..52c4ac300968c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceIdentifier.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceIdentifier.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Azure resource identifier. */ +/** + * Azure resource identifier. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "type") @JsonTypeName("AzureResource") @Immutable @@ -20,13 +22,15 @@ public final class AzureResourceIdentifier extends ResourceIdentifier { @JsonProperty(value = "azureResourceId", access = JsonProperty.Access.WRITE_ONLY) private String azureResourceId; - /** Creates an instance of AzureResourceIdentifier class. */ + /** + * Creates an instance of AzureResourceIdentifier class. + */ public AzureResourceIdentifier() { } /** * Get the azureResourceId property: ARM resource identifier for the cloud resource being alerted on. - * + * * @return the azureResourceId value. */ public String azureResourceId() { @@ -35,7 +39,7 @@ public String azureResourceId() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceLink.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceLink.java index fb5c7218394af..6daa6a30dff48 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceLink.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureResourceLink.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Describes an Azure resource with kind. */ +/** + * Describes an Azure resource with kind. + */ @Immutable public final class AzureResourceLink { /* @@ -16,13 +18,15 @@ public final class AzureResourceLink { @JsonProperty(value = "id", access = JsonProperty.Access.WRITE_ONLY) private String id; - /** Creates an instance of AzureResourceLink class. */ + /** + * Creates an instance of AzureResourceLink class. + */ public AzureResourceLink() { } /** * Get the id property: Azure resource Id. - * + * * @return the id value. */ public String id() { @@ -31,7 +35,7 @@ public String id() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureServersSetting.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureServersSetting.java new file mode 100644 index 0000000000000..50d6ef79f6e22 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureServersSetting.java @@ -0,0 +1,81 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsAzureSettingProperties; +import com.azure.resourcemanager.security.fluent.models.ServerVulnerabilityAssessmentsSettingInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * A vulnerability assessments setting on Azure servers in the defined scope. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") +@JsonTypeName("AzureServersSetting") +@Fluent +public final class AzureServersSetting extends ServerVulnerabilityAssessmentsSettingInner { + /* + * The vulnerability assessments setting properties on Azure servers in the defined scope. + */ + @JsonProperty(value = "properties") + private ServerVulnerabilityAssessmentsAzureSettingProperties innerProperties; + + /** + * Creates an instance of AzureServersSetting class. + */ + public AzureServersSetting() { + } + + /** + * Get the innerProperties property: The vulnerability assessments setting properties on Azure servers in the + * defined scope. + * + * @return the innerProperties value. + */ + private ServerVulnerabilityAssessmentsAzureSettingProperties innerProperties() { + return this.innerProperties; + } + + /** + * Get the selectedProvider property: The selected vulnerability assessments provider on Azure servers in the + * defined scope. + * + * @return the selectedProvider value. + */ + public ServerVulnerabilityAssessmentsAzureSettingSelectedProvider selectedProvider() { + return this.innerProperties() == null ? null : this.innerProperties().selectedProvider(); + } + + /** + * Set the selectedProvider property: The selected vulnerability assessments provider on Azure servers in the + * defined scope. + * + * @param selectedProvider the selectedProvider value to set. + * @return the AzureServersSetting object itself. + */ + public AzureServersSetting + withSelectedProvider(ServerVulnerabilityAssessmentsAzureSettingSelectedProvider selectedProvider) { + if (this.innerProperties() == null) { + this.innerProperties = new ServerVulnerabilityAssessmentsAzureSettingProperties(); + } + this.innerProperties().withSelectedProvider(selectedProvider); + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + @Override + public void validate() { + super.validate(); + if (innerProperties() != null) { + innerProperties().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureTrackedResourceLocation.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureTrackedResourceLocation.java index c48378441c4cc..bb5adbfb4f261 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureTrackedResourceLocation.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/AzureTrackedResourceLocation.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Describes an Azure resource with location. */ +/** + * Describes an Azure resource with location. + */ @Fluent public class AzureTrackedResourceLocation { /* @@ -16,13 +18,15 @@ public class AzureTrackedResourceLocation { @JsonProperty(value = "location") private String location; - /** Creates an instance of AzureTrackedResourceLocation class. */ + /** + * Creates an instance of AzureTrackedResourceLocation class. + */ public AzureTrackedResourceLocation() { } /** * Get the location property: Location where the resource is stored. - * + * * @return the location value. */ public String location() { @@ -31,7 +35,7 @@ public String location() { /** * Set the location property: Location where the resource is stored. - * + * * @param location the location value to set. * @return the AzureTrackedResourceLocation object itself. */ @@ -42,7 +46,7 @@ public AzureTrackedResourceLocation withLocation(String location) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Baseline.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Baseline.java index 99852d05bed64..a057ceaf1e39a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Baseline.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Baseline.java @@ -9,7 +9,9 @@ import java.time.OffsetDateTime; import java.util.List; -/** Baseline details. */ +/** + * Baseline details. + */ @Fluent public final class Baseline { /* @@ -24,13 +26,15 @@ public final class Baseline { @JsonProperty(value = "updatedTime") private OffsetDateTime updatedTime; - /** Creates an instance of Baseline class. */ + /** + * Creates an instance of Baseline class. + */ public Baseline() { } /** * Get the expectedResults property: Expected results. - * + * * @return the expectedResults value. */ public List> expectedResults() { @@ -39,7 +43,7 @@ public List> expectedResults() { /** * Set the expectedResults property: Expected results. - * + * * @param expectedResults the expectedResults value to set. * @return the Baseline object itself. */ @@ -50,7 +54,7 @@ public Baseline withExpectedResults(List> expectedResults) { /** * Get the updatedTime property: Baseline update time (UTC). - * + * * @return the updatedTime value. */ public OffsetDateTime updatedTime() { @@ -59,7 +63,7 @@ public OffsetDateTime updatedTime() { /** * Set the updatedTime property: Baseline update time (UTC). - * + * * @param updatedTime the updatedTime value to set. * @return the Baseline object itself. */ @@ -70,7 +74,7 @@ public Baseline withUpdatedTime(OffsetDateTime updatedTime) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BaselineAdjustedResult.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BaselineAdjustedResult.java index dfd5b9663173c..4bf91b01f9efd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BaselineAdjustedResult.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BaselineAdjustedResult.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The rule result adjusted with baseline. */ +/** + * The rule result adjusted with baseline. + */ @Fluent public final class BaselineAdjustedResult { /* @@ -35,13 +37,15 @@ public final class BaselineAdjustedResult { @JsonProperty(value = "resultsOnlyInBaseline") private List> resultsOnlyInBaseline; - /** Creates an instance of BaselineAdjustedResult class. */ + /** + * Creates an instance of BaselineAdjustedResult class. + */ public BaselineAdjustedResult() { } /** * Get the baseline property: Baseline details. - * + * * @return the baseline value. */ public Baseline baseline() { @@ -50,7 +54,7 @@ public Baseline baseline() { /** * Set the baseline property: Baseline details. - * + * * @param baseline the baseline value to set. * @return the BaselineAdjustedResult object itself. */ @@ -61,7 +65,7 @@ public BaselineAdjustedResult withBaseline(Baseline baseline) { /** * Get the status property: The rule result status. - * + * * @return the status value. */ public RuleStatus status() { @@ -70,7 +74,7 @@ public RuleStatus status() { /** * Set the status property: The rule result status. - * + * * @param status the status value to set. * @return the BaselineAdjustedResult object itself. */ @@ -81,7 +85,7 @@ public BaselineAdjustedResult withStatus(RuleStatus status) { /** * Get the resultsNotInBaseline property: Results the are not in baseline. - * + * * @return the resultsNotInBaseline value. */ public List> resultsNotInBaseline() { @@ -90,7 +94,7 @@ public List> resultsNotInBaseline() { /** * Set the resultsNotInBaseline property: Results the are not in baseline. - * + * * @param resultsNotInBaseline the resultsNotInBaseline value to set. * @return the BaselineAdjustedResult object itself. */ @@ -101,7 +105,7 @@ public BaselineAdjustedResult withResultsNotInBaseline(List> result /** * Get the resultsOnlyInBaseline property: Results the are in baseline. - * + * * @return the resultsOnlyInBaseline value. */ public List> resultsOnlyInBaseline() { @@ -110,7 +114,7 @@ public List> resultsOnlyInBaseline() { /** * Set the resultsOnlyInBaseline property: Results the are in baseline. - * + * * @param resultsOnlyInBaseline the resultsOnlyInBaseline value to set. * @return the BaselineAdjustedResult object itself. */ @@ -121,7 +125,7 @@ public BaselineAdjustedResult withResultsOnlyInBaseline(List> resul /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BenchmarkReference.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BenchmarkReference.java index 06dc061d3069e..86edae55926c7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BenchmarkReference.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BenchmarkReference.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The benchmark references. */ +/** + * The benchmark references. + */ @Fluent public final class BenchmarkReference { /* @@ -22,13 +24,15 @@ public final class BenchmarkReference { @JsonProperty(value = "reference") private String reference; - /** Creates an instance of BenchmarkReference class. */ + /** + * Creates an instance of BenchmarkReference class. + */ public BenchmarkReference() { } /** * Get the benchmark property: The benchmark name. - * + * * @return the benchmark value. */ public String benchmark() { @@ -37,7 +41,7 @@ public String benchmark() { /** * Set the benchmark property: The benchmark name. - * + * * @param benchmark the benchmark value to set. * @return the BenchmarkReference object itself. */ @@ -48,7 +52,7 @@ public BenchmarkReference withBenchmark(String benchmark) { /** * Get the reference property: The benchmark reference. - * + * * @return the reference value. */ public String reference() { @@ -57,7 +61,7 @@ public String reference() { /** * Set the reference property: The benchmark reference. - * + * * @param reference the reference value to set. * @return the BenchmarkReference object itself. */ @@ -68,7 +72,7 @@ public BenchmarkReference withReference(String reference) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BuiltInInfoType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BuiltInInfoType.java new file mode 100644 index 0000000000000..f579d5a558ef9 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BuiltInInfoType.java @@ -0,0 +1,107 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.UUID; + +/** + * Pre-configured sensitive information type. + */ +@Fluent +public final class BuiltInInfoType { + /* + * Display name of the info type + */ + @JsonProperty(value = "name") + private String name; + + /* + * Id of the info type + */ + @JsonProperty(value = "id") + private UUID id; + + /* + * Category of the built-in info type + */ + @JsonProperty(value = "type") + private String type; + + /** + * Creates an instance of BuiltInInfoType class. + */ + public BuiltInInfoType() { + } + + /** + * Get the name property: Display name of the info type. + * + * @return the name value. + */ + public String name() { + return this.name; + } + + /** + * Set the name property: Display name of the info type. + * + * @param name the name value to set. + * @return the BuiltInInfoType object itself. + */ + public BuiltInInfoType withName(String name) { + this.name = name; + return this; + } + + /** + * Get the id property: Id of the info type. + * + * @return the id value. + */ + public UUID id() { + return this.id; + } + + /** + * Set the id property: Id of the info type. + * + * @param id the id value to set. + * @return the BuiltInInfoType object itself. + */ + public BuiltInInfoType withId(UUID id) { + this.id = id; + return this; + } + + /** + * Get the type property: Category of the built-in info type. + * + * @return the type value. + */ + public String type() { + return this.type; + } + + /** + * Set the type property: Category of the built-in info type. + * + * @param type the type value to set. + * @return the BuiltInInfoType object itself. + */ + public BuiltInInfoType withType(String type) { + this.type = type; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BundleType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BundleType.java index e16b0198c97e3..2c284f4c2bc28 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BundleType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/BundleType.java @@ -8,38 +8,58 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Alert Simulator supported bundles. */ +/** + * Alert Simulator supported bundles. + */ public final class BundleType extends ExpandableStringEnum { - /** Static value AppServices for BundleType. */ + /** + * Static value AppServices for BundleType. + */ public static final BundleType APP_SERVICES = fromString("AppServices"); - /** Static value DNS for BundleType. */ + /** + * Static value DNS for BundleType. + */ public static final BundleType DNS = fromString("DNS"); - /** Static value KeyVaults for BundleType. */ + /** + * Static value KeyVaults for BundleType. + */ public static final BundleType KEY_VAULTS = fromString("KeyVaults"); - /** Static value KubernetesService for BundleType. */ + /** + * Static value KubernetesService for BundleType. + */ public static final BundleType KUBERNETES_SERVICE = fromString("KubernetesService"); - /** Static value ResourceManager for BundleType. */ + /** + * Static value ResourceManager for BundleType. + */ public static final BundleType RESOURCE_MANAGER = fromString("ResourceManager"); - /** Static value SqlServers for BundleType. */ + /** + * Static value SqlServers for BundleType. + */ public static final BundleType SQL_SERVERS = fromString("SqlServers"); - /** Static value StorageAccounts for BundleType. */ + /** + * Static value StorageAccounts for BundleType. + */ public static final BundleType STORAGE_ACCOUNTS = fromString("StorageAccounts"); - /** Static value VirtualMachines for BundleType. */ + /** + * Static value VirtualMachines for BundleType. + */ public static final BundleType VIRTUAL_MACHINES = fromString("VirtualMachines"); - /** Static value CosmosDbs for BundleType. */ + /** + * Static value CosmosDbs for BundleType. + */ public static final BundleType COSMOS_DBS = fromString("CosmosDbs"); /** * Creates a new instance of BundleType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -48,7 +68,7 @@ public BundleType() { /** * Creates or finds a BundleType from its string representation. - * + * * @param name a name to look for. * @return the corresponding BundleType. */ @@ -59,7 +79,7 @@ public static BundleType fromString(String name) { /** * Gets known BundleType values. - * + * * @return known BundleType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Categories.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Categories.java index deadc86057452..0f3ad301c78d3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Categories.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Categories.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The categories of resource that is at risk when the assessment is unhealthy. */ +/** + * The categories of resource that is at risk when the assessment is unhealthy. + */ public final class Categories extends ExpandableStringEnum { - /** Static value Compute for Categories. */ + /** + * Static value Compute for Categories. + */ public static final Categories COMPUTE = fromString("Compute"); - /** Static value Networking for Categories. */ + /** + * Static value Networking for Categories. + */ public static final Categories NETWORKING = fromString("Networking"); - /** Static value Data for Categories. */ + /** + * Static value Data for Categories. + */ public static final Categories DATA = fromString("Data"); - /** Static value IdentityAndAccess for Categories. */ + /** + * Static value IdentityAndAccess for Categories. + */ public static final Categories IDENTITY_AND_ACCESS = fromString("IdentityAndAccess"); - /** Static value IoT for Categories. */ + /** + * Static value IoT for Categories. + */ public static final Categories IOT = fromString("IoT"); /** * Creates a new instance of Categories value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public Categories() { /** * Creates or finds a Categories from its string representation. - * + * * @param name a name to look for. * @return the corresponding Categories. */ @@ -47,7 +59,7 @@ public static Categories fromString(String name) { /** * Gets known Categories values. - * + * * @return known Categories values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CategoryConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CategoryConfiguration.java new file mode 100644 index 0000000000000..337cfc02f7ae8 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CategoryConfiguration.java @@ -0,0 +1,98 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Severity level per category configuration for PR Annotations. + */ +@Fluent +public final class CategoryConfiguration { + /* + * Gets or sets minimum severity level for a given category. + */ + @JsonProperty(value = "minimumSeverityLevel") + private String minimumSeverityLevel; + + /* + * Rule categories. + * Code - code scanning results. + * Artifact scanning results. + * Dependencies scanning results. + * IaC results. + * Secrets scanning results. + * Container scanning results. + */ + @JsonProperty(value = "category") + private RuleCategory category; + + /** + * Creates an instance of CategoryConfiguration class. + */ + public CategoryConfiguration() { + } + + /** + * Get the minimumSeverityLevel property: Gets or sets minimum severity level for a given category. + * + * @return the minimumSeverityLevel value. + */ + public String minimumSeverityLevel() { + return this.minimumSeverityLevel; + } + + /** + * Set the minimumSeverityLevel property: Gets or sets minimum severity level for a given category. + * + * @param minimumSeverityLevel the minimumSeverityLevel value to set. + * @return the CategoryConfiguration object itself. + */ + public CategoryConfiguration withMinimumSeverityLevel(String minimumSeverityLevel) { + this.minimumSeverityLevel = minimumSeverityLevel; + return this; + } + + /** + * Get the category property: Rule categories. + * Code - code scanning results. + * Artifact scanning results. + * Dependencies scanning results. + * IaC results. + * Secrets scanning results. + * Container scanning results. + * + * @return the category value. + */ + public RuleCategory category() { + return this.category; + } + + /** + * Set the category property: Rule categories. + * Code - code scanning results. + * Artifact scanning results. + * Dependencies scanning results. + * IaC results. + * Secrets scanning results. + * Container scanning results. + * + * @param category the category value to set. + * @return the CategoryConfiguration object itself. + */ + public CategoryConfiguration withCategory(RuleCategory category) { + this.category = category; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefExternalSecuritySolution.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefExternalSecuritySolution.java index 7fd9efb1e94b8..4935e5233d231 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefExternalSecuritySolution.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefExternalSecuritySolution.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Represents a security solution which sends CEF logs to an OMS workspace. */ +/** + * Represents a security solution which sends CEF logs to an OMS workspace. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") @JsonTypeName("CEF") @Fluent @@ -21,13 +23,15 @@ public final class CefExternalSecuritySolution extends ExternalSecuritySolutionI @JsonProperty(value = "properties") private CefSolutionProperties properties; - /** Creates an instance of CefExternalSecuritySolution class. */ + /** + * Creates an instance of CefExternalSecuritySolution class. + */ public CefExternalSecuritySolution() { } /** * Get the properties property: The external security solution properties for CEF solutions. - * + * * @return the properties value. */ public CefSolutionProperties properties() { @@ -36,7 +40,7 @@ public CefSolutionProperties properties() { /** * Set the properties property: The external security solution properties for CEF solutions. - * + * * @param properties the properties value to set. * @return the CefExternalSecuritySolution object itself. */ @@ -47,7 +51,7 @@ public CefExternalSecuritySolution withProperties(CefSolutionProperties properti /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefSolutionProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefSolutionProperties.java index aa7d3c3e7e1b2..bfb05b218fdf4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefSolutionProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CefSolutionProperties.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The external security solution properties for CEF solutions. */ +/** + * The external security solution properties for CEF solutions. + */ @Fluent public final class CefSolutionProperties extends ExternalSecuritySolutionProperties { /* @@ -28,13 +30,15 @@ public final class CefSolutionProperties extends ExternalSecuritySolutionPropert @JsonProperty(value = "lastEventReceived") private String lastEventReceived; - /** Creates an instance of CefSolutionProperties class. */ + /** + * Creates an instance of CefSolutionProperties class. + */ public CefSolutionProperties() { } /** * Get the hostname property: The hostname property. - * + * * @return the hostname value. */ public String hostname() { @@ -43,7 +47,7 @@ public String hostname() { /** * Set the hostname property: The hostname property. - * + * * @param hostname the hostname value to set. * @return the CefSolutionProperties object itself. */ @@ -54,7 +58,7 @@ public CefSolutionProperties withHostname(String hostname) { /** * Get the agent property: The agent property. - * + * * @return the agent value. */ public String agent() { @@ -63,7 +67,7 @@ public String agent() { /** * Set the agent property: The agent property. - * + * * @param agent the agent value to set. * @return the CefSolutionProperties object itself. */ @@ -74,7 +78,7 @@ public CefSolutionProperties withAgent(String agent) { /** * Get the lastEventReceived property: The lastEventReceived property. - * + * * @return the lastEventReceived value. */ public String lastEventReceived() { @@ -83,7 +87,7 @@ public String lastEventReceived() { /** * Set the lastEventReceived property: The lastEventReceived property. - * + * * @param lastEventReceived the lastEventReceived value to set. * @return the CefSolutionProperties object itself. */ @@ -92,21 +96,27 @@ public CefSolutionProperties withLastEventReceived(String lastEventReceived) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CefSolutionProperties withDeviceVendor(String deviceVendor) { super.withDeviceVendor(deviceVendor); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CefSolutionProperties withDeviceType(String deviceType) { super.withDeviceType(deviceType); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public CefSolutionProperties withWorkspace(ConnectedWorkspace workspace) { super.withWorkspace(workspace); @@ -115,7 +125,7 @@ public CefSolutionProperties withWorkspace(ConnectedWorkspace workspace) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudName.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudName.java index 667eb58402de6..7ea279843cc95 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudName.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudName.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The multi cloud resource's cloud name. */ +/** + * The multi cloud resource's cloud name. + */ public final class CloudName extends ExpandableStringEnum { - /** Static value Azure for CloudName. */ + /** + * Static value Azure for CloudName. + */ public static final CloudName AZURE = fromString("Azure"); - /** Static value AWS for CloudName. */ + /** + * Static value AWS for CloudName. + */ public static final CloudName AWS = fromString("AWS"); - /** Static value GCP for CloudName. */ + /** + * Static value GCP for CloudName. + */ public static final CloudName GCP = fromString("GCP"); - /** Static value Github for CloudName. */ + /** + * Static value Github for CloudName. + */ public static final CloudName GITHUB = fromString("Github"); - /** Static value AzureDevOps for CloudName. */ + /** + * Static value AzureDevOps for CloudName. + */ public static final CloudName AZURE_DEV_OPS = fromString("AzureDevOps"); - /** Static value GitLab for CloudName. */ + /** + * Static value GitLab for CloudName. + */ public static final CloudName GIT_LAB = fromString("GitLab"); /** * Creates a new instance of CloudName value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public CloudName() { /** * Creates or finds a CloudName from its string representation. - * + * * @param name a name to look for. * @return the corresponding CloudName. */ @@ -50,7 +64,7 @@ public static CloudName fromString(String name) { /** * Gets known CloudName values. - * + * * @return known CloudName values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudOffering.java index d1a7b48b8634d..c72e9254799af 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CloudOffering.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The security offering details. */ +/** + * The security offering details. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -34,8 +36,7 @@ @JsonSubTypes.Type(name = "DefenderForDevOpsGithub", value = DefenderForDevOpsGithubOffering.class), @JsonSubTypes.Type(name = "DefenderForDevOpsAzureDevOps", value = DefenderForDevOpsAzureDevOpsOffering.class), @JsonSubTypes.Type(name = "CspmMonitorGitLab", value = CspmMonitorGitLabOffering.class), - @JsonSubTypes.Type(name = "DefenderForDevOpsGitLab", value = DefenderForDevOpsGitLabOffering.class) -}) + @JsonSubTypes.Type(name = "DefenderForDevOpsGitLab", value = DefenderForDevOpsGitLabOffering.class) }) @Immutable public class CloudOffering { /* @@ -44,13 +45,15 @@ public class CloudOffering { @JsonProperty(value = "description", access = JsonProperty.Access.WRITE_ONLY) private String description; - /** Creates an instance of CloudOffering class. */ + /** + * Creates an instance of CloudOffering class. + */ public CloudOffering() { } /** * Get the description property: The offering description. - * + * * @return the description value. */ public String description() { @@ -59,7 +62,7 @@ public String description() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Code.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Code.java index 487d31012408d..03946dd3e0c9b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Code.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Code.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The operation status code. */ +/** + * The operation status code. + */ public final class Code extends ExpandableStringEnum { - /** Static value Succeeded for Code. */ + /** + * Static value Succeeded for Code. + */ public static final Code SUCCEEDED = fromString("Succeeded"); - /** Static value Failed for Code. */ + /** + * Static value Failed for Code. + */ public static final Code FAILED = fromString("Failed"); /** * Creates a new instance of Code value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public Code() { /** * Creates or finds a Code from its string representation. - * + * * @param name a name to look for. * @return the corresponding Code. */ @@ -38,7 +44,7 @@ public static Code fromString(String name) { /** * Gets known Code values. - * + * * @return known Code values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliance.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliance.java index 7d14f9285a725..01358544b4d42 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliance.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliance.java @@ -8,32 +8,34 @@ import java.time.OffsetDateTime; import java.util.List; -/** An immutable client-side representation of Compliance. */ +/** + * An immutable client-side representation of Compliance. + */ public interface Compliance { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the assessmentTimestampUtcDate property: The timestamp when the Compliance calculation was conducted. - * + * * @return the assessmentTimestampUtcDate value. */ OffsetDateTime assessmentTimestampUtcDate(); @@ -41,21 +43,21 @@ public interface Compliance { /** * Gets the resourceCount property: The resource count of the given subscription for which the Compliance * calculation was conducted (needed for Management Group Compliance calculation). - * + * * @return the resourceCount value. */ Integer resourceCount(); /** * Gets the assessmentResult property: An array of segment, which is the actually the compliance assessment. - * + * * @return the assessmentResult value. */ List assessmentResult(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.ComplianceInner object. - * + * * @return the inner object. */ ComplianceInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceList.java index c325572625cc5..cdb5d3c4b33be 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of Compliance objects response. */ +/** + * List of Compliance objects response. + */ @Fluent public final class ComplianceList { /* @@ -24,13 +26,15 @@ public final class ComplianceList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ComplianceList class. */ + /** + * Creates an instance of ComplianceList class. + */ public ComplianceList() { } /** * Get the value property: List of Compliance objects. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of Compliance objects. - * + * * @param value the value value to set. * @return the ComplianceList object itself. */ @@ -50,7 +54,7 @@ public ComplianceList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResult.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResult.java index a762e476d2592..bfb4965af144f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResult.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResult.java @@ -6,39 +6,41 @@ import com.azure.resourcemanager.security.fluent.models.ComplianceResultInner; -/** An immutable client-side representation of ComplianceResult. */ +/** + * An immutable client-side representation of ComplianceResult. + */ public interface ComplianceResult { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the resourceStatus property: The status of the resource regarding a single assessment. - * + * * @return the resourceStatus value. */ ResourceStatus resourceStatus(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.ComplianceResultInner object. - * + * * @return the inner object. */ ComplianceResultInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResultList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResultList.java index 53df27fd84992..2e0a1a18bdecd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResultList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResultList.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of compliance results response. */ +/** + * List of compliance results response. + */ @Fluent public final class ComplianceResultList { /* @@ -25,13 +27,15 @@ public final class ComplianceResultList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ComplianceResultList class. */ + /** + * Creates an instance of ComplianceResultList class. + */ public ComplianceResultList() { } /** * Get the value property: List of compliance results. - * + * * @return the value value. */ public List value() { @@ -40,7 +44,7 @@ public List value() { /** * Set the value property: List of compliance results. - * + * * @param value the value value to set. * @return the ComplianceResultList object itself. */ @@ -51,7 +55,7 @@ public ComplianceResultList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -60,14 +64,13 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { if (value() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property value in model ComplianceResultList")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property value in model ComplianceResultList")); } else { value().forEach(e -> e.validate()); } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResults.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResults.java index dfea3d0538114..5b32e6ad86328 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResults.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceResults.java @@ -8,13 +8,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ComplianceResults. */ +/** + * Resource collection API of ComplianceResults. + */ public interface ComplianceResults { /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -24,9 +26,9 @@ public interface ComplianceResults { /** * Security compliance results in the subscription. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -37,7 +39,7 @@ public interface ComplianceResults { /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @param context The context to associate with this operation. @@ -50,7 +52,7 @@ public interface ComplianceResults { /** * Security Compliance Result. - * + * * @param resourceId The identifier of the resource. * @param complianceResultName name of the desired assessment compliance result. * @throws IllegalArgumentException thrown if parameters fail the validation. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceSegment.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceSegment.java index 92424ee5d4b5e..8f4728c35f835 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceSegment.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ComplianceSegment.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** A segment of a compliance assessment. */ +/** + * A segment of a compliance assessment. + */ @Immutable public final class ComplianceSegment { /* @@ -22,13 +24,15 @@ public final class ComplianceSegment { @JsonProperty(value = "percentage", access = JsonProperty.Access.WRITE_ONLY) private Double percentage; - /** Creates an instance of ComplianceSegment class. */ + /** + * Creates an instance of ComplianceSegment class. + */ public ComplianceSegment() { } /** * Get the segmentType property: The segment type, e.g. compliant, non-compliance, insufficient coverage, N/A, etc. - * + * * @return the segmentType value. */ public String segmentType() { @@ -37,7 +41,7 @@ public String segmentType() { /** * Get the percentage property: The size (%) of the segment. - * + * * @return the percentage value. */ public Double percentage() { @@ -46,7 +50,7 @@ public Double percentage() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliances.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliances.java index 3aed2c4ad78b1..9bc6c3f347e12 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliances.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Compliances.java @@ -8,13 +8,15 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Compliances. */ +/** + * Resource collection API of Compliances. + */ public interface Compliances { /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -24,9 +26,9 @@ public interface Compliances { /** * The Compliance scores of the specific management group. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -37,9 +39,9 @@ public interface Compliances { /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -51,9 +53,9 @@ public interface Compliances { /** * Details of a specific Compliance. - * + * * @param scope Scope of the query, can be subscription (/subscriptions/0b06d9ea-afe6-4779-bd59-30e5c2d9d13f) or - * management group (/providers/Microsoft.Management/managementGroups/mgName). + * management group (/providers/Microsoft.Management/managementGroups/mgName). * @param complianceName name of the Compliance. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConfigurationStatus.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConfigurationStatus.java index ca570c91256db..d5ef61efa78d1 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConfigurationStatus.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConfigurationStatus.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The configuration status of the machines group or machine or rule. */ +/** + * The configuration status of the machines group or machine or rule. + */ public final class ConfigurationStatus extends ExpandableStringEnum { - /** Static value Configured for ConfigurationStatus. */ + /** + * Static value Configured for ConfigurationStatus. + */ public static final ConfigurationStatus CONFIGURED = fromString("Configured"); - /** Static value NotConfigured for ConfigurationStatus. */ + /** + * Static value NotConfigured for ConfigurationStatus. + */ public static final ConfigurationStatus NOT_CONFIGURED = fromString("NotConfigured"); - /** Static value InProgress for ConfigurationStatus. */ + /** + * Static value InProgress for ConfigurationStatus. + */ public static final ConfigurationStatus IN_PROGRESS = fromString("InProgress"); - /** Static value Failed for ConfigurationStatus. */ + /** + * Static value Failed for ConfigurationStatus. + */ public static final ConfigurationStatus FAILED = fromString("Failed"); - /** Static value NoStatus for ConfigurationStatus. */ + /** + * Static value NoStatus for ConfigurationStatus. + */ public static final ConfigurationStatus NO_STATUS = fromString("NoStatus"); /** * Creates a new instance of ConfigurationStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public ConfigurationStatus() { /** * Creates or finds a ConfigurationStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding ConfigurationStatus. */ @@ -47,7 +59,7 @@ public static ConfigurationStatus fromString(String name) { /** * Gets known ConfigurationStatus values. - * + * * @return known ConfigurationStatus values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectableResource.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectableResource.java index 19acfd395a631..2c6fd74ee58e2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectableResource.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectableResource.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes the allowed inbound and outbound traffic of an Azure resource. */ +/** + * Describes the allowed inbound and outbound traffic of an Azure resource. + */ @Immutable public final class ConnectableResource { /* @@ -29,13 +31,15 @@ public final class ConnectableResource { @JsonProperty(value = "outboundConnectedResources", access = JsonProperty.Access.WRITE_ONLY) private List outboundConnectedResources; - /** Creates an instance of ConnectableResource class. */ + /** + * Creates an instance of ConnectableResource class. + */ public ConnectableResource() { } /** * Get the id property: The Azure resource id. - * + * * @return the id value. */ public String id() { @@ -45,7 +49,7 @@ public String id() { /** * Get the inboundConnectedResources property: The list of Azure resources that the resource has inbound allowed * connection from. - * + * * @return the inboundConnectedResources value. */ public List inboundConnectedResources() { @@ -55,7 +59,7 @@ public List inboundConnectedResources() { /** * Get the outboundConnectedResources property: The list of Azure resources that the resource has outbound allowed * connection to. - * + * * @return the outboundConnectedResources value. */ public List outboundConnectedResources() { @@ -64,7 +68,7 @@ public List outboundConnectedResources() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedResource.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedResource.java index 222924164abd2..c2f50d81704c7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedResource.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedResource.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** Describes properties of a connected resource. */ +/** + * Describes properties of a connected resource. + */ @Immutable public final class ConnectedResource { /* @@ -28,13 +30,15 @@ public final class ConnectedResource { @JsonProperty(value = "udpPorts", access = JsonProperty.Access.WRITE_ONLY) private String udpPorts; - /** Creates an instance of ConnectedResource class. */ + /** + * Creates an instance of ConnectedResource class. + */ public ConnectedResource() { } /** * Get the connectedResourceId property: The Azure resource id of the connected resource. - * + * * @return the connectedResourceId value. */ public String connectedResourceId() { @@ -43,7 +47,7 @@ public String connectedResourceId() { /** * Get the tcpPorts property: The allowed tcp ports. - * + * * @return the tcpPorts value. */ public String tcpPorts() { @@ -52,7 +56,7 @@ public String tcpPorts() { /** * Get the udpPorts property: The allowed udp ports. - * + * * @return the udpPorts value. */ public String udpPorts() { @@ -61,7 +65,7 @@ public String udpPorts() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedWorkspace.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedWorkspace.java index 34a0ad3b5ccf9..22bb02e956c30 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedWorkspace.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectedWorkspace.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Represents an OMS workspace to which the solution is connected. */ +/** + * Represents an OMS workspace to which the solution is connected. + */ @Fluent public final class ConnectedWorkspace { /* @@ -16,13 +18,15 @@ public final class ConnectedWorkspace { @JsonProperty(value = "id") private String id; - /** Creates an instance of ConnectedWorkspace class. */ + /** + * Creates an instance of ConnectedWorkspace class. + */ public ConnectedWorkspace() { } /** * Get the id property: Azure resource ID of the connected OMS workspace. - * + * * @return the id value. */ public String id() { @@ -31,7 +35,7 @@ public String id() { /** * Set the id property: Azure resource ID of the connected OMS workspace. - * + * * @param id the id value to set. * @return the ConnectedWorkspace object itself. */ @@ -42,7 +46,7 @@ public ConnectedWorkspace withId(String id) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionFromIpNotAllowed.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionFromIpNotAllowed.java index 09db2166904f3..70e8dee8aea34 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionFromIpNotAllowed.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionFromIpNotAllowed.java @@ -9,23 +9,31 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Inbound connection from an ip that isn't allowed. Allow list consists of ipv4 or ipv6 range in CIDR notation. */ +/** + * Inbound connection from an ip that isn't allowed. Allow list consists of ipv4 or ipv6 range in CIDR notation. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("ConnectionFromIpNotAllowed") @Fluent public final class ConnectionFromIpNotAllowed extends AllowlistCustomAlertRule { - /** Creates an instance of ConnectionFromIpNotAllowed class. */ + /** + * Creates an instance of ConnectionFromIpNotAllowed class. + */ public ConnectionFromIpNotAllowed() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ConnectionFromIpNotAllowed withAllowlistValues(List allowlistValues) { super.withAllowlistValues(allowlistValues); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ConnectionFromIpNotAllowed withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -34,7 +42,7 @@ public ConnectionFromIpNotAllowed withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionStrings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionStrings.java deleted file mode 100644 index c5da6b30ddaf0..0000000000000 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionStrings.java +++ /dev/null @@ -1,25 +0,0 @@ -// Copyright (c) Microsoft Corporation. All rights reserved. -// Licensed under the MIT License. -// Code generated by Microsoft (R) AutoRest Code Generator. - -package com.azure.resourcemanager.security.models; - -import com.azure.resourcemanager.security.fluent.models.ConnectionStringsInner; -import java.util.List; - -/** An immutable client-side representation of ConnectionStrings. */ -public interface ConnectionStrings { - /** - * Gets the value property: Connection strings. - * - * @return the value value. - */ - List value(); - - /** - * Gets the inner com.azure.resourcemanager.security.fluent.models.ConnectionStringsInner object. - * - * @return the inner object. - */ - ConnectionStringsInner innerModel(); -} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionToIpNotAllowed.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionToIpNotAllowed.java index 59841c97d2d05..44efda923dae0 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionToIpNotAllowed.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionToIpNotAllowed.java @@ -9,23 +9,31 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** Outbound connection to an ip that isn't allowed. Allow list consists of ipv4 or ipv6 range in CIDR notation. */ +/** + * Outbound connection to an ip that isn't allowed. Allow list consists of ipv4 or ipv6 range in CIDR notation. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("ConnectionToIpNotAllowed") @Fluent public final class ConnectionToIpNotAllowed extends AllowlistCustomAlertRule { - /** Creates an instance of ConnectionToIpNotAllowed class. */ + /** + * Creates an instance of ConnectionToIpNotAllowed class. + */ public ConnectionToIpNotAllowed() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ConnectionToIpNotAllowed withAllowlistValues(List allowlistValues) { super.withAllowlistValues(allowlistValues); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public ConnectionToIpNotAllowed withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -34,7 +42,7 @@ public ConnectionToIpNotAllowed withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionType.java index f26f0c83f24b5..5277fab3b3035 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectionType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for ConnectionType. */ +/** + * Defines values for ConnectionType. + */ public final class ConnectionType extends ExpandableStringEnum { - /** Static value Internal for ConnectionType. */ + /** + * Static value Internal for ConnectionType. + */ public static final ConnectionType INTERNAL = fromString("Internal"); - /** Static value External for ConnectionType. */ + /** + * Static value External for ConnectionType. + */ public static final ConnectionType EXTERNAL = fromString("External"); /** * Creates a new instance of ConnectionType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ConnectionType() { /** * Creates or finds a ConnectionType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ConnectionType. */ @@ -38,7 +44,7 @@ public static ConnectionType fromString(String name) { /** * Gets known ConnectionType values. - * + * * @return known ConnectionType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSetting.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSetting.java index 5a13750d1686c..0542d4b9ef658 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSetting.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSetting.java @@ -7,25 +7,27 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.ConnectorSettingInner; -/** An immutable client-side representation of ConnectorSetting. */ +/** + * An immutable client-side representation of ConnectorSetting. + */ public interface ConnectorSetting { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); @@ -33,7 +35,7 @@ public interface ConnectorSetting { /** * Gets the hybridComputeSettings property: Settings for hybrid compute management. These settings are relevant only * for Arc autoProvision (Hybrid Compute). - * + * * @return the hybridComputeSettings value. */ HybridComputeSettingsProperties hybridComputeSettings(); @@ -41,26 +43,34 @@ public interface ConnectorSetting { /** * Gets the authenticationDetails property: Settings for authentication management, these settings are relevant only * for the cloud connector. - * + * * @return the authenticationDetails value. */ AuthenticationDetailsProperties authenticationDetails(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.ConnectorSettingInner object. - * + * * @return the inner object. */ ConnectorSettingInner innerModel(); - /** The entirety of the ConnectorSetting definition. */ + /** + * The entirety of the ConnectorSetting definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithCreate { } - /** The ConnectorSetting definition stages. */ + + /** + * The ConnectorSetting definition stages. + */ interface DefinitionStages { - /** The first stage of the ConnectorSetting definition. */ + /** + * The first stage of the ConnectorSetting definition. + */ interface Blank extends WithCreate { } + /** * The stage of the ConnectorSetting definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. @@ -69,105 +79,123 @@ interface WithCreate extends DefinitionStages.WithHybridComputeSettings, DefinitionStages.WithAuthenticationDetails { /** * Executes the create request. - * + * * @return the created resource. */ ConnectorSetting create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ ConnectorSetting create(Context context); } - /** The stage of the ConnectorSetting definition allowing to specify hybridComputeSettings. */ + + /** + * The stage of the ConnectorSetting definition allowing to specify hybridComputeSettings. + */ interface WithHybridComputeSettings { /** * Specifies the hybridComputeSettings property: Settings for hybrid compute management. These settings are * relevant only for Arc autoProvision (Hybrid Compute).. - * + * * @param hybridComputeSettings Settings for hybrid compute management. These settings are relevant only for - * Arc autoProvision (Hybrid Compute). + * Arc autoProvision (Hybrid Compute). * @return the next definition stage. */ WithCreate withHybridComputeSettings(HybridComputeSettingsProperties hybridComputeSettings); } - /** The stage of the ConnectorSetting definition allowing to specify authenticationDetails. */ + + /** + * The stage of the ConnectorSetting definition allowing to specify authenticationDetails. + */ interface WithAuthenticationDetails { /** * Specifies the authenticationDetails property: Settings for authentication management, these settings are * relevant only for the cloud connector.. - * + * * @param authenticationDetails Settings for authentication management, these settings are relevant only for - * the cloud connector. + * the cloud connector. * @return the next definition stage. */ WithCreate withAuthenticationDetails(AuthenticationDetailsProperties authenticationDetails); } } + /** * Begins update for the ConnectorSetting resource. - * + * * @return the stage of resource update. */ ConnectorSetting.Update update(); - /** The template for ConnectorSetting update. */ + /** + * The template for ConnectorSetting update. + */ interface Update extends UpdateStages.WithHybridComputeSettings, UpdateStages.WithAuthenticationDetails { /** * Executes the update request. - * + * * @return the updated resource. */ ConnectorSetting apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ ConnectorSetting apply(Context context); } - /** The ConnectorSetting update stages. */ + + /** + * The ConnectorSetting update stages. + */ interface UpdateStages { - /** The stage of the ConnectorSetting update allowing to specify hybridComputeSettings. */ + /** + * The stage of the ConnectorSetting update allowing to specify hybridComputeSettings. + */ interface WithHybridComputeSettings { /** * Specifies the hybridComputeSettings property: Settings for hybrid compute management. These settings are * relevant only for Arc autoProvision (Hybrid Compute).. - * + * * @param hybridComputeSettings Settings for hybrid compute management. These settings are relevant only for - * Arc autoProvision (Hybrid Compute). + * Arc autoProvision (Hybrid Compute). * @return the next definition stage. */ Update withHybridComputeSettings(HybridComputeSettingsProperties hybridComputeSettings); } - /** The stage of the ConnectorSetting update allowing to specify authenticationDetails. */ + + /** + * The stage of the ConnectorSetting update allowing to specify authenticationDetails. + */ interface WithAuthenticationDetails { /** * Specifies the authenticationDetails property: Settings for authentication management, these settings are * relevant only for the cloud connector.. - * + * * @param authenticationDetails Settings for authentication management, these settings are relevant only for - * the cloud connector. + * the cloud connector. * @return the next definition stage. */ Update withAuthenticationDetails(AuthenticationDetailsProperties authenticationDetails); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ ConnectorSetting refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSettingList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSettingList.java index f593b57393806..cb7207bf37d95 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSettingList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ConnectorSettingList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** For a subscription, list of all cloud account connectors and their settings. */ +/** + * For a subscription, list of all cloud account connectors and their settings. + */ @Fluent public final class ConnectorSettingList { /* @@ -24,13 +26,15 @@ public final class ConnectorSettingList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ConnectorSettingList class. */ + /** + * Creates an instance of ConnectorSettingList class. + */ public ConnectorSettingList() { } /** * Get the value property: List of all the cloud account connector settings. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of all the cloud account connector settings. - * + * * @param value the value value to set. * @return the ConnectorSettingList object itself. */ @@ -50,7 +54,7 @@ public ConnectorSettingList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Connectors.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Connectors.java index 0fbe5c0e8ca4b..1d72270a40bd6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Connectors.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Connectors.java @@ -8,33 +8,35 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of Connectors. */ +/** + * Resource collection API of Connectors. + */ public interface Connectors { /** * Cloud accounts connectors of a subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ PagedIterable list(); /** * Cloud accounts connectors of a subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return for a subscription, list of all cloud account connectors and their settings as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -46,7 +48,7 @@ public interface Connectors { /** * Details of a specific cloud account connector. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -57,7 +59,7 @@ public interface Connectors { /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -69,7 +71,7 @@ public interface Connectors { /** * Delete a cloud account connector from a subscription. - * + * * @param connectorName Name of the cloud account connector. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -79,7 +81,7 @@ public interface Connectors { /** * Details of a specific cloud account connector. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -90,7 +92,7 @@ public interface Connectors { /** * Details of a specific cloud account connector. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -102,7 +104,7 @@ public interface Connectors { /** * Delete a cloud account connector from a subscription. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -112,7 +114,7 @@ public interface Connectors { /** * Delete a cloud account connector from a subscription. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -124,7 +126,7 @@ public interface Connectors { /** * Begins definition for a new ConnectorSetting resource. - * + * * @param name resource name. * @return the first stage of the new ConnectorSetting definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ContainerRegistryVulnerabilityProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ContainerRegistryVulnerabilityProperties.java index 490b6e85e689a..bf2651b150956 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ContainerRegistryVulnerabilityProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ContainerRegistryVulnerabilityProperties.java @@ -13,7 +13,9 @@ import java.util.List; import java.util.Map; -/** Additional context fields for container registry Vulnerability assessment. */ +/** + * Additional context fields for container registry Vulnerability assessment. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "assessedResourceType") @JsonTypeName("ContainerRegistryVulnerability") @Immutable @@ -67,14 +69,16 @@ public final class ContainerRegistryVulnerabilityProperties extends AdditionalDa @JsonProperty(value = "imageDigest", access = JsonProperty.Access.WRITE_ONLY) private String imageDigest; - /** Creates an instance of ContainerRegistryVulnerabilityProperties class. */ + /** + * Creates an instance of ContainerRegistryVulnerabilityProperties class. + */ public ContainerRegistryVulnerabilityProperties() { } /** * Get the type property: Vulnerability Type. e.g: Vulnerability, Potential Vulnerability, Information Gathered, * Vulnerability. - * + * * @return the type value. */ public String type() { @@ -83,7 +87,7 @@ public String type() { /** * Get the cvss property: Dictionary from cvss version to cvss details object. - * + * * @return the cvss value. */ public Map cvss() { @@ -92,7 +96,7 @@ public Map cvss() { /** * Get the patchable property: Indicates whether a patch is available or not. - * + * * @return the patchable value. */ public Boolean patchable() { @@ -101,7 +105,7 @@ public Boolean patchable() { /** * Get the cve property: List of CVEs. - * + * * @return the cve value. */ public List cve() { @@ -110,7 +114,7 @@ public List cve() { /** * Get the publishedTime property: Published time. - * + * * @return the publishedTime value. */ public OffsetDateTime publishedTime() { @@ -119,7 +123,7 @@ public OffsetDateTime publishedTime() { /** * Get the vendorReferences property: The vendorReferences property. - * + * * @return the vendorReferences value. */ public List vendorReferences() { @@ -128,7 +132,7 @@ public List vendorReferences() { /** * Get the repositoryName property: Name of the repository which the vulnerable image belongs to. - * + * * @return the repositoryName value. */ public String repositoryName() { @@ -137,7 +141,7 @@ public String repositoryName() { /** * Get the imageDigest property: Digest of the vulnerable image. - * + * * @return the imageDigest value. */ public String imageDigest() { @@ -146,21 +150,18 @@ public String imageDigest() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (cvss() != null) { - cvss() - .values() - .forEach( - e -> { - if (e != null) { - e.validate(); - } - }); + cvss().values().forEach(e -> { + if (e != null) { + e.validate(); + } + }); } if (cve() != null) { cve().forEach(e -> e.validate()); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ControlType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ControlType.java index b2502b132198f..db3d2ac06d2a5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ControlType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ControlType.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The type of security control (for example, BuiltIn). */ +/** + * The type of security control (for example, BuiltIn). + */ public final class ControlType extends ExpandableStringEnum { - /** Static value BuiltIn for ControlType. */ + /** + * Static value BuiltIn for ControlType. + */ public static final ControlType BUILT_IN = fromString("BuiltIn"); - /** Static value Custom for ControlType. */ + /** + * Static value Custom for ControlType. + */ public static final ControlType CUSTOM = fromString("Custom"); /** * Creates a new instance of ControlType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ControlType() { /** * Creates or finds a ControlType from its string representation. - * + * * @param name a name to look for. * @return the corresponding ControlType. */ @@ -38,7 +44,7 @@ public static ControlType fromString(String name) { /** * Gets known ControlType values. - * + * * @return known ControlType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOffering.java index 3ead5e6e1dca6..bcb1d32469a18 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The CSPM monitoring for AWS offering. */ +/** + * The CSPM monitoring for AWS offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("CspmMonitorAws") @Fluent @@ -20,13 +22,15 @@ public final class CspmMonitorAwsOffering extends CloudOffering { @JsonProperty(value = "nativeCloudConnection") private CspmMonitorAwsOfferingNativeCloudConnection nativeCloudConnection; - /** Creates an instance of CspmMonitorAwsOffering class. */ + /** + * Creates an instance of CspmMonitorAwsOffering class. + */ public CspmMonitorAwsOffering() { } /** * Get the nativeCloudConnection property: The native cloud connection configuration. - * + * * @return the nativeCloudConnection value. */ public CspmMonitorAwsOfferingNativeCloudConnection nativeCloudConnection() { @@ -35,19 +39,19 @@ public CspmMonitorAwsOfferingNativeCloudConnection nativeCloudConnection() { /** * Set the nativeCloudConnection property: The native cloud connection configuration. - * + * * @param nativeCloudConnection the nativeCloudConnection value to set. * @return the CspmMonitorAwsOffering object itself. */ - public CspmMonitorAwsOffering withNativeCloudConnection( - CspmMonitorAwsOfferingNativeCloudConnection nativeCloudConnection) { + public CspmMonitorAwsOffering + withNativeCloudConnection(CspmMonitorAwsOfferingNativeCloudConnection nativeCloudConnection) { this.nativeCloudConnection = nativeCloudConnection; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOfferingNativeCloudConnection.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOfferingNativeCloudConnection.java index 0c58266fdcb86..53368e794c002 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOfferingNativeCloudConnection.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAwsOfferingNativeCloudConnection.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The native cloud connection configuration. */ +/** + * The native cloud connection configuration. + */ @Fluent public final class CspmMonitorAwsOfferingNativeCloudConnection { /* @@ -16,13 +18,15 @@ public final class CspmMonitorAwsOfferingNativeCloudConnection { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of CspmMonitorAwsOfferingNativeCloudConnection class. */ + /** + * Creates an instance of CspmMonitorAwsOfferingNativeCloudConnection class. + */ public CspmMonitorAwsOfferingNativeCloudConnection() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the CspmMonitorAwsOfferingNativeCloudConnection object itself. */ @@ -42,7 +46,7 @@ public CspmMonitorAwsOfferingNativeCloudConnection withCloudRoleArn(String cloud /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAzureDevOpsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAzureDevOpsOffering.java index 37effcc24def8..97dc15f77abe9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAzureDevOpsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorAzureDevOpsOffering.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The CSPM monitoring for AzureDevOps offering. */ +/** + * The CSPM monitoring for AzureDevOps offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("CspmMonitorAzureDevOps") @Immutable public final class CspmMonitorAzureDevOpsOffering extends CloudOffering { - /** Creates an instance of CspmMonitorAzureDevOpsOffering class. */ + /** + * Creates an instance of CspmMonitorAzureDevOpsOffering class. + */ public CspmMonitorAzureDevOpsOffering() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOffering.java index 48f5c5f63a00b..eda75d55d126b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The CSPM monitoring for GCP offering. */ +/** + * The CSPM monitoring for GCP offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("CspmMonitorGcp") @Fluent @@ -20,13 +22,15 @@ public final class CspmMonitorGcpOffering extends CloudOffering { @JsonProperty(value = "nativeCloudConnection") private CspmMonitorGcpOfferingNativeCloudConnection nativeCloudConnection; - /** Creates an instance of CspmMonitorGcpOffering class. */ + /** + * Creates an instance of CspmMonitorGcpOffering class. + */ public CspmMonitorGcpOffering() { } /** * Get the nativeCloudConnection property: The native cloud connection configuration. - * + * * @return the nativeCloudConnection value. */ public CspmMonitorGcpOfferingNativeCloudConnection nativeCloudConnection() { @@ -35,19 +39,19 @@ public CspmMonitorGcpOfferingNativeCloudConnection nativeCloudConnection() { /** * Set the nativeCloudConnection property: The native cloud connection configuration. - * + * * @param nativeCloudConnection the nativeCloudConnection value to set. * @return the CspmMonitorGcpOffering object itself. */ - public CspmMonitorGcpOffering withNativeCloudConnection( - CspmMonitorGcpOfferingNativeCloudConnection nativeCloudConnection) { + public CspmMonitorGcpOffering + withNativeCloudConnection(CspmMonitorGcpOfferingNativeCloudConnection nativeCloudConnection) { this.nativeCloudConnection = nativeCloudConnection; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOfferingNativeCloudConnection.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOfferingNativeCloudConnection.java index 3b653d78726c7..7b461f5889502 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOfferingNativeCloudConnection.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGcpOfferingNativeCloudConnection.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The native cloud connection configuration. */ +/** + * The native cloud connection configuration. + */ @Fluent public final class CspmMonitorGcpOfferingNativeCloudConnection { /* @@ -22,13 +24,15 @@ public final class CspmMonitorGcpOfferingNativeCloudConnection { @JsonProperty(value = "serviceAccountEmailAddress") private String serviceAccountEmailAddress; - /** Creates an instance of CspmMonitorGcpOfferingNativeCloudConnection class. */ + /** + * Creates an instance of CspmMonitorGcpOfferingNativeCloudConnection class. + */ public CspmMonitorGcpOfferingNativeCloudConnection() { } /** * Get the workloadIdentityProviderId property: The GCP workload identity provider id for the offering. - * + * * @return the workloadIdentityProviderId value. */ public String workloadIdentityProviderId() { @@ -37,19 +41,19 @@ public String workloadIdentityProviderId() { /** * Set the workloadIdentityProviderId property: The GCP workload identity provider id for the offering. - * + * * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. * @return the CspmMonitorGcpOfferingNativeCloudConnection object itself. */ - public CspmMonitorGcpOfferingNativeCloudConnection withWorkloadIdentityProviderId( - String workloadIdentityProviderId) { + public CspmMonitorGcpOfferingNativeCloudConnection + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { this.workloadIdentityProviderId = workloadIdentityProviderId; return this; } /** * Get the serviceAccountEmailAddress property: The service account email address in GCP for this offering. - * + * * @return the serviceAccountEmailAddress value. */ public String serviceAccountEmailAddress() { @@ -58,19 +62,19 @@ public String serviceAccountEmailAddress() { /** * Set the serviceAccountEmailAddress property: The service account email address in GCP for this offering. - * + * * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. * @return the CspmMonitorGcpOfferingNativeCloudConnection object itself. */ - public CspmMonitorGcpOfferingNativeCloudConnection withServiceAccountEmailAddress( - String serviceAccountEmailAddress) { + public CspmMonitorGcpOfferingNativeCloudConnection + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { this.serviceAccountEmailAddress = serviceAccountEmailAddress; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGitLabOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGitLabOffering.java index c4968ba49cf13..be0f785b499eb 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGitLabOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGitLabOffering.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The CSPM (Cloud security posture management) monitoring for gitlab offering. */ +/** + * The CSPM (Cloud security posture management) monitoring for gitlab offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("CspmMonitorGitLab") @Immutable public final class CspmMonitorGitLabOffering extends CloudOffering { - /** Creates an instance of CspmMonitorGitLabOffering class. */ + /** + * Creates an instance of CspmMonitorGitLabOffering class. + */ public CspmMonitorGitLabOffering() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGithubOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGithubOffering.java index 1039d0c712fdc..5612245c734ee 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGithubOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CspmMonitorGithubOffering.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The CSPM monitoring for github offering. */ +/** + * The CSPM monitoring for github offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("CspmMonitorGithub") @Immutable public final class CspmMonitorGithubOffering extends CloudOffering { - /** Creates an instance of CspmMonitorGithubOffering class. */ + /** + * Creates an instance of CspmMonitorGithubOffering class. + */ public CspmMonitorGithubOffering() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAlertRule.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAlertRule.java index 6b17e8b182ee6..e65668c0394d7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAlertRule.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAlertRule.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** A custom alert rule. */ +/** + * A custom alert rule. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -19,8 +21,7 @@ @JsonTypeName("CustomAlertRule") @JsonSubTypes({ @JsonSubTypes.Type(name = "ThresholdCustomAlertRule", value = ThresholdCustomAlertRule.class), - @JsonSubTypes.Type(name = "ListCustomAlertRule", value = ListCustomAlertRule.class) -}) + @JsonSubTypes.Type(name = "ListCustomAlertRule", value = ListCustomAlertRule.class) }) @Fluent public class CustomAlertRule { /* @@ -41,13 +42,15 @@ public class CustomAlertRule { @JsonProperty(value = "isEnabled", required = true) private boolean isEnabled; - /** Creates an instance of CustomAlertRule class. */ + /** + * Creates an instance of CustomAlertRule class. + */ public CustomAlertRule() { } /** * Get the displayName property: The display name of the custom alert. - * + * * @return the displayName value. */ public String displayName() { @@ -56,7 +59,7 @@ public String displayName() { /** * Get the description property: The description of the custom alert. - * + * * @return the description value. */ public String description() { @@ -65,7 +68,7 @@ public String description() { /** * Get the isEnabled property: Status of the custom alert. - * + * * @return the isEnabled value. */ public boolean isEnabled() { @@ -74,7 +77,7 @@ public boolean isEnabled() { /** * Set the isEnabled property: Status of the custom alert. - * + * * @param isEnabled the isEnabled value to set. * @return the CustomAlertRule object itself. */ @@ -85,7 +88,7 @@ public CustomAlertRule withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomation.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomation.java index 59ff105ba2b42..31e7cbdf895b2 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomation.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomation.java @@ -8,32 +8,34 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.CustomAssessmentAutomationInner; -/** An immutable client-side representation of CustomAssessmentAutomation. */ +/** + * An immutable client-side representation of CustomAssessmentAutomation. + */ public interface CustomAssessmentAutomation { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); @@ -41,28 +43,28 @@ public interface CustomAssessmentAutomation { /** * Gets the compressedQuery property: GZip encoded KQL query representing the assessment automation results * required. - * + * * @return the compressedQuery value. */ String compressedQuery(); /** * Gets the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @return the supportedCloud value. */ SupportedCloudEnum supportedCloud(); /** * Gets the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @return the severity value. */ SeverityEnum severity(); /** * Gets the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @return the displayName value. */ String displayName(); @@ -70,7 +72,7 @@ public interface CustomAssessmentAutomation { /** * Gets the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @return the description value. */ String description(); @@ -78,7 +80,7 @@ public interface CustomAssessmentAutomation { /** * Gets the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @return the remediationDescription value. */ String remediationDescription(); @@ -86,141 +88,167 @@ public interface CustomAssessmentAutomation { /** * Gets the assessmentKey property: The assessment metadata key used when an assessment is generated for this * assessment automation. - * + * * @return the assessmentKey value. */ String assessmentKey(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.CustomAssessmentAutomationInner object. - * + * * @return the inner object. */ CustomAssessmentAutomationInner innerModel(); - /** The entirety of the CustomAssessmentAutomation definition. */ + /** + * The entirety of the CustomAssessmentAutomation definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The CustomAssessmentAutomation definition stages. */ + + /** + * The CustomAssessmentAutomation definition stages. + */ interface DefinitionStages { - /** The first stage of the CustomAssessmentAutomation definition. */ + /** + * The first stage of the CustomAssessmentAutomation definition. + */ interface Blank extends WithResourceGroup { } - /** The stage of the CustomAssessmentAutomation definition allowing to specify parent resource. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @return the next definition stage. */ WithCreate withExistingResourceGroup(String resourceGroupName); } + /** * The stage of the CustomAssessmentAutomation definition which contains all the minimum required properties for * the resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithCompressedQuery, - DefinitionStages.WithSupportedCloud, - DefinitionStages.WithSeverity, - DefinitionStages.WithDisplayName, - DefinitionStages.WithDescription, - DefinitionStages.WithRemediationDescription { + interface WithCreate extends DefinitionStages.WithCompressedQuery, DefinitionStages.WithSupportedCloud, + DefinitionStages.WithSeverity, DefinitionStages.WithDisplayName, DefinitionStages.WithDescription, + DefinitionStages.WithRemediationDescription { /** * Executes the create request. - * + * * @return the created resource. */ CustomAssessmentAutomation create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ CustomAssessmentAutomation create(Context context); } - /** The stage of the CustomAssessmentAutomation definition allowing to specify compressedQuery. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify compressedQuery. + */ interface WithCompressedQuery { /** * Specifies the compressedQuery property: Base 64 encoded KQL query representing the assessment automation * results required.. - * + * * @param compressedQuery Base 64 encoded KQL query representing the assessment automation results required. * @return the next definition stage. */ WithCreate withCompressedQuery(String compressedQuery); } - /** The stage of the CustomAssessmentAutomation definition allowing to specify supportedCloud. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify supportedCloud. + */ interface WithSupportedCloud { /** * Specifies the supportedCloud property: Relevant cloud for the custom assessment automation.. - * + * * @param supportedCloud Relevant cloud for the custom assessment automation. * @return the next definition stage. */ WithCreate withSupportedCloud(SupportedCloudEnum supportedCloud); } - /** The stage of the CustomAssessmentAutomation definition allowing to specify severity. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify severity. + */ interface WithSeverity { /** * Specifies the severity property: The severity to relate to the assessments generated by this assessment * automation.. - * + * * @param severity The severity to relate to the assessments generated by this assessment automation. * @return the next definition stage. */ WithCreate withSeverity(SeverityEnum severity); } - /** The stage of the CustomAssessmentAutomation definition allowing to specify displayName. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify displayName. + */ interface WithDisplayName { /** * Specifies the displayName property: The display name of the assessments generated by this assessment * automation.. - * + * * @param displayName The display name of the assessments generated by this assessment automation. * @return the next definition stage. */ WithCreate withDisplayName(String displayName); } - /** The stage of the CustomAssessmentAutomation definition allowing to specify description. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify description. + */ interface WithDescription { /** * Specifies the description property: The description to relate to the assessments generated by this * assessment automation.. - * + * * @param description The description to relate to the assessments generated by this assessment automation. * @return the next definition stage. */ WithCreate withDescription(String description); } - /** The stage of the CustomAssessmentAutomation definition allowing to specify remediationDescription. */ + + /** + * The stage of the CustomAssessmentAutomation definition allowing to specify remediationDescription. + */ interface WithRemediationDescription { /** * Specifies the remediationDescription property: The remediation description to relate to the assessments * generated by this assessment automation.. - * + * * @param remediationDescription The remediation description to relate to the assessments generated by this - * assessment automation. + * assessment automation. * @return the next definition stage. */ WithCreate withRemediationDescription(String remediationDescription); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ CustomAssessmentAutomation refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationRequest.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationRequest.java index 0c48f14f9c28b..a8eac3b820950 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationRequest.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationRequest.java @@ -9,7 +9,9 @@ import com.azure.resourcemanager.security.fluent.models.CustomAssessmentAutomationRequestProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** Custom Assessment Automation request. */ +/** + * Custom Assessment Automation request. + */ @Fluent public final class CustomAssessmentAutomationRequest extends ProxyResource { /* @@ -18,13 +20,15 @@ public final class CustomAssessmentAutomationRequest extends ProxyResource { @JsonProperty(value = "properties") private CustomAssessmentAutomationRequestProperties innerProperties; - /** Creates an instance of CustomAssessmentAutomationRequest class. */ + /** + * Creates an instance of CustomAssessmentAutomationRequest class. + */ public CustomAssessmentAutomationRequest() { } /** * Get the innerProperties property: describes Custom Assessment Automation request properties. - * + * * @return the innerProperties value. */ private CustomAssessmentAutomationRequestProperties innerProperties() { @@ -34,7 +38,7 @@ private CustomAssessmentAutomationRequestProperties innerProperties() { /** * Get the compressedQuery property: Base 64 encoded KQL query representing the assessment automation results * required. - * + * * @return the compressedQuery value. */ public String compressedQuery() { @@ -44,7 +48,7 @@ public String compressedQuery() { /** * Set the compressedQuery property: Base 64 encoded KQL query representing the assessment automation results * required. - * + * * @param compressedQuery the compressedQuery value to set. * @return the CustomAssessmentAutomationRequest object itself. */ @@ -58,7 +62,7 @@ public CustomAssessmentAutomationRequest withCompressedQuery(String compressedQu /** * Get the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @return the supportedCloud value. */ public SupportedCloudEnum supportedCloud() { @@ -67,7 +71,7 @@ public SupportedCloudEnum supportedCloud() { /** * Set the supportedCloud property: Relevant cloud for the custom assessment automation. - * + * * @param supportedCloud the supportedCloud value to set. * @return the CustomAssessmentAutomationRequest object itself. */ @@ -81,7 +85,7 @@ public CustomAssessmentAutomationRequest withSupportedCloud(SupportedCloudEnum s /** * Get the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @return the severity value. */ public SeverityEnum severity() { @@ -90,7 +94,7 @@ public SeverityEnum severity() { /** * Set the severity property: The severity to relate to the assessments generated by this assessment automation. - * + * * @param severity the severity value to set. * @return the CustomAssessmentAutomationRequest object itself. */ @@ -104,7 +108,7 @@ public CustomAssessmentAutomationRequest withSeverity(SeverityEnum severity) { /** * Get the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @return the displayName value. */ public String displayName() { @@ -113,7 +117,7 @@ public String displayName() { /** * Set the displayName property: The display name of the assessments generated by this assessment automation. - * + * * @param displayName the displayName value to set. * @return the CustomAssessmentAutomationRequest object itself. */ @@ -128,7 +132,7 @@ public CustomAssessmentAutomationRequest withDisplayName(String displayName) { /** * Get the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @return the description value. */ public String description() { @@ -138,7 +142,7 @@ public String description() { /** * Set the description property: The description to relate to the assessments generated by this assessment * automation. - * + * * @param description the description value to set. * @return the CustomAssessmentAutomationRequest object itself. */ @@ -153,7 +157,7 @@ public CustomAssessmentAutomationRequest withDescription(String description) { /** * Get the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @return the remediationDescription value. */ public String remediationDescription() { @@ -163,7 +167,7 @@ public String remediationDescription() { /** * Set the remediationDescription property: The remediation description to relate to the assessments generated by * this assessment automation. - * + * * @param remediationDescription the remediationDescription value to set. * @return the CustomAssessmentAutomationRequest object itself. */ @@ -177,7 +181,7 @@ public CustomAssessmentAutomationRequest withRemediationDescription(String remed /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomations.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomations.java index 5a3c288c946fd..664d26aeac330 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomations.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomations.java @@ -8,33 +8,35 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of CustomAssessmentAutomations. */ +/** + * Resource collection API of CustomAssessmentAutomations. + */ public interface CustomAssessmentAutomations { /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response}. + * {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String customAssessmentAutomationName, Context context); /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -45,11 +47,11 @@ Response getByResourceGroupWithResponse( /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -57,16 +59,16 @@ Response getByResourceGroupWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteByResourceGroupWithResponse( - String resourceGroupName, String customAssessmentAutomationName, Context context); + Response deleteByResourceGroupWithResponse(String resourceGroupName, String customAssessmentAutomationName, + Context context); /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customAssessmentAutomationName Name of the Custom Assessment Automation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -76,11 +78,11 @@ Response deleteByResourceGroupWithResponse( /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -90,11 +92,11 @@ Response deleteByResourceGroupWithResponse( /** * List custom assessment automations in a subscription and a resource group - * - *

List custom assessment automations by provided subscription and resource group. - * + * + * List custom assessment automations by provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -105,9 +107,9 @@ Response deleteByResourceGroupWithResponse( /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of Custom Assessment Automations as paginated response with {@link PagedIterable}. @@ -116,9 +118,9 @@ Response deleteByResourceGroupWithResponse( /** * List custom assessment automations in a subscription - * - *

List custom assessment automations by provided subscription. - * + * + * List custom assessment automations by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -129,38 +131,38 @@ Response deleteByResourceGroupWithResponse( /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response}. + * {@link Response}. */ CustomAssessmentAutomation getById(String id); /** * Gets a custom assessment automation - * - *

Gets a single custom assessment automation by name for the provided subscription and resource group. - * + * + * Gets a single custom assessment automation by name for the provided subscription and resource group. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom assessment automation by name for the provided subscription and resource group along with - * {@link Response}. + * {@link Response}. */ Response getByIdWithResponse(String id, Context context); /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -170,9 +172,9 @@ Response deleteByResourceGroupWithResponse( /** * Deletes a custom assessment automation - * - *

Deletes a custom assessment automation by name for a provided subscription. - * + * + * Deletes a custom assessment automation by name for a provided subscription. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -184,7 +186,7 @@ Response deleteByResourceGroupWithResponse( /** * Begins definition for a new CustomAssessmentAutomation resource. - * + * * @param name resource name. * @return the first stage of the new CustomAssessmentAutomation definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationsListResult.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationsListResult.java index 12e9233976c19..f2817eb84c43e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationsListResult.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomAssessmentAutomationsListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of Custom Assessment Automations. */ +/** + * A list of Custom Assessment Automations. + */ @Fluent public final class CustomAssessmentAutomationsListResult { /* @@ -24,13 +26,15 @@ public final class CustomAssessmentAutomationsListResult { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of CustomAssessmentAutomationsListResult class. */ + /** + * Creates an instance of CustomAssessmentAutomationsListResult class. + */ public CustomAssessmentAutomationsListResult() { } /** * Get the value property: Collection of Custom Assessment Automations. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The link used to get the next page of operations. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Set the nextLink property: The link used to get the next page of operations. - * + * * @param nextLink the nextLink value to set. * @return the CustomAssessmentAutomationsListResult object itself. */ @@ -59,7 +63,7 @@ public CustomAssessmentAutomationsListResult withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignment.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignment.java index fcdf9e4547fec..0ddee9631222c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignment.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignment.java @@ -8,32 +8,34 @@ import com.azure.core.util.Context; import com.azure.resourcemanager.security.fluent.models.CustomEntityStoreAssignmentInner; -/** An immutable client-side representation of CustomEntityStoreAssignment. */ +/** + * An immutable client-side representation of CustomEntityStoreAssignment. + */ public interface CustomEntityStoreAssignment { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the systemData property: Azure Resource Manager metadata containing createdBy and modifiedBy information. - * + * * @return the systemData value. */ SystemData systemData(); @@ -41,45 +43,56 @@ public interface CustomEntityStoreAssignment { /** * Gets the principal property: The principal assigned with entity store. Format of principal is: [AAD * type]=[PrincipalObjectId];[TenantId]. - * + * * @return the principal value. */ String principal(); /** * Gets the entityStoreDatabaseLink property: The link to entity store database. - * + * * @return the entityStoreDatabaseLink value. */ String entityStoreDatabaseLink(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.CustomEntityStoreAssignmentInner object. - * + * * @return the inner object. */ CustomEntityStoreAssignmentInner innerModel(); - /** The entirety of the CustomEntityStoreAssignment definition. */ + /** + * The entirety of the CustomEntityStoreAssignment definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithResourceGroup, DefinitionStages.WithCreate { } - /** The CustomEntityStoreAssignment definition stages. */ + + /** + * The CustomEntityStoreAssignment definition stages. + */ interface DefinitionStages { - /** The first stage of the CustomEntityStoreAssignment definition. */ + /** + * The first stage of the CustomEntityStoreAssignment definition. + */ interface Blank extends WithResourceGroup { } - /** The stage of the CustomEntityStoreAssignment definition allowing to specify parent resource. */ + + /** + * The stage of the CustomEntityStoreAssignment definition allowing to specify parent resource. + */ interface WithResourceGroup { /** * Specifies resourceGroupName. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @return the next definition stage. */ WithCreate withExistingResourceGroup(String resourceGroupName); } + /** * The stage of the CustomEntityStoreAssignment definition which contains all the minimum required properties * for the resource to be created, but also allows for any other optional properties to be specified. @@ -87,42 +100,46 @@ interface WithResourceGroup { interface WithCreate extends DefinitionStages.WithPrincipal { /** * Executes the create request. - * + * * @return the created resource. */ CustomEntityStoreAssignment create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ CustomEntityStoreAssignment create(Context context); } - /** The stage of the CustomEntityStoreAssignment definition allowing to specify principal. */ + + /** + * The stage of the CustomEntityStoreAssignment definition allowing to specify principal. + */ interface WithPrincipal { /** * Specifies the principal property: The principal assigned with entity store. If not provided, will use * caller principal. Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. - * + * * @param principal The principal assigned with entity store. If not provided, will use caller principal. - * Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. + * Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. * @return the next definition stage. */ WithCreate withPrincipal(String principal); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ CustomEntityStoreAssignment refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentRequest.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentRequest.java index 915238a0ec4a3..29b7242af9899 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentRequest.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentRequest.java @@ -8,7 +8,9 @@ import com.azure.resourcemanager.security.fluent.models.CustomEntityStoreAssignmentRequestProperties; import com.fasterxml.jackson.annotation.JsonProperty; -/** describes the custom entity store assignment request. */ +/** + * describes the custom entity store assignment request. + */ @Fluent public final class CustomEntityStoreAssignmentRequest { /* @@ -17,13 +19,15 @@ public final class CustomEntityStoreAssignmentRequest { @JsonProperty(value = "properties") private CustomEntityStoreAssignmentRequestProperties innerProperties; - /** Creates an instance of CustomEntityStoreAssignmentRequest class. */ + /** + * Creates an instance of CustomEntityStoreAssignmentRequest class. + */ public CustomEntityStoreAssignmentRequest() { } /** * Get the innerProperties property: describes properties of custom entity store assignment request. - * + * * @return the innerProperties value. */ private CustomEntityStoreAssignmentRequestProperties innerProperties() { @@ -31,9 +35,9 @@ private CustomEntityStoreAssignmentRequestProperties innerProperties() { } /** - * Get the principal property: The principal assigned with entity store. If not provided, will use caller principal. - * Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. - * + * Get the principal property: The principal assigned with entity store. If not provided, will use caller + * principal. Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. + * * @return the principal value. */ public String principal() { @@ -41,9 +45,9 @@ public String principal() { } /** - * Set the principal property: The principal assigned with entity store. If not provided, will use caller principal. - * Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. - * + * Set the principal property: The principal assigned with entity store. If not provided, will use caller + * principal. Format of principal is: [AAD type]=[PrincipalObjectId];[TenantId]. + * * @param principal the principal value to set. * @return the CustomEntityStoreAssignmentRequest object itself. */ @@ -57,7 +61,7 @@ public CustomEntityStoreAssignmentRequest withPrincipal(String principal) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignments.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignments.java index 6267ac669e8e0..5293562f0ddad 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignments.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignments.java @@ -8,33 +8,35 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of CustomEntityStoreAssignments. */ +/** + * Resource collection API of CustomEntityStoreAssignments. + */ public interface CustomEntityStoreAssignments { /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response}. + * with {@link Response}. */ - Response getByResourceGroupWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context); + Response getByResourceGroupWithResponse(String resourceGroupName, + String customEntityStoreAssignmentName, Context context); /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -45,11 +47,11 @@ Response getByResourceGroupWithResponse( /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -57,16 +59,16 @@ Response getByResourceGroupWithResponse( * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteByResourceGroupWithResponse( - String resourceGroupName, String customEntityStoreAssignmentName, Context context); + Response deleteByResourceGroupWithResponse(String resourceGroupName, String customEntityStoreAssignmentName, + Context context); /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param customEntityStoreAssignmentName Name of the custom entity store assignment. Generated name is GUID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -76,11 +78,11 @@ Response deleteByResourceGroupWithResponse( /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -90,11 +92,11 @@ Response deleteByResourceGroupWithResponse( /** * List custom entity store assignments in a subscription and a resource group - * - *

List custom entity store assignments by a provided subscription and resource group. - * + * + * List custom entity store assignments by a provided subscription and resource group. + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -105,9 +107,9 @@ Response deleteByResourceGroupWithResponse( /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of custom entity store assignments as paginated response with {@link PagedIterable}. @@ -116,9 +118,9 @@ Response deleteByResourceGroupWithResponse( /** * List custom entity store assignments in a subscription - * - *

List custom entity store assignments by provided subscription. - * + * + * List custom entity store assignments by provided subscription. + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -129,38 +131,38 @@ Response deleteByResourceGroupWithResponse( /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response}. + * with {@link Response}. */ CustomEntityStoreAssignment getById(String id); /** * Gets a custom entity store assignment - * - *

Gets a single custom entity store assignment by name for the provided subscription and resource group. - * + * + * Gets a single custom entity store assignment by name for the provided subscription and resource group. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a single custom entity store assignment by name for the provided subscription and resource group along - * with {@link Response}. + * with {@link Response}. */ Response getByIdWithResponse(String id, Context context); /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -170,9 +172,9 @@ Response deleteByResourceGroupWithResponse( /** * Deleted a custom entity store assignment - * - *

Delete a custom entity store assignment by name for a provided subscription. - * + * + * Delete a custom entity store assignment by name for a provided subscription. + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -184,7 +186,7 @@ Response deleteByResourceGroupWithResponse( /** * Begins definition for a new CustomEntityStoreAssignment resource. - * + * * @param name resource name. * @return the first stage of the new CustomEntityStoreAssignment definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentsListResult.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentsListResult.java index 72fd9d924f5f2..6f78441d7b681 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentsListResult.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/CustomEntityStoreAssignmentsListResult.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** A list of custom entity store assignments. */ +/** + * A list of custom entity store assignments. + */ @Fluent public final class CustomEntityStoreAssignmentsListResult { /* @@ -24,13 +26,15 @@ public final class CustomEntityStoreAssignmentsListResult { @JsonProperty(value = "nextLink") private String nextLink; - /** Creates an instance of CustomEntityStoreAssignmentsListResult class. */ + /** + * Creates an instance of CustomEntityStoreAssignmentsListResult class. + */ public CustomEntityStoreAssignmentsListResult() { } /** * Get the value property: Collection of custom entity store assignments. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Get the nextLink property: The link used to get the next page of operations. - * + * * @return the nextLink value. */ public String nextLink() { @@ -48,7 +52,7 @@ public String nextLink() { /** * Set the nextLink property: The link used to get the next page of operations. - * + * * @param nextLink the nextLink value to set. * @return the CustomEntityStoreAssignmentsListResult object itself. */ @@ -59,7 +63,7 @@ public CustomEntityStoreAssignmentsListResult withNextLink(String nextLink) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cve.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cve.java index d7ce9a9681b1f..b3b119b353b81 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cve.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cve.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** CVE details. */ +/** + * CVE details. + */ @Immutable public final class Cve { /* @@ -22,13 +24,15 @@ public final class Cve { @JsonProperty(value = "link", access = JsonProperty.Access.WRITE_ONLY) private String link; - /** Creates an instance of Cve class. */ + /** + * Creates an instance of Cve class. + */ public Cve() { } /** * Get the title property: CVE title. - * + * * @return the title value. */ public String title() { @@ -37,7 +41,7 @@ public String title() { /** * Get the link property: Link url. - * + * * @return the link value. */ public String link() { @@ -46,7 +50,7 @@ public String link() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cvss.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cvss.java index 2e1f3a22d5347..585b172d61a7b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cvss.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Cvss.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Immutable; import com.fasterxml.jackson.annotation.JsonProperty; -/** CVSS details. */ +/** + * CVSS details. + */ @Immutable public final class Cvss { /* @@ -16,13 +18,15 @@ public final class Cvss { @JsonProperty(value = "base", access = JsonProperty.Access.WRITE_ONLY) private Float base; - /** Creates an instance of Cvss class. */ + /** + * Creates an instance of Cvss class. + */ public Cvss() { } /** * Get the base property: CVSS base. - * + * * @return the base value. */ public Float base() { @@ -31,7 +35,7 @@ public Float base() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataExportSettings.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataExportSettings.java index ed4306d488793..45e905b3f2367 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataExportSettings.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataExportSettings.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** Represents a data export setting. */ +/** + * Represents a data export setting. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "kind") @JsonTypeName("DataExportSettings") @Fluent @@ -22,13 +24,15 @@ public final class DataExportSettings extends SettingInner { @JsonProperty(value = "properties") private DataExportSettingProperties innerProperties; - /** Creates an instance of DataExportSettings class. */ + /** + * Creates an instance of DataExportSettings class. + */ public DataExportSettings() { } /** * Get the innerProperties property: Data export setting data. - * + * * @return the innerProperties value. */ private DataExportSettingProperties innerProperties() { @@ -37,7 +41,7 @@ private DataExportSettingProperties innerProperties() { /** * Get the enabled property: Is the data export setting enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -46,7 +50,7 @@ public Boolean enabled() { /** * Set the enabled property: Is the data export setting enabled. - * + * * @param enabled the enabled value to set. * @return the DataExportSettings object itself. */ @@ -60,7 +64,7 @@ public DataExportSettings withEnabled(Boolean enabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataSource.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataSource.java index 8327adbc4f490..bb31fb4acbcba 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataSource.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DataSource.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for DataSource. */ +/** + * Defines values for DataSource. + */ public final class DataSource extends ExpandableStringEnum { - /** Static value TwinData for DataSource. */ + /** + * Static value TwinData for DataSource. + */ public static final DataSource TWIN_DATA = fromString("TwinData"); /** * Creates a new instance of DataSource value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public DataSource() { /** * Creates or finds a DataSource from its string representation. - * + * * @param name a name to look for. * @return the corresponding DataSource. */ @@ -35,7 +39,7 @@ public static DataSource fromString(String name) { /** * Gets known DataSource values. - * + * * @return known DataSource values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOffering.java index 0d4bb91cd0c71..431d129663d9e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The CSPM P1 for AWS offering. */ +/** + * The CSPM P1 for AWS offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderCspmAws") @Fluent @@ -32,13 +34,33 @@ public final class DefenderCspmAwsOffering extends CloudOffering { @JsonProperty(value = "databasesDspm") private DefenderCspmAwsOfferingDatabasesDspm databasesDspm; - /** Creates an instance of DefenderCspmAwsOffering class. */ + /* + * Defenders CSPM Cloud infrastructure entitlement management (CIEM) offering configurations + */ + @JsonProperty(value = "ciem") + private DefenderCspmAwsOfferingCiem ciem; + + /* + * The Microsoft Defender container image assessment configuration + */ + @JsonProperty(value = "mdcContainersImageAssessment") + private DefenderCspmAwsOfferingMdcContainersImageAssessment mdcContainersImageAssessment; + + /* + * The Microsoft Defender container agentless discovery K8s configuration + */ + @JsonProperty(value = "mdcContainersAgentlessDiscoveryK8s") + private DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S; + + /** + * Creates an instance of DefenderCspmAwsOffering class. + */ public DefenderCspmAwsOffering() { } /** * Get the vmScanners property: The Microsoft Defender for Server VM scanning configuration. - * + * * @return the vmScanners value. */ public DefenderCspmAwsOfferingVmScanners vmScanners() { @@ -47,7 +69,7 @@ public DefenderCspmAwsOfferingVmScanners vmScanners() { /** * Set the vmScanners property: The Microsoft Defender for Server VM scanning configuration. - * + * * @param vmScanners the vmScanners value to set. * @return the DefenderCspmAwsOffering object itself. */ @@ -58,7 +80,7 @@ public DefenderCspmAwsOffering withVmScanners(DefenderCspmAwsOfferingVmScanners /** * Get the dataSensitivityDiscovery property: The Microsoft Defender Data Sensitivity discovery configuration. - * + * * @return the dataSensitivityDiscovery value. */ public DefenderCspmAwsOfferingDataSensitivityDiscovery dataSensitivityDiscovery() { @@ -67,19 +89,19 @@ public DefenderCspmAwsOfferingDataSensitivityDiscovery dataSensitivityDiscovery( /** * Set the dataSensitivityDiscovery property: The Microsoft Defender Data Sensitivity discovery configuration. - * + * * @param dataSensitivityDiscovery the dataSensitivityDiscovery value to set. * @return the DefenderCspmAwsOffering object itself. */ - public DefenderCspmAwsOffering withDataSensitivityDiscovery( - DefenderCspmAwsOfferingDataSensitivityDiscovery dataSensitivityDiscovery) { + public DefenderCspmAwsOffering + withDataSensitivityDiscovery(DefenderCspmAwsOfferingDataSensitivityDiscovery dataSensitivityDiscovery) { this.dataSensitivityDiscovery = dataSensitivityDiscovery; return this; } /** * Get the databasesDspm property: The databases DSPM configuration. - * + * * @return the databasesDspm value. */ public DefenderCspmAwsOfferingDatabasesDspm databasesDspm() { @@ -88,7 +110,7 @@ public DefenderCspmAwsOfferingDatabasesDspm databasesDspm() { /** * Set the databasesDspm property: The databases DSPM configuration. - * + * * @param databasesDspm the databasesDspm value to set. * @return the DefenderCspmAwsOffering object itself. */ @@ -97,9 +119,75 @@ public DefenderCspmAwsOffering withDatabasesDspm(DefenderCspmAwsOfferingDatabase return this; } + /** + * Get the ciem property: Defenders CSPM Cloud infrastructure entitlement management (CIEM) offering + * configurations. + * + * @return the ciem value. + */ + public DefenderCspmAwsOfferingCiem ciem() { + return this.ciem; + } + + /** + * Set the ciem property: Defenders CSPM Cloud infrastructure entitlement management (CIEM) offering + * configurations. + * + * @param ciem the ciem value to set. + * @return the DefenderCspmAwsOffering object itself. + */ + public DefenderCspmAwsOffering withCiem(DefenderCspmAwsOfferingCiem ciem) { + this.ciem = ciem; + return this; + } + + /** + * Get the mdcContainersImageAssessment property: The Microsoft Defender container image assessment configuration. + * + * @return the mdcContainersImageAssessment value. + */ + public DefenderCspmAwsOfferingMdcContainersImageAssessment mdcContainersImageAssessment() { + return this.mdcContainersImageAssessment; + } + + /** + * Set the mdcContainersImageAssessment property: The Microsoft Defender container image assessment configuration. + * + * @param mdcContainersImageAssessment the mdcContainersImageAssessment value to set. + * @return the DefenderCspmAwsOffering object itself. + */ + public DefenderCspmAwsOffering withMdcContainersImageAssessment( + DefenderCspmAwsOfferingMdcContainersImageAssessment mdcContainersImageAssessment) { + this.mdcContainersImageAssessment = mdcContainersImageAssessment; + return this; + } + + /** + * Get the mdcContainersAgentlessDiscoveryK8S property: The Microsoft Defender container agentless discovery K8s + * configuration. + * + * @return the mdcContainersAgentlessDiscoveryK8S value. + */ + public DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S() { + return this.mdcContainersAgentlessDiscoveryK8S; + } + + /** + * Set the mdcContainersAgentlessDiscoveryK8S property: The Microsoft Defender container agentless discovery K8s + * configuration. + * + * @param mdcContainersAgentlessDiscoveryK8S the mdcContainersAgentlessDiscoveryK8S value to set. + * @return the DefenderCspmAwsOffering object itself. + */ + public DefenderCspmAwsOffering withMdcContainersAgentlessDiscoveryK8S( + DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S) { + this.mdcContainersAgentlessDiscoveryK8S = mdcContainersAgentlessDiscoveryK8S; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override @@ -114,5 +202,14 @@ public void validate() { if (databasesDspm() != null) { databasesDspm().validate(); } + if (ciem() != null) { + ciem().validate(); + } + if (mdcContainersImageAssessment() != null) { + mdcContainersImageAssessment().validate(); + } + if (mdcContainersAgentlessDiscoveryK8S() != null) { + mdcContainersAgentlessDiscoveryK8S().validate(); + } } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiem.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiem.java new file mode 100644 index 0000000000000..faad3cf060499 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiem.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Defenders CSPM Cloud infrastructure entitlement management (CIEM) offering configurations. + */ +@Fluent +public final class DefenderCspmAwsOfferingCiem { + /* + * Defender CSPM CIEM discovery configuration + */ + @JsonProperty(value = "ciemDiscovery") + private DefenderCspmAwsOfferingCiemDiscovery ciemDiscovery; + + /* + * Defender CSPM CIEM AWS OIDC (open id connect) configuration + */ + @JsonProperty(value = "ciemOidc") + private DefenderCspmAwsOfferingCiemOidc ciemOidc; + + /** + * Creates an instance of DefenderCspmAwsOfferingCiem class. + */ + public DefenderCspmAwsOfferingCiem() { + } + + /** + * Get the ciemDiscovery property: Defender CSPM CIEM discovery configuration. + * + * @return the ciemDiscovery value. + */ + public DefenderCspmAwsOfferingCiemDiscovery ciemDiscovery() { + return this.ciemDiscovery; + } + + /** + * Set the ciemDiscovery property: Defender CSPM CIEM discovery configuration. + * + * @param ciemDiscovery the ciemDiscovery value to set. + * @return the DefenderCspmAwsOfferingCiem object itself. + */ + public DefenderCspmAwsOfferingCiem withCiemDiscovery(DefenderCspmAwsOfferingCiemDiscovery ciemDiscovery) { + this.ciemDiscovery = ciemDiscovery; + return this; + } + + /** + * Get the ciemOidc property: Defender CSPM CIEM AWS OIDC (open id connect) configuration. + * + * @return the ciemOidc value. + */ + public DefenderCspmAwsOfferingCiemOidc ciemOidc() { + return this.ciemOidc; + } + + /** + * Set the ciemOidc property: Defender CSPM CIEM AWS OIDC (open id connect) configuration. + * + * @param ciemOidc the ciemOidc value to set. + * @return the DefenderCspmAwsOfferingCiem object itself. + */ + public DefenderCspmAwsOfferingCiem withCiemOidc(DefenderCspmAwsOfferingCiemOidc ciemOidc) { + this.ciemOidc = ciemOidc; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (ciemDiscovery() != null) { + ciemDiscovery().validate(); + } + if (ciemOidc() != null) { + ciemOidc().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemDiscovery.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemDiscovery.java new file mode 100644 index 0000000000000..f7a6f2eee5cbd --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemDiscovery.java @@ -0,0 +1,54 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Defender CSPM CIEM discovery configuration. + */ +@Fluent +public final class DefenderCspmAwsOfferingCiemDiscovery { + /* + * The cloud role ARN in AWS for CIEM discovery + */ + @JsonProperty(value = "cloudRoleArn") + private String cloudRoleArn; + + /** + * Creates an instance of DefenderCspmAwsOfferingCiemDiscovery class. + */ + public DefenderCspmAwsOfferingCiemDiscovery() { + } + + /** + * Get the cloudRoleArn property: The cloud role ARN in AWS for CIEM discovery. + * + * @return the cloudRoleArn value. + */ + public String cloudRoleArn() { + return this.cloudRoleArn; + } + + /** + * Set the cloudRoleArn property: The cloud role ARN in AWS for CIEM discovery. + * + * @param cloudRoleArn the cloudRoleArn value to set. + * @return the DefenderCspmAwsOfferingCiemDiscovery object itself. + */ + public DefenderCspmAwsOfferingCiemDiscovery withCloudRoleArn(String cloudRoleArn) { + this.cloudRoleArn = cloudRoleArn; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemOidc.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemOidc.java new file mode 100644 index 0000000000000..7d4577dbbb03e --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingCiemOidc.java @@ -0,0 +1,82 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Defender CSPM CIEM AWS OIDC (open id connect) configuration. + */ +@Fluent +public final class DefenderCspmAwsOfferingCiemOidc { + /* + * The cloud role ARN in AWS for CIEM oidc connection + */ + @JsonProperty(value = "cloudRoleArn") + private String cloudRoleArn; + + /* + * the azure active directory app name used of authenticating against AWS + */ + @JsonProperty(value = "azureActiveDirectoryAppName") + private String azureActiveDirectoryAppName; + + /** + * Creates an instance of DefenderCspmAwsOfferingCiemOidc class. + */ + public DefenderCspmAwsOfferingCiemOidc() { + } + + /** + * Get the cloudRoleArn property: The cloud role ARN in AWS for CIEM oidc connection. + * + * @return the cloudRoleArn value. + */ + public String cloudRoleArn() { + return this.cloudRoleArn; + } + + /** + * Set the cloudRoleArn property: The cloud role ARN in AWS for CIEM oidc connection. + * + * @param cloudRoleArn the cloudRoleArn value to set. + * @return the DefenderCspmAwsOfferingCiemOidc object itself. + */ + public DefenderCspmAwsOfferingCiemOidc withCloudRoleArn(String cloudRoleArn) { + this.cloudRoleArn = cloudRoleArn; + return this; + } + + /** + * Get the azureActiveDirectoryAppName property: the azure active directory app name used of authenticating against + * AWS. + * + * @return the azureActiveDirectoryAppName value. + */ + public String azureActiveDirectoryAppName() { + return this.azureActiveDirectoryAppName; + } + + /** + * Set the azureActiveDirectoryAppName property: the azure active directory app name used of authenticating against + * AWS. + * + * @param azureActiveDirectoryAppName the azureActiveDirectoryAppName value to set. + * @return the DefenderCspmAwsOfferingCiemOidc object itself. + */ + public DefenderCspmAwsOfferingCiemOidc withAzureActiveDirectoryAppName(String azureActiveDirectoryAppName) { + this.azureActiveDirectoryAppName = azureActiveDirectoryAppName; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDataSensitivityDiscovery.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDataSensitivityDiscovery.java index 4db399753ec84..45b882cdb39cf 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDataSensitivityDiscovery.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDataSensitivityDiscovery.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Microsoft Defender Data Sensitivity discovery configuration. */ +/** + * The Microsoft Defender Data Sensitivity discovery configuration. + */ @Fluent public final class DefenderCspmAwsOfferingDataSensitivityDiscovery { /* @@ -22,13 +24,15 @@ public final class DefenderCspmAwsOfferingDataSensitivityDiscovery { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderCspmAwsOfferingDataSensitivityDiscovery class. */ + /** + * Creates an instance of DefenderCspmAwsOfferingDataSensitivityDiscovery class. + */ public DefenderCspmAwsOfferingDataSensitivityDiscovery() { } /** * Get the enabled property: Is Microsoft Defender Data Sensitivity discovery enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Microsoft Defender Data Sensitivity discovery enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderCspmAwsOfferingDataSensitivityDiscovery object itself. */ @@ -48,7 +52,7 @@ public DefenderCspmAwsOfferingDataSensitivityDiscovery withEnabled(Boolean enabl /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -57,7 +61,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderCspmAwsOfferingDataSensitivityDiscovery object itself. */ @@ -68,7 +72,7 @@ public DefenderCspmAwsOfferingDataSensitivityDiscovery withCloudRoleArn(String c /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDatabasesDspm.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDatabasesDspm.java index 15821e5d425b7..c202ed2c0d92d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDatabasesDspm.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingDatabasesDspm.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The databases DSPM configuration. */ +/** + * The databases DSPM configuration. + */ @Fluent public final class DefenderCspmAwsOfferingDatabasesDspm { /* @@ -22,13 +24,15 @@ public final class DefenderCspmAwsOfferingDatabasesDspm { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderCspmAwsOfferingDatabasesDspm class. */ + /** + * Creates an instance of DefenderCspmAwsOfferingDatabasesDspm class. + */ public DefenderCspmAwsOfferingDatabasesDspm() { } /** * Get the enabled property: Is databases DSPM protection enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is databases DSPM protection enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderCspmAwsOfferingDatabasesDspm object itself. */ @@ -48,7 +52,7 @@ public DefenderCspmAwsOfferingDatabasesDspm withEnabled(Boolean enabled) { /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -57,7 +61,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderCspmAwsOfferingDatabasesDspm object itself. */ @@ -68,7 +72,7 @@ public DefenderCspmAwsOfferingDatabasesDspm withCloudRoleArn(String cloudRoleArn /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S.java new file mode 100644 index 0000000000000..52fc063bf334f --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Microsoft Defender container agentless discovery K8s configuration. + */ +@Fluent +public final class DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S { + /* + * Is Microsoft Defender container agentless discovery K8s enabled + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The cloud role ARN in AWS for this feature + */ + @JsonProperty(value = "cloudRoleArn") + private String cloudRoleArn; + + /** + * Creates an instance of DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S class. + */ + public DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S() { + } + + /** + * Get the enabled property: Is Microsoft Defender container agentless discovery K8s enabled. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Is Microsoft Defender container agentless discovery K8s enabled. + * + * @param enabled the enabled value to set. + * @return the DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @return the cloudRoleArn value. + */ + public String cloudRoleArn() { + return this.cloudRoleArn; + } + + /** + * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @param cloudRoleArn the cloudRoleArn value to set. + * @return the DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderCspmAwsOfferingMdcContainersAgentlessDiscoveryK8S withCloudRoleArn(String cloudRoleArn) { + this.cloudRoleArn = cloudRoleArn; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersImageAssessment.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersImageAssessment.java new file mode 100644 index 0000000000000..cfa11595913f8 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingMdcContainersImageAssessment.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Microsoft Defender container image assessment configuration. + */ +@Fluent +public final class DefenderCspmAwsOfferingMdcContainersImageAssessment { + /* + * Is Microsoft Defender container image assessment enabled + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The cloud role ARN in AWS for this feature + */ + @JsonProperty(value = "cloudRoleArn") + private String cloudRoleArn; + + /** + * Creates an instance of DefenderCspmAwsOfferingMdcContainersImageAssessment class. + */ + public DefenderCspmAwsOfferingMdcContainersImageAssessment() { + } + + /** + * Get the enabled property: Is Microsoft Defender container image assessment enabled. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Is Microsoft Defender container image assessment enabled. + * + * @param enabled the enabled value to set. + * @return the DefenderCspmAwsOfferingMdcContainersImageAssessment object itself. + */ + public DefenderCspmAwsOfferingMdcContainersImageAssessment withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @return the cloudRoleArn value. + */ + public String cloudRoleArn() { + return this.cloudRoleArn; + } + + /** + * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @param cloudRoleArn the cloudRoleArn value to set. + * @return the DefenderCspmAwsOfferingMdcContainersImageAssessment object itself. + */ + public DefenderCspmAwsOfferingMdcContainersImageAssessment withCloudRoleArn(String cloudRoleArn) { + this.cloudRoleArn = cloudRoleArn; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScanners.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScanners.java index 8854c0967acf4..d0b1864ad5ee4 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScanners.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScanners.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Microsoft Defender for Server VM scanning configuration. */ +/** + * The Microsoft Defender for Server VM scanning configuration. + */ @Fluent public final class DefenderCspmAwsOfferingVmScanners { /* @@ -22,13 +24,15 @@ public final class DefenderCspmAwsOfferingVmScanners { @JsonProperty(value = "configuration") private DefenderCspmAwsOfferingVmScannersConfiguration configuration; - /** Creates an instance of DefenderCspmAwsOfferingVmScanners class. */ + /** + * Creates an instance of DefenderCspmAwsOfferingVmScanners class. + */ public DefenderCspmAwsOfferingVmScanners() { } /** * Get the enabled property: Is Microsoft Defender for Server VM scanning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Microsoft Defender for Server VM scanning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderCspmAwsOfferingVmScanners object itself. */ @@ -48,7 +52,7 @@ public DefenderCspmAwsOfferingVmScanners withEnabled(Boolean enabled) { /** * Get the configuration property: configuration for Microsoft Defender for Server VM scanning. - * + * * @return the configuration value. */ public DefenderCspmAwsOfferingVmScannersConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderCspmAwsOfferingVmScannersConfiguration configuration() { /** * Set the configuration property: configuration for Microsoft Defender for Server VM scanning. - * + * * @param configuration the configuration value to set. * @return the DefenderCspmAwsOfferingVmScanners object itself. */ - public DefenderCspmAwsOfferingVmScanners withConfiguration( - DefenderCspmAwsOfferingVmScannersConfiguration configuration) { + public DefenderCspmAwsOfferingVmScanners + withConfiguration(DefenderCspmAwsOfferingVmScannersConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScannersConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScannersConfiguration.java index e70b7f13e8c28..181e39d30344b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScannersConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderCspmAwsOfferingVmScannersConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** configuration for Microsoft Defender for Server VM scanning. */ +/** + * configuration for Microsoft Defender for Server VM scanning. + */ @Fluent public final class DefenderCspmAwsOfferingVmScannersConfiguration { /* @@ -31,13 +33,15 @@ public final class DefenderCspmAwsOfferingVmScannersConfiguration { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map exclusionTags; - /** Creates an instance of DefenderCspmAwsOfferingVmScannersConfiguration class. */ + /** + * Creates an instance of DefenderCspmAwsOfferingVmScannersConfiguration class. + */ public DefenderCspmAwsOfferingVmScannersConfiguration() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -46,7 +50,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderCspmAwsOfferingVmScannersConfiguration object itself. */ @@ -57,7 +61,7 @@ public DefenderCspmAwsOfferingVmScannersConfiguration withCloudRoleArn(String cl /** * Get the scanningMode property: The scanning mode for the VM scan. - * + * * @return the scanningMode value. */ public ScanningMode scanningMode() { @@ -66,7 +70,7 @@ public ScanningMode scanningMode() { /** * Set the scanningMode property: The scanning mode for the VM scan. - * + * * @param scanningMode the scanningMode value to set. * @return the DefenderCspmAwsOfferingVmScannersConfiguration object itself. */ @@ -77,7 +81,7 @@ public DefenderCspmAwsOfferingVmScannersConfiguration withScanningMode(ScanningM /** * Get the exclusionTags property: VM tags that indicates that VM should not be scanned. - * + * * @return the exclusionTags value. */ public Map exclusionTags() { @@ -86,7 +90,7 @@ public Map exclusionTags() { /** * Set the exclusionTags property: VM tags that indicates that VM should not be scanned. - * + * * @param exclusionTags the exclusionTags value to set. * @return the DefenderCspmAwsOfferingVmScannersConfiguration object itself. */ @@ -97,7 +101,7 @@ public DefenderCspmAwsOfferingVmScannersConfiguration withExclusionTags(Map exclusionTags; + + /** + * Creates an instance of DefenderCspmGcpOfferingVmScannersConfiguration class. + */ + public DefenderCspmGcpOfferingVmScannersConfiguration() { + } + + /** + * Get the scanningMode property: The scanning mode for the VM scan. + * + * @return the scanningMode value. + */ + public ScanningMode scanningMode() { + return this.scanningMode; + } + + /** + * Set the scanningMode property: The scanning mode for the VM scan. + * + * @param scanningMode the scanningMode value to set. + * @return the DefenderCspmGcpOfferingVmScannersConfiguration object itself. + */ + public DefenderCspmGcpOfferingVmScannersConfiguration withScanningMode(ScanningMode scanningMode) { + this.scanningMode = scanningMode; + return this; + } + + /** + * Get the exclusionTags property: VM tags that indicates that VM should not be scanned. + * + * @return the exclusionTags value. + */ + public Map exclusionTags() { + return this.exclusionTags; + } + + /** + * Set the exclusionTags property: VM tags that indicates that VM should not be scanned. + * + * @param exclusionTags the exclusionTags value to set. + * @return the DefenderCspmGcpOfferingVmScannersConfiguration object itself. + */ + public DefenderCspmGcpOfferingVmScannersConfiguration withExclusionTags(Map exclusionTags) { + this.exclusionTags = exclusionTags; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOffering.java index d95d5ca67fb67..d17ff6a867bab 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for Databases AWS offering. */ +/** + * The Defender for Databases AWS offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForDatabasesAws") @Fluent @@ -32,13 +34,15 @@ public final class DefenderFoDatabasesAwsOffering extends CloudOffering { @JsonProperty(value = "databasesDspm") private DefenderFoDatabasesAwsOfferingDatabasesDspm databasesDspm; - /** Creates an instance of DefenderFoDatabasesAwsOffering class. */ + /** + * Creates an instance of DefenderFoDatabasesAwsOffering class. + */ public DefenderFoDatabasesAwsOffering() { } /** * Get the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @return the arcAutoProvisioning value. */ public DefenderFoDatabasesAwsOfferingArcAutoProvisioning arcAutoProvisioning() { @@ -47,19 +51,19 @@ public DefenderFoDatabasesAwsOfferingArcAutoProvisioning arcAutoProvisioning() { /** * Set the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @param arcAutoProvisioning the arcAutoProvisioning value to set. * @return the DefenderFoDatabasesAwsOffering object itself. */ - public DefenderFoDatabasesAwsOffering withArcAutoProvisioning( - DefenderFoDatabasesAwsOfferingArcAutoProvisioning arcAutoProvisioning) { + public DefenderFoDatabasesAwsOffering + withArcAutoProvisioning(DefenderFoDatabasesAwsOfferingArcAutoProvisioning arcAutoProvisioning) { this.arcAutoProvisioning = arcAutoProvisioning; return this; } /** * Get the rds property: The RDS configuration. - * + * * @return the rds value. */ public DefenderFoDatabasesAwsOfferingRds rds() { @@ -68,7 +72,7 @@ public DefenderFoDatabasesAwsOfferingRds rds() { /** * Set the rds property: The RDS configuration. - * + * * @param rds the rds value to set. * @return the DefenderFoDatabasesAwsOffering object itself. */ @@ -79,7 +83,7 @@ public DefenderFoDatabasesAwsOffering withRds(DefenderFoDatabasesAwsOfferingRds /** * Get the databasesDspm property: The databases data security posture management (DSPM) configuration. - * + * * @return the databasesDspm value. */ public DefenderFoDatabasesAwsOfferingDatabasesDspm databasesDspm() { @@ -88,7 +92,7 @@ public DefenderFoDatabasesAwsOfferingDatabasesDspm databasesDspm() { /** * Set the databasesDspm property: The databases data security posture management (DSPM) configuration. - * + * * @param databasesDspm the databasesDspm value to set. * @return the DefenderFoDatabasesAwsOffering object itself. */ @@ -99,7 +103,7 @@ public DefenderFoDatabasesAwsOffering withDatabasesDspm(DefenderFoDatabasesAwsOf /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioning.java index b08183d5e9f38..be528f28a6e6b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The ARC autoprovisioning configuration. */ +/** + * The ARC autoprovisioning configuration. + */ @Fluent public final class DefenderFoDatabasesAwsOfferingArcAutoProvisioning { /* @@ -28,13 +30,15 @@ public final class DefenderFoDatabasesAwsOfferingArcAutoProvisioning { @JsonProperty(value = "configuration") private DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration configuration; - /** Creates an instance of DefenderFoDatabasesAwsOfferingArcAutoProvisioning class. */ + /** + * Creates an instance of DefenderFoDatabasesAwsOfferingArcAutoProvisioning class. + */ public DefenderFoDatabasesAwsOfferingArcAutoProvisioning() { } /** * Get the enabled property: Is arc auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -43,7 +47,7 @@ public Boolean enabled() { /** * Set the enabled property: Is arc auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderFoDatabasesAwsOfferingArcAutoProvisioning object itself. */ @@ -54,7 +58,7 @@ public DefenderFoDatabasesAwsOfferingArcAutoProvisioning withEnabled(Boolean ena /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -63,7 +67,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderFoDatabasesAwsOfferingArcAutoProvisioning object itself. */ @@ -74,7 +78,7 @@ public DefenderFoDatabasesAwsOfferingArcAutoProvisioning withCloudRoleArn(String /** * Get the configuration property: Configuration for servers Arc auto provisioning. - * + * * @return the configuration value. */ public DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration configuration() { @@ -83,19 +87,19 @@ public DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration configurat /** * Set the configuration property: Configuration for servers Arc auto provisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderFoDatabasesAwsOfferingArcAutoProvisioning object itself. */ - public DefenderFoDatabasesAwsOfferingArcAutoProvisioning withConfiguration( - DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration configuration) { + public DefenderFoDatabasesAwsOfferingArcAutoProvisioning + withConfiguration(DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration.java index a4796bf24775d..6e7ca9c54548e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration for servers Arc auto provisioning. */ +/** + * Configuration for servers Arc auto provisioning. + */ @Fluent public final class DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration { /* @@ -22,13 +24,15 @@ public final class DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguratio @JsonProperty(value = "privateLinkScope") private String privateLinkScope; - /** Creates an instance of DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration class. */ + /** + * Creates an instance of DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration class. + */ public DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration() { } /** * Get the proxy property: Optional http proxy endpoint to use for the Arc agent. - * + * * @return the proxy value. */ public String proxy() { @@ -37,7 +41,7 @@ public String proxy() { /** * Set the proxy property: Optional http proxy endpoint to use for the Arc agent. - * + * * @param proxy the proxy value to set. * @return the DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration object itself. */ @@ -48,7 +52,7 @@ public DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration withProxy( /** * Get the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @return the privateLinkScope value. */ public String privateLinkScope() { @@ -57,19 +61,19 @@ public String privateLinkScope() { /** * Set the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @param privateLinkScope the privateLinkScope value to set. * @return the DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration object itself. */ - public DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration withPrivateLinkScope( - String privateLinkScope) { + public DefenderFoDatabasesAwsOfferingArcAutoProvisioningConfiguration + withPrivateLinkScope(String privateLinkScope) { this.privateLinkScope = privateLinkScope; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingDatabasesDspm.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingDatabasesDspm.java index f4e4b88c87353..e6fe9024911ed 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingDatabasesDspm.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingDatabasesDspm.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The databases data security posture management (DSPM) configuration. */ +/** + * The databases data security posture management (DSPM) configuration. + */ @Fluent public final class DefenderFoDatabasesAwsOfferingDatabasesDspm { /* @@ -22,13 +24,15 @@ public final class DefenderFoDatabasesAwsOfferingDatabasesDspm { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderFoDatabasesAwsOfferingDatabasesDspm class. */ + /** + * Creates an instance of DefenderFoDatabasesAwsOfferingDatabasesDspm class. + */ public DefenderFoDatabasesAwsOfferingDatabasesDspm() { } /** * Get the enabled property: Is databases data security posture management (DSPM) protection enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is databases data security posture management (DSPM) protection enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderFoDatabasesAwsOfferingDatabasesDspm object itself. */ @@ -48,7 +52,7 @@ public DefenderFoDatabasesAwsOfferingDatabasesDspm withEnabled(Boolean enabled) /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -57,7 +61,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderFoDatabasesAwsOfferingDatabasesDspm object itself. */ @@ -68,7 +72,7 @@ public DefenderFoDatabasesAwsOfferingDatabasesDspm withCloudRoleArn(String cloud /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingRds.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingRds.java index 73c9f6277ef59..ac0c430af8240 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingRds.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderFoDatabasesAwsOfferingRds.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The RDS configuration. */ +/** + * The RDS configuration. + */ @Fluent public final class DefenderFoDatabasesAwsOfferingRds { /* @@ -22,13 +24,15 @@ public final class DefenderFoDatabasesAwsOfferingRds { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderFoDatabasesAwsOfferingRds class. */ + /** + * Creates an instance of DefenderFoDatabasesAwsOfferingRds class. + */ public DefenderFoDatabasesAwsOfferingRds() { } /** * Get the enabled property: Is RDS protection enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is RDS protection enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderFoDatabasesAwsOfferingRds object itself. */ @@ -48,7 +52,7 @@ public DefenderFoDatabasesAwsOfferingRds withEnabled(Boolean enabled) { /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -57,7 +61,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderFoDatabasesAwsOfferingRds object itself. */ @@ -68,7 +72,7 @@ public DefenderFoDatabasesAwsOfferingRds withCloudRoleArn(String cloudRoleArn) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOffering.java index 9877dfd1b9384..e94faaf094a5e 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for Containers AWS offering. */ +/** + * The Defender for Containers AWS offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForContainersAws") @Fluent @@ -74,13 +76,27 @@ public final class DefenderForContainersAwsOffering extends CloudOffering { @JsonProperty(value = "scubaExternalId") private String scubaExternalId; - /** Creates an instance of DefenderForContainersAwsOffering class. */ + /* + * The Microsoft Defender container image assessment configuration + */ + @JsonProperty(value = "mdcContainersImageAssessment") + private DefenderForContainersAwsOfferingMdcContainersImageAssessment mdcContainersImageAssessment; + + /* + * The Microsoft Defender container agentless discovery K8s configuration + */ + @JsonProperty(value = "mdcContainersAgentlessDiscoveryK8s") + private DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S; + + /** + * Creates an instance of DefenderForContainersAwsOffering class. + */ public DefenderForContainersAwsOffering() { } /** * Get the kubernetesService property: The kubernetes service connection configuration. - * + * * @return the kubernetesService value. */ public DefenderForContainersAwsOfferingKubernetesService kubernetesService() { @@ -89,19 +105,19 @@ public DefenderForContainersAwsOfferingKubernetesService kubernetesService() { /** * Set the kubernetesService property: The kubernetes service connection configuration. - * + * * @param kubernetesService the kubernetesService value to set. * @return the DefenderForContainersAwsOffering object itself. */ - public DefenderForContainersAwsOffering withKubernetesService( - DefenderForContainersAwsOfferingKubernetesService kubernetesService) { + public DefenderForContainersAwsOffering + withKubernetesService(DefenderForContainersAwsOfferingKubernetesService kubernetesService) { this.kubernetesService = kubernetesService; return this; } /** * Get the kubernetesScubaReader property: The kubernetes to scuba connection configuration. - * + * * @return the kubernetesScubaReader value. */ public DefenderForContainersAwsOfferingKubernetesScubaReader kubernetesScubaReader() { @@ -110,19 +126,19 @@ public DefenderForContainersAwsOfferingKubernetesScubaReader kubernetesScubaRead /** * Set the kubernetesScubaReader property: The kubernetes to scuba connection configuration. - * + * * @param kubernetesScubaReader the kubernetesScubaReader value to set. * @return the DefenderForContainersAwsOffering object itself. */ - public DefenderForContainersAwsOffering withKubernetesScubaReader( - DefenderForContainersAwsOfferingKubernetesScubaReader kubernetesScubaReader) { + public DefenderForContainersAwsOffering + withKubernetesScubaReader(DefenderForContainersAwsOfferingKubernetesScubaReader kubernetesScubaReader) { this.kubernetesScubaReader = kubernetesScubaReader; return this; } /** * Get the cloudWatchToKinesis property: The cloudwatch to kinesis connection configuration. - * + * * @return the cloudWatchToKinesis value. */ public DefenderForContainersAwsOfferingCloudWatchToKinesis cloudWatchToKinesis() { @@ -131,19 +147,19 @@ public DefenderForContainersAwsOfferingCloudWatchToKinesis cloudWatchToKinesis() /** * Set the cloudWatchToKinesis property: The cloudwatch to kinesis connection configuration. - * + * * @param cloudWatchToKinesis the cloudWatchToKinesis value to set. * @return the DefenderForContainersAwsOffering object itself. */ - public DefenderForContainersAwsOffering withCloudWatchToKinesis( - DefenderForContainersAwsOfferingCloudWatchToKinesis cloudWatchToKinesis) { + public DefenderForContainersAwsOffering + withCloudWatchToKinesis(DefenderForContainersAwsOfferingCloudWatchToKinesis cloudWatchToKinesis) { this.cloudWatchToKinesis = cloudWatchToKinesis; return this; } /** * Get the kinesisToS3 property: The kinesis to s3 connection configuration. - * + * * @return the kinesisToS3 value. */ public DefenderForContainersAwsOfferingKinesisToS3 kinesisToS3() { @@ -152,7 +168,7 @@ public DefenderForContainersAwsOfferingKinesisToS3 kinesisToS3() { /** * Set the kinesisToS3 property: The kinesis to s3 connection configuration. - * + * * @param kinesisToS3 the kinesisToS3 value to set. * @return the DefenderForContainersAwsOffering object itself. */ @@ -163,7 +179,7 @@ public DefenderForContainersAwsOffering withKinesisToS3(DefenderForContainersAws /** * Get the containerVulnerabilityAssessment property: The container vulnerability assessment configuration. - * + * * @return the containerVulnerabilityAssessment value. */ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessment containerVulnerabilityAssessment() { @@ -172,7 +188,7 @@ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessment containe /** * Set the containerVulnerabilityAssessment property: The container vulnerability assessment configuration. - * + * * @param containerVulnerabilityAssessment the containerVulnerabilityAssessment value to set. * @return the DefenderForContainersAwsOffering object itself. */ @@ -183,8 +199,9 @@ public DefenderForContainersAwsOffering withContainerVulnerabilityAssessment( } /** - * Get the containerVulnerabilityAssessmentTask property: The container vulnerability assessment task configuration. - * + * Get the containerVulnerabilityAssessmentTask property: The container vulnerability assessment task + * configuration. + * * @return the containerVulnerabilityAssessmentTask value. */ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask containerVulnerabilityAssessmentTask() { @@ -192,8 +209,9 @@ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask cont } /** - * Set the containerVulnerabilityAssessmentTask property: The container vulnerability assessment task configuration. - * + * Set the containerVulnerabilityAssessmentTask property: The container vulnerability assessment task + * configuration. + * * @param containerVulnerabilityAssessmentTask the containerVulnerabilityAssessmentTask value to set. * @return the DefenderForContainersAwsOffering object itself. */ @@ -205,7 +223,7 @@ public DefenderForContainersAwsOffering withContainerVulnerabilityAssessmentTask /** * Get the enableContainerVulnerabilityAssessment property: Enable container vulnerability assessment feature. - * + * * @return the enableContainerVulnerabilityAssessment value. */ public Boolean enableContainerVulnerabilityAssessment() { @@ -214,19 +232,19 @@ public Boolean enableContainerVulnerabilityAssessment() { /** * Set the enableContainerVulnerabilityAssessment property: Enable container vulnerability assessment feature. - * + * * @param enableContainerVulnerabilityAssessment the enableContainerVulnerabilityAssessment value to set. * @return the DefenderForContainersAwsOffering object itself. */ - public DefenderForContainersAwsOffering withEnableContainerVulnerabilityAssessment( - Boolean enableContainerVulnerabilityAssessment) { + public DefenderForContainersAwsOffering + withEnableContainerVulnerabilityAssessment(Boolean enableContainerVulnerabilityAssessment) { this.enableContainerVulnerabilityAssessment = enableContainerVulnerabilityAssessment; return this; } /** * Get the autoProvisioning property: Is audit logs pipeline auto provisioning enabled. - * + * * @return the autoProvisioning value. */ public Boolean autoProvisioning() { @@ -235,7 +253,7 @@ public Boolean autoProvisioning() { /** * Set the autoProvisioning property: Is audit logs pipeline auto provisioning enabled. - * + * * @param autoProvisioning the autoProvisioning value to set. * @return the DefenderForContainersAwsOffering object itself. */ @@ -247,7 +265,7 @@ public DefenderForContainersAwsOffering withAutoProvisioning(Boolean autoProvisi /** * Get the kubeAuditRetentionTime property: The retention time in days of kube audit logs set on the CloudWatch log * group. - * + * * @return the kubeAuditRetentionTime value. */ public Long kubeAuditRetentionTime() { @@ -257,7 +275,7 @@ public Long kubeAuditRetentionTime() { /** * Set the kubeAuditRetentionTime property: The retention time in days of kube audit logs set on the CloudWatch log * group. - * + * * @param kubeAuditRetentionTime the kubeAuditRetentionTime value to set. * @return the DefenderForContainersAwsOffering object itself. */ @@ -268,7 +286,7 @@ public DefenderForContainersAwsOffering withKubeAuditRetentionTime(Long kubeAudi /** * Get the scubaExternalId property: The externalId used by the data reader to prevent the confused deputy attack. - * + * * @return the scubaExternalId value. */ public String scubaExternalId() { @@ -277,7 +295,7 @@ public String scubaExternalId() { /** * Set the scubaExternalId property: The externalId used by the data reader to prevent the confused deputy attack. - * + * * @param scubaExternalId the scubaExternalId value to set. * @return the DefenderForContainersAwsOffering object itself. */ @@ -286,9 +304,53 @@ public DefenderForContainersAwsOffering withScubaExternalId(String scubaExternal return this; } + /** + * Get the mdcContainersImageAssessment property: The Microsoft Defender container image assessment configuration. + * + * @return the mdcContainersImageAssessment value. + */ + public DefenderForContainersAwsOfferingMdcContainersImageAssessment mdcContainersImageAssessment() { + return this.mdcContainersImageAssessment; + } + + /** + * Set the mdcContainersImageAssessment property: The Microsoft Defender container image assessment configuration. + * + * @param mdcContainersImageAssessment the mdcContainersImageAssessment value to set. + * @return the DefenderForContainersAwsOffering object itself. + */ + public DefenderForContainersAwsOffering withMdcContainersImageAssessment( + DefenderForContainersAwsOfferingMdcContainersImageAssessment mdcContainersImageAssessment) { + this.mdcContainersImageAssessment = mdcContainersImageAssessment; + return this; + } + + /** + * Get the mdcContainersAgentlessDiscoveryK8S property: The Microsoft Defender container agentless discovery K8s + * configuration. + * + * @return the mdcContainersAgentlessDiscoveryK8S value. + */ + public DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S() { + return this.mdcContainersAgentlessDiscoveryK8S; + } + + /** + * Set the mdcContainersAgentlessDiscoveryK8S property: The Microsoft Defender container agentless discovery K8s + * configuration. + * + * @param mdcContainersAgentlessDiscoveryK8S the mdcContainersAgentlessDiscoveryK8S value to set. + * @return the DefenderForContainersAwsOffering object itself. + */ + public DefenderForContainersAwsOffering withMdcContainersAgentlessDiscoveryK8S( + DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S) { + this.mdcContainersAgentlessDiscoveryK8S = mdcContainersAgentlessDiscoveryK8S; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override @@ -312,5 +374,11 @@ public void validate() { if (containerVulnerabilityAssessmentTask() != null) { containerVulnerabilityAssessmentTask().validate(); } + if (mdcContainersImageAssessment() != null) { + mdcContainersImageAssessment().validate(); + } + if (mdcContainersAgentlessDiscoveryK8S() != null) { + mdcContainersAgentlessDiscoveryK8S().validate(); + } } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingCloudWatchToKinesis.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingCloudWatchToKinesis.java index d27c5a0afc7bd..59df0dad19dc3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingCloudWatchToKinesis.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingCloudWatchToKinesis.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The cloudwatch to kinesis connection configuration. */ +/** + * The cloudwatch to kinesis connection configuration. + */ @Fluent public final class DefenderForContainersAwsOfferingCloudWatchToKinesis { /* @@ -16,13 +18,15 @@ public final class DefenderForContainersAwsOfferingCloudWatchToKinesis { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForContainersAwsOfferingCloudWatchToKinesis class. */ + /** + * Creates an instance of DefenderForContainersAwsOfferingCloudWatchToKinesis class. + */ public DefenderForContainersAwsOfferingCloudWatchToKinesis() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS used by CloudWatch to transfer data into Kinesis. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS used by CloudWatch to transfer data into Kinesis. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForContainersAwsOfferingCloudWatchToKinesis object itself. */ @@ -42,7 +46,7 @@ public DefenderForContainersAwsOfferingCloudWatchToKinesis withCloudRoleArn(Stri /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessment.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessment.java index a1348cee8522e..4ec0d73678dd3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessment.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessment.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The container vulnerability assessment configuration. */ +/** + * The container vulnerability assessment configuration. + */ @Fluent public final class DefenderForContainersAwsOfferingContainerVulnerabilityAssessment { /* @@ -16,13 +18,15 @@ public final class DefenderForContainersAwsOfferingContainerVulnerabilityAssessm @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForContainersAwsOfferingContainerVulnerabilityAssessment class. */ + /** + * Creates an instance of DefenderForContainersAwsOfferingContainerVulnerabilityAssessment class. + */ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessment() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForContainersAwsOfferingContainerVulnerabilityAssessment object itself. */ @@ -42,7 +46,7 @@ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessment withClou /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask.java index 711d36db19c0a..b1750e8084e00 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The container vulnerability assessment task configuration. */ +/** + * The container vulnerability assessment task configuration. + */ @Fluent public final class DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask { /* @@ -16,13 +18,15 @@ public final class DefenderForContainersAwsOfferingContainerVulnerabilityAssessm @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask class. */ + /** + * Creates an instance of DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask class. + */ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask object itself. */ @@ -42,7 +46,7 @@ public DefenderForContainersAwsOfferingContainerVulnerabilityAssessmentTask with /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKinesisToS3.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKinesisToS3.java index 1dc228f61ddf5..23ab2b9a459d6 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKinesisToS3.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKinesisToS3.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The kinesis to s3 connection configuration. */ +/** + * The kinesis to s3 connection configuration. + */ @Fluent public final class DefenderForContainersAwsOfferingKinesisToS3 { /* @@ -16,13 +18,15 @@ public final class DefenderForContainersAwsOfferingKinesisToS3 { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForContainersAwsOfferingKinesisToS3 class. */ + /** + * Creates an instance of DefenderForContainersAwsOfferingKinesisToS3 class. + */ public DefenderForContainersAwsOfferingKinesisToS3() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS used by Kinesis to transfer data into S3. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS used by Kinesis to transfer data into S3. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForContainersAwsOfferingKinesisToS3 object itself. */ @@ -42,7 +46,7 @@ public DefenderForContainersAwsOfferingKinesisToS3 withCloudRoleArn(String cloud /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesScubaReader.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesScubaReader.java index 13533efccb8ff..669a805c78411 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesScubaReader.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesScubaReader.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The kubernetes to scuba connection configuration. */ +/** + * The kubernetes to scuba connection configuration. + */ @Fluent public final class DefenderForContainersAwsOfferingKubernetesScubaReader { /* @@ -16,13 +18,15 @@ public final class DefenderForContainersAwsOfferingKubernetesScubaReader { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForContainersAwsOfferingKubernetesScubaReader class. */ + /** + * Creates an instance of DefenderForContainersAwsOfferingKubernetesScubaReader class. + */ public DefenderForContainersAwsOfferingKubernetesScubaReader() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature used for reading data. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature used for reading data. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForContainersAwsOfferingKubernetesScubaReader object itself. */ @@ -42,7 +46,7 @@ public DefenderForContainersAwsOfferingKubernetesScubaReader withCloudRoleArn(St /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesService.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesService.java index 391c18cb98e0e..3639cd51c2631 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesService.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingKubernetesService.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The kubernetes service connection configuration. */ +/** + * The kubernetes service connection configuration. + */ @Fluent public final class DefenderForContainersAwsOfferingKubernetesService { /* @@ -16,13 +18,15 @@ public final class DefenderForContainersAwsOfferingKubernetesService { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForContainersAwsOfferingKubernetesService class. */ + /** + * Creates an instance of DefenderForContainersAwsOfferingKubernetesService class. + */ public DefenderForContainersAwsOfferingKubernetesService() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature used for provisioning resources. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature used for provisioning resources. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForContainersAwsOfferingKubernetesService object itself. */ @@ -42,7 +46,7 @@ public DefenderForContainersAwsOfferingKubernetesService withCloudRoleArn(String /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S.java new file mode 100644 index 0000000000000..e442242e6ea0c --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Microsoft Defender container agentless discovery K8s configuration. + */ +@Fluent +public final class DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S { + /* + * Is Microsoft Defender container agentless discovery K8s enabled + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The cloud role ARN in AWS for this feature + */ + @JsonProperty(value = "cloudRoleArn") + private String cloudRoleArn; + + /** + * Creates an instance of DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S class. + */ + public DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S() { + } + + /** + * Get the enabled property: Is Microsoft Defender container agentless discovery K8s enabled. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Is Microsoft Defender container agentless discovery K8s enabled. + * + * @param enabled the enabled value to set. + * @return the DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @return the cloudRoleArn value. + */ + public String cloudRoleArn() { + return this.cloudRoleArn; + } + + /** + * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @param cloudRoleArn the cloudRoleArn value to set. + * @return the DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderForContainersAwsOfferingMdcContainersAgentlessDiscoveryK8S withCloudRoleArn(String cloudRoleArn) { + this.cloudRoleArn = cloudRoleArn; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersImageAssessment.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersImageAssessment.java new file mode 100644 index 0000000000000..f1206de4ecf18 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersAwsOfferingMdcContainersImageAssessment.java @@ -0,0 +1,80 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Microsoft Defender container image assessment configuration. + */ +@Fluent +public final class DefenderForContainersAwsOfferingMdcContainersImageAssessment { + /* + * Is Microsoft Defender container image assessment enabled + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The cloud role ARN in AWS for this feature + */ + @JsonProperty(value = "cloudRoleArn") + private String cloudRoleArn; + + /** + * Creates an instance of DefenderForContainersAwsOfferingMdcContainersImageAssessment class. + */ + public DefenderForContainersAwsOfferingMdcContainersImageAssessment() { + } + + /** + * Get the enabled property: Is Microsoft Defender container image assessment enabled. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Is Microsoft Defender container image assessment enabled. + * + * @param enabled the enabled value to set. + * @return the DefenderForContainersAwsOfferingMdcContainersImageAssessment object itself. + */ + public DefenderForContainersAwsOfferingMdcContainersImageAssessment withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @return the cloudRoleArn value. + */ + public String cloudRoleArn() { + return this.cloudRoleArn; + } + + /** + * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. + * + * @param cloudRoleArn the cloudRoleArn value to set. + * @return the DefenderForContainersAwsOfferingMdcContainersImageAssessment object itself. + */ + public DefenderForContainersAwsOfferingMdcContainersImageAssessment withCloudRoleArn(String cloudRoleArn) { + this.cloudRoleArn = cloudRoleArn; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOffering.java index 4f6379839b469..efe381e51a0ff 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The containers GCP offering. */ +/** + * The containers GCP offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForContainersGcp") @Fluent @@ -44,13 +46,27 @@ public final class DefenderForContainersGcpOffering extends CloudOffering { @JsonProperty(value = "policyAgentAutoProvisioningFlag") private Boolean policyAgentAutoProvisioningFlag; - /** Creates an instance of DefenderForContainersGcpOffering class. */ + /* + * The Microsoft Defender Container image assessment configuration + */ + @JsonProperty(value = "mdcContainersImageAssessment") + private DefenderForContainersGcpOfferingMdcContainersImageAssessment mdcContainersImageAssessment; + + /* + * The Microsoft Defender Container agentless discovery configuration + */ + @JsonProperty(value = "mdcContainersAgentlessDiscoveryK8s") + private DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S; + + /** + * Creates an instance of DefenderForContainersGcpOffering class. + */ public DefenderForContainersGcpOffering() { } /** * Get the nativeCloudConnection property: The native cloud connection configuration. - * + * * @return the nativeCloudConnection value. */ public DefenderForContainersGcpOfferingNativeCloudConnection nativeCloudConnection() { @@ -59,19 +75,19 @@ public DefenderForContainersGcpOfferingNativeCloudConnection nativeCloudConnecti /** * Set the nativeCloudConnection property: The native cloud connection configuration. - * + * * @param nativeCloudConnection the nativeCloudConnection value to set. * @return the DefenderForContainersGcpOffering object itself. */ - public DefenderForContainersGcpOffering withNativeCloudConnection( - DefenderForContainersGcpOfferingNativeCloudConnection nativeCloudConnection) { + public DefenderForContainersGcpOffering + withNativeCloudConnection(DefenderForContainersGcpOfferingNativeCloudConnection nativeCloudConnection) { this.nativeCloudConnection = nativeCloudConnection; return this; } /** * Get the dataPipelineNativeCloudConnection property: The native cloud connection configuration. - * + * * @return the dataPipelineNativeCloudConnection value. */ public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection dataPipelineNativeCloudConnection() { @@ -80,7 +96,7 @@ public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection dataPip /** * Set the dataPipelineNativeCloudConnection property: The native cloud connection configuration. - * + * * @param dataPipelineNativeCloudConnection the dataPipelineNativeCloudConnection value to set. * @return the DefenderForContainersGcpOffering object itself. */ @@ -92,7 +108,7 @@ public DefenderForContainersGcpOffering withDataPipelineNativeCloudConnection( /** * Get the auditLogsAutoProvisioningFlag property: Is audit logs data collection enabled. - * + * * @return the auditLogsAutoProvisioningFlag value. */ public Boolean auditLogsAutoProvisioningFlag() { @@ -101,7 +117,7 @@ public Boolean auditLogsAutoProvisioningFlag() { /** * Set the auditLogsAutoProvisioningFlag property: Is audit logs data collection enabled. - * + * * @param auditLogsAutoProvisioningFlag the auditLogsAutoProvisioningFlag value to set. * @return the DefenderForContainersGcpOffering object itself. */ @@ -113,7 +129,7 @@ public DefenderForContainersGcpOffering withAuditLogsAutoProvisioningFlag(Boolea /** * Get the defenderAgentAutoProvisioningFlag property: Is Microsoft Defender for Cloud Kubernetes agent auto * provisioning enabled. - * + * * @return the defenderAgentAutoProvisioningFlag value. */ public Boolean defenderAgentAutoProvisioningFlag() { @@ -123,19 +139,19 @@ public Boolean defenderAgentAutoProvisioningFlag() { /** * Set the defenderAgentAutoProvisioningFlag property: Is Microsoft Defender for Cloud Kubernetes agent auto * provisioning enabled. - * + * * @param defenderAgentAutoProvisioningFlag the defenderAgentAutoProvisioningFlag value to set. * @return the DefenderForContainersGcpOffering object itself. */ - public DefenderForContainersGcpOffering withDefenderAgentAutoProvisioningFlag( - Boolean defenderAgentAutoProvisioningFlag) { + public DefenderForContainersGcpOffering + withDefenderAgentAutoProvisioningFlag(Boolean defenderAgentAutoProvisioningFlag) { this.defenderAgentAutoProvisioningFlag = defenderAgentAutoProvisioningFlag; return this; } /** * Get the policyAgentAutoProvisioningFlag property: Is Policy Kubernetes agent auto provisioning enabled. - * + * * @return the policyAgentAutoProvisioningFlag value. */ public Boolean policyAgentAutoProvisioningFlag() { @@ -144,19 +160,63 @@ public Boolean policyAgentAutoProvisioningFlag() { /** * Set the policyAgentAutoProvisioningFlag property: Is Policy Kubernetes agent auto provisioning enabled. - * + * * @param policyAgentAutoProvisioningFlag the policyAgentAutoProvisioningFlag value to set. * @return the DefenderForContainersGcpOffering object itself. */ - public DefenderForContainersGcpOffering withPolicyAgentAutoProvisioningFlag( - Boolean policyAgentAutoProvisioningFlag) { + public DefenderForContainersGcpOffering + withPolicyAgentAutoProvisioningFlag(Boolean policyAgentAutoProvisioningFlag) { this.policyAgentAutoProvisioningFlag = policyAgentAutoProvisioningFlag; return this; } + /** + * Get the mdcContainersImageAssessment property: The Microsoft Defender Container image assessment configuration. + * + * @return the mdcContainersImageAssessment value. + */ + public DefenderForContainersGcpOfferingMdcContainersImageAssessment mdcContainersImageAssessment() { + return this.mdcContainersImageAssessment; + } + + /** + * Set the mdcContainersImageAssessment property: The Microsoft Defender Container image assessment configuration. + * + * @param mdcContainersImageAssessment the mdcContainersImageAssessment value to set. + * @return the DefenderForContainersGcpOffering object itself. + */ + public DefenderForContainersGcpOffering withMdcContainersImageAssessment( + DefenderForContainersGcpOfferingMdcContainersImageAssessment mdcContainersImageAssessment) { + this.mdcContainersImageAssessment = mdcContainersImageAssessment; + return this; + } + + /** + * Get the mdcContainersAgentlessDiscoveryK8S property: The Microsoft Defender Container agentless discovery + * configuration. + * + * @return the mdcContainersAgentlessDiscoveryK8S value. + */ + public DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S() { + return this.mdcContainersAgentlessDiscoveryK8S; + } + + /** + * Set the mdcContainersAgentlessDiscoveryK8S property: The Microsoft Defender Container agentless discovery + * configuration. + * + * @param mdcContainersAgentlessDiscoveryK8S the mdcContainersAgentlessDiscoveryK8S value to set. + * @return the DefenderForContainersGcpOffering object itself. + */ + public DefenderForContainersGcpOffering withMdcContainersAgentlessDiscoveryK8S( + DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S mdcContainersAgentlessDiscoveryK8S) { + this.mdcContainersAgentlessDiscoveryK8S = mdcContainersAgentlessDiscoveryK8S; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override @@ -168,5 +228,11 @@ public void validate() { if (dataPipelineNativeCloudConnection() != null) { dataPipelineNativeCloudConnection().validate(); } + if (mdcContainersImageAssessment() != null) { + mdcContainersImageAssessment().validate(); + } + if (mdcContainersAgentlessDiscoveryK8S() != null) { + mdcContainersAgentlessDiscoveryK8S().validate(); + } } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection.java index 263ea9aa2cc4a..be2aadb8c4e32 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The native cloud connection configuration. */ +/** + * The native cloud connection configuration. + */ @Fluent public final class DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection { /* @@ -22,14 +24,16 @@ public final class DefenderForContainersGcpOfferingDataPipelineNativeCloudConnec @JsonProperty(value = "workloadIdentityProviderId") private String workloadIdentityProviderId; - /** Creates an instance of DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection class. */ + /** + * Creates an instance of DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection class. + */ public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection() { } /** * Get the serviceAccountEmailAddress property: The data collection service account email address in GCP for this * offering. - * + * * @return the serviceAccountEmailAddress value. */ public String serviceAccountEmailAddress() { @@ -39,12 +43,12 @@ public String serviceAccountEmailAddress() { /** * Set the serviceAccountEmailAddress property: The data collection service account email address in GCP for this * offering. - * + * * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. * @return the DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection object itself. */ - public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection withServiceAccountEmailAddress( - String serviceAccountEmailAddress) { + public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { this.serviceAccountEmailAddress = serviceAccountEmailAddress; return this; } @@ -52,7 +56,7 @@ public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection withSer /** * Get the workloadIdentityProviderId property: The data collection GCP workload identity provider id for this * offering. - * + * * @return the workloadIdentityProviderId value. */ public String workloadIdentityProviderId() { @@ -62,19 +66,19 @@ public String workloadIdentityProviderId() { /** * Set the workloadIdentityProviderId property: The data collection GCP workload identity provider id for this * offering. - * + * * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. * @return the DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection object itself. */ - public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection withWorkloadIdentityProviderId( - String workloadIdentityProviderId) { + public DefenderForContainersGcpOfferingDataPipelineNativeCloudConnection + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { this.workloadIdentityProviderId = workloadIdentityProviderId; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S.java new file mode 100644 index 0000000000000..826addd4b26e4 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S.java @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Microsoft Defender Container agentless discovery configuration. + */ +@Fluent +public final class DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S { + /* + * Is Microsoft Defender container agentless discovery enabled + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The workload identity provider id in GCP for this feature + */ + @JsonProperty(value = "workloadIdentityProviderId") + private String workloadIdentityProviderId; + + /* + * The service account email address in GCP for this feature + */ + @JsonProperty(value = "serviceAccountEmailAddress") + private String serviceAccountEmailAddress; + + /** + * Creates an instance of DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S class. + */ + public DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S() { + } + + /** + * Get the enabled property: Is Microsoft Defender container agentless discovery enabled. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Is Microsoft Defender container agentless discovery enabled. + * + * @param enabled the enabled value to set. + * @return the DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the workloadIdentityProviderId property: The workload identity provider id in GCP for this feature. + * + * @return the workloadIdentityProviderId value. + */ + public String workloadIdentityProviderId() { + return this.workloadIdentityProviderId; + } + + /** + * Set the workloadIdentityProviderId property: The workload identity provider id in GCP for this feature. + * + * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. + * @return the DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { + this.workloadIdentityProviderId = workloadIdentityProviderId; + return this; + } + + /** + * Get the serviceAccountEmailAddress property: The service account email address in GCP for this feature. + * + * @return the serviceAccountEmailAddress value. + */ + public String serviceAccountEmailAddress() { + return this.serviceAccountEmailAddress; + } + + /** + * Set the serviceAccountEmailAddress property: The service account email address in GCP for this feature. + * + * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. + * @return the DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S object itself. + */ + public DefenderForContainersGcpOfferingMdcContainersAgentlessDiscoveryK8S + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { + this.serviceAccountEmailAddress = serviceAccountEmailAddress; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersImageAssessment.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersImageAssessment.java new file mode 100644 index 0000000000000..7d4321448beab --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingMdcContainersImageAssessment.java @@ -0,0 +1,108 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * The Microsoft Defender Container image assessment configuration. + */ +@Fluent +public final class DefenderForContainersGcpOfferingMdcContainersImageAssessment { + /* + * Is Microsoft Defender container image assessment enabled + */ + @JsonProperty(value = "enabled") + private Boolean enabled; + + /* + * The workload identity provider id in GCP for this feature + */ + @JsonProperty(value = "workloadIdentityProviderId") + private String workloadIdentityProviderId; + + /* + * The service account email address in GCP for this feature + */ + @JsonProperty(value = "serviceAccountEmailAddress") + private String serviceAccountEmailAddress; + + /** + * Creates an instance of DefenderForContainersGcpOfferingMdcContainersImageAssessment class. + */ + public DefenderForContainersGcpOfferingMdcContainersImageAssessment() { + } + + /** + * Get the enabled property: Is Microsoft Defender container image assessment enabled. + * + * @return the enabled value. + */ + public Boolean enabled() { + return this.enabled; + } + + /** + * Set the enabled property: Is Microsoft Defender container image assessment enabled. + * + * @param enabled the enabled value to set. + * @return the DefenderForContainersGcpOfferingMdcContainersImageAssessment object itself. + */ + public DefenderForContainersGcpOfferingMdcContainersImageAssessment withEnabled(Boolean enabled) { + this.enabled = enabled; + return this; + } + + /** + * Get the workloadIdentityProviderId property: The workload identity provider id in GCP for this feature. + * + * @return the workloadIdentityProviderId value. + */ + public String workloadIdentityProviderId() { + return this.workloadIdentityProviderId; + } + + /** + * Set the workloadIdentityProviderId property: The workload identity provider id in GCP for this feature. + * + * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. + * @return the DefenderForContainersGcpOfferingMdcContainersImageAssessment object itself. + */ + public DefenderForContainersGcpOfferingMdcContainersImageAssessment + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { + this.workloadIdentityProviderId = workloadIdentityProviderId; + return this; + } + + /** + * Get the serviceAccountEmailAddress property: The service account email address in GCP for this feature. + * + * @return the serviceAccountEmailAddress value. + */ + public String serviceAccountEmailAddress() { + return this.serviceAccountEmailAddress; + } + + /** + * Set the serviceAccountEmailAddress property: The service account email address in GCP for this feature. + * + * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. + * @return the DefenderForContainersGcpOfferingMdcContainersImageAssessment object itself. + */ + public DefenderForContainersGcpOfferingMdcContainersImageAssessment + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { + this.serviceAccountEmailAddress = serviceAccountEmailAddress; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingNativeCloudConnection.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingNativeCloudConnection.java index bc0088a59a9d4..6cdcebcf95b73 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingNativeCloudConnection.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForContainersGcpOfferingNativeCloudConnection.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The native cloud connection configuration. */ +/** + * The native cloud connection configuration. + */ @Fluent public final class DefenderForContainersGcpOfferingNativeCloudConnection { /* @@ -22,13 +24,15 @@ public final class DefenderForContainersGcpOfferingNativeCloudConnection { @JsonProperty(value = "workloadIdentityProviderId") private String workloadIdentityProviderId; - /** Creates an instance of DefenderForContainersGcpOfferingNativeCloudConnection class. */ + /** + * Creates an instance of DefenderForContainersGcpOfferingNativeCloudConnection class. + */ public DefenderForContainersGcpOfferingNativeCloudConnection() { } /** * Get the serviceAccountEmailAddress property: The service account email address in GCP for this offering. - * + * * @return the serviceAccountEmailAddress value. */ public String serviceAccountEmailAddress() { @@ -37,19 +41,19 @@ public String serviceAccountEmailAddress() { /** * Set the serviceAccountEmailAddress property: The service account email address in GCP for this offering. - * + * * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. * @return the DefenderForContainersGcpOfferingNativeCloudConnection object itself. */ - public DefenderForContainersGcpOfferingNativeCloudConnection withServiceAccountEmailAddress( - String serviceAccountEmailAddress) { + public DefenderForContainersGcpOfferingNativeCloudConnection + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { this.serviceAccountEmailAddress = serviceAccountEmailAddress; return this; } /** * Get the workloadIdentityProviderId property: The GCP workload identity provider id for this offering. - * + * * @return the workloadIdentityProviderId value. */ public String workloadIdentityProviderId() { @@ -58,19 +62,19 @@ public String workloadIdentityProviderId() { /** * Set the workloadIdentityProviderId property: The GCP workload identity provider id for this offering. - * + * * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. * @return the DefenderForContainersGcpOfferingNativeCloudConnection object itself. */ - public DefenderForContainersGcpOfferingNativeCloudConnection withWorkloadIdentityProviderId( - String workloadIdentityProviderId) { + public DefenderForContainersGcpOfferingNativeCloudConnection + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { this.workloadIdentityProviderId = workloadIdentityProviderId; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOffering.java index 0dc9a2ad7509e..516ba4e4b86e3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for Databases GCP offering configurations. */ +/** + * The Defender for Databases GCP offering configurations. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForDatabasesGcp") @Fluent @@ -24,16 +26,17 @@ public final class DefenderForDatabasesGcpOffering extends CloudOffering { * The native cloud connection configuration */ @JsonProperty(value = "defenderForDatabasesArcAutoProvisioning") - private DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning - defenderForDatabasesArcAutoProvisioning; + private DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning defenderForDatabasesArcAutoProvisioning; - /** Creates an instance of DefenderForDatabasesGcpOffering class. */ + /** + * Creates an instance of DefenderForDatabasesGcpOffering class. + */ public DefenderForDatabasesGcpOffering() { } /** * Get the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @return the arcAutoProvisioning value. */ public DefenderForDatabasesGcpOfferingArcAutoProvisioning arcAutoProvisioning() { @@ -42,19 +45,19 @@ public DefenderForDatabasesGcpOfferingArcAutoProvisioning arcAutoProvisioning() /** * Set the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @param arcAutoProvisioning the arcAutoProvisioning value to set. * @return the DefenderForDatabasesGcpOffering object itself. */ - public DefenderForDatabasesGcpOffering withArcAutoProvisioning( - DefenderForDatabasesGcpOfferingArcAutoProvisioning arcAutoProvisioning) { + public DefenderForDatabasesGcpOffering + withArcAutoProvisioning(DefenderForDatabasesGcpOfferingArcAutoProvisioning arcAutoProvisioning) { this.arcAutoProvisioning = arcAutoProvisioning; return this; } /** * Get the defenderForDatabasesArcAutoProvisioning property: The native cloud connection configuration. - * + * * @return the defenderForDatabasesArcAutoProvisioning value. */ public DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning @@ -64,20 +67,19 @@ public DefenderForDatabasesGcpOffering withArcAutoProvisioning( /** * Set the defenderForDatabasesArcAutoProvisioning property: The native cloud connection configuration. - * + * * @param defenderForDatabasesArcAutoProvisioning the defenderForDatabasesArcAutoProvisioning value to set. * @return the DefenderForDatabasesGcpOffering object itself. */ public DefenderForDatabasesGcpOffering withDefenderForDatabasesArcAutoProvisioning( - DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning - defenderForDatabasesArcAutoProvisioning) { + DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning defenderForDatabasesArcAutoProvisioning) { this.defenderForDatabasesArcAutoProvisioning = defenderForDatabasesArcAutoProvisioning; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioning.java index a1af1380b1b12..a530e306c0443 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The ARC autoprovisioning configuration. */ +/** + * The ARC autoprovisioning configuration. + */ @Fluent public final class DefenderForDatabasesGcpOfferingArcAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForDatabasesGcpOfferingArcAutoProvisioning { @JsonProperty(value = "configuration") private DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration configuration; - /** Creates an instance of DefenderForDatabasesGcpOfferingArcAutoProvisioning class. */ + /** + * Creates an instance of DefenderForDatabasesGcpOfferingArcAutoProvisioning class. + */ public DefenderForDatabasesGcpOfferingArcAutoProvisioning() { } /** * Get the enabled property: Is arc auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is arc auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForDatabasesGcpOfferingArcAutoProvisioning object itself. */ @@ -48,7 +52,7 @@ public DefenderForDatabasesGcpOfferingArcAutoProvisioning withEnabled(Boolean en /** * Get the configuration property: Configuration for servers Arc auto provisioning. - * + * * @return the configuration value. */ public DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration configura /** * Set the configuration property: Configuration for servers Arc auto provisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForDatabasesGcpOfferingArcAutoProvisioning object itself. */ - public DefenderForDatabasesGcpOfferingArcAutoProvisioning withConfiguration( - DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration configuration) { + public DefenderForDatabasesGcpOfferingArcAutoProvisioning + withConfiguration(DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration.java index e44b9dc662fec..b9ab82a01006c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration for servers Arc auto provisioning. */ +/** + * Configuration for servers Arc auto provisioning. + */ @Fluent public final class DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration { /* @@ -22,13 +24,15 @@ public final class DefenderForDatabasesGcpOfferingArcAutoProvisioningConfigurati @JsonProperty(value = "privateLinkScope") private String privateLinkScope; - /** Creates an instance of DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration class. */ + /** + * Creates an instance of DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration class. + */ public DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration() { } /** * Get the proxy property: Optional http proxy endpoint to use for the Arc agent. - * + * * @return the proxy value. */ public String proxy() { @@ -37,7 +41,7 @@ public String proxy() { /** * Set the proxy property: Optional http proxy endpoint to use for the Arc agent. - * + * * @param proxy the proxy value to set. * @return the DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration object itself. */ @@ -48,7 +52,7 @@ public DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration withProxy /** * Get the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @return the privateLinkScope value. */ public String privateLinkScope() { @@ -57,19 +61,19 @@ public String privateLinkScope() { /** * Set the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @param privateLinkScope the privateLinkScope value to set. * @return the DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration object itself. */ - public DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration withPrivateLinkScope( - String privateLinkScope) { + public DefenderForDatabasesGcpOfferingArcAutoProvisioningConfiguration + withPrivateLinkScope(String privateLinkScope) { this.privateLinkScope = privateLinkScope; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning.java index 552d3f4dd765a..c5b55c56eb266 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The native cloud connection configuration. */ +/** + * The native cloud connection configuration. + */ @Fluent public final class DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoPro @JsonProperty(value = "workloadIdentityProviderId") private String workloadIdentityProviderId; - /** Creates an instance of DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning class. */ + /** + * Creates an instance of DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning class. + */ public DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning() { } /** * Get the serviceAccountEmailAddress property: The service account email address in GCP for this offering. - * + * * @return the serviceAccountEmailAddress value. */ public String serviceAccountEmailAddress() { @@ -37,19 +41,19 @@ public String serviceAccountEmailAddress() { /** * Set the serviceAccountEmailAddress property: The service account email address in GCP for this offering. - * + * * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. * @return the DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning object itself. */ - public DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning withServiceAccountEmailAddress( - String serviceAccountEmailAddress) { + public DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { this.serviceAccountEmailAddress = serviceAccountEmailAddress; return this; } /** * Get the workloadIdentityProviderId property: The GCP workload identity provider id for this offering. - * + * * @return the workloadIdentityProviderId value. */ public String workloadIdentityProviderId() { @@ -58,19 +62,19 @@ public String workloadIdentityProviderId() { /** * Set the workloadIdentityProviderId property: The GCP workload identity provider id for this offering. - * + * * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. * @return the DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning object itself. */ - public DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning withWorkloadIdentityProviderId( - String workloadIdentityProviderId) { + public DefenderForDatabasesGcpOfferingDefenderForDatabasesArcAutoProvisioning + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { this.workloadIdentityProviderId = workloadIdentityProviderId; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsAzureDevOpsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsAzureDevOpsOffering.java index 841f547633802..baa72bbb62623 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsAzureDevOpsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsAzureDevOpsOffering.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for DevOps for Azure DevOps offering. */ +/** + * The Defender for DevOps for Azure DevOps offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForDevOpsAzureDevOps") @Immutable public final class DefenderForDevOpsAzureDevOpsOffering extends CloudOffering { - /** Creates an instance of DefenderForDevOpsAzureDevOpsOffering class. */ + /** + * Creates an instance of DefenderForDevOpsAzureDevOpsOffering class. + */ public DefenderForDevOpsAzureDevOpsOffering() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGitLabOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGitLabOffering.java index e194b6df990bf..a513bd44073e9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGitLabOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGitLabOffering.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for DevOps for Gitlab offering. */ +/** + * The Defender for DevOps for Gitlab offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForDevOpsGitLab") @Immutable public final class DefenderForDevOpsGitLabOffering extends CloudOffering { - /** Creates an instance of DefenderForDevOpsGitLabOffering class. */ + /** + * Creates an instance of DefenderForDevOpsGitLabOffering class. + */ public DefenderForDevOpsGitLabOffering() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGithubOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGithubOffering.java index 22df7340c130b..5c3790fd9d915 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGithubOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForDevOpsGithubOffering.java @@ -8,18 +8,22 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for DevOps for Github offering. */ +/** + * The Defender for DevOps for Github offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForDevOpsGithub") @Immutable public final class DefenderForDevOpsGithubOffering extends CloudOffering { - /** Creates an instance of DefenderForDevOpsGithubOffering class. */ + /** + * Creates an instance of DefenderForDevOpsGithubOffering class. + */ public DefenderForDevOpsGithubOffering() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOffering.java index e261069b5d20c..5006235df374a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for Servers AWS offering. */ +/** + * The Defender for Servers AWS offering. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForServersAws") @Fluent @@ -50,13 +52,15 @@ public final class DefenderForServersAwsOffering extends CloudOffering { @JsonProperty(value = "vmScanners") private DefenderForServersAwsOfferingVmScanners vmScanners; - /** Creates an instance of DefenderForServersAwsOffering class. */ + /** + * Creates an instance of DefenderForServersAwsOffering class. + */ public DefenderForServersAwsOffering() { } /** * Get the defenderForServers property: The Defender for servers connection configuration. - * + * * @return the defenderForServers value. */ public DefenderForServersAwsOfferingDefenderForServers defenderForServers() { @@ -65,19 +69,19 @@ public DefenderForServersAwsOfferingDefenderForServers defenderForServers() { /** * Set the defenderForServers property: The Defender for servers connection configuration. - * + * * @param defenderForServers the defenderForServers value to set. * @return the DefenderForServersAwsOffering object itself. */ - public DefenderForServersAwsOffering withDefenderForServers( - DefenderForServersAwsOfferingDefenderForServers defenderForServers) { + public DefenderForServersAwsOffering + withDefenderForServers(DefenderForServersAwsOfferingDefenderForServers defenderForServers) { this.defenderForServers = defenderForServers; return this; } /** * Get the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @return the arcAutoProvisioning value. */ public DefenderForServersAwsOfferingArcAutoProvisioning arcAutoProvisioning() { @@ -86,19 +90,19 @@ public DefenderForServersAwsOfferingArcAutoProvisioning arcAutoProvisioning() { /** * Set the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @param arcAutoProvisioning the arcAutoProvisioning value to set. * @return the DefenderForServersAwsOffering object itself. */ - public DefenderForServersAwsOffering withArcAutoProvisioning( - DefenderForServersAwsOfferingArcAutoProvisioning arcAutoProvisioning) { + public DefenderForServersAwsOffering + withArcAutoProvisioning(DefenderForServersAwsOfferingArcAutoProvisioning arcAutoProvisioning) { this.arcAutoProvisioning = arcAutoProvisioning; return this; } /** * Get the vaAutoProvisioning property: The Vulnerability Assessment autoprovisioning configuration. - * + * * @return the vaAutoProvisioning value. */ public DefenderForServersAwsOfferingVaAutoProvisioning vaAutoProvisioning() { @@ -107,19 +111,19 @@ public DefenderForServersAwsOfferingVaAutoProvisioning vaAutoProvisioning() { /** * Set the vaAutoProvisioning property: The Vulnerability Assessment autoprovisioning configuration. - * + * * @param vaAutoProvisioning the vaAutoProvisioning value to set. * @return the DefenderForServersAwsOffering object itself. */ - public DefenderForServersAwsOffering withVaAutoProvisioning( - DefenderForServersAwsOfferingVaAutoProvisioning vaAutoProvisioning) { + public DefenderForServersAwsOffering + withVaAutoProvisioning(DefenderForServersAwsOfferingVaAutoProvisioning vaAutoProvisioning) { this.vaAutoProvisioning = vaAutoProvisioning; return this; } /** * Get the mdeAutoProvisioning property: The Microsoft Defender for Endpoint autoprovisioning configuration. - * + * * @return the mdeAutoProvisioning value. */ public DefenderForServersAwsOfferingMdeAutoProvisioning mdeAutoProvisioning() { @@ -128,19 +132,19 @@ public DefenderForServersAwsOfferingMdeAutoProvisioning mdeAutoProvisioning() { /** * Set the mdeAutoProvisioning property: The Microsoft Defender for Endpoint autoprovisioning configuration. - * + * * @param mdeAutoProvisioning the mdeAutoProvisioning value to set. * @return the DefenderForServersAwsOffering object itself. */ - public DefenderForServersAwsOffering withMdeAutoProvisioning( - DefenderForServersAwsOfferingMdeAutoProvisioning mdeAutoProvisioning) { + public DefenderForServersAwsOffering + withMdeAutoProvisioning(DefenderForServersAwsOfferingMdeAutoProvisioning mdeAutoProvisioning) { this.mdeAutoProvisioning = mdeAutoProvisioning; return this; } /** * Get the subPlan property: configuration for the servers offering subPlan. - * + * * @return the subPlan value. */ public DefenderForServersAwsOfferingSubPlan subPlan() { @@ -149,7 +153,7 @@ public DefenderForServersAwsOfferingSubPlan subPlan() { /** * Set the subPlan property: configuration for the servers offering subPlan. - * + * * @param subPlan the subPlan value to set. * @return the DefenderForServersAwsOffering object itself. */ @@ -160,7 +164,7 @@ public DefenderForServersAwsOffering withSubPlan(DefenderForServersAwsOfferingSu /** * Get the vmScanners property: The Microsoft Defender for Server VM scanning configuration. - * + * * @return the vmScanners value. */ public DefenderForServersAwsOfferingVmScanners vmScanners() { @@ -169,7 +173,7 @@ public DefenderForServersAwsOfferingVmScanners vmScanners() { /** * Set the vmScanners property: The Microsoft Defender for Server VM scanning configuration. - * + * * @param vmScanners the vmScanners value to set. * @return the DefenderForServersAwsOffering object itself. */ @@ -180,7 +184,7 @@ public DefenderForServersAwsOffering withVmScanners(DefenderForServersAwsOfferin /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioning.java index 0d1ff8f1543e5..b05c9ab0d3866 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The ARC autoprovisioning configuration. */ +/** + * The ARC autoprovisioning configuration. + */ @Fluent public final class DefenderForServersAwsOfferingArcAutoProvisioning { /* @@ -28,13 +30,15 @@ public final class DefenderForServersAwsOfferingArcAutoProvisioning { @JsonProperty(value = "configuration") private DefenderForServersAwsOfferingArcAutoProvisioningConfiguration configuration; - /** Creates an instance of DefenderForServersAwsOfferingArcAutoProvisioning class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingArcAutoProvisioning class. + */ public DefenderForServersAwsOfferingArcAutoProvisioning() { } /** * Get the enabled property: Is arc auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -43,7 +47,7 @@ public Boolean enabled() { /** * Set the enabled property: Is arc auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersAwsOfferingArcAutoProvisioning object itself. */ @@ -54,7 +58,7 @@ public DefenderForServersAwsOfferingArcAutoProvisioning withEnabled(Boolean enab /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -63,7 +67,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForServersAwsOfferingArcAutoProvisioning object itself. */ @@ -74,7 +78,7 @@ public DefenderForServersAwsOfferingArcAutoProvisioning withCloudRoleArn(String /** * Get the configuration property: Configuration for servers Arc auto provisioning. - * + * * @return the configuration value. */ public DefenderForServersAwsOfferingArcAutoProvisioningConfiguration configuration() { @@ -83,19 +87,19 @@ public DefenderForServersAwsOfferingArcAutoProvisioningConfiguration configurati /** * Set the configuration property: Configuration for servers Arc auto provisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersAwsOfferingArcAutoProvisioning object itself. */ - public DefenderForServersAwsOfferingArcAutoProvisioning withConfiguration( - DefenderForServersAwsOfferingArcAutoProvisioningConfiguration configuration) { + public DefenderForServersAwsOfferingArcAutoProvisioning + withConfiguration(DefenderForServersAwsOfferingArcAutoProvisioningConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioningConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioningConfiguration.java index 327bf081344c5..6f7d1336ac192 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioningConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingArcAutoProvisioningConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration for servers Arc auto provisioning. */ +/** + * Configuration for servers Arc auto provisioning. + */ @Fluent public final class DefenderForServersAwsOfferingArcAutoProvisioningConfiguration { /* @@ -22,13 +24,15 @@ public final class DefenderForServersAwsOfferingArcAutoProvisioningConfiguration @JsonProperty(value = "privateLinkScope") private String privateLinkScope; - /** Creates an instance of DefenderForServersAwsOfferingArcAutoProvisioningConfiguration class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingArcAutoProvisioningConfiguration class. + */ public DefenderForServersAwsOfferingArcAutoProvisioningConfiguration() { } /** * Get the proxy property: Optional HTTP proxy endpoint to use for the Arc agent. - * + * * @return the proxy value. */ public String proxy() { @@ -37,7 +41,7 @@ public String proxy() { /** * Set the proxy property: Optional HTTP proxy endpoint to use for the Arc agent. - * + * * @param proxy the proxy value to set. * @return the DefenderForServersAwsOfferingArcAutoProvisioningConfiguration object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersAwsOfferingArcAutoProvisioningConfiguration withProxy(S /** * Get the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @return the privateLinkScope value. */ public String privateLinkScope() { @@ -57,7 +61,7 @@ public String privateLinkScope() { /** * Set the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @param privateLinkScope the privateLinkScope value to set. * @return the DefenderForServersAwsOfferingArcAutoProvisioningConfiguration object itself. */ @@ -68,7 +72,7 @@ public DefenderForServersAwsOfferingArcAutoProvisioningConfiguration withPrivate /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingDefenderForServers.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingDefenderForServers.java index aa05784fa66b1..98df7fa47fe8b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingDefenderForServers.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingDefenderForServers.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Defender for servers connection configuration. */ +/** + * The Defender for servers connection configuration. + */ @Fluent public final class DefenderForServersAwsOfferingDefenderForServers { /* @@ -16,13 +18,15 @@ public final class DefenderForServersAwsOfferingDefenderForServers { @JsonProperty(value = "cloudRoleArn") private String cloudRoleArn; - /** Creates an instance of DefenderForServersAwsOfferingDefenderForServers class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingDefenderForServers class. + */ public DefenderForServersAwsOfferingDefenderForServers() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -31,7 +35,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForServersAwsOfferingDefenderForServers object itself. */ @@ -42,7 +46,7 @@ public DefenderForServersAwsOfferingDefenderForServers withCloudRoleArn(String c /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingMdeAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingMdeAutoProvisioning.java index 0bef82c51ccb0..44d32a6f8ef55 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingMdeAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingMdeAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Microsoft Defender for Endpoint autoprovisioning configuration. */ +/** + * The Microsoft Defender for Endpoint autoprovisioning configuration. + */ @Fluent public final class DefenderForServersAwsOfferingMdeAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForServersAwsOfferingMdeAutoProvisioning { @JsonProperty(value = "configuration") private Object configuration; - /** Creates an instance of DefenderForServersAwsOfferingMdeAutoProvisioning class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingMdeAutoProvisioning class. + */ public DefenderForServersAwsOfferingMdeAutoProvisioning() { } /** * Get the enabled property: Is Microsoft Defender for Endpoint auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Microsoft Defender for Endpoint auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersAwsOfferingMdeAutoProvisioning object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersAwsOfferingMdeAutoProvisioning withEnabled(Boolean enab /** * Get the configuration property: configuration for Microsoft Defender for Endpoint autoprovisioning. - * + * * @return the configuration value. */ public Object configuration() { @@ -57,7 +61,7 @@ public Object configuration() { /** * Set the configuration property: configuration for Microsoft Defender for Endpoint autoprovisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersAwsOfferingMdeAutoProvisioning object itself. */ @@ -68,7 +72,7 @@ public DefenderForServersAwsOfferingMdeAutoProvisioning withConfiguration(Object /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingSubPlan.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingSubPlan.java index b9f80ce25dca0..0c4a52d242267 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingSubPlan.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingSubPlan.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** configuration for the servers offering subPlan. */ +/** + * configuration for the servers offering subPlan. + */ @Fluent public final class DefenderForServersAwsOfferingSubPlan { /* @@ -16,13 +18,15 @@ public final class DefenderForServersAwsOfferingSubPlan { @JsonProperty(value = "type") private SubPlan type; - /** Creates an instance of DefenderForServersAwsOfferingSubPlan class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingSubPlan class. + */ public DefenderForServersAwsOfferingSubPlan() { } /** * Get the type property: The available sub plans. - * + * * @return the type value. */ public SubPlan type() { @@ -31,7 +35,7 @@ public SubPlan type() { /** * Set the type property: The available sub plans. - * + * * @param type the type value to set. * @return the DefenderForServersAwsOfferingSubPlan object itself. */ @@ -42,7 +46,7 @@ public DefenderForServersAwsOfferingSubPlan withType(SubPlan type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioning.java index ec83a52f46eba..f0e803a21b440 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Vulnerability Assessment autoprovisioning configuration. */ +/** + * The Vulnerability Assessment autoprovisioning configuration. + */ @Fluent public final class DefenderForServersAwsOfferingVaAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForServersAwsOfferingVaAutoProvisioning { @JsonProperty(value = "configuration") private DefenderForServersAwsOfferingVaAutoProvisioningConfiguration configuration; - /** Creates an instance of DefenderForServersAwsOfferingVaAutoProvisioning class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingVaAutoProvisioning class. + */ public DefenderForServersAwsOfferingVaAutoProvisioning() { } /** * Get the enabled property: Is Vulnerability Assessment auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Vulnerability Assessment auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersAwsOfferingVaAutoProvisioning object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersAwsOfferingVaAutoProvisioning withEnabled(Boolean enabl /** * Get the configuration property: configuration for Vulnerability Assessment autoprovisioning. - * + * * @return the configuration value. */ public DefenderForServersAwsOfferingVaAutoProvisioningConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderForServersAwsOfferingVaAutoProvisioningConfiguration configuratio /** * Set the configuration property: configuration for Vulnerability Assessment autoprovisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersAwsOfferingVaAutoProvisioning object itself. */ - public DefenderForServersAwsOfferingVaAutoProvisioning withConfiguration( - DefenderForServersAwsOfferingVaAutoProvisioningConfiguration configuration) { + public DefenderForServersAwsOfferingVaAutoProvisioning + withConfiguration(DefenderForServersAwsOfferingVaAutoProvisioningConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioningConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioningConfiguration.java index 1afa0a454e4cd..4f2d54b3101db 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioningConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVaAutoProvisioningConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** configuration for Vulnerability Assessment autoprovisioning. */ +/** + * configuration for Vulnerability Assessment autoprovisioning. + */ @Fluent public final class DefenderForServersAwsOfferingVaAutoProvisioningConfiguration { /* @@ -16,13 +18,15 @@ public final class DefenderForServersAwsOfferingVaAutoProvisioningConfiguration @JsonProperty(value = "type") private Type type; - /** Creates an instance of DefenderForServersAwsOfferingVaAutoProvisioningConfiguration class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingVaAutoProvisioningConfiguration class. + */ public DefenderForServersAwsOfferingVaAutoProvisioningConfiguration() { } /** * Get the type property: The Vulnerability Assessment solution to be provisioned. Can be either 'TVM' or 'Qualys'. - * + * * @return the type value. */ public Type type() { @@ -31,7 +35,7 @@ public Type type() { /** * Set the type property: The Vulnerability Assessment solution to be provisioned. Can be either 'TVM' or 'Qualys'. - * + * * @param type the type value to set. * @return the DefenderForServersAwsOfferingVaAutoProvisioningConfiguration object itself. */ @@ -42,7 +46,7 @@ public DefenderForServersAwsOfferingVaAutoProvisioningConfiguration withType(Typ /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScanners.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScanners.java index bc3612691f4e9..29c7e1e4184d7 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScanners.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScanners.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Microsoft Defender for Server VM scanning configuration. */ +/** + * The Microsoft Defender for Server VM scanning configuration. + */ @Fluent public final class DefenderForServersAwsOfferingVmScanners { /* @@ -22,13 +24,15 @@ public final class DefenderForServersAwsOfferingVmScanners { @JsonProperty(value = "configuration") private DefenderForServersAwsOfferingVmScannersConfiguration configuration; - /** Creates an instance of DefenderForServersAwsOfferingVmScanners class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingVmScanners class. + */ public DefenderForServersAwsOfferingVmScanners() { } /** * Get the enabled property: Is Microsoft Defender for Server VM scanning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Microsoft Defender for Server VM scanning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersAwsOfferingVmScanners object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersAwsOfferingVmScanners withEnabled(Boolean enabled) { /** * Get the configuration property: configuration for Microsoft Defender for Server VM scanning. - * + * * @return the configuration value. */ public DefenderForServersAwsOfferingVmScannersConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderForServersAwsOfferingVmScannersConfiguration configuration() { /** * Set the configuration property: configuration for Microsoft Defender for Server VM scanning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersAwsOfferingVmScanners object itself. */ - public DefenderForServersAwsOfferingVmScanners withConfiguration( - DefenderForServersAwsOfferingVmScannersConfiguration configuration) { + public DefenderForServersAwsOfferingVmScanners + withConfiguration(DefenderForServersAwsOfferingVmScannersConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScannersConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScannersConfiguration.java index 0d8b78178250b..aded5d5c2c083 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScannersConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersAwsOfferingVmScannersConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** configuration for Microsoft Defender for Server VM scanning. */ +/** + * configuration for Microsoft Defender for Server VM scanning. + */ @Fluent public final class DefenderForServersAwsOfferingVmScannersConfiguration { /* @@ -31,13 +33,15 @@ public final class DefenderForServersAwsOfferingVmScannersConfiguration { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map exclusionTags; - /** Creates an instance of DefenderForServersAwsOfferingVmScannersConfiguration class. */ + /** + * Creates an instance of DefenderForServersAwsOfferingVmScannersConfiguration class. + */ public DefenderForServersAwsOfferingVmScannersConfiguration() { } /** * Get the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @return the cloudRoleArn value. */ public String cloudRoleArn() { @@ -46,7 +50,7 @@ public String cloudRoleArn() { /** * Set the cloudRoleArn property: The cloud role ARN in AWS for this feature. - * + * * @param cloudRoleArn the cloudRoleArn value to set. * @return the DefenderForServersAwsOfferingVmScannersConfiguration object itself. */ @@ -57,7 +61,7 @@ public DefenderForServersAwsOfferingVmScannersConfiguration withCloudRoleArn(Str /** * Get the scanningMode property: The scanning mode for the VM scan. - * + * * @return the scanningMode value. */ public ScanningMode scanningMode() { @@ -66,7 +70,7 @@ public ScanningMode scanningMode() { /** * Set the scanningMode property: The scanning mode for the VM scan. - * + * * @param scanningMode the scanningMode value to set. * @return the DefenderForServersAwsOfferingVmScannersConfiguration object itself. */ @@ -77,7 +81,7 @@ public DefenderForServersAwsOfferingVmScannersConfiguration withScanningMode(Sca /** * Get the exclusionTags property: VM tags that indicates that VM should not be scanned. - * + * * @return the exclusionTags value. */ public Map exclusionTags() { @@ -86,7 +90,7 @@ public Map exclusionTags() { /** * Set the exclusionTags property: VM tags that indicates that VM should not be scanned. - * + * * @param exclusionTags the exclusionTags value to set. * @return the DefenderForServersAwsOfferingVmScannersConfiguration object itself. */ @@ -97,7 +101,7 @@ public DefenderForServersAwsOfferingVmScannersConfiguration withExclusionTags(Ma /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOffering.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOffering.java index 336fa465a9c99..1c02333bbcde9 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOffering.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOffering.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The Defender for Servers GCP offering configurations. */ +/** + * The Defender for Servers GCP offering configurations. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "offeringType") @JsonTypeName("DefenderForServersGcp") @Fluent @@ -50,13 +52,15 @@ public final class DefenderForServersGcpOffering extends CloudOffering { @JsonProperty(value = "vmScanners") private DefenderForServersGcpOfferingVmScanners vmScanners; - /** Creates an instance of DefenderForServersGcpOffering class. */ + /** + * Creates an instance of DefenderForServersGcpOffering class. + */ public DefenderForServersGcpOffering() { } /** * Get the defenderForServers property: The Defender for servers connection configuration. - * + * * @return the defenderForServers value. */ public DefenderForServersGcpOfferingDefenderForServers defenderForServers() { @@ -65,19 +69,19 @@ public DefenderForServersGcpOfferingDefenderForServers defenderForServers() { /** * Set the defenderForServers property: The Defender for servers connection configuration. - * + * * @param defenderForServers the defenderForServers value to set. * @return the DefenderForServersGcpOffering object itself. */ - public DefenderForServersGcpOffering withDefenderForServers( - DefenderForServersGcpOfferingDefenderForServers defenderForServers) { + public DefenderForServersGcpOffering + withDefenderForServers(DefenderForServersGcpOfferingDefenderForServers defenderForServers) { this.defenderForServers = defenderForServers; return this; } /** * Get the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @return the arcAutoProvisioning value. */ public DefenderForServersGcpOfferingArcAutoProvisioning arcAutoProvisioning() { @@ -86,19 +90,19 @@ public DefenderForServersGcpOfferingArcAutoProvisioning arcAutoProvisioning() { /** * Set the arcAutoProvisioning property: The ARC autoprovisioning configuration. - * + * * @param arcAutoProvisioning the arcAutoProvisioning value to set. * @return the DefenderForServersGcpOffering object itself. */ - public DefenderForServersGcpOffering withArcAutoProvisioning( - DefenderForServersGcpOfferingArcAutoProvisioning arcAutoProvisioning) { + public DefenderForServersGcpOffering + withArcAutoProvisioning(DefenderForServersGcpOfferingArcAutoProvisioning arcAutoProvisioning) { this.arcAutoProvisioning = arcAutoProvisioning; return this; } /** * Get the vaAutoProvisioning property: The Vulnerability Assessment autoprovisioning configuration. - * + * * @return the vaAutoProvisioning value. */ public DefenderForServersGcpOfferingVaAutoProvisioning vaAutoProvisioning() { @@ -107,19 +111,19 @@ public DefenderForServersGcpOfferingVaAutoProvisioning vaAutoProvisioning() { /** * Set the vaAutoProvisioning property: The Vulnerability Assessment autoprovisioning configuration. - * + * * @param vaAutoProvisioning the vaAutoProvisioning value to set. * @return the DefenderForServersGcpOffering object itself. */ - public DefenderForServersGcpOffering withVaAutoProvisioning( - DefenderForServersGcpOfferingVaAutoProvisioning vaAutoProvisioning) { + public DefenderForServersGcpOffering + withVaAutoProvisioning(DefenderForServersGcpOfferingVaAutoProvisioning vaAutoProvisioning) { this.vaAutoProvisioning = vaAutoProvisioning; return this; } /** * Get the mdeAutoProvisioning property: The Microsoft Defender for Endpoint autoprovisioning configuration. - * + * * @return the mdeAutoProvisioning value. */ public DefenderForServersGcpOfferingMdeAutoProvisioning mdeAutoProvisioning() { @@ -128,19 +132,19 @@ public DefenderForServersGcpOfferingMdeAutoProvisioning mdeAutoProvisioning() { /** * Set the mdeAutoProvisioning property: The Microsoft Defender for Endpoint autoprovisioning configuration. - * + * * @param mdeAutoProvisioning the mdeAutoProvisioning value to set. * @return the DefenderForServersGcpOffering object itself. */ - public DefenderForServersGcpOffering withMdeAutoProvisioning( - DefenderForServersGcpOfferingMdeAutoProvisioning mdeAutoProvisioning) { + public DefenderForServersGcpOffering + withMdeAutoProvisioning(DefenderForServersGcpOfferingMdeAutoProvisioning mdeAutoProvisioning) { this.mdeAutoProvisioning = mdeAutoProvisioning; return this; } /** * Get the subPlan property: configuration for the servers offering subPlan. - * + * * @return the subPlan value. */ public DefenderForServersGcpOfferingSubPlan subPlan() { @@ -149,7 +153,7 @@ public DefenderForServersGcpOfferingSubPlan subPlan() { /** * Set the subPlan property: configuration for the servers offering subPlan. - * + * * @param subPlan the subPlan value to set. * @return the DefenderForServersGcpOffering object itself. */ @@ -160,7 +164,7 @@ public DefenderForServersGcpOffering withSubPlan(DefenderForServersGcpOfferingSu /** * Get the vmScanners property: The Microsoft Defender for Server VM scanning configuration. - * + * * @return the vmScanners value. */ public DefenderForServersGcpOfferingVmScanners vmScanners() { @@ -169,7 +173,7 @@ public DefenderForServersGcpOfferingVmScanners vmScanners() { /** * Set the vmScanners property: The Microsoft Defender for Server VM scanning configuration. - * + * * @param vmScanners the vmScanners value to set. * @return the DefenderForServersGcpOffering object itself. */ @@ -180,7 +184,7 @@ public DefenderForServersGcpOffering withVmScanners(DefenderForServersGcpOfferin /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioning.java index 62a8766f49c51..0063ee864c6c8 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The ARC autoprovisioning configuration. */ +/** + * The ARC autoprovisioning configuration. + */ @Fluent public final class DefenderForServersGcpOfferingArcAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForServersGcpOfferingArcAutoProvisioning { @JsonProperty(value = "configuration") private DefenderForServersGcpOfferingArcAutoProvisioningConfiguration configuration; - /** Creates an instance of DefenderForServersGcpOfferingArcAutoProvisioning class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingArcAutoProvisioning class. + */ public DefenderForServersGcpOfferingArcAutoProvisioning() { } /** * Get the enabled property: Is arc auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is arc auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersGcpOfferingArcAutoProvisioning object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersGcpOfferingArcAutoProvisioning withEnabled(Boolean enab /** * Get the configuration property: Configuration for servers Arc auto provisioning. - * + * * @return the configuration value. */ public DefenderForServersGcpOfferingArcAutoProvisioningConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderForServersGcpOfferingArcAutoProvisioningConfiguration configurati /** * Set the configuration property: Configuration for servers Arc auto provisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersGcpOfferingArcAutoProvisioning object itself. */ - public DefenderForServersGcpOfferingArcAutoProvisioning withConfiguration( - DefenderForServersGcpOfferingArcAutoProvisioningConfiguration configuration) { + public DefenderForServersGcpOfferingArcAutoProvisioning + withConfiguration(DefenderForServersGcpOfferingArcAutoProvisioningConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioningConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioningConfiguration.java index ff5655dfab0a0..bff15ad5ccf7b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioningConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingArcAutoProvisioningConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Configuration for servers Arc auto provisioning. */ +/** + * Configuration for servers Arc auto provisioning. + */ @Fluent public final class DefenderForServersGcpOfferingArcAutoProvisioningConfiguration { /* @@ -22,13 +24,15 @@ public final class DefenderForServersGcpOfferingArcAutoProvisioningConfiguration @JsonProperty(value = "privateLinkScope") private String privateLinkScope; - /** Creates an instance of DefenderForServersGcpOfferingArcAutoProvisioningConfiguration class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingArcAutoProvisioningConfiguration class. + */ public DefenderForServersGcpOfferingArcAutoProvisioningConfiguration() { } /** * Get the proxy property: Optional HTTP proxy endpoint to use for the Arc agent. - * + * * @return the proxy value. */ public String proxy() { @@ -37,7 +41,7 @@ public String proxy() { /** * Set the proxy property: Optional HTTP proxy endpoint to use for the Arc agent. - * + * * @param proxy the proxy value to set. * @return the DefenderForServersGcpOfferingArcAutoProvisioningConfiguration object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersGcpOfferingArcAutoProvisioningConfiguration withProxy(S /** * Get the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @return the privateLinkScope value. */ public String privateLinkScope() { @@ -57,7 +61,7 @@ public String privateLinkScope() { /** * Set the privateLinkScope property: Optional Arc private link scope resource id to link the Arc agent. - * + * * @param privateLinkScope the privateLinkScope value to set. * @return the DefenderForServersGcpOfferingArcAutoProvisioningConfiguration object itself. */ @@ -68,7 +72,7 @@ public DefenderForServersGcpOfferingArcAutoProvisioningConfiguration withPrivate /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingDefenderForServers.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingDefenderForServers.java index 1533e9171d673..64fd7880fb158 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingDefenderForServers.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingDefenderForServers.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Defender for servers connection configuration. */ +/** + * The Defender for servers connection configuration. + */ @Fluent public final class DefenderForServersGcpOfferingDefenderForServers { /* @@ -22,13 +24,15 @@ public final class DefenderForServersGcpOfferingDefenderForServers { @JsonProperty(value = "serviceAccountEmailAddress") private String serviceAccountEmailAddress; - /** Creates an instance of DefenderForServersGcpOfferingDefenderForServers class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingDefenderForServers class. + */ public DefenderForServersGcpOfferingDefenderForServers() { } /** * Get the workloadIdentityProviderId property: The workload identity provider id in GCP for this feature. - * + * * @return the workloadIdentityProviderId value. */ public String workloadIdentityProviderId() { @@ -37,19 +41,19 @@ public String workloadIdentityProviderId() { /** * Set the workloadIdentityProviderId property: The workload identity provider id in GCP for this feature. - * + * * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. * @return the DefenderForServersGcpOfferingDefenderForServers object itself. */ - public DefenderForServersGcpOfferingDefenderForServers withWorkloadIdentityProviderId( - String workloadIdentityProviderId) { + public DefenderForServersGcpOfferingDefenderForServers + withWorkloadIdentityProviderId(String workloadIdentityProviderId) { this.workloadIdentityProviderId = workloadIdentityProviderId; return this; } /** * Get the serviceAccountEmailAddress property: The service account email address in GCP for this feature. - * + * * @return the serviceAccountEmailAddress value. */ public String serviceAccountEmailAddress() { @@ -58,19 +62,19 @@ public String serviceAccountEmailAddress() { /** * Set the serviceAccountEmailAddress property: The service account email address in GCP for this feature. - * + * * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. * @return the DefenderForServersGcpOfferingDefenderForServers object itself. */ - public DefenderForServersGcpOfferingDefenderForServers withServiceAccountEmailAddress( - String serviceAccountEmailAddress) { + public DefenderForServersGcpOfferingDefenderForServers + withServiceAccountEmailAddress(String serviceAccountEmailAddress) { this.serviceAccountEmailAddress = serviceAccountEmailAddress; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingMdeAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingMdeAutoProvisioning.java index 51bb244074163..f99c83dc070ce 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingMdeAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingMdeAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Microsoft Defender for Endpoint autoprovisioning configuration. */ +/** + * The Microsoft Defender for Endpoint autoprovisioning configuration. + */ @Fluent public final class DefenderForServersGcpOfferingMdeAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForServersGcpOfferingMdeAutoProvisioning { @JsonProperty(value = "configuration") private Object configuration; - /** Creates an instance of DefenderForServersGcpOfferingMdeAutoProvisioning class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingMdeAutoProvisioning class. + */ public DefenderForServersGcpOfferingMdeAutoProvisioning() { } /** * Get the enabled property: Is Microsoft Defender for Endpoint auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Microsoft Defender for Endpoint auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersGcpOfferingMdeAutoProvisioning object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersGcpOfferingMdeAutoProvisioning withEnabled(Boolean enab /** * Get the configuration property: configuration for Microsoft Defender for Endpoint autoprovisioning. - * + * * @return the configuration value. */ public Object configuration() { @@ -57,7 +61,7 @@ public Object configuration() { /** * Set the configuration property: configuration for Microsoft Defender for Endpoint autoprovisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersGcpOfferingMdeAutoProvisioning object itself. */ @@ -68,7 +72,7 @@ public DefenderForServersGcpOfferingMdeAutoProvisioning withConfiguration(Object /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingSubPlan.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingSubPlan.java index 22ba934ffb525..805e38440176c 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingSubPlan.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingSubPlan.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** configuration for the servers offering subPlan. */ +/** + * configuration for the servers offering subPlan. + */ @Fluent public final class DefenderForServersGcpOfferingSubPlan { /* @@ -16,13 +18,15 @@ public final class DefenderForServersGcpOfferingSubPlan { @JsonProperty(value = "type") private SubPlan type; - /** Creates an instance of DefenderForServersGcpOfferingSubPlan class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingSubPlan class. + */ public DefenderForServersGcpOfferingSubPlan() { } /** * Get the type property: The available sub plans. - * + * * @return the type value. */ public SubPlan type() { @@ -31,7 +35,7 @@ public SubPlan type() { /** * Set the type property: The available sub plans. - * + * * @param type the type value to set. * @return the DefenderForServersGcpOfferingSubPlan object itself. */ @@ -42,7 +46,7 @@ public DefenderForServersGcpOfferingSubPlan withType(SubPlan type) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioning.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioning.java index f6b41f749fd8c..add710540f995 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioning.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioning.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Vulnerability Assessment autoprovisioning configuration. */ +/** + * The Vulnerability Assessment autoprovisioning configuration. + */ @Fluent public final class DefenderForServersGcpOfferingVaAutoProvisioning { /* @@ -22,13 +24,15 @@ public final class DefenderForServersGcpOfferingVaAutoProvisioning { @JsonProperty(value = "configuration") private DefenderForServersGcpOfferingVaAutoProvisioningConfiguration configuration; - /** Creates an instance of DefenderForServersGcpOfferingVaAutoProvisioning class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingVaAutoProvisioning class. + */ public DefenderForServersGcpOfferingVaAutoProvisioning() { } /** * Get the enabled property: Is Vulnerability Assessment auto provisioning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Vulnerability Assessment auto provisioning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersGcpOfferingVaAutoProvisioning object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersGcpOfferingVaAutoProvisioning withEnabled(Boolean enabl /** * Get the configuration property: configuration for Vulnerability Assessment autoprovisioning. - * + * * @return the configuration value. */ public DefenderForServersGcpOfferingVaAutoProvisioningConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderForServersGcpOfferingVaAutoProvisioningConfiguration configuratio /** * Set the configuration property: configuration for Vulnerability Assessment autoprovisioning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersGcpOfferingVaAutoProvisioning object itself. */ - public DefenderForServersGcpOfferingVaAutoProvisioning withConfiguration( - DefenderForServersGcpOfferingVaAutoProvisioningConfiguration configuration) { + public DefenderForServersGcpOfferingVaAutoProvisioning + withConfiguration(DefenderForServersGcpOfferingVaAutoProvisioningConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioningConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioningConfiguration.java index 4c51b2de77486..afceeffabd351 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioningConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVaAutoProvisioningConfiguration.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** configuration for Vulnerability Assessment autoprovisioning. */ +/** + * configuration for Vulnerability Assessment autoprovisioning. + */ @Fluent public final class DefenderForServersGcpOfferingVaAutoProvisioningConfiguration { /* @@ -16,13 +18,15 @@ public final class DefenderForServersGcpOfferingVaAutoProvisioningConfiguration @JsonProperty(value = "type") private Type type; - /** Creates an instance of DefenderForServersGcpOfferingVaAutoProvisioningConfiguration class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingVaAutoProvisioningConfiguration class. + */ public DefenderForServersGcpOfferingVaAutoProvisioningConfiguration() { } /** * Get the type property: The Vulnerability Assessment solution to be provisioned. Can be either 'TVM' or 'Qualys'. - * + * * @return the type value. */ public Type type() { @@ -31,7 +35,7 @@ public Type type() { /** * Set the type property: The Vulnerability Assessment solution to be provisioned. Can be either 'TVM' or 'Qualys'. - * + * * @param type the type value to set. * @return the DefenderForServersGcpOfferingVaAutoProvisioningConfiguration object itself. */ @@ -42,7 +46,7 @@ public DefenderForServersGcpOfferingVaAutoProvisioningConfiguration withType(Typ /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScanners.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScanners.java index d530ace03cb64..067d8b5669a22 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScanners.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScanners.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The Microsoft Defender for Server VM scanning configuration. */ +/** + * The Microsoft Defender for Server VM scanning configuration. + */ @Fluent public final class DefenderForServersGcpOfferingVmScanners { /* @@ -22,13 +24,15 @@ public final class DefenderForServersGcpOfferingVmScanners { @JsonProperty(value = "configuration") private DefenderForServersGcpOfferingVmScannersConfiguration configuration; - /** Creates an instance of DefenderForServersGcpOfferingVmScanners class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingVmScanners class. + */ public DefenderForServersGcpOfferingVmScanners() { } /** * Get the enabled property: Is Microsoft Defender for Server VM scanning enabled. - * + * * @return the enabled value. */ public Boolean enabled() { @@ -37,7 +41,7 @@ public Boolean enabled() { /** * Set the enabled property: Is Microsoft Defender for Server VM scanning enabled. - * + * * @param enabled the enabled value to set. * @return the DefenderForServersGcpOfferingVmScanners object itself. */ @@ -48,7 +52,7 @@ public DefenderForServersGcpOfferingVmScanners withEnabled(Boolean enabled) { /** * Get the configuration property: configuration for Microsoft Defender for Server VM scanning. - * + * * @return the configuration value. */ public DefenderForServersGcpOfferingVmScannersConfiguration configuration() { @@ -57,19 +61,19 @@ public DefenderForServersGcpOfferingVmScannersConfiguration configuration() { /** * Set the configuration property: configuration for Microsoft Defender for Server VM scanning. - * + * * @param configuration the configuration value to set. * @return the DefenderForServersGcpOfferingVmScanners object itself. */ - public DefenderForServersGcpOfferingVmScanners withConfiguration( - DefenderForServersGcpOfferingVmScannersConfiguration configuration) { + public DefenderForServersGcpOfferingVmScanners + withConfiguration(DefenderForServersGcpOfferingVmScannersConfiguration configuration) { this.configuration = configuration; return this; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScannersConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScannersConfiguration.java index aedf865b1f868..0dd6d187d6988 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScannersConfiguration.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForServersGcpOfferingVmScannersConfiguration.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** configuration for Microsoft Defender for Server VM scanning. */ +/** + * configuration for Microsoft Defender for Server VM scanning. + */ @Fluent public final class DefenderForServersGcpOfferingVmScannersConfiguration { /* @@ -25,13 +27,15 @@ public final class DefenderForServersGcpOfferingVmScannersConfiguration { @JsonInclude(value = JsonInclude.Include.NON_NULL, content = JsonInclude.Include.ALWAYS) private Map exclusionTags; - /** Creates an instance of DefenderForServersGcpOfferingVmScannersConfiguration class. */ + /** + * Creates an instance of DefenderForServersGcpOfferingVmScannersConfiguration class. + */ public DefenderForServersGcpOfferingVmScannersConfiguration() { } /** * Get the scanningMode property: The scanning mode for the VM scan. - * + * * @return the scanningMode value. */ public ScanningMode scanningMode() { @@ -40,7 +44,7 @@ public ScanningMode scanningMode() { /** * Set the scanningMode property: The scanning mode for the VM scan. - * + * * @param scanningMode the scanningMode value to set. * @return the DefenderForServersGcpOfferingVmScannersConfiguration object itself. */ @@ -51,7 +55,7 @@ public DefenderForServersGcpOfferingVmScannersConfiguration withScanningMode(Sca /** * Get the exclusionTags property: VM tags that indicate that VM should not be scanned. - * + * * @return the exclusionTags value. */ public Map exclusionTags() { @@ -60,7 +64,7 @@ public Map exclusionTags() { /** * Set the exclusionTags property: VM tags that indicate that VM should not be scanned. - * + * * @param exclusionTags the exclusionTags value to set. * @return the DefenderForServersGcpOfferingVmScannersConfiguration object itself. */ @@ -71,7 +75,7 @@ public DefenderForServersGcpOfferingVmScannersConfiguration withExclusionTags(Ma /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorageSetting.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorageSetting.java new file mode 100644 index 0000000000000..24ec7a8744bc1 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorageSetting.java @@ -0,0 +1,262 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.DefenderForStorageSettingInner; + +/** + * An immutable client-side representation of DefenderForStorageSetting. + */ +public interface DefenderForStorageSetting { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the isEnabled property: Indicates whether Defender for Storage is enabled on this storage account. + * + * @return the isEnabled value. + */ + Boolean isEnabled(); + + /** + * Gets the overrideSubscriptionLevelSettings property: Indicates whether the settings defined for this storage + * account should override the settings defined for the subscription. + * + * @return the overrideSubscriptionLevelSettings value. + */ + Boolean overrideSubscriptionLevelSettings(); + + /** + * Gets the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic to send scan + * results to. + * + * @return the scanResultsEventGridTopicResourceId value. + */ + String scanResultsEventGridTopicResourceId(); + + /** + * Gets the operationStatus property: Upon failure or partial success. Additional data describing Malware Scanning + * enable/disable operation. + * + * @return the operationStatus value. + */ + OperationStatus operationStatus(); + + /** + * Gets the isEnabledMalwareScanningIsEnabled property: Indicates whether On Upload malware scanning should be + * enabled. + * + * @return the isEnabledMalwareScanningIsEnabled value. + */ + Boolean isEnabledMalwareScanningIsEnabled(); + + /** + * Gets the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * + * @return the capGBPerMonth value. + */ + Integer capGBPerMonth(); + + /** + * Gets the isEnabledSensitiveDataDiscoveryIsEnabled property: Indicates whether Sensitive Data Discovery should be + * enabled. + * + * @return the isEnabledSensitiveDataDiscoveryIsEnabled value. + */ + Boolean isEnabledSensitiveDataDiscoveryIsEnabled(); + + /** + * Gets the operationStatusSensitiveDataDiscoveryOperationStatus property: Upon failure or partial success. + * Additional data describing Sensitive Data Discovery enable/disable operation. + * + * @return the operationStatusSensitiveDataDiscoveryOperationStatus value. + */ + OperationStatus operationStatusSensitiveDataDiscoveryOperationStatus(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.DefenderForStorageSettingInner object. + * + * @return the inner object. + */ + DefenderForStorageSettingInner innerModel(); + + /** + * The entirety of the DefenderForStorageSetting definition. + */ + interface Definition extends DefinitionStages.Blank, DefinitionStages.WithScope, DefinitionStages.WithCreate { + } + + /** + * The DefenderForStorageSetting definition stages. + */ + interface DefinitionStages { + /** + * The first stage of the DefenderForStorageSetting definition. + */ + interface Blank extends WithScope { + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify parent resource. + */ + interface WithScope { + /** + * Specifies resourceId. + * + * @param resourceId The identifier of the resource. + * @return the next definition stage. + */ + WithCreate withExistingResourceId(String resourceId); + } + + /** + * The stage of the DefenderForStorageSetting definition which contains all the minimum required properties for + * the resource to be created, but also allows for any other optional properties to be specified. + */ + interface WithCreate + extends DefinitionStages.WithIsEnabled, DefinitionStages.WithOverrideSubscriptionLevelSettings, + DefinitionStages.WithScanResultsEventGridTopicResourceId, + DefinitionStages.WithIsEnabledMalwareScanningIsEnabled, DefinitionStages.WithCapGBPerMonth, + DefinitionStages.WithIsEnabledSensitiveDataDiscoveryIsEnabled { + /** + * Executes the create request. + * + * @return the created resource. + */ + DefenderForStorageSetting create(); + + /** + * Executes the create request. + * + * @param context The context to associate with this operation. + * @return the created resource. + */ + DefenderForStorageSetting create(Context context); + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify isEnabled. + */ + interface WithIsEnabled { + /** + * Specifies the isEnabled property: Indicates whether Defender for Storage is enabled on this storage + * account.. + * + * @param isEnabled Indicates whether Defender for Storage is enabled on this storage account. + * @return the next definition stage. + */ + WithCreate withIsEnabled(Boolean isEnabled); + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify overrideSubscriptionLevelSettings. + */ + interface WithOverrideSubscriptionLevelSettings { + /** + * Specifies the overrideSubscriptionLevelSettings property: Indicates whether the settings defined for this + * storage account should override the settings defined for the subscription.. + * + * @param overrideSubscriptionLevelSettings Indicates whether the settings defined for this storage account + * should override the settings defined for the subscription. + * @return the next definition stage. + */ + WithCreate withOverrideSubscriptionLevelSettings(Boolean overrideSubscriptionLevelSettings); + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify + * scanResultsEventGridTopicResourceId. + */ + interface WithScanResultsEventGridTopicResourceId { + /** + * Specifies the scanResultsEventGridTopicResourceId property: Optional. Resource id of an Event Grid Topic + * to send scan results to.. + * + * @param scanResultsEventGridTopicResourceId Optional. Resource id of an Event Grid Topic to send scan + * results to. + * @return the next definition stage. + */ + WithCreate withScanResultsEventGridTopicResourceId(String scanResultsEventGridTopicResourceId); + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify isEnabledMalwareScanningIsEnabled. + */ + interface WithIsEnabledMalwareScanningIsEnabled { + /** + * Specifies the isEnabledMalwareScanningIsEnabled property: Indicates whether On Upload malware scanning + * should be enabled.. + * + * @param isEnabledMalwareScanningIsEnabled Indicates whether On Upload malware scanning should be enabled. + * @return the next definition stage. + */ + WithCreate withIsEnabledMalwareScanningIsEnabled(Boolean isEnabledMalwareScanningIsEnabled); + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify capGBPerMonth. + */ + interface WithCapGBPerMonth { + /** + * Specifies the capGBPerMonth property: Defines the max GB to be scanned per Month. Set to -1 if no capping + * is needed.. + * + * @param capGBPerMonth Defines the max GB to be scanned per Month. Set to -1 if no capping is needed. + * @return the next definition stage. + */ + WithCreate withCapGBPerMonth(Integer capGBPerMonth); + } + + /** + * The stage of the DefenderForStorageSetting definition allowing to specify + * isEnabledSensitiveDataDiscoveryIsEnabled. + */ + interface WithIsEnabledSensitiveDataDiscoveryIsEnabled { + /** + * Specifies the isEnabledSensitiveDataDiscoveryIsEnabled property: Indicates whether Sensitive Data + * Discovery should be enabled.. + * + * @param isEnabledSensitiveDataDiscoveryIsEnabled Indicates whether Sensitive Data Discovery should be + * enabled. + * @return the next definition stage. + */ + WithCreate withIsEnabledSensitiveDataDiscoveryIsEnabled(Boolean isEnabledSensitiveDataDiscoveryIsEnabled); + } + } + + /** + * Refreshes the resource to sync with Azure. + * + * @return the refreshed resource. + */ + DefenderForStorageSetting refresh(); + + /** + * Refreshes the resource to sync with Azure. + * + * @param context The context to associate with this operation. + * @return the refreshed resource. + */ + DefenderForStorageSetting refresh(Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorages.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorages.java new file mode 100644 index 0000000000000..4762c3ebfa315 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DefenderForStorages.java @@ -0,0 +1,69 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of DefenderForStorages. + */ +public interface DefenderForStorages { + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response}. + */ + Response getWithResponse(String resourceId, SettingName settingName, Context context); + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param resourceId The identifier of the resource. + * @param settingName Defender for Storage setting name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account. + */ + DefenderForStorageSetting get(String resourceId, SettingName settingName); + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param id the resource ID. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response}. + */ + DefenderForStorageSetting getById(String id); + + /** + * Gets the Defender for Storage settings for the specified storage account. + * + * @param id the resource ID. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return the Defender for Storage settings for the specified storage account along with {@link Response}. + */ + Response getByIdWithResponse(String id, Context context); + + /** + * Begins definition for a new DefenderForStorageSetting resource. + * + * @param name resource name. + * @return the first stage of the new DefenderForStorageSetting definition. + */ + DefenderForStorageSetting.DefinitionStages.Blank define(SettingName name); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DenylistCustomAlertRule.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DenylistCustomAlertRule.java index 184d7b23a1f56..85be1995b28fd 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DenylistCustomAlertRule.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DenylistCustomAlertRule.java @@ -11,7 +11,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** A custom alert rule that checks if a value (depends on the custom alert type) is denied. */ +/** + * A custom alert rule that checks if a value (depends on the custom alert type) is denied. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("DenylistCustomAlertRule") @Fluent @@ -22,13 +24,15 @@ public final class DenylistCustomAlertRule extends ListCustomAlertRule { @JsonProperty(value = "denylistValues", required = true) private List denylistValues; - /** Creates an instance of DenylistCustomAlertRule class. */ + /** + * Creates an instance of DenylistCustomAlertRule class. + */ public DenylistCustomAlertRule() { } /** * Get the denylistValues property: The values to deny. The format of the values depends on the rule type. - * + * * @return the denylistValues value. */ public List denylistValues() { @@ -37,7 +41,7 @@ public List denylistValues() { /** * Set the denylistValues property: The values to deny. The format of the values depends on the rule type. - * + * * @param denylistValues the denylistValues value to set. * @return the DenylistCustomAlertRule object itself. */ @@ -46,7 +50,9 @@ public DenylistCustomAlertRule withDenylistValues(List denylistValues) { return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DenylistCustomAlertRule withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -55,17 +61,15 @@ public DenylistCustomAlertRule withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (denylistValues() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property denylistValues in model DenylistCustomAlertRule")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property denylistValues in model DenylistCustomAlertRule")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfiguration.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfiguration.java new file mode 100644 index 0000000000000..4b148f082fd3c --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfiguration.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.management.SystemData; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; + +/** + * An immutable client-side representation of DevOpsConfiguration. + */ +public interface DevOpsConfiguration { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the systemData property: Metadata pertaining to creation and last modification of the resource. + * + * @return the systemData value. + */ + SystemData systemData(); + + /** + * Gets the properties property: DevOps Configuration properties. + * + * @return the properties value. + */ + DevOpsConfigurationProperties properties(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner object. + * + * @return the inner object. + */ + DevOpsConfigurationInner innerModel(); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationListResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationListResponse.java new file mode 100644 index 0000000000000..ac060a5d6efb3 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationListResponse.java @@ -0,0 +1,85 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * List of RP resources which supports pagination. + */ +@Fluent +public final class DevOpsConfigurationListResponse { + /* + * Gets or sets list of resources. + */ + @JsonProperty(value = "value") + private List value; + + /* + * Gets or sets next link to scroll over the results. + */ + @JsonProperty(value = "nextLink") + private String nextLink; + + /** + * Creates an instance of DevOpsConfigurationListResponse class. + */ + public DevOpsConfigurationListResponse() { + } + + /** + * Get the value property: Gets or sets list of resources. + * + * @return the value value. + */ + public List value() { + return this.value; + } + + /** + * Set the value property: Gets or sets list of resources. + * + * @param value the value value to set. + * @return the DevOpsConfigurationListResponse object itself. + */ + public DevOpsConfigurationListResponse withValue(List value) { + this.value = value; + return this; + } + + /** + * Get the nextLink property: Gets or sets next link to scroll over the results. + * + * @return the nextLink value. + */ + public String nextLink() { + return this.nextLink; + } + + /** + * Set the nextLink property: Gets or sets next link to scroll over the results. + * + * @param nextLink the nextLink value to set. + * @return the DevOpsConfigurationListResponse object itself. + */ + public DevOpsConfigurationListResponse withNextLink(String nextLink) { + this.nextLink = nextLink; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (value() != null) { + value().forEach(e -> e.validate()); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationProperties.java new file mode 100644 index 0000000000000..137e8311394d5 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurationProperties.java @@ -0,0 +1,194 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.time.OffsetDateTime; +import java.util.List; + +/** + * DevOps Configuration properties. + */ +@Fluent +public final class DevOpsConfigurationProperties { + /* + * Gets or sets resource status message. + */ + @JsonProperty(value = "provisioningStatusMessage", access = JsonProperty.Access.WRITE_ONLY) + private String provisioningStatusMessage; + + /* + * Gets or sets time when resource was last checked. + */ + @JsonProperty(value = "provisioningStatusUpdateTimeUtc", access = JsonProperty.Access.WRITE_ONLY) + private OffsetDateTime provisioningStatusUpdateTimeUtc; + + /* + * The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + */ + @JsonProperty(value = "provisioningState") + private DevOpsProvisioningState provisioningState; + + /* + * Authorization payload. + */ + @JsonProperty(value = "authorization") + private Authorization authorization; + + /* + * AutoDiscovery states. + */ + @JsonProperty(value = "autoDiscovery") + private AutoDiscovery autoDiscovery; + + /* + * List of top-level inventory to select when AutoDiscovery is disabled. + * This field is ignored when AutoDiscovery is enabled. + */ + @JsonProperty(value = "topLevelInventoryList") + private List topLevelInventoryList; + + /** + * Creates an instance of DevOpsConfigurationProperties class. + */ + public DevOpsConfigurationProperties() { + } + + /** + * Get the provisioningStatusMessage property: Gets or sets resource status message. + * + * @return the provisioningStatusMessage value. + */ + public String provisioningStatusMessage() { + return this.provisioningStatusMessage; + } + + /** + * Get the provisioningStatusUpdateTimeUtc property: Gets or sets time when resource was last checked. + * + * @return the provisioningStatusUpdateTimeUtc value. + */ + public OffsetDateTime provisioningStatusUpdateTimeUtc() { + return this.provisioningStatusUpdateTimeUtc; + } + + /** + * Get the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @return the provisioningState value. + */ + public DevOpsProvisioningState provisioningState() { + return this.provisioningState; + } + + /** + * Set the provisioningState property: The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + * + * @param provisioningState the provisioningState value to set. + * @return the DevOpsConfigurationProperties object itself. + */ + public DevOpsConfigurationProperties withProvisioningState(DevOpsProvisioningState provisioningState) { + this.provisioningState = provisioningState; + return this; + } + + /** + * Get the authorization property: Authorization payload. + * + * @return the authorization value. + */ + public Authorization authorization() { + return this.authorization; + } + + /** + * Set the authorization property: Authorization payload. + * + * @param authorization the authorization value to set. + * @return the DevOpsConfigurationProperties object itself. + */ + public DevOpsConfigurationProperties withAuthorization(Authorization authorization) { + this.authorization = authorization; + return this; + } + + /** + * Get the autoDiscovery property: AutoDiscovery states. + * + * @return the autoDiscovery value. + */ + public AutoDiscovery autoDiscovery() { + return this.autoDiscovery; + } + + /** + * Set the autoDiscovery property: AutoDiscovery states. + * + * @param autoDiscovery the autoDiscovery value to set. + * @return the DevOpsConfigurationProperties object itself. + */ + public DevOpsConfigurationProperties withAutoDiscovery(AutoDiscovery autoDiscovery) { + this.autoDiscovery = autoDiscovery; + return this; + } + + /** + * Get the topLevelInventoryList property: List of top-level inventory to select when AutoDiscovery is disabled. + * This field is ignored when AutoDiscovery is enabled. + * + * @return the topLevelInventoryList value. + */ + public List topLevelInventoryList() { + return this.topLevelInventoryList; + } + + /** + * Set the topLevelInventoryList property: List of top-level inventory to select when AutoDiscovery is disabled. + * This field is ignored when AutoDiscovery is enabled. + * + * @param topLevelInventoryList the topLevelInventoryList value to set. + * @return the DevOpsConfigurationProperties object itself. + */ + public DevOpsConfigurationProperties withTopLevelInventoryList(List topLevelInventoryList) { + this.topLevelInventoryList = topLevelInventoryList; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (authorization() != null) { + authorization().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurations.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurations.java new file mode 100644 index 0000000000000..b1e34bdf97810 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsConfigurations.java @@ -0,0 +1,147 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.http.rest.PagedIterable; +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; +import com.azure.resourcemanager.security.fluent.models.DevOpsConfigurationInner; + +/** + * Resource collection API of DevOpsConfigurations. + */ +public interface DevOpsConfigurations { + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName); + + /** + * List DevOps Configurations. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return list of RP resources which supports pagination as paginated response with {@link PagedIterable}. + */ + PagedIterable list(String resourceGroupName, String securityConnectorName, Context context); + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String securityConnectorName, + Context context); + + /** + * Gets a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return a DevOps Configuration. + */ + DevOpsConfiguration get(String resourceGroupName, String securityConnectorName); + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + DevOpsConfiguration createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration); + + /** + * Creates or updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + DevOpsConfiguration createOrUpdate(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context); + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + DevOpsConfiguration update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration); + + /** + * Updates a DevOps Configuration. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param devOpsConfiguration The DevOps configuration resource payload. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devOps Configuration resource. + */ + DevOpsConfiguration update(String resourceGroupName, String securityConnectorName, + DevOpsConfigurationInner devOpsConfiguration, Context context); + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void deleteByResourceGroup(String resourceGroupName, String securityConnectorName); + + /** + * Deletes a DevOps Connector. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + */ + void delete(String resourceGroupName, String securityConnectorName, Context context); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsOperationResults.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsOperationResults.java new file mode 100644 index 0000000000000..da45533089d98 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsOperationResults.java @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.http.rest.Response; +import com.azure.core.util.Context; + +/** + * Resource collection API of DevOpsOperationResults. + */ +public interface DevOpsOperationResults { + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @param context The context to associate with this operation. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result along with {@link Response}. + */ + Response getWithResponse(String resourceGroupName, String securityConnectorName, + String operationResultId, Context context); + + /** + * Get devops long running operation result. + * + * @param resourceGroupName The name of the resource group. The name is case insensitive. + * @param securityConnectorName The security connector name. + * @param operationResultId The operation result Id. + * @throws IllegalArgumentException thrown if parameters fail the validation. + * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. + * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. + * @return devops long running operation result. + */ + OperationStatusResult get(String resourceGroupName, String securityConnectorName, String operationResultId); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsProvisioningState.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsProvisioningState.java new file mode 100644 index 0000000000000..cf6fecd3e6096 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DevOpsProvisioningState.java @@ -0,0 +1,86 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * The provisioning state of the resource. + * + * Pending - Provisioning pending. + * Failed - Provisioning failed. + * Succeeded - Successful provisioning. + * Canceled - Provisioning canceled. + * PendingDeletion - Deletion pending. + * DeletionSuccess - Deletion successful. + * DeletionFailure - Deletion failure. + */ +public final class DevOpsProvisioningState extends ExpandableStringEnum { + /** + * Static value Succeeded for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState SUCCEEDED = fromString("Succeeded"); + + /** + * Static value Failed for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState FAILED = fromString("Failed"); + + /** + * Static value Canceled for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState CANCELED = fromString("Canceled"); + + /** + * Static value Pending for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState PENDING = fromString("Pending"); + + /** + * Static value PendingDeletion for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState PENDING_DELETION = fromString("PendingDeletion"); + + /** + * Static value DeletionSuccess for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState DELETION_SUCCESS = fromString("DeletionSuccess"); + + /** + * Static value DeletionFailure for DevOpsProvisioningState. + */ + public static final DevOpsProvisioningState DELETION_FAILURE = fromString("DeletionFailure"); + + /** + * Creates a new instance of DevOpsProvisioningState value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public DevOpsProvisioningState() { + } + + /** + * Creates or finds a DevOpsProvisioningState from its string representation. + * + * @param name a name to look for. + * @return the corresponding DevOpsProvisioningState. + */ + @JsonCreator + public static DevOpsProvisioningState fromString(String name) { + return fromString(name, DevOpsProvisioningState.class); + } + + /** + * Gets known DevOpsProvisioningState values. + * + * @return known DevOpsProvisioningState values. + */ + public static Collection values() { + return values(DevOpsProvisioningState.class); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroup.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroup.java index 8cf36f7d77a0a..246f68a1ad583 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroup.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroup.java @@ -8,228 +8,265 @@ import com.azure.resourcemanager.security.fluent.models.DeviceSecurityGroupInner; import java.util.List; -/** An immutable client-side representation of DeviceSecurityGroup. */ +/** + * An immutable client-side representation of DeviceSecurityGroup. + */ public interface DeviceSecurityGroup { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the thresholdRules property: The list of custom alert threshold rules. - * + * * @return the thresholdRules value. */ List thresholdRules(); /** * Gets the timeWindowRules property: The list of custom alert time-window rules. - * + * * @return the timeWindowRules value. */ List timeWindowRules(); /** * Gets the allowlistRules property: The allow-list custom alert rules. - * + * * @return the allowlistRules value. */ List allowlistRules(); /** * Gets the denylistRules property: The deny-list custom alert rules. - * + * * @return the denylistRules value. */ List denylistRules(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.DeviceSecurityGroupInner object. - * + * * @return the inner object. */ DeviceSecurityGroupInner innerModel(); - /** The entirety of the DeviceSecurityGroup definition. */ + /** + * The entirety of the DeviceSecurityGroup definition. + */ interface Definition extends DefinitionStages.Blank, DefinitionStages.WithScope, DefinitionStages.WithCreate { } - /** The DeviceSecurityGroup definition stages. */ + + /** + * The DeviceSecurityGroup definition stages. + */ interface DefinitionStages { - /** The first stage of the DeviceSecurityGroup definition. */ + /** + * The first stage of the DeviceSecurityGroup definition. + */ interface Blank extends WithScope { } - /** The stage of the DeviceSecurityGroup definition allowing to specify parent resource. */ + + /** + * The stage of the DeviceSecurityGroup definition allowing to specify parent resource. + */ interface WithScope { /** * Specifies resourceId. - * + * * @param resourceId The identifier of the resource. * @return the next definition stage. */ WithCreate withExistingResourceId(String resourceId); } + /** * The stage of the DeviceSecurityGroup definition which contains all the minimum required properties for the * resource to be created, but also allows for any other optional properties to be specified. */ - interface WithCreate - extends DefinitionStages.WithThresholdRules, - DefinitionStages.WithTimeWindowRules, - DefinitionStages.WithAllowlistRules, - DefinitionStages.WithDenylistRules { + interface WithCreate extends DefinitionStages.WithThresholdRules, DefinitionStages.WithTimeWindowRules, + DefinitionStages.WithAllowlistRules, DefinitionStages.WithDenylistRules { /** * Executes the create request. - * + * * @return the created resource. */ DeviceSecurityGroup create(); /** * Executes the create request. - * + * * @param context The context to associate with this operation. * @return the created resource. */ DeviceSecurityGroup create(Context context); } - /** The stage of the DeviceSecurityGroup definition allowing to specify thresholdRules. */ + + /** + * The stage of the DeviceSecurityGroup definition allowing to specify thresholdRules. + */ interface WithThresholdRules { /** * Specifies the thresholdRules property: The list of custom alert threshold rules.. - * + * * @param thresholdRules The list of custom alert threshold rules. * @return the next definition stage. */ WithCreate withThresholdRules(List thresholdRules); } - /** The stage of the DeviceSecurityGroup definition allowing to specify timeWindowRules. */ + + /** + * The stage of the DeviceSecurityGroup definition allowing to specify timeWindowRules. + */ interface WithTimeWindowRules { /** * Specifies the timeWindowRules property: The list of custom alert time-window rules.. - * + * * @param timeWindowRules The list of custom alert time-window rules. * @return the next definition stage. */ WithCreate withTimeWindowRules(List timeWindowRules); } - /** The stage of the DeviceSecurityGroup definition allowing to specify allowlistRules. */ + + /** + * The stage of the DeviceSecurityGroup definition allowing to specify allowlistRules. + */ interface WithAllowlistRules { /** * Specifies the allowlistRules property: The allow-list custom alert rules.. - * + * * @param allowlistRules The allow-list custom alert rules. * @return the next definition stage. */ WithCreate withAllowlistRules(List allowlistRules); } - /** The stage of the DeviceSecurityGroup definition allowing to specify denylistRules. */ + + /** + * The stage of the DeviceSecurityGroup definition allowing to specify denylistRules. + */ interface WithDenylistRules { /** * Specifies the denylistRules property: The deny-list custom alert rules.. - * + * * @param denylistRules The deny-list custom alert rules. * @return the next definition stage. */ WithCreate withDenylistRules(List denylistRules); } } + /** * Begins update for the DeviceSecurityGroup resource. - * + * * @return the stage of resource update. */ DeviceSecurityGroup.Update update(); - /** The template for DeviceSecurityGroup update. */ - interface Update - extends UpdateStages.WithThresholdRules, - UpdateStages.WithTimeWindowRules, - UpdateStages.WithAllowlistRules, - UpdateStages.WithDenylistRules { + /** + * The template for DeviceSecurityGroup update. + */ + interface Update extends UpdateStages.WithThresholdRules, UpdateStages.WithTimeWindowRules, + UpdateStages.WithAllowlistRules, UpdateStages.WithDenylistRules { /** * Executes the update request. - * + * * @return the updated resource. */ DeviceSecurityGroup apply(); /** * Executes the update request. - * + * * @param context The context to associate with this operation. * @return the updated resource. */ DeviceSecurityGroup apply(Context context); } - /** The DeviceSecurityGroup update stages. */ + + /** + * The DeviceSecurityGroup update stages. + */ interface UpdateStages { - /** The stage of the DeviceSecurityGroup update allowing to specify thresholdRules. */ + /** + * The stage of the DeviceSecurityGroup update allowing to specify thresholdRules. + */ interface WithThresholdRules { /** * Specifies the thresholdRules property: The list of custom alert threshold rules.. - * + * * @param thresholdRules The list of custom alert threshold rules. * @return the next definition stage. */ Update withThresholdRules(List thresholdRules); } - /** The stage of the DeviceSecurityGroup update allowing to specify timeWindowRules. */ + + /** + * The stage of the DeviceSecurityGroup update allowing to specify timeWindowRules. + */ interface WithTimeWindowRules { /** * Specifies the timeWindowRules property: The list of custom alert time-window rules.. - * + * * @param timeWindowRules The list of custom alert time-window rules. * @return the next definition stage. */ Update withTimeWindowRules(List timeWindowRules); } - /** The stage of the DeviceSecurityGroup update allowing to specify allowlistRules. */ + + /** + * The stage of the DeviceSecurityGroup update allowing to specify allowlistRules. + */ interface WithAllowlistRules { /** * Specifies the allowlistRules property: The allow-list custom alert rules.. - * + * * @param allowlistRules The allow-list custom alert rules. * @return the next definition stage. */ Update withAllowlistRules(List allowlistRules); } - /** The stage of the DeviceSecurityGroup update allowing to specify denylistRules. */ + + /** + * The stage of the DeviceSecurityGroup update allowing to specify denylistRules. + */ interface WithDenylistRules { /** * Specifies the denylistRules property: The deny-list custom alert rules.. - * + * * @param denylistRules The deny-list custom alert rules. * @return the next definition stage. */ Update withDenylistRules(List denylistRules); } } + /** * Refreshes the resource to sync with Azure. - * + * * @return the refreshed resource. */ DeviceSecurityGroup refresh(); /** * Refreshes the resource to sync with Azure. - * + * * @param context The context to associate with this operation. * @return the refreshed resource. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroupList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroupList.java index 037c1991ff6e3..fcb9cf9fd1b24 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroupList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroupList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** List of device security groups. */ +/** + * List of device security groups. + */ @Fluent public final class DeviceSecurityGroupList { /* @@ -24,13 +26,15 @@ public final class DeviceSecurityGroupList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DeviceSecurityGroupList class. */ + /** + * Creates an instance of DeviceSecurityGroupList class. + */ public DeviceSecurityGroupList() { } /** * Get the value property: List of device security group objects. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: List of device security group objects. - * + * * @param value the value value to set. * @return the DeviceSecurityGroupList object itself. */ @@ -50,7 +54,7 @@ public DeviceSecurityGroupList withValue(List value) { /** * Get the nextLink property: The URI to fetch the next page. - * + * * @return the nextLink value. */ public String nextLink() { @@ -59,7 +63,7 @@ public String nextLink() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroups.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroups.java index 83cc69b063b73..79f7aa53697dc 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroups.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DeviceSecurityGroups.java @@ -8,11 +8,13 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of DeviceSecurityGroups. */ +/** + * Resource collection API of DeviceSecurityGroups. + */ public interface DeviceSecurityGroups { /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -23,7 +25,7 @@ public interface DeviceSecurityGroups { /** * Use this method get the list of device security groups for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -35,10 +37,10 @@ public interface DeviceSecurityGroups { /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -49,10 +51,10 @@ public interface DeviceSecurityGroups { /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -62,25 +64,25 @@ public interface DeviceSecurityGroups { /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return the {@link Response}. */ - Response deleteByResourceGroupWithResponse( - String resourceId, String deviceSecurityGroupName, Context context); + Response deleteByResourceGroupWithResponse(String resourceId, String deviceSecurityGroupName, + Context context); /** * User this method to deletes the device security group. - * + * * @param resourceId The identifier of the resource. * @param deviceSecurityGroupName The name of the device security group. Note that the name of the device security - * group is case insensitive. + * group is case insensitive. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. @@ -89,7 +91,7 @@ Response deleteByResourceGroupWithResponse( /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -100,7 +102,7 @@ Response deleteByResourceGroupWithResponse( /** * Use this method to get the device security group for the specified IoT Hub resource. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -112,7 +114,7 @@ Response deleteByResourceGroupWithResponse( /** * User this method to deletes the device security group. - * + * * @param id the resource ID. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. @@ -122,7 +124,7 @@ Response deleteByResourceGroupWithResponse( /** * User this method to deletes the device security group. - * + * * @param id the resource ID. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -134,7 +136,7 @@ Response deleteByResourceGroupWithResponse( /** * Begins definition for a new DeviceSecurityGroup resource. - * + * * @param name resource name. * @return the first stage of the new DeviceSecurityGroup definition. */ diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DirectMethodInvokesNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DirectMethodInvokesNotInAllowedRange.java index 2a932e08c998c..c58b67522e473 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DirectMethodInvokesNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DirectMethodInvokesNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of direct method invokes is not in allowed range. */ +/** + * Number of direct method invokes is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("DirectMethodInvokesNotInAllowedRange") @Fluent public final class DirectMethodInvokesNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of DirectMethodInvokesNotInAllowedRange class. */ + /** + * Creates an instance of DirectMethodInvokesNotInAllowedRange class. + */ public DirectMethodInvokesNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DirectMethodInvokesNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DirectMethodInvokesNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DirectMethodInvokesNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public DirectMethodInvokesNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public DirectMethodInvokesNotInAllowedRange withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Direction.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Direction.java index 9042afdbef2e8..fa058da533dba 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Direction.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Direction.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The rule's direction. */ +/** + * The rule's direction. + */ public final class Direction extends ExpandableStringEnum { - /** Static value Inbound for Direction. */ + /** + * Static value Inbound for Direction. + */ public static final Direction INBOUND = fromString("Inbound"); - /** Static value Outbound for Direction. */ + /** + * Static value Outbound for Direction. + */ public static final Direction OUTBOUND = fromString("Outbound"); /** * Creates a new instance of Direction value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public Direction() { /** * Creates or finds a Direction from its string representation. - * + * * @param name a name to look for. * @return the corresponding Direction. */ @@ -38,7 +44,7 @@ public static Direction fromString(String name) { /** * Gets known Direction values. - * + * * @return known Direction values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolution.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolution.java index ffb98c4d786db..88a44bda8f142 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolution.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolution.java @@ -6,67 +6,69 @@ import com.azure.resourcemanager.security.fluent.models.DiscoveredSecuritySolutionInner; -/** An immutable client-side representation of DiscoveredSecuritySolution. */ +/** + * An immutable client-side representation of DiscoveredSecuritySolution. + */ public interface DiscoveredSecuritySolution { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: Location where the resource is stored. - * + * * @return the location value. */ String location(); /** * Gets the securityFamily property: The security family of the discovered solution. - * + * * @return the securityFamily value. */ SecurityFamily securityFamily(); /** * Gets the offer property: The security solutions' image offer. - * + * * @return the offer value. */ String offer(); /** * Gets the publisher property: The security solutions' image publisher. - * + * * @return the publisher value. */ String publisher(); /** * Gets the sku property: The security solutions' image sku. - * + * * @return the sku value. */ String sku(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.DiscoveredSecuritySolutionInner object. - * + * * @return the inner object. */ DiscoveredSecuritySolutionInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolutionList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolutionList.java index b20c1a5413ad9..bd6bf1354a239 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolutionList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/DiscoveredSecuritySolutionList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The DiscoveredSecuritySolutionList model. */ +/** + * The DiscoveredSecuritySolutionList model. + */ @Fluent public final class DiscoveredSecuritySolutionList { /* @@ -24,13 +26,15 @@ public final class DiscoveredSecuritySolutionList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of DiscoveredSecuritySolutionList class. */ + /** + * Creates an instance of DiscoveredSecuritySolutionList class. + */ public DiscoveredSecuritySolutionList() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: The value property. - * + * * @param value the value value to set. * @return the DiscoveredSecuritySolutionList object itself. */ @@ -50,7 +54,7 @@ public DiscoveredSecuritySolutionList withValue(List list(); /** * Gets a list of discovered Security Solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of discovered Security Solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of discovered Security Solutions for the subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ PagedIterable listByHomeRegion(String ascLocation); /** * Gets a list of discovered Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a list of discovered Security Solutions for the subscription and location as paginated response with - * {@link PagedIterable}. + * {@link PagedIterable}. */ PagedIterable listByHomeRegion(String ascLocation, Context context); /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -73,16 +75,16 @@ public interface DiscoveredSecuritySolutions { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific discovered Security Solution along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String ascLocation, String discoveredSecuritySolutionName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String discoveredSecuritySolutionName, Context context); /** * Gets a specific discovered Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param discoveredSecuritySolutionName Name of a discovered security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EffectiveNetworkSecurityGroups.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EffectiveNetworkSecurityGroups.java index de51e989459e7..d9f78c61a3497 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EffectiveNetworkSecurityGroups.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EffectiveNetworkSecurityGroups.java @@ -8,7 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** Describes the Network Security Groups effective on a network interface. */ +/** + * Describes the Network Security Groups effective on a network interface. + */ @Fluent public final class EffectiveNetworkSecurityGroups { /* @@ -23,13 +25,15 @@ public final class EffectiveNetworkSecurityGroups { @JsonProperty(value = "networkSecurityGroups") private List networkSecurityGroups; - /** Creates an instance of EffectiveNetworkSecurityGroups class. */ + /** + * Creates an instance of EffectiveNetworkSecurityGroups class. + */ public EffectiveNetworkSecurityGroups() { } /** * Get the networkInterface property: The Azure resource ID of the network interface. - * + * * @return the networkInterface value. */ public String networkInterface() { @@ -38,7 +42,7 @@ public String networkInterface() { /** * Set the networkInterface property: The Azure resource ID of the network interface. - * + * * @param networkInterface the networkInterface value to set. * @return the EffectiveNetworkSecurityGroups object itself. */ @@ -49,7 +53,7 @@ public EffectiveNetworkSecurityGroups withNetworkInterface(String networkInterfa /** * Get the networkSecurityGroups property: The Network Security Groups effective on the network interface. - * + * * @return the networkSecurityGroups value. */ public List networkSecurityGroups() { @@ -58,7 +62,7 @@ public List networkSecurityGroups() { /** * Set the networkSecurityGroups property: The Network Security Groups effective on the network interface. - * + * * @param networkSecurityGroups the networkSecurityGroups value to set. * @return the EffectiveNetworkSecurityGroups object itself. */ @@ -69,7 +73,7 @@ public EffectiveNetworkSecurityGroups withNetworkSecurityGroups(List net /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EndOfSupportStatus.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EndOfSupportStatus.java index ffd0f46c7c4fc..8d83a3237130d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EndOfSupportStatus.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EndOfSupportStatus.java @@ -8,27 +8,39 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** End of support status. */ +/** + * End of support status. + */ public final class EndOfSupportStatus extends ExpandableStringEnum { - /** Static value None for EndOfSupportStatus. */ + /** + * Static value None for EndOfSupportStatus. + */ public static final EndOfSupportStatus NONE = fromString("None"); - /** Static value noLongerSupported for EndOfSupportStatus. */ + /** + * Static value noLongerSupported for EndOfSupportStatus. + */ public static final EndOfSupportStatus NO_LONGER_SUPPORTED = fromString("noLongerSupported"); - /** Static value versionNoLongerSupported for EndOfSupportStatus. */ + /** + * Static value versionNoLongerSupported for EndOfSupportStatus. + */ public static final EndOfSupportStatus VERSION_NO_LONGER_SUPPORTED = fromString("versionNoLongerSupported"); - /** Static value upcomingNoLongerSupported for EndOfSupportStatus. */ + /** + * Static value upcomingNoLongerSupported for EndOfSupportStatus. + */ public static final EndOfSupportStatus UPCOMING_NO_LONGER_SUPPORTED = fromString("upcomingNoLongerSupported"); - /** Static value upcomingVersionNoLongerSupported for EndOfSupportStatus. */ - public static final EndOfSupportStatus UPCOMING_VERSION_NO_LONGER_SUPPORTED = - fromString("upcomingVersionNoLongerSupported"); + /** + * Static value upcomingVersionNoLongerSupported for EndOfSupportStatus. + */ + public static final EndOfSupportStatus UPCOMING_VERSION_NO_LONGER_SUPPORTED + = fromString("upcomingVersionNoLongerSupported"); /** * Creates a new instance of EndOfSupportStatus value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -37,7 +49,7 @@ public EndOfSupportStatus() { /** * Creates or finds a EndOfSupportStatus from its string representation. - * + * * @param name a name to look for. * @return the corresponding EndOfSupportStatus. */ @@ -48,7 +60,7 @@ public static EndOfSupportStatus fromString(String name) { /** * Gets known EndOfSupportStatus values. - * + * * @return known EndOfSupportStatus values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Enforce.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Enforce.java new file mode 100644 index 0000000000000..491f1e20353f6 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Enforce.java @@ -0,0 +1,55 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.util.ExpandableStringEnum; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Collection; + +/** + * If set to "False", it allows the descendants of this scope to override the pricing configuration set on this scope + * (allows setting inherited="False"). If set to "True", it prevents overrides and forces this pricing configuration on + * all the descendants of this scope. This field is only available for subscription-level pricing. + */ +public final class Enforce extends ExpandableStringEnum { + /** + * Static value False for Enforce. + */ + public static final Enforce FALSE = fromString("False"); + + /** + * Static value True for Enforce. + */ + public static final Enforce TRUE = fromString("True"); + + /** + * Creates a new instance of Enforce value. + * + * @deprecated Use the {@link #fromString(String)} factory method. + */ + @Deprecated + public Enforce() { + } + + /** + * Creates or finds a Enforce from its string representation. + * + * @param name a name to look for. + * @return the corresponding Enforce. + */ + @JsonCreator + public static Enforce fromString(String name) { + return fromString(name, Enforce.class); + } + + /** + * Gets known Enforce values. + * + * @return known Enforce values. + */ + public static Collection values() { + return values(Enforce.class); + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementMode.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementMode.java index 6235187ef4705..85e1cdc5ccc4a 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementMode.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementMode.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The application control policy enforcement/protection mode of the machine group. */ +/** + * The application control policy enforcement/protection mode of the machine group. + */ public final class EnforcementMode extends ExpandableStringEnum { - /** Static value Audit for EnforcementMode. */ + /** + * Static value Audit for EnforcementMode. + */ public static final EnforcementMode AUDIT = fromString("Audit"); - /** Static value Enforce for EnforcementMode. */ + /** + * Static value Enforce for EnforcementMode. + */ public static final EnforcementMode ENFORCE = fromString("Enforce"); - /** Static value None for EnforcementMode. */ + /** + * Static value None for EnforcementMode. + */ public static final EnforcementMode NONE = fromString("None"); /** * Creates a new instance of EnforcementMode value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public EnforcementMode() { /** * Creates or finds a EnforcementMode from its string representation. - * + * * @param name a name to look for. * @return the corresponding EnforcementMode. */ @@ -41,7 +49,7 @@ public static EnforcementMode fromString(String name) { /** * Gets known EnforcementMode values. - * + * * @return known EnforcementMode values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementSupport.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementSupport.java index 89f498dd851a8..42bd2ade16c91 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementSupport.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnforcementSupport.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The machine supportability of Enforce feature. */ +/** + * The machine supportability of Enforce feature. + */ public final class EnforcementSupport extends ExpandableStringEnum { - /** Static value Supported for EnforcementSupport. */ + /** + * Static value Supported for EnforcementSupport. + */ public static final EnforcementSupport SUPPORTED = fromString("Supported"); - /** Static value NotSupported for EnforcementSupport. */ + /** + * Static value NotSupported for EnforcementSupport. + */ public static final EnforcementSupport NOT_SUPPORTED = fromString("NotSupported"); - /** Static value Unknown for EnforcementSupport. */ + /** + * Static value Unknown for EnforcementSupport. + */ public static final EnforcementSupport UNKNOWN = fromString("Unknown"); /** * Creates a new instance of EnforcementSupport value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public EnforcementSupport() { /** * Creates or finds a EnforcementSupport from its string representation. - * + * * @param name a name to look for. * @return the corresponding EnforcementSupport. */ @@ -41,7 +49,7 @@ public static EnforcementSupport fromString(String name) { /** * Gets known EnforcementSupport values. - * + * * @return known EnforcementSupport values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentData.java index 105f62c59a09e..02b1a6a87a3ba 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentData.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The security connector environment data. */ +/** + * The security connector environment data. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -21,17 +23,18 @@ @JsonSubTypes.Type(name = "GcpProject", value = GcpProjectEnvironmentData.class), @JsonSubTypes.Type(name = "GithubScope", value = GithubScopeEnvironmentData.class), @JsonSubTypes.Type(name = "AzureDevOpsScope", value = AzureDevOpsScopeEnvironmentData.class), - @JsonSubTypes.Type(name = "GitlabScope", value = GitlabScopeEnvironmentData.class) -}) + @JsonSubTypes.Type(name = "GitlabScope", value = GitlabScopeEnvironmentData.class) }) @Immutable public class EnvironmentData { - /** Creates an instance of EnvironmentData class. */ + /** + * Creates an instance of EnvironmentData class. + */ public EnvironmentData() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentDetails.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentDetails.java index 4bc4556cbae7e..2b66a9a895d51 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentDetails.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentDetails.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The environment details of the resource. */ +/** + * The environment details of the resource. + */ @Fluent public final class EnvironmentDetails { /* @@ -42,14 +44,16 @@ public final class EnvironmentDetails { @JsonProperty(value = "tenantId") private String tenantId; - /** Creates an instance of EnvironmentDetails class. */ + /** + * Creates an instance of EnvironmentDetails class. + */ public EnvironmentDetails() { } /** - * Get the nativeResourceId property: The native resource id of the resource (in case of Azure - the resource Id, in - * case of MC - the native resource id). - * + * Get the nativeResourceId property: The native resource id of the resource (in case of Azure - the resource Id, + * in case of MC - the native resource id). + * * @return the nativeResourceId value. */ public String nativeResourceId() { @@ -57,9 +61,9 @@ public String nativeResourceId() { } /** - * Set the nativeResourceId property: The native resource id of the resource (in case of Azure - the resource Id, in - * case of MC - the native resource id). - * + * Set the nativeResourceId property: The native resource id of the resource (in case of Azure - the resource Id, + * in case of MC - the native resource id). + * * @param nativeResourceId the nativeResourceId value to set. * @return the EnvironmentDetails object itself. */ @@ -71,7 +75,7 @@ public EnvironmentDetails withNativeResourceId(String nativeResourceId) { /** * Get the environmentHierarchyId property: The hierarchy id of the connector (in case of Azure - the subscription * Id, in case of MC - the hierarchyId id). - * + * * @return the environmentHierarchyId value. */ public String environmentHierarchyId() { @@ -81,7 +85,7 @@ public String environmentHierarchyId() { /** * Set the environmentHierarchyId property: The hierarchy id of the connector (in case of Azure - the subscription * Id, in case of MC - the hierarchyId id). - * + * * @param environmentHierarchyId the environmentHierarchyId value to set. * @return the EnvironmentDetails object itself. */ @@ -93,7 +97,7 @@ public EnvironmentDetails withEnvironmentHierarchyId(String environmentHierarchy /** * Get the organizationalHierarchyId property: The organizational hierarchy id of the connector (in case of Azure - * the subscription Id, in case of MC - the organizational hierarchyId id). - * + * * @return the organizationalHierarchyId value. */ public String organizationalHierarchyId() { @@ -103,7 +107,7 @@ public String organizationalHierarchyId() { /** * Set the organizationalHierarchyId property: The organizational hierarchy id of the connector (in case of Azure - * the subscription Id, in case of MC - the organizational hierarchyId id). - * + * * @param organizationalHierarchyId the organizationalHierarchyId value to set. * @return the EnvironmentDetails object itself. */ @@ -114,7 +118,7 @@ public EnvironmentDetails withOrganizationalHierarchyId(String organizationalHie /** * Get the subscriptionId property: The subscription Id. - * + * * @return the subscriptionId value. */ public String subscriptionId() { @@ -123,7 +127,7 @@ public String subscriptionId() { /** * Set the subscriptionId property: The subscription Id. - * + * * @param subscriptionId the subscriptionId value to set. * @return the EnvironmentDetails object itself. */ @@ -134,7 +138,7 @@ public EnvironmentDetails withSubscriptionId(String subscriptionId) { /** * Get the tenantId property: The tenant Id. - * + * * @return the tenantId value. */ public String tenantId() { @@ -143,7 +147,7 @@ public String tenantId() { /** * Set the tenantId property: The tenant Id. - * + * * @param tenantId the tenantId value to set. * @return the EnvironmentDetails object itself. */ @@ -154,7 +158,7 @@ public EnvironmentDetails withTenantId(String tenantId) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentType.java index c70a011dfa98d..406dc205abf54 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EnvironmentType.java @@ -8,26 +8,38 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The type of the environment data. */ +/** + * The type of the environment data. + */ public final class EnvironmentType extends ExpandableStringEnum { - /** Static value AwsAccount for EnvironmentType. */ + /** + * Static value AwsAccount for EnvironmentType. + */ public static final EnvironmentType AWS_ACCOUNT = fromString("AwsAccount"); - /** Static value GcpProject for EnvironmentType. */ + /** + * Static value GcpProject for EnvironmentType. + */ public static final EnvironmentType GCP_PROJECT = fromString("GcpProject"); - /** Static value GithubScope for EnvironmentType. */ + /** + * Static value GithubScope for EnvironmentType. + */ public static final EnvironmentType GITHUB_SCOPE = fromString("GithubScope"); - /** Static value AzureDevOpsScope for EnvironmentType. */ + /** + * Static value AzureDevOpsScope for EnvironmentType. + */ public static final EnvironmentType AZURE_DEV_OPS_SCOPE = fromString("AzureDevOpsScope"); - /** Static value GitlabScope for EnvironmentType. */ + /** + * Static value GitlabScope for EnvironmentType. + */ public static final EnvironmentType GITLAB_SCOPE = fromString("GitlabScope"); /** * Creates a new instance of EnvironmentType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -36,7 +48,7 @@ public EnvironmentType() { /** * Creates or finds a EnvironmentType from its string representation. - * + * * @param name a name to look for. * @return the corresponding EnvironmentType. */ @@ -47,7 +59,7 @@ public static EnvironmentType fromString(String name) { /** * Gets known EnvironmentType values. - * + * * @return known EnvironmentType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Etag.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Etag.java index 0e0178c4057fc..bf6efadba6151 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Etag.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Etag.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Entity tag is used for comparing two or more entities from the same requested resource. */ +/** + * Entity tag is used for comparing two or more entities from the same requested resource. + */ @Fluent public class Etag { /* @@ -16,13 +18,15 @@ public class Etag { @JsonProperty(value = "etag") private String etag; - /** Creates an instance of Etag class. */ + /** + * Creates an instance of Etag class. + */ public Etag() { } /** * Get the etag property: Entity tag is used for comparing two or more entities from the same requested resource. - * + * * @return the etag value. */ public String etag() { @@ -31,7 +35,7 @@ public String etag() { /** * Set the etag property: Entity tag is used for comparing two or more entities from the same requested resource. - * + * * @param etag the etag value to set. * @return the Etag object itself. */ @@ -42,7 +46,7 @@ public Etag withEtag(String etag) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EventSource.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EventSource.java index 39678faa99075..ac6777138c5ec 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EventSource.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/EventSource.java @@ -8,45 +8,79 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** A valid event source type. */ +/** + * A valid event source type. + */ public final class EventSource extends ExpandableStringEnum { - /** Static value Assessments for EventSource. */ + /** + * Static value Assessments for EventSource. + */ public static final EventSource ASSESSMENTS = fromString("Assessments"); - /** Static value AssessmentsSnapshot for EventSource. */ + /** + * Static value AssessmentsSnapshot for EventSource. + */ public static final EventSource ASSESSMENTS_SNAPSHOT = fromString("AssessmentsSnapshot"); - /** Static value SubAssessments for EventSource. */ + /** + * Static value SubAssessments for EventSource. + */ public static final EventSource SUB_ASSESSMENTS = fromString("SubAssessments"); - /** Static value SubAssessmentsSnapshot for EventSource. */ + /** + * Static value SubAssessmentsSnapshot for EventSource. + */ public static final EventSource SUB_ASSESSMENTS_SNAPSHOT = fromString("SubAssessmentsSnapshot"); - /** Static value Alerts for EventSource. */ + /** + * Static value Alerts for EventSource. + */ public static final EventSource ALERTS = fromString("Alerts"); - /** Static value SecureScores for EventSource. */ + /** + * Static value SecureScores for EventSource. + */ public static final EventSource SECURE_SCORES = fromString("SecureScores"); - /** Static value SecureScoresSnapshot for EventSource. */ + /** + * Static value SecureScoresSnapshot for EventSource. + */ public static final EventSource SECURE_SCORES_SNAPSHOT = fromString("SecureScoresSnapshot"); - /** Static value SecureScoreControls for EventSource. */ + /** + * Static value SecureScoreControls for EventSource. + */ public static final EventSource SECURE_SCORE_CONTROLS = fromString("SecureScoreControls"); - /** Static value SecureScoreControlsSnapshot for EventSource. */ + /** + * Static value SecureScoreControlsSnapshot for EventSource. + */ public static final EventSource SECURE_SCORE_CONTROLS_SNAPSHOT = fromString("SecureScoreControlsSnapshot"); - /** Static value RegulatoryComplianceAssessment for EventSource. */ + /** + * Static value RegulatoryComplianceAssessment for EventSource. + */ public static final EventSource REGULATORY_COMPLIANCE_ASSESSMENT = fromString("RegulatoryComplianceAssessment"); - /** Static value RegulatoryComplianceAssessmentSnapshot for EventSource. */ - public static final EventSource REGULATORY_COMPLIANCE_ASSESSMENT_SNAPSHOT = - fromString("RegulatoryComplianceAssessmentSnapshot"); + /** + * Static value RegulatoryComplianceAssessmentSnapshot for EventSource. + */ + public static final EventSource REGULATORY_COMPLIANCE_ASSESSMENT_SNAPSHOT + = fromString("RegulatoryComplianceAssessmentSnapshot"); + + /** + * Static value AttackPaths for EventSource. + */ + public static final EventSource ATTACK_PATHS = fromString("AttackPaths"); + + /** + * Static value AttackPathsSnapshot for EventSource. + */ + public static final EventSource ATTACK_PATHS_SNAPSHOT = fromString("AttackPathsSnapshot"); /** * Creates a new instance of EventSource value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -55,7 +89,7 @@ public EventSource() { /** * Creates or finds a EventSource from its string representation. - * + * * @param name a name to look for. * @return the corresponding EventSource. */ @@ -66,7 +100,7 @@ public static EventSource fromString(String name) { /** * Gets known EventSource values. - * + * * @return known EventSource values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExecuteGovernanceRuleParams.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExecuteGovernanceRuleParams.java index 985f61f947582..3d62749f52d4f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExecuteGovernanceRuleParams.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExecuteGovernanceRuleParams.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** Governance rule execution parameters. */ +/** + * Governance rule execution parameters. + */ @Fluent public final class ExecuteGovernanceRuleParams { /* @@ -16,13 +18,15 @@ public final class ExecuteGovernanceRuleParams { @JsonProperty(value = "override") private Boolean override; - /** Creates an instance of ExecuteGovernanceRuleParams class. */ + /** + * Creates an instance of ExecuteGovernanceRuleParams class. + */ public ExecuteGovernanceRuleParams() { } /** * Get the override property: Describe if governance rule should be override. - * + * * @return the override value. */ public Boolean override() { @@ -31,7 +35,7 @@ public Boolean override() { /** * Set the override property: Describe if governance rule should be override. - * + * * @param override the override value to set. * @return the ExecuteGovernanceRuleParams object itself. */ @@ -42,7 +46,7 @@ public ExecuteGovernanceRuleParams withOverride(Boolean override) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandControlsEnum.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandControlsEnum.java index 05be0c5ea44ff..346c4c40f240d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandControlsEnum.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandControlsEnum.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for ExpandControlsEnum. */ +/** + * Defines values for ExpandControlsEnum. + */ public final class ExpandControlsEnum extends ExpandableStringEnum { - /** Static value definition for ExpandControlsEnum. */ + /** + * Static value definition for ExpandControlsEnum. + */ public static final ExpandControlsEnum DEFINITION = fromString("definition"); /** * Creates a new instance of ExpandControlsEnum value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ExpandControlsEnum() { /** * Creates or finds a ExpandControlsEnum from its string representation. - * + * * @param name a name to look for. * @return the corresponding ExpandControlsEnum. */ @@ -35,7 +39,7 @@ public static ExpandControlsEnum fromString(String name) { /** * Gets known ExpandControlsEnum values. - * + * * @return known ExpandControlsEnum values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandEnum.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandEnum.java index 2a5584e0fb553..e47ff3a294e62 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandEnum.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExpandEnum.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for ExpandEnum. */ +/** + * Defines values for ExpandEnum. + */ public final class ExpandEnum extends ExpandableStringEnum { - /** Static value links for ExpandEnum. */ + /** + * Static value links for ExpandEnum. + */ public static final ExpandEnum LINKS = fromString("links"); - /** Static value metadata for ExpandEnum. */ + /** + * Static value metadata for ExpandEnum. + */ public static final ExpandEnum METADATA = fromString("metadata"); /** * Creates a new instance of ExpandEnum value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -27,7 +33,7 @@ public ExpandEnum() { /** * Creates or finds a ExpandEnum from its string representation. - * + * * @param name a name to look for. * @return the corresponding ExpandEnum. */ @@ -38,7 +44,7 @@ public static ExpandEnum fromString(String name) { /** * Gets known ExpandEnum values. - * + * * @return known ExpandEnum values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExportData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExportData.java index d24f2e1271f8d..b44944af9beff 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExportData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExportData.java @@ -8,14 +8,18 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** Defines values for ExportData. */ +/** + * Defines values for ExportData. + */ public final class ExportData extends ExpandableStringEnum { - /** Static value RawEvents for ExportData. */ + /** + * Static value RawEvents for ExportData. + */ public static final ExportData RAW_EVENTS = fromString("RawEvents"); /** * Creates a new instance of ExportData value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -24,7 +28,7 @@ public ExportData() { /** * Creates or finds a ExportData from its string representation. - * + * * @param name a name to look for. * @return the corresponding ExportData. */ @@ -35,7 +39,7 @@ public static ExportData fromString(String name) { /** * Gets known ExportData values. - * + * * @return known ExportData values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Extension.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Extension.java index 08661bf274768..0e35dfc2e31c5 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Extension.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/Extension.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Map; -/** A plan's extension properties. */ +/** + * A plan's extension properties. + */ @Fluent public final class Extension { /* @@ -48,9 +50,11 @@ public final class Extension { * Optional. A status describing the success/failure of the extension's enablement/disablement operation. */ @JsonProperty(value = "operationStatus", access = JsonProperty.Access.WRITE_ONLY) - private OperationStatus operationStatus; + private OperationStatusAutoGenerated operationStatus; - /** Creates an instance of Extension class. */ + /** + * Creates an instance of Extension class. + */ public Extension() { } @@ -58,16 +62,18 @@ public Extension() { * Get the name property: The extension name. Supported values are: * <br><br>**AgentlessDiscoveryForKubernetes** - API-based discovery of information about Kubernetes * cluster architecture, workload objects, and setup. Required for Kubernetes inventory, identity and network - * exposure detection, attack path analysis and risk hunting as part of the cloud security explorer. Available for - * CloudPosture plan.<br><br>**OnUploadMalwareScanning** - Limits the GB to be scanned per month for - * each storage account within the subscription. Once this limit reached on a given storage account, Blobs won't be - * scanned during current calendar month. Available for StorageAccounts - * plan.<br><br>**SensitiveDataDiscovery** - Sensitive data discovery identifies Blob storage container - * with sensitive data such as credentials, credit cards, and more, to help prioritize and investigate security - * events. Available for StorageAccounts and CloudPosture - * plans.<br><br>**ContainerRegistriesVulnerabilityAssessments** - Provides vulnerability management for - * images stored in your container registries. Available for CloudPosture and Containers plans. - * + * exposure detection, attack path analysis and risk hunting as part of the cloud security explorer. + * Available for CloudPosture plan.<br><br>**OnUploadMalwareScanning** - Limits the GB to be scanned + * per month for each storage account within the subscription. Once this limit reached on a given storage account, + * Blobs won't be scanned during current calendar month. + * Available for StorageAccounts plan.<br><br>**SensitiveDataDiscovery** - Sensitive data discovery + * identifies Blob storage container with sensitive data such as credentials, credit cards, and more, to help + * prioritize and investigate security events. + * Available for StorageAccounts and CloudPosture + * plans.<br><br>**ContainerRegistriesVulnerabilityAssessments** - Provides vulnerability management + * for images stored in your container registries. + * Available for CloudPosture and Containers plans. + * * @return the name value. */ public String name() { @@ -78,16 +84,18 @@ public String name() { * Set the name property: The extension name. Supported values are: * <br><br>**AgentlessDiscoveryForKubernetes** - API-based discovery of information about Kubernetes * cluster architecture, workload objects, and setup. Required for Kubernetes inventory, identity and network - * exposure detection, attack path analysis and risk hunting as part of the cloud security explorer. Available for - * CloudPosture plan.<br><br>**OnUploadMalwareScanning** - Limits the GB to be scanned per month for - * each storage account within the subscription. Once this limit reached on a given storage account, Blobs won't be - * scanned during current calendar month. Available for StorageAccounts - * plan.<br><br>**SensitiveDataDiscovery** - Sensitive data discovery identifies Blob storage container - * with sensitive data such as credentials, credit cards, and more, to help prioritize and investigate security - * events. Available for StorageAccounts and CloudPosture - * plans.<br><br>**ContainerRegistriesVulnerabilityAssessments** - Provides vulnerability management for - * images stored in your container registries. Available for CloudPosture and Containers plans. - * + * exposure detection, attack path analysis and risk hunting as part of the cloud security explorer. + * Available for CloudPosture plan.<br><br>**OnUploadMalwareScanning** - Limits the GB to be scanned + * per month for each storage account within the subscription. Once this limit reached on a given storage account, + * Blobs won't be scanned during current calendar month. + * Available for StorageAccounts plan.<br><br>**SensitiveDataDiscovery** - Sensitive data discovery + * identifies Blob storage container with sensitive data such as credentials, credit cards, and more, to help + * prioritize and investigate security events. + * Available for StorageAccounts and CloudPosture + * plans.<br><br>**ContainerRegistriesVulnerabilityAssessments** - Provides vulnerability management + * for images stored in your container registries. + * Available for CloudPosture and Containers plans. + * * @param name the name value to set. * @return the Extension object itself. */ @@ -98,7 +106,7 @@ public Extension withName(String name) { /** * Get the isEnabled property: Indicates whether the extension is enabled. - * + * * @return the isEnabled value. */ public IsEnabled isEnabled() { @@ -107,7 +115,7 @@ public IsEnabled isEnabled() { /** * Set the isEnabled property: Indicates whether the extension is enabled. - * + * * @param isEnabled the isEnabled value to set. * @return the Extension object itself. */ @@ -118,7 +126,7 @@ public Extension withIsEnabled(IsEnabled isEnabled) { /** * Get the additionalExtensionProperties property: Property values associated with the extension. - * + * * @return the additionalExtensionProperties value. */ public Map additionalExtensionProperties() { @@ -127,7 +135,7 @@ public Map additionalExtensionProperties() { /** * Set the additionalExtensionProperties property: Property values associated with the extension. - * + * * @param additionalExtensionProperties the additionalExtensionProperties value to set. * @return the Extension object itself. */ @@ -139,16 +147,16 @@ public Extension withAdditionalExtensionProperties(Map additiona /** * Get the operationStatus property: Optional. A status describing the success/failure of the extension's * enablement/disablement operation. - * + * * @return the operationStatus value. */ - public OperationStatus operationStatus() { + public OperationStatusAutoGenerated operationStatus() { return this.operationStatus; } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { @@ -157,9 +165,8 @@ public void validate() { .logExceptionAsError(new IllegalArgumentException("Missing required property name in model Extension")); } if (isEnabled() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException("Missing required property isEnabled in model Extension")); + throw LOGGER.logExceptionAsError( + new IllegalArgumentException("Missing required property isEnabled in model Extension")); } if (operationStatus() != null) { operationStatus().validate(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolution.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolution.java index 59722076e72b4..87dfa11c82686 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolution.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolution.java @@ -6,39 +6,41 @@ import com.azure.resourcemanager.security.fluent.models.ExternalSecuritySolutionInner; -/** An immutable client-side representation of ExternalSecuritySolution. */ +/** + * An immutable client-side representation of ExternalSecuritySolution. + */ public interface ExternalSecuritySolution { /** * Gets the id property: Fully qualified resource Id for the resource. - * + * * @return the id value. */ String id(); /** * Gets the name property: The name of the resource. - * + * * @return the name value. */ String name(); /** * Gets the type property: The type of the resource. - * + * * @return the type value. */ String type(); /** * Gets the location property: Location where the resource is stored. - * + * * @return the location value. */ String location(); /** * Gets the inner com.azure.resourcemanager.security.fluent.models.ExternalSecuritySolutionInner object. - * + * * @return the inner object. */ ExternalSecuritySolutionInner innerModel(); diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionKind.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionKind.java index cbbc0b7acb37c..974e187fc666b 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionKind.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionKind.java @@ -8,20 +8,28 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The kind of the external solution. */ +/** + * The kind of the external solution. + */ public final class ExternalSecuritySolutionKind extends ExpandableStringEnum { - /** Static value CEF for ExternalSecuritySolutionKind. */ + /** + * Static value CEF for ExternalSecuritySolutionKind. + */ public static final ExternalSecuritySolutionKind CEF = fromString("CEF"); - /** Static value ATA for ExternalSecuritySolutionKind. */ + /** + * Static value ATA for ExternalSecuritySolutionKind. + */ public static final ExternalSecuritySolutionKind ATA = fromString("ATA"); - /** Static value AAD for ExternalSecuritySolutionKind. */ + /** + * Static value AAD for ExternalSecuritySolutionKind. + */ public static final ExternalSecuritySolutionKind AAD = fromString("AAD"); /** * Creates a new instance of ExternalSecuritySolutionKind value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -30,7 +38,7 @@ public ExternalSecuritySolutionKind() { /** * Creates or finds a ExternalSecuritySolutionKind from its string representation. - * + * * @param name a name to look for. * @return the corresponding ExternalSecuritySolutionKind. */ @@ -41,7 +49,7 @@ public static ExternalSecuritySolutionKind fromString(String name) { /** * Gets known ExternalSecuritySolutionKind values. - * + * * @return known ExternalSecuritySolutionKind values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionList.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionList.java index 8afc46f34ad08..0a2ddc31fdc7f 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionList.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutionList.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; -/** The ExternalSecuritySolutionList model. */ +/** + * The ExternalSecuritySolutionList model. + */ @Fluent public final class ExternalSecuritySolutionList { /* @@ -24,13 +26,15 @@ public final class ExternalSecuritySolutionList { @JsonProperty(value = "nextLink", access = JsonProperty.Access.WRITE_ONLY) private String nextLink; - /** Creates an instance of ExternalSecuritySolutionList class. */ + /** + * Creates an instance of ExternalSecuritySolutionList class. + */ public ExternalSecuritySolutionList() { } /** * Get the value property: The value property. - * + * * @return the value value. */ public List value() { @@ -39,7 +43,7 @@ public List value() { /** * Set the value property: The value property. - * + * * @param value the value value to set. * @return the ExternalSecuritySolutionList object itself. */ @@ -50,7 +54,7 @@ public ExternalSecuritySolutionList withValue(List additionalProperties; + @JsonIgnore + private Map additionalProperties; - /** Creates an instance of ExternalSecuritySolutionProperties class. */ + /** + * Creates an instance of ExternalSecuritySolutionProperties class. + */ public ExternalSecuritySolutionProperties() { } /** * Get the deviceVendor property: The deviceVendor property. - * + * * @return the deviceVendor value. */ public String deviceVendor() { @@ -53,7 +58,7 @@ public String deviceVendor() { /** * Set the deviceVendor property: The deviceVendor property. - * + * * @param deviceVendor the deviceVendor value to set. * @return the ExternalSecuritySolutionProperties object itself. */ @@ -64,7 +69,7 @@ public ExternalSecuritySolutionProperties withDeviceVendor(String deviceVendor) /** * Get the deviceType property: The deviceType property. - * + * * @return the deviceType value. */ public String deviceType() { @@ -73,7 +78,7 @@ public String deviceType() { /** * Set the deviceType property: The deviceType property. - * + * * @param deviceType the deviceType value to set. * @return the ExternalSecuritySolutionProperties object itself. */ @@ -84,7 +89,7 @@ public ExternalSecuritySolutionProperties withDeviceType(String deviceType) { /** * Get the workspace property: Represents an OMS workspace to which the solution is connected. - * + * * @return the workspace value. */ public ConnectedWorkspace workspace() { @@ -93,7 +98,7 @@ public ConnectedWorkspace workspace() { /** * Set the workspace property: Represents an OMS workspace to which the solution is connected. - * + * * @param workspace the workspace value to set. * @return the ExternalSecuritySolutionProperties object itself. */ @@ -104,7 +109,7 @@ public ExternalSecuritySolutionProperties withWorkspace(ConnectedWorkspace works /** * Get the additionalProperties property: The solution properties (correspond to the solution kind). - * + * * @return the additionalProperties value. */ @JsonAnyGetter @@ -114,7 +119,7 @@ public Map additionalProperties() { /** * Set the additionalProperties property: The solution properties (correspond to the solution kind). - * + * * @param additionalProperties the additionalProperties value to set. * @return the ExternalSecuritySolutionProperties object itself. */ @@ -133,7 +138,7 @@ void withAdditionalProperties(String key, Object value) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutions.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutions.java index 60dc430b9b7fb..5eb50a8611f43 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutions.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/ExternalSecuritySolutions.java @@ -8,64 +8,66 @@ import com.azure.core.http.rest.Response; import com.azure.core.util.Context; -/** Resource collection API of ExternalSecuritySolutions. */ +/** + * Resource collection API of ExternalSecuritySolutions. + */ public interface ExternalSecuritySolutions { /** * Gets a list of external security solutions for the subscription. - * + * * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external security solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of external security solutions for the subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(); /** * Gets a list of external security solutions for the subscription. - * + * * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external security solutions for the subscription as paginated response with {@link - * PagedIterable}. + * @return a list of external security solutions for the subscription as paginated response with + * {@link PagedIterable}. */ PagedIterable list(Context context); /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedIterable}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedIterable}. */ PagedIterable listByHomeRegion(String ascLocation); /** * Gets a list of external Security Solutions for the subscription and location. - * + * * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. - * @return a list of external Security Solutions for the subscription and location as paginated response with {@link - * PagedIterable}. + * @return a list of external Security Solutions for the subscription and location as paginated response with + * {@link PagedIterable}. */ PagedIterable listByHomeRegion(String ascLocation, Context context); /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @param context The context to associate with this operation. * @throws IllegalArgumentException thrown if parameters fail the validation. @@ -73,16 +75,16 @@ public interface ExternalSecuritySolutions { * @throws RuntimeException all other wrapped checked exceptions if the request fails to be sent. * @return a specific external Security Solution along with {@link Response}. */ - Response getWithResponse( - String resourceGroupName, String ascLocation, String externalSecuritySolutionsName, Context context); + Response getWithResponse(String resourceGroupName, String ascLocation, + String externalSecuritySolutionsName, Context context); /** * Gets a specific external Security Solution. - * + * * @param resourceGroupName The name of the resource group within the user's subscription. The name is case - * insensitive. + * insensitive. * @param ascLocation The location where ASC stores the data of the subscription. can be retrieved from Get - * locations. + * locations. * @param externalSecuritySolutionsName Name of an external security solution. * @throws IllegalArgumentException thrown if parameters fail the validation. * @throws com.azure.core.management.exception.ManagementException thrown if the request is rejected by server. diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FailedLocalLoginsNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FailedLocalLoginsNotInAllowedRange.java index 47e4200079025..9e9243734c011 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FailedLocalLoginsNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FailedLocalLoginsNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of failed local logins is not in allowed range. */ +/** + * Number of failed local logins is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("FailedLocalLoginsNotInAllowedRange") @Fluent public final class FailedLocalLoginsNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of FailedLocalLoginsNotInAllowedRange class. */ + /** + * Creates an instance of FailedLocalLoginsNotInAllowedRange class. + */ public FailedLocalLoginsNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FailedLocalLoginsNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FailedLocalLoginsNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FailedLocalLoginsNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FailedLocalLoginsNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public FailedLocalLoginsNotInAllowedRange withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileType.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileType.java index f6830ab5e8042..3873d1fd88c48 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileType.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileType.java @@ -8,29 +8,43 @@ import com.fasterxml.jackson.annotation.JsonCreator; import java.util.Collection; -/** The type of the file (for Linux files - Executable is used). */ +/** + * The type of the file (for Linux files - Executable is used). + */ public final class FileType extends ExpandableStringEnum { - /** Static value Exe for FileType. */ + /** + * Static value Exe for FileType. + */ public static final FileType EXE = fromString("Exe"); - /** Static value Dll for FileType. */ + /** + * Static value Dll for FileType. + */ public static final FileType DLL = fromString("Dll"); - /** Static value Msi for FileType. */ + /** + * Static value Msi for FileType. + */ public static final FileType MSI = fromString("Msi"); - /** Static value Script for FileType. */ + /** + * Static value Script for FileType. + */ public static final FileType SCRIPT = fromString("Script"); - /** Static value Executable for FileType. */ + /** + * Static value Executable for FileType. + */ public static final FileType EXECUTABLE = fromString("Executable"); - /** Static value Unknown for FileType. */ + /** + * Static value Unknown for FileType. + */ public static final FileType UNKNOWN = fromString("Unknown"); /** * Creates a new instance of FileType value. - * + * * @deprecated Use the {@link #fromString(String)} factory method. */ @Deprecated @@ -39,7 +53,7 @@ public FileType() { /** * Creates or finds a FileType from its string representation. - * + * * @param name a name to look for. * @return the corresponding FileType. */ @@ -50,7 +64,7 @@ public static FileType fromString(String name) { /** * Gets known FileType values. - * + * * @return known FileType values. */ public static Collection values() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileUploadsNotInAllowedRange.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileUploadsNotInAllowedRange.java index 69f8cf762e203..df16679e6723d 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileUploadsNotInAllowedRange.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/FileUploadsNotInAllowedRange.java @@ -9,37 +9,49 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.time.Duration; -/** Number of file uploads is not in allowed range. */ +/** + * Number of file uploads is not in allowed range. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "ruleType") @JsonTypeName("FileUploadsNotInAllowedRange") @Fluent public final class FileUploadsNotInAllowedRange extends TimeWindowCustomAlertRule { - /** Creates an instance of FileUploadsNotInAllowedRange class. */ + /** + * Creates an instance of FileUploadsNotInAllowedRange class. + */ public FileUploadsNotInAllowedRange() { } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FileUploadsNotInAllowedRange withTimeWindowSize(Duration timeWindowSize) { super.withTimeWindowSize(timeWindowSize); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FileUploadsNotInAllowedRange withMinThreshold(int minThreshold) { super.withMinThreshold(minThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FileUploadsNotInAllowedRange withMaxThreshold(int maxThreshold) { super.withMaxThreshold(maxThreshold); return this; } - /** {@inheritDoc} */ + /** + * {@inheritDoc} + */ @Override public FileUploadsNotInAllowedRange withIsEnabled(boolean isEnabled) { super.withIsEnabled(isEnabled); @@ -48,7 +60,7 @@ public FileUploadsNotInAllowedRange withIsEnabled(boolean isEnabled) { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpCredentialsDetailsProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpCredentialsDetailsProperties.java index ee933d133b168..d49551c2b70e3 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpCredentialsDetailsProperties.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpCredentialsDetailsProperties.java @@ -11,8 +11,8 @@ import com.fasterxml.jackson.annotation.JsonTypeName; /** - * GCP cloud account connector based service to service credentials, the credentials are composed of the organization ID - * and a JSON API key (write only). + * GCP cloud account connector based service to service credentials, the credentials are composed of the organization + * ID and a JSON API key (write only). */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "authenticationType") @JsonTypeName("gcpCredentials") @@ -84,13 +84,15 @@ public final class GcpCredentialsDetailsProperties extends AuthenticationDetails @JsonProperty(value = "clientX509CertUrl", required = true) private String clientX509CertUrl; - /** Creates an instance of GcpCredentialsDetailsProperties class. */ + /** + * Creates an instance of GcpCredentialsDetailsProperties class. + */ public GcpCredentialsDetailsProperties() { } /** * Get the organizationId property: The organization ID of the GCP cloud account. - * + * * @return the organizationId value. */ public String organizationId() { @@ -99,7 +101,7 @@ public String organizationId() { /** * Set the organizationId property: The organization ID of the GCP cloud account. - * + * * @param organizationId the organizationId value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -110,7 +112,7 @@ public GcpCredentialsDetailsProperties withOrganizationId(String organizationId) /** * Get the type property: Type field of the API key (write only). - * + * * @return the type value. */ public String type() { @@ -119,7 +121,7 @@ public String type() { /** * Set the type property: Type field of the API key (write only). - * + * * @param type the type value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -130,7 +132,7 @@ public GcpCredentialsDetailsProperties withType(String type) { /** * Get the projectId property: Project ID field of the API key (write only). - * + * * @return the projectId value. */ public String projectId() { @@ -139,7 +141,7 @@ public String projectId() { /** * Set the projectId property: Project ID field of the API key (write only). - * + * * @param projectId the projectId value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -150,7 +152,7 @@ public GcpCredentialsDetailsProperties withProjectId(String projectId) { /** * Get the privateKeyId property: Private key ID field of the API key (write only). - * + * * @return the privateKeyId value. */ public String privateKeyId() { @@ -159,7 +161,7 @@ public String privateKeyId() { /** * Set the privateKeyId property: Private key ID field of the API key (write only). - * + * * @param privateKeyId the privateKeyId value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -170,7 +172,7 @@ public GcpCredentialsDetailsProperties withPrivateKeyId(String privateKeyId) { /** * Get the privateKey property: Private key field of the API key (write only). - * + * * @return the privateKey value. */ public String privateKey() { @@ -179,7 +181,7 @@ public String privateKey() { /** * Set the privateKey property: Private key field of the API key (write only). - * + * * @param privateKey the privateKey value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -190,7 +192,7 @@ public GcpCredentialsDetailsProperties withPrivateKey(String privateKey) { /** * Get the clientEmail property: Client email field of the API key (write only). - * + * * @return the clientEmail value. */ public String clientEmail() { @@ -199,7 +201,7 @@ public String clientEmail() { /** * Set the clientEmail property: Client email field of the API key (write only). - * + * * @param clientEmail the clientEmail value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -210,7 +212,7 @@ public GcpCredentialsDetailsProperties withClientEmail(String clientEmail) { /** * Get the clientId property: Client ID field of the API key (write only). - * + * * @return the clientId value. */ public String clientId() { @@ -219,7 +221,7 @@ public String clientId() { /** * Set the clientId property: Client ID field of the API key (write only). - * + * * @param clientId the clientId value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -230,7 +232,7 @@ public GcpCredentialsDetailsProperties withClientId(String clientId) { /** * Get the authUri property: Auth URI field of the API key (write only). - * + * * @return the authUri value. */ public String authUri() { @@ -239,7 +241,7 @@ public String authUri() { /** * Set the authUri property: Auth URI field of the API key (write only). - * + * * @param authUri the authUri value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -250,7 +252,7 @@ public GcpCredentialsDetailsProperties withAuthUri(String authUri) { /** * Get the tokenUri property: Token URI field of the API key (write only). - * + * * @return the tokenUri value. */ public String tokenUri() { @@ -259,7 +261,7 @@ public String tokenUri() { /** * Set the tokenUri property: Token URI field of the API key (write only). - * + * * @param tokenUri the tokenUri value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -270,7 +272,7 @@ public GcpCredentialsDetailsProperties withTokenUri(String tokenUri) { /** * Get the authProviderX509CertUrl property: Auth provider x509 certificate URL field of the API key (write only). - * + * * @return the authProviderX509CertUrl value. */ public String authProviderX509CertUrl() { @@ -279,7 +281,7 @@ public String authProviderX509CertUrl() { /** * Set the authProviderX509CertUrl property: Auth provider x509 certificate URL field of the API key (write only). - * + * * @param authProviderX509CertUrl the authProviderX509CertUrl value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -290,7 +292,7 @@ public GcpCredentialsDetailsProperties withAuthProviderX509CertUrl(String authPr /** * Get the clientX509CertUrl property: Client x509 certificate URL field of the API key (write only). - * + * * @return the clientX509CertUrl value. */ public String clientX509CertUrl() { @@ -299,7 +301,7 @@ public String clientX509CertUrl() { /** * Set the clientX509CertUrl property: Client x509 certificate URL field of the API key (write only). - * + * * @param clientX509CertUrl the clientX509CertUrl value to set. * @return the GcpCredentialsDetailsProperties object itself. */ @@ -310,77 +312,55 @@ public GcpCredentialsDetailsProperties withClientX509CertUrl(String clientX509Ce /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override public void validate() { super.validate(); if (organizationId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property organizationId in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property organizationId in model GcpCredentialsDetailsProperties")); } if (type() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property type in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property type in model GcpCredentialsDetailsProperties")); } if (projectId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property projectId in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property projectId in model GcpCredentialsDetailsProperties")); } if (privateKeyId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property privateKeyId in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property privateKeyId in model GcpCredentialsDetailsProperties")); } if (privateKey() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property privateKey in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property privateKey in model GcpCredentialsDetailsProperties")); } if (clientEmail() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property clientEmail in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property clientEmail in model GcpCredentialsDetailsProperties")); } if (clientId() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property clientId in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property clientId in model GcpCredentialsDetailsProperties")); } if (authUri() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property authUri in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property authUri in model GcpCredentialsDetailsProperties")); } if (tokenUri() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property tokenUri in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property tokenUri in model GcpCredentialsDetailsProperties")); } if (authProviderX509CertUrl() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property authProviderX509CertUrl in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property authProviderX509CertUrl in model GcpCredentialsDetailsProperties")); } if (clientX509CertUrl() == null) { - throw LOGGER - .logExceptionAsError( - new IllegalArgumentException( - "Missing required property clientX509CertUrl in model GcpCredentialsDetailsProperties")); + throw LOGGER.logExceptionAsError(new IllegalArgumentException( + "Missing required property clientX509CertUrl in model GcpCredentialsDetailsProperties")); } } diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalData.java index 4dabcf13f8202..43504796b61ce 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalData.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The gcpOrganization data. */ +/** + * The gcpOrganization data. + */ @JsonTypeInfo( use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, @@ -18,17 +20,18 @@ @JsonTypeName("GcpOrganizationalData") @JsonSubTypes({ @JsonSubTypes.Type(name = "Organization", value = GcpOrganizationalDataOrganization.class), - @JsonSubTypes.Type(name = "Member", value = GcpOrganizationalDataMember.class) -}) + @JsonSubTypes.Type(name = "Member", value = GcpOrganizationalDataMember.class) }) @Immutable public class GcpOrganizationalData { - /** Creates an instance of GcpOrganizationalData class. */ + /** + * Creates an instance of GcpOrganizationalData class. + */ public GcpOrganizationalData() { } /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataMember.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataMember.java index 7fda4d5a6317f..188ad3f1d0764 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataMember.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataMember.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The gcpOrganization data for the member account. */ +/** + * The gcpOrganization data for the member account. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "organizationMembershipType") @JsonTypeName("Member") @Fluent @@ -26,14 +28,16 @@ public final class GcpOrganizationalDataMember extends GcpOrganizationalData { @JsonProperty(value = "managementProjectNumber") private String managementProjectNumber; - /** Creates an instance of GcpOrganizationalDataMember class. */ + /** + * Creates an instance of GcpOrganizationalDataMember class. + */ public GcpOrganizationalDataMember() { } /** * Get the parentHierarchyId property: If the multi cloud account is not of membership type organization, this will * be the ID of the project's parent. - * + * * @return the parentHierarchyId value. */ public String parentHierarchyId() { @@ -43,7 +47,7 @@ public String parentHierarchyId() { /** * Set the parentHierarchyId property: If the multi cloud account is not of membership type organization, this will * be the ID of the project's parent. - * + * * @param parentHierarchyId the parentHierarchyId value to set. * @return the GcpOrganizationalDataMember object itself. */ @@ -54,7 +58,7 @@ public GcpOrganizationalDataMember withParentHierarchyId(String parentHierarchyI /** * Get the managementProjectNumber property: The GCP management project number from organizational onboarding. - * + * * @return the managementProjectNumber value. */ public String managementProjectNumber() { @@ -63,7 +67,7 @@ public String managementProjectNumber() { /** * Set the managementProjectNumber property: The GCP management project number from organizational onboarding. - * + * * @param managementProjectNumber the managementProjectNumber value to set. * @return the GcpOrganizationalDataMember object itself. */ @@ -74,7 +78,7 @@ public GcpOrganizationalDataMember withManagementProjectNumber(String management /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataOrganization.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataOrganization.java index fc324cde83264..d573f0c0e1274 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataOrganization.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpOrganizationalDataOrganization.java @@ -10,7 +10,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import java.util.List; -/** The gcpOrganization data for the parent account. */ +/** + * The gcpOrganization data for the parent account. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "organizationMembershipType") @JsonTypeName("Organization") @Fluent @@ -40,14 +42,16 @@ public final class GcpOrganizationalDataOrganization extends GcpOrganizationalDa @JsonProperty(value = "organizationName", access = JsonProperty.Access.WRITE_ONLY) private String organizationName; - /** Creates an instance of GcpOrganizationalDataOrganization class. */ + /** + * Creates an instance of GcpOrganizationalDataOrganization class. + */ public GcpOrganizationalDataOrganization() { } /** * Get the excludedProjectNumbers property: If the multi cloud account is of membership type organization, list of * accounts excluded from offering. - * + * * @return the excludedProjectNumbers value. */ public List excludedProjectNumbers() { @@ -57,7 +61,7 @@ public List excludedProjectNumbers() { /** * Set the excludedProjectNumbers property: If the multi cloud account is of membership type organization, list of * accounts excluded from offering. - * + * * @param excludedProjectNumbers the excludedProjectNumbers value to set. * @return the GcpOrganizationalDataOrganization object itself. */ @@ -69,7 +73,7 @@ public GcpOrganizationalDataOrganization withExcludedProjectNumbers(List /** * Get the serviceAccountEmailAddress property: The service account email address which represents the organization * level permissions container. - * + * * @return the serviceAccountEmailAddress value. */ public String serviceAccountEmailAddress() { @@ -79,7 +83,7 @@ public String serviceAccountEmailAddress() { /** * Set the serviceAccountEmailAddress property: The service account email address which represents the organization * level permissions container. - * + * * @param serviceAccountEmailAddress the serviceAccountEmailAddress value to set. * @return the GcpOrganizationalDataOrganization object itself. */ @@ -91,7 +95,7 @@ public GcpOrganizationalDataOrganization withServiceAccountEmailAddress(String s /** * Get the workloadIdentityProviderId property: The GCP workload identity provider id which represents the * permissions required to auto provision security connectors. - * + * * @return the workloadIdentityProviderId value. */ public String workloadIdentityProviderId() { @@ -101,7 +105,7 @@ public String workloadIdentityProviderId() { /** * Set the workloadIdentityProviderId property: The GCP workload identity provider id which represents the * permissions required to auto provision security connectors. - * + * * @param workloadIdentityProviderId the workloadIdentityProviderId value to set. * @return the GcpOrganizationalDataOrganization object itself. */ @@ -112,7 +116,7 @@ public GcpOrganizationalDataOrganization withWorkloadIdentityProviderId(String w /** * Get the organizationName property: GCP organization name. - * + * * @return the organizationName value. */ public String organizationName() { @@ -121,7 +125,7 @@ public String organizationName() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectDetails.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectDetails.java index ac6bbd2acf3f0..45d89bb89da79 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectDetails.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectDetails.java @@ -7,7 +7,9 @@ import com.azure.core.annotation.Fluent; import com.fasterxml.jackson.annotation.JsonProperty; -/** The details about the project represented by the security connector. */ +/** + * The details about the project represented by the security connector. + */ @Fluent public final class GcpProjectDetails { /* @@ -34,13 +36,15 @@ public final class GcpProjectDetails { @JsonProperty(value = "projectName", access = JsonProperty.Access.WRITE_ONLY) private String projectName; - /** Creates an instance of GcpProjectDetails class. */ + /** + * Creates an instance of GcpProjectDetails class. + */ public GcpProjectDetails() { } /** * Get the projectNumber property: The unique GCP Project number. - * + * * @return the projectNumber value. */ public String projectNumber() { @@ -49,7 +53,7 @@ public String projectNumber() { /** * Set the projectNumber property: The unique GCP Project number. - * + * * @param projectNumber the projectNumber value to set. * @return the GcpProjectDetails object itself. */ @@ -60,7 +64,7 @@ public GcpProjectDetails withProjectNumber(String projectNumber) { /** * Get the projectId property: The GCP Project id. - * + * * @return the projectId value. */ public String projectId() { @@ -69,7 +73,7 @@ public String projectId() { /** * Set the projectId property: The GCP Project id. - * + * * @param projectId the projectId value to set. * @return the GcpProjectDetails object itself. */ @@ -80,7 +84,7 @@ public GcpProjectDetails withProjectId(String projectId) { /** * Get the workloadIdentityPoolId property: The GCP workload identity federation pool id. - * + * * @return the workloadIdentityPoolId value. */ public String workloadIdentityPoolId() { @@ -89,7 +93,7 @@ public String workloadIdentityPoolId() { /** * Get the projectName property: GCP project name. - * + * * @return the projectName value. */ public String projectName() { @@ -98,7 +102,7 @@ public String projectName() { /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ public void validate() { diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectEnvironmentData.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectEnvironmentData.java index 68697c94c1f44..84cacbffcf5aa 100644 --- a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectEnvironmentData.java +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GcpProjectEnvironmentData.java @@ -9,7 +9,9 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeName; -/** The GCP project connector environment data. */ +/** + * The GCP project connector environment data. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "environmentType") @JsonTypeName("GcpProject") @Fluent @@ -26,13 +28,21 @@ public final class GcpProjectEnvironmentData extends EnvironmentData { @JsonProperty(value = "projectDetails") private GcpProjectDetails projectDetails; - /** Creates an instance of GcpProjectEnvironmentData class. */ + /* + * Scan interval in hours (value should be between 1-hour to 24-hours) + */ + @JsonProperty(value = "scanInterval") + private Long scanInterval; + + /** + * Creates an instance of GcpProjectEnvironmentData class. + */ public GcpProjectEnvironmentData() { } /** * Get the organizationalData property: The Gcp project's organizational data. - * + * * @return the organizationalData value. */ public GcpOrganizationalData organizationalData() { @@ -41,7 +51,7 @@ public GcpOrganizationalData organizationalData() { /** * Set the organizationalData property: The Gcp project's organizational data. - * + * * @param organizationalData the organizationalData value to set. * @return the GcpProjectEnvironmentData object itself. */ @@ -52,7 +62,7 @@ public GcpProjectEnvironmentData withOrganizationalData(GcpOrganizationalData or /** * Get the projectDetails property: The Gcp project's details. - * + * * @return the projectDetails value. */ public GcpProjectDetails projectDetails() { @@ -61,7 +71,7 @@ public GcpProjectDetails projectDetails() { /** * Set the projectDetails property: The Gcp project's details. - * + * * @param projectDetails the projectDetails value to set. * @return the GcpProjectEnvironmentData object itself. */ @@ -70,9 +80,29 @@ public GcpProjectEnvironmentData withProjectDetails(GcpProjectDetails projectDet return this; } + /** + * Get the scanInterval property: Scan interval in hours (value should be between 1-hour to 24-hours). + * + * @return the scanInterval value. + */ + public Long scanInterval() { + return this.scanInterval; + } + + /** + * Set the scanInterval property: Scan interval in hours (value should be between 1-hour to 24-hours). + * + * @param scanInterval the scanInterval value to set. + * @return the GcpProjectEnvironmentData object itself. + */ + public GcpProjectEnvironmentData withScanInterval(Long scanInterval) { + this.scanInterval = scanInterval; + return this; + } + /** * Validates the instance. - * + * * @throws IllegalArgumentException thrown if the instance is not valid. */ @Override diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsListResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsListResponse.java new file mode 100644 index 0000000000000..f60c947fad029 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsListResponse.java @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsListResponseInner; +import java.util.List; + +/** + * An immutable client-side representation of GetSensitivitySettingsListResponse. + */ +public interface GetSensitivitySettingsListResponse { + /** + * Gets the value property: The value property. + * + * @return the value value. + */ + List value(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsListResponseInner object. + * + * @return the inner object. + */ + GetSensitivitySettingsListResponseInner innerModel(); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponse.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponse.java new file mode 100644 index 0000000000000..7db855c1ede45 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponse.java @@ -0,0 +1,47 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner; + +/** + * An immutable client-side representation of GetSensitivitySettingsResponse. + */ +public interface GetSensitivitySettingsResponse { + /** + * Gets the id property: Fully qualified resource Id for the resource. + * + * @return the id value. + */ + String id(); + + /** + * Gets the name property: The name of the resource. + * + * @return the name value. + */ + String name(); + + /** + * Gets the type property: The type of the resource. + * + * @return the type value. + */ + String type(); + + /** + * Gets the properties property: The sensitivity settings properties. + * + * @return the properties value. + */ + GetSensitivitySettingsResponseProperties properties(); + + /** + * Gets the inner com.azure.resourcemanager.security.fluent.models.GetSensitivitySettingsResponseInner object. + * + * @return the inner object. + */ + GetSensitivitySettingsResponseInner innerModel(); +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponseProperties.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponseProperties.java new file mode 100644 index 0000000000000..a15a267937fca --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponseProperties.java @@ -0,0 +1,146 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.UUID; + +/** + * The sensitivity settings properties. + */ +@Fluent +public final class GetSensitivitySettingsResponseProperties { + /* + * List of selected sensitive info types' IDs. + */ + @JsonProperty(value = "sensitiveInfoTypesIds") + private List sensitiveInfoTypesIds; + + /* + * The order of the sensitivity threshold label. Any label at or above this order will be considered sensitive. If + * set to -1, sensitivity by labels is turned off + */ + @JsonProperty(value = "sensitivityThresholdLabelOrder") + private Float sensitivityThresholdLabelOrder; + + /* + * The id of the sensitivity threshold label. Any label at or above this rank will be considered sensitive. + */ + @JsonProperty(value = "sensitivityThresholdLabelId") + private UUID sensitivityThresholdLabelId; + + /* + * Microsoft information protection built-in and custom information types, labels, and integration status. + */ + @JsonProperty(value = "mipInformation") + private GetSensitivitySettingsResponsePropertiesMipInformation mipInformation; + + /** + * Creates an instance of GetSensitivitySettingsResponseProperties class. + */ + public GetSensitivitySettingsResponseProperties() { + } + + /** + * Get the sensitiveInfoTypesIds property: List of selected sensitive info types' IDs. + * + * @return the sensitiveInfoTypesIds value. + */ + public List sensitiveInfoTypesIds() { + return this.sensitiveInfoTypesIds; + } + + /** + * Set the sensitiveInfoTypesIds property: List of selected sensitive info types' IDs. + * + * @param sensitiveInfoTypesIds the sensitiveInfoTypesIds value to set. + * @return the GetSensitivitySettingsResponseProperties object itself. + */ + public GetSensitivitySettingsResponseProperties withSensitiveInfoTypesIds(List sensitiveInfoTypesIds) { + this.sensitiveInfoTypesIds = sensitiveInfoTypesIds; + return this; + } + + /** + * Get the sensitivityThresholdLabelOrder property: The order of the sensitivity threshold label. Any label at or + * above this order will be considered sensitive. If set to -1, sensitivity by labels is turned off. + * + * @return the sensitivityThresholdLabelOrder value. + */ + public Float sensitivityThresholdLabelOrder() { + return this.sensitivityThresholdLabelOrder; + } + + /** + * Set the sensitivityThresholdLabelOrder property: The order of the sensitivity threshold label. Any label at or + * above this order will be considered sensitive. If set to -1, sensitivity by labels is turned off. + * + * @param sensitivityThresholdLabelOrder the sensitivityThresholdLabelOrder value to set. + * @return the GetSensitivitySettingsResponseProperties object itself. + */ + public GetSensitivitySettingsResponseProperties + withSensitivityThresholdLabelOrder(Float sensitivityThresholdLabelOrder) { + this.sensitivityThresholdLabelOrder = sensitivityThresholdLabelOrder; + return this; + } + + /** + * Get the sensitivityThresholdLabelId property: The id of the sensitivity threshold label. Any label at or above + * this rank will be considered sensitive. + * + * @return the sensitivityThresholdLabelId value. + */ + public UUID sensitivityThresholdLabelId() { + return this.sensitivityThresholdLabelId; + } + + /** + * Set the sensitivityThresholdLabelId property: The id of the sensitivity threshold label. Any label at or above + * this rank will be considered sensitive. + * + * @param sensitivityThresholdLabelId the sensitivityThresholdLabelId value to set. + * @return the GetSensitivitySettingsResponseProperties object itself. + */ + public GetSensitivitySettingsResponseProperties withSensitivityThresholdLabelId(UUID sensitivityThresholdLabelId) { + this.sensitivityThresholdLabelId = sensitivityThresholdLabelId; + return this; + } + + /** + * Get the mipInformation property: Microsoft information protection built-in and custom information types, labels, + * and integration status. + * + * @return the mipInformation value. + */ + public GetSensitivitySettingsResponsePropertiesMipInformation mipInformation() { + return this.mipInformation; + } + + /** + * Set the mipInformation property: Microsoft information protection built-in and custom information types, labels, + * and integration status. + * + * @param mipInformation the mipInformation value to set. + * @return the GetSensitivitySettingsResponseProperties object itself. + */ + public GetSensitivitySettingsResponseProperties + withMipInformation(GetSensitivitySettingsResponsePropertiesMipInformation mipInformation) { + this.mipInformation = mipInformation; + return this; + } + + /** + * Validates the instance. + * + * @throws IllegalArgumentException thrown if the instance is not valid. + */ + public void validate() { + if (mipInformation() != null) { + mipInformation().validate(); + } + } +} diff --git a/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponsePropertiesMipInformation.java b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponsePropertiesMipInformation.java new file mode 100644 index 0000000000000..fa0a02f11ba06 --- /dev/null +++ b/sdk/security/azure-resourcemanager-security/src/main/java/com/azure/resourcemanager/security/models/GetSensitivitySettingsResponsePropertiesMipInformation.java @@ -0,0 +1,144 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +// Code generated by Microsoft (R) AutoRest Code Generator. + +package com.azure.resourcemanager.security.models; + +import com.azure.core.annotation.Fluent; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; + +/** + * Microsoft information protection built-in and custom information types, labels, and integration status. + */ +@Fluent +public final class GetSensitivitySettingsResponsePropertiesMipInformation { + /* + * Microsoft information protection integration status + */ + @JsonProperty(value = "mipIntegrationStatus") + private MipIntegrationStatus mipIntegrationStatus; + + /* + * List of Microsoft information protection sensitivity labels + */ + @JsonProperty(value = "labels") + private List

See the Guava User Guide article on {@code Splitter}. + * + * @author Julien Silland + * @author Jesse Wilson + * @author Kevin Bourrillion + * @author Louis Wasserman + * @since 1.0 + */ +public final class Splitter { + private final CharMatcher trimmer; + private final boolean omitEmptyStrings; + private final Strategy strategy; + private final int limit; + + private Splitter(Strategy strategy) { + this(strategy, false, CharMatcher.none(), Integer.MAX_VALUE); + } + + private Splitter(Strategy strategy, boolean omitEmptyStrings, CharMatcher trimmer, int limit) { + this.strategy = strategy; + this.omitEmptyStrings = omitEmptyStrings; + this.trimmer = trimmer; + this.limit = limit; + } + + /** + * Returns a splitter that uses the given single-character separator. For example, {@code + * Splitter.on(',').split("foo,,bar")} returns an iterable containing {@code ["foo", "", "bar"]}. + * + * @param separator the character to recognize as a separator + * @return a splitter, with default settings, that recognizes that separator + */ + public static Splitter on(char separator) { + return on(CharMatcher.is(separator)); + } + + /** + * Returns a splitter that considers any single character matched by the given {@code CharMatcher} + * to be a separator. For example, {@code + * Splitter.on(CharMatcher.anyOf(";,")).split("foo,;bar,quux")} returns an iterable containing + * {@code ["foo", "", "bar", "quux"]}. + * + * @param separatorMatcher a {@link CharMatcher} that determines whether a character is a + * separator + * @return a splitter, with default settings, that uses this matcher + */ + public static Splitter on(final CharMatcher separatorMatcher) { + checkNotNull(separatorMatcher); + + return new Splitter( + new Strategy() { + @Override + public SplittingIterator iterator(Splitter splitter, final CharSequence toSplit) { + return new SplittingIterator(splitter, toSplit) { + @Override + int separatorStart(int start) { + return separatorMatcher.indexIn(toSplit, start); + } + + @Override + int separatorEnd(int separatorPosition) { + return separatorPosition + 1; + } + }; + } + }); + } + + /** + * Returns a splitter that uses the given fixed string as a separator. For example, {@code + * Splitter.on(", ").split("foo, bar,baz")} returns an iterable containing {@code ["foo", + * "bar,baz"]}. + * + * @param separator the literal, nonempty string to recognize as a separator + * @return a splitter, with default settings, that recognizes that separator + */ + public static Splitter on(final String separator) { + checkArgument(separator.length() != 0, "The separator may not be the empty string."); + if (separator.length() == 1) { + return Splitter.on(separator.charAt(0)); + } + return new Splitter( + new Strategy() { + @Override + public SplittingIterator iterator(Splitter splitter, CharSequence toSplit) { + return new SplittingIterator(splitter, toSplit) { + @Override + public int separatorStart(int start) { + int separatorLength = separator.length(); + + positions: + for (int p = start, last = toSplit.length() - separatorLength; p <= last; p++) { + for (int i = 0; i < separatorLength; i++) { + if (toSplit.charAt(i + p) != separator.charAt(i)) { + continue positions; + } + } + return p; + } + return -1; + } + + @Override + public int separatorEnd(int separatorPosition) { + return separatorPosition + separator.length(); + } + }; + } + }); + } + + /** + * Returns a splitter that considers any subsequence matching {@code pattern} to be a separator. + * For example, {@code Splitter.on(Pattern.compile("\r?\n")).split(entireFile)} splits a string + * into lines whether it uses DOS-style or UNIX-style line terminators. + * + * @param separatorPattern the pattern that determines whether a subsequence is a separator. This + * pattern may not match the empty string. + * @return a splitter, with default settings, that uses this pattern + * @throws IllegalArgumentException if {@code separatorPattern} matches the empty string + */ + public static Splitter on(Pattern separatorPattern) { + return on(new JdkPattern(separatorPattern)); + } + + private static Splitter on(final CommonPattern separatorPattern) { + checkArgument( + !separatorPattern.matcher("").matches(), + "The pattern may not match the empty string: %s", + separatorPattern); + + return new Splitter( + new Strategy() { + @Override + public SplittingIterator iterator(final Splitter splitter, CharSequence toSplit) { + final CommonMatcher matcher = separatorPattern.matcher(toSplit); + return new SplittingIterator(splitter, toSplit) { + @Override + public int separatorStart(int start) { + return matcher.find(start) ? matcher.start() : -1; + } + + @Override + public int separatorEnd(int separatorPosition) { + return matcher.end(); + } + }; + } + }); + } + + /** + * Returns a splitter that considers any subsequence matching a given pattern (regular expression) + * to be a separator. For example, {@code Splitter.onPattern("\r?\n").split(entireFile)} splits a + * string into lines whether it uses DOS-style or UNIX-style line terminators. This is equivalent + * to {@code Splitter.on(Pattern.compile(pattern))}. + * + * @param separatorPattern the pattern that determines whether a subsequence is a separator. This + * pattern may not match the empty string. + * @return a splitter, with default settings, that uses this pattern + * @throws IllegalArgumentException if {@code separatorPattern} matches the empty string or is a + * malformed expression + */ + public static Splitter onPattern(String separatorPattern) { + return on(Platform.compilePattern(separatorPattern)); + } + + /** + * Returns a splitter that divides strings into pieces of the given length. For example, {@code + * Splitter.fixedLength(2).split("abcde")} returns an iterable containing {@code ["ab", "cd", + * "e"]}. The last piece can be smaller than {@code length} but will never be empty. + * + *